From bfd1c3eb8c6066152d86e8d9032c5df0632304f4 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Thu, 15 Feb 2024 11:33:15 -0600 Subject: [PATCH 0001/1019] Adding simple custom partition sort order option to RewriteManifests Spark Action --- .palantir/revapi.yml | 4 + .../iceberg/actions/RewriteManifests.java | 11 ++ .../actions/RewriteManifestsSparkAction.java | 72 +++++++- .../actions/TestRewriteManifestsAction.java | 164 ++++++++++++++++++ 4 files changed, 250 insertions(+), 1 deletion(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index a41d3ddfb8df..86537211f42a 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -873,6 +873,10 @@ acceptedBreaks: new: "method void org.apache.iceberg.encryption.Ciphers::()" justification: "Static utility class - should not have public constructor" "1.4.0": + org.apache.iceberg:iceberg-api: + - code: "java.method.addedToInterface" + new: "method org.apache.iceberg.actions.RewriteManifests org.apache.iceberg.actions.RewriteManifests::sort(java.util.List)" + justification: "New method added to Spark RewriteManifests" org.apache.iceberg:iceberg-core: - code: "java.class.defaultSerializationChanged" old: "class org.apache.iceberg.PartitionData" diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index 57b35aaf6596..1295960b0cb8 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.actions; +import java.util.List; import java.util.function.Predicate; import org.apache.iceberg.ManifestFile; @@ -44,6 +45,16 @@ public interface RewriteManifests */ RewriteManifests rewriteIf(Predicate predicate); + /** + * Rewrite manifests in a given order, based on partition columns + * + *

If not set, manifests will be rewritten in the order of the table's spec. + * + * @param partitionSortOrder a list of partition field names + * @return this for method chaining + */ + RewriteManifests sort(List partitionSortOrder); + /** * Passes a location where the staged manifests should be written. * diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 5b1d6165695b..7e1ea00b0beb 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -19,10 +19,13 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.MetadataTableType.ENTRIES; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.functions.concat_ws; import java.io.Serializable; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.function.Function; import java.util.function.Predicate; @@ -90,6 +93,7 @@ public class RewriteManifestsSparkAction public static final String USE_CACHING = "use-caching"; public static final boolean USE_CACHING_DEFAULT = false; + private List partitionSortColumns = null; private static final Logger LOG = LoggerFactory.getLogger(RewriteManifestsSparkAction.class); private static final RewriteManifests.Result EMPTY_RESULT = ImmutableRewriteManifests.Result.builder() @@ -97,6 +101,37 @@ public class RewriteManifestsSparkAction .addedManifests(ImmutableList.of()) .build(); + /** + * Supply an optional set of partition columns to sort the rewritten manifests by. Expects real + * column names used for partitioning; will resolve to the proper hidden partition names. + * + * @param partitionSortOrder - Order of partitions to sort manifests by + * @return this action + */ + @Override + public RewriteManifestsSparkAction sort(List partitionSortOrder) { + // Build up a mapping of input column -> partition column name (AKA x -> x_bucket_1000) + Map partitionFieldMap = + spec.fields().stream() + .map( + partitionField -> + Map.entry( + spec.schema().findField(partitionField.sourceId()).name(), + partitionField.name())) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + + // Check if these partitions are included in the spec + Preconditions.checkArgument( + partitionSortOrder.stream().allMatch(partitionFieldMap::containsKey), + "Invalid sorting columns. Expected to match column names used for partitioning in spec: %s." + + " Available partitionable columns: %s", + this.spec.specId(), + partitionFieldMap.keySet()); + this.partitionSortColumns = + partitionSortOrder.stream().map(partitionFieldMap::get).collect(Collectors.toList()); + return this; + } + private final Table table; private final int formatVersion; private final long targetManifestSizeBytes; @@ -208,7 +243,15 @@ private RewriteManifests.Result rewriteManifests(ManifestContent content) { if (spec.isUnpartitioned()) { newManifests = writeUnpartitionedManifests(content, manifestEntryDF, targetNumManifests); } else { - newManifests = writePartitionedManifests(content, manifestEntryDF, targetNumManifests); + if (partitionSortColumns != null) { + LOG.info( + "Sorting manifests for specId {} by partition columns in order of {} ", + spec.specId(), + partitionSortColumns); + newManifests = writeSortedManifests(content, manifestEntryDF, targetNumManifests); + } else { + newManifests = writePartitionedManifests(content, manifestEntryDF, targetNumManifests); + } } return ImmutableRewriteManifests.Result.builder() @@ -247,6 +290,33 @@ private List writeUnpartitionedManifests( return writeFunc.apply(transformedManifestEntryDF).collectAsList(); } + private List writeSortedManifests( + ManifestContent content, Dataset manifestEntryDF, int numManifests) { + + // Map the top level partition column names to the column name referenced within the manifest + // entry dataframe + Column[] actualPartitionColumns = + partitionSortColumns.stream() + .map(p -> col("data_file.partition." + p)) + .toArray(Column[]::new); + + // Form a new temporary column to sort/cluster manifests on, based on the custom sort + // order provided + Dataset clusteredManifestEntryDF = + manifestEntryDF.withColumn( + "__clustering_column__", concat_ws("::", actualPartitionColumns)); + + return withReusableDS( + clusteredManifestEntryDF, + df -> { + WriteManifests writeFunc = newWriteManifestsFunc(content, df.schema()); + Column partitionColumn = df.col("__clustering_column__"); + Dataset transformedDF = + repartitionAndSort(df, partitionColumn, numManifests).drop("__clustering_column__"); + return writeFunc.apply(transformedDF).collectAsList(); + }); + } + private List writePartitionedManifests( ManifestContent content, Dataset manifestEntryDF, int numManifests) { 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..fca93e8e5acc 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 @@ -34,9 +34,12 @@ import java.io.File; import java.io.IOException; import java.nio.file.Path; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Random; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -65,6 +68,7 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.io.OutputFile; +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.relocated.com.google.common.collect.Lists; @@ -73,6 +77,7 @@ import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.TestBase; import org.apache.iceberg.spark.source.ThreeColumnRecord; +import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; @@ -84,6 +89,7 @@ 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 TestRewriteManifestsAction extends TestBase { @@ -466,6 +472,164 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); } + @TestTemplate + public void testRewriteManifestsPartitionedTableWithInvalidSortColumnsThowsException() + throws IOException { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").bucket("c3", 10).build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + SparkActions actions = SparkActions.get(); + + // c2 is not a partition column, cannot use for sorting + List badSortKeys1 = ImmutableList.of("c1", "c2"); + assertThatThrownBy( + () -> + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sort(badSortKeys1) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .message() + .contains("Invalid sorting columns"); + + // c3_bucket is the hidden internal partition name, sort() expects the un-hidden partition + // column names + List badSortKeys2 = ImmutableList.of("c1", "c3_bucket"); + assertThatThrownBy( + () -> + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sort(badSortKeys2) + .execute()) + .isInstanceOf(IllegalArgumentException.class) + .message() + .contains("Invalid sorting columns"); + } + + @TestTemplate + public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOException { + Random random = new Random(); + + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 3).bucket("c3", 10).build(); + Map options = Maps.newHashMap(); + options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); + options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); + Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + + List dataFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + dataFiles.add( + newDataFile( + table, + TestHelpers.Row.of( + new Object[] { + fileOrdinal, String.valueOf(random.nextInt() * 100), random.nextInt(10) + }))); + } + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 1 manifests before rewrite").hasSize(1); + + // Capture the c3 partition's lower and upper bounds - used for later test assertions + Integer c3PartitionMin = + Conversions.fromByteBuffer( + Types.IntegerType.get(), manifests.get(0).partitions().get(2).lowerBound()); + Integer c3PartitionMax = + Conversions.fromByteBuffer( + Types.IntegerType.get(), manifests.get(0).partitions().get(2).upperBound()); + + // Set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(manifests.get(0).length() / 2)) + .commit(); + + SparkActions actions = SparkActions.get(); + + String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); + + List manifestSortKeys = ImmutableList.of("c3", "c2", "c1"); + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sort(manifestSortKeys) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .stagingLocation(stagingLocation) + .execute(); + + table.refresh(); + List newManifests = table.currentSnapshot().allManifests(table.io()); + + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + assertManifestsLocation(result.addedManifests(), stagingLocation); + + assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); + + // Rewritten manifests are clustered by c3_bucket - each should contain only a subset of the + // lower and upper bounds + // of the partition 'c3'. + List> c3Boundaries = + newManifests.stream() + .map(manifest -> manifest.partitions().get(2)) + .sorted( + Comparator.comparing( + ptn -> Conversions.fromByteBuffer(Types.IntegerType.get(), ptn.lowerBound()))) + .map( + p -> + new Tuple2( + Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), + Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) + .collect(Collectors.toList()); + + List lowers = c3Boundaries.stream().map(t -> t._1).collect(Collectors.toList()); + List uppers = c3Boundaries.stream().map(t -> t._2).collect(Collectors.toList()); + + // With custom sorting, this looks like + // - manifest 1 -> [lower bound = 0, upper bound = 4] + // - manifest 2 -> [lower bound = 4, upper bound = 9] + // Without the custom sorting, each manifest tracks the full range of c3 upper/lower bounds. + // AKA they look like + // - manifest 1 -> [lower bound = 0, upper bound = 9] + // - manifest 2 -> [lower bound = 0, upper bound = 9] + // So the upper bound of the partitions tracked by the first file should be LEQ the lower bounds + // of the second. Etc + assertThat(uppers.get(0)) + .withFailMessage( + "Upper bound of first manifest partition should be LEQ lower bound of second") + .isLessThanOrEqualTo(lowers.get(1)); + + // Each file should contain less than the full c3 partition span + c3Boundaries.forEach( + boundary -> { + assertThat(boundary._2 - boundary._1) + .withFailMessage( + "Manifest should contain less than the full range of c3 bucket partitions") + .isLessThanOrEqualTo(c3PartitionMax - c3PartitionMin); + }); + + // c3's Bucket(10) partition means our true lower bound = 0 and true upper bound is 9. The first + // manifest should + // include the lower bound of 0, and the last should have the upper bound of 9 + assertThat(lowers.get(0)) + .withFailMessage("Lower bound of first manifest partition should be 0") + .isEqualTo(c3PartitionMin); + assertThat(uppers.get(uppers.size() - 1)) + .withFailMessage("Lower bound of first manifest partition should be 0") + .isEqualTo(c3PartitionMax); + } + @TestTemplate public void testRewriteManifestsWithPredicate() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); From feda66c58dee7b0d4c996c2d386d06aae5d69385 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Wed, 21 Feb 2024 15:39:45 -0600 Subject: [PATCH 0002/1019] Updated custom sort order for RewriteManifests to have a default implementation and to accept transform names --- .palantir/revapi.yml | 4 - .../iceberg/actions/RewriteManifests.java | 12 +- .../actions/RewriteManifestsSparkAction.java | 129 ++++++++---------- .../actions/TestRewriteManifestsAction.java | 17 +-- 4 files changed, 74 insertions(+), 88 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 86537211f42a..a41d3ddfb8df 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -873,10 +873,6 @@ acceptedBreaks: new: "method void org.apache.iceberg.encryption.Ciphers::()" justification: "Static utility class - should not have public constructor" "1.4.0": - org.apache.iceberg:iceberg-api: - - code: "java.method.addedToInterface" - new: "method org.apache.iceberg.actions.RewriteManifests org.apache.iceberg.actions.RewriteManifests::sort(java.util.List)" - justification: "New method added to Spark RewriteManifests" org.apache.iceberg:iceberg-core: - code: "java.class.defaultSerializationChanged" old: "class org.apache.iceberg.PartitionData" diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index 1295960b0cb8..ae0d967a05c0 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -46,14 +46,18 @@ public interface RewriteManifests RewriteManifests rewriteIf(Predicate predicate); /** - * Rewrite manifests in a given order, based on partition columns + * Rewrite manifests in a given order, based on partition transforms * - *

If not set, manifests will be rewritten in the order of the table's spec. + *

If not set, manifests will be rewritten in the order of the transforms in the table's + * current partition spec. * - * @param partitionSortOrder a list of partition field names + * @param partitionSortOrder a list of partition field transform names * @return this for method chaining */ - RewriteManifests sort(List partitionSortOrder); + default RewriteManifests sort(List partitionSortOrder) { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement sort(List)"); + } /** * Passes a location where the staged manifests should be written. diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 7e1ea00b0beb..53e6d83944f6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -25,7 +25,7 @@ import java.io.Serializable; import java.util.Iterator; import java.util.List; -import java.util.Map; +import java.util.Set; import java.util.UUID; import java.util.function.Function; import java.util.function.Predicate; @@ -40,6 +40,7 @@ import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Partitioning; import org.apache.iceberg.RollingManifestWriter; @@ -100,38 +101,6 @@ public class RewriteManifestsSparkAction .rewrittenManifests(ImmutableList.of()) .addedManifests(ImmutableList.of()) .build(); - - /** - * Supply an optional set of partition columns to sort the rewritten manifests by. Expects real - * column names used for partitioning; will resolve to the proper hidden partition names. - * - * @param partitionSortOrder - Order of partitions to sort manifests by - * @return this action - */ - @Override - public RewriteManifestsSparkAction sort(List partitionSortOrder) { - // Build up a mapping of input column -> partition column name (AKA x -> x_bucket_1000) - Map partitionFieldMap = - spec.fields().stream() - .map( - partitionField -> - Map.entry( - spec.schema().findField(partitionField.sourceId()).name(), - partitionField.name())) - .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); - - // Check if these partitions are included in the spec - Preconditions.checkArgument( - partitionSortOrder.stream().allMatch(partitionFieldMap::containsKey), - "Invalid sorting columns. Expected to match column names used for partitioning in spec: %s." - + " Available partitionable columns: %s", - this.spec.specId(), - partitionFieldMap.keySet()); - this.partitionSortColumns = - partitionSortOrder.stream().map(partitionFieldMap::get).collect(Collectors.toList()); - return this; - } - private final Table table; private final int formatVersion; private final long targetManifestSizeBytes; @@ -195,6 +164,34 @@ public RewriteManifestsSparkAction stagingLocation(String newStagingLocation) { return this; } + /** + * Supply an optional set of partition transform column names to sort the rewritten manifests by. + * Expects exact transformed column names used for partitioning; not the raw column names used for + * partitioning. E.G. supply 'data_bucket' and not 'data' for a partition defined as bucket(N, + * data) + * + * @param partitionSortOrder - Order of partitions to sort manifests by + * @return this action + */ + @Override + public RewriteManifestsSparkAction sort(List partitionSortOrder) { + // Collect set of allowable partition columns to sort on + Set availablePartitionNames = + spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + + // Check if these partitions are included in the spec + Preconditions.checkArgument( + partitionSortOrder.stream().allMatch(availablePartitionNames::contains), + "Cannot use custom sort order to rewrite manifests '%s'. All partition columns must be " + + "defined in the current partition spec: %s. Choose from the available partitionable columns: %s", + partitionSortColumns, + this.spec.specId(), + availablePartitionNames); + this.partitionSortColumns = partitionSortOrder; + + return this; + } + @Override public RewriteManifests.Result execute() { String desc = String.format("Rewriting manifests in %s", table.name()); @@ -243,15 +240,7 @@ private RewriteManifests.Result rewriteManifests(ManifestContent content) { if (spec.isUnpartitioned()) { newManifests = writeUnpartitionedManifests(content, manifestEntryDF, targetNumManifests); } else { - if (partitionSortColumns != null) { - LOG.info( - "Sorting manifests for specId {} by partition columns in order of {} ", - spec.specId(), - partitionSortColumns); - newManifests = writeSortedManifests(content, manifestEntryDF, targetNumManifests); - } else { - newManifests = writePartitionedManifests(content, manifestEntryDF, targetNumManifests); - } + newManifests = writePartitionedManifests(content, manifestEntryDF, targetNumManifests); } return ImmutableRewriteManifests.Result.builder() @@ -290,42 +279,42 @@ private List writeUnpartitionedManifests( return writeFunc.apply(transformedManifestEntryDF).collectAsList(); } - private List writeSortedManifests( + private List writePartitionedManifests( ManifestContent content, Dataset manifestEntryDF, int numManifests) { - // Map the top level partition column names to the column name referenced within the manifest - // entry dataframe - Column[] actualPartitionColumns = - partitionSortColumns.stream() - .map(p -> col("data_file.partition." + p)) - .toArray(Column[]::new); - - // Form a new temporary column to sort/cluster manifests on, based on the custom sort - // order provided - Dataset clusteredManifestEntryDF = - manifestEntryDF.withColumn( - "__clustering_column__", concat_ws("::", actualPartitionColumns)); + // Extract desired clustering/sorting criteria into a dedicated column + Dataset clusteredManifestEntryDF; + String clusteringColumnName = "__clustering_column__"; + + if (partitionSortColumns != null) { + LOG.info( + "Sorting manifests for specId {} by partition columns in order of {} ", + spec.specId(), + partitionSortColumns); + + // Map the top level partition column names to the column name referenced within the manifest + // entry dataframe + Column[] actualPartitionColumns = + partitionSortColumns.stream() + .map(p -> col("data_file.partition." + p)) + .toArray(Column[]::new); + + // Form a new temporary column to sort/cluster manifests on, based on the custom sort + // order provided + clusteredManifestEntryDF = + manifestEntryDF.withColumn(clusteringColumnName, concat_ws("::", actualPartitionColumns)); + } else { + clusteredManifestEntryDF = + manifestEntryDF.withColumn(clusteringColumnName, col("data_file.partition")); + } return withReusableDS( clusteredManifestEntryDF, df -> { WriteManifests writeFunc = newWriteManifestsFunc(content, df.schema()); - Column partitionColumn = df.col("__clustering_column__"); + Column partitionColumn = df.col(clusteringColumnName); Dataset transformedDF = - repartitionAndSort(df, partitionColumn, numManifests).drop("__clustering_column__"); - return writeFunc.apply(transformedDF).collectAsList(); - }); - } - - private List writePartitionedManifests( - ManifestContent content, Dataset manifestEntryDF, int numManifests) { - - return withReusableDS( - manifestEntryDF, - df -> { - WriteManifests writeFunc = newWriteManifestsFunc(content, df.schema()); - Column partitionColumn = df.col("data_file.partition"); - Dataset transformedDF = repartitionAndSort(df, partitionColumn, numManifests); + repartitionAndSort(df, partitionColumn, numManifests).drop(clusteringColumnName); return writeFunc.apply(transformedDF).collectAsList(); }); } 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 fca93e8e5acc..12202f8e7c8f 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 @@ -494,11 +494,12 @@ public void testRewriteManifestsPartitionedTableWithInvalidSortColumnsThowsExcep .execute()) .isInstanceOf(IllegalArgumentException.class) .message() - .contains("Invalid sorting columns"); + .contains("Cannot use custom sort order"); - // c3_bucket is the hidden internal partition name, sort() expects the un-hidden partition - // column names - List badSortKeys2 = ImmutableList.of("c1", "c3_bucket"); + // c3_bucket is the correct internal partition name to use, c3 is the untransformed colum name, + // sort() expects + // the hidden partition column names + List badSortKeys2 = ImmutableList.of("c1", "c3"); assertThatThrownBy( () -> actions @@ -508,7 +509,7 @@ public void testRewriteManifestsPartitionedTableWithInvalidSortColumnsThowsExcep .execute()) .isInstanceOf(IllegalArgumentException.class) .message() - .contains("Invalid sorting columns"); + .contains("Cannot use custom sort order"); } @TestTemplate @@ -556,16 +557,13 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc SparkActions actions = SparkActions.get(); - String stagingLocation = java.nio.file.Files.createTempDirectory(temp, "junit").toString(); - - List manifestSortKeys = ImmutableList.of("c3", "c2", "c1"); + List manifestSortKeys = ImmutableList.of("c3_bucket", "c2_trunc", "c1"); RewriteManifests.Result result = actions .rewriteManifests(table) .rewriteIf(manifest -> true) .sort(manifestSortKeys) .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) - .stagingLocation(stagingLocation) .execute(); table.refresh(); @@ -573,7 +571,6 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc assertThat(result.rewrittenManifests()).hasSize(1); assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); - assertManifestsLocation(result.addedManifests(), stagingLocation); assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); From 46235514906acb729d24008aea271d3a1d394117 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Wed, 21 Feb 2024 15:47:25 -0600 Subject: [PATCH 0003/1019] Replaced Scala Tuple with Pair utility class in RewriteManifests test --- .../spark/actions/TestRewriteManifestsAction.java | 15 +++++++-------- 1 file changed, 7 insertions(+), 8 deletions(-) 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 12202f8e7c8f..fbfc02109579 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 @@ -89,7 +89,6 @@ 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 TestRewriteManifestsAction extends TestBase { @@ -577,7 +576,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc // Rewritten manifests are clustered by c3_bucket - each should contain only a subset of the // lower and upper bounds // of the partition 'c3'. - List> c3Boundaries = + List> c3Boundaries = newManifests.stream() .map(manifest -> manifest.partitions().get(2)) .sorted( @@ -585,13 +584,13 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc ptn -> Conversions.fromByteBuffer(Types.IntegerType.get(), ptn.lowerBound()))) .map( p -> - new Tuple2( - Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), - Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) + Pair.of( + (Integer) Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), + (Integer) Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) .collect(Collectors.toList()); - List lowers = c3Boundaries.stream().map(t -> t._1).collect(Collectors.toList()); - List uppers = c3Boundaries.stream().map(t -> t._2).collect(Collectors.toList()); + List lowers = c3Boundaries.stream().map(t -> t.first()).collect(Collectors.toList()); + List uppers = c3Boundaries.stream().map(t -> t.second()).collect(Collectors.toList()); // With custom sorting, this looks like // - manifest 1 -> [lower bound = 0, upper bound = 4] @@ -610,7 +609,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc // Each file should contain less than the full c3 partition span c3Boundaries.forEach( boundary -> { - assertThat(boundary._2 - boundary._1) + assertThat(boundary.second() - boundary.first()) .withFailMessage( "Manifest should contain less than the full range of c3 bucket partitions") .isLessThanOrEqualTo(c3PartitionMax - c3PartitionMin); From 394c71389efe51724d732d6728b6daa3f6f4a3f5 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Thu, 22 Feb 2024 21:04:10 -0600 Subject: [PATCH 0004/1019] Fixed styling, changed method for clustering columns to be based on a struct, not a concatted string --- .../actions/RewriteManifestsSparkAction.java | 19 ++++++++++--------- .../actions/TestRewriteManifestsAction.java | 6 ++++-- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 53e6d83944f6..cb9042414fbe 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -20,7 +20,6 @@ import static org.apache.iceberg.MetadataTableType.ENTRIES; import static org.apache.spark.sql.functions.col; -import static org.apache.spark.sql.functions.concat_ws; import java.io.Serializable; import java.util.Iterator; @@ -74,6 +73,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.functions; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -177,16 +177,16 @@ public RewriteManifestsSparkAction stagingLocation(String newStagingLocation) { public RewriteManifestsSparkAction sort(List partitionSortOrder) { // Collect set of allowable partition columns to sort on Set availablePartitionNames = - spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); // Check if these partitions are included in the spec Preconditions.checkArgument( - partitionSortOrder.stream().allMatch(availablePartitionNames::contains), - "Cannot use custom sort order to rewrite manifests '%s'. All partition columns must be " - + "defined in the current partition spec: %s. Choose from the available partitionable columns: %s", - partitionSortColumns, - this.spec.specId(), - availablePartitionNames); + partitionSortOrder.stream().allMatch(availablePartitionNames::contains), + "Cannot use custom sort order to rewrite manifests '%s'. All partition columns must be " + + "defined in the current partition spec: %s. Choose from the available partitionable columns: %s", + partitionSortColumns, + this.spec.specId(), + availablePartitionNames); this.partitionSortColumns = partitionSortOrder; return this; @@ -302,7 +302,8 @@ private List writePartitionedManifests( // Form a new temporary column to sort/cluster manifests on, based on the custom sort // order provided clusteredManifestEntryDF = - manifestEntryDF.withColumn(clusteringColumnName, concat_ws("::", actualPartitionColumns)); + manifestEntryDF.withColumn( + clusteringColumnName, functions.struct(actualPartitionColumns)); } else { clusteredManifestEntryDF = manifestEntryDF.withColumn(clusteringColumnName, col("data_file.partition")); 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 fbfc02109579..40bbbc6b953b 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 @@ -585,8 +585,10 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc .map( p -> Pair.of( - (Integer) Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), - (Integer) Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) + (Integer) + Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), + (Integer) + Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) .collect(Collectors.toList()); List lowers = c3Boundaries.stream().map(t -> t.first()).collect(Collectors.toList()); From 9e1ed71bd45540cd068b4aac59c43a46eb1d47ee Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Fri, 23 Feb 2024 13:07:50 -0600 Subject: [PATCH 0005/1019] Added custom sorting with user supplied function api to RewriteManifests --- .../iceberg/actions/RewriteManifests.java | 26 ++- .../actions/CustomDataFileSorterUdf.java | 55 ++++++ .../actions/RewriteManifestsSparkAction.java | 70 ++++--- .../actions/TestRewriteManifestsAction.java | 183 ++++++++++++++++-- 4 files changed, 291 insertions(+), 43 deletions(-) create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index ae0d967a05c0..aab2a2c59118 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -19,7 +19,9 @@ package org.apache.iceberg.actions; import java.util.List; +import java.util.function.Function; import java.util.function.Predicate; +import org.apache.iceberg.DataFile; import org.apache.iceberg.ManifestFile; /** An action that rewrites manifests. */ @@ -46,19 +48,37 @@ public interface RewriteManifests RewriteManifests rewriteIf(Predicate predicate); /** - * Rewrite manifests in a given order, based on partition transforms + * Rewrite manifests in a given order, based on partition field names + * + *

Supply an optional set of partition field names to sort the rewritten manifests by. Expects + * exact transformed column names used for partitioning; not the raw columnnames that partitions + * are derived from. E.G. supply 'data_bucket' and not 'data' for a bucket(N, data) partition + * definition * *

If not set, manifests will be rewritten in the order of the transforms in the table's * current partition spec. * - * @param partitionSortOrder a list of partition field transform names + * @param partitionFieldSortOrder a list of partition field names * @return this for method chaining */ - default RewriteManifests sort(List partitionSortOrder) { + default RewriteManifests sort(List partitionFieldSortOrder) { throw new UnsupportedOperationException( this.getClass().getName() + " doesn't implement sort(List)"); } + /** + * Rewrite manifests in a given order, dictated by a custom Function + * + *

Supply a Function which will apply its own custom clustering logic based on supplied {@link + * org.apache.iceberg.DataFile} attributes. + * + * @param sortStrategyFunction A Function that returns a String to be used for manifest clustering + * @return this method for chaining + */ + default RewriteManifests sort(Function sortStrategyFunction) { + throw new UnsupportedOperationException(); + } + /** * Passes a location where the staged manifests should be written. * diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java new file mode 100644 index 000000000000..f158521675c6 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java @@ -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. + */ +package org.apache.iceberg.spark.actions; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.spark.SparkDataFile; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.api.java.UDF1; +import org.apache.spark.sql.types.StructType; + +/** + * A UserDefinedFunction that is built specifically to translate its inputs into a {@link + * org.apache.iceberg.DataFile} appearing object and pass that along to a runtime supplied Function. + * The result of the execution of which must be a String. + */ +public class CustomDataFileSorterUdf implements UDF1, Serializable { + // Supply how the DataFile should be interpreted from a raw Row. + private Types.StructType dataFileType; + private StructType dataFileSparkType; + private Function clusteringFunction; + + public CustomDataFileSorterUdf( + Function clusteringFunction, + Types.StructType dataFileType, + StructType dataFileSparkType) { + this.dataFileType = dataFileType; + this.dataFileSparkType = dataFileSparkType; + this.clusteringFunction = clusteringFunction; + } + + @Override + public String call(Row dataFile) throws Exception { + SparkDataFile wrapper = new SparkDataFile(dataFileType, dataFileSparkType); + return this.clusteringFunction.apply(wrapper.wrap(dataFile)); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index cb9042414fbe..8b5d1f342793 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -73,7 +73,9 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.expressions.UserDefinedFunction; import org.apache.spark.sql.functions; +import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -94,13 +96,15 @@ public class RewriteManifestsSparkAction public static final String USE_CACHING = "use-caching"; public static final boolean USE_CACHING_DEFAULT = false; - private List partitionSortColumns = null; private static final Logger LOG = LoggerFactory.getLogger(RewriteManifestsSparkAction.class); private static final RewriteManifests.Result EMPTY_RESULT = ImmutableRewriteManifests.Result.builder() .rewrittenManifests(ImmutableList.of()) .addedManifests(ImmutableList.of()) .build(); + + private static final String CUSTOM_CLUSTERING_COLUMN_NAME = "__clustering_column__"; + private final Table table; private final int formatVersion; private final long targetManifestSizeBytes; @@ -110,6 +114,10 @@ public class RewriteManifestsSparkAction private Predicate predicate = manifest -> true; private String outputLocation = null; + private List partitionFieldSortOrder = null; + + private Function partitionSortFunction = null; + RewriteManifestsSparkAction(SparkSession spark, Table table) { super(spark); this.table = table; @@ -164,31 +172,28 @@ public RewriteManifestsSparkAction stagingLocation(String newStagingLocation) { return this; } - /** - * Supply an optional set of partition transform column names to sort the rewritten manifests by. - * Expects exact transformed column names used for partitioning; not the raw column names used for - * partitioning. E.G. supply 'data_bucket' and not 'data' for a partition defined as bucket(N, - * data) - * - * @param partitionSortOrder - Order of partitions to sort manifests by - * @return this action - */ @Override - public RewriteManifestsSparkAction sort(List partitionSortOrder) { + public RewriteManifestsSparkAction sort(List sortOrder) { // Collect set of allowable partition columns to sort on Set availablePartitionNames = spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); - // Check if these partitions are included in the spec + // Check if these partition fields are included in the spec Preconditions.checkArgument( - partitionSortOrder.stream().allMatch(availablePartitionNames::contains), + sortOrder.stream().allMatch(availablePartitionNames::contains), "Cannot use custom sort order to rewrite manifests '%s'. All partition columns must be " + "defined in the current partition spec: %s. Choose from the available partitionable columns: %s", - partitionSortColumns, + sortOrder, this.spec.specId(), availablePartitionNames); - this.partitionSortColumns = partitionSortOrder; + this.partitionFieldSortOrder = sortOrder; + return this; + } + + @Override + public RewriteManifests sort(Function partitionFieldsSortStrategy) { + this.partitionSortFunction = partitionFieldsSortStrategy; return this; } @@ -284,18 +289,36 @@ private List writePartitionedManifests( // Extract desired clustering/sorting criteria into a dedicated column Dataset clusteredManifestEntryDF; - String clusteringColumnName = "__clustering_column__"; - if (partitionSortColumns != null) { + if (partitionSortFunction != null) { + LOG.info( + "Sorting manifests for specId {} using custom sorting function", + spec.specId(), + partitionSortFunction); + Types.StructType partitionType = DataFile.getType(table.spec().partitionType()); + StructType dataFileSchema = manifestEntryDF.select("data_file.*").schema(); + + // Create a UDF to wrap the custom partitionSortFunction call + UserDefinedFunction clusteringUdf = + functions.udf( + new CustomDataFileSorterUdf( + this.partitionSortFunction, partitionType, dataFileSchema), + DataTypes.StringType); + // Apply supplied partitionSortFunction function to the data_file datums within this dataframe + // The results are stored as a String in the new __clustering_column__ + clusteredManifestEntryDF = + manifestEntryDF.withColumn( + CUSTOM_CLUSTERING_COLUMN_NAME, clusteringUdf.apply(col("data_file"))); + } else if (partitionFieldSortOrder != null) { LOG.info( "Sorting manifests for specId {} by partition columns in order of {} ", spec.specId(), - partitionSortColumns); + partitionFieldSortOrder); // Map the top level partition column names to the column name referenced within the manifest // entry dataframe Column[] actualPartitionColumns = - partitionSortColumns.stream() + partitionFieldSortOrder.stream() .map(p -> col("data_file.partition." + p)) .toArray(Column[]::new); @@ -303,19 +326,20 @@ private List writePartitionedManifests( // order provided clusteredManifestEntryDF = manifestEntryDF.withColumn( - clusteringColumnName, functions.struct(actualPartitionColumns)); + CUSTOM_CLUSTERING_COLUMN_NAME, functions.struct(actualPartitionColumns)); } else { clusteredManifestEntryDF = - manifestEntryDF.withColumn(clusteringColumnName, col("data_file.partition")); + manifestEntryDF.withColumn(CUSTOM_CLUSTERING_COLUMN_NAME, col("data_file.partition")); } return withReusableDS( clusteredManifestEntryDF, df -> { WriteManifests writeFunc = newWriteManifestsFunc(content, df.schema()); - Column partitionColumn = df.col(clusteringColumnName); + Column partitionColumn = df.col(CUSTOM_CLUSTERING_COLUMN_NAME); Dataset transformedDF = - repartitionAndSort(df, partitionColumn, numManifests).drop(clusteringColumnName); + repartitionAndSort(df, partitionColumn, numManifests) + .drop(CUSTOM_CLUSTERING_COLUMN_NAME); return writeFunc.apply(transformedDF).collectAsList(); }); } 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 40bbbc6b953b..78cb97c339b7 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 @@ -33,13 +33,16 @@ import java.io.File; import java.io.IOException; +import java.io.Serializable; import java.nio.file.Path; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Random; import java.util.UUID; +import java.util.function.Function; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -472,8 +475,7 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { } @TestTemplate - public void testRewriteManifestsPartitionedTableWithInvalidSortColumnsThowsException() - throws IOException { + public void testRewriteManifestsPartitionedTableWithInvalidSortColumns() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").bucket("c3", 10).build(); Map options = Maps.newHashMap(); options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); @@ -492,12 +494,13 @@ public void testRewriteManifestsPartitionedTableWithInvalidSortColumnsThowsExcep .sort(badSortKeys1) .execute()) .isInstanceOf(IllegalArgumentException.class) - .message() - .contains("Cannot use custom sort order"); + .hasMessage( + "Cannot use custom sort order to rewrite manifests '[c1, c2]'. All partition columns " + + "must be defined in the current partition spec: 0. Choose from the available " + + "partitionable columns: [c3_bucket, c1]"); - // c3_bucket is the correct internal partition name to use, c3 is the untransformed colum name, - // sort() expects - // the hidden partition column names + // c3_bucket is the correct internal partition name to use, c3 is the untransformed column name, + // sort() expects the hidden partition column names List badSortKeys2 = ImmutableList.of("c1", "c3"); assertThatThrownBy( () -> @@ -507,8 +510,10 @@ public void testRewriteManifestsPartitionedTableWithInvalidSortColumnsThowsExcep .sort(badSortKeys2) .execute()) .isInstanceOf(IllegalArgumentException.class) - .message() - .contains("Cannot use custom sort order"); + .hasMessage( + "Cannot use custom sort order to rewrite manifests '[c1, c3]'. All partition columns " + + "must be defined in the current partition spec: 0. Choose from the available " + + "partitionable columns: [c3_bucket, c1]"); } @TestTemplate @@ -517,10 +522,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 3).bucket("c3", 10).build(); - Map options = Maps.newHashMap(); - options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); - options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); - Table table = TABLES.create(SCHEMA, spec, options, tableLocation); + Table table = TABLES.create(SCHEMA, spec, tableLocation); List dataFiles = Lists.newArrayList(); for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { @@ -604,16 +606,14 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc // So the upper bound of the partitions tracked by the first file should be LEQ the lower bounds // of the second. Etc assertThat(uppers.get(0)) - .withFailMessage( - "Upper bound of first manifest partition should be LEQ lower bound of second") + .as("Upper bound of first manifest partition should be LEQ lower bound of second") .isLessThanOrEqualTo(lowers.get(1)); // Each file should contain less than the full c3 partition span c3Boundaries.forEach( boundary -> { assertThat(boundary.second() - boundary.first()) - .withFailMessage( - "Manifest should contain less than the full range of c3 bucket partitions") + .as("Manifest should contain less than the full range of c3 bucket partitions") .isLessThanOrEqualTo(c3PartitionMax - c3PartitionMin); }); @@ -628,6 +628,155 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc .isEqualTo(c3PartitionMax); } + @TestTemplate + public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws IOException { + Random random = new Random(); + + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 3).bucket("c3", 10).build(); + Table table = TABLES.create(SCHEMA, spec, tableLocation); + + List dataFiles = Lists.newArrayList(); + for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { + dataFiles.add( + newDataFile( + table, + TestHelpers.Row.of( + new Object[] { + fileOrdinal, String.valueOf(random.nextInt() * 100), random.nextInt(10) + }))); + } + ManifestFile appendManifest = writeManifest(table, dataFiles); + table.newFastAppend().appendManifest(appendManifest).commit(); + + List manifests = table.currentSnapshot().allManifests(table.io()); + assertThat(manifests).as("Should have 1 manifests before rewrite").hasSize(1); + + // Capture the c3 partition's lower and upper bounds - used for later test assertions + Integer c3PartitionMin = + Conversions.fromByteBuffer( + Types.IntegerType.get(), manifests.get(0).partitions().get(2).lowerBound()); + Integer c3PartitionMax = + Conversions.fromByteBuffer( + Types.IntegerType.get(), manifests.get(0).partitions().get(2).upperBound()); + + // Set the target manifest size to a small value to force splitting records into multiple files + table + .updateProperties() + .set( + TableProperties.MANIFEST_TARGET_SIZE_BYTES, + String.valueOf(manifests.get(0).length() / 3)) + .commit(); + + SparkActions actions = SparkActions.get(); + + // This is the main point of this test! + // Let's say I want to sort manifests into buckets 0-4 and 5 - 9 + // I know my Table and partition Spec, so can do that programmatically + Function test = + (Function & Serializable) + (dataFile) -> { + StructLike partition = dataFile.partition(); + // Find the ordinal index for the c3 partition column for this data file + int c3Index = + IntStream.range(0, spec.fields().size()) + .filter(i -> spec.fields().get(i).name().contains("c3")) + .findFirst() + .getAsInt(); + Object c3BucketValue = partition.get(c3Index, Object.class); + + // Return one string for the lower values, one for the upper. RewriteManifests + // will cluster datafiles together in manifests according to this value. + return (Integer) c3BucketValue < 5 ? "cluster=LT_5" : "cluster=GTE_5"; + }; + + RewriteManifests.Result result = + actions + .rewriteManifests(table) + .rewriteIf(manifest -> true) + .sort(test) + .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) + .execute(); + + table.refresh(); + List newManifests = table.currentSnapshot().allManifests(table.io()); + + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + + assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); + + // Rewritten manifests are clustered by c3_bucket - each should contain only a subset of the + // lower and upper bounds of the partition 'c3'. + List> c3Boundaries = + newManifests.stream() + .map(manifest -> manifest.partitions().get(2)) + .sorted( + Comparator.comparing( + ptn -> Conversions.fromByteBuffer(Types.IntegerType.get(), ptn.lowerBound()))) + .map( + p -> + Pair.of( + (Integer) + Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), + (Integer) + Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) + .collect(Collectors.toList()); + + List lowers = c3Boundaries.stream().map(t -> t.first()).collect(Collectors.toList()); + List uppers = c3Boundaries.stream().map(t -> t.second()).collect(Collectors.toList()); + + // The custom function sorts datafiles by having c3 bucket partitions in the range 0-4 and 5-9. + // Internally, that looks like + // +--------------------+---------------------+ + // |partition |__clustering_column__| + // +--------------------+---------------------+ + // |{0, -531806488, 0} |cluster=LT_5 | + // |{1, 385955472, 7} |cluster=GTE_5 | + // |{2, 604077840, 6} |cluster=GTE_5 | + // |{3, 1875302972, 4} |cluster=LT_5 | + // |{4, -1772544904, 0} |cluster=LT_5 | + // |{5, 172551248, 7} |cluster=GTE_5 | + // ... + // After rewriting, this looks like + // - manifest 1 -> [lower bound = 0, upper bound = 4] + // - manifest 2 -> [lower bound = 5, upper bound = 9] + // Without the custom sorting, each manifest tracks the full range of c3 upper/lower bounds. + // AKA they look like + // - manifest 1 -> [lower bound = 0, upper bound = 9] + // - manifest 2 -> [lower bound = 0, upper bound = 9] + // So the upper bound of the partitions tracked by the first file should be LEQ the lower bounds + // of the second. Etc + assertThat(uppers.get(0)) + .withFailMessage( + "Upper bound of first manifest partition should be LESS THAN the lower bound of second") + .isLessThan(lowers.get(1)); + + // Each file should contain less than the full c3 partition span + c3Boundaries.forEach( + boundary -> { + assertThat(boundary.second() - boundary.first()) + .withFailMessage( + "Manifest should contain less than the full range of c3 bucket partitions") + .isLessThan(c3PartitionMax - c3PartitionMin); + }); + + // c3's Bucket(10) partition means our true lower bound = 0 and true upper bound is 9. The first + // manifest should include the lower bound of 0, and the last should have the upper bound of 9 + assertThat(lowers.get(0)) + .withFailMessage("Lower bound of first manifest c3 bucket partition should be 0") + .isEqualTo(0); + assertThat(uppers.get(0)) + .withFailMessage("Upper bound of first manifest c3 bucket partition should be 4") + .isEqualTo(4); + assertThat(lowers.get(1)) + .withFailMessage("Lower bound of second manifest c3 bucket partition should be 5") + .isEqualTo(5); + assertThat(uppers.get(1)) + .withFailMessage("Upper bound of second manifest c3 bucket partition should be 9") + .isEqualTo(9); + } + @TestTemplate public void testRewriteManifestsWithPredicate() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build(); From 3fe4fd86d713cc3d971a576c509c57e507d7daaf Mon Sep 17 00:00:00 2001 From: Jack Ye Date: Mon, 4 Mar 2024 15:59:53 -0800 Subject: [PATCH 0006/1019] Refactor partition field sort order to be a function --- .../actions/RewriteManifestsSparkAction.java | 73 +++++++++++++------ 1 file changed, 52 insertions(+), 21 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 8b5d1f342793..5270274c6168 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -24,6 +24,7 @@ import java.io.Serializable; import java.util.Iterator; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.function.Function; @@ -44,6 +45,7 @@ import org.apache.iceberg.Partitioning; import org.apache.iceberg.RollingManifestWriter; import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; @@ -56,13 +58,16 @@ 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.Maps; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.SparkContentFile; import org.apache.iceberg.spark.SparkDataFile; import org.apache.iceberg.spark.SparkDeleteFile; import org.apache.iceberg.spark.source.SerializableTableWithSize; +import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.SerializableFunction; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; @@ -114,8 +119,6 @@ public class RewriteManifestsSparkAction private Predicate predicate = manifest -> true; private String outputLocation = null; - private List partitionFieldSortOrder = null; - private Function partitionSortFunction = null; RewriteManifestsSparkAction(SparkSession spark, Table table) { @@ -187,7 +190,7 @@ public RewriteManifestsSparkAction sort(List sortOrder) { this.spec.specId(), availablePartitionNames); - this.partitionFieldSortOrder = sortOrder; + this.partitionSortFunction = new PartitionFieldSortFunction(spec, sortOrder); return this; } @@ -309,24 +312,6 @@ private List writePartitionedManifests( clusteredManifestEntryDF = manifestEntryDF.withColumn( CUSTOM_CLUSTERING_COLUMN_NAME, clusteringUdf.apply(col("data_file"))); - } else if (partitionFieldSortOrder != null) { - LOG.info( - "Sorting manifests for specId {} by partition columns in order of {} ", - spec.specId(), - partitionFieldSortOrder); - - // Map the top level partition column names to the column name referenced within the manifest - // entry dataframe - Column[] actualPartitionColumns = - partitionFieldSortOrder.stream() - .map(p -> col("data_file.partition." + p)) - .toArray(Column[]::new); - - // Form a new temporary column to sort/cluster manifests on, based on the custom sort - // order provided - clusteredManifestEntryDF = - manifestEntryDF.withColumn( - CUSTOM_CLUSTERING_COLUMN_NAME, functions.struct(actualPartitionColumns)); } else { clusteredManifestEntryDF = manifestEntryDF.withColumn(CUSTOM_CLUSTERING_COLUMN_NAME, col("data_file.partition")); @@ -628,4 +613,50 @@ private Table table() { return tableBroadcast.value(); } } + + static class PartitionFieldSortFunction implements SerializableFunction { + + private final PartitionSpec spec; + private final List partitionFieldSortOrder; + + public PartitionFieldSortFunction(PartitionSpec spec, List partitionFieldSortOrder) { + this.spec = spec; + this.partitionFieldSortOrder = partitionFieldSortOrder; + } + + @Override + public String apply(DataFile dataFile) { + Map fieldIndices = Maps.newHashMap(); + int idx = 0; + for (PartitionField field : spec.fields()) { + fieldIndices.put(field.name(), idx); + idx++; + } + + StringBuilder sb = new StringBuilder(); + for (java.lang.String fieldName : partitionFieldSortOrder) { + Preconditions.checkArgument( + fieldIndices.containsKey(fieldName), + "Cannot find partition field with name %s in spec %s", + fieldName, + spec); + int fieldIndex = fieldIndices.get(fieldName); + PartitionField field = spec.fields().get(fieldIndex); + Type fieldType = spec.partitionType().field(field.fieldId()).type(); + Class javaClass = spec.javaClasses()[fieldIndex]; + java.lang.String fieldValue = + field + .transform() + .toHumanString(fieldType, get(dataFile.partition(), fieldIndex, javaClass)); + sb.append(fieldValue); + } + + return sb.toString(); + } + + @SuppressWarnings("unchecked") + private T get(StructLike data, int pos, Class javaClass) { + return data.get(pos, (Class) javaClass); + } + } } From 3cfa48a9993dcb8f41c39f13cd24943433003751 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Mon, 4 Mar 2024 20:52:15 -0600 Subject: [PATCH 0007/1019] Moved custom clustering UDF as static class within RewriteManifests. Updated test messaging --- .../iceberg/actions/RewriteManifests.java | 3 +- .../actions/CustomDataFileSorterUdf.java | 55 ------------ .../actions/RewriteManifestsSparkAction.java | 83 +++++++++---------- .../actions/TestRewriteManifestsAction.java | 16 ++-- 4 files changed, 48 insertions(+), 109 deletions(-) delete mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index aab2a2c59118..688e3b0ed67f 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -76,7 +76,8 @@ default RewriteManifests sort(List partitionFieldSortOrder) { * @return this method for chaining */ default RewriteManifests sort(Function sortStrategyFunction) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement sort(Function)"); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java deleted file mode 100644 index f158521675c6..000000000000 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/CustomDataFileSorterUdf.java +++ /dev/null @@ -1,55 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.spark.actions; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.spark.SparkDataFile; -import org.apache.iceberg.types.Types; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.api.java.UDF1; -import org.apache.spark.sql.types.StructType; - -/** - * A UserDefinedFunction that is built specifically to translate its inputs into a {@link - * org.apache.iceberg.DataFile} appearing object and pass that along to a runtime supplied Function. - * The result of the execution of which must be a String. - */ -public class CustomDataFileSorterUdf implements UDF1, Serializable { - // Supply how the DataFile should be interpreted from a raw Row. - private Types.StructType dataFileType; - private StructType dataFileSparkType; - private Function clusteringFunction; - - public CustomDataFileSorterUdf( - Function clusteringFunction, - Types.StructType dataFileType, - StructType dataFileSparkType) { - this.dataFileType = dataFileType; - this.dataFileSparkType = dataFileSparkType; - this.clusteringFunction = clusteringFunction; - } - - @Override - public String call(Row dataFile) throws Exception { - SparkDataFile wrapper = new SparkDataFile(dataFileType, dataFileSparkType); - return this.clusteringFunction.apply(wrapper.wrap(dataFile)); - } -} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 5270274c6168..325f20af7a87 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -24,7 +24,6 @@ import java.io.Serializable; import java.util.Iterator; import java.util.List; -import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.function.Function; @@ -45,7 +44,6 @@ import org.apache.iceberg.Partitioning; import org.apache.iceberg.RollingManifestWriter; import org.apache.iceberg.Snapshot; -import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; @@ -58,16 +56,13 @@ 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.Maps; import org.apache.iceberg.spark.JobGroupInfo; import org.apache.iceberg.spark.SparkContentFile; import org.apache.iceberg.spark.SparkDataFile; import org.apache.iceberg.spark.SparkDeleteFile; import org.apache.iceberg.spark.source.SerializableTableWithSize; -import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.SerializableFunction; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; @@ -78,6 +73,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.api.java.UDF1; import org.apache.spark.sql.expressions.UserDefinedFunction; import org.apache.spark.sql.functions; import org.apache.spark.sql.types.DataTypes; @@ -119,6 +115,7 @@ public class RewriteManifestsSparkAction private Predicate predicate = manifest -> true; private String outputLocation = null; + private List partitionFieldSortOrder = null; private Function partitionSortFunction = null; RewriteManifestsSparkAction(SparkSession spark, Table table) { @@ -190,13 +187,13 @@ public RewriteManifestsSparkAction sort(List sortOrder) { this.spec.specId(), availablePartitionNames); - this.partitionSortFunction = new PartitionFieldSortFunction(spec, sortOrder); + this.partitionFieldSortOrder = sortOrder; return this; } @Override public RewriteManifests sort(Function partitionFieldsSortStrategy) { - this.partitionSortFunction = partitionFieldsSortStrategy; + this.partitionSortFunction = (Function & Serializable) partitionFieldsSortStrategy; return this; } @@ -312,6 +309,24 @@ private List writePartitionedManifests( clusteredManifestEntryDF = manifestEntryDF.withColumn( CUSTOM_CLUSTERING_COLUMN_NAME, clusteringUdf.apply(col("data_file"))); + } else if (partitionFieldSortOrder != null) { + LOG.info( + "Sorting manifests for specId {} by partition columns in order of {} ", + spec.specId(), + partitionFieldSortOrder); + + // Map the top level partition column names to the column name referenced within the manifest + // entry dataframe + Column[] actualPartitionColumns = + partitionFieldSortOrder.stream() + .map(p -> col("data_file.partition." + p)) + .toArray(Column[]::new); + + // Form a new temporary column to sort/cluster manifests on, based on the custom sort + // order provided + clusteredManifestEntryDF = + manifestEntryDF.withColumn( + CUSTOM_CLUSTERING_COLUMN_NAME, functions.struct(actualPartitionColumns)); } else { clusteredManifestEntryDF = manifestEntryDF.withColumn(CUSTOM_CLUSTERING_COLUMN_NAME, col("data_file.partition")); @@ -614,49 +629,27 @@ private Table table() { } } - static class PartitionFieldSortFunction implements SerializableFunction { - private final PartitionSpec spec; - private final List partitionFieldSortOrder; + // UDF that will execute supplied custom manifest clustering function + static class CustomDataFileSorterUdf implements UDF1, Serializable { + // Supply how the DataFile should be interpreted from a raw Row. + private Types.StructType dataFileType; + private StructType dataFileSparkType; + private Function clusteringFunction; - public PartitionFieldSortFunction(PartitionSpec spec, List partitionFieldSortOrder) { - this.spec = spec; - this.partitionFieldSortOrder = partitionFieldSortOrder; + public CustomDataFileSorterUdf( + Function clusteringFunction, + Types.StructType dataFileType, + StructType dataFileSparkType) { + this.dataFileType = dataFileType; + this.dataFileSparkType = dataFileSparkType; + this.clusteringFunction = (Function & Serializable) clusteringFunction; } @Override - public String apply(DataFile dataFile) { - Map fieldIndices = Maps.newHashMap(); - int idx = 0; - for (PartitionField field : spec.fields()) { - fieldIndices.put(field.name(), idx); - idx++; - } - - StringBuilder sb = new StringBuilder(); - for (java.lang.String fieldName : partitionFieldSortOrder) { - Preconditions.checkArgument( - fieldIndices.containsKey(fieldName), - "Cannot find partition field with name %s in spec %s", - fieldName, - spec); - int fieldIndex = fieldIndices.get(fieldName); - PartitionField field = spec.fields().get(fieldIndex); - Type fieldType = spec.partitionType().field(field.fieldId()).type(); - Class javaClass = spec.javaClasses()[fieldIndex]; - java.lang.String fieldValue = - field - .transform() - .toHumanString(fieldType, get(dataFile.partition(), fieldIndex, javaClass)); - sb.append(fieldValue); - } - - return sb.toString(); - } - - @SuppressWarnings("unchecked") - private T get(StructLike data, int pos, Class javaClass) { - return data.get(pos, (Class) javaClass); + public String call(Row dataFile) throws Exception { + SparkDataFile wrapper = new SparkDataFile(dataFileType, dataFileSparkType); + return this.clusteringFunction.apply(wrapper.wrap(dataFile)); } } } 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 78cb97c339b7..7c2d1e9d9304 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 @@ -621,10 +621,10 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc // manifest should // include the lower bound of 0, and the last should have the upper bound of 9 assertThat(lowers.get(0)) - .withFailMessage("Lower bound of first manifest partition should be 0") + .as("Lower bound of first manifest partition should be 0") .isEqualTo(c3PartitionMin); assertThat(uppers.get(uppers.size() - 1)) - .withFailMessage("Lower bound of first manifest partition should be 0") + .as("Lower bound of first manifest partition should be 0") .isEqualTo(c3PartitionMax); } @@ -748,7 +748,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws // So the upper bound of the partitions tracked by the first file should be LEQ the lower bounds // of the second. Etc assertThat(uppers.get(0)) - .withFailMessage( + .as( "Upper bound of first manifest partition should be LESS THAN the lower bound of second") .isLessThan(lowers.get(1)); @@ -756,7 +756,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws c3Boundaries.forEach( boundary -> { assertThat(boundary.second() - boundary.first()) - .withFailMessage( + .as( "Manifest should contain less than the full range of c3 bucket partitions") .isLessThan(c3PartitionMax - c3PartitionMin); }); @@ -764,16 +764,16 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws // c3's Bucket(10) partition means our true lower bound = 0 and true upper bound is 9. The first // manifest should include the lower bound of 0, and the last should have the upper bound of 9 assertThat(lowers.get(0)) - .withFailMessage("Lower bound of first manifest c3 bucket partition should be 0") + .as("Lower bound of first manifest c3 bucket partition should be 0") .isEqualTo(0); assertThat(uppers.get(0)) - .withFailMessage("Upper bound of first manifest c3 bucket partition should be 4") + .as("Upper bound of first manifest c3 bucket partition should be 4") .isEqualTo(4); assertThat(lowers.get(1)) - .withFailMessage("Lower bound of second manifest c3 bucket partition should be 5") + .as("Lower bound of second manifest c3 bucket partition should be 5") .isEqualTo(5); assertThat(uppers.get(1)) - .withFailMessage("Upper bound of second manifest c3 bucket partition should be 9") + .as("Upper bound of second manifest c3 bucket partition should be 9") .isEqualTo(9); } From a2b69b9093313de8c113a6f363d4ac1ee79d68b4 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Mon, 4 Mar 2024 23:10:47 -0600 Subject: [PATCH 0008/1019] Rebased, updated docs --- .../iceberg/actions/RewriteManifests.java | 2 +- .../actions/RewriteManifestsSparkAction.java | 40 +++++++++---------- .../actions/TestRewriteManifestsAction.java | 6 +-- 3 files changed, 23 insertions(+), 25 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index 688e3b0ed67f..8fbc91dcd67a 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -77,7 +77,7 @@ default RewriteManifests sort(List partitionFieldSortOrder) { */ default RewriteManifests sort(Function sortStrategyFunction) { throw new UnsupportedOperationException( - this.getClass().getName() + " doesn't implement sort(Function)"); + this.getClass().getName() + " doesn't implement sort(Function)"); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 325f20af7a87..a99bb54f1f46 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -193,7 +193,8 @@ public RewriteManifestsSparkAction sort(List sortOrder) { @Override public RewriteManifests sort(Function partitionFieldsSortStrategy) { - this.partitionSortFunction = (Function & Serializable) partitionFieldsSortStrategy; + this.partitionSortFunction = + (Function & Serializable) partitionFieldsSortStrategy; return this; } @@ -310,23 +311,23 @@ private List writePartitionedManifests( manifestEntryDF.withColumn( CUSTOM_CLUSTERING_COLUMN_NAME, clusteringUdf.apply(col("data_file"))); } else if (partitionFieldSortOrder != null) { - LOG.info( - "Sorting manifests for specId {} by partition columns in order of {} ", - spec.specId(), - partitionFieldSortOrder); - - // Map the top level partition column names to the column name referenced within the manifest - // entry dataframe - Column[] actualPartitionColumns = - partitionFieldSortOrder.stream() - .map(p -> col("data_file.partition." + p)) - .toArray(Column[]::new); - - // Form a new temporary column to sort/cluster manifests on, based on the custom sort - // order provided - clusteredManifestEntryDF = - manifestEntryDF.withColumn( - CUSTOM_CLUSTERING_COLUMN_NAME, functions.struct(actualPartitionColumns)); + LOG.info( + "Sorting manifests for specId {} by partition columns in order of {} ", + spec.specId(), + partitionFieldSortOrder); + + // Map the top level partition column names to the column name referenced within the manifest + // entry dataframe + Column[] actualPartitionColumns = + partitionFieldSortOrder.stream() + .map(p -> col("data_file.partition." + p)) + .toArray(Column[]::new); + + // Form a new temporary column to sort/cluster manifests on, based on the custom sort + // order provided + clusteredManifestEntryDF = + manifestEntryDF.withColumn( + CUSTOM_CLUSTERING_COLUMN_NAME, functions.struct(actualPartitionColumns)); } else { clusteredManifestEntryDF = manifestEntryDF.withColumn(CUSTOM_CLUSTERING_COLUMN_NAME, col("data_file.partition")); @@ -629,7 +630,6 @@ private Table table() { } } - // UDF that will execute supplied custom manifest clustering function static class CustomDataFileSorterUdf implements UDF1, Serializable { // Supply how the DataFile should be interpreted from a raw Row. @@ -637,7 +637,7 @@ static class CustomDataFileSorterUdf implements UDF1, Serializable private StructType dataFileSparkType; private Function clusteringFunction; - public CustomDataFileSorterUdf( + CustomDataFileSorterUdf( Function clusteringFunction, Types.StructType dataFileType, StructType dataFileSparkType) { 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 7c2d1e9d9304..869aa17306f4 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 @@ -748,16 +748,14 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws // So the upper bound of the partitions tracked by the first file should be LEQ the lower bounds // of the second. Etc assertThat(uppers.get(0)) - .as( - "Upper bound of first manifest partition should be LESS THAN the lower bound of second") + .as("Upper bound of first manifest partition should be LESS THAN the lower bound of second") .isLessThan(lowers.get(1)); // Each file should contain less than the full c3 partition span c3Boundaries.forEach( boundary -> { assertThat(boundary.second() - boundary.first()) - .as( - "Manifest should contain less than the full range of c3 bucket partitions") + .as("Manifest should contain less than the full range of c3 bucket partitions") .isLessThan(c3PartitionMax - c3PartitionMin); }); From 3960aec17211ca91b3776ef72caa1a573e4c4e9f Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Wed, 6 Mar 2024 11:27:15 -0600 Subject: [PATCH 0009/1019] Renamed custom manifest rewriting to cluster instead of sort --- .../iceberg/actions/RewriteManifests.java | 24 +++++---- .../actions/RewriteManifestsSparkAction.java | 50 +++++++++---------- .../actions/TestRewriteManifestsAction.java | 41 ++++++++------- 3 files changed, 61 insertions(+), 54 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index 8fbc91dcd67a..603af96f5800 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -50,20 +50,23 @@ public interface RewriteManifests /** * Rewrite manifests in a given order, based on partition field names * - *

Supply an optional set of partition field names to sort the rewritten manifests by. Expects - * exact transformed column names used for partitioning; not the raw columnnames that partitions - * are derived from. E.G. supply 'data_bucket' and not 'data' for a bucket(N, data) partition - * definition + *

Supply an optional set of partition field names to cluster the rewritten manifests by. For + * example, given a table PARTITIONED BY (a, b, c, d), you may wish to rewrite and cluster + * manifests by ('d', 'b') only, based on your query patterns. Rewriting Manifests in this way + * will yield manifest_lists that point to manifest_files containing data files for common 'd' and + * 'b' partitions. * *

If not set, manifests will be rewritten in the order of the transforms in the table's * current partition spec. * - * @param partitionFieldSortOrder a list of partition field names + * @param partitionFieldClustering Exact transformed column names used for partitioning; not the + * raw column names that partitions are derived from. E.G. supply 'data_bucket' and not 'data' + * for a bucket(N, data) partition * definition * @return this for method chaining */ - default RewriteManifests sort(List partitionFieldSortOrder) { + default RewriteManifests clusterBy(List partitionFieldClustering) { throw new UnsupportedOperationException( - this.getClass().getName() + " doesn't implement sort(List)"); + this.getClass().getName() + " doesn't implement clusterBy(List)"); } /** @@ -72,12 +75,13 @@ default RewriteManifests sort(List partitionFieldSortOrder) { *

Supply a Function which will apply its own custom clustering logic based on supplied {@link * org.apache.iceberg.DataFile} attributes. * - * @param sortStrategyFunction A Function that returns a String to be used for manifest clustering + * @param clusterStrategyFunction A Function that returns a String to be used for manifest + * clustering * @return this method for chaining */ - default RewriteManifests sort(Function sortStrategyFunction) { + default RewriteManifests clusterBy(Function clusterStrategyFunction) { throw new UnsupportedOperationException( - this.getClass().getName() + " doesn't implement sort(Function)"); + this.getClass().getName() + " doesn't implement clusterBy(Function)"); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index a99bb54f1f46..648a3197730b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -115,8 +115,8 @@ public class RewriteManifestsSparkAction private Predicate predicate = manifest -> true; private String outputLocation = null; - private List partitionFieldSortOrder = null; - private Function partitionSortFunction = null; + private List partitionFieldClustering = null; + private Function partitionClusteringFunction = null; RewriteManifestsSparkAction(SparkSession spark, Table table) { super(spark); @@ -173,28 +173,28 @@ public RewriteManifestsSparkAction stagingLocation(String newStagingLocation) { } @Override - public RewriteManifestsSparkAction sort(List sortOrder) { - // Collect set of allowable partition columns to sort on + public RewriteManifestsSparkAction clusterBy(List clusteringColumns) { + // Collect set of allowable partition columns to cluster on Set availablePartitionNames = spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); // Check if these partition fields are included in the spec Preconditions.checkArgument( - sortOrder.stream().allMatch(availablePartitionNames::contains), - "Cannot use custom sort order to rewrite manifests '%s'. All partition columns must be " + clusteringColumns.stream().allMatch(availablePartitionNames::contains), + "Cannot use custom clustering to rewrite manifests '%s'. All partition columns must be " + "defined in the current partition spec: %s. Choose from the available partitionable columns: %s", - sortOrder, + clusteringColumns, this.spec.specId(), availablePartitionNames); - this.partitionFieldSortOrder = sortOrder; + this.partitionFieldClustering = clusteringColumns; return this; } @Override - public RewriteManifests sort(Function partitionFieldsSortStrategy) { - this.partitionSortFunction = - (Function & Serializable) partitionFieldsSortStrategy; + public RewriteManifests clusterBy(Function clusteringFunction) { + this.partitionClusteringFunction = + (Function & Serializable) clusteringFunction; return this; } @@ -288,42 +288,42 @@ private List writeUnpartitionedManifests( private List writePartitionedManifests( ManifestContent content, Dataset manifestEntryDF, int numManifests) { - // Extract desired clustering/sorting criteria into a dedicated column + // Extract desired clustering criteria into a dedicated column Dataset clusteredManifestEntryDF; - if (partitionSortFunction != null) { + if (partitionClusteringFunction != null) { LOG.info( - "Sorting manifests for specId {} using custom sorting function", + "Sorting manifests for specId {} using custom clustering function", spec.specId(), - partitionSortFunction); + partitionClusteringFunction); Types.StructType partitionType = DataFile.getType(table.spec().partitionType()); StructType dataFileSchema = manifestEntryDF.select("data_file.*").schema(); - // Create a UDF to wrap the custom partitionSortFunction call + // Create a UDF to wrap the custom partitionClusteringFunction call UserDefinedFunction clusteringUdf = functions.udf( - new CustomDataFileSorterUdf( - this.partitionSortFunction, partitionType, dataFileSchema), + new CustomDataFileClusteringUdf( + this.partitionClusteringFunction, partitionType, dataFileSchema), DataTypes.StringType); // Apply supplied partitionSortFunction function to the data_file datums within this dataframe // The results are stored as a String in the new __clustering_column__ clusteredManifestEntryDF = manifestEntryDF.withColumn( CUSTOM_CLUSTERING_COLUMN_NAME, clusteringUdf.apply(col("data_file"))); - } else if (partitionFieldSortOrder != null) { + } else if (partitionFieldClustering != null) { LOG.info( - "Sorting manifests for specId {} by partition columns in order of {} ", + "Clustering manifests for specId {} by partition columns by {} ", spec.specId(), - partitionFieldSortOrder); + partitionFieldClustering); // Map the top level partition column names to the column name referenced within the manifest // entry dataframe Column[] actualPartitionColumns = - partitionFieldSortOrder.stream() + partitionFieldClustering.stream() .map(p -> col("data_file.partition." + p)) .toArray(Column[]::new); - // Form a new temporary column to sort/cluster manifests on, based on the custom sort + // Form a new temporary column to cluster manifests on, based on the custom clustering columns // order provided clusteredManifestEntryDF = manifestEntryDF.withColumn( @@ -631,13 +631,13 @@ private Table table() { } // UDF that will execute supplied custom manifest clustering function - static class CustomDataFileSorterUdf implements UDF1, Serializable { + static class CustomDataFileClusteringUdf implements UDF1, Serializable { // Supply how the DataFile should be interpreted from a raw Row. private Types.StructType dataFileType; private StructType dataFileSparkType; private Function clusteringFunction; - CustomDataFileSorterUdf( + CustomDataFileClusteringUdf( Function clusteringFunction, Types.StructType dataFileType, StructType dataFileSparkType) { 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 869aa17306f4..6058c29dabeb 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 @@ -475,7 +475,8 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { } @TestTemplate - public void testRewriteManifestsPartitionedTableWithInvalidSortColumns() throws IOException { + public void testRewriteManifestsPartitionedTableWithInvalidClusteringColumns() + throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").bucket("c3", 10).build(); Map options = Maps.newHashMap(); options.put(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)); @@ -484,40 +485,40 @@ public void testRewriteManifestsPartitionedTableWithInvalidSortColumns() throws SparkActions actions = SparkActions.get(); - // c2 is not a partition column, cannot use for sorting - List badSortKeys1 = ImmutableList.of("c1", "c2"); + // c2 is not a partition column, cannot use for clustering + List badClusteringKeys1 = ImmutableList.of("c1", "c2"); assertThatThrownBy( () -> actions .rewriteManifests(table) .rewriteIf(manifest -> true) - .sort(badSortKeys1) + .clusterBy(badClusteringKeys1) .execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot use custom sort order to rewrite manifests '[c1, c2]'. All partition columns " + "Cannot use custom clustering to rewrite manifests '[c1, c2]'. All partition columns " + "must be defined in the current partition spec: 0. Choose from the available " + "partitionable columns: [c3_bucket, c1]"); // c3_bucket is the correct internal partition name to use, c3 is the untransformed column name, - // sort() expects the hidden partition column names - List badSortKeys2 = ImmutableList.of("c1", "c3"); + // clusterBy() expects the hidden partition column names + List badClusteringKeys2 = ImmutableList.of("c1", "c3"); assertThatThrownBy( () -> actions .rewriteManifests(table) .rewriteIf(manifest -> true) - .sort(badSortKeys2) + .clusterBy(badClusteringKeys2) .execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot use custom sort order to rewrite manifests '[c1, c3]'. All partition columns " + "Cannot use custom clustering to rewrite manifests '[c1, c3]'. All partition columns " + "must be defined in the current partition spec: 0. Choose from the available " + "partitionable columns: [c3_bucket, c1]"); } @TestTemplate - public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOException { + public void testRewriteManifestsPartitionedTableWithCustomClustering() throws IOException { Random random = new Random(); PartitionSpec spec = @@ -558,12 +559,12 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc SparkActions actions = SparkActions.get(); - List manifestSortKeys = ImmutableList.of("c3_bucket", "c2_trunc", "c1"); + List manifestClusterKeys = ImmutableList.of("c3_bucket", "c2_trunc", "c1"); RewriteManifests.Result result = actions .rewriteManifests(table) .rewriteIf(manifest -> true) - .sort(manifestSortKeys) + .clusterBy(manifestClusterKeys) .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) .execute(); @@ -596,10 +597,10 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc List lowers = c3Boundaries.stream().map(t -> t.first()).collect(Collectors.toList()); List uppers = c3Boundaries.stream().map(t -> t.second()).collect(Collectors.toList()); - // With custom sorting, this looks like + // With custom clustering, this looks like // - manifest 1 -> [lower bound = 0, upper bound = 4] // - manifest 2 -> [lower bound = 4, upper bound = 9] - // Without the custom sorting, each manifest tracks the full range of c3 upper/lower bounds. + // Without the custom clustering, each manifest tracks the full range of c3 upper/lower bounds. // AKA they look like // - manifest 1 -> [lower bound = 0, upper bound = 9] // - manifest 2 -> [lower bound = 0, upper bound = 9] @@ -629,7 +630,8 @@ public void testRewriteManifestsPartitionedTableWithCustomSorting() throws IOExc } @TestTemplate - public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws IOException { + public void testRewriteManifestsPartitionedTableWithCustomClusteringFunction() + throws IOException { Random random = new Random(); PartitionSpec spec = @@ -671,7 +673,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws SparkActions actions = SparkActions.get(); // This is the main point of this test! - // Let's say I want to sort manifests into buckets 0-4 and 5 - 9 + // Let's say I want to cluster manifests into buckets 0-4 and 5 - 9 // I know my Table and partition Spec, so can do that programmatically Function test = (Function & Serializable) @@ -694,7 +696,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws actions .rewriteManifests(table) .rewriteIf(manifest -> true) - .sort(test) + .clusterBy(test) .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) .execute(); @@ -726,7 +728,8 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws List lowers = c3Boundaries.stream().map(t -> t.first()).collect(Collectors.toList()); List uppers = c3Boundaries.stream().map(t -> t.second()).collect(Collectors.toList()); - // The custom function sorts datafiles by having c3 bucket partitions in the range 0-4 and 5-9. + // The custom function clusters datafiles by having c3 bucket partitions in the range 0-4 and + // 5-9. // Internally, that looks like // +--------------------+---------------------+ // |partition |__clustering_column__| @@ -741,7 +744,7 @@ public void testRewriteManifestsPartitionedTableWithCustomSortFunction() throws // After rewriting, this looks like // - manifest 1 -> [lower bound = 0, upper bound = 4] // - manifest 2 -> [lower bound = 5, upper bound = 9] - // Without the custom sorting, each manifest tracks the full range of c3 upper/lower bounds. + // Without the custom clustering, each manifest tracks the full range of c3 upper/lower bounds. // AKA they look like // - manifest 1 -> [lower bound = 0, upper bound = 9] // - manifest 2 -> [lower bound = 0, upper bound = 9] From 64f000ba780d50f97125b8e2855cbaf177a6bbcd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 19:55:20 +0100 Subject: [PATCH 0010/1019] Build: Bump datamodel-code-generator from 0.25.2 to 0.25.3 (#9639) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.2 to 0.25.3. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.2...0.25.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 5c83ae84d444..a2135acda0be 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.25.2 +datamodel-code-generator==0.25.3 From 13ef3210009bc14d863e4e33b6297a5df775e71b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 19:55:35 +0100 Subject: [PATCH 0011/1019] Build: Bump mkdocs-material from 9.5.3 to 9.5.7 (#9638) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.3 to 9.5.7. - [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.3...9.5.7) --- 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 abae9c1dc0ea..b4533dc8ce49 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.3 +mkdocs-material==9.5.7 mkdocs-material-extensions==1.3 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix mkdocs-redirects==1.2.1 From 4779f73375e766e514351fa40a80399bda033076 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 19:56:10 +0100 Subject: [PATCH 0012/1019] Build: Bump org.xerial:sqlite-jdbc from 3.44.0.0 to 3.45.1.0 (#9634) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.44.0.0 to 3.45.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.44.0.0...3.45.1.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 2304daf704f2..3b166c646e82 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,7 +84,7 @@ spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" -sqlite-jdbc = "3.44.0.0" +sqlite-jdbc = "3.45.1.0" testcontainers = "1.19.3" tez010 = "0.10.2" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 774f7c7758d4b9902c9bf38342ea813f30a34aab Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 20:11:11 +0100 Subject: [PATCH 0013/1019] Build: Bump mkdocs-material-extensions from 1.3 to 1.3.1 (#9160) Bumps [mkdocs-material-extensions](https://github.com/facelessuser/mkdocs-material-extensions) from 1.3 to 1.3.1. - [Release notes](https://github.com/facelessuser/mkdocs-material-extensions/releases) - [Changelog](https://github.com/facelessuser/mkdocs-material-extensions/blob/master/changelog.md) - [Commits](https://github.com/facelessuser/mkdocs-material-extensions/compare/1.3...1.3.1) --- updated-dependencies: - dependency-name: mkdocs-material-extensions 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 b4533dc8ce49..15c2958f4823 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -18,6 +18,6 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 mkdocs-material==9.5.7 -mkdocs-material-extensions==1.3 +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 61e503e7cee6ec0091ab460830fad56802765c9e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 20:11:50 +0100 Subject: [PATCH 0014/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.0.0 to 1.0.1 (#9317) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.0.0 to 1.0.1. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.0.0...1.0.1) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 3b166c646e82..fda2b5fb4c4d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -74,7 +74,7 @@ object-client-bundle = "3.3.2" orc = "1.9.2" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.0.0" +roaringbitmap = "1.0.1" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" From 3dbf7cf6c3f38038ba949cf0170a90d44caae1d6 Mon Sep 17 00:00:00 2001 From: Abid Mohammed Date: Tue, 6 Feb 2024 14:57:07 -0800 Subject: [PATCH 0015/1019] Core: Only trim trailing slash when warehouse location is not root path (#9619) Co-authored-by: Abid Mohammed Co-authored-by: Eduard Tudenhoefner --- .../org/apache/iceberg/util/LocationUtil.java | 2 +- .../apache/iceberg/util/TestLocationUtil.java | 26 +++++++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java index 192786155785..400307149238 100644 --- a/core/src/main/java/org/apache/iceberg/util/LocationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/LocationUtil.java @@ -28,7 +28,7 @@ public static String stripTrailingSlash(String path) { Preconditions.checkArgument(!Strings.isNullOrEmpty(path), "path must not be null or empty"); String result = path; - while (result.endsWith("/")) { + while (!result.endsWith("://") && result.endsWith("/")) { result = result.substring(0, result.length() - 1); } return result; diff --git a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java index f557352299b7..277500461659 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java @@ -58,4 +58,30 @@ public void testStripTrailingSlashWithInvalidPath() { .hasMessage("path must not be null or empty"); } } + + @Test + void testDoNotStripTrailingSlashForRootPath() { + String rootPath = "blobstore://"; + assertThat(LocationUtil.stripTrailingSlash(rootPath)) + .as("Should be root path") + .isEqualTo(rootPath); + } + + @Test + void testStripTrailingSlashForRootPathWithTrailingSlash() { + String rootPath = "blobstore://"; + String rootPathWithTrailingSlash = rootPath + "/"; + assertThat(LocationUtil.stripTrailingSlash(rootPathWithTrailingSlash)) + .as("Should be root path") + .isEqualTo(rootPath); + } + + @Test + void testStripTrailingSlashForRootPathWithTrailingSlashes() { + String rootPath = "blobstore://"; + String rootPathWithMultipleTrailingSlash = rootPath + "///"; + assertThat(LocationUtil.stripTrailingSlash(rootPathWithMultipleTrailingSlash)) + .as("Should be root path") + .isEqualTo(rootPath); + } } From bb90c661bb571bc3c873129a8721574ccc68286b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 17:27:12 -0800 Subject: [PATCH 0016/1019] Build: Bump io.delta:delta-spark_2.12 from 3.0.0 to 3.1.0 (#9631) --- 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 fda2b5fb4c4d..c91e1128ad72 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -34,7 +34,7 @@ awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" delta-standalone = "0.6.0" -delta-spark = "3.0.0" +delta-spark = "3.1.0" esotericsoftware-kryo = "4.0.2" errorprone-annotations = "2.24.1" findbugs-jsr305 = "3.0.2" From cc8a9083a47c342c7bb62ac1087e12bde5b679ef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 18:17:40 -0800 Subject: [PATCH 0017/1019] Build: Bump io.delta:delta-standalone_2.12 from 0.6.0 to 3.1.0 (#9636) --- 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 c91e1128ad72..8b01d1d50799 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -33,7 +33,7 @@ azuresdk-bom = "1.2.18" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" -delta-standalone = "0.6.0" +delta-standalone = "3.1.0" delta-spark = "3.1.0" esotericsoftware-kryo = "4.0.2" errorprone-annotations = "2.24.1" From 8303147765a0ea67fc8036b03a4ecb3f465a5ef0 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 7 Feb 2024 08:13:22 +0100 Subject: [PATCH 0018/1019] Spark: Move the Writer to a visitor (#9440) * Spark: Move the Writer to a visitor * Add UUID * Add newline * Whitespace nits --- .../spark/data/SparkParquetWriters.java | 169 ++++++++++++------ 1 file changed, 119 insertions(+), 50 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index af6f65a089b6..b2e95df59eb8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.UUID; import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; import org.apache.iceberg.parquet.ParquetValueWriter; @@ -48,11 +49,9 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.ByteType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; -import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; @@ -90,7 +89,6 @@ public ParquetValueWriter struct( writers.add(newOption(struct.getType(i), fieldWriters.get(i))); sparkTypes.add(sparkFields[i].dataType()); } - return new InternalRowWriter(writers, sparkTypes); } @@ -136,46 +134,126 @@ private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter wr return ParquetValueWriters.option(fieldType, maxD, writer); } + private static class LogicalTypeAnnotationParquetValueWriterVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final ColumnDescriptor desc; + private final PrimitiveType primitive; + + LogicalTypeAnnotationParquetValueWriterVisitor( + ColumnDescriptor desc, PrimitiveType primitive) { + this.desc = desc; + this.primitive = primitive; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.UUIDLogicalTypeAnnotation uuidLogicalType) { + return Optional.of(uuids(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(mapLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(listLogicalType); + } + + @Override + public Optional> visit(DecimalLogicalTypeAnnotation decimal) { + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return Optional.of(decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale())); + case INT64: + return Optional.of(decimalAsLong(desc, decimal.getPrecision(), decimal.getScale())); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of(decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale())); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(ParquetValueWriters.ints(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int bitWidth = intLogicalType.getBitWidth(); + if (bitWidth <= 8) { + return Optional.of(ParquetValueWriters.tinyints(desc)); + } else if (bitWidth <= 16) { + return Optional.of(ParquetValueWriters.shorts(desc)); + } else if (bitWidth <= 32) { + return Optional.of(ParquetValueWriters.ints(desc)); + } else { + return Optional.of(ParquetValueWriters.longs(desc)); + } + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(byteArrays(desc)); + } + } + @Override public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return utf8Strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(sType, desc); - case INT_64: - case TIME_MICROS: - case TIMESTAMP_MICROS: - return ParquetValueWriters.longs(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueWriterVisitor(desc, primitive)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); } switch (primitive.getPrimitiveTypeName()) { @@ -188,7 +266,7 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) case BOOLEAN: return ParquetValueWriters.booleans(desc); case INT32: - return ints(sType, desc); + return ParquetValueWriters.ints(desc); case INT64: return ParquetValueWriters.longs(desc); case FLOAT: @@ -201,15 +279,6 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) } } - private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { - if (type instanceof ByteType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof ShortType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { return new UTF8StringWriter(desc); } From a96a866c169990f01fa269e9c4d10ff164d30dcb Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 7 Feb 2024 08:17:53 +0100 Subject: [PATCH 0019/1019] Azure: Bump Azurite container (#9668) To check if it works against the latest version of the Azure SDK: https://github.com/apache/iceberg/pull/9571 --- .../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 1ae320b45493..ff2b9cbc5d0c 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.26.0"; + private static final String DEFAULT_TAG = "3.29.0"; private static final String LOG_WAIT_REGEX = "Azurite Blob service is successfully listening at .*"; From cf77ab7f018ee53d0106395cc571ffce2ba26175 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 7 Feb 2024 08:32:43 +0100 Subject: [PATCH 0020/1019] Spark: Handle concurrently dropped view during CREATE OR REPLACE (#9623) --- .../datasources/v2/CreateV2ViewExec.scala | 73 +++++++++++-------- .../apache/iceberg/spark/SparkCatalog.java | 4 +- 2 files changed, 46 insertions(+), 31 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index 388d391a4cae..d2db0f4992fe 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -21,6 +21,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.SupportsReplaceView import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier @@ -59,45 +60,24 @@ case class CreateV2ViewExec( // CREATE OR REPLACE VIEW catalog match { case c: SupportsReplaceView => - c.replaceView( - ident, - queryText, - currentCatalog, - currentNamespace, - viewSchema, - queryColumnNames.toArray, - columnAliases.toArray, - columnComments.map(c => c.orNull).toArray, - newProperties.asJava) + try { + replaceView(c, currentCatalog, currentNamespace, newProperties) + } catch { + // view might have been concurrently dropped during replace + case _: NoSuchViewException => + replaceView(c, currentCatalog, currentNamespace, newProperties) + } case _ => if (catalog.viewExists(ident)) { catalog.dropView(ident) } - catalog.createView( - ident, - queryText, - currentCatalog, - currentNamespace, - viewSchema, - queryColumnNames.toArray, - columnAliases.toArray, - columnComments.map(c => c.orNull).toArray, - newProperties.asJava) + createView(currentCatalog, currentNamespace, newProperties) } } else { try { // CREATE VIEW [IF NOT EXISTS] - catalog.createView( - ident, - queryText, - currentCatalog, - currentNamespace, - viewSchema, - queryColumnNames.toArray, - columnAliases.toArray, - columnComments.map(c => c.orNull).toArray, - newProperties.asJava) + createView(currentCatalog, currentNamespace, newProperties) } catch { case _: ViewAlreadyExistsException if allowExisting => // Ignore } @@ -106,6 +86,39 @@ case class CreateV2ViewExec( Nil } + private def replaceView( + supportsReplaceView: SupportsReplaceView, + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { + supportsReplaceView.replaceView( + ident, + queryText, + currentCatalog, + currentNamespace, + viewSchema, + queryColumnNames.toArray, + columnAliases.toArray, + columnComments.map(c => c.orNull).toArray, + newProperties.asJava) + } + + private def createView( + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { + catalog.createView( + ident, + queryText, + currentCatalog, + currentNamespace, + viewSchema, + queryColumnNames.toArray, + columnAliases.toArray, + columnComments.map(c => c.orNull).toArray, + newProperties.asJava) + } + override def simpleString(maxFields: Int): String = { s"CreateV2ViewExec: ${ident}" } 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 0483926c4cd1..7357a4683bed 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 @@ -619,7 +619,7 @@ public View replaceView( String[] columnAliases, String[] columnComments, Map properties) - throws NoSuchNamespaceException { + throws NoSuchNamespaceException, NoSuchViewException { if (null != asViewCatalog) { Schema icebergSchema = SparkSchemaUtil.convert(schema); @@ -643,6 +643,8 @@ public View replaceView( return new SparkView(catalogName, view); } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { throw new NoSuchNamespaceException(currentNamespace); + } catch (org.apache.iceberg.exceptions.NoSuchViewException e) { + throw new NoSuchViewException(ident); } } From c693af42194f3b425c8c7bd90deb9653c12b29e5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Feb 2024 09:10:22 +0100 Subject: [PATCH 0021/1019] Build: Bump com.azure:azure-sdk-bom from 1.2.18 to 1.2.20 (#9571) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.18 to 1.2.20. - [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.18...azure-sdk-bom_1.2.20) --- 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 8b01d1d50799..0cbe70c3b6f1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.25.2" awaitility = "4.2.0" awssdk-bom = "2.23.17" -azuresdk-bom = "1.2.18" +azuresdk-bom = "1.2.20" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From f853e8cb42b451a6c5fb0d5f6aa65f5e97c10de3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Feb 2024 09:10:52 +0100 Subject: [PATCH 0022/1019] Build: Bump org.testcontainers:testcontainers from 1.19.3 to 1.19.4 (#9577) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.3 to 1.19.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.19.3...1.19.4) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 0cbe70c3b6f1..4b805159e9bf 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -85,7 +85,7 @@ spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" sqlite-jdbc = "3.45.1.0" -testcontainers = "1.19.3" +testcontainers = "1.19.4" tez010 = "0.10.2" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 2a6a89daaa49d06be01dfcf212d756171bf8c792 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 7 Feb 2024 09:43:41 +0100 Subject: [PATCH 0023/1019] Spark 3.3: Move the Writer to a visitor (#9672) Backport of https://github.com/apache/iceberg/pull/9440 --- .../spark/data/SparkParquetWriters.java | 168 +++++++++++++----- 1 file changed, 119 insertions(+), 49 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index af6f65a089b6..8baea6c5ab59 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.UUID; import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; import org.apache.iceberg.parquet.ParquetValueWriter; @@ -48,11 +49,9 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.ByteType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; -import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; @@ -136,46 +135,126 @@ private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter wr return ParquetValueWriters.option(fieldType, maxD, writer); } + private static class LogicalTypeAnnotationParquetValueWriterVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final ColumnDescriptor desc; + private final PrimitiveType primitive; + + LogicalTypeAnnotationParquetValueWriterVisitor( + ColumnDescriptor desc, PrimitiveType primitive) { + this.desc = desc; + this.primitive = primitive; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.UUIDLogicalTypeAnnotation uuidLogicalType) { + return Optional.of(uuids(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(mapLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(listLogicalType); + } + + @Override + public Optional> visit(DecimalLogicalTypeAnnotation decimal) { + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return Optional.of(decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale())); + case INT64: + return Optional.of(decimalAsLong(desc, decimal.getPrecision(), decimal.getScale())); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of(decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale())); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(ParquetValueWriters.ints(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int bitWidth = intLogicalType.getBitWidth(); + if (bitWidth <= 8) { + return Optional.of(ParquetValueWriters.tinyints(desc)); + } else if (bitWidth <= 16) { + return Optional.of(ParquetValueWriters.shorts(desc)); + } else if (bitWidth <= 32) { + return Optional.of(ParquetValueWriters.ints(desc)); + } else { + return Optional.of(ParquetValueWriters.longs(desc)); + } + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(byteArrays(desc)); + } + } + @Override public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return utf8Strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(sType, desc); - case INT_64: - case TIME_MICROS: - case TIMESTAMP_MICROS: - return ParquetValueWriters.longs(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueWriterVisitor(desc, primitive)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); } switch (primitive.getPrimitiveTypeName()) { @@ -188,7 +267,7 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) case BOOLEAN: return ParquetValueWriters.booleans(desc); case INT32: - return ints(sType, desc); + return ParquetValueWriters.ints(desc); case INT64: return ParquetValueWriters.longs(desc); case FLOAT: @@ -201,15 +280,6 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) } } - private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { - if (type instanceof ByteType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof ShortType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { return new UTF8StringWriter(desc); } From 76ce3a0a896f75410b91b782c33a02dbc2f7cf96 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 7 Feb 2024 09:44:23 +0100 Subject: [PATCH 0024/1019] Spark 3.4: Move the Writer to a visitor (#9673) Backport of https://github.com/apache/iceberg/pull/9440 --- .../spark/data/SparkParquetWriters.java | 168 +++++++++++++----- 1 file changed, 119 insertions(+), 49 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index af6f65a089b6..8baea6c5ab59 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; import java.util.NoSuchElementException; +import java.util.Optional; import java.util.UUID; import org.apache.iceberg.parquet.ParquetValueReaders.ReusableEntry; import org.apache.iceberg.parquet.ParquetValueWriter; @@ -48,11 +49,9 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.ArrayType; -import org.apache.spark.sql.types.ByteType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; -import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; @@ -136,46 +135,126 @@ private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter wr return ParquetValueWriters.option(fieldType, maxD, writer); } + private static class LogicalTypeAnnotationParquetValueWriterVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final ColumnDescriptor desc; + private final PrimitiveType primitive; + + LogicalTypeAnnotationParquetValueWriterVisitor( + ColumnDescriptor desc, PrimitiveType primitive) { + this.desc = desc; + this.primitive = primitive; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(utf8Strings(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.UUIDLogicalTypeAnnotation uuidLogicalType) { + return Optional.of(uuids(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.MapLogicalTypeAnnotation mapLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(mapLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.ListLogicalTypeAnnotation listLogicalType) { + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(listLogicalType); + } + + @Override + public Optional> visit(DecimalLogicalTypeAnnotation decimal) { + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return Optional.of(decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale())); + case INT64: + return Optional.of(decimalAsLong(desc, decimal.getPrecision(), decimal.getScale())); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of(decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale())); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(ParquetValueWriters.ints(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(ParquetValueWriters.longs(desc)); + } + return Optional.empty(); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int bitWidth = intLogicalType.getBitWidth(); + if (bitWidth <= 8) { + return Optional.of(ParquetValueWriters.tinyints(desc)); + } else if (bitWidth <= 16) { + return Optional.of(ParquetValueWriters.shorts(desc)); + } else if (bitWidth <= 32) { + return Optional.of(ParquetValueWriters.ints(desc)); + } else { + return Optional.of(ParquetValueWriters.longs(desc)); + } + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(byteArrays(desc)); + } + } + @Override public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) { ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return utf8Strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(sType, desc); - case INT_64: - case TIME_MICROS: - case TIMESTAMP_MICROS: - return ParquetValueWriters.longs(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueWriterVisitor(desc, primitive)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); } switch (primitive.getPrimitiveTypeName()) { @@ -188,7 +267,7 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) case BOOLEAN: return ParquetValueWriters.booleans(desc); case INT32: - return ints(sType, desc); + return ParquetValueWriters.ints(desc); case INT64: return ParquetValueWriters.longs(desc); case FLOAT: @@ -201,15 +280,6 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) } } - private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { - if (type instanceof ByteType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof ShortType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { return new UTF8StringWriter(desc); } From a7cd6afa084dda4d81c23c5e7d8a67225ca5644c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 7 Feb 2024 10:35:35 +0100 Subject: [PATCH 0025/1019] Spark: Avoid NPE when catalog config doesn't have "type" set (#9676) We do the same check in `SparkTestBaseWithCatalog` to avoid a potential NPE when a catalog configuration doesn't have the `type` set --- .../test/java/org/apache/iceberg/spark/TestBaseWithCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 be0d8c6439a2..c3c958abf0fd 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 @@ -101,7 +101,7 @@ public void before() { catalogConfig.forEach( (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value)); - if (catalogConfig.get("type").equalsIgnoreCase("hadoop")) { + if ("hadoop".equalsIgnoreCase(catalogConfig.get("type"))) { spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); } From 0a3b30a95b2a191c21e2a286264bf31cca8dd4b1 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 7 Feb 2024 15:18:06 +0100 Subject: [PATCH 0026/1019] Spark 3.4: Handle concurrently dropped view during CREATE OR REPLACE (#9677) --- .../datasources/v2/CreateV2ViewExec.scala | 73 +++++++++++-------- .../apache/iceberg/spark/SparkCatalog.java | 4 +- 2 files changed, 46 insertions(+), 31 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala index 388d391a4cae..d2db0f4992fe 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateV2ViewExec.scala @@ -21,6 +21,7 @@ package org.apache.spark.sql.execution.datasources.v2 import org.apache.iceberg.spark.SupportsReplaceView import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchViewException import org.apache.spark.sql.catalyst.analysis.ViewAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.connector.catalog.Identifier @@ -59,45 +60,24 @@ case class CreateV2ViewExec( // CREATE OR REPLACE VIEW catalog match { case c: SupportsReplaceView => - c.replaceView( - ident, - queryText, - currentCatalog, - currentNamespace, - viewSchema, - queryColumnNames.toArray, - columnAliases.toArray, - columnComments.map(c => c.orNull).toArray, - newProperties.asJava) + try { + replaceView(c, currentCatalog, currentNamespace, newProperties) + } catch { + // view might have been concurrently dropped during replace + case _: NoSuchViewException => + replaceView(c, currentCatalog, currentNamespace, newProperties) + } case _ => if (catalog.viewExists(ident)) { catalog.dropView(ident) } - catalog.createView( - ident, - queryText, - currentCatalog, - currentNamespace, - viewSchema, - queryColumnNames.toArray, - columnAliases.toArray, - columnComments.map(c => c.orNull).toArray, - newProperties.asJava) + createView(currentCatalog, currentNamespace, newProperties) } } else { try { // CREATE VIEW [IF NOT EXISTS] - catalog.createView( - ident, - queryText, - currentCatalog, - currentNamespace, - viewSchema, - queryColumnNames.toArray, - columnAliases.toArray, - columnComments.map(c => c.orNull).toArray, - newProperties.asJava) + createView(currentCatalog, currentNamespace, newProperties) } catch { case _: ViewAlreadyExistsException if allowExisting => // Ignore } @@ -106,6 +86,39 @@ case class CreateV2ViewExec( Nil } + private def replaceView( + supportsReplaceView: SupportsReplaceView, + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { + supportsReplaceView.replaceView( + ident, + queryText, + currentCatalog, + currentNamespace, + viewSchema, + queryColumnNames.toArray, + columnAliases.toArray, + columnComments.map(c => c.orNull).toArray, + newProperties.asJava) + } + + private def createView( + currentCatalog: String, + currentNamespace: Array[String], + newProperties: Map[String, String]) = { + catalog.createView( + ident, + queryText, + currentCatalog, + currentNamespace, + viewSchema, + queryColumnNames.toArray, + columnAliases.toArray, + columnComments.map(c => c.orNull).toArray, + newProperties.asJava) + } + override def simpleString(maxFields: Int): String = { s"CreateV2ViewExec: ${ident}" } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index f3b91c4145b8..d72a281fe825 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -619,7 +619,7 @@ public View replaceView( String[] columnAliases, String[] columnComments, Map properties) - throws NoSuchNamespaceException { + throws NoSuchNamespaceException, NoSuchViewException { if (null != asViewCatalog) { Schema icebergSchema = SparkSchemaUtil.convert(schema); @@ -643,6 +643,8 @@ public View replaceView( return new SparkView(catalogName, view); } catch (org.apache.iceberg.exceptions.NoSuchNamespaceException e) { throw new NoSuchNamespaceException(currentNamespace); + } catch (org.apache.iceberg.exceptions.NoSuchViewException e) { + throw new NoSuchViewException(ident); } } From 7305d22f1368a632bcc1c1818dae27d31e3abad1 Mon Sep 17 00:00:00 2001 From: Scott Teal Date: Wed, 7 Feb 2024 16:54:59 -0800 Subject: [PATCH 0027/1019] Docs: Add/update Snowflake (#9669) * Update sidebar Add Snowflake and make it closer to alphabetical order, keeping open source toward the top, vendors toward the bottom * Update Snowflake's description Include latest docs links * Add Snowflake without disrupting ordering --- docs/mkdocs.yml | 1 + site/docs/vendors.md | 3 +-- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 061daf135bbd..e6b6a0052f16 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -55,6 +55,7 @@ nav: - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog - Amazon Athena: https://docs.aws.amazon.com/athena/latest/ug/querying-iceberg.html - Amazon EMR: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html + - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - Doris: https://doris.apache.org/docs/dev/lakehouse/multi-catalog/iceberg - Integrations: diff --git a/site/docs/vendors.md b/site/docs/vendors.md index ecafd8d8ae07..7609dcdf1903 100644 --- a/site/docs/vendors.md +++ b/site/docs/vendors.md @@ -59,8 +59,7 @@ IOMETE is a fully-managed ready to use, batteries included Data Platform. IOMETE PuppyGraph is a cloud-native graph analytics engine that enables users to query one or more relational data stores as a unified graph model. This eliminates the overhead of deploying and maintaining a siloed graph database system, with no ETL required. [PuppyGraph’s native Apache Iceberg integration](https://docs.puppygraph.com/user-manual/getting-started/iceberg) adds native graph capabilities to your existing data lake in an easy and performant way. ### [Snowflake](http://snowflake.com/) - -[Snowflake](https://www.snowflake.com/data-cloud/) is a single, cross-cloud platform that enables every organization to mobilize their data with Snowflake’s Data Cloud. Snowflake supports Apache Iceberg by offering [native support for Iceberg Tables](https://www.snowflake.com/blog/iceberg-tables-powering-open-standards-with-snowflake-innovations/) for full DML as well as connectors to [External Tables](https://www.snowflake.com/blog/expanding-the-data-cloud-with-apache-iceberg/) for read-only access. +[Snowflake](https://www.snowflake.com/en/) is a single, cross-cloud platform that enables every organization to mobilize their data with Snowflake’s Data Cloud. Snowflake supports Apache Iceberg by offering [Snowflake-managed Iceberg Tables](https://docs.snowflake.com/en/user-guide/tables-iceberg#use-snowflake-as-the-iceberg-catalog) for full DML as well as [externally managed Iceberg Tables with catalog integrations](https://docs.snowflake.com/en/user-guide/tables-iceberg#use-a-catalog-integration) for read-only access. ### [Starburst](http://starburst.io) From 9f9ac3d7f625e67bfe81a8cad3e1f69ebbb221f5 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 8 Feb 2024 03:35:16 -0800 Subject: [PATCH 0028/1019] open-api: Use openapi-generator-gradle-plugin for validating specification (#9344) Co-authored-by: Steve Zhang --- build.gradle | 17 ++++++++++ open-api/rest-catalog-open-api.py | 11 ++++++- open-api/rest-catalog-open-api.yaml | 50 ++++++++++++++--------------- settings.gradle | 2 ++ 4 files changed, 54 insertions(+), 26 deletions(-) diff --git a/build.gradle b/build.gradle index d99c6fa2a06b..00ca99e9456a 100644 --- a/build.gradle +++ b/build.gradle @@ -41,6 +41,7 @@ buildscript { classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.1' classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.0.0' + classpath 'org.openapitools:openapi-generator-gradle-plugin:6.6.0' } } @@ -531,6 +532,13 @@ project(':iceberg-aws') { classpath = sourceSets.integration.runtimeClasspath jvmArgs += project.property('extraJvmArgs') } + + def s3SignerSpec = "$projectDir/src/main/resources/s3-signer-open-api.yaml" + tasks.register('validateS3SignerSpec', org.openapitools.generator.gradle.plugin.tasks.ValidateTask) { + inputSpec.set(s3SignerSpec) + recommend.set(true) + } + check.dependsOn('validateS3SignerSpec') } project(':iceberg-azure') { @@ -964,6 +972,15 @@ project(':iceberg-snowflake') { } } +project(':iceberg-open-api') { + def restCatalogSpec = "$projectDir/rest-catalog-open-api.yaml" + tasks.register('validateRESTCatalogSpec', org.openapitools.generator.gradle.plugin.tasks.ValidateTask) { + inputSpec.set(restCatalogSpec) + recommend.set(true) + } + check.dependsOn('validateRESTCatalogSpec') +} + @Memoized boolean versionFileExists() { return file('version.txt').exists() diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 719e77b0ad33..c8889271b203 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -424,7 +424,16 @@ class AssertDefaultSortOrderId(TableRequirement): class ViewRequirement(BaseModel): - __root__: Any = Field(..., discriminator='type') + type: str + + +class AssertViewUUID(ViewRequirement): + """ + The view UUID must match the requirement's `uuid` + """ + + type: Literal['assert-view-uuid'] + uuid: str class RegisterTableRequest(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index b21527946216..1048425147e9 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2632,26 +2632,26 @@ components: propertyName: type mapping: assert-view-uuid: '#/components/schemas/AssertViewUUID' - type: object - required: - - type - properties: - type: - type: "string" + type: object + required: + - type + properties: + type: + type: "string" - AssertViewUUID: - allOf: - - $ref: "#/components/schemas/ViewRequirement" - description: The view UUID must match the requirement's `uuid` - required: - - type - - uuid - properties: - type: - type: string - enum: [ "assert-view-uuid" ] - uuid: - type: string + AssertViewUUID: + allOf: + - $ref: "#/components/schemas/ViewRequirement" + description: The view UUID must match the requirement's `uuid` + required: + - type + - uuid + properties: + type: + type: string + enum: [ "assert-view-uuid" ] + uuid: + type: string LoadTableResult: description: | @@ -3363,12 +3363,12 @@ components: application/json: schema: $ref: '#/components/schemas/IcebergErrorResponse' - example: { - "error": { - "message": "The server does not support this operation", - "type": "UnsupportedOperationException", - "code": 406 - } } + example: { + "error": { + "message": "The server does not support this operation", + "type": "UnsupportedOperationException", + "code": 406 + } } CreateNamespaceResponse: description: diff --git a/settings.gradle b/settings.gradle index 3fd93db24928..15bb83754e14 100644 --- a/settings.gradle +++ b/settings.gradle @@ -42,6 +42,7 @@ include 'dell' include 'snowflake' include 'delta-lake' include 'kafka-connect' +include 'open-api' project(':bom').name = 'iceberg-bom' project(':api').name = 'iceberg-api' @@ -67,6 +68,7 @@ project(':dell').name = 'iceberg-dell' project(':snowflake').name = 'iceberg-snowflake' project(':delta-lake').name = 'iceberg-delta-lake' project(':kafka-connect').name = 'iceberg-kafka-connect' +project(':open-api').name = 'iceberg-open-api' if (null != System.getProperty("allVersions")) { System.setProperty("flinkVersions", System.getProperty("knownFlinkVersions")) From dca61f128653193b67f0ae15df317c21c446997f Mon Sep 17 00:00:00 2001 From: Muna Bedan <45054928+munabedan@users.noreply.github.com> Date: Thu, 8 Feb 2024 17:47:53 +0300 Subject: [PATCH 0029/1019] Docs: Fix hidden-partition-animation not showing (#9686) --- site/docs/assets/lottie/hidden-partitioning-animation.json | 1 + site/overrides/home.html | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) create mode 100644 site/docs/assets/lottie/hidden-partitioning-animation.json diff --git a/site/docs/assets/lottie/hidden-partitioning-animation.json b/site/docs/assets/lottie/hidden-partitioning-animation.json new file mode 100644 index 000000000000..0689a8f0b930 --- /dev/null +++ b/site/docs/assets/lottie/hidden-partitioning-animation.json @@ -0,0 +1 @@ +{"v":"5.5.7","meta":{"g":"LottieFiles AE 0.1.20","a":"Blockcities","k":"block, shares, fractional","d":"Fractionalized Block","tc":"#000000"},"fr":30,"ip":0,"op":600,"w":2500,"h":2160,"nm":"TechnoCub","ddd":0,"assets":[{"id":"comp_0","layers":[{"ddd":0,"ind":1,"ty":4,"nm":"04","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,125.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill 2","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[0.992156863213,0.258823543787,0.992156863213,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0002');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[48.5,-391.5],[-1240,357],[48,1100],[1343.5,352.5]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":5,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":false},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":true},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":2,"ty":4,"nm":"06","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,125.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill 2","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[0.992156863213,0.258823543787,0.992156863213,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0002');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[-1239,1846.059],[48.25,2588.812],[48,1100],[-1239.5,356.875]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":5,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":false},{"ty":"fl","c":{"a":0,"k":[0.15294117647058825,0.47843137254901963,0.7450980392156863,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":true},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":3,"ty":4,"nm":"05","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,125.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill 2","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[0.992156863213,0.258823543787,0.992156863213,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0002');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[1343.751,1842.45],[48.251,2589.95],[48,1100],[1343.5,352.5]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":5,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":false},{"ty":"fl","c":{"a":0,"k":[0.40784313725490196,0.7607843137254902,0.9215686274509803,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":true},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":4,"ty":4,"nm":"03","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,125.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Gradient Ramp","np":10,"mn":"ADBE Ramp","ix":1,"en":1,"ef":[{"ty":3,"nm":"Start of Ramp","mn":"ADBE Ramp-0001","ix":1,"v":{"a":0,"k":[382.317,210.45],"ix":1}},{"ty":2,"nm":"Start Color","mn":"ADBE Ramp-0002","ix":2,"v":{"a":0,"k":[0.20000000298,0.490196079016,0.941176474094,1],"ix":2,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0002');"}},{"ty":3,"nm":"End of Ramp","mn":"ADBE Ramp-0003","ix":3,"v":{"a":0,"k":[2434.217,250.383],"ix":3}},{"ty":2,"nm":"End Color","mn":"ADBE Ramp-0004","ix":4,"v":{"a":0,"k":[0.098039217293,0.901960790157,0.847058832645,1],"ix":4,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0003');"}},{"ty":7,"nm":"Ramp Shape","mn":"ADBE Ramp-0005","ix":5,"v":{"a":0,"k":1,"ix":5}},{"ty":0,"nm":"Ramp Scatter","mn":"ADBE Ramp-0006","ix":6,"v":{"a":0,"k":6,"ix":6}},{"ty":0,"nm":"Blend With Original","mn":"ADBE Ramp-0007","ix":7,"v":{"a":0,"k":0,"ix":7}},{"ty":6,"nm":"","mn":"ADBE Ramp-0008","ix":8,"v":0}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[48.5,-391.5],[-1240,357],[48,1100],[1343.5,352.5]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":5,"ty":4,"nm":"02","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,125.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Gradient Ramp","np":10,"mn":"ADBE Ramp","ix":1,"en":1,"ef":[{"ty":3,"nm":"Start of Ramp","mn":"ADBE Ramp-0001","ix":1,"v":{"a":0,"k":[2562.317,1114.45],"ix":1}},{"ty":2,"nm":"Start Color","mn":"ADBE Ramp-0002","ix":2,"v":{"a":0,"k":[0.20000000298,0.490196079016,0.941176474094,1],"ix":2,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0002');"}},{"ty":3,"nm":"End of Ramp","mn":"ADBE Ramp-0003","ix":3,"v":{"a":0,"k":[198.217,494.383],"ix":3}},{"ty":2,"nm":"End Color","mn":"ADBE Ramp-0004","ix":4,"v":{"a":0,"k":[0.098039217293,0.901960790157,0.847058832645,1],"ix":4,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0003');"}},{"ty":7,"nm":"Ramp Shape","mn":"ADBE Ramp-0005","ix":5,"v":{"a":0,"k":1,"ix":5}},{"ty":0,"nm":"Ramp Scatter","mn":"ADBE Ramp-0006","ix":6,"v":{"a":0,"k":6,"ix":6}},{"ty":0,"nm":"Blend With Original","mn":"ADBE Ramp-0007","ix":7,"v":{"a":0,"k":0,"ix":7}},{"ty":6,"nm":"","mn":"ADBE Ramp-0008","ix":8,"v":0}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[-1239,1846.059],[48.25,2588.812],[48,1100],[-1239.5,356.875]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.15294117647058825,0.47843137254901963,0.7450980392156863,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":6,"ty":4,"nm":"01","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,125.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Gradient Ramp","np":10,"mn":"ADBE Ramp","ix":1,"en":1,"ef":[{"ty":3,"nm":"Start of Ramp","mn":"ADBE Ramp-0001","ix":1,"v":{"a":0,"k":[-537.683,1418.45],"ix":1}},{"ty":2,"nm":"Start Color","mn":"ADBE Ramp-0002","ix":2,"v":{"a":0,"k":[0.20000000298,0.490196079016,0.941176474094,1],"ix":2,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0002');"}},{"ty":3,"nm":"End of Ramp","mn":"ADBE Ramp-0003","ix":3,"v":{"a":0,"k":[2050.217,-85.617],"ix":3}},{"ty":2,"nm":"End Color","mn":"ADBE Ramp-0004","ix":4,"v":{"a":0,"k":[0.098039217293,0.901960790157,0.847058832645,1],"ix":4,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0003');"}},{"ty":7,"nm":"Ramp Shape","mn":"ADBE Ramp-0005","ix":5,"v":{"a":0,"k":1,"ix":5}},{"ty":0,"nm":"Ramp Scatter","mn":"ADBE Ramp-0006","ix":6,"v":{"a":0,"k":6,"ix":6}},{"ty":0,"nm":"Blend With Original","mn":"ADBE Ramp-0007","ix":7,"v":{"a":0,"k":0,"ix":7}},{"ty":6,"nm":"","mn":"ADBE Ramp-0008","ix":8,"v":0}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[1343.751,1842.45],[48.251,2589.95],[48,1100],[1343.5,352.5]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.40784313725490196,0.7607843137254902,0.9215686274509803,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0}]},{"id":"comp_1","layers":[{"ddd":0,"ind":1,"ty":4,"nm":"08","td":1,"sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":18,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[96,195.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Linear Wipe","np":5,"mn":"ADBE Linear Wipe","ix":1,"en":1,"ef":[{"ty":0,"nm":"Transition Completion","mn":"ADBE Linear Wipe-0001","ix":1,"v":{"a":0,"k":57,"ix":1}},{"ty":0,"nm":"Wipe Angle","mn":"ADBE Linear Wipe-0002","ix":2,"v":{"a":0,"k":320,"ix":2}},{"ty":0,"nm":"Feather","mn":"ADBE Linear Wipe-0003","ix":3,"v":{"a":0,"k":923,"ix":3}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0],[0,0]],"v":[[4771.751,5338.449],[48.251,2589.95],[48,1100],[1343.5,352.5],[7411.557,3834]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":5,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":2,"ty":0,"nm":"Part","tt":1,"refId":"comp_2","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":121,"ix":10},"p":{"a":0,"k":[136,384.8,0],"ix":2},"a":{"a":0,"k":[300,1080,0],"ix":1},"s":{"a":0,"k":[252,252,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[1,0,0,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0001');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]}],"w":600,"h":1080,"ip":0,"op":600,"st":0,"bm":16},{"ddd":0,"ind":3,"ty":4,"nm":"07","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":18,"s":[35]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[96,195.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[1,0,0,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0001');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]},{"ty":5,"nm":"Linear Wipe","np":5,"mn":"ADBE Linear Wipe","ix":2,"en":1,"ef":[{"ty":0,"nm":"Transition Completion","mn":"ADBE Linear Wipe-0001","ix":1,"v":{"a":0,"k":57,"ix":1}},{"ty":0,"nm":"Wipe Angle","mn":"ADBE Linear Wipe-0002","ix":2,"v":{"a":0,"k":320,"ix":2}},{"ty":0,"nm":"Feather","mn":"ADBE Linear Wipe-0003","ix":3,"v":{"a":0,"k":923,"ix":3}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0],[0,0]],"v":[[4771.751,5338.449],[48.251,2589.95],[48,1100],[1343.5,352.5],[7411.557,3834]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":5,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":4,"ty":4,"nm":"01","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[96,195.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[1,0,0,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0001');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]},{"ty":5,"nm":"Fast Blur (Legacy)","np":5,"mn":"ADBE Fast Blur","ix":2,"en":1,"ef":[{"ty":0,"nm":"Blurriness","mn":"ADBE Fast Blur-0001","ix":1,"v":{"a":0,"k":43,"ix":1}},{"ty":7,"nm":"Blur Dimensions","mn":"ADBE Fast Blur-0002","ix":2,"v":{"a":0,"k":1,"ix":2}},{"ty":7,"nm":"Repeat Edge Pixels","mn":"ADBE Fast Blur-0003","ix":3,"v":{"a":0,"k":0,"ix":3}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[1343.751,1842.45],[48.251,2589.95],[48,1100],[1343.5,352.5]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.40784313725490196,0.7607843137254902,0.9215686274509803,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0}]},{"id":"comp_2","layers":[{"ddd":0,"ind":1,"ty":9,"nm":"Part.mov","cl":"mov","td":1,"sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[300,540,0],"ix":2},"a":{"a":0,"k":[300,540,0],"ix":1},"s":{"a":0,"k":[100,100,100],"ix":6}},"ao":0,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":2,"ty":4,"nm":"Shape Layer 1","tt":3,"sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[300,540,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[100,100,100],"ix":6}},"ao":0,"shapes":[{"ty":"gr","it":[{"ty":"rc","d":1,"s":{"a":0,"k":[600,1080],"ix":2},"p":{"a":0,"k":[0,0],"ix":3},"r":{"a":0,"k":0,"ix":4},"nm":"Rectangle Path 1","mn":"ADBE Vector Shape - Rect","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":39,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Rectangle 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0}]},{"id":"comp_3","layers":[{"ddd":0,"ind":1,"ty":4,"nm":"04","td":1,"sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":18,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,725.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Linear Wipe","np":5,"mn":"ADBE Linear Wipe","ix":1,"en":1,"ef":[{"ty":0,"nm":"Transition Completion","mn":"ADBE Linear Wipe-0001","ix":1,"v":{"a":0,"k":47,"ix":1}},{"ty":0,"nm":"Wipe Angle","mn":"ADBE Linear Wipe-0002","ix":2,"v":{"a":0,"k":180,"ix":2}},{"ty":0,"nm":"Feather","mn":"ADBE Linear Wipe-0003","ix":3,"v":{"a":0,"k":923,"ix":3}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0],[0,0]],"v":[[-1233.5,-2313.5],[-1240,4197],[48,4940],[1343.5,4192.5],[1344.775,-2312.003]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":2,"ty":0,"nm":"Part","tt":1,"refId":"comp_2","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[392,600,0],"ix":2},"a":{"a":0,"k":[300,540,0],"ix":1},"s":{"a":0,"k":[252,252,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[1,0,0,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0001');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]}],"w":600,"h":1080,"ip":0,"op":600,"st":0,"bm":16},{"ddd":0,"ind":3,"ty":4,"nm":"03","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":18,"s":[35]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,725.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Linear Wipe","np":5,"mn":"ADBE Linear Wipe","ix":1,"en":1,"ef":[{"ty":0,"nm":"Transition Completion","mn":"ADBE Linear Wipe-0001","ix":1,"v":{"a":0,"k":47,"ix":1}},{"ty":0,"nm":"Wipe Angle","mn":"ADBE Linear Wipe-0002","ix":2,"v":{"a":0,"k":180,"ix":2}},{"ty":0,"nm":"Feather","mn":"ADBE Linear Wipe-0003","ix":3,"v":{"a":0,"k":923,"ix":3}}]},{"ty":21,"nm":"Fill","np":9,"mn":"ADBE Fill","ix":2,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[1,0,0,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0001');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0],[0,0]],"v":[[-1233.5,-2313.5],[-1240,4197],[48,4940],[1343.5,4192.5],[1344.775,-2312.003]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":4,"ty":4,"nm":"05","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[386,1685.5,0],"ix":2},"a":{"a":0,"k":[0,0,0],"ix":1},"s":{"a":0,"k":[25,25,100],"ix":6}},"ao":0,"ef":[{"ty":21,"nm":"Fill","np":9,"mn":"ADBE Fill","ix":1,"en":1,"ef":[{"ty":10,"nm":"Fill Mask","mn":"ADBE Fill-0001","ix":1,"v":{"a":0,"k":0,"ix":1}},{"ty":7,"nm":"All Masks","mn":"ADBE Fill-0007","ix":2,"v":{"a":0,"k":0,"ix":2}},{"ty":2,"nm":"Color","mn":"ADBE Fill-0002","ix":3,"v":{"a":0,"k":[1,0,0,1],"ix":3,"x":"var $bm_rt;\n$bm_rt = comp('TechnoCub').layer('Customize').effect('Customize 2')('Pseudo/rve2e2f0a9gMD-0001');"}},{"ty":7,"nm":"Invert","mn":"ADBE Fill-0006","ix":4,"v":{"a":0,"k":0,"ix":4}},{"ty":0,"nm":"Horizontal Feather","mn":"ADBE Fill-0003","ix":5,"v":{"a":0,"k":0,"ix":5}},{"ty":0,"nm":"Vertical Feather","mn":"ADBE Fill-0004","ix":6,"v":{"a":0,"k":0,"ix":6}},{"ty":0,"nm":"Opacity","mn":"ADBE Fill-0005","ix":7,"v":{"a":0,"k":1,"ix":7}}]},{"ty":5,"nm":"Fast Blur (Legacy)","np":5,"mn":"ADBE Fast Blur","ix":2,"en":1,"ef":[{"ty":0,"nm":"Blurriness","mn":"ADBE Fast Blur-0001","ix":1,"v":{"a":0,"k":43,"ix":1}},{"ty":7,"nm":"Blur Dimensions","mn":"ADBE Fast Blur-0002","ix":2,"v":{"a":0,"k":1,"ix":2}},{"ty":7,"nm":"Repeat Edge Pixels","mn":"ADBE Fast Blur-0003","ix":3,"v":{"a":0,"k":0,"ix":3}}]}],"shapes":[{"ty":"gr","it":[{"ind":0,"ty":"sh","ix":1,"ks":{"a":0,"k":{"i":[[0,0],[0,0],[0,0],[0,0]],"o":[[0,0],[0,0],[0,0],[0,0]],"v":[[48.5,-391.5],[-1240,357],[48,1100],[1343.5,352.5]],"c":true},"ix":2},"nm":"Path 1","mn":"ADBE Vector Shape - Group","hd":false},{"ty":"st","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":3},"o":{"a":0,"k":100,"ix":4},"w":{"a":0,"k":3,"ix":5},"lc":1,"lj":1,"ml":4,"bm":0,"nm":"Stroke 1","mn":"ADBE Vector Graphic - Stroke","hd":true},{"ty":"fl","c":{"a":0,"k":[0.7215686274509804,0.8823529411764706,0.9529411764705882,1],"ix":4},"o":{"a":0,"k":100,"ix":5},"r":1,"bm":0,"nm":"Fill 1","mn":"ADBE Vector Graphic - Fill","hd":false},{"ty":"tr","p":{"a":0,"k":[0,0],"ix":2},"a":{"a":0,"k":[0,0],"ix":1},"s":{"a":0,"k":[100,100],"ix":3},"r":{"a":0,"k":0,"ix":6},"o":{"a":0,"k":100,"ix":7},"sk":{"a":0,"k":0,"ix":4},"sa":{"a":0,"k":0,"ix":5},"nm":"Transform"}],"nm":"Shape 1","np":3,"cix":2,"bm":0,"ix":1,"mn":"ADBE Vector Group","hd":false}],"ip":0,"op":600,"st":0,"bm":0}]}],"layers":[{"ddd":0,"ind":1,"ty":12,"nm":"Customize","hd":true,"sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[-320,250,0],"ix":2},"a":{"a":0,"k":[1250,800,0],"ix":1},"s":{"a":0,"k":[80,100,100],"ix":6}},"ao":0,"ef":[{"ty":5,"nm":"Customize 2","np":5,"mn":"Pseudo/rve2e2f0a9gMD","ix":1,"en":1,"ef":[{"ty":2,"nm":"Light_Color","mn":"Pseudo/rve2e2f0a9gMD-0001","ix":1,"v":{"a":0,"k":[1,1,1,1],"ix":1}},{"ty":2,"nm":"01_Elem_Color_01","mn":"Pseudo/rve2e2f0a9gMD-0002","ix":2,"v":{"a":0,"k":[1,0.258823543787,0.996078431606,1],"ix":2}},{"ty":2,"nm":"01_Elem_Color_02","mn":"Pseudo/rve2e2f0a9gMD-0003","ix":3,"v":{"a":0,"k":[0.286274522543,0.06274510175,0.603921592236,1],"ix":3}}]}],"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":2,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1212.566,1139.024,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":3,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1102.533,1075.568,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":4,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.18,1011.685,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":5,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[882.146,948.229,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":6,"ty":0,"nm":"Lig_02","refId":"comp_1","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":41,"s":[0]},{"t":45,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[577.152,767.262,0],"ix":2},"a":{"a":0,"k":[1000,1000,0],"ix":1},"s":{"a":0,"k":[-34.185,-34.185,100],"ix":6}},"ao":0,"w":2000,"h":2000,"ip":41,"op":600,"st":41,"bm":16},{"ddd":0,"ind":7,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[771.579,884.559,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[598.518,784.141,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.25,"y":1},"o":{"x":0.5,"y":0},"t":361,"s":[487.416,719.189,0],"to":[0,0,0],"ti":[0,0,0]},{"t":599,"s":[598.518,784.141,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":8,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1323.026,1074.714,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":9,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1212.993,1011.258,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":10,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1102.64,947.374,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":11,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.607,883.919,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":12,"ty":0,"nm":"Lig_02","refId":"comp_1","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":46,"s":[0]},{"t":50,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[689.108,704.02,0],"ix":2},"a":{"a":0,"k":[1000,1000,0],"ix":1},"s":{"a":0,"k":[-34.185,-34.185,100],"ix":6}},"ao":0,"w":2000,"h":2000,"ip":46,"op":600,"st":46,"bm":16},{"ddd":0,"ind":13,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[882.04,820.249,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[829.48,789.482,0],"to":[0,0,0],"ti":[0,0,0]},{"t":597,"s":[631.207,675.39,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":14,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1433.914,1010.403,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":15,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1323.881,946.947,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":16,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1213.528,883.064,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[1213.528,336.104,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":299,"s":[1213.528,404.474,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":486,"s":[1213.528,370.289,0],"to":[0,0,0],"ti":[0,0,0]},{"t":599,"s":[1213.528,438.659,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":17,"ty":0,"nm":"Lig_01","refId":"comp_3","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1215.13,563.434,0],"ix":2},"a":{"a":0,"k":[400,1000,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":2000,"ip":55,"op":600,"st":55,"bm":16},{"ddd":0,"ind":18,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1103.494,819.608,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":19,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.927,755.938,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":20,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1544.374,946.093,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":21,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1434.341,882.637,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":22,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1323.988,818.753,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":23,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1213.955,755.297,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":24,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1103.388,691.628,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":25,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1655.262,881.568,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[1655.262,702.097,0],"to":[0,0,0],"ti":[0,0,0]},{"t":598,"s":[1655.262,633.727,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":26,"ty":0,"nm":"Lig_01","refId":"comp_3","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":40,"s":[0]},{"t":48,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1654.835,681.372,0],"ix":2},"a":{"a":0,"k":[400,1000,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":2000,"ip":40,"op":600,"st":40,"bm":16},{"ddd":0,"ind":27,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1545.229,818.112,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":28,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1434.876,754.229,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":29,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1324.843,690.773,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":30,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1214.275,627.104,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":31,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1212.566,1266.791,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":32,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1102.533,1203.335,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":33,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.18,1139.452,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":34,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[882.146,1075.996,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[645.415,1212.736,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.167,"y":0},"t":406,"s":[538.587,1275.978,0],"to":[0,0,0],"ti":[0,0,0]},{"t":599,"s":[567.644,1258.885,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":35,"ty":0,"nm":"Lig_02","refId":"comp_1","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[583.989,1253.117,0],"ix":2},"a":{"a":0,"k":[1000,1000,0],"ix":1},"s":{"a":0,"k":[-34.185,34.185,100],"ix":6}},"ao":0,"w":2000,"h":2000,"ip":63,"op":600,"st":63,"bm":16},{"ddd":0,"ind":36,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[771.579,1012.326,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":37,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1323.026,1202.48,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[1598.216,1359.731,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":295,"s":[1555.485,1334.947,0],"to":[0,0,0],"ti":[0,0,0]},{"t":599,"s":[1598.216,1359.731,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":38,"ty":0,"nm":"Lig_02","refId":"comp_1","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1620.222,1377.465,0],"ix":2},"a":{"a":0,"k":[1000,1000,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":2000,"h":2000,"ip":61,"op":600,"st":61,"bm":16},{"ddd":0,"ind":39,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1433.914,1138.17,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":40,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1544.374,1073.859,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":41,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1655.262,1009.335,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":42,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1212.566,1394.13,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":43,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1102.533,1330.674,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":44,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.18,1266.791,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":45,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[882.146,1203.335,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":46,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[771.579,1139.665,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":47,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1323.026,1329.819,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":48,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1433.914,1265.509,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":49,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1544.374,1201.198,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[1858.877,1382.379,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":373,"s":[1939.211,1427.674,0],"to":[0,0,0],"ti":[0,0,0]},{"t":598,"s":[1893.916,1401.181,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":50,"ty":0,"nm":"Lig_02","refId":"comp_1","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1843.28,1379.601,0],"ix":2},"a":{"a":0,"k":[1000,1000,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":2000,"h":2000,"ip":60,"op":600,"st":60,"bm":16},{"ddd":0,"ind":51,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1655.262,1136.674,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":52,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1212.566,1521.896,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":53,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1102.533,1458.441,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[812.815,1625.093,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":280,"s":[879.903,1586.421,0],"to":[0,0,0],"ti":[0,0,0]},{"t":599,"s":[812.815,1625.093,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":54,"ty":0,"nm":"Lig_02","refId":"comp_1","sr":1,"ks":{"o":{"a":0,"k":100,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[808.756,1632.998,0],"ix":2},"a":{"a":0,"k":[1000,1000,0],"ix":1},"s":{"a":0,"k":[-34.185,34.185,100],"ix":6}},"ao":0,"w":2000,"h":2000,"ip":61,"op":600,"st":61,"bm":16},{"ddd":0,"ind":55,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.18,1394.557,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":56,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[882.146,1331.101,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":57,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[771.579,1267.432,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":58,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1323.026,1457.586,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":59,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1433.914,1393.275,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":60,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1544.374,1328.965,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":61,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1655.262,1264.441,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":62,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1212.566,1649.663,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":63,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1102.533,1586.207,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":64,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[992.18,1522.324,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":65,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[882.146,1458.868,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":66,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[771.579,1395.198,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":67,"ty":0,"nm":"Lig_01","refId":"comp_3","sr":1,"ks":{"o":{"a":0,"k":80,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1323.026,1778.711,0],"ix":2},"a":{"a":0,"k":[400,1000,0],"ix":1},"s":{"a":0,"k":[34.185,-34.185,100],"ix":6}},"ao":0,"w":800,"h":2000,"ip":39,"op":600,"st":39,"bm":16},{"ddd":0,"ind":68,"ty":0,"nm":"Lig_01","refId":"comp_3","sr":1,"ks":{"o":{"a":0,"k":80,"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1433.487,1716.811,0],"ix":2},"a":{"a":0,"k":[400,1000,0],"ix":1},"s":{"a":0,"k":[34.185,-34.185,100],"ix":6}},"ao":0,"w":800,"h":2000,"ip":39,"op":600,"st":39,"bm":16},{"ddd":0,"ind":69,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1323.026,1585.352,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[1323.026,1844.304,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":305,"s":[1323.026,1758.841,0],"to":[0,0,0],"ti":[0,0,0]},{"t":598,"s":[1323.026,1857.978,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":70,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":1,"k":[{"i":{"x":0.25,"y":1},"o":{"x":0.333,"y":0},"t":35,"s":[1433.914,1521.042,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":122,"s":[1433.914,1621.033,0],"to":[0,0,0],"ti":[0,0,0]},{"i":{"x":0.5,"y":1},"o":{"x":0.5,"y":0},"t":351,"s":[1433.914,1791.958,0],"to":[0,0,0],"ti":[0,0,0]},{"t":599,"s":[1433.914,1621.033,0]}],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":71,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1544.374,1456.731,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0},{"ddd":0,"ind":72,"ty":0,"nm":"Cbe","refId":"comp_0","sr":1,"ks":{"o":{"a":1,"k":[{"i":{"x":[0.667],"y":[1]},"o":{"x":[0.333],"y":[0]},"t":0,"s":[0]},{"t":32,"s":[100]}],"ix":11},"r":{"a":0,"k":0,"ix":10},"p":{"a":0,"k":[1655.262,1392.207,0],"ix":2},"a":{"a":0,"k":[400,400,0],"ix":1},"s":{"a":0,"k":[34.185,34.185,100],"ix":6}},"ao":0,"w":800,"h":800,"ip":0,"op":600,"st":0,"bm":0}],"markers":[]} \ No newline at end of file diff --git a/site/overrides/home.html b/site/overrides/home.html index 5ab9907f70b3..a79c44c746c2 100644 --- a/site/overrides/home.html +++ b/site/overrides/home.html @@ -216,7 +216,7 @@

Hidden Partitioning

- Date: Thu, 8 Feb 2024 19:28:44 +0100 Subject: [PATCH 0030/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.14.4 to 3.14.5 (#9570) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.14.4 to 3.14.5. - [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.14.4...v3.14.5) --- 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 4b805159e9bf..4b88d63efeb4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -78,7 +78,7 @@ roaringbitmap = "1.0.1" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.14.4" +snowflake-jdbc = "3.14.5" spark-hive33 = "3.3.4" spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" From ee92a71d56e144793795e91de7e41eed8d2386ed Mon Sep 17 00:00:00 2001 From: Marc Cenac <547446+mrcnc@users.noreply.github.com> Date: Fri, 9 Feb 2024 02:33:12 -0600 Subject: [PATCH 0031/1019] OpenAPI: Spec updates for statistics (#9690) --- open-api/rest-catalog-open-api.py | 24 +++++++++++++ open-api/rest-catalog-open-api.yaml | 54 +++++++++++++++++++++++++++++ 2 files changed, 78 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index c8889271b203..c193b53e9452 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -347,6 +347,11 @@ class RemoveStatisticsUpdate(BaseUpdate): snapshot_id: int = Field(..., alias='snapshot-id') +class RemovePartitionStatisticsUpdate(BaseUpdate): + action: Literal['remove-partition-statistics'] + snapshot_id: int = Field(..., alias='snapshot-id') + + class TableRequirement(BaseModel): type: str @@ -618,6 +623,12 @@ class BlobMetadata(BaseModel): properties: Optional[Dict[str, Any]] = None +class PartitionStatisticsFile(BaseModel): + snapshot_id: int = Field(..., alias='snapshot-id') + statistics_path: str = Field(..., alias='statistics-path') + file_size_in_bytes: int = Field(..., alias='file-size-in-bytes') + + class CreateNamespaceRequest(BaseModel): namespace: Namespace properties: Optional[Dict[str, str]] = Field( @@ -638,6 +649,13 @@ class TransformTerm(BaseModel): term: Reference +class SetPartitionStatisticsUpdate(BaseUpdate): + action: Literal['set-partition-statistics'] + partition_statistics: PartitionStatisticsFile = Field( + ..., alias='partition-statistics' + ) + + class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -754,6 +772,12 @@ class TableMetadata(BaseModel): last_sequence_number: Optional[int] = Field(None, alias='last-sequence-number') snapshot_log: Optional[SnapshotLog] = Field(None, alias='snapshot-log') metadata_log: Optional[MetadataLog] = Field(None, alias='metadata-log') + statistics_files: Optional[List[StatisticsFile]] = Field( + None, alias='statistics-files' + ) + partition_statistics_files: Optional[List[PartitionStatisticsFile]] = Field( + None, alias='partition-statistics-files' + ) class ViewMetadata(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 1048425147e9..e1d3c7df2264 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2058,6 +2058,15 @@ components: $ref: '#/components/schemas/SnapshotLog' metadata-log: $ref: '#/components/schemas/MetadataLog' + # statistics + statistics-files: + type: array + items: + $ref: '#/components/schemas/StatisticsFile' + partition-statistics-files: + type: array + items: + $ref: '#/components/schemas/PartitionStatisticsFile' SQLViewRepresentation: type: object @@ -2181,6 +2190,8 @@ components: set-current-view-version: '#/components/schemas/SetCurrentViewVersionUpdate' set-statistics: '#/components/schemas/SetStatisticsUpdate' remove-statistics: '#/components/schemas/RemoveStatisticsUpdate' + set-partition-statistics: '#/components/schemas/SetPartitionStatisticsUpdate' + remove-partition-statistics: '#/components/schemas/RemovePartitionStatisticsUpdate' type: object required: - action @@ -2456,6 +2467,33 @@ components: type: integer format: int64 + SetPartitionStatisticsUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + required: + - action + - partition-statistics + properties: + action: + type: string + enum: [ "set-partition-statistics" ] + partition-statistics: + $ref: '#/components/schemas/PartitionStatisticsFile' + + RemovePartitionStatisticsUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + required: + - action + - snapshot-id + properties: + action: + type: string + enum: [ "remove-partition-statistics" ] + snapshot-id: + type: integer + format: int64 + TableUpdate: anyOf: - $ref: '#/components/schemas/AssignUUIDUpdate' @@ -3270,6 +3308,22 @@ components: properties: type: object + PartitionStatisticsFile: + type: object + required: + - snapshot-id + - statistics-path + - file-size-in-bytes + properties: + snapshot-id: + type: integer + format: int64 + statistics-path: + type: string + file-size-in-bytes: + type: integer + format: int64 + ############################# # Reusable Response Objects # From f39ce18dd79eee9638aa49dfd652c7a8183fa865 Mon Sep 17 00:00:00 2001 From: Muna Bedan <45054928+munabedan@users.noreply.github.com> Date: Sun, 11 Feb 2024 22:35:12 +0300 Subject: [PATCH 0032/1019] Docs: Fix broken strike-through markup (#9696) --- site/mkdocs.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 2cd19c72be19..9f75bad96d67 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -68,6 +68,7 @@ markdown_extensions: - pymdownx.tabbed: alternate_style: true - pymdownx.mark + - pymdownx.tilde - attr_list - pymdownx.emoji: emoji_index: !!python/name:material.extensions.emoji.twemoji From c3859a4ef0ec0c58c101b93cca2c475be3e69b44 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Feb 2024 20:52:49 +0100 Subject: [PATCH 0033/1019] Build: Bump mkdocs-material from 9.5.7 to 9.5.9 (#9708) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.7 to 9.5.9. - [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.7...9.5.9) --- 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 15c2958f4823..64fa41cab2af 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.7 +mkdocs-material==9.5.9 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 0ff08daf34314b50ae73f280e5ad97044a0d1bd9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Feb 2024 20:53:21 +0100 Subject: [PATCH 0034/1019] Build: Bump org.testcontainers:testcontainers from 1.19.4 to 1.19.5 (#9704) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.4 to 1.19.5. - [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.19.4...1.19.5) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 4b88d63efeb4..f6bce201ca2a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -85,7 +85,7 @@ spark-hive35 = "3.5.0" spring-boot = "2.5.4" spring-web = "5.3.30" sqlite-jdbc = "3.45.1.0" -testcontainers = "1.19.4" +testcontainers = "1.19.5" tez010 = "0.10.2" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From e8133ecf02f47bad7022c1126f0211bf29034ccc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 11 Feb 2024 20:55:48 +0100 Subject: [PATCH 0035/1019] Build: Bump io.airlift:aircompressor from 0.25 to 0.26 (#9700) Bumps [io.airlift:aircompressor](https://github.com/airlift/aircompressor) from 0.25 to 0.26. - [Commits](https://github.com/airlift/aircompressor/compare/0.25...0.26) --- updated-dependencies: - dependency-name: io.airlift:aircompressor 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 f6bce201ca2a..82bda5d9f09f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -23,7 +23,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" -aircompressor = "0.25" +aircompressor = "0.26" arrow = "14.0.2" avro = "1.11.3" assertj-core = "3.25.2" From 53f704bb54929a3312176469aa6d5e7ac91dd706 Mon Sep 17 00:00:00 2001 From: DongDongLee Date: Tue, 13 Feb 2024 19:20:19 +0800 Subject: [PATCH 0036/1019] MR: Migrate parameterized tests to JUni5 (#9711) --- .../iceberg/mr/TestIcebergInputFormats.java | 96 +++--- .../HiveIcebergStorageHandlerTestUtils.java | 17 +- ...estHiveIcebergStorageHandlerLocalScan.java | 220 ++++++------- .../TestHiveIcebergStorageHandlerNoScan.java | 293 +++++++++--------- ...TestHiveIcebergStorageHandlerTimezone.java | 73 +++-- ...stHiveIcebergStorageHandlerWithEngine.java | 282 +++++++++-------- ...ergStorageHandlerWithMultipleCatalogs.java | 71 +++-- .../apache/iceberg/mr/hive/TestTables.java | 67 ++-- 8 files changed, 549 insertions(+), 570 deletions(-) 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 8d0aad44cfc0..c9d5d487de38 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java @@ -19,10 +19,14 @@ package org.apache.iceberg.mr; 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; import java.io.UncheckedIOException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.util.List; import java.util.Map; import java.util.Set; @@ -40,6 +44,9 @@ import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DataFile; 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; @@ -60,16 +67,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergInputFormats { public static final List> TESTED_INPUT_FORMATS = @@ -90,7 +93,7 @@ public class TestIcebergInputFormats { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("date").bucket("id", 1).build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; // before variables private Configuration conf; @@ -98,23 +101,26 @@ public class TestIcebergInputFormats { private InputFormatConfig.ConfigBuilder builder; // parametrized variables - private final TestInputFormat.Factory testInputFormat; - private final FileFormat fileFormat; + @Parameter(index = 0) + private TestInputFormat.Factory testInputFormat; + + @Parameter(index = 1) + private FileFormat fileFormat; - @Before + @BeforeEach public void before() throws IOException { conf = new Configuration(); conf.set(CatalogUtil.ICEBERG_CATALOG_TYPE, Catalogs.LOCATION); HadoopTables tables = new HadoopTables(conf); - File location = temp.newFolder(testInputFormat.name(), fileFormat.name()); - Assert.assertTrue(location.delete()); + File location = temp.resolve(Paths.get(testInputFormat.name(), fileFormat.name())).toFile(); + assertThat(location).doesNotExist(); helper = new TestHelper(conf, tables, location.toString(), SCHEMA, SPEC, fileFormat, temp); builder = new InputFormatConfig.ConfigBuilder(conf).readFrom(location.toString()); } - @Parameterized.Parameters(name = "testInputFormat = {0}, fileFormat = {1}") + @Parameters(name = "testInputFormat = {0}, fileFormat = {1}") public static Object[][] parameters() { Object[][] parameters = new Object[TESTED_INPUT_FORMATS.size() * TESTED_FILE_FORMATS.size()][2]; @@ -122,20 +128,14 @@ public static Object[][] parameters() { for (TestInputFormat.Factory inputFormat : TESTED_INPUT_FORMATS) { for (String fileFormat : TESTED_FILE_FORMATS) { - parameters[idx++] = new Object[] {inputFormat, fileFormat}; + parameters[idx++] = new Object[] {inputFormat, FileFormat.fromString(fileFormat)}; } } return parameters; } - public TestIcebergInputFormats( - TestInputFormat.Factory testInputFormat, String fileFormat) { - this.testInputFormat = testInputFormat; - this.fileFormat = FileFormat.fromString(fileFormat); - } - - @Test + @TestTemplate public void testUnpartitionedTable() throws Exception { helper.createUnpartitionedTable(); List expectedRecords = helper.generateRandomRecords(1, 0L); @@ -144,7 +144,7 @@ public void testUnpartitionedTable() throws Exception { testInputFormat.create(builder.conf()).validate(expectedRecords); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { helper.createTable(); List expectedRecords = helper.generateRandomRecords(1, 0L); @@ -154,7 +154,7 @@ public void testPartitionedTable() throws Exception { testInputFormat.create(builder.conf()).validate(expectedRecords); } - @Test + @TestTemplate public void testFilterExp() throws Exception { helper.createTable(); @@ -171,7 +171,7 @@ public void testFilterExp() throws Exception { testInputFormat.create(builder.conf()).validate(expectedRecords); } - @Test + @TestTemplate public void testResiduals() throws Exception { helper.createTable(); @@ -198,7 +198,7 @@ public void testResiduals() throws Exception { testInputFormat.create(builder.conf()).validate(writeRecords); } - @Test + @TestTemplate public void testFailedResidualFiltering() throws Exception { helper.createTable(); @@ -213,20 +213,20 @@ public void testFailedResidualFiltering() throws Exception { .filter( Expressions.and(Expressions.equal("date", "2020-03-20"), Expressions.equal("id", 0))); - Assertions.assertThatThrownBy(() -> testInputFormat.create(builder.conf())) + assertThatThrownBy(() -> testInputFormat.create(builder.conf())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage( "Filter expression ref(name=\"id\") == 0 is not completely satisfied. Additional rows can be returned not satisfied by the filter expression"); builder.usePigTuples(); - Assertions.assertThatThrownBy(() -> testInputFormat.create(builder.conf())) + assertThatThrownBy(() -> testInputFormat.create(builder.conf())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage( "Filter expression ref(name=\"id\") == 0 is not completely satisfied. Additional rows can be returned not satisfied by the filter expression"); } - @Test + @TestTemplate public void testProjection() throws Exception { helper.createTable(); List inputRecords = helper.generateRandomRecords(1, 0L); @@ -237,8 +237,8 @@ public void testProjection() throws Exception { List outputRecords = testInputFormat.create(builder.conf()).getRecords(); - Assert.assertEquals(inputRecords.size(), outputRecords.size()); - Assert.assertEquals(projection.asStruct(), outputRecords.get(0).struct()); + assertThat(outputRecords).hasSameSizeAs(inputRecords); + assertThat(outputRecords.get(0).struct()).isEqualTo(projection.asStruct()); } private static final Schema LOG_SCHEMA = @@ -251,7 +251,7 @@ public void testProjection() throws Exception { private static final PartitionSpec IDENTITY_PARTITION_SPEC = PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build(); - @Test + @TestTemplate public void testIdentityPartitionProjections() throws Exception { helper.createTable(LOG_SCHEMA, IDENTITY_PARTITION_SPEC); List inputRecords = helper.generateRandomRecords(10, 0L); @@ -310,19 +310,19 @@ private void validateIdentityPartitionProjections( for (int pos = 0; pos < inputRecords.size(); pos++) { Record inputRecord = inputRecords.get(pos); Record actualRecord = actualRecords.get(pos); - Assert.assertEquals( - "Projected schema should match", projectedSchema.asStruct(), actualRecord.struct()); + assertThat(actualRecord.struct()) + .as("Projected schema should match") + .isEqualTo(projectedSchema.asStruct()); for (String name : fieldNames) { - Assert.assertEquals( - "Projected field " + name + " should match", - inputRecord.getField(name), - actualRecord.getField(name)); + assertThat(actualRecord.getField(name)) + .as("Projected field " + name + " should match") + .isEqualTo(inputRecord.getField(name)); } } } - @Test + @TestTemplate public void testSnapshotReads() throws Exception { helper.createUnpartitionedTable(); @@ -336,26 +336,26 @@ public void testSnapshotReads() throws Exception { testInputFormat.create(builder.conf()).validate(expectedRecords); } - @Test + @TestTemplate public void testLocality() throws Exception { helper.createUnpartitionedTable(); List expectedRecords = helper.generateRandomRecords(1, 0L); helper.appendToTable(null, expectedRecords); for (InputSplit split : testInputFormat.create(builder.conf()).getSplits()) { - Assert.assertArrayEquals(new String[] {"*"}, split.getLocations()); + assertThat(split.getLocations()).containsExactly("*"); } builder.preferLocality(); for (InputSplit split : testInputFormat.create(builder.conf()).getSplits()) { - Assert.assertArrayEquals(new String[] {"localhost"}, split.getLocations()); + assertThat(split.getLocations()).containsExactly("localhost"); } } - @Test + @TestTemplate public void testCustomCatalog() throws IOException { - String warehouseLocation = temp.newFolder("hadoop_catalog").getAbsolutePath(); + String warehouseLocation = temp.resolve("hadoop_catalog").toAbsolutePath().toString(); conf.set("warehouse.location", warehouseLocation); conf.set(InputFormatConfig.CATALOG_NAME, Catalogs.ICEBERG_DEFAULT_CATALOG_NAME); conf.set( @@ -402,7 +402,7 @@ public List getSplits() { } public void validate(List expected) { - Assert.assertEquals(expected, records); + assertThat(records).isEqualTo(expected); } public interface Factory { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java index f20b28a35fd2..72b5034051da 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergStorageHandlerTestUtils.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.iceberg.mr.TestHelper; import org.apache.iceberg.types.Types; import org.apache.orc.OrcConf; -import org.junit.rules.TemporaryFolder; public class HiveIcebergStorageHandlerTestUtils { static final FileFormat[] FILE_FORMATS = @@ -78,22 +78,17 @@ static TestHiveShell shell(Map configs) { } static TestTables testTables( - TestHiveShell shell, TestTables.TestTableType testTableType, TemporaryFolder temp) - throws IOException { + TestHiveShell shell, TestTables.TestTableType testTableType, Path temp) throws IOException { return testTables(shell, testTableType, temp, Catalogs.ICEBERG_DEFAULT_CATALOG_NAME); } static TestTables testTables( - TestHiveShell shell, - TestTables.TestTableType testTableType, - TemporaryFolder temp, - String catalogName) + TestHiveShell shell, TestTables.TestTableType testTableType, Path temp, String catalogName) throws IOException { return testTableType.instance(shell.metastore().hiveConf(), temp, catalogName); } - static void init( - TestHiveShell shell, TestTables testTables, TemporaryFolder temp, String engine) { + static void init(TestHiveShell shell, TestTables testTables, Path temp, String engine) { shell.openSession(); for (Map.Entry property : testTables.properties().entrySet()) { @@ -101,8 +96,8 @@ static void init( } shell.setHiveSessionValue("hive.execution.engine", engine); - shell.setHiveSessionValue("hive.jar.directory", temp.getRoot().getAbsolutePath()); - shell.setHiveSessionValue("tez.staging-dir", temp.getRoot().getAbsolutePath()); + shell.setHiveSessionValue("hive.jar.directory", temp.toAbsolutePath().toString()); + shell.setHiveSessionValue("tez.staging-dir", temp.toAbsolutePath().toString()); } static void close(TestHiveShell shell) throws Exception { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java index 5aeb825e7ba5..fd15a9a62621 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java @@ -19,16 +19,19 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.runners.Parameterized.Parameter; -import static org.junit.runners.Parameterized.Parameters; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.math.BigDecimal; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; import java.util.List; import java.util.Map; 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.PartitionSpecParser; import org.apache.iceberg.Schema; @@ -45,18 +48,15 @@ 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.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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; + +@ExtendWith(ParameterizedTestExtension.class) public class TestHiveIcebergStorageHandlerLocalScan { @Parameters(name = "fileFormat={0}, catalog={1}") @@ -83,25 +83,25 @@ public static Collection parameters() { private TestTables testTables; - @Parameter(0) - public FileFormat fileFormat; + @Parameter(index = 0) + private FileFormat fileFormat; - @Parameter(1) - public TestTables.TestTableType testTableType; + @Parameter(index = 1) + private TestTables.TestTableType testTableType; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @BeforeClass + @BeforeAll public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell(); } - @AfterClass + @AfterAll public static void afterClass() throws Exception { shell.stop(); } - @Before + @BeforeEach public void before() throws IOException { testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); // Uses spark as an engine so we can detect if we unintentionally try to use any execution @@ -109,21 +109,21 @@ public void before() throws IOException { HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "spark"); } - @After + @AfterEach public void after() throws Exception { HiveIcebergStorageHandlerTestUtils.close(shell); } - @Test + @TestTemplate public void testScanEmptyTable() throws IOException { Schema emptySchema = new Schema(required(1, "empty", Types.StringType.get())); testTables.createTable(shell, "empty", emptySchema, fileFormat, ImmutableList.of()); List rows = shell.executeStatement("SELECT * FROM default.empty"); - Assert.assertEquals(0, rows.size()); + assertThat(rows).isEmpty(); } - @Test + @TestTemplate public void testScanTable() throws IOException { testTables.createTable( shell, @@ -135,13 +135,14 @@ public void testScanTable() throws IOException { // Single fetch task: no MR job. List rows = shell.executeStatement("SELECT * FROM default.customers"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {0L, "Alice", "Brown"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Bob", "Green"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Trudy", "Pink"}, rows.get(2)); + assertThat(rows) + .containsExactly( + new Object[] {0L, "Alice", "Brown"}, + new Object[] {1L, "Bob", "Green"}, + new Object[] {2L, "Trudy", "Pink"}); } - @Test + @TestTemplate public void testScanTableCaseInsensitive() throws IOException { testTables.createTable( shell, @@ -152,22 +153,22 @@ public void testScanTableCaseInsensitive() throws IOException { List rows = shell.executeStatement("SELECT * FROM default.customers"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {0L, "Alice", "Brown"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Bob", "Green"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Trudy", "Pink"}, rows.get(2)); + assertThat(rows) + .containsExactly( + new Object[] {0L, "Alice", "Brown"}, + new Object[] {1L, "Bob", "Green"}, + new Object[] {2L, "Trudy", "Pink"}); rows = shell.executeStatement( "SELECT * FROM default.customers where CustomER_Id < 2 " + "and first_name in ('Alice', 'Bob')"); - Assert.assertEquals(2, rows.size()); - Assert.assertArrayEquals(new Object[] {0L, "Alice", "Brown"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Bob", "Green"}, rows.get(1)); + assertThat(rows) + .containsExactly(new Object[] {0L, "Alice", "Brown"}, new Object[] {1L, "Bob", "Green"}); } - @Test + @TestTemplate public void testDecimalTableWithPredicateLiterals() throws IOException { Schema schema = new Schema(required(1, "decimal_field", Types.DecimalType.of(7, 2))); List records = @@ -181,28 +182,23 @@ public void testDecimalTableWithPredicateLiterals() throws IOException { // Use integer literal in predicate List rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field >= 85"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {"85.00"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {"100.56"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(2)); + assertThat(rows) + .containsExactly(new Object[] {"85.00"}, new Object[] {"100.56"}, new Object[] {"100.57"}); // Use decimal literal in predicate with smaller scale than schema type definition rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 99.1"); - Assert.assertEquals(2, rows.size()); - Assert.assertArrayEquals(new Object[] {"100.56"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(1)); + assertThat(rows).containsExactly(new Object[] {"100.56"}, new Object[] {"100.57"}); // Use decimal literal in predicate with higher scale than schema type definition rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 100.565"); - Assert.assertEquals(1, rows.size()); - Assert.assertArrayEquals(new Object[] {"100.57"}, rows.get(0)); + assertThat(rows).containsExactly(new Object[] {"100.57"}); // Use decimal literal in predicate with the same scale as schema type definition rows = shell.executeStatement("SELECT * FROM default.dec_test where decimal_field > 640.34"); - Assert.assertEquals(0, rows.size()); + assertThat(rows).isEmpty(); } - @Test + @TestTemplate public void testColumnSelection() throws IOException { testTables.createTable( shell, @@ -214,37 +210,37 @@ public void testColumnSelection() throws IOException { List outOfOrderColumns = shell.executeStatement("SELECT first_name, customer_id, last_name FROM default.customers"); - Assert.assertEquals(3, outOfOrderColumns.size()); - Assert.assertArrayEquals(new Object[] {"Alice", 0L, "Brown"}, outOfOrderColumns.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", 1L, "Green"}, outOfOrderColumns.get(1)); - Assert.assertArrayEquals(new Object[] {"Trudy", 2L, "Pink"}, outOfOrderColumns.get(2)); + assertThat(outOfOrderColumns) + .containsExactly( + new Object[] {"Alice", 0L, "Brown"}, + new Object[] {"Bob", 1L, "Green"}, + new Object[] {"Trudy", 2L, "Pink"}); List allButFirstColumn = shell.executeStatement("SELECT first_name, last_name FROM default.customers"); - Assert.assertEquals(3, allButFirstColumn.size()); - Assert.assertArrayEquals(new Object[] {"Alice", "Brown"}, allButFirstColumn.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", "Green"}, allButFirstColumn.get(1)); - Assert.assertArrayEquals(new Object[] {"Trudy", "Pink"}, allButFirstColumn.get(2)); + assertThat(allButFirstColumn) + .containsExactly( + new Object[] {"Alice", "Brown"}, + new Object[] {"Bob", "Green"}, + new Object[] {"Trudy", "Pink"}); List allButMiddleColumn = shell.executeStatement("SELECT customer_id, last_name FROM default.customers"); - Assert.assertEquals(3, allButMiddleColumn.size()); - Assert.assertArrayEquals(new Object[] {0L, "Brown"}, allButMiddleColumn.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Green"}, allButMiddleColumn.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Pink"}, allButMiddleColumn.get(2)); + assertThat(allButMiddleColumn) + .containsExactly( + new Object[] {0L, "Brown"}, new Object[] {1L, "Green"}, new Object[] {2L, "Pink"}); List allButLastColumn = shell.executeStatement("SELECT customer_id, first_name FROM default.customers"); - Assert.assertEquals(3, allButLastColumn.size()); - Assert.assertArrayEquals(new Object[] {0L, "Alice"}, allButLastColumn.get(0)); - Assert.assertArrayEquals(new Object[] {1L, "Bob"}, allButLastColumn.get(1)); - Assert.assertArrayEquals(new Object[] {2L, "Trudy"}, allButLastColumn.get(2)); + assertThat(allButLastColumn) + .containsExactly( + new Object[] {0L, "Alice"}, new Object[] {1L, "Bob"}, new Object[] {2L, "Trudy"}); } - @Test + @TestTemplate public void selectSameColumnTwice() throws IOException { testTables.createTable( shell, @@ -256,13 +252,14 @@ public void selectSameColumnTwice() throws IOException { List columns = shell.executeStatement("SELECT first_name, first_name FROM default.customers"); - Assert.assertEquals(3, columns.size()); - Assert.assertArrayEquals(new Object[] {"Alice", "Alice"}, columns.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", "Bob"}, columns.get(1)); - Assert.assertArrayEquals(new Object[] {"Trudy", "Trudy"}, columns.get(2)); + assertThat(columns) + .containsExactly( + new Object[] {"Alice", "Alice"}, + new Object[] {"Bob", "Bob"}, + new Object[] {"Trudy", "Trudy"}); } - @Test + @TestTemplate public void testCreateTableWithColumnSpecification() throws IOException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); Map> data = Maps.newHashMapWithExpectedSize(1); @@ -283,7 +280,7 @@ public void testCreateTableWithColumnSpecification() throws IOException { data); } - @Test + @TestTemplate public void testCreateTableWithColumnSpecificationPartitioned() throws IOException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); PartitionSpec spec = @@ -313,7 +310,7 @@ public void testCreateTableWithColumnSpecificationPartitioned() throws IOExcepti identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, data); } - @Test + @TestTemplate public void testCreatePartitionedTableByProperty() throws IOException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); PartitionSpec spec = @@ -355,7 +352,7 @@ public void testCreatePartitionedTableByProperty() throws IOException { identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, data); } - @Test + @TestTemplate public void testCreateTableWithColumnSpecificationMultilevelPartitioned() throws IOException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); PartitionSpec spec = @@ -387,7 +384,7 @@ public void testCreateTableWithColumnSpecificationMultilevelPartitioned() throws identifier, createSql, HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, spec, data); } - @Test + @TestTemplate public void testArrayOfPrimitivesInTable() throws IOException { Schema schema = new Schema( @@ -404,12 +401,12 @@ public void testArrayOfPrimitivesInTable() throws IOException { String.format( "SELECT arrayofprimitives[%d] FROM default.arraytable " + "LIMIT 1 OFFSET %d", j, i)); - Assert.assertEquals(expectedList.get(j), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(expectedList.get(j)); } } } - @Test + @TestTemplate public void testArrayOfArraysInTable() throws IOException { Schema schema = new Schema( @@ -430,13 +427,13 @@ public void testArrayOfArraysInTable() throws IOException { String.format( "SELECT arrayofarrays[%d][%d] FROM default.arraytable " + "LIMIT 1 OFFSET %d", j, k, i)); - Assert.assertEquals(expectedInnerList.get(k).toString(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(expectedInnerList.get(k).toString()); } } } } - @Test + @TestTemplate public void testArrayOfMapsInTable() throws IOException { Schema schema = new Schema( @@ -460,13 +457,13 @@ public void testArrayOfMapsInTable() throws IOException { String.format( "SELECT arrayofmaps[%d][\"%s\"] FROM default.arraytable LIMIT 1 OFFSET %d", j, entry.getKey(), i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(entry.getValue()); } } } } - @Test + @TestTemplate public void testArrayOfStructsInTable() throws IOException { Schema schema = new Schema( @@ -493,14 +490,16 @@ public void testArrayOfStructsInTable() throws IOException { + "OFFSET %d", j, j, j, i)); GenericRecord genericRecord = (GenericRecord) expectedList.get(j); - Assert.assertEquals(genericRecord.getField("something"), queryResult.get(0)[0]); - Assert.assertEquals(genericRecord.getField("someone"), queryResult.get(0)[1]); - Assert.assertEquals(genericRecord.getField("somewhere"), queryResult.get(0)[2]); + assertThat(queryResult.get(0)) + .containsExactly( + genericRecord.getField("something"), + genericRecord.getField("someone"), + genericRecord.getField("somewhere")); } } } - @Test + @TestTemplate public void testMapOfPrimitivesInTable() throws IOException { Schema schema = new Schema( @@ -519,12 +518,12 @@ public void testMapOfPrimitivesInTable() throws IOException { String.format( "SELECT mapofprimitives[\"%s\"] " + "FROM default.maptable LIMIT 1 OFFSET %d", entry.getKey(), i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(entry.getValue()); } } } - @Test + @TestTemplate public void testMapOfArraysInTable() throws IOException { Schema schema = new Schema( @@ -549,13 +548,13 @@ public void testMapOfArraysInTable() throws IOException { String.format( "SELECT mapofarrays[\"%s\"]" + "[%d] FROM maptable LIMIT 1 OFFSET %d", entry.getKey(), j, i)); - Assert.assertEquals(expectedList.get(j).toString(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(expectedList.get(j).toString()); } } } } - @Test + @TestTemplate public void testMapOfMapsInTable() throws IOException { Schema schema = new Schema( @@ -581,13 +580,13 @@ public void testMapOfMapsInTable() throws IOException { String.format( "SELECT mapofmaps[\"%s\"]" + "[\"%s\"] FROM maptable LIMIT 1 OFFSET %d", entry.getKey(), innerEntry.getKey(), i)); - Assert.assertEquals(innerEntry.getValue(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(innerEntry.getValue()); } } } } - @Test + @TestTemplate public void testMapOfStructsInTable() throws IOException { Schema schema = new Schema( @@ -616,14 +615,16 @@ public void testMapOfStructsInTable() throws IOException { + "OFFSET %d", entry.getKey(), entry.getKey(), entry.getKey(), i)); GenericRecord genericRecord = (GenericRecord) entry.getValue(); - Assert.assertEquals(genericRecord.getField("something"), queryResult.get(0)[0]); - Assert.assertEquals(genericRecord.getField("someone"), queryResult.get(0)[1]); - Assert.assertEquals(genericRecord.getField("somewhere"), queryResult.get(0)[2]); + assertThat(queryResult.get(0)) + .containsExactly( + genericRecord.getField("something"), + genericRecord.getField("someone"), + genericRecord.getField("somewhere")); } } } - @Test + @TestTemplate public void testStructOfPrimitivesInTable() throws IOException { Schema schema = new Schema( @@ -643,12 +644,12 @@ public void testStructOfPrimitivesInTable() throws IOException { String.format( "SELECT structofprimitives.key, structofprimitives.value FROM default.structtable LIMIT 1 OFFSET %d", i)); - Assert.assertEquals(expectedStruct.getField("key"), queryResult.get(0)[0]); - Assert.assertEquals(expectedStruct.getField("value"), queryResult.get(0)[1]); + assertThat(queryResult.get(0)) + .containsExactly(expectedStruct.getField("key"), expectedStruct.getField("value")); } } - @Test + @TestTemplate public void testStructOfArraysInTable() throws IOException { Schema schema = new Schema( @@ -670,7 +671,7 @@ public void testStructOfArraysInTable() throws IOException { String.format( "SELECT structofarrays.names[%d] FROM default.structtable LIMIT 1 OFFSET %d", j, i)); - Assert.assertEquals(expectedList.get(j), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(expectedList.get(j)); } expectedList = (List) expectedStruct.getField("birthdays"); for (int j = 0; j < expectedList.size(); j++) { @@ -679,12 +680,12 @@ public void testStructOfArraysInTable() throws IOException { String.format( "SELECT structofarrays.birthdays[%d] FROM default.structtable LIMIT 1 OFFSET %d", j, i)); - Assert.assertEquals(expectedList.get(j).toString(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(expectedList.get(j).toString()); } } } - @Test + @TestTemplate public void testStructOfMapsInTable() throws IOException { Schema schema = new Schema( @@ -714,7 +715,7 @@ public void testStructOfMapsInTable() throws IOException { String.format( "SELECT structofmaps.map1[\"%s\"] from default.structtable LIMIT 1 OFFSET %d", entry.getKey(), i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(entry.getValue()); } expectedMap = (Map) expectedStruct.getField("map2"); for (Map.Entry entry : expectedMap.entrySet()) { @@ -723,12 +724,12 @@ public void testStructOfMapsInTable() throws IOException { String.format( "SELECT structofmaps.map2[\"%s\"] from default.structtable LIMIT 1 OFFSET %d", entry.getKey(), i)); - Assert.assertEquals(entry.getValue(), queryResult.get(0)[0]); + assertThat(queryResult.get(0)[0]).isEqualTo(entry.getValue()); } } } - @Test + @TestTemplate public void testStructOfStructsInTable() throws IOException { Schema schema = new Schema( @@ -754,8 +755,9 @@ public void testStructOfStructsInTable() throws IOException { "SELECT structofstructs.struct1.key, structofstructs.struct1.value FROM default.structtable " + "LIMIT 1 OFFSET %d", i)); - Assert.assertEquals(expectedInnerStruct.getField("key"), queryResult.get(0)[0]); - Assert.assertEquals(expectedInnerStruct.getField("value"), queryResult.get(0)[1]); + assertThat(queryResult.get(0)) + .containsExactly( + expectedInnerStruct.getField("key"), expectedInnerStruct.getField("value")); } } @@ -769,8 +771,8 @@ private void runCreateAndReadTest( shell.executeStatement(createSQL); org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(expectedSchema.asStruct(), icebergTable.schema().asStruct()); - Assert.assertEquals(expectedSpec, icebergTable.spec()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(expectedSchema.asStruct()); + assertThat(icebergTable.spec()).isEqualTo(expectedSpec); List expected = Lists.newArrayList(); for (StructLike partition : data.keySet()) { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 81e2ffcc84da..534cc7d7476c 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -20,8 +20,9 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.junit.runners.Parameterized.Parameter; -import static org.junit.runners.Parameterized.Parameters; +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.Collection; @@ -38,6 +39,9 @@ import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.BaseTable; 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.PartitionSpecParser; import org.apache.iceberg.Schema; @@ -61,20 +65,15 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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; + +@ExtendWith(ParameterizedTestExtension.class) public class TestHiveIcebergStorageHandlerNoScan { private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); @@ -140,22 +139,21 @@ public static Collection parameters() { private TestTables testTables; - @Parameter(0) - public TestTables.TestTableType testTableType; + @Parameter private TestTables.TestTableType testTableType; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private java.nio.file.Path temp; - @BeforeClass + @BeforeAll public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell(); } - @AfterClass + @AfterAll public static void afterClass() throws Exception { shell.stop(); } - @Before + @BeforeEach public void before() throws IOException { testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); // Uses spark as an engine so we can detect if we unintentionally try to use any execution @@ -163,12 +161,12 @@ public void before() throws IOException { HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "spark"); } - @After + @AfterEach public void after() throws Exception { HiveIcebergStorageHandlerTestUtils.close(shell); } - @Test + @TestTemplate public void testCreateDropTable() throws TException, IOException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -195,10 +193,9 @@ public void testCreateDropTable() throws TException, IOException, InterruptedExc // Check the Iceberg table data org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals( - HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.asStruct(), - icebergTable.schema().asStruct()); - Assert.assertEquals(PartitionSpec.unpartitioned(), icebergTable.spec()); + assertThat(icebergTable.schema().asStruct()) + .isEqualTo(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.asStruct()); + assertThat(icebergTable.spec()).isEqualTo(PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = shell.metastore().getTable("default", "customers"); @@ -210,7 +207,7 @@ public void testCreateDropTable() throws TException, IOException, InterruptedExc shell.executeStatement("DROP TABLE customers"); // Check if the table was really dropped even from the Catalog - Assertions.assertThatThrownBy(() -> testTables.loadTable(identifier)) + assertThatThrownBy(() -> testTables.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); } else { @@ -220,7 +217,7 @@ public void testCreateDropTable() throws TException, IOException, InterruptedExc shell.executeStatement("DROP TABLE customers"); // Check if we drop an exception when trying to load the table - Assertions.assertThatThrownBy(() -> testTables.loadTable(identifier)) + assertThatThrownBy(() -> testTables.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: default.customers"); // Check if the files are removed @@ -229,13 +226,13 @@ public void testCreateDropTable() throws TException, IOException, InterruptedExc // if table directory has been deleted, we're good. This is the expected behavior in Hive4. // if table directory exists, its contents should have been cleaned up, save for an empty // metadata dir (Hive3). - Assert.assertEquals(1, fs.listStatus(hmsTableLocation).length); - Assert.assertEquals(0, fs.listStatus(new Path(hmsTableLocation, "metadata")).length); + assertThat(fs.listStatus(hmsTableLocation)).hasSize(1); + assertThat(fs.listStatus(new Path(hmsTableLocation, "metadata"))).isEmpty(); } } } - @Test + @TestTemplate public void testCreateDropTableNonDefaultCatalog() throws TException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); String catalogName = "nondefaultcatalog"; @@ -257,18 +254,17 @@ public void testCreateDropTableNonDefaultCatalog() throws TException, Interrupte shell.executeStatement(createSql); Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals( - HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.asStruct(), - icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()) + .isEqualTo(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.asStruct()); shell.executeStatement("DROP TABLE default.customers"); // Check if the table was really dropped even from the Catalog - Assertions.assertThatThrownBy(() -> testTables.loadTable(identifier)) + assertThatThrownBy(() -> testTables.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); } - @Test + @TestTemplate public void testCreateTableWithoutSpec() { TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -288,10 +284,10 @@ public void testCreateTableWithoutSpec() { // Check the Iceberg table partition data org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(PartitionSpec.unpartitioned(), icebergTable.spec()); + assertThat(icebergTable.spec()).isEqualTo(PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testCreateTableWithUnpartitionedSpec() { TableIdentifier identifier = TableIdentifier.of("default", "customers"); // We need the location for HadoopTable based tests only @@ -317,10 +313,10 @@ public void testCreateTableWithUnpartitionedSpec() { // Check the Iceberg table partition data org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(SPEC, icebergTable.spec()); + assertThat(icebergTable.spec()).isEqualTo(SPEC); } - @Test + @TestTemplate public void testCreateTableWithFormatV2ThroughTableProperty() { TableIdentifier identifier = TableIdentifier.of("default", "customers"); // We need the location for HadoopTable based tests only @@ -351,13 +347,12 @@ public void testCreateTableWithFormatV2ThroughTableProperty() { // Check the Iceberg table partition data org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals( - "should create table using format v2", - 2, - ((BaseTable) icebergTable).operations().current().formatVersion()); + assertThat(((BaseTable) icebergTable).operations().current().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); } - @Test + @TestTemplate public void testDeleteBackingTable() throws TException, IOException, InterruptedException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -398,23 +393,23 @@ public void testDeleteBackingTable() throws TException, IOException, Interrupted shell.executeStatement("DROP TABLE customers"); // Check if we drop an exception when trying to drop the table - Assertions.assertThatThrownBy(() -> testTables.loadTable(identifier)) + assertThatThrownBy(() -> testTables.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: default.customers"); // Check if the files are kept FileSystem fs = Util.getFs(hmsTableLocation, shell.getHiveConf()); - Assert.assertEquals(1, fs.listStatus(hmsTableLocation).length); - Assert.assertEquals(1, fs.listStatus(new Path(hmsTableLocation, "metadata")).length); + assertThat(fs.listStatus(hmsTableLocation)).hasSize(1); + assertThat(fs.listStatus(new Path(hmsTableLocation, "metadata"))).hasSize(1); } } - @Test + @TestTemplate public void testDropTableWithCorruptedMetadata() throws TException, IOException, InterruptedException { - Assume.assumeTrue( - "Only HiveCatalog attempts to load the Iceberg table prior to dropping it.", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + assumeThat(testTableType) + .as("Only HiveCatalog attempts to load the Iceberg table prior to dropping it.") + .isEqualTo(TestTables.TestTableType.HIVE_CATALOG); // create test table TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -441,17 +436,17 @@ public void testDropTableWithCorruptedMetadata() // check if HMS table is nonetheless still droppable shell.executeStatement(String.format("DROP TABLE %s", identifier)); - Assertions.assertThatThrownBy(() -> testTables.loadTable(identifier)) + assertThatThrownBy(() -> testTables.loadTable(identifier)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: default.customers"); } - @Test + @TestTemplate public void testCreateTableError() { TableIdentifier identifier = TableIdentifier.of("default", "withShell2"); // Wrong schema - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> shell.executeStatement( "CREATE EXTERNAL TABLE withShell2 " @@ -470,7 +465,7 @@ public void testCreateTableError() { .hasMessageContaining("Unrecognized token 'WrongSchema'"); // Missing schema, we try to get the schema from the table and fail - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> shell.executeStatement( "CREATE EXTERNAL TABLE withShell2 " @@ -483,7 +478,7 @@ public void testCreateTableError() { if (!testTables.locationForCreateTableSQL(identifier).isEmpty()) { // Only test this if the location is required - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> shell.executeStatement( "CREATE EXTERNAL TABLE withShell2 " @@ -503,7 +498,7 @@ public void testCreateTableError() { } } - @Test + @TestTemplate public void testCreateTableAboveExistingTable() throws IOException { // Create the Iceberg table testTables.createIcebergTable( @@ -515,7 +510,7 @@ public void testCreateTableAboveExistingTable() throws IOException { if (testTableType == TestTables.TestTableType.HIVE_CATALOG) { // In HiveCatalog we just expect an exception since the table is already exists - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> shell.executeStatement( "CREATE EXTERNAL TABLE customers " @@ -542,14 +537,14 @@ public void testCreateTableAboveExistingTable() throws IOException { } } - @Test + @TestTemplate public void testCreatePartitionedTableWithPropertiesAndWithColumnSpecification() { PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) .identity("last_name") .build(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> shell.executeStatement( "CREATE EXTERNAL TABLE customers (customer_id BIGINT) " @@ -568,7 +563,7 @@ public void testCreatePartitionedTableWithPropertiesAndWithColumnSpecification() "Provide only one of the following: Hive partition specification, or the iceberg.mr.table.partition.spec property"); } - @Test + @TestTemplate public void testCreateTableWithColumnSpecificationHierarchy() { TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -586,10 +581,10 @@ public void testCreateTableWithColumnSpecificationHierarchy() { // Check the Iceberg table data org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(COMPLEX_SCHEMA.asStruct(), icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(COMPLEX_SCHEMA.asStruct()); } - @Test + @TestTemplate public void testCreateTableWithAllSupportedTypes() { TableIdentifier identifier = TableIdentifier.of("default", "all_types"); Schema allSupportedSchema = @@ -616,10 +611,10 @@ public void testCreateTableWithAllSupportedTypes() { // Check the Iceberg table data org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals(allSupportedSchema.asStruct(), icebergTable.schema().asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(allSupportedSchema.asStruct()); } - @Test + @TestTemplate public void testCreateTableWithNotSupportedTypes() { TableIdentifier identifier = TableIdentifier.of("default", "not_supported_types"); // Can not create INTERVAL types from normal create table, so leave them out from this test @@ -631,7 +626,7 @@ public void testCreateTableWithNotSupportedTypes() { "CHAR(1)", Types.StringType.get()); for (String notSupportedType : notSupportedTypes.keySet()) { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> shell.executeStatement( "CREATE EXTERNAL TABLE not_supported_types " @@ -647,7 +642,7 @@ public void testCreateTableWithNotSupportedTypes() { } } - @Test + @TestTemplate public void testCreateTableWithNotSupportedTypesWithAutoConversion() { TableIdentifier identifier = TableIdentifier.of("default", "not_supported_types"); // Can not create INTERVAL types from normal create table, so leave them out from this test @@ -674,13 +669,13 @@ public void testCreateTableWithNotSupportedTypesWithAutoConversion() { + testTables.propertiesForCreateTableSQL(ImmutableMap.of())); org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals( - notSupportedTypes.get(notSupportedType), icebergTable.schema().columns().get(0).type()); + assertThat(icebergTable.schema().columns().get(0).type()) + .isEqualTo(notSupportedTypes.get(notSupportedType)); shell.executeStatement("DROP TABLE not_supported_types"); } } - @Test + @TestTemplate public void testCreateTableWithColumnComments() { TableIdentifier identifier = TableIdentifier.of("default", "comment_table"); shell.executeStatement( @@ -694,20 +689,18 @@ public void testCreateTableWithColumnComments() { org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); List rows = shell.executeStatement("DESCRIBE default.comment_table"); - Assert.assertEquals(icebergTable.schema().columns().size(), rows.size()); + assertThat(rows).hasSameSizeAs(icebergTable.schema().columns()); for (int i = 0; i < icebergTable.schema().columns().size(); i++) { Types.NestedField field = icebergTable.schema().columns().get(i); - Assert.assertArrayEquals( - new Object[] { - field.name(), - HiveSchemaUtil.convert(field.type()).getTypeName(), - (field.doc() != null ? field.doc() : "from deserializer") - }, - rows.get(i)); + assertThat(rows.get(i)) + .containsExactly( + field.name(), + HiveSchemaUtil.convert(field.type()).getTypeName(), + (field.doc() != null ? field.doc() : "from deserializer")); } } - @Test + @TestTemplate public void testCreateTableWithoutColumnComments() { TableIdentifier identifier = TableIdentifier.of("default", "without_comment_table"); shell.executeStatement( @@ -720,19 +713,19 @@ public void testCreateTableWithoutColumnComments() { org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); List rows = shell.executeStatement("DESCRIBE default.without_comment_table"); - Assert.assertEquals(icebergTable.schema().columns().size(), rows.size()); + assertThat(rows).hasSameSizeAs(icebergTable.schema().columns()); for (int i = 0; i < icebergTable.schema().columns().size(); i++) { Types.NestedField field = icebergTable.schema().columns().get(i); - Assert.assertNull(field.doc()); - Assert.assertArrayEquals( - new Object[] { - field.name(), HiveSchemaUtil.convert(field.type()).getTypeName(), "from deserializer" - }, - rows.get(i)); + assertThat(field.doc()).isNull(); + assertThat(rows.get(i)) + .containsExactly( + field.name(), + HiveSchemaUtil.convert(field.type()).getTypeName(), + (field.doc() != null ? field.doc() : "from deserializer")); } } - @Test + @TestTemplate public void testIcebergAndHmsTableProperties() throws Exception { TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -779,38 +772,36 @@ public void testIcebergAndHmsTableProperties() throws Exception { if (HiveVersion.min(HiveVersion.HIVE_3)) { expectedIcebergProperties.put("bucketing_version", "2"); } - Assert.assertEquals(expectedIcebergProperties, icebergTable.properties()); + assertThat(icebergTable.properties()).isEqualTo((expectedIcebergProperties)); if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - Assert.assertEquals(14, hmsParams.size()); - Assert.assertEquals("initial_val", hmsParams.get("custom_property")); - Assert.assertEquals("TRUE", hmsParams.get(InputFormatConfig.EXTERNAL_TABLE_PURGE)); - Assert.assertEquals("TRUE", hmsParams.get("EXTERNAL")); - Assert.assertEquals("true", hmsParams.get(TableProperties.ENGINE_HIVE_ENABLED)); - Assert.assertEquals( - HiveIcebergStorageHandler.class.getName(), - hmsParams.get(hive_metastoreConstants.META_TABLE_STORAGE)); - Assert.assertEquals( - BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(), - hmsParams.get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); - Assert.assertEquals( - hmsParams.get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP), - getCurrentSnapshotForHiveCatalogTable(icebergTable)); - Assert.assertNull( - hmsParams.get(BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP)); - Assert.assertNotNull(hmsParams.get(hive_metastoreConstants.DDL_TIME)); - Assert.assertNotNull(hmsParams.get(InputFormatConfig.PARTITION_SPEC)); + assertThat(hmsParams) + .hasSize(14) + .containsEntry("custom_property", "initial_val") + .containsEntry(InputFormatConfig.EXTERNAL_TABLE_PURGE, "TRUE") + .containsEntry("EXTERNAL", "TRUE") + .containsEntry(TableProperties.ENGINE_HIVE_ENABLED, "true") + .containsEntry( + hive_metastoreConstants.META_TABLE_STORAGE, HiveIcebergStorageHandler.class.getName()) + .containsEntry( + BaseMetastoreTableOperations.TABLE_TYPE_PROP, + BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase()) + .containsEntry( + BaseMetastoreTableOperations.METADATA_LOCATION_PROP, + getCurrentSnapshotForHiveCatalogTable(icebergTable)) + .doesNotContainKey(BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP) + .containsKey(hive_metastoreConstants.DDL_TIME) + .containsKey(InputFormatConfig.PARTITION_SPEC); } else { - Assert.assertEquals(8, hmsParams.size()); - Assert.assertNull(hmsParams.get(TableProperties.ENGINE_HIVE_ENABLED)); + assertThat(hmsParams).hasSize(8).doesNotContainKey(TableProperties.ENGINE_HIVE_ENABLED); } // Check HMS inputformat/outputformat/serde - Assert.assertEquals(HiveIcebergInputFormat.class.getName(), hmsTable.getSd().getInputFormat()); - Assert.assertEquals( - HiveIcebergOutputFormat.class.getName(), hmsTable.getSd().getOutputFormat()); - Assert.assertEquals( - HiveIcebergSerDe.class.getName(), hmsTable.getSd().getSerdeInfo().getSerializationLib()); + assertThat(hmsTable.getSd().getInputFormat()).isEqualTo(HiveIcebergInputFormat.class.getName()); + assertThat(hmsTable.getSd().getOutputFormat()) + .isEqualTo(HiveIcebergOutputFormat.class.getName()); + assertThat(hmsTable.getSd().getSerdeInfo().getSerializationLib()) + .isEqualTo(HiveIcebergSerDe.class.getName()); // Add two new properties to the Iceberg table and update an existing one icebergTable @@ -827,30 +818,29 @@ public void testIcebergAndHmsTableProperties() throws Exception { .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { - // 2 newly-added properties + previous_metadata_location prop + explicit Parquet compression - Assert.assertEquals(17, hmsParams.size()); - Assert.assertEquals("true", hmsParams.get("new_prop_1")); - Assert.assertEquals("false", hmsParams.get("new_prop_2")); - Assert.assertEquals("new_val", hmsParams.get("custom_property")); + assertThat(hmsParams) + .hasSize(17) + .containsEntry("new_prop_1", "true") + .containsEntry("new_prop_2", "false") + .containsEntry("custom_property", "new_val"); String prevSnapshot = getCurrentSnapshotForHiveCatalogTable(icebergTable); icebergTable.refresh(); String newSnapshot = getCurrentSnapshotForHiveCatalogTable(icebergTable); - Assert.assertEquals( - hmsParams.get(BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP), - prevSnapshot); - Assert.assertEquals( - hmsParams.get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP), newSnapshot); + assertThat(hmsParams) + .containsEntry(BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP, prevSnapshot) + .containsEntry(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, newSnapshot); } else { - Assert.assertEquals(8, hmsParams.size()); + assertThat(hmsParams).hasSize(8); } // Remove some Iceberg props and see if they're removed from HMS table props as well if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { icebergTable.updateProperties().remove("custom_property").remove("new_prop_1").commit(); hmsParams = shell.metastore().getTable("default", "customers").getParameters(); - Assert.assertFalse(hmsParams.containsKey("custom_property")); - Assert.assertFalse(hmsParams.containsKey("new_prop_1")); - Assert.assertTrue(hmsParams.containsKey("new_prop_2")); + assertThat(hmsParams) + .doesNotContainKey("custom_property") + .doesNotContainKey("new_prop_1") + .containsKey("new_prop_2"); } // append some data and check whether HMS stats are aligned with snapshot summary @@ -860,23 +850,20 @@ public void testIcebergAndHmsTableProperties() throws Exception { shell.getHiveConf(), icebergTable, FileFormat.PARQUET, null, records); hmsParams = shell.metastore().getTable("default", "customers").getParameters(); Map summary = icebergTable.currentSnapshot().summary(); - Assert.assertEquals( - summary.get(SnapshotSummary.TOTAL_DATA_FILES_PROP), - hmsParams.get(StatsSetupConst.NUM_FILES)); - Assert.assertEquals( - summary.get(SnapshotSummary.TOTAL_RECORDS_PROP), - hmsParams.get(StatsSetupConst.ROW_COUNT)); - Assert.assertEquals( - summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP), - hmsParams.get(StatsSetupConst.TOTAL_SIZE)); + assertThat(hmsParams) + .containsEntry( + StatsSetupConst.NUM_FILES, summary.get(SnapshotSummary.TOTAL_DATA_FILES_PROP)) + .containsEntry(StatsSetupConst.ROW_COUNT, summary.get(SnapshotSummary.TOTAL_RECORDS_PROP)) + .containsEntry( + StatsSetupConst.TOTAL_SIZE, summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP)); } } - @Test + @TestTemplate public void testIcebergHMSPropertiesTranslation() throws Exception { - Assume.assumeTrue( - "Iceberg - HMS property translation is only relevant for HiveCatalog", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + assumeThat(testTableType) + .as("Iceberg - HMS property translation is only relevant for HiveCatalog") + .isEqualTo(TestTables.TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -895,8 +882,9 @@ public void testIcebergHMSPropertiesTranslation() throws Exception { // Check that HMS table prop was translated to equivalent Iceberg prop (purge -> gc.enabled) org.apache.iceberg.Table icebergTable = testTables.loadTable(identifier); - Assert.assertEquals("false", icebergTable.properties().get(GC_ENABLED)); - Assert.assertNull(icebergTable.properties().get(InputFormatConfig.EXTERNAL_TABLE_PURGE)); + assertThat(icebergTable.properties()) + .containsEntry(GC_ENABLED, "false") + .doesNotContainKey(InputFormatConfig.EXTERNAL_TABLE_PURGE); // Change Iceberg prop icebergTable.updateProperties().set(GC_ENABLED, "true").commit(); @@ -904,11 +892,12 @@ public void testIcebergHMSPropertiesTranslation() throws Exception { // Check that Iceberg prop was translated to equivalent HMS prop (gc.enabled -> purge) Map hmsParams = shell.metastore().getTable("default", "customers").getParameters(); - Assert.assertEquals("true", hmsParams.get(InputFormatConfig.EXTERNAL_TABLE_PURGE)); - Assert.assertNull(hmsParams.get(GC_ENABLED)); + assertThat(hmsParams) + .containsEntry(InputFormatConfig.EXTERNAL_TABLE_PURGE, "true") + .doesNotContainKey(GC_ENABLED); } - @Test + @TestTemplate public void testDropTableWithAppendedData() throws IOException { TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -931,11 +920,11 @@ public void testDropTableWithAppendedData() throws IOException { shell.executeStatement("DROP TABLE customers"); } - @Test + @TestTemplate public void testDropHiveTableWithoutUnderlyingTable() throws IOException { - Assume.assumeFalse( - "Not relevant for HiveCatalog", - testTableType.equals(TestTables.TestTableType.HIVE_CATALOG)); + assumeThat(testTableType) + .as("Not relevant for HiveCatalog") + .isNotEqualTo(TestTables.TestTableType.HIVE_CATALOG); TableIdentifier identifier = TableIdentifier.of("default", "customers"); // Create the Iceberg table in non-HiveCatalog diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java index 86e3baf8e759..a7aa5126e2e2 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java @@ -19,10 +19,10 @@ package org.apache.iceberg.mr.hive; import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.junit.runners.Parameterized.Parameter; -import static org.junit.runners.Parameterized.Parameters; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; +import java.nio.file.Path; import java.text.DateFormat; import java.time.LocalDate; import java.time.LocalDateTime; @@ -33,24 +33,24 @@ import org.apache.hadoop.hive.serde2.io.DateWritable; import org.apache.hadoop.hive.serde2.io.TimestampWritable; 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.common.DynFields; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.TestHelper; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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; + +@ExtendWith(ParameterizedTestExtension.class) public class TestHiveIcebergStorageHandlerTimezone { private static final Optional> dateFormat = Optional.ofNullable( @@ -82,22 +82,21 @@ public static Collection parameters() { private TestTables testTables; - @Parameter(0) - public String timezoneString; + @Parameter private String timezoneString; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @BeforeClass + @BeforeAll public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell(); } - @AfterClass + @AfterAll public static void afterClass() throws Exception { shell.stop(); } - @Before + @BeforeEach public void before() throws IOException { TimeZone.setDefault(TimeZone.getTimeZone(timezoneString)); @@ -115,12 +114,12 @@ public void before() throws IOException { HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "spark"); } - @After + @AfterEach public void after() throws Exception { HiveIcebergStorageHandlerTestUtils.close(shell); } - @Test + @TestTemplate public void testDateQuery() throws IOException { Schema dateSchema = new Schema(optional(1, "d_date", Types.DateType.get())); @@ -134,24 +133,24 @@ public void testDateQuery() throws IOException { List result = shell.executeStatement("SELECT * from date_test WHERE d_date='2020-01-21'"); - Assert.assertEquals(1, result.size()); - Assert.assertEquals("2020-01-21", result.get(0)[0]); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]).isEqualTo("2020-01-21"); result = shell.executeStatement( "SELECT * from date_test WHERE d_date in ('2020-01-21', '2020-01-22')"); - Assert.assertEquals(1, result.size()); - Assert.assertEquals("2020-01-21", result.get(0)[0]); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]).isEqualTo("2020-01-21"); result = shell.executeStatement("SELECT * from date_test WHERE d_date > '2020-01-21'"); - Assert.assertEquals(1, result.size()); - Assert.assertEquals("2020-01-24", result.get(0)[0]); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]).isEqualTo("2020-01-24"); result = shell.executeStatement("SELECT * from date_test WHERE d_date='2020-01-20'"); - Assert.assertEquals(0, result.size()); + assertThat(result).isEmpty(); } - @Test + @TestTemplate public void testTimestampQuery() throws IOException { Schema timestampSchema = new Schema(optional(1, "d_ts", Types.TimestampType.withoutZone())); @@ -165,21 +164,21 @@ public void testTimestampQuery() throws IOException { List result = shell.executeStatement("SELECT d_ts FROM ts_test WHERE d_ts='2019-02-22 09:44:54.2'"); - Assert.assertEquals(1, result.size()); - Assert.assertEquals("2019-02-22 09:44:54.2", result.get(0)[0]); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]).isEqualTo("2019-02-22 09:44:54.2"); result = shell.executeStatement( "SELECT * FROM ts_test WHERE d_ts in ('2017-01-01 22:30:57.1', '2019-02-22 09:44:54.2')"); - Assert.assertEquals(1, result.size()); - Assert.assertEquals("2019-02-22 09:44:54.2", result.get(0)[0]); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]).isEqualTo("2019-02-22 09:44:54.2"); result = shell.executeStatement("SELECT d_ts FROM ts_test WHERE d_ts < '2019-02-22 09:44:54.2'"); - Assert.assertEquals(1, result.size()); - Assert.assertEquals("2019-01-22 09:44:54.1", result.get(0)[0]); + assertThat(result).hasSize(1); + assertThat(result.get(0)[0]).isEqualTo("2019-01-22 09:44:54.1"); result = shell.executeStatement("SELECT * FROM ts_test WHERE d_ts='2017-01-01 22:30:57.3'"); - Assert.assertEquals(0, result.size()); + assertThat(result).isEmpty(); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java index 09242ead0969..c8e91de9b859 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java @@ -20,11 +20,11 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.Assume.assumeTrue; -import static org.junit.runners.Parameterized.Parameter; -import static org.junit.runners.Parameterized.Parameters; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; +import java.nio.file.Path; import java.util.Collection; import java.util.List; import java.util.Map; @@ -34,6 +34,9 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.ql.exec.mr.ExecMapper; 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; @@ -50,20 +53,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 200_000, unit = TimeUnit.MILLISECONDS) public class TestHiveIcebergStorageHandlerWithEngine { private static final String[] EXECUTION_ENGINES = new String[] {"tez", "mr"}; @@ -150,33 +150,31 @@ public static Collection parameters() { private TestTables testTables; - @Parameter(0) - public FileFormat fileFormat; + @Parameter(index = 0) + private FileFormat fileFormat; - @Parameter(1) - public String executionEngine; + @Parameter(index = 1) + private String executionEngine; - @Parameter(2) - public TestTables.TestTableType testTableType; + @Parameter(index = 2) + private TestTables.TestTableType testTableType; - @Parameter(3) - public boolean isVectorized; + @Parameter(index = 3) + private boolean isVectorized; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @Rule public Timeout timeout = new Timeout(200_000, TimeUnit.MILLISECONDS); - - @BeforeClass + @BeforeAll public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell(); } - @AfterClass + @AfterAll public static void afterClass() throws Exception { shell.stop(); } - @Before + @BeforeEach public void before() throws IOException { testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, executionEngine); @@ -189,7 +187,7 @@ public void before() throws IOException { } } - @After + @AfterEach public void after() throws Exception { HiveIcebergStorageHandlerTestUtils.close(shell); // Mixing mr and tez jobs within the same JVM can cause problems. Mr jobs set the ExecMapper @@ -203,7 +201,7 @@ public void after() throws Exception { ExecMapper.setDone(false); } - @Test + @TestTemplate public void testScanTable() throws IOException { testTables.createTable( shell, @@ -217,13 +215,12 @@ public void testScanTable() throws IOException { shell.executeStatement( "SELECT first_name, customer_id FROM default.customers ORDER BY customer_id DESC"); - Assert.assertEquals(3, descRows.size()); - Assert.assertArrayEquals(new Object[] {"Trudy", 2L}, descRows.get(0)); - Assert.assertArrayEquals(new Object[] {"Bob", 1L}, descRows.get(1)); - Assert.assertArrayEquals(new Object[] {"Alice", 0L}, descRows.get(2)); + assertThat(descRows) + .containsExactly( + new Object[] {"Trudy", 2L}, new Object[] {"Bob", 1L}, new Object[] {"Alice", 0L}); } - @Test + @TestTemplate public void testCBOWithSelectedColumnsNonOverlapJoin() throws IOException { shell.setHiveSessionValue("hive.cbo.enable", true); @@ -235,13 +232,14 @@ public void testCBOWithSelectedColumnsNonOverlapJoin() throws IOException { "SELECT o.order_id, o.customer_id, o.total, p.name " + "FROM default.orders o JOIN default.products p ON o.product_id = p.id ORDER BY o.order_id"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {100L, 0L, 11.11d, "skirt"}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {101L, 0L, 22.22d, "tee"}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {102L, 1L, 33.33d, "watch"}, rows.get(2)); + assertThat(rows) + .containsExactly( + new Object[] {100L, 0L, 11.11d, "skirt"}, + new Object[] {101L, 0L, 22.22d, "tee"}, + new Object[] {102L, 1L, 33.33d, "watch"}); } - @Test + @TestTemplate public void testDescribeTable() throws IOException { testTables.createTable( shell, @@ -250,18 +248,17 @@ public void testDescribeTable() throws IOException { fileFormat, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); List rows = shell.executeStatement("DESCRIBE default.customers"); - Assert.assertEquals( - HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.columns().size(), rows.size()); + assertThat(rows).hasSameSizeAs(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.columns()); for (int i = 0; i < HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.columns().size(); i++) { Types.NestedField field = HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA.columns().get(i); String comment = field.doc() == null ? "from deserializer" : field.doc(); - Assert.assertArrayEquals( - new Object[] {field.name(), HiveSchemaUtil.convert(field.type()).getTypeName(), comment}, - rows.get(i)); + assertThat(rows.get(i)) + .containsExactly( + field.name(), HiveSchemaUtil.convert(field.type()).getTypeName(), comment); } } - @Test + @TestTemplate public void testCBOWithSelectedColumnsOverlapJoin() throws IOException { shell.setHiveSessionValue("hive.cbo.enable", true); testTables.createTable( @@ -278,13 +275,12 @@ public void testCBOWithSelectedColumnsOverlapJoin() throws IOException { + "FROM default.orders o JOIN default.customers c ON o.customer_id = c.customer_id " + "ORDER BY o.order_id DESC"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {"Bob", 102L}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {"Alice", 101L}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {"Alice", 100L}, rows.get(2)); + assertThat(rows) + .containsExactly( + new Object[] {"Bob", 102L}, new Object[] {"Alice", 101L}, new Object[] {"Alice", 100L}); } - @Test + @TestTemplate public void testCBOWithSelfJoin() throws IOException { shell.setHiveSessionValue("hive.cbo.enable", true); @@ -295,13 +291,14 @@ public void testCBOWithSelfJoin() throws IOException { "SELECT o1.order_id, o1.customer_id, o1.total " + "FROM default.orders o1 JOIN default.orders o2 ON o1.order_id = o2.order_id ORDER BY o1.order_id"); - Assert.assertEquals(3, rows.size()); - Assert.assertArrayEquals(new Object[] {100L, 0L, 11.11d}, rows.get(0)); - Assert.assertArrayEquals(new Object[] {101L, 0L, 22.22d}, rows.get(1)); - Assert.assertArrayEquals(new Object[] {102L, 1L, 33.33d}, rows.get(2)); + assertThat(rows) + .containsExactly( + new Object[] {100L, 0L, 11.11d}, + new Object[] {101L, 0L, 22.22d}, + new Object[] {102L, 1L, 33.33d}); } - @Test + @TestTemplate public void testJoinTablesSupportedTypes() throws IOException { for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { Type type = SUPPORTED_TYPES.get(i); @@ -333,14 +330,13 @@ public void testJoinTablesSupportedTypes() throws IOException { + columnName + "=s." + columnName); - Assert.assertEquals( - "Non matching record count for table " + tableName + " with type " + type, - 1, - queryResult.size()); + assertThat(queryResult) + .as("Non matching record count for table " + tableName + " with type " + type) + .hasSize(1); } } - @Test + @TestTemplate public void testSelectDistinctFromTable() throws IOException { for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { Type type = SUPPORTED_TYPES.get(i); @@ -363,13 +359,13 @@ public void testSelectDistinctFromTable() throws IOException { shell.executeStatement( "select count(distinct(" + columnName + ")) from default." + tableName); int distinctIds = ((Long) queryResult.get(0)[0]).intValue(); - Assert.assertEquals(tableName, size, distinctIds); + assertThat(distinctIds).as(tableName).isEqualTo(size); } } - @Test + @TestTemplate public void testInsert() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Table table = testTables.createTable( @@ -400,9 +396,9 @@ record -> table, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 0); } - @Test + @TestTemplate public void testInsertSupportedTypes() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); for (int i = 0; i < SUPPORTED_TYPES.size(); i++) { Type type = SUPPORTED_TYPES.get(i); // TODO: remove this filter when issue #1881 is resolved @@ -437,9 +433,9 @@ public void testInsertSupportedTypes() throws IOException { * * @throws IOException If there is an underlying IOException */ - @Test + @TestTemplate public void testInsertFromSelect() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Table table = testTables.createTable( @@ -462,9 +458,9 @@ public void testInsertFromSelect() throws IOException { * * @throws IOException If there is an underlying IOException */ - @Test + @TestTemplate public void testInsertFromSelectWithOrderBy() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Table table = testTables.createTable( @@ -483,9 +479,9 @@ public void testInsertFromSelectWithOrderBy() throws IOException { HiveIcebergTestUtils.validateData(table, records, 0); } - @Test + @TestTemplate public void testInsertFromSelectWithProjection() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Table table = testTables.createTable( @@ -508,9 +504,9 @@ public void testInsertFromSelectWithProjection() throws IOException { HiveIcebergTestUtils.validateData(table, expected, 0); } - @Test + @TestTemplate public void testInsertUsingSourceTableWithSharedColumnsNames() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); List records = HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS; PartitionSpec spec = @@ -550,9 +546,9 @@ public void testInsertUsingSourceTableWithSharedColumnsNames() throws IOExceptio HiveIcebergTestUtils.validateData(table, expected, 0); } - @Test + @TestTemplate public void testInsertFromJoiningTwoIcebergTables() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) @@ -589,9 +585,9 @@ public void testInsertFromJoiningTwoIcebergTables() throws IOException { table, HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 0); } - @Test + @TestTemplate public void testWriteArrayOfPrimitivesInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -600,9 +596,9 @@ public void testWriteArrayOfPrimitivesInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteArrayOfArraysInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -615,9 +611,9 @@ public void testWriteArrayOfArraysInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteArrayOfMapsInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -632,9 +628,9 @@ public void testWriteArrayOfMapsInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteArrayOfStructsInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -651,9 +647,9 @@ public void testWriteArrayOfStructsInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteMapOfPrimitivesInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -665,9 +661,9 @@ public void testWriteMapOfPrimitivesInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteMapOfArraysInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -683,9 +679,9 @@ public void testWriteMapOfArraysInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteMapOfMapsInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -702,9 +698,9 @@ public void testWriteMapOfMapsInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteMapOfStructsInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -723,9 +719,9 @@ public void testWriteMapOfStructsInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteStructOfPrimitivesInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -739,9 +735,9 @@ public void testWriteStructOfPrimitivesInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteStructOfArraysInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -756,9 +752,9 @@ public void testWriteStructOfArraysInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteStructOfMapsInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -780,9 +776,9 @@ public void testWriteStructOfMapsInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testWriteStructOfStructsInTable() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -800,9 +796,9 @@ public void testWriteStructOfStructsInTable() throws IOException { testComplexTypeWrite(schema, records); } - @Test + @TestTemplate public void testPartitionedWrite() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) @@ -824,9 +820,9 @@ public void testPartitionedWrite() throws IOException { HiveIcebergTestUtils.validateData(table, records, 0); } - @Test + @TestTemplate public void testIdentityPartitionedWrite() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) @@ -848,9 +844,9 @@ public void testIdentityPartitionedWrite() throws IOException { HiveIcebergTestUtils.validateData(table, records, 0); } - @Test + @TestTemplate public void testMultilevelIdentityPartitionedWrite() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); PartitionSpec spec = PartitionSpec.builderFor(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA) @@ -873,9 +869,9 @@ public void testMultilevelIdentityPartitionedWrite() throws IOException { HiveIcebergTestUtils.validateData(table, records, 0); } - @Test + @TestTemplate public void testMultiTableInsert() throws IOException { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); testTables.createTable( shell, @@ -941,11 +937,11 @@ public void testMultiTableInsert() throws IOException { /** * Fix vectorized parquet issue-4403. */ - @Test + @TestTemplate public void testStructMapWithNull() throws IOException { - Assume.assumeTrue( - "Vectorized parquet throw class cast exception see : issue 4403", - !("PARQUET".equals(fileFormat.name()) && isVectorized)); + assumeThat(!("PARQUET".equals(fileFormat.name()) && isVectorized)) + .as("Vectorized parquet throw class cast exception see : issue 4403") + .isTrue(); Schema schema = new Schema( required(1, "id", Types.LongType.get()), @@ -968,17 +964,18 @@ public void testStructMapWithNull() throws IOException { List results = shell.executeStatement("select mapofstructs['context'].someone FROM mapwithnull"); - Assert.assertEquals(1, results.size()); - Assert.assertNull(results.get(0)[0]); + assertThat(results).hasSize(1); + assertThat(results.get(0)[0]).isNull(); } - @Test + @TestTemplate public void testWriteWithDefaultWriteFormat() { - Assume.assumeTrue( - "Testing the default file format is enough for a single scenario.", - executionEngine.equals("mr") - && testTableType == TestTables.TestTableType.HIVE_CATALOG - && fileFormat == FileFormat.ORC); + assumeThat( + executionEngine.equals("mr") + && testTableType == TestTables.TestTableType.HIVE_CATALOG + && fileFormat == FileFormat.ORC) + .as("Testing the default file format is enough for a single scenario.") + .isTrue(); TableIdentifier identifier = TableIdentifier.of("default", "customers"); @@ -994,12 +991,11 @@ public void testWriteWithDefaultWriteFormat() { shell.executeStatement(String.format("INSERT INTO %s VALUES (10, 'Linda')", identifier)); List results = shell.executeStatement(String.format("SELECT * FROM %s", identifier)); - Assert.assertEquals(1, results.size()); - Assert.assertEquals(10L, results.get(0)[0]); - Assert.assertEquals("Linda", results.get(0)[1]); + assertThat(results).hasSize(1); + assertThat(results.get(0)).containsExactly(10L, "Linda"); } - @Test + @TestTemplate public void testInsertEmptyResultSet() throws IOException { Table source = testTables.createTable( @@ -1029,12 +1025,12 @@ public void testInsertEmptyResultSet() throws IOException { HiveIcebergTestUtils.validateData(target, ImmutableList.of(), 0); } - @Test + @TestTemplate public void testStatsPopulation() throws Exception { - Assume.assumeTrue("Tez write is not implemented yet", executionEngine.equals("mr")); - Assume.assumeTrue( - "Only HiveCatalog can remove stats which become obsolete", - testTableType == TestTables.TestTableType.HIVE_CATALOG); + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); + assumeThat(testTableType) + .as("Only HiveCatalog can remove stats which become obsolete") + .isEqualTo(TestTables.TestTableType.HIVE_CATALOG); shell.setHiveSessionValue(HiveConf.ConfVars.HIVESTATSAUTOGATHER.varname, true); // create the table using a catalog which supports updating Hive stats (KEEP_HIVE_STATS is true) @@ -1059,8 +1055,8 @@ public void testStatsPopulation() throws Exception { .getTable(identifier) .getParameters() .get(StatsSetupConst.COLUMN_STATS_ACCURATE); - Assert.assertTrue( - stats.startsWith("{\"BASIC_STATS\":\"true\"")); // it's followed by column stats in Hive3 + assertThat(stats) + .startsWith("{\"BASIC_STATS\":\"true\""); // it's followed by column stats in Hive3 // Create a Catalog where the KEEP_HIVE_STATS is false shell.metastore().hiveConf().set(ConfigProperties.KEEP_HIVE_STATS, StatsSetupConst.FALSE); @@ -1082,7 +1078,7 @@ public void testStatsPopulation() throws Exception { .getTable(identifier) .getParameters() .get(StatsSetupConst.COLUMN_STATS_ACCURATE); - Assert.assertNull(stats); + assertThat(stats).isNull(); // insert some data again using Hive catalog, and check the stats are back shell.executeStatement(insert); @@ -1092,8 +1088,8 @@ public void testStatsPopulation() throws Exception { .getTable(identifier) .getParameters() .get(StatsSetupConst.COLUMN_STATS_ACCURATE); - Assert.assertTrue( - stats.startsWith("{\"BASIC_STATS\":\"true\"")); // it's followed by column stats in Hive3 + assertThat(stats) + .startsWith("{\"BASIC_STATS\":\"true\""); // it's followed by column stats in Hive3 } /** @@ -1105,9 +1101,9 @@ public void testStatsPopulation() throws Exception { * * @throws Exception - any test error */ - @Test + @TestTemplate public void testVectorizedOrcMultipleSplits() throws Exception { - assumeTrue(isVectorized && FileFormat.ORC.equals(fileFormat)); + assumeThat(isVectorized && FileFormat.ORC.equals(fileFormat)).isTrue(); // This data will be held by a ~870kB ORC file List records = @@ -1132,12 +1128,12 @@ public void testVectorizedOrcMultipleSplits() throws Exception { shell.setHiveSessionValue(InputFormatConfig.SPLIT_SIZE, "210000"); List result = shell.executeStatement("SELECT * FROM targettab ORDER BY last_name"); - Assert.assertEquals(20000, result.size()); + assertThat(result).hasSize(20000); } - @Test + @TestTemplate public void testRemoveAndAddBackColumnFromIcebergTable() throws IOException { - assumeTrue(isVectorized && FileFormat.PARQUET.equals(fileFormat)); + assumeThat(isVectorized && FileFormat.PARQUET.equals(fileFormat)).isTrue(); // Create an Iceberg table with the columns customer_id, first_name and last_name with some // initial data. Table icebergTable = @@ -1267,7 +1263,7 @@ private StringBuilder buildComplexTypeInnerQuery(Object field, Type type) { if (type instanceof Types.ListType) { query.append("array("); List elements = (List) field; - Assert.assertFalse("Hive can not handle empty array() inserts", elements.isEmpty()); + assertThat(elements).as("Hive can not handle empty array() inserts").isNotEmpty(); Type innerType = ((Types.ListType) type).fields().get(0).type(); if (!elements.isEmpty()) { elements.forEach(e -> query.append(buildComplexTypeInnerQuery(e, innerType))); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java index b24959bbe8e7..c2cf8f675007 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithMultipleCatalogs.java @@ -18,27 +18,30 @@ */ package org.apache.iceberg.mr.hive; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.Collection; import java.util.List; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.data.Record; import org.apache.iceberg.mr.InputFormatConfig; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.After; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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; + +@ExtendWith(ParameterizedTestExtension.class) public class TestHiveIcebergStorageHandlerWithMultipleCatalogs { private static final String[] EXECUTION_ENGINES = new String[] {"tez", "mr"}; @@ -46,32 +49,32 @@ public class TestHiveIcebergStorageHandlerWithMultipleCatalogs { private static final String OTHERCATALOGNAME = "table2_catalog"; private static TestHiveShell shell; - @Parameterized.Parameter(0) - public FileFormat fileFormat1; + @Parameter(index = 0) + private FileFormat fileFormat1; - @Parameterized.Parameter(1) - public FileFormat fileFormat2; + @Parameter(index = 1) + private FileFormat fileFormat2; - @Parameterized.Parameter(2) - public String executionEngine; + @Parameter(index = 2) + private String executionEngine; - @Parameterized.Parameter(3) - public TestTables.TestTableType testTableType1; + @Parameter(index = 3) + private TestTables.TestTableType testTableType1; - @Parameterized.Parameter(4) - public String table1CatalogName; + @Parameter(index = 4) + private String table1CatalogName; - @Parameterized.Parameter(5) - public TestTables.TestTableType testTableType2; + @Parameter(index = 5) + private TestTables.TestTableType testTableType2; - @Parameterized.Parameter(6) - public String table2CatalogName; + @Parameter(index = 6) + private String table2CatalogName; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private TestTables testTables1; private TestTables testTables2; - @Parameterized.Parameters( + @Parameters( name = "fileFormat1={0}, fileFormat2={1}, engine={2}, tableType1={3}, catalogName1={4}, " + "tableType2={5}, catalogName2={6}") @@ -102,17 +105,17 @@ public static Collection parameters() { return testParams; } - @BeforeClass + @BeforeAll public static void beforeClass() { shell = HiveIcebergStorageHandlerTestUtils.shell(); } - @AfterClass + @AfterAll public static void afterClass() throws Exception { shell.stop(); } - @Before + @BeforeEach public void before() throws IOException { testTables1 = HiveIcebergStorageHandlerTestUtils.testTables( @@ -132,12 +135,12 @@ public void before() throws IOException { .forEach(e -> shell.setHiveSessionValue(e.getKey(), e.getValue())); } - @After + @AfterEach public void after() throws Exception { HiveIcebergStorageHandlerTestUtils.close(shell); } - @Test + @TestTemplate public void testJoinTablesFromDifferentCatalogs() throws IOException { createAndAddRecords( testTables1, @@ -155,7 +158,7 @@ public void testJoinTablesFromDifferentCatalogs() throws IOException { "SELECT c2.customer_id, c2.first_name, c2.last_name " + "FROM default.customers2 c2 JOIN default.customers1 c1 ON c2.customer_id = c1.customer_id " + "ORDER BY c2.customer_id"); - Assert.assertEquals(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS.size(), rows.size()); + assertThat(rows).hasSameSizeAs(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS); HiveIcebergTestUtils.validateData( Lists.newArrayList(HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS), HiveIcebergTestUtils.valueForRow(HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, rows), diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java index faeb7d0df75e..8c8cf894c9f1 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java @@ -18,9 +18,12 @@ */ package org.apache.iceberg.mr.hive; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; -import java.io.UncheckedIOException; +import java.nio.file.Path; +import java.nio.file.Paths; import java.sql.Timestamp; import java.time.LocalDateTime; import java.time.OffsetDateTime; @@ -57,11 +60,8 @@ 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.relocated.com.google.common.collect.ObjectArrays; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; // Helper class for setting up and testing various catalog implementations abstract class TestTables { @@ -74,16 +74,16 @@ abstract class TestTables { }; private final Tables tables; - protected final TemporaryFolder temp; + protected final Path temp; protected final String catalog; - protected TestTables(Tables tables, TemporaryFolder temp, String catalogName) { + protected TestTables(Tables tables, Path temp, String catalogName) { this.tables = tables; this.temp = temp; this.catalog = catalogName; } - protected TestTables(Catalog catalog, TemporaryFolder temp, String catalogName) { + protected TestTables(Catalog catalog, Path temp, String catalogName) { this(new CatalogToTables(catalog), temp, catalogName); } @@ -417,18 +417,17 @@ static class CustomCatalogTestTables extends TestTables { private final String warehouseLocation; - CustomCatalogTestTables(Configuration conf, TemporaryFolder temp, String catalogName) - throws IOException { + CustomCatalogTestTables(Configuration conf, Path temp, String catalogName) throws IOException { this( conf, temp, (HiveVersion.min(HiveVersion.HIVE_3) ? "file:" : "") - + temp.newFolder("custom", "warehouse").toString(), + + temp.resolve(Paths.get("custom", "warehouse")), catalogName); } CustomCatalogTestTables( - Configuration conf, TemporaryFolder temp, String warehouseLocation, String catalogName) { + Configuration conf, Path temp, String warehouseLocation, String catalogName) { super(new TestCatalogs.CustomHadoopCatalog(conf, warehouseLocation), temp, catalogName); this.warehouseLocation = warehouseLocation; } @@ -452,18 +451,17 @@ static class HadoopCatalogTestTables extends TestTables { private final String warehouseLocation; - HadoopCatalogTestTables(Configuration conf, TemporaryFolder temp, String catalogName) - throws IOException { + HadoopCatalogTestTables(Configuration conf, Path temp, String catalogName) throws IOException { this( conf, temp, (HiveVersion.min(HiveVersion.HIVE_3) ? "file:" : "") - + temp.newFolder("hadoop", "warehouse").toString(), + + temp.resolve(Paths.get("hadoop", "warehouse")), catalogName); } HadoopCatalogTestTables( - Configuration conf, TemporaryFolder temp, String warehouseLocation, String catalogName) { + Configuration conf, Path temp, String warehouseLocation, String catalogName) { super(new HadoopCatalog(conf, warehouseLocation), temp, catalogName); this.warehouseLocation = warehouseLocation; } @@ -484,7 +482,8 @@ public String locationForCreateTableSQL(TableIdentifier identifier) { } static class HadoopTestTables extends TestTables { - HadoopTestTables(Configuration conf, TemporaryFolder temp) { + + HadoopTestTables(Configuration conf, Path temp) { super(new HadoopTables(conf), temp, Catalogs.ICEBERG_HADOOP_TABLE_NAME); } @@ -492,32 +491,32 @@ static class HadoopTestTables extends TestTables { public String identifier(String tableIdentifier) { final File location; - try { - TableIdentifier identifier = TableIdentifier.parse(tableIdentifier); - location = - temp.newFolder(ObjectArrays.concat(identifier.namespace().levels(), identifier.name())); - } catch (IOException ioe) { - throw new UncheckedIOException(ioe); - } + TableIdentifier identifier = TableIdentifier.parse(tableIdentifier); + location = + temp.resolve( + Joiner.on(File.separator).join(identifier.namespace().levels()) + + File.separator + + identifier.name()) + .toFile(); - Assert.assertTrue(location.delete()); + assertThat(location).doesNotExist(); return location.toString(); } @Override public String locationForCreateTableSQL(TableIdentifier identifier) { - return "LOCATION '" + temp.getRoot().getPath() + tablePath(identifier) + "' "; + return "LOCATION '" + temp + tablePath(identifier) + "' "; } @Override public Table loadTable(TableIdentifier identifier) { - return tables().load(temp.getRoot().getPath() + TestTables.tablePath(identifier)); + return tables().load(temp + TestTables.tablePath(identifier)); } } static class HiveTestTables extends TestTables { - HiveTestTables(Configuration conf, TemporaryFolder temp, String catalogName) { + HiveTestTables(Configuration conf, Path temp, String catalogName) { super( CatalogUtil.loadCatalog( HiveCatalog.class.getName(), @@ -569,36 +568,32 @@ private String getStringValueForInsert(Object value, Type type) { enum TestTableType { HADOOP_TABLE { @Override - public TestTables instance( - Configuration conf, TemporaryFolder temporaryFolder, String catalogName) { + public TestTables instance(Configuration conf, Path temporaryFolder, String catalogName) { return new HadoopTestTables(conf, temporaryFolder); } }, HADOOP_CATALOG { @Override - public TestTables instance( - Configuration conf, TemporaryFolder temporaryFolder, String catalogName) + public TestTables instance(Configuration conf, Path temporaryFolder, String catalogName) throws IOException { return new HadoopCatalogTestTables(conf, temporaryFolder, catalogName); } }, CUSTOM_CATALOG { @Override - public TestTables instance( - Configuration conf, TemporaryFolder temporaryFolder, String catalogName) + public TestTables instance(Configuration conf, Path temporaryFolder, String catalogName) throws IOException { return new CustomCatalogTestTables(conf, temporaryFolder, catalogName); } }, HIVE_CATALOG { @Override - public TestTables instance( - Configuration conf, TemporaryFolder temporaryFolder, String catalogName) { + public TestTables instance(Configuration conf, Path temporaryFolder, String catalogName) { return new HiveTestTables(conf, temporaryFolder, catalogName); } }; public abstract TestTables instance( - Configuration conf, TemporaryFolder temporaryFolder, String catalogName) throws IOException; + Configuration conf, Path temporaryFolder, String catalogName) throws IOException; } } From a2c1e3bdc8c4cb1b07a6156fbaa8edd3cd688e28 Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Wed, 14 Feb 2024 21:12:22 +0100 Subject: [PATCH 0037/1019] Upgrade Nessie to 0.77.1 (#9726) --- build.gradle | 2 +- gradle/libs.versions.toml | 4 ++-- .../test/java/org/apache/iceberg/nessie/BaseTestIceberg.java | 2 +- .../java/org/apache/iceberg/nessie/TestNessieCatalog.java | 2 +- .../java/org/apache/iceberg/nessie/TestNessieViewCatalog.java | 2 +- 5 files changed, 6 insertions(+), 6 deletions(-) diff --git a/build.gradle b/build.gradle index 00ca99e9456a..90675cfb9800 100644 --- a/build.gradle +++ b/build.gradle @@ -919,7 +919,7 @@ project(':iceberg-nessie') { if (JavaVersion.current().isJava11Compatible()) { testImplementation libs.nessie.jaxrs.testextension - testImplementation libs.nessie.versioned.storage.inmemory + testImplementation libs.nessie.versioned.storage.inmemory.tests testImplementation libs.nessie.versioned.storage.testextension // Need to "pull in" el-api explicitly :( testImplementation libs.jakarta.el.api diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 82bda5d9f09f..d20f1055f85f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -67,7 +67,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.76.3" +nessie = "0.77.1" netty-buffer = "4.1.97.Final" netty-buffer-compat = "4.1.68.Final" object-client-bundle = "3.3.2" @@ -208,7 +208,7 @@ mockito-junit-jupiter = { module = "org.mockito:mockito-junit-jupiter", version. mockserver-client-java = { module = "org.mock-server:mockserver-client-java", version.ref = "mockserver" } mockserver-netty = { module = "org.mock-server:mockserver-netty", version.ref = "mockserver" } nessie-jaxrs-testextension = { module = "org.projectnessie.nessie:nessie-jaxrs-testextension", version.ref = "nessie" } -nessie-versioned-storage-inmemory = { module = "org.projectnessie.nessie:nessie-versioned-storage-inmemory", version.ref = "nessie" } +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" } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java index 4cb768f2aa64..c2fa8d4da197 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java @@ -70,7 +70,7 @@ import org.projectnessie.model.Reference; import org.projectnessie.model.Tag; import org.projectnessie.versioned.storage.common.persist.Persist; -import org.projectnessie.versioned.storage.inmemory.InmemoryBackendTestFactory; +import org.projectnessie.versioned.storage.inmemorytests.InmemoryBackendTestFactory; import org.projectnessie.versioned.storage.testextension.NessieBackend; import org.projectnessie.versioned.storage.testextension.NessiePersist; import org.projectnessie.versioned.storage.testextension.PersistExtension; 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 107106f0835b..dbe8f92cdf81 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java @@ -47,7 +47,7 @@ import org.projectnessie.model.Reference; import org.projectnessie.model.Tag; import org.projectnessie.versioned.storage.common.persist.Persist; -import org.projectnessie.versioned.storage.inmemory.InmemoryBackendTestFactory; +import org.projectnessie.versioned.storage.inmemorytests.InmemoryBackendTestFactory; import org.projectnessie.versioned.storage.testextension.NessieBackend; import org.projectnessie.versioned.storage.testextension.NessiePersist; import org.projectnessie.versioned.storage.testextension.PersistExtension; diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java index 005e700f431b..cb2c61e9dcb8 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieViewCatalog.java @@ -50,7 +50,7 @@ import org.projectnessie.model.Reference; import org.projectnessie.model.Tag; import org.projectnessie.versioned.storage.common.persist.Persist; -import org.projectnessie.versioned.storage.inmemory.InmemoryBackendTestFactory; +import org.projectnessie.versioned.storage.inmemorytests.InmemoryBackendTestFactory; import org.projectnessie.versioned.storage.testextension.NessieBackend; import org.projectnessie.versioned.storage.testextension.NessiePersist; import org.projectnessie.versioned.storage.testextension.PersistExtension; From 3b60ca2e3b9f59711cefa874baf4f624ca62baae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 14 Feb 2024 21:14:26 +0100 Subject: [PATCH 0038/1019] Build: Bump org.assertj:assertj-core from 3.25.2 to 3.25.3 (#9706) Bumps [org.assertj:assertj-core](https://github.com/assertj/assertj) from 3.25.2 to 3.25.3. - [Release notes](https://github.com/assertj/assertj/releases) - [Commits](https://github.com/assertj/assertj/compare/assertj-build-3.25.2...assertj-build-3.25.3) --- updated-dependencies: - dependency-name: org.assertj:assertj-core 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 d20f1055f85f..14747e7e5c13 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -26,7 +26,7 @@ antlr = "4.9.3" aircompressor = "0.26" arrow = "14.0.2" avro = "1.11.3" -assertj-core = "3.25.2" +assertj-core = "3.25.3" awaitility = "4.2.0" awssdk-bom = "2.23.17" azuresdk-bom = "1.2.20" From aa003d54ffa673823919247ec5e4159d13401bcb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 14 Feb 2024 21:15:32 +0100 Subject: [PATCH 0039/1019] Build: Bump tez010 from 0.10.2 to 0.10.3 (#9702) Bumps `tez010` from 0.10.2 to 0.10.3. Updates `org.apache.tez:tez-dag` from 0.10.2 to 0.10.3 Updates `org.apache.tez:tez-mapreduce` from 0.10.2 to 0.10.3 --- updated-dependencies: - dependency-name: org.apache.tez:tez-dag dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.tez:tez-mapreduce 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 14747e7e5c13..821fddb4d6c7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -86,7 +86,7 @@ spring-boot = "2.5.4" spring-web = "5.3.30" sqlite-jdbc = "3.45.1.0" testcontainers = "1.19.5" -tez010 = "0.10.2" +tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above [libraries] From 55998481db1268146dd203462f8a05797949e84e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 14 Feb 2024 21:16:18 +0100 Subject: [PATCH 0040/1019] Build: Bump software.amazon.awssdk:bom from 2.23.17 to 2.24.0 (#9701) Bumps software.amazon.awssdk:bom from 2.23.17 to 2.24.0. --- 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 821fddb4d6c7..a16c44340742 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "14.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.0" -awssdk-bom = "2.23.17" +awssdk-bom = "2.24.0" azuresdk-bom = "1.2.20" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 2b22701f921446beaa93906696b00806706c613a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 14 Feb 2024 21:18:01 +0100 Subject: [PATCH 0041/1019] Build: Bump arrow from 14.0.2 to 15.0.0 (#9574) Bumps `arrow` from 14.0.2 to 15.0.0. Updates `org.apache.arrow:arrow-memory-netty` from 14.0.2 to 15.0.0 Updates `org.apache.arrow:arrow-vector` from 14.0.2 to 15.0.0 - [Commits](https://github.com/apache/arrow/compare/go/v14.0.2...go/v15.0.0) --- updated-dependencies: - dependency-name: org.apache.arrow:arrow-memory-netty dependency-type: direct:production update-type: version-update:semver-major - dependency-name: org.apache.arrow:arrow-vector dependency-type: direct:production update-type: version-update:semver-major ... 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 a16c44340742..5a919142272b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -24,7 +24,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.26" -arrow = "14.0.2" +arrow = "15.0.0" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.0" From 9d7c2482b9f2a08f8fe65e04a2b8800333e3ee90 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 15 Feb 2024 11:19:28 +0100 Subject: [PATCH 0042/1019] Core: Only write view history when currentVersionId changes (#9725) --- .../org/apache/iceberg/view/ViewMetadata.java | 20 ++- .../responses/TestLoadViewResponseParser.java | 12 -- .../apache/iceberg/view/TestViewMetadata.java | 136 ++++++++++++++---- .../iceberg/view/TestViewMetadataParser.java | 48 ++++--- 4 files changed, 152 insertions(+), 64 deletions(-) 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 fa75c352f198..68d8c6458701 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -31,6 +31,7 @@ import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.Schema; import org.apache.iceberg.exceptions.ValidationException; +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.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -156,6 +157,7 @@ class Builder { // internal change tracking private Integer lastAddedVersionId = null; private Integer lastAddedSchemaId = null; + private ViewHistoryEntry historyEntry = null; // indexes private final Map versionsById; @@ -243,6 +245,12 @@ public Builder setCurrentVersionId(int newVersionId) { changes.add(new MetadataUpdate.SetCurrentViewVersion(newVersionId)); } + this.historyEntry = + ImmutableViewHistoryEntry.builder() + .timestampMillis(version.timestampMillis()) + .versionId(version.versionId()) + .build(); + return this; } @@ -307,12 +315,6 @@ private int addVersionInternal(ViewVersion newVersion) { changes.add(new MetadataUpdate.AddViewVersion(version)); } - history.add( - ImmutableViewHistoryEntry.builder() - .timestampMillis(version.timestampMillis()) - .versionId(version.versionId()) - .build()); - this.lastAddedVersionId = newVersionId; return newVersionId; @@ -438,6 +440,10 @@ public ViewMetadata build() { metadataLocation == null || changes.isEmpty(), "Cannot create view metadata with a metadata location and changes"); + if (null != historyEntry) { + history.add(historyEntry); + } + int historySize = PropertyUtil.propertyAsInt( properties, @@ -479,6 +485,7 @@ public ViewMetadata build() { metadataLocation); } + @VisibleForTesting static List expireVersions( Map versionsById, int numVersionsToKeep) { // version ids are assigned sequentially. keep the latest versions by ID. @@ -493,6 +500,7 @@ static List expireVersions( return retainedVersions; } + @VisibleForTesting static List updateHistory(List history, Set ids) { List retainedHistory = Lists.newArrayList(); for (ViewHistoryEntry entry : history) { 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 d94d035596c9..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 @@ -137,12 +137,6 @@ public void roundTripSerde() { + " \"representations\" : [ ]\n" + " } ],\n" + " \"version-log\" : [ {\n" - + " \"timestamp-ms\" : 23,\n" - + " \"version-id\" : 1\n" - + " }, {\n" - + " \"timestamp-ms\" : 24,\n" - + " \"version-id\" : 2\n" - + " }, {\n" + " \"timestamp-ms\" : 25,\n" + " \"version-id\" : 3\n" + " } ]\n" @@ -235,12 +229,6 @@ public void roundTripSerdeWithConfig() { + " \"representations\" : [ ]\n" + " } ],\n" + " \"version-log\" : [ {\n" - + " \"timestamp-ms\" : 23,\n" - + " \"version-id\" : 1\n" - + " }, {\n" - + " \"timestamp-ms\" : 24,\n" - + " \"version-id\" : 2\n" - + " }, {\n" + " \"timestamp-ms\" : 25,\n" + " \"version-id\" : 3\n" + " } ]\n" diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java index e60fe3b285b5..70f87e397435 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -23,7 +23,6 @@ import java.util.List; import java.util.Map; -import java.util.Set; import java.util.UUID; import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.Schema; @@ -63,8 +62,11 @@ public void testExpiration() { ViewVersion v2 = newViewVersion(2, "select count(1) as count from t2"); Map versionsById = ImmutableMap.of(1, v1, 2, v2, 3, v3); - List retainedVersions = ViewMetadata.Builder.expireVersions(versionsById, 2); - assertThat(retainedVersions).hasSameElementsAs(ImmutableList.of(v2, v3)); + assertThat(ViewMetadata.Builder.expireVersions(versionsById, 3)) + .containsExactlyInAnyOrder(v1, v2, v3); + assertThat(ViewMetadata.Builder.expireVersions(versionsById, 2)) + .containsExactlyInAnyOrder(v2, v3); + assertThat(ViewMetadata.Builder.expireVersions(versionsById, 1)).containsExactly(v3); } @Test @@ -73,26 +75,38 @@ public void testUpdateHistory() { ViewVersion v2 = newViewVersion(2, "select count(1) as count from t2"); ViewVersion v3 = newViewVersion(3, "select count from t1"); - Set versionsById = ImmutableSet.of(2, 3); - - List history = - ImmutableList.of( - ImmutableViewHistoryEntry.builder() - .versionId(v1.versionId()) - .timestampMillis(v1.timestampMillis()) - .build(), - ImmutableViewHistoryEntry.builder() - .versionId(v2.versionId()) - .timestampMillis(v2.timestampMillis()) - .build(), - ImmutableViewHistoryEntry.builder() - .versionId(v3.versionId()) - .timestampMillis(v3.timestampMillis()) - .build()); - - List retainedHistory = - ViewMetadata.Builder.updateHistory(history, versionsById); - assertThat(retainedHistory).hasSameElementsAs(history.subList(1, 3)); + ViewHistoryEntry one = + ImmutableViewHistoryEntry.builder() + .versionId(v1.versionId()) + .timestampMillis(v1.timestampMillis()) + .build(); + ViewHistoryEntry two = + ImmutableViewHistoryEntry.builder() + .versionId(v2.versionId()) + .timestampMillis(v2.timestampMillis()) + .build(); + ViewHistoryEntry three = + ImmutableViewHistoryEntry.builder() + .versionId(v3.versionId()) + .timestampMillis(v3.timestampMillis()) + .build(); + + assertThat( + ViewMetadata.Builder.updateHistory( + ImmutableList.of(one, two, three), ImmutableSet.of(1, 2, 3))) + .containsExactly(one, two, three); + + // one was an invalid entry in the history, so all previous elements are removed + assertThat( + ViewMetadata.Builder.updateHistory( + ImmutableList.of(three, two, one, two, three), ImmutableSet.of(2, 3))) + .containsExactly(two, three); + + // two was an invalid entry in the history, so all previous elements are removed + assertThat( + ViewMetadata.Builder.updateHistory( + ImmutableList.of(one, two, three, one, three), ImmutableSet.of(1, 3))) + .containsExactly(three, one, three); } @Test @@ -233,7 +247,7 @@ public void invalidVersionHistorySizeToKeep() { } @Test - public void viewHistoryNormalization() { + public void viewVersionHistoryNormalization() { Map properties = ImmutableMap.of(ViewProperties.VERSION_HISTORY_SIZE, "2"); ViewVersion viewVersionOne = newViewVersion(1, "select * from ns.tbl"); ViewVersion viewVersionTwo = newViewVersion(2, "select count(*) from ns.tbl"); @@ -252,12 +266,12 @@ public void viewHistoryNormalization() { // the first build will not expire versions that were added in the builder assertThat(originalViewMetadata.versions()).hasSize(3); - assertThat(originalViewMetadata.history()).hasSize(3); + assertThat(originalViewMetadata.history()).hasSize(1); // rebuild the metadata to expire older versions ViewMetadata viewMetadata = ViewMetadata.buildFrom(originalViewMetadata).build(); assertThat(viewMetadata.versions()).hasSize(2); - assertThat(viewMetadata.history()).hasSize(2); + assertThat(viewMetadata.history()).hasSize(1); // make sure that metadata changes reflect the current state after the history was adjusted, // meaning that the first view version shouldn't be included @@ -314,6 +328,74 @@ public void viewHistoryNormalization() { .isEqualTo(-1); } + @Test + public void viewVersionHistoryIsCorrectlyRetained() { + Map properties = ImmutableMap.of(ViewProperties.VERSION_HISTORY_SIZE, "2"); + ViewVersion viewVersionOne = newViewVersion(1, "select * from ns.tbl"); + ViewVersion viewVersionTwo = newViewVersion(2, "select count(*) from ns.tbl"); + ViewVersion viewVersionThree = newViewVersion(3, "select count(*) as count from ns.tbl"); + + ViewMetadata originalViewMetadata = + ViewMetadata.builder() + .setProperties(properties) + .setLocation("location") + .addSchema(new Schema(Types.NestedField.required(1, "x", Types.LongType.get()))) + .addVersion(viewVersionOne) + .addVersion(viewVersionTwo) + .addVersion(viewVersionThree) + .setCurrentVersionId(3) + .build(); + + assertThat(originalViewMetadata.versions()) + .hasSize(3) + .containsExactlyInAnyOrder(viewVersionOne, viewVersionTwo, viewVersionThree); + assertThat(originalViewMetadata.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(3); + + // rebuild the metadata to expire older versions + ViewMetadata viewMetadata = ViewMetadata.buildFrom(originalViewMetadata).build(); + assertThat(viewMetadata.versions()) + .hasSize(2) + // there is no requirement about the order of versions + .containsExactlyInAnyOrder(viewVersionThree, viewVersionTwo); + assertThat(viewMetadata.history()) + .hasSize(1) + .first() + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(3); + + ViewMetadata updated = ViewMetadata.buildFrom(viewMetadata).setCurrentVersionId(2).build(); + assertThat(updated.versions()) + .hasSize(2) + .containsExactlyInAnyOrder(viewVersionTwo, viewVersionThree); + assertThat(updated.history()) + .hasSize(2) + .element(0) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(3); + assertThat(updated.history()).element(1).extracting(ViewHistoryEntry::versionId).isEqualTo(2); + + ViewMetadata view = ViewMetadata.buildFrom(updated).setCurrentVersionId(3).build(); + assertThat(view.versions()) + .hasSize(2) + .containsExactlyInAnyOrder(viewVersionTwo, viewVersionThree); + assertThat(view.history()) + .hasSize(3) + .element(0) + .extracting(ViewHistoryEntry::versionId) + .isEqualTo(3); + assertThat(view.history()).element(1).extracting(ViewHistoryEntry::versionId).isEqualTo(2); + assertThat(view.history()).element(2).extracting(ViewHistoryEntry::versionId).isEqualTo(3); + + // viewVersionId 1 has been removed from versions, so this should fail + assertThatThrownBy(() -> ViewMetadata.buildFrom(view).setCurrentVersionId(1).build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set current version to unknown version: 1"); + } + @Test public void viewMetadataAndMetadataChanges() { Map properties = ImmutableMap.of("key1", "prop1", "key2", "prop2"); @@ -340,7 +422,7 @@ public void viewMetadataAndMetadataChanges() { assertThat(viewMetadata.versions()) .hasSize(3) .containsExactly(viewVersionOne, viewVersionTwo, viewVersionThree); - assertThat(viewMetadata.history()).hasSize(3); + assertThat(viewMetadata.history()).hasSize(1); assertThat(viewMetadata.currentVersionId()).isEqualTo(3); assertThat(viewMetadata.currentVersion()).isEqualTo(viewVersionThree); assertThat(viewMetadata.formatVersion()).isEqualTo(ViewMetadata.DEFAULT_VIEW_FORMAT_VERSION); diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java index d2372363723b..7784fdc4ed04 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadataParser.java @@ -101,16 +101,20 @@ public void readAndWriteValidViewMetadata() throws Exception { String json = readViewMetadataInputFile("org/apache/iceberg/view/ValidViewMetadata.json"); ViewMetadata expectedViewMetadata = - ViewMetadata.builder() - .assignUUID("fa6506c3-7681-40c8-86dc-e36561f83385") - .addSchema(TEST_SCHEMA) - .addVersion(version1) - .addVersion(version2) - .setLocation("s3://bucket/test/location") - .setProperties( - ImmutableMap.of("some-key", "some-value", ViewProperties.COMMENT, "some-comment")) + ViewMetadata.buildFrom( + ViewMetadata.builder() + .assignUUID("fa6506c3-7681-40c8-86dc-e36561f83385") + .addSchema(TEST_SCHEMA) + .addVersion(version1) + .addVersion(version2) + .setLocation("s3://bucket/test/location") + .setProperties( + ImmutableMap.of( + "some-key", "some-value", ViewProperties.COMMENT, "some-comment")) + .setCurrentVersionId(1) + .upgradeFormatVersion(1) + .build()) .setCurrentVersionId(2) - .upgradeFormatVersion(1) .build(); ViewMetadata actual = ViewMetadataParser.fromJson(json); @@ -213,17 +217,23 @@ public void viewMetadataWithMetadataLocation() throws Exception { String metadataLocation = "s3://bucket/test/location/metadata/v1.metadata.json"; ViewMetadata expectedViewMetadata = ViewMetadata.buildFrom( - ViewMetadata.builder() - .assignUUID("fa6506c3-7681-40c8-86dc-e36561f83385") - .addSchema(TEST_SCHEMA) - .addVersion(version1) - .addVersion(version2) - .setLocation("s3://bucket/test/location") - .setProperties( - ImmutableMap.of( - "some-key", "some-value", ViewProperties.COMMENT, "some-comment")) + ViewMetadata.buildFrom( + ViewMetadata.builder() + .assignUUID("fa6506c3-7681-40c8-86dc-e36561f83385") + .addSchema(TEST_SCHEMA) + .addVersion(version1) + .addVersion(version2) + .setLocation("s3://bucket/test/location") + .setProperties( + ImmutableMap.of( + "some-key", + "some-value", + ViewProperties.COMMENT, + "some-comment")) + .setCurrentVersionId(1) + .upgradeFormatVersion(1) + .build()) .setCurrentVersionId(2) - .upgradeFormatVersion(1) .build()) .setMetadataLocation(metadataLocation) .build(); From 27a7be61dcc45f3ab9876c5ffa3a85887ce0c766 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 15 Feb 2024 20:18:48 +0100 Subject: [PATCH 0043/1019] Core: Add strictness flag to prevent loss of view representation when replacing a view (#9620) --- .../org/apache/iceberg/view/ViewMetadata.java | 39 ++- .../apache/iceberg/view/ViewProperties.java | 2 + .../apache/iceberg/view/TestViewMetadata.java | 302 +++++++++++++++++- .../apache/iceberg/view/ViewCatalogTests.java | 5 +- .../iceberg/nessie/BaseTestIceberg.java | 2 + .../iceberg/spark/extensions/TestViews.java | 102 ++++++ .../iceberg/spark/extensions/TestViews.java | 102 ++++++ 7 files changed, 550 insertions(+), 4 deletions(-) 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 68d8c6458701..ae837ff96882 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewMetadata.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Comparator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Set; @@ -158,6 +159,7 @@ class Builder { private Integer lastAddedVersionId = null; private Integer lastAddedSchemaId = null; private ViewHistoryEntry historyEntry = null; + private ViewVersion previousViewVersion = null; // indexes private final Map versionsById; @@ -187,6 +189,7 @@ private Builder(ViewMetadata base) { this.location = base.location(); this.uuid = base.uuid(); this.metadataLocation = null; + this.previousViewVersion = base.currentVersion(); } public Builder upgradeFormatVersion(int newFormatVersion) { @@ -298,9 +301,9 @@ private int addVersionInternal(ViewVersion newVersion) { if (repr instanceof SQLViewRepresentation) { SQLViewRepresentation sql = (SQLViewRepresentation) repr; Preconditions.checkArgument( - dialects.add(sql.dialect()), + dialects.add(sql.dialect().toLowerCase(Locale.ROOT)), "Invalid view version: Cannot add multiple queries for dialect %s", - sql.dialect()); + sql.dialect().toLowerCase(Locale.ROOT)); } } @@ -444,6 +447,14 @@ public ViewMetadata build() { history.add(historyEntry); } + if (null != previousViewVersion + && !PropertyUtil.propertyAsBoolean( + properties, + ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, + ViewProperties.REPLACE_DROP_DIALECT_ALLOWED_DEFAULT)) { + checkIfDialectIsDropped(previousViewVersion, versionsById.get(currentVersionId)); + } + int historySize = PropertyUtil.propertyAsInt( properties, @@ -518,5 +529,29 @@ static List updateHistory(List history, Set< private Stream changes(Class updateClass) { return changes.stream().filter(updateClass::isInstance).map(updateClass::cast); } + + private void checkIfDialectIsDropped(ViewVersion previous, ViewVersion current) { + Set baseDialects = sqlDialectsFor(previous); + Set updatedDialects = sqlDialectsFor(current); + + Preconditions.checkState( + updatedDialects.containsAll(baseDialects), + "Cannot replace view due to loss of view dialects (%s=false):\nPrevious dialects: %s\nNew dialects: %s", + ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, + baseDialects, + updatedDialects); + } + + private Set sqlDialectsFor(ViewVersion viewVersion) { + Set dialects = Sets.newHashSet(); + for (ViewRepresentation repr : viewVersion.representations()) { + if (repr instanceof SQLViewRepresentation) { + SQLViewRepresentation sql = (SQLViewRepresentation) repr; + dialects.add(sql.dialect().toLowerCase(Locale.ROOT)); + } + } + + return dialects; + } } } diff --git a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java index 12b63659bcb2..ecade26bdec6 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewProperties.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewProperties.java @@ -26,6 +26,8 @@ public class ViewProperties { public static final String METADATA_COMPRESSION = "write.metadata.compression-codec"; public static final String METADATA_COMPRESSION_DEFAULT = "gzip"; public static final String COMMENT = "comment"; + public static final String REPLACE_DROP_DIALECT_ALLOWED = "replace.drop-dialect.allowed"; + public static final boolean REPLACE_DROP_DIALECT_ALLOWED_DEFAULT = false; private ViewProperties() {} } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java index 70f87e397435..60e615a99310 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewMetadata.java @@ -858,7 +858,7 @@ public void viewMetadataWithMultipleSQLForSameDialect() { .build()) .addRepresentations( ImmutableSQLViewRepresentation.builder() - .dialect("spark") + .dialect("SpArK") .sql("select * from ns.tbl2") .build()) .build()) @@ -881,4 +881,304 @@ public void lastAddedSchemaFailure() { .isInstanceOf(ValidationException.class) .hasMessage("Cannot set last added schema: no schema has been added"); } + + @Test + public void droppingDialectFailsByDefault() { + Schema schema = new Schema(Types.NestedField.required(1, "x", Types.LongType.get())); + ViewRepresentation spark = + ImmutableSQLViewRepresentation.builder().dialect("spark").sql("select * from tbl").build(); + ViewRepresentation trino = + ImmutableSQLViewRepresentation.builder().dialect("trino").sql("select * from tbl").build(); + + ViewMetadata metadata = + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema(schema) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(spark) + .build(), + schema) + .build(); + + assertThatThrownBy( + () -> + ViewMetadata.buildFrom(metadata) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(trino) + .build(), + schema) + .build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [spark]\n" + + "New dialects: [trino]"); + } + + @Test + public void droppingDialectDoesNotFailWhenAllowed() { + Schema schema = new Schema(Types.NestedField.required(1, "x", Types.LongType.get())); + ViewRepresentation spark = + ImmutableSQLViewRepresentation.builder().dialect("spark").sql("select * from tbl").build(); + ViewRepresentation trino = + ImmutableSQLViewRepresentation.builder().dialect("trino").sql("select * from tbl").build(); + + ViewMetadata metadata = + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema(schema) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(spark) + .build(), + schema) + .build(); + + assertThat(metadata.currentVersion().representations()).containsExactly(spark); + + // replacing spark with trino should work when dropping the dialect is allowed + ViewMetadata updated = + ViewMetadata.buildFrom(metadata) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(trino) + .build(), + schema) + .setProperties(ImmutableMap.of(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "true")) + .build(); + + assertThat(updated.currentVersion().representations()).containsExactly(trino); + } + + @Test + public void droppingDialectDoesNotFailWhenKeepingPreviousRepresentation() { + Schema schema = new Schema(Types.NestedField.required(1, "x", Types.LongType.get())); + ViewRepresentation spark = + ImmutableSQLViewRepresentation.builder().dialect("spark").sql("select * from tbl").build(); + ViewRepresentation trino = + ImmutableSQLViewRepresentation.builder().dialect("trino").sql("select * from tbl").build(); + + ViewMetadata metadata = + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema(schema) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(spark) + .build(), + schema) + .build(); + + // keeping the previous representation and adding a new one shouldn't fail + ViewMetadata updated = + ViewMetadata.buildFrom(metadata) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(spark) + .addRepresentations(trino) + .build(), + schema) + .build(); + + assertThat(updated.currentVersion().representations()).containsExactly(spark, trino); + } + + @Test + public void droppingDialectDoesNotFailWhenAddingNewRepresentation() { + Schema schema = new Schema(Types.NestedField.required(1, "x", Types.LongType.get())); + ViewMetadata metadata = + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema(schema) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .build(), + schema) + .build(); + + // this shouldn't fail as there are no previous representations + assertThat(ViewMetadata.buildFrom(metadata).build().currentVersion().representations()) + .isEmpty(); + + ViewRepresentation spark = + ImmutableSQLViewRepresentation.builder().dialect("spark").sql("select * from tbl").build(); + ViewMetadata updated = + ViewMetadata.buildFrom(metadata) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + // this shouldn't fail as there are no previous representations + .addRepresentations(spark) + .build(), + schema) + .build(); + + assertThat(updated.currentVersion().representations()).containsExactly(spark); + } + + @Test + public void droppingDialectFailsWhenSwitchingViewVersionId() { + Schema schema = new Schema(Types.NestedField.required(1, "x", Types.LongType.get())); + ViewRepresentation spark = + ImmutableSQLViewRepresentation.builder().dialect("spark").sql("select * from tbl").build(); + ViewRepresentation trino = + ImmutableSQLViewRepresentation.builder().dialect("trino").sql("select * from tbl").build(); + + ViewVersion sparkOnly = + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(spark) + .build(); + ViewVersion trinoOnly = + ImmutableViewVersion.builder() + .versionId(2) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(trino) + .build(); + ViewVersion trinoAndSpark = + ImmutableViewVersion.builder() + .versionId(3) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(trino) + .addRepresentations(spark) + .build(); + + // add all different view versions and set it to using sparkOnly + ViewMetadata metadata = + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema(schema) + .addVersion(sparkOnly) + .addVersion(trinoOnly) + .addVersion(trinoAndSpark) + .setCurrentVersionId(1) + .build(); + + assertThatThrownBy( + () -> + ViewMetadata.buildFrom(metadata) + // switches from sparkOnly to trinoOnly. Intermediate versionId updates are + // ignored + .setCurrentVersionId(trinoAndSpark.versionId()) + .setCurrentVersionId(sparkOnly.versionId()) + .setCurrentVersionId(trinoOnly.versionId()) + .build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [spark]\n" + + "New dialects: [trino]"); + + assertThatThrownBy( + () -> + ViewMetadata.buildFrom( + ViewMetadata.buildFrom(metadata) + .setCurrentVersionId(trinoAndSpark.versionId()) + .build()) + // switches from trinoAndSpark back to trinoOnly. Intermediate versionId + // updates are ignored + .setCurrentVersionId(sparkOnly.versionId()) + .setCurrentVersionId(trinoOnly.versionId()) + .build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino, spark]\n" + + "New dialects: [trino]"); + } + + @Test + public void droppingDialectAllowedAndThenDisallowed() { + Schema schema = new Schema(Types.NestedField.required(1, "x", Types.LongType.get())); + ViewRepresentation spark = + ImmutableSQLViewRepresentation.builder().dialect("spark").sql("select * from tbl").build(); + ViewRepresentation trino = + ImmutableSQLViewRepresentation.builder().dialect("trino").sql("select * from tbl").build(); + + ViewMetadata metadata = + ViewMetadata.builder() + .setLocation("custom-location") + .addSchema(schema) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + .addRepresentations(spark) + .build(), + schema) + .build(); + + ViewMetadata updated = + ViewMetadata.buildFrom(metadata) + .setCurrentVersion( + ImmutableViewVersion.builder() + .versionId(1) + .schemaId(0) + .timestampMillis(System.currentTimeMillis()) + .defaultNamespace(Namespace.empty()) + // this drops spark and is allowed + .addRepresentations(trino) + .build(), + schema) + .setProperties(ImmutableMap.of(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "true")) + .build(); + + assertThat(updated.currentVersion().representations()).containsExactly(trino); + + assertThatThrownBy( + () -> + ViewMetadata.buildFrom(updated) + // switches back to spark and isn't allowed + .setCurrentVersionId(1) + .setProperties( + ImmutableMap.of(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "false")) + .build()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino]\n" + + "New dialects: [spark]"); + } } diff --git a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java index 58727e4588ff..b3765bb1eae7 100644 --- a/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/view/ViewCatalogTests.java @@ -863,7 +863,8 @@ public void createOrReplaceView(boolean useCreateOrReplace) { .withDefaultNamespace(identifier.namespace()) .withQuery("trino", "select count(*) from ns.tbl") .withProperty("replacedProp1", "val1") - .withProperty("replacedProp2", "val2"); + .withProperty("replacedProp2", "val2") + .withProperty(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "true"); View replacedView = useCreateOrReplace ? viewBuilder.createOrReplace() : viewBuilder.replace(); // validate replaced view settings @@ -1092,6 +1093,7 @@ public void replaceViewVersion() { .withDefaultNamespace(identifier.namespace()) .withQuery(trino.dialect(), trino.sql()) .withQuery(spark.dialect(), spark.sql()) + .withProperty(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "true") .create(); assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); @@ -1552,6 +1554,7 @@ public void concurrentReplaceViewVersion() { .withSchema(SCHEMA) .withDefaultNamespace(identifier.namespace()) .withQuery("trino", "select * from ns.tbl") + .withProperty(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "true") .create(); assertThat(catalog().viewExists(identifier)).as("View should exist").isTrue(); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java index c2fa8d4da197..d56d07f7d27a 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java @@ -53,6 +53,7 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.view.BaseView; import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewProperties; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.extension.ExtendWith; @@ -200,6 +201,7 @@ protected View createView( .withSchema(schema) .withDefaultNamespace(tableIdentifier.namespace()) .withQuery("spark", "select * from ns.tbl") + .withProperty(ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, "true") .create(); } 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 9e78c54518c3..ea9ccc9133fe 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 @@ -39,12 +39,18 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableSQLViewRepresentation; +import org.apache.iceberg.view.SQLViewRepresentation; import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewHistoryEntry; +import org.apache.iceberg.view.ViewProperties; +import org.apache.iceberg.view.ViewVersion; 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.NoSuchTableException; import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -1604,6 +1610,102 @@ public void createOrReplaceViewKeepsViewHistory() { .asStruct()); } + @Test + public void replacingTrinoViewShouldFail() { + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino]\n" + + "New dialects: [spark]"); + } + + @Test + public void replacingTrinoAndSparkViewShouldFail() { + String viewName = viewName("trinoAndSparkView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino, spark]\n" + + "New dialects: [spark]"); + } + + @Test + public void replacingViewWithDialectDropAllowed() { + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + // allowing to drop the trino dialect should replace the view + sql( + "CREATE OR REPLACE VIEW %s TBLPROPERTIES ('%s'='true') AS SELECT id FROM %s", + viewName, ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, tableName); + + View view = viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.currentVersion().representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); + + // trino view should show up in the view versions & history + assertThat(view.history()).hasSize(2); + assertThat(view.history()).element(0).extracting(ViewHistoryEntry::versionId).isEqualTo(1); + assertThat(view.history()).element(1).extracting(ViewHistoryEntry::versionId).isEqualTo(2); + + assertThat(view.versions()).hasSize(2); + assertThat(view.versions()).element(0).extracting(ViewVersion::versionId).isEqualTo(1); + assertThat(view.versions()).element(1).extracting(ViewVersion::versionId).isEqualTo(2); + + assertThat(Lists.newArrayList(view.versions()).get(0).representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("trino").sql(sql).build()); + + assertThat(Lists.newArrayList(view.versions()).get(1).representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); + } + private void insertRows(int numRows) throws NoSuchTableException { List records = Lists.newArrayListWithCapacity(numRows); for (int i = 1; i <= numRows; i++) { 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 ee23d4a766c1..267628cd06e6 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 @@ -39,12 +39,18 @@ import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.ImmutableSQLViewRepresentation; +import org.apache.iceberg.view.SQLViewRepresentation; import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewHistoryEntry; +import org.apache.iceberg.view.ViewProperties; +import org.apache.iceberg.view.ViewVersion; 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.NoSuchTableException; import org.apache.spark.sql.catalyst.catalog.SessionCatalog; +import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -1604,6 +1610,102 @@ public void createOrReplaceViewKeepsViewHistory() { .asStruct()); } + @Test + public void replacingTrinoViewShouldFail() { + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino]\n" + + "New dialects: [spark]"); + } + + @Test + public void replacingTrinoAndSparkViewShouldFail() { + String viewName = viewName("trinoAndSparkView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot replace view due to loss of view dialects (replace.drop-dialect.allowed=false):\n" + + "Previous dialects: [trino, spark]\n" + + "New dialects: [spark]"); + } + + @Test + public void replacingViewWithDialectDropAllowed() { + String viewName = viewName("trinoView"); + String sql = String.format("SELECT id FROM %s", tableName); + + ViewCatalog viewCatalog = viewCatalog(); + + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("trino", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema(sql)) + .create(); + + // allowing to drop the trino dialect should replace the view + sql( + "CREATE OR REPLACE VIEW %s TBLPROPERTIES ('%s'='true') AS SELECT id FROM %s", + viewName, ViewProperties.REPLACE_DROP_DIALECT_ALLOWED, tableName); + + View view = viewCatalog.loadView(TableIdentifier.of(NAMESPACE, viewName)); + assertThat(view.currentVersion().representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); + + // trino view should show up in the view versions & history + assertThat(view.history()).hasSize(2); + assertThat(view.history()).element(0).extracting(ViewHistoryEntry::versionId).isEqualTo(1); + assertThat(view.history()).element(1).extracting(ViewHistoryEntry::versionId).isEqualTo(2); + + assertThat(view.versions()).hasSize(2); + assertThat(view.versions()).element(0).extracting(ViewVersion::versionId).isEqualTo(1); + assertThat(view.versions()).element(1).extracting(ViewVersion::versionId).isEqualTo(2); + + assertThat(Lists.newArrayList(view.versions()).get(0).representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("trino").sql(sql).build()); + + assertThat(Lists.newArrayList(view.versions()).get(1).representations()) + .hasSize(1) + .first() + .asInstanceOf(InstanceOfAssertFactories.type(SQLViewRepresentation.class)) + .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); + } + private void insertRows(int numRows) throws NoSuchTableException { List records = Lists.newArrayListWithCapacity(numRows); for (int i = 1; i <= numRows; i++) { From 8e23c726e4d5027f86155697a2026b08a51a2ab6 Mon Sep 17 00:00:00 2001 From: Drew Gallardo Date: Fri, 16 Feb 2024 03:46:22 -0800 Subject: [PATCH 0044/1019] Core: Make InMemoryFileIO map shared across instances (#9722) --- .../iceberg/inmemory/InMemoryFileIO.java | 10 ++++----- .../iceberg/inmemory/TestInMemoryFileIO.java | 21 ++++++++++++++++++- .../apache/iceberg/rest/TestRESTCatalog.java | 7 ++++++- 3 files changed, 31 insertions(+), 7 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryFileIO.java b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryFileIO.java index 056929a20111..19208a1e0ce3 100644 --- a/core/src/main/java/org/apache/iceberg/inmemory/InMemoryFileIO.java +++ b/core/src/main/java/org/apache/iceberg/inmemory/InMemoryFileIO.java @@ -28,22 +28,22 @@ public class InMemoryFileIO implements FileIO { - private final Map inMemoryFiles = Maps.newConcurrentMap(); + private static final Map IN_MEMORY_FILES = Maps.newConcurrentMap(); private boolean closed = false; public void addFile(String location, byte[] contents) { Preconditions.checkState(!closed, "Cannot call addFile after calling close()"); - inMemoryFiles.put(location, contents); + IN_MEMORY_FILES.put(location, contents); } public boolean fileExists(String location) { - return inMemoryFiles.containsKey(location); + return IN_MEMORY_FILES.containsKey(location); } @Override public InputFile newInputFile(String location) { Preconditions.checkState(!closed, "Cannot call newInputFile after calling close()"); - byte[] contents = inMemoryFiles.get(location); + byte[] contents = IN_MEMORY_FILES.get(location); if (null == contents) { throw new NotFoundException("No in-memory file found for location: %s", location); } @@ -59,7 +59,7 @@ public OutputFile newOutputFile(String location) { @Override public void deleteFile(String location) { Preconditions.checkState(!closed, "Cannot call deleteFile after calling close()"); - if (null == inMemoryFiles.remove(location)) { + if (null == IN_MEMORY_FILES.remove(location)) { throw new NotFoundException("No in-memory file found for location: %s", location); } } diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java index 5f9ec3fbf274..12f5bf84ccba 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java @@ -21,17 +21,18 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.util.UUID; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NotFoundException; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestInMemoryFileIO { - String location = "s3://foo/bar.txt"; @Test public void testBasicEndToEnd() throws IOException { InMemoryFileIO fileIO = new InMemoryFileIO(); + String location = randomLocation(); Assertions.assertThat(fileIO.fileExists(location)).isFalse(); OutputStream outputStream = fileIO.newOutputFile(location).create(); @@ -66,6 +67,7 @@ public void testDeleteFileNotFound() { @Test public void testCreateNoOverwrite() { + String location = randomLocation(); InMemoryFileIO fileIO = new InMemoryFileIO(); fileIO.addFile(location, "hello world".getBytes()); Assertions.assertThatExceptionOfType(AlreadyExistsException.class) @@ -74,6 +76,7 @@ public void testCreateNoOverwrite() { @Test public void testOverwriteBeforeAndAfterClose() throws IOException { + String location = randomLocation(); byte[] oldData = "old data".getBytes(); byte[] newData = "new data".getBytes(); @@ -108,4 +111,20 @@ public void testOverwriteBeforeAndAfterClose() throws IOException { inputStream.close(); Assertions.assertThat(new String(buf)).isEqualTo("new data"); } + + @Test + public void testFilesAreSharedAcrossMultipleInstances() { + String location = randomLocation(); + InMemoryFileIO fileIO = new InMemoryFileIO(); + fileIO.addFile(location, "hello world".getBytes()); + + InMemoryFileIO fileIO2 = new InMemoryFileIO(); + Assertions.assertThat(fileIO2.fileExists(location)) + .isTrue() + .as("Files should be shared across all InMemoryFileIO instances"); + } + + private String randomLocation() { + return "s3://foo/" + UUID.randomUUID(); + } } 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 f196787d5636..34b3e2222235 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -180,7 +180,12 @@ public T execute( restCatalog.initialize( "prod", ImmutableMap.of( - CatalogProperties.URI, httpServer.getURI().toString(), "credential", "catalog:12345")); + CatalogProperties.URI, + httpServer.getURI().toString(), + CatalogProperties.FILE_IO_IMPL, + "org.apache.iceberg.inmemory.InMemoryFileIO", + "credential", + "catalog:12345")); } @SuppressWarnings("unchecked") From f19cd6bcfd4a155a8cb80ba359449132e1b844ff Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 16 Feb 2024 18:40:01 +0100 Subject: [PATCH 0045/1019] Core: Properly suppress historical snapshots when building TableMetadata with suppressHistoricalSnapshots() (#9234) While using ``` TableMetadata.buildFrom(originalResponse.tableMetadata()) .suppressHistoricalSnapshots() ``` already suppresses historical snapshots correctly, we still end up with all snapshots when calling `build()`. This is because `build()` exists early when there are no changes in https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/TableMetadata.java#L1391-L1393: ``` if (!hasChanges()) { return base; } ``` This then causes to return all snapshots from `base` metadata. --- .../org/apache/iceberg/TableMetadata.java | 6 +++- .../apache/iceberg/rest/TestRESTCatalog.java | 35 +++++++++++++++++++ 2 files changed, 40 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 b9061a3107ac..8e687950a6f2 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -886,6 +886,7 @@ public static class Builder { private final Map refs; private final Map> statisticsFiles; private final Map> partitionStatisticsFiles; + private boolean suppressHistoricalSnapshots = false; // change tracking private final List changes; @@ -1288,6 +1289,7 @@ public Builder removeStatistics(long snapshotId) { * @return this for method chaining */ public Builder suppressHistoricalSnapshots() { + this.suppressHistoricalSnapshots = true; Set refSnapshotIds = refs.values().stream().map(SnapshotRef::snapshotId).collect(Collectors.toSet()); Set suppressedSnapshotIds = Sets.difference(snapshotsById.keySet(), refSnapshotIds); @@ -1408,7 +1410,9 @@ public Builder setPreviousFileLocation(String previousFileLocation) { private boolean hasChanges() { return changes.size() != startingChangeCount || (discardChanges && !changes.isEmpty()) - || metadataLocation != null; + || metadataLocation != null + || suppressHistoricalSnapshots + || null != snapshotsSupplier; } public TableMetadata build() { 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 34b3e2222235..0e260a4e53d5 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -47,6 +47,7 @@ import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.Transaction; @@ -81,6 +82,7 @@ import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; +import org.assertj.core.api.InstanceOfAssertFactories; import org.awaitility.Awaitility; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.gzip.GzipHandler; @@ -820,6 +822,13 @@ public void testTableSnapshotLoading() { .suppressHistoricalSnapshots() .build(); + // don't call snapshots() directly as that would cause to load all snapshots. Instead, + // make sure the snapshots field holds exactly 1 snapshot + Assertions.assertThat(refsMetadata) + .extracting("snapshots") + .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class)) + .hasSize(1); + return LoadTableResponse.builder() .withTableMetadata(refsMetadata) .addAllConfig(originalResponse.config()) @@ -917,6 +926,17 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { .toBranch(branch) .commit(); + table + .newFastAppend() + .appendFile( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(10) + .withRecordCount(2) + .build()) + .toBranch(branch) + .commit(); + ResourcePaths paths = ResourcePaths.forCatalogProperties(Maps.newHashMap()); // Respond with only referenced snapshots @@ -928,6 +948,14 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { .suppressHistoricalSnapshots() .build(); + // don't call snapshots() directly as that would cause to load all snapshots. Instead, + // make sure the snapshots field holds exactly 2 snapshots (the latest snapshot for main + // and the branch) + Assertions.assertThat(refsMetadata) + .extracting("snapshots") + .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class)) + .hasSize(2); + return LoadTableResponse.builder() .withTableMetadata(refsMetadata) .addAllConfig(originalResponse.config()) @@ -1037,6 +1065,13 @@ public void lazySnapshotLoadingWithDivergedHistory() { .suppressHistoricalSnapshots() .build(); + // don't call snapshots() directly as that would cause to load all snapshots. Instead, + // make sure the snapshots field holds exactly 1 snapshot + Assertions.assertThat(refsMetadata) + .extracting("snapshots") + .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class)) + .hasSize(1); + return LoadTableResponse.builder() .withTableMetadata(refsMetadata) .addAllConfig(originalResponse.config()) From a33fab9869c0a8c7f4eb6b5d48f6bb66ef292b53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Sat, 17 Feb 2024 01:58:25 +0100 Subject: [PATCH 0046/1019] Core: Add view support on the JDBC catalog (#9487) --- .../org/apache/iceberg/jdbc/JdbcCatalog.java | 178 ++++++- .../iceberg/jdbc/JdbcTableOperations.java | 90 ++-- .../org/apache/iceberg/jdbc/JdbcUtil.java | 495 ++++++++++++++++-- .../iceberg/jdbc/JdbcViewOperations.java | 206 ++++++++ .../apache/iceberg/jdbc/TestJdbcCatalog.java | 152 ++++++ .../org/apache/iceberg/jdbc/TestJdbcUtil.java | 5 +- .../iceberg/jdbc/TestJdbcViewCatalog.java | 66 +++ 7 files changed, 1069 insertions(+), 123 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java create mode 100644 core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 0bab6ade4c85..8d0a93b727f5 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -37,7 +38,6 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.TableMetadata; @@ -49,6 +49,7 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; @@ -60,16 +61,21 @@ 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.LocationUtil; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.BaseMetastoreViewCatalog; +import org.apache.iceberg.view.ViewOperations; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class JdbcCatalog extends BaseMetastoreCatalog +public class JdbcCatalog extends BaseMetastoreViewCatalog implements Configurable, SupportsNamespaces { public static final String PROPERTY_PREFIX = "jdbc."; private static final String NAMESPACE_EXISTS_PROPERTY = "exists"; private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class); private static final Joiner SLASH = Joiner.on("/"); + static final String VIEW_WARNING_LOG_MESSAGE = + "JDBC catalog is initialized without view support. To auto-migrate the database's schema and enable view support, set jdbc.add-view-support=true"; private FileIO io; private String catalogName = "jdbc"; @@ -81,6 +87,7 @@ public class JdbcCatalog extends BaseMetastoreCatalog private final Function, JdbcClientPool> clientPoolBuilder; private final boolean initializeCatalogTables; private CloseableGroup closeableGroup; + private JdbcUtil.SchemaVersion schemaVersion = JdbcUtil.SchemaVersion.V1; public JdbcCatalog() { this(null, null, true); @@ -158,14 +165,17 @@ private void initializeCatalogTables() throws InterruptedException, SQLException dbMeta.getTables( null /* catalog name */, null /* schemaPattern */, - JdbcUtil.CATALOG_TABLE_NAME /* tableNamePattern */, + JdbcUtil.CATALOG_TABLE_VIEW_NAME /* tableNamePattern */, null /* types */); if (tableExists.next()) { + updateCatalogTables(conn); return true; } - LOG.debug("Creating table {} to store iceberg catalog", JdbcUtil.CATALOG_TABLE_NAME); - return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_TABLE).execute(); + LOG.debug( + "Creating table {} to store iceberg catalog tables", + JdbcUtil.CATALOG_TABLE_VIEW_NAME); + return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_SQL).execute(); }); connections.run( @@ -185,14 +195,39 @@ private void initializeCatalogTables() throws InterruptedException, SQLException LOG.debug( "Creating table {} to store iceberg catalog namespace properties", JdbcUtil.NAMESPACE_PROPERTIES_TABLE_NAME); - return conn.prepareStatement(JdbcUtil.CREATE_NAMESPACE_PROPERTIES_TABLE).execute(); + return conn.prepareStatement(JdbcUtil.CREATE_NAMESPACE_PROPERTIES_TABLE_SQL).execute(); }); } + private void updateCatalogTables(Connection connection) throws SQLException { + DatabaseMetaData dbMeta = connection.getMetaData(); + ResultSet typeColumn = + dbMeta.getColumns(null, null, JdbcUtil.CATALOG_TABLE_VIEW_NAME, JdbcUtil.RECORD_TYPE); + if (typeColumn.next()) { + LOG.debug("{} already supports views", JdbcUtil.CATALOG_TABLE_VIEW_NAME); + } else { + if (PropertyUtil.propertyAsBoolean( + catalogProperties, JdbcUtil.ADD_VIEW_SUPPORT_PROPERTY, false)) { + connection.prepareStatement(JdbcUtil.UPDATE_CATALOG_SQL).execute(); + } else { + LOG.warn(VIEW_WARNING_LOG_MESSAGE); + schemaVersion = JdbcUtil.SchemaVersion.V0; + } + } + } + @Override protected TableOperations newTableOps(TableIdentifier tableIdentifier) { return new JdbcTableOperations( - connections, io, catalogName, tableIdentifier, catalogProperties); + connections, io, catalogName, tableIdentifier, catalogProperties, schemaVersion); + } + + @Override + protected ViewOperations newViewOps(TableIdentifier viewIdentifier) { + if (schemaVersion != JdbcUtil.SchemaVersion.V1) { + throw new UnsupportedOperationException(VIEW_WARNING_LOG_MESSAGE); + } + return new JdbcViewOperations(connections, io, catalogName, viewIdentifier, catalogProperties); } @Override @@ -217,7 +252,9 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { int deletedRecords = execute( - JdbcUtil.DROP_TABLE_SQL, + (schemaVersion == JdbcUtil.SchemaVersion.V1) + ? JdbcUtil.V1_DROP_TABLE_SQL + : JdbcUtil.V0_DROP_TABLE_SQL, catalogName, JdbcUtil.namespaceToString(identifier.namespace()), identifier.name()); @@ -245,13 +282,35 @@ public List listTables(Namespace namespace) { row -> JdbcUtil.stringToTableIdentifier( row.getString(JdbcUtil.TABLE_NAMESPACE), row.getString(JdbcUtil.TABLE_NAME)), - JdbcUtil.LIST_TABLES_SQL, + (schemaVersion == JdbcUtil.SchemaVersion.V1) + ? JdbcUtil.V1_LIST_TABLE_SQL + : JdbcUtil.V0_LIST_TABLE_SQL, catalogName, JdbcUtil.namespaceToString(namespace)); } @Override public void renameTable(TableIdentifier from, TableIdentifier to) { + if (from.equals(to)) { + return; + } + + if (!tableExists(from)) { + throw new NoSuchTableException("Table does not exist: %s", from); + } + + if (!namespaceExists(to.namespace())) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", to.namespace()); + } + + if (viewExists(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); + } + + if (tableExists(to)) { + throw new AlreadyExistsException("Table already exists: %s", to); + } + int updatedRecords = execute( err -> { @@ -261,7 +320,9 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { throw new AlreadyExistsException("Table already exists: %s", to); } }, - JdbcUtil.RENAME_TABLE_SQL, + (schemaVersion == JdbcUtil.SchemaVersion.V1) + ? JdbcUtil.V1_RENAME_TABLE_SQL + : JdbcUtil.V0_RENAME_TABLE_SQL, JdbcUtil.namespaceToString(to.namespace()), to.name(), catalogName, @@ -315,7 +376,7 @@ public List listNamespaces() { namespaces.addAll( fetch( row -> JdbcUtil.stringToNamespace(row.getString(JdbcUtil.TABLE_NAMESPACE)), - JdbcUtil.LIST_ALL_TABLE_NAMESPACES_SQL, + JdbcUtil.LIST_ALL_NAMESPACES_SQL, catalogName)); namespaces.addAll( fetch( @@ -503,6 +564,101 @@ public boolean namespaceExists(Namespace namespace) { return JdbcUtil.namespaceExists(catalogName, connections, namespace); } + @Override + public boolean dropView(TableIdentifier identifier) { + if (schemaVersion != JdbcUtil.SchemaVersion.V1) { + throw new UnsupportedOperationException(VIEW_WARNING_LOG_MESSAGE); + } + + int deletedRecords = + execute( + JdbcUtil.DROP_VIEW_SQL, + catalogName, + JdbcUtil.namespaceToString(identifier.namespace()), + identifier.name()); + + if (deletedRecords == 0) { + LOG.info("Skipping drop, view does not exist: {}", identifier); + return false; + } + + LOG.info("Dropped view: {}", identifier); + return true; + } + + @Override + public List listViews(Namespace namespace) { + if (schemaVersion != JdbcUtil.SchemaVersion.V1) { + throw new UnsupportedOperationException(VIEW_WARNING_LOG_MESSAGE); + } + + if (!namespaceExists(namespace)) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + } + + return fetch( + row -> + JdbcUtil.stringToTableIdentifier( + row.getString(JdbcUtil.TABLE_NAMESPACE), row.getString(JdbcUtil.TABLE_NAME)), + JdbcUtil.LIST_VIEW_SQL, + catalogName, + JdbcUtil.namespaceToString(namespace)); + } + + @Override + public void renameView(TableIdentifier from, TableIdentifier to) { + if (schemaVersion != JdbcUtil.SchemaVersion.V1) { + throw new UnsupportedOperationException(VIEW_WARNING_LOG_MESSAGE); + } + + if (from.equals(to)) { + return; + } + + if (!namespaceExists(to.namespace())) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", to.namespace()); + } + + if (!viewExists(from)) { + throw new NoSuchViewException("View does not exist"); + } + + if (tableExists(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. Table already exists", from, to); + } + + if (viewExists(to)) { + throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); + } + + int updatedRecords = + execute( + err -> { + // SQLite doesn't set SQLState or throw SQLIntegrityConstraintViolationException + if (err instanceof SQLIntegrityConstraintViolationException + || (err.getMessage() != null && err.getMessage().contains("constraint failed"))) { + throw new AlreadyExistsException( + "Cannot rename %s to %s. View already exists", from, to); + } + }, + JdbcUtil.RENAME_VIEW_SQL, + JdbcUtil.namespaceToString(to.namespace()), + to.name(), + catalogName, + JdbcUtil.namespaceToString(from.namespace()), + from.name()); + + if (updatedRecords == 1) { + LOG.info("Renamed view from {}, to {}", from, to); + } else if (updatedRecords == 0) { + throw new NoSuchViewException("View does not exist: %s", from); + } else { + LOG.warn( + "Rename operation affected {} rows: the catalog view's primary key assumption has been violated", + updatedRecords); + } + } + private int execute(String sql, String... args) { return execute(err -> {}, sql, args); } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java index 6a7d594dd9f6..68d75b8e4f5f 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java @@ -19,8 +19,6 @@ package org.apache.iceberg.jdbc; import java.sql.DataTruncation; -import java.sql.PreparedStatement; -import java.sql.ResultSet; import java.sql.SQLException; import java.sql.SQLIntegrityConstraintViolationException; import java.sql.SQLNonTransientConnectionException; @@ -39,7 +37,6 @@ import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.io.FileIO; 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.PropertyUtil; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -52,18 +49,21 @@ class JdbcTableOperations extends BaseMetastoreTableOperations { private final FileIO fileIO; private final JdbcClientPool connections; private final Map catalogProperties; + private final JdbcUtil.SchemaVersion schemaVersion; protected JdbcTableOperations( JdbcClientPool dbConnPool, FileIO fileIO, String catalogName, TableIdentifier tableIdentifier, - Map catalogProperties) { + Map catalogProperties, + JdbcUtil.SchemaVersion schemaVersion) { this.catalogName = catalogName; this.tableIdentifier = tableIdentifier; this.fileIO = fileIO; this.connections = dbConnPool; this.catalogProperties = catalogProperties; + this.schemaVersion = schemaVersion; } @Override @@ -71,7 +71,7 @@ public void doRefresh() { Map table; try { - table = getTable(); + table = JdbcUtil.loadTable(schemaVersion, connections, catalogName, tableIdentifier); } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new UncheckedInterruptedException(e, "Interrupted during refresh"); @@ -105,7 +105,8 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { boolean newTable = base == null; String newMetadataLocation = writeNewMetadataIfRequired(newTable, metadata); try { - Map table = getTable(); + Map table = + JdbcUtil.loadTable(schemaVersion, connections, catalogName, tableIdentifier); if (base != null) { validateMetadataLocation(table, base); @@ -140,6 +141,7 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { if (e.getMessage().contains("constraint failed")) { throw new AlreadyExistsException("Table already exists: %s", tableIdentifier); } + throw new UncheckedSQLException(e, "Unknown failure"); } catch (InterruptedException e) { Thread.currentThread().interrupt(); @@ -150,20 +152,13 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { private void updateTable(String newMetadataLocation, String oldMetadataLocation) throws SQLException, InterruptedException { int updatedRecords = - connections.run( - conn -> { - try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.DO_COMMIT_SQL)) { - // UPDATE - sql.setString(1, newMetadataLocation); - sql.setString(2, oldMetadataLocation); - // WHERE - sql.setString(3, catalogName); - sql.setString(4, JdbcUtil.namespaceToString(tableIdentifier.namespace())); - sql.setString(5, tableIdentifier.name()); - sql.setString(6, oldMetadataLocation); - return sql.executeUpdate(); - } - }); + JdbcUtil.updateTable( + schemaVersion, + connections, + catalogName, + tableIdentifier, + newMetadataLocation, + oldMetadataLocation); if (updatedRecords == 1) { LOG.debug("Successfully committed to existing table: {}", tableIdentifier); @@ -182,18 +177,23 @@ private void createTable(String newMetadataLocation) throws SQLException, Interr tableIdentifier, catalogName, namespace); } + if (schemaVersion == JdbcUtil.SchemaVersion.V1 + && JdbcUtil.viewExists(catalogName, connections, tableIdentifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", tableIdentifier); + } + + if (JdbcUtil.tableExists(schemaVersion, catalogName, connections, tableIdentifier)) { + throw new AlreadyExistsException("Table already exists: %s", tableIdentifier); + } + int insertRecord = - connections.run( - conn -> { - try (PreparedStatement sql = - conn.prepareStatement(JdbcUtil.DO_COMMIT_CREATE_TABLE_SQL)) { - sql.setString(1, catalogName); - sql.setString(2, JdbcUtil.namespaceToString(namespace)); - sql.setString(3, tableIdentifier.name()); - sql.setString(4, newMetadataLocation); - return sql.executeUpdate(); - } - }); + JdbcUtil.doCommitCreateTable( + schemaVersion, + connections, + catalogName, + namespace, + tableIdentifier, + newMetadataLocation); if (insertRecord == 1) { LOG.debug("Successfully committed to new table: {}", tableIdentifier); @@ -223,32 +223,4 @@ public FileIO io() { protected String tableName() { return tableIdentifier.toString(); } - - private Map getTable() - throws UncheckedSQLException, SQLException, InterruptedException { - return connections.run( - conn -> { - Map table = Maps.newHashMap(); - - try (PreparedStatement sql = conn.prepareStatement(JdbcUtil.GET_TABLE_SQL)) { - sql.setString(1, catalogName); - sql.setString(2, JdbcUtil.namespaceToString(tableIdentifier.namespace())); - sql.setString(3, tableIdentifier.name()); - ResultSet rs = sql.executeQuery(); - - if (rs.next()) { - table.put(JdbcUtil.CATALOG_NAME, rs.getString(JdbcUtil.CATALOG_NAME)); - table.put(JdbcUtil.TABLE_NAMESPACE, rs.getString(JdbcUtil.TABLE_NAMESPACE)); - table.put(JdbcUtil.TABLE_NAME, rs.getString(JdbcUtil.TABLE_NAME)); - table.put(METADATA_LOCATION_PROP, rs.getString(METADATA_LOCATION_PROP)); - table.put( - PREVIOUS_METADATA_LOCATION_PROP, rs.getString(PREVIOUS_METADATA_LOCATION_PROP)); - } - - rs.close(); - } - - return table; - }); - } } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 4a515d1329ed..20a2f7ea6238 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -25,31 +25,62 @@ import java.util.Map; import java.util.Properties; import java.util.Set; +import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; 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.base.Splitter; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; final class JdbcUtil { // property to control strict-mode (aka check if namespace exists when creating a table) static final String STRICT_MODE_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "strict-mode"; + // property to control if view support is added to the existing database + static final String ADD_VIEW_SUPPORT_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "add-view-support"; - // Catalog Table - static final String CATALOG_TABLE_NAME = "iceberg_tables"; + enum SchemaVersion { + V0, + V1 + } + + // Catalog Table & View + static final String CATALOG_TABLE_VIEW_NAME = "iceberg_tables"; static final String CATALOG_NAME = "catalog_name"; - static final String TABLE_NAMESPACE = "table_namespace"; static final String TABLE_NAME = "table_name"; + static final String TABLE_NAMESPACE = "table_namespace"; + static final String RECORD_TYPE = "iceberg_type"; + static final String TABLE_RECORD_TYPE = "TABLE"; + static final String VIEW_RECORD_TYPE = "VIEW"; - static final String DO_COMMIT_SQL = + private static final String V1_DO_COMMIT_SQL = + "UPDATE " + + CATALOG_TABLE_VIEW_NAME + + " SET " + + JdbcTableOperations.METADATA_LOCATION_PROP + + " = ? , " + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + + " = ?" + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + + " = ? AND " + + JdbcTableOperations.METADATA_LOCATION_PROP + + " = ? AND " + + RECORD_TYPE + + " = ?"; + private static final String V0_DO_COMMIT_SQL = "UPDATE " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " SET " + JdbcTableOperations.METADATA_LOCATION_PROP + " = ? , " + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP - + " = ? " + + " = ?" + " WHERE " + CATALOG_NAME + " = ? AND " @@ -59,9 +90,9 @@ final class JdbcUtil { + " = ? AND " + JdbcTableOperations.METADATA_LOCATION_PROP + " = ?"; - static final String CREATE_CATALOG_TABLE = + static final String CREATE_CATALOG_SQL = "CREATE TABLE " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + "(" + CATALOG_NAME + " VARCHAR(255) NOT NULL," @@ -73,6 +104,8 @@ final class JdbcUtil { + " VARCHAR(1000)," + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + " VARCHAR(1000)," + + RECORD_TYPE + + " VARCHAR(100)," + "PRIMARY KEY (" + CATALOG_NAME + ", " @@ -81,85 +114,242 @@ final class JdbcUtil { + TABLE_NAME + ")" + ")"; - static final String GET_TABLE_SQL = + static final String UPDATE_CATALOG_SQL = + "ALTER TABLE " + CATALOG_TABLE_VIEW_NAME + " ADD COLUMN " + RECORD_TYPE + " VARCHAR(5)"; + + private static final String GET_VIEW_SQL = "SELECT * FROM " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " = ? AND " + TABLE_NAME - + " = ? "; - static final String LIST_TABLES_SQL = + + " = ? AND " + + RECORD_TYPE + + " = " + + "'" + + VIEW_RECORD_TYPE + + "'"; + private static final String V1_GET_TABLE_SQL = + "SELECT * FROM " + + CATALOG_TABLE_VIEW_NAME + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + + " = ? AND (" + + RECORD_TYPE + + " = " + + "'" + + TABLE_RECORD_TYPE + + "'" + + " OR " + + RECORD_TYPE + + " IS NULL)"; + private static final String V0_GET_TABLE_SQL = "SELECT * FROM " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + " = ?"; - static final String RENAME_TABLE_SQL = + static final String LIST_VIEW_SQL = + "SELECT * FROM " + + CATALOG_TABLE_VIEW_NAME + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + RECORD_TYPE + + " = " + + "'" + + VIEW_RECORD_TYPE + + "'"; + static final String V1_LIST_TABLE_SQL = + "SELECT * FROM " + + CATALOG_TABLE_VIEW_NAME + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND (" + + RECORD_TYPE + + " = " + + "'" + + TABLE_RECORD_TYPE + + "'" + + " OR " + + RECORD_TYPE + + " IS NULL)"; + static final String V0_LIST_TABLE_SQL = + "SELECT * FROM " + + CATALOG_TABLE_VIEW_NAME + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ?"; + static final String RENAME_VIEW_SQL = "UPDATE " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " SET " + TABLE_NAMESPACE - + " = ? , " + + " = ?, " + TABLE_NAME - + " = ? " + + " = ?" + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " = ? AND " + TABLE_NAME - + " = ? "; - static final String DROP_TABLE_SQL = + + " = ? AND " + + RECORD_TYPE + + " = " + + "'" + + VIEW_RECORD_TYPE + + "'"; + static final String V1_RENAME_TABLE_SQL = + "UPDATE " + + CATALOG_TABLE_VIEW_NAME + + " SET " + + TABLE_NAMESPACE + + " = ?, " + + TABLE_NAME + + " = ?" + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + + " = ? AND (" + + RECORD_TYPE + + " = " + + "'" + + TABLE_RECORD_TYPE + + "'" + + " OR " + + RECORD_TYPE + + " IS NULL)"; + static final String V0_RENAME_TABLE_SQL = + "UPDATE " + + CATALOG_TABLE_VIEW_NAME + + " SET " + + TABLE_NAMESPACE + + " = ?, " + + TABLE_NAME + + " = ?" + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + + " = ?"; + static final String DROP_VIEW_SQL = "DELETE FROM " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + " = ? AND " + + RECORD_TYPE + + " = " + + "'" + + VIEW_RECORD_TYPE + + "'"; + static final String V1_DROP_TABLE_SQL = + "DELETE FROM " + + CATALOG_TABLE_VIEW_NAME + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + TABLE_NAME - + " = ? "; - static final String GET_NAMESPACE_SQL = + + " = ? AND (" + + RECORD_TYPE + + " = " + + "'" + + TABLE_RECORD_TYPE + + "'" + + " OR " + + RECORD_TYPE + + " IS NULL)"; + static final String V0_DROP_TABLE_SQL = + "DELETE FROM " + + CATALOG_TABLE_VIEW_NAME + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + + " = ?"; + private static final String GET_NAMESPACE_SQL = "SELECT " + TABLE_NAMESPACE + " FROM " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " WHERE " + CATALOG_NAME + " = ? AND " - + " ( " + + " (" + TABLE_NAMESPACE + " = ? OR " + TABLE_NAMESPACE - + " LIKE ? ESCAPE '\\' " - + " ) " + + " LIKE ? ESCAPE '\\')" + " LIMIT 1"; static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT " + TABLE_NAMESPACE + " FROM " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " WHERE " + CATALOG_NAME + " = ? AND " + TABLE_NAMESPACE + " LIKE ?"; - static final String LIST_ALL_TABLE_NAMESPACES_SQL = + static final String LIST_ALL_NAMESPACES_SQL = "SELECT DISTINCT " + TABLE_NAMESPACE + " FROM " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " WHERE " + CATALOG_NAME + " = ?"; - static final String DO_COMMIT_CREATE_TABLE_SQL = + private static final String V1_DO_COMMIT_CREATE_SQL = + "INSERT INTO " + + CATALOG_TABLE_VIEW_NAME + + " (" + + CATALOG_NAME + + ", " + + TABLE_NAMESPACE + + ", " + + TABLE_NAME + + ", " + + JdbcTableOperations.METADATA_LOCATION_PROP + + ", " + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + + ", " + + RECORD_TYPE + + ") " + + " VALUES (?,?,?,?,null,?)"; + private static final String V0_DO_COMMIT_CREATE_SQL = "INSERT INTO " - + CATALOG_TABLE_NAME + + CATALOG_TABLE_VIEW_NAME + " (" + CATALOG_NAME + ", " @@ -179,7 +369,7 @@ final class JdbcUtil { static final String NAMESPACE_PROPERTY_KEY = "property_key"; static final String NAMESPACE_PROPERTY_VALUE = "property_value"; - static final String CREATE_NAMESPACE_PROPERTIES_TABLE = + static final String CREATE_NAMESPACE_PROPERTIES_TABLE_SQL = "CREATE TABLE " + NAMESPACE_PROPERTIES_TABLE_NAME + "(" @@ -278,20 +468,20 @@ final class JdbcUtil { private JdbcUtil() {} - public static Namespace stringToNamespace(String namespace) { + static Namespace stringToNamespace(String namespace) { Preconditions.checkArgument(namespace != null, "Invalid namespace %s", namespace); return Namespace.of(Iterables.toArray(SPLITTER_DOT.split(namespace), String.class)); } - public static String namespaceToString(Namespace namespace) { + static String namespaceToString(Namespace namespace) { return JOINER_DOT.join(namespace.levels()); } - public static TableIdentifier stringToTableIdentifier(String tableNamespace, String tableName) { + static TableIdentifier stringToTableIdentifier(String tableNamespace, String tableName) { return TableIdentifier.of(JdbcUtil.stringToNamespace(tableNamespace), tableName); } - public static Properties filterAndRemovePrefix(Map properties, String prefix) { + static Properties filterAndRemovePrefix(Map properties, String prefix) { Properties result = new Properties(); properties.forEach( (key, value) -> { @@ -303,7 +493,218 @@ public static Properties filterAndRemovePrefix(Map properties, S return result; } - public static String updatePropertiesStatement(int size) { + private static int update( + boolean isTable, + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + TableIdentifier identifier, + String newMetadataLocation, + String oldMetadataLocation) + throws SQLException, InterruptedException { + return connections.run( + conn -> { + try (PreparedStatement sql = + conn.prepareStatement( + (schemaVersion == SchemaVersion.V1) ? V1_DO_COMMIT_SQL : V0_DO_COMMIT_SQL)) { + // UPDATE + sql.setString(1, newMetadataLocation); + sql.setString(2, oldMetadataLocation); + // WHERE + sql.setString(3, catalogName); + sql.setString(4, namespaceToString(identifier.namespace())); + sql.setString(5, identifier.name()); + sql.setString(6, oldMetadataLocation); + if (schemaVersion == SchemaVersion.V1) { + sql.setString(7, isTable ? TABLE_RECORD_TYPE : VIEW_RECORD_TYPE); + } + + return sql.executeUpdate(); + } + }); + } + + static int updateTable( + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + TableIdentifier tableIdentifier, + String newMetadataLocation, + String oldMetadataLocation) + throws SQLException, InterruptedException { + return update( + true, + schemaVersion, + connections, + catalogName, + tableIdentifier, + newMetadataLocation, + oldMetadataLocation); + } + + static int updateView( + JdbcClientPool connections, + String catalogName, + TableIdentifier viewIdentifier, + String newMetadataLocation, + String oldMetadataLocation) + throws SQLException, InterruptedException { + return update( + false, + SchemaVersion.V1, + connections, + catalogName, + viewIdentifier, + newMetadataLocation, + oldMetadataLocation); + } + + private static Map tableOrView( + boolean isTable, + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + TableIdentifier identifier) + throws SQLException, InterruptedException { + return connections.run( + conn -> { + Map tableOrView = Maps.newHashMap(); + + try (PreparedStatement sql = + conn.prepareStatement( + isTable + ? ((schemaVersion == SchemaVersion.V1) ? V1_GET_TABLE_SQL : V0_GET_TABLE_SQL) + : GET_VIEW_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, namespaceToString(identifier.namespace())); + sql.setString(3, identifier.name()); + ResultSet rs = sql.executeQuery(); + + if (rs.next()) { + tableOrView.put(CATALOG_NAME, rs.getString(CATALOG_NAME)); + tableOrView.put(TABLE_NAMESPACE, rs.getString(TABLE_NAMESPACE)); + tableOrView.put(TABLE_NAME, rs.getString(TABLE_NAME)); + tableOrView.put( + BaseMetastoreTableOperations.METADATA_LOCATION_PROP, + rs.getString(BaseMetastoreTableOperations.METADATA_LOCATION_PROP)); + tableOrView.put( + BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP, + rs.getString(BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP)); + } + + rs.close(); + } + + return tableOrView; + }); + } + + static Map loadTable( + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + TableIdentifier identifier) + throws SQLException, InterruptedException { + return tableOrView(true, schemaVersion, connections, catalogName, identifier); + } + + static Map loadView( + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + TableIdentifier identifier) + throws SQLException, InterruptedException { + return tableOrView(false, schemaVersion, connections, catalogName, identifier); + } + + private static int doCommitCreate( + boolean isTable, + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + Namespace namespace, + TableIdentifier identifier, + String newMetadataLocation) + throws SQLException, InterruptedException { + return connections.run( + conn -> { + try (PreparedStatement sql = + conn.prepareStatement( + (schemaVersion == SchemaVersion.V1) + ? V1_DO_COMMIT_CREATE_SQL + : V0_DO_COMMIT_CREATE_SQL)) { + sql.setString(1, catalogName); + sql.setString(2, namespaceToString(namespace)); + sql.setString(3, identifier.name()); + sql.setString(4, newMetadataLocation); + if (schemaVersion == SchemaVersion.V1) { + sql.setString(5, isTable ? TABLE_RECORD_TYPE : VIEW_RECORD_TYPE); + } + + return sql.executeUpdate(); + } + }); + } + + static int doCommitCreateTable( + SchemaVersion schemaVersion, + JdbcClientPool connections, + String catalogName, + Namespace namespace, + TableIdentifier tableIdentifier, + String newMetadataLocation) + throws SQLException, InterruptedException { + return doCommitCreate( + true, + schemaVersion, + connections, + catalogName, + namespace, + tableIdentifier, + newMetadataLocation); + } + + static int doCommitCreateView( + JdbcClientPool connections, + String catalogName, + Namespace namespace, + TableIdentifier viewIdentifier, + String newMetadataLocation) + throws SQLException, InterruptedException { + return doCommitCreate( + false, + SchemaVersion.V1, + connections, + catalogName, + namespace, + viewIdentifier, + newMetadataLocation); + } + + static boolean viewExists( + String catalogName, JdbcClientPool connections, TableIdentifier viewIdentifier) { + return exists( + connections, + GET_VIEW_SQL, + catalogName, + namespaceToString(viewIdentifier.namespace()), + viewIdentifier.name()); + } + + static boolean tableExists( + SchemaVersion schemaVersion, + String catalogName, + JdbcClientPool connections, + TableIdentifier tableIdentifier) { + return exists( + connections, + (schemaVersion == SchemaVersion.V1) ? V1_GET_TABLE_SQL : V0_GET_TABLE_SQL, + catalogName, + namespaceToString(tableIdentifier.namespace()), + tableIdentifier.name()); + } + + static String updatePropertiesStatement(int size) { StringBuilder sqlStatement = new StringBuilder( "UPDATE " @@ -314,6 +715,7 @@ public static String updatePropertiesStatement(int size) { for (int i = 0; i < size; i += 1) { sqlStatement.append(" WHEN " + NAMESPACE_PROPERTY_KEY + " = ? THEN ?"); } + sqlStatement.append( " END WHERE " + CATALOG_NAME @@ -329,7 +731,7 @@ public static String updatePropertiesStatement(int size) { return sqlStatement.toString(); } - public static String insertPropertiesStatement(int size) { + static String insertPropertiesStatement(int size) { StringBuilder sqlStatement = new StringBuilder(JdbcUtil.INSERT_NAMESPACE_PROPERTIES_SQL); for (int i = 0; i < size; i++) { @@ -342,7 +744,7 @@ public static String insertPropertiesStatement(int size) { return sqlStatement.toString(); } - public static String deletePropertiesStatement(Set properties) { + static String deletePropertiesStatement(Set properties) { StringBuilder sqlStatement = new StringBuilder(JdbcUtil.DELETE_NAMESPACE_PROPERTIES_SQL); String values = String.join(",", Collections.nCopies(properties.size(), String.valueOf('?'))); sqlStatement.append("(").append(values).append(")"); @@ -358,25 +760,16 @@ static boolean namespaceExists( // catalog.db can exists as: catalog.db.ns1 or catalog.db.ns1.ns2 String namespaceStartsWith = namespaceEquals.replace("\\", "\\\\").replace("_", "\\_").replace("%", "\\%") + ".%"; - if (exists( - connections, - JdbcUtil.GET_NAMESPACE_SQL, - catalogName, - namespaceEquals, - namespaceStartsWith)) { + if (exists(connections, GET_NAMESPACE_SQL, catalogName, namespaceEquals, namespaceStartsWith)) { return true; } - if (exists( + return exists( connections, JdbcUtil.GET_NAMESPACE_PROPERTIES_SQL, catalogName, namespaceEquals, - namespaceStartsWith)) { - return true; - } - - return false; + namespaceStartsWith); } @SuppressWarnings("checkstyle:NestedTryDepth") diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java new file mode 100644 index 000000000000..2ded12b102ae --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.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.jdbc; + +import java.sql.DataTruncation; +import java.sql.SQLException; +import java.sql.SQLIntegrityConstraintViolationException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.sql.SQLWarning; +import java.util.Map; +import java.util.Objects; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.BaseViewOperations; +import org.apache.iceberg.view.ViewMetadata; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** JDBC implementation of Iceberg ViewOperations. */ +public class JdbcViewOperations extends BaseViewOperations { + + private static final Logger LOG = LoggerFactory.getLogger(JdbcViewOperations.class); + private final String catalogName; + private final TableIdentifier viewIdentifier; + private final FileIO fileIO; + private final JdbcClientPool connections; + private final Map catalogProperties; + + protected JdbcViewOperations( + JdbcClientPool dbConnPool, + FileIO fileIO, + String catalogName, + TableIdentifier viewIdentifier, + Map catalogProperties) { + this.catalogName = catalogName; + this.viewIdentifier = viewIdentifier; + this.fileIO = fileIO; + this.connections = dbConnPool; + this.catalogProperties = catalogProperties; + } + + @Override + protected void doRefresh() { + Map view; + + try { + view = JdbcUtil.loadView(JdbcUtil.SchemaVersion.V1, connections, catalogName, viewIdentifier); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during refresh"); + } catch (SQLException e) { + // SQL exception happened when getting view from catalog + throw new UncheckedSQLException( + e, "Failed to get view %s from catalog %s", viewIdentifier, catalogName); + } + + if (view.isEmpty()) { + if (currentMetadataLocation() != null) { + throw new NoSuchViewException("View does not exist: %s", viewIdentifier); + } else { + this.disableRefresh(); + return; + } + } + + String newMetadataLocation = view.get(JdbcTableOperations.METADATA_LOCATION_PROP); + Preconditions.checkState( + newMetadataLocation != null, "Invalid view %s: metadata location is null", viewIdentifier); + refreshFromMetadataLocation(newMetadataLocation); + } + + @Override + protected void doCommit(ViewMetadata base, ViewMetadata metadata) { + String newMetadataLocation = writeNewMetadataIfRequired(metadata); + try { + Map view = + JdbcUtil.loadView(JdbcUtil.SchemaVersion.V1, connections, catalogName, viewIdentifier); + if (base != null) { + validateMetadataLocation(view, base); + String oldMetadataLocation = base.metadataFileLocation(); + // Start atomic update + LOG.debug("Committing existing view: {}", viewName()); + updateView(newMetadataLocation, oldMetadataLocation); + } else { + // view does not exist, create it + LOG.debug("Committing new view: {}", viewName()); + createView(newMetadataLocation); + } + + } catch (SQLIntegrityConstraintViolationException e) { + if (currentMetadataLocation() == null) { + throw new AlreadyExistsException(e, "View already exists: %s", viewIdentifier); + } else { + throw new UncheckedSQLException(e, "View already exists: %s", viewIdentifier); + } + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException(e, "Database Connection timeout"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException(e, "Database Connection failed"); + } catch (DataTruncation e) { + throw new UncheckedSQLException(e, "Database data truncation error"); + } catch (SQLWarning e) { + throw new UncheckedSQLException(e, "Database warning"); + } catch (SQLException e) { + // SQLite doesn't set SQLState or throw SQLIntegrityConstraintViolationException + if (e.getMessage().contains("constraint failed")) { + throw new AlreadyExistsException("View already exists: %s", viewIdentifier); + } + + throw new UncheckedSQLException(e, "Unknown failure"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during commit"); + } + } + + @Override + protected String viewName() { + return viewIdentifier.toString(); + } + + @Override + protected FileIO io() { + return fileIO; + } + + private void validateMetadataLocation(Map view, ViewMetadata base) { + String catalogMetadataLocation = view.get(JdbcTableOperations.METADATA_LOCATION_PROP); + String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; + + if (!Objects.equals(baseMetadataLocation, catalogMetadataLocation)) { + throw new CommitFailedException( + "Cannot commit %s: metadata location %s has changed from %s", + viewIdentifier, baseMetadataLocation, catalogMetadataLocation); + } + } + + private void updateView(String newMetadataLocation, String oldMetadataLocation) + throws SQLException, InterruptedException { + int updatedRecords = + JdbcUtil.updateView( + connections, catalogName, viewIdentifier, newMetadataLocation, oldMetadataLocation); + + if (updatedRecords == 1) { + LOG.debug("Successfully committed to existing view: {}", viewIdentifier); + } else { + throw new CommitFailedException( + "Failed to update view %s from catalog %s", viewIdentifier, catalogName); + } + } + + private void createView(String newMetadataLocation) throws SQLException, InterruptedException { + Namespace namespace = viewIdentifier.namespace(); + if (PropertyUtil.propertyAsBoolean(catalogProperties, JdbcUtil.STRICT_MODE_PROPERTY, false) + && !JdbcUtil.namespaceExists(catalogName, connections, namespace)) { + throw new NoSuchNamespaceException( + "Cannot create view %s in catalog %s. Namespace %s does not exist", + viewIdentifier, catalogName, namespace); + } + + if (JdbcUtil.tableExists(JdbcUtil.SchemaVersion.V1, catalogName, connections, viewIdentifier)) { + throw new AlreadyExistsException("Table with same name already exists: %s", viewIdentifier); + } + + if (JdbcUtil.viewExists(catalogName, connections, viewIdentifier)) { + throw new AlreadyExistsException("View already exists: %s", viewIdentifier); + } + + int insertRecord = + JdbcUtil.doCommitCreateView( + connections, catalogName, namespace, viewIdentifier, newMetadataLocation); + + if (insertRecord == 1) { + LOG.debug("Successfully committed to new view: {}", viewIdentifier); + } else { + throw new CommitFailedException( + "Failed to create view %s in catalog %s", viewIdentifier, catalogName); + } + } +} 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 fb6e6b591c87..3c39ed3c1321 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -29,6 +29,8 @@ import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Paths; +import java.sql.Connection; +import java.sql.SQLException; import java.util.List; import java.util.Map; import java.util.Set; @@ -77,6 +79,7 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.sqlite.SQLiteDataSource; public class TestJdbcCatalog extends CatalogTests { @@ -158,6 +161,91 @@ public void testInitialize() { jdbcCatalog.initialize("test_jdbc_catalog", properties); } + @Test + public void testSchemaIsMigratedToAddViewSupport() throws Exception { + // as this test uses different connections, we can't use memory database (as it's per + // connection), but a file database instead + java.nio.file.Path dbFile = Files.createTempFile("icebergSchemaUpdate", "db"); + String jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); + + initLegacySchema(jdbcUrl); + + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, jdbcUrl); + properties.put(JdbcUtil.ADD_VIEW_SUPPORT_PROPERTY, "true"); + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.setConf(conf); + jdbcCatalog.initialize("TEST", properties); + + TableIdentifier tableOne = TableIdentifier.of("namespace1", "table1"); + TableIdentifier tableTwo = TableIdentifier.of("namespace2", "table2"); + assertThat(jdbcCatalog.listTables(Namespace.of("namespace1"))) + .hasSize(1) + .containsExactly(tableOne); + + assertThat(jdbcCatalog.listTables(Namespace.of("namespace2"))) + .hasSize(1) + .containsExactly(tableTwo); + + assertThat(jdbcCatalog.listViews(Namespace.of("namespace1"))).isEmpty(); + + TableIdentifier view = TableIdentifier.of("namespace1", "view"); + jdbcCatalog + .buildView(view) + .withQuery("spark", "select * from tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(Namespace.of("namespace1")) + .create(); + + assertThat(jdbcCatalog.listViews(Namespace.of("namespace1"))).hasSize(1).containsExactly(view); + } + + @Test + public void testLegacySchemaSupport() throws Exception { + // as this test uses different connection, we can't use memory database (as it's per + // connection), but a + // file database instead + java.nio.file.Path dbFile = Files.createTempFile("icebergOldSchema", "db"); + String jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); + + initLegacySchema(jdbcUrl); + + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, jdbcUrl); + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.setConf(conf); + jdbcCatalog.initialize("TEST", properties); + + TableIdentifier tableOne = TableIdentifier.of("namespace1", "table1"); + TableIdentifier tableTwo = TableIdentifier.of("namespace2", "table2"); + + assertThat(jdbcCatalog.listTables(Namespace.of("namespace1"))) + .hasSize(1) + .containsExactly(tableOne); + + assertThat(jdbcCatalog.listTables(Namespace.of("namespace2"))) + .hasSize(1) + .containsExactly(tableTwo); + + TableIdentifier newTable = TableIdentifier.of("namespace1", "table2"); + jdbcCatalog.buildTable(newTable, SCHEMA).create(); + + assertThat(jdbcCatalog.listTables(Namespace.of("namespace1"))) + .hasSize(2) + .containsExactly(tableOne, newTable); + + assertThatThrownBy(() -> jdbcCatalog.listViews(Namespace.of("namespace1"))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage(JdbcCatalog.VIEW_WARNING_LOG_MESSAGE); + + assertThatThrownBy( + () -> jdbcCatalog.buildView(TableIdentifier.of("namespace1", "view")).create()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage(JdbcCatalog.VIEW_WARNING_LOG_MESSAGE); + } + @Test public void testCreateTableBuilder() { TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); @@ -854,4 +942,68 @@ public void report(MetricsReport report) { COUNTER.incrementAndGet(); } } + + private void initLegacySchema(String jdbcUrl) throws SQLException { + SQLiteDataSource dataSource = new SQLiteDataSource(); + dataSource.setUrl(jdbcUrl); + + try (Connection connection = dataSource.getConnection()) { + // create "old style" SQL schema + connection + .prepareStatement( + "CREATE TABLE " + + JdbcUtil.CATALOG_TABLE_VIEW_NAME + + "(" + + JdbcUtil.CATALOG_NAME + + " VARCHAR(255) NOT NULL," + + JdbcUtil.TABLE_NAMESPACE + + " VARCHAR(255) NOT NULL," + + JdbcUtil.TABLE_NAME + + " VARCHAR(255) NOT NULL," + + JdbcTableOperations.METADATA_LOCATION_PROP + + " VARCHAR(1000)," + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + + " VARCHAR(1000)," + + "PRIMARY KEY(" + + JdbcUtil.CATALOG_NAME + + "," + + JdbcUtil.TABLE_NAMESPACE + + "," + + JdbcUtil.TABLE_NAME + + "))") + .executeUpdate(); + connection + .prepareStatement( + "INSERT INTO " + + JdbcUtil.CATALOG_TABLE_VIEW_NAME + + "(" + + JdbcUtil.CATALOG_NAME + + "," + + JdbcUtil.TABLE_NAMESPACE + + "," + + JdbcUtil.TABLE_NAME + + "," + + JdbcTableOperations.METADATA_LOCATION_PROP + + "," + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + + ") VALUES('TEST','namespace1','table1',null,null)") + .execute(); + connection + .prepareStatement( + "INSERT INTO " + + JdbcUtil.CATALOG_TABLE_VIEW_NAME + + "(" + + JdbcUtil.CATALOG_NAME + + "," + + JdbcUtil.TABLE_NAMESPACE + + "," + + JdbcUtil.TABLE_NAME + + "," + + JdbcTableOperations.METADATA_LOCATION_PROP + + "," + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + + ") VALUES('TEST','namespace2','table2',null,null)") + .execute(); + } + } } 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 0dec6eb83ebb..7dde37d4b510 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.jdbc; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import java.util.Properties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestJdbcUtil { @@ -42,6 +43,6 @@ public void testFilterAndRemovePrefix() { Properties actual = JdbcUtil.filterAndRemovePrefix(input, "jdbc."); - Assertions.assertThat(expected).isEqualTo(actual); + assertThat(expected).isEqualTo(actual); } } diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java new file mode 100644 index 000000000000..c7bdb158cdf2 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java @@ -0,0 +1,66 @@ +/* + * 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.jdbc; + +import java.util.Map; +import java.util.UUID; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.view.ViewCatalogTests; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; + +public class TestJdbcViewCatalog extends ViewCatalogTests { + + private JdbcCatalog catalog; + + @TempDir private java.nio.file.Path tableDir; + + @BeforeEach + public void before() { + 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()); + + catalog = new JdbcCatalog(); + catalog.setConf(new Configuration()); + catalog.initialize("testCatalog", properties); + } + + @Override + protected JdbcCatalog catalog() { + return catalog; + } + + @Override + protected Catalog tableCatalog() { + return catalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return true; + } +} From 62be61ad2a43ee619164329f9d7f482145190fab Mon Sep 17 00:00:00 2001 From: Sung Yun <107272191+syun64@users.noreply.github.com> Date: Fri, 16 Feb 2024 18:35:53 -0700 Subject: [PATCH 0047/1019] Support usage of Separate OIDC Authorization Server URI (#8976) * configurable token-uri * more tests * lint * minor changes * suppress style * address feedback * adopt review feedback * update config name * lint * adopt review comments --- .../aws/s3/signer/S3V4RestSignerClient.java | 16 +- .../org/apache/iceberg/rest/HTTPClient.java | 29 +- .../iceberg/rest/RESTSessionCatalog.java | 7 +- .../iceberg/rest/auth/OAuth2Properties.java | 3 + .../apache/iceberg/rest/auth/OAuth2Util.java | 86 ++++- .../iceberg/rest/RESTCatalogAdapter.java | 66 ++-- .../apache/iceberg/rest/TestRESTCatalog.java | 342 +++++++++++++----- 7 files changed, 391 insertions(+), 158 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java index 99f1588f0c53..f284822161ca 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java @@ -41,6 +41,7 @@ import org.apache.iceberg.rest.ErrorHandlers; import org.apache.iceberg.rest.HTTPClient; import org.apache.iceberg.rest.RESTClient; +import org.apache.iceberg.rest.ResourcePaths; import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession; @@ -111,6 +112,12 @@ public String credential() { return properties().get(OAuth2Properties.CREDENTIAL); } + /** Token endpoint URI to fetch token from if the Rest Catalog is not the authorization server. */ + @Value.Lazy + public String oauth2ServerUri() { + return properties().getOrDefault(OAuth2Properties.OAUTH2_SERVER_URI, ResourcePaths.tokens()); + } + /** A Bearer token supplier which will be used for interaction with the server. */ @Value.Default public Supplier token() { @@ -199,7 +206,8 @@ private AuthSession authSession() { tokenRefreshExecutor(), token, expiresAtMillis(properties()), - new AuthSession(ImmutableMap.of(), token, null, credential(), SCOPE))); + new AuthSession( + ImmutableMap.of(), token, null, credential(), SCOPE, oauth2ServerUri()))); } if (credentialProvided()) { @@ -208,10 +216,12 @@ private AuthSession authSession() { credential(), id -> { AuthSession session = - new AuthSession(ImmutableMap.of(), null, null, credential(), SCOPE); + new AuthSession( + ImmutableMap.of(), null, null, credential(), SCOPE, oauth2ServerUri()); long startTimeMillis = System.currentTimeMillis(); OAuthTokenResponse authResponse = - OAuth2Util.fetchToken(httpClient(), session.headers(), credential(), SCOPE); + OAuth2Util.fetchToken( + httpClient(), session.headers(), credential(), SCOPE, oauth2ServerUri()); return AuthSession.fromTokenResponse( httpClient(), tokenRefreshExecutor(), authResponse, startTimeMillis, session); }); diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index 55ed1422b3e1..936663095492 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -205,16 +205,24 @@ private static void throwFailure( } private URI buildUri(String path, Map params) { - String baseUri = String.format("%s/%s", uri, path); + // if full path is provided, use the input path as path + if (path.startsWith("/")) { + throw new RESTException( + "Received a malformed path for a REST request: %s. Paths should not start with /", path); + } + String fullPath = + (path.startsWith("https://") || path.startsWith("http://")) + ? path + : String.format("%s/%s", uri, path); try { - URIBuilder builder = new URIBuilder(baseUri); + URIBuilder builder = new URIBuilder(fullPath); if (params != null) { params.forEach(builder::addParameter); } return builder.build(); } catch (URISyntaxException e) { throw new RESTException( - "Failed to create request URI from base %s, params %s", baseUri, params); + "Failed to create request URI from base %s, params %s", fullPath, params); } } @@ -223,7 +231,7 @@ private URI buildUri(String path, Map params) { * * @param method - HTTP method, such as GET, POST, HEAD, etc. * @param queryParams - A map of query parameters - * @param path - URL path to send the request to + * @param path - URI to send the request to * @param requestBody - Content to place in the request body * @param responseType - Class of the Response type. Needs to have serializer registered with * ObjectMapper @@ -250,7 +258,7 @@ private T execute( * * @param method - HTTP method, such as GET, POST, HEAD, etc. * @param queryParams - A map of query parameters - * @param path - URL path to send the request to + * @param path - URL to send the request to * @param requestBody - Content to place in the request body * @param responseType - Class of the Response type. Needs to have serializer registered with * ObjectMapper @@ -270,11 +278,6 @@ private T execute( Map headers, Consumer errorHandler, Consumer> responseHeaders) { - if (path.startsWith("/")) { - throw new RESTException( - "Received a malformed path for a REST request: %s. Paths should not start with /", path); - } - HttpUriRequestBase request = new HttpUriRequestBase(method.name(), buildUri(path, queryParams)); if (requestBody instanceof Map) { @@ -459,9 +462,9 @@ private Builder(Map properties) { this.properties = properties; } - public Builder uri(String baseUri) { - Preconditions.checkNotNull(baseUri, "Invalid uri for http client: null"); - this.uri = RESTUtil.stripTrailingSlash(baseUri); + public Builder uri(String path) { + Preconditions.checkNotNull(path, "Invalid uri for http client: null"); + this.uri = RESTUtil.stripTrailingSlash(path); 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 5f660f0f4fe8..87519cfd9dd2 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -178,11 +178,14 @@ public void initialize(String name, Map unresolved) { OAuthTokenResponse authResponse; String credential = props.get(OAuth2Properties.CREDENTIAL); String scope = props.getOrDefault(OAuth2Properties.SCOPE, OAuth2Properties.CATALOG_SCOPE); + String oauth2ServerUri = + props.getOrDefault(OAuth2Properties.OAUTH2_SERVER_URI, ResourcePaths.tokens()); try (RESTClient initClient = clientBuilder.apply(props)) { Map initHeaders = RESTUtil.merge(configHeaders(props), OAuth2Util.authHeaders(initToken)); if (credential != null && !credential.isEmpty()) { - authResponse = OAuth2Util.fetchToken(initClient, initHeaders, credential, scope); + authResponse = + OAuth2Util.fetchToken(initClient, initHeaders, credential, scope, oauth2ServerUri); Map authHeaders = RESTUtil.merge(initHeaders, OAuth2Util.authHeaders(authResponse.token())); config = fetchConfig(initClient, authHeaders, props); @@ -209,7 +212,7 @@ public void initialize(String name, Map unresolved) { this.paths = ResourcePaths.forCatalogProperties(mergedProps); String token = mergedProps.get(OAuth2Properties.TOKEN); - this.catalogAuth = new AuthSession(baseHeaders, null, null, credential, scope); + this.catalogAuth = new AuthSession(baseHeaders, null, null, credential, scope, oauth2ServerUri); if (authResponse != null) { this.catalogAuth = AuthSession.fromTokenResponse( diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java index 8bd75c5ebb33..e1a9181d164d 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java @@ -27,6 +27,9 @@ private OAuth2Properties() {} /** A credential to exchange for a token in the OAuth2 client credentials flow. */ public static final String CREDENTIAL = "credential"; + /** Token endpoint URI to fetch token from if the Rest Catalog is not the authorization server. */ + public static final String OAUTH2_SERVER_URI = "oauth2-server-uri"; + /** * Interval in milliseconds to wait before attempting to exchange the configured catalog Bearer * token. By default, token exchange will be attempted after 1 hour. diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 678642830cda..ad1821a3f2b6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -134,7 +134,8 @@ private static OAuthTokenResponse refreshToken( Map headers, String subjectToken, String subjectTokenType, - String scope) { + String scope, + String oauth2ServerUri) { Map request = tokenExchangeRequest( subjectToken, @@ -143,7 +144,7 @@ private static OAuthTokenResponse refreshToken( OAuthTokenResponse response = client.postForm( - ResourcePaths.tokens(), + oauth2ServerUri, request, OAuthTokenResponse.class, headers, @@ -160,7 +161,8 @@ public static OAuthTokenResponse exchangeToken( String subjectTokenType, String actorToken, String actorTokenType, - String scope) { + String scope, + String oauth2ServerUri) { Map request = tokenExchangeRequest( subjectToken, @@ -171,7 +173,7 @@ public static OAuthTokenResponse exchangeToken( OAuthTokenResponse response = client.postForm( - ResourcePaths.tokens(), + oauth2ServerUri, request, OAuthTokenResponse.class, headers, @@ -181,15 +183,38 @@ public static OAuthTokenResponse exchangeToken( return response; } + public static OAuthTokenResponse exchangeToken( + RESTClient client, + Map headers, + String subjectToken, + String subjectTokenType, + String actorToken, + String actorTokenType, + String scope) { + return exchangeToken( + client, + headers, + subjectToken, + subjectTokenType, + actorToken, + actorTokenType, + scope, + ResourcePaths.tokens()); + } + public static OAuthTokenResponse fetchToken( - RESTClient client, Map headers, String credential, String scope) { + RESTClient client, + Map headers, + String credential, + String scope, + String oauth2ServerUri) { Map request = clientCredentialsRequest( credential, scope != null ? ImmutableList.of(scope) : ImmutableList.of()); OAuthTokenResponse response = client.postForm( - ResourcePaths.tokens(), + oauth2ServerUri, request, OAuthTokenResponse.class, headers, @@ -199,6 +224,12 @@ public static OAuthTokenResponse fetchToken( return response; } + public static OAuthTokenResponse fetchToken( + RESTClient client, Map headers, String credential, String scope) { + + return fetchToken(client, headers, credential, scope, ResourcePaths.tokens()); + } + private static Map tokenExchangeRequest( String subjectToken, String subjectTokenType, List scopes) { return tokenExchangeRequest(subjectToken, subjectTokenType, null, null, scopes); @@ -361,7 +392,26 @@ public static class AuthSession { private final String credential; private final String scope; private volatile boolean keepRefreshed = true; + private final String oauth2ServerUri; + public AuthSession( + Map baseHeaders, + String token, + String tokenType, + String credential, + String scope, + String oauth2ServerUri) { + this.headers = RESTUtil.merge(baseHeaders, authHeaders(token)); + this.token = token; + this.tokenType = tokenType; + this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); + this.credential = credential; + this.scope = scope; + this.oauth2ServerUri = oauth2ServerUri; + } + + /** @deprecated since 1.5.0, will be removed in 1.6.0 */ + @Deprecated public AuthSession( Map baseHeaders, String token, @@ -374,6 +424,7 @@ public AuthSession( this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); this.credential = credential; this.scope = scope; + this.oauth2ServerUri = ResourcePaths.tokens(); } public Map headers() { @@ -404,6 +455,10 @@ public String credential() { return credential; } + public String oauth2ServerUri() { + return oauth2ServerUri; + } + @VisibleForTesting static void setTokenRefreshNumRetries(int retries) { tokenRefreshNumRetries = retries; @@ -415,7 +470,8 @@ static void setTokenRefreshNumRetries(int retries) { * @return A new {@link AuthSession} with empty headers. */ public static AuthSession empty() { - return new AuthSession(ImmutableMap.of(), null, null, null, OAuth2Properties.CATALOG_SCOPE); + return new AuthSession( + ImmutableMap.of(), null, null, null, OAuth2Properties.CATALOG_SCOPE, null); } /** @@ -470,14 +526,14 @@ private OAuthTokenResponse refreshCurrentToken(RESTClient client) { return refreshExpiredToken(client); } else { // attempt a normal refresh - return refreshToken(client, headers(), token, tokenType, scope); + return refreshToken(client, headers(), token, tokenType, scope, oauth2ServerUri); } } private OAuthTokenResponse refreshExpiredToken(RESTClient client) { if (credential != null) { Map basicHeaders = RESTUtil.merge(headers(), basicAuthHeaders(credential)); - return refreshToken(client, basicHeaders, token, tokenType, scope); + return refreshToken(client, basicHeaders, token, tokenType, scope, oauth2ServerUri); } return null; @@ -533,7 +589,8 @@ public static AuthSession fromAccessToken( token, OAuth2Properties.ACCESS_TOKEN_TYPE, parent.credential(), - parent.scope()); + parent.scope(), + parent.oauth2ServerUri()); long startTimeMillis = System.currentTimeMillis(); Long expiresAtMillis = session.expiresAtMillis(); @@ -571,7 +628,8 @@ public static AuthSession fromCredential( AuthSession parent) { long startTimeMillis = System.currentTimeMillis(); OAuthTokenResponse response = - fetchToken(client, parent.headers(), credential, parent.scope()); + fetchToken( + client, parent.headers(), credential, parent.scope(), parent.oauth2ServerUri()); return fromTokenResponse(client, executor, response, startTimeMillis, parent, credential); } @@ -598,7 +656,8 @@ private static AuthSession fromTokenResponse( response.token(), response.issuedTokenType(), credential, - parent.scope()); + parent.scope(), + parent.oauth2ServerUri()); Long expiresAtMillis = session.expiresAtMillis(); if (null == expiresAtMillis && response.expiresInSeconds() != null) { @@ -627,7 +686,8 @@ public static AuthSession fromTokenExchange( tokenType, parent.token(), parent.tokenType(), - parent.scope()); + parent.scope(), + parent.oauth2ServerUri()); return fromTokenResponse(client, executor, response, startTimeMillis, parent); } } 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 1974838ede32..7fccc4e974e3 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -112,6 +112,8 @@ enum HTTPMethod { enum Route { TOKENS(HTTPMethod.POST, "v1/oauth/tokens", null, OAuthTokenResponse.class), + SEPARATE_AUTH_TOKENS_URI( + HTTPMethod.POST, "https://auth-server.com/token", null, OAuthTokenResponse.class), CONFIG(HTTPMethod.GET, "v1/config", null, ConfigResponse.class), LIST_NAMESPACES(HTTPMethod.GET, "v1/namespaces", null, ListNamespacesResponse.class), CREATE_NAMESPACE( @@ -245,43 +247,40 @@ public Class responseClass() { } } + private static OAuthTokenResponse handleOAuthRequest(Object body) { + Map request = (Map) castRequest(Map.class, body); + String grantType = request.get("grant_type"); + switch (grantType) { + case "client_credentials": + return OAuthTokenResponse.builder() + .withToken("client-credentials-token:sub=" + request.get("client_id")) + .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") + .withTokenType("Bearer") + .build(); + + case "urn:ietf:params:oauth:grant-type:token-exchange": + String actor = request.get("actor_token"); + String token = + String.format( + "token-exchange-token:sub=%s%s", + request.get("subject_token"), actor != null ? ",act=" + actor : ""); + return OAuthTokenResponse.builder() + .withToken(token) + .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") + .withTokenType("Bearer") + .build(); + + default: + throw new UnsupportedOperationException("Unsupported grant_type: " + grantType); + } + } + @SuppressWarnings({"MethodLength", "checkstyle:CyclomaticComplexity"}) public T handleRequest( Route route, Map vars, Object body, Class responseType) { switch (route) { case TOKENS: - { - @SuppressWarnings("unchecked") - Map request = (Map) castRequest(Map.class, body); - String grantType = request.get("grant_type"); - switch (grantType) { - case "client_credentials": - return castResponse( - responseType, - OAuthTokenResponse.builder() - .withToken("client-credentials-token:sub=" + request.get("client_id")) - .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") - .withTokenType("Bearer") - .build()); - - case "urn:ietf:params:oauth:grant-type:token-exchange": - String actor = request.get("actor_token"); - String token = - String.format( - "token-exchange-token:sub=%s%s", - request.get("subject_token"), actor != null ? ",act=" + actor : ""); - return castResponse( - responseType, - OAuthTokenResponse.builder() - .withToken(token) - .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") - .withTokenType("Bearer") - .build()); - - default: - throw new UnsupportedOperationException("Unsupported grant_type: " + grantType); - } - } + return castResponse(responseType, handleOAuthRequest(body)); case CONFIG: return castResponse(responseType, ConfigResponse.builder().build()); @@ -469,6 +468,9 @@ public T handleRequest( } default: + if (responseType == OAuthTokenResponse.class) { + return castResponse(responseType, handleOAuthRequest(body)); + } } return null; 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 0e260a4e53d5..cd7710d460a9 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -353,8 +353,7 @@ public void testCatalogBasicBearerToken() { any()); } - @Test - public void testCatalogCredential() { + public void testCatalogCredentialNoOauth2ServerUri() { Map emptyHeaders = ImmutableMap.of(); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -400,8 +399,64 @@ public void testCatalogCredential() { any()); } - @Test - public void testCatalogBearerTokenWithClientCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogCredential(String oauth2ServerUri) { + Map emptyHeaders = ImmutableMap.of(); + Map catalogHeaders = + ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); + + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize( + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "credential", + "catalog:secret", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); + + Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + + // no token or credential for catalog token exchange + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.POST), + eq(oauth2ServerUri), + any(), + any(), + eq(OAuthTokenResponse.class), + eq(emptyHeaders), + any()); + // no token or credential for config + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/config"), + any(), + any(), + eq(ConfigResponse.class), + eq(catalogHeaders), + any()); + // use the catalog token for all interactions + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/namespaces/ns/tables/table"), + any(), + any(), + eq(LoadTableResponse.class), + eq(catalogHeaders), + any()); + } + + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogBearerTokenWithClientCredential(String oauth2ServerUri) { Map contextHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=user"); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer bearer-token"); @@ -417,7 +472,14 @@ public void testCatalogBearerTokenWithClientCredential() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", "bearer-token")); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "token", + "bearer-token", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); @@ -435,7 +497,7 @@ public void testCatalogBearerTokenWithClientCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -453,8 +515,9 @@ public void testCatalogBearerTokenWithClientCredential() { any()); } - @Test - public void testCatalogCredentialWithClientCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogCredentialWithClientCredential(String oauth2ServerUri) { Map emptyHeaders = ImmutableMap.of(); Map contextHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=user"); @@ -472,7 +535,14 @@ public void testCatalogCredentialWithClientCredential() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", "catalog:secret")); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "credential", + "catalog:secret", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); @@ -480,7 +550,7 @@ public void testCatalogCredentialWithClientCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -500,7 +570,7 @@ public void testCatalogCredentialWithClientCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -518,8 +588,9 @@ public void testCatalogCredentialWithClientCredential() { any()); } - @Test - public void testCatalogBearerTokenAndCredentialWithClientCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogBearerTokenAndCredentialWithClientCredential(String oauth2ServerUri) { Map contextHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=user"); Map initHeaders = ImmutableMap.of("Authorization", "Bearer bearer-token"); @@ -544,7 +615,9 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential() { "credential", "catalog:secret", "token", - "bearer-token")); + "bearer-token", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); @@ -552,7 +625,7 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -572,7 +645,7 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -590,8 +663,9 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential() { any()); } - @Test - public void testClientBearerToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientBearerToken(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of( @@ -602,11 +676,13 @@ public void testClientBearerToken() { "urn:ietf:params:oauth:token-type:jwt", "jwt-token", "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), - ImmutableMap.of("Authorization", "Bearer client-bearer-token")); + ImmutableMap.of("Authorization", "Bearer client-bearer-token"), + oauth2ServerUri); } - @Test - public void testClientCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientCredential(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of( @@ -616,11 +692,13 @@ public void testClientCredential() { "urn:ietf:params:oauth:token-type:jwt", "jwt-token", "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), - ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=user")); + ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=user"), + oauth2ServerUri); } - @Test - public void testClientIDToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientIDToken(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of( @@ -630,11 +708,13 @@ public void testClientIDToken() { "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( - "Authorization", "Bearer token-exchange-token:sub=id-token,act=bearer-token")); + "Authorization", "Bearer token-exchange-token:sub=id-token,act=bearer-token"), + oauth2ServerUri); } - @Test - public void testClientAccessToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientAccessToken(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of( @@ -643,11 +723,13 @@ public void testClientAccessToken() { "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( - "Authorization", "Bearer token-exchange-token:sub=access-token,act=bearer-token")); + "Authorization", "Bearer token-exchange-token:sub=access-token,act=bearer-token"), + oauth2ServerUri); } - @Test - public void testClientJWTToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientJWTToken(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of( @@ -655,31 +737,39 @@ public void testClientJWTToken() { "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( - "Authorization", "Bearer token-exchange-token:sub=jwt-token,act=bearer-token")); + "Authorization", "Bearer token-exchange-token:sub=jwt-token,act=bearer-token"), + oauth2ServerUri); } - @Test - public void testClientSAML2Token() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientSAML2Token(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of( "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( - "Authorization", "Bearer token-exchange-token:sub=saml2-token,act=bearer-token")); + "Authorization", "Bearer token-exchange-token:sub=saml2-token,act=bearer-token"), + oauth2ServerUri); } - @Test - public void testClientSAML1Token() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientSAML1Token(String oauth2ServerUri) { testClientAuth( "bearer-token", ImmutableMap.of("urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( - "Authorization", "Bearer token-exchange-token:sub=saml1-token,act=bearer-token")); + "Authorization", "Bearer token-exchange-token:sub=saml1-token,act=bearer-token"), + oauth2ServerUri); } private void testClientAuth( - String catalogToken, Map credentials, Map expectedHeaders) { + String catalogToken, + Map credentials, + Map expectedHeaders, + String oauth2ServerUri) { Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer " + catalogToken); RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); @@ -690,7 +780,14 @@ private void testClientAuth( RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", catalogToken)); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "token", + catalogToken, + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); @@ -710,7 +807,7 @@ private void testClientAuth( Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -729,18 +826,21 @@ private void testClientAuth( any()); } - @Test - public void testTableBearerToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testTableBearerToken(String oauth2ServerUri) { testTableAuth( "catalog", ImmutableMap.of("urn:ietf:params:oauth:token-type:id_token", "id-token"), ImmutableMap.of("token", "table-bearer-token"), ImmutableMap.of("Authorization", "Bearer token-exchange-token:sub=id-token,act=catalog"), - ImmutableMap.of("Authorization", "Bearer table-bearer-token")); + ImmutableMap.of("Authorization", "Bearer table-bearer-token"), + oauth2ServerUri); } - @Test - public void testTableIDToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testTableIDToken(String oauth2ServerUri) { testTableAuth( "catalog", ImmutableMap.of("urn:ietf:params:oauth:token-type:id_token", "id-token"), @@ -748,17 +848,20 @@ public void testTableIDToken() { ImmutableMap.of("Authorization", "Bearer token-exchange-token:sub=id-token,act=catalog"), ImmutableMap.of( "Authorization", - "Bearer token-exchange-token:sub=table-id-token,act=token-exchange-token:sub=id-token,act=catalog")); + "Bearer token-exchange-token:sub=table-id-token,act=token-exchange-token:sub=id-token,act=catalog"), + oauth2ServerUri); } - @Test - public void testTableCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testTableCredential(String oauth2ServerUri) { testTableAuth( "catalog", ImmutableMap.of("urn:ietf:params:oauth:token-type:id_token", "id-token"), ImmutableMap.of("credential", "table-user:secret"), ImmutableMap.of("Authorization", "Bearer token-exchange-token:sub=id-token,act=catalog"), - ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=table-user")); + ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=table-user"), + oauth2ServerUri); } @Test @@ -914,7 +1017,6 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { String branch = "divergedBranch"; table.manageSnapshots().createBranch(branch, table.currentSnapshot().snapshotId()).commit(); - // branch and main are diverged now table .newFastAppend() .appendFile( @@ -1095,12 +1197,14 @@ public void lazySnapshotLoadingWithDivergedHistory() { assertThat(refsTables.history()).hasSize(numSnapshots); } + @SuppressWarnings("MethodLength") public void testTableAuth( String catalogToken, Map credentials, Map tableConfig, Map expectedContextHeaders, - Map expectedTableHeaders) { + Map expectedTableHeaders, + String oauth2ServerUri) { TableIdentifier ident = TableIdentifier.of("ns", "table"); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer " + catalogToken); @@ -1145,7 +1249,14 @@ public void testTableAuth( RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", catalogToken)); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "token", + catalogToken, + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Schema expectedSchema = new Schema( @@ -1181,7 +1292,7 @@ public void testTableAuth( Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1205,7 +1316,7 @@ public void testTableAuth( Mockito.verify(adapter, times(1)) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1247,8 +1358,9 @@ public void testTableAuth( any()); } - @Test - public void testCatalogTokenRefresh() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogTokenRefresh(String oauth2ServerUri) { Map emptyHeaders = ImmutableMap.of(); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -1271,7 +1383,7 @@ public void testCatalogTokenRefresh() { .when(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1285,7 +1397,14 @@ public void testCatalogTokenRefresh() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", "catalog:secret")); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "credential", + "catalog:secret", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Awaitility.await() .atMost(5, TimeUnit.SECONDS) @@ -1295,7 +1414,7 @@ public void testCatalogTokenRefresh() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1323,7 +1442,7 @@ public void testCatalogTokenRefresh() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1345,7 +1464,7 @@ public void testCatalogTokenRefresh() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(secondRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1354,8 +1473,9 @@ public void testCatalogTokenRefresh() { }); } - @Test - public void testCatalogRefreshedTokenIsUsed() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogRefreshedTokenIsUsed(String oauth2ServerUri) { Map emptyHeaders = ImmutableMap.of(); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -1378,7 +1498,7 @@ public void testCatalogRefreshedTokenIsUsed() { .when(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1392,7 +1512,14 @@ public void testCatalogRefreshedTokenIsUsed() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", "catalog:secret")); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "credential", + "catalog:secret", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Awaitility.await() .atMost(5, TimeUnit.SECONDS) @@ -1406,7 +1533,7 @@ public void testCatalogRefreshedTokenIsUsed() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1434,7 +1561,7 @@ public void testCatalogRefreshedTokenIsUsed() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1531,8 +1658,9 @@ public void testCatalogExpiredBearerTokenRefreshWithoutCredential() { catalog.initialize("prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", token)); } - @Test - public void testCatalogExpiredBearerTokenIsRefreshedWithCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogExpiredBearerTokenIsRefreshedWithCredential(String oauth2ServerUri) { // expires at epoch second = 1 String token = "eyJhbGciOiJIUzI1NiIsInR5cCI6IkpXVCJ9.eyJzdWIiOiIxMjM0NTY3ODkwIiwibmFtZSI6IkpvaG4gRG9lIiwiaWF0IjoxNTE2MjM5MDIyLCJleHAiOjF9.gQADTbdEv-rpDWKSkGLbmafyB5UUjTdm9B_1izpuZ6E"; @@ -1551,7 +1679,14 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); // the init token at the catalog level is a valid token catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", credential)); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "credential", + credential, + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); @@ -1559,7 +1694,7 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1585,7 +1720,7 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1602,7 +1737,7 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(secondRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1662,8 +1797,9 @@ public void testCatalogValidBearerTokenIsNotRefreshed() { any()); } - @Test - public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh(String oauth2ServerUri) { Map emptyHeaders = ImmutableMap.of(); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -1689,7 +1825,7 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { .when(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1708,7 +1844,7 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { .when(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1725,7 +1861,14 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize( - "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", credential)); + "prod", + ImmutableMap.of( + CatalogProperties.URI, + "ignored", + "credential", + credential, + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Awaitility.await() .atMost(5, TimeUnit.SECONDS) @@ -1739,7 +1882,7 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1762,7 +1905,7 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { Mockito.verify(adapter, times(2)) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1774,7 +1917,7 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1798,8 +1941,9 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh() { }); } - @Test - public void testCatalogWithCustomTokenScope() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogWithCustomTokenScope(String oauth2ServerUri) { Map emptyHeaders = ImmutableMap.of(); Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -1822,7 +1966,7 @@ public void testCatalogWithCustomTokenScope() { .when(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1844,7 +1988,9 @@ public void testCatalogWithCustomTokenScope() { "credential", "catalog:secret", OAuth2Properties.SCOPE, - scope)); + scope, + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Awaitility.await() .atMost(5, TimeUnit.SECONDS) @@ -1854,7 +2000,7 @@ public void testCatalogWithCustomTokenScope() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1882,7 +2028,7 @@ public void testCatalogWithCustomTokenScope() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(firstRefreshRequest::equals), eq(OAuthTokenResponse.class), @@ -1891,8 +2037,9 @@ public void testCatalogWithCustomTokenScope() { }); } - @Test - public void testCatalogTokenRefreshDisabledWithToken() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogTokenRefreshDisabledWithToken(String oauth2ServerUri) { String token = "some-token"; Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer " + token); @@ -1914,7 +2061,7 @@ public void testCatalogTokenRefreshDisabledWithToken() { .when(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), any(), eq(OAuthTokenResponse.class), @@ -1935,7 +2082,9 @@ public void testCatalogTokenRefreshDisabledWithToken() { OAuth2Properties.TOKEN, token, OAuth2Properties.TOKEN_REFRESH_ENABLED, - "false")); + "false", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); Mockito.verify(adapter) .execute( @@ -1948,8 +2097,9 @@ public void testCatalogTokenRefreshDisabledWithToken() { any()); } - @Test - public void testCatalogTokenRefreshDisabledWithCredential() { + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testCatalogTokenRefreshDisabledWithCredential(String oauth2ServerUri) { Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=catalog"); @@ -1968,7 +2118,9 @@ public void testCatalogTokenRefreshDisabledWithCredential() { OAuth2Properties.CREDENTIAL, "catalog:12345", OAuth2Properties.TOKEN_REFRESH_ENABLED, - "false")); + "false", + OAuth2Properties.OAUTH2_SERVER_URI, + oauth2ServerUri)); // fetch token from client credential Map fetchTokenFromCredential = @@ -1984,7 +2136,7 @@ public void testCatalogTokenRefreshDisabledWithCredential() { Mockito.verify(adapter) .execute( eq(HTTPMethod.POST), - eq("v1/oauth/tokens"), + eq(oauth2ServerUri), any(), Mockito.argThat(fetchTokenFromCredential::equals), eq(OAuthTokenResponse.class), From c4330b4011d2cc888f75457a4da4c91b31ff1213 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sat, 17 Feb 2024 17:16:57 +0100 Subject: [PATCH 0048/1019] Core: Don't reset snapshotLog when replacing Table (#9732) --- .../org/apache/iceberg/TableMetadata.java | 12 +- .../org/apache/iceberg/TestTableMetadata.java | 25 ++++ .../spark/extensions/TestMetadataTables.java | 112 ++++++++++++++++++ 3 files changed, 148 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 8e687950a6f2..9587c57a0fd2 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -702,7 +702,7 @@ public TableMetadata buildReplacement( return new Builder(this) .upgradeFormatVersion(newFormatVersion) - .removeRef(SnapshotRef.MAIN_BRANCH) + .resetMainBranch() .setCurrentSchema(freshSchema, newLastColumnId.get()) .setDefaultPartitionSpec(freshSpec) .setDefaultSortOrder(freshSortOrder) @@ -1258,6 +1258,16 @@ public Builder removeRef(String name) { return this; } + private Builder resetMainBranch() { + this.currentSnapshotId = -1; + SnapshotRef ref = refs.remove(SnapshotRef.MAIN_BRANCH); + if (ref != null) { + changes.add(new MetadataUpdate.RemoveSnapshotRef(SnapshotRef.MAIN_BRANCH)); + } + + return this; + } + public Builder setStatistics(long snapshotId, StatisticsFile statisticsFile) { Preconditions.checkNotNull(statisticsFile, "statisticsFile is null"); Preconditions.checkArgument( diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 0e5b32595708..826f3ad1e778 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1742,4 +1742,29 @@ private String createManifestListWithManifestFile( return localInput(manifestList).location(); } + + @Test + public void buildReplacementKeepsSnapshotLog() throws Exception { + TableMetadata metadata = + TableMetadataParser.fromJson(readTableMetadataInputFile("TableMetadataV2Valid.json")); + Assertions.assertThat(metadata.currentSnapshot()).isNotNull(); + Assertions.assertThat(metadata.snapshots()).hasSize(2); + Assertions.assertThat(metadata.snapshotLog()).hasSize(2); + + TableMetadata replacement = + metadata.buildReplacement( + metadata.schema(), + metadata.spec(), + metadata.sortOrder(), + metadata.location(), + metadata.properties()); + + Assertions.assertThat(replacement.currentSnapshot()).isNull(); + Assertions.assertThat(replacement.snapshots()) + .hasSize(2) + .containsExactlyElementsOf(metadata.snapshots()); + Assertions.assertThat(replacement.snapshotLog()) + .hasSize(2) + .containsExactlyElementsOf(metadata.snapshotLog()); + } } 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 21439163848d..50376589b671 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.util.Comparator; @@ -30,6 +31,7 @@ import org.apache.commons.collections.ListUtils; import org.apache.iceberg.FileContent; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; @@ -721,4 +723,114 @@ private boolean partitionMatch(Record file, String partValue) { Record partition = (Record) file.get(4); return partValue.equals(partition.get(0).toString()); } + + @Test + public void metadataLogEntriesAfterReplacingTable() throws Exception { + sql( + "CREATE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES " + + "('format-version'='2')", + tableName); + + Table table = Spark3Util.loadIcebergTable(spark, tableName); + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + assertThat(tableMetadata.snapshots()).isEmpty(); + assertThat(tableMetadata.snapshotLog()).isEmpty(); + assertThat(tableMetadata.currentSnapshot()).isNull(); + + Object[] firstEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + null, + null, + null); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)).containsExactly(firstEntry); + + sql("INSERT INTO %s (id, data) VALUES (1, 'a')", tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(1); + assertThat(tableMetadata.snapshotLog()).hasSize(1); + Snapshot currentSnapshot = tableMetadata.currentSnapshot(); + + Object[] secondEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry); + + sql("INSERT INTO %s (id, data) VALUES (1, 'a')", tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(2); + assertThat(tableMetadata.snapshotLog()).hasSize(2); + currentSnapshot = tableMetadata.currentSnapshot(); + + Object[] thirdEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry, thirdEntry); + + sql( + "CREATE OR REPLACE TABLE %s (id bigint, data string) " + + "USING iceberg " + + "PARTITIONED BY (data) " + + "TBLPROPERTIES " + + "('format-version'='2')", + tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(2); + assertThat(tableMetadata.snapshotLog()).hasSize(2); + + // currentSnapshot is null but the metadata_log_entries will refer to the last snapshot from the + // snapshotLog + assertThat(tableMetadata.currentSnapshot()).isNull(); + HistoryEntry historyEntry = tableMetadata.snapshotLog().get(1); + Snapshot lastSnapshot = tableMetadata.snapshot(historyEntry.snapshotId()); + + Object[] fourthEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + lastSnapshot.snapshotId(), + lastSnapshot.schemaId(), + lastSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry, thirdEntry, fourthEntry); + + sql("INSERT INTO %s (id, data) VALUES (1, 'a')", tableName); + + tableMetadata = ((HasTableOperations) table).operations().refresh(); + assertThat(tableMetadata.snapshots()).hasSize(3); + assertThat(tableMetadata.snapshotLog()).hasSize(3); + currentSnapshot = tableMetadata.currentSnapshot(); + + Object[] fifthEntry = + row( + DateTimeUtils.toJavaTimestamp(tableMetadata.lastUpdatedMillis() * 1000), + tableMetadata.metadataFileLocation(), + currentSnapshot.snapshotId(), + currentSnapshot.schemaId(), + currentSnapshot.sequenceNumber()); + + assertThat(sql("SELECT * FROM %s.metadata_log_entries", tableName)) + .containsExactly(firstEntry, secondEntry, thirdEntry, fourthEntry, fifthEntry); + } } From 4b8968d96e1f8cc9a51b50cb932176545d7b159e Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Mon, 19 Feb 2024 15:39:11 +0800 Subject: [PATCH 0049/1019] Aliyun: Add security token to client properties (#9671) --- .../iceberg/aliyun/AliyunClientFactories.java | 20 ++++++++++++++----- .../iceberg/aliyun/AliyunProperties.java | 17 ++++++++++++++++ .../aliyun/TestAliyunClientFactories.java | 15 +++++++++++++- .../iceberg/aliyun/oss/TestOSSFileIO.java | 3 +++ 4 files changed, 49 insertions(+), 6 deletions(-) diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunClientFactories.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunClientFactories.java index 5807f9bfe119..e91d07721cd8 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunClientFactories.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunClientFactories.java @@ -23,6 +23,7 @@ import java.util.Map; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.util.PropertyUtil; public class AliyunClientFactories { @@ -90,11 +91,20 @@ public OSS newOSSClient() { aliyunProperties, "Cannot create aliyun oss client before initializing the AliyunClientFactory."); - return new OSSClientBuilder() - .build( - aliyunProperties.ossEndpoint(), - aliyunProperties.accessKeyId(), - aliyunProperties.accessKeySecret()); + if (Strings.isNullOrEmpty(aliyunProperties.securityToken())) { + return new OSSClientBuilder() + .build( + aliyunProperties.ossEndpoint(), + aliyunProperties.accessKeyId(), + aliyunProperties.accessKeySecret()); + } else { + return new OSSClientBuilder() + .build( + aliyunProperties.ossEndpoint(), + aliyunProperties.accessKeyId(), + aliyunProperties.accessKeySecret(), + aliyunProperties.securityToken()); + } } @Override diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunProperties.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunProperties.java index 623b55263af7..4de784f40c09 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunProperties.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/AliyunProperties.java @@ -52,6 +52,17 @@ public class AliyunProperties implements Serializable { */ public static final String CLIENT_ACCESS_KEY_SECRET = "client.access-key-secret"; + /** + * Aliyun supports Security Token Service (STS) to generate temporary access credentials to + * authorize a user to access the Object Storage Service (OSS) resources within a specific period + * of time. In this way, user does not have to share the AccessKey pair and ensures higher level + * of data security. + * + *

For more information about how to obtain a security token, see: + * https://www.alibabacloud.com/help/en/vod/user-guide/sts-tokens + */ + public static final String CLIENT_SECURITY_TOKEN = "client.security-token"; + /** * The implementation class of {@link AliyunClientFactory} to customize Aliyun client * configurations. If set, all Aliyun clients will be initialized by the specified factory. If not @@ -68,6 +79,7 @@ public class AliyunProperties implements Serializable { private final String ossEndpoint; private final String accessKeyId; private final String accessKeySecret; + private final String securityToken; private final String ossStagingDirectory; public AliyunProperties() { @@ -79,6 +91,7 @@ public AliyunProperties(Map properties) { this.ossEndpoint = properties.get(OSS_ENDPOINT); this.accessKeyId = properties.get(CLIENT_ACCESS_KEY_ID); this.accessKeySecret = properties.get(CLIENT_ACCESS_KEY_SECRET); + this.securityToken = properties.get(CLIENT_SECURITY_TOKEN); this.ossStagingDirectory = PropertyUtil.propertyAsString( @@ -97,6 +110,10 @@ public String accessKeySecret() { return accessKeySecret; } + public String securityToken() { + return securityToken; + } + public String ossStagingDirectory() { return ossStagingDirectory; } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java index 03df4af70b33..a329a3bda7c3 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java @@ -42,8 +42,17 @@ public void testLoadDefault() { .as("Should have no Aliyun properties set") .isNull(); + Assertions.assertThat(defaultFactory.aliyunProperties().securityToken()) + .as("Should have no security token") + .isNull(); + AliyunClientFactory defaultFactoryWithConfig = - AliyunClientFactories.from(ImmutableMap.of(AliyunProperties.CLIENT_ACCESS_KEY_ID, "key")); + AliyunClientFactories.from( + ImmutableMap.of( + AliyunProperties.CLIENT_ACCESS_KEY_ID, + "key", + AliyunProperties.CLIENT_SECURITY_TOKEN, + "token")); Assertions.assertThat(defaultFactoryWithConfig) .as("Should load default when factory impl not configured") .isInstanceOf(AliyunClientFactories.DefaultAliyunClientFactory.class); @@ -51,6 +60,10 @@ public void testLoadDefault() { Assertions.assertThat(defaultFactoryWithConfig.aliyunProperties().accessKeyId()) .as("Should have access key set") .isEqualTo("key"); + + Assertions.assertThat(defaultFactoryWithConfig.aliyunProperties().securityToken()) + .as("Should have security token set") + .isEqualTo("token"); } @Test diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java index 1cc8f45467e2..a4db1b9d04ed 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java @@ -158,6 +158,9 @@ public void serializeClient() throws URISyntaxException { Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getSecretAccessKey()) .as("Should have expected secret key") .isEqualTo(accessSecret); + Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getSecurityToken()) + .as("Should have no security token") + .isNull(); } private FileIO fileIO() { From 4d2602e27bc5c7c890ee13aba2248db358c7ce05 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Feb 2024 10:24:19 +0100 Subject: [PATCH 0050/1019] Build: Bump org.immutables:value from 2.10.0 to 2.10.1 (#9749) Bumps [org.immutables:value](https://github.com/immutables/immutables) from 2.10.0 to 2.10.1. - [Release notes](https://github.com/immutables/immutables/releases) - [Commits](https://github.com/immutables/immutables/compare/2.10.0...2.10.1) --- updated-dependencies: - dependency-name: org.immutables:value 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 5a919142272b..4c18a95b048b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -48,7 +48,7 @@ hadoop3-client = "3.3.6" httpcomponents-httpclient5 = "5.3.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" -immutables-value = "2.10.0" +immutables-value = "2.10.1" jackson-annotations = "2.16.0" jackson-bom = "2.14.2" jackson-dataformat-xml = "2.16.1" From e0d50a68695f2660e0b07f67bc63a3aa987cef37 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Feb 2024 10:24:46 +0100 Subject: [PATCH 0051/1019] Build: Bump software.amazon.awssdk:bom from 2.24.0 to 2.24.5 (#9743) Bumps software.amazon.awssdk:bom from 2.24.0 to 2.24.5. --- 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 4c18a95b048b..3555b5f90699 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.0" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.0" -awssdk-bom = "2.24.0" +awssdk-bom = "2.24.5" azuresdk-bom = "1.2.20" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 651e6ed2d6253bfde6b61c6f64da81a689c47763 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Feb 2024 10:25:20 +0100 Subject: [PATCH 0052/1019] Build: Bump io.netty:netty-buffer from 4.1.68.Final to 4.1.107.Final (#9744) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.68.Final to 4.1.107.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.68.Final...netty-4.1.107.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 3555b5f90699..853dc8c21cec 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -68,8 +68,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.77.1" -netty-buffer = "4.1.97.Final" -netty-buffer-compat = "4.1.68.Final" +netty-buffer = "4.1.107.Final" +netty-buffer-compat = "4.1.107.Final" object-client-bundle = "3.3.2" orc = "1.9.2" parquet = "1.13.1" From 43378bb6a290dd800f6f3f297076f6ad4ec90f03 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Feb 2024 10:26:01 +0100 Subject: [PATCH 0053/1019] Build: Bump datamodel-code-generator from 0.25.3 to 0.25.4 (#9742) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.3 to 0.25.4. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.3...0.25.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 a2135acda0be..4ffabfde3e07 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.25.3 +datamodel-code-generator==0.25.4 From 8a8417534cc191fcd2f2a67752ee7edcfc5fe06f Mon Sep 17 00:00:00 2001 From: pvary Date: Mon, 19 Feb 2024 18:08:18 +0100 Subject: [PATCH 0054/1019] Allow creating metadata tables based on SerializableTable instances (#9735) --- .../apache/iceberg/MetadataTableUtils.java | 11 +++++- .../org/apache/iceberg/SerializableTable.java | 7 +++- .../hadoop/TestTableSerialization.java | 34 ++++++++++++++++--- 3 files changed, 46 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MetadataTableUtils.java b/core/src/main/java/org/apache/iceberg/MetadataTableUtils.java index d045bdbe83c1..adb0f18ba1ad 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataTableUtils.java +++ b/core/src/main/java/org/apache/iceberg/MetadataTableUtils.java @@ -32,9 +32,18 @@ public static boolean hasMetadataTableName(TableIdentifier identifier) { public static Table createMetadataTableInstance(Table table, MetadataTableType type) { if (table instanceof BaseTable) { return createMetadataTableInstance(table, metadataTableName(table.name(), type), type); + } else if (table instanceof HasTableOperations) { + return createMetadataTableInstance( + ((HasTableOperations) table).operations(), + table.name(), + metadataTableName(table.name(), type), + type); } else { throw new IllegalArgumentException( - String.format("Cannot create metadata table for table %s: not a base table", table)); + String.format( + "Cannot create metadata table for table %s: " + + "table is not a base table or does not have table operations", + table)); } } diff --git a/core/src/main/java/org/apache/iceberg/SerializableTable.java b/core/src/main/java/org/apache/iceberg/SerializableTable.java index 42e7af72d4e8..9e0055a10376 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -49,7 +49,7 @@ *

Note: loading the complete metadata from a large number of nodes can overwhelm the * storage. */ -public class SerializableTable implements Table, Serializable { +public class SerializableTable implements Table, HasTableOperations, Serializable { private final String name; private final String location; @@ -388,6 +388,11 @@ public Transaction newTransaction() { throw new UnsupportedOperationException(errorMsg("newTransaction")); } + @Override + public StaticTableOperations operations() { + return (StaticTableOperations) ((BaseTable) lazyTable()).operations(); + } + private String errorMsg(String operation) { return String.format("Operation %s is not supported after the table is serialized", operation); } 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 ed5b5361df01..78a724225bac 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -29,10 +29,12 @@ import org.apache.iceberg.FileScanTask; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; import org.apache.iceberg.PositionDeletesScanTask; import org.apache.iceberg.PositionDeletesTable; import org.apache.iceberg.ScanTask; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TestHelpers; @@ -43,6 +45,8 @@ import org.apache.iceberg.types.Types; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestTableSerialization extends HadoopTableTestBase { @@ -58,6 +62,9 @@ public void testSerializableTable() throws IOException, ClassNotFoundException { Table serializableTable = SerializableTable.copyOf(table); TestHelpers.assertSerializedAndLoadedMetadata( serializableTable, TestHelpers.KryoHelpers.roundTripSerialize(serializableTable)); + Assertions.assertThat(serializableTable).isInstanceOf(HasTableOperations.class); + Assertions.assertThat(((HasTableOperations) serializableTable).operations()) + .isInstanceOf(StaticTableOperations.class); } @Test @@ -118,16 +125,19 @@ public void testSerializableTablePlanning() throws IOException { Assertions.assertThat(deserializedFiles).isNotEqualTo(getFiles(table)); } - @Test - public void testSerializableMetadataTablesPlanning() throws IOException { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testSerializableMetadataTablesPlanning(boolean fromSerialized) throws IOException { table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); - table.newAppend().appendFile(FILE_A).commit(); + Table sourceTable = + fromSerialized ? (SerializableTable) SerializableTable.copyOf(table) : table; + Map serialized = Maps.newHashMap(); Map> expected = Maps.newHashMap(); for (MetadataTableType type : MetadataTableType.values()) { - Table metaTable = getMetaDataTable(table, type); + Table metaTable = MetadataTableUtils.createMetadataTableInstance(sourceTable, type); // Serialize the table serialized.put(type, serializeToBytes(metaTable)); @@ -153,6 +163,22 @@ public void testSerializableMetadataTablesPlanning() throws IOException { } } + @Test + public void testMetadataTableFromSerializedTable() { + table.newAppend().appendFile(FILE_A).commit(); + + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + + Table metaFromOriginal = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.ENTRIES); + Table metaFromSerializable = + MetadataTableUtils.createMetadataTableInstance( + serializableTable, MetadataTableType.ENTRIES); + + // Check that the data is correct + TestHelpers.assertSerializedAndLoadedMetadata(metaFromOriginal, metaFromSerializable); + } + private static Table getMetaDataTable(Table table, MetadataTableType type) { return TABLES.load( ((HasTableOperations) table).operations().current().metadataFileLocation() + "#" + type); From d6734085713f16fc6ba7647f165ddece433872c6 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 19 Feb 2024 14:42:51 -0800 Subject: [PATCH 0055/1019] API: Extend FileIO and add EncryptingFileIO. (#9592) --- .../iceberg/encryption/EncryptingFileIO.java | 214 ++++++++++++++++++ .../java/org/apache/iceberg/io/FileIO.java | 28 +++ .../org/apache/iceberg/ManifestFiles.java | 40 ++-- .../NativeEncryptionOutputFile.java | 24 +- .../encryption/StandardEncryptionManager.java | 4 + .../org/apache/iceberg/io/ContentCache.java | 162 ++++++------- .../hadoop/TestCatalogUtilDropTable.java | 42 ++-- .../iceberg/spark/source/BaseReader.java | 23 +- 8 files changed, 384 insertions(+), 153 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java diff --git a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java new file mode 100644 index 000000000000..ea63b29754bc --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.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.encryption; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Map; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +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; + +public class EncryptingFileIO implements FileIO, Serializable { + public static EncryptingFileIO create(FileIO io, EncryptionManager em) { + if (io instanceof EncryptingFileIO) { + return (EncryptingFileIO) io; + } + + return new EncryptingFileIO(io, em); + } + + private final FileIO io; + private final EncryptionManager em; + + EncryptingFileIO(FileIO io, EncryptionManager em) { + this.io = io; + this.em = em; + } + + public Map bulkDecrypt(Iterable> files) { + Iterable decrypted = em.decrypt(Iterables.transform(files, this::wrap)); + + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (InputFile in : decrypted) { + builder.put(in.location(), in); + } + + return builder.buildKeepingLast(); + } + + public EncryptionManager encryptionManager() { + return em; + } + + @Override + public InputFile newInputFile(String path) { + return io.newInputFile(path); + } + + @Override + public InputFile newInputFile(String path, long length) { + return io.newInputFile(path, length); + } + + @Override + public InputFile newInputFile(DataFile file) { + return newInputFile((ContentFile) file); + } + + @Override + public InputFile newInputFile(DeleteFile file) { + return newInputFile((ContentFile) file); + } + + private InputFile newInputFile(ContentFile file) { + if (file.keyMetadata() != null) { + return newDecryptingInputFile( + file.path().toString(), file.fileSizeInBytes(), file.keyMetadata()); + } else { + return newInputFile(file.path().toString(), file.fileSizeInBytes()); + } + } + + @Override + public InputFile newInputFile(ManifestFile manifest) { + if (manifest.keyMetadata() != null) { + return newDecryptingInputFile(manifest.path(), manifest.length(), manifest.keyMetadata()); + } else { + return newInputFile(manifest.path(), manifest.length()); + } + } + + public InputFile newDecryptingInputFile(String path, ByteBuffer buffer) { + return em.decrypt(wrap(io.newInputFile(path), buffer)); + } + + public InputFile newDecryptingInputFile(String path, long length, ByteBuffer buffer) { + // TODO: is the length correct for the encrypted file? It may be the length of the plaintext + // stream + return em.decrypt(wrap(io.newInputFile(path, length), buffer)); + } + + @Override + public OutputFile newOutputFile(String path) { + return io.newOutputFile(path); + } + + public EncryptedOutputFile newEncryptingOutputFile(String path) { + OutputFile plainOutputFile = io.newOutputFile(path); + return em.encrypt(plainOutputFile); + } + + @Override + public void deleteFile(String path) { + io.deleteFile(path); + } + + @Override + public void close() { + io.close(); + + if (em instanceof Closeable) { + try { + ((Closeable) em).close(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close encryption manager", e); + } + } + } + + private SimpleEncryptedInputFile wrap(ContentFile file) { + InputFile encryptedInputFile = io.newInputFile(file.path().toString(), file.fileSizeInBytes()); + return new SimpleEncryptedInputFile(encryptedInputFile, toKeyMetadata(file.keyMetadata())); + } + + private static SimpleEncryptedInputFile wrap(InputFile encryptedInputFile, ByteBuffer buffer) { + return new SimpleEncryptedInputFile(encryptedInputFile, toKeyMetadata(buffer)); + } + + private static EncryptionKeyMetadata toKeyMetadata(ByteBuffer buffer) { + return buffer != null ? new SimpleKeyMetadata(buffer) : EmptyKeyMetadata.get(); + } + + private static class SimpleEncryptedInputFile implements EncryptedInputFile { + private final InputFile encryptedInputFile; + private final EncryptionKeyMetadata keyMetadata; + + private SimpleEncryptedInputFile( + InputFile encryptedInputFile, EncryptionKeyMetadata keyMetadata) { + this.encryptedInputFile = encryptedInputFile; + this.keyMetadata = keyMetadata; + } + + @Override + public InputFile encryptedInputFile() { + return encryptedInputFile; + } + + @Override + public EncryptionKeyMetadata keyMetadata() { + return keyMetadata; + } + } + + private static class SimpleKeyMetadata implements EncryptionKeyMetadata { + private final ByteBuffer metadataBuffer; + + private SimpleKeyMetadata(ByteBuffer metadataBuffer) { + this.metadataBuffer = metadataBuffer; + } + + @Override + public ByteBuffer buffer() { + return metadataBuffer; + } + + @Override + public EncryptionKeyMetadata copy() { + return new SimpleKeyMetadata(metadataBuffer.duplicate()); + } + } + + private static class EmptyKeyMetadata implements EncryptionKeyMetadata { + private static final EmptyKeyMetadata INSTANCE = new EmptyKeyMetadata(); + + private static EmptyKeyMetadata get() { + return INSTANCE; + } + + @Override + public ByteBuffer buffer() { + return null; + } + + @Override + public EncryptionKeyMetadata copy() { + return this; + } + } +} 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 928f09eb20b6..fc6a53367f21 100644 --- a/api/src/main/java/org/apache/iceberg/io/FileIO.java +++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java @@ -21,6 +21,10 @@ import java.io.Closeable; import java.io.Serializable; import java.util.Map; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** * Pluggable module for reading, writing, and deleting files. @@ -42,6 +46,30 @@ default InputFile newInputFile(String path, long length) { return newInputFile(path); } + default InputFile newInputFile(DataFile file) { + Preconditions.checkArgument( + file.keyMetadata() == null, + "Cannot decrypt data file: {} (use EncryptingFileIO)", + file.path()); + return newInputFile(file.path().toString()); + } + + default InputFile newInputFile(DeleteFile file) { + Preconditions.checkArgument( + file.keyMetadata() == null, + "Cannot decrypt delete file: {} (use EncryptingFileIO)", + file.path()); + return newInputFile(file.path().toString()); + } + + default InputFile newInputFile(ManifestFile manifest) { + Preconditions.checkArgument( + manifest.keyMetadata() == null, + "Cannot decrypt manifest: {} (use EncryptingFileIO)", + manifest.path()); + return newInputFile(manifest.path()); + } + /** Get a {@link OutputFile} instance to write bytes to the file at the given path. */ OutputFile newOutputFile(String path); diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index c23ab667a41b..f7b1add6bdc5 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -126,7 +126,7 @@ public static ManifestReader read( manifest.content() == ManifestContent.DATA, "Cannot read a delete manifest with a ManifestReader: %s", manifest); - InputFile file = newInputFile(io, manifest.path(), manifest.length()); + InputFile file = newInputFile(io, manifest); InheritableMetadata inheritableMetadata = InheritableMetadataFactory.fromManifest(manifest); return new ManifestReader<>( file, manifest.partitionSpecId(), specsById, inheritableMetadata, FileType.DATA_FILES); @@ -181,7 +181,7 @@ public static ManifestReader readDeleteManifest( manifest.content() == ManifestContent.DELETES, "Cannot read a data manifest with a DeleteManifestReader: %s", manifest); - InputFile file = newInputFile(io, manifest.path(), manifest.length()); + InputFile file = newInputFile(io, manifest); InheritableMetadata inheritableMetadata = InheritableMetadataFactory.fromManifest(manifest); return new ManifestReader<>( file, manifest.partitionSpecId(), specsById, inheritableMetadata, FileType.DELETE_FILES); @@ -345,34 +345,24 @@ private static ManifestFile copyManifestInternal( return writer.toManifestFile(); } - private static InputFile newInputFile(FileIO io, String path, long length) { - boolean enabled; - - try { - enabled = cachingEnabled(io); - } catch (UnsupportedOperationException e) { - // There is an issue reading io.properties(). Disable caching. - enabled = false; - } - - if (enabled) { - ContentCache cache = contentCache(io); - Preconditions.checkNotNull( - cache, - "ContentCache creation failed. Check that all manifest caching configurations has valid value."); - LOG.debug("FileIO-level cache stats: {}", CONTENT_CACHES.stats()); - return cache.tryCache(io, path, length); + private static InputFile newInputFile(FileIO io, ManifestFile manifest) { + InputFile input = io.newInputFile(manifest); + if (cachingEnabled(io)) { + return contentCache(io).tryCache(input); } - // caching is not enable for this io or caught RuntimeException. - return io.newInputFile(path, length); + return input; } static boolean cachingEnabled(FileIO io) { - return PropertyUtil.propertyAsBoolean( - io.properties(), - CatalogProperties.IO_MANIFEST_CACHE_ENABLED, - CatalogProperties.IO_MANIFEST_CACHE_ENABLED_DEFAULT); + try { + return PropertyUtil.propertyAsBoolean( + io.properties(), + CatalogProperties.IO_MANIFEST_CACHE_ENABLED, + CatalogProperties.IO_MANIFEST_CACHE_ENABLED_DEFAULT); + } catch (UnsupportedOperationException e) { + return false; + } } static long cacheDurationMs(FileIO io) { diff --git a/core/src/main/java/org/apache/iceberg/encryption/NativeEncryptionOutputFile.java b/core/src/main/java/org/apache/iceberg/encryption/NativeEncryptionOutputFile.java index 0d0d5da8a677..6d095252cb99 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/NativeEncryptionOutputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/NativeEncryptionOutputFile.java @@ -18,13 +18,35 @@ */ package org.apache.iceberg.encryption; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; /** An {@link EncryptedOutputFile} that can be used for format-native encryption. */ -public interface NativeEncryptionOutputFile extends EncryptedOutputFile { +public interface NativeEncryptionOutputFile extends EncryptedOutputFile, OutputFile { @Override NativeEncryptionKeyMetadata keyMetadata(); /** An {@link OutputFile} instance for the underlying (plaintext) output stream. */ OutputFile plainOutputFile(); + + @Override + default PositionOutputStream create() { + return encryptingOutputFile().create(); + } + + @Override + default PositionOutputStream createOrOverwrite() { + return encryptingOutputFile().createOrOverwrite(); + } + + @Override + default String location() { + return encryptingOutputFile().location(); + } + + @Override + default InputFile toInputFile() { + return encryptingOutputFile().toInputFile(); + } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java index 185f4d6f81bb..119d2a5f9ae2 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java @@ -61,6 +61,10 @@ public NativeEncryptionOutputFile encrypt(OutputFile plainOutput) { @Override public NativeEncryptionInputFile decrypt(EncryptedInputFile encrypted) { // this input file will lazily parse key metadata in case the file is not an AES GCM stream. + if (encrypted instanceof NativeEncryptionInputFile) { + return (NativeEncryptionInputFile) encrypted; + } + return new StandardDecryptedInputFile(encrypted); } 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 c999f3f333f6..7942c69d5d77 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -32,7 +32,6 @@ import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.ValidationException; 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.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -40,8 +39,8 @@ /** * Class that provides file-content caching during reading. * - *

The file-content caching is initiated by calling {@link ContentCache#tryCache(FileIO, String, - * long)}. Given a FileIO, a file location string, and file length that is within allowed limit, + *

The file-content caching is initiated by calling {@link ContentCache#tryCache(InputFile)}. + * Given a FileIO, a file location string, and file length that is within allowed limit, * ContentCache will return a {@link CachingInputFile} that is backed by the cache. Calling {@link * CachingInputFile#newStream()} will return a {@link ByteBufferInputStream} backed by list of * {@link ByteBuffer} from the cache if such file-content exist in the cache. If the file-content @@ -56,7 +55,7 @@ public class ContentCache { private final long expireAfterAccessMs; private final long maxTotalBytes; private final long maxContentLength; - private final Cache cache; + private final Cache cache; /** * Constructor for ContentCache class. @@ -86,11 +85,11 @@ public ContentCache(long expireAfterAccessMs, long maxTotalBytes, long maxConten builder .maximumWeight(maxTotalBytes) .weigher( - (Weigher) + (Weigher) (key, value) -> (int) Math.min(value.length, Integer.MAX_VALUE)) .softValues() .removalListener( - (location, cacheEntry, cause) -> + (location, fileContent, cause) -> LOG.debug("Evicted {} from ContentCache ({})", location, cause)) .recordStats() .build(); @@ -112,14 +111,24 @@ public CacheStats stats() { return cache.stats(); } - public CacheEntry get(String key, Function mappingFunction) { + /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ + @Deprecated + public CacheEntry get(String key, Function mappingFunction) { return cache.get(key, mappingFunction); } + /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ + @Deprecated public CacheEntry getIfPresent(String location) { return cache.getIfPresent(location); } + /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ + @Deprecated + public InputFile tryCache(FileIO io, String location, long length) { + return tryCache(io.newInputFile(location, length)); + } + /** * Try cache the file-content of file in the given location upon stream reading. * @@ -127,17 +136,15 @@ public CacheEntry getIfPresent(String location) { * and no caching will be done for that file. Otherwise, this method will return a {@link * CachingInputFile} that serve file reads backed by ContentCache. * - * @param io a FileIO associated with the location. - * @param location URL/path of a file accessible by io. - * @param length the known length of such file. + * @param input an InputFile to cache * @return a {@link CachingInputFile} if length is within allowed limit. Otherwise, a regular * {@link InputFile} for given location. */ - public InputFile tryCache(FileIO io, String location, long length) { - if (length <= maxContentLength) { - return new CachingInputFile(this, io, location, length); + public InputFile tryCache(InputFile input) { + if (input.getLength() <= maxContentLength) { + return new CachingInputFile(this, input); } - return io.newInputFile(location, length); + return input; } public void invalidate(String key) { @@ -166,11 +173,15 @@ public String toString() { .toString(); } - private static class CacheEntry { + /** @deprecated will be removed in 1.7; use {@link FileContent} instead. */ + @Deprecated + private static class CacheEntry {} + + private static class FileContent extends CacheEntry { private final long length; private final List buffers; - private CacheEntry(long length, List buffers) { + private FileContent(long length, List buffers) { this.length = length; this.buffers = buffers; } @@ -187,34 +198,20 @@ private CacheEntry(long length, List buffers) { */ private static class CachingInputFile implements InputFile { private final ContentCache contentCache; - private final FileIO io; - private final String location; - private final long length; - private InputFile fallbackInputFile = null; + private final InputFile input; - private CachingInputFile(ContentCache cache, FileIO io, String location, long length) { + private CachingInputFile(ContentCache cache, InputFile input) { this.contentCache = cache; - this.io = io; - this.location = location; - this.length = length; - } - - private InputFile wrappedInputFile() { - if (fallbackInputFile == null) { - fallbackInputFile = io.newInputFile(location, length); - } - return fallbackInputFile; + this.input = input; } @Override public long getLength() { - CacheEntry buf = contentCache.getIfPresent(location); + FileContent buf = contentCache.cache.getIfPresent(input.location()); if (buf != null) { return buf.length; - } else if (fallbackInputFile != null) { - return fallbackInputFile.getLength(); } else { - return length; + return input.getLength(); } } @@ -232,80 +229,63 @@ public long getLength() { @Override public SeekableInputStream newStream() { try { - // read from cache if file length is less than or equal to maximum length allowed to - // cache. - if (getLength() <= contentCache.maxContentLength()) { - return cachedStream(); - } - - // fallback to non-caching input stream. - return wrappedInputFile().newStream(); + return cachedStream(); } catch (FileNotFoundException e) { - throw new NotFoundException( - e, "Failed to open input stream for file %s: %s", location, e.toString()); + throw new NotFoundException(e, "Failed to open file: %s", input.location()); } catch (IOException e) { - throw new UncheckedIOException( - String.format("Failed to open input stream for file %s: %s", location, e), e); + return input.newStream(); } } @Override public String location() { - return location; + return input.location(); } @Override public boolean exists() { - CacheEntry buf = contentCache.getIfPresent(location); - return buf != null || wrappedInputFile().exists(); - } - - private CacheEntry cacheEntry() { - long start = System.currentTimeMillis(); - try (SeekableInputStream stream = wrappedInputFile().newStream()) { - long fileLength = getLength(); - long totalBytesToRead = fileLength; - List buffers = Lists.newArrayList(); - - while (totalBytesToRead > 0) { - // read the stream in 4MB chunk - int bytesToRead = (int) Math.min(BUFFER_CHUNK_SIZE, totalBytesToRead); - byte[] buf = new byte[bytesToRead]; - int bytesRead = IOUtil.readRemaining(stream, buf, 0, bytesToRead); - totalBytesToRead -= bytesRead; - - if (bytesRead < bytesToRead) { - // Read less than it should be, possibly hitting EOF. Abandon caching by throwing - // IOException and let the caller fallback to non-caching input file. - throw new IOException( - String.format( - "Expected to read %d bytes, but only %d bytes read.", - fileLength, fileLength - totalBytesToRead)); - } else { - buffers.add(ByteBuffer.wrap(buf)); - } - } - - CacheEntry newEntry = new CacheEntry(fileLength, buffers); - LOG.debug("cacheEntry took {} ms for {}", (System.currentTimeMillis() - start), location); - return newEntry; - } catch (IOException ex) { - throw new UncheckedIOException(ex); - } + FileContent buf = contentCache.cache.getIfPresent(input.location()); + return buf != null || input.exists(); } private SeekableInputStream cachedStream() throws IOException { try { - CacheEntry entry = contentCache.get(location, k -> cacheEntry()); - Preconditions.checkNotNull( - entry, "CacheEntry should not be null when there is no RuntimeException occurs"); - LOG.debug("Cache stats: {}", contentCache.stats()); - return ByteBufferInputStream.wrap(entry.buffers); + FileContent content = contentCache.cache.get(input.location(), k -> download(input)); + return ByteBufferInputStream.wrap(content.buffers); } catch (UncheckedIOException ex) { throw ex.getCause(); - } catch (RuntimeException ex) { - throw new IOException("Caught an error while reading from cache", ex); } } } + + private static FileContent download(InputFile input) { + try (SeekableInputStream stream = input.newStream()) { + long fileLength = input.getLength(); + long totalBytesToRead = fileLength; + List buffers = Lists.newArrayList(); + + while (totalBytesToRead > 0) { + // read the stream in chunks + int bytesToRead = (int) Math.min(BUFFER_CHUNK_SIZE, totalBytesToRead); + byte[] buf = new byte[bytesToRead]; + int bytesRead = IOUtil.readRemaining(stream, buf, 0, bytesToRead); + totalBytesToRead -= bytesRead; + + if (bytesRead < bytesToRead) { + // Read less than it should be, possibly hitting EOF. Abandon caching by throwing + // IOException and let the caller fallback to non-caching input file. + throw new IOException( + String.format( + "Failed to read %d bytes: %d bytes in stream", + fileLength, fileLength - totalBytesToRead)); + } else { + buffers.add(ByteBuffer.wrap(buf)); + } + } + + return new FileContent(fileLength, buffers); + } catch (IOException ex) { + throw new UncheckedIOException(ex); + } + } } 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 9cb44b1341cb..e3d2b4d8db92 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java @@ -28,6 +28,8 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.ImmutableGenericPartitionStatisticsFile; @@ -90,13 +92,7 @@ public void dropTableDataDeletesExpectedFiles() throws IOException { .as("should have 1 partition stats file") .containsExactly(partitionStatisticsFile.path()); - FileIO fileIO = Mockito.mock(FileIO.class); - Mockito.when(fileIO.newInputFile(Mockito.anyString())) - .thenAnswer(invocation -> table.io().newInputFile(invocation.getArgument(0))); - Mockito.when(fileIO.newInputFile(Mockito.anyString(), Mockito.anyLong())) - .thenAnswer( - invocation -> - table.io().newInputFile(invocation.getArgument(0), invocation.getArgument(1))); + FileIO fileIO = createMockFileIO(table.io()); CatalogUtil.dropTableData(fileIO, tableMetadata); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); @@ -141,14 +137,7 @@ public void dropTableDataDoNotThrowWhenDeletesFail() { TableMetadata tableMetadata = readMetadataVersion(3); Set snapshotSet = Sets.newHashSet(table.snapshots()); - FileIO fileIO = Mockito.mock(FileIO.class); - Mockito.when(fileIO.newInputFile(Mockito.anyString())) - .thenAnswer(invocation -> table.io().newInputFile(invocation.getArgument(0))); - Mockito.when(fileIO.newInputFile(Mockito.anyString(), Mockito.anyLong())) - .thenAnswer( - invocation -> - table.io().newInputFile(invocation.getArgument(0), invocation.getArgument(1))); - Mockito.doThrow(new RuntimeException()).when(fileIO).deleteFile(ArgumentMatchers.anyString()); + FileIO fileIO = createMockFileIO(table.io()); CatalogUtil.dropTableData(fileIO, tableMetadata); Mockito.verify( @@ -176,9 +165,7 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { Assertions.assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); Assertions.assertThat(metadataLocations).as("should have 4 metadata locations").hasSize(4); - FileIO fileIO = Mockito.mock(FileIO.class); - Mockito.when(fileIO.newInputFile(Mockito.anyString())) - .thenAnswer(invocation -> table.io().newInputFile(invocation.getArgument(0))); + FileIO fileIO = createMockFileIO(table.io()); CatalogUtil.dropTableData(fileIO, tableMetadata); ArgumentCaptor argumentCaptor = ArgumentCaptor.forClass(String.class); @@ -201,6 +188,25 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { .containsAll(metadataLocations); } + private static FileIO createMockFileIO(FileIO wrapped) { + FileIO mockIO = Mockito.mock(FileIO.class); + + Mockito.when(mockIO.newInputFile(Mockito.anyString())) + .thenAnswer(invocation -> wrapped.newInputFile((String) invocation.getArgument(0))); + Mockito.when(mockIO.newInputFile(Mockito.anyString(), Mockito.anyLong())) + .thenAnswer( + invocation -> + wrapped.newInputFile(invocation.getArgument(0), invocation.getArgument(1))); + Mockito.when(mockIO.newInputFile(Mockito.any(ManifestFile.class))) + .thenAnswer(invocation -> wrapped.newInputFile((ManifestFile) invocation.getArgument(0))); + Mockito.when(mockIO.newInputFile(Mockito.any(DataFile.class))) + .thenAnswer(invocation -> wrapped.newInputFile((DataFile) invocation.getArgument(0))); + Mockito.when(mockIO.newInputFile(Mockito.any(DeleteFile.class))) + .thenAnswer(invocation -> wrapped.newInputFile((DeleteFile) invocation.getArgument(0))); + + return mockIO; + } + private static Set manifestListLocations(Set snapshotSet) { return snapshotSet.stream().map(Snapshot::manifestListLocation).collect(Collectors.toSet()); } 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 c2b3e7c2dc56..f67171708bb2 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 @@ -46,14 +46,11 @@ import org.apache.iceberg.data.DeleteFilter; import org.apache.iceberg.data.DeleteLoader; import org.apache.iceberg.deletes.DeleteCounter; -import org.apache.iceberg.encryption.EncryptedFiles; -import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.NameMapping; import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.spark.SparkExecutorCache; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; @@ -184,25 +181,15 @@ protected InputFile getInputFile(String location) { private Map inputFiles() { if (lazyInputFiles == null) { - Stream encryptedFiles = - taskGroup.tasks().stream().flatMap(this::referencedFiles).map(this::toEncryptedInputFile); - - // decrypt with the batch call to avoid multiple RPCs to a key server, if possible - Iterable decryptedFiles = table.encryption().decrypt(encryptedFiles::iterator); - - Map files = Maps.newHashMapWithExpectedSize(taskGroup.tasks().size()); - decryptedFiles.forEach(decrypted -> files.putIfAbsent(decrypted.location(), decrypted)); - this.lazyInputFiles = ImmutableMap.copyOf(files); + this.lazyInputFiles = + EncryptingFileIO.create(table().io(), table().encryption()) + .bulkDecrypt( + () -> taskGroup.tasks().stream().flatMap(this::referencedFiles).iterator()); } return lazyInputFiles; } - private EncryptedInputFile toEncryptedInputFile(ContentFile file) { - InputFile inputFile = table.io().newInputFile(file.path().toString()); - return EncryptedFiles.encryptedInput(inputFile, file.keyMetadata()); - } - protected Map constantsMap(ContentScanTask task, Schema readSchema) { if (readSchema.findField(MetadataColumns.PARTITION_COLUMN_ID) != null) { StructType partitionType = Partitioning.partitionType(table); From b4046768549c3d82336c7b45185876bb14ed7d56 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 20 Feb 2024 00:14:52 +0100 Subject: [PATCH 0056/1019] Core: Don't fail if a REST service doesn't support views (#9754) --- .../iceberg/exceptions/BadRequestException.java | 6 +++--- .../iceberg/exceptions/ForbiddenException.java | 6 +++--- .../org/apache/iceberg/rest/RESTSessionCatalog.java | 12 ++++++++++-- 3 files changed, 16 insertions(+), 8 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/exceptions/BadRequestException.java b/api/src/main/java/org/apache/iceberg/exceptions/BadRequestException.java index c699c1ab22ab..4518cea8f5a1 100644 --- a/api/src/main/java/org/apache/iceberg/exceptions/BadRequestException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/BadRequestException.java @@ -21,14 +21,14 @@ import com.google.errorprone.annotations.FormatMethod; /** Exception thrown on HTTP 400 - Bad Request */ -public class BadRequestException extends RuntimeException implements CleanableFailure { +public class BadRequestException extends RESTException implements CleanableFailure { @FormatMethod public BadRequestException(String message, Object... args) { - super(String.format(message, args)); + super(message, args); } @FormatMethod public BadRequestException(Throwable cause, String message, Object... args) { - super(String.format(message, args), cause); + super(cause, message, args); } } diff --git a/api/src/main/java/org/apache/iceberg/exceptions/ForbiddenException.java b/api/src/main/java/org/apache/iceberg/exceptions/ForbiddenException.java index f66b417fa775..e80cb800d02c 100644 --- a/api/src/main/java/org/apache/iceberg/exceptions/ForbiddenException.java +++ b/api/src/main/java/org/apache/iceberg/exceptions/ForbiddenException.java @@ -21,14 +21,14 @@ import com.google.errorprone.annotations.FormatMethod; /** Exception thrown on HTTP 403 Forbidden - Failed authorization checks. */ -public class ForbiddenException extends RuntimeException implements CleanableFailure { +public class ForbiddenException extends RESTException implements CleanableFailure { @FormatMethod public ForbiddenException(String message, Object... args) { - super(String.format(message, args)); + super(message, args); } @FormatMethod public ForbiddenException(Throwable cause, String message, Object... args) { - super(String.format(message, args), cause); + super(cause, message, args); } } 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 87519cfd9dd2..742caa9494f4 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -59,6 +59,7 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; @@ -717,8 +718,15 @@ public Transaction createTransaction() { @Override public Transaction replaceTransaction() { - if (viewExists(context, ident)) { - throw new AlreadyExistsException("View with same name already exists: %s", ident); + try { + if (viewExists(context, ident)) { + throw new AlreadyExistsException("View with same name already exists: %s", ident); + } + } catch (RESTException | UnsupportedOperationException e) { + // don't fail if the server doesn't support views, which could be due to: + // 1. server or backing catalog doesn't support views + // 2. newer client talks to an older server that doesn't support views + LOG.debug("Failed to check whether view {} exists", ident, e); } LoadTableResponse response = loadInternal(context, ident, snapshotMode); From 45fb717a3464ac202c45810d63f6a894af41c454 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 20 Feb 2024 17:17:46 +0900 Subject: [PATCH 0057/1019] Spark: Migrate tests to JUnit5 (#9670) --- spark/v3.5/build.gradle | 1 + .../SparkRowLevelOperationsTestBase.java | 101 +++--- .../extensions/TestConflictValidation.java | 43 ++- .../extensions/TestCopyOnWriteDelete.java | 43 +-- .../extensions/TestCopyOnWriteMerge.java | 40 +-- .../extensions/TestCopyOnWriteUpdate.java | 40 +-- .../iceberg/spark/extensions/TestDelete.java | 298 +++++++++--------- .../iceberg/spark/extensions/TestMerge.java | 276 ++++++++-------- .../extensions/TestMergeOnReadDelete.java | 50 +-- .../extensions/TestMergeOnReadMerge.java | 29 +- .../extensions/TestMergeOnReadUpdate.java | 32 +- .../iceberg/spark/extensions/TestUpdate.java | 274 ++++++++-------- .../spark/extensions/TestWriteAborts.java | 27 +- 13 files changed, 531 insertions(+), 723 deletions(-) diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 5b07a3c3847d..d4cb67a2bbf3 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -166,6 +166,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop + testImplementation libs.awaitility // Required because we remove antlr plugin dependencies from the compile configuration, see note above runtimeOnly libs.antlr.runtime 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 e4605a765823..ea1040dcf0ac 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 @@ -45,11 +45,15 @@ import java.util.Map; import java.util.Random; import java.util.Set; +import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; @@ -69,41 +73,30 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; -import org.junit.Assert; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public abstract class SparkRowLevelOperationsTestBase extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public abstract class SparkRowLevelOperationsTestBase extends ExtensionsTestBase { private static final Random RANDOM = ThreadLocalRandom.current(); - protected final String fileFormat; - protected final boolean vectorized; - protected final String distributionMode; - protected final boolean fanoutEnabled; - protected final String branch; - protected final PlanningMode planningMode; - - public SparkRowLevelOperationsTestBase( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super(catalogName, implementation, config); - this.fileFormat = fileFormat; - this.vectorized = vectorized; - this.distributionMode = distributionMode; - this.fanoutEnabled = fanoutEnabled; - this.branch = branch; - this.planningMode = planningMode; - } + @Parameter(index = 3) + protected FileFormat fileFormat; + + @Parameter(index = 4) + protected boolean vectorized; + + @Parameter(index = 5) + protected String distributionMode; + + @Parameter(index = 6) + protected boolean fanoutEnabled; + + @Parameter(index = 7) + protected String branch; + + @Parameter(index = 8) + protected PlanningMode planningMode; @Parameters( name = @@ -118,7 +111,7 @@ public static Object[][] parameters() { ImmutableMap.of( "type", "hive", "default-namespace", "default"), - "orc", + FileFormat.ORC, true, WRITE_DISTRIBUTION_MODE_NONE, true, @@ -131,7 +124,7 @@ public static Object[][] parameters() { ImmutableMap.of( "type", "hive", "default-namespace", "default"), - "parquet", + FileFormat.PARQUET, true, WRITE_DISTRIBUTION_MODE_NONE, false, @@ -142,7 +135,7 @@ public static Object[][] parameters() { "testhadoop", SparkCatalog.class.getName(), ImmutableMap.of("type", "hadoop"), - "parquet", + FileFormat.PARQUET, RANDOM.nextBoolean(), WRITE_DISTRIBUTION_MODE_HASH, true, @@ -160,7 +153,7 @@ public static Object[][] parameters() { "cache-enabled", "false" // Spark will delete tables using v1, leaving the cache out of sync ), - "avro", + FileFormat.AVRO, false, WRITE_DISTRIBUTION_MODE_RANGE, false, @@ -188,18 +181,18 @@ protected void initTable() { planningMode.modeName()); switch (fileFormat) { - case "parquet": + case PARQUET: sql( "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')", tableName, PARQUET_VECTORIZATION_ENABLED, vectorized); break; - case "orc": + case ORC: sql( "ALTER TABLE %s SET TBLPROPERTIES('%s' '%b')", tableName, ORC_VECTORIZATION_ENABLED, vectorized); break; - case "avro": - Assert.assertFalse(vectorized); + case AVRO: + assertThat(vectorized).isFalse(); break; } @@ -303,7 +296,7 @@ protected void validateSnapshot( String deletedDataFiles, String addedDeleteFiles, String addedDataFiles) { - Assert.assertEquals("Operation must match", operation, snapshot.operation()); + assertThat(snapshot.operation()).as("Operation must match").isEqualTo(operation); validateProperty(snapshot, CHANGED_PARTITION_COUNT_PROP, changedPartitionCount); validateProperty(snapshot, DELETED_FILES_PROP, deletedDataFiles); validateProperty(snapshot, ADDED_DELETE_FILES_PROP, addedDeleteFiles); @@ -312,20 +305,22 @@ protected void validateSnapshot( protected void validateProperty(Snapshot snapshot, String property, Set expectedValues) { String actual = snapshot.summary().get(property); - Assert.assertTrue( - "Snapshot property " - + property - + " has unexpected value, actual = " - + actual - + ", expected one of : " - + String.join(",", expectedValues), - expectedValues.contains(actual)); + assertThat(actual) + .as( + "Snapshot property " + + property + + " has unexpected value, actual = " + + actual + + ", expected one of : " + + String.join(",", expectedValues)) + .isIn(expectedValues); } protected void validateProperty(Snapshot snapshot, String property, String expectedValue) { String actual = snapshot.summary().get(property); - Assert.assertEquals( - "Snapshot property " + property + " has unexpected value.", expectedValue, actual); + assertThat(actual) + .as("Snapshot property " + property + " has unexpected value.") + .isEqualTo(expectedValue); } protected void sleep(long millis) { @@ -338,7 +333,9 @@ protected void sleep(long millis) { protected DataFile writeDataFile(Table table, List records) { try { - OutputFile file = Files.localOutput(temp.newFile()); + OutputFile file = + Files.localOutput( + temp.resolve(fileFormat.addExtension(UUID.randomUUID().toString())).toFile()); DataWriter dataWriter = Parquet.writeData(file) @@ -384,7 +381,7 @@ protected boolean supportsVectorization() { } private boolean isParquet() { - return fileFormat.equalsIgnoreCase(FileFormat.PARQUET.name()); + return fileFormat.equals(FileFormat.PARQUET); } private boolean isCopyOnWrite() { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java index aaa275db9c16..cc2bd91cf802 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -19,8 +19,8 @@ package org.apache.iceberg.spark.extensions; import java.util.List; -import java.util.Map; import org.apache.iceberg.IsolationLevel; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -30,18 +30,15 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.functions; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +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; -public class TestConflictValidation extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestConflictValidation extends ExtensionsTestBase { - public TestConflictValidation( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before + @BeforeEach public void createTables() { sql( "CREATE TABLE %s (id int, data string) USING iceberg " @@ -53,12 +50,12 @@ public void createTables() { sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testOverwriteFilterSerializableIsolation() throws Exception { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); @@ -90,7 +87,7 @@ public void testOverwriteFilterSerializableIsolation() throws Exception { .overwrite(functions.col("id").equalTo(1)); } - @Test + @TestTemplate public void testOverwriteFilterSerializableIsolation2() throws Exception { List records = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "b")); @@ -127,7 +124,7 @@ public void testOverwriteFilterSerializableIsolation2() throws Exception { .overwrite(functions.col("id").equalTo(1)); } - @Test + @TestTemplate public void testOverwriteFilterSerializableIsolation3() throws Exception { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); @@ -161,7 +158,7 @@ public void testOverwriteFilterSerializableIsolation3() throws Exception { .overwrite(functions.col("id").equalTo(1)); } - @Test + @TestTemplate public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { Table table = validationCatalog.loadTable(tableIdent); @@ -192,7 +189,7 @@ public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { .overwrite(functions.col("id").equalTo(1)); } - @Test + @TestTemplate public void testOverwriteFilterSnapshotIsolation() throws Exception { List records = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "b")); @@ -229,7 +226,7 @@ public void testOverwriteFilterSnapshotIsolation() throws Exception { .overwrite(functions.col("id").equalTo(1)); } - @Test + @TestTemplate public void testOverwriteFilterSnapshotIsolation2() throws Exception { Table table = validationCatalog.loadTable(tableIdent); long snapshotId = table.currentSnapshot().snapshotId(); @@ -246,7 +243,7 @@ public void testOverwriteFilterSnapshotIsolation2() throws Exception { .overwrite(functions.col("id").equalTo(1)); } - @Test + @TestTemplate public void testOverwritePartitionSerializableIsolation() throws Exception { Table table = validationCatalog.loadTable(tableIdent); final long snapshotId = table.currentSnapshot().snapshotId(); @@ -278,7 +275,7 @@ public void testOverwritePartitionSerializableIsolation() throws Exception { .overwritePartitions(); } - @Test + @TestTemplate public void testOverwritePartitionSnapshotIsolation() throws Exception { List records = Lists.newArrayList(new SimpleRecord(1, "a"), new SimpleRecord(1, "b")); @@ -313,7 +310,7 @@ public void testOverwritePartitionSnapshotIsolation() throws Exception { .overwritePartitions(); } - @Test + @TestTemplate public void testOverwritePartitionSnapshotIsolation2() throws Exception { Table table = validationCatalog.loadTable(tableIdent); final long snapshotId = table.currentSnapshot().snapshotId(); @@ -347,7 +344,7 @@ public void testOverwritePartitionSnapshotIsolation2() throws Exception { .overwritePartitions(); } - @Test + @TestTemplate public void testOverwritePartitionSnapshotIsolation3() throws Exception { Table table = validationCatalog.loadTable(tableIdent); final long snapshotId = table.currentSnapshot().snapshotId(); @@ -364,7 +361,7 @@ public void testOverwritePartitionSnapshotIsolation3() throws Exception { .overwritePartitions(); } - @Test + @TestTemplate public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { Table table = validationCatalog.loadTable(tableIdent); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java index 9ebe73da3323..edddb3bbb8fd 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Collections; import java.util.Map; @@ -32,7 +34,7 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; -import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -40,7 +42,6 @@ import org.apache.iceberg.data.GenericRecord; 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.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSQLProperties; @@ -49,45 +50,23 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) public class TestCopyOnWriteDelete extends TestDelete { - public TestCopyOnWriteDelete( - String catalogName, - String implementation, - Map config, - String fileFormat, - Boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - @Override protected Map extraTableProperties() { return ImmutableMap.of( TableProperties.DELETE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); } - @Test + @TestTemplate public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception { // this test can only be run with Hive tables as it requires a reliable lock // also, the table cache must be enabled so that the same table instance can be reused - Assume.assumeTrue(catalogName.equalsIgnoreCase("testhive")); + assumeThat(catalogName).isEqualToIgnoringCase("testhive"); createAndInitUnpartitionedTable(); createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); @@ -167,10 +146,10 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public void testRuntimeFilteringWithPreservedDataGrouping() throws NoSuchTableException { createAndInitPartitionedTable(); @@ -188,7 +167,7 @@ public void testRuntimeFilteringWithPreservedDataGrouping() throws NoSuchTableEx withSQLConf(sqlConf, () -> sql("DELETE FROM %s WHERE id = 2", commitTarget())); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); validateCopyOnWrite(currentSnapshot, "1", "1", "1"); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index 6b6819a924e8..dfd22e96ddc1 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Collections; import java.util.Map; @@ -31,7 +33,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.DataFile; -import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -39,7 +40,6 @@ import org.apache.iceberg.data.GenericRecord; 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.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSQLProperties; @@ -47,45 +47,21 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; public class TestCopyOnWriteMerge extends TestMerge { - public TestCopyOnWriteMerge( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - @Override protected Map extraTableProperties() { return ImmutableMap.of( TableProperties.MERGE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); } - @Test + @TestTemplate public synchronized void testMergeWithConcurrentTableRefresh() throws Exception { // this test can only be run with Hive tables as it requires a reliable lock // also, the table cache must be enabled so that the same table instance can be reused - Assume.assumeTrue(catalogName.equalsIgnoreCase("testhive")); + assumeThat(catalogName).isEqualToIgnoringCase("testhive"); createAndInitTable("id INT, dep STRING"); createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); @@ -165,10 +141,10 @@ public synchronized void testMergeWithConcurrentTableRefresh() throws Exception } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public void testRuntimeFilteringWithReportedPartitioning() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -199,7 +175,7 @@ public void testRuntimeFilteringWithReportedPartitioning() { commitTarget())); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); validateCopyOnWrite(currentSnapshot, "1", "1", "1"); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java index 4354a1019c69..e9767c66bafc 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -31,7 +33,6 @@ import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; -import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -39,52 +40,27 @@ import org.apache.iceberg.data.GenericRecord; 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.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; public class TestCopyOnWriteUpdate extends TestUpdate { - public TestCopyOnWriteUpdate( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - @Override protected Map extraTableProperties() { return ImmutableMap.of( TableProperties.UPDATE_MODE, RowLevelOperationMode.COPY_ON_WRITE.modeName()); } - @Test + @TestTemplate public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception { // this test can only be run with Hive tables as it requires a reliable lock // also, the table cache must be enabled so that the same table instance can be reused - Assume.assumeTrue(catalogName.equalsIgnoreCase("testhive")); + assumeThat(catalogName).isEqualToIgnoringCase("testhive"); createAndInitTable("id INT, dep STRING"); @@ -163,10 +139,10 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public void testRuntimeFilteringWithReportedPartitioning() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -187,7 +163,7 @@ public void testRuntimeFilteringWithReportedPartitioning() { withSQLConf(sqlConf, () -> sql("UPDATE %s SET id = -1 WHERE id = 2", commitTarget())); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); validateCopyOnWrite(currentSnapshot, "1", "1", "1"); 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 9bc46d05713f..e9a8c13be56a 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 @@ -27,6 +27,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; @@ -45,8 +46,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; @@ -57,7 +59,6 @@ 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; -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.apache.iceberg.spark.Spark3Util; @@ -77,42 +78,21 @@ import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromTable; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Test; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestDelete extends SparkRowLevelOperationsTestBase { - public TestDelete( - String catalogName, - String implementation, - Map config, - String fileFormat, - Boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - - @BeforeClass + @BeforeAll public static void setupSparkConf() { spark.conf().set("spark.sql.shuffle.partitions", "4"); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS deleted_id"); @@ -120,7 +100,7 @@ public void removeTables() { sql("DROP TABLE IF EXISTS parquet_table"); } - @Test + @TestTemplate public void testDeleteWithVectorizedReads() throws NoSuchTableException { assumeThat(supportsVectorization()).isTrue(); @@ -135,7 +115,7 @@ public void testDeleteWithVectorizedReads() throws NoSuchTableException { assertAllBatchScansVectorized(plan); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -150,7 +130,7 @@ public void testDeleteWithVectorizedReads() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id ASC", selectTarget())); } - @Test + @TestTemplate public void testCoalesceDelete() throws Exception { createAndInitUnpartitionedTable(); @@ -193,7 +173,7 @@ public void testCoalesceDelete() throws Exception { () -> { SparkPlan plan = executeAndKeepPlan("DELETE FROM %s WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -214,11 +194,12 @@ public void testCoalesceDelete() throws Exception { validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "1"); } - Assert.assertEquals( - "Row count must match", 200L, scalarSql("SELECT COUNT(*) FROM %s", commitTarget())); + assertThat(scalarSql("SELECT COUNT(*) FROM %s", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); } - @Test + @TestTemplate public void testSkewDelete() throws Exception { createAndInitPartitionedTable(); @@ -261,7 +242,7 @@ public void testSkewDelete() throws Exception { () -> { SparkPlan plan = executeAndKeepPlan("DELETE FROM %s WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -283,11 +264,12 @@ public void testSkewDelete() throws Exception { validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); } - Assert.assertEquals( - "Row count must match", 200L, scalarSql("SELECT COUNT(*) FROM %s", commitTarget())); + assertThat(scalarSql("SELECT COUNT(*) FROM %s", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); } - @Test + @TestTemplate public void testDeleteWithoutScanningTable() throws Exception { createAndInitPartitionedTable(); @@ -308,10 +290,10 @@ public void testDeleteWithoutScanningTable() throws Exception { LogicalPlan parsed = parsePlan("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); LogicalPlan analyzed = spark.sessionState().analyzer().execute(parsed); - Assertions.assertThat(analyzed).isInstanceOf(RowLevelWrite.class); + assertThat(analyzed).isInstanceOf(RowLevelWrite.class); LogicalPlan optimized = OptimizeMetadataOnlyDeleteFromTable.apply(analyzed); - Assertions.assertThat(optimized).isInstanceOf(DeleteFromTableWithFilters.class); + assertThat(optimized).isInstanceOf(DeleteFromTableWithFilters.class); }); sql("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); @@ -322,9 +304,11 @@ public void testDeleteWithoutScanningTable() throws Exception { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDeleteFileThenMetadataDelete() throws Exception { - Assume.assumeFalse("Avro does not support metadata delete", fileFormat.equals("avro")); + assumeThat(fileFormat) + .as("Avro does not support metadata delete") + .isNotEqualTo(FileFormat.AVRO); createAndInitUnpartitionedTable(); createBranchIfNeeded(); sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware'), (null, 'hr')", commitTarget()); @@ -339,8 +323,9 @@ public void testDeleteFileThenMetadataDelete() throws Exception { sql("DELETE FROM %s AS t WHERE t.id = 1", commitTarget()); List dataFilesAfter = TestHelpers.dataFiles(table, branch); - Assert.assertTrue( - "Data file should have been removed", dataFilesBefore.size() > dataFilesAfter.size()); + assertThat(dataFilesAfter) + .as("Data file should have been removed") + .hasSizeLessThan(dataFilesBefore.size()); assertEquals( "Should have expected rows", @@ -348,7 +333,7 @@ public void testDeleteFileThenMetadataDelete() throws Exception { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithPartitionedTable() throws Exception { createAndInitPartitionedTable(); @@ -364,8 +349,9 @@ public void testDeleteWithPartitionedTable() throws Exception { sql("SELECT * FROM %s ORDER BY id", tableName)); List rowLevelDeletePartitions = spark.sql("SELECT * FROM " + tableName + ".partitions ").collectAsList(); - Assert.assertEquals( - "row level delete does not reduce number of partition", 2, rowLevelDeletePartitions.size()); + assertThat(rowLevelDeletePartitions) + .as("row level delete does not reduce number of partition") + .hasSize(2); // partition aligned delete sql("DELETE FROM %s WHERE dep = 'hr'", tableName); @@ -376,11 +362,10 @@ public void testDeleteWithPartitionedTable() throws Exception { sql("SELECT * FROM %s ORDER BY id", tableName)); List actualPartitions = spark.sql("SELECT * FROM " + tableName + ".partitions ").collectAsList(); - Assert.assertEquals( - "partition aligned delete results in 1 partition", 1, actualPartitions.size()); + assertThat(actualPartitions).as("partition aligned delete results in 1 partition").hasSize(1); } - @Test + @TestTemplate public void testDeleteWithFalseCondition() { createAndInitUnpartitionedTable(); @@ -390,7 +375,7 @@ public void testDeleteWithFalseCondition() { sql("DELETE FROM %s WHERE id = 1 AND id > 20", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); assertEquals( "Should have expected rows", @@ -398,16 +383,16 @@ public void testDeleteWithFalseCondition() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDeleteFromEmptyTable() { - Assume.assumeFalse("Custom branch does not exist for empty table", "test".equals(branch)); + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); createAndInitUnpartitionedTable(); sql("DELETE FROM %s WHERE id IN (1)", commitTarget()); sql("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); assertEquals( "Should have expected rows", @@ -415,9 +400,9 @@ public void testDeleteFromEmptyTable() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDeleteFromNonExistingCustomBranch() { - Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); + assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); createAndInitUnpartitionedTable(); Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) @@ -425,7 +410,7 @@ public void testDeleteFromNonExistingCustomBranch() { .hasMessage("Cannot use branch (does not exist): test"); } - @Test + @TestTemplate public void testExplain() { createAndInitUnpartitionedTable(); @@ -437,7 +422,7 @@ public void testExplain() { sql("EXPLAIN DELETE FROM %s WHERE true", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 1 snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); assertEquals( "Should have expected rows", @@ -445,7 +430,7 @@ public void testExplain() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", commitTarget())); } - @Test + @TestTemplate public void testDeleteWithAlias() { createAndInitUnpartitionedTable(); @@ -460,7 +445,7 @@ public void testDeleteWithAlias() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { createAndInitPartitionedTable(); @@ -471,7 +456,7 @@ public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { sql("DELETE FROM %s WHERE id = 2", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -486,7 +471,7 @@ public void testDeleteWithDynamicFileFiltering() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); } - @Test + @TestTemplate public void testDeleteNonExistingRecords() { createAndInitPartitionedTable(); @@ -496,11 +481,11 @@ public void testDeleteNonExistingRecords() { sql("DELETE FROM %s AS t WHERE t.id > 10", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); - if (fileFormat.equals("orc") || fileFormat.equals("parquet")) { + if (fileFormat.equals(FileFormat.ORC) || fileFormat.equals(FileFormat.PARQUET)) { validateDelete(currentSnapshot, "0", null); } else { if (mode(table) == COPY_ON_WRITE) { @@ -516,7 +501,7 @@ public void testDeleteNonExistingRecords() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithoutCondition() { createAndInitPartitionedTable(); @@ -528,7 +513,7 @@ public void testDeleteWithoutCondition() { sql("DELETE FROM %s", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); // should be a delete instead of an overwrite as it is done through a metadata operation Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); @@ -538,7 +523,7 @@ public void testDeleteWithoutCondition() { "Should have expected rows", ImmutableList.of(), sql("SELECT * FROM %s", commitTarget())); } - @Test + @TestTemplate public void testDeleteUsingMetadataWithComplexCondition() { createAndInitPartitionedTable(); @@ -552,7 +537,7 @@ public void testDeleteUsingMetadataWithComplexCondition() { commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); // should be a delete instead of an overwrite as it is done through a metadata operation Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); @@ -564,7 +549,7 @@ public void testDeleteUsingMetadataWithComplexCondition() { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithArbitraryPartitionPredicates() { createAndInitPartitionedTable(); @@ -577,7 +562,7 @@ public void testDeleteWithArbitraryPartitionPredicates() { sql("DELETE FROM %s WHERE id = 10 OR dep LIKE '%%ware'", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); // should be an overwrite since cannot be executed using a metadata operation Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); @@ -593,7 +578,7 @@ public void testDeleteWithArbitraryPartitionPredicates() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithNonDeterministicCondition() { createAndInitPartitionedTable(); @@ -606,7 +591,7 @@ public void testDeleteWithNonDeterministicCondition() { .hasMessageContaining("The operator expects a deterministic expression"); } - @Test + @TestTemplate public void testDeleteWithFoldableConditions() { createAndInitPartitionedTable(); @@ -642,10 +627,10 @@ public void testDeleteWithFoldableConditions() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); } - @Test + @TestTemplate public void testDeleteWithNullConditions() { createAndInitPartitionedTable(); @@ -677,13 +662,13 @@ public void testDeleteWithNullConditions() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); validateDelete(currentSnapshot, "1", "1"); } - @Test + @TestTemplate public void testDeleteWithInAndNotInConditions() { createAndInitUnpartitionedTable(); @@ -709,9 +694,9 @@ public void testDeleteWithInAndNotInConditions() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithMultipleRowGroupsParquet() throws NoSuchTableException { - Assume.assumeTrue(fileFormat.equalsIgnoreCase("parquet")); + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); createAndInitPartitionedTable(); @@ -732,15 +717,15 @@ public void testDeleteWithMultipleRowGroupsParquet() throws NoSuchTableException df.coalesce(1).writeTo(tableName).append(); createBranchIfNeeded(); - Assert.assertEquals(200, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); // delete a record from one of two row groups and copy over the second one sql("DELETE FROM %s WHERE id IN (200, 201)", commitTarget()); - Assert.assertEquals(199, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()).isEqualTo(199); } - @Test + @TestTemplate public void testDeleteWithConditionOnNestedColumn() { createAndInitNestedColumnsTable(); @@ -759,7 +744,7 @@ public void testDeleteWithConditionOnNestedColumn() { "Should have expected rows", ImmutableList.of(), sql("SELECT id FROM %s", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithInSubquery() throws NoSuchTableException { createAndInitUnpartitionedTable(); @@ -806,7 +791,7 @@ public void testDeleteWithInSubquery() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithMultiColumnInSubquery() throws NoSuchTableException { createAndInitUnpartitionedTable(); @@ -824,7 +809,7 @@ public void testDeleteWithMultiColumnInSubquery() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithNotInSubquery() throws NoSuchTableException { createAndInitUnpartitionedTable(); @@ -884,9 +869,9 @@ public void testDeleteWithNotInSubquery() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteOnNonIcebergTableNotSupported() { - Assume.assumeTrue(catalogName.equalsIgnoreCase("spark_catalog")); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); sql("CREATE TABLE parquet_table (c1 INT, c2 INT) USING parquet"); @@ -895,7 +880,7 @@ public void testDeleteOnNonIcebergTableNotSupported() { .hasMessageContaining("does not support DELETE"); } - @Test + @TestTemplate public void testDeleteWithExistSubquery() throws NoSuchTableException { createAndInitUnpartitionedTable(); @@ -940,7 +925,7 @@ public void testDeleteWithExistSubquery() throws NoSuchTableException { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithNotExistsSubquery() throws NoSuchTableException { createAndInitUnpartitionedTable(); @@ -976,7 +961,7 @@ public void testDeleteWithNotExistsSubquery() throws NoSuchTableException { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testDeleteWithScalarSubquery() throws NoSuchTableException { createAndInitUnpartitionedTable(); @@ -997,7 +982,7 @@ public void testDeleteWithScalarSubquery() throws NoSuchTableException { }); } - @Test + @TestTemplate public void testDeleteThatRequiresGroupingBeforeWrite() throws NoSuchTableException { createAndInitPartitionedTable(); @@ -1015,20 +1000,21 @@ public void testDeleteThatRequiresGroupingBeforeWrite() throws NoSuchTableExcept spark.conf().set("spark.sql.shuffle.partitions", "1"); sql("DELETE FROM %s t WHERE id IN (SELECT * FROM deleted_id)", commitTarget()); - Assert.assertEquals( - "Should have expected num of rows", 8L, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()) + .as("Should have expected num of rows") + .isEqualTo(8L); } finally { spark.conf().set("spark.sql.shuffle.partitions", originalNumOfShufflePartitions); } } - @Test + @TestTemplate public synchronized void testDeleteWithSerializableIsolation() throws InterruptedException { // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); // if caching is off, the table is eagerly refreshed during runtime filtering // this can cause a validation exception as concurrent changes would be visible - Assume.assumeTrue(cachingCatalogEnabled()); + assumeThat(cachingCatalogEnabled()).isTrue(); createAndInitUnpartitionedTable(); createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); @@ -1052,9 +1038,11 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte executorService.submit( () -> { for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> barrier.get() >= currentNumOperations * 2); sql("DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", commitTarget()); @@ -1074,9 +1062,11 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte record.set(1, "hr"); // dep for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); if (!shouldAppend.get()) { return; @@ -1090,7 +1080,6 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte } appendFiles.commit(); - sleep(10); } barrier.incrementAndGet(); @@ -1109,17 +1098,17 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public synchronized void testDeleteWithSnapshotIsolation() throws InterruptedException, ExecutionException { // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); // if caching is off, the table is eagerly refreshed during runtime filtering // this can cause a validation exception as concurrent changes would be visible - Assume.assumeTrue(cachingCatalogEnabled()); + assumeThat(cachingCatalogEnabled()).isTrue(); createAndInitUnpartitionedTable(); createOrReplaceView("deleted_id", Collections.singletonList(1), Encoders.INT()); @@ -1143,9 +1132,11 @@ public synchronized void testDeleteWithSnapshotIsolation() executorService.submit( () -> { for (int numOperations = 0; numOperations < 20; numOperations++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> barrier.get() >= currentNumOperations * 2); sql("DELETE FROM %s WHERE id IN (SELECT * FROM deleted_id)", commitTarget()); @@ -1165,9 +1156,11 @@ public synchronized void testDeleteWithSnapshotIsolation() record.set(1, "hr"); // dep for (int numOperations = 0; numOperations < 20; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); if (!shouldAppend.get()) { return; @@ -1181,7 +1174,6 @@ public synchronized void testDeleteWithSnapshotIsolation() } appendFiles.commit(); - sleep(10); } barrier.incrementAndGet(); @@ -1196,10 +1188,10 @@ public synchronized void testDeleteWithSnapshotIsolation() } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public void testDeleteRefreshesRelationCache() throws NoSuchTableException { createAndInitPartitionedTable(); @@ -1220,7 +1212,7 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException { sql("DELETE FROM %s WHERE id = 1", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -1241,7 +1233,7 @@ public void testDeleteRefreshesRelationCache() throws NoSuchTableException { spark.sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testDeleteWithMultipleSpecs() { createAndInitTable("id INT, dep STRING, category STRING"); @@ -1267,7 +1259,7 @@ public void testDeleteWithMultipleSpecs() { sql("DELETE FROM %s WHERE id IN (1, 3, 5, 7)", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 5 snapshots", 5, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 5 snapshots").hasSize(5); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -1282,9 +1274,9 @@ public void testDeleteWithMultipleSpecs() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testDeleteToWapBranch() throws NoSuchTableException { - Assume.assumeTrue("WAP branch only works for table identifier without branch", branch == null); + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); createAndInitPartitionedTable(); sql( @@ -1296,40 +1288,36 @@ public void testDeleteToWapBranch() throws NoSuchTableException { ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> { sql("DELETE FROM %s t WHERE id=0", tableName); - Assert.assertEquals( - "Should have expected num of rows when reading table", - 2L, - spark.table(tableName).count()); - Assert.assertEquals( - "Should have expected num of rows when reading WAP branch", - 2L, - spark.table(tableName + ".branch_wap").count()); - Assert.assertEquals( - "Should not modify main branch", 3L, spark.table(tableName + ".branch_main").count()); + assertThat(spark.table(tableName).count()) + .as("Should have expected num of rows when reading table") + .isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_wap").count()) + .as("Should have expected num of rows when reading WAP branch") + .isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch") + .isEqualTo(3L); }); withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> { sql("DELETE FROM %s t WHERE id=1", tableName); - Assert.assertEquals( - "Should have expected num of rows when reading table with multiple writes", - 1L, - spark.table(tableName).count()); - Assert.assertEquals( - "Should have expected num of rows when reading WAP branch with multiple writes", - 1L, - spark.table(tableName + ".branch_wap").count()); - Assert.assertEquals( - "Should not modify main branch with multiple writes", - 3L, - spark.table(tableName + ".branch_main").count()); + assertThat(spark.table(tableName).count()) + .as("Should have expected num of rows when reading table with multiple writes") + .isEqualTo(1L); + assertThat(spark.table(tableName + ".branch_wap").count()) + .as("Should have expected num of rows when reading WAP branch with multiple writes") + .isEqualTo(1L); + assertThat(spark.table(tableName + ".branch_main").count()) + .as("Should not modify main branch with multiple writes") + .isEqualTo(3L); }); } - @Test + @TestTemplate public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableException { - Assume.assumeTrue("Test must have branch name part in table identifier", branch != null); + assumeThat(branch).as("Test must have branch name part in table identifier").isNotNull(); createAndInitPartitionedTable(); sql( @@ -1349,7 +1337,7 @@ public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableE branch))); } - @Test + @TestTemplate public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableException { assumeThat(branch) .as("Run only if custom WAP branch is not main") @@ -1367,9 +1355,9 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), () -> { sql("DELETE FROM %s t WHERE id=1", tableName); - Assertions.assertThat(spark.table(tableName).count()).isEqualTo(2L); - Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); - Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + assertThat(spark.table(tableName).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_main").count()) .as("Should not modify main branch") .isEqualTo(3L); }); @@ -1377,9 +1365,9 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), () -> { sql("DELETE FROM %s t", tableName); - Assertions.assertThat(spark.table(tableName).count()).isEqualTo(0L); - Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); - Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + assertThat(spark.table(tableName).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_main").count()) .as("Should not modify main branch") .isEqualTo(3L); }); 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 2694d522c913..9f8ac9c993c2 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 @@ -28,6 +28,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; @@ -45,7 +46,7 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; @@ -69,48 +70,25 @@ import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Test; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; public abstract class TestMerge extends SparkRowLevelOperationsTestBase { - public TestMerge( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - - @BeforeClass + @BeforeAll public static void setupSparkConf() { spark.conf().set("spark.sql.shuffle.partitions", "4"); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS source"); } - @Test + @TestTemplate public void testMergeWithAllClauses() { createAndInitTable( "id INT, dep STRING", @@ -152,7 +130,7 @@ public void testMergeWithAllClauses() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOneNotMatchedBySourceClause() { createAndInitTable( "id INT, dep STRING", @@ -180,7 +158,7 @@ public void testMergeWithOneNotMatchedBySourceClause() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeNotMatchedBySourceClausesPartitionedTable() { createAndInitTable( "id INT, dep STRING", @@ -209,7 +187,7 @@ public void testMergeNotMatchedBySourceClausesPartitionedTable() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithVectorizedReads() { assumeThat(supportsVectorization()).isTrue(); @@ -251,7 +229,7 @@ public void testMergeWithVectorizedReads() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testCoalesceMerge() { createAndInitTable("id INT, salary INT, dep STRING"); @@ -322,13 +300,12 @@ public void testCoalesceMerge() { validateProperty(currentSnapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "1"); } - Assert.assertEquals( - "Row count must match", - 400L, - scalarSql("SELECT COUNT(*) FROM %s WHERE salary = -1", commitTarget())); + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE salary = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(400L); } - @Test + @TestTemplate public void testSkewMerge() { createAndInitTable("id INT, salary INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -382,7 +359,7 @@ public void testSkewMerge() { + "WHEN MATCHED THEN " + " UPDATE SET salary = -1 ", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -406,13 +383,12 @@ public void testSkewMerge() { validateProperty(currentSnapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); } - Assert.assertEquals( - "Row count must match", - 400L, - scalarSql("SELECT COUNT(*) FROM %s WHERE salary = -1", commitTarget())); + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE salary = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(400L); } - @Test + @TestTemplate public void testMergeConditionSplitIntoTargetPredicateAndJoinCondition() { createAndInitTable( "id INT, salary INT, dep STRING, sub_dep STRING", @@ -461,7 +437,7 @@ public void testMergeConditionSplitIntoTargetPredicateAndJoinCondition() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithStaticPredicatePushDown() { createAndInitTable("id BIGINT, dep STRING"); @@ -478,7 +454,7 @@ public void testMergeWithStaticPredicatePushDown() { Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); String dataFilesCount = snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP); - Assert.assertEquals("Must have 2 files before MERGE", "2", dataFilesCount); + assertThat(dataFilesCount).as("Must have 2 files before MERGE").isEqualTo("2"); createOrReplaceView( "source", "{ \"id\": 1, \"dep\": \"finance\" }\n" + "{ \"id\": 2, \"dep\": \"hardware\" }"); @@ -516,9 +492,9 @@ public void testMergeWithStaticPredicatePushDown() { sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); } - @Test + @TestTemplate public void testMergeIntoEmptyTargetInsertAllNonMatchingRows() { - Assume.assumeFalse("Custom branch does not exist for empty table", "test".equals(branch)); + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); createAndInitTable("id INT, dep STRING"); createOrReplaceView( @@ -547,9 +523,9 @@ public void testMergeIntoEmptyTargetInsertAllNonMatchingRows() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeIntoEmptyTargetInsertOnlyMatchingRows() { - Assume.assumeFalse("Custom branch does not exist for empty table", "test".equals(branch)); + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); createAndInitTable("id INT, dep STRING"); createOrReplaceView( @@ -577,7 +553,7 @@ public void testMergeIntoEmptyTargetInsertOnlyMatchingRows() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOnlyUpdateClause() { createAndInitTable( "id INT, dep STRING", @@ -608,7 +584,7 @@ public void testMergeWithOnlyUpdateClause() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOnlyUpdateNullUnmatchedValues() { createAndInitTable( "id INT, value INT", "{ \"id\": 1, \"value\": 2 }\n" + "{ \"id\": 6, \"value\": null }"); @@ -634,7 +610,7 @@ public void testMergeWithOnlyUpdateNullUnmatchedValues() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOnlyUpdateSingleFieldNullUnmatchedValues() { createAndInitTable( "id INT, value INT", "{ \"id\": 1, \"value\": 2 }\n" + "{ \"id\": 6, \"value\": null }"); @@ -660,7 +636,7 @@ public void testMergeWithOnlyUpdateSingleFieldNullUnmatchedValues() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOnlyDeleteNullUnmatchedValues() { createAndInitTable( "id INT, value INT", "{ \"id\": 1, \"value\": 2 }\n" + "{ \"id\": 6, \"value\": null }"); @@ -680,7 +656,7 @@ public void testMergeWithOnlyDeleteNullUnmatchedValues() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOnlyUpdateClauseAndNullValues() { createAndInitTable( "id INT, dep STRING", @@ -713,7 +689,7 @@ public void testMergeWithOnlyUpdateClauseAndNullValues() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOnlyDeleteClause() { createAndInitTable( "id INT, dep STRING", @@ -743,7 +719,7 @@ public void testMergeWithOnlyDeleteClause() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMatchedAndNotMatchedClauses() { createAndInitTable( "id INT, dep STRING", @@ -778,7 +754,7 @@ public void testMergeWithMatchedAndNotMatchedClauses() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithAllCausesWithExplicitColumnSpecification() { createAndInitTable( "id INT, dep STRING", @@ -813,7 +789,7 @@ public void testMergeWithAllCausesWithExplicitColumnSpecification() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithSourceCTE() { createAndInitTable( "id INT, dep STRING", @@ -849,7 +825,7 @@ public void testMergeWithSourceCTE() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithSourceFromSetOps() { createAndInitTable( "id INT, dep STRING", @@ -889,7 +865,7 @@ public void testMergeWithSourceFromSetOps() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithOneMatchingBranchButMultipleSourceRowsForTargetRow() { createAndInitTable( "id INT, dep STRING", @@ -924,7 +900,7 @@ public void testMergeWithOneMatchingBranchButMultipleSourceRowsForTargetRow() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { createAndInitTable( "id INT, dep STRING", @@ -961,7 +937,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceEnabledHashShuffleJoin() { createAndInitTable( @@ -1003,7 +979,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqualityCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"emp-id-one\" }"); @@ -1042,7 +1018,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqua sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActions() { createAndInitTable( "id INT, dep STRING", @@ -1076,7 +1052,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotMatchedActionsNoEqualityCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"emp-id-one\" }"); @@ -1110,7 +1086,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleUpdatesForTargetRow() { createAndInitTable( "id INT, dep STRING", @@ -1148,7 +1124,7 @@ public void testMergeWithMultipleUpdatesForTargetRow() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithUnconditionalDelete() { createAndInitTable( "id INT, dep STRING", @@ -1181,7 +1157,7 @@ public void testMergeWithUnconditionalDelete() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithSingleConditionalDelete() { createAndInitTable( "id INT, dep STRING", @@ -1217,7 +1193,7 @@ public void testMergeWithSingleConditionalDelete() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testMergeWithIdentityTransform() { for (DistributionMode mode : DistributionMode.values()) { createAndInitTable("id INT, dep STRING"); @@ -1263,7 +1239,7 @@ public void testMergeWithIdentityTransform() { } } - @Test + @TestTemplate public void testMergeWithDaysTransform() { for (DistributionMode mode : DistributionMode.values()) { createAndInitTable("id INT, ts TIMESTAMP"); @@ -1311,7 +1287,7 @@ public void testMergeWithDaysTransform() { } } - @Test + @TestTemplate public void testMergeWithBucketTransform() { for (DistributionMode mode : DistributionMode.values()) { createAndInitTable("id INT, dep STRING"); @@ -1357,7 +1333,7 @@ public void testMergeWithBucketTransform() { } } - @Test + @TestTemplate public void testMergeWithTruncateTransform() { for (DistributionMode mode : DistributionMode.values()) { createAndInitTable("id INT, dep STRING"); @@ -1403,7 +1379,7 @@ public void testMergeWithTruncateTransform() { } } - @Test + @TestTemplate public void testMergeIntoPartitionedAndOrderedTable() { for (DistributionMode mode : DistributionMode.values()) { createAndInitTable("id INT, dep STRING"); @@ -1450,7 +1426,7 @@ public void testMergeIntoPartitionedAndOrderedTable() { } } - @Test + @TestTemplate public void testSelfMerge() { createAndInitTable( "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1473,7 +1449,7 @@ public void testSelfMerge() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testSelfMergeWithCaching() { createAndInitTable( "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1498,7 +1474,7 @@ public void testSelfMergeWithCaching() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", commitTarget())); } - @Test + @TestTemplate public void testMergeWithSourceAsSelfSubquery() { createAndInitTable( "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1523,13 +1499,13 @@ public void testMergeWithSourceAsSelfSubquery() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public synchronized void testMergeWithSerializableIsolation() throws InterruptedException { // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); // if caching is off, the table is eagerly refreshed during runtime filtering // this can cause a validation exception as concurrent changes would be visible - Assume.assumeTrue(cachingCatalogEnabled()); + assumeThat(cachingCatalogEnabled()).isTrue(); createAndInitTable("id INT, dep STRING"); createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); @@ -1553,9 +1529,11 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted executorService.submit( () -> { for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> barrier.get() >= currentNumOperations * 2); sql( "MERGE INTO %s t USING source s " @@ -1580,9 +1558,11 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted record.set(1, "hr"); // dep for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); if (!shouldAppend.get()) { return; @@ -1595,7 +1575,6 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted appendFiles.toBranch(branch); } appendFiles.commit(); - sleep(10); } barrier.incrementAndGet(); @@ -1614,17 +1593,17 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public synchronized void testMergeWithSnapshotIsolation() throws InterruptedException, ExecutionException { // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); // if caching is off, the table is eagerly refreshed during runtime filtering // this can cause a validation exception as concurrent changes would be visible - Assume.assumeTrue(cachingCatalogEnabled()); + assumeThat(cachingCatalogEnabled()).isTrue(); createAndInitTable("id INT, dep STRING"); createOrReplaceView("source", Collections.singletonList(1), Encoders.INT()); @@ -1648,9 +1627,11 @@ public synchronized void testMergeWithSnapshotIsolation() executorService.submit( () -> { for (int numOperations = 0; numOperations < 20; numOperations++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> barrier.get() >= currentNumOperations * 2); sql( "MERGE INTO %s t USING source s " @@ -1675,9 +1656,11 @@ public synchronized void testMergeWithSnapshotIsolation() record.set(1, "hr"); // dep for (int numOperations = 0; numOperations < 20; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); if (!shouldAppend.get()) { return; @@ -1691,7 +1674,6 @@ public synchronized void testMergeWithSnapshotIsolation() } appendFiles.commit(); - sleep(10); } barrier.incrementAndGet(); @@ -1706,10 +1688,10 @@ public synchronized void testMergeWithSnapshotIsolation() } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public void testMergeWithExtraColumnsInSource() { createAndInitTable( "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1739,7 +1721,7 @@ public void testMergeWithExtraColumnsInSource() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithNullsInTargetAndSource() { createAndInitTable( "id INT, v STRING", "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1767,7 +1749,7 @@ public void testMergeWithNullsInTargetAndSource() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); } - @Test + @TestTemplate public void testMergeWithNullSafeEquals() { createAndInitTable( "id INT, v STRING", "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1794,7 +1776,7 @@ public void testMergeWithNullSafeEquals() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); } - @Test + @TestTemplate public void testMergeWithNullCondition() { createAndInitTable( "id INT, v STRING", "{ \"id\": null, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1822,7 +1804,7 @@ public void testMergeWithNullCondition() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); } - @Test + @TestTemplate public void testMergeWithNullActionConditions() { createAndInitTable( "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1873,7 +1855,7 @@ public void testMergeWithNullActionConditions() { "Output should match", expectedRows2, sql("SELECT * FROM %s ORDER BY v", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleMatchingActions() { createAndInitTable( "id INT, v STRING", "{ \"id\": 1, \"v\": \"v1\" }\n" + "{ \"id\": 2, \"v\": \"v2\" }"); @@ -1902,9 +1884,9 @@ public void testMergeWithMultipleMatchingActions() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY v", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException { - Assume.assumeTrue(fileFormat.equalsIgnoreCase("parquet")); + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -1928,7 +1910,7 @@ public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException df.coalesce(1).writeTo(tableName).append(); createBranchIfNeeded(); - Assert.assertEquals(200, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); // update a record from one of two row groups and copy over the second one sql( @@ -1938,10 +1920,10 @@ public void testMergeWithMultipleRowGroupsParquet() throws NoSuchTableException + " UPDATE SET dep = 'x'", commitTarget()); - Assert.assertEquals(200, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); } - @Test + @TestTemplate public void testMergeInsertOnly() { createAndInitTable( "id STRING, v STRING", @@ -1973,7 +1955,7 @@ public void testMergeInsertOnly() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeInsertOnlyWithCondition() { createAndInitTable("id INTEGER, v INTEGER", "{ \"id\": 1, \"v\": 1 }"); createOrReplaceView( @@ -1999,7 +1981,7 @@ public void testMergeInsertOnlyWithCondition() { "Output should match", expectedRows, sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeAlignsUpdateAndInsertActions() { createAndInitTable("id INT, a INT, b STRING", "{ \"id\": 1, \"a\": 2, \"b\": \"str\" }"); createOrReplaceView( @@ -2022,7 +2004,7 @@ public void testMergeAlignsUpdateAndInsertActions() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeMixedCaseAlignsUpdateAndInsertActions() { createAndInitTable("id INT, a INT, b STRING", "{ \"id\": 1, \"a\": 2, \"b\": \"str\" }"); createOrReplaceView( @@ -2054,7 +2036,7 @@ public void testMergeMixedCaseAlignsUpdateAndInsertActions() { sql("SELECT * FROM %s WHERE b = 'new_str_2'ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeUpdatesNestedStructFields() { createAndInitTable( "id INT, s STRUCT,m:MAP>>", @@ -2101,7 +2083,7 @@ public void testMergeUpdatesNestedStructFields() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithInferredCasts() { createAndInitTable("id INT, s STRING", "{ \"id\": 1, \"s\": \"value\" }"); createOrReplaceView("source", "{ \"id\": 1, \"c1\": -2}"); @@ -2120,7 +2102,7 @@ public void testMergeWithInferredCasts() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeModifiesNullStruct() { createAndInitTable("id INT, s STRUCT", "{ \"id\": 1, \"s\": null }"); createOrReplaceView("source", "{ \"id\": 1, \"n1\": -10 }"); @@ -2138,7 +2120,7 @@ public void testMergeModifiesNullStruct() { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testMergeRefreshesRelationCache() { createAndInitTable("id INT, name STRING", "{ \"id\": 1, \"name\": \"n1\" }"); createOrReplaceView("source", "{ \"id\": 1, \"name\": \"n2\" }"); @@ -2164,7 +2146,7 @@ public void testMergeRefreshesRelationCache() { spark.sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testMergeWithMultipleNotMatchedActions() { createAndInitTable("id INT, dep STRING", "{ \"id\": 0, \"dep\": \"emp-id-0\" }"); @@ -2197,7 +2179,7 @@ public void testMergeWithMultipleNotMatchedActions() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithMultipleConditionalNotMatchedActions() { createAndInitTable("id INT, dep STRING", "{ \"id\": 0, \"dep\": \"emp-id-0\" }"); @@ -2229,7 +2211,7 @@ public void testMergeWithMultipleConditionalNotMatchedActions() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeResolvesColumnsByName() { createAndInitTable( "id INT, badge INT, dep STRING", @@ -2264,7 +2246,7 @@ public void testMergeResolvesColumnsByName() { sql("SELECT id, badge, dep FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns() { // ensures that MERGE INTO will resolve into the correct action even if no columns // or otherwise unresolved expressions exist in the query (testing SPARK-34962) @@ -2299,7 +2281,7 @@ public void testMergeShouldResolveWhenThereAreNoUnresolvedExpressionsOrColumns() sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithTableWithNonNullableColumn() { createAndInitTable( "id INT NOT NULL, dep STRING", @@ -2333,7 +2315,7 @@ public void testMergeWithTableWithNonNullableColumn() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithNonExistingColumns() { createAndInitTable( "id INT, c STRUCT>", @@ -2377,7 +2359,7 @@ public void testMergeWithNonExistingColumns() { "A column or function parameter with name `invalid_col` cannot be resolved"); } - @Test + @TestTemplate public void testMergeWithInvalidColumnsInInsert() { createAndInitTable( "id INT, c STRUCT> NOT NULL", @@ -2420,7 +2402,7 @@ public void testMergeWithInvalidColumnsInInsert() { .hasMessageContaining("No assignment for 'c'"); } - @Test + @TestTemplate public void testMergeWithMissingOptionalColumnsInInsert() { createAndInitTable("id INT, value LONG", "{ \"id\": 1, \"value\": 100}"); createOrReplaceView("source", "{ \"c1\": 2, \"c2\": 200 }"); @@ -2440,7 +2422,7 @@ public void testMergeWithMissingOptionalColumnsInInsert() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testMergeWithInvalidUpdates() { createAndInitTable( "id INT, a ARRAY>, m MAP", @@ -2469,7 +2451,7 @@ public void testMergeWithInvalidUpdates() { .hasMessageContaining("Updating nested fields is only supported for StructType"); } - @Test + @TestTemplate public void testMergeWithConflictingUpdates() { createAndInitTable( "id INT, c STRUCT>", @@ -2508,7 +2490,7 @@ public void testMergeWithConflictingUpdates() { .hasMessageContaining("Conflicting assignments for 'c'"); } - @Test + @TestTemplate public void testMergeWithInvalidAssignmentsAnsi() { createAndInitTable( "id INT NOT NULL, s STRUCT> NOT NULL", @@ -2575,7 +2557,7 @@ public void testMergeWithInvalidAssignmentsAnsi() { }); } - @Test + @TestTemplate public void testMergeWithInvalidAssignmentsStrict() { createAndInitTable( "id INT NOT NULL, s STRUCT> NOT NULL", @@ -2643,7 +2625,7 @@ public void testMergeWithInvalidAssignmentsStrict() { }); } - @Test + @TestTemplate public void testMergeWithNonDeterministicConditions() { createAndInitTable( "id INT, c STRUCT>", @@ -2697,7 +2679,7 @@ public void testMergeWithNonDeterministicConditions() { "MERGE operation contains unsupported INSERT condition. Non-deterministic expressions are not allowed"); } - @Test + @TestTemplate public void testMergeWithAggregateExpressions() { createAndInitTable( "id INT, c STRUCT>", @@ -2752,7 +2734,7 @@ public void testMergeWithAggregateExpressions() { "MERGE operation contains unsupported INSERT condition. Aggregates are not allowed"); } - @Test + @TestTemplate public void testMergeWithSubqueriesInConditions() { createAndInitTable( "id INT, c STRUCT>", @@ -2807,7 +2789,7 @@ public void testMergeWithSubqueriesInConditions() { "MERGE operation contains unsupported INSERT condition. Subqueries are not allowed"); } - @Test + @TestTemplate public void testMergeWithTargetColumnsInInsertConditions() { createAndInitTable("id INT, c2 INT", "{ \"id\": 1, \"c2\": 2 }"); createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); @@ -2824,7 +2806,7 @@ public void testMergeWithTargetColumnsInInsertConditions() { .hasMessageContaining("A column or function parameter with name `c2` cannot be resolved"); } - @Test + @TestTemplate public void testMergeWithNonIcebergTargetTableNotSupported() { createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); @@ -2844,7 +2826,7 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { * Tests a merge where both the source and target are evaluated to be partitioned by * SingePartition at planning time but DynamicFileFilterExec will return an empty target. */ - @Test + @TestTemplate public void testMergeSinglePartitionPartitioning() { // This table will only have a single file and a single partition createAndInitTable("id INT", "{\"id\": -1}"); @@ -2865,9 +2847,9 @@ public void testMergeSinglePartitionPartitioning() { assertEquals("Should correctly add the non-matching rows", expectedRows, result); } - @Test + @TestTemplate public void testMergeEmptyTable() { - Assume.assumeFalse("Custom branch does not exist for empty table", "test".equals(branch)); + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); // This table will only have a single file and a single partition createAndInitTable("id INT", null); @@ -2886,9 +2868,9 @@ public void testMergeEmptyTable() { assertEquals("Should correctly add the non-matching rows", expectedRows, result); } - @Test + @TestTemplate public void testMergeNonExistingBranch() { - Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); + assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); createAndInitTable("id INT", null); // Coalesce forces our source into a SinglePartition distribution @@ -2904,9 +2886,9 @@ public void testMergeNonExistingBranch() { .hasMessage("Cannot use branch (does not exist): test"); } - @Test + @TestTemplate public void testMergeToWapBranch() { - Assume.assumeTrue("WAP branch only works for table identifier without branch", branch == null); + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); createAndInitTable("id INT", "{\"id\": -1}"); ImmutableList originalRows = ImmutableList.of(row(-1)); @@ -2965,9 +2947,9 @@ public void testMergeToWapBranch() { }); } - @Test + @TestTemplate public void testMergeToWapBranchWithTableBranchIdentifier() { - Assume.assumeTrue("Test must have branch name part in table identifier", branch != null); + assumeThat(branch).as("Test must have branch name part in table identifier").isNotNull(); createAndInitTable("id INT", "{\"id\": -1}"); sql( @@ -3004,9 +2986,9 @@ private void checkJoinAndFilterConditions(String query, String join, String iceb SparkPlan sparkPlan = executeAndKeepPlan(() -> sql(query)); String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); - Assertions.assertThat(planAsString).as("Join should match").contains(join + "\n"); + assertThat(planAsString).as("Join should match").contains(join + "\n"); - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Pushed filters must match") .contains("[filters=" + icebergFilters + ","); }); 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 91600d4df08d..0d6f3f395f01 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 @@ -26,7 +26,7 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; @@ -44,34 +44,13 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) public class TestMergeOnReadDelete extends TestDelete { - public TestMergeOnReadDelete( - String catalogName, - String implementation, - Map config, - String fileFormat, - Boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - @Override protected Map extraTableProperties() { return ImmutableMap.of( @@ -81,12 +60,12 @@ protected Map extraTableProperties() { RowLevelOperationMode.MERGE_ON_READ.modeName()); } - @Parameterized.AfterParam - public static void clearTestSparkCatalogCache() { + @BeforeEach + public void clearTestSparkCatalogCache() { TestSparkCatalog.clearTables(); } - @Test + @TestTemplate public void testDeleteWithExecutorCacheLocality() throws NoSuchTableException { createAndInitPartitionedTable(); @@ -110,12 +89,12 @@ public void testDeleteWithExecutorCacheLocality() throws NoSuchTableException { }); } - @Test + @TestTemplate public void testDeleteFileGranularity() throws NoSuchTableException { checkDeleteFileGranularity(DeleteGranularity.FILE); } - @Test + @TestTemplate public void testDeletePartitionGranularity() throws NoSuchTableException { checkDeleteFileGranularity(DeleteGranularity.PARTITION); } @@ -150,7 +129,7 @@ private void checkDeleteFileGranularity(DeleteGranularity deleteGranularity) sql("SELECT * FROM %s ORDER BY id ASC, dep ASC", selectTarget())); } - @Test + @TestTemplate public void testCommitUnknownException() { createAndInitTable("id INT, dep STRING, category STRING"); @@ -208,7 +187,7 @@ public void testCommitUnknownException() { sql("SELECT * FROM %s ORDER BY id", "dummy_catalog.default.table")); } - @Test + @TestTemplate public void testAggregatePushDownInMergeOnReadDelete() { createAndInitTable("id LONG, data INT"); sql( @@ -228,8 +207,9 @@ public void testAggregatePushDownInMergeOnReadDelete() { explainContainsPushDownAggregates = true; } - Assert.assertFalse( - "min/max/count not pushed down for deleted", explainContainsPushDownAggregates); + assertThat(explainContainsPushDownAggregates) + .as("min/max/count not pushed down for deleted") + .isFalse(); List actual = sql(select, selectTarget()); List expected = Lists.newArrayList(); 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 f9c13d828cd3..71ca3421f28d 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 @@ -21,7 +21,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -31,32 +30,10 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; public class TestMergeOnReadMerge extends TestMerge { - public TestMergeOnReadMerge( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - @Override protected Map extraTableProperties() { return ImmutableMap.of( @@ -66,12 +43,12 @@ protected Map extraTableProperties() { RowLevelOperationMode.MERGE_ON_READ.modeName()); } - @Test + @TestTemplate public void testMergeDeleteFileGranularity() { checkMergeDeleteGranularity(DeleteGranularity.FILE); } - @Test + @TestTemplate public void testMergeDeletePartitionGranularity() { checkMergeDeleteGranularity(DeleteGranularity.PARTITION); } 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 45ef343b2dfe..391fae4ea696 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,7 @@ import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; -import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -30,32 +30,12 @@ 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.SnapshotUtil; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) public class TestMergeOnReadUpdate extends TestUpdate { - public TestMergeOnReadUpdate( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - @Override protected Map extraTableProperties() { return ImmutableMap.of( @@ -65,12 +45,12 @@ protected Map extraTableProperties() { RowLevelOperationMode.MERGE_ON_READ.modeName()); } - @Test + @TestTemplate public void testUpdateFileGranularity() { checkUpdateFileGranularity(DeleteGranularity.FILE); } - @Test + @TestTemplate public void testUpdatePartitionGranularity() { checkUpdateFileGranularity(DeleteGranularity.PARTITION); } 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 4cc8845befee..09ec61a6ad4e 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 @@ -31,6 +31,7 @@ import static org.apache.iceberg.TableProperties.UPDATE_MODE; import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; @@ -47,7 +48,8 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.PlanningMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; @@ -72,42 +74,21 @@ import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Test; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestUpdate extends SparkRowLevelOperationsTestBase { - public TestUpdate( - String catalogName, - String implementation, - Map config, - String fileFormat, - boolean vectorized, - String distributionMode, - boolean fanoutEnabled, - String branch, - PlanningMode planningMode) { - super( - catalogName, - implementation, - config, - fileFormat, - vectorized, - distributionMode, - fanoutEnabled, - branch, - planningMode); - } - - @BeforeClass + @BeforeAll public static void setupSparkConf() { spark.conf().set("spark.sql.shuffle.partitions", "4"); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS updated_id"); @@ -115,7 +96,7 @@ public void removeTables() { sql("DROP TABLE IF EXISTS deleted_employee"); } - @Test + @TestTemplate public void testUpdateWithVectorizedReads() { assumeThat(supportsVectorization()).isTrue(); @@ -134,7 +115,7 @@ public void testUpdateWithVectorizedReads() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testCoalesceUpdate() { createAndInitTable("id INT, dep STRING"); @@ -177,7 +158,7 @@ public void testCoalesceUpdate() { () -> { SparkPlan plan = executeAndKeepPlan("UPDATE %s SET id = -1 WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -198,13 +179,12 @@ public void testCoalesceUpdate() { validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "1"); } - Assert.assertEquals( - "Row count must match", - 200L, - scalarSql("SELECT COUNT(*) FROM %s WHERE id = -1", commitTarget())); + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE id = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); } - @Test + @TestTemplate public void testSkewUpdate() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -248,7 +228,7 @@ public void testSkewUpdate() { () -> { SparkPlan plan = executeAndKeepPlan("UPDATE %s SET id = -1 WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -270,13 +250,12 @@ public void testSkewUpdate() { validateProperty(snapshot, SnapshotSummary.ADDED_DELETE_FILES_PROP, "4"); } - Assert.assertEquals( - "Row count must match", - 200L, - scalarSql("SELECT COUNT(*) FROM %s WHERE id = -1", commitTarget())); + assertThat(scalarSql("SELECT COUNT(*) FROM %s WHERE id = -1", commitTarget())) + .as("Row count must match") + .isEqualTo(200L); } - @Test + @TestTemplate public void testExplain() { createAndInitTable("id INT, dep STRING"); @@ -288,7 +267,7 @@ public void testExplain() { sql("EXPLAIN UPDATE %s SET dep = 'invalid' WHERE true", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 1 snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 1 snapshot").hasSize(1); assertEquals( "Should have expected rows", @@ -296,16 +275,16 @@ public void testExplain() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testUpdateEmptyTable() { - Assume.assumeFalse("Custom branch does not exist for empty table", "test".equals(branch)); + assumeThat(branch).as("Custom branch does not exist for empty table").isNotEqualTo("test"); createAndInitTable("id INT, dep STRING"); sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget()); sql("UPDATE %s SET id = -1 WHERE dep = 'hr'", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); assertEquals( "Should have expected rows", @@ -313,9 +292,9 @@ public void testUpdateEmptyTable() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testUpdateNonExistingCustomBranch() { - Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); + assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); createAndInitTable("id INT, dep STRING"); Assertions.assertThatThrownBy( @@ -324,7 +303,7 @@ public void testUpdateNonExistingCustomBranch() { .hasMessage("Cannot use branch (does not exist): test"); } - @Test + @TestTemplate public void testUpdateWithAlias() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"a\" }"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -332,7 +311,7 @@ public void testUpdateWithAlias() { sql("UPDATE %s AS t SET t.dep = 'invalid'", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); assertEquals( "Should have expected rows", @@ -340,7 +319,7 @@ public void testUpdateWithAlias() { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testUpdateAlignsAssignments() { createAndInitTable("id INT, c1 INT, c2 INT"); @@ -355,7 +334,7 @@ public void testUpdateAlignsAssignments() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithUnsupportedPartitionPredicate() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -371,7 +350,7 @@ public void testUpdateWithUnsupportedPartitionPredicate() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithDynamicFileFiltering() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -385,7 +364,7 @@ public void testUpdateWithDynamicFileFiltering() { sql("UPDATE %s SET id = cast('-1' AS INT) WHERE id = 2", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -400,7 +379,7 @@ public void testUpdateWithDynamicFileFiltering() { sql("SELECT * FROM %s ORDER BY id, dep", commitTarget())); } - @Test + @TestTemplate public void testUpdateNonExistingRecords() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -411,7 +390,7 @@ public void testUpdateNonExistingRecords() { sql("UPDATE %s SET id = -1 WHERE id > 10", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 2 snapshots").hasSize(2); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -426,7 +405,7 @@ public void testUpdateNonExistingRecords() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithoutCondition() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -448,13 +427,13 @@ public void testUpdateWithoutCondition() { }); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); - Assert.assertEquals("Operation must match", OVERWRITE, currentSnapshot.operation()); + assertThat(currentSnapshot.operation()).as("Operation must match").isEqualTo(OVERWRITE); if (mode(table) == COPY_ON_WRITE) { - Assert.assertEquals("Operation must match", OVERWRITE, currentSnapshot.operation()); + assertThat(currentSnapshot.operation()).as("Operation must match").isEqualTo(OVERWRITE); validateProperty(currentSnapshot, CHANGED_PARTITION_COUNT_PROP, "2"); validateProperty(currentSnapshot, DELETED_FILES_PROP, "3"); validateProperty(currentSnapshot, ADDED_FILES_PROP, ImmutableSet.of("2", "3")); @@ -468,7 +447,7 @@ public void testUpdateWithoutCondition() { sql("SELECT * FROM %s ORDER BY dep ASC", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithNullConditions() { createAndInitTable("id INT, dep STRING"); @@ -501,7 +480,7 @@ public void testUpdateWithNullConditions() { sql("SELECT * FROM %s ORDER BY id", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithInAndNotInConditions() { createAndInitTable("id INT, dep STRING"); @@ -531,9 +510,9 @@ public void testUpdateWithInAndNotInConditions() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithMultipleRowGroupsParquet() throws NoSuchTableException { - Assume.assumeTrue(fileFormat.equalsIgnoreCase("parquet")); + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -555,15 +534,15 @@ public void testUpdateWithMultipleRowGroupsParquet() throws NoSuchTableException df.coalesce(1).writeTo(tableName).append(); createBranchIfNeeded(); - Assert.assertEquals(200, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); // update a record from one of two row groups and copy over the second one sql("UPDATE %s SET id = -1 WHERE id IN (200, 201)", commitTarget()); - Assert.assertEquals(200, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()).isEqualTo(200); } - @Test + @TestTemplate public void testUpdateNestedStructFields() { createAndInitTable( "id INT, s STRUCT,m:MAP>>", @@ -596,7 +575,7 @@ public void testUpdateNestedStructFields() { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithUserDefinedDistribution() { createAndInitTable("id INT, c2 INT, c3 INT"); sql("ALTER TABLE %s ADD PARTITION FIELD bucket(8, c3)", tableName); @@ -633,13 +612,13 @@ public void testUpdateWithUserDefinedDistribution() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public synchronized void testUpdateWithSerializableIsolation() throws InterruptedException { // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); // if caching is off, the table is eagerly refreshed during runtime filtering // this can cause a validation exception as concurrent changes would be visible - Assume.assumeTrue(cachingCatalogEnabled()); + assumeThat(cachingCatalogEnabled()).isTrue(); createAndInitTable("id INT, dep STRING"); @@ -662,9 +641,11 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte executorService.submit( () -> { for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> barrier.get() >= currentNumOperations * 2); sql("UPDATE %s SET id = -1 WHERE id = 1", commitTarget()); @@ -684,9 +665,11 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte record.set(1, "hr"); // dep for (int numOperations = 0; numOperations < Integer.MAX_VALUE; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); if (!shouldAppend.get()) { return; @@ -700,7 +683,6 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte } appendFiles.commit(); - sleep(10); } barrier.incrementAndGet(); @@ -719,17 +701,17 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public synchronized void testUpdateWithSnapshotIsolation() throws InterruptedException, ExecutionException { // cannot run tests with concurrency for Hadoop tables without atomic renames - Assume.assumeFalse(catalogName.equalsIgnoreCase("testhadoop")); + assumeThat(catalogName).isNotEqualToIgnoringCase("testhadoop"); // if caching is off, the table is eagerly refreshed during runtime filtering // this can cause a validation exception as concurrent changes would be visible - Assume.assumeTrue(cachingCatalogEnabled()); + assumeThat(cachingCatalogEnabled()).isTrue(); createAndInitTable("id INT, dep STRING"); @@ -752,9 +734,11 @@ public synchronized void testUpdateWithSnapshotIsolation() executorService.submit( () -> { for (int numOperations = 0; numOperations < 20; numOperations++) { - while (barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> barrier.get() >= currentNumOperations * 2); sql("UPDATE %s SET id = -1 WHERE id = 1", tableName); @@ -774,9 +758,11 @@ public synchronized void testUpdateWithSnapshotIsolation() record.set(1, "hr"); // dep for (int numOperations = 0; numOperations < 20; numOperations++) { - while (shouldAppend.get() && barrier.get() < numOperations * 2) { - sleep(10); - } + int currentNumOperations = numOperations; + Awaitility.await() + .pollInterval(10, TimeUnit.MILLISECONDS) + .atMost(5, TimeUnit.SECONDS) + .until(() -> !shouldAppend.get() || barrier.get() >= currentNumOperations * 2); if (!shouldAppend.get()) { return; @@ -790,7 +776,6 @@ public synchronized void testUpdateWithSnapshotIsolation() } appendFiles.commit(); - sleep(10); } barrier.incrementAndGet(); @@ -805,10 +790,10 @@ public synchronized void testUpdateWithSnapshotIsolation() } executorService.shutdown(); - Assert.assertTrue("Timeout", executorService.awaitTermination(2, TimeUnit.MINUTES)); + assertThat(executorService.awaitTermination(2, TimeUnit.MINUTES)).as("Timeout").isTrue(); } - @Test + @TestTemplate public void testUpdateWithInferredCasts() { createAndInitTable("id INT, s STRING", "{ \"id\": 1, \"s\": \"value\" }"); @@ -820,7 +805,7 @@ public void testUpdateWithInferredCasts() { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testUpdateModifiesNullStruct() { createAndInitTable("id INT, s STRUCT", "{ \"id\": 1, \"s\": null }"); @@ -832,7 +817,7 @@ public void testUpdateModifiesNullStruct() { sql("SELECT * FROM %s", selectTarget())); } - @Test + @TestTemplate public void testUpdateRefreshesRelationCache() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -857,7 +842,7 @@ public void testUpdateRefreshesRelationCache() { sql("UPDATE %s SET id = -1 WHERE id = 1", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -879,7 +864,7 @@ public void testUpdateRefreshesRelationCache() { spark.sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testUpdateWithInSubquery() { createAndInitTable("id INT, dep STRING"); @@ -929,7 +914,7 @@ public void testUpdateWithInSubquery() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithInSubqueryAndDynamicFileFiltering() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -947,7 +932,7 @@ public void testUpdateWithInSubqueryAndDynamicFileFiltering() { sql("UPDATE %s SET id = -1 WHERE id IN (SELECT * FROM updated_id)", commitTarget()); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should have 3 snapshots").hasSize(3); Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); if (mode(table) == COPY_ON_WRITE) { @@ -962,7 +947,7 @@ public void testUpdateWithInSubqueryAndDynamicFileFiltering() { sql("SELECT * FROM %s ORDER BY id, dep", commitTarget())); } - @Test + @TestTemplate public void testUpdateWithSelfSubquery() { createAndInitTable("id INT, dep STRING"); @@ -998,7 +983,7 @@ public void testUpdateWithSelfSubquery() { sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithMultiColumnInSubquery() { createAndInitTable("id INT, dep STRING"); @@ -1022,7 +1007,7 @@ public void testUpdateWithMultiColumnInSubquery() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithNotInSubquery() { createAndInitTable("id INT, dep STRING"); @@ -1060,7 +1045,7 @@ public void testUpdateWithNotInSubquery() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST, dep", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithExistSubquery() { createAndInitTable("id INT, dep STRING"); @@ -1112,7 +1097,7 @@ public void testUpdateWithExistSubquery() { sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithNotExistsSubquery() { createAndInitTable("id INT, dep STRING"); @@ -1155,7 +1140,7 @@ public void testUpdateWithNotExistsSubquery() { sql("SELECT * FROM %s ORDER BY id, dep", selectTarget())); } - @Test + @TestTemplate public void testUpdateWithScalarSubquery() { createAndInitTable("id INT, dep STRING"); @@ -1182,7 +1167,7 @@ public void testUpdateWithScalarSubquery() { }); } - @Test + @TestTemplate public void testUpdateThatRequiresGroupingBeforeWrite() { createAndInitTable("id INT, dep STRING"); sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); @@ -1220,14 +1205,15 @@ public void testUpdateThatRequiresGroupingBeforeWrite() { spark.conf().set("spark.sql.shuffle.partitions", "1"); sql("UPDATE %s t SET id = -1 WHERE id IN (SELECT * FROM updated_id)", commitTarget()); - Assert.assertEquals( - "Should have expected num of rows", 12L, spark.table(commitTarget()).count()); + assertThat(spark.table(commitTarget()).count()) + .as("Should have expected num of rows") + .isEqualTo(12L); } finally { spark.conf().set("spark.sql.shuffle.partitions", originalNumOfShufflePartitions); } } - @Test + @TestTemplate public void testUpdateWithVectorization() { createAndInitTable("id INT, dep STRING"); @@ -1250,7 +1236,7 @@ public void testUpdateWithVectorization() { }); } - @Test + @TestTemplate public void testUpdateModifyPartitionSourceField() throws NoSuchTableException { createAndInitTable("id INT, dep STRING, country STRING"); @@ -1290,10 +1276,10 @@ public void testUpdateModifyPartitionSourceField() throws NoSuchTableException { sql( "UPDATE %s SET id = -1 WHERE id IN (10, 11, 12, 13, 14, 15, 16, 17, 18, 19)", commitTarget()); - Assert.assertEquals(30L, scalarSql("SELECT count(*) FROM %s WHERE id = -1", selectTarget())); + assertThat(scalarSql("SELECT count(*) FROM %s WHERE id = -1", selectTarget())).isEqualTo(30L); } - @Test + @TestTemplate public void testUpdateWithStaticPredicatePushdown() { createAndInitTable("id INT, dep STRING"); @@ -1310,7 +1296,7 @@ public void testUpdateWithStaticPredicatePushdown() { Snapshot snapshot = SnapshotUtil.latestSnapshot(table, branch); String dataFilesCount = snapshot.summary().get(SnapshotSummary.TOTAL_DATA_FILES_PROP); - Assert.assertEquals("Must have 2 files before UPDATE", "2", dataFilesCount); + assertThat(dataFilesCount).as("Must have 2 files before UPDATE").isEqualTo("2"); // remove the data file from the 'hr' partition to ensure it is not scanned DataFile dataFile = Iterables.getOnlyElement(snapshot.addedDataFiles(table.io())); @@ -1326,7 +1312,7 @@ public void testUpdateWithStaticPredicatePushdown() { }); } - @Test + @TestTemplate public void testUpdateWithInvalidUpdates() { createAndInitTable( "id INT, a ARRAY>, m MAP", @@ -1341,7 +1327,7 @@ public void testUpdateWithInvalidUpdates() { .hasMessageContaining("Updating nested fields is only supported for StructType"); } - @Test + @TestTemplate public void testUpdateWithConflictingAssignments() { createAndInitTable( "id INT, c STRUCT>", "{ \"id\": 0, \"s\": null }"); @@ -1365,7 +1351,7 @@ public void testUpdateWithConflictingAssignments() { .hasMessageContaining("Conflicting assignments for 'c'"); } - @Test + @TestTemplate public void testUpdateWithInvalidAssignmentsAnsi() { createAndInitTable( "id INT NOT NULL, s STRUCT> NOT NULL", @@ -1401,7 +1387,7 @@ public void testUpdateWithInvalidAssignmentsAnsi() { }); } - @Test + @TestTemplate public void testUpdateWithInvalidAssignmentsStrict() { createAndInitTable( "id INT NOT NULL, s STRUCT> NOT NULL", @@ -1437,7 +1423,7 @@ public void testUpdateWithInvalidAssignmentsStrict() { }); } - @Test + @TestTemplate public void testUpdateWithNonDeterministicCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); @@ -1447,7 +1433,7 @@ public void testUpdateWithNonDeterministicCondition() { .hasMessageContaining("The operator expects a deterministic expression"); } - @Test + @TestTemplate public void testUpdateOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); @@ -1456,9 +1442,9 @@ public void testUpdateOnNonIcebergTableNotSupported() { .hasMessage("UPDATE TABLE is not supported temporarily."); } - @Test + @TestTemplate public void testUpdateToWAPBranch() { - Assume.assumeTrue("WAP branch only works for table identifier without branch", branch == null); + assumeThat(branch).as("WAP branch only works for table identifier without branch").isNull(); createAndInitTable( "id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }\n" + "{ \"id\": 2, \"dep\": \"a\" }"); @@ -1470,42 +1456,36 @@ public void testUpdateToWAPBranch() { ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> { sql("UPDATE %s SET dep='hr' WHERE dep='a'", tableName); - Assert.assertEquals( - "Should have expected num of rows when reading table", - 2L, - sql("SELECT * FROM %s WHERE dep='hr'", tableName).size()); - Assert.assertEquals( - "Should have expected num of rows when reading WAP branch", - 2L, - sql("SELECT * FROM %s.branch_wap WHERE dep='hr'", tableName).size()); - Assert.assertEquals( - "Should not modify main branch", - 1L, - sql("SELECT * FROM %s.branch_main WHERE dep='hr'", tableName).size()); + assertThat(sql("SELECT * FROM %s WHERE dep='hr'", tableName)) + .as("Should have expected num of rows when reading table") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_wap WHERE dep='hr'", tableName)) + .as("Should have expected num of rows when reading WAP branch") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_main WHERE dep='hr'", tableName)) + .as("Should not modify main branch") + .hasSize(1); }); withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> { sql("UPDATE %s SET dep='b' WHERE dep='hr'", tableName); - Assert.assertEquals( - "Should have expected num of rows when reading table with multiple writes", - 2L, - sql("SELECT * FROM %s WHERE dep='b'", tableName).size()); - Assert.assertEquals( - "Should have expected num of rows when reading WAP branch with multiple writes", - 2L, - sql("SELECT * FROM %s.branch_wap WHERE dep='b'", tableName).size()); - Assert.assertEquals( - "Should not modify main branch with multiple writes", - 0L, - sql("SELECT * FROM %s.branch_main WHERE dep='b'", tableName).size()); + assertThat(sql("SELECT * FROM %s WHERE dep='b'", tableName)) + .as("Should have expected num of rows when reading table with multiple writes") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_wap WHERE dep='b'", tableName)) + .as("Should have expected num of rows when reading WAP branch with multiple writes") + .hasSize(2); + assertThat(sql("SELECT * FROM %s.branch_main WHERE dep='b'", tableName)) + .as("Should not modify main branch with multiple writes") + .hasSize(0); }); } - @Test + @TestTemplate public void testUpdateToWapBranchWithTableBranchIdentifier() { - Assume.assumeTrue("Test must have branch name part in table identifier", branch != null); + assumeThat(branch).as("Test must have branch name part in table identifier").isNotNull(); createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); sql( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 4d87099572b8..256e654b7775 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -22,6 +22,8 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.BulkDeletionFailureException; @@ -38,15 +40,14 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestWriteAborts extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestWriteAborts extends ExtensionsTestBase { - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -74,20 +75,14 @@ public static Object[][] parameters() { }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - public TestWriteAborts(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testBatchAppend() throws Exception { - String dataLocation = temp.newFolder().toString(); + String dataLocation = temp.toFile().toString(); sql( "CREATE TABLE %s (id INT, data STRING) " From 59c78a452f77d7da7a3a8d1b2460d873b90a93e8 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 20 Feb 2024 17:51:05 +0800 Subject: [PATCH 0058/1019] Infra: Fix issue template labels (#9759) Co-authored-by: Zhang, Manu --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 2 +- .github/ISSUE_TEMPLATE/iceberg_improvement.yml | 2 +- .github/ISSUE_TEMPLATE/iceberg_question.yml | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 7fadcd02f7d4..6ad07b60be71 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -20,7 +20,7 @@ --- name: Iceberg Bug report 🐞 description: Problems, bugs and issues with Apache Iceberg -labels: ["kind:bug"] +labels: ["bug"] body: - type: dropdown attributes: diff --git a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml index d85a8f182135..d25a4b707386 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml @@ -20,7 +20,7 @@ --- name: Iceberg Improvement / Feature Request description: New features with Apache Iceberg -labels: ["kind:feature request"] +labels: ["improvement"] body: - type: textarea attributes: diff --git a/.github/ISSUE_TEMPLATE/iceberg_question.yml b/.github/ISSUE_TEMPLATE/iceberg_question.yml index 25da32f4a874..da3513cd4276 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_question.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_question.yml @@ -20,7 +20,7 @@ --- name: Iceberg Question description: Questions around Apache Iceberg -labels: ["kind:question"] +labels: ["question"] body: - type: markdown attributes: From d1d2534f8849ae2f67171c658ee9c7e8882ceeec Mon Sep 17 00:00:00 2001 From: Reo Date: Wed, 21 Feb 2024 12:35:07 +0800 Subject: [PATCH 0059/1019] Flink 1.18: Fix continuous enumerator lost enumeration history state when restore from checkpoint. (#9762) --- .../flink/source/enumerator/ContinuousIcebergEnumerator.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index ff68103b2b9a..c7021b9c6847 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -83,6 +83,7 @@ public ContinuousIcebergEnumerator( if (enumState != null) { this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); + this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); } } From 20096b4dd751d8ce5f83e6d7c5df2f21e576b378 Mon Sep 17 00:00:00 2001 From: Reo Date: Wed, 21 Feb 2024 12:47:25 +0800 Subject: [PATCH 0060/1019] Flink 1.18: Fix iceberg source plan parallelism not effective. (#9761) --- .../src/main/java/org/apache/iceberg/flink/FlinkReadConf.java | 1 + 1 file changed, 1 insertion(+) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java index d53ea73f9342..804a956ec9b9 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java @@ -178,6 +178,7 @@ public long limit() { public int workerPoolSize() { return confParser .intConf() + .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) .parse(); From df96c6ddd0cdef757b2546ed470743e879b9a13f Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Tue, 20 Feb 2024 23:07:27 -0800 Subject: [PATCH 0061/1019] API: Fix EncryptingFileIO factory method (#9757) --- .../org/apache/iceberg/encryption/EncryptingFileIO.java | 9 +++++++-- .../java/org/apache/iceberg/spark/source/BaseReader.java | 2 +- 2 files changed, 8 insertions(+), 3 deletions(-) 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 ea63b29754bc..0203361844a5 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptingFileIO.java @@ -35,9 +35,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; public class EncryptingFileIO implements FileIO, Serializable { - public static EncryptingFileIO create(FileIO io, EncryptionManager em) { + public static EncryptingFileIO combine(FileIO io, EncryptionManager em) { if (io instanceof EncryptingFileIO) { - return (EncryptingFileIO) io; + EncryptingFileIO encryptingIO = (EncryptingFileIO) io; + if (encryptingIO.em == em) { + return encryptingIO; + } + + return combine(encryptingIO.io, em); } return new EncryptingFileIO(io, em); 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 f67171708bb2..a982309ec16d 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 @@ -182,7 +182,7 @@ protected InputFile getInputFile(String location) { private Map inputFiles() { if (lazyInputFiles == null) { this.lazyInputFiles = - EncryptingFileIO.create(table().io(), table().encryption()) + EncryptingFileIO.combine(table().io(), table().encryption()) .bulkDecrypt( () -> taskGroup.tasks().stream().flatMap(this::referencedFiles).iterator()); } From 94bb100460f69801d050c0f533513c229cbd5d6f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Wed, 21 Feb 2024 12:15:55 +0100 Subject: [PATCH 0062/1019] Core: Use V0 SQL schema as default / rename jdbc.add-view-support to jdbc.schema-version (#9765) --- .../org/apache/iceberg/jdbc/JdbcCatalog.java | 158 ++++++++++-------- .../org/apache/iceberg/jdbc/JdbcUtil.java | 8 +- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 70 +++++--- .../iceberg/jdbc/TestJdbcViewCatalog.java | 1 + 4 files changed, 138 insertions(+), 99 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 8d0a93b727f5..73c7e0586528 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.io.UncheckedIOException; -import java.sql.Connection; import java.sql.DatabaseMetaData; import java.sql.PreparedStatement; import java.sql.ResultSet; @@ -75,7 +74,7 @@ public class JdbcCatalog extends BaseMetastoreViewCatalog private static final Logger LOG = LoggerFactory.getLogger(JdbcCatalog.class); private static final Joiner SLASH = Joiner.on("/"); static final String VIEW_WARNING_LOG_MESSAGE = - "JDBC catalog is initialized without view support. To auto-migrate the database's schema and enable view support, set jdbc.add-view-support=true"; + "JDBC catalog is initialized without view support. To auto-migrate the database's schema and enable view support, set jdbc.schema-version=V1"; private FileIO io; private String catalogName = "jdbc"; @@ -87,7 +86,7 @@ public class JdbcCatalog extends BaseMetastoreViewCatalog private final Function, JdbcClientPool> clientPoolBuilder; private final boolean initializeCatalogTables; private CloseableGroup closeableGroup; - private JdbcUtil.SchemaVersion schemaVersion = JdbcUtil.SchemaVersion.V1; + private JdbcUtil.SchemaVersion schemaVersion = JdbcUtil.SchemaVersion.V0; public JdbcCatalog() { this(null, null, true); @@ -136,10 +135,60 @@ public void initialize(String name, Map properties) { this.connections = new JdbcClientPool(uri, properties); } + if (initializeCatalogTables) { + initializeCatalogTables(); + } + + updateSchemaIfRequired(); + + this.closeableGroup = new CloseableGroup(); + closeableGroup.addCloseable(metricsReporter()); + closeableGroup.addCloseable(connections); + closeableGroup.setSuppressCloseFailure(true); + } + + private void initializeCatalogTables() { + LOG.trace("Creating database tables (if missing) to store iceberg catalog"); try { - if (initializeCatalogTables) { - initializeCatalogTables(); - } + connections.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + JdbcUtil.CATALOG_TABLE_VIEW_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + return true; + } + + LOG.debug( + "Creating table {} to store iceberg catalog tables", + JdbcUtil.CATALOG_TABLE_VIEW_NAME); + return conn.prepareStatement(JdbcUtil.V0_CREATE_CATALOG_SQL).execute(); + }); + + connections.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + JdbcUtil.NAMESPACE_PROPERTIES_TABLE_NAME /* tableNamePattern */, + null /* types */); + + if (tableExists.next()) { + return true; + } + + LOG.debug( + "Creating table {} to store iceberg catalog namespace properties", + JdbcUtil.NAMESPACE_PROPERTIES_TABLE_NAME); + return conn.prepareStatement(JdbcUtil.CREATE_NAMESPACE_PROPERTIES_TABLE_SQL).execute(); + }); + } catch (SQLTimeoutException e) { throw new UncheckedSQLException(e, "Cannot initialize JDBC catalog: Query timed out"); } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { @@ -150,69 +199,44 @@ public void initialize(String name, Map properties) { Thread.currentThread().interrupt(); throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); } - this.closeableGroup = new CloseableGroup(); - closeableGroup.addCloseable(metricsReporter()); - closeableGroup.addCloseable(connections); - closeableGroup.setSuppressCloseFailure(true); } - private void initializeCatalogTables() throws InterruptedException, SQLException { - LOG.trace("Creating database tables (if missing) to store iceberg catalog"); - connections.run( - conn -> { - DatabaseMetaData dbMeta = conn.getMetaData(); - ResultSet tableExists = - dbMeta.getTables( - null /* catalog name */, - null /* schemaPattern */, - JdbcUtil.CATALOG_TABLE_VIEW_NAME /* tableNamePattern */, - null /* types */); - if (tableExists.next()) { - updateCatalogTables(conn); - return true; - } - - LOG.debug( - "Creating table {} to store iceberg catalog tables", - JdbcUtil.CATALOG_TABLE_VIEW_NAME); - return conn.prepareStatement(JdbcUtil.CREATE_CATALOG_SQL).execute(); - }); - - connections.run( - conn -> { - DatabaseMetaData dbMeta = conn.getMetaData(); - ResultSet tableExists = - dbMeta.getTables( - null /* catalog name */, - null /* schemaPattern */, - JdbcUtil.NAMESPACE_PROPERTIES_TABLE_NAME /* tableNamePattern */, - null /* types */); - - if (tableExists.next()) { - return true; - } - - LOG.debug( - "Creating table {} to store iceberg catalog namespace properties", - JdbcUtil.NAMESPACE_PROPERTIES_TABLE_NAME); - return conn.prepareStatement(JdbcUtil.CREATE_NAMESPACE_PROPERTIES_TABLE_SQL).execute(); - }); - } - - private void updateCatalogTables(Connection connection) throws SQLException { - DatabaseMetaData dbMeta = connection.getMetaData(); - ResultSet typeColumn = - dbMeta.getColumns(null, null, JdbcUtil.CATALOG_TABLE_VIEW_NAME, JdbcUtil.RECORD_TYPE); - if (typeColumn.next()) { - LOG.debug("{} already supports views", JdbcUtil.CATALOG_TABLE_VIEW_NAME); - } else { - if (PropertyUtil.propertyAsBoolean( - catalogProperties, JdbcUtil.ADD_VIEW_SUPPORT_PROPERTY, false)) { - connection.prepareStatement(JdbcUtil.UPDATE_CATALOG_SQL).execute(); - } else { - LOG.warn(VIEW_WARNING_LOG_MESSAGE); - schemaVersion = JdbcUtil.SchemaVersion.V0; - } + private void updateSchemaIfRequired() { + try { + connections.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet typeColumn = + dbMeta.getColumns( + null, null, JdbcUtil.CATALOG_TABLE_VIEW_NAME, JdbcUtil.RECORD_TYPE); + if (typeColumn.next()) { + LOG.debug("{} already supports views", JdbcUtil.CATALOG_TABLE_VIEW_NAME); + schemaVersion = JdbcUtil.SchemaVersion.V1; + return true; + } else { + if (PropertyUtil.propertyAsString( + catalogProperties, + JdbcUtil.SCHEMA_VERSION_PROPERTY, + JdbcUtil.SchemaVersion.V0.name()) + .equalsIgnoreCase(JdbcUtil.SchemaVersion.V1.name())) { + LOG.debug("{} is being updated to support views", JdbcUtil.CATALOG_TABLE_VIEW_NAME); + schemaVersion = JdbcUtil.SchemaVersion.V1; + return conn.prepareStatement(JdbcUtil.V1_UPDATE_CATALOG_SQL).execute(); + } else { + LOG.warn(VIEW_WARNING_LOG_MESSAGE); + return true; + } + } + }); + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException(e, "Cannot update JDBC catalog: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException(e, "Cannot update JDBC catalog: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot check and eventually update SQL schema"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); } } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 20a2f7ea6238..077c33321be4 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -38,7 +38,7 @@ final class JdbcUtil { // property to control strict-mode (aka check if namespace exists when creating a table) static final String STRICT_MODE_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "strict-mode"; // property to control if view support is added to the existing database - static final String ADD_VIEW_SUPPORT_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "add-view-support"; + static final String SCHEMA_VERSION_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "schema-version"; enum SchemaVersion { V0, @@ -90,7 +90,7 @@ enum SchemaVersion { + " = ? AND " + JdbcTableOperations.METADATA_LOCATION_PROP + " = ?"; - static final String CREATE_CATALOG_SQL = + static final String V0_CREATE_CATALOG_SQL = "CREATE TABLE " + CATALOG_TABLE_VIEW_NAME + "(" @@ -104,8 +104,6 @@ enum SchemaVersion { + " VARCHAR(1000)," + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + " VARCHAR(1000)," - + RECORD_TYPE - + " VARCHAR(100)," + "PRIMARY KEY (" + CATALOG_NAME + ", " @@ -114,7 +112,7 @@ enum SchemaVersion { + TABLE_NAME + ")" + ")"; - static final String UPDATE_CATALOG_SQL = + static final String V1_UPDATE_CATALOG_SQL = "ALTER TABLE " + CATALOG_TABLE_VIEW_NAME + " ADD COLUMN " + RECORD_TYPE + " VARCHAR(5)"; private static final String GET_VIEW_SQL = 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 3c39ed3c1321..f9bbb527be1e 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -140,6 +140,7 @@ private JdbcCatalog initCatalog(String catalogName, Map props) { properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); warehouseLocation = this.tableDir.toAbsolutePath().toString(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); properties.put("type", "jdbc"); @@ -153,6 +154,7 @@ public void testInitialize() { Map properties = Maps.newHashMap(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); properties.put(CatalogProperties.URI, "jdbc:sqlite:file::memory:?icebergDB"); + properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); JdbcCatalog jdbcCatalog = new JdbcCatalog(); jdbcCatalog.setConf(conf); jdbcCatalog.initialize("test_jdbc_catalog", properties); @@ -161,6 +163,41 @@ public void testInitialize() { jdbcCatalog.initialize("test_jdbc_catalog", properties); } + @Test + public void testInitSchemaV0() { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, "jdbc:sqlite:file::memory:?icebergDBV0"); + properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V0.name()); + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.setConf(conf); + jdbcCatalog.initialize("v0catalog", properties); + + TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("ns1"), "tbl"); + Table table = + jdbcCatalog + .buildTable(tableIdent, SCHEMA) + .withPartitionSpec(PARTITION_SPEC) + .withProperty("key1", "value1") + .withProperty("key2", "value2") + .create(); + + assertThat(table.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1").containsEntry("key2", "value2"); + + assertThat(jdbcCatalog.listTables(Namespace.of("ns1"))).hasSize(1).contains(tableIdent); + + assertThatThrownBy(() -> jdbcCatalog.listViews(Namespace.of("namespace1"))) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage(JdbcCatalog.VIEW_WARNING_LOG_MESSAGE); + + assertThatThrownBy( + () -> jdbcCatalog.buildView(TableIdentifier.of("namespace1", "view")).create()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage(JdbcCatalog.VIEW_WARNING_LOG_MESSAGE); + } + @Test public void testSchemaIsMigratedToAddViewSupport() throws Exception { // as this test uses different connections, we can't use memory database (as it's per @@ -173,7 +210,7 @@ public void testSchemaIsMigratedToAddViewSupport() throws Exception { Map properties = Maps.newHashMap(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); properties.put(CatalogProperties.URI, jdbcUrl); - properties.put(JdbcUtil.ADD_VIEW_SUPPORT_PROPERTY, "true"); + properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); JdbcCatalog jdbcCatalog = new JdbcCatalog(); jdbcCatalog.setConf(conf); jdbcCatalog.initialize("TEST", properties); @@ -201,8 +238,9 @@ public void testSchemaIsMigratedToAddViewSupport() throws Exception { assertThat(jdbcCatalog.listViews(Namespace.of("namespace1"))).hasSize(1).containsExactly(view); } - @Test - public void testLegacySchemaSupport() throws Exception { + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testExistingV0SchemaSupport(boolean initializeCatalogTables) throws Exception { // as this test uses different connection, we can't use memory database (as it's per // connection), but a // file database instead @@ -214,7 +252,7 @@ public void testLegacySchemaSupport() throws Exception { Map properties = Maps.newHashMap(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); properties.put(CatalogProperties.URI, jdbcUrl); - JdbcCatalog jdbcCatalog = new JdbcCatalog(); + JdbcCatalog jdbcCatalog = new JdbcCatalog(null, null, initializeCatalogTables); jdbcCatalog.setConf(conf); jdbcCatalog.initialize("TEST", properties); @@ -949,29 +987,7 @@ private void initLegacySchema(String jdbcUrl) throws SQLException { try (Connection connection = dataSource.getConnection()) { // create "old style" SQL schema - connection - .prepareStatement( - "CREATE TABLE " - + JdbcUtil.CATALOG_TABLE_VIEW_NAME - + "(" - + JdbcUtil.CATALOG_NAME - + " VARCHAR(255) NOT NULL," - + JdbcUtil.TABLE_NAMESPACE - + " VARCHAR(255) NOT NULL," - + JdbcUtil.TABLE_NAME - + " VARCHAR(255) NOT NULL," - + JdbcTableOperations.METADATA_LOCATION_PROP - + " VARCHAR(1000)," - + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP - + " VARCHAR(1000)," - + "PRIMARY KEY(" - + JdbcUtil.CATALOG_NAME - + "," - + JdbcUtil.TABLE_NAMESPACE - + "," - + JdbcUtil.TABLE_NAME - + "))") - .executeUpdate(); + connection.prepareStatement(JdbcUtil.V0_CREATE_CATALOG_SQL).executeUpdate(); connection .prepareStatement( "INSERT INTO " diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java index c7bdb158cdf2..8c0255764251 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java @@ -43,6 +43,7 @@ public void before() { 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()); From fc35ec29305453a7e7b554c697951d53ee934148 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 21 Feb 2024 13:53:01 +0100 Subject: [PATCH 0063/1019] Infra: Add Kafka Connect as a label (#9769) --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 1 + .github/ISSUE_TEMPLATE/iceberg_improvement.yml | 1 + 2 files changed, 2 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 6ad07b60be71..51efe443f5f5 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -66,6 +66,7 @@ body: - Athena - PrestoDB - Flink + - Kafka Connect - Impala - Hive - Other diff --git a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml index d25a4b707386..f3d6d6de923a 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml @@ -45,6 +45,7 @@ body: - Athena - PrestoDB - Flink + - Kafka Connect - Impala - Hive - Other From 65ea620c53e635031e72c22f1e1aa52f6f7f5593 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Wed, 21 Feb 2024 16:57:42 +0100 Subject: [PATCH 0064/1019] Core: Only test if view exists when using SchemaVersion.V1 during table rename (#9770) --- .../org/apache/iceberg/jdbc/JdbcCatalog.java | 33 +++++++++- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 2 - .../jdbc/TestJdbcCatalogWithV1Schema.java | 66 +++++++++++++++++++ 3 files changed, 98 insertions(+), 3 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 73c7e0586528..dc7352743e88 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -39,8 +39,10 @@ import java.util.stream.Stream; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Schema; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; @@ -313,6 +315,7 @@ public List listTables(Namespace namespace) { JdbcUtil.namespaceToString(namespace)); } + @SuppressWarnings("checkstyle:CyclomaticComplexity") @Override public void renameTable(TableIdentifier from, TableIdentifier to) { if (from.equals(to)) { @@ -327,7 +330,7 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { throw new NoSuchNamespaceException("Namespace does not exist: %s", to.namespace()); } - if (viewExists(to)) { + if (schemaVersion == JdbcUtil.SchemaVersion.V1 && viewExists(to)) { throw new AlreadyExistsException("Cannot rename %s to %s. View already exists", from, to); } @@ -812,4 +815,32 @@ private boolean deleteProperties(Namespace namespace, Set properties) { protected Map properties() { return catalogProperties == null ? ImmutableMap.of() : catalogProperties; } + + @Override + public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { + return new ViewAwareTableBuilder(identifier, schema); + } + + /** + * The purpose of this class is to add view detection only when SchemaVersion.V1 schema is used + * when replacing a table. + */ + protected class ViewAwareTableBuilder extends BaseMetastoreCatalogTableBuilder { + + private final TableIdentifier identifier; + + public ViewAwareTableBuilder(TableIdentifier identifier, Schema schema) { + super(identifier, schema); + this.identifier = identifier; + } + + @Override + public Transaction replaceTransaction() { + if (schemaVersion == JdbcUtil.SchemaVersion.V1 && viewExists(identifier)) { + throw new AlreadyExistsException("View with same name already exists: %s", identifier); + } + + return super.replaceTransaction(); + } + } } 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 f9bbb527be1e..48ad71734799 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -140,7 +140,6 @@ private JdbcCatalog initCatalog(String catalogName, Map props) { properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); - properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); warehouseLocation = this.tableDir.toAbsolutePath().toString(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); properties.put("type", "jdbc"); @@ -154,7 +153,6 @@ public void testInitialize() { Map properties = Maps.newHashMap(); properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); properties.put(CatalogProperties.URI, "jdbc:sqlite:file::memory:?icebergDB"); - properties.put(JdbcUtil.SCHEMA_VERSION_PROPERTY, JdbcUtil.SchemaVersion.V1.name()); JdbcCatalog jdbcCatalog = new JdbcCatalog(); jdbcCatalog.setConf(conf); jdbcCatalog.initialize("test_jdbc_catalog", properties); diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java new file mode 100644 index 000000000000..b47c216ffced --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalogWithV1Schema.java @@ -0,0 +1,66 @@ +/* + * 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.jdbc; + +import java.util.Map; +import java.util.UUID; +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.Maps; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.io.TempDir; + +public class TestJdbcCatalogWithV1Schema extends CatalogTests { + + private JdbcCatalog catalog; + + @TempDir private java.nio.file.Path tableDir; + + @Override + protected JdbcCatalog catalog() { + return catalog; + } + + @Override + protected boolean supportsNamespaceProperties() { + return true; + } + + @Override + protected boolean supportsNestedNamespaces() { + return true; + } + + @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); + } +} From 5609cf9ded6fa98b2cbb1abd8582bd4631ab2684 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 22 Feb 2024 16:41:28 +0100 Subject: [PATCH 0065/1019] docs: Fix listings on Release page / Update Multi-engine support (#9775) --- site/docs/multi-engine-support.md | 1 + site/docs/releases.md | 668 +++++++++++++++--------------- 2 files changed, 335 insertions(+), 334 deletions(-) diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index cd7fddf3224d..e137bdda1735 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -67,6 +67,7 @@ Each engine version undergoes the following lifecycle stages: | 3.2 | Deprecated | 0.13.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.2_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.2_2.12-{{ icebergVersion }}.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.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) | * [1] Spark 3.1 shares the same runtime jar `iceberg-spark3-runtime` with Spark 3.0 before Iceberg 0.13.0 diff --git a/site/docs/releases.md b/site/docs/releases.md index 13ee45edd9ca..e55c6ed46f3f 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -87,7 +87,7 @@ The 1.4.2 patch release addresses fixing a remaining case where split offsets should be ignored when they are deemed invalid. * Core - - Core: Ignore split offsets array when split offset is past file length ([\#8925](https://github.com/apache/iceberg/pull/8925)) + - Ignore split offsets array when split offset is past file length ([\#8925](https://github.com/apache/iceberg/pull/8925)) ### 1.4.1 Release @@ -95,12 +95,12 @@ Apache Iceberg 1.4.1 was released on October 23, 2023. The 1.4.1 release addresses various issues identified in the 1.4.0 release. * Core - - Core: Do not use a lazy split offset list in manifests ([\#8834](https://github.com/apache/iceberg/pull/8834)) - - Core: Ignore split offsets when the last split offset is past the file length ([\#8860](https://github.com/apache/iceberg/pull/8860)) + - Do not use a lazy split offset list in manifests ([\#8834](https://github.com/apache/iceberg/pull/8834)) + - Ignore split offsets when the last split offset is past the file length ([\#8860](https://github.com/apache/iceberg/pull/8860)) * AWS - - Avoid static global credentials provider which doesn't play well with lifecycle management ([\#8677](https://github.com/apache/iceberg/pull/8677)) + - Avoid static global credentials provider which doesn't play well with lifecycle management ([\#8677](https://github.com/apache/iceberg/pull/8677)) * Flink - - Reverting the default custom partitioner for bucket column ([\#8848](https://github.com/apache/iceberg/pull/8848)) + - Reverting the default custom partitioner for bucket column ([\#8848](https://github.com/apache/iceberg/pull/8848)) ### 1.4.0 release @@ -108,96 +108,96 @@ Apache Iceberg 1.4.0 was released on October 4, 2023. The 1.4.0 release adds a variety of new features and bug fixes. * API - - Implement bound expression sanitization ([\#8149](https://github.com/apache/iceberg/pull/8149)) - - Remove overflow checks in `DefaultCounter` causing performance issues ([\#8297](https://github.com/apache/iceberg/pull/8297)) - - Support incremental scanning with branch ([\#5984](https://github.com/apache/iceberg/pull/5984)) - - Add a validation API to `DeleteFiles` which validates files exist ([\#8525](https://github.com/apache/iceberg/pull/8525)) + - Implement bound expression sanitization ([\#8149](https://github.com/apache/iceberg/pull/8149)) + - Remove overflow checks in `DefaultCounter` causing performance issues ([\#8297](https://github.com/apache/iceberg/pull/8297)) + - Support incremental scanning with branch ([\#5984](https://github.com/apache/iceberg/pull/5984)) + - Add a validation API to `DeleteFiles` which validates files exist ([\#8525](https://github.com/apache/iceberg/pull/8525)) * Core - - Use V2 format by default in new tables ([\#8381](https://github.com/apache/iceberg/pull/8381)) - - Use `zstd` compression for Parquet by default in new tables ([\#8593](https://github.com/apache/iceberg/pull/8593)) - - Add strict metadata cleanup mode and enable it by default ([\#8397](https://github.com/apache/iceberg/pull/8397)) ([\#8599](https://github.com/apache/iceberg/pull/8599)) - - Avoid generating huge manifests during commits ([\#6335](https://github.com/apache/iceberg/pull/6335)) - - Add a writer for unordered position deletes ([\#7692](https://github.com/apache/iceberg/pull/7692)) - - Optimize `DeleteFileIndex` ([\#8157](https://github.com/apache/iceberg/pull/8157)) - - Optimize lookup in `DeleteFileIndex` without useful bounds ([\#8278](https://github.com/apache/iceberg/pull/8278)) - - Optimize split offsets handling ([\#8336](https://github.com/apache/iceberg/pull/8336)) - - Optimize computing user-facing state in data tasks ([\#8346](https://github.com/apache/iceberg/pull/8346)) - - Don't persist useless file and position bounds for deletes ([\#8360](https://github.com/apache/iceberg/pull/8360)) - - Don't persist counts for paths and positions in position delete files ([\#8590](https://github.com/apache/iceberg/pull/8590)) - - Support setting system-level properties via environmental variables ([\#5659](https://github.com/apache/iceberg/pull/5659)) - - Add JSON parser for `ContentFile` and `FileScanTask` ([\#6934](https://github.com/apache/iceberg/pull/6934)) - - Add REST spec and request for commits to multiple tables ([\#7741](https://github.com/apache/iceberg/pull/7741)) - - Add REST API for committing changes against multiple tables ([\#7569](https://github.com/apache/iceberg/pull/7569)) - - Default to exponential retry strategy in REST client ([\#8366](https://github.com/apache/iceberg/pull/8366)) - - Support registering tables with REST session catalog ([\#6512](https://github.com/apache/iceberg/pull/6512)) - - Add last updated timestamp and snapshot ID to partitions metadata table ([\#7581](https://github.com/apache/iceberg/pull/7581)) - - Add total data size to partitions metadata table ([\#7920](https://github.com/apache/iceberg/pull/7920)) - - Extend `ResolvingFileIO` to support bulk operations ([\#7976](https://github.com/apache/iceberg/pull/7976)) - - Key metadata in Avro format ([\#6450](https://github.com/apache/iceberg/pull/6450)) - - Add AES GCM encryption stream ([\#3231](https://github.com/apache/iceberg/pull/3231)) - - Fix a connection leak in streaming delete filters ([\#8132](https://github.com/apache/iceberg/pull/8132)) - - Fix lazy snapshot loading history ([\#8470](https://github.com/apache/iceberg/pull/8470)) - - Fix unicode handling in HTTPClient ([\#8046](https://github.com/apache/iceberg/pull/8046)) - - Fix paths for unpartitioned specs in writers ([\#7685](https://github.com/apache/iceberg/pull/7685)) - - Fix OOM caused by Avro decoder caching ([\#7791](https://github.com/apache/iceberg/pull/7791)) + - Use V2 format by default in new tables ([\#8381](https://github.com/apache/iceberg/pull/8381)) + - Use `zstd` compression for Parquet by default in new tables ([\#8593](https://github.com/apache/iceberg/pull/8593)) + - Add strict metadata cleanup mode and enable it by default ([\#8397](https://github.com/apache/iceberg/pull/8397)) ([\#8599](https://github.com/apache/iceberg/pull/8599)) + - Avoid generating huge manifests during commits ([\#6335](https://github.com/apache/iceberg/pull/6335)) + - Add a writer for unordered position deletes ([\#7692](https://github.com/apache/iceberg/pull/7692)) + - Optimize `DeleteFileIndex` ([\#8157](https://github.com/apache/iceberg/pull/8157)) + - Optimize lookup in `DeleteFileIndex` without useful bounds ([\#8278](https://github.com/apache/iceberg/pull/8278)) + - Optimize split offsets handling ([\#8336](https://github.com/apache/iceberg/pull/8336)) + - Optimize computing user-facing state in data tasks ([\#8346](https://github.com/apache/iceberg/pull/8346)) + - Don't persist useless file and position bounds for deletes ([\#8360](https://github.com/apache/iceberg/pull/8360)) + - Don't persist counts for paths and positions in position delete files ([\#8590](https://github.com/apache/iceberg/pull/8590)) + - Support setting system-level properties via environmental variables ([\#5659](https://github.com/apache/iceberg/pull/5659)) + - Add JSON parser for `ContentFile` and `FileScanTask` ([\#6934](https://github.com/apache/iceberg/pull/6934)) + - Add REST spec and request for commits to multiple tables ([\#7741](https://github.com/apache/iceberg/pull/7741)) + - Add REST API for committing changes against multiple tables ([\#7569](https://github.com/apache/iceberg/pull/7569)) + - Default to exponential retry strategy in REST client ([\#8366](https://github.com/apache/iceberg/pull/8366)) + - Support registering tables with REST session catalog ([\#6512](https://github.com/apache/iceberg/pull/6512)) + - Add last updated timestamp and snapshot ID to partitions metadata table ([\#7581](https://github.com/apache/iceberg/pull/7581)) + - Add total data size to partitions metadata table ([\#7920](https://github.com/apache/iceberg/pull/7920)) + - Extend `ResolvingFileIO` to support bulk operations ([\#7976](https://github.com/apache/iceberg/pull/7976)) + - Key metadata in Avro format ([\#6450](https://github.com/apache/iceberg/pull/6450)) + - Add AES GCM encryption stream ([\#3231](https://github.com/apache/iceberg/pull/3231)) + - Fix a connection leak in streaming delete filters ([\#8132](https://github.com/apache/iceberg/pull/8132)) + - Fix lazy snapshot loading history ([\#8470](https://github.com/apache/iceberg/pull/8470)) + - Fix unicode handling in HTTPClient ([\#8046](https://github.com/apache/iceberg/pull/8046)) + - Fix paths for unpartitioned specs in writers ([\#7685](https://github.com/apache/iceberg/pull/7685)) + - Fix OOM caused by Avro decoder caching ([\#7791](https://github.com/apache/iceberg/pull/7791)) * Spark - - Added support for Spark 3.5 - - Code for DELETE, UPDATE, and MERGE commands has moved to Spark, and all related extensions have been dropped from Iceberg. - - Support for WHEN NOT MATCHED BY SOURCE clause in MERGE. - - Column pruning in merge-on-read operations. - - Ability to request a bigger advisory partition size for the final write to produce well-sized output files without harming the job parallelism. - - Dropped support for Spark 3.1 - - Deprecated support for Spark 3.2 - - Support vectorized reads for merge-on-read operations in Spark 3.4 and 3.5 ([\#8466](https://github.com/apache/iceberg/pull/8466)) - - Increase default advisory partition size for writes in Spark 3.5 ([\#8660](https://github.com/apache/iceberg/pull/8660)) - - Support distributed planning in Spark 3.4 and 3.5 ([\#8123](https://github.com/apache/iceberg/pull/8123)) - - Support pushing down system functions by V2 filters in Spark 3.4 and 3.5 ([\#7886](https://github.com/apache/iceberg/pull/7886)) - - Support fanout position delta writers in Spark 3.4 and 3.5 ([\#7703](https://github.com/apache/iceberg/pull/7703)) - - Use fanout writers for unsorted tables by default in Spark 3.5 ([\#8621](https://github.com/apache/iceberg/pull/8621)) - - Support multiple shuffle partitions per file in compaction in Spark 3.4 and 3.5 ([\#7897](https://github.com/apache/iceberg/pull/7897)) - - Output net changes across snapshots for carryover rows in CDC ([\#7326](https://github.com/apache/iceberg/pull/7326)) - - Display read metrics on Spark SQL UI ([\#7447](https://github.com/apache/iceberg/pull/7447)) ([\#8445](https://github.com/apache/iceberg/pull/8445)) - - Adjust split size to benefit from cluster parallelism in Spark 3.4 and 3.5 ([\#7714](https://github.com/apache/iceberg/pull/7714)) - - Add `fast_forward` procedure ([\#8081](https://github.com/apache/iceberg/pull/8081)) - - Support filters when rewriting position deletes ([\#7582](https://github.com/apache/iceberg/pull/7582)) - - Support setting current snapshot with ref ([\#8163](https://github.com/apache/iceberg/pull/8163)) - - Make backup table name configurable during migration ([\#8227](https://github.com/apache/iceberg/pull/8227)) - - Add write and SQL options to override compression config ([\#8313](https://github.com/apache/iceberg/pull/8313)) - - Correct partition transform functions to match the spec ([\#8192](https://github.com/apache/iceberg/pull/8192)) - - Enable extra commit properties with metadata delete ([\#7649](https://github.com/apache/iceberg/pull/7649)) + - Added support for Spark 3.5 + - Code for DELETE, UPDATE, and MERGE commands has moved to Spark, and all related extensions have been dropped from Iceberg. + - Support for WHEN NOT MATCHED BY SOURCE clause in MERGE. + - Column pruning in merge-on-read operations. + - Ability to request a bigger advisory partition size for the final write to produce well-sized output files without harming the job parallelism. + - Dropped support for Spark 3.1 + - Deprecated support for Spark 3.2 + - Support vectorized reads for merge-on-read operations in Spark 3.4 and 3.5 ([\#8466](https://github.com/apache/iceberg/pull/8466)) + - Increase default advisory partition size for writes in Spark 3.5 ([\#8660](https://github.com/apache/iceberg/pull/8660)) + - Support distributed planning in Spark 3.4 and 3.5 ([\#8123](https://github.com/apache/iceberg/pull/8123)) + - Support pushing down system functions by V2 filters in Spark 3.4 and 3.5 ([\#7886](https://github.com/apache/iceberg/pull/7886)) + - Support fanout position delta writers in Spark 3.4 and 3.5 ([\#7703](https://github.com/apache/iceberg/pull/7703)) + - Use fanout writers for unsorted tables by default in Spark 3.5 ([\#8621](https://github.com/apache/iceberg/pull/8621)) + - Support multiple shuffle partitions per file in compaction in Spark 3.4 and 3.5 ([\#7897](https://github.com/apache/iceberg/pull/7897)) + - Output net changes across snapshots for carryover rows in CDC ([\#7326](https://github.com/apache/iceberg/pull/7326)) + - Display read metrics on Spark SQL UI ([\#7447](https://github.com/apache/iceberg/pull/7447)) ([\#8445](https://github.com/apache/iceberg/pull/8445)) + - Adjust split size to benefit from cluster parallelism in Spark 3.4 and 3.5 ([\#7714](https://github.com/apache/iceberg/pull/7714)) + - Add `fast_forward` procedure ([\#8081](https://github.com/apache/iceberg/pull/8081)) + - Support filters when rewriting position deletes ([\#7582](https://github.com/apache/iceberg/pull/7582)) + - Support setting current snapshot with ref ([\#8163](https://github.com/apache/iceberg/pull/8163)) + - Make backup table name configurable during migration ([\#8227](https://github.com/apache/iceberg/pull/8227)) + - Add write and SQL options to override compression config ([\#8313](https://github.com/apache/iceberg/pull/8313)) + - Correct partition transform functions to match the spec ([\#8192](https://github.com/apache/iceberg/pull/8192)) + - Enable extra commit properties with metadata delete ([\#7649](https://github.com/apache/iceberg/pull/7649)) * Flink - - Add possibility of ordering the splits based on the file sequence number ([\#7661](https://github.com/apache/iceberg/pull/7661)) - - Fix serialization in `TableSink` with anonymous object ([\#7866](https://github.com/apache/iceberg/pull/7866)) - - Switch to `FileScanTaskParser` for JSON serialization of `IcebergSourceSplit` ([\#7978](https://github.com/apache/iceberg/pull/7978)) - - Custom partitioner for bucket partitions ([\#7161](https://github.com/apache/iceberg/pull/7161)) - - Implement data statistics coordinator to aggregate data statistics from operator subtasks ([\#7360](https://github.com/apache/iceberg/pull/7360)) - - Support alter table column ([\#7628](https://github.com/apache/iceberg/pull/7628)) + - Add possibility of ordering the splits based on the file sequence number ([\#7661](https://github.com/apache/iceberg/pull/7661)) + - Fix serialization in `TableSink` with anonymous object ([\#7866](https://github.com/apache/iceberg/pull/7866)) + - Switch to `FileScanTaskParser` for JSON serialization of `IcebergSourceSplit` ([\#7978](https://github.com/apache/iceberg/pull/7978)) + - Custom partitioner for bucket partitions ([\#7161](https://github.com/apache/iceberg/pull/7161)) + - Implement data statistics coordinator to aggregate data statistics from operator subtasks ([\#7360](https://github.com/apache/iceberg/pull/7360)) + - Support alter table column ([\#7628](https://github.com/apache/iceberg/pull/7628)) * Parquet - - Add encryption config to read and write builders ([\#2639](https://github.com/apache/iceberg/pull/2639)) - - Skip writing bloom filters for deletes ([\#7617](https://github.com/apache/iceberg/pull/7617)) - - Cache codecs by name and level ([\#8182](https://github.com/apache/iceberg/pull/8182)) - - Fix decimal data reading from `ParquetAvroValueReaders` ([\#8246](https://github.com/apache/iceberg/pull/8246)) - - Handle filters with transforms by assuming data must be scanned ([\#8243](https://github.com/apache/iceberg/pull/8243)) + - Add encryption config to read and write builders ([\#2639](https://github.com/apache/iceberg/pull/2639)) + - Skip writing bloom filters for deletes ([\#7617](https://github.com/apache/iceberg/pull/7617)) + - Cache codecs by name and level ([\#8182](https://github.com/apache/iceberg/pull/8182)) + - Fix decimal data reading from `ParquetAvroValueReaders` ([\#8246](https://github.com/apache/iceberg/pull/8246)) + - Handle filters with transforms by assuming data must be scanned ([\#8243](https://github.com/apache/iceberg/pull/8243)) * ORC - - Handle filters with transforms by assuming the filter matches ([\#8244](https://github.com/apache/iceberg/pull/8244)) + - Handle filters with transforms by assuming the filter matches ([\#8244](https://github.com/apache/iceberg/pull/8244)) * Vendor Integrations - - GCP: Fix single byte read in `GCSInputStream` ([\#8071](https://github.com/apache/iceberg/pull/8071)) - - GCP: Add properties for OAtuh2 and update library ([\#8073](https://github.com/apache/iceberg/pull/8073)) - - GCP: Add prefix and bulk operations to `GCSFileIO` ([\#8168](https://github.com/apache/iceberg/pull/8168)) - - GCP: Add bundle jar for GCP-related dependencies ([\#8231](https://github.com/apache/iceberg/pull/8231)) - - GCP: Add range reads to `GCSInputStream` ([\#8301](https://github.com/apache/iceberg/pull/8301)) - - AWS: Add bundle jar for AWS-related dependencies ([\#8261](https://github.com/apache/iceberg/pull/8261)) - - AWS: support config storage class for `S3FileIO` ([\#8154](https://github.com/apache/iceberg/pull/8154)) - - AWS: Add `FileIO` tracker/closer to Glue catalog ([\#8315](https://github.com/apache/iceberg/pull/8315)) - - AWS: Update S3 signer spec to allow an optional string body in `S3SignRequest` ([\#8361](https://github.com/apache/iceberg/pull/8361)) - - Azure: Add `FileIO` that supports ADLSv2 storage ([\#8303](https://github.com/apache/iceberg/pull/8303)) - - Azure: Make `ADLSFileIO` implement `DelegateFileIO` ([\#8563](https://github.com/apache/iceberg/pull/8563)) - - Nessie: Provide better commit message on table registration ([\#8385](https://github.com/apache/iceberg/pull/8385)) + - GCP: Fix single byte read in `GCSInputStream` ([\#8071](https://github.com/apache/iceberg/pull/8071)) + - GCP: Add properties for OAtuh2 and update library ([\#8073](https://github.com/apache/iceberg/pull/8073)) + - GCP: Add prefix and bulk operations to `GCSFileIO` ([\#8168](https://github.com/apache/iceberg/pull/8168)) + - GCP: Add bundle jar for GCP-related dependencies ([\#8231](https://github.com/apache/iceberg/pull/8231)) + - GCP: Add range reads to `GCSInputStream` ([\#8301](https://github.com/apache/iceberg/pull/8301)) + - AWS: Add bundle jar for AWS-related dependencies ([\#8261](https://github.com/apache/iceberg/pull/8261)) + - AWS: support config storage class for `S3FileIO` ([\#8154](https://github.com/apache/iceberg/pull/8154)) + - AWS: Add `FileIO` tracker/closer to Glue catalog ([\#8315](https://github.com/apache/iceberg/pull/8315)) + - AWS: Update S3 signer spec to allow an optional string body in `S3SignRequest` ([\#8361](https://github.com/apache/iceberg/pull/8361)) + - Azure: Add `FileIO` that supports ADLSv2 storage ([\#8303](https://github.com/apache/iceberg/pull/8303)) + - Azure: Make `ADLSFileIO` implement `DelegateFileIO` ([\#8563](https://github.com/apache/iceberg/pull/8563)) + - Nessie: Provide better commit message on table registration ([\#8385](https://github.com/apache/iceberg/pull/8385)) * Dependencies - - Bump Nessie to 0.71.0 - - Bump ORC to 1.9.1 - - Bump Arrow to 12.0.1 - - Bump AWS Java SDK to 2.20.131 + - Bump Nessie to 0.71.0 + - Bump ORC to 1.9.1 + - Bump Arrow to 12.0.1 + - Bump AWS Java SDK to 2.20.131 ### 1.3.1 release @@ -206,15 +206,15 @@ Apache Iceberg 1.3.1 was released on July 25, 2023. The 1.3.1 release addresses various issues identified in the 1.3.0 release. * Core - - Table Metadata parser now accepts null for fields: current-snapshot-id, properties, and snapshots ([\#8064](https://github.com/apache/iceberg/pull/8064)) + - Table Metadata parser now accepts null for fields: current-snapshot-id, properties, and snapshots ([\#8064](https://github.com/apache/iceberg/pull/8064)) * Hive - - Fix HiveCatalog deleting metadata on failures in checking lock status ([\#7931](https://github.com/apache/iceberg/pull/7931)) + - Fix HiveCatalog deleting metadata on failures in checking lock status ([\#7931](https://github.com/apache/iceberg/pull/7931)) * Spark - - Fix RewritePositionDeleteFiles failure for certain partition types ([\#8059](https://github.com/apache/iceberg/pull/8059)) - - Fix RewriteDataFiles concurrency edge-case on commit timeouts ([\#7933](https://github.com/apache/iceberg/pull/7933)) - - Fix partition-level DELETE operations for WAP branches ([\#7900](https://github.com/apache/iceberg/pull/7900)) + - Fix RewritePositionDeleteFiles failure for certain partition types ([\#8059](https://github.com/apache/iceberg/pull/8059)) + - Fix RewriteDataFiles concurrency edge-case on commit timeouts ([\#7933](https://github.com/apache/iceberg/pull/7933)) + - Fix partition-level DELETE operations for WAP branches ([\#7900](https://github.com/apache/iceberg/pull/7900)) * Flink - - FlinkCatalog creation no longer creates the default database ([\#8039](https://github.com/apache/iceberg/pull/8039)) + - FlinkCatalog creation no longer creates the default database ([\#8039](https://github.com/apache/iceberg/pull/8039)) ### 1.3.0 release @@ -222,50 +222,50 @@ Apache Iceberg 1.3.0 was released on May 30th, 2023. The 1.3.0 release adds a variety of new features and bug fixes. * Core - - Expose file and data sequence numbers in ContentFile ([\#7555](https://github.com/apache/iceberg/pull/7555)) - - Improve bit density in object storage layout ([\#7128](https://github.com/apache/iceberg/pull/7128)) - - Store split offsets for delete files ([\#7011](https://github.com/apache/iceberg/pull/7011)) - - Readable metrics in entries metadata table ([\#7539](https://github.com/apache/iceberg/pull/7539)) - - Delete file stats in partitions metadata table ([\#6661](https://github.com/apache/iceberg/pull/6661)) - - Optimized vectorized reads for Parquet Decimal ([\#3249](https://github.com/apache/iceberg/pull/3249)) - - Vectorized reads for Parquet INT96 timestamps in imported data ([\#6962](https://github.com/apache/iceberg/pull/6962)) - - Support selected vector with ORC row and batch readers ([\#7197](https://github.com/apache/iceberg/pull/7197)) - - Clean up expired metastore clients ([\#7310](https://github.com/apache/iceberg/pull/7310)) - - Support for deleting old partition spec columns in V1 tables ([\#7398](https://github.com/apache/iceberg/pull/7398)) + - Expose file and data sequence numbers in ContentFile ([\#7555](https://github.com/apache/iceberg/pull/7555)) + - Improve bit density in object storage layout ([\#7128](https://github.com/apache/iceberg/pull/7128)) + - Store split offsets for delete files ([\#7011](https://github.com/apache/iceberg/pull/7011)) + - Readable metrics in entries metadata table ([\#7539](https://github.com/apache/iceberg/pull/7539)) + - Delete file stats in partitions metadata table ([\#6661](https://github.com/apache/iceberg/pull/6661)) + - Optimized vectorized reads for Parquet Decimal ([\#3249](https://github.com/apache/iceberg/pull/3249)) + - Vectorized reads for Parquet INT96 timestamps in imported data ([\#6962](https://github.com/apache/iceberg/pull/6962)) + - Support selected vector with ORC row and batch readers ([\#7197](https://github.com/apache/iceberg/pull/7197)) + - Clean up expired metastore clients ([\#7310](https://github.com/apache/iceberg/pull/7310)) + - Support for deleting old partition spec columns in V1 tables ([\#7398](https://github.com/apache/iceberg/pull/7398)) * Spark - - Initial support for Spark 3.4 - - Removed integration for Spark 2.4 - - Support for storage-partitioned joins with mismatching keys in Spark 3.4 (MERGE commands) ([\#7424](https://github.com/apache/iceberg/pull/7424)) - - Support for TimestampNTZ in Spark 3.4 ([\#7553](https://github.com/apache/iceberg/pull/7553)) - - Ability to handle skew during writes in Spark 3.4 ([\#7520](https://github.com/apache/iceberg/pull/7520)) - - Ability to coalesce small tasks during writes in Spark 3.4 ([\#7532](https://github.com/apache/iceberg/pull/7532)) - - Distribution and ordering enhancements in Spark 3.4 ([\#7637](https://github.com/apache/iceberg/pull/7637)) - - Action for rewriting position deletes ([\#7389](https://github.com/apache/iceberg/pull/7389)) - - Procedure for rewriting position deletes ([\#7572](https://github.com/apache/iceberg/pull/7572)) - - Avoid local sort for MERGE cardinality check ([\#7558](https://github.com/apache/iceberg/pull/7558)) - - Support for rate limits in Structured Streaming ([\#4479](https://github.com/apache/iceberg/pull/4479)) - - Read and write support for UUIDs ([\#7399](https://github.com/apache/iceberg/pull/7399)) - - Concurrent compaction is enabled by default ([\#6907](https://github.com/apache/iceberg/pull/6907)) - - Support for metadata columns in changelog tables ([\#7152](https://github.com/apache/iceberg/pull/7152)) - - Add file group failure info for data compaction ([\#7361](https://github.com/apache/iceberg/pull/7361)) + - Initial support for Spark 3.4 + - Removed integration for Spark 2.4 + - Support for storage-partitioned joins with mismatching keys in Spark 3.4 (MERGE commands) ([\#7424](https://github.com/apache/iceberg/pull/7424)) + - Support for TimestampNTZ in Spark 3.4 ([\#7553](https://github.com/apache/iceberg/pull/7553)) + - Ability to handle skew during writes in Spark 3.4 ([\#7520](https://github.com/apache/iceberg/pull/7520)) + - Ability to coalesce small tasks during writes in Spark 3.4 ([\#7532](https://github.com/apache/iceberg/pull/7532)) + - Distribution and ordering enhancements in Spark 3.4 ([\#7637](https://github.com/apache/iceberg/pull/7637)) + - Action for rewriting position deletes ([\#7389](https://github.com/apache/iceberg/pull/7389)) + - Procedure for rewriting position deletes ([\#7572](https://github.com/apache/iceberg/pull/7572)) + - Avoid local sort for MERGE cardinality check ([\#7558](https://github.com/apache/iceberg/pull/7558)) + - Support for rate limits in Structured Streaming ([\#4479](https://github.com/apache/iceberg/pull/4479)) + - Read and write support for UUIDs ([\#7399](https://github.com/apache/iceberg/pull/7399)) + - Concurrent compaction is enabled by default ([\#6907](https://github.com/apache/iceberg/pull/6907)) + - Support for metadata columns in changelog tables ([\#7152](https://github.com/apache/iceberg/pull/7152)) + - Add file group failure info for data compaction ([\#7361](https://github.com/apache/iceberg/pull/7361)) * Flink - - Initial support for Flink 1.17 - - Removed integration for Flink 1.14 - - Data statistics operator to collect traffic distribution for guiding smart shuffling ([\#6382](https://github.com/apache/iceberg/pull/6382)) - - Data statistics operator sends local data statistics to coordinator and receives aggregated data statistics from coordinator for smart shuffling ([\#7269](https://github.com/apache/iceberg/pull/7269)) - - Exposed write parallelism in SQL hints ([\#7039](https://github.com/apache/iceberg/pull/7039)) - - Row-level filtering ([\#7109](https://github.com/apache/iceberg/pull/7109)) - - Use starting sequence number by default when rewriting data files ([\#7218](https://github.com/apache/iceberg/pull/7218)) - - Config for max allowed consecutive planning failures in IcebergSource before failing the job ([\#7571](https://github.com/apache/iceberg/pull/7571)) + - Initial support for Flink 1.17 + - Removed integration for Flink 1.14 + - Data statistics operator to collect traffic distribution for guiding smart shuffling ([\#6382](https://github.com/apache/iceberg/pull/6382)) + - Data statistics operator sends local data statistics to coordinator and receives aggregated data statistics from coordinator for smart shuffling ([\#7269](https://github.com/apache/iceberg/pull/7269)) + - Exposed write parallelism in SQL hints ([\#7039](https://github.com/apache/iceberg/pull/7039)) + - Row-level filtering ([\#7109](https://github.com/apache/iceberg/pull/7109)) + - Use starting sequence number by default when rewriting data files ([\#7218](https://github.com/apache/iceberg/pull/7218)) + - Config for max allowed consecutive planning failures in IcebergSource before failing the job ([\#7571](https://github.com/apache/iceberg/pull/7571)) * Vendor Integrations - - AWS: Use Apache HTTP client as default AWS HTTP client ([\#7119](https://github.com/apache/iceberg/pull/7119)) - - AWS: Prevent token refresh scheduling on every sign request ([\#7270](https://github.com/apache/iceberg/pull/7270)) - - AWS: Disable local credentials if remote signing is enabled ([\#7230](https://github.com/apache/iceberg/pull/7230)) + - AWS: Use Apache HTTP client as default AWS HTTP client ([\#7119](https://github.com/apache/iceberg/pull/7119)) + - AWS: Prevent token refresh scheduling on every sign request ([\#7270](https://github.com/apache/iceberg/pull/7270)) + - AWS: Disable local credentials if remote signing is enabled ([\#7230](https://github.com/apache/iceberg/pull/7230)) * Dependencies - - Bump Arrow to 12.0.0 - - Bump ORC to 1.8.3 - - Bump Parquet to 1.13.1 - - Bump Nessie to 0.59.0 + - Bump Arrow to 12.0.0 + - Bump ORC to 1.8.3 + - Bump Parquet to 1.13.1 + - Bump Nessie to 0.59.0 ### 1.2.1 release @@ -274,17 +274,17 @@ The 1.2.1 release is a patch release to address various issues identified in the Here is an overview: * CORE - - REST: fix previous locations for refs-only load [\#7284](https://github.com/apache/iceberg/pull/7284) - - Parse snapshot-id as long in remove-statistics update [\#7235](https://github.com/apache/iceberg/pull/7235) + - REST: fix previous locations for refs-only load [\#7284](https://github.com/apache/iceberg/pull/7284) + - Parse snapshot-id as long in remove-statistics update [\#7235](https://github.com/apache/iceberg/pull/7235) * Spark - - Broadcast table instead of file IO in rewrite manifests [\#7263](https://github.com/apache/iceberg/pull/7263) - - Revert "Spark: Add "Iceberg" prefix to SparkTable name string for SparkUI [\#7273](https://github.com/apache/iceberg/pull/7273) + - Broadcast table instead of file IO in rewrite manifests [\#7263](https://github.com/apache/iceberg/pull/7263) + - Revert "Spark: Add "Iceberg" prefix to SparkTable name string for SparkUI [\#7273](https://github.com/apache/iceberg/pull/7273) * AWS - - Make AuthSession cache static [\#7289](https://github.com/apache/iceberg/pull/7289) - - Abort S3 input stream on close if not EOS [\#7262](https://github.com/apache/iceberg/pull/7262) - - Disable local credentials if remote signing is enabled [\#7230](https://github.com/apache/iceberg/pull/7230) - - Prevent token refresh scheduling on every sign request [\#7270](https://github.com/apache/iceberg/pull/7270) - - S3 Credentials provider support in DefaultAwsClientFactory [\#7066](https://github.com/apache/iceberg/pull/7066) + - Make AuthSession cache static [\#7289](https://github.com/apache/iceberg/pull/7289) + - Abort S3 input stream on close if not EOS [\#7262](https://github.com/apache/iceberg/pull/7262) + - Disable local credentials if remote signing is enabled [\#7230](https://github.com/apache/iceberg/pull/7230) + - Prevent token refresh scheduling on every sign request [\#7270](https://github.com/apache/iceberg/pull/7270) + - S3 Credentials provider support in DefaultAwsClientFactory [\#7066](https://github.com/apache/iceberg/pull/7066) ### 1.2.0 release @@ -293,57 +293,57 @@ The 1.2.0 release adds a variety of new features and bug fixes. Here is an overview: * Core - - Added AES GCM encrpytion stream spec ([\#5432](https://github.com/apache/iceberg/pull/5432)) - - Added support for Delta Lake to Iceberg table conversion ([\#6449](https://github.com/apache/iceberg/pull/6449), [\#6880](https://github.com/apache/iceberg/pull/6880)) - - Added support for `position_deletes` metadata table ([\#6365](https://github.com/apache/iceberg/pull/6365), [\#6716](https://github.com/apache/iceberg/pull/6716)) - - Added support for scan and commit metrics reporter that is pluggable through catalog ([\#6404](https://github.com/apache/iceberg/pull/6404), [\#6246](https://github.com/apache/iceberg/pull/6246), [\#6410](https://github.com/apache/iceberg/pull/6410)) - - Added support for branch commit for all operations ([\#4926](https://github.com/apache/iceberg/pull/4926), [\#5010](https://github.com/apache/iceberg/pull/5010)) - - Added `FileIO` support for ORC readers and writers ([\#6293](https://github.com/apache/iceberg/pull/6293)) - - Updated all actions to leverage bulk delete whenever possible ([\#6682](https://github.com/apache/iceberg/pull/6682)) - - Updated snapshot ID definition in Puffin spec to support statistics file reuse ([\#6272](https://github.com/apache/iceberg/pull/6267)) - - Added human-readable metrics information in `files` metadata table ([\#5376](https://github.com/apache/iceberg/pull/5376)) - - Fixed incorrect Parquet row group skipping when min and max values are `NaN` ([\#6517](https://github.com/apache/iceberg/pull/6517)) - - Fixed a bug that location provider could generate paths with double slash (`//`) which is not compatible in a Hadoop file system ([\#6777](https://github.com/apache/iceberg/pull/6777)) - - Fixed metadata table time travel failure for tables that performed schema evolution ([\#6980](https://github.com/apache/iceberg/pull/6980)) + - Added AES GCM encrpytion stream spec ([\#5432](https://github.com/apache/iceberg/pull/5432)) + - Added support for Delta Lake to Iceberg table conversion ([\#6449](https://github.com/apache/iceberg/pull/6449), [\#6880](https://github.com/apache/iceberg/pull/6880)) + - Added support for `position_deletes` metadata table ([\#6365](https://github.com/apache/iceberg/pull/6365), [\#6716](https://github.com/apache/iceberg/pull/6716)) + - Added support for scan and commit metrics reporter that is pluggable through catalog ([\#6404](https://github.com/apache/iceberg/pull/6404), [\#6246](https://github.com/apache/iceberg/pull/6246), [\#6410](https://github.com/apache/iceberg/pull/6410)) + - Added support for branch commit for all operations ([\#4926](https://github.com/apache/iceberg/pull/4926), [\#5010](https://github.com/apache/iceberg/pull/5010)) + - Added `FileIO` support for ORC readers and writers ([\#6293](https://github.com/apache/iceberg/pull/6293)) + - Updated all actions to leverage bulk delete whenever possible ([\#6682](https://github.com/apache/iceberg/pull/6682)) + - Updated snapshot ID definition in Puffin spec to support statistics file reuse ([\#6272](https://github.com/apache/iceberg/pull/6267)) + - Added human-readable metrics information in `files` metadata table ([\#5376](https://github.com/apache/iceberg/pull/5376)) + - Fixed incorrect Parquet row group skipping when min and max values are `NaN` ([\#6517](https://github.com/apache/iceberg/pull/6517)) + - Fixed a bug that location provider could generate paths with double slash (`//`) which is not compatible in a Hadoop file system ([\#6777](https://github.com/apache/iceberg/pull/6777)) + - Fixed metadata table time travel failure for tables that performed schema evolution ([\#6980](https://github.com/apache/iceberg/pull/6980)) * Spark - - Added time range query support for changelog table ([\#6350](https://github.com/apache/iceberg/pull/6350)) - - Added changelog view procedure for v1 table ([\#6012](https://github.com/apache/iceberg/pull/6012)) - - Added support for storage partition joins to improve read and write performance ([\#6371](https://github.com/apache/iceberg/pull/6371)) - - Updated default Arrow environment settings to improve read performance ([\#6550](https://github.com/apache/iceberg/pull/6550)) - - Added aggregate pushdown support for `min`, `max` and `count` to improve read performance ([\#6622](https://github.com/apache/iceberg/pull/6622)) - - Updated default distribution mode settings to improve write performance ([\#6828](https://github.com/apache/iceberg/pull/6828), [\#6838](https://github.com/apache/iceberg/pull/6838)) - - Updated DELETE to perform metadata-only update whenever possible to improve write performance ([\#6899](https://github.com/apache/iceberg/pull/6899)) - - Improved predicate pushdown support for write operations ([\#6636](https://github.com/apache/iceberg/pull/6633)) - - Added support for reading a branch or tag through table identifier and `VERSION AS OF` (a.k.a. `FOR SYSTEM_VERSION AS OF`) SQL syntax ([\#6717](https://github.com/apache/iceberg/pull/6717), [\#6575](https://github.com/apache/iceberg/pull/6575)) - - Added support for writing to a branch through identifier or through write-audit-publish (WAP) workflow settings ([\#6965](https://github.com/apache/iceberg/pull/6965), [\#7050](https://github.com/apache/iceberg/pull/7050)) - - Added DDL SQL extensions to create, replace and drop a branch or tag ([\#6638](https://github.com/apache/iceberg/pull/6638), [\#6637](https://github.com/apache/iceberg/pull/6637), [\#6752](https://github.com/apache/iceberg/pull/6752), [\#6807](https://github.com/apache/iceberg/pull/6807)) - - Added UDFs for `years`, `months`, `days` and `hours` transforms ([\#6207](https://github.com/apache/iceberg/pull/6207), [\#6261](https://github.com/apache/iceberg/pull/6261), [\#6300](https://github.com/apache/iceberg/pull/6300), [\#6339](https://github.com/apache/iceberg/pull/6339)) - - Added partition related stats for `add_files` procedure result ([\#6797](https://github.com/apache/iceberg/pull/6797)) - - Fixed a bug that `rewrite_manifests` procedure produced a new manifest even when there was no rewrite performed ([\#6659](https://github.com/apache/iceberg/pull/6695)) - - Fixed a bug that statistics files were not cleaned up in `expire_snapshots` procedure ([\#6090](https://github.com/apache/iceberg/pull/6090)) + - Added time range query support for changelog table ([\#6350](https://github.com/apache/iceberg/pull/6350)) + - Added changelog view procedure for v1 table ([\#6012](https://github.com/apache/iceberg/pull/6012)) + - Added support for storage partition joins to improve read and write performance ([\#6371](https://github.com/apache/iceberg/pull/6371)) + - Updated default Arrow environment settings to improve read performance ([\#6550](https://github.com/apache/iceberg/pull/6550)) + - Added aggregate pushdown support for `min`, `max` and `count` to improve read performance ([\#6622](https://github.com/apache/iceberg/pull/6622)) + - Updated default distribution mode settings to improve write performance ([\#6828](https://github.com/apache/iceberg/pull/6828), [\#6838](https://github.com/apache/iceberg/pull/6838)) + - Updated DELETE to perform metadata-only update whenever possible to improve write performance ([\#6899](https://github.com/apache/iceberg/pull/6899)) + - Improved predicate pushdown support for write operations ([\#6636](https://github.com/apache/iceberg/pull/6633)) + - Added support for reading a branch or tag through table identifier and `VERSION AS OF` (a.k.a. `FOR SYSTEM_VERSION AS OF`) SQL syntax ([\#6717](https://github.com/apache/iceberg/pull/6717), [\#6575](https://github.com/apache/iceberg/pull/6575)) + - Added support for writing to a branch through identifier or through write-audit-publish (WAP) workflow settings ([\#6965](https://github.com/apache/iceberg/pull/6965), [\#7050](https://github.com/apache/iceberg/pull/7050)) + - Added DDL SQL extensions to create, replace and drop a branch or tag ([\#6638](https://github.com/apache/iceberg/pull/6638), [\#6637](https://github.com/apache/iceberg/pull/6637), [\#6752](https://github.com/apache/iceberg/pull/6752), [\#6807](https://github.com/apache/iceberg/pull/6807)) + - Added UDFs for `years`, `months`, `days` and `hours` transforms ([\#6207](https://github.com/apache/iceberg/pull/6207), [\#6261](https://github.com/apache/iceberg/pull/6261), [\#6300](https://github.com/apache/iceberg/pull/6300), [\#6339](https://github.com/apache/iceberg/pull/6339)) + - Added partition related stats for `add_files` procedure result ([\#6797](https://github.com/apache/iceberg/pull/6797)) + - Fixed a bug that `rewrite_manifests` procedure produced a new manifest even when there was no rewrite performed ([\#6659](https://github.com/apache/iceberg/pull/6695)) + - Fixed a bug that statistics files were not cleaned up in `expire_snapshots` procedure ([\#6090](https://github.com/apache/iceberg/pull/6090)) * Flink - - Added support for metadata tables ([\#6222](https://github.com/apache/iceberg/pull/6222)) - - Added support for read options in Flink source ([\#5967](https://github.com/apache/iceberg/pull/5967)) - - Added support for reading and writing Avro `GenericRecord` ([\#6557](https://github.com/apache/iceberg/pull/6557), [\#6584](https://github.com/apache/iceberg/pull/6584)) - - Added support for reading a branch or tag and write to a branch ([\#6660](https://github.com/apache/iceberg/pull/6660), [\#5029](https://github.com/apache/iceberg/pull/5029)) - - Added throttling support for streaming read ([\#6299](https://github.com/apache/iceberg/pull/6299)) - - Added support for multiple sinks for the same table in the same job ([\#6528](https://github.com/apache/iceberg/pull/6528)) - - Fixed a bug that metrics config was not applied to equality and position deletes ([\#6271](https://github.com/apache/iceberg/pull/6271), [\#6313](https://github.com/apache/iceberg/pull/6313)) + - Added support for metadata tables ([\#6222](https://github.com/apache/iceberg/pull/6222)) + - Added support for read options in Flink source ([\#5967](https://github.com/apache/iceberg/pull/5967)) + - Added support for reading and writing Avro `GenericRecord` ([\#6557](https://github.com/apache/iceberg/pull/6557), [\#6584](https://github.com/apache/iceberg/pull/6584)) + - Added support for reading a branch or tag and write to a branch ([\#6660](https://github.com/apache/iceberg/pull/6660), [\#5029](https://github.com/apache/iceberg/pull/5029)) + - Added throttling support for streaming read ([\#6299](https://github.com/apache/iceberg/pull/6299)) + - Added support for multiple sinks for the same table in the same job ([\#6528](https://github.com/apache/iceberg/pull/6528)) + - Fixed a bug that metrics config was not applied to equality and position deletes ([\#6271](https://github.com/apache/iceberg/pull/6271), [\#6313](https://github.com/apache/iceberg/pull/6313)) * Vendor Integrations - - Added Snowflake catalog integration ([\#6428](https://github.com/apache/iceberg/pull/6428)) - - Added AWS sigV4 authentication support for REST catalog ([\#6951](https://github.com/apache/iceberg/pull/6951)) - - Added support for AWS S3 remote signing ([\#6169](https://github.com/apache/iceberg/pull/6169), [\#6835](https://github.com/apache/iceberg/pull/6835), [\#7080](https://github.com/apache/iceberg/pull/7080)) - - Updated AWS Glue catalog to skip table version archive by default ([\#6919](https://github.com/apache/iceberg/pull/6916)) - - Updated AWS Glue catalog to not require a warehouse location ([\#6586](https://github.com/apache/iceberg/pull/6586)) - - Fixed a bug that a bucket-only AWS S3 location such as `s3://my-bucket` could not be parsed ([\#6352](https://github.com/apache/iceberg/pull/6352)) - - Fixed a bug that unnecessary HTTP client dependencies had to be included to use any AWS integration ([\#6746](https://github.com/apache/iceberg/pull/6746)) - - Fixed a bug that AWS Glue catalog did not respect custom catalog ID when determining default warehouse location ([\#6223](https://github.com/apache/iceberg/pull/6223)) - - Fixes a bug that AWS DynamoDB catalog namespace listing result was incomplete ([\#6823](https://github.com/apache/iceberg/pull/6823)) + - Added Snowflake catalog integration ([\#6428](https://github.com/apache/iceberg/pull/6428)) + - Added AWS sigV4 authentication support for REST catalog ([\#6951](https://github.com/apache/iceberg/pull/6951)) + - Added support for AWS S3 remote signing ([\#6169](https://github.com/apache/iceberg/pull/6169), [\#6835](https://github.com/apache/iceberg/pull/6835), [\#7080](https://github.com/apache/iceberg/pull/7080)) + - Updated AWS Glue catalog to skip table version archive by default ([\#6919](https://github.com/apache/iceberg/pull/6916)) + - Updated AWS Glue catalog to not require a warehouse location ([\#6586](https://github.com/apache/iceberg/pull/6586)) + - Fixed a bug that a bucket-only AWS S3 location such as `s3://my-bucket` could not be parsed ([\#6352](https://github.com/apache/iceberg/pull/6352)) + - Fixed a bug that unnecessary HTTP client dependencies had to be included to use any AWS integration ([\#6746](https://github.com/apache/iceberg/pull/6746)) + - Fixed a bug that AWS Glue catalog did not respect custom catalog ID when determining default warehouse location ([\#6223](https://github.com/apache/iceberg/pull/6223)) + - Fixes a bug that AWS DynamoDB catalog namespace listing result was incomplete ([\#6823](https://github.com/apache/iceberg/pull/6823)) * Dependencies - - Upgraded ORC to 1.8.1 ([\#6349](https://github.com/apache/iceberg/pull/6349)) - - Upgraded Jackson to 2.14.1 ([\#6168](https://github.com/apache/iceberg/pull/6168)) - - Upgraded AWS SDK V2 to 2.20.18 ([\#7003](https://github.com/apache/iceberg/pull/7003)) - - Upgraded Nessie to 0.50.0 ([\#6875](https://github.com/apache/iceberg/pull/6875)) + - Upgraded ORC to 1.8.1 ([\#6349](https://github.com/apache/iceberg/pull/6349)) + - Upgraded Jackson to 2.14.1 ([\#6168](https://github.com/apache/iceberg/pull/6168)) + - Upgraded AWS SDK V2 to 2.20.18 ([\#7003](https://github.com/apache/iceberg/pull/7003)) + - Upgraded Nessie to 0.50.0 ([\#6875](https://github.com/apache/iceberg/pull/6875)) For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.2.0). @@ -356,24 +356,24 @@ and adds a variety of new features. Here is an overview: * Core - - Puffin statistics have been [added to the Table API](https://github.com/apache/iceberg/pull/4945) - - Support for [Table scan reporting](https://github.com/apache/iceberg/pull/5268), which enables collection of statistics of the table scans. - - [Add file sequence number to ManifestEntry](https://github.com/apache/iceberg/pull/6002) - - [Support register table](https://github.com/apache/iceberg/pull/5037) for all the catalogs (previously it was only for Hive) - - [Support performing merge appends and delete files on branches](https://github.com/apache/iceberg/pull/5618) - - [Improved Expire Snapshots FileCleanupStrategy](https://github.com/apache/iceberg/pull/5669) - - [SnapshotProducer supports branch writes](https://github.com/apache/iceberg/pull/4926) + - Puffin statistics have been [added to the Table API](https://github.com/apache/iceberg/pull/4945) + - Support for [Table scan reporting](https://github.com/apache/iceberg/pull/5268), which enables collection of statistics of the table scans. + - [Add file sequence number to ManifestEntry](https://github.com/apache/iceberg/pull/6002) + - [Support register table](https://github.com/apache/iceberg/pull/5037) for all the catalogs (previously it was only for Hive) + - [Support performing merge appends and delete files on branches](https://github.com/apache/iceberg/pull/5618) + - [Improved Expire Snapshots FileCleanupStrategy](https://github.com/apache/iceberg/pull/5669) + - [SnapshotProducer supports branch writes](https://github.com/apache/iceberg/pull/4926) * Spark - - [Support for aggregate expressions](https://github.com/apache/iceberg/pull/5961) - - [SparkChangelogTable for querying changelogs](https://github.com/apache/iceberg/pull/5740) - - Dropped support for Apache Spark 3.0 + - [Support for aggregate expressions](https://github.com/apache/iceberg/pull/5961) + - [SparkChangelogTable for querying changelogs](https://github.com/apache/iceberg/pull/5740) + - Dropped support for Apache Spark 3.0 * Flink - - [FLIP-27 reader is supported in SQL](https://github.com/apache/iceberg/pull/5318) - - Added support for Flink 1.16, dropped support for Flink 1.13 + - [FLIP-27 reader is supported in SQL](https://github.com/apache/iceberg/pull/5318) + - Added support for Flink 1.16, dropped support for Flink 1.13 * Dependencies - - [AWS SDK: 2.17.257](https://github.com/apache/iceberg/pull/5612) - - [Nessie: 0.44](https://github.com/apache/iceberg/pull/6008) - - [Apache ORC: 1.8.0](https://github.com/apache/iceberg/pull/5699) (Also, supports [setting bloom filters on row groups](https://github.com/apache/iceberg/pull/5313/files)) + - [AWS SDK: 2.17.257](https://github.com/apache/iceberg/pull/5612) + - [Nessie: 0.44](https://github.com/apache/iceberg/pull/6008) + - [Apache ORC: 1.8.0](https://github.com/apache/iceberg/pull/5699) (Also, supports [setting bloom filters on row groups](https://github.com/apache/iceberg/pull/5313/files)) For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.1.0). @@ -396,19 +396,19 @@ This release includes all bug fixes from the 0.14.x patch releases. #### Notable bug fixes * API - - API: Fix ID assignment in schema merging ([#5395](https://github.com/apache/iceberg/pull/5395)) + - API: Fix ID assignment in schema merging ([#5395](https://github.com/apache/iceberg/pull/5395)) * Core - - Core: Fix snapshot log with intermediate transaction snapshots ([#5568](https://github.com/apache/iceberg/pull/5568)) - - Core: Fix exception handling in BaseTaskWriter ([#5683](https://github.com/apache/iceberg/pull/5683)) - - Core: Support deleting tables without metadata files ([#5510](https://github.com/apache/iceberg/pull/5510)) - - Core: Add CommitStateUnknownException handling to REST ([#5694](https://github.com/apache/iceberg/pull/5694)) + - Fix snapshot log with intermediate transaction snapshots ([#5568](https://github.com/apache/iceberg/pull/5568)) + - Fix exception handling in BaseTaskWriter ([#5683](https://github.com/apache/iceberg/pull/5683)) + - Support deleting tables without metadata files ([#5510](https://github.com/apache/iceberg/pull/5510)) + - Add CommitStateUnknownException handling to REST ([#5694](https://github.com/apache/iceberg/pull/5694)) * Spark - - Spark: Fix stats in rewrite metadata action ([#5691](https://github.com/apache/iceberg/pull/5691)) + - Spark: Fix stats in rewrite metadata action ([#5691](https://github.com/apache/iceberg/pull/5691)) * File Formats - - Parquet: Close zstd input stream early to avoid memory pressure ([#5681](https://github.com/apache/iceberg/pull/5681)) + - Parquet: Close zstd input stream early to avoid memory pressure ([#5681](https://github.com/apache/iceberg/pull/5681)) * Vendor Integrations - - Core, AWS: Fix Kryo serialization failure for FileIO ([#5437](https://github.com/apache/iceberg/pull/5437)) - - AWS: S3OutputStream - failure to close should persist on subsequent close calls ([#5311](https://github.com/apache/iceberg/pull/5311)) + - Core, AWS: Fix Kryo serialization failure for FileIO ([#5437](https://github.com/apache/iceberg/pull/5437)) + - AWS: S3OutputStream - failure to close should persist on subsequent close calls ([#5311](https://github.com/apache/iceberg/pull/5311)) ### 0.14.0 release @@ -431,109 +431,109 @@ Apache Iceberg 0.14.0 was released on 16 July 2022. #### High-level features * API - - Added IcebergBuild to expose Iceberg version and build information - - Added binary compatibility checking to the build ([#4638](https://github.com/apache/iceberg/pull/4638), [#4798](https://github.com/apache/iceberg/pull/4798)) - - Added a new IncrementalAppendScan interface and planner implementation ([#4580](https://github.com/apache/iceberg/pull/4580)) - - Added a new IncrementalChangelogScan interface ([#4870](https://github.com/apache/iceberg/pull/4870)) - - Refactored the ScanTask hierarchy to create new task types for changelog scans ([#5077](https://github.com/apache/iceberg/pull/5077)) - - Added expression sanitizer ([#4672](https://github.com/apache/iceberg/pull/4672)) - - Added utility to check expression equivalence ([#4947](https://github.com/apache/iceberg/pull/4947)) - - Added support for serializing FileIO instances using initialization properties ([#5178](https://github.com/apache/iceberg/pull/5178)) - - Updated Snapshot methods to accept a FileIO to read metadata files, deprecated old methods ([#4873](https://github.com/apache/iceberg/pull/4873)) - - Added optional interfaces to FileIO, for batch deletes ([#4052](https://github.com/apache/iceberg/pull/4052)), prefix operations ([#5096](https://github.com/apache/iceberg/pull/5096)), and ranged reads ([#4608](https://github.com/apache/iceberg/pull/4608)) + - Added IcebergBuild to expose Iceberg version and build information + - Added binary compatibility checking to the build ([#4638](https://github.com/apache/iceberg/pull/4638), [#4798](https://github.com/apache/iceberg/pull/4798)) + - Added a new IncrementalAppendScan interface and planner implementation ([#4580](https://github.com/apache/iceberg/pull/4580)) + - Added a new IncrementalChangelogScan interface ([#4870](https://github.com/apache/iceberg/pull/4870)) + - Refactored the ScanTask hierarchy to create new task types for changelog scans ([#5077](https://github.com/apache/iceberg/pull/5077)) + - Added expression sanitizer ([#4672](https://github.com/apache/iceberg/pull/4672)) + - Added utility to check expression equivalence ([#4947](https://github.com/apache/iceberg/pull/4947)) + - Added support for serializing FileIO instances using initialization properties ([#5178](https://github.com/apache/iceberg/pull/5178)) + - Updated Snapshot methods to accept a FileIO to read metadata files, deprecated old methods ([#4873](https://github.com/apache/iceberg/pull/4873)) + - Added optional interfaces to FileIO, for batch deletes ([#4052](https://github.com/apache/iceberg/pull/4052)), prefix operations ([#5096](https://github.com/apache/iceberg/pull/5096)), and ranged reads ([#4608](https://github.com/apache/iceberg/pull/4608)) * Core - - Added a common client for REST-based catalog services that uses a change-based protocol ([#4320](https://github.com/apache/iceberg/pull/4320), [#4319](https://github.com/apache/iceberg/pull/4319)) - - Added Puffin, a file format for statistics and index payloads or sketches ([#4944](https://github.com/apache/iceberg/pull/4944), [#4537](https://github.com/apache/iceberg/pull/4537)) - - Added snapshot references to track tags and branches ([#4019](https://github.com/apache/iceberg/pull/4019)) - - ManageSnapshots now supports multiple operations using transactions, and added branch and tag operations ([#4128](https://github.com/apache/iceberg/pull/4128), [#4071](https://github.com/apache/iceberg/pull/4071)) - - ReplacePartitions and OverwriteFiles now support serializable isolation ([#2925](https://github.com/apache/iceberg/pull/2925), [#4052](https://github.com/apache/iceberg/pull/4052)) - - Added new metadata tables: `data_files` ([#4336](https://github.com/apache/iceberg/pull/4336)), `delete_files` ([#4243](https://github.com/apache/iceberg/pull/4243)), `all_delete_files`, and `all_files` ([#4694](https://github.com/apache/iceberg/pull/4694)) - - Added deleted files to the `files` metadata table ([#4336](https://github.com/apache/iceberg/pull/4336)) and delete file counts to the `manifests` table ([#4764](https://github.com/apache/iceberg/pull/4764)) - - Added support for predicate pushdown for the `all_data_files` metadata table ([#4382](https://github.com/apache/iceberg/pull/4382)) and the `all_manifests` table ([#4736](https://github.com/apache/iceberg/pull/4736)) - - Added support for catalogs to default table properties on creation ([#4011](https://github.com/apache/iceberg/pull/4011)) - - Updated sort order construction to ensure all partition fields are added to avoid partition closed failures ([#5131](https://github.com/apache/iceberg/pull/5131)) + - Added a common client for REST-based catalog services that uses a change-based protocol ([#4320](https://github.com/apache/iceberg/pull/4320), [#4319](https://github.com/apache/iceberg/pull/4319)) + - Added Puffin, a file format for statistics and index payloads or sketches ([#4944](https://github.com/apache/iceberg/pull/4944), [#4537](https://github.com/apache/iceberg/pull/4537)) + - Added snapshot references to track tags and branches ([#4019](https://github.com/apache/iceberg/pull/4019)) + - ManageSnapshots now supports multiple operations using transactions, and added branch and tag operations ([#4128](https://github.com/apache/iceberg/pull/4128), [#4071](https://github.com/apache/iceberg/pull/4071)) + - ReplacePartitions and OverwriteFiles now support serializable isolation ([#2925](https://github.com/apache/iceberg/pull/2925), [#4052](https://github.com/apache/iceberg/pull/4052)) + - Added new metadata tables: `data_files` ([#4336](https://github.com/apache/iceberg/pull/4336)), `delete_files` ([#4243](https://github.com/apache/iceberg/pull/4243)), `all_delete_files`, and `all_files` ([#4694](https://github.com/apache/iceberg/pull/4694)) + - Added deleted files to the `files` metadata table ([#4336](https://github.com/apache/iceberg/pull/4336)) and delete file counts to the `manifests` table ([#4764](https://github.com/apache/iceberg/pull/4764)) + - Added support for predicate pushdown for the `all_data_files` metadata table ([#4382](https://github.com/apache/iceberg/pull/4382)) and the `all_manifests` table ([#4736](https://github.com/apache/iceberg/pull/4736)) + - Added support for catalogs to default table properties on creation ([#4011](https://github.com/apache/iceberg/pull/4011)) + - Updated sort order construction to ensure all partition fields are added to avoid partition closed failures ([#5131](https://github.com/apache/iceberg/pull/5131)) * Spark - - Spark 3.3 is now supported ([#5056](https://github.com/apache/iceberg/pull/5056)) - - Added SQL time travel using `AS OF` syntax in Spark 3.3 ([#5156](https://github.com/apache/iceberg/pull/5156)) - - Scala 2.13 is now supported for Spark 3.2 and 3.3 ([#4009](https://github.com/apache/iceberg/pull/4009)) - - Added support for the `mergeSchema` option for DataFrame writes ([#4154](https://github.com/apache/iceberg/pull/4154)) - - MERGE and UPDATE queries now support the lazy / merge-on-read strategy ([#3984](https://github.com/apache/iceberg/pull/3984), [#4047](https://github.com/apache/iceberg/pull/4047)) - - Added zorder rewrite strategy to the `rewrite_data_files` stored procedure and action ([#3983](https://github.com/apache/iceberg/pull/3983), [#4902](https://github.com/apache/iceberg/pull/4902)) - - Added a `register_table` stored procedure to create tables from metadata JSON files ([#4810](https://github.com/apache/iceberg/pull/4810)) - - Added a `publish_changes` stored procedure to publish staged commits by ID ([#4715](https://github.com/apache/iceberg/pull/4715)) - - Added `CommitMetadata` helper class to set snapshot summary properties from SQL ([#4956](https://github.com/apache/iceberg/pull/4956)) - - Added support to supply a file listing to remove orphan data files procedure and action ([#4503](https://github.com/apache/iceberg/pull/4503)) - - Added FileIO metrics to the Spark UI ([#4030](https://github.com/apache/iceberg/pull/4030), [#4050](https://github.com/apache/iceberg/pull/4050)) - - DROP TABLE now supports the PURGE flag ([#3056](https://github.com/apache/iceberg/pull/3056)) - - Added support for custom isolation level for dynamic partition overwrites ([#2925](https://github.com/apache/iceberg/pull/2925)) and filter overwrites ([#4293](https://github.com/apache/iceberg/pull/4293)) - - Schema identifier fields are now shown in table properties ([#4475](https://github.com/apache/iceberg/pull/4475)) - - Abort cleanup now supports parallel execution ([#4704](https://github.com/apache/iceberg/pull/4704)) + - Spark 3.3 is now supported ([#5056](https://github.com/apache/iceberg/pull/5056)) + - Added SQL time travel using `AS OF` syntax in Spark 3.3 ([#5156](https://github.com/apache/iceberg/pull/5156)) + - Scala 2.13 is now supported for Spark 3.2 and 3.3 ([#4009](https://github.com/apache/iceberg/pull/4009)) + - Added support for the `mergeSchema` option for DataFrame writes ([#4154](https://github.com/apache/iceberg/pull/4154)) + - MERGE and UPDATE queries now support the lazy / merge-on-read strategy ([#3984](https://github.com/apache/iceberg/pull/3984), [#4047](https://github.com/apache/iceberg/pull/4047)) + - Added zorder rewrite strategy to the `rewrite_data_files` stored procedure and action ([#3983](https://github.com/apache/iceberg/pull/3983), [#4902](https://github.com/apache/iceberg/pull/4902)) + - Added a `register_table` stored procedure to create tables from metadata JSON files ([#4810](https://github.com/apache/iceberg/pull/4810)) + - Added a `publish_changes` stored procedure to publish staged commits by ID ([#4715](https://github.com/apache/iceberg/pull/4715)) + - Added `CommitMetadata` helper class to set snapshot summary properties from SQL ([#4956](https://github.com/apache/iceberg/pull/4956)) + - Added support to supply a file listing to remove orphan data files procedure and action ([#4503](https://github.com/apache/iceberg/pull/4503)) + - Added FileIO metrics to the Spark UI ([#4030](https://github.com/apache/iceberg/pull/4030), [#4050](https://github.com/apache/iceberg/pull/4050)) + - DROP TABLE now supports the PURGE flag ([#3056](https://github.com/apache/iceberg/pull/3056)) + - Added support for custom isolation level for dynamic partition overwrites ([#2925](https://github.com/apache/iceberg/pull/2925)) and filter overwrites ([#4293](https://github.com/apache/iceberg/pull/4293)) + - Schema identifier fields are now shown in table properties ([#4475](https://github.com/apache/iceberg/pull/4475)) + - Abort cleanup now supports parallel execution ([#4704](https://github.com/apache/iceberg/pull/4704)) * Flink - - Flink 1.15 is now supported ([#4553](https://github.com/apache/iceberg/pull/4553)) - - Flink 1.12 support was removed ([#4551](https://github.com/apache/iceberg/pull/4551)) - - Added a FLIP-27 source and builder to 1.14 and 1.15 ([#5109](https://github.com/apache/iceberg/pull/5109)) - - Added an option to set the monitor interval ([#4887](https://github.com/apache/iceberg/pull/4887)) and an option to limit the number of snapshots in a streaming read planning operation ([#4943](https://github.com/apache/iceberg/pull/4943)) - - Added support for write options, like `write-format` to Flink sink builder ([#3998](https://github.com/apache/iceberg/pull/3998)) - - Added support for task locality when reading from HDFS ([#3817](https://github.com/apache/iceberg/pull/3817)) - - Use Hadoop configuration files from `hadoop-conf-dir` property ([#4622](https://github.com/apache/iceberg/pull/4622)) + - Flink 1.15 is now supported ([#4553](https://github.com/apache/iceberg/pull/4553)) + - Flink 1.12 support was removed ([#4551](https://github.com/apache/iceberg/pull/4551)) + - Added a FLIP-27 source and builder to 1.14 and 1.15 ([#5109](https://github.com/apache/iceberg/pull/5109)) + - Added an option to set the monitor interval ([#4887](https://github.com/apache/iceberg/pull/4887)) and an option to limit the number of snapshots in a streaming read planning operation ([#4943](https://github.com/apache/iceberg/pull/4943)) + - Added support for write options, like `write-format` to Flink sink builder ([#3998](https://github.com/apache/iceberg/pull/3998)) + - Added support for task locality when reading from HDFS ([#3817](https://github.com/apache/iceberg/pull/3817)) + - Use Hadoop configuration files from `hadoop-conf-dir` property ([#4622](https://github.com/apache/iceberg/pull/4622)) * Vendor integrations - - Added Dell ECS integration ([#3376](https://github.com/apache/iceberg/pull/3376), [#4221](https://github.com/apache/iceberg/pull/4221)) - - JDBC catalog now supports namespace properties ([#3275](https://github.com/apache/iceberg/pull/3275)) - - AWS Glue catalog supports native Glue locking ([#4166](https://github.com/apache/iceberg/pull/4166)) - - AWS S3FileIO supports using S3 access points ([#4334](https://github.com/apache/iceberg/pull/4334)), bulk operations ([#4052](https://github.com/apache/iceberg/pull/4052), [#5096](https://github.com/apache/iceberg/pull/5096)), ranged reads ([#4608](https://github.com/apache/iceberg/pull/4608)), and tagging at write time or in place of deletes ([#4259](https://github.com/apache/iceberg/pull/4259), [#4342](https://github.com/apache/iceberg/pull/4342)) - - AWS GlueCatalog supports passing LakeFormation credentials ([#4280](https://github.com/apache/iceberg/pull/4280)) - - AWS DynamoDB catalog and lock supports overriding the DynamoDB endpoint ([#4726](https://github.com/apache/iceberg/pull/4726)) - - Nessie now supports namespaces and namespace properties ([#4385](https://github.com/apache/iceberg/pull/4385), [#4610](https://github.com/apache/iceberg/pull/4610)) - - Nessie now passes most common catalog tests ([#4392](https://github.com/apache/iceberg/pull/4392)) + - Added Dell ECS integration ([#3376](https://github.com/apache/iceberg/pull/3376), [#4221](https://github.com/apache/iceberg/pull/4221)) + - JDBC catalog now supports namespace properties ([#3275](https://github.com/apache/iceberg/pull/3275)) + - AWS Glue catalog supports native Glue locking ([#4166](https://github.com/apache/iceberg/pull/4166)) + - AWS S3FileIO supports using S3 access points ([#4334](https://github.com/apache/iceberg/pull/4334)), bulk operations ([#4052](https://github.com/apache/iceberg/pull/4052), [#5096](https://github.com/apache/iceberg/pull/5096)), ranged reads ([#4608](https://github.com/apache/iceberg/pull/4608)), and tagging at write time or in place of deletes ([#4259](https://github.com/apache/iceberg/pull/4259), [#4342](https://github.com/apache/iceberg/pull/4342)) + - AWS GlueCatalog supports passing LakeFormation credentials ([#4280](https://github.com/apache/iceberg/pull/4280)) + - AWS DynamoDB catalog and lock supports overriding the DynamoDB endpoint ([#4726](https://github.com/apache/iceberg/pull/4726)) + - Nessie now supports namespaces and namespace properties ([#4385](https://github.com/apache/iceberg/pull/4385), [#4610](https://github.com/apache/iceberg/pull/4610)) + - Nessie now passes most common catalog tests ([#4392](https://github.com/apache/iceberg/pull/4392)) * Parquet - - Added support for row group skipping using Parquet bloom filters ([#4938](https://github.com/apache/iceberg/pull/4938)) - - Added table configuration options for writing Parquet bloom filters ([#5035](https://github.com/apache/iceberg/pull/5035)) + - Added support for row group skipping using Parquet bloom filters ([#4938](https://github.com/apache/iceberg/pull/4938)) + - Added table configuration options for writing Parquet bloom filters ([#5035](https://github.com/apache/iceberg/pull/5035)) * ORC - - Support file rolling at a target file size ([#4419](https://github.com/apache/iceberg/pull/4419)) - - Support table compression settings, `write.orc.compression-codec` and `write.orc.compression-strategy` ([#4273](https://github.com/apache/iceberg/pull/4273)) + - Support file rolling at a target file size ([#4419](https://github.com/apache/iceberg/pull/4419)) + - Support table compression settings, `write.orc.compression-codec` and `write.orc.compression-strategy` ([#4273](https://github.com/apache/iceberg/pull/4273)) #### Performance improvements * Core - - Fixed manifest file handling in scan planning to open manifests in the planning threadpool ([#5206](https://github.com/apache/iceberg/pull/5206)) - - Avoided an extra S3 HEAD request by passing file length when opening manifest files ([#5207](https://github.com/apache/iceberg/pull/5207)) - - Refactored Arrow vectorized readers to avoid extra dictionary copies ([#5137](https://github.com/apache/iceberg/pull/5137)) - - Improved Arrow decimal handling to improve decimal performance ([#5168](https://github.com/apache/iceberg/pull/5168), [#5198](https://github.com/apache/iceberg/pull/5198)) - - Added support for Avro files with Zstd compression ([#4083](https://github.com/apache/iceberg/pull/4083)) - - Column metrics are now disabled by default after the first 32 columns ([#3959](https://github.com/apache/iceberg/pull/3959), [#5215](https://github.com/apache/iceberg/pull/5215)) - - Updated delete filters to copy row wrappers to avoid expensive type analysis ([#5249](https://github.com/apache/iceberg/pull/5249)) - - Snapshot expiration supports parallel execution ([#4148](https://github.com/apache/iceberg/pull/4148)) - - Manifest updates can use a custom thread pool ([#4146](https://github.com/apache/iceberg/pull/4146)) + - Fixed manifest file handling in scan planning to open manifests in the planning threadpool ([#5206](https://github.com/apache/iceberg/pull/5206)) + - Avoided an extra S3 HEAD request by passing file length when opening manifest files ([#5207](https://github.com/apache/iceberg/pull/5207)) + - Refactored Arrow vectorized readers to avoid extra dictionary copies ([#5137](https://github.com/apache/iceberg/pull/5137)) + - Improved Arrow decimal handling to improve decimal performance ([#5168](https://github.com/apache/iceberg/pull/5168), [#5198](https://github.com/apache/iceberg/pull/5198)) + - Added support for Avro files with Zstd compression ([#4083](https://github.com/apache/iceberg/pull/4083)) + - Column metrics are now disabled by default after the first 32 columns ([#3959](https://github.com/apache/iceberg/pull/3959), [#5215](https://github.com/apache/iceberg/pull/5215)) + - Updated delete filters to copy row wrappers to avoid expensive type analysis ([#5249](https://github.com/apache/iceberg/pull/5249)) + - Snapshot expiration supports parallel execution ([#4148](https://github.com/apache/iceberg/pull/4148)) + - Manifest updates can use a custom thread pool ([#4146](https://github.com/apache/iceberg/pull/4146)) * Spark - - Parquet vectorized reads are enabled by default ([#4196](https://github.com/apache/iceberg/pull/4196)) - - Scan statistics now adjust row counts for split data files ([#4446](https://github.com/apache/iceberg/pull/4446)) - - Implemented `SupportsReportStatistics` in `ScanBuilder` to work around SPARK-38962 ([#5136](https://github.com/apache/iceberg/pull/5136)) - - Updated Spark tables to avoid expensive (and inaccurate) size estimation ([#5225](https://github.com/apache/iceberg/pull/5225)) + - Parquet vectorized reads are enabled by default ([#4196](https://github.com/apache/iceberg/pull/4196)) + - Scan statistics now adjust row counts for split data files ([#4446](https://github.com/apache/iceberg/pull/4446)) + - Implemented `SupportsReportStatistics` in `ScanBuilder` to work around SPARK-38962 ([#5136](https://github.com/apache/iceberg/pull/5136)) + - Updated Spark tables to avoid expensive (and inaccurate) size estimation ([#5225](https://github.com/apache/iceberg/pull/5225)) * Flink - - Operators will now use a worker pool per job ([#4177](https://github.com/apache/iceberg/pull/4177)) - - Fixed `ClassCastException` thrown when reading arrays from Parquet ([#4432](https://github.com/apache/iceberg/pull/4432)) + - Operators will now use a worker pool per job ([#4177](https://github.com/apache/iceberg/pull/4177)) + - Fixed `ClassCastException` thrown when reading arrays from Parquet ([#4432](https://github.com/apache/iceberg/pull/4432)) * Hive - - Added vectorized Parquet reads for Hive 3 ([#3980](https://github.com/apache/iceberg/pull/3980)) - - Improved generic reader performance using copy instead of create ([#4218](https://github.com/apache/iceberg/pull/4218)) + - Added vectorized Parquet reads for Hive 3 ([#3980](https://github.com/apache/iceberg/pull/3980)) + - Improved generic reader performance using copy instead of create ([#4218](https://github.com/apache/iceberg/pull/4218)) #### Notable bug fixes This release includes all bug fixes from the 0.13.x patch releases. * Core - - Fixed an exception thrown when metadata-only deletes encounter delete files that are partially matched ([#4304](https://github.com/apache/iceberg/pull/4304)) - - Fixed transaction retries for changes without validations, like schema updates, that could ignore an update ([#4464](https://github.com/apache/iceberg/pull/4464)) - - Fixed failures when reading metadata tables with evolved partition specs ([#4520](https://github.com/apache/iceberg/pull/4520), [#4560](https://github.com/apache/iceberg/pull/4560)) - - Fixed delete files dropped when a manifest is rewritten following a format version upgrade ([#4514](https://github.com/apache/iceberg/pull/4514)) - - Fixed missing metadata files resulting from an OOM during commit cleanup ([#4673](https://github.com/apache/iceberg/pull/4673)) - - Updated logging to use sanitized expressions to avoid leaking values ([#4672](https://github.com/apache/iceberg/pull/4672)) + - Fixed an exception thrown when metadata-only deletes encounter delete files that are partially matched ([#4304](https://github.com/apache/iceberg/pull/4304)) + - Fixed transaction retries for changes without validations, like schema updates, that could ignore an update ([#4464](https://github.com/apache/iceberg/pull/4464)) + - Fixed failures when reading metadata tables with evolved partition specs ([#4520](https://github.com/apache/iceberg/pull/4520), [#4560](https://github.com/apache/iceberg/pull/4560)) + - Fixed delete files dropped when a manifest is rewritten following a format version upgrade ([#4514](https://github.com/apache/iceberg/pull/4514)) + - Fixed missing metadata files resulting from an OOM during commit cleanup ([#4673](https://github.com/apache/iceberg/pull/4673)) + - Updated logging to use sanitized expressions to avoid leaking values ([#4672](https://github.com/apache/iceberg/pull/4672)) * Spark - - Fixed Spark to skip calling abort when CommitStateUnknownException is thrown ([#4687](https://github.com/apache/iceberg/pull/4687)) - - Fixed MERGE commands with mixed case identifiers ([#4848](https://github.com/apache/iceberg/pull/4848)) + - Fixed Spark to skip calling abort when CommitStateUnknownException is thrown ([#4687](https://github.com/apache/iceberg/pull/4687)) + - Fixed MERGE commands with mixed case identifiers ([#4848](https://github.com/apache/iceberg/pull/4848)) * Flink - - Fixed table property update failures when tables have a primary key ([#4561](https://github.com/apache/iceberg/pull/4561)) + - Fixed table property update failures when tables have a primary key ([#4561](https://github.com/apache/iceberg/pull/4561)) * Integrations - - JDBC catalog behavior has been updated to pass common catalog tests ([#4220](https://github.com/apache/iceberg/pull/4220), [#4231](https://github.com/apache/iceberg/pull/4231)) + - JDBC catalog behavior has been updated to pass common catalog tests ([#4220](https://github.com/apache/iceberg/pull/4220), [#4231](https://github.com/apache/iceberg/pull/4231)) #### Dependency changes @@ -624,59 +624,59 @@ Apache Iceberg 0.13.0 was released on February 4th, 2022. **High-level features:** * **Core** - * Catalog caching now supports cache expiration through catalog property `cache.expiration-interval-ms` [[\#3543](https://github.com/apache/iceberg/pull/3543)] - * Catalog now supports registration of Iceberg table from a given metadata file location [[\#3851](https://github.com/apache/iceberg/pull/3851)] - * Hadoop catalog can be used with S3 and other file systems safely by using a lock manager [[\#3663](https://github.com/apache/iceberg/pull/3663)] + * Catalog caching now supports cache expiration through catalog property `cache.expiration-interval-ms` [[\#3543](https://github.com/apache/iceberg/pull/3543)] + * Catalog now supports registration of Iceberg table from a given metadata file location [[\#3851](https://github.com/apache/iceberg/pull/3851)] + * Hadoop catalog can be used with S3 and other file systems safely by using a lock manager [[\#3663](https://github.com/apache/iceberg/pull/3663)] * **Vendor Integrations** - * Google Cloud Storage (GCS) `FileIO` is supported with optimized read and write using GCS streaming transfer [[\#3711](https://github.com/apache/iceberg/pull/3711)] - * Aliyun Object Storage Service (OSS) `FileIO` is supported [[\#3553](https://github.com/apache/iceberg/pull/3553)] - * Any S3-compatible storage (e.g. MinIO) can now be accessed through AWS `S3FileIO` with custom endpoint and credential configurations [[\#3656](https://github.com/apache/iceberg/pull/3656)] [[\#3658](https://github.com/apache/iceberg/pull/3658)] - * AWS `S3FileIO` now supports server-side checksum validation [[\#3813](https://github.com/apache/iceberg/pull/3813)] - * AWS `GlueCatalog` now displays more table information including table location, description [[\#3467](https://github.com/apache/iceberg/pull/3467)] and columns [[\#3888](https://github.com/apache/iceberg/pull/3888)] - * Using multiple `FileIO`s based on file path scheme is supported by configuring a `ResolvingFileIO` [[\#3593](https://github.com/apache/iceberg/pull/3593)] + * Google Cloud Storage (GCS) `FileIO` is supported with optimized read and write using GCS streaming transfer [[\#3711](https://github.com/apache/iceberg/pull/3711)] + * Aliyun Object Storage Service (OSS) `FileIO` is supported [[\#3553](https://github.com/apache/iceberg/pull/3553)] + * Any S3-compatible storage (e.g. MinIO) can now be accessed through AWS `S3FileIO` with custom endpoint and credential configurations [[\#3656](https://github.com/apache/iceberg/pull/3656)] [[\#3658](https://github.com/apache/iceberg/pull/3658)] + * AWS `S3FileIO` now supports server-side checksum validation [[\#3813](https://github.com/apache/iceberg/pull/3813)] + * AWS `GlueCatalog` now displays more table information including table location, description [[\#3467](https://github.com/apache/iceberg/pull/3467)] and columns [[\#3888](https://github.com/apache/iceberg/pull/3888)] + * Using multiple `FileIO`s based on file path scheme is supported by configuring a `ResolvingFileIO` [[\#3593](https://github.com/apache/iceberg/pull/3593)] * **Spark** - * Spark 3.2 is supported [[\#3335](https://github.com/apache/iceberg/pull/3335)] with merge-on-read `DELETE` [[\#3970](https://github.com/apache/iceberg/pull/3970)] - * `RewriteDataFiles` action now supports sort-based table optimization [[\#2829](https://github.com/apache/iceberg/pull/2829)] and merge-on-read delete compaction [[\#3454](https://github.com/apache/iceberg/pull/3454)]. The corresponding Spark call procedure `rewrite_data_files` is also supported [[\#3375](https://github.com/apache/iceberg/pull/3375)] - * Time travel queries now use snapshot schema instead of the table's latest schema [[\#3722](https://github.com/apache/iceberg/pull/3722)] - * Spark vectorized reads now support row-level deletes [[\#3557](https://github.com/apache/iceberg/pull/3557)] [[\#3287](https://github.com/apache/iceberg/pull/3287)] - * `add_files` procedure now skips duplicated files by default (can be turned off with the `check_duplicate_files` flag) [[\#2895](https://github.com/apache/iceberg/issues/2779)], skips folder without file [[\#2895](https://github.com/apache/iceberg/issues/3455)] and partitions with `null` values [[\#2895](https://github.com/apache/iceberg/issues/3778)] instead of throwing exception, and supports partition pruning for faster table import [[\#3745](https://github.com/apache/iceberg/issues/3745)] + * Spark 3.2 is supported [[\#3335](https://github.com/apache/iceberg/pull/3335)] with merge-on-read `DELETE` [[\#3970](https://github.com/apache/iceberg/pull/3970)] + * `RewriteDataFiles` action now supports sort-based table optimization [[\#2829](https://github.com/apache/iceberg/pull/2829)] and merge-on-read delete compaction [[\#3454](https://github.com/apache/iceberg/pull/3454)]. The corresponding Spark call procedure `rewrite_data_files` is also supported [[\#3375](https://github.com/apache/iceberg/pull/3375)] + * Time travel queries now use snapshot schema instead of the table's latest schema [[\#3722](https://github.com/apache/iceberg/pull/3722)] + * Spark vectorized reads now support row-level deletes [[\#3557](https://github.com/apache/iceberg/pull/3557)] [[\#3287](https://github.com/apache/iceberg/pull/3287)] + * `add_files` procedure now skips duplicated files by default (can be turned off with the `check_duplicate_files` flag) [[\#2895](https://github.com/apache/iceberg/issues/2779)], skips folder without file [[\#2895](https://github.com/apache/iceberg/issues/3455)] and partitions with `null` values [[\#2895](https://github.com/apache/iceberg/issues/3778)] instead of throwing exception, and supports partition pruning for faster table import [[\#3745](https://github.com/apache/iceberg/issues/3745)] * **Flink** - * Flink 1.13 and 1.14 are supported [[\#3116](https://github.com/apache/iceberg/pull/3116)] [[\#3434](https://github.com/apache/iceberg/pull/3434)] - * Flink connector support is supported [[\#2666](https://github.com/apache/iceberg/pull/2666)] - * Upsert write option is supported [[\#2863](https://github.com/apache/iceberg/pull/2863)] + * Flink 1.13 and 1.14 are supported [[\#3116](https://github.com/apache/iceberg/pull/3116)] [[\#3434](https://github.com/apache/iceberg/pull/3434)] + * Flink connector support is supported [[\#2666](https://github.com/apache/iceberg/pull/2666)] + * Upsert write option is supported [[\#2863](https://github.com/apache/iceberg/pull/2863)] * **Hive** - * Table listing in Hive catalog can now skip non-Iceberg tables by disabling flag `list-all-tables` [[\#3908](https://github.com/apache/iceberg/pull/3908)] - * Hive tables imported to Iceberg can now be read by `IcebergInputFormat` [[\#3312](https://github.com/apache/iceberg/pull/3312)] + * Table listing in Hive catalog can now skip non-Iceberg tables by disabling flag `list-all-tables` [[\#3908](https://github.com/apache/iceberg/pull/3908)] + * Hive tables imported to Iceberg can now be read by `IcebergInputFormat` [[\#3312](https://github.com/apache/iceberg/pull/3312)] * **File Formats** - * ORC now supports writing delete file [[\#3248](https://github.com/apache/iceberg/pull/3248)] [[\#3250](https://github.com/apache/iceberg/pull/3250)] [[\#3366](https://github.com/apache/iceberg/pull/3366)] + * ORC now supports writing delete file [[\#3248](https://github.com/apache/iceberg/pull/3248)] [[\#3250](https://github.com/apache/iceberg/pull/3250)] [[\#3366](https://github.com/apache/iceberg/pull/3366)] **Important bug fixes:** * **Core** - * Iceberg new data file root path is configured through `write.data.path` going forward. `write.folder-storage.path` and `write.object-storage.path` are deprecated [[\#3094](https://github.com/apache/iceberg/pull/3094)] - * Catalog commit status is `UNKNOWN` instead of `FAILURE` when new metadata location cannot be found in snapshot history [[\#3717](https://github.com/apache/iceberg/pull/3717)] - * Dropping table now also deletes old metadata files instead of leaving them strained [[\#3622](https://github.com/apache/iceberg/pull/3622)] - * `history` and `snapshots` metadata tables can now query tables with no current snapshot instead of returning empty [[\#3812](https://github.com/apache/iceberg/pull/3812)] + * Iceberg new data file root path is configured through `write.data.path` going forward. `write.folder-storage.path` and `write.object-storage.path` are deprecated [[\#3094](https://github.com/apache/iceberg/pull/3094)] + * Catalog commit status is `UNKNOWN` instead of `FAILURE` when new metadata location cannot be found in snapshot history [[\#3717](https://github.com/apache/iceberg/pull/3717)] + * Dropping table now also deletes old metadata files instead of leaving them strained [[\#3622](https://github.com/apache/iceberg/pull/3622)] + * `history` and `snapshots` metadata tables can now query tables with no current snapshot instead of returning empty [[\#3812](https://github.com/apache/iceberg/pull/3812)] * **Vendor Integrations** - * Using cloud service integrations such as AWS `GlueCatalog` and `S3FileIO` no longer fail when missing Hadoop dependencies in the execution environment [[\#3590](https://github.com/apache/iceberg/pull/3590)] - * AWS clients are now auto-closed when related `FileIO` or `Catalog` is closed. There is no need to close the AWS clients separately [[\#2878](https://github.com/apache/iceberg/pull/2878)] + * Using cloud service integrations such as AWS `GlueCatalog` and `S3FileIO` no longer fail when missing Hadoop dependencies in the execution environment [[\#3590](https://github.com/apache/iceberg/pull/3590)] + * AWS clients are now auto-closed when related `FileIO` or `Catalog` is closed. There is no need to close the AWS clients separately [[\#2878](https://github.com/apache/iceberg/pull/2878)] * **Spark** - * For Spark >= 3.1, `REFRESH TABLE` can now be used with Spark session catalog instead of throwing exception [[\#3072](https://github.com/apache/iceberg/pull/3072)] - * Insert overwrite mode now skips partition with 0 record instead of failing the write operation [[\#2895](https://github.com/apache/iceberg/issues/2895)] - * Spark snapshot expiration action now supports custom `FileIO` instead of just `HadoopFileIO` [[\#3089](https://github.com/apache/iceberg/pull/3089)] - * `REPLACE TABLE AS SELECT` can now work with tables with columns that have changed partition transform. Each old partition field of the same column is converted to a void transform with a different name [[\#3421](https://github.com/apache/iceberg/issues/3421)] - * Spark SQL filters containing binary or fixed literals can now be pushed down instead of throwing exception [[\#3728](https://github.com/apache/iceberg/pull/3728)] + * For Spark >= 3.1, `REFRESH TABLE` can now be used with Spark session catalog instead of throwing exception [[\#3072](https://github.com/apache/iceberg/pull/3072)] + * Insert overwrite mode now skips partition with 0 record instead of failing the write operation [[\#2895](https://github.com/apache/iceberg/issues/2895)] + * Spark snapshot expiration action now supports custom `FileIO` instead of just `HadoopFileIO` [[\#3089](https://github.com/apache/iceberg/pull/3089)] + * `REPLACE TABLE AS SELECT` can now work with tables with columns that have changed partition transform. Each old partition field of the same column is converted to a void transform with a different name [[\#3421](https://github.com/apache/iceberg/issues/3421)] + * Spark SQL filters containing binary or fixed literals can now be pushed down instead of throwing exception [[\#3728](https://github.com/apache/iceberg/pull/3728)] * **Flink** - * A `ValidationException` will be thrown if a user configures both `catalog-type` and `catalog-impl`. Previously it chose to use `catalog-type`. The new behavior brings Flink consistent with Spark and Hive [[\#3308](https://github.com/apache/iceberg/issues/3308)] - * Changelog tables can now be queried without `RowData` serialization issues [[\#3240](https://github.com/apache/iceberg/pull/3240)] - * `java.sql.Time` data type can now be written without data overflow problem [[\#3740](https://github.com/apache/iceberg/pull/3740)] - * Avro position delete files can now be read without encountering `NullPointerException` [[\#3540](https://github.com/apache/iceberg/pull/3540)] + * A `ValidationException` will be thrown if a user configures both `catalog-type` and `catalog-impl`. Previously it chose to use `catalog-type`. The new behavior brings Flink consistent with Spark and Hive [[\#3308](https://github.com/apache/iceberg/issues/3308)] + * Changelog tables can now be queried without `RowData` serialization issues [[\#3240](https://github.com/apache/iceberg/pull/3240)] + * `java.sql.Time` data type can now be written without data overflow problem [[\#3740](https://github.com/apache/iceberg/pull/3740)] + * Avro position delete files can now be read without encountering `NullPointerException` [[\#3540](https://github.com/apache/iceberg/pull/3540)] * **Hive** - * Hive catalog can now be initialized with a `null` Hadoop configuration instead of throwing exception [[\#3252](https://github.com/apache/iceberg/pull/3252)] - * Table creation can now succeed instead of throwing exception when some columns do not have comments [[\#3531](https://github.com/apache/iceberg/pull/3531)] + * Hive catalog can now be initialized with a `null` Hadoop configuration instead of throwing exception [[\#3252](https://github.com/apache/iceberg/pull/3252)] + * Table creation can now succeed instead of throwing exception when some columns do not have comments [[\#3531](https://github.com/apache/iceberg/pull/3531)] * **File Formats** - * Parquet file writing issue is fixed for string data with over 16 unparseable chars (e.g. high/low surrogates) [[\#3760](https://github.com/apache/iceberg/pull/3760)] - * ORC vectorized read is now configured using `read.orc.vectorization.batch-size` instead of `read.parquet.vectorization.batch-size` [[\#3133](https://github.com/apache/iceberg/pull/3133)] + * Parquet file writing issue is fixed for string data with over 16 unparseable chars (e.g. high/low surrogates) [[\#3760](https://github.com/apache/iceberg/pull/3760)] + * ORC vectorized read is now configured using `read.orc.vectorization.batch-size` instead of `read.parquet.vectorization.batch-size` [[\#3133](https://github.com/apache/iceberg/pull/3133)] **Other notable changes:** From ea1ce01797630c48c15e700033fc8387dc1ce321 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 23 Feb 2024 01:03:03 +0800 Subject: [PATCH 0066/1019] Infra: Don't run Delta Conversion CI on changes to site folder (#9780) --- .github/workflows/delta-conversion-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index d30d60bf6db0..c89fbb6060af 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -51,6 +51,7 @@ on: - 'flink/**' - 'pig/**' - 'docs/**' + - 'site/**' - 'open-api/**' - 'format/**' - '.gitattributes' From b78016dd21391dbb5601372dc9e4c3a0148867dc Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Sat, 24 Feb 2024 00:17:23 +0800 Subject: [PATCH 0067/1019] Docs: Sync specs to site via symlinks (#9779) --- format/spec.md | 4 +- site/docs/gcm-stream-spec.md | 86 +-- site/docs/puffin-spec.md | 144 +--- site/docs/spec.md | 1294 +--------------------------------- site/docs/view-spec.md | 328 +-------- site/nav.yml | 1 + 6 files changed, 7 insertions(+), 1850 deletions(-) mode change 100644 => 120000 site/docs/gcm-stream-spec.md mode change 100644 => 120000 site/docs/puffin-spec.md mode change 100644 => 120000 site/docs/spec.md mode change 100644 => 120000 site/docs/view-spec.md diff --git a/format/spec.md b/format/spec.md index bc655c49dc57..efc21e904b84 100644 --- a/format/spec.md +++ b/format/spec.md @@ -787,7 +787,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 }}/index.html?org/apache/iceberg/hadoop/HadoopTableOperations.html). #### Metastore Tables @@ -803,7 +803,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 }}/index.html?org/apache/iceberg/BaseMetastoreTableOperations.html). ### Delete Formats diff --git a/site/docs/gcm-stream-spec.md b/site/docs/gcm-stream-spec.md deleted file mode 100644 index d2074e02ab6d..000000000000 --- a/site/docs/gcm-stream-spec.md +++ /dev/null @@ -1,85 +0,0 @@ ---- -title: "AES GCM Stream Spec" ---- - - -# AES GCM Stream file format extension - -## Background and Motivation - -Iceberg supports a number of data file formats. Two of these formats (Parquet and ORC) have built-in encryption capabilities, that allow to protect sensitive information in the data files. However, besides the data files, Iceberg tables also have metadata files, that keep sensitive information too (e.g., min/max values in manifest files, or bloom filter bitsets in puffin files). Metadata file formats (AVRO, JSON, Puffin) don't have encryption support. - -Moreover, with the exception of Parquet, no Iceberg data or metadata file format supports integrity verification, required for end-to-end tamper proofing of Iceberg tables. - -This document specifies details of a simple file format extension that adds encryption and tamper-proofing to any existing file format. - -## Goals - -* Metadata encryption: enable encryption of manifests, manifest lists, snapshots and stats. -* Avro data encryption: enable encryption of data files in tables that use the Avro format. -* Support read splitting: enable seekable decrypting streams that can be used with splittable formats like Avro. -* Tamper proofing of Iceberg data and metadata files. - -## Overview - -The output stream, produced by a metadata or data writer, is split into equal-size blocks (plus last block that can be shorter). Each block is enciphered (encrypted/signed) with a given encryption key, and stored in a file in the AES GCM Stream format. Upon reading, the stored cipherblocks are verified for integrity; then decrypted and passed to metadata or data readers. - -## Encryption algorithm - -AES GCM Stream uses the standard AEG GCM cipher, and supports all AES key sizes: 128, 192 and 256 bits. - -AES GCM is an authenticated encryption. Besides data confidentiality (encryption), it supports two levels of integrity verification (authentication): of the data (default), and of the data combined with an optional AAD (“additional authenticated data”). An AAD is a free text to be authenticated, together with the data. The structure of AES GCM Stream AADs is described below. - -AES GCM requires a unique vector to be provided for each encrypted block. In this document, the unique input to GCM encryption is called nonce (“number used once”). AES GCM Stream encryption uses the RBG-based (random bit generator) nonce construction as defined in the section 8.2.2 of the NIST SP 800-38D document. For each encrypted block, AES GCM Stream generates a unique nonce with a length of 12 bytes (96 bits). - -## Format specification - -### File structure - -The AES GCM Stream files have the following structure - -``` -Magic BlockLength CipherBlock₁ CipherBlock₂ ... CipherBlockₙ -``` - -where - -- `Magic` is four bytes 0x41, 0x47, 0x53, 0x31 ("AGS1", short for: AES GCM Stream, version 1) -- `BlockLength` is four bytes (little endian) integer keeping the length of the equal-size split blocks before encryption. The length is specified in bytes. -- `CipherBlockᵢ` is the i-th enciphered block in the file, with the structure defined below. - -### Cipher Block structure - -Cipher blocks have the following structure - -| nonce | ciphertext | tag | -|-------|------------|-----| - -where - -- `nonce` is the AES GCM nonce, with a length of 12 bytes. -- `ciphertext` is the encrypted block. Its length is identical to the length of the block before encryption ("plaintext"). The length of all plaintext blocks, except the last, is `BlockLength` bytes. The last block has a non-zero length <= `BlockLength`. -- `tag` is the AES GCM tag, with a length of 16 bytes. - -AES GCM Stream encrypts all blocks by the GCM cipher, without padding. The AES GCM cipher must be implemented by a cryptographic provider according to the NIST SP 800-38D specification. In AES GCM Stream, an input to the GCM cipher is an AES encryption key, a nonce, a plaintext and an AAD (described below). The output is a ciphertext with the length equal to that of plaintext, and a 16-byte authentication tag used to verify the ciphertext and AAD integrity. - -### Additional Authenticated Data - -The AES GCM cipher protects against byte replacement inside a ciphertext block - but, without an AAD, it can't prevent replacement of one ciphertext block with another (encrypted with the same key). AES GCM Stream leverages AADs to protect against swapping ciphertext blocks inside a file or between files. AES GCM Stream can also protect against swapping full files - for example, replacement of a metadata file with an old version. AADs are built to reflects the identity of a file and of the blocks inside the file. - -AES GCM Stream constructs a block AAD from two components: an AAD prefix - a string provided by Iceberg for the file (with the file ID), and an AAD suffix - the block sequence number in the file, as an int in a 4-byte little-endian form. The block AAD is a direct concatenation of the prefix and suffix parts. diff --git a/site/docs/gcm-stream-spec.md b/site/docs/gcm-stream-spec.md new file mode 120000 index 000000000000..1d4b317de8eb --- /dev/null +++ b/site/docs/gcm-stream-spec.md @@ -0,0 +1 @@ +../../format/gcm-stream-spec.md \ No newline at end of file diff --git a/site/docs/puffin-spec.md b/site/docs/puffin-spec.md deleted file mode 100644 index 1fe9e5b9bc56..000000000000 --- a/site/docs/puffin-spec.md +++ /dev/null @@ -1,143 +0,0 @@ ---- -title: "Puffin Spec" ---- - - -# Puffin file format - -This is a specification for Puffin, a file format designed to store -information such as indexes and statistics about data managed in an -Iceberg table that cannot be stored directly within the Iceberg manifest. A -Puffin file contains arbitrary pieces of information (here called "blobs"), -along with metadata necessary to interpret them. The blobs supported by Iceberg -are documented at [Blob types](#blob-types). - -## Format specification - -A file conforming to the Puffin file format specification should have the structure -as described below. - -### Versions - -Currently, there is a single version of the Puffin file format, described below. - -### File structure - -The Puffin file has the following structure - -``` -Magic Blob₁ Blob₂ ... Blobₙ Footer -``` - -where - -- `Magic` is four bytes 0x50, 0x46, 0x41, 0x31 (short for: Puffin _Fratercula - arctica_, version 1), -- `Blobᵢ` is i-th blob contained in the file, to be interpreted by application - according to the footer, -- `Footer` is defined below. - -### Footer structure - -Footer has the following structure - -``` -Magic FooterPayload FooterPayloadSize Flags Magic -``` - -where - -- `Magic`: four bytes, same as at the beginning of the file -- `FooterPayload`: optionally compressed, UTF-8 encoded JSON payload describing the - blobs in the file, with the structure described below -- `FooterPayloadSize`: a length in bytes of the `FooterPayload` (after compression, - if compressed), stored as 4 byte integer -- `Flags`: 4 bytes for boolean flags - - byte 0 (first) - - bit 0 (lowest bit): whether `FooterPayload` is compressed - - all other bits are reserved for future use and should be set to 0 on write - - all other bytes are reserved for future use and should be set to 0 on write - -A 4 byte integer is always signed, in a two's complement representation, stored -little-endian. - -### Footer Payload - -Footer payload bytes is either uncompressed or LZ4-compressed (as a single -[LZ4 compression frame](https://github.com/lz4/lz4/blob/77d1b93f72628af7bbde0243b4bba9205c3138d9/doc/lz4_Frame_format.md) -with content size present), UTF-8 encoded JSON payload representing a single -`FileMetadata` object. - -#### FileMetadata - -`FileMetadata` has the following fields - - -| Field Name | Field Type | Required | Description | -| ---------- | --------------------------------------- | -------- | ----------- | -| blobs | list of BlobMetadata objects | yes | -| properties | JSON object with string property values | no | storage for arbitrary meta-information, like writer identification/version. See [Common properties](#common-properties) for properties that are recommended to be set by a writer. - -#### BlobMetadata - -`BlobMetadata` has the following fields - -| Field Name | Field Type | Required | Description | -|-------------------|-----------------------------------------|----------| ----------- | -| type | JSON string | yes | See [Blob types](#blob-types) -| fields | JSON list of ints | yes | List of field IDs the blob was computed for; the order of items is used to compute sketches stored in the blob. -| snapshot-id | JSON long | yes | ID of the Iceberg table's snapshot the blob was computed from. -| sequence-number | JSON long | yes | Sequence number of the Iceberg table's snapshot the blob was computed from. -| offset | JSON long | yes | The offset in the file where the blob contents start -| length | JSON long | yes | The length of the blob stored in the file (after compression, if compressed) -| compression-codec | JSON string | no | See [Compression codecs](#compression-codecs). If omitted, the data is assumed to be uncompressed. -| properties | JSON object with string property values | no | storage for arbitrary meta-information about the blob - -### Blob types - -The blobs can be of a type listed below - -#### `apache-datasketches-theta-v1` blob type - -A serialized form of a "compact" Theta sketch produced by the [Apache -DataSketches](https://datasketches.apache.org/) library. The sketch is obtained by -constructing Alpha family sketch with default seed, and feeding it with individual -distinct values converted to bytes using Iceberg's single-value serialization. - -The blob metadata for this blob may include following properties: - -- `ndv`: estimate of number of distinct values, derived from the sketch. - -### Compression codecs - -The data can also be uncompressed. If it is compressed the codec should be one of -codecs listed below. For maximal interoperability, other codecs are not supported. - -| Codec name | Description | -|------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| -| lz4 | Single [LZ4 compression frame](https://github.com/lz4/lz4/blob/77d1b93f72628af7bbde0243b4bba9205c3138d9/doc/lz4_Frame_format.md), with content size present | -| zstd | Single [Zstandard compression frame](https://github.com/facebook/zstd/blob/8af64f41161f6c2e0ba842006fe238c664a6a437/doc/zstd_compression_format.md#zstandard-frames), with content size present | -__ - -### Common properties - -When writing a Puffin file it is recommended to set the following fields in the -[FileMetadata](#filemetadata)'s `properties` field. - -- `created-by` - human-readable identification of the application writing the file, - along with its version. Example "Trino version 381". diff --git a/site/docs/puffin-spec.md b/site/docs/puffin-spec.md new file mode 120000 index 000000000000..370457769aa3 --- /dev/null +++ b/site/docs/puffin-spec.md @@ -0,0 +1 @@ +../../format/puffin-spec.md \ No newline at end of file diff --git a/site/docs/spec.md b/site/docs/spec.md deleted file mode 100644 index 2ff625b76581..000000000000 --- a/site/docs/spec.md +++ /dev/null @@ -1,1293 +0,0 @@ ---- -title: "Spec" ---- - - -# Iceberg Table Spec - -This is a specification for the Iceberg table format that is designed to manage a large, slow-changing collection of files in a distributed file system or key-value store as a table. - -## Format Versioning - -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 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 of the Iceberg spec adds row-level updates and deletes for analytic tables with immutable files. - -The primary change in version 2 adds delete files to encode rows that are deleted in existing data files. This version can be used to delete or replace individual rows in immutable data files without rewriting the files. - -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). - - -## Goals - -* **Serializable isolation** -- Reads will be isolated from concurrent writes and always use a committed snapshot of a table’s data. Writes will support removing and adding files in a single operation and are never partially visible. Readers will not acquire locks. -* **Speed** -- Operations will use O(1) remote calls to plan the files for a scan and not O(n) where n grows with the size of the table, like the number of partitions or files. -* **Scale** -- Job planning will be handled primarily by clients and not bottleneck on a central metadata store. Metadata will include information needed for cost-based optimization. -* **Evolution** -- Tables will support full schema and partition spec evolution. Schema evolution supports safe column add, drop, reorder and rename, including in nested structures. -* **Dependable types** -- Tables will provide well-defined and dependable support for a core set of types. -* **Storage separation** -- Partitioning will be table configuration. Reads will be planned using predicates on data values, not partition values. Tables will support evolving partition schemes. -* **Formats** -- Underlying data file formats will support identical schema evolution rules and types. Both read-optimized and write-optimized formats will be available. - -## Overview - -![Iceberg snapshot structure](assets/images/iceberg-metadata.png) - -This table format tracks individual data files in a table instead of directories. This allows writers to create data files in-place and only adds files to the table in an explicit commit. - -Table state is maintained in metadata files. All changes to table state create a new metadata file and replace the old metadata with an atomic swap. The table metadata file tracks the table schema, partitioning config, custom properties, and snapshots of the table contents. A snapshot represents the state of a table at some time and is used to access the complete set of data files in the table. - -Data files in snapshots are tracked by one or more manifest files that contain a row for each data file in the table, the file's partition data, and its metrics. The data in a snapshot is the union of all files in its manifests. Manifest files are reused across snapshots to avoid rewriting metadata that is slow-changing. Manifests can track data files with any subset of a table and are not associated with partitions. - -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 - -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. - -Writers create table metadata files optimistically, assuming that the current version will not be changed before the writer's commit. Once a writer has created an update, it commits by swapping the table’s metadata file pointer from the base version to the new version. - -If the snapshot on which an update is based is no longer current, the writer must retry the update based on the new current version. Some operations support retry by re-applying metadata changes and committing, under well-defined conditions. For example, a change that rewrites files can be applied to a new table snapshot if all of the rewritten files are still in the table. - -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 - -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. - -All manifests, data files, and delete files created for a snapshot inherit the snapshot's sequence number. Manifest file metadata in the manifest list stores a manifest's sequence number. New data and metadata file entries are written with `null` in place of a sequence number, which is replaced with the manifest's sequence number at read time. When a data or delete file is written to a new manifest (as "existing"), the inherited sequence number is written to ensure it does not change after it is first inherited. - -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 are stored in delete files. - -There are two ways to encode a row-level delete: - -* [_Position deletes_](#position-delete-files) mark a row deleted by data file path and the row position in the data file -* [_Equality deletes_](#equality-delete-files) mark a row deleted by one or more column values, like `id = 5` - -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 - -Iceberg only requires that file systems support the following operations: - -* **In-place write** -- Files are not moved or altered once they are written. -* **Seekable reads** -- Data file formats require seek support. -* **Deletes** -- Tables delete files that are no longer used. - -These requirements are compatible with object stores, like S3. - -Tables do not require random-access writes. Once written, data and metadata files are immutable until they are deleted. - -Tables do not require rename, except for tables that use atomic rename to implement the commit operation for new metadata files. - - -## Specification - -#### Terms - -* **Schema** -- Names and types of fields in a table. -* **Partition spec** -- A definition of how partition values are derived from data fields. -* **Snapshot** -- The state of a table at some point in time, including the set of all data files. -* **Manifest list** -- A file that lists manifest files; one per snapshot. -* **Manifest** -- A file that lists data or delete files; a subset of a snapshot. -* **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 - -Some tables in this spec have columns that specify requirements for v1 and v2 tables. These requirements are intended for writers when adding metadata files to a table with the given version. - -| Requirement | Write behavior | -|-------------|----------------| -| (blank) | The field should be omitted | -| _optional_ | The field can be written | -| _required_ | The field must be written | - -Readers should be more permissive because v1 metadata files are allowed in v2 tables so that tables can be upgraded to v2 without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected v2 read behavior: - -| v1 | v2 | v2 read behavior | -|------------|------------|------------------| -| | _optional_ | Read the field as _optional_ | -| | _required_ | Read the field as _optional_; it may be missing in v1 files | -| _optional_ | | Ignore the field | -| _optional_ | _optional_ | Read the field as _optional_ | -| _optional_ | _required_ | Read the field as _optional_; it may be missing in v1 files | -| _required_ | | Ignore the field | -| _required_ | _optional_ | Read the field as _optional_ | -| _required_ | _required_ | Fill in a default or throw an exception if the field is missing | - -Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. - -### 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 - -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). - -A **`list`** is a collection of values with some element type. The element field has an integer id that is unique in the table schema. Elements can be either optional or required. Element types may be any type. - -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 type | Description | Requirements | -|--------------------|--------------------------------------------------------------------------|--------------------------------------------------| -| **`boolean`** | True or false | | -| **`int`** | 32-bit signed integers | Can promote to `long` | -| **`long`** | 64-bit signed integers | | -| **`float`** | [32-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | Can promote to double | -| **`double`** | [64-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | | -| **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed [1], precision must be 38 or less | -| **`date`** | Calendar date without timezone or time | | -| **`time`** | Time of day without date, timezone | Microsecond precision [2] | -| **`timestamp`** | Timestamp without timezone | Microsecond precision [2] | -| **`timestamptz`** | Timestamp with timezone | Stored as UTC [2] | -| **`string`** | Arbitrary-length character sequences | Encoded with UTF-8 [3] | -| **`uuid`** | Universally unique identifiers | Should use 16-byte fixed | -| **`fixed(L)`** | Fixed-length byte array of length L | | -| **`binary`** | Arbitrary-length byte array | | - -Notes: - -1. Decimal scale is fixed and cannot be changed by schema evolution. Precision can only be widened. -2. All time and timestamp values are stored with microsecond precision. - - Timestamps _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). - - Timestamps _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). Timestamp values are stored as a long that encodes microseconds from the unix epoch. -3. Character strings must be stored as UTF-8 encoded byte arrays. - -For details on how to serialize a schema to JSON, see Appendix C. - - -#### 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: -- `initial-default` is used to populate the field's value for all records that were written before the field was added to the schema -- `write-default` is used to populate the field's value for any records written after the field was added to the schema, if the writer does not supply the field's value - -The `initial-default` is set only when a field is added to an existing schema. The `write-default` is initially set to the same value as `initial-default` and can be changed through schema evolution. If either default is not set for an optional field, then the default value is null for compatibility with older spec versions. - -The `initial-default` and `write-default` produce SQL default value behavior, without rewriting data files. SQL default value behavior when a field is added handles all existing rows as though the rows were written with the new field's default value. Default value changes may only affect future records and all known fields are written into data files. Omitting a known field when writing a data file is never allowed. The write default for a field must be written if a field is not supplied to a write. If the write default for a required field is not set, the writer must fail. - -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 - -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). - -Evolution applies changes to the table's current schema to produce a new schema that is identified by a unique schema ID, is added to the table's list of schemas, and is set as the table's current schema. - -Valid type promotions are: - -* `int` to `long` -* `float` to `double` -* `decimal(P, S)` to `decimal(P', S)` if `P' > P` -- widen the precision of decimal types. - -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. - -Grouping a subset of a struct’s fields into a nested struct is **not** allowed, nor is moving fields from a nested struct into its immediate parent struct (`struct ↔ struct>`). Evolving primitive types to structs is **not** allowed, nor is evolving a single-field struct to a primitive (`map ↔ map>`). - -Struct evolution requires the following rules for default values: - -* The `initial-default` must be set when a field is added and cannot change -* The `write-default` must be set when a field is added and may change -* When a required field is added, both defaults must be set to a non-null value -* When an optional field is added, the defaults may be null and should be explicitly set -* When a new field is added to a struct with a default value, updating the struct's default is optional -* If a field value is missing from a struct's `initial-default`, the field's `initial-default` must be used for the field -* 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 - -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. If a field id is missing from a data file, its value for each row should be `null`. - -For example, a file may be written with schema `1: a int, 2: b string, 3: c double` and read using projection schema `3: measurement, 2: name, 4: a`. This must select file columns `c` (renamed to `measurement`), `b` (now called `name`), and a column of `null` values called `a`; in that order. - -Tables may also define a property `schema.name-mapping.default` with a JSON name mapping containing a list of field mapping objects. These mappings provide fallback field ids to be used when a data file does not contain field id information. Each object should contain - -* `names`: A required list of 0 or more names for a field. -* `field-id`: An optional Iceberg field ID used when a field's name is present in `names` -* `fields`: An optional list of field mappings for child field of structs, maps, and lists. - -Field mapping fields are constrained by the following rules: - -* A name may contain `.` but this refers to a literal name, not a nested field. For example, `a.b` refers to a field named `a.b`, not child field `b` of field `a`. -* Each child field should be defined with their own field mapping under `fields`. -* Multiple values for `names` may be mapped to a single field ID to support cases where a field may have different names in different data files. For example, all Avro field aliases should be listed in `names`. -* Fields which exist only in the Iceberg schema and not in imported data files may use an empty `names` list. -* Fields that exist in imported files but not in the Iceberg schema may omit `field-id`. -* List types should contain a mapping in `fields` for `element`. -* Map types should contain mappings in `fields` for `key` and `value`. -* Struct types should contain mappings in `fields` for their child fields. - -For details on serialization, see [Appendix C](#name-mapping-serialization). - -#### 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). - -Two rows are the "same"---that is, the rows represent the same entity---if the identifier fields are equal. However, uniqueness of rows by this identifier is not guaranteed or required by Iceberg and it is the responsibility of processing engines or data providers to enforce. - -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 - -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. - -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 | - - -### 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.) - -Tables are configured with a **partition spec** that defines how to produce a tuple of partition values from a record. A partition spec has a list of fields that consist of: - -* A **source column id** from the table’s schema -* A **partition field id** that is used to identify a partition field and is unique within a partition spec. In v2 table metadata, it is unique across all partition specs. -* A **transform** that is applied to the source column to produce a partition value -* A **partition name** - -The source column, selected by id, must be a primitive type and cannot be contained in a map or list, but may be nested in a struct. For details on how to serialize a partition spec to JSON, see Appendix C. - -Partition specs capture the transform from table data to partition values. This is used to transform predicates to partition predicates, in addition to transforming data values. Deriving partition predicates from column predicates on the table data is used to separate the logical queries from physical storage: the partitioning can change and the correct partition filters are always derived from column predicates. This simplifies queries because users don’t have to supply both logical predicates and partition predicates. For more information, see Scan Planning below. - - -#### Partition Transforms - -| Transform name | Description | Source types | Result type | -|-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------| -| **`identity`** | Source value, unmodified | Any | Source type | -| **`bucket[N]`** | Hash of value, mod `N` (see below) | `int`, `long`, `decimal`, `date`, `time`, `timestamp`, `timestamptz`, `string`, `uuid`, `fixed`, `binary` | `int` | -| **`truncate[W]`** | Value truncated to width `W` (see below) | `int`, `long`, `decimal`, `string` | Source type | -| **`year`** | Extract a date or timestamp year, as years from 1970 | `date`, `timestamp`, `timestamptz` | `int` | -| **`month`** | Extract a date or timestamp month, as months from 1970-01-01 | `date`, `timestamp`, `timestamptz` | `int` | -| **`day`** | Extract a date or timestamp day, as days from 1970-01-01 | `date`, `timestamp`, `timestamptz` | `int` | -| **`hour`** | Extract a timestamp hour, as hours from 1970-01-01 00:00:00 | `timestamp`, `timestamptz` | `int` | -| **`void`** | Always produces `null` | Any | Source type or `int` | - -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 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. - -Transforms are parameterized by a number of buckets [1], `N`. The hash mod `N` must produce a positive value by first discarding the sign bit of the hash value. In pseudo-code, the function is: - -``` - def bucket_N(x) = (murmur3_x86_32_hash(x) & Integer.MAX_VALUE) % N -``` - -Notes: - -1. Changing the number of buckets as a table grows is possible by evolving the partition spec. - -For hash function details by type, see Appendix B. - - -#### Truncate Transform Details - -| **Type** | **Config** | **Truncate specification** | **Examples** | -|---------------|-----------------------|------------------------------------------------------------------|----------------------------------| -| **`int`** | `W`, width | `v - (v % W)` remainders must be positive [1] | `W=10`: `1` → `0`, `-1` → `-10` | -| **`long`** | `W`, width | `v - (v % W)` remainders must be positive [1] | `W=10`: `1` → `0`, `-1` → `-10` | -| **`decimal`** | `W`, width (no scale) | `scaled_W = decimal(W, scale(v))` `v - (v % scaled_W)` [1, 2] | `W=50`, `s=2`: `10.65` → `10.50` | -| **`string`** | `L`, length | Substring of length `L`: `v.substring(0, L)` [3] | `L=3`: `iceberg` → `ice` | - -Notes: - -1. The remainder, `v % W`, must be positive. For languages where `%` can produce negative values, the correct truncate function is: `v - (((v % W) + W) % W)` -2. The width, `W`, used to truncate decimal values is applied using the scale of the decimal column to avoid additional (and potentially conflicting) parameters. -3. Strings are truncated to a valid UTF-8 string with no more than `L` code points. - - -#### Partition Evolution - -Table partitioning can be evolved by adding, removing, renaming, or reordering partition spec fields. - -Changing a partition spec produces a new spec identified by a unique spec ID that is added to the table's list of partition specs and may be set as the table's default spec. - -When evolving a spec, changes should not cause partition field IDs to change because the partition field IDs are used as the partition tuple field IDs in manifest files. - -In v2, partition field IDs must be explicitly tracked for each partition field. New IDs are assigned based on the last assigned partition ID in table metadata. - -In v1, partition field IDs were not tracked, but were assigned sequentially starting at 1000 in the reference implementation. This assignment caused problems when reading metadata tables based on manifest files from multiple specs because partition fields with the same ID may contain different data types. For compatibility with old versions, the following rules are recommended for partition evolution in v1 tables: - -1. Do not reorder partition fields -2. Do not drop partition fields; instead replace the field's transform with the `void` transform -3. Only add partition fields at the end of the previous partition spec - - -### 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**. - -A sort order is defined by a sort order id and a list of sort fields. The order of the sort fields within the list defines the order in which the sort is applied to the data. Each sort field consists of: - -* A **source column id** from the table's schema -* A **transform** that is used to produce values to be sorted on from the source column. This is the same transform as described in [partition transforms](#partition-transforms). -* A **sort direction**, that can only be either `asc` or `desc` -* A **null order** that describes the order of null values when sorted. Can only be either `nulls-first` or `nulls-last` - -Order id `0` is reserved for the unsorted order. - -Sorting floating-point numbers should produce the following behavior: `-NaN` < `-Infinity` < `-value` < `-0` < `0` < `value` < `Infinity` < `NaN`. This aligns with the implementation of Java floating-point types comparisons. - -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 - -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. - -A manifest is a valid Iceberg data file: files must use valid Iceberg formats, schemas, and column projection. - -A manifest may store either data files or delete files, but not both because manifests that contain delete files are scanned first during job planning. Whether a manifest is a data manifest or a delete manifest is stored in manifest metadata. - -A manifest stores files for a single partition spec. When a table’s partition spec changes, old files remain in the older manifest and newer files are written to a new manifest. This is required because a manifest file’s schema is based on its partition spec (see below). The partition spec of each manifest is also used to transform predicates on the table's data rows into predicates on partition values that are used during job planning to select files from a manifest. - -A manifest file must store the partition spec and other metadata as properties in the Avro file's key-value metadata: - -| v1 | v2 | Key | Value | -|------------|------------|---------------------|------------------------------------------------------------------------------| -| _required_ | _required_ | `schema` | JSON representation of the table schema at the time the manifest was written | -| _optional_ | _required_ | `schema-id` | ID of the schema used to write the manifest as a string | -| _required_ | _required_ | `partition-spec` | JSON fields representation of the partition spec used to write the manifest | -| _optional_ | _required_ | `partition-spec-id` | ID of the partition spec used to write the manifest as a string | -| _optional_ | _required_ | `format-version` | Table format version number of the manifest as a string | -| | _required_ | `content` | Type of content files tracked by the manifest: "data" or "deletes" | - -The schema of a manifest file is a struct called `manifest_entry` with the following fields: - -| v1 | v2 | Field id, name | Type | Description | -| ---------- | ---------- |-------------------------------|-----------------------------------------------------------|-------------| -| _required_ | _required_ | **`0 status`** | `int` with meaning: `0: EXISTING` `1: ADDED` `2: DELETED` | Used to track additions and deletions. Deletes are informational only and not used in scans. | -| _required_ | _optional_ | **`1 snapshot_id`** | `long` | Snapshot id where the file was added, or deleted if status is 2. Inherited when null. | -| | _optional_ | **`3 sequence_number`** | `long` | Data sequence number of the file. Inherited when null and status is 1 (added). | -| | _optional_ | **`4 file_sequence_number`** | `long` | File sequence number indicating when the file was added. Inherited when null and status is 1 (added). | -| _required_ | _required_ | **`2 data_file`** | `data_file` `struct` (see below) | File path, partition tuple, metrics, ... | - -`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]. | - -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. - -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. - -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 - -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. - -When a file is added to the dataset, its manifest entry should store the snapshot ID in which the file was added and set status to 1 (added). - -When a file is replaced or deleted from the dataset, its manifest entry fields store the snapshot ID in which the file was deleted and status 2 (deleted). The file may be deleted from the file system when the snapshot in which it was deleted is garbage collected, assuming that older snapshots have also been garbage collected [1]. - -Iceberg v2 adds data and file sequence numbers to the entry and makes the snapshot ID optional. Values for these fields are inherited from manifest metadata when `null`. That is, if the field is `null` for an entry, then the entry must inherit its value from the manifest file's metadata, stored in the manifest list. -The `sequence_number` field represents the data sequence number and must never change after a file is added to the dataset. The data sequence number represents a relative age of the file content and should be used for planning which delete files apply to a data file. -The `file_sequence_number` field represents the sequence number of the snapshot that added the file and must also remain unchanged upon assigning at commit. The file sequence number can't be used for pruning delete files as the data within the file may have an older data sequence number. -The data and file sequence numbers are inherited only if the entry status is 1 (added). If the entry status is 0 (existing) or 2 (deleted), the entry must include both sequence numbers explicitly. - -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 - -Manifests track the sequence number when a data or delete file was added to the table. - -When adding a new file, its data and file sequence numbers are set to `null` because the snapshot's sequence number is not assigned until the snapshot is successfully committed. When reading, sequence numbers are inherited by replacing `null` with the manifest's sequence number from the manifest list. -It is also possible to add a new file with data that logically belongs to an older sequence number. In that case, the data sequence number must be provided explicitly and not inherited. However, the file sequence number must be always assigned when the snapshot is successfully committed. - -When writing an existing file to a new manifest or marking an existing file as deleted, the data and file sequence numbers must be non-null and set to the original values that were either inherited or provided at the commit time. - -Inheriting sequence numbers through the metadata tree allows writing a new manifest without a known sequence number, so that a manifest can be written once and reused in commit retries. To change a sequence number for a retry, only the manifest list must be rewritten. - -When reading v1 manifests with no sequence number column, sequence numbers for all files must default to 0. - - -### 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 | - -The snapshot summary's `operation` field is used by some operations, like snapshot expiration, to skip processing certain snapshots. Possible `operation` values are: - -* `append` -- Only data files were added and no files were removed. -* `replace` -- Data and delete files were added and removed without changing table data; i.e., compaction, changing the data file format, or relocating data files. -* `overwrite` -- Data and delete files were added and removed in a logical overwrite operation. -* `delete` -- Data files were removed and their contents logically deleted and/or delete files were added to delete rows. - -Data and delete files for a snapshot can be stored in more than one manifest. This enables: - -* Appends can add a new manifest to minimize the amount of data written, instead of adding new records by rewriting and appending to an existing manifest. (This is called a “fast append”.) -* Tables can use multiple partition specs. A table’s partition configuration can evolve if, for example, its data volume changes. Each manifest uses a single partition spec, and queries do not need to change because partition filters are derived from data predicates. -* Large tables can be split across multiple manifests so that implementations can parallelize job planning or reduce the cost of rewriting a manifest. - -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. - - -#### Manifest Lists - -Snapshots are embedded in table metadata, but the list of manifests for a snapshot are stored in a separate manifest list file. - -A new manifest list is written for each attempt to commit a snapshot because the list of manifests always changes to produce a new snapshot. When a manifest list is written, the (optimistic) sequence number of the snapshot is written for all new manifest files tracked by the list. - -A manifest list includes summary metadata that can be used to avoid scanning all of the manifests in a snapshot when planning a table scan. This includes the number of added, existing, and deleted files, and a summary of values for each field of the partition spec used to write the manifest. - -A manifest list is a valid Iceberg data file: files must use valid Iceberg formats, schemas, and column projection. - -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 | - -`field_summary` is a struct with the following fields: - -| v1 | v2 | Field id, name | Type | Description | -| ---------- | ---------- |-------------------------|---------------|-------------| -| _required_ | _required_ | **`509 contains_null`** | `boolean` | Whether the manifest contains at least one partition with a null value for the field | -| _optional_ | _optional_ | **`518 contains_nan`** | `boolean` | Whether the manifest contains at least one partition with a NaN value for the field | -| _optional_ | _optional_ | **`510 lower_bound`** | `bytes` [1] | Lower bound for the non-null, non-NaN values in the partition field, or null if all values are null or NaN [2] | -| _optional_ | _optional_ | **`511 upper_bound`** | `bytes` [1] | Upper bound for the non-null, non-NaN values in the partition field, or null if all values are null or NaN [2] | - -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. - -#### 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. - -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. - -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. - -For example, an `events` table with a timestamp column named `ts` that is partitioned by `ts_day=day(ts)` is queried by users with ranges over the timestamp column: `ts > X`. The inclusive projection is `ts_day >= day(X)`, which is used to select files that may have matching rows. Note that, in most cases, timestamps just before `X` will be included in the scan because the file contains rows that match the predicate and rows that do not match the predicate. - -Scan predicates are also used to filter data and delete files using column bounds and counts that are stored by field id in manifests. The same filter logic can be used for both data and delete files because both store metrics of the rows either inserted or deleted. If metrics show that a delete file has no rows that match a scan predicate, it may be ignored just as a data file would be ignored [2]. - -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. - -* A _position_ delete file must be applied to a data file when all of the following are true: - - 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 to the delete file's partition -* 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 and partition values) is equal to the delete file's partition _or_ the delete file's partition spec is unpartitioned - -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. - - -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. - -#### Snapshot Reference - -Iceberg tables keep track of branches and tags using snapshot references. -Tags are labels for individual snapshots. Branches are mutable named references that can be updated by committing a new snapshot as the branch's referenced snapshot using the [Commit Conflict Resolution and Retry](#commit-conflict-resolution-and-retry) procedures. - -The snapshot reference object records all the information of a reference including snapshot ID, reference type and [Snapshot Retention Policy](#snapshot-retention-policy). - -| v1 | v2 | Field name | Type | Description | -| ---------- | ---------- | ---------------------------- | --------- | ----------- | -| _required_ | _required_ | **`snapshot-id`** | `long` | A reference's snapshot ID. The tagged snapshot or latest snapshot of a branch. | -| _required_ | _required_ | **`type`** | `string` | Type of the reference, `tag` or `branch` | -| _optional_ | _optional_ | **`min-snapshots-to-keep`** | `int` | For `branch` type only, a positive number for the minimum number of snapshots to keep in a branch while expiring snapshots. Defaults to table property `history.expire.min-snapshots-to-keep`. | -| _optional_ | _optional_ | **`max-snapshot-age-ms`** | `long` | For `branch` type only, a positive number for the max age of snapshots to keep when expiring, including the latest snapshot. Defaults to table property `history.expire.max-snapshot-age-ms`. | -| _optional_ | _optional_ | **`max-ref-age-ms`** | `long` | For snapshot references except the `main` branch, a positive number for the max age of the snapshot reference to keep while expiring snapshots. Defaults to table property `history.expire.max-ref-age-ms`. The `main` branch never expires. | - -Valid snapshot references are stored as the values of the `refs` map in table metadata. For serialization, see Appendix C. - -#### Snapshot Retention Policy - -Table snapshots expire and are removed from metadata to allow removed or replaced data files to be physically deleted. -The snapshot expiration procedure removes snapshots from table metadata and applies the table's retention policy. -Retention policy can be configured both globally and on snapshot reference through properties `min-snapshots-to-keep`, `max-snapshot-age-ms` and `max-ref-age-ms`. - -When expiring snapshots, retention policies in table and snapshot references are evaluated in the following way: - -1. Start with an empty set of snapshots to retain -2. Remove any refs (other than main) where the referenced snapshot is older than `max-ref-age-ms` -3. For each branch and tag, add the referenced snapshot to the retained set -4. For each branch, add its ancestors to the retained set until: - 1. The snapshot is older than `max-snapshot-age-ms`, AND - 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 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 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). | - -For serialization details, see Appendix C. - -#### 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 -many statistics files associated with different table snapshots. - -Statistics files metadata within `statistics` table metadata field is a struct with the following fields: - -| v1 | v2 | Field name | Type | Description | -|----|----|------------|------|-------------| -| _required_ | _required_ | **`snapshot-id`** | `string` | ID of the Iceberg table's snapshot the statistics file is associated with. | -| _required_ | _required_ | **`statistics-path`** | `string` | Path of the statistics file. See [Puffin file format](puffin-spec.md). | -| _required_ | _required_ | **`file-size-in-bytes`** | `long` | Size of the statistics file. | -| _required_ | _required_ | **`file-footer-size-in-bytes`** | `long` | Total size of the statistics file's footer (not the footer payload size). See [Puffin file format](puffin-spec.md) for footer definition. | -| _optional_ | _optional_ | **`key-metadata`** | Base64-encoded implementation-specific key metadata for encryption. | -| _required_ | _required_ | **`blob-metadata`** | `list` (see below) | A list of the blob metadata for statistics contained in the file with structure described below. | - -Blob metadata is a struct with the following fields: - -| v1 | v2 | Field name | Type | Description | -|----|----|------------|------|-------------| -| _required_ | _required_ | **`type`** | `string` | Type of the blob. Matches Blob type in the Puffin file. | -| _required_ | _required_ | **`snapshot-id`** | `long` | ID of the Iceberg table's snapshot the blob was computed from. | -| _required_ | _required_ | **`sequence-number`** | `long` | Sequence number of the Iceberg table's snapshot the blob was computed from. | -| _required_ | _required_ | **`fields`** | `list` | Ordered list of fields, given by field ID, on which the statistic was calculated. | -| _optional_ | _optional_ | **`properties`** | `map` | Additional properties associated with the statistic. Subset of Blob properties in the Puffin file. | - - -#### 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. - -* Append operations have no requirements and can always be applied. -* Replace operations must verify that the files that will be deleted are still in the table. Examples of replace operations include format changes (replace an Avro file with a Parquet file) and compactions (several files are replaced with a single file that contains the same rows). -* Delete operations must verify that specific files to delete are still in the table. Delete operations based on expressions can always be applied (e.g., where timestamp < X). -* 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 - -An atomic swap can be implemented using atomic rename in file systems that support it, like HDFS or most local file systems [1]. - -Each version of table metadata is stored in a metadata folder under the table’s base location using a file naming scheme that includes a version number, `V`: `v.metadata.json`. To commit a new metadata version, `V+1`, the writer performs the following steps: - -1. Read the current table metadata version `V`. -2. Create new table metadata based on version `V`. -3. Write the new table metadata to a unique file: `.metadata.json`. -4. Rename the unique file to the well-known file for version `V`: `v.metadata.json`. - 1. If the rename succeeds, the commit succeeded and `V+1` is the table’s current version - 2. If the rename fails, go back to step 1. - -Notes: - -1. The file system table scheme is implemented in [HadoopTableOperations](javadoc/latest/index.html?org/apache/iceberg/hadoop/HadoopTableOperations.html). - -#### 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. - -Each version of table metadata is stored in a metadata folder under the table’s base location using a naming scheme that includes a version and UUID: `-.metadata.json`. To commit a new metadata version, `V+1`, the writer performs the following steps: - -1. Create a new table metadata file based on the current metadata. -2. Write the new table metadata to a unique file: `-.metadata.json`. -3. Request that the metastore swap the table’s metadata pointer from the location of `V` to the location of `V+1`. - 1. If the swap succeeds, the commit succeeded. `V` was still the latest metadata version and the metadata file for `V+1` is now the current metadata. - 2. If the swap fails, another writer has already created `V+1`. The current writer goes back to step 1. - -Notes: - -1. The metastore table scheme is partly implemented in [BaseMetastoreTableOperations](javadoc/latest/index.html?org/apache/iceberg/BaseMetastoreTableOperations.html). - - -### Delete Formats - -This section details how to encode row-level deletes in Iceberg delete files. Row-level deletes are not supported in v1. - -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. - -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-based delete files identify deleted rows by file and position in one or more data files, and may optionally contain the deleted row. - -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: - -| Field id, name | Type | Description | -|-----------------------------|----------------------------|-------------| -| **`2147483546 file_path`** | `string` | Full URI of a data file with FS scheme. This must match the `file_path` of the target data file in a manifest entry | -| **`2147483545 pos`** | `long` | Ordinal position of a deleted row in the target data file identified by `file_path`, starting at `0` | -| **`2147483544 row`** | `required struct<...>` [1] | Deleted row values. Omit the column when not storing deleted rows. | - -1. When present in the delete file, `row` is required because all delete entries must include the row values. - -When the deleted row column is present, its schema may be any subset of the table schema and must use field ids matching the table. - -To ensure the accuracy of statistics, all delete entries must include row values, or the column must be omitted (this is why the column type is `required`). - -The rows in the delete file must be sorted by `file_path` then `pos` to optimize filtering rows while scanning. - -* 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 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. - -Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). Float and double columns cannot be used as delete columns in equality delete files. - -A data row is deleted if its values are equal to all delete columns for any row in an equality delete file that applies to the row's data file (see [`Scan Planning`](#scan-planning)). - -Each row of the delete file produces one equality predicate that matches any row where the delete columns are equal. Multiple columns can be thought of as an `AND` of equality predicates. A `null` value in a delete column matches a row if the row's value is `null`, equivalent to `col IS NULL`. - -For example, a table with the following data: - -```text - 1: id | 2: category | 3: name --------|-------------|--------- - 1 | marsupial | Koala - 2 | toy | Teddy - 3 | NULL | Grizzly - 4 | NULL | Polar -``` - -The delete `id = 3` could be written as either of the following equality delete files: - -```text -equality_ids=[1] - - 1: id -------- - 3 -``` - -```text -equality_ids=[1] - - 1: id | 2: category | 3: name --------|-------------|--------- - 3 | NULL | Grizzly -``` - -The delete `id = 4 AND category IS NULL` could be written as the following equality delete file: - -```text -equality_ids=[1, 2] - - 1: id | 2: category | 3: name --------|-------------|--------- - 4 | NULL | Polar -``` - -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 - -Manifests hold the same statistics for delete files and data files. For delete files, the metrics describe the values that were deleted. - - -## Appendix A: Format-specific Requirements - - -### Avro - -**Data Type Mappings** - -Values should be stored in Avro using the Avro types and logical type annotations in the table below. - -Optional fields, array elements, and map values must be wrapped in an Avro `union` with `null`. This is the only union type allowed in Iceberg data files. - -Optional fields must always set the Avro field default value to null. - -Maps with non-string keys must use an array representation with the `map` logical type. The array representation or Avro’s map type may be used for maps with string keys. - -|Type|Avro type|Notes| -|--- |--- |--- | -|**`boolean`**|`boolean`|| -|**`int`**|`int`|| -|**`long`**|`long`|| -|**`float`**|`float`|| -|**`double`**|`double`|| -|**`decimal(P,S)`**|`{ "type": "fixed",`
  `"size": minBytesRequired(P),`
  `"logicalType": "decimal",`
  `"precision": P,`
  `"scale": S }`|Stored as fixed using the minimum number of bytes for the given precision.| -|**`date`**|`{ "type": "int",`
  `"logicalType": "date" }`|Stores days from the 1970-01-01.| -|**`time`**|`{ "type": "long",`
  `"logicalType": "time-micros" }`|Stores microseconds from midnight.| -|**`timestamp`**|`{ "type": "long",`
  `"logicalType": "timestamp-micros",`
  `"adjust-to-utc": false }`|Stores microseconds from 1970-01-01 00:00:00.000000.| -|**`timestamptz`**|`{ "type": "long",`
  `"logicalType": "timestamp-micros",`
  `"adjust-to-utc": true }`|Stores microseconds from 1970-01-01 00:00:00.000000 UTC.| -|**`string`**|`string`|| -|**`uuid`**|`{ "type": "fixed",`
  `"size": 16,`
  `"logicalType": "uuid" }`|| -|**`fixed(L)`**|`{ "type": "fixed",`
  `"size": L }`|| -|**`binary`**|`bytes`|| -|**`struct`**|`record`|| -|**`list`**|`array`|| -|**`map`**|`array` of key-value records, or `map` when keys are strings (optional).|Array storage must use logical type name `map` and must store elements that are 2-field records. The first field is a non-null key and the second field is the value.| - - -**Field IDs** - -Iceberg struct, list, and map types identify nested types by ID. When writing data to Avro files, these IDs must be stored in the Avro schema to support ID-based column pruning. - -IDs are stored as JSON integers in the following locations: - -|ID|Avro schema location|Property|Example| -|--- |--- |--- |--- | -|**Struct field**|Record field object|`field-id`|`{ "type": "record", ...`
  `"fields": [`
    `{ "name": "l",`
      `"type": ["null", "long"],`
      `"default": null,`
      `"field-id": 8 }`
  `] }`| -|**List element**|Array schema object|`element-id`|`{ "type": "array",`
  `"items": "int",`
  `"element-id": 9 }`| -|**String map key**|Map schema object|`key-id`|`{ "type": "map",`
  `"values": "int",`
  `"key-id": 10,`
  `"value-id": 11 }`| -|**String map value**|Map schema object|`value-id`|| -|**Map key, value**|Key, value fields in the element record.|`field-id`|`{ "type": "array",`
  `"logicalType": "map",`
  `"items": {`
    `"type": "record",`
    `"name": "k12_v13",`
    `"fields": [`
      `{ "name": "key",`
        `"type": "int",`
        `"field-id": 12 },`
      `{ "name": "value",`
        `"type": "string",`
        `"field-id": 13 }`
    `] } }`| - -Note that the string map case is for maps where the key type is a string. Using Avro’s map type in this case is optional. Maps with string keys may be stored as arrays. - - -### Parquet - -**Data Type Mappings** - -Values should be stored in Parquet using the types and logical type annotations in the table below. Column IDs are required. - -Lists must use the [3-level representation](https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists). - -| Type | Parquet physical type | Logical type | Notes | -|--------------------|--------------------------------------------------------------------|---------------------------------------------|----------------------------------------------------------------| -| **`boolean`** | `boolean` | | | -| **`int`** | `int` | | | -| **`long`** | `long` | | | -| **`float`** | `float` | | | -| **`double`** | `double` | | | -| **`decimal(P,S)`** | `P <= 9`: `int32`,
`P <= 18`: `int64`,
`fixed` otherwise | `DECIMAL(P,S)` | Fixed must use the minimum number of bytes that can store `P`. | -| **`date`** | `int32` | `DATE` | Stores days from the 1970-01-01. | -| **`time`** | `int64` | `TIME_MICROS` with `adjustToUtc=false` | Stores microseconds from midnight. | -| **`timestamp`** | `int64` | `TIMESTAMP_MICROS` with `adjustToUtc=false` | Stores microseconds from 1970-01-01 00:00:00.000000. | -| **`timestamptz`** | `int64` | `TIMESTAMP_MICROS` with `adjustToUtc=true` | Stores microseconds from 1970-01-01 00:00:00.000000 UTC. | -| **`string`** | `binary` | `UTF8` | Encoding must be UTF-8. | -| **`uuid`** | `fixed_len_byte_array[16]` | `UUID` | | -| **`fixed(L)`** | `fixed_len_byte_array[L]` | | | -| **`binary`** | `binary` | | | -| **`struct`** | `group` | | | -| **`list`** | `3-level list` | `LIST` | See Parquet docs for 3-level representation. | -| **`map`** | `3-level map` | `MAP` | See Parquet docs for 3-level representation. | - - -### ORC - -**Data Type Mappings** - -| Type | ORC type | ORC type attributes | Notes | -|--------------------|---------------------|------------------------------------------------------|-----------------------------------------------------------------------------------------| -| **`boolean`** | `boolean` | | | -| **`int`** | `int` | | ORC `tinyint` and `smallint` would also map to **`int`**. | -| **`long`** | `long` | | | -| **`float`** | `float` | | | -| **`double`** | `double` | | | -| **`decimal(P,S)`** | `decimal` | | | -| **`date`** | `date` | | | -| **`time`** | `long` | `iceberg.long-type`=`TIME` | Stores microseconds from midnight. | -| **`timestamp`** | `timestamp` | | [1] | -| **`timestamptz`** | `timestamp_instant` | | [1] | -| **`string`** | `string` | | ORC `varchar` and `char` would also map to **`string`**. | -| **`uuid`** | `binary` | `iceberg.binary-type`=`UUID` | | -| **`fixed(L)`** | `binary` | `iceberg.binary-type`=`FIXED` & `iceberg.length`=`L` | The length would not be checked by the ORC reader and should be checked by the adapter. | -| **`binary`** | `binary` | | | -| **`struct`** | `struct` | | | -| **`list`** | `array` | | | -| **`map`** | `map` | | | - -Notes: - -1. ORC's [TimestampColumnVector](https://orc.apache.org/api/hive-storage-api/org/apache/hadoop/hive/ql/exec/vector/TimestampColumnVector.html) consists of a time field (milliseconds since epoch) and a nanos field (nanoseconds within the second). Hence the milliseconds within the second are reported twice; once in the time field and again in the nanos field. The read adapter should only use milliseconds within the second from one of these fields. The write adapter should also report milliseconds within the second twice; once in the time field and again in the nanos field. ORC writer is expected to correctly consider millis information from one of the fields. More details at https://issues.apache.org/jira/browse/ORC-546 - -One of the interesting challenges with this is how to map Iceberg’s schema evolution (id based) on to ORC’s (name based). In theory, we could use Iceberg’s column ids as the column and field names, but that would be inconvenient. - -The column IDs must be stored in ORC type attributes using the key `iceberg.id`, and `iceberg.required` to store `"true"` if the Iceberg column is required, otherwise it will be optional. - -Iceberg would build the desired reader schema with their schema evolution rules and pass that down to the ORC reader, which would then use its schema evolution to map that to the writer’s schema. Basically, Iceberg would need to change the names of columns and fields to get the desired mapping. - -|Iceberg writer|ORC writer|Iceberg reader|ORC reader| -|--- |--- |--- |--- | -|`struct`|`struct`|`struct`|`struct`| -|`struct>`|`struct>`|`struct>`|`struct>`| - -## Appendix B: 32-bit Hash Requirements - -The 32-bit hash implementation is 32-bit Murmur3 hash, x86 variant, seeded with 0. - -| Primitive type | Hash specification | Test value | -|--------------------|-------------------------------------------|--------------------------------------------| -| **`int`** | `hashLong(long(v))` [1] | `34` → `2017239379` | -| **`long`** | `hashBytes(littleEndianBytes(v))` | `34L` → `2017239379` | -| **`decimal(P,S)`** | `hashBytes(minBigEndian(unscaled(v)))`[2] | `14.20` → `-500754589` | -| **`date`** | `hashInt(daysFromUnixEpoch(v))` | `2017-11-16` → `-653330422` | -| **`time`** | `hashLong(microsecsFromMidnight(v))` | `22:31:08` → `-662762989` | -| **`timestamp`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-2047944441` | -| **`timestamptz`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00`→ `-2047944441` | -| **`string`** | `hashBytes(utf8Bytes(v))` | `iceberg` → `1210000089` | -| **`uuid`** | `hashBytes(uuidBytes(v))` [3] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | -| **`fixed(L)`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | -| **`binary`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | - -The types below are not currently valid for bucketing, and so are not hashed. However, if that changes and a hash value is needed, the following table shall apply: - -| Primitive type | Hash specification | Test value | -|--------------------|-------------------------------------------|--------------------------------------------| -| **`boolean`** | `false: hashInt(0)`, `true: hashInt(1)` | `true` → `1392991556` | -| **`float`** | `hashLong(doubleToLongBits(double(v))` [4]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | -| **`double`** | `hashLong(doubleToLongBits(v))` [4]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | - -Notes: - -1. Integer and long hash results must be identical for all integer values. This ensures that schema evolution does not change bucket partition values if integer types are promoted. -2. Decimal values are hashed using the minimum number of bytes required to hold the unscaled value as a two’s complement big-endian; this representation does not include padding bytes required for storage in a fixed-length array. -Hash results are not dependent on decimal scale, which is part of the type, not the data value. -3. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: -`F7 9C 3E 09 67 7C 4B BD A4 79 3F 34 9C B7 85 E7` -4. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. - -## Appendix C: JSON serialization - - -### Schemas - -Schemas are serialized as a JSON object with the same fields as a struct in the table below, and the following additional fields: - -| v1 | v2 |Field|JSON representation|Example| -| ---------- | ---------- |--- |--- |--- | -| _optional_ | _required_ |**`schema-id`**|`JSON int`|`0`| -| _optional_ | _optional_ |**`identifier-field-ids`**|`JSON list of ints`|`[1, 2]`| - -Types are serialized according to this table: - -|Type|JSON representation|Example| -|--- |--- |--- | -|**`boolean`**|`JSON string: "boolean"`|`"boolean"`| -|**`int`**|`JSON string: "int"`|`"int"`| -|**`long`**|`JSON string: "long"`|`"long"`| -|**`float`**|`JSON string: "float"`|`"float"`| -|**`double`**|`JSON string: "double"`|`"double"`| -|**`date`**|`JSON string: "date"`|`"date"`| -|**`time`**|`JSON string: "time"`|`"time"`| -|**`timestamp without zone`**|`JSON string: "timestamp"`|`"timestamp"`| -|**`timestamp with zone`**|`JSON string: "timestamptz"`|`"timestamptz"`| -|**`string`**|`JSON string: "string"`|`"string"`| -|**`uuid`**|`JSON string: "uuid"`|`"uuid"`| -|**`fixed(L)`**|`JSON string: "fixed[]"`|`"fixed[16]"`| -|**`binary`**|`JSON string: "binary"`|`"binary"`| -|**`decimal(P, S)`**|`JSON string: "decimal(

,)"`|`"decimal(9,2)"`,
`"decimal(9, 2)"`| -|**`struct`**|`JSON object: {`
  `"type": "struct",`
  `"fields": [ {`
    `"id": ,`
    `"name": ,`
    `"required": ,`
    `"type": ,`
    `"doc": ,`
    `"initial-default": ,`
    `"write-default": `
    `}, ...`
  `] }`|`{`
  `"type": "struct",`
  `"fields": [ {`
    `"id": 1,`
    `"name": "id",`
    `"required": true,`
    `"type": "uuid",`
    `"initial-default": "0db3e2a8-9d1d-42b9-aa7b-74ebe558dceb",`
    `"write-default": "ec5911be-b0a7-458c-8438-c9a3e53cffae"`
  `}, {`
    `"id": 2,`
    `"name": "data",`
    `"required": false,`
    `"type": {`
      `"type": "list",`
      `...`
    `}`
  `} ]`
`}`| -|**`list`**|`JSON object: {`
  `"type": "list",`
  `"element-id": ,`
  `"element-required": `
  `"element": `
`}`|`{`
  `"type": "list",`
  `"element-id": 3,`
  `"element-required": true,`
  `"element": "string"`
`}`| -|**`map`**|`JSON object: {`
  `"type": "map",`
  `"key-id": ,`
  `"key": ,`
  `"value-id": ,`
  `"value-required": `
  `"value": `
`}`|`{`
  `"type": "map",`
  `"key-id": 4,`
  `"key": "string",`
  `"value-id": 5,`
  `"value-required": false,`
  `"value": "double"`
`}`| - -Note that default values are serialized using the JSON single-value serialization in [Appendix D](#appendix-d-single-value-serialization). - - -### Partition Specs - -Partition specs are serialized as a JSON object with the following fields: - -|Field|JSON representation|Example| -|--- |--- |--- | -|**`spec-id`**|`JSON int`|`0`| -|**`fields`**|`JSON list: [`
  `,`
  `...`
`]`|`[ {`
  `"source-id": 4,`
  `"field-id": 1000,`
  `"name": "ts_day",`
  `"transform": "day"`
`}, {`
  `"source-id": 1,`
  `"field-id": 1001,`
  `"name": "id_bucket",`
  `"transform": "bucket[16]"`
`} ]`| - -Each partition field in the fields list is stored as an object. See the table for more detail: - -|Transform or Field|JSON representation|Example| -|--- |--- |--- | -|**`identity`**|`JSON string: "identity"`|`"identity"`| -|**`bucket[N]`**|`JSON string: "bucket[]"`|`"bucket[16]"`| -|**`truncate[W]`**|`JSON string: "truncate[]"`|`"truncate[20]"`| -|**`year`**|`JSON string: "year"`|`"year"`| -|**`month`**|`JSON string: "month"`|`"month"`| -|**`day`**|`JSON string: "day"`|`"day"`| -|**`hour`**|`JSON string: "hour"`|`"hour"`| -|**`Partition Field`**|`JSON object: {`
  `"source-id": ,`
  `"field-id": ,`
  `"name": ,`
  `"transform": `
`}`|`{`
  `"source-id": 1,`
  `"field-id": 1000,`
  `"name": "id_bucket",`
  `"transform": "bucket[16]"`
`}`| - -In some cases partition specs are stored using only the field list instead of the object format that includes the spec ID, like the deprecated `partition-spec` field in table metadata. The object format should be used unless otherwise noted in this spec. - -The `field-id` property was added for each partition field in v2. In v1, the reference implementation assigned field ids sequentially in each spec starting at 1,000. See Partition Evolution for more details. - -### Sort Orders - -Sort orders are serialized as a list of JSON object, each of which contains the following fields: - -|Field|JSON representation|Example| -|--- |--- |--- | -|**`order-id`**|`JSON int`|`1`| -|**`fields`**|`JSON list: [`
  `,`
  `...`
`]`|`[ {`
  ` "transform": "identity",`
  ` "source-id": 2,`
  ` "direction": "asc",`
  ` "null-order": "nulls-first"`
  `}, {`
  ` "transform": "bucket[4]",`
  ` "source-id": 3,`
  ` "direction": "desc",`
  ` "null-order": "nulls-last"`
`} ]`| - -Each sort field in the fields list is stored as an object with the following properties: - -|Field|JSON representation|Example| -|--- |--- |--- | -|**`Sort Field`**|`JSON object: {`
  `"transform": ,`
  `"source-id": ,`
  `"direction": ,`
  `"null-order": `
`}`|`{`
  ` "transform": "bucket[4]",`
  ` "source-id": 3,`
  ` "direction": "desc",`
  ` "null-order": "nulls-last"`
`}`| - -The following table describes the possible values for the some of the field within sort field: - -|Field|JSON representation|Possible values| -|--- |--- |--- | -|**`direction`**|`JSON string`|`"asc", "desc"`| -|**`null-order`**|`JSON string`|`"nulls-first", "nulls-last"`| - - -### Table Metadata and Snapshots - -Table metadata is serialized as a JSON object according to the following table. Snapshots are not serialized separately. Instead, they are stored in the table metadata JSON. - -|Metadata field|JSON representation|Example| -|--- |--- |--- | -|**`format-version`**|`JSON int`|`1`| -|**`table-uuid`**|`JSON string`|`"fb072c92-a02b-11e9-ae9c-1bb7bc9eca94"`| -|**`location`**|`JSON string`|`"s3://b/wh/data.db/table"`| -|**`last-updated-ms`**|`JSON long`|`1515100955770`| -|**`last-column-id`**|`JSON int`|`22`| -|**`schema`**|`JSON schema (object)`|`See above, read schemas instead`| -|**`schemas`**|`JSON schemas (list of objects)`|`See above`| -|**`current-schema-id`**|`JSON int`|`0`| -|**`partition-spec`**|`JSON partition fields (list)`|`See above, read partition-specs instead`| -|**`partition-specs`**|`JSON partition specs (list of objects)`|`See above`| -|**`default-spec-id`**|`JSON int`|`0`| -|**`last-partition-id`**|`JSON int`|`1000`| -|**`properties`**|`JSON object: {`
  `"": "",`
  `...`
`}`|`{`
  `"write.format.default": "avro",`
  `"commit.retry.num-retries": "4"`
`}`| -|**`current-snapshot-id`**|`JSON long`|`3051729675574597004`| -|**`snapshots`**|`JSON list of objects: [ {`
  `"snapshot-id": ,`
  `"timestamp-ms": ,`
  `"summary": {`
    `"operation": ,`
    `... },`
  `"manifest-list": "",`
  `"schema-id": ""`
  `},`
  `...`
`]`|`[ {`
  `"snapshot-id": 3051729675574597004,`
  `"timestamp-ms": 1515100955770,`
  `"summary": {`
    `"operation": "append"`
  `},`
  `"manifest-list": "s3://b/wh/.../s1.avro"`
  `"schema-id": 0`
`} ]`| -|**`snapshot-log`**|`JSON list of objects: [`
  `{`
  `"snapshot-id": ,`
  `"timestamp-ms": `
  `},`
  `...`
`]`|`[ {`
  `"snapshot-id": 30517296...,`
  `"timestamp-ms": 1515100...`
`} ]`| -|**`metadata-log`**|`JSON list of objects: [`
  `{`
  `"metadata-file": ,`
  `"timestamp-ms": `
  `},`
  `...`
`]`|`[ {`
  `"metadata-file": "s3://bucket/.../v1.json",`
  `"timestamp-ms": 1515100...`
`} ]` | -|**`sort-orders`**|`JSON sort orders (list of sort field object)`|`See above`| -|**`default-sort-order-id`**|`JSON int`|`0`| -|**`refs`**|`JSON map with string key and object value:`
`{`
  `"": {`
  `"snapshot-id": ,`
  `"type": ,`
  `"max-ref-age-ms": ,`
  `...`
  `}`
  `...`
`}`|`{`
  `"test": {`
  `"snapshot-id": 123456789000,`
  `"type": "tag",`
  `"max-ref-age-ms": 10000000`
  `}`
`}`| - -### Name Mapping Serialization - -Name mapping is serialized as a list of field mapping JSON Objects which are serialized as follows - -|Field mapping field|JSON representation|Example| -|--- |--- |--- | -|**`names`**|`JSON list of strings`|`["latitude", "lat"]`| -|**`field_id`**|`JSON int`|`1`| -|**`fields`**|`JSON field mappings (list of objects)`|`[{ `
  `"field-id": 4,`
  `"names": ["latitude", "lat"]`
`}, {`
  `"field-id": 5,`
  `"names": ["longitude", "long"]`
`}]`| - -Example -```json -[ { "field-id": 1, "names": ["id", "record_id"] }, - { "field-id": 2, "names": ["data"] }, - { "field-id": 3, "names": ["location"], "fields": [ - { "field-id": 4, "names": ["latitude", "lat"] }, - { "field-id": 5, "names": ["longitude", "long"] } - ] } ] -``` - -### Content File (Data and Delete) Serialization - -Content file (data or delete) is serialized as a JSON object according to the following table. - -| Metadata field |JSON representation|Example| -|--------------------------|--- |--- | -| **`spec-id`** |`JSON int`|`1`| -| **`content`** |`JSON string`|`DATA`, `POSITION_DELETES`, `EQUALITY_DELETES`| -| **`file-path`** |`JSON string`|`"s3://b/wh/data.db/table"`| -| **`file-format`** |`JSON string`|`AVRO`, `ORC`, `PARQUET`| -| **`partition`** |`JSON object: Partition data tuple using partition field ids for the struct field ids`|`{"1000":1}`| -| **`record-count`** |`JSON long`|`1`| -| **`file-size-in-bytes`** |`JSON long`|`1024`| -| **`column-sizes`** |`JSON object: Map from column id to the total size on disk of all regions that store the column.`|`{"keys":[3,4],"values":[100,200]}`| -| **`value-counts`** |`JSON object: Map from column id to number of values in the column (including null and NaN values)`|`{"keys":[3,4],"values":[90,180]}`| -| **`null-value-counts`** |`JSON object: Map from column id to number of null values in the column`|`{"keys":[3,4],"values":[10,20]}`| -| **`nan-value-counts`** |`JSON object: Map from column id to number of NaN values in the column`|`{"keys":[3,4],"values":[0,0]}`| -| **`lower-bounds`** |`JSON object: Map from column id to lower bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["01000000","02000000"]}`| -| **`upper-bounds`** |`JSON object: Map from column id to upper bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["05000000","0A000000"]}`| -| **`key-metadata`** |`JSON string: Encryption key metadata binary serialized as hexadecimal string`|`00000000000000000000000000000000`| -| **`split-offsets`** |`JSON list of long: Split offsets for the data file`|`[128,256]`| -| **`equality-ids`** |`JSON list of int: Field ids used to determine row equality in equality delete files`|`[1]`| -| **`sort-order-id`** |`JSON int`|`1`| - -### File Scan Task Serialization - -File scan task is serialized as a JSON object according to the following table. - -| Metadata field |JSON representation|Example| -|--------------------------|--- |--- | -| **`schema`** |`JSON object`|`See above, read schemas instead`| -| **`spec`** |`JSON object`|`See above, read partition specs instead`| -| **`data-file`** |`JSON object`|`See above, read content file instead`| -| **`delete-files`** |`JSON list of objects`|`See above, read content file instead`| -| **`residual-filter`** |`JSON object: residual filter expression`|`{"type":"eq","term":"id","value":1}`| - -## Appendix D: Single-value serialization - -### Binary single-value serialization - -This serialization scheme is for storing single values as individual binary values in the lower and upper bounds maps of manifest files. - -| Type | Binary serialization | -|------------------------------|--------------------------------------------------------------------------------------------------------------| -| **`boolean`** | `0x00` for false, non-zero byte for true | -| **`int`** | Stored as 4-byte little-endian | -| **`long`** | Stored as 8-byte little-endian | -| **`float`** | Stored as 4-byte little-endian | -| **`double`** | Stored as 8-byte little-endian | -| **`date`** | Stores days from the 1970-01-01 in an 4-byte little-endian int | -| **`time`** | Stores microseconds from midnight in an 8-byte little-endian long | -| **`timestamp without zone`** | Stores microseconds from 1970-01-01 00:00:00.000000 in an 8-byte little-endian long | -| **`timestamp with zone`** | Stores microseconds from 1970-01-01 00:00:00.000000 UTC in an 8-byte little-endian long | -| **`string`** | UTF-8 bytes (without length) | -| **`uuid`** | 16-byte big-endian value, see example in Appendix B | -| **`fixed(L)`** | Binary value | -| **`binary`** | Binary value (without length) | -| **`decimal(P, S)`** | Stores unscaled value as two’s-complement big-endian binary, using the minimum number of bytes for the value | -| **`struct`** | Not supported | -| **`list`** | Not supported | -| **`map`** | Not supported | - -### JSON single-value serialization - - Single values are serialized as JSON by type according to the following table: - -| Type | JSON representation | Example | Description | -| ------------------ | ----------------------------------------- | ------------------------------------------ | -- | -| **`boolean`** | **`JSON boolean`** | `true` | | -| **`int`** | **`JSON int`** | `34` | | -| **`long`** | **`JSON long`** | `34` | | -| **`float`** | **`JSON number`** | `1.0` | | -| **`double`** | **`JSON number`** | `1.0` | | -| **`decimal(P,S)`** | **`JSON string`** | `"14.20"`, `"2E+20"` | Stores the string representation of the decimal value, specifically, for values with a positive scale, the number of digits to the right of the decimal point is used to indicate scale, for values with a negative scale, the scientific notation is used and the exponent must equal the negated scale | -| **`date`** | **`JSON string`** | `"2017-11-16"` | Stores ISO-8601 standard date | -| **`time`** | **`JSON string`** | `"22:31:08.123456"` | Stores ISO-8601 standard time with microsecond precision | -| **`timestamp`** | **`JSON string`** | `"2017-11-16T22:31:08.123456"` | Stores ISO-8601 standard timestamp with microsecond precision; must not include a zone offset | -| **`timestamptz`** | **`JSON string`** | `"2017-11-16T22:31:08.123456+00:00"` | Stores ISO-8601 standard timestamp with microsecond precision; must include a zone offset and it must be '+00:00' | -| **`string`** | **`JSON string`** | `"iceberg"` | | -| **`uuid`** | **`JSON string`** | `"f79c3e09-677c-4bbd-a479-3f349cb785e7"` | Stores the lowercase uuid string | -| **`fixed(L)`** | **`JSON string`** | `"000102ff"` | Stored as a hexadecimal string | -| **`binary`** | **`JSON string`** | `"000102ff"` | Stored as a hexadecimal string | -| **`struct`** | **`JSON object by field ID`** | `{"1": 1, "2": "bar"}` | Stores struct fields using the field ID as the JSON field name; field values are stored using this JSON single-value format | -| **`list`** | **`JSON array of values`** | `[1, 2, 3]` | Stores a JSON array of values that are serialized using this JSON single-value format | -| **`map`** | **`JSON object of key and value arrays`** | `{ "keys": ["a", "b"], "values": [1, 2] }` | Stores arrays of keys and values; individual keys and values are serialized using this JSON single-value format | - - -## Appendix E: Format version changes - -### Version 3 - -Default values are added to struct fields in v3. - -* The `write-default` is a forward-compatible change because it is only used at write time. Old writers will fail because the field is missing. -* Tables with `initial-default` will be read correctly by older readers if `initial-default` is always null for optional fields. Otherwise, old readers will default optional columns with null. Old readers will fail to read required fields which are populated by `initial-default` because that default is not supported. - -### Version 2 - -Writing v1 metadata: - -* Table metadata field `last-sequence-number` should not be written -* Snapshot field `sequence-number` should not be written -* Manifest list field `sequence-number` should not be written -* Manifest list field `min-sequence-number` should not be written -* Manifest list field `content` must be 0 (data) or omitted -* Manifest entry field `sequence_number` should not be written -* Manifest entry field `file_sequence_number` should not be written -* Data file field `content` must be 0 (data) or omitted - -Reading v1 metadata for v2: - -* Table metadata field `last-sequence-number` must default to 0 -* Snapshot field `sequence-number` must default to 0 -* Manifest list field `sequence-number` must default to 0 -* Manifest list field `min-sequence-number` must default to 0 -* Manifest list field `content` must default to 0 (data) -* Manifest entry field `sequence_number` must default to 0 -* Manifest entry field `file_sequence_number` must default to 0 -* Data file field `content` must default to 0 (data) - -Writing v2 metadata: - -* Table metadata JSON: - * `last-sequence-number` was added and is required; default to 0 when reading v1 metadata - * `table-uuid` is now required - * `current-schema-id` is now required - * `schemas` is now required - * `partition-specs` is now required - * `default-spec-id` is now required - * `last-partition-id` is now required - * `sort-orders` is now required - * `default-sort-order-id` is now required - * `schema` is no longer required and should be omitted; use `schemas` and `current-schema-id` instead - * `partition-spec` is no longer required and should be omitted; use `partition-specs` and `default-spec-id` instead -* Snapshot JSON: - * `sequence-number` was added and is required; default to 0 when reading v1 metadata - * `manifest-list` is now required - * `manifests` is no longer required and should be omitted; always use `manifest-list` instead -* Manifest list `manifest_file`: - * `content` was added and is required; 0=data, 1=deletes; default to 0 when reading v1 manifest lists - * `sequence_number` was added and is required - * `min_sequence_number` was added and is required - * `added_files_count` is now required - * `existing_files_count` is now required - * `deleted_files_count` is now required - * `added_rows_count` is now required - * `existing_rows_count` is now required - * `deleted_rows_count` is now required -* Manifest key-value metadata: - * `schema-id` is now required - * `partition-spec-id` is now required - * `format-version` is now required - * `content` was added and is required (must be "data" or "deletes") -* Manifest `manifest_entry`: - * `snapshot_id` is now optional to support inheritance - * `sequence_number` was added and is optional, to support inheritance - * `file_sequence_number` was added and is optional, to support inheritance -* Manifest `data_file`: - * `content` was added and is required; 0=data, 1=position deletes, 2=equality deletes; default to 0 when reading v1 manifests - * `equality_ids` was added, to be used for equality deletes only - * `block_size_in_bytes` was removed (breaks v1 reader compatibility) - * `file_ordinal` was removed - * `sort_columns` was removed - -Note that these requirements apply when writing data to a v2 table. Tables that are upgraded from v1 may contain metadata that does not follow these requirements. Implementations should remain backward-compatible with v1 metadata requirements. diff --git a/site/docs/spec.md b/site/docs/spec.md new file mode 120000 index 000000000000..975c95866515 --- /dev/null +++ b/site/docs/spec.md @@ -0,0 +1 @@ +../../format/spec.md \ No newline at end of file diff --git a/site/docs/view-spec.md b/site/docs/view-spec.md deleted file mode 100644 index 9c6ba3413be2..000000000000 --- a/site/docs/view-spec.md +++ /dev/null @@ -1,327 +0,0 @@ ---- -title: "View Spec" ---- - - -# Iceberg View Spec - -## Background and Motivation - -Most compute engines (e.g. Trino and Apache Spark) support views. A view is a logical table that can be referenced by future queries. Views do not contain any data. Instead, the query stored by the view is executed every time the view is referenced by another query. - -Each compute engine stores the metadata of the view in its proprietary format in the metastore of choice. Thus, views created from one engine can not be read or altered easily from another engine even when engines share the metastore as well as the storage system. This document standardizes the view metadata for ease of sharing the views across engines. - -## Goals - -* A common metadata format for view metadata, similar to how Iceberg supports a common table format for tables. - -## Overview - -View metadata storage mirrors how Iceberg table metadata is stored and retrieved. View metadata is maintained in metadata files. All changes to view state create a new view metadata file and completely replace the old metadata using an atomic swap. Like Iceberg tables, this atomic swap is delegated to the metastore that tracks tables and/or views by name. The view metadata file tracks the view schema, custom properties, current and past versions, as well as other metadata. - -Each metadata file is self-sufficient. It contains the history of the last few versions of the view and can be used to roll back the view to a previous version. - -### Metadata Location - -An atomic swap of one view metadata file for another provides the basis for making atomic changes. Readers use the version of the view that was current when they loaded the view metadata and are not affected by changes until they refresh and pick up a new metadata location. - -Writers create view metadata files optimistically, assuming that the current metadata location will not be changed before the writer's commit. Once a writer has created an update, it commits by swapping the view's metadata file pointer from the base location to the new location. - -## Specification - -### Terms - -* **Schema** -- Names and types of fields in a view. -* **Version** -- The state of a view at some point in time. - -### View Metadata - -The view version metadata file has the following fields: - -| Requirement | Field name | Description | -|-------------|----------------------|-------------| -| _required_ | `view-uuid` | A UUID that identifies the view, generated when the view is created. Implementations must throw an exception if a view's UUID does not match the expected UUID after refreshing metadata | -| _required_ | `format-version` | An integer version number for the view format; must be 1 | -| _required_ | `location` | The view's base location; used to create metadata file locations | -| _required_ | `schemas` | A list of known schemas | -| _required_ | `current-version-id` | ID of the current version of the view (`version-id`) | -| _required_ | `versions` | A list of known [versions](#versions) of the view [1] | -| _required_ | `version-log` | A list of [version log](#version-log) entries with the timestamp and `version-id` for every change to `current-version-id` | -| _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. - -#### Versions - -Each version in `versions` is a struct with the following fields: - -| Requirement | Field name | Description | -|-------------|---------------------|-------------------------------------------------------------------------------| -| _required_ | `version-id` | ID for the version | -| _required_ | `schema-id` | ID of the schema for the view version | -| _required_ | `timestamp-ms` | Timestamp when the version was created (ms from epoch) | -| _required_ | `summary` | A string to string map of [summary metadata](#summary) about the version | -| _required_ | `representations` | A list of [representations](#representations) for the view definition | -| _optional_ | `default-catalog` | Catalog name to use when a reference in the SELECT does not contain a catalog | -| _required_ | `default-namespace` | Namespace to use when a reference in the SELECT is a single identifier | - -When `default-catalog` is `null` or not set, the catalog in which the view is stored must be used as the default catalog. - -#### Summary - -Summary is a string to string map of metadata about a view version. Common metadata keys are documented here. - -| Requirement | Key | Value | -|-------------|------------------|-------| -| _required_ | `operation` | Operation that caused this metadata to be created; must be `create` or `replace` | -| _optional_ | `engine-name` | Name of the engine that created the view version | -| _optional_ | `engine-version` | Version of the engine that created the view version | - -#### Representations - -View definitions can be represented in multiple ways. Representations are documented ways to express a view definition. - -A view version can have more than one representation. All representations for a version must express the same underlying definition. Engines are free to choose the representation to use. - -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. - -##### SQL representation - -The SQL representation stores the view definition as a SQL SELECT, with metadata such as the SQL dialect. - -A view version can have multiple SQL representations of different dialects, but only one SQL representation per dialect. - -| Requirement | Field name | Type | Description | -|-------------|---------------------|----------------|-------------| -| _required_ | `type` | `string` | Must be `sql` | -| _required_ | `sql` | `string` | A SQL SELECT statement | -| _required_ | `dialect` | `string` | The dialect of the `sql` SELECT statement (e.g., "trino" or "spark") | - -For example: - -```sql -USE prod.default -``` -```sql -CREATE OR REPLACE VIEW event_agg ( - event_count COMMENT 'Count of events', - event_date) AS -SELECT - COUNT(1), CAST(event_ts AS DATE) -FROM events -GROUP BY 2 -``` - -This create statement would produce the following `sql` representation metadata: - -| Field name | Value | -|---------------------|-------| -| `type` | `"sql"` | -| `sql` | `"SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2"` | -| `dialect` | `"spark"` | - -If a create statement does not include column names or comments before `AS`, the fields should be omitted. - -The `event_count` (with the `Count of events` comment) and `event_date` field aliases must be part of the view version's `schema`. - -#### Version log - -The version log tracks changes to the view's current version. This is the view's history and allows reconstructing what version of the view would have been used at some point in time. - -Note that this is not the version's creation time, which is stored in each version's metadata. A version can appear multiple times in the version log, indicating that the view definition was rolled back. - -Each entry in `version-log` is a struct with the following fields: - -| Requirement | Field name | Description | -|-------------|----------------|-------------| -| _required_ | `timestamp-ms` | Timestamp when the view's `current-version-id` was updated (ms from epoch) | -| _required_ | `version-id` | ID that `current-version-id` was set to | - -## Appendix A: An Example - -The JSON metadata file format is described using an example below. - -Imagine the following sequence of operations: - -```sql -USE prod.default -``` -```sql -CREATE OR REPLACE VIEW event_agg ( - event_count COMMENT 'Count of events', - event_date) -COMMENT 'Daily event counts' -AS -SELECT - COUNT(1), CAST(event_ts AS DATE) -FROM events -GROUP BY 2 -``` - - -The metadata JSON file created looks as follows. - -The path is intentionally similar to the path for Iceberg tables and uses a `metadata` directory. - -``` -s3://bucket/warehouse/default.db/event_agg/metadata/00001-(uuid).metadata.json -``` -``` -{ - "view-uuid": "fa6506c3-7681-40c8-86dc-e36561f83385", - "format-version" : 1, - "location" : "s3://bucket/warehouse/default.db/event_agg", - "current-version-id" : 1, - "properties" : { - "comment" : "Daily event counts" - }, - "versions" : [ { - "version-id" : 1, - "timestamp-ms" : 1573518431292, - "schema-id" : 1, - "default-catalog" : "prod", - "default-namespace" : [ "default" ], - "summary" : { - "operation" : "create", - "engine-name" : "Spark", - "engineVersion" : "3.3.2" - }, - "representations" : [ { - "type" : "sql", - "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2", - "dialect" : "spark" - } ] - } ], - "schemas": [ { - "schema-id": 1, - "type" : "struct", - "fields" : [ { - "id" : 1, - "name" : "event_count", - "required" : false, - "type" : "int", - "doc" : "Count of events" - }, { - "id" : 2, - "name" : "event_date", - "required" : false, - "type" : "date" - } ] - } ], - "version-log" : [ { - "timestamp-ms" : 1573518431292, - "version-id" : 1 - } ] -} -``` - -Each change creates a new metadata JSON file. - -```sql -USE prod.other_db; -CREATE OR REPLACE VIEW default.event_agg ( - event_count, - event_date) -AS -SELECT - COUNT(1), CAST(event_ts AS DATE) -FROM prod.default.events -GROUP BY 2 -``` - -Updating the view produces a new metadata file that completely replaces the old: - -``` -s3://bucket/warehouse/default.db/event_agg/metadata/00002-(uuid).metadata.json -``` -``` -{ - "view-uuid": "fa6506c3-7681-40c8-86dc-e36561f83385", - "format-version" : 1, - "location" : "s3://bucket/warehouse/default.db/event_agg", - "current-version-id" : 1, - "properties" : { - "comment" : "Daily event counts" - }, - "versions" : [ { - "version-id" : 1, - "timestamp-ms" : 1573518431292, - "schema-id" : 1, - "default-catalog" : "prod", - "default-namespace" : [ "default" ], - "summary" : { - "operation" : "create", - "engine-name" : "Spark", - "engineVersion" : "3.3.2" - }, - "representations" : [ { - "type" : "sql", - "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM events\nGROUP BY 2", - "dialect" : "spark" - } ] - }, { - "version-id" : 2, - "timestamp-ms" : 1573518981593, - "schema-id" : 1, - "default-catalog" : "prod", - "default-namespace" : [ "default" ], - "summary" : { - "operation" : "create", - "engine-name" : "Spark", - "engineVersion" : "3.3.2" - }, - "representations" : [ { - "type" : "sql", - "sql" : "SELECT\n COUNT(1), CAST(event_ts AS DATE)\nFROM prod.default.events\nGROUP BY 2", - "dialect" : "spark" - } ] - } ], - "schemas": [ { - "schema-id": 1, - "type" : "struct", - "fields" : [ { - "id" : 1, - "name" : "event_count", - "required" : false, - "type" : "int", - "doc" : "Count of events" - }, { - "id" : 2, - "name" : "event_date", - "required" : false, - "type" : "date" - } ] - } ], - "version-log" : [ { - "timestamp-ms" : 1573518431292, - "version-id" : 1 - }, { - "timestamp-ms" : 1573518981593, - "version-id" : 2 - } ] -} -``` diff --git a/site/docs/view-spec.md b/site/docs/view-spec.md new file mode 120000 index 000000000000..7a150c43981e --- /dev/null +++ b/site/docs/view-spec.md @@ -0,0 +1 @@ +../../format/view-spec.md \ No newline at end of file diff --git a/site/nav.yml b/site/nav.yml index b588076fb59f..a12fd89724e2 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -36,6 +36,7 @@ nav: - Spec: spec.md - View spec: view-spec.md - Puffin spec: puffin-spec.md + - AES GCM Stream spec: gcm-stream-spec.md - Multi-engine support: multi-engine-support.md - How to release: how-to-release.md - Terms: terms.md From c3f43073a0c2a7ed86658b63092becce4866b676 Mon Sep 17 00:00:00 2001 From: Drew Gallardo Date: Fri, 23 Feb 2024 08:41:01 -0800 Subject: [PATCH 0068/1019] AWS: Deprecate DynamoDB catalog (#9783) --- .../org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java | 7 ++++++- .../iceberg/aws/dynamodb/DynamoDbTableOperations.java | 2 ++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java index 0c991af75076..f14aebe48036 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -84,7 +84,12 @@ import software.amazon.awssdk.services.dynamodb.model.TransactWriteItemsRequest; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; -/** DynamoDB implementation of Iceberg catalog */ +/** + * DynamoDB implementation of Iceberg catalog + * + * @deprecated since 1.6.0, will be removed in 2.0.0 + */ +@Deprecated public class DynamoDbCatalog extends BaseMetastoreCatalog implements SupportsNamespaces, Configurable { diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java index a1a330b11889..c5c70abb4126 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java @@ -44,6 +44,8 @@ import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; +/** @deprecated since 1.6.0, will be removed in 2.0.0 */ +@Deprecated class DynamoDbTableOperations extends BaseMetastoreTableOperations { private static final Logger LOG = LoggerFactory.getLogger(DynamoDbTableOperations.class); From a549aee598e2b2c18a8d340b94227bf466f7ed90 Mon Sep 17 00:00:00 2001 From: Drew Gallardo Date: Fri, 23 Feb 2024 11:29:54 -0800 Subject: [PATCH 0069/1019] AWS: Adjust Deprecation Version for DynamoDB Catalog to 1.5.0 (#9788) --- .../java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java | 2 +- .../apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java | 2 +- docs/docs/aws.md | 1 + 3 files changed, 3 insertions(+), 2 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java index f14aebe48036..393c6d7cdf10 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -87,7 +87,7 @@ /** * DynamoDB implementation of Iceberg catalog * - * @deprecated since 1.6.0, will be removed in 2.0.0 + * @deprecated since 1.5.0, will be removed in 2.0.0 */ @Deprecated public class DynamoDbCatalog extends BaseMetastoreCatalog diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java index c5c70abb4126..17e6c2c55a95 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java @@ -44,7 +44,7 @@ import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; -/** @deprecated since 1.6.0, will be removed in 2.0.0 */ +/** @deprecated since 1.5.0, will be removed in 2.0.0 */ @Deprecated class DynamoDbTableOperations extends BaseMetastoreTableOperations { diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 2bd6636670ee..eb217e424d44 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -226,6 +226,7 @@ PARTITIONED BY (category); ``` ### DynamoDB Catalog +**Deprecated:** As of version 1.5.0, the DynamoDB Catalog is planned for deprecation in version 2.0.0. Iceberg supports using a [DynamoDB](https://aws.amazon.com/dynamodb) table to record and manage database and table information. From c32699c272ba57831f915fb6a561ee23d12fda51 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Sat, 24 Feb 2024 19:52:39 +0900 Subject: [PATCH 0070/1019] Spark: Migrate procedure tests to JUnit5 (#9760) --- .../extensions/TestAncestorsOfProcedure.java | 27 ++-- .../extensions/TestCallStatementParser.java | 59 ++++----- .../spark/extensions/TestChangelogTable.java | 61 +++++---- .../TestCherrypickSnapshotProcedure.java | 24 ++-- .../TestCreateChangelogViewProcedure.java | 59 ++++----- .../TestExpireSnapshotsProcedure.java | 120 ++++++++---------- .../TestFastForwardBranchProcedure.java | 27 ++-- .../extensions/TestMigrateTableProcedure.java | 109 ++++++++-------- .../TestPublishChangesProcedure.java | 24 ++-- .../TestRegisterTableProcedure.java | 43 +++---- .../TestRemoveOrphanFilesProcedure.java | 110 +++++++--------- .../TestRewriteDataFilesProcedure.java | 96 +++++++------- .../TestRewriteManifestsProcedure.java | 111 ++++++++-------- .../TestRewritePositionDeleteFiles.java | 52 ++++---- ...stRewritePositionDeleteFilesProcedure.java | 65 +++++----- .../TestRollbackToSnapshotProcedure.java | 37 +++--- .../TestRollbackToTimestampProcedure.java | 31 ++--- .../TestSetCurrentSnapshotProcedure.java | 35 +++-- .../TestSnapshotTableProcedure.java | 92 +++++++------- .../spark/extensions/TestWriteAborts.java | 6 +- 20 files changed, 558 insertions(+), 630 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index 3282cdf381ce..437b4c368232 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -19,27 +19,24 @@ package org.apache.iceberg.spark.extensions; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestAncestorsOfProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestAncestorsOfProcedure extends ExtensionsTestBase { - public TestAncestorsOfProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testAncestorOfUsingEmptyArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -60,7 +57,7 @@ public void testAncestorOfUsingEmptyArgs() { output); } - @Test + @TestTemplate public void testAncestorOfUsingSnapshotId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -84,7 +81,7 @@ public void testAncestorOfUsingSnapshotId() { sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, preSnapshotId)); } - @Test + @TestTemplate public void testAncestorOfWithRollBack() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); Table table = validationCatalog.loadTable(tableIdent); @@ -128,7 +125,7 @@ public void testAncestorOfWithRollBack() { sql("CALL %s.system.ancestors_of('%s', %dL)", catalogName, tableIdent, thirdSnapshotId)); } - @Test + @TestTemplate public void testAncestorOfUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -145,7 +142,7 @@ public void testAncestorOfUsingNamedArgs() { catalogName, firstSnapshotId, tableIdent)); } - @Test + @TestTemplate public void testInvalidAncestorOfCases() { Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) .isInstanceOf(AnalysisException.class) 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 5c41604afae1..490127ab52f5 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 @@ -18,11 +18,13 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static scala.collection.JavaConverters.seqAsJavaList; + import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.expressions.Expression; @@ -38,22 +40,16 @@ import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import scala.collection.JavaConverters; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; public class TestCallStatementParser { - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private static SparkSession spark = null; private static ParserInterface parser = null; - @BeforeClass + @BeforeAll public static void startSpark() { TestCallStatementParser.spark = SparkSession.builder() @@ -64,7 +60,7 @@ public static void startSpark() { TestCallStatementParser.parser = spark.sessionState().sqlParser(); } - @AfterClass + @AfterAll public static void stopSpark() { SparkSession currentSpark = TestCallStatementParser.spark; TestCallStatementParser.spark = null; @@ -76,10 +72,9 @@ public static void stopSpark() { public void testCallWithPositionalArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL c.n.func(1, '2', 3L, true, 1.0D, 9.0e1, 900e-1BD)"); - Assert.assertEquals( - ImmutableList.of("c", "n", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).containsExactly("c", "n", "func"); - Assert.assertEquals(7, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(7); checkArg(call, 0, 1, DataTypes.IntegerType); checkArg(call, 1, "2", DataTypes.StringType); @@ -94,10 +89,9 @@ public void testCallWithPositionalArgs() throws ParseException { public void testCallWithNamedArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, c2 => '2', c3 => true)"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); - Assert.assertEquals(3, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(3); checkArg(call, 0, "c1", 1, DataTypes.IntegerType); checkArg(call, 1, "c2", "2", DataTypes.StringType); @@ -107,10 +101,9 @@ public void testCallWithNamedArgs() throws ParseException { @Test public void testCallWithMixedArgs() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(c1 => 1, '2')"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); - Assert.assertEquals(2, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(2); checkArg(call, 0, "c1", 1, DataTypes.IntegerType); checkArg(call, 1, "2", DataTypes.StringType); @@ -121,10 +114,9 @@ public void testCallWithTimestampArg() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func(TIMESTAMP '2017-02-03T10:37:30.00Z')"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); - Assert.assertEquals(1, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(1); checkArg( call, 0, Timestamp.from(Instant.parse("2017-02-03T10:37:30.00Z")), DataTypes.TimestampType); @@ -134,10 +126,9 @@ public void testCallWithTimestampArg() throws ParseException { public void testCallWithVarSubstitution() throws ParseException { CallStatement call = (CallStatement) parser.parsePlan("CALL cat.system.func('${spark.extra.prop}')"); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); - Assert.assertEquals(1, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(1); checkArg(call, 0, "value", DataTypes.StringType); } @@ -165,10 +156,9 @@ public void testCallStripsComments() throws ParseException { "CALL -- a line ending comment\n" + "cat.system.func('${spark.extra.prop}')"); for (String sqlText : callStatementsWithComments) { CallStatement call = (CallStatement) parser.parsePlan(sqlText); - Assert.assertEquals( - ImmutableList.of("cat", "system", "func"), JavaConverters.seqAsJavaList(call.name())); + assertThat(seqAsJavaList(call.name())).containsExactly("cat", "system", "func"); - Assert.assertEquals(1, call.args().size()); + assertThat(seqAsJavaList(call.args())).hasSize(1); checkArg(call, 0, "value", DataTypes.StringType); } @@ -188,7 +178,7 @@ private void checkArg( if (expectedName != null) { NamedArgument arg = checkCast(call.args().apply(index), NamedArgument.class); - Assert.assertEquals(expectedName, arg.name()); + assertThat(arg.name()).isEqualTo(expectedName); } else { CallArgument arg = call.args().apply(index); checkCast(arg, PositionalArgument.class); @@ -196,8 +186,8 @@ private void checkArg( Expression expectedExpr = toSparkLiteral(expectedValue, expectedType); Expression actualExpr = call.args().apply(index).expr(); - Assert.assertEquals("Arg types must match", expectedExpr.dataType(), actualExpr.dataType()); - Assert.assertEquals("Arg must match", expectedExpr, actualExpr); + assertThat(actualExpr.dataType()).as("Arg types must match").isEqualTo(expectedExpr.dataType()); + assertThat(actualExpr).as("Arg must match").isEqualTo(expectedExpr); } private Literal toSparkLiteral(Object value, DataType dataType) { @@ -205,8 +195,7 @@ private Literal toSparkLiteral(Object value, DataType dataType) { } private T checkCast(Object value, Class expectedClass) { - Assert.assertTrue( - "Expected instance of " + expectedClass.getName(), expectedClass.isInstance(value)); + assertThat(value).isInstanceOf(expectedClass); return expectedClass.cast(value); } } 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 fe44fa3dec52..daf45626753b 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 @@ -21,59 +21,56 @@ import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; -import java.util.Map; import org.apache.iceberg.DataOperations; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Row; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized.Parameters; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestChangelogTable extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestChangelogTable extends ExtensionsTestBase { - @Parameters(name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}, formatVersion = {3}") public static Object[][] parameters() { return new Object[][] { { - 1, SparkCatalogConfig.SPARK.catalogName(), SparkCatalogConfig.SPARK.implementation(), - SparkCatalogConfig.SPARK.properties() + SparkCatalogConfig.SPARK.properties(), + 1 }, { - 2, SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties() + SparkCatalogConfig.HIVE.properties(), + 2 } }; } - private final int formatVersion; + @Parameter(index = 3) + private int formatVersion; - public TestChangelogTable( - int formatVersion, String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - this.formatVersion = formatVersion; - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDataFilters() { createTableWithDefaultRows(); @@ -97,7 +94,7 @@ public void testDataFilters() { sql("SELECT * FROM %s.changes WHERE id = 3 ORDER BY _change_ordinal, id", tableName)); } - @Test + @TestTemplate public void testOverwrites() { createTableWithDefaultRows(); @@ -119,7 +116,7 @@ public void testOverwrites() { changelogRecords(snap2, snap3)); } - @Test + @TestTemplate public void testQueryWithTimeRange() { createTable(); @@ -189,7 +186,7 @@ public void testQueryWithTimeRange() { changelogRecords(rightAfterSnap2, snap3.timestampMillis() - 1)); } - @Test + @TestTemplate public void testTimeRangeValidation() { createTableWithDefaultRows(); @@ -207,7 +204,7 @@ public void testTimeRangeValidation() { .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); } - @Test + @TestTemplate public void testMetadataDeletes() { createTableWithDefaultRows(); @@ -220,7 +217,7 @@ public void testMetadataDeletes() { table.refresh(); Snapshot snap3 = table.currentSnapshot(); - Assert.assertEquals("Operation must match", DataOperations.DELETE, snap3.operation()); + assertThat(snap3.operation()).as("Operation must match").isEqualTo(DataOperations.DELETE); assertEquals( "Rows should match", @@ -228,7 +225,7 @@ public void testMetadataDeletes() { changelogRecords(snap2, snap3)); } - @Test + @TestTemplate public void testExistingEntriesInNewDataManifestsAreIgnored() { sql( "CREATE TABLE %s (id INT, data STRING) " @@ -252,7 +249,7 @@ public void testExistingEntriesInNewDataManifestsAreIgnored() { table.refresh(); Snapshot snap2 = table.currentSnapshot(); - Assert.assertEquals("Manifest number must match", 1, snap2.dataManifests(table.io()).size()); + assertThat(snap2.dataManifests(table.io())).as("Manifest number must match").hasSize(1); assertEquals( "Rows should match", @@ -260,14 +257,14 @@ public void testExistingEntriesInNewDataManifestsAreIgnored() { changelogRecords(snap1, snap2)); } - @Test + @TestTemplate public void testManifestRewritesAreIgnored() { createTableWithDefaultRows(); sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Num snapshots must match", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Num snapshots must match").hasSize(3); assertEquals( "Should have expected rows", @@ -275,7 +272,7 @@ public void testManifestRewritesAreIgnored() { sql("SELECT id, _change_type FROM %s.changes ORDER BY id", tableName)); } - @Test + @TestTemplate public void testMetadataColumns() { createTableWithDefaultRows(); List rows = @@ -284,7 +281,7 @@ public void testMetadataColumns() { tableName); String file1 = rows.get(0)[1].toString(); - Assert.assertTrue(file1.startsWith("file:/")); + assertThat(file1).startsWith("file:/"); String file2 = rows.get(1)[1].toString(); assertEquals( 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 c9af319f5c35..794d77980c74 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 @@ -21,7 +21,6 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import java.util.List; -import java.util.Map; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,22 +31,17 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestCherrypickSnapshotProcedure extends SparkExtensionsTestBase { +public class TestCherrypickSnapshotProcedure extends ExtensionsTestBase { - public TestCherrypickSnapshotProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testCherrypickSnapshotUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -84,7 +78,7 @@ public void testCherrypickSnapshotUsingPositionalArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testCherrypickSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -121,7 +115,7 @@ public void testCherrypickSnapshotUsingNamedArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testCherrypickSnapshotRefreshesRelationCache() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -157,7 +151,7 @@ public void testCherrypickSnapshotRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -167,7 +161,7 @@ public void testCherrypickInvalidSnapshot() { .hasMessage("Cannot cherry-pick unknown snapshot ID: -1"); } - @Test + @TestTemplate public void testInvalidCherrypickSnapshotCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index 9aa4bd3d7c8c..bc60759bd038 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -18,31 +18,28 @@ */ package org.apache.iceberg.spark.extensions; -import static org.junit.Assert.assertThrows; +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.ChangelogOperation; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkReadOptions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateChangelogViewProcedure extends ExtensionsTestBase { private static final String DELETE = ChangelogOperation.DELETE.name(); private static final String INSERT = ChangelogOperation.INSERT.name(); private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name(); private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name(); - public TestCreateChangelogViewProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -62,7 +59,7 @@ private void createTableWithIdentifierField() { sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); } - @Test + @TestTemplate public void testCustomizedViewName() { createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); @@ -92,10 +89,10 @@ public void testCustomizedViewName() { "cdc_view"); long rowCount = sql("select * from %s", "cdc_view").stream().count(); - Assert.assertEquals(2, rowCount); + assertThat(rowCount).isEqualTo(2); } - @Test + @TestTemplate public void testNoSnapshotIdInput() { createTableWithTwoColumns(); sql("INSERT INTO %s VALUES (1, 'a')", tableName); @@ -126,7 +123,7 @@ public void testNoSnapshotIdInput() { sql("select * from %s order by _change_ordinal, id", viewName)); } - @Test + @TestTemplate public void testTimestampsBasedQuery() { createTableWithTwoColumns(); long beginning = System.currentTimeMillis(); @@ -186,7 +183,7 @@ public void testTimestampsBasedQuery() { sql("select * from %s order by _change_ordinal, id", returns.get(0)[0])); } - @Test + @TestTemplate public void testUpdate() { createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); @@ -217,7 +214,7 @@ public void testUpdate() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testUpdateWithIdentifierField() { createTableWithIdentifierField(); @@ -245,7 +242,7 @@ public void testUpdateWithIdentifierField() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testUpdateWithFilter() { createTableWithTwoColumns(); sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName); @@ -277,7 +274,7 @@ public void testUpdateWithFilter() { sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testUpdateWithMultipleIdentifierColumns() { createTableWithThreeColumns(); @@ -309,7 +306,7 @@ public void testUpdateWithMultipleIdentifierColumns() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testRemoveCarryOvers() { createTableWithThreeColumns(); @@ -343,7 +340,7 @@ public void testRemoveCarryOvers() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testRemoveCarryOversWithoutUpdatedRows() { createTableWithThreeColumns(); @@ -375,7 +372,7 @@ public void testRemoveCarryOversWithoutUpdatedRows() { sql("select * from %s order by _change_ordinal, id, data", viewName)); } - @Test + @TestTemplate public void testNetChangesWithRemoveCarryOvers() { // partitioned by id createTableWithThreeColumns(); @@ -428,15 +425,15 @@ public void testNetChangesWithRemoveCarryOvers() { sql("select * from %s order by _change_ordinal, data", viewName)); } - @Test + @TestTemplate public void testNetChangesWithComputeUpdates() { createTableWithTwoColumns(); - assertThrows( - "Should fail because net_changes is not supported with computing updates", - IllegalArgumentException.class, - () -> - sql( - "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", - catalogName, tableName)); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'), net_changes => true)", + catalogName, tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Not support net changes with update images"); } } 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 7dacce5487d6..46847a03ca7b 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 @@ -19,15 +19,16 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.nio.file.Files; import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -43,7 +44,6 @@ import org.apache.iceberg.puffin.Puffin; import org.apache.iceberg.puffin.PuffinWriter; 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.spark.SparkCatalog; import org.apache.iceberg.spark.data.TestHelpers; @@ -52,23 +52,17 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestExpireSnapshotsProcedure extends SparkExtensionsTestBase { +public class TestExpireSnapshotsProcedure extends ExtensionsTestBase { - public TestExpireSnapshotsProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testExpireSnapshotsInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -77,7 +71,7 @@ public void testExpireSnapshotsInEmptyTable() { "Should not delete any files", ImmutableList.of(row(0L, 0L, 0L, 0L, 0L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotsUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -95,7 +89,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { Timestamp secondSnapshotTimestamp = Timestamp.from(Instant.ofEpochMilli(secondSnapshot.timestampMillis())); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); // expire without retainLast param List output1 = @@ -107,7 +101,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { table.refresh(); - Assert.assertEquals("Should expire one snapshot", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should expire one snapshot").hasSize(1); sql("INSERT OVERWRITE %s VALUES (3, 'c')", tableName); sql("INSERT INTO TABLE %s VALUES (4, 'd')", tableName); @@ -122,7 +116,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); - Assert.assertEquals("Should be 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 3 snapshots").hasSize(3); // expire with retainLast param List output = @@ -133,7 +127,7 @@ public void testExpireSnapshotsUsingPositionalArgs() { "Procedure output must match", ImmutableList.of(row(2L, 0L, 0L, 2L, 1L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -142,7 +136,7 @@ public void testExpireSnapshotUsingNamedArgs() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -156,7 +150,7 @@ public void testExpireSnapshotUsingNamedArgs() { "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotsGCDisabled() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -168,7 +162,7 @@ public void testExpireSnapshotsGCDisabled() { .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } - @Test + @TestTemplate public void testInvalidExpireSnapshotsCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) @@ -194,7 +188,7 @@ public void testInvalidExpireSnapshotsCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testResolvingTableInAnotherCatalog() throws IOException { String anotherCatalog = "another_" + catalogName; spark.conf().set("spark.sql.catalog." + anotherCatalog, SparkCatalog.class.getName()); @@ -203,7 +197,7 @@ public void testResolvingTableInAnotherCatalog() throws IOException { .conf() .set( "spark.sql.catalog." + anotherCatalog + ".warehouse", - "file:" + temp.newFolder().toString()); + Files.createTempDirectory(temp, "junit").toFile().toURI().toString()); sql( "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", @@ -218,7 +212,7 @@ public void testResolvingTableInAnotherCatalog() throws IOException { .hasMessageStartingWith("Cannot run procedure in catalog"); } - @Test + @TestTemplate public void testConcurrentExpireSnapshots() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -241,7 +235,7 @@ public void testConcurrentExpireSnapshots() { output); } - @Test + @TestTemplate public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -262,7 +256,7 @@ public void testConcurrentExpireSnapshotsWithInvalidInput() { .hasMessage("max_concurrent_deletes should have value > 0, value: -1"); } - @Test + @TestTemplate public void testExpireDeleteFiles() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" @@ -284,9 +278,8 @@ public void testExpireDeleteFiles() throws Exception { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); - Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); + 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().path())); @@ -306,13 +299,14 @@ public void testExpireDeleteFiles() throws Exception { sql("INSERT INTO TABLE %s VALUES (6, 'f')", tableName); // this txn removes the file reference table.refresh(); - Assert.assertEquals( - "Should have no delete manifests", 0, TestHelpers.deleteManifests(table).size()); - Assert.assertEquals("Should have no delete files", 0, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteManifests(table)).as("Should have no delete manifests").hasSize(0); + assertThat(TestHelpers.deleteFiles(table)).as("Should have no delete files").hasSize(0); FileSystem localFs = FileSystem.getLocal(new Configuration()); - Assert.assertTrue("Delete manifest should still exist", localFs.exists(deleteManifestPath)); - Assert.assertTrue("Delete file should still exist", localFs.exists(deleteFilePath)); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should still exist") + .isTrue(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should still exist").isTrue(); Timestamp currentTimestamp = Timestamp.from(Instant.ofEpochMilli(System.currentTimeMillis())); List output = @@ -324,11 +318,13 @@ public void testExpireDeleteFiles() throws Exception { "Should deleted 1 data and pos delete file and 4 manifests and lists (one for each txn)", ImmutableList.of(row(1L, 1L, 0L, 4L, 4L, 0L)), output); - Assert.assertFalse("Delete manifest should be removed", localFs.exists(deleteManifestPath)); - Assert.assertFalse("Delete file should be removed", localFs.exists(deleteFilePath)); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should be removed") + .isFalse(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should be removed").isFalse(); } - @Test + @TestTemplate public void testExpireSnapshotWithStreamResultsEnabled() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -337,7 +333,7 @@ public void testExpireSnapshotWithStreamResultsEnabled() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -354,7 +350,7 @@ public void testExpireSnapshotWithStreamResultsEnabled() { "Procedure output must match", ImmutableList.of(row(0L, 0L, 0L, 0L, 1L, 0L)), output); } - @Test + @TestTemplate public void testExpireSnapshotsWithSnapshotId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -363,7 +359,7 @@ public void testExpireSnapshotsWithSnapshotId() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); // Expiring the snapshot specified by snapshot_id should keep only a single snapshot. long firstSnapshotId = table.currentSnapshot().parentId(); @@ -373,16 +369,14 @@ public void testExpireSnapshotsWithSnapshotId() { // There should only be one single snapshot left. table.refresh(); - Assert.assertEquals("Should be 1 snapshots", 1, Iterables.size(table.snapshots())); - Assert.assertEquals( - "Snapshot ID should not be present", - 0, - Iterables.size( - Iterables.filter( - table.snapshots(), snapshot -> snapshot.snapshotId() == firstSnapshotId))); + assertThat(table.snapshots()).as("Should be 1 snapshots").hasSize(1); + assertThat(table.snapshots()) + .as("Snapshot ID should not be present") + .filteredOn(snapshot -> snapshot.snapshotId() == firstSnapshotId) + .hasSize(0); } - @Test + @TestTemplate public void testExpireSnapshotShouldFailForCurrentSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -390,7 +384,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); Assertions.assertThatThrownBy( () -> @@ -406,7 +400,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { .hasMessageStartingWith("Cannot expire"); } - @Test + @TestTemplate public void testExpireSnapshotsProcedureWorksWithSqlComments() { // Ensure that systems such as dbt, that inject comments into the generated SQL files, will // work with Iceberg-specific DDL @@ -417,7 +411,7 @@ public void testExpireSnapshotsProcedureWorksWithSqlComments() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -436,10 +430,10 @@ public void testExpireSnapshotsProcedureWorksWithSqlComments() { table.refresh(); - Assert.assertEquals("Should be 1 snapshot remaining", 1, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).as("Should be 1 snapshot remaining").hasSize(1); } - @Test + @TestTemplate public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); @@ -471,26 +465,26 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.statisticsFiles()) + assertThat(table.statisticsFiles()) .as( "Statistics file entry in TableMetadata should be present only for the snapshot %s", statisticsFile2.snapshotId()) .extracting(StatisticsFile::snapshotId) .containsExactly(statisticsFile2.snapshotId()); - Assertions.assertThat(new File(statsFileLocation1)) + assertThat(new File(statsFileLocation1)) .as("Statistics file should not exist for snapshot %s", statisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(statsFileLocation2)) + assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); } - @Test + @TestTemplate public void testExpireSnapshotsWithPartitionStatisticFiles() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (10, 'abc')", tableName); @@ -516,25 +510,23 @@ public void testExpireSnapshotsWithPartitionStatisticFiles() { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]) - .as("should be 1 deleted partition statistics file") - .isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted partition statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.partitionStatisticsFiles()) + assertThat(table.partitionStatisticsFiles()) .as( "partition statistics file entry in TableMetadata should be present only for the snapshot %s", partitionStatisticsFile2.snapshotId()) .extracting(PartitionStatisticsFile::snapshotId) .containsExactly(partitionStatisticsFile2.snapshotId()); - Assertions.assertThat(new File(partitionStatsFileLocation1)) + assertThat(new File(partitionStatsFileLocation1)) .as( "partition statistics file should not exist for snapshot %s", partitionStatisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(partitionStatsFileLocation2)) + assertThat(new File(partitionStatsFileLocation2)) .as( "partition statistics file should exist for snapshot %s", partitionStatisticsFile2.snapshotId()) 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 0c99c3e07f6b..fd8ee7d91bdc 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 @@ -23,7 +23,6 @@ import java.util.Arrays; import java.util.List; -import java.util.Map; import java.util.stream.Collectors; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; @@ -31,21 +30,17 @@ 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.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestFastForwardBranchProcedure extends SparkExtensionsTestBase { - public TestFastForwardBranchProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestFastForwardBranchProcedure extends ExtensionsTestBase { - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testFastForwardBranchUsingPositionalArgs() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -96,7 +91,7 @@ public void testFastForwardBranchUsingPositionalArgs() { sql("SELECT * FROM %s order by id", tableName)); } - @Test + @TestTemplate public void testFastForwardBranchUsingNamedArgs() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -129,7 +124,7 @@ public void testFastForwardBranchUsingNamedArgs() { sql("SELECT * FROM %s order by id", tableName)); } - @Test + @TestTemplate public void testFastForwardWhenTargetIsNotAncestorFails() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -163,7 +158,7 @@ public void testFastForwardWhenTargetIsNotAncestorFails() { .hasMessage("Cannot fast-forward: main is not an ancestor of testBranch"); } - @Test + @TestTemplate public void testInvalidFastForwardBranchCases() { assertThatThrownBy( () -> @@ -189,7 +184,7 @@ public void testInvalidFastForwardBranchCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testFastForwardNonExistingToRefFails() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); assertThatThrownBy( @@ -201,7 +196,7 @@ public void testFastForwardNonExistingToRefFails() { .hasMessage("Ref does not exist: non_existing_branch"); } - @Test + @TestTemplate public void testFastForwardNonMain() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -231,7 +226,7 @@ public void testFastForwardNonMain() { .containsExactly(row(branch1, branch1Snapshot.snapshotId(), branch2Snapshot.snapshotId())); } - @Test + @TestTemplate public void testFastForwardNonExistingFromMainCreatesBranch() { sql("CREATE TABLE %s (id int NOT NULL, data string) USING iceberg", tableName); String branch1 = "branch1"; 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 0d23e4bbf068..26a888356ca6 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 @@ -18,51 +18,46 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.IOException; +import java.nio.file.Files; import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestMigrateTableProcedure extends SparkExtensionsTestBase { - - public TestMigrateTableProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - @After +@ExtendWith(ParameterizedTestExtension.class) +public class TestMigrateTableProcedure extends ExtensionsTestBase { + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); } - @Test + @TestTemplate public void testMigrate() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + 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 LOCATION '%s'", tableName, location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); String tableLocation = createdTable.location().replace("file:", ""); - Assert.assertEquals("Table should have original location", location, tableLocation); + assertThat(tableLocation).as("Table should have original location").isEqualTo(location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -74,10 +69,10 @@ public void testMigrate() throws IOException { sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); } - @Test + @TestTemplate public void testMigrateWithOptions() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + 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 LOCATION '%s'", tableName, location); @@ -86,15 +81,15 @@ public void testMigrateWithOptions() throws IOException { Object result = scalarSql("CALL %s.system.migrate('%s', map('foo', 'bar'))", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); Map props = createdTable.properties(); - Assert.assertEquals("Should have extra property set", "bar", props.get("foo")); + assertThat(props).containsEntry("foo", "bar"); String tableLocation = createdTable.location().replace("file:", ""); - Assert.assertEquals("Table should have original location", location, tableLocation); + assertThat(tableLocation).as("Table should have original location").isEqualTo(location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -106,10 +101,10 @@ public void testMigrateWithOptions() throws IOException { sql("DROP TABLE IF EXISTS %s", tableName + "_BACKUP_"); } - @Test + @TestTemplate public void testMigrateWithDropBackup() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + 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 LOCATION '%s'", tableName, location); @@ -118,14 +113,14 @@ public void testMigrateWithDropBackup() throws IOException { Object result = scalarSql( "CALL %s.system.migrate(table => '%s', drop_backup => true)", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); - Assert.assertFalse(spark.catalog().tableExists(tableName + "_BACKUP_")); + assertThat(result).as("Should have added one file").isEqualTo(1L); + assertThat(spark.catalog().tableExists(tableName + "_BACKUP_")).isFalse(); } - @Test + @TestTemplate public void testMigrateWithBackupTableName() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + 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 LOCATION '%s'", tableName, location); @@ -137,16 +132,16 @@ public void testMigrateWithBackupTableName() throws IOException { "CALL %s.system.migrate(table => '%s', backup_table_name => '%s')", catalogName, tableName, backupTableName); - Assertions.assertThat(result).isEqualTo(1L); + assertThat(result).isEqualTo(1L); String dbName = tableName.split("\\.")[0]; - Assertions.assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); + assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); } - @Test + @TestTemplate public void testMigrateWithInvalidMetricsConfig() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -160,11 +155,11 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { .hasMessageStartingWith("Invalid metrics config"); } - @Test + @TestTemplate public void testMigrateWithConflictingProps() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); @@ -172,7 +167,7 @@ public void testMigrateWithConflictingProps() throws IOException { Object result = scalarSql("CALL %s.system.migrate('%s', map('migrated', 'false'))", catalogName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( "Should have expected rows", @@ -180,10 +175,10 @@ public void testMigrateWithConflictingProps() throws IOException { sql("SELECT * FROM %s", tableName)); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals("Should override user value", "true", table.properties().get("migrated")); + assertThat(table.properties()).containsEntry("migrated", "true"); } - @Test + @TestTemplate public void testInvalidMigrateCases() { Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) .isInstanceOf(AnalysisException.class) @@ -199,10 +194,10 @@ public void testInvalidMigrateCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testMigratePartitionWithSpecialCharacter() throws IOException { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string, dt date) USING parquet " + "PARTITIONED BY (data, dt) LOCATION '%s'", @@ -216,25 +211,25 @@ public void testMigratePartitionWithSpecialCharacter() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testMigrateEmptyPartitionedTable() throws Exception { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + 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); Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); - Assert.assertEquals(0L, result); + assertThat(result).isEqualTo(0L); } - @Test + @TestTemplate public void testMigrateEmptyTable() throws Exception { - Assume.assumeTrue(catalogName.equals("spark_catalog")); - String location = temp.newFolder().toString(); + 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 LOCATION '%s'", tableName, location); Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); - Assert.assertEquals(0L, result); + assertThat(result).isEqualTo(0L); } } 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 94c4af3ad56d..7121c59df541 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 @@ -21,7 +21,6 @@ import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import java.util.List; -import java.util.Map; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,22 +31,17 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestPublishChangesProcedure extends SparkExtensionsTestBase { +public class TestPublishChangesProcedure extends ExtensionsTestBase { - public TestPublishChangesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testApplyWapChangesUsingPositionalArgs() { String wapId = "wap_id_1"; sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -83,7 +77,7 @@ public void testApplyWapChangesUsingPositionalArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testApplyWapChangesUsingNamedArgs() { String wapId = "wap_id_1"; sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -121,7 +115,7 @@ public void testApplyWapChangesUsingNamedArgs() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testApplyWapChangesRefreshesRelationCache() { String wapId = "wap_id_1"; sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -153,7 +147,7 @@ public void testApplyWapChangesRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -163,7 +157,7 @@ public void testApplyInvalidWapId() { .hasMessage("Cannot apply unknown WAP ID 'not_valid'"); } - @Test + @TestTemplate public void testInvalidApplyWapChangesCases() { Assertions.assertThatThrownBy( () -> diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java index 2f1165e9cd5e..3047dccd959b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRegisterTableProcedure.java @@ -18,40 +18,39 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; -import java.util.Map; import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.spark.Spark3Util; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.functions; import org.apache.spark.sql.types.DataTypes; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +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; -public class TestRegisterTableProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRegisterTableProcedure extends ExtensionsTestBase { - private final String targetName; + private String targetName; - public TestRegisterTableProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); + @BeforeEach + public void setTargetName() { targetName = tableName("register_table"); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @After + @AfterEach public void dropTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", targetName); } - @Test + @TestTemplate public void testRegisterTable() throws NoSuchTableException, ParseException { long numRows = 1000; @@ -70,16 +69,16 @@ public void testRegisterTable() throws NoSuchTableException, ParseException { List result = sql("CALL %s.system.register_table('%s', '%s')", catalogName, targetName, metadataJson); - Assert.assertEquals("Current Snapshot is not correct", currentSnapshotId, result.get(0)[0]); + assertThat(result.get(0)[0]).as("Current Snapshot is not correct").isEqualTo(currentSnapshotId); List original = sql("SELECT * FROM %s", tableName); List registered = sql("SELECT * FROM %s", targetName); assertEquals("Registered table rows should match original table rows", original, registered); - Assert.assertEquals( - "Should have the right row count in the procedure result", numRows, result.get(0)[1]); - Assert.assertEquals( - "Should have the right datafile count in the procedure result", - originalFileCount, - result.get(0)[2]); + assertThat(result.get(0)[1]) + .as("Should have the right row count in the procedure result") + .isEqualTo(numRows); + assertThat(result.get(0)[2]) + .as("Should have the right datafile count in the procedure result") + .isEqualTo(originalFileCount); } } 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 80b515d34417..5d48daa74f40 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -29,7 +30,6 @@ import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; @@ -65,28 +65,18 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestRemoveOrphanFilesProcedure extends SparkExtensionsTestBase { +public class TestRemoveOrphanFilesProcedure extends ExtensionsTestBase { - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - public TestRemoveOrphanFilesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s PURGE", tableName); sql("DROP TABLE IF EXISTS p PURGE"); } - @Test + @TestTemplate public void testRemoveOrphanFilesInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -97,7 +87,7 @@ public void testRemoveOrphanFilesInEmptyTable() { assertEquals("Should have no rows", ImmutableList.of(), sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRemoveOrphanFilesInDataFolder() throws IOException { if (catalogName.equals("testhadoop")) { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -106,9 +96,8 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder()); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } - sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); @@ -143,7 +132,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be orphan files in the data folder", 1, output2.size()); + assertThat(output2).as("Should be orphan files in the data folder").hasSize(1); // the previous call should have deleted all orphan files List output3 = @@ -152,7 +141,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be no more orphan files in the data folder", 0, output3.size()); + assertThat(output3).as("Should be no more orphan files in the data folder").hasSize(0); assertEquals( "Should have expected rows", @@ -160,7 +149,7 @@ public void testRemoveOrphanFilesInDataFolder() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRemoveOrphanFilesDryRun() throws IOException { if (catalogName.equals("testhadoop")) { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -169,7 +158,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder()); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -194,7 +183,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { + "older_than => TIMESTAMP '%s'," + "dry_run => true)", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be one orphan files", 1, output1.size()); + assertThat(output1).as("Should be one orphan files").hasSize(1); // actually delete orphans List output2 = @@ -203,7 +192,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be one orphan files", 1, output2.size()); + assertThat(output2).as("Should be one orphan files").hasSize(1); // the previous call should have deleted all orphan files List output3 = @@ -212,7 +201,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be no more orphan files", 0, output3.size()); + assertThat(output3).as("Should be no more orphan files").hasSize(0); assertEquals( "Should have expected rows", @@ -220,7 +209,7 @@ public void testRemoveOrphanFilesDryRun() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRemoveOrphanFilesGCDisabled() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -236,7 +225,7 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, GC_ENABLED); } - @Test + @TestTemplate public void testRemoveOrphanFilesWap() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -255,7 +244,7 @@ public void testRemoveOrphanFilesWap() { assertEquals("Should be no orphan files", ImmutableList.of(), output); } - @Test + @TestTemplate public void testInvalidRemoveOrphanFilesCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) @@ -281,7 +270,7 @@ public void testInvalidRemoveOrphanFilesCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testConcurrentRemoveOrphanFiles() throws IOException { if (catalogName.equals("testhadoop")) { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -290,7 +279,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { // correctly while dropping tables through spark_catalog sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder()); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -321,7 +310,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { + "max_concurrent_deletes => %s," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, 4, currentTimestamp); - Assert.assertEquals("Should be orphan files in the data folder", 4, output.size()); + assertThat(output).as("Should be orphan files in the data folder").hasSize(4); // the previous call should have deleted all orphan files List output3 = @@ -331,7 +320,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { + "max_concurrent_deletes => %s," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, 4, currentTimestamp); - Assert.assertEquals("Should be no more orphan files in the data folder", 0, output3.size()); + assertThat(output3).as("Should be no more orphan files in the data folder").hasSize(0); assertEquals( "Should have expected rows", @@ -339,7 +328,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -397,7 +386,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .hasMessage("Invalid last_modified column: StringType is not a timestamp"); } - @Test + @TestTemplate public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { sql( "CREATE TABLE %s (id int, data string) USING iceberg TBLPROPERTIES" @@ -418,9 +407,8 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { sql("DELETE FROM %s WHERE id=1", tableName); Table table = Spark3Util.loadIcebergTable(spark, tableName); - Assert.assertEquals( - "Should have 1 delete manifest", 1, TestHelpers.deleteManifests(table).size()); - Assert.assertEquals("Should have 1 delete file", 1, TestHelpers.deleteFiles(table).size()); + 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().path())); @@ -436,20 +424,22 @@ public void testRemoveOrphanFilesWithDeleteFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assert.assertEquals("Should be no orphan files", 0, output.size()); + assertThat(output).as("Should be no orphan files").hasSize(0); FileSystem localFs = FileSystem.getLocal(new Configuration()); - Assert.assertTrue("Delete manifest should still exist", localFs.exists(deleteManifestPath)); - Assert.assertTrue("Delete file should still exist", localFs.exists(deleteFilePath)); + assertThat(localFs.exists(deleteManifestPath)) + .as("Delete manifest should still exist") + .isTrue(); + assertThat(localFs.exists(deleteFilePath)).as("Delete file should still exist").isTrue(); records.remove(new SimpleRecord(1, "a")); Dataset resultDF = spark.read().format("iceberg").load(tableName); List actualRecords = resultDF.as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Rows must match", records, actualRecords); + assertThat(actualRecords).as("Rows must match").isEqualTo(records); } - @Test + @TestTemplate public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { sql( "CREATE TABLE %s USING iceberg " @@ -505,12 +495,10 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) - .as("stats file length") - .isEqualTo(statisticsFile.fileSizeInBytes()); + assertThat(statsLocation).exists(); + assertThat(statsLocation).hasSize(statisticsFile.fileSizeInBytes()); transaction = table.newTransaction(); transaction.updateStatistics().removeStatistics(statisticsFile.snapshotId()).commit(); @@ -522,14 +510,14 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly(statsLocation.toURI().toString()); - Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation).doesNotExist(); } - @Test + @TestTemplate public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception { sql( "CREATE TABLE %s USING iceberg " @@ -555,11 +543,9 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(new File(partitionStatsLocation)) - .as("partition stats file should exist") - .exists(); + assertThat(new File(partitionStatsLocation)).as("partition stats file should exist").exists(); removePartitionStatsTxn(table, partitionStatisticsFile); @@ -569,11 +555,11 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly("file:" + partitionStatsLocation); - Assertions.assertThat(new File(partitionStatsLocation)) + assertThat(new File(partitionStatsLocation)) .as("partition stats file should be deleted") .doesNotExist(); } @@ -598,7 +584,7 @@ private static void commitPartitionStatsTxn( transaction.commitTransaction(); } - @Test + @TestTemplate public void testRemoveOrphanFilesProcedureWithPrefixMode() throws NoSuchTableException, ParseException, IOException { if (catalogName.equals("testhadoop")) { @@ -606,7 +592,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() } else { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", - tableName, temp.newFolder().toURI().toString()); + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); } Table table = Spark3Util.loadIcebergTable(spark, tableName); String location = table.location(); @@ -663,7 +649,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() + "equal_schemes => map('file1', 'file')," + "file_list_view => '%s')", catalogName, tableIdent, fileListViewName); - Assert.assertEquals(0, orphanFiles.size()); + assertThat(orphanFiles).isEmpty(); // Test with no equal schemes Assertions.assertThatThrownBy( 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 477aa2a1d958..b01438d39dfc 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 @@ -19,11 +19,13 @@ 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.Arrays; import java.util.List; import java.util.Map; import java.util.stream.IntStream; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; @@ -43,51 +45,47 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.internal.SQLConf; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRewriteDataFilesProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteDataFilesProcedure extends ExtensionsTestBase { private static final String QUOTED_SPECIAL_CHARS_TABLE_NAME = "`table:with.special:chars`"; - public TestRewriteDataFilesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @BeforeClass + @BeforeAll public static void setupSpark() { // disable AQE as tests assume that writes generate a particular number of files spark.conf().set(SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), "false"); } - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); } - @Test + @TestTemplate public void testZOrderSortExpression() { List order = ExtendedParser.parseSortOrder(spark, "c1, zorder(c2, c3)"); - Assert.assertEquals("Should parse 2 order fields", 2, order.size()); - Assert.assertEquals( - "First field should be a ref", "c1", ((NamedReference) order.get(0).term()).name()); - Assert.assertTrue("Second field should be zorder", order.get(1).term() instanceof Zorder); + assertThat(order).as("Should parse 2 order fields").hasSize(2); + assertThat(((NamedReference) order.get(0).term()).name()) + .as("First field should be a ref") + .isEqualTo("c1"); + assertThat(order.get(1).term()).as("Second field should be zorder").isInstanceOf(Zorder.class); } - @Test + @TestTemplate public void testRewriteDataFilesInEmptyTable() { createTable(); List output = sql("CALL %s.system.rewrite_data_files('%s')", catalogName, tableIdent); assertEquals("Procedure output must match", ImmutableList.of(row(0, 0, 0L, 0)), output); } - @Test + @TestTemplate public void testRewriteDataFilesOnPartitionTable() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -111,7 +109,7 @@ public void testRewriteDataFilesOnPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesOnNonPartitionTable() { createTable(); // create 10 files under non-partitioned table @@ -135,7 +133,7 @@ public void testRewriteDataFilesOnNonPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithOptions() { createTable(); // create 10 files under non-partitioned table @@ -157,7 +155,7 @@ public void testRewriteDataFilesWithOptions() { assertEquals("Data should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithSortStrategy() { createTable(); // create 10 files under non-partitioned table @@ -185,7 +183,7 @@ public void testRewriteDataFilesWithSortStrategy() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithSortStrategyAndMultipleShufflePartitionsPerFile() { createTable(); insertData(10 /* file count */); @@ -220,7 +218,7 @@ public void testRewriteDataFilesWithSortStrategyAndMultipleShufflePartitionsPerF assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRewriteDataFilesWithZOrder() { createTable(); // create 10 files under non-partitioned table @@ -261,7 +259,7 @@ public void testRewriteDataFilesWithZOrder() { assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() { createTable(); insertData(10 /* file count */); @@ -297,7 +295,7 @@ public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void testRewriteDataFilesWithFilter() { createTable(); // create 10 files under non-partitioned table @@ -325,7 +323,7 @@ public void testRewriteDataFilesWithFilter() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithDeterministicTrueFilter() { createTable(); // create 10 files under non-partitioned table @@ -349,7 +347,7 @@ public void testRewriteDataFilesWithDeterministicTrueFilter() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithDeterministicFalseFilter() { createTable(); // create 10 files under non-partitioned table @@ -368,7 +366,7 @@ public void testRewriteDataFilesWithDeterministicFalseFilter() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithFilterOnPartitionTable() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -396,11 +394,11 @@ public void testRewriteDataFilesWithFilterOnPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithFilterOnOnBucketExpression() { // currently spark session catalog only resolve to v1 functions instead of desired v2 functions // https://github.com/apache/spark/blob/branch-3.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L2070-L2083 - Assume.assumeFalse(catalogName.equals(SparkCatalogConfig.SPARK.catalogName())); + assumeThat(catalogName).isNotEqualTo(SparkCatalogConfig.SPARK.catalogName()); createBucketPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') insertData(10); @@ -428,7 +426,7 @@ public void testRewriteDataFilesWithFilterOnOnBucketExpression() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithInFilterOnPartitionTable() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -456,7 +454,7 @@ public void testRewriteDataFilesWithInFilterOnPartitionTable() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteDataFilesWithAllPossibleFilters() { createPartitionTable(); // create 5 files for each partition (c2 = 'foo' and c2 = 'bar') @@ -523,11 +521,11 @@ public void testRewriteDataFilesWithAllPossibleFilters() { // " where => 'c2 like \"%s\"')", catalogName, tableIdent, "%car%"); } - @Test + @TestTemplate public void testRewriteDataFilesWithPossibleV2Filters() { // currently spark session catalog only resolve to v1 functions instead of desired v2 functions // https://github.com/apache/spark/blob/branch-3.4/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala#L2070-L2083 - Assume.assumeFalse(catalogName.equals(SparkCatalogConfig.SPARK.catalogName())); + assumeThat(catalogName).isNotEqualTo(SparkCatalogConfig.SPARK.catalogName()); SystemFunctionPushDownHelper.createPartitionedTable(spark, tableName, "id"); sql( @@ -556,7 +554,7 @@ public void testRewriteDataFilesWithPossibleV2Filters() { catalogName, tableIdent, catalogName); } - @Test + @TestTemplate public void testRewriteDataFilesWithInvalidInputs() { createTable(); // create 2 files under non-partitioned table @@ -654,7 +652,7 @@ public void testRewriteDataFilesWithInvalidInputs() { "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)"); } - @Test + @TestTemplate public void testInvalidCasesForRewriteDataFiles() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) @@ -680,9 +678,9 @@ public void testInvalidCasesForRewriteDataFiles() { .hasMessage("Cannot handle an empty identifier for parameter 'table'"); } - @Test + @TestTemplate public void testBinPackTableWithSpecialChars() { - Assume.assumeTrue(catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())); + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); TableIdentifier identifier = TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); @@ -712,12 +710,12 @@ public void testBinPackTableWithSpecialChars() { List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); assertEquals("Data after compaction should not change", expectedRecords, actualRecords); - Assert.assertEquals("Table cache must be empty", 0, SparkTableCache.get().size()); + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isEqualTo(0); } - @Test + @TestTemplate public void testSortTableWithSpecialChars() { - Assume.assumeTrue(catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())); + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); TableIdentifier identifier = TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); @@ -752,12 +750,12 @@ public void testSortTableWithSpecialChars() { List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); assertEquals("Data after compaction should not change", expectedRecords, actualRecords); - Assert.assertEquals("Table cache must be empty", 0, SparkTableCache.get().size()); + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isEqualTo(0); } - @Test + @TestTemplate public void testZOrderTableWithSpecialChars() { - Assume.assumeTrue(catalogName.equals(SparkCatalogConfig.HADOOP.catalogName())); + assumeThat(catalogName).isEqualTo(SparkCatalogConfig.HADOOP.catalogName()); TableIdentifier identifier = TableIdentifier.of("default", QUOTED_SPECIAL_CHARS_TABLE_NAME.replaceAll("`", "")); @@ -792,10 +790,10 @@ public void testZOrderTableWithSpecialChars() { List actualRecords = currentData(tableName(QUOTED_SPECIAL_CHARS_TABLE_NAME)); assertEquals("Data after compaction should not change", expectedRecords, actualRecords); - Assert.assertEquals("Table cache must be empty", 0, SparkTableCache.get().size()); + assertThat(SparkTableCache.get().size()).as("Table cache must be empty").isEqualTo(0); } - @Test + @TestTemplate public void testDefaultSortOrder() { createTable(); // add a default sort order for a table @@ -828,7 +826,7 @@ public void testDefaultSortOrder() { assertEquals("Data after compaction should not change", expectedRecords, actualRecords); } - @Test + @TestTemplate public void testRewriteWithUntranslatedOrUnconvertedFilter() { createTable(); Assertions.assertThatThrownBy( 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 b230a6912984..e527eadeb081 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 @@ -19,11 +19,12 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; import java.sql.Date; import java.sql.Timestamp; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -32,30 +33,26 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRewriteManifestsProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteManifestsProcedure extends ExtensionsTestBase { - public TestRewriteManifestsProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTable() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRewriteManifestsInEmptyTable() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); assertEquals("Procedure output must match", ImmutableList.of(row(0, 0)), output); } - @Test + @TestTemplate public void testRewriteLargeManifests() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -64,8 +61,9 @@ public void testRewriteLargeManifests() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); sql("ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", tableName); @@ -74,11 +72,12 @@ public void testRewriteLargeManifests() { table.refresh(); - Assert.assertEquals( - "Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); } - @Test + @TestTemplate public void testRewriteManifestsNoOp() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -87,8 +86,9 @@ public void testRewriteManifestsNoOp() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); List output = sql("CALL %s.system.rewrite_manifests('%s')", catalogName, tableIdent); // should not rewrite any manifests for no-op (output of rewrite is same as before and after) @@ -96,11 +96,12 @@ public void testRewriteManifestsNoOp() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() { withSQLConf( ImmutableMap.of("spark.sql.datetime.java8API.enabled", "true"), @@ -126,8 +127,9 @@ public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); sql( "ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", @@ -139,12 +141,13 @@ public void testRewriteLargeManifestsOnDatePartitionedTableWithJava8APIEnabled() table.refresh(); - Assert.assertEquals( - "Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); }); } - @Test + @TestTemplate public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnabled() { withSQLConf( ImmutableMap.of("spark.sql.datetime.java8API.enabled", "true"), @@ -174,8 +177,9 @@ public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnab Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 1 manifest", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); sql( "ALTER TABLE %s SET TBLPROPERTIES ('commit.manifest.target-size-bytes' '1')", @@ -187,12 +191,13 @@ public void testRewriteLargeManifestsOnTimestampPartitionedTableWithJava8APIEnab table.refresh(); - Assert.assertEquals( - "Must have 4 manifests", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); }); } - @Test + @TestTemplate public void testRewriteSmallManifestsWithSnapshotIdInheritance() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -209,8 +214,9 @@ public void testRewriteSmallManifestsWithSnapshotIdInheritance() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 4 manifest", 4, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 4 manifests") + .hasSize(4); List output = sql("CALL %s.system.rewrite_manifests(table => '%s')", catalogName, tableIdent); @@ -218,11 +224,12 @@ public void testRewriteSmallManifestsWithSnapshotIdInheritance() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testRewriteSmallManifestsWithoutCaching() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -233,8 +240,9 @@ public void testRewriteSmallManifestsWithoutCaching() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 2 manifest") + .hasSize(2); List output = sql( @@ -244,11 +252,12 @@ public void testRewriteSmallManifestsWithoutCaching() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testRewriteManifestsCaseInsensitiveArgs() { sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg PARTITIONED BY (data)", @@ -259,8 +268,9 @@ public void testRewriteManifestsCaseInsensitiveArgs() { Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals( - "Must have 2 manifest", 2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 2 manifests") + .hasSize(2); List output = sql( @@ -270,11 +280,12 @@ public void testRewriteManifestsCaseInsensitiveArgs() { table.refresh(); - Assert.assertEquals( - "Must have 1 manifests", 1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Must have 1 manifest") + .hasSize(1); } - @Test + @TestTemplate public void testInvalidRewriteManifestsCases() { Assertions.assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) @@ -305,7 +316,7 @@ public void testInvalidRewriteManifestsCases() { .hasMessage("Cannot handle an empty identifier for argument table"); } - @Test + @TestTemplate public void testReplacePartitionField() { sql( "CREATE TABLE %s (id int, ts timestamp, day_of_ts date) USING iceberg PARTITIONED BY (day_of_ts)", @@ -339,7 +350,7 @@ public void testReplacePartitionField() { sql("SELECT * FROM %s WHERE ts < current_timestamp() order by 1 asc", tableName)); } - @Test + @TestTemplate public void testWriteManifestWithSpecId() { sql( "CREATE TABLE %s (id int, dt string, hr string) USING iceberg PARTITIONED BY (dt)", 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 4c5082cabbb6..f7329e841800 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 @@ -32,6 +32,7 @@ import java.time.LocalDateTime; import java.util.List; import java.util.Map; +import java.util.UUID; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.ContentFile; @@ -42,6 +43,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PositionDeletesScanTask; import org.apache.iceberg.RowDelta; import org.apache.iceberg.ScanTask; @@ -69,13 +71,10 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.types.StructType; -import org.junit.After; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; -public class TestRewritePositionDeleteFiles extends SparkExtensionsTestBase { +public class TestRewritePositionDeleteFiles extends ExtensionsTestBase { private static final Map CATALOG_PROPS = ImmutableMap.of( @@ -89,8 +88,7 @@ public class TestRewritePositionDeleteFiles extends SparkExtensionsTestBase { private static final int DELETE_FILES_PER_PARTITION = 2; private static final int DELETE_FILE_SIZE = 10; - @Parameterized.Parameters( - name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") + @Parameters(name = "formatVersion = {0}, catalogName = {1}, implementation = {2}, config = {3}") public static Object[][] parameters() { return new Object[][] { { @@ -101,19 +99,12 @@ public static Object[][] parameters() { }; } - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - public TestRewritePositionDeleteFiles( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void cleanup() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testDatePartition() throws Exception { createTable("date"); Date baseDate = Date.valueOf("2023-01-01"); @@ -121,14 +112,14 @@ public void testDatePartition() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testBooleanPartition() throws Exception { createTable("boolean"); insertData(i -> i % 2 == 0, 2); testDanglingDelete(2); } - @Test + @TestTemplate public void testTimestampPartition() throws Exception { createTable("timestamp"); Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); @@ -136,7 +127,7 @@ public void testTimestampPartition() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testTimestampNtz() throws Exception { createTable("timestamp_ntz"); LocalDateTime baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00").toLocalDateTime(); @@ -144,14 +135,14 @@ public void testTimestampNtz() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testBytePartition() throws Exception { createTable("byte"); insertData(i -> i); testDanglingDelete(); } - @Test + @TestTemplate public void testDecimalPartition() throws Exception { createTable("decimal(18, 10)"); BigDecimal baseDecimal = new BigDecimal("1.0"); @@ -159,35 +150,35 @@ public void testDecimalPartition() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testBinaryPartition() throws Exception { createTable("binary"); insertData(i -> java.nio.ByteBuffer.allocate(4).putInt(i).array()); testDanglingDelete(); } - @Test + @TestTemplate public void testCharPartition() throws Exception { createTable("char(10)"); insertData(Object::toString); testDanglingDelete(); } - @Test + @TestTemplate public void testVarcharPartition() throws Exception { createTable("varchar(10)"); insertData(Object::toString); testDanglingDelete(); } - @Test + @TestTemplate public void testIntPartition() throws Exception { createTable("int"); insertData(i -> i); testDanglingDelete(); } - @Test + @TestTemplate public void testDaysPartitionTransform() throws Exception { createTable("timestamp", PARTITION_COL, String.format("days(%s)", PARTITION_COL)); Timestamp baseTimestamp = Timestamp.valueOf("2023-01-01 15:30:00"); @@ -195,14 +186,14 @@ public void testDaysPartitionTransform() throws Exception { testDanglingDelete(); } - @Test + @TestTemplate public void testNullTransform() throws Exception { createTable("int"); insertData(i -> i == 0 ? null : 1, 2); testDanglingDelete(2); } - @Test + @TestTemplate public void testPartitionColWithDot() throws Exception { String partitionColWithDot = "`partition.col`"; createTable("int", partitionColWithDot, partitionColWithDot); @@ -318,7 +309,8 @@ private void writePosDeletesForFiles(Table table, List files) throws I counter++; if (counter == deleteFileSize) { // Dump to file and reset variables - OutputFile output = Files.localOutput(temp.newFile()); + OutputFile output = + Files.localOutput(temp.resolve(UUID.randomUUID().toString()).toFile()); deleteFiles.add(writeDeleteFile(table, output, partition, deletes)); counter = 0; deletes.clear(); 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 5dde5d698ee4..585db39a3c44 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 @@ -20,26 +20,24 @@ import static org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP; import static org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP; +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.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Encoders; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestRewritePositionDeleteFilesProcedure extends SparkExtensionsTestBase { - - public TestRewritePositionDeleteFilesProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewritePositionDeleteFilesProcedure extends ExtensionsTestBase { private void createTable() throws Exception { createTable(false); @@ -79,12 +77,12 @@ private void createTable(boolean partitioned) throws Exception { .append(); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testExpireDeleteFilesAll() throws Exception { createTable(); @@ -92,7 +90,7 @@ public void testExpireDeleteFilesAll() throws Exception { sql("DELETE FROM %s WHERE id=2", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals(2, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(2); List output = sql( @@ -114,10 +112,10 @@ public void testExpireDeleteFilesAll() throws Exception { Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), output); - Assert.assertEquals(1, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(1); } - @Test + @TestTemplate public void testExpireDeleteFilesNoOption() throws Exception { createTable(); @@ -128,7 +126,7 @@ public void testExpireDeleteFilesNoOption() throws Exception { sql("DELETE FROM %s WHERE id=5", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals(5, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(5); List output = sql( @@ -148,7 +146,7 @@ public void testExpireDeleteFilesNoOption() throws Exception { output); } - @Test + @TestTemplate public void testExpireDeleteFilesFilter() throws Exception { createTable(true); @@ -160,7 +158,7 @@ public void testExpireDeleteFilesFilter() throws Exception { sql("DELETE FROM %s WHERE id = 3 and data='h'", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertEquals(6, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(6); List output = sql( @@ -184,29 +182,30 @@ public void testExpireDeleteFilesFilter() throws Exception { Long.valueOf(snapshotSummary.get(ADDED_FILE_SIZE_PROP)))), output); - Assert.assertEquals(4, TestHelpers.deleteFiles(table).size()); + assertThat(TestHelpers.deleteFiles(table)).hasSize(4); } - @Test + @TestTemplate public void testInvalidOption() throws Exception { createTable(); - Assert.assertThrows( - "Cannot use options [foo], they are not supported by the action or the rewriter BIN-PACK", - IllegalArgumentException.class, - () -> - sql( - "CALL %s.system.rewrite_position_delete_files(" - + "table => '%s'," - + "options => map(" - + "'foo', 'bar'))", - catalogName, tableIdent)); + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + + "options => map(" + + "'foo', 'bar'))", + catalogName, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot use options [foo], they are not supported by the action or the rewriter BIN-PACK"); } - @Test + @TestTemplate public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(encode(data, \"utf-8\"), 2) = \"fo\"')", @@ -214,7 +213,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(data, 2) = \"fo\"')", 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 1b4cd2b4a3f6..ba833bcee35e 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 @@ -18,8 +18,10 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -29,23 +31,18 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Test; - -public class TestRollbackToSnapshotProcedure extends SparkExtensionsTestBase { - - public TestRollbackToSnapshotProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - @After +@ExtendWith(ParameterizedTestExtension.class) +public class TestRollbackToSnapshotProcedure extends ExtensionsTestBase { + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRollbackToSnapshotUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -80,7 +77,7 @@ public void testRollbackToSnapshotUsingPositionalArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -115,7 +112,7 @@ public void testRollbackToSnapshotUsingNamedArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToSnapshotRefreshesRelationCache() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -155,7 +152,7 @@ public void testRollbackToSnapshotRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testRollbackToSnapshotWithQuotedIdentifiers() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -200,9 +197,9 @@ public void testRollbackToSnapshotWithQuotedIdentifiers() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToSnapshotWithoutExplicitCatalog() { - Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -236,7 +233,7 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -246,7 +243,7 @@ public void testRollbackToInvalidSnapshot() { .hasMessage("Cannot roll back to unknown snapshot id: -1"); } - @Test + @TestTemplate public void testInvalidRollbackToSnapshotCases() { Assertions.assertThatThrownBy( () -> 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 80af6e7f5f4c..d9366c08dea3 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 @@ -18,9 +18,10 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.time.LocalDateTime; import java.util.List; -import java.util.Map; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -29,23 +30,17 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Test; - -public class TestRollbackToTimestampProcedure extends SparkExtensionsTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; - public TestRollbackToTimestampProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +public class TestRollbackToTimestampProcedure extends ExtensionsTestBase { - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testRollbackToTimestampUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -83,7 +78,7 @@ public void testRollbackToTimestampUsingPositionalArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToTimestampUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -121,7 +116,7 @@ public void testRollbackToTimestampUsingNamedArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToTimestampRefreshesRelationCache() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -164,7 +159,7 @@ public void testRollbackToTimestampRefreshesRelationCache() { sql("UNCACHE TABLE tmp"); } - @Test + @TestTemplate public void testRollbackToTimestampWithQuotedIdentifiers() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -210,9 +205,9 @@ public void testRollbackToTimestampWithQuotedIdentifiers() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testRollbackToTimestampWithoutExplicitCatalog() { - Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -251,7 +246,7 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; 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 e894ba4ff0ae..52278026f829 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 @@ -19,9 +19,10 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -30,23 +31,19 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assume; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestSetCurrentSnapshotProcedure extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestSetCurrentSnapshotProcedure extends ExtensionsTestBase { - public TestSetCurrentSnapshotProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testSetCurrentSnapshotUsingPositionalArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -81,7 +78,7 @@ public void testSetCurrentSnapshotUsingPositionalArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSetCurrentSnapshotUsingNamedArgs() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -116,7 +113,7 @@ public void testSetCurrentSnapshotUsingNamedArgs() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSetCurrentSnapshotWap() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, WRITE_AUDIT_PUBLISH_ENABLED); @@ -149,9 +146,9 @@ public void testSetCurrentSnapshotWap() { sql("SELECT * FROM %s", tableName)); } - @Test + @TestTemplate public void tesSetCurrentSnapshotWithoutExplicitCatalog() { - Assume.assumeTrue("Working only with the session catalog", "spark_catalog".equals(catalogName)); + assumeThat(catalogName).as("Working only with the session catalog").isEqualTo("spark_catalog"); sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -185,7 +182,7 @@ public void tesSetCurrentSnapshotWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSetCurrentSnapshotToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); @@ -195,7 +192,7 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { .hasMessage("Cannot roll back to unknown snapshot id: -1"); } - @Test + @TestTemplate public void testInvalidRollbackToSnapshotCases() { Assertions.assertThatThrownBy( () -> @@ -248,7 +245,7 @@ public void testInvalidRollbackToSnapshotCases() { .hasMessage("Either snapshot_id or ref must be provided, not both"); } - @Test + @TestTemplate public void testSetCurrentSnapshotToRef() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); sql("INSERT INTO TABLE %s VALUES (1, 'a')", 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 367d95d8f619..69bc35e9ea33 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 @@ -18,40 +18,36 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.io.IOException; +import java.nio.file.Files; import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotTableProcedure extends ExtensionsTestBase { private static final String sourceName = "spark_catalog.default.source"; // Currently we can only Snapshot only out of the Spark Session Catalog - public TestSnapshotTableProcedure( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s PURGE", sourceName); } - @Test + @TestTemplate public void testSnapshot() throws IOException { - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -59,11 +55,13 @@ public void testSnapshot() throws IOException { Object result = scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); String tableLocation = createdTable.location(); - Assert.assertNotEquals("Table should not have the original location", location, tableLocation); + assertThat(tableLocation) + .as("Table should not have the original location") + .isNotEqualTo(location); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -73,9 +71,9 @@ public void testSnapshot() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSnapshotWithProperties() throws IOException { - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -85,15 +83,17 @@ public void testSnapshotWithProperties() throws IOException { "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('foo','bar'))", catalogName, sourceName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); Table createdTable = validationCatalog.loadTable(tableIdent); String tableLocation = createdTable.location(); - Assert.assertNotEquals("Table should not have the original location", location, tableLocation); + assertThat(tableLocation) + .as("Table should not have the original location") + .isNotEqualTo(location); Map props = createdTable.properties(); - Assert.assertEquals("Should have extra property set", "bar", props.get("foo")); + assertThat(props).as("Should have extra property set").containsEntry("foo", "bar"); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -103,13 +103,13 @@ public void testSnapshotWithProperties() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testSnapshotWithAlternateLocation() throws IOException { - Assume.assumeTrue( - "No Snapshoting with Alternate locations with Hadoop Catalogs", - !catalogName.contains("hadoop")); - String location = temp.newFolder().toString(); - String snapshotLocation = temp.newFolder().toString(); + assumeThat(catalogName) + .as("No Snapshoting with Alternate locations with Hadoop Catalogs") + .doesNotContain("hadoop"); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + String snapshotLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -120,11 +120,12 @@ public void testSnapshotWithAlternateLocation() throws IOException { catalogName, sourceName, tableName, snapshotLocation) .get(0); - Assert.assertEquals("Should have added one file", 1L, result[0]); + assertThat(result[0]).as("Should have added one file").isEqualTo(1L); String storageLocation = validationCatalog.loadTable(tableIdent).location(); - Assert.assertEquals( - "Snapshot should be made at specified location", snapshotLocation, storageLocation); + assertThat(storageLocation) + .as("Snapshot should be made at specified location") + .isEqualTo(snapshotLocation); sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); @@ -134,9 +135,9 @@ public void testSnapshotWithAlternateLocation() throws IOException { sql("SELECT * FROM %s ORDER BY id", tableName)); } - @Test + @TestTemplate public void testDropTable() throws IOException { - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -144,7 +145,7 @@ public void testDropTable() throws IOException { Object result = scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( "Should have expected rows", @@ -159,9 +160,9 @@ public void testDropTable() throws IOException { sql("SELECT * FROM %s", sourceName)); } - @Test + @TestTemplate public void testSnapshotWithConflictingProps() throws IOException { - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); @@ -174,7 +175,7 @@ public void testSnapshotWithConflictingProps() throws IOException { + "table => '%s'," + "properties => map('%s', 'true', 'snapshot', 'false'))", catalogName, sourceName, tableName, TableProperties.GC_ENABLED); - Assert.assertEquals("Should have added one file", 1L, result); + assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( "Should have expected rows", @@ -183,14 +184,15 @@ public void testSnapshotWithConflictingProps() throws IOException { Table table = validationCatalog.loadTable(tableIdent); Map props = table.properties(); - Assert.assertEquals("Should override user value", "true", props.get("snapshot")); - Assert.assertEquals( - "Should override user value", "false", props.get(TableProperties.GC_ENABLED)); + assertThat(props).as("Should override user value").containsEntry("snapshot", "true"); + assertThat(props) + .as("Should override user value") + .containsEntry(TableProperties.GC_ENABLED, "false"); } - @Test + @TestTemplate public void testInvalidSnapshotsCases() throws IOException { - String location = temp.newFolder().toString(); + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 256e654b7775..a40e66f5bf2b 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -81,8 +83,8 @@ public void removeTables() { } @TestTemplate - public void testBatchAppend() throws Exception { - String dataLocation = temp.toFile().toString(); + public void testBatchAppend() throws IOException { + String dataLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id INT, data STRING) " From 38a6de9629ee82cec05cb64baeea763e3d8017b3 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 25 Feb 2024 19:22:43 +0100 Subject: [PATCH 0071/1019] Spark: Fail if temp functions are used in views (#9675) --- .../analysis/RewriteViewCommands.scala | 49 ++++-- .../iceberg/spark/extensions/TestViews.java | 155 ++++++++++++++++-- 2 files changed, 176 insertions(+), 28 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index ef67f1ef2263..ff7f8fe494c9 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -33,9 +33,11 @@ import org.apache.spark.sql.catalyst.plans.logical.views.DropIcebergView import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_FUNCTION import org.apache.spark.sql.connector.catalog.CatalogManager import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.LookupCatalog +import scala.collection.mutable /** * ResolveSessionCatalog exits early for some v2 View commands, @@ -43,6 +45,8 @@ import org.apache.spark.sql.connector.catalog.LookupCatalog */ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { @@ -83,6 +87,13 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi catalogManager.v1SessionCatalog.isTempView(nameParts) } + private def isTempFunction(nameParts: Seq[String]): Boolean = { + if (nameParts.size > 1) { + return false + } + catalogManager.v1SessionCatalog.isTemporaryFunction(nameParts.asFunctionIdentifier) + } + private object ResolvedIdent { def unapply(unresolved: UnresolvedIdentifier): Option[ResolvedIdentifier] = unresolved match { case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) => @@ -102,20 +113,20 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi private def verifyTemporaryObjectsDontExist( name: Identifier, child: LogicalPlan): Unit = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val tempViews = collectTemporaryViews(child) - tempViews.foreach { nameParts => - throw new AnalysisException( - errorClass = "INVALID_TEMP_OBJ_REFERENCE", - messageParameters = Map( - "obj" -> "VIEW", - "objName" -> name.name(), - "tempObj" -> "VIEW", - "tempObjName" -> nameParts.quoted)) + if (tempViews.nonEmpty) { + throw invalidRefToTempObject(name, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") } - // TODO: check for temp function names + val tempFunctions = collectTemporaryFunctions(child) + if (tempFunctions.nonEmpty) { + throw invalidRefToTempObject(name, tempFunctions.mkString("[", ", ", "]"), "function") + } + } + + private def invalidRefToTempObject(name: Identifier, tempObjectNames: String, tempObjectType: String) = { + new AnalysisException(String.format("Cannot create view %s that references temporary %s: %s", + name, tempObjectType, tempObjectNames)) } /** @@ -149,4 +160,20 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi None } } + + /** + * Collect the names of all temporary functions. + */ + private def collectTemporaryFunctions(child: LogicalPlan): Seq[String] = { + val tempFunctions = new mutable.HashSet[String]() + child.resolveExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { + case f @ UnresolvedFunction(nameParts, _, _, _, _) if isTempFunction(nameParts) => + tempFunctions += nameParts.head + f + case e: SubqueryExpression => + tempFunctions ++= collectTemporaryFunctions(e.plan) + e + } + tempFunctions.toSeq + } } 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 267628cd06e6..6ffcb82679c6 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 @@ -477,6 +477,39 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException .hasMessageContaining("cannot be found"); } + @Test + public void readFromViewReferencingTempFunction() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewReferencingTempFunction"); + String functionName = "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'", + functionName); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); + + // it wouldn't be possible to reference a TEMP FUNCTION if the view had been created via SQL, + // but this can't be prevented when using the API directly + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThat(sql(sql)).hasSize(1).containsExactly(row(5.5)); + + // reading from a view that references a TEMP FUNCTION shouldn't be possible + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The function") + .hasMessageContaining(functionName) + .hasMessageContaining("cannot be found"); + } + @Test public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); @@ -947,9 +980,9 @@ public void createViewReferencingTempView() throws NoSuchTableException { assertThatThrownBy( () -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewReferencingTempView, tempView)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot create the persistent object") - .hasMessageContaining(viewReferencingTempView) - .hasMessageContaining("of the type VIEW because it references to the temporary object") + .hasMessageContaining( + String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -970,10 +1003,59 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { "CREATE VIEW %s AS SELECT id FROM global_temp.%s", viewReferencingTempView, globalTempView)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot create the persistent object") - .hasMessageContaining(viewReferencingTempView) - .hasMessageContaining("of the type VIEW because it references to the temporary object") - .hasMessageContaining(globalTempView); + .hasMessageContaining( + String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); + } + + @Test + public void createViewReferencingTempFunction() { + String viewName = viewName("viewReferencingTemporaryFunction"); + String functionName = "test_avg_func"; + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + // creating a view that references a TEMP FUNCTION shouldn't be possible + assertThatThrownBy( + () -> sql("CREATE VIEW %s AS SELECT %s(id) FROM %s", viewName, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + + @Test + public void createViewReferencingQualifiedTempFunction() { + String viewName = viewName("viewReferencingTemporaryFunction"); + String functionName = "test_avg_func_qualified"; + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + // TEMP Function can't be referenced using catalog.schema.name + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT %s.%s.%s(id) FROM %s", + viewName, catalogName, NAMESPACE, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve function") + .hasMessageContaining( + String.format("`%s`.`%s`.`%s`", catalogName, NAMESPACE, functionName)); + + // TEMP Function can't be referenced using schema.name + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT %s.%s(id) FROM %s", + viewName, NAMESPACE, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve function") + .hasMessageContaining(String.format("`%s`.`%s`", NAMESPACE, functionName)); } @Test @@ -1118,12 +1200,32 @@ public void createViewWithCTEReferencingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot create the persistent object") - .hasMessageContaining(viewName) - .hasMessageContaining("of the type VIEW because it references to the temporary object") + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempViewInCTE); } + @Test + public void createViewWithCTEReferencingTempFunction() { + String viewName = "viewWithCTEReferencingTempFunction"; + String functionName = "avg_function_in_cte"; + String sql = + String.format( + "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " + + "SELECT avg, count(1) AS count FROM avg_data GROUP BY max", + functionName, tableName); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + @Test public void createViewWithNonExistingQueryColumn() { assertThatThrownBy( @@ -1147,9 +1249,9 @@ public void createViewWithSubqueryExpressionUsingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create the persistent object %s", viewName)) - .hasMessageContaining( - String.format("because it references to the temporary object %s", tempView)); + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(tempView); } @Test @@ -1167,10 +1269,29 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create the persistent object %s", viewName)) - .hasMessageContaining( - String.format( - "because it references to the temporary object global_temp.%s", globalTempView)); + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); + } + + @Test + public void createViewWithSubqueryExpressionUsingTempFunction() { + String viewName = viewName("viewWithSubqueryExpression"); + String functionName = "avg_function_in_subquery"; + String sql = + String.format( + "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", + tableName, functionName, tableName); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); } @Test From 1bcab39a4bc75c5c8c1f98fdedf09ec8ac02dde0 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 26 Feb 2024 10:02:35 +0100 Subject: [PATCH 0072/1019] Spark 3.4: Fail if temp functions are used in views (#9809) --- .../analysis/RewriteViewCommands.scala | 51 ++++-- .../iceberg/spark/extensions/TestViews.java | 155 ++++++++++++++++-- 2 files changed, 178 insertions(+), 28 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 151622191fad..e48357971211 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -33,11 +33,13 @@ import org.apache.spark.sql.catalyst.plans.logical.views.DropIcebergView import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_FUNCTION 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.LookupCatalog import org.apache.spark.sql.connector.catalog.ViewCatalog +import scala.collection.mutable /** * ResolveSessionCatalog exits early for some v2 View commands, @@ -45,6 +47,8 @@ import org.apache.spark.sql.connector.catalog.ViewCatalog */ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] with LookupCatalog { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { @@ -85,6 +89,13 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi catalogManager.v1SessionCatalog.isTempView(nameParts) } + private def isTempFunction(nameParts: Seq[String]): Boolean = { + if (nameParts.size > 1) { + return false + } + catalogManager.v1SessionCatalog.isTemporaryFunction(nameParts.asFunctionIdentifier) + } + object ResolvedIdent { def unapply(unresolved: UnresolvedIdentifier): Option[ResolvedIdentifier] = unresolved match { case UnresolvedIdentifier(nameParts, true) if isTempView(nameParts) => @@ -104,18 +115,20 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi private def verifyTemporaryObjectsDontExist( name: Identifier, child: LogicalPlan): Unit = { - import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ - val tempViews = collectTemporaryViews(child) - tempViews.foreach { nameParts => - throw new AnalysisException(String.format("Cannot create the persistent object %s" + - " of the type VIEW because it references to the temporary object %s of" + - " the type VIEW. Please make the temporary object %s" + - " persistent, or make the persistent object %s temporary", - name.name(), nameParts.quoted, nameParts.quoted, name.name())) - }; - - // TODO: check for temp function names + if (tempViews.nonEmpty) { + throw invalidRefToTempObject(name, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") + } + + val tempFunctions = collectTemporaryFunctions(child) + if (tempFunctions.nonEmpty) { + throw invalidRefToTempObject(name, tempFunctions.mkString("[", ", ", "]"), "function") + } + } + + private def invalidRefToTempObject(name: Identifier, tempObjectNames: String, tempObjectType: String) = { + new AnalysisException(String.format("Cannot create view %s that references temporary %s: %s", + name, tempObjectType, tempObjectNames)) } /** @@ -149,4 +162,20 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi None } } + + /** + * Collect the names of all temporary functions. + */ + private def collectTemporaryFunctions(child: LogicalPlan): Seq[String] = { + val tempFunctions = new mutable.HashSet[String]() + child.resolveExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { + case f @ UnresolvedFunction(nameParts, _, _, _, _) if isTempFunction(nameParts) => + tempFunctions += nameParts.head + f + case e: SubqueryExpression => + tempFunctions ++= collectTemporaryFunctions(e.plan) + e + } + tempFunctions.toSeq + } } 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 ea9ccc9133fe..26bcb03ba101 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 @@ -477,6 +477,39 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException .hasMessageContaining("cannot be found"); } + @Test + public void readFromViewReferencingTempFunction() throws NoSuchTableException { + insertRows(10); + String viewName = viewName("viewReferencingTempFunction"); + String functionName = "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'", + functionName); + + ViewCatalog viewCatalog = viewCatalog(); + Schema schema = tableCatalog().loadTable(TableIdentifier.of(NAMESPACE, tableName)).schema(); + + // it wouldn't be possible to reference a TEMP FUNCTION if the view had been created via SQL, + // but this can't be prevented when using the API directly + viewCatalog + .buildView(TableIdentifier.of(NAMESPACE, viewName)) + .withQuery("spark", sql) + .withDefaultNamespace(NAMESPACE) + .withDefaultCatalog(catalogName) + .withSchema(schema) + .create(); + + assertThat(sql(sql)).hasSize(1).containsExactly(row(5.5)); + + // reading from a view that references a TEMP FUNCTION shouldn't be possible + assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("The function") + .hasMessageContaining(functionName) + .hasMessageContaining("cannot be found"); + } + @Test public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); @@ -947,9 +980,9 @@ public void createViewReferencingTempView() throws NoSuchTableException { assertThatThrownBy( () -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewReferencingTempView, tempView)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot create the persistent object") - .hasMessageContaining(viewReferencingTempView) - .hasMessageContaining("of the type VIEW because it references to the temporary object") + .hasMessageContaining( + String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -970,10 +1003,59 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { "CREATE VIEW %s AS SELECT id FROM global_temp.%s", viewReferencingTempView, globalTempView)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot create the persistent object") - .hasMessageContaining(viewReferencingTempView) - .hasMessageContaining("of the type VIEW because it references to the temporary object") - .hasMessageContaining(globalTempView); + .hasMessageContaining( + String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); + } + + @Test + public void createViewReferencingTempFunction() { + String viewName = viewName("viewReferencingTemporaryFunction"); + String functionName = "test_avg_func"; + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + // creating a view that references a TEMP FUNCTION shouldn't be possible + assertThatThrownBy( + () -> sql("CREATE VIEW %s AS SELECT %s(id) FROM %s", viewName, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + + @Test + public void createViewReferencingQualifiedTempFunction() { + String viewName = viewName("viewReferencingTemporaryFunction"); + String functionName = "test_avg_func_qualified"; + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + // TEMP Function can't be referenced using catalog.schema.name + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT %s.%s.%s(id) FROM %s", + viewName, catalogName, NAMESPACE, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve function") + .hasMessageContaining( + String.format("`%s`.`%s`.`%s`", catalogName, NAMESPACE, functionName)); + + // TEMP Function can't be referenced using schema.name + assertThatThrownBy( + () -> + sql( + "CREATE VIEW %s AS SELECT %s.%s(id) FROM %s", + viewName, NAMESPACE, functionName, tableName)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve function") + .hasMessageContaining(String.format("`%s`.`%s`", NAMESPACE, functionName)); } @Test @@ -1118,12 +1200,32 @@ public void createViewWithCTEReferencingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("Cannot create the persistent object") - .hasMessageContaining(viewName) - .hasMessageContaining("of the type VIEW because it references to the temporary object") + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempViewInCTE); } + @Test + public void createViewWithCTEReferencingTempFunction() { + String viewName = "viewWithCTEReferencingTempFunction"; + String functionName = "avg_function_in_cte"; + String sql = + String.format( + "WITH avg_data AS (SELECT %s(id) as avg FROM %s) " + + "SELECT avg, count(1) AS count FROM avg_data GROUP BY max", + functionName, tableName); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); + } + @Test public void createViewWithNonExistingQueryColumn() { assertThatThrownBy( @@ -1147,9 +1249,9 @@ public void createViewWithSubqueryExpressionUsingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create the persistent object %s", viewName)) - .hasMessageContaining( - String.format("because it references to the temporary object %s", tempView)); + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(tempView); } @Test @@ -1167,10 +1269,29 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create the persistent object %s", viewName)) - .hasMessageContaining( - String.format( - "because it references to the temporary object global_temp.%s", globalTempView)); + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary view:") + .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); + } + + @Test + public void createViewWithSubqueryExpressionUsingTempFunction() { + String viewName = viewName("viewWithSubqueryExpression"); + String functionName = "avg_function_in_subquery"; + String sql = + String.format( + "SELECT * FROM %s WHERE id < (SELECT %s(id) FROM %s)", + tableName, functionName, tableName); + + sql( + "CREATE TEMPORARY FUNCTION %s AS 'org.apache.hadoop.hive.ql.udf.generic.GenericUDAFAverage'", + functionName); + + assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining("that references temporary function:") + .hasMessageContaining(functionName); } @Test From ec5710a1fc8d6848927a61287da4b4dd7384ecbf Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 26 Feb 2024 10:54:18 +0100 Subject: [PATCH 0073/1019] Spark: Include catalog name in view errors (#9807) --- .../sql/catalyst/analysis/CheckViews.scala | 11 ++++--- .../analysis/RewriteViewCommands.scala | 18 ++++++----- .../iceberg/spark/extensions/TestViews.java | 30 ++++++++++++------- 3 files changed, 35 insertions(+), 24 deletions(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 00799a769c46..9a2dee997a10 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans.logical.AlterViewAs import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View -import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.SchemaUtils @@ -33,9 +32,9 @@ object CheckViews extends (LogicalPlan => Unit) { override def apply(plan: LogicalPlan): Unit = { plan foreach { - case CreateIcebergView(ResolvedIdentifier(_: ViewCatalog, ident), _, query, columnAliases, _, + case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, _, _, _, _, _, _) => - verifyColumnCount(ident, columnAliases, query) + verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) case AlterViewAs(ResolvedV2View(_, _), _, _) => @@ -45,20 +44,20 @@ object CheckViews extends (LogicalPlan => Unit) { } } - private def verifyColumnCount(ident: Identifier, columns: Seq[String], query: LogicalPlan): Unit = { + private def verifyColumnCount(ident: ResolvedIdentifier, columns: Seq[String], query: LogicalPlan): Unit = { if (columns.nonEmpty) { if (columns.length > query.output.length) { throw new AnalysisException( errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.NOT_ENOUGH_DATA_COLUMNS", messageParameters = Map( - "viewName" -> ident.toString, + "viewName" -> String.format("%s.%s", ident.catalog.name(), ident.identifier), "viewColumns" -> columns.mkString(", "), "dataColumns" -> query.output.map(c => c.name).mkString(", "))) } else if (columns.length < query.output.length) { throw new AnalysisException( errorClass = "CREATE_VIEW_COLUMN_ARITY_MISMATCH.TOO_MANY_DATA_COLUMNS", messageParameters = Map( - "viewName" -> ident.toString, + "viewName" -> String.format("%s.%s", ident.catalog.name(), ident.identifier), "viewColumns" -> columns.mkString(", "), "dataColumns" -> query.output.map(c => c.name).mkString(", "))) } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index ff7f8fe494c9..22c0bb330179 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -35,7 +35,6 @@ import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_FUNCTION import org.apache.spark.sql.connector.catalog.CatalogManager -import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.LookupCatalog import scala.collection.mutable @@ -56,7 +55,7 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case CreateView(ResolvedIdent(resolved), userSpecifiedColumns, comment, properties, Some(queryText), query, allowExisting, replace) => val q = CTESubstitution.apply(query) - verifyTemporaryObjectsDontExist(resolved.identifier, q) + verifyTemporaryObjectsDontExist(resolved, q) CreateIcebergView(child = resolved, queryText = queryText, query = q, @@ -111,22 +110,25 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi * Permanent views are not allowed to reference temp objects */ private def verifyTemporaryObjectsDontExist( - name: Identifier, + identifier: ResolvedIdentifier, child: LogicalPlan): Unit = { val tempViews = collectTemporaryViews(child) if (tempViews.nonEmpty) { - throw invalidRefToTempObject(name, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") + throw invalidRefToTempObject(identifier, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") } val tempFunctions = collectTemporaryFunctions(child) if (tempFunctions.nonEmpty) { - throw invalidRefToTempObject(name, tempFunctions.mkString("[", ", ", "]"), "function") + throw invalidRefToTempObject(identifier, tempFunctions.mkString("[", ", ", "]"), "function") } } - private def invalidRefToTempObject(name: Identifier, tempObjectNames: String, tempObjectType: String) = { - new AnalysisException(String.format("Cannot create view %s that references temporary %s: %s", - name, tempObjectType, tempObjectNames)) + private def invalidRefToTempObject( + identifier: ResolvedIdentifier, + tempObjectNames: String, + tempObjectType: String) = { + new AnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s", + identifier.catalog.name(), identifier.identifier, tempObjectType, tempObjectNames)) } /** 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 6ffcb82679c6..f1e6580a3e19 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 @@ -981,7 +981,8 @@ public void createViewReferencingTempView() throws NoSuchTableException { () -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewReferencingTempView, tempView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + String.format( + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -1004,7 +1005,8 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { viewReferencingTempView, globalTempView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + String.format( + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); } @@ -1022,7 +1024,8 @@ public void createViewReferencingTempFunction() { assertThatThrownBy( () -> sql("CREATE VIEW %s AS SELECT %s(id) FROM %s", viewName, functionName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1073,7 +1076,8 @@ public void createViewWithMismatchedColumnCounts() { assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("not enough data columns") .hasMessageContaining("View columns: id, data") .hasMessageContaining("Data columns: id"); @@ -1081,7 +1085,8 @@ public void createViewWithMismatchedColumnCounts() { assertThatThrownBy( () -> sql("CREATE VIEW %s (id) AS SELECT id, data FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("too many data columns") .hasMessageContaining("View columns: id") .hasMessageContaining("Data columns: id, data"); @@ -1200,7 +1205,8 @@ public void createViewWithCTEReferencingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempViewInCTE); } @@ -1221,7 +1227,8 @@ public void createViewWithCTEReferencingTempFunction() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1249,7 +1256,8 @@ public void createViewWithSubqueryExpressionUsingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -1269,7 +1277,8 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); } @@ -1289,7 +1298,8 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } From f6892bb7ecbdf0b4e3e43f10a13d9760f4b1a14c Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 26 Feb 2024 19:07:18 +0800 Subject: [PATCH 0074/1019] Build: Ignore major version update in dependabot (#9806) --- .github/dependabot.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index 86d4e469ea5b..ab81fe8de75f 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -36,4 +36,7 @@ updates: interval: "weekly" day: "sunday" open-pull-requests-limit: 5 + - ignore: + dependency-name: "*" + update-types: ["version-update:semver-major"] From 37a9eea1f70a7fe04fbae59826a88716a0b77bd8 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 26 Feb 2024 12:38:07 +0100 Subject: [PATCH 0075/1019] Docs: Sync contributing page / refer to website for contributing (#9776) --- CONTRIBUTING.md | 336 +--------------------------------------- site/docs/contribute.md | 151 ++++++++++++------ 2 files changed, 103 insertions(+), 384 deletions(-) diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index e160aaec17c0..49f803cbf3ad 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -19,337 +19,5 @@ # Contributing -In this document, you will find some guidelines on contributing to Apache Iceberg. Please keep in mind that none of -these are hard rules and they're meant as a collection of helpful suggestions to make contributing as seamless of an -experience as possible. - -If you are thinking of contributing but first would like to discuss the change you wish to make, we welcome you to -head over to the [Community](https://iceberg.apache.org/community/) page on the official Iceberg documentation site -to find a number of ways to connect with the community, including slack and our mailing lists. Of course, always feel -free to just open a [new issue](https://github.com/apache/iceberg/issues/new) in the GitHub repo. - -## Pull Request Process - -Pull requests are the preferred mechanism for contributing to Iceberg -* PRs are automatically labeled based on the content by our github-actions labeling action -* It's helpful to include a prefix in the summary that provides context to PR reviewers, such as `Build:`, `Docs:`, `Spark:`, `Flink:`, `Core:`, `API:` -* If a PR is related to an issue, adding `Closes #1234` in the PR description will automatically close the issue and helps keep the project clean -* If a PR is posted for visibility and isn't necessarily ready for review or merging, be sure to convert the PR to a draft - -## Building the Project Locally - -Please refer to the [Building](https://github.com/apache/iceberg#building) section of the main readme for instructions -on how to build iceberg locally. - -## Website and Documentation Updates - -The [Iceberg website](https://iceberg.apache.org/) and documentations are hosted in a different repository [iceberg-docs](https://github.com/apache/iceberg-docs). -Read the repository README for contribution guidelines for the website and documentation. - -## Semantic Versioning - -Apache Iceberg leverages [semantic versioning](https://semver.org/#semantic-versioning-200) to ensure compatibility -for developers and users of the iceberg libraries as APIs and implementations evolve. The requirements and -guarantees provided depend on the subproject as described below: - -### Major Version Deprecations Required - -__Modules__ -`iceberg-api` - -The API subproject is the main interface for developers and users of the Iceberg API and therefore has the strongest -guarantees. Evolution of the interfaces in this subproject are enforced by [Revapi](https://revapi.org/) and require -explicit acknowledgement of API changes. - -All public interfaces and classes require one major version for deprecation cycle. Any backward incompatible changes -should be annotated as `@Deprecated` and removed for the next major release. Backward compatible changes are allowed -within major versions. - -### Minor Version Deprecations Required - -__Modules__ -`iceberg-common` -`iceberg-core` -`iceberg-data` -`iceberg-orc` -`iceberg-parquet` - -Changes to public interfaces and classes in the subprojects listed above require a deprecation cycle of one minor -release. These projects contain common and internal code used by other projects and can evolve within a major release. -Minor release deprecation will provide other subprojects and external projects notice and opportunity to transition -to new implementations. - -### Minor Version Deprecations Discretionary - -__modules__ (All modules not referenced above) - -Other modules are less likely to be extended directly and modifications should make a good faith effort to follow a -minor version deprecation cycle. If there are significant structural or design changes that result in deprecations -being difficult to orchestrate, it is up to the committers to decide if deprecation is necessary. - -## Deprecation Notices - -All interfaces, classes, and methods targeted for deprecation must include the following: - -1. `@Deprecated` annotation on the appropriate element -2. `@depreceted` javadoc comment including: the version for removal, the appropriate alternative for usage -3. Replacement of existing code paths that use the deprecated behavior - -Example: - -```java - /** - * Set the sequence number for this manifest entry. - * - * @param sequenceNumber a sequence number - * @deprecated since 1.0.0, will be removed in 1.1.0; use dataSequenceNumber() instead. - */ - @Deprecated - void sequenceNumber(long sequenceNumber); -``` - -## Adding new functionality without breaking APIs -Ideally, we'd want to add new functionality without breaking existing APIs, especially within the scope of the API modules that are being checked by [Revapi](https://revapi.org/). - -Let's assume we'd want to add a `createBranch(String name)` method to the `ManageSnapshots` API. - -The most straight-forward way would be to add the below code: - -```java -public interface ManageSnapshots extends PendingUpdate { - // existing code... - - // adding this method introduces an API-breaking change - ManageSnapshots createBranch(String name); -} -``` - -And then add the implementation: -```java -public class SnapshotManager implements ManageSnapshots { - // existing code... - - @Override - public ManageSnapshots createBranch(String name, long snapshotId) { - updateSnapshotReferencesOperation().createBranch(name, snapshotId); - return this; - } -} -``` - -### Checking for API breakages -Running `./gradlew revapi` will flag this as an API-breaking change: -``` -./gradlew revapi -> Task :iceberg-api:revapi FAILED -> Task :iceberg-api:showDeprecationRulesOnRevApiFailure FAILED - -1: Task failed with an exception. ------------ -* What went wrong: -Execution failed for task ':iceberg-api:revapi'. -> There were Java public API/ABI breaks reported by revapi: - - java.method.addedToInterface: Method was added to an interface. - - old: - new: method org.apache.iceberg.ManageSnapshots org.apache.iceberg.ManageSnapshots::createBranch(java.lang.String) - - SOURCE: BREAKING, BINARY: NON_BREAKING, SEMANTIC: POTENTIALLY_BREAKING - - From old archive: - From new archive: iceberg-api-1.4.0-SNAPSHOT.jar - - If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for: - - * Just this break: - ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \ - --code "java.method.addedToInterface" \ - --new "method org.apache.iceberg.ManageSnapshots org.apache.iceberg.ManageSnapshots::createBranch(java.lang.String)" - * All breaks in this project: - ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}" - * All breaks in all projects: - ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}" - ---------------------------------------------------------------------------------------------------- - -``` - -### Adding a default implementation -In order to avoid breaking the API, we can add a default implementation that throws an `UnsupportedOperationException`: -```java -public interface ManageSnapshots extends PendingUpdate { - // existing code... - - // introduces new code without breaking the API - default ManageSnapshots createBranch(String name) { - throw new UnsupportedOperationException(this.getClass().getName() + " doesn't implement createBranch(String)"); - } -} -``` - -## Style - -For Java styling, check out the section -[Setting up IDE and Code Style](https://iceberg.apache.org/contribute/#setting-up-ide-and-code-style) from the -documentation site. - -### Java style guidelines - -#### Line breaks - -Continuation indents are 2 indents (4 spaces) from the start of the previous line. - -Try to break long lines at the same semantic level to make code more readable. -* Don't use the same level of indentation for arguments to different methods -* Don't use the same level of indentation for arguments and chained methods - -```java - // BAD: hard to see arguments passed to the same method - doSomething(new ArgumentClass(1, - 2), - 3); - - // GOOD: break lines at the same semantic level - doSomething( - new ArgumentClass(1, 2), - 3); - - // BAD: arguments and chained methods mixed - SomeObject myNewObject = SomeObject.builder(schema, partitionSpec - sortOrder) - .withProperty("x", "1") - .build() - - // GOOD: method calls at the same level, arguments indented - SomeObject myNewObject = SomeObject - .builder(schema, partitionSpec, - sortOrder) - .withProperty("x", "1") - .build() -``` - -#### Method naming - -1. Make method names as short as possible, while being clear. Omit needless words. -2. Avoid `get` in method names, unless an object must be a Java bean. - * In most cases, replace `get` with a more specific verb that describes what is happening in the method, like `find` or `fetch`. - * If there isn't a more specific verb or the method is a getter, omit `get` because it isn't helpful to readers and makes method names longer. -3. Where possible, use words and conjugations that form correct sentences in English when read - * For example, `Transform.preservesOrder()` reads correctly in an if statement: `if (transform.preservesOrder()) { ... }` - -#### Boolean arguments - -Avoid boolean arguments to methods that are not `private` to avoid confusing invocations like `sendMessage(false)`. It is better to create two methods with names and behavior, even if both are implemented by one internal method. - -```java - // prefer exposing suppressFailure in method names - public void sendMessageIgnoreFailure() { - sendMessageInternal(true); - } - - public void sendMessage() { - sendMessageInternal(false); - } - - private void sendMessageInternal(boolean suppressFailure) { - ... - } -``` - -When passing boolean arguments to existing or external methods, use inline comments to help the reader understand actions without an IDE. - -```java - // BAD: it is not clear what false controls - dropTable(identifier, false); - - // GOOD: these uses of dropTable are clear to the reader - dropTable(identifier, true /* purge data */); - dropTable(identifier, purge); -``` - -#### Config naming - -1. Use `-` to link words in one concept - * For example, preferred convection `access-key-id` rather than `access.key.id` -2. Use `.` to create a hierarchy of config groups - * For example, `s3` in `s3.access-key-id`, `s3.secret-access-key` - -## Testing - -### AssertJ - -Prefer using [AssertJ](https://github.com/assertj/assertj) assertions as those provide a rich and intuitive set of strongly-typed assertions. -Checks can be expressed in a fluent way and [AssertJ](https://github.com/assertj/assertj) provides rich context when assertions fail. -Additionally, [AssertJ](https://github.com/assertj/assertj) has powerful testing capabilities on collections and exceptions. -Please refer to the [usage guide](https://assertj.github.io/doc/#assertj-core-assertions-guide) for additional examples. - -```java -// bad: will only say true != false when check fails -assertTrue(x instanceof Xyz); - -// better: will show type of x when check fails -assertThat(x).isInstanceOf(Xyz.class); - -// bad: will only say true != false when check fails -assertTrue(catalog.listNamespaces().containsAll(expected)); - -// better: will show content of expected and of catalog.listNamespaces() if check fails -assertThat(catalog.listNamespaces()).containsAll(expected); -``` -```java -// ok -assertNotNull(metadataFileLocations); -assertEquals(metadataFileLocations.size(), 4); - -// better: will show the content of metadataFileLocations if check fails -assertThat(metadataFileLocations).isNotNull().hasSize(4); - -// or -assertThat(metadataFileLocations).isNotNull().hasSameSizeAs(expected).hasSize(4); -``` - -```java -// bad -try { - catalog.createNamespace(deniedNamespace); - Assert.fail("this should fail"); -} catch (Exception e) { - assertEquals(AccessDeniedException.class, e.getClass()); - assertEquals("User 'testUser' has no permission to create namespace", e.getMessage()); -} - -// better -assertThatThrownBy(() -> catalog.createNamespace(deniedNamespace)) - .isInstanceOf(AccessDeniedException.class) - .hasMessage("User 'testUser' has no permission to create namespace"); -``` -Checks on exceptions should always make sure to assert that a particular exception message has occurred. - - -### Awaitility - -Avoid using `Thread.sleep()` in tests as it leads to long test durations and flaky behavior if a condition takes slightly longer than expected. - -```java -deleteTablesAsync(); -Thread.sleep(3000L); -assertThat(tables()).isEmpty(); -``` - -A better alternative is using [Awaitility](https://github.com/awaitility/awaitility) to make sure `tables()` are eventually empty. The below example will run the check -with a default polling interval of **100 millis**: - -```java -deleteTablesAsync(); -Awaitility.await("Tables were not deleted") - .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertThat(tables()).isEmpty()); -``` - -Please refer to the [usage guide](https://github.com/awaitility/awaitility/wiki/Usage) of [Awaitility](https://github.com/awaitility/awaitility) for more usage examples. - - -### JUnit4 / JUnit5 - -Iceberg currently uses a mix of JUnit4 (`org.junit` imports) and JUnit5 (`org.junit.jupiter.api` imports) tests. To allow an easier migration to JUnit5 in the future, new test classes -that are being added to the codebase should be written purely in JUnit5 where possible. +Please refer to the [contributing](https://iceberg.apache.org/contribute/) section for instructions +on how to contribute to Iceberg. diff --git a/site/docs/contribute.md b/site/docs/contribute.md index fdc1ef46d8a7..e7fe35d14c06 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -96,6 +96,7 @@ The API subproject is the main interface for developers and users of the Iceberg guarantees. Evolution of the interfaces in this subproject are enforced by [Revapi](https://revapi.org/) and require explicit acknowledgement of API changes. + All public interfaces and classes require one major version for deprecation cycle. Any backward incompatible changes should be annotated as `@Deprecated` and removed for the next major release. Backward compatible changes are allowed within major versions. @@ -111,6 +112,7 @@ __Modules__ Changes to public interfaces and classes in the subprojects listed above require a deprecation cycle of one minor release. + These projects contain common and internal code used by other projects and can evolve within a major release. Minor release deprecation will provide other subprojects and external projects notice and opportunity to transition to new implementations. @@ -121,6 +123,7 @@ __modules__ (All modules not referenced above) Other modules are less likely to be extended directly and modifications should make a good faith effort to follow a minor version deprecation cycle. + If there are significant structural or design changes that result in deprecations being difficult to orchestrate, it is up to the committers to decide if deprecation is necessary. @@ -145,6 +148,89 @@ Example: void sequenceNumber(long sequenceNumber); ``` +## Adding new functionality without breaking APIs +When adding new functionality, make sure to avoid breaking existing APIs, especially within the scope of the API modules that are being checked by [Revapi](https://revapi.org/). + +Assume adding a `createBranch(String name)` method to the `ManageSnapshots` API. + +The most straight-forward way would be to add the below code: + +```java +public interface ManageSnapshots extends PendingUpdate { + // existing code... + + // adding this method introduces an API-breaking change + ManageSnapshots createBranch(String name); +} +``` + +And then add the implementation: + +```java +public class SnapshotManager implements ManageSnapshots { + // existing code... + + @Override + public ManageSnapshots createBranch(String name, long snapshotId) { + updateSnapshotReferencesOperation().createBranch(name, snapshotId); + return this; + } +} +``` + +### Checking for API breakages + +Running `./gradlew revapi` will flag this as an API-breaking change: + +``` +./gradlew revapi +> Task :iceberg-api:revapi FAILED +> Task :iceberg-api:showDeprecationRulesOnRevApiFailure FAILED + +1: Task failed with an exception. +----------- +* What went wrong: +Execution failed for task ':iceberg-api:revapi'. +> There were Java public API/ABI breaks reported by revapi: + + java.method.addedToInterface: Method was added to an interface. + + old: + new: method org.apache.iceberg.ManageSnapshots org.apache.iceberg.ManageSnapshots::createBranch(java.lang.String) + + SOURCE: BREAKING, BINARY: NON_BREAKING, SEMANTIC: POTENTIALLY_BREAKING + + From old archive: + From new archive: iceberg-api-1.4.0-SNAPSHOT.jar + + If this is an acceptable break that will not harm your users, you can ignore it in future runs like so for: + + * Just this break: + ./gradlew :iceberg-api:revapiAcceptBreak --justification "{why this break is ok}" \ + --code "java.method.addedToInterface" \ + --new "method org.apache.iceberg.ManageSnapshots org.apache.iceberg.ManageSnapshots::createBranch(java.lang.String)" + * All breaks in this project: + ./gradlew :iceberg-api:revapiAcceptAllBreaks --justification "{why this break is ok}" + * All breaks in all projects: + ./gradlew revapiAcceptAllBreaks --justification "{why this break is ok}" + ---------------------------------------------------------------------------------------------------- + +``` + +### Adding a default implementation + +To avoid breaking the API, add a default implementation that throws an `UnsupportedOperationException`:` + +```java +public interface ManageSnapshots extends PendingUpdate { + // existing code... + + // introduces new code without breaking the API + default ManageSnapshots createBranch(String name) { + throw new UnsupportedOperationException(this.getClass().getName() + " doesn't implement createBranch(String)"); + } +} +``` ## Iceberg Code Contribution Guidelines @@ -217,10 +303,10 @@ adding a Copyright profile: 1. Make method names as short as possible, while being clear. Omit needless words. 2. Avoid `get` in method names, unless an object must be a Java bean. - * In most cases, replace `get` with a more specific verb that describes what is happening in the method, like `find` or `fetch`. - * If there isn't a more specific verb or the method is a getter, omit `get` because it isn't helpful to readers and makes method names longer. + * In most cases, replace `get` with a more specific verb that describes what is happening in the method, like `find` or `fetch`. + * If there isn't a more specific verb or the method is a getter, omit `get` because it isn't helpful to readers and makes method names longer. 3. Where possible, use words and conjugations that form correct sentences in English when read - * For example, `Transform.preservesOrder()` reads correctly in an if statement: `if (transform.preservesOrder()) { ... }` + * For example, `Transform.preservesOrder()` reads correctly in an if statement: `if (transform.preservesOrder()) { ... }` #### Boolean arguments @@ -292,6 +378,18 @@ assertThat(metadataFileLocations).isNotNull().hasSize(4); // or assertThat(metadataFileLocations).isNotNull().hasSameSizeAs(expected).hasSize(4); ``` +```java +// if any key doesn't exist, it won't show the content of the map +assertThat(map.get("key1")).isEqualTo("value1"); +assertThat(map.get("key2")).isNotNull(); +assertThat(map.get("key3")).startsWith("3.5"); + +// better: all checks can be combined and the content of the map will be shown if any check fails +assertThat(map) + .containsEntry("key1", "value1") + .containsKey("key2") + .hasEntrySatisfying("key3", v -> assertThat(v).startsWith("3.5")); +``` ```java // bad @@ -346,50 +444,3 @@ no "push a single button to get a performance comparison" solution available, th post the results on the PR. See [Benchmarks](benchmarks.md) for a summary of available benchmarks and how to run them. - -## Website and Documentation Updates - -Currently, there is an [iceberg-docs](https://github.com/apache/iceberg-docs) repository -which contains the HTML/CSS and other files needed for the [Iceberg website](https://iceberg.apache.org/). -The [docs folder](https://github.com/apache/iceberg/tree/master/docs) in the Iceberg repository contains -the markdown content for the documentation site. All markdown changes should still be made -to this repository. - -### Submitting Pull Requests - -Changes to the markdown contents should be submitted directly to this repository. - -Changes to the website appearance (e.g. HTML, CSS changes) should be submitted to the [iceberg-docs repository](https://github.com/apache/iceberg-docs) against the `main` branch. - -Changes to the documentation of old Iceberg versions should be submitted to the [iceberg-docs repository](https://github.com/apache/iceberg-docs) against the specific version branch. - -### Reporting Issues - -All issues related to the doc website should still be submitted to the [Iceberg repository](https://github.com/apache/iceberg). -The GitHub Issues feature of the [iceberg-docs repository](https://github.com/apache/iceberg-docs) is disabled. - -### Running Locally - -Clone the [iceberg-docs](https://github.com/apache/iceberg-docs) repository to run the website locally: -```shell -git clone git@github.com:apache/iceberg-docs.git -cd iceberg-docs -``` - -To start the landing page site locally, run: -```shell -cd landing-page && hugo serve -``` - -To start the documentation site locally, run: -```shell -cd docs && hugo serve -``` - -If you would like to see how the latest website looks based on the documentation in the Iceberg repository, you can copy docs to the iceberg-docs repository by: -```shell -rm -rf docs/content/docs -rm -rf landing-page/content/common -cp -r /docs/versioned docs/content/docs -cp -r /docs/common landing-page/content/common -``` From 0612553f5f7919aa83474c087ee378b0272d0c10 Mon Sep 17 00:00:00 2001 From: Reo Date: Mon, 26 Feb 2024 20:58:15 +0800 Subject: [PATCH 0076/1019] Flink 1.16, 1.17: Fix continuous enumerator lost enumeration history state when restore from checkpoint (#9812) --- .../flink/source/enumerator/ContinuousIcebergEnumerator.java | 1 + .../flink/source/enumerator/ContinuousIcebergEnumerator.java | 1 + 2 files changed, 2 insertions(+) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index b1dadfb9a69a..55451b105885 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -75,6 +75,7 @@ public ContinuousIcebergEnumerator( if (enumState != null) { this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); + this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index b1dadfb9a69a..55451b105885 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -75,6 +75,7 @@ public ContinuousIcebergEnumerator( if (enumState != null) { this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); + this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); } } From a2c5f27226d39f3612098f1fa647d1d0e3cb793d Mon Sep 17 00:00:00 2001 From: Reo Date: Mon, 26 Feb 2024 20:58:52 +0800 Subject: [PATCH 0077/1019] Flink 1.16, 1.17: Fix iceberg source plan parallelism not effective (#9811) --- .../src/main/java/org/apache/iceberg/flink/FlinkReadConf.java | 1 + .../src/main/java/org/apache/iceberg/flink/FlinkReadConf.java | 1 + 2 files changed, 2 insertions(+) diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java index d53ea73f9342..804a956ec9b9 100644 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java @@ -178,6 +178,7 @@ public long limit() { public int workerPoolSize() { return confParser .intConf() + .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) .parse(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java index d53ea73f9342..804a956ec9b9 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java @@ -178,6 +178,7 @@ public long limit() { public int workerPoolSize() { return confParser .intConf() + .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) .parse(); From 9ca830c2673739a36845fe1e2a44d871c9b7f9eb Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 26 Feb 2024 15:51:43 +0100 Subject: [PATCH 0078/1019] Spark 3.4: Include catalog name in view errors (#9810) --- .../sql/catalyst/analysis/CheckViews.scala | 17 ++++++----- .../analysis/RewriteViewCommands.scala | 17 +++++------ .../iceberg/spark/extensions/TestViews.java | 30 ++++++++++++------- .../analysis/RewriteViewCommands.scala | 7 ++--- 4 files changed, 38 insertions(+), 33 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 5e9429ac3005..4debc4d343a0 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.plans.logical.AlterViewAs import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View -import org.apache.spark.sql.connector.catalog.Identifier import org.apache.spark.sql.connector.catalog.ViewCatalog import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.util.SchemaUtils @@ -33,9 +32,9 @@ object CheckViews extends (LogicalPlan => Unit) { override def apply(plan: LogicalPlan): Unit = { plan foreach { - case CreateIcebergView(ResolvedIdentifier(_: ViewCatalog, ident), _, query, columnAliases, _, + case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, _, _, _, _, _, _) => - verifyColumnCount(ident, columnAliases, query) + verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) case AlterViewAs(ResolvedV2View(_, _), _, _) => @@ -45,16 +44,18 @@ object CheckViews extends (LogicalPlan => Unit) { } } - private def verifyColumnCount(ident: Identifier, columns: Seq[String], query: LogicalPlan): Unit = { + private def verifyColumnCount(ident: ResolvedIdentifier, columns: Seq[String], query: LogicalPlan): Unit = { if (columns.nonEmpty) { if (columns.length > query.output.length) { - throw new AnalysisException(String.format("Cannot create view %s, the reason is not enough data columns:\n" + + throw new AnalysisException(String.format("Cannot create view %s.%s, the reason is not enough data columns:\n" + "View columns: %s\n" + - "Data columns: %s", ident.toString, columns.mkString(", "), query.output.map(c => c.name).mkString(", "))) + "Data columns: %s", ident.catalog.name(), ident.identifier, columns.mkString(", "), + query.output.map(c => c.name).mkString(", "))) } else if (columns.length < query.output.length) { - throw new AnalysisException(String.format("Cannot create view %s, the reason is too many data columns:\n" + + throw new AnalysisException(String.format("Cannot create view %s.%s, the reason is too many data columns:\n" + "View columns: %s\n" + - "Data columns: %s", ident.toString, columns.mkString(", "), query.output.map(c => c.name).mkString(", "))) + "Data columns: %s", ident.catalog.name(), ident.identifier, columns.mkString(", "), + query.output.map(c => c.name).mkString(", "))) } } } diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index e48357971211..ece851a25b13 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -35,10 +35,7 @@ import org.apache.spark.sql.catalyst.plans.logical.views.ShowIcebergViews import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.UNRESOLVED_FUNCTION 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.LookupCatalog -import org.apache.spark.sql.connector.catalog.ViewCatalog import scala.collection.mutable /** @@ -58,7 +55,7 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case CreateView(ResolvedIdent(resolved), userSpecifiedColumns, comment, properties, Some(queryText), query, allowExisting, replace) => val q = CTESubstitution.apply(query) - verifyTemporaryObjectsDontExist(resolved.identifier, q) + verifyTemporaryObjectsDontExist(resolved, q) CreateIcebergView(child = resolved, queryText = queryText, query = q, @@ -113,22 +110,22 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi * Permanent views are not allowed to reference temp objects */ private def verifyTemporaryObjectsDontExist( - name: Identifier, + identifier: ResolvedIdentifier, child: LogicalPlan): Unit = { val tempViews = collectTemporaryViews(child) if (tempViews.nonEmpty) { - throw invalidRefToTempObject(name, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") + throw invalidRefToTempObject(identifier, tempViews.map(v => v.quoted).mkString("[", ", ", "]"), "view") } val tempFunctions = collectTemporaryFunctions(child) if (tempFunctions.nonEmpty) { - throw invalidRefToTempObject(name, tempFunctions.mkString("[", ", ", "]"), "function") + throw invalidRefToTempObject(identifier, tempFunctions.mkString("[", ", ", "]"), "function") } } - private def invalidRefToTempObject(name: Identifier, tempObjectNames: String, tempObjectType: String) = { - new AnalysisException(String.format("Cannot create view %s that references temporary %s: %s", - name, tempObjectType, tempObjectNames)) + private def invalidRefToTempObject(ident: ResolvedIdentifier, tempObjectNames: String, tempObjectType: String) = { + new AnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s", + ident.catalog.name(), ident.identifier, tempObjectType, tempObjectNames)) } /** 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 26bcb03ba101..7314b4bb0f65 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 @@ -981,7 +981,8 @@ public void createViewReferencingTempView() throws NoSuchTableException { () -> sql("CREATE VIEW %s AS SELECT id FROM %s", viewReferencingTempView, tempView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + String.format( + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -1004,7 +1005,8 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { viewReferencingTempView, globalTempView)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( - String.format("Cannot create view %s.%s", NAMESPACE, viewReferencingTempView)) + String.format( + "Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewReferencingTempView)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); } @@ -1022,7 +1024,8 @@ public void createViewReferencingTempFunction() { assertThatThrownBy( () -> sql("CREATE VIEW %s AS SELECT %s(id) FROM %s", viewName, functionName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1073,7 +1076,8 @@ public void createViewWithMismatchedColumnCounts() { assertThatThrownBy( () -> sql("CREATE VIEW %s (id, data) AS SELECT id FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("not enough data columns") .hasMessageContaining("View columns: id, data") .hasMessageContaining("Data columns: id"); @@ -1081,7 +1085,8 @@ public void createViewWithMismatchedColumnCounts() { assertThatThrownBy( () -> sql("CREATE VIEW %s (id) AS SELECT id, data FROM %s", viewName, tableName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("too many data columns") .hasMessageContaining("View columns: id") .hasMessageContaining("Data columns: id, data"); @@ -1200,7 +1205,8 @@ public void createViewWithCTEReferencingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempViewInCTE); } @@ -1221,7 +1227,8 @@ public void createViewWithCTEReferencingTempFunction() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } @@ -1249,7 +1256,8 @@ public void createViewWithSubqueryExpressionUsingTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(tempView); } @@ -1269,7 +1277,8 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary view:") .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); } @@ -1289,7 +1298,8 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { assertThatThrownBy(() -> sql("CREATE VIEW %s AS %s", viewName, sql)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining(String.format("Cannot create view %s.%s", NAMESPACE, viewName)) + .hasMessageContaining( + String.format("Cannot create view %s.%s.%s", catalogName, NAMESPACE, viewName)) .hasMessageContaining("that references temporary function:") .hasMessageContaining(functionName); } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 22c0bb330179..ef0ed5c996de 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -123,12 +123,9 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi } } - private def invalidRefToTempObject( - identifier: ResolvedIdentifier, - tempObjectNames: String, - tempObjectType: String) = { + private def invalidRefToTempObject(ident: ResolvedIdentifier, tempObjectNames: String, tempObjectType: String) = { new AnalysisException(String.format("Cannot create view %s.%s that references temporary %s: %s", - identifier.catalog.name(), identifier.identifier, tempObjectType, tempObjectNames)) + ident.catalog.name(), ident.identifier, tempObjectType, tempObjectNames)) } /** From 0b9a4803f1cec928f7545afaf100e33f9d19a1b3 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Mon, 26 Feb 2024 15:21:31 -0800 Subject: [PATCH 0079/1019] Spec: Clarify multi-arg transform behavior for different versions (#9661) --- format/spec.md | 59 ++++++++++++++++++++++++++++++++++++++++---------- 1 file changed, 47 insertions(+), 12 deletions(-) diff --git a/format/spec.md b/format/spec.md index efc21e904b84..306cdb77757b 100644 --- a/format/spec.md +++ b/format/spec.md @@ -29,6 +29,8 @@ This is a specification for the Iceberg table format that is designed to manage Versions 1 and 2 of the Iceberg spec are complete and adopted by the community. +**Version 3 is under active development and has not been formally adopted.** + 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 @@ -301,7 +303,7 @@ Tables are configured with a **partition spec** that defines how to produce a tu * A **transform** that is applied to the source column(s) to produce a partition value * A **partition name** -The source column, selected by id, must be a primitive type and cannot be contained in a map or list, but may be nested in a struct. For details on how to serialize a partition spec to JSON, see Appendix C. +The source columns, selected by ids, must be a primitive type and cannot be contained in a map or list, but may be nested in a struct. For details on how to serialize a partition spec to JSON, see Appendix C. Partition specs capture the transform from table data to partition values. This is used to transform predicates to partition predicates, in addition to transforming data values. Deriving partition predicates from column predicates on the table data is used to separate the logical queries from physical storage: the partitioning can change and the correct partition filters are always derived from column predicates. This simplifies queries because users don’t have to supply both logical predicates and partition predicates. For more information, see Scan Planning below. @@ -388,6 +390,8 @@ A sort order is defined by a sort order id and a list of sort fields. The order * A **sort direction**, that can only be either `asc` or `desc` * A **null order** that describes the order of null values when sorted. Can only be either `nulls-first` or `nulls-last` +For details on how to serialize a sort order to JSON, see Appendix C. + Order id `0` is reserved for the unsorted order. Sorting floating-point numbers should produce the following behavior: `-NaN` < `-Infinity` < `-value` < `-0` < `0` < `value` < `Infinity` < `NaN`. This aligns with the implementation of Java floating-point types comparisons. @@ -1117,7 +1121,17 @@ Partition specs are serialized as a JSON object with the following fields: |**`spec-id`**|`JSON int`|`0`| |**`fields`**|`JSON list: [`
  `,`
  `...`
`]`|`[ {`
  `"source-id": 4,`
  `"field-id": 1000,`
  `"name": "ts_day",`
  `"transform": "day"`
`}, {`
  `"source-id": 1,`
  `"field-id": 1001,`
  `"name": "id_bucket",`
  `"transform": "bucket[16]"`
`} ]`| -Each partition field in the fields list is stored as an object. See the table for more detail: +Each partition field in `fields` is stored as a JSON object with the following properties. + +| V1 | V2 | V3 | Field | JSON representation | Example | +|----------|----------|----------|------------------|---------------------|--------------| +| required | required | omitted | **`source-id`** | `JSON int` | 1 | +| optional | optional | required | **`source-ids`** | `JSON list of ints` | `[1,2]` | +| | required | required | **`field-id`** | `JSON int` | 1000 | +| required | required | required | **`name`** | `JSON string` | `id_bucket` | +| required | required | required | **`transform`** | `JSON string` | `bucket[16]` | + +Supported partition transforms are listed below. |Transform or Field|JSON representation|Example| |--- |--- |--- | @@ -1128,16 +1142,14 @@ Each partition field in the fields list is stored as an object. See the table fo |**`month`**|`JSON string: "month"`|`"month"`| |**`day`**|`JSON string: "day"`|`"day"`| |**`hour`**|`JSON string: "hour"`|`"hour"`| -|**`Partition Field`** [1,2]|`JSON object: {`
  `"source-id": ,`
  `"field-id": ,`
  `"name": ,`
  `"transform": `
`}`|`{`
  `"source-id": 1,`
  `"field-id": 1000,`
  `"name": "id_bucket",`
  `"transform": "bucket[16]"`
`}`| In some cases partition specs are stored using only the field list instead of the object format that includes the spec ID, like the deprecated `partition-spec` field in table metadata. The object format should be used unless otherwise noted in this spec. The `field-id` property was added for each partition field in v2. In v1, the reference implementation assigned field ids sequentially in each spec starting at 1,000. See Partition Evolution for more details. -Notes: +In v3 metadata, writers must use only `source-ids` because v3 requires reader support for multi-arg transforms. In v1 and v2 metadata, writers must always write `source-id`; for multi-arg transforms, writers must produce `source-ids` and set `source-id` to the first ID from the field ID list. -1. For partition fields with a transform with a single argument, the ID of the source field is set on `source-id`, and `source-ids` is omitted. -2. For partition fields with a transform of multiple arguments, the IDs of the source fields are set on `source-ids`. To preserve backward compatibility, `source-id` is set to -1. +Older versions of the reference implementation can read tables with transforms unknown to it, ignoring them. But other implementations may break if they encounter unknown transforms. All v3 readers are required to read tables with unknown transforms, ignoring them. Writers should not write using partition specs that use unknown transforms. ### Sort Orders @@ -1150,13 +1162,17 @@ Sort orders are serialized as a list of JSON object, each of which contains the Each sort field in the fields list is stored as an object with the following properties: -|Field|JSON representation|Example| -|--- |--- |--- | -|**`Sort Field`** [1,2]|`JSON object: {`
  `"transform": ,`
  `"source-id": ,`
  `"direction": ,`
  `"null-order": `
`}`|`{`
  ` "transform": "bucket[4]",`
  ` "source-id": 3,`
  ` "direction": "desc",`
  ` "null-order": "nulls-last"`
`}`| +| V1 | V2 | V3 | Field | JSON representation | Example | +|----------|----------|----------|------------------|---------------------|-------------| +| required | required | required | **`transform`** | `JSON string` | `bucket[4]` | +| required | required | omitted | **`source-id`** | `JSON int` | 1 | +| | | required | **`source-ids`** | `JSON list of ints` | `[1,2]` | +| required | required | required | **`direction`** | `JSON string` | `asc` | +| required | required | required | **`null-order`** | `JSON string` | `nulls-last`| -Notes: -1. For sort fields with a transform with a single argument, the ID of the source field is set on `source-id`, and `source-ids` is omitted. -2. For sort fields with a transform of multiple arguments, the IDs of the source fields are set on `source-ids`. To preserve backward compatibility, `source-id` is set to -1. +In v3 metadata, writers must use only `source-ids` because v3 requires reader support for multi-arg transforms. In v1 and v2 metadata, writers must always write `source-id`; for multi-arg transforms, writers must produce `source-ids` and set `source-id` to the first ID from the field ID list. + +Older versions of the reference implementation can read tables with transforms unknown to it, ignoring them. But other implementations may break if they encounter unknown transforms. All v3 readers are required to read tables with unknown transforms, ignoring them. The following table describes the possible values for the some of the field within sort field: @@ -1314,6 +1330,25 @@ Default values are added to struct fields in v3. Types `timestamp_ns` and `timestamptz_ns` are added in v3. +All readers are required to read tables with unknown partition transforms, ignoring them. + +Writing v3 metadata: + +* Partition Field and Sort Field JSON: + * `source-ids` was added and is required + * `source-id` is no longer required and should be omitted; always use `source-ids` instead + +Reading v1 or v2 metadata for v3: + +* Partition Field and Sort Field JSON: + * `source-ids` should default to a single-value list of the value of `source-id` + +Writing v1 or v2 metadata: + +* Partition Field and Sort Field JSON: + * 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` + ### Version 2 Writing v1 metadata: From 835affb970a79920e79617ee42312ba27191cd3c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 27 Feb 2024 08:32:18 +0100 Subject: [PATCH 0080/1019] Spark 3.4, 3.5: Use current namespace for SHOW VIEWS cmd (#9787) This uses the current namespace when `SHOW VIEWS` is executed without specifying any namespace but `USE ` was defined. This behavior is similar to what's being done for V1 `ShowViews` in https://github.com/apache/spark/blob/branch-3.5/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveCatalogs.scala#L39-L44 --- .../analysis/RewriteViewCommands.scala | 3 +- .../iceberg/spark/extensions/TestViews.java | 31 +++++++++++++++++++ .../analysis/RewriteViewCommands.scala | 3 +- .../iceberg/spark/extensions/TestViews.java | 31 +++++++++++++++++++ 4 files changed, 66 insertions(+), 2 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index ece851a25b13..377025f92644 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -68,7 +68,8 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case ShowViews(UnresolvedNamespace(Seq()), pattern, output) if ViewUtil.isViewCatalog(catalogManager.currentCatalog) => - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, Seq.empty), pattern, output) + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, output) case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)), pattern, output) if ViewUtil.isViewCatalog(catalog) => 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 7314b4bb0f65..5bef3003b421 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 @@ -1466,6 +1466,37 @@ public void showViews() throws NoSuchTableException { row("global_temp", "globalviewforlisting", true), tempView); } + @Test + public void showViewsWithCurrentNamespace() { + String namespaceOne = "show_views_ns1"; + String namespaceTwo = "show_views_ns2"; + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceOne); + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceTwo); + + // create one view in each namespace + sql("CREATE VIEW %s.%s AS SELECT * FROM %s.%s", namespaceOne, viewOne, NAMESPACE, tableName); + sql("CREATE VIEW %s.%s AS SELECT * FROM %s.%s", namespaceTwo, viewTwo, NAMESPACE, tableName); + + Object[] v1 = row(namespaceOne, viewOne, false); + Object[] v2 = row(namespaceTwo, viewTwo, false); + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceOne)) + .contains(v1) + .doesNotContain(v2); + sql("USE %s", namespaceOne); + assertThat(sql("SHOW VIEWS")).contains(v1).doesNotContain(v2); + assertThat(sql("SHOW VIEWS LIKE 'viewOne*'")).contains(v1).doesNotContain(v2); + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceTwo)) + .contains(v2) + .doesNotContain(v1); + sql("USE %s", namespaceTwo); + assertThat(sql("SHOW VIEWS")).contains(v2).doesNotContain(v1); + assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); + } + @Test public void showCreateSimpleView() { String viewName = "showCreateSimpleView"; diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index ef0ed5c996de..011626c280b6 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -68,7 +68,8 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi case ShowViews(UnresolvedNamespace(Seq()), pattern, output) if ViewUtil.isViewCatalog(catalogManager.currentCatalog) => - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, Seq.empty), pattern, output) + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, output) case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)), pattern, output) if ViewUtil.isViewCatalog(catalog) => 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 f1e6580a3e19..ac02561a3c76 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 @@ -1466,6 +1466,37 @@ public void showViews() throws NoSuchTableException { row("global_temp", "globalviewforlisting", true), tempView); } + @Test + public void showViewsWithCurrentNamespace() { + String namespaceOne = "show_views_ns1"; + String namespaceTwo = "show_views_ns2"; + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceOne); + sql("CREATE NAMESPACE IF NOT EXISTS %s", namespaceTwo); + + // create one view in each namespace + sql("CREATE VIEW %s.%s AS SELECT * FROM %s.%s", namespaceOne, viewOne, NAMESPACE, tableName); + sql("CREATE VIEW %s.%s AS SELECT * FROM %s.%s", namespaceTwo, viewTwo, NAMESPACE, tableName); + + Object[] v1 = row(namespaceOne, viewOne, false); + Object[] v2 = row(namespaceTwo, viewTwo, false); + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceOne)) + .contains(v1) + .doesNotContain(v2); + sql("USE %s", namespaceOne); + assertThat(sql("SHOW VIEWS")).contains(v1).doesNotContain(v2); + assertThat(sql("SHOW VIEWS LIKE 'viewOne*'")).contains(v1).doesNotContain(v2); + + assertThat(sql("SHOW VIEWS IN %s.%s", catalogName, namespaceTwo)) + .contains(v2) + .doesNotContain(v1); + sql("USE %s", namespaceTwo); + assertThat(sql("SHOW VIEWS")).contains(v2).doesNotContain(v1); + assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); + } + @Test public void showCreateSimpleView() { String viewName = "showCreateSimpleView"; From 7fd3e77e924eb5a729936903a51e42564521fb38 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 27 Feb 2024 14:11:51 +0530 Subject: [PATCH 0081/1019] AWS: Revert DynamoDb deprecation for 1.5.0 (#9815) --- .../org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java | 7 +------ .../iceberg/aws/dynamodb/DynamoDbTableOperations.java | 2 -- docs/docs/aws.md | 1 - 3 files changed, 1 insertion(+), 9 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java index 393c6d7cdf10..0c991af75076 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbCatalog.java @@ -84,12 +84,7 @@ import software.amazon.awssdk.services.dynamodb.model.TransactWriteItemsRequest; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; -/** - * DynamoDB implementation of Iceberg catalog - * - * @deprecated since 1.5.0, will be removed in 2.0.0 - */ -@Deprecated +/** DynamoDB implementation of Iceberg catalog */ public class DynamoDbCatalog extends BaseMetastoreCatalog implements SupportsNamespaces, Configurable { diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java index 17e6c2c55a95..a1a330b11889 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbTableOperations.java @@ -44,8 +44,6 @@ import software.amazon.awssdk.services.dynamodb.model.PutItemRequest; import software.amazon.awssdk.services.dynamodb.model.UpdateItemRequest; -/** @deprecated since 1.5.0, will be removed in 2.0.0 */ -@Deprecated class DynamoDbTableOperations extends BaseMetastoreTableOperations { private static final Logger LOG = LoggerFactory.getLogger(DynamoDbTableOperations.class); diff --git a/docs/docs/aws.md b/docs/docs/aws.md index eb217e424d44..2bd6636670ee 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -226,7 +226,6 @@ PARTITIONED BY (category); ``` ### DynamoDB Catalog -**Deprecated:** As of version 1.5.0, the DynamoDB Catalog is planned for deprecation in version 2.0.0. Iceberg supports using a [DynamoDB](https://aws.amazon.com/dynamodb) table to record and manage database and table information. From e6a0458790c423ef8b0ca56f4168948dbb581198 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 27 Feb 2024 23:19:02 +0900 Subject: [PATCH 0082/1019] Spark: Migrate tests to JUnit5 (#9790) --- spark/v3.5/build.gradle | 2 + .../extensions/TestAlterTableSchema.java | 6 +- ...TestMetaColumnProjectionWithStageScan.java | 28 ++- .../spark/extensions/TestMetadataTables.java | 214 ++++++++++-------- ...ePartitionedJoinsInRowLevelOperations.java | 57 +++-- .../TestSystemFunctionPushDownDQL.java | 83 ++++--- .../iceberg/spark/extensions/TestViews.java | 171 +++++++------- .../org/apache/iceberg/spark/SmokeTest.java | 95 ++++---- 8 files changed, 337 insertions(+), 319 deletions(-) diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index d4cb67a2bbf3..b29ba6761ebc 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -239,6 +239,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio integrationImplementation "org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}" integrationImplementation "org.apache.spark:spark-hive_${scalaVersion}:${libs.versions.spark.hive35.get()}" integrationImplementation libs.junit.vintage.engine + integrationImplementation libs.junit.jupiter integrationImplementation libs.slf4j.simple integrationImplementation libs.assertj.core integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') @@ -288,6 +289,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio } task integrationTest(type: Test) { + useJUnitPlatform() description = "Test Spark3 Runtime Jar against Spark ${sparkMajorVersion}" group = "verification" jvmArgs += project.property('extraJvmArgs') diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java index 91126776528a..3b2427eea7ca 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -24,7 +24,6 @@ import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -74,8 +73,9 @@ public void testSetIdentifierFields() { public void testSetInvalidIdentifierFields() { sql("CREATE TABLE %s (id bigint NOT NULL, id2 bigint) USING iceberg", tableName); Table table = validationCatalog.loadTable(tableIdent); - Assert.assertTrue( - "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); + assertThat(table.schema().identifierFieldIds()) + .as("Table should start without identifier") + .isEmpty(); Assertions.assertThatThrownBy( () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) .isInstanceOf(IllegalArgumentException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java index e9013848cf11..b783a006ef73 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -18,9 +18,12 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; -import java.util.Map; import java.util.UUID; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.ScanTask; import org.apache.iceberg.Table; import org.apache.iceberg.io.CloseableIterable; @@ -34,19 +37,14 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Test; -import org.junit.runners.Parameterized; - -public class TestMetaColumnProjectionWithStageScan extends SparkExtensionsTestBase { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - public TestMetaColumnProjectionWithStageScan( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetaColumnProjectionWithStageScan extends ExtensionsTestBase { - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -57,7 +55,7 @@ public static Object[][] parameters() { }; } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -68,7 +66,7 @@ private void stageTask( taskSetManager.stageTasks(tab, fileSetID, Lists.newArrayList(tasks)); } - @Test + @TestTemplate public void testReadStageTableMeta() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" @@ -103,7 +101,7 @@ public void testReadStageTableMeta() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .load(tableLocation); - Assertions.assertThat(scanDF2.columns().length).isEqualTo(2); + assertThat(scanDF2.columns()).hasSize(2); } try (CloseableIterable tasks = table.newBatchScan().planFiles()) { 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 50376589b671..a22cf61ec8c9 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 @@ -24,7 +24,6 @@ import java.io.IOException; 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.avro.generic.GenericData.Record; @@ -33,6 +32,7 @@ import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Schema; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; @@ -53,22 +53,19 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.catalyst.util.DateTimeUtils; import org.apache.spark.sql.types.StructType; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestMetadataTables extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetadataTables extends ExtensionsTestBase { - public TestMetadataTables(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testUnpartitionedTable() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" @@ -92,8 +89,8 @@ public void testUnpartitionedTable() throws Exception { Table table = Spark3Util.loadIcebergTable(spark, tableName); List expectedDataManifests = TestHelpers.dataManifests(table); List expectedDeleteManifests = TestHelpers.deleteManifests(table); - Assert.assertEquals("Should have 1 data manifest", 1, expectedDataManifests.size()); - Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); + assertThat(expectedDataManifests).as("Should have 1 data manifest").hasSize(1); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); Schema entriesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(); Schema filesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".files").schema(); @@ -101,13 +98,12 @@ public void testUnpartitionedTable() throws Exception { // check delete files table Dataset actualDeleteFilesDs = spark.sql("SELECT * FROM " + tableName + ".delete_files"); List actualDeleteFiles = TestHelpers.selectNonDerived(actualDeleteFilesDs).collectAsList(); - Assert.assertEquals( - "Metadata table should return one delete file", 1, actualDeleteFiles.size()); + assertThat(actualDeleteFiles).as("Metadata table should return one delete file").hasSize(1); List expectedDeleteFiles = expectedEntries( table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, null); - Assert.assertEquals("Should be one delete file manifest entry", 1, expectedDeleteFiles.size()); + assertThat(expectedDeleteFiles).as("Should be one delete file manifest entry").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDeleteFilesDs), expectedDeleteFiles.get(0), @@ -116,11 +112,11 @@ public void testUnpartitionedTable() throws Exception { // check data files table Dataset actualDataFilesDs = spark.sql("SELECT * FROM " + tableName + ".data_files"); List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); - Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); List expectedDataFiles = expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - Assert.assertEquals("Should be one data file manifest entry", 1, expectedDataFiles.size()); + assertThat(expectedDataFiles).as("Should be one data file manifest entry").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDataFilesDs), expectedDataFiles.get(0), @@ -131,19 +127,19 @@ public void testUnpartitionedTable() throws Exception { spark.sql("SELECT * FROM " + tableName + ".files ORDER BY content"); List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); - Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); List expectedFiles = Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) .collect(Collectors.toList()); - Assert.assertEquals("Should have two files manifest entries", 2, expectedFiles.size()); + assertThat(expectedFiles).as("Should have two files manifest entries").hasSize(2); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0)); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1)); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) " @@ -177,8 +173,8 @@ public void testPartitionedTable() throws Exception { List expectedDataManifests = TestHelpers.dataManifests(table); List expectedDeleteManifests = TestHelpers.deleteManifests(table); - Assert.assertEquals("Should have 2 data manifests", 2, expectedDataManifests.size()); - Assert.assertEquals("Should have 2 delete manifests", 2, expectedDeleteManifests.size()); + assertThat(expectedDataManifests).as("Should have 2 data manifest").hasSize(2); + assertThat(expectedDeleteManifests).as("Should have 2 delete manifest").hasSize(2); Schema filesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".delete_files").schema(); @@ -187,15 +183,13 @@ public void testPartitionedTable() throws Exception { List expectedDeleteFiles = expectedEntries( table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - Assert.assertEquals( - "Should have one delete file manifest entry", 1, expectedDeleteFiles.size()); + assertThat(expectedDeleteFiles).as("Should have one delete file manifest entry").hasSize(1); Dataset actualDeleteFilesDs = spark.sql("SELECT * FROM " + tableName + ".delete_files " + "WHERE partition.data='a'"); List actualDeleteFiles = actualDeleteFilesDs.collectAsList(); - Assert.assertEquals( - "Metadata table should return one delete file", 1, actualDeleteFiles.size()); + assertThat(actualDeleteFiles).as("Metadata table should return one delete file").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDeleteFilesDs), expectedDeleteFiles.get(0), @@ -204,13 +198,13 @@ public void testPartitionedTable() throws Exception { // Check data files table List expectedDataFiles = expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - Assert.assertEquals("Should have one data file manifest entry", 1, expectedDataFiles.size()); + assertThat(expectedDataFiles).as("Should have one data file manifest entry").hasSize(1); Dataset actualDataFilesDs = spark.sql("SELECT * FROM " + tableName + ".data_files " + "WHERE partition.data='a'"); List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); - Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDataFilesDs), expectedDataFiles.get(0), @@ -218,32 +212,29 @@ public void testPartitionedTable() throws Exception { List actualPartitionsWithProjection = spark.sql("SELECT file_count FROM " + tableName + ".partitions ").collectAsList(); - Assert.assertEquals( - "Metadata table should return two partitions record", - 2, - actualPartitionsWithProjection.size()); - for (int i = 0; i < 2; ++i) { - Assert.assertEquals(1, actualPartitionsWithProjection.get(i).get(0)); - } + assertThat(actualPartitionsWithProjection) + .as("Metadata table should return two partitions record") + .hasSize(2) + .containsExactly(RowFactory.create(1), RowFactory.create(1)); // Check files table List expectedFiles = Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) .collect(Collectors.toList()); - Assert.assertEquals("Should have two file manifest entries", 2, expectedFiles.size()); + assertThat(expectedFiles).as("Should have two file manifest entries").hasSize(2); Dataset actualFilesDs = spark.sql( "SELECT * FROM " + tableName + ".files " + "WHERE partition.data='a' ORDER BY content"); List actualFiles = TestHelpers.selectNonDerived(actualFilesDs).collectAsList(); - Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0)); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1)); } - @Test + @TestTemplate public void testAllFilesUnpartitioned() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) USING iceberg TBLPROPERTIES" @@ -267,13 +258,13 @@ public void testAllFilesUnpartitioned() throws Exception { Table table = Spark3Util.loadIcebergTable(spark, tableName); List expectedDataManifests = TestHelpers.dataManifests(table); - Assert.assertEquals("Should have 1 data manifest", 1, expectedDataManifests.size()); + assertThat(expectedDataManifests).as("Should have 1 data manifest").hasSize(1); List expectedDeleteManifests = TestHelpers.deleteManifests(table); - Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); // Clear table to test whether 'all_files' can read past files List results = sql("DELETE FROM %s", tableName); - Assert.assertEquals("Table should be cleared", 0, results.size()); + assertThat(results).as("Table should be cleared").isEmpty(); Schema entriesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(); Schema filesTableSchema = @@ -285,8 +276,8 @@ public void testAllFilesUnpartitioned() throws Exception { List expectedDataFiles = expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - Assert.assertEquals("Should be one data file manifest entry", 1, expectedDataFiles.size()); - Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); + assertThat(expectedDataFiles).as("Should be one data file manifest entry").hasSize(1); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDataFilesDs), expectedDataFiles.get(0), @@ -299,9 +290,8 @@ public void testAllFilesUnpartitioned() throws Exception { List expectedDeleteFiles = expectedEntries( table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, null); - Assert.assertEquals("Should be one delete file manifest entry", 1, expectedDeleteFiles.size()); - Assert.assertEquals( - "Metadata table should return one delete file", 1, actualDeleteFiles.size()); + assertThat(expectedDeleteFiles).as("Should be one delete file manifest entry").hasSize(1); + assertThat(actualDeleteFiles).as("Metadata table should return one delete file").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDeleteFilesDs), expectedDeleteFiles.get(0), @@ -313,12 +303,12 @@ public void testAllFilesUnpartitioned() throws Exception { List actualFiles = actualFilesDs.collectAsList(); List expectedFiles = ListUtils.union(expectedDataFiles, expectedDeleteFiles); expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles, actualFiles); } - @Test + @TestTemplate public void testAllFilesPartitioned() throws Exception { // Create table and insert data sql( @@ -350,13 +340,13 @@ public void testAllFilesPartitioned() throws Exception { Table table = Spark3Util.loadIcebergTable(spark, tableName); List expectedDataManifests = TestHelpers.dataManifests(table); - Assert.assertEquals("Should have 2 data manifests", 2, expectedDataManifests.size()); + assertThat(expectedDataManifests).as("Should have 2 data manifests").hasSize(2); List expectedDeleteManifests = TestHelpers.deleteManifests(table); - Assert.assertEquals("Should have 1 delete manifest", 1, expectedDeleteManifests.size()); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); // Clear table to test whether 'all_files' can read past files List results = sql("DELETE FROM %s", tableName); - Assert.assertEquals("Table should be cleared", 0, results.size()); + assertThat(results).as("Table should be cleared").isEmpty(); Schema entriesTableSchema = Spark3Util.loadIcebergTable(spark, tableName + ".entries").schema(); Schema filesTableSchema = @@ -368,8 +358,8 @@ public void testAllFilesPartitioned() throws Exception { List actualDataFiles = TestHelpers.selectNonDerived(actualDataFilesDs).collectAsList(); List expectedDataFiles = expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - Assert.assertEquals("Should be one data file manifest entry", 1, expectedDataFiles.size()); - Assert.assertEquals("Metadata table should return one data file", 1, actualDataFiles.size()); + assertThat(expectedDataFiles).as("Should be one data file manifest entry").hasSize(1); + assertThat(actualDataFiles).as("Metadata table should return one data file").hasSize(1); TestHelpers.assertEqualsSafe( SparkSchemaUtil.convert(TestHelpers.selectNonDerived(actualDataFilesDs).schema()) .asStruct(), @@ -384,8 +374,8 @@ public void testAllFilesPartitioned() throws Exception { List expectedDeleteFiles = expectedEntries( table, FileContent.POSITION_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - Assert.assertEquals("Should be one data file manifest entry", 1, expectedDeleteFiles.size()); - Assert.assertEquals("Metadata table should return one data file", 1, actualDeleteFiles.size()); + assertThat(expectedDeleteFiles).as("Should be one data file manifest entry").hasSize(1); + assertThat(actualDeleteFiles).as("Metadata table should return one data file").hasSize(1); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDeleteFilesDs), @@ -403,12 +393,12 @@ public void testAllFilesPartitioned() throws Exception { List expectedFiles = ListUtils.union(expectedDataFiles, expectedDeleteFiles); expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - Assert.assertEquals("Metadata table should return two files", 2, actualFiles.size()); + assertThat(actualFiles).as("Metadata table should return two files").hasSize(2); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualDataFilesDs), expectedFiles, actualFiles); } - @Test + @TestTemplate public void testMetadataLogEntries() throws Exception { // Create table and insert data sql( @@ -465,8 +455,9 @@ public void testMetadataLogEntries() throws Exception { sql( "SELECT * FROM %s.metadata_log_entries WHERE latest_snapshot_id = %s", tableName, currentSnapshotId); - Assert.assertEquals( - "metadataLogEntries table should return 1 row", 1, metadataLogWithFilters.size()); + assertThat(metadataLogWithFilters) + .as("metadataLogEntries table should return 1 row") + .hasSize(1); assertEquals( "Result should match the latest snapshot entry", ImmutableList.of( @@ -487,15 +478,16 @@ public void testMetadataLogEntries() throws Exception { metadataFiles.add(tableMetadata.metadataFileLocation()); List metadataLogWithProjection = sql("SELECT file FROM %s.metadata_log_entries", tableName); - Assert.assertEquals( - "metadataLogEntries table should return 3 rows", 3, metadataLogWithProjection.size()); + assertThat(metadataLogWithProjection) + .as("metadataLogEntries table should return 3 rows") + .hasSize(3); assertEquals( "metadataLog entry should be of same file", metadataFiles.stream().map(this::row).collect(Collectors.toList()), metadataLogWithProjection); } - @Test + @TestTemplate public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception { // Create table and insert data sql( @@ -545,7 +537,7 @@ public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception { List expectedFiles = expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - Assert.assertEquals("actualFiles size should be 2", 2, actualFiles.size()); + assertThat(actualFiles).as("actualFiles size should be 2").hasSize(2); TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(0), actualFiles.get(0)); @@ -553,13 +545,12 @@ public void testFilesTableTimeTravelWithSchemaEvolution() throws Exception { TestHelpers.assertEqualsSafe( TestHelpers.nonDerivedSchema(actualFilesDs), expectedFiles.get(1), actualFiles.get(1)); - Assert.assertEquals( - "expectedFiles and actualFiles size should be the same", - actualFiles.size(), - expectedFiles.size()); + assertThat(actualFiles) + .as("expectedFiles and actualFiles size should be the same") + .hasSameSizeAs(expectedFiles); } - @Test + @TestTemplate public void testSnapshotReferencesMetatable() throws Exception { // Create table and insert data sql( @@ -605,43 +596,64 @@ public void testSnapshotReferencesMetatable() throws Exception { .commit(); // Check refs table List references = spark.sql("SELECT * FROM " + tableName + ".refs").collectAsList(); - Assert.assertEquals("Refs table should return 3 rows", 3, references.size()); + assertThat(references).as("Refs table should return 3 rows").hasSize(3); List branches = spark.sql("SELECT * FROM " + tableName + ".refs WHERE type='BRANCH'").collectAsList(); - Assert.assertEquals("Refs table should return 2 branches", 2, branches.size()); + assertThat(branches).as("Refs table should return 2 branches").hasSize(2); List tags = spark.sql("SELECT * FROM " + tableName + ".refs WHERE type='TAG'").collectAsList(); - Assert.assertEquals("Refs table should return 1 tag", 1, tags.size()); + assertThat(tags).as("Refs table should return 1 tag").hasSize(1); // Check branch entries in refs table List mainBranch = spark .sql("SELECT * FROM " + tableName + ".refs WHERE name = 'main' AND type='BRANCH'") .collectAsList(); - Assert.assertEquals("main", mainBranch.get(0).getAs("name")); - Assert.assertEquals("BRANCH", mainBranch.get(0).getAs("type")); - Assert.assertEquals(currentSnapshotId, mainBranch.get(0).getAs("snapshot_id")); + assertThat(mainBranch) + .hasSize(1) + .containsExactly(RowFactory.create("main", "BRANCH", currentSnapshotId, null, null, null)); + assertThat(mainBranch.get(0).schema().fieldNames()) + .containsExactly( + "name", + "type", + "snapshot_id", + "max_reference_age_in_ms", + "min_snapshots_to_keep", + "max_snapshot_age_in_ms"); List testBranch = spark .sql("SELECT * FROM " + tableName + ".refs WHERE name = 'testBranch' AND type='BRANCH'") .collectAsList(); - Assert.assertEquals("testBranch", testBranch.get(0).getAs("name")); - Assert.assertEquals("BRANCH", testBranch.get(0).getAs("type")); - Assert.assertEquals(currentSnapshotId, testBranch.get(0).getAs("snapshot_id")); - Assert.assertEquals(Long.valueOf(10), testBranch.get(0).getAs("max_reference_age_in_ms")); - Assert.assertEquals(Integer.valueOf(20), testBranch.get(0).getAs("min_snapshots_to_keep")); - Assert.assertEquals(Long.valueOf(30), testBranch.get(0).getAs("max_snapshot_age_in_ms")); + assertThat(testBranch) + .hasSize(1) + .containsExactly( + RowFactory.create("testBranch", "BRANCH", currentSnapshotId, 10L, 20L, 30L)); + assertThat(testBranch.get(0).schema().fieldNames()) + .containsExactly( + "name", + "type", + "snapshot_id", + "max_reference_age_in_ms", + "min_snapshots_to_keep", + "max_snapshot_age_in_ms"); // Check tag entries in refs table List testTag = spark .sql("SELECT * FROM " + tableName + ".refs WHERE name = 'testTag' AND type='TAG'") .collectAsList(); - Assert.assertEquals("testTag", testTag.get(0).getAs("name")); - Assert.assertEquals("TAG", testTag.get(0).getAs("type")); - Assert.assertEquals(currentSnapshotId, testTag.get(0).getAs("snapshot_id")); - Assert.assertEquals(Long.valueOf(50), testTag.get(0).getAs("max_reference_age_in_ms")); + assertThat(testTag) + .hasSize(1) + .containsExactly(RowFactory.create("testTag", "TAG", currentSnapshotId, 50L, null, null)); + assertThat(testTag.get(0).schema().fieldNames()) + .containsExactly( + "name", + "type", + "snapshot_id", + "max_reference_age_in_ms", + "min_snapshots_to_keep", + "max_snapshot_age_in_ms"); // Check projection in refs table List testTagProjection = @@ -651,12 +663,12 @@ public void testSnapshotReferencesMetatable() throws Exception { + tableName + ".refs where type='TAG'") .collectAsList(); - Assert.assertEquals("testTag", testTagProjection.get(0).getAs("name")); - Assert.assertEquals("TAG", testTagProjection.get(0).getAs("type")); - Assert.assertEquals(currentSnapshotId, testTagProjection.get(0).getAs("snapshot_id")); - Assert.assertEquals( - Long.valueOf(50), testTagProjection.get(0).getAs("max_reference_age_in_ms")); - Assert.assertNull(testTagProjection.get(0).getAs("min_snapshots_to_keep")); + assertThat(testTagProjection) + .hasSize(1) + .containsExactly(RowFactory.create("testTag", "TAG", currentSnapshotId, 50L, null)); + assertThat(testTagProjection.get(0).schema().fieldNames()) + .containsExactly( + "name", "type", "snapshot_id", "max_reference_age_in_ms", "min_snapshots_to_keep"); List mainBranchProjection = spark @@ -665,21 +677,23 @@ public void testSnapshotReferencesMetatable() throws Exception { + tableName + ".refs WHERE name = 'main' AND type = 'BRANCH'") .collectAsList(); - Assert.assertEquals("main", mainBranchProjection.get(0).getAs("name")); - Assert.assertEquals("BRANCH", mainBranchProjection.get(0).getAs("type")); + assertThat(mainBranchProjection) + .hasSize(1) + .containsExactly(RowFactory.create("main", "BRANCH")); + assertThat(mainBranchProjection.get(0).schema().fieldNames()).containsExactly("name", "type"); List testBranchProjection = spark .sql( - "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM " + "SELECT name, type, snapshot_id, max_reference_age_in_ms FROM " + tableName + ".refs WHERE name = 'testBranch' AND type = 'BRANCH'") .collectAsList(); - Assert.assertEquals("testBranch", testBranchProjection.get(0).getAs("name")); - Assert.assertEquals("BRANCH", testBranchProjection.get(0).getAs("type")); - Assert.assertEquals(currentSnapshotId, testBranchProjection.get(0).getAs("snapshot_id")); - Assert.assertEquals( - Long.valueOf(10), testBranchProjection.get(0).getAs("max_reference_age_in_ms")); + assertThat(testBranchProjection) + .hasSize(1) + .containsExactly(RowFactory.create("testBranch", "BRANCH", currentSnapshotId, 10L)); + assertThat(testBranchProjection.get(0).schema().fieldNames()) + .containsExactly("name", "type", "snapshot_id", "max_reference_age_in_ms"); } /** @@ -724,7 +738,7 @@ private boolean partitionMatch(Record file, String partValue) { return partValue.equals(partition.get(0).toString()); } - @Test + @TestTemplate public void metadataLogEntriesAfterReplacingTable() throws Exception { sql( "CREATE TABLE %s (id bigint, data string) " diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java index 00130b4d7e14..ce609450c097 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java @@ -20,9 +20,12 @@ import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.commons.lang3.StringUtils; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -31,13 +34,12 @@ import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestStoragePartitionedJoinsInRowLevelOperations extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStoragePartitionedJoinsInRowLevelOperations extends ExtensionsTestBase { private static final String OTHER_TABLE_NAME = "other_table"; @@ -68,7 +70,7 @@ public class TestStoragePartitionedJoinsInRowLevelOperations extends SparkExtens SparkSQLProperties.PRESERVE_DATA_GROUPING, "true"); - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -79,23 +81,18 @@ public static Object[][] parameters() { }; } - public TestStoragePartitionedJoinsInRowLevelOperations( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); sql("DROP TABLE IF EXISTS %s", tableName(OTHER_TABLE_NAME)); } - @Test + @TestTemplate public void testCopyOnWriteDeleteWithoutShuffles() { checkDelete(COPY_ON_WRITE); } - @Test + @TestTemplate public void testMergeOnReadDeleteWithoutShuffles() { checkDelete(MERGE_ON_READ); } @@ -139,10 +136,10 @@ private void checkDelete(RowLevelOperationMode mode) { String planAsString = plan.toString(); if (mode == COPY_ON_WRITE) { int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); - Assert.assertEquals("Should be 1 shuffle with SPJ", 1, actualNumShuffles); - Assertions.assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + assertThat(actualNumShuffles).as("Should be 1 shuffle with SPJ").isEqualTo(1); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); } else { - Assertions.assertThat(planAsString).doesNotContain("Exchange"); + assertThat(planAsString).doesNotContain("Exchange"); } }); @@ -158,12 +155,12 @@ private void checkDelete(RowLevelOperationMode mode) { sql("SELECT * FROM %s ORDER BY id, salary", tableName)); } - @Test + @TestTemplate public void testCopyOnWriteUpdateWithoutShuffles() { checkUpdate(COPY_ON_WRITE); } - @Test + @TestTemplate public void testMergeOnReadUpdateWithoutShuffles() { checkUpdate(MERGE_ON_READ); } @@ -207,10 +204,10 @@ private void checkUpdate(RowLevelOperationMode mode) { String planAsString = plan.toString(); if (mode == COPY_ON_WRITE) { int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); - Assert.assertEquals("Should be 1 shuffle with SPJ", 1, actualNumShuffles); - Assertions.assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + assertThat(actualNumShuffles).as("Should be 1 shuffle with SPJ").isEqualTo(1); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); } else { - Assertions.assertThat(planAsString).doesNotContain("Exchange"); + assertThat(planAsString).doesNotContain("Exchange"); } }); @@ -227,22 +224,22 @@ private void checkUpdate(RowLevelOperationMode mode) { sql("SELECT * FROM %s ORDER BY id, salary", tableName)); } - @Test + @TestTemplate public void testCopyOnWriteMergeWithoutShuffles() { checkMerge(COPY_ON_WRITE, false /* with ON predicate */); } - @Test + @TestTemplate public void testCopyOnWriteMergeWithoutShufflesWithPredicate() { checkMerge(COPY_ON_WRITE, true /* with ON predicate */); } - @Test + @TestTemplate public void testMergeOnReadMergeWithoutShuffles() { checkMerge(MERGE_ON_READ, false /* with ON predicate */); } - @Test + @TestTemplate public void testMergeOnReadMergeWithoutShufflesWithPredicate() { checkMerge(MERGE_ON_READ, true /* with ON predicate */); } @@ -294,10 +291,10 @@ private void checkMerge(RowLevelOperationMode mode, boolean withPredicate) { String planAsString = plan.toString(); if (mode == COPY_ON_WRITE) { int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); - Assert.assertEquals("Should be 1 shuffle with SPJ", 1, actualNumShuffles); - Assertions.assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + assertThat(actualNumShuffles).as("Should be 1 shuffle with SPJ").isEqualTo(1); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); } else { - Assertions.assertThat(planAsString).doesNotContain("Exchange"); + assertThat(planAsString).doesNotContain("Exchange"); } }); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java index 7f2857cce0b9..f6102bab69b0 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java @@ -37,9 +37,11 @@ import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToHourOrdinal; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToMonthOrdinal; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToYearOrdinal; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; -import java.util.Map; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.spark.source.PlanUtils; @@ -49,19 +51,15 @@ import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runners.Parameterized; - -public class TestSystemFunctionPushDownDQL extends SparkExtensionsTestBase { - public TestSystemFunctionPushDownDQL( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } +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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSystemFunctionPushDownDQL extends ExtensionsTestBase { - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -72,23 +70,24 @@ public static Object[][] parameters() { }; } - @Before + @BeforeEach public void before() { + super.before(); sql("USE %s", catalogName); } - @After + @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); } - @Test + @TestTemplate public void testYearsFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testYearsFunction(false); } - @Test + @TestTemplate public void testYearsFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "years(ts)"); testYearsFunction(true); @@ -107,16 +106,16 @@ private void testYearsFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, equal(year("ts"), targetYears)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual).hasSize(5); } - @Test + @TestTemplate public void testMonthsFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testMonthsFunction(false); } - @Test + @TestTemplate public void testMonthsFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "months(ts)"); testMonthsFunction(true); @@ -135,16 +134,16 @@ private void testMonthsFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, greaterThan(month("ts"), targetMonths)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual).hasSize(5); } - @Test + @TestTemplate public void testDaysFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testDaysFunction(false); } - @Test + @TestTemplate public void testDaysFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "days(ts)"); testDaysFunction(true); @@ -165,16 +164,16 @@ private void testDaysFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, lessThan(day("ts"), targetDays)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual).hasSize(5); } - @Test + @TestTemplate public void testHoursFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testHoursFunction(false); } - @Test + @TestTemplate public void testHoursFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "hours(ts)"); testHoursFunction(true); @@ -193,16 +192,16 @@ private void testHoursFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, greaterThanOrEqual(hour("ts"), targetHours)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(8); + assertThat(actual).hasSize(8); } - @Test + @TestTemplate public void testBucketLongFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testBucketLongFunction(false); } - @Test + @TestTemplate public void testBucketLongFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "bucket(5, id)"); testBucketLongFunction(true); @@ -221,16 +220,16 @@ private void testBucketLongFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, lessThanOrEqual(bucket("id", 5), target)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual).hasSize(5); } - @Test + @TestTemplate public void testBucketStringFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testBucketStringFunction(false); } - @Test + @TestTemplate public void testBucketStringFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "bucket(5, data)"); testBucketStringFunction(true); @@ -249,16 +248,16 @@ private void testBucketStringFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, notEqual(bucket("data", 5), target)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(8); + assertThat(actual).hasSize(8); } - @Test + @TestTemplate public void testTruncateFunctionOnUnpartitionedTable() { createUnpartitionedTable(spark, tableName); testTruncateFunction(false); } - @Test + @TestTemplate public void testTruncateFunctionOnPartitionedTable() { createPartitionedTable(spark, tableName, "truncate(4, data)"); testTruncateFunction(true); @@ -278,7 +277,7 @@ private void testTruncateFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, equal(truncate("data", 4), target)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual).hasSize(5); } private void checkExpressions( @@ -286,18 +285,18 @@ private void checkExpressions( List staticInvokes = PlanUtils.collectSparkExpressions( optimizedPlan, expression -> expression instanceof StaticInvoke); - Assertions.assertThat(staticInvokes).isEmpty(); + assertThat(staticInvokes).isEmpty(); List applyExpressions = PlanUtils.collectSparkExpressions( optimizedPlan, expression -> expression instanceof ApplyFunctionExpression); if (partitioned) { - Assertions.assertThat(applyExpressions).isEmpty(); + assertThat(applyExpressions).isEmpty(); } else { - Assertions.assertThat(applyExpressions.size()).isEqualTo(1); + assertThat(applyExpressions).hasSize(1); ApplyFunctionExpression expression = (ApplyFunctionExpression) applyExpressions.get(0); - Assertions.assertThat(expression.name()).isEqualTo(expectedFunctionName); + assertThat(expression.name()).isEqualTo(expectedFunctionName); } } @@ -305,9 +304,9 @@ private void checkPushedFilters( LogicalPlan optimizedPlan, org.apache.iceberg.expressions.Expression expected) { List pushedFilters = PlanUtils.collectPushDownFilters(optimizedPlan); - Assertions.assertThat(pushedFilters.size()).isEqualTo(1); + assertThat(pushedFilters).hasSize(1); org.apache.iceberg.expressions.Expression actual = pushedFilters.get(0); - Assertions.assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)) + assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)) .as("Pushed filter should match") .isTrue(); } 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 ac02561a3c76..786d0bd1c660 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,11 +23,12 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; -import java.util.Map; import java.util.Random; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; @@ -51,30 +52,32 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.catalog.SessionCatalog; import org.assertj.core.api.InstanceOfAssertFactories; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; -import org.junit.runners.Parameterized; +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; -public class TestViews extends SparkExtensionsTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestViews extends ExtensionsTestBase { private static final Namespace NAMESPACE = Namespace.of("default"); private final String tableName = "table"; - @Before + @BeforeEach 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); } - @After + @AfterEach public void removeTable() { sql("USE %s", catalogName); sql("DROP TABLE IF EXISTS %s", tableName); } - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { { @@ -85,11 +88,7 @@ public static Object[][] parameters() { }; } - public TestViews(String catalog, String implementation, Map properties) { - super(catalog, implementation, properties); - } - - @Test + @TestTemplate public void readFromView() throws NoSuchTableException { insertRows(10); String viewName = "simpleView"; @@ -115,7 +114,7 @@ public void readFromView() throws NoSuchTableException { .containsExactlyInAnyOrderElementsOf(expected); } - @Test + @TestTemplate public void readFromTrinoView() throws NoSuchTableException { insertRows(10); String viewName = "trinoView"; @@ -140,7 +139,7 @@ public void readFromTrinoView() throws NoSuchTableException { .containsExactlyInAnyOrderElementsOf(expected); } - @Test + @TestTemplate public void readFromMultipleViews() throws NoSuchTableException { insertRows(6); String viewName = "firstView"; @@ -173,7 +172,7 @@ public void readFromMultipleViews() throws NoSuchTableException { .containsExactlyInAnyOrder(row(4), row(5), row(6)); } - @Test + @TestTemplate public void readFromViewUsingNonExistingTable() throws NoSuchTableException { insertRows(10); String viewName = "viewWithNonExistingTable"; @@ -197,7 +196,7 @@ public void readFromViewUsingNonExistingTable() throws NoSuchTableException { catalogName, NAMESPACE)); } - @Test + @TestTemplate public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableException { insertRows(10); String viewName = "viewWithNonExistingColumn"; @@ -219,7 +218,7 @@ public void readFromViewUsingNonExistingTableColumn() throws NoSuchTableExceptio "A column or function parameter with name `non_existing` cannot be resolved"); } - @Test + @TestTemplate public void readFromViewUsingInvalidSQL() throws NoSuchTableException { insertRows(10); String viewName = "viewWithInvalidSQL"; @@ -241,7 +240,7 @@ public void readFromViewUsingInvalidSQL() throws NoSuchTableException { String.format("The view `%s` cannot be displayed due to invalid view text", viewName)); } - @Test + @TestTemplate public void readFromViewWithStaleSchema() throws NoSuchTableException { insertRows(10); String viewName = "staleView"; @@ -267,7 +266,7 @@ public void readFromViewWithStaleSchema() throws NoSuchTableException { .hasMessageContaining("A column or function parameter with name `data` cannot be resolved"); } - @Test + @TestTemplate public void readFromViewHiddenByTempView() throws NoSuchTableException { insertRows(10); String viewName = "viewHiddenByTempView"; @@ -294,7 +293,7 @@ public void readFromViewHiddenByTempView() throws NoSuchTableException { .containsExactlyInAnyOrderElementsOf(expected); } - @Test + @TestTemplate public void readFromViewWithGlobalTempView() throws NoSuchTableException { insertRows(10); String viewName = "viewWithGlobalTempView"; @@ -324,7 +323,7 @@ public void readFromViewWithGlobalTempView() throws NoSuchTableException { IntStream.rangeClosed(6, 10).mapToObj(this::row).collect(Collectors.toList())); } - @Test + @TestTemplate public void readFromViewReferencingAnotherView() throws NoSuchTableException { insertRows(10); String firstView = "viewBeingReferencedInAnotherView"; @@ -355,7 +354,7 @@ public void readFromViewReferencingAnotherView() throws NoSuchTableException { .containsExactly(row(5)); } - @Test + @TestTemplate public void readFromViewReferencingTempView() throws NoSuchTableException { insertRows(10); String tempView = "tempViewBeingReferencedInAnotherView"; @@ -391,7 +390,7 @@ public void readFromViewReferencingTempView() throws NoSuchTableException { .hasMessageContaining("cannot be found"); } - @Test + @TestTemplate public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTableException { insertRows(10); String innerViewName = "inner_view"; @@ -439,7 +438,7 @@ public void readFromViewReferencingAnotherViewHiddenByTempView() throws NoSuchTa .containsExactlyInAnyOrderElementsOf(expectedViewRows); } - @Test + @TestTemplate public void readFromViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); String globalTempView = "globalTempViewBeingReferenced"; @@ -477,7 +476,7 @@ public void readFromViewReferencingGlobalTempView() throws NoSuchTableException .hasMessageContaining("cannot be found"); } - @Test + @TestTemplate public void readFromViewReferencingTempFunction() throws NoSuchTableException { insertRows(10); String viewName = viewName("viewReferencingTempFunction"); @@ -510,7 +509,7 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { .hasMessageContaining("cannot be found"); } - @Test + @TestTemplate public void readFromViewWithCTE() throws NoSuchTableException { insertRows(10); String viewName = "viewWithCTE"; @@ -533,7 +532,7 @@ public void readFromViewWithCTE() throws NoSuchTableException { assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); } - @Test + @TestTemplate public void rewriteFunctionIdentifier() { String viewName = "rewriteFunctionIdentifier"; String sql = "SELECT iceberg_version() AS version"; @@ -559,7 +558,7 @@ public void rewriteFunctionIdentifier() { .containsExactly(row(IcebergBuild.version())); } - @Test + @TestTemplate public void builtinFunctionIdentifierNotRewritten() { String viewName = "builtinFunctionIdentifierNotRewritten"; String sql = "SELECT trim(' abc ') AS result"; @@ -578,7 +577,7 @@ public void builtinFunctionIdentifierNotRewritten() { assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row("abc")); } - @Test + @TestTemplate public void rewriteFunctionIdentifierWithNamespace() { String viewName = "rewriteFunctionIdentifierWithNamespace"; String sql = "SELECT system.bucket(100, 'a') AS bucket_result, 'a' AS value"; @@ -605,7 +604,7 @@ public void rewriteFunctionIdentifierWithNamespace() { .containsExactly(row(50, "a")); } - @Test + @TestTemplate public void fullFunctionIdentifier() { String viewName = "fullFunctionIdentifier"; String sql = @@ -629,7 +628,7 @@ public void fullFunctionIdentifier() { .containsExactly(row(50, "a")); } - @Test + @TestTemplate public void fullFunctionIdentifierNotRewrittenLoadFailure() { String viewName = "fullFunctionIdentifierNotRewrittenLoadFailure"; String sql = "SELECT spark_catalog.system.bucket(100, 'a') AS bucket_result, 'a' AS value"; @@ -674,7 +673,7 @@ private Catalog tableCatalog() { return Spark3Util.loadIcebergCatalog(spark, catalogName); } - @Test + @TestTemplate public void renameView() throws NoSuchTableException { insertRows(10); String viewName = viewName("originalView"); @@ -700,7 +699,7 @@ public void renameView() throws NoSuchTableException { .containsExactlyInAnyOrderElementsOf(expected); } - @Test + @TestTemplate public void renameViewHiddenByTempView() throws NoSuchTableException { insertRows(10); String viewName = viewName("originalView"); @@ -739,7 +738,7 @@ public void renameViewHiddenByTempView() throws NoSuchTableException { assertThat(viewCatalog.viewExists(TableIdentifier.of(NAMESPACE, renamedView))).isTrue(); } - @Test + @TestTemplate public void renameViewToDifferentTargetCatalog() { String viewName = viewName("originalView"); String renamedView = viewName("renamedView"); @@ -761,14 +760,14 @@ public void renameViewToDifferentTargetCatalog() { "Cannot move view between catalogs: from=spark_with_views and to=spark_catalog"); } - @Test + @TestTemplate public void renameNonExistingView() { assertThatThrownBy(() -> sql("ALTER VIEW non_existing RENAME TO target")) .isInstanceOf(AnalysisException.class) .hasMessageContaining("The table or view `non_existing` cannot be found"); } - @Test + @TestTemplate public void renameViewTargetAlreadyExistsAsView() { String viewName = viewName("renameViewSource"); String target = viewName("renameViewTarget"); @@ -798,7 +797,7 @@ public void renameViewTargetAlreadyExistsAsView() { String.format("Cannot create view default.%s because it already exists", target)); } - @Test + @TestTemplate public void renameViewTargetAlreadyExistsAsTable() { String viewName = viewName("renameViewSource"); String target = viewName("renameViewTarget"); @@ -821,7 +820,7 @@ public void renameViewTargetAlreadyExistsAsTable() { String.format("Cannot create view default.%s because it already exists", target)); } - @Test + @TestTemplate public void dropView() { String viewName = "viewToBeDropped"; String sql = String.format("SELECT id FROM %s", tableName); @@ -843,14 +842,14 @@ public void dropView() { assertThat(viewCatalog.viewExists(identifier)).isFalse(); } - @Test + @TestTemplate public void dropNonExistingView() { assertThatThrownBy(() -> sql("DROP VIEW non_existing")) .isInstanceOf(AnalysisException.class) .hasMessageContaining("The view %s.%s cannot be found", NAMESPACE, "non_existing"); } - @Test + @TestTemplate public void dropViewIfExists() { String viewName = "viewToBeDropped"; String sql = String.format("SELECT id FROM %s", tableName); @@ -875,7 +874,7 @@ public void dropViewIfExists() { } /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ - @Test + @TestTemplate public void dropGlobalTempView() { String globalTempView = "globalViewToBeDropped"; sql("CREATE GLOBAL TEMPORARY VIEW %s AS SELECT id FROM %s", globalTempView, tableName); @@ -886,7 +885,7 @@ public void dropGlobalTempView() { } /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ - @Test + @TestTemplate public void dropTempView() { String tempView = "tempViewToBeDropped"; sql("CREATE TEMPORARY VIEW %s AS SELECT id FROM %s", tempView, tableName); @@ -897,7 +896,7 @@ public void dropTempView() { } /** The purpose of this test is mainly to make sure that normal view deletion isn't messed up */ - @Test + @TestTemplate public void dropV1View() { String v1View = "v1ViewToBeDropped"; sql("USE spark_catalog"); @@ -928,7 +927,7 @@ private String viewName(String viewName) { return viewName + new Random().nextInt(1000000); } - @Test + @TestTemplate public void createViewIfNotExists() { String viewName = "viewThatAlreadyExists"; sql("CREATE VIEW %s AS SELECT id FROM %s", viewName, tableName); @@ -945,7 +944,7 @@ public void createViewIfNotExists() { () -> sql("CREATE VIEW IF NOT EXISTS %s AS SELECT id FROM %s", viewName, tableName)); } - @Test + @TestTemplate public void createOrReplaceView() throws NoSuchTableException { insertRows(6); String viewName = viewName("simpleView"); @@ -961,14 +960,14 @@ public void createOrReplaceView() throws NoSuchTableException { .containsExactlyInAnyOrder(row(4), row(5), row(6)); } - @Test + @TestTemplate public void createViewWithInvalidSQL() { assertThatThrownBy(() -> sql("CREATE VIEW simpleViewWithInvalidSQL AS invalid SQL")) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Syntax error"); } - @Test + @TestTemplate public void createViewReferencingTempView() throws NoSuchTableException { insertRows(10); String tempView = "temporaryViewBeingReferencedInAnotherView"; @@ -987,7 +986,7 @@ public void createViewReferencingTempView() throws NoSuchTableException { .hasMessageContaining(tempView); } - @Test + @TestTemplate public void createViewReferencingGlobalTempView() throws NoSuchTableException { insertRows(10); String globalTempView = "globalTemporaryViewBeingReferenced"; @@ -1011,7 +1010,7 @@ public void createViewReferencingGlobalTempView() throws NoSuchTableException { .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); } - @Test + @TestTemplate public void createViewReferencingTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); String functionName = "test_avg_func"; @@ -1030,7 +1029,7 @@ public void createViewReferencingTempFunction() { .hasMessageContaining(functionName); } - @Test + @TestTemplate public void createViewReferencingQualifiedTempFunction() { String viewName = viewName("viewReferencingTemporaryFunction"); String functionName = "test_avg_func_qualified"; @@ -1061,7 +1060,7 @@ public void createViewReferencingQualifiedTempFunction() { .hasMessageContaining(String.format("`%s`.`%s`", NAMESPACE, functionName)); } - @Test + @TestTemplate public void createViewUsingNonExistingTable() { assertThatThrownBy( () -> sql("CREATE VIEW viewWithNonExistingTable AS SELECT id FROM non_existing")) @@ -1069,7 +1068,7 @@ public void createViewUsingNonExistingTable() { .hasMessageContaining("The table or view `non_existing` cannot be found"); } - @Test + @TestTemplate public void createViewWithMismatchedColumnCounts() { String viewName = "viewWithMismatchedColumnCounts"; @@ -1092,7 +1091,7 @@ public void createViewWithMismatchedColumnCounts() { .hasMessageContaining("Data columns: id, data"); } - @Test + @TestTemplate public void createViewWithColumnAliases() throws NoSuchTableException { insertRows(6); String viewName = "viewWithColumnAliases"; @@ -1128,7 +1127,7 @@ public void createViewWithColumnAliases() throws NoSuchTableException { .containsExactlyInAnyOrder(row(1), row(2), row(3)); } - @Test + @TestTemplate public void createViewWithDuplicateColumnNames() { assertThatThrownBy( () -> @@ -1139,7 +1138,7 @@ public void createViewWithDuplicateColumnNames() { .hasMessageContaining("The column `new_id` already exists"); } - @Test + @TestTemplate public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableException { insertRows(3); String viewName = "viewWithDuplicateQueryColumnNames"; @@ -1157,7 +1156,7 @@ public void createViewWithDuplicateQueryColumnNames() throws NoSuchTableExceptio .containsExactlyInAnyOrder(row(1, 1), row(2, 2), row(3, 3)); } - @Test + @TestTemplate public void createViewWithCTE() throws NoSuchTableException { insertRows(10); String viewName = "simpleViewWithCTE"; @@ -1172,7 +1171,7 @@ public void createViewWithCTE() throws NoSuchTableException { assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); } - @Test + @TestTemplate public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTableException { insertRows(10); String viewName = "viewWithConflictingNamesForCTEAndTempView"; @@ -1191,7 +1190,7 @@ public void createViewWithConflictingNamesForCTEAndTempView() throws NoSuchTable assertThat(sql("SELECT * FROM %s", viewName)).hasSize(1).containsExactly(row(10, 1L)); } - @Test + @TestTemplate public void createViewWithCTEReferencingTempView() { String viewName = "viewWithCTEReferencingTempView"; String tempViewInCTE = "tempViewInCTE"; @@ -1211,7 +1210,7 @@ public void createViewWithCTEReferencingTempView() { .hasMessageContaining(tempViewInCTE); } - @Test + @TestTemplate public void createViewWithCTEReferencingTempFunction() { String viewName = "viewWithCTEReferencingTempFunction"; String functionName = "avg_function_in_cte"; @@ -1233,7 +1232,7 @@ public void createViewWithCTEReferencingTempFunction() { .hasMessageContaining(functionName); } - @Test + @TestTemplate public void createViewWithNonExistingQueryColumn() { assertThatThrownBy( () -> @@ -1245,7 +1244,7 @@ public void createViewWithNonExistingQueryColumn() { "A column or function parameter with name `non_existing` cannot be resolved"); } - @Test + @TestTemplate public void createViewWithSubqueryExpressionUsingTempView() { String viewName = "viewWithSubqueryExpression"; String tempView = "simpleTempView"; @@ -1262,7 +1261,7 @@ public void createViewWithSubqueryExpressionUsingTempView() { .hasMessageContaining(tempView); } - @Test + @TestTemplate public void createViewWithSubqueryExpressionUsingGlobalTempView() { String viewName = "simpleViewWithSubqueryExpression"; String globalTempView = "simpleGlobalTempView"; @@ -1283,7 +1282,7 @@ public void createViewWithSubqueryExpressionUsingGlobalTempView() { .hasMessageContaining(String.format("%s.%s", "global_temp", globalTempView)); } - @Test + @TestTemplate public void createViewWithSubqueryExpressionUsingTempFunction() { String viewName = viewName("viewWithSubqueryExpression"); String functionName = "avg_function_in_subquery"; @@ -1304,7 +1303,7 @@ public void createViewWithSubqueryExpressionUsingTempFunction() { .hasMessageContaining(functionName); } - @Test + @TestTemplate public void createViewWithSubqueryExpressionInFilterThatIsRewritten() throws NoSuchTableException { insertRows(5); @@ -1329,7 +1328,7 @@ public void createViewWithSubqueryExpressionInFilterThatIsRewritten() .containsExactly(row(5)); } - @Test + @TestTemplate public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSuchTableException { insertRows(3); String viewName = viewName("viewWithSubqueryExpression"); @@ -1354,7 +1353,7 @@ public void createViewWithSubqueryExpressionInQueryThatIsRewritten() throws NoSu .containsExactly(row(3), row(3), row(3)); } - @Test + @TestTemplate public void describeView() { String viewName = "describeView"; @@ -1363,7 +1362,7 @@ public void describeView() { .containsExactly(row("id", "int", ""), row("data", "string", "")); } - @Test + @TestTemplate public void describeExtendedView() { String viewName = "describeExtendedView"; String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); @@ -1388,7 +1387,7 @@ public void describeExtendedView() { "")); } - @Test + @TestTemplate public void showViewProperties() { String viewName = "showViewProps"; @@ -1399,7 +1398,7 @@ public void showViewProperties() { .contains(row("key1", "val1"), row("key2", "val2")); } - @Test + @TestTemplate public void showViewPropertiesByKey() { String viewName = "showViewPropsByKey"; @@ -1418,7 +1417,7 @@ public void showViewPropertiesByKey() { catalogName, NAMESPACE, viewName))); } - @Test + @TestTemplate public void showViews() throws NoSuchTableException { insertRows(6); String sql = String.format("SELECT * from %s", tableName); @@ -1466,7 +1465,7 @@ public void showViews() throws NoSuchTableException { row("global_temp", "globalviewforlisting", true), tempView); } - @Test + @TestTemplate public void showViewsWithCurrentNamespace() { String namespaceOne = "show_views_ns1"; String namespaceTwo = "show_views_ns2"; @@ -1497,7 +1496,7 @@ public void showViewsWithCurrentNamespace() { assertThat(sql("SHOW VIEWS LIKE 'viewTwo*'")).contains(v2).doesNotContain(v1); } - @Test + @TestTemplate public void showCreateSimpleView() { String viewName = "showCreateSimpleView"; String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); @@ -1518,7 +1517,7 @@ public void showCreateSimpleView() { assertThat(sql("SHOW CREATE TABLE %s", viewName)).containsExactly(row(expected)); } - @Test + @TestTemplate public void showCreateComplexView() { String viewName = "showCreateComplexView"; String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); @@ -1545,7 +1544,7 @@ public void showCreateComplexView() { assertThat(sql("SHOW CREATE TABLE %s", viewName)).containsExactly(row(expected)); } - @Test + @TestTemplate public void alterViewSetProperties() { String viewName = "viewWithSetProperties"; @@ -1567,7 +1566,7 @@ public void alterViewSetProperties() { .containsEntry("comment", "view comment"); } - @Test + @TestTemplate public void alterViewSetReservedProperties() { String viewName = "viewWithSetReservedProperties"; @@ -1598,7 +1597,7 @@ public void alterViewSetReservedProperties() { .hasMessageContaining("Cannot set reserved property: 'spark.query-column-names'"); } - @Test + @TestTemplate public void alterViewUnsetProperties() { String viewName = "viewWithUnsetProperties"; sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1619,7 +1618,7 @@ public void alterViewUnsetProperties() { .containsEntry("comment", "view comment"); } - @Test + @TestTemplate public void alterViewUnsetUnknownProperty() { String viewName = "viewWithUnsetUnknownProp"; sql("CREATE VIEW %s AS SELECT id FROM %s WHERE id <= 3", viewName, tableName); @@ -1633,7 +1632,7 @@ public void alterViewUnsetUnknownProperty() { () -> sql("ALTER VIEW %s UNSET TBLPROPERTIES IF EXISTS ('unknown-key')", viewName)); } - @Test + @TestTemplate public void alterViewUnsetReservedProperties() { String viewName = "viewWithUnsetReservedProperties"; @@ -1669,7 +1668,7 @@ public void alterViewUnsetReservedProperties() { .hasMessageContaining("Cannot unset reserved property: 'spark.query-column-names'"); } - @Test + @TestTemplate public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { insertRows(6); String viewName = viewName("viewWithColumnAliases"); @@ -1715,7 +1714,7 @@ public void createOrReplaceViewWithColumnAliases() throws NoSuchTableException { assertThat(second.doc()).isEqualTo("new ID"); } - @Test + @TestTemplate public void alterViewIsNotSupported() throws NoSuchTableException { insertRows(6); String viewName = "alteredView"; @@ -1733,7 +1732,7 @@ public void alterViewIsNotSupported() throws NoSuchTableException { "ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead"); } - @Test + @TestTemplate public void createOrReplaceViewKeepsViewHistory() { String viewName = viewName("viewWithHistoryAfterReplace"); String sql = String.format("SELECT id, data FROM %s WHERE id <= 3", tableName); @@ -1772,7 +1771,7 @@ public void createOrReplaceViewKeepsViewHistory() { .asStruct()); } - @Test + @TestTemplate public void replacingTrinoViewShouldFail() { String viewName = viewName("trinoView"); String sql = String.format("SELECT id FROM %s", tableName); @@ -1795,7 +1794,7 @@ public void replacingTrinoViewShouldFail() { + "New dialects: [spark]"); } - @Test + @TestTemplate public void replacingTrinoAndSparkViewShouldFail() { String viewName = viewName("trinoAndSparkView"); String sql = String.format("SELECT id FROM %s", tableName); @@ -1819,7 +1818,7 @@ public void replacingTrinoAndSparkViewShouldFail() { + "New dialects: [spark]"); } - @Test + @TestTemplate public void replacingViewWithDialectDropAllowed() { String viewName = viewName("trinoView"); String sql = String.format("SELECT id FROM %s", tableName); diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 25d7e7471588..207fca3cc216 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -19,22 +19,19 @@ package org.apache.iceberg.spark; import java.io.IOException; -import java.util.Map; +import java.nio.file.Files; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.spark.extensions.SparkExtensionsTestBase; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -public class SmokeTest extends SparkExtensionsTestBase { - - public SmokeTest(String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @Before +import org.apache.iceberg.spark.extensions.ExtensionsTestBase; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class SmokeTest extends ExtensionsTestBase { + @AfterEach public void dropTable() { sql("DROP TABLE IF EXISTS %s", tableName); } @@ -42,30 +39,32 @@ public void dropTable() { // Run through our Doc's Getting Started Example // TODO Update doc example so that it can actually be run, modifications were required for this // test suite to run - @Test + @TestTemplate public void testGettingStarted() throws IOException { // Creating a table sql("CREATE TABLE %s (id bigint, data string) USING iceberg", tableName); // Writing sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); - Assert.assertEquals( - "Should have inserted 3 rows", 3L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); + Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + .as("Should have inserted 3 rows") + .isEqualTo(3L); sql("DROP TABLE IF EXISTS source PURGE"); sql( "CREATE TABLE source (id bigint, data string) USING parquet LOCATION '%s'", - temp.newFolder()); + Files.createTempDirectory(temp, "junit")); sql("INSERT INTO source VALUES (10, 'd'), (11, 'ee')"); sql("INSERT INTO %s SELECT id, data FROM source WHERE length(data) = 1", tableName); - Assert.assertEquals( - "Table should now have 4 rows", 4L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); + Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + .as("Table should now have 4 rows") + .isEqualTo(4L); sql("DROP TABLE IF EXISTS updates PURGE"); sql( "CREATE TABLE updates (id bigint, data string) USING parquet LOCATION '%s'", - temp.newFolder()); + Files.createTempDirectory(temp, "junit")); sql("INSERT INTO updates VALUES (1, 'x'), (2, 'x'), (4, 'z')"); sql( @@ -73,31 +72,31 @@ public void testGettingStarted() throws IOException { + "WHEN MATCHED THEN UPDATE SET t.data = u.data\n" + "WHEN NOT MATCHED THEN INSERT *", tableName); - Assert.assertEquals( - "Table should now have 5 rows", 5L, scalarSql("SELECT COUNT(*) FROM %s", tableName)); - Assert.assertEquals( - "Record 1 should now have data x", - "x", - scalarSql("SELECT data FROM %s WHERE id = 1", tableName)); + Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + .as("Table should now have 5 rows") + .isEqualTo(5L); + Assertions.assertThat(scalarSql("SELECT data FROM %s WHERE id = 1", tableName)) + .as("Record 1 should now have data x") + .isEqualTo("x"); // Reading - Assert.assertEquals( - "There should be 2 records with data x", - 2L, - scalarSql("SELECT count(1) as count FROM %s WHERE data = 'x' GROUP BY data ", tableName)); + Assertions.assertThat( + scalarSql( + "SELECT count(1) as count FROM %s WHERE data = 'x' GROUP BY data ", tableName)) + .as("There should be 2 records with data x") + .isEqualTo(2L); // Not supported because of Spark limitation if (!catalogName.equals("spark_catalog")) { - Assert.assertEquals( - "There should be 3 snapshots", - 3L, - scalarSql("SELECT COUNT(*) FROM %s.snapshots", tableName)); + Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s.snapshots", tableName)) + .as("There should be 3 snapshots") + .isEqualTo(3L); } } // From Spark DDL Docs section - @Test - public void testAlterTable() throws NoSuchTableException { + @TestTemplate + public void testAlterTable() { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); @@ -108,7 +107,9 @@ public void testAlterTable() throws NoSuchTableException { sql("ALTER TABLE %s ADD PARTITION FIELD years(ts)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, category) AS shard", tableName); table = getTable(); - Assert.assertEquals("Table should have 4 partition fields", 4, table.spec().fields().size()); + Assertions.assertThat(table.spec().fields()) + .as("Table should have 4 partition fields") + .hasSize(4); // Drop Examples sql("ALTER TABLE %s DROP PARTITION FIELD bucket(16, id)", tableName); @@ -117,17 +118,21 @@ public void testAlterTable() throws NoSuchTableException { sql("ALTER TABLE %s DROP PARTITION FIELD shard", tableName); table = getTable(); - Assert.assertTrue("Table should be unpartitioned", table.spec().isUnpartitioned()); + Assertions.assertThat(table.spec().isUnpartitioned()) + .as("Table should be unpartitioned") + .isTrue(); // Sort order examples sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); sql("ALTER TABLE %s WRITE ORDERED BY category ASC, id DESC", tableName); sql("ALTER TABLE %s WRITE ORDERED BY category ASC NULLS LAST, id DESC NULLS FIRST", tableName); table = getTable(); - Assert.assertEquals("Table should be sorted on 2 fields", 2, table.sortOrder().fields().size()); + Assertions.assertThat(table.sortOrder().fields()) + .as("Table should be sorted on 2 fields") + .hasSize(2); } - @Test + @TestTemplate public void testCreateTable() { sql("DROP TABLE IF EXISTS %s", tableName("first")); sql("DROP TABLE IF EXISTS %s", tableName("second")); @@ -150,7 +155,9 @@ public void testCreateTable() { + "PARTITIONED BY (category)", tableName("second")); Table second = getTable("second"); - Assert.assertEquals("Should be partitioned on 1 column", 1, second.spec().fields().size()); + Assertions.assertThat(second.spec().fields()) + .as("Should be partitioned on 1 column") + .hasSize(1); sql( "CREATE TABLE %s (\n" @@ -162,7 +169,9 @@ public void testCreateTable() { + "PARTITIONED BY (bucket(16, id), days(ts), category)", tableName("third")); Table third = getTable("third"); - Assert.assertEquals("Should be partitioned on 3 columns", 3, third.spec().fields().size()); + Assertions.assertThat(third.spec().fields()) + .as("Should be partitioned on 3 columns") + .hasSize(3); } private Table getTable(String name) { From 00881e711f90c7ad43f6c962c0edef5d2e031c83 Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 27 Feb 2024 23:20:08 +0800 Subject: [PATCH 0083/1019] API: Fix typo in method comment of SortOrder and SortOrderBuilder (#9816) --- api/src/main/java/org/apache/iceberg/SortOrder.java | 2 +- .../main/java/org/apache/iceberg/SortOrderBuilder.java | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/SortOrder.java b/api/src/main/java/org/apache/iceberg/SortOrder.java index d0041cefc1c4..e98e05bd1e88 100644 --- a/api/src/main/java/org/apache/iceberg/SortOrder.java +++ b/api/src/main/java/org/apache/iceberg/SortOrder.java @@ -212,7 +212,7 @@ public Builder asc(Term term, NullOrder nullOrder) { } /** - * Add an expression term to the sort, ascending with the given null order. + * Add an expression term to the sort, descending with the given null order. * * @param term an expression term * @param nullOrder a null order (first or last) diff --git a/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java b/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java index 4dc8fa46e708..4a112e21c26c 100644 --- a/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java +++ b/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java @@ -65,7 +65,7 @@ default R asc(Term term) { R asc(Term term, NullOrder nullOrder); /** - * Add a field to the sort by field name, ascending with nulls first. + * Add a field to the sort by field name, descending with nulls first. * * @param name a field name * @return this for method chaining @@ -75,7 +75,7 @@ default R desc(String name) { } /** - * Add a field to the sort by field name, ascending with the given null order. + * Add a field to the sort by field name, descending with the given null order. * * @param name a field name * @param nullOrder a null order (first or last) @@ -86,7 +86,7 @@ default R desc(String name, NullOrder nullOrder) { } /** - * Add an expression term to the sort, ascending with nulls first. + * Add an expression term to the sort, descending with nulls first. * * @param term an expression term * @return this for method chaining @@ -96,7 +96,7 @@ default R desc(Term term) { } /** - * Add an expression term to the sort, ascending with the given null order. + * Add an expression term to the sort, descending with the given null order. * * @param term an expression term * @param nullOrder a null order (first or last) From c3006f964df6845c988e5569d86292cde02148df Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 27 Feb 2024 19:54:04 +0100 Subject: [PATCH 0084/1019] Spark: Improve error msg when function can't be loaded (#9814) --- .../java/org/apache/iceberg/spark/extensions/TestViews.java | 5 ++--- .../java/org/apache/iceberg/spark/SupportsFunctions.java | 4 +++- .../java/org/apache/iceberg/spark/TestFunctionCatalog.java | 4 ++-- 3 files changed, 7 insertions(+), 6 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 786d0bd1c660..bd611b936a59 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 @@ -504,9 +504,8 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { // reading from a view that references a TEMP FUNCTION shouldn't be possible assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("The function") - .hasMessageContaining(functionName) - .hasMessageContaining("cannot be found"); + .hasMessageStartingWith( + String.format("Cannot load function: %s.%s.%s", catalogName, NAMESPACE, functionName)); } @TestTemplate diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java index 34897d2b4c01..1aa08b02cead 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java @@ -24,6 +24,7 @@ import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import scala.Option; interface SupportsFunctions extends FunctionCatalog { @@ -58,6 +59,7 @@ default UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionExce } } - throw new NoSuchFunctionException(ident); + throw new NoSuchFunctionException( + String.format("Cannot load function: %s.%s", name(), ident), Option.empty()); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java index 5789b43e7d52..6ae22f52b593 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java @@ -87,13 +87,13 @@ public void testLoadFunctions() throws NoSuchFunctionException { asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))) .isInstanceOf(NoSuchFunctionException.class) .hasMessageStartingWith( - "[ROUTINE_NOT_FOUND] The function default.iceberg_version cannot be found."); + String.format("Cannot load function: %s.default.iceberg_version", catalogName)); Identifier undefinedFunction = Identifier.of(SYSTEM_NAMESPACE, "undefined_function"); assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) .isInstanceOf(NoSuchFunctionException.class) .hasMessageStartingWith( - "[ROUTINE_NOT_FOUND] The function system.undefined_function cannot be found."); + String.format("Cannot load function: %s.system.undefined_function", catalogName)); assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) .isInstanceOf(AnalysisException.class) From 067ca7fd3c5fa3eb5254fd4b87135d1c7061d165 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 28 Feb 2024 17:28:51 +0900 Subject: [PATCH 0085/1019] Spark: Remove/migrate remaining JUnit4 tests (#9817) --- .../extensions/SparkExtensionsTestBase.java | 71 ----- .../iceberg/spark/SparkCatalogTestBase.java | 62 ---- .../apache/iceberg/spark/SparkTestBase.java | 287 ------------------ .../spark/SparkTestBaseWithCatalog.java | 130 -------- .../spark/functions/TestSparkFunctions.java | 2 +- .../source/TestSparkMetadataColumns.java | 9 +- 6 files changed, 6 insertions(+), 555 deletions(-) delete mode 100644 spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java delete mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.java deleted file mode 100644 index 4f137f5b8dac..000000000000 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkExtensionsTestBase.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.extensions; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; - -import java.util.Map; -import java.util.Random; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.spark.SparkCatalogTestBase; -import org.apache.iceberg.spark.SparkTestBase; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.internal.SQLConf; -import org.junit.BeforeClass; - -public abstract class SparkExtensionsTestBase extends SparkCatalogTestBase { - - private static final Random RANDOM = ThreadLocalRandom.current(); - - public SparkExtensionsTestBase( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } - - @BeforeClass - public static void startMetastoreAndSpark() { - SparkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - SparkTestBase.hiveConf = metastore.hiveConf(); - - SparkTestBase.spark = - SparkSession.builder() - .master("local[2]") - .config("spark.testing", "true") - .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") - .config("spark.sql.extensions", IcebergSparkSessionExtensions.class.getName()) - .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) - .config("spark.sql.shuffle.partitions", "4") - .config("spark.sql.hive.metastorePartitionPruningFallbackOnException", "true") - .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") - .config( - SQLConf.ADAPTIVE_EXECUTION_ENABLED().key(), String.valueOf(RANDOM.nextBoolean())) - .enableHiveSupport() - .getOrCreate(); - - SparkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java deleted file mode 100644 index 89323c26100c..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkCatalogTestBase.java +++ /dev/null @@ -1,62 +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; - -import java.util.Map; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class SparkCatalogTestBase extends SparkTestBaseWithCatalog { - - // these parameters are broken out to avoid changes that need to modify lots of test suites - @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") - public static Object[][] parameters() { - return new Object[][] { - { - SparkCatalogConfig.HIVE.catalogName(), - SparkCatalogConfig.HIVE.implementation(), - SparkCatalogConfig.HIVE.properties() - }, - { - SparkCatalogConfig.HADOOP.catalogName(), - SparkCatalogConfig.HADOOP.implementation(), - SparkCatalogConfig.HADOOP.properties() - }, - { - SparkCatalogConfig.SPARK.catalogName(), - SparkCatalogConfig.SPARK.implementation(), - SparkCatalogConfig.SPARK.properties() - } - }; - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - public SparkCatalogTestBase(SparkCatalogConfig config) { - super(config); - } - - public SparkCatalogTestBase( - String catalogName, String implementation, Map config) { - super(catalogName, implementation, config); - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.java deleted file mode 100644 index 82b36528996f..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBase.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.spark; - -import static org.apache.hadoop.hive.conf.HiveConf.ConfVars.METASTOREURIS; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.net.URI; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.List; -import java.util.Map; -import java.util.TimeZone; -import java.util.concurrent.TimeoutException; -import java.util.concurrent.atomic.AtomicReference; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Encoders; -import org.apache.spark.sql.Row; -import org.apache.spark.sql.SparkSession; -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.apache.spark.sql.execution.QueryExecution; -import org.apache.spark.sql.execution.SparkPlan; -import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec; -import org.apache.spark.sql.internal.SQLConf; -import org.apache.spark.sql.util.QueryExecutionListener; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; - -public abstract class SparkTestBase extends SparkTestHelperBase { - - protected static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static SparkSession spark = null; - protected static JavaSparkContext sparkContext = null; - protected static HiveCatalog catalog = null; - - @BeforeClass - public static void startMetastoreAndSpark() { - SparkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - SparkTestBase.hiveConf = metastore.hiveConf(); - - SparkTestBase.spark = - SparkSession.builder() - .master("local[2]") - .config(SQLConf.PARTITION_OVERWRITE_MODE().key(), "dynamic") - .config("spark.hadoop." + METASTOREURIS.varname, hiveConf.get(METASTOREURIS.varname)) - .config("spark.sql.legacy.respectNullabilityInTextDatasetConversion", "true") - .enableHiveSupport() - .getOrCreate(); - - SparkTestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); - - SparkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - - try { - catalog.createNamespace(Namespace.of("default")); - } catch (AlreadyExistsException ignored) { - // the default namespace already exists. ignore the create error - } - } - - @AfterClass - public static void stopMetastoreAndSpark() throws Exception { - SparkTestBase.catalog = null; - if (metastore != null) { - metastore.stop(); - SparkTestBase.metastore = null; - } - if (spark != null) { - spark.stop(); - SparkTestBase.spark = null; - SparkTestBase.sparkContext = null; - } - } - - protected long waitUntilAfter(long timestampMillis) { - long current = System.currentTimeMillis(); - while (current <= timestampMillis) { - current = System.currentTimeMillis(); - } - return current; - } - - protected List sql(String query, Object... args) { - List rows = spark.sql(String.format(query, args)).collectAsList(); - if (rows.size() < 1) { - return ImmutableList.of(); - } - - return rowsToJava(rows); - } - - protected Object scalarSql(String query, Object... args) { - List rows = sql(query, args); - Assert.assertEquals("Scalar SQL should return one row", 1, rows.size()); - Object[] row = Iterables.getOnlyElement(rows); - Assert.assertEquals("Scalar SQL should return one value", 1, row.length); - return row[0]; - } - - protected Object[] row(Object... values) { - return values; - } - - protected static String dbPath(String dbName) { - return metastore.getDatabasePath(dbName); - } - - protected void withUnavailableFiles(Iterable> files, Action action) { - Iterable fileLocations = Iterables.transform(files, file -> file.path().toString()); - withUnavailableLocations(fileLocations, action); - } - - private void move(String location, String newLocation) { - Path path = Paths.get(URI.create(location)); - Path tempPath = Paths.get(URI.create(newLocation)); - - try { - Files.move(path, tempPath); - } catch (IOException e) { - throw new UncheckedIOException("Failed to move: " + location, e); - } - } - - protected void withUnavailableLocations(Iterable locations, Action action) { - for (String location : locations) { - move(location, location + "_temp"); - } - - try { - action.invoke(); - } finally { - for (String location : locations) { - move(location + "_temp", location); - } - } - } - - protected void withDefaultTimeZone(String zoneId, Action action) { - TimeZone currentZone = TimeZone.getDefault(); - try { - TimeZone.setDefault(TimeZone.getTimeZone(zoneId)); - action.invoke(); - } finally { - TimeZone.setDefault(currentZone); - } - } - - protected void withSQLConf(Map conf, Action action) { - 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); - } - }); - } - } - - protected Dataset jsonToDF(String schema, String... records) { - Dataset jsonDF = spark.createDataset(ImmutableList.copyOf(records), Encoders.STRING()); - return spark.read().schema(schema).json(jsonDF); - } - - protected void append(String table, String... jsonRecords) { - try { - String schema = spark.table(table).schema().toDDL(); - Dataset df = jsonToDF(schema, jsonRecords); - df.coalesce(1).writeTo(table).append(); - } catch (NoSuchTableException e) { - throw new RuntimeException("Failed to write data", e); - } - } - - protected String tablePropsAsString(Map tableProps) { - StringBuilder stringBuilder = new StringBuilder(); - - for (Map.Entry property : tableProps.entrySet()) { - if (stringBuilder.length() > 0) { - stringBuilder.append(", "); - } - stringBuilder.append(String.format("'%s' '%s'", property.getKey(), property.getValue())); - } - - return stringBuilder.toString(); - } - - protected SparkPlan executeAndKeepPlan(String query, Object... args) { - return executeAndKeepPlan(() -> sql(query, args)); - } - - protected SparkPlan executeAndKeepPlan(Action action) { - AtomicReference executedPlanRef = new AtomicReference<>(); - - QueryExecutionListener listener = - new QueryExecutionListener() { - @Override - public void onSuccess(String funcName, QueryExecution qe, long durationNs) { - executedPlanRef.set(qe.executedPlan()); - } - - @Override - public void onFailure(String funcName, QueryExecution qe, Exception exception) {} - }; - - spark.listenerManager().register(listener); - - action.invoke(); - - try { - spark.sparkContext().listenerBus().waitUntilEmpty(); - } catch (TimeoutException e) { - throw new RuntimeException("Timeout while waiting for processing events", e); - } - - SparkPlan executedPlan = executedPlanRef.get(); - if (executedPlan instanceof AdaptiveSparkPlanExec) { - return ((AdaptiveSparkPlanExec) executedPlan).executedPlan(); - } else { - return executedPlan; - } - } - - @FunctionalInterface - protected interface Action { - void invoke(); - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java deleted file mode 100644 index d5708c9e575e..000000000000 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestBaseWithCatalog.java +++ /dev/null @@ -1,130 +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; - -import java.io.File; -import java.io.IOException; -import java.util.Map; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.PlanningMode; -import org.apache.iceberg.TableProperties; -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.hadoop.HadoopCatalog; -import org.apache.iceberg.util.PropertyUtil; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -public abstract class SparkTestBaseWithCatalog extends SparkTestBase { - protected static File warehouse = null; - - @BeforeClass - public static void createWarehouse() throws IOException { - SparkTestBaseWithCatalog.warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); - } - - @AfterClass - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); - } - } - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - protected final String catalogName; - protected final Map catalogConfig; - protected final Catalog validationCatalog; - protected final SupportsNamespaces validationNamespaceCatalog; - protected final TableIdentifier tableIdent = TableIdentifier.of(Namespace.of("default"), "table"); - protected final String tableName; - - public SparkTestBaseWithCatalog() { - this(SparkCatalogConfig.HADOOP); - } - - public SparkTestBaseWithCatalog(SparkCatalogConfig config) { - this(config.catalogName(), config.implementation(), config.properties()); - } - - 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; - - spark.conf().set("spark.sql.catalog." + catalogName, implementation); - config.forEach( - (key, value) -> spark.conf().set("spark.sql.catalog." + catalogName + "." + key, value)); - - if ("hadoop".equalsIgnoreCase(config.get("type"))) { - spark.conf().set("spark.sql.catalog." + catalogName + ".warehouse", "file:" + warehouse); - } - - this.tableName = - (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default.table"; - - sql("CREATE NAMESPACE IF NOT EXISTS default"); - } - - protected String tableName(String name) { - return (catalogName.equals("spark_catalog") ? "" : catalogName + ".") + "default." + name; - } - - protected String commitTarget() { - return tableName; - } - - protected String selectTarget() { - return tableName; - } - - protected boolean cachingCatalogEnabled() { - return PropertyUtil.propertyAsBoolean( - catalogConfig, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); - } - - protected void configurePlanningMode(PlanningMode planningMode) { - configurePlanningMode(tableName, planningMode); - } - - protected void configurePlanningMode(String table, PlanningMode planningMode) { - sql( - "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", - table, - TableProperties.DATA_PLANNING_MODE, - planningMode.modeName(), - TableProperties.DELETE_PLANNING_MODE, - planningMode.modeName()); - } -} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java index 34308e77d27a..ee264cb090fd 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java @@ -23,7 +23,7 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSparkFunctions { 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 2924cd5db5b2..0fb0ac2f05df 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 @@ -27,6 +27,7 @@ import static org.apache.spark.sql.functions.lit; 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.nio.file.Files; @@ -59,7 +60,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.junit.Assume; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -140,7 +140,8 @@ public void dropTable() { @TestTemplate public void testSpecAndPartitionMetadataColumns() { // TODO: support metadata structs in vectorized ORC reads - Assume.assumeFalse(fileFormat == FileFormat.ORC && vectorized); + assumeThat(fileFormat).isNotEqualTo(FileFormat.ORC); + assumeThat(vectorized).isFalse(); sql("INSERT INTO TABLE %s VALUES (1, 'a1', 'b1')", TABLE_NAME); @@ -173,7 +174,7 @@ public void testSpecAndPartitionMetadataColumns() { @TestTemplate public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET); + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); table.updateProperties().set(PARQUET_ROW_GROUP_SIZE_BYTES, "100").commit(); @@ -197,7 +198,7 @@ public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTable @TestTemplate public void testPositionMetadataColumnWithMultipleBatches() throws NoSuchTableException { - Assume.assumeTrue(fileFormat == FileFormat.PARQUET); + assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); table.updateProperties().set(PARQUET_BATCH_SIZE, "1000").commit(); From e479b5534d068872c9b772a6a788b8c245ecf210 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 28 Feb 2024 16:03:12 +0100 Subject: [PATCH 0086/1019] Spark 3.4: Improve error msg when function can't be loaded (#9821) --- .../java/org/apache/iceberg/spark/extensions/TestViews.java | 5 ++--- .../java/org/apache/iceberg/spark/SupportsFunctions.java | 4 +++- .../java/org/apache/iceberg/spark/TestFunctionCatalog.java | 4 ++-- 3 files changed, 7 insertions(+), 6 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 5bef3003b421..5d1cb2db612b 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 @@ -505,9 +505,8 @@ public void readFromViewReferencingTempFunction() throws NoSuchTableException { // reading from a view that references a TEMP FUNCTION shouldn't be possible assertThatThrownBy(() -> sql("SELECT * FROM %s", viewName)) .isInstanceOf(AnalysisException.class) - .hasMessageContaining("The function") - .hasMessageContaining(functionName) - .hasMessageContaining("cannot be found"); + .hasMessageStartingWith( + String.format("Cannot load function: %s.%s.%s", catalogName, NAMESPACE, functionName)); } @Test diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java index 34897d2b4c01..1aa08b02cead 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SupportsFunctions.java @@ -24,6 +24,7 @@ import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; +import scala.Option; interface SupportsFunctions extends FunctionCatalog { @@ -58,6 +59,7 @@ default UnboundFunction loadFunction(Identifier ident) throws NoSuchFunctionExce } } - throw new NoSuchFunctionException(ident); + throw new NoSuchFunctionException( + String.format("Cannot load function: %s.%s", name(), ident), Option.empty()); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java index be91c8d6370b..cad3478f3eba 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java @@ -89,13 +89,13 @@ public void testLoadFunctions() throws NoSuchFunctionException { asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))) .isInstanceOf(NoSuchFunctionException.class) .hasMessageStartingWith( - "[ROUTINE_NOT_FOUND] The function default.iceberg_version cannot be found."); + String.format("Cannot load function: %s.default.iceberg_version", catalogName)); Identifier undefinedFunction = Identifier.of(SYSTEM_NAMESPACE, "undefined_function"); Assertions.assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) .isInstanceOf(NoSuchFunctionException.class) .hasMessageStartingWith( - "[ROUTINE_NOT_FOUND] The function system.undefined_function cannot be found."); + String.format("Cannot load function: %s.system.undefined_function", catalogName)); Assertions.assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) .isInstanceOf(AnalysisException.class) From c2e3f6eab085e9b1a34946dea66dc8a60350267b Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Wed, 28 Feb 2024 20:37:59 +0530 Subject: [PATCH 0087/1019] Core, Spark: Fix build warning related to Javadoc link tag (#9823) --- .../main/java/org/apache/iceberg/actions/BaseCommitService.java | 2 +- .../java/org/apache/iceberg/spark/procedures/BaseProcedure.java | 2 +- .../java/org/apache/iceberg/spark/procedures/BaseProcedure.java | 2 +- .../java/org/apache/iceberg/spark/procedures/BaseProcedure.java | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java b/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java index dbe888681a5f..d8e5b753cbd2 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java @@ -138,7 +138,7 @@ public void start() { /** * Places a file group in the queue and commits a batch of file groups if {@link - * #rewritesPerCommit} number of file groups are present in the queue. + * BaseCommitService#rewritesPerCommit} number of file groups are present in the queue. * * @param group file group to eventually be committed */ diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java index d7c01936ee02..fb8bdc252df5 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java @@ -201,7 +201,7 @@ TableCatalog tableCatalog() { /** * Closes this procedure's executor service if a new one was created with {@link - * #executorService(int, String)}. Does not block for any remaining tasks. + * BaseProcedure#executorService(int, String)}. Does not block for any remaining tasks. */ protected void closeService() { if (executorService != null) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java index d7c01936ee02..fb8bdc252df5 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java @@ -201,7 +201,7 @@ TableCatalog tableCatalog() { /** * Closes this procedure's executor service if a new one was created with {@link - * #executorService(int, String)}. Does not block for any remaining tasks. + * BaseProcedure#executorService(int, String)}. Does not block for any remaining tasks. */ protected void closeService() { if (executorService != null) { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java index d7c01936ee02..fb8bdc252df5 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java @@ -201,7 +201,7 @@ TableCatalog tableCatalog() { /** * Closes this procedure's executor service if a new one was created with {@link - * #executorService(int, String)}. Does not block for any remaining tasks. + * BaseProcedure#executorService(int, String)}. Does not block for any remaining tasks. */ protected void closeService() { if (executorService != null) { From 97bbdbc60d05675563e79a27b98b5bf88feec528 Mon Sep 17 00:00:00 2001 From: Alok Thatikunta Date: Wed, 28 Feb 2024 22:18:48 +0530 Subject: [PATCH 0088/1019] OpenAPI: Fix URL pointing to catalog properties (#9825) --- open-api/rest-catalog-open-api.yaml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index e1d3c7df2264..d23c65e87c89 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -99,7 +99,7 @@ paths: Common catalog configuration settings are documented at - https://iceberg.apache.org/configuration/#catalog-properties + https://iceberg.apache.org/docs/latest/configuration/#catalog-properties " responses: 200: From 7d04d8237ec9b7147ad66113a3ef67c6c4e9a0c1 Mon Sep 17 00:00:00 2001 From: Drew Gallardo Date: Wed, 28 Feb 2024 08:57:38 -0800 Subject: [PATCH 0089/1019] REST spec: Add ContentFile types to spec for the PreplanTable and PlanTable API (#9717) --- open-api/rest-catalog-open-api.py | 220 ++++++++++++++++++++++ open-api/rest-catalog-open-api.yaml | 275 ++++++++++++++++++++++++++++ 2 files changed, 495 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index c193b53e9452..3173a6efb810 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -17,7 +17,9 @@ from __future__ import annotations +from datetime import date from typing import Any, Dict, List, Literal, Optional, Union +from uuid import UUID from pydantic import BaseModel, Extra, Field @@ -629,6 +631,183 @@ class PartitionStatisticsFile(BaseModel): file_size_in_bytes: int = Field(..., alias='file-size-in-bytes') +class BooleanTypeValue(BaseModel): + __root__: bool = Field(..., example=True) + + +class IntegerTypeValue(BaseModel): + __root__: int = Field(..., example=42) + + +class LongTypeValue(BaseModel): + __root__: int = Field(..., example=9223372036854775807) + + +class FloatTypeValue(BaseModel): + __root__: float = Field(..., example=3.14) + + +class DoubleTypeValue(BaseModel): + __root__: float = Field(..., example=123.456) + + +class DecimalTypeValue(BaseModel): + __root__: str = Field( + ..., + description="Decimal type values are serialized as strings. Decimals with a positive scale serialize as numeric plain text, while decimals with a negative scale use scientific notation and the exponent will be equal to the negated scale. For instance, a decimal with a positive scale is '123.4500', with zero scale is '2', and with a negative scale is '2E+20'", + example='123.4500', + ) + + +class StringTypeValue(BaseModel): + __root__: str = Field(..., example='hello') + + +class UUIDTypeValue(BaseModel): + __root__: UUID = Field( + ..., + description='UUID type values are serialized as a 36-character lowercase string in standard UUID format as specified by RFC-4122', + example='eb26bdb1-a1d8-4aa6-990e-da940875492c', + max_length=36, + min_length=36, + regex='^[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}$', + ) + + +class DateTypeValue(BaseModel): + __root__: date = Field( + ..., + description="Date type values follow the 'YYYY-MM-DD' ISO-8601 standard date format", + example='2007-12-03', + ) + + +class TimeTypeValue(BaseModel): + __root__: str = Field( + ..., + description="Time type values follow the 'HH:MM:SS.ssssss' ISO-8601 format with microsecond precision", + example='22:31:08.123456', + ) + + +class TimestampTypeValue(BaseModel): + __root__: str = Field( + ..., + description="Timestamp type values follow the 'YYYY-MM-DDTHH:MM:SS.ssssss' ISO-8601 format with microsecond precision", + example='2007-12-03T10:15:30.123456', + ) + + +class TimestampTzTypeValue(BaseModel): + __root__: str = Field( + ..., + description="TimestampTz type values follow the 'YYYY-MM-DDTHH:MM:SS.ssssss+00:00' ISO-8601 format with microsecond precision, and a timezone offset (+00:00 for UTC)", + example='2007-12-03T10:15:30.123456+00:00', + ) + + +class TimestampNanoTypeValue(BaseModel): + __root__: str = Field( + ..., + description="Timestamp_ns type values follow the 'YYYY-MM-DDTHH:MM:SS.sssssssss' ISO-8601 format with nanosecond precision", + example='2007-12-03T10:15:30.123456789', + ) + + +class TimestampTzNanoTypeValue(BaseModel): + __root__: str = Field( + ..., + description="Timestamp_ns type values follow the 'YYYY-MM-DDTHH:MM:SS.sssssssss+00:00' ISO-8601 format with nanosecond precision, and a timezone offset (+00:00 for UTC)", + example='2007-12-03T10:15:30.123456789+00:00', + ) + + +class FixedTypeValue(BaseModel): + __root__: str = Field( + ..., + description='Fixed length type values are stored and serialized as an uppercase hexadecimal string preserving the fixed length', + example='78797A', + ) + + +class BinaryTypeValue(BaseModel): + __root__: str = Field( + ..., + description='Binary type values are stored and serialized as an uppercase hexadecimal string', + example='78797A', + ) + + +class CountMap(BaseModel): + keys: Optional[List[IntegerTypeValue]] = Field( + None, description='List of integer column ids for each corresponding value' + ) + values: Optional[List[LongTypeValue]] = Field( + None, description="List of Long values, matched to 'keys' by index" + ) + + +class PrimitiveTypeValue(BaseModel): + __root__: Union[ + BooleanTypeValue, + IntegerTypeValue, + LongTypeValue, + FloatTypeValue, + DoubleTypeValue, + DecimalTypeValue, + StringTypeValue, + UUIDTypeValue, + DateTypeValue, + TimeTypeValue, + TimestampTypeValue, + TimestampTzTypeValue, + TimestampNanoTypeValue, + TimestampTzNanoTypeValue, + FixedTypeValue, + BinaryTypeValue, + ] + + +class FileFormat(BaseModel): + __root__: Literal['avro', 'orc', 'parquet'] + + +class ContentFile(BaseModel): + content: str + file_path: str = Field(..., alias='file-path') + file_format: FileFormat = Field(..., alias='file-format') + spec_id: int = Field(..., alias='spec-id') + partition: Optional[List[PrimitiveTypeValue]] = Field( + None, + description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', + example=[1, 'bar'], + ) + file_size_in_bytes: int = Field( + ..., alias='file-size-in-bytes', description='Total file size in bytes' + ) + record_count: int = Field( + ..., alias='record-count', description='Number of records in the file' + ) + key_metadata: Optional[BinaryTypeValue] = Field( + None, alias='key-metadata', description='Encryption key metadata blob' + ) + split_offsets: Optional[List[int]] = Field( + None, alias='split-offsets', description='List of splittable offsets' + ) + sort_order_id: Optional[int] = Field(None, alias='sort-order-id') + + +class PositionDeleteFile(ContentFile): + content: Literal['position-deletes'] + + +class EqualityDeleteFile(ContentFile): + content: Literal['equality-deletes'] + equality_ids: Optional[List[int]] = Field( + None, alias='equality-ids', description='List of equality field IDs' + ) + + class CreateNamespaceRequest(BaseModel): namespace: Namespace properties: Optional[Dict[str, str]] = Field( @@ -668,6 +847,47 @@ class StatisticsFile(BaseModel): blob_metadata: List[BlobMetadata] = Field(..., alias='blob-metadata') +class ValueMap(BaseModel): + keys: Optional[List[IntegerTypeValue]] = Field( + None, description='List of integer column ids for each corresponding value' + ) + values: Optional[List[PrimitiveTypeValue]] = Field( + None, description="List of primitive type values, matched to 'keys' by index" + ) + + +class DataFile(ContentFile): + content: Literal['data'] + column_sizes: Optional[CountMap] = Field( + None, + alias='column-sizes', + description='Map of column id to total count, including null and NaN', + ) + value_counts: Optional[CountMap] = Field( + None, alias='value-counts', description='Map of column id to null value count' + ) + null_value_counts: Optional[CountMap] = Field( + None, + alias='null-value-counts', + description='Map of column id to null value count', + ) + nan_value_counts: Optional[CountMap] = Field( + None, + alias='nan-value-counts', + description='Map of column id to number of NaN values in the column', + ) + lower_bounds: Optional[ValueMap] = Field( + None, + alias='lower-bounds', + description='Map of column id to lower bound primitive type values', + ) + upper_bounds: Optional[ValueMap] = Field( + None, + alias='upper-bounds', + description='Map of column id to upper bound primitive type values', + ) + + class Term(BaseModel): __root__: Union[Reference, TransformTerm] diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index d23c65e87c89..b95c5759e9c2 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -3324,6 +3324,281 @@ components: type: integer format: int64 + BooleanTypeValue: + type: boolean + example: true + + IntegerTypeValue: + type: integer + example: 42 + + LongTypeValue: + type: integer + format: int64 + example: 9223372036854775807 + + FloatTypeValue: + type: number + format: float + example: 3.14 + + DoubleTypeValue: + type: number + format: double + example: 123.456 + + DecimalTypeValue: + type: string + description: + "Decimal type values are serialized as strings. Decimals with a positive scale serialize as numeric plain + text, while decimals with a negative scale use scientific notation and the exponent will be equal to the + negated scale. For instance, a decimal with a positive scale is '123.4500', with zero scale is '2', + and with a negative scale is '2E+20'" + example: "123.4500" + + StringTypeValue: + type: string + example: "hello" + + UUIDTypeValue: + type: string + format: uuid + pattern: '^[0-9a-f]{8}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{4}-[0-9a-f]{12}$' + maxLength: 36 + minLength: 36 + description: + "UUID type values are serialized as a 36-character lowercase string in standard UUID format as specified + by RFC-4122" + example: "eb26bdb1-a1d8-4aa6-990e-da940875492c" + + DateTypeValue: + type: string + format: date + description: + "Date type values follow the 'YYYY-MM-DD' ISO-8601 standard date format" + example: "2007-12-03" + + TimeTypeValue: + type: string + description: + "Time type values follow the 'HH:MM:SS.ssssss' ISO-8601 format with microsecond precision" + example: "22:31:08.123456" + + TimestampTypeValue: + type: string + description: + "Timestamp type values follow the 'YYYY-MM-DDTHH:MM:SS.ssssss' ISO-8601 format with microsecond precision" + example: "2007-12-03T10:15:30.123456" + + TimestampTzTypeValue: + type: string + description: + "TimestampTz type values follow the 'YYYY-MM-DDTHH:MM:SS.ssssss+00:00' ISO-8601 format with microsecond precision, + and a timezone offset (+00:00 for UTC)" + example: "2007-12-03T10:15:30.123456+00:00" + + TimestampNanoTypeValue: + type: string + description: + "Timestamp_ns type values follow the 'YYYY-MM-DDTHH:MM:SS.sssssssss' ISO-8601 format with nanosecond precision" + example: "2007-12-03T10:15:30.123456789" + + TimestampTzNanoTypeValue: + type: string + description: + "Timestamp_ns type values follow the 'YYYY-MM-DDTHH:MM:SS.sssssssss+00:00' ISO-8601 format with nanosecond + precision, and a timezone offset (+00:00 for UTC)" + example: "2007-12-03T10:15:30.123456789+00:00" + + FixedTypeValue: + type: string + description: + "Fixed length type values are stored and serialized as an uppercase hexadecimal string + preserving the fixed length" + example: "78797A" + + BinaryTypeValue: + type: string + description: + "Binary type values are stored and serialized as an uppercase hexadecimal string" + example: "78797A" + + CountMap: + type: object + properties: + keys: + type: array + items: + $ref: '#/components/schemas/IntegerTypeValue' + description: "List of integer column ids for each corresponding value" + values: + type: array + items: + $ref: '#/components/schemas/LongTypeValue' + description: "List of Long values, matched to 'keys' by index" + example: + { + "keys": [ 1, 2 ], + "values": [ 100,200 ] + } + + ValueMap: + type: object + properties: + keys: + type: array + items: + $ref: '#/components/schemas/IntegerTypeValue' + description: "List of integer column ids for each corresponding value" + values: + type: array + items: + $ref: '#/components/schemas/PrimitiveTypeValue' + description: "List of primitive type values, matched to 'keys' by index" + example: + { + "keys": [ 1, 2 ], + "values": [ 100, "test" ] + } + + PrimitiveTypeValue: + oneOf: + - $ref: '#/components/schemas/BooleanTypeValue' + - $ref: '#/components/schemas/IntegerTypeValue' + - $ref: '#/components/schemas/LongTypeValue' + - $ref: '#/components/schemas/FloatTypeValue' + - $ref: '#/components/schemas/DoubleTypeValue' + - $ref: '#/components/schemas/DecimalTypeValue' + - $ref: '#/components/schemas/StringTypeValue' + - $ref: '#/components/schemas/UUIDTypeValue' + - $ref: '#/components/schemas/DateTypeValue' + - $ref: '#/components/schemas/TimeTypeValue' + - $ref: '#/components/schemas/TimestampTypeValue' + - $ref: '#/components/schemas/TimestampTzTypeValue' + - $ref: '#/components/schemas/TimestampNanoTypeValue' + - $ref: '#/components/schemas/TimestampTzNanoTypeValue' + - $ref: '#/components/schemas/FixedTypeValue' + - $ref: '#/components/schemas/BinaryTypeValue' + + FileFormat: + type: string + enum: + - avro + - orc + - parquet + + ContentFile: + discriminator: + propertyName: content + mapping: + data: '#/components/schemas/DataFile' + position-deletes: '#/components/schemas/PositionDeleteFile' + equality-deletes: '#/components/schemas/EqualityDeleteFile' + type: object + required: + - spec-id + - content + - file-path + - file-format + - file-size-in-bytes + - record-count + properties: + content: + type: string + file-path: + type: string + file-format: + $ref: '#/components/schemas/FileFormat' + spec-id: + type: integer + partition: + type: array + items: + $ref: '#/components/schemas/PrimitiveTypeValue' + description: + "A list of partition field values ordered based on the fields of the partition spec specified by the + `spec-id`" + example: [1, "bar"] + file-size-in-bytes: + type: integer + format: int64 + description: "Total file size in bytes" + record-count: + type: integer + format: int64 + description: "Number of records in the file" + key-metadata: + allOf: + - $ref: '#/components/schemas/BinaryTypeValue' + description: "Encryption key metadata blob" + split-offsets: + type: array + items: + type: integer + format: int64 + description: "List of splittable offsets" + sort-order-id: + type: integer + + DataFile: + allOf: + - $ref: '#/components/schemas/ContentFile' + type: object + required: + - content + properties: + content: + type: string + enum: [ "data" ] + column-sizes: + allOf: + - $ref: '#/components/schemas/CountMap' + description: "Map of column id to total count, including null and NaN" + value-counts: + allOf: + - $ref: '#/components/schemas/CountMap' + description: "Map of column id to null value count" + null-value-counts: + allOf: + - $ref: '#/components/schemas/CountMap' + description: "Map of column id to null value count" + nan-value-counts: + allOf: + - $ref: '#/components/schemas/CountMap' + description: "Map of column id to number of NaN values in the column" + lower-bounds: + allOf: + - $ref: '#/components/schemas/ValueMap' + description: "Map of column id to lower bound primitive type values" + upper-bounds: + allOf: + - $ref: '#/components/schemas/ValueMap' + description: "Map of column id to upper bound primitive type values" + + PositionDeleteFile: + allOf: + - $ref: '#/components/schemas/ContentFile' + required: + - content + properties: + content: + type: string + enum: [ "position-deletes" ] + + EqualityDeleteFile: + allOf: + - $ref: '#/components/schemas/ContentFile' + required: + - content + properties: + content: + type: string + enum: [ "equality-deletes" ] + equality-ids: + type: array + items: + type: integer + description: "List of equality field IDs" ############################# # Reusable Response Objects # From 25b26339425f4767144612e867a5651daec09327 Mon Sep 17 00:00:00 2001 From: Rahil C <32500120+rahil-c@users.noreply.github.com> Date: Thu, 29 Feb 2024 10:37:42 -0800 Subject: [PATCH 0090/1019] Add pagination to open api spec for listing of namespaces, tables, views (#9660) * Add pagination to open api spec for listing of namespaces, tables, views * Rev for pagination in open api spec * Fix writing in pagination descriptions * Use opaque token * Rev on pagination pr writing * Rev on pagination pr writing py file * Rev on description for pageSize * revise pageToken description * minor rev and make int for pageSize * minor rev and remove emtpy val true for pageSize * use only PageToken instead of NextPageToken * address jack and nastra comments * address final comments * remove they and use min 1 for pageSize --------- Co-authored-by: Rahil Chertara --- open-api/rest-catalog-open-api.py | 9 +++++++ open-api/rest-catalog-open-api.yaml | 42 +++++++++++++++++++++++++++++ 2 files changed, 51 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 3173a6efb810..77dcad9cfba6 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -77,6 +77,13 @@ class Namespace(BaseModel): ) +class PageToken(BaseModel): + __root__: str = Field( + ..., + description='An opaque token which allows clients to make use of pagination for a list API (e.g. ListTables). Clients will initiate the first paginated request by sending an empty `pageToken` e.g. `GET /tables?pageToken` or `GET /tables?pageToken=` signaling to the service that the response should be paginated.\nServers that support pagination will recognize `pageToken` and return a `next-page-token` in response if there are more results available. After the initial request, it is expected that the value of `next-page-token` from the last response is used in the subsequent request. Servers that do not support pagination will ignore `next-page-token` and return all results.', + ) + + class TableIdentifier(BaseModel): namespace: Namespace name: str @@ -597,10 +604,12 @@ class GetNamespaceResponse(BaseModel): class ListTablesResponse(BaseModel): + next_page_token: Optional[PageToken] = Field(None, alias='next-page-token') identifiers: Optional[List[TableIdentifier]] = Field(None, unique_items=True) class ListNamespacesResponse(BaseModel): + next_page_token: Optional[PageToken] = Field(None, alias='next-page-token') namespaces: Optional[List[Namespace]] = Field(None, unique_items=True) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index b95c5759e9c2..77aabc834adb 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -201,6 +201,8 @@ paths: If `parent` is not provided, all top-level namespaces should be listed. operationId: listNamespaces parameters: + - $ref: '#/components/parameters/page-token' + - $ref: '#/components/parameters/page-size' - name: parent in: query description: @@ -448,6 +450,9 @@ paths: summary: List all table identifiers underneath a given namespace description: Return all table identifiers under this namespace operationId: listTables + parameters: + - $ref: '#/components/parameters/page-token' + - $ref: '#/components/parameters/page-size' responses: 200: $ref: '#/components/responses/ListTablesResponse' @@ -1070,6 +1075,9 @@ paths: summary: List all view identifiers underneath a given namespace description: Return all view identifiers under this namespace operationId: listViews + parameters: + - $ref: '#/components/parameters/page-token' + - $ref: '#/components/parameters/page-size' responses: 200: $ref: '#/components/responses/ListTablesResponse' @@ -1482,6 +1490,25 @@ components: explode: false example: "vended-credentials,remote-signing" + page-token: + name: pageToken + in: query + required: false + allowEmptyValue: true + schema: + $ref: '#/components/schemas/PageToken' + + page-size: + name: pageSize + in: query + description: + For servers that support pagination, this signals an upper bound of the number of results that a client will receive. + For servers that do not support pagination, clients may receive results larger than the indicated `pageSize`. + required: false + schema: + type: integer + minimum: 1 + ############################## # Application Schema Objects # ############################## @@ -1581,6 +1608,17 @@ components: type: string example: [ "accounting", "tax" ] + PageToken: + description: + An opaque token which allows clients to make use of pagination for a list API (e.g. ListTables). + Clients will initiate the first paginated request by sending an empty `pageToken` e.g. `GET /tables?pageToken` or `GET /tables?pageToken=` + signaling to the service that the response should be paginated. + + Servers that support pagination will recognize `pageToken` and return a `next-page-token` in response if there are more results available. + After the initial request, it is expected that the value of `next-page-token` from the last response is used in the subsequent request. + Servers that do not support pagination will ignore `next-page-token` and return all results. + type: string + TableIdentifier: type: object required: @@ -3207,6 +3245,8 @@ components: ListTablesResponse: type: object properties: + next-page-token: + $ref: '#/components/schemas/PageToken' identifiers: type: array uniqueItems: true @@ -3216,6 +3256,8 @@ components: ListNamespacesResponse: type: object properties: + next-page-token: + $ref: '#/components/schemas/PageToken' namespaces: type: array uniqueItems: true From fc7794ad6dae308560a574fe4652ab8ee38fce06 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 29 Feb 2024 17:02:34 -0600 Subject: [PATCH 0091/1019] Core: FileIO Reflection Error Message is Misleading (#9840) --- core/src/main/java/org/apache/iceberg/CatalogUtil.java | 2 +- core/src/test/java/org/apache/iceberg/TestCatalogUtil.java | 4 +++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index ccecfb19070b..e09be748f2ee 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -334,7 +334,7 @@ public static FileIO loadFileIO(String impl, Map properties, Obj .buildChecked(); } catch (NoSuchMethodException e) { throw new IllegalArgumentException( - String.format("Cannot initialize FileIO, missing no-arg constructor: %s", impl), e); + String.format("Cannot initialize FileIO implementation %s: %s", impl, e.getMessage()), e); } FileIO fileIO; diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 33172aa7f09c..5ac0ebbafc90 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -160,7 +160,9 @@ public void loadCustomFileIO_badArg() { Assertions.assertThatThrownBy( () -> CatalogUtil.loadFileIO(TestFileIOBadArg.class.getName(), Maps.newHashMap(), null)) .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot initialize FileIO, missing no-arg constructor"); + .hasMessageStartingWith( + "Cannot initialize FileIO implementation " + + "org.apache.iceberg.TestCatalogUtil$TestFileIOBadArg: Cannot find constructor"); } @Test From 77496d66642db0f493a373a3aa185ac54291fbcf Mon Sep 17 00:00:00 2001 From: "Brian \"bits\" Olsen" Date: Sat, 2 Mar 2024 13:03:35 -0800 Subject: [PATCH 0092/1019] Docs: Fix image on spec (#9843) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index 306cdb77757b..d9fbacf2328a 100644 --- a/format/spec.md +++ b/format/spec.md @@ -60,7 +60,7 @@ In addition to row-level deletes, version 2 makes some requirements stricter for ## Overview -![Iceberg snapshot structure](../../../img/iceberg-metadata.png) +![Iceberg snapshot structure](assets/images/iceberg-metadata.png) This table format tracks individual data files in a table instead of directories. This allows writers to create data files in-place and only adds files to the table in an explicit commit. From 259a48ff936ffc1784f38dd87ae82104dfdbd515 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 2 Mar 2024 14:05:42 -0800 Subject: [PATCH 0093/1019] Core: Fix REST catalog handling when the service has no view support (#9853) * Core: Fix REST catalog handling when the service has no view support. * Fix error message. * Improve the error message. --- .../iceberg/rest/RESTSessionCatalog.java | 22 ++++++++++++++----- 1 file changed, 16 insertions(+), 6 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 742caa9494f4..a34f738c318e 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -1058,12 +1058,22 @@ public List listViews(SessionContext context, Namespace namespa public View loadView(SessionContext context, TableIdentifier identifier) { checkViewIdentifierIsValid(identifier); - LoadViewResponse response = - client.get( - paths.view(identifier), - LoadViewResponse.class, - headers(context), - ErrorHandlers.viewErrorHandler()); + LoadViewResponse response; + try { + response = + client.get( + paths.view(identifier), + LoadViewResponse.class, + headers(context), + ErrorHandlers.viewErrorHandler()); + } catch (UnsupportedOperationException | RESTException e) { + // Normally, copying an exception message is a bad practice but engines may show just the + // message and suppress the exception cause when the view does not exist. Since 401 and 403 + // responses can trigger this case, including the message increases the chances that the "Not + // authorized" or "Forbidden" message is preserved and shown. + throw new NoSuchViewException( + e, "Unable to load view %s.%s: %s", name(), identifier, e.getMessage()); + } AuthSession session = tableSession(response.config(), session(context)); ViewMetadata metadata = response.metadata(); From 2c97ca7bd7e624f0c536f34aa029b9a6bc2e0428 Mon Sep 17 00:00:00 2001 From: "Brian \"bits\" Olsen" Date: Sun, 3 Mar 2024 12:42:55 -0800 Subject: [PATCH 0094/1019] Site: Remove embedded calendar, replace with links (#9854) --- site/docs/community.md | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/site/docs/community.md b/site/docs/community.md index 6d39ce96aad6..ac028a46e9c3 100644 --- a/site/docs/community.md +++ b/site/docs/community.md @@ -48,12 +48,8 @@ Please note that this link may occasionally break when Slack does an upgrade. If This calendar contains two calendar feeds: -* Iceberg Community Events - Events such as conferences and meetups, aimed to educate and inspire Iceberg users. -* Iceberg Dev Events - Events such as the triweekly Iceberg sync, aimed to discuss the project roadmap and how to implement features. - -You can subscribe to either or both of these calendars by clicking the "+ Google Calendar" icon on the bottom right. - - +* [Iceberg Community Events](https://calendar.google.com/calendar/u/0?cid=NTkzYmIwMGJmZTQ1N2QzMTkxNDEzNTBkZDI0Yzk2NGYzOWJkYmQ5ZmQyNDMyODFhODYzMmEwMDk2M2EyMWQ4NkBncm91cC5jYWxlbmRhci5nb29nbGUuY29t){:target="_blank"} : Events such as conferences and meetups, aimed to educate and inspire Iceberg users. +* [Iceberg Dev Events](https://calendar.google.com/calendar/u/0?cid=MzkwNWQ0OTJmMWI0NTBiYTA3MTJmMmFlNmFmYTc2ZWI3NTdmMTNkODUyMjBjYzAzYWE0NTI3ODg1YWRjNTYyOUBncm91cC5jYWxlbmRhci5nb29nbGUuY29t){:target="_blank"} : Events such as the triweekly Iceberg sync, aimed to discuss the project roadmap and how to implement features. ## Mailing Lists From 0576139be6716bb4a673a6baac059aee7ec98ba3 Mon Sep 17 00:00:00 2001 From: "Brian \"bits\" Olsen" Date: Sun, 3 Mar 2024 12:44:39 -0800 Subject: [PATCH 0095/1019] Site: Update for ASF site guidelines (#9729) Co-authored-by: Muna Bedan --- .../docs/assets/images/asf-estd-1999-logo.png | Bin 0 -> 22681 bytes site/docs/assets/javascript/lottie-player.js | 77 +++++++ site/docs/assets/javascript/termynal.js | 197 ++++++++++++++++++ site/docs/assets/stylesheets/extra.css | 86 +++----- site/docs/assets/stylesheets/fonts.css | 76 +++++++ site/docs/assets/stylesheets/termynal.css | 101 +++++++++ site/mkdocs.yml | 3 +- site/nav.yml | 3 +- site/overrides/home.html | 10 +- site/overrides/partials/cto.html | 21 -- site/overrides/partials/footer.html | 144 +++++++++++-- 11 files changed, 617 insertions(+), 101 deletions(-) create mode 100644 site/docs/assets/images/asf-estd-1999-logo.png create mode 100644 site/docs/assets/javascript/lottie-player.js create mode 100644 site/docs/assets/javascript/termynal.js create mode 100644 site/docs/assets/stylesheets/fonts.css create mode 100644 site/docs/assets/stylesheets/termynal.css delete mode 100644 site/overrides/partials/cto.html diff --git a/site/docs/assets/images/asf-estd-1999-logo.png b/site/docs/assets/images/asf-estd-1999-logo.png new file mode 100644 index 0000000000000000000000000000000000000000..3aa13cfa11bd5aea65bf3bb4f7de18c7c43c2168 GIT binary patch literal 22681 zcmXtf19V*9*LG~HF()<}tMSBk8rx1|n-kl%)i_NW+cw(R{?h;N{q|Zjb7$7Mb2gv7 z&$;_VD9TGBBj6)|fq@|drNorMz`$ESuN&cDKHm`>m4Tlh@D5U1&R}4OnEzhjU>RAs zpFe^-D@%%i)l3qee13s4|0eeh46Hs5@zoF-3`~;=DE3X&1N^MZ&qa61hx)$j0$D0v zUF^40)^nD?0pyuxG$=gJhH`Qx@t6PRS;@v?QB!8)%DouWoyKV-fOO1>bIk&#L1h>u zizFrf$0cs$o9C)$Ym8XQnsM%ZIN@WHUacQ`5>XV0amTSyZ=gX6i6#Ne>~N!V>*DR| zaALdb2^_^AZ`&6oEbYfd_lxh=)Z05u-T!FW$iQF*C*{5U2!5D{7J?fhFaWLKcd&D} zYPswd`}vE3;V0Ff)a@_Aq7f=w;UEN=R8uw{M^Sp=;W<}4Vs&|xGSuED`P;6G-$tM=8y z%IIbpj9oP%M&Q7Hx@C=9_ZC5w0=VAv%IS*r&eflCyc&Gb|NS!qCz85Tl<=n>+#2ZW z6I6#p%v5HzrEIi;Bjj8A?i;$cp_OjML;vYa)Nq6`aE<<)=aA(D8#pL2U$Us$gITWz zlx1SKB~7|=k+BNX59|A%em}>`?cL{IWQwuzl8Ys0gjYz(&IRKH`cfogH&@HqqDwmt zp$w-zlcD&BeA2Jbeh`7tiT>X)QdHnwzB>Ej@=C?{P&7TfIhnIXFlCHxs;0 z^PQ+1Se$=E5NfMn*ZTfCAPHrzWh<@%^VL*iZd8h_f;drm;h3GaS&dA+oa#VG+y@2H zz0|sx>z~$<2V1DPB27h^c&YuwbiavoW{}h2FsT6!oxvg1;1yOD=Sw;YrKbG>{ISY8 z|8Zux23k`PYTeR#eu|a(Eei4|FcJ3?mvTc-M~*-ivNVkf3Wrc0v5bUPBR;UmZ960V z&kndPjO{$~mrhS9%R1c?k<#FJSB@s+0A8i!t!qGi`dN$|qjb=-{a~hYgg@bbw!l~e zt^3}GkxY>85ekPI7yp?QR=X=JZl%+~Yc>IQjT%6~^pfSn`&qpfEJh8Ki*hhy1mzAa zL;U{My+& z0pQD{Gh-^_(suu|UAV2NUFWl#tO)77vIs*T#e#J(dD*p`q{bYI@y7KvpTCCOfEwbg zjl>O5A~{*@CHwCN?5F)c4LHz3+WO?!)Cg^(3QDSK$1%IoASUIl6hbEqfM|A%i1U-g zR>8Xf;v{bdqrfVnP{y_lDpC4Jq7|+s_h}JIh^HLBC zr;;>(rGaLl@E6kJpJKqJ#bJEhF(2y92_;Ryl3EKV*=Y;wi>>|eiu_FKAV4(q^vilc zPet;2)-xg6=(xz{*8jymOAE(8HBvUnVSn0_VN5SY{?1VW zDz;)K)HV=7+L9*WDpIAkph`)ajf@_0JorBJnYU)LZ>n-U+dvP)SbNvD?u98Ap+2x#02PIbaaZJoR*P|_-f9+9M-KLWCv;n!i}fFd4b&I^wzX63 zcr)*dPZ)_-!-P?!q+oP2)rurK)aA*MvX_)xm`*{ED0Js2$UKjigp;p}pB5Pvo%~xi z&-qD;J1mQ3a-#u+MryJfd(=Y>$r$@R*U4G>J?C?|!~{l6MIV1~8R+vLzM#;-C8TLK#Kvmfo#ZC7li(ON=I(I)$8FJQ?zLYn!o3 z23Em_#(Wt5IMi!A#lZbi{U&yY3o079K{XITY-&)M%}{iN7)^nV|CmnoCgFuk`tAM7 zZsc3YkwP7`jcD#PY(POe2z5@6`fnluf5vZi&OKqLQj-m#c@_6xBl8y%%$1FTq<(+R zbkn^QCna83JF%gJ%5MS<#fYsQyO2m^kGL2T9OM?XsDbacWxE3EKYe#u)&|}Qv)}^b zhnGc#VPHhvbp~iVDKbGdF~k2=_!3^%dxXpnT~1n>s2qxq2>J`7@j%6ulR*|!b8snJ zHrztVW&Tk$MZ~`lH!CEB@l@59uyB_^U{M(rk(71-{e_yzmBxs0YhiHD z;%Hd?ZWkVpL7d5J8+yW#K+7x*j^3FK__{i4*nmaI&PI|Lu(tur-#dP_r$AV(&!pPx zd2>eOcKvvD)p4eEHJ+}58uU4#cxry>i0%1rUuL+nik7mDvpJ8L`>%JOQhu!BghFye z*sU`^T;Wk=$&?syzU_=Zs*i#T$;v~FzZ!G^>!Wbu9{wXSLNHXA>83m5_R!C{m?jPq{=N zb{u)JXI9k{i^i6tQcY<(JZ}3+^^3iF3aYHW(Gmu*Kz+BTh%1sR0>wgf)Zq8}xJ;8X zsW^wiJ)TvMLix{_fjN||Bp<1$Qm#eHM0C`kX&I1pXPP=TUAZ(UC#j+>_RHK)iB*}B zH9C9ALQ{agAI3d|GO)sAED{1*l}PpWo$yk~r9`Hf^B-#4hUNfTmwe58sHLPaW{Q53 z$^vclx+Hjy*psn97UN+D1|qkeT0_km#n(bgJOF4&!`XLC0+XC^6qST^glZJ?3JW<>elHwpqKvXvq}#AcG>hu!+vON z)sp&J$~Ox$NaGGfN>L(&?cmNlPIi*#o28bP`Jv_gE6lxgwJ30=EI|Ptn15cZ7o0=U z%Ju#ZNv%ZGu8|Ct{DTCWo@ex|2NSTG8b<#CokuG9|YK=N4 zH?sjx`ZM=;pjLYXxFdgm$tH_~hA8H&PX%zASyi*@ImJXwvxBmv#ArPFXo^gbGaGsB zi<%mNGqs^euA0ICq+Sa?Iewk5>hk4`34%8tCmOYO-0lA|ktU$9^t3$!yU7|F&`e8> zqU9U|psD%AQiJg@=4lgv)#i{+zJNHy zFYnv@4123=Cnnajx{nZ2fB63;mSeix=zX`Rd|1hsE$Ybt_!Kj>^0Bt_%!9n9kieo= zf4(3){rDm1qr8lEu^+d8KwSB6jG&X;dMj5jTi-d5_5$ZHVZuh{M*`*lDl#U~i{*8@ zdf9l4?R(nUol!ph5{8r_KrvU12zb*RK-&I?!5B@~ooNz5IFn|@l1vz|gpkp-;6Umf z2w>i09Z7R3jAa%WplC_FUrZJ^Z2BC8L#T&1-fpp(}9ocivJ-ej$T9!b!*;ws@nI* zQLlQZF+(3dC@~vTG>mM_jNjh-CH%dy3^l8j;c=F=zTy`09A7BQDdftPu-j~TQ(d=J zPY77je|oEXav}cnU>v=u8miU{-^YBV{lvmNWHV)_3Q`3|)}8x58HM?XaPeV<%CfvR z((gEbEz&SFg}++p_>b%}_uEid@s&XQOkV4mPH#c7tU!Z&*y+3JmczCAm%Dx)TV)a; z$l^`V_Ak#$>1ei}Nxh^$GZqO%>pnCSO>XI@Q96xDS;o+JrZ=Sn*YF{Yy(mKWbk{b@ zdW0ift@!c>z)72Cl)j$!eb~C;{HINPJpu%WA6xb&@j4AV=vFE0LMrZ#!fQI3eiQR5 zqR;qx=RxV&gog!v%e=xmomZmGmlm?59&MN~6``3`9KBS&Pp_3)|5ZDxCXHG&I8l(_ zz$Aka#ghbHzc{z!_Z8BQoNalCe$SnA$3wM?o326 z`tWwQMa&!4BMC0{bqz-aV1^Yp8zAOR2hS93X-qX`l2(ie-EMyVvXrv?k8w_dy}i9M zN8?hdiJ^9y?E-R?HR9OCxRkNo`dV+wBbCk?K&w5fV3OoH51n6H9S1ru<0(c_uW~0( z8(*<=1Ub`op($)JXIPghL_n3}RW%lmm zmza;xvV@L5+UUno$g@MW`~(l%rmtk!!Y{526c9{HY1o%X3e^m3GJM z_qun26^@jL6ip+H8Yl%d#iG))j{bH_jhG3YHCTu2xf3j(d?q6FleevB8}4^1xEacG zCc$FBl$Sb>ym;R>?B@GQ4DMe(Xdr$pcOdh!;j~x@BLq;<7D^^_HOfQ(c&cNQ0DoI+ zymK32X;?__n_V~a2htD`_4g^6M?anY&^^E1AMU8Q@(!d4>ul*ry861Zdu?59)~Q~m z#e`V{tX`<{1(D#!1A$S)%CzrdJn@);x}RDo(%j;yCP0F0z)q_V9iar3K~M#PM9Jy) z`qSI^#aJCZjHrYF8j(PB7}Fp(04w;4Q#-;a%8hb^wZ(|nw zeb9qmiFJgS_*5dSJB+ja;rET!4?W1GBRPY4H`+@6t%T9adccK`{i%8W3fhn}V?85c1nEjpKe%Oz?|$kkj%$aRV*s{)l<3(#!XOys7tbCeI84;$w4 zs?F(-A%fq+%unYXg)y*hrVE=y9?|==&6WF&S9%{@FOvSn_n>IZZDQtbVC5LC3Kl9_ z8`Mb39a~f>FjOL8z(lC0*_JMXn;6} zS2(ZcJC&?V;Ko8Hds>YUo)?I*VViWGLP*o|r{TH)k)BK0=71=bv1emg-S=GNqyw?( zr_)@>c1sW>JYV!0>hE~)3b0;LT8Hy(8~af3_|?i8&zXgaNaP@DPSHqfkFynj$TvL= zxRm)gAz>i0D{+|!27zdIXKg5O6W>cQA-_#qf- zzf)lbcylVH#af0UHYC7jG_a_HTsW^15l%f(7|-O_+$MwVoYWciw~STS#6;-L1g#R1 zS-x{bSOafg3$*);7I9iAcog5a>dHU9Ja-hDY>TOD5_&(g!8?w zEVZ5;{I_4#4;ViR5q=GL%~i}FC3Ik1#I>S7n0NXQDH2NgEPIcC)&B9Y;U4v8>C059 z8+YNO%7!0`hTGX`rq=fwU4GxsRnTzEjg);#z;=5i;Y6c>M>Xe|uFzzgF)TM2IuEbKhoGC*8Kw*}5=F9Vk z=>E$7NQaH`T#KCtO@k`o26pjk!QPN)Wddk}-E0sKSO(uNpIJr1)jz$O5;#k~Bt=Z~BQ5TPqI{NR#LHr)6vhl;sHXX|libdnr>- zKO7dCgJ_z4+_yxJZaov9(=1I3rn{3209?sKP|?Am&vWi+|1QT*kIbRp6I*dDXHu5|jr-7y=-P>~juhy_B* zIIJPI>&C({>FkidJVw+r#Ofl_>@#e()pLtcxQ=4SDdK`PZ@6R<0QE#Jl@gzoia9vl zIB{T#Ps9u8lz`+QWrBCyOmq+6IXX8V6CE4(Ap1lIsFL28K8aTf5#x zQ-2+MKzY|C770$Js;|k`3J<~>6abqz&FhVH^HJ;s$(*zO&dDe|N}v1=Z6h=7@MOZN zB>u|E;txg)p@gEc=*{hK$oD^}B!B&be@uVO{009wrMc&!$qVR^MPKKsTUI9x6fI|b z33xy?`5`_I1wPIj`@7r0(`g-zz;JGa@!&6aBtw%1Cu_mh4`Vl6-sKRVkqjBX zT?t3PykWR(*87bkQ`P}V&9ROt$UI9ma0~$*z61uo@8=%`WpU6T4D$TyBh@GLSnqH1 zmdPzZ?VlPH!o#7jV_$UFICs=*`Y9qXIg#fpB+VmnCa=H8JitfFjG-Bp$0emy_yS7u z4|ylNFr-42tqQ@W%UVF`t6wvEqS=ARk>mvRifZNXYMBSPVw94rENBJP$V)&g=6GVe z*%It8i}9PWhTxIggR1?fvHF;JiX@`A;2J?5q{|`CuG+Rg^9{jK{!CEko!-53jTkGu-s?|pyxh@dlg;$%wOm1#`T1x9#$uV>NJ;dQn6R!v0(rqA%P_ug6x zJcS&kHPe-Y3b%mIlJa8F213-TcPRZaJrcppP-KN0)o;9ww~wGi!Yr4!Cq|Wy#R%6Di;pN?}p$@)8lx>9@kSbr0nO zL&TvnB|ZUhXY8iT?OG8dCA=+IOf`vjm(hfJ*idUlOpz0#aIy+iz}iFm<-?PeMb@{U zba#?L7AC1ehqCQkg?Y>Z(5m+Mf*U^I=S~NYgNHys%i>kA8d2gV2=X*r5dsv;oLRq+xN#K}S;u6eKe=y7GraV(3Rj0HR*K-=GX+G<6nJ&Lfj6>KDP#A$g@or=umnc*yD zv6e@`X%KPd+p^Bq8BixvOr<$hq#?M@_C~INgW%%l^fY6aI88aD#B*c<2xDRBkGDOD zO2TQ&!xqKu8@V~knDGn{PY$K1wFDWlF5I9I(8_>HQIoCsQ@P%fd>&=gbm5tUw3t&L zXxrvitN2bK;fQCKN6E@~!-_dPggakVA#M5GX~%cubYys})KqXE83;L5`@JLkMviGJ za(SG_W?O#asvHU_3aAls(J!m}qS;(ONd@@_Y{fp%M`_KvaTrnAL99)ef#fgD zZ1XXb4R?FOd?@5ErPXE@`R9Jk^|bOZy*t0?aK}eiZAOXmi859S`t(yu@_X?rD~4Sn zEj?sICYie>Nh7}b%2O$kN-Jhay`G9Xt4!Mrdk~C#%C_e2rln<0oAoGZZq?zMX0v0V z4X25D5EDc>^+BMLcx-EI?m8Cr3SiAk4;3X}e0Y(0uZ_~a) zTS+3joJh0)pvE|B)-lH`cXmmlq^3S1A1)HfMXbg zXjSEHAM?w68O4Acu}-U6I_VH@wJbaSP+{mW=gAk|xWJejnP{GOQD6I^mgd@fWkrnsQ_{l;IKp$7 zHd5Y-Ln|D`Ex3_bN&+;m(iHdNc{9TqBm!wWh;>;px;bThA1O-9BJM^cau8vnGC@#w z!k1OeIWgR}YKXeR3W+XE|1!$B?LrRc93E~_>Q?iigk??q)GuRMcl!*kj8?TYm!A6? z%p2dIsE{4V7nDR48iF$XpG$@lk2PoUKFw{^QY|QigR>EeOwnrZ*&~23t41G0a`U`G+hnX%I2oAgKO>ov=QRSr?B#K+Oj-QoC)+CJ!7%-MitT$87 zYPjF1dYOAfPEi*>sA4btNuF}PD76TsX7N-iUjPppi(|(mIx$>6htf)bc0wO>z>F=i zt}aWi>!J800-rbffW-69R*hHA&%562vaSND@T*m~C#tJ*zWBP)FwPlIVIMXSk<7M* zAOial31@f@ddx`s&=3Mp6RQNud8~&S3W1ih-JnbA? z&3-QF*-tcy6t|FopZ=ssP^d*p)Vf{)VD!Wz5V7Od%;KP}6&utF_Pvgu9^bLZ%p|X| z_FMb-V7l$QM&BomW4lLGw@$6}yk7dNz;ea~PlXwAH63jtHj^xSG+n~#G|hZ#Q#GDl z*+m{c5j)i)*9}b9U?s!!g-%%`Zw2V2gg4uBN`Q<&|j&L&UiZH%5?) zneZ86;SOdMqJqhIoB-t**{s=?Q$SjK+K6EI2TQb7@Q|=y&H2(!3vnv}E{qAuF2TTR zBsEs0gSx@ET#UbpwW}*|*@|A`eTF>h$?baAa#U-%Whmdh`uej$WfTO%w(gQv#$VFD z^iHbD{saE6zhC=fjdA3bwhx4xN0E#E-DPpC#Z7p1gmqdYl=8gSUPEi_lja5F(CB0sRMQC$Czz(;1F}Er5m9A`haWSOfTV zre2n)z=;?nod}7@Xw7TXd5ViMG!!x-8C*eX04IM2tWI2G+OCb*y8SrA`eCqZo~J>{ zb%u~rqu-6wtbw(!x!dsUtG0cqexGBF^;@<@-vSbdVQ}OBY%)E#7uo7Z7wT`?J<=?t zjp9(d38@ue%a;u3=K9{stm=Q*4qJEqu<;j5XT!a z&c`y{r-NZ=3bEuGiND2SRm)?Oqmv`F3KMUO33T#REi%&;j~@10j3Vp>p6Qnq_j!i1n)c#APyN=SF7mfQ}(iq_c0EWtcrlxMdede_U}>NA6k1$`n-$TL_zIGta-RFV8!X+CNm)fSK&2b@gP5#2C_I?RAnIniP!qLT z!&Ybvp3;d|rKvIVGRWtJy&e-fP*&(}uSM0x4r#`Pjg(h##?}0jFUgS!BWuHr-_=mb zRq)fCK#=GWYPcs?13o-T&lic;z__p!n9__Cdi+OZUi~5ywo-id(rh){B{3_b0~1!l z?6=^M@98Li{WrekY19w#gR03_>Wf4J2K2yq<1`-5gs3r!^vH|cy5wC=Ln&CZP4h)R zZ8YOJ3{vw6XoO{8@WlNiVOIu27&}xz+iaK?0Vv=M?)V8Juk}R3tLc(#d(}_CSb8h- zbZD4VmGW*#f0r3cE*dV9iBPh)nnAfsC*P{18`0>B8(Q2rb(yyH9s+GnanS-N)`U(Q z-p5GQOBvY>0^?fww2s?e{4jWnIEPYCwO{c!8(S<85&Jv1le{-xvYXCc!gw@Ox--N$bm1Ec8By-hWTAsjWd9BE6QvIu&4a-L?V9d>~c(_xeiBxT$M zjX@IJewJu`m z{$vk!#AiAJ61Ix1yQ!#%%`Q>1g)7#6Gd7&2;#NAopq?l^Fp-ZTe>+8kzz4RWRb`|G zxm@N6OeW-(bqOF>`>n(h5ev4GAIr1b3Eb>D7d}0*ssw~5sgdwF$#1)MhHc)^9K&vlx zeCo27;3-hF$tvN)E~K(jO=Vck1uhk<=J1b;_gg`<7sk>Sfr5#Dg_d`5fCy!HzbjNB z3cE3={vy`-!@#*{e1!sWnwgJjwFekR_inD5m?$WnaddhS>iRIkIx~7mi>6RxmLh7- z#||61)5pZA9*cY2x*Ss+k!VqpCnrB$RCxa&SFT@MUL}P4ft*%0q1rDC)y@wyydzKv zTBm7F{RT3IKkhYLrW3~KA3ojg`&v;F8b%*m%ru8OS* zyW8HiW&q%LUZ0_v-riI^zxcR8>tgRt4Fm{3B!}>)D7Lk(uz=fl7e(mqoCk@rh~b1) z)9T6?+HtbUn#jw=jJv`90)lBJC$AMgHn0M3P2AyxX}b~&AcoRHgji>a6i?-4uHH+J zdC&Qq0OCF@#43-Q(a<7S_BRgdOGL2Uqde2}|lsq;5t)SMi_J&S{0B zbXdeFVzEz{2$9yNL6{9#3$_|%oc=+wpq!{J#xz!WmYtYiMk%mn4UZ8G8v#e8l0;FA zWQhpbVjaS1pe+IzHFc(V|4{KKhBEEyWj|%35F&WAzIY$#IsGWxy9%^Wz}IEgQ7cWr zQtTPA;Mp=EU4~ALxUImBRSh`m>o0(vQOeAOJm0a(bG4o3d^@0`7s7o>b-``Fe|WQ( zEv&W+4`Rnshk=VqlZdntEmE``%o!H<@#hXc6J}n@g+Er>#wH4366<}$wQ+YTcb5BO z8c7$j{7b|plh+qV2bGADE2(L=6fk=motvTwK2@k z#0&<=3)}#yel^ZEYG)R7K(;W&n6#e#m|ZR|V~F}fU`k=dh?G#?t8_m%R|_uF6tBzo0a0J1f)J}EA$Qu(w}{ftis_H#XsK_qLiL3c?6l*Uf+MLnFGhMK(+ z8`Kyfof;$Y+vmz(69%!%y)!WEPNO^V<>nIg&bTcfLyCXyXwKun@V3eECO&@jeh4xB zguN(AfPKjDw@E(BRh&>XB9!Shd4dJobHG48sw0-W}AWitMCKUk8+we z6{L$N-5#O+OTp6q-wN*h+6Y}i&Zw0=|I+Li)_y7LC>ELeF55L^bX*e z=`<-(s;Yj3?Y1g8eOySJ;B>c#o1MjtDB}qpMZvGW-gQW3 z5vPq)HJkdrHe)B1AUr-&awr^WBWXF>=rfs;8D+Y$ld!iNL8)@Fm1D7(38W;enfYP( zVjhv^wiCKQ%upzzCjqb{^kXHL%vJhG(jQ<7t>k34yphr(ce{(O6&TOj)9}spr$IuA zb-Ztz69<3IFxRoYhcKl3Jo<2Ck8SHY*ICXA9soLY{l*4y`C z%6F``Z>^Gzqu9QvEJl~+~_P$C!+ z1tQPYyu{;a$ifpfTIy~k+@_2CpZ#Gob6Qmst;y{4_^gbRCP|_+6zqum@;QTMe6PfI z)vCDLmpxF8qU?SVCF(%0qZ8Ru72MZbXbK0H(fpMa^v3oS7QK)(t_=#0WOG5ADV7v< znzy|bEI6=NJq91~49g3fss3D)|E>p&bSDPB57de8lu0jD`Y-LWnLhh72~*DxiGg&X z;wY=RpdOwjg@zW_qCn3*o?AZA%&?{8ZY`^v+|;^p1$8)GQ7BS&qnjm0^TN0{F_V!YdQny@AOjKBgFZ!xFfagXD7j%h5knJQJgY|5k9yKCj5N8ZyO z-M|ZKX&X|^Gl3cUh!#dp7TQAWrABly@ zVGyI>&coN=Kd#qATeCMH^Mk^4D|-Gwjm{WYgFR$L+EP6D!J|8J2RIP0oWDmlRlJD< zeumpBRM6txj9s2%PVY&nd)anV00nr_x!36}2o-yLlc$#=C9L+`63MtIxz8e}QXj>$ z=A+{g5`LHKxl*E2tX%oFUHFD#Mi}p}#@XKks^sC#ggGPNc?Ik0$_9uIQn2`&A5szn z^}OK;x4-^L+oc&Q&J26?J#&Ba`>|vCD1aJv$#MJ&5>sf%AEBp(j|y^B%x4%?wo1eU zWINF7c?wgd0E`g*i9DB?Z(Zy}vnAQSoh@YF2N!7@=rC@XH1dO{O-Y-tDqqg&w+HPP zOd>#J6bbto2bs}~=uJctVfVX9L3(75IBh&h@91*#!9fu2@%gnzNlh z>X*`xeC(@Lc>lTkW)N-(sxD=u(p+wUo_6s-p!US9 zR>f_#z#|5DnA>wWgy#mK?8MsG$?}Hy3-4lxISio;MeB`oCxg+yVt!b^^GVq}Zs?ds zVHyT-*3%VzqC+O8uf`g$y)Cv81;-@C$_CRUVIY*TAi*+1EG13A5Dwr>S`x$yILIub zTeF^^5(S+?I%}W?M2GxYH<$;z9nwdDBIMof5wwe#D8L4Qba2dF_FogAPH0CI-$;=S zqBoq}5{`0jk460tiumr<`sil4d`fnTS zge*&)C8tR?L@j)W;q9(OYn6vVao6H}a+{{f>n0!qsvEzJVhQ-)GsOmr$`fVmYh(1R z3QJkF)r!K}%JD>T#xt}ZL_uy{r*;W-3f*_X^ym`K)?0(TA(E)=+dEy)nyr|mJnP@< zKZF#fUb=?~k7$BHT5_|-WfRzVdS50)$bojdJJx z4w}GXfe0AzyMix)p{_N*HBUhEL+-p+c@bY$q)^87HY7oW3FyLQ9+qFM`2GgLv7?P$ zv=+4>SZ4^^NGfs79H!hk|6=yN9T|C`^EPv$$t200SgbN9|C-l7`bG)pnZ2GMAu!2O2tX~{F&sI66-2>mv{35_ zwXKD!^Y-mph~<{ciRmn@5?#v2yyRz7%i~}mJzNR0o~9D?s6||=t>v|c(SvcK2|&r; z8S!!aAn{~?>kd{(VC#4Jwfsf*b40oTm#FODa3u! z*_1@MdCyF}1g9$GuRQ07Hml?#&5xM8B+l)CK8P^Z)~1j}i3zx25{hJur1o>_kP(1q zWMDvI@_Okyw$D@=G!mFeYX1m}eS_R6+_RTIgZ$Tv{+VAgJZ0Wz6mq73N(p`7^+7$?)lbFW&MbD-DoyAM z3jDx*H=W%BTH1k`VhDlJHmcSN##E_AW%bR#n0?C0?Zl7ek5w{J8xtMTJDSMTUwz5S z2iW|#(Q^Z8EuW$>Ff5`np6CjFBv8-A?&+FTbWKT5lC`jNM>VKvQ;lOJO#aRehTRQH zCKnYXxUGBJf)fl=Jq$VIgb!%p!DH<07t@*DW74E7p8&Br)qBJ_WG6147m>bo`M;WP zjuZW#47^P?im&}Xxt-zsoCOr~HQXrl2$Nyy_ekK$?@Cq#5l}DU<^C;o2_GW_a(ghT z1#X;Hdr}c{qrT9(4-*(aQ*VCe?@n#OJo2|V;x`X-o=4vWv@=)QCK!zE zPPgUW{O|Nje*p@kIZZjsy{2?H4`Vj+h^l66!mlJz;4w~MOp2C%j?sa~CJE&`Jv-iX zjFrq6p~z_@^og6>is#t5cM6z^4Fe*3gC zHvjALgSQyNzC|Q?TOX4$kTjW3zuMxW6p^bjA)ByRdOsnULNV*A576U##u;U6)0zJp zZ%Kk>Sj3H94vBUcr>bmlXJGoVm-{wSPWZ1|^s`?yuliQ!w}{Fpsu-UiO-dr3bTCTO zVh`f{=0^P=2d!s?Kw12fhD4gp-Sz2ogh`Z_Xw`!n7^lh&XK>8_+Ts{8^3O=Sr?ws> zv1(JyR=kjS-qu^to1W1r$F4Uc`Lg|z+uY|t4Z%0gZ zz5k^A*D8z{Rbs*C$hJB#Y5nCr97-d($szq?Sm;ML*v%Mmjf@KMNVrR>a%!2$C@*V6 z*$4q`G4FF3MDhFs;Xy5d%SFSW*}qd@GZ?G22z3p{+QE~beY&FL9Pm7rdjVjjdpYm?H8?27w-Nl-RzqKVr~RBSXt&j!{b-|@o0ynb z53eh9lRm{VY$<5&dES0x)Jj4yCPH^d?DhV1WW1CJd)|4`J$Bylr)JQ(Kv4HV@BV<< zuiX-)b$YQ_t%Ia?GA;c=6H78B?Dui{N6csDYUXaLC4RYc<&Ns_W>@E#IT`$67C|$x zL+I{!@P<=RkSUJxB%SRg-+KOU+ju)8^97FiB!FbVTarA4kG5duHmP=e}$~gwf ztAX`Ho6^>W*Vc;V3xb03cXVc1n1~3IRxihCoJxu2)&*D&-5yg;Td}1f12FF^ z1p2%%I#|ZW#tuG?j)OaYSx``5E%3gl(<35sna7d%Sv`-Zo9hr4)#o;yhF$y`;}w3} zZa`dC*xB@QZ@pF2@99n(@=5^OY{StxvJpSVttBc5YLN!oA9(I|Wnfvs z@aN1^ImO@;Yuv5lFmHD|17Gw%o1=8K+S9{=&1pPs4zsJEZ&b=NL!N$I$$9%7Ikn$c zQ7H)AVPQD%g!u?5GJPQlg&En3NVNMB_SEZh+gCS$czGmu2Nk72N|`1_!Mfcljj!-#iWL zb2a8$)^BfUwI$vZ2@UCkkWCZ&)<8AHGH0dQ0zbYw^mrB6(o%s0?|cBZByio~3ghUW z(|d~=+|fJFN`DPG&UKGHjXsP$;-jV^M!`XtmSlAbKLv@KpdDH6d0Bn^&-J!Y+^)`A zqhK@jZZ199>#0vNZk()-<`6E6Ni14=B-ZAhrhm+^7w5JrEC|`t)4vw0=W?wd`)^m} zuk`6sSX`|;1j!D(854}=&6hEQ&X1RObM$@UrgjA0iI3z%smiKFq!o)X(rx7pwg$3> zQ#&g}-?{>qy_UBZPprAu`@8p9yKm&L4EZ@DZ6|l03hKzce@;QtlxGdzUBv#9~&e~ zS#G0HIHhro*x%2EA{N=jtK+X6jyAla5#>MFqod{^_>{QhJC!~c0v_&y1Z?{N83BbiS` znLjWi0Q@}MyU}L`(c5_E5a`XOKSb>eXdWQzA8#pdxZ0aU*7rm;|M_JD%L_j4hO|jB z1>Srhu8Ei69FCh`>7$Axh>VlJHtMml?*_(J$ew73c-60RsT?u0jfUVn!n zijhsq;g(;nsUSA_RD4nb+yt0u{pcIL+cF1SJ3TnM8~*Esww1`W7UglgX-&T$=&_lMKp>3P+fWNTf(b^np^-?xjc#6L+H0`L(a zorRxnTUKideI)-1g6^DNPR z)gm6uo(6P3TQQzZ`XgJXi>>lQib z-gV|XVl}d2bA?#z^NtQz7R*mDL=f+o=9Logq6WL(NC&f)(6=%u6+1S*&5y7^YO@Cz z2`(2Jttd~4c`vM?1N)j$+13~2_d4{Y)G)KxHG7~4CO;rAJJSxv!cz%<;(5YA26eB0 zBD1>e8z;I^hMC8ml?b4bBfcnU09;L~ye1UvWOc_C7l(1VWZ!>mACbWpmR8d92}QMS zuEVz+Uhpy?0OBqAG>;KNApg>W9NV0aHSY*{>zw*n&~B(#uRI))`{%wxR{u*ocguIk zlNXHc{IZ!)2jbH2)CrFb z*^24$MYJUMSUMJtn4FyV?P^QwuJpEEPAwfhZC>k-?+t~Ivy&|af?pq$Waglk4Yq?# zA-1Xw^Z(j^(RMty{4&4w_~^k3H>kgzEr?!97uTY;3VZ*;y$n3U8%lAQJdxka`oQ8D z*e;M;P)#1A^)~uE*`v1gmpD;_p*!(QJed43P=PoDj2hCpZNgZ}?#sx#+w0?10=xNS zuV-HQ6eqfR59E>^bK{i3jd}cN4eeI|Vo+^}uY-mnWkioa{)!w+nY6>~O;1Lcm>5LM zNMu;9%(4;S<@pP0f+uDRyoI7T8=qLC>`#E#2s-#eG}muZ1>&Ur#Z5Sm37E@{=V>~{ z{}(tH$LKXcYU5U#l7mXzr+vqZ_Fz<#IB7a;1|vxnWI)$sM7B8~Nh_-IQ+AA|DoN^- z1uTPu^yR2pO$a_keYPvvk`!3~A=1Esb(GeZmc*4-`av7fn*lEc^`BBHI1B3l)V7?8 zyT5vnjRn5b_`Hd0t@N)YyCS~$dteiw-z`#M3vGYvh|0MhG|1zCC-3p{G9>PUlwDf; zfFzxC>ag)JJBd53I&+VjKs6zKdre@s zQcFe%jq5a$3%D3(=IoQ%tABDRbM=ceunj>kA==i)&zzpzr}fx*yeLeoVL;}?|bsyovx~S(l z7k=zLE;Zk_)#nWW)`d`7MW`Ot_P6!0|J}GD$257Kjoyvx*WV+5f~QFpj_rJU=I~jh zA+xB}dyg(G6gTyHp@>5NwX7vD&YAePaK2H$+T$|I)b@iEh7nzl zk7_Xzico;&@MNe(O(*i*@bSN6XHK7LnV85>Zbqg{{Pj-juw4}#^&3_ilcbj*#^^FrnkmZ3B>+_$ApCDnJiTzm|* zRIiYOlIyr13j*jUYMvJ*t({`GOYdg5!`>81oiK@V*JA;Q2bY)N-Y>qEb(WUc`%(0o z(%y{)`ct}1#D=pMhdsdsJA~wQ2yfJ(KEXcRC)|C1^4c@bpIrU?zR$ll@yIi;Ju_vA zO@Z8Q;~r#sNl;BD2S&k?z3ot?kq4} zswRoZ24wK3OI6{U+k&*me_&4YfhyY6E@*sIX z8hdaYD2ofIAZ!v5j>bHD$3Y1ZCl-pEC%u)+Wm3A$7!S^*cYu~JF5WihL>iU0v~ILL z+24#uF;3(=u6nPL+?M8XC+xiwc*NJrOZ@#gKN~v>8Q2ZQWdVmt%~&ZbH4qZpUL0VM z%Gn9LN1y!Dt6!J{i-yor#JOAZ#fc}MeA1`cj9It1$WRrhyoqpTlXxvj?Siw?9L-9Z zxVeb7o(W1T#KQcaLdMmstduh}s4F$uqz^p-QcbdRIHP+QA2q+;?VQg2`0;V6nk1EE zz(rFjDG<&{8efQnl86wmXO<}^1P@vmQ0x1_fqep{&cpP=A^|+yC=FCS8WHMd+hYiH z59oQL*i3p#adMV3sm2Ivdfqrr&>U=w6Ey7aiD-YQwKH*bB-5zifu!aG>2UzZ3Tom zdN&rNGJO2;rzV(F{uI7fFX6TSB;J$R+Ivz2RG;3%!J#ZSdy$|GkXsY9yaPRS(%S z&L(9yv8aM(s`$A49daZf2NG17j2gQQ+1f}Sjm_rpWoYQxNcK9C7Vz^l*?Op-w~NJt z^Tpy|t^Eh`FPuJ=g&u*cB(tKt4hgejnq|@;O7U;aRFcx(BE%8n{-4xNhXoWco75$cQjr1wAvjFW5n9l5X?`x}MBw6h)Aw4?Wej*HH@R0?uPY~RHuoI)d#o~b2N zbusSXq#yh^Vk6K%ziU_D4B4SbTd|q3s2}*iZh=~de~4d;c)v|lY^RbyqQpW`DZ8k_sa*{#a@MsecyTo%-12%cU zvZq`0E){g&>7K=XPNENtv|*1-JU6nxX|-`XP7_1XdqKxV=ZqcvK0PO{BR-aNY&5Ycl}a9|yYYF-Hrfwq zTv`q47H!_WpZ{EK1}&Vo4p5)N^V+Ap8f)(fp2sOX&bgo>9{jU2xt8qrjrL0#D;@u~ z0G%6m+E1KJ64~vyDHj@Jj#!;l<4dn)r=I%N>|U8dX%k{wdJ*`5(94$l^ry#$XqJd7 z-r|lO$;Rm<(2I+I@|60iQc;1!_}6dONdYCMLZ#9gsWw9ph$r90y{&?s1jSI99Z**lXY9C8xr?}vtH<$9-5YqvM8>I+~N zRY8Z08~0jao;UN{(z3NYZw5LWPgbU#18M4L`#mt{)U;jwlMBCoALbxj9KREfMHo4hskz#g=^IIqFkcEYXg+=e6Epp4G&(S7nN`xE) zK|>MyxvQBOSl)pQ zN4fO8OIAr*SXfwCSZp?0mfT1sNiP2MT?D0VnFG?o!otF$H_$S;>k`v>Xp- zVPRomVX+BlSuH6<-%-IR)&JU4;8<8#SXfwW1X_~Zq)O6(G(`@U0%lc`78Vv37F~^& zB)9AD%%=eurFANNQk4e_3kwSii!Mh~wIrz|1+ZaC z)EPM3sw6EeEG#U#9!<&ZXQ+}CRHB_!lCxGNX|eUt4Rz8s3ft*i35}|I$F=QrpB9^k zriqjP;Mbqa9b;pki#^u*V=$nK?ezD{U;c7~ zKF4tTKK9&m&#gKpDj=%PWo3Z#oyDCySMP_~-MF#D&YXERb{}-jip3>1FtAHR2lr;z zDcU~iK5flWZov!K&KaU8(7MQ~>QvEFG0NySZhZtuU!4Y$^f7%a-2_Z|2!BOZK#7x|E7YVi1 z>f3-S*gM>}gS|zSoxM5{~7aZpwScfpbpffw_Y9ooJ33_KP&jskwBP>neNoq~Y5r0Wug zgtqy|PmE=8GJOuES056J9qU4wa^3Oc<3&*?Hz8qV858GTn97ZtWk{B@)F)0HUc&p7 z$7@m4_v`nmkmw7MLZYZ4XlH2dzpM_;nIb0GIwgf9yC26r`fDgU2vN-*44@n4}m~~ShG#H{RD9IIP^ivGP z$!5}n+i(ZngYb^G85 zSacPVB{v2`Wl`}0a0*|W{qKFS=s3s$0AQ!p*SCRs2aXu9dS?w(FGAEw!gb+%z~O0^ zvzJf>Hi5RD8Jw;AagH3V!boj6hK|y2RDbV|2{#3#NMu$j0ji@ZZdaD=0$ze5{f^lWo(J>r?*OfHU3U@5?i!IMO%TlaXq(+2HOC3l z193+}jzB@sR`Hc>juo18HG9WfLgH0l5fxNPI^cqv2a9z98NwNqo(79{(9lkd>OgW1 zsa?pVx@e#INfHZAEDoWhMTVVIxXl2np=MTAZcab-)Kk;XKKpFjFPBzz&YeCzIc@fV zDz+&cNy&aR9Xr4?_SHUKyNH8WB->bD-yR=208*{uxjv}R^(>w%EU&0@9UIH$b0{d) z!Ao)~33vl;V~iX{R@v}w+*o-GpuOY|C@9V1!~m9;p>1*_hY=2Z#sT&;sS5Fa`pe4< zii^@__7XOclL;awBym*xmi-`bCQqFB++*ZHy0RjG^WY?^HTMz=(lPwfFWpsKxxSbO zw1qC*ym=n`{HvuUB@GMC{?hbW4yvnoUQr&TsJ3Kq0HwTHJlWIC zf#LpsDAKcl*RD|E;u#3o%gaG|76resPy9XO16V#5(Hxy={75_1o_ zzzSc#`b8fs)&<(;VXKvQ5Un_rL{&=!?c7j6lAA`7yM(iI8UQaqi%>4~%tKNgMG0`U zx`wHLXg@*}>IsvV#bVvD`tmqVj4jeO&;fFbs^;|bmIvjyL49XXy2(SWBSi7cnOr`u zFC5@KB!YNyYtx~m(eYkipzAOEX7@1(6kmw_j3II5Dit8FQ-LHq!yRW4D&lF!WjV7i z#2nt6eKY|Mo+heLn77K)v7g z9p41RlG_ri9CqQK3Fy7ze8Ysl!A?+|z1*T58MS;C>11 zGT)48g0p-!mL$E!9pClqMITA=jy)Mo92cq|7X}8Z894McIFHiFNhvLMiI*Q;OKHPG zRjvRw1$PK7vHGTOihaYW#z0hBj*x%BER#+>4oY}T6q@yk>dt*A!3_sxr3J9)9;_la zQb{iUbpD9TIHUw`dG=~y(IsF})u2HIr`t56CkFw>*;h+}m_e!K8ML+d{rv*SF2nf> zNlSvHKWGsYw2c}nuUr-*Ns`9#StNz@~ zdBwG*bESY|1~(_>=lNfd{Yl!27U&u{CvoszSc}>)331n*5SQ&E+LoL<=-lPC zC4=|$zbdc5$4sBpcD`8c({WUvNNlg*b4DBi8T=6E_T8e0ExhAN}JkILThVn!^0~HHUedshB z+{82vmXkP;W+9@YLiN}wJl3kpiUF$q7#;`sr4{kW>YcIla$N=zb6#SZk3C4d^IE*ZHwGCK-VA12L)hp+I8Ai$A;uC zB{Yf;7qe(Pe*!GJ4y)CYfs+oHbm6j6n?=t6A7sA*Vj?M9 zsw3E(`L1n4`;;RI9Kb>E9vtg^3K2NdV6dmm9w;dJHOrWosjM{i1|%C6+PFM&MF%~| z%Pb)@B-b7ZX*Y;2E@9TASS6@fX|dSVL#RH3?} zkvgzQ$4yQawTs2HkY1MY8}a38d)kMSs=ZIR?E)UtV%#w`hnU9m4$qnFnj|6PL}H(` zjXB}~p78>3k&Omw6P|zmd4Xj3uv8i2bAWvf;BjQ}I4+!dv3eX@b-57JXRf|ai|#?q z<48C0fBj$n=4BB4Fd9f7{PBqX^ZK>GUYf{|9G| ze5hW3C;)dyx!}Tez-A?}u&}VO z=xIpF?f(0-6#OzwnF4OF2rIc+SXfwC^fV;7mCODRsv}V)DQ0{>Z@0P#2<+CTsR002ovPDHLkV1m4PbKL*{ literal 0 HcmV?d00001 diff --git a/site/docs/assets/javascript/lottie-player.js b/site/docs/assets/javascript/lottie-player.js new file mode 100644 index 000000000000..146eea83437c --- /dev/null +++ b/site/docs/assets/javascript/lottie-player.js @@ -0,0 +1,77 @@ +!function(t,e){"object"==typeof exports&&"undefined"!=typeof module?e(exports):"function"==typeof define&&define.amd?define(["exports"],e):e((t="undefined"!=typeof globalThis?globalThis:t||self)["lottie-player"]={})}(this,(function(exports){"use strict";function _asyncIterator(t){var e,r,i,s=2;for("undefined"!=typeof Symbol&&(r=Symbol.asyncIterator,i=Symbol.iterator);s--;){if(r&&null!=(e=t[r]))return e.call(t);if(i&&null!=(e=t[i]))return new AsyncFromSyncIterator(e.call(t));r="@@asyncIterator",i="@@iterator"}throw new TypeError("Object is not async iterable")}function AsyncFromSyncIterator(t){function e(t){if(Object(t)!==t)return Promise.reject(new TypeError(t+" is not an object."));var e=t.done;return Promise.resolve(t.value).then((function(t){return{value:t,done:e}}))}return AsyncFromSyncIterator=function(t){this.s=t,this.n=t.next},AsyncFromSyncIterator.prototype={s:null,n:null,next:function(){return e(this.n.apply(this.s,arguments))},return:function(t){var r=this.s.return;return void 0===r?Promise.resolve({value:t,done:!0}):e(r.apply(this.s,arguments))},throw:function(t){var r=this.s.return;return void 0===r?Promise.reject(t):e(r.apply(this.s,arguments))}},new AsyncFromSyncIterator(t)}var REACT_ELEMENT_TYPE;function _jsx(t,e,r,i){REACT_ELEMENT_TYPE||(REACT_ELEMENT_TYPE="function"==typeof Symbol&&Symbol.for&&Symbol.for("react.element")||60103);var s=t&&t.defaultProps,a=arguments.length-3;if(e||0===a||(e={children:void 0}),1===a)e.children=i;else if(a>1){for(var n=new Array(a),o=0;o]+)>/g,(function(t,e){return"$"+a[e]})))}if("function"==typeof s){var n=this;return t[Symbol.replace].call(this,r,(function(){var t=arguments;return"object"!=typeof t[t.length-1]&&(t=[].slice.call(t)).push(i(t,n)),s.apply(this,t)}))}return t[Symbol.replace].call(this,r,s)},_wrapRegExp.apply(this,arguments)}function _AwaitValue(t){this.wrapped=t}function _AsyncGenerator(t){var e,r;function i(e,r){try{var a=t[e](r),n=a.value,o=n instanceof _AwaitValue;Promise.resolve(o?n.wrapped:n).then((function(t){o?i("return"===e?"return":"next",t):s(a.done?"return":"normal",t)}),(function(t){i("throw",t)}))}catch(t){s("throw",t)}}function s(t,s){switch(t){case"return":e.resolve({value:s,done:!0});break;case"throw":e.reject(s);break;default:e.resolve({value:s,done:!1})}(e=e.next)?i(e.key,e.arg):r=null}this._invoke=function(t,s){return new Promise((function(a,n){var o={key:t,arg:s,resolve:a,reject:n,next:null};r?r=r.next=o:(e=r=o,i(t,s))}))},"function"!=typeof t.return&&(this.return=void 0)}function _wrapAsyncGenerator(t){return function(){return new _AsyncGenerator(t.apply(this,arguments))}}function _awaitAsyncGenerator(t){return new _AwaitValue(t)}function _asyncGeneratorDelegate(t,e){var r={},i=!1;function s(r,s){return i=!0,s=new Promise((function(e){e(t[r](s))})),{done:!1,value:e(s)}}return r["undefined"!=typeof Symbol&&Symbol.iterator||"@@iterator"]=function(){return this},r.next=function(t){return i?(i=!1,t):s("next",t)},"function"==typeof t.throw&&(r.throw=function(t){if(i)throw i=!1,t;return s("throw",t)}),"function"==typeof t.return&&(r.return=function(t){return i?(i=!1,t):s("return",t)}),r}function asyncGeneratorStep(t,e,r,i,s,a,n){try{var o=t[a](n),h=o.value}catch(t){return void r(t)}o.done?e(h):Promise.resolve(h).then(i,s)}function _asyncToGenerator(t){return function(){var e=this,r=arguments;return new Promise((function(i,s){var a=t.apply(e,r);function n(t){asyncGeneratorStep(a,i,s,n,o,"next",t)}function o(t){asyncGeneratorStep(a,i,s,n,o,"throw",t)}n(void 0)}))}}function _classCallCheck(t,e){if(!(t instanceof e))throw new TypeError("Cannot call a class as a function")}function _defineProperties(t,e){for(var r=0;r=0||(s[r]=t[r]);return s}function _objectWithoutProperties(t,e){if(null==t)return{};var r,i,s=_objectWithoutPropertiesLoose(t,e);if(Object.getOwnPropertySymbols){var a=Object.getOwnPropertySymbols(t);for(i=0;i=0||Object.prototype.propertyIsEnumerable.call(t,r)&&(s[r]=t[r])}return s}function _assertThisInitialized(t){if(void 0===t)throw new ReferenceError("this hasn't been initialised - super() hasn't been called");return t}function _possibleConstructorReturn(t,e){if(e&&("object"==typeof e||"function"==typeof e))return e;if(void 0!==e)throw new TypeError("Derived constructors may only return object or undefined");return _assertThisInitialized(t)}function _createSuper(t){var e=_isNativeReflectConstruct();return function(){var r,i=_getPrototypeOf(t);if(e){var s=_getPrototypeOf(this).constructor;r=Reflect.construct(i,arguments,s)}else r=i.apply(this,arguments);return _possibleConstructorReturn(this,r)}}function _superPropBase(t,e){for(;!Object.prototype.hasOwnProperty.call(t,e)&&null!==(t=_getPrototypeOf(t)););return t}function _get(){return _get="undefined"!=typeof Reflect&&Reflect.get?Reflect.get:function(t,e,r){var i=_superPropBase(t,e);if(i){var s=Object.getOwnPropertyDescriptor(i,e);return s.get?s.get.call(arguments.length<3?t:r):s.value}},_get.apply(this,arguments)}function set(t,e,r,i){return set="undefined"!=typeof Reflect&&Reflect.set?Reflect.set:function(t,e,r,i){var s,a=_superPropBase(t,e);if(a){if((s=Object.getOwnPropertyDescriptor(a,e)).set)return s.set.call(i,r),!0;if(!s.writable)return!1}if(s=Object.getOwnPropertyDescriptor(i,e)){if(!s.writable)return!1;s.value=r,Object.defineProperty(i,e,s)}else _defineProperty(i,e,r);return!0},set(t,e,r,i)}function _set(t,e,r,i,s){if(!set(t,e,r,i||t)&&s)throw new Error("failed to set property");return r}function _taggedTemplateLiteral(t,e){return e||(e=t.slice(0)),Object.freeze(Object.defineProperties(t,{raw:{value:Object.freeze(e)}}))}function _taggedTemplateLiteralLoose(t,e){return e||(e=t.slice(0)),t.raw=e,t}function _readOnlyError(t){throw new TypeError('"'+t+'" is read-only')}function _writeOnlyError(t){throw new TypeError('"'+t+'" is write-only')}function _classNameTDZError(t){throw new Error('Class "'+t+'" cannot be referenced in computed property keys.')}function _temporalUndefined(){}function _tdz(t){throw new ReferenceError(t+" is not defined - temporal dead zone")}function _temporalRef(t,e){return t===_temporalUndefined?_tdz(e):t}function _slicedToArray(t,e){return _arrayWithHoles(t)||_iterableToArrayLimit(t,e)||_unsupportedIterableToArray(t,e)||_nonIterableRest()}function _slicedToArrayLoose(t,e){return _arrayWithHoles(t)||_iterableToArrayLimitLoose(t,e)||_unsupportedIterableToArray(t,e)||_nonIterableRest()}function _toArray(t){return _arrayWithHoles(t)||_iterableToArray(t)||_unsupportedIterableToArray(t)||_nonIterableRest()}function _toConsumableArray(t){return _arrayWithoutHoles(t)||_iterableToArray(t)||_unsupportedIterableToArray(t)||_nonIterableSpread()}function _arrayWithoutHoles(t){if(Array.isArray(t))return _arrayLikeToArray(t)}function _arrayWithHoles(t){if(Array.isArray(t))return t}function _maybeArrayLike(t,e,r){if(e&&!Array.isArray(e)&&"number"==typeof e.length){var i=e.length;return _arrayLikeToArray(e,void 0!==r&&rt.length)&&(e=t.length);for(var r=0,i=new Array(e);r=t.length?{done:!0}:{done:!1,value:t[i++]}},e:function(t){throw t},f:s}}throw new TypeError("Invalid attempt to iterate non-iterable instance.\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.")}var a,n=!0,o=!1;return{s:function(){r=r.call(t)},n:function(){var t=r.next();return n=t.done,t},e:function(t){o=!0,a=t},f:function(){try{n||null==r.return||r.return()}finally{if(o)throw a}}}}function _createForOfIteratorHelperLoose(t,e){var r="undefined"!=typeof Symbol&&t[Symbol.iterator]||t["@@iterator"];if(r)return(r=r.call(t)).next.bind(r);if(Array.isArray(t)||(r=_unsupportedIterableToArray(t))||e&&t&&"number"==typeof t.length){r&&(t=r);var i=0;return function(){return i>=t.length?{done:!0}:{done:!1,value:t[i++]}}}throw new TypeError("Invalid attempt to iterate non-iterable instance.\nIn order to be iterable, non-array objects must have a [Symbol.iterator]() method.")}function _skipFirstGeneratorNext(t){return function(){var e=t.apply(this,arguments);return e.next(),e}}function _toPrimitive(t,e){if("object"!=typeof t||null===t)return t;var r=t[Symbol.toPrimitive];if(void 0!==r){var i=r.call(t,e||"default");if("object"!=typeof i)return i;throw new TypeError("@@toPrimitive must return a primitive value.")}return("string"===e?String:Number)(t)}function _toPropertyKey(t){var e=_toPrimitive(t,"string");return"symbol"==typeof e?e:String(e)}function _initializerWarningHelper(t,e){throw new Error("Decorating class property failed. Please ensure that proposal-class-properties is enabled and runs after the decorators transform.")}function _initializerDefineProperty(t,e,r,i){r&&Object.defineProperty(t,e,{enumerable:r.enumerable,configurable:r.configurable,writable:r.writable,value:r.initializer?r.initializer.call(i):void 0})}function _applyDecoratedDescriptor(t,e,r,i,s){var a={};return Object.keys(i).forEach((function(t){a[t]=i[t]})),a.enumerable=!!a.enumerable,a.configurable=!!a.configurable,("value"in a||a.initializer)&&(a.writable=!0),a=r.slice().reverse().reduce((function(r,i){return i(t,e,r)||r}),a),s&&void 0!==a.initializer&&(a.value=a.initializer?a.initializer.call(s):void 0,a.initializer=void 0),void 0===a.initializer&&(Object.defineProperty(t,e,a),a=null),a}_AsyncGenerator.prototype["function"==typeof Symbol&&Symbol.asyncIterator||"@@asyncIterator"]=function(){return this},_AsyncGenerator.prototype.next=function(t){return this._invoke("next",t)},_AsyncGenerator.prototype.throw=function(t){return this._invoke("throw",t)},_AsyncGenerator.prototype.return=function(t){return this._invoke("return",t)};var id=0;function _classPrivateFieldLooseKey(t){return"__private_"+id+++"_"+t}function _classPrivateFieldLooseBase(t,e){if(!Object.prototype.hasOwnProperty.call(t,e))throw new TypeError("attempted to use private field on non-instance");return t}function _classPrivateFieldGet(t,e){return _classApplyDescriptorGet(t,_classExtractFieldDescriptor(t,e,"get"))}function _classPrivateFieldSet(t,e,r){return _classApplyDescriptorSet(t,_classExtractFieldDescriptor(t,e,"set"),r),r}function _classPrivateFieldDestructureSet(t,e){return _classApplyDescriptorDestructureSet(t,_classExtractFieldDescriptor(t,e,"set"))}function _classExtractFieldDescriptor(t,e,r){if(!e.has(t))throw new TypeError("attempted to "+r+" private field on non-instance");return e.get(t)}function _classStaticPrivateFieldSpecGet(t,e,r){return _classCheckPrivateStaticAccess(t,e),_classCheckPrivateStaticFieldDescriptor(r,"get"),_classApplyDescriptorGet(t,r)}function _classStaticPrivateFieldSpecSet(t,e,r,i){return _classCheckPrivateStaticAccess(t,e),_classCheckPrivateStaticFieldDescriptor(r,"set"),_classApplyDescriptorSet(t,r,i),i}function _classStaticPrivateMethodGet(t,e,r){return _classCheckPrivateStaticAccess(t,e),r}function _classStaticPrivateMethodSet(){throw new TypeError("attempted to set read only static private field")}function _classApplyDescriptorGet(t,e){return e.get?e.get.call(t):e.value}function _classApplyDescriptorSet(t,e,r){if(e.set)e.set.call(t,r);else{if(!e.writable)throw new TypeError("attempted to set read only private field");e.value=r}}function _classApplyDescriptorDestructureSet(t,e){if(e.set)return"__destrObj"in e||(e.__destrObj={set value(r){e.set.call(t,r)}}),e.__destrObj;if(!e.writable)throw new TypeError("attempted to set read only private field");return e}function _classStaticPrivateFieldDestructureSet(t,e,r){return _classCheckPrivateStaticAccess(t,e),_classCheckPrivateStaticFieldDescriptor(r,"set"),_classApplyDescriptorDestructureSet(t,r)}function _classCheckPrivateStaticAccess(t,e){if(t!==e)throw new TypeError("Private static access of wrong provenance")}function _classCheckPrivateStaticFieldDescriptor(t,e){if(void 0===t)throw new TypeError("attempted to "+e+" private static field before its declaration")}function _decorate(t,e,r,i){var s=_getDecoratorsApi();if(i)for(var a=0;a=0;a--){var n=e[t.placement];n.splice(n.indexOf(t.key),1);var o=this.fromElementDescriptor(t),h=this.toElementFinisherExtras((0,s[a])(o)||o);t=h.element,this.addElementPlacement(t,e),h.finisher&&i.push(h.finisher);var l=h.extras;if(l){for(var p=0;p=0;i--){var s=this.fromClassDescriptor(t),a=this.toClassDescriptor((0,e[i])(s)||s);if(void 0!==a.finisher&&r.push(a.finisher),void 0!==a.elements){t=a.elements;for(var n=0;n=0;o--)(s=t[o])&&(n=(a<3?s(n):a>3?s(e,r,n):s(e,r))||n);return a>3&&n&&Object.defineProperty(e,r,n),n}function __param(t,e){return function(r,i){e(r,i,t)}}function __metadata(t,e){if("object"==typeof Reflect&&"function"==typeof Reflect.metadata)return Reflect.metadata(t,e)}function __awaiter(t,e,r,i){return new(r||(r=Promise))((function(s,a){function n(t){try{h(i.next(t))}catch(t){a(t)}}function o(t){try{h(i.throw(t))}catch(t){a(t)}}function h(t){var e;t.done?s(t.value):(e=t.value,e instanceof r?e:new r((function(t){t(e)}))).then(n,o)}h((i=i.apply(t,e||[])).next())}))}function __generator(t,e){var r,i,s,a,n={label:0,sent:function(){if(1&s[0])throw s[1];return s[1]},trys:[],ops:[]};return a={next:o(0),throw:o(1),return:o(2)},"function"==typeof Symbol&&(a[Symbol.iterator]=function(){return this}),a;function o(a){return function(o){return function(a){if(r)throw new TypeError("Generator is already executing.");for(;n;)try{if(r=1,i&&(s=2&a[0]?i.return:a[0]?i.throw||((s=i.return)&&s.call(i),0):i.next)&&!(s=s.call(i,a[1])).done)return s;switch(i=0,s&&(a=[2&a[0],s.value]),a[0]){case 0:case 1:s=a;break;case 4:return n.label++,{value:a[1],done:!1};case 5:n.label++,i=a[1],a=[0];continue;case 7:a=n.ops.pop(),n.trys.pop();continue;default:if(!(s=n.trys,(s=s.length>0&&s[s.length-1])||6!==a[0]&&2!==a[0])){n=0;continue}if(3===a[0]&&(!s||a[1]>s[0]&&a[1]=t.length&&(t=void 0),{value:t&&t[i++],done:!t}}};throw new TypeError(e?"Object is not iterable.":"Symbol.iterator is not defined.")}function __read(t,e){var r="function"==typeof Symbol&&t[Symbol.iterator];if(!r)return t;var i,s,a=r.call(t),n=[];try{for(;(void 0===e||e-- >0)&&!(i=a.next()).done;)n.push(i.value)}catch(t){s={error:t}}finally{try{i&&!i.done&&(r=a.return)&&r.call(a)}finally{if(s)throw s.error}}return n}function __spread(){for(var t=[],e=0;e1||o(t,e)}))})}function o(t,e){try{!function(t){t.value instanceof __await?Promise.resolve(t.value.v).then(h,l):p(a[0][2],t)}(s[t](e))}catch(t){p(a[0][3],t)}}function h(t){o("next",t)}function l(t){o("throw",t)}function p(t,e){t(e),a.shift(),a.length&&o(a[0][0],a[0][1])}}function __asyncDelegator(t){var e,r;return e={},i("next"),i("throw",(function(t){throw t})),i("return"),e[Symbol.iterator]=function(){return this},e;function i(i,s){e[i]=t[i]?function(e){return(r=!r)?{value:__await(t[i](e)),done:"return"===i}:s?s(e):e}:s}}function __asyncValues(t){if(!Symbol.asyncIterator)throw new TypeError("Symbol.asyncIterator is not defined.");var e,r=t[Symbol.asyncIterator];return r?r.call(t):(t="function"==typeof __values?__values(t):t[Symbol.iterator](),e={},i("next"),i("throw"),i("return"),e[Symbol.asyncIterator]=function(){return this},e);function i(r){e[r]=t[r]&&function(e){return new Promise((function(i,s){(function(t,e,r,i){Promise.resolve(i).then((function(e){t({value:e,done:r})}),e)})(i,s,(e=t[r](e)).done,e.value)}))}}}function __makeTemplateObject(t,e){return Object.defineProperty?Object.defineProperty(t,"raw",{value:e}):t.raw=e,t}var __setModuleDefault=Object.create?function(t,e){Object.defineProperty(t,"default",{enumerable:!0,value:e})}:function(t,e){t.default=e};function __importStar(t){if(t&&t.__esModule)return t;var e={};if(null!=t)for(var r in t)"default"!==r&&Object.prototype.hasOwnProperty.call(t,r)&&__createBinding(e,t,r);return __setModuleDefault(e,t),e}function __importDefault(t){return t&&t.__esModule?t:{default:t}}function __classPrivateFieldGet(t,e,r,i){if("a"===r&&!i)throw new TypeError("Private accessor was defined without a getter");if("function"==typeof e?t!==e||!i:!e.has(t))throw new TypeError("Cannot read private member from an object whose class did not declare it");return"m"===r?i:"a"===r?i.call(t):i?i.value:e.get(t)}function __classPrivateFieldSet(t,e,r,i,s){if("m"===i)throw new TypeError("Private method is not writable");if("a"===i&&!s)throw new TypeError("Private accessor was defined without a setter");if("function"==typeof e?t!==e||!s:!e.has(t))throw new TypeError("Cannot write private member to an object whose class did not declare it");return"a"===i?s.call(t,r):s?s.value=r:e.set(t,r),r +/** + * @license + * Copyright 2019 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */}var t$3=window.ShadowRoot&&(void 0===window.ShadyCSS||window.ShadyCSS.nativeShadow)&&"adoptedStyleSheets"in Document.prototype&&"replace"in CSSStyleSheet.prototype,e$8=Symbol(),n$5=new Map;class s$3{constructor(t,e){if(this._$cssResult$=!0,e!==e$8)throw Error("CSSResult is not constructable. Use `unsafeCSS` or `css` instead.");this.cssText=t}get styleSheet(){var t=n$5.get(this.cssText);return t$3&&void 0===t&&(n$5.set(this.cssText,t=new CSSStyleSheet),t.replaceSync(this.cssText)),t}toString(){return this.cssText}}var o$5=t=>new s$3("string"==typeof t?t:t+"",e$8),r$3=function(t){for(var e=arguments.length,r=new Array(e>1?e-1:0),i=1;ie+(t=>{if(!0===t._$cssResult$)return t.cssText;if("number"==typeof t)return t;throw Error("Value passed to 'css' function must be a 'css' function result: "+t+". Use 'unsafeCSS' to pass non-literal values, but take care to ensure page security.")})(r)+t[i+1]),t[0]);return new s$3(s,e$8)},i$3=(t,e)=>{t$3?t.adoptedStyleSheets=e.map((t=>t instanceof CSSStyleSheet?t:t.styleSheet)):e.forEach((e=>{var r=document.createElement("style"),i=window.litNonce;void 0!==i&&r.setAttribute("nonce",i),r.textContent=e.cssText,t.appendChild(r)}))},S$1=t$3?t=>t:t=>t instanceof CSSStyleSheet?(t=>{var e="";for(var r of t.cssRules)e+=r.cssText;return o$5(e)})(t):t +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */,s$2,e$7=window.trustedTypes,r$2=e$7?e$7.emptyScript:"",h$2=window.reactiveElementPolyfillSupport,o$4={toAttribute(t,e){switch(e){case Boolean:t=t?r$2:null;break;case Object:case Array:t=null==t?t:JSON.stringify(t)}return t},fromAttribute(t,e){var r=t;switch(e){case Boolean:r=null!==t;break;case Number:r=null===t?null:Number(t);break;case Object:case Array:try{r=JSON.parse(t)}catch(t){r=null}}return r}},n$4=(t,e)=>e!==t&&(e==e||t==t),l$3={attribute:!0,type:String,converter:o$4,reflect:!1,hasChanged:n$4},t$2;class a$1 extends HTMLElement{constructor(){super(),this._$Et=new Map,this.isUpdatePending=!1,this.hasUpdated=!1,this._$Ei=null,this.o()}static addInitializer(t){var e;null!==(e=this.l)&&void 0!==e||(this.l=[]),this.l.push(t)}static get observedAttributes(){this.finalize();var t=[];return this.elementProperties.forEach(((e,r)=>{var i=this._$Eh(r,e);void 0!==i&&(this._$Eu.set(i,r),t.push(i))})),t}static createProperty(t){var e=arguments.length>1&&void 0!==arguments[1]?arguments[1]:l$3;if(e.state&&(e.attribute=!1),this.finalize(),this.elementProperties.set(t,e),!e.noAccessor&&!this.prototype.hasOwnProperty(t)){var r="symbol"==typeof t?Symbol():"__"+t,i=this.getPropertyDescriptor(t,r,e);void 0!==i&&Object.defineProperty(this.prototype,t,i)}}static getPropertyDescriptor(t,e,r){return{get(){return this[e]},set(i){var s=this[t];this[e]=i,this.requestUpdate(t,s,r)},configurable:!0,enumerable:!0}}static getPropertyOptions(t){return this.elementProperties.get(t)||l$3}static finalize(){if(this.hasOwnProperty("finalized"))return!1;this.finalized=!0;var t=Object.getPrototypeOf(this);if(t.finalize(),this.elementProperties=new Map(t.elementProperties),this._$Eu=new Map,this.hasOwnProperty("properties")){var e=this.properties,r=[...Object.getOwnPropertyNames(e),...Object.getOwnPropertySymbols(e)];for(var i of r)this.createProperty(i,e[i])}return this.elementStyles=this.finalizeStyles(this.styles),!0}static finalizeStyles(t){var e=[];if(Array.isArray(t)){var r=new Set(t.flat(1/0).reverse());for(var i of r)e.unshift(S$1(i))}else void 0!==t&&e.push(S$1(t));return e}static _$Eh(t,e){var r=e.attribute;return!1===r?void 0:"string"==typeof r?r:"string"==typeof t?t.toLowerCase():void 0}o(){var t;this._$Ep=new Promise((t=>this.enableUpdating=t)),this._$AL=new Map,this._$Em(),this.requestUpdate(),null===(t=this.constructor.l)||void 0===t||t.forEach((t=>t(this)))}addController(t){var e,r;(null!==(e=this._$Eg)&&void 0!==e?e:this._$Eg=[]).push(t),void 0!==this.renderRoot&&this.isConnected&&(null===(r=t.hostConnected)||void 0===r||r.call(t))}removeController(t){var e;null===(e=this._$Eg)||void 0===e||e.splice(this._$Eg.indexOf(t)>>>0,1)}_$Em(){this.constructor.elementProperties.forEach(((t,e)=>{this.hasOwnProperty(e)&&(this._$Et.set(e,this[e]),delete this[e])}))}createRenderRoot(){var t,e=null!==(t=this.shadowRoot)&&void 0!==t?t:this.attachShadow(this.constructor.shadowRootOptions);return i$3(e,this.constructor.elementStyles),e}connectedCallback(){var t;void 0===this.renderRoot&&(this.renderRoot=this.createRenderRoot()),this.enableUpdating(!0),null===(t=this._$Eg)||void 0===t||t.forEach((t=>{var e;return null===(e=t.hostConnected)||void 0===e?void 0:e.call(t)}))}enableUpdating(t){}disconnectedCallback(){var t;null===(t=this._$Eg)||void 0===t||t.forEach((t=>{var e;return null===(e=t.hostDisconnected)||void 0===e?void 0:e.call(t)}))}attributeChangedCallback(t,e,r){this._$AK(t,r)}_$ES(t,e){var r,i,s=arguments.length>2&&void 0!==arguments[2]?arguments[2]:l$3,a=this.constructor._$Eh(t,s);if(void 0!==a&&!0===s.reflect){var n=(null!==(i=null===(r=s.converter)||void 0===r?void 0:r.toAttribute)&&void 0!==i?i:o$4.toAttribute)(e,s.type);this._$Ei=t,null==n?this.removeAttribute(a):this.setAttribute(a,n),this._$Ei=null}}_$AK(t,e){var r,i,s,a=this.constructor,n=a._$Eu.get(t);if(void 0!==n&&this._$Ei!==n){var o=a.getPropertyOptions(n),h=o.converter,l=null!==(s=null!==(i=null===(r=h)||void 0===r?void 0:r.fromAttribute)&&void 0!==i?i:"function"==typeof h?h:null)&&void 0!==s?s:o$4.fromAttribute;this._$Ei=n,this[n]=l(e,o.type),this._$Ei=null}}requestUpdate(t,e,r){var i=!0;void 0!==t&&(((r=r||this.constructor.getPropertyOptions(t)).hasChanged||n$4)(this[t],e)?(this._$AL.has(t)||this._$AL.set(t,e),!0===r.reflect&&this._$Ei!==t&&(void 0===this._$E_&&(this._$E_=new Map),this._$E_.set(t,r))):i=!1),!this.isUpdatePending&&i&&(this._$Ep=this._$EC())}_$EC(){var t=this;return _asyncToGenerator((function*(){t.isUpdatePending=!0;try{yield t._$Ep}catch(e){Promise.reject(e)}var e=t.scheduleUpdate();return null!=e&&(yield e),!t.isUpdatePending}))()}scheduleUpdate(){return this.performUpdate()}performUpdate(){var t;if(this.isUpdatePending){this.hasUpdated,this._$Et&&(this._$Et.forEach(((t,e)=>this[e]=t)),this._$Et=void 0);var e=!1,r=this._$AL;try{(e=this.shouldUpdate(r))?(this.willUpdate(r),null===(t=this._$Eg)||void 0===t||t.forEach((t=>{var e;return null===(e=t.hostUpdate)||void 0===e?void 0:e.call(t)})),this.update(r)):this._$EU()}catch(t){throw e=!1,this._$EU(),t}e&&this._$AE(r)}}willUpdate(t){}_$AE(t){var e;null===(e=this._$Eg)||void 0===e||e.forEach((t=>{var e;return null===(e=t.hostUpdated)||void 0===e?void 0:e.call(t)})),this.hasUpdated||(this.hasUpdated=!0,this.firstUpdated(t)),this.updated(t)}_$EU(){this._$AL=new Map,this.isUpdatePending=!1}get updateComplete(){return this.getUpdateComplete()}getUpdateComplete(){return this._$Ep}shouldUpdate(t){return!0}update(t){void 0!==this._$E_&&(this._$E_.forEach(((t,e)=>this._$ES(e,this[e],t))),this._$E_=void 0),this._$EU()}updated(t){}firstUpdated(t){}}a$1.finalized=!0,a$1.elementProperties=new Map,a$1.elementStyles=[],a$1.shadowRootOptions={mode:"open"},null==h$2||h$2({ReactiveElement:a$1}),(null!==(s$2=globalThis.reactiveElementVersions)&&void 0!==s$2?s$2:globalThis.reactiveElementVersions=[]).push("1.2.1");var i$2=globalThis.trustedTypes,s$1=i$2?i$2.createPolicy("lit-html",{createHTML:t=>t}):void 0,e$6="lit$".concat((Math.random()+"").slice(9),"$"),o$3="?"+e$6,n$3="<".concat(o$3,">"),l$2=document,h$1=function(){var t=arguments.length>0&&void 0!==arguments[0]?arguments[0]:"";return l$2.createComment(t)},r$1=t=>null===t||"object"!=typeof t&&"function"!=typeof t,d=Array.isArray,u=t=>{var e;return d(t)||"function"==typeof(null===(e=t)||void 0===e?void 0:e[Symbol.iterator])},c=/<(?:(!--|\/[^a-zA-Z])|(\/?[a-zA-Z][^>\s]*)|(\/?$))/g,v=/-->/g,a=/>/g,f=/>|[ \n \r](?:([^\s"'>=/]+)([ \n \r]*=[ \n \r]*(?:[^ \n \r"'`<>=]|("|')|))|$)/g,_=/'/g,m=/"/g,g=/^(?:script|style|textarea)$/i,p=t=>function(e){for(var r=arguments.length,i=new Array(r>1?r-1:0),s=1;s{var i,s,a=null!==(i=null==r?void 0:r.renderBefore)&&void 0!==i?i:e,n=a._$litPart$;if(void 0===n){var o=null!==(s=null==r?void 0:r.renderBefore)&&void 0!==s?s:null;a._$litPart$=n=new N(e.insertBefore(h$1(),o),o,void 0,null!=r?r:{})}return n._$AI(t),n},A=l$2.createTreeWalker(l$2,129,null,!1),C=(t,e)=>{for(var r,i=t.length-1,s=[],n=2===e?"":"",o=c,h=0;h"===u[0]?(o=null!=r?r:c,d=-1):void 0===u[1]?d=-2:(d=o.lastIndex-u[2].length,p=u[1],o=void 0===u[3]?f:'"'===u[3]?m:_):o===m||o===_?o=f:o===v||o===a?o=c:(o=f,r=void 0);var b=o===f&&t[h+1].startsWith("/>")?" ":"";n+=o===c?l+n$3:d>=0?(s.push(p),l.slice(0,d)+"$lit$"+l.slice(d)+e$6+b):l+e$6+(-2===d?(s.push(void 0),h):b)}var P=n+(t[i]||"")+(2===e?"":"");if(!Array.isArray(t)||!t.hasOwnProperty("raw"))throw Error("invalid template strings array");return[void 0!==s$1?s$1.createHTML(P):P,s]};class E{constructor(t,e){var r,{strings:i,_$litType$:s}=t;this.parts=[];var a=0,n=0,o=i.length-1,h=this.parts,[l,p]=C(i,s);if(this.el=E.createElement(l,e),A.currentNode=this.el.content,2===s){var c=this.el.content,f=c.firstChild;f.remove(),c.append(...f.childNodes)}for(;null!==(r=A.nextNode())&&h.length0){r.textContent=i$2?i$2.emptyScript:"";for(var x=0;x2&&void 0!==arguments[2]?arguments[2]:t,o=arguments.length>3?arguments[3]:void 0;if(e===b)return e;var h=void 0!==o?null===(r=n._$Cl)||void 0===r?void 0:r[o]:n._$Cu,l=r$1(e)?void 0:e._$litDirective$;return(null==h?void 0:h.constructor)!==l&&(null===(i=null==h?void 0:h._$AO)||void 0===i||i.call(h,!1),void 0===l?h=void 0:(h=new l(t))._$AT(t,n,o),void 0!==o?(null!==(s=(a=n)._$Cl)&&void 0!==s?s:a._$Cl=[])[o]=h:n._$Cu=h),void 0!==h&&(e=P(t,h._$AS(t,e.values),h,o)),e}class V{constructor(t,e){this.v=[],this._$AN=void 0,this._$AD=t,this._$AM=e}get parentNode(){return this._$AM.parentNode}get _$AU(){return this._$AM._$AU}p(t){var e,{el:{content:r},parts:i}=this._$AD,s=(null!==(e=null==t?void 0:t.creationScope)&&void 0!==e?e:l$2).importNode(r,!0);A.currentNode=s;for(var a=A.nextNode(),n=0,o=0,h=i[0];void 0!==h;){if(n===h.index){var l=void 0;2===h.type?l=new N(a,a.nextSibling,this,t):1===h.type?l=new h.ctor(a,h.name,h.strings,this,t):6===h.type&&(l=new L(a,this,t)),this.v.push(l),h=i[++o]}n!==(null==h?void 0:h.index)&&(a=A.nextNode(),n++)}return s}m(t){var e=0;for(var r of this.v)void 0!==r&&(void 0!==r.strings?(r._$AI(t,r,e),e+=r.strings.length-2):r._$AI(t[e])),e++}}class N{constructor(t,e,r,i){var s;this.type=2,this._$AH=w,this._$AN=void 0,this._$AA=t,this._$AB=e,this._$AM=r,this.options=i,this._$Cg=null===(s=null==i?void 0:i.isConnected)||void 0===s||s}get _$AU(){var t,e;return null!==(e=null===(t=this._$AM)||void 0===t?void 0:t._$AU)&&void 0!==e?e:this._$Cg}get parentNode(){var t=this._$AA.parentNode,e=this._$AM;return void 0!==e&&11===t.nodeType&&(t=e.parentNode),t}get startNode(){return this._$AA}get endNode(){return this._$AB}_$AI(t){t=P(this,t,arguments.length>1&&void 0!==arguments[1]?arguments[1]:this),r$1(t)?t===w||null==t||""===t?(this._$AH!==w&&this._$AR(),this._$AH=w):t!==this._$AH&&t!==b&&this.$(t):void 0!==t._$litType$?this.T(t):void 0!==t.nodeType?this.S(t):u(t)?this.A(t):this.$(t)}M(t){var e=arguments.length>1&&void 0!==arguments[1]?arguments[1]:this._$AB;return this._$AA.parentNode.insertBefore(t,e)}S(t){this._$AH!==t&&(this._$AR(),this._$AH=this.M(t))}$(t){this._$AH!==w&&r$1(this._$AH)?this._$AA.nextSibling.data=t:this.S(l$2.createTextNode(t)),this._$AH=t}T(t){var e,{values:r,_$litType$:i}=t,s="number"==typeof i?this._$AC(t):(void 0===i.el&&(i.el=E.createElement(i.h,this.options)),i);if((null===(e=this._$AH)||void 0===e?void 0:e._$AD)===s)this._$AH.m(r);else{var a=new V(s,this),n=a.p(this.options);a.m(r),this.S(n),this._$AH=a}}_$AC(t){var e=T.get(t.strings);return void 0===e&&T.set(t.strings,e=new E(t)),e}A(t){d(this._$AH)||(this._$AH=[],this._$AR());var e,r=this._$AH,i=0;for(var s of t)i===r.length?r.push(e=new N(this.M(h$1()),this.M(h$1()),this,this.options)):e=r[i],e._$AI(s),i++;i0&&void 0!==arguments[0]?arguments[0]:this._$AA.nextSibling,r=arguments.length>1?arguments[1]:void 0;for(null===(t=this._$AP)||void 0===t||t.call(this,!1,!0,r);e&&e!==this._$AB;){var i=e.nextSibling;e.remove(),e=i}}setConnected(t){var e;void 0===this._$AM&&(this._$Cg=t,null===(e=this._$AP)||void 0===e||e.call(this,t))}}class S{constructor(t,e,r,i,s){this.type=1,this._$AH=w,this._$AN=void 0,this.element=t,this.name=e,this._$AM=i,this.options=s,r.length>2||""!==r[0]||""!==r[1]?(this._$AH=Array(r.length-1).fill(new String),this.strings=r):this._$AH=w}get tagName(){return this.element.tagName}get _$AU(){return this._$AM._$AU}_$AI(t){var e=arguments.length>1&&void 0!==arguments[1]?arguments[1]:this,r=arguments.length>2?arguments[2]:void 0,i=arguments.length>3?arguments[3]:void 0,s=this.strings,a=!1;if(void 0===s)t=P(this,t,e,0),(a=!r$1(t)||t!==this._$AH&&t!==b)&&(this._$AH=t);else{var n,o,h=t;for(t=s[0],n=0;n1&&void 0!==arguments[1]?arguments[1]:this,0))&&void 0!==e?e:w)!==b){var r=this._$AH,i=t===w&&r!==w||t.capture!==r.capture||t.once!==r.once||t.passive!==r.passive,s=t!==w&&(r===w||i);i&&this.element.removeEventListener(this.name,this,r),s&&this.element.addEventListener(this.name,this,t),this._$AH=t}}handleEvent(t){var e,r;"function"==typeof this._$AH?this._$AH.call(null!==(r=null===(e=this.options)||void 0===e?void 0:e.host)&&void 0!==r?r:this.element,t):this._$AH.handleEvent(t)}}class L{constructor(t,e,r){this.element=t,this.type=6,this._$AN=void 0,this._$AM=e,this.options=r}get _$AU(){return this._$AM._$AU}_$AI(t){P(this,t)}}var R={P:"$lit$",V:e$6,L:o$3,I:1,N:C,R:V,D:u,j:P,H:N,O:S,F:H,B:I,W:M,Z:L},z=window.litHtmlPolyfillSupport,l$1,o$2;null==z||z(E,N),(null!==(t$2=globalThis.litHtmlVersions)&&void 0!==t$2?t$2:globalThis.litHtmlVersions=[]).push("2.1.2");var r=a$1;class s extends a$1{constructor(){super(...arguments),this.renderOptions={host:this},this._$Dt=void 0}createRenderRoot(){var t,e,r=super.createRenderRoot();return null!==(t=(e=this.renderOptions).renderBefore)&&void 0!==t||(e.renderBefore=r.firstChild),r}update(t){var e=this.render();this.hasUpdated||(this.renderOptions.isConnected=this.isConnected),super.update(t),this._$Dt=x(e,this.renderRoot,this.renderOptions)}connectedCallback(){var t;super.connectedCallback(),null===(t=this._$Dt)||void 0===t||t.setConnected(!0)}disconnectedCallback(){var t;super.disconnectedCallback(),null===(t=this._$Dt)||void 0===t||t.setConnected(!1)}render(){return b}}s.finalized=!0,s._$litElement$=!0,null===(l$1=globalThis.litElementHydrateSupport)||void 0===l$1||l$1.call(globalThis,{LitElement:s});var n$2=globalThis.litElementPolyfillSupport;null==n$2||n$2({LitElement:s});var h={_$AK:(t,e,r)=>{t._$AK(e,r)},_$AL:t=>t._$AL};(null!==(o$2=globalThis.litElementVersions)&&void 0!==o$2?o$2:globalThis.litElementVersions=[]).push("3.1.2"); +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */ +var n$1=t=>e=>"function"==typeof e?((t,e)=>(window.customElements.define(t,e),e))(t,e):((t,e)=>{var{kind:r,elements:i}=e;return{kind:r,elements:i,finisher(e){window.customElements.define(t,e)}}})(t,e) +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */,i$1=(t,e)=>"method"===e.kind&&e.descriptor&&!("value"in e.descriptor)?_objectSpread2(_objectSpread2({},e),{},{finisher(r){r.createProperty(e.key,t)}}):{kind:"field",key:Symbol(),placement:"own",descriptor:{},originalKey:e.key,initializer(){"function"==typeof e.initializer&&(this[e.key]=e.initializer.call(this))},finisher(r){r.createProperty(e.key,t)}};function e$5(t){return(e,r)=>void 0!==r?((t,e,r)=>{e.constructor.createProperty(r,t)})(t,e,r):i$1(t,e)} +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */function t$1(t){return e$5(_objectSpread2(_objectSpread2({},t),{},{state:!0}))} +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */var e$4=(t,e,r)=>{Object.defineProperty(e,r,t)},t=(t,e)=>({kind:"method",placement:"prototype",key:e.key,descriptor:t}),o$1=t=>{var{finisher:e,descriptor:r}=t;return(t,i)=>{var s;if(void 0===i){var a=null!==(s=t.originalKey)&&void 0!==s?s:t.key,n=null!=r?{kind:"method",placement:"prototype",key:a,descriptor:r(t.key)}:_objectSpread2(_objectSpread2({},t),{},{key:a});return null!=e&&(n.finisher=function(t){e(t,a)}),n}var o=t.constructor;void 0!==r&&Object.defineProperty(t,i,r(i)),null==e||e(o,i)}},n; +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */function e$3(t){return o$1({finisher:(e,r)=>{Object.assign(e.prototype[r],t)}})} +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */function i(t,e){return o$1({descriptor:r=>{var i={get(){var e,r;return null!==(r=null===(e=this.renderRoot)||void 0===e?void 0:e.querySelector(t))&&void 0!==r?r:null},enumerable:!0,configurable:!0};if(e){var s="symbol"==typeof r?Symbol():"__"+r;i.get=function(){var e,r;return void 0===this[s]&&(this[s]=null!==(r=null===(e=this.renderRoot)||void 0===e?void 0:e.querySelector(t))&&void 0!==r?r:null),this[s]}}return i}})} +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */function e$2(t){return o$1({descriptor:e=>({get(){var e,r;return null!==(r=null===(e=this.renderRoot)||void 0===e?void 0:e.querySelectorAll(t))&&void 0!==r?r:[]},enumerable:!0,configurable:!0})})} +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */function e$1(t){return o$1({descriptor:e=>({get(){var e=this;return _asyncToGenerator((function*(){var r;return yield e.updateComplete,null===(r=e.renderRoot)||void 0===r?void 0:r.querySelector(t)}))()},enumerable:!0,configurable:!0})})} +/** + * @license + * Copyright 2021 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */var e=null!=(null===(n=window.HTMLSlotElement)||void 0===n?void 0:n.prototype.assignedElements)?(t,e)=>t.assignedElements(e):(t,e)=>t.assignedNodes(e).filter((t=>t.nodeType===Node.ELEMENT_NODE));function l(t){var{slot:r,selector:i}=null!=t?t:{};return o$1({descriptor:s=>({get(){var s,a="slot"+(r?"[name=".concat(r,"]"):":not([name])"),n=null===(s=this.renderRoot)||void 0===s?void 0:s.querySelector(a),o=null!=n?e(n,t):[];return i?o.filter((t=>t.matches(i))):o},enumerable:!0,configurable:!0})})} +/** + * @license + * Copyright 2017 Google LLC + * SPDX-License-Identifier: BSD-3-Clause + */function o(t,e,r){var i,s=t;return"object"==typeof t?(s=t.slot,i=t):i={flatten:e},r?l({slot:s,flatten:e,selector:r}):o$1({descriptor:t=>({get(){var t,e,r="slot"+(s?"[name=".concat(s,"]"):":not([name])"),a=null===(t=this.renderRoot)||void 0===t?void 0:t.querySelector(r);return null!==(e=null==a?void 0:a.assignedNodes(i))&&void 0!==e?e:[]},enumerable:!0,configurable:!0})})}var commonjsGlobal="undefined"!=typeof globalThis?globalThis:"undefined"!=typeof window?window:"undefined"!=typeof global?global:"undefined"!=typeof self?self:{};function getDefaultExportFromCjs(t){return t&&t.__esModule&&Object.prototype.hasOwnProperty.call(t,"default")?t.default:t}function getDefaultExportFromNamespaceIfPresent(t){return t&&Object.prototype.hasOwnProperty.call(t,"default")?t.default:t}function getDefaultExportFromNamespaceIfNotNamed(t){return t&&Object.prototype.hasOwnProperty.call(t,"default")&&1===Object.keys(t).length?t.default:t}function getAugmentedNamespace(t){if(t.__esModule)return t;var e=Object.defineProperty({},"__esModule",{value:!0});return Object.keys(t).forEach((function(r){var i=Object.getOwnPropertyDescriptor(t,r);Object.defineProperty(e,r,i.get?i:{enumerable:!0,get:function(){return t[r]}})})),e}function commonjsRequire(t){throw new Error('Could not dynamically require "'+t+'". Please configure the dynamicRequireTargets or/and ignoreDynamicRequires option of @rollup/plugin-commonjs appropriately for this require call to work.')}var lottie$1={exports:{}};(function(module,exports){var factory;"undefined"!=typeof navigator&&(factory=function(){var svgNS="http://www.w3.org/2000/svg",locationHref="",_useWebWorker=!1,initialDefaultFrame=-999999,setWebWorker=function(t){_useWebWorker=!!t},getWebWorker=function(){return _useWebWorker},setLocationHref=function(t){locationHref=t},getLocationHref=function(){return locationHref};function createTag(t){return document.createElement(t)}function extendPrototype(t,e){var r,i,s=t.length;for(r=0;r1?r[1]=1:r[1]<=0&&(r[1]=0),HSVtoRGB(r[0],r[1],r[2])}function addBrightnessToRGB(t,e){var r=RGBtoHSV(255*t[0],255*t[1],255*t[2]);return r[2]+=e,r[2]>1?r[2]=1:r[2]<0&&(r[2]=0),HSVtoRGB(r[0],r[1],r[2])}function addHueToRGB(t,e){var r=RGBtoHSV(255*t[0],255*t[1],255*t[2]);return r[0]+=e/360,r[0]>1?r[0]-=1:r[0]<0&&(r[0]+=1),HSVtoRGB(r[0],r[1],r[2])}var rgbToHex=function(){var t,e,r=[];for(t=0;t<256;t+=1)e=t.toString(16),r[t]=1===e.length?"0"+e:e;return function(t,e,i){return t<0&&(t=0),e<0&&(e=0),i<0&&(i=0),"#"+r[t]+r[e]+r[i]}}(),setSubframeEnabled=function(t){subframeEnabled=!!t},getSubframeEnabled=function(){return subframeEnabled},setExpressionsPlugin=function(t){expressionsPlugin=t},getExpressionsPlugin=function(){return expressionsPlugin},setExpressionInterfaces=function(t){expressionsInterfaces=t},getExpressionInterfaces=function(){return expressionsInterfaces},setDefaultCurveSegments=function(t){defaultCurveSegments=t},getDefaultCurveSegments=function(){return defaultCurveSegments},setIdPrefix=function(t){idPrefix$1=t},getIdPrefix=function(){return idPrefix$1};function createNS(t){return document.createElementNS(svgNS,t)}function _typeof$5(t){return _typeof$5="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol&&t!==Symbol.prototype?"symbol":typeof t},_typeof$5(t)}var dataManager=function(){var t,e,r=1,i=[],s={onmessage:function(){},postMessage:function(e){t({data:e})}},_workerSelf={postMessage:function(t){s.onmessage({data:t})}};function a(){e||(e=function(e){if(window.Worker&&window.Blob&&getWebWorker()){var r=new Blob(["var _workerSelf = self; self.onmessage = ",e.toString()],{type:"text/javascript"}),i=URL.createObjectURL(r);return new Worker(i)}return t=e,s}((function(t){if(_workerSelf.dataManager||(_workerSelf.dataManager=function(){function t(s,a){var n,o,h,l,p,f,u=s.length;for(o=0;o=0;e-=1)if("sh"===t[e].ty)if(t[e].ks.k.i)i(t[e].ks.k);else for(a=t[e].ks.k.length,s=0;sr[0]||!(r[0]>t[0])&&(t[1]>r[1]||!(r[1]>t[1])&&(t[2]>r[2]||!(r[2]>t[2])&&null))}var a,n=function(){var t=[4,4,14];function e(t){var e,r,i,s=t.length;for(e=0;e=0;r-=1)if("sh"===t[r].ty)if(t[r].ks.k.i)t[r].ks.k.c=t[r].closed;else for(s=t[r].ks.k.length,i=0;i500)&&(this._imageLoaded(),clearInterval(r)),e+=1}.bind(this),50)}function a(t){var e={assetData:t},r=i(t,this.assetsPath,this.path);return dataManager.loadData(r,function(t){e.img=t,this._footageLoaded()}.bind(this),function(){e.img={},this._footageLoaded()}.bind(this)),e}function n(){this._imageLoaded=e.bind(this),this._footageLoaded=r.bind(this),this.testImageLoaded=s.bind(this),this.createFootageData=a.bind(this),this.assetsPath="",this.path="",this.totalImages=0,this.totalFootages=0,this.loadedAssets=0,this.loadedFootagesCount=0,this.imagesLoadedCb=null,this.images=[]}return n.prototype={loadAssets:function(t,e){var r;this.imagesLoadedCb=e;var i=t.length;for(r=0;rthis.animationData.op&&(this.animationData.op=t.op,this.totalFrames=Math.floor(t.op-this.animationData.ip));var e,r,i=this.animationData.layers,s=i.length,a=t.layers,n=a.length;for(r=0;rthis.timeCompleted&&(this.currentFrame=this.timeCompleted),this.trigger("enterFrame"),this.renderFrame(),this.trigger("drawnFrame")},AnimationItem.prototype.renderFrame=function(){if(!1!==this.isLoaded&&this.renderer)try{this.expressionsPlugin&&this.expressionsPlugin.resetFrame(),this.renderer.renderFrame(this.currentFrame+this.firstFrame)}catch(t){this.triggerRenderFrameError(t)}},AnimationItem.prototype.play=function(t){t&&this.name!==t||!0===this.isPaused&&(this.isPaused=!1,this.trigger("_play"),this.audioController.resume(),this._idle&&(this._idle=!1,this.trigger("_active")))},AnimationItem.prototype.pause=function(t){t&&this.name!==t||!1===this.isPaused&&(this.isPaused=!0,this.trigger("_pause"),this._idle=!0,this.trigger("_idle"),this.audioController.pause())},AnimationItem.prototype.togglePause=function(t){t&&this.name!==t||(!0===this.isPaused?this.play():this.pause())},AnimationItem.prototype.stop=function(t){t&&this.name!==t||(this.pause(),this.playCount=0,this._completedLoop=!1,this.setCurrentRawFrameValue(0))},AnimationItem.prototype.getMarkerData=function(t){for(var e,r=0;r=this.totalFrames-1&&this.frameModifier>0?this.loop&&this.playCount!==this.loop?e>=this.totalFrames?(this.playCount+=1,this.checkSegments(e%this.totalFrames)||(this.setCurrentRawFrameValue(e%this.totalFrames),this._completedLoop=!0,this.trigger("loopComplete"))):this.setCurrentRawFrameValue(e):this.checkSegments(e>this.totalFrames?e%this.totalFrames:0)||(r=!0,e=this.totalFrames-1):e<0?this.checkSegments(e%this.totalFrames)||(!this.loop||this.playCount--<=0&&!0!==this.loop?(r=!0,e=0):(this.setCurrentRawFrameValue(this.totalFrames+e%this.totalFrames),this._completedLoop?this.trigger("loopComplete"):this._completedLoop=!0)):this.setCurrentRawFrameValue(e),r&&(this.setCurrentRawFrameValue(e),this.pause(),this.trigger("complete"))}},AnimationItem.prototype.adjustSegment=function(t,e){this.playCount=0,t[1]0&&(this.playSpeed<0?this.setSpeed(-this.playSpeed):this.setDirection(-1)),this.totalFrames=t[0]-t[1],this.timeCompleted=this.totalFrames,this.firstFrame=t[1],this.setCurrentRawFrameValue(this.totalFrames-.001-e)):t[1]>t[0]&&(this.frameModifier<0&&(this.playSpeed<0?this.setSpeed(-this.playSpeed):this.setDirection(1)),this.totalFrames=t[1]-t[0],this.timeCompleted=this.totalFrames,this.firstFrame=t[0],this.setCurrentRawFrameValue(.001+e)),this.trigger("segmentStart")},AnimationItem.prototype.setSegment=function(t,e){var r=-1;this.isPaused&&(this.currentRawFrame+this.firstFramee&&(r=e-t)),this.firstFrame=t,this.totalFrames=e-t,this.timeCompleted=this.totalFrames,-1!==r&&this.goToAndStop(r,!0)},AnimationItem.prototype.playSegments=function(t,e){if(e&&(this.segments.length=0),"object"===_typeof$4(t[0])){var r,i=t.length;for(r=0;r=0;r-=1)e[r].animation.destroy(t)},t.freeze=function(){n=!0},t.unfreeze=function(){n=!1,d()},t.setVolume=function(t,r){var s;for(s=0;s=.001?function(t,e,r,i){for(var s=0;s<4;++s){var a=h(e,r,i);if(0===a)return e;e-=(o(e,r,i)-t)/a}return e}(t,l,e,i):0===p?l:function(t,e,r,i,s){var a,n,h=0;do{(a=o(n=e+(r-e)/2,i,s)-t)>0?r=n:e=n}while(Math.abs(a)>1e-7&&++h<10);return n}(t,a,a+r,e,i)}},t}(),pooling={double:function(t){return t.concat(createSizedArray(t.length))}},poolFactory=function(t,e,r){var i=0,s=t,a=createSizedArray(s);return{newElement:function(){return i?a[i-=1]:e()},release:function(t){i===s&&(a=pooling.double(a),s*=2),r&&r(t),a[i]=t,i+=1}}},bezierLengthPool=poolFactory(8,(function(){return{addedLength:0,percents:createTypedArray("float32",getDefaultCurveSegments()),lengths:createTypedArray("float32",getDefaultCurveSegments())}})),segmentsLengthPool=poolFactory(8,(function(){return{lengths:[],totalLength:0}}),(function(t){var e,r=t.lengths.length;for(e=0;e-.001&&n<.001}var r=function(t,e,r,i){var s,a,n,o,h,l,p=getDefaultCurveSegments(),c=0,f=[],u=[],d=bezierLengthPool.newElement();for(n=r.length,s=0;sn?-1:1,l=!0;l;)if(i[a]<=n&&i[a+1]>n?(o=(n-i[a])/(i[a+1]-i[a]),l=!1):a+=h,a<0||a>=s-1){if(a===s-1)return r[a];l=!1}return r[a]+(r[a+1]-r[a])*o}var h=createTypedArray("float32",8);return{getSegmentsLength:function(t){var e,i=segmentsLengthPool.newElement(),s=t.c,a=t.v,n=t.o,o=t.i,h=t._length,l=i.lengths,p=0;for(e=0;e1&&(a=1);var p,c=o(a,l),f=o(n=n>1?1:n,l),u=e.length,d=1-c,m=1-f,y=d*d*d,g=c*d*d*3,v=c*c*d*3,b=c*c*c,_=d*d*m,P=c*d*m+d*c*m+d*d*f,S=c*c*m+d*c*f+c*d*f,E=c*c*f,x=d*m*m,C=c*m*m+d*f*m+d*m*f,A=c*f*m+d*f*f+c*m*f,w=c*f*f,k=m*m*m,T=f*m*m+m*f*m+m*m*f,M=f*f*m+m*f*f+f*m*f,D=f*f*f;for(p=0;pu?f>d?f-u-d:d-u-f:d>u?d-u-f:u-f-d)>-1e-4&&c<1e-4}}}var bez=bezFunction(),initFrame=initialDefaultFrame,mathAbs=Math.abs;function interpolateValue(t,e){var r,i=this.offsetTime;"multidimensional"===this.propType&&(r=createTypedArray("float32",this.pv.length));for(var s,a,n,o,h,l,p,c,f,u=e.lastIndex,d=u,m=this.keyframes.length-1,y=!0;y;){if(s=this.keyframes[d],a=this.keyframes[d+1],d===m-1&&t>=a.t-i){s.h&&(s=a),u=0;break}if(a.t-i>t){u=d;break}d=v||t=v?_.points.length-1:0;for(h=_.points[P].point.length,o=0;o=x&&E=v?(r[0]=g[0],r[1]=g[1],r[2]=g[2]):t<=b?(r[0]=s.s[0],r[1]=s.s[1],r[2]=s.s[2]):quaternionToEuler(r,slerp(createQuaternion(s.s),createQuaternion(g),(t-b)/(v-b)));else for(d=0;d=v?l=1:t1e-6?(i=Math.acos(s),a=Math.sin(i),n=Math.sin((1-r)*i)/a,o=Math.sin(r*i)/a):(n=1-r,o=r),h[0]=n*l+o*u,h[1]=n*p+o*d,h[2]=n*c+o*m,h[3]=n*f+o*y,h}function quaternionToEuler(t,e){var r=e[0],i=e[1],s=e[2],a=e[3],n=Math.atan2(2*i*a-2*r*s,1-2*i*i-2*s*s),o=Math.asin(2*r*i+2*s*a),h=Math.atan2(2*r*a-2*i*s,1-2*r*r-2*s*s);t[0]=n/degToRads,t[1]=o/degToRads,t[2]=h/degToRads}function createQuaternion(t){var e=t[0]*degToRads,r=t[1]*degToRads,i=t[2]*degToRads,s=Math.cos(e/2),a=Math.cos(r/2),n=Math.cos(i/2),o=Math.sin(e/2),h=Math.sin(r/2),l=Math.sin(i/2);return[o*h*n+s*a*l,o*a*n+s*h*l,s*h*n-o*a*l,s*a*n-o*h*l]}function getValueAtCurrentTime(){var t=this.comp.renderedFrame-this.offsetTime,e=this.keyframes[0].t-this.offsetTime,r=this.keyframes[this.keyframes.length-1].t-this.offsetTime;if(!(t===this._caching.lastFrame||this._caching.lastFrame!==initFrame&&(this._caching.lastFrame>=r&&t>=r||this._caching.lastFrame=t&&(this._caching._lastKeyframeIndex=-1,this._caching.lastIndex=0);var i=this.interpolateValue(t,this._caching);this.pv=i}return this._caching.lastFrame=t,this.pv}function setVValue(t){var e;if("unidimensional"===this.propType)e=t*this.mult,mathAbs(this.v-e)>1e-5&&(this.v=e,this._mdf=!0);else for(var r=0,i=this.v.length;r1e-5&&(this.v[r]=e,this._mdf=!0),r+=1}function processEffectsSequence(){if(this.elem.globalData.frameId!==this.frameId&&this.effectsSequence.length)if(this.lock)this.setVValue(this.pv);else{var t;this.lock=!0,this._mdf=this._isFirstFrame;var e=this.effectsSequence.length,r=this.kf?this.pv:this.data.k;for(t=0;t=this._maxLength&&this.doubleArrayLength(),r){case"v":a=this.v;break;case"i":a=this.i;break;case"o":a=this.o;break;default:a=[]}(!a[i]||a[i]&&!s)&&(a[i]=pointPool.newElement()),a[i][0]=t,a[i][1]=e},ShapePath.prototype.setTripleAt=function(t,e,r,i,s,a,n,o){this.setXYAt(t,e,"v",n,o),this.setXYAt(r,i,"o",n,o),this.setXYAt(s,a,"i",n,o)},ShapePath.prototype.reverse=function(){var t=new ShapePath;t.setPathData(this.c,this._length);var e=this.v,r=this.o,i=this.i,s=0;this.c&&(t.setTripleAt(e[0][0],e[0][1],i[0][0],i[0][1],r[0][0],r[0][1],0,!1),s=1);var a,n=this._length-1,o=this._length;for(a=s;a=u[u.length-1].t-this.offsetTime)i=u[u.length-1].s?u[u.length-1].s[0]:u[u.length-2].e[0],a=!0;else{for(var d,m,y,g=f,v=u.length-1,b=!0;b&&(d=u[g],!((m=u[g+1]).t-this.offsetTime>t));)g=m.t-this.offsetTime)p=1;else if(ti&&e>i)||(this._caching.lastIndex=s0||t>-1e-6&&t<0?i(1e4*t)/1e4:t}function I(){var t=this.props;return"matrix("+F(t[0])+","+F(t[1])+","+F(t[4])+","+F(t[5])+","+F(t[12])+","+F(t[13])+")"}return function(){this.reset=s,this.rotate=a,this.rotateX=n,this.rotateY=o,this.rotateZ=h,this.skew=p,this.skewFromAxis=c,this.shear=l,this.scale=f,this.setTransform=u,this.translate=d,this.transform=m,this.multiply=y,this.applyToPoint=P,this.applyToX=S,this.applyToY=E,this.applyToZ=x,this.applyToPointArray=T,this.applyToTriplePoints=k,this.applyToPointStringified=M,this.toCSS=D,this.to2dCSS=I,this.clone=b,this.cloneFromProps=_,this.equals=v,this.inversePoints=w,this.inversePoint=A,this.getInverseMatrix=C,this._t=this.transform,this.isIdentity=g,this._identity=!0,this._identityCalculated=!1,this.props=createTypedArray("float32",16),this.reset()}}();function _typeof$3(t){return _typeof$3="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol&&t!==Symbol.prototype?"symbol":typeof t},_typeof$3(t)}var lottie={},standalone="__[STANDALONE]__",animationData="__[ANIMATIONDATA]__",renderer="";function setLocation(t){setLocationHref(t)}function searchAnimations(){!0===standalone?animationManager.searchAnimations(animationData,standalone,renderer):animationManager.searchAnimations()}function setSubframeRendering(t){setSubframeEnabled(t)}function setPrefix(t){setIdPrefix(t)}function loadAnimation(t){return!0===standalone&&(t.animationData=JSON.parse(animationData)),animationManager.loadAnimation(t)}function setQuality(t){if("string"==typeof t)switch(t){case"high":setDefaultCurveSegments(200);break;default:case"medium":setDefaultCurveSegments(50);break;case"low":setDefaultCurveSegments(10)}else!isNaN(t)&&t>1&&setDefaultCurveSegments(t);getDefaultCurveSegments()>=50?roundValues(!1):roundValues(!0)}function inBrowser(){return"undefined"!=typeof navigator}function installPlugin(t,e){"expressions"===t&&setExpressionsPlugin(e)}function getFactory(t){switch(t){case"propertyFactory":return PropertyFactory;case"shapePropertyFactory":return ShapePropertyFactory;case"matrix":return Matrix;default:return null}}function checkReady(){"complete"===document.readyState&&(clearInterval(readyStateCheckInterval),searchAnimations())}function getQueryVariable(t){for(var e=queryString.split("&"),r=0;r=1?a.push({s:t-1,e:e-1}):(a.push({s:t,e:1}),a.push({s:0,e:e-1}));var n,o,h=[],l=a.length;for(n=0;ni+r||(p=o.s*s<=i?0:(o.s*s-i)/r,c=o.e*s>=i+r?1:(o.e*s-i)/r,h.push([p,c]))}return h.length||h.push([0,0]),h},TrimModifier.prototype.releasePathsData=function(t){var e,r=t.length;for(e=0;e1?1+a:this.s.v<0?0+a:this.s.v+a)>(r=this.e.v>1?1+a:this.e.v<0?0+a:this.e.v+a)){var n=e;e=r,r=n}e=1e-4*Math.round(1e4*e),r=1e-4*Math.round(1e4*r),this.sValue=e,this.eValue=r}else e=this.sValue,r=this.eValue;var o,h,l,p,c,f=this.shapes.length,u=0;if(r===e)for(s=0;s=0;s-=1)if((d=this.shapes[s]).shape._mdf){for((m=d.localShapeCollection).releaseShapes(),2===this.m&&f>1?(g=this.calculateShapeEdges(e,r,d.totalShapeLength,_,u),_+=d.totalShapeLength):g=[[v,b]],h=g.length,o=0;o=1?y.push({s:d.totalShapeLength*(v-1),e:d.totalShapeLength*(b-1)}):(y.push({s:d.totalShapeLength*v,e:d.totalShapeLength}),y.push({s:0,e:d.totalShapeLength*(b-1)}));var P=this.addShapes(d,y[0]);if(y[0].s!==y[0].e){if(y.length>1)if(d.shape.paths.shapes[d.shape.paths._length-1].c){var S=P.pop();this.addPaths(P,m),P=this.addShapes(d,y[1],S)}else this.addPaths(P,m),P=this.addShapes(d,y[1]);this.addPaths(P,m)}}d.shape.paths=m}}},TrimModifier.prototype.addPaths=function(t,e){var r,i=t.length;for(r=0;re.e){r.c=!1;break}e.s<=d&&e.e>=d+n.addedLength?(this.addSegment(f[i].v[s-1],f[i].o[s-1],f[i].i[s],f[i].v[s],r,o,y),y=!1):(l=bez.getNewSegment(f[i].v[s-1],f[i].v[s],f[i].o[s-1],f[i].i[s],(e.s-d)/n.addedLength,(e.e-d)/n.addedLength,h[s-1]),this.addSegmentFromArray(l,r,o,y),y=!1,r.c=!1),d+=n.addedLength,o+=1}if(f[i].c&&h.length){if(n=h[s-1],d<=e.e){var g=h[s-1].addedLength;e.s<=d&&e.e>=d+g?(this.addSegment(f[i].v[s-1],f[i].o[s-1],f[i].i[0],f[i].v[0],r,o,y),y=!1):(l=bez.getNewSegment(f[i].v[s-1],f[i].v[0],f[i].o[s-1],f[i].i[0],(e.s-d)/g,(e.e-d)/g,h[s-1]),this.addSegmentFromArray(l,r,o,y),y=!1,r.c=!1)}else r.c=!1;d+=n.addedLength,o+=1}if(r._length&&(r.setXYAt(r.v[p][0],r.v[p][1],"i",p),r.setXYAt(r.v[r._length-1][0],r.v[r._length-1][1],"o",r._length-1)),d>e.e)break;i=this.p.keyframes[this.p.keyframes.length-1].t?(i=this.p.getValueAtTime(this.p.keyframes[this.p.keyframes.length-1].t/r,0),s=this.p.getValueAtTime((this.p.keyframes[this.p.keyframes.length-1].t-.05)/r,0)):(i=this.p.pv,s=this.p.getValueAtTime((this.p._caching.lastFrame+this.p.offsetTime-.01)/r,this.p.offsetTime));else if(this.px&&this.px.keyframes&&this.py.keyframes&&this.px.getValueAtTime&&this.py.getValueAtTime){i=[],s=[];var a=this.px,n=this.py;a._caching.lastFrame+a.offsetTime<=a.keyframes[0].t?(i[0]=a.getValueAtTime((a.keyframes[0].t+.01)/r,0),i[1]=n.getValueAtTime((n.keyframes[0].t+.01)/r,0),s[0]=a.getValueAtTime(a.keyframes[0].t/r,0),s[1]=n.getValueAtTime(n.keyframes[0].t/r,0)):a._caching.lastFrame+a.offsetTime>=a.keyframes[a.keyframes.length-1].t?(i[0]=a.getValueAtTime(a.keyframes[a.keyframes.length-1].t/r,0),i[1]=n.getValueAtTime(n.keyframes[n.keyframes.length-1].t/r,0),s[0]=a.getValueAtTime((a.keyframes[a.keyframes.length-1].t-.01)/r,0),s[1]=n.getValueAtTime((n.keyframes[n.keyframes.length-1].t-.01)/r,0)):(i=[a.pv,n.pv],s[0]=a.getValueAtTime((a._caching.lastFrame+a.offsetTime-.01)/r,a.offsetTime),s[1]=n.getValueAtTime((n._caching.lastFrame+n.offsetTime-.01)/r,n.offsetTime))}else i=s=t;this.v.rotate(-Math.atan2(i[1]-s[1],i[0]-s[0]))}this.data.p&&this.data.p.s?this.data.p.z?this.v.translate(this.px.v,this.py.v,-this.pz.v):this.v.translate(this.px.v,this.py.v,0):this.v.translate(this.p.v[0],this.p.v[1],-this.p.v[2])}this.frameId=this.elem.globalData.frameId}},precalculateMatrix:function(){if(this.appliedTransformations=0,this.pre.reset(),!this.a.effectsSequence.length&&(this.pre.translate(-this.a.v[0],-this.a.v[1],this.a.v[2]),this.appliedTransformations=1,!this.s.effectsSequence.length)){if(this.pre.scale(this.s.v[0],this.s.v[1],this.s.v[2]),this.appliedTransformations=2,this.sk){if(this.sk.effectsSequence.length||this.sa.effectsSequence.length)return;this.pre.skewFromAxis(-this.sk.v,this.sa.v),this.appliedTransformations=3}this.r?this.r.effectsSequence.length||(this.pre.rotate(-this.r.v),this.appliedTransformations=4):this.rz.effectsSequence.length||this.ry.effectsSequence.length||this.rx.effectsSequence.length||this.or.effectsSequence.length||(this.pre.rotateZ(-this.rz.v).rotateY(this.ry.v).rotateX(this.rx.v).rotateZ(-this.or.v[2]).rotateY(this.or.v[1]).rotateX(this.or.v[0]),this.appliedTransformations=4)}},autoOrient:function(){}},extendPrototype([DynamicPropertyContainer],e),e.prototype.addDynamicProperty=function(t){this._addDynamicProperty(t),this.elem.addDynamicProperty(t),this._isDirty=!0},e.prototype._addDynamicProperty=DynamicPropertyContainer.prototype.addDynamicProperty,{getTransformProperty:function(t,r,i){return new e(t,r,i)}}}();function RepeaterModifier(){}function RoundCornersModifier(){}function floatEqual(t,e){return 1e5*Math.abs(t-e)<=Math.min(Math.abs(t),Math.abs(e))}function floatZero(t){return Math.abs(t)<=1e-5}function lerp(t,e,r){return t*(1-r)+e*r}function lerpPoint(t,e,r){return[lerp(t[0],e[0],r),lerp(t[1],e[1],r)]}function quadRoots(t,e,r){if(0===t)return[];var i=e*e-4*t*r;if(i<0)return[];var s=-e/(2*t);if(0===i)return[s];var a=Math.sqrt(i)/(2*t);return[s-a,s+a]}function polynomialCoefficients(t,e,r,i){return[3*e-t-3*r+i,3*t-6*e+3*r,-3*t+3*e,t]}function singlePoint(t){return new PolynomialBezier(t,t,t,t,!1)}function PolynomialBezier(t,e,r,i,s){s&&pointEqual(t,e)&&(e=lerpPoint(t,i,1/3)),s&&pointEqual(r,i)&&(r=lerpPoint(t,i,2/3));var a=polynomialCoefficients(t[0],e[0],r[0],i[0]),n=polynomialCoefficients(t[1],e[1],r[1],i[1]);this.a=[a[0],n[0]],this.b=[a[1],n[1]],this.c=[a[2],n[2]],this.d=[a[3],n[3]],this.points=[t,e,r,i]}function extrema(t,e){var r=t.points[0][e],i=t.points[t.points.length-1][e];if(r>i){var s=i;i=r,r=s}for(var a=quadRoots(3*t.a[e],2*t.b[e],t.c[e]),n=0;n0&&a[n]<1){var o=t.point(a[n])[e];oi&&(i=o)}return{min:r,max:i}}function intersectData(t,e,r){var i=t.boundingBox();return{cx:i.cx,cy:i.cy,width:i.width,height:i.height,bez:t,t:(e+r)/2,t1:e,t2:r}}function splitData(t){var e=t.bez.split(.5);return[intersectData(e[0],t.t1,t.t),intersectData(e[1],t.t,t.t2)]}function boxIntersect(t,e){return 2*Math.abs(t.cx-e.cx)=a||t.width<=i&&t.height<=i&&e.width<=i&&e.height<=i)s.push([t.t,e.t]);else{var n=splitData(t),o=splitData(e);intersectsImpl(n[0],o[0],r+1,i,s,a),intersectsImpl(n[0],o[1],r+1,i,s,a),intersectsImpl(n[1],o[0],r+1,i,s,a),intersectsImpl(n[1],o[1],r+1,i,s,a)}}function crossProduct(t,e){return[t[1]*e[2]-t[2]*e[1],t[2]*e[0]-t[0]*e[2],t[0]*e[1]-t[1]*e[0]]}function lineIntersection(t,e,r,i){var s=[t[0],t[1],1],a=[e[0],e[1],1],n=[r[0],r[1],1],o=[i[0],i[1],1],h=crossProduct(crossProduct(s,a),crossProduct(n,o));return floatZero(h[2])?null:[h[0]/h[2],h[1]/h[2]]}function polarOffset(t,e,r){return[t[0]+Math.cos(e)*r,t[1]-Math.sin(e)*r]}function pointDistance(t,e){return Math.hypot(t[0]-e[0],t[1]-e[1])}function pointEqual(t,e){return floatEqual(t[0],e[0])&&floatEqual(t[1],e[1])}function ZigZagModifier(){}function setPoint(t,e,r,i,s,a,n){var o=r-Math.PI/2,h=r+Math.PI/2,l=e[0]+Math.cos(r)*i*s,p=e[1]-Math.sin(r)*i*s;t.setTripleAt(l,p,l+Math.cos(o)*a,p-Math.sin(o)*a,l+Math.cos(h)*n,p-Math.sin(h)*n,t.length())}function getPerpendicularVector(t,e){var r=[e[0]-t[0],e[1]-t[1]],i=.5*-Math.PI;return[Math.cos(i)*r[0]-Math.sin(i)*r[1],Math.sin(i)*r[0]+Math.cos(i)*r[1]]}function getProjectingAngle(t,e){var r=0===e?t.length()-1:e-1,i=(e+1)%t.length(),s=getPerpendicularVector(t.v[r],t.v[i]);return Math.atan2(0,1)-Math.atan2(s[1],s[0])}function zigZagCorner(t,e,r,i,s,a,n){var o=getProjectingAngle(e,r),h=e.v[r%e._length],l=e.v[0===r?e._length-1:r-1],p=e.v[(r+1)%e._length],c=2===a?Math.sqrt(Math.pow(h[0]-l[0],2)+Math.pow(h[1]-l[1],2)):0,f=2===a?Math.sqrt(Math.pow(h[0]-p[0],2)+Math.pow(h[1]-p[1],2)):0;setPoint(t,e.v[r%e._length],o,n,i,f/(2*(s+1)),c/(2*(s+1)),a)}function zigZagSegment(t,e,r,i,s,a){for(var n=0;n1&&e.length>1&&(s=getIntersection(t[0],e[e.length-1]))?[[t[0].split(s[0])[0]],[e[e.length-1].split(s[1])[1]]]:[r,i]}function pruneIntersections(t){for(var e,r=1;r1&&(e=pruneSegmentIntersection(t[t.length-1],t[0]),t[t.length-1]=e[0],t[0]=e[1]),t}function offsetSegmentSplit(t,e){var r,i,s,a,n=t.inflectionPoints();if(0===n.length)return[offsetSegment(t,e)];if(1===n.length||floatEqual(n[1],1))return r=(s=t.split(n[0]))[0],i=s[1],[offsetSegment(r,e),offsetSegment(i,e)];r=(s=t.split(n[0]))[0];var o=(n[1]-n[0])/(1-n[0]);return a=(s=s[1].split(o))[0],i=s[1],[offsetSegment(r,e),offsetSegment(a,e),offsetSegment(i,e)]}function OffsetPathModifier(){}function getFontProperties(t){for(var e=t.fStyle?t.fStyle.split(" "):[],r="normal",i="normal",s=e.length,a=0;a0;)r-=1,this._elements.unshift(e[r]);this.dynamicProperties.length?this.k=!0:this.getValue(!0)},RepeaterModifier.prototype.resetElements=function(t){var e,r=t.length;for(e=0;e0?Math.floor(f):Math.ceil(f),m=this.pMatrix.props,y=this.rMatrix.props,g=this.sMatrix.props;this.pMatrix.reset(),this.rMatrix.reset(),this.sMatrix.reset(),this.tMatrix.reset(),this.matrix.reset();var v,b,_=0;if(f>0){for(;_d;)this.applyTransforms(this.pMatrix,this.rMatrix,this.sMatrix,this.tr,1,!0),_-=1;u&&(this.applyTransforms(this.pMatrix,this.rMatrix,this.sMatrix,this.tr,-u,!0),_-=u)}for(i=1===this.data.m?0:this._currentCopies-1,s=1===this.data.m?1:-1,a=this._currentCopies;a;){if(b=(r=(e=this.elemsData[i].it)[e.length-1].transform.mProps.v.props).length,e[e.length-1].transform.mProps._mdf=!0,e[e.length-1].transform.op._mdf=!0,e[e.length-1].transform.op.v=1===this._currentCopies?this.so.v:this.so.v+(this.eo.v-this.so.v)*(i/(this._currentCopies-1)),0!==_){for((0!==i&&1===s||i!==this._currentCopies-1&&-1===s)&&this.applyTransforms(this.pMatrix,this.rMatrix,this.sMatrix,this.tr,1,!1),this.matrix.transform(y[0],y[1],y[2],y[3],y[4],y[5],y[6],y[7],y[8],y[9],y[10],y[11],y[12],y[13],y[14],y[15]),this.matrix.transform(g[0],g[1],g[2],g[3],g[4],g[5],g[6],g[7],g[8],g[9],g[10],g[11],g[12],g[13],g[14],g[15]),this.matrix.transform(m[0],m[1],m[2],m[3],m[4],m[5],m[6],m[7],m[8],m[9],m[10],m[11],m[12],m[13],m[14],m[15]),v=0;v0&&i<1?[e]:[]:[e-i,e+i].filter((function(t){return t>0&&t<1}))},PolynomialBezier.prototype.split=function(t){if(t<=0)return[singlePoint(this.points[0]),this];if(t>=1)return[this,singlePoint(this.points[this.points.length-1])];var e=lerpPoint(this.points[0],this.points[1],t),r=lerpPoint(this.points[1],this.points[2],t),i=lerpPoint(this.points[2],this.points[3],t),s=lerpPoint(e,r,t),a=lerpPoint(r,i,t),n=lerpPoint(s,a,t);return[new PolynomialBezier(this.points[0],e,s,n,!0),new PolynomialBezier(n,a,i,this.points[3],!0)]},PolynomialBezier.prototype.bounds=function(){return{x:extrema(this,0),y:extrema(this,1)}},PolynomialBezier.prototype.boundingBox=function(){var t=this.bounds();return{left:t.x.min,right:t.x.max,top:t.y.min,bottom:t.y.max,width:t.x.max-t.x.min,height:t.y.max-t.y.min,cx:(t.x.max+t.x.min)/2,cy:(t.y.max+t.y.min)/2}},PolynomialBezier.prototype.intersections=function(t,e,r){void 0===e&&(e=2),void 0===r&&(r=7);var i=[];return intersectsImpl(intersectData(this,0,1),intersectData(t,0,1),0,e,i,r),i},PolynomialBezier.shapeSegment=function(t,e){var r=(e+1)%t.length();return new PolynomialBezier(t.v[e],t.o[e],t.i[r],t.v[r],!0)},PolynomialBezier.shapeSegmentInverted=function(t,e){var r=(e+1)%t.length();return new PolynomialBezier(t.v[r],t.i[r],t.o[e],t.v[e],!0)},extendPrototype([ShapeModifier],ZigZagModifier),ZigZagModifier.prototype.initModifierProperties=function(t,e){this.getValue=this.processKeys,this.amplitude=PropertyFactory.getProp(t,e.s,0,null,this),this.frequency=PropertyFactory.getProp(t,e.r,0,null,this),this.pointsType=PropertyFactory.getProp(t,e.pt,0,null,this),this._isAnimated=0!==this.amplitude.effectsSequence.length||0!==this.frequency.effectsSequence.length||0!==this.pointsType.effectsSequence.length},ZigZagModifier.prototype.processPath=function(t,e,r,i){var s=t._length,a=shapePool.newElement();if(a.c=t.c,t.c||(s-=1),0===s)return a;var n=-1,o=PolynomialBezier.shapeSegment(t,0);zigZagCorner(a,t,0,e,r,i,n);for(var h=0;h=0;a-=1)o=PolynomialBezier.shapeSegmentInverted(t,a),l.push(offsetSegmentSplit(o,e));l=pruneIntersections(l);var p=null,c=null;for(a=0;a=55296&&r<=56319){var i=t.charCodeAt(1);i>=56320&&i<=57343&&(e=1024*(r-55296)+i-56320+65536)}return e}function o(t){var e=n(t);return e>=127462&&e<=127487}var h=function(){this.fonts=[],this.chars=null,this.typekitLoaded=0,this.isLoaded=!1,this._warned=!1,this.initTime=Date.now(),this.setIsLoadedBinded=this.setIsLoaded.bind(this),this.checkLoadedFontsBinded=this.checkLoadedFonts.bind(this)};h.isModifier=function(t,e){var r=t.toString(16)+e.toString(16);return-1!==i.indexOf(r)},h.isZeroWidthJoiner=function(t){return 8205===t},h.isFlagEmoji=function(t){return o(t.substr(0,2))&&o(t.substr(2,2))},h.isRegionalCode=o,h.isCombinedCharacter=function(t){return-1!==e.indexOf(t)},h.isRegionalFlag=function(t,e){var i=n(t.substr(e,2));if(i!==r)return!1;var s=0;for(e+=2;s<5;){if((i=n(t.substr(e,2)))<917601||i>917626)return!1;s+=1,e+=2}return 917631===n(t.substr(e,2))},h.isVariationSelector=function(t){return 65039===t},h.BLACK_FLAG_CODE_POINT=r;var l={addChars:function(t){if(t){var e;this.chars||(this.chars=[]);var r,i,s=t.length,a=this.chars.length;for(e=0;e0&&(p=!1),p){var c=createTag("style");c.setAttribute("f-forigin",i[r].fOrigin),c.setAttribute("f-origin",i[r].origin),c.setAttribute("f-family",i[r].fFamily),c.type="text/css",c.innerText="@font-face {font-family: "+i[r].fFamily+"; font-style: normal; src: url('"+i[r].fPath+"');}",e.appendChild(c)}}else if("g"===i[r].fOrigin||1===i[r].origin){for(h=document.querySelectorAll('link[f-forigin="g"], link[f-origin="1"]'),l=0;lt?!0!==this.isInRange&&(this.globalData._mdf=!0,this._mdf=!0,this.isInRange=!0,this.show()):!1!==this.isInRange&&(this.globalData._mdf=!0,this.isInRange=!1,this.hide())},renderRenderable:function(){var t,e=this.renderableComponents.length;for(t=0;t.1)&&this.audio.seek(this._currentTime/this.globalData.frameRate):(this.audio.play(),this.audio.seek(this._currentTime/this.globalData.frameRate),this._isPlaying=!0))},AudioElement.prototype.show=function(){},AudioElement.prototype.hide=function(){this.audio.pause(),this._isPlaying=!1},AudioElement.prototype.pause=function(){this.audio.pause(),this._isPlaying=!1,this._canPlay=!1},AudioElement.prototype.resume=function(){this._canPlay=!0},AudioElement.prototype.setRate=function(t){this.audio.rate(t)},AudioElement.prototype.volume=function(t){this._volumeMultiplier=t,this._previousVolume=t*this._volume,this.audio.volume(this._previousVolume)},AudioElement.prototype.getBaseElement=function(){return null},AudioElement.prototype.destroy=function(){},AudioElement.prototype.sourceRectAtTime=function(){},AudioElement.prototype.initExpressions=function(){},BaseRenderer.prototype.checkLayers=function(t){var e,r,i=this.layers.length;for(this.completeLayers=!0,e=i-1;e>=0;e-=1)this.elements[e]||(r=this.layers[e]).ip-r.st<=t-this.layers[e].st&&r.op-r.st>t-this.layers[e].st&&this.buildItem(e),this.completeLayers=!!this.elements[e]&&this.completeLayers;this.checkPendingElements()},BaseRenderer.prototype.createItem=function(t){switch(t.ty){case 2:return this.createImage(t);case 0:return this.createComp(t);case 1:return this.createSolid(t);case 3:default:return this.createNull(t);case 4:return this.createShape(t);case 5:return this.createText(t);case 6:return this.createAudio(t);case 13:return this.createCamera(t);case 15:return this.createFootage(t)}},BaseRenderer.prototype.createCamera=function(){throw new Error("You're using a 3d camera. Try the html renderer.")},BaseRenderer.prototype.createAudio=function(t){return new AudioElement(t,this.globalData,this)},BaseRenderer.prototype.createFootage=function(t){return new FootageElement(t,this.globalData,this)},BaseRenderer.prototype.buildAllItems=function(){var t,e=this.layers.length;for(t=0;t0&&(this.maskElement.setAttribute("id",y),this.element.maskedElement.setAttribute(v,"url("+getLocationHref()+"#"+y+")"),a.appendChild(this.maskElement)),this.viewData.length&&this.element.addRenderableComponent(this)}TransformElement.prototype={initTransform:function(){var t=new Matrix;this.finalTransform={mProp:this.data.ks?TransformPropertyFactory.getTransformProperty(this,this.data.ks,this):{o:0},_matMdf:!1,_localMatMdf:!1,_opMdf:!1,mat:t,localMat:t,localOpacity:1},this.data.ao&&(this.finalTransform.mProp.autoOriented=!0),this.data.ty},renderTransform:function(){if(this.finalTransform._opMdf=this.finalTransform.mProp.o._mdf||this._isFirstFrame,this.finalTransform._matMdf=this.finalTransform.mProp._mdf||this._isFirstFrame,this.hierarchy){var t,e=this.finalTransform.mat,r=0,i=this.hierarchy.length;if(!this.finalTransform._matMdf)for(;r1&&(a+=" C"+e.o[i-1][0]+","+e.o[i-1][1]+" "+e.i[0][0]+","+e.i[0][1]+" "+e.v[0][0]+","+e.v[0][1]),r.lastPath!==a){var n="";r.elem&&(e.c&&(n=t.inv?this.solidPath+a:a),r.elem.setAttribute("d",n)),r.lastPath=a}},MaskElement.prototype.destroy=function(){this.element=null,this.globalData=null,this.maskElement=null,this.data=null,this.masksProperties=null};var filtersFactory=function(){var t={createFilter:function(t,e){var r=createNS("filter");return r.setAttribute("id",t),!0!==e&&(r.setAttribute("filterUnits","objectBoundingBox"),r.setAttribute("x","0%"),r.setAttribute("y","0%"),r.setAttribute("width","100%"),r.setAttribute("height","100%")),r},createAlphaToLuminanceFilter:function(){var t=createNS("feColorMatrix");return t.setAttribute("type","matrix"),t.setAttribute("color-interpolation-filters","sRGB"),t.setAttribute("values","0 0 0 1 0 0 0 0 1 0 0 0 0 1 0 0 0 0 1 1"),t}};return t}(),featureSupport=function(){var t={maskType:!0,svgLumaHidden:!0,offscreenCanvas:"undefined"!=typeof OffscreenCanvas};return(/MSIE 10/i.test(navigator.userAgent)||/MSIE 9/i.test(navigator.userAgent)||/rv:11.0/i.test(navigator.userAgent)||/Edge\/\d./i.test(navigator.userAgent))&&(t.maskType=!1),/firefox/i.test(navigator.userAgent)&&(t.svgLumaHidden=!1),t}(),registeredEffects$1={},idPrefix="filter_result_";function SVGEffects(t){var e,r,i="SourceGraphic",s=t.data.ef?t.data.ef.length:0,a=createElementID(),n=filtersFactory.createFilter(a,!0),o=0;for(this.filters=[],e=0;e=0&&!this.shapeModifiers[t].processShapes(this._isFirstFrame);t-=1);}},searchProcessedElement:function(t){for(var e=this.processedElements,r=0,i=e.length;r.01)return!1;r+=1}return!0},GradientProperty.prototype.checkCollapsable=function(){if(this.o.length/2!=this.c.length/4)return!1;if(this.data.k.k[0].s)for(var t=0,e=this.data.k.k.length;t0;)h=i.transformers[d].mProps._mdf||h,u-=1,d-=1;if(h)for(u=y-i.styles[p].lvl,d=i.transformers.length-1;u>0;)f.multiply(i.transformers[d].mProps.v),u-=1,d-=1}else f=t;if(n=(c=i.sh.paths)._length,h){for(o="",a=0;a=1?v=.99:v<=-1&&(v=-.99);var b=o*v,_=Math.cos(g+e.a.v)*b+p[0],P=Math.sin(g+e.a.v)*b+p[1];h.setAttribute("fx",_),h.setAttribute("fy",P),l&&!e.g._collapsable&&(e.of.setAttribute("fx",_),e.of.setAttribute("fy",P))}}function h(t,e,r){var i=e.style,s=e.d;s&&(s._mdf||r)&&s.dashStr&&(i.pElem.setAttribute("stroke-dasharray",s.dashStr),i.pElem.setAttribute("stroke-dashoffset",s.dashoffset[0])),e.c&&(e.c._mdf||r)&&i.pElem.setAttribute("stroke","rgb("+bmFloor(e.c.v[0])+","+bmFloor(e.c.v[1])+","+bmFloor(e.c.v[2])+")"),(e.o._mdf||r)&&i.pElem.setAttribute("stroke-opacity",e.o.v),(e.w._mdf||r)&&(i.pElem.setAttribute("stroke-width",e.w.v),i.msElem&&i.msElem.setAttribute("stroke-width",e.w.v))}return{createRenderFunction:function(t){switch(t.ty){case"fl":return a;case"gf":return o;case"gs":return n;case"st":return h;case"sh":case"el":case"rc":case"sr":return s;case"tr":return r;case"no":return i;default:return null}}}}();function SVGShapeElement(t,e,r){this.shapes=[],this.shapesData=t.shapes,this.stylesList=[],this.shapeModifiers=[],this.itemsData=[],this.processedElements=[],this.animatedContents=[],this.initElement(t,e,r),this.prevViewData=[]}function LetterProps(t,e,r,i,s,a){this.o=t,this.sw=e,this.sc=r,this.fc=i,this.m=s,this.p=a,this._mdf={o:!0,sw:!!e,sc:!!r,fc:!!i,m:!0,p:!0}}function TextProperty(t,e){this._frameId=initialDefaultFrame,this.pv="",this.v="",this.kf=!1,this._isFirstFrame=!0,this._mdf=!1,e.d&&e.d.sid&&(e.d=t.globalData.slotManager.getProp(e.d)),this.data=e,this.elem=t,this.comp=this.elem.comp,this.keysIndex=0,this.canResize=!1,this.minimumFontSize=1,this.effectsSequence=[],this.currentData={ascent:0,boxWidth:this.defaultBoxWidth,f:"",fStyle:"",fWeight:"",fc:"",j:"",justifyOffset:"",l:[],lh:0,lineWidths:[],ls:"",of:"",s:"",sc:"",sw:0,t:0,tr:0,sz:0,ps:null,fillColorAnim:!1,strokeColorAnim:!1,strokeWidthAnim:!1,yOffset:0,finalSize:0,finalText:[],finalLineHeight:0,__complete:!1},this.copyData(this.currentData,this.data.d.k[0].s),this.searchProperty()||this.completeTextData(this.currentData)}extendPrototype([BaseElement,TransformElement,SVGBaseElement,IShapeElement,HierarchyElement,FrameElement,RenderableDOMElement],SVGShapeElement),SVGShapeElement.prototype.initSecondaryElement=function(){},SVGShapeElement.prototype.identityMatrix=new Matrix,SVGShapeElement.prototype.buildExpressionInterface=function(){},SVGShapeElement.prototype.createContent=function(){this.searchShapes(this.shapesData,this.itemsData,this.prevViewData,this.layerElement,0,[],!0),this.filterUniqueShapes()},SVGShapeElement.prototype.filterUniqueShapes=function(){var t,e,r,i,s=this.shapes.length,a=this.stylesList.length,n=[],o=!1;for(r=0;r1&&o&&this.setShapesAsAnimated(n)}},SVGShapeElement.prototype.setShapesAsAnimated=function(t){var e,r=t.length;for(e=0;e=0;o-=1){if((f=this.searchProcessedElement(t[o]))?e[o]=r[f-1]:t[o]._render=n,"fl"===t[o].ty||"st"===t[o].ty||"gf"===t[o].ty||"gs"===t[o].ty||"no"===t[o].ty)f?e[o].style.closed=!1:e[o]=this.createStyleElement(t[o],s),t[o]._render&&e[o].style.pElem.parentNode!==i&&i.appendChild(e[o].style.pElem),m.push(e[o].style);else if("gr"===t[o].ty){if(f)for(l=e[o].it.length,h=0;h1,this.kf&&this.addEffect(this.getKeyframeValue.bind(this)),this.kf},TextProperty.prototype.addEffect=function(t){this.effectsSequence.push(t),this.elem.addDynamicProperty(this)},TextProperty.prototype.getValue=function(t){if(this.elem.globalData.frameId!==this.frameId&&this.effectsSequence.length||t){this.currentData.t=this.data.d.k[this.keysIndex].s.t;var e=this.currentData,r=this.keysIndex;if(this.lock)this.setCurrentData(this.currentData);else{var i;this.lock=!0,this._mdf=!1;var s=this.effectsSequence.length,a=t||this.data.d.k[this.keysIndex].s;for(i=0;ie);)r+=1;return this.keysIndex!==r&&(this.keysIndex=r),this.data.d.k[this.keysIndex].s},TextProperty.prototype.buildFinalText=function(t){for(var e,r,i=[],s=0,a=t.length,n=!1,o=!1,h="";s=55296&&e<=56319?FontManager.isRegionalFlag(t,s)?h=t.substr(s,14):(r=t.charCodeAt(s+1))>=56320&&r<=57343&&(FontManager.isModifier(e,r)?(h=t.substr(s,2),n=!0):h=FontManager.isFlagEmoji(t.substr(s,4))?t.substr(s,4):t.substr(s,2)):e>56319?(r=t.charCodeAt(s+1),FontManager.isVariationSelector(e)&&(n=!0)):FontManager.isZeroWidthJoiner(e)&&(n=!0,o=!0),n?(i[i.length-1]+=h,n=!1):i.push(h),s+=h.length;return i},TextProperty.prototype.completeTextData=function(t){t.__complete=!0;var e,r,i,s,a,n,o,h=this.elem.globalData.fontManager,l=this.data,p=[],c=0,f=l.m.g,u=0,d=0,m=0,y=[],g=0,v=0,b=h.getFontByName(t.f),_=0,P=getFontProperties(b);t.fWeight=P.weight,t.fStyle=P.style,t.finalSize=t.s,t.finalText=this.buildFinalText(t.t),r=t.finalText.length,t.finalLineHeight=t.lh;var S,E=t.tr/1e3*t.finalSize;if(t.sz)for(var x,C,A=!0,w=t.sz[0],k=t.sz[1];A;){x=0,g=0,r=(C=this.buildFinalText(t.t)).length,E=t.tr/1e3*t.finalSize;var T=-1;for(e=0;ew&&" "!==C[e]?(-1===T?r+=1:e=T,x+=t.finalLineHeight||1.2*t.finalSize,C.splice(e,T===e?1:0,"\r"),T=-1,g=0):(g+=_,g+=E);x+=b.ascent*t.finalSize/100,this.canResize&&t.finalSize>this.minimumFontSize&&kv?g:v,g=-2*E,s="",i=!0,m+=1):s=M,h.chars?(o=h.getCharData(M,b.fStyle,h.getFontByName(t.f).fFamily),_=i?0:o.w*t.finalSize/100):_=h.measureText(s,t.f,t.finalSize)," "===M?D+=_+E:(g+=_+E+D,D=0),p.push({l:_,an:_,add:u,n:i,anIndexes:[],val:s,line:m,animatorJustifyOffset:0}),2==f){if(u+=_,""===s||" "===s||e===r-1){for(""!==s&&" "!==s||(u-=_);d<=e;)p[d].an=u,p[d].ind=c,p[d].extra=_,d+=1;c+=1,u=0}}else if(3==f){if(u+=_,""===s||e===r-1){for(""===s&&(u-=_);d<=e;)p[d].an=u,p[d].ind=c,p[d].extra=_,d+=1;u=0,c+=1}}else p[c].ind=c,p[c].extra=0,c+=1;if(t.l=p,v=g>v?g:v,y.push(g),t.sz)t.boxWidth=t.sz[0],t.justifyOffset=0;else switch(t.boxWidth=v,t.j){case 1:t.justifyOffset=-t.boxWidth;break;case 2:t.justifyOffset=-t.boxWidth/2;break;default:t.justifyOffset=0}t.lineWidths=y;var F,I,R,L,V=l.a;n=V.length;var B=[];for(a=0;a0?s=this.ne.v/100:a=-this.ne.v/100,this.xe.v>0?n=1-this.xe.v/100:o=1+this.xe.v/100;var h=BezierFactory.getBezierEasing(s,a,n,o).get,l=0,p=this.finalS,c=this.finalE,f=this.data.sh;if(2===f)l=h(l=c===p?i>=c?1:0:t(0,e(.5/(c-p)+(i-p)/(c-p),1)));else if(3===f)l=h(l=c===p?i>=c?0:1:1-t(0,e(.5/(c-p)+(i-p)/(c-p),1)));else if(4===f)c===p?l=0:(l=t(0,e(.5/(c-p)+(i-p)/(c-p),1)))<.5?l*=2:l=1-2*(l-.5),l=h(l);else if(5===f){if(c===p)l=0;else{var u=c-p,d=-u/2+(i=e(t(0,i+.5-p),c-p)),m=u/2;l=Math.sqrt(1-d*d/(m*m))}l=h(l)}else 6===f?(c===p?l=0:(i=e(t(0,i+.5-p),c-p),l=(1+Math.cos(Math.PI+2*Math.PI*i/(c-p)))/2),l=h(l)):(i>=r(p)&&(l=t(0,e(i-p<0?e(c,1)-(p-i):c-i,1))),l=h(l));if(100!==this.sm.v){var y=.01*this.sm.v;0===y&&(y=1e-8);var g=.5-.5*y;l1&&(l=1)}return l*this.a.v},getValue:function(t){this.iterateDynamicProperties(),this._mdf=t||this._mdf,this._currentTextLength=this.elem.textProperty.currentData.l.length||0,t&&2===this.data.r&&(this.e.v=this._currentTextLength);var e=2===this.data.r?1:100/this.data.totalChars,r=this.o.v/e,i=this.s.v/e+r,s=this.e.v/e+r;if(i>s){var a=i;i=s,s=a}this.finalS=i,this.finalE=s}},extendPrototype([DynamicPropertyContainer],i),{getTextSelectorProp:function(t,e,r){return new i(t,e,r)}}}();function TextAnimatorDataProperty(t,e,r){var i={propType:!1},s=PropertyFactory.getProp,a=e.a;this.a={r:a.r?s(t,a.r,0,degToRads,r):i,rx:a.rx?s(t,a.rx,0,degToRads,r):i,ry:a.ry?s(t,a.ry,0,degToRads,r):i,sk:a.sk?s(t,a.sk,0,degToRads,r):i,sa:a.sa?s(t,a.sa,0,degToRads,r):i,s:a.s?s(t,a.s,1,.01,r):i,a:a.a?s(t,a.a,1,0,r):i,o:a.o?s(t,a.o,0,.01,r):i,p:a.p?s(t,a.p,1,0,r):i,sw:a.sw?s(t,a.sw,0,0,r):i,sc:a.sc?s(t,a.sc,1,0,r):i,fc:a.fc?s(t,a.fc,1,0,r):i,fh:a.fh?s(t,a.fh,0,0,r):i,fs:a.fs?s(t,a.fs,0,.01,r):i,fb:a.fb?s(t,a.fb,0,.01,r):i,t:a.t?s(t,a.t,0,0,r):i},this.s=TextSelectorProp.getTextSelectorProp(t,e.s,r),this.s.t=e.s.t}function TextAnimatorProperty(t,e,r){this._isFirstFrame=!0,this._hasMaskedPath=!1,this._frameId=-1,this._textData=t,this._renderType=e,this._elem=r,this._animatorsData=createSizedArray(this._textData.a.length),this._pathData={},this._moreOptions={alignment:{}},this.renderedLetters=[],this.lettersChangedFlag=!1,this.initDynamicPropertyContainer(r)}function ITextElement(){}TextAnimatorProperty.prototype.searchProperties=function(){var t,e,r=this._textData.a.length,i=PropertyFactory.getProp;for(t=0;t=o+ot||!d?(v=(o+ot-l)/h.partialLength,O=u.point[0]+(h.point[0]-u.point[0])*v,$=u.point[1]+(h.point[1]-u.point[1])*v,x.translate(-P[0]*w[s].an*.005,-P[1]*L*.01),p=!1):d&&(l+=h.partialLength,(c+=1)>=d.length&&(c=0,m[f+=1]?d=m[f].points:_.v.c?(c=0,d=m[f=0].points):(l-=h.partialLength,d=null)),d&&(u=h,y=(h=d[c]).partialLength));B=w[s].an/2-w[s].add,x.translate(-B,0,0)}else B=w[s].an/2-w[s].add,x.translate(-B,0,0),x.translate(-P[0]*w[s].an*.005,-P[1]*L*.01,0);for(D=0;Dt?this.textSpans[t].span:createNS(h?"g":"text"),y<=t){if(n.setAttribute("stroke-linecap","butt"),n.setAttribute("stroke-linejoin","round"),n.setAttribute("stroke-miterlimit","4"),this.textSpans[t].span=n,h){var g=createNS("g");n.appendChild(g),this.textSpans[t].childSpan=g}this.textSpans[t].span=n,this.layerElement.appendChild(n)}n.style.display="inherit"}if(l.reset(),p&&(o[t].n&&(c=-d,f+=r.yOffset,f+=u?1:0,u=!1),this.applyTextPropertiesToMatrix(r,l,o[t].line,c,f),c+=o[t].l||0,c+=d),h){var v;if(1===(m=this.globalData.fontManager.getCharData(r.finalText[t],i.fStyle,this.globalData.fontManager.getFontByName(r.f).fFamily)).t)v=new SVGCompElement(m.data,this.globalData,this);else{var b=emptyShapeData;m.data&&m.data.shapes&&(b=this.buildShapeData(m.data,r.finalSize)),v=new SVGShapeElement(b,this.globalData,this)}if(this.textSpans[t].glyph){var _=this.textSpans[t].glyph;this.textSpans[t].childSpan.removeChild(_.layerElement),_.destroy()}this.textSpans[t].glyph=v,v._debug=!0,v.prepareFrame(0),v.renderFrame(),this.textSpans[t].childSpan.appendChild(v.layerElement),1===m.t&&this.textSpans[t].childSpan.setAttribute("transform","scale("+r.finalSize/100+","+r.finalSize/100+")")}else p&&n.setAttribute("transform","translate("+l.props[12]+","+l.props[13]+")"),n.textContent=o[t].val,n.setAttributeNS("http://www.w3.org/XML/1998/namespace","xml:space","preserve")}p&&n&&n.setAttribute("d","")}else{var P=this.textContainer,S="start";switch(r.j){case 1:S="end";break;case 2:S="middle";break;default:S="start"}P.setAttribute("text-anchor",S),P.setAttribute("letter-spacing",d);var E=this.buildTextContents(r.finalText);for(e=E.length,f=r.ps?r.ps[1]+r.ascent:0,t=0;t=0;e-=1)(this.completeLayers||this.elements[e])&&this.elements[e].prepareFrame(t-this.layers[e].st);if(this.globalData._mdf)for(e=0;e=0;r-=1)(this.completeLayers||this.elements[r])&&(this.elements[r].prepareFrame(this.renderedFrame-this.layers[r].st),this.elements[r]._mdf&&(this._mdf=!0))}},ICompElement.prototype.renderInnerContent=function(){var t,e=this.layers.length;for(t=0;t=0;r-=1)t.finalTransform.multiply(t.transforms[r].transform.mProps.v);t._mdf=s},processSequences:function(t){var e,r=this.sequenceList.length;for(e=0;e=1){this.buffers=[];var t=this.globalData.canvasContext,e=assetLoader.createCanvas(t.canvas.width,t.canvas.height);this.buffers.push(e);var r=assetLoader.createCanvas(t.canvas.width,t.canvas.height);this.buffers.push(r),this.data.tt>=3&&!document._isProxy&&assetLoader.loadLumaCanvas()}this.canvasContext=this.globalData.canvasContext,this.transformCanvas=this.globalData.transformCanvas,this.renderableEffectsManager=new CVEffects(this),this.searchEffectTransforms()},createContent:function(){},setBlendMode:function(){var t=this.globalData;if(t.blendMode!==this.data.bm){t.blendMode=this.data.bm;var e=getBlendMode(this.data.bm);t.canvasContext.globalCompositeOperation=e}},createRenderableComponents:function(){this.maskManager=new CVMaskElement(this.data,this),this.transformEffects=this.renderableEffectsManager.getEffects(effectTypes.TRANSFORM_EFFECT)},hideElement:function(){this.hidden||this.isInRange&&!this.isTransparent||(this.hidden=!0)},showElement:function(){this.isInRange&&!this.isTransparent&&(this.hidden=!1,this._isFirstFrame=!0,this.maskManager._isFirstFrame=!0)},clearCanvas:function(t){t.clearRect(this.transformCanvas.tx,this.transformCanvas.ty,this.transformCanvas.w*this.transformCanvas.sx,this.transformCanvas.h*this.transformCanvas.sy)},prepareLayer:function(){if(this.data.tt>=1){var t=this.buffers[0].getContext("2d");this.clearCanvas(t),t.drawImage(this.canvasContext.canvas,0,0),this.currentTransform=this.canvasContext.getTransform(),this.canvasContext.setTransform(1,0,0,1,0,0),this.clearCanvas(this.canvasContext),this.canvasContext.setTransform(this.currentTransform)}},exitLayer:function(){if(this.data.tt>=1){var t=this.buffers[1],e=t.getContext("2d");if(this.clearCanvas(e),e.drawImage(this.canvasContext.canvas,0,0),this.canvasContext.setTransform(1,0,0,1,0,0),this.clearCanvas(this.canvasContext),this.canvasContext.setTransform(this.currentTransform),this.comp.getElementById("tp"in this.data?this.data.tp:this.data.ind-1).renderFrame(!0),this.canvasContext.setTransform(1,0,0,1,0,0),this.data.tt>=3&&!document._isProxy){var r=assetLoader.getLumaCanvas(this.canvasContext.canvas);r.getContext("2d").drawImage(this.canvasContext.canvas,0,0),this.clearCanvas(this.canvasContext),this.canvasContext.drawImage(r,0,0)}this.canvasContext.globalCompositeOperation=operationsMap[this.data.tt],this.canvasContext.drawImage(t,0,0),this.canvasContext.globalCompositeOperation="destination-over",this.canvasContext.drawImage(this.buffers[0],0,0),this.canvasContext.setTransform(this.currentTransform),this.canvasContext.globalCompositeOperation="source-over"}},renderFrame:function(t){if(!this.hidden&&!this.data.hd&&(1!==this.data.td||t)){this.renderTransform(),this.renderRenderable(),this.renderLocalTransform(),this.setBlendMode();var e=0===this.data.ty;this.prepareLayer(),this.globalData.renderer.save(e),this.globalData.renderer.ctxTransform(this.finalTransform.localMat.props),this.globalData.renderer.ctxOpacity(this.finalTransform.localOpacity),this.renderInnerContent(),this.globalData.renderer.restore(e),this.exitLayer(),this.maskManager.hasMasks&&this.globalData.renderer.restore(!0),this._isFirstFrame&&(this._isFirstFrame=!1)}},destroy:function(){this.canvasContext=null,this.data=null,this.globalData=null,this.maskManager.destroy()},mHelper:new Matrix},CVBaseElement.prototype.hide=CVBaseElement.prototype.hideElement,CVBaseElement.prototype.show=CVBaseElement.prototype.showElement,CVShapeData.prototype.setAsAnimated=SVGShapeData.prototype.setAsAnimated,extendPrototype([BaseElement,TransformElement,CVBaseElement,IShapeElement,HierarchyElement,FrameElement,RenderableElement],CVShapeElement),CVShapeElement.prototype.initElement=RenderableDOMElement.prototype.initElement,CVShapeElement.prototype.transformHelper={opacity:1,_opMdf:!1},CVShapeElement.prototype.dashResetter=[],CVShapeElement.prototype.createContent=function(){this.searchShapes(this.shapesData,this.itemsData,this.prevViewData,!0,[])},CVShapeElement.prototype.createStyleElement=function(t,e){var r={data:t,type:t.ty,preTransforms:this.transformsManager.addTransformSequence(e),transforms:[],elements:[],closed:!0===t.hd},i={};if("fl"===t.ty||"st"===t.ty?(i.c=PropertyFactory.getProp(this,t.c,1,255,this),i.c.k||(r.co="rgb("+bmFloor(i.c.v[0])+","+bmFloor(i.c.v[1])+","+bmFloor(i.c.v[2])+")")):"gf"!==t.ty&&"gs"!==t.ty||(i.s=PropertyFactory.getProp(this,t.s,1,null,this),i.e=PropertyFactory.getProp(this,t.e,1,null,this),i.h=PropertyFactory.getProp(this,t.h||{k:0},0,.01,this),i.a=PropertyFactory.getProp(this,t.a||{k:0},0,degToRads,this),i.g=new GradientProperty(this,t.g,this)),i.o=PropertyFactory.getProp(this,t.o,0,.01,this),"st"===t.ty||"gs"===t.ty){if(r.lc=lineCapEnum[t.lc||2],r.lj=lineJoinEnum[t.lj||2],1==t.lj&&(r.ml=t.ml),i.w=PropertyFactory.getProp(this,t.w,0,null,this),i.w.k||(r.wi=i.w.v),t.d){var s=new DashProperty(this,t.d,"canvas",this);i.d=s,i.d.k||(r.da=i.d.dashArray,r.do=i.d.dashoffset[0])}}else r.r=2===t.r?"evenodd":"nonzero";return this.stylesList.push(r),i.style=r,i},CVShapeElement.prototype.createGroupElement=function(){return{it:[],prevViewData:[]}},CVShapeElement.prototype.createTransformElement=function(t){return{transform:{opacity:1,_opMdf:!1,key:this.transformsManager.getNewKey(),op:PropertyFactory.getProp(this,t.o,0,.01,this),mProps:TransformPropertyFactory.getTransformProperty(this,t,this)}}},CVShapeElement.prototype.createShapeElement=function(t){var e=new CVShapeData(this,t,this.stylesList,this.transformsManager);return this.shapes.push(e),this.addShapeToModifiers(e),e},CVShapeElement.prototype.reloadShapes=function(){var t;this._isFirstFrame=!0;var e=this.itemsData.length;for(t=0;t=0;a-=1){if((h=this.searchProcessedElement(t[a]))?e[a]=r[h-1]:t[a]._shouldRender=i,"fl"===t[a].ty||"st"===t[a].ty||"gf"===t[a].ty||"gs"===t[a].ty)h?e[a].style.closed=!1:e[a]=this.createStyleElement(t[a],d),f.push(e[a].style);else if("gr"===t[a].ty){if(h)for(o=e[a].it.length,n=0;n=0;s-=1)"tr"===e[s].ty?(a=r[s].transform,this.renderShapeTransform(t,a)):"sh"===e[s].ty||"el"===e[s].ty||"rc"===e[s].ty||"sr"===e[s].ty?this.renderPath(e[s],r[s]):"fl"===e[s].ty?this.renderFill(e[s],r[s],a):"st"===e[s].ty?this.renderStroke(e[s],r[s],a):"gf"===e[s].ty||"gs"===e[s].ty?this.renderGradientFill(e[s],r[s],a):"gr"===e[s].ty?this.renderShape(a,e[s].it,r[s].it):e[s].ty;i&&this.drawLayer()},CVShapeElement.prototype.renderStyledShape=function(t,e){if(this._isFirstFrame||e._mdf||t.transforms._mdf){var r,i,s,a=t.trNodes,n=e.paths,o=n._length;a.length=0;var h=t.transforms.finalTransform;for(s=0;s=1?c=.99:c<=-1&&(c=-.99);var f=l*c,u=Math.cos(p+e.a.v)*f+o[0],d=Math.sin(p+e.a.v)*f+o[1];i=n.createRadialGradient(u,d,0,o[0],o[1],l)}var m=t.g.p,y=e.g.c,g=1;for(a=0;ao&&"xMidYMid slice"===h||ns&&"meet"===o||as&&"slice"===o)?(r-this.transformCanvas.w*(i/this.transformCanvas.h))/2*this.renderConfig.dpr:"xMax"===l&&(as&&"slice"===o)?(r-this.transformCanvas.w*(i/this.transformCanvas.h))*this.renderConfig.dpr:0,this.transformCanvas.ty="YMid"===p&&(a>s&&"meet"===o||as&&"meet"===o||a=0;t-=1)this.elements[t]&&this.elements[t].destroy&&this.elements[t].destroy();this.elements.length=0,this.globalData.canvasContext=null,this.animationItem.container=null,this.destroyed=!0},CanvasRendererBase.prototype.renderFrame=function(t,e){if((this.renderedFrame!==t||!0!==this.renderConfig.clearCanvas||e)&&!this.destroyed&&-1!==t){var r;this.renderedFrame=t,this.globalData.frameNum=t-this.animationItem._isFirstFrame,this.globalData.frameId+=1,this.globalData._mdf=!this.renderConfig.clearCanvas||e,this.globalData.projectInterface.currentFrame=t;var i=this.layers.length;for(this.completeLayers||this.checkLayers(t),r=i-1;r>=0;r-=1)(this.completeLayers||this.elements[r])&&this.elements[r].prepareFrame(t-this.layers[r].st);if(this.globalData._mdf){for(!0===this.renderConfig.clearCanvas?this.canvasContext.clearRect(0,0,this.transformCanvas.w,this.transformCanvas.h):this.save(),r=i-1;r>=0;r-=1)(this.completeLayers||this.elements[r])&&this.elements[r].renderFrame();!0!==this.renderConfig.clearCanvas&&this.restore()}}},CanvasRendererBase.prototype.buildItem=function(t){var e=this.elements;if(!e[t]&&99!==this.layers[t].ty){var r=this.createItem(this.layers[t],this,this.globalData);e[t]=r,r.initExpressions()}},CanvasRendererBase.prototype.checkPendingElements=function(){for(;this.pendingElements.length;)this.pendingElements.pop().checkParenting()},CanvasRendererBase.prototype.hide=function(){this.animationItem.container.style.display="none"},CanvasRendererBase.prototype.show=function(){this.animationItem.container.style.display="block"},CVContextData.prototype.duplicate=function(){var t=2*this._length,e=0;for(e=this._length;e=0;t-=1)(this.completeLayers||this.elements[t])&&this.elements[t].renderFrame()},CVCompElement.prototype.destroy=function(){var t;for(t=this.layers.length-1;t>=0;t-=1)this.elements[t]&&this.elements[t].destroy();this.layers=null,this.elements=null},CVCompElement.prototype.createComp=function(t){return new CVCompElement(t,this.globalData,this)},extendPrototype([CanvasRendererBase],CanvasRenderer),CanvasRenderer.prototype.createComp=function(t){return new CVCompElement(t,this.globalData,this)},HBaseElement.prototype={checkBlendMode:function(){},initRendererElement:function(){this.baseElement=createTag(this.data.tg||"div"),this.data.hasMask?(this.svgElement=createNS("svg"),this.layerElement=createNS("g"),this.maskedElement=this.layerElement,this.svgElement.appendChild(this.layerElement),this.baseElement.appendChild(this.svgElement)):this.layerElement=this.baseElement,styleDiv(this.baseElement)},createContainerElements:function(){this.renderableEffectsManager=new CVEffects(this),this.transformedElement=this.baseElement,this.maskedElement=this.layerElement,this.data.ln&&this.layerElement.setAttribute("id",this.data.ln),this.data.cl&&this.layerElement.setAttribute("class",this.data.cl),0!==this.data.bm&&this.setBlendMode()},renderElement:function(){var t=this.transformedElement?this.transformedElement.style:{};if(this.finalTransform._matMdf){var e=this.finalTransform.mat.toCSS();t.transform=e,t.webkitTransform=e}this.finalTransform._opMdf&&(t.opacity=this.finalTransform.mProp.o.v)},renderFrame:function(){this.data.hd||this.hidden||(this.renderTransform(),this.renderRenderable(),this.renderElement(),this.renderInnerContent(),this._isFirstFrame&&(this._isFirstFrame=!1))},destroy:function(){this.layerElement=null,this.transformedElement=null,this.matteElement&&(this.matteElement=null),this.maskManager&&(this.maskManager.destroy(),this.maskManager=null)},createRenderableComponents:function(){this.maskManager=new MaskElement(this.data,this,this.globalData)},addEffects:function(){},setMatte:function(){}},HBaseElement.prototype.getBaseElement=SVGBaseElement.prototype.getBaseElement,HBaseElement.prototype.destroyBaseElement=HBaseElement.prototype.destroy,HBaseElement.prototype.buildElementParenting=BaseRenderer.prototype.buildElementParenting,extendPrototype([BaseElement,TransformElement,HBaseElement,HierarchyElement,FrameElement,RenderableDOMElement],HSolidElement),HSolidElement.prototype.createContent=function(){var t;this.data.hasMask?((t=createNS("rect")).setAttribute("width",this.data.sw),t.setAttribute("height",this.data.sh),t.setAttribute("fill",this.data.sc),this.svgElement.setAttribute("width",this.data.sw),this.svgElement.setAttribute("height",this.data.sh)):((t=createTag("div")).style.width=this.data.sw+"px",t.style.height=this.data.sh+"px",t.style.backgroundColor=this.data.sc),this.layerElement.appendChild(t)},extendPrototype([BaseElement,TransformElement,HSolidElement,SVGShapeElement,HBaseElement,HierarchyElement,FrameElement,RenderableElement],HShapeElement),HShapeElement.prototype._renderShapeFrame=HShapeElement.prototype.renderInnerContent,HShapeElement.prototype.createContent=function(){var t;if(this.baseElement.style.fontSize=0,this.data.hasMask)this.layerElement.appendChild(this.shapesContainer),t=this.svgElement;else{t=createNS("svg");var e=this.comp.data?this.comp.data:this.globalData.compSize;t.setAttribute("width",e.w),t.setAttribute("height",e.h),t.appendChild(this.shapesContainer),this.layerElement.appendChild(t)}this.searchShapes(this.shapesData,this.itemsData,this.prevViewData,this.shapesContainer,0,[],!0),this.filterUniqueShapes(),this.shapeCont=t},HShapeElement.prototype.getTransformedPoint=function(t,e){var r,i=t.length;for(r=0;r0&&o<1&&c[f].push(this.calculateF(o,t,e,r,i,f)):(h=a*a-4*n*s)>=0&&((l=(-a+bmSqrt(h))/(2*s))>0&&l<1&&c[f].push(this.calculateF(l,t,e,r,i,f)),(p=(-a-bmSqrt(h))/(2*s))>0&&p<1&&c[f].push(this.calculateF(p,t,e,r,i,f))));this.shapeBoundingBox.left=bmMin.apply(null,c[0]),this.shapeBoundingBox.top=bmMin.apply(null,c[1]),this.shapeBoundingBox.right=bmMax.apply(null,c[0]),this.shapeBoundingBox.bottom=bmMax.apply(null,c[1])},HShapeElement.prototype.calculateF=function(t,e,r,i,s,a){return bmPow(1-t,3)*e[a]+3*bmPow(1-t,2)*t*r[a]+3*(1-t)*bmPow(t,2)*i[a]+bmPow(t,3)*s[a]},HShapeElement.prototype.calculateBoundingBox=function(t,e){var r,i=t.length;for(r=0;rr&&(r=s)}r*=t.mult}else r=t.v*t.mult;e.x-=r,e.xMax+=r,e.y-=r,e.yMax+=r},HShapeElement.prototype.currentBoxContains=function(t){return this.currentBBox.x<=t.x&&this.currentBBox.y<=t.y&&this.currentBBox.width+this.currentBBox.x>=t.x+t.width&&this.currentBBox.height+this.currentBBox.y>=t.y+t.height},HShapeElement.prototype.renderInnerContent=function(){if(this._renderShapeFrame(),!this.hidden&&(this._isFirstFrame||this._mdf)){var t=this.tempBoundingBox,e=999999;if(t.x=e,t.xMax=-e,t.y=e,t.yMax=-e,this.calculateBoundingBox(this.itemsData,t),t.width=t.xMax=0;t-=1){var i=this.hierarchy[t].finalTransform.mProp;this.mat.translate(-i.p.v[0],-i.p.v[1],i.p.v[2]),this.mat.rotateX(-i.or.v[0]).rotateY(-i.or.v[1]).rotateZ(i.or.v[2]),this.mat.rotateX(-i.rx.v).rotateY(-i.ry.v).rotateZ(i.rz.v),this.mat.scale(1/i.s.v[0],1/i.s.v[1],1/i.s.v[2]),this.mat.translate(i.a.v[0],i.a.v[1],i.a.v[2])}if(this.p?this.mat.translate(-this.p.v[0],-this.p.v[1],this.p.v[2]):this.mat.translate(-this.px.v,-this.py.v,this.pz.v),this.a){var s;s=this.p?[this.p.v[0]-this.a.v[0],this.p.v[1]-this.a.v[1],this.p.v[2]-this.a.v[2]]:[this.px.v-this.a.v[0],this.py.v-this.a.v[1],this.pz.v-this.a.v[2]];var a=Math.sqrt(Math.pow(s[0],2)+Math.pow(s[1],2)+Math.pow(s[2],2)),n=[s[0]/a,s[1]/a,s[2]/a],o=Math.sqrt(n[2]*n[2]+n[0]*n[0]),h=Math.atan2(n[1],o),l=Math.atan2(n[0],-n[2]);this.mat.rotateY(l).rotateX(-h)}this.mat.rotateX(-this.rx.v).rotateY(-this.ry.v).rotateZ(this.rz.v),this.mat.rotateX(-this.or.v[0]).rotateY(-this.or.v[1]).rotateZ(this.or.v[2]),this.mat.translate(this.globalData.compSize.w/2,this.globalData.compSize.h/2,0),this.mat.translate(0,0,this.pe.v);var p=!this._prevMat.equals(this.mat);if((p||this.pe._mdf)&&this.comp.threeDElements){var c,f,u;for(e=this.comp.threeDElements.length,t=0;t=t)return this.threeDElements[e].perspectiveElem;e+=1}return null},HybridRendererBase.prototype.createThreeDContainer=function(t,e){var r,i,s=createTag("div");styleDiv(s);var a=createTag("div");if(styleDiv(a),"3d"===e){(r=s.style).width=this.globalData.compSize.w+"px",r.height=this.globalData.compSize.h+"px";var n="50% 50%";r.webkitTransformOrigin=n,r.mozTransformOrigin=n,r.transformOrigin=n;var o="matrix3d(1,0,0,0,0,1,0,0,0,0,1,0,0,0,0,1)";(i=a.style).transform=o,i.webkitTransform=o}s.appendChild(a);var h={container:a,perspectiveElem:s,startPos:t,endPos:t,type:e};return this.threeDElements.push(h),h},HybridRendererBase.prototype.build3dContainers=function(){var t,e,r=this.layers.length,i="";for(t=0;t=0;t-=1)this.resizerElem.appendChild(this.threeDElements[t].perspectiveElem)},HybridRendererBase.prototype.addTo3dContainer=function(t,e){for(var r=0,i=this.threeDElements.length;rn?(t=s/this.globalData.compSize.w,e=s/this.globalData.compSize.w,r=0,i=(a-this.globalData.compSize.h*(s/this.globalData.compSize.w))/2):(t=a/this.globalData.compSize.h,e=a/this.globalData.compSize.h,r=(s-this.globalData.compSize.w*(a/this.globalData.compSize.h))/2,i=0);var o=this.resizerElem.style;o.webkitTransform="matrix3d("+t+",0,0,0,0,"+e+",0,0,0,0,1,0,"+r+","+i+",0,1)",o.transform=o.webkitTransform},HybridRendererBase.prototype.renderFrame=SVGRenderer.prototype.renderFrame,HybridRendererBase.prototype.hide=function(){this.resizerElem.style.display="none"},HybridRendererBase.prototype.show=function(){this.resizerElem.style.display="block"},HybridRendererBase.prototype.initItems=function(){if(this.buildAllItems(),this.camera)this.camera.setup();else{var t,e=this.globalData.compSize.w,r=this.globalData.compSize.h,i=this.threeDElements.length;for(t=0;t=n;)t/=2,e/=2,r>>>=1;return(t+r)/e};return v.int32=function(){return 0|g.g(4)},v.quick=function(){return g.g(4)/4294967296},v.double=v,c(f(g.S),t),(u.pass||d||function(t,r,i,s){return s&&(s.S&&l(s,g),t.state=function(){return l(g,{})}),i?(e.random=t,r):t})(v,y,"global"in u?u.global:this==e,u.state)},c(e.random(),t)}function initialize$2(t){seedRandom([],t)}var propTypes={SHAPE:"shape"};function _typeof$1(t){return _typeof$1="function"==typeof Symbol&&"symbol"==typeof Symbol.iterator?function(t){return typeof t}:function(t){return t&&"function"==typeof Symbol&&t.constructor===Symbol&&t!==Symbol.prototype?"symbol":typeof t},_typeof$1(t)}var ExpressionManager=function(){var ob={},Math=BMMath,window=null,document=null,XMLHttpRequest=null,fetch=null,frames=null,_lottieGlobal={};function resetFrame(){_lottieGlobal={}}function $bm_isInstanceOfArray(t){return t.constructor===Array||t.constructor===Float32Array}function isNumerable(t,e){return"number"===t||e instanceof Number||"boolean"===t||"string"===t}function $bm_neg(t){var e=_typeof$1(t);if("number"===e||t instanceof Number||"boolean"===e)return-t;if($bm_isInstanceOfArray(t)){var r,i=t.length,s=[];for(r=0;rr){var i=r;r=e,e=i}return Math.min(Math.max(t,e),r)}function radiansToDegrees(t){return t/degToRads}var radians_to_degrees=radiansToDegrees;function degreesToRadians(t){return t*degToRads}var degrees_to_radians=radiansToDegrees,helperLengthArray=[0,0,0,0,0,0];function length(t,e){if("number"==typeof t||t instanceof Number)return e=e||0,Math.abs(t-e);var r;e||(e=helperLengthArray);var i=Math.min(t.length,e.length),s=0;for(r=0;r.5?l/(2-n-o):l/(n+o),n){case i:e=(s-a)/l+(s1&&(r-=1),r<1/6?t+6*(e-t)*r:r<.5?e:r<2/3?t+(e-t)*(2/3-r)*6:t}function hslToRgb(t){var e,r,i,s=t[0],a=t[1],n=t[2];if(0===a)e=n,i=n,r=n;else{var o=n<.5?n*(1+a):n+a-n*a,h=2*n-o;e=hue2rgb(h,o,s+1/3),r=hue2rgb(h,o,s),i=hue2rgb(h,o,s-1/3)}return[e,r,i,t[3]]}function linear(t,e,r,i,s){if(void 0!==i&&void 0!==s||(i=e,s=r,e=0,r=1),r=r)return s;var n,o=r===e?0:(t-e)/(r-e);if(!i.length)return i+(s-i)*o;var h=i.length,l=createTypedArray("float32",h);for(n=0;n1){for(i=0;i1?e=1:e<0&&(e=0);var n=t(e);if($bm_isInstanceOfArray(s)){var o,h=s.length,l=createTypedArray("float32",h);for(o=0;odata.k[e].t&&tdata.k[e+1].t-t?(r=e+2,i=data.k[e+1].t):(r=e+1,i=data.k[e].t);break}}-1===r&&(r=e+1,i=data.k[e].t)}else r=0,i=0;var a={};return a.index=r,a.time=i/elem.comp.globalData.frameRate,a}function key(t){var e,r,i;if(!data.k.length||"number"==typeof data.k[0])throw new Error("The property has no keyframe at index "+t);t-=1,e={time:data.k[t].t/elem.comp.globalData.frameRate,value:[]};var s=Object.prototype.hasOwnProperty.call(data.k[t],"s")?data.k[t].s:data.k[t-1].e;for(i=s.length,r=0;rl.length-1)&&(e=l.length-1),i=p-(s=l[l.length-1-e].t)),"pingpong"===t){if(Math.floor((h-s)/i)%2!=0)return this.getValueAtTime((i-(h-s)%i+s)/this.comp.globalData.frameRate,0)}else{if("offset"===t){var c=this.getValueAtTime(s/this.comp.globalData.frameRate,0),f=this.getValueAtTime(p/this.comp.globalData.frameRate,0),u=this.getValueAtTime(((h-s)%i+s)/this.comp.globalData.frameRate,0),d=Math.floor((h-s)/i);if(this.pv.length){for(n=(o=new Array(c.length)).length,a=0;a=p)return this.pv;if(r?s=p+(i=e?Math.abs(this.elem.comp.globalData.frameRate*e):Math.max(0,this.elem.data.op-p)):((!e||e>l.length-1)&&(e=l.length-1),i=(s=l[e].t)-p),"pingpong"===t){if(Math.floor((p-h)/i)%2==0)return this.getValueAtTime(((p-h)%i+p)/this.comp.globalData.frameRate,0)}else{if("offset"===t){var c=this.getValueAtTime(p/this.comp.globalData.frameRate,0),f=this.getValueAtTime(s/this.comp.globalData.frameRate,0),u=this.getValueAtTime((i-(p-h)%i+p)/this.comp.globalData.frameRate,0),d=Math.floor((p-h)/i)+1;if(this.pv.length){for(n=(o=new Array(c.length)).length,a=0;a1?(s+t-a)/(e-1):1,o=0,h=0;for(r=this.pv.length?createTypedArray("float32",this.pv.length):0;on){var p=o,c=r.c&&o===h-1?0:o+1,f=(n-l)/a[o].addedLength;i=bez.getPointInSegment(r.v[p],r.v[c],r.o[p],r.i[c],f,a[o]);break}l+=a[o].addedLength,o+=1}return i||(i=r.c?[r.v[0][0],r.v[0][1]]:[r.v[r._length-1][0],r.v[r._length-1][1]]),i},vectorOnPath:function(t,e,r){1==t?t=this.v.c:0==t&&(t=.999);var i=this.pointOnPath(t,e),s=this.pointOnPath(t+.001,e),a=s[0]-i[0],n=s[1]-i[1],o=Math.sqrt(Math.pow(a,2)+Math.pow(n,2));return 0===o?[0,0]:"tangent"===r?[a/o,n/o]:[-n/o,a/o]},tangentOnPath:function(t,e){return this.vectorOnPath(t,e,"tangent")},normalOnPath:function(t,e){return this.vectorOnPath(t,e,"normal")},setGroupProperty:expressionHelpers.setGroupProperty,getValueAtTime:expressionHelpers.getStaticValueAtTime},extendPrototype([l],o),extendPrototype([l],h),h.prototype.getValueAtTime=function(t){return this._cachingAtTime||(this._cachingAtTime={shapeValue:shapePool.clone(this.pv),lastIndex:0,lastTime:initialDefaultFrame}),t*=this.elem.globalData.frameRate,(t-=this.offsetTime)!==this._cachingAtTime.lastTime&&(this._cachingAtTime.lastIndex=this._cachingAtTime.lastTime=l?u<0?i:s:i+f*Math.pow((a-t)/u,1/r),p[c]=n,c+=1,o+=256/255;return p.join(" ")},SVGProLevelsFilter.prototype.renderFrame=function(t){if(t||this.filterManager._mdf){var e,r=this.filterManager.effectElements;this.feFuncRComposed&&(t||r[3].p._mdf||r[4].p._mdf||r[5].p._mdf||r[6].p._mdf||r[7].p._mdf)&&(e=this.getTableValue(r[3].p.v,r[4].p.v,r[5].p.v,r[6].p.v,r[7].p.v),this.feFuncRComposed.setAttribute("tableValues",e),this.feFuncGComposed.setAttribute("tableValues",e),this.feFuncBComposed.setAttribute("tableValues",e)),this.feFuncR&&(t||r[10].p._mdf||r[11].p._mdf||r[12].p._mdf||r[13].p._mdf||r[14].p._mdf)&&(e=this.getTableValue(r[10].p.v,r[11].p.v,r[12].p.v,r[13].p.v,r[14].p.v),this.feFuncR.setAttribute("tableValues",e)),this.feFuncG&&(t||r[17].p._mdf||r[18].p._mdf||r[19].p._mdf||r[20].p._mdf||r[21].p._mdf)&&(e=this.getTableValue(r[17].p.v,r[18].p.v,r[19].p.v,r[20].p.v,r[21].p.v),this.feFuncG.setAttribute("tableValues",e)),this.feFuncB&&(t||r[24].p._mdf||r[25].p._mdf||r[26].p._mdf||r[27].p._mdf||r[28].p._mdf)&&(e=this.getTableValue(r[24].p.v,r[25].p.v,r[26].p.v,r[27].p.v,r[28].p.v),this.feFuncB.setAttribute("tableValues",e)),this.feFuncA&&(t||r[31].p._mdf||r[32].p._mdf||r[33].p._mdf||r[34].p._mdf||r[35].p._mdf)&&(e=this.getTableValue(r[31].p.v,r[32].p.v,r[33].p.v,r[34].p.v,r[35].p.v),this.feFuncA.setAttribute("tableValues",e))}},extendPrototype([SVGComposableEffect],SVGDropShadowEffect),SVGDropShadowEffect.prototype.renderFrame=function(t){if(t||this.filterManager._mdf){if((t||this.filterManager.effectElements[4].p._mdf)&&this.feGaussianBlur.setAttribute("stdDeviation",this.filterManager.effectElements[4].p.v/4),t||this.filterManager.effectElements[0].p._mdf){var e=this.filterManager.effectElements[0].p.v;this.feFlood.setAttribute("flood-color",rgbToHex(Math.round(255*e[0]),Math.round(255*e[1]),Math.round(255*e[2])))}if((t||this.filterManager.effectElements[1].p._mdf)&&this.feFlood.setAttribute("flood-opacity",this.filterManager.effectElements[1].p.v/255),t||this.filterManager.effectElements[2].p._mdf||this.filterManager.effectElements[3].p._mdf){var r=this.filterManager.effectElements[3].p.v,i=(this.filterManager.effectElements[2].p.v-90)*degToRads,s=r*Math.cos(i),a=r*Math.sin(i);this.feOffset.setAttribute("dx",s),this.feOffset.setAttribute("dy",a)}}};var _svgMatteSymbols=[];function SVGMatte3Effect(t,e,r){this.initialized=!1,this.filterManager=e,this.filterElem=t,this.elem=r,r.matteElement=createNS("g"),r.matteElement.appendChild(r.layerElement),r.matteElement.appendChild(r.transformedElement),r.baseElement=r.matteElement}function SVGGaussianBlurEffect(t,e,r,i){t.setAttribute("x","-100%"),t.setAttribute("y","-100%"),t.setAttribute("width","300%"),t.setAttribute("height","300%"),this.filterManager=e;var s=createNS("feGaussianBlur");s.setAttribute("result",i),t.appendChild(s),this.feGaussianBlur=s}function TransformEffect(){}function SVGTransformEffect(t,e){this.init(e)}function CVTransformEffect(t){this.init(t)}return SVGMatte3Effect.prototype.findSymbol=function(t){for(var e=0,r=_svgMatteSymbols.length;eObject.prototype.hasOwnProperty.call(t,e)))}function fromURL(t){return _fromURL.apply(this,arguments)}function _fromURL(){return(_fromURL=_asyncToGenerator((function*(t){if("string"!=typeof t)throw new Error("The url value must be a string");var e;try{var r=new URL(t),i=yield fetch(r.toString());e=yield i.json()}catch(t){throw new Error("An error occurred while trying to load the Lottie file from URL")}return e}))).apply(this,arguments)}exports.PlayerState=void 0,PlayerState=exports.PlayerState||(exports.PlayerState={}),PlayerState.Destroyed="destroyed",PlayerState.Error="error",PlayerState.Frozen="frozen",PlayerState.Loading="loading",PlayerState.Paused="paused",PlayerState.Playing="playing",PlayerState.Stopped="stopped",exports.PlayMode=void 0,PlayMode=exports.PlayMode||(exports.PlayMode={}),PlayMode.Bounce="bounce",PlayMode.Normal="normal",exports.PlayerEvents=void 0,PlayerEvents=exports.PlayerEvents||(exports.PlayerEvents={}),PlayerEvents.Complete="complete",PlayerEvents.Destroyed="destroyed",PlayerEvents.Error="error",PlayerEvents.Frame="frame",PlayerEvents.Freeze="freeze",PlayerEvents.Load="load",PlayerEvents.Loop="loop",PlayerEvents.Pause="pause",PlayerEvents.Play="play",PlayerEvents.Ready="ready",PlayerEvents.Rendered="rendered",PlayerEvents.Stop="stop",exports.LottiePlayer=class extends s{constructor(){super(...arguments),this.autoplay=!1,this.background="transparent",this.controls=!1,this.currentState=exports.PlayerState.Loading,this.description="Lottie animation",this.direction=1,this.disableCheck=!1,this.disableShadowDOM=!1,this.hover=!1,this.intermission=1,this.loop=!1,this.mode=exports.PlayMode.Normal,this.preserveAspectRatio="xMidYMid meet",this.renderer="svg",this.speed=1,this._io=void 0,this._counter=1}load(t){var e=this;return _asyncToGenerator((function*(){var r={container:e.container,loop:!1,autoplay:!1,renderer:e.renderer,rendererSettings:Object.assign({preserveAspectRatio:e.preserveAspectRatio,clearCanvas:!1,progressiveLoad:!0,hideOnTransparent:!0},e.viewBoxSize&&{viewBoxSize:e.viewBoxSize})};try{var i=parseSrc(t),s={},a="string"==typeof i?"path":"animationData";e._lottie&&e._lottie.destroy(),e.webworkers&&lottie$1.exports.useWebWorker(!0),e._lottie=lottie$1.exports.loadAnimation(Object.assign(Object.assign({},r),{[a]:i})),e._attachEventListeners(),e.disableCheck||("path"===a?(s=yield fromURL(i),a="animationData"):s=i,isLottie(s)||(e.currentState=exports.PlayerState.Error,e.dispatchEvent(new CustomEvent(exports.PlayerEvents.Error))))}catch(t){e.currentState=exports.PlayerState.Error,e.dispatchEvent(new CustomEvent(exports.PlayerEvents.Error))}}))()}getLottie(){return this._lottie}getVersions(){return{lottieWebVersion:LOTTIE_WEB_VERSION,lottiePlayerVersion:LOTTIE_PLAYER_VERSION}}play(){this._lottie&&(this._lottie.play(),this.currentState=exports.PlayerState.Playing,this.dispatchEvent(new CustomEvent(exports.PlayerEvents.Play)))}pause(){this._lottie&&(this._lottie.pause(),this.currentState=exports.PlayerState.Paused,this.dispatchEvent(new CustomEvent(exports.PlayerEvents.Pause)))}stop(){this._lottie&&(this._counter=1,this._lottie.stop(),this.currentState=exports.PlayerState.Stopped,this.dispatchEvent(new CustomEvent(exports.PlayerEvents.Stop)))}destroy(){this._lottie&&(this._lottie.destroy(),this._lottie=null,this.currentState=exports.PlayerState.Destroyed,this.dispatchEvent(new CustomEvent(exports.PlayerEvents.Destroyed)),this.remove())}seek(t){if(this._lottie){var e=/^(\d+)(%?)$/.exec(t.toString());if(e){var r="%"===e[2]?this._lottie.totalFrames*Number(e[1])/100:Number(e[1]);this.seeker=r,this.currentState===exports.PlayerState.Playing?this._lottie.goToAndPlay(r,!0):(this._lottie.goToAndStop(r,!0),this._lottie.pause())}}}snapshot(){var t=!(arguments.length>0&&void 0!==arguments[0])||arguments[0];if(this.shadowRoot){var e=this.shadowRoot.querySelector(".animation svg"),r=(new XMLSerializer).serializeToString(e);if(t){var i=document.createElement("a");i.href="data:image/svg+xml;charset=utf-8,".concat(encodeURIComponent(r)),i.download="download_".concat(this.seeker,".svg"),document.body.appendChild(i),i.click(),document.body.removeChild(i)}return r}}setSpeed(){var t=arguments.length>0&&void 0!==arguments[0]?arguments[0]:1;this._lottie&&this._lottie.setSpeed(t)}setDirection(t){this._lottie&&this._lottie.setDirection(t)}setLooping(t){this._lottie&&(this.loop=t,this._lottie.loop=t)}togglePlay(){return this.currentState===exports.PlayerState.Playing?this.pause():this.play()}toggleLooping(){this.setLooping(!this.loop)}resize(){this._lottie&&this._lottie.resize()}static get styles(){return styles}disconnectedCallback(){this.isConnected||(this._io&&(this._io.disconnect(),this._io=void 0),document.removeEventListener("visibilitychange",(()=>this._onVisibilityChange())),this.destroy())}render(){var t=this.controls?"main controls":"main",e=this.controls?"animation controls":"animation";return $(_templateObject||(_templateObject=_taggedTemplateLiteral([' \n \n ',"\n \n ","\n "])),t,this.description,e,this.background,this.currentState===exports.PlayerState.Error?$(_templateObject2||(_templateObject2=_taggedTemplateLiteral(['

']))):void 0,this.controls&&!this.disableShadowDOM?this.renderControls():void 0)}createRenderRoot(){return this.disableShadowDOM&&(this.style.display="block"),this.disableShadowDOM?this:super.createRenderRoot()}firstUpdated(){"IntersectionObserver"in window&&(this._io=new IntersectionObserver((t=>{t[0].isIntersecting?this.currentState===exports.PlayerState.Frozen&&this.play():this.currentState===exports.PlayerState.Playing&&this.freeze()})),this._io.observe(this.container)),void 0!==document.hidden&&document.addEventListener("visibilitychange",(()=>this._onVisibilityChange())),this.src&&this.load(this.src),this.dispatchEvent(new CustomEvent(exports.PlayerEvents.Rendered))}renderControls(){var t=this.currentState===exports.PlayerState.Playing,e=this.currentState===exports.PlayerState.Paused,r=this.currentState===exports.PlayerState.Stopped;return $(_templateObject3||(_templateObject3=_taggedTemplateLiteral(['\n \n \n ','\n \n \n \n \n \n \n \n \n \n '])),this.togglePlay,t||e?"active":"",$(t?_templateObject4||(_templateObject4=_taggedTemplateLiteral(['Apache License, Version 2.0.

site_name: Apache Iceberg theme: @@ -51,6 +49,7 @@ plugins: - search - macros - monorepo + - privacy - offline: enabled: !ENV [OFFLINE, false] diff --git a/site/nav.yml b/site/nav.yml index a12fd89724e2..c7d9a26ae0e9 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -41,8 +41,9 @@ nav: - How to release: how-to-release.md - Terms: terms.md - ASF: - - Sponsorship: https://www.apache.org/foundation/sponsorship.html + - 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 - License: https://www.apache.org/licenses/ - Security: https://www.apache.org/security/ - Sponsors: https://www.apache.org/foundation/thanks.html diff --git a/site/overrides/home.html b/site/overrides/home.html index a79c44c746c2..f6479d46ea9b 100644 --- a/site/overrides/home.html +++ b/site/overrides/home.html @@ -4,7 +4,7 @@ {{ super() }} @@ -215,7 +215,7 @@

Hidden Partitioning

- + Data Compaction {% endblock %} @@ -315,7 +315,3 @@

Data Compaction

{% block content %} {{ super() }} {% endblock %} - -{% block footer%} - {% include "partials/cto.html" %} -{% endblock %} diff --git a/site/overrides/partials/cto.html b/site/overrides/partials/cto.html deleted file mode 100644 index a64e2533e5e8..000000000000 --- a/site/overrides/partials/cto.html +++ /dev/null @@ -1,21 +0,0 @@ -
-
- Apache Iceberg, Iceberg, Apache, the Apache feather logo, and the Apache Iceberg project logo - are
either registered trademarks or trademarks of The Apache Software Foundation. Copyright © - 2023
The Apache Software Foundation, Licensed under the Apache License, Version 2.0.

-
- -
-
diff --git a/site/overrides/partials/footer.html b/site/overrides/partials/footer.html index 9567e202f4ae..adb4acc0b0b2 100644 --- a/site/overrides/partials/footer.html +++ b/site/overrides/partials/footer.html @@ -1,22 +1,132 @@ - From f9b5d31846f93f95a70a9ac522dadce470cc7e65 Mon Sep 17 00:00:00 2001 From: big face cat <731030576@qq.com> Date: Tue, 5 Mar 2024 10:29:50 +0800 Subject: [PATCH 0096/1019] Flink: Supports specifying comment for iceberg fields in create table and addcolumn syntax using flinksql (#9606) Co-authored-by: huyuanfeng --- .../apache/iceberg/flink/FlinkCatalog.java | 10 +-- .../flink/FlinkDynamicTableFactory.java | 23 +++--- .../apache/iceberg/flink/FlinkSchemaUtil.java | 77 ++++++++++++++----- .../flink/util/FlinkAlterTableUtil.java | 6 +- .../iceberg/flink/TestFlinkCatalogTable.java | 18 ++++- 5 files changed, 95 insertions(+), 39 deletions(-) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index f022c8abcb00..86295d78cc13 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -38,6 +38,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; @@ -390,17 +391,16 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + "create table without 'connector'='iceberg' related properties in an iceberg table."); } - - createIcebergTable(tablePath, table, ignoreIfExists); + Preconditions.checkArgument(table instanceof ResolvedCatalogTable, "table should be resolved"); + createIcebergTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists); } - void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolean ignoreIfExists) throws CatalogException, TableAlreadyExistException { validateFlinkTable(table); - Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema()); + Schema icebergSchema = FlinkSchemaUtil.convert(table.getResolvedSchema()); PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); - ImmutableMap.Builder properties = ImmutableMap.builder(); String location = null; for (Map.Entry entry : table.getOptions().entrySet()) { diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java index 8e1f420b722d..b7f1be4b93fb 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -24,11 +24,10 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -84,9 +83,9 @@ public FlinkDynamicTableFactory(FlinkCatalog catalog) { @Override public DynamicTableSource createDynamicTableSource(Context context) { ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map tableProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map tableProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { @@ -94,7 +93,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { } else { tableLoader = createTableLoader( - catalogTable, + resolvedCatalogTable, tableProps, objectIdentifier.getDatabaseName(), objectIdentifier.getObjectName()); @@ -106,9 +105,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map writeProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map writeProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { @@ -116,7 +115,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { } else { tableLoader = createTableLoader( - catalogTable, + resolvedCatalogTable, writeProps, objectIdentifier.getDatabaseName(), objectIdentifier.getObjectName()); @@ -147,7 +146,7 @@ public String factoryIdentifier() { } private static TableLoader createTableLoader( - CatalogBaseTable catalogBaseTable, + ResolvedCatalogTable resolvedCatalogTable, Map tableProps, String databaseName, String tableName) { @@ -187,7 +186,7 @@ private static TableLoader createTableLoader( // Create table if not exists in the external catalog. if (!flinkCatalog.tableExists(objectPath)) { try { - flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true); + flinkCatalog.createIcebergTable(objectPath, resolvedCatalogTable, true); } catch (TableAlreadyExistException e) { throw new AlreadyExistsException( e, diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index a6b53879ad80..4790dc85bf28 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -20,7 +20,10 @@ import java.util.List; import java.util.Set; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.utils.TypeConversions; @@ -55,35 +58,69 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** Convert the flink table schema to apache iceberg schema. */ + /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); Preconditions.checkArgument( - schemaType instanceof RowType, "Schema logical type should be RowType."); + schemaType instanceof RowType, "Schema logical type should be row type."); RowType root = (RowType) schemaType; Type converted = root.accept(new FlinkTypeToType(root)); - Schema iSchema = new Schema(converted.asStructType().fields()); - return freshIdentifierFieldIds(iSchema, schema); + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (schema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, schema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } + } + + /** Convert the flink table schema to apache iceberg schema with column comment. */ + public static Schema convert(ResolvedSchema flinkSchema) { + List tableColumns = flinkSchema.getColumns(); + // copy from org.apache.flink.table.api.Schema#toRowDataType + DataTypes.Field[] fields = + tableColumns.stream() + .map( + column -> { + if (column.getComment().isPresent()) { + return DataTypes.FIELD( + column.getName(), column.getDataType(), column.getComment().get()); + } else { + return DataTypes.FIELD(column.getName(), column.getDataType()); + } + }) + .toArray(DataTypes.Field[]::new); + + LogicalType schemaType = DataTypes.ROW(fields).notNull().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be row type."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } } - private static Schema freshIdentifierFieldIds(Schema iSchema, TableSchema schema) { + private static Schema freshIdentifierFieldIds(Schema icebergSchema, List primaryKeys) { // Locate the identifier field id list. Set identifierFieldIds = Sets.newHashSet(); - if (schema.getPrimaryKey().isPresent()) { - for (String column : schema.getPrimaryKey().get().getColumns()) { - Types.NestedField field = iSchema.findField(column); - Preconditions.checkNotNull( - field, - "Cannot find field ID for the primary key column %s in schema %s", - column, - iSchema); - identifierFieldIds.add(field.fieldId()); - } + for (String primaryKey : primaryKeys) { + Types.NestedField field = icebergSchema.findField(primaryKey); + Preconditions.checkNotNull( + field, + "Cannot find field ID for the primary key column %s in schema %s", + primaryKey, + icebergSchema); + identifierFieldIds.add(field.fieldId()); } - - return new Schema(iSchema.schemaId(), iSchema.asStruct().fields(), identifierFieldIds); + return new Schema( + icebergSchema.schemaId(), icebergSchema.asStruct().fields(), identifierFieldIds); } /** @@ -109,7 +146,11 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { // fix types that can't be represented in Flink (UUID) Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); - return freshIdentifierFieldIds(fixedSchema, flinkSchema); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(fixedSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return fixedSchema; + } } /** diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java index f0b9bf64fb1a..2bbc9cf208fe 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java @@ -132,9 +132,11 @@ public static void applySchemaChanges( flinkColumn.getName()); Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); if (flinkColumn.getDataType().getLogicalType().isNullable()) { - pendingUpdate.addColumn(flinkColumn.getName(), icebergType); + pendingUpdate.addColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); } else { - pendingUpdate.addRequiredColumn(flinkColumn.getName(), icebergType); + pendingUpdate.addRequiredColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); } } else if (change instanceof TableChange.ModifyColumn) { TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index ef0802d8693d..eaa92e32c49d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -226,6 +226,19 @@ public void testCreatePartitionTable() throws TableNotExistException { assertThat(catalogTable.getPartitionKeys()).isEqualTo(Collections.singletonList("dt")); } + @TestTemplate + public void testCreateTableWithColumnComment() { + sql("CREATE TABLE tl(id BIGINT COMMENT 'comment - id', data STRING COMMENT 'comment - data')"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get(), "comment - id"), + Types.NestedField.optional(2, "data", Types.StringType.get(), "comment - data")) + .asStruct()); + } + @TestTemplate public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); @@ -316,14 +329,15 @@ public void testAlterTableAddColumn() { Types.NestedField.optional(2, "dt", Types.StringType.get())) .asStruct()); // Add multiple columns - sql("ALTER TABLE tl ADD (col1 STRING, col2 BIGINT)"); + sql("ALTER TABLE tl ADD (col1 STRING COMMENT 'comment for col1', col2 BIGINT)"); Schema schemaAfter2 = table("tl").schema(); assertThat(schemaAfter2.asStruct()) .isEqualTo( new Schema( Types.NestedField.optional(1, "id", Types.LongType.get()), Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional( + 3, "col1", Types.StringType.get(), "comment for col1"), Types.NestedField.optional(4, "col2", Types.LongType.get())) .asStruct()); // Adding a required field should fail because Iceberg's SchemaUpdate does not allow From cd7fa7320b7628a8940f23082cd986017891d859 Mon Sep 17 00:00:00 2001 From: big face cat <731030576@qq.com> Date: Tue, 5 Mar 2024 23:32:50 +0800 Subject: [PATCH 0097/1019] Flink:backport PR to 1.17 #9606 : Supports specifying comment for iceberg fields in create table and addcolumn syntax using flinksql (#9868) Co-authored-by: huyuanfeng --- .../apache/iceberg/flink/FlinkCatalog.java | 9 ++- .../flink/FlinkDynamicTableFactory.java | 23 +++--- .../apache/iceberg/flink/FlinkSchemaUtil.java | 77 ++++++++++++++----- .../flink/util/FlinkAlterTableUtil.java | 6 +- .../iceberg/flink/TestFlinkCatalogTable.java | 18 ++++- 5 files changed, 95 insertions(+), 38 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java index f022c8abcb00..988465a38f4d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -38,6 +38,7 @@ import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.CatalogTableImpl; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.TableChange; import org.apache.flink.table.catalog.exceptions.CatalogException; import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; @@ -390,15 +391,15 @@ public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ig + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + "create table without 'connector'='iceberg' related properties in an iceberg table."); } - - createIcebergTable(tablePath, table, ignoreIfExists); + Preconditions.checkArgument(table instanceof ResolvedCatalogTable, "table should be resolved"); + createIcebergTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists); } - void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolean ignoreIfExists) throws CatalogException, TableAlreadyExistException { validateFlinkTable(table); - Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema()); + Schema icebergSchema = FlinkSchemaUtil.convert(table.getResolvedSchema()); PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); ImmutableMap.Builder properties = ImmutableMap.builder(); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java index 8e1f420b722d..b7f1be4b93fb 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -24,11 +24,10 @@ import org.apache.flink.configuration.ConfigOptions; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogBaseTable; import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogTable; import org.apache.flink.table.catalog.ObjectIdentifier; import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; import org.apache.flink.table.connector.sink.DynamicTableSink; @@ -84,9 +83,9 @@ public FlinkDynamicTableFactory(FlinkCatalog catalog) { @Override public DynamicTableSource createDynamicTableSource(Context context) { ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map tableProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map tableProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { @@ -94,7 +93,7 @@ public DynamicTableSource createDynamicTableSource(Context context) { } else { tableLoader = createTableLoader( - catalogTable, + resolvedCatalogTable, tableProps, objectIdentifier.getDatabaseName(), objectIdentifier.getObjectName()); @@ -106,9 +105,9 @@ public DynamicTableSource createDynamicTableSource(Context context) { @Override public DynamicTableSink createDynamicTableSink(Context context) { ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map writeProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map writeProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); TableLoader tableLoader; if (catalog != null) { @@ -116,7 +115,7 @@ public DynamicTableSink createDynamicTableSink(Context context) { } else { tableLoader = createTableLoader( - catalogTable, + resolvedCatalogTable, writeProps, objectIdentifier.getDatabaseName(), objectIdentifier.getObjectName()); @@ -147,7 +146,7 @@ public String factoryIdentifier() { } private static TableLoader createTableLoader( - CatalogBaseTable catalogBaseTable, + ResolvedCatalogTable resolvedCatalogTable, Map tableProps, String databaseName, String tableName) { @@ -187,7 +186,7 @@ private static TableLoader createTableLoader( // Create table if not exists in the external catalog. if (!flinkCatalog.tableExists(objectPath)) { try { - flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true); + flinkCatalog.createIcebergTable(objectPath, resolvedCatalogTable, true); } catch (TableAlreadyExistException e) { throw new AlreadyExistsException( e, diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index a6b53879ad80..4790dc85bf28 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -20,7 +20,10 @@ import java.util.List; import java.util.Set; +import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; import org.apache.flink.table.types.logical.LogicalType; import org.apache.flink.table.types.logical.RowType; import org.apache.flink.table.types.utils.TypeConversions; @@ -55,35 +58,69 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** Convert the flink table schema to apache iceberg schema. */ + /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); Preconditions.checkArgument( - schemaType instanceof RowType, "Schema logical type should be RowType."); + schemaType instanceof RowType, "Schema logical type should be row type."); RowType root = (RowType) schemaType; Type converted = root.accept(new FlinkTypeToType(root)); - Schema iSchema = new Schema(converted.asStructType().fields()); - return freshIdentifierFieldIds(iSchema, schema); + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (schema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, schema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } + } + + /** Convert the flink table schema to apache iceberg schema with column comment. */ + public static Schema convert(ResolvedSchema flinkSchema) { + List tableColumns = flinkSchema.getColumns(); + // copy from org.apache.flink.table.api.Schema#toRowDataType + DataTypes.Field[] fields = + tableColumns.stream() + .map( + column -> { + if (column.getComment().isPresent()) { + return DataTypes.FIELD( + column.getName(), column.getDataType(), column.getComment().get()); + } else { + return DataTypes.FIELD(column.getName(), column.getDataType()); + } + }) + .toArray(DataTypes.Field[]::new); + + LogicalType schemaType = DataTypes.ROW(fields).notNull().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be row type."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } } - private static Schema freshIdentifierFieldIds(Schema iSchema, TableSchema schema) { + private static Schema freshIdentifierFieldIds(Schema icebergSchema, List primaryKeys) { // Locate the identifier field id list. Set identifierFieldIds = Sets.newHashSet(); - if (schema.getPrimaryKey().isPresent()) { - for (String column : schema.getPrimaryKey().get().getColumns()) { - Types.NestedField field = iSchema.findField(column); - Preconditions.checkNotNull( - field, - "Cannot find field ID for the primary key column %s in schema %s", - column, - iSchema); - identifierFieldIds.add(field.fieldId()); - } + for (String primaryKey : primaryKeys) { + Types.NestedField field = icebergSchema.findField(primaryKey); + Preconditions.checkNotNull( + field, + "Cannot find field ID for the primary key column %s in schema %s", + primaryKey, + icebergSchema); + identifierFieldIds.add(field.fieldId()); } - - return new Schema(iSchema.schemaId(), iSchema.asStruct().fields(), identifierFieldIds); + return new Schema( + icebergSchema.schemaId(), icebergSchema.asStruct().fields(), identifierFieldIds); } /** @@ -109,7 +146,11 @@ public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { // fix types that can't be represented in Flink (UUID) Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); - return freshIdentifierFieldIds(fixedSchema, flinkSchema); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(fixedSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return fixedSchema; + } } /** diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java index f0b9bf64fb1a..2bbc9cf208fe 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java @@ -132,9 +132,11 @@ public static void applySchemaChanges( flinkColumn.getName()); Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); if (flinkColumn.getDataType().getLogicalType().isNullable()) { - pendingUpdate.addColumn(flinkColumn.getName(), icebergType); + pendingUpdate.addColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); } else { - pendingUpdate.addRequiredColumn(flinkColumn.getName(), icebergType); + pendingUpdate.addRequiredColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); } } else if (change instanceof TableChange.ModifyColumn) { TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index ef0802d8693d..eaa92e32c49d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -226,6 +226,19 @@ public void testCreatePartitionTable() throws TableNotExistException { assertThat(catalogTable.getPartitionKeys()).isEqualTo(Collections.singletonList("dt")); } + @TestTemplate + public void testCreateTableWithColumnComment() { + sql("CREATE TABLE tl(id BIGINT COMMENT 'comment - id', data STRING COMMENT 'comment - data')"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get(), "comment - id"), + Types.NestedField.optional(2, "data", Types.StringType.get(), "comment - data")) + .asStruct()); + } + @TestTemplate public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); @@ -316,14 +329,15 @@ public void testAlterTableAddColumn() { Types.NestedField.optional(2, "dt", Types.StringType.get())) .asStruct()); // Add multiple columns - sql("ALTER TABLE tl ADD (col1 STRING, col2 BIGINT)"); + sql("ALTER TABLE tl ADD (col1 STRING COMMENT 'comment for col1', col2 BIGINT)"); Schema schemaAfter2 = table("tl").schema(); assertThat(schemaAfter2.asStruct()) .isEqualTo( new Schema( Types.NestedField.optional(1, "id", Types.LongType.get()), Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional( + 3, "col1", Types.StringType.get(), "comment for col1"), Types.NestedField.optional(4, "col2", Types.LongType.get())) .asStruct()); // Adding a required field should fail because Iceberg's SchemaUpdate does not allow From 9536f34b368e876d517739628fb838ac0421344d Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Mar 2024 19:30:48 +0100 Subject: [PATCH 0098/1019] Build: Don't publish iceberg-open-api module (#9871) --- deploy.gradle | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/deploy.gradle b/deploy.gradle index ad4dbb5263b9..3e2eda2a5a60 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -22,6 +22,11 @@ if (project.hasProperty('release') && jdkVersion != '8') { } subprojects { + if (it.name == 'iceberg-open-api') { + // don't publish iceberg-open-api + return + } + def isBom = it.name == 'iceberg-bom' apply plugin: 'maven-publish' From 88e0f1afb77d43cb5e222d206a2aa9a88404fd6e Mon Sep 17 00:00:00 2001 From: "Brian \"bits\" Olsen" Date: Wed, 6 Mar 2024 04:09:22 -0600 Subject: [PATCH 0099/1019] Docs: Update specs from hugo to mkdocs format (#9861) --- format/gcm-stream-spec.md | 3 --- format/puffin-spec.md | 3 --- format/spec.md | 13 +++++-------- format/view-spec.md | 3 --- 4 files changed, 5 insertions(+), 17 deletions(-) diff --git a/format/gcm-stream-spec.md b/format/gcm-stream-spec.md index d51982ce5c92..eb531cf9051d 100644 --- a/format/gcm-stream-spec.md +++ b/format/gcm-stream-spec.md @@ -1,8 +1,5 @@ --- title: "AES GCM Stream Spec" -url: gcm-stream-spec -toc: true -disableSidebar: true --- -## Rewrite files action. +## Rewrite files action -Iceberg provides API to rewrite small files into large files by submitting Flink batch jobs. The behavior of this Flink action is the same as Spark's [rewriteDataFiles](maintenance.md#compact-data-files). +Iceberg provides API to rewrite small files into large files by submitting Flink batch jobs. The behavior of this Flink action is the same as Spark's [rewriteDataFiles](../maintenance.md#compact-data-files). ```java import org.apache.iceberg.flink.actions.Actions; diff --git a/docs/docs/flink-connector.md b/docs/docs/flink-connector.md index 025e9aee92ea..260a5c581493 100644 --- a/docs/docs/flink-connector.md +++ b/docs/docs/flink-connector.md @@ -29,13 +29,13 @@ To create the table in Flink SQL by using SQL syntax `CREATE TABLE test (..) WIT * `connector`: Use the constant `iceberg`. * `catalog-name`: User-specified catalog name. It's required because the connector don't have any default value. * `catalog-type`: `hive` or `hadoop` for built-in catalogs (defaults to `hive`), or left unset for custom catalog implementations using `catalog-impl`. -* `catalog-impl`: The fully-qualified class name of a custom catalog implementation. Must be set if `catalog-type` is unset. See also [custom catalog](flink.md#adding-catalogs) for more details. +* `catalog-impl`: The fully-qualified class name of a custom catalog implementation. Must be set if `catalog-type` is unset. See also [custom catalog](../flink.md#adding-catalogs) for more details. * `catalog-database`: The iceberg database name in the backend catalog, use the current flink database name by default. * `catalog-table`: The iceberg table name in the backend catalog. Default to use the table name in the flink `CREATE TABLE` sentence. ## Table managed in Hive catalog. -Before executing the following SQL, please make sure you've configured the Flink SQL client correctly according to the [quick start documentation](flink.md). +Before executing the following SQL, please make sure you've configured the Flink SQL client correctly according to the [quick start documentation](../flink.md). The following SQL will create a Flink table in the current Flink catalog, which maps to the iceberg table `default_database.flink_table` managed in iceberg catalog. @@ -138,4 +138,4 @@ SELECT * FROM flink_table; 3 rows in set ``` -For more details, please refer to the Iceberg [Flink documentation](flink.md). +For more details, please refer to the Iceberg [Flink documentation](../flink.md). diff --git a/docs/docs/flink-ddl.md b/docs/docs/flink-ddl.md index c2b3051fde8d..681a018865aa 100644 --- a/docs/docs/flink-ddl.md +++ b/docs/docs/flink-ddl.md @@ -150,7 +150,7 @@ Table create commands support the commonly used [Flink create clauses](https://n * `PARTITION BY (column1, column2, ...)` to configure partitioning, Flink does not yet support hidden partitioning. * `COMMENT 'table document'` to set a table description. -* `WITH ('key'='value', ...)` to set [table configuration](configuration.md) which will be stored in Iceberg table properties. +* `WITH ('key'='value', ...)` to set [table configuration](../configuration.md) which will be stored in Iceberg table properties. Currently, it does not support computed column and watermark definition etc. diff --git a/docs/docs/flink-queries.md b/docs/docs/flink-queries.md index 431a5554f248..036d95a4953a 100644 --- a/docs/docs/flink-queries.md +++ b/docs/docs/flink-queries.md @@ -75,7 +75,7 @@ SET table.exec.iceberg.use-flip27-source = true; ### 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) +refer to [Flink Configuration](../flink-configuration.md#read-options) ```sql --- Read from branch b1 diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index 46bc9bb2c605..c41b367deaed 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -59,20 +59,20 @@ Iceberg supports `UPSERT` based on the primary key when writing data into v2 tab 1. Enable the `UPSERT` mode as table-level property `write.upsert.enabled`. Here is an example SQL statement to set the table property when creating a table. It would be applied for all write paths to this table (batch or streaming) unless overwritten by write options as described later. -```sql -CREATE TABLE `hive_catalog`.`default`.`sample` ( - `id` INT COMMENT 'unique id', - `data` STRING NOT NULL, - PRIMARY KEY(`id`) NOT ENFORCED -) with ('format-version'='2', 'write.upsert.enabled'='true'); -``` + ```sql + CREATE TABLE `hive_catalog`.`default`.`sample` ( + `id` INT COMMENT 'unique id', + `data` STRING NOT NULL, + PRIMARY KEY(`id`) NOT ENFORCED + ) with ('format-version'='2', 'write.upsert.enabled'='true'); + ``` -2. Enabling `UPSERT` mode using `upsert-enabled` in the [write options](#write-options) provides more flexibility than a table level config. Note that you still need to use v2 table format and specify the primary key when creating the table. +2. Enabling `UPSERT` mode using `upsert-enabled` in the [write options](#write-options) provides more flexibility than a table level config. Note that you still need to use v2 table format and specify the [primary key](../flink-ddl.md/#primary-key) or [identifier fields](../../spec.md#identifier-field-ids) when creating the table. -```sql -INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ -... -``` + ```sql + INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ + ... + ``` !!! info OVERWRITE and UPSERT can't be set together. In UPSERT mode, if the table is partitioned, the partition fields should be included in equality fields. @@ -85,7 +85,7 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ Iceberg support writing to iceberg table from different DataStream input. -### Appending data. +### Appending data Flink supports writing `DataStream` and `DataStream` to the sink iceberg table natively. @@ -185,7 +185,7 @@ FlinkSink.builderFor( ### Branch Writes Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink` -For more information on branches please refer to [branches](branching.md). +For more information on branches please refer to [branches](../branching.md). ```java FlinkSink.forRowData(input) .tableLoader(tableLoader) @@ -262,7 +262,7 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ ... ``` -Check out all the options here: [write-options](flink-configuration.md#write-options) +Check out all the options here: [write-options](../flink-configuration.md#write-options) ## Notes diff --git a/docs/docs/flink.md b/docs/docs/flink.md index bfad96840b43..7f27a280eb91 100644 --- a/docs/docs/flink.md +++ b/docs/docs/flink.md @@ -271,7 +271,7 @@ env.execute("Test Iceberg DataStream"); ### Branch Writes Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink` -For more information on branches please refer to [branches](branching.md). +For more information on branches please refer to [branches](../branching.md). ```java FlinkSink.forRowData(input) .tableLoader(tableLoader) From a82e957bb3bf12905eb4fb038d9369d45b8b8d03 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 11 Mar 2024 11:55:13 +0100 Subject: [PATCH 0112/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.0.1 to 1.0.5 (#9911) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.0.1 to 1.0.5. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.0.1...1.0.5) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 853dc8c21cec..3d4a027ae247 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -74,7 +74,7 @@ object-client-bundle = "3.3.2" orc = "1.9.2" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.0.1" +roaringbitmap = "1.0.5" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" From 1fd5c13c7b08261f9cc6af916cebe0dd9bcf0299 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 11 Mar 2024 16:28:07 +0530 Subject: [PATCH 0113/1019] Site: Update release notes for 1.5.0 (#9835) * Site: Update release notes for 1.5.0 * Address comments * add view PRs --- site/docs/multi-engine-support.md | 30 ++++++------ site/docs/releases.md | 76 +++++++++++++++++++++++++++++-- 2 files changed, 86 insertions(+), 20 deletions(-) diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index e137bdda1735..b0667361ef5c 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -63,11 +63,11 @@ Each engine version undergoes the following lifecycle stages: | ---------- | ------------------ | ----------------------- |------------------------| ------------------ | | 2.4 | End of Life | 0.7.0-incubating | 1.2.1 | [iceberg-spark-runtime-2.4](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-2.4/1.2.1/iceberg-spark-runtime-2.4-1.2.1.jar) | | 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 | Deprecated | 0.13.0 | {{ icebergVersion }} | [iceberg-spark-runtime-3.2_2.12](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.2_2.12-{{ icebergVersion }}.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.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) | +| 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.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) | * [1] Spark 3.1 shares the same runtime jar `iceberg-spark3-runtime` with Spark 3.0 before Iceberg 0.13.0 @@ -76,16 +76,16 @@ Each engine version undergoes the following lifecycle stages: Based on the guideline of the Flink community, only the latest 2 minor versions are actively maintained. Users should continuously upgrade their Flink version to stay up-to-date. -| Version | Lifecycle Stage | Initial Iceberg Support | Latest Iceberg Support | Latest Runtime Jar | -| ------- | --------------- | ----------------------- | ---------------------- | ------------------------------------------------------------ | -| 1.11 | End of Life | 0.9.0 | 0.12.1 | [iceberg-flink-runtime](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime/0.12.1/iceberg-flink-runtime-0.12.1.jar) | -| 1.12 | End of Life | 0.12.0 | 0.13.1 | [iceberg-flink-runtime-1.12](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.12/0.13.2/iceberg-flink-runtime-1.12-0.13.2.jar) [3] | -| 1.13 | End of Life | 0.13.0 | 1.0.0 | [iceberg-flink-runtime-1.13](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.13/1.2.0/iceberg-flink-runtime-1.13-1.0.0.jar) | -| 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 | {{ icebergVersion }} | [iceberg-flink-runtime-1.15](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.15/{{ icebergVersion }}/iceberg-flink-runtime-1.15-{{ icebergVersion }}.jar) | -| 1.16 | Deprecated | 1.1.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.16](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) | -| 1.17 | Maintained | 1.3.0 | {{ icebergVersion }} | [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.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) | +| Version | Lifecycle Stage | Initial Iceberg Support | Latest Iceberg Support | Latest Runtime Jar | +| ------- | --------------- | ----------------------- |------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| 1.11 | End of Life | 0.9.0 | 0.12.1 | [iceberg-flink-runtime](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime/0.12.1/iceberg-flink-runtime-0.12.1.jar) | +| 1.12 | End of Life | 0.12.0 | 0.13.1 | [iceberg-flink-runtime-1.12](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.12/0.13.2/iceberg-flink-runtime-1.12-0.13.2.jar) [3] | +| 1.13 | End of Life | 0.13.0 | 1.0.0 | [iceberg-flink-runtime-1.13](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.13/1.2.0/iceberg-flink-runtime-1.13-1.0.0.jar) | +| 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 | Deprecated | 1.1.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.16](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) | +| 1.17 | Maintained | 1.3.0 | {{ icebergVersion }} | [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.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) | * [3] Flink 1.12 shares the same runtime jar `iceberg-flink-runtime` with Flink 1.11 before Iceberg 0.13.0 diff --git a/site/docs/releases.md b/site/docs/releases.md index e55c6ed46f3f..85997e6d6a08 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -26,10 +26,9 @@ The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apach * [{{ icebergVersion }} Spark 3.5\_2.12 runtime Jar](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) -- [3.5\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.5_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.5_2.13-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Spark 3.4\_2.12 runtime Jar](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.4\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.4_2.13-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Spark 3.3\_2.12 runtime Jar](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\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.13-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Spark 3.2\_2.12 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.12/{{ icebergVersion }}/iceberg-spark-runtime-3.2_2.12-{{ icebergVersion }}.jar) -- [3.2\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.2_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.2_2.13-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Flink 1.18 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.17 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.16 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Flink 1.15 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.15/{{ icebergVersion }}/iceberg-flink-runtime-1.15-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Hive runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-hive-runtime/{{ icebergVersion }}/iceberg-hive-runtime-{{ icebergVersion }}.jar) * [{{ icebergVersion }} aws-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-aws-bundle/{{ icebergVersion }}/iceberg-aws-bundle-{{ icebergVersion }}.jar) * [{{ icebergVersion }} gcp-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-gcp-bundle/{{ icebergVersion }}/iceberg-gcp-bundle-{{ icebergVersion }}.jar) @@ -67,6 +66,76 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ``` +### 1.5.0 release + +Apache Iceberg 1.5.0 was released on March 11, 2024. +The 1.5.0 release adds a variety of new features and bug fixes. + +* API + - Extend FileIO and add EncryptingFileIO. ([\#9592](https://github.com/apache/iceberg/pull/9592)) + - Track partition statistics in TableMetadata ([\#8502](https://github.com/apache/iceberg/pull/8502)) + - Add sqlFor API to views to handle resolving a representation for a dialect([\#9247](https://github.com/apache/iceberg/pull/9247)) +* Core + - Add view support for REST catalog ([\#7913](https://github.com/apache/iceberg/pull/7913)) + - Add view support for JDBC catalog ([\#9487](https://github.com/apache/iceberg/pull/9487)) + - Add catalog type for glue,jdbc,nessie ([\#9647](https://github.com/apache/iceberg/pull/9647)) + - Support Avro file encryption with AES GCM streams ([\#9436](https://github.com/apache/iceberg/pull/9436)) + - Add ApplyNameMapping for Avro ([\#9347](https://github.com/apache/iceberg/pull/9347)) + - Add StandardEncryptionManager ([\#9277](https://github.com/apache/iceberg/pull/9277)) + - Add REST catalog table session cache ([\#8920](https://github.com/apache/iceberg/pull/8920)) + - Support view metadata compression ([\#8552](https://github.com/apache/iceberg/pull/8552)) + - Track partition statistics in TableMetadata ([\#8502](https://github.com/apache/iceberg/pull/8502)) + - Enable column statistics filtering after planning ([\#8803](https://github.com/apache/iceberg/pull/8803)) +* Spark + - Remove support for Spark 3.2 ([\#9295](https://github.com/apache/iceberg/pull/9295)) + - Support views via SQL for Spark 3.4 and 3.5 ([\#9423](https://github.com/apache/iceberg/pull/9423), [\#9421](https://github.com/apache/iceberg/pull/9421), [\#9343](https://github.com/apache/iceberg/pull/9343)), ([\#9513](https://github.com/apache/iceberg/pull/9513), ([\#9582](https://github.com/apache/iceberg/pull/9582) + - Support executor cache locality ([\#9563](https://github.com/apache/iceberg/pull/9563)) + - Added support for delete manifest rewrites ([\#9020](https://github.com/apache/iceberg/pull/9020)) + - Support encrypted output files ([\#9435](https://github.com/apache/iceberg/pull/9435)) + - Add Spark UI metrics from Iceberg scan metrics ([\#8717](https://github.com/apache/iceberg/pull/8717)) + - Parallelize reading files in add_files procedure ([\#9274](https://github.com/apache/iceberg/pull/9274)) + - Support file and partition delete granularity ([\#9384](https://github.com/apache/iceberg/pull/9384)) +* Flink + - Remove Flink 1.15 + - Adds support for 1.18 version [\#9211](https://github.com/apache/iceberg/pull/9211) + - Emit watermarks from the IcebergSource ([\#8553](https://github.com/apache/iceberg/pull/8553)) + - Watermark read options ([\#9346](https://github.com/apache/iceberg/pull/9346)) +* Parquet + - Support reading INT96 column in row group filter ([\#8988](https://github.com/apache/iceberg/pull/8988)) + - Add system config for unsafe Parquet ID fallback. ([\#9324](https://github.com/apache/iceberg/pull/9324)) +* Kafka-Connect + - Initial project setup and event data structures ([\#8701](https://github.com/apache/iceberg/pull/8701)) + - Sink connector with data writers and converters ([\#9466](https://github.com/apache/iceberg/pull/9466)) +* Spec + - Add partition stats spec ([\#7105](https://github.com/apache/iceberg/pull/7105)) + - add nanosecond timestamp types ([\#8683](https://github.com/apache/iceberg/pull/8683)) + - Add multi-arg transform ([\#8579](https://github.com/apache/iceberg/pull/8579)) +* Vendor Integrations + - AWS: Support setting description for Glue table ([\#9530](https://github.com/apache/iceberg/pull/9530)) + - AWS: Update S3FileIO test to run when CLIENT_FACTORY is not set ([\#9541](https://github.com/apache/iceberg/pull/9541)) + - AWS: Add S3 Access Grants Integration ([\#9385](https://github.com/apache/iceberg/pull/9385)) + - AWS: Glue catalog strip trailing slash on DB URI ([\#8870](https://github.com/apache/iceberg/pull/8870)) + - Azure: Add FileIO that supports ADLSv2 storage ([\#8303](https://github.com/apache/iceberg/pull/8303)) + - Azure: Make ADLSFileIO implement DelegateFileIO ([\#8563](https://github.com/apache/iceberg/pull/8563)) + - Nessie: Support views for NessieCatalog ([\#8909](https://github.com/apache/iceberg/pull/8909)) + - Nessie: Strip trailing slash for warehouse location ([\#9415](https://github.com/apache/iceberg/pull/9415)) + - Nessie: Infer default API version from URI ([\#9459](https://github.com/apache/iceberg/pull/9459)) +* Dependencies + - Bump Nessie to 0.77.1 + - Bump ORC to 1.9.2 + - Bump Arrow to 15.0.0 + - Bump AWS Java SDK to 2.24.5 + - Bump Azure Java SDK to 1.2.20 + - Bump Google cloud libraries to 26.28.0 + +Note: +1. To enable view support for JDBC catalog, configure `jdbc.schema-version` to `V1` in catalog properties. + +For more details, please visit Github. +https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.5.0 + +## Past releases + ### 1.4.3 Release Apache Iceberg 1.4.3 was released on December 27, 2023. The main issue it solves is missing files from a transaction retry with conflicting manifests. It is recommended to upgrade if you use transactions. @@ -77,9 +146,6 @@ Apache Iceberg 1.4.3 was released on December 27, 2023. The main issue it solves - Core: Expired Snapshot files in a transaction should be deleted by [@bartash](https://github.com/bartash) in [#9223](https://github.com/apache/iceberg/pull/9223) - Core: Fix missing delete files from transaction by [@nastra](https://github.com/nastra) in [#9356](https://github.com/apache/iceberg/pull/9356) - -## Past releases - ### 1.4.2 Release Apache Iceberg 1.4.2 was released on November 2, 2023. From 2a9a630815479cd0752a2719a797d91c3b828d5c Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Mon, 11 Mar 2024 22:53:24 +0900 Subject: [PATCH 0114/1019] Core: Migrate tests to JUnit5 (#9892) --- .../apache/iceberg/LocalTableOperations.java | 10 +- .../java/org/apache/iceberg/TestSnapshot.java | 152 +++---- .../org/apache/iceberg/TestSnapshotJson.java | 92 ++--- .../apache/iceberg/TestSnapshotLoading.java | 43 +- .../apache/iceberg/TestSnapshotManager.java | 381 +++++++++--------- .../apache/iceberg/TestSnapshotRefParser.java | 67 +-- .../apache/iceberg/TestSnapshotSelection.java | 42 +- .../apache/iceberg/TestSnapshotSummary.java | 62 +-- .../org/apache/iceberg/TestTableMetadata.java | 9 +- 9 files changed, 412 insertions(+), 446 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/LocalTableOperations.java b/core/src/test/java/org/apache/iceberg/LocalTableOperations.java index 27767801cd4e..699ef05c1e62 100644 --- a/core/src/test/java/org/apache/iceberg/LocalTableOperations.java +++ b/core/src/test/java/org/apache/iceberg/LocalTableOperations.java @@ -18,21 +18,23 @@ */ package org.apache.iceberg; +import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Map; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.rules.TemporaryFolder; class LocalTableOperations implements TableOperations { - private final TemporaryFolder temp; + private final Path temp; + private final FileIO io; private final Map createdMetadataFilePaths = Maps.newHashMap(); - LocalTableOperations(TemporaryFolder temp) { + LocalTableOperations(Path temp) { this.temp = temp; this.io = new TestTables.LocalFileIO(); } @@ -63,7 +65,7 @@ public String metadataFileLocation(String fileName) { fileName, name -> { try { - return temp.newFile(name).getAbsolutePath(); + return File.createTempFile("junit", null, temp.toFile()).getAbsolutePath(); } catch (IOException e) { throw new RuntimeIOException(e); } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index 11b7c4a7cce4..2ec6abd4e428 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -18,27 +18,25 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSnapshot extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestSnapshot(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshot extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testAppendFilesFromTable() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -58,7 +56,7 @@ public void testAppendFilesFromTable() { validateSnapshot(oldSnapshot, newSnapshot, FILE_A, FILE_B); } - @Test + @TestTemplate public void testAppendFoundFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -81,7 +79,7 @@ public void testAppendFoundFiles() { validateSnapshot(oldSnapshot, newSnapshot, FILE_A, FILE_B); } - @Test + @TestTemplate public void testCachedDataFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -98,27 +96,25 @@ public void testCachedDataFiles() { Snapshot thirdSnapshot = table.currentSnapshot(); Iterable removedDataFiles = thirdSnapshot.removedDataFiles(FILE_IO); - Assert.assertEquals("Must have 1 removed data file", 1, Iterables.size(removedDataFiles)); + assertThat(removedDataFiles).as("Must have 1 removed data file").hasSize(1); DataFile removedDataFile = Iterables.getOnlyElement(removedDataFiles); - Assert.assertEquals("Path must match", FILE_A.path(), removedDataFile.path()); - Assert.assertEquals("Spec ID must match", FILE_A.specId(), removedDataFile.specId()); - Assert.assertEquals("Partition must match", FILE_A.partition(), removedDataFile.partition()); + assertThat(removedDataFile.path()).isEqualTo(FILE_A.path()); + assertThat(removedDataFile.specId()).isEqualTo(FILE_A.specId()); + assertThat(removedDataFile.partition()).isEqualTo(FILE_A.partition()); Iterable addedDataFiles = thirdSnapshot.addedDataFiles(FILE_IO); - Assert.assertEquals("Must have 1 added data file", 1, Iterables.size(addedDataFiles)); + assertThat(addedDataFiles).as("Must have 1 added data file").hasSize(1); DataFile addedDataFile = Iterables.getOnlyElement(addedDataFiles); - Assert.assertEquals("Path must match", thirdSnapshotDataFile.path(), addedDataFile.path()); - Assert.assertEquals( - "Spec ID must match", thirdSnapshotDataFile.specId(), addedDataFile.specId()); - Assert.assertEquals( - "Partition must match", thirdSnapshotDataFile.partition(), addedDataFile.partition()); + assertThat(addedDataFile.path()).isEqualTo(thirdSnapshotDataFile.path()); + assertThat(addedDataFile.specId()).isEqualTo(thirdSnapshotDataFile.specId()); + assertThat(addedDataFile.partition()).isEqualTo(thirdSnapshotDataFile.partition()); } - @Test + @TestTemplate public void testCachedDeleteFiles() { - Assume.assumeTrue("Delete files only supported in V2", formatVersion >= 2); + assumeThat(formatVersion).as("Delete files only supported in V2").isGreaterThanOrEqualTo(2); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -148,30 +144,23 @@ public void testCachedDeleteFiles() { Snapshot thirdSnapshot = table.currentSnapshot(); Iterable removedDeleteFiles = thirdSnapshot.removedDeleteFiles(FILE_IO); - Assert.assertEquals("Must have 1 removed delete file", 1, Iterables.size(removedDeleteFiles)); + assertThat(removedDeleteFiles).as("Must have 1 removed delete file").hasSize(1); DeleteFile removedDeleteFile = Iterables.getOnlyElement(removedDeleteFiles); - Assert.assertEquals( - "Path must match", secondSnapshotDeleteFile.path(), removedDeleteFile.path()); - Assert.assertEquals( - "Spec ID must match", secondSnapshotDeleteFile.specId(), removedDeleteFile.specId()); - Assert.assertEquals( - "Partition must match", - secondSnapshotDeleteFile.partition(), - removedDeleteFile.partition()); + assertThat(removedDeleteFile.path()).isEqualTo(secondSnapshotDeleteFile.path()); + assertThat(removedDeleteFile.specId()).isEqualTo(secondSnapshotDeleteFile.specId()); + assertThat(removedDeleteFile.partition()).isEqualTo(secondSnapshotDeleteFile.partition()); Iterable addedDeleteFiles = thirdSnapshot.addedDeleteFiles(FILE_IO); - Assert.assertEquals("Must have 1 added delete file", 1, Iterables.size(addedDeleteFiles)); + assertThat(addedDeleteFiles).as("Must have 1 added delete file").hasSize(1); DeleteFile addedDeleteFile = Iterables.getOnlyElement(addedDeleteFiles); - Assert.assertEquals("Path must match", thirdSnapshotDeleteFile.path(), addedDeleteFile.path()); - Assert.assertEquals( - "Spec ID must match", thirdSnapshotDeleteFile.specId(), addedDeleteFile.specId()); - Assert.assertEquals( - "Partition must match", thirdSnapshotDeleteFile.partition(), addedDeleteFile.partition()); + assertThat(addedDeleteFile.path()).isEqualTo(thirdSnapshotDeleteFile.path()); + assertThat(addedDeleteFile.specId()).isEqualTo(thirdSnapshotDeleteFile.specId()); + assertThat(addedDeleteFile.partition()).isEqualTo(thirdSnapshotDeleteFile.partition()); } - @Test + @TestTemplate public void testSequenceNumbersInAddedDataFiles() { long expectedSequenceNumber = 0L; if (formatVersion >= 2) { @@ -193,22 +182,21 @@ private void runAddedDataFileSequenceNumberTest(long expectedSequenceNumber) { Snapshot snapshot = table.currentSnapshot(); Iterable addedDataFiles = snapshot.addedDataFiles(table.io()); - Assert.assertEquals( - "Sequence number mismatch in Snapshot", expectedSequenceNumber, snapshot.sequenceNumber()); + assertThat(snapshot.sequenceNumber()) + .as("Sequence number mismatch in Snapshot") + .isEqualTo(expectedSequenceNumber); for (DataFile df : addedDataFiles) { - Assert.assertEquals( - "Data sequence number mismatch", - expectedSequenceNumber, - df.dataSequenceNumber().longValue()); - Assert.assertEquals( - "File sequence number mismatch", - expectedSequenceNumber, - df.fileSequenceNumber().longValue()); + assertThat(df.dataSequenceNumber().longValue()) + .as("Data sequence number mismatch") + .isEqualTo(expectedSequenceNumber); + assertThat(df.fileSequenceNumber().longValue()) + .as("File sequence number mismatch") + .isEqualTo(expectedSequenceNumber); } } - @Test + @TestTemplate public void testSequenceNumbersInRemovedDataFiles() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -239,28 +227,24 @@ private void runRemovedDataFileSequenceNumberTest( Snapshot snapshot = table.currentSnapshot(); Iterable removedDataFiles = snapshot.removedDataFiles(table.io()); - Assert.assertEquals("Must have 1 removed data file", 1, Iterables.size(removedDataFiles)); + assertThat(removedDataFiles).as("Must have 1 removed data file").hasSize(1); DataFile removedDataFile = Iterables.getOnlyElement(removedDataFiles); - Assert.assertEquals( - "Sequence number mismatch in Snapshot", - expectedSnapshotSequenceNumber, - snapshot.sequenceNumber()); - - Assert.assertEquals( - "Data sequence number mismatch", - expectedFileSequenceNumber, - removedDataFile.dataSequenceNumber().longValue()); - Assert.assertEquals( - "File sequence number mismatch", - expectedFileSequenceNumber, - removedDataFile.fileSequenceNumber().longValue()); + assertThat(snapshot.sequenceNumber()) + .as("Sequence number mismatch in Snapshot") + .isEqualTo(expectedSnapshotSequenceNumber); + assertThat(removedDataFile.dataSequenceNumber().longValue()) + .as("Data sequence number mismatch") + .isEqualTo(expectedFileSequenceNumber); + assertThat(removedDataFile.fileSequenceNumber().longValue()) + .as("File sequence number mismatch") + .isEqualTo(expectedFileSequenceNumber); } - @Test + @TestTemplate public void testSequenceNumbersInAddedDeleteFiles() { - Assume.assumeTrue("Delete files only supported in V2", formatVersion >= 2); + assumeThat(formatVersion).as("Delete files only supported in V2").isGreaterThanOrEqualTo(2); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -277,20 +261,18 @@ private void runAddedDeleteFileSequenceNumberTest( Snapshot snapshot = table.currentSnapshot(); Iterable addedDeleteFiles = snapshot.addedDeleteFiles(table.io()); - Assert.assertEquals("Must have 1 added delete file", 1, Iterables.size(addedDeleteFiles)); + assertThat(addedDeleteFiles).as("Must have 1 added delete file").hasSize(1); DeleteFile addedDeleteFile = Iterables.getOnlyElement(addedDeleteFiles); - Assert.assertEquals( - "Sequence number mismatch in Snapshot", expectedSequenceNumber, snapshot.sequenceNumber()); - - Assert.assertEquals( - "Data sequence number mismatch", - expectedSequenceNumber, - addedDeleteFile.dataSequenceNumber().longValue()); - Assert.assertEquals( - "File sequence number mismatch", - expectedSequenceNumber, - addedDeleteFile.fileSequenceNumber().longValue()); + assertThat(snapshot.sequenceNumber()) + .as("Sequence number mismatch in Snapshot") + .isEqualTo(expectedSequenceNumber); + assertThat(addedDeleteFile.dataSequenceNumber().longValue()) + .as("Data sequence number mismatch") + .isEqualTo(expectedSequenceNumber); + assertThat(addedDeleteFile.fileSequenceNumber().longValue()) + .as("File sequence number mismatch") + .isEqualTo(expectedSequenceNumber); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java index e384e571e162..ee1239074997 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotJson.java @@ -19,21 +19,19 @@ package org.apache.iceberg; import static org.apache.iceberg.Files.localInput; +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.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestSnapshotJson { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; public TableOperations ops = new LocalTableOperations(temp); @@ -49,12 +47,11 @@ public void testJsonConversion() throws IOException { String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); - Assert.assertEquals("Snapshot ID should match", expected.snapshotId(), snapshot.snapshotId()); - Assert.assertEquals( - "Files should match", expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); - Assert.assertNull("Operation should be null", snapshot.operation()); - Assert.assertNull("Summary should be null", snapshot.summary()); - Assert.assertEquals("Schema ID should match", Integer.valueOf(1), snapshot.schemaId()); + assertThat(snapshot.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(snapshot.allManifests(ops.io())).isEqualTo(expected.allManifests(ops.io())); + assertThat(snapshot.operation()).isNull(); + assertThat(snapshot.summary()).isNull(); + assertThat(snapshot.schemaId()).isEqualTo(1); } @Test @@ -69,12 +66,11 @@ public void testJsonConversionWithoutSchemaId() throws IOException { String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); - Assert.assertEquals("Snapshot ID should match", expected.snapshotId(), snapshot.snapshotId()); - Assert.assertEquals( - "Files should match", expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); - Assert.assertNull("Operation should be null", snapshot.operation()); - Assert.assertNull("Summary should be null", snapshot.summary()); - Assert.assertNull("Schema ID should be null", snapshot.schemaId()); + assertThat(snapshot.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(snapshot.allManifests(ops.io())).isEqualTo(expected.allManifests(ops.io())); + assertThat(snapshot.operation()).isNull(); + assertThat(snapshot.summary()).isNull(); + assertThat(snapshot.schemaId()).isNull(); } @Test @@ -98,20 +94,17 @@ public void testJsonConversionWithOperation() throws IOException { String json = SnapshotParser.toJson(expected); Snapshot snapshot = SnapshotParser.fromJson(json); - Assert.assertEquals("Sequence number should default to 0 for v1", 0, snapshot.sequenceNumber()); - Assert.assertEquals("Snapshot ID should match", expected.snapshotId(), snapshot.snapshotId()); - Assert.assertEquals( - "Timestamp should match", expected.timestampMillis(), snapshot.timestampMillis()); - Assert.assertEquals("Parent ID should match", expected.parentId(), snapshot.parentId()); - Assert.assertEquals( - "Manifest list should match", - expected.manifestListLocation(), - snapshot.manifestListLocation()); - Assert.assertEquals( - "Files should match", expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); - Assert.assertEquals("Operation should match", expected.operation(), snapshot.operation()); - Assert.assertEquals("Summary should match", expected.summary(), snapshot.summary()); - Assert.assertEquals("Schema ID should match", expected.schemaId(), snapshot.schemaId()); + assertThat(snapshot.sequenceNumber()) + .as("Sequence number should default to 0 for v1") + .isEqualTo(0); + assertThat(snapshot.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(snapshot.timestampMillis()).isEqualTo(expected.timestampMillis()); + assertThat(snapshot.parentId()).isEqualTo(expected.parentId()); + assertThat(snapshot.manifestListLocation()).isEqualTo(expected.manifestListLocation()); + assertThat(snapshot.allManifests(ops.io())).isEqualTo(expected.allManifests(ops.io())); + assertThat(snapshot.operation()).isEqualTo(expected.operation()); + assertThat(snapshot.summary()).isEqualTo(expected.summary()); + assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); } @Test @@ -149,29 +142,26 @@ public void testJsonConversionWithV1Manifests() { timestampMillis); String json = SnapshotParser.toJson(expected, true); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(json).isEqualTo(expectedJson); Snapshot snapshot = SnapshotParser.fromJson(json); - Assertions.assertThat(snapshot).isEqualTo(expected); - - Assert.assertEquals("Sequence number should default to 0 for v1", 0, snapshot.sequenceNumber()); - Assert.assertEquals("Snapshot ID should match", expected.snapshotId(), snapshot.snapshotId()); - Assert.assertEquals( - "Timestamp should match", expected.timestampMillis(), snapshot.timestampMillis()); - Assert.assertEquals("Parent ID should match", expected.parentId(), snapshot.parentId()); - Assert.assertEquals( - "Manifest list should match", - expected.manifestListLocation(), - snapshot.manifestListLocation()); - Assert.assertEquals( - "Files should match", expected.allManifests(ops.io()), snapshot.allManifests(ops.io())); - Assert.assertEquals("Operation should match", expected.operation(), snapshot.operation()); - Assert.assertEquals("Summary should match", expected.summary(), snapshot.summary()); - Assert.assertEquals("Schema ID should match", expected.schemaId(), snapshot.schemaId()); + assertThat(snapshot).isEqualTo(expected); + + assertThat(snapshot.sequenceNumber()) + .as("Sequence number should default to 0 for v1") + .isEqualTo(0); + assertThat(snapshot.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(snapshot.timestampMillis()).isEqualTo(expected.timestampMillis()); + assertThat(snapshot.parentId()).isEqualTo(expected.parentId()); + assertThat(snapshot.manifestListLocation()).isEqualTo(expected.manifestListLocation()); + assertThat(snapshot.allManifests(ops.io())).isEqualTo(expected.allManifests(ops.io())); + assertThat(snapshot.operation()).isEqualTo(expected.operation()); + assertThat(snapshot.summary()).isEqualTo(expected.summary()); + assertThat(snapshot.schemaId()).isEqualTo(expected.schemaId()); } private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) throws IOException { - File manifestList = temp.newFile("manifests" + UUID.randomUUID()); + File manifestList = File.createTempFile("manifests", null, temp.toFile()); manifestList.deleteOnExit(); List manifests = diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java index 25a48ce0622b..89312201265d 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java @@ -20,9 +20,11 @@ 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.times; import static org.mockito.Mockito.verify; +import java.util.Arrays; import java.util.List; import java.util.function.Predicate; import java.util.stream.Collectors; @@ -31,22 +33,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.SerializableSupplier; -import org.assertj.core.api.Assumptions; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.Mockito; -@RunWith(Parameterized.class) -public class TestSnapshotLoading extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotLoading extends TestBase { - public TestSnapshotLoading(int formatVersion) { - super(formatVersion); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } private Snapshot currentSnapshot; @@ -56,7 +53,7 @@ public TestSnapshotLoading(int formatVersion) { private SerializableSupplier> snapshotsSupplierMock; - @Before + @BeforeEach public void before() { table.newFastAppend().appendFile(FILE_A).commit(); table.newFastAppend().appendFile(FILE_B).commit(); @@ -87,7 +84,7 @@ public List get() { .build(); } - @Test + @TestTemplate public void testSnapshotsAreLoadedOnce() { latestTableMetadata.snapshots(); latestTableMetadata.snapshots(); @@ -99,7 +96,7 @@ public void testSnapshotsAreLoadedOnce() { .containsExactlyElementsOf(originalTableMetadata.snapshots()); } - @Test + @TestTemplate public void testCurrentAndMainSnapshotDoesNotLoad() { latestTableMetadata.currentSnapshot(); latestTableMetadata.snapshot(latestTableMetadata.ref(SnapshotRef.MAIN_BRANCH).snapshotId()); @@ -107,7 +104,7 @@ public void testCurrentAndMainSnapshotDoesNotLoad() { verify(snapshotsSupplierMock, times(0)).get(); } - @Test + @TestTemplate public void testUnloadedSnapshotLoadsOnce() { Snapshot unloadedSnapshot = allSnapshots.stream().filter(s -> !s.equals(currentSnapshot)).findFirst().get(); @@ -118,7 +115,7 @@ public void testUnloadedSnapshotLoadsOnce() { verify(snapshotsSupplierMock, times(1)).get(); } - @Test + @TestTemplate public void testCurrentTableScanDoesNotLoad() { latestTableMetadata.currentSnapshot(); @@ -130,9 +127,9 @@ public void testCurrentTableScanDoesNotLoad() { verify(snapshotsSupplierMock, times(0)).get(); } - @Test + @TestTemplate public void testFutureSnapshotsAreRemoved() { - Assumptions.assumeThat(formatVersion) + assumeThat(formatVersion) .as("Future snapshots are only removed for V2 tables") .isGreaterThan(1); @@ -152,7 +149,7 @@ public void testFutureSnapshotsAreRemoved() { .containsExactlyInAnyOrderElementsOf(originalTableMetadata.snapshots()); } - @Test + @TestTemplate public void testRemovedCurrentSnapshotFails() { List snapshotsMissingCurrent = allSnapshots.stream() @@ -174,7 +171,7 @@ public void testRemovedCurrentSnapshotFails() { .hasMessage("Invalid table metadata: Cannot find current version"); } - @Test + @TestTemplate public void testRemovedRefSnapshotFails() { Snapshot referencedSnapshot = allSnapshots.stream().filter(Predicate.isEqual(currentSnapshot).negate()).findFirst().get(); @@ -194,7 +191,7 @@ public void testRemovedRefSnapshotFails() { .hasMessageEndingWith("does not exist in the existing snapshots list"); } - @Test + @TestTemplate public void testBuildingNewMetadataTriggersSnapshotLoad() { TableMetadata newTableMetadata = TableMetadata.buildFrom(latestTableMetadata).removeRef(SnapshotRef.MAIN_BRANCH).build(); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java index fd22ae24d0e4..88233dd99097 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java @@ -18,16 +18,18 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestSnapshotManager extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotManager extends TestBase { // replacement for FILE_A static final DataFile REPLACEMENT_FILE_A = @@ -47,16 +49,12 @@ public class TestSnapshotManager extends TableTestBase { .withRecordCount(1) .build(); - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestSnapshotManager(int formatVersion) { - super(formatVersion); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testCherryPickDynamicOverwrite() { table.newAppend().appendFile(FILE_A).commit(); @@ -64,8 +62,9 @@ public void testCherryPickDynamicOverwrite() { table.newReplacePartitions().addFile(REPLACEMENT_FILE_A).stageOnly().commit(); Snapshot staged = Iterables.getLast(table.snapshots()); - Assert.assertEquals( - "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); // add another append so that the original commit can't be fast-forwarded table.newAppend().appendFile(FILE_B).commit(); @@ -73,21 +72,23 @@ public void testCherryPickDynamicOverwrite() { // pick the snapshot into the current state table.manageSnapshots().cherrypick(staged.snapshotId()).commit(); - Assert.assertNotEquals( - "Should not fast-forward", staged.snapshotId(), table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Should not fast-forward") + .isNotEqualTo(staged.snapshotId()); validateTableFiles(table, FILE_B, REPLACEMENT_FILE_A); } - @Test + @TestTemplate public void testCherryPickDynamicOverwriteWithoutParent() { - Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); // stage an overwrite that replaces FILE_A table.newReplacePartitions().addFile(REPLACEMENT_FILE_A).stageOnly().commit(); Snapshot staged = Iterables.getLast(table.snapshots()); - Assert.assertEquals( - "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); // add another append so that the original commit can't be fast-forwarded table.newAppend().appendFile(FILE_B).commit(); @@ -95,12 +96,13 @@ public void testCherryPickDynamicOverwriteWithoutParent() { // pick the snapshot into the current state table.manageSnapshots().cherrypick(staged.snapshotId()).commit(); - Assert.assertNotEquals( - "Should not fast-forward", staged.snapshotId(), table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Should not fast-forward") + .isNotEqualTo(staged.snapshotId()); validateTableFiles(table, FILE_B, REPLACEMENT_FILE_A); } - @Test + @TestTemplate public void testCherryPickDynamicOverwriteConflict() { table.newAppend().appendFile(FILE_A).commit(); @@ -108,27 +110,26 @@ public void testCherryPickDynamicOverwriteConflict() { table.newReplacePartitions().addFile(REPLACEMENT_FILE_A).stageOnly().commit(); Snapshot staged = Iterables.getLast(table.snapshots()); - Assert.assertEquals( - "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); // add another append so that the original commit can't be fast-forwarded table.newAppend().appendFile(CONFLICT_FILE_A).commit(); long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) + assertThatThrownBy(() -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot cherry-pick replace partitions with changed partition"); - Assert.assertEquals( - "Failed cherry-pick should not change the table state", - lastSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Failed cherry-pick should not change the table state") + .isEqualTo(lastSnapshotId); validateTableFiles(table, FILE_A, CONFLICT_FILE_A); } - @Test + @TestTemplate public void testCherryPickDynamicOverwriteDeleteConflict() { table.newAppend().appendFile(FILE_A).commit(); @@ -136,8 +137,9 @@ public void testCherryPickDynamicOverwriteDeleteConflict() { table.newReplacePartitions().addFile(REPLACEMENT_FILE_A).stageOnly().commit(); Snapshot staged = Iterables.getLast(table.snapshots()); - Assert.assertEquals( - "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); // add FILE_B s table.newAppend().appendFile(FILE_B).commit(); @@ -147,19 +149,17 @@ public void testCherryPickDynamicOverwriteDeleteConflict() { long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) + assertThatThrownBy(() -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Missing required files to delete"); - Assert.assertEquals( - "Failed cherry-pick should not change the table state", - lastSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Failed cherry-pick should not change the table state") + .isEqualTo(lastSnapshotId); validateTableFiles(table, FILE_B); } - @Test + @TestTemplate public void testCherryPickFromBranch() { table.newAppend().appendFile(FILE_A).commit(); long branchSnapshotId = table.currentSnapshot().snapshotId(); @@ -177,20 +177,18 @@ public void testCherryPickFromBranch() { long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().cherrypick(replaceSnapshotId).commit()) + assertThatThrownBy(() -> table.manageSnapshots().cherrypick(replaceSnapshotId).commit()) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( "Cannot cherry-pick overwrite not based on an ancestor of the current state"); - Assert.assertEquals( - "Failed cherry-pick should not change the table state", - lastSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Failed cherry-pick should not change the table state") + .isEqualTo(lastSnapshotId); validateTableFiles(table, FILE_A); } - @Test + @TestTemplate public void testCherryPickOverwrite() { table.newAppend().appendFile(FILE_A).commit(); @@ -198,99 +196,94 @@ public void testCherryPickOverwrite() { table.newOverwrite().deleteFile(FILE_A).addFile(REPLACEMENT_FILE_A).stageOnly().commit(); Snapshot staged = Iterables.getLast(table.snapshots()); - Assert.assertEquals( - "Should find the staged overwrite snapshot", DataOperations.OVERWRITE, staged.operation()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); // add another append so that the original commit can't be fast-forwarded table.newAppend().appendFile(FILE_B).commit(); long lastSnapshotId = table.currentSnapshot().snapshotId(); // pick the snapshot into the current state - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) + assertThatThrownBy(() -> table.manageSnapshots().cherrypick(staged.snapshotId()).commit()) .isInstanceOf(ValidationException.class) .hasMessageEndingWith("not append, dynamic overwrite, or fast-forward"); - Assert.assertEquals( - "Failed cherry-pick should not change the table state", - lastSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()) + .as("Failed cherry-pick should not change the table state") + .isEqualTo(lastSnapshotId); validateTableFiles(table, FILE_A, FILE_B); } - @Test + @TestTemplate public void testCreateBranch() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); // Test a basic case of creating a branch table.manageSnapshots().createBranch("branch1", snapshotId).commit(); SnapshotRef expectedBranch = table.ops().refresh().ref("branch1"); - Assert.assertTrue( - expectedBranch != null - && expectedBranch.equals(SnapshotRef.branchBuilder(snapshotId).build())); + assertThat(expectedBranch).isNotNull().isEqualTo(SnapshotRef.branchBuilder(snapshotId).build()); } - @Test + @TestTemplate public void testCreateBranchWithoutSnapshotId() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); // Test a basic case of creating a branch table.manageSnapshots().createBranch("branch1").commit(); SnapshotRef actualBranch = table.ops().refresh().ref("branch1"); - Assertions.assertThat(actualBranch).isNotNull(); - Assertions.assertThat(actualBranch).isEqualTo(SnapshotRef.branchBuilder(snapshotId).build()); + assertThat(actualBranch).isNotNull().isEqualTo(SnapshotRef.branchBuilder(snapshotId).build()); } - @Test + @TestTemplate public void testCreateBranchOnEmptyTable() { table.manageSnapshots().createBranch("branch1").commit(); SnapshotRef mainSnapshotRef = table.ops().refresh().ref(SnapshotRef.MAIN_BRANCH); - Assertions.assertThat(mainSnapshotRef).isNull(); + assertThat(mainSnapshotRef).isNull(); SnapshotRef branch1SnapshotRef = table.ops().refresh().ref("branch1"); - Assertions.assertThat(branch1SnapshotRef).isNotNull(); - Assertions.assertThat(branch1SnapshotRef.minSnapshotsToKeep()).isNull(); - Assertions.assertThat(branch1SnapshotRef.maxSnapshotAgeMs()).isNull(); - Assertions.assertThat(branch1SnapshotRef.maxRefAgeMs()).isNull(); + assertThat(branch1SnapshotRef).isNotNull(); + assertThat(branch1SnapshotRef.minSnapshotsToKeep()).isNull(); + assertThat(branch1SnapshotRef.maxSnapshotAgeMs()).isNull(); + assertThat(branch1SnapshotRef.maxRefAgeMs()).isNull(); Snapshot snapshot = table.snapshot(branch1SnapshotRef.snapshotId()); - Assertions.assertThat(snapshot.parentId()).isNull(); - Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); } - @Test + @TestTemplate public void testCreateBranchOnEmptyTableFailsWhenRefAlreadyExists() { table.manageSnapshots().createBranch("branch1").commit(); // Trying to create a branch with an existing name should fail - Assertions.assertThatThrownBy(() -> table.manageSnapshots().createBranch("branch1").commit()) + assertThatThrownBy(() -> table.manageSnapshots().createBranch("branch1").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref branch1 already exists"); // Trying to create another branch within the same chain - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.manageSnapshots().createBranch("branch2").createBranch("branch2").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref branch2 already exists"); } - @Test + @TestTemplate public void testCreateBranchFailsWhenRefAlreadyExists() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); // Trying to create a branch with an existing name should fail - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().createBranch("branch1", snapshotId).commit()) + assertThatThrownBy(() -> table.manageSnapshots().createBranch("branch1", snapshotId).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref branch1 already exists"); // Trying to create another branch within the same chain - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .manageSnapshots() @@ -301,7 +294,7 @@ public void testCreateBranchFailsWhenRefAlreadyExists() { .hasMessage("Ref branch2 already exists"); } - @Test + @TestTemplate public void testCreateTag() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -309,24 +302,22 @@ public void testCreateTag() { table.manageSnapshots().createTag("tag1", snapshotId).commit(); SnapshotRef expectedTag = table.ops().refresh().ref("tag1"); - Assert.assertTrue( - expectedTag != null && expectedTag.equals(SnapshotRef.tagBuilder(snapshotId).build())); + assertThat(expectedTag).isNotNull().isEqualTo(SnapshotRef.tagBuilder(snapshotId).build()); } - @Test + @TestTemplate public void testCreateTagFailsWhenRefAlreadyExists() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag("tag1", snapshotId).commit(); // Trying to create a tag with an existing name should fail - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().createTag("tag1", snapshotId).commit()) + assertThatThrownBy(() -> table.manageSnapshots().createTag("tag1", snapshotId).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref tag1 already exists"); // Trying to create another tag within the same chain - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .manageSnapshots() @@ -337,7 +328,7 @@ public void testCreateTagFailsWhenRefAlreadyExists() { .hasMessage("Ref tag2 already exists"); } - @Test + @TestTemplate public void testRemoveBranch() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -347,31 +338,29 @@ public void testRemoveBranch() { TableMetadata updated = table.ops().refresh(); SnapshotRef expectedBranch = updated.ref("branch1"); - Assert.assertNull(expectedBranch); + assertThat(expectedBranch).isNull(); // Test chained creating and removal of branch and tag table.manageSnapshots().createBranch("branch2", snapshotId).removeBranch("branch2").commit(); updated = table.ops().refresh(); - Assert.assertNull(updated.ref("branch2")); + assertThat(updated.ref("branch2")).isNull(); } - @Test + @TestTemplate public void testRemovingNonExistingBranchFails() { - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().removeBranch("non-existing").commit()) + assertThatThrownBy(() -> table.manageSnapshots().removeBranch("non-existing").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Branch does not exist: non-existing"); } - @Test + @TestTemplate public void testRemovingMainBranchFails() { - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().removeBranch(SnapshotRef.MAIN_BRANCH).commit()) + assertThatThrownBy(() -> table.manageSnapshots().removeBranch(SnapshotRef.MAIN_BRANCH).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot remove main branch"); } - @Test + @TestTemplate public void testRemoveTag() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -380,22 +369,22 @@ public void testRemoveTag() { table.manageSnapshots().removeTag("tag1").commit(); TableMetadata updated = table.ops().refresh(); SnapshotRef expectedTag = updated.ref("tag1"); - Assert.assertNull(expectedTag); + assertThat(expectedTag).isNull(); // Test chained creating and removal of a tag table.manageSnapshots().createTag("tag2", snapshotId).removeTag("tag2").commit(); - Assert.assertEquals(updated, table.ops().refresh()); - Assert.assertNull(updated.ref("tag2")); + assertThat(table.ops().refresh()).isEqualTo(updated); + assertThat(updated.ref("tag2")).isNull(); } - @Test + @TestTemplate public void testRemovingNonExistingTagFails() { - Assertions.assertThatThrownBy(() -> table.manageSnapshots().removeTag("non-existing").commit()) + assertThatThrownBy(() -> table.manageSnapshots().removeTag("non-existing").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Tag does not exist: non-existing"); } - @Test + @TestTemplate public void testReplaceBranch() { table.newAppend().appendFile(FILE_A).set("wap.id", "123").stageOnly().commit(); Snapshot firstSnapshot = Iterables.getOnlyElement(table.snapshots()); @@ -404,63 +393,61 @@ public void testReplaceBranch() { Snapshot secondSnapshot = Iterables.get(table.snapshots(), 1); table.manageSnapshots().createBranch("branch2", secondSnapshot.snapshotId()).commit(); table.manageSnapshots().replaceBranch("branch1", "branch2").commit(); - Assert.assertEquals( - table.ops().refresh().ref("branch1").snapshotId(), secondSnapshot.snapshotId()); + assertThat(secondSnapshot.snapshotId()) + .isEqualTo(table.ops().refresh().ref("branch1").snapshotId()); } - @Test + @TestTemplate public void testReplaceBranchNonExistingToBranchFails() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.manageSnapshots().replaceBranch("branch1", "non-existing").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref does not exist: non-existing"); } - @Test + @TestTemplate public void testFastForwardBranchNonExistingFromBranchCreatesTheBranch() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); table.manageSnapshots().fastForwardBranch("new-branch", "branch1").commit(); - Assertions.assertThat(table.ops().current().ref("new-branch").isBranch()).isTrue(); - Assertions.assertThat(table.ops().current().ref("new-branch").snapshotId()) - .isEqualTo(snapshotId); + assertThat(table.ops().current().ref("new-branch").isBranch()).isTrue(); + assertThat(table.ops().current().ref("new-branch").snapshotId()).isEqualTo(snapshotId); } - @Test + @TestTemplate public void testReplaceBranchNonExistingFromBranchCreatesTheBranch() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); table.manageSnapshots().replaceBranch("new-branch", "branch1").commit(); - Assertions.assertThat(table.ops().current().ref("new-branch").isBranch()).isTrue(); - Assertions.assertThat(table.ops().current().ref("new-branch").snapshotId()) - .isEqualTo(snapshotId); + assertThat(table.ops().current().ref("new-branch").isBranch()).isTrue(); + assertThat(table.ops().current().ref("new-branch").snapshotId()).isEqualTo(snapshotId); } - @Test + @TestTemplate public void testFastForwardBranchNonExistingToFails() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createBranch("branch1", snapshotId).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.manageSnapshots().fastForwardBranch("branch1", "non-existing").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref does not exist: non-existing"); } - @Test + @TestTemplate public void testFastForward() { table.newAppend().appendFile(FILE_A).commit(); table.newAppend().appendFile(FILE_B).set("wap.id", "123456789").stageOnly().commit(); - Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(1); table.manageSnapshots().createBranch("new-branch-at-staged-snapshot", 2).commit(); table @@ -468,10 +455,10 @@ public void testFastForward() { .fastForwardBranch(SnapshotRef.MAIN_BRANCH, "new-branch-at-staged-snapshot") .commit(); - Assert.assertEquals(table.currentSnapshot().snapshotId(), 2); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(2); } - @Test + @TestTemplate public void testFastForwardWhenFromIsNotAncestorFails() { table.newAppend().appendFile(FILE_A).commit(); @@ -485,7 +472,7 @@ public void testFastForwardWhenFromIsNotAncestorFails() { final String newBranch = "new-branch-at-staged-snapshot"; table.manageSnapshots().createBranch(newBranch, snapshot).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .manageSnapshots() @@ -496,7 +483,7 @@ public void testFastForwardWhenFromIsNotAncestorFails() { "Cannot fast-forward: main is not an ancestor of new-branch-at-staged-snapshot"); } - @Test + @TestTemplate public void testReplaceTag() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -505,10 +492,10 @@ public void testReplaceTag() { table.newAppend().appendFile(FILE_B).commit(); long currentSnapshot = table.ops().refresh().currentSnapshot().snapshotId(); table.manageSnapshots().replaceTag("tag1", currentSnapshot).commit(); - Assert.assertEquals(table.ops().refresh().ref("tag1").snapshotId(), currentSnapshot); + assertThat(currentSnapshot).isEqualTo(table.ops().refresh().ref("tag1").snapshotId()); } - @Test + @TestTemplate public void testUpdatingBranchRetention() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -520,8 +507,8 @@ public void testUpdatingBranchRetention() { .setMaxSnapshotAgeMs("branch1", 20000) .commit(); TableMetadata updated = table.ops().refresh(); - Assert.assertEquals(20000, (long) updated.ref("branch1").maxSnapshotAgeMs()); - Assert.assertEquals(10, (long) updated.ref("branch1").minSnapshotsToKeep()); + assertThat(updated.ref("branch1").maxSnapshotAgeMs()).isEqualTo(20000); + assertThat(updated.ref("branch1").minSnapshotsToKeep()).isEqualTo(10); // Test creating and updating in a chain table .manageSnapshots() @@ -530,16 +517,16 @@ public void testUpdatingBranchRetention() { .setMaxSnapshotAgeMs("branch2", 20000) .commit(); updated = table.ops().refresh(); - Assert.assertEquals(20000, (long) updated.ref("branch2").maxSnapshotAgeMs()); - Assert.assertEquals(10, (long) updated.ref("branch2").minSnapshotsToKeep()); + assertThat(updated.ref("branch2").maxSnapshotAgeMs()).isEqualTo(20000); + assertThat(updated.ref("branch2").minSnapshotsToKeep()).isEqualTo(10); } - @Test + @TestTemplate public void testSettingBranchRetentionOnTagFails() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .manageSnapshots() @@ -549,7 +536,7 @@ public void testSettingBranchRetentionOnTagFails() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Tags do not support setting minSnapshotsToKeep"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .manageSnapshots() @@ -560,7 +547,7 @@ public void testSettingBranchRetentionOnTagFails() { .hasMessage("Tags do not support setting maxSnapshotAgeMs"); } - @Test + @TestTemplate public void testUpdatingBranchMaxRefAge() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -570,11 +557,10 @@ public void testUpdatingBranchMaxRefAge() { table.manageSnapshots().createBranch("branch1", snapshotId).commit(); table.manageSnapshots().setMaxRefAgeMs("branch1", 10000).commit(); TableMetadata updated = table.ops().refresh(); - Assert.assertEquals(maxRefAgeMs, (long) updated.ref("branch1").maxRefAgeMs()); - Assert.assertEquals(maxRefAgeMs, (long) updated.ref("branch1").maxRefAgeMs()); + assertThat(updated.ref("branch1").maxRefAgeMs()).isEqualTo(maxRefAgeMs); } - @Test + @TestTemplate public void testUpdatingTagMaxRefAge() { table.newAppend().appendFile(FILE_A).commit(); long snapshotId = table.currentSnapshot().snapshotId(); @@ -585,7 +571,7 @@ public void testUpdatingTagMaxRefAge() { table.manageSnapshots().setMaxRefAgeMs("tag1", maxRefAgeMs).commit(); TableMetadata updated = table.ops().refresh(); - Assert.assertEquals(maxRefAgeMs, (long) updated.ref("tag1").maxRefAgeMs()); + assertThat(updated.ref("tag1").maxRefAgeMs()).isEqualTo(maxRefAgeMs); // Test creating and updating in a chain table @@ -594,10 +580,10 @@ public void testUpdatingTagMaxRefAge() { .setMaxRefAgeMs("tag2", maxRefAgeMs) .commit(); updated = table.ops().refresh(); - Assert.assertEquals(maxRefAgeMs, (long) updated.ref("tag2").maxRefAgeMs()); + assertThat(updated.ref("tag2").maxRefAgeMs()).isEqualTo(maxRefAgeMs); } - @Test + @TestTemplate public void testRenameBranch() { table.newAppend().appendFile(FILE_A).commit(); table.newAppend().appendFile(FILE_A).commit(); @@ -607,8 +593,8 @@ public void testRenameBranch() { table.manageSnapshots().createBranch("branch1", snapshotId).commit(); table.manageSnapshots().renameBranch("branch1", "branch2").commit(); TableMetadata updated = table.ops().refresh(); - Assert.assertNull(updated.ref("branch1")); - Assert.assertEquals(updated.ref("branch2"), SnapshotRef.branchBuilder(snapshotId).build()); + assertThat(updated.ref("branch1")).isNull(); + assertThat(SnapshotRef.branchBuilder(snapshotId).build()).isEqualTo(updated.ref("branch2")); table .manageSnapshots() @@ -617,13 +603,13 @@ public void testRenameBranch() { .commit(); updated = table.ops().refresh(); - Assert.assertNull(updated.ref("branch3")); - Assert.assertEquals(updated.ref("branch4"), SnapshotRef.branchBuilder(snapshotId).build()); + assertThat(updated.ref("branch3")).isNull(); + assertThat(SnapshotRef.branchBuilder(snapshotId).build()).isEqualTo(updated.ref("branch4")); } - @Test + @TestTemplate public void testFailRenamingMainBranch() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .manageSnapshots() @@ -633,16 +619,16 @@ public void testFailRenamingMainBranch() { .hasMessage("Cannot rename main branch"); } - @Test + @TestTemplate public void testRenamingNonExistingBranchFails() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.manageSnapshots().renameBranch("some-missing-branch", "some-branch").commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Branch does not exist: some-missing-branch"); } - @Test + @TestTemplate public void testCreateReferencesAndRollback() { table.newAppend().appendFile(FILE_A).commit(); table.newAppend().appendFile(FILE_A).commit(); @@ -656,15 +642,15 @@ public void testCreateReferencesAndRollback() { .commit(); TableMetadata current = table.ops().current(); - Assert.assertEquals(current.currentSnapshot().snapshotId(), 1); + assertThat(current.currentSnapshot().snapshotId()).isEqualTo(1); SnapshotRef actualTag = current.ref("tag1"); SnapshotRef actualBranch = current.ref("branch1"); - Assert.assertEquals(1, current.currentSnapshot().snapshotId()); - Assert.assertEquals(SnapshotRef.branchBuilder(snapshotPriorToRollback).build(), actualBranch); - Assert.assertEquals(SnapshotRef.tagBuilder(snapshotPriorToRollback).build(), actualTag); + assertThat(current.currentSnapshot().snapshotId()).isEqualTo(1); + assertThat(actualBranch).isEqualTo(SnapshotRef.branchBuilder(snapshotPriorToRollback).build()); + assertThat(actualTag).isEqualTo(SnapshotRef.tagBuilder(snapshotPriorToRollback).build()); } - @Test + @TestTemplate public void testCreateReferencesAndCherrypick() { table.newAppend().appendFile(FILE_A).commit(); @@ -681,15 +667,15 @@ public void testCreateReferencesAndCherrypick() { .commit(); TableMetadata current = table.ops().current(); - Assert.assertEquals(current.currentSnapshot().snapshotId(), 2); + assertThat(current.currentSnapshot().snapshotId()).isEqualTo(2); SnapshotRef actualTag = current.ref("tag1"); SnapshotRef actualBranch = current.ref("branch1"); - Assert.assertEquals(2, current.currentSnapshot().snapshotId()); - Assert.assertEquals(SnapshotRef.branchBuilder(1).build(), actualBranch); - Assert.assertEquals(SnapshotRef.tagBuilder(1).build(), actualTag); + assertThat(current.currentSnapshot().snapshotId()).isEqualTo(2); + assertThat(actualBranch).isEqualTo(SnapshotRef.branchBuilder(1).build()); + assertThat(actualTag).isEqualTo(SnapshotRef.tagBuilder(1).build()); } - @Test + @TestTemplate public void testAttemptToRollbackToCurrentSnapshot() { table.newAppend().appendFile(FILE_A).commit(); @@ -700,7 +686,7 @@ public void testAttemptToRollbackToCurrentSnapshot() { table.manageSnapshots().rollbackTo(currentSnapshotId).commit(); } - @Test + @TestTemplate public void testSnapshotManagerThroughTransaction() { table.newAppend().appendFile(FILE_A).commit(); Snapshot snapshotAfterFirstAppend = readMetadata().currentSnapshot(); @@ -708,40 +694,43 @@ public void testSnapshotManagerThroughTransaction() { table.newAppend().appendFile(FILE_B).commit(); validateSnapshot(snapshotAfterFirstAppend, readMetadata().currentSnapshot(), FILE_B); - Assert.assertEquals("Table should be on version 2 after appending twice", 2, (int) version()); + assertThat(version()).as("Table should be on version 2 after appending twice").isEqualTo(2); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when transaction is created", base, readMetadata()); - Assert.assertEquals( - "Table should be on version 2 after creating transaction", 2, (int) version()); + assertThat(readMetadata()) + .as("Base metadata should not change when transaction is created") + .isSameAs(base); + assertThat(version()) + .as("Table should be on version 2 after creating transaction") + .isEqualTo(2); ManageSnapshots manageSnapshots = txn.manageSnapshots(); - Assert.assertNotNull(manageSnapshots); + assertThat(manageSnapshots).isNotNull(); - Assert.assertSame( - "Base metadata should not change when manageSnapshots is created", base, readMetadata()); - Assert.assertEquals( - "Table should be on version 2 after creating manageSnapshots", 2, (int) version()); + assertThat(readMetadata()) + .as("Base metadata should not change when manageSnapshots is created") + .isSameAs(base); + assertThat(version()) + .as("Table should be on version 2 after creating manageSnapshots") + .isEqualTo(2); manageSnapshots.rollbackTo(snapshotAfterFirstAppend.snapshotId()).commit(); - Assert.assertSame( - "Base metadata should not change when invoking rollbackTo", base, readMetadata()); - Assert.assertEquals( - "Table should be on version 2 after invoking rollbackTo", 2, (int) version()); + assertThat(readMetadata()) + .as("Base metadata should not change when invoking rollbackTo") + .isSameAs(base); + assertThat(version()).as("Table should be on version 2 after invoking rollbackTo").isEqualTo(2); txn.commitTransaction(); - Assert.assertEquals(snapshotAfterFirstAppend, readMetadata().currentSnapshot()); + assertThat(readMetadata().currentSnapshot()).isEqualTo(snapshotAfterFirstAppend); validateSnapshot(null, snapshotAfterFirstAppend, FILE_A); - Assert.assertEquals( - "Table should be on version 3 after invoking rollbackTo", 3, (int) version()); + assertThat(version()).as("Table should be on version 3 after invoking rollbackTo").isEqualTo(3); } - @Test + @TestTemplate public void testSnapshotManagerThroughTransactionMultiOperation() { table.newAppend().appendFile(FILE_A).commit(); Snapshot snapshotAfterFirstAppend = readMetadata().currentSnapshot(); @@ -749,32 +738,30 @@ public void testSnapshotManagerThroughTransactionMultiOperation() { table.newAppend().appendFile(FILE_B).commit(); validateSnapshot(snapshotAfterFirstAppend, readMetadata().currentSnapshot(), FILE_B); - Assert.assertEquals("Table should be on version 2 after appending twice", 2, (int) version()); + assertThat(version()).as("Table should be on version 2 after appending twice").isEqualTo(2); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); txn.manageSnapshots().rollbackTo(snapshotAfterFirstAppend.snapshotId()).commit(); txn.updateProperties().set("some_prop", "some_prop_value").commit(); - Assert.assertSame( - "Base metadata should not change when transaction is not committed", base, readMetadata()); - Assert.assertEquals( - "Table should remain on version 2 when transaction is not committed", 2, (int) version()); + assertThat(readMetadata()) + .as("Base metadata should not change when transaction is not committed") + .isSameAs(base); + assertThat(version()) + .as("Table should remain on version 2 when transaction is not committed") + .isEqualTo(2); txn.commitTransaction(); - Assert.assertEquals(snapshotAfterFirstAppend, readMetadata().currentSnapshot()); - Assert.assertEquals( - "Table should be on version 3 after invoking rollbackTo", 3, (int) version()); + assertThat(readMetadata().currentSnapshot()).isEqualTo(snapshotAfterFirstAppend); + assertThat(version()).as("Table should be on version 3 after invoking rollbackTo").isEqualTo(3); } - @Test - public void testSnapshotManagerInvalidParameters() throws Exception { - Assert.assertThrows( - "Incorrect input transaction: null", - IllegalArgumentException.class, - () -> { - new SnapshotManager(null); - }); + @TestTemplate + public void testSnapshotManagerInvalidParameters() { + assertThatThrownBy(() -> new SnapshotManager(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid input transaction: null"); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java b/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java index d8940367e1a5..52e937a7745e 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotRefParser.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.junit.jupiter.api.Test; public class TestSnapshotRefParser { @@ -28,24 +29,27 @@ public class TestSnapshotRefParser { public void testTagToJsonDefault() { String json = "{\"snapshot-id\":1,\"type\":\"tag\"}"; SnapshotRef ref = SnapshotRef.tagBuilder(1L).build(); - Assert.assertEquals( - "Should be able to serialize default tag", json, SnapshotRefParser.toJson(ref)); + assertThat(SnapshotRefParser.toJson(ref)) + .as("Should be able to serialize default tag") + .isEqualTo(json); } @Test public void testTagToJsonAllFields() { String json = "{\"snapshot-id\":1,\"type\":\"tag\",\"max-ref-age-ms\":1}"; SnapshotRef ref = SnapshotRef.tagBuilder(1L).maxRefAgeMs(1L).build(); - Assert.assertEquals( - "Should be able to serialize tag with all fields", json, SnapshotRefParser.toJson(ref)); + assertThat(SnapshotRefParser.toJson(ref)) + .as("Should be able to serialize tag with all fields") + .isEqualTo(json); } @Test public void testBranchToJsonDefault() { String json = "{\"snapshot-id\":1,\"type\":\"branch\"}"; SnapshotRef ref = SnapshotRef.branchBuilder(1L).build(); - Assert.assertEquals( - "Should be able to serialize default branch", json, SnapshotRefParser.toJson(ref)); + assertThat(SnapshotRefParser.toJson(ref)) + .as("Should be able to serialize default branch") + .isEqualTo(json); } @Test @@ -59,32 +63,36 @@ public void testBranchToJsonAllFields() { .maxSnapshotAgeMs(3L) .maxRefAgeMs(4L) .build(); - Assert.assertEquals( - "Should be able to serialize branch with all fields", json, SnapshotRefParser.toJson(ref)); + assertThat(SnapshotRefParser.toJson(ref)) + .as("Should be able to serialize branch with all fields") + .isEqualTo(json); } @Test public void testTagFromJsonDefault() { String json = "{\"snapshot-id\":1,\"type\":\"tag\"}"; SnapshotRef ref = SnapshotRef.tagBuilder(1L).build(); - Assert.assertEquals( - "Should be able to deserialize default tag", ref, SnapshotRefParser.fromJson(json)); + assertThat(SnapshotRefParser.fromJson(json)) + .as("Should be able to deserialize default tag") + .isEqualTo(ref); } @Test public void testTagFromJsonAllFields() { String json = "{\"snapshot-id\":1,\"type\":\"tag\",\"max-ref-age-ms\":1}"; SnapshotRef ref = SnapshotRef.tagBuilder(1L).maxRefAgeMs(1L).build(); - Assert.assertEquals( - "Should be able to deserialize tag with all fields", ref, SnapshotRefParser.fromJson(json)); + assertThat(SnapshotRefParser.fromJson(json)) + .as("Should be able to deserialize tag with all fields") + .isEqualTo(ref); } @Test public void testBranchFromJsonDefault() { String json = "{\"snapshot-id\":1,\"type\":\"branch\"}"; SnapshotRef ref = SnapshotRef.branchBuilder(1L).build(); - Assert.assertEquals( - "Should be able to deserialize default branch", ref, SnapshotRefParser.fromJson(json)); + assertThat(SnapshotRefParser.fromJson(json)) + .as("Should be able to deserialize default branch") + .isEqualTo(ref); } @Test @@ -98,21 +106,20 @@ public void testBranchFromJsonAllFields() { .maxSnapshotAgeMs(3L) .maxRefAgeMs(4L) .build(); - Assert.assertEquals( - "Should be able to deserialize branch with all fields", - ref, - SnapshotRefParser.fromJson(json)); + assertThat(SnapshotRefParser.fromJson(json)) + .as("Should be able to deserialize branch with all fields") + .isEqualTo(ref); } @Test public void testFailParsingWhenNullOrEmptyJson() { String nullJson = null; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(nullJson)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(nullJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot parse snapshot ref from invalid JSON"); String emptyJson = ""; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(emptyJson)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot parse snapshot ref from invalid JSON"); } @@ -120,12 +127,12 @@ public void testFailParsingWhenNullOrEmptyJson() { @Test public void testFailParsingWhenMissingRequiredFields() { String refMissingType = "{\"snapshot-id\":1}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(refMissingType)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(refMissingType)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot parse missing string"); String refMissingSnapshotId = "{\"type\":\"branch\"}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(refMissingSnapshotId)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(refMissingSnapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot parse missing long"); } @@ -134,31 +141,31 @@ public void testFailParsingWhenMissingRequiredFields() { public void testFailWhenFieldsHaveInvalidValues() { String invalidSnapshotId = "{\"snapshot-id\":\"invalid-snapshot-id\",\"type\":\"not-a-valid-tag-type\"}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidSnapshotId)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidSnapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: snapshot-id: \"invalid-snapshot-id\""); String invalidTagType = "{\"snapshot-id\":1,\"type\":\"not-a-valid-tag-type\"}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidTagType)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidTagType)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid snapshot ref type: not-a-valid-tag-type"); String invalidRefAge = "{\"snapshot-id\":1,\"type\":\"tag\",\"max-ref-age-ms\":\"not-a-valid-value\"}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidRefAge)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidRefAge)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: max-ref-age-ms: \"not-a-valid-value\""); String invalidSnapshotsToKeep = "{\"snapshot-id\":1,\"type\":\"branch\", " + "\"min-snapshots-to-keep\":\"invalid-number\"}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidSnapshotsToKeep)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidSnapshotsToKeep)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: min-snapshots-to-keep: \"invalid-number\""); String invalidMaxSnapshotAge = "{\"snapshot-id\":1,\"type\":\"branch\", " + "\"max-snapshot-age-ms\":\"invalid-age\"}"; - Assertions.assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidMaxSnapshotAge)) + assertThatThrownBy(() -> SnapshotRefParser.fromJson(invalidMaxSnapshotAge)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: max-snapshot-age-ms: \"invalid-age\""); } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java index 1a9f4646e81a..7ce59e9df1c9 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java @@ -18,29 +18,27 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSnapshotSelection extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - public TestSnapshotSelection(int formatVersion) { - super(formatVersion); +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotSelection extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testSnapshotSelectionById() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(0); table.newFastAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); @@ -48,12 +46,12 @@ public void testSnapshotSelectionById() { table.newFastAppend().appendFile(FILE_B).commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals("Table should have two snapshots", 2, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(2); validateSnapshot(null, table.snapshot(firstSnapshot.snapshotId()), FILE_A); validateSnapshot(firstSnapshot, table.snapshot(secondSnapshot.snapshotId()), FILE_B); } - @Test + @TestTemplate public void testSnapshotStatsForAddedFiles() { DataFile fileWithStats = DataFiles.builder(SPEC) @@ -76,12 +74,12 @@ public void testSnapshotStatsForAddedFiles() { Snapshot snapshot = table.currentSnapshot(); Iterable addedFiles = snapshot.addedDataFiles(table.io()); - Assert.assertEquals(1, Iterables.size(addedFiles)); + assertThat(addedFiles).hasSize(1); DataFile dataFile = Iterables.getOnlyElement(addedFiles); - Assert.assertNotNull("Value counts should be not null", dataFile.valueCounts()); - Assert.assertNotNull("Null value counts should be not null", dataFile.nullValueCounts()); - Assert.assertNotNull("Lower bounds should be not null", dataFile.lowerBounds()); - Assert.assertNotNull("Upper bounds should be not null", dataFile.upperBounds()); + assertThat(dataFile.valueCounts()).isNotNull(); + assertThat(dataFile.nullValueCounts()).isNotNull(); + assertThat(dataFile.lowerBounds()).isNotNull(); + assertThat(dataFile.upperBounds()).isNotNull(); } private ByteBuffer longToBuffer(long value) { diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index e5653b4c65d7..75b98bd4dca1 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -18,40 +18,41 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.List; import java.util.Map; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestSnapshotSummary extends TableTestBase { - public TestSnapshotSummary(int formatVersion) { - super(formatVersion); - } +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotSummary extends TestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testFileSizeSummary() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(0); // fast append table.newFastAppend().appendFile(FILE_A).commit(); Map summary = table.currentSnapshot().summary(); - Assert.assertEquals("10", summary.get(SnapshotSummary.ADDED_FILE_SIZE_PROP)); - Assert.assertNull(summary.get(SnapshotSummary.REMOVED_FILE_SIZE_PROP)); - Assert.assertEquals("10", summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP)); + assertThat(summary) + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .doesNotContainKey(SnapshotSummary.REMOVED_FILE_SIZE_PROP); // merge append table.newAppend().appendFile(FILE_B).commit(); summary = table.currentSnapshot().summary(); - Assert.assertEquals("10", summary.get(SnapshotSummary.ADDED_FILE_SIZE_PROP)); - Assert.assertNull(summary.get(SnapshotSummary.REMOVED_FILE_SIZE_PROP)); - Assert.assertEquals("20", summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP)); + assertThat(summary) + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "20") + .doesNotContainKey(SnapshotSummary.REMOVED_FILE_SIZE_PROP); table .newOverwrite() @@ -62,18 +63,20 @@ public void testFileSizeSummary() { .addFile(FILE_D) .commit(); summary = table.currentSnapshot().summary(); - Assert.assertEquals("30", summary.get(SnapshotSummary.ADDED_FILE_SIZE_PROP)); - Assert.assertEquals("20", summary.get(SnapshotSummary.REMOVED_FILE_SIZE_PROP)); - Assert.assertEquals("30", summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP)); + assertThat(summary) + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "30") + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "20") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "30"); table.newDelete().deleteFile(FILE_C).deleteFile(FILE_D).commit(); summary = table.currentSnapshot().summary(); - Assert.assertNull(summary.get(SnapshotSummary.ADDED_FILE_SIZE_PROP)); - Assert.assertEquals("20", summary.get(SnapshotSummary.REMOVED_FILE_SIZE_PROP)); - Assert.assertEquals("10", summary.get(SnapshotSummary.TOTAL_FILE_SIZE_PROP)); + assertThat(summary) + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "20") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .doesNotContainKey(SnapshotSummary.ADDED_FILE_SIZE_PROP); } - @Test + @TestTemplate public void testFileSizeSummaryWithDeletes() { if (formatVersion == 1) { return; @@ -83,7 +86,8 @@ public void testFileSizeSummaryWithDeletes() { table.refresh(); Map summary = table.currentSnapshot().summary(); - Assert.assertEquals("1", summary.get(SnapshotSummary.ADD_EQ_DELETE_FILES_PROP)); - Assert.assertEquals("1", summary.get(SnapshotSummary.ADD_POS_DELETE_FILES_PROP)); + assertThat(summary) + .containsEntry(SnapshotSummary.ADD_EQ_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADD_POS_DELETE_FILES_PROP, "1"); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 826f3ad1e778..1d7d754d8df2 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -61,9 +61,8 @@ import org.apache.iceberg.util.JsonUtil; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestTableMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; @@ -87,7 +86,7 @@ public class TestTableMetadata { .desc(Expressions.bucket("z", 4), NullOrder.NULLS_LAST) .build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; public TableOperations ops = new LocalTableOperations(temp); @@ -1731,7 +1730,7 @@ public void testNoTrailingLocationSlash() { private String createManifestListWithManifestFile( long snapshotId, Long parentSnapshotId, String manifestFile) throws IOException { - File manifestList = temp.newFile("manifests" + UUID.randomUUID()); + File manifestList = File.createTempFile("manifests", null, temp.toFile()); manifestList.deleteOnExit(); try (ManifestListWriter writer = From e8b2e9c7ce160d2a11909d3ba23514819cef8452 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Mon, 11 Mar 2024 19:26:03 +0530 Subject: [PATCH 0115/1019] Data, Flink, Spark: Migrate TestAppenderFactory and subclasses to JUnit5 (#9862) --- .../iceberg/TestGenericAppenderFactory.java | 7 +- .../iceberg/io/TestAppenderFactory.java | 111 +++---- .../apache/iceberg/io/TestBaseTaskWriter.java | 74 ++--- .../io/TestTaskEqualityDeltaWriter.java | 291 +++++++++--------- .../flink/sink/TestFlinkAppenderFactory.java | 7 +- .../flink/sink/TestFlinkAppenderFactory.java | 7 +- .../flink/sink/TestFlinkAppenderFactory.java | 7 +- .../source/TestSparkAppenderFactory.java | 7 +- .../source/TestSparkAppenderFactory.java | 7 +- .../source/TestSparkAppenderFactory.java | 7 +- 10 files changed, 253 insertions(+), 272 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/TestGenericAppenderFactory.java b/data/src/test/java/org/apache/iceberg/TestGenericAppenderFactory.java index 8f27e734b42c..2e4a7b885903 100644 --- a/data/src/test/java/org/apache/iceberg/TestGenericAppenderFactory.java +++ b/data/src/test/java/org/apache/iceberg/TestGenericAppenderFactory.java @@ -30,12 +30,7 @@ public class TestGenericAppenderFactory extends TestAppenderFactory { - private final GenericRecord gRecord; - - public TestGenericAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.gRecord = GenericRecord.create(SCHEMA); - } + private final GenericRecord gRecord = GenericRecord.create(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( 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 b4ae6fd893ca..0b9d60bcc192 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestAppenderFactory.java @@ -18,16 +18,23 @@ */ package org.apache.iceberg.io; +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 java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; @@ -46,45 +53,39 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class TestAppenderFactory extends TableTestBase { - private static final int FORMAT_V2 = 2; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; - private final FileFormat format; - private final boolean partitioned; +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestAppenderFactory extends TestBase { + private static final int FORMAT_V2 = 2; private PartitionKey partition = null; private OutputFileFactory fileFactory = null; - @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {"avro", false}, - new Object[] {"avro", true}, - new Object[] {"orc", false}, - new Object[] {"orc", true}, - new Object[] {"parquet", false}, - new Object[] {"parquet", true} - }; - } - - public TestAppenderFactory(String fileFormat, boolean partitioned) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); - this.partitioned = partitioned; + @Parameter(index = 1) + protected FileFormat format; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "formatVersion = {0}, FileFormat={1}, partitioned={2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {FORMAT_V2, FileFormat.AVRO, false}, + new Object[] {FORMAT_V2, FileFormat.AVRO, true}, + new Object[] {FORMAT_V2, FileFormat.ORC, false}, + new Object[] {FORMAT_V2, FileFormat.ORC, true}, + new Object[] {FORMAT_V2, FileFormat.PARQUET, false}, + new Object[] {FORMAT_V2, FileFormat.PARQUET, true}); } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); // created by table create + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -157,7 +158,7 @@ private DataFile prepareDataFile(List rowSet, FileAppenderFactory appender return writer.toDataFile(); } - @Test + @TestTemplate public void testDataWriter() throws IOException { FileAppenderFactory appenderFactory = createAppenderFactory(null, null, null); @@ -166,11 +167,12 @@ public void testDataWriter() throws IOException { table.newRowDelta().addRows(dataFile).commit(); - Assert.assertEquals( - "Should have the expected records.", expectedRowSet(rowSet), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have the expected records.") + .isEqualTo(expectedRowSet(rowSet)); } - @Test + @TestTemplate public void testEqDeleteWriter() throws IOException { List equalityFieldIds = Lists.newArrayList(table.schema().findField("id").fieldId()); Schema eqDeleteRowSchema = table.schema().select("id"); @@ -198,18 +200,20 @@ public void testEqDeleteWriter() throws IOException { GenericRecord gRecord = GenericRecord.create(eqDeleteRowSchema); Set expectedDeletes = Sets.newHashSet(gRecord.copy("id", 1), gRecord.copy("id", 3), gRecord.copy("id", 5)); - Assert.assertEquals( - expectedDeletes, - Sets.newHashSet(createReader(eqDeleteRowSchema, out.encryptingOutputFile().toInputFile()))); + assertThat( + Sets.newHashSet( + createReader(eqDeleteRowSchema, out.encryptingOutputFile().toInputFile()))) + .isEqualTo(expectedDeletes); table.newRowDelta().addDeletes(eqDeleteWriter.toDeleteFile()).commit(); List expected = Lists.newArrayList(createRow(2, "bbb"), createRow(4, "ddd")); - Assert.assertEquals( - "Should have the expected records", expectedRowSet(expected), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have the expected records") + .isEqualTo(expectedRowSet(expected)); } - @Test + @TestTemplate public void testPosDeleteWriter() throws IOException { // Initialize FileAppenderFactory without pos-delete row schema. FileAppenderFactory appenderFactory = createAppenderFactory(null, null, null); @@ -241,9 +245,9 @@ public void testPosDeleteWriter() throws IOException { gRecord.copy("file_path", dataFile.path(), "pos", 0L), gRecord.copy("file_path", dataFile.path(), "pos", 2L), gRecord.copy("file_path", dataFile.path(), "pos", 4L)); - Assert.assertEquals( - expectedDeletes, - Sets.newHashSet(createReader(pathPosSchema, out.encryptingOutputFile().toInputFile()))); + assertThat( + Sets.newHashSet(createReader(pathPosSchema, out.encryptingOutputFile().toInputFile()))) + .isEqualTo(expectedDeletes); table .newRowDelta() @@ -254,11 +258,12 @@ public void testPosDeleteWriter() throws IOException { .commit(); List expected = Lists.newArrayList(createRow(2, "bbb"), createRow(4, "ddd")); - Assert.assertEquals( - "Should have the expected records", expectedRowSet(expected), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have the expected records") + .isEqualTo(expectedRowSet(expected)); } - @Test + @TestTemplate public void testPosDeleteWriterWithRowSchema() throws IOException { FileAppenderFactory appenderFactory = createAppenderFactory(null, null, table.schema()); @@ -308,9 +313,10 @@ public void testPosDeleteWriterWithRowSchema() throws IOException { 4L, "row", rowRecord.copy("id", 5, "data", "eee"))); - Assert.assertEquals( - expectedDeletes, - Sets.newHashSet(createReader(pathPosRowSchema, out.encryptingOutputFile().toInputFile()))); + assertThat( + Sets.newHashSet( + createReader(pathPosRowSchema, out.encryptingOutputFile().toInputFile()))) + .isEqualTo(expectedDeletes); table .newRowDelta() @@ -321,8 +327,9 @@ public void testPosDeleteWriterWithRowSchema() throws IOException { .commit(); List expected = Lists.newArrayList(createRow(2, "bbb"), createRow(4, "ddd")); - Assert.assertEquals( - "Should have the expected records", expectedRowSet(expected), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have the expected records") + .isEqualTo(expectedRowSet(expected)); } private CloseableIterable createReader(Schema schema, InputFile inputFile) { 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 0c478ad802d1..2a235f5d7428 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -27,46 +29,47 @@ import java.util.List; import java.util.stream.Collectors; 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.RowDelta; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestBaseTaskWriter extends TableTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestBaseTaskWriter extends TestBase { private static final int FORMAT_V2 = 2; - private final FileFormat format; private final GenericRecord gRecord = GenericRecord.create(SCHEMA); private OutputFileFactory fileFactory = null; private FileAppenderFactory appenderFactory = null; - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } + @Parameter(index = 1) + protected FileFormat format; - public TestBaseTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); + @Parameters(name = "formatVersion = {0}, FileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {FORMAT_V2, FileFormat.AVRO}, + new Object[] {FORMAT_V2, FileFormat.ORC}, + new Object[] {FORMAT_V2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); // created by table create + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -90,23 +93,23 @@ private Record createRecord(Integer id, String data) { return gRecord.copy("id", id, "data", data); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TestTaskWriter writer = createTaskWriter(128 * 1024 * 1024)) { writer.close(); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(0); + assertThat(result.deleteFiles()).hasSize(0); writer.close(); result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(0); + assertThat(result.deleteFiles()).hasSize(0); } } - @Test + @TestTemplate public void testAbort() throws IOException { List records = Lists.newArrayList(); for (int i = 0; i < 2000; i++) { @@ -128,18 +131,18 @@ public void testAbort() throws IOException { Files.list(Paths.get(tableDir.getPath(), "data")) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals("Should have 4 files but the files are: " + files, 4, files.size()); + assertThat(files).as("Should have 4 files but the files are: " + files).hasSize(4); // Abort to clean all delete files and data files. taskWriter.abort(); } for (Path path : files) { - Assert.assertFalse(Files.exists(path)); + assertThat(path).doesNotExist(); } } - @Test + @TestTemplate public void testRollIfExceedTargetFileSize() throws IOException { List records = Lists.newArrayListWithCapacity(8000); for (int i = 0; i < 2000; i++) { @@ -156,8 +159,8 @@ public void testRollIfExceedTargetFileSize() throws IOException { } result = taskWriter.complete(); - Assert.assertEquals(8, result.dataFiles().length); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(8); + assertThat(result.deleteFiles()).hasSize(0); } RowDelta rowDelta = table.newRowDelta(); @@ -178,8 +181,8 @@ public void testRollIfExceedTargetFileSize() throws IOException { } result = taskWriter.complete(); - Assert.assertEquals(8, result.dataFiles().length); - Assert.assertEquals(8, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(8); + assertThat(result.deleteFiles()).hasSize(8); } rowDelta = table.newRowDelta(); @@ -187,8 +190,9 @@ public void testRollIfExceedTargetFileSize() throws IOException { Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); rowDelta.commit(); - Assert.assertEquals( - "Should have expected records", expectedRowSet(expected), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo(expectedRowSet(expected)); } private StructLikeSet expectedRowSet(Iterable records) { 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 28543277a50d..4910b74a2abe 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -28,12 +30,15 @@ import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.GenericRecord; @@ -48,18 +53,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ArrayUtil; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestTaskEqualityDeltaWriter extends TableTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTaskEqualityDeltaWriter extends TestBase { private static final int FORMAT_V2 = 2; private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024L; - - private final FileFormat format; private final GenericRecord gRecord = GenericRecord.create(SCHEMA); private final GenericRecord posRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema()); @@ -67,21 +68,22 @@ public class TestTaskEqualityDeltaWriter extends TableTestBase { private int idFieldId; private int dataFieldId; - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } + @Parameter(index = 1) + protected FileFormat format; - public TestTaskEqualityDeltaWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); + @Parameters(name = "formatVersion = {0}, FileFormat = {0}") + protected static List parameters() { + return Arrays.asList( + new Object[] {FORMAT_V2, FileFormat.AVRO}, + new Object[] {FORMAT_V2, FileFormat.ORC}, + new Object[] {FORMAT_V2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); // created by table create + this.tableDir = java.nio.file.Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -98,7 +100,7 @@ private Record createRecord(Integer id, String data) { return gRecord.copy("id", id, "data", data); } - @Test + @TestTemplate public void testPureInsert() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(idFieldId, dataFieldId); Schema eqDeleteRowSchema = table.schema(); @@ -113,11 +115,12 @@ public void testPureInsert() throws IOException { } WriteResult result = deltaWriter.complete(); - Assert.assertEquals("Should only have a data file.", 1, result.dataFiles().length); - Assert.assertEquals("Should have no delete file", 0, result.deleteFiles().length); + assertThat(result.dataFiles()).as("Should only have a data file.").hasSize(1); + assertThat(result.deleteFiles()).as("Should have no delete file").hasSize(0); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(expected), actualRowSet("*")); + assertThat(expectedRowSet(expected)) + .as("Should have expected records") + .isEqualTo(actualRowSet("*")); deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); for (int i = 20; i < 30; i++) { @@ -127,14 +130,15 @@ public void testPureInsert() throws IOException { deltaWriter.write(record); } result = deltaWriter.complete(); - Assert.assertEquals("Should only have a data file.", 1, result.dataFiles().length); - Assert.assertEquals("Should have no delete file", 0, result.deleteFiles().length); + assertThat(result.dataFiles()).as("Should only have a data file.").hasSize(1); + assertThat(result.deleteFiles()).as("Should have no delete file").hasSize(0); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(expected), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo(expectedRowSet(expected)); } - @Test + @TestTemplate public void testInsertDuplicatedKey() throws IOException { List equalityFieldIds = Lists.newArrayList(idFieldId); Schema eqDeleteRowSchema = table.schema(); @@ -152,48 +156,49 @@ public void testInsertDuplicatedKey() throws IOException { WriteResult result = deltaWriter.complete(); commitTransaction(result); - Assert.assertEquals("Should have a data file.", 1, result.dataFiles().length); - Assert.assertEquals("Should have a pos-delete file", 1, result.deleteFiles().length); + assertThat(result.dataFiles()).as("Should have a data file.").hasSize(1); + assertThat(result.deleteFiles()).as("Should have a pos-delete file").hasSize(1); DeleteFile posDeleteFile = result.deleteFiles()[0]; - Assert.assertEquals( - "Should be a pos-delete file", FileContent.POSITION_DELETES, posDeleteFile.content()); - Assert.assertEquals(1, result.referencedDataFiles().length); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( + assertThat(posDeleteFile.content()) + .as("Should be a pos-delete file") + .isEqualTo(FileContent.POSITION_DELETES); + assertThat(result.referencedDataFiles()).hasSize(1); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo( + expectedRowSet( + ImmutableList.of( + createRecord(4, "eee"), + createRecord(3, "fff"), + createRecord(2, "ggg"), + createRecord(1, "hhh")))); + + // Check records in the data file. + DataFile dataFile = result.dataFiles()[0]; + assertThat(readRecordsAsList(table.schema(), dataFile.path())) + .isEqualTo( ImmutableList.of( + createRecord(1, "aaa"), + createRecord(2, "bbb"), + createRecord(3, "ccc"), + createRecord(4, "ddd"), createRecord(4, "eee"), createRecord(3, "fff"), createRecord(2, "ggg"), - createRecord(1, "hhh"))), - actualRowSet("*")); - - // Check records in the data file. - DataFile dataFile = result.dataFiles()[0]; - Assert.assertEquals( - ImmutableList.of( - createRecord(1, "aaa"), - createRecord(2, "bbb"), - createRecord(3, "ccc"), - createRecord(4, "ddd"), - createRecord(4, "eee"), - createRecord(3, "fff"), - createRecord(2, "ggg"), - createRecord(1, "hhh")), - readRecordsAsList(table.schema(), dataFile.path())); + createRecord(1, "hhh"))); // Check records in the pos-delete file. Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); - Assert.assertEquals( - 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)), - readRecordsAsList(posDeleteSchema, posDeleteFile.path())); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) + .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))); } - @Test + @TestTemplate public void testUpsertSameRow() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(idFieldId, dataFieldId); Schema eqDeleteRowSchema = table.schema(); @@ -208,36 +213,35 @@ public void testUpsertSameRow() throws IOException { deltaWriter.write(record); WriteResult result = deltaWriter.complete(); - Assert.assertEquals("Should have a data file.", 1, result.dataFiles().length); - Assert.assertEquals("Should have a pos-delete file.", 1, result.deleteFiles().length); + assertThat(result.dataFiles()).as("Should have a data file.").hasSize(1); + assertThat(result.deleteFiles()).as("Should have a pos-delete file").hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have an expected record", - expectedRowSet(ImmutableList.of(record)), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have an expected record") + .isEqualTo(expectedRowSet(ImmutableList.of(record))); // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - Assert.assertEquals( - ImmutableList.of(record, record), readRecordsAsList(table.schema(), dataFile.path())); + assertThat(readRecordsAsList(table.schema(), dataFile.path())) + .isEqualTo(ImmutableList.of(record, record)); // Check records in the pos-delete file. DeleteFile posDeleteFile = result.deleteFiles()[0]; - Assert.assertEquals( - ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L)), - readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.path())); + assertThat(readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.path())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); deltaWriter.delete(record); result = deltaWriter.complete(); - Assert.assertEquals("Should have 0 data file.", 0, result.dataFiles().length); - Assert.assertEquals("Should have 1 eq-delete file", 1, result.deleteFiles().length); + assertThat(result.dataFiles()).as("Should have 0 data file.").hasSize(0); + assertThat(result.deleteFiles()).as("Should have 1 eq-delete file").hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have no record", expectedRowSet(ImmutableList.of()), actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have no record") + .isEqualTo(expectedRowSet(ImmutableList.of())); } - @Test + @TestTemplate public void testUpsertData() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(dataFieldId); Schema eqDeleteRowSchema = table.schema().select("data"); @@ -251,22 +255,22 @@ public void testUpsertData() throws IOException { // Commit the 1th transaction. WriteResult result = deltaWriter.complete(); - Assert.assertEquals("Should have a data file", 1, result.dataFiles().length); - Assert.assertEquals( - "Should have a pos-delete file for deduplication purpose", 1, result.deleteFiles().length); - Assert.assertEquals( - "Should be pos-delete file", - FileContent.POSITION_DELETES, - result.deleteFiles()[0].content()); - Assert.assertEquals(1, result.referencedDataFiles().length); + assertThat(result.dataFiles()).as("Should have a data file").hasSize(1); + assertThat(result.deleteFiles()) + .as("Should have a pos-delete file for deduplication purpose") + .hasSize(1); + assertThat(result.deleteFiles()[0].content()) + .as("Should be pos-delete file") + .isEqualTo(FileContent.POSITION_DELETES); + assertThat(result.referencedDataFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - ImmutableList.of( - createRecord(2, "bbb"), createRecord(3, "aaa"), createRecord(4, "ccc"))), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo( + expectedRowSet( + ImmutableList.of( + createRecord(2, "bbb"), createRecord(3, "aaa"), createRecord(4, "ccc")))); // Start the 2nd transaction. deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); @@ -290,38 +294,38 @@ public void testUpsertData() throws IOException { // Commit the 2nd transaction. result = deltaWriter.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(2, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(2); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(ImmutableList.of(createRecord(6, "aaa"), createRecord(7, "ccc"))), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo( + expectedRowSet(ImmutableList.of(createRecord(6, "aaa"), createRecord(7, "ccc")))); // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - Assert.assertEquals( - ImmutableList.of(createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc")), - readRecordsAsList(table.schema(), dataFile.path())); + assertThat(readRecordsAsList(table.schema(), dataFile.path())) + .isEqualTo( + ImmutableList.of( + createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc"))); // Check records in the eq-delete file. DeleteFile eqDeleteFile = result.deleteFiles()[0]; - Assert.assertEquals(FileContent.EQUALITY_DELETES, eqDeleteFile.content()); - Assert.assertEquals( - ImmutableList.of(keyFunc.apply("aaa"), keyFunc.apply("ccc"), keyFunc.apply("bbb")), - readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())); + assertThat(eqDeleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())) + .isEqualTo( + ImmutableList.of(keyFunc.apply("aaa"), keyFunc.apply("ccc"), keyFunc.apply("bbb"))); // Check records in the pos-delete file. DeleteFile posDeleteFile = result.deleteFiles()[1]; Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); - Assert.assertEquals(FileContent.POSITION_DELETES, posDeleteFile.content()); - Assert.assertEquals( - ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L)), - readRecordsAsList(posDeleteSchema, posDeleteFile.path())); + assertThat(posDeleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); } - @Test + @TestTemplate public void testUpsertDataWithFullRowSchema() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(dataFieldId); Schema eqDeleteRowSchema = table.schema(); @@ -335,22 +339,22 @@ public void testUpsertDataWithFullRowSchema() throws IOException { // Commit the 1th transaction. WriteResult result = deltaWriter.complete(); - Assert.assertEquals("Should have a data file", 1, result.dataFiles().length); - Assert.assertEquals( - "Should have a pos-delete file for deduplication purpose", 1, result.deleteFiles().length); - Assert.assertEquals( - "Should be pos-delete file", - FileContent.POSITION_DELETES, - result.deleteFiles()[0].content()); - Assert.assertEquals(1, result.referencedDataFiles().length); + assertThat(result.dataFiles()).as("Should have a data file").hasSize(1); + assertThat(result.deleteFiles()) + .as("Should have a pos-delete file for deduplication purpose") + .hasSize(1); + assertThat(result.deleteFiles()[0].content()) + .as("Should be pos-delete file") + .isEqualTo(FileContent.POSITION_DELETES); + assertThat(result.referencedDataFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - ImmutableList.of( - createRecord(2, "bbb"), createRecord(3, "aaa"), createRecord(4, "ccc"))), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo( + expectedRowSet( + ImmutableList.of( + createRecord(2, "bbb"), createRecord(3, "aaa"), createRecord(4, "ccc")))); // Start the 2nd transaction. deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); @@ -372,36 +376,37 @@ public void testUpsertDataWithFullRowSchema() throws IOException { // Commit the 2nd transaction. result = deltaWriter.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(2, result.deleteFiles().length); - Assert.assertEquals(1, result.referencedDataFiles().length); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(2); + assertThat(result.referencedDataFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(ImmutableList.of(createRecord(6, "aaa"), createRecord(7, "ccc"))), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .as("Should have expected records") + .isEqualTo( + expectedRowSet(ImmutableList.of(createRecord(6, "aaa"), createRecord(7, "ccc")))); // Check records in the data file. DataFile dataFile = result.dataFiles()[0]; - Assert.assertEquals( - ImmutableList.of(createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc")), - readRecordsAsList(table.schema(), dataFile.path())); + assertThat(readRecordsAsList(table.schema(), dataFile.path())) + .isEqualTo( + ImmutableList.of( + createRecord(5, "aaa"), createRecord(6, "aaa"), createRecord(7, "ccc"))); // Check records in the eq-delete file. DeleteFile eqDeleteFile = result.deleteFiles()[0]; - Assert.assertEquals(FileContent.EQUALITY_DELETES, eqDeleteFile.content()); - Assert.assertEquals( - ImmutableList.of(createRecord(3, "aaa"), createRecord(4, "ccc"), createRecord(2, "bbb")), - readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())); + assertThat(eqDeleteFile.content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(readRecordsAsList(eqDeleteRowSchema, eqDeleteFile.path())) + .isEqualTo( + ImmutableList.of( + createRecord(3, "aaa"), createRecord(4, "ccc"), createRecord(2, "bbb"))); // Check records in the pos-delete file. DeleteFile posDeleteFile = result.deleteFiles()[1]; Schema posDeleteSchema = DeleteSchemaUtil.pathPosSchema(); - Assert.assertEquals(FileContent.POSITION_DELETES, posDeleteFile.content()); - Assert.assertEquals( - ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L)), - readRecordsAsList(posDeleteSchema, posDeleteFile.path())); + assertThat(posDeleteFile.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(readRecordsAsList(posDeleteSchema, posDeleteFile.path())) + .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); } private void commitTransaction(WriteResult result) { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java index d25b2792ac65..dd89f43483b0 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java @@ -32,12 +32,7 @@ public class TestFlinkAppenderFactory extends TestAppenderFactory { - private final RowType rowType; - - public TestFlinkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.rowType = FlinkSchemaUtil.convert(SCHEMA); - } + private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java index d25b2792ac65..dd89f43483b0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java @@ -32,12 +32,7 @@ public class TestFlinkAppenderFactory extends TestAppenderFactory { - private final RowType rowType; - - public TestFlinkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.rowType = FlinkSchemaUtil.convert(SCHEMA); - } + private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java index d25b2792ac65..dd89f43483b0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java @@ -32,12 +32,7 @@ public class TestFlinkAppenderFactory extends TestAppenderFactory { - private final RowType rowType; - - public TestFlinkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.rowType = FlinkSchemaUtil.convert(SCHEMA); - } + private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( 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 index 3fb2a630fe81..1f4c613f749e 100644 --- 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 @@ -32,12 +32,7 @@ public class TestSparkAppenderFactory extends TestAppenderFactory { - private final StructType sparkType; - - public TestSparkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.sparkType = SparkSchemaUtil.convert(SCHEMA); - } + private final StructType sparkType = SparkSchemaUtil.convert(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( 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 index 3fb2a630fe81..1f4c613f749e 100644 --- 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 @@ -32,12 +32,7 @@ public class TestSparkAppenderFactory extends TestAppenderFactory { - private final StructType sparkType; - - public TestSparkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.sparkType = SparkSchemaUtil.convert(SCHEMA); - } + private final StructType sparkType = SparkSchemaUtil.convert(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( 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 index 3fb2a630fe81..1f4c613f749e 100644 --- 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 @@ -32,12 +32,7 @@ public class TestSparkAppenderFactory extends TestAppenderFactory { - private final StructType sparkType; - - public TestSparkAppenderFactory(String fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - this.sparkType = SparkSchemaUtil.convert(SCHEMA); - } + private final StructType sparkType = SparkSchemaUtil.convert(SCHEMA); @Override protected FileAppenderFactory createAppenderFactory( From 4c5c67920c4d9d1bf4a863f1091ccaac6b3fa63e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 11 Mar 2024 20:10:44 +0530 Subject: [PATCH 0116/1019] Infra: Add 1.5.0 to issue template (#9778) --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 51efe443f5f5..f2ea3f8aab4c 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.4.3 (latest release)" + - "1.5.0 (latest release)" + - "1.4.3" - "1.4.2" - "1.4.1" - "1.4.0" @@ -38,14 +39,6 @@ body: - "1.2.0" - "1.1.0" - "1.0.0" - - "0.14.1" - - "0.14.0" - - "0.13.1" - - "0.13.0" - - "0.12.1" - - "0.12.0" - - "0.11.1" - - "0.11.0" - "main (development)" validations: required: false From 970171846396d538d666ca23e450da049384965e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 11 Mar 2024 20:13:35 +0530 Subject: [PATCH 0117/1019] Update ASF DOAP file (#9922) --- doap.rdf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doap.rdf b/doap.rdf index 5b82f864e51e..d3ef2e34497c 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.4.3 - 2023-12-27 - 1.4.3 + 1.5.0 + 2024-03-11 + 1.5.0 From 0aef4c95cdbad831fb35ad36ad9ffaf99708fc58 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 11 Mar 2024 20:14:07 +0530 Subject: [PATCH 0118/1019] Build: Let revapi compare against 1.5.0 (#9777) --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 90675cfb9800..64b4299d09bd 100644 --- a/build.gradle +++ b/build.gradle @@ -144,7 +144,7 @@ subprojects { revapi { oldGroup = project.group oldName = project.name - oldVersion = "1.4.0" + oldVersion = "1.5.0" } tasks.register('showDeprecationRulesOnRevApiFailure') { From 36bbd2364ca9e40e02d33b4036179a817355cca5 Mon Sep 17 00:00:00 2001 From: "Brian \"bits\" Olsen" Date: Mon, 11 Mar 2024 16:21:15 -0500 Subject: [PATCH 0119/1019] Update site to 1.5.0 docs (#9931) --- site/mkdocs.yml | 2 +- site/nav.yml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 2b3630ad4f71..da7810f2ed18 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.4.3' + icebergVersion: '1.5.0' social: - icon: fontawesome/regular/comments link: 'https://iceberg.apache.org/community/' diff --git a/site/nav.yml b/site/nav.yml index c7d9a26ae0e9..f315e0ae9616 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -22,6 +22,7 @@ nav: - Hive: hive-quickstart.md - Docs: - latest: '!include docs/docs/latest/mkdocs.yml' + - 1.5.0: '!include docs/docs/1.5.0/mkdocs.yml' - 1.4.3: '!include docs/docs/1.4.3/mkdocs.yml' - 1.4.2: '!include docs/docs/1.4.2/mkdocs.yml' - 1.4.1: '!include docs/docs/1.4.1/mkdocs.yml' From 8ad7a69dc9881c3fbad2ba736b7a972c50c644e4 Mon Sep 17 00:00:00 2001 From: Himadri Pal Date: Mon, 11 Mar 2024 14:22:28 -0700 Subject: [PATCH 0120/1019] Make OAuth `audience` and `resource` configurable (#9839) --- .../aws/s3/signer/S3V4RestSignerClient.java | 28 ++- .../iceberg/rest/RESTSessionCatalog.java | 8 +- .../iceberg/rest/auth/OAuth2Properties.java | 6 + .../apache/iceberg/rest/auth/OAuth2Util.java | 163 +++++++++++++++--- .../apache/iceberg/rest/TestRESTCatalog.java | 76 ++++++-- 5 files changed, 234 insertions(+), 47 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java index f284822161ca..cdbdfb3d869e 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java @@ -118,6 +118,11 @@ public String oauth2ServerUri() { return properties().getOrDefault(OAuth2Properties.OAUTH2_SERVER_URI, ResourcePaths.tokens()); } + @Value.Lazy + public Map optionalOAuthParams() { + return OAuth2Util.buildOptionalParam(properties()); + } + /** A Bearer token supplier which will be used for interaction with the server. */ @Value.Default public Supplier token() { @@ -207,7 +212,13 @@ private AuthSession authSession() { token, expiresAtMillis(properties()), new AuthSession( - ImmutableMap.of(), token, null, credential(), SCOPE, oauth2ServerUri()))); + ImmutableMap.of(), + token, + null, + credential(), + SCOPE, + oauth2ServerUri(), + optionalOAuthParams()))); } if (credentialProvided()) { @@ -217,11 +228,22 @@ private AuthSession authSession() { id -> { AuthSession session = new AuthSession( - ImmutableMap.of(), null, null, credential(), SCOPE, oauth2ServerUri()); + ImmutableMap.of(), + null, + null, + credential(), + SCOPE, + oauth2ServerUri(), + optionalOAuthParams()); long startTimeMillis = System.currentTimeMillis(); OAuthTokenResponse authResponse = OAuth2Util.fetchToken( - httpClient(), session.headers(), credential(), SCOPE, oauth2ServerUri()); + httpClient(), + session.headers(), + credential(), + SCOPE, + oauth2ServerUri(), + optionalOAuthParams()); return AuthSession.fromTokenResponse( httpClient(), tokenRefreshExecutor(), authResponse, startTimeMillis, session); }); 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 a34f738c318e..96aa14b128da 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -179,6 +179,7 @@ public void initialize(String name, Map unresolved) { OAuthTokenResponse authResponse; String credential = props.get(OAuth2Properties.CREDENTIAL); String scope = props.getOrDefault(OAuth2Properties.SCOPE, OAuth2Properties.CATALOG_SCOPE); + Map optionalOAuthParams = OAuth2Util.buildOptionalParam(props); String oauth2ServerUri = props.getOrDefault(OAuth2Properties.OAUTH2_SERVER_URI, ResourcePaths.tokens()); try (RESTClient initClient = clientBuilder.apply(props)) { @@ -186,7 +187,8 @@ public void initialize(String name, Map unresolved) { RESTUtil.merge(configHeaders(props), OAuth2Util.authHeaders(initToken)); if (credential != null && !credential.isEmpty()) { authResponse = - OAuth2Util.fetchToken(initClient, initHeaders, credential, scope, oauth2ServerUri); + OAuth2Util.fetchToken( + initClient, initHeaders, credential, scope, oauth2ServerUri, optionalOAuthParams); Map authHeaders = RESTUtil.merge(initHeaders, OAuth2Util.authHeaders(authResponse.token())); config = fetchConfig(initClient, authHeaders, props); @@ -213,7 +215,9 @@ public void initialize(String name, Map unresolved) { this.paths = ResourcePaths.forCatalogProperties(mergedProps); String token = mergedProps.get(OAuth2Properties.TOKEN); - this.catalogAuth = new AuthSession(baseHeaders, null, null, credential, scope, oauth2ServerUri); + this.catalogAuth = + new AuthSession( + baseHeaders, null, null, credential, scope, oauth2ServerUri, optionalOAuthParams); if (authResponse != null) { this.catalogAuth = AuthSession.fromTokenResponse( diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java index e1a9181d164d..295e24519129 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Properties.java @@ -49,6 +49,12 @@ private OAuth2Properties() {} /** Additional scope for OAuth2. */ public static final String SCOPE = "scope"; + /** Optional param audience for OAuth2. */ + public static final String AUDIENCE = "audience"; + + /** Optional param resource for OAuth2. */ + public static final String RESOURCE = "resource"; + /** Scope for OAuth2 flows. */ public static final String CATALOG_SCOPE = "catalog"; diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index ad1821a3f2b6..9e36694508d9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -40,6 +40,7 @@ import org.apache.iceberg.relocated.com.google.common.base.Splitter; 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.Sets; import org.apache.iceberg.rest.ErrorHandlers; import org.apache.iceberg.rest.RESTClient; @@ -129,18 +130,40 @@ public static String toScope(Iterable scopes) { return SCOPE_JOINER.join(scopes); } + public static Map buildOptionalParam(Map properties) { + // these are some options oauth params based on specification + // for any new optional oauth param, define the constant and add the constant to this list + Set optionalParamKeys = + ImmutableSet.of(OAuth2Properties.AUDIENCE, OAuth2Properties.RESOURCE); + ImmutableMap.Builder optionalParamBuilder = ImmutableMap.builder(); + // add scope too, + optionalParamBuilder.put( + OAuth2Properties.SCOPE, + properties.getOrDefault(OAuth2Properties.SCOPE, OAuth2Properties.CATALOG_SCOPE)); + // add all other parameters + for (String key : optionalParamKeys) { + String value = properties.get(key); + if (value != null) { + optionalParamBuilder.put(key, value); + } + } + return optionalParamBuilder.buildKeepingLast(); + } + private static OAuthTokenResponse refreshToken( RESTClient client, Map headers, String subjectToken, String subjectTokenType, String scope, - String oauth2ServerUri) { + String oauth2ServerUri, + Map optionalOAuthParams) { Map request = tokenExchangeRequest( subjectToken, subjectTokenType, - scope != null ? ImmutableList.of(scope) : ImmutableList.of()); + scope != null ? ImmutableList.of(scope) : ImmutableList.of(), + optionalOAuthParams); OAuthTokenResponse response = client.postForm( @@ -162,14 +185,16 @@ public static OAuthTokenResponse exchangeToken( String actorToken, String actorTokenType, String scope, - String oauth2ServerUri) { + String oauth2ServerUri, + Map optionalParams) { Map request = tokenExchangeRequest( subjectToken, subjectTokenType, actorToken, actorTokenType, - scope != null ? ImmutableList.of(scope) : ImmutableList.of()); + scope != null ? ImmutableList.of(scope) : ImmutableList.of(), + optionalParams); OAuthTokenResponse response = client.postForm( @@ -199,7 +224,29 @@ public static OAuthTokenResponse exchangeToken( actorToken, actorTokenType, scope, - ResourcePaths.tokens()); + ResourcePaths.tokens(), + ImmutableMap.of()); + } + + public static OAuthTokenResponse exchangeToken( + RESTClient client, + Map headers, + String subjectToken, + String subjectTokenType, + String actorToken, + String actorTokenType, + String scope, + String oauth2ServerUri) { + return exchangeToken( + client, + headers, + subjectToken, + subjectTokenType, + actorToken, + actorTokenType, + scope, + oauth2ServerUri, + ImmutableMap.of()); } public static OAuthTokenResponse fetchToken( @@ -207,10 +254,13 @@ public static OAuthTokenResponse fetchToken( Map headers, String credential, String scope, - String oauth2ServerUri) { + String oauth2ServerUri, + Map optionalParams) { Map request = clientCredentialsRequest( - credential, scope != null ? ImmutableList.of(scope) : ImmutableList.of()); + credential, + scope != null ? ImmutableList.of(scope) : ImmutableList.of(), + optionalParams); OAuthTokenResponse response = client.postForm( @@ -227,12 +277,27 @@ public static OAuthTokenResponse fetchToken( public static OAuthTokenResponse fetchToken( RESTClient client, Map headers, String credential, String scope) { - return fetchToken(client, headers, credential, scope, ResourcePaths.tokens()); + return fetchToken( + client, headers, credential, scope, ResourcePaths.tokens(), ImmutableMap.of()); + } + + public static OAuthTokenResponse fetchToken( + RESTClient client, + Map headers, + String credential, + String scope, + String oauth2ServerUri) { + + return fetchToken(client, headers, credential, scope, oauth2ServerUri, ImmutableMap.of()); } private static Map tokenExchangeRequest( - String subjectToken, String subjectTokenType, List scopes) { - return tokenExchangeRequest(subjectToken, subjectTokenType, null, null, scopes); + String subjectToken, + String subjectTokenType, + List scopes, + Map optionalOAuthParams) { + return tokenExchangeRequest( + subjectToken, subjectTokenType, null, null, scopes, optionalOAuthParams); } private static Map tokenExchangeRequest( @@ -240,7 +305,8 @@ private static Map tokenExchangeRequest( String subjectTokenType, String actorToken, String actorTokenType, - List scopes) { + List scopes, + Map optionalParams) { Preconditions.checkArgument( VALID_TOKEN_TYPES.contains(subjectTokenType), "Invalid token type: %s", subjectTokenType); Preconditions.checkArgument( @@ -257,8 +323,9 @@ private static Map tokenExchangeRequest( formData.put(ACTOR_TOKEN, actorToken); formData.put(ACTOR_TOKEN_TYPE, actorTokenType); } + formData.putAll(optionalParams); - return formData.build(); + return formData.buildKeepingLast(); } private static Pair parseCredential(String credential) { @@ -278,13 +345,17 @@ private static Pair parseCredential(String credential) { } private static Map clientCredentialsRequest( - String credential, List scopes) { + String credential, List scopes, Map optionalOAuthParams) { Pair credentialPair = parseCredential(credential); - return clientCredentialsRequest(credentialPair.first(), credentialPair.second(), scopes); + return clientCredentialsRequest( + credentialPair.first(), credentialPair.second(), scopes, optionalOAuthParams); } private static Map clientCredentialsRequest( - String clientId, String clientSecret, List scopes) { + String clientId, + String clientSecret, + List scopes, + Map optionalOAuthParams) { ImmutableMap.Builder formData = ImmutableMap.builder(); formData.put(GRANT_TYPE, CLIENT_CREDENTIALS); if (clientId != null) { @@ -292,8 +363,9 @@ private static Map clientCredentialsRequest( } formData.put(CLIENT_SECRET, clientSecret); formData.put(SCOPE, toScope(scopes)); + formData.putAll(optionalOAuthParams); - return formData.build(); + return formData.buildKeepingLast(); } public static String tokenResponseToJson(OAuthTokenResponse response) { @@ -394,13 +466,16 @@ public static class AuthSession { private volatile boolean keepRefreshed = true; private final String oauth2ServerUri; + private Map optionalOAuthParams = ImmutableMap.of(); + public AuthSession( Map baseHeaders, String token, String tokenType, String credential, String scope, - String oauth2ServerUri) { + String oauth2ServerUri, + Map optionalOAuthParams) { this.headers = RESTUtil.merge(baseHeaders, authHeaders(token)); this.token = token; this.tokenType = tokenType; @@ -408,6 +483,7 @@ public AuthSession( this.credential = credential; this.scope = scope; this.oauth2ServerUri = oauth2ServerUri; + this.optionalOAuthParams = optionalOAuthParams; } /** @deprecated since 1.5.0, will be removed in 1.6.0 */ @@ -427,6 +503,25 @@ public AuthSession( this.oauth2ServerUri = ResourcePaths.tokens(); } + /** @deprecated since 1.6.0, will be removed in 1.7.0 */ + @Deprecated + public AuthSession( + Map baseHeaders, + String token, + String tokenType, + String credential, + String scope, + String oauth2ServerUri) { + this.headers = RESTUtil.merge(baseHeaders, authHeaders(token)); + this.token = token; + this.tokenType = tokenType; + this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); + this.credential = credential; + this.scope = scope; + this.oauth2ServerUri = oauth2ServerUri; + this.optionalOAuthParams = ImmutableMap.of(); + } + public Map headers() { return headers; } @@ -459,6 +554,10 @@ public String oauth2ServerUri() { return oauth2ServerUri; } + public Map optionalOAuthParams() { + return optionalOAuthParams; + } + @VisibleForTesting static void setTokenRefreshNumRetries(int retries) { tokenRefreshNumRetries = retries; @@ -471,7 +570,13 @@ static void setTokenRefreshNumRetries(int retries) { */ public static AuthSession empty() { return new AuthSession( - ImmutableMap.of(), null, null, null, OAuth2Properties.CATALOG_SCOPE, null); + ImmutableMap.of(), + null, + null, + null, + OAuth2Properties.CATALOG_SCOPE, + null, + ImmutableMap.of()); } /** @@ -526,14 +631,16 @@ private OAuthTokenResponse refreshCurrentToken(RESTClient client) { return refreshExpiredToken(client); } else { // attempt a normal refresh - return refreshToken(client, headers(), token, tokenType, scope, oauth2ServerUri); + return refreshToken( + client, headers(), token, tokenType, scope, oauth2ServerUri, optionalOAuthParams); } } private OAuthTokenResponse refreshExpiredToken(RESTClient client) { if (credential != null) { Map basicHeaders = RESTUtil.merge(headers(), basicAuthHeaders(credential)); - return refreshToken(client, basicHeaders, token, tokenType, scope, oauth2ServerUri); + return refreshToken( + client, basicHeaders, token, tokenType, scope, oauth2ServerUri, optionalOAuthParams); } return null; @@ -590,7 +697,8 @@ public static AuthSession fromAccessToken( OAuth2Properties.ACCESS_TOKEN_TYPE, parent.credential(), parent.scope(), - parent.oauth2ServerUri()); + parent.oauth2ServerUri(), + parent.optionalOAuthParams()); long startTimeMillis = System.currentTimeMillis(); Long expiresAtMillis = session.expiresAtMillis(); @@ -629,7 +737,12 @@ public static AuthSession fromCredential( long startTimeMillis = System.currentTimeMillis(); OAuthTokenResponse response = fetchToken( - client, parent.headers(), credential, parent.scope(), parent.oauth2ServerUri()); + client, + parent.headers(), + credential, + parent.scope(), + parent.oauth2ServerUri(), + parent.optionalOAuthParams()); return fromTokenResponse(client, executor, response, startTimeMillis, parent, credential); } @@ -657,7 +770,8 @@ private static AuthSession fromTokenResponse( response.issuedTokenType(), credential, parent.scope(), - parent.oauth2ServerUri()); + parent.oauth2ServerUri(), + parent.optionalOAuthParams()); Long expiresAtMillis = session.expiresAtMillis(); if (null == expiresAtMillis && response.expiresInSeconds() != null) { @@ -687,7 +801,8 @@ public static AuthSession fromTokenExchange( parent.token(), parent.tokenType(), parent.scope(), - parent.oauth2ServerUri()); + parent.oauth2ServerUri(), + parent.optionalOAuthParams()); return fromTokenResponse(client, executor, response, startTimeMillis, parent); } } 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 db67cfd4bb3f..18d832b3cd46 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -678,7 +678,8 @@ public void testClientBearerToken(String oauth2ServerUri) { "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of("Authorization", "Bearer client-bearer-token"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); } @ParameterizedTest @@ -694,7 +695,8 @@ public void testClientCredential(String oauth2ServerUri) { "urn:ietf:params:oauth:token-type:saml2", "saml2-token", "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=user"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); } @ParameterizedTest @@ -710,7 +712,8 @@ public void testClientIDToken(String oauth2ServerUri) { "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=id-token,act=bearer-token"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); } @ParameterizedTest @@ -725,7 +728,25 @@ public void testClientAccessToken(String oauth2ServerUri) { "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=access-token,act=bearer-token"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); + } + + @ParameterizedTest + @ValueSource(strings = {"v1/oauth/tokens", "https://auth-server.com/token"}) + public void testClientAccessTokenWithOptionalParams(String oauth2ServerUri) { + testClientAuth( + "bearer-token", + ImmutableMap.of( + "urn:ietf:params:oauth:token-type:access_token", "access-token", + "urn:ietf:params:oauth:token-type:jwt", "jwt-token", + "urn:ietf:params:oauth:token-type:saml2", "saml2-token", + "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), + ImmutableMap.of( + "Authorization", "Bearer token-exchange-token:sub=access-token,act=bearer-token"), + oauth2ServerUri, + ImmutableMap.of( + "scope", "custom_scope", "audience", "test_audience", "resource", "test_resource")); } @ParameterizedTest @@ -739,7 +760,8 @@ public void testClientJWTToken(String oauth2ServerUri) { "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=jwt-token,act=bearer-token"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); } @ParameterizedTest @@ -752,7 +774,8 @@ public void testClientSAML2Token(String oauth2ServerUri) { "urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=saml2-token,act=bearer-token"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); } @ParameterizedTest @@ -763,14 +786,16 @@ public void testClientSAML1Token(String oauth2ServerUri) { ImmutableMap.of("urn:ietf:params:oauth:token-type:saml1", "saml1-token"), ImmutableMap.of( "Authorization", "Bearer token-exchange-token:sub=saml1-token,act=bearer-token"), - oauth2ServerUri); + oauth2ServerUri, + ImmutableMap.of()); } private void testClientAuth( String catalogToken, Map credentials, Map expectedHeaders, - String oauth2ServerUri) { + String oauth2ServerUri, + Map optionalOAuthParams) { Map catalogHeaders = ImmutableMap.of("Authorization", "Bearer " + catalogToken); RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); @@ -780,15 +805,16 @@ private void testClientAuth( UUID.randomUUID().toString(), "user", credentials, ImmutableMap.of()); RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); - catalog.initialize( - "prod", - ImmutableMap.of( - CatalogProperties.URI, - "ignored", - "token", - catalogToken, - OAuth2Properties.OAUTH2_SERVER_URI, - oauth2ServerUri)); + + ImmutableMap.Builder propertyBuilder = ImmutableMap.builder(); + Map initializationProperties = + propertyBuilder + .put(CatalogProperties.URI, "ignored") + .put("token", catalogToken) + .put(OAuth2Properties.OAUTH2_SERVER_URI, oauth2ServerUri) + .putAll(optionalOAuthParams) + .build(); + catalog.initialize("prod", initializationProperties); Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); @@ -815,7 +841,6 @@ private void testClientAuth( eq(catalogHeaders), any()); } - Mockito.verify(adapter) .execute( eq(HTTPMethod.GET), @@ -825,6 +850,21 @@ private void testClientAuth( eq(LoadTableResponse.class), eq(expectedHeaders), any()); + if (!optionalOAuthParams.isEmpty()) { + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.POST), + eq(oauth2ServerUri), + any(), + Mockito.argThat( + body -> + ((Map) body) + .keySet() + .containsAll(optionalOAuthParams.keySet())), + eq(OAuthTokenResponse.class), + eq(catalogHeaders), + any()); + } } @ParameterizedTest From c554b414e9f4e611df2e81652f6e832e651c714e Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 11 Mar 2024 16:52:48 -0700 Subject: [PATCH 0121/1019] Spark 3.5: Fix system function pushdown in CoW row-level commands (#9873) --- .../optimizer/ReplaceStaticInvoke.scala | 56 ++- .../spark/extensions/SparkPlanUtil.java | 48 +++ ...mFunctionPushDownInRowLevelOperations.java | 348 ++++++++++++++++++ .../spark/functions/BaseScalarFunction.java | 40 ++ .../spark/functions/BucketFunction.java | 3 +- .../iceberg/spark/functions/DaysFunction.java | 3 +- .../spark/functions/HoursFunction.java | 5 +- .../functions/IcebergVersionFunction.java | 3 +- .../spark/functions/MonthsFunction.java | 3 +- .../spark/functions/TruncateFunction.java | 3 +- .../spark/functions/YearsFunction.java | 3 +- 11 files changed, 485 insertions(+), 30 deletions(-) create mode 100644 spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala index 1f0e164d8467..d5c4cb84a02a 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala @@ -22,12 +22,20 @@ import org.apache.iceberg.spark.functions.SparkFunctions import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression import org.apache.spark.sql.catalyst.expressions.BinaryComparison import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.In +import org.apache.spark.sql.catalyst.expressions.InSet import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.ReplaceData import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.BINARY_COMPARISON +import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND import org.apache.spark.sql.catalyst.trees.TreePattern.FILTER +import org.apache.spark.sql.catalyst.trees.TreePattern.IN +import org.apache.spark.sql.catalyst.trees.TreePattern.INSET +import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN import org.apache.spark.sql.connector.catalog.functions.ScalarFunction import org.apache.spark.sql.types.StructField import org.apache.spark.sql.types.StructType @@ -40,21 +48,39 @@ import org.apache.spark.sql.types.StructType object ReplaceStaticInvoke extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning (_.containsAllPatterns(BINARY_COMPARISON, FILTER)) { - case filter @ Filter(condition, _) => - val newCondition = condition.transformWithPruning(_.containsPattern(BINARY_COMPARISON)) { - case c @ BinaryComparison(left: StaticInvoke, right) if canReplace(left) && right.foldable => - c.withNewChildren(Seq(replaceStaticInvoke(left), right)) - - case c @ BinaryComparison(left, right: StaticInvoke) if canReplace(right) && left.foldable => - c.withNewChildren(Seq(left, replaceStaticInvoke(right))) - } - - if (newCondition fastEquals condition) { - filter - } else { - filter.copy(condition = newCondition) - } + plan.transformWithPruning (_.containsAnyPattern(COMMAND, FILTER, JOIN)) { + case replace @ ReplaceData(_, cond, _, _, _, _) => + replaceStaticInvoke(replace, cond, newCond => replace.copy(condition = newCond)) + + case join @ Join(_, _, _, Some(cond), _) => + replaceStaticInvoke(join, cond, newCond => join.copy(condition = Some(newCond))) + + case filter @ Filter(cond, _) => + replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond)) + } + + private def replaceStaticInvoke[T <: LogicalPlan]( + node: T, + condition: Expression, + copy: Expression => T): T = { + val newCondition = replaceStaticInvoke(condition) + if (newCondition fastEquals condition) node else copy(newCondition) + } + + private def replaceStaticInvoke(condition: Expression): Expression = { + condition.transformWithPruning(_.containsAnyPattern(BINARY_COMPARISON, IN, INSET)) { + case in @ In(value: StaticInvoke, _) if canReplace(value) => + in.copy(value = replaceStaticInvoke(value)) + + case in @ InSet(value: StaticInvoke, _) if canReplace(value) => + in.copy(child = replaceStaticInvoke(value)) + + case c @ BinaryComparison(left: StaticInvoke, right) if canReplace(left) && right.foldable => + c.withNewChildren(Seq(replaceStaticInvoke(left), right)) + + case c @ BinaryComparison(left, right: StaticInvoke) if canReplace(right) && left.foldable => + c.withNewChildren(Seq(left, replaceStaticInvoke(right))) + } } private def replaceStaticInvoke(invoke: StaticInvoke): Expression = { diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java index 4f7c3ebadbc5..830d07d86eab 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java @@ -20,12 +20,17 @@ import static scala.collection.JavaConverters.seqAsJavaListConverter; +import java.util.Collection; import java.util.List; +import java.util.function.Predicate; import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.execution.CommandResultExec; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper; import org.apache.spark.sql.execution.datasources.v2.BatchScanExec; +import scala.PartialFunction; import scala.collection.Seq; public class SparkPlanUtil { @@ -53,6 +58,49 @@ private static SparkPlan actualPlan(SparkPlan plan) { } } + public static List collectExprs( + SparkPlan sparkPlan, Predicate predicate) { + Seq> seq = + SPARK_HELPER.collect( + sparkPlan, + new PartialFunction>() { + @Override + public List apply(SparkPlan plan) { + List exprs = Lists.newArrayList(); + + for (Expression expr : toJavaList(plan.expressions())) { + exprs.addAll(collectExprs(expr, predicate)); + } + + return exprs; + } + + @Override + public boolean isDefinedAt(SparkPlan plan) { + return true; + } + }); + return toJavaList(seq).stream().flatMap(Collection::stream).collect(Collectors.toList()); + } + + private static List collectExprs( + Expression expression, Predicate predicate) { + Seq seq = + expression.collect( + new PartialFunction() { + @Override + public Expression apply(Expression expr) { + return expr; + } + + @Override + public boolean isDefinedAt(Expression expr) { + return predicate.test(expr); + } + }); + return toJavaList(seq); + } + private static List toJavaList(Seq seq) { return seqAsJavaListConverter(seq).asJava(); } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java new file mode 100644 index 000000000000..8cb92224fc22 --- /dev/null +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java @@ -0,0 +1,348 @@ +/* + * 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.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; +import org.apache.spark.sql.execution.CommandResultExec; +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec; +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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSystemFunctionPushDownInRowLevelOperations extends ExtensionsTestBase { + + private static final String CHANGES_TABLE_NAME = "changes"; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + } + }; + } + + @BeforeEach + public void beforeEach() { + sql("USE %s", catalogName); + } + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s PURGE", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", tableName(CHANGES_TABLE_NAME)); + } + + @TestTemplate + public void testCopyOnWriteDeleteBucketTransformInPredicate() { + initTable("bucket(4, dep)"); + checkDelete(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testMergeOnReadDeleteBucketTransformInPredicate() { + initTable("bucket(4, dep)"); + checkDelete(MERGE_ON_READ, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testCopyOnWriteDeleteBucketTransformEqPredicate() { + initTable("bucket(4, dep)"); + checkDelete(COPY_ON_WRITE, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testMergeOnReadDeleteBucketTransformEqPredicate() { + initTable("bucket(4, dep)"); + checkDelete(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testCopyOnWriteDeleteYearsTransform() { + initTable("years(ts)"); + checkDelete(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @TestTemplate + public void testMergeOnReadDeleteYearsTransform() { + initTable("years(ts)"); + checkDelete(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @TestTemplate + public void testCopyOnWriteDeleteMonthsTransform() { + initTable("months(ts)"); + checkDelete(COPY_ON_WRITE, "system.months(ts) <= 250"); + } + + @TestTemplate + public void testMergeOnReadDeleteMonthsTransform() { + initTable("months(ts)"); + checkDelete(MERGE_ON_READ, "system.months(ts) > 250"); + } + + @TestTemplate + public void testCopyOnWriteDeleteDaysTransform() { + initTable("days(ts)"); + checkDelete(COPY_ON_WRITE, "system.days(ts) <= date('2000-01-03 00:00:00')"); + } + + @TestTemplate + public void testMergeOnReadDeleteDaysTransform() { + initTable("days(ts)"); + checkDelete(MERGE_ON_READ, "system.days(ts) > date('2000-01-03 00:00:00')"); + } + + @TestTemplate + public void testCopyOnWriteDeleteHoursTransform() { + initTable("hours(ts)"); + checkDelete(COPY_ON_WRITE, "system.hours(ts) <= 100000"); + } + + @TestTemplate + public void testMergeOnReadDeleteHoursTransform() { + initTable("hours(ts)"); + checkDelete(MERGE_ON_READ, "system.hours(ts) > 100000"); + } + + @TestTemplate + public void testCopyOnWriteDeleteTruncateTransform() { + initTable("truncate(1, dep)"); + checkDelete(COPY_ON_WRITE, "system.truncate(1, dep) = 'i'"); + } + + @TestTemplate + public void testMergeOnReadDeleteTruncateTransform() { + initTable("truncate(1, dep)"); + checkDelete(MERGE_ON_READ, "system.truncate(1, dep) = 'i'"); + } + + @TestTemplate + public void testCopyOnWriteUpdateBucketTransform() { + initTable("bucket(4, dep)"); + checkUpdate(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testMergeOnReadUpdateBucketTransform() { + initTable("bucket(4, dep)"); + checkUpdate(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testCopyOnWriteUpdateYearsTransform() { + initTable("years(ts)"); + checkUpdate(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @TestTemplate + public void testMergeOnReadUpdateYearsTransform() { + initTable("years(ts)"); + checkUpdate(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @TestTemplate + public void testCopyOnWriteMergeBucketTransform() { + initTable("bucket(4, dep)"); + checkMerge(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @TestTemplate + public void testMergeOnReadMergeBucketTransform() { + initTable("bucket(4, dep)"); + checkMerge(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @TestTemplate + public void testCopyOnWriteMergeYearsTransform() { + initTable("years(ts)"); + checkMerge(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @TestTemplate + public void testMergeOnReadMergeYearsTransform() { + initTable("years(ts)"); + checkMerge(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @TestTemplate + public void testCopyOnWriteMergeTruncateTransform() { + initTable("truncate(1, dep)"); + checkMerge(COPY_ON_WRITE, "system.truncate(1, dep) = 'i'"); + } + + @TestTemplate + public void testMergeOnReadMergeTruncateTransform() { + initTable("truncate(1, dep)"); + checkMerge(MERGE_ON_READ, "system.truncate(1, dep) = 'i'"); + } + + private void checkDelete(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.DELETE_MODE, + mode.modeName(), + TableProperties.DELETE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = spark.table(tableName).where(cond).limit(2).select("id"); + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + + List calls = + executeAndCollectFunctionCalls( + "DELETE FROM %s t WHERE %s AND t.id IN (SELECT id FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME)); + // CoW planning currently does not optimize post-scan filters in DELETE + int expectedCallCount = mode == COPY_ON_WRITE ? 1 : 0; + assertThat(calls).hasSize(expectedCallCount); + + assertEquals( + "Should have no matching rows", + ImmutableList.of(), + sql( + "SELECT * FROM %s WHERE %s AND id IN (SELECT * FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME))); + }); + } + + private void checkUpdate(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.UPDATE_MODE, + mode.modeName(), + TableProperties.UPDATE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = spark.table(tableName).where(cond).limit(2).select("id"); + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + + List calls = + executeAndCollectFunctionCalls( + "UPDATE %s t SET t.salary = -1 WHERE %s AND t.id IN (SELECT id FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME)); + // CoW planning currently does not optimize post-scan filters in UPDATE + int expectedCallCount = mode == COPY_ON_WRITE ? 2 : 0; + assertThat(calls).hasSize(expectedCallCount); + + assertEquals( + "Should have correct updates", + sql("SELECT id FROM %s", tableName(CHANGES_TABLE_NAME)), + sql("SELECT id FROM %s WHERE %s AND salary = -1", tableName, cond)); + }); + } + + private void checkMerge(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.MERGE_MODE, + mode.modeName(), + TableProperties.MERGE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = + spark.table(tableName).where(cond).limit(2).selectExpr("id + 1 as id"); + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + + List calls = + executeAndCollectFunctionCalls( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.id AND %s " + + "WHEN MATCHED THEN " + + " UPDATE SET salary = -1 " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT (id, salary, dep, ts) VALUES (100, -1, 'hr', null)", + tableName, tableName(CHANGES_TABLE_NAME), cond); + assertThat(calls).isEmpty(); + + assertEquals( + "Should have correct updates", + sql("SELECT id FROM %s", tableName(CHANGES_TABLE_NAME)), + sql("SELECT id FROM %s WHERE %s AND salary = -1", tableName, cond)); + }); + } + + private List executeAndCollectFunctionCalls(String query, Object... args) { + CommandResultExec command = (CommandResultExec) executeAndKeepPlan(query, args); + V2TableWriteExec write = (V2TableWriteExec) command.commandPhysicalPlan(); + return SparkPlanUtil.collectExprs( + write.query(), + expr -> expr instanceof StaticInvoke || expr instanceof ApplyFunctionExpression); + } + + private List findIrrelevantFileLocations(String cond) { + return spark + .table(tableName) + .where("NOT " + cond) + .select(MetadataColumns.FILE_PATH.name()) + .distinct() + .as(Encoders.STRING()) + .collectAsList(); + } + + private void initTable(String transform) { + sql( + "CREATE TABLE %s (id BIGINT, salary INT, dep STRING, ts TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (%s)", + tableName, transform); + + append( + tableName, + "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 4, \"salary\": 400, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }", + "{ \"id\": 5, \"salary\": 500, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }", + "{ \"id\": 6, \"salary\": 600, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }"); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java new file mode 100644 index 000000000000..5ec44f314180 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.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.spark.functions; + +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; + +abstract class BaseScalarFunction implements ScalarFunction { + @Override + public int hashCode() { + return canonicalName().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof ScalarFunction)) { + return false; + } + + ScalarFunction that = (ScalarFunction) other; + return canonicalName().equals(that.canonicalName()); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java index af3c67a4bb63..c3de3d48dbcc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java @@ -25,7 +25,6 @@ import org.apache.iceberg.util.BucketUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.ByteType; @@ -115,7 +114,7 @@ public String name() { return "bucket"; } - public abstract static class BucketBase implements ScalarFunction { + public abstract static class BucketBase extends BaseScalarFunction { public static int apply(int numBuckets, int hashedValue) { return (hashedValue & Integer.MAX_VALUE) % numBuckets; } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java index b8d28b73f42f..f52edd9b208f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DateType; @@ -61,7 +60,7 @@ public String name() { return "days"; } - private abstract static class BaseToDaysFunction implements ScalarFunction { + private abstract static class BaseToDaysFunction extends BaseScalarFunction { @Override public String name() { return "days"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java index 18697e1c16fb..660a182f0e78 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.TimestampNTZType; @@ -58,7 +57,7 @@ public String name() { return "hours"; } - public static class TimestampToHoursFunction implements ScalarFunction { + public static class TimestampToHoursFunction extends BaseScalarFunction { // magic method used in codegen public static int invoke(long micros) { return DateTimeUtil.microsToHours(micros); @@ -91,7 +90,7 @@ public Integer produceResult(InternalRow input) { } } - public static class TimestampNtzToHoursFunction implements ScalarFunction { + public static class TimestampNtzToHoursFunction extends BaseScalarFunction { // magic method used in codegen public static int invoke(long micros) { return DateTimeUtil.microsToHours(micros); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java index 9cd059377ce3..689a0f4cb4df 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.IcebergBuild; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; @@ -55,7 +54,7 @@ public String name() { // Implementing class cannot be private, otherwise Spark is unable to access the static invoke // function during code-gen and calling the function fails - static class IcebergVersionFunctionImpl implements ScalarFunction { + static class IcebergVersionFunctionImpl extends BaseScalarFunction { private static final UTF8String VERSION = UTF8String.fromString(IcebergBuild.version()); // magic function used in code-gen. must be named `invoke`. diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java index 1d38014461c1..353d850f86e2 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DateType; @@ -61,7 +60,7 @@ public String name() { return "months"; } - private abstract static class BaseToMonthsFunction implements ScalarFunction { + private abstract static class BaseToMonthsFunction extends BaseScalarFunction { @Override public String name() { return "months"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java index 8cfb529e1028..fac90c9efee6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java @@ -27,7 +27,6 @@ import org.apache.iceberg.util.TruncateUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.ByteType; @@ -108,7 +107,7 @@ public String name() { return "truncate"; } - public abstract static class TruncateBase implements ScalarFunction { + public abstract static class TruncateBase extends BaseScalarFunction { @Override public String name() { return "truncate"; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java index 02642e657d76..cfd1b0e8d002 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DateType; @@ -61,7 +60,7 @@ public String name() { return "years"; } - private abstract static class BaseToYearsFunction implements ScalarFunction { + private abstract static class BaseToYearsFunction extends BaseScalarFunction { @Override public String name() { return "years"; From fcda6851e9d26e84e7d8bd29249307ea5d173764 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 12 Mar 2024 10:17:21 +0530 Subject: [PATCH 0122/1019] Docs: Fix release notes indentation (#9933) --- site/docs/releases.md | 102 +++++++++++++++++++++--------------------- 1 file changed, 50 insertions(+), 52 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 85997e6d6a08..89c963f25a0b 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -72,67 +72,65 @@ Apache Iceberg 1.5.0 was released on March 11, 2024. The 1.5.0 release adds a variety of new features and bug fixes. * API - - Extend FileIO and add EncryptingFileIO. ([\#9592](https://github.com/apache/iceberg/pull/9592)) - - Track partition statistics in TableMetadata ([\#8502](https://github.com/apache/iceberg/pull/8502)) - - Add sqlFor API to views to handle resolving a representation for a dialect([\#9247](https://github.com/apache/iceberg/pull/9247)) + - Extend FileIO and add EncryptingFileIO. ([\#9592](https://github.com/apache/iceberg/pull/9592)) + - Track partition statistics in TableMetadata ([\#8502](https://github.com/apache/iceberg/pull/8502)) + - Add sqlFor API to views to handle resolving a representation for a dialect([\#9247](https://github.com/apache/iceberg/pull/9247)) * Core - - Add view support for REST catalog ([\#7913](https://github.com/apache/iceberg/pull/7913)) - - Add view support for JDBC catalog ([\#9487](https://github.com/apache/iceberg/pull/9487)) - - Add catalog type for glue,jdbc,nessie ([\#9647](https://github.com/apache/iceberg/pull/9647)) - - Support Avro file encryption with AES GCM streams ([\#9436](https://github.com/apache/iceberg/pull/9436)) - - Add ApplyNameMapping for Avro ([\#9347](https://github.com/apache/iceberg/pull/9347)) - - Add StandardEncryptionManager ([\#9277](https://github.com/apache/iceberg/pull/9277)) - - Add REST catalog table session cache ([\#8920](https://github.com/apache/iceberg/pull/8920)) - - Support view metadata compression ([\#8552](https://github.com/apache/iceberg/pull/8552)) - - Track partition statistics in TableMetadata ([\#8502](https://github.com/apache/iceberg/pull/8502)) - - Enable column statistics filtering after planning ([\#8803](https://github.com/apache/iceberg/pull/8803)) + - Add view support for REST catalog ([\#7913](https://github.com/apache/iceberg/pull/7913)) + - Add view support for JDBC catalog ([\#9487](https://github.com/apache/iceberg/pull/9487)) + - Add catalog type for glue,jdbc,nessie ([\#9647](https://github.com/apache/iceberg/pull/9647)) + - Support Avro file encryption with AES GCM streams ([\#9436](https://github.com/apache/iceberg/pull/9436)) + - Add ApplyNameMapping for Avro ([\#9347](https://github.com/apache/iceberg/pull/9347)) + - Add StandardEncryptionManager ([\#9277](https://github.com/apache/iceberg/pull/9277)) + - Add REST catalog table session cache ([\#8920](https://github.com/apache/iceberg/pull/8920)) + - Support view metadata compression ([\#8552](https://github.com/apache/iceberg/pull/8552)) + - Track partition statistics in TableMetadata ([\#8502](https://github.com/apache/iceberg/pull/8502)) + - Enable column statistics filtering after planning ([\#8803](https://github.com/apache/iceberg/pull/8803)) * Spark - - Remove support for Spark 3.2 ([\#9295](https://github.com/apache/iceberg/pull/9295)) - - Support views via SQL for Spark 3.4 and 3.5 ([\#9423](https://github.com/apache/iceberg/pull/9423), [\#9421](https://github.com/apache/iceberg/pull/9421), [\#9343](https://github.com/apache/iceberg/pull/9343)), ([\#9513](https://github.com/apache/iceberg/pull/9513), ([\#9582](https://github.com/apache/iceberg/pull/9582) - - Support executor cache locality ([\#9563](https://github.com/apache/iceberg/pull/9563)) - - Added support for delete manifest rewrites ([\#9020](https://github.com/apache/iceberg/pull/9020)) - - Support encrypted output files ([\#9435](https://github.com/apache/iceberg/pull/9435)) - - Add Spark UI metrics from Iceberg scan metrics ([\#8717](https://github.com/apache/iceberg/pull/8717)) - - Parallelize reading files in add_files procedure ([\#9274](https://github.com/apache/iceberg/pull/9274)) - - Support file and partition delete granularity ([\#9384](https://github.com/apache/iceberg/pull/9384)) + - Remove support for Spark 3.2 ([\#9295](https://github.com/apache/iceberg/pull/9295)) + - Support views via SQL for Spark 3.4 and 3.5 ([\#9423](https://github.com/apache/iceberg/pull/9423), [\#9421](https://github.com/apache/iceberg/pull/9421), [\#9343](https://github.com/apache/iceberg/pull/9343)), ([\#9513](https://github.com/apache/iceberg/pull/9513), ([\#9582](https://github.com/apache/iceberg/pull/9582) + - Support executor cache locality ([\#9563](https://github.com/apache/iceberg/pull/9563)) + - Added support for delete manifest rewrites ([\#9020](https://github.com/apache/iceberg/pull/9020)) + - Support encrypted output files ([\#9435](https://github.com/apache/iceberg/pull/9435)) + - Add Spark UI metrics from Iceberg scan metrics ([\#8717](https://github.com/apache/iceberg/pull/8717)) + - Parallelize reading files in add_files procedure ([\#9274](https://github.com/apache/iceberg/pull/9274)) + - Support file and partition delete granularity ([\#9384](https://github.com/apache/iceberg/pull/9384)) * Flink - - Remove Flink 1.15 - - Adds support for 1.18 version [\#9211](https://github.com/apache/iceberg/pull/9211) - - Emit watermarks from the IcebergSource ([\#8553](https://github.com/apache/iceberg/pull/8553)) - - Watermark read options ([\#9346](https://github.com/apache/iceberg/pull/9346)) + - Remove Flink 1.15 + - Adds support for 1.18 version [\#9211](https://github.com/apache/iceberg/pull/9211) + - Emit watermarks from the IcebergSource ([\#8553](https://github.com/apache/iceberg/pull/8553)) + - Watermark read options ([\#9346](https://github.com/apache/iceberg/pull/9346)) * Parquet - - Support reading INT96 column in row group filter ([\#8988](https://github.com/apache/iceberg/pull/8988)) - - Add system config for unsafe Parquet ID fallback. ([\#9324](https://github.com/apache/iceberg/pull/9324)) + - Support reading INT96 column in row group filter ([\#8988](https://github.com/apache/iceberg/pull/8988)) + - Add system config for unsafe Parquet ID fallback. ([\#9324](https://github.com/apache/iceberg/pull/9324)) * Kafka-Connect - - Initial project setup and event data structures ([\#8701](https://github.com/apache/iceberg/pull/8701)) - - Sink connector with data writers and converters ([\#9466](https://github.com/apache/iceberg/pull/9466)) + - Initial project setup and event data structures ([\#8701](https://github.com/apache/iceberg/pull/8701)) + - Sink connector with data writers and converters ([\#9466](https://github.com/apache/iceberg/pull/9466)) * Spec - - Add partition stats spec ([\#7105](https://github.com/apache/iceberg/pull/7105)) - - add nanosecond timestamp types ([\#8683](https://github.com/apache/iceberg/pull/8683)) - - Add multi-arg transform ([\#8579](https://github.com/apache/iceberg/pull/8579)) + - Add partition stats spec ([\#7105](https://github.com/apache/iceberg/pull/7105)) + - add nanosecond timestamp types ([\#8683](https://github.com/apache/iceberg/pull/8683)) + - Add multi-arg transform ([\#8579](https://github.com/apache/iceberg/pull/8579)) * Vendor Integrations - - AWS: Support setting description for Glue table ([\#9530](https://github.com/apache/iceberg/pull/9530)) - - AWS: Update S3FileIO test to run when CLIENT_FACTORY is not set ([\#9541](https://github.com/apache/iceberg/pull/9541)) - - AWS: Add S3 Access Grants Integration ([\#9385](https://github.com/apache/iceberg/pull/9385)) - - AWS: Glue catalog strip trailing slash on DB URI ([\#8870](https://github.com/apache/iceberg/pull/8870)) - - Azure: Add FileIO that supports ADLSv2 storage ([\#8303](https://github.com/apache/iceberg/pull/8303)) - - Azure: Make ADLSFileIO implement DelegateFileIO ([\#8563](https://github.com/apache/iceberg/pull/8563)) - - Nessie: Support views for NessieCatalog ([\#8909](https://github.com/apache/iceberg/pull/8909)) - - Nessie: Strip trailing slash for warehouse location ([\#9415](https://github.com/apache/iceberg/pull/9415)) - - Nessie: Infer default API version from URI ([\#9459](https://github.com/apache/iceberg/pull/9459)) + - AWS: Support setting description for Glue table ([\#9530](https://github.com/apache/iceberg/pull/9530)) + - AWS: Update S3FileIO test to run when CLIENT_FACTORY is not set ([\#9541](https://github.com/apache/iceberg/pull/9541)) + - AWS: Add S3 Access Grants Integration ([\#9385](https://github.com/apache/iceberg/pull/9385)) + - AWS: Glue catalog strip trailing slash on DB URI ([\#8870](https://github.com/apache/iceberg/pull/8870)) + - Azure: Add FileIO that supports ADLSv2 storage ([\#8303](https://github.com/apache/iceberg/pull/8303)) + - Azure: Make ADLSFileIO implement DelegateFileIO ([\#8563](https://github.com/apache/iceberg/pull/8563)) + - Nessie: Support views for NessieCatalog ([\#8909](https://github.com/apache/iceberg/pull/8909)) + - Nessie: Strip trailing slash for warehouse location ([\#9415](https://github.com/apache/iceberg/pull/9415)) + - Nessie: Infer default API version from URI ([\#9459](https://github.com/apache/iceberg/pull/9459)) * Dependencies - - Bump Nessie to 0.77.1 - - Bump ORC to 1.9.2 - - Bump Arrow to 15.0.0 - - Bump AWS Java SDK to 2.24.5 - - Bump Azure Java SDK to 1.2.20 - - Bump Google cloud libraries to 26.28.0 - -Note: -1. To enable view support for JDBC catalog, configure `jdbc.schema-version` to `V1` in catalog properties. + - Bump Nessie to 0.77.1 + - Bump ORC to 1.9.2 + - Bump Arrow to 15.0.0 + - Bump AWS Java SDK to 2.24.5 + - Bump Azure Java SDK to 1.2.20 + - Bump Google cloud libraries to 26.28.0 + +* Note: To enable view support for JDBC catalog, configure `jdbc.schema-version` to `V1` in catalog properties. -For more details, please visit Github. -https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.5.0 +For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.5.0). ## Past releases From 08e941dde171b7c5abfd5c4e71540b28c352fce1 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 12 Mar 2024 12:30:22 +0530 Subject: [PATCH 0123/1019] Flink: Bump minor versions (#9875) --- .../java/org/apache/iceberg/flink/util/TestFlinkPackage.java | 2 +- .../java/org/apache/iceberg/flink/util/TestFlinkPackage.java | 2 +- gradle/libs.versions.toml | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index 23cefa834757..9d99193fb1be 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -28,7 +28,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - Assert.assertEquals("1.16.2", FlinkPackage.version()); + Assert.assertEquals("1.16.3", FlinkPackage.version()); } @Test diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index ceb3ed558698..eda340dedaf5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -28,7 +28,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - Assert.assertEquals("1.17.1", FlinkPackage.version()); + Assert.assertEquals("1.17.2", FlinkPackage.version()); } @Test diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 3d4a027ae247..4b0094a8ffa1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -38,8 +38,8 @@ delta-spark = "3.1.0" esotericsoftware-kryo = "4.0.2" errorprone-annotations = "2.24.1" findbugs-jsr305 = "3.0.2" -flink116 = { strictly = "1.16.2"} -flink117 = { strictly = "1.17.1"} +flink116 = { strictly = "1.16.3"} +flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} google-libraries-bom = "26.28.0" guava = "33.0.0-jre" From 34efd9d89a730f7a435e731e93f730d1cdae5930 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 12 Mar 2024 15:44:56 +0800 Subject: [PATCH 0124/1019] Docs: Enhance Spark pages (#9920) 1. Fix internal links 2. Move `Type Compatibility` section from spark-writes.md to spark-getting-started.md --- docs/docs/spark-configuration.md | 2 +- docs/docs/spark-ddl.md | 16 ++--- docs/docs/spark-getting-started.md | 90 ++++++++++++++++++++----- docs/docs/spark-procedures.md | 8 +-- docs/docs/spark-queries.md | 4 +- docs/docs/spark-structured-streaming.md | 10 +-- docs/docs/spark-writes.md | 68 ++----------------- 7 files changed, 99 insertions(+), 99 deletions(-) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index e9382e701e49..5e9c6e5d1147 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -78,7 +78,7 @@ Both catalogs are configured using properties nested under the catalog name. Com | 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 | -Additional properties can be found in common [catalog configuration](configuration.md#catalog-properties). +Additional properties can be found in common [catalog configuration](../configuration.md#catalog-properties). ### Using catalogs diff --git a/docs/docs/spark-ddl.md b/docs/docs/spark-ddl.md index ea85d5c9919a..e1376ddcf667 100644 --- a/docs/docs/spark-ddl.md +++ b/docs/docs/spark-ddl.md @@ -33,14 +33,14 @@ CREATE TABLE prod.db.sample ( USING iceberg; ``` -Iceberg will convert the column type in Spark to corresponding Iceberg type. Please check the section of [type compatibility on creating table](spark-writes.md#spark-type-to-iceberg-type) for details. +Iceberg will convert the column type in Spark to corresponding Iceberg type. Please check the section of [type compatibility on creating table](../spark-getting-started.md#spark-type-to-iceberg-type) for details. Table create commands, including CTAS and RTAS, support the full range of Spark create clauses, including: * `PARTITIONED BY (partition-expressions)` to configure partitioning * `LOCATION '(fully-qualified-uri)'` to set the table location * `COMMENT 'table documentation'` to set a table description -* `TBLPROPERTIES ('key'='value', ...)` to set [table configuration](configuration.md) +* `TBLPROPERTIES ('key'='value', ...)` to set [table configuration](../configuration.md) Create commands may also set the default format with the `USING` clause. This is only supported for `SparkCatalog` because Spark handles the `USING` clause differently for the built-in catalog. @@ -59,7 +59,7 @@ USING iceberg PARTITIONED BY (category); ``` -The `PARTITIONED BY` clause supports transform expressions to create [hidden partitions](partitioning.md). +The `PARTITIONED BY` clause supports transform expressions to create [hidden partitions](../partitioning.md). ```sql CREATE TABLE prod.db.sample ( @@ -86,7 +86,7 @@ Note: Old syntax of `years(ts)`, `months(ts)`, `days(ts)` and `hours(ts)` are al ## `CREATE TABLE ... AS SELECT` -Iceberg supports CTAS as an atomic operation when using a [`SparkCatalog`](spark-configuration.md#catalog-configuration). CTAS is supported, but is not atomic when using [`SparkSessionCatalog`](spark-configuration.md#replacing-the-session-catalog). +Iceberg supports CTAS as an atomic operation when using a [`SparkCatalog`](../spark-configuration.md#catalog-configuration). CTAS is supported, but is not atomic when using [`SparkSessionCatalog`](../spark-configuration.md#replacing-the-session-catalog). ```sql CREATE TABLE prod.db.sample @@ -106,7 +106,7 @@ AS SELECT ... ## `REPLACE TABLE ... AS SELECT` -Iceberg supports RTAS as an atomic operation when using a [`SparkCatalog`](spark-configuration.md#catalog-configuration). RTAS is supported, but is not atomic when using [`SparkSessionCatalog`](spark-configuration.md#replacing-the-session-catalog). +Iceberg supports RTAS as an atomic operation when using a [`SparkCatalog`](../spark-configuration.md#catalog-configuration). RTAS is supported, but is not atomic when using [`SparkSessionCatalog`](../spark-configuration.md#replacing-the-session-catalog). Atomic table replacement creates a new snapshot with the results of the `SELECT` query, but keeps table history. @@ -168,7 +168,7 @@ Iceberg has full `ALTER TABLE` support in Spark 3, including: * Widening the type of `int`, `float`, and `decimal` fields * Making required columns optional -In addition, [SQL extensions](spark-configuration.md#sql-extensions) can be used to add support for partition evolution and setting a table's write order +In addition, [SQL extensions](../spark-configuration.md#sql-extensions) can be used to add support for partition evolution and setting a table's write order ### `ALTER TABLE ... RENAME TO` @@ -184,7 +184,7 @@ ALTER TABLE prod.db.sample SET TBLPROPERTIES ( ); ``` -Iceberg uses table properties to control table behavior. For a list of available properties, see [Table configuration](configuration.md). +Iceberg uses table properties to control table behavior. For a list of available properties, see [Table configuration](../configuration.md). `UNSET` is used to remove properties: @@ -325,7 +325,7 @@ ALTER TABLE prod.db.sample DROP COLUMN point.z; ## `ALTER TABLE` SQL extensions -These commands are available in Spark 3 when using Iceberg [SQL extensions](spark-configuration.md#sql-extensions). +These commands are available in Spark 3 when using Iceberg [SQL extensions](../spark-configuration.md#sql-extensions). ### `ALTER TABLE ... ADD PARTITION FIELD` diff --git a/docs/docs/spark-getting-started.md b/docs/docs/spark-getting-started.md index d9dd70e4fa33..72642cc6e14f 100644 --- a/docs/docs/spark-getting-started.md +++ b/docs/docs/spark-getting-started.md @@ -40,7 +40,7 @@ spark-shell --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:{{ iceb ### Adding catalogs -Iceberg comes with [catalogs](spark-configuration.md#catalogs) that enable SQL commands to manage tables and load them by name. Catalogs are configured using properties under `spark.sql.catalog.(catalog_name)`. +Iceberg comes with [catalogs](../spark-configuration.md#catalogs) that enable SQL commands to manage tables and load them by name. Catalogs are configured using properties under `spark.sql.catalog.(catalog_name)`. This command creates a path-based catalog named `local` for tables under `$PWD/warehouse` and adds support for Iceberg tables to Spark's built-in catalog: @@ -56,7 +56,7 @@ spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:{{ iceber ### Creating a table -To create your first Iceberg table in Spark, use the `spark-sql` shell or `spark.sql(...)` to run a [`CREATE TABLE`](spark-ddl.md#create-table) command: +To create your first Iceberg table in Spark, use the `spark-sql` shell or `spark.sql(...)` to run a [`CREATE TABLE`](../spark-ddl.md#create-table) command: ```sql -- local is the path-based catalog defined above @@ -65,21 +65,21 @@ CREATE TABLE local.db.table (id bigint, data string) USING iceberg; Iceberg catalogs support the full range of SQL DDL commands, including: -* [`CREATE TABLE ... PARTITIONED BY`](spark-ddl.md#create-table) -* [`CREATE TABLE ... AS SELECT`](spark-ddl.md#create-table-as-select) -* [`ALTER TABLE`](spark-ddl.md#alter-table) -* [`DROP TABLE`](spark-ddl.md#drop-table) +* [`CREATE TABLE ... PARTITIONED BY`](../spark-ddl.md#create-table) +* [`CREATE TABLE ... AS SELECT`](../spark-ddl.md#create-table-as-select) +* [`ALTER TABLE`](../spark-ddl.md#alter-table) +* [`DROP TABLE`](../spark-ddl.md#drop-table) ### Writing -Once your table is created, insert data using [`INSERT INTO`](spark-writes.md#insert-into): +Once your table is created, insert data using [`INSERT INTO`](../spark-writes.md#insert-into): ```sql INSERT INTO local.db.table VALUES (1, 'a'), (2, 'b'), (3, 'c'); INSERT INTO local.db.table SELECT id, data FROM source WHERE length(data) = 1; ``` -Iceberg also adds row-level SQL updates to Spark, [`MERGE INTO`](spark-writes.md#merge-into) and [`DELETE FROM`](spark-writes.md#delete-from): +Iceberg also adds row-level SQL updates to Spark, [`MERGE INTO`](../spark-writes.md#merge-into) and [`DELETE FROM`](../spark-writes.md#delete-from): ```sql MERGE INTO local.db.target t USING (SELECT * FROM updates) u ON t.id = u.id @@ -87,7 +87,7 @@ WHEN MATCHED THEN UPDATE SET t.count = t.count + u.count WHEN NOT MATCHED THEN INSERT *; ``` -Iceberg supports writing DataFrames using the new [v2 DataFrame write API](spark-writes.md#writing-with-dataframes): +Iceberg supports writing DataFrames using the new [v2 DataFrame write API](../spark-writes.md#writing-with-dataframes): ```scala spark.table("source").select("id", "data") @@ -106,7 +106,7 @@ FROM local.db.table GROUP BY data; ``` -SQL is also the recommended way to [inspect tables](spark-queries.md#inspecting-tables). To view all snapshots in a table, use the `snapshots` metadata table: +SQL is also the recommended way to [inspect tables](../spark-queries.md#inspecting-tables). To view all snapshots in a table, use the `snapshots` metadata table: ```sql SELECT * FROM local.db.table.snapshots; ``` @@ -121,18 +121,78 @@ SELECT * FROM local.db.table.snapshots; +-------------------------+----------------+-----------+-----------+----------------------------------------------------+-----+ ``` -[DataFrame reads](spark-queries.md#querying-with-dataframes) are supported and can now reference tables by name using `spark.table`: +[DataFrame reads](../spark-queries.md#querying-with-dataframes) are supported and can now reference tables by name using `spark.table`: ```scala val df = spark.table("local.db.table") df.count() ``` +### Type compatibility + +Spark and Iceberg support different set of types. Iceberg does the type conversion automatically, but not for all combinations, +so you may want to understand the type conversion in Iceberg in prior to design the types of columns in your tables. + +#### Spark type to Iceberg type + +This type conversion table describes how Spark types are converted to the Iceberg types. The conversion applies on both creating Iceberg table and writing to Iceberg table via Spark. + +| Spark | Iceberg | Notes | +|-----------------|----------------------------|-------| +| boolean | boolean | | +| short | integer | | +| byte | integer | | +| integer | integer | | +| long | long | | +| float | float | | +| double | double | | +| date | date | | +| timestamp | timestamp with timezone | | +| timestamp_ntz | timestamp without timezone | | +| char | string | | +| varchar | string | | +| string | string | | +| binary | binary | | +| decimal | decimal | | +| struct | struct | | +| array | list | | +| map | map | | + +!!! info + The table is based on representing conversion during creating table. In fact, broader supports are applied on write. Here're some points on write: + + * Iceberg numeric types (`integer`, `long`, `float`, `double`, `decimal`) support promotion during writes. e.g. You can write Spark types `short`, `byte`, `integer`, `long` to Iceberg type `long`. + * You can write to Iceberg `fixed` type using Spark `binary` type. Note that assertion on the length will be performed. + +#### Iceberg type to Spark type + +This type conversion table describes how Iceberg types are converted to the Spark types. The conversion applies on reading from Iceberg table via Spark. + +| Iceberg | Spark | Note | +|----------------------------|-------------------------|---------------| +| boolean | boolean | | +| integer | integer | | +| long | long | | +| float | float | | +| double | double | | +| date | date | | +| time | | Not supported | +| timestamp with timezone | timestamp | | +| timestamp without timezone | timestamp_ntz | | +| string | string | | +| uuid | string | | +| fixed | binary | | +| binary | binary | | +| decimal | decimal | | +| struct | struct | | +| list | array | | +| map | map | | + ### Next steps Next, you can learn more about Iceberg tables in Spark: -* [DDL commands](spark-ddl.md): `CREATE`, `ALTER`, and `DROP` -* [Querying data](spark-queries.md): `SELECT` queries and metadata tables -* [Writing data](spark-writes.md): `INSERT INTO` and `MERGE INTO` -* [Maintaining tables](spark-procedures.md) with stored procedures +* [DDL commands](../spark-ddl.md): `CREATE`, `ALTER`, and `DROP` +* [Querying data](../spark-queries.md): `SELECT` queries and metadata tables +* [Writing data](../spark-writes.md): `INSERT INTO` and `MERGE INTO` +* [Maintaining tables](../spark-procedures.md) with stored procedures diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 6b3cb06c3af7..ee8a8466c70d 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -20,7 +20,7 @@ title: "Procedures" # Spark Procedures -To use Iceberg in Spark, first configure [Spark catalogs](spark-configuration.md). Stored procedures are only available when using [Iceberg SQL extensions](spark-configuration.md#sql-extensions) in Spark 3. +To use Iceberg in Spark, first configure [Spark catalogs](../spark-configuration.md). Stored procedures are only available when using [Iceberg SQL extensions](../spark-configuration.md#sql-extensions) in Spark 3. ## Usage @@ -272,7 +272,7 @@ the `expire_snapshots` procedure will never remove files which are still require | `stream_results` | | boolean | When true, deletion files will be sent to Spark driver by RDD partition (by default, all the files will be sent to Spark driver). This option is recommended to set to `true` to prevent Spark driver OOM from large file size | | `snapshot_ids` | | array of long | Array of snapshot IDs to expire. | -If `older_than` and `retain_last` are omitted, the table's [expiration properties](configuration.md#table-behavior-properties) will be used. +If `older_than` and `retain_last` are omitted, the table's [expiration properties](../configuration.md#table-behavior-properties) will be used. Snapshots that are still referenced by branches or tags won't be removed. By default, branches and tags never expire, but their retention policy can be changed with the table property `history.expire.max-ref-age-ms`. The `main` branch never expires. #### Output @@ -357,7 +357,7 @@ Iceberg can compact data files in parallel using Spark with the `rewriteDataFile | `partial-progress.max-commits` | 10 | Maximum amount of commits that this rewrite is allowed to produce 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 | +| `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 | | `min-file-size-bytes` | 75% of target file size | Files under this threshold will be considered for rewriting regardless of any other criteria | | `max-file-size-bytes` | 180% of target file size | Files with sizes above this threshold will be considered for rewriting regardless of any other criteria | | `min-input-files` | 5 | Any file group exceeding this number of files will be rewritten regardless of other criteria | @@ -480,7 +480,7 @@ Dangling deletes are always filtered out during rewriting. | `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 | | `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` | 67108864 (64MB, default value of `write.delete.target-file-size-bytes` from [table properties](configuration.md#write-properties)) | Target output file size | +| `target-file-size-bytes` | 67108864 (64MB, default value of `write.delete.target-file-size-bytes` from [table properties](../configuration.md#write-properties)) | Target output file size | | `min-file-size-bytes` | 75% of target file size | Files under this threshold will be considered for rewriting regardless of any other criteria | | `max-file-size-bytes` | 180% of target file size | Files with sizes above this threshold will be considered for rewriting regardless of any other criteria | | `min-input-files` | 5 | Any file group exceeding this number of files will be rewritten regardless of other criteria | diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index 536c136d7e55..092ed6b1d636 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -20,11 +20,11 @@ title: "Queries" # Spark Queries -To use Iceberg in Spark, first configure [Spark catalogs](spark-configuration.md). Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. +To use Iceberg in Spark, first configure [Spark catalogs](../spark-configuration.md). Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. ## Querying with SQL -In Spark 3, tables use identifiers that include a [catalog name](spark-configuration.md#using-catalogs). +In Spark 3, tables use identifiers that include a [catalog name](../spark-configuration.md#using-catalogs). ```sql SELECT * FROM prod.db.table; -- catalog: prod, namespace: db, table: table diff --git a/docs/docs/spark-structured-streaming.md b/docs/docs/spark-structured-streaming.md index 0ac753808d9e..50799042073f 100644 --- a/docs/docs/spark-structured-streaming.md +++ b/docs/docs/spark-structured-streaming.md @@ -68,7 +68,7 @@ Iceberg supports `append` and `complete` output modes: * `append`: appends the rows of every micro-batch to the table * `complete`: replaces the table contents every micro-batch -Prior to starting the streaming query, ensure you created the table. Refer to the [SQL create table](spark-ddl.md#create-table) documentation to learn how to create the Iceberg table. +Prior to starting the streaming query, ensure you created the table. Refer to the [SQL create table](../spark-ddl.md#create-table) documentation to learn how to create the Iceberg table. Iceberg doesn't support experimental [continuous processing](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#continuous-processing), as it doesn't provide the interface to "commit" the output. @@ -76,7 +76,7 @@ Iceberg doesn't support experimental [continuous processing](https://spark.apach Iceberg requires sorting data by partition per task prior to writing the data. In Spark tasks are split by Spark partition. against partitioned table. For batch queries you're encouraged to do explicit sort to fulfill the requirement -(see [here](spark-writes.md#writing-distribution-modes)), but the approach would bring additional latency as +(see [here](../spark-writes.md#writing-distribution-modes)), but the approach would bring additional latency as repartition and sort are considered as heavy operations for streaming workload. To avoid additional latency, you can enable fanout writer to eliminate the requirement. @@ -107,13 +107,13 @@ documents how to configure the interval. ### Expire old snapshots -Each batch written to a table produces a new snapshot. Iceberg tracks snapshots in table metadata until they are expired. Snapshots accumulate quickly with frequent commits, so it is highly recommended that tables written by streaming queries are [regularly maintained](maintenance.md#expire-snapshots). [Snapshot expiration](spark-procedures.md#expire_snapshots) is the procedure of removing the metadata and any data files that are no longer needed. By default, the procedure will expire the snapshots older than five days. +Each batch written to a table produces a new snapshot. Iceberg tracks snapshots in table metadata until they are expired. Snapshots accumulate quickly with frequent commits, so it is highly recommended that tables written by streaming queries are [regularly maintained](../maintenance.md#expire-snapshots). [Snapshot expiration](../spark-procedures.md#expire_snapshots) is the procedure of removing the metadata and any data files that are no longer needed. By default, the procedure will expire the snapshots older than five days. ### Compacting data files -The amount of data written from a streaming process is typically small, which can cause the table metadata to track lots of small files. [Compacting small files into larger files](maintenance.md#compact-data-files) reduces the metadata needed by the table, and increases query efficiency. Iceberg and Spark [comes with the `rewrite_data_files` procedure](spark-procedures.md#rewrite_data_files). +The amount of data written from a streaming process is typically small, which can cause the table metadata to track lots of small files. [Compacting small files into larger files](../maintenance.md#compact-data-files) reduces the metadata needed by the table, and increases query efficiency. Iceberg and Spark [comes with the `rewrite_data_files` procedure](../spark-procedures.md#rewrite_data_files). ### Rewrite manifests To optimize write latency on a streaming workload, Iceberg can write the new snapshot with a "fast" append that does not automatically compact manifests. -This could lead lots of small manifest files. Iceberg can [rewrite the number of manifest files to improve query performance](maintenance.md#rewrite-manifests). Iceberg and Spark [come with the `rewrite_manifests` procedure](spark-procedures.md#rewrite_manifests). +This could lead lots of small manifest files. Iceberg can [rewrite the number of manifest files to improve query performance](../maintenance.md#rewrite-manifests). Iceberg and Spark [come with the `rewrite_manifests` procedure](../spark-procedures.md#rewrite_manifests). diff --git a/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index 1baea04deed5..efc15e7e35fc 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -20,9 +20,9 @@ title: "Writes" # Spark Writes -To use Iceberg in Spark, first configure [Spark catalogs](spark-configuration.md). +To use Iceberg in Spark, first configure [Spark catalogs](../spark-configuration.md). -Some plans are only available when using [Iceberg SQL extensions](spark-configuration.md#sql-extensions) in Spark 3. +Some plans are only available when using [Iceberg SQL extensions](../spark-configuration.md#sql-extensions) in Spark 3. Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. Spark DSv2 is an evolving API with different levels of support in Spark versions: @@ -200,7 +200,7 @@ Branch writes can also be performed as part of a write-audit-publish (WAP) workf Note WAP branch and branch identifier cannot both be specified. Also, the branch must exist before performing the write. The operation does **not** create the branch if it does not exist. -For more information on branches please refer to [branches](branching.md). +For more information on branches please refer to [branches](../branching.md). ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. @@ -385,7 +385,7 @@ sort-order. Further division and coalescing of tasks may take place because of When writing data to Iceberg with Spark, it's important to note that Spark cannot write a file larger than a Spark task and a file cannot span an Iceberg partition boundary. This means although Iceberg will always roll over a file -when it grows to [`write.target-file-size-bytes`](configuration.md#write-properties), but unless the Spark task is +when it grows to [`write.target-file-size-bytes`](../configuration.md#write-properties), but unless the Spark task is large enough that will not happen. The size of the file created on disk will also be much smaller than the Spark task since the on disk data will be both compressed and in columnar format as opposed to Spark's uncompressed row representation. This means a 100 megabyte Spark task will create a file much smaller than 100 megabytes even if that @@ -404,63 +404,3 @@ columnar-compressed size, so a larger value than the target file size will need in-memory size to on disk size is data dependent. Future work in Spark should allow Iceberg to automatically adjust this parameter at write time to match the `write.target-file-size-bytes`. -## Type compatibility - -Spark and Iceberg support different set of types. Iceberg does the type conversion automatically, but not for all combinations, -so you may want to understand the type conversion in Iceberg in prior to design the types of columns in your tables. - -### Spark type to Iceberg type - -This type conversion table describes how Spark types are converted to the Iceberg types. The conversion applies on both creating Iceberg table and writing to Iceberg table via Spark. - -| Spark | Iceberg | Notes | -|-----------------|----------------------------|-------| -| boolean | boolean | | -| short | integer | | -| byte | integer | | -| integer | integer | | -| long | long | | -| float | float | | -| double | double | | -| date | date | | -| timestamp | timestamp with timezone | | -| timestamp_ntz | timestamp without timezone | | -| char | string | | -| varchar | string | | -| string | string | | -| binary | binary | | -| decimal | decimal | | -| struct | struct | | -| array | list | | -| map | map | | - -!!! info - The table is based on representing conversion during creating table. In fact, broader supports are applied on write. Here're some points on write: - - * Iceberg numeric types (`integer`, `long`, `float`, `double`, `decimal`) support promotion during writes. e.g. You can write Spark types `short`, `byte`, `integer`, `long` to Iceberg type `long`. - * You can write to Iceberg `fixed` type using Spark `binary` type. Note that assertion on the length will be performed. - -### Iceberg type to Spark type - -This type conversion table describes how Iceberg types are converted to the Spark types. The conversion applies on reading from Iceberg table via Spark. - -| Iceberg | Spark | Note | -|----------------------------|-------------------------|---------------| -| boolean | boolean | | -| integer | integer | | -| long | long | | -| float | float | | -| double | double | | -| date | date | | -| time | | Not supported | -| timestamp with timezone | timestamp | | -| timestamp without timezone | timestamp_ntz | | -| string | string | | -| uuid | string | | -| fixed | binary | | -| binary | binary | | -| decimal | decimal | | -| struct | struct | | -| list | array | | -| map | map | | - From 27958e5e1f6f371a5f823e298af2352e87fde96c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 12 Mar 2024 17:51:03 +0100 Subject: [PATCH 0125/1019] Build: Align Jackson versions (#9925) --- build.gradle | 3 ++- gradle/libs.versions.toml | 4 ---- hive3/build.gradle | 2 +- mr/build.gradle | 3 ++- 4 files changed, 5 insertions(+), 7 deletions(-) diff --git a/build.gradle b/build.gradle index 64b4299d09bd..c21813a4d4ae 100644 --- a/build.gradle +++ b/build.gradle @@ -440,7 +440,8 @@ project(':iceberg-aliyun') { exclude group: 'com.google.code.gson', module: 'gson' } - testImplementation libs.jackson.dataformat.xml + 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) { diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 4b0094a8ffa1..867c87374e8e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -49,9 +49,7 @@ httpcomponents-httpclient5 = "5.3.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" -jackson-annotations = "2.16.0" jackson-bom = "2.14.2" -jackson-dataformat-xml = "2.16.1" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above jackson212 = { strictly = "2.12.3"} jackson213 = { strictly = "2.13.4"} @@ -193,8 +191,6 @@ flink118-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "f flink118-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink118" } flink118-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink118" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } -jackson-annotations = { module = "com.fasterxml.jackson.core:jackson-annotations", version.ref = "jackson-annotations" } -jackson-dataformat-xml = { module = "com.fasterxml.jackson.dataformat:jackson-dataformat-xml", version.ref = "jackson-dataformat-xml" } jakarta-el-api = { module = "jakarta.el:jakarta.el-api", version.ref = "jakarta-el-api" } jetty-server = { module = "org.eclipse.jetty:jetty-server", version.ref = "jetty" } jetty-servlet = { module = "org.eclipse.jetty:jetty-servlet", version.ref = "jetty" } diff --git a/hive3/build.gradle b/hive3/build.gradle index fe0d3a9b45fa..f8bb50951fb1 100644 --- a/hive3/build.gradle +++ b/hive3/build.gradle @@ -94,7 +94,7 @@ project(':iceberg-hive3') { testImplementation libs.avro.avro testImplementation libs.calcite.core testImplementation libs.kryo.shaded - testImplementation libs.jackson.annotations + testImplementation platform(libs.jackson.bom) testImplementation(libs.hive3.service) { exclude group: 'org.apache.hive', module: 'hive-exec' exclude group: 'org.apache.orc' diff --git a/mr/build.gradle b/mr/build.gradle index 37e665d6d2e6..d2bf9290d8ae 100644 --- a/mr/build.gradle +++ b/mr/build.gradle @@ -68,7 +68,8 @@ project(':iceberg-mr') { testImplementation libs.avro.avro testImplementation libs.calcite.core testImplementation libs.kryo.shaded - testImplementation libs.jackson.annotations + testImplementation platform(libs.jackson.bom) + testImplementation "com.fasterxml.jackson.core:jackson-annotations" testImplementation(libs.hive2.service) { exclude group: 'org.apache.hive', module: 'hive-exec' } From a227a9ce4173bad42eab9d837627e27a3f3984bc Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 13 Mar 2024 16:13:31 +0530 Subject: [PATCH 0126/1019] Docs: Update site docs (#9946) - Resolve Nessie version for site docs - Add release notes link to template --- site/docs/how-to-release.md | 2 ++ site/mkdocs.yml | 1 + 2 files changed, 3 insertions(+) diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index c775a718c472..de3bcf958c67 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -294,6 +294,8 @@ evolution. This release can be downloaded from: https://www.apache.org/dyn/closer.cgi/iceberg// +Release notes: https://iceberg.apache.org/releases/#XYZ-release + Java artifacts are available from Maven Central. Thanks to everyone for contributing! diff --git a/site/mkdocs.yml b/site/mkdocs.yml index da7810f2ed18..45df8d93bd39 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -79,6 +79,7 @@ markdown_extensions: extra: icebergVersion: '1.5.0' + nessieVersion: '0.77.1' social: - icon: fontawesome/regular/comments link: 'https://iceberg.apache.org/community/' From 08b3650c4ecdf3d074b465d6dfa742bc01f52962 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 14 Mar 2024 00:54:20 +0800 Subject: [PATCH 0127/1019] Docs: Enhance create_changelog_view usage (#9889) --- docs/docs/spark-procedures.md | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index ee8a8466c70d..7dc0d1a2aab7 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -756,8 +756,8 @@ Creates a view that contains the changes from a given table. | `table` | ✔️ | string | Name of the source table for the changelog | | `changelog_view` | | string | Name of the view to create | | `options` | | map | A map of Spark read options to use | -| `net_changes` | | boolean | Whether to output net changes (see below for more information). Defaults to false. | -| `compute_updates` | | boolean | Whether to compute pre/post update images (see below for more information). Defaults to false. | +| `net_changes` | | boolean | Whether to output net changes (see below for more information). Defaults to false. It must be false when `compute_updates` is true. | +| `compute_updates` | | boolean | Whether to compute pre/post update images (see below for more information). Defaults to true if `identifer_columns` are provided; otherwise, defaults to false. | | `identifier_columns` | | array | The list of identifier columns to compute updates. If the argument `compute_updates` is set to true and `identifier_columns` are not provided, the table’s current identifier fields will be used. | Here is a list of commonly used Spark read options: @@ -823,7 +823,10 @@ second snapshot deleted 1 record. |2 | Bob |INSERT |0 |5390529835796506035| |1 | Alice |DELETE |1 |8764748981452218370| -Create a changelog view that computes net changes. It removes intermediate changes and only outputs the net changes. +#### Net Changes + +The procedure can remove intermediate changes across multiple snapshots, and only outputs the net changes. Here is an example to create a changelog view that computes net changes. + ```sql CALL spark_catalog.system.create_changelog_view( table => 'db.tbl', From 0b3e3ff612cd5de39a9641c4e98971367487e419 Mon Sep 17 00:00:00 2001 From: ggershinsky Date: Wed, 13 Mar 2024 19:48:09 +0200 Subject: [PATCH 0128/1019] API, Core: Support manifest encryption (#8252) --- .../iceberg/io/PositionOutputStream.java | 11 + .../apache/iceberg/BaseRewriteManifests.java | 6 +- .../java/org/apache/iceberg/FastAppend.java | 8 +- .../org/apache/iceberg/ManifestFiles.java | 45 +++- .../org/apache/iceberg/ManifestWriter.java | 18 +- .../iceberg/MergingSnapshotProducer.java | 8 +- .../org/apache/iceberg/SnapshotProducer.java | 20 +- .../java/org/apache/iceberg/V1Metadata.java | 10 +- .../java/org/apache/iceberg/V2Metadata.java | 3 +- .../apache/iceberg/avro/AvroFileAppender.java | 2 +- .../encryption/AesGcmOutputStream.java | 5 + .../iceberg/encryption/EncryptedFiles.java | 4 + .../iceberg/TestManifestEncryption.java | 247 ++++++++++++++++++ 13 files changed, 358 insertions(+), 29 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestManifestEncryption.java diff --git a/api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java b/api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java index b80549fbdef8..5092863809e2 100644 --- a/api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java +++ b/api/src/main/java/org/apache/iceberg/io/PositionOutputStream.java @@ -29,4 +29,15 @@ public abstract class PositionOutputStream extends OutputStream { * @throws IOException If the underlying stream throws IOException */ public abstract long getPos() throws IOException; + + /** + * Return the current stored length of the output. Can differ from the current position for + * encrypting streams, and for other non-length-preserving streams. + * + * @return current stored length in bytes + * @throws IOException If the underlying stream throws IOException + */ + public long storedLength() throws IOException { + return getPos(); + } } diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index c70dda2bd6d0..e8fbfef2ca1a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -32,10 +32,10 @@ import java.util.function.Function; import java.util.function.Predicate; import java.util.stream.Collectors; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.io.OutputFile; 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.Iterables; @@ -154,8 +154,8 @@ public RewriteManifests addManifest(ManifestFile manifest) { private ManifestFile copyManifest(ManifestFile manifest) { TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest.path()); - OutputFile newFile = newManifestOutput(); + InputFile toCopy = ops.io().newInputFile(manifest); + EncryptedOutputFile newFile = newManifestOutputFile(); return ManifestFiles.copyRewriteManifest( current.formatVersion(), manifest.partitionSpecId(), diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index e0919d9c7b2f..14e776a92d6a 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -22,11 +22,11 @@ import java.util.List; import java.util.Map; 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.io.OutputFile; 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.Lists; @@ -121,14 +121,14 @@ public FastAppend appendManifest(ManifestFile manifest) { private ManifestFile copyManifest(ManifestFile manifest) { TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest.path()); - OutputFile newManifestPath = newManifestOutput(); + InputFile toCopy = ops.io().newInputFile(manifest); + EncryptedOutputFile newManifestFile = newManifestOutputFile(); return ManifestFiles.copyAppendManifest( current.formatVersion(), manifest.partitionSpecId(), toCopy, current.specsById(), - newManifestPath, + newManifestFile, snapshotId(), summaryBuilder); } diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index f7b1add6bdc5..9009f19ec947 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -25,6 +25,8 @@ import org.apache.iceberg.ManifestReader.FileType; import org.apache.iceberg.avro.AvroEncoderUtil; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.ContentCache; @@ -157,11 +159,29 @@ public static ManifestWriter write(PartitionSpec spec, OutputFile outp */ public static ManifestWriter write( int formatVersion, PartitionSpec spec, OutputFile outputFile, Long snapshotId) { + return write( + formatVersion, spec, EncryptedFiles.plainAsEncryptedOutput(outputFile), snapshotId); + } + + /** + * Create a new {@link ManifestWriter} for the given format version. + * + * @param formatVersion a target format version + * @param spec a {@link PartitionSpec} + * @param encryptedOutputFile an {@link EncryptedOutputFile} where the manifest will be written + * @param snapshotId a snapshot ID for the manifest entries, or null for an inherited ID + * @return a manifest writer + */ + public static ManifestWriter write( + int formatVersion, + PartitionSpec spec, + EncryptedOutputFile encryptedOutputFile, + Long snapshotId) { switch (formatVersion) { case 1: - return new ManifestWriter.V1Writer(spec, outputFile, snapshotId); + return new ManifestWriter.V1Writer(spec, encryptedOutputFile, snapshotId); case 2: - return new ManifestWriter.V2Writer(spec, outputFile, snapshotId); + return new ManifestWriter.V2Writer(spec, encryptedOutputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -198,6 +218,21 @@ public static ManifestReader readDeleteManifest( */ public static ManifestWriter writeDeleteManifest( int formatVersion, PartitionSpec spec, OutputFile outputFile, Long snapshotId) { + return writeDeleteManifest( + formatVersion, spec, EncryptedFiles.plainAsEncryptedOutput(outputFile), snapshotId); + } + + /** + * Create a new {@link ManifestWriter} for the given format version. + * + * @param formatVersion a target format version + * @param spec a {@link PartitionSpec} + * @param outputFile an {@link EncryptedOutputFile} where the manifest will be written + * @param snapshotId a snapshot ID for the manifest entries, or null for an inherited ID + * @return a manifest writer + */ + public static ManifestWriter writeDeleteManifest( + int formatVersion, PartitionSpec spec, EncryptedOutputFile outputFile, Long snapshotId) { switch (formatVersion) { case 1: throw new IllegalArgumentException("Cannot write delete files in a v1 table"); @@ -254,7 +289,7 @@ static ManifestFile copyAppendManifest( int specId, InputFile toCopy, Map specsById, - OutputFile outputFile, + EncryptedOutputFile outputFile, long snapshotId, SnapshotSummary.Builder summaryBuilder) { // use metadata that will add the current snapshot's ID for the rewrite @@ -278,7 +313,7 @@ static ManifestFile copyRewriteManifest( int specId, InputFile toCopy, Map specsById, - OutputFile outputFile, + EncryptedOutputFile outputFile, long snapshotId, SnapshotSummary.Builder summaryBuilder) { // for a rewritten manifest all snapshot ids should be set. use empty metadata to throw an @@ -302,7 +337,7 @@ static ManifestFile copyRewriteManifest( private static ManifestFile copyManifestInternal( int formatVersion, ManifestReader reader, - OutputFile outputFile, + EncryptedOutputFile outputFile, long snapshotId, SnapshotSummary.Builder summaryBuilder, ManifestEntry.Status allowedEntryStatus) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 4865ccfc3b2d..cea907ddaca1 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -19,7 +19,9 @@ package org.apache.iceberg; import java.io.IOException; +import java.nio.ByteBuffer; import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.OutputFile; @@ -37,6 +39,7 @@ public abstract class ManifestWriter> implements FileAp static final long UNASSIGNED_SEQ = -1L; private final OutputFile file; + private final ByteBuffer keyMetadataBuffer; private final int specId; private final FileAppender> writer; private final Long snapshotId; @@ -52,13 +55,14 @@ public abstract class ManifestWriter> implements FileAp private long deletedRows = 0L; private Long minDataSequenceNumber = null; - private ManifestWriter(PartitionSpec spec, OutputFile file, Long snapshotId) { - this.file = file; + private ManifestWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + this.file = file.encryptingOutputFile(); this.specId = spec.specId(); - this.writer = newAppender(spec, file); + this.writer = newAppender(spec, this.file); this.snapshotId = snapshotId; this.reused = new GenericManifestEntry<>(spec.partitionType()); this.stats = new PartitionSummary(spec); + this.keyMetadataBuffer = (file.keyMetadata() == null) ? null : file.keyMetadata().buffer(); } protected abstract ManifestEntry prepare(ManifestEntry entry); @@ -204,7 +208,7 @@ public ManifestFile toManifestFile() { deletedFiles, deletedRows, stats.summaries(), - null); + keyMetadataBuffer); } @Override @@ -216,7 +220,7 @@ public void close() throws IOException { static class V2Writer extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; - V2Writer(PartitionSpec spec, OutputFile file, Long snapshotId) { + V2Writer(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { super(spec, file, snapshotId); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -250,7 +254,7 @@ protected FileAppender> newAppender( static class V2DeleteWriter extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; - V2DeleteWriter(PartitionSpec spec, OutputFile file, Long snapshotId) { + V2DeleteWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { super(spec, file, snapshotId); this.entryWrapper = new V2Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); } @@ -289,7 +293,7 @@ protected ManifestContent content() { static class V1Writer extends ManifestWriter { private final V1Metadata.IndexedManifestEntry entryWrapper; - V1Writer(PartitionSpec spec, OutputFile file, Long snapshotId) { + V1Writer(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { super(spec, file, snapshotId); this.entryWrapper = new V1Metadata.IndexedManifestEntry(spec.partitionType()); } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 5d3ec6e35f0d..c1dc6b58b784 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -30,6 +30,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; @@ -38,7 +39,6 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.io.OutputFile; 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; @@ -275,14 +275,14 @@ protected void add(ManifestFile manifest) { private ManifestFile copyManifest(ManifestFile manifest) { TableMetadata current = ops.current(); - InputFile toCopy = ops.io().newInputFile(manifest.path()); - OutputFile newManifestPath = newManifestOutput(); + InputFile toCopy = ops.io().newInputFile(manifest); + EncryptedOutputFile newManifestFile = newManifestOutputFile(); return ManifestFiles.copyAppendManifest( current.formatVersion(), manifest.partitionSpecId(), toCopy, current.specsById(), - newManifestPath, + newManifestFile, snapshotId(), appendedManifestsSummary); } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 757d0b78bca7..1c4491a6841a 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -43,6 +43,8 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Consumer; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.events.CreateSnapshotEvent; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.exceptions.CleanableFailure; @@ -255,7 +257,6 @@ public Snapshot apply() { .run(index -> manifestFiles[index] = manifestsWithMetadata.get(manifests.get(index))); writer.addAll(Arrays.asList(manifestFiles)); - } catch (IOException e) { throw new RuntimeIOException(e, "Failed to write manifest list file"); } @@ -499,6 +500,11 @@ protected OutputFile manifestListPath() { "snap-%d-%d-%s", snapshotId(), attempt.incrementAndGet(), commitUUID)))); } + /** + * @deprecated will be removed in 1.7.0; Use {@link SnapshotProducer#newManifestOutputFile} + * instead + */ + @Deprecated protected OutputFile newManifestOutput() { return ops.io() .newOutputFile( @@ -506,14 +512,22 @@ protected OutputFile newManifestOutput() { FileFormat.AVRO.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement()))); } + protected EncryptedOutputFile newManifestOutputFile() { + String manifestFileLocation = + ops.metadataFileLocation( + FileFormat.AVRO.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement())); + return EncryptingFileIO.combine(ops.io(), ops.encryption()) + .newEncryptingOutputFile(manifestFileLocation); + } + protected ManifestWriter newManifestWriter(PartitionSpec spec) { return ManifestFiles.write( - ops.current().formatVersion(), spec, newManifestOutput(), snapshotId()); + ops.current().formatVersion(), spec, newManifestOutputFile(), snapshotId()); } protected ManifestWriter newDeleteManifestWriter(PartitionSpec spec) { return ManifestFiles.writeDeleteManifest( - ops.current().formatVersion(), spec, newManifestOutput(), snapshotId()); + ops.current().formatVersion(), spec, newManifestOutputFile(), snapshotId()); } protected RollingManifestWriter newRollingManifestWriter(PartitionSpec spec) { diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index fddcee037488..30b04cd73124 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -43,7 +43,8 @@ private V1Metadata() {} ManifestFile.PARTITION_SUMMARIES, ManifestFile.ADDED_ROWS_COUNT, ManifestFile.EXISTING_ROWS_COUNT, - ManifestFile.DELETED_ROWS_COUNT); + ManifestFile.DELETED_ROWS_COUNT, + ManifestFile.KEY_METADATA); /** * A wrapper class to write any ManifestFile implementation to Avro using the v1 schema. @@ -97,6 +98,8 @@ public Object get(int pos) { return existingRowsCount(); case 10: return deletedRowsCount(); + case 11: + return keyMetadata(); default: throw new UnsupportedOperationException("Unknown field ordinal: " + pos); } @@ -187,6 +190,11 @@ public List partitions() { return wrapped.partitions(); } + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + @Override public ManifestFile copy() { return wrapped.copy(); diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index ba6bb4a31960..8f3b71d39925 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -47,7 +47,8 @@ private V2Metadata() {} ManifestFile.ADDED_ROWS_COUNT.asRequired(), ManifestFile.EXISTING_ROWS_COUNT.asRequired(), ManifestFile.DELETED_ROWS_COUNT.asRequired(), - ManifestFile.PARTITION_SUMMARIES); + ManifestFile.PARTITION_SUMMARIES, + ManifestFile.KEY_METADATA); /** * A wrapper class to write any ManifestFile implementation to Avro using the v2 write schema. diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java index 47e3c2c2849d..c69a8626f6c9 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java @@ -80,7 +80,7 @@ public Metrics metrics() { public long length() { if (stream != null) { try { - return stream.getPos(); + return stream.storedLength(); } catch (IOException e) { throw new RuntimeIOException(e, "Failed to get stream length"); } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index b4f723cca3e7..da437b7540db 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -129,6 +129,11 @@ public void close() throws IOException { targetStream.close(); } + @Override + public long storedLength() throws IOException { + return targetStream.storedLength(); + } + private void writeHeader() throws IOException { targetStream.write(HEADER_BYTES); isHeaderWritten = true; diff --git a/core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java b/core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java index c0fc41ca1385..b0e2e74d9336 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java +++ b/core/src/main/java/org/apache/iceberg/encryption/EncryptedFiles.java @@ -57,5 +57,9 @@ public static EncryptedOutputFile encryptedOutput( encryptedOutputFile, BaseEncryptionKeyMetadata.fromByteArray(keyMetadata)); } + public static EncryptedOutputFile plainAsEncryptedOutput(OutputFile encryptingOutputFile) { + return new BaseEncryptedOutputFile(encryptingOutputFile, EncryptionKeyMetadata.EMPTY); + } + private EncryptedFiles() {} } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java new file mode 100644 index 000000000000..b64324ec573a --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java @@ -0,0 +1,247 @@ +/* + * 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.required; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.avro.InvalidAvroMagicException; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptingFileIO; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.EncryptionTestHelpers; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +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.types.Conversions; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestManifestEncryption { + private static final FileIO FILE_IO = new TestTables.LocalFileIO(); + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "timestamp", Types.TimestampType.withZone()), + required(3, "category", Types.StringType.get()), + required(4, "data", Types.StringType.get()), + required(5, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA) + .identity("category") + .hour("timestamp") + .bucket("id", 16) + .build(); + + private static final long SNAPSHOT_ID = 987134631982734L; + private static final String PATH = + "s3://bucket/table/category=cheesy/timestamp_hour=10/id_bucket=3/file.avro"; + private static final FileFormat FORMAT = FileFormat.AVRO; + private static final PartitionData PARTITION = + DataFiles.data(SPEC, "category=cheesy/timestamp_hour=10/id_bucket=3"); + private static final Metrics METRICS = + new Metrics( + 1587L, + ImmutableMap.of(1, 15L, 2, 122L, 3, 4021L, 4, 9411L, 5, 15L), // sizes + ImmutableMap.of(1, 100L, 2, 100L, 3, 100L, 4, 100L, 5, 100L), // value counts + ImmutableMap.of(1, 0L, 2, 0L, 3, 0L, 4, 0L, 5, 0L), // null value counts + ImmutableMap.of(5, 10L), // nan value counts + ImmutableMap.of(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1)), // lower bounds + ImmutableMap.of(1, Conversions.toByteBuffer(Types.IntegerType.get(), 1))); // upper bounds + private static final List OFFSETS = ImmutableList.of(4L); + private static final Integer SORT_ORDER_ID = 2; + + private static final ByteBuffer CONTENT_KEY_METADATA = ByteBuffer.allocate(100); + + private static final DataFile DATA_FILE = + new GenericDataFile( + 0, + PATH, + FORMAT, + PARTITION, + 150972L, + METRICS, + CONTENT_KEY_METADATA, + OFFSETS, + SORT_ORDER_ID); + + private static final List EQUALITY_IDS = ImmutableList.of(1); + private static final int[] EQUALITY_ID_ARR = new int[] {1}; + + private static final DeleteFile DELETE_FILE = + new GenericDeleteFile( + SPEC.specId(), + FileContent.EQUALITY_DELETES, + PATH, + FORMAT, + PARTITION, + 22905L, + METRICS, + EQUALITY_ID_ARR, + SORT_ORDER_ID, + null, + CONTENT_KEY_METADATA); + + private static final EncryptionManager ENCRYPTION_MANAGER = + EncryptionTestHelpers.createEncryptionManager(); + + @Rule public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testV1Write() throws IOException { + ManifestFile manifest = writeManifest(1); + checkEntry( + readManifest(manifest), + ManifestWriter.UNASSIGNED_SEQ, + ManifestWriter.UNASSIGNED_SEQ, + FileContent.DATA); + } + + @Test + public void testV2Write() throws IOException { + ManifestFile manifest = writeManifest(2); + checkEntry( + readManifest(manifest), + ManifestWriter.UNASSIGNED_SEQ, + ManifestWriter.UNASSIGNED_SEQ, + FileContent.DATA); + } + + @Test + public void testV2WriteDelete() throws IOException { + ManifestFile manifest = writeDeleteManifest(2); + checkEntry( + readDeleteManifest(manifest), + ManifestWriter.UNASSIGNED_SEQ, + ManifestWriter.UNASSIGNED_SEQ, + FileContent.EQUALITY_DELETES); + } + + void checkEntry( + ManifestEntry entry, + Long expectedDataSequenceNumber, + Long expectedFileSequenceNumber, + FileContent content) { + Assert.assertEquals("Status", ManifestEntry.Status.ADDED, entry.status()); + Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, entry.snapshotId()); + Assert.assertEquals( + "Data sequence number", expectedDataSequenceNumber, entry.dataSequenceNumber()); + Assert.assertEquals( + "File sequence number", expectedFileSequenceNumber, entry.fileSequenceNumber()); + checkDataFile(entry.file(), content); + } + + void checkDataFile(ContentFile dataFile, FileContent content) { + // DataFile is the superclass of DeleteFile, so this method can check both + Assert.assertEquals("Content", content, dataFile.content()); + Assert.assertEquals("Path", PATH, dataFile.path()); + Assert.assertEquals("Format", FORMAT, dataFile.format()); + Assert.assertEquals("Partition", PARTITION, dataFile.partition()); + Assert.assertEquals("Record count", METRICS.recordCount(), (Long) dataFile.recordCount()); + Assert.assertEquals("Column sizes", METRICS.columnSizes(), dataFile.columnSizes()); + Assert.assertEquals("Value counts", METRICS.valueCounts(), dataFile.valueCounts()); + Assert.assertEquals("Null value counts", METRICS.nullValueCounts(), dataFile.nullValueCounts()); + Assert.assertEquals("NaN value counts", METRICS.nanValueCounts(), dataFile.nanValueCounts()); + Assert.assertEquals("Lower bounds", METRICS.lowerBounds(), dataFile.lowerBounds()); + Assert.assertEquals("Upper bounds", METRICS.upperBounds(), dataFile.upperBounds()); + Assert.assertEquals("Sort order id", SORT_ORDER_ID, dataFile.sortOrderId()); + if (dataFile.content() == FileContent.EQUALITY_DELETES) { + Assert.assertEquals(EQUALITY_IDS, dataFile.equalityFieldIds()); + } else { + Assert.assertNull(dataFile.equalityFieldIds()); + } + } + + private ManifestFile writeManifest(int formatVersion) throws IOException { + return writeManifest(DATA_FILE, formatVersion); + } + + private ManifestFile writeManifest(DataFile file, int formatVersion) throws IOException { + OutputFile manifestFile = + Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + EncryptedOutputFile encryptedManifest = ENCRYPTION_MANAGER.encrypt(manifestFile); + ManifestWriter writer = + ManifestFiles.write(formatVersion, SPEC, encryptedManifest, SNAPSHOT_ID); + try { + writer.add(file); + } finally { + writer.close(); + } + return writer.toManifestFile(); + } + + private ManifestEntry readManifest(ManifestFile manifest) throws IOException { + // First try to read without decryption + Assertions.assertThatThrownBy( + () -> + ManifestFiles.read( + manifest, + EncryptingFileIO.combine(FILE_IO, PlaintextEncryptionManager.instance()), + null)) + .isInstanceOf(RuntimeIOException.class) + .hasMessageContaining("Failed to open file") + .hasCauseInstanceOf(InvalidAvroMagicException.class); + + try (CloseableIterable> reader = + ManifestFiles.read(manifest, EncryptingFileIO.combine(FILE_IO, ENCRYPTION_MANAGER), null) + .entries()) { + List> files = Lists.newArrayList(reader); + Assert.assertEquals("Should contain only one data file", 1, files.size()); + return files.get(0); + } + } + + private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { + OutputFile manifestFile = + Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + EncryptedOutputFile encryptedManifest = ENCRYPTION_MANAGER.encrypt(manifestFile); + ManifestWriter writer = + ManifestFiles.writeDeleteManifest(formatVersion, SPEC, encryptedManifest, SNAPSHOT_ID); + try { + writer.add(DELETE_FILE); + } finally { + writer.close(); + } + return writer.toManifestFile(); + } + + private ManifestEntry readDeleteManifest(ManifestFile manifest) throws IOException { + try (CloseableIterable> reader = + ManifestFiles.readDeleteManifest( + manifest, EncryptingFileIO.combine(FILE_IO, ENCRYPTION_MANAGER), null) + .entries()) { + List> entries = Lists.newArrayList(reader); + Assert.assertEquals("Should contain only one delete file", 1, entries.size()); + return entries.get(0); + } + } +} From 589099c805eb1c848762698394552da169090b38 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 14 Mar 2024 03:18:09 -0700 Subject: [PATCH 0129/1019] Docs: Clarify table property on metrics for inferred column defaults (#9865) --- docs/docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 54d01bb3d383..d5e33529c0dd 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -61,7 +61,7 @@ Iceberg tables support table properties to configure table behavior, like the de | write.orc.bloom.filter.fpp | 0.05 | False positive probability for Bloom filter (must > 0.0 and < 1.0) | | write.location-provider.impl | null | Optional custom implementation for LocationProvider | | write.metadata.compression-codec | none | Metadata compression codec; none or gzip | -| write.metadata.metrics.max-inferred-column-defaults | 100 | Defines the maximum number of columns for which metrics are collected | +| write.metadata.metrics.max-inferred-column-defaults | 100 | Defines the maximum number of top level columns for which metrics are collected. Number of stored metrics can be higher than this limit for a table with nested fields | | write.metadata.metrics.default | truncate(16) | Default metrics mode for all columns in the table; none, counts, truncate(length), or full | | 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 | From c8fc09f3d6fd72bf383e051365f3a8c327ba7ae1 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Fri, 15 Mar 2024 16:26:43 +0900 Subject: [PATCH 0130/1019] Core: Migrate tests to JUnit5 (#9927) --- .../iceberg/TableMetadataParserCodecTest.java | 29 +- .../iceberg/TableMetadataParserTest.java | 41 +- .../iceberg/TestMetadataTableFilters.java | 176 +++--- .../iceberg/TestMetadataUpdateParser.java | 392 ++++++------ .../org/apache/iceberg/TestTableMetadata.java | 571 +++++++----------- .../TestTableMetadataSerialization.java | 65 +- 6 files changed, 524 insertions(+), 750 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TableMetadataParserCodecTest.java b/core/src/test/java/org/apache/iceberg/TableMetadataParserCodecTest.java index 597eb50fd384..a1aaa9a17f3c 100644 --- a/core/src/test/java/org/apache/iceberg/TableMetadataParserCodecTest.java +++ b/core/src/test/java/org/apache/iceberg/TableMetadataParserCodecTest.java @@ -18,36 +18,37 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.TableMetadataParser.Codec; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TableMetadataParserCodecTest { @Test public void testCompressionCodec() { - Assert.assertEquals(Codec.GZIP, Codec.fromName("gzip")); - Assert.assertEquals(Codec.GZIP, Codec.fromName("gZiP")); - Assert.assertEquals(Codec.GZIP, Codec.fromFileName("v3.gz.metadata.json")); - Assert.assertEquals(Codec.GZIP, Codec.fromFileName("v3-f326-4b66-a541-7b1c.gz.metadata.json")); - Assert.assertEquals(Codec.GZIP, Codec.fromFileName("v3-f326-4b66-a541-7b1c.metadata.json.gz")); - Assert.assertEquals(Codec.NONE, Codec.fromName("none")); - Assert.assertEquals(Codec.NONE, Codec.fromName("nOnE")); - Assert.assertEquals(Codec.NONE, Codec.fromFileName("v3.metadata.json")); - Assert.assertEquals(Codec.NONE, Codec.fromFileName("v3-f326-4b66-a541-7b1c.metadata.json")); + assertThat(Codec.fromName("gzip")).isEqualTo(Codec.GZIP); + assertThat(Codec.fromName("gZiP")).isEqualTo(Codec.GZIP); + assertThat(Codec.fromFileName("v3.gz.metadata.json")).isEqualTo(Codec.GZIP); + assertThat(Codec.fromFileName("v3-f326-4b66-a541-7b1c.gz.metadata.json")).isEqualTo(Codec.GZIP); + assertThat(Codec.fromFileName("v3-f326-4b66-a541-7b1c.metadata.json.gz")).isEqualTo(Codec.GZIP); + assertThat(Codec.fromName("none")).isEqualTo(Codec.NONE); + assertThat(Codec.fromName("nOnE")).isEqualTo(Codec.NONE); + assertThat(Codec.fromFileName("v3.metadata.json")).isEqualTo(Codec.NONE); + assertThat(Codec.fromFileName("v3-f326-4b66-a541-7b1c.metadata.json")).isEqualTo(Codec.NONE); } @Test public void testInvalidCodecName() { - Assertions.assertThatThrownBy(() -> Codec.fromName("invalid")) + assertThatThrownBy(() -> Codec.fromName("invalid")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid codec name: invalid"); } @Test public void testInvalidFileName() { - Assertions.assertThatThrownBy(() -> Codec.fromFileName("path/to/file.parquet")) + assertThatThrownBy(() -> Codec.fromFileName("path/to/file.parquet")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("path/to/file.parquet is not a valid metadata file"); } diff --git a/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java index 77a2e89a6fa8..5581818aa1f0 100644 --- a/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java +++ b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.TableMetadata.newTableMetadata; import static org.apache.iceberg.TableMetadataParser.getFileExtension; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.FileInputStream; @@ -29,6 +30,8 @@ import java.io.InputStream; import java.nio.file.Path; import java.nio.file.Paths; +import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.zip.GZIPInputStream; import java.util.zip.ZipException; @@ -37,30 +40,24 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types.BooleanType; -import org.junit.After; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TableMetadataParserTest { private static final Schema SCHEMA = new Schema(optional(1, "b", BooleanType.get())); - @Parameterized.Parameters(name = "codecName = {0}") - public static Object[] parameters() { - return new Object[] {"none", "gzip"}; + @Parameters(name = "codecName = {0}") + private static List parameters() { + return Arrays.asList("none", "gzip"); } - private final String codecName; + @Parameter private String codecName; - public TableMetadataParserTest(String codecName) { - this.codecName = codecName; - } - - @Test - public void testCompressionProperty() throws IOException { + @TestTemplate + public void testGzipCompressionProperty() throws IOException { Codec codec = Codec.fromName(codecName); String fileExtension = getFileExtension(codec); String fileName = "v3" + fileExtension; @@ -70,13 +67,13 @@ public void testCompressionProperty() throws IOException { String location = "file://tmp/db/table"; TableMetadata metadata = newTableMetadata(SCHEMA, unpartitioned(), location, properties); TableMetadataParser.write(metadata, outputFile); - Assert.assertEquals(codec == Codec.GZIP, isCompressed(fileName)); + assertThat(isCompressed(fileName)).isEqualTo(codec == Codec.GZIP); TableMetadata actualMetadata = TableMetadataParser.read((FileIO) null, Files.localInput(new File(fileName))); verifyMetadata(metadata, actualMetadata); } - @After + @AfterEach public void cleanup() throws IOException { Codec codec = Codec.fromName(codecName); Path metadataFilePath = Paths.get("v3" + getFileExtension(codec)); @@ -84,10 +81,10 @@ public void cleanup() throws IOException { } private void verifyMetadata(TableMetadata expected, TableMetadata actual) { - Assert.assertEquals(expected.schema().asStruct(), actual.schema().asStruct()); - Assert.assertEquals(expected.location(), actual.location()); - Assert.assertEquals(expected.lastColumnId(), actual.lastColumnId()); - Assert.assertEquals(expected.properties(), actual.properties()); + assertThat(actual.schema().asStruct()).isEqualTo(expected.schema().asStruct()); + assertThat(actual.location()).isEqualTo(expected.location()); + assertThat(actual.lastColumnId()).isEqualTo(expected.lastColumnId()); + assertThat(actual.properties()).isEqualTo(expected.properties()); } private boolean isCompressed(String path) throws IOException { diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index df950d9c0cfa..fadaeb079330 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -18,24 +18,24 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.Arrays; +import java.util.List; import java.util.Set; -import java.util.stream.StreamSupport; 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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestMetadataTableFilters extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetadataTableFilters extends TestBase { private static final Set aggFileTables = Sets.newHashSet( @@ -44,34 +44,29 @@ public class TestMetadataTableFilters extends TableTestBase { MetadataTableType.ALL_FILES, MetadataTableType.ALL_ENTRIES); - private final MetadataTableType type; - - @Parameterized.Parameters(name = "table_type = {0}, format = {1}") - public static Object[][] parameters() { - return new Object[][] { - {MetadataTableType.DATA_FILES, 1}, - {MetadataTableType.DATA_FILES, 2}, - {MetadataTableType.DELETE_FILES, 2}, - {MetadataTableType.FILES, 1}, - {MetadataTableType.FILES, 2}, - {MetadataTableType.ALL_DATA_FILES, 1}, - {MetadataTableType.ALL_DATA_FILES, 2}, - {MetadataTableType.ALL_DELETE_FILES, 2}, - {MetadataTableType.ALL_FILES, 1}, - {MetadataTableType.ALL_FILES, 2}, - {MetadataTableType.ENTRIES, 1}, - {MetadataTableType.ENTRIES, 2}, - {MetadataTableType.ALL_ENTRIES, 1}, - {MetadataTableType.ALL_ENTRIES, 2} - }; - } - - public TestMetadataTableFilters(MetadataTableType type, int formatVersion) { - super(formatVersion); - this.type = type; + @Parameter(index = 1) + private MetadataTableType type; + + @Parameters(name = "formatVersion = {0}, table_type = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, MetadataTableType.DATA_FILES}, + new Object[] {2, MetadataTableType.DATA_FILES}, + new Object[] {2, MetadataTableType.DELETE_FILES}, + new Object[] {1, MetadataTableType.FILES}, + new Object[] {2, MetadataTableType.FILES}, + new Object[] {1, MetadataTableType.ALL_DATA_FILES}, + new Object[] {2, MetadataTableType.ALL_DATA_FILES}, + new Object[] {2, MetadataTableType.ALL_DELETE_FILES}, + new Object[] {1, MetadataTableType.ALL_FILES}, + new Object[] {2, MetadataTableType.ALL_FILES}, + new Object[] {1, MetadataTableType.ENTRIES}, + new Object[] {2, MetadataTableType.ENTRIES}, + new Object[] {1, MetadataTableType.ALL_ENTRIES}, + new Object[] {2, MetadataTableType.ALL_ENTRIES}); } - @Before + @BeforeEach @Override public void setupTable() throws Exception { super.setupTable(); @@ -99,10 +94,7 @@ public void setupTable() throws Exception { .newDelete() .deleteFromRowFilter(Expressions.alwaysTrue()) .commit(); // Removes all entries - Assert.assertEquals( - "Current snapshot should be made empty", - 0, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).isEmpty(); } } @@ -195,21 +187,21 @@ private Expression dummyExpression() { } } - @Test + @TestTemplate public void testNoFilter() { Table metadataTable = createMetadataTable(); TableScan scan = metadataTable.newScan().select(partitionColumn("data_bucket")); CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(4), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(4)); validateFileScanTasks(tasks, 0); validateFileScanTasks(tasks, 1); validateFileScanTasks(tasks, 2); validateFileScanTasks(tasks, 3); } - @Test + @TestTemplate public void testAnd() { Table metadataTable = createMetadataTable(); @@ -218,23 +210,23 @@ public void testAnd() { TableScan scan = metadataTable.newScan().filter(and); CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(1), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(1)); validateFileScanTasks(tasks, 0); } - @Test + @TestTemplate public void testLt() { Table metadataTable = createMetadataTable(); Expression lt = Expressions.lessThan(partitionColumn("data_bucket"), 2); TableScan scan = metadataTable.newScan().filter(lt); CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(2), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(2)); validateFileScanTasks(tasks, 0); validateFileScanTasks(tasks, 1); } - @Test + @TestTemplate public void testOr() { Table metadataTable = createMetadataTable(); @@ -244,14 +236,14 @@ public void testOr() { CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(4), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(4)); validateFileScanTasks(tasks, 0); validateFileScanTasks(tasks, 1); validateFileScanTasks(tasks, 2); validateFileScanTasks(tasks, 3); } - @Test + @TestTemplate public void testNot() { Table metadataTable = createMetadataTable(); @@ -259,12 +251,12 @@ public void testNot() { TableScan scan = metadataTable.newScan().filter(not); CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(2), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(2)); validateFileScanTasks(tasks, 2); validateFileScanTasks(tasks, 3); } - @Test + @TestTemplate public void testIn() { Table metadataTable = createMetadataTable(); @@ -272,20 +264,20 @@ public void testIn() { TableScan scan = metadataTable.newScan().filter(set); CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(2), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(2)); validateFileScanTasks(tasks, 2); validateFileScanTasks(tasks, 3); } - @Test + @TestTemplate public void testNotNull() { Table metadataTable = createMetadataTable(); Expression unary = Expressions.notNull(partitionColumn("data_bucket")); TableScan scan = metadataTable.newScan().filter(unary); CloseableIterable tasks = scan.planFiles(); - Assert.assertEquals(expectedScanTaskCount(4), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(4)); validateFileScanTasks(tasks, 0); validateFileScanTasks(tasks, 1); @@ -293,7 +285,7 @@ public void testNotNull() { validateFileScanTasks(tasks, 3); } - @Test + @TestTemplate public void testPlanTasks() { Table metadataTable = createMetadataTable(); @@ -302,13 +294,13 @@ public void testPlanTasks() { TableScan scan = metadataTable.newScan().filter(and); CloseableIterable tasks = scan.planTasks(); - Assert.assertEquals(1, Iterables.size(tasks)); + assertThat(tasks).hasSize(1); validateCombinedScanTasks(tasks, 0); } - @Test + @TestTemplate public void testPartitionSpecEvolutionRemovalV1() { - Assume.assumeTrue(formatVersion == 1); + assumeThat(formatVersion).isEqualTo(1); // Change spec and add two data files table.updateSpec().removeField(Expressions.bucket("data", 16)).addField("id").commit(); @@ -348,10 +340,7 @@ public void testPartitionSpecEvolutionRemovalV1() { .newDelete() .deleteFromRowFilter(Expressions.alwaysTrue()) .commit(); // Removes all entries - Assert.assertEquals( - "Current snapshot should be made empty", - 0, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).isEmpty(); } Table metadataTable = createMetadataTable(); @@ -361,7 +350,7 @@ public void testPartitionSpecEvolutionRemovalV1() { CloseableIterable tasks = scan.planFiles(); // All 4 original data files written by old spec, plus one data file written by new spec - Assert.assertEquals(expectedScanTaskCount(5), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(5)); filter = Expressions.and(Expressions.equal(partitionColumn("data_bucket"), 0), dummyExpression()); @@ -370,12 +359,12 @@ public void testPartitionSpecEvolutionRemovalV1() { // 1 original data file written by old spec (V1 filters out new specs which don't have this // value) - Assert.assertEquals(expectedScanTaskCount(1), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(1)); } - @Test + @TestTemplate public void testPartitionSpecEvolutionRemovalV2() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); // Change spec and add two data and delete files each table.updateSpec().removeField(Expressions.bucket("data", 16)).addField("id").commit(); @@ -433,10 +422,7 @@ public void testPartitionSpecEvolutionRemovalV2() { .newDelete() .deleteFromRowFilter(Expressions.alwaysTrue()) .commit(); // Removes all entries - Assert.assertEquals( - "Current snapshot should be made empty", - 0, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).isEmpty(); } Table metadataTable = createMetadataTable(); @@ -447,7 +433,7 @@ public void testPartitionSpecEvolutionRemovalV2() { // All 4 original data/delete files written by old spec, plus one new data file/delete file // written by new spec - Assert.assertEquals(expectedScanTaskCount(5), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(5)); filter = Expressions.and(Expressions.equal(partitionColumn("data_bucket"), 0), dummyExpression()); @@ -456,12 +442,12 @@ public void testPartitionSpecEvolutionRemovalV2() { // 1 original data/delete files written by old spec, plus both of new data file/delete file // written by new spec - Assert.assertEquals(expectedScanTaskCount(3), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(3)); } - @Test + @TestTemplate public void testPartitionSpecEvolutionAdditiveV1() { - Assume.assumeTrue(formatVersion == 1); + assumeThat(formatVersion).isEqualTo(1); // Change spec and add two data files table.updateSpec().addField("id").commit(); @@ -503,10 +489,7 @@ public void testPartitionSpecEvolutionAdditiveV1() { .newDelete() .deleteFromRowFilter(Expressions.alwaysTrue()) .commit(); // Removes all entries - Assert.assertEquals( - "Current snapshot should be made empty", - 0, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).isEmpty(); } Table metadataTable = createMetadataTable(); @@ -517,7 +500,7 @@ public void testPartitionSpecEvolutionAdditiveV1() { // All 4 original data/delete files written by old spec, plus one new data file written by new // spec - Assert.assertEquals(expectedScanTaskCount(5), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(5)); filter = Expressions.and(Expressions.equal(partitionColumn("data_bucket"), 0), dummyExpression()); @@ -525,12 +508,12 @@ public void testPartitionSpecEvolutionAdditiveV1() { tasks = scan.planFiles(); // 1 original data file written by old spec, plus 1 new data file written by new spec - Assert.assertEquals(expectedScanTaskCount(2), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(2)); } - @Test + @TestTemplate public void testPartitionSpecEvolutionAdditiveV2() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); // Change spec and add two data and delete files each table.updateSpec().addField("id").commit(); @@ -588,10 +571,7 @@ public void testPartitionSpecEvolutionAdditiveV2() { .newDelete() .deleteFromRowFilter(Expressions.alwaysTrue()) .commit(); // Removes all entries - Assert.assertEquals( - "Current snapshot should be made empty", - 0, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).isEmpty(); } Table metadataTable = createMetadataTable(); @@ -602,7 +582,7 @@ public void testPartitionSpecEvolutionAdditiveV2() { // All 4 original data/delete files written by old spec, plus one new data file/delete file // written by new spec - Assert.assertEquals(expectedScanTaskCount(5), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(5)); filter = Expressions.and(Expressions.equal(partitionColumn("data_bucket"), 0), dummyExpression()); @@ -611,22 +591,24 @@ public void testPartitionSpecEvolutionAdditiveV2() { // 1 original data/delete files written by old spec, plus 1 of new data file/delete file written // by new spec - Assert.assertEquals(expectedScanTaskCount(2), Iterables.size(tasks)); + assertThat(tasks).hasSize(expectedScanTaskCount(2)); } private void validateFileScanTasks(CloseableIterable fileScanTasks, int partValue) { - Assert.assertTrue( - "File scan tasks do not include correct file", - StreamSupport.stream(fileScanTasks.spliterator(), false) - .anyMatch(t -> manifestHasPartition(manifest(t), partValue))); + assertThat(fileScanTasks) + .as("File scan tasks do not include correct file") + .anyMatch(t -> manifestHasPartition(manifest(t), partValue)); } private void validateCombinedScanTasks(CloseableIterable tasks, int partValue) { - Assert.assertTrue( - "File scan tasks do not include correct partition value", - StreamSupport.stream(tasks.spliterator(), false) - .flatMap(c -> c.files().stream().map(this::manifest)) - .anyMatch(m -> manifestHasPartition(m, partValue))); + assertThat(tasks) + .as("File scan tasks do not include correct partition value") + .allSatisfy( + task -> { + assertThat(task.files()) + .map(this::manifest) + .anyMatch(m -> manifestHasPartition(m, partValue)); + }); } private boolean manifestHasPartition(ManifestFile mf, int partValue) { diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java index 64e01f8cd8ff..bfed6ebebe2c 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataUpdateParser.java @@ -19,14 +19,16 @@ package org.apache.iceberg; import static org.apache.iceberg.Files.localInput; +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; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; -import java.util.Objects; import java.util.Set; -import java.util.UUID; import java.util.stream.IntStream; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -38,15 +40,12 @@ import org.apache.iceberg.util.Pair; import org.apache.iceberg.view.ImmutableViewVersion; import org.apache.iceberg.view.ViewVersion; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestMetadataUpdateParser { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private static final Schema ID_DATA_SCHEMA = new Schema( @@ -59,7 +58,7 @@ public void testMetadataUpdateWithoutActionCannotDeserialize() { ImmutableList.of("{\"action\":null,\"format-version\":2}", "{\"format-version\":2}"); for (String json : invalidJson) { - Assertions.assertThatThrownBy(() -> MetadataUpdateParser.fromJson(json)) + assertThatThrownBy(() -> MetadataUpdateParser.fromJson(json)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse metadata update. Missing field: action"); } @@ -81,10 +80,9 @@ public void testAssignUUIDFromJson() { String expected = "{\"action\":\"assign-uuid\",\"uuid\":\"9510c070-5e6d-4b40-bf40-a8915bb76e5d\"}"; MetadataUpdate actual = new MetadataUpdate.AssignUUID(uuid); - Assert.assertEquals( - "Assign UUID should convert to the correct JSON value", - expected, - MetadataUpdateParser.toJson(actual)); + assertThat(MetadataUpdateParser.toJson(actual)) + .as("Assign UUID should convert to the correct JSON value") + .isEqualTo(expected); } /** UpgradeFormatVersion * */ @@ -104,10 +102,9 @@ public void testUpgradeFormatVersionFromJson() { String expected = "{\"action\":\"upgrade-format-version\",\"format-version\":2}"; MetadataUpdate.UpgradeFormatVersion actual = new MetadataUpdate.UpgradeFormatVersion(formatVersion); - Assert.assertEquals( - "Upgrade format version should convert to the correct JSON value", - expected, - MetadataUpdateParser.toJson(actual)); + assertThat(MetadataUpdateParser.toJson(actual)) + .as("Upgrade format version should convert to the correct JSON value") + .isEqualTo(expected); } /** AddSchema * */ @@ -145,7 +142,9 @@ public void testAddSchemaToJson() { SchemaParser.toJson(schema), lastColumnId); MetadataUpdate update = new MetadataUpdate.AddSchema(schema, lastColumnId); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals("Add schema should convert to the correct JSON value", expected, actual); + assertThat(actual) + .as("Add schema should convert to the correct JSON value") + .isEqualTo(expected); } /** SetCurrentSchema * */ @@ -165,8 +164,9 @@ public void testSetCurrentSchemaToJson() { String expected = String.format("{\"action\":\"%s\",\"schema-id\":%d}", action, schemaId); MetadataUpdate update = new MetadataUpdate.SetCurrentSchema(schemaId); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set current schema should convert to the correct JSON value", expected, actual); + assertThat(actual) + .as("Set current schema should convert to the correct JSON value") + .isEqualTo(expected); } /** AddPartitionSpec * */ @@ -277,8 +277,9 @@ public void testAddPartitionSpecToJson() { .build(); MetadataUpdate update = new MetadataUpdate.AddPartitionSpec(expectedSpec); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Add partition spec should convert to the correct JSON value", expected, actual); + assertThat(actual) + .as("Add partition spec should convert to the correct JSON value") + .isEqualTo(expected); } /** SetDefaultPartitionSpec * */ @@ -289,8 +290,9 @@ public void testSetDefaultPartitionSpecToJson() { String expected = String.format("{\"action\":\"%s\",\"spec-id\":%d}", action, specId); MetadataUpdate update = new MetadataUpdate.SetDefaultPartitionSpec(specId); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set default partition spec should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Set default partition spec should serialize to the correct JSON value") + .isEqualTo(expected); } @Test @@ -319,10 +321,9 @@ public void testAddSortOrderToJson() { String.format( "{\"action\":\"%s\",\"sort-order\":%s}", action, SortOrderParser.toJson(sortOrder)); MetadataUpdate update = new MetadataUpdate.AddSortOrder(sortOrder); - Assert.assertEquals( - "Add sort order should serialize to the correct JSON value", - expected, - MetadataUpdateParser.toJson(update)); + assertThat(MetadataUpdateParser.toJson(update)) + .as("Add sort order should serialize to the correct JSON value") + .isEqualTo(expected); } @Test @@ -352,8 +353,9 @@ public void testSetDefaultSortOrderToJson() { String.format("{\"action\":\"%s\",\"sort-order-id\":%d}", action, sortOrderId); MetadataUpdate update = new MetadataUpdate.SetDefaultSortOrder(sortOrderId); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set default sort order should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Set default sort order should serialize to the correct JSON value") + .isEqualTo(expected); } @Test @@ -389,8 +391,9 @@ public void testAddSnapshotToJson() throws IOException { String expected = String.format("{\"action\":\"%s\",\"snapshot\":%s}", action, snapshotJson); MetadataUpdate update = new MetadataUpdate.AddSnapshot(snapshot); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Add snapshot should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Add snapshot should serialize to the correct JSON value") + .isEqualTo(expected); } @Test @@ -435,8 +438,9 @@ public void testRemoveSnapshotsToJson() { String expected = String.format("{\"action\":\"%s\",\"snapshot-ids\":[2]}", action); MetadataUpdate update = new MetadataUpdate.RemoveSnapshot(snapshotId); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Remove snapshots should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Remove snapshots should serialize to the correct JSON value") + .isEqualTo(expected); } /** RemoveSnapshotRef * */ @@ -454,10 +458,9 @@ public void testRemoveSnapshotRefToJson() { String snapshotRef = "snapshot-ref"; String expected = "{\"action\":\"remove-snapshot-ref\",\"ref-name\":\"snapshot-ref\"}"; MetadataUpdate actual = new MetadataUpdate.RemoveSnapshotRef(snapshotRef); - Assert.assertEquals( - "RemoveSnapshotRef should convert to the correct JSON value", - expected, - MetadataUpdateParser.toJson(actual)); + assertThat(MetadataUpdateParser.toJson(actual)) + .as("RemoveSnapshotRef should convert to the correct JSON value") + .isEqualTo(expected); } /** SetSnapshotRef * */ @@ -599,10 +602,10 @@ public void testSetSnapshotRefTagToJsonDefault() { new MetadataUpdate.SetSnapshotRef( refName, snapshotId, type, minSnapshotsToKeep, maxSnapshotAgeMs, maxRefAgeMs); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set snapshot ref should serialize to the correct JSON value for tag with default fields", - expected, - actual); + assertThat(actual) + .as( + "Set snapshot ref should serialize to the correct JSON value for tag with default fields") + .isEqualTo(expected); } @Test @@ -620,10 +623,9 @@ public void testSetSnapshotRefTagToJsonAllFields() { new MetadataUpdate.SetSnapshotRef( refName, snapshotId, type, minSnapshotsToKeep, maxSnapshotAgeMs, maxRefAgeMs); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set snapshot ref should serialize to the correct JSON value for tag with all fields", - expected, - actual); + assertThat(actual) + .as("Set snapshot ref should serialize to the correct JSON value for tag with all fields") + .isEqualTo(expected); } @Test @@ -640,10 +642,10 @@ public void testSetSnapshotRefBranchToJsonDefault() { new MetadataUpdate.SetSnapshotRef( refName, snapshotId, type, minSnapshotsToKeep, maxSnapshotAgeMs, maxRefAgeMs); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set snapshot ref should serialize to the correct JSON value for branch with default fields", - expected, - actual); + assertThat(actual) + .as( + "Set snapshot ref should serialize to the correct JSON value for branch with default fields") + .isEqualTo(expected); } @Test @@ -661,10 +663,10 @@ public void testSetSnapshotRefBranchToJsonAllFields() { new MetadataUpdate.SetSnapshotRef( refName, snapshotId, type, minSnapshotsToKeep, maxSnapshotAgeMs, maxRefAgeMs); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set snapshot ref should serialize to the correct JSON value for branch with all fields", - expected, - actual); + assertThat(actual) + .as( + "Set snapshot ref should serialize to the correct JSON value for branch with all fields") + .isEqualTo(expected); } /** SetProperties */ @@ -703,7 +705,7 @@ public void testSetPropertiesFromJsonFailsWhenDeserializingNullValues() { props.put("prop2", null); String propsMap = "{\"prop1\":\"val1\",\"prop2\":null}"; String json = String.format("{\"action\":\"%s\",\"updated\":%s}", action, propsMap); - Assertions.assertThatThrownBy(() -> MetadataUpdateParser.fromJson(json)) + assertThatThrownBy(() -> MetadataUpdateParser.fromJson(json)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: prop2: null"); } @@ -719,8 +721,9 @@ public void testSetPropertiesToJson() { String expected = String.format("{\"action\":\"%s\",\"updates\":%s}", action, propsMap); MetadataUpdate update = new MetadataUpdate.SetProperties(props); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Set properties should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Set properties should serialize to the correct JSON value") + .isEqualTo(expected); } /** RemoveProperties */ @@ -755,8 +758,9 @@ public void testRemovePropertiesToJson() { String expected = String.format("{\"action\":\"%s\",\"removals\":%s}", action, toRemoveAsJSON); MetadataUpdate update = new MetadataUpdate.RemoveProperties(toRemove); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Remove properties should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Remove properties should serialize to the correct JSON value") + .isEqualTo(expected); } /** SetLocation */ @@ -776,8 +780,9 @@ public void testSetLocationToJson() { String expected = String.format("{\"action\":\"%s\",\"location\":\"%s\"}", action, location); MetadataUpdate update = new MetadataUpdate.SetLocation(location); String actual = MetadataUpdateParser.toJson(update); - Assert.assertEquals( - "Remove properties should serialize to the correct JSON value", expected, actual); + assertThat(actual) + .as("Remove properties should serialize to the correct JSON value") + .isEqualTo(expected); } @Test @@ -807,10 +812,9 @@ public void testSetStatistics() { ImmutableMap.of("prop-key", "prop-value"))))); assertEquals( MetadataUpdateParser.SET_STATISTICS, expected, MetadataUpdateParser.fromJson(json)); - Assert.assertEquals( - "Set statistics should convert to the correct JSON value", - json, - MetadataUpdateParser.toJson(expected)); + assertThat(MetadataUpdateParser.toJson(expected)) + .as("Set statistics should convert to the correct JSON value") + .isEqualTo(json); } @Test @@ -819,10 +823,9 @@ public void testRemoveStatistics() { MetadataUpdate expected = new MetadataUpdate.RemoveStatistics(1940541653261589030L); assertEquals( MetadataUpdateParser.REMOVE_STATISTICS, expected, MetadataUpdateParser.fromJson(json)); - Assert.assertEquals( - "Remove statistics should convert to the correct JSON value", - json, - MetadataUpdateParser.toJson(expected)); + assertThat(MetadataUpdateParser.toJson(expected)) + .as("Remove statistics should convert to the correct JSON value") + .isEqualTo(json); } /** AddViewVersion */ @@ -864,7 +867,7 @@ public void testAddViewVersionToJson() { action); MetadataUpdate update = new MetadataUpdate.AddViewVersion(viewVersion); - Assertions.assertThat(MetadataUpdateParser.toJson(update)).isEqualTo(expected); + assertThat(MetadataUpdateParser.toJson(update)).isEqualTo(expected); } /** SetCurrentViewVersion */ @@ -881,7 +884,7 @@ public void testSetCurrentViewVersionToJson() { String action = MetadataUpdateParser.SET_CURRENT_VIEW_VERSION; String expected = String.format("{\"action\":\"%s\",\"view-version-id\":23}", action); MetadataUpdate update = new MetadataUpdate.SetCurrentViewVersion(23); - Assertions.assertThat(MetadataUpdateParser.toJson(update)).isEqualTo(expected); + assertThat(MetadataUpdateParser.toJson(update)).isEqualTo(expected); } @Test @@ -904,10 +907,9 @@ public void testSetPartitionStatistics() { MetadataUpdateParser.SET_PARTITION_STATISTICS, expected, MetadataUpdateParser.fromJson(json)); - Assert.assertEquals( - "Set partition statistics should convert to the correct JSON value", - json, - MetadataUpdateParser.toJson(expected)); + assertThat(MetadataUpdateParser.toJson(expected)) + .as("Set partition statistics should convert to the correct JSON value") + .isEqualTo(json); } @Test @@ -919,10 +921,9 @@ public void testRemovePartitionStatistics() { MetadataUpdateParser.REMOVE_PARTITION_STATISTICS, expected, MetadataUpdateParser.fromJson(json)); - Assert.assertEquals( - "Remove partition statistics should convert to the correct JSON value", - json, - MetadataUpdateParser.toJson(expected)); + assertThat(MetadataUpdateParser.toJson(expected)) + .as("Remove partition statistics should convert to the correct JSON value") + .isEqualTo(json); } public void assertEquals( @@ -1030,49 +1031,45 @@ public void assertEquals( (MetadataUpdate.SetCurrentViewVersion) actualUpdate); break; default: - Assert.fail("Unrecognized metadata update action: " + action); + fail("Unrecognized metadata update action: " + action); } } private static void assertEqualsAssignUUID( MetadataUpdate.AssignUUID expected, MetadataUpdate.AssignUUID actual) { - Assert.assertEquals("UUIDs should be equal", expected.uuid(), actual.uuid()); + assertThat(actual.uuid()).isEqualTo(expected.uuid()); } private static void assertEqualsUpgradeFormatVersion( MetadataUpdate.UpgradeFormatVersion expected, MetadataUpdate.UpgradeFormatVersion actual) { - Assert.assertEquals( - "Format version should be equal", expected.formatVersion(), actual.formatVersion()); + assertThat(actual.formatVersion()).isEqualTo(expected.formatVersion()); } private static void assertEqualsAddSchema( MetadataUpdate.AddSchema expected, MetadataUpdate.AddSchema actual) { - Assert.assertTrue("Schemas should be the same", expected.schema().sameSchema(actual.schema())); - Assert.assertEquals( - "Last column id should be equal", expected.lastColumnId(), actual.lastColumnId()); + assertThat(actual.schema().asStruct()).isEqualTo(expected.schema().asStruct()); + assertThat(actual.lastColumnId()).isEqualTo(expected.lastColumnId()); } private static void assertEqualsSetCurrentSchema( MetadataUpdate.SetCurrentSchema expected, MetadataUpdate.SetCurrentSchema actual) { - Assert.assertEquals("Schema id should be equal", expected.schemaId(), actual.schemaId()); + assertThat(actual.schemaId()).isEqualTo(expected.schemaId()); } private static void assertEqualsSetDefaultPartitionSpec( MetadataUpdate.SetDefaultPartitionSpec expected, MetadataUpdate.SetDefaultPartitionSpec actual) { - Assertions.assertThat(actual.specId()).isEqualTo(expected.specId()); + assertThat(actual.specId()).isEqualTo(expected.specId()); } private static void assertEqualsAddPartitionSpec( MetadataUpdate.AddPartitionSpec expected, MetadataUpdate.AddPartitionSpec actual) { - Assert.assertEquals( - "Unbound partition specs should have the same spec id", - expected.spec().specId(), - actual.spec().specId()); - Assert.assertEquals( - "Unbound partition specs should have the same number of fields", - expected.spec().fields().size(), - actual.spec().fields().size()); + assertThat(actual.spec().specId()) + .as("Unbound partition specs should have the same spec id") + .isEqualTo(expected.spec().specId()); + assertThat(actual.spec().fields()) + .as("Unbound partition specs should have the same number of fields") + .hasSameSizeAs(expected.spec().fields()); IntStream.range(0, expected.spec().fields().size()) .forEachOrdered( @@ -1081,27 +1078,23 @@ private static void assertEqualsAddPartitionSpec( expected.spec().fields().get(i); UnboundPartitionSpec.UnboundPartitionField actualField = actual.spec().fields().get(i); - Assert.assertTrue( - "Fields of the unbound partition spec should be the same", - Objects.equals(expectedField.partitionId(), actualField.partitionId()) - && expectedField.name().equals(actualField.name()) - && Objects.equals( - expectedField.transformAsString(), actualField.transformAsString()) - && expectedField.sourceId() == actualField.sourceId()); + assertThat(actualField.partitionId()).isEqualTo(expectedField.partitionId()); + assertThat(actualField.name()).isEqualTo(expectedField.name()); + assertThat(actualField.transformAsString()) + .isEqualTo(expectedField.transformAsString()); + assertThat(actualField.sourceId()).isEqualTo(expectedField.sourceId()); }); } private static void assertEqualsAddSortOrder( MetadataUpdate.AddSortOrder expected, MetadataUpdate.AddSortOrder actual) { - Assert.assertEquals( - "Order id of the sort order should be the same", - expected.sortOrder().orderId(), - actual.sortOrder().orderId()); + assertThat(actual.sortOrder().orderId()) + .as("Order id of the sort order should be the same") + .isEqualTo(expected.sortOrder().orderId()); - Assert.assertEquals( - "Sort orders should have the same number of fields", - expected.sortOrder().fields().size(), - actual.sortOrder().fields().size()); + assertThat(actual.sortOrder().fields()) + .as("Sort orders should have the same number of fields") + .hasSameSizeAs(expected.sortOrder().fields()); IntStream.range(0, expected.sortOrder().fields().size()) .forEachOrdered( @@ -1109,45 +1102,31 @@ private static void assertEqualsAddSortOrder( UnboundSortOrder.UnboundSortField expectedField = expected.sortOrder().fields().get(i); UnboundSortOrder.UnboundSortField actualField = actual.sortOrder().fields().get(i); - Assert.assertTrue( - "Fields of the sort order should be the same", - expectedField.sourceId() == actualField.sourceId() - && expectedField.nullOrder().equals(actualField.nullOrder()) - && expectedField.direction().equals(actualField.direction()) - && Objects.equals( - expectedField.transformAsString(), actualField.transformAsString())); + assertThat(actualField.sourceId()).isEqualTo(expectedField.sourceId()); + assertThat(actualField.nullOrder()).isEqualTo(expectedField.nullOrder()); + assertThat(actualField.direction()).isEqualTo(expectedField.direction()); + assertThat(actualField.transformAsString()) + .isEqualTo(expectedField.transformAsString()); }); } private static void assertEqualsSetDefaultSortOrder( MetadataUpdate.SetDefaultSortOrder expected, MetadataUpdate.SetDefaultSortOrder actual) { - Assert.assertEquals( - "Sort order id should be the same", expected.sortOrderId(), actual.sortOrderId()); + assertThat(actual.sortOrderId()).isEqualTo(expected.sortOrderId()); } private static void assertEqualsSetStatistics( MetadataUpdate.SetStatistics expected, MetadataUpdate.SetStatistics actual) { - Assert.assertEquals("Snapshot IDs should be equal", expected.snapshotId(), actual.snapshotId()); - Assert.assertEquals( - "Statistics files snapshot IDs should be equal", - expected.statisticsFile().snapshotId(), - actual.statisticsFile().snapshotId()); - Assert.assertEquals( - "Statistics files paths should be equal", - expected.statisticsFile().path(), - actual.statisticsFile().path()); - Assert.assertEquals( - "Statistics files size should be equal", - expected.statisticsFile().fileSizeInBytes(), - actual.statisticsFile().fileSizeInBytes()); - Assert.assertEquals( - "Statistics files footer size should be equal", - expected.statisticsFile().fileFooterSizeInBytes(), - actual.statisticsFile().fileFooterSizeInBytes()); - Assert.assertEquals( - "Statistics blob list size should be equal", - expected.statisticsFile().blobMetadata().size(), - actual.statisticsFile().blobMetadata().size()); + assertThat(actual.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(actual.statisticsFile().snapshotId()) + .isEqualTo(expected.statisticsFile().snapshotId()); + assertThat(actual.statisticsFile().path()).isEqualTo(expected.statisticsFile().path()); + assertThat(actual.statisticsFile().fileSizeInBytes()) + .isEqualTo(expected.statisticsFile().fileSizeInBytes()); + assertThat(actual.statisticsFile().fileFooterSizeInBytes()) + .isEqualTo(expected.statisticsFile().fileFooterSizeInBytes()); + assertThat(actual.statisticsFile().blobMetadata()) + .hasSameSizeAs(expected.statisticsFile().blobMetadata()); Streams.zip( expected.statisticsFile().blobMetadata().stream(), @@ -1158,123 +1137,90 @@ private static void assertEqualsSetStatistics( BlobMetadata expectedBlob = pair.first(); BlobMetadata actualBlob = pair.second(); - Assert.assertEquals( - "Expected blob type should be equal", expectedBlob.type(), actualBlob.type()); - Assert.assertEquals( - "Expected blob fields should be equal", - expectedBlob.fields(), - actualBlob.fields()); - Assert.assertEquals( - "Expected blob source snapshot ID should be equal", - expectedBlob.sourceSnapshotId(), - actualBlob.sourceSnapshotId()); - Assert.assertEquals( - "Expected blob source snapshot sequence number should be equal", - expectedBlob.sourceSnapshotSequenceNumber(), - actualBlob.sourceSnapshotSequenceNumber()); - Assert.assertEquals( - "Expected blob properties should be equal", - expectedBlob.properties(), - actualBlob.properties()); + assertThat(actualBlob.type()).isEqualTo(expectedBlob.type()); + assertThat(actualBlob.fields()).isEqualTo(expectedBlob.fields()); + assertThat(actualBlob.sourceSnapshotId()).isEqualTo(expectedBlob.sourceSnapshotId()); + assertThat(actualBlob.sourceSnapshotSequenceNumber()) + .isEqualTo(expectedBlob.sourceSnapshotSequenceNumber()); + assertThat(actualBlob.properties()).isEqualTo(expectedBlob.properties()); }); } private static void assertEqualsRemoveStatistics( MetadataUpdate.RemoveStatistics expected, MetadataUpdate.RemoveStatistics actual) { - Assert.assertEquals( - "Snapshots to remove should be the same", expected.snapshotId(), actual.snapshotId()); + assertThat(actual.snapshotId()) + .as("Snapshots to remove should be the same") + .isEqualTo(expected.snapshotId()); } private static void assertEqualsSetPartitionStatistics( MetadataUpdate.SetPartitionStatistics expected, MetadataUpdate.SetPartitionStatistics actual) { - Assert.assertEquals("Snapshot IDs should be equal", expected.snapshotId(), actual.snapshotId()); - Assert.assertEquals( - "Partition Statistics files snapshot IDs should be equal", - expected.partitionStatisticsFile().snapshotId(), - actual.partitionStatisticsFile().snapshotId()); - Assert.assertEquals( - "Partition statistics files paths should be equal", - expected.partitionStatisticsFile().path(), - actual.partitionStatisticsFile().path()); - Assert.assertEquals( - "Partition statistics file size should be equal", - expected.partitionStatisticsFile().fileSizeInBytes(), - actual.partitionStatisticsFile().fileSizeInBytes()); + assertThat(actual.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(actual.partitionStatisticsFile().snapshotId()) + .isEqualTo(expected.partitionStatisticsFile().snapshotId()); + assertThat(actual.partitionStatisticsFile().path()) + .isEqualTo(expected.partitionStatisticsFile().path()); + assertThat(actual.partitionStatisticsFile().fileSizeInBytes()) + .isEqualTo(expected.partitionStatisticsFile().fileSizeInBytes()); } private static void assertEqualsRemovePartitionStatistics( MetadataUpdate.RemovePartitionStatistics expected, MetadataUpdate.RemovePartitionStatistics actual) { - Assert.assertEquals( - "Snapshots to remove should be the same", expected.snapshotId(), actual.snapshotId()); + assertThat(actual.snapshotId()).isEqualTo(expected.snapshotId()); } private static void assertEqualsAddSnapshot( MetadataUpdate.AddSnapshot expected, MetadataUpdate.AddSnapshot actual) { - Assert.assertEquals( - "Snapshot ID should be equal", - expected.snapshot().snapshotId(), - actual.snapshot().snapshotId()); - Assert.assertEquals( - "Manifest list location should be equal", - expected.snapshot().manifestListLocation(), - actual.snapshot().manifestListLocation()); - Assertions.assertThat(actual.snapshot().summary()) + assertThat(actual.snapshot().snapshotId()).isEqualTo(expected.snapshot().snapshotId()); + assertThat(actual.snapshot().manifestListLocation()) + .isEqualTo(expected.snapshot().manifestListLocation()); + assertThat(actual.snapshot().summary()) .as("Snapshot summary should be equivalent") .containsExactlyEntriesOf(expected.snapshot().summary()); - Assert.assertEquals( - "Snapshot Parent ID should be equal", - expected.snapshot().parentId(), - actual.snapshot().parentId()); - Assert.assertEquals( - "Snapshot timestamp should be equal", - expected.snapshot().timestampMillis(), - actual.snapshot().timestampMillis()); - Assert.assertEquals( - "Snapshot schema id should be equal", - expected.snapshot().schemaId(), - actual.snapshot().schemaId()); + assertThat(actual.snapshot().parentId()).isEqualTo(expected.snapshot().parentId()); + assertThat(actual.snapshot().timestampMillis()) + .isEqualTo(expected.snapshot().timestampMillis()); + assertThat(actual.snapshot().schemaId()).isEqualTo(expected.snapshot().schemaId()); } private static void assertEqualsRemoveSnapshots( MetadataUpdate.RemoveSnapshot expected, MetadataUpdate.RemoveSnapshot actual) { - Assert.assertEquals( - "Snapshots to remove should be the same", expected.snapshotId(), actual.snapshotId()); + assertThat(actual.snapshotId()) + .as("Snapshots to remove should be the same") + .isEqualTo(expected.snapshotId()); } private static void assertEqualsSetSnapshotRef( MetadataUpdate.SetSnapshotRef expected, MetadataUpdate.SetSnapshotRef actual) { // Non-null fields - Assert.assertNotNull("Snapshot ref name should not be null", actual.name()); - Assert.assertEquals("Snapshot ref name should be equal", expected.name(), actual.name()); - Assert.assertEquals("Snapshot ID should be equal", expected.snapshotId(), actual.snapshotId()); - Assert.assertNotNull("Snapshot reference type should not be null", actual.type()); - Assert.assertEquals("Snapshot reference type should be equal", expected.type(), actual.type()); + assertThat(actual.name()).isNotNull().isEqualTo(expected.name()); + assertThat(actual.snapshotId()).isEqualTo(expected.snapshotId()); + assertThat(actual.type()).isNotNull().isEqualTo(expected.type()); // Nullable fields - Assert.assertEquals( - "Min snapshots to keep should be equal when present and null when missing or explicitly null", - expected.minSnapshotsToKeep(), - actual.minSnapshotsToKeep()); - Assert.assertEquals( - "Max snapshot age ms should be equal when present and null when missing or explicitly null", - expected.maxSnapshotAgeMs(), - actual.maxSnapshotAgeMs()); - Assert.assertEquals( - "Max ref age ms should be equal when present and null when missing or explicitly null", - expected.maxRefAgeMs(), - actual.maxRefAgeMs()); + assertThat(actual.minSnapshotsToKeep()) + .as( + "Min snapshots to keep should be equal when present and null when missing or explicitly null") + .isEqualTo(expected.minSnapshotsToKeep()); + assertThat(actual.maxSnapshotAgeMs()) + .as( + "Max snapshot age ms should be equal when present and null when missing or explicitly null") + .isEqualTo(expected.maxSnapshotAgeMs()); + assertThat(actual.maxRefAgeMs()) + .as("Max ref age ms should be equal when present and null when missing or explicitly null") + .isEqualTo(expected.maxRefAgeMs()); } private static void assertEqualsRemoveSnapshotRef( MetadataUpdate.RemoveSnapshotRef expected, MetadataUpdate.RemoveSnapshotRef actual) { - Assertions.assertThat(actual.name()).isEqualTo(expected.name()); + assertThat(actual.name()).isEqualTo(expected.name()); } private static void assertEqualsSetProperties( MetadataUpdate.SetProperties expected, MetadataUpdate.SetProperties actual) { - Assertions.assertThat(actual.updated()) + assertThat(actual.updated()) .as("Properties to set / update should not be null") .isNotNull() .as("Properties to set / update should be the same") @@ -1283,7 +1229,7 @@ private static void assertEqualsSetProperties( private static void assertEqualsRemoveProperties( MetadataUpdate.RemoveProperties expected, MetadataUpdate.RemoveProperties actual) { - Assertions.assertThat(actual.removed()) + assertThat(actual.removed()) .as("Properties to remove should not be null") .isNotNull() .as("Properties to remove should be equal") @@ -1292,22 +1238,22 @@ private static void assertEqualsRemoveProperties( private static void assertEqualsSetLocation( MetadataUpdate.SetLocation expected, MetadataUpdate.SetLocation actual) { - Assert.assertEquals("Location should be the same", expected.location(), actual.location()); + assertThat(actual.location()).isEqualTo(expected.location()); } private static void assertEqualsAddViewVersion( MetadataUpdate.AddViewVersion expected, MetadataUpdate.AddViewVersion actual) { - Assertions.assertThat(actual.viewVersion()).isEqualTo(expected.viewVersion()); + assertThat(actual.viewVersion()).isEqualTo(expected.viewVersion()); } private static void assertEqualsSetCurrentViewVersion( MetadataUpdate.SetCurrentViewVersion expected, MetadataUpdate.SetCurrentViewVersion actual) { - Assertions.assertThat(actual.versionId()).isEqualTo(expected.versionId()); + assertThat(actual.versionId()).isEqualTo(expected.versionId()); } private String createManifestListWithManifestFiles(long snapshotId, Long parentSnapshotId) throws IOException { - File manifestList = temp.newFile("manifests" + UUID.randomUUID()); + File manifestList = File.createTempFile("manifests", null, temp.toFile()); manifestList.deleteOnExit(); List manifests = diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 1d7d754d8df2..8075372d09c1 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -30,6 +30,8 @@ import static org.apache.iceberg.TableMetadataParser.SNAPSHOTS; import static org.apache.iceberg.TestHelpers.assertSameSchemaList; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.entry; import com.fasterxml.jackson.core.JsonGenerator; import java.io.File; @@ -59,8 +61,6 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -183,72 +183,34 @@ public void testJsonConversion() throws Exception { String asJson = TableMetadataParser.toJson(expected); TableMetadata metadata = TableMetadataParser.fromJson(asJson); - Assert.assertEquals( - "Format version should match", expected.formatVersion(), metadata.formatVersion()); - Assert.assertEquals("Table UUID should match", expected.uuid(), metadata.uuid()); - Assert.assertEquals("Table location should match", expected.location(), metadata.location()); - Assert.assertEquals( - "Last sequence number should match", - expected.lastSequenceNumber(), - metadata.lastSequenceNumber()); - Assert.assertEquals( - "Last column ID should match", expected.lastColumnId(), metadata.lastColumnId()); - Assert.assertEquals( - "Current schema id should match", expected.currentSchemaId(), metadata.currentSchemaId()); + assertThat(metadata.formatVersion()).isEqualTo(expected.formatVersion()); + assertThat(metadata.uuid()).isEqualTo(expected.uuid()); + assertThat(metadata.location()).isEqualTo(expected.location()); + assertThat(metadata.lastSequenceNumber()).isEqualTo(expected.lastSequenceNumber()); + assertThat(metadata.lastColumnId()).isEqualTo(expected.lastColumnId()); + assertThat(metadata.currentSchemaId()).isEqualTo(expected.currentSchemaId()); assertSameSchemaList(expected.schemas(), metadata.schemas()); - Assert.assertEquals( - "Partition spec should match", expected.spec().toString(), metadata.spec().toString()); - Assert.assertEquals( - "Default spec ID should match", expected.defaultSpecId(), metadata.defaultSpecId()); - Assert.assertEquals("PartitionSpec map should match", expected.specs(), metadata.specs()); - Assert.assertEquals( - "lastAssignedFieldId across all PartitionSpecs should match", - expected.spec().lastAssignedFieldId(), - metadata.lastAssignedPartitionId()); - Assert.assertEquals( - "Default sort ID should match", - expected.defaultSortOrderId(), - metadata.defaultSortOrderId()); - Assert.assertEquals("Sort order should match", expected.sortOrder(), metadata.sortOrder()); - Assert.assertEquals( - "Sort order map should match", expected.sortOrders(), metadata.sortOrders()); - Assert.assertEquals("Properties should match", expected.properties(), metadata.properties()); - Assert.assertEquals( - "Snapshot logs should match", expected.snapshotLog(), metadata.snapshotLog()); - Assert.assertEquals( - "Current snapshot ID should match", - currentSnapshotId, - metadata.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Parent snapshot ID should match", - (Long) previousSnapshotId, - metadata.currentSnapshot().parentId()); - Assert.assertEquals( - "Current snapshot files should match", - currentSnapshot.allManifests(ops.io()), - metadata.currentSnapshot().allManifests(ops.io())); - Assert.assertEquals( - "Schema ID for current snapshot should match", - (Integer) 7, - metadata.currentSnapshot().schemaId()); - Assert.assertEquals( - "Previous snapshot ID should match", - previousSnapshotId, - metadata.snapshot(previousSnapshotId).snapshotId()); - Assert.assertEquals( - "Previous snapshot files should match", - previousSnapshot.allManifests(ops.io()), - metadata.snapshot(previousSnapshotId).allManifests(ops.io())); - Assert.assertNull( - "Previous snapshot's schema ID should be null", - metadata.snapshot(previousSnapshotId).schemaId()); - Assert.assertEquals( - "Statistics files should match", statisticsFiles, metadata.statisticsFiles()); - Assert.assertEquals( - "Partition statistics files should match", - partitionStatisticsFiles, - metadata.partitionStatisticsFiles()); - Assert.assertEquals("Refs map should match", refs, metadata.refs()); + assertThat(metadata.spec().toString()).isEqualTo(expected.spec().toString()); + assertThat(metadata.defaultSpecId()).isEqualTo(expected.defaultSpecId()); + assertThat(metadata.specs()).isEqualTo(expected.specs()); + assertThat(metadata.lastAssignedPartitionId()).isEqualTo(expected.spec().lastAssignedFieldId()); + assertThat(metadata.defaultSortOrderId()).isEqualTo(expected.defaultSortOrderId()); + assertThat(metadata.sortOrder()).isEqualTo(expected.sortOrder()); + assertThat(metadata.sortOrders()).isEqualTo(expected.sortOrders()); + assertThat(metadata.properties()).isEqualTo(expected.properties()); + assertThat(metadata.snapshotLog()).isEqualTo(expected.snapshotLog()); + assertThat(metadata.currentSnapshot().snapshotId()).isEqualTo(currentSnapshotId); + assertThat(metadata.currentSnapshot().parentId()).isEqualTo(previousSnapshotId); + assertThat(metadata.currentSnapshot().allManifests(ops.io())) + .isEqualTo(currentSnapshot.allManifests(ops.io())); + assertThat(metadata.currentSnapshot().schemaId()).isEqualTo(7); + assertThat(metadata.snapshot(previousSnapshotId).snapshotId()).isEqualTo(previousSnapshotId); + assertThat(metadata.snapshot(previousSnapshotId).allManifests(ops.io())) + .isEqualTo(previousSnapshot.allManifests(ops.io())); + assertThat(metadata.snapshot(previousSnapshotId).schemaId()).isNull(); + assertThat(metadata.statisticsFiles()).isEqualTo(statisticsFiles); + assertThat(metadata.partitionStatisticsFiles()).isEqualTo(partitionStatisticsFiles); + assertThat(metadata.refs()).isEqualTo(refs); } @Test @@ -310,74 +272,40 @@ public void testBackwardCompat() throws Exception { String asJson = toJsonWithoutSpecAndSchemaList(expected); TableMetadata metadata = TableMetadataParser.fromJson(asJson); - Assert.assertEquals( - "Format version should match", expected.formatVersion(), metadata.formatVersion()); - Assert.assertNull("Table UUID should not be assigned", metadata.uuid()); - Assert.assertEquals("Table location should match", expected.location(), metadata.location()); - Assert.assertEquals( - "Last sequence number should default to 0", - expected.lastSequenceNumber(), - metadata.lastSequenceNumber()); - Assert.assertEquals( - "Last column ID should match", expected.lastColumnId(), metadata.lastColumnId()); - Assert.assertEquals( - "Current schema ID should be default to TableMetadata.INITIAL_SCHEMA_ID", - TableMetadata.INITIAL_SCHEMA_ID, - metadata.currentSchemaId()); - Assert.assertEquals("Schemas size should match", 1, metadata.schemas().size()); - Assert.assertEquals( - "Schemas should contain the schema", - metadata.schemas().get(0).asStruct(), - schema.asStruct()); - Assert.assertEquals( - "Partition spec should be the default", - expected.spec().toString(), - metadata.spec().toString()); - Assert.assertEquals( - "Default spec ID should default to TableMetadata.INITIAL_SPEC_ID", - TableMetadata.INITIAL_SPEC_ID, - metadata.defaultSpecId()); - Assert.assertEquals("PartitionSpec should contain the spec", 1, metadata.specs().size()); - Assert.assertTrue( - "PartitionSpec should contain the spec", metadata.specs().get(0).compatibleWith(spec)); - Assert.assertEquals( - "PartitionSpec should have ID TableMetadata.INITIAL_SPEC_ID", - TableMetadata.INITIAL_SPEC_ID, - metadata.specs().get(0).specId()); - Assert.assertEquals( - "lastAssignedFieldId across all PartitionSpecs should match", - expected.spec().lastAssignedFieldId(), - metadata.lastAssignedPartitionId()); - Assert.assertEquals("Properties should match", expected.properties(), metadata.properties()); - Assert.assertEquals( - "Snapshot logs should match", expected.snapshotLog(), metadata.snapshotLog()); - Assert.assertEquals( - "Current snapshot ID should match", - currentSnapshotId, - metadata.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Parent snapshot ID should match", - (Long) previousSnapshotId, - metadata.currentSnapshot().parentId()); - Assert.assertEquals( - "Current snapshot files should match", - currentSnapshot.allManifests(ops.io()), - metadata.currentSnapshot().allManifests(ops.io())); - Assert.assertNull( - "Current snapshot's schema ID should be null", metadata.currentSnapshot().schemaId()); - Assert.assertEquals( - "Previous snapshot ID should match", - previousSnapshotId, - metadata.snapshot(previousSnapshotId).snapshotId()); - Assert.assertEquals( - "Previous snapshot files should match", - previousSnapshot.allManifests(ops.io()), - metadata.snapshot(previousSnapshotId).allManifests(ops.io())); - Assert.assertEquals( - "Snapshot logs should match", expected.previousFiles(), metadata.previousFiles()); - Assert.assertNull( - "Previous snapshot's schema ID should be null", - metadata.snapshot(previousSnapshotId).schemaId()); + assertThat(metadata.formatVersion()).isEqualTo(expected.formatVersion()); + assertThat(metadata.uuid()).as("Table UUID should not be assigned").isNull(); + assertThat(metadata.location()).isEqualTo(expected.location()); + assertThat(metadata.lastSequenceNumber()) + .as("Last sequence number should default to 0") + .isEqualTo(expected.lastSequenceNumber()); + + assertThat(metadata.lastColumnId()).isEqualTo(expected.lastColumnId()); + assertThat(metadata.currentSchemaId()) + .as("Current schema ID should be default to TableMetadata.INITIAL_SCHEMA_ID") + .isEqualTo(TableMetadata.INITIAL_SCHEMA_ID); + assertThat(metadata.schemas()).hasSize(1); + assertThat(metadata.schemas().get(0).asStruct()).isEqualTo(schema.asStruct()); + assertThat(metadata.spec().toString()).isEqualTo(expected.spec().toString()); + assertThat(metadata.defaultSpecId()).isEqualTo(TableMetadata.INITIAL_SPEC_ID); + assertThat(metadata.specs()).hasSize(1); + assertThat(metadata.specs()) + .first() + .satisfies(partitionSpec -> partitionSpec.compatibleWith(spec)); + assertThat(metadata.specs().get(0).specId()).isEqualTo(TableMetadata.INITIAL_SPEC_ID); + assertThat(metadata.lastAssignedPartitionId()).isEqualTo(expected.spec().lastAssignedFieldId()); + assertThat(metadata.properties()).isEqualTo(expected.properties()); + assertThat(metadata.snapshotLog()).isEqualTo(expected.snapshotLog()); + assertThat(metadata.currentSnapshot().snapshotId()).isEqualTo(currentSnapshotId); + assertThat(metadata.currentSnapshot().parentId()).isEqualTo(previousSnapshotId); + assertThat(metadata.currentSnapshot().allManifests(ops.io())) + .as("Current snapshot files should match") + .isEqualTo(currentSnapshot.allManifests(ops.io())); + assertThat(metadata.currentSnapshot().schemaId()).isNull(); + assertThat(metadata.snapshot(previousSnapshotId).snapshotId()).isEqualTo(previousSnapshotId); + assertThat(metadata.snapshot(previousSnapshotId).allManifests(ops.io())) + .isEqualTo(previousSnapshot.allManifests(ops.io())); + assertThat(metadata.previousFiles()).isEqualTo(expected.previousFiles()); + assertThat(metadata.snapshot(previousSnapshotId).schemaId()).isNull(); } @Test @@ -421,7 +349,7 @@ public void testInvalidMainBranch() throws IOException { Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(previousSnapshotId).build()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new TableMetadata( null, @@ -466,7 +394,7 @@ public void testMainWithoutCurrent() throws IOException { Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new TableMetadata( null, @@ -506,7 +434,7 @@ public void testBranchSnapshotMissing() { Map refs = ImmutableMap.of("main", SnapshotRef.branchBuilder(snapshotId).build()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new TableMetadata( null, @@ -641,8 +569,7 @@ public void testJsonWithPreviousMetadataLog() throws Exception { String asJson = TableMetadataParser.toJson(base); TableMetadata metadataFromJson = TableMetadataParser.fromJson(asJson); - Assert.assertEquals( - "Metadata logs should match", previousMetadataLog, metadataFromJson.previousFiles()); + assertThat(metadataFromJson.previousFiles()).isEqualTo(previousMetadataLog); } @Test @@ -726,9 +653,8 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { Set removedPreviousMetadata = Sets.newHashSet(base.previousFiles()); removedPreviousMetadata.removeAll(metadata.previousFiles()); - Assert.assertEquals( - "Metadata logs should match", previousMetadataLog, metadata.previousFiles()); - Assert.assertEquals("Removed Metadata logs should be empty", 0, removedPreviousMetadata.size()); + assertThat(metadata.previousFiles()).isEqualTo(previousMetadataLog); + assertThat(removedPreviousMetadata).isEmpty(); } @Test @@ -827,12 +753,9 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { removedPreviousMetadata.addAll(base.previousFiles()); removedPreviousMetadata.removeAll(metadata.previousFiles()); - Assert.assertEquals( - "Metadata logs should match", previousMetadataLog.subList(1, 6), metadata.previousFiles()); - Assert.assertEquals( - "Removed Metadata logs should contain 1", - previousMetadataLog.subList(0, 1), - ImmutableList.copyOf(removedPreviousMetadata)); + assertThat(metadata.previousFiles()).isEqualTo(previousMetadataLog.subList(1, 6)); + assertThat(ImmutableList.copyOf(removedPreviousMetadata)) + .isEqualTo(previousMetadataLog.subList(0, 1)); } @Test @@ -931,17 +854,14 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { removedPreviousMetadata.addAll(base.previousFiles()); removedPreviousMetadata.removeAll(metadata.previousFiles()); - Assert.assertEquals( - "Metadata logs should match", previousMetadataLog.subList(4, 6), metadata.previousFiles()); - Assert.assertEquals( - "Removed Metadata logs should contain 4", - previousMetadataLog.subList(0, 4), - ImmutableList.copyOf(removedPreviousMetadata)); + assertThat(metadata.previousFiles()).isEqualTo(previousMetadataLog.subList(4, 6)); + assertThat(ImmutableList.copyOf(removedPreviousMetadata)) + .isEqualTo(previousMetadataLog.subList(0, 4)); } @Test public void testV2UUIDValidation() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new TableMetadata( null, @@ -975,7 +895,7 @@ public void testV2UUIDValidation() { @Test public void testVersionValidation() { int unsupportedVersion = TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new TableMetadata( null, @@ -1010,14 +930,14 @@ public void testVersionValidation() { public void testParserVersionValidation() throws Exception { String supportedVersion1 = readTableMetadataInputFile("TableMetadataV1Valid.json"); TableMetadata parsed1 = TableMetadataParser.fromJson(supportedVersion1); - Assert.assertNotNull("Should successfully read supported metadata version", parsed1); + assertThat(parsed1).as("Should successfully read supported metadata version").isNotNull(); String supportedVersion2 = readTableMetadataInputFile("TableMetadataV2Valid.json"); TableMetadata parsed2 = TableMetadataParser.fromJson(supportedVersion2); - Assert.assertNotNull("Should successfully read supported metadata version", parsed2); + assertThat(parsed2).as("Should successfully read supported metadata version").isNotNull(); String unsupportedVersion = readTableMetadataInputFile("TableMetadataUnsupportedVersion.json"); - Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot read unsupported version"); } @@ -1026,7 +946,7 @@ public void testParserVersionValidation() throws Exception { public void testParserV2PartitionSpecsValidation() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingPartitionSpecs.json"); - Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("partition-specs must exist in format v2"); } @@ -1035,7 +955,7 @@ public void testParserV2PartitionSpecsValidation() throws Exception { public void testParserV2LastAssignedFieldIdValidation() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingLastPartitionId.json"); - Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("last-partition-id must exist in format v2"); } @@ -1043,7 +963,7 @@ public void testParserV2LastAssignedFieldIdValidation() throws Exception { @Test public void testParserV2SortOrderValidation() throws Exception { String unsupportedVersion = readTableMetadataInputFile("TableMetadataV2MissingSortOrder.json"); - Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) + assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupportedVersion)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("sort-orders must exist in format v2"); } @@ -1051,7 +971,7 @@ public void testParserV2SortOrderValidation() throws Exception { @Test public void testParserV2CurrentSchemaIdValidation() throws Exception { String unsupported = readTableMetadataInputFile("TableMetadataV2CurrentSchemaNotFound.json"); - Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) + assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find schema with current-schema-id=2 from schemas"); } @@ -1059,7 +979,7 @@ public void testParserV2CurrentSchemaIdValidation() throws Exception { @Test public void testParserV2SchemasValidation() throws Exception { String unsupported = readTableMetadataInputFile("TableMetadataV2MissingSchemas.json"); - Assertions.assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) + assertThatThrownBy(() -> TableMetadataParser.fromJson(unsupported)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("schemas must exist in format v2"); } @@ -1095,7 +1015,7 @@ public void testNewTableMetadataReassignmentAllIds() throws Exception { .add(3, 1001, "z_partition", Transforms.bucket(8)) .build(); - Assert.assertEquals(expected, metadata.spec()); + assertThat(metadata.spec()).isEqualTo(expected); } @Test @@ -1117,7 +1037,7 @@ public void testInvalidUpdatePartitionSpecForV1Table() throws Exception { ImmutableMap.of(), 1); - Assertions.assertThatThrownBy(() -> metadata.updatePartitionSpec(spec)) + assertThatThrownBy(() -> metadata.updatePartitionSpec(spec)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Spec does not use sequential IDs that are required in v1"); } @@ -1134,7 +1054,7 @@ public void testBuildReplacementForV1Table() { TableMetadata metadata = TableMetadata.newTableMetadata( schema, spec, SortOrder.unsorted(), location, ImmutableMap.of(), 1); - Assert.assertEquals(spec, metadata.spec()); + assertThat(metadata.spec()).isEqualTo(spec); Schema updatedSchema = new Schema( @@ -1153,10 +1073,10 @@ public void testBuildReplacementForV1Table() { .add(2, 1001, "y", Transforms.alwaysNull()) .add(3, 1002, "z_bucket", Transforms.bucket(8)) .build(); - Assert.assertEquals( - "Should reassign the partition field IDs and reuse any existing IDs for equivalent fields", - expected, - updated.spec()); + assertThat(updated.spec()) + .as( + "Should reassign the partition field IDs and reuse any existing IDs for equivalent fields") + .isEqualTo(expected); } @Test @@ -1171,7 +1091,7 @@ public void testBuildReplacementForV2Table() { TableMetadata metadata = TableMetadata.newTableMetadata( schema, spec, SortOrder.unsorted(), location, ImmutableMap.of(), 2); - Assert.assertEquals(spec, metadata.spec()); + assertThat(metadata.spec()).isEqualTo(spec); Schema updatedSchema = new Schema( @@ -1188,10 +1108,10 @@ public void testBuildReplacementForV2Table() { .add(3, 1002, "z_bucket", Transforms.bucket(8)) .add(1, 1000, "x", Transforms.identity()) .build(); - Assert.assertEquals( - "Should reassign the partition field IDs and reuse any existing IDs for equivalent fields", - expected, - updated.spec()); + assertThat(updated.spec()) + .as( + "Should reassign the partition field IDs and reuse any existing IDs for equivalent fields") + .isEqualTo(expected); } @Test @@ -1201,11 +1121,10 @@ public void testSortOrder() { TableMetadata meta = TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); - Assert.assertTrue("Should default to unsorted order", meta.sortOrder().isUnsorted()); - Assert.assertSame( - "Should detect identical unsorted order", - meta, - meta.replaceSortOrder(SortOrder.unsorted())); + assertThat(meta.sortOrder().isUnsorted()).isTrue(); + assertThat(meta.replaceSortOrder(SortOrder.unsorted())) + .as("Should detect identical unsorted order") + .isSameAs(meta); } @Test @@ -1217,49 +1136,34 @@ public void testUpdateSortOrder() { TableMetadata sortedByX = TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), order, null, ImmutableMap.of()); - Assert.assertEquals("Should have 1 sort order", 1, sortedByX.sortOrders().size()); - Assert.assertEquals("Should use orderId 1", 1, sortedByX.sortOrder().orderId()); - Assert.assertEquals("Should be sorted by one field", 1, sortedByX.sortOrder().fields().size()); - Assert.assertEquals( - "Should use the table's field ids", 1, sortedByX.sortOrder().fields().get(0).sourceId()); - Assert.assertEquals( - "Should be ascending", - SortDirection.ASC, - sortedByX.sortOrder().fields().get(0).direction()); - Assert.assertEquals( - "Should be nulls first", - NullOrder.NULLS_FIRST, - sortedByX.sortOrder().fields().get(0).nullOrder()); + assertThat(sortedByX.sortOrders()).hasSize(1); + assertThat(sortedByX.sortOrder().orderId()).isEqualTo(1); + assertThat(sortedByX.sortOrder().fields()).hasSize(1); + assertThat(sortedByX.sortOrder().fields().get(0).sourceId()).isEqualTo(1); + assertThat(sortedByX.sortOrder().fields().get(0).direction()).isEqualTo(SortDirection.ASC); + assertThat(sortedByX.sortOrder().fields().get(0).nullOrder()).isEqualTo(NullOrder.NULLS_FIRST); // build an equivalent order with the correct schema SortOrder newOrder = SortOrder.builderFor(sortedByX.schema()).asc("x").build(); TableMetadata alsoSortedByX = sortedByX.replaceSortOrder(newOrder); - Assert.assertSame("Should detect current sortOrder and not update", alsoSortedByX, sortedByX); + assertThat(sortedByX) + .as("Should detect current sortOrder and not update") + .isSameAs(alsoSortedByX); TableMetadata unsorted = alsoSortedByX.replaceSortOrder(SortOrder.unsorted()); - Assert.assertEquals("Should have 2 sort orders", 2, unsorted.sortOrders().size()); - Assert.assertEquals("Should use orderId 0", 0, unsorted.sortOrder().orderId()); - Assert.assertTrue("Should be unsorted", unsorted.sortOrder().isUnsorted()); + assertThat(unsorted.sortOrders()).hasSize(2); + assertThat(unsorted.sortOrder().orderId()).isEqualTo(0); + assertThat(unsorted.sortOrder().isUnsorted()).isTrue(); TableMetadata sortedByXDesc = unsorted.replaceSortOrder(SortOrder.builderFor(unsorted.schema()).desc("x").build()); - Assert.assertEquals("Should have 3 sort orders", 3, sortedByXDesc.sortOrders().size()); - Assert.assertEquals("Should use orderId 2", 2, sortedByXDesc.sortOrder().orderId()); - Assert.assertEquals( - "Should be sorted by one field", 1, sortedByXDesc.sortOrder().fields().size()); - Assert.assertEquals( - "Should use the table's field ids", - 1, - sortedByXDesc.sortOrder().fields().get(0).sourceId()); - Assert.assertEquals( - "Should be ascending", - SortDirection.DESC, - sortedByXDesc.sortOrder().fields().get(0).direction()); - Assert.assertEquals( - "Should be nulls first", - NullOrder.NULLS_FIRST, - sortedByX.sortOrder().fields().get(0).nullOrder()); + assertThat(sortedByXDesc.sortOrders()).hasSize(3); + assertThat(sortedByXDesc.sortOrder().orderId()).isEqualTo(2); + assertThat(sortedByXDesc.sortOrder().fields()).hasSize(1); + assertThat(sortedByXDesc.sortOrder().fields().get(0).sourceId()).isEqualTo(1); + assertThat(sortedByXDesc.sortOrder().fields().get(0).direction()).isEqualTo(SortDirection.DESC); + assertThat(sortedByX.sortOrder().fields().get(0).nullOrder()).isEqualTo(NullOrder.NULLS_FIRST); } @Test @@ -1269,8 +1173,7 @@ public void testStatistics() { TableMetadata meta = TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); - Assert.assertEquals( - "Should default to no statistics files", ImmutableList.of(), meta.statisticsFiles()); + assertThat(meta.statisticsFiles()).as("Should default to no statistics files").isEmpty(); } @Test @@ -1289,12 +1192,12 @@ public void testSetStatistics() { 43, "/some/path/to/stats/file", 128, 27, ImmutableList.of())) .build(); - Assertions.assertThat(withStatistics.statisticsFiles()) + assertThat(withStatistics.statisticsFiles()) .as("There should be one statistics file registered") .hasSize(1); StatisticsFile statisticsFile = Iterables.getOnlyElement(withStatistics.statisticsFiles()); - Assert.assertEquals("Statistics file snapshot", 43L, statisticsFile.snapshotId()); - Assert.assertEquals("Statistics file path", "/some/path/to/stats/file", statisticsFile.path()); + assertThat(statisticsFile.snapshotId()).isEqualTo(43L); + assertThat(statisticsFile.path()).isEqualTo("/some/path/to/stats/file"); TableMetadata withStatisticsReplaced = TableMetadata.buildFrom(withStatistics) @@ -1304,12 +1207,12 @@ public void testSetStatistics() { 43, "/some/path/to/stats/file2", 128, 27, ImmutableList.of())) .build(); - Assertions.assertThat(withStatisticsReplaced.statisticsFiles()) + assertThat(withStatisticsReplaced.statisticsFiles()) .as("There should be one statistics file registered") .hasSize(1); statisticsFile = Iterables.getOnlyElement(withStatisticsReplaced.statisticsFiles()); - Assert.assertEquals("Statistics file snapshot", 43L, statisticsFile.snapshotId()); - Assert.assertEquals("Statistics file path", "/some/path/to/stats/file2", statisticsFile.path()); + assertThat(statisticsFile.snapshotId()).isEqualTo(43L); + assertThat(statisticsFile.path()).isEqualTo("/some/path/to/stats/file2"); } @Test @@ -1330,19 +1233,18 @@ public void testRemoveStatistics() { 44, "/some/path/to/stats/file2", 128, 27, ImmutableList.of())) .build(); - Assert.assertSame( - "Should detect no statistics to remove", - meta, - TableMetadata.buildFrom(meta).removeStatistics(42L).build()); + assertThat(TableMetadata.buildFrom(meta).removeStatistics(42L).build()) + .as("Should detect no statistics to remove") + .isSameAs(meta); TableMetadata withOneRemoved = TableMetadata.buildFrom(meta).removeStatistics(43).build(); - Assertions.assertThat(withOneRemoved.statisticsFiles()) + assertThat(withOneRemoved.statisticsFiles()) .as("There should be one statistics file retained") .hasSize(1); StatisticsFile statisticsFile = Iterables.getOnlyElement(withOneRemoved.statisticsFiles()); - Assert.assertEquals("Statistics file snapshot", 44L, statisticsFile.snapshotId()); - Assert.assertEquals("Statistics file path", "/some/path/to/stats/file2", statisticsFile.path()); + assertThat(statisticsFile.snapshotId()).isEqualTo(44L); + assertThat(statisticsFile.path()).isEqualTo("/some/path/to/stats/file2"); } @Test @@ -1352,10 +1254,9 @@ public void testPartitionStatistics() { TableMetadata meta = TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); - Assert.assertEquals( - "Should default to no partition statistics files", - ImmutableList.of(), - meta.partitionStatisticsFiles()); + assertThat(meta.partitionStatisticsFiles()) + .as("Should default to no partition statistics files") + .isEmpty(); } @Test @@ -1376,18 +1277,15 @@ public void testSetPartitionStatistics() { .build()) .build(); - Assertions.assertThat(withPartitionStatistics.partitionStatisticsFiles()) + assertThat(withPartitionStatistics.partitionStatisticsFiles()) .as("There should be one partition statistics file registered") .hasSize(1); PartitionStatisticsFile partitionStatisticsFile = Iterables.getOnlyElement(withPartitionStatistics.partitionStatisticsFiles()); - Assert.assertEquals("Statistics file snapshot", 43L, partitionStatisticsFile.snapshotId()); - Assert.assertEquals( - "Statistics file path", - "/some/path/to/partition/stats/file.parquet", - partitionStatisticsFile.path()); - Assert.assertEquals( - "Statistics file size in bytes", 42L, partitionStatisticsFile.fileSizeInBytes()); + assertThat(partitionStatisticsFile.snapshotId()).isEqualTo(43L); + assertThat(partitionStatisticsFile.path()) + .isEqualTo("/some/path/to/partition/stats/file.parquet"); + assertThat(partitionStatisticsFile.fileSizeInBytes()).isEqualTo(42L); TableMetadata withStatisticsReplaced = TableMetadata.buildFrom(withPartitionStatistics) @@ -1399,18 +1297,15 @@ public void testSetPartitionStatistics() { .build()) .build(); - Assertions.assertThat(withStatisticsReplaced.partitionStatisticsFiles()) + assertThat(withStatisticsReplaced.partitionStatisticsFiles()) .as("There should be one statistics file registered") .hasSize(1); partitionStatisticsFile = Iterables.getOnlyElement(withStatisticsReplaced.partitionStatisticsFiles()); - Assert.assertEquals("Statistics file snapshot", 43L, partitionStatisticsFile.snapshotId()); - Assert.assertEquals( - "Statistics file path", - "/some/path/to/partition/stats/file2.parquet", - partitionStatisticsFile.path()); - Assert.assertEquals( - "Statistics file size in bytes", 48L, partitionStatisticsFile.fileSizeInBytes()); + assertThat(partitionStatisticsFile.snapshotId()).isEqualTo(43L); + assertThat(partitionStatisticsFile.path()) + .isEqualTo("/some/path/to/partition/stats/file2.parquet"); + assertThat(partitionStatisticsFile.fileSizeInBytes()).isEqualTo(48L); } @Test @@ -1435,44 +1330,40 @@ public void testRemovePartitionStatistics() { .build()) .build(); - Assert.assertSame( - "Should detect no partition statistics to remove", - meta, - TableMetadata.buildFrom(meta).removePartitionStatistics(42L).build()); + assertThat(TableMetadata.buildFrom(meta).removePartitionStatistics(42L).build()) + .as("Should detect no partition statistics to remove") + .isSameAs(meta); TableMetadata withOneRemoved = TableMetadata.buildFrom(meta).removePartitionStatistics(43).build(); - Assertions.assertThat(withOneRemoved.partitionStatisticsFiles()) + assertThat(withOneRemoved.partitionStatisticsFiles()) .as("There should be one partition statistics file retained") .hasSize(1); PartitionStatisticsFile partitionStatisticsFile = Iterables.getOnlyElement(withOneRemoved.partitionStatisticsFiles()); - Assert.assertEquals("Statistics file snapshot", 44L, partitionStatisticsFile.snapshotId()); - Assert.assertEquals( - "Statistics file path", - "/some/path/to/partition/stats/file2.parquet", - partitionStatisticsFile.path()); - Assert.assertEquals( - "Statistics file size in bytes", 49L, partitionStatisticsFile.fileSizeInBytes()); + assertThat(partitionStatisticsFile.snapshotId()).isEqualTo(44L); + assertThat(partitionStatisticsFile.path()) + .isEqualTo("/some/path/to/partition/stats/file2.parquet"); + assertThat(partitionStatisticsFile.fileSizeInBytes()).isEqualTo(49L); } @Test public void testParseSchemaIdentifierFields() throws Exception { String data = readTableMetadataInputFile("TableMetadataV2Valid.json"); TableMetadata parsed = TableMetadataParser.fromJson(data); - Assert.assertEquals(Sets.newHashSet(), parsed.schemasById().get(0).identifierFieldIds()); - Assert.assertEquals(Sets.newHashSet(1, 2), parsed.schemasById().get(1).identifierFieldIds()); + assertThat(parsed.schemasById().get(0).identifierFieldIds()).isEmpty(); + assertThat(parsed.schemasById().get(1).identifierFieldIds()).containsExactly(1, 2); } @Test public void testParseMinimal() throws Exception { String data = readTableMetadataInputFile("TableMetadataV2ValidMinimal.json"); TableMetadata parsed = TableMetadataParser.fromJson(data); - Assertions.assertThat(parsed.snapshots()).isEmpty(); - Assertions.assertThat(parsed.snapshotLog()).isEmpty(); - Assertions.assertThat(parsed.properties()).isEmpty(); - Assertions.assertThat(parsed.previousFiles()).isEmpty(); + assertThat(parsed.snapshots()).isEmpty(); + assertThat(parsed.snapshotLog()).isEmpty(); + assertThat(parsed.properties()).isEmpty(); + assertThat(parsed.previousFiles()).isEmpty(); } @Test @@ -1488,8 +1379,8 @@ public void testUpdateSchemaIdentifierFields() { Lists.newArrayList(Types.NestedField.required(1, "x", Types.StringType.get())), Sets.newHashSet(1)); TableMetadata newMeta = meta.updateSchema(newSchema, 1); - Assert.assertEquals(2, newMeta.schemas().size()); - Assert.assertEquals(Sets.newHashSet(1), newMeta.schema().identifierFieldIds()); + assertThat(newMeta.schemas()).hasSize(2); + assertThat(newMeta.schema().identifierFieldIds()).containsExactly(1); } @Test @@ -1499,16 +1390,10 @@ public void testUpdateSchema() { TableMetadata freshTable = TableMetadata.newTableMetadata( schema, PartitionSpec.unpartitioned(), null, ImmutableMap.of()); - Assert.assertEquals( - "Should use TableMetadata.INITIAL_SCHEMA_ID for current schema id", - TableMetadata.INITIAL_SCHEMA_ID, - freshTable.currentSchemaId()); + assertThat(freshTable.currentSchemaId()).isEqualTo(TableMetadata.INITIAL_SCHEMA_ID); assertSameSchemaList(ImmutableList.of(schema), freshTable.schemas()); - Assert.assertEquals( - "Should have expected schema upon return", - schema.asStruct(), - freshTable.schema().asStruct()); - Assert.assertEquals("Should return expected last column id", 1, freshTable.lastColumnId()); + assertThat(freshTable.schema().asStruct()).isEqualTo(schema.asStruct()); + assertThat(freshTable.lastColumnId()).isEqualTo(1); // update schema Schema schema2 = @@ -1516,14 +1401,11 @@ public void testUpdateSchema() { Types.NestedField.required(1, "y", Types.LongType.get(), "comment"), Types.NestedField.required(2, "x", Types.StringType.get())); TableMetadata twoSchemasTable = freshTable.updateSchema(schema2, 2); - Assert.assertEquals("Should have current schema id as 1", 1, twoSchemasTable.currentSchemaId()); + assertThat(twoSchemasTable.currentSchemaId()).isEqualTo(1); assertSameSchemaList( ImmutableList.of(schema, new Schema(1, schema2.columns())), twoSchemasTable.schemas()); - Assert.assertEquals( - "Should have expected schema upon return", - schema2.asStruct(), - twoSchemasTable.schema().asStruct()); - Assert.assertEquals("Should return expected last column id", 2, twoSchemasTable.lastColumnId()); + assertThat(twoSchemasTable.schema().asStruct()).isEqualTo(schema2.asStruct()); + assertThat(twoSchemasTable.lastColumnId()).isEqualTo(2); // update schema with the same schema and last column ID as current shouldn't cause change Schema sameSchema2 = @@ -1531,35 +1413,25 @@ public void testUpdateSchema() { Types.NestedField.required(1, "y", Types.LongType.get(), "comment"), Types.NestedField.required(2, "x", Types.StringType.get())); TableMetadata sameSchemaTable = twoSchemasTable.updateSchema(sameSchema2, 2); - Assert.assertSame("Should return same table metadata", twoSchemasTable, sameSchemaTable); + 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); - Assert.assertEquals( - "Should have current schema id as 1", 1, differentColumnIdTable.currentSchemaId()); + assertThat(differentColumnIdTable.currentSchemaId()).isEqualTo(1); assertSameSchemaList( ImmutableList.of(schema, new Schema(1, schema2.columns())), differentColumnIdTable.schemas()); - Assert.assertEquals( - "Should have expected schema upon return", - schema2.asStruct(), - differentColumnIdTable.schema().asStruct()); - Assert.assertEquals( - "Should return expected last column id", 3, differentColumnIdTable.lastColumnId()); + assertThat(differentColumnIdTable.schema().asStruct()).isEqualTo(schema2.asStruct()); + assertThat(differentColumnIdTable.lastColumnId()).isEqualTo(3); // update schema with old schema does not change schemas TableMetadata revertSchemaTable = differentColumnIdTable.updateSchema(schema, 3); - Assert.assertEquals( - "Should have current schema id as 0", 0, revertSchemaTable.currentSchemaId()); + assertThat(revertSchemaTable.currentSchemaId()).isEqualTo(0); assertSameSchemaList( ImmutableList.of(schema, new Schema(1, schema2.columns())), revertSchemaTable.schemas()); - Assert.assertEquals( - "Should have expected schema upon return", - schema.asStruct(), - revertSchemaTable.schema().asStruct()); - Assert.assertEquals( - "Should return expected last column id", 3, revertSchemaTable.lastColumnId()); + assertThat(revertSchemaTable.schema().asStruct()).isEqualTo(schema.asStruct()); + assertThat(revertSchemaTable.lastColumnId()).isEqualTo(3); // create new schema will use the largest schema id + 1 Schema schema3 = @@ -1568,18 +1440,13 @@ public void testUpdateSchema() { Types.NestedField.required(4, "x", Types.StringType.get()), Types.NestedField.required(6, "z", Types.IntegerType.get())); TableMetadata threeSchemaTable = revertSchemaTable.updateSchema(schema3, 6); - Assert.assertEquals( - "Should have current schema id as 2", 2, threeSchemaTable.currentSchemaId()); + assertThat(threeSchemaTable.currentSchemaId()).isEqualTo(2); assertSameSchemaList( ImmutableList.of( schema, new Schema(1, schema2.columns()), new Schema(2, schema3.columns())), threeSchemaTable.schemas()); - Assert.assertEquals( - "Should have expected schema upon return", - schema3.asStruct(), - threeSchemaTable.schema().asStruct()); - Assert.assertEquals( - "Should return expected last column id", 6, threeSchemaTable.lastColumnId()); + assertThat(threeSchemaTable.schema().asStruct()).isEqualTo(schema3.asStruct()); + assertThat(threeSchemaTable.lastColumnId()).isEqualTo(6); } @Test @@ -1640,56 +1507,53 @@ public void testUpgradeV1MetadataToV2ThroughTableProperty() { meta.replaceProperties( ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key2", "val2")); - Assert.assertEquals( - "format version should be configured based on the format-version key", - 2, - meta.formatVersion()); - Assert.assertEquals( - "should not contain format-version but should contain new properties", - ImmutableMap.of("key2", "val2"), - meta.properties()); + assertThat(meta.formatVersion()) + .as("format version should be configured based on the format-version key") + .isEqualTo(2); + assertThat(meta.properties()) + .as("should not contain format-version but should contain new properties") + .containsExactly(entry("key2", "val2")); } @Test public void testParseStatisticsFiles() throws Exception { String data = readTableMetadataInputFile("TableMetadataStatisticsFiles.json"); TableMetadata parsed = TableMetadataParser.fromJson(data); - Assertions.assertThat(parsed.statisticsFiles()).as("parsed statistics files").hasSize(1); - Assert.assertEquals( - "parsed statistics file", - new GenericStatisticsFile( - 3055729675574597004L, - "s3://a/b/stats.puffin", - 413, - 42, - ImmutableList.of( - new GenericBlobMetadata( - "ndv", 3055729675574597004L, 1, ImmutableList.of(1), ImmutableMap.of()))), - Iterables.getOnlyElement(parsed.statisticsFiles())); + assertThat(parsed.statisticsFiles()).hasSize(1); + assertThat(parsed.statisticsFiles()) + .hasSize(1) + .first() + .isEqualTo( + new GenericStatisticsFile( + 3055729675574597004L, + "s3://a/b/stats.puffin", + 413, + 42, + ImmutableList.of( + new GenericBlobMetadata( + "ndv", 3055729675574597004L, 1, ImmutableList.of(1), ImmutableMap.of())))); } @Test public void testParsePartitionStatisticsFiles() throws Exception { String data = readTableMetadataInputFile("TableMetadataPartitionStatisticsFiles.json"); TableMetadata parsed = TableMetadataParser.fromJson(data); - Assertions.assertThat(parsed.partitionStatisticsFiles()) - .as("parsed partition statistics files") - .hasSize(1); - Assert.assertEquals( - "parsed partition statistics file", - ImmutableGenericPartitionStatisticsFile.builder() - .snapshotId(3055729675574597004L) - .path("s3://a/b/partition-stats.parquet") - .fileSizeInBytes(43L) - .build(), - Iterables.getOnlyElement(parsed.partitionStatisticsFiles())); + assertThat(parsed.partitionStatisticsFiles()) + .hasSize(1) + .first() + .isEqualTo( + ImmutableGenericPartitionStatisticsFile.builder() + .snapshotId(3055729675574597004L) + .path("s3://a/b/partition-stats.parquet") + .fileSizeInBytes(43L) + .build()); } @Test public void testNoReservedPropertyForTableMetadataCreation() { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TableMetadata.newTableMetadata( schema, @@ -1702,7 +1566,7 @@ public void testNoReservedPropertyForTableMetadataCreation() { .hasMessage( "Table properties should not contain reserved properties, but got {format-version=1}"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TableMetadata.newTableMetadata( schema, @@ -1722,10 +1586,9 @@ public void testNoTrailingLocationSlash() { TableMetadata meta = TableMetadata.newTableMetadata( TEST_SCHEMA, SPEC_5, SORT_ORDER_3, locationWithSlash, Collections.emptyMap()); - Assert.assertEquals( - "Metadata should never return a location ending in a slash", - locationWithoutSlash, - meta.location()); + assertThat(meta.location()) + .as("Metadata should never return a location ending in a slash") + .isEqualTo(locationWithoutSlash); } private String createManifestListWithManifestFile( @@ -1746,9 +1609,9 @@ private String createManifestListWithManifestFile( public void buildReplacementKeepsSnapshotLog() throws Exception { TableMetadata metadata = TableMetadataParser.fromJson(readTableMetadataInputFile("TableMetadataV2Valid.json")); - Assertions.assertThat(metadata.currentSnapshot()).isNotNull(); - Assertions.assertThat(metadata.snapshots()).hasSize(2); - Assertions.assertThat(metadata.snapshotLog()).hasSize(2); + assertThat(metadata.currentSnapshot()).isNotNull(); + assertThat(metadata.snapshots()).hasSize(2); + assertThat(metadata.snapshotLog()).hasSize(2); TableMetadata replacement = metadata.buildReplacement( @@ -1758,11 +1621,9 @@ public void buildReplacementKeepsSnapshotLog() throws Exception { metadata.location(), metadata.properties()); - Assertions.assertThat(replacement.currentSnapshot()).isNull(); - Assertions.assertThat(replacement.snapshots()) - .hasSize(2) - .containsExactlyElementsOf(metadata.snapshots()); - Assertions.assertThat(replacement.snapshotLog()) + assertThat(replacement.currentSnapshot()).isNull(); + assertThat(replacement.snapshots()).hasSize(2).containsExactlyElementsOf(metadata.snapshots()); + assertThat(replacement.snapshotLog()) .hasSize(2) .containsExactlyElementsOf(metadata.snapshotLog()); } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java index d343586ef778..f47968434bfe 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java @@ -19,29 +19,26 @@ package org.apache.iceberg; import static org.apache.iceberg.TestHelpers.assertSameSchemaList; +import static org.assertj.core.api.Assertions.assertThat; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.ObjectInputStream; import java.io.ObjectOutputStream; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestTableMetadataSerialization extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +@ExtendWith(ParameterizedTestExtension.class) +public class TestTableMetadataSerialization extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestTableMetadataSerialization(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testSerialization() throws Exception { // add a commit to the metadata so there is at least one snapshot, and history table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -59,31 +56,21 @@ public void testSerialization() throws Exception { result = (TableMetadata) reader.readObject(); } - Assert.assertEquals( - "Metadata file location should match", - meta.metadataFileLocation(), - result.metadataFileLocation()); - Assert.assertEquals("UUID should match", meta.uuid(), result.uuid()); - Assert.assertEquals("Location should match", meta.location(), result.location()); - Assert.assertEquals( - "Last updated should match", meta.lastUpdatedMillis(), result.lastUpdatedMillis()); - Assert.assertEquals("Last column id", meta.lastColumnId(), result.lastColumnId()); - Assert.assertEquals( - "Schema should match", meta.schema().asStruct(), result.schema().asStruct()); + assertThat(result.metadataFileLocation()).isEqualTo(meta.metadataFileLocation()); + assertThat(result.uuid()).isEqualTo(meta.uuid()); + assertThat(result.location()).isEqualTo(meta.location()); + assertThat(result.lastUpdatedMillis()).isEqualTo(meta.lastUpdatedMillis()); + assertThat(result.lastColumnId()).isEqualTo(meta.lastColumnId()); + assertThat(result.schema().asStruct()).isEqualTo(meta.schema().asStruct()); assertSameSchemaList(meta.schemas(), result.schemas()); - Assert.assertEquals( - "Current schema id should match", meta.currentSchemaId(), result.currentSchemaId()); - Assert.assertEquals("Spec should match", meta.defaultSpecId(), result.defaultSpecId()); - Assert.assertEquals("Spec list should match", meta.specs(), result.specs()); - Assert.assertEquals("Properties should match", meta.properties(), result.properties()); - Assert.assertEquals( - "Current snapshot ID should match", - meta.currentSnapshot().snapshotId(), - result.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Snapshots should match", - Lists.transform(meta.snapshots(), Snapshot::snapshotId), - Lists.transform(result.snapshots(), Snapshot::snapshotId)); - Assert.assertEquals("History should match", meta.snapshotLog(), result.snapshotLog()); + assertThat(result.currentSchemaId()).isEqualTo(meta.currentSchemaId()); + assertThat(result.defaultSpecId()).isEqualTo(meta.defaultSpecId()); + assertThat(result.specs()).isEqualTo(meta.specs()); + assertThat(result.properties()).isEqualTo(meta.properties()); + assertThat(result.currentSnapshot().snapshotId()) + .isEqualTo(meta.currentSnapshot().snapshotId()); + assertThat(Lists.transform(result.snapshots(), Snapshot::snapshotId)) + .isEqualTo(Lists.transform(meta.snapshots(), Snapshot::snapshotId)); + assertThat(result.snapshotLog()).isEqualTo(meta.snapshotLog()); } } From c31706d20026b760e9895e24334afbd4e827ac8b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 15 Mar 2024 09:41:32 +0100 Subject: [PATCH 0131/1019] docs: Remove roadmap (#9941) For now. It is outdated and might confuse users. --- site/docs/roadmap.md | 56 -------------------------------------------- site/nav.yml | 1 - 2 files changed, 57 deletions(-) delete mode 100644 site/docs/roadmap.md diff --git a/site/docs/roadmap.md b/site/docs/roadmap.md deleted file mode 100644 index 7df1114c76b6..000000000000 --- a/site/docs/roadmap.md +++ /dev/null @@ -1,56 +0,0 @@ ---- -title: "Roadmap" ---- - - -# Roadmap Overview - -This roadmap outlines projects that the Iceberg community is working on. -Each high-level item links to a Github project board that tracks the current status. -Related design docs will be linked on the planning boards. - -# General - -* [Multi-table transaction support](https://github.com/apache/iceberg/projects/30) -* [Views Support](https://github.com/apache/iceberg/projects/29) -* [Change Data Capture (CDC) Support](https://github.com/apache/iceberg/projects/26) -* [Snapshot tagging and branching](https://github.com/apache/iceberg/projects/4) -* [Inline file compaction](https://github.com/apache/iceberg/projects/14) -* [Delete File compaction](https://github.com/apache/iceberg/projects/10) -* [Z-ordering / Space-filling curves](https://github.com/apache/iceberg/projects/16) -* [Support UPSERT](https://github.com/apache/iceberg/projects/15) - -# Clients -_Python, Rust, and Go projects are pointing to their respective repositories which include -their own issues as the implementations are not final._ - -* [Add the Iceberg Python Client](https://github.com/apache/iceberg-python) -* [Add the Iceberg Rust Client](https://github.com/apache/iceberg-rust) -* [Add the Iceberg Go Client](https://github.com/apache/iceberg-go) - -# Spec V2 - -* [Views Spec](https://github.com/apache/iceberg/projects/6) -* [DSv2 streaming improvements](https://github.com/apache/iceberg/projects/2) -* [Secondary indexes](https://github.com/apache/iceberg/projects/17) - -# Spec V3 - -* [Encryption](https://github.com/apache/iceberg/projects/5) -* [Relative paths](https://github.com/apache/iceberg/projects/18) -* [Default field values](https://github.com/apache/iceberg/projects/19) diff --git a/site/nav.yml b/site/nav.yml index f315e0ae9616..aa5c77b46122 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -28,7 +28,6 @@ nav: - 1.4.1: '!include docs/docs/1.4.1/mkdocs.yml' - 1.4.0: '!include docs/docs/1.4.0/mkdocs.yml' - Releases: releases.md - - Roadmap: roadmap.md - Blogs: blogs.md - Talks: talks.md - Vendors: vendors.md From a4ef1964fbfef2f4795a72fa0e20dcb23aca247c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 15 Mar 2024 09:50:25 +0100 Subject: [PATCH 0132/1019] Docs: document view properties (#9961) --- docs/docs/view-configuration.md | 43 +++++++++++++++++++++++++++++++++ docs/mkdocs.yml | 2 ++ 2 files changed, 45 insertions(+) create mode 100644 docs/docs/view-configuration.md diff --git a/docs/docs/view-configuration.md b/docs/docs/view-configuration.md new file mode 100644 index 000000000000..e8b02e32f23c --- /dev/null +++ b/docs/docs/view-configuration.md @@ -0,0 +1,43 @@ +--- +title: "Configuration" +--- + + +# Configuration + +## View properties + +Iceberg views support properties to configure view behavior. Below is an overview of currently available view properties. + + +| Property | Default | Description | +|--------------------------------------------|---------|------------------------------------------------------------------------------------| +| write.metadata.compression-codec | gzip | Metadata compression codec: `none` or `gzip` | +| version.history.num-entries | 10 | Controls the number of `versions` to retain | +| replace.drop-dialect.allowed | false | Controls whether a SQL dialect is allowed to be dropped during a replace operation | + + +### View behavior properties + + +| Property | Default | Description | +|-------------------------------------|---------------------|--------------------------------------------------------------------| +| commit.retry.num-retries | 4 | Number of times to retry a commit before failing | +| commit.retry.min-wait-ms | 100 | Minimum time in milliseconds to wait before retrying a commit | +| commit.retry.max-wait-ms | 60000 (1 min) | Maximum time in milliseconds to wait before retrying a commit | +| commit.retry.total-timeout-ms | 1800000 (30 min) | Total retry timeout period in milliseconds for a commit | diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index e6b6a0052f16..a8804ed04dc4 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -31,6 +31,8 @@ nav: - performance.md - reliability.md - schemas.md + - Views: + - view-configuration.md - Spark: - spark-getting-started.md - spark-configuration.md From e23a1c0b4dc88ccc8db9a1079605bc1aa61edda7 Mon Sep 17 00:00:00 2001 From: Gang Wu Date: Fri, 15 Mar 2024 22:24:23 +0800 Subject: [PATCH 0133/1019] Parquet: Refactor BasePageIterator to add initRepetitionLevelsReader (#9751) --- .../iceberg/parquet/BasePageIterator.java | 23 ++++++++++++++----- 1 file changed, 17 insertions(+), 6 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java index 75989e8f649b..6389177bff63 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/BasePageIterator.java @@ -81,6 +81,21 @@ protected abstract void initDefinitionLevelsReader( protected abstract void initDefinitionLevelsReader( DataPageV2 dataPageV2, ColumnDescriptor descriptor) throws IOException; + protected void initRepetitionLevelsReader( + DataPageV1 dataPageV1, ColumnDescriptor descriptor, ByteBufferInputStream in, int count) + throws IOException { + ValuesReader rlReader = + dataPageV1.getRlEncoding().getValuesReader(descriptor, ValuesType.REPETITION_LEVEL); + this.repetitionLevels = new ValuesReaderIntIterator(rlReader); + rlReader.initFromPage(count, in); + } + + protected void initRepetitionLevelsReader(DataPageV2 dataPageV2, ColumnDescriptor descriptor) + throws IOException { + this.repetitionLevels = + newRLEIterator(descriptor.getMaxRepetitionLevel(), dataPageV2.getRepetitionLevels()); + } + public int currentPageCount() { return triplesCount; } @@ -112,15 +127,12 @@ public ValuesReader visit(DataPageV2 dataPageV2) { protected void initFromPage(DataPageV1 initPage) { this.triplesCount = initPage.getValueCount(); - ValuesReader rlReader = - initPage.getRlEncoding().getValuesReader(desc, ValuesType.REPETITION_LEVEL); - this.repetitionLevels = new ValuesReaderIntIterator(rlReader); try { BytesInput bytes = initPage.getBytes(); LOG.debug("page size {} bytes and {} records", bytes.size(), triplesCount); LOG.debug("reading repetition levels at 0"); ByteBufferInputStream in = bytes.toInputStream(); - rlReader.initFromPage(triplesCount, in); + initRepetitionLevelsReader(initPage, desc, in, triplesCount); LOG.debug("reading definition levels at {}", in.position()); initDefinitionLevelsReader(initPage, desc, in, triplesCount); LOG.debug("reading data at {}", in.position()); @@ -132,9 +144,8 @@ protected void initFromPage(DataPageV1 initPage) { protected void initFromPage(DataPageV2 initPage) { this.triplesCount = initPage.getValueCount(); - this.repetitionLevels = - newRLEIterator(desc.getMaxRepetitionLevel(), initPage.getRepetitionLevels()); try { + initRepetitionLevelsReader(initPage, desc); initDefinitionLevelsReader(initPage, desc); LOG.debug("page data size {} bytes and {} records", initPage.getData().size(), triplesCount); initDataReader(initPage.getDataEncoding(), initPage.getData().toInputStream(), triplesCount); From a26cc504ba61fbda962af001f3a3a26fc3ece157 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 08:33:55 +0100 Subject: [PATCH 0134/1019] Build: Bump com.google.errorprone:error_prone_annotations (#9972) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.24.1 to 2.26.1. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.24.1...v2.26.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 867c87374e8e..3b7a8e08fab1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -36,7 +36,7 @@ calcite = "1.10.0" delta-standalone = "3.1.0" delta-spark = "3.1.0" esotericsoftware-kryo = "4.0.2" -errorprone-annotations = "2.24.1" +errorprone-annotations = "2.26.1" findbugs-jsr305 = "3.0.2" flink116 = { strictly = "1.16.3"} flink117 = { strictly = "1.17.2"} From 68c7b4ba489c438488acb009f0747d377b8d973d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 08:35:27 +0100 Subject: [PATCH 0135/1019] Build: Bump org.awaitility:awaitility from 4.2.0 to 4.2.1 (#9970) Bumps [org.awaitility:awaitility](https://github.com/awaitility/awaitility) from 4.2.0 to 4.2.1. - [Changelog](https://github.com/awaitility/awaitility/blob/master/changelog.txt) - [Commits](https://github.com/awaitility/awaitility/compare/awaitility-4.2.0...awaitility-4.2.1) --- updated-dependencies: - dependency-name: org.awaitility:awaitility 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 3b7a8e08fab1..f8d912afb4b4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ aircompressor = "0.26" arrow = "15.0.0" avro = "1.11.3" assertj-core = "3.25.3" -awaitility = "4.2.0" +awaitility = "4.2.1" awssdk-bom = "2.24.5" azuresdk-bom = "1.2.20" awssdk-s3accessgrants = "2.0.0" From b88fbed13248f13623fa873b88ce43ec2e262574 Mon Sep 17 00:00:00 2001 From: Alex Merced Date: Mon, 18 Mar 2024 03:37:45 -0400 Subject: [PATCH 0136/1019] Docs: Add 13 Dremio Blogs + Fix a few incorrect dates (#9967) --- site/docs/blogs.md | 68 +++++++++++++++++++++++++++++++++++++++++++--- 1 file changed, 64 insertions(+), 4 deletions(-) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 24e472c19500..746eef97d663 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -22,6 +22,41 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. +### [The Apache Iceberg Lakehouse: The Great Data Equalizer](https://amdatalakehouse.substack.com/p/the-apache-iceberg-lakehouse-the) +**Date**: March 6th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [Data Lakehouse Versioning Comparison: (Nessie, Apache Iceberg, LakeFS)](https://www.dremio.com/blog/data-lakehouse-versioning-comparison-nessie-apache-iceberg-lakefs/) +**Date**: March 5th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [What is Lakehouse Management?: Git-for-Data, Automated Apache Iceberg Table Maintenance and more](https://www.dremio.com/blog/what-is-lakehouse-management-git-for-data-automated-apache-iceberg-table-maintenance-and-more/) +**Date**: February 23rd, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [What is DataOps? Automating Data Management on the Apache Iceberg Lakehouse](https://www.dremio.com/blog/what-is-dataops-automating-data-management-on-the-apache-iceberg-lakehouse/) +**Date**: February 23rd, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [What is the Data Lakehouse and the Role of Apache Iceberg, Nessie and Dremio?](https://amdatalakehouse.substack.com/p/the-apache-iceberg-lakehouse-the) +**Date**: February 21st, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [Ingesting Data Into Apache Iceberg Tables with Dremio: A Unified Path to Iceberg](https://www.dremio.com/blog/ingesting-data-into-apache-iceberg-tables-with-dremio-a-unified-path-to-iceberg/) +**Date**: February 1st, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [Open Source and the Data Lakehouse: Apache Arrow, Apache Iceberg, Nessie and Dremio](https://www.dremio.com/blog/open-source-and-the-data-lakehouse-apache-arrow-apache-iceberg-nessie-and-dremio/) +**Date**: February 1st, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How not to use Apache Iceberg](https://medium.com/@ajanthabhat/how-not-to-use-apache-iceberg-046ae7e7c884) **Date**: January 23rd, 2024, **Company**: Dremio @@ -37,6 +72,16 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ayush Saxena](https://www.linkedin.com/in/ayush151/) +### [Getting Started with Flink SQL and Apache Iceberg](https://www.dremio.com/blog/getting-started-with-flink-sql-and-apache-iceberg/) +**Date**: August 8th, 2023, **Company**: Dremio + +**Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + +### [Using Flink with Apache Iceberg and Nessie](https://www.dremio.com/blog/using-flink-with-apache-iceberg-and-nessie/) +**Date**: July 28th, 2023, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [From Hive Tables to Iceberg Tables: Hassle-Free](https://blog.cloudera.com/from-hive-tables-to-iceberg-tables-hassle-free/) **Date**: July 14th, 2023, **Company**: Cloudera @@ -57,23 +102,38 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Rui Li](https://www.linkedin.com/in/rui-li-19282979/) +### [How to Convert JSON Files Into an Apache Iceberg Table with Dremio](https://www.dremio.com/blog/how-to-convert-json-files-into-an-apache-iceberg-table-with-dremio/) +**Date**: May 31st, 2023, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [Deep Dive Into Configuring Your Apache Iceberg Catalog with Apache Spark](https://www.dremio.com/blog/deep-dive-into-configuring-your-apache-iceberg-catalog-with-apache-spark/) +**Date**: May 31st, 2023, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + +### [Streamlining Data Quality in Apache Iceberg with write-audit-publish & branching](https://www.dremio.com/blog/streamlining-data-quality-in-apache-iceberg-with-write-audit-publish-branching/) +**Date**: May 19th, 2023, **Company**: Dremio + +**Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [Introducing the Apache Iceberg Catalog Migration Tool](https://www.dremio.com/blog/introducing-the-apache-iceberg-catalog-migration-tool/) -**Date**: May 12th, 2022, **Company**: Dremio +**Date**: May 12th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) ### [3 Ways to Use Python with Apache Iceberg](https://www.dremio.com/blog/3-ways-to-use-python-with-apache-iceberg/) -**Date**: April 12th, 2022, **Company**: Dremio +**Date**: April 12th, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) ### [3 Ways to Convert a Delta Lake Table Into an Apache Iceberg Table](https://www.dremio.com/blog/3-ways-to-convert-a-delta-lake-table-into-an-apache-iceberg-table/) -**Date**: April 3rd, 2022, **Company**: Dremio +**Date**: April 3rd, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) ### [How to Convert CSV Files into an Apache Iceberg table with Dremio](https://www.dremio.com/blog/how-to-convert-csv-files-into-an-apache-iceberg-table-with-dremio/) -**Date**: April 3rd, 2022, **Company**: Dremio +**Date**: April 3rd, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) From 36d2d99d9400c2be800a668debeb09e99009b0e8 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 18 Mar 2024 15:39:05 +0800 Subject: [PATCH 0137/1019] Build: Fix ignoring major version update in dependabot (#9981) I got the config wrong in the previous attempt #9806. This PR fixes it following the [official example](https://github.blog/changelog/2021-05-21-dependabot-version-updates-can-now-ignore-major-minor-patch-releases/) --- .github/dependabot.yml | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/.github/dependabot.yml b/.github/dependabot.yml index ab81fe8de75f..51a34758fe09 100644 --- a/.github/dependabot.yml +++ b/.github/dependabot.yml @@ -24,19 +24,24 @@ updates: schedule: interval: "weekly" day: "sunday" + ignore: + - dependency-name: "*" + update-types: ["version-update:semver-major"] - package-ecosystem: "gradle" directory: "/" schedule: interval: "weekly" day: "sunday" open-pull-requests-limit: 50 + ignore: + - dependency-name: "*" + update-types: ["version-update:semver-major"] - package-ecosystem: "pip" directory: "/" schedule: interval: "weekly" day: "sunday" open-pull-requests-limit: 5 - - ignore: - dependency-name: "*" - update-types: ["version-update:semver-major"] - + ignore: + - dependency-name: "*" + update-types: ["version-update:semver-major"] From 8f8db7f32bc4ce3905e3200dfc8f1010ce61a61a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 08:40:22 +0100 Subject: [PATCH 0138/1019] Build: Bump nessie from 0.77.1 to 0.79.0 (#9976) Bumps `nessie` from 0.77.1 to 0.79.0. Updates `org.projectnessie.nessie:nessie-client` from 0.77.1 to 0.79.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.77.1 to 0.79.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.77.1 to 0.79.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.77.1 to 0.79.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 f8d912afb4b4..26d0a79e57b0 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.77.1" +nessie = "0.79.0" netty-buffer = "4.1.107.Final" netty-buffer-compat = "4.1.107.Final" object-client-bundle = "3.3.2" From 0cbc6a3fd6565a618ad6467d935dd72f0df8a86d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 08:40:39 +0100 Subject: [PATCH 0139/1019] Build: Bump datamodel-code-generator from 0.25.4 to 0.25.5 (#9979) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.4 to 0.25.5. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.4...0.25.5) --- 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 4ffabfde3e07..5b819b796841 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.25.4 +datamodel-code-generator==0.25.5 From 1bd67f13c50a5604fbfe750adad1f2d96d938445 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 08:41:20 +0100 Subject: [PATCH 0140/1019] Build: Bump mkdocs-material from 9.5.9 to 9.5.14 (#9983) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.9 to 9.5.14. - [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.9...9.5.14) --- 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 64fa41cab2af..e1efc7cbb2f9 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.9 +mkdocs-material==9.5.14 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 cc7924493606b3dac8a926f14f75fd1d00bcfb0c Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Mon, 18 Mar 2024 17:00:42 +0900 Subject: [PATCH 0141/1019] Core: Migrate tests to JUnit5 (#9964) --- .../apache/iceberg/TestFormatVersions.java | 38 ++-- .../apache/iceberg/TestLocationProvider.java | 138 ++++++------- .../apache/iceberg/TestManifestCaching.java | 68 ++++--- .../apache/iceberg/TestManifestCleanup.java | 89 ++++----- .../iceberg/TestManifestEncryption.java | 65 +++--- .../iceberg/TestManifestListVersions.java | 185 ++++++++--------- .../iceberg/TestManifestReaderStats.java | 146 +++++++------- .../apache/iceberg/TestManifestWriter.java | 187 ++++++++---------- .../iceberg/TestManifestWriterVersions.java | 123 ++++++------ 9 files changed, 479 insertions(+), 560 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java index b4f80088d2cc..2f6a01c6e603 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -18,50 +18,54 @@ */ package org.apache.iceberg; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -public class TestFormatVersions extends TableTestBase { - public TestFormatVersions() { - super(1); +import java.util.Arrays; +import java.util.List; +import org.junit.jupiter.api.TestTemplate; + +public class TestFormatVersions extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1); } - @Test + @TestTemplate public void testDefaultFormatVersion() { - Assert.assertEquals("Should default to v1", 1, table.ops().current().formatVersion()); + assertThat(table.ops().current().formatVersion()).isEqualTo(1); } - @Test + @TestTemplate public void testFormatVersionUpgrade() { TableOperations ops = table.ops(); TableMetadata base = ops.current(); ops.commit(base, base.upgradeToFormatVersion(2)); - Assert.assertEquals("Should report v2", 2, ops.current().formatVersion()); + assertThat(ops.current().formatVersion()).isEqualTo(2); } - @Test + @TestTemplate public void testFormatVersionDowngrade() { TableOperations ops = table.ops(); TableMetadata base = ops.current(); ops.commit(base, base.upgradeToFormatVersion(2)); - Assert.assertEquals("Should report v2", 2, ops.current().formatVersion()); + assertThat(ops.current().formatVersion()).isEqualTo(2); - Assertions.assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(1)) + assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot downgrade v2 table to v1"); - Assert.assertEquals("Should report v2", 2, ops.current().formatVersion()); + assertThat(ops.current().formatVersion()).isEqualTo(2); } - @Test + @TestTemplate public void testFormatVersionUpgradeNotSupported() { TableOperations ops = table.ops(); TableMetadata base = ops.current(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ops.commit( base, @@ -69,6 +73,6 @@ public void testFormatVersionUpgradeNotSupported() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot upgrade table to unsupported format version: v3 (supported: v2)"); - Assert.assertEquals("Should report v1", 1, ops.current().formatVersion()); + assertThat(ops.current().formatVersion()).isEqualTo(1); } } diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 6afc7f0fe715..1b9c6581200a 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -18,27 +18,22 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; import java.util.List; import java.util.Map; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.base.Splitter; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestLocationProvider extends TableTestBase { - @Parameterized.Parameters - public static Object[][] parameters() { - return new Object[][] { - new Object[] {1}, new Object[] {2}, - }; - } - - public TestLocationProvider(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestLocationProvider extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } // publicly visible for testing to be dynamically loaded @@ -99,29 +94,25 @@ public static class InvalidNoInterfaceDynamicallyLoadedLocationProvider { // Default no-arg constructor is present, but does not impelemnt interface LocationProvider } - @Test + @TestTemplate public void testDefaultLocationProvider() { this.table.updateProperties().commit(); this.table.locationProvider().newDataLocation("my_file"); - Assert.assertEquals( - "Default data path should have table location as root", - String.format("%s/data/%s", this.table.location(), "my_file"), - this.table.locationProvider().newDataLocation("my_file")); + assertThat(this.table.locationProvider().newDataLocation("my_file")) + .isEqualTo(String.format("%s/data/%s", this.table.location(), "my_file")); } - @Test + @TestTemplate public void testDefaultLocationProviderWithCustomDataLocation() { this.table.updateProperties().set(TableProperties.WRITE_DATA_LOCATION, "new_location").commit(); this.table.locationProvider().newDataLocation("my_file"); - Assert.assertEquals( - "Default location provider should allow custom path location", - "new_location/my_file", - this.table.locationProvider().newDataLocation("my_file")); + assertThat(this.table.locationProvider().newDataLocation("my_file")) + .isEqualTo("new_location/my_file"); } - @Test + @TestTemplate public void testNoArgDynamicallyLoadedLocationProvider() { String invalidImpl = String.format( @@ -133,13 +124,11 @@ public void testNoArgDynamicallyLoadedLocationProvider() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, invalidImpl) .commit(); - Assert.assertEquals( - "Custom provider should take base table location", - "test_no_arg_provider/my_file", - this.table.locationProvider().newDataLocation("my_file")); + assertThat(this.table.locationProvider().newDataLocation("my_file")) + .isEqualTo("test_no_arg_provider/my_file"); } - @Test + @TestTemplate public void testTwoArgDynamicallyLoadedLocationProvider() { this.table .updateProperties() @@ -151,17 +140,15 @@ public void testTwoArgDynamicallyLoadedLocationProvider() { TwoArgDynamicallyLoadedLocationProvider.class.getSimpleName())) .commit(); - Assert.assertTrue( - String.format("Table should load impl defined in its properties"), - this.table.locationProvider() instanceof TwoArgDynamicallyLoadedLocationProvider); + assertThat(this.table.locationProvider()) + .as("Table should load impl defined in its properties") + .isInstanceOf(TwoArgDynamicallyLoadedLocationProvider.class); - Assert.assertEquals( - "Custom provider should take base table location", - String.format("%s/test_custom_provider/%s", this.table.location(), "my_file"), - this.table.locationProvider().newDataLocation("my_file")); + assertThat(this.table.locationProvider().newDataLocation("my_file")) + .isEqualTo(String.format("%s/test_custom_provider/%s", this.table.location(), "my_file")); } - @Test + @TestTemplate public void testDynamicallyLoadedLocationProviderNotFound() { String nonExistentImpl = String.format( @@ -173,7 +160,7 @@ public void testDynamicallyLoadedLocationProviderNotFound() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, nonExistentImpl) .commit(); - Assertions.assertThatThrownBy(() -> table.locationProvider()) + assertThatThrownBy(() -> table.locationProvider()) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith( String.format( @@ -185,7 +172,7 @@ public void testDynamicallyLoadedLocationProviderNotFound() { + "taking in the string base table location and its property string map."); } - @Test + @TestTemplate public void testInvalidNoInterfaceDynamicallyLoadedLocationProvider() { String invalidImpl = String.format( @@ -197,7 +184,7 @@ public void testInvalidNoInterfaceDynamicallyLoadedLocationProvider() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, invalidImpl) .commit(); - Assertions.assertThatThrownBy(() -> table.locationProvider()) + assertThatThrownBy(() -> table.locationProvider()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( String.format( @@ -205,7 +192,7 @@ public void testInvalidNoInterfaceDynamicallyLoadedLocationProvider() { LocationProvider.class)); } - @Test + @TestTemplate public void testInvalidArgTypesDynamicallyLoadedLocationProvider() { String invalidImpl = String.format( @@ -217,7 +204,7 @@ public void testInvalidArgTypesDynamicallyLoadedLocationProvider() { .set(TableProperties.WRITE_LOCATION_PROVIDER_IMPL, invalidImpl) .commit(); - Assertions.assertThatThrownBy(() -> table.locationProvider()) + assertThatThrownBy(() -> table.locationProvider()) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith( String.format( @@ -225,13 +212,13 @@ public void testInvalidArgTypesDynamicallyLoadedLocationProvider() { invalidImpl, LocationProvider.class)); } - @Test + @TestTemplate public void testObjectStorageLocationProviderPathResolution() { table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "true").commit(); - Assert.assertTrue( - "default data location should be used when object storage path not set", - table.locationProvider().newDataLocation("file").contains(table.location() + "/data")); + assertThat(table.locationProvider().newDataLocation("file")) + .as("default data location should be used when object storage path not set") + .contains(table.location() + "/data"); String folderPath = "s3://random/folder/location"; table @@ -239,32 +226,32 @@ public void testObjectStorageLocationProviderPathResolution() { .set(TableProperties.WRITE_FOLDER_STORAGE_LOCATION, folderPath) .commit(); - Assert.assertTrue( - "folder storage path should be used when set", - table.locationProvider().newDataLocation("file").contains(folderPath)); + assertThat(table.locationProvider().newDataLocation("file")) + .as("folder storage path should be used when set") + .contains(folderPath); String objectPath = "s3://random/object/location"; table.updateProperties().set(TableProperties.OBJECT_STORE_PATH, objectPath).commit(); - Assert.assertTrue( - "object storage path should be used when set", - table.locationProvider().newDataLocation("file").contains(objectPath)); + assertThat(table.locationProvider().newDataLocation("file")) + .as("object storage path should be used when set") + .contains(objectPath); String dataPath = "s3://random/data/location"; table.updateProperties().set(TableProperties.WRITE_DATA_LOCATION, dataPath).commit(); - Assert.assertTrue( - "write data path should be used when set", - table.locationProvider().newDataLocation("file").contains(dataPath)); + assertThat(table.locationProvider().newDataLocation("file")) + .as("write data path should be used when set") + .contains(dataPath); } - @Test + @TestTemplate public void testDefaultStorageLocationProviderPathResolution() { table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "false").commit(); - Assert.assertTrue( - "default data location should be used when object storage path not set", - table.locationProvider().newDataLocation("file").contains(table.location() + "/data")); + assertThat(table.locationProvider().newDataLocation("file")) + .as("default data location should be used when object storage path not set") + .contains(table.location() + "/data"); String folderPath = "s3://random/folder/location"; table @@ -272,19 +259,19 @@ public void testDefaultStorageLocationProviderPathResolution() { .set(TableProperties.WRITE_FOLDER_STORAGE_LOCATION, folderPath) .commit(); - Assert.assertTrue( - "folder storage path should be used when set", - table.locationProvider().newDataLocation("file").contains(folderPath)); + assertThat(table.locationProvider().newDataLocation("file")) + .as("folder storage path should be used when set") + .contains(folderPath); String dataPath = "s3://random/data/location"; table.updateProperties().set(TableProperties.WRITE_DATA_LOCATION, dataPath).commit(); - Assert.assertTrue( - "write data path should be used when set", - table.locationProvider().newDataLocation("file").contains(dataPath)); + assertThat(table.locationProvider().newDataLocation("file")) + .as("write data path should be used when set") + .contains(dataPath); } - @Test + @TestTemplate public void testObjectStorageWithinTableLocation() { table.updateProperties().set(TableProperties.OBJECT_STORE_ENABLED, "true").commit(); @@ -292,11 +279,10 @@ public void testObjectStorageWithinTableLocation() { String relativeLocation = fileLocation.replaceFirst(table.location(), ""); List parts = Splitter.on("/").splitToList(relativeLocation); - Assert.assertEquals("Should contain 4 parts", 4, parts.size()); - Assert.assertTrue("First part should be empty", parts.get(0).isEmpty()); - Assert.assertEquals("Second part should be data", "data", parts.get(1)); - Assert.assertFalse("Third part should be a hash value", parts.get(2).isEmpty()); - Assert.assertEquals( - "Fourth part should be the file name passed in", "test.parquet", parts.get(3)); + assertThat(parts).hasSize(4); + 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"); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCaching.java b/core/src/test/java/org/apache/iceberg/TestManifestCaching.java index 3b67cb3e695e..4469dece2f25 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCaching.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCaching.java @@ -19,10 +19,13 @@ package org.apache.iceberg; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import com.github.benmanes.caffeine.cache.Cache; import com.google.common.testing.GcFinalization; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import java.util.Map; import java.util.UUID; @@ -35,13 +38,10 @@ import org.apache.iceberg.io.ContentCache; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestManifestCaching { @@ -54,7 +54,7 @@ public class TestManifestCaching { // Partition spec used to create tables static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).bucket("data", 16).build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testPlanWithCache() throws Exception { @@ -66,7 +66,7 @@ public void testPlanWithCache() throws Exception { "true"); Table table = createTable(properties); ContentCache cache = ManifestFiles.contentCache(table.io()); - Assert.assertEquals(0, cache.estimatedCacheSize()); + assertThat(cache.estimatedCacheSize()).isEqualTo(0); int numFiles = 4; List files16Mb = newFiles(numFiles, 16 * 1024 * 1024); @@ -75,20 +75,22 @@ public void testPlanWithCache() throws Exception { // planTask with SPLIT_SIZE half of the file size TableScan scan1 = table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(8 * 1024 * 1024)); - Assert.assertEquals( - "Should get 2 tasks per file", numFiles * 2, Iterables.size(scan1.planTasks())); - Assert.assertEquals( - "All manifest files should be cached", numFiles, cache.estimatedCacheSize()); - Assert.assertEquals( - "All manifest files should be recently loaded", numFiles, cache.stats().loadCount()); + assertThat(scan1.planTasks()).hasSize(numFiles * 2); + assertThat(cache.estimatedCacheSize()) + .as("All manifest files should be cached") + .isEqualTo(numFiles); + assertThat(cache.stats().loadCount()) + .as("All manifest files should be recently loaded") + .isEqualTo(numFiles); long missCount = cache.stats().missCount(); // planFiles and verify that cache size still the same TableScan scan2 = table.newScan(); - Assert.assertEquals("Should get 1 tasks per file", numFiles, Iterables.size(scan2.planFiles())); - Assert.assertEquals("Cache size should remain the same", numFiles, cache.estimatedCacheSize()); - Assert.assertEquals( - "All manifest file reads should hit cache", missCount, cache.stats().missCount()); + assertThat(scan2.planFiles()).hasSize(numFiles); + assertThat(cache.estimatedCacheSize()).isEqualTo(numFiles); + assertThat(cache.stats().missCount()) + .as("All manifest file reads should hit cache") + .isEqualTo(missCount); ManifestFiles.dropCache(table.io()); } @@ -110,12 +112,14 @@ public void testPlanWithSmallCache() throws Exception { // We should never hit cache. TableScan scan = table.newScan(); ContentCache cache = ManifestFiles.contentCache(scan.table().io()); - Assert.assertEquals(1, cache.maxContentLength()); - Assert.assertEquals(1, cache.maxTotalBytes()); - Assert.assertEquals("Should get 1 tasks per file", numFiles, Iterables.size(scan.planFiles())); - Assert.assertEquals("Cache should be empty", 0, cache.estimatedCacheSize()); - Assert.assertEquals("File should not be loaded through cache", 0, cache.stats().loadCount()); - Assert.assertEquals("Cache should not serve file", 0, cache.stats().requestCount()); + assertThat(cache.maxContentLength()).isEqualTo(1); + assertThat(cache.maxTotalBytes()).isEqualTo(1); + assertThat(scan.planFiles()).hasSize(numFiles); + assertThat(cache.estimatedCacheSize()).isEqualTo(0); + assertThat(cache.stats().loadCount()) + .as("File should not be loaded through cache") + .isEqualTo(0); + assertThat(cache.stats().requestCount()).as("Cache should not serve file").isEqualTo(0); ManifestFiles.dropCache(scan.table().io()); } @@ -140,8 +144,8 @@ public void testUniqueCache() throws Exception { ContentCache cache1 = ManifestFiles.contentCache(table1.io()); ContentCache cache2 = ManifestFiles.contentCache(table2.io()); ContentCache cache3 = ManifestFiles.contentCache(table2.io()); - Assert.assertNotSame(cache1, cache2); - Assert.assertSame(cache2, cache3); + assertThat(cache2).isNotSameAs(cache1); + assertThat(cache3).isSameAs(cache2); ManifestFiles.dropCache(table1.io()); ManifestFiles.dropCache(table2.io()); @@ -161,7 +165,7 @@ public void testRecreateCache() throws Exception { ManifestFiles.dropCache(table.io()); ContentCache cache2 = ManifestFiles.contentCache(table.io()); - Assert.assertNotSame(cache1, cache2); + assertThat(cache2).isNotSameAs(cache1); ManifestFiles.dropCache(table.io()); } @@ -193,10 +197,10 @@ public void testWeakFileIOReferenceCleanUp() { // Verify that manifestCache evicts all FileIO except the firstIO and lastIO. ContentCache cache1 = contentCache(manifestCache, firstIO); ContentCache cacheN = contentCache(manifestCache, lastIO); - Assert.assertSame(firstCache, cache1); - Assert.assertSame(lastCache, cacheN); - Assert.assertEquals(maxIO, manifestCache.stats().loadCount()); - Assert.assertEquals(maxIO - 2, manifestCache.stats().evictionCount()); + assertThat(cache1).isSameAs(firstCache); + assertThat(cacheN).isSameAs(lastCache); + assertThat(manifestCache.stats().loadCount()).isEqualTo(maxIO); + assertThat(manifestCache.stats().evictionCount()).isEqualTo(maxIO - 2); } /** @@ -241,7 +245,9 @@ private HadoopCatalog hadoopCatalog(Map catalogProperties) throw "hadoop", ImmutableMap.builder() .putAll(catalogProperties) - .put(CatalogProperties.WAREHOUSE_LOCATION, temp.newFolder().getAbsolutePath()) + .put( + CatalogProperties.WAREHOUSE_LOCATION, + Files.createTempDirectory(temp, "junit").toFile().getAbsolutePath()) .buildOrThrow()); return hadoopCatalog; } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java index f7b989065be9..b5f6d05cc6a0 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java @@ -18,84 +18,71 @@ */ package org.apache.iceberg; -import org.apache.iceberg.expressions.Expressions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestManifestCleanup extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } +import static org.assertj.core.api.Assertions.assertThat; - public TestManifestCleanup(int formatVersion) { - super(formatVersion); +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.expressions.Expressions; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestManifestCleanup extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testDelete() { - Assert.assertEquals("Table should start with no manifests", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals( - "Table should have one append manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Table should have one append manifest") + .hasSize(1); table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - Assert.assertEquals( - "Table should have one delete manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Table should have one delete manifest") + .hasSize(1); table.newAppend().commit(); - Assert.assertEquals( - "Table should have no manifests", - 0, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).isEmpty(); } - @Test + @TestTemplate public void testPartialDelete() { - Assert.assertEquals("Table should start with no manifests", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot s1 = table.currentSnapshot(); - Assert.assertEquals( - "Table should have one append manifest", 1, s1.allManifests(table.io()).size()); + assertThat(s1.allManifests(table.io())).as("Table should have one append manifest").hasSize(1); table.newDelete().deleteFile(FILE_B).commit(); Snapshot s2 = table.currentSnapshot(); - Assert.assertEquals( - "Table should have one mixed manifest", 1, s2.allManifests(table.io()).size()); + assertThat(s2.allManifests(table.io())).as("Table should have one mixed manifest").hasSize(1); table.newAppend().commit(); Snapshot s3 = table.currentSnapshot(); - Assert.assertEquals( - "Table should have the same manifests", - s2.allManifests(table.io()), - s3.allManifests(table.io())); + assertThat(s3.allManifests(table.io())).isEqualTo(s2.allManifests(table.io())); } - @Test + @TestTemplate public void testOverwrite() { - Assert.assertEquals("Table should start with no manifests", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals( - "Table should have one append manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Table should have one append manifest") + .hasSize(1); table .newOverwrite() @@ -104,10 +91,9 @@ public void testOverwrite() { .addFile(FILE_D) .commit(); - Assert.assertEquals( - "Table should have one delete manifest and one append manifest", - 2, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Table should have one delete manifest and one append manifest") + .hasSize(2); table .newOverwrite() @@ -116,9 +102,8 @@ public void testOverwrite() { .addFile(FILE_B) .commit(); - Assert.assertEquals( - "Table should have one delete manifest and one append manifest", - 2, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Table should have one delete manifest and one append manifest") + .hasSize(2); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java index b64324ec573a..13e8985cdb56 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestEncryption.java @@ -19,9 +19,13 @@ package org.apache.iceberg; 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; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.List; import org.apache.avro.InvalidAvroMagicException; import org.apache.iceberg.encryption.EncryptedOutputFile; @@ -38,11 +42,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestManifestEncryption { private static final FileIO FILE_IO = new TestTables.LocalFileIO(); @@ -114,7 +115,7 @@ public class TestManifestEncryption { private static final EncryptionManager ENCRYPTION_MANAGER = EncryptionTestHelpers.createEncryptionManager(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testV1Write() throws IOException { @@ -151,33 +152,31 @@ void checkEntry( Long expectedDataSequenceNumber, Long expectedFileSequenceNumber, FileContent content) { - Assert.assertEquals("Status", ManifestEntry.Status.ADDED, entry.status()); - Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, entry.snapshotId()); - Assert.assertEquals( - "Data sequence number", expectedDataSequenceNumber, entry.dataSequenceNumber()); - Assert.assertEquals( - "File sequence number", expectedFileSequenceNumber, entry.fileSequenceNumber()); + assertThat(entry.status()).isEqualTo(ManifestEntry.Status.ADDED); + assertThat(entry.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(entry.dataSequenceNumber()).isEqualTo(expectedDataSequenceNumber); + assertThat(entry.fileSequenceNumber()).isEqualTo(expectedFileSequenceNumber); checkDataFile(entry.file(), content); } void checkDataFile(ContentFile dataFile, FileContent content) { // DataFile is the superclass of DeleteFile, so this method can check both - Assert.assertEquals("Content", content, dataFile.content()); - Assert.assertEquals("Path", PATH, dataFile.path()); - Assert.assertEquals("Format", FORMAT, dataFile.format()); - Assert.assertEquals("Partition", PARTITION, dataFile.partition()); - Assert.assertEquals("Record count", METRICS.recordCount(), (Long) dataFile.recordCount()); - Assert.assertEquals("Column sizes", METRICS.columnSizes(), dataFile.columnSizes()); - Assert.assertEquals("Value counts", METRICS.valueCounts(), dataFile.valueCounts()); - Assert.assertEquals("Null value counts", METRICS.nullValueCounts(), dataFile.nullValueCounts()); - Assert.assertEquals("NaN value counts", METRICS.nanValueCounts(), dataFile.nanValueCounts()); - Assert.assertEquals("Lower bounds", METRICS.lowerBounds(), dataFile.lowerBounds()); - Assert.assertEquals("Upper bounds", METRICS.upperBounds(), dataFile.upperBounds()); - Assert.assertEquals("Sort order id", SORT_ORDER_ID, dataFile.sortOrderId()); + assertThat(dataFile.content()).isEqualTo(content); + assertThat(dataFile.path()).isEqualTo(PATH); + assertThat(dataFile.format()).isEqualTo(FORMAT); + assertThat(dataFile.partition()).isEqualTo(PARTITION); + assertThat(dataFile.recordCount()).isEqualTo(METRICS.recordCount()); + assertThat(dataFile.columnSizes()).isEqualTo(METRICS.columnSizes()); + assertThat(dataFile.valueCounts()).isEqualTo(METRICS.valueCounts()); + assertThat(dataFile.nullValueCounts()).isEqualTo(METRICS.nullValueCounts()); + assertThat(dataFile.nanValueCounts()).isEqualTo(METRICS.nanValueCounts()); + assertThat(dataFile.lowerBounds()).isEqualTo(METRICS.lowerBounds()); + assertThat(dataFile.upperBounds()).isEqualTo(METRICS.upperBounds()); + assertThat(dataFile.sortOrderId()).isEqualTo(SORT_ORDER_ID); if (dataFile.content() == FileContent.EQUALITY_DELETES) { - Assert.assertEquals(EQUALITY_IDS, dataFile.equalityFieldIds()); + assertThat(dataFile.equalityFieldIds()).isEqualTo(EQUALITY_IDS); } else { - Assert.assertNull(dataFile.equalityFieldIds()); + assertThat(dataFile.equalityFieldIds()).isNull(); } } @@ -187,7 +186,9 @@ private ManifestFile writeManifest(int formatVersion) throws IOException { private ManifestFile writeManifest(DataFile file, int formatVersion) throws IOException { OutputFile manifestFile = - Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + Files.localOutput( + FileFormat.AVRO.addExtension( + File.createTempFile("manifest", null, temp.toFile()).toString())); EncryptedOutputFile encryptedManifest = ENCRYPTION_MANAGER.encrypt(manifestFile); ManifestWriter writer = ManifestFiles.write(formatVersion, SPEC, encryptedManifest, SNAPSHOT_ID); @@ -201,7 +202,7 @@ private ManifestFile writeManifest(DataFile file, int formatVersion) throws IOEx private ManifestEntry readManifest(ManifestFile manifest) throws IOException { // First try to read without decryption - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ManifestFiles.read( manifest, @@ -215,14 +216,16 @@ private ManifestEntry readManifest(ManifestFile manifest) throws IOExc ManifestFiles.read(manifest, EncryptingFileIO.combine(FILE_IO, ENCRYPTION_MANAGER), null) .entries()) { List> files = Lists.newArrayList(reader); - Assert.assertEquals("Should contain only one data file", 1, files.size()); + assertThat(files).hasSize(1); return files.get(0); } } private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { OutputFile manifestFile = - Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + Files.localOutput( + FileFormat.AVRO.addExtension( + File.createTempFile("manifest", null, temp.toFile()).toString())); EncryptedOutputFile encryptedManifest = ENCRYPTION_MANAGER.encrypt(manifestFile); ManifestWriter writer = ManifestFiles.writeDeleteManifest(formatVersion, SPEC, encryptedManifest, SNAPSHOT_ID); @@ -240,7 +243,7 @@ private ManifestEntry readDeleteManifest(ManifestFile manifest) thro manifest, EncryptingFileIO.combine(FILE_IO, ENCRYPTION_MANAGER), null) .entries()) { List> entries = Lists.newArrayList(reader); - Assert.assertEquals("Should contain only one delete file", 1, entries.size()); + assertThat(entries).hasSize(1); return entries.get(0); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java index 5b8df081c3c2..afbee9be1375 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestListVersions.java @@ -18,9 +18,13 @@ */ 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 java.io.IOException; import java.nio.ByteBuffer; +import java.nio.file.Path; import java.util.Collection; import java.util.List; import org.apache.avro.AvroRuntimeException; @@ -39,11 +43,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestManifestListVersions { private static final String PATH = "s3://bucket/table/m1.avro"; @@ -98,11 +99,11 @@ public class TestManifestListVersions { PARTITION_SUMMARIES, KEY_METADATA); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testV1WriteDeleteManifest() { - Assertions.assertThatThrownBy(() -> writeManifestList(TEST_DELETE_MANIFEST, 1)) + assertThatThrownBy(() -> writeManifestList(TEST_DELETE_MANIFEST, 1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot store delete manifests in a v1 table"); } @@ -112,24 +113,21 @@ public void testV1Write() throws IOException { ManifestFile manifest = writeAndReadManifestList(1); // v2 fields are not written and are defaulted - Assert.assertEquals( - "Should not contain sequence number, default to 0", 0, manifest.sequenceNumber()); - Assert.assertEquals( - "Should not contain min sequence number, default to 0", 0, manifest.minSequenceNumber()); + assertThat(manifest.sequenceNumber()).isEqualTo(0); + assertThat(manifest.minSequenceNumber()).isEqualTo(0); // v1 fields are read correctly, even though order changed - Assert.assertEquals("Path", PATH, manifest.path()); - Assert.assertEquals("Length", LENGTH, manifest.length()); - Assert.assertEquals("Spec id", SPEC_ID, manifest.partitionSpecId()); - Assert.assertEquals("Content", ManifestContent.DATA, manifest.content()); - Assert.assertEquals("Snapshot id", SNAPSHOT_ID, (long) manifest.snapshotId()); - Assert.assertEquals("Added files count", ADDED_FILES, (int) manifest.addedFilesCount()); - Assert.assertEquals( - "Existing files count", EXISTING_FILES, (int) manifest.existingFilesCount()); - Assert.assertEquals("Deleted files count", DELETED_FILES, (int) manifest.deletedFilesCount()); - Assert.assertEquals("Added rows count", ADDED_ROWS, (long) manifest.addedRowsCount()); - Assert.assertEquals("Existing rows count", EXISTING_ROWS, (long) manifest.existingRowsCount()); - Assert.assertEquals("Deleted rows count", DELETED_ROWS, (long) manifest.deletedRowsCount()); + assertThat(manifest.path()).isEqualTo(PATH); + assertThat(manifest.length()).isEqualTo(LENGTH); + assertThat(manifest.partitionSpecId()).isEqualTo(SPEC_ID); + assertThat(manifest.content()).isEqualTo(ManifestContent.DATA); + assertThat(manifest.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(manifest.addedFilesCount()).isEqualTo(ADDED_FILES); + assertThat(manifest.existingFilesCount()).isEqualTo(EXISTING_FILES); + assertThat(manifest.deletedFilesCount()).isEqualTo(DELETED_FILES); + assertThat(manifest.addedRowsCount()).isEqualTo(ADDED_ROWS); + assertThat(manifest.existingRowsCount()).isEqualTo(EXISTING_ROWS); + assertThat(manifest.deletedRowsCount()).isEqualTo(DELETED_ROWS); } @Test @@ -137,20 +135,19 @@ public void testV2Write() throws IOException { ManifestFile manifest = writeAndReadManifestList(2); // all v2 fields should be read correctly - Assert.assertEquals("Path", PATH, manifest.path()); - Assert.assertEquals("Length", LENGTH, manifest.length()); - Assert.assertEquals("Spec id", SPEC_ID, manifest.partitionSpecId()); - Assert.assertEquals("Content", ManifestContent.DATA, manifest.content()); - Assert.assertEquals("Sequence number", SEQ_NUM, manifest.sequenceNumber()); - Assert.assertEquals("Min sequence number", MIN_SEQ_NUM, manifest.minSequenceNumber()); - Assert.assertEquals("Snapshot id", SNAPSHOT_ID, (long) manifest.snapshotId()); - Assert.assertEquals("Added files count", ADDED_FILES, (int) manifest.addedFilesCount()); - Assert.assertEquals("Added rows count", ADDED_ROWS, (long) manifest.addedRowsCount()); - Assert.assertEquals( - "Existing files count", EXISTING_FILES, (int) manifest.existingFilesCount()); - Assert.assertEquals("Existing rows count", EXISTING_ROWS, (long) manifest.existingRowsCount()); - Assert.assertEquals("Deleted files count", DELETED_FILES, (int) manifest.deletedFilesCount()); - Assert.assertEquals("Deleted rows count", DELETED_ROWS, (long) manifest.deletedRowsCount()); + assertThat(manifest.path()).isEqualTo(PATH); + assertThat(manifest.length()).isEqualTo(LENGTH); + assertThat(manifest.partitionSpecId()).isEqualTo(SPEC_ID); + assertThat(manifest.content()).isEqualTo(ManifestContent.DATA); + assertThat(manifest.sequenceNumber()).isEqualTo(SEQ_NUM); + assertThat(manifest.minSequenceNumber()).isEqualTo(MIN_SEQ_NUM); + assertThat(manifest.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(manifest.addedFilesCount()).isEqualTo(ADDED_FILES); + assertThat(manifest.addedRowsCount()).isEqualTo(ADDED_ROWS); + assertThat(manifest.existingFilesCount()).isEqualTo(EXISTING_FILES); + assertThat(manifest.existingRowsCount()).isEqualTo(EXISTING_ROWS); + assertThat(manifest.deletedFilesCount()).isEqualTo(DELETED_FILES); + assertThat(manifest.deletedRowsCount()).isEqualTo(DELETED_ROWS); } @Test @@ -159,20 +156,16 @@ public void testV1ForwardCompatibility() throws IOException { GenericData.Record generic = readGeneric(manifestList, V1Metadata.MANIFEST_LIST_SCHEMA); // v1 metadata should match even though order changed - Assert.assertEquals("Path", PATH, generic.get("manifest_path").toString()); - Assert.assertEquals("Length", LENGTH, generic.get("manifest_length")); - Assert.assertEquals("Spec id", SPEC_ID, generic.get("partition_spec_id")); - Assert.assertEquals("Snapshot id", SNAPSHOT_ID, (long) generic.get("added_snapshot_id")); - Assert.assertEquals("Added files count", ADDED_FILES, (int) generic.get("added_files_count")); - Assert.assertEquals( - "Existing files count", EXISTING_FILES, (int) generic.get("existing_files_count")); - Assert.assertEquals( - "Deleted files count", DELETED_FILES, (int) generic.get("deleted_files_count")); - Assert.assertEquals("Added rows count", ADDED_ROWS, (long) generic.get("added_rows_count")); - Assert.assertEquals( - "Existing rows count", EXISTING_ROWS, (long) generic.get("existing_rows_count")); - Assert.assertEquals( - "Deleted rows count", DELETED_ROWS, (long) generic.get("deleted_rows_count")); + assertThat(generic.get("manifest_path")).asString().isEqualTo(PATH); + assertThat(generic.get("manifest_length")).isEqualTo(LENGTH); + assertThat(generic.get("partition_spec_id")).isEqualTo(SPEC_ID); + assertThat(generic.get("added_snapshot_id")).isEqualTo(SNAPSHOT_ID); + assertThat(generic.get("added_files_count")).isEqualTo(ADDED_FILES); + assertThat(generic.get("existing_files_count")).isEqualTo(EXISTING_FILES); + assertThat(generic.get("deleted_files_count")).isEqualTo(DELETED_FILES); + assertThat(generic.get("added_rows_count")).isEqualTo(ADDED_ROWS); + assertThat(generic.get("existing_rows_count")).isEqualTo(EXISTING_ROWS); + assertThat(generic.get("deleted_rows_count")).isEqualTo(DELETED_ROWS); assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); @@ -186,20 +179,16 @@ public void testV2ForwardCompatibility() throws IOException { GenericData.Record generic = readGeneric(manifestList, V1Metadata.MANIFEST_LIST_SCHEMA); // v1 metadata should match even though order changed - Assert.assertEquals("Path", PATH, generic.get("manifest_path").toString()); - Assert.assertEquals("Length", LENGTH, generic.get("manifest_length")); - Assert.assertEquals("Spec id", SPEC_ID, generic.get("partition_spec_id")); - Assert.assertEquals("Snapshot id", SNAPSHOT_ID, (long) generic.get("added_snapshot_id")); - Assert.assertEquals("Added files count", ADDED_FILES, (int) generic.get("added_files_count")); - Assert.assertEquals( - "Existing files count", EXISTING_FILES, (int) generic.get("existing_files_count")); - Assert.assertEquals( - "Deleted files count", DELETED_FILES, (int) generic.get("deleted_files_count")); - Assert.assertEquals("Added rows count", ADDED_ROWS, (long) generic.get("added_rows_count")); - Assert.assertEquals( - "Existing rows count", EXISTING_ROWS, (long) generic.get("existing_rows_count")); - Assert.assertEquals( - "Deleted rows count", DELETED_ROWS, (long) generic.get("deleted_rows_count")); + assertThat(generic.get("manifest_path")).asString().isEqualTo(PATH); + assertThat(generic.get("manifest_length")).isEqualTo(LENGTH); + assertThat(generic.get("partition_spec_id")).isEqualTo(SPEC_ID); + assertThat(generic.get("added_snapshot_id")).isEqualTo(SNAPSHOT_ID); + assertThat(generic.get("added_files_count")).isEqualTo(ADDED_FILES); + assertThat(generic.get("existing_files_count")).isEqualTo(EXISTING_FILES); + assertThat(generic.get("deleted_files_count")).isEqualTo(DELETED_FILES); + assertThat(generic.get("added_rows_count")).isEqualTo(ADDED_ROWS); + assertThat(generic.get("existing_rows_count")).isEqualTo(EXISTING_ROWS); + assertThat(generic.get("deleted_rows_count")).isEqualTo(DELETED_ROWS); assertEmptyAvroField(generic, ManifestFile.MANIFEST_CONTENT.name()); assertEmptyAvroField(generic, ManifestFile.SEQUENCE_NUMBER.name()); assertEmptyAvroField(generic, ManifestFile.MIN_SEQUENCE_NUMBER.name()); @@ -207,8 +196,8 @@ public void testV2ForwardCompatibility() throws IOException { @Test public void testManifestsWithoutRowStats() throws IOException { - File manifestListFile = temp.newFile("manifest-list.avro"); - Assert.assertTrue(manifestListFile.delete()); + File manifestListFile = File.createTempFile("manifest-list", ".avro", temp.toFile()); + assertThat(manifestListFile.delete()).isTrue(); Collection columnNamesWithoutRowStats = ImmutableList.of( @@ -250,18 +239,15 @@ public void testManifestsWithoutRowStats() throws IOException { List files = ManifestLists.read(outputFile.toInputFile()); ManifestFile manifest = Iterables.getOnlyElement(files); - Assert.assertTrue("Added files should be present", manifest.hasAddedFiles()); - Assert.assertEquals("Added files count should match", 2, (int) manifest.addedFilesCount()); - Assert.assertNull("Added rows count should be null", manifest.addedRowsCount()); - - Assert.assertTrue("Existing files should be present", manifest.hasExistingFiles()); - Assert.assertEquals( - "Existing files count should match", 3, (int) manifest.existingFilesCount()); - Assert.assertNull("Existing rows count should be null", manifest.existingRowsCount()); - - Assert.assertTrue("Deleted files should be present", manifest.hasDeletedFiles()); - Assert.assertEquals("Deleted files count should match", 4, (int) manifest.deletedFilesCount()); - Assert.assertNull("Deleted rows count should be null", manifest.deletedRowsCount()); + assertThat(manifest.hasAddedFiles()).isTrue(); + assertThat(manifest.addedFilesCount()).isEqualTo(2); + assertThat(manifest.addedRowsCount()).isNull(); + assertThat(manifest.hasExistingFiles()).isTrue(); + assertThat(manifest.existingFilesCount()).isEqualTo(3); + assertThat(manifest.existingRowsCount()).isNull(); + assertThat(manifest.hasDeletedFiles()).isTrue(); + assertThat(manifest.deletedFilesCount()).isEqualTo(4); + assertThat(manifest.deletedRowsCount()).isNull(); } @Test @@ -298,36 +284,19 @@ public void testManifestsPartitionSummary() throws IOException { List files = ManifestLists.read(manifestList); ManifestFile returnedManifest = Iterables.getOnlyElement(files); - Assert.assertEquals( - "Number of partition field summaries should match", - 2, - returnedManifest.partitions().size()); + assertThat(returnedManifest.partitions()).hasSize(2); ManifestFile.PartitionFieldSummary first = returnedManifest.partitions().get(0); - Assert.assertFalse( - "First partition field summary should not contain null", first.containsNull()); - Assert.assertNull("First partition field summary has unknown NaN", first.containsNaN()); - Assert.assertEquals( - "Lower bound for first partition field summary should match", - firstSummaryLowerBound, - first.lowerBound()); - Assert.assertEquals( - "Upper bound for first partition field summary should match", - firstSummaryUpperBound, - first.upperBound()); + assertThat(first.containsNull()).isFalse(); + assertThat(first.containsNaN()).isNull(); + assertThat(first.lowerBound()).isEqualTo(firstSummaryLowerBound); + assertThat(first.upperBound()).isEqualTo(firstSummaryUpperBound); ManifestFile.PartitionFieldSummary second = returnedManifest.partitions().get(1); - Assert.assertTrue("Second partition field summary should contain null", second.containsNull()); - Assert.assertFalse( - "Second partition field summary should not contain NaN", second.containsNaN()); - Assert.assertEquals( - "Lower bound for second partition field summary should match", - secondSummaryLowerBound, - second.lowerBound()); - Assert.assertEquals( - "Upper bound for second partition field summary should match", - secondSummaryUpperBound, - second.upperBound()); + assertThat(second.containsNull()).isTrue(); + assertThat(second.containsNaN()).isFalse(); + assertThat(second.lowerBound()).isEqualTo(secondSummaryLowerBound); + assertThat(second.upperBound()).isEqualTo(secondSummaryUpperBound); } private InputFile writeManifestList(ManifestFile manifest, int formatVersion) throws IOException { @@ -348,7 +317,7 @@ private GenericData.Record readGeneric(InputFile manifestList, Schema schema) th try (CloseableIterable files = Avro.read(manifestList).project(schema).reuseContainers(false).build()) { List records = Lists.newLinkedList(files); - Assert.assertEquals("Should contain one manifest", 1, records.size()); + assertThat(records).hasSize(1); return records.get(0); } } @@ -356,12 +325,12 @@ private GenericData.Record readGeneric(InputFile manifestList, Schema schema) th private ManifestFile writeAndReadManifestList(int formatVersion) throws IOException { List manifests = ManifestLists.read(writeManifestList(TEST_MANIFEST, formatVersion)); - Assert.assertEquals("Should contain one manifest", 1, manifests.size()); + assertThat(manifests).hasSize(1); return manifests.get(0); } private void assertEmptyAvroField(GenericRecord record, String field) { - Assertions.assertThatThrownBy(() -> record.get(field)) + assertThatThrownBy(() -> record.get(field)) .isInstanceOf(AvroRuntimeException.class) .hasMessage("Not a valid schema field: " + field); } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java index 082800238a0a..4c60a943f76c 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java @@ -18,8 +18,13 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; import java.util.Map; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; @@ -27,21 +32,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestManifestReaderStats extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestManifestReaderStats(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestManifestReaderStats extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } private static final Map VALUE_COUNT = ImmutableMap.of(3, 3L); @@ -65,7 +63,7 @@ public TestManifestReaderStats(int formatVersion) { .withMetrics(METRICS) .build(); - @Test + @TestTemplate public void testReadIncludesFullStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = ManifestFiles.read(manifest, FILE_IO)) { @@ -75,7 +73,7 @@ public void testReadIncludesFullStats() throws IOException { } } - @Test + @TestTemplate public void testReadEntriesWithFilterIncludesFullStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -86,7 +84,7 @@ public void testReadEntriesWithFilterIncludesFullStats() throws IOException { } } - @Test + @TestTemplate public void testReadIteratorWithFilterIncludesFullStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -96,7 +94,7 @@ public void testReadIteratorWithFilterIncludesFullStats() throws IOException { } } - @Test + @TestTemplate public void testReadEntriesWithFilterAndSelectIncludesFullStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -109,7 +107,7 @@ public void testReadEntriesWithFilterAndSelectIncludesFullStats() throws IOExcep } } - @Test + @TestTemplate public void testReadIteratorWithFilterAndSelectDropsStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -121,7 +119,7 @@ public void testReadIteratorWithFilterAndSelectDropsStats() throws IOException { } } - @Test + @TestTemplate public void testReadIteratorWithFilterAndSelectRecordCountDropsStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -133,7 +131,7 @@ public void testReadIteratorWithFilterAndSelectRecordCountDropsStats() throws IO } } - @Test + @TestTemplate public void testReadIteratorWithFilterAndSelectStatsIncludesFullStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -148,7 +146,7 @@ public void testReadIteratorWithFilterAndSelectStatsIncludesFullStats() throws I } } - @Test + @TestTemplate public void testReadIteratorWithProjectStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -156,18 +154,18 @@ public void testReadIteratorWithProjectStats() throws IOException { .project(new Schema(ImmutableList.of(DataFile.FILE_PATH, DataFile.VALUE_COUNTS)))) { DataFile entry = reader.iterator().next(); - Assert.assertEquals(FILE_PATH, entry.path()); - Assert.assertEquals(VALUE_COUNT, entry.valueCounts()); - Assert.assertNull(entry.columnSizes()); - Assert.assertNull(entry.nullValueCounts()); - Assert.assertNull(entry.nanValueCounts()); - Assert.assertNull(entry.lowerBounds()); - Assert.assertNull(entry.upperBounds()); + assertThat(entry.path()).isEqualTo(FILE_PATH); + assertThat(entry.valueCounts()).isEqualTo(VALUE_COUNT); + assertThat(entry.columnSizes()).isNull(); + assertThat(entry.nullValueCounts()).isNull(); + assertThat(entry.nanValueCounts()).isNull(); + assertThat(entry.lowerBounds()).isNull(); + assertThat(entry.upperBounds()).isNull(); assertNullRecordCount(entry); } } - @Test + @TestTemplate public void testReadEntriesWithSelectNotProjectStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -177,20 +175,20 @@ public void testReadEntriesWithSelectNotProjectStats() throws IOException { DataFile dataFile = entry.file(); // selected field is populated - Assert.assertEquals(FILE_PATH, dataFile.path()); + assertThat(dataFile.path()).isEqualTo(FILE_PATH); // not selected fields are all null and not projected - Assert.assertNull(dataFile.columnSizes()); - Assert.assertNull(dataFile.valueCounts()); - Assert.assertNull(dataFile.nullValueCounts()); - Assert.assertNull(dataFile.lowerBounds()); - Assert.assertNull(dataFile.upperBounds()); - Assert.assertNull(dataFile.nanValueCounts()); + assertThat(dataFile.columnSizes()).isNull(); + assertThat(dataFile.valueCounts()).isNull(); + assertThat(dataFile.nullValueCounts()).isNull(); + assertThat(dataFile.nanValueCounts()).isNull(); + assertThat(dataFile.lowerBounds()).isNull(); + assertThat(dataFile.upperBounds()).isNull(); assertNullRecordCount(dataFile); } } - @Test + @TestTemplate public void testReadEntriesWithSelectCertainStatNotProjectStats() throws IOException { ManifestFile manifest = writeManifest(1000L, FILE); try (ManifestReader reader = @@ -199,82 +197,76 @@ public void testReadEntriesWithSelectCertainStatNotProjectStats() throws IOExcep DataFile dataFile = reader.iterator().next(); // selected fields are populated - Assert.assertEquals(VALUE_COUNT, dataFile.valueCounts()); - Assert.assertEquals(FILE_PATH, dataFile.path()); + assertThat(dataFile.path()).isEqualTo(FILE_PATH); + assertThat(dataFile.valueCounts()).isEqualTo(VALUE_COUNT); // not selected fields are all null and not projected - Assert.assertNull(dataFile.columnSizes()); - Assert.assertNull(dataFile.nullValueCounts()); - Assert.assertNull(dataFile.nanValueCounts()); - Assert.assertNull(dataFile.lowerBounds()); - Assert.assertNull(dataFile.upperBounds()); + assertThat(dataFile.columnSizes()).isNull(); + assertThat(dataFile.nullValueCounts()).isNull(); + assertThat(dataFile.nanValueCounts()).isNull(); + assertThat(dataFile.lowerBounds()).isNull(); + assertThat(dataFile.upperBounds()).isNull(); assertNullRecordCount(dataFile); } } private void assertFullStats(DataFile dataFile) { - Assert.assertEquals(3, dataFile.recordCount()); - Assert.assertNull(dataFile.columnSizes()); - Assert.assertEquals(VALUE_COUNT, dataFile.valueCounts()); - Assert.assertEquals(NULL_VALUE_COUNTS, dataFile.nullValueCounts()); - Assert.assertEquals(NAN_VALUE_COUNTS, dataFile.nanValueCounts()); - Assert.assertEquals(LOWER_BOUNDS, dataFile.lowerBounds()); - Assert.assertEquals(UPPER_BOUNDS, dataFile.upperBounds()); + assertThat(dataFile.recordCount()).isEqualTo(3); + assertThat(dataFile.columnSizes()).isNull(); + assertThat(dataFile.valueCounts()).isEqualTo(VALUE_COUNT); + assertThat(dataFile.nullValueCounts()).isEqualTo(NULL_VALUE_COUNTS); + assertThat(dataFile.nanValueCounts()).isEqualTo(NAN_VALUE_COUNTS); + assertThat(dataFile.lowerBounds()).isEqualTo(LOWER_BOUNDS); + assertThat(dataFile.upperBounds()).isEqualTo(UPPER_BOUNDS); if (dataFile.valueCounts() != null) { - Assertions.assertThatThrownBy( - () -> dataFile.valueCounts().clear(), "Should not be modifiable") + assertThatThrownBy(() -> dataFile.valueCounts().clear(), "Should not be modifiable") .isInstanceOf(UnsupportedOperationException.class); } if (dataFile.nullValueCounts() != null) { - Assertions.assertThatThrownBy( - () -> dataFile.nullValueCounts().clear(), "Should not be modifiable") + assertThatThrownBy(() -> dataFile.nullValueCounts().clear(), "Should not be modifiable") .isInstanceOf(UnsupportedOperationException.class); } if (dataFile.nanValueCounts() != null) { - Assertions.assertThatThrownBy( - () -> dataFile.nanValueCounts().clear(), "Should not be modifiable") + assertThatThrownBy(() -> dataFile.nanValueCounts().clear(), "Should not be modifiable") .isInstanceOf(UnsupportedOperationException.class); } if (dataFile.upperBounds() != null) { - Assertions.assertThatThrownBy( - () -> dataFile.upperBounds().clear(), "Should not be modifiable") + assertThatThrownBy(() -> dataFile.upperBounds().clear(), "Should not be modifiable") .isInstanceOf(UnsupportedOperationException.class); } if (dataFile.lowerBounds() != null) { - Assertions.assertThatThrownBy( - () -> dataFile.lowerBounds().clear(), "Should not be modifiable") + assertThatThrownBy(() -> dataFile.lowerBounds().clear(), "Should not be modifiable") .isInstanceOf(UnsupportedOperationException.class); } if (dataFile.columnSizes() != null) { - Assertions.assertThatThrownBy( - () -> dataFile.columnSizes().clear(), "Should not be modifiable") + assertThatThrownBy(() -> dataFile.columnSizes().clear(), "Should not be modifiable") .isInstanceOf(UnsupportedOperationException.class); } - Assert.assertEquals(FILE_PATH, dataFile.path()); // always select file path in all test cases + assertThat(dataFile.path()).isEqualTo(FILE_PATH); // always select file path in all test cases } private void assertStatsDropped(DataFile dataFile) { - Assert.assertEquals( - 3, dataFile.recordCount()); // record count is not considered as droppable stats - Assert.assertNull(dataFile.columnSizes()); - Assert.assertNull(dataFile.valueCounts()); - Assert.assertNull(dataFile.nullValueCounts()); - Assert.assertNull(dataFile.lowerBounds()); - Assert.assertNull(dataFile.upperBounds()); - Assert.assertNull(dataFile.nanValueCounts()); - - Assert.assertEquals(FILE_PATH, dataFile.path()); // always select file path in all test cases + assertThat(dataFile.recordCount()) + .isEqualTo(3); // record count is not considered as droppable stats + assertThat(dataFile.columnSizes()).isNull(); + assertThat(dataFile.valueCounts()).isNull(); + assertThat(dataFile.nullValueCounts()).isNull(); + assertThat(dataFile.nanValueCounts()).isNull(); + assertThat(dataFile.lowerBounds()).isNull(); + assertThat(dataFile.upperBounds()).isNull(); + + assertThat(dataFile.path()).isEqualTo(FILE_PATH); // always select file path in all test cases } private void assertNullRecordCount(DataFile dataFile) { // record count is a primitive type, accessing null record count will throw NPE - Assertions.assertThatThrownBy(dataFile::recordCount).isInstanceOf(NullPointerException.class); + assertThatThrownBy(dataFile::recordCount).isInstanceOf(NullPointerException.class); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index 17a41f418a8e..eb7910a79fc5 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -18,38 +18,33 @@ */ package org.apache.iceberg; +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.io.UncheckedIOException; +import java.util.Arrays; import java.util.List; import java.util.UUID; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.assertj.core.api.Assumptions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestManifestWriter extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestManifestWriter(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestManifestWriter extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 250; private static final long SMALL_FILE_SIZE = 10L; - @Test + @TestTemplate public void testManifestStats() throws IOException { ManifestFile manifest = writeManifest( @@ -64,22 +59,18 @@ public void testManifestStats() throws IOException { manifestEntry(Status.DELETED, null, newFile(5)), manifestEntry(Status.DELETED, null, newFile(2))); - Assert.assertTrue("Added files should be present", manifest.hasAddedFiles()); - Assert.assertEquals("Added files count should match", 4, (int) manifest.addedFilesCount()); - Assert.assertEquals("Added rows count should match", 40L, (long) manifest.addedRowsCount()); - - Assert.assertTrue("Existing files should be present", manifest.hasExistingFiles()); - Assert.assertEquals( - "Existing files count should match", 3, (int) manifest.existingFilesCount()); - Assert.assertEquals( - "Existing rows count should match", 26L, (long) manifest.existingRowsCount()); - - Assert.assertTrue("Deleted files should be present", manifest.hasDeletedFiles()); - Assert.assertEquals("Deleted files count should match", 2, (int) manifest.deletedFilesCount()); - Assert.assertEquals("Deleted rows count should match", 7L, (long) manifest.deletedRowsCount()); + assertThat(manifest.hasAddedFiles()).isTrue(); + assertThat(manifest.addedFilesCount()).isEqualTo(4); + assertThat(manifest.addedRowsCount()).isEqualTo(40); + assertThat(manifest.hasExistingFiles()).isTrue(); + assertThat(manifest.existingFilesCount()).isEqualTo(3); + assertThat(manifest.existingRowsCount()).isEqualTo(26); + assertThat(manifest.hasDeletedFiles()).isTrue(); + assertThat(manifest.deletedFilesCount()).isEqualTo(2); + assertThat(manifest.deletedRowsCount()).isEqualTo(7); } - @Test + @TestTemplate public void testManifestPartitionStats() throws IOException { ManifestFile manifest = writeManifest( @@ -89,48 +80,44 @@ public void testManifestPartitionStats() throws IOException { manifestEntry(Status.DELETED, null, newFile(2, TestHelpers.Row.of(3)))); List partitions = manifest.partitions(); - Assert.assertEquals("Partition field summaries count should match", 1, partitions.size()); + assertThat(partitions).hasSize(1); ManifestFile.PartitionFieldSummary partitionFieldSummary = partitions.get(0); - Assert.assertFalse("contains_null should be false", partitionFieldSummary.containsNull()); - Assert.assertFalse("contains_nan should be false", partitionFieldSummary.containsNaN()); - Assert.assertEquals( - "Lower bound should match", - Integer.valueOf(1), - Conversions.fromByteBuffer(Types.IntegerType.get(), partitionFieldSummary.lowerBound())); - Assert.assertEquals( - "Upper bound should match", - Integer.valueOf(3), - Conversions.fromByteBuffer(Types.IntegerType.get(), partitionFieldSummary.upperBound())); + assertThat(partitionFieldSummary.containsNull()).isFalse(); + assertThat(partitionFieldSummary.containsNaN()).isFalse(); + assertThat( + (Integer) + Conversions.fromByteBuffer( + Types.IntegerType.get(), partitionFieldSummary.lowerBound())) + .isEqualTo(1); + assertThat( + (Integer) + Conversions.fromByteBuffer( + Types.IntegerType.get(), partitionFieldSummary.upperBound())) + .isEqualTo(3); } - @Test + @TestTemplate public void testWriteManifestWithSequenceNumber() throws IOException { - Assume.assumeTrue("sequence number is only valid for format version > 1", formatVersion > 1); - File manifestFile = temp.newFile("manifest.avro"); - Assert.assertTrue(manifestFile.delete()); + assumeThat(formatVersion).isGreaterThan(1); + File manifestFile = File.createTempFile("manifest", ".avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = ManifestFiles.write(formatVersion, table.spec(), outputFile, 1L); writer.add(newFile(10, TestHelpers.Row.of(1)), 1000L); writer.close(); ManifestFile manifest = writer.toManifestFile(); - Assert.assertEquals("Manifest should have no sequence number", -1L, manifest.sequenceNumber()); + assertThat(manifest.sequenceNumber()).isEqualTo(-1); ManifestReader manifestReader = ManifestFiles.read(manifest, table.io()); for (ManifestEntry entry : manifestReader.entries()) { - Assert.assertEquals( - "Custom data sequence number should be used for all manifest entries", - 1000L, - (long) entry.dataSequenceNumber()); - Assert.assertEquals( - "File sequence number must be unassigned", - ManifestWriter.UNASSIGNED_SEQ, - entry.fileSequenceNumber().longValue()); + assertThat(entry.dataSequenceNumber()).isEqualTo(1000); + assertThat(entry.fileSequenceNumber()).isEqualTo(ManifestWriter.UNASSIGNED_SEQ); } } - @Test + @TestTemplate public void testCommitManifestWithExplicitDataSequenceNumber() throws IOException { - Assume.assumeTrue("Sequence numbers are valid for format version > 1", formatVersion > 1); + assumeThat(formatVersion).isGreaterThan(1); DataFile file1 = newFile(50); DataFile file2 = newFile(50); @@ -143,10 +130,7 @@ public void testCommitManifestWithExplicitDataSequenceNumber() throws IOExceptio manifestEntry(Status.ADDED, null, dataSequenceNumber, null, file1), manifestEntry(Status.ADDED, null, dataSequenceNumber, null, file2)); - Assert.assertEquals( - "Manifest should have no sequence number before commit", - ManifestWriter.UNASSIGNED_SEQ, - manifest.sequenceNumber()); + assertThat(manifest.sequenceNumber()).isEqualTo(ManifestWriter.UNASSIGNED_SEQ); table.newFastAppend().appendManifest(manifest).commit(); @@ -154,15 +138,9 @@ public void testCommitManifestWithExplicitDataSequenceNumber() throws IOExceptio ManifestFile committedManifest = table.currentSnapshot().dataManifests(table.io()).get(0); - Assert.assertEquals( - "Committed manifest sequence number must be correct", - 1L, - committedManifest.sequenceNumber()); + assertThat(committedManifest.sequenceNumber()).isEqualTo(1); - Assert.assertEquals( - "Committed manifest min sequence number must be correct", - dataSequenceNumber, - committedManifest.minSequenceNumber()); + assertThat(committedManifest.minSequenceNumber()).isEqualTo(dataSequenceNumber); validateManifest( committedManifest, @@ -173,9 +151,9 @@ public void testCommitManifestWithExplicitDataSequenceNumber() throws IOExceptio statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() throws IOException { - Assume.assumeTrue("Sequence numbers are valid for format version > 1", formatVersion > 1); + assumeThat(formatVersion).isGreaterThan(1); DataFile file1 = newFile(50); DataFile file2 = newFile(50); @@ -194,10 +172,7 @@ public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() thr manifestEntry(Status.EXISTING, appendSnapshotId, appendSequenceNumber, null, file1), manifestEntry(Status.EXISTING, appendSnapshotId, appendSequenceNumber, null, file2)); - Assert.assertEquals( - "Manifest should have no sequence number before commit", - ManifestWriter.UNASSIGNED_SEQ, - newManifest.sequenceNumber()); + assertThat(newManifest.sequenceNumber()).isEqualTo(ManifestWriter.UNASSIGNED_SEQ); table.rewriteManifests().deleteManifest(originalManifest).addManifest(newManifest).commit(); @@ -205,15 +180,9 @@ public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() thr ManifestFile committedManifest = table.currentSnapshot().dataManifests(table.io()).get(0); - Assert.assertEquals( - "Committed manifest sequence number must be correct", - rewriteSnapshot.sequenceNumber(), - committedManifest.sequenceNumber()); + assertThat(committedManifest.sequenceNumber()).isEqualTo(rewriteSnapshot.sequenceNumber()); - Assert.assertEquals( - "Committed manifest min sequence number must be correct", - appendSequenceNumber, - committedManifest.minSequenceNumber()); + assertThat(committedManifest.minSequenceNumber()).isEqualTo(appendSequenceNumber); validateManifest( committedManifest, @@ -224,30 +193,30 @@ public void testCommitManifestWithExistingEntriesWithoutFileSequenceNumber() thr statuses(Status.EXISTING, Status.EXISTING)); } - @Test + @TestTemplate public void testRollingManifestWriterNoRecords() throws IOException { RollingManifestWriter writer = newRollingWriteManifest(SMALL_FILE_SIZE); writer.close(); - Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + assertThat(writer.toManifestFiles()).isEmpty(); writer.close(); - Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + assertThat(writer.toManifestFiles()).isEmpty(); } - @Test + @TestTemplate public void testRollingDeleteManifestWriterNoRecords() throws IOException { - Assumptions.assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isGreaterThan(1); RollingManifestWriter writer = newRollingWriteDeleteManifest(SMALL_FILE_SIZE); writer.close(); - Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + assertThat(writer.toManifestFiles()).isEmpty(); writer.close(); - Assertions.assertThat(writer.toManifestFiles()).isEmpty(); + assertThat(writer.toManifestFiles()).isEmpty(); } - @Test + @TestTemplate public void testRollingManifestWriterSplitFiles() throws IOException { RollingManifestWriter writer = newRollingWriteManifest(SMALL_FILE_SIZE); @@ -278,7 +247,7 @@ public void testRollingManifestWriterSplitFiles() throws IOException { writer.close(); List manifestFiles = writer.toManifestFiles(); - Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + assertThat(manifestFiles).hasSize(3); checkManifests( manifestFiles, @@ -291,7 +260,7 @@ public void testRollingManifestWriterSplitFiles() throws IOException { writer.close(); manifestFiles = writer.toManifestFiles(); - Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + assertThat(manifestFiles).hasSize(3); checkManifests( manifestFiles, @@ -303,9 +272,9 @@ public void testRollingManifestWriterSplitFiles() throws IOException { deletedRowCounts); } - @Test + @TestTemplate public void testRollingDeleteManifestWriterSplitFiles() throws IOException { - Assumptions.assumeThat(formatVersion).isGreaterThan(1); + assumeThat(formatVersion).isGreaterThan(1); RollingManifestWriter writer = newRollingWriteDeleteManifest(SMALL_FILE_SIZE); int[] addedFileCounts = new int[3]; @@ -334,7 +303,7 @@ public void testRollingDeleteManifestWriterSplitFiles() throws IOException { writer.close(); List manifestFiles = writer.toManifestFiles(); - Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + assertThat(manifestFiles).hasSize(3); checkManifests( manifestFiles, @@ -347,7 +316,7 @@ public void testRollingDeleteManifestWriterSplitFiles() throws IOException { writer.close(); manifestFiles = writer.toManifestFiles(); - Assertions.assertThat(manifestFiles.size()).isEqualTo(3); + assertThat(manifestFiles).hasSize(3); checkManifests( manifestFiles, @@ -370,17 +339,17 @@ private void checkManifests( for (int i = 0; i < manifests.size(); i++) { ManifestFile manifest = manifests.get(i); - Assertions.assertThat(manifest.hasAddedFiles()).isTrue(); - Assertions.assertThat(manifest.addedFilesCount()).isEqualTo(addedFileCounts[i]); - Assertions.assertThat(manifest.addedRowsCount()).isEqualTo(addedRowCounts[i]); + assertThat(manifest.hasAddedFiles()).isTrue(); + assertThat(manifest.addedFilesCount()).isEqualTo(addedFileCounts[i]); + assertThat(manifest.addedRowsCount()).isEqualTo(addedRowCounts[i]); - Assertions.assertThat(manifest.hasExistingFiles()).isTrue(); - Assertions.assertThat(manifest.existingFilesCount()).isEqualTo(existingFileCounts[i]); - Assertions.assertThat(manifest.existingRowsCount()).isEqualTo(existingRowCounts[i]); + assertThat(manifest.hasExistingFiles()).isTrue(); + assertThat(manifest.existingFilesCount()).isEqualTo(existingFileCounts[i]); + assertThat(manifest.existingRowsCount()).isEqualTo(existingRowCounts[i]); - Assertions.assertThat(manifest.hasDeletedFiles()).isTrue(); - Assertions.assertThat(manifest.deletedFilesCount()).isEqualTo(deletedFileCounts[i]); - Assertions.assertThat(manifest.deletedRowsCount()).isEqualTo(deletedRowCounts[i]); + assertThat(manifest.hasDeletedFiles()).isTrue(); + assertThat(manifest.deletedFilesCount()).isEqualTo(deletedFileCounts[i]); + assertThat(manifest.deletedRowsCount()).isEqualTo(deletedRowCounts[i]); } } @@ -430,7 +399,9 @@ private RollingManifestWriter newRollingWriteDeleteManifest(long tar private OutputFile newManifestFile() { try { - return Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + return Files.localOutput( + FileFormat.AVRO.addExtension( + File.createTempFile("manifest", null, temp.toFile()).toString())); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java index 740791b255d5..1d5c34fa4b16 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriterVersions.java @@ -19,8 +19,12 @@ package org.apache.iceberg; 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; +import java.nio.file.Path; import java.util.List; import org.apache.iceberg.inmemory.InMemoryOutputFile; import org.apache.iceberg.io.CloseableIterable; @@ -33,11 +37,8 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestManifestWriterVersions { private static final FileIO FILE_IO = new TestTables.LocalFileIO(); @@ -97,7 +98,7 @@ public class TestManifestWriterVersions { null, null); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testV1Write() throws IOException { @@ -112,7 +113,7 @@ public void testV1Write() throws IOException { @Test public void testV1WriteDelete() { - Assertions.assertThatThrownBy(() -> writeDeleteManifest(1)) + assertThatThrownBy(() -> writeDeleteManifest(1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot write delete files in a v1 table"); } @@ -130,7 +131,7 @@ public void testV1WriteWithInheritance() throws IOException { public void testV2Write() throws IOException { ManifestFile manifest = writeManifest(2); checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ); - Assert.assertEquals("Content", ManifestContent.DATA, manifest.content()); + assertThat(manifest.content()).isEqualTo(ManifestContent.DATA); checkEntry( readManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, @@ -142,7 +143,7 @@ public void testV2Write() throws IOException { public void testV2WriteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeManifest(2), 2); checkManifest(manifest, SEQUENCE_NUMBER); - Assert.assertEquals("Content", ManifestContent.DATA, manifest.content()); + assertThat(manifest.content()).isEqualTo(ManifestContent.DATA); // v2 should use the correct sequence number by inheriting it checkEntry(readManifest(manifest), SEQUENCE_NUMBER, SEQUENCE_NUMBER, FileContent.DATA); @@ -152,7 +153,7 @@ public void testV2WriteWithInheritance() throws IOException { public void testV2WriteDelete() throws IOException { ManifestFile manifest = writeDeleteManifest(2); checkManifest(manifest, ManifestWriter.UNASSIGNED_SEQ); - Assert.assertEquals("Content", ManifestContent.DELETES, manifest.content()); + assertThat(manifest.content()).isEqualTo(ManifestContent.DELETES); checkEntry( readDeleteManifest(manifest), ManifestWriter.UNASSIGNED_SEQ, @@ -164,7 +165,7 @@ public void testV2WriteDelete() throws IOException { public void testV2WriteDeleteWithInheritance() throws IOException { ManifestFile manifest = writeAndReadManifestList(writeDeleteManifest(2), 2); checkManifest(manifest, SEQUENCE_NUMBER); - Assert.assertEquals("Content", ManifestContent.DELETES, manifest.content()); + assertThat(manifest.content()).isEqualTo(ManifestContent.DELETES); // v2 should use the correct sequence number by inheriting it checkEntry( @@ -214,69 +215,65 @@ void checkEntry( Long expectedDataSequenceNumber, Long expectedFileSequenceNumber, FileContent content) { - Assert.assertEquals("Status", ManifestEntry.Status.ADDED, entry.status()); - Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, entry.snapshotId()); - Assert.assertEquals( - "Data sequence number", expectedDataSequenceNumber, entry.dataSequenceNumber()); - Assert.assertEquals( - "File sequence number", expectedFileSequenceNumber, entry.fileSequenceNumber()); + assertThat(entry.status()).isEqualTo(ManifestEntry.Status.ADDED); + assertThat(entry.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(entry.dataSequenceNumber()).isEqualTo(expectedDataSequenceNumber); + assertThat(entry.fileSequenceNumber()).isEqualTo(expectedFileSequenceNumber); checkDataFile(entry.file(), content); } void checkRewrittenEntry( ManifestEntry entry, Long expectedSequenceNumber, FileContent content) { - Assert.assertEquals("Status", ManifestEntry.Status.EXISTING, entry.status()); - Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, entry.snapshotId()); - Assert.assertEquals("Data sequence number", expectedSequenceNumber, entry.dataSequenceNumber()); + assertThat(entry.status()).isEqualTo(ManifestEntry.Status.EXISTING); + assertThat(entry.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(entry.dataSequenceNumber()).isEqualTo(expectedSequenceNumber); checkDataFile(entry.file(), content); } void checkDataFile(ContentFile dataFile, FileContent content) { // DataFile is the superclass of DeleteFile, so this method can check both - Assert.assertEquals("Content", content, dataFile.content()); - Assert.assertEquals("Path", PATH, dataFile.path()); - Assert.assertEquals("Format", FORMAT, dataFile.format()); - Assert.assertEquals("Partition", PARTITION, dataFile.partition()); - Assert.assertEquals("Record count", METRICS.recordCount(), (Long) dataFile.recordCount()); - Assert.assertEquals("Column sizes", METRICS.columnSizes(), dataFile.columnSizes()); - Assert.assertEquals("Value counts", METRICS.valueCounts(), dataFile.valueCounts()); - Assert.assertEquals("Null value counts", METRICS.nullValueCounts(), dataFile.nullValueCounts()); - Assert.assertEquals("NaN value counts", METRICS.nanValueCounts(), dataFile.nanValueCounts()); - Assert.assertEquals("Lower bounds", METRICS.lowerBounds(), dataFile.lowerBounds()); - Assert.assertEquals("Upper bounds", METRICS.upperBounds(), dataFile.upperBounds()); - Assert.assertEquals("Sort order id", SORT_ORDER_ID, dataFile.sortOrderId()); + assertThat(dataFile.content()).isEqualTo(content); + assertThat(dataFile.path()).isEqualTo(PATH); + assertThat(dataFile.format()).isEqualTo(FORMAT); + assertThat(dataFile.partition()).isEqualTo(PARTITION); + assertThat(dataFile.recordCount()).isEqualTo(METRICS.recordCount()); + assertThat(dataFile.columnSizes()).isEqualTo(METRICS.columnSizes()); + assertThat(dataFile.valueCounts()).isEqualTo(METRICS.valueCounts()); + assertThat(dataFile.nullValueCounts()).isEqualTo(METRICS.nullValueCounts()); + assertThat(dataFile.nanValueCounts()).isEqualTo(METRICS.nanValueCounts()); + assertThat(dataFile.lowerBounds()).isEqualTo(METRICS.lowerBounds()); + assertThat(dataFile.upperBounds()).isEqualTo(METRICS.upperBounds()); + assertThat(dataFile.sortOrderId()).isEqualTo(SORT_ORDER_ID); if (dataFile.content() == FileContent.EQUALITY_DELETES) { - Assert.assertEquals(EQUALITY_IDS, dataFile.equalityFieldIds()); + assertThat(dataFile.equalityFieldIds()).isEqualTo(EQUALITY_IDS); } else { - Assert.assertNull(dataFile.equalityFieldIds()); + assertThat(dataFile.equalityFieldIds()).isNull(); } } void checkManifest(ManifestFile manifest, long expectedSequenceNumber) { - Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, manifest.snapshotId()); - Assert.assertEquals("Sequence number", expectedSequenceNumber, manifest.sequenceNumber()); - Assert.assertEquals( - "Min sequence number", expectedSequenceNumber, manifest.minSequenceNumber()); - Assert.assertEquals("Added files count", (Integer) 1, manifest.addedFilesCount()); - Assert.assertEquals("Existing files count", (Integer) 0, manifest.existingFilesCount()); - Assert.assertEquals("Deleted files count", (Integer) 0, manifest.deletedFilesCount()); - Assert.assertEquals("Added rows count", METRICS.recordCount(), manifest.addedRowsCount()); - Assert.assertEquals("Existing rows count", (Long) 0L, manifest.existingRowsCount()); - Assert.assertEquals("Deleted rows count", (Long) 0L, manifest.deletedRowsCount()); + assertThat(manifest.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(manifest.sequenceNumber()).isEqualTo(expectedSequenceNumber); + assertThat(manifest.minSequenceNumber()).isEqualTo(expectedSequenceNumber); + assertThat(manifest.addedFilesCount()).isEqualTo(1); + assertThat(manifest.existingFilesCount()).isEqualTo(0); + assertThat(manifest.deletedFilesCount()).isEqualTo(0); + assertThat(manifest.addedRowsCount()).isEqualTo(METRICS.recordCount()); + assertThat(manifest.existingRowsCount()).isEqualTo(0); + assertThat(manifest.deletedRowsCount()).isEqualTo(0); } void checkRewrittenManifest( ManifestFile manifest, long expectedSequenceNumber, long expectedMinSequenceNumber) { - Assert.assertEquals("Snapshot ID", (Long) SNAPSHOT_ID, manifest.snapshotId()); - Assert.assertEquals("Sequence number", expectedSequenceNumber, manifest.sequenceNumber()); - Assert.assertEquals( - "Min sequence number", expectedMinSequenceNumber, manifest.minSequenceNumber()); - Assert.assertEquals("Added files count", (Integer) 0, manifest.addedFilesCount()); - Assert.assertEquals("Existing files count", (Integer) 1, manifest.existingFilesCount()); - Assert.assertEquals("Deleted files count", (Integer) 0, manifest.deletedFilesCount()); - Assert.assertEquals("Added rows count", (Long) 0L, manifest.addedRowsCount()); - Assert.assertEquals("Existing rows count", METRICS.recordCount(), manifest.existingRowsCount()); - Assert.assertEquals("Deleted rows count", (Long) 0L, manifest.deletedRowsCount()); + assertThat(manifest.snapshotId()).isEqualTo(SNAPSHOT_ID); + assertThat(manifest.sequenceNumber()).isEqualTo(expectedSequenceNumber); + assertThat(manifest.minSequenceNumber()).isEqualTo(expectedMinSequenceNumber); + assertThat(manifest.addedFilesCount()).isEqualTo(0); + assertThat(manifest.existingFilesCount()).isEqualTo(1); + assertThat(manifest.deletedFilesCount()).isEqualTo(0); + assertThat(manifest.addedRowsCount()).isEqualTo(0); + assertThat(manifest.existingRowsCount()).isEqualTo(METRICS.recordCount()); + assertThat(manifest.deletedRowsCount()).isEqualTo(0); } private InputFile writeManifestList(ManifestFile manifest, int formatVersion) throws IOException { @@ -296,14 +293,16 @@ private InputFile writeManifestList(ManifestFile manifest, int formatVersion) th private ManifestFile writeAndReadManifestList(ManifestFile manifest, int formatVersion) throws IOException { List manifests = ManifestLists.read(writeManifestList(manifest, formatVersion)); - Assert.assertEquals("Should contain one manifest", 1, manifests.size()); + assertThat(manifests).hasSize(1); return manifests.get(0); } private ManifestFile rewriteManifest(ManifestFile manifest, int formatVersion) throws IOException { OutputFile manifestFile = - Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + Files.localOutput( + FileFormat.AVRO.addExtension( + File.createTempFile("manifest", null, temp.toFile()).toString())); ManifestWriter writer = ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); try { @@ -320,7 +319,9 @@ private ManifestFile writeManifest(int formatVersion) throws IOException { private ManifestFile writeManifest(DataFile file, int formatVersion) throws IOException { OutputFile manifestFile = - Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + Files.localOutput( + FileFormat.AVRO.addExtension( + File.createTempFile("manifest", null, temp.toFile()).toString())); ManifestWriter writer = ManifestFiles.write(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); try { @@ -335,14 +336,16 @@ private ManifestEntry readManifest(ManifestFile manifest) throws IOExc try (CloseableIterable> reader = ManifestFiles.read(manifest, FILE_IO).entries()) { List> files = Lists.newArrayList(reader); - Assert.assertEquals("Should contain only one data file", 1, files.size()); + assertThat(files).hasSize(1); return files.get(0); } } private ManifestFile writeDeleteManifest(int formatVersion) throws IOException { OutputFile manifestFile = - Files.localOutput(FileFormat.AVRO.addExtension(temp.newFile().toString())); + Files.localOutput( + FileFormat.AVRO.addExtension( + File.createTempFile("manifest", null, temp.toFile()).toString())); ManifestWriter writer = ManifestFiles.writeDeleteManifest(formatVersion, SPEC, manifestFile, SNAPSHOT_ID); try { @@ -357,7 +360,7 @@ private ManifestEntry readDeleteManifest(ManifestFile manifest) thro try (CloseableIterable> reader = ManifestFiles.readDeleteManifest(manifest, FILE_IO, null).entries()) { List> entries = Lists.newArrayList(reader); - Assert.assertEquals("Should contain only one data file", 1, entries.size()); + assertThat(entries).hasSize(1); return entries.get(0); } } From 47afbb2e0125b0a62573ed8e7114f3486ac28799 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 11:02:52 +0100 Subject: [PATCH 0142/1019] Build: Bump spring-boot from 2.5.4 to 2.7.18 (#9985) Bumps `spring-boot` from 2.5.4 to 2.7.18. Updates `org.springframework.boot:spring-boot-starter-jetty` from 2.5.4 to 2.7.18 - [Release notes](https://github.com/spring-projects/spring-boot/releases) - [Commits](https://github.com/spring-projects/spring-boot/compare/v2.5.4...v2.7.18) Updates `org.springframework.boot:spring-boot-starter-web` from 2.5.4 to 2.7.18 - [Release notes](https://github.com/spring-projects/spring-boot/releases) - [Commits](https://github.com/spring-projects/spring-boot/compare/v2.5.4...v2.7.18) --- updated-dependencies: - dependency-name: org.springframework.boot:spring-boot-starter-jetty dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: org.springframework.boot:spring-boot-starter-web 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 26d0a79e57b0..ec7e91afe2ea 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -80,7 +80,7 @@ snowflake-jdbc = "3.14.5" spark-hive33 = "3.3.4" spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" -spring-boot = "2.5.4" +spring-boot = "2.7.18" spring-web = "5.3.30" sqlite-jdbc = "3.45.1.0" testcontainers = "1.19.5" From 2c2128757881ccbac8bac8756468480bb8d87d3c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 11:53:19 +0100 Subject: [PATCH 0143/1019] Build: Bump org.springframework:spring-web from 5.3.30 to 5.3.33 (#9989) Bumps [org.springframework:spring-web](https://github.com/spring-projects/spring-framework) from 5.3.30 to 5.3.33. - [Release notes](https://github.com/spring-projects/spring-framework/releases) - [Commits](https://github.com/spring-projects/spring-framework/compare/v5.3.30...v5.3.33) --- updated-dependencies: - dependency-name: org.springframework:spring-web 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 ec7e91afe2ea..17693a69d27b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -81,7 +81,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.2" spark-hive35 = "3.5.0" spring-boot = "2.7.18" -spring-web = "5.3.30" +spring-web = "5.3.33" sqlite-jdbc = "3.45.1.0" testcontainers = "1.19.5" tez010 = "0.10.3" From 0a9fb2e272c01a7ea97ec62edf0469cd6dee6112 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 11:53:56 +0100 Subject: [PATCH 0144/1019] Build: Bump jetty from 9.4.53.v20231009 to 9.4.54.v20240208 (#9982) Bumps `jetty` from 9.4.53.v20231009 to 9.4.54.v20240208. Updates `org.eclipse.jetty:jetty-server` from 9.4.53.v20231009 to 9.4.54.v20240208 Updates `org.eclipse.jetty:jetty-servlet` from 9.4.53.v20231009 to 9.4.54.v20240208 --- updated-dependencies: - dependency-name: org.eclipse.jetty:jetty-server dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.eclipse.jetty:jetty-servlet 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 17693a69d27b..d1a26280ca37 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -58,7 +58,7 @@ jackson215 = { strictly = "2.15.2"} jakarta-el-api = "3.0.3" jaxb-api = "2.3.1" jaxb-runtime = "2.3.3" -jetty = "9.4.53.v20231009" +jetty = "9.4.54.v20240208" junit = "5.10.1" kafka = "3.6.1" kryo-shaded = "4.0.3" From e3035a14208954eaf5296d37f2f81e51ccd69fb4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 18 Mar 2024 14:13:57 +0100 Subject: [PATCH 0145/1019] Build: Bump guava from 33.0.0-jre to 33.1.0-jre (#9977) Bumps `guava` from 33.0.0-jre to 33.1.0-jre. Updates `com.google.guava:guava` from 33.0.0-jre to 33.1.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.0.0-jre to 33.1.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 d1a26280ca37..34e9fc427ca9 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -42,7 +42,7 @@ flink116 = { strictly = "1.16.3"} flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} google-libraries-bom = "26.28.0" -guava = "33.0.0-jre" +guava = "33.1.0-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" httpcomponents-httpclient5 = "5.3.1" From 8931563060470b54982144dd767644dce7076608 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 18 Mar 2024 16:42:14 +0100 Subject: [PATCH 0146/1019] API: Fix `TestStrictMetricsEvaluator` assertion message (#9992) All values (`5` and `6`) are not between the upper and lower bound of `[30, 79]`. --- .../apache/iceberg/expressions/TestStrictMetricsEvaluator.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java index 2266389abab0..82aaff02149a 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -592,7 +592,7 @@ public void testIntegerNotIn() { boolean shouldRead = new StrictMetricsEvaluator(SCHEMA, notIn("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)) .eval(FILE); - assertThat(shouldRead).as("Should not match: all values !=5 and !=6").isTrue(); + assertThat(shouldRead).as("Should match: all values !=5 and !=6").isTrue(); shouldRead = new StrictMetricsEvaluator(SCHEMA, notIn("id", INT_MIN_VALUE - 1, INT_MIN_VALUE)) From 4010d2f722bb20124b964fdc508887370df5a2dd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 19 Mar 2024 08:58:10 +0100 Subject: [PATCH 0147/1019] Build: Bump arrow from 15.0.0 to 15.0.1 (#9910) Bumps `arrow` from 15.0.0 to 15.0.1. Updates `org.apache.arrow:arrow-memory-netty` from 15.0.0 to 15.0.1 Updates `org.apache.arrow:arrow-vector` from 15.0.0 to 15.0.1 - [Commits](https://github.com/apache/arrow/compare/go/v15.0.0...go/v15.0.1) --- updated-dependencies: - dependency-name: org.apache.arrow:arrow-memory-netty dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.arrow:arrow-vector 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 34e9fc427ca9..ce571331529f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -24,7 +24,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.26" -arrow = "15.0.0" +arrow = "15.0.1" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" From c54e58393d87ded02858ff7f0d9c82a037d3564c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 19 Mar 2024 17:05:41 +0100 Subject: [PATCH 0148/1019] AWS, Core: Replace .withFailMessage() usage with .as() (#10000) We almost never want to use `.withFailMessage()` as that will override any enriched contextual information that AssertJ generally provides about actual/expected. It's better to use `.as()` to add some description to the check being done, which allows to still show contextual information about actual/expected if the assertion ever fails. --- .../iceberg/aws/AwsClientPropertiesTest.java | 33 +++++++++---------- .../iceberg/aws/HttpClientPropertiesTest.java | 12 +++---- .../aws/TestS3FileIOAwsClientFactories.java | 4 +-- .../aws/s3/TestS3FileIOProperties.java | 6 ++-- .../apache/iceberg/catalog/CatalogTests.java | 22 ++++++------- 5 files changed, 36 insertions(+), 41 deletions(-) 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 7f57fbcb2d10..fd3bea68088b 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java @@ -46,7 +46,7 @@ public void testApplyClientRegion() { Mockito.verify(mockS3ClientBuilder).region(regionArgumentCaptor.capture()); Region region = regionArgumentCaptor.getValue(); Assertions.assertThat(region.id()) - .withFailMessage("region parameter should match what is set in CLIENT_REGION") + .as("region parameter should match what is set in CLIENT_REGION") .isEqualTo("us-east-1"); } @@ -56,9 +56,9 @@ public void testDefaultCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider = awsClientProperties.credentialsProvider(null, null, null); - Assertions.assertThat(credentialsProvider instanceof DefaultCredentialsProvider) - .withFailMessage("Should use default credentials if nothing is set") - .isTrue(); + Assertions.assertThat(credentialsProvider) + .as("Should use default credentials if nothing is set") + .isInstanceOf(DefaultCredentialsProvider.class); } @Test @@ -70,7 +70,7 @@ public void testCreatesNewInstanceOfDefaultCredentialsConfiguration() { awsClientProperties.credentialsProvider(null, null, null); Assertions.assertThat(credentialsProvider) - .withFailMessage("Should create a new instance in each call") + .as("Should create a new instance in each call") .isNotSameAs(credentialsProvider2); } @@ -81,17 +81,15 @@ public void testBasicCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider = awsClientProperties.credentialsProvider("key", "secret", null); - Assertions.assertThat(credentialsProvider.resolveCredentials() instanceof AwsBasicCredentials) - .withFailMessage( - "Should use basic credentials if access key ID and secret access key are set") - .isTrue(); + Assertions.assertThat(credentialsProvider.resolveCredentials()) + .as("Should use basic credentials if access key ID and secret access key are set") + .isInstanceOf(AwsBasicCredentials.class); Assertions.assertThat(credentialsProvider.resolveCredentials().accessKeyId()) - .withFailMessage("The access key id should be the same as the one set by tag ACCESS_KEY_ID") + .as("The access key id should be the same as the one set by tag ACCESS_KEY_ID") .isEqualTo("key"); Assertions.assertThat(credentialsProvider.resolveCredentials().secretAccessKey()) - .withFailMessage( - "The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") + .as("The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") .isEqualTo("secret"); } @@ -102,15 +100,14 @@ public void testSessionCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider = awsClientProperties.credentialsProvider("key", "secret", "token"); - Assertions.assertThat(credentialsProvider.resolveCredentials() instanceof AwsSessionCredentials) - .withFailMessage("Should use session credentials if session token is set") - .isTrue(); + Assertions.assertThat(credentialsProvider.resolveCredentials()) + .as("Should use session credentials if session token is set") + .isInstanceOf(AwsSessionCredentials.class); Assertions.assertThat(credentialsProvider.resolveCredentials().accessKeyId()) - .withFailMessage("The access key id should be the same as the one set by tag ACCESS_KEY_ID") + .as("The access key id should be the same as the one set by tag ACCESS_KEY_ID") .isEqualTo("key"); Assertions.assertThat(credentialsProvider.resolveCredentials().secretAccessKey()) - .withFailMessage( - "The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") + .as("The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") .isEqualTo("secret"); } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java b/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java index 270c4c1dc090..7c6e5c19256d 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java @@ -45,9 +45,9 @@ public void testUrlHttpClientConfiguration() { Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assertions.assertThat(capturedHttpClientBuilder instanceof UrlConnectionHttpClient.Builder) - .withFailMessage("Should use url connection http client") - .isTrue(); + Assertions.assertThat(capturedHttpClientBuilder) + .as("Should use url connection http client") + .isInstanceOf(UrlConnectionHttpClient.Builder.class); } @Test @@ -62,9 +62,9 @@ public void testApacheHttpClientConfiguration() { httpClientProperties.applyHttpClientConfigurations(mockS3ClientBuilder); Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assertions.assertThat(capturedHttpClientBuilder instanceof ApacheHttpClient.Builder) - .withFailMessage("Should use apache http client") - .isTrue(); + Assertions.assertThat(capturedHttpClientBuilder) + .as("Should use apache http client") + .isInstanceOf(ApacheHttpClient.Builder.class); } @Test diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java index f0bf9f7067e3..f09d4e543f17 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java @@ -35,7 +35,7 @@ public void testS3FileIOImplCatalogPropertyDefined() { "org.apache.iceberg.aws.s3.DefaultS3FileIOAwsClientFactory"); Object factoryImpl = S3FileIOAwsClientFactories.initialize(properties); Assertions.assertThat(factoryImpl) - .withFailMessage( + .as( "should instantiate an object of type S3FileIOAwsClientFactory when s3.client-factory-impl is set") .isInstanceOf(S3FileIOAwsClientFactory.class); } @@ -46,7 +46,7 @@ public void testS3FileIOImplCatalogPropertyNotDefined() { Map properties = Maps.newHashMap(); Object factoryImpl = S3FileIOAwsClientFactories.initialize(properties); Assertions.assertThat(factoryImpl) - .withFailMessage( + .as( "should instantiate an object of type AwsClientFactory when s3.client-factory-impl is not set") .isInstanceOf(AwsClientFactory.class); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index 8dda5e16338e..2ed8a9471d66 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -447,13 +447,13 @@ public void testApplyS3ServiceConfigurations() { S3Configuration s3Configuration = s3ConfigurationCaptor.getValue(); Assertions.assertThat(s3Configuration.pathStyleAccessEnabled()) - .withFailMessage("s3 path style access enabled parameter should be set to true") + .as("s3 path style access enabled parameter should be set to true") .isTrue(); Assertions.assertThat(s3Configuration.useArnRegionEnabled()) - .withFailMessage("s3 use arn region enabled parameter should be set to true") + .as("s3 use arn region enabled parameter should be set to true") .isTrue(); Assertions.assertThat(s3Configuration.accelerateModeEnabled()) - .withFailMessage("s3 acceleration mode enabled parameter should be set to true") + .as("s3 acceleration mode enabled parameter should be set to true") .isFalse(); } 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 d6b9f6d12074..6904449cc4c3 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -380,17 +380,17 @@ public void testListNamespaces() { catalog.createNamespace(ns1); Assertions.assertThat(catalog.listNamespaces()) - .withFailMessage("Should include newdb_1") + .as("Should include newdb_1") .hasSameElementsAs(concat(starting, ns1)); catalog.createNamespace(ns2); Assertions.assertThat(catalog.listNamespaces()) - .withFailMessage("Should include newdb_1 and newdb_2") + .as("Should include newdb_1 and newdb_2") .hasSameElementsAs(concat(starting, ns1, ns2)); catalog.dropNamespace(ns1); Assertions.assertThat(catalog.listNamespaces()) - .withFailMessage("Should include newdb_2, not newdb_1") + .as("Should include newdb_2, not newdb_1") .hasSameElementsAs(concat(starting, ns2)); catalog.dropNamespace(ns2); @@ -415,36 +415,34 @@ public void testListNestedNamespaces() { catalog.createNamespace(parent); Assertions.assertThat(catalog.listNamespaces()) - .withFailMessage("Should include parent") + .as("Should include parent") .hasSameElementsAs(concat(starting, parent)); Assertions.assertThat(catalog.listNamespaces(parent)) - .withFailMessage("Should have no children in newly created parent namespace") + .as("Should have no children in newly created parent namespace") .isEmpty(); catalog.createNamespace(child1); Assertions.assertThat(catalog.listNamespaces(parent)) - .withFailMessage("Should include child1") + .as("Should include child1") .hasSameElementsAs(ImmutableList.of(child1)); catalog.createNamespace(child2); Assertions.assertThat(catalog.listNamespaces(parent)) - .withFailMessage("Should include child1 and child2") + .as("Should include child1 and child2") .hasSameElementsAs(ImmutableList.of(child1, child2)); Assertions.assertThat(catalog.listNamespaces()) - .withFailMessage("Should not change listing the root") + .as("Should not change listing the root") .hasSameElementsAs(concat(starting, parent)); catalog.dropNamespace(child1); Assertions.assertThat(catalog.listNamespaces(parent)) - .withFailMessage("Should include only child2") + .as("Should include only child2") .hasSameElementsAs(ImmutableList.of(child2)); catalog.dropNamespace(child2); - Assertions.assertThat(catalog.listNamespaces(parent)) - .withFailMessage("Should be empty") - .isEmpty(); + Assertions.assertThat(catalog.listNamespaces(parent)).as("Should be empty").isEmpty(); } @Test From 72fae6f8e8a7738fbe01a581e5a9fc6d2862a983 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 20 Mar 2024 16:06:38 +0900 Subject: [PATCH 0149/1019] Core: Migrate tests to JUnit5 (#9999) --- .../apache/iceberg/TestCommitReporting.java | 19 +- .../apache/iceberg/TestCreateTransaction.java | 330 +++++-------- .../apache/iceberg/TestPartitionSpecInfo.java | 94 ++-- .../iceberg/TestPartitionSpecParser.java | 35 +- .../org/apache/iceberg/TestPartitioning.java | 94 ++-- .../iceberg/TestReplaceTransaction.java | 255 ++++------ .../iceberg/TestSetPartitionStatistics.java | 64 +-- .../org/apache/iceberg/TestSetStatistics.java | 78 ++- .../org/apache/iceberg/TestTransaction.java | 462 +++++++----------- 9 files changed, 594 insertions(+), 837 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index 08c4ac33d6fd..48b9c8d72bd2 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -21,21 +21,26 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.ScanPlanningAndReportingTestBase.TestMetricsReporter; import org.apache.iceberg.metrics.CommitMetricsResult; import org.apache.iceberg.metrics.CommitReport; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestCommitReporting extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestCommitReporting extends TestBase { private final TestMetricsReporter reporter = new TestMetricsReporter(); - public TestCommitReporting() { - super(2); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2); } - @Test + @TestTemplate public void addAndDeleteDataFiles() { String tableName = "add-and-delete-data-files"; Table table = @@ -80,7 +85,7 @@ public void addAndDeleteDataFiles() { assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(0L); } - @Test + @TestTemplate public void addAndDeleteDeleteFiles() { String tableName = "add-and-delete-delete-files"; Table table = @@ -150,7 +155,7 @@ public void addAndDeleteDeleteFiles() { assertThat(metrics.totalFilesSizeInBytes().value()).isEqualTo(0L); } - @Test + @TestTemplate public void addAndDeleteManifests() throws IOException { String tableName = "add-and-delete-manifests"; Table table = diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 4240184e913c..0c6b50b37792 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -20,70 +20,61 @@ import static org.apache.iceberg.PartitionSpec.unpartitioned; 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; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.exceptions.CommitFailedException; 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.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestCreateTransaction extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestCreateTransaction extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestCreateTransaction(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testCreateTransaction() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "test_create", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_create")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_create")); + assertThat(TestTables.readMetadata("test_create")).isNull(); + assertThat(TestTables.metadataVersion("test_create")).isNull(); txn.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_create"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_create")); - Assert.assertEquals("Should have 0 manifest files", 0, listManifestFiles(tableDir).size()); - - Assert.assertEquals( - "Table schema should match with reassigned IDs", - TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should not have any snapshots", 0, meta.snapshots().size()); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_create")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).isEmpty(); + + assertThat(meta.schema().asStruct()) + .isEqualTo(TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).isEmpty(); } - @Test + @TestTemplate public void testCreateTransactionAndUpdateSchema() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "test_create", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_create")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_create")); + assertThat(TestTables.readMetadata("test_create")).isNull(); + assertThat(TestTables.metadataVersion("test_create")).isNull(); txn.updateSchema() .allowIncompatibleChanges() @@ -94,10 +85,9 @@ public void testCreateTransactionAndUpdateSchema() throws IOException { txn.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_create"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_create")); - Assert.assertEquals("Should have 0 manifest files", 0, listManifestFiles(tableDir).size()); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_create")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).isEmpty(); Schema resultSchema = new Schema( @@ -107,263 +97,195 @@ public void testCreateTransactionAndUpdateSchema() throws IOException { required(3, "col", Types.StringType.get())), Sets.newHashSet(1, 3)); - Assert.assertEquals( - "Table schema should match with reassigned IDs", - resultSchema.asStruct(), - meta.schema().asStruct()); - Assert.assertEquals( - "Table schema identifier should match", - resultSchema.identifierFieldIds(), - meta.schema().identifierFieldIds()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should not have any snapshots", 0, meta.snapshots().size()); + assertThat(meta.schema().asStruct()).isEqualTo(resultSchema.asStruct()); + assertThat(meta.schema().identifierFieldIds()).isEqualTo(resultSchema.identifierFieldIds()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).isEmpty(); } - @Test + @TestTemplate public void testCreateAndAppendWithTransaction() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "test_append", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertNull( - "Appending in a transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); + assertThat(TestTables.metadataVersion("test_append")).isNull(); txn.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_append"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_append")); - Assert.assertEquals("Should have 1 manifest file", 1, listManifestFiles(tableDir).size()); - - Assert.assertEquals( - "Table schema should match with reassigned IDs", - TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should have one snapshot", 1, meta.snapshots().size()); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_append")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).hasSize(1); + + assertThat(meta.schema().asStruct()) + .isEqualTo(TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).hasSize(1); validateSnapshot(null, meta.currentSnapshot(), FILE_A, FILE_B); } - @Test + @TestTemplate public void testCreateAndAppendWithTable() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "test_append", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")) + .isEqualTo(TestTables.readMetadata("test_append")); + assertThat(TestTables.metadataVersion("test_append")).isNull(); - Assert.assertTrue( - "Should return a transaction table", - txn.table() instanceof BaseTransaction.TransactionTable); + assertThat(txn.table()).isInstanceOf(BaseTransaction.TransactionTable.class); txn.table().newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertNull( - "Appending in a transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); + assertThat(TestTables.metadataVersion("test_append")).isNull(); txn.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_append"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_append")); - Assert.assertEquals("Should have 1 manifest file", 1, listManifestFiles(tableDir).size()); - - Assert.assertEquals( - "Table schema should match with reassigned IDs", - TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should have one snapshot", 1, meta.snapshots().size()); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_append")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).hasSize(1); + + assertThat(meta.schema().asStruct()) + .isEqualTo(TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).hasSize(1); validateSnapshot(null, meta.currentSnapshot(), FILE_A, FILE_B); } - @Test + @TestTemplate public void testCreateAndUpdatePropertiesWithTransaction() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "test_properties", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_properties")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("test_properties")); + assertThat(TestTables.readMetadata("test_properties")).isNull(); + assertThat(TestTables.metadataVersion("test_properties")).isNull(); txn.updateProperties().set("test-property", "test-value").commit(); - Assert.assertNull( - "Adding properties in a transaction should not commit metadata", - TestTables.readMetadata("test_properties")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("test_properties")); + assertThat(TestTables.readMetadata("test_properties")).isNull(); + assertThat(TestTables.metadataVersion("test_properties")).isNull(); txn.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_properties"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_properties")); - Assert.assertEquals("Should have 0 manifest files", 0, listManifestFiles(tableDir).size()); - - Assert.assertEquals( - "Table schema should match with reassigned IDs", - TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should not have any snapshots", 0, meta.snapshots().size()); - Assert.assertEquals("Should have one table property", 1, meta.properties().size()); - Assert.assertEquals( - "Should have correct table property value", - "test-value", - meta.properties().get("test-property")); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_properties")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).isEmpty(); + + assertThat(meta.schema().asStruct()) + .isEqualTo(TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).isEmpty(); + assertThat(meta.properties()).hasSize(1).containsEntry("test-property", "test-value"); } - @Test + @TestTemplate public void testCreateAndUpdatePropertiesWithTable() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "test_properties", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_properties")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("test_properties")); + assertThat(TestTables.readMetadata("test_properties")).isNull(); + assertThat(TestTables.metadataVersion("test_properties")).isNull(); - Assert.assertTrue( - "Should return a transaction table", - txn.table() instanceof BaseTransaction.TransactionTable); + assertThat(txn.table()).isInstanceOf(BaseTransaction.TransactionTable.class); txn.table().updateProperties().set("test-property", "test-value").commit(); - Assert.assertNull( - "Adding properties in a transaction should not commit metadata", - TestTables.readMetadata("test_properties")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("test_properties")); + assertThat(TestTables.readMetadata("test_properties")).isNull(); + assertThat(TestTables.metadataVersion("test_properties")).isNull(); txn.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_properties"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_properties")); - Assert.assertEquals("Should have 0 manifest files", 0, listManifestFiles(tableDir).size()); - - Assert.assertEquals( - "Table schema should match with reassigned IDs", - TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should not have any snapshots", 0, meta.snapshots().size()); - Assert.assertEquals("Should have one table property", 1, meta.properties().size()); - Assert.assertEquals( - "Should have correct table property value", - "test-value", - meta.properties().get("test-property")); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_properties")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).hasSize(0); + + assertThat(meta.schema().asStruct()) + .isEqualTo(TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).isEmpty(); + assertThat(meta.properties()).hasSize(1).containsEntry("test-property", "test-value"); } - @Test + @TestTemplate public void testCreateDetectsUncommittedChange() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "uncommitted_change", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("uncommitted_change")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("uncommitted_change")); + assertThat(TestTables.readMetadata("uncommitted_change")).isNull(); + assertThat(TestTables.metadataVersion("uncommitted_change")).isNull(); txn.updateProperties().set("test-property", "test-value"); // not committed - Assertions.assertThatThrownBy(txn::newDelete) + assertThatThrownBy(txn::newDelete) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot create new DeleteFiles: last operation has not committed"); } - @Test + @TestTemplate public void testCreateDetectsUncommittedChangeOnCommit() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Transaction txn = TestTables.beginCreate(tableDir, "uncommitted_change", SCHEMA, unpartitioned()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("uncommitted_change")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("uncommitted_change")); + assertThat(TestTables.readMetadata("uncommitted_change")).isNull(); + assertThat(TestTables.metadataVersion("uncommitted_change")).isNull(); txn.updateProperties().set("test-property", "test-value"); // not committed - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot commit transaction: last operation has not committed"); } - @Test + @TestTemplate public void testCreateTransactionConflict() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + 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 txn.newAppend().appendFile(FILE_A).commit(); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_conflict")); - Assert.assertNull( - "Should have no metadata version", TestTables.metadataVersion("test_conflict")); + assertThat(TestTables.readMetadata("test_conflict")).isNull(); + assertThat(TestTables.metadataVersion("test_conflict")).isNull(); Table conflict = TestTables.create(tableDir, "test_conflict", SCHEMA, unpartitioned(), formatVersion); - Assert.assertEquals( - "Table schema should match with reassigned IDs", - TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct(), - conflict.schema().asStruct()); - Assert.assertEquals( - "Table spec should match conflict table, not transaction table", - unpartitioned(), - conflict.spec()); - Assert.assertFalse( - "Table should not have any snapshots", conflict.snapshots().iterator().hasNext()); - - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThat(conflict.schema().asStruct()) + .isEqualTo(TypeUtil.assignIncreasingFreshIds(SCHEMA).asStruct()); + assertThat(conflict.spec()).isEqualTo(unpartitioned()); + assertThat(conflict.snapshots()).isEmpty(); + + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith("Commit failed: table was updated"); - Assert.assertEquals( - "Should clean up metadata", - Sets.newHashSet(), - Sets.newHashSet(listManifestFiles(tableDir))); + assertThat(listManifestFiles(tableDir)).isEmpty(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index 46f38b97f2a4..a82ac6636d4a 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -19,46 +19,45 @@ package org.apache.iceberg; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.entry; import java.io.File; import java.io.IOException; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) public class TestPartitionSpecInfo { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; + private final Schema schema = new Schema( required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get())); private File tableDir = null; - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - private final int formatVersion; - - public TestPartitionSpecInfo(int formatVersion) { - this.formatVersion = formatVersion; - } + @Parameter private int formatVersion; - @Before + @BeforeEach public void setupTableDir() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); } - @After + @AfterEach public void cleanupTables() { TestTables.clearTables(); } @@ -68,7 +67,7 @@ public void testSpecIsUnpartitionedForVoidTranforms() { PartitionSpec spec = PartitionSpec.builderFor(schema).alwaysNull("id").alwaysNull("data").build(); - Assert.assertTrue(spec.isUnpartitioned()); + assertThat(spec.isUnpartitioned()).isTrue(); } @Test @@ -76,11 +75,12 @@ public void testSpecInfoUnpartitionedTable() { PartitionSpec spec = PartitionSpec.unpartitioned(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); - Assert.assertTrue(spec.isUnpartitioned()); - Assert.assertEquals(spec, table.spec()); - Assert.assertEquals(spec.lastAssignedFieldId(), table.spec().lastAssignedFieldId()); - Assert.assertEquals(ImmutableMap.of(spec.specId(), spec), table.specs()); - Assert.assertNull(table.specs().get(Integer.MAX_VALUE)); + assertThat(spec.isUnpartitioned()).isTrue(); + assertThat(table.spec()).isEqualTo(spec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(spec.lastAssignedFieldId()); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec)) + .doesNotContainKey(Integer.MAX_VALUE); } @Test @@ -88,10 +88,11 @@ public void testSpecInfoPartitionedTable() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); - Assert.assertEquals(spec, table.spec()); - Assert.assertEquals(spec.lastAssignedFieldId(), table.spec().lastAssignedFieldId()); - Assert.assertEquals(ImmutableMap.of(spec.specId(), spec), table.specs()); - Assert.assertNull(table.specs().get(Integer.MAX_VALUE)); + assertThat(table.spec()).isEqualTo(spec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(spec.lastAssignedFieldId()); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec)) + .doesNotContainKey(Integer.MAX_VALUE); } @Test @@ -99,7 +100,7 @@ public void testColumnDropWithPartitionSpecEvolution() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); - Assert.assertEquals(spec, table.spec()); + assertThat(table.spec()).isEqualTo(spec); TableMetadata base = TestTables.readMetadata("test"); PartitionSpec newSpec = @@ -111,14 +112,11 @@ public void testColumnDropWithPartitionSpecEvolution() { final Schema expectedSchema = new Schema(required(2, "data", Types.StringType.get())); - Assert.assertEquals(newSpec, table.spec()); - Assert.assertEquals(newSpec, table.specs().get(newSpec.specId())); - Assert.assertEquals(spec, table.specs().get(spec.specId())); - Assert.assertEquals( - ImmutableMap.of(spec.specId(), spec, newSpec.specId(), newSpec), table.specs()); - Assert.assertNull(table.specs().get(Integer.MAX_VALUE)); - Assert.assertTrue( - "Schema must have only \"data\" column", table.schema().sameSchema(expectedSchema)); + assertThat(table.spec()).isEqualTo(newSpec); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec), entry(newSpec.specId(), newSpec)) + .doesNotContainKey(Integer.MAX_VALUE); + assertThat(table.schema().asStruct()).isEqualTo(expectedSchema.asStruct()); } @Test @@ -126,18 +124,16 @@ public void testSpecInfoPartitionSpecEvolutionForV1Table() { PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 4).build(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); - Assert.assertEquals(spec, table.spec()); + assertThat(table.spec()).isEqualTo(spec); TableMetadata base = TestTables.readMetadata("test"); PartitionSpec newSpec = PartitionSpec.builderFor(table.schema()).bucket("data", 10).withSpecId(1).build(); table.ops().commit(base, base.updatePartitionSpec(newSpec)); - Assert.assertEquals(newSpec, table.spec()); - Assert.assertEquals(newSpec, table.specs().get(newSpec.specId())); - Assert.assertEquals(spec, table.specs().get(spec.specId())); - Assert.assertEquals( - ImmutableMap.of(spec.specId(), spec, newSpec.specId(), newSpec), table.specs()); - Assert.assertNull(table.specs().get(Integer.MAX_VALUE)); + assertThat(table.spec()).isEqualTo(newSpec); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec), entry(newSpec.specId(), newSpec)) + .doesNotContainKey(Integer.MAX_VALUE); } } diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java index 1d88e97f9925..ad8861f53685 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java @@ -18,12 +18,18 @@ */ package org.apache.iceberg; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; -public class TestPartitionSpecParser extends TableTestBase { - public TestPartitionSpecParser() { - super(1); +import java.util.Arrays; +import java.util.List; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestPartitionSpecParser extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1); } @Test @@ -38,7 +44,7 @@ public void testToJsonForV1Table() { + " \"field-id\" : 1000\n" + " } ]\n" + "}"; - Assert.assertEquals(expected, PartitionSpecParser.toJson(table.spec(), true)); + assertThat(PartitionSpecParser.toJson(table.spec(), true)).isEqualTo(expected); PartitionSpec spec = PartitionSpec.builderFor(table.schema()).bucket("id", 8).bucket("data", 16).build(); @@ -60,7 +66,7 @@ public void testToJsonForV1Table() { + " \"field-id\" : 1001\n" + " } ]\n" + "}"; - Assert.assertEquals(expected, PartitionSpecParser.toJson(table.spec(), true)); + assertThat(PartitionSpecParser.toJson(table.spec(), true)).isEqualTo(expected); } @Test @@ -83,10 +89,10 @@ public void testFromJsonWithFieldId() { PartitionSpec spec = PartitionSpecParser.fromJson(table.schema(), specString); - Assert.assertEquals(2, spec.fields().size()); + assertThat(spec.fields()).hasSize(2); // should be the field ids in the JSON - Assert.assertEquals(1001, spec.fields().get(0).fieldId()); - Assert.assertEquals(1000, spec.fields().get(1).fieldId()); + assertThat(spec.fields().get(0).fieldId()).isEqualTo(1001); + assertThat(spec.fields().get(1).fieldId()).isEqualTo(1000); } @Test @@ -107,17 +113,16 @@ public void testFromJsonWithoutFieldId() { PartitionSpec spec = PartitionSpecParser.fromJson(table.schema(), specString); - Assert.assertEquals(2, spec.fields().size()); + assertThat(spec.fields()).hasSize(2); // should be the default assignment - Assert.assertEquals(1000, spec.fields().get(0).fieldId()); - Assert.assertEquals(1001, spec.fields().get(1).fieldId()); + assertThat(spec.fields().get(0).fieldId()).isEqualTo(1000); + assertThat(spec.fields().get(1).fieldId()).isEqualTo(1001); } @Test public void testTransforms() { for (PartitionSpec spec : PartitionSpecTestBase.SPECS) { - Assert.assertEquals( - "To/from JSON should produce equal partition spec", spec, roundTripJSON(spec)); + assertThat(roundTripJSON(spec)).isEqualTo(spec); } } diff --git a/core/src/test/java/org/apache/iceberg/TestPartitioning.java b/core/src/test/java/org/apache/iceberg/TestPartitioning.java index 4de62e3cfee3..91f0fe95c2fa 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitioning.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitioning.java @@ -19,21 +19,22 @@ package org.apache.iceberg; 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; +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.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestPartitioning { @@ -51,15 +52,15 @@ public class TestPartitioning { private static final PartitionSpec BY_DATA_CATEGORY_BUCKET_SPEC = PartitionSpec.builderFor(SCHEMA).identity("data").bucket("category", 8).build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private File tableDir = null; - @Before + @BeforeEach public void setupTableDir() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); } - @After + @AfterEach public void cleanupTables() { TestTables.clearTables(); } @@ -71,19 +72,19 @@ public void testPartitionTypeWithSpecEvolutionInV1Tables() { table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of( NestedField.optional(1000, "data", Types.StringType.get()), NestedField.optional(1001, "category_bucket_8", Types.IntegerType.get())); StructType actualType = Partitioning.partitionType(table); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); table.updateSpec().removeField("data").removeField("category_bucket_8").commit(); - Assert.assertEquals("Should have 3 specs", 3, table.specs().size()); - Assert.assertTrue("PartitionSpec should be unpartitioned", table.spec().isUnpartitioned()); + assertThat(table.specs()).hasSize(3); + assertThat(table.spec().isUnpartitioned()).isTrue(); } @Test @@ -93,14 +94,14 @@ public void testPartitionTypeWithSpecEvolutionInV2Tables() { table.updateSpec().removeField("data").addField("category").commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of( NestedField.optional(1000, "data", Types.StringType.get()), NestedField.optional(1001, "category", Types.StringType.get())); StructType actualType = Partitioning.partitionType(table); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -118,7 +119,7 @@ public void testPartitionTypeWithRenamesInV1Table() { NestedField.optional(1000, "p2", Types.StringType.get()), NestedField.optional(1001, "category", Types.StringType.get())); StructType actualType = Partitioning.partitionType(table); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -136,7 +137,7 @@ public void testPartitionTypeWithAddingBackSamePartitionFieldInV1Table() { NestedField.optional(1000, "data_1000", Types.StringType.get()), NestedField.optional(1001, "data", Types.StringType.get())); StructType actualType = Partitioning.partitionType(table); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -152,7 +153,7 @@ public void testPartitionTypeWithAddingBackSamePartitionFieldInV2Table() { StructType expectedType = StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); StructType actualType = Partitioning.partitionType(table); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -166,9 +167,9 @@ public void testPartitionTypeWithIncompatibleSpecEvolution() { TableMetadata current = ops.current(); ops.commit(current, current.updatePartitionSpec(newSpec)); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); - Assertions.assertThatThrownBy(() -> Partitioning.partitionType(table)) + assertThatThrownBy(() -> Partitioning.partitionType(table)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Conflicting partition fields"); } @@ -180,12 +181,12 @@ public void testGroupingKeyTypeWithSpecEvolutionInV1Tables() { table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -195,12 +196,12 @@ public void testGroupingKeyTypeWithSpecEvolutionInV2Tables() { table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -211,12 +212,12 @@ public void testGroupingKeyTypeWithDroppedPartitionFieldInV1Tables() { table.updateSpec().removeField(Expressions.bucket("category", 8)).commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -227,12 +228,12 @@ public void testGroupingKeyTypeWithDroppedPartitionFieldInV2Tables() { table.updateSpec().removeField(Expressions.bucket("category", 8)).commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -248,7 +249,7 @@ public void testGroupingKeyTypeWithRenamesInV1Table() { StructType expectedType = StructType.of(NestedField.optional(1000, "p2", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -264,7 +265,7 @@ public void testGroupingKeyTypeWithRenamesInV2Table() { StructType expectedType = StructType.of(NestedField.optional(1000, "p2", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -274,11 +275,11 @@ public void testGroupingKeyTypeWithEvolvedIntoUnpartitionedSpecV1Table() { table.updateSpec().removeField("data").commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -288,11 +289,11 @@ public void testGroupingKeyTypeWithEvolvedIntoUnpartitionedSpecV2Table() { table.updateSpec().removeField("data").commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -307,7 +308,7 @@ public void testGroupingKeyTypeWithAddingBackSamePartitionFieldInV1Table() { StructType expectedType = StructType.of(NestedField.optional(1000, "category", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -322,7 +323,7 @@ public void testGroupingKeyTypeWithAddingBackSamePartitionFieldInV2Table() { StructType expectedType = StructType.of(NestedField.optional(1000, "category", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -331,11 +332,11 @@ public void testGroupingKeyTypeWithOnlyUnpartitionedSpec() { TestTables.create( tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), V1_FORMAT_VERSION); - Assert.assertEquals("Should have 1 spec", 1, table.specs().size()); + assertThat(table.specs()).hasSize(1); StructType expectedType = StructType.of(); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -346,11 +347,11 @@ public void testGroupingKeyTypeWithEvolvedUnpartitionedSpec() { table.updateSpec().addField(Expressions.bucket("category", 8)).commit(); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); StructType expectedType = StructType.of(); StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -363,7 +364,7 @@ public void testGroupingKeyTypeWithProjectedSchema() { StructType expectedType = StructType.of(NestedField.optional(1001, "data", Types.StringType.get())); StructType actualType = Partitioning.groupingKeyType(projectedSchema, table.specs().values()); - Assert.assertEquals("Types must match", expectedType, actualType); + assertThat(actualType).isEqualTo(expectedType); } @Test @@ -377,10 +378,9 @@ public void testGroupingKeyTypeWithIncompatibleSpecEvolution() { TableMetadata current = ops.current(); ops.commit(current, current.updatePartitionSpec(newSpec)); - Assert.assertEquals("Should have 2 specs", 2, table.specs().size()); + assertThat(table.specs()).hasSize(2); - Assertions.assertThatThrownBy( - () -> Partitioning.groupingKeyType(table.schema(), table.specs().values())) + assertThatThrownBy(() -> Partitioning.groupingKeyType(table.schema(), table.specs().values())) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Conflicting partition fields"); } @@ -403,6 +403,6 @@ public void testDeletingPartitionField() { .identity("id") .build(); - Assert.assertEquals("The spec should be there", spec, table.spec()); + assertThat(table.spec()).isEqualTo(spec); } } diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index b338d00696dd..3a6d2017eb82 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -22,9 +22,15 @@ import static org.apache.iceberg.PartitionSpec.unpartitioned; import static org.apache.iceberg.SortDirection.ASC; 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; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; @@ -37,32 +43,24 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestReplaceTransaction extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestReplaceTransaction extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestReplaceTransaction(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testReplaceTransactionWithCustomSortOrder() { Snapshot start = table.currentSnapshot(); Schema schema = table.schema(); table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); @@ -75,10 +73,9 @@ public void testReplaceTransactionWithCustomSortOrder() { table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should match previous schema", schema.asStruct(), table.schema().asStruct()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot()).isNull(); + assertThat(table.schema().asStruct()).isEqualTo(schema.asStruct()); PartitionSpec v2Expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).build(); V2Assert.assertEquals("Table should have an unpartitioned spec", v2Expected, table.spec()); @@ -90,18 +87,17 @@ public void testReplaceTransactionWithCustomSortOrder() { .build(); V1Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); - Assert.assertEquals("Table should have 2 orders", 2, table.sortOrders().size()); + assertThat(table.sortOrders()).hasSize(2); SortOrder sortOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 1, sortOrder.orderId()); - Assert.assertEquals("Order must have 1 field", 1, sortOrder.fields().size()); - Assert.assertEquals("Direction must match ", ASC, sortOrder.fields().get(0).direction()); - Assert.assertEquals( - "Null order must match ", NULLS_FIRST, sortOrder.fields().get(0).nullOrder()); + assertThat(sortOrder.orderId()).isEqualTo(1); + assertThat(sortOrder.fields()).hasSize(1); + assertThat(sortOrder.fields().get(0).direction()).isEqualTo(ASC); + assertThat(sortOrder.fields().get(0).nullOrder()).isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(); - Assert.assertEquals("Transform must match", transform, sortOrder.fields().get(0).transform()); + assertThat(sortOrder.fields().get(0).transform()).isEqualTo(transform); } - @Test + @TestTemplate public void testReplaceTransaction() { Schema newSchema = new Schema( @@ -113,7 +109,7 @@ public void testReplaceTransaction() { table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); @@ -122,10 +118,9 @@ public void testReplaceTransaction() { table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should match previous schema", schema.asStruct(), table.schema().asStruct()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot()).isNull(); + assertThat(table.schema().asStruct()).isEqualTo(schema.asStruct()); PartitionSpec v2Expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).build(); V2Assert.assertEquals("Table should have an unpartitioned spec", v2Expected, table.spec()); @@ -137,15 +132,16 @@ public void testReplaceTransaction() { .build(); V1Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); - Assert.assertEquals("Table should have 1 order", 1, table.sortOrders().size()); - Assert.assertEquals("Table order ID should match", 0, table.sortOrder().orderId()); - Assert.assertTrue("Table should be unsorted", table.sortOrder().isUnsorted()); + assertThat(table.sortOrders()).hasSize(1); + assertThat(table.sortOrder().orderId()).isEqualTo(0); + assertThat(table.sortOrder().isUnsorted()).isTrue(); } - @Test + @TestTemplate public void testReplaceWithIncompatibleSchemaUpdate() { - Assume.assumeTrue( - "Fails early for v1 tables because partition spec cannot drop a field", formatVersion == 2); + assumeThat(formatVersion) + .as("Fails early for v1 tables because partition spec cannot drop a field") + .isEqualTo(2); Schema newSchema = new Schema(required(4, "obj_id", Types.IntegerType.get())); @@ -153,7 +149,7 @@ public void testReplaceWithIncompatibleSchemaUpdate() { table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); @@ -162,15 +158,13 @@ public void testReplaceWithIncompatibleSchemaUpdate() { table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should use new schema, not compatible with previous", - new Schema(required(3, "obj_id", Types.IntegerType.get())).asStruct(), - table.schema().asStruct()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot()).isNull(); + assertThat(table.schema().asStruct()) + .isEqualTo(new Schema(required(3, "obj_id", Types.IntegerType.get())).asStruct()); } - @Test + @TestTemplate public void testReplaceWithNewPartitionSpec() { PartitionSpec newSpec = PartitionSpec.unpartitioned(); @@ -179,7 +173,7 @@ public void testReplaceWithNewPartitionSpec() { table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); @@ -188,12 +182,9 @@ public void testReplaceWithNewPartitionSpec() { table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNull("Table should not have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should use new schema, not compatible with previous", - schema.asStruct(), - table.schema().asStruct()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot()).isNull(); + assertThat(table.schema().asStruct()).isEqualTo(schema.asStruct()); PartitionSpec v2Expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).build(); V2Assert.assertEquals("Table should have an unpartitioned spec", v2Expected, table.spec()); @@ -206,14 +197,14 @@ public void testReplaceWithNewPartitionSpec() { V1Assert.assertEquals("Table should have a spec with one void field", v1Expected, table.spec()); } - @Test + @TestTemplate public void testReplaceWithNewData() { Snapshot start = table.currentSnapshot(); Schema schema = table.schema(); table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); @@ -225,19 +216,16 @@ public void testReplaceWithNewData() { table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNotNull("Table should have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should use new schema, not compatible with previous", - schema.asStruct(), - table.schema().asStruct()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot()).isNotNull(); + assertThat(table.schema().asStruct()).isEqualTo(schema.asStruct()); validateSnapshot(null, table.currentSnapshot(), FILE_B, FILE_C, FILE_D); } - @Test + @TestTemplate public void testReplaceDetectsUncommittedChangeOnCommit() { - Assert.assertEquals("Version should be 0", 0L, (long) version()); + assertThat(version()).isEqualTo(0); Transaction replace = TestTables.beginReplace(tableDir, "test", table.schema(), table.spec()); @@ -247,16 +235,16 @@ public void testReplaceDetectsUncommittedChangeOnCommit() { .appendFile(FILE_C) .appendFile(FILE_D); - Assertions.assertThatThrownBy(replace::commitTransaction) + assertThatThrownBy(replace::commitTransaction) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot commit transaction: last operation has not committed"); - Assert.assertEquals("Version should be 0", 0L, (long) version()); + assertThat(version()).isEqualTo(0); } - @Test + @TestTemplate public void testReplaceDetectsUncommittedChangeOnTableCommit() { - Assert.assertEquals("Version should be 0", 0L, (long) version()); + assertThat(version()).isEqualTo(0); Transaction replace = TestTables.beginReplace(tableDir, "test", table.schema(), table.spec()); @@ -267,21 +255,21 @@ public void testReplaceDetectsUncommittedChangeOnTableCommit() { .appendFile(FILE_C) .appendFile(FILE_D); - Assertions.assertThatThrownBy(replace::commitTransaction) + assertThatThrownBy(replace::commitTransaction) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot commit transaction: last operation has not committed"); - Assert.assertEquals("Version should be 0", 0L, (long) version()); + assertThat(version()).isEqualTo(0); } - @Test + @TestTemplate public void testReplaceTransactionRetry() { Snapshot start = table.currentSnapshot(); Schema schema = table.schema(); table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); @@ -296,23 +284,20 @@ public void testReplaceTransactionRetry() { table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNotNull("Table should have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should use new schema, not compatible with previous", - schema.asStruct(), - table.schema().asStruct()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot()).isNotNull(); + assertThat(table.schema().asStruct()).isEqualTo(schema.asStruct()); validateSnapshot(null, table.currentSnapshot(), FILE_B, FILE_C, FILE_D); } - @Test + @TestTemplate public void testReplaceTransactionConflict() { Snapshot start = table.currentSnapshot(); table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(start, table.currentSnapshot(), FILE_A); Set manifests = Sets.newHashSet(listManifestFiles()); @@ -324,63 +309,52 @@ public void testReplaceTransactionConflict() { // keep failing to trigger eventual transaction failure ((TestTables.TestTableOperations) ((BaseTransaction) replace).ops()).failCommits(100); - Assertions.assertThatThrownBy(replace::commitTransaction) + assertThatThrownBy(replace::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); table.refresh(); validateSnapshot(start, table.currentSnapshot(), FILE_A); - Assert.assertEquals( - "Should clean up replace manifests", manifests, Sets.newHashSet(listManifestFiles())); + assertThat(listManifestFiles()).containsExactlyElementsOf(manifests); } - @Test + @TestTemplate public void testReplaceToCreateAndAppend() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + 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()); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); + assertThat(TestTables.metadataVersion("test_append")).isNull(); - Assert.assertTrue( - "Should return a transaction table", - replace.table() instanceof BaseTransaction.TransactionTable); + assertThat(replace.table()).isInstanceOf(BaseTransaction.TransactionTable.class); replace.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertNull( - "Appending in a transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); + assertThat(TestTables.metadataVersion("test_append")).isNull(); replace.commitTransaction(); TableMetadata meta = TestTables.readMetadata("test_append"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_append")); - Assert.assertEquals("Should have 1 manifest file", 1, listManifestFiles(tableDir).size()); - - Assert.assertEquals( - "Table schema should match with reassigned IDs", - assignFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should have one snapshot", 1, meta.snapshots().size()); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_append")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).hasSize(1); + + assertThat(meta.schema().asStruct()).isEqualTo(assignFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).hasSize(1); validateSnapshot(null, meta.currentSnapshot(), FILE_A, FILE_B); } - @Test + @TestTemplate public void testReplaceTransactionWithUnknownState() { Schema newSchema = new Schema( @@ -392,7 +366,7 @@ public void testReplaceTransactionWithUnknownState() { table.newAppend().appendFile(FILE_A).commit(); - Assert.assertEquals("Version should be 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1L); validateSnapshot(start, table.currentSnapshot(), FILE_A); TestTables.TestTableOperations ops = @@ -409,26 +383,23 @@ public void testReplaceTransactionWithUnknownState() { replace.newAppend().appendFile(FILE_B).commit(); - Assertions.assertThatThrownBy(replace::commitTransaction) + assertThatThrownBy(replace::commitTransaction) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("datacenter on fire"); table.refresh(); - Assert.assertEquals("Version should be 2", 2L, (long) version()); - Assert.assertNotNull("Table should have a current snapshot", table.currentSnapshot()); - Assert.assertEquals( - "Schema should use new schema, not compatible with previous", - schema.asStruct(), - table.schema().asStruct()); - Assert.assertEquals("Should have 4 files in metadata", 4, countAllMetadataFiles(tableDir)); + assertThat(version()).isEqualTo(2L); + assertThat(table.currentSnapshot()).isNotNull(); + assertThat(table.schema().asStruct()).isEqualTo(schema.asStruct()); + assertThat(countAllMetadataFiles(tableDir)).isEqualTo(4); validateSnapshot(null, table.currentSnapshot(), FILE_B); } - @Test + @TestTemplate public void testCreateTransactionWithUnknownState() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // this table doesn't exist. TestTables.TestTableOperations ops = @@ -443,38 +414,28 @@ public void testCreateTransactionWithUnknownState() throws IOException { ImmutableMap.of(), ops); - Assert.assertNull( - "Starting a create transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); + assertThat(TestTables.metadataVersion("test_append")).isNull(); - Assert.assertTrue( - "Should return a transaction table", - replace.table() instanceof BaseTransaction.TransactionTable); + assertThat(replace.table()).isInstanceOf(BaseTransaction.TransactionTable.class); replace.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertNull( - "Appending in a transaction should not commit metadata", - TestTables.readMetadata("test_append")); - Assert.assertNull("Should have no metadata version", TestTables.metadataVersion("test_append")); + assertThat(TestTables.readMetadata("test_append")).isNull(); + assertThat(TestTables.metadataVersion("test_append")).isNull(); - Assertions.assertThatThrownBy(replace::commitTransaction) + assertThatThrownBy(replace::commitTransaction) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("datacenter on fire"); TableMetadata meta = TestTables.readMetadata("test_append"); - Assert.assertNotNull("Table metadata should be created after transaction commits", meta); - Assert.assertEquals( - "Should have metadata version 0", 0, (int) TestTables.metadataVersion("test_append")); - Assert.assertEquals("Should have 1 manifest file", 1, listManifestFiles(tableDir).size()); - Assert.assertEquals("Should have 2 files in metadata", 2, countAllMetadataFiles(tableDir)); - Assert.assertEquals( - "Table schema should match with reassigned IDs", - assignFreshIds(SCHEMA).asStruct(), - meta.schema().asStruct()); - Assert.assertEquals("Table spec should match", unpartitioned(), meta.spec()); - Assert.assertEquals("Table should have one snapshot", 1, meta.snapshots().size()); + assertThat(meta).isNotNull(); + assertThat(TestTables.metadataVersion("test_append")).isEqualTo(0); + assertThat(listManifestFiles(tableDir)).hasSize(1); + assertThat(countAllMetadataFiles(tableDir)).isEqualTo(2); + assertThat(meta.schema().asStruct()).isEqualTo(assignFreshIds(SCHEMA).asStruct()); + assertThat(meta.spec()).isEqualTo(unpartitioned()); + assertThat(meta.snapshots()).hasSize(1); validateSnapshot(null, meta.currentSnapshot(), FILE_A, FILE_B); } diff --git a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java index 2ab5a141133f..3ebe047e90b7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java @@ -18,36 +18,32 @@ */ package org.apache.iceberg; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSetPartitionStatistics extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.List; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSetPartitionStatistics extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestSetPartitionStatistics(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testEmptyUpdateStatistics() { assertTableMetadataVersion(0); TableMetadata base = readMetadata(); table.updatePartitionStatistics().commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, table.ops().current()); + assertThat(table.ops().current()).isSameAs(base); assertTableMetadataVersion(1); } - @Test + @TestTemplate public void testEmptyTransactionalUpdateStatistics() { assertTableMetadataVersion(0); TableMetadata base = readMetadata(); @@ -56,12 +52,11 @@ public void testEmptyTransactionalUpdateStatistics() { transaction.updatePartitionStatistics().commit(); transaction.commitTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, table.ops().current()); + assertThat(table.ops().current()).isSameAs(base); assertTableMetadataVersion(0); } - @Test + @TestTemplate public void testUpdateStatistics() { // Create a snapshot table.newFastAppend().commit(); @@ -80,17 +75,11 @@ public void testUpdateStatistics() { TableMetadata metadata = readMetadata(); assertTableMetadataVersion(2); - Assert.assertEquals( - "Table snapshot should be the same after setting partition statistics file", - snapshotId, - metadata.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Table metadata should have partition statistics files", - ImmutableList.of(partitionStatisticsFile), - metadata.partitionStatisticsFiles()); + assertThat(metadata.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(metadata.partitionStatisticsFiles()).containsExactly(partitionStatisticsFile); } - @Test + @TestTemplate public void testRemoveStatistics() { // Create a snapshot table.newFastAppend().commit(); @@ -109,23 +98,16 @@ public void testRemoveStatistics() { TableMetadata metadata = readMetadata(); assertTableMetadataVersion(2); - Assert.assertEquals( - "Table metadata should have partition statistics files", - ImmutableList.of(partitionStatisticsFile), - metadata.partitionStatisticsFiles()); + assertThat(metadata.partitionStatisticsFiles()).containsExactly(partitionStatisticsFile); table.updatePartitionStatistics().removePartitionStatistics(snapshotId).commit(); metadata = readMetadata(); assertTableMetadataVersion(3); - Assert.assertEquals( - "Table metadata should have no partition statistics files", - ImmutableList.of(), - metadata.partitionStatisticsFiles()); + assertThat(metadata.partitionStatisticsFiles()).isEmpty(); } private void assertTableMetadataVersion(int expected) { - Assert.assertEquals( - String.format("Table should be on version %s", expected), expected, (int) version()); + assertThat(version()).isEqualTo(expected); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java index f594e08d1e91..41941e3c6630 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java @@ -18,55 +18,51 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSetStatistics extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestSetStatistics(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSetStatistics extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testEmptyUpdateStatistics() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); table.updateStatistics().commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, table.ops().current()); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(table.ops().current()).isSameAs(base); + assertThat(version()).isEqualTo(1); } - @Test + @TestTemplate public void testEmptyTransactionalUpdateStatistics() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); Transaction transaction = table.newTransaction(); transaction.updateStatistics().commit(); transaction.commitTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, table.ops().current()); - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(table.ops().current()).isSameAs(base); + assertThat(version()).isEqualTo(0); } - @Test + @TestTemplate public void testUpdateStatistics() { // Create a snapshot table.newFastAppend().commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); long snapshotId = base.currentSnapshot().snapshotId(); @@ -87,22 +83,16 @@ public void testUpdateStatistics() { table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); TableMetadata metadata = readMetadata(); - Assert.assertEquals("Table should be on version 2", 2, (int) version()); - Assert.assertEquals( - "Table snapshot should be the same after setting statistics file", - snapshotId, - metadata.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Table metadata should have statistics files", - ImmutableList.of(statisticsFile), - metadata.statisticsFiles()); + assertThat(version()).isEqualTo(2); + assertThat(metadata.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(metadata.statisticsFiles()).containsExactly(statisticsFile); } - @Test + @TestTemplate public void testRemoveStatistics() { // Create a snapshot table.newFastAppend().commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); long snapshotId = base.currentSnapshot().snapshotId(); @@ -113,19 +103,13 @@ public void testRemoveStatistics() { table.updateStatistics().setStatistics(snapshotId, statisticsFile).commit(); TableMetadata metadata = readMetadata(); - Assert.assertEquals("Table should be on version 2", 2, (int) version()); - Assert.assertEquals( - "Table metadata should have statistics files", - ImmutableList.of(statisticsFile), - metadata.statisticsFiles()); + assertThat(version()).isEqualTo(2); + assertThat(metadata.statisticsFiles()).containsExactly(statisticsFile); table.updateStatistics().removeStatistics(snapshotId).commit(); metadata = readMetadata(); - Assert.assertEquals("Table should be on version 3", 3, (int) version()); - Assert.assertEquals( - "Table metadata should have no statistics files", - ImmutableList.of(), - metadata.statisticsFiles()); + assertThat(version()).isEqualTo(3); + assertThat(metadata.statisticsFiles()).isEmpty(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 74892dbfbf7c..393494da1283 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -18,8 +18,13 @@ */ package org.apache.iceberg; +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.util.Arrays; import java.util.List; import java.util.Set; import java.util.UUID; @@ -30,65 +35,54 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.assertj.core.api.Assumptions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestTransaction extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestTransaction(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTransaction extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testEmptyTransaction() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); txn.commitTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); } - @Test + @TestTemplate public void testSingleOperationTransaction() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertSame( - "Base metadata should not change when an append is committed", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after append", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); txn.commitTransaction(); validateSnapshot(base.currentSnapshot(), readMetadata().currentSnapshot(), FILE_A, FILE_B); - Assert.assertEquals("Table should be on version 1 after commit", 1, (int) version()); + assertThat(version()).isEqualTo(1); } - @Test + @TestTemplate public void testMultipleOperationTransaction() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); table.newAppend().appendFile(FILE_C).commit(); List initialHistory = table.history(); @@ -97,15 +91,13 @@ public void testMultipleOperationTransaction() { Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); Snapshot appendSnapshot = txn.table().currentSnapshot(); @@ -113,55 +105,45 @@ public void testMultipleOperationTransaction() { Snapshot deleteSnapshot = txn.table().currentSnapshot(); - Assert.assertSame( - "Base metadata should not change when an append is committed", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after append", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 2 after commit", 2, (int) version()); - Assert.assertEquals( - "Table should have two manifest after commit", - 2, - readMetadata().currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Table snapshot should be the delete snapshot", - deleteSnapshot, - readMetadata().currentSnapshot()); + assertThat(version()).isEqualTo(2); + assertThat(readMetadata().currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(readMetadata().currentSnapshot()).isEqualTo(deleteSnapshot); validateManifestEntries( readMetadata().currentSnapshot().allManifests(table.io()).get(0), ids(deleteSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); - Assert.assertEquals( - "Table should have a snapshot for each operation", 3, readMetadata().snapshots().size()); + assertThat(readMetadata().snapshots()).hasSize(3); validateManifestEntries( readMetadata().snapshots().get(1).allManifests(table.io()).get(0), ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); - Assertions.assertThat(table.history()).containsAll(initialHistory); + assertThat(table.history()).containsAll(initialHistory); } - @Test + @TestTemplate public void testMultipleOperationTransactionFromTable() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); Snapshot appendSnapshot = txn.table().currentSnapshot(); @@ -169,29 +151,21 @@ public void testMultipleOperationTransactionFromTable() { Snapshot deleteSnapshot = txn.table().currentSnapshot(); - Assert.assertSame( - "Base metadata should not change when an append is committed", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after append", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 1 after commit", 1, (int) version()); - Assert.assertEquals( - "Table should have one manifest after commit", - 1, - readMetadata().currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Table snapshot should be the delete snapshot", - deleteSnapshot, - readMetadata().currentSnapshot()); + assertThat(version()).isEqualTo(1); + assertThat(readMetadata().currentSnapshot().allManifests(table.io())).hasSize(1); + assertThat(readMetadata().currentSnapshot()).isEqualTo(deleteSnapshot); validateManifestEntries( readMetadata().currentSnapshot().allManifests(table.io()).get(0), ids(deleteSnapshot.snapshotId(), appendSnapshot.snapshotId()), files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); - Assert.assertEquals( - "Table should have a snapshot for each operation", 2, readMetadata().snapshots().size()); + assertThat(readMetadata().snapshots()).hasSize(2); validateManifestEntries( readMetadata().snapshots().get(0).allManifests(table.io()).get(0), ids(appendSnapshot.snapshotId(), appendSnapshot.snapshotId()), @@ -199,165 +173,151 @@ public void testMultipleOperationTransactionFromTable() { statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testDetectsUncommittedChange() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B); // not committed - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); - Assertions.assertThatThrownBy(txn::newDelete) + assertThatThrownBy(txn::newDelete) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot create new DeleteFiles: last operation has not committed"); } - @Test + @TestTemplate public void testDetectsUncommittedChangeOnCommit() { - Assert.assertEquals("Table should be on version 0", 0, (int) version()); + assertThat(version()).isEqualTo(0); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B); // not committed - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 0 after txn create", 0, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(0); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(IllegalStateException.class) .hasMessage("Cannot commit transaction: last operation has not committed"); } - @Test + @TestTemplate public void testTransactionConflict() { // set retries to 0 to catch the failure table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "0").commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after append", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); // cause the transaction commit to fail table.ops().failCommits(1); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); } - @Test + @TestTemplate public void testTransactionRetry() { // use only one retry table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "1").commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Set appendManifests = Sets.newHashSet(txn.table().currentSnapshot().allManifests(table.io())); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after append", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); // cause the transaction commit to fail table.ops().failCommits(1); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 2 after commit", 2, (int) version()); + assertThat(version()).isEqualTo(2); - Assert.assertEquals( - "Should reuse manifests from initial append commit", - appendManifests, - Sets.newHashSet(table.currentSnapshot().allManifests(table.io()))); + assertThat(Sets.newHashSet(table.currentSnapshot().allManifests(table.io()))) + .isEqualTo(appendManifests); } - @Test + @TestTemplate public void testTransactionRetryMergeAppend() { // use only one retry table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "1").commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Set appendManifests = Sets.newHashSet(txn.table().currentSnapshot().allManifests(table.io())); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after append", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); // cause the transaction commit to fail table.newAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); - Assert.assertEquals("Table should be on version 2 after real append", 2, (int) version()); + assertThat(version()).isEqualTo(2); Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests(table.io())); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 3 after commit", 3, (int) version()); + assertThat(version()).isEqualTo(3); Set expectedManifests = Sets.newHashSet(); expectedManifests.addAll(appendManifests); expectedManifests.addAll(conflictAppendManifests); - Assert.assertEquals( - "Should reuse manifests from initial append commit and conflicting append", - expectedManifests, - Sets.newHashSet(table.currentSnapshot().allManifests(table.io()))); + assertThat(Sets.newHashSet(table.currentSnapshot().allManifests(table.io()))) + .isEqualTo(expectedManifests); } - @Test + @TestTemplate public void testMultipleUpdateTransactionRetryMergeCleanup() { // use only one retry and aggressively merge manifests table @@ -366,59 +326,49 @@ public void testMultipleUpdateTransactionRetryMergeCleanup() { .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0") .commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.updateProperties().set("test-property", "test-value").commit(); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals( - "Append should create one manifest", - 1, - txn.table().currentSnapshot().allManifests(table.io()).size()); + assertThat(txn.table().currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile appendManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after append", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); // cause the transaction commit to fail table.newAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); - Assert.assertEquals("Table should be on version 2 after real append", 2, (int) version()); + assertThat(version()).isEqualTo(2); Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests(table.io())); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 3 after commit", 3, (int) version()); + assertThat(version()).isEqualTo(3); Set previousManifests = Sets.newHashSet(); previousManifests.add(appendManifest); previousManifests.addAll(conflictAppendManifests); - Assert.assertEquals( - "Should merge both commit manifests into a single manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); - Assert.assertFalse( - "Should merge both commit manifests into a new manifest", - previousManifests.contains(table.currentSnapshot().allManifests(table.io()).get(0))); + assertThat(table.currentSnapshot().allManifests(table.io())) + .hasSize(1) + .doesNotContainAnyElementsOf(previousManifests); - Assert.assertFalse( - "Append manifest should be deleted", new File(appendManifest.path()).exists()); + assertThat(new File(appendManifest.path())).doesNotExist(); } - @Test + @TestTemplate public void testTransactionRetrySchemaUpdate() { // use only one retry table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "1").commit(); @@ -434,18 +384,15 @@ public void testTransactionRetrySchemaUpdate() { table.updateSchema().addColumn("another-column", Types.IntegerType.get()).commit(); int conflictingSchemaId = table.schema().schemaId(); - Assert.assertEquals( - "Both schema IDs should be the same in order to cause a conflict", - conflictingSchemaId, - schemaId); + assertThat(schemaId).isEqualTo(conflictingSchemaId); // commit the transaction for adding "new-column" - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessage("Table metadata refresh is required"); } - @Test + @TestTemplate public void testTransactionRetryMergeCleanup() { // use only one retry and aggressively merge manifests table @@ -454,61 +401,50 @@ public void testTransactionRetryMergeCleanup() { .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0") .commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); TableMetadata base = readMetadata(); Transaction txn = table.newTransaction(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after txn create", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); txn.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals( - "Append should create one manifest", - 1, - txn.table().currentSnapshot().allManifests(table.io()).size()); + assertThat(txn.table().currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile appendManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 1 after append", 1, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(1); // cause the transaction commit to fail table.newAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); - Assert.assertEquals("Table should be on version 2 after real append", 2, (int) version()); + assertThat(version()).isEqualTo(2); Set conflictAppendManifests = Sets.newHashSet(table.currentSnapshot().allManifests(table.io())); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 3 after commit", 3, (int) version()); + assertThat(version()).isEqualTo(3); Set previousManifests = Sets.newHashSet(); previousManifests.add(appendManifest); previousManifests.addAll(conflictAppendManifests); - Assert.assertEquals( - "Should merge both commit manifests into a single manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); - Assert.assertFalse( - "Should merge both commit manifests into a new manifest", - previousManifests.contains(table.currentSnapshot().allManifests(table.io()).get(0))); - - Assert.assertFalse( - "Append manifest should be deleted", new File(appendManifest.path()).exists()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .hasSize(1) + .doesNotContainAnyElementsOf(previousManifests); + assertThat(new File(appendManifest.path())).doesNotExist(); } - @Test + @TestTemplate public void testTransactionRetryAndAppendManifestsWithoutSnapshotIdInheritance() throws Exception { // this test assumes append manifests are rewritten, which only happens in V1 tables - Assumptions.assumeThat(formatVersion).isEqualTo(1); + assumeThat(formatVersion).isEqualTo(1); // use only one retry and aggressively merge manifests table @@ -517,22 +453,18 @@ public void testTransactionRetryAndAppendManifestsWithoutSnapshotIdInheritance() .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0") .commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals("Table should be on version 2 after append", 2, (int) version()); - Assert.assertEquals( - "Append should create one manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile v1manifest = table.currentSnapshot().allManifests(table.io()).get(0); TableMetadata base = readMetadata(); // create a manifest append - OutputFile manifestLocation = - Files.localOutput("/tmp/" + UUID.randomUUID().toString() + ".avro"); + OutputFile manifestLocation = Files.localOutput("/tmp/" + UUID.randomUUID() + ".avro"); ManifestWriter writer = ManifestFiles.write(table.spec(), manifestLocation); try { writer.add(FILE_D); @@ -544,14 +476,10 @@ public void testTransactionRetryAndAppendManifestsWithoutSnapshotIdInheritance() txn.newAppend().appendManifest(writer.toManifestFile()).commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 2 after txn create", 2, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(2); - Assert.assertEquals( - "Append should have one merged manifest", - 1, - txn.table().currentSnapshot().allManifests(table.io()).size()); + assertThat(txn.table().currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile mergedManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); // find the initial copy of the appended manifest @@ -562,40 +490,34 @@ public void testTransactionRetryAndAppendManifestsWithoutSnapshotIdInheritance() path -> !v1manifest.path().contains(path) && !mergedManifest.path().contains(path))); - Assert.assertTrue( - "Transaction should hijack the delete of the original copied manifest", - ((BaseTransaction) txn).deletedFiles().contains(copiedAppendManifest)); - Assert.assertTrue( - "Copied append manifest should not be deleted yet", - new File(copiedAppendManifest).exists()); + assertThat(((BaseTransaction) txn).deletedFiles()) + .as("Transaction should hijack the delete of the original copied manifest") + .contains(copiedAppendManifest); + assertThat(new File(copiedAppendManifest)).exists(); // cause the transaction commit to fail and retry table.newAppend().appendFile(FILE_C).commit(); - Assert.assertEquals("Table should be on version 3 after real append", 3, (int) version()); + assertThat(version()).isEqualTo(3); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 4 after commit", 4, (int) version()); - - Assert.assertTrue( - "Transaction should hijack the delete of the original copied manifest", - ((BaseTransaction) txn).deletedFiles().contains(copiedAppendManifest)); - Assert.assertFalse( - "Append manifest should be deleted", new File(copiedAppendManifest).exists()); - Assert.assertTrue( - "Transaction should hijack the delete of the first merged manifest", - ((BaseTransaction) txn).deletedFiles().contains(mergedManifest.path())); - Assert.assertFalse( - "Append manifest should be deleted", new File(mergedManifest.path()).exists()); - - Assert.assertEquals( - "Should merge all commit manifests into a single manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(version()).isEqualTo(4); + + assertThat(((BaseTransaction) txn).deletedFiles()) + .as("Transaction should hijack the delete of the original copied manifest") + .contains(copiedAppendManifest); + + assertThat(new File(copiedAppendManifest)).doesNotExist(); + assertThat(((BaseTransaction) txn).deletedFiles()) + .as("Transaction should hijack the delete of the first merged manifest") + .contains(mergedManifest.path()); + assertThat(new File(mergedManifest.path())).doesNotExist(); + + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); } - @Test + @TestTemplate public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() throws Exception { // use only one retry and aggressively merge manifests table @@ -605,15 +527,12 @@ public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() th .set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true") .commit(); - Assert.assertEquals("Table should be on version 1", 1, (int) version()); + assertThat(version()).isEqualTo(1); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals("Table should be on version 2 after append", 2, (int) version()); - Assert.assertEquals( - "Append should create one manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(version()).isEqualTo(2); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); TableMetadata base = readMetadata(); @@ -622,46 +541,37 @@ public void testTransactionRetryAndAppendManifestsWithSnapshotIdInheritance() th ManifestFile appendManifest = writeManifestWithName("input.m0", FILE_D); txn.newAppend().appendManifest(appendManifest).commit(); - Assert.assertSame( - "Base metadata should not change when commit is created", base, readMetadata()); - Assert.assertEquals("Table should be on version 2 after txn create", 2, (int) version()); + assertThat(readMetadata()).isSameAs(base); + assertThat(version()).isEqualTo(2); - Assert.assertEquals( - "Append should have one merged manifest", - 1, - txn.table().currentSnapshot().allManifests(table.io()).size()); + assertThat(txn.table().currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile mergedManifest = txn.table().currentSnapshot().allManifests(table.io()).get(0); // cause the transaction commit to fail and retry table.newAppend().appendFile(FILE_C).commit(); - Assert.assertEquals("Table should be on version 3 after real append", 3, (int) version()); + assertThat(version()).isEqualTo(3); txn.commitTransaction(); - Assert.assertEquals("Table should be on version 4 after commit", 4, (int) version()); + assertThat(version()).isEqualTo(4); - Assert.assertTrue( - "Transaction should hijack the delete of the original append manifest", - ((BaseTransaction) txn).deletedFiles().contains(appendManifest.path())); - Assert.assertFalse( - "Append manifest should be deleted", new File(appendManifest.path()).exists()); + assertThat(((BaseTransaction) txn).deletedFiles()) + .as("Transaction should hijack the delete of the original append manifest") + .contains(appendManifest.path()); + assertThat(new File(appendManifest.path())).doesNotExist(); - Assert.assertTrue( - "Transaction should hijack the delete of the first merged manifest", - ((BaseTransaction) txn).deletedFiles().contains(mergedManifest.path())); - Assert.assertFalse( - "Merged append manifest should be deleted", new File(mergedManifest.path()).exists()); + assertThat(((BaseTransaction) txn).deletedFiles()) + .as("Transaction should hijack the delete of the first merged manifest") + .contains(mergedManifest.path()); + assertThat(new File(appendManifest.path())).doesNotExist(); - Assert.assertEquals( - "Should merge all commit manifests into a single manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); } - @Test + @TestTemplate public void testTransactionNoCustomDeleteFunc() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .newTransaction() @@ -673,7 +583,7 @@ public void testTransactionNoCustomDeleteFunc() { .hasMessage("Cannot set delete callback more than once"); } - @Test + @TestTemplate public void testTransactionFastAppends() { table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); @@ -686,10 +596,10 @@ public void testTransactionFastAppends() { txn.commitTransaction(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Expected 2 manifests", 2, manifests.size()); + assertThat(manifests).hasSize(2); } - @Test + @TestTemplate public void testTransactionRewriteManifestsAppendedDirectly() throws IOException { Table table = load(); @@ -706,7 +616,7 @@ public void testTransactionRewriteManifestsAppendedDirectly() throws IOException long secondSnapshotId = table.currentSnapshot().snapshotId(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 2 manifests after 2 appends", 2, manifests.size()); + assertThat(manifests).hasSize(2); ManifestFile newManifest = writeManifest( @@ -726,11 +636,10 @@ public void testTransactionRewriteManifestsAppendedDirectly() throws IOException long finalSnapshotId = table.currentSnapshot().snapshotId(); long finalSnapshotTimestamp = System.currentTimeMillis(); - Assert.assertTrue( - "Append manifest should not be deleted", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); List finalManifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should have 1 final manifest", 1, finalManifests.size()); + assertThat(finalManifests).hasSize(1); validateManifestEntries( finalManifests.get(0), @@ -743,30 +652,29 @@ public void testTransactionRewriteManifestsAppendedDirectly() throws IOException table.expireSnapshots().expireOlderThan(finalSnapshotTimestamp + 1).retainLast(1).commit(); - Assert.assertFalse( - "Append manifest should be deleted on expiry", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).doesNotExist(); } - @Test + @TestTemplate public void testSimpleTransactionNotDeletingMetadataOnUnknownSate() throws IOException { Table table = TestTables.tableWithCommitSucceedButStateUnknown(tableDir, "test"); Transaction transaction = table.newTransaction(); transaction.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThatThrownBy(transaction::commitTransaction) + assertThatThrownBy(transaction::commitTransaction) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("datacenter on fire"); // Make sure metadata files still exist Snapshot current = table.currentSnapshot(); List manifests = current.allManifests(table.io()); - Assert.assertEquals("Should have 1 manifest file", 1, manifests.size()); - Assert.assertTrue("Manifest file should exist", new File(manifests.get(0).path()).exists()); - Assert.assertEquals("Should have 2 files in metadata", 2, countAllMetadataFiles(tableDir)); + assertThat(manifests).hasSize(1); + assertThat(new File(manifests.get(0).path())).exists(); + assertThat(countAllMetadataFiles(tableDir)).isEqualTo(2); } - @Test + @TestTemplate public void testTransactionRecommit() { // update table settings to merge when there are 3 manifests table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "3").commit(); @@ -781,18 +689,14 @@ public void testTransactionRecommit() { AppendFiles append = transaction.newAppend().appendFile(FILE_D); Snapshot pending = append.apply(); - Assert.assertEquals( - "Should produce 1 pending merged manifest", 1, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(1); // because a merge happened, the appended manifest is deleted the by append operation append.commit(); // concurrently commit FILE_A without a transaction to cause the previous append to retry table.newAppend().appendFile(FILE_C).commit(); - Assert.assertEquals( - "Should produce 1 committed merged manifest", - 1, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); transaction.commitTransaction(); @@ -807,9 +711,7 @@ public void testTransactionRecommit() { FILE_C.path().toString(), FILE_D.path().toString()); - Assert.assertEquals("Should contain all committed files", expectedPaths, paths); - - Assert.assertEquals( - "Should produce 2 manifests", 2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(paths).isEqualTo(expectedPaths); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); } } From 29648efb706281b3676de8f92f4c0af759fac9d9 Mon Sep 17 00:00:00 2001 From: Jay Chia <17691182+jaychia@users.noreply.github.com> Date: Wed, 20 Mar 2024 04:01:21 -0700 Subject: [PATCH 0150/1019] Docs: Add Daft into Iceberg documentation (#9836) --- docs/docs/daft.md | 145 ++++++++++++++++++++++++++++++++++++++++++++++ docs/mkdocs.yml | 1 + 2 files changed, 146 insertions(+) create mode 100644 docs/docs/daft.md diff --git a/docs/docs/daft.md b/docs/docs/daft.md new file mode 100644 index 000000000000..da78b7eb6ccd --- /dev/null +++ b/docs/docs/daft.md @@ -0,0 +1,145 @@ +--- +title: "Daft" +--- + + +# Daft + +[Daft](www.getdaft.io) is a distributed query engine written in Python and Rust, two fast-growing ecosystems in the data engineering and machine learning industry. + +It exposes its flavor of the familiar [Python DataFrame API](https://www.getdaft.io/projects/docs/en/latest/api_docs/dataframe.html) which is a common abstraction over querying tables of data in the Python data ecosystem. + +Daft DataFrames are a powerful interface to power use-cases across ML/AI training, batch inference, feature engineering and traditional analytics. Daft's tight integration with Iceberg unlocks novel capabilities for both traditional analytics and Pythonic ML workloads on your data catalog. + +## Enabling Iceberg support in Daft + +[PyIceberg](https://py.iceberg.apache.org/) supports reading of Iceberg tables into Daft DataFrames. + +To use Iceberg with Daft, ensure that the [PyIceberg](https://py.iceberg.apache.org/) library is also installed in your current Python environment. + +``` +pip install getdaft pyiceberg +``` + +## Querying Iceberg using Daft + +Daft interacts natively with [PyIceberg](https://py.iceberg.apache.org/) to read Iceberg tables. + +### Reading Iceberg tables + +**Setup Steps** + +To follow along with this code, first create an Iceberg table following [the Spark Quickstart tutorial](https://iceberg.apache.org/spark-quickstart/). PyIceberg must then be correctly configured by ensuring that the `~/.pyiceberg.yaml` file contains an appropriate catalog entry: + +``` +catalog: + default: + # URL to the Iceberg REST server Docker container + uri: http://localhost:8181 + # URL and credentials for the MinIO Docker container + s3.endpoint: http://localhost:9000 + s3.access-key-id: admin + s3.secret-access-key: password +``` + +Here is how the Iceberg table `demo.nyc.taxis` can be loaded into Daft: + +``` py +import daft +from pyiceberg.catalog import load_catalog + +# Configure Daft to use the local MinIO Docker container for any S3 operations +daft.set_planning_config( + default_io_config=daft.io.IOConfig( + s3=daft.io.S3Config(endpoint_url="http://localhost:9000"), + ) +) + +# Load a PyIceberg table into Daft, and show the first few rows +table = load_catalog("default").load_table("nyc.taxis") +df = daft.read_iceberg(table) +df.show() +``` + +``` +╭───────────┬─────────┬───────────────┬─────────────┬────────────────────╮ +│ vendor_id ┆ trip_id ┆ trip_distance ┆ fare_amount ┆ store_and_fwd_flag │ +│ --- ┆ --- ┆ --- ┆ --- ┆ --- │ +│ Int64 ┆ Int64 ┆ Float32 ┆ Float64 ┆ Utf8 │ +╞═══════════╪═════════╪═══════════════╪═════════════╪════════════════════╡ +│ 1 ┆ 1000371 ┆ 1.8 ┆ 15.32 ┆ N │ +├╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ +│ 1 ┆ 1000374 ┆ 8.4 ┆ 42.13 ┆ Y │ +├╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ +│ 2 ┆ 1000372 ┆ 2.5 ┆ 22.15 ┆ N │ +├╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ +│ 2 ┆ 1000373 ┆ 0.9 ┆ 9.01 ┆ N │ +╰───────────┴─────────┴───────────────┴─────────────┴────────────────────╯ + +(Showing first 4 of 4 rows) +``` + +Note that the operation above will produce a warning from PyIceberg that "no partition filter was specified" and that "this will result in a full table scan". Any filter operations on the Daft dataframe, `df`, will [push down the filters](https://iceberg.apache.org/docs/latest/performance/#data-filtering), correctly account for [hidden partitioning](https://iceberg.apache.org/docs/latest/partitioning/), and utilize [table statistics to inform query planning](https://iceberg.apache.org/docs/latest/performance/#scan-planning) for efficient reads. + +Let's try the above query again, but this time with a filter applied on the table's partition column `"vendor_id"` which Daft will correctly use to elide a full table scan. + +``` py +df = df.where(df["vendor_id"] > 1) +df.show() +``` + +``` +╭───────────┬─────────┬───────────────┬─────────────┬────────────────────╮ +│ vendor_id ┆ trip_id ┆ trip_distance ┆ fare_amount ┆ store_and_fwd_flag │ +│ --- ┆ --- ┆ --- ┆ --- ┆ --- │ +│ Int64 ┆ Int64 ┆ Float32 ┆ Float64 ┆ Utf8 │ +╞═══════════╪═════════╪═══════════════╪═════════════╪════════════════════╡ +│ 2 ┆ 1000372 ┆ 2.5 ┆ 22.15 ┆ N │ +├╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌┼╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌╌┤ +│ 2 ┆ 1000373 ┆ 0.9 ┆ 9.01 ┆ N │ +╰───────────┴─────────┴───────────────┴─────────────┴────────────────────╯ + +(Showing first 2 of 2 rows) +``` + +### Type compatibility + +Daft and Iceberg have compatible type systems. Here are how types are converted across the two systems. + + +| Iceberg | Daft | +|---------|------| +| **Primitive Types** | +| `boolean` | [`daft.DataType.bool()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.bool) | +| `int` | [`daft.DataType.int32()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.int32) | +| `long` | [`daft.DataType.int64()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.int64) | +| `float` | [`daft.DataType.float32()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.float32) | +| `double` | [`daft.DataType.float64()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.float64) | +| `decimal(precision, scale)` | [`daft.DataType.decimal128(precision, scale)`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.decimal128) | +| `date` | [`daft.DataType.date()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.date) | +| `time` | [`daft.DataType.time(timeunit="us")`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.int64) | +| `timestamp` | [`daft.DataType.timestamp(timeunit="us", timezone=None)`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.timestamp) | +| `timestampz` | [`daft.DataType.timestamp(timeunit="us", timezone="UTC")`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.timestamp) | +| `string` | [`daft.DataType.string()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.string) | +| `uuid` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.binary) | +| `fixed(L)` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.binary) | +| `binary` | [`daft.DataType.binary()`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.binary) | +| **Nested Types** | +| `struct(**fields)` | [`daft.DataType.struct(**fields)`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.struct) | +| `list(child_type)` | [`daft.DataType.list(child_type)`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.list) | +| `map(K, V)` | [`daft.DataType.map(K, V)`](https://www.getdaft.io/projects/docs/en/latest/api_docs/datatype.html#daft.DataType.map) | diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index a8804ed04dc4..a803f4585e41 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -51,6 +51,7 @@ nav: - flink-configuration.md - hive.md - Trino: https://trino.io/docs/current/connector/iceberg.html + - Daft: daft.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 22d26c51fd671de5e2e92d49622e394ffec75825 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Thu, 21 Mar 2024 00:30:19 +0900 Subject: [PATCH 0151/1019] Core: Migrate tests to JUnit5 (#9994) --- .../org/apache/iceberg/TestDeleteFiles.java | 115 ++-- .../org/apache/iceberg/TestFastAppend.java | 307 ++++------ .../org/apache/iceberg/TestMergeAppend.java | 561 +++++++----------- .../org/apache/iceberg/TestOverwrite.java | 104 ++-- .../iceberg/TestOverwriteWithValidation.java | 226 ++++--- .../org/apache/iceberg/TestRewriteFiles.java | 304 +++++----- 6 files changed, 675 insertions(+), 942 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index 63fc7010c49c..18e3de240170 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -19,30 +19,29 @@ package org.apache.iceberg; 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.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; -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.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeWrapper; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestDeleteFiles extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeleteFiles extends TestBase { private static final DataFile DATA_FILE_BUCKET_0_IDS_0_2 = DataFiles.builder(SPEC) @@ -78,36 +77,31 @@ public class TestDeleteFiles extends TableTestBase { )) .build(); - private final String branch; + @Parameter(index = 1) + private String branch; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} - }; + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}); } - public TestDeleteFiles(int formatVersion, String branch) { - super(formatVersion); - this.branch = branch; - } - - @Test + @TestTemplate public void testMultipleDeletes() { commit( table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C), branch); Snapshot append = latestSnapshot(readMetadata(), branch); - Assert.assertEquals("Metadata should be at version 1", 1L, (long) version()); + assertThat(version()).isEqualTo(1); validateSnapshot(null, append, FILE_A, FILE_B, FILE_C); commit(table, table.newDelete().deleteFile(FILE_A), branch); Snapshot delete1 = latestSnapshot(readMetadata(), branch); - Assert.assertEquals("Metadata should be at version 2", 2L, (long) version()); - Assert.assertEquals("Should have 1 manifest", 1, delete1.allManifests(FILE_IO).size()); + assertThat(version()).isEqualTo(2); + assertThat(delete1.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( delete1.allManifests(table.io()).get(0), ids(delete1.snapshotId(), append.snapshotId(), append.snapshotId()), @@ -115,8 +109,8 @@ public void testMultipleDeletes() { statuses(Status.DELETED, Status.EXISTING, Status.EXISTING)); Snapshot delete2 = commit(table, table.newDelete().deleteFile(FILE_B), branch); - Assert.assertEquals("Metadata should be at version 3", 3L, (long) version()); - Assert.assertEquals("Should have 1 manifest", 1, delete2.allManifests(FILE_IO).size()); + assertThat(version()).isEqualTo(3); + assertThat(delete2.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( delete2.allManifests(FILE_IO).get(0), ids(delete2.snapshotId(), append.snapshotId()), @@ -124,7 +118,7 @@ public void testMultipleDeletes() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { PartitionSpec spec = table.spec(); @@ -169,7 +163,7 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { table.newFastAppend().appendFile(firstDataFile).appendFile(secondDataFile), branch); - Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests(FILE_IO).size()); + assertThat(initialSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( initialSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), initialSnapshot.snapshotId()), @@ -178,7 +172,7 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { // delete the first data file Snapshot deleteSnapshot = commit(table, table.newDelete().deleteFile(firstDataFile), branch); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + assertThat(deleteSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( deleteSnapshot.allManifests(FILE_IO).get(0), ids(deleteSnapshot.snapshotId(), initialSnapshot.snapshotId()), @@ -190,7 +184,7 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { Snapshot finalSnapshot = commit(table, table.newDelete().deleteFromRowFilter(Expressions.lessThan("id", 7)), branch); - Assert.assertEquals("Should have 1 manifest", 1, finalSnapshot.allManifests(FILE_IO).size()); + assertThat(finalSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( finalSnapshot.allManifests(FILE_IO).get(0), ids(finalSnapshot.snapshotId()), @@ -198,7 +192,7 @@ public void testAlreadyDeletedFilesAreIgnoredDuringDeletesByRowFilter() { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testDeleteSomeFilesByRowFilterWithoutPartitionPredicates() { // add both data files Snapshot initialSnapshot = @@ -210,7 +204,7 @@ public void testDeleteSomeFilesByRowFilterWithoutPartitionPredicates() { .appendFile(DATA_FILE_BUCKET_0_IDS_8_10), branch); - Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests(FILE_IO).size()); + assertThat(initialSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( initialSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), initialSnapshot.snapshotId()), @@ -222,7 +216,7 @@ public void testDeleteSomeFilesByRowFilterWithoutPartitionPredicates() { commit( table, table.newDelete().deleteFromRowFilter(Expressions.greaterThan("id", 5)), branch); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + assertThat(deleteSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( deleteSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), deleteSnapshot.snapshotId()), @@ -230,7 +224,7 @@ public void testDeleteSomeFilesByRowFilterWithoutPartitionPredicates() { statuses(Status.EXISTING, Status.DELETED)); } - @Test + @TestTemplate public void testDeleteSomeFilesByRowFilterWithCombinedPredicates() { // add both data files Snapshot initialSnapshot = @@ -242,7 +236,7 @@ public void testDeleteSomeFilesByRowFilterWithCombinedPredicates() { .appendFile(DATA_FILE_BUCKET_0_IDS_8_10), branch); - Assert.assertEquals("Should have 1 manifest", 1, initialSnapshot.allManifests(FILE_IO).size()); + assertThat(initialSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( initialSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), initialSnapshot.snapshotId()), @@ -255,7 +249,7 @@ public void testDeleteSomeFilesByRowFilterWithCombinedPredicates() { Expression predicate = Expressions.and(partPredicate, rowPredicate); Snapshot deleteSnapshot = commit(table, table.newDelete().deleteFromRowFilter(predicate), branch); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + assertThat(deleteSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( deleteSnapshot.allManifests(FILE_IO).get(0), ids(initialSnapshot.snapshotId(), deleteSnapshot.snapshotId()), @@ -263,9 +257,9 @@ public void testDeleteSomeFilesByRowFilterWithCombinedPredicates() { statuses(Status.EXISTING, Status.DELETED)); } - @Test + @TestTemplate public void testCannotDeleteFileWhereNotAllRowsMatchPartitionFilter() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); table .updateSpec() @@ -285,7 +279,7 @@ public void testCannotDeleteFileWhereNotAllRowsMatchPartitionFilter() { commit(table, table.newFastAppend().appendFile(dataFile), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -295,18 +289,18 @@ public void testCannotDeleteFileWhereNotAllRowsMatchPartitionFilter() { .hasMessageStartingWith("Cannot delete file where some, but not all, rows match filter"); } - @Test + @TestTemplate public void testDeleteCaseSensitivity() { commit(table, table.newFastAppend().appendFile(DATA_FILE_BUCKET_0_IDS_0_2), branch); Expression rowFilter = Expressions.lessThan("iD", 5); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit(table, table.newDelete().deleteFromRowFilter(rowFilter), branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot find field 'iD'"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -319,7 +313,7 @@ public void testDeleteCaseSensitivity() { commit( table, table.newDelete().deleteFromRowFilter(rowFilter).caseSensitive(false), branch); - Assert.assertEquals("Should have 1 manifest", 1, deleteSnapshot.allManifests(FILE_IO).size()); + assertThat(deleteSnapshot.allManifests(FILE_IO)).hasSize(1); validateManifestEntries( deleteSnapshot.allManifests(FILE_IO).get(0), ids(deleteSnapshot.snapshotId()), @@ -327,7 +321,7 @@ public void testDeleteCaseSensitivity() { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testDeleteFilesOnIndependentBranches() { String testBranch = "testBranch"; table.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit(); @@ -355,7 +349,7 @@ public void testDeleteFilesOnIndependentBranches() { statuses(Status.EXISTING, Status.DELETED, Status.DELETED)); } - @Test + @TestTemplate public void testDeleteWithCollision() { Schema schema = new Schema(Types.NestedField.of(0, false, "x", Types.StringType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).identity("x").build(); @@ -367,9 +361,8 @@ public void testDeleteWithCollision() { PartitionData partitionTwo = new PartitionData(spec.partitionType()); partitionTwo.set(0, "BB"); - Assert.assertEquals( - StructLikeWrapper.forType(spec.partitionType()).set(partitionOne).hashCode(), - StructLikeWrapper.forType(spec.partitionType()).set(partitionTwo).hashCode()); + assertThat(StructLikeWrapper.forType(spec.partitionType()).set(partitionTwo).hashCode()) + .isEqualTo(StructLikeWrapper.forType(spec.partitionType()).set(partitionOne).hashCode()); DataFile testFileOne = DataFiles.builder(spec) @@ -394,10 +387,7 @@ public void testDeleteWithCollision() { .map(s -> ((PartitionData) s.partition()).copy()) .collect(Collectors.toList()); - Assert.assertEquals( - "We should have both partitions", - ImmutableList.of(partitionOne, partitionTwo), - beforeDeletePartitions); + assertThat(beforeDeletePartitions).containsExactly(partitionOne, partitionTwo); collisionTable.newDelete().deleteFromRowFilter(Expressions.equal("x", "BB")).commit(); @@ -406,13 +396,10 @@ public void testDeleteWithCollision() { .map(s -> ((PartitionData) s.partition()).copy()) .collect(Collectors.toList()); - Assert.assertEquals( - "We should have deleted partitionTwo", - ImmutableList.of(partitionOne), - afterDeletePartitions); + assertThat(afterDeletePartitions).containsExactly(partitionOne); } - @Test + @TestTemplate public void testDeleteValidateFileExistence() { commit(table, table.newFastAppend().appendFile(FILE_B), branch); Snapshot delete = @@ -423,12 +410,12 @@ public void testDeleteValidateFileExistence() { files(FILE_B), statuses(Status.DELETED)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit(table, table.newDelete().deleteFile(FILE_B).validateFilesExist(), branch)) .isInstanceOf(ValidationException.class); } - @Test + @TestTemplate public void testDeleteFilesNoValidation() { commit(table, table.newFastAppend().appendFile(FILE_B), branch); Snapshot delete1 = commit(table, table.newDelete().deleteFile(FILE_B), branch); @@ -439,8 +426,8 @@ public void testDeleteFilesNoValidation() { statuses(Status.DELETED)); Snapshot delete2 = commit(table, table.newDelete().deleteFile(FILE_B), branch); - Assertions.assertThat(delete2.allManifests(FILE_IO).isEmpty()).isTrue(); - Assertions.assertThat(delete2.removedDataFiles(FILE_IO).iterator().hasNext()).isFalse(); + assertThat(delete2.allManifests(FILE_IO)).isEmpty(); + assertThat(delete2.removedDataFiles(FILE_IO)).isEmpty(); } private static ByteBuffer longToBuffer(long value) { diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index a871b4e00c24..c3fc710ebffb 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -18,8 +18,12 @@ */ 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 java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.stream.Collectors; @@ -28,31 +32,23 @@ 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.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFastAppend extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestFastAppend(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFastAppend extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testEmptyTableAppend() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals( - "Table should start with last-sequence-number 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -68,14 +64,13 @@ public void testEmptyTableAppend() { "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); } - @Test + @TestTemplate public void testEmptyTableAppendManifest() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals( - "Table should start with last-sequence-number 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifest(FILE_A, FILE_B); table.newFastAppend().appendManifest(manifest).commit(); @@ -86,16 +81,13 @@ public void testEmptyTableAppendManifest() throws IOException { ManifestFile committedManifest = Iterables.getOnlyElement(snap.allManifests(FILE_IO)); if (formatVersion == 1) { - Assertions.assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); + assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); } else { - Assertions.assertThat(committedManifest.path()).isEqualTo(manifest.path()); + assertThat(committedManifest.path()).isEqualTo(manifest.path()); } // validate that the metadata summary is correct when using appendManifest - Assert.assertEquals( - "Summary metadata should include 2 added files", - "2", - snap.summary().get("added-data-files")); + assertThat(snap.summary()).containsEntry("added-data-files", "2"); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber()); V2Assert.assertEquals( @@ -105,14 +97,13 @@ public void testEmptyTableAppendManifest() throws IOException { "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); } - @Test + @TestTemplate public void testEmptyTableAppendFilesAndManifest() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals( - "Table should start with last-sequence-number 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifest(FILE_A, FILE_B); table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).appendManifest(manifest).commit(); @@ -135,9 +126,9 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { files(FILE_A, FILE_B)); if (formatVersion == 1) { - Assertions.assertThat(snap.allManifests(FILE_IO).get(1).path()).isNotEqualTo(manifest.path()); + assertThat(snap.allManifests(FILE_IO).get(1).path()).isNotEqualTo(manifest.path()); } else { - Assertions.assertThat(snap.allManifests(FILE_IO).get(1).path()).isEqualTo(manifest.path()); + assertThat(snap.allManifests(FILE_IO).get(1).path()).isEqualTo(manifest.path()); } V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap.sequenceNumber()); @@ -148,35 +139,32 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); } - @Test + @TestTemplate public void testNonEmptyTableAppend() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); TableMetadata base = readMetadata(); - Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNotNull(); List v2manifests = base.currentSnapshot().allManifests(FILE_IO); - Assert.assertEquals("Should have one existing manifest", 1, v2manifests.size()); + assertThat(v2manifests).hasSize(1); // prepare a new append Snapshot pending = table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).apply(); - Assert.assertNotEquals( - "Snapshots should have unique IDs", - base.currentSnapshot().snapshotId(), - pending.snapshotId()); + assertThat(pending.snapshotId()).isNotEqualTo(base.currentSnapshot().snapshotId()); validateSnapshot(base.currentSnapshot(), pending, FILE_C, FILE_D); } - @Test + @TestTemplate public void testNoMerge() { table.newAppend().appendFile(FILE_A).commit(); table.newFastAppend().appendFile(FILE_B).commit(); TableMetadata base = readMetadata(); - Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNotNull(); List v3manifests = base.currentSnapshot().allManifests(FILE_IO); - Assert.assertEquals("Should have 2 existing manifests", 2, v3manifests.size()); + assertThat(v3manifests).hasSize(2); // prepare a new append Snapshot pending = table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).apply(); @@ -186,12 +174,12 @@ public void testNoMerge() { ids.add(snapshot.snapshotId()); } ids.add(pending.snapshotId()); - Assert.assertEquals("Snapshots should have 3 unique IDs", 3, ids.size()); + assertThat(ids).hasSize(3); validateSnapshot(base.currentSnapshot(), pending, FILE_C, FILE_D); } - @Test + @TestTemplate public void testRefreshBeforeApply() { // load a new copy of the table that will not be refreshed by the commit Table stale = load(); @@ -199,9 +187,9 @@ public void testRefreshBeforeApply() { table.newAppend().appendFile(FILE_A).commit(); TableMetadata base = readMetadata(); - Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNotNull(); List v2manifests = base.currentSnapshot().allManifests(FILE_IO); - Assert.assertEquals("Should have 1 existing manifest", 1, v2manifests.size()); + assertThat(v2manifests).hasSize(1); // commit from the stale table AppendFiles append = stale.newFastAppend().appendFile(FILE_D); @@ -211,7 +199,7 @@ public void testRefreshBeforeApply() { validateSnapshot(base.currentSnapshot(), pending, FILE_D); } - @Test + @TestTemplate public void testRefreshBeforeCommit() { // commit from the stale table AppendFiles append = table.newFastAppend().appendFile(FILE_D); @@ -222,9 +210,9 @@ public void testRefreshBeforeCommit() { table.newAppend().appendFile(FILE_A).commit(); TableMetadata base = readMetadata(); - Assert.assertNotNull("Should have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNotNull(); List v2manifests = base.currentSnapshot().allManifests(FILE_IO); - Assert.assertEquals("Should have 1 existing manifest", 1, v2manifests.size()); + assertThat(v2manifests).hasSize(1); append.commit(); @@ -236,13 +224,10 @@ public void testRefreshBeforeCommit() { List committedManifests = Lists.newArrayList(committed.currentSnapshot().allManifests(FILE_IO)); committedManifests.removeAll(base.currentSnapshot().allManifests(FILE_IO)); - Assert.assertEquals( - "Should reused manifest created by apply", - pending.allManifests(FILE_IO).get(0), - committedManifests.get(0)); + assertThat(committedManifests.get(0)).isEqualTo(pending.allManifests(FILE_IO).get(0)); } - @Test + @TestTemplate public void testFailure() { // inject 5 failures TestTables.TestTableOperations ops = table.ops(); @@ -251,16 +236,16 @@ public void testFailure() { AppendFiles append = table.newFastAppend().appendFile(FILE_B); Snapshot pending = append.apply(); ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); - Assertions.assertThatThrownBy(append::commit) + assertThatThrownBy(append::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertFalse("Should clean up new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).doesNotExist(); } - @Test + @TestTemplate public void testAppendManifestCleanup() throws IOException { // inject 5 failures TestTables.TestTableOperations ops = table.ops(); @@ -270,25 +255,25 @@ public void testAppendManifestCleanup() throws IOException { AppendFiles append = table.newFastAppend().appendManifest(manifest); Snapshot pending = append.apply(); ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); if (formatVersion == 1) { - Assertions.assertThat(newManifest.path()).isNotEqualTo(manifest.path()); + assertThat(newManifest.path()).isNotEqualTo(manifest.path()); } else { - Assertions.assertThat(newManifest.path()).isEqualTo(manifest.path()); + assertThat(newManifest.path()).isEqualTo(manifest.path()); } - Assertions.assertThatThrownBy(append::commit) + assertThatThrownBy(append::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); if (formatVersion == 1) { - Assertions.assertThat(new File(newManifest.path())).doesNotExist(); + assertThat(new File(newManifest.path())).doesNotExist(); } else { - Assertions.assertThat(new File(newManifest.path())).exists(); + assertThat(new File(newManifest.path())).exists(); } } - @Test + @TestTemplate public void testRecoveryWithManifestList() { table.updateProperties().set(TableProperties.MANIFEST_LISTS_ENABLED, "true").commit(); @@ -299,20 +284,18 @@ public void testRecoveryWithManifestList() { AppendFiles append = table.newFastAppend().appendFile(FILE_B); Snapshot pending = append.apply(); ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); append.commit(); TableMetadata metadata = readMetadata(); validateSnapshot(null, metadata.currentSnapshot(), FILE_B); - Assert.assertTrue("Should commit same new manifest", new File(newManifest.path()).exists()); - Assert.assertTrue( - "Should commit the same new manifest", - metadata.currentSnapshot().allManifests(FILE_IO).contains(newManifest)); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); } - @Test + @TestTemplate public void testRecoveryWithoutManifestList() { table.updateProperties().set(TableProperties.MANIFEST_LISTS_ENABLED, "false").commit(); @@ -323,27 +306,25 @@ public void testRecoveryWithoutManifestList() { AppendFiles append = table.newFastAppend().appendFile(FILE_B); Snapshot pending = append.apply(); ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); append.commit(); TableMetadata metadata = readMetadata(); validateSnapshot(null, metadata.currentSnapshot(), FILE_B); - Assert.assertTrue("Should commit same new manifest", new File(newManifest.path()).exists()); - Assert.assertTrue( - "Should commit the same new manifest", - metadata.currentSnapshot().allManifests(FILE_IO).contains(newManifest)); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); } - @Test + @TestTemplate public void testAppendManifestWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); ManifestFile manifest = writeManifest(FILE_A, FILE_B); table.newFastAppend().appendManifest(manifest).commit(); @@ -351,7 +332,7 @@ public void testAppendManifestWithSnapshotIdInheritance() throws IOException { Snapshot snapshot = table.currentSnapshot(); List manifests = table.currentSnapshot().allManifests(FILE_IO); ManifestFile committedManifest = Iterables.getOnlyElement(manifests); - Assertions.assertThat(committedManifest.path()).isEqualTo(manifest.path()); + assertThat(committedManifest.path()).isEqualTo(manifest.path()); validateManifestEntries( manifests.get(0), @@ -360,32 +341,21 @@ public void testAppendManifestWithSnapshotIdInheritance() throws IOException { statuses(Status.ADDED, Status.ADDED)); // validate that the metadata summary is correct when using appendManifest - Assert.assertEquals( - "Summary metadata should include 2 added files", - "2", - snapshot.summary().get("added-data-files")); - Assert.assertEquals( - "Summary metadata should include 2 added records", - "2", - snapshot.summary().get("added-records")); - Assert.assertEquals( - "Summary metadata should include 2 files in total", - "2", - snapshot.summary().get("total-data-files")); - Assert.assertEquals( - "Summary metadata should include 2 records in total", - "2", - snapshot.summary().get("total-records")); + assertThat(snapshot.summary()) + .containsEntry("added-data-files", "2") + .containsEntry("added-records", "2") + .containsEntry("total-data-files", "2") + .containsEntry("total-records", "2"); } - @Test + @TestTemplate public void testAppendManifestFailureWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "1").commit(); @@ -396,36 +366,36 @@ public void testAppendManifestFailureWithSnapshotIdInheritance() throws IOExcept AppendFiles append = table.newAppend(); append.appendManifest(manifest); - Assertions.assertThatThrownBy(append::commit) + assertThatThrownBy(append::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertTrue("Append manifest should not be deleted", new File(manifest.path()).exists()); + assertThat(new File(manifest.path())).exists(); } - @Test + @TestTemplate public void testInvalidAppendManifest() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); ManifestFile manifestWithExistingFiles = writeManifest("manifest-file-1.avro", manifestEntry(Status.EXISTING, null, FILE_A)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.newFastAppend().appendManifest(manifestWithExistingFiles).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot append manifest with existing files"); ManifestFile manifestWithDeletedFiles = writeManifest("manifest-file-2.avro", manifestEntry(Status.DELETED, null, FILE_A)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.newFastAppend().appendManifest(manifestWithDeletedFiles).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot append manifest with deleted files"); } - @Test + @TestTemplate public void testPartitionSummariesOnUnpartitionedTable() { Table table = TestTables.create( @@ -447,7 +417,7 @@ public void testPartitionSummariesOnUnpartitionedTable() { .build()) .commit(); - Assertions.assertThat( + assertThat( table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet())) @@ -455,7 +425,7 @@ public void testPartitionSummariesOnUnpartitionedTable() { .isEmpty(); } - @Test + @TestTemplate public void testDefaultPartitionSummaries() { table.newFastAppend().appendFile(FILE_A).commit(); @@ -463,23 +433,14 @@ public void testDefaultPartitionSummaries() { table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet()); - Assert.assertEquals( - "Should include no partition summaries by default", 0, partitionSummaryKeys.size()); - - String summariesIncluded = - table - .currentSnapshot() - .summary() - .getOrDefault(SnapshotSummary.PARTITION_SUMMARY_PROP, "false"); - Assert.assertEquals( - "Should not set partition-summaries-included to true", "false", summariesIncluded); - - String changedPartitions = - table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); - Assert.assertEquals("Should set changed partition count", "1", changedPartitions); + assertThat(partitionSummaryKeys).isEmpty(); + + assertThat(table.currentSnapshot().summary()) + .doesNotContainKey(SnapshotSummary.PARTITION_SUMMARY_PROP) + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1"); } - @Test + @TestTemplate public void testIncludedPartitionSummaries() { table.updateProperties().set(TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, "1").commit(); @@ -489,32 +450,17 @@ public void testIncludedPartitionSummaries() { table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet()); - Assert.assertEquals("Should include a partition summary", 1, partitionSummaryKeys.size()); - - String summariesIncluded = - table - .currentSnapshot() - .summary() - .getOrDefault(SnapshotSummary.PARTITION_SUMMARY_PROP, "false"); - Assert.assertEquals( - "Should set partition-summaries-included to true", "true", summariesIncluded); - - String changedPartitions = - table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); - Assert.assertEquals("Should set changed partition count", "1", changedPartitions); - - String partitionSummary = - table - .currentSnapshot() - .summary() - .get(SnapshotSummary.CHANGED_PARTITION_PREFIX + "data_bucket=0"); - Assert.assertEquals( - "Summary should include 1 file with 1 record that is 10 bytes", - "added-data-files=1,added-records=1,added-files-size=10", - partitionSummary); + assertThat(partitionSummaryKeys).hasSize(1); + + assertThat(table.currentSnapshot().summary()) + .containsEntry(SnapshotSummary.PARTITION_SUMMARY_PROP, "true") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry( + SnapshotSummary.CHANGED_PARTITION_PREFIX + "data_bucket=0", + "added-data-files=1,added-records=1,added-files-size=10"); } - @Test + @TestTemplate public void testIncludedPartitionSummaryLimit() { table.updateProperties().set(TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, "1").commit(); @@ -524,69 +470,56 @@ public void testIncludedPartitionSummaryLimit() { table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet()); - Assert.assertEquals( - "Should include no partition summaries, over limit", 0, partitionSummaryKeys.size()); - - String summariesIncluded = - table - .currentSnapshot() - .summary() - .getOrDefault(SnapshotSummary.PARTITION_SUMMARY_PROP, "false"); - Assert.assertEquals( - "Should not set partition-summaries-included to true", "false", summariesIncluded); - - String changedPartitions = - table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); - Assert.assertEquals("Should set changed partition count", "2", changedPartitions); + assertThat(partitionSummaryKeys).isEmpty(); + + assertThat(table.currentSnapshot().summary()) + .doesNotContainKey(SnapshotSummary.PARTITION_SUMMARY_PROP) + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "2"); } - @Test + @TestTemplate public void testAppendToExistingBranch() { table.newFastAppend().appendFile(FILE_A).commit(); table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.newFastAppend().appendFile(FILE_B).toBranch("branch").commit(); - int branchSnapshot = 2; - Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); - Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(1); + assertThat(table.ops().current().ref("branch").snapshotId()).isEqualTo(2); } - @Test + @TestTemplate public void testAppendCreatesBranchIfNeeded() { table.newFastAppend().appendFile(FILE_A).commit(); table.newFastAppend().appendFile(FILE_B).toBranch("branch").commit(); - int branchSnapshot = 2; - Assert.assertEquals(table.currentSnapshot().snapshotId(), 1); - Assert.assertNotNull(table.ops().current().ref("branch")); - Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(1); + assertThat(table.ops().current().ref("branch")).isNotNull(); + assertThat(table.ops().current().ref("branch").snapshotId()).isEqualTo(2); } - @Test + @TestTemplate public void testAppendToBranchEmptyTable() { table.newFastAppend().appendFile(FILE_B).toBranch("branch").commit(); - int branchSnapshot = 1; - Assert.assertNull(table.currentSnapshot()); - Assert.assertNotNull(table.ops().current().ref("branch")); - Assert.assertEquals(table.ops().current().ref("branch").snapshotId(), branchSnapshot); + assertThat(table.currentSnapshot()).isNull(); + assertThat(table.ops().current().ref("branch")).isNotNull(); + assertThat(table.ops().current().ref("branch").snapshotId()).isEqualTo(1); } - @Test + @TestTemplate public void testAppendToNullBranchFails() { - Assertions.assertThatThrownBy(() -> table.newFastAppend().appendFile(FILE_A).toBranch(null)) + assertThatThrownBy(() -> table.newFastAppend().appendFile(FILE_A).toBranch(null)) .as("Invalid branch") .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid branch name: null"); } - @Test + @TestTemplate public void testAppendToTagFails() { table.newFastAppend().appendFile(FILE_A).commit(); table.manageSnapshots().createTag("some-tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( - () -> table.newFastAppend().appendFile(FILE_A).toBranch("some-tag").commit()) + assertThatThrownBy(() -> table.newFastAppend().appendFile(FILE_A).toBranch("some-tag").commit()) .as("Invalid branch") .isInstanceOf(IllegalArgumentException.class) .hasMessage( diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 39c9ac4b6c21..17d6bd5a19ba 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -20,9 +20,12 @@ import static org.apache.iceberg.relocated.com.google.common.collect.Iterators.concat; 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 java.io.File; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.concurrent.Executors; @@ -31,55 +34,43 @@ import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.CommitFailedException; 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.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestMergeAppend extends TableTestBase { - private final String branch; - - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} - }; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMergeAppend extends TestBase { + @Parameter(index = 1) + private String branch; + + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}); } - public TestMergeAppend(int formatVersion, String branch) { - super(formatVersion); - this.branch = branch; - } - - @Test + @TestTemplate public void testEmptyTableAppend() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); Snapshot committedSnapshot = commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); - Assert.assertNotNull("Should create a snapshot", committedSnapshot); + assertThat(committedSnapshot).isNotNull(); V1Assert.assertEquals( "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - committedSnapshot.allManifests(table.io()).size()); + assertThat(committedSnapshot.allManifests(table.io())).hasSize(1); long snapshotId = committedSnapshot.snapshotId(); @@ -92,18 +83,18 @@ public void testEmptyTableAppend() { statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testEmptyTableAppendManifest() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifest(FILE_A, FILE_B); Snapshot committedSnapshot = commit(table, table.newAppend().appendManifest(manifest), branch); - Assert.assertNotNull("Should create a snapshot", committedSnapshot); + assertThat(committedSnapshot).isNotNull(); V1Assert.assertEquals( "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals( @@ -111,9 +102,9 @@ public void testEmptyTableAppendManifest() throws IOException { List manifests = committedSnapshot.allManifests(table.io()); ManifestFile committedManifest = Iterables.getOnlyElement(manifests); if (formatVersion == 1) { - Assertions.assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); + assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); } else { - Assertions.assertThat(committedManifest.path()).isEqualTo(manifest.path()); + assertThat(committedManifest.path()).isEqualTo(manifest.path()); } long snapshotId = committedSnapshot.snapshotId(); @@ -126,19 +117,16 @@ public void testEmptyTableAppendManifest() throws IOException { statuses(Status.ADDED, Status.ADDED)); // validate that the metadata summary is correct when using appendManifest - Assert.assertEquals( - "Summary metadata should include 2 added files", - "2", - committedSnapshot.summary().get("added-data-files")); + assertThat(committedSnapshot.summary()).containsEntry("added-data-files", "2"); } - @Test + @TestTemplate public void testEmptyTableAppendFilesAndManifest() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifest(FILE_A, FILE_B); Snapshot committedSnapshot = @@ -147,15 +135,12 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { table.newAppend().appendFile(FILE_C).appendFile(FILE_D).appendManifest(manifest), branch); - Assert.assertNotNull("Should create a snapshot", committedSnapshot); + assertThat(committedSnapshot).isNotNull(); V1Assert.assertEquals( "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); - Assert.assertEquals( - "Should create 2 manifests for initial write", - 2, - committedSnapshot.allManifests(table.io()).size()); + assertThat(committedSnapshot.allManifests(table.io())).hasSize(2); long snapshotId = committedSnapshot.snapshotId(); @@ -163,9 +148,9 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { ManifestFile committedManifest2 = committedSnapshot.allManifests(table.io()).get(1); if (formatVersion == 1) { - Assertions.assertThat(committedManifest2.path()).isNotEqualTo(manifest.path()); + assertThat(committedManifest2.path()).isNotEqualTo(manifest.path()); } else { - Assertions.assertThat(committedManifest2.path()).isEqualTo(manifest.path()); + assertThat(committedManifest2.path()).isEqualTo(manifest.path()); } validateManifest( @@ -185,13 +170,13 @@ public void testEmptyTableAppendFilesAndManifest() throws IOException { statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testAppendWithManifestScanExecutor() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); AtomicInteger scanThreadsIndex = new AtomicInteger(0); Snapshot snapshot = commit( @@ -212,20 +197,22 @@ public void testAppendWithManifestScanExecutor() { return thread; })), branch); - Assert.assertTrue("Thread should be created in provided pool", scanThreadsIndex.get() > 0); - Assert.assertNotNull("Should create a snapshot", snapshot); + assertThat(scanThreadsIndex.get()) + .as("Thread should be created in provided pool") + .isGreaterThan(0); + assertThat(snapshot).isNotNull(); } - @Test + @TestTemplate public void testMergeWithAppendFilesAndManifest() throws IOException { // merge all manifests for this test table.updateProperties().set("commit.manifest.min-count-to-merge", "1").commit(); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifest(FILE_A, FILE_B); Snapshot committedSnapshot = @@ -234,7 +221,7 @@ public void testMergeWithAppendFilesAndManifest() throws IOException { table.newAppend().appendFile(FILE_C).appendFile(FILE_D).appendManifest(manifest), branch); - Assert.assertNotNull("Should create a snapshot", committedSnapshot); + assertThat(committedSnapshot).isNotNull(); V1Assert.assertEquals( "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals( @@ -245,7 +232,7 @@ public void testMergeWithAppendFilesAndManifest() throws IOException { List manifests = committedSnapshot.allManifests(table.io()); ManifestFile committedManifest = Iterables.getOnlyElement(manifests); - Assertions.assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); + assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); validateManifest( committedManifest, @@ -256,17 +243,17 @@ public void testMergeWithAppendFilesAndManifest() throws IOException { statuses(Status.ADDED, Status.ADDED, Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testMergeWithExistingManifest() { // merge all manifests for this test table.updateProperties().set("commit.manifest.min-count-to-merge", "1").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); Snapshot commitBefore = commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); - Assert.assertNotNull("Should create a snapshot", commitBefore); + assertThat(commitBefore).isNotNull(); V1Assert.assertEquals( "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals( @@ -276,10 +263,7 @@ public void testMergeWithExistingManifest() { long baseId = commitBefore.snapshotId(); validateSnapshot(null, commitBefore, 1, FILE_A, FILE_B); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - commitBefore.allManifests(table.io()).size()); + assertThat(commitBefore.allManifests(table.io())).hasSize(1); ManifestFile initialManifest = commitBefore.allManifests(table.io()).get(0); validateManifest( initialManifest, @@ -296,13 +280,9 @@ public void testMergeWithExistingManifest() { V2Assert.assertEquals( "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); - Assert.assertEquals( - "Should contain 1 merged manifest for second write", - 1, - committedAfter.allManifests(table.io()).size()); + assertThat(committedAfter.allManifests(table.io())).hasSize(1); ManifestFile newManifest = committedAfter.allManifests(table.io()).get(0); - Assert.assertNotEquals( - "Should not contain manifest from initial write", initialManifest, newManifest); + assertThat(newManifest).isNotEqualTo(initialManifest); long snapshotId = committedAfter.snapshotId(); @@ -315,9 +295,9 @@ public void testMergeWithExistingManifest() { statuses(Status.ADDED, Status.ADDED, Status.EXISTING, Status.EXISTING)); } - @Test + @TestTemplate public void testManifestMergeMinCount() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); table .updateProperties() .set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "2") @@ -328,8 +308,8 @@ public void testManifestMergeMinCount() throws IOException { .commit(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifestWithName("FILE_A", FILE_A); ManifestFile manifest2 = writeManifestWithName("FILE_C", FILE_C); @@ -351,10 +331,7 @@ public void testManifestMergeMinCount() throws IOException { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - Assert.assertEquals( - "Should contain 2 merged manifest for first write", - 2, - snap1.allManifests(table.io()).size()); + assertThat(snap1.allManifests(table.io())).hasSize(2); validateManifest( snap1.allManifests(table.io()).get(0), dataSeqs(1L), @@ -391,10 +368,7 @@ public void testManifestMergeMinCount() throws IOException { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - Assert.assertEquals( - "Should contain 3 merged manifest for second write", - 3, - snap2.allManifests(table.io()).size()); + assertThat(snap2.allManifests(table.io())).hasSize(3); validateManifest( snap2.allManifests(table.io()).get(0), dataSeqs(2L), @@ -418,15 +392,12 @@ public void testManifestMergeMinCount() throws IOException { statuses(Status.EXISTING, Status.EXISTING, Status.EXISTING)); // validate that the metadata summary is correct when using appendManifest - Assert.assertEquals( - "Summary metadata should include 3 added files", - "3", - snap2.summary().get("added-data-files")); + assertThat(snap2.summary()).containsEntry("added-data-files", "3"); } - @Test + @TestTemplate public void testManifestsMergeIntoOne() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); Snapshot snap1 = commit(table, table.newAppend().appendFile(FILE_A), branch); TableMetadata base = readMetadata(); V2Assert.assertEquals("Snapshot sequence number should be 1", 1, snap1.sequenceNumber()); @@ -435,7 +406,7 @@ public void testManifestsMergeIntoOne() throws IOException { "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); long commitId1 = snap1.snapshotId(); - Assert.assertEquals("Should contain 1 manifest", 1, snap1.allManifests(table.io()).size()); + assertThat(snap1.allManifests(table.io())).hasSize(1); validateManifest( snap1.allManifests(table.io()).get(0), dataSeqs(1L), @@ -452,7 +423,7 @@ public void testManifestsMergeIntoOne() throws IOException { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - Assert.assertEquals("Should contain 2 manifests", 2, snap2.allManifests(table.io()).size()); + assertThat(snap2.allManifests(table.io())).hasSize(2); validateManifest( snap2.allManifests(table.io()).get(0), dataSeqs(2L), @@ -484,7 +455,7 @@ public void testManifestsMergeIntoOne() throws IOException { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); - Assert.assertEquals("Should contain 3 manifests", 3, snap3.allManifests(table.io()).size()); + assertThat(snap3.allManifests(table.io())).hasSize(3); long commitId3 = snap3.snapshotId(); validateManifest( snap3.allManifests(table.io()).get(0), @@ -527,8 +498,7 @@ public void testManifestsMergeIntoOne() throws IOException { "Table should end with last-sequence-number 0", 0, base.lastSequenceNumber()); long commitId4 = snap4.snapshotId(); - Assert.assertEquals( - "Should only contains 1 merged manifest", 1, snap4.allManifests(table.io()).size()); + assertThat(snap4.allManifests(table.io())).hasSize(1); validateManifest( snap4.allManifests(table.io()).get(0), dataSeqs(4L, 3L, 2L, 1L), @@ -538,14 +508,14 @@ public void testManifestsMergeIntoOne() throws IOException { statuses(Status.ADDED, Status.EXISTING, Status.EXISTING, Status.EXISTING)); } - @Test + @TestTemplate public void testManifestDoNotMergeMinCount() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); table.updateProperties().set("commit.manifest.min-count-to-merge", "4").commit(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.currentSnapshot()).isNull(); + assertThat(base.lastSequenceNumber()).isEqualTo(0); ManifestFile manifest = writeManifest(FILE_A, FILE_B); ManifestFile manifest2 = writeManifestWithName("FILE_C", FILE_C); @@ -560,14 +530,14 @@ public void testManifestDoNotMergeMinCount() throws IOException { .appendManifest(manifest3), branch); - Assert.assertNotNull("Should create a snapshot", committed); + assertThat(committed).isNotNull(); V1Assert.assertEquals( "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); V2Assert.assertEquals( "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); List manifests = committed.allManifests(table.io()); - Assertions.assertThat(manifests).hasSize(3); + assertThat(manifests).hasSize(3); ManifestFile committedManifest = manifests.get(0); ManifestFile committedManifest2 = manifests.get(1); @@ -576,13 +546,13 @@ public void testManifestDoNotMergeMinCount() throws IOException { long snapshotId = committed.snapshotId(); if (formatVersion == 1) { - Assertions.assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); - Assertions.assertThat(committedManifest2.path()).isNotEqualTo(manifest2.path()); - Assertions.assertThat(committedManifest3.path()).isNotEqualTo(manifest3.path()); + assertThat(committedManifest.path()).isNotEqualTo(manifest.path()); + assertThat(committedManifest2.path()).isNotEqualTo(manifest2.path()); + assertThat(committedManifest3.path()).isNotEqualTo(manifest3.path()); } else { - Assertions.assertThat(committedManifest.path()).isEqualTo(manifest.path()); - Assertions.assertThat(committedManifest2.path()).isEqualTo(manifest2.path()); - Assertions.assertThat(committedManifest3.path()).isEqualTo(manifest3.path()); + assertThat(committedManifest.path()).isEqualTo(manifest.path()); + assertThat(committedManifest2.path()).isEqualTo(manifest2.path()); + assertThat(committedManifest3.path()).isEqualTo(manifest3.path()); } validateManifest( @@ -608,19 +578,16 @@ public void testManifestDoNotMergeMinCount() throws IOException { statuses(Status.ADDED)); // validate that the metadata summary is correct when using appendManifest - Assert.assertEquals( - "Summary metadata should include 4 added files", - "4", - committed.summary().get("added-data-files")); + assertThat(committed.summary()).containsEntry("added-data-files", "4"); } - @Test + @TestTemplate public void testMergeWithExistingManifestAfterDelete() { // merge all manifests for this test table.updateProperties().set("commit.manifest.min-count-to-merge", "1").commit(); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); Snapshot snap = commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -628,8 +595,7 @@ public void testMergeWithExistingManifestAfterDelete() { TableMetadata base = readMetadata(); long baseId = snap.snapshotId(); - Assert.assertEquals( - "Should create 1 manifest for initial write", 1, snap.allManifests(table.io()).size()); + assertThat(snap.allManifests(table.io())).hasSize(1); ManifestFile initialManifest = snap.allManifests(table.io()).get(0); validateManifest( initialManifest, @@ -650,10 +616,7 @@ public void testMergeWithExistingManifestAfterDelete() { TableMetadata delete = readMetadata(); long deleteId = latestSnapshot(table, branch).snapshotId(); - Assert.assertEquals( - "Should create 1 filtered manifest for delete", - 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); ManifestFile deleteManifest = deleteSnapshot.allManifests(table.io()).get(0); validateManifest( @@ -674,13 +637,9 @@ public void testMergeWithExistingManifestAfterDelete() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should contain 1 merged manifest for second write", - 1, - committedSnapshot.allManifests(table.io()).size()); + assertThat(committedSnapshot.allManifests(table.io())).hasSize(1); ManifestFile newManifest = committedSnapshot.allManifests(table.io()).get(0); - Assert.assertNotEquals( - "Should not contain manifest from initial write", initialManifest, newManifest); + assertThat(newManifest).isNotEqualTo(initialManifest); long snapshotId = committedSnapshot.snapshotId(); @@ -692,13 +651,13 @@ public void testMergeWithExistingManifestAfterDelete() { statuses(Status.ADDED, Status.ADDED, Status.EXISTING)); } - @Test + @TestTemplate public void testMinMergeCount() { // only merge when there are at least 4 manifests table.updateProperties().set("commit.manifest.min-count-to-merge", "4").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); Snapshot snap1 = commit(table, table.newFastAppend().appendFile(FILE_A), branch); long idFileA = snap1.snapshotId(); @@ -708,18 +667,14 @@ public void testMinMergeCount() { long idFileB = snap2.snapshotId(); validateSnapshot(snap1, snap2, 2, FILE_B); - Assert.assertEquals( - "Should have 2 manifests from setup writes", 2, snap2.allManifests(table.io()).size()); + assertThat(snap2.allManifests(table.io())).hasSize(2); Snapshot snap3 = commit(table, table.newAppend().appendFile(FILE_C), branch); long idFileC = snap3.snapshotId(); validateSnapshot(snap2, snap3, 3, FILE_C); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should have 3 unmerged manifests", - 3, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(3); Set unmerged = Sets.newHashSet(latestSnapshot(table, branch).allManifests(table.io())); @@ -730,12 +685,10 @@ public void testMinMergeCount() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should contain 1 merged manifest after the 4th write", - 1, - committed.allManifests(table.io()).size()); + assertThat(committed.allManifests(table.io())).hasSize(1); + ManifestFile newManifest = committed.allManifests(table.io()).get(0); - Assert.assertFalse("Should not contain previous manifests", unmerged.contains(newManifest)); + assertThat(unmerged).doesNotContain(newManifest); long lastSnapshotId = committed.snapshotId(); @@ -748,13 +701,13 @@ public void testMinMergeCount() { statuses(Status.ADDED, Status.EXISTING, Status.EXISTING, Status.EXISTING)); } - @Test + @TestTemplate public void testMergeSizeTargetWithExistingManifest() { // use a small limit on manifest size to prevent merging table.updateProperties().set(TableProperties.MANIFEST_TARGET_SIZE_BYTES, "10").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); Snapshot snap = commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -762,8 +715,7 @@ public void testMergeSizeTargetWithExistingManifest() { TableMetadata base = readMetadata(); long baseId = snap.snapshotId(); - Assert.assertEquals( - "Should create 1 manifest for initial write", 1, snap.allManifests(table.io()).size()); + assertThat(snap.allManifests(table.io())).hasSize(1); ManifestFile initialManifest = snap.allManifests(table.io()).get(0); validateManifest( initialManifest, @@ -782,13 +734,9 @@ public void testMergeSizeTargetWithExistingManifest() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should contain 2 unmerged manifests after second write", - 2, - committed.allManifests(table.io()).size()); + assertThat(committed.allManifests(table.io())).hasSize(2); ManifestFile newManifest = committed.allManifests(table.io()).get(0); - Assert.assertNotEquals( - "Should not contain manifest from initial write", initialManifest, newManifest); + assertThat(newManifest).isNotEqualTo(initialManifest); long pendingId = committed.snapshotId(); validateManifest( @@ -808,7 +756,7 @@ public void testMergeSizeTargetWithExistingManifest() { statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testChangedPartitionSpec() { Snapshot snap = commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -816,8 +764,7 @@ public void testChangedPartitionSpec() { validateSnapshot(null, snap, 1, FILE_A, FILE_B); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should create 1 manifest for initial write", 1, snap.allManifests(table.io()).size()); + assertThat(snap.allManifests(table.io())).hasSize(1); ManifestFile initialManifest = snap.allManifests(table.io()).get(0); validateManifest( initialManifest, @@ -856,8 +803,7 @@ public void testChangedPartitionSpec() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should use 2 manifest files", 2, lastSnapshot.allManifests(table.io()).size()); + assertThat(lastSnapshot.allManifests(table.io())).hasSize(2); // new manifest comes first validateManifest( @@ -868,13 +814,13 @@ public void testChangedPartitionSpec() { files(newFileY), statuses(Status.ADDED)); - Assert.assertEquals( - "Second manifest should be the initial manifest with the old spec", - initialManifest, - lastSnapshot.allManifests(table.io()).get(1)); + assertThat(lastSnapshot.allManifests(table.io())) + .as("Second manifest should be the initial manifest with the old spec") + .element(1) + .isEqualTo(initialManifest); } - @Test + @TestTemplate public void testChangedPartitionSpecMergeExisting() { Snapshot snap1 = commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -888,7 +834,7 @@ public void testChangedPartitionSpecMergeExisting() { validateSnapshot(snap1, snap2, 2, FILE_B); TableMetadata base = readMetadata(); - Assert.assertEquals("Should contain 2 manifests", 2, snap2.allManifests(table.io()).size()); + assertThat(snap2.allManifests(table.io())).hasSize(2); ManifestFile manifest = snap2.allManifests(table.io()).get(0); // build the new spec using the table's schema, which uses fresh IDs @@ -919,11 +865,8 @@ public void testChangedPartitionSpecMergeExisting() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should use 2 manifest files", 2, lastSnapshot.allManifests(table.io()).size()); - Assert.assertFalse( - "First manifest should not be in the new snapshot", - lastSnapshot.allManifests(table.io()).contains(manifest)); + assertThat(lastSnapshot.allManifests(table.io())).hasSize(2); + assertThat(lastSnapshot.allManifests(table.io())).doesNotContain(manifest); validateManifest( lastSnapshot.allManifests(table.io()).get(0), @@ -941,11 +884,11 @@ public void testChangedPartitionSpecMergeExisting() { statuses(Status.EXISTING, Status.EXISTING)); } - @Test + @TestTemplate public void testFailure() { // merge all manifests for this test table.updateProperties().set("commit.manifest.min-count-to-merge", "1").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); Snapshot snap = commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -969,16 +912,16 @@ public void testFailure() { AppendFiles append = table.newAppend().appendFile(FILE_B); Snapshot pending = apply(append, branch); - Assert.assertEquals("Should merge to 1 manifest", 1, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(1); ManifestFile newManifest = pending.allManifests(table.io()).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); validateManifest( newManifest, ids(pending.snapshotId(), baseId), concat(files(FILE_B), files(initialManifest))); - Assertions.assertThatThrownBy(() -> commit(table, append, branch)) + assertThatThrownBy(() -> commit(table, append, branch)) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); @@ -986,10 +929,7 @@ public void testFailure() { "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should only contain 1 manifest file", - 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); validateManifest( latestSnapshot(table, branch).allManifests(table.io()).get(0), @@ -999,10 +939,10 @@ public void testFailure() { files(initialManifest), statuses(Status.ADDED)); - Assert.assertFalse("Should clean up new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).doesNotExist(); } - @Test + @TestTemplate public void testAppendManifestCleanup() throws IOException { // inject 5 failures TestTables.TestTableOperations ops = table.ops(); @@ -1012,14 +952,14 @@ public void testAppendManifestCleanup() throws IOException { AppendFiles append = table.newAppend().appendManifest(manifest); Snapshot pending = apply(append, branch); ManifestFile newManifest = pending.allManifests(table.io()).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); if (formatVersion == 1) { - Assertions.assertThat(newManifest.path()).isNotEqualTo(manifest.path()); + assertThat(newManifest.path()).isNotEqualTo(manifest.path()); } else { - Assertions.assertThat(newManifest.path()).isEqualTo(manifest.path()); + assertThat(newManifest.path()).isEqualTo(manifest.path()); } - Assertions.assertThatThrownBy(() -> commit(table, append, branch)) + assertThatThrownBy(() -> commit(table, append, branch)) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); V2Assert.assertEquals( @@ -1028,18 +968,18 @@ public void testAppendManifestCleanup() throws IOException { "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); if (formatVersion == 1) { - Assertions.assertThat(new File(newManifest.path())).doesNotExist(); + assertThat(new File(newManifest.path())).doesNotExist(); } else { - Assertions.assertThat(new File(newManifest.path())).exists(); + assertThat(new File(newManifest.path())).exists(); } } - @Test + @TestTemplate public void testRecovery() { // merge all manifests for this test table.updateProperties().set("commit.manifest.min-count-to-merge", "1").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); Snapshot current = commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -1063,10 +1003,10 @@ public void testRecovery() { AppendFiles append = table.newAppend().appendFile(FILE_B); Snapshot pending = apply(append, branch); - Assert.assertEquals("Should merge to 1 manifest", 1, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(1); ManifestFile newManifest = pending.allManifests(table.io()).get(0); - Assert.assertTrue("Should create new manifest", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); validateManifest( newManifest, ids(pending.snapshotId(), baseId), @@ -1088,14 +1028,10 @@ public void testRecovery() { "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); TableMetadata metadata = readMetadata(); - Assert.assertTrue("Should reuse the new manifest", new File(newManifest.path()).exists()); - Assert.assertEquals( - "Should commit the same new manifest during retry", - Lists.newArrayList(newManifest), - snapshot.allManifests(table.io())); - - Assert.assertEquals( - "Should only contain 1 merged manifest file", 1, snapshot.allManifests(table.io()).size()); + assertThat(new File(newManifest.path())).exists(); + assertThat(snapshot.allManifests(table.io())).containsExactly(newManifest); + + assertThat(snapshot.allManifests(table.io())).hasSize(1); ManifestFile manifestFile = snapshot.allManifests(table.io()).get(0); validateManifest( manifestFile, @@ -1106,15 +1042,15 @@ public void testRecovery() { statuses(Status.ADDED, Status.EXISTING)); } - @Test + @TestTemplate public void testAppendManifestWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); ManifestFile manifest = writeManifest(FILE_A, FILE_B); Snapshot snapshot = commit(table, table.newAppend().appendManifest(manifest), branch); @@ -1123,9 +1059,9 @@ public void testAppendManifestWithSnapshotIdInheritance() throws IOException { validateSnapshot(null, snapshot, 1, FILE_A, FILE_B); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals("Should have 1 committed manifest", 1, manifests.size()); + assertThat(manifests).hasSize(1); ManifestFile manifestFile = snapshot.allManifests(table.io()).get(0); - Assertions.assertThat(manifestFile.path()).isEqualTo(manifest.path()); + assertThat(manifestFile.path()).isEqualTo(manifest.path()); validateManifest( manifestFile, dataSeqs(1L, 1L), @@ -1135,33 +1071,22 @@ public void testAppendManifestWithSnapshotIdInheritance() throws IOException { statuses(Status.ADDED, Status.ADDED)); // validate that the metadata summary is correct when using appendManifest - Assert.assertEquals( - "Summary metadata should include 2 added files", - "2", - snapshot.summary().get("added-data-files")); - Assert.assertEquals( - "Summary metadata should include 2 added records", - "2", - snapshot.summary().get("added-records")); - Assert.assertEquals( - "Summary metadata should include 2 files in total", - "2", - snapshot.summary().get("total-data-files")); - Assert.assertEquals( - "Summary metadata should include 2 records in total", - "2", - snapshot.summary().get("total-records")); + assertThat(snapshot.summary()) + .containsEntry("added-data-files", "2") + .containsEntry("added-records", "2") + .containsEntry("total-data-files", "2") + .containsEntry("total-records", "2"); } - @Test + @TestTemplate public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); @@ -1171,7 +1096,7 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO long commitId1 = snap1.snapshotId(); validateSnapshot(null, snap1, 1, FILE_A, FILE_B); - Assert.assertEquals("Should have only 1 manifest", 1, snap1.allManifests(table.io()).size()); + assertThat(snap1.allManifests(table.io())).hasSize(1); validateManifest( snap1.allManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -1179,8 +1104,7 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO ids(commitId1, commitId1), files(FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED)); - Assert.assertTrue( - "Unmerged append manifest should not be deleted", new File(manifest1.path()).exists()); + assertThat(new File(manifest1.path())).exists(); ManifestFile manifest2 = writeManifestWithName("manifest-file-2.avro", FILE_C, FILE_D); Snapshot snap2 = commit(table, table.newAppend().appendManifest(manifest2), branch); @@ -1192,8 +1116,7 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Manifests should be merged into 1", 1, snap2.allManifests(table.io()).size()); + assertThat(snap2.allManifests(table.io())).hasSize(1); validateManifest( latestSnapshot(table, branch).allManifests(table.io()).get(0), dataSeqs(2L, 2L, 1L, 1L), @@ -1202,19 +1125,18 @@ public void testMergedAppendManifestCleanupWithSnapshotIdInheritance() throws IO files(FILE_C, FILE_D, FILE_A, FILE_B), statuses(Status.ADDED, Status.ADDED, Status.EXISTING, Status.EXISTING)); - Assert.assertFalse( - "Merged append manifest should be deleted", new File(manifest2.path()).exists()); + assertThat(new File(manifest2.path())).doesNotExist(); } - @Test + @TestTemplate public void testAppendManifestFailureWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); table.updateProperties().set(TableProperties.COMMIT_NUM_RETRIES, "1").commit(); @@ -1225,40 +1147,40 @@ public void testAppendManifestFailureWithSnapshotIdInheritance() throws IOExcept AppendFiles append = table.newAppend(); append.appendManifest(manifest); - Assertions.assertThatThrownBy(() -> commit(table, append, branch)) + assertThatThrownBy(() -> commit(table, append, branch)) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertTrue("Append manifest should not be deleted", new File(manifest.path()).exists()); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); + assertThat(new File(manifest.path())).exists(); } - @Test + @TestTemplate public void testInvalidAppendManifest() throws IOException { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.currentSnapshot()); + assertThat(base.currentSnapshot()).isNull(); ManifestFile manifestWithExistingFiles = writeManifest("manifest-file-1.avro", manifestEntry(Status.EXISTING, null, FILE_A)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit(table, table.newAppend().appendManifest(manifestWithExistingFiles), branch)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot append manifest with existing files"); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); ManifestFile manifestWithDeletedFiles = writeManifest("manifest-file-2.avro", manifestEntry(Status.DELETED, null, FILE_A)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit(table, table.newAppend().appendManifest(manifestWithDeletedFiles), branch)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot append manifest with deleted files"); - Assert.assertEquals("Last sequence number should be 0", 0, readMetadata().lastSequenceNumber()); + assertThat(readMetadata().lastSequenceNumber()).isEqualTo(0); } - @Test + @TestTemplate public void testUpdatePartitionSpecFieldIdsForV1Table() { TableMetadata base = readMetadata(); @@ -1273,35 +1195,35 @@ public void testUpdatePartitionSpecFieldIdsForV1Table() { // commit the new partition spec to the table manually table.ops().commit(base, base.updatePartitionSpec(newSpec)); - Assert.assertEquals("Last sequence number should be 0", 0, base.lastSequenceNumber()); + assertThat(base.lastSequenceNumber()).isEqualTo(0); List partitionSpecs = table.ops().current().specs(); PartitionSpec partitionSpec = partitionSpecs.get(0); - Assert.assertEquals(1000, partitionSpec.lastAssignedFieldId()); + assertThat(partitionSpec.lastAssignedFieldId()).isEqualTo(1000); Types.StructType structType = partitionSpec.partitionType(); List fields = structType.fields(); - Assert.assertEquals(1, fields.size()); - Assert.assertEquals("data_bucket", fields.get(0).name()); - Assert.assertEquals(1000, fields.get(0).fieldId()); + assertThat(fields).hasSize(1); + assertThat(fields.get(0).name()).isEqualTo("data_bucket"); + assertThat(fields.get(0).fieldId()).isEqualTo(1000); partitionSpec = partitionSpecs.get(1); - Assert.assertEquals(1003, partitionSpec.lastAssignedFieldId()); + assertThat(partitionSpec.lastAssignedFieldId()).isEqualTo(1003); structType = partitionSpec.partitionType(); fields = structType.fields(); - Assert.assertEquals(4, fields.size()); - Assert.assertEquals("id_bucket", fields.get(0).name()); - Assert.assertEquals(1000, fields.get(0).fieldId()); - Assert.assertEquals("data", fields.get(1).name()); - Assert.assertEquals(1001, fields.get(1).fieldId()); - Assert.assertEquals("data_bucket", fields.get(2).name()); - Assert.assertEquals(1002, fields.get(2).fieldId()); - Assert.assertEquals("data_partition", fields.get(3).name()); - Assert.assertEquals(1003, fields.get(3).fieldId()); + assertThat(fields).hasSize(4); + assertThat(fields.get(0).name()).isEqualTo("id_bucket"); + assertThat(fields.get(0).fieldId()).isEqualTo(1000); + assertThat(fields.get(1).name()).isEqualTo("data"); + assertThat(fields.get(1).fieldId()).isEqualTo(1001); + assertThat(fields.get(2).name()).isEqualTo("data_bucket"); + assertThat(fields.get(2).fieldId()).isEqualTo(1002); + assertThat(fields.get(3).name()).isEqualTo("data_partition"); + assertThat(fields.get(3).fieldId()).isEqualTo(1003); } - @Test + @TestTemplate public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { Snapshot snap = commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -1309,8 +1231,7 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { validateSnapshot(null, snap, 1, FILE_A); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should create 1 manifest for initial write", 1, snap.allManifests(table.io()).size()); + assertThat(snap.allManifests(table.io())).hasSize(1); ManifestFile initialManifest = snap.allManifests(table.io()).get(0); validateManifest( initialManifest, @@ -1349,8 +1270,7 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { V1Assert.assertEquals( "Table should end with last-sequence-number 0", 0, readMetadata().lastSequenceNumber()); - Assert.assertEquals( - "Should use 2 manifest files", 2, committedSnapshot.allManifests(table.io()).size()); + assertThat(committedSnapshot.allManifests(table.io())).hasSize(2); // new manifest comes first validateManifest( @@ -1361,10 +1281,10 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { files(newFile), statuses(Status.ADDED)); - Assert.assertEquals( - "Second manifest should be the initial manifest with the old spec", - initialManifest, - committedSnapshot.allManifests(table.io()).get(1)); + assertThat(committedSnapshot.allManifests(table.io())) + .as("Second manifest should be the initial manifest with the old spec") + .element(1) + .isEqualTo(initialManifest); // field ids of manifest entries in two manifests with different specs of the same source field // should be different @@ -1375,11 +1295,11 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { .next(); Types.NestedField field = ((PartitionData) entry.file().partition()).getPartitionType().fields().get(0); - Assert.assertEquals(1000, field.fieldId()); - Assert.assertEquals("id_bucket", field.name()); + assertThat(field.fieldId()).isEqualTo(1000); + assertThat(field.name()).isEqualTo("id_bucket"); field = ((PartitionData) entry.file().partition()).getPartitionType().fields().get(1); - Assert.assertEquals(1001, field.fieldId()); - Assert.assertEquals("data_bucket", field.name()); + assertThat(field.fieldId()).isEqualTo(1001); + assertThat(field.name()).isEqualTo("data_bucket"); entry = ManifestFiles.read(committedSnapshot.allManifests(table.io()).get(1), FILE_IO) @@ -1387,11 +1307,11 @@ public void testManifestEntryFieldIdsForChangedPartitionSpecForV1Table() { .iterator() .next(); field = ((PartitionData) entry.file().partition()).getPartitionType().fields().get(0); - Assert.assertEquals(1000, field.fieldId()); - Assert.assertEquals("data_bucket", field.name()); + assertThat(field.fieldId()).isEqualTo(1000); + assertThat(field.name()).isEqualTo("data_bucket"); } - @Test + @TestTemplate public void testDefaultPartitionSummaries() { table.newFastAppend().appendFile(FILE_A).commit(); @@ -1399,23 +1319,14 @@ public void testDefaultPartitionSummaries() { table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet()); - Assert.assertEquals( - "Should include no partition summaries by default", 0, partitionSummaryKeys.size()); - - String summariesIncluded = - table - .currentSnapshot() - .summary() - .getOrDefault(SnapshotSummary.PARTITION_SUMMARY_PROP, "false"); - Assert.assertEquals( - "Should not set partition-summaries-included to true", "false", summariesIncluded); - - String changedPartitions = - table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); - Assert.assertEquals("Should set changed partition count", "1", changedPartitions); + assertThat(partitionSummaryKeys).isEmpty(); + + assertThat(table.currentSnapshot().summary()) + .doesNotContainKey(SnapshotSummary.PARTITION_SUMMARY_PROP) + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1"); } - @Test + @TestTemplate public void testIncludedPartitionSummaries() { table.updateProperties().set(TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, "1").commit(); @@ -1425,32 +1336,17 @@ public void testIncludedPartitionSummaries() { table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet()); - Assert.assertEquals("Should include a partition summary", 1, partitionSummaryKeys.size()); - - String summariesIncluded = - table - .currentSnapshot() - .summary() - .getOrDefault(SnapshotSummary.PARTITION_SUMMARY_PROP, "false"); - Assert.assertEquals( - "Should set partition-summaries-included to true", "true", summariesIncluded); - - String changedPartitions = - table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); - Assert.assertEquals("Should set changed partition count", "1", changedPartitions); - - String partitionSummary = - table - .currentSnapshot() - .summary() - .get(SnapshotSummary.CHANGED_PARTITION_PREFIX + "data_bucket=0"); - Assert.assertEquals( - "Summary should include 1 file with 1 record that is 10 bytes", - "added-data-files=1,added-records=1,added-files-size=10", - partitionSummary); + assertThat(partitionSummaryKeys).hasSize(1); + + assertThat(table.currentSnapshot().summary()) + .containsEntry(SnapshotSummary.PARTITION_SUMMARY_PROP, "true") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry( + SnapshotSummary.CHANGED_PARTITION_PREFIX + "data_bucket=0", + "added-data-files=1,added-records=1,added-files-size=10"); } - @Test + @TestTemplate public void testIncludedPartitionSummaryLimit() { table.updateProperties().set(TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, "1").commit(); @@ -1460,19 +1356,10 @@ public void testIncludedPartitionSummaryLimit() { table.currentSnapshot().summary().keySet().stream() .filter(key -> key.startsWith(SnapshotSummary.CHANGED_PARTITION_PREFIX)) .collect(Collectors.toSet()); - Assert.assertEquals( - "Should include no partition summaries, over limit", 0, partitionSummaryKeys.size()); - - String summariesIncluded = - table - .currentSnapshot() - .summary() - .getOrDefault(SnapshotSummary.PARTITION_SUMMARY_PROP, "false"); - Assert.assertEquals( - "Should not set partition-summaries-included to true", "false", summariesIncluded); - - String changedPartitions = - table.currentSnapshot().summary().get(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP); - Assert.assertEquals("Should set changed partition count", "2", changedPartitions); + assertThat(partitionSummaryKeys).isEmpty(); + + assertThat(table.currentSnapshot().summary()) + .doesNotContainKey(SnapshotSummary.PARTITION_SUMMARY_PROP) + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "2"); } } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index e5665d6714fc..15b5547cc456 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -24,24 +24,26 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; 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 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; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestOverwrite extends TableTestBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestOverwrite extends TestBase { private static final Schema DATE_SCHEMA = new Schema( required(1, "id", Types.LongType.get()), @@ -104,21 +106,16 @@ public class TestOverwrite extends TableTestBase { )) .build(); - private final String branch; - - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} - }; - } + @Parameter(index = 1) + private String branch; - public TestOverwrite(int formatVersion, String branch) { - super(formatVersion); - this.branch = branch; + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}); } private static ByteBuffer longToBuffer(long value) { @@ -127,10 +124,10 @@ private static ByteBuffer longToBuffer(long value) { private Table table = null; - @Before + @BeforeEach public void createTestTable() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_BY_DATE, formatVersion); @@ -138,7 +135,7 @@ public void createTestTable() throws IOException { commit(table, table.newAppend().appendFile(FILE_0_TO_4).appendFile(FILE_5_TO_9), branch); } - @Test + @TestTemplate public void testOverwriteWithoutAppend() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); long baseId = latestSnapshot(base, branch).snapshotId(); @@ -147,11 +144,8 @@ public void testOverwriteWithoutAppend() { long overwriteId = latestSnapshot(table, branch).snapshotId(); - Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); - Assert.assertEquals( - "Table should have one manifest", - 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(overwriteId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), @@ -160,7 +154,7 @@ public void testOverwriteWithoutAppend() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testOverwriteFailsDelete() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); long baseId = @@ -171,15 +165,14 @@ public void testOverwriteFailsDelete() { .newOverwrite() .overwriteByRowFilter(and(equal("date", "2018-06-09"), lessThan("id", 9))); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot delete file where some, but not all, rows match filter"); - Assert.assertEquals( - "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); + assertThat(latestSnapshot(base, branch).snapshotId()).isEqualTo(baseId); } - @Test + @TestTemplate public void testOverwriteWithAppendOutsideOfDelete() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); Snapshot latestSnapshot = latestSnapshot(base, branch); @@ -195,11 +188,8 @@ public void testOverwriteWithAppendOutsideOfDelete() { long overwriteId = latestSnapshot(table, branch).snapshotId(); - Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); - Assert.assertEquals( - "Table should have 2 manifests", - 2, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(overwriteId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); // manifest is not merged because it is less than the minimum validateManifestEntries( @@ -215,7 +205,7 @@ public void testOverwriteWithAppendOutsideOfDelete() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testOverwriteWithMergedAppendOutsideOfDelete() { // ensure the overwrite results in a merge table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); @@ -234,11 +224,8 @@ public void testOverwriteWithMergedAppendOutsideOfDelete() { long overwriteId = latestSnapshot(table, branch).snapshotId(); - Assert.assertNotEquals("Should create a new snapshot", baseId, overwriteId); - Assert.assertEquals( - "Table should have one merged manifest", - 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(overwriteId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), @@ -247,7 +234,7 @@ public void testOverwriteWithMergedAppendOutsideOfDelete() { statuses(Status.ADDED, Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testValidatedOverwriteWithAppendOutsideOfDelete() { // ensure the overwrite results in a merge table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); @@ -263,15 +250,14 @@ public void testValidatedOverwriteWithAppendOutsideOfDelete() { .addFile(FILE_10_TO_14) // in 2018-06-09, NOT in 2018-06-08 .validateAddedFilesMatchOverwriteFilter(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot append file with rows that do not match filter"); - Assert.assertEquals( - "Should not create a new snapshot", baseId, latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(baseId); } - @Test + @TestTemplate public void testValidatedOverwriteWithAppendOutsideOfDeleteMetrics() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); long baseId = @@ -284,15 +270,14 @@ public void testValidatedOverwriteWithAppendOutsideOfDeleteMetrics() { .addFile(FILE_10_TO_14) // in 2018-06-09 matches, but IDs are outside range .validateAddedFilesMatchOverwriteFilter(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot append file with rows that do not match filter"); - Assert.assertEquals( - "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); + assertThat(latestSnapshot(base, branch).snapshotId()).isEqualTo(baseId); } - @Test + @TestTemplate public void testValidatedOverwriteWithAppendSuccess() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); long baseId = @@ -305,11 +290,10 @@ public void testValidatedOverwriteWithAppendSuccess() { .addFile(FILE_10_TO_14) // in 2018-06-09 matches and IDs are inside range .validateAddedFilesMatchOverwriteFilter(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot append file with rows that do not match filter"); - Assert.assertEquals( - "Should not create a new snapshot", baseId, latestSnapshot(base, branch).snapshotId()); + assertThat(latestSnapshot(base, branch).snapshotId()).isEqualTo(baseId); } } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java index a4ccb4018c9a..7cb50df919de 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwriteWithValidation.java @@ -26,26 +26,28 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; 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.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; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestOverwriteWithValidation extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestOverwriteWithValidation extends TestBase { private static final String TABLE_NAME = "overwrite_table"; @@ -179,21 +181,16 @@ public class TestOverwriteWithValidation extends TableTestBase { private static final Expression EXPRESSION_DAY_2_ANOTHER_ID_RANGE = greaterThanOrEqual("id", 10L); - private final String branch; + @Parameter(index = 1) + private String branch; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} - }; - } - - public TestOverwriteWithValidation(int formatVersion, String branch) { - super(formatVersion); - this.branch = branch; + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}); } private static ByteBuffer longToBuffer(long value) { @@ -202,26 +199,26 @@ private static ByteBuffer longToBuffer(long value) { private Table table = null; - @Before + @BeforeEach public void before() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); this.table = TestTables.create(tableDir, TABLE_NAME, DATE_SCHEMA, PARTITION_SPEC, formatVersion); } - @Test + @TestTemplate public void testOverwriteEmptyTableNotValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newOverwrite().addFile(FILE_DAY_2_MODIFIED), branch); validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteEmptyTableStrictValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit( table, @@ -235,9 +232,9 @@ public void testOverwriteEmptyTableStrictValidated() { validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteEmptyTableValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit( table, @@ -251,7 +248,7 @@ public void testOverwriteEmptyTableValidated() { validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteTableNotValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -263,7 +260,7 @@ public void testOverwriteTableNotValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteTableStrictValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -284,7 +281,7 @@ public void testOverwriteTableStrictValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteTableValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -305,7 +302,7 @@ public void testOverwriteTableValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteCompatibleAdditionNotValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); @@ -321,7 +318,7 @@ public void testOverwriteCompatibleAdditionNotValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteCompatibleAdditionStrictValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); @@ -340,17 +337,14 @@ public void testOverwriteCompatibleAdditionStrictValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found conflicting files"); - Assert.assertEquals( - "Should not create a new snapshot", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testOverwriteCompatibleAdditionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); @@ -373,7 +367,7 @@ public void testOverwriteCompatibleAdditionValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteCompatibleDeletionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -396,7 +390,7 @@ public void testOverwriteCompatibleDeletionValidated() { validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteIncompatibleAdditionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); @@ -414,17 +408,14 @@ public void testOverwriteIncompatibleAdditionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found conflicting files"); - Assert.assertEquals( - "Should not create a new snapshot", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testOverwriteIncompatibleDeletionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -443,17 +434,14 @@ public void testOverwriteIncompatibleDeletionValidated() { commit(table, table.newDelete().deleteFile(FILE_DAY_2), branch); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Missing required files to delete:"); - Assert.assertEquals( - "Should not create a new snapshot", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testOverwriteCompatibleRewriteAllowed() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -477,13 +465,10 @@ public void testOverwriteCompatibleRewriteAllowed() { commit(table, overwrite, branch); - Assert.assertNotEquals( - "Should successfully commit", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isNotEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testOverwriteCompatibleExpirationAdditionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); // id 1 @@ -508,7 +493,7 @@ public void testOverwriteCompatibleExpirationAdditionValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteCompatibleExpirationDeletionValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); // id 1 @@ -533,7 +518,7 @@ public void testOverwriteCompatibleExpirationDeletionValidated() { validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteIncompatibleExpirationValidated() { commit(table, table.newAppend().appendFile(FILE_DAY_1), branch); // id 1 @@ -553,19 +538,16 @@ public void testOverwriteIncompatibleExpirationValidated() { table.expireSnapshots().expireSnapshotId(2L).commit(); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot determine history"); - Assert.assertEquals( - "Should not create a new snapshot", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testOverwriteIncompatibleBaseExpirationEmptyTableValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); OverwriteFiles overwrite = table @@ -581,19 +563,16 @@ public void testOverwriteIncompatibleBaseExpirationEmptyTableValidated() { table.expireSnapshots().expireSnapshotId(1L).commit(); long committedSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot determine history"); - Assert.assertEquals( - "Should not create a new snapshot", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testOverwriteAnotherRangeValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); OverwriteFiles overwrite = table @@ -609,9 +588,9 @@ public void testOverwriteAnotherRangeValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testOverwriteAnotherRangeWithinPartitionValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); Expression conflictDetectionFilter = and(EXPRESSION_DAY_2, EXPRESSION_DAY_2_ID_RANGE); OverwriteFiles overwrite = @@ -628,9 +607,9 @@ public void testOverwriteAnotherRangeWithinPartitionValidated() { validateBranchFiles(table, branch, FILE_DAY_2_ANOTHER_RANGE, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testTransactionCompatibleAdditionValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_2), branch); @@ -653,9 +632,9 @@ public void testTransactionCompatibleAdditionValidated() { validateBranchFiles(table, branch, FILE_DAY_1, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testTransactionIncompatibleAdditionValidated() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); Transaction txn = table.newTransaction(); @@ -672,21 +651,18 @@ public void testTransactionIncompatibleAdditionValidated() { commit(table, overwrite, branch); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found conflicting files"); - Assert.assertEquals( - "Should not create a new snapshot", - committedSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(committedSnapshotId); } - @Test + @TestTemplate public void testConcurrentConflictingPositionDeletes() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -704,16 +680,16 @@ public void testConcurrentConflictingPositionDeletes() { commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES), branch); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, found new delete for replaced data file"); } - @Test + @TestTemplate public void testConcurrentConflictingPositionDeletesOverwriteByFilter() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -731,14 +707,14 @@ public void testConcurrentConflictingPositionDeletesOverwriteByFilter() { commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_POS_DELETES), branch); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found new conflicting delete"); } - @Test + @TestTemplate public void testConcurrentConflictingDataFileDeleteOverwriteByFilter() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -755,14 +731,14 @@ public void testConcurrentConflictingDataFileDeleteOverwriteByFilter() { commit(table, table.newOverwrite().deleteFile(FILE_DAY_2), branch); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found conflicting deleted files"); } - @Test + @TestTemplate public void testConcurrentNonConflictingDataFileDeleteOverwriteByFilter() { - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -784,11 +760,11 @@ public void testConcurrentNonConflictingDataFileDeleteOverwriteByFilter() { validateBranchFiles(table, branch, FILE_DAY_2_MODIFIED); } - @Test + @TestTemplate public void testConcurrentNonConflictingPositionDeletes() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -812,11 +788,11 @@ public void testConcurrentNonConflictingPositionDeletes() { validateBranchDeleteFiles(table, branch, FILE_DAY_1_POS_DELETES); } - @Test + @TestTemplate public void testConcurrentNonConflictingPositionDeletesOverwriteByFilter() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -840,11 +816,11 @@ public void testConcurrentNonConflictingPositionDeletesOverwriteByFilter() { validateBranchDeleteFiles(table, branch, FILE_DAY_1_POS_DELETES); } - @Test + @TestTemplate public void testConcurrentConflictingEqualityDeletes() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -862,16 +838,16 @@ public void testConcurrentConflictingEqualityDeletes() { commit(table, table.newRowDelta().addDeletes(FILE_DAY_2_EQ_DELETES), branch); - Assertions.assertThatThrownBy(() -> commit(table, overwrite, branch)) + assertThatThrownBy(() -> commit(table, overwrite, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, found new delete for replaced data file"); } - @Test + @TestTemplate public void testConcurrentNonConflictingEqualityDeletes() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit( table, @@ -898,11 +874,11 @@ public void testConcurrentNonConflictingEqualityDeletes() { validateBranchDeleteFiles(table, branch, FILE_DAY_2_ANOTHER_RANGE_EQ_DELETES); } - @Test + @TestTemplate public void testOverwriteByFilterInheritsConflictDetectionFilter() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -926,7 +902,7 @@ public void testOverwriteByFilterInheritsConflictDetectionFilter() { validateBranchDeleteFiles(table, branch, FILE_DAY_1_POS_DELETES); } - @Test + @TestTemplate public void testOverwriteCaseSensitivity() { commit(table, table.newAppend().appendFile(FILE_DAY_1).appendFile(FILE_DAY_2), branch); @@ -934,7 +910,7 @@ public void testOverwriteCaseSensitivity() { Expression rowFilter = equal("dAtE", "2018-06-09"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -947,7 +923,7 @@ public void testOverwriteCaseSensitivity() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot find field 'dAtE'"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -962,7 +938,7 @@ public void testOverwriteCaseSensitivity() { .hasMessageStartingWith("Cannot find field 'dAtE'"); // binding should succeed and trigger the validation - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -977,11 +953,11 @@ public void testOverwriteCaseSensitivity() { .hasMessageStartingWith("Found conflicting files"); } - @Test + @TestTemplate public void testMetadataOnlyDeleteWithPositionDeletes() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); - Assert.assertNull("Should be empty table", latestSnapshot(table, branch)); + assertThat(latestSnapshot(table, branch)).isNull(); commit( table, diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index 5b868d3d3642..948eda552817 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -22,50 +22,45 @@ import static org.apache.iceberg.ManifestEntry.Status.DELETED; import static org.apache.iceberg.ManifestEntry.Status.EXISTING; 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.File; +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; import org.mockito.internal.util.collections.Sets; -@RunWith(Parameterized.class) -public class TestRewriteFiles extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteFiles extends TestBase { - private final String branch; + @Parameter(index = 1) + private String branch; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} - }; + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}); } - public TestRewriteFiles(int formatVersion, String branch) { - super(formatVersion); - this.branch = branch; - } - - @Test + @TestTemplate public void testEmptyTable() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); TableMetadata base = readMetadata(); - Assert.assertNull("Should not have a current snapshot", base.ref(branch)); + assertThat(base.ref(branch)).isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -74,7 +69,7 @@ public void testEmptyTable() { .isInstanceOf(ValidationException.class) .hasMessage("Missing required files to delete: /path/to/data-a.parquet"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -90,11 +85,11 @@ public void testEmptyTable() { .hasMessage("Missing required files to delete: /path/to/data-a-deletes.parquet"); } - @Test + @TestTemplate public void testAddOnly() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Collections.emptySet()), @@ -102,7 +97,7 @@ public void testAddOnly() { .isInstanceOf(ValidationException.class) .hasMessage("Missing required files to delete: /path/to/data-a.parquet"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table @@ -117,7 +112,7 @@ public void testAddOnly() { .hasMessage( "Delete files to add must be empty because there's no delete file to be rewritten"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table @@ -133,11 +128,11 @@ public void testAddOnly() { "Delete files to add must be empty because there's no delete file to be rewritten"); } - @Test + @TestTemplate public void testDeleteOnly() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table.newRewrite().rewriteFiles(Collections.emptySet(), Sets.newSet(FILE_A)), @@ -145,7 +140,7 @@ public void testDeleteOnly() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Files to delete cannot be empty"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table @@ -159,7 +154,7 @@ public void testDeleteOnly() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Files to delete cannot be empty"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table @@ -174,28 +169,23 @@ public void testDeleteOnly() { .hasMessage("Files to delete cannot be empty"); } - @Test + @TestTemplate public void testDeleteWithDuplicateEntriesInManifest() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); commit( table, table.newAppend().appendFile(FILE_A).appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); long baseSnapshotId = latestSnapshot(base, branch).snapshotId(); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - latestSnapshot(base, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(base, branch).allManifests(table.io())).hasSize(1); ManifestFile initialManifest = latestSnapshot(base, branch).allManifests(table.io()).get(0); Snapshot pending = apply(table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)), branch); - Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); - Assert.assertFalse( - "Should not contain manifest from initial write", - pending.allManifests(table.io()).contains(initialManifest)); + assertThat(pending.allManifests(table.io())).hasSize(2); + assertThat(pending.allManifests(table.io())).doesNotContain(initialManifest); long pendingId = pending.snapshotId(); @@ -209,30 +199,25 @@ public void testDeleteWithDuplicateEntriesInManifest() { statuses(DELETED, DELETED, EXISTING)); // We should only get the 3 manifests that this test is expected to add. - Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(3); } - @Test + @TestTemplate public void testAddAndDelete() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); long baseSnapshotId = latestSnapshot(base, branch).snapshotId(); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); ManifestFile initialManifest = latestSnapshot(table, branch).allManifests(table.io()).get(0); Snapshot pending = apply(table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_C)), branch); - Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); - Assert.assertFalse( - "Should not contain manifest from initial write", - pending.allManifests(table.io()).contains(initialManifest)); + assertThat(pending.allManifests(table.io())).hasSize(2); + assertThat(pending.allManifests(table.io())).doesNotContain(initialManifest); long pendingId = pending.snapshotId(); @@ -246,14 +231,15 @@ public void testAddAndDelete() { statuses(DELETED, EXISTING)); // We should only get the 3 manifests that this test is expected to add. - Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(3); } - @Test + @TestTemplate public void testRewriteDataAndDeleteFiles() { - Assume.assumeTrue( - "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assumeThat(formatVersion) + .as("Rewriting delete files is only supported in iceberg format v2 or later") + .isGreaterThan(1); + assertThat(listManifestFiles()).isEmpty(); commit( table, @@ -269,8 +255,7 @@ public void testRewriteDataAndDeleteFiles() { TableMetadata base = readMetadata(); Snapshot baseSnap = latestSnapshot(base, branch); long baseSnapshotId = baseSnap.snapshotId(); - Assert.assertEquals( - "Should create 2 manifests for initial write", 2, baseSnap.allManifests(table.io()).size()); + assertThat(baseSnap.allManifests(table.io())).hasSize(2); List initialManifests = baseSnap.allManifests(table.io()); validateManifestEntries( @@ -299,10 +284,8 @@ public void testRewriteDataAndDeleteFiles() { ImmutableSet.of()), branch); - Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); - Assert.assertFalse( - "Should not contain manifest from initial write", - pending.allManifests(table.io()).stream().anyMatch(initialManifests::contains)); + assertThat(pending.allManifests(table.io())).hasSize(3); + assertThat(pending.allManifests(table.io())).doesNotContainAnyElementsOf(initialManifests); long pendingId = pending.snapshotId(); validateManifestEntries( @@ -323,14 +306,15 @@ public void testRewriteDataAndDeleteFiles() { statuses(DELETED, EXISTING)); // We should only get the 5 manifests that this test is expected to add. - Assert.assertEquals("Only 5 manifests should exist", 5, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(5); } - @Test + @TestTemplate public void testRewriteDataAndAssignOldSequenceNumber() { - Assume.assumeTrue( - "Sequence number is only supported in iceberg format v2. ", formatVersion > 1); - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assumeThat(formatVersion) + .as("Sequence number is only supported in iceberg format v2 or later") + .isGreaterThan(1); + assertThat(listManifestFiles()).isEmpty(); commit( table, @@ -346,8 +330,7 @@ public void testRewriteDataAndAssignOldSequenceNumber() { TableMetadata base = readMetadata(); Snapshot baseSnap = latestSnapshot(base, branch); long baseSnapshotId = baseSnap.snapshotId(); - Assert.assertEquals( - "Should create 2 manifests for initial write", 2, baseSnap.allManifests(table.io()).size()); + assertThat(baseSnap.allManifests(table.io())).hasSize(2); List initialManifests = baseSnap.allManifests(table.io()); validateManifestEntries( @@ -373,24 +356,15 @@ public void testRewriteDataAndAssignOldSequenceNumber() { .rewriteFiles(ImmutableSet.of(FILE_A), ImmutableSet.of(FILE_D), oldSequenceNumber), branch); - Assert.assertEquals("Should contain 3 manifest", 3, pending.allManifests(table.io()).size()); - Assert.assertFalse( - "Should not contain data manifest from initial write", - pending.dataManifests(table.io()).stream().anyMatch(initialManifests::contains)); + assertThat(pending.allManifests(table.io())).hasSize(3); + assertThat(pending.dataManifests(table.io())).doesNotContainAnyElementsOf(initialManifests); long pendingId = pending.snapshotId(); ManifestFile newManifest = pending.allManifests(table.io()).get(0); validateManifestEntries(newManifest, ids(pendingId), files(FILE_D), statuses(ADDED)); - for (ManifestEntry entry : ManifestFiles.read(newManifest, FILE_IO).entries()) { - Assert.assertEquals( - "Should have old sequence number for manifest entries", - oldSequenceNumber, - (long) entry.dataSequenceNumber()); - } - Assert.assertEquals( - "Should use new sequence number for the manifest file", - oldSequenceNumber + 1, - newManifest.sequenceNumber()); + assertThat(ManifestFiles.read(newManifest, FILE_IO).entries()) + .allSatisfy(entry -> assertThat(entry.dataSequenceNumber()).isEqualTo(oldSequenceNumber)); + assertThat(newManifest.sequenceNumber()).isEqualTo(oldSequenceNumber + 1); validateManifestEntries( pending.allManifests(table.io()).get(1), @@ -407,10 +381,10 @@ public void testRewriteDataAndAssignOldSequenceNumber() { statuses(ADDED, ADDED)); // We should only get the 4 manifests that this test is expected to add. - Assert.assertEquals("Only 4 manifests should exist", 4, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(4); } - @Test + @TestTemplate public void testFailure() { commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -420,28 +394,29 @@ public void testFailure() { table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)); Snapshot pending = apply(rewrite, branch); - Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(2); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); ManifestFile manifest2 = pending.allManifests(table.io()).get(1); validateManifestEntries(manifest1, ids(pending.snapshotId()), files(FILE_B), statuses(ADDED)); validateManifestEntries(manifest2, ids(pending.snapshotId()), files(FILE_A), statuses(DELETED)); - Assertions.assertThatThrownBy(() -> commit(table, rewrite, branch)) + assertThatThrownBy(() -> commit(table, rewrite, branch)) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertFalse("Should clean up new manifest", new File(manifest1.path()).exists()); - Assert.assertFalse("Should clean up new manifest", new File(manifest2.path()).exists()); + assertThat(new File(manifest1.path())).doesNotExist(); + assertThat(new File(manifest2.path())).doesNotExist(); // As commit failed all the manifests added with rewrite should be cleaned up - Assert.assertEquals("Only 1 manifest should exist", 1, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(1); } - @Test + @TestTemplate public void testFailureWhenRewriteBothDataAndDeleteFiles() { - Assume.assumeTrue( - "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); + assumeThat(formatVersion) + .as("Rewriting delete files is only supported in iceberg format v2 or later") + .isGreaterThan(1); commit( table, @@ -468,7 +443,7 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { ImmutableSet.of()); Snapshot pending = apply(rewrite, branch); - Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(3); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); ManifestFile manifest2 = pending.allManifests(table.io()).get(1); ManifestFile manifest3 = pending.allManifests(table.io()).get(2); @@ -493,19 +468,19 @@ public void testFailureWhenRewriteBothDataAndDeleteFiles() { files(FILE_A_DELETES, FILE_B_DELETES), statuses(DELETED, DELETED)); - Assertions.assertThatThrownBy(rewrite::commit) + assertThatThrownBy(rewrite::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertFalse("Should clean up new manifest", new File(manifest1.path()).exists()); - Assert.assertFalse("Should clean up new manifest", new File(manifest2.path()).exists()); - Assert.assertFalse("Should clean up new manifest", new File(manifest3.path()).exists()); + assertThat(new File(manifest1.path())).doesNotExist(); + assertThat(new File(manifest2.path())).doesNotExist(); + assertThat(new File(manifest3.path())).doesNotExist(); // As commit failed all the manifests added with rewrite should be cleaned up - Assert.assertEquals("Only 2 manifest should exist", 2, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(2); } - @Test + @TestTemplate public void testRecovery() { commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -515,7 +490,7 @@ public void testRecovery() { table.newRewrite().rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)); Snapshot pending = apply(rewrite, branch); - Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(2); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); ManifestFile manifest2 = pending.allManifests(table.io()).get(1); @@ -524,23 +499,21 @@ public void testRecovery() { commit(table, rewrite, branch); - Assert.assertTrue("Should reuse the manifest for appends", new File(manifest1.path()).exists()); - Assert.assertTrue( - "Should reuse the manifest with deletes", new File(manifest2.path()).exists()); + assertThat(new File(manifest1.path())).exists(); + assertThat(new File(manifest2.path())).exists(); TableMetadata metadata = readMetadata(); - Assert.assertTrue( - "Should commit the manifest for append", - latestSnapshot(metadata, branch).allManifests(table.io()).contains(manifest2)); + assertThat(latestSnapshot(metadata, branch).allManifests(table.io())).contains(manifest2); // 2 manifests added by rewrite and 1 original manifest should be found. - Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(3); } - @Test + @TestTemplate public void testRecoverWhenRewriteBothDataAndDeleteFiles() { - Assume.assumeTrue( - "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); + assumeThat(formatVersion) + .as("Rewriting delete files is only supported in iceberg format v2 or later") + .isGreaterThan(1); commit( table, @@ -567,7 +540,7 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { ImmutableSet.of()); Snapshot pending = apply(rewrite, branch); - Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(3); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); ManifestFile manifest2 = pending.allManifests(table.io()).get(1); ManifestFile manifest3 = pending.allManifests(table.io()).get(2); @@ -590,25 +563,24 @@ public void testRecoverWhenRewriteBothDataAndDeleteFiles() { commit(table, rewrite, branch); - Assert.assertTrue("Should reuse new manifest", new File(manifest1.path()).exists()); - Assert.assertTrue("Should reuse new manifest", new File(manifest2.path()).exists()); - Assert.assertTrue("Should reuse new manifest", new File(manifest3.path()).exists()); + assertThat(new File(manifest1.path())).exists(); + assertThat(new File(manifest2.path())).exists(); + assertThat(new File(manifest3.path())).exists(); TableMetadata metadata = readMetadata(); List committedManifests = Lists.newArrayList(manifest1, manifest2, manifest3); - Assert.assertEquals( - "Should committed the manifests", - latestSnapshot(metadata, branch).allManifests(table.io()), - committedManifests); + assertThat(latestSnapshot(metadata, branch).allManifests(table.io())) + .isEqualTo(committedManifests); // As commit success all the manifests added with rewrite should be available. - Assert.assertEquals("Only 5 manifest should exist", 5, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(5); } - @Test + @TestTemplate public void testReplaceEqualityDeletesWithPositionDeletes() { - Assume.assumeTrue( - "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); + assumeThat(formatVersion) + .as("Rewriting delete files is only supported in iceberg format v2 or later") + .isGreaterThan(1); commit(table, table.newRowDelta().addRows(FILE_A2).addDeletes(FILE_A2_DELETES), branch); @@ -624,7 +596,7 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { ImmutableSet.of(), ImmutableSet.of(FILE_B_DELETES)); Snapshot pending = apply(rewrite, branch); - Assert.assertEquals("Should produce 3 manifests", 3, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(3); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); ManifestFile manifest2 = pending.allManifests(table.io()).get(1); ManifestFile manifest3 = pending.allManifests(table.io()).get(2); @@ -649,25 +621,24 @@ public void testReplaceEqualityDeletesWithPositionDeletes() { commit(table, rewrite, branch); - Assert.assertTrue("Should reuse new manifest", new File(manifest1.path()).exists()); - Assert.assertTrue("Should reuse new manifest", new File(manifest2.path()).exists()); - Assert.assertTrue("Should reuse new manifest", new File(manifest3.path()).exists()); + assertThat(new File(manifest1.path())).exists(); + assertThat(new File(manifest2.path())).exists(); + assertThat(new File(manifest3.path())).exists(); metadata = readMetadata(); List committedManifests = Lists.newArrayList(manifest1, manifest2, manifest3); - Assert.assertEquals( - "Should committed the manifests", - latestSnapshot(metadata, branch).allManifests(table.io()), - committedManifests); + assertThat(latestSnapshot(metadata, branch).allManifests(table.io())) + .isEqualTo(committedManifests); // As commit success all the manifests added with rewrite should be available. - Assert.assertEquals("4 manifests should exist", 4, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(4); } - @Test + @TestTemplate public void testRemoveAllDeletes() { - Assume.assumeTrue( - "Rewriting delete files is only supported in iceberg format v2. ", formatVersion > 1); + assumeThat(formatVersion) + .as("Rewriting delete files is only supported in iceberg format v2 or later") + .isGreaterThan(1); commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); @@ -681,7 +652,7 @@ public void testRemoveAllDeletes() { ImmutableSet.of(), ImmutableSet.of()); Snapshot pending = apply(rewrite, branch); - Assert.assertEquals("Should produce 2 manifests", 2, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(2); ManifestFile manifest1 = pending.allManifests(table.io()).get(0); ManifestFile manifest2 = pending.allManifests(table.io()).get(1); @@ -697,32 +668,28 @@ public void testRemoveAllDeletes() { commit(table, rewrite, branch); - Assert.assertTrue("Should reuse the new manifest", new File(manifest1.path()).exists()); - Assert.assertTrue("Should reuse the new manifest", new File(manifest2.path()).exists()); + assertThat(new File(manifest1.path())).exists(); + assertThat(new File(manifest2.path())).exists(); TableMetadata metadata = readMetadata(); List committedManifests = Lists.newArrayList(manifest1, manifest2); - Assert.assertTrue( - "Should committed the manifests", - latestSnapshot(metadata, branch).allManifests(table.io()).containsAll(committedManifests)); + assertThat(latestSnapshot(metadata, branch).allManifests(table.io())) + .containsAll(committedManifests); // As commit success all the manifests added with rewrite should be available. - Assert.assertEquals("4 manifests should exist", 4, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(4); } - @Test + @TestTemplate public void testDeleteNonExistentFile() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - latestSnapshot(base, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(base, branch).allManifests(table.io())).hasSize(1); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -731,26 +698,23 @@ public void testDeleteNonExistentFile() { .isInstanceOf(ValidationException.class) .hasMessage("Missing required files to delete: /path/to/data-c.parquet"); - Assert.assertEquals("Only 1 manifests should exist", 1, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(1); } - @Test + @TestTemplate public void testAlreadyDeletedFile() { - Assert.assertEquals("Table should start empty", 0, listManifestFiles().size()); + assertThat(listManifestFiles()).isEmpty(); commit(table, table.newAppend().appendFile(FILE_A), branch); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - latestSnapshot(base, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(base, branch).allManifests(table.io())).hasSize(1); RewriteFiles rewrite = table.newRewrite(); Snapshot pending = apply(rewrite.rewriteFiles(Sets.newSet(FILE_A), Sets.newSet(FILE_B)), branch); - Assert.assertEquals("Should contain 2 manifest", 2, pending.allManifests(table.io()).size()); + assertThat(pending.allManifests(table.io())).hasSize(2); long pendingId = pending.snapshotId(); @@ -765,7 +729,7 @@ public void testAlreadyDeletedFile() { commit(table, rewrite, branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -774,12 +738,14 @@ public void testAlreadyDeletedFile() { .isInstanceOf(ValidationException.class) .hasMessage("Missing required files to delete: /path/to/data-a.parquet"); - Assert.assertEquals("Only 3 manifests should exist", 3, listManifestFiles().size()); + assertThat(listManifestFiles()).hasSize(3); } - @Test + @TestTemplate public void testNewDeleteFile() { - Assume.assumeTrue("Delete files are only supported in v2", formatVersion > 1); + assumeThat(formatVersion) + .as("Rewriting delete files is only supported in iceberg format v2 or later") + .isGreaterThan(1); commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -789,7 +755,7 @@ public void testNewDeleteFile() { long snapshotAfterDeletes = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> apply( table From 4f248197f0592780537436360cf768af9e76756d Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Wed, 20 Mar 2024 12:45:58 -0700 Subject: [PATCH 0152/1019] Add issue template and docs for iceberg proposals (#9932) * Add issue template and docs for iceberg proposals * Add vote clarification * Update site/docs/contribute.md Co-authored-by: Renjie Liu * Update site/docs/contribute.md Co-authored-by: Renjie Liu * Update .github/ISSUE_TEMPLATE/iceberg_proposal.yml Co-authored-by: Eduard Tudenhoefner * Fix issue template link * Address comments --------- Co-authored-by: Renjie Liu Co-authored-by: Eduard Tudenhoefner --- .github/ISSUE_TEMPLATE/iceberg_proposal.yml | 50 +++++++++++++++++++++ site/docs/contribute.md | 36 +++++++++++++++ 2 files changed, 86 insertions(+) create mode 100644 .github/ISSUE_TEMPLATE/iceberg_proposal.yml diff --git a/.github/ISSUE_TEMPLATE/iceberg_proposal.yml b/.github/ISSUE_TEMPLATE/iceberg_proposal.yml new file mode 100644 index 000000000000..af96b5399e15 --- /dev/null +++ b/.github/ISSUE_TEMPLATE/iceberg_proposal.yml @@ -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. +# + +--- +name: Iceberg Improvement Proposal +description: Propose a Spec change or major feature +labels: ["proposal"] +body: + - type: markdown + attributes: + value: "Please see documentation site for information on [contributing proposals](https://iceberg.apache.org/contribute/#apache-iceberg-improvement-proposals)" + - type: textarea + attributes: + label: Proposed Change + description: Please describe the proposal and elaborate on the use case and motivation + validations: + required: true + - type: input + attributes: + label: Proposal document + description: | + Link to the proposal document. Google Docs is preferred format to allow for public + comment and sharing + - type: checkboxes + attributes: + label: Specifications + description: Which specifications are affected by this proposal? + options: + - label: Table + - label: View + - label: REST + - label: Puffin + - label: Encryption + - label: Other \ No newline at end of file diff --git a/site/docs/contribute.md b/site/docs/contribute.md index e7fe35d14c06..2c9add8df28c 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -46,6 +46,42 @@ The Iceberg community prefers to receive contributions as [Github pull requests] * If a PR is posted for visibility and isn't necessarily ready for review or merging, be sure to convert the PR to a draft +## Apache Iceberg Improvement Proposals + +### What is an improvement proposal? + +An improvement proposal is a major change to Apache Iceberg that may require changes to an existing specification, creation +of a new specification, or significant additions/changes to any of the existing Iceberg implementations. Changes that are large in +scope need to be considered carefully and incorporate feedback from many community stakeholders. + +### What should a proposal include? + +1. A GitHub issue created using the `Apache Iceberg Improvement Proposal` template +2. A document including the following: + * Motivation for the change + * Implementation proposal + * Breaking changes/incompatibilities + * Alternatives considered +3. A discussion thread initiated in the dev list with the Subject: '[DISCUSS] ' + +### Who can submit a proposal? + +Anyone can submit a proposal, but be considerate and submit only if you plan on contributing to the implementation. + +### Where can I find current proposals? + +Current proposals are tracked in GitHub issues with the label [Proposal][iceberg-proposals] + +### How are proposals adopted? + +Once general consensus has been reached, a vote should be raised on the dev list. The vote follows the ASF +[code modification][apache-vote] model with three positive PMC votes required and no lazy consensus modifier. +The voting process should be held in good faith to reinforce and affirm the agreed upon proposal, not to +settle disagreements or to force a decision. + +[iceberg-proposals]: https://github.com/apache/iceberg/issues?q=is%3Aissue+is%3Aopen+label%3Aproposal+ +[apache-vote]: https://www.apache.org/foundation/voting.html#apache-voting-process + ## Building the Project Locally Iceberg is built using Gradle with Java 8 or Java 11. From a720e86d7c75a80c0c419b9f6137119d79798367 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Fri, 22 Mar 2024 01:50:00 +0900 Subject: [PATCH 0153/1019] Core: Migrate tests to JUnit5 (#10014) --- .../apache/iceberg/TestReplacePartitions.java | 209 ++++------ .../iceberg/TestScanDataFileColumns.java | 47 +-- .../org/apache/iceberg/TestScanSummary.java | 133 +++--- .../iceberg/TestScansAndSchemaEvolution.java | 45 +- .../iceberg/TestSchemaAndMappingUpdate.java | 224 ++++------ .../java/org/apache/iceberg/TestSchemaID.java | 129 ++---- .../org/apache/iceberg/TestSchemaUpdate.java | 389 ++++++++---------- .../iceberg/TestTableUpdatePartitionSpec.java | 97 +++-- .../iceberg/TestTimestampPartitions.java | 32 +- .../iceberg/TestUpdatePartitionSpec.java | 272 +++++------- 10 files changed, 667 insertions(+), 910 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index e657e7fc43ca..a25920a1d733 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -19,20 +19,22 @@ package org.apache.iceberg; 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.File; import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.ValidationException; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestReplacePartitions extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestReplacePartitions extends TestBase { static final DataFile FILE_E = DataFiles.builder(SPEC) @@ -73,24 +75,19 @@ public class TestReplacePartitions extends TableTestBase { .withRecordCount(1) .build(); - private final String branch; + @Parameter(index = 1) + private String branch; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, "main"}, - new Object[] {1, "testBranch"}, - new Object[] {2, "main"}, - new Object[] {2, "testBranch"} - }; + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, "main"}, + new Object[] {1, "testBranch"}, + new Object[] {2, "main"}, + new Object[] {2, "testBranch"}); } - public TestReplacePartitions(int formatVersion, String branch) { - super(formatVersion); - this.branch = branch; - } - - @Test + @TestTemplate public void testReplaceOnePartition() { commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -100,11 +97,8 @@ public void testReplaceOnePartition() { commit(table, table.newReplacePartitions().addFile(FILE_E), branch); long replaceId = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); - Assert.assertEquals( - "Table should have 2 manifests", - 2, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(replaceId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); // manifest is not merged because it is less than the minimum validateManifestEntries( @@ -120,7 +114,7 @@ public void testReplaceOnePartition() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testReplaceAndMergeOnePartition() { // ensure the overwrite results in a merge table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); @@ -133,11 +127,8 @@ public void testReplaceAndMergeOnePartition() { commit(table, table.newReplacePartitions().addFile(FILE_E), branch); long replaceId = latestSnapshot(table, branch).snapshotId(); - Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); - Assert.assertEquals( - "Table should have 1 manifest", - 1, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(replaceId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), @@ -146,37 +137,31 @@ public void testReplaceAndMergeOnePartition() { statuses(Status.ADDED, Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testReplaceWithUnpartitionedTable() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Table unpartitioned = TestTables.create( tableDir, "unpartitioned", SCHEMA, PartitionSpec.unpartitioned(), formatVersion); - Assert.assertEquals( - "Table version should be 0", 0, (long) TestTables.metadataVersion("unpartitioned")); + assertThat(TestTables.metadataVersion("unpartitioned")).isEqualTo(0); commit(table, unpartitioned.newAppend().appendFile(FILE_A), branch); // make sure the data was successfully added - Assert.assertEquals( - "Table version should be 1", 1, (long) TestTables.metadataVersion("unpartitioned")); + assertThat(TestTables.metadataVersion("unpartitioned")).isEqualTo(1); validateSnapshot( null, latestSnapshot(TestTables.readMetadata("unpartitioned"), branch), FILE_A); ReplacePartitions replacePartitions = unpartitioned.newReplacePartitions().addFile(FILE_B); commit(table, replacePartitions, branch); - Assert.assertEquals( - "Table version should be 2", 2, (long) TestTables.metadataVersion("unpartitioned")); + assertThat(TestTables.metadataVersion("unpartitioned")).isEqualTo(2); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); long replaceId = latestSnapshot(replaceMetadata, branch).snapshotId(); - Assert.assertEquals( - "Table should have 2 manifests", - 2, - latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).size()); + assertThat(latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io())).hasSize(2); validateManifestEntries( latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).get(0), @@ -191,10 +176,10 @@ public void testReplaceWithUnpartitionedTable() throws IOException { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Table unpartitioned = TestTables.create( @@ -203,30 +188,24 @@ public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { // ensure the overwrite results in a merge unpartitioned.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); - Assert.assertEquals( - "Table version should be 1", 1, (long) TestTables.metadataVersion("unpartitioned")); + assertThat(TestTables.metadataVersion("unpartitioned")).isEqualTo(1); AppendFiles appendFiles = unpartitioned.newAppend().appendFile(FILE_A); commit(table, appendFiles, branch); // make sure the data was successfully added - Assert.assertEquals( - "Table version should be 2", 2, (long) TestTables.metadataVersion("unpartitioned")); + assertThat(TestTables.metadataVersion("unpartitioned")).isEqualTo(2); validateSnapshot( null, latestSnapshot(TestTables.readMetadata("unpartitioned"), branch), FILE_A); ReplacePartitions replacePartitions = unpartitioned.newReplacePartitions().addFile(FILE_B); commit(table, replacePartitions, branch); - Assert.assertEquals( - "Table version should be 3", 3, (long) TestTables.metadataVersion("unpartitioned")); + assertThat(TestTables.metadataVersion("unpartitioned")).isEqualTo(3); TableMetadata replaceMetadata = TestTables.readMetadata("unpartitioned"); long replaceId = latestSnapshot(replaceMetadata, branch).snapshotId(); - Assert.assertEquals( - "Table should have 1 manifest", - 1, - latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).size()); + assertThat(latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io())).hasSize(1); validateManifestEntries( latestSnapshot(replaceMetadata, branch).allManifests(unpartitioned.io()).get(0), @@ -235,7 +214,7 @@ public void testReplaceAndMergeWithUnpartitionedTable() throws IOException { statuses(Status.ADDED, Status.DELETED)); } - @Test + @TestTemplate public void testValidationFailure() { commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -245,17 +224,14 @@ public void testValidationFailure() { ReplacePartitions replace = table.newReplacePartitions().addFile(FILE_F).addFile(FILE_G).validateAppendOnly(); - Assertions.assertThatThrownBy(() -> commit(table, replace, branch)) + assertThatThrownBy(() -> commit(table, replace, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit file that conflicts with existing partition"); - Assert.assertEquals( - "Should not create a new snapshot", - baseId, - latestSnapshot(readMetadata(), branch).snapshotId()); + assertThat(latestSnapshot(readMetadata(), branch).snapshotId()).isEqualTo(baseId); } - @Test + @TestTemplate public void testValidationSuccess() { commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -265,11 +241,8 @@ public void testValidationSuccess() { commit(table, table.newReplacePartitions().addFile(FILE_G).validateAppendOnly(), branch); long replaceId = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertNotEquals("Should create a new snapshot", baseId, replaceId); - Assert.assertEquals( - "Table should have 2 manifests", - 2, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(replaceId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); // manifest is not merged because it is less than the minimum validateManifestEntries( @@ -285,7 +258,7 @@ public void testValidationSuccess() { statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testValidationNotInvoked() { commit(table, table.newFastAppend().appendFile(FILE_A), branch); @@ -309,10 +282,7 @@ public void testValidationNotInvoked() { branch); long replaceId = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertEquals( - "Table should have 2 manifest", - 2, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(replaceId, replaceId), @@ -325,13 +295,13 @@ public void testValidationNotInvoked() { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testValidateWithDefaultSnapshotId() { commit(table, table.newReplacePartitions().addFile(FILE_A), branch); // Concurrent Replace Partitions should fail with ValidationException ReplacePartitions replace = table.newReplacePartitions(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -347,7 +317,7 @@ public void testValidateWithDefaultSnapshotId() { + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]"); } - @Test + @TestTemplate public void testConcurrentReplaceConflict() { commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -357,7 +327,7 @@ public void testConcurrentReplaceConflict() { // Concurrent Replace Partitions should fail with ValidationException commit(table, table.newReplacePartitions().addFile(FILE_A), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -375,7 +345,7 @@ public void testConcurrentReplaceConflict() { + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]"); } - @Test + @TestTemplate public void testConcurrentReplaceNoConflict() { commit(table, table.newFastAppend().appendFile(FILE_A), branch); @@ -397,10 +367,7 @@ public void testConcurrentReplaceNoConflict() { branch); long id3 = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertEquals( - "Table should have 2 manifests", - 2, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(id3), @@ -413,7 +380,7 @@ public void testConcurrentReplaceNoConflict() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testConcurrentReplaceConflictNonPartitioned() { Table unpartitioned = TestTables.create( @@ -426,7 +393,7 @@ public void testConcurrentReplaceConflictNonPartitioned() { // Concurrent ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newReplacePartitions().addFile(FILE_UNPARTITIONED_A), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -443,7 +410,7 @@ public void testConcurrentReplaceConflictNonPartitioned() { + "[/path/to/data-unpartitioned-a.parquet]"); } - @Test + @TestTemplate public void testAppendReplaceConflict() { commit(table, table.newFastAppend().appendFile(FILE_A), branch); @@ -453,7 +420,7 @@ public void testAppendReplaceConflict() { // Concurrent Append and ReplacePartition should fail with ValidationException commit(table, table.newFastAppend().appendFile(FILE_B), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -471,7 +438,7 @@ public void testAppendReplaceConflict() { + "[data_bucket=0, data_bucket=1]: [/path/to/data-b.parquet]"); } - @Test + @TestTemplate public void testAppendReplaceNoConflict() { commit(table, table.newFastAppend().appendFile(FILE_A), branch); @@ -494,10 +461,7 @@ public void testAppendReplaceNoConflict() { branch); long id3 = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertEquals( - "Table should have 3 manifests", - 3, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(3); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(id3), @@ -515,7 +479,7 @@ public void testAppendReplaceNoConflict() { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testAppendReplaceConflictNonPartitioned() { Table unpartitioned = TestTables.create( @@ -528,7 +492,7 @@ public void testAppendReplaceConflictNonPartitioned() { // Concurrent Append and ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newAppend().appendFile(FILE_UNPARTITIONED_A), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -545,9 +509,9 @@ public void testAppendReplaceConflictNonPartitioned() { + "[/path/to/data-unpartitioned-a.parquet]"); } - @Test + @TestTemplate public void testDeleteReplaceConflict() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); commit(table, table.newFastAppend().appendFile(FILE_A), branch); TableMetadata base = readMetadata(); @@ -557,7 +521,7 @@ public void testDeleteReplaceConflict() { commit( table, table.newRowDelta().addDeletes(FILE_A_DELETES).validateFromSnapshot(baseId), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -574,9 +538,9 @@ public void testDeleteReplaceConflict() { + "[data_bucket=0]: [/path/to/data-a-deletes.parquet]"); } - @Test + @TestTemplate public void testDeleteReplaceConflictNonPartitioned() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); Table unpartitioned = TestTables.create( @@ -589,7 +553,7 @@ public void testDeleteReplaceConflictNonPartitioned() { // Concurrent Delete and ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newRowDelta().addDeletes(FILE_UNPARTITIONED_A_DELETES), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -606,9 +570,9 @@ public void testDeleteReplaceConflictNonPartitioned() { + "[/path/to/data-unpartitioned-a-deletes.parquet]"); } - @Test + @TestTemplate public void testDeleteReplaceNoConflict() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); commit(table, table.newFastAppend().appendFile(FILE_A), branch); long id1 = latestSnapshot(readMetadata(), branch).snapshotId(); @@ -638,10 +602,7 @@ public void testDeleteReplaceNoConflict() { long id3 = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertEquals( - "Table should have 3 manifest", - 3, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(3); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(id3), @@ -661,9 +622,9 @@ public void testDeleteReplaceNoConflict() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testOverwriteReplaceConflict() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); commit(table, table.newFastAppend().appendFile(FILE_A), branch); TableMetadata base = readMetadata(); @@ -672,7 +633,7 @@ public void testOverwriteReplaceConflict() { // Concurrent Overwrite and ReplacePartition should fail with ValidationException commit(table, table.newOverwrite().deleteFile(FILE_A), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -689,9 +650,9 @@ public void testOverwriteReplaceConflict() { + "[data_bucket=0]: [/path/to/data-a.parquet]"); } - @Test + @TestTemplate public void testOverwriteReplaceNoConflict() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); TableMetadata base = readMetadata(); @@ -713,10 +674,7 @@ public void testOverwriteReplaceNoConflict() { long finalId = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertEquals( - "Table should have 2 manifest", - 2, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(finalId), @@ -729,9 +687,9 @@ public void testOverwriteReplaceNoConflict() { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testOverwriteReplaceConflictNonPartitioned() { - Assume.assumeTrue(formatVersion == 2); + assumeThat(formatVersion).isEqualTo(2); Table unpartitioned = TestTables.create( @@ -745,7 +703,7 @@ public void testOverwriteReplaceConflictNonPartitioned() { // Concurrent Overwrite and ReplacePartitions should fail with ValidationException commit(table, unpartitioned.newOverwrite().deleteFile(FILE_UNPARTITIONED_A), branch); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -762,7 +720,7 @@ public void testOverwriteReplaceConflictNonPartitioned() { + "[/path/to/data-unpartitioned-a.parquet]"); } - @Test + @TestTemplate public void testValidateOnlyDeletes() { commit(table, table.newAppend().appendFile(FILE_A), branch); long baseId = latestSnapshot(readMetadata(), branch).snapshotId(); @@ -780,10 +738,7 @@ public void testValidateOnlyDeletes() { branch); long finalId = latestSnapshot(readMetadata(), branch).snapshotId(); - Assert.assertEquals( - "Table should have 3 manifest", - 3, - latestSnapshot(table, branch).allManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(3); validateManifestEntries( latestSnapshot(table, branch).allManifests(table.io()).get(0), ids(finalId), @@ -801,7 +756,7 @@ public void testValidateOnlyDeletes() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testEmptyPartitionPathWithUnpartitionedTable() { DataFiles.builder(PartitionSpec.unpartitioned()).withPartitionPath(""); } diff --git a/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java b/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java index 8baec3184fd7..408d0047e0af 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java +++ b/core/src/test/java/org/apache/iceberg/TestScanDataFileColumns.java @@ -20,21 +20,22 @@ 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.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestScanDataFileColumns { private static final Schema SCHEMA = @@ -44,15 +45,15 @@ public class TestScanDataFileColumns { private static final Configuration CONF = new Configuration(); private static final Tables TABLES = new HadoopTables(CONF); - @Rule public final TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private String tableLocation = null; private Table table = null; - @Before + @BeforeEach public void createTables() throws IOException { - File location = temp.newFolder("shared"); - Assert.assertTrue(location.delete()); + File location = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(location.delete()).isTrue(); this.tableLocation = location.toString(); this.table = TABLES.create( @@ -115,10 +116,10 @@ public void createTables() throws IOException { public void testColumnStatsIgnored() { // stats columns should be suppressed by default for (FileScanTask fileTask : table.newScan().planFiles()) { - Assert.assertNull(fileTask.file().valueCounts()); - Assert.assertNull(fileTask.file().nullValueCounts()); - Assert.assertNull(fileTask.file().lowerBounds()); - Assert.assertNull(fileTask.file().upperBounds()); + assertThat(fileTask.file().valueCounts()).isNull(); + assertThat(fileTask.file().nullValueCounts()).isNull(); + assertThat(fileTask.file().lowerBounds()).isNull(); + assertThat(fileTask.file().upperBounds()).isNull(); } } @@ -126,11 +127,11 @@ public void testColumnStatsIgnored() { public void testColumnStatsLoading() { // stats columns should be suppressed by default for (FileScanTask fileTask : table.newScan().includeColumnStats().planFiles()) { - Assert.assertEquals(2, fileTask.file().valueCounts().size()); - Assert.assertEquals(2, fileTask.file().nullValueCounts().size()); - Assert.assertEquals(2, fileTask.file().lowerBounds().size()); - Assert.assertEquals(2, fileTask.file().upperBounds().size()); - Assert.assertEquals(2, fileTask.file().columnSizes().size()); + assertThat(fileTask.file().valueCounts()).hasSize(2); + assertThat(fileTask.file().nullValueCounts()).hasSize(2); + assertThat(fileTask.file().lowerBounds()).hasSize(2); + assertThat(fileTask.file().upperBounds()).hasSize(2); + assertThat(fileTask.file().columnSizes()).hasSize(2); } } @@ -139,11 +140,11 @@ public void testColumnStatsPartial() { // stats columns should be suppressed by default for (FileScanTask fileTask : table.newScan().includeColumnStats(ImmutableSet.of("id")).planFiles()) { - Assert.assertEquals(1, fileTask.file().valueCounts().size()); - Assert.assertEquals(1, fileTask.file().nullValueCounts().size()); - Assert.assertEquals(1, fileTask.file().lowerBounds().size()); - Assert.assertEquals(1, fileTask.file().upperBounds().size()); - Assert.assertEquals(1, fileTask.file().columnSizes().size()); + assertThat(fileTask.file().valueCounts()).hasSize(1); + assertThat(fileTask.file().nullValueCounts()).hasSize(1); + assertThat(fileTask.file().lowerBounds()).hasSize(1); + assertThat(fileTask.file().upperBounds()).hasSize(1); + assertThat(fileTask.file().columnSizes()).hasSize(1); } } diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java index 59986016614a..b326274842a3 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -25,28 +25,24 @@ import static org.apache.iceberg.expressions.Expressions.greaterThanOrEqual; import static org.apache.iceberg.expressions.Expressions.lessThan; import static org.apache.iceberg.expressions.Expressions.lessThanOrEqual; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestScanSummary extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestScanSummary extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestScanSummary(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testSnapshotTimeRangeValidation() { long t0 = System.currentTimeMillis(); @@ -76,12 +72,8 @@ public void testSnapshotTimeRangeValidation() { // expire the first snapshot table.expireSnapshots().expireOlderThan(t1).commit(); - Assert.assertEquals( - "Should have one snapshot", 1, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "Snapshot should be the second snapshot created", - secondSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.snapshots()).hasSize(1); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(secondSnapshotId); // this should include the first snapshot, but it was removed from the dataset TableScan scan = @@ -90,63 +82,56 @@ public void testSnapshotTimeRangeValidation() { .filter(greaterThanOrEqual("dateCreated", t0)) .filter(lessThan("dateCreated", t2)); - Assertions.assertThatThrownBy(() -> new ScanSummary.Builder(scan).build()) + assertThatThrownBy(() -> new ScanSummary.Builder(scan).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot satisfy time filters: time range may include expired snapshots"); } - @Test + @TestTemplate public void testTimestampRanges() { long lower = 1542750188523L; long upper = 1542750695131L; - Assert.assertEquals( - "Should use inclusive bound", - Pair.of(Long.MIN_VALUE, upper), - timestampRange(ImmutableList.of(lessThanOrEqual("ts_ms", upper)))); - - Assert.assertEquals( - "Should use lower value for upper bound", - Pair.of(Long.MIN_VALUE, upper), - timestampRange( - ImmutableList.of( - lessThanOrEqual("ts_ms", upper + 918234), lessThanOrEqual("ts_ms", upper)))); - - Assert.assertEquals( - "Should make upper bound inclusive", - Pair.of(Long.MIN_VALUE, upper - 1), - timestampRange(ImmutableList.of(lessThan("ts_ms", upper)))); - - Assert.assertEquals( - "Should use inclusive bound", - Pair.of(lower, Long.MAX_VALUE), - timestampRange(ImmutableList.of(greaterThanOrEqual("ts_ms", lower)))); - - Assert.assertEquals( - "Should use upper value for lower bound", - Pair.of(lower, Long.MAX_VALUE), - timestampRange( - ImmutableList.of( - greaterThanOrEqual("ts_ms", lower - 918234), greaterThanOrEqual("ts_ms", lower)))); - - Assert.assertEquals( - "Should make lower bound inclusive", - Pair.of(lower + 1, Long.MAX_VALUE), - timestampRange(ImmutableList.of(greaterThan("ts_ms", lower)))); - - Assert.assertEquals( - "Should set both bounds for equals", - Pair.of(lower, lower), - timestampRange(ImmutableList.of(equal("ts_ms", lower)))); - - Assert.assertEquals( - "Should set both bounds", - Pair.of(lower, upper - 1), - timestampRange( - ImmutableList.of(greaterThanOrEqual("ts_ms", lower), lessThan("ts_ms", upper)))); + assertThat(timestampRange(ImmutableList.of(lessThanOrEqual("ts_ms", upper)))) + .isEqualTo(Pair.of(Long.MIN_VALUE, upper)); + + assertThat( + timestampRange( + ImmutableList.of( + lessThanOrEqual("ts_ms", upper + 918234), lessThanOrEqual("ts_ms", upper)))) + .as("Should use lower value for upper bound") + .isEqualTo(Pair.of(Long.MIN_VALUE, upper)); + + assertThat(timestampRange(ImmutableList.of(lessThan("ts_ms", upper)))) + .as("Should make upper bound inclusive") + .isEqualTo(Pair.of(Long.MIN_VALUE, upper - 1)); + + assertThat(timestampRange(ImmutableList.of(greaterThanOrEqual("ts_ms", lower)))) + .isEqualTo(Pair.of(lower, Long.MAX_VALUE)); + + assertThat( + timestampRange( + ImmutableList.of( + greaterThanOrEqual("ts_ms", lower - 918234), + greaterThanOrEqual("ts_ms", lower)))) + .as("Should use upper value for lower bound") + .isEqualTo(Pair.of(lower, Long.MAX_VALUE)); + + assertThat(timestampRange(ImmutableList.of(greaterThan("ts_ms", lower)))) + .as("Should make lower bound inclusive") + .isEqualTo(Pair.of(lower + 1, Long.MAX_VALUE)); + + assertThat(timestampRange(ImmutableList.of(equal("ts_ms", lower)))) + .isEqualTo(Pair.of(lower, lower)); + + assertThat( + timestampRange( + ImmutableList.of(greaterThanOrEqual("ts_ms", lower), lessThan("ts_ms", upper)))) + .as("Should set both bounds and make upper bound inclusive") + .isEqualTo(Pair.of(lower, upper - 1)); // >= lower and < lower is an empty range - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> timestampRange( ImmutableList.of(greaterThanOrEqual("ts_ms", lower), lessThan("ts_ms", lower)))) @@ -154,11 +139,11 @@ public void testTimestampRanges() { .hasMessageStartingWith("No timestamps can match filters"); } - @Test + @TestTemplate public void testToMillis() { long millis = 1542750947417L; - Assert.assertEquals(1542750947000L, toMillis(millis / 1000)); - Assert.assertEquals(1542750947417L, toMillis(millis)); - Assert.assertEquals(1542750947417L, toMillis(millis * 1000 + 918)); + assertThat(toMillis(millis / 1000)).isEqualTo(1542750947000L); + assertThat(toMillis(millis)).isEqualTo(1542750947417L); + assertThat(toMillis(millis * 1000 + 918)).isEqualTo(1542750947417L); } } diff --git a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java index 5b7886fee20b..0d5b1bc7066b 100644 --- a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java @@ -19,9 +19,13 @@ package org.apache.iceberg; 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.nio.file.Path; +import java.util.Arrays; import java.util.List; import java.util.UUID; import org.apache.avro.generic.GenericData; @@ -33,15 +37,12 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.AfterEach; +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 TestScansAndSchemaEvolution { private static final Schema SCHEMA = new Schema( @@ -52,18 +53,14 @@ public class TestScansAndSchemaEvolution { private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("part").build(); - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public final int formatVersion; - - public TestScansAndSchemaEvolution(int formatVersion) { - this.formatVersion = formatVersion; - } + @Parameter private int formatVersion; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private DataFile createDataFile(String partValue) throws IOException { List expected = RandomAvroData.generate(SCHEMA, 100, 0L); @@ -87,15 +84,15 @@ private DataFile createDataFile(String partValue) throws IOException { .build(); } - @After + @AfterEach public void cleanupTables() { TestTables.clearTables(); } - @Test + @TestTemplate public void testPartitionSourceRename() throws IOException { - File location = temp.newFolder(); - Assert.assertTrue(location.delete()); // should be created by table create + File location = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(location.delete()).isTrue(); // should be created by table create Table table = TestTables.create(location, "test", SCHEMA, SPEC, formatVersion); @@ -107,13 +104,13 @@ public void testPartitionSourceRename() throws IOException { List tasks = Lists.newArrayList(table.newScan().filter(Expressions.equal("part", "one")).planFiles()); - Assert.assertEquals("Should produce 1 matching file task", 1, tasks.size()); + assertThat(tasks).hasSize(1); table.updateSchema().renameColumn("part", "p").commit(); // plan the scan using the new name in a filter tasks = Lists.newArrayList(table.newScan().filter(Expressions.equal("p", "one")).planFiles()); - Assert.assertEquals("Should produce 1 matching file task", 1, tasks.size()); + assertThat(tasks).hasSize(1); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java index 5b8aff686fbc..3697678d63f0 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java @@ -19,7 +19,11 @@ package org.apache.iceberg; import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.Arrays; +import java.util.List; import java.util.Objects; import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; @@ -32,24 +36,17 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSchemaAndMappingUpdate extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSchemaAndMappingUpdate extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestSchemaAndMappingUpdate(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testAddPrimitiveColumn() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -64,15 +61,12 @@ public void testAddPrimitiveColumn() { validateUnchanged(mapping, updated); MappedField newMapping = updated.find("count"); - Assert.assertNotNull("Mapping for new column should be added", newMapping); - Assert.assertEquals( - "Mapping should use the assigned field ID", - (Integer) table.schema().findField("count").fieldId(), - updated.find("count").id()); - Assert.assertNull("Should not contain a nested mapping", updated.find("count").nestedMapping()); + assertThat(newMapping).isNotNull(); + assertThat(updated.find("count").id()).isEqualTo(table.schema().findField("count").fieldId()); + assertThat(updated.find("count").nestedMapping()).isNull(); } - @Test + @TestTemplate public void testAddStructColumn() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -94,31 +88,22 @@ public void testAddStructColumn() { validateUnchanged(mapping, updated); MappedField newMapping = updated.find("location"); - Assert.assertNotNull("Mapping for new column should be added", newMapping); - - Assert.assertEquals( - "Mapping should use the assigned field ID", - (Integer) table.schema().findField("location").fieldId(), - updated.find("location").id()); - Assert.assertNotNull( - "Should contain a nested mapping", updated.find("location").nestedMapping()); - - Assert.assertEquals( - "Mapping should use the assigned field ID", - (Integer) table.schema().findField("location.lat").fieldId(), - updated.find("location.lat").id()); - Assert.assertNull( - "Should not contain a nested mapping", updated.find("location.lat").nestedMapping()); - - Assert.assertEquals( - "Mapping should use the assigned field ID", - (Integer) table.schema().findField("location.long").fieldId(), - updated.find("location.long").id()); - Assert.assertNull( - "Should not contain a nested mapping", updated.find("location.long").nestedMapping()); + assertThat(newMapping).isNotNull(); + + assertThat(updated.find("location").id()) + .isEqualTo(table.schema().findField("location").fieldId()); + assertThat(updated.find("location").nestedMapping()).isNotNull(); + + assertThat(updated.find("location.lat").id()) + .isEqualTo(table.schema().findField("location.lat").fieldId()); + assertThat(updated.find("location.lat").nestedMapping()).isNull(); + + assertThat(updated.find("location.long").id()) + .isEqualTo(table.schema().findField("location.long").fieldId()); + assertThat(updated.find("location.long").nestedMapping()).isNull(); } - @Test + @TestTemplate public void testRenameColumn() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -137,14 +122,12 @@ public void testRenameColumn() { updated); MappedField updatedMapping = updated.find(idColumnId); - Assert.assertNotNull("Mapping for id column should exist", updatedMapping); - Assert.assertEquals( - "Should add the new column name to the existing mapping", - MappedField.of(idColumnId, ImmutableList.of("id", "object_id")), - updatedMapping); + assertThat(updatedMapping) + .isNotNull() + .isEqualTo(MappedField.of(idColumnId, ImmutableList.of("id", "object_id"))); } - @Test + @TestTemplate public void testDeleteColumn() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -160,7 +143,7 @@ public void testDeleteColumn() { validateUnchanged(mapping, updated); } - @Test + @TestTemplate public void testModificationWithMetricsMetrics() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -171,7 +154,7 @@ public void testModificationWithMetricsMetrics() { .set("write.metadata.metrics.column.id", "full") .commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .updateProperties() @@ -184,25 +167,19 @@ public void testModificationWithMetricsMetrics() { // Re-naming a column with metrics succeeds; table.updateSchema().renameColumn("id", "bloop").commit(); - Assert.assertNotNull( - "Make sure the metrics config now has bloop", - table.properties().get(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "bloop")); - Assert.assertNull( - "Make sure the metrics config no longer has id", - table.properties().get(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id")); + assertThat(table.properties()) + .containsEntry(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "bloop", "full") + .doesNotContainKey(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id"); // Deleting a column with metrics succeeds table.updateSchema().deleteColumn("bloop").commit(); // Make sure no more reference to bloop in the metrics config - Assert.assertNull( - "Make sure the metrics config no longer has id", - table.properties().get(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id")); - Assert.assertNull( - "Make sure the metrics config no longer has bloop", - table.properties().get(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "bloop")); + assertThat(table.properties()) + .doesNotContainKey(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "id") + .doesNotContainKey(TableProperties.METRICS_MODE_COLUMN_CONF_PREFIX + "bloop"); } - @Test + @TestTemplate public void testModificationWithParquetBloomConfig() { table .updateProperties() @@ -210,20 +187,17 @@ public void testModificationWithParquetBloomConfig() { .commit(); table.updateSchema().renameColumn("id", "ID").commit(); - Assert.assertNotNull( - "Parquet bloom config for new column name ID should exists", - table.properties().get(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "ID")); - Assert.assertNull( - "Parquet bloom config for old column name id should not exists", - table.properties().get(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id")); + assertThat(table.properties()) + .containsEntry(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "ID", "true") + .doesNotContainKey(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id"); table.updateSchema().deleteColumn("ID").commit(); - Assert.assertNull( - "Parquet bloom config for dropped column name ID should not exists", - table.properties().get(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "ID")); + assertThat(table.properties()) + .doesNotContainKey( + table.properties().get(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "ID")); } - @Test + @TestTemplate public void testDeleteAndAddColumnReassign() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -248,20 +222,18 @@ public void testDeleteAndAddColumnReassign() { updated); MappedField newMapping = updated.find("id"); - Assert.assertNotNull("Mapping for id column should exist", newMapping); - Assert.assertEquals( - "Mapping should use the new field ID", (Integer) idColumnId, newMapping.id()); - Assert.assertNull("Should not contain a nested mapping", newMapping.nestedMapping()); + assertThat(newMapping).isNotNull(); + assertThat(newMapping.id()).isEqualTo(idColumnId); + assertThat(newMapping.nestedMapping()).isNull(); MappedField updatedMapping = updated.find(startIdColumnId); - Assert.assertNotNull("Mapping for original id column should exist", updatedMapping); - Assert.assertEquals( - "Mapping should use the original field ID", (Integer) startIdColumnId, updatedMapping.id()); - Assert.assertFalse("Should not use id as a name", updatedMapping.names().contains("id")); - Assert.assertNull("Should not contain a nested mapping", updatedMapping.nestedMapping()); + assertThat(updatedMapping).isNotNull(); + assertThat(updatedMapping.id()).isEqualTo(startIdColumnId); + assertThat(updatedMapping.names()).doesNotContain("id"); + assertThat(updatedMapping.nestedMapping()).isNull(); } - @Test + @TestTemplate public void testDeleteAndRenameColumnReassign() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -286,22 +258,19 @@ public void testDeleteAndRenameColumnReassign() { updated); MappedField newMapping = updated.find("id"); - Assert.assertNotNull("Mapping for id column should exist", newMapping); - Assert.assertEquals( - "Mapping should use the new field ID", (Integer) idColumnId, newMapping.id()); - Assert.assertEquals( - "Should have both names", Sets.newHashSet("id", "data"), newMapping.names()); - Assert.assertNull("Should not contain a nested mapping", newMapping.nestedMapping()); + assertThat(newMapping).isNotNull(); + assertThat(newMapping.id()).isEqualTo(idColumnId); + assertThat(newMapping.names()).isEqualTo(Sets.newHashSet("id", "data")); + assertThat(newMapping.nestedMapping()).isNull(); MappedField updatedMapping = updated.find(startIdColumnId); - Assert.assertNotNull("Mapping for original id column should exist", updatedMapping); - Assert.assertEquals( - "Mapping should use the original field ID", (Integer) startIdColumnId, updatedMapping.id()); - Assert.assertFalse("Should not use id as a name", updatedMapping.names().contains("id")); - Assert.assertNull("Should not contain a nested mapping", updatedMapping.nestedMapping()); + assertThat(updatedMapping).isNotNull(); + assertThat(updatedMapping.id()).isEqualTo(startIdColumnId); + assertThat(updatedMapping.names()).doesNotContain("id"); + assertThat(updatedMapping.nestedMapping()).isNull(); } - @Test + @TestTemplate public void testRenameAndAddColumnReassign() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -314,10 +283,8 @@ public void testRenameAndAddColumnReassign() { NameMapping afterRename = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - "Renamed column should have both names", - Sets.newHashSet("id", "object_id"), - afterRename.find(startIdColumnId).names()); + assertThat(afterRename.find(startIdColumnId).names()) + .isEqualTo(Sets.newHashSet("id", "object_id")); // add a new column with the renamed column's old name // also, rename the original column again to ensure its names are handled correctly @@ -338,21 +305,18 @@ public void testRenameAndAddColumnReassign() { updated); MappedField newMapping = updated.find("id"); - Assert.assertNotNull("Mapping for id column should exist", newMapping); - Assert.assertEquals( - "Mapping should use the new field ID", (Integer) idColumnId, newMapping.id()); - Assert.assertNull("Should not contain a nested mapping", newMapping.nestedMapping()); + assertThat(newMapping).isNotNull(); + assertThat(newMapping.id()).isEqualTo(idColumnId); + assertThat(newMapping.nestedMapping()).isNull(); MappedField updatedMapping = updated.find(startIdColumnId); - Assert.assertNotNull("Mapping for original id column should exist", updatedMapping); - Assert.assertEquals( - "Mapping should use the original field ID", (Integer) startIdColumnId, updatedMapping.id()); - Assert.assertEquals( - "Should not use id as a name", Sets.newHashSet("object_id", "oid"), updatedMapping.names()); - Assert.assertNull("Should not contain a nested mapping", updatedMapping.nestedMapping()); + assertThat(updatedMapping).isNotNull(); + assertThat(updatedMapping.id()).isEqualTo(startIdColumnId); + assertThat(updatedMapping.names()).isEqualTo(Sets.newHashSet("object_id", "oid")); + assertThat(updatedMapping.nestedMapping()).isNull(); } - @Test + @TestTemplate public void testRenameAndRenameColumnReassign() { NameMapping mapping = MappingUtil.create(table.schema()); String mappingJson = NameMappingParser.toJson(mapping); @@ -365,10 +329,8 @@ public void testRenameAndRenameColumnReassign() { NameMapping afterRename = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - "Renamed column should have both names", - Sets.newHashSet("id", "object_id"), - afterRename.find(startIdColumnId).names()); + assertThat(afterRename.find(startIdColumnId).names()) + .isEqualTo(Sets.newHashSet("id", "object_id")); // rename the data column to the renamed column's old name // also, rename the original column again to ensure its names are handled correctly @@ -385,28 +347,23 @@ public void testRenameAndRenameColumnReassign() { updated); MappedField newMapping = updated.find("id"); - Assert.assertNotNull("Mapping for id column should exist", newMapping); - Assert.assertEquals( - "Renamed column should have both names", Sets.newHashSet("id", "data"), newMapping.names()); - Assert.assertEquals( - "Mapping should use the new field ID", (Integer) idColumnId, newMapping.id()); - Assert.assertNull("Should not contain a nested mapping", newMapping.nestedMapping()); + assertThat(newMapping).isNotNull(); + assertThat(newMapping.names()).isEqualTo(Sets.newHashSet("id", "data")); + assertThat(newMapping.id()).isEqualTo(idColumnId); + assertThat(newMapping.nestedMapping()).isNull(); MappedField updatedMapping = updated.find(startIdColumnId); - Assert.assertNotNull("Mapping for original id column should exist", updatedMapping); - Assert.assertEquals( - "Mapping should use the original field ID", (Integer) startIdColumnId, updatedMapping.id()); - Assert.assertEquals( - "Should not use id as a name", Sets.newHashSet("object_id", "oid"), updatedMapping.names()); - Assert.assertNull("Should not contain a nested mapping", updatedMapping.nestedMapping()); + assertThat(updatedMapping).isNotNull(); + assertThat(updatedMapping.id()).isEqualTo(startIdColumnId); + assertThat(updatedMapping.names()).isEqualTo(Sets.newHashSet("object_id", "oid")); + assertThat(updatedMapping.nestedMapping()).isNull(); } /** Asserts that the fields in the original mapping are unchanged in the updated mapping. */ private void validateUnchanged(NameMapping original, NameMapping updated) { MappedFields updatedFields = updated.asMappedFields(); for (MappedField field : original.asMappedFields().fields()) { - Assert.assertEquals( - "Existing fields should not change", field, updatedFields.field(field.id())); + assertThat(updatedFields.field(field.id())).isEqualTo(field); } } @@ -414,8 +371,7 @@ private void validateUnchanged(NameMapping original, NameMapping updated) { private void validateUnchanged(Iterable fields, NameMapping updated) { MappedFields updatedFields = updated.asMappedFields(); for (MappedField field : fields) { - Assert.assertEquals( - "Existing fields should not change", field, updatedFields.field(field.id())); + assertThat(updatedFields.field(field.id())).isEqualTo(field); } } } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaID.java b/core/src/test/java/org/apache/iceberg/TestSchemaID.java index f27fd92fa3c1..33ac84d20801 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaID.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaID.java @@ -20,32 +20,26 @@ 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.util.Arrays; +import java.util.List; import java.util.Map; import java.util.function.Function; import java.util.stream.Collectors; -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.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestSchemaID extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestSchemaID extends TestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestSchemaID(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testNoChange() { int onlyId = table.schema().schemaId(); Map onlySchemaMap = schemaMap(table.schema()); @@ -54,46 +48,30 @@ public void testNoChange() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); TestHelpers.assertSameSchemaMap(onlySchemaMap, table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be the current", - table.schema().schemaId(), - (int) table.currentSnapshot().schemaId()); + assertThat(table.currentSnapshot().schemaId()).isEqualTo(table.schema().schemaId()); - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of(onlyId), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.snapshots()).extracting(Snapshot::schemaId).containsExactly(onlyId); // remove file from table table.newDelete().deleteFile(FILE_A).commit(); TestHelpers.assertSameSchemaMap(onlySchemaMap, table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be the current", - table.schema().schemaId(), - (int) table.currentSnapshot().schemaId()); + assertThat(table.currentSnapshot().schemaId()).isEqualTo(table.schema().schemaId()); - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of(onlyId, onlyId), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.snapshots()).extracting(Snapshot::schemaId).containsExactly(onlyId, onlyId); // add file to table table.newFastAppend().appendFile(FILE_A2).commit(); TestHelpers.assertSameSchemaMap(onlySchemaMap, table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be the current", - table.schema().schemaId(), - (int) table.currentSnapshot().schemaId()); - - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of(onlyId, onlyId, onlyId), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.currentSnapshot().schemaId()).isEqualTo(table.schema().schemaId()); + + assertThat(table.snapshots()) + .extracting(Snapshot::schemaId) + .containsExactly(onlyId, onlyId, onlyId); } - @Test + @TestTemplate public void testSchemaIdChangeInSchemaUpdate() { Schema originalSchema = table.schema(); @@ -101,15 +79,11 @@ public void testSchemaIdChangeInSchemaUpdate() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); TestHelpers.assertSameSchemaMap(schemaMap(table.schema()), table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be the current", - table.schema().schemaId(), - (int) table.currentSnapshot().schemaId()); + assertThat(table.currentSnapshot().schemaId()).isEqualTo(table.schema().schemaId()); - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of(originalSchema.schemaId()), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.snapshots()) + .extracting(Snapshot::schemaId) + .containsExactly(originalSchema.schemaId()); // update schema table.updateSchema().addColumn("data2", Types.StringType.get()).commit(); @@ -122,50 +96,37 @@ public void testSchemaIdChangeInSchemaUpdate() { optional(3, "data2", Types.StringType.get())); TestHelpers.assertSameSchemaMap(schemaMap(originalSchema, updatedSchema), table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be old since update schema doesn't create new snapshot", - originalSchema.schemaId(), - (int) table.currentSnapshot().schemaId()); - Assert.assertEquals( - "Current schema should match", updatedSchema.asStruct(), table.schema().asStruct()); - - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of(originalSchema.schemaId()), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.currentSnapshot().schemaId()) + .as( + "Current snapshot's schemaId should be old since update schema doesn't create new snapshot") + .isEqualTo(originalSchema.schemaId()); + assertThat(table.schema().asStruct()).isEqualTo(updatedSchema.asStruct()); + + assertThat(table.snapshots()) + .extracting(Snapshot::schemaId) + .containsExactly(originalSchema.schemaId()); // remove file from table table.newDelete().deleteFile(FILE_A).commit(); TestHelpers.assertSameSchemaMap(schemaMap(originalSchema, updatedSchema), table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be the current", - updatedSchema.schemaId(), - (int) table.currentSnapshot().schemaId()); - Assert.assertEquals( - "Current schema should match", updatedSchema.asStruct(), table.schema().asStruct()); - - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of(originalSchema.schemaId(), updatedSchema.schemaId()), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.currentSnapshot().schemaId()).isEqualTo(updatedSchema.schemaId()); + assertThat(table.schema().asStruct()).isEqualTo(updatedSchema.asStruct()); + assertThat(table.snapshots()) + .extracting(Snapshot::schemaId) + .containsExactly(originalSchema.schemaId(), updatedSchema.schemaId()); // add files to table table.newAppend().appendFile(FILE_A2).commit(); TestHelpers.assertSameSchemaMap(schemaMap(originalSchema, updatedSchema), table.schemas()); - Assert.assertEquals( - "Current snapshot's schemaId should be the current", - updatedSchema.schemaId(), - (int) table.currentSnapshot().schemaId()); - Assert.assertEquals( - "Current schema should match", updatedSchema.asStruct(), table.schema().asStruct()); - - Assert.assertEquals( - "Schema ids should be correct in snapshots", - ImmutableList.of( - originalSchema.schemaId(), updatedSchema.schemaId(), updatedSchema.schemaId()), - Lists.transform(Lists.newArrayList(table.snapshots()), Snapshot::schemaId)); + assertThat(table.currentSnapshot().schemaId()).isEqualTo(updatedSchema.schemaId()); + assertThat(table.schema().asStruct()).isEqualTo(updatedSchema.asStruct()); + + assertThat(table.snapshots()) + .extracting(Snapshot::schemaId) + .containsExactly( + originalSchema.schemaId(), updatedSchema.schemaId(), updatedSchema.schemaId()); } private Map schemaMap(Schema... schemas) { diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index 1d903dfbb1a5..943ff9f51acd 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -20,6 +20,8 @@ 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.util.List; import java.util.Set; @@ -30,9 +32,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSchemaUpdate { private static final Schema SCHEMA = @@ -84,7 +84,7 @@ public class TestSchemaUpdate { @Test public void testNoChanges() { Schema identical = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).apply(); - Assert.assertEquals("Should not include any changes", SCHEMA.asStruct(), identical.asStruct()); + assertThat(identical.asStruct()).isEqualTo(SCHEMA.asStruct()); } @Test @@ -114,10 +114,7 @@ public void testDeleteFields() { Schema del = new SchemaUpdate(SCHEMA, 19).deleteColumn(name).apply(); - Assert.assertEquals( - "Should match projection with '" + name + "' removed", - TypeUtil.project(SCHEMA, selected).asStruct(), - del.asStruct()); + assertThat(del.asStruct()).isEqualTo(TypeUtil.project(SCHEMA, selected).asStruct()); } } @@ -148,10 +145,7 @@ public void testDeleteFieldsCaseSensitiveDisabled() { Schema del = new SchemaUpdate(SCHEMA, 19).caseSensitive(false).deleteColumn(name).apply(); - Assert.assertEquals( - "Should match projection with '" + name + "' removed", - TypeUtil.project(SCHEMA, selected).asStruct(), - del.asStruct()); + assertThat(del.asStruct()).isEqualTo(TypeUtil.project(SCHEMA, selected).asStruct()); } } @@ -206,7 +200,7 @@ public void testUpdateTypes() { .updateColumn("locations.long", Types.DoubleType.get()) .apply(); - Assert.assertEquals("Should convert types", expected, updated.asStruct()); + assertThat(updated.asStruct()).isEqualTo(expected); } @Test @@ -261,7 +255,7 @@ public void testUpdateTypesCaseInsensitive() { .updateColumn("Locations.Long", Types.DoubleType.get()) .apply(); - Assert.assertEquals("Should convert types", expected, updated.asStruct()); + assertThat(updated.asStruct()).isEqualTo(expected); } @Test @@ -299,13 +293,12 @@ public void testUpdateFailure() { if (fromType.equals(toType) || allowedUpdates.contains(Pair.of(fromType, toType))) { Schema expected = new Schema(required(1, "col", toType)); Schema result = new SchemaUpdate(fromSchema, 1).updateColumn("col", toType).apply(); - Assert.assertEquals("Should allow update", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); continue; } String typeChange = fromType + " -> " + toType.toString(); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(fromSchema, 1).updateColumn("col", toType)) + assertThatThrownBy(() -> new SchemaUpdate(fromSchema, 1).updateColumn("col", toType)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column type: col: " + typeChange); } @@ -366,7 +359,7 @@ public void testRename() { .renameColumn("points.y", "y.y") // has a '.' in the field name .apply(); - Assert.assertEquals("Should rename all fields", expected, renamed.asStruct()); + assertThat(renamed.asStruct()).isEqualTo(expected); } @Test @@ -424,7 +417,7 @@ public void testRenameCaseInsensitive() { .renameColumn("Points.y", "y.y") // has a '.' in the field name .apply(); - Assert.assertEquals("Should rename all fields", expected, renamed.asStruct()); + assertThat(renamed.asStruct()).isEqualTo(expected); } @Test @@ -483,7 +476,7 @@ public void testAddFields() { .addColumn("points", "t.t", Types.LongType.get()) // name with '.' .apply(); - Assert.assertEquals("Should match with added fields", expected.asStruct(), added.asStruct()); + assertThat(added.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -506,8 +499,7 @@ public void testAddNestedStruct() { Schema result = new SchemaUpdate(schema, 1).addColumn("location", struct).apply(); - Assert.assertEquals( - "Should add struct and reassign column IDs", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -546,8 +538,7 @@ public void testAddNestedMapOfStructs() { Schema result = new SchemaUpdate(schema, 1).addColumn("locations", map).apply(); - Assert.assertEquals( - "Should add map and reassign column IDs", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -574,8 +565,7 @@ public void testAddNestedListOfStructs() { Schema result = new SchemaUpdate(schema, 1).addColumn("locations", list).apply(); - Assert.assertEquals( - "Should add map and reassign column IDs", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -586,7 +576,7 @@ public void testAddRequiredColumn() { required(1, "id", Types.IntegerType.get()), required(2, "data", Types.StringType.get())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 1).addRequiredColumn("data", Types.StringType.get())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Incompatible change: cannot add required column: data"); @@ -597,14 +587,14 @@ public void testAddRequiredColumn() { .addRequiredColumn("data", Types.StringType.get()) .apply(); - Assert.assertEquals("Should add required column", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test public void testAddRequiredColumnCaseInsensitive() { Schema schema = new Schema(required(1, "id", Types.IntegerType.get())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 1) .caseSensitive(false) @@ -622,8 +612,7 @@ public void testMakeColumnOptional() { Schema result = new SchemaUpdate(schema, 1).makeColumnOptional("id").apply(); - Assert.assertEquals( - "Should update column to be optional", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -631,7 +620,7 @@ public void testRequireColumn() { Schema schema = new Schema(optional(1, "id", Types.IntegerType.get())); Schema expected = new Schema(required(1, "id", Types.IntegerType.get())); - Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 1).requireColumn("id")) + assertThatThrownBy(() -> new SchemaUpdate(schema, 1).requireColumn("id")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column nullability: id: optional -> required"); @@ -641,8 +630,7 @@ public void testRequireColumn() { Schema result = new SchemaUpdate(schema, 1).allowIncompatibleChanges().requireColumn("id").apply(); - Assert.assertEquals( - "Should update column to be required", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -657,8 +645,7 @@ public void testRequireColumnCaseInsensitive() { .requireColumn("ID") .apply(); - Assert.assertEquals( - "Should update column to be required", expected.asStruct(), result.asStruct()); + assertThat(result.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -729,13 +716,13 @@ public void testMixedChanges() { "locations", "description", Types.StringType.get(), "Location description") .apply(); - Assert.assertEquals("Should match with added fields", expected.asStruct(), updated.asStruct()); + assertThat(updated.asStruct()).isEqualTo(expected.asStruct()); } @Test public void testAmbiguousAdd() { // preferences.booleans could be top-level or a field of preferences - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.addColumn("preferences.booleans", Types.BooleanType.get()); @@ -746,7 +733,7 @@ public void testAmbiguousAdd() { @Test public void testAddAlreadyExists() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.addColumn("preferences", "feature1", Types.BooleanType.get()); @@ -754,7 +741,7 @@ public void testAddAlreadyExists() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot add column, name already exists: preferences.feature1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.addColumn("preferences", Types.BooleanType.get()); @@ -774,7 +761,7 @@ public void testDeleteThenAdd() { .addColumn("id", optional(2, "id", Types.IntegerType.get()).type()) .apply(); - Assert.assertEquals("Should match with added fields", expected.asStruct(), updated.asStruct()); + assertThat(updated.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -827,13 +814,12 @@ public void testDeleteThenAddNested() { .addColumn("preferences", "feature1", Types.BooleanType.get()) .apply(); - Assert.assertEquals( - "Should match with added fields", expectedNested.asStruct(), updatedNested.asStruct()); + assertThat(updatedNested.asStruct()).isEqualTo(expectedNested.asStruct()); } @Test public void testDeleteMissingColumn() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.deleteColumn("col"); @@ -844,7 +830,7 @@ public void testDeleteMissingColumn() { @Test public void testAddDeleteConflict() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.addColumn("col", Types.IntegerType.get()).deleteColumn("col"); @@ -852,7 +838,7 @@ public void testAddDeleteConflict() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot delete missing column: col"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update @@ -865,7 +851,7 @@ public void testAddDeleteConflict() { @Test public void testRenameMissingColumn() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.renameColumn("col", "fail"); @@ -876,7 +862,7 @@ public void testRenameMissingColumn() { @Test public void testRenameDeleteConflict() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.renameColumn("id", "col").deleteColumn("id"); @@ -884,7 +870,7 @@ public void testRenameDeleteConflict() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot delete a column that has updates: id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.renameColumn("id", "col").deleteColumn("col"); @@ -895,7 +881,7 @@ public void testRenameDeleteConflict() { @Test public void testDeleteRenameConflict() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.deleteColumn("id").renameColumn("id", "identifier"); @@ -906,7 +892,7 @@ public void testDeleteRenameConflict() { @Test public void testUpdateMissingColumn() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.updateColumn("col", Types.DateType.get()); @@ -917,7 +903,7 @@ public void testUpdateMissingColumn() { @Test public void testUpdateDeleteConflict() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.updateColumn("id", Types.LongType.get()).deleteColumn("id"); @@ -928,7 +914,7 @@ public void testUpdateDeleteConflict() { @Test public void testDeleteUpdateConflict() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { UpdateSchema update = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID); update.deleteColumn("id").updateColumn("id", Types.LongType.get()); @@ -939,7 +925,7 @@ public void testDeleteUpdateConflict() { @Test public void testDeleteMapKey() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .deleteColumn("locations.key") @@ -950,7 +936,7 @@ public void testDeleteMapKey() { @Test public void testAddFieldToMapKey() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .addColumn("locations.key", "address_line_2", Types.StringType.get()) @@ -961,7 +947,7 @@ public void testAddFieldToMapKey() { @Test public void testAlterMapKey() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .updateColumn("locations.key.zip", Types.LongType.get()) @@ -978,7 +964,7 @@ public void testUpdateMapKey() { 1, "m", Types.MapType.ofOptional(2, 3, Types.IntegerType.get(), Types.DoubleType.get()))); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 3).updateColumn("m.key", Types.LongType.get()).apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot update map keys: map"); @@ -987,7 +973,7 @@ public void testUpdateMapKey() { @Test public void testUpdateAddedColumnDoc() { Schema schema = new Schema(required(1, "i", Types.IntegerType.get())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 3) .addColumn("value", Types.LongType.get()) @@ -1000,7 +986,7 @@ public void testUpdateAddedColumnDoc() { @Test public void testUpdateDeletedColumnDoc() { Schema schema = new Schema(required(1, "i", Types.IntegerType.get())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 3) .deleteColumn("i") @@ -1035,7 +1021,7 @@ public void testMultipleMoves() { .moveBefore("d", "a") .apply(); - Assert.assertEquals("Schema should match", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1049,7 +1035,7 @@ public void testMoveTopLevelColumnFirst() { Schema actual = new SchemaUpdate(schema, 2).moveFirst("data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1063,7 +1049,7 @@ public void testMoveTopLevelColumnBeforeFirst() { Schema actual = new SchemaUpdate(schema, 2).moveBefore("data", "id").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1077,7 +1063,7 @@ public void testMoveTopLevelColumnAfterLast() { Schema actual = new SchemaUpdate(schema, 2).moveAfter("id", "data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1095,7 +1081,7 @@ public void testMoveTopLevelColumnAfter() { Schema actual = new SchemaUpdate(schema, 3).moveAfter("ts", "id").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1113,7 +1099,7 @@ public void testMoveTopLevelColumnBefore() { Schema actual = new SchemaUpdate(schema, 3).moveBefore("ts", "data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1139,7 +1125,7 @@ public void testMoveNestedFieldFirst() { Schema actual = new SchemaUpdate(schema, 4).moveFirst("struct.data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1165,7 +1151,7 @@ public void testMoveNestedFieldBeforeFirst() { Schema actual = new SchemaUpdate(schema, 4).moveBefore("struct.data", "struct.count").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1191,7 +1177,7 @@ public void testMoveNestedFieldAfterLast() { Schema actual = new SchemaUpdate(schema, 4).moveAfter("struct.count", "struct.data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1219,7 +1205,7 @@ public void testMoveNestedFieldAfter() { Schema actual = new SchemaUpdate(schema, 5).moveAfter("struct.ts", "struct.count").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1247,7 +1233,7 @@ public void testMoveNestedFieldBefore() { Schema actual = new SchemaUpdate(schema, 5).moveBefore("struct.ts", "struct.data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1279,7 +1265,7 @@ public void testMoveListElementField() { Schema actual = new SchemaUpdate(schema, 6).moveBefore("list.ts", "list.data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1315,7 +1301,7 @@ public void testMoveMapValueStructField() { Schema actual = new SchemaUpdate(schema, 7).moveBefore("map.ts", "map.data").apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1335,7 +1321,7 @@ public void testMoveAddedTopLevelColumn() { .moveAfter("ts", "id") .apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1358,7 +1344,7 @@ public void testMoveAddedTopLevelColumnAfterAddedColumn() { .moveAfter("count", "ts") .apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1389,7 +1375,7 @@ public void testMoveAddedNestedStructField() { .moveBefore("struct.ts", "struct.count") .apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1423,7 +1409,7 @@ public void testMoveAddedNestedStructFieldBeforeAddedColumn() { .moveBefore("struct.size", "struct.ts") .apply(); - Assert.assertEquals("Should move data first", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -1432,11 +1418,11 @@ public void testMoveSelfReferenceFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveBefore("id", "id").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveBefore("id", "id").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move id before itself"); - Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveAfter("id", "id").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveAfter("id", "id").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move id after itself"); } @@ -1447,17 +1433,15 @@ public void testMoveMissingColumnFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - Assertions.assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveFirst("items").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveFirst("items").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move missing column: items"); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 2).moveBefore("items", "id").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveBefore("items", "id").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move missing column: items"); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 2).moveAfter("items", "data").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveAfter("items", "data").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move missing column: items"); } @@ -1468,7 +1452,7 @@ public void testMoveBeforeAddFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 2) .moveFirst("ts") @@ -1477,7 +1461,7 @@ public void testMoveBeforeAddFails() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move missing column: ts"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 2) .moveBefore("ts", "id") @@ -1486,7 +1470,7 @@ public void testMoveBeforeAddFails() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move missing column: ts"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schema, 2) .moveAfter("ts", "data") @@ -1502,13 +1486,11 @@ public void testMoveMissingReferenceColumnFails() { new Schema( required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 2).moveBefore("id", "items").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveBefore("id", "items").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move id before missing column: items"); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 2).moveAfter("data", "items").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 2).moveAfter("data", "items").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move data after missing column: items"); } @@ -1524,8 +1506,7 @@ public void testMovePrimitiveMapKeyFails() { "map", Types.MapType.ofRequired(4, 5, Types.StringType.get(), Types.StringType.get()))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 5).moveBefore("map.key", "map.value").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 5).moveBefore("map.key", "map.value").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move fields in non-struct type: map"); } @@ -1541,8 +1522,7 @@ public void testMovePrimitiveMapValueFails() { "map", Types.MapType.ofRequired(4, 5, Types.StringType.get(), Types.StructType.of()))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 5).moveBefore("map.value", "map.key").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 5).moveBefore("map.value", "map.key").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move fields in non-struct type: map>"); } @@ -1555,8 +1535,7 @@ public void testMovePrimitiveListElementFails() { required(2, "data", Types.StringType.get()), optional(3, "list", Types.ListType.ofRequired(4, Types.StringType.get()))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 4).moveBefore("list.element", "list").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 4).moveBefore("list.element", "list").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move fields in non-struct type: list"); } @@ -1574,8 +1553,7 @@ public void testMoveTopLevelBetweenStructsFails() { required(4, "x", Types.IntegerType.get()), required(5, "y", Types.IntegerType.get())))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 5).moveBefore("a", "struct.x").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 5).moveBefore("a", "struct.x").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move field a to a different struct"); } @@ -1597,8 +1575,7 @@ public void testMoveBetweenStructsFails() { required(5, "x", Types.IntegerType.get()), required(6, "y", Types.IntegerType.get())))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(schema, 6).moveBefore("s2.x", "s1.a").apply()) + assertThatThrownBy(() -> new SchemaUpdate(schema, 6).moveBefore("s2.x", "s1.a").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot move field s2.x to a different struct"); } @@ -1608,10 +1585,9 @@ public void testAddExistingIdentifierFields() { Schema newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("id").apply(); - Assert.assertEquals( - "add an existing field as identifier field should succeed", - Sets.newHashSet(newSchema.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("add an existing field as identifier field should succeed") + .containsExactly(newSchema.findField("id").fieldId()); } @Test @@ -1623,11 +1599,11 @@ public void testAddNewIdentifierFieldColumns() { .setIdentifierFields("id", "new_field") .apply(); - Assert.assertEquals( - "add column then set as identifier should succeed", - Sets.newHashSet( - newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("add column then set as identifier should succeed") + .isEqualTo( + Sets.newHashSet( + newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId())); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) @@ -1636,11 +1612,11 @@ public void testAddNewIdentifierFieldColumns() { .addRequiredColumn("new_field", Types.StringType.get()) .apply(); - Assert.assertEquals( - "set identifier then add column should succeed", - Sets.newHashSet( - newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("set identifier then add column should succeed") + .isEqualTo( + Sets.newHashSet( + newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId())); } @Test @@ -1660,10 +1636,9 @@ public void testAddNestedIdentifierFieldColumns() { .setIdentifierFields("required_struct.field") .apply(); - Assert.assertEquals( - "set existing nested field as identifier should succeed", - Sets.newHashSet(newSchema.findField("required_struct.field").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("set existing nested field as identifier should succeed") + .containsExactly(newSchema.findField("required_struct.field").fieldId()); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) @@ -1676,10 +1651,9 @@ public void testAddNestedIdentifierFieldColumns() { .setIdentifierFields("new.field") .apply(); - Assert.assertEquals( - "set newly added nested field as identifier should succeed", - Sets.newHashSet(newSchema.findField("new.field").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("set newly added nested field as identifier should succeed") + .containsExactly(newSchema.findField("new.field").fieldId()); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) @@ -1696,10 +1670,9 @@ public void testAddNestedIdentifierFieldColumns() { .setIdentifierFields("new.field.nested") .apply(); - Assert.assertEquals( - "set newly added multi-layer nested field as identifier should succeed", - Sets.newHashSet(newSchema.findField("new.field.nested").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("set newly added multi-layer nested field as identifier should succeed") + .containsExactly(newSchema.findField("new.field.nested").fieldId()); } @Test @@ -1711,11 +1684,11 @@ public void testAddDottedIdentifierFieldColumns() { .setIdentifierFields("id", "dot.field") .apply(); - Assert.assertEquals( - "add a field with dot as identifier should succeed", - Sets.newHashSet( - newSchema.findField("id").fieldId(), newSchema.findField("dot.field").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("add a field with dot as identifier should succeed") + .isEqualTo( + Sets.newHashSet( + newSchema.findField("id").fieldId(), newSchema.findField("dot.field").fieldId())); } @Test @@ -1733,22 +1706,19 @@ public void testRemoveIdentifierFields() { .setIdentifierFields("new_field", "new_field2") .apply(); - Assert.assertEquals( - "remove an identifier field should succeed", - Sets.newHashSet( - newSchema.findField("new_field").fieldId(), - newSchema.findField("new_field2").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("remove an identifier field should succeed") + .isEqualTo( + Sets.newHashSet( + newSchema.findField("new_field").fieldId(), + newSchema.findField("new_field2").fieldId())); newSchema = new SchemaUpdate(newSchema, SCHEMA_LAST_COLUMN_ID) .setIdentifierFields(Sets.newHashSet()) .apply(); - Assert.assertEquals( - "remove all identifier fields should succeed", - Sets.newHashSet(), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()).isEmpty(); } @SuppressWarnings("MethodLength") @@ -1760,29 +1730,25 @@ public void testSetIdentifierFieldsFails() { required(2, "float", Types.FloatType.get()), required(3, "double", Types.DoubleType.get())); - Assertions.assertThatThrownBy( - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(999))) + assertThatThrownBy(() -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(999))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot add fieldId 999 as an identifier field: field does not exist"); - Assertions.assertThatThrownBy( - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(1))) + assertThatThrownBy(() -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(1))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot add field id as an identifier field: not a required field"); - Assertions.assertThatThrownBy( - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(2))) + assertThatThrownBy(() -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(2))) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot add field float as an identifier field: must not be float or double field"); - Assertions.assertThatThrownBy( - () -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(3))) + assertThatThrownBy(() -> new Schema(testSchema.asStruct().fields(), ImmutableSet.of(3))) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot add field double as an identifier field: must not be float or double field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .setIdentifierFields("unknown") @@ -1791,7 +1757,7 @@ public void testSetIdentifierFieldsFails() { .hasMessage( "Cannot add field unknown as an identifier field: not found in current schema or added columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .setIdentifierFields("locations") @@ -1800,13 +1766,13 @@ public void testSetIdentifierFieldsFails() { .hasMessage( "Cannot add field locations as an identifier field: not a primitive type field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("data").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot add field data as an identifier field: not a required field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .setIdentifierFields("locations.key.zip") @@ -1816,7 +1782,7 @@ public void testSetIdentifierFieldsFails() { "Cannot add field zip as an identifier field: must not be nested in " + SCHEMA.findField("locations")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) .setIdentifierFields("points.element.x") @@ -1865,7 +1831,7 @@ public void testSetIdentifierFieldsFails() { int lastColId = SCHEMA_LAST_COLUMN_ID + 15; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, lastColId) .setIdentifierFields("required_list.element.x") @@ -1875,19 +1841,19 @@ public void testSetIdentifierFieldsFails() { "Cannot add field x as an identifier field: must not be nested in " + newSchema.findField("required_list")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, lastColId).setIdentifierFields("col_double").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot add field col_double as an identifier field: must not be float or double field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, lastColId).setIdentifierFields("col_float").apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot add field col_float as an identifier field: must not be float or double field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, lastColId) .setIdentifierFields("new_map.value.val_col") @@ -1897,7 +1863,7 @@ public void testSetIdentifierFieldsFails() { "Cannot add field val_col as an identifier field: must not be nested in " + newSchema.findField("new_map")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, lastColId) .setIdentifierFields("new.fields.element.nested") @@ -1907,7 +1873,7 @@ public void testSetIdentifierFieldsFails() { "Cannot add field nested as an identifier field: must not be nested in " + newSchema.findField("new.fields")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, lastColId) .setIdentifierFields("preferences.feature1") @@ -1923,23 +1889,23 @@ public void testDeleteIdentifierFieldColumns() { Schema schemaWithIdentifierFields = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("id").apply(); - Assert.assertEquals( - "delete column and then reset identifier field should succeed", - Sets.newHashSet(), - new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) - .deleteColumn("id") - .setIdentifierFields(Sets.newHashSet()) - .apply() - .identifierFieldIds()); + assertThat( + new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) + .deleteColumn("id") + .setIdentifierFields(Sets.newHashSet()) + .apply() + .identifierFieldIds()) + .as("delete column and then reset identifier field should succeed") + .isEmpty(); - Assert.assertEquals( - "delete reset identifier field and then delete column should succeed", - Sets.newHashSet(), - new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) - .setIdentifierFields(Sets.newHashSet()) - .deleteColumn("id") - .apply() - .identifierFieldIds()); + assertThat( + new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) + .setIdentifierFields(Sets.newHashSet()) + .deleteColumn("id") + .apply() + .identifierFieldIds()) + .as("delete reset identifier field and then delete column should succeed") + .isEmpty(); } @Test @@ -1947,7 +1913,7 @@ public void testDeleteIdentifierFieldColumnsFails() { Schema schemaWithIdentifierFields = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID).setIdentifierFields("id").apply(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) .deleteColumn("id") @@ -1970,7 +1936,7 @@ public void testDeleteContainingNestedIdentifierFieldColumnsFails() { .setIdentifierFields("out.nested") .apply(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new SchemaUpdate(newSchema, SCHEMA_LAST_COLUMN_ID + 2).deleteColumn("out").apply()) .isInstanceOf(IllegalArgumentException.class) @@ -1989,10 +1955,9 @@ public void testRenameIdentifierFields() { .renameColumn("id", "id2") .apply(); - Assert.assertEquals( - "rename should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("rename should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); } @Test @@ -2005,28 +1970,25 @@ public void testMoveIdentifierFields() { .moveAfter("id", "locations") .apply(); - Assert.assertEquals( - "move after should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("move after should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); newSchema = new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) .moveBefore("id", "locations") .apply(); - Assert.assertEquals( - "move before should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("move before should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); newSchema = new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID).moveFirst("id").apply(); - Assert.assertEquals( - "move first should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("move first should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); } @Test @@ -2040,10 +2002,9 @@ public void testMoveIdentifierFieldsCaseInsensitive() { .moveAfter("iD", "locations") .apply(); - Assert.assertEquals( - "move after should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("move after should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); newSchema = new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) @@ -2051,10 +2012,9 @@ public void testMoveIdentifierFieldsCaseInsensitive() { .moveBefore("ID", "locations") .apply(); - Assert.assertEquals( - "move before should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("move before should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); newSchema = new SchemaUpdate(schemaWithIdentifierFields, SCHEMA_LAST_COLUMN_ID) @@ -2062,10 +2022,9 @@ public void testMoveIdentifierFieldsCaseInsensitive() { .moveFirst("ID") .apply(); - Assert.assertEquals( - "move first should not affect identifier fields", - Sets.newHashSet(SCHEMA.findField("id").fieldId()), - newSchema.identifierFieldIds()); + assertThat(newSchema.identifierFieldIds()) + .as("move first should not affect identifier fields") + .containsExactly(SCHEMA.findField("id").fieldId()); } @Test @@ -2088,8 +2047,7 @@ public void testMoveTopDeletedColumnAfterAnotherColumn() { .addRequiredColumn("id", Types.IntegerType.get()) .moveAfter("id", "data") .apply(); - Assert.assertEquals( - "Should move deleted column correctly", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -2112,8 +2070,7 @@ public void testMoveTopDeletedColumnBeforeAnotherColumn() { .addRequiredColumn("id", Types.IntegerType.get()) .moveBefore("id", "data_1") .apply(); - Assert.assertEquals( - "Should move deleted column correctly", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -2136,8 +2093,7 @@ public void testMoveTopDeletedColumnToFirst() { .addRequiredColumn("id", Types.IntegerType.get()) .moveFirst("id") .apply(); - Assert.assertEquals( - "Should move deleted column correctly", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -2171,8 +2127,7 @@ public void testMoveDeletedNestedStructFieldAfterAnotherColumn() { .moveAfter("struct.data", "struct.count") .apply(); - Assert.assertEquals( - "Should move deleted nested column correctly", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -2206,8 +2161,7 @@ public void testMoveDeletedNestedStructFieldBeforeAnotherColumn() { .moveBefore("struct.data", "struct.data_1") .apply(); - Assert.assertEquals( - "Should move deleted nested column correctly", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -2241,7 +2195,6 @@ public void testMoveDeletedNestedStructFieldToFirst() { .moveFirst("struct.data") .apply(); - Assert.assertEquals( - "Should move deleted nested column correctly", expected.asStruct(), actual.asStruct()); + assertThat(actual.asStruct()).isEqualTo(expected.asStruct()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java index f3bfdf669e33..a4e587068e74 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java @@ -21,38 +21,33 @@ import static org.apache.iceberg.expressions.Expressions.bucket; import static org.apache.iceberg.expressions.Expressions.truncate; import static org.apache.iceberg.expressions.Expressions.year; +import static org.assertj.core.api.Assertions.assertThat; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestTableUpdatePartitionSpec extends TableTestBase { - - @Parameterized.Parameters - public static Object[][] parameters() { - return new Object[][] { - new Object[] {1}, new Object[] {2}, - }; - } +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestTableUpdatePartitionSpec extends TestBase { - public TestTableUpdatePartitionSpec(int formatVersion) { - super(formatVersion); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Before + @BeforeEach public void verifyInitialSpec() { PartitionSpec initialSpec = PartitionSpec.builderFor(table.schema()).bucket("data", 16).build(); - Assert.assertEquals("Should use the expected initial spec", initialSpec, table.spec()); - Assert.assertEquals(1000, table.spec().lastAssignedFieldId()); - Assert.assertEquals(0, table.spec().specId()); + assertThat(table.spec()).isEqualTo(initialSpec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1000); + assertThat(table.spec().specId()).isEqualTo(0); } - @Test + @TestTemplate public void testCommitUpdatedSpec() { table.updateSpec().addField(bucket("id", 8)).commit(); @@ -62,8 +57,10 @@ public void testCommitUpdatedSpec() { .bucket("data", 16) .bucket("id", 8, "id_bucket_8") .build(); - Assert.assertEquals("Should append a partition field to the spec", evolvedSpec, table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); + assertThat(table.spec()) + .as("Should append a partition field to the spec") + .isEqualTo(evolvedSpec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); table .updateSpec() @@ -90,10 +87,10 @@ public void testCommitUpdatedSpec() { .build(), table.spec()); - Assert.assertEquals(1002, table.spec().lastAssignedFieldId()); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1002); } - @Test + @TestTemplate public void testNoopCommit() { TableMetadata current = table.ops().current(); int currentVersion = TestTables.metadataVersion("test"); @@ -102,20 +99,20 @@ public void testNoopCommit() { table.updateSpec().commit(); TableMetadata updated = table.ops().current(); Integer updatedVersion = TestTables.metadataVersion("test"); - Assert.assertEquals(current, updated); + assertThat(updated).isEqualTo(current); currentVersion += 1; - Assert.assertEquals(currentVersion, updatedVersion.intValue()); + assertThat(updatedVersion).isEqualTo(currentVersion); // no-op commit due to no-op rename table.updateSpec().renameField("data_bucket", "data_bucket").commit(); updated = table.ops().current(); updatedVersion = TestTables.metadataVersion("test"); - Assert.assertEquals(current, updated); + assertThat(updated).isEqualTo(current); currentVersion += 1; - Assert.assertEquals(currentVersion, updatedVersion.intValue()); + assertThat(updatedVersion).isEqualTo(currentVersion); } - @Test + @TestTemplate public void testRenameField() { table .updateSpec() @@ -130,8 +127,8 @@ public void testRenameField() { .bucket("id", 8, "id_bucket_8") .build(); - Assert.assertEquals("should match evolved spec", evolvedSpec, table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); + assertThat(table.spec()).isEqualTo(evolvedSpec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); table .updateSpec() @@ -147,11 +144,11 @@ public void testRenameField() { .truncate("id", 4, "id_trunc_4") .build(); - Assert.assertEquals("should match evolved spec", evolvedSpec, table.spec()); - Assert.assertEquals(1002, table.spec().lastAssignedFieldId()); + assertThat(table.spec()).isEqualTo(evolvedSpec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1002); } - @Test + @TestTemplate public void testRenameOnlyEvolution() { table.updateSpec().renameField("data_bucket", "data_partition").commit(); @@ -161,11 +158,11 @@ public void testRenameOnlyEvolution() { .bucket("data", 16, "data_partition") .build(); - Assert.assertEquals("should match evolved spec", evolvedSpec, table.spec()); - Assert.assertEquals(1000, table.spec().lastAssignedFieldId()); + assertThat(table.spec()).isEqualTo(evolvedSpec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1000); } - @Test + @TestTemplate public void testRemoveAndAddField() { table.updateSpec().removeField("data_bucket").addField(bucket("id", 8)).commit(); @@ -186,10 +183,10 @@ public void testRemoveAndAddField() { .build(), table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); } - @Test + @TestTemplate public void testRemoveAndAddYearField() { table.updateSchema().addColumn("year_field", Types.DateType.get()).commit(); table.updateSpec().addField(year("year_field")).commit(); @@ -201,8 +198,8 @@ public void testRemoveAndAddYearField() { .year("year_field") .build(); - Assert.assertEquals("should match evolved spec", evolvedSpec, table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); + assertThat(table.spec()).isEqualTo(evolvedSpec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); table.updateSpec().removeField("year_field_year").addField(year("year_field")).commit(); @@ -224,10 +221,10 @@ public void testRemoveAndAddYearField() { .build(), table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); } - @Test + @TestTemplate public void testAddAndRemoveField() { table.updateSpec().addField(bucket("data", 6)).removeField("data_bucket").commit(); @@ -246,10 +243,10 @@ public void testAddAndRemoveField() { .add(2, 1001, "data_bucket_6", Transforms.bucket(6)) .build(), table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); } - @Test + @TestTemplate public void testAddAfterLastFieldRemoved() { table.updateSpec().removeField("data_bucket").commit(); @@ -268,7 +265,7 @@ public void testAddAfterLastFieldRemoved() { table.spec()); V2Assert.assertEquals( "Should match the last assigned field id", 999, table.spec().lastAssignedFieldId()); - Assert.assertEquals(1000, table.ops().current().lastAssignedPartitionId()); + assertThat(table.ops().current().lastAssignedPartitionId()).isEqualTo(1000); table.updateSpec().addField(bucket("id", 8)).commit(); @@ -287,7 +284,7 @@ public void testAddAfterLastFieldRemoved() { .add(1, 1001, "id_bucket_8", Transforms.bucket(8)) .build(), table.spec()); - Assert.assertEquals(1001, table.spec().lastAssignedFieldId()); - Assert.assertEquals(1001, table.ops().current().lastAssignedPartitionId()); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); + assertThat(table.ops().current().lastAssignedPartitionId()).isEqualTo(1001); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java index 7cf993307e3d..08714dec01f1 100644 --- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java @@ -20,27 +20,25 @@ 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.Files; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestTimestampPartitions extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +@ExtendWith(ParameterizedTestExtension.class) +public class TestTimestampPartitions extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestTimestampPartitions(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testPartitionAppend() throws IOException { Schema dateSchema = new Schema( @@ -58,8 +56,8 @@ public void testPartitionAppend() throws IOException { .withPartitionPath("date=2018-06-08") .build(); - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); this.table = TestTables.create( @@ -67,7 +65,7 @@ public void testPartitionAppend() throws IOException { table.newAppend().appendFile(dataFile).commit(); long id = table.currentSnapshot().snapshotId(); - Assert.assertEquals(table.currentSnapshot().allManifests(table.io()).size(), 1); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); validateManifestEntries( table.currentSnapshot().allManifests(table.io()).get(0), ids(id), diff --git a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java index 5ecf138870ad..33b003cfd8c5 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java @@ -25,18 +25,18 @@ import static org.apache.iceberg.expressions.Expressions.ref; import static org.apache.iceberg.expressions.Expressions.truncate; import static org.apache.iceberg.expressions.Expressions.year; +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.ImmutableList; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestUpdatePartitionSpec extends TableTestBase { +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestUpdatePartitionSpec extends TestBase { private static final Schema SCHEMA = new Schema( Types.NestedField.required(1, "id", Types.LongType.get()), @@ -52,76 +52,72 @@ public class TestUpdatePartitionSpec extends TableTestBase { .bucket("id", 16, "shard") .build(); - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestUpdatePartitionSpec(int formatVersion) { - super(formatVersion); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Test + @TestTemplate public void testAddIdentityByName() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED).addField("category").apply(); PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).identity("category").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddIdentityByTerm() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED).addField(ref("category")).apply(); PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).identity("category").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddYear() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED).addField(year("ts")).apply(); PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).year("ts").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddMonth() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED).addField(month("ts")).apply(); PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).month("ts").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddDay() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED).addField(day("ts")).apply(); PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).day("ts").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddHour() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED).addField(hour("ts")).apply(); PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).hour("ts").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddBucket() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) @@ -132,10 +128,10 @@ public void testAddBucket() { PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).bucket("id", 16, "id_bucket_16").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddTruncate() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) @@ -146,10 +142,10 @@ public void testAddTruncate() { PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).truncate("data", 4, "data_trunc_4").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddNamedPartition() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) @@ -158,10 +154,10 @@ public void testAddNamedPartition() { PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).bucket("id", 16, "shard").build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddToExisting() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) @@ -176,10 +172,10 @@ public void testAddToExisting() { .truncate("data", 4, "data_trunc_4") .build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testMultipleAdds() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) @@ -197,10 +193,10 @@ public void testMultipleAdds() { .truncate("data", 4, "prefix") .build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testAddHourToDay() { // multiple partitions for the same source with different time granularity is not allowed by the // builder, but is @@ -211,15 +207,13 @@ public void testAddHourToDay() { PartitionSpec byHour = new BaseUpdatePartitionSpec(formatVersion, byDay).addField(hour("ts")).apply(); - Assert.assertEquals( - "Should have a day and an hour time field", - ImmutableList.of( + assertThat(byHour.fields()) + .containsExactly( new PartitionField(2, 1000, "ts_day", Transforms.day()), - new PartitionField(2, 1001, "ts_hour", Transforms.hour())), - byHour.fields()); + new PartitionField(2, 1001, "ts_hour", Transforms.hour())); } - @Test + @TestTemplate public void testAddMultipleBuckets() { PartitionSpec bucket16 = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) @@ -235,10 +229,10 @@ public void testAddMultipleBuckets() { .bucket("id", 8, "id_bucket_8") .build(); - Assert.assertEquals("Should have multiple bucket partition fields", expected, bucket8); + assertThat(bucket8).isEqualTo(expected); } - @Test + @TestTemplate public void testRemoveIdentityByName() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).removeField("category").apply(); @@ -261,7 +255,7 @@ public void testRemoveIdentityByName() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRemoveBucketByName() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).removeField("shard").apply(); @@ -284,7 +278,7 @@ public void testRemoveBucketByName() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRemoveIdentityByEquivalent() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) @@ -309,7 +303,7 @@ public void testRemoveIdentityByEquivalent() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRemoveDayByEquivalent() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).removeField(day("ts")).apply(); @@ -332,7 +326,7 @@ public void testRemoveDayByEquivalent() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRemoveBucketByEquivalent() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) @@ -354,7 +348,7 @@ public void testRemoveBucketByEquivalent() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRename() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) @@ -364,10 +358,10 @@ public void testRename() { PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).identity("category").day("ts").bucket("id", 16).build(); - Assert.assertEquals("Should match expected spec", expected, updated); + assertThat(updated).isEqualTo(expected); } - @Test + @TestTemplate public void testMultipleChanges() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) @@ -396,7 +390,7 @@ public void testMultipleChanges() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testAddDeletedName() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) @@ -418,9 +412,9 @@ public void testAddDeletedName() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRemoveNewlyAddedFieldByName() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField("prefix", truncate("data", 4)) @@ -429,9 +423,9 @@ public void testRemoveNewlyAddedFieldByName() { .hasMessageStartingWith("Cannot delete newly added field"); } - @Test + @TestTemplate public void testRemoveNewlyAddedFieldByTransform() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField("prefix", truncate("data", 4)) @@ -440,9 +434,9 @@ public void testRemoveNewlyAddedFieldByTransform() { .hasMessageStartingWith("Cannot delete newly added field"); } - @Test + @TestTemplate public void testAddAlreadyAddedFieldByTransform() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField("prefix", truncate("data", 4)) @@ -451,9 +445,9 @@ public void testAddAlreadyAddedFieldByTransform() { .hasMessageStartingWith("Cannot add duplicate partition field"); } - @Test + @TestTemplate public void testAddAlreadyAddedFieldByName() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField("prefix", truncate("data", 4)) @@ -462,9 +456,9 @@ public void testAddAlreadyAddedFieldByName() { .hasMessageStartingWith("Cannot add duplicate partition field"); } - @Test + @TestTemplate public void testAddRedundantTimePartition() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) .addField(day("ts")) @@ -472,7 +466,7 @@ public void testAddRedundantTimePartition() { .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot add redundant partition field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField(hour("ts")) // does not conflict with day because day already exists @@ -481,78 +475,66 @@ public void testAddRedundantTimePartition() { .hasMessageStartingWith("Cannot add redundant partition"); } - @Test + @TestTemplate public void testNoEffectAddDeletedSameFieldWithSameName() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .removeField("shard") .addField("shard", bucket("id", 16)) .apply(); - Assert.assertEquals(PARTITIONED, updated); + assertThat(updated).isEqualTo(PARTITIONED); updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .removeField("shard") .addField(bucket("id", 16)) .apply(); - Assert.assertEquals(PARTITIONED, updated); + assertThat(updated).isEqualTo(PARTITIONED); } - @Test + @TestTemplate public void testGenerateNewSpecAddDeletedSameFieldWithDifferentName() { PartitionSpec updated = new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .removeField("shard") .addField("new_shard", bucket("id", 16)) .apply(); - Assert.assertEquals("Should match expected field size", 3, updated.fields().size()); - Assert.assertEquals( - "Should match expected field name", "category", updated.fields().get(0).name()); - Assert.assertEquals( - "Should match expected field name", "ts_day", updated.fields().get(1).name()); - Assert.assertEquals( - "Should match expected field name", "new_shard", updated.fields().get(2).name()); - Assert.assertEquals( - "Should match expected field transform", - "identity", - updated.fields().get(0).transform().toString()); - Assert.assertEquals( - "Should match expected field transform", - "day", - updated.fields().get(1).transform().toString()); - Assert.assertEquals( - "Should match expected field transform", - "bucket[16]", - updated.fields().get(2).transform().toString()); - } - - @Test + assertThat(updated.fields()).hasSize(3); + assertThat(updated.fields().get(0).name()).isEqualTo("category"); + assertThat(updated.fields().get(1).name()).isEqualTo("ts_day"); + assertThat(updated.fields().get(2).name()).isEqualTo("new_shard"); + assertThat(updated.fields().get(0).transform()).asString().isEqualTo("identity"); + assertThat(updated.fields().get(1).transform()).asString().isEqualTo("day"); + assertThat(updated.fields().get(2).transform()).asString().isEqualTo("bucket[16]"); + } + + @TestTemplate public void testAddDuplicateByName() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField("category")) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot add duplicate partition field"); } - @Test + @TestTemplate public void testAddDuplicateByRef() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField(ref("category"))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot add duplicate partition field"); } - @Test + @TestTemplate public void testAddDuplicateTransform() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).addField(bucket("id", 16))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot add duplicate partition field"); } - @Test + @TestTemplate public void testAddNamedDuplicate() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField("b16", bucket("id", 16))) @@ -560,17 +542,17 @@ public void testAddNamedDuplicate() { .hasMessageStartingWith("Cannot add duplicate partition field"); } - @Test + @TestTemplate public void testRemoveUnknownFieldByName() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED).removeField("moon")) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find partition field to remove"); } - @Test + @TestTemplate public void testRemoveUnknownFieldByEquivalent() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .removeField(hour("ts")) // day(ts) exists @@ -579,9 +561,9 @@ public void testRemoveUnknownFieldByEquivalent() { .hasMessageStartingWith("Cannot find partition field to remove"); } - @Test + @TestTemplate public void testRenameUnknownField() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .renameField("shake", "seal")) @@ -589,9 +571,9 @@ public void testRenameUnknownField() { .hasMessage("Cannot find partition field to rename: shake"); } - @Test + @TestTemplate public void testRenameAfterAdd() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .addField("data_trunc", truncate("data", 4)) @@ -600,9 +582,9 @@ public void testRenameAfterAdd() { .hasMessage("Cannot rename newly added partition field: data_trunc"); } - @Test + @TestTemplate public void testRenameAndDelete() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .renameField("shard", "id_bucket") @@ -611,9 +593,9 @@ public void testRenameAndDelete() { .hasMessage("Cannot rename and delete partition field: shard"); } - @Test + @TestTemplate public void testDeleteAndRename() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new BaseUpdatePartitionSpec(formatVersion, PARTITIONED) .removeField(bucket("id", 16)) @@ -622,7 +604,7 @@ public void testDeleteAndRename() { .hasMessage("Cannot delete and rename partition field: shard"); } - @Test + @TestTemplate public void testRemoveAndAddMultiTimes() { PartitionSpec addFirstTime = new BaseUpdatePartitionSpec(formatVersion, UNPARTITIONED) @@ -644,29 +626,15 @@ public void testRemoveAndAddMultiTimes() { .apply(); if (formatVersion == 1) { - Assert.assertEquals("Should match expected spec field size", 3, updated.fields().size()); - - Assert.assertTrue( - "Should match expected field name", - updated.fields().get(0).name().matches("^ts_date(?:_\\d+)+$")); - Assert.assertTrue( - "Should match expected field name", - updated.fields().get(1).name().matches("^ts_date_(?:\\d+)+$")); - Assert.assertEquals( - "Should match expected field name", "ts_date", updated.fields().get(2).name()); - - Assert.assertEquals( - "Should match expected field transform", - "void", - updated.fields().get(0).transform().toString()); - Assert.assertEquals( - "Should match expected field transform", - "void", - updated.fields().get(1).transform().toString()); - Assert.assertEquals( - "Should match expected field transform", - "month", - updated.fields().get(2).transform().toString()); + assertThat(updated.fields()).hasSize(3); + + assertThat(updated.fields().get(0).name()).matches("^ts_date(?:_\\d+)+$"); + assertThat(updated.fields().get(1).name()).matches("^ts_date(?:_\\d+)+$"); + assertThat(updated.fields().get(2).name()).isEqualTo("ts_date"); + + assertThat(updated.fields().get(0).transform()).asString().isEqualTo("void"); + assertThat(updated.fields().get(1).transform()).asString().isEqualTo("void"); + assertThat(updated.fields().get(2).transform()).asString().isEqualTo("month"); } PartitionSpec v2Expected = PartitionSpec.builderFor(SCHEMA).month("ts", "ts_date").build(); @@ -674,7 +642,7 @@ public void testRemoveAndAddMultiTimes() { V2Assert.assertEquals("Should match expected spec", v2Expected, updated); } - @Test + @TestTemplate public void testRemoveAndUpdateWithDifferentTransformation() { PartitionSpec expected = PartitionSpec.builderFor(SCHEMA).month("ts", "ts_transformed").build(); PartitionSpec updated = @@ -684,30 +652,16 @@ public void testRemoveAndUpdateWithDifferentTransformation() { .apply(); if (formatVersion == 1) { - Assert.assertEquals("Should match expected spec field size", 2, updated.fields().size()); - Assert.assertEquals( - "Should match expected field name", - "ts_transformed_1000", - updated.fields().get(0).name()); - Assert.assertEquals( - "Should match expected field name", "ts_transformed", updated.fields().get(1).name()); - - Assert.assertEquals( - "Should match expected field transform", - "void", - updated.fields().get(0).transform().toString()); - Assert.assertEquals( - "Should match expected field transform", - "day", - updated.fields().get(1).transform().toString()); + assertThat(updated.fields()).hasSize(2); + assertThat(updated.fields().get(0).name()).isEqualTo("ts_transformed_1000"); + assertThat(updated.fields().get(1).name()).isEqualTo("ts_transformed"); + + assertThat(updated.fields().get(0).transform()).asString().isEqualTo("void"); + assertThat(updated.fields().get(1).transform()).asString().isEqualTo("day"); } else { - Assert.assertEquals("Should match expected spec field size", 1, updated.fields().size()); - Assert.assertEquals( - "Should match expected field name", "ts_transformed", updated.fields().get(0).name()); - Assert.assertEquals( - "Should match expected field transform", - "day", - updated.fields().get(0).transform().toString()); + assertThat(updated.fields()).hasSize(1); + assertThat(updated.fields().get(0).name()).isEqualTo("ts_transformed"); + assertThat(updated.fields().get(0).transform()).asString().isEqualTo("day"); } } From b8b1f6697eb4ea191f10cbf5a32f719eb0f7882e Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Thu, 21 Mar 2024 14:08:16 -0700 Subject: [PATCH 0154/1019] Kafka Connect: Record converters (#9641) --- .../connect/events/TableReference.java | 7 +- .../iceberg/connect/IcebergSinkConfig.java | 6 +- .../iceberg/connect/IcebergSinkConnector.java | 5 +- .../iceberg/connect/data/IcebergWriter.java | 32 +- .../iceberg/connect/data/NoOpWriter.java | 2 +- .../connect/data/PartitionedAppendWriter.java | 4 +- .../iceberg/connect/data/RecordConverter.java | 517 ++++++++++ .../iceberg/connect/data/SchemaUpdate.java | 44 +- .../iceberg/connect/data/SchemaUtils.java | 14 +- .../iceberg/connect/data/Utilities.java | 20 +- .../connect/IcebergSinkConnectorTest.java | 4 +- .../connect/data/RecordConverterTest.java | 936 ++++++++++++++++++ 12 files changed, 1526 insertions(+), 65 deletions(-) create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java diff --git a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java index d1400f58f74c..50eaa1050485 100644 --- a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java +++ b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.connect.events; -import static java.util.stream.Collectors.toList; - import java.util.Arrays; import java.util.List; +import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; import org.apache.avro.util.Utf8; @@ -96,7 +95,9 @@ public void put(int i, Object v) { return; case NAMESPACE: this.namespace = - v == null ? null : ((List) v).stream().map(Utf8::toString).collect(toList()); + v == null + ? null + : ((List) v).stream().map(Utf8::toString).collect(Collectors.toList()); return; case NAME: this.name = v == null ? null : v.toString(); 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 aa1ecdd5d1ba..d1572fbff37b 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 @@ -18,8 +18,6 @@ */ package org.apache.iceberg.connect; -import static java.util.stream.Collectors.toList; - import java.io.InputStream; import java.nio.file.Files; import java.nio.file.Paths; @@ -28,6 +26,7 @@ import java.util.Map; import java.util.Properties; import java.util.regex.Pattern; +import java.util.stream.Collectors; import org.apache.iceberg.IcebergBuild; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -72,7 +71,6 @@ public class IcebergSinkConfig extends AbstractConfig { private static final String TABLES_DEFAULT_COMMIT_BRANCH = "iceberg.tables.default-commit-branch"; private static final String TABLES_DEFAULT_ID_COLUMNS = "iceberg.tables.default-id-columns"; private static final String TABLES_DEFAULT_PARTITION_BY = "iceberg.tables.default-partition-by"; - // FIXME: add config for CDC and upsert mode private static final String TABLES_AUTO_CREATE_ENABLED_PROP = "iceberg.tables.auto-create-enabled"; private static final String TABLES_EVOLVE_SCHEMA_ENABLED_PROP = @@ -365,7 +363,7 @@ static List stringToList(String value, String regex) { return ImmutableList.of(); } - return Arrays.stream(value.split(regex)).map(String::trim).collect(toList()); + return Arrays.stream(value.split(regex)).map(String::trim).collect(Collectors.toList()); } public String controlTopic() { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java index 485b209302d5..8be8518f4407 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java @@ -18,11 +18,10 @@ */ package org.apache.iceberg.connect; -import static java.util.stream.Collectors.toList; - import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.kafka.common.config.ConfigDef; @@ -60,7 +59,7 @@ public List> taskConfigs(int maxTasks) { map.put(IcebergSinkConfig.INTERNAL_TRANSACTIONAL_SUFFIX_PROP, txnSuffix + i); return map; }) - .collect(toList()); + .collect(Collectors.toList()); } @Override diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java index da88b3b50ffe..27ffc4de9973 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java @@ -38,8 +38,7 @@ public class IcebergWriter implements RecordWriter { private final IcebergSinkConfig config; private final List writerResults; - // FIXME: update this when the record converter is added - // private RecordConverter recordConverter; + private RecordConverter recordConverter; private TaskWriter writer; public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { @@ -52,19 +51,15 @@ public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { private void initNewWriter() { this.writer = Utilities.createTableWriter(table, tableName, config); - // FIXME: update this when the record converter is added - // this.recordConverter = new RecordConverter(table, config); + this.recordConverter = new RecordConverter(table, config); } @Override public void write(SinkRecord record) { try { - // TODO: config to handle tombstones instead of always ignoring? + // ignore tombstones... if (record.value() != null) { Record row = convertToRow(record); - - // FIXME: add CDC operation support - writer.write(row); } } catch (Exception e) { @@ -77,8 +72,25 @@ public void write(SinkRecord record) { } private Record convertToRow(SinkRecord record) { - // FIXME: update this when the record converter is added - return null; + if (!config.evolveSchemaEnabled()) { + return recordConverter.convert(record.value()); + } + + SchemaUpdate.Consumer updates = new SchemaUpdate.Consumer(); + Record row = recordConverter.convert(record.value(), updates); + + if (!updates.empty()) { + // complete the current file + flush(); + // apply the schema updates, this will refresh the table + SchemaUtils.applySchemaUpdates(table, updates); + // initialize a new writer with the new schema + initNewWriter(); + // convert the row again, this time using the new table schema + row = recordConverter.convert(record.value(), null); + } + + return row; } private void flush() { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java index 31abe09cf1a4..64ca44f03209 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java @@ -21,7 +21,7 @@ import java.util.List; import org.apache.kafka.connect.sink.SinkRecord; -public class NoOpWriter implements RecordWriter { +class NoOpWriter implements RecordWriter { @Override public void write(SinkRecord record) { // NO-OP diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/PartitionedAppendWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/PartitionedAppendWriter.java index 1d429e44e675..ad8b5715a99b 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/PartitionedAppendWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/PartitionedAppendWriter.java @@ -29,12 +29,12 @@ import org.apache.iceberg.io.OutputFileFactory; import org.apache.iceberg.io.PartitionedFanoutWriter; -public class PartitionedAppendWriter extends PartitionedFanoutWriter { +class PartitionedAppendWriter extends PartitionedFanoutWriter { private final PartitionKey partitionKey; private final InternalRecordWrapper wrapper; - public PartitionedAppendWriter( + PartitionedAppendWriter( PartitionSpec spec, FileFormat format, FileAppenderFactory appenderFactory, 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 new file mode 100644 index 000000000000..406a2cba4526 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordConverter.java @@ -0,0 +1,517 @@ +/* + * 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.connect.data; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.math.BigDecimal; +import java.math.RoundingMode; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.format.DateTimeFormatter; +import java.time.format.DateTimeFormatterBuilder; +import java.time.format.DateTimeParseException; +import java.time.temporal.Temporal; +import java.util.Base64; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.MapType; +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.DateTimeUtil; +import org.apache.kafka.connect.data.Struct; + +class RecordConverter { + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final DateTimeFormatter OFFSET_TIMESTAMP_FORMAT = + new DateTimeFormatterBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HHmm", "Z") + .toFormatter(); + + private final Schema tableSchema; + private final NameMapping nameMapping; + private final IcebergSinkConfig config; + private final Map> structNameMap = Maps.newHashMap(); + + RecordConverter(Table table, IcebergSinkConfig config) { + this.tableSchema = table.schema(); + this.nameMapping = createNameMapping(table); + this.config = config; + } + + Record convert(Object data) { + return convert(data, null); + } + + Record convert(Object data, SchemaUpdate.Consumer schemaUpdateConsumer) { + if (data instanceof Struct || data instanceof Map) { + return convertStructValue(data, tableSchema.asStruct(), -1, schemaUpdateConsumer); + } + throw new UnsupportedOperationException("Cannot convert type: " + data.getClass().getName()); + } + + private NameMapping createNameMapping(Table table) { + String nameMappingString = table.properties().get(TableProperties.DEFAULT_NAME_MAPPING); + return nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; + } + + private Object convertValue( + Object value, Type type, int fieldId, SchemaUpdate.Consumer schemaUpdateConsumer) { + if (value == null) { + return null; + } + switch (type.typeId()) { + case STRUCT: + return convertStructValue(value, type.asStructType(), fieldId, schemaUpdateConsumer); + case LIST: + return convertListValue(value, type.asListType(), schemaUpdateConsumer); + case MAP: + return convertMapValue(value, type.asMapType(), schemaUpdateConsumer); + case INTEGER: + return convertInt(value); + case LONG: + return convertLong(value); + case FLOAT: + return convertFloat(value); + case DOUBLE: + return convertDouble(value); + case DECIMAL: + return convertDecimal(value, (DecimalType) type); + case BOOLEAN: + return convertBoolean(value); + case STRING: + return convertString(value); + case UUID: + return convertUUID(value); + case BINARY: + case FIXED: + return convertBase64Binary(value); + case DATE: + return convertDateValue(value); + case TIME: + return convertTimeValue(value); + case TIMESTAMP: + return convertTimestampValue(value, (TimestampType) type); + } + throw new UnsupportedOperationException("Unsupported type: " + type.typeId()); + } + + protected GenericRecord convertStructValue( + Object value, + StructType schema, + int parentFieldId, + SchemaUpdate.Consumer schemaUpdateConsumer) { + if (value instanceof Map) { + return convertToStruct((Map) value, schema, parentFieldId, schemaUpdateConsumer); + } else if (value instanceof Struct) { + return convertToStruct((Struct) value, schema, parentFieldId, schemaUpdateConsumer); + } + throw new IllegalArgumentException("Cannot convert to struct: " + value.getClass().getName()); + } + + /** + * This method will be called for records when there is no record schema. Also, when there is no + * schema, we infer that map values are struct types. This method might also be called if the + * field value is a map but the Iceberg type is a struct. This can happen if the Iceberg table + * schema is not managed by the sink, i.e. created manually. + */ + private GenericRecord convertToStruct( + Map map, + StructType schema, + int structFieldId, + SchemaUpdate.Consumer schemaUpdateConsumer) { + GenericRecord result = GenericRecord.create(schema); + map.forEach( + (recordFieldNameObj, recordFieldValue) -> { + String recordFieldName = recordFieldNameObj.toString(); + NestedField tableField = lookupStructField(recordFieldName, schema, structFieldId); + if (tableField == null) { + // add the column if schema evolution is on, otherwise skip the value, + // skip the add column if we can't infer the type + if (schemaUpdateConsumer != null) { + Type type = SchemaUtils.inferIcebergType(recordFieldValue, config); + if (type != null) { + String parentFieldName = + structFieldId < 0 ? null : tableSchema.findColumnName(structFieldId); + schemaUpdateConsumer.addColumn(parentFieldName, recordFieldName, type); + } + } + } else { + result.setField( + tableField.name(), + convertValue( + recordFieldValue, + tableField.type(), + tableField.fieldId(), + schemaUpdateConsumer)); + } + }); + return result; + } + + /** This method will be called for records and struct values when there is a record schema. */ + private GenericRecord convertToStruct( + Struct struct, + StructType schema, + int structFieldId, + SchemaUpdate.Consumer schemaUpdateConsumer) { + GenericRecord result = GenericRecord.create(schema); + struct + .schema() + .fields() + .forEach( + recordField -> { + NestedField tableField = lookupStructField(recordField.name(), schema, structFieldId); + if (tableField == null) { + // add the column if schema evolution is on, otherwise skip the value + if (schemaUpdateConsumer != null) { + String parentFieldName = + structFieldId < 0 ? null : tableSchema.findColumnName(structFieldId); + Type type = SchemaUtils.toIcebergType(recordField.schema(), config); + schemaUpdateConsumer.addColumn(parentFieldName, recordField.name(), type); + } + } else { + boolean hasSchemaUpdates = false; + if (schemaUpdateConsumer != null) { + // update the type if needed and schema evolution is on + PrimitiveType evolveDataType = + SchemaUtils.needsDataTypeUpdate(tableField.type(), recordField.schema()); + if (evolveDataType != null) { + String fieldName = tableSchema.findColumnName(tableField.fieldId()); + schemaUpdateConsumer.updateType(fieldName, evolveDataType); + hasSchemaUpdates = true; + } + // make optional if needed and schema evolution is on + if (tableField.isRequired() && recordField.schema().isOptional()) { + String fieldName = tableSchema.findColumnName(tableField.fieldId()); + schemaUpdateConsumer.makeOptional(fieldName); + hasSchemaUpdates = true; + } + } + if (!hasSchemaUpdates) { + result.setField( + tableField.name(), + convertValue( + struct.get(recordField), + tableField.type(), + tableField.fieldId(), + schemaUpdateConsumer)); + } + } + }); + return result; + } + + private NestedField lookupStructField(String fieldName, StructType schema, int structFieldId) { + if (nameMapping == null) { + return config.schemaCaseInsensitive() + ? schema.caseInsensitiveField(fieldName) + : schema.field(fieldName); + } + + return structNameMap + .computeIfAbsent(structFieldId, notUsed -> createStructNameMap(schema)) + .get(fieldName); + } + + private Map createStructNameMap(StructType schema) { + Map map = Maps.newHashMap(); + schema + .fields() + .forEach( + col -> { + MappedField mappedField = nameMapping.find(col.fieldId()); + if (mappedField != null && !mappedField.names().isEmpty()) { + mappedField.names().forEach(name -> map.put(name, col)); + } else { + map.put(col.name(), col); + } + }); + return map; + } + + protected List convertListValue( + Object value, ListType type, SchemaUpdate.Consumer schemaUpdateConsumer) { + Preconditions.checkArgument(value instanceof List); + List list = (List) value; + return list.stream() + .map( + element -> { + int fieldId = type.fields().get(0).fieldId(); + return convertValue(element, type.elementType(), fieldId, schemaUpdateConsumer); + }) + .collect(Collectors.toList()); + } + + protected Map convertMapValue( + Object value, MapType type, SchemaUpdate.Consumer schemaUpdateConsumer) { + Preconditions.checkArgument(value instanceof Map); + Map map = (Map) value; + Map result = Maps.newHashMap(); + map.forEach( + (k, v) -> { + int keyFieldId = type.fields().get(0).fieldId(); + int valueFieldId = type.fields().get(1).fieldId(); + result.put( + convertValue(k, type.keyType(), keyFieldId, schemaUpdateConsumer), + convertValue(v, type.valueType(), valueFieldId, schemaUpdateConsumer)); + }); + return result; + } + + protected int convertInt(Object value) { + if (value instanceof Number) { + return ((Number) value).intValue(); + } else if (value instanceof String) { + return Integer.parseInt((String) value); + } + throw new IllegalArgumentException("Cannot convert to int: " + value.getClass().getName()); + } + + protected long convertLong(Object value) { + if (value instanceof Number) { + return ((Number) value).longValue(); + } else if (value instanceof String) { + return Long.parseLong((String) value); + } + throw new IllegalArgumentException("Cannot convert to long: " + value.getClass().getName()); + } + + protected float convertFloat(Object value) { + if (value instanceof Number) { + return ((Number) value).floatValue(); + } else if (value instanceof String) { + return Float.parseFloat((String) value); + } + throw new IllegalArgumentException("Cannot convert to float: " + value.getClass().getName()); + } + + protected double convertDouble(Object value) { + if (value instanceof Number) { + return ((Number) value).doubleValue(); + } else if (value instanceof String) { + return Double.parseDouble((String) value); + } + throw new IllegalArgumentException("Cannot convert to double: " + value.getClass().getName()); + } + + protected BigDecimal convertDecimal(Object value, DecimalType type) { + BigDecimal bigDecimal; + if (value instanceof BigDecimal) { + bigDecimal = (BigDecimal) value; + } else if (value instanceof Number) { + Number num = (Number) value; + Double dbl = num.doubleValue(); + if (dbl.equals(Math.floor(dbl))) { + bigDecimal = BigDecimal.valueOf(num.longValue()); + } else { + bigDecimal = BigDecimal.valueOf(dbl); + } + } else if (value instanceof String) { + bigDecimal = new BigDecimal((String) value); + } else { + throw new IllegalArgumentException( + "Cannot convert to BigDecimal: " + value.getClass().getName()); + } + return bigDecimal.setScale(type.scale(), RoundingMode.HALF_UP); + } + + protected boolean convertBoolean(Object value) { + if (value instanceof Boolean) { + return (boolean) value; + } else if (value instanceof String) { + return Boolean.parseBoolean((String) value); + } + throw new IllegalArgumentException("Cannot convert to boolean: " + value.getClass().getName()); + } + + protected String convertString(Object value) { + try { + if (value instanceof String) { + return (String) value; + } else if (value instanceof Number || value instanceof Boolean) { + return value.toString(); + } else if (value instanceof Map || value instanceof List) { + return MAPPER.writeValueAsString(value); + } else if (value instanceof Struct) { + Struct struct = (Struct) value; + byte[] data = config.jsonConverter().fromConnectData(null, struct.schema(), struct); + return new String(data, StandardCharsets.UTF_8); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + throw new IllegalArgumentException("Cannot convert to string: " + value.getClass().getName()); + } + + protected UUID convertUUID(Object value) { + if (value instanceof String) { + return UUID.fromString((String) value); + } else if (value instanceof UUID) { + return (UUID) value; + } + throw new IllegalArgumentException("Cannot convert to UUID: " + value.getClass().getName()); + } + + protected ByteBuffer convertBase64Binary(Object value) { + if (value instanceof String) { + return ByteBuffer.wrap(Base64.getDecoder().decode((String) value)); + } else if (value instanceof byte[]) { + return ByteBuffer.wrap((byte[]) value); + } else if (value instanceof ByteBuffer) { + return (ByteBuffer) value; + } + throw new IllegalArgumentException("Cannot convert to binary: " + value.getClass().getName()); + } + + @SuppressWarnings("JavaUtilDate") + protected LocalDate convertDateValue(Object value) { + if (value instanceof Number) { + int days = ((Number) value).intValue(); + return DateTimeUtil.dateFromDays(days); + } else if (value instanceof String) { + return LocalDate.parse((String) value); + } else if (value instanceof LocalDate) { + return (LocalDate) value; + } else if (value instanceof Date) { + int days = (int) (((Date) value).getTime() / 1000 / 60 / 60 / 24); + return DateTimeUtil.dateFromDays(days); + } + throw new RuntimeException("Cannot convert date: " + value); + } + + @SuppressWarnings("JavaUtilDate") + protected LocalTime convertTimeValue(Object value) { + if (value instanceof Number) { + long millis = ((Number) value).longValue(); + return DateTimeUtil.timeFromMicros(millis * 1000); + } else if (value instanceof String) { + return LocalTime.parse((String) value); + } else if (value instanceof LocalTime) { + return (LocalTime) value; + } else if (value instanceof Date) { + long millis = ((Date) value).getTime(); + return DateTimeUtil.timeFromMicros(millis * 1000); + } + throw new RuntimeException("Cannot convert time: " + value); + } + + protected Temporal convertTimestampValue(Object value, TimestampType type) { + if (type.shouldAdjustToUTC()) { + return convertOffsetDateTime(value); + } + return convertLocalDateTime(value); + } + + @SuppressWarnings("JavaUtilDate") + private OffsetDateTime convertOffsetDateTime(Object value) { + if (value instanceof Number) { + long millis = ((Number) value).longValue(); + return DateTimeUtil.timestamptzFromMicros(millis * 1000); + } else if (value instanceof String) { + return parseOffsetDateTime((String) value); + } else if (value instanceof OffsetDateTime) { + return (OffsetDateTime) value; + } else if (value instanceof LocalDateTime) { + return ((LocalDateTime) value).atOffset(ZoneOffset.UTC); + } else if (value instanceof Date) { + return DateTimeUtil.timestamptzFromMicros(((Date) value).getTime() * 1000); + } + throw new RuntimeException( + "Cannot convert timestamptz: " + value + ", type: " + value.getClass()); + } + + private OffsetDateTime parseOffsetDateTime(String str) { + String tsStr = ensureTimestampFormat(str); + try { + return OFFSET_TIMESTAMP_FORMAT.parse(tsStr, OffsetDateTime::from); + } catch (DateTimeParseException e) { + return LocalDateTime.parse(tsStr, DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .atOffset(ZoneOffset.UTC); + } + } + + @SuppressWarnings("JavaUtilDate") + private LocalDateTime convertLocalDateTime(Object value) { + if (value instanceof Number) { + long millis = ((Number) value).longValue(); + return DateTimeUtil.timestampFromMicros(millis * 1000); + } else if (value instanceof String) { + return parseLocalDateTime((String) value); + } else if (value instanceof LocalDateTime) { + return (LocalDateTime) value; + } else if (value instanceof OffsetDateTime) { + return ((OffsetDateTime) value).toLocalDateTime(); + } else if (value instanceof Date) { + return DateTimeUtil.timestampFromMicros(((Date) value).getTime() * 1000); + } + throw new RuntimeException( + "Cannot convert timestamp: " + value + ", type: " + value.getClass()); + } + + private LocalDateTime parseLocalDateTime(String str) { + String tsStr = ensureTimestampFormat(str); + try { + return LocalDateTime.parse(tsStr, DateTimeFormatter.ISO_LOCAL_DATE_TIME); + } catch (DateTimeParseException e) { + return OFFSET_TIMESTAMP_FORMAT.parse(tsStr, OffsetDateTime::from).toLocalDateTime(); + } + } + + private String ensureTimestampFormat(String str) { + String result = str; + if (result.charAt(10) == ' ') { + result = result.substring(0, 10) + 'T' + result.substring(11); + } + if (result.length() > 22 + && (result.charAt(19) == '+' || result.charAt(19) == '-') + && result.charAt(22) == ':') { + result = result.substring(0, 19) + result.substring(19).replace(":", ""); + } + return result; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUpdate.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUpdate.java index 2bb0e65f204b..809bea84dcc2 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUpdate.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUpdate.java @@ -24,97 +24,97 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Type.PrimitiveType; -public class SchemaUpdate { +class SchemaUpdate { - public static class Consumer { + static class Consumer { private final Map addColumns = Maps.newHashMap(); private final Map updateTypes = Maps.newHashMap(); private final Map makeOptionals = Maps.newHashMap(); - public Collection addColumns() { + Collection addColumns() { return addColumns.values(); } - public Collection updateTypes() { + Collection updateTypes() { return updateTypes.values(); } - public Collection makeOptionals() { + Collection makeOptionals() { return makeOptionals.values(); } - public boolean empty() { + boolean empty() { return addColumns.isEmpty() && updateTypes.isEmpty() && makeOptionals.isEmpty(); } - public void addColumn(String parentName, String name, Type type) { + void addColumn(String parentName, String name, Type type) { AddColumn addCol = new AddColumn(parentName, name, type); addColumns.put(addCol.key(), addCol); } - public void updateType(String name, PrimitiveType type) { + void updateType(String name, PrimitiveType type) { updateTypes.put(name, new UpdateType(name, type)); } - public void makeOptional(String name) { + void makeOptional(String name) { makeOptionals.put(name, new MakeOptional(name)); } } - public static class AddColumn extends SchemaUpdate { + static class AddColumn extends SchemaUpdate { private final String parentName; private final String name; private final Type type; - public AddColumn(String parentName, String name, Type type) { + AddColumn(String parentName, String name, Type type) { this.parentName = parentName; this.name = name; this.type = type; } - public String parentName() { + String parentName() { return parentName; } - public String name() { + String name() { return name; } - public String key() { + String key() { return parentName == null ? name : parentName + "." + name; } - public Type type() { + Type type() { return type; } } - public static class UpdateType extends SchemaUpdate { + static class UpdateType extends SchemaUpdate { private final String name; private final PrimitiveType type; - public UpdateType(String name, PrimitiveType type) { + UpdateType(String name, PrimitiveType type) { this.name = name; this.type = type; } - public String name() { + String name() { return name; } - public PrimitiveType type() { + PrimitiveType type() { return type; } } - public static class MakeOptional extends SchemaUpdate { + static class MakeOptional extends SchemaUpdate { private final String name; - public MakeOptional(String name) { + MakeOptional(String name) { this.name = name; } - public String name() { + String name() { return name; } } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java index 64fa89041c29..a2e0729fd506 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SchemaUtils.java @@ -65,13 +65,13 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class SchemaUtils { +class SchemaUtils { private static final Logger LOG = LoggerFactory.getLogger(SchemaUtils.class); private static final Pattern TRANSFORM_REGEX = Pattern.compile("(\\w+)\\((.+)\\)"); - public static PrimitiveType needsDataTypeUpdate(Type currentIcebergType, Schema valueSchema) { + static PrimitiveType needsDataTypeUpdate(Type currentIcebergType, Schema valueSchema) { if (currentIcebergType.typeId() == TypeID.FLOAT && valueSchema.type() == Schema.Type.FLOAT64) { return DoubleType.get(); } @@ -81,7 +81,7 @@ public static PrimitiveType needsDataTypeUpdate(Type currentIcebergType, Schema return null; } - public static void applySchemaUpdates(Table table, SchemaUpdate.Consumer updates) { + static void applySchemaUpdates(Table table, SchemaUpdate.Consumer updates) { if (updates == null || updates.empty()) { // no updates to apply return; @@ -150,7 +150,7 @@ private static boolean isOptional(org.apache.iceberg.Schema schema, MakeOptional return field.isOptional(); } - public static PartitionSpec createPartitionSpec( + static PartitionSpec createPartitionSpec( org.apache.iceberg.Schema schema, List partitionBy) { if (partitionBy.isEmpty()) { return PartitionSpec.unpartitioned(); @@ -209,11 +209,11 @@ private static Pair transformArgPair(String argsStr) { return Pair.of(parts.get(0).trim(), Integer.parseInt(parts.get(1).trim())); } - public static Type toIcebergType(Schema valueSchema, IcebergSinkConfig config) { + static Type toIcebergType(Schema valueSchema, IcebergSinkConfig config) { return new SchemaGenerator(config).toIcebergType(valueSchema); } - public static Type inferIcebergType(Object value, IcebergSinkConfig config) { + static Type inferIcebergType(Object value, IcebergSinkConfig config) { return new SchemaGenerator(config).inferIcebergType(value); } @@ -290,7 +290,7 @@ Type toIcebergType(Schema valueSchema) { } @SuppressWarnings("checkstyle:CyclomaticComplexity") - public Type inferIcebergType(Object value) { + Type inferIcebergType(Object value) { if (value == null) { return null; } else if (value instanceof String) { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java index ec13b003a21a..4ff83f777527 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java @@ -18,12 +18,6 @@ */ package org.apache.iceberg.connect.data; -import static java.util.stream.Collectors.toSet; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; -import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES; -import static org.apache.iceberg.TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT; - import java.io.IOException; import java.net.URL; import java.nio.file.Files; @@ -33,9 +27,11 @@ import java.util.Map; import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynConstructors; @@ -175,12 +171,16 @@ public static TaskWriter createTableWriter( Map tableProps = Maps.newHashMap(table.properties()); tableProps.putAll(config.writeProps()); - String formatStr = tableProps.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + String formatStr = + tableProps.getOrDefault( + TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); FileFormat format = FileFormat.fromString(formatStr); long targetFileSize = PropertyUtil.propertyAsLong( - tableProps, WRITE_TARGET_FILE_SIZE_BYTES, WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); + tableProps, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT); Set identifierFieldIds = table.schema().identifierFieldIds(); @@ -197,7 +197,7 @@ public static TaskWriter createTableWriter( } return field.fieldId(); }) - .collect(toSet()); + .collect(Collectors.toSet()); } FileAppenderFactory appenderFactory; @@ -224,8 +224,6 @@ public static TaskWriter createTableWriter( .format(format) .build(); - // FIXME: add delta writers - TaskWriter writer; if (table.spec().isUnpartitioned()) { writer = diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConnectorTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConnectorTest.java index 86502794b224..c8f563a13911 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConnectorTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/IcebergSinkConnectorTest.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.connect; -import static org.apache.iceberg.connect.IcebergSinkConfig.INTERNAL_TRANSACTIONAL_SUFFIX_PROP; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -35,6 +34,7 @@ public void testTaskConfigs() { connector.start(ImmutableMap.of()); List> configs = connector.taskConfigs(3); assertThat(configs).hasSize(3); - configs.forEach(map -> assertThat(map).containsKey(INTERNAL_TRANSACTIONAL_SUFFIX_PROP)); + configs.forEach( + map -> assertThat(map).containsKey(IcebergSinkConfig.INTERNAL_TRANSACTIONAL_SUFFIX_PROP)); } } 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 new file mode 100644 index 000000000000..b494a9da85d3 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordConverterTest.java @@ -0,0 +1,936 @@ +/* + * 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.connect.data; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.Temporal; +import java.util.Base64; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.SchemaUpdate.AddColumn; +import org.apache.iceberg.connect.data.SchemaUpdate.UpdateType; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.mapping.MappedField; +import org.apache.iceberg.mapping.NameMapping; +import org.apache.iceberg.mapping.NameMappingParser; +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.types.Type; +import org.apache.iceberg.types.Types.BinaryType; +import org.apache.iceberg.types.Types.BooleanType; +import org.apache.iceberg.types.Types.DateType; +import org.apache.iceberg.types.Types.DecimalType; +import org.apache.iceberg.types.Types.DoubleType; +import org.apache.iceberg.types.Types.FixedType; +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.NestedField; +import org.apache.iceberg.types.Types.StringType; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampType; +import org.apache.iceberg.types.Types.UUIDType; +import org.apache.kafka.connect.data.Decimal; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Time; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.json.JsonConverterConfig; +import org.apache.kafka.connect.storage.ConverterConfig; +import org.apache.kafka.connect.storage.ConverterType; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class RecordConverterTest { + + private static final ObjectMapper MAPPER = new ObjectMapper(); + + private static final org.apache.iceberg.Schema SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required(20, "i", IntegerType.get()), + NestedField.required(21, "l", LongType.get()), + NestedField.required(22, "d", DateType.get()), + NestedField.required(23, "t", TimeType.get()), + NestedField.required(24, "ts", TimestampType.withoutZone()), + NestedField.required(25, "tsz", TimestampType.withZone()), + NestedField.required(26, "fl", FloatType.get()), + NestedField.required(27, "do", DoubleType.get()), + NestedField.required(28, "dec", DecimalType.of(9, 2)), + NestedField.required(29, "s", StringType.get()), + NestedField.required(30, "b", BooleanType.get()), + NestedField.required(31, "u", UUIDType.get()), + NestedField.required(32, "f", FixedType.ofLength(3)), + NestedField.required(33, "bi", BinaryType.get()), + NestedField.required(34, "li", ListType.ofRequired(35, StringType.get())), + NestedField.required( + 36, "ma", MapType.ofRequired(37, 38, StringType.get(), StringType.get())), + NestedField.optional(39, "extra", StringType.get())); + + // we have 1 unmapped column so exclude that from the count + private static final int MAPPED_CNT = SCHEMA.columns().size() - 1; + + private static final org.apache.iceberg.Schema NESTED_SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required(1, "ii", IntegerType.get()), + NestedField.required(2, "st", SCHEMA.asStruct())); + + private static final org.apache.iceberg.Schema SIMPLE_SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required(1, "ii", IntegerType.get()), + NestedField.required(2, "st", StringType.get())); + + private static final org.apache.iceberg.Schema ID_SCHEMA = + new org.apache.iceberg.Schema(NestedField.required(1, "ii", IntegerType.get())); + + private static final org.apache.iceberg.Schema STRUCT_IN_LIST_SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required(100, "stli", ListType.ofRequired(101, NESTED_SCHEMA.asStruct()))); + + private static final org.apache.iceberg.Schema STRUCT_IN_LIST_BASIC_SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required(100, "stli", ListType.ofRequired(101, ID_SCHEMA.asStruct()))); + + private static final org.apache.iceberg.Schema STRUCT_IN_MAP_SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required( + 100, + "stma", + MapType.ofRequired(101, 102, StringType.get(), NESTED_SCHEMA.asStruct()))); + + private static final org.apache.iceberg.Schema STRUCT_IN_MAP_BASIC_SCHEMA = + new org.apache.iceberg.Schema( + NestedField.required( + 100, "stma", MapType.ofRequired(101, 102, StringType.get(), ID_SCHEMA.asStruct()))); + + private static final Schema CONNECT_SCHEMA = + SchemaBuilder.struct() + .field("i", Schema.INT32_SCHEMA) + .field("l", Schema.INT64_SCHEMA) + .field("d", org.apache.kafka.connect.data.Date.SCHEMA) + .field("t", Time.SCHEMA) + .field("ts", Timestamp.SCHEMA) + .field("tsz", Timestamp.SCHEMA) + .field("fl", Schema.FLOAT32_SCHEMA) + .field("do", Schema.FLOAT64_SCHEMA) + .field("dec", Decimal.schema(2)) + .field("s", Schema.STRING_SCHEMA) + .field("b", Schema.BOOLEAN_SCHEMA) + .field("u", Schema.STRING_SCHEMA) + .field("f", Schema.BYTES_SCHEMA) + .field("bi", Schema.BYTES_SCHEMA) + .field("li", SchemaBuilder.array(Schema.STRING_SCHEMA)) + .field("ma", SchemaBuilder.map(Schema.STRING_SCHEMA, Schema.STRING_SCHEMA)); + + private static final Schema CONNECT_NESTED_SCHEMA = + SchemaBuilder.struct().field("ii", Schema.INT32_SCHEMA).field("st", CONNECT_SCHEMA); + + private static final Schema CONNECT_STRUCT_IN_LIST_SCHEMA = + SchemaBuilder.struct().field("stli", SchemaBuilder.array(CONNECT_NESTED_SCHEMA)).build(); + + private static final Schema CONNECT_STRUCT_IN_MAP_SCHEMA = + SchemaBuilder.struct() + .field("stma", SchemaBuilder.map(Schema.STRING_SCHEMA, CONNECT_NESTED_SCHEMA)) + .build(); + + private static final LocalDate DATE_VAL = LocalDate.parse("2023-05-18"); + private static final LocalTime TIME_VAL = LocalTime.parse("07:14:21"); + private static final LocalDateTime TS_VAL = LocalDateTime.parse("2023-05-18T07:14:21"); + private static final OffsetDateTime TSZ_VAL = OffsetDateTime.parse("2023-05-18T07:14:21Z"); + private static final BigDecimal DEC_VAL = new BigDecimal("12.34"); + private static final String STR_VAL = "foobar"; + private static final UUID UUID_VAL = UUID.randomUUID(); + private static final ByteBuffer BYTES_VAL = ByteBuffer.wrap(new byte[] {1, 2, 3}); + private static final List LIST_VAL = ImmutableList.of("hello", "world"); + private static final Map MAP_VAL = ImmutableMap.of("one", "1", "two", "2"); + + private static final JsonConverter JSON_CONVERTER = new JsonConverter(); + + private IcebergSinkConfig config; + + @BeforeAll + public static void beforeAll() { + JSON_CONVERTER.configure( + ImmutableMap.of( + JsonConverterConfig.SCHEMAS_ENABLE_CONFIG, + false, + ConverterConfig.TYPE_CONFIG, + ConverterType.VALUE.getName())); + } + + @BeforeEach + public void before() { + this.config = mock(IcebergSinkConfig.class); + when(config.jsonConverter()).thenReturn(JSON_CONVERTER); + } + + @Test + public void testMapConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map data = createMapData(); + Record record = converter.convert(data); + assertRecordValues(record); + } + + @Test + public void testNestedMapConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(NESTED_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map nestedData = createNestedMapData(); + Record record = converter.convert(nestedData); + assertNestedRecordValues(record); + } + + @Test + @SuppressWarnings("unchecked") + public void testMapToString() throws Exception { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map nestedData = createNestedMapData(); + Record record = converter.convert(nestedData); + + String str = (String) record.getField("st"); + Map map = (Map) MAPPER.readValue(str, Map.class); + assertThat(map).hasSize(MAPPED_CNT); + } + + @Test + public void testMapValueInListConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_LIST_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map data = createNestedMapData(); + Record record = converter.convert(ImmutableMap.of("stli", ImmutableList.of(data, data))); + List fieldVal = (List) record.getField("stli"); + + Record elementVal = (Record) fieldVal.get(0); + assertNestedRecordValues(elementVal); + } + + @Test + public void testMapValueInMapConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_MAP_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map data = createNestedMapData(); + Record record = + converter.convert(ImmutableMap.of("stma", ImmutableMap.of("key1", data, "key2", data))); + + Map fieldVal = (Map) record.getField("stma"); + Record mapVal = (Record) fieldVal.get("key1"); + assertNestedRecordValues(mapVal); + } + + @Test + public void testStructConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct data = createStructData(); + Record record = converter.convert(data); + assertRecordValues(record); + } + + @Test + public void testNestedStructConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(NESTED_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct nestedData = createNestedStructData(); + Record record = converter.convert(nestedData); + assertNestedRecordValues(record); + } + + @Test + @SuppressWarnings("unchecked") + public void testStructToString() throws Exception { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct nestedData = createNestedStructData(); + Record record = converter.convert(nestedData); + + String str = (String) record.getField("st"); + Map map = (Map) MAPPER.readValue(str, Map.class); + assertThat(map).hasSize(MAPPED_CNT); + } + + @Test + public void testStructValueInListConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_LIST_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct data = createNestedStructData(); + Struct struct = + new Struct(CONNECT_STRUCT_IN_LIST_SCHEMA).put("stli", ImmutableList.of(data, data)); + Record record = converter.convert(struct); + + List fieldVal = (List) record.getField("stli"); + Record elementVal = (Record) fieldVal.get(0); + assertNestedRecordValues(elementVal); + } + + @Test + public void testStructValueInMapConvert() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_MAP_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct data = createNestedStructData(); + Struct struct = + new Struct(CONNECT_STRUCT_IN_MAP_SCHEMA) + .put("stma", ImmutableMap.of("key1", data, "key2", data)); + Record record = converter.convert(struct); + + Map fieldVal = (Map) record.getField("stma"); + Record mapVal = (Record) fieldVal.get("key1"); + assertNestedRecordValues(mapVal); + } + + @Test + public void testNameMapping() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + NameMapping nameMapping = NameMapping.of(MappedField.of(1, ImmutableList.of("renamed_ii"))); + when(table.properties()) + .thenReturn( + ImmutableMap.of( + TableProperties.DEFAULT_NAME_MAPPING, NameMappingParser.toJson(nameMapping))); + + RecordConverter converter = new RecordConverter(table, config); + + Map data = ImmutableMap.of("renamed_ii", 123); + Record record = converter.convert(data); + assertThat(record.getField("ii")).isEqualTo(123); + } + + @ParameterizedTest + @ValueSource(booleans = {false, true}) + public void testCaseSensitivity(boolean caseInsensitive) { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + when(config.schemaCaseInsensitive()).thenReturn(caseInsensitive); + + RecordConverter converter = new RecordConverter(table, config); + + Map mapData = ImmutableMap.of("II", 123); + Record record1 = converter.convert(mapData); + + Struct structData = + new Struct(SchemaBuilder.struct().field("II", Schema.INT32_SCHEMA).build()).put("II", 123); + Record record2 = converter.convert(structData); + + if (caseInsensitive) { + assertThat(record1.getField("ii")).isEqualTo(123); + assertThat(record2.getField("ii")).isEqualTo(123); + } else { + assertThat(record1.getField("ii")).isEqualTo(null); + assertThat(record2.getField("ii")).isEqualTo(null); + } + } + + @Test + public void testIntConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + RecordConverter converter = new RecordConverter(table, config); + + int expectedInt = 123; + + ImmutableList.of("123", 123.0f, 123.0d, 123L, expectedInt) + .forEach( + input -> { + int val = converter.convertInt(input); + assertThat(val).isEqualTo(expectedInt); + }); + } + + @Test + public void testLongConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + RecordConverter converter = new RecordConverter(table, config); + + long expectedLong = 123L; + + ImmutableList.of("123", 123.0f, 123.0d, 123, expectedLong) + .forEach( + input -> { + long val = converter.convertLong(input); + assertThat(val).isEqualTo(expectedLong); + }); + } + + @Test + public void testFloatConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + RecordConverter converter = new RecordConverter(table, config); + + float expectedFloat = 123f; + + ImmutableList.of("123", 123, 123L, 123d, expectedFloat) + .forEach( + input -> { + float val = converter.convertFloat(input); + assertThat(val).isEqualTo(expectedFloat); + }); + } + + @Test + public void testDoubleConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + RecordConverter converter = new RecordConverter(table, config); + + double expectedDouble = 123d; + + ImmutableList.of("123", 123, 123L, 123f, expectedDouble) + .forEach( + input -> { + double val = converter.convertDouble(input); + assertThat(val).isEqualTo(expectedDouble); + }); + } + + @Test + public void testDecimalConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + + RecordConverter converter = new RecordConverter(table, config); + + BigDecimal expected = new BigDecimal("123.45"); + + ImmutableList.of("123.45", 123.45d, expected) + .forEach( + input -> { + BigDecimal decimal = converter.convertDecimal(input, DecimalType.of(10, 2)); + assertThat(decimal).isEqualTo(expected); + }); + + BigDecimal expected2 = new BigDecimal(123); + + ImmutableList.of("123", 123, expected2) + .forEach( + input -> { + BigDecimal decimal = converter.convertDecimal(input, DecimalType.of(10, 0)); + assertThat(decimal).isEqualTo(expected2); + }); + } + + @Test + public void testDateConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + LocalDate expected = LocalDate.of(2023, 11, 15); + + List inputList = + ImmutableList.of( + "2023-11-15", + expected.toEpochDay(), + expected, + new Date(Duration.ofDays(expected.toEpochDay()).toMillis())); + + inputList.forEach( + input -> { + Temporal ts = converter.convertDateValue(input); + assertThat(ts).isEqualTo(expected); + }); + } + + @Test + public void testTimeConversion() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + LocalTime expected = LocalTime.of(7, 51, 30, 888_000_000); + + List inputList = + ImmutableList.of( + "07:51:30.888", + expected.toNanoOfDay() / 1000 / 1000, + expected, + new Date(expected.toNanoOfDay() / 1000 / 1000)); + + inputList.forEach( + input -> { + Temporal ts = converter.convertTimeValue(input); + assertThat(ts).isEqualTo(expected); + }); + } + + @Test + public void testTimestampWithZoneConversion() { + OffsetDateTime expected = OffsetDateTime.parse("2023-05-18T11:22:33Z"); + long expectedMillis = expected.toInstant().toEpochMilli(); + assertTimestampConvert(expected, expectedMillis, TimestampType.withZone()); + + // zone should be respected + expected = OffsetDateTime.parse("2023-05-18T03:22:33-08:00"); + List additionalInput = + ImmutableList.of( + "2023-05-18T03:22:33-08", + "2023-05-18 03:22:33-08", + "2023-05-18T03:22:33-08:00", + "2023-05-18 03:22:33-08:00", + "2023-05-18T03:22:33-0800", + "2023-05-18 03:22:33-0800"); + assertTimestampConvert(expected, additionalInput, TimestampType.withZone()); + } + + @Test + public void testTimestampWithoutZoneConversion() { + LocalDateTime expected = LocalDateTime.parse("2023-05-18T11:22:33"); + long expectedMillis = expected.atZone(ZoneOffset.UTC).toInstant().toEpochMilli(); + assertTimestampConvert(expected, expectedMillis, TimestampType.withoutZone()); + + // zone should be ignored + List additionalInput = + ImmutableList.of( + "2023-05-18T11:22:33-08", + "2023-05-18 11:22:33-08", + "2023-05-18T11:22:33-08:00", + "2023-05-18 11:22:33-08:00", + "2023-05-18T11:22:33-0800", + "2023-05-18 11:22:33-0800"); + assertTimestampConvert(expected, additionalInput, TimestampType.withoutZone()); + } + + private void assertTimestampConvert(Temporal expected, long expectedMillis, TimestampType type) { + List inputList = + Lists.newArrayList( + "2023-05-18T11:22:33Z", + "2023-05-18 11:22:33Z", + "2023-05-18T11:22:33+00", + "2023-05-18 11:22:33+00", + "2023-05-18T11:22:33+00:00", + "2023-05-18 11:22:33+00:00", + "2023-05-18T11:22:33+0000", + "2023-05-18 11:22:33+0000", + "2023-05-18T11:22:33", + "2023-05-18 11:22:33", + expectedMillis, + new Date(expectedMillis), + OffsetDateTime.ofInstant(Instant.ofEpochMilli(expectedMillis), ZoneOffset.UTC), + LocalDateTime.ofInstant(Instant.ofEpochMilli(expectedMillis), ZoneOffset.UTC)); + + assertTimestampConvert(expected, inputList, type); + } + + private void assertTimestampConvert( + Temporal expected, List inputList, TimestampType type) { + Table table = mock(Table.class); + when(table.schema()).thenReturn(SIMPLE_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + inputList.forEach( + input -> { + Temporal ts = converter.convertTimestampValue(input, type); + assertThat(ts).isEqualTo(expected); + }); + } + + @Test + public void testMissingColumnDetectionMap() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(ID_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map data = Maps.newHashMap(createMapData()); + data.put("null", null); + + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(data, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(MAPPED_CNT); + + Map newColMap = Maps.newHashMap(); + addCols.forEach(addCol -> newColMap.put(addCol.name(), addCol)); + + assertTypesAddedFromMap(col -> newColMap.get(col).type()); + + // null values should be ignored + assertThat(newColMap).doesNotContainKey("null"); + } + + @Test + public void testMissingColumnDetectionMapNested() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(ID_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map nestedData = createNestedMapData(); + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(nestedData, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(1); + + AddColumn addCol = addCols.iterator().next(); + assertThat(addCol.name()).isEqualTo("st"); + + StructType addedType = addCol.type().asStructType(); + assertThat(addedType.fields()).hasSize(MAPPED_CNT); + assertTypesAddedFromMap(col -> addedType.field(col).type()); + } + + @Test + public void testMissingColumnDetectionMapListValue() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_LIST_BASIC_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Map nestedData = createNestedMapData(); + Map map = ImmutableMap.of("stli", ImmutableList.of(nestedData, nestedData)); + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(map, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(1); + + AddColumn addCol = addCols.iterator().next(); + assertThat(addCol.parentName()).isEqualTo("stli.element"); + assertThat(addCol.name()).isEqualTo("st"); + + StructType nestedElementType = addCol.type().asStructType(); + assertThat(nestedElementType.fields()).hasSize(MAPPED_CNT); + assertTypesAddedFromMap(col -> nestedElementType.field(col).type()); + } + + private void assertTypesAddedFromMap(Function fn) { + assertThat(fn.apply("i")).isInstanceOf(LongType.class); + assertThat(fn.apply("l")).isInstanceOf(LongType.class); + assertThat(fn.apply("d")).isInstanceOf(StringType.class); + assertThat(fn.apply("t")).isInstanceOf(StringType.class); + assertThat(fn.apply("ts")).isInstanceOf(StringType.class); + assertThat(fn.apply("tsz")).isInstanceOf(StringType.class); + assertThat(fn.apply("fl")).isInstanceOf(DoubleType.class); + assertThat(fn.apply("do")).isInstanceOf(DoubleType.class); + assertThat(fn.apply("dec")).isInstanceOf(StringType.class); + assertThat(fn.apply("s")).isInstanceOf(StringType.class); + assertThat(fn.apply("b")).isInstanceOf(BooleanType.class); + assertThat(fn.apply("u")).isInstanceOf(StringType.class); + assertThat(fn.apply("f")).isInstanceOf(StringType.class); + assertThat(fn.apply("bi")).isInstanceOf(StringType.class); + assertThat(fn.apply("li")).isInstanceOf(ListType.class); + assertThat(fn.apply("ma")).isInstanceOf(StructType.class); + } + + @Test + public void testMissingColumnDetectionStruct() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(ID_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct data = createStructData(); + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(data, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(MAPPED_CNT); + + Map newColMap = Maps.newHashMap(); + addCols.forEach(addCol -> newColMap.put(addCol.name(), addCol)); + + assertTypesAddedFromStruct(col -> newColMap.get(col).type()); + } + + @Test + public void testMissingColumnDetectionStructNested() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(ID_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct nestedData = createNestedStructData(); + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(nestedData, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(1); + + AddColumn addCol = addCols.iterator().next(); + assertThat(addCol.name()).isEqualTo("st"); + + StructType addedType = addCol.type().asStructType(); + assertThat(addedType.fields()).hasSize(MAPPED_CNT); + assertTypesAddedFromStruct(col -> addedType.field(col).type()); + } + + @Test + public void testMissingColumnDetectionStructListValue() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_LIST_BASIC_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct nestedData = createNestedStructData(); + Struct struct = + new Struct(CONNECT_STRUCT_IN_LIST_SCHEMA) + .put("stli", ImmutableList.of(nestedData, nestedData)); + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(struct, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(1); + + AddColumn addCol = addCols.iterator().next(); + assertThat(addCol.parentName()).isEqualTo("stli.element"); + assertThat(addCol.name()).isEqualTo("st"); + + StructType nestedElementType = addCol.type().asStructType(); + assertThat(nestedElementType.fields()).hasSize(MAPPED_CNT); + assertTypesAddedFromStruct(col -> nestedElementType.field(col).type()); + } + + @Test + public void testMissingColumnDetectionStructMapValue() { + Table table = mock(Table.class); + when(table.schema()).thenReturn(STRUCT_IN_MAP_BASIC_SCHEMA); + RecordConverter converter = new RecordConverter(table, config); + + Struct nestedData = createNestedStructData(); + Struct struct = + new Struct(CONNECT_STRUCT_IN_MAP_SCHEMA) + .put("stma", ImmutableMap.of("key1", nestedData, "key2", nestedData)); + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(struct, consumer); + Collection addCols = consumer.addColumns(); + + assertThat(addCols).hasSize(1); + + AddColumn addCol = addCols.iterator().next(); + assertThat(addCol.parentName()).isEqualTo("stma.value"); + assertThat(addCol.name()).isEqualTo("st"); + + StructType nestedValueType = addCol.type().asStructType(); + assertThat(nestedValueType.fields()).hasSize(MAPPED_CNT); + assertTypesAddedFromStruct(col -> nestedValueType.field(col).type()); + } + + private void assertTypesAddedFromStruct(Function fn) { + assertThat(fn.apply("i")).isInstanceOf(IntegerType.class); + assertThat(fn.apply("l")).isInstanceOf(LongType.class); + assertThat(fn.apply("d")).isInstanceOf(DateType.class); + assertThat(fn.apply("t")).isInstanceOf(TimeType.class); + assertThat(fn.apply("ts")).isInstanceOf(TimestampType.class); + assertThat(fn.apply("tsz")).isInstanceOf(TimestampType.class); + assertThat(fn.apply("fl")).isInstanceOf(FloatType.class); + assertThat(fn.apply("do")).isInstanceOf(DoubleType.class); + assertThat(fn.apply("dec")).isInstanceOf(DecimalType.class); + assertThat(fn.apply("s")).isInstanceOf(StringType.class); + assertThat(fn.apply("b")).isInstanceOf(BooleanType.class); + assertThat(fn.apply("u")).isInstanceOf(StringType.class); + assertThat(fn.apply("f")).isInstanceOf(BinaryType.class); + assertThat(fn.apply("bi")).isInstanceOf(BinaryType.class); + assertThat(fn.apply("li")).isInstanceOf(ListType.class); + assertThat(fn.apply("ma")).isInstanceOf(MapType.class); + } + + @Test + public void testEvolveTypeDetectionStruct() { + org.apache.iceberg.Schema tableSchema = + new org.apache.iceberg.Schema( + NestedField.required(1, "ii", IntegerType.get()), + NestedField.required(2, "ff", FloatType.get())); + + Table table = mock(Table.class); + when(table.schema()).thenReturn(tableSchema); + RecordConverter converter = new RecordConverter(table, config); + + Schema valueSchema = + SchemaBuilder.struct().field("ii", Schema.INT64_SCHEMA).field("ff", Schema.FLOAT64_SCHEMA); + Struct data = new Struct(valueSchema).put("ii", 11L).put("ff", 22d); + + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(data, consumer); + Collection updates = consumer.updateTypes(); + + assertThat(updates).hasSize(2); + + Map updateMap = Maps.newHashMap(); + updates.forEach(update -> updateMap.put(update.name(), update)); + + assertThat(updateMap.get("ii").type()).isInstanceOf(LongType.class); + assertThat(updateMap.get("ff").type()).isInstanceOf(DoubleType.class); + } + + @Test + public void testEvolveTypeDetectionStructNested() { + org.apache.iceberg.Schema structColSchema = + new org.apache.iceberg.Schema( + NestedField.required(1, "ii", IntegerType.get()), + NestedField.required(2, "ff", FloatType.get())); + + org.apache.iceberg.Schema tableSchema = + new org.apache.iceberg.Schema( + NestedField.required(3, "i", IntegerType.get()), + NestedField.required(4, "st", structColSchema.asStruct())); + + Table table = mock(Table.class); + when(table.schema()).thenReturn(tableSchema); + RecordConverter converter = new RecordConverter(table, config); + + Schema structSchema = + SchemaBuilder.struct().field("ii", Schema.INT64_SCHEMA).field("ff", Schema.FLOAT64_SCHEMA); + Schema schema = + SchemaBuilder.struct().field("i", Schema.INT32_SCHEMA).field("st", structSchema); + Struct structValue = new Struct(structSchema).put("ii", 11L).put("ff", 22d); + Struct data = new Struct(schema).put("i", 1).put("st", structValue); + + SchemaUpdate.Consumer consumer = new SchemaUpdate.Consumer(); + converter.convert(data, consumer); + Collection updates = consumer.updateTypes(); + + assertThat(updates).hasSize(2); + + Map updateMap = Maps.newHashMap(); + updates.forEach(update -> updateMap.put(update.name(), update)); + + assertThat(updateMap.get("st.ii").type()).isInstanceOf(LongType.class); + assertThat(updateMap.get("st.ff").type()).isInstanceOf(DoubleType.class); + } + + private Map createMapData() { + return ImmutableMap.builder() + .put("i", 1) + .put("l", 2L) + .put("d", DATE_VAL.toString()) + .put("t", TIME_VAL.toString()) + .put("ts", TS_VAL.toString()) + .put("tsz", TSZ_VAL.toString()) + .put("fl", 1.1f) + .put("do", 2.2d) + .put("dec", DEC_VAL.toString()) + .put("s", STR_VAL) + .put("b", true) + .put("u", UUID_VAL.toString()) + .put("f", Base64.getEncoder().encodeToString(BYTES_VAL.array())) + .put("bi", Base64.getEncoder().encodeToString(BYTES_VAL.array())) + .put("li", LIST_VAL) + .put("ma", MAP_VAL) + .build(); + } + + private Map createNestedMapData() { + return ImmutableMap.builder().put("ii", 11).put("st", createMapData()).build(); + } + + private Struct createStructData() { + return new Struct(CONNECT_SCHEMA) + .put("i", 1) + .put("l", 2L) + .put("d", new Date(DATE_VAL.toEpochDay() * 24 * 60 * 60 * 1000L)) + .put("t", new Date(TIME_VAL.toNanoOfDay() / 1_000_000)) + .put("ts", Date.from(TS_VAL.atZone(ZoneOffset.UTC).toInstant())) + .put("tsz", Date.from(TSZ_VAL.toInstant())) + .put("fl", 1.1f) + .put("do", 2.2d) + .put("dec", DEC_VAL) + .put("s", STR_VAL) + .put("b", true) + .put("u", UUID_VAL.toString()) + .put("f", BYTES_VAL.array()) + .put("bi", BYTES_VAL.array()) + .put("li", LIST_VAL) + .put("ma", MAP_VAL); + } + + private Struct createNestedStructData() { + return new Struct(CONNECT_NESTED_SCHEMA).put("ii", 11).put("st", createStructData()); + } + + private void assertRecordValues(Record record) { + GenericRecord rec = (GenericRecord) record; + assertThat(rec.getField("i")).isEqualTo(1); + assertThat(rec.getField("l")).isEqualTo(2L); + assertThat(rec.getField("d")).isEqualTo(DATE_VAL); + assertThat(rec.getField("t")).isEqualTo(TIME_VAL); + assertThat(rec.getField("ts")).isEqualTo(TS_VAL); + assertThat(rec.getField("tsz")).isEqualTo(TSZ_VAL); + assertThat(rec.getField("fl")).isEqualTo(1.1f); + assertThat(rec.getField("do")).isEqualTo(2.2d); + 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("bi")).isEqualTo(BYTES_VAL); + assertThat(rec.getField("li")).isEqualTo(LIST_VAL); + assertThat(rec.getField("ma")).isEqualTo(MAP_VAL); + } + + private void assertNestedRecordValues(Record record) { + GenericRecord rec = (GenericRecord) record; + assertThat(rec.getField("ii")).isEqualTo(11); + assertRecordValues((GenericRecord) rec.getField("st")); + } +} From 57b43eeaed33bbe6c254d0418b25390e184051ab Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 22 Mar 2024 08:19:29 +0100 Subject: [PATCH 0155/1019] Core: Use as type parameter instead of raw type for SnapshotUpdate (#10015) --- .../test/java/org/apache/iceberg/TableTestBase.java | 10 +++++----- core/src/test/java/org/apache/iceberg/TestBase.java | 10 +++++----- .../src/test/java/org/apache/iceberg/TestRowDelta.java | 8 ++++---- 3 files changed, 14 insertions(+), 14 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java index c3db85910138..a92497f88e87 100644 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/TableTestBase.java @@ -375,24 +375,24 @@ void validateSnapshot(Snapshot old, Snapshot snap, long sequenceNumber, DataFile } @SuppressWarnings("checkstyle:HiddenField") - Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { + Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { Snapshot snapshot; if (branch.equals(SnapshotRef.MAIN_BRANCH)) { snapshotUpdate.commit(); snapshot = table.currentSnapshot(); } else { - ((SnapshotProducer) snapshotUpdate.toBranch(branch)).commit(); + ((SnapshotProducer) snapshotUpdate.toBranch(branch)).commit(); snapshot = table.snapshot(branch); } return snapshot; } - Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { + Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return ((SnapshotProducer) snapshotUpdate).apply(); + return ((SnapshotProducer) snapshotUpdate).apply(); } else { - return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); + return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index ebd527ded306..10aa57abf6f3 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -366,24 +366,24 @@ void validateSnapshot(Snapshot old, Snapshot snap, long sequenceNumber, DataFile } @SuppressWarnings("checkstyle:HiddenField") - Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { + Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { Snapshot snapshot; if (branch.equals(SnapshotRef.MAIN_BRANCH)) { snapshotUpdate.commit(); snapshot = table.currentSnapshot(); } else { - ((SnapshotProducer) snapshotUpdate.toBranch(branch)).commit(); + ((SnapshotProducer) snapshotUpdate.toBranch(branch)).commit(); snapshot = table.snapshot(branch); } return snapshot; } - Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { + Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return ((SnapshotProducer) snapshotUpdate).apply(); + return ((SnapshotProducer) snapshotUpdate).apply(); } else { - return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); + return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index fa04f36d367f..56bab52edaf9 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -62,7 +62,7 @@ public TestRowDelta(String branch) { @Test public void testAddDeleteFile() { - SnapshotUpdate rowDelta = + SnapshotUpdate rowDelta = table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES); commit(table, rowDelta, branch); @@ -97,7 +97,7 @@ public void testAddDeleteFile() { @Test public void testValidateDataFilesExistDefaults() { - SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); + SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); commit(table, rowDelta1, branch); @@ -105,12 +105,12 @@ public void testValidateDataFilesExistDefaults() { long validateFromSnapshotId = latestSnapshot(table, branch).snapshotId(); // overwrite FILE_A - SnapshotUpdate rowDelta2 = table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2); + SnapshotUpdate rowDelta2 = table.newOverwrite().deleteFile(FILE_A).addFile(FILE_A2); commit(table, rowDelta2, branch); // delete FILE_B - SnapshotUpdate rowDelta3 = table.newDelete().deleteFile(FILE_B); + SnapshotUpdate rowDelta3 = table.newDelete().deleteFile(FILE_B); commit(table, rowDelta3, branch); From c4e20a4a25460cb6b15bd8ce23e8a02f55334d07 Mon Sep 17 00:00:00 2001 From: "Brian \"bits\" Olsen" Date: Fri, 22 Mar 2024 02:23:36 -0500 Subject: [PATCH 0156/1019] Docs: Add local nightly build to test current docs changes (#9943) --- site/README.md | 13 ++++++++++++- site/dev/common.sh | 26 +++++++++++++++++++++++++- site/nav.yml | 1 + 3 files changed, 38 insertions(+), 2 deletions(-) diff --git a/site/README.md b/site/README.md index 63394635b463..6cb4f4907ebf 100644 --- a/site/README.md +++ b/site/README.md @@ -60,7 +60,12 @@ The static Iceberg site pages are Markdown files that live at `/site/docs/*.md`. ``` ### Building the versioned docs -The Iceberg versioned docs are committed in the [orphan `docs` branch](https://github.com/apache/iceberg/tree/docs) and mounted using [git worktree](https://git-scm.com/docs/git-worktree) at build time. The `docs` branch contains the versioned documenation source files at the root. These versions are mounted at the `/site/docs/docs/` directory at build time. The `latest` version, is a soft link to the most recent [semver version](https://semver.org/) in the `docs` branch. There is also an [orphan `javadoc` branch](https://github.com/apache/iceberg/tree/javadoc) that contains prior staticly generated versions of the javadocs mounted at `/site/docs/javadoc/` during build time. +The Iceberg versioned docs are committed in two [orphan](https://git-scm.com/docs/gitglossary#Documentation/gitglossary.txt-aiddeforphanaorphan) branches and mounted using [git worktree](https://git-scm.com/docs/git-worktree) at build time: + + 1. [`docs`](https://github.com/apache/iceberg/tree/docs) - contains the state of the documenation source files (`/docs`) during release. These versions are mounted at the `/site/docs/docs/` directory at build time. + 1. [`javadoc`](https://github.com/apache/iceberg/tree/javadoc) - contains prior statically generated versions of the javadocs mounted at `/site/docs/javadoc/` directory at build time. + +The `latest` version, is a soft link to the most recent [semver version](https://semver.org/) in the `docs` branch. The `nightly` version, is a soft link to the current local state of the `/docs` markdown files. The docs are built, run, and released using [make](https://www.gnu.org/software/make/manual/make.html). The [Makefile](Makefile) and the [common shell script](dev/common.sh) support the following command: @@ -84,11 +89,13 @@ This step will generate the staged source code which blends into the original so ./site/ └── docs    ├── docs +    │ ├── nightly (symlink to /docs/)    │ ├── latest (symlink to /site/docs/1.4.0/)    │ ├── 1.4.0    │ ├── 1.3.1    │   └── ... ├── javadoc +    │ ├── nightly (currently points to latest)    │ ├── latest    │ ├── 1.4.0    │ ├── 1.3.1 @@ -143,6 +150,9 @@ As mentioned in the MkDocs section, when you build MkDocs `mkdocs build`, MkDocs ./site/ ├── docs │   ├── docs +│   │   ├── nightly +│   │   │ ├── docs +│   │   │ └── mkdocs.yml │   │   ├── latest │   │   │ ├── docs │   │   │ └── mkdocs.yml @@ -150,6 +160,7 @@ As mentioned in the MkDocs section, when you build MkDocs `mkdocs build`, MkDocs │   │   ├── docs │   │ └── mkdocs.yml │   └─ javadoc +│   ├── nightly │   ├── latest │   └── 1.4.0 └── mkdocs.yml diff --git a/site/dev/common.sh b/site/dev/common.sh index 6eb0cf34a9c0..481628aa4dbd 100755 --- a/site/dev/common.sh +++ b/site/dev/common.sh @@ -85,6 +85,26 @@ assert_not_empty () { fi } +# Creates a 'nightly' version of the documentation that points to the current versioned docs +# located at the root-level `/docs` directory. +create_nightly () { + echo " --> create nightly" + + # Remove any existing 'nightly' directory and recreate it + rm -rf docs/docs/nightly/ + mkdir docs/docs/nightly/ + + # Create symbolic links and copy configuration files for the 'nightly' documentation + ln -s "../../../../docs/docs/" docs/docs/nightly/docs + cp "../docs/mkdocs.yml" docs/docs/nightly/ + + cd docs/docs/ + + # Update version information within the 'nightly' documentation + update_version "nightly" + cd - +} + # Finds and retrieves the latest version of the documentation based on the directory structure. # Assumes the documentation versions are numeric folders within 'docs/docs/'. get_latest_version () { @@ -189,6 +209,9 @@ pull_versioned_docs () { # Create the 'latest' version of documentation create_latest "${latest_version}" + + # Create the 'nightly' version of documentation + create_nightly } # Cleans up artifacts and temporary files generated during documentation management. @@ -198,8 +221,9 @@ clean () { # Temporarily disable script exit on errors to ensure cleanup continues set +e - # Remove 'latest' directories and related Git worktrees + # Remove temp directories and related Git worktrees rm -rf docs/docs/latest &> /dev/null + rm -rf docs/docs/nightly &> /dev/null git worktree remove docs/docs &> /dev/null git worktree remove docs/javadoc &> /dev/null diff --git a/site/nav.yml b/site/nav.yml index aa5c77b46122..4a3548433636 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -21,6 +21,7 @@ nav: - Spark: spark-quickstart.md - Hive: hive-quickstart.md - Docs: + - nightly: '!include docs/docs/nightly/mkdocs.yml' - latest: '!include docs/docs/latest/mkdocs.yml' - 1.5.0: '!include docs/docs/1.5.0/mkdocs.yml' - 1.4.3: '!include docs/docs/1.4.3/mkdocs.yml' From e498c958b0bd76a4eed506aa4b3cb9c8d27cdea5 Mon Sep 17 00:00:00 2001 From: Rahil C <32500120+rahil-c@users.noreply.github.com> Date: Fri, 22 Mar 2024 17:58:14 -0400 Subject: [PATCH 0157/1019] Spec: Fix REST pagination requirements based on new feedback (#9917) Co-authored-by: Rahil Chertara --- open-api/rest-catalog-open-api.py | 6 +++--- open-api/rest-catalog-open-api.yaml | 26 ++++++++++++++++++++------ 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 77dcad9cfba6..7bd97b69885f 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -78,9 +78,9 @@ class Namespace(BaseModel): class PageToken(BaseModel): - __root__: str = Field( - ..., - description='An opaque token which allows clients to make use of pagination for a list API (e.g. ListTables). Clients will initiate the first paginated request by sending an empty `pageToken` e.g. `GET /tables?pageToken` or `GET /tables?pageToken=` signaling to the service that the response should be paginated.\nServers that support pagination will recognize `pageToken` and return a `next-page-token` in response if there are more results available. After the initial request, it is expected that the value of `next-page-token` from the last response is used in the subsequent request. Servers that do not support pagination will ignore `next-page-token` and return all results.', + __root__: Optional[str] = Field( + None, + description='An opaque token that allows clients to make use of pagination for list APIs (e.g. ListTables). Clients may initiate the first paginated request by sending an empty query parameter `pageToken` to the server.\nServers that support pagination should identify the `pageToken` parameter and return a `next-page-token` in the response if there are more results available. After the initial request, the value of `next-page-token` from each response must be used as the `pageToken` parameter value for the next request. The server must return `null` value for the `next-page-token` in the last response.\nServers that support pagination must return all results in a single response with the value of `next-page-token` set to `null` if the query parameter `pageToken` is not set in the request.\nServers that do not support pagination should ignore the `pageToken` parameter and return all results in a single response. The `next-page-token` must be omitted from the response.\nClients must interpret either `null` or missing response value of `next-page-token` as the end of the listing results.', ) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 77aabc834adb..161d5e0fcff8 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -1610,14 +1610,28 @@ components: PageToken: description: - An opaque token which allows clients to make use of pagination for a list API (e.g. ListTables). - Clients will initiate the first paginated request by sending an empty `pageToken` e.g. `GET /tables?pageToken` or `GET /tables?pageToken=` - signaling to the service that the response should be paginated. + An opaque token that allows clients to make use of pagination for list APIs + (e.g. ListTables). Clients may initiate the first paginated request by sending an empty + query parameter `pageToken` to the server. + + Servers that support pagination should identify the `pageToken` parameter and return a + `next-page-token` in the response if there are more results available. After the initial + request, the value of `next-page-token` from each response must be used as the `pageToken` + parameter value for the next request. The server must return `null` value for the + `next-page-token` in the last response. + + Servers that support pagination must return all results in a single response with the value + of `next-page-token` set to `null` if the query parameter `pageToken` is not set in the + request. + + Servers that do not support pagination should ignore the `pageToken` parameter and return + all results in a single response. The `next-page-token` must be omitted from the response. + + Clients must interpret either `null` or missing response value of `next-page-token` as + the end of the listing results. - Servers that support pagination will recognize `pageToken` and return a `next-page-token` in response if there are more results available. - After the initial request, it is expected that the value of `next-page-token` from the last response is used in the subsequent request. - Servers that do not support pagination will ignore `next-page-token` and return all results. type: string + nullable: true TableIdentifier: type: object From a04c1a72a3acde119384bea8a079bd1ca8448f28 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sat, 23 Mar 2024 11:37:10 +0100 Subject: [PATCH 0158/1019] docs: Add links checker (#9965) * docs: Add links checker * Comments * Fix broken paths * Fix moar links * Last few --- .github/workflows/docs-check-links.yml | 40 +++++++++++++++++++++++++ README.md | 7 ++--- docs/docs/configuration.md | 8 ++--- docs/docs/daft.md | 2 +- docs/docs/flink-actions.md | 2 +- docs/docs/flink-connector.md | 6 ++-- docs/docs/flink-ddl.md | 2 +- docs/docs/flink-queries.md | 2 +- docs/docs/flink-writes.md | 10 +++---- docs/docs/flink.md | 35 +++++++++++----------- docs/docs/spark-configuration.md | 4 +-- docs/docs/spark-ddl.md | 16 +++++----- docs/docs/spark-getting-started.md | 31 +++++++++---------- docs/docs/spark-procedures.md | 8 ++--- docs/docs/spark-queries.md | 4 +-- docs/docs/spark-structured-streaming.md | 10 +++---- docs/docs/spark-writes.md | 10 +++---- format/spec.md | 2 +- site/README.md | 2 +- site/docs/blogs.md | 11 ++++--- site/docs/how-to-release.md | 2 +- site/docs/multi-engine-support.md | 12 ++++++++ site/docs/releases.md | 7 +++-- site/docs/spark-quickstart.md | 1 + site/docs/vendors.md | 9 ++++-- site/link-checker-config.json | 23 ++++++++++++++ 26 files changed, 172 insertions(+), 94 deletions(-) create mode 100644 .github/workflows/docs-check-links.yml create mode 100644 site/link-checker-config.json diff --git a/.github/workflows/docs-check-links.yml b/.github/workflows/docs-check-links.yml new file mode 100644 index 000000000000..9de842813dc3 --- /dev/null +++ b/.github/workflows/docs-check-links.yml @@ -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. +# + +name: Check Markdown docs links + +on: + push: + paths: + - docs/** + - site/** + branches: + - 'main' + pull_request: + workflow_dispatch: + +jobs: + markdown-link-check: + runs-on: ubuntu-latest + steps: + - uses: actions/checkout@v4 + - uses: gaurav-nelson/github-action-markdown-link-check@v1 + with: + config-file: 'site/link-checker-config.json' + use-verbose-mode: yes diff --git a/README.md b/README.md index 3ba4f74f871a..8d36b212b953 100644 --- a/README.md +++ b/README.md @@ -17,7 +17,7 @@ - under the License. --> -![Iceberg](https://iceberg.apache.org/docs/latest/img/Iceberg-logo.png) +![Iceberg](https://iceberg.apache.org/assets/images/Iceberg-logo.svg) [![](https://github.com/apache/iceberg/actions/workflows/java-ci.yml/badge.svg)](https://github.com/apache/iceberg/actions/workflows/java-ci.yml) [![Slack](https://img.shields.io/badge/chat-on%20Slack-brightgreen.svg)](https://apache-iceberg.slack.com/) @@ -37,11 +37,8 @@ The core Java library is located in this repository and is the reference impleme [Documentation][iceberg-docs] is available for all libraries and integrations. -Current work is tracked in the [roadmap][roadmap]. - [iceberg-docs]: https://iceberg.apache.org/docs/latest/ -[iceberg-spec]: https://iceberg.apache.org/spec -[roadmap]: https://iceberg.apache.org/roadmap/ +[iceberg-spec]: https://iceberg.apache.org/spec/ ## Collaboration diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index d5e33529c0dd..ec7af06ca7f9 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -108,9 +108,9 @@ Iceberg tables support table properties to configure table behavior, like the de Reserved table properties are only used to control behaviors when creating or updating a table. The value of these properties are not persisted as a part of the table metadata. -| Property | Default | Description | -| -------------- | -------- | ------------------------------------------------------------- | -| format-version | 2 | Table's format version (can be 1 or 2) as defined in the [Spec](../../../spec/#format-versioning). Defaults to 2 since version 1.4.0. | +| Property | Default | Description | +| -------------- | -------- |--------------------------------------------------------------------------------------------------------------------------------------| +| format-version | 2 | Table's format version (can be 1 or 2) as defined in the [Spec](../../spec.md#format-versioning). Defaults to 2 since version 1.4.0. | ### Compatibility flags @@ -131,7 +131,7 @@ Iceberg catalogs support using catalog properties to configure catalog behaviors | clients | 2 | client pool size | | cache-enabled | true | Whether to cache catalog entries | | cache.expiration-interval-ms | 30000 | How long catalog entries are locally cached, in milliseconds; 0 disables caching, negative values disable expiration | -| metrics-reporter-impl | org.apache.iceberg.metrics.LoggingMetricsReporter | Custom `MetricsReporter` implementation to use in a catalog. See the [Metrics reporting](../metrics-reporting.md) section for additional details | +| metrics-reporter-impl | org.apache.iceberg.metrics.LoggingMetricsReporter | Custom `MetricsReporter` implementation to use in a catalog. See the [Metrics reporting](metrics-reporting.md) section for additional details | `HadoopCatalog` and `HiveCatalog` can access the properties in their constructors. Any other custom catalog can access the properties by implementing `Catalog.initialize(catalogName, catalogProperties)`. diff --git a/docs/docs/daft.md b/docs/docs/daft.md index da78b7eb6ccd..71030e394957 100644 --- a/docs/docs/daft.md +++ b/docs/docs/daft.md @@ -20,7 +20,7 @@ title: "Daft" # Daft -[Daft](www.getdaft.io) is a distributed query engine written in Python and Rust, two fast-growing ecosystems in the data engineering and machine learning industry. +[Daft](https://www.getdaft.io/) is a distributed query engine written in Python and Rust, two fast-growing ecosystems in the data engineering and machine learning industry. It exposes its flavor of the familiar [Python DataFrame API](https://www.getdaft.io/projects/docs/en/latest/api_docs/dataframe.html) which is a common abstraction over querying tables of data in the Python data ecosystem. diff --git a/docs/docs/flink-actions.md b/docs/docs/flink-actions.md index 4e54732c3b1a..c058795fd079 100644 --- a/docs/docs/flink-actions.md +++ b/docs/docs/flink-actions.md @@ -20,7 +20,7 @@ title: "Flink Actions" ## Rewrite files action -Iceberg provides API to rewrite small files into large files by submitting Flink batch jobs. The behavior of this Flink action is the same as Spark's [rewriteDataFiles](../maintenance.md#compact-data-files). +Iceberg provides API to rewrite small files into large files by submitting Flink batch jobs. The behavior of this Flink action is the same as Spark's [rewriteDataFiles](maintenance.md#compact-data-files). ```java import org.apache.iceberg.flink.actions.Actions; diff --git a/docs/docs/flink-connector.md b/docs/docs/flink-connector.md index 260a5c581493..025e9aee92ea 100644 --- a/docs/docs/flink-connector.md +++ b/docs/docs/flink-connector.md @@ -29,13 +29,13 @@ To create the table in Flink SQL by using SQL syntax `CREATE TABLE test (..) WIT * `connector`: Use the constant `iceberg`. * `catalog-name`: User-specified catalog name. It's required because the connector don't have any default value. * `catalog-type`: `hive` or `hadoop` for built-in catalogs (defaults to `hive`), or left unset for custom catalog implementations using `catalog-impl`. -* `catalog-impl`: The fully-qualified class name of a custom catalog implementation. Must be set if `catalog-type` is unset. See also [custom catalog](../flink.md#adding-catalogs) for more details. +* `catalog-impl`: The fully-qualified class name of a custom catalog implementation. Must be set if `catalog-type` is unset. See also [custom catalog](flink.md#adding-catalogs) for more details. * `catalog-database`: The iceberg database name in the backend catalog, use the current flink database name by default. * `catalog-table`: The iceberg table name in the backend catalog. Default to use the table name in the flink `CREATE TABLE` sentence. ## Table managed in Hive catalog. -Before executing the following SQL, please make sure you've configured the Flink SQL client correctly according to the [quick start documentation](../flink.md). +Before executing the following SQL, please make sure you've configured the Flink SQL client correctly according to the [quick start documentation](flink.md). The following SQL will create a Flink table in the current Flink catalog, which maps to the iceberg table `default_database.flink_table` managed in iceberg catalog. @@ -138,4 +138,4 @@ SELECT * FROM flink_table; 3 rows in set ``` -For more details, please refer to the Iceberg [Flink documentation](../flink.md). +For more details, please refer to the Iceberg [Flink documentation](flink.md). diff --git a/docs/docs/flink-ddl.md b/docs/docs/flink-ddl.md index 681a018865aa..c2b3051fde8d 100644 --- a/docs/docs/flink-ddl.md +++ b/docs/docs/flink-ddl.md @@ -150,7 +150,7 @@ Table create commands support the commonly used [Flink create clauses](https://n * `PARTITION BY (column1, column2, ...)` to configure partitioning, Flink does not yet support hidden partitioning. * `COMMENT 'table document'` to set a table description. -* `WITH ('key'='value', ...)` to set [table configuration](../configuration.md) which will be stored in Iceberg table properties. +* `WITH ('key'='value', ...)` to set [table configuration](configuration.md) which will be stored in Iceberg table properties. Currently, it does not support computed column and watermark definition etc. diff --git a/docs/docs/flink-queries.md b/docs/docs/flink-queries.md index 036d95a4953a..431a5554f248 100644 --- a/docs/docs/flink-queries.md +++ b/docs/docs/flink-queries.md @@ -75,7 +75,7 @@ SET table.exec.iceberg.use-flip27-source = true; ### 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) +refer to [Flink Configuration](flink-configuration.md#read-options) ```sql --- Read from branch b1 diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index c41b367deaed..ef1e602c8212 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -67,7 +67,7 @@ Iceberg supports `UPSERT` based on the primary key when writing data into v2 tab ) with ('format-version'='2', 'write.upsert.enabled'='true'); ``` -2. Enabling `UPSERT` mode using `upsert-enabled` in the [write options](#write-options) provides more flexibility than a table level config. Note that you still need to use v2 table format and specify the [primary key](../flink-ddl.md/#primary-key) or [identifier fields](../../spec.md#identifier-field-ids) when creating the table. +2. Enabling `UPSERT` mode using `upsert-enabled` in the [write options](#write-options) provides more flexibility than a table level config. Note that you still need to use v2 table format and specify the [primary key](flink-ddl.md/#primary-key) or [identifier fields](../../spec.md#identifier-field-ids) when creating the table. ```sql INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ @@ -185,7 +185,7 @@ FlinkSink.builderFor( ### Branch Writes Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink` -For more information on branches please refer to [branches](../branching.md). +For more information on branches please refer to [branches](branching.md). ```java FlinkSink.forRowData(input) .tableLoader(tableLoader) @@ -262,13 +262,13 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ ... ``` -Check out all the options here: [write-options](../flink-configuration.md#write-options) +Check out all the options here: [write-options](flink-configuration.md#write-options) ## Notes Flink streaming write jobs rely on snapshot summary to keep the last committed checkpoint ID, and -store uncommitted data as temporary files. Therefore, [expiring snapshots](../maintenance.md#expire-snapshots) -and [deleting orphan files](../maintenance.md#delete-orphan-files) could possibly corrupt +store uncommitted data as temporary files. Therefore, [expiring snapshots](maintenance.md#expire-snapshots) +and [deleting orphan files](maintenance.md#delete-orphan-files) could possibly corrupt 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. diff --git a/docs/docs/flink.md b/docs/docs/flink.md index 7f27a280eb91..b8ab694ad9bc 100644 --- a/docs/docs/flink.md +++ b/docs/docs/flink.md @@ -22,22 +22,22 @@ title: "Flink Getting Started" Apache Iceberg supports both [Apache Flink](https://flink.apache.org/)'s DataStream API and Table API. See the [Multi-Engine Support](../../multi-engine-support.md#apache-flink) page for the integration of Apache Flink. -| Feature support | Flink | Notes | -| ----------------------------------------------------------- |-------|----------------------------------------------------------------------------------------| -| [SQL create catalog](../flink-ddl.md#create-catalog) | ✔️ | | -| [SQL create database](../flink-ddl.md#create-database) | ✔️ | | -| [SQL create table](../flink-ddl.md#create-table) | ✔️ | | -| [SQL create table like](../flink-ddl.md#create-table-like) | ✔️ | | -| [SQL alter table](../flink-ddl.md#alter-table) | ✔️ | Only support altering table properties, column and partition changes are not supported | -| [SQL drop_table](../flink-ddl.md#drop-table) | ✔️ | | -| [SQL select](../flink-queries.md#reading-with-sql) | ✔️ | Support both streaming and batch mode | -| [SQL insert into](../flink-writes.md#insert-into) | ✔️ ️ | Support both streaming and batch mode | -| [SQL insert overwrite](../flink-writes.md#insert-overwrite) | ✔️ ️ | | -| [DataStream read](../flink-queries.md#reading-with-datastream) | ✔️ ️ | | -| [DataStream append](../flink-writes.md#appending-data) | ✔️ ️ | | -| [DataStream overwrite](../flink-writes.md#overwrite-data) | ✔️ ️ | | -| [Metadata tables](../flink-queries.md#inspecting-tables) | ✔️ | | -| [Rewrite files action](../flink-actions.md#rewrite-files-action) | ✔️ ️ | | +| Feature support | Flink | Notes | +| -------------------------------------------------------- |-------|----------------------------------------------------------------------------------------| +| [SQL create catalog](flink-ddl.md#create-catalog) | ✔️ | | +| [SQL create database](flink-ddl.md#create-database) | ✔️ | | +| [SQL create table](flink-ddl.md#create-table) | ✔️ | | +| [SQL create table like](flink-ddl.md#create-table-like) | ✔️ | | +| [SQL alter table](flink-ddl.md#alter-table) | ✔️ | Only support altering table properties, column and partition changes are not supported | +| [SQL drop_table](flink-ddl.md#drop-table) | ✔️ | | +| [SQL select](flink-queries.md#reading-with-sql) | ✔️ | Support both streaming and batch mode | +| [SQL insert into](flink-writes.md#insert-into) | ✔️ ️ | Support both streaming and batch mode | +| [SQL insert overwrite](flink-writes.md#insert-overwrite) | ✔️ ️ | | +| [DataStream read](flink-queries.md#reading-with-datastream) | ✔️ ️ | | +| [DataStream append](flink-writes.md#appending-data) | ✔️ ️ | | +| [DataStream overwrite](flink-writes.md#overwrite-data) | ✔️ ️ | | +| [Metadata tables](flink-queries.md#inspecting-tables) | ✔️ | | +| [Rewrite files action](flink-actions.md#rewrite-files-action) | ✔️ ️ | | ## Preparation when using Flink SQL Client @@ -69,6 +69,7 @@ export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath` ./bin/start-cluster.sh ``` + Start the Flink SQL client. There is a separate `flink-runtime` module in the Iceberg project to generate a bundled jar, which could be loaded by Flink SQL client directly. To build the `flink-runtime` bundled jar manually, build the `iceberg` project, and it will generate the jar under `/flink-runtime/build/libs`. Or download the `flink-runtime` jar from the [Apache repository](https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/). ```bash @@ -271,7 +272,7 @@ env.execute("Test Iceberg DataStream"); ### Branch Writes Writing to branches in Iceberg tables is also supported via the `toBranch` API in `FlinkSink` -For more information on branches please refer to [branches](../branching.md). +For more information on branches please refer to [branches](branching.md). ```java FlinkSink.forRowData(input) .tableLoader(tableLoader) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 5e9c6e5d1147..6ac4f1e9c82a 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -78,7 +78,7 @@ Both catalogs are configured using properties nested under the catalog name. Com | 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 | -Additional properties can be found in common [catalog configuration](../configuration.md#catalog-properties). +Additional properties can be found in common [catalog configuration](configuration.md#catalog-properties). ### Using catalogs @@ -185,7 +185,7 @@ df.write | fanout-enabled | false | Overrides this table's write.spark.fanout.enabled | | check-ordering | true | Checks if input schema and table schema are same | | isolation-level | null | Desired isolation level for Dataframe overwrite operations. `null` => no checks (for idempotent writes), `serializable` => check for concurrent inserts or deletes in destination partitions, `snapshot` => checks for concurrent deletes in destination partitions. | -| validate-from-snapshot-id | null | If isolation level is set, id of base snapshot from which to check concurrent write conflicts into a table. Should be the snapshot before any reads from the table. Can be obtained via [Table API](../api.md#table-metadata) or [Snapshots table](../spark-queries.md#snapshots). If null, the table's oldest known snapshot is used. | +| validate-from-snapshot-id | null | If isolation level is set, id of base snapshot from which to check concurrent write conflicts into a table. Should be the snapshot before any reads from the table. Can be obtained via [Table API](api.md#table-metadata) or [Snapshots table](spark-queries.md#snapshots). If null, the table's oldest known snapshot is used. | | compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | | compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | | compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | diff --git a/docs/docs/spark-ddl.md b/docs/docs/spark-ddl.md index e1376ddcf667..8b30710997c8 100644 --- a/docs/docs/spark-ddl.md +++ b/docs/docs/spark-ddl.md @@ -33,14 +33,14 @@ CREATE TABLE prod.db.sample ( USING iceberg; ``` -Iceberg will convert the column type in Spark to corresponding Iceberg type. Please check the section of [type compatibility on creating table](../spark-getting-started.md#spark-type-to-iceberg-type) for details. +Iceberg will convert the column type in Spark to corresponding Iceberg type. Please check the section of [type compatibility on creating table](spark-getting-started.md#spark-type-to-iceberg-type) for details. Table create commands, including CTAS and RTAS, support the full range of Spark create clauses, including: * `PARTITIONED BY (partition-expressions)` to configure partitioning * `LOCATION '(fully-qualified-uri)'` to set the table location * `COMMENT 'table documentation'` to set a table description -* `TBLPROPERTIES ('key'='value', ...)` to set [table configuration](../configuration.md) +* `TBLPROPERTIES ('key'='value', ...)` to set [table configuration](configuration.md) Create commands may also set the default format with the `USING` clause. This is only supported for `SparkCatalog` because Spark handles the `USING` clause differently for the built-in catalog. @@ -59,7 +59,7 @@ USING iceberg PARTITIONED BY (category); ``` -The `PARTITIONED BY` clause supports transform expressions to create [hidden partitions](../partitioning.md). +The `PARTITIONED BY` clause supports transform expressions to create [hidden partitions](partitioning.md). ```sql CREATE TABLE prod.db.sample ( @@ -86,7 +86,7 @@ Note: Old syntax of `years(ts)`, `months(ts)`, `days(ts)` and `hours(ts)` are al ## `CREATE TABLE ... AS SELECT` -Iceberg supports CTAS as an atomic operation when using a [`SparkCatalog`](../spark-configuration.md#catalog-configuration). CTAS is supported, but is not atomic when using [`SparkSessionCatalog`](../spark-configuration.md#replacing-the-session-catalog). +Iceberg supports CTAS as an atomic operation when using a [`SparkCatalog`](spark-configuration.md#catalog-configuration). CTAS is supported, but is not atomic when using [`SparkSessionCatalog`](spark-configuration.md#replacing-the-session-catalog). ```sql CREATE TABLE prod.db.sample @@ -106,7 +106,7 @@ AS SELECT ... ## `REPLACE TABLE ... AS SELECT` -Iceberg supports RTAS as an atomic operation when using a [`SparkCatalog`](../spark-configuration.md#catalog-configuration). RTAS is supported, but is not atomic when using [`SparkSessionCatalog`](../spark-configuration.md#replacing-the-session-catalog). +Iceberg supports RTAS as an atomic operation when using a [`SparkCatalog`](spark-configuration.md#catalog-configuration). RTAS is supported, but is not atomic when using [`SparkSessionCatalog`](spark-configuration.md#replacing-the-session-catalog). Atomic table replacement creates a new snapshot with the results of the `SELECT` query, but keeps table history. @@ -168,7 +168,7 @@ Iceberg has full `ALTER TABLE` support in Spark 3, including: * Widening the type of `int`, `float`, and `decimal` fields * Making required columns optional -In addition, [SQL extensions](../spark-configuration.md#sql-extensions) can be used to add support for partition evolution and setting a table's write order +In addition, [SQL extensions](spark-configuration.md#sql-extensions) can be used to add support for partition evolution and setting a table's write order ### `ALTER TABLE ... RENAME TO` @@ -184,7 +184,7 @@ ALTER TABLE prod.db.sample SET TBLPROPERTIES ( ); ``` -Iceberg uses table properties to control table behavior. For a list of available properties, see [Table configuration](../configuration.md). +Iceberg uses table properties to control table behavior. For a list of available properties, see [Table configuration](configuration.md). `UNSET` is used to remove properties: @@ -325,7 +325,7 @@ ALTER TABLE prod.db.sample DROP COLUMN point.z; ## `ALTER TABLE` SQL extensions -These commands are available in Spark 3 when using Iceberg [SQL extensions](../spark-configuration.md#sql-extensions). +These commands are available in Spark 3 when using Iceberg [SQL extensions](spark-configuration.md#sql-extensions). ### `ALTER TABLE ... ADD PARTITION FIELD` diff --git a/docs/docs/spark-getting-started.md b/docs/docs/spark-getting-started.md index 72642cc6e14f..2bcdbd23eb1e 100644 --- a/docs/docs/spark-getting-started.md +++ b/docs/docs/spark-getting-started.md @@ -35,12 +35,13 @@ spark-shell --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:{{ iceb ``` !!! info + If you want to include Iceberg in your Spark installation, add the [`iceberg-spark-runtime-3.5_2.12` Jar](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) to Spark's `jars` folder. ### Adding catalogs -Iceberg comes with [catalogs](../spark-configuration.md#catalogs) that enable SQL commands to manage tables and load them by name. Catalogs are configured using properties under `spark.sql.catalog.(catalog_name)`. +Iceberg comes with [catalogs](spark-configuration.md#catalogs) that enable SQL commands to manage tables and load them by name. Catalogs are configured using properties under `spark.sql.catalog.(catalog_name)`. This command creates a path-based catalog named `local` for tables under `$PWD/warehouse` and adds support for Iceberg tables to Spark's built-in catalog: @@ -56,7 +57,7 @@ spark-sql --packages org.apache.iceberg:iceberg-spark-runtime-3.5_2.12:{{ iceber ### Creating a table -To create your first Iceberg table in Spark, use the `spark-sql` shell or `spark.sql(...)` to run a [`CREATE TABLE`](../spark-ddl.md#create-table) command: +To create your first Iceberg table in Spark, use the `spark-sql` shell or `spark.sql(...)` to run a [`CREATE TABLE`](spark-ddl.md#create-table) command: ```sql -- local is the path-based catalog defined above @@ -65,21 +66,21 @@ CREATE TABLE local.db.table (id bigint, data string) USING iceberg; Iceberg catalogs support the full range of SQL DDL commands, including: -* [`CREATE TABLE ... PARTITIONED BY`](../spark-ddl.md#create-table) -* [`CREATE TABLE ... AS SELECT`](../spark-ddl.md#create-table-as-select) -* [`ALTER TABLE`](../spark-ddl.md#alter-table) -* [`DROP TABLE`](../spark-ddl.md#drop-table) +* [`CREATE TABLE ... PARTITIONED BY`](spark-ddl.md#create-table) +* [`CREATE TABLE ... AS SELECT`](spark-ddl.md#create-table-as-select) +* [`ALTER TABLE`](spark-ddl.md#alter-table) +* [`DROP TABLE`](spark-ddl.md#drop-table) ### Writing -Once your table is created, insert data using [`INSERT INTO`](../spark-writes.md#insert-into): +Once your table is created, insert data using [`INSERT INTO`](spark-writes.md#insert-into): ```sql INSERT INTO local.db.table VALUES (1, 'a'), (2, 'b'), (3, 'c'); INSERT INTO local.db.table SELECT id, data FROM source WHERE length(data) = 1; ``` -Iceberg also adds row-level SQL updates to Spark, [`MERGE INTO`](../spark-writes.md#merge-into) and [`DELETE FROM`](../spark-writes.md#delete-from): +Iceberg also adds row-level SQL updates to Spark, [`MERGE INTO`](spark-writes.md#merge-into) and [`DELETE FROM`](spark-writes.md#delete-from): ```sql MERGE INTO local.db.target t USING (SELECT * FROM updates) u ON t.id = u.id @@ -87,7 +88,7 @@ WHEN MATCHED THEN UPDATE SET t.count = t.count + u.count WHEN NOT MATCHED THEN INSERT *; ``` -Iceberg supports writing DataFrames using the new [v2 DataFrame write API](../spark-writes.md#writing-with-dataframes): +Iceberg supports writing DataFrames using the new [v2 DataFrame write API](spark-writes.md#writing-with-dataframes): ```scala spark.table("source").select("id", "data") @@ -106,7 +107,7 @@ FROM local.db.table GROUP BY data; ``` -SQL is also the recommended way to [inspect tables](../spark-queries.md#inspecting-tables). To view all snapshots in a table, use the `snapshots` metadata table: +SQL is also the recommended way to [inspect tables](spark-queries.md#inspecting-tables). To view all snapshots in a table, use the `snapshots` metadata table: ```sql SELECT * FROM local.db.table.snapshots; ``` @@ -121,7 +122,7 @@ SELECT * FROM local.db.table.snapshots; +-------------------------+----------------+-----------+-----------+----------------------------------------------------+-----+ ``` -[DataFrame reads](../spark-queries.md#querying-with-dataframes) are supported and can now reference tables by name using `spark.table`: +[DataFrame reads](spark-queries.md#querying-with-dataframes) are supported and can now reference tables by name using `spark.table`: ```scala val df = spark.table("local.db.table") @@ -192,7 +193,7 @@ This type conversion table describes how Iceberg types are converted to the Spar Next, you can learn more about Iceberg tables in Spark: -* [DDL commands](../spark-ddl.md): `CREATE`, `ALTER`, and `DROP` -* [Querying data](../spark-queries.md): `SELECT` queries and metadata tables -* [Writing data](../spark-writes.md): `INSERT INTO` and `MERGE INTO` -* [Maintaining tables](../spark-procedures.md) with stored procedures +* [DDL commands](spark-ddl.md): `CREATE`, `ALTER`, and `DROP` +* [Querying data](spark-queries.md): `SELECT` queries and metadata tables +* [Writing data](spark-writes.md): `INSERT INTO` and `MERGE INTO` +* [Maintaining tables](spark-procedures.md) with stored procedures diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 7dc0d1a2aab7..dc439c04c855 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -20,7 +20,7 @@ title: "Procedures" # Spark Procedures -To use Iceberg in Spark, first configure [Spark catalogs](../spark-configuration.md). Stored procedures are only available when using [Iceberg SQL extensions](../spark-configuration.md#sql-extensions) in Spark 3. +To use Iceberg in Spark, first configure [Spark catalogs](spark-configuration.md). Stored procedures are only available when using [Iceberg SQL extensions](spark-configuration.md#sql-extensions) in Spark 3. ## Usage @@ -272,7 +272,7 @@ the `expire_snapshots` procedure will never remove files which are still require | `stream_results` | | boolean | When true, deletion files will be sent to Spark driver by RDD partition (by default, all the files will be sent to Spark driver). This option is recommended to set to `true` to prevent Spark driver OOM from large file size | | `snapshot_ids` | | array of long | Array of snapshot IDs to expire. | -If `older_than` and `retain_last` are omitted, the table's [expiration properties](../configuration.md#table-behavior-properties) will be used. +If `older_than` and `retain_last` are omitted, the table's [expiration properties](configuration.md#table-behavior-properties) will be used. Snapshots that are still referenced by branches or tags won't be removed. By default, branches and tags never expire, but their retention policy can be changed with the table property `history.expire.max-ref-age-ms`. The `main` branch never expires. #### Output @@ -357,7 +357,7 @@ Iceberg can compact data files in parallel using Spark with the `rewriteDataFile | `partial-progress.max-commits` | 10 | Maximum amount of commits that this rewrite is allowed to produce 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 | +| `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 | | `min-file-size-bytes` | 75% of target file size | Files under this threshold will be considered for rewriting regardless of any other criteria | | `max-file-size-bytes` | 180% of target file size | Files with sizes above this threshold will be considered for rewriting regardless of any other criteria | | `min-input-files` | 5 | Any file group exceeding this number of files will be rewritten regardless of other criteria | @@ -480,7 +480,7 @@ Dangling deletes are always filtered out during rewriting. | `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 | | `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` | 67108864 (64MB, default value of `write.delete.target-file-size-bytes` from [table properties](../configuration.md#write-properties)) | Target output file size | +| `target-file-size-bytes` | 67108864 (64MB, default value of `write.delete.target-file-size-bytes` from [table properties](configuration.md#write-properties)) | Target output file size | | `min-file-size-bytes` | 75% of target file size | Files under this threshold will be considered for rewriting regardless of any other criteria | | `max-file-size-bytes` | 180% of target file size | Files with sizes above this threshold will be considered for rewriting regardless of any other criteria | | `min-input-files` | 5 | Any file group exceeding this number of files will be rewritten regardless of other criteria | diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index 092ed6b1d636..536c136d7e55 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -20,11 +20,11 @@ title: "Queries" # Spark Queries -To use Iceberg in Spark, first configure [Spark catalogs](../spark-configuration.md). Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. +To use Iceberg in Spark, first configure [Spark catalogs](spark-configuration.md). Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. ## Querying with SQL -In Spark 3, tables use identifiers that include a [catalog name](../spark-configuration.md#using-catalogs). +In Spark 3, tables use identifiers that include a [catalog name](spark-configuration.md#using-catalogs). ```sql SELECT * FROM prod.db.table; -- catalog: prod, namespace: db, table: table diff --git a/docs/docs/spark-structured-streaming.md b/docs/docs/spark-structured-streaming.md index 50799042073f..0ac753808d9e 100644 --- a/docs/docs/spark-structured-streaming.md +++ b/docs/docs/spark-structured-streaming.md @@ -68,7 +68,7 @@ Iceberg supports `append` and `complete` output modes: * `append`: appends the rows of every micro-batch to the table * `complete`: replaces the table contents every micro-batch -Prior to starting the streaming query, ensure you created the table. Refer to the [SQL create table](../spark-ddl.md#create-table) documentation to learn how to create the Iceberg table. +Prior to starting the streaming query, ensure you created the table. Refer to the [SQL create table](spark-ddl.md#create-table) documentation to learn how to create the Iceberg table. Iceberg doesn't support experimental [continuous processing](https://spark.apache.org/docs/latest/structured-streaming-programming-guide.html#continuous-processing), as it doesn't provide the interface to "commit" the output. @@ -76,7 +76,7 @@ Iceberg doesn't support experimental [continuous processing](https://spark.apach Iceberg requires sorting data by partition per task prior to writing the data. In Spark tasks are split by Spark partition. against partitioned table. For batch queries you're encouraged to do explicit sort to fulfill the requirement -(see [here](../spark-writes.md#writing-distribution-modes)), but the approach would bring additional latency as +(see [here](spark-writes.md#writing-distribution-modes)), but the approach would bring additional latency as repartition and sort are considered as heavy operations for streaming workload. To avoid additional latency, you can enable fanout writer to eliminate the requirement. @@ -107,13 +107,13 @@ documents how to configure the interval. ### Expire old snapshots -Each batch written to a table produces a new snapshot. Iceberg tracks snapshots in table metadata until they are expired. Snapshots accumulate quickly with frequent commits, so it is highly recommended that tables written by streaming queries are [regularly maintained](../maintenance.md#expire-snapshots). [Snapshot expiration](../spark-procedures.md#expire_snapshots) is the procedure of removing the metadata and any data files that are no longer needed. By default, the procedure will expire the snapshots older than five days. +Each batch written to a table produces a new snapshot. Iceberg tracks snapshots in table metadata until they are expired. Snapshots accumulate quickly with frequent commits, so it is highly recommended that tables written by streaming queries are [regularly maintained](maintenance.md#expire-snapshots). [Snapshot expiration](spark-procedures.md#expire_snapshots) is the procedure of removing the metadata and any data files that are no longer needed. By default, the procedure will expire the snapshots older than five days. ### Compacting data files -The amount of data written from a streaming process is typically small, which can cause the table metadata to track lots of small files. [Compacting small files into larger files](../maintenance.md#compact-data-files) reduces the metadata needed by the table, and increases query efficiency. Iceberg and Spark [comes with the `rewrite_data_files` procedure](../spark-procedures.md#rewrite_data_files). +The amount of data written from a streaming process is typically small, which can cause the table metadata to track lots of small files. [Compacting small files into larger files](maintenance.md#compact-data-files) reduces the metadata needed by the table, and increases query efficiency. Iceberg and Spark [comes with the `rewrite_data_files` procedure](spark-procedures.md#rewrite_data_files). ### Rewrite manifests To optimize write latency on a streaming workload, Iceberg can write the new snapshot with a "fast" append that does not automatically compact manifests. -This could lead lots of small manifest files. Iceberg can [rewrite the number of manifest files to improve query performance](../maintenance.md#rewrite-manifests). Iceberg and Spark [come with the `rewrite_manifests` procedure](../spark-procedures.md#rewrite_manifests). +This could lead lots of small manifest files. Iceberg can [rewrite the number of manifest files to improve query performance](maintenance.md#rewrite-manifests). Iceberg and Spark [come with the `rewrite_manifests` procedure](spark-procedures.md#rewrite_manifests). diff --git a/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index efc15e7e35fc..626dee6c96e6 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -20,9 +20,9 @@ title: "Writes" # Spark Writes -To use Iceberg in Spark, first configure [Spark catalogs](../spark-configuration.md). +To use Iceberg in Spark, first configure [Spark catalogs](spark-configuration.md). -Some plans are only available when using [Iceberg SQL extensions](../spark-configuration.md#sql-extensions) in Spark 3. +Some plans are only available when using [Iceberg SQL extensions](spark-configuration.md#sql-extensions) in Spark 3. Iceberg uses Apache Spark's DataSourceV2 API for data source and catalog implementations. Spark DSv2 is an evolving API with different levels of support in Spark versions: @@ -200,7 +200,7 @@ Branch writes can also be performed as part of a write-audit-publish (WAP) workf Note WAP branch and branch identifier cannot both be specified. Also, the branch must exist before performing the write. The operation does **not** create the branch if it does not exist. -For more information on branches please refer to [branches](../branching.md). +For more information on branches please refer to [branches](branching.md). ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. @@ -364,7 +364,7 @@ There are 3 options for `write.distribution-mode` This mode does not request any shuffles or sort to be performed automatically by Spark. Because no work is done automatically by Spark, the data must be *manually* sorted by partition value. The data must be sorted either within each spark task, or globally within the entire dataset. A global sort will minimize the number of output files. -A sort can be avoided by using the Spark [write fanout](../spark-configuration.md#write-options) property but this will cause all +A sort can be avoided by using the Spark [write fanout](spark-configuration.md#write-options) property but this will cause all file handles to remain open until each write task has completed. * `hash` - This mode is the new default and requests that Spark uses a hash-based exchange to shuffle the incoming write data before writing. @@ -385,7 +385,7 @@ sort-order. Further division and coalescing of tasks may take place because of When writing data to Iceberg with Spark, it's important to note that Spark cannot write a file larger than a Spark task and a file cannot span an Iceberg partition boundary. This means although Iceberg will always roll over a file -when it grows to [`write.target-file-size-bytes`](../configuration.md#write-properties), but unless the Spark task is +when it grows to [`write.target-file-size-bytes`](configuration.md#write-properties), but unless the Spark task is large enough that will not happen. The size of the file created on disk will also be much smaller than the Spark task since the on disk data will be both compressed and in columnar format as opposed to Spark's uncompressed row representation. This means a 100 megabyte Spark task will create a file much smaller than 100 megabytes even if that diff --git a/format/spec.md b/format/spec.md index 397057a97456..ab6f3494830c 100644 --- a/format/spec.md +++ b/format/spec.md @@ -57,7 +57,7 @@ In addition to row-level deletes, version 2 makes some requirements stricter for ## Overview -![Iceberg snapshot structure](assets/images/iceberg-metadata.png) +![Iceberg snapshot structure](https://iceberg.apache.org/assets/images/iceberg-metadata.png) This table format tracks individual data files in a table instead of directories. This allows writers to create data files in-place and only adds files to the table in an explicit commit. diff --git a/site/README.md b/site/README.md index 6cb4f4907ebf..b1f9310e8bf8 100644 --- a/site/README.md +++ b/site/README.md @@ -74,7 +74,6 @@ The docs are built, run, and released using [make](https://www.gnu.org/software/ > [clean](dev/clean.sh): Clean the local site. > [deploy](dev/deploy.sh): Clean, build, and deploy the Iceberg docs site. > help: Show help for each of the Makefile recipes. -> [release](dev/release.sh): Release the current `/docs` as `ICEBERG_VERSION` (`make release ICEBERG_VERSION=`). > [serve](dev/serve.sh): Clean, build, and run the site locally. To scaffold the versioned docs and build the project, run the `build` recipe. @@ -103,6 +102,7 @@ This step will generate the staged source code which blends into the original so └─.asf.yaml ``` + To run this, run the `serve` recipe, which runs the `build` recipe and calls `mkdocs serve`. This will run locally at . ``` make serve diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 746eef97d663..1714ce50405d 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -22,6 +22,7 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. + ### [The Apache Iceberg Lakehouse: The Great Data Equalizer](https://amdatalakehouse.substack.com/p/the-apache-iceberg-lakehouse-the) **Date**: March 6th, 2024, **Company**: Dremio @@ -42,6 +43,7 @@ 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/) + ### [What is the Data Lakehouse and the Role of Apache Iceberg, Nessie and Dremio?](https://amdatalakehouse.substack.com/p/the-apache-iceberg-lakehouse-the) **Date**: February 21st, 2024, **Company**: Dremio @@ -147,6 +149,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Iceberg Tables: Catalog Support Now Available](https://www.snowflake.com/blog/iceberg-tables-catalog-support-available-now/) **Date**: March 29th, 2023, **Company**: Snowflake @@ -362,6 +365,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/), [Kyle Bendickson](https://www.linkedin.com/in/kylebendickson/) + ### [Expanding the Data Cloud with Apache Iceberg](https://www.snowflake.com/blog/expanding-the-data-cloud-with-apache-iceberg/) **Date**: January 21st, 2022, **Company**: Snowflake @@ -377,11 +381,6 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) -### [Using Flink CDC to synchronize data from MySQL sharding tables and build real-time data lake](https://ververica.github.io/flink-cdc-connectors/master/content/quickstart/build-real-time-data-lake-tutorial.html) -**Date**: November 11th, 2021, **Company**: Ververica, Alibaba Cloud - -**Author**: [Yuxia Luo](https://github.com/luoyuxia), [Jark Wu](https://github.com/wuchong), [Zheng Hu](https://www.linkedin.com/in/zheng-hu-37017683/) - ### [Metadata Indexing in Iceberg](https://tabular.io/blog/iceberg-metadata-indexing/) **Date**: October 10th, 2021, **Company**: Tabular @@ -450,7 +449,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f ### [High Throughput Ingestion with Iceberg](https://medium.com/adobetech/high-throughput-ingestion-with-iceberg-ccf7877a413f) **Date**: Dec 22nd, 2020, **Company**: Adobe -**Author**: [Andrei Ionescu](http://linkedin.com/in/andreiionescu), [Shone Sadler](https://www.linkedin.com/in/shonesadler/), [Anil Malkani](https://www.linkedin.com/in/anil-malkani-52861a/) +**Author**: [Andrei Ionescu](https://www.linkedin.com/in/andreiionescu), [Shone Sadler](https://www.linkedin.com/in/shonesadler/), [Anil Malkani](https://www.linkedin.com/in/anil-malkani-52861a/) ### [Optimizing data warehouse storage](https://netflixtechblog.com/optimizing-data-warehouse-storage-7b94a48fdcbe) **Date**: Dec 21st, 2020, **Company**: Netflix diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index de3bcf958c67..f65b9d6c3941 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -376,7 +376,7 @@ The last step is to update the `main` branch in `iceberg-docs` to set the latest A PR needs to be published in the `iceberg-docs` repository with the following changes: 1. Update variable `latestVersions.iceberg` to the new release version in `landing-page/config.toml` 2. Update variable `latestVersions.iceberg` to the new release version and -`versions.nessie` to the version of `org.projectnessie.nessie:*` from [versions.props](https://github.com/apache/iceberg/blob/master/versions.props) in `docs/config.toml` +`versions.nessie` to the version of `org.projectnessie.nessie:*` from [mkdocs.yml](https://github.com/apache/iceberg/blob/main/site/mkdocs.yml) in `docs/config.toml` 3. Update list `versions` with the new release in `landing-page/config.toml` 4. Update list `versions` with the new release in `docs/config.toml` 5. Mark the current latest release notes to past releases under `landing-page/content/common/release-notes.md` diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index b0667361ef5c..ce4de4bdc1b8 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -59,6 +59,8 @@ Each engine version undergoes the following lifecycle stages: ### Apache Spark + + | Version | Lifecycle Stage | Initial Iceberg Support | Latest Iceberg Support | Latest Runtime Jar | | ---------- | ------------------ | ----------------------- |------------------------| ------------------ | | 2.4 | End of Life | 0.7.0-incubating | 1.2.1 | [iceberg-spark-runtime-2.4](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-2.4/1.2.1/iceberg-spark-runtime-2.4-1.2.1.jar) | @@ -69,6 +71,8 @@ Each engine version undergoes the following lifecycle stages: | 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) | + + * [1] Spark 3.1 shares the same runtime jar `iceberg-spark3-runtime` with Spark 3.0 before Iceberg 0.13.0 ### Apache Flink @@ -76,6 +80,8 @@ Each engine version undergoes the following lifecycle stages: Based on the guideline of the Flink community, only the latest 2 minor versions are actively maintained. Users should continuously upgrade their Flink version to stay up-to-date. + + | Version | Lifecycle Stage | Initial Iceberg Support | Latest Iceberg Support | Latest Runtime Jar | | ------- | --------------- | ----------------------- |------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| | 1.11 | End of Life | 0.9.0 | 0.12.1 | [iceberg-flink-runtime](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime/0.12.1/iceberg-flink-runtime-0.12.1.jar) | @@ -87,15 +93,21 @@ Users should continuously upgrade their Flink version to stay up-to-date. | 1.17 | Maintained | 1.3.0 | {{ icebergVersion }} | [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.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) | + + * [3] Flink 1.12 shares the same runtime jar `iceberg-flink-runtime` with Flink 1.11 before Iceberg 0.13.0 ### Apache Hive + + | 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) | | 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) | + + ## Developer Guide ### Maintaining existing engine versions diff --git a/site/docs/releases.md b/site/docs/releases.md index 89c963f25a0b..09b7f439385b 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -20,6 +20,8 @@ title: "Releases" ## Downloads + + The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apache/iceberg/releases/tag/apache-iceberg-{{ icebergVersion }}). * [{{ icebergVersion }} source tar.gz](https://www.apache.org/dyn/closer.cgi/iceberg/apache-iceberg-{{ icebergVersion }}/apache-iceberg-{{ icebergVersion }}.tar.gz) -- [signature](https://downloads.apache.org/iceberg/apache-iceberg-{{ icebergVersion }}/apache-iceberg-{{ icebergVersion }}.tar.gz.asc) -- [sha512](https://downloads.apache.org/iceberg/apache-iceberg-{{ icebergVersion }}/apache-iceberg-{{ icebergVersion }}.tar.gz.sha512) @@ -34,6 +36,8 @@ The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apach * [{{ icebergVersion }} gcp-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-gcp-bundle/{{ icebergVersion }}/iceberg-gcp-bundle-{{ icebergVersion }}.jar) * [{{ icebergVersion }} azure-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-azure-bundle/{{ icebergVersion }}/iceberg-azure-bundle-{{ icebergVersion }}.jar) + + To use Iceberg in Spark or Flink, download the runtime JAR for your engine version and add it to the jars folder of your installation. To use Iceberg in Hive 2 or Hive 3, download the Hive runtime JAR and add it to Hive using `ADD JAR`. @@ -970,6 +974,3 @@ A more exhaustive list of changes is available under the [0.10.0 release milesto ### 0.7.0 * Git tag: [apache-iceberg-0.7.0-incubating](https://github.com/apache/iceberg/releases/tag/apache-iceberg-0.7.0-incubating) -* [0.7.0-incubating source tar.gz](https://www.apache.org/dyn/closer.cgi/incubator/iceberg/apache-iceberg-0.7.0-incubating/apache-iceberg-0.7.0-incubating.tar.gz) -- [signature](https://dist.apache.org/repos/dist/release/incubator/iceberg/apache-iceberg-0.7.0-incubating/apache-iceberg-0.7.0-incubating.tar.gz.asc) -- [sha512](https://dist.apache.org/repos/dist/release/incubator/iceberg/apache-iceberg-0.7.0-incubating/apache-iceberg-0.7.0-incubating.tar.gz.sha512) -* [0.7.0-incubating Spark 2.4 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime/0.7.0-incubating/iceberg-spark-runtime-0.7.0-incubating.jar) - diff --git a/site/docs/spark-quickstart.md b/site/docs/spark-quickstart.md index 9601bcbdb0f8..5a940009f9a3 100644 --- a/site/docs/spark-quickstart.md +++ b/site/docs/spark-quickstart.md @@ -335,6 +335,7 @@ If you already have a Spark environment, you can add Iceberg, using the `--packa If you want to include Iceberg in your Spark installation, add the Iceberg Spark runtime to Spark's `jars` folder. You can download the runtime by visiting to the [Releases](releases.md) page. + [spark-runtime-jar]: 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 #### Learn More diff --git a/site/docs/vendors.md b/site/docs/vendors.md index 7609dcdf1903..d549219e5c10 100644 --- a/site/docs/vendors.md +++ b/site/docs/vendors.md @@ -58,10 +58,13 @@ IOMETE is a fully-managed ready to use, batteries included Data Platform. IOMETE PuppyGraph is a cloud-native graph analytics engine that enables users to query one or more relational data stores as a unified graph model. This eliminates the overhead of deploying and maintaining a siloed graph database system, with no ETL required. [PuppyGraph’s native Apache Iceberg integration](https://docs.puppygraph.com/user-manual/getting-started/iceberg) adds native graph capabilities to your existing data lake in an easy and performant way. -### [Snowflake](http://snowflake.com/) -[Snowflake](https://www.snowflake.com/en/) is a single, cross-cloud platform that enables every organization to mobilize their data with Snowflake’s Data Cloud. Snowflake supports Apache Iceberg by offering [Snowflake-managed Iceberg Tables](https://docs.snowflake.com/en/user-guide/tables-iceberg#use-snowflake-as-the-iceberg-catalog) for full DML as well as [externally managed Iceberg Tables with catalog integrations](https://docs.snowflake.com/en/user-guide/tables-iceberg#use-a-catalog-integration) for read-only access. + +### [Snowflake](https://snowflake.com/) -### [Starburst](http://starburst.io) + +[Snowflake](https://www.snowflake.com/) is a single, cross-cloud platform that enables every organization to mobilize their data with Snowflake’s Data Cloud. Snowflake supports Apache Iceberg by offering [Snowflake-managed Iceberg Tables](https://docs.snowflake.com/en/user-guide/tables-iceberg#use-snowflake-as-the-iceberg-catalog) for full DML as well as [externally managed Iceberg Tables with catalog integrations](https://docs.snowflake.com/en/user-guide/tables-iceberg#use-a-catalog-integration) for read-only access. + +### [Starburst](https://starburst.io) Starburst is a commercial offering for the [Trino query engine](https://trino.io). Trino is a distributed MPP SQL query engine that can query data in Iceberg at interactive speeds. Trino also enables you to join Iceberg tables with an [array of other systems](https://trino.io/docs/current/connector.html). Starburst offers both an [enterprise deployment](https://www.starburst.io/platform/starburst-enterprise/) and a [fully managed service](https://www.starburst.io/platform/starburst-galaxy/) to make managing and scaling Trino a flawless experience. Starburst also provides customer support and houses many of the original contributors to the open-source project that know Trino best. Learn more about [the Starburst Iceberg connector](https://docs.starburst.io/latest/connector/iceberg.html). diff --git a/site/link-checker-config.json b/site/link-checker-config.json new file mode 100644 index 000000000000..8eed0c163404 --- /dev/null +++ b/site/link-checker-config.json @@ -0,0 +1,23 @@ +{ + "ignorePatterns": [ + { + "pattern": "^https://www.linkedin.com/" + }, + { + "pattern": "^https://mvnrepository.com/" + }, + { + "pattern": "^../../javadoc" + } + ], + "replacementPatterns": [ + { + "pattern": "^docs/latest/", + "replacement": "{{BASEURL}}/docs/docs/" + }, + { + "pattern": "^../../", + "replacement": "{{BASEURL}}/site/docs/" + } + ] +} \ No newline at end of file From 568d94ca00e980990aa384a7a471f732ee0f7e29 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 24 Mar 2024 06:14:58 +0100 Subject: [PATCH 0159/1019] Build: Bump mkdocs-material from 9.5.14 to 9.5.15 (#10031) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.14 to 9.5.15. - [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.14...9.5.15) --- 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 e1efc7cbb2f9..e170b507fb44 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.14 +mkdocs-material==9.5.15 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 f4e840051cac9526736a0b12de2fe54fdf6f78b5 Mon Sep 17 00:00:00 2001 From: Alex Merced Date: Mon, 25 Mar 2024 04:41:45 -0400 Subject: [PATCH 0160/1019] Docs: Fix link to blog post (#10028) Co-authored-by: Fokko Driesprong --- site/docs/blogs.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 1714ce50405d..cf4e3254981b 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -44,7 +44,7 @@ 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/) -### [What is the Data Lakehouse and the Role of Apache Iceberg, Nessie and Dremio?](https://amdatalakehouse.substack.com/p/the-apache-iceberg-lakehouse-the) +### [What is the Data Lakehouse and the Role of Apache Iceberg, Nessie and Dremio?](https://amdatalakehouse.substack.com/p/what-is-the-data-lakehouse-and-the) **Date**: February 21st, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) From db15e80257e375aa63683532a7c5f5b852a889ef Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 26 Mar 2024 00:23:30 +0900 Subject: [PATCH 0161/1019] Core: Migrate tests to JUnit5 (#10027) --- .../java/org/apache/iceberg/TestBase.java | 9 + .../org/apache/iceberg/TestMetricsModes.java | 144 ++--- .../apache/iceberg/TestMicroBatchBuilder.java | 105 ++-- .../apache/iceberg/TestRemoveSnapshots.java | 586 ++++++++---------- .../apache/iceberg/TestRewriteManifests.java | 319 +++++----- .../java/org/apache/iceberg/TestRowDelta.java | 439 +++++-------- .../iceberg/TestSchemaAndMappingUpdate.java | 14 +- .../org/apache/iceberg/TestSchemaUpdate.java | 22 +- .../iceberg/TestSequenceNumberForV2Table.java | 45 +- .../apache/iceberg/TestSingleValueParser.java | 50 +- .../org/apache/iceberg/TestSortOrder.java | 247 ++++---- .../apache/iceberg/TestSortOrderParser.java | 39 +- .../org/apache/iceberg/TestSplitPlanning.java | 110 ++-- .../iceberg/TestV1ToV2RowDeltaDelete.java | 161 +++-- .../org/apache/iceberg/V2TableTestBase.java | 10 +- 15 files changed, 1029 insertions(+), 1271 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 10aa57abf6f3..dc7bf0e8d8cb 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -211,6 +211,15 @@ List listManifestFiles(File tableDirToList) { && Files.getFileExtension(name).equalsIgnoreCase("avro"))); } + List listManifestLists(File tableDirToList) { + return Lists.newArrayList( + new File(tableDirToList, "metadata") + .listFiles( + (dir, name) -> + name.startsWith("snap") + && Files.getFileExtension(name).equalsIgnoreCase("avro"))); + } + public static long countAllMetadataFiles(File tableDir) { return Arrays.stream(new File(tableDir, "metadata").listFiles()) .filter(f -> f.isFile()) diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java index 564eb03ec04b..31ae459df506 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java @@ -19,9 +19,15 @@ package org.apache.iceberg; 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; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; import java.util.Map; import org.apache.iceberg.MetricsModes.Counts; import org.apache.iceberg.MetricsModes.Full; @@ -29,56 +35,48 @@ import org.apache.iceberg.MetricsModes.Truncate; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestMetricsModes { +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; - private final int formatVersion; +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetricsModes { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } + @Parameter private int formatVersion; - public TestMetricsModes(int formatVersion) { - this.formatVersion = formatVersion; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; - @After + @AfterEach public void after() { TestTables.clearTables(); } - @Test + @TestTemplate public void testMetricsModeParsing() { - Assert.assertEquals(None.get(), MetricsModes.fromString("none")); - Assert.assertEquals(None.get(), MetricsModes.fromString("nOnE")); - Assert.assertEquals(Counts.get(), MetricsModes.fromString("counts")); - Assert.assertEquals(Counts.get(), MetricsModes.fromString("coUntS")); - Assert.assertEquals(Truncate.withLength(1), MetricsModes.fromString("truncate(1)")); - Assert.assertEquals(Truncate.withLength(10), MetricsModes.fromString("truNcAte(10)")); - Assert.assertEquals(Full.get(), MetricsModes.fromString("full")); - Assert.assertEquals(Full.get(), MetricsModes.fromString("FULL")); + assertThat(MetricsModes.fromString("none")).isEqualTo(None.get()); + assertThat(MetricsModes.fromString("nOnE")).isEqualTo(None.get()); + assertThat(MetricsModes.fromString("counts")).isEqualTo(Counts.get()); + assertThat(MetricsModes.fromString("coUntS")).isEqualTo(Counts.get()); + assertThat(MetricsModes.fromString("truncate(1)")).isEqualTo(Truncate.withLength(1)); + assertThat(MetricsModes.fromString("truNcAte(10)")).isEqualTo(Truncate.withLength(10)); + assertThat(MetricsModes.fromString("full")).isEqualTo(Full.get()); + assertThat(MetricsModes.fromString("FULL")).isEqualTo(Full.get()); } - @Test + @TestTemplate public void testInvalidTruncationLength() { - Assertions.assertThatThrownBy(() -> MetricsModes.fromString("truncate(0)")) + assertThatThrownBy(() -> MetricsModes.fromString("truncate(0)")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Truncate length should be positive"); } - @Test + @TestTemplate public void testInvalidColumnModeValue() { Map properties = ImmutableMap.of( @@ -88,13 +86,12 @@ public void testInvalidColumnModeValue() { "troncate(5)"); MetricsConfig config = MetricsConfig.fromProperties(properties); - Assert.assertEquals( - "Invalid mode should be defaulted to table default (full)", - MetricsModes.Full.get(), - config.columnMode("col")); + assertThat(config.columnMode("col")) + .as("Invalid mode should be defaulted to table default (full)") + .isEqualTo(MetricsModes.Full.get()); } - @Test + @TestTemplate public void testInvalidDefaultColumnModeValue() { Map properties = ImmutableMap.of( @@ -104,16 +101,15 @@ public void testInvalidDefaultColumnModeValue() { "troncate(5)"); MetricsConfig config = MetricsConfig.fromProperties(properties); - Assert.assertEquals( - "Invalid mode should be defaulted to library default (truncate(16))", - MetricsModes.Truncate.withLength(16), - config.columnMode("col")); + assertThat(config.columnMode("col")) + .as("Invalid mode should be defaulted to library default (truncate(16))") + .isEqualTo(MetricsModes.Truncate.withLength(16)); } - @Test + @TestTemplate public void testMetricsConfigSortedColsDefault() throws Exception { - File tableDir = temp.newFolder(); - tableDir.delete(); // created by table create + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Schema schema = new Schema( @@ -133,26 +129,24 @@ public void testMetricsConfigSortedColsDefault() throws Exception { .commit(); MetricsConfig config = MetricsConfig.forTable(testTable); - Assert.assertEquals( - "Non-sorted existing column should not be overridden", - Counts.get(), - config.columnMode("col1")); - Assert.assertEquals( - "Sorted column defaults should not override user specified config", - None.get(), - config.columnMode("col2")); - Assert.assertEquals( - "Unspecified sorted column should use default", - Truncate.withLength(16), - config.columnMode("col3")); - Assert.assertEquals( - "Unspecified normal column should use default", Counts.get(), config.columnMode("col4")); + assertThat(config.columnMode("col1")) + .as("Non-sorted existing column should not be overridden") + .isEqualTo(Counts.get()); + assertThat(config.columnMode("col2")) + .as("Sorted column defaults should not override user specified config") + .isEqualTo(None.get()); + assertThat(config.columnMode("col3")) + .as("Unspecified sorted column should use default") + .isEqualTo(Truncate.withLength(16)); + assertThat(config.columnMode("col4")) + .as("Unspecified normal column should use default") + .isEqualTo(Counts.get()); } - @Test + @TestTemplate public void testMetricsConfigSortedColsDefaultByInvalid() throws Exception { - File tableDir = temp.newFolder(); - tableDir.delete(); // created by table create + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Schema schema = new Schema( @@ -171,17 +165,15 @@ public void testMetricsConfigSortedColsDefaultByInvalid() throws Exception { .commit(); MetricsConfig config = MetricsConfig.forTable(testTable); - Assert.assertEquals( - "Non-sorted existing column should not be overridden by sorted column", - Full.get(), - config.columnMode("col1")); - Assert.assertEquals( - "Original default applies as user entered invalid mode for sorted column", - Counts.get(), - config.columnMode("col2")); + assertThat(config.columnMode("col1")) + .as("Non-sorted existing column should not be overridden by sorted column") + .isEqualTo(Full.get()); + assertThat(config.columnMode("col2")) + .as("Original default applies as user entered invalid mode for sorted column") + .isEqualTo(Counts.get()); } - @Test + @TestTemplate public void testMetricsConfigInferredDefaultModeLimit() throws IOException { Schema schema = new Schema( @@ -189,8 +181,8 @@ public void testMetricsConfigInferredDefaultModeLimit() throws IOException { required(2, "col2", Types.IntegerType.get()), required(3, "col3", Types.IntegerType.get())); - File tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); Table table = TestTables.create( @@ -209,10 +201,8 @@ public void testMetricsConfigInferredDefaultModeLimit() throws IOException { MetricsConfig config = MetricsConfig.forTable(table); - Assert.assertEquals( - "Should use default mode for col1", Truncate.withLength(16), config.columnMode("col1")); - Assert.assertEquals( - "Should use default mode for col2", Truncate.withLength(16), config.columnMode("col2")); - Assert.assertEquals("Should use None for col3", None.get(), config.columnMode("col3")); + assertThat(config.columnMode("col1")).isEqualTo(Truncate.withLength(16)); + assertThat(config.columnMode("col2")).isEqualTo(Truncate.withLength(16)); + assertThat(config.columnMode("col3")).isEqualTo(None.get()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java index deb6e7c8ad32..733bb0bb38fd 100644 --- a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java +++ b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java @@ -18,34 +18,31 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; import java.util.Collections; import java.util.List; import org.apache.iceberg.MicroBatches.MicroBatch; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestMicroBatchBuilder extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestMicroBatchBuilder(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMicroBatchBuilder extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Before + @BeforeEach public void setupTableProperties() { table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "3").commit(); } - @Test + @TestTemplate public void testGenerateMicroBatch() { add(table.newAppend(), files("A", "B", "C", "D", "E")); @@ -53,42 +50,42 @@ public void testGenerateMicroBatch() { MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(0, 6, Long.MAX_VALUE, true); - Assert.assertEquals(batch.snapshotId(), 1L); - Assert.assertEquals(batch.startFileIndex(), 0); - Assert.assertEquals(batch.endFileIndex(), 5); - Assert.assertEquals(batch.sizeInBytes(), 50); - Assert.assertTrue(batch.lastIndexOfSnapshot()); + assertThat(batch.snapshotId()).isEqualTo(1L); + assertThat(batch.startFileIndex()).isEqualTo(0); + assertThat(batch.endFileIndex()).isEqualTo(5); + assertThat(batch.sizeInBytes()).isEqualTo(50); + assertThat(batch.lastIndexOfSnapshot()).isTrue(); filesMatch(Lists.newArrayList("A", "B", "C", "D", "E"), filesToScan(batch.tasks())); MicroBatch batch1 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(0, 1, 15L, true); - Assert.assertEquals(batch1.endFileIndex(), 1); - Assert.assertEquals(batch1.sizeInBytes(), 10); - Assert.assertFalse(batch1.lastIndexOfSnapshot()); + assertThat(batch1.endFileIndex()).isEqualTo(1); + assertThat(batch1.sizeInBytes()).isEqualTo(10); + assertThat(batch1.lastIndexOfSnapshot()).isFalse(); filesMatch(Lists.newArrayList("A"), filesToScan(batch1.tasks())); MicroBatch batch2 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch1.endFileIndex(), 4, 30L, true); - Assert.assertEquals(batch2.endFileIndex(), 4); - Assert.assertEquals(batch2.sizeInBytes(), 30); - Assert.assertFalse(batch2.lastIndexOfSnapshot()); + assertThat(batch2.endFileIndex()).isEqualTo(4); + assertThat(batch2.sizeInBytes()).isEqualTo(30); + assertThat(batch2.lastIndexOfSnapshot()).isFalse(); filesMatch(Lists.newArrayList("B", "C", "D"), filesToScan(batch2.tasks())); MicroBatch batch3 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch2.endFileIndex(), 5, 50L, true); - Assert.assertEquals(batch3.endFileIndex(), 5); - Assert.assertEquals(batch3.sizeInBytes(), 10); - Assert.assertTrue(batch3.lastIndexOfSnapshot()); + assertThat(batch3.endFileIndex()).isEqualTo(5); + assertThat(batch3.sizeInBytes()).isEqualTo(10); + assertThat(batch3.lastIndexOfSnapshot()).isTrue(); filesMatch(Lists.newArrayList("E"), filesToScan(batch3.tasks())); } - @Test + @TestTemplate public void testGenerateMicroBatchWithSmallTargetSize() { add(table.newAppend(), files("A", "B", "C", "D", "E")); @@ -96,57 +93,57 @@ public void testGenerateMicroBatchWithSmallTargetSize() { MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(0, 1, 10L, true); - Assert.assertEquals(batch.snapshotId(), 1L); - Assert.assertEquals(batch.startFileIndex(), 0); - Assert.assertEquals(batch.endFileIndex(), 1); - Assert.assertEquals(batch.sizeInBytes(), 10); - Assert.assertFalse(batch.lastIndexOfSnapshot()); + assertThat(batch.snapshotId()).isEqualTo(1L); + assertThat(batch.startFileIndex()).isEqualTo(0); + assertThat(batch.endFileIndex()).isEqualTo(1); + assertThat(batch.sizeInBytes()).isEqualTo(10); + assertThat(batch.lastIndexOfSnapshot()).isFalse(); filesMatch(Lists.newArrayList("A"), filesToScan(batch.tasks())); MicroBatch batch1 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch.endFileIndex(), 2, 5L, true); - Assert.assertEquals(batch1.endFileIndex(), 2); - Assert.assertEquals(batch1.sizeInBytes(), 10); + assertThat(batch1.endFileIndex()).isEqualTo(2); + assertThat(batch1.sizeInBytes()).isEqualTo(10); filesMatch(Lists.newArrayList("B"), filesToScan(batch1.tasks())); - Assert.assertFalse(batch1.lastIndexOfSnapshot()); + assertThat(batch1.lastIndexOfSnapshot()).isFalse(); MicroBatch batch2 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch1.endFileIndex(), 3, 10L, true); - Assert.assertEquals(batch2.endFileIndex(), 3); - Assert.assertEquals(batch2.sizeInBytes(), 10); + assertThat(batch2.endFileIndex()).isEqualTo(3); + assertThat(batch2.sizeInBytes()).isEqualTo(10); filesMatch(Lists.newArrayList("C"), filesToScan(batch2.tasks())); - Assert.assertFalse(batch2.lastIndexOfSnapshot()); + assertThat(batch2.lastIndexOfSnapshot()).isFalse(); MicroBatch batch3 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch2.endFileIndex(), 4, 10L, true); - Assert.assertEquals(batch3.endFileIndex(), 4); - Assert.assertEquals(batch3.sizeInBytes(), 10); + assertThat(batch3.endFileIndex()).isEqualTo(4); + assertThat(batch3.sizeInBytes()).isEqualTo(10); filesMatch(Lists.newArrayList("D"), filesToScan(batch3.tasks())); - Assert.assertFalse(batch3.lastIndexOfSnapshot()); + assertThat(batch3.lastIndexOfSnapshot()).isFalse(); MicroBatch batch4 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch3.endFileIndex(), 5, 5L, true); - Assert.assertEquals(batch4.endFileIndex(), 5); - Assert.assertEquals(batch4.sizeInBytes(), 10); + assertThat(batch4.endFileIndex()).isEqualTo(5); + assertThat(batch4.sizeInBytes()).isEqualTo(10); filesMatch(Lists.newArrayList("E"), filesToScan(batch4.tasks())); - Assert.assertTrue(batch4.lastIndexOfSnapshot()); + assertThat(batch4.lastIndexOfSnapshot()).isTrue(); MicroBatch batch5 = MicroBatches.from(table.snapshot(1L), table.io()) .specsById(table.specs()) .generate(batch4.endFileIndex(), 5, 5L, true); - Assert.assertEquals(batch5.endFileIndex(), 5); - Assert.assertEquals(batch5.sizeInBytes(), 0); - Assert.assertTrue(Iterables.isEmpty(batch5.tasks())); - Assert.assertTrue(batch5.lastIndexOfSnapshot()); + assertThat(batch5.endFileIndex()).isEqualTo(5); + assertThat(batch5.sizeInBytes()).isEqualTo(0); + assertThat(batch5.tasks()).isEmpty(); + assertThat(batch5.lastIndexOfSnapshot()).isTrue(); } private static DataFile file(String name) { @@ -190,6 +187,6 @@ private static List filesToScan(Iterable tasks) { private static void filesMatch(List expected, List actual) { Collections.sort(expected); Collections.sort(actual); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 11e2daca6ab0..6a47a24fb494 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -18,11 +18,16 @@ */ package org.apache.iceberg; +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.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; +import java.util.Arrays; import java.util.List; import java.util.Set; import java.util.UUID; @@ -42,30 +47,21 @@ 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.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestRemoveSnapshots extends TableTestBase { - private final boolean incrementalCleanup; - - @Parameterized.Parameters(name = "formatVersion = {0}, incrementalCleanup = {1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {1, true}, - new Object[] {2, true}, - new Object[] {1, false}, - new Object[] {2, false} - }; - } - - public TestRemoveSnapshots(int formatVersion, boolean incrementalCleanup) { - super(formatVersion); - this.incrementalCleanup = incrementalCleanup; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRemoveSnapshots extends TestBase { + @Parameter(index = 1) + private boolean incrementalCleanup; + + @Parameters(name = "formatVersion = {0}, incrementalCleanup = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, true}, + new Object[] {2, true}, + new Object[] {1, false}, + new Object[] {2, false}); } private long waitUntilAfter(long timestampMillis) { @@ -76,7 +72,7 @@ private long waitUntilAfter(long timestampMillis) { return current; } - @Test + @TestTemplate public void testExpireOlderThan() { table.newAppend().appendFile(FILE_A).commit(); @@ -94,35 +90,26 @@ public void testExpireOlderThan() { removeSnapshots(table).expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertEquals( - "Should remove only the expired manifest list location", - Sets.newHashSet(firstSnapshot.manifestListLocation()), - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())).isNull(); + assertThat(deletedFiles).containsExactly(firstSnapshot.manifestListLocation()); } - @Test + @TestTemplate public void testExpireOlderThanWithDelete() { table.newAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).hasSize(1); waitUntilAfter(table.currentSnapshot().timestampMillis()); table.newDelete().deleteFile(FILE_A).commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create replace manifest with a rewritten manifest", - 1, - secondSnapshot.allManifests(table.io()).size()); + assertThat(secondSnapshot.allManifests(table.io())) + .as("Should create replace manifest with a rewritten manifest") + .hasSize(1); table.newAppend().appendFile(FILE_B).commit(); @@ -136,34 +123,29 @@ public void testExpireOlderThanWithDelete() { removeSnapshots(table).expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the second oldest snapshot", - table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", - Sets.newHashSet( - firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot - .allManifests(table.io()) - .get(0) - .path(), // manifest was rewritten for delete - secondSnapshot.manifestListLocation(), // snapshot expired - secondSnapshot - .allManifests(table.io()) - .get(0) - .path(), // manifest contained only deletes, was dropped - FILE_A.path()), // deleted - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())).isNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())).isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file") + .isEqualTo( + Sets.newHashSet( + firstSnapshot.manifestListLocation(), // snapshot expired + firstSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete + secondSnapshot.manifestListLocation(), // snapshot expired + secondSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest contained only deletes, was dropped + FILE_A.path() // deleted + )); } - @Test + @TestTemplate public void testExpireOlderThanWithDeleteInMergedManifests() { // merge every commit table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); @@ -171,8 +153,7 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).hasSize(1); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -182,10 +163,9 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { .commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should replace manifest with a rewritten manifest", - 1, - secondSnapshot.allManifests(table.io()).size()); + assertThat(secondSnapshot.allManifests(table.io())) + .as("Should replace manifest with a rewritten manifest") + .hasSize(1); table .newFastAppend() // do not merge to keep the last snapshot's manifest valid @@ -202,30 +182,25 @@ public void testExpireOlderThanWithDeleteInMergedManifests() { removeSnapshots(table).expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNull( - "Expire should remove the oldest snapshot", table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the second oldest snapshot", - table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and deleted data file", - Sets.newHashSet( - firstSnapshot.manifestListLocation(), // snapshot expired - firstSnapshot - .allManifests(table.io()) - .get(0) - .path(), // manifest was rewritten for delete - secondSnapshot.manifestListLocation(), // snapshot expired - FILE_A.path()), // deleted - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())).isNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())).isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and deleted data file") + .isEqualTo( + Sets.newHashSet( + firstSnapshot.manifestListLocation(), // snapshot expired + firstSnapshot + .allManifests(table.io()) + .get(0) + .path(), // manifest was rewritten for delete + secondSnapshot.manifestListLocation(), // snapshot expired + FILE_A.path() // deleted + )); } - @Test + @TestTemplate public void testExpireOlderThanWithRollback() { // merge every commit table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "0").commit(); @@ -233,8 +208,7 @@ public void testExpireOlderThanWithRollback() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).hasSize(1); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -244,8 +218,7 @@ public void testExpireOlderThanWithRollback() { Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); - Assert.assertEquals( - "Should add one new manifest for append", 1, secondSnapshotManifests.size()); + assertThat(secondSnapshotManifests).hasSize(1); table.manageSnapshots().rollbackTo(firstSnapshot.snapshotId()).commit(); @@ -257,32 +230,30 @@ public void testExpireOlderThanWithRollback() { removeSnapshots(table).expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNotNull( - "Expire should keep the oldest snapshot, current", - table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", - Sets.newHashSet( - secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path()), // manifest is no longer referenced - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should keep the oldest snapshot, current") + .isNotNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the orphaned snapshot") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and reverted appended data file") + .isEqualTo( + Sets.newHashSet( + secondSnapshot.manifestListLocation(), // snapshot expired + Iterables.getOnlyElement(secondSnapshotManifests) + .path()) // manifest is no longer referenced + ); } - @Test + @TestTemplate public void testExpireOlderThanWithRollbackAndMergedManifests() { table.newAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should create one manifest", 1, firstSnapshot.allManifests(table.io()).size()); + assertThat(firstSnapshot.allManifests(table.io())).hasSize(1); waitUntilAfter(table.currentSnapshot().timestampMillis()); @@ -292,8 +263,7 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { Set secondSnapshotManifests = Sets.newHashSet(secondSnapshot.allManifests(table.io())); secondSnapshotManifests.removeAll(firstSnapshot.allManifests(table.io())); - Assert.assertEquals( - "Should add one new manifest for append", 1, secondSnapshotManifests.size()); + assertThat(secondSnapshotManifests).hasSize(1); table.manageSnapshots().rollbackTo(firstSnapshot.snapshotId()).commit(); @@ -305,27 +275,28 @@ public void testExpireOlderThanWithRollbackAndMergedManifests() { removeSnapshots(table).expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Expire should not change current snapshot", - snapshotId, - table.currentSnapshot().snapshotId()); - Assert.assertNotNull( - "Expire should keep the oldest snapshot, current", - table.snapshot(firstSnapshot.snapshotId())); - Assert.assertNull( - "Expire should remove the orphaned snapshot", table.snapshot(secondSnapshot.snapshotId())); - - Assert.assertEquals( - "Should remove expired manifest lists and reverted appended data file", - Sets.newHashSet( - secondSnapshot.manifestListLocation(), // snapshot expired - Iterables.getOnlyElement(secondSnapshotManifests) - .path(), // manifest is no longer referenced - FILE_B.path()), // added, but rolled back - deletedFiles); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(snapshotId); + assertThat(table.snapshot(firstSnapshot.snapshotId())) + .as("Expire should keep the oldest snapshot, current") + .isNotNull(); + assertThat(table.snapshot(secondSnapshot.snapshotId())) + .as("Expire should remove the orphaned snapshot") + .isNull(); + + assertThat(deletedFiles) + .as("Should remove expired manifest lists and reverted appended data file") + .isEqualTo( + Sets.newHashSet( + secondSnapshot.manifestListLocation(), // snapshot expired + secondSnapshotManifests.stream() + .findFirst() + .get() + .path(), // manifest is no longer referenced + FILE_B.path()) // added, but rolled back + ); } - @Test + @TestTemplate public void testRetainLastWithExpireOlderThan() { long t0 = System.currentTimeMillis(); table @@ -361,13 +332,11 @@ public void testRetainLastWithExpireOlderThan() { // Retain last 2 snapshots removeSnapshots(table).expireOlderThan(t3).retainLast(2).commit(); - Assert.assertEquals( - "Should have two snapshots.", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); + assertThat(table.snapshots()).hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).isNull(); } - @Test + @TestTemplate public void testRetainLastWithExpireById() { long t0 = System.currentTimeMillis(); table @@ -403,13 +372,11 @@ public void testRetainLastWithExpireById() { // Retain last 3 snapshots, but explicitly remove the first snapshot removeSnapshots(table).expireSnapshotId(firstSnapshotId).retainLast(3).commit(); - Assert.assertEquals( - "Should have two snapshots.", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); + assertThat(table.snapshots()).hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).isNull(); } - @Test + @TestTemplate public void testRetainNAvailableSnapshotsWithTransaction() { long t0 = System.currentTimeMillis(); table @@ -442,23 +409,19 @@ public void testRetainNAvailableSnapshotsWithTransaction() { t3 = System.currentTimeMillis(); } - Assert.assertEquals( - "Should be 3 manifest lists", 3, listManifestLists(table.location()).size()); + assertThat(listManifestFiles(new File(table.location()))).hasSize(3); // Retain last 2 snapshots, which means 1 is deleted. Transaction tx = table.newTransaction(); removeSnapshots(tx.table()).expireOlderThan(t3).retainLast(2).commit(); tx.commitTransaction(); - Assert.assertEquals( - "Should have two snapshots.", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should not present.", null, table.snapshot(firstSnapshotId)); - Assert.assertEquals( - "Should be 2 manifest lists", 2, listManifestLists(table.location()).size()); + assertThat(table.snapshots()).hasSize(2); + assertThat(table.snapshot(firstSnapshotId)).isNull(); + assertThat(listManifestLists(new File(table.location()))).hasSize(2); } - @Test + @TestTemplate public void testRetainLastWithTooFewSnapshots() { long t0 = System.currentTimeMillis(); table @@ -486,15 +449,11 @@ public void testRetainLastWithTooFewSnapshots() { // Retain last 3 snapshots removeSnapshots(table).expireOlderThan(t2).retainLast(3).commit(); - Assert.assertEquals( - "Should have two snapshots", 2, Lists.newArrayList(table.snapshots()).size()); - Assert.assertEquals( - "First snapshot should still present", - firstSnapshotId, - table.snapshot(firstSnapshotId).snapshotId()); + assertThat(table.snapshots()).hasSize(2); + assertThat(table.snapshot(firstSnapshotId).snapshotId()).isEqualTo(firstSnapshotId); } - @Test + @TestTemplate public void testRetainNLargerThanCurrentSnapshots() { // Append 3 files table @@ -532,11 +491,10 @@ public void testRetainNLargerThanCurrentSnapshots() { removeSnapshots(tx.table()).expireOlderThan(t3).retainLast(4).commit(); tx.commitTransaction(); - Assert.assertEquals( - "Should have three snapshots.", 3, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(3); } - @Test + @TestTemplate public void testRetainLastKeepsExpiringSnapshot() { long t0 = System.currentTimeMillis(); table @@ -582,13 +540,11 @@ public void testRetainLastKeepsExpiringSnapshot() { // Retain last 2 snapshots and expire older than t3 removeSnapshots(table).expireOlderThan(secondSnapshot.timestampMillis()).retainLast(2).commit(); - Assert.assertEquals( - "Should have three snapshots.", 3, Lists.newArrayList(table.snapshots()).size()); - Assert.assertNotNull( - "Second snapshot should present.", table.snapshot(secondSnapshot.snapshotId())); + assertThat(table.snapshots()).hasSize(3); + assertThat(table.snapshot(secondSnapshot.snapshotId())).isNotNull(); } - @Test + @TestTemplate public void testExpireOlderThanMultipleCalls() { long t0 = System.currentTimeMillis(); table @@ -628,13 +584,11 @@ public void testExpireOlderThanMultipleCalls() { .expireOlderThan(thirdSnapshot.timestampMillis()) .commit(); - Assert.assertEquals( - "Should have one snapshots.", 1, Lists.newArrayList(table.snapshots()).size()); - Assert.assertNull( - "Second snapshot should not present.", table.snapshot(secondSnapshot.snapshotId())); + assertThat(table.snapshots()).hasSize(1); + assertThat(table.snapshot(secondSnapshot.snapshotId())).isNull(); } - @Test + @TestTemplate public void testRetainLastMultipleCalls() { long t0 = System.currentTimeMillis(); table @@ -670,20 +624,18 @@ public void testRetainLastMultipleCalls() { // Retain last 2 snapshots and expire older than t3 removeSnapshots(table).expireOlderThan(t3).retainLast(2).retainLast(1).commit(); - Assert.assertEquals( - "Should have one snapshots.", 1, Lists.newArrayList(table.snapshots()).size()); - Assert.assertNull( - "Second snapshot should not present.", table.snapshot(secondSnapshot.snapshotId())); + assertThat(table.snapshots()).hasSize(1); + assertThat(table.snapshot(secondSnapshot.snapshotId())).isNull(); } - @Test + @TestTemplate public void testRetainZeroSnapshots() { - Assertions.assertThatThrownBy(() -> removeSnapshots(table).retainLast(0).commit()) + assertThatThrownBy(() -> removeSnapshots(table).retainLast(0).commit()) .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(); @@ -700,10 +652,10 @@ public void testScanExpiredManifestInValidSnapshotAppend() { removeSnapshots(table).expireOlderThan(t3).deleteWith(deletedFiles::add).commit(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + assertThat(deletedFiles).contains(FILE_A.path().toString()); } - @Test + @TestTemplate public void testScanExpiredManifestInValidSnapshotFastAppend() { table .updateProperties() @@ -726,10 +678,10 @@ public void testScanExpiredManifestInValidSnapshotFastAppend() { removeSnapshots(table).expireOlderThan(t3).deleteWith(deletedFiles::add).commit(); - Assert.assertTrue("FILE_A should be deleted", deletedFiles.contains(FILE_A.path().toString())); + assertThat(deletedFiles).contains(FILE_A.path().toString()); } - @Test + @TestTemplate public void dataFilesCleanup() throws IOException { table.newFastAppend().appendFile(FILE_A).commit(); @@ -763,11 +715,11 @@ public void dataFilesCleanup() throws IOException { removeSnapshots(table).expireOlderThan(t4).deleteWith(deletedFiles::add).commit(); - 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())); + assertThat(deletedFiles).contains(FILE_A.path().toString()); + assertThat(deletedFiles).contains(FILE_B.path().toString()); } - @Test + @TestTemplate public void dataFilesCleanupWithParallelTasks() throws IOException { table.newFastAppend().appendFile(FILE_A).commit(); @@ -833,17 +785,18 @@ public void dataFilesCleanupWithParallelTasks() throws IOException { // Verifies that the delete methods ran in the threads created by the provided ExecutorService // ThreadFactory - Assert.assertEquals( - deleteThreads, - 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("Thread should be created in provided pool", planThreadsIndex.get() > 0); + assertThat(deleteThreads) + .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(planThreadsIndex.get()) + .as("Thread should be created in provided pool") + .isGreaterThan(0); } - @Test + @TestTemplate public void noDataFileCleanup() throws IOException { table.newFastAppend().appendFile(FILE_A).commit(); @@ -866,14 +819,14 @@ public void noDataFileCleanup() throws IOException { .deleteWith(deletedFiles::add) .commit(); - Assert.assertTrue("No files should have been deleted", deletedFiles.isEmpty()); + assertThat(deletedFiles).isEmpty(); } /** * 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(); @@ -918,18 +871,17 @@ public void testWithExpiringDanglingStageCommit() { expectedDeletes.add(file.path()); } }); - Assert.assertSame( - "Files deleted count should be expected", expectedDeletes.size(), deletedFiles.size()); + assertThat(deletedFiles).isEqualTo(expectedDeletes); // Take the diff expectedDeletes.removeAll(deletedFiles); - Assert.assertTrue("Exactly same files should be deleted", expectedDeletes.isEmpty()); + assertThat(expectedDeletes).isEmpty(); } /** * 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(); @@ -938,7 +890,7 @@ public void testWithCherryPickTableSnapshot() { // `B` commit Set deletedAFiles = Sets.newHashSet(); table.newOverwrite().addFile(FILE_B).deleteFile(FILE_A).deleteWith(deletedAFiles::add).commit(); - Assert.assertTrue("No files should be physically deleted", deletedAFiles.isEmpty()); + assertThat(deletedAFiles).isEmpty(); // pick the snapshot 'B` Snapshot snapshotB = readMetadata().currentSnapshot(); @@ -971,7 +923,7 @@ public void testWithCherryPickTableSnapshot() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + assertThat(deletedFiles).doesNotContain(item.path().toString()); }); }); } @@ -980,7 +932,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(); @@ -1016,7 +968,7 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + assertThat(deletedFiles).doesNotContain(item.path().toString()); }); }); @@ -1033,23 +985,23 @@ public void testWithExpiringStagedThenCherrypick() { i.addedDataFiles(table.io()) .forEach( item -> { - Assert.assertFalse(deletedFiles.contains(item.path().toString())); + assertThat(deletedFiles).doesNotContain(item.path().toString()); }); }); } - @Test + @TestTemplate public void testExpireSnapshotsWhenGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); table.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThatThrownBy(() -> table.expireSnapshots()) + assertThatThrownBy(() -> table.expireSnapshots()) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } - @Test + @TestTemplate public void testExpireWithDefaultRetainLast() { table.newAppend().appendFile(FILE_A).commit(); @@ -1057,7 +1009,7 @@ public void testExpireWithDefaultRetainLast() { table.newAppend().appendFile(FILE_C).commit(); - Assert.assertEquals("Expected 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); table.updateProperties().set(TableProperties.MIN_SNAPSHOTS_TO_KEEP, "3").commit(); @@ -1070,13 +1022,12 @@ public void testExpireWithDefaultRetainLast() { .deleteWith(deletedFiles::add) .commit(); - Assert.assertEquals( - "Should not change current snapshot", snapshotBeforeExpiration, table.currentSnapshot()); - Assert.assertEquals("Should keep 3 snapshots", 3, Iterables.size(table.snapshots())); - Assert.assertTrue("Should not delete data", deletedFiles.isEmpty()); + assertThat(table.currentSnapshot()).isEqualTo(snapshotBeforeExpiration); + assertThat(table.snapshots()).hasSize(3); + assertThat(deletedFiles).isEmpty(); } - @Test + @TestTemplate public void testExpireWithDefaultSnapshotAge() { table.newAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); @@ -1093,7 +1044,7 @@ public void testExpireWithDefaultSnapshotAge() { waitUntilAfter(thirdSnapshot.timestampMillis()); - Assert.assertEquals("Expected 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); table.updateProperties().set(TableProperties.MAX_SNAPSHOT_AGE_MS, "1").commit(); @@ -1102,19 +1053,16 @@ public void testExpireWithDefaultSnapshotAge() { // rely solely on default configs removeSnapshots(table).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Should not change current snapshot", thirdSnapshot, table.currentSnapshot()); - Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); - Assert.assertEquals( - "Should remove expired manifest lists", - Sets.newHashSet( - firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation()), - deletedFiles); + assertThat(table.currentSnapshot()).isEqualTo(thirdSnapshot); + assertThat(table.snapshots()).hasSize(1); + assertThat(deletedFiles) + .containsExactlyInAnyOrder( + firstSnapshot.manifestListLocation(), secondSnapshot.manifestListLocation()); } - @Test + @TestTemplate public void testExpireWithDeleteFiles() { - Assume.assumeTrue("Delete files only supported in V2 spec", formatVersion == 2); + assumeThat(formatVersion).as("Delete files only supported in V2 spec").isEqualTo(2); // Data Manifest => File_A table.newAppend().appendFile(FILE_A).commit(); @@ -1124,10 +1072,8 @@ public void testExpireWithDeleteFiles() { // Delete Manifest => FILE_A_DELETES table.newRowDelta().addDeletes(FILE_A_DELETES).commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals( - "Should have 1 data manifest", 1, secondSnapshot.dataManifests(table.io()).size()); - Assert.assertEquals( - "Should have 1 delete manifest", 1, secondSnapshot.deleteManifests(table.io()).size()); + assertThat(secondSnapshot.dataManifests(table.io())).hasSize(1); + assertThat(secondSnapshot.deleteManifests(table.io())).hasSize(1); // FILE_A and FILE_A_DELETES move into "DELETED" state table @@ -1142,8 +1088,7 @@ public void testExpireWithDeleteFiles() { thirdSnapshot.allManifests(table.io()).stream() .filter(ManifestFile::hasDeletedFiles) .collect(Collectors.toSet()); - Assert.assertEquals( - "Should have two manifests of deleted files", 2, manifestOfDeletedFiles.size()); + assertThat(manifestOfDeletedFiles).hasSize(2); // Need one more commit before manifests of files of DELETED state get cleared from current // snapshot. @@ -1155,24 +1100,24 @@ public void testExpireWithDeleteFiles() { Set deletedFiles = Sets.newHashSet(); removeSnapshots(table).expireOlderThan(fourthSnapshotTs).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals( - "Should remove old delete files and delete file manifests", - ImmutableSet.builder() - .add(FILE_A.path()) - .add(FILE_A_DELETES.path()) - .add(firstSnapshot.manifestListLocation()) - .add(secondSnapshot.manifestListLocation()) - .add(thirdSnapshot.manifestListLocation()) - .addAll(manifestPaths(secondSnapshot, table.io())) - .addAll( - manifestOfDeletedFiles.stream() - .map(ManifestFile::path) - .collect(Collectors.toList())) - .build(), - deletedFiles); + assertThat(deletedFiles) + .as("Should remove old delete files and delete file manifests") + .isEqualTo( + ImmutableSet.builder() + .add(FILE_A.path()) + .add(FILE_A_DELETES.path()) + .add(firstSnapshot.manifestListLocation()) + .add(secondSnapshot.manifestListLocation()) + .add(thirdSnapshot.manifestListLocation()) + .addAll(manifestPaths(secondSnapshot, table.io())) + .addAll( + manifestOfDeletedFiles.stream() + .map(ManifestFile::path) + .collect(Collectors.toList())) + .build()); } - @Test + @TestTemplate public void testTagExpiration() { table.newAppend().appendFile(FILE_A).commit(); @@ -1194,12 +1139,12 @@ public void testTagExpiration() { removeSnapshots(table).cleanExpiredFiles(false).commit(); - Assert.assertNull(table.ops().current().ref("tag")); - Assert.assertNotNull(table.ops().current().ref("branch")); - Assert.assertNotNull(table.ops().current().ref(SnapshotRef.MAIN_BRANCH)); + assertThat(table.ops().current().ref("tag")).isNull(); + assertThat(table.ops().current().ref("branch")).isNotNull(); + assertThat(table.ops().current().ref(SnapshotRef.MAIN_BRANCH)).isNotNull(); } - @Test + @TestTemplate public void testBranchExpiration() { table.newAppend().appendFile(FILE_A).commit(); @@ -1221,12 +1166,12 @@ public void testBranchExpiration() { removeSnapshots(table).cleanExpiredFiles(false).commit(); - Assert.assertNull(table.ops().current().ref("branch")); - Assert.assertNotNull(table.ops().current().ref("tag")); - Assert.assertNotNull(table.ops().current().ref(SnapshotRef.MAIN_BRANCH)); + assertThat(table.ops().current().ref("branch")).isNull(); + assertThat(table.ops().current().ref("tag")).isNotNull(); + assertThat(table.ops().current().ref(SnapshotRef.MAIN_BRANCH)).isNotNull(); } - @Test + @TestTemplate public void testMultipleRefsAndCleanExpiredFilesFailsForIncrementalCleanup() { table.newAppend().appendFile(FILE_A).commit(); table.newDelete().deleteFile(FILE_A).commit(); @@ -1234,7 +1179,7 @@ public void testMultipleRefsAndCleanExpiredFilesFailsForIncrementalCleanup() { waitUntilAfter(table.currentSnapshot().timestampMillis()); RemoveSnapshots removeSnapshots = (RemoveSnapshots) table.expireSnapshots(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> removeSnapshots .withIncrementalCleanup(true) @@ -1245,7 +1190,7 @@ public void testMultipleRefsAndCleanExpiredFilesFailsForIncrementalCleanup() { .hasMessage("Cannot incrementally clean files for tables with more than 1 ref"); } - @Test + @TestTemplate public void testExpireWithStatisticsFiles() throws IOException { table.newAppend().appendFile(FILE_A).commit(); String statsFileLocation1 = statsFileLocation(table.location()); @@ -1266,24 +1211,24 @@ public void testExpireWithStatisticsFiles() throws IOException { statsFileLocation2, table.io()); commitStats(table, statisticsFile2); - Assert.assertEquals("Should have 2 statistics file", 2, table.statisticsFiles().size()); + assertThat(table.statisticsFiles()).hasSize(2); long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); removeSnapshots(table).expireOlderThan(tAfterCommits).commit(); // only the current snapshot and its stats file should be retained - Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); - Assertions.assertThat(table.statisticsFiles()) + assertThat(table.snapshots()).hasSize(1); + assertThat(table.statisticsFiles()) .hasSize(1) .extracting(StatisticsFile::snapshotId) .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); - Assertions.assertThat(new File(statsFileLocation1)).doesNotExist(); - Assertions.assertThat(new File(statsFileLocation2)).exists(); + assertThat(new File(statsFileLocation1)).doesNotExist(); + assertThat(new File(statsFileLocation2)).exists(); } - @Test + @TestTemplate public void testExpireWithStatisticsFilesWithReuse() throws IOException { table.newAppend().appendFile(FILE_A).commit(); String statsFileLocation1 = statsFileLocation(table.location()); @@ -1303,24 +1248,24 @@ public void testExpireWithStatisticsFilesWithReuse() throws IOException { reuseStatsFile(table.currentSnapshot().snapshotId(), statisticsFile1); commitStats(table, statisticsFile2); - Assert.assertEquals("Should have 2 statistics file", 2, table.statisticsFiles().size()); + assertThat(table.statisticsFiles()).hasSize(2); long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); removeSnapshots(table).expireOlderThan(tAfterCommits).commit(); // only the current snapshot and its stats file (reused from previous snapshot) should be // retained - Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); - Assertions.assertThat(table.statisticsFiles()) + assertThat(table.snapshots()).hasSize(1); + assertThat(table.statisticsFiles()) .hasSize(1) .extracting(StatisticsFile::snapshotId) .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); // the reused stats file should exist. - Assertions.assertThat(new File(statsFileLocation1)).exists(); + assertThat(new File(statsFileLocation1)).exists(); } - @Test + @TestTemplate public void testExpireWithPartitionStatisticsFiles() throws IOException { table.newAppend().appendFile(FILE_A).commit(); String statsFileLocation1 = statsFileLocation(table.location()); @@ -1335,25 +1280,24 @@ public void testExpireWithPartitionStatisticsFiles() throws IOException { writePartitionStatsFile( table.currentSnapshot().snapshotId(), statsFileLocation2, table.io()); commitPartitionStats(table, statisticsFile2); - Assert.assertEquals( - "Should have 2 partition statistics file", 2, table.partitionStatisticsFiles().size()); + assertThat(table.partitionStatisticsFiles()).hasSize(2); long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); removeSnapshots(table).expireOlderThan(tAfterCommits).commit(); // only the current snapshot and its stats file should be retained - Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); - Assertions.assertThat(table.partitionStatisticsFiles()) + assertThat(table.snapshots()).hasSize(1); + assertThat(table.partitionStatisticsFiles()) .hasSize(1) .extracting(PartitionStatisticsFile::snapshotId) .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); - Assertions.assertThat(new File(statsFileLocation1)).doesNotExist(); - Assertions.assertThat(new File(statsFileLocation2)).exists(); + assertThat(new File(statsFileLocation1)).doesNotExist(); + assertThat(new File(statsFileLocation2)).exists(); } - @Test + @TestTemplate public void testExpireWithPartitionStatisticsFilesWithReuse() throws IOException { table.newAppend().appendFile(FILE_A).commit(); String statsFileLocation1 = statsFileLocation(table.location()); @@ -1370,25 +1314,24 @@ public void testExpireWithPartitionStatisticsFilesWithReuse() throws IOException reusePartitionStatsFile(table.currentSnapshot().snapshotId(), statisticsFile1); commitPartitionStats(table, statisticsFile2); - Assert.assertEquals( - "Should have 2 partition statistics file", 2, table.partitionStatisticsFiles().size()); + assertThat(table.partitionStatisticsFiles()).hasSize(2); long tAfterCommits = waitUntilAfter(table.currentSnapshot().timestampMillis()); removeSnapshots(table).expireOlderThan(tAfterCommits).commit(); // only the current snapshot and its stats file (reused from previous snapshot) should be // retained - Assert.assertEquals("Should keep 1 snapshot", 1, Iterables.size(table.snapshots())); - Assertions.assertThat(table.partitionStatisticsFiles()) + assertThat(table.snapshots()).hasSize(1); + assertThat(table.partitionStatisticsFiles()) .hasSize(1) .extracting(PartitionStatisticsFile::snapshotId) .as("Should contain only the statistics file of snapshot2") .isEqualTo(Lists.newArrayList(statisticsFile2.snapshotId())); // the reused stats file should exist. - Assertions.assertThat(new File(statsFileLocation1)).exists(); + assertThat(new File(statsFileLocation1)).exists(); } - @Test + @TestTemplate public void testFailRemovingSnapshotWhenStillReferencedByBranch() { table.newAppend().appendFile(FILE_A).commit(); @@ -1400,13 +1343,12 @@ public void testFailRemovingSnapshotWhenStillReferencedByBranch() { table.manageSnapshots().createBranch("branch", snapshotId).commit(); - Assertions.assertThatThrownBy( - () -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()) + assertThatThrownBy(() -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot expire 2. Still referenced by refs: [branch]"); } - @Test + @TestTemplate public void testFailRemovingSnapshotWhenStillReferencedByTag() { table.newAppend().appendFile(FILE_A).commit(); @@ -1417,13 +1359,12 @@ public void testFailRemovingSnapshotWhenStillReferencedByTag() { // commit another snapshot so the first one isn't referenced by main table.newAppend().appendFile(FILE_B).commit(); - Assertions.assertThatThrownBy( - () -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()) + assertThatThrownBy(() -> removeSnapshots(table).expireSnapshotId(snapshotId).commit()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot expire 1. Still referenced by refs: [tag]"); } - @Test + @TestTemplate public void testRetainUnreferencedSnapshotsWithinExpirationAge() { table.newAppend().appendFile(FILE_A).commit(); @@ -1436,10 +1377,10 @@ public void testRetainUnreferencedSnapshotsWithinExpirationAge() { removeSnapshots(table).expireOlderThan(expireTimestampSnapshotA).commit(); - Assert.assertEquals(2, table.ops().current().snapshots().size()); + assertThat(table.ops().current().snapshots()).hasSize(2); } - @Test + @TestTemplate public void testUnreferencedSnapshotParentOfTag() { table.newAppend().appendFile(FILE_A).commit(); @@ -1468,12 +1409,13 @@ public void testUnreferencedSnapshotParentOfTag() { .cleanExpiredFiles(false) .commit(); - Assert.assertNull( - "Should remove unreferenced snapshot beneath a tag", table.snapshot(expiredSnapshotId)); - Assert.assertEquals(2, table.ops().current().snapshots().size()); + assertThat(table.snapshot(expiredSnapshotId)) + .as("Should remove unreferenced snapshot beneath a tag") + .isNull(); + assertThat(table.ops().current().snapshots()).hasSize(2); } - @Test + @TestTemplate public void testSnapshotParentOfBranchNotUnreferenced() { // similar to testUnreferencedSnapshotParentOfTag, but checks that branch history is not // considered unreferenced @@ -1505,11 +1447,13 @@ public void testSnapshotParentOfBranchNotUnreferenced() { .cleanExpiredFiles(false) .commit(); - Assert.assertNotNull("Should not remove snapshot beneath a branch", table.snapshot(snapshotId)); - Assert.assertEquals(3, table.ops().current().snapshots().size()); + assertThat(table.snapshot(snapshotId)) + .as("Should not remove snapshot beneath a branch") + .isNotNull(); + assertThat(table.ops().current().snapshots()).hasSize(3); } - @Test + @TestTemplate public void testMinSnapshotsToKeepMultipleBranches() { table.newAppend().appendFile(FILE_A).commit(); long initialSnapshotId = table.currentSnapshot().snapshotId(); @@ -1520,7 +1464,7 @@ public void testMinSnapshotsToKeepMultipleBranches() { long branchSnapshotId = append.apply().snapshotId(); append.commit(); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); long maxSnapshotAgeMs = 1; long expirationTime = System.currentTimeMillis() + maxSnapshotAgeMs; @@ -1543,20 +1487,18 @@ public void testMinSnapshotsToKeepMultipleBranches() { waitUntilAfter(expirationTime); table.expireSnapshots().cleanExpiredFiles(false).commit(); - Assert.assertEquals( - "Should have 3 snapshots (none removed)", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); // stop retaining snapshots from the branch table.manageSnapshots().setMinSnapshotsToKeep("branch", 1).commit(); removeSnapshots(table).cleanExpiredFiles(false).commit(); - Assert.assertEquals( - "Should have 2 snapshots (initial removed)", 2, Iterables.size(table.snapshots())); - Assert.assertNull(table.ops().current().snapshot(initialSnapshotId)); + assertThat(table.snapshots()).hasSize(2); + assertThat(table.ops().current().snapshot(initialSnapshotId)).isNull(); } - @Test + @TestTemplate public void testMaxSnapshotAgeMultipleBranches() { table.newAppend().appendFile(FILE_A).commit(); long initialSnapshotId = table.currentSnapshot().snapshotId(); @@ -1580,7 +1522,7 @@ public void testMaxSnapshotAgeMultipleBranches() { long branchSnapshotId = append.apply().snapshotId(); append.commit(); - Assert.assertEquals("Should have 3 snapshots", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); // retain all snapshots on branch (including the initial snapshot) table @@ -1592,20 +1534,18 @@ public void testMaxSnapshotAgeMultipleBranches() { removeSnapshots(table).cleanExpiredFiles(false).commit(); - Assert.assertEquals( - "Should have 3 snapshots (none removed)", 3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); // allow the initial snapshot to age off from branch table.manageSnapshots().setMaxSnapshotAgeMs("branch", ageMs).commit(); table.expireSnapshots().cleanExpiredFiles(false).commit(); - Assert.assertEquals( - "Should have 2 snapshots (initial removed)", 2, Iterables.size(table.snapshots())); - Assert.assertNull(table.ops().current().snapshot(initialSnapshotId)); + assertThat(table.snapshots()).hasSize(2); + assertThat(table.ops().current().snapshot(initialSnapshotId)).isNull(); } - @Test + @TestTemplate public void testRetainFilesOnRetainedBranches() { // Append a file to main and test branch String testBranch = "test-branch"; @@ -1628,8 +1568,8 @@ public void testRetainFilesOnRetainedBranches() { expectedDeletes.addAll(manifestPaths(deletionA, table.io())); table.expireSnapshots().expireOlderThan(tAfterCommits).deleteWith(deletedFiles::add).commit(); - Assert.assertEquals(2, Iterables.size(table.snapshots())); - Assert.assertEquals(expectedDeletes, deletedFiles); + assertThat(table.snapshots()).hasSize(2); + assertThat(deletedFiles).isEqualTo(expectedDeletes); // Delete A on test branch table.newDelete().deleteFile(FILE_A).toBranch(testBranch).commit(); @@ -1655,8 +1595,8 @@ public void testRetainFilesOnRetainedBranches() { expectedDeletes.addAll(manifestPaths(branchDelete, table.io())); expectedDeletes.add(FILE_A.path().toString()); - Assert.assertEquals(2, Iterables.size(table.snapshots())); - Assert.assertEquals(expectedDeletes, deletedFiles); + assertThat(table.snapshots()).hasSize(2); + assertThat(deletedFiles).isEqualTo(expectedDeletes); } private Set manifestPaths(Snapshot snapshot, FileIO io) { diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 8cc7e440686d..ef42fc1793da 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; 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.spy; import static org.mockito.Mockito.when; @@ -40,24 +41,17 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestRewriteManifests extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestRewriteManifests extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestRewriteManifests(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testRewriteManifestsAppendedDirectly() throws IOException { Table table = load(); @@ -70,18 +64,18 @@ public void testRewriteManifestsAppendedDirectly() throws IOException { table.newFastAppend().appendManifest(newManifest).commit(); long appendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); table.rewriteManifests().clusterBy(file -> "").commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(1, manifests.size()); + assertThat(manifests).hasSize(1); validateManifestEntries( manifests.get(0), ids(appendId), files(FILE_A), statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testRewriteManifestsWithScanExecutor() throws IOException { Table table = load(); @@ -93,7 +87,7 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { table.newFastAppend().appendManifest(newManifest).commit(); - Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); AtomicInteger scanThreadsIndex = new AtomicInteger(0); table .rewriteManifests() @@ -111,11 +105,13 @@ public void testRewriteManifestsWithScanExecutor() throws IOException { .commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(1, manifests.size()); - Assert.assertTrue("Thread should be created in provided pool", scanThreadsIndex.get() > 0); + assertThat(manifests).hasSize(1); + assertThat(scanThreadsIndex.get()) + .as("Thread should be created in provided pool") + .isGreaterThan(0); } - @Test + @TestTemplate public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOException { Table table = load(); @@ -131,12 +127,12 @@ public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOExceptio table.newFastAppend().appendFile(FILE_B).commit(); long fileAppendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); table.rewriteManifests().clusterBy(file -> "").commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Manifests must be merged into 1", 1, manifests.size()); + assertThat(manifests).hasSize(1); // get the correct file order List files; @@ -158,20 +154,20 @@ public void testRewriteManifestsGeneratedAndAppendedDirectly() throws IOExceptio statuses(ManifestEntry.Status.EXISTING, ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testReplaceManifestsSeparate() { Table table = load(); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); long appendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); // cluster by path will split the manifest into two table.rewriteManifests().clusterBy(file -> file.path()).commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(2, manifests.size()); + assertThat(manifests).hasSize(2); manifests.sort(Comparator.comparing(ManifestFile::path)); validateManifestEntries( @@ -180,7 +176,7 @@ public void testReplaceManifestsSeparate() { manifests.get(1), ids(appendId), files(FILE_B), statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testReplaceManifestsConsolidate() throws IOException { Table table = load(); @@ -189,14 +185,14 @@ public void testReplaceManifestsConsolidate() throws IOException { table.newFastAppend().appendFile(FILE_B).commit(); long appendIdB = table.currentSnapshot().snapshotId(); - Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); // cluster by constant will combine manifests into one table.rewriteManifests().clusterBy(file -> "file").commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(1, manifests.size()); + assertThat(manifests).hasSize(1); // get the file order correct List files; @@ -218,7 +214,7 @@ public void testReplaceManifestsConsolidate() throws IOException { statuses(ManifestEntry.Status.EXISTING, ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testReplaceManifestsWithFilter() throws IOException { Table table = load(); @@ -231,7 +227,7 @@ public void testReplaceManifestsWithFilter() throws IOException { table.newFastAppend().appendFile(FILE_C).commit(); long appendIdC = table.currentSnapshot().snapshotId(); - Assert.assertEquals(3, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(3); // keep the file A manifest, combine the other two @@ -249,7 +245,7 @@ public void testReplaceManifestsWithFilter() throws IOException { .commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(2, manifests.size()); + assertThat(manifests).hasSize(2); // get the file order correct List files; @@ -273,13 +269,13 @@ public void testReplaceManifestsWithFilter() throws IOException { manifests.get(1), ids(appendIdA), files(FILE_A), statuses(ManifestEntry.Status.ADDED)); } - @Test + @TestTemplate public void testReplaceManifestsMaxSize() { Table table = load(); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); long appendId = table.currentSnapshot().snapshotId(); - Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); // cluster by constant will combine manifests into one but small target size will create one per // entry @@ -288,7 +284,7 @@ public void testReplaceManifestsMaxSize() { rewriteManifests.clusterBy(file -> "file").commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(2, manifests.size()); + assertThat(manifests).hasSize(2); manifests.sort(Comparator.comparing(ManifestFile::path)); validateManifestEntries( @@ -297,7 +293,7 @@ public void testReplaceManifestsMaxSize() { manifests.get(1), ids(appendId), files(FILE_B), statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testConcurrentRewriteManifest() throws IOException { Table table = load(); table.newFastAppend().appendFile(FILE_A).commit(); @@ -323,14 +319,14 @@ public void testConcurrentRewriteManifest() throws IOException { }) .commit(); - Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); // commit the rewrite manifests in progress - this should perform a full rewrite as the manifest // with file B is no longer part of the snapshot rewrite.commit(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(1, manifests.size()); + assertThat(manifests).hasSize(1); // get the file order correct List files; @@ -352,7 +348,7 @@ public void testConcurrentRewriteManifest() throws IOException { statuses(ManifestEntry.Status.EXISTING, ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testAppendDuringRewriteManifest() { Table table = load(); table.newFastAppend().appendFile(FILE_A).commit(); @@ -366,7 +362,7 @@ public void testAppendDuringRewriteManifest() { table.newFastAppend().appendFile(FILE_B).commit(); long appendIdB = table.currentSnapshot().snapshotId(); - Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); // commit the rewrite manifests in progress rewrite.commit(); @@ -376,7 +372,7 @@ public void testAppendDuringRewriteManifest() { // have a single cluster key, rewritten one should be the first in the list List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(2, manifests.size()); + assertThat(manifests).hasSize(2); validateManifestEntries( manifests.get(0), ids(appendIdA), files(FILE_A), statuses(ManifestEntry.Status.EXISTING)); @@ -384,7 +380,7 @@ public void testAppendDuringRewriteManifest() { manifests.get(1), ids(appendIdB), files(FILE_B), statuses(ManifestEntry.Status.ADDED)); } - @Test + @TestTemplate public void testRewriteManifestDuringAppend() { Table table = load(); table.newFastAppend().appendFile(FILE_A).commit(); @@ -397,14 +393,14 @@ public void testRewriteManifestDuringAppend() { // rewrite the manifests - only affects the first table.rewriteManifests().clusterBy(file -> "file").commit(); - Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); // commit the append in progress append.commit(); long appendIdB = table.currentSnapshot().snapshotId(); List manifests = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals(2, manifests.size()); + assertThat(manifests).hasSize(2); // last append should be the first in the list @@ -414,15 +410,15 @@ public void testRewriteManifestDuringAppend() { manifests.get(1), ids(appendIdA), files(FILE_A), statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testBasicManifestReplacement() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); @@ -445,7 +441,7 @@ public void testBasicManifestReplacement() throws IOException { Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(3, manifests.size()); + assertThat(manifests).hasSize(3); if (formatVersion == 1) { assertThat(manifests.get(0).path()).isNotEqualTo(firstNewManifest.path()); @@ -476,9 +472,9 @@ public void testBasicManifestReplacement() throws IOException { statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); } - @Test + @TestTemplate public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.updateProperties().set(SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); @@ -486,7 +482,7 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); @@ -509,7 +505,7 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(3, manifests.size()); + assertThat(manifests).hasSize(3); assertThat(manifests.get(0).path()).isEqualTo(firstNewManifest.path()); assertThat(manifests.get(1).path()).isEqualTo(secondNewManifest.path()); @@ -538,17 +534,14 @@ public void testBasicManifestReplacementWithSnapshotIdInheritance() throws IOExc table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); } - @Test + @TestTemplate public void testWithMultiplePartitionSpec() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - base.currentSnapshot().allManifests(table.io()).size()); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); int initialPartitionSpecId = initialManifest.partitionSpecId(); @@ -580,8 +573,7 @@ public void testWithMultiplePartitionSpec() throws IOException { table.newAppend().appendFile(newFileZ).commit(); - Assert.assertEquals( - "Should use 3 manifest files", 3, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(3); RewriteManifests rewriteManifests = table.rewriteManifests(); // try to cluster in 1 manifest file, but because of 2 partition specs @@ -589,40 +581,33 @@ public void testWithMultiplePartitionSpec() throws IOException { rewriteManifests.clusterBy(dataFile -> "file").commit(); List manifestFiles = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals( - "Rewrite manifest should produce 2 manifest files", 2, manifestFiles.size()); + assertThat(manifestFiles).as("Rewrite manifest should produce 2 manifest files").hasSize(2); - Assert.assertEquals( - "2 manifest files should have different partitionSpecId", - true, - manifestFiles.get(0).partitionSpecId() != manifestFiles.get(1).partitionSpecId()); + assertThat(manifestFiles.get(1).partitionSpecId()) + .as("2 manifest files should have different partitionSpecId") + .isNotEqualTo(manifestFiles.get(0).partitionSpecId()); matchNumberOfManifestFileWithSpecId(manifestFiles, initialPartitionSpecId, 1); matchNumberOfManifestFileWithSpecId(manifestFiles, table.ops().current().spec().specId(), 1); - Assert.assertEquals( - "first manifest file should have 2 data files", - Integer.valueOf(2), - manifestFiles.get(0).existingFilesCount()); + assertThat(manifestFiles.get(0).existingFilesCount()) + .as("first manifest file should have 2 data files") + .isEqualTo(2); - Assert.assertEquals( - "second manifest file should have 2 data files", - Integer.valueOf(2), - manifestFiles.get(1).existingFilesCount()); + assertThat(manifestFiles.get(1).existingFilesCount()) + .as("second manifest file should have 2 data files") + .isEqualTo(2); } - @Test + @TestTemplate public void testManifestSizeWithMultiplePartitionSpec() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); TableMetadata base = readMetadata(); - Assert.assertEquals( - "Should create 1 manifest for initial write", - 1, - base.currentSnapshot().allManifests(table.io()).size()); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(1); ManifestFile initialManifest = base.currentSnapshot().allManifests(table.io()).get(0); int initialPartitionSpecId = initialManifest.partitionSpecId(); @@ -653,10 +638,9 @@ public void testManifestSizeWithMultiplePartitionSpec() throws IOException { table.newAppend().appendFile(newFileZ).commit(); - Assert.assertEquals( - "Rewrite manifests should produce 3 manifest files", - 3, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())) + .as("Rewrite manifests should produce 3 manifest files") + .hasSize(3); // cluster by constant will combine manifests into one but small target size will create one per // entry @@ -667,42 +651,28 @@ public void testManifestSizeWithMultiplePartitionSpec() throws IOException { rewriteManifests.clusterBy(dataFile -> "file").commit(); List manifestFiles = table.currentSnapshot().allManifests(table.io()); - Assert.assertEquals("Should use 4 manifest files", 4, manifestFiles.size()); + assertThat(manifestFiles).hasSize(4); matchNumberOfManifestFileWithSpecId(manifestFiles, initialPartitionSpecId, 2); matchNumberOfManifestFileWithSpecId(manifestFiles, table.ops().current().spec().specId(), 2); - Assert.assertEquals( - "first manifest file should have 1 data files", - Integer.valueOf(1), - manifestFiles.get(0).existingFilesCount()); - - Assert.assertEquals( - "second manifest file should have 1 data files", - Integer.valueOf(1), - manifestFiles.get(1).existingFilesCount()); - - Assert.assertEquals( - "third manifest file should have 1 data files", - Integer.valueOf(1), - manifestFiles.get(2).existingFilesCount()); - - Assert.assertEquals( - "fourth manifest file should have 1 data files", - Integer.valueOf(1), - manifestFiles.get(3).existingFilesCount()); + assertThat(manifestFiles.get(0).existingFilesCount()).isEqualTo(1); + + assertThat(manifestFiles.get(1).existingFilesCount()).isEqualTo(1); + assertThat(manifestFiles.get(2).existingFilesCount()).isEqualTo(1); + assertThat(manifestFiles.get(3).existingFilesCount()).isEqualTo(1); } - @Test + @TestTemplate public void testManifestReplacementConcurrentAppend() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); ManifestFile firstNewManifest = @@ -722,13 +692,13 @@ public void testManifestReplacementConcurrentAppend() throws IOException { table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); Snapshot secondSnapshot = table.currentSnapshot(); - Assert.assertEquals(2, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(2); rewriteManifests.commit(); Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(3, manifests.size()); + assertThat(manifests).hasSize(3); validateSummary(snapshot, 1, 1, 2, 0); @@ -751,9 +721,9 @@ public void testManifestReplacementConcurrentAppend() throws IOException { statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); } - @Test + @TestTemplate public void testManifestReplacementConcurrentDelete() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.updateProperties().set(MANIFEST_MERGE_ENABLED, "false").commit(); @@ -761,7 +731,7 @@ public void testManifestReplacementConcurrentDelete() throws IOException { Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); @@ -788,7 +758,7 @@ public void testManifestReplacementConcurrentDelete() throws IOException { Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(3, manifests.size()); + assertThat(manifests).hasSize(3); validateSummary(snapshot, 1, 1, 2, 0); @@ -811,15 +781,15 @@ public void testManifestReplacementConcurrentDelete() throws IOException { statuses(ManifestEntry.Status.DELETED, ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testManifestReplacementConcurrentConflictingDelete() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); ManifestFile firstNewManifest = @@ -838,20 +808,20 @@ public void testManifestReplacementConcurrentConflictingDelete() throws IOExcept table.newDelete().deleteFile(FILE_A).commit(); - Assertions.assertThatThrownBy(rewriteManifests::commit) + assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Manifest is missing"); } - @Test + @TestTemplate public void testManifestReplacementCombinedWithRewrite() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newFastAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_B).commit(); @@ -862,7 +832,7 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { table.newFastAppend().appendFile(FILE_D).commit(); - Assert.assertEquals(4, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(4); ManifestFile newManifest = writeManifest( @@ -886,7 +856,7 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(3, manifests.size()); + assertThat(manifests).hasSize(3); validateSummary(snapshot, 3, 1, 2, 2); @@ -903,9 +873,9 @@ public void testManifestReplacementCombinedWithRewrite() throws IOException { statuses(ManifestEntry.Status.ADDED)); } - @Test + @TestTemplate public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.updateProperties().set(MANIFEST_MERGE_ENABLED, "false").commit(); @@ -913,7 +883,7 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_B).commit(); @@ -922,7 +892,7 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws table.newFastAppend().appendFile(FILE_C).commit(); - Assert.assertEquals(3, Iterables.size(table.snapshots())); + assertThat(table.snapshots()).hasSize(3); ManifestEntry entry = manifestEntry(ManifestEntry.Status.EXISTING, firstSnapshot.snapshotId(), FILE_A); @@ -945,7 +915,7 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(2, manifests.size()); + assertThat(manifests).hasSize(2); validateSummary(snapshot, 3, 0, 2, 1); @@ -962,15 +932,15 @@ public void testManifestReplacementCombinedWithRewriteConcurrentDelete() throws statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testInvalidUsage() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snapshot = table.currentSnapshot(); List manifests = snapshot.allManifests(table.io()); - Assert.assertEquals(1, manifests.size()); + assertThat(manifests).hasSize(1); ManifestFile manifest = manifests.get(0); ManifestEntry appendEntry = @@ -980,7 +950,7 @@ public void testInvalidUsage() throws IOException { ManifestFile invalidAddedFileManifest = writeManifest("manifest-file-2.avro", appendEntry); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .rewriteManifests() @@ -997,7 +967,7 @@ public void testInvalidUsage() throws IOException { ManifestFile invalidDeletedFileManifest = writeManifest("manifest-file-3.avro", deleteEntry); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .rewriteManifests() @@ -1007,28 +977,28 @@ public void testInvalidUsage() throws IOException { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot add manifest with deleted files"); - Assertions.assertThatThrownBy(() -> table.rewriteManifests().deleteManifest(manifest).commit()) + assertThatThrownBy(() -> table.rewriteManifests().deleteManifest(manifest).commit()) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( "Replaced and created manifests must have the same number of active files"); } - @Test + @TestTemplate public void testManifestReplacementFailure() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.newFastAppend().appendFile(FILE_A).commit(); Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_B).commit(); Snapshot secondSnapshot = table.currentSnapshot(); List secondSnapshotManifests = secondSnapshot.allManifests(table.io()); - Assert.assertEquals(2, secondSnapshotManifests.size()); + assertThat(secondSnapshotManifests).hasSize(2); ManifestFile secondSnapshotManifest = secondSnapshotManifests.get(0); ManifestFile newManifest = @@ -1046,16 +1016,16 @@ public void testManifestReplacementFailure() throws IOException { rewriteManifests.deleteManifest(secondSnapshotManifest); rewriteManifests.addManifest(newManifest); - Assertions.assertThatThrownBy(rewriteManifests::commit) + assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertTrue("New manifest should not be deleted", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); } - @Test + @TestTemplate public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOException { - Assert.assertNull("Table should be empty", table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); table.updateProperties().set(SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); @@ -1063,14 +1033,14 @@ public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOE Snapshot firstSnapshot = table.currentSnapshot(); List firstSnapshotManifests = firstSnapshot.allManifests(table.io()); - Assert.assertEquals(1, firstSnapshotManifests.size()); + assertThat(firstSnapshotManifests).hasSize(1); ManifestFile firstSnapshotManifest = firstSnapshotManifests.get(0); table.newFastAppend().appendFile(FILE_B).commit(); Snapshot secondSnapshot = table.currentSnapshot(); List secondSnapshotManifests = secondSnapshot.allManifests(table.io()); - Assert.assertEquals(2, secondSnapshotManifests.size()); + assertThat(secondSnapshotManifests).hasSize(2); ManifestFile secondSnapshotManifest = secondSnapshotManifests.get(0); ManifestFile newManifest = @@ -1088,27 +1058,27 @@ public void testManifestReplacementFailureWithSnapshotIdInheritance() throws IOE rewriteManifests.deleteManifest(secondSnapshotManifest); rewriteManifests.addManifest(newManifest); - Assertions.assertThatThrownBy(rewriteManifests::commit) + assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); - Assert.assertTrue("New manifest should not be deleted", new File(newManifest.path()).exists()); + assertThat(new File(newManifest.path())).exists(); } - @Test + @TestTemplate public void testRewriteManifestsOnBranchUnsupported() { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); - Assert.assertEquals(1, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(1); - Assertions.assertThatThrownBy(() -> table.rewriteManifests().toBranch("someBranch").commit()) + assertThatThrownBy(() -> table.rewriteManifests().toBranch("someBranch").commit()) .isInstanceOf(UnsupportedOperationException.class) .hasMessage( "Cannot commit to branch someBranch: org.apache.iceberg.BaseRewriteManifests does not support branch commits"); } - @Test + @TestTemplate public void testRewriteDataManifestsPreservesDeletes() { assumeThat(formatVersion).isGreaterThan(1); @@ -1170,7 +1140,7 @@ public void testRewriteDataManifestsPreservesDeletes() { statuses(ManifestEntry.Status.ADDED, ManifestEntry.Status.ADDED)); } - @Test + @TestTemplate public void testReplaceDeleteManifestsOnly() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -1256,7 +1226,7 @@ public void testReplaceDeleteManifestsOnly() throws IOException { statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testReplaceDataAndDeleteManifests() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -1375,7 +1345,7 @@ public void testReplaceDataAndDeleteManifests() throws IOException { statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testDeleteManifestReplacementConcurrentAppend() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -1478,7 +1448,7 @@ public void testDeleteManifestReplacementConcurrentAppend() throws IOException { statuses(ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -1586,7 +1556,7 @@ public void testDeleteManifestReplacementConcurrentDeleteFileRemoval() throws IO statuses(ManifestEntry.Status.DELETED, ManifestEntry.Status.EXISTING)); } - @Test + @TestTemplate public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -1632,12 +1602,12 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I table.newRewrite().deleteFile(FILE_A_DELETES).commit(); // the rewrite must fail as the original delete manifest was replaced concurrently - Assertions.assertThatThrownBy(rewriteManifests::commit) + assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Manifest is missing"); } - @Test + @TestTemplate public void testDeleteManifestReplacementFailure() throws IOException { assumeThat(formatVersion).isGreaterThan(1); @@ -1693,7 +1663,7 @@ public void testDeleteManifestReplacementFailure() throws IOException { rewriteManifests.addManifest(newDeleteManifest); // the rewrite must fail - Assertions.assertThatThrownBy(rewriteManifests::commit) + assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); @@ -1717,18 +1687,11 @@ private List sortedDataManifests(FileIO io, Snapshot snapshot) { private void validateSummary( Snapshot snapshot, int replaced, int kept, int created, int entryCount) { Map summary = snapshot.summary(); - Assert.assertEquals( - "Replaced manifest count should match", - replaced, - Integer.parseInt(summary.get("manifests-replaced"))); - Assert.assertEquals( - "Kept manifest count should match", kept, Integer.parseInt(summary.get("manifests-kept"))); - Assert.assertEquals( - "Created manifest count should match", - created, - Integer.parseInt(summary.get("manifests-created"))); - Assert.assertEquals( - "Entry count should match", entryCount, Integer.parseInt(summary.get("entries-processed"))); + assertThat(summary) + .containsEntry("manifests-replaced", String.valueOf(replaced)) + .containsEntry("manifests-kept", String.valueOf(kept)) + .containsEntry("manifests-created", String.valueOf(created)) + .containsEntry("entries-processed", String.valueOf(entryCount)); } private void matchNumberOfManifestFileWithSpecId( @@ -1740,12 +1703,12 @@ private void matchNumberOfManifestFileWithSpecId( .filter(m -> m.partitionSpecId() == toBeMatchedPartitionSpecId) .count(); - Assert.assertEquals( - "manifest list should have " - + numberOfManifestWithPartitionSpecID - + " manifests matching this partitionSpecId " - + toBeMatchedPartitionSpecId, - numberOfManifestWithPartitionSpecID, - matchedManifestsCounter); + assertThat(matchedManifestsCounter) + .as( + "manifest list should have " + + numberOfManifestWithPartitionSpecID + + " manifests matching this partitionSpecId " + + toBeMatchedPartitionSpecId) + .isEqualTo(numberOfManifestWithPartitionSpecID); } } diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 56bab52edaf9..867e4b062f4d 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -27,7 +27,10 @@ import static org.apache.iceberg.SnapshotSummary.TOTAL_DELETE_FILES_PROP; import static org.apache.iceberg.SnapshotSummary.TOTAL_POS_DELETES_PROP; 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 java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Set; @@ -38,44 +41,34 @@ 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.Sets; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestRowDelta extends V2TableTestBase { - private final String branch; + @Parameter(index = 1) + private String branch; - @Parameterized.Parameters(name = "branch = {0}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {"main"}, new Object[] {"testBranch"}, - }; + @Parameters(name = "formatVersion = {0}, branch = {1}") + protected static List parameters() { + return Arrays.asList(new Object[] {2, "main"}, new Object[] {2, "testBranch"}); } - public TestRowDelta(String branch) { - this.branch = branch; - } - - @Test + @TestTemplate public void testAddDeleteFile() { SnapshotUpdate rowDelta = table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES); commit(table, rowDelta, branch); Snapshot snap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 1", 1, snap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); - Assert.assertEquals( - "Delta commit should use operation 'overwrite'", - DataOperations.OVERWRITE, - snap.operation()); - - Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size()); + assertThat(snap.sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); + assertThat(snap.operation()) + .as("Delta commit should use operation 'overwrite'") + .isEqualTo(DataOperations.OVERWRITE); + assertThat(snap.dataManifests(table.io())).hasSize(1); + validateManifest( snap.dataManifests(table.io()).get(0), dataSeqs(1L), @@ -84,8 +77,7 @@ public void testAddDeleteFile() { files(FILE_A), statuses(Status.ADDED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( snap.deleteManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -95,7 +87,7 @@ public void testAddDeleteFile() { statuses(Status.ADDED, Status.ADDED)); } - @Test + @TestTemplate public void testValidateDataFilesExistDefaults() { SnapshotUpdate rowDelta1 = table.newAppend().appendFile(FILE_A).appendFile(FILE_B); @@ -116,7 +108,7 @@ public void testValidateDataFilesExistDefaults() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -129,15 +121,11 @@ public void testValidateDataFilesExistDefaults() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); - Assert.assertEquals( - "Table state should not be modified by failed RowDelta operation", - deleteSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()) + .as("Table state should not be modified by failed RowDelta operation") + .isEqualTo(deleteSnapshotId); - Assert.assertEquals( - "Table should not have any delete manifests", - 0, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).isEmpty(); commit( table, @@ -148,10 +136,7 @@ public void testValidateDataFilesExistDefaults() { .validateFromSnapshot(validateFromSnapshotId), branch); - Assert.assertEquals( - "Table should have one new delete manifest", - 1, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).hasSize(1); ManifestFile deletes = latestSnapshot(table, branch).deleteManifests(table.io()).get(0); validateDeleteManifest( deletes, @@ -162,7 +147,7 @@ public void testValidateDataFilesExistDefaults() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testValidateDataFilesExistOverwrite() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -174,7 +159,7 @@ public void testValidateDataFilesExistOverwrite() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -187,18 +172,14 @@ public void testValidateDataFilesExistOverwrite() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); - Assert.assertEquals( - "Table state should not be modified by failed RowDelta operation", - deleteSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()) + .as("Table state should not be modified by failed RowDelta operation") + .isEqualTo(deleteSnapshotId); - Assert.assertEquals( - "Table should not have any delete manifests", - 0, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).isEmpty(); } - @Test + @TestTemplate public void testValidateDataFilesExistReplacePartitions() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -210,7 +191,7 @@ public void testValidateDataFilesExistReplacePartitions() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -223,18 +204,14 @@ public void testValidateDataFilesExistReplacePartitions() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); - Assert.assertEquals( - "Table state should not be modified by failed RowDelta operation", - deleteSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()) + .as("Table state should not be modified by failed RowDelta operation") + .isEqualTo(deleteSnapshotId); - Assert.assertEquals( - "Table should not have any delete manifests", - 0, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).isEmpty(); } - @Test + @TestTemplate public void testValidateDataFilesExistFromSnapshot() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -259,11 +236,10 @@ public void testValidateDataFilesExistFromSnapshot() { branch); Snapshot snap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 3, snap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); + assertThat(snap.sequenceNumber()).isEqualTo(3); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(3); - Assert.assertEquals("Should have 2 data manifests", 2, snap.dataManifests(table.io()).size()); + assertThat(snap.dataManifests(table.io())).hasSize(2); // manifest with FILE_A2 added validateManifest( snap.dataManifests(table.io()).get(0), @@ -282,8 +258,7 @@ public void testValidateDataFilesExistFromSnapshot() { files(FILE_A, FILE_B), statuses(Status.DELETED, Status.EXISTING)); - Assert.assertEquals( - "Should have 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( snap.deleteManifests(table.io()).get(0), dataSeqs(3L), @@ -293,7 +268,7 @@ public void testValidateDataFilesExistFromSnapshot() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testValidateDataFilesExistRewrite() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -308,7 +283,7 @@ public void testValidateDataFilesExistRewrite() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -321,18 +296,14 @@ public void testValidateDataFilesExistRewrite() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); - Assert.assertEquals( - "Table state should not be modified by failed RowDelta operation", - deleteSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()) + .as("Table state should not be modified by failed RowDelta operation") + .isEqualTo(deleteSnapshotId); - Assert.assertEquals( - "Table should not have any delete manifests", - 0, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).isEmpty(); } - @Test + @TestTemplate public void testValidateDataFilesExistValidateDeletes() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_B), branch); @@ -344,7 +315,7 @@ public void testValidateDataFilesExistValidateDeletes() { long deleteSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -358,18 +329,14 @@ public void testValidateDataFilesExistValidateDeletes() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); - Assert.assertEquals( - "Table state should not be modified by failed RowDelta operation", - deleteSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()) + .as("Table state should not be modified by failed RowDelta operation") + .isEqualTo(deleteSnapshotId); - Assert.assertEquals( - "Table should not have any delete manifests", - 0, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).isEmpty(); } - @Test + @TestTemplate public void testValidateNoConflicts() { commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -381,7 +348,7 @@ public void testValidateNoConflicts() { long appendSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> commit( table, @@ -396,18 +363,14 @@ public void testValidateNoConflicts() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found conflicting files"); - Assert.assertEquals( - "Table state should not be modified by failed RowDelta operation", - appendSnapshotId, - latestSnapshot(table, branch).snapshotId()); + assertThat(latestSnapshot(table, branch).snapshotId()) + .as("Table state should not be modified by failed RowDelta operation") + .isEqualTo(appendSnapshotId); - Assert.assertEquals( - "Table should not have any delete manifests", - 0, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).isEmpty(); } - @Test + @TestTemplate public void testValidateNoConflictsFromSnapshot() { commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -433,11 +396,10 @@ public void testValidateNoConflictsFromSnapshot() { branch); Snapshot snap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 3, snap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); + assertThat(snap.sequenceNumber()).isEqualTo(3); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(3); - Assert.assertEquals("Should have 2 data manifests", 2, snap.dataManifests(table.io()).size()); + assertThat(snap.dataManifests(table.io())).hasSize(2); // manifest with FILE_A2 added validateManifest( snap.dataManifests(table.io()).get(0), @@ -456,8 +418,7 @@ public void testValidateNoConflictsFromSnapshot() { files(FILE_A), statuses(Status.ADDED)); - Assert.assertEquals( - "Should have 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( snap.deleteManifests(table.io()).get(0), dataSeqs(3L), @@ -467,7 +428,7 @@ public void testValidateNoConflictsFromSnapshot() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testOverwriteWithDeleteFile() { commit( table, @@ -475,12 +436,8 @@ public void testOverwriteWithDeleteFile() { branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assert.assertEquals( - "Commit should produce sequence number 1", - 1, - latestSnapshot(table, branch).sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); // overwriting by a filter will also remove delete files that match because all matching data // files are removed. @@ -492,11 +449,10 @@ public void testOverwriteWithDeleteFile() { branch); Snapshot snap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); + assertThat(snap.sequenceNumber()).isEqualTo(2); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(2); - Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size()); + assertThat(snap.dataManifests(table.io())).hasSize(1); validateManifest( snap.dataManifests(table.io()).get(0), dataSeqs(1L), @@ -505,8 +461,7 @@ public void testOverwriteWithDeleteFile() { files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( snap.deleteManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -516,7 +471,7 @@ public void testOverwriteWithDeleteFile() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testReplacePartitionsWithDeleteFile() { commit( table, @@ -524,24 +479,18 @@ public void testReplacePartitionsWithDeleteFile() { branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assert.assertEquals( - "Commit should produce sequence number 1", - 1, - latestSnapshot(table, branch).sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); // overwriting the partition will also remove delete files that match because all matching data // files are removed. commit(table, table.newReplacePartitions().addFile(FILE_A2), branch); Snapshot snap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); + assertThat(snap.sequenceNumber()).isEqualTo(2); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(2); - Assert.assertEquals( - "Should produce 2 data manifests", 2, snap.dataManifests(table.io()).size()); + assertThat(snap.dataManifests(table.io())).hasSize(2); int deleteManifestPos = snap.dataManifests(table.io()).get(0).deletedFilesCount() > 0 ? 0 : 1; validateManifest( snap.dataManifests(table.io()).get(deleteManifestPos), @@ -559,8 +508,7 @@ public void testReplacePartitionsWithDeleteFile() { files(FILE_A2), statuses(Status.ADDED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( snap.deleteManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -570,7 +518,7 @@ public void testReplacePartitionsWithDeleteFile() { statuses(Status.DELETED, Status.EXISTING)); } - @Test + @TestTemplate public void testDeleteByExpressionWithDeleteFile() { commit( table, @@ -578,23 +526,18 @@ public void testDeleteByExpressionWithDeleteFile() { branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assert.assertEquals( - "Commit should produce sequence number 1", - 1, - latestSnapshot(table, branch).sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); // deleting with a filter will also remove delete files that match because all matching data // files are removed. commit(table, table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()), branch); Snapshot snap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 2, snap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); + assertThat(snap.sequenceNumber()).isEqualTo(2); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(2); - Assert.assertEquals("Should produce 1 data manifest", 1, snap.dataManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateManifest( snap.dataManifests(table.io()).get(0), dataSeqs(1L), @@ -603,8 +546,7 @@ public void testDeleteByExpressionWithDeleteFile() { files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, snap.deleteManifests(table.io()).size()); + assertThat(snap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( snap.deleteManifests(table.io()).get(0), dataSeqs(1L, 1L), @@ -614,28 +556,22 @@ public void testDeleteByExpressionWithDeleteFile() { statuses(Status.DELETED, Status.DELETED)); } - @Test + @TestTemplate public void testDeleteDataFileWithDeleteFile() { commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assert.assertEquals( - "Commit should produce sequence number 1", - 1, - latestSnapshot(table, branch).sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); // deleting a specific data file will not affect a delete file commit(table, table.newDelete().deleteFile(FILE_A), branch); Snapshot deleteSnap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 2, deleteSnap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); + assertThat(deleteSnap.sequenceNumber()).isEqualTo(2); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(2); - Assert.assertEquals( - "Should produce 1 data manifest", 1, deleteSnap.dataManifests(table.io()).size()); + assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); validateManifest( deleteSnap.dataManifests(table.io()).get(0), dataSeqs(1L), @@ -644,8 +580,7 @@ public void testDeleteDataFileWithDeleteFile() { files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, deleteSnap.deleteManifests(table.io()).size()); + assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( deleteSnap.deleteManifests(table.io()).get(0), dataSeqs(1L), @@ -662,14 +597,11 @@ public void testDeleteDataFileWithDeleteFile() { commit(table, table.newDelete().deleteFile("no-such-file"), branch); Snapshot nextSnap = latestSnapshot(table, branch); - Assert.assertEquals("Append should produce sequence number 3", 3, nextSnap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); - - Assert.assertEquals( - "Should have 0 data manifests", 0, nextSnap.dataManifests(table.io()).size()); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, nextSnap.deleteManifests(table.io()).size()); + assertThat(nextSnap.sequenceNumber()).isEqualTo(3); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(3); + + assertThat(nextSnap.dataManifests(table.io())).isEmpty(); + assertThat(nextSnap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( nextSnap.deleteManifests(table.io()).get(0), dataSeqs(1L), @@ -679,28 +611,22 @@ public void testDeleteDataFileWithDeleteFile() { statuses(Status.DELETED)); } - @Test + @TestTemplate public void testFastAppendDoesNotRemoveStaleDeleteFiles() { commit(table, table.newRowDelta().addRows(FILE_A).addDeletes(FILE_A_DELETES), branch); long deltaSnapshotId = latestSnapshot(table, branch).snapshotId(); - Assert.assertEquals( - "Commit should produce sequence number 1", - 1, - latestSnapshot(table, branch).sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + assertThat(latestSnapshot(table, branch).sequenceNumber()).isEqualTo(1); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(1); // deleting a specific data file will not affect a delete file commit(table, table.newDelete().deleteFile(FILE_A), branch); Snapshot deleteSnap = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 2", 2, deleteSnap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 2", 2, table.ops().current().lastSequenceNumber()); + assertThat(deleteSnap.sequenceNumber()).isEqualTo(2); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(2); - Assert.assertEquals( - "Should produce 1 data manifest", 1, deleteSnap.dataManifests(table.io()).size()); + assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); validateManifest( deleteSnap.dataManifests(table.io()).get(0), dataSeqs(1L), @@ -709,8 +635,7 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { files(FILE_A), statuses(Status.DELETED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, deleteSnap.deleteManifests(table.io()).size()); + assertThat(deleteSnap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( deleteSnap.deleteManifests(table.io()).get(0), dataSeqs(1L), @@ -724,12 +649,10 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { commit(table, table.newFastAppend().appendFile(FILE_B), branch); Snapshot nextSnap = latestSnapshot(table, branch); - Assert.assertEquals("Append should produce sequence number 3", 3, nextSnap.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 3", 3, table.ops().current().lastSequenceNumber()); + assertThat(nextSnap.sequenceNumber()).isEqualTo(3); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(3); - Assert.assertEquals( - "Should have 2 data manifests", 2, nextSnap.dataManifests(table.io()).size()); + assertThat(nextSnap.dataManifests(table.io())).hasSize(2); int deleteManifestPos = nextSnap.dataManifests(table.io()).get(0).deletedFilesCount() > 0 ? 0 : 1; validateManifest( @@ -748,8 +671,7 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { files(FILE_B), statuses(Status.ADDED)); - Assert.assertEquals( - "Should produce 1 delete manifest", 1, nextSnap.deleteManifests(table.io()).size()); + assertThat(nextSnap.deleteManifests(table.io())).hasSize(1); validateDeleteManifest( nextSnap.deleteManifests(table.io()).get(0), dataSeqs(1L), @@ -759,7 +681,7 @@ public void testFastAppendDoesNotRemoveStaleDeleteFiles() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testValidateDataFilesExistWithConflictDetectionFilter() { // change the spec to be partitioned by data table @@ -820,10 +742,7 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { // commit the delta for partition A commit(table, rowDelta, branch); - Assert.assertEquals( - "Table should have one new delete manifest", - 1, - latestSnapshot(table, branch).deleteManifests(table.io()).size()); + assertThat(latestSnapshot(table, branch).deleteManifests(table.io())).hasSize(1); ManifestFile deletes = latestSnapshot(table, branch).deleteManifests(table.io()).get(0); validateDeleteManifest( deletes, @@ -834,7 +753,7 @@ public void testValidateDataFilesExistWithConflictDetectionFilter() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { // change the spec to be partitioned by data table @@ -881,12 +800,12 @@ public void testValidateDataFilesDoNotExistWithConflictDetectionFilter() { // concurrently delete the file for partition A commit(table, table.newDelete().deleteFile(dataFile1), branch); - Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + assertThatThrownBy(() -> commit(table, rowDelta, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); } - @Test + @TestTemplate public void testAddDeleteFilesMultipleSpecs() { // enable partition summaries table.updateProperties().set(TableProperties.WRITE_PARTITION_SUMMARY_LIMIT, "10").commit(); @@ -898,7 +817,7 @@ public void testAddDeleteFilesMultipleSpecs() { // remove the only partition field to make the spec unpartitioned table.updateSpec().removeField(Expressions.bucket("data", 16)).commit(); - Assert.assertTrue("Spec must be unpartitioned", table.spec().isUnpartitioned()); + assertThat(table.spec().isUnpartitioned()).isTrue(); // append an unpartitioned data file DataFile secondSnapshotDataFile = newDataFile(""); @@ -911,7 +830,7 @@ public void testAddDeleteFilesMultipleSpecs() { DataFile thirdSnapshotDataFile = newDataFile("data=abc"); commit(table, table.newAppend().appendFile(thirdSnapshotDataFile), branch); - Assert.assertEquals("Should have 3 specs", 3, table.specs().size()); + assertThat(table.specs()).hasSize(3); // commit a row delta with 1 data file and 3 delete files where delete files have different // specs @@ -931,40 +850,32 @@ public void testAddDeleteFilesMultipleSpecs() { branch); Snapshot snapshot = latestSnapshot(table, branch); - Assert.assertEquals("Commit should produce sequence number 4", 4, snapshot.sequenceNumber()); - Assert.assertEquals( - "Last sequence number should be 4", 4, table.ops().current().lastSequenceNumber()); - Assert.assertEquals( - "Delta commit should be 'overwrite'", DataOperations.OVERWRITE, snapshot.operation()); + assertThat(snapshot.sequenceNumber()).isEqualTo(4); + assertThat(table.ops().current().lastSequenceNumber()).isEqualTo(4); + assertThat(snapshot.operation()).isEqualTo(DataOperations.OVERWRITE); Map summary = snapshot.summary(); - Assert.assertEquals( - "Should change 4 partitions", "4", summary.get(CHANGED_PARTITION_COUNT_PROP)); - Assert.assertEquals("Should add 1 data file", "1", summary.get(ADDED_FILES_PROP)); - Assert.assertEquals("Should have 4 data files", "4", summary.get(TOTAL_DATA_FILES_PROP)); - Assert.assertEquals("Should add 3 delete files", "3", summary.get(ADDED_DELETE_FILES_PROP)); - Assert.assertEquals("Should have 3 delete files", "3", summary.get(TOTAL_DELETE_FILES_PROP)); - Assert.assertEquals("Should add 3 position deletes", "3", summary.get(ADDED_POS_DELETES_PROP)); - Assert.assertEquals("Should have 3 position deletes", "3", summary.get(TOTAL_POS_DELETES_PROP)); - - Assert.assertTrue( - "Partition metrics must be correct", - summary - .get(CHANGED_PARTITION_PREFIX + "data_bucket=0") - .contains(ADDED_DELETE_FILES_PROP + "=1")); - Assert.assertTrue( - "Partition metrics must be correct", - summary - .get(CHANGED_PARTITION_PREFIX + "data=abc") - .contains(ADDED_DELETE_FILES_PROP + "=1")); - Assert.assertTrue( - "Partition metrics must be correct", - summary.get(CHANGED_PARTITION_PREFIX + "data=xyz").contains(ADDED_FILES_PROP + "=1")); + assertThat(summary) + .containsEntry(CHANGED_PARTITION_COUNT_PROP, "4") + .containsEntry(ADDED_FILES_PROP, "1") + .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") + .hasEntrySatisfying( + CHANGED_PARTITION_PREFIX + "data_bucket=0", + v -> assertThat(v).contains(ADDED_DELETE_FILES_PROP + "=1")) + .hasEntrySatisfying( + CHANGED_PARTITION_PREFIX + "data=abc", + v -> assertThat(v).contains(ADDED_DELETE_FILES_PROP + "=1")) + .hasEntrySatisfying( + CHANGED_PARTITION_PREFIX + "data=xyz", + v -> assertThat(v).contains(ADDED_FILES_PROP + "=1")); // 3 appends + 1 row delta - Assert.assertEquals( - "Should have 4 data manifest", 4, snapshot.dataManifests(table.io()).size()); + assertThat(snapshot.dataManifests(table.io())).hasSize(4); validateManifest( snapshot.dataManifests(table.io()).get(0), dataSeqs(4L), @@ -974,12 +885,10 @@ public void testAddDeleteFilesMultipleSpecs() { statuses(Status.ADDED)); // each delete file goes into a separate manifest as the specs are different - Assert.assertEquals( - "Should produce 3 delete manifest", 3, snapshot.deleteManifests(table.io()).size()); + assertThat(snapshot.deleteManifests(table.io())).hasSize(3); ManifestFile firstDeleteManifest = snapshot.deleteManifests(table.io()).get(2); - Assert.assertEquals( - "Spec must match", firstSnapshotDataFile.specId(), firstDeleteManifest.partitionSpecId()); + assertThat(firstDeleteManifest.partitionSpecId()).isEqualTo(firstSnapshotDataFile.specId()); validateDeleteManifest( firstDeleteManifest, dataSeqs(4L), @@ -989,8 +898,7 @@ public void testAddDeleteFilesMultipleSpecs() { statuses(Status.ADDED)); ManifestFile secondDeleteManifest = snapshot.deleteManifests(table.io()).get(1); - Assert.assertEquals( - "Spec must match", secondSnapshotDataFile.specId(), secondDeleteManifest.partitionSpecId()); + assertThat(secondDeleteManifest.partitionSpecId()).isEqualTo(secondSnapshotDataFile.specId()); validateDeleteManifest( secondDeleteManifest, dataSeqs(4L), @@ -1000,8 +908,7 @@ public void testAddDeleteFilesMultipleSpecs() { statuses(Status.ADDED)); ManifestFile thirdDeleteManifest = snapshot.deleteManifests(table.io()).get(0); - Assert.assertEquals( - "Spec must match", thirdSnapshotDataFile.specId(), thirdDeleteManifest.partitionSpecId()); + assertThat(thirdDeleteManifest.partitionSpecId()).isEqualTo(thirdSnapshotDataFile.specId()); validateDeleteManifest( thirdDeleteManifest, dataSeqs(4L), @@ -1011,7 +918,7 @@ public void testAddDeleteFilesMultipleSpecs() { statuses(Status.ADDED)); } - @Test + @TestTemplate public void testManifestMergingMultipleSpecs() { // make sure we enable manifest merging table @@ -1027,7 +934,7 @@ public void testManifestMergingMultipleSpecs() { // remove the only partition field to make the spec unpartitioned table.updateSpec().removeField(Expressions.bucket("data", 16)).commit(); - Assert.assertTrue("Spec must be unpartitioned", table.spec().isUnpartitioned()); + assertThat(table.spec().isUnpartitioned()).isTrue(); // append an unpartitioned data file DataFile secondSnapshotDataFile = newDataFile(""); @@ -1045,10 +952,8 @@ public void testManifestMergingMultipleSpecs() { Snapshot thirdSnapshot = latestSnapshot(table, branch); // 2 appends and 1 row delta where delete files belong to different specs - Assert.assertEquals( - "Should have 2 data manifest", 2, thirdSnapshot.dataManifests(table.io()).size()); - Assert.assertEquals( - "Should have 2 delete manifest", 2, thirdSnapshot.deleteManifests(table.io()).size()); + assertThat(thirdSnapshot.dataManifests(table.io())).hasSize(2); + 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"); @@ -1062,14 +967,11 @@ public void testManifestMergingMultipleSpecs() { Snapshot fourthSnapshot = latestSnapshot(table, branch); // make sure merging respects spec boundaries - Assert.assertEquals( - "Should have 2 data manifest", 2, fourthSnapshot.dataManifests(table.io()).size()); - Assert.assertEquals( - "Should have 2 delete manifest", 2, fourthSnapshot.deleteManifests(table.io()).size()); + assertThat(fourthSnapshot.dataManifests(table.io())).hasSize(2); + assertThat(fourthSnapshot.deleteManifests(table.io())).hasSize(2); ManifestFile firstDeleteManifest = fourthSnapshot.deleteManifests(table.io()).get(1); - Assert.assertEquals( - "Spec must match", firstSnapshotDataFile.specId(), firstDeleteManifest.partitionSpecId()); + assertThat(firstDeleteManifest.partitionSpecId()).isEqualTo(firstSnapshotDataFile.specId()); validateDeleteManifest( firstDeleteManifest, dataSeqs(4L, 3L), @@ -1079,8 +981,7 @@ public void testManifestMergingMultipleSpecs() { statuses(Status.ADDED, Status.EXISTING)); ManifestFile secondDeleteManifest = fourthSnapshot.deleteManifests(table.io()).get(0); - Assert.assertEquals( - "Spec must match", secondSnapshotDataFile.specId(), secondDeleteManifest.partitionSpecId()); + assertThat(secondDeleteManifest.partitionSpecId()).isEqualTo(secondSnapshotDataFile.specId()); validateDeleteManifest( secondDeleteManifest, dataSeqs(4L, 3L), @@ -1090,7 +991,7 @@ public void testManifestMergingMultipleSpecs() { statuses(Status.ADDED, Status.EXISTING)); } - @Test + @TestTemplate public void testAbortMultipleSpecs() { // append a partitioned data file DataFile firstSnapshotDataFile = newDataFile("data_bucket=0"); @@ -1099,7 +1000,7 @@ public void testAbortMultipleSpecs() { // remove the only partition field to make the spec unpartitioned table.updateSpec().removeField(Expressions.bucket("data", 16)).commit(); - Assert.assertTrue("Spec must be unpartitioned", table.spec().isUnpartitioned()); + assertThat(table.spec().isUnpartitioned()).isTrue(); // append an unpartitioned data file DataFile secondSnapshotDataFile = newDataFile(""); @@ -1127,15 +1028,15 @@ public void testAbortMultipleSpecs() { // perform a conflicting concurrent operation commit(table, table.newDelete().deleteFile(firstSnapshotDataFile), branch); - Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + assertThatThrownBy(() -> commit(table, rowDelta, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, missing data files"); // we should clean up 1 manifest list and 2 delete manifests - Assert.assertEquals("Should delete 3 files", 3, deletedFiles.size()); + assertThat(deletedFiles).hasSize(3); } - @Test + @TestTemplate public void testConcurrentConflictingRowDelta() { commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -1164,12 +1065,12 @@ public void testConcurrentConflictingRowDelta() { .validateNoConflictingDataFiles() .commit(); - Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + assertThatThrownBy(() -> commit(table, rowDelta, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found new conflicting delete files"); } - @Test + @TestTemplate public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { commit(table, table.newAppend().appendFile(FILE_A), branch); @@ -1195,12 +1096,12 @@ public void testConcurrentConflictingRowDeltaWithoutAppendValidation() { .validateNoConflictingDataFiles() .commit(); - Assertions.assertThatThrownBy(() -> commit(table, rowDelta, branch)) + assertThatThrownBy(() -> commit(table, rowDelta, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found new conflicting delete files"); } - @Test + @TestTemplate public void testConcurrentNonConflictingRowDelta() { // change the spec to be partitioned by data table @@ -1278,7 +1179,7 @@ public void testConcurrentNonConflictingRowDelta() { validateBranchDeleteFiles(table, branch, deleteFile1, deleteFile2); } - @Test + @TestTemplate public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumber() { // change the spec to be partitioned by data table @@ -1328,7 +1229,7 @@ public void testConcurrentNonConflictingRowDeltaAndRewriteFilesWithSequenceNumbe validateBranchFiles(table, branch, dataFile2); } - @Test + @TestTemplate public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { table.updateProperties().set(TableProperties.MANIFEST_MIN_MERGE_COUNT, "1").commit(); // change the spec to be partitioned by data @@ -1376,9 +1277,9 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { table.refresh(); List dataManifests = latestSnapshot(table, branch).dataManifests(table.io()); - Assert.assertEquals("should have 1 data manifest", 1, dataManifests.size()); + assertThat(dataManifests).hasSize(1); ManifestFile mergedDataManifest = dataManifests.get(0); - Assert.assertEquals("Manifest seq number must match", 3L, mergedDataManifest.sequenceNumber()); + assertThat(mergedDataManifest.sequenceNumber()).isEqualTo(3); long currentSnapshotId = latestSnapshot(table, branch).snapshotId(); @@ -1391,7 +1292,7 @@ public void testRowDeltaAndRewriteFilesMergeManifestsWithSequenceNumber() { statuses(Status.ADDED, Status.DELETED)); } - @Test + @TestTemplate public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() { // change the spec to be partitioned by data table @@ -1433,12 +1334,12 @@ public void testConcurrentConflictingRowDeltaAndRewriteFilesWithSequenceNumber() commit(table, rowDelta, branch); - Assertions.assertThatThrownBy(() -> commit(table, rewriteFiles, branch)) + assertThatThrownBy(() -> commit(table, rewriteFiles, branch)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot commit, found new position delete for replaced data file"); } - @Test + @TestTemplate public void testRowDeltaCaseSensitivity() { commit(table, table.newAppend().appendFile(FILE_A).appendFile(FILE_A2), branch); @@ -1448,7 +1349,7 @@ public void testRowDeltaCaseSensitivity() { Expression conflictDetectionFilter = Expressions.equal(Expressions.bucket("dAtA", 16), 0); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .newRowDelta() @@ -1463,7 +1364,7 @@ public void testRowDeltaCaseSensitivity() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot find field 'dAtA'"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .newRowDelta() @@ -1480,7 +1381,7 @@ public void testRowDeltaCaseSensitivity() { .hasMessageStartingWith("Cannot find field 'dAtA'"); // binding should succeed and trigger the validation - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .newRowDelta() diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java index 3697678d63f0..1bd1761ffc60 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java @@ -260,7 +260,7 @@ public void testDeleteAndRenameColumnReassign() { MappedField newMapping = updated.find("id"); assertThat(newMapping).isNotNull(); assertThat(newMapping.id()).isEqualTo(idColumnId); - assertThat(newMapping.names()).isEqualTo(Sets.newHashSet("id", "data")); + assertThat(newMapping.names()).containsExactly("data", "id"); assertThat(newMapping.nestedMapping()).isNull(); MappedField updatedMapping = updated.find(startIdColumnId); @@ -283,8 +283,7 @@ public void testRenameAndAddColumnReassign() { NameMapping afterRename = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - assertThat(afterRename.find(startIdColumnId).names()) - .isEqualTo(Sets.newHashSet("id", "object_id")); + assertThat(afterRename.find(startIdColumnId).names()).containsExactly("id", "object_id"); // add a new column with the renamed column's old name // also, rename the original column again to ensure its names are handled correctly @@ -312,7 +311,7 @@ public void testRenameAndAddColumnReassign() { MappedField updatedMapping = updated.find(startIdColumnId); assertThat(updatedMapping).isNotNull(); assertThat(updatedMapping.id()).isEqualTo(startIdColumnId); - assertThat(updatedMapping.names()).isEqualTo(Sets.newHashSet("object_id", "oid")); + assertThat(updatedMapping.names()).containsExactly("oid", "object_id"); assertThat(updatedMapping.nestedMapping()).isNull(); } @@ -329,8 +328,7 @@ public void testRenameAndRenameColumnReassign() { NameMapping afterRename = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - assertThat(afterRename.find(startIdColumnId).names()) - .isEqualTo(Sets.newHashSet("id", "object_id")); + assertThat(afterRename.find(startIdColumnId).names()).containsExactly("id", "object_id"); // rename the data column to the renamed column's old name // also, rename the original column again to ensure its names are handled correctly @@ -348,14 +346,14 @@ public void testRenameAndRenameColumnReassign() { MappedField newMapping = updated.find("id"); assertThat(newMapping).isNotNull(); - assertThat(newMapping.names()).isEqualTo(Sets.newHashSet("id", "data")); + assertThat(newMapping.names()).containsExactly("data", "id"); assertThat(newMapping.id()).isEqualTo(idColumnId); assertThat(newMapping.nestedMapping()).isNull(); MappedField updatedMapping = updated.find(startIdColumnId); assertThat(updatedMapping).isNotNull(); assertThat(updatedMapping.id()).isEqualTo(startIdColumnId); - assertThat(updatedMapping.names()).isEqualTo(Sets.newHashSet("object_id", "oid")); + assertThat(updatedMapping.names()).containsExactly("oid", "object_id"); assertThat(updatedMapping.nestedMapping()).isNull(); } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java index 943ff9f51acd..2b91a408850e 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUpdate.java @@ -1601,9 +1601,8 @@ public void testAddNewIdentifierFieldColumns() { assertThat(newSchema.identifierFieldIds()) .as("add column then set as identifier should succeed") - .isEqualTo( - Sets.newHashSet( - newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId())); + .containsExactly( + newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId()); newSchema = new SchemaUpdate(SCHEMA, SCHEMA_LAST_COLUMN_ID) @@ -1614,9 +1613,8 @@ public void testAddNewIdentifierFieldColumns() { assertThat(newSchema.identifierFieldIds()) .as("set identifier then add column should succeed") - .isEqualTo( - Sets.newHashSet( - newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId())); + .containsExactly( + newSchema.findField("id").fieldId(), newSchema.findField("new_field").fieldId()); } @Test @@ -1686,9 +1684,8 @@ public void testAddDottedIdentifierFieldColumns() { assertThat(newSchema.identifierFieldIds()) .as("add a field with dot as identifier should succeed") - .isEqualTo( - Sets.newHashSet( - newSchema.findField("id").fieldId(), newSchema.findField("dot.field").fieldId())); + .containsExactly( + newSchema.findField("id").fieldId(), newSchema.findField("dot.field").fieldId()); } @Test @@ -1708,10 +1705,9 @@ public void testRemoveIdentifierFields() { assertThat(newSchema.identifierFieldIds()) .as("remove an identifier field should succeed") - .isEqualTo( - Sets.newHashSet( - newSchema.findField("new_field").fieldId(), - newSchema.findField("new_field2").fieldId())); + .containsExactly( + newSchema.findField("new_field").fieldId(), + newSchema.findField("new_field2").fieldId()); newSchema = new SchemaUpdate(newSchema, SCHEMA_LAST_COLUMN_ID) diff --git a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java index 86842b681278..7845f305e3c9 100644 --- a/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java +++ b/core/src/test/java/org/apache/iceberg/TestSequenceNumberForV2Table.java @@ -18,21 +18,26 @@ */ package org.apache.iceberg; +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.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestSequenceNumberForV2Table extends TableTestBase { +public class TestSequenceNumberForV2Table extends TestBase { - public TestSequenceNumberForV2Table() { - super(2); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2); } - @Test + @TestTemplate public void testRewrite() { table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); @@ -96,7 +101,7 @@ public void testRewrite() { } } - @Test + @TestTemplate public void testCommitConflict() { AppendFiles appendA = table.newFastAppend(); appendA.appendFile(FILE_A).apply(); @@ -105,7 +110,7 @@ public void testCommitConflict() { table.ops().failCommits(1); - Assertions.assertThatThrownBy(() -> table.newFastAppend().appendFile(FILE_B).commit()) + assertThatThrownBy(() -> table.newFastAppend().appendFile(FILE_B).commit()) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); @@ -144,7 +149,7 @@ public void testCommitConflict() { "Last sequence number should be 3", 3, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testRollBack() { table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); @@ -183,7 +188,7 @@ public void testRollBack() { "Last sequence number should be 3", 3, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testSingleTransaction() { Transaction txn = table.newTransaction(); txn.newAppend().appendFile(FILE_A).commit(); @@ -198,7 +203,7 @@ public void testSingleTransaction() { "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testConcurrentTransaction() { Transaction txn1 = table.newTransaction(); Transaction txn2 = table.newTransaction(); @@ -264,7 +269,7 @@ public void testConcurrentTransaction() { "Last sequence number should be 4", 4, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testMultipleOperationsTransaction() { Transaction txn = table.newTransaction(); txn.newFastAppend().appendFile(FILE_A).commit(); @@ -298,7 +303,7 @@ public void testMultipleOperationsTransaction() { "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testExpirationInTransaction() { table.newFastAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); @@ -310,7 +315,7 @@ public void testExpirationInTransaction() { V2Assert.assertEquals( "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); V2Assert.assertEquals( - "Should be 1 manifest list", 1, listManifestLists(table.location()).size()); + "Should be 1 manifest list", 1, listManifestLists(new File(table.location())).size()); table.newAppend().appendFile(FILE_B).commit(); Snapshot snap2 = table.currentSnapshot(); @@ -322,7 +327,7 @@ public void testExpirationInTransaction() { V2Assert.assertEquals( "Last sequence number should be 2", 2, readMetadata().lastSequenceNumber()); V2Assert.assertEquals( - "Should be 2 manifest lists", 2, listManifestLists(table.location()).size()); + "Should be 2 manifest lists", 2, listManifestLists(new File(table.location())).size()); Transaction txn = table.newTransaction(); txn.expireSnapshots().expireSnapshotId(commitId1).commit(); @@ -332,10 +337,10 @@ public void testExpirationInTransaction() { V2Assert.assertEquals( "Should be 1 manifest list as 1 was deleted", 1, - listManifestLists(table.location()).size()); + listManifestLists(new File(table.location())).size()); } - @Test + @TestTemplate public void testTransactionFailure() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); Snapshot snap1 = table.currentSnapshot(); @@ -359,7 +364,7 @@ public void testTransactionFailure() { Transaction txn = table.newTransaction(); txn.newAppend().appendFile(FILE_C).commit(); - Assertions.assertThatThrownBy(txn::commitTransaction) + assertThatThrownBy(txn::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessage("Injected failure"); @@ -367,7 +372,7 @@ public void testTransactionFailure() { "Last sequence number should be 1", 1, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testCherryPicking() { table.newAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); @@ -415,7 +420,7 @@ public void testCherryPicking() { "Last sequence number should be 4", 4, readMetadata().lastSequenceNumber()); } - @Test + @TestTemplate public void testCherryPickFastForward() { table.newAppend().appendFile(FILE_A).commit(); Snapshot snap1 = table.currentSnapshot(); diff --git a/core/src/test/java/org/apache/iceberg/TestSingleValueParser.java b/core/src/test/java/org/apache/iceberg/TestSingleValueParser.java index e04ba440ae3f..cc1578b0e081 100644 --- a/core/src/test/java/org/apache/iceberg/TestSingleValueParser.java +++ b/core/src/test/java/org/apache/iceberg/TestSingleValueParser.java @@ -20,14 +20,15 @@ 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 java.util.Locale; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSingleValueParser { @@ -114,22 +115,18 @@ public void testValidDefaults() throws IOException { public void testInvalidFixed() { Type expectedType = Types.FixedType.ofLength(2); String defaultJson = "\"111ff\""; - Exception exception = - Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default fixed[2] value")); + assertThatThrownBy(() -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse default fixed[2] value"); } @Test public void testInvalidUUID() { Type expectedType = Types.UUIDType.get(); String defaultJson = "\"eb26bdb1-a1d8-4aa6-990e-da940875492c-abcde\""; - Exception exception = - Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue(exception.getMessage().startsWith("Cannot parse default as a uuid value")); + assertThatThrownBy(() -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse default as a uuid value"); } @Test @@ -137,36 +134,27 @@ public void testInvalidMap() { Type expectedType = Types.MapType.ofOptional(1, 2, Types.IntegerType.get(), Types.StringType.get()); String defaultJson = "{\"keys\": [1, 2, 3], \"values\": [\"foo\", \"bar\"]}"; - Exception exception = - Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue( - exception.getMessage().startsWith("Cannot parse default as a map value")); + assertThatThrownBy(() -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse default as a map value"); } @Test public void testInvalidDecimal() { Type expectedType = Types.DecimalType.of(5, 2); String defaultJson = "123.456"; - Exception exception = - Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue( - exception.getMessage().startsWith("Cannot parse default as a decimal(5, 2) value")); + assertThatThrownBy(() -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse default as a decimal(5, 2) value"); } @Test public void testInvalidTimestamptz() { Type expectedType = Types.TimestampType.withZone(); String defaultJson = "\"2007-12-03T10:15:30+01:00\""; - Exception exception = - Assert.assertThrows( - IllegalArgumentException.class, - () -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)); - Assert.assertTrue( - exception.getMessage().startsWith("Cannot parse default as a timestamptz value")); + assertThatThrownBy(() -> defaultValueParseAndUnParseRoundTrip(expectedType, defaultJson)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot parse default as a timestamptz value"); } // serialize to json and deserialize back should return the same result @@ -176,6 +164,6 @@ private static String defaultValueParseAndUnParseRoundTrip(Type type, String def } private static void jsonStringEquals(String s1, String s2) throws IOException { - Assert.assertEquals(JsonUtil.mapper().readTree(s1), JsonUtil.mapper().readTree(s2)); + assertThat(JsonUtil.mapper().readTree(s2)).isEqualTo(JsonUtil.mapper().readTree(s1)); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index 8fbc4e11fbc3..a3ba69a808b3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -24,26 +24,27 @@ import static org.apache.iceberg.expressions.Expressions.truncate; 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; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SortOrderUtil; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestSortOrder { // column ids will be reassigned during table creation @@ -69,59 +70,52 @@ public class TestSortOrder { required(30, "ext", Types.StringType.get()), required(42, "Ext1", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; + private File tableDir = null; - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - private final int formatVersion; - - public TestSortOrder(int formatVersion) { - this.formatVersion = formatVersion; - } + @Parameter private int formatVersion; - @Before + @BeforeEach public void setupTableDir() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); } - @After + @AfterEach public void cleanupTables() { TestTables.clearTables(); } - @Test + @TestTemplate public void testSortOrderBuilder() { - Assert.assertEquals( - "Should be able to build unsorted order", - SortOrder.unsorted(), - SortOrder.builderFor(SCHEMA).withOrderId(0).build()); + assertThat(SortOrder.builderFor(SCHEMA).withOrderId(0).build()).isEqualTo(SortOrder.unsorted()); - Assertions.assertThatThrownBy( - () -> SortOrder.builderFor(SCHEMA).asc("data").withOrderId(0).build()) + assertThatThrownBy(() -> SortOrder.builderFor(SCHEMA).asc("data").withOrderId(0).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Sort order ID 0 is reserved for unsorted order"); - Assertions.assertThatThrownBy(() -> SortOrder.builderFor(SCHEMA).withOrderId(1).build()) + assertThatThrownBy(() -> SortOrder.builderFor(SCHEMA).withOrderId(1).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Unsorted order ID must be 0"); } - @Test + @TestTemplate public void testDefaultOrder() { PartitionSpec spec = PartitionSpec.unpartitioned(); TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, formatVersion); - Assert.assertEquals("Expected 1 sort order", 1, table.sortOrders().size()); + assertThat(table.sortOrders()).hasSize(1); SortOrder actualOrder = table.sortOrder(); - Assert.assertEquals("Order ID must match", 0, actualOrder.orderId()); - Assert.assertTrue("Order must unsorted", actualOrder.isUnsorted()); + assertThat(actualOrder.orderId()).isEqualTo(0); + assertThat(actualOrder.isUnsorted()).isTrue(); } - @Test + @TestTemplate public void testFreshIds() { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).withSpecId(5).identity("data").build(); SortOrder order = @@ -133,20 +127,16 @@ public void testFreshIds() { TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, formatVersion); - Assert.assertEquals("Expected 1 sort order", 1, table.sortOrders().size()); - Assert.assertTrue( - "Order ID must be fresh", - table.sortOrders().containsKey(TableMetadata.INITIAL_SORT_ORDER_ID)); + assertThat(table.sortOrders()).hasSize(1).containsKey(TableMetadata.INITIAL_SORT_ORDER_ID); SortOrder actualOrder = table.sortOrder(); - Assert.assertEquals( - "Order ID must be fresh", TableMetadata.INITIAL_SORT_ORDER_ID, actualOrder.orderId()); - Assert.assertEquals("Order must have 2 fields", 2, actualOrder.fields().size()); - Assert.assertEquals("Field id must be fresh", 8, actualOrder.fields().get(0).sourceId()); - Assert.assertEquals("Field id must be fresh", 2, actualOrder.fields().get(1).sourceId()); + assertThat(actualOrder.orderId()).isEqualTo(TableMetadata.INITIAL_SORT_ORDER_ID); + assertThat(actualOrder.fields()).hasSize(2); + assertThat(actualOrder.fields().get(0).sourceId()).isEqualTo(8); + assertThat(actualOrder.fields().get(1).sourceId()).isEqualTo(2); } - @Test + @TestTemplate public void testCompatibleOrders() { SortOrder order1 = SortOrder.builderFor(SCHEMA).withOrderId(9).asc("s.id", NULLS_LAST).build(); @@ -175,37 +165,37 @@ public void testCompatibleOrders() { SortOrder.builderFor(SCHEMA).withOrderId(11).desc("s.id", NULLS_LAST).build(); // an unsorted order satisfies only itself - Assert.assertTrue(SortOrder.unsorted().satisfies(SortOrder.unsorted())); - Assert.assertFalse(SortOrder.unsorted().satisfies(order1)); - Assert.assertFalse(SortOrder.unsorted().satisfies(order2)); - Assert.assertFalse(SortOrder.unsorted().satisfies(order3)); - Assert.assertFalse(SortOrder.unsorted().satisfies(order4)); - Assert.assertFalse(SortOrder.unsorted().satisfies(order5)); + assertThat(SortOrder.unsorted().satisfies(SortOrder.unsorted())).isTrue(); + assertThat(SortOrder.unsorted().satisfies(order1)).isFalse(); + assertThat(SortOrder.unsorted().satisfies(order2)).isFalse(); + assertThat(SortOrder.unsorted().satisfies(order3)).isFalse(); + assertThat(SortOrder.unsorted().satisfies(order4)).isFalse(); + assertThat(SortOrder.unsorted().satisfies(order5)).isFalse(); // any ordering satisfies an unsorted ordering - Assert.assertTrue(order1.satisfies(SortOrder.unsorted())); - Assert.assertTrue(order2.satisfies(SortOrder.unsorted())); - Assert.assertTrue(order3.satisfies(SortOrder.unsorted())); - Assert.assertTrue(order4.satisfies(SortOrder.unsorted())); - Assert.assertTrue(order5.satisfies(SortOrder.unsorted())); + assertThat(order1.satisfies(SortOrder.unsorted())).isTrue(); + assertThat(order2.satisfies(SortOrder.unsorted())).isTrue(); + assertThat(order3.satisfies(SortOrder.unsorted())).isTrue(); + assertThat(order4.satisfies(SortOrder.unsorted())).isTrue(); + assertThat(order5.satisfies(SortOrder.unsorted())).isTrue(); // order1 has the same fields but different sort direction compared to order5 - Assert.assertFalse(order1.satisfies(order5)); + assertThat(order1.satisfies(order5)).isFalse(); // order2 has more fields than order1 and is compatible - Assert.assertTrue(order2.satisfies(order1)); + assertThat(order2.satisfies(order1)).isTrue(); // order2 has more fields than order5 but is incompatible - Assert.assertFalse(order2.satisfies(order5)); + assertThat(order2.satisfies(order5)).isFalse(); // order2 has the same fields but different null order compared to order3 - Assert.assertFalse(order2.satisfies(order3)); + assertThat(order2.satisfies(order3)).isFalse(); // order2 has the same fields but different sort direction compared to order4 - Assert.assertFalse(order2.satisfies(order4)); + assertThat(order2.satisfies(order4)).isFalse(); // order1 has fewer fields than order2 and is incompatible - Assert.assertFalse(order1.satisfies(order2)); + assertThat(order1.satisfies(order2)).isFalse(); } - @Test + @TestTemplate public void testSatisfiesTruncateFieldOrder() { SortOrder id = SortOrder.builderFor(SCHEMA).asc("data", NULLS_LAST).build(); SortOrder truncate4 = @@ -213,36 +203,35 @@ public void testSatisfiesTruncateFieldOrder() { SortOrder truncate2 = SortOrder.builderFor(SCHEMA).asc(Expressions.truncate("data", 2), NULLS_LAST).build(); - Assert.assertTrue(id.satisfies(truncate2)); - Assert.assertTrue(id.satisfies(truncate4)); - Assert.assertFalse(truncate2.satisfies(id)); - Assert.assertFalse(truncate4.satisfies(id)); - Assert.assertTrue(truncate4.satisfies(truncate2)); - Assert.assertFalse(truncate2.satisfies(truncate4)); + assertThat(id.satisfies(truncate2)).isTrue(); + assertThat(truncate2.satisfies(id)).isFalse(); + assertThat(truncate4.satisfies(id)).isFalse(); + assertThat(truncate4.satisfies(truncate2)).isTrue(); + assertThat(truncate2.satisfies(truncate4)).isFalse(); } - @Test + @TestTemplate public void testSatisfiesDateFieldOrder() { SortOrder id = SortOrder.builderFor(SCHEMA).asc("d", NULLS_LAST).build(); SortOrder year = SortOrder.builderFor(SCHEMA).asc(Expressions.year("d"), NULLS_LAST).build(); SortOrder month = SortOrder.builderFor(SCHEMA).asc(Expressions.month("d"), NULLS_LAST).build(); SortOrder day = SortOrder.builderFor(SCHEMA).asc(Expressions.day("d"), NULLS_LAST).build(); - Assert.assertTrue(id.satisfies(year)); - Assert.assertTrue(id.satisfies(month)); - Assert.assertTrue(id.satisfies(day)); - Assert.assertFalse(year.satisfies(id)); - Assert.assertFalse(month.satisfies(id)); - Assert.assertFalse(day.satisfies(id)); - Assert.assertTrue(day.satisfies(year)); - Assert.assertTrue(day.satisfies(month)); - Assert.assertTrue(month.satisfies(year)); - Assert.assertFalse(month.satisfies(day)); - Assert.assertFalse(year.satisfies(day)); - Assert.assertFalse(year.satisfies(month)); + assertThat(id.satisfies(year)).isTrue(); + assertThat(id.satisfies(month)).isTrue(); + assertThat(id.satisfies(day)).isTrue(); + assertThat(year.satisfies(id)).isFalse(); + assertThat(month.satisfies(id)).isFalse(); + assertThat(day.satisfies(id)).isFalse(); + assertThat(day.satisfies(year)).isTrue(); + assertThat(day.satisfies(month)).isTrue(); + assertThat(month.satisfies(year)).isTrue(); + assertThat(month.satisfies(day)).isFalse(); + assertThat(year.satisfies(day)).isFalse(); + assertThat(year.satisfies(month)).isFalse(); } - @Test + @TestTemplate public void testSatisfiesTimestampFieldOrder() { SortOrder id = SortOrder.builderFor(SCHEMA).asc("ts", NULLS_LAST).build(); SortOrder year = SortOrder.builderFor(SCHEMA).asc(Expressions.year("ts"), NULLS_LAST).build(); @@ -250,41 +239,40 @@ public void testSatisfiesTimestampFieldOrder() { SortOrder day = SortOrder.builderFor(SCHEMA).asc(Expressions.day("ts"), NULLS_LAST).build(); SortOrder hour = SortOrder.builderFor(SCHEMA).asc(Expressions.hour("ts"), NULLS_LAST).build(); - Assert.assertTrue(id.satisfies(year)); - Assert.assertTrue(id.satisfies(month)); - Assert.assertTrue(id.satisfies(day)); - Assert.assertTrue(id.satisfies(hour)); - Assert.assertFalse(year.satisfies(id)); - Assert.assertFalse(month.satisfies(id)); - Assert.assertFalse(day.satisfies(id)); - Assert.assertFalse(hour.satisfies(id)); - Assert.assertTrue(hour.satisfies(year)); - Assert.assertTrue(hour.satisfies(month)); - Assert.assertTrue(hour.satisfies(day)); - Assert.assertTrue(day.satisfies(year)); - Assert.assertTrue(day.satisfies(month)); - Assert.assertFalse(day.satisfies(hour)); - Assert.assertTrue(month.satisfies(year)); - Assert.assertFalse(month.satisfies(day)); - Assert.assertFalse(month.satisfies(hour)); - Assert.assertFalse(year.satisfies(day)); - Assert.assertFalse(year.satisfies(month)); - Assert.assertFalse(year.satisfies(hour)); + assertThat(id.satisfies(year)).isTrue(); + assertThat(id.satisfies(month)).isTrue(); + assertThat(id.satisfies(day)).isTrue(); + assertThat(id.satisfies(hour)).isTrue(); + assertThat(year.satisfies(id)).isFalse(); + assertThat(month.satisfies(id)).isFalse(); + assertThat(day.satisfies(id)).isFalse(); + assertThat(hour.satisfies(id)).isFalse(); + assertThat(hour.satisfies(year)).isTrue(); + assertThat(hour.satisfies(month)).isTrue(); + assertThat(hour.satisfies(day)).isTrue(); + assertThat(day.satisfies(year)).isTrue(); + assertThat(day.satisfies(month)).isTrue(); + assertThat(day.satisfies(hour)).isFalse(); + assertThat(month.satisfies(year)).isTrue(); + assertThat(month.satisfies(day)).isFalse(); + assertThat(month.satisfies(hour)).isFalse(); + assertThat(year.satisfies(day)).isFalse(); + assertThat(year.satisfies(month)).isFalse(); + assertThat(year.satisfies(hour)).isFalse(); } - @Test + @TestTemplate public void testSameOrder() { SortOrder order1 = SortOrder.builderFor(SCHEMA).withOrderId(9).asc("s.id", NULLS_LAST).build(); SortOrder order2 = SortOrder.builderFor(SCHEMA).withOrderId(10).asc("s.id", NULLS_LAST).build(); // orders have different ids but are logically the same - Assert.assertNotEquals("Orders must not be equal", order1, order2); - Assert.assertTrue("Orders must be equivalent", order1.sameOrder(order2)); - Assert.assertTrue("Orders must be equivalent", order2.sameOrder(order1)); + assertThat(order2).isNotEqualTo(order1); + assertThat(order2.fields()).isEqualTo(order1.fields()); } - @Test + @TestTemplate public void testSchemaEvolutionWithSortOrder() { PartitionSpec spec = PartitionSpec.unpartitioned(); SortOrder order = @@ -295,14 +283,13 @@ public void testSchemaEvolutionWithSortOrder() { table.updateSchema().renameColumn("s.id", "s.id2").commit(); SortOrder actualOrder = table.sortOrder(); - Assert.assertEquals( - "Order ID must match", TableMetadata.INITIAL_SORT_ORDER_ID, actualOrder.orderId()); - Assert.assertEquals("Order must have 2 fields", 2, actualOrder.fields().size()); - Assert.assertEquals("Field id must match", 8, actualOrder.fields().get(0).sourceId()); - Assert.assertEquals("Field id must match", 2, actualOrder.fields().get(1).sourceId()); + assertThat(actualOrder.orderId()).isEqualTo(TableMetadata.INITIAL_SORT_ORDER_ID); + assertThat(actualOrder.fields()).hasSize(2); + assertThat(actualOrder.fields().get(0).sourceId()).isEqualTo(8); + assertThat(actualOrder.fields().get(1).sourceId()).isEqualTo(2); } - @Test + @TestTemplate public void testColumnDropWithSortOrder() { PartitionSpec spec = PartitionSpec.unpartitioned(); @@ -316,16 +303,14 @@ public void testColumnDropWithSortOrder() { table.updateSchema().deleteColumn("id").commit(); SortOrder actualOrder = table.sortOrder(); - Assert.assertEquals( - "Order ID must match", TableMetadata.INITIAL_SORT_ORDER_ID + 1, actualOrder.orderId()); - Assert.assertEquals( - "Schema must have one less column", initialColSize - 1, table.schema().columns().size()); + assertThat(actualOrder.orderId()).isEqualTo(TableMetadata.INITIAL_SORT_ORDER_ID + 1); + assertThat(table.schema().columns()).hasSize(initialColSize - 1); // ensure that the table metadata can be serialized and reloaded with an invalid order TableMetadataParser.fromJson(TableMetadataParser.toJson(table.ops().current())); } - @Test + @TestTemplate public void testIncompatibleSchemaEvolutionWithSortOrder() { PartitionSpec spec = PartitionSpec.unpartitioned(); SortOrder order = @@ -333,26 +318,26 @@ public void testIncompatibleSchemaEvolutionWithSortOrder() { TestTables.TestTable table = TestTables.create(tableDir, "test", SCHEMA, spec, order, formatVersion); - Assertions.assertThatThrownBy(() -> table.updateSchema().deleteColumn("s.id").commit()) + assertThatThrownBy(() -> table.updateSchema().deleteColumn("s.id").commit()) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot find source column for sort field"); } - @Test + @TestTemplate public void testEmptySortOrder() { SortOrder order = SortOrder.builderFor(SCHEMA).build(); - Assert.assertEquals("Order must be unsorted", SortOrder.unsorted(), order); + assertThat(order).isEqualTo(SortOrder.unsorted()); } - @Test + @TestTemplate public void testSortedColumnNames() { SortOrder order = SortOrder.builderFor(SCHEMA).withOrderId(10).asc("s.id").desc(truncate("data", 10)).build(); Set sortedCols = SortOrderUtil.orderPreservingSortedColumns(order); - Assert.assertEquals(ImmutableSet.of("s.id", "data"), sortedCols); + assertThat(sortedCols).containsExactly("s.id", "data"); } - @Test + @TestTemplate public void testPreservingOrderSortedColumnNames() { SortOrder order = SortOrder.builderFor(SCHEMA) @@ -361,13 +346,13 @@ public void testPreservingOrderSortedColumnNames() { .desc(truncate("data", 10)) .build(); Set sortedCols = SortOrderUtil.orderPreservingSortedColumns(order); - Assert.assertEquals(ImmutableSet.of("data"), sortedCols); + assertThat(sortedCols).containsExactly("data"); } - @Test + @TestTemplate public void testCaseSensitiveSortedColumnNames() { String fieldName = "ext1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> SortOrder.builderFor(SCHEMA) .caseSensitive(true) @@ -380,6 +365,6 @@ public void testCaseSensitiveSortedColumnNames() { SortOrder ext1 = SortOrder.builderFor(SCHEMA).caseSensitive(false).withOrderId(10).asc("ext1").build(); SortField sortField = ext1.fields().get(0); - Assert.assertEquals(sortField.sourceId(), SCHEMA.findField("Ext1").fieldId()); + assertThat(SCHEMA.findField("Ext1").fieldId()).isEqualTo(sortField.sourceId()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrderParser.java b/core/src/test/java/org/apache/iceberg/TestSortOrderParser.java index 5aba9e9ad533..adcc130c2390 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrderParser.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrderParser.java @@ -20,18 +20,23 @@ import static org.apache.iceberg.NullOrder.NULLS_FIRST; import static org.apache.iceberg.SortDirection.DESC; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.transforms.UnknownTransform; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestSortOrderParser extends TableTestBase { - public TestSortOrderParser() { - super(1); +@ExtendWith(ParameterizedTestExtension.class) +public class TestSortOrderParser extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1); } - @Test + @TestTemplate public void testUnknownTransforms() { String jsonString = "{\n" @@ -46,16 +51,18 @@ public void testUnknownTransforms() { SortOrder order = SortOrderParser.fromJson(table.schema(), jsonString); - Assert.assertEquals(10, order.orderId()); - Assert.assertEquals(1, order.fields().size()); - Assertions.assertThat(order.fields().get(0).transform()).isInstanceOf(UnknownTransform.class); - Assert.assertEquals("custom_transform", order.fields().get(0).transform().toString()); - Assert.assertEquals(2, order.fields().get(0).sourceId()); - Assert.assertEquals(DESC, order.fields().get(0).direction()); - Assert.assertEquals(NULLS_FIRST, order.fields().get(0).nullOrder()); + assertThat(order.orderId()).isEqualTo(10); + assertThat(order.fields()).hasSize(1); + assertThat(order.fields().get(0).transform()) + .isInstanceOf(UnknownTransform.class) + .asString() + .isEqualTo("custom_transform"); + assertThat(order.fields().get(0).sourceId()).isEqualTo(2); + assertThat(order.fields().get(0).direction()).isEqualTo(DESC); + assertThat(order.fields().get(0).nullOrder()).isEqualTo(NULLS_FIRST); } - @Test + @TestTemplate public void invalidSortDirection() { String jsonString = "{\n" @@ -68,7 +75,7 @@ public void invalidSortDirection() { + " } ]\n" + "}"; - Assertions.assertThatThrownBy(() -> SortOrderParser.fromJson(table.schema(), jsonString)) + assertThatThrownBy(() -> SortOrderParser.fromJson(table.schema(), jsonString)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid sort direction: invalid"); } diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index ab83b277509c..04bb2ae215d8 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -19,9 +19,14 @@ package org.apache.iceberg; 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 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.UUID; import java.util.stream.Collectors; @@ -33,17 +38,13 @@ 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.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestSplitPlanning extends TableTestBase { +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 TestSplitPlanning extends TestBase { private static final Configuration CONF = new Configuration(); private static final HadoopTables TABLES = new HadoopTables(CONF); @@ -51,22 +52,19 @@ public class TestSplitPlanning extends TableTestBase { new Schema( optional(1, "id", Types.IntegerType.get()), optional(2, "data", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private Table table = null; + @TempDir private Path temp; - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } + private Table table = null; - public TestSplitPlanning(int formatVersion) { - super(formatVersion); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - File tableDir = temp.newFolder(); + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); String tableLocation = tableDir.toURI().toString(); table = TABLES.create(SCHEMA, tableLocation); table @@ -77,19 +75,19 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testBasicSplitPlanning() { List files128Mb = newFiles(4, 128 * 1024 * 1024); appendFiles(files128Mb); // we expect 4 bins since split size is 128MB and we have 4 files 128MB each - Assert.assertEquals(4, Iterables.size(table.newScan().planTasks())); + assertThat(table.newScan().planTasks()).hasSize(4); List files32Mb = newFiles(16, 32 * 1024 * 1024); appendFiles(files32Mb); // we expect 8 bins after we add 16 files 32MB each as they will form additional 4 bins - Assert.assertEquals(8, Iterables.size(table.newScan().planTasks())); + assertThat(table.newScan().planTasks()).hasSize(8); } - @Test + @TestTemplate public void testSplitPlanningWithSmallFiles() { List files60Mb = newFiles(50, 60 * 1024 * 1024); List files5Kb = newFiles(370, 5 * 1024); @@ -101,10 +99,10 @@ public void testSplitPlanningWithSmallFiles() { // as "read.split.open-file-cost" is 4MB, each of the original 25 bins will get at most 2 files // so 50 of 370 files will be packed into the existing 25 bins and the remaining 320 files // will form additional 10 bins, resulting in 35 bins in total - Assert.assertEquals(35, Iterables.size(table.newScan().planTasks())); + assertThat(table.newScan().planTasks()).hasSize(35); } - @Test + @TestTemplate public void testSplitPlanningWithNoMinWeight() { table.updateProperties().set(TableProperties.SPLIT_OPEN_FILE_COST, "0").commit(); List files60Mb = newFiles(2, 60 * 1024 * 1024); @@ -112,30 +110,30 @@ public void testSplitPlanningWithNoMinWeight() { Iterable files = Iterables.concat(files60Mb, files5Kb); appendFiles(files); // all small files will be packed into one bin as "read.split.open-file-cost" is set to 0 - Assert.assertEquals(1, Iterables.size(table.newScan().planTasks())); + assertThat(table.newScan().planTasks()).hasSize(1); } - @Test + @TestTemplate public void testSplitPlanningWithOverridenSize() { List files128Mb = newFiles(4, 128 * 1024 * 1024); appendFiles(files128Mb); // we expect 2 bins since we are overriding split size in scan with 256MB TableScan scan = table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(256L * 1024 * 1024)); - Assert.assertEquals(2, Iterables.size(scan.planTasks())); + assertThat(scan.planTasks()).hasSize(2); } - @Test + @TestTemplate public void testSplitPlanningWithOverriddenSizeForMetadataJsonFile() { List files8Mb = newFiles(32, 8 * 1024 * 1024, FileFormat.METADATA); appendFiles(files8Mb); // we expect 16 bins since we are overriding split size in scan with 16MB TableScan scan = table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(16L * 1024 * 1024)); - Assert.assertEquals(16, Iterables.size(scan.planTasks())); + assertThat(scan.planTasks()).hasSize(16); } - @Test + @TestTemplate public void testSplitPlanningWithOverriddenSizeForLargeMetadataJsonFile() { List files128Mb = newFiles(4, 128 * 1024 * 1024, FileFormat.METADATA); appendFiles(files128Mb); @@ -143,10 +141,10 @@ public void testSplitPlanningWithOverriddenSizeForLargeMetadataJsonFile() { // splittable TableScan scan = table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(8L * 1024 * 1024)); - Assert.assertEquals(4, Iterables.size(scan.planTasks())); + assertThat(scan.planTasks()).hasSize(4); } - @Test + @TestTemplate public void testSplitPlanningWithOverridenLookback() { List files120Mb = newFiles(1, 120 * 1024 * 1024); List file128Mb = newFiles(1, 128 * 1024 * 1024); @@ -155,15 +153,15 @@ public void testSplitPlanningWithOverridenLookback() { // we expect 2 bins from non-overriden table properties TableScan scan = table.newScan().option(TableProperties.SPLIT_LOOKBACK, "1"); CloseableIterable tasks = scan.planTasks(); - Assert.assertEquals(2, Iterables.size(tasks)); + assertThat(tasks).hasSize(2); // since lookback was overridden to 1, we expect the first bin to be the largest of the two. CombinedScanTask combinedScanTask = tasks.iterator().next(); FileScanTask task = combinedScanTask.files().iterator().next(); - Assert.assertEquals(128 * 1024 * 1024, task.length()); + assertThat(task.length()).isEqualTo(128 * 1024 * 1024); } - @Test + @TestTemplate public void testSplitPlanningWithOverridenOpenCostSize() { List files16Mb = newFiles(16, 16 * 1024 * 1024); appendFiles(files16Mb); @@ -173,18 +171,18 @@ public void testSplitPlanningWithOverridenOpenCostSize() { table .newScan() .option(TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(32L * 1024 * 1024)); - Assert.assertEquals(4, Iterables.size(scan.planTasks())); + assertThat(scan.planTasks()).hasSize(4); } - @Test + @TestTemplate public void testSplitPlanningWithNegativeValues() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(-10)).planTasks()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Split size must be > 0: -10"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .newScan() @@ -193,7 +191,7 @@ public void testSplitPlanningWithNegativeValues() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Split planning lookback must be > 0: -10"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> table .newScan() @@ -203,7 +201,7 @@ public void testSplitPlanningWithNegativeValues() { .hasMessage("File open cost must be >= 0: -10"); } - @Test + @TestTemplate public void testSplitPlanningWithOffsets() { List files16Mb = newFiles(16, 16 * 1024 * 1024, 2); appendFiles(files16Mb); @@ -212,11 +210,10 @@ public void testSplitPlanningWithOffsets() { // 1 split per row group TableScan scan = table.newScan().option(TableProperties.SPLIT_SIZE, String.valueOf(10L * 1024 * 1024)); - Assert.assertEquals( - "We should get one task per row group", 32, Iterables.size(scan.planTasks())); + assertThat(scan.planTasks()).hasSize(32); } - @Test + @TestTemplate public void testSplitPlanningWithOffsetsUnableToSplit() { List files16Mb = newFiles(16, 16 * 1024 * 1024, 2); appendFiles(files16Mb); @@ -228,11 +225,10 @@ public void testSplitPlanningWithOffsetsUnableToSplit() { .newScan() .option(TableProperties.SPLIT_OPEN_FILE_COST, String.valueOf(0)) .option(TableProperties.SPLIT_SIZE, String.valueOf(4L * 1024 * 1024)); - Assert.assertEquals( - "We should still only get 2 tasks per file", 32, Iterables.size(scan.planTasks())); + assertThat(scan.planTasks()).hasSize(32); } - @Test + @TestTemplate public void testBasicSplitPlanningDeleteFiles() { table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); List files128Mb = newDeleteFiles(4, 128 * 1024 * 1024); @@ -240,14 +236,14 @@ public void testBasicSplitPlanningDeleteFiles() { PositionDeletesTable posDeletesTable = new PositionDeletesTable(table); // we expect 4 bins since split size is 128MB and we have 4 files 128MB each - Assert.assertEquals(4, Iterables.size(posDeletesTable.newBatchScan().planTasks())); + assertThat(posDeletesTable.newBatchScan().planTasks()).hasSize(4); List files32Mb = newDeleteFiles(16, 32 * 1024 * 1024); appendDeleteFiles(files32Mb); // we expect 8 bins after we add 16 files 32MB each as they will form additional 4 bins - Assert.assertEquals(8, Iterables.size(posDeletesTable.newBatchScan().planTasks())); + assertThat(posDeletesTable.newBatchScan().planTasks()).hasSize(8); } - @Test + @TestTemplate public void testBasicSplitPlanningDeleteFilesWithSplitOffsets() { table.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit(); List files128Mb = newDeleteFiles(4, 128 * 1024 * 1024, 8); @@ -266,19 +262,19 @@ public void testBasicSplitPlanningDeleteFilesWithSplitOffsets() { long previousOffset = -1; for (ScanTask task : group.tasks()) { tasksPerGroup++; - Assert.assertTrue(task instanceof SplitPositionDeletesScanTask); + assertThat(task).isInstanceOf(SplitPositionDeletesScanTask.class); SplitPositionDeletesScanTask splitPosDelTask = (SplitPositionDeletesScanTask) task; if (previousOffset != -1) { - Assert.assertEquals(splitPosDelTask.start(), previousOffset); + assertThat(previousOffset).isEqualTo(splitPosDelTask.start()); } previousOffset = splitPosDelTask.start() + splitPosDelTask.length(); } - Assert.assertEquals("Should have 1 task as result of task merge", 1, tasksPerGroup); + assertThat(tasksPerGroup).isEqualTo(1); totalTaskGroups++; } // we expect 8 bins since split size is 64MB - Assert.assertEquals(8, totalTaskGroups); + assertThat(totalTaskGroups).isEqualTo(8); } catch (IOException e) { throw new RuntimeException(e); } diff --git a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java index 6d38ab427315..28695e2fffe0 100644 --- a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java +++ b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java @@ -20,19 +20,18 @@ import static org.apache.iceberg.expressions.Expressions.bucket; import static org.apache.iceberg.expressions.Expressions.equal; +import static org.assertj.core.api.Assertions.assertThat; +import java.util.Arrays; import java.util.List; -import java.util.Optional; -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.Assert; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestV1ToV2RowDeltaDelete extends TableTestBase { +public class TestV1ToV2RowDeltaDelete extends TestBase { - public TestV1ToV2RowDeltaDelete() { - super(1 /* table format version */); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1); } static final DeleteFile FILE_A_POS_1 = @@ -55,20 +54,18 @@ public TestV1ToV2RowDeltaDelete() { private void verifyManifestSequenceNumber( ManifestFile mf, long sequenceNum, long minSequenceNum) { - Assert.assertEquals( - "sequence number should be " + sequenceNum, mf.sequenceNumber(), sequenceNum); - Assert.assertEquals( - "min sequence number should be " + minSequenceNum, mf.minSequenceNumber(), minSequenceNum); + assertThat(sequenceNum).isEqualTo(mf.sequenceNumber()); + assertThat(minSequenceNum).isEqualTo(mf.minSequenceNumber()); } - @Test + @TestTemplate public void testPartitionedTableWithPartitionEqDeletes() { table.newAppend().appendFile(FILE_A).appendFile(FILE_B).appendFile(FILE_C).commit(); List dataManifests = table.currentSnapshot().dataManifests(table.io()); List deleteManifests = table.currentSnapshot().deleteManifests(table.io()); - Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); - Assert.assertEquals("Should have zero delete manifest file", 0, deleteManifests.size()); + assertThat(dataManifests).hasSize(1); + assertThat(deleteManifests).isEmpty(); ManifestFile dataManifest = dataManifests.get(0); verifyManifestSequenceNumber(dataManifest, 0, 0); @@ -81,56 +78,51 @@ public void testPartitionedTableWithPartitionEqDeletes() { dataManifests = table.currentSnapshot().dataManifests(ops.io()); deleteManifests = table.currentSnapshot().deleteManifests(ops.io()); - Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); - Assert.assertEquals("Should have one delete manifest file", 1, deleteManifests.size()); - Assert.assertEquals(dataManifest, dataManifests.get(0)); // data manifest not changed + assertThat(dataManifests).hasSize(1).first().isEqualTo(dataManifest); + assertThat(deleteManifests).hasSize(1); ManifestFile deleteManifest = deleteManifests.get(0); verifyManifestSequenceNumber(deleteManifest, 1, 1); - List tasks = Lists.newArrayList(table.newScan().planFiles().iterator()); - Assert.assertEquals("Should have three task", 3, tasks.size()); - Optional task = - tasks.stream().filter(t -> t.file().path().equals(FILE_A.path())).findFirst(); - Assert.assertTrue(task.isPresent()); - Assert.assertEquals("Should have one associated delete file", 1, task.get().deletes().size()); - Assert.assertEquals( - "Should have only pos delete file", FILE_A_EQ_1.path(), task.get().deletes().get(0).path()); + assertThat(table.newScan().planFiles()) + .hasSize(3) + .filteredOn(fileScanTask -> fileScanTask.file().path().equals(FILE_A.path())) + .first() + .satisfies( + fileScanTask -> { + assertThat(fileScanTask.deletes()).hasSize(1); + assertThat(fileScanTask.deletes().get(0).path()).isEqualTo(FILE_A_EQ_1.path()); + }); // first commit after row-delta changes table.newDelete().deleteFile(FILE_B).commit(); dataManifests = table.currentSnapshot().dataManifests(ops.io()); deleteManifests = table.currentSnapshot().deleteManifests(ops.io()); - Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); - Assert.assertEquals("Should have one delete manifest file", 1, deleteManifests.size()); + assertThat(dataManifests).hasSize(1).first().isNotEqualTo(dataManifest); + assertThat(deleteManifests).hasSize(1).first().isEqualTo(deleteManifest); ManifestFile dataManifest2 = dataManifests.get(0); verifyManifestSequenceNumber(dataManifest2, 2, 0); - Assert.assertNotEquals(dataManifest, dataManifest2); - Assert.assertEquals(deleteManifest, deleteManifests.get(0)); // delete manifest not changed - tasks = Lists.newArrayList(table.newScan().planFiles().iterator()); - Assert.assertEquals("Should have two task", 2, tasks.size()); - task = tasks.stream().filter(t -> t.file().path().equals(FILE_A.path())).findFirst(); - Assert.assertTrue(task.isPresent()); - Assert.assertEquals("Should have one associated delete file", 1, task.get().deletes().size()); + assertThat(table.newScan().planFiles()) + .hasSize(2) + .filteredOn(fileScanTask -> fileScanTask.file().path().equals(FILE_A.path())) + .first() + .satisfies(fileScanTask -> assertThat(fileScanTask.deletes()).hasSize(1)); // second commit after row-delta changes table.newDelete().deleteFile(FILE_C).commit(); dataManifests = table.currentSnapshot().dataManifests(ops.io()); deleteManifests = table.currentSnapshot().deleteManifests(ops.io()); - Assert.assertEquals("Should have one data manifest file", 1, dataManifests.size()); - Assert.assertEquals("Should have one delete manifest file", 1, deleteManifests.size()); - ManifestFile dataManifest3 = dataManifests.get(0); - verifyManifestSequenceNumber(dataManifest3, 3, 0); - Assert.assertNotEquals(dataManifest2, dataManifest3); - Assert.assertEquals(deleteManifest, deleteManifests.get(0)); // delete manifest not changed - tasks = Lists.newArrayList(table.newScan().planFiles().iterator()); - Assert.assertEquals("Should have one task", 1, tasks.size()); - task = tasks.stream().filter(t -> t.file().path().equals(FILE_A.path())).findFirst(); - Assert.assertTrue(task.isPresent()); - Assert.assertEquals("Should have one associated delete file", 1, task.get().deletes().size()); + assertThat(dataManifests).hasSize(1).first().isNotEqualTo(dataManifest2); + assertThat(deleteManifests).hasSize(1).first().isEqualTo(deleteManifest); + verifyManifestSequenceNumber(dataManifests.get(0), 3, 0); + assertThat(table.newScan().planFiles()) + .hasSize(1) + .filteredOn(fileScanTask -> fileScanTask.file().path().equals(FILE_A.path())) + .first() + .satisfies(fileScanTask -> assertThat(fileScanTask.deletes()).hasSize(1)); } - @Test + @TestTemplate public void testPartitionedTableWithUnrelatedPartitionDeletes() { table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); @@ -141,27 +133,29 @@ public void testPartitionedTableWithUnrelatedPartitionDeletes() { table.newRowDelta().addDeletes(FILE_A_POS_1).addDeletes(FILE_A_EQ_1).commit(); - List tasks = Lists.newArrayList(table.newScan().planFiles().iterator()); - Assert.assertEquals("Should have three task", 3, tasks.size()); - Assert.assertEquals( - "Should have the correct data file path", FILE_B.path(), tasks.get(0).file().path()); - Assert.assertEquals( - "Should have zero associated delete file", 0, tasks.get(0).deletes().size()); + assertThat(table.newScan().planFiles()) + .hasSize(3) + .first() + .satisfies( + fileScanTask -> { + assertThat(fileScanTask.file().path()).isEqualTo(FILE_B.path()); + assertThat(fileScanTask.deletes()).isEmpty(); + }); table.newDelete().deleteFile(FILE_B).commit(); - tasks = Lists.newArrayList(table.newScan().planFiles().iterator()); - Assert.assertEquals("Should have two task", 2, tasks.size()); - Assert.assertEquals( - "Should have zero associated delete file", 0, tasks.get(0).deletes().size()); + assertThat(table.newScan().planFiles()) + .hasSize(2) + .first() + .satisfies(fileScanTask -> assertThat(fileScanTask.deletes()).isEmpty()); table.newDelete().deleteFile(FILE_C).commit(); - tasks = Lists.newArrayList(table.newScan().planFiles().iterator()); - Assert.assertEquals("Should have one task", 1, tasks.size()); - Assert.assertEquals( - "Should have zero associated delete file", 0, tasks.get(0).deletes().size()); + assertThat(table.newScan().planFiles()) + .hasSize(1) + .first() + .satisfies(fileScanTask -> assertThat(fileScanTask.deletes()).isEmpty()); } - @Test + @TestTemplate public void testPartitionedTableWithExistingDeleteFile() { table.updateProperties().set(TableProperties.MANIFEST_MERGE_ENABLED, "false").commit(); @@ -182,30 +176,18 @@ public void testPartitionedTableWithExistingDeleteFile() { .set(TableProperties.MANIFEST_MERGE_ENABLED, "true") .commit(); - Assert.assertEquals( - "Should have two delete manifests", - 2, - table.currentSnapshot().deleteManifests(table.io()).size()); + assertThat(table.currentSnapshot().deleteManifests(table.io())).hasSize(2); // merge delete manifests table.newAppend().appendFile(FILE_B).commit(); - Assert.assertEquals( - "Should have one delete manifest", - 1, - table.currentSnapshot().deleteManifests(table.io()).size()); - Assert.assertEquals( - "Should have zero added delete file", - 0, - table.currentSnapshot().deleteManifests(table.io()).get(0).addedFilesCount().intValue()); - Assert.assertEquals( - "Should have zero deleted delete file", - 0, - table.currentSnapshot().deleteManifests(table.io()).get(0).deletedFilesCount().intValue()); - Assert.assertEquals( - "Should have two existing delete files", - 2, - table.currentSnapshot().deleteManifests(table.io()).get(0).existingFilesCount().intValue()); + assertThat(table.currentSnapshot().deleteManifests(table.io())).hasSize(1); + assertThat(table.currentSnapshot().deleteManifests(table.io()).get(0).addedFilesCount()) + .isEqualTo(0); + assertThat(table.currentSnapshot().deleteManifests(table.io()).get(0).deletedFilesCount()) + .isEqualTo(0); + assertThat(table.currentSnapshot().deleteManifests(table.io()).get(0).existingFilesCount()) + .isEqualTo(2); List tasks = Lists.newArrayList( @@ -214,19 +196,16 @@ public void testPartitionedTableWithExistingDeleteFile() { .filter(equal(bucket("data", BUCKETS_NUMBER), 0)) .planFiles() .iterator()); - Assert.assertEquals("Should have one task", 1, tasks.size()); + assertThat(tasks).hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should have the correct data file path", FILE_A.path(), task.file().path()); - Assert.assertEquals("Should have two associated delete files", 2, task.deletes().size()); - Assert.assertEquals( - "Should have expected delete files", - Sets.newHashSet(FILE_A_EQ_1.path(), FILE_A_POS_1.path()), - Sets.newHashSet(Iterables.transform(task.deletes(), ContentFile::path))); + assertThat(task.file().path()).isEqualTo(FILE_A.path()); + 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()); } - @Test + @TestTemplate public void testSequenceNumbersInUpgradedTables() { // add initial data table.newAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); diff --git a/core/src/test/java/org/apache/iceberg/V2TableTestBase.java b/core/src/test/java/org/apache/iceberg/V2TableTestBase.java index 5e46f927f545..7e9dd5e85a00 100644 --- a/core/src/test/java/org/apache/iceberg/V2TableTestBase.java +++ b/core/src/test/java/org/apache/iceberg/V2TableTestBase.java @@ -18,8 +18,12 @@ */ package org.apache.iceberg; -public class V2TableTestBase extends TableTestBase { - public V2TableTestBase() { - super(2); +import java.util.Arrays; +import java.util.List; + +public class V2TableTestBase extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(2); } } From 6f48bd90e77fc81f3ab01bd7b24ffa5f139d01f8 Mon Sep 17 00:00:00 2001 From: Csenger Geza Date: Mon, 25 Mar 2024 19:01:42 +0000 Subject: [PATCH 0162/1019] Add Iceberg version to UserAgent in S3 requests (#9963) This allows developers to monitor which version of Iceberg they have deployed to a cluster (for example, S3 Access Logs contain the user agent field). Co-authored-by: Geza Csenger --- .../org/apache/iceberg/aws/AwsClientFactories.java | 1 + .../aws/s3/DefaultS3FileIOAwsClientFactory.java | 1 + .../apache/iceberg/aws/s3/S3FileIOProperties.java | 14 ++++++++++++++ .../iceberg/aws/s3/TestS3FileIOProperties.java | 10 ++++++++++ 4 files changed, 26 insertions(+) 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 580e7303ffca..81c7bd6b4bab 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -113,6 +113,7 @@ public S3Client s3() { b -> s3FileIOProperties.applyCredentialConfigurations(awsClientProperties, b)) .applyMutation(s3FileIOProperties::applySignerConfiguration) .applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations) + .applyMutation(s3FileIOProperties::applyUserAgentConfigurations) .build(); } diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java index a65910612fe6..18b40000a91a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java @@ -54,6 +54,7 @@ public S3Client s3() { awsClientProperties, s3ClientBuilder)) .applyMutation(s3FileIOProperties::applySignerConfiguration) .applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations) + .applyMutation(s3FileIOProperties::applyUserAgentConfigurations) .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 9aad784be865..857f35e710ab 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 @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.aws.AwsClientProperties; import org.apache.iceberg.aws.glue.GlueCatalog; import org.apache.iceberg.aws.s3.signer.S3V4RestSignerClient; @@ -375,6 +376,14 @@ public class S3FileIOProperties implements Serializable { public static final boolean PRELOAD_CLIENT_ENABLED_DEFAULT = false; + /** + * User Agent Prefix set by the S3 client. + * + *

This allows developers to monitor which version of Iceberg they have deployed to a cluster + * (for example, through the S3 Access Logs, which contain the user agent field). + */ + private static final String S3_FILE_IO_USER_AGENT = "s3fileio/" + EnvironmentContext.get(); + private String sseType; private String sseKey; private String sseMd5; @@ -819,6 +828,11 @@ public void applyS3AccessGrantsConfigurations(T buil } } + public void applyUserAgentConfigurations(T builder) { + builder.overrideConfiguration( + c -> c.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, S3_FILE_IO_USER_AGENT)); + } + /** * Dynamically load the http client builder to avoid runtime deps requirements of any optional SDK * Plugins diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index 2ed8a9471d66..658b5b781969 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -478,4 +478,14 @@ public void testApplyEndpointConfiguration() { s3FileIOProperties.applyEndpointConfigurations(mockS3ClientBuilder); Mockito.verify(mockS3ClientBuilder).endpointOverride(Mockito.any(URI.class)); } + + @Test + public void testApplyUserAgentConfigurations() { + Map properties = Maps.newHashMap(); + S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(properties); + S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); + s3FileIOProperties.applyUserAgentConfigurations(mockS3ClientBuilder); + + Mockito.verify(mockS3ClientBuilder).overrideConfiguration(Mockito.any(Consumer.class)); + } } From 8aa6dd93727b22dc887305f973a2fd1efb87832d Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 25 Mar 2024 15:03:33 -0700 Subject: [PATCH 0163/1019] Core, Spark: Fix handling of null binary values when sorting with zorder (#10026) --- .../apache/iceberg/util/ZOrderByteUtils.java | 5 +++ .../iceberg/util/TestZOrderByteUtil.java | 10 ++++++ .../TestRewriteDataFilesProcedure.java | 35 +++++++++++++++++++ .../TestRewriteDataFilesProcedure.java | 35 +++++++++++++++++++ .../TestRewriteDataFilesProcedure.java | 35 +++++++++++++++++++ 5 files changed, 120 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java index 923f3dc2d5c6..c687fc4e03dc 100644 --- a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java +++ b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java @@ -145,6 +145,11 @@ public static ByteBuffer stringToOrderedBytes( @SuppressWarnings("ByteBufferBackingArray") public static ByteBuffer byteTruncateOrFill(byte[] val, int length, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, length); + if (val == null) { + Arrays.fill(bytes.array(), 0, length, (byte) 0x00); + return bytes; + } + if (val.length < length) { bytes.put(val, 0, val.length); Arrays.fill(bytes.array(), val.length, length, (byte) 0x00); diff --git a/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java b/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java index 13e7c843c79f..d05843e20849 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestZOrderByteUtil.java @@ -391,4 +391,14 @@ public void testByteTruncateOrFill() { .isEqualTo(stringCompare); } } + + @Test + public void testByteTruncatedOrFillNullIsZeroArray() { + ByteBuffer buffer = ByteBuffer.allocate(128); + byte[] actualBytes = ZOrderByteUtils.byteTruncateOrFill(null, 128, buffer).array(); + ByteBuffer expected = ByteBuffer.allocate(128); + Arrays.fill(expected.array(), 0, 128, (byte) 0x00); + + assertThat(actualBytes).isEqualTo(expected.array()); + } } 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 0cdde158bde3..80cacbf376ac 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 @@ -218,6 +218,41 @@ public void testRewriteDataFilesWithZOrder() { assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } + @Test + public void testRewriteDataFilesWithZOrderNullBinaryColumn() { + sql("CREATE TABLE %s (c1 int, c2 string, c3 binary) USING iceberg", tableName); + + for (int i = 0; i < 5; i++) { + sql("INSERT INTO %s values (1, 'foo', null), (2, 'bar', null)", tableName); + } + + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', sort_order => 'zorder(c2,c3)')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + assertThat(output.get(0)).hasSize(3); + assertThat(snapshotSummary()) + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, String.valueOf(output.get(0)[2])); + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactly( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + } + @Test public void testRewriteDataFilesWithFilter() { createTable(); 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 477aa2a1d958..009ab41228c8 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 @@ -261,6 +261,41 @@ public void testRewriteDataFilesWithZOrder() { assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } + @Test + public void testRewriteDataFilesWithZOrderNullBinaryColumn() { + sql("CREATE TABLE %s (c1 int, c2 string, c3 binary) USING iceberg", tableName); + + for (int i = 0; i < 5; i++) { + sql("INSERT INTO %s values (1, 'foo', null), (2, 'bar', null)", tableName); + } + + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', sort_order => 'zorder(c2,c3)')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + assertThat(output.get(0)).hasSize(4); + assertThat(snapshotSummary()) + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, String.valueOf(output.get(0)[2])); + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactly( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + } + @Test public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() { createTable(); 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 b01438d39dfc..9ba886db4516 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 @@ -259,6 +259,41 @@ public void testRewriteDataFilesWithZOrder() { assertEquals("Should have expected rows", expectedRows, sql("SELECT * FROM %s", tableName)); } + @TestTemplate + public void testRewriteDataFilesWithZOrderNullBinaryColumn() { + sql("CREATE TABLE %s (c1 int, c2 string, c3 binary) USING iceberg", tableName); + + for (int i = 0; i < 5; i++) { + sql("INSERT INTO %s values (1, 'foo', null), (2, 'bar', null)", tableName); + } + + List output = + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + + "strategy => 'sort', sort_order => 'zorder(c2,c3)')", + catalogName, tableIdent); + + assertEquals( + "Action should rewrite 10 data files and add 1 data files", + row(10, 1), + Arrays.copyOf(output.get(0), 2)); + assertThat(output.get(0)).hasSize(4); + assertThat(snapshotSummary()) + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, String.valueOf(output.get(0)[2])); + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactly( + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(2, "bar", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null), + row(1, "foo", null)); + } + @TestTemplate public void testRewriteDataFilesWithZOrderAndMultipleShufflePartitionsPerFile() { createTable(); From 5f372dcbfb48401800ea014b46f8bd77aee3fb28 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Tue, 26 Mar 2024 12:19:22 +0530 Subject: [PATCH 0164/1019] Hive: Extract common code to be re-used for View support (#10001) --- .../apache/iceberg/BaseMetastoreCatalog.java | 21 +--- .../iceberg/BaseMetastoreOperations.java | 118 ++++++++++++++++++ .../iceberg/BaseMetastoreTableOperations.java | 100 +++++---------- .../java/org/apache/iceberg/CatalogUtil.java | 24 ++++ .../iceberg/view/BaseViewOperations.java | 3 +- .../org/apache/iceberg/TestCatalogUtil.java | 26 ++++ .../org/apache/iceberg/hive/HiveCatalog.java | 52 +++++--- .../iceberg/hive/HiveOperationsBase.java | 68 +++++++++- .../iceberg/hive/HiveTableOperations.java | 39 ++---- 9 files changed, 314 insertions(+), 137 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/BaseMetastoreOperations.java diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java index bb7d5a0ffd9d..e794b3121dc3 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreCatalog.java @@ -285,26 +285,7 @@ private Map tableOverrideProperties() { } protected static String fullTableName(String catalogName, TableIdentifier identifier) { - StringBuilder sb = new StringBuilder(); - - if (catalogName.contains("/") || catalogName.contains(":")) { - // use / for URI-like names: thrift://host:port/db.table - sb.append(catalogName); - if (!catalogName.endsWith("/")) { - sb.append("/"); - } - } else { - // use . for non-URI named catalogs: prod.db.table - sb.append(catalogName).append("."); - } - - for (String level : identifier.namespace().levels()) { - sb.append(level).append("."); - } - - sb.append(identifier.name()); - - return sb.toString(); + return CatalogUtil.fullTableName(catalogName, identifier); } protected MetricsReporter metricsReporter() { diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreOperations.java new file mode 100644 index 000000000000..09c2249046f4 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreOperations.java @@ -0,0 +1,118 @@ +/* + * 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.TableProperties.COMMIT_NUM_STATUS_CHECKS; +import static org.apache.iceberg.TableProperties.COMMIT_NUM_STATUS_CHECKS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MAX_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MAX_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MIN_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MIN_WAIT_MS_DEFAULT; +import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS; +import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS_DEFAULT; + +import java.util.Map; +import java.util.concurrent.atomic.AtomicReference; +import java.util.function.Supplier; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class BaseMetastoreOperations { + private static final Logger LOG = LoggerFactory.getLogger(BaseMetastoreOperations.class); + + public enum CommitStatus { + FAILURE, + SUCCESS, + UNKNOWN + } + + /** + * Attempt to load the content and see if any current or past metadata location matches the one we + * were attempting to set. This is used as a last resort when we are dealing with exceptions that + * may indicate the commit has failed but don't have proof that this is the case. Note that all + * the previous locations must also be searched on the chance that a second committer was able to + * successfully commit on top of our commit. + * + * @param tableOrViewName full name of the Table/View + * @param newMetadataLocation the path of the new commit file + * @param properties properties for retry + * @param commitStatusSupplier check if the latest metadata presents or not using metadata + * location for table. + * @return Commit Status of Success, Failure or Unknown + */ + protected CommitStatus checkCommitStatus( + String tableOrViewName, + String newMetadataLocation, + Map properties, + Supplier commitStatusSupplier) { + int maxAttempts = + PropertyUtil.propertyAsInt( + properties, COMMIT_NUM_STATUS_CHECKS, COMMIT_NUM_STATUS_CHECKS_DEFAULT); + long minWaitMs = + PropertyUtil.propertyAsLong( + properties, COMMIT_STATUS_CHECKS_MIN_WAIT_MS, COMMIT_STATUS_CHECKS_MIN_WAIT_MS_DEFAULT); + long maxWaitMs = + PropertyUtil.propertyAsLong( + properties, COMMIT_STATUS_CHECKS_MAX_WAIT_MS, COMMIT_STATUS_CHECKS_MAX_WAIT_MS_DEFAULT); + long totalRetryMs = + PropertyUtil.propertyAsLong( + properties, + COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS, + COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS_DEFAULT); + + AtomicReference status = new AtomicReference<>(CommitStatus.UNKNOWN); + + Tasks.foreach(newMetadataLocation) + .retry(maxAttempts) + .suppressFailureWhenFinished() + .exponentialBackoff(minWaitMs, maxWaitMs, totalRetryMs, 2.0) + .onFailure( + (location, checkException) -> + LOG.error("Cannot check if commit to {} exists.", tableOrViewName, checkException)) + .run( + location -> { + boolean commitSuccess = commitStatusSupplier.get(); + + if (commitSuccess) { + LOG.info( + "Commit status check: Commit to {} of {} succeeded", + tableOrViewName, + newMetadataLocation); + status.set(CommitStatus.SUCCESS); + } else { + LOG.warn( + "Commit status check: Commit to {} of {} unknown, new metadata location is not current " + + "or in history", + tableOrViewName, + newMetadataLocation); + } + }); + + if (status.get() == CommitStatus.UNKNOWN) { + LOG.error( + "Cannot determine commit state to {}. Failed during checking {} times. " + + "Treating commit state as unknown.", + tableOrViewName, + maxAttempts); + } + return status.get(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 2fccef5a0ab3..6443cf6e60ea 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -18,15 +18,6 @@ */ package org.apache.iceberg; -import static org.apache.iceberg.TableProperties.COMMIT_NUM_STATUS_CHECKS; -import static org.apache.iceberg.TableProperties.COMMIT_NUM_STATUS_CHECKS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MAX_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MAX_WAIT_MS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MIN_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_MIN_WAIT_MS_DEFAULT; -import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS; -import static org.apache.iceberg.TableProperties.COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS_DEFAULT; - import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; @@ -44,12 +35,12 @@ 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.LocationUtil; -import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class BaseMetastoreTableOperations implements TableOperations { +public abstract class BaseMetastoreTableOperations extends BaseMetastoreOperations + implements TableOperations { private static final Logger LOG = LoggerFactory.getLogger(BaseMetastoreTableOperations.class); public static final String TABLE_TYPE_PROP = "table_type"; @@ -291,6 +282,11 @@ public long newSnapshotId() { }; } + /** + * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link + * BaseMetastoreOperations.CommitStatus} instead + */ + @Deprecated protected enum CommitStatus { FAILURE, SUCCESS, @@ -309,65 +305,29 @@ protected enum CommitStatus { * @return Commit Status of Success, Failure or Unknown */ protected CommitStatus checkCommitStatus(String newMetadataLocation, TableMetadata config) { - int maxAttempts = - PropertyUtil.propertyAsInt( - config.properties(), COMMIT_NUM_STATUS_CHECKS, COMMIT_NUM_STATUS_CHECKS_DEFAULT); - long minWaitMs = - PropertyUtil.propertyAsLong( - config.properties(), - COMMIT_STATUS_CHECKS_MIN_WAIT_MS, - COMMIT_STATUS_CHECKS_MIN_WAIT_MS_DEFAULT); - long maxWaitMs = - PropertyUtil.propertyAsLong( - config.properties(), - COMMIT_STATUS_CHECKS_MAX_WAIT_MS, - COMMIT_STATUS_CHECKS_MAX_WAIT_MS_DEFAULT); - long totalRetryMs = - PropertyUtil.propertyAsLong( - config.properties(), - COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS, - COMMIT_STATUS_CHECKS_TOTAL_WAIT_MS_DEFAULT); - - AtomicReference status = new AtomicReference<>(CommitStatus.UNKNOWN); - - Tasks.foreach(newMetadataLocation) - .retry(maxAttempts) - .suppressFailureWhenFinished() - .exponentialBackoff(minWaitMs, maxWaitMs, totalRetryMs, 2.0) - .onFailure( - (location, checkException) -> - LOG.error("Cannot check if commit to {} exists.", tableName(), checkException)) - .run( - location -> { - TableMetadata metadata = refresh(); - String currentMetadataFileLocation = metadata.metadataFileLocation(); - boolean commitSuccess = - currentMetadataFileLocation.equals(newMetadataLocation) - || metadata.previousFiles().stream() - .anyMatch(log -> log.file().equals(newMetadataLocation)); - if (commitSuccess) { - LOG.info( - "Commit status check: Commit to {} of {} succeeded", - tableName(), - newMetadataLocation); - status.set(CommitStatus.SUCCESS); - } else { - LOG.warn( - "Commit status check: Commit to {} of {} unknown, new metadata location is not current " - + "or in history", - tableName(), - newMetadataLocation); - } - }); - - if (status.get() == CommitStatus.UNKNOWN) { - LOG.error( - "Cannot determine commit state to {}. Failed during checking {} times. " - + "Treating commit state as unknown.", - tableName(), - maxAttempts); - } - return status.get(); + return CommitStatus.valueOf( + checkCommitStatus( + tableName(), + newMetadataLocation, + config.properties(), + () -> checkCurrentMetadataLocation(newMetadataLocation)) + .name()); + } + + /** + * Validate if the new metadata location is the current metadata location or present within + * previous metadata files. + * + * @param newMetadataLocation newly written metadata location + * @return true if the new metadata location is the current metadata location or present within + * previous metadata files. + */ + private boolean checkCurrentMetadataLocation(String newMetadataLocation) { + TableMetadata metadata = refresh(); + String currentMetadataFileLocation = metadata.metadataFileLocation(); + return currentMetadataFileLocation.equals(newMetadataLocation) + || metadata.previousFiles().stream() + .anyMatch(log -> log.file().equals(newMetadataLocation)); } private String newTableMetadataFilePath(TableMetadata meta, int newVersion) { diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index e09be748f2ee..4846dbb8e9a9 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.common.DynClasses; import org.apache.iceberg.common.DynConstructors; import org.apache.iceberg.common.DynMethods; @@ -473,4 +474,27 @@ public static MetricsReporter loadMetricsReporter(Map properties return reporter; } + + public static String fullTableName(String catalogName, TableIdentifier identifier) { + StringBuilder sb = new StringBuilder(); + + if (catalogName.contains("/") || catalogName.contains(":")) { + // use / for URI-like names: thrift://host:port/db.table + sb.append(catalogName); + if (!catalogName.endsWith("/")) { + sb.append("/"); + } + } else { + // use . for non-URI named catalogs: prod.db.table + sb.append(catalogName).append("."); + } + + for (String level : identifier.namespace().levels()) { + sb.append(level).append("."); + } + + sb.append(identifier.name()); + + return sb.toString(); + } } diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java index 766d217346e0..7a4f546b8860 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; import java.util.function.Predicate; +import org.apache.iceberg.BaseMetastoreOperations; import org.apache.iceberg.TableMetadataParser; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; @@ -35,7 +36,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public abstract class BaseViewOperations implements ViewOperations { +public abstract class BaseViewOperations extends BaseMetastoreOperations implements ViewOperations { private static final Logger LOG = LoggerFactory.getLogger(BaseViewOperations.class); private static final String METADATA_FOLDER_NAME = "metadata"; diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 5ac0ebbafc90..878ca36a10ef 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -223,6 +223,32 @@ public void loadCustomMetricsReporter_badClass() { .hasMessageContaining("does not implement MetricsReporter"); } + @Test + public void fullTableNameWithDifferentValues() { + String uriTypeCatalogName = "thrift://host:port/db.table"; + String namespace = "ns"; + String nameSpaceWithTwoLevels = "ns.l2"; + String tableName = "tbl"; + TableIdentifier tableIdentifier = TableIdentifier.of(namespace, tableName); + Assertions.assertThat(CatalogUtil.fullTableName(uriTypeCatalogName, tableIdentifier)) + .isEqualTo(uriTypeCatalogName + "/" + namespace + "." + tableName); + + tableIdentifier = TableIdentifier.of(nameSpaceWithTwoLevels, tableName); + Assertions.assertThat(CatalogUtil.fullTableName(uriTypeCatalogName, tableIdentifier)) + .isEqualTo(uriTypeCatalogName + "/" + nameSpaceWithTwoLevels + "." + tableName); + + Assertions.assertThat(CatalogUtil.fullTableName(uriTypeCatalogName + "/", tableIdentifier)) + .isEqualTo(uriTypeCatalogName + "/" + nameSpaceWithTwoLevels + "." + tableName); + + String nonUriCatalogName = "test.db.catalog"; + Assertions.assertThat(CatalogUtil.fullTableName(nonUriCatalogName, tableIdentifier)) + .isEqualTo(nonUriCatalogName + "." + nameSpaceWithTwoLevels + "." + tableName); + + String pathStyleCatalogName = "/test/db"; + Assertions.assertThat(CatalogUtil.fullTableName(pathStyleCatalogName, tableIdentifier)) + .isEqualTo(pathStyleCatalogName + "/" + nameSpaceWithTwoLevels + "." + tableName); + } + public static class TestCatalog extends BaseMetastoreCatalog { private String catalogName; 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 34f9e1da4319..b4f49e29fc49 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 @@ -129,20 +129,9 @@ public List listTables(Namespace namespace) { .map(t -> TableIdentifier.of(namespace, t)) .collect(Collectors.toList()); } else { - List tableObjects = - clients.run(client -> client.getTableObjectsByName(database, tableNames)); tableIdentifiers = - tableObjects.stream() - .filter( - table -> - table.getParameters() != null - && BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE - .equalsIgnoreCase( - table - .getParameters() - .get(BaseMetastoreTableOperations.TABLE_TYPE_PROP))) - .map(table -> TableIdentifier.of(namespace, table.getTableName())) - .collect(Collectors.toList()); + listIcebergTables( + tableNames, namespace, BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE); } LOG.debug( @@ -222,6 +211,28 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { @Override public void renameTable(TableIdentifier from, TableIdentifier originalTo) { + renameTableOrView(from, originalTo, HiveOperationsBase.ContentType.TABLE); + } + + private List listIcebergTables( + List tableNames, Namespace namespace, String tableTypeProp) + throws TException, InterruptedException { + List
tableObjects = + clients.run(client -> client.getTableObjectsByName(namespace.level(0), tableNames)); + return tableObjects.stream() + .filter( + table -> + table.getParameters() != null + && tableTypeProp.equalsIgnoreCase( + table.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP))) + .map(table -> TableIdentifier.of(namespace, table.getTableName())) + .collect(Collectors.toList()); + } + + private void renameTableOrView( + TableIdentifier from, + TableIdentifier originalTo, + HiveOperationsBase.ContentType contentType) { if (!isValidIdentifier(from)) { throw new NoSuchTableException("Invalid identifier: %s", from); } @@ -239,7 +250,7 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) { try { Table table = clients.run(client -> client.getTable(fromDatabase, fromName)); - HiveOperationsBase.validateTableIsIceberg(table, fullTableName(name, from)); + validateTableIsIcebergTableOrView(contentType, table, CatalogUtil.fullTableName(name, from)); table.setDbName(toDatabase); table.setTableName(to.name()); @@ -250,7 +261,7 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) { return null; }); - LOG.info("Renamed table from {}, to {}", from, to); + LOG.info("Renamed {} from {}, to {}", contentType.value(), from, to); } catch (NoSuchObjectException e) { throw new NoSuchTableException("Table does not exist: %s", from); @@ -273,6 +284,17 @@ public void renameTable(TableIdentifier from, TableIdentifier originalTo) { } } + private void validateTableIsIcebergTableOrView( + HiveOperationsBase.ContentType contentType, Table table, String fullName) { + switch (contentType) { + case TABLE: + HiveOperationsBase.validateTableIsIceberg(table, fullName); + break; + case VIEW: + throw new UnsupportedOperationException("View is not supported."); + } + } + @Override public void createNamespace(Namespace namespace, Map meta) { Preconditions.checkArgument( diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java index ea24fe4e1133..a93577a35c73 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -22,11 +22,14 @@ import java.util.Map; import org.apache.hadoop.hive.metastore.IMetaStoreClient; import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.SerDeInfo; import org.apache.hadoop.hive.metastore.api.StorageDescriptor; import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.iceberg.BaseMetastoreOperations; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.ClientPool; +import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; @@ -52,6 +55,21 @@ interface HiveOperationsBase { String NO_LOCK_EXPECTED_KEY = "expected_parameter_key"; String NO_LOCK_EXPECTED_VALUE = "expected_parameter_value"; + enum ContentType { + TABLE("Table"), + VIEW("View"); + + private final String value; + + ContentType(String value) { + this.value = value; + } + + public String value() { + return value; + } + } + TableType tableType(); ClientPool metaClients(); @@ -62,6 +80,15 @@ interface HiveOperationsBase { String table(); + default Table loadHmsTable() throws TException, InterruptedException { + try { + return metaClients().run(client -> client.getTable(database(), table())); + } catch (NoSuchObjectException nte) { + LOG.trace("Table not found {}", database() + "." + table(), nte); + return null; + } + } + default Map hmsEnvContext(String metadataLocation) { return metadataLocation == null ? ImmutableMap.of() @@ -76,11 +103,19 @@ default boolean exposeInHmsProperties() { return maxHiveTablePropertySize() > 0; } + /** + * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link #setSchema(Schema, Map)} instead + */ + @Deprecated default void setSchema(TableMetadata metadata, Map parameters) { + setSchema(metadata.schema(), parameters); + } + + default void setSchema(Schema schema, Map parameters) { parameters.remove(TableProperties.CURRENT_SCHEMA); - if (exposeInHmsProperties() && metadata.schema() != null) { - String schema = SchemaParser.toJson(metadata.schema()); - setField(parameters, TableProperties.CURRENT_SCHEMA, schema); + if (exposeInHmsProperties() && schema != null) { + String jsonSchema = SchemaParser.toJson(schema); + setField(parameters, TableProperties.CURRENT_SCHEMA, jsonSchema); } } @@ -123,13 +158,23 @@ default void persistTable(Table hmsTable, boolean updateHiveTable, String metada } } + /** + * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link #storageDescriptor(Schema, + * String, boolean)} instead + */ + @Deprecated static StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { + return storageDescriptor(metadata.schema(), metadata.location(), hiveEngineEnabled); + } + static StorageDescriptor storageDescriptor( + Schema schema, String location, boolean hiveEngineEnabled) { final StorageDescriptor storageDescriptor = new StorageDescriptor(); - storageDescriptor.setCols(HiveSchemaUtil.convert(metadata.schema())); - storageDescriptor.setLocation(metadata.location()); + storageDescriptor.setCols(HiveSchemaUtil.convert(schema)); + storageDescriptor.setLocation(location); SerDeInfo serDeInfo = new SerDeInfo(); serDeInfo.setParameters(Maps.newHashMap()); + if (hiveEngineEnabled) { storageDescriptor.setInputFormat("org.apache.iceberg.mr.hive.HiveIcebergInputFormat"); storageDescriptor.setOutputFormat("org.apache.iceberg.mr.hive.HiveIcebergOutputFormat"); @@ -139,6 +184,7 @@ static StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveE storageDescriptor.setInputFormat("org.apache.hadoop.mapred.FileInputFormat"); serDeInfo.setSerializationLib("org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"); } + storageDescriptor.setSerdeInfo(serDeInfo); return storageDescriptor; } @@ -154,6 +200,18 @@ static void cleanupMetadata(FileIO io, String commitStatus, String metadataLocat } } + static void cleanupMetadataAndUnlock( + FileIO io, + BaseMetastoreOperations.CommitStatus commitStatus, + String metadataLocation, + HiveLock lock) { + try { + cleanupMetadata(io, commitStatus.name(), metadataLocation); + } finally { + lock.unlock(); + } + } + default Table newHmsTable(String hmsTableOwner) { Preconditions.checkNotNull(hmsTableOwner, "'hmsOwner' parameter can't be null"); final long currentTimeMillis = System.currentTimeMillis(); 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 a3750b9f3101..bae074d55d24 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 @@ -36,6 +36,7 @@ import org.apache.hadoop.hive.metastore.api.NoSuchObjectException; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants; +import org.apache.iceberg.BaseMetastoreOperations; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.ClientPool; import org.apache.iceberg.PartitionSpecParser; @@ -174,7 +175,8 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { boolean hiveEngineEnabled = hiveEngineEnabled(metadata, conf); boolean keepHiveStats = conf.getBoolean(ConfigProperties.KEEP_HIVE_STATS, false); - CommitStatus commitStatus = CommitStatus.FAILURE; + BaseMetastoreOperations.CommitStatus commitStatus = + BaseMetastoreOperations.CommitStatus.FAILURE; boolean updateHiveTable = false; HiveLock lock = lockObject(metadata); @@ -203,7 +205,9 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { tbl.setSd( HiveOperationsBase.storageDescriptor( - metadata, hiveEngineEnabled)); // set to pickup any schema changes + metadata.schema(), + metadata.location(), + hiveEngineEnabled)); // set to pickup any schema changes String metadataLocation = tbl.getParameters().get(METADATA_LOCATION_PROP); String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; @@ -240,9 +244,9 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { tbl, updateHiveTable, hiveLockEnabled(metadata, conf) ? null : baseMetadataLocation); lock.ensureActive(); - commitStatus = CommitStatus.SUCCESS; + commitStatus = BaseMetastoreOperations.CommitStatus.SUCCESS; } catch (LockException le) { - commitStatus = CommitStatus.UNKNOWN; + commitStatus = BaseMetastoreOperations.CommitStatus.UNKNOWN; throw new CommitStateUnknownException( "Failed to heartbeat for hive lock while " + "committing changes. This can lead to a concurrent commit attempt be able to overwrite this commit. " @@ -282,7 +286,9 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { database, tableName, e); - commitStatus = checkCommitStatus(newMetadataLocation, metadata); + commitStatus = + BaseMetastoreOperations.CommitStatus.valueOf( + checkCommitStatus(newMetadataLocation, metadata).name()); switch (commitStatus) { case SUCCESS: break; @@ -304,23 +310,13 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { throw new CommitFailedException(e); } finally { - cleanupMetadataAndUnlock(commitStatus, newMetadataLocation, lock); + HiveOperationsBase.cleanupMetadataAndUnlock(io(), commitStatus, newMetadataLocation, lock); } LOG.info( "Committed to table {} with the new metadata location {}", fullName, newMetadataLocation); } - @VisibleForTesting - Table loadHmsTable() throws TException, InterruptedException { - try { - return metaClients.run(client -> client.getTable(database, tableName)); - } catch (NoSuchObjectException nte) { - LOG.trace("Table not found {}", fullName, nte); - return null; - } - } - private void setHmsTableParameters( String newMetadataLocation, Table tbl, @@ -376,7 +372,7 @@ private void setHmsTableParameters( } setSnapshotStats(metadata, parameters); - setSchema(metadata, parameters); + setSchema(metadata.schema(), parameters); setPartitionSpec(metadata, parameters); setSortOrder(metadata, parameters); @@ -467,15 +463,6 @@ public ClientPool metaClients() { return metaClients; } - private void cleanupMetadataAndUnlock( - CommitStatus commitStatus, String metadataLocation, HiveLock lock) { - try { - HiveOperationsBase.cleanupMetadata(io(), commitStatus.name(), metadataLocation); - } finally { - lock.unlock(); - } - } - /** * Returns if the hive engine related values should be enabled on the table, or not. * From 5ab38857a81a2407b666a7b747c70ae46f6f893e Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Tue, 26 Mar 2024 20:10:56 +0530 Subject: [PATCH 0165/1019] Hive: Add test to make sure iceberg table with same name as hive table can't be created (#9980) --- .../apache/iceberg/hive/HiveTableTest.java | 47 +++++++++++++++++-- 1 file changed, 43 insertions(+), 4 deletions(-) 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 0fa6c94bf154..6d8e9b4391c3 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 @@ -68,6 +68,7 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.hadoop.ConfigProperties; @@ -79,6 +80,8 @@ import org.apache.thrift.TException; 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.EnumSource; public class HiveTableTest extends HiveTableBaseTest { static final String NON_DEFAULT_DATABASE = "nondefault"; @@ -334,7 +337,8 @@ public void testListTables() throws TException, IOException { // create a hive table String hiveTableName = "test_hive_table"; - org.apache.hadoop.hive.metastore.api.Table hiveTable = createHiveTable(hiveTableName); + org.apache.hadoop.hive.metastore.api.Table hiveTable = + createHiveTable(hiveTableName, TableType.EXTERNAL_TABLE); HIVE_METASTORE_EXTENSION.metastoreClient().createTable(hiveTable); catalog.setListAllTables(false); @@ -349,8 +353,43 @@ public void testListTables() throws TException, IOException { HIVE_METASTORE_EXTENSION.metastoreClient().dropTable(DB_NAME, hiveTableName); } - private org.apache.hadoop.hive.metastore.api.Table createHiveTable(String hiveTableName) - throws IOException { + @ParameterizedTest + @EnumSource( + value = TableType.class, + names = {"EXTERNAL_TABLE", "VIRTUAL_VIEW", "MANAGED_TABLE"}) + public void testHiveTableAndIcebergTableWithSameName(TableType tableType) + throws TException, IOException { + + assertThat(catalog.listTables(TABLE_IDENTIFIER.namespace())) + .hasSize(1) + .containsExactly(TABLE_IDENTIFIER); + + // create a hive table with a defined table type. + String hiveTableName = "test_hive_table"; + TableIdentifier identifier = TableIdentifier.of(DB_NAME, hiveTableName); + HIVE_METASTORE_EXTENSION + .metastoreClient() + .createTable(createHiveTable(hiveTableName, tableType)); + + catalog.setListAllTables(true); + assertThat(catalog.listTables(TABLE_IDENTIFIER.namespace())) + .hasSize(2) + .containsExactly(TABLE_IDENTIFIER, identifier); + catalog.setListAllTables(false); // reset to default. + + // create an iceberg table with the same name + assertThatThrownBy(() -> catalog.createTable(identifier, schema, PartitionSpec.unpartitioned())) + .isInstanceOf(NoSuchIcebergTableException.class) + .hasMessageStartingWith(String.format("Not an iceberg table: hive.%s", identifier)); + + assertThat(catalog.tableExists(identifier)).isFalse(); + + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); + HIVE_METASTORE_EXTENSION.metastoreClient().dropTable(DB_NAME, hiveTableName); + } + + private org.apache.hadoop.hive.metastore.api.Table createHiveTable( + String hiveTableName, TableType type) throws IOException { Map parameters = Maps.newHashMap(); parameters.put( serdeConstants.SERIALIZATION_CLASS, "org.apache.hadoop.hive.serde2.thrift.test.IntString"); @@ -387,7 +426,7 @@ private org.apache.hadoop.hive.metastore.api.Table createHiveTable(String hiveTa Maps.newHashMap(), "viewOriginalText", "viewExpandedText", - TableType.EXTERNAL_TABLE.name()); + type.name()); return hiveTable; } From 9969d1333450ae05c8b219cead6de580c8377903 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 26 Mar 2024 23:58:48 +0800 Subject: [PATCH 0166/1019] Build: Bump Spark from 3.5 to 3.5.1 (#9832) --- 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 ce571331529f..79ef9c78a396 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -79,7 +79,7 @@ slf4j = "1.7.36" snowflake-jdbc = "3.14.5" spark-hive33 = "3.3.4" spark-hive34 = "3.4.2" -spark-hive35 = "3.5.0" +spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.33" sqlite-jdbc = "3.45.1.0" From 8125f63caa57761408f7aa428078ac8314b97598 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 27 Mar 2024 07:58:18 +0100 Subject: [PATCH 0167/1019] Spark: Fail on recursive cycle in view (#9834) --- .../sql/catalyst/analysis/CheckViews.scala | 52 ++++++++++- .../iceberg/spark/extensions/TestViews.java | 87 +++++++++++++++++++ 2 files changed, 138 insertions(+), 1 deletion(-) diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 9a2dee997a10..b8cd1020298d 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -20,8 +20,12 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.AlterViewAs import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.catalyst.plans.logical.View import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View import org.apache.spark.sql.connector.catalog.ViewCatalog @@ -30,12 +34,18 @@ import org.apache.spark.sql.util.SchemaUtils object CheckViews extends (LogicalPlan => Unit) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + override def apply(plan: LogicalPlan): Unit = { plan foreach { case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, - _, _, _, _, _, _) => + _, _, _, _, replace, _) => verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) + if (replace) { + val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier + checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) + } case AlterViewAs(ResolvedV2View(_, _), _, _) => throw new AnalysisException("ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") @@ -63,4 +73,44 @@ object CheckViews extends (LogicalPlan => Unit) { } } } + + private def checkCyclicViewReference( + viewIdent: Seq[String], + plan: LogicalPlan, + cyclePath: Seq[Seq[String]]): Unit = { + plan match { + case sub@SubqueryAlias(_, Project(_, _)) => + val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name + checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children) + case v1View: View => + val currentViewIdent: Seq[String] = v1View.desc.identifier.nameParts + checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, v1View.children) + case _ => + plan.children.foreach(child => checkCyclicViewReference(viewIdent, child, cyclePath)) + } + + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => + checkCyclicViewReference(viewIdent, e.plan, cyclePath) + None + case _ => None + }) + } + + private def checkIfRecursiveView( + viewIdent: Seq[String], + currentViewIdent: Seq[String], + cyclePath: Seq[Seq[String]], + children: Seq[LogicalPlan] + ): Unit = { + val newCyclePath = cyclePath :+ currentViewIdent + if (currentViewIdent == viewIdent) { + throw new AnalysisException(String.format("Recursive cycle in view detected: %s (cycle: %s)", + viewIdent.asIdentifier, newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) + } else { + children.foreach { c => + checkCyclicViewReference(viewIdent, c, newCyclePath) + } + } + } } 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 bd611b936a59..3cc1e32d00a1 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 @@ -1866,6 +1866,93 @@ public void replacingViewWithDialectDropAllowed() { .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); } + @TestTemplate + public void createViewWithRecursiveCycle() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // 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", catalogName, 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) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + + @TestTemplate + public void createViewWithRecursiveCycleToV1View() { + String viewOne = viewName("view_one"); + String viewTwo = viewName("view_two"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("USE spark_catalog"); + sql("CREATE VIEW %s AS SELECT * FROM %s.%s.%s", viewTwo, catalogName, NAMESPACE, viewOne); + + 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 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) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + + @TestTemplate + public void createViewWithRecursiveCycleInCTE() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // CTE points to viewTwo + String sql = + String.format( + "WITH max_by_data AS (SELECT max(id) as max FROM %s) " + + "SELECT max, count(1) AS count FROM max_by_data GROUP BY max", + viewTwo); + + // viewOne points to CTE, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + + @TestTemplate + public void createViewWithRecursiveCycleInSubqueryExpression() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // subquery expression points to viewTwo + String sql = + String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, viewTwo); + + // viewOne points to subquery expression, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + private void insertRows(int numRows) throws NoSuchTableException { List records = Lists.newArrayListWithCapacity(numRows); for (int i = 1; i <= numRows; i++) { From 07d843501f5192852fb742ed63a66c172c831328 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Wed, 27 Mar 2024 15:27:51 +0800 Subject: [PATCH 0168/1019] Build: disable link-check for existing medium blog posts (#10042) --- site/docs/blogs.md | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index cf4e3254981b..4e94c9e71b08 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -59,16 +59,19 @@ 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/) + ### [How not to use Apache Iceberg](https://medium.com/@ajanthabhat/how-not-to-use-apache-iceberg-046ae7e7c884) **Date**: January 23rd, 2024, **Company**: Dremio **Authors**: [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [Apache Hive-4.x with Iceberg Branches & Tags](https://medium.com/@ayushtkn/apache-hive-4-x-with-iceberg-branches-tags-3d52293ac0bf/) **Date**: October 12th, 2023, **Company**: Cloudera **Authors**: [Ayush Saxena](https://www.linkedin.com/in/ayush151/) + ### [Apache Hive 4.x With Apache Iceberg](https://medium.com/@ayushtkn/apache-hive-4-x-with-apache-iceberg-part-i-355e7a380725/) **Date**: October 12th, 2023, **Company**: Cloudera @@ -99,6 +102,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Riza Suminto](https://www.linkedin.com/in/rizasuminto/) + ### [How Bilibili Builds OLAP Data Lakehouse with Apache Iceberg](https://medium.com/@lirui.fudan/how-bilibili-builds-olap-data-lakehouse-with-apache-iceberg-9f3408e53f9) **Date**: June 14th, 2023, **Company**: Bilibili @@ -165,21 +169,25 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Benny Chow](https://www.linkedin.com/in/bechow/) + ### [Understanding Iceberg Table Metadata](https://medium.com/snowflake/understanding-iceberg-table-metadata-b1209fbcc7c3) **Date**: January 30st, 2023, **Company**: Snowflake **Author**: [Phani Raj](https://www.linkedin.com/in/phani-raj-9830a31b/) + ### [Creating and managing Apache Iceberg tables using serverless features and without coding](https://medium.com/snowflake/creating-and-managing-apache-iceberg-tables-using-serverless-features-and-without-coding-14d2198cf5b5) **Date**: January 27th, 2023, **Company**: Snowflake **Author**: [Parag Jain](https://www.linkedin.com/in/paragjainsa/) + ### [Getting started with Apache Iceberg](https://medium.com/snowflake/getting-started-with-apache-iceberg-80f338921a31) **Date**: January 27th, 2023, **Company**: Snowflake **Author**: [Jedidiah Rajbhushan](https://www.linkedin.com/in/jrajbhushan/) + ### [How Apache Iceberg enables ACID compliance for data lakes](https://medium.com/snowflake/how-apache-iceberg-enables-acid-compliance-for-data-lakes-9069ae783b60/) **Date**: January 13th, 2023, **Company**: Snowflake From f994262f4ee057efbe0cc3c6ba560ea4e08b0045 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 27 Mar 2024 16:59:55 +0100 Subject: [PATCH 0169/1019] Spark 3.4: Fail on recursive cycle in view (#10048) --- .../sql/catalyst/analysis/CheckViews.scala | 52 ++++++++++- .../iceberg/spark/extensions/TestViews.java | 87 +++++++++++++++++++ 2 files changed, 138 insertions(+), 1 deletion(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala index 4debc4d343a0..685b85a0d75f 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckViews.scala @@ -20,8 +20,12 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression import org.apache.spark.sql.catalyst.plans.logical.AlterViewAs import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.Project +import org.apache.spark.sql.catalyst.plans.logical.SubqueryAlias +import org.apache.spark.sql.catalyst.plans.logical.View import org.apache.spark.sql.catalyst.plans.logical.views.CreateIcebergView import org.apache.spark.sql.catalyst.plans.logical.views.ResolvedV2View import org.apache.spark.sql.connector.catalog.ViewCatalog @@ -30,12 +34,18 @@ import org.apache.spark.sql.util.SchemaUtils object CheckViews extends (LogicalPlan => Unit) { + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + override def apply(plan: LogicalPlan): Unit = { plan foreach { case CreateIcebergView(resolvedIdent@ResolvedIdentifier(_: ViewCatalog, _), _, query, columnAliases, _, - _, _, _, _, _, _) => + _, _, _, _, replace, _) => verifyColumnCount(resolvedIdent, columnAliases, query) SchemaUtils.checkColumnNameDuplication(query.schema.fieldNames, SQLConf.get.resolver) + if (replace) { + val viewIdent: Seq[String] = resolvedIdent.catalog.name() +: resolvedIdent.identifier.asMultipartIdentifier + checkCyclicViewReference(viewIdent, query, Seq(viewIdent)) + } case AlterViewAs(ResolvedV2View(_, _), _, _) => throw new AnalysisException("ALTER VIEW AS is not supported. Use CREATE OR REPLACE VIEW instead") @@ -59,4 +69,44 @@ object CheckViews extends (LogicalPlan => Unit) { } } } + + private def checkCyclicViewReference( + viewIdent: Seq[String], + plan: LogicalPlan, + cyclePath: Seq[Seq[String]]): Unit = { + plan match { + case sub@SubqueryAlias(_, Project(_, _)) => + val currentViewIdent: Seq[String] = sub.identifier.qualifier :+ sub.identifier.name + checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, sub.children) + case v1View: View => + val currentViewIdent: Seq[String] = v1View.desc.identifier.nameParts + checkIfRecursiveView(viewIdent, currentViewIdent, cyclePath, v1View.children) + case _ => + plan.children.foreach(child => checkCyclicViewReference(viewIdent, child, cyclePath)) + } + + plan.expressions.flatMap(_.flatMap { + case e: SubqueryExpression => + checkCyclicViewReference(viewIdent, e.plan, cyclePath) + None + case _ => None + }) + } + + private def checkIfRecursiveView( + viewIdent: Seq[String], + currentViewIdent: Seq[String], + cyclePath: Seq[Seq[String]], + children: Seq[LogicalPlan] + ): Unit = { + val newCyclePath = cyclePath :+ currentViewIdent + if (currentViewIdent == viewIdent) { + throw new AnalysisException(String.format("Recursive cycle in view detected: %s (cycle: %s)", + viewIdent.asIdentifier, newCyclePath.map(p => p.mkString(".")).mkString(" -> "))) + } else { + children.foreach { c => + checkCyclicViewReference(viewIdent, c, newCyclePath) + } + } + } } 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 5d1cb2db612b..624b4e354937 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 @@ -1867,6 +1867,93 @@ public void replacingViewWithDialectDropAllowed() { .isEqualTo(ImmutableSQLViewRepresentation.builder().dialect("spark").sql(sql).build()); } + @Test + public void createViewWithRecursiveCycle() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // 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", catalogName, 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) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + + @Test + public void createViewWithRecursiveCycleToV1View() { + String viewOne = viewName("view_one"); + String viewTwo = viewName("view_two"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("USE spark_catalog"); + sql("CREATE VIEW %s AS SELECT * FROM %s.%s.%s", viewTwo, catalogName, NAMESPACE, viewOne); + + 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 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) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + + @Test + public void createViewWithRecursiveCycleInCTE() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // CTE points to viewTwo + String sql = + String.format( + "WITH max_by_data AS (SELECT max(id) as max FROM %s) " + + "SELECT max, count(1) AS count FROM max_by_data GROUP BY max", + viewTwo); + + // viewOne points to CTE, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + + @Test + public void createViewWithRecursiveCycleInSubqueryExpression() { + String viewOne = viewName("viewOne"); + String viewTwo = viewName("viewTwo"); + + sql("CREATE VIEW %s AS SELECT * FROM %s", viewOne, tableName); + // viewTwo points to viewOne + sql("CREATE VIEW %s AS SELECT * FROM %s", viewTwo, viewOne); + + // subquery expression points to viewTwo + String sql = + String.format("SELECT * FROM %s WHERE id = (SELECT id FROM %s)", tableName, viewTwo); + + // viewOne points to subquery expression, creating a recursive cycle + String view1 = String.format("%s.%s.%s", catalogName, NAMESPACE, viewOne); + String cycle = String.format("%s -> %s -> %s", view1, viewTwo, view1); + assertThatThrownBy(() -> sql("CREATE OR REPLACE VIEW %s AS %s", viewOne, sql)) + .isInstanceOf(AnalysisException.class) + .hasMessageStartingWith( + String.format("Recursive cycle in view detected: %s (cycle: %s)", view1, cycle)); + } + private void insertRows(int numRows) throws NoSuchTableException { List records = Lists.newArrayListWithCapacity(numRows); for (int i = 1; i <= numRows; i++) { From 5725031d12f73cdebff5d7d23bcc94ebf019f44f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:18:49 +0100 Subject: [PATCH 0170/1019] Build: Bump org.xerial:sqlite-jdbc from 3.45.1.0 to 3.45.2.0 (#9974) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.45.1.0 to 3.45.2.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.45.1.0...3.45.2.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 79ef9c78a396..5749c6f82d54 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ spark-hive34 = "3.4.2" spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.33" -sqlite-jdbc = "3.45.1.0" +sqlite-jdbc = "3.45.2.0" testcontainers = "1.19.5" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 3d50e353e5d68da09ecd9f564548e4607799b4ec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:19:20 +0100 Subject: [PATCH 0171/1019] Build: Bump io.netty:netty-buffer from 4.1.107.Final to 4.1.108.Final (#10032) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.107.Final to 4.1.108.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.107.Final...netty-4.1.108.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 5749c6f82d54..05b1369cd765 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,8 +66,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.79.0" -netty-buffer = "4.1.107.Final" -netty-buffer-compat = "4.1.107.Final" +netty-buffer = "4.1.108.Final" +netty-buffer-compat = "4.1.108.Final" object-client-bundle = "3.3.2" orc = "1.9.2" parquet = "1.13.1" From e872865a6644b8e8f85cd56ad84a38da51d901b8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:20:00 +0100 Subject: [PATCH 0172/1019] Build: Bump arrow from 15.0.1 to 15.0.2 (#10034) Bumps `arrow` from 15.0.1 to 15.0.2. Updates `org.apache.arrow:arrow-memory-netty` from 15.0.1 to 15.0.2 Updates `org.apache.arrow:arrow-vector` from 15.0.1 to 15.0.2 - [Commits](https://github.com/apache/arrow/compare/go/v15.0.1...go/v15.0.2) --- updated-dependencies: - dependency-name: org.apache.arrow:arrow-memory-netty dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.arrow:arrow-vector 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 05b1369cd765..df757c9690ff 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -24,7 +24,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.26" -arrow = "15.0.1" +arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" From 4aef0b76dce883312b63575d5ae327226568b74e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 09:33:10 -0700 Subject: [PATCH 0173/1019] Build: Bump kafka from 3.6.1 to 3.7.0 (#9855) Bumps `kafka` from 3.6.1 to 3.7.0. Updates `org.apache.kafka:kafka-clients` from 3.6.1 to 3.7.0 Updates `org.apache.kafka:connect-api` from 3.6.1 to 3.7.0 Updates `org.apache.kafka:connect-json` from 3.6.1 to 3.7.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 df757c9690ff..0e89682a6997 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -60,7 +60,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.3" jetty = "9.4.54.v20240208" junit = "5.10.1" -kafka = "3.6.1" +kafka = "3.7.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" From cb19e192b680cfe90dba6106e5a777414ec6572b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:35:38 +0100 Subject: [PATCH 0174/1019] Build: Bump orc from 1.9.2 to 1.9.3 (#10033) Bumps `orc` from 1.9.2 to 1.9.3. Updates `org.apache.orc:orc-core` from 1.9.2 to 1.9.3 Updates `org.apache.orc:orc-tools` from 1.9.2 to 1.9.3 --- 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 0e89682a6997..00ab2e4c548f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -69,7 +69,7 @@ nessie = "0.79.0" netty-buffer = "4.1.108.Final" netty-buffer-compat = "4.1.108.Final" object-client-bundle = "3.3.2" -orc = "1.9.2" +orc = "1.9.3" parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.0.5" From 90f21d9e7d563ead017a8824608cd810fa98fe73 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 17:40:26 +0100 Subject: [PATCH 0175/1019] Build: Bump com.azure:azure-sdk-bom from 1.2.20 to 1.2.21 (#9857) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.20 to 1.2.21. - [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.20...azure-sdk-bom_1.2.21) --- 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 00ab2e4c548f..298a8a74984d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" awssdk-bom = "2.24.5" -azuresdk-bom = "1.2.20" +azuresdk-bom = "1.2.21" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From f817ac835b31f0f5a8e0842525c6dbde30a1f170 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 27 Mar 2024 20:10:50 +0100 Subject: [PATCH 0176/1019] Build: Bump com.esotericsoftware:kryo from 4.0.2 to 4.0.3 (#9984) Bumps [com.esotericsoftware:kryo](https://github.com/EsotericSoftware/kryo) from 4.0.2 to 4.0.3. - [Release notes](https://github.com/EsotericSoftware/kryo/releases) - [Commits](https://github.com/EsotericSoftware/kryo/compare/kryo-parent-4.0.2...kryo-parent-4.0.3) --- updated-dependencies: - dependency-name: com.esotericsoftware:kryo 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 298a8a74984d..cec16519417a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -35,7 +35,7 @@ caffeine = "2.9.3" calcite = "1.10.0" delta-standalone = "3.1.0" delta-spark = "3.1.0" -esotericsoftware-kryo = "4.0.2" +esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.26.1" findbugs-jsr305 = "3.0.2" flink116 = { strictly = "1.16.3"} From efb24f04fd11f7535235a60fc47a6ddca1539f6b Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 27 Mar 2024 13:46:49 -0700 Subject: [PATCH 0177/1019] Flink: implement range partitioner for map data statistics (#9321) --- .../shuffle/MapRangePartitionerBenchmark.java | 199 ++++++++ .../sink/shuffle/MapRangePartitioner.java | 381 +++++++++++++++ .../sink/shuffle/TestMapRangePartitioner.java | 448 ++++++++++++++++++ jmh.gradle | 5 + 4 files changed, 1033 insertions(+) create mode 100644 flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +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.apache.iceberg.types.Types; +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.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..fb1a8f03a65c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,381 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +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; +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.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator comparator; + private final Map mapStatistics; + private final double closeFileCostInWeightPercentage; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + // lazily computed due to the need of numPartitions + private Map assignment; + private NavigableMap sortedStatsWithCloseFileCost; + + MapRangePartitioner( + Schema schema, + SortOrder sortOrder, + MapDataStatistics dataStatistics, + double closeFileCostInWeightPercentage) { + dataStatistics + .statistics() + .entrySet() + .forEach( + entry -> + Preconditions.checkArgument( + entry.getValue() > 0, + "Invalid statistics: weight is 0 for key %s", + entry.getKey())); + + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.mapStatistics = dataStatistics.statistics(); + this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // assignment table can only be built lazily when first referenced here, + // because number of partitions (downstream subtasks) is needed. + // the numPartitions is not available in the constructor. + Map assignmentMap = assignment(numPartitions); + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = assignmentMap.get(sortKey); + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + } + return (int) (newSortKeyCounter % numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostInWeight); + } + + return assignment; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostInWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostInWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostInWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks.clear(); + subtaskWeights.clear(); + currentKey = null; + } + } + + return assignmentMap; + } + + /** Subtask assignment for a key */ + @VisibleForTesting + static class KeyAssignment { + private final int[] assignedSubtasks; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It + * could also be multiple subtasks if the key has heavy weight that should be handled by + * multiple subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the + * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain + * values as [10, 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostInWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostInWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostInWeight)); + + this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.length == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks[0]; + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.length, + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks[position]; + } + } + + @Override + public int hashCode() { + return 31 * Arrays.hashCode(assignedSubtasks) + + Arrays.hashCode(subtaskWeightsExcludingCloseCost); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Arrays.equals(assignedSubtasks, that.assignedSubtasks) + && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) + .toString(); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..92eb71acc834 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.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.flink.sink.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 8; + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 9; + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 9; + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + partitioner + .mapStatistics() + .forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + Assertions.assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/jmh.gradle b/jmh.gradle index 076899239430..ea317cc2eea1 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -21,10 +21,15 @@ if (jdkVersion != '8' && jdkVersion != '11' && jdkVersion != '17') { throw new GradleException("The JMH benchmarks must be run with JDK 8 or JDK 11 or JDK 17") } +def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getProperty("sparkVersions") : System.getProperty("defaultSparkVersions")).split(",") def scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") def jmhProjects = [project(":iceberg-core"), project(":iceberg-data")] +if (flinkVersions.contains("1.17")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.17")) +} + if (sparkVersions.contains("3.3")) { jmhProjects.add(project(":iceberg-spark:iceberg-spark-3.3_${scalaVersion}")) jmhProjects.add(project(":iceberg-spark:iceberg-spark-extensions-3.3_${scalaVersion}")) From 8fe26374d645dcf9666d445d64e70f3304ead80f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 28 Mar 2024 11:38:26 +0100 Subject: [PATCH 0178/1019] Build: Bump software.amazon.awssdk:bom from 2.24.5 to 2.25.18 (#10050) Bumps software.amazon.awssdk:bom from 2.24.5 to 2.25.18. --- 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 cec16519417a..a0fb79bcd363 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.24.5" +awssdk-bom = "2.25.18" azuresdk-bom = "1.2.21" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 1a179eccf7305b0e9a21278e4334f811a5a278d4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 28 Mar 2024 11:39:25 +0100 Subject: [PATCH 0179/1019] CI: Run Markdown links checker only when `{docs,site}/**` changes (#10049) --- .github/workflows/docs-check-links.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/docs-check-links.yml b/.github/workflows/docs-check-links.yml index 9de842813dc3..4275a2d6efc4 100644 --- a/.github/workflows/docs-check-links.yml +++ b/.github/workflows/docs-check-links.yml @@ -27,6 +27,9 @@ on: branches: - 'main' pull_request: + paths: + - docs/** + - site/** workflow_dispatch: jobs: From decb76824624cb0f5b11485251c984b259c850ca Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Thu, 28 Mar 2024 23:21:31 +0900 Subject: [PATCH 0180/1019] Core, Data: Migrate tests to JUnit5 (#10039) --- .../apache/iceberg/FilterFilesTestBase.java | 11 +- .../java/org/apache/iceberg/TestMetrics.java | 170 +++--- .../org/apache/iceberg/TestWapWorkflow.java | 554 ++++++------------ .../iceberg/actions/TestCommitService.java | 39 +- .../actions/TestSizeBasedRewriter.java | 26 +- .../apache/iceberg/avro/AvroTestHelpers.java | 24 +- .../avro/TestNameMappingWithAvroSchema.java | 13 +- .../iceberg/encryption/TestGcmStreams.java | 104 ++-- .../TestStandardKeyMetadataParser.java | 13 +- .../iceberg/io/TestOutputFileFactory.java | 48 +- .../iceberg/mapping/TestMappingUpdates.java | 172 +++--- .../iceberg/mapping/TestNameMapping.java | 81 +-- .../apache/iceberg/orc/TestOrcMetrics.java | 28 +- .../iceberg/parquet/TestParquetMetrics.java | 17 +- 14 files changed, 554 insertions(+), 746 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java index ad92d0f662a3..bb4bb282a330 100644 --- a/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java +++ b/core/src/test/java/org/apache/iceberg/FilterFilesTestBase.java @@ -19,7 +19,7 @@ package org.apache.iceberg; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -28,7 +28,6 @@ import java.nio.file.Path; import java.util.Map; import org.apache.iceberg.expressions.Expressions; -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.Conversions; import org.apache.iceberg.types.Types; @@ -119,10 +118,10 @@ private void testFilterFiles(Table table) { table.refresh(); ScanT emptyScan = newScan(table).filter(Expressions.equal("id", 5)); - assertEquals(0, Iterables.size(emptyScan.planFiles())); + assertThat(emptyScan.planFiles()).isEmpty(); ScanT nonEmptyScan = newScan(table).filter(Expressions.equal("id", 1)); - assertEquals(1, Iterables.size(nonEmptyScan.planFiles())); + assertThat(nonEmptyScan.planFiles()).hasSize(1); } private void testCaseInsensitiveFilterFiles(Table table) { @@ -153,9 +152,9 @@ private void testCaseInsensitiveFilterFiles(Table table) { table.refresh(); ScanT emptyScan = newScan(table).caseSensitive(false).filter(Expressions.equal("ID", 5)); - assertEquals(0, Iterables.size(emptyScan.planFiles())); + assertThat(emptyScan.planFiles()).hasSize(0); ScanT nonEmptyScan = newScan(table).caseSensitive(false).filter(Expressions.equal("ID", 1)); - assertEquals(1, Iterables.size(nonEmptyScan.planFiles())); + assertThat(nonEmptyScan.planFiles()).hasSize(1); } } diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index 32bc6299ce1b..424e0e0a7b93 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -21,6 +21,8 @@ import static org.apache.iceberg.types.Conversions.fromByteBuffer; 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.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -28,9 +30,11 @@ 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; import java.util.Map; -import java.util.Objects; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.InputFile; @@ -56,21 +60,19 @@ import org.apache.iceberg.types.Types.TimeType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.util.DateTimeUtil; -import org.junit.After; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.io.TempDir; /** Tests for Metrics. */ public abstract class TestMetrics { - protected TestMetrics(int formatVersion) { - this.formatVersion = formatVersion; + @Parameters(name = "formatVersion = {0}") + public static List parameters() { + return Arrays.asList(1, 2); } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir public Path temp; private static final StructType LEAF_STRUCT_TYPE = StructType.of( @@ -113,10 +115,10 @@ protected TestMetrics(int formatVersion) { private static final Record NAN_ONLY_RECORD = createRecordWithFloatAndDouble(Float.NaN, Double.NaN); - private final int formatVersion; + @Parameter private int formatVersion; private final byte[] fixed = "abcd".getBytes(StandardCharsets.UTF_8); - @After + @AfterEach public void after() { TestTables.clearTables(); } @@ -146,7 +148,7 @@ public boolean supportsSmallRowGroups() { protected abstract OutputFile createOutputFile() throws IOException; - @Test + @TestTemplate public void testMetricsForRepeatedValues() throws IOException { Record record = GenericRecord.create(SIMPLE_SCHEMA); record.setField("booleanCol", true); @@ -164,7 +166,7 @@ public void testMetricsForRepeatedValues() throws IOException { record.setField("timestampColBelowEpoch", DateTimeUtil.timestampFromMicros(0L)); Metrics metrics = getMetrics(SIMPLE_SCHEMA, record, record); - Assert.assertEquals(2L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(2L); assertCounts(1, 2L, 0L, metrics); assertCounts(2, 2L, 0L, metrics); assertCounts(3, 2L, 2L, metrics); @@ -180,7 +182,7 @@ public void testMetricsForRepeatedValues() throws IOException { assertCounts(13, 2L, 0L, metrics); } - @Test + @TestTemplate public void testMetricsForTopLevelFields() throws IOException { Record firstRecord = GenericRecord.create(SIMPLE_SCHEMA); firstRecord.setField("booleanCol", true); @@ -212,7 +214,7 @@ public void testMetricsForTopLevelFields() throws IOException { secondRecord.setField("timestampColBelowEpoch", DateTimeUtil.timestampFromMicros(-7_000L)); Metrics metrics = getMetrics(SIMPLE_SCHEMA, firstRecord, secondRecord); - Assert.assertEquals(2L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(2L); assertCounts(1, 2L, 0L, metrics); assertBounds(1, BooleanType.get(), false, true, metrics); assertCounts(2, 2L, 0L, metrics); @@ -255,7 +257,7 @@ public void testMetricsForTopLevelFields() throws IOException { } } - @Test + @TestTemplate public void testMetricsForDecimals() throws IOException { Schema schema = new Schema( @@ -269,7 +271,7 @@ public void testMetricsForDecimals() throws IOException { record.setField("decimalAsFixed", new BigDecimal("5.80")); Metrics metrics = getMetrics(schema, record); - Assert.assertEquals(1L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(1); assertCounts(1, 1L, 0L, metrics); assertBounds(1, DecimalType.of(4, 2), new BigDecimal("2.55"), new BigDecimal("2.55"), metrics); assertCounts(2, 1L, 0L, metrics); @@ -278,10 +280,10 @@ public void testMetricsForDecimals() throws IOException { assertBounds(3, DecimalType.of(22, 2), new BigDecimal("5.80"), new BigDecimal("5.80"), metrics); } - @Test + @TestTemplate public void testMetricsForNestedStructFields() throws IOException { Metrics metrics = getMetrics(NESTED_SCHEMA, buildNestedTestRecord()); - Assert.assertEquals(1L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(1L); assertCounts(1, 1L, 0L, metrics); assertBounds(1, IntegerType.get(), Integer.MAX_VALUE, Integer.MAX_VALUE, metrics); assertCounts(3, 1L, 0L, metrics); @@ -299,7 +301,7 @@ public void testMetricsForNestedStructFields() throws IOException { assertBounds(7, DoubleType.get(), null, null, metrics); } - @Test + @TestTemplate public void testMetricsModeForNestedStructFields() throws IOException { Map properties = ImmutableMap.of( @@ -310,9 +312,9 @@ public void testMetricsModeForNestedStructFields() throws IOException { MetricsConfig config = MetricsConfig.fromProperties(properties); Metrics metrics = getMetrics(NESTED_SCHEMA, config, buildNestedTestRecord()); - Assert.assertEquals(1L, (long) metrics.recordCount()); - Assert.assertEquals(1, metrics.lowerBounds().size()); - Assert.assertEquals(1, metrics.upperBounds().size()); + assertThat(metrics.recordCount()).isEqualTo(1L); + assertThat(metrics.lowerBounds()).hasSize(1); + assertThat(metrics.upperBounds()).hasSize(1); assertBounds(3, LongType.get(), 100L, 100L, metrics); } @@ -331,7 +333,7 @@ private Record buildNestedTestRecord() { return record; } - @Test + @TestTemplate public void testMetricsForListAndMapElements() throws IOException { StructType structType = StructType.of( @@ -352,7 +354,7 @@ public void testMetricsForListAndMapElements() throws IOException { record.set(1, map); Metrics metrics = getMetrics(schema, record); - Assert.assertEquals(1L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(1L); if (fileFormat() != FileFormat.ORC) { assertCounts(1, 1L, 0L, metrics); assertCounts(2, 1L, 0L, metrics); @@ -371,7 +373,7 @@ public void testMetricsForListAndMapElements() throws IOException { assertBounds(7, structType, null, null, metrics); } - @Test + @TestTemplate public void testMetricsForNullColumns() throws IOException { Schema schema = new Schema(optional(1, "intCol", IntegerType.get())); Record firstRecord = GenericRecord.create(schema); @@ -380,15 +382,15 @@ public void testMetricsForNullColumns() throws IOException { secondRecord.setField("intCol", null); Metrics metrics = getMetrics(schema, firstRecord, secondRecord); - Assert.assertEquals(2L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(2L); assertCounts(1, 2L, 2L, metrics); assertBounds(1, IntegerType.get(), null, null, metrics); } - @Test + @TestTemplate public void testMetricsForNaNColumns() throws IOException { Metrics metrics = getMetrics(FLOAT_DOUBLE_ONLY_SCHEMA, NAN_ONLY_RECORD, NAN_ONLY_RECORD); - Assert.assertEquals(2L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(2L); assertCounts(1, 2L, 0L, 2L, metrics); assertCounts(2, 2L, 0L, 2L, metrics); @@ -396,7 +398,7 @@ public void testMetricsForNaNColumns() throws IOException { assertBounds(2, DoubleType.get(), null, null, metrics); } - @Test + @TestTemplate public void testColumnBoundsWithNaNValueAtFront() throws IOException { Metrics metrics = getMetrics( @@ -404,7 +406,7 @@ public void testColumnBoundsWithNaNValueAtFront() throws IOException { NAN_ONLY_RECORD, FLOAT_DOUBLE_RECORD_1, FLOAT_DOUBLE_RECORD_2); - Assert.assertEquals(3L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(3L); assertCounts(1, 3L, 0L, 1L, metrics); assertCounts(2, 3L, 0L, 1L, metrics); @@ -412,7 +414,7 @@ public void testColumnBoundsWithNaNValueAtFront() throws IOException { assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); } - @Test + @TestTemplate public void testColumnBoundsWithNaNValueInMiddle() throws IOException { Metrics metrics = getMetrics( @@ -420,7 +422,7 @@ public void testColumnBoundsWithNaNValueInMiddle() throws IOException { FLOAT_DOUBLE_RECORD_1, NAN_ONLY_RECORD, FLOAT_DOUBLE_RECORD_2); - Assert.assertEquals(3L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(3L); assertCounts(1, 3L, 0L, 1L, metrics); assertCounts(2, 3L, 0L, 1L, metrics); @@ -428,7 +430,7 @@ public void testColumnBoundsWithNaNValueInMiddle() throws IOException { assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); } - @Test + @TestTemplate public void testColumnBoundsWithNaNValueAtEnd() throws IOException { Metrics metrics = getMetrics( @@ -436,7 +438,7 @@ public void testColumnBoundsWithNaNValueAtEnd() throws IOException { FLOAT_DOUBLE_RECORD_1, FLOAT_DOUBLE_RECORD_2, NAN_ONLY_RECORD); - Assert.assertEquals(3L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(3L); assertCounts(1, 3L, 0L, 1L, metrics); assertCounts(2, 3L, 0L, 1L, metrics); @@ -444,10 +446,11 @@ public void testColumnBoundsWithNaNValueAtEnd() throws IOException { assertBounds(2, DoubleType.get(), 3.4D, 7.8D, metrics); } - @Test + @TestTemplate public void testMetricsForTopLevelWithMultipleRowGroup() throws Exception { - Assume.assumeTrue( - "Skip test for formats that do not support small row groups", supportsSmallRowGroups()); + assumeThat(supportsSmallRowGroups()) + .as("Skip test for formats that do not support small row groups") + .isTrue(); int recordCount = 201; List records = Lists.newArrayListWithExpectedSize(recordCount); @@ -479,11 +482,11 @@ public void testMetricsForTopLevelWithMultipleRowGroup() throws Exception { SIMPLE_SCHEMA, outputFile, records.toArray(new Record[0])); InputFile recordsFile = outputFile.toInputFile(); - Assert.assertNotNull(recordsFile); + assertThat(recordsFile).isNotNull(); // rowgroup size should be > 1 - Assert.assertEquals(3, splitCount(recordsFile)); + assertThat(splitCount(recordsFile)).isEqualTo(3); - Assert.assertEquals(201L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(201L); assertCounts(1, 201L, 0L, metrics); assertBounds(1, Types.BooleanType.get(), false, true, metrics); assertBounds(2, Types.IntegerType.get(), 1, 201, metrics); @@ -498,10 +501,11 @@ public void testMetricsForTopLevelWithMultipleRowGroup() throws Exception { 6, Types.DecimalType.of(10, 2), new BigDecimal("2.00"), new BigDecimal("201.00"), metrics); } - @Test + @TestTemplate public void testMetricsForNestedStructFieldsWithMultipleRowGroup() throws IOException { - Assume.assumeTrue( - "Skip test for formats that do not support small row groups", supportsSmallRowGroups()); + assumeThat(supportsSmallRowGroups()) + .as("Skip test for formats that do not support small row groups") + .isTrue(); int recordCount = 201; List records = Lists.newArrayListWithExpectedSize(recordCount); @@ -527,11 +531,11 @@ public void testMetricsForNestedStructFieldsWithMultipleRowGroup() throws IOExce NESTED_SCHEMA, outputFile, records.toArray(new Record[0])); InputFile recordsFile = outputFile.toInputFile(); - Assert.assertNotNull(recordsFile); + assertThat(recordsFile).isNotNull(); // rowgroup size should be > 1 - Assert.assertEquals(3, splitCount(recordsFile)); + assertThat(splitCount(recordsFile)).isEqualTo(3); - Assert.assertEquals(201L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(201L); assertCounts(1, 201L, 0L, metrics); assertBounds(1, IntegerType.get(), 1, 201, metrics); assertCounts(3, 201L, 0L, metrics); @@ -549,15 +553,15 @@ public void testMetricsForNestedStructFieldsWithMultipleRowGroup() throws IOExce assertBounds(7, DoubleType.get(), null, null, metrics); } - @Test + @TestTemplate public void testNoneMetricsMode() throws IOException { Metrics metrics = getMetrics( NESTED_SCHEMA, MetricsConfig.fromProperties(ImmutableMap.of("write.metadata.metrics.default", "none")), buildNestedTestRecord()); - Assert.assertEquals(1L, (long) metrics.recordCount()); - Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull)); + assertThat(metrics.recordCount()).isEqualTo(1L); + assertThat(metrics.columnSizes()).doesNotContainValue(null); assertCounts(1, null, null, metrics); assertBounds(1, Types.IntegerType.get(), null, null, metrics); assertCounts(3, null, null, metrics); @@ -570,7 +574,7 @@ public void testNoneMetricsMode() throws IOException { assertBounds(7, Types.DoubleType.get(), null, null, metrics); } - @Test + @TestTemplate public void testCountsMetricsMode() throws IOException { Metrics metrics = getMetrics( @@ -578,8 +582,8 @@ public void testCountsMetricsMode() throws IOException { MetricsConfig.fromProperties( ImmutableMap.of("write.metadata.metrics.default", "counts")), buildNestedTestRecord()); - Assert.assertEquals(1L, (long) metrics.recordCount()); - Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull)); + assertThat(metrics.recordCount()).isEqualTo(1L); + assertThat(metrics.columnSizes()).doesNotContainValue(null); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.IntegerType.get(), null, null, metrics); assertCounts(3, 1L, 0L, metrics); @@ -592,15 +596,15 @@ public void testCountsMetricsMode() throws IOException { assertBounds(7, Types.DoubleType.get(), null, null, metrics); } - @Test + @TestTemplate public void testFullMetricsMode() throws IOException { Metrics metrics = getMetrics( NESTED_SCHEMA, MetricsConfig.fromProperties(ImmutableMap.of("write.metadata.metrics.default", "full")), buildNestedTestRecord()); - Assert.assertEquals(1L, (long) metrics.recordCount()); - Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull)); + assertThat(metrics.recordCount()).isEqualTo(1L); + assertThat(metrics.columnSizes()).doesNotContainValue(null); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.IntegerType.get(), Integer.MAX_VALUE, Integer.MAX_VALUE, metrics); assertCounts(3, 1L, 0L, metrics); @@ -618,7 +622,7 @@ public void testFullMetricsMode() throws IOException { assertBounds(7, Types.DoubleType.get(), null, null, metrics); } - @Test + @TestTemplate public void testTruncateStringMetricsMode() throws IOException { String colName = "str_to_truncate"; Schema singleStringColSchema = new Schema(required(1, colName, Types.StringType.get())); @@ -636,13 +640,13 @@ public void testTruncateStringMetricsMode() throws IOException { CharBuffer expectedMinBound = CharBuffer.wrap("Lorem ipsu"); CharBuffer expectedMaxBound = CharBuffer.wrap("Lorem ipsv"); - Assert.assertEquals(1L, (long) metrics.recordCount()); - Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull)); + assertThat(metrics.recordCount()).isEqualTo(1L); + assertThat(metrics.columnSizes()).doesNotContainValue(null); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.StringType.get(), expectedMinBound, expectedMaxBound, metrics); } - @Test + @TestTemplate public void testTruncateBinaryMetricsMode() throws IOException { String colName = "bin_to_truncate"; Schema singleBinaryColSchema = new Schema(required(1, colName, Types.BinaryType.get())); @@ -660,16 +664,16 @@ public void testTruncateBinaryMetricsMode() throws IOException { ByteBuffer expectedMinBounds = ByteBuffer.wrap(new byte[] {0x1, 0x2, 0x3, 0x4, 0x5}); ByteBuffer expectedMaxBounds = ByteBuffer.wrap(new byte[] {0x1, 0x2, 0x3, 0x4, 0x6}); - Assert.assertEquals(1L, (long) metrics.recordCount()); - Assert.assertTrue(metrics.columnSizes().values().stream().allMatch(Objects::nonNull)); + assertThat(metrics.recordCount()).isEqualTo(1L); + assertThat(metrics.columnSizes()).doesNotContainValue(null); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.BinaryType.get(), expectedMinBounds, expectedMaxBounds, metrics); } - @Test + @TestTemplate public void testSortedColumnMetrics() throws IOException { - File tableDir = temp.newFolder(); - tableDir.delete(); // created by table create + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created by table create SortOrder sortOrder = SortOrder.builderFor(SIMPLE_SCHEMA) @@ -719,7 +723,7 @@ public void testSortedColumnMetrics() throws IOException { Metrics metrics = getMetrics(SIMPLE_SCHEMA, MetricsConfig.forTable(table), firstRecord, secondRecord); - Assert.assertEquals(2L, (long) metrics.recordCount()); + assertThat(metrics.recordCount()).isEqualTo(2L); assertBounds(1, BooleanType.get(), false, true, metrics); assertBounds(2, IntegerType.get(), Integer.MIN_VALUE, Integer.MAX_VALUE, metrics); assertBounds(3, LongType.get(), Long.MIN_VALUE, Long.MAX_VALUE, metrics); @@ -729,10 +733,10 @@ public void testSortedColumnMetrics() throws IOException { assertBounds(8, DateType.get(), 1500, 3000, metrics); } - @Test + @TestTemplate public void testMetricsForSortedNestedStructFields() throws IOException { - File tableDir = temp.newFolder(); - tableDir.delete(); // created by table create + File tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created by table create SortOrder sortOrder = SortOrder.builderFor(NESTED_SCHEMA) @@ -766,12 +770,9 @@ protected void assertCounts(int fieldId, Long valueCount, Long nullValueCount, M protected void assertCounts( int fieldId, Long valueCount, Long nullValueCount, Long nanValueCount, Metrics metrics) { - Map valueCounts = metrics.valueCounts(); - Map nullValueCounts = metrics.nullValueCounts(); - Map nanValueCounts = metrics.nanValueCounts(); - Assert.assertEquals(valueCount, valueCounts.get(fieldId)); - Assert.assertEquals(nullValueCount, nullValueCounts.get(fieldId)); - Assert.assertEquals(nanValueCount, nanValueCounts.get(fieldId)); + assertThat(metrics.valueCounts().get(fieldId)).isEqualTo(valueCount); + assertThat(metrics.nullValueCounts().get(fieldId)).isEqualTo(nullValueCount); + assertThat(metrics.nanValueCounts().get(fieldId)).isEqualTo(nanValueCount); } protected void assertBounds( @@ -779,11 +780,16 @@ protected void assertBounds( Map lowerBounds = metrics.lowerBounds(); Map upperBounds = metrics.upperBounds(); - Assert.assertEquals( - lowerBound, - lowerBounds.containsKey(fieldId) ? fromByteBuffer(type, lowerBounds.get(fieldId)) : null); - Assert.assertEquals( - upperBound, - upperBounds.containsKey(fieldId) ? fromByteBuffer(type, upperBounds.get(fieldId)) : null); + if (lowerBounds.containsKey(fieldId)) { + assertThat((Object) fromByteBuffer(type, lowerBounds.get(fieldId))).isEqualTo(lowerBound); + } else { + assertThat(lowerBound).isNull(); + } + + if (upperBounds.containsKey(fieldId)) { + assertThat((Object) fromByteBuffer(type, upperBounds.get(fieldId))).isEqualTo(upperBound); + } else { + assertThat(upperBound).isNull(); + } } } diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java index c65d469e3262..58b83f43f399 100644 --- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java +++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java @@ -18,35 +18,32 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.exceptions.CherrypickAncestorCommitException; import org.apache.iceberg.exceptions.DuplicateWAPCommitException; import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestWapWorkflow extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestWapWorkflow(int formatVersion) { - super(formatVersion); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestWapWorkflow extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - @Before + @BeforeEach public void setupTableProperties() { table.updateProperties().set(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, "true").commit(); } - @Test + @TestTemplate public void testCherryPickOverwrite() { table.newAppend().appendFile(FILE_A).commit(); @@ -68,7 +65,7 @@ public void testCherryPickOverwrite() { validateTableFiles(table, FILE_B); } - @Test + @TestTemplate public void testCherryPickOverwriteFailsIfCurrentHasChanged() { table.newAppend().appendFile(FILE_A).commit(); @@ -87,8 +84,7 @@ public void testCherryPickOverwriteFailsIfCurrentHasChanged() { .get(); // try to cherry-pick, which should fail because the overwrite's parent is no longer current - Assertions.assertThatThrownBy( - () -> table.manageSnapshots().cherrypick(overwrite.snapshotId()).commit()) + assertThatThrownBy(() -> table.manageSnapshots().cherrypick(overwrite.snapshotId()).commit()) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot cherry-pick snapshot 2: not append, dynamic overwrite, or fast-forward"); @@ -97,7 +93,7 @@ public void testCherryPickOverwriteFailsIfCurrentHasChanged() { validateTableFiles(table, FILE_A, FILE_C); } - @Test + @TestTemplate public void testCurrentSnapshotOperation() { table.newAppend().appendFile(FILE_A).commit(); @@ -109,38 +105,27 @@ public void testCurrentSnapshotOperation() { Snapshot wapSnapshot = base.snapshots().get(1); - Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Snapshot should have wap id in summary", "123456789", wapSnapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(2); + assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // do setCurrentSnapshot table.manageSnapshots().setCurrentSnapshot(wapSnapshot.snapshotId()).commit(); base = readMetadata(); - Assert.assertEquals( - "Current snapshot should be what we rolled back to", - wapSnapshot.snapshotId(), - base.currentSnapshot().snapshotId()); - Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should contain manifests for both files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(wapSnapshot.snapshotId()); + assertThat(base.snapshots()).hasSize(2); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); } - @Test + @TestTemplate public void testSetCurrentSnapshotNoWAP() { table.newAppend().appendFile(FILE_A).commit(); @@ -154,24 +139,16 @@ public void testSetCurrentSnapshotNoWAP() { table.manageSnapshots().setCurrentSnapshot(firstSnapshotId).commit(); base = readMetadata(); - Assert.assertEquals( - "Current snapshot should be what we rolled back to", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should contain manifests for both files", - 1, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 3, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshots()).hasSize(2); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(1); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(3); } - @Test + @TestTemplate public void testRollbackOnInvalidNonAncestor() { table.newAppend().appendFile(FILE_A).commit(); @@ -183,42 +160,31 @@ public void testRollbackOnInvalidNonAncestor() { Snapshot wapSnapshot = base.snapshots().get(1); - Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Snapshot should have wap id in summary", "123456789", wapSnapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(2); + assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // do rollback - Assertions.assertThatThrownBy( + assertThatThrownBy( // rollback to snapshot that is not an ancestor () -> table.manageSnapshots().rollbackTo(wapSnapshot.snapshotId()).commit()) .isInstanceOf(ValidationException.class) .hasMessage("Cannot roll back to snapshot, not an ancestor of the current state: 2"); base = readMetadata(); - Assert.assertEquals( - "Current snapshot should be what we rolled back to", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals("Metadata should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should contain manifests for one snapshot", - 1, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshots()).hasSize(2); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(1); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); } - @Test + @TestTemplate public void testRollbackAndCherrypick() { // first snapshot table.newAppend().appendFile(FILE_A).commit(); @@ -239,26 +205,22 @@ public void testRollbackAndCherrypick() { // rollback to first snapshot table.manageSnapshots().rollbackTo(firstSnapshotId).commit(); base = readMetadata(); - Assert.assertEquals( - "Should be at first snapshot", firstSnapshotId, base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should have all three snapshots in the system", 3, base.snapshots().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshots()).hasSize(3); // fast forward to third snapshot table.manageSnapshots().cherrypick(thirdSnapshot.snapshotId()).commit(); base = readMetadata(); - Assert.assertEquals( - "Current state should be at third snapshot", 4, base.currentSnapshot().snapshotId()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(4); // fast forward to 2nd snapshot table.manageSnapshots().cherrypick(secondSnapshot.snapshotId()).commit(); base = readMetadata(); - Assert.assertEquals( - "Current state should be at second snapshot", 5, base.currentSnapshot().snapshotId()); - Assert.assertEquals("Count all snapshots", 5, base.snapshots().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(5); + assertThat(base.snapshots()).hasSize(5); } - @Test + @TestTemplate public void testRollbackToTime() { // first snapshot @@ -279,13 +241,11 @@ public void testRollbackToTime() { table.manageSnapshots().rollbackToTime(secondSnapshot.timestampMillis()).commit(); base = readMetadata(); - Assert.assertEquals( - "Should be at first snapshot", firstSnapshotId, base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should have all three snapshots in the system", 3, base.snapshots().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshots()).hasSize(3); } - @Test + @TestTemplate public void testWithCherryPicking() { table.newAppend().appendFile(FILE_A).commit(); @@ -299,15 +259,12 @@ public void testWithCherryPicking() { // pick the snapshot that's staged but not committed Snapshot wapSnapshot = base.snapshots().get(1); - Assert.assertEquals("Should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should have first wap id in summary", "123456789", wapSnapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(2); + assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // cherry-pick snapshot table.manageSnapshots().cherrypick(wapSnapshot.snapshotId()).commit(); @@ -315,24 +272,16 @@ public void testWithCherryPicking() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals( - "Current snapshot should be fast-forwarded to wap snapshot", - wapSnapshot.snapshotId(), - base.currentSnapshot().snapshotId()); - Assert.assertEquals("Should have two snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should contain manifests for both files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(wapSnapshot.snapshotId()); + assertThat(base.snapshots()).hasSize(2); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); } - @Test + @TestTemplate public void testWithTwoPhaseCherryPicking() { table.newAppend().appendFile(FILE_A).commit(); @@ -352,25 +301,15 @@ public void testWithTwoPhaseCherryPicking() { Snapshot wap1Snapshot = base.snapshots().get(1); Snapshot wap2Snapshot = base.snapshots().get(2); - Assert.assertEquals("Should have three snapshots", 3, base.snapshots().size()); - Assert.assertEquals( - "Should have first wap id in summary", "123456789", wap1Snapshot.summary().get("wap.id")); - Assert.assertEquals( - "Should have second wap id in summary", "987654321", wap2Snapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Parent snapshot id should be same for first WAP snapshot", - firstSnapshotId, - wap1Snapshot.parentId().longValue()); - Assert.assertEquals( - "Parent snapshot id should be same for second WAP snapshot", - firstSnapshotId, - wap2Snapshot.parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(3); + assertThat(wap1Snapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(wap2Snapshot.summary()).containsEntry("wap.id", "987654321"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(wap1Snapshot.parentId()).isEqualTo(firstSnapshotId); + assertThat(wap2Snapshot.parentId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // load current snapshot parentSnapshot = base.currentSnapshot(); @@ -380,24 +319,15 @@ public void testWithTwoPhaseCherryPicking() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals( - "Current snapshot should be set to one after wap snapshot", - parentSnapshot.snapshotId() + 1, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should contain manifests for both files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Parent snapshot id should change to latest snapshot before commit", - parentSnapshot.snapshotId(), - base.currentSnapshot().parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId() + 1); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.currentSnapshot().parentId()) + .as("Parent snapshot id should change to latest snapshot before commit") + .isEqualTo(parentSnapshot.snapshotId()); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); // load current snapshot parentSnapshot = base.currentSnapshot(); @@ -407,27 +337,19 @@ public void testWithTwoPhaseCherryPicking() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals( - "Current snapshot should be set to one after wap snapshot", - parentSnapshot.snapshotId() + 1 /* one fast-forwarded snapshot */ + 1, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should contain manifests for both files", - 3, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Parent snapshot id should change to latest snapshot before commit", - parentSnapshot.snapshotId(), - base.currentSnapshot().parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 3, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()) + .isEqualTo(parentSnapshot.snapshotId() + 1 /* one fast-forwarded snapshot */ + 1); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(3); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.currentSnapshot().parentId()) + .as("Parent snapshot id should change to latest snapshot before commit") + .isEqualTo(parentSnapshot.snapshotId()); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(3); } - @Test + @TestTemplate public void testWithCommitsBetweenCherryPicking() { table.newAppend().appendFile(FILE_A).commit(); TableMetadata base = readMetadata(); @@ -446,25 +368,15 @@ public void testWithCommitsBetweenCherryPicking() { Snapshot wap1Snapshot = base.snapshots().get(1); Snapshot wap2Snapshot = base.snapshots().get(2); - Assert.assertEquals("Should have three snapshots", 3, base.snapshots().size()); - Assert.assertEquals( - "Should have first wap id in summary", "123456789", wap1Snapshot.summary().get("wap.id")); - Assert.assertEquals( - "Should have second wap id in summary", "987654321", wap2Snapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Parent snapshot id should be same for first WAP snapshot", - firstSnapshotId, - wap1Snapshot.parentId().longValue()); - Assert.assertEquals( - "Parent snapshot id should be same for second WAP snapshot", - firstSnapshotId, - wap2Snapshot.parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(3); + assertThat(wap1Snapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(wap2Snapshot.summary()).containsEntry("wap.id", "987654321"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(wap1Snapshot.parentId()).isEqualTo(firstSnapshotId); + assertThat(wap2Snapshot.parentId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // load current snapshot parentSnapshot = base.currentSnapshot(); @@ -473,17 +385,12 @@ public void testWithCommitsBetweenCherryPicking() { table.newAppend().appendFile(FILE_D).commit(); base = readMetadata(); - Assert.assertEquals("Should have four snapshots", 4, base.snapshots().size()); - Assert.assertEquals( - "Current snapshot should carry over the parent snapshot", - parentSnapshot.snapshotId(), - base.currentSnapshot().parentId().longValue()); - Assert.assertEquals( - "Should contain manifests for two files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(4); + assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId()); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); // load current snapshot parentSnapshot = base.currentSnapshot(); @@ -493,25 +400,14 @@ public void testWithCommitsBetweenCherryPicking() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals("Should have five snapshots", 5, base.snapshots().size()); - Assert.assertEquals( - "Current snapshot should be set to one after wap snapshot", - parentSnapshot.snapshotId() + 1, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should contain manifests for three files", - 3, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Parent snapshot id should point to same snapshot", - parentSnapshot.snapshotId(), - base.currentSnapshot().parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 3, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(5); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId() + 1); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(3); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId()); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(3); // load current snapshot parentSnapshot = base.currentSnapshot(); @@ -521,28 +417,17 @@ public void testWithCommitsBetweenCherryPicking() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals("Should have all the snapshots", 6, base.snapshots().size()); - Assert.assertEquals( - "Current snapshot should be set to one after wap snapshot", - parentSnapshot.snapshotId() + 1, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should contain manifests for four files", - 4, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Parent snapshot id should point to same snapshot", - parentSnapshot.snapshotId(), - base.currentSnapshot().parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 4, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(6); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId() + 1); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(4); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId()); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(4); } - @Test + @TestTemplate public void testWithCherryPickingWithCommitRetry() { table.newAppend().appendFile(FILE_A).commit(); @@ -559,19 +444,13 @@ public void testWithCherryPickingWithCommitRetry() { // pick the snapshot that's staged but not committed Snapshot wap1Snapshot = base.snapshots().get(1); - Assert.assertEquals("Should have two snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should have first wap id in summary", "123456789", wap1Snapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Parent snapshot id should be same for first WAP snapshot", - firstSnapshotId, - wap1Snapshot.parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(2); + assertThat(wap1Snapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(wap1Snapshot.parentId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // load current snapshot base = readMetadata(); @@ -583,27 +462,16 @@ public void testWithCherryPickingWithCommitRetry() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals( - "Current snapshot should be set to one after wap snapshot", - parentSnapshot.snapshotId() + 1, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Should contain manifests for both files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should not contain redundant append due to retry", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Parent snapshot id should change to latest snapshot before commit", - parentSnapshot.snapshotId(), - base.currentSnapshot().parentId().longValue()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(parentSnapshot.snapshotId() + 1); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.currentSnapshot().parentId()).isEqualTo(parentSnapshot.snapshotId()); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); } - @Test + @TestTemplate public void testCherrypickingAncestor() { table.newAppend().appendFile(FILE_A).commit(); @@ -617,15 +485,12 @@ public void testCherrypickingAncestor() { // pick the snapshot that's staged but not committed Snapshot wapSnapshot = base.snapshots().get(1); - Assert.assertEquals("Should have both snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should have first wap id in summary", "123456789", wapSnapshot.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(2); + assertThat(wapSnapshot.summary()).containsEntry("wap.id", "123456789"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // cherry-pick snapshot table.manageSnapshots().cherrypick(wapSnapshot.snapshotId()).commit(); @@ -634,30 +499,22 @@ public void testCherrypickingAncestor() { // check if the effective current snapshot is set to the new snapshot created // as a result of the cherry-pick operation - Assert.assertEquals( - "Current snapshot should be fast-forwarded to wap snapshot", - wapSnapshot.snapshotId(), - base.currentSnapshot().snapshotId()); - Assert.assertEquals("Should have two snapshots", 2, base.snapshots().size()); - Assert.assertEquals( - "Should contain manifests for both files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); - - Assertions.assertThatThrownBy( + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(wapPublishedId); + assertThat(base.snapshots()).hasSize(2); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); + + assertThatThrownBy( // duplicate cherry-pick snapshot () -> table.manageSnapshots().cherrypick(firstSnapshotId).commit()) .isInstanceOf(CherrypickAncestorCommitException.class) .hasMessage("Cannot cherrypick snapshot 1: already an ancestor"); } - @Test + @TestTemplate public void testDuplicateCherrypick() { table.newAppend().appendFile(FILE_A).commit(); TableMetadata base = readMetadata(); @@ -673,39 +530,26 @@ public void testDuplicateCherrypick() { Snapshot wapSnapshot1 = base.snapshots().get(1); Snapshot wapSnapshot2 = base.snapshots().get(2); - Assert.assertEquals("Should have both snapshots", 3, base.snapshots().size()); - Assert.assertEquals( - "Should have wap id in first wap snapshot summary", - "123456789", - wapSnapshot1.summary().get("wap.id")); - Assert.assertEquals( - "Should have wap id in second wap snapshot summary", - "123456789", - wapSnapshot2.summary().get("wap.id")); - Assert.assertEquals( - "Current snapshot should be first commit's snapshot", - firstSnapshotId, - base.currentSnapshot().snapshotId()); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 1, base.snapshotLog().size()); + assertThat(base.snapshots()).hasSize(3); + assertThat(wapSnapshot1.summary()).containsEntry("wap.id", "123456789"); + assertThat(wapSnapshot2.summary()).containsEntry("wap.id", "123456789"); + assertThat(base.currentSnapshot().snapshotId()).isEqualTo(firstSnapshotId); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(1); // cherry-pick snapshot table.manageSnapshots().cherrypick(wapSnapshot1.snapshotId()).commit(); base = readMetadata(); - Assert.assertEquals("Should have three snapshots", 3, base.snapshots().size()); - Assert.assertEquals( - "Should contain manifests for both files", - 2, - base.currentSnapshot().allManifests(table.io()).size()); - Assert.assertEquals( - "Should contain append from last commit", - 1, - Iterables.size(base.currentSnapshot().addedDataFiles(table.io()))); - Assert.assertEquals( - "Snapshot log should indicate number of snapshots committed", 2, base.snapshotLog().size()); - - Assertions.assertThatThrownBy( + assertThat(base.snapshots()).hasSize(3); + assertThat(base.currentSnapshot().allManifests(table.io())).hasSize(2); + assertThat(base.currentSnapshot().addedDataFiles(table.io())).hasSize(1); + assertThat(base.snapshotLog()) + .as("Snapshot log should indicate number of snapshots committed") + .hasSize(2); + + assertThatThrownBy( // duplicate cherry-pick snapshot () -> table.manageSnapshots().cherrypick(wapSnapshot2.snapshotId()).commit()) .isInstanceOf(DuplicateWAPCommitException.class) @@ -713,7 +557,7 @@ public void testDuplicateCherrypick() { "Duplicate request to cherry pick wap id that was published already: 123456789"); } - @Test + @TestTemplate public void testNonWapCherrypick() { table.newAppend().appendFile(FILE_A).commit(); TableMetadata base = readMetadata(); @@ -727,41 +571,29 @@ public void testNonWapCherrypick() { base = readMetadata(); long thirdSnapshotId = base.currentSnapshot().snapshotId(); - Assert.assertEquals( - "Should be pointing to third snapshot", - thirdSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(thirdSnapshotId); // NOOP commit table.manageSnapshots().commit(); - Assert.assertEquals( - "Should still be pointing to third snapshot", - thirdSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(thirdSnapshotId); // Rollback to second snapshot table.manageSnapshots().rollbackTo(secondSnapshotId).commit(); - Assert.assertEquals( - "Should be pointing to second snapshot", - secondSnapshotId, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(secondSnapshotId); // Cherrypick down to third table.manageSnapshots().cherrypick(thirdSnapshotId).commit(); - Assert.assertEquals( - "Should be re-using wap snapshot after cherrypick", - 3, - table.currentSnapshot().snapshotId()); + assertThat(table.currentSnapshot().snapshotId()).isEqualTo(3); // try double cherrypicking of the third snapshot - Assertions.assertThatThrownBy( + assertThatThrownBy( // double cherrypicking of second snapshot () -> table.manageSnapshots().cherrypick(thirdSnapshotId).commit()) .isInstanceOf(CherrypickAncestorCommitException.class) .hasMessage("Cannot cherrypick snapshot 3: already an ancestor"); // try cherrypicking an ancestor - Assertions.assertThatThrownBy( + assertThatThrownBy( // double cherrypicking of second snapshot () -> table.manageSnapshots().cherrypick(firstSnapshotId).commit()) .isInstanceOf(CherrypickAncestorCommitException.class) diff --git a/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java b/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java index 1aae6483337f..2d2e78a81557 100644 --- a/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java +++ b/core/src/test/java/org/apache/iceberg/actions/TestCommitService.java @@ -19,31 +19,36 @@ package org.apache.iceberg.actions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; +import java.util.Arrays; +import java.util.List; import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; import java.util.stream.IntStream; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableTestBase; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.Tasks; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Test; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -public class TestCommitService extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestCommitService extends TestBase { - public TestCommitService() { - super(1); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1); } - @Test + @TestTemplate public void testCommittedResultsCorrectly() { CustomCommitService commitService = new CustomCommitService(table, 5, 10000); commitService.start(); @@ -55,10 +60,10 @@ public void testCommittedResultsCorrectly() { Set expected = Sets.newHashSet(IntStream.range(0, 100).iterator()); Set actual = Sets.newHashSet(commitService.results()); - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); } - @Test + @TestTemplate public void testAbortFileGroupsAfterTimeout() { CustomCommitService commitService = new CustomCommitService(table, 5, 200); commitService.start(); @@ -90,7 +95,7 @@ public void testAbortFileGroupsAfterTimeout() { // simulate timeout on the main thread, which then tries to abort file groups [5-7]. // This tests the race conditions, as the committerService is also trying to commit groups // [5-7]. - Assertions.assertThatThrownBy(commitService::close) + assertThatThrownBy(commitService::close) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Timeout occurred when waiting for commits"); @@ -102,14 +107,12 @@ public void testAbortFileGroupsAfterTimeout() { .untilAsserted(() -> assertThat(commitService.completedRewritesAllCommitted()).isTrue()); if (commitService.aborted.isEmpty()) { // All file groups are committed - Assertions.assertThat(commitService.results()) - .isEqualTo(ImmutableList.of(0, 1, 2, 3, 4, 5, 6, 7)); + assertThat(commitService.results()).containsExactly(0, 1, 2, 3, 4, 5, 6, 7); } else { // File groups [5-7] are aborted - Assertions.assertThat(commitService.results()) - .doesNotContainAnyElementsOf(commitService.aborted); - Assertions.assertThat(commitService.results()).isEqualTo(ImmutableList.of(0, 1, 2, 3, 4)); - Assertions.assertThat(commitService.aborted).isEqualTo(ImmutableSet.of(5, 6, 7)); + assertThat(commitService.results()).doesNotContainAnyElementsOf(commitService.aborted); + assertThat(commitService.results()).containsExactly(0, 1, 2, 3, 4); + assertThat(commitService.aborted).containsExactly(5, 6, 7); } } diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java index c33bbc6f6d1e..a9a23d587ac9 100644 --- a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java +++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java @@ -20,33 +20,31 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Set; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.MockFileScanTask; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestSizeBasedRewriter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestSizeBasedRewriter extends TestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestSizeBasedRewriter(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testSplitSizeLowerBound() { SizeBasedDataFileRewriterImpl rewriter = new SizeBasedDataFileRewriterImpl(table); diff --git a/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java index af35e27f5b22..03108376eb4b 100644 --- a/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java +++ b/core/src/test/java/org/apache/iceberg/avro/AvroTestHelpers.java @@ -19,6 +19,7 @@ package org.apache.iceberg.avro; import static org.apache.iceberg.avro.AvroSchemaUtil.toOption; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Arrays; import java.util.List; @@ -28,7 +29,6 @@ import org.apache.avro.generic.GenericData.Record; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; class AvroTestHelpers { @@ -81,7 +81,7 @@ static void assertEquals(Types.StructType struct, Record expected, Record actual static void assertEquals(Types.ListType list, List expected, List actual) { Type elementType = list.elementType(); - Assertions.assertThat(actual).as("List size should match").hasSameSizeAs(expected); + assertThat(actual).as("List size should match").hasSameSizeAs(expected); for (int i = 0; i < expected.size(); i += 1) { Object expectedValue = expected.get(i); @@ -94,7 +94,7 @@ static void assertEquals(Types.ListType list, List expected, List actual) static void assertEquals(Types.MapType map, Map expected, Map actual) { Type valueType = map.valueType(); - Assertions.assertThat(actual).as("Map keys should match").hasSameSizeAs(expected); + assertThat(actual).as("Map keys should match").hasSameSizeAs(expected); for (Object expectedKey : expected.keySet()) { Object expectedValue = expected.get(expectedKey); @@ -123,25 +123,21 @@ private static void assertEquals(Type type, Object expected, Object actual) { case FIXED: case BINARY: case DECIMAL: - Assertions.assertThat(actual) - .as("Primitive value should be equal to expected") - .isEqualTo(expected); + assertThat(actual).as("Primitive value should be equal to expected").isEqualTo(expected); break; case STRUCT: - Assertions.assertThat(expected) - .as("Expected should be a Record") - .isInstanceOf(Record.class); - Assertions.assertThat(actual).as("Actual should be a Record").isInstanceOf(Record.class); + assertThat(expected).as("Expected should be a Record").isInstanceOf(Record.class); + assertThat(actual).as("Actual should be a Record").isInstanceOf(Record.class); assertEquals(type.asStructType(), (Record) expected, (Record) actual); break; case LIST: - Assertions.assertThat(expected).as("Expected should be a List").isInstanceOf(List.class); - Assertions.assertThat(actual).as("Actual should be a List").isInstanceOf(List.class); + assertThat(expected).as("Expected should be a List").isInstanceOf(List.class); + assertThat(actual).as("Actual should be a List").isInstanceOf(List.class); assertEquals(type.asListType(), (List) expected, (List) actual); break; case MAP: - Assertions.assertThat(expected).as("Expected should be a Map").isInstanceOf(Map.class); - Assertions.assertThat(actual).as("Actual should be a Map").isInstanceOf(Map.class); + assertThat(expected).as("Expected should be a Map").isInstanceOf(Map.class); + assertThat(actual).as("Actual should be a Map").isInstanceOf(Map.class); assertEquals(type.asMapType(), (Map) expected, (Map) actual); break; default: diff --git a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java index 9c8ea8453df5..686456b3b16f 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.avro; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.avro.Schema; import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.MappedFields; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestNameMappingWithAvroSchema { @Test @@ -132,9 +133,9 @@ public void testNameMappingWithAvroSchema() { MappedField.of(13, "string"), MappedField.of(14, "int"))))), MappedField.of(21, "timezone"), MappedField.of(22, "bitmap")))); - Assert.assertEquals( - expected, - AvroWithPartnerByStructureVisitor.visit( - icebergSchema.asStruct(), schema, nameMappingWithAvroSchema)); + assertThat( + AvroWithPartnerByStructureVisitor.visit( + icebergSchema.asStruct(), schema, nameMappingWithAvroSchema)) + .isEqualTo(expected); } } diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index a954cf760baa..af910f9fdd45 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -18,10 +18,14 @@ */ package org.apache.iceberg.encryption; +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.channels.FileChannel; +import java.nio.file.Path; import java.nio.file.StandardOpenOption; import java.util.Arrays; import java.util.Random; @@ -29,15 +33,12 @@ import org.apache.iceberg.Files; import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.io.SeekableInputStream; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestGcmStreams { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testEmptyFile() throws IOException { @@ -48,7 +49,7 @@ public void testEmptyFile() throws IOException { random.nextBytes(aadPrefix); byte[] readBytes = new byte[1]; - File testFile = temp.newFile(); + File testFile = File.createTempFile("test", null, temp.toFile()); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); @@ -56,20 +57,20 @@ public void testEmptyFile() throws IOException { encryptedStream.close(); AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); - Assert.assertEquals("File size", 0, decryptedFile.getLength()); + assertThat(decryptedFile.getLength()).isEqualTo(0); try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { - Assert.assertEquals("Read empty stream", -1, decryptedStream.read(readBytes)); + assertThat(decryptedStream.read(readBytes)).as("Read empty stream").isEqualTo(-1); } // check that the AAD is still verified, even for an empty file byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length); badAAD[1] -= 1; // modify the AAD slightly AesGcmInputFile badAADFile = new AesGcmInputFile(Files.localInput(testFile), key, badAAD); - Assert.assertEquals("File size", 0, badAADFile.getLength()); + assertThat(badAADFile.getLength()).isEqualTo(0); try (SeekableInputStream decryptedStream = badAADFile.newStream()) { - Assertions.assertThatThrownBy(() -> decryptedStream.read(readBytes)) + assertThatThrownBy(() -> decryptedStream.read(readBytes)) .isInstanceOf(RuntimeException.class) .hasCauseInstanceOf(AEADBadTagException.class) .hasMessageContaining("GCM tag check failed"); @@ -86,7 +87,7 @@ public void testAADValidation() throws IOException { byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block random.nextBytes(content); - File testFile = temp.newFile(); + File testFile = File.createTempFile("test", null, temp.toFile()); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); @@ -96,27 +97,24 @@ public void testAADValidation() throws IOException { // verify the data can be read correctly with the right AAD AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); - Assert.assertEquals("File size", content.length, decryptedFile.getLength()); + assertThat(decryptedFile.getLength()).isEqualTo(content.length); try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; int bytesRead = decryptedStream.read(readContent); - Assert.assertEquals("Bytes read should match bytes written", content.length, bytesRead); - Assert.assertEquals( - "Content should match", - ByteBuffer.wrap(content), - ByteBuffer.wrap(readContent, 0, bytesRead)); + assertThat(bytesRead).as("Bytes read should match bytes written").isEqualTo(content.length); + assertThat(ByteBuffer.wrap(readContent, 0, bytesRead)).isEqualTo(ByteBuffer.wrap(content)); } // test with the wrong AAD byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length); badAAD[1] -= 1; // modify the AAD slightly AesGcmInputFile badAADFile = new AesGcmInputFile(Files.localInput(testFile), key, badAAD); - Assert.assertEquals("File size", content.length, badAADFile.getLength()); + assertThat(badAADFile.getLength()).isEqualTo(content.length); try (SeekableInputStream decryptedStream = badAADFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; - Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + assertThatThrownBy(() -> decryptedStream.read(readContent)) .isInstanceOf(RuntimeException.class) .hasCauseInstanceOf(AEADBadTagException.class) .hasMessageContaining("GCM tag check failed"); @@ -132,7 +130,7 @@ public void testAADValidation() throws IOException { // read with the correct AAD and verify the tag check fails try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; - Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + assertThatThrownBy(() -> decryptedStream.read(readContent)) .isInstanceOf(RuntimeException.class) .hasCauseInstanceOf(AEADBadTagException.class) .hasMessageContaining("GCM tag check failed"); @@ -149,7 +147,7 @@ public void testCorruptNonce() throws IOException { byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block random.nextBytes(content); - File testFile = temp.newFile(); + File testFile = File.createTempFile("test", null, temp.toFile()); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); @@ -159,16 +157,13 @@ public void testCorruptNonce() throws IOException { // verify the data can be read correctly with the right AAD AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); - Assert.assertEquals("File size", content.length, decryptedFile.getLength()); + assertThat(decryptedFile.getLength()).isEqualTo(content.length); try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; int bytesRead = decryptedStream.read(readContent); - Assert.assertEquals("Bytes read should match bytes written", content.length, bytesRead); - Assert.assertEquals( - "Content should match", - ByteBuffer.wrap(content), - ByteBuffer.wrap(readContent, 0, bytesRead)); + assertThat(bytesRead).as("Bytes read should match bytes written").isEqualTo(content.length); + assertThat(ByteBuffer.wrap(readContent, 0, bytesRead)).isEqualTo(ByteBuffer.wrap(content)); } // replace the first block's nonce @@ -181,7 +176,7 @@ public void testCorruptNonce() throws IOException { // read with the correct AAD and verify the read fails try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; - Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + assertThatThrownBy(() -> decryptedStream.read(readContent)) .isInstanceOf(RuntimeException.class) .hasCauseInstanceOf(AEADBadTagException.class) .hasMessageContaining("GCM tag check failed"); @@ -198,7 +193,7 @@ public void testCorruptCiphertext() throws IOException { byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block random.nextBytes(content); - File testFile = temp.newFile(); + File testFile = File.createTempFile("test", null, temp.toFile()); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); @@ -208,16 +203,13 @@ public void testCorruptCiphertext() throws IOException { // verify the data can be read correctly with the right AAD AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); - Assert.assertEquals("File size", content.length, decryptedFile.getLength()); + assertThat(decryptedFile.getLength()).isEqualTo(content.length); try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; int bytesRead = decryptedStream.read(readContent); - Assert.assertEquals("Bytes read should match bytes written", content.length, bytesRead); - Assert.assertEquals( - "Content should match", - ByteBuffer.wrap(content), - ByteBuffer.wrap(readContent, 0, bytesRead)); + assertThat(bytesRead).as("Bytes read should match bytes written").isEqualTo(content.length); + assertThat(ByteBuffer.wrap(readContent, 0, bytesRead)).isEqualTo(ByteBuffer.wrap(content)); } // replace part of the first block's content @@ -230,7 +222,7 @@ public void testCorruptCiphertext() throws IOException { // read with the correct AAD and verify the read fails try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; - Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + assertThatThrownBy(() -> decryptedStream.read(readContent)) .isInstanceOf(RuntimeException.class) .hasCauseInstanceOf(AEADBadTagException.class) .hasMessageContaining("GCM tag check failed"); @@ -260,7 +252,7 @@ public void testRandomWriteRead() throws IOException { byte[] key = new byte[keyLength]; random.nextBytes(key); random.nextBytes(aadPrefix); - File testFile = temp.newFile(); + File testFile = File.createTempFile("test", null, temp.toFile()); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); @@ -277,17 +269,19 @@ public void testRandomWriteRead() throws IOException { } encryptedStream.write(testFileContents, offset, chunkLen); offset += chunkLen; - Assert.assertEquals("Position", offset, encryptedStream.getPos()); + assertThat(encryptedStream.getPos()).isEqualTo(offset); left -= chunkLen; } encryptedStream.close(); - Assert.assertEquals("Final position in closed stream", offset, encryptedStream.getPos()); + assertThat(encryptedStream.getPos()) + .as("Final position in closed stream") + .isEqualTo(offset); AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); SeekableInputStream decryptedStream = decryptedFile.newStream(); - Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); + assertThat(decryptedFile.getLength()).isEqualTo(testFileSize); byte[] chunk = new byte[testFileSize]; @@ -303,26 +297,26 @@ public void testRandomWriteRead() throws IOException { decryptedStream.seek(pos); int len = decryptedStream.read(chunk, 0, chunkLen); - Assert.assertEquals("Read length", len, chunkLen); + assertThat(chunkLen).isEqualTo(len); long pos2 = decryptedStream.getPos(); - Assert.assertEquals("Position", pos + len, pos2); + assertThat(pos2).isEqualTo(pos + len); ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, pos, chunkLen); - Assert.assertEquals("Read contents", bb1, bb2); + assertThat(bb2).isEqualTo(bb1); // Test skip long toSkip = random.nextInt(testFileSize); long skipped = decryptedStream.skip(toSkip); if (pos2 + toSkip < testFileSize) { - Assert.assertEquals("Skipped", toSkip, skipped); + assertThat(skipped).isEqualTo(toSkip); } else { - Assert.assertEquals("Skipped", (testFileSize - pos2), skipped); + assertThat(skipped).isEqualTo(testFileSize - pos2); } int pos3 = (int) decryptedStream.getPos(); - Assert.assertEquals("Position", pos2 + skipped, pos3); + assertThat(pos3).isEqualTo(pos2 + skipped); chunkLen = random.nextInt(testFileSize); left = testFileSize - pos3; @@ -334,7 +328,7 @@ public void testRandomWriteRead() throws IOException { decryptedStream.read(chunk, 0, chunkLen); bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); bb2 = ByteBuffer.wrap(testFileContents, pos3, chunkLen); - Assert.assertEquals("Read contents", bb1, bb2); + assertThat(bb2).isEqualTo(bb1); } decryptedStream.close(); @@ -357,7 +351,7 @@ public void testAlignedWriteRead() throws IOException { byte[] aadPrefix = new byte[16]; random.nextBytes(aadPrefix); - File testFile = temp.newFile(); + File testFile = File.createTempFile("test", null, temp.toFile()); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); @@ -374,17 +368,17 @@ public void testAlignedWriteRead() throws IOException { encryptedStream.write(testFileContents, offset, chunkLen); offset += chunkLen; - Assert.assertEquals("Position", offset, encryptedStream.getPos()); + assertThat(encryptedStream.getPos()).isEqualTo(offset); left -= chunkLen; } encryptedStream.close(); - Assert.assertEquals("Final position in closed stream", offset, encryptedStream.getPos()); + assertThat(encryptedStream.getPos()).as("Final position in closed stream").isEqualTo(offset); AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); SeekableInputStream decryptedStream = decryptedFile.newStream(); - Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); + assertThat(decryptedFile.getLength()).isEqualTo(testFileSize); offset = 0; chunkLen = Ciphers.PLAIN_BLOCK_SIZE; @@ -399,12 +393,12 @@ public void testAlignedWriteRead() throws IOException { decryptedStream.seek(offset); int len = decryptedStream.read(chunk, 0, chunkLen); - Assert.assertEquals("Read length", len, chunkLen); - Assert.assertEquals("Position", offset + len, decryptedStream.getPos()); + assertThat(chunkLen).isEqualTo(len); + assertThat(decryptedStream.getPos()).isEqualTo(offset + len); ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, offset, chunkLen); - Assert.assertEquals("Read contents", bb1, bb2); + assertThat(bb2).isEqualTo(bb1); offset += len; left = testFileSize - offset; diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java b/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java index 889506cb93e4..d73f52da4661 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.encryption; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStandardKeyMetadataParser { @@ -35,14 +36,14 @@ public void testParser() { ByteBuffer serialized = metadata.buffer(); StandardKeyMetadata parsedMetadata = StandardKeyMetadata.parse(serialized); - Assert.assertEquals(parsedMetadata.encryptionKey(), encryptionKey); - Assert.assertEquals(parsedMetadata.aadPrefix(), aadPrefix); + assertThat(encryptionKey).isEqualTo(parsedMetadata.encryptionKey()); + assertThat(aadPrefix).isEqualTo(parsedMetadata.aadPrefix()); } @Test public void testUnsupportedVersion() { ByteBuffer badBuffer = ByteBuffer.wrap(new byte[] {0x02}); - Assertions.assertThatThrownBy(() -> StandardKeyMetadata.parse(badBuffer)) + assertThatThrownBy(() -> StandardKeyMetadata.parse(badBuffer)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot resolve schema for version: 2"); } diff --git a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java index f7c81ae879c9..ceffeb3749da 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java +++ b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java @@ -18,36 +18,35 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Arrays; +import java.util.List; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestOutputFileFactory extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestOutputFileFactory extends TestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } private static final int PARTITION_ID = 1; private static final int TASK_ID = 100; - public TestOutputFileFactory(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testOutputFileFactoryWithCustomFormat() { table.updateProperties().defaultFormat(FileFormat.ORC).commit(); @@ -55,11 +54,10 @@ public void testOutputFileFactoryWithCustomFormat() { OutputFileFactory.builderFor(table, PARTITION_ID, TASK_ID).format(FileFormat.AVRO).build(); String location = fileFactory.newOutputFile().encryptingOutputFile().location(); - Assert.assertEquals( - "File format should be correct", FileFormat.AVRO, FileFormat.fromFileName(location)); + assertThat(FileFormat.fromFileName(location)).isEqualTo(FileFormat.AVRO); } - @Test + @TestTemplate public void testOutputFileFactoryWithMultipleSpecs() { OutputFileFactory fileFactory = OutputFileFactory.builderFor(table, PARTITION_ID, TASK_ID).operationId("append").build(); @@ -67,18 +65,17 @@ public void testOutputFileFactoryWithMultipleSpecs() { EncryptedOutputFile unpartitionedFile = fileFactory.newOutputFile(PartitionSpec.unpartitioned(), null); String unpartitionedFileLocation = unpartitionedFile.encryptingOutputFile().location(); - Assert.assertTrue(unpartitionedFileLocation.endsWith("data/00001-100-append-00001.parquet")); + assertThat(unpartitionedFileLocation).endsWith("data/00001-100-append-00001.parquet"); Record record = GenericRecord.create(table.schema()).copy(ImmutableMap.of("data", "aaa")); PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); partitionKey.partition(record); EncryptedOutputFile partitionedFile = fileFactory.newOutputFile(table.spec(), partitionKey); String partitionedFileLocation = partitionedFile.encryptingOutputFile().location(); - Assert.assertTrue( - partitionedFileLocation.endsWith("data_bucket=7/00001-100-append-00002.parquet")); + assertThat(partitionedFileLocation).endsWith("data_bucket=7/00001-100-append-00002.parquet"); } - @Test + @TestTemplate public void testWithCustomSuffix() { OutputFileFactory fileFactory = OutputFileFactory.builderFor(table, PARTITION_ID, TASK_ID) @@ -89,15 +86,14 @@ public void testWithCustomSuffix() { EncryptedOutputFile unpartitionedFile = fileFactory.newOutputFile(PartitionSpec.unpartitioned(), null); String unpartitionedFileLocation = unpartitionedFile.encryptingOutputFile().location(); - Assertions.assertThat(unpartitionedFileLocation) - .endsWith("data/00001-100-append-00001-suffix.parquet"); + assertThat(unpartitionedFileLocation).endsWith("data/00001-100-append-00001-suffix.parquet"); Record record = GenericRecord.create(table.schema()).copy(ImmutableMap.of("data", "aaa")); PartitionKey partitionKey = new PartitionKey(table.spec(), table.schema()); partitionKey.partition(record); EncryptedOutputFile partitionedFile = fileFactory.newOutputFile(table.spec(), partitionKey); String partitionedFileLocation = partitionedFile.encryptingOutputFile().location(); - Assertions.assertThat(partitionedFileLocation) + assertThat(partitionedFileLocation) .endsWith("data_bucket=7/00001-100-append-00002-suffix.parquet"); } } diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java index 6b59095225d8..b9ae9dc273f2 100644 --- a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java +++ b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java @@ -19,28 +19,27 @@ package org.apache.iceberg.mapping; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestMappingUpdates extends TableTestBase { - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMappingUpdates extends TestBase { + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestMappingUpdates(int formatVersion) { - super(formatVersion); - } - - @Test + @TestTemplate public void testAddColumnMappingUpdate() { NameMapping mapping = MappingUtil.create(table.schema()); table @@ -48,22 +47,21 @@ public void testAddColumnMappingUpdate() { .set(TableProperties.DEFAULT_NAME_MAPPING, NameMappingParser.toJson(mapping)) .commit(); - Assert.assertEquals( - MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")), - mapping.asMappedFields()); + assertThat(mapping.asMappedFields()) + .isEqualTo(MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data"))); table.updateSchema().addColumn("ts", Types.TimestampType.withZone()).commit(); NameMapping updated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), MappedField.of(2, "data"), MappedField.of(3, "ts")), - updated.asMappedFields()); + assertThat(updated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), MappedField.of(2, "data"), MappedField.of(3, "ts"))); } - @Test + @TestTemplate public void testAddNestedColumnMappingUpdate() { NameMapping mapping = MappingUtil.create(table.schema()); table @@ -71,9 +69,8 @@ public void testAddNestedColumnMappingUpdate() { .set(TableProperties.DEFAULT_NAME_MAPPING, NameMappingParser.toJson(mapping)) .commit(); - Assert.assertEquals( - MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")), - mapping.asMappedFields()); + assertThat(mapping.asMappedFields()) + .isEqualTo(MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data"))); table .updateSchema() @@ -86,32 +83,32 @@ public void testAddNestedColumnMappingUpdate() { NameMapping updated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), - MappedField.of(2, "data"), - MappedField.of( - 3, "point", MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y")))), - updated.asMappedFields()); + assertThat(updated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), + MappedField.of(2, "data"), + MappedField.of( + 3, "point", MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y"))))); table.updateSchema().addColumn("point", "z", Types.DoubleType.get()).commit(); NameMapping pointUpdated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), - MappedField.of(2, "data"), - MappedField.of( - 3, - "point", - MappedFields.of( - MappedField.of(4, "x"), MappedField.of(5, "y"), MappedField.of(6, "z")))), - pointUpdated.asMappedFields()); + assertThat(pointUpdated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), + MappedField.of(2, "data"), + MappedField.of( + 3, + "point", + MappedFields.of( + MappedField.of(4, "x"), MappedField.of(5, "y"), MappedField.of(6, "z"))))); } - @Test + @TestTemplate public void testRenameMappingUpdate() { NameMapping mapping = MappingUtil.create(table.schema()); table @@ -119,22 +116,21 @@ public void testRenameMappingUpdate() { .set(TableProperties.DEFAULT_NAME_MAPPING, NameMappingParser.toJson(mapping)) .commit(); - Assert.assertEquals( - MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")), - mapping.asMappedFields()); + assertThat(mapping.asMappedFields()) + .isEqualTo(MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data"))); table.updateSchema().renameColumn("id", "object_id").commit(); NameMapping updated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, ImmutableList.of("id", "object_id")), MappedField.of(2, "data")), - updated.asMappedFields()); + assertThat(updated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, ImmutableList.of("id", "object_id")), MappedField.of(2, "data"))); } - @Test + @TestTemplate public void testRenameNestedFieldMappingUpdate() { NameMapping mapping = MappingUtil.create(table.schema()); table @@ -153,33 +149,33 @@ public void testRenameNestedFieldMappingUpdate() { NameMapping updated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), - MappedField.of(2, "data"), - MappedField.of( - 3, "point", MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y")))), - updated.asMappedFields()); + assertThat(updated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), + MappedField.of(2, "data"), + MappedField.of( + 3, "point", MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y"))))); table.updateSchema().renameColumn("point.x", "X").renameColumn("point.y", "Y").commit(); NameMapping pointUpdated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), - MappedField.of(2, "data"), - MappedField.of( - 3, - "point", - MappedFields.of( - MappedField.of(4, ImmutableList.of("x", "X")), - MappedField.of(5, ImmutableList.of("y", "Y"))))), - pointUpdated.asMappedFields()); + assertThat(pointUpdated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), + MappedField.of(2, "data"), + MappedField.of( + 3, + "point", + MappedFields.of( + MappedField.of(4, ImmutableList.of("x", "X")), + MappedField.of(5, ImmutableList.of("y", "Y")))))); } - @Test + @TestTemplate public void testRenameComplexFieldMappingUpdate() { NameMapping mapping = MappingUtil.create(table.schema()); table @@ -198,27 +194,27 @@ public void testRenameComplexFieldMappingUpdate() { NameMapping updated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), - MappedField.of(2, "data"), - MappedField.of( - 3, "point", MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y")))), - updated.asMappedFields()); + assertThat(updated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), + MappedField.of(2, "data"), + MappedField.of( + 3, "point", MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y"))))); table.updateSchema().renameColumn("point", "p2").commit(); NameMapping pointUpdated = NameMappingParser.fromJson(table.properties().get(TableProperties.DEFAULT_NAME_MAPPING)); - Assert.assertEquals( - MappedFields.of( - MappedField.of(1, "id"), - MappedField.of(2, "data"), - MappedField.of( - 3, - ImmutableList.of("point", "p2"), - MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y")))), - pointUpdated.asMappedFields()); + assertThat(pointUpdated.asMappedFields()) + .isEqualTo( + MappedFields.of( + MappedField.of(1, "id"), + MappedField.of(2, "data"), + MappedField.of( + 3, + ImmutableList.of("point", "p2"), + MappedFields.of(MappedField.of(4, "x"), MappedField.of(5, "y"))))); } } diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java b/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java index bde99b4113ab..d30a93d50d49 100644 --- a/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java +++ b/core/src/test/java/org/apache/iceberg/mapping/TestNameMapping.java @@ -19,12 +19,12 @@ package org.apache.iceberg.mapping; 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 org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestNameMapping { @Test @@ -36,7 +36,7 @@ public void testFlatSchemaToMapping() { MappedFields expected = MappedFields.of(MappedField.of(1, "id"), MappedField.of(2, "data")); NameMapping mapping = MappingUtil.create(schema); - Assert.assertEquals(expected, mapping.asMappedFields()); + assertThat(mapping.asMappedFields()).isEqualTo(expected); } @Test @@ -62,7 +62,7 @@ public void testNestedStructSchemaToMapping() { MappedFields.of(MappedField.of(4, "latitude"), MappedField.of(5, "longitude")))); NameMapping mapping = MappingUtil.create(schema); - Assert.assertEquals(expected, mapping.asMappedFields()); + assertThat(mapping.asMappedFields()).isEqualTo(expected); } @Test @@ -84,7 +84,7 @@ public void testMapSchemaToMapping() { 3, "map", MappedFields.of(MappedField.of(4, "key"), MappedField.of(5, "value")))); NameMapping mapping = MappingUtil.create(schema); - Assert.assertEquals(expected, mapping.asMappedFields()); + assertThat(mapping.asMappedFields()).isEqualTo(expected); } @Test @@ -117,7 +117,7 @@ public void testComplexKeyMapSchemaToMapping() { MappedField.of(5, "value")))); NameMapping mapping = MappingUtil.create(schema); - Assert.assertEquals(expected, mapping.asMappedFields()); + assertThat(mapping.asMappedFields()).isEqualTo(expected); } @Test @@ -152,7 +152,7 @@ public void testComplexValueMapSchemaToMapping() { MappedFields.of(MappedField.of(6, "x"), MappedField.of(7, "y")))))); NameMapping mapping = MappingUtil.create(schema); - Assert.assertEquals(expected, mapping.asMappedFields()); + assertThat(mapping.asMappedFields()).isEqualTo(expected); } @Test @@ -170,13 +170,13 @@ public void testListSchemaToMapping() { MappedField.of(3, "list", MappedFields.of(MappedField.of(4, "element")))); NameMapping mapping = MappingUtil.create(schema); - Assert.assertEquals(expected, mapping.asMappedFields()); + assertThat(mapping.asMappedFields()).isEqualTo(expected); } @Test public void testFailsDuplicateId() { // the schema can be created because ID indexing is lazy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new Schema( required(1, "id", Types.LongType.get()), @@ -187,7 +187,7 @@ public void testFailsDuplicateId() { @Test public void testFailsDuplicateName() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new NameMapping(MappedFields.of(MappedField.of(1, "x"), MappedField.of(2, "x")))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Multiple entries with same key: x=2 and x=1"); @@ -227,17 +227,17 @@ public void testMappingFindById() { NameMapping mapping = MappingUtil.create(schema); - Assert.assertNull("Should not return a field mapping for a missing ID", mapping.find(100)); - Assert.assertEquals(MappedField.of(2, "data"), mapping.find(2)); - Assert.assertEquals(MappedField.of(6, "x"), mapping.find(6)); - Assert.assertEquals(MappedField.of(9, "element"), mapping.find(9)); - Assert.assertEquals(MappedField.of(11, "latitude"), mapping.find(11)); - Assert.assertEquals( - MappedField.of( - 10, - "location", - MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12, "longitude"))), - mapping.find(10)); + assertThat(mapping.find(100)).as("Should not return a field mapping for a missing ID").isNull(); + assertThat(mapping.find(2)).isEqualTo(MappedField.of(2, "data")); + assertThat(mapping.find(6)).isEqualTo(MappedField.of(6, "x")); + assertThat(mapping.find(9)).isEqualTo(MappedField.of(9, "element")); + assertThat(mapping.find(11)).isEqualTo(MappedField.of(11, "latitude")); + assertThat(mapping.find(10)) + .isEqualTo( + MappedField.of( + 10, + "location", + MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12, "longitude")))); } @Test @@ -266,20 +266,27 @@ public void testMappingFindByName() { NameMapping mapping = MappingUtil.create(schema); - Assert.assertNull( - "Should not return a field mapping for a nested name", mapping.find("element")); - Assert.assertNull("Should not return a field mapping for a nested name", mapping.find("x")); - Assert.assertNull("Should not return a field mapping for a nested name", mapping.find("key")); - Assert.assertNull("Should not return a field mapping for a nested name", mapping.find("value")); - Assert.assertEquals(MappedField.of(2, "data"), mapping.find("data")); - Assert.assertEquals(MappedField.of(6, "x"), mapping.find("map", "value", "x")); - Assert.assertEquals(MappedField.of(9, "element"), mapping.find("list", "element")); - Assert.assertEquals(MappedField.of(11, "latitude"), mapping.find("location", "latitude")); - Assert.assertEquals( - MappedField.of( - 10, - "location", - MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12, "longitude"))), - mapping.find("location")); + assertThat(mapping.find("element")) + .as("Should not return a field mapping for a nested name") + .isNull(); + assertThat(mapping.find("x")) + .as("Should not return a field mapping for a nested name") + .isNull(); + assertThat(mapping.find("key")) + .as("Should not return a field mapping for a nested name") + .isNull(); + assertThat(mapping.find("value")) + .as("Should not return a field mapping for a nested name") + .isNull(); + assertThat(mapping.find("data")).isEqualTo(MappedField.of(2, "data")); + assertThat(mapping.find("map", "value", "x")).isEqualTo(MappedField.of(6, "x")); + assertThat(mapping.find("list", "element")).isEqualTo(MappedField.of(9, "element")); + assertThat(mapping.find("location", "latitude")).isEqualTo(MappedField.of(11, "latitude")); + assertThat(mapping.find("location")) + .isEqualTo( + MappedField.of( + 10, + "location", + MappedFields.of(MappedField.of(11, "latitude"), MappedField.of(12, "longitude")))); } } diff --git a/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java b/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java index 724970bc09ed..d85a73dbd233 100644 --- a/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java +++ b/data/src/test/java/org/apache/iceberg/orc/TestOrcMetrics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.orc; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Map; @@ -26,6 +28,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.Metrics; import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Schema; import org.apache.iceberg.TestMetrics; import org.apache.iceberg.data.Record; @@ -37,29 +40,18 @@ 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.Type; -import org.junit.Assert; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.extension.ExtendWith; /** Test Metrics for ORC. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestOrcMetrics extends TestMetrics { static final ImmutableSet BINARY_TYPES = ImmutableSet.of(Type.TypeID.BINARY, Type.TypeID.FIXED, Type.TypeID.UUID); - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestOrcMetrics(int formatVersion) { - super(formatVersion); - } - @Override protected OutputFile createOutputFile() throws IOException { - File tmpFolder = temp.newFolder("orc"); + File tmpFolder = java.nio.file.Files.createTempDirectory(temp, "orc").toFile(); String filename = UUID.randomUUID().toString(); return Files.localOutput(new File(tmpFolder, FileFormat.ORC.addExtension(filename))); } @@ -119,12 +111,8 @@ private boolean isBinaryType(Type type) { protected void assertBounds( int fieldId, Type type, T lowerBound, T upperBound, Metrics metrics) { if (isBinaryType(type)) { - Assert.assertFalse( - "ORC binary field should not have lower bounds.", - metrics.lowerBounds().containsKey(fieldId)); - Assert.assertFalse( - "ORC binary field should not have upper bounds.", - metrics.upperBounds().containsKey(fieldId)); + assertThat(metrics.lowerBounds()).doesNotContainKey(fieldId); + assertThat(metrics.upperBounds()).doesNotContainKey(fieldId); return; } super.assertBounds(fieldId, type, lowerBound, upperBound, metrics); diff --git a/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java b/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java index f363e5d979d3..545cff9311d9 100644 --- a/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java +++ b/data/src/test/java/org/apache/iceberg/parquet/TestParquetMetrics.java @@ -26,6 +26,7 @@ import org.apache.iceberg.Files; import org.apache.iceberg.Metrics; import org.apache.iceberg.MetricsConfig; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Schema; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TestMetrics; @@ -37,24 +38,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.parquet.hadoop.ParquetFileReader; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.extension.ExtendWith; /** Test Metrics for Parquet. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestParquetMetrics extends TestMetrics { private static final Map SMALL_ROW_GROUP_CONFIG = ImmutableMap.of(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, "1600"); - @Parameterized.Parameters(name = "formatVersion = {0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestParquetMetrics(int formatVersion) { - super(formatVersion); - } - @Override public FileFormat fileFormat() { return FileFormat.PARQUET; @@ -62,7 +53,7 @@ public FileFormat fileFormat() { @Override protected OutputFile createOutputFile() throws IOException { - File tmpFolder = temp.newFolder("parquet"); + File tmpFolder = java.nio.file.Files.createTempDirectory(temp, "parquet").toFile(); String filename = UUID.randomUUID().toString(); return Files.localOutput(new File(tmpFolder, FileFormat.PARQUET.addExtension(filename))); } From fb305c7494c77e4b17c7f6a6169eb4a1173f42d5 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 28 Mar 2024 23:17:10 +0800 Subject: [PATCH 0181/1019] Build: disable link-check for all medium blog posts (#10057) --- site/docs/blogs.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 4e94c9e71b08..e7cda2f997a6 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -409,6 +409,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Jason Hughes](https://www.linkedin.com/in/jasonhhughes/) + ### [Migrating to Apache Iceberg at Adobe Experience Platform](https://medium.com/adobetech/migrating-to-apache-iceberg-at-adobe-experience-platform-40fa80f8b8de) **Date**: Jun 17th, 2021, **Company**: Adobe @@ -439,11 +440,13 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Susan Hall](https://thenewstack.io/author/susanhall/) + ### [A Short Introduction to Apache Iceberg](https://medium.com/expedia-group-tech/a-short-introduction-to-apache-iceberg-d34f628b6799) **Date**: Jan 26th, 2021, **Company**: Expedia **Author**: [Christine Mathiesen](https://www.linkedin.com/in/christine-mathiesen-676a98159/) + ### [Taking Query Optimizations to the Next Level with Iceberg](https://medium.com/adobetech/taking-query-optimizations-to-the-next-level-with-iceberg-6c968b83cd6f) **Date**: Jan 14th, 2021, **Company**: Adobe @@ -454,6 +457,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Zihan Li](https://www.linkedin.com/in/zihan-li-0a8a15149/), [Sudarshan Vasudevan](https://www.linkedin.com/in/suddu/), [Lei Sun](https://www.linkedin.com/in/lei-s-a93138a0/), [Shirshanka Das](https://www.linkedin.com/in/shirshankadas/) + ### [High Throughput Ingestion with Iceberg](https://medium.com/adobetech/high-throughput-ingestion-with-iceberg-ccf7877a413f) **Date**: Dec 22nd, 2020, **Company**: Adobe @@ -464,6 +468,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Anupom Syam](https://www.linkedin.com/in/anupom/) + ### [Iceberg at Adobe](https://medium.com/adobetech/iceberg-at-adobe-88cf1950e866) **Date**: Dec 3rd, 2020, **Company**: Adobe From 133f3da7b90d4461b2050f830b245525b78c607e Mon Sep 17 00:00:00 2001 From: Abid Mohammed Date: Fri, 29 Mar 2024 10:32:01 -0700 Subject: [PATCH 0182/1019] [core] fix #9997 - Handle s3a file upload interrupt which results in table metadata pointing to files that doesn't exist (#9998) Co-authored-by: Abid Mohammed --- .../apache/iceberg/hadoop/HadoopStreams.java | 11 +++ .../hadoop/fs/s3a/S3ABlockOutputStream.java | 67 +++++++++++++++++++ .../iceberg/hadoop/TestHadoopStreams.java | 53 +++++++++++++++ 3 files changed, 131 insertions(+) create mode 100644 core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java create mode 100644 core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java index 44023326a0cf..f9b43b684666 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java @@ -187,6 +187,17 @@ public void flush() throws IOException { public void close() throws IOException { stream.close(); this.closed = true; + // {@link org.apache.hadoop.fs.s3a.S3ABlockOutputStream#close()} calls {@link + // org.apache.hadoop.fs.s3a.S3ABlockOutputStream#putObject()} + // which doesn't throw an exception when interrupted. + // Need to check the interrupted flag to detect failed object upload + // and propagate the error up. + if (Thread.interrupted() + && "org.apache.hadoop.fs.s3a.S3ABlockOutputStream" + .equals(stream.getWrappedStream().getClass().getName())) { + throw new IOException( + "S3ABlockOutputStream failed to upload object after stream was closed"); + } } @SuppressWarnings("checkstyle:NoFinalizer") diff --git a/core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java b/core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.java new file mode 100644 index 000000000000..63bddf7b79c2 --- /dev/null +++ b/core/src/test/java/org/apache/hadoop/fs/s3a/S3ABlockOutputStream.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.hadoop.fs.s3a; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +/** mock class for testing hadoop s3a writer */ +public class S3ABlockOutputStream extends OutputStream { + public ExecutorService mockCloseService; + public Future mockUploadOnClose; + + public S3ABlockOutputStream() { + mockCloseService = Executors.newSingleThreadExecutor(); + } + + @Override + public void write(int b) throws IOException { + throw new IOException("mocked class, do not use"); + } + + @Override + public void close() throws IOException { + try { + mockUploadOnClose = + mockCloseService.submit( + () -> { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + // ignore + } + }); + mockUploadOnClose.get(); + } catch (CancellationException | InterruptedException e) { + // mock interrupt in S3ABlockOutputStream#putObject + Thread.currentThread().interrupt(); + } catch (Exception e) { + throw new RuntimeException(e); + } + super.close(); + } + + public void interruptClose() { + mockUploadOnClose.cancel(true); + } +} diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java new file mode 100644 index 000000000000..09b478e4a6c1 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.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.hadoop; + +import java.io.IOException; +import java.util.concurrent.Executors; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.s3a.S3ABlockOutputStream; +import org.apache.iceberg.io.PositionOutputStream; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +class TestHadoopStreams { + + @Test + void closeShouldThrowIOExceptionWhenInterrupted() throws Exception { + + S3ABlockOutputStream s3ABlockOutputStream = new S3ABlockOutputStream(); + FSDataOutputStream fsDataOutputStream = new FSDataOutputStream(s3ABlockOutputStream, null); + PositionOutputStream wrap = HadoopStreams.wrap(fsDataOutputStream); + // interrupt mock upload on close after a delay + Executors.newSingleThreadExecutor() + .execute( + () -> { + try { + Thread.sleep(1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + s3ABlockOutputStream.interruptClose(); + }); + + Assertions.assertThatThrownBy(wrap::close) + .isInstanceOf(IOException.class) + .hasMessage("S3ABlockOutputStream failed to upload object after stream was closed"); + } +} From ae1d4301fa0fb067e0f424fc97ff8669a3b194fb Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sat, 30 Mar 2024 09:07:45 +0100 Subject: [PATCH 0183/1019] Spark: Clarify schema behavior when working with branches (#10055) --- docs/docs/branching.md | 91 +++++++++++++++++++ docs/docs/spark-queries.md | 23 ++++- docs/docs/spark-writes.md | 4 + .../apache/iceberg/spark/sql/TestSelect.java | 59 ++++++++++++ 4 files changed, 176 insertions(+), 1 deletion(-) diff --git a/docs/docs/branching.md b/docs/docs/branching.md index e944e4eb3bc2..3379264d8a5f 100644 --- a/docs/docs/branching.md +++ b/docs/docs/branching.md @@ -113,3 +113,94 @@ Creating, querying and writing to branches and tags are supported in the Iceberg - [Spark Branch Writes](spark-writes.md#writing-to-branches) - [Flink Reads](flink-queries.md#reading-branches-and-tags-with-SQL) - [Flink Branch Writes](flink-writes.md#branch-writes) + + +## Schema selection with branches and tags + +It is important to understand that the schema tracked for a table is valid across all branches. +When working with branches, the table's schema is used as that's the schema being validated when writing data to a branch. +On the other hands, querying a tag uses the snapshot's schema, which is the schema id that snapshot pointed to when the snapshot was created. + +The below examples show which schema is being used when working with branches. + +Create a table and insert some data: + +```sql +CREATE TABLE db.table (id bigint, data string, col float); +INSERT INTO db.table values (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 3.0); +SELECT * FROM db.table; +1 a 1.0 +2 b 2.0 +3 c 3.0 +``` + +Create a branch `test_branch` that points to the current snapshot and read data from the branch: + +```sql +ALTER TABLE db.table CREATE BRANCH test_branch; + +SELECT * FROM db.table.branch_test_branch; +1 a 1.0 +2 b 2.0 +3 c 3.0 +``` + +Modify the table's schema by dropping the `col` column and adding a new column named `new_col`: + +```sql +ALTER TABLE db.table drop column float; + +ALTER TABLE db.table add column new_col date; + +INSERT INTO db.table values (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05')); + +SELECT * FROM db.table; +1 a NULL +2 b NULL +3 c NULL +4 d 2024-04-04 +5 e 2024-05-05 +``` + +Querying the head of the branch using one of the below statements will return data using the **table's schema**: + +```sql +SELECT * FROM db.table.branch_test_branch; +1 a NULL +2 b NULL +3 c NULL + +SELECT * FROM db.table VERSION AS OF 'test_branch'; +1 a NULL +2 b NULL +3 c NULL +``` + +Performing a time travel query using the snapshot id uses the **snapshot's schema**: + +```sql + +SELECT * FROM db.table.refs; +test_branch BRANCH 8109744798576441359 NULL NULL NULL +main BRANCH 6910357365743665710 NULL NULL NULL + + +SELECT * FROM db.table VERSION AS OF 8109744798576441359; +1 a 1.0 +2 b 2.0 +3 c 3.0 +``` + +When writing to the branch, the **table's schema** is used for validation: + +```sql + +INSERT INTO db.table.branch_test_branch values (6, 'e', date('2024-06-06')), (7, 'g', date('2024-07-07')); + +SELECT * FROM db.table.branch_test_branch; +6 e 2024-06-06 +7 g 2024-07-07 +1 a NULL +2 b NULL +3 c NULL +``` diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index 536c136d7e55..b606d849a692 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -82,7 +82,7 @@ The `VERSION AS OF` clause can contain a long snapshot ID or a string branch or If this is not desired, rename the tag or branch with a well-defined prefix such as 'snapshot-1'. -```sql +```sql -- time travel to October 26, 1986 at 01:21:00 SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00'; @@ -124,6 +124,27 @@ SELECT * FROM prod.db.table.`tag_historical-snapshot`; Note that the identifier with branch or tag may not be used in combination with `VERSION AS OF`. + +#### Schema selection in time travel queries + +The different time travel queries mentioned in the previous section can use either the snapshot's schema or the table's schema: + +```sql +-- time travel to October 26, 1986 at 01:21:00 -> uses the snapshot's schema +SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00'; + +-- time travel to snapshot with id 10963874102873L -> uses the snapshot's schema +SELECT * FROM prod.db.table VERSION AS OF 10963874102873; + +-- time travel to the head of audit-branch -> uses the table's schema +SELECT * FROM prod.db.table VERSION AS OF 'audit-branch'; +SELECT * FROM prod.db.table.`branch_audit-branch`; + +-- time travel to the snapshot referenced by the tag historical-snapshot -> uses the snapshot's schema +SELECT * FROM prod.db.table VERSION AS OF 'historical-snapshot'; +SELECT * FROM prod.db.table.`tag_historical-snapshot`; +``` + #### DataFrame To select a specific table snapshot or the snapshot at some time in the DataFrame API, Iceberg supports four Spark read options: diff --git a/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index 626dee6c96e6..96fcc5f7ce32 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -201,6 +201,10 @@ Note WAP branch and branch identifier cannot both be specified. Also, the branch must exist before performing the write. The operation does **not** create the branch if it does not exist. For more information on branches please refer to [branches](branching.md). + +!!! info + Note: When writing to a branch, the current schema of the table will be used for validation. + ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 80d7d8787e27..4c99a38d29fc 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -23,6 +23,7 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.text.SimpleDateFormat; +import java.util.Arrays; import java.util.Date; import java.util.List; import java.util.concurrent.TimeUnit; @@ -348,6 +349,64 @@ public void testBranchReference() { assertEquals("Snapshot at specific branch reference name", expected, fromDF); } + @TestTemplate + public void readAndWriteWithBranchAfterSchemaChange() { + Table table = validationCatalog.loadTable(tableIdent); + String branchName = "test_branch"; + table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); + + List expected = + Arrays.asList(row(1L, "a", 1.0f), row(2L, "b", 2.0f), row(3L, "c", Float.NaN)); + assertThat(sql("SELECT * FROM %s", tableName)).containsExactlyElementsOf(expected); + + // change schema on the table and add more data + sql("ALTER TABLE %s DROP COLUMN float", tableName); + sql("ALTER TABLE %s ADD COLUMN new_col date", tableName); + sql( + "INSERT INTO %s VALUES (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05'))", + tableName); + + // time-travel query using snapshot id should return the snapshot's schema + long branchSnapshotId = table.refs().get(branchName).snapshotId(); + assertThat(sql("SELECT * FROM %s VERSION AS OF %s", tableName, branchSnapshotId)) + .containsExactlyElementsOf(expected); + + // querying the head of the branch should return the table's schema + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); + + if (!"spark_catalog".equals(catalogName)) { + // querying the head of the branch using 'branch_' should return the table's schema + assertThat(sql("SELECT * FROM %s.branch_%s", tableName, branchName)) + .containsExactly(row(1L, "a", null), row(2L, "b", null), row(3L, "c", null)); + } + + // writing to a branch uses the table's schema + sql( + "INSERT INTO %s.branch_%s VALUES (6L, 'f', cast('2023-06-06' as date)), (7L, 'g', cast('2023-07-07' as date))", + tableName, branchName); + + // querying the head of the branch returns the table's schema + assertThat(sql("SELECT * FROM %s VERSION AS OF '%s'", tableName, branchName)) + .containsExactlyInAnyOrder( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", null), + row(6L, "f", java.sql.Date.valueOf("2023-06-06")), + row(7L, "g", java.sql.Date.valueOf("2023-07-07"))); + + // using DataFrameReader with the 'branch' option should return the table's schema + Dataset df = + spark.read().format("iceberg").option(SparkReadOptions.BRANCH, branchName).load(tableName); + assertThat(rowsToJava(df.collectAsList())) + .containsExactlyInAnyOrder( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", null), + row(6L, "f", java.sql.Date.valueOf("2023-06-06")), + row(7L, "g", java.sql.Date.valueOf("2023-07-07"))); + } + @TestTemplate public void testUnknownReferenceAsOf() { assertThatThrownBy(() -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) From 4e58d27fe74c5aa9a522a8c94d3f156537b21d93 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Sat, 30 Mar 2024 13:01:28 -0700 Subject: [PATCH 0184/1019] Flink: backport PR #9321 for range partitioner on map statistics (#10061) --- .../shuffle/MapRangePartitionerBenchmark.java | 199 ++++++++ .../sink/shuffle/MapRangePartitioner.java | 381 +++++++++++++++ .../sink/shuffle/TestMapRangePartitioner.java | 448 ++++++++++++++++++ .../shuffle/MapRangePartitionerBenchmark.java | 199 ++++++++ .../sink/shuffle/MapRangePartitioner.java | 381 +++++++++++++++ .../sink/shuffle/TestMapRangePartitioner.java | 448 ++++++++++++++++++ jmh.gradle | 8 + 7 files changed, 2064 insertions(+) create mode 100644 flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java create mode 100644 flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +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.apache.iceberg.types.Types; +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.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..fb1a8f03a65c --- /dev/null +++ b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,381 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +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; +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.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator comparator; + private final Map mapStatistics; + private final double closeFileCostInWeightPercentage; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + // lazily computed due to the need of numPartitions + private Map assignment; + private NavigableMap sortedStatsWithCloseFileCost; + + MapRangePartitioner( + Schema schema, + SortOrder sortOrder, + MapDataStatistics dataStatistics, + double closeFileCostInWeightPercentage) { + dataStatistics + .statistics() + .entrySet() + .forEach( + entry -> + Preconditions.checkArgument( + entry.getValue() > 0, + "Invalid statistics: weight is 0 for key %s", + entry.getKey())); + + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.mapStatistics = dataStatistics.statistics(); + this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // assignment table can only be built lazily when first referenced here, + // because number of partitions (downstream subtasks) is needed. + // the numPartitions is not available in the constructor. + Map assignmentMap = assignment(numPartitions); + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = assignmentMap.get(sortKey); + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + } + return (int) (newSortKeyCounter % numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostInWeight); + } + + return assignment; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostInWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostInWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostInWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks.clear(); + subtaskWeights.clear(); + currentKey = null; + } + } + + return assignmentMap; + } + + /** Subtask assignment for a key */ + @VisibleForTesting + static class KeyAssignment { + private final int[] assignedSubtasks; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It + * could also be multiple subtasks if the key has heavy weight that should be handled by + * multiple subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the + * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain + * values as [10, 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostInWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostInWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostInWeight)); + + this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.length == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks[0]; + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.length, + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks[position]; + } + } + + @Override + public int hashCode() { + return 31 * Arrays.hashCode(assignedSubtasks) + + Arrays.hashCode(subtaskWeightsExcludingCloseCost); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Arrays.equals(assignedSubtasks, that.assignedSubtasks) + && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) + .toString(); + } + } +} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..92eb71acc834 --- /dev/null +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.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.flink.sink.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 8; + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 9; + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 9; + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + partitioner + .mapStatistics() + .forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + Assertions.assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +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.apache.iceberg.types.Types; +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.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..fb1a8f03a65c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,381 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +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; +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.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator comparator; + private final Map mapStatistics; + private final double closeFileCostInWeightPercentage; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + // lazily computed due to the need of numPartitions + private Map assignment; + private NavigableMap sortedStatsWithCloseFileCost; + + MapRangePartitioner( + Schema schema, + SortOrder sortOrder, + MapDataStatistics dataStatistics, + double closeFileCostInWeightPercentage) { + dataStatistics + .statistics() + .entrySet() + .forEach( + entry -> + Preconditions.checkArgument( + entry.getValue() > 0, + "Invalid statistics: weight is 0 for key %s", + entry.getKey())); + + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.mapStatistics = dataStatistics.statistics(); + this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // assignment table can only be built lazily when first referenced here, + // because number of partitions (downstream subtasks) is needed. + // the numPartitions is not available in the constructor. + Map assignmentMap = assignment(numPartitions); + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = assignmentMap.get(sortKey); + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + } + return (int) (newSortKeyCounter % numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostInWeight); + } + + return assignment; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostInWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostInWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostInWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks.clear(); + subtaskWeights.clear(); + currentKey = null; + } + } + + return assignmentMap; + } + + /** Subtask assignment for a key */ + @VisibleForTesting + static class KeyAssignment { + private final int[] assignedSubtasks; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It + * could also be multiple subtasks if the key has heavy weight that should be handled by + * multiple subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the + * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain + * values as [10, 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostInWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostInWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostInWeight)); + + this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.length == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks[0]; + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.length, + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks[position]; + } + } + + @Override + public int hashCode() { + return 31 * Arrays.hashCode(assignedSubtasks) + + Arrays.hashCode(subtaskWeightsExcludingCloseCost); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Arrays.equals(assignedSubtasks, that.assignedSubtasks) + && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) + .toString(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..92eb71acc834 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.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.flink.sink.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 8; + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 9; + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 9; + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + partitioner + .mapStatistics() + .forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + Assertions.assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/jmh.gradle b/jmh.gradle index ea317cc2eea1..de50162cb099 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -26,10 +26,18 @@ def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getPro def scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") def jmhProjects = [project(":iceberg-core"), project(":iceberg-data")] +if (flinkVersions.contains("1.16")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.16")) +} + if (flinkVersions.contains("1.17")) { jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.17")) } +if (flinkVersions.contains("1.18")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.18")) +} + if (sparkVersions.contains("3.3")) { jmhProjects.add(project(":iceberg-spark:iceberg-spark-3.3_${scalaVersion}")) jmhProjects.add(project(":iceberg-spark:iceberg-spark-extensions-3.3_${scalaVersion}")) From 1cb2c85c8f8406acda03c648a0b28d9c1f09627a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 31 Mar 2024 12:55:18 +0200 Subject: [PATCH 0185/1019] Spark: Don't allow branch_ usage with TIMESTAMP AS OF (#10059) --- .../main/java/org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../test/java/org/apache/iceberg/spark/sql/TestSelect.java | 6 ++++++ .../main/java/org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../test/java/org/apache/iceberg/spark/sql/TestSelect.java | 6 ++++++ .../main/java/org/apache/iceberg/spark/SparkCatalog.java | 2 +- .../test/java/org/apache/iceberg/spark/sql/TestSelect.java | 5 +++++ 6 files changed, 20 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index 48decf995fbb..5638af071ad7 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -205,7 +205,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); // convert the timestamp to milliseconds as Spark passes microseconds diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 54d8dd01b92e..2265dec4763c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -453,6 +453,12 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using branch_b1 in the table identifier and TIMESTAMP AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @Test diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java index d72a281fe825..7894adee410d 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkCatalog.java @@ -215,7 +215,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); // convert the timestamp to milliseconds as Spark passes microseconds diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 93bb21b41a2b..85d340c8b27b 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -448,6 +448,12 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using branch_b1 in the table identifier and TIMESTAMP AS OF + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @Test 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 7357a4683bed..0be9fd948466 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 @@ -215,7 +215,7 @@ public Table loadTable(Identifier ident, long timestamp) throws NoSuchTableExcep SparkTable sparkTable = (SparkTable) table; Preconditions.checkArgument( - sparkTable.snapshotId() == null, + sparkTable.snapshotId() == null && sparkTable.branch() == null, "Cannot do time-travel based on both table identifier and AS OF"); // convert the timestamp to milliseconds as Spark passes microseconds diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 4c99a38d29fc..3ecfc60b49b4 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -529,6 +529,11 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); + + // using branch_b1 in the table identifier and TIMESTAMP AS OF + assertThatThrownBy(() -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @TestTemplate From 2b38d55d77930f285ae9f8efb745bb11d5491ed8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 Mar 2024 23:08:23 +0200 Subject: [PATCH 0186/1019] Build: Bump software.amazon.awssdk:bom from 2.25.18 to 2.25.21 (#10072) Bumps software.amazon.awssdk:bom from 2.25.18 to 2.25.21. --- 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 a0fb79bcd363..68e9ddee6dde 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.18" +awssdk-bom = "2.25.21" azuresdk-bom = "1.2.21" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 6b687461eff33911d228a20f7640c176a66d3bf7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 31 Mar 2024 23:08:51 +0200 Subject: [PATCH 0187/1019] Build: Bump org.glassfish.jaxb:jaxb-runtime from 2.3.3 to 2.3.9 (#9988) Bumps org.glassfish.jaxb:jaxb-runtime from 2.3.3 to 2.3.9. --- updated-dependencies: - dependency-name: org.glassfish.jaxb:jaxb-runtime 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 68e9ddee6dde..261fd983d386 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -57,7 +57,7 @@ jackson214 = { strictly = "2.14.2"} jackson215 = { strictly = "2.15.2"} jakarta-el-api = "3.0.3" jaxb-api = "2.3.1" -jaxb-runtime = "2.3.3" +jaxb-runtime = "2.3.9" jetty = "9.4.54.v20240208" junit = "5.10.1" kafka = "3.7.0" From 6bdafedc5f1da97c9a2140ef14c3da36bf3dcea5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 2 Apr 2024 08:42:44 +0200 Subject: [PATCH 0188/1019] Build: Bump com.azure:azure-sdk-bom from 1.2.21 to 1.2.22 (#10071) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.21 to 1.2.22. - [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.21...azure-sdk-bom_1.2.22) --- 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 261fd983d386..41b87a22faf4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" awssdk-bom = "2.25.21" -azuresdk-bom = "1.2.21" +azuresdk-bom = "1.2.22" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From 0288ab6d6935dcec95e2d6113101bf4b2d92ac09 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 2 Apr 2024 20:12:29 +0900 Subject: [PATCH 0189/1019] Core, Flink, Spark: Migrate remaining subclasses of TableTestBase to JUnit5 (#10063) --- .../org/apache/iceberg/TableTestBase.java | 753 ------------------ .../apache/iceberg/TestContentFileParser.java | 36 +- .../iceberg/TestFileScanTaskParser.java | 13 +- .../iceberg/util/TestTableScanUtil.java | 10 +- .../data/TestGenericFileWriterFactory.java | 5 - .../iceberg/io/TestFileWriterFactory.java | 189 ++--- .../iceberg/io/TestPartitioningWriters.java | 85 +- .../iceberg/io/TestPositionDeltaWriters.java | 49 +- .../iceberg/io/TestRollingFileWriters.java | 64 +- .../org/apache/iceberg/io/WriterTestBase.java | 8 +- .../flink/sink/TestDeltaTaskWriter.java | 62 +- .../sink/TestFlinkFileWriterFactory.java | 5 - .../sink/TestFlinkPartitioningWriters.java | 5 - .../sink/TestFlinkPositionDeltaWriters.java | 5 - .../sink/TestFlinkRollingFileWriters.java | 5 - .../flink/sink/TestIcebergFilesCommitter.java | 87 +- .../source/TestStreamingMonitorFunction.java | 43 +- .../source/TestStreamingReaderOperator.java | 38 +- .../flink/sink/TestDeltaTaskWriter.java | 62 +- .../sink/TestFlinkFileWriterFactory.java | 5 - .../sink/TestFlinkPartitioningWriters.java | 5 - .../sink/TestFlinkPositionDeltaWriters.java | 5 - .../sink/TestFlinkRollingFileWriters.java | 5 - .../flink/sink/TestIcebergFilesCommitter.java | 87 +- .../source/TestStreamingMonitorFunction.java | 43 +- .../source/TestStreamingReaderOperator.java | 38 +- .../flink/sink/TestDeltaTaskWriter.java | 62 +- .../sink/TestFlinkFileWriterFactory.java | 5 - .../sink/TestFlinkPartitioningWriters.java | 10 +- .../sink/TestFlinkPositionDeltaWriters.java | 5 - .../sink/TestFlinkRollingFileWriters.java | 5 - .../flink/sink/TestIcebergFilesCommitter.java | 87 +- .../source/TestStreamingMonitorFunction.java | 43 +- .../source/TestStreamingReaderOperator.java | 38 +- .../source/TestSparkFileWriterFactory.java | 5 - .../source/TestSparkPartitioningWriters.java | 5 - .../source/TestSparkPositionDeltaWriters.java | 5 - .../source/TestSparkRollingFileWriters.java | 5 - .../source/TestSparkFileWriterFactory.java | 5 - .../source/TestSparkPartitioningWriters.java | 5 - .../source/TestSparkPositionDeltaWriters.java | 5 - .../source/TestSparkRollingFileWriters.java | 5 - .../source/TestSparkFileWriterFactory.java | 5 - .../source/TestSparkPartitioningWriters.java | 5 - .../source/TestSparkPositionDeltaWriters.java | 5 - .../source/TestSparkRollingFileWriters.java | 5 - 46 files changed, 571 insertions(+), 1456 deletions(-) delete mode 100644 core/src/test/java/org/apache/iceberg/TableTestBase.java diff --git a/core/src/test/java/org/apache/iceberg/TableTestBase.java b/core/src/test/java/org/apache/iceberg/TableTestBase.java deleted file mode 100644 index a92497f88e87..000000000000 --- a/core/src/test/java/org/apache/iceberg/TableTestBase.java +++ /dev/null @@ -1,753 +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; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.Arrays; -import java.util.Iterator; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -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.Iterators; -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.Files; -import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -public class TableTestBase { - // Schema passed to create tables - public static final Schema SCHEMA = - new Schema( - required(3, "id", Types.IntegerType.get()), required(4, "data", Types.StringType.get())); - - protected static final int BUCKETS_NUMBER = 16; - - // Partition spec used to create tables - public static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).bucket("data", BUCKETS_NUMBER).build(); - - static final DataFile FILE_A = - DataFiles.builder(SPEC) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DataFile FILE_A2 = - DataFiles.builder(SPEC) - .withPath("/path/to/data-a-2.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DeleteFile FILE_A_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofPositionDeletes() - .withPath("/path/to/data-a-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - // Equality delete files. - static final DeleteFile FILE_A2_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofEqualityDeletes(1) - .withPath("/path/to/data-a2-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=0") - .withRecordCount(1) - .build(); - static final DataFile FILE_B = - DataFiles.builder(SPEC) - .withPath("/path/to/data-b.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=1") // easy way to set partition data for now - .withRecordCount(1) - .withSplitOffsets(ImmutableList.of(1L)) - .build(); - static final DeleteFile FILE_B_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofPositionDeletes() - .withPath("/path/to/data-b-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=1") // 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("data_bucket=2") // easy way to set partition data for now - .withRecordCount(1) - .withSplitOffsets(ImmutableList.of(2L, 8L)) - .build(); - static final DeleteFile FILE_C2_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofEqualityDeletes(1) - .withPath("/path/to/data-c-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=2") // 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("data_bucket=3") // easy way to set partition data for now - .withRecordCount(1) - .withSplitOffsets(ImmutableList.of(0L, 3L, 6L)) - .build(); - static final DeleteFile FILE_D2_DELETES = - FileMetadata.deleteFileBuilder(SPEC) - .ofEqualityDeletes(1) - .withPath("/path/to/data-d-deletes.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("data_bucket=3") // easy way to set partition data for now - .withRecordCount(1) - .build(); - static final DataFile FILE_WITH_STATS = - DataFiles.builder(SPEC) - .withPath("/path/to/data-with-stats.parquet") - .withMetrics( - new Metrics( - 10L, - ImmutableMap.of(3, 100L, 4, 200L), // column sizes - ImmutableMap.of(3, 90L, 4, 180L), // value counts - ImmutableMap.of(3, 10L, 4, 20L), // null value counts - ImmutableMap.of(3, 0L, 4, 0L), // nan value counts - ImmutableMap.of( - 3, - Conversions.toByteBuffer(Types.IntegerType.get(), 1), - 4, - Conversions.toByteBuffer(Types.IntegerType.get(), 2)), // lower bounds - ImmutableMap.of( - 3, - Conversions.toByteBuffer(Types.IntegerType.get(), 5), - 4, - Conversions.toByteBuffer(Types.IntegerType.get(), 10)) // upperbounds - )) - .withFileSizeInBytes(350) - .build(); - - static final FileIO FILE_IO = new TestTables.LocalFileIO(); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - protected File tableDir = null; - protected File metadataDir = null; - public TestTables.TestTable table = null; - - protected final int formatVersion; - - @SuppressWarnings("checkstyle:MemberName") - protected final TableAssertions V1Assert; - - @SuppressWarnings("checkstyle:MemberName") - protected final TableAssertions V2Assert; - - public TableTestBase(int formatVersion) { - this.formatVersion = formatVersion; - this.V1Assert = new TableAssertions(1, formatVersion); - this.V2Assert = new TableAssertions(2, formatVersion); - } - - @Before - public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); - tableDir.delete(); // created by table create - - this.metadataDir = new File(tableDir, "metadata"); - this.table = create(SCHEMA, SPEC); - } - - @After - public void cleanupTables() { - TestTables.clearTables(); - } - - List listManifestFiles() { - return listManifestFiles(tableDir); - } - - List listManifestFiles(File tableDirToList) { - return Lists.newArrayList( - new File(tableDirToList, "metadata") - .listFiles( - (dir, name) -> - !name.startsWith("snap") - && Files.getFileExtension(name).equalsIgnoreCase("avro"))); - } - - List listManifestLists(String tableDirToList) { - return Lists.newArrayList( - new File(tableDirToList, "metadata") - .listFiles( - (dir, name) -> - name.startsWith("snap") - && Files.getFileExtension(name).equalsIgnoreCase("avro"))); - } - - public static long countAllMetadataFiles(File tableDir) { - return Arrays.stream(new File(tableDir, "metadata").listFiles()) - .filter(f -> f.isFile()) - .count(); - } - - protected TestTables.TestTable create(Schema schema, PartitionSpec spec) { - return TestTables.create(tableDir, "test", schema, spec, formatVersion); - } - - TestTables.TestTable load() { - return TestTables.load(tableDir, "test"); - } - - Integer version() { - return TestTables.metadataVersion("test"); - } - - public TableMetadata readMetadata() { - return TestTables.readMetadata("test"); - } - - ManifestFile writeManifest(DataFile... files) throws IOException { - return writeManifest(null, files); - } - - ManifestFile writeManifest(Long snapshotId, DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - ManifestFile writeManifest(String fileName, ManifestEntry... entries) throws IOException { - return writeManifest(null, fileName, entries); - } - - ManifestFile writeManifest(Long snapshotId, ManifestEntry... entries) throws IOException { - return writeManifest(snapshotId, "input.m0.avro", entries); - } - - @SuppressWarnings("unchecked") - > ManifestFile writeManifest( - Long snapshotId, String fileName, ManifestEntry... entries) throws IOException { - File manifestFile = temp.newFile(fileName); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer; - if (entries[0].file() instanceof DataFile) { - writer = - (ManifestWriter) - ManifestFiles.write(formatVersion, table.spec(), outputFile, snapshotId); - } else { - writer = - (ManifestWriter) - ManifestFiles.writeDeleteManifest( - formatVersion, table.spec(), outputFile, snapshotId); - } - try { - for (ManifestEntry entry : entries) { - writer.addEntry((ManifestEntry) entry); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - ManifestFile writeDeleteManifest(int newFormatVersion, Long snapshotId, DeleteFile... deleteFiles) - throws IOException { - OutputFile manifestFile = - org.apache.iceberg.Files.localOutput( - FileFormat.AVRO.addExtension(temp.newFile().toString())); - ManifestWriter writer = - ManifestFiles.writeDeleteManifest(newFormatVersion, SPEC, manifestFile, snapshotId); - try { - for (DeleteFile deleteFile : deleteFiles) { - writer.add(deleteFile); - } - } finally { - writer.close(); - } - return writer.toManifestFile(); - } - - ManifestFile writeManifestWithName(String name, DataFile... files) throws IOException { - File manifestFile = temp.newFile(name + ".avro"); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = table.ops().io().newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = - ManifestFiles.write(formatVersion, table.spec(), outputFile, null); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - > ManifestEntry manifestEntry( - ManifestEntry.Status status, Long snapshotId, F file) { - return manifestEntry(status, snapshotId, 0L, 0L, file); - } - - > ManifestEntry manifestEntry( - ManifestEntry.Status status, - Long snapshotId, - Long dataSequenceNumber, - Long fileSequenceNumber, - F file) { - - GenericManifestEntry entry = new GenericManifestEntry<>(table.spec().partitionType()); - switch (status) { - case ADDED: - if (dataSequenceNumber != null && dataSequenceNumber != 0) { - return entry.wrapAppend(snapshotId, dataSequenceNumber, file); - } else { - return entry.wrapAppend(snapshotId, file); - } - case EXISTING: - return entry.wrapExisting(snapshotId, dataSequenceNumber, fileSequenceNumber, file); - case DELETED: - return entry.wrapDelete(snapshotId, dataSequenceNumber, fileSequenceNumber, file); - default: - throw new IllegalArgumentException("Unexpected entry status: " + status); - } - } - - void validateSnapshot(Snapshot old, Snapshot snap, DataFile... newFiles) { - validateSnapshot(old, snap, null, newFiles); - } - - void validateSnapshot(Snapshot old, Snapshot snap, long sequenceNumber, DataFile... newFiles) { - validateSnapshot(old, snap, (Long) sequenceNumber, newFiles); - } - - @SuppressWarnings("checkstyle:HiddenField") - Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { - Snapshot snapshot; - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - snapshotUpdate.commit(); - snapshot = table.currentSnapshot(); - } else { - ((SnapshotProducer) snapshotUpdate.toBranch(branch)).commit(); - snapshot = table.snapshot(branch); - } - - return snapshot; - } - - Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return ((SnapshotProducer) snapshotUpdate).apply(); - } else { - return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); - } - } - - void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile... newFiles) { - Assert.assertEquals( - "Should not change delete manifests", - old != null ? Sets.newHashSet(old.deleteManifests(FILE_IO)) : ImmutableSet.of(), - Sets.newHashSet(snap.deleteManifests(FILE_IO))); - List oldManifests = old != null ? old.dataManifests(FILE_IO) : ImmutableList.of(); - - // copy the manifests to a modifiable list and remove the existing manifests - List newManifests = Lists.newArrayList(snap.dataManifests(FILE_IO)); - for (ManifestFile oldManifest : oldManifests) { - Assert.assertTrue( - "New snapshot should contain old manifests", newManifests.remove(oldManifest)); - } - - Assert.assertEquals( - "Should create 1 new manifest and reuse old manifests", 1, newManifests.size()); - ManifestFile manifest = newManifests.get(0); - - long id = snap.snapshotId(); - Iterator newPaths = paths(newFiles).iterator(); - - for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) { - DataFile file = entry.file(); - if (sequenceNumber != null) { - V1Assert.assertEquals( - "Data sequence number should default to 0", 0, entry.dataSequenceNumber().longValue()); - V1Assert.assertEquals( - "Data sequence number should default to 0", - 0, - entry.file().dataSequenceNumber().longValue()); - - V2Assert.assertEquals( - "Data sequence number should match expected", - sequenceNumber, - entry.dataSequenceNumber()); - V2Assert.assertEquals( - "Data sequence number should match expected", - sequenceNumber, - entry.file().dataSequenceNumber()); - V2Assert.assertEquals( - "Sequence number should match expected", - snap.sequenceNumber(), - entry.dataSequenceNumber().longValue()); - - V2Assert.assertEquals( - "File sequence number should match expected", - sequenceNumber, - entry.file().fileSequenceNumber()); - V2Assert.assertEquals( - "File sequence number should match expected", - snap.sequenceNumber(), - entry.file().fileSequenceNumber().longValue()); - } - Assert.assertEquals("Path should match expected", newPaths.next(), file.path().toString()); - Assert.assertEquals("File's snapshot ID should match", id, (long) entry.snapshotId()); - } - - Assert.assertFalse("Should find all files in the manifest", newPaths.hasNext()); - - Assert.assertEquals("Schema ID should match", table.schema().schemaId(), (int) snap.schemaId()); - } - - void validateTableFiles(Table tbl, DataFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DataFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().planFiles()) { - actualFilePaths.add(task.file().path()); - } - Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); - } - - void validateBranchFiles(Table tbl, String ref, DataFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DataFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().useRef(ref).planFiles()) { - actualFilePaths.add(task.file().path()); - } - Assert.assertEquals("Files should match", expectedFilePaths, actualFilePaths); - } - - void validateBranchDeleteFiles(Table tbl, String branch, DeleteFile... expectedFiles) { - Set expectedFilePaths = Sets.newHashSet(); - for (DeleteFile file : expectedFiles) { - expectedFilePaths.add(file.path()); - } - Set actualFilePaths = Sets.newHashSet(); - for (FileScanTask task : tbl.newScan().useRef(branch).planFiles()) { - for (DeleteFile file : task.deletes()) { - actualFilePaths.add(file.path()); - } - } - Assert.assertEquals("Delete files should match", expectedFilePaths, actualFilePaths); - } - - List paths(DataFile... dataFiles) { - List paths = Lists.newArrayListWithExpectedSize(dataFiles.length); - for (DataFile file : dataFiles) { - paths.add(file.path().toString()); - } - return paths; - } - - void validateManifest( - ManifestFile manifest, Iterator ids, Iterator expectedFiles) { - validateManifest(manifest, null, null, ids, expectedFiles, null); - } - - void validateManifest( - ManifestFile manifest, - Iterator dataSeqs, - Iterator fileSeqs, - Iterator ids, - Iterator expectedFiles) { - validateManifest(manifest, dataSeqs, fileSeqs, ids, expectedFiles, null); - } - - void validateManifest( - ManifestFile manifest, - Iterator dataSeqs, - Iterator fileSeqs, - Iterator ids, - Iterator expectedFiles, - Iterator statuses) { - for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) { - DataFile file = entry.file(); - DataFile expected = expectedFiles.next(); - - validateManifestSequenceNumbers(entry, dataSeqs, fileSeqs); - - Assert.assertEquals( - "Path should match expected", expected.path().toString(), file.path().toString()); - Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId()); - if (statuses != null) { - Assert.assertEquals("Status should match expected", statuses.next(), entry.status()); - } - } - - Assert.assertFalse("Should find all files in the manifest", expectedFiles.hasNext()); - } - - void validateDeleteManifest( - ManifestFile manifest, - Iterator dataSeqs, - Iterator fileSeqs, - Iterator ids, - Iterator expectedFiles, - Iterator statuses) { - for (ManifestEntry entry : - ManifestFiles.readDeleteManifest(manifest, FILE_IO, null).entries()) { - DeleteFile file = entry.file(); - DeleteFile expected = expectedFiles.next(); - - validateManifestSequenceNumbers(entry, dataSeqs, fileSeqs); - - Assert.assertEquals( - "Path should match expected", expected.path().toString(), file.path().toString()); - Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId()); - Assert.assertEquals("Status should match expected", statuses.next(), entry.status()); - } - - Assert.assertFalse("Should find all files in the manifest", expectedFiles.hasNext()); - } - - private > void validateManifestSequenceNumbers( - ManifestEntry entry, Iterator dataSeqs, Iterator fileSeqs) { - if (dataSeqs != null) { - V1Assert.assertEquals( - "Data sequence number should default to 0", 0, entry.dataSequenceNumber().longValue()); - V1Assert.assertEquals( - "Data sequence number should default to 0", - 0, - entry.file().dataSequenceNumber().longValue()); - - Long expectedSequenceNumber = dataSeqs.next(); - V2Assert.assertEquals( - "Data sequence number should match expected", - expectedSequenceNumber, - entry.dataSequenceNumber()); - V2Assert.assertEquals( - "Data sequence number should match expected", - expectedSequenceNumber, - entry.file().dataSequenceNumber()); - } - - if (fileSeqs != null) { - V1Assert.assertEquals( - "File sequence number should default to 0", (Long) 0L, entry.fileSequenceNumber()); - V1Assert.assertEquals( - "File sequence number should default to 0", (Long) 0L, entry.file().fileSequenceNumber()); - - Long expectedFileSequenceNumber = fileSeqs.next(); - V2Assert.assertEquals( - "File sequence number should match", - expectedFileSequenceNumber, - entry.fileSequenceNumber()); - V2Assert.assertEquals( - "File sequence number should match", - expectedFileSequenceNumber, - entry.file().fileSequenceNumber()); - } - } - - protected DataFile newDataFile(String partitionPath) { - return DataFiles.builder(table.spec()) - .withPath("/path/to/data-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); - } - - protected DeleteFile newDeleteFile(int specId, String partitionPath) { - PartitionSpec spec = table.specs().get(specId); - return FileMetadata.deleteFileBuilder(spec) - .ofPositionDeletes() - .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); - } - - protected DeleteFile newEqualityDeleteFile(int specId, String partitionPath, int... fieldIds) { - PartitionSpec spec = table.specs().get(specId); - return FileMetadata.deleteFileBuilder(spec) - .ofEqualityDeletes(fieldIds) - .withPath("/path/to/delete-" + UUID.randomUUID() + ".parquet") - .withFileSizeInBytes(10) - .withPartitionPath(partitionPath) - .withRecordCount(1) - .build(); - } - - protected PositionDelete positionDelete(CharSequence path, long pos, T row) { - PositionDelete positionDelete = PositionDelete.create(); - return positionDelete.set(path, pos, row); - } - - protected void withUnavailableLocations(Iterable locations, Action action) { - for (String location : locations) { - move(location, location + "_temp"); - } - - try { - action.invoke(); - } finally { - for (String location : locations) { - move(location + "_temp", location); - } - } - } - - private void move(String location, String newLocation) { - Path path = Paths.get(location); - Path tempPath = Paths.get(newLocation); - - try { - java.nio.file.Files.move(path, tempPath); - } catch (IOException e) { - throw new UncheckedIOException("Failed to move: " + location, e); - } - } - - static void validateManifestEntries( - ManifestFile manifest, - Iterator ids, - Iterator expectedFiles, - Iterator expectedStatuses) { - for (ManifestEntry entry : ManifestFiles.read(manifest, FILE_IO).entries()) { - DataFile file = entry.file(); - DataFile expected = expectedFiles.next(); - final ManifestEntry.Status expectedStatus = expectedStatuses.next(); - Assert.assertEquals( - "Path should match expected", expected.path().toString(), file.path().toString()); - Assert.assertEquals("Snapshot ID should match expected ID", ids.next(), entry.snapshotId()); - Assert.assertEquals("Entry status should match expected ID", expectedStatus, entry.status()); - } - - Assert.assertFalse("Should find all files in the manifest", expectedFiles.hasNext()); - } - - static Iterator statuses(ManifestEntry.Status... statuses) { - return Iterators.forArray(statuses); - } - - static Iterator dataSeqs(Long... seqs) { - return Iterators.forArray(seqs); - } - - static Iterator fileSeqs(Long... seqs) { - return Iterators.forArray(seqs); - } - - static Iterator ids(Long... ids) { - return Iterators.forArray(ids); - } - - static Iterator files(DataFile... files) { - return Iterators.forArray(files); - } - - static Iterator files(DeleteFile... files) { - return Iterators.forArray(files); - } - - static Iterator files(ManifestFile manifest) { - return ManifestFiles.read(manifest, FILE_IO).iterator(); - } - - /** Used for assertions that only apply if the table version is v2. */ - protected static class TableAssertions { - private boolean enabled; - - private TableAssertions(int validForVersion, int formatVersion) { - this.enabled = validForVersion == formatVersion; - } - - void disable() { - this.enabled = false; - } - - void enable() { - this.enabled = true; - } - - void assertEquals(String context, int expected, int actual) { - if (enabled) { - Assert.assertEquals(context, expected, actual); - } - } - - void assertEquals(String context, long expected, long actual) { - if (enabled) { - Assert.assertEquals(context, expected, actual); - } - } - - void assertEquals(String context, Object expected, Object actual) { - if (enabled) { - Assert.assertEquals(context, expected, actual); - } - } - } - - @FunctionalInterface - protected interface Action { - void invoke(); - } -} diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 4fda388159ec..1e185b6544e1 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -37,24 +37,24 @@ public class TestContentFileParser { @Test public void testNullArguments() throws Exception { - Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(null, TableTestBase.SPEC)) + Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(null, TestBase.SPEC)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid content file: null"); - Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(TableTestBase.FILE_A, null)) + Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(TestBase.FILE_A, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid partition spec: null"); Assertions.assertThatThrownBy( - () -> ContentFileParser.toJson(TableTestBase.FILE_A, TableTestBase.SPEC, null)) + () -> ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON generator: null"); - Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(null, TableTestBase.SPEC)) + Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(null, TestBase.SPEC)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON node for content file: null"); - String jsonStr = ContentFileParser.toJson(TableTestBase.FILE_A, TableTestBase.SPEC); + String jsonStr = ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, null)) .isInstanceOf(IllegalArgumentException.class) @@ -96,13 +96,13 @@ private static Stream provideSpecAndDataFile() { dataFileWithAllOptional(PartitionSpec.unpartitioned()), dataFileJsonWithAllOptional(PartitionSpec.unpartitioned())), Arguments.of( - TableTestBase.SPEC, - dataFileWithRequiredOnly(TableTestBase.SPEC), - dataFileJsonWithRequiredOnly(TableTestBase.SPEC)), + TestBase.SPEC, + dataFileWithRequiredOnly(TestBase.SPEC), + dataFileJsonWithRequiredOnly(TestBase.SPEC)), Arguments.of( - TableTestBase.SPEC, - dataFileWithAllOptional(TableTestBase.SPEC), - dataFileJsonWithAllOptional(TableTestBase.SPEC))); + TestBase.SPEC, + dataFileWithAllOptional(TestBase.SPEC), + dataFileJsonWithAllOptional(TestBase.SPEC))); } private static DataFile dataFileWithRequiredOnly(PartitionSpec spec) { @@ -182,7 +182,7 @@ private static DataFile dataFileWithAllOptional(PartitionSpec spec) { .withSplitOffsets(Arrays.asList(128L, 256L)) .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[16])) .withSortOrder( - SortOrder.builderFor(TableTestBase.SCHEMA) + SortOrder.builderFor(TestBase.SCHEMA) .withOrderId(1) .sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST) .build()); @@ -206,13 +206,13 @@ private static Stream provideSpecAndDeleteFile() { deleteFileWithAllOptional(PartitionSpec.unpartitioned()), deleteFileJsonWithAllOptional(PartitionSpec.unpartitioned())), Arguments.of( - TableTestBase.SPEC, - deleteFileWithRequiredOnly(TableTestBase.SPEC), - deleteFileJsonWithRequiredOnly(TableTestBase.SPEC)), + TestBase.SPEC, + deleteFileWithRequiredOnly(TestBase.SPEC), + deleteFileJsonWithRequiredOnly(TestBase.SPEC)), Arguments.of( - TableTestBase.SPEC, - deleteFileWithAllOptional(TableTestBase.SPEC), - deleteFileJsonWithAllOptional(TableTestBase.SPEC))); + TestBase.SPEC, + deleteFileWithAllOptional(TestBase.SPEC), + deleteFileJsonWithAllOptional(TestBase.SPEC))); } private static DeleteFile deleteFileWithRequiredOnly(PartitionSpec spec) { diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 42785c7a778e..73bfba996ebf 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -41,7 +41,7 @@ public void testNullArguments() { @ParameterizedTest @ValueSource(booleans = {true, false}) public void testParser(boolean caseSensitive) { - PartitionSpec spec = TableTestBase.SPEC; + PartitionSpec spec = TestBase.SPEC; FileScanTask fileScanTask = createScanTask(spec, caseSensitive); String jsonStr = FileScanTaskParser.toJson(fileScanTask); Assertions.assertThat(jsonStr).isEqualTo(expectedFileScanTaskJson()); @@ -58,9 +58,9 @@ private FileScanTask createScanTask(PartitionSpec spec, boolean caseSensitive) { } return new BaseFileScanTask( - TableTestBase.FILE_A, - new DeleteFile[] {TableTestBase.FILE_A_DELETES, TableTestBase.FILE_A2_DELETES}, - SchemaParser.toJson(TableTestBase.SCHEMA), + TestBase.FILE_A, + new DeleteFile[] {TestBase.FILE_A_DELETES, TestBase.FILE_A2_DELETES}, + SchemaParser.toJson(TestBase.SCHEMA), PartitionSpecParser.toJson(spec), residualEvaluator); } @@ -99,10 +99,7 @@ private static void assertFileScanTaskEquals( Assertions.assertThat(actual.spec()).isEqualTo(expected.spec()); Assertions.assertThat( ExpressionUtil.equivalent( - expected.residual(), - actual.residual(), - TableTestBase.SCHEMA.asStruct(), - caseSensitive)) + expected.residual(), actual.residual(), TestBase.SCHEMA.asStruct(), caseSensitive)) .as("Residual expression should match") .isTrue(); } 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 cfe3cb625e4f..d868fba90190 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java @@ -42,7 +42,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SplittableScanTask; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; @@ -136,7 +136,7 @@ public void testTaskGroupPlanning() { @Test public void testTaskGroupPlanningCorruptedOffset() { DataFile dataFile = - DataFiles.builder(TableTestBase.SPEC) + DataFiles.builder(TestBase.SPEC) .withPath("/path/to/data-a.parquet") .withFileSizeInBytes(10) .withPartitionPath("data_bucket=0") @@ -146,14 +146,14 @@ public void testTaskGroupPlanningCorruptedOffset() { .build(); ResidualEvaluator residualEvaluator = - ResidualEvaluator.of(TableTestBase.SPEC, Expressions.equal("id", 1), false); + ResidualEvaluator.of(TestBase.SPEC, Expressions.equal("id", 1), false); BaseFileScanTask baseFileScanTask = new BaseFileScanTask( dataFile, null, - SchemaParser.toJson(TableTestBase.SCHEMA), - PartitionSpecParser.toJson(TableTestBase.SPEC), + SchemaParser.toJson(TestBase.SCHEMA), + PartitionSpecParser.toJson(TestBase.SPEC), residualEvaluator); List baseFileScanTasks = ImmutableList.of(baseFileScanTask); diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java b/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java index 6752234f5ef5..2ab908c8c701 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.data; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -28,10 +27,6 @@ public class TestGenericFileWriterFactory extends TestFileWriterFactory { - public TestGenericFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, 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 152679abbde5..996a434f97f3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java +++ b/data/src/test/java/org/apache/iceberg/io/TestFileWriterFactory.java @@ -21,14 +21,21 @@ import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH; import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS; import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; +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.Files; +import java.util.Arrays; import java.util.List; import java.util.Map; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; 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.StructLike; @@ -50,43 +57,35 @@ import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestFileWriterFactory extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO, false}, - new Object[] {FileFormat.AVRO, true}, - new Object[] {FileFormat.PARQUET, false}, - new Object[] {FileFormat.PARQUET, true}, - new Object[] {FileFormat.ORC, false}, - new Object[] {FileFormat.ORC, true} - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, Partitioned = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO, false}, + new Object[] {2, FileFormat.AVRO, true}, + new Object[] {2, FileFormat.PARQUET, false}, + new Object[] {2, FileFormat.PARQUET, true}, + new Object[] {2, FileFormat.ORC, false}, + new Object[] {2, FileFormat.ORC, true}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final String PARTITION_VALUE = "aaa"; - private final FileFormat fileFormat; - private final boolean partitioned; + @Parameter(index = 1) + private FileFormat fileFormat; + + @Parameter(index = 2) + private boolean partitioned; private StructLike partition = null; private OutputFileFactory fileFactory = null; private List dataRows; - public TestFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - this.partitioned = partitioned; - } - protected abstract StructLikeSet toSet(Iterable records); protected FileFormat format() { @@ -94,10 +93,10 @@ protected FileFormat format() { } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); - Assert.assertTrue(tableDir.delete()); // created during table creation + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -116,7 +115,7 @@ public void setupTable() throws Exception { toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa"), toRow(5, "aaa")); } - @Test + @TestTemplate public void testDataWriter() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -124,10 +123,10 @@ public void testDataWriter() throws IOException { table.newRowDelta().addRows(dataFile).commit(); - Assert.assertEquals("Records should match", toSet(dataRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(dataRows)); } - @Test + @TestTemplate public void testEqualityDeleteWriter() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -151,19 +150,19 @@ public void testEqualityDeleteWriter() throws IOException { deleteRecord.copy("id", 1), deleteRecord.copy("id", 3), deleteRecord.copy("id", 5)); InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); List actualDeletes = readFile(equalityDeleteRowSchema, inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + assertThat(actualDeletes).isEqualTo(expectedDeletes); // commit the written delete file table.newRowDelta().addDeletes(deleteFile).commit(); // verify the delete file is applied correctly List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(4, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { - Assume.assumeFalse("Table must start unpartitioned", partitioned); + assumeThat(partitioned).isFalse(); List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -172,8 +171,9 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // write an unpartitioned data file DataFile firstDataFile = writeData(writerFactory, dataRows, table.spec(), partition); - Assert.assertEquals( - "First data file must be unpartitioned", 0, firstDataFile.partition().size()); + assertThat(firstDataFile.partition().size()) + .as("First data file must be unpartitioned") + .isEqualTo(0); List deletes = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa")); @@ -181,8 +181,9 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // write an unpartitioned delete file DeleteFile firstDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition); - Assert.assertEquals( - "First delete file must be unpartitioned", 0, firstDeleteFile.partition().size()); + assertThat(firstDeleteFile.partition().size()) + .as("First delete file must be unpartitioned") + .isEqualTo(0); // commit the first data and delete files table.newAppend().appendFile(firstDataFile).commit(); @@ -195,14 +196,16 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // write a partitioned data file DataFile secondDataFile = writeData(writerFactory, dataRows, table.spec(), partition); - Assert.assertEquals( - "Second data file must be partitioned", 1, secondDataFile.partition().size()); + assertThat(secondDataFile.partition().size()) + .as("Second data file must be partitioned") + .isEqualTo(1); // write a partitioned delete file DeleteFile secondDeleteFile = writeEqualityDeletes(writerFactory, deletes, table.spec(), partition); - Assert.assertEquals( - "Second delete file must be artitioned", 1, secondDeleteFile.partition().size()); + assertThat(secondDeleteFile.partition().size()) + .as("Second delete file must be partitioned") + .isEqualTo(1); // commit the second data and delete files table.newAppend().appendFile(secondDataFile).commit(); @@ -210,10 +213,10 @@ public void testEqualityDeleteWriterWithMultipleSpecs() throws IOException { // verify both delete files are applied correctly List expectedRows = ImmutableList.of(toRow(5, "aaa"), toRow(5, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testPositionDeleteWriter() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -232,21 +235,19 @@ public void testPositionDeleteWriter() throws IOException { CharSequenceSet referencedDataFiles = result.second(); if (fileFormat == FileFormat.AVRO) { - Assert.assertNull(deleteFile.lowerBounds()); - Assert.assertNull(deleteFile.upperBounds()); - Assert.assertNull(deleteFile.columnSizes()); + assertThat(deleteFile.lowerBounds()).isNull(); + assertThat(deleteFile.upperBounds()).isNull(); + assertThat(deleteFile.columnSizes()).isNull(); } else { - Assert.assertEquals(1, referencedDataFiles.size()); - Assert.assertEquals(2, deleteFile.lowerBounds().size()); - Assert.assertTrue(deleteFile.lowerBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertEquals(2, deleteFile.upperBounds().size()); - Assert.assertTrue(deleteFile.upperBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertEquals(2, deleteFile.columnSizes().size()); + assertThat(referencedDataFiles).hasSize(1); + assertThat(deleteFile.lowerBounds()).hasSize(2).containsKey(DELETE_FILE_PATH.fieldId()); + assertThat(deleteFile.upperBounds()).hasSize(2).containsKey(DELETE_FILE_PATH.fieldId()); + assertThat(deleteFile.columnSizes()).hasSize(2); } - Assert.assertNull(deleteFile.valueCounts()); - Assert.assertNull(deleteFile.nullValueCounts()); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(deleteFile.valueCounts()).isNull(); + assertThat(deleteFile.nullValueCounts()).isNull(); + assertThat(deleteFile.nanValueCounts()).isNull(); // verify the written delete file GenericRecord deleteRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema()); @@ -258,7 +259,7 @@ public void testPositionDeleteWriter() throws IOException { DELETE_FILE_PATH.name(), dataFile.path(), DELETE_FILE_POS.name(), 4L)); InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); List actualDeletes = readFile(DeleteSchemaUtil.pathPosSchema(), inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + assertThat(actualDeletes).isEqualTo(expectedDeletes); // commit the data and delete files table @@ -271,10 +272,10 @@ public void testPositionDeleteWriter() throws IOException { // verify the delete file is applied correctly List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(4, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testPositionDeleteWriterWithRow() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema(), table.schema()); @@ -290,31 +291,33 @@ public void testPositionDeleteWriterWithRow() throws IOException { CharSequenceSet referencedDataFiles = result.second(); if (fileFormat == FileFormat.AVRO) { - Assert.assertNull(deleteFile.lowerBounds()); - Assert.assertNull(deleteFile.upperBounds()); - Assert.assertNull(deleteFile.columnSizes()); - Assert.assertNull(deleteFile.valueCounts()); - Assert.assertNull(deleteFile.nullValueCounts()); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(deleteFile.lowerBounds()).isNull(); + assertThat(deleteFile.upperBounds()).isNull(); + assertThat(deleteFile.columnSizes()).isNull(); + assertThat(deleteFile.valueCounts()).isNull(); + assertThat(deleteFile.nullValueCounts()).isNull(); + assertThat(deleteFile.nanValueCounts()).isNull(); } else { - Assert.assertEquals(1, referencedDataFiles.size()); - Assert.assertEquals(4, deleteFile.lowerBounds().size()); - Assert.assertTrue(deleteFile.lowerBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertTrue(deleteFile.lowerBounds().containsKey(DELETE_FILE_POS.fieldId())); + assertThat(referencedDataFiles).hasSize(1); + assertThat(deleteFile.lowerBounds()) + .hasSize(4) + .containsKey(DELETE_FILE_PATH.fieldId()) + .containsKey(DELETE_FILE_POS.fieldId()); for (Types.NestedField column : table.schema().columns()) { - Assert.assertTrue(deleteFile.lowerBounds().containsKey(column.fieldId())); + assertThat(deleteFile.lowerBounds()).containsKey(column.fieldId()); } - Assert.assertEquals(4, deleteFile.upperBounds().size()); - Assert.assertTrue(deleteFile.upperBounds().containsKey(DELETE_FILE_PATH.fieldId())); - Assert.assertTrue(deleteFile.upperBounds().containsKey(DELETE_FILE_POS.fieldId())); + assertThat(deleteFile.upperBounds()) + .hasSize(4) + .containsKey(DELETE_FILE_PATH.fieldId()) + .containsKey(DELETE_FILE_POS.fieldId()); for (Types.NestedField column : table.schema().columns()) { - Assert.assertTrue(deleteFile.upperBounds().containsKey(column.fieldId())); + assertThat(deleteFile.upperBounds()).containsKey(column.fieldId()); } // ORC also contains metrics for the deleted row struct, not just actual data fields - Assert.assertTrue(deleteFile.columnSizes().size() >= 4); - Assert.assertTrue(deleteFile.valueCounts().size() >= 2); - Assert.assertTrue(deleteFile.nullValueCounts().size() >= 2); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(deleteFile.columnSizes()).hasSizeGreaterThanOrEqualTo(4); + assertThat(deleteFile.valueCounts()).hasSizeGreaterThanOrEqualTo(2); + assertThat(deleteFile.nullValueCounts()).hasSizeGreaterThanOrEqualTo(2); + assertThat(deleteFile.nanValueCounts()).isNull(); } // verify the written delete file @@ -332,7 +335,7 @@ public void testPositionDeleteWriterWithRow() throws IOException { List expectedDeletes = ImmutableList.of(deleteRecord.copy(deleteRecordColumns)); InputFile inputDeleteFile = table.io().newInputFile(deleteFile.path().toString()); List actualDeletes = readFile(positionDeleteSchema, inputDeleteFile); - Assert.assertEquals("Delete records must match", expectedDeletes, actualDeletes); + assertThat(actualDeletes).isEqualTo(expectedDeletes); // commit the data and delete files table @@ -346,10 +349,10 @@ public void testPositionDeleteWriterWithRow() throws IOException { // verify the delete file is applied correctly List expectedRows = ImmutableList.of(toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa"), toRow(5, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testPositionDeleteWriterMultipleDataFiles() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -369,17 +372,17 @@ public void testPositionDeleteWriterMultipleDataFiles() throws IOException { CharSequenceSet referencedDataFiles = result.second(); // verify the written delete file has NO lower and upper bounds - Assert.assertEquals(2, referencedDataFiles.size()); - Assert.assertNull(deleteFile.lowerBounds()); - Assert.assertNull(deleteFile.upperBounds()); - Assert.assertNull(deleteFile.valueCounts()); - Assert.assertNull(deleteFile.nullValueCounts()); - Assert.assertNull(deleteFile.nanValueCounts()); + assertThat(referencedDataFiles).hasSize(2); + assertThat(deleteFile.lowerBounds()).isNull(); + assertThat(deleteFile.upperBounds()).isNull(); + assertThat(deleteFile.valueCounts()).isNull(); + assertThat(deleteFile.nullValueCounts()).isNull(); + assertThat(deleteFile.nanValueCounts()).isNull(); if (fileFormat == FileFormat.AVRO) { - Assert.assertNull(deleteFile.columnSizes()); + assertThat(deleteFile.columnSizes()).isNull(); } else { - Assert.assertEquals(2, deleteFile.columnSizes().size()); + assertThat(deleteFile.columnSizes()).hasSize(2); } // commit the data and delete files @@ -402,7 +405,7 @@ public void testPositionDeleteWriterMultipleDataFiles() throws IOException { toRow(2, "aaa"), toRow(3, "aaa"), toRow(4, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } private DataFile writeData( 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 d76774326272..88d3c16c2dca 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -22,9 +22,14 @@ 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; 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.RowDelta; import org.apache.iceberg.Schema; @@ -35,33 +40,27 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestPartitioningWriters extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.ORC}, - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final long TARGET_FILE_SIZE = 128L * 1024 * 1024; - private final FileFormat fileFormat; - private OutputFileFactory fileFactory = null; + @Parameter(index = 1) + private FileFormat fileFormat; - public TestPartitioningWriters(FileFormat fileFormat) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - } + private OutputFileFactory fileFactory = null; protected abstract StructLikeSet toSet(Iterable records); @@ -70,9 +69,9 @@ protected FileFormat format() { } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -80,7 +79,7 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @Test + @TestTemplate public void testClusteredDataWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); ClusteredDataWriter writer = @@ -93,7 +92,7 @@ public void testClusteredDataWriterNoRecords() throws IOException { Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testClusteredDataWriterMultiplePartitions() throws IOException { table.updateSpec().addField(Expressions.ref("data")).commit(); @@ -124,7 +123,7 @@ public void testClusteredDataWriterMultiplePartitions() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testClusteredDataWriterOutOfOrderPartitions() throws IOException { table.updateSpec().addField(Expressions.ref("data")).commit(); @@ -149,7 +148,7 @@ public void testClusteredDataWriterOutOfOrderPartitions() throws IOException { writer.close(); } - @Test + @TestTemplate public void testClusteredEqualityDeleteWriterNoRecords() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -170,7 +169,7 @@ public void testClusteredEqualityDeleteWriterNoRecords() throws IOException { Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testClusteredEqualityDeleteWriterMultipleSpecs() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -236,7 +235,7 @@ public void testClusteredEqualityDeleteWriterMultipleSpecs() throws IOException Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testClusteredEqualityDeleteWriterOutOfOrderSpecsAndPartitions() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -280,12 +279,12 @@ public void testClusteredEqualityDeleteWriterOutOfOrderSpecsAndPartitions() thro writer.close(); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterNoRecordsPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterNoRecordsFileGranularity() throws IOException { checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity.FILE); } @@ -308,13 +307,13 @@ private void checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity delet Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterMultipleSpecsPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterMultipleSpecsFileGranularity() throws IOException { checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity.FILE); } @@ -387,13 +386,13 @@ private void checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity d Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitionsPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitionsFileGranularity() throws IOException { checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions(DeleteGranularity.FILE); @@ -455,12 +454,12 @@ private void checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions( writer.close(); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterPartitionGranularity() throws IOException { checkClusteredPositionDeleteWriterGranularity(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testClusteredPositionDeleteWriterFileGranularity() throws IOException { checkClusteredPositionDeleteWriterGranularity(DeleteGranularity.FILE); } @@ -510,7 +509,7 @@ private void checkClusteredPositionDeleteWriterGranularity(DeleteGranularity del assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } - @Test + @TestTemplate public void testFanoutDataWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); FanoutDataWriter writer = @@ -523,7 +522,7 @@ public void testFanoutDataWriterNoRecords() throws IOException { Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testFanoutDataWriterMultiplePartitions() throws IOException { table.updateSpec().addField(Expressions.ref("data")).commit(); @@ -554,12 +553,12 @@ public void testFanoutDataWriterMultiplePartitions() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterNoRecordsPartitionGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterNoRecordsFileGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity.FILE); } @@ -582,13 +581,13 @@ private void checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity dele Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterOutOfOrderRecordsPartitionGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterOutOfOrderRecordsFileGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords(DeleteGranularity.FILE); @@ -671,12 +670,12 @@ private void checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords( Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterPartitionGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity.PARTITION); } - @Test + @TestTemplate public void testFanoutPositionOnlyDeleteWriterFileGranularity() throws IOException { checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity.FILE); } 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 aa1118306a4c..2de7bf7324c0 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java @@ -20,43 +20,42 @@ 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; import org.apache.iceberg.DeleteFile; 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.RowDelta; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.StructLikeSet; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestPositionDeltaWriters extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.PARQUET} - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final long TARGET_FILE_SIZE = 128L * 1024 * 1024; - private final FileFormat fileFormat; - private OutputFileFactory fileFactory = null; + @Parameter(index = 1) + private FileFormat fileFormat; - public TestPositionDeltaWriters(FileFormat fileFormat) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - } + private OutputFileFactory fileFactory = null; protected abstract StructLikeSet toSet(Iterable records); @@ -65,9 +64,9 @@ protected FileFormat format() { } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -75,7 +74,7 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @Test + @TestTemplate public void testPositionDeltaWithOneDataWriter() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -100,7 +99,7 @@ public void testPositionDeltaWithOneDataWriter() throws IOException { Assert.assertEquals("Must not reference data files", 0, referencedDataFiles.length); } - @Test + @TestTemplate public void testPositionDeltaInsertOnly() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -136,7 +135,7 @@ public void testPositionDeltaInsertOnly() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testPositionDeltaDeleteOnly() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); @@ -192,7 +191,7 @@ public void testPositionDeltaDeleteOnly() throws IOException { Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); } - @Test + @TestTemplate public void testPositionDeltaMultipleSpecs() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); 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 b229d4871f55..f6d5d41b72b3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java @@ -20,8 +20,13 @@ 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; +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.StructLike; @@ -29,51 +34,46 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public abstract class TestRollingFileWriters extends WriterTestBase { - @Parameterized.Parameters(name = "FileFormat={0}, Partitioned={1}") - public static Object[] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO, false}, - new Object[] {FileFormat.AVRO, true}, - new Object[] {FileFormat.PARQUET, false}, - new Object[] {FileFormat.PARQUET, true}, - new Object[] {FileFormat.ORC, false}, - new Object[] {FileFormat.ORC, true} - }; + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, Partitioned = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO, false}, + new Object[] {2, FileFormat.AVRO, true}, + new Object[] {2, FileFormat.PARQUET, false}, + new Object[] {2, FileFormat.PARQUET, true}, + new Object[] {2, FileFormat.ORC, false}, + new Object[] {2, FileFormat.ORC, true}); } - private static final int TABLE_FORMAT_VERSION = 2; private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 1000; private static final long DEFAULT_FILE_SIZE = 128L * 1024 * 1024; private static final long SMALL_FILE_SIZE = 2L; private static final String PARTITION_VALUE = "aaa"; - private final FileFormat fileFormat; - private final boolean partitioned; + @Parameter(index = 1) + private FileFormat fileFormat; + + @Parameter(index = 2) + private boolean partitioned; + private StructLike partition = null; private OutputFileFactory fileFactory = null; - public TestRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(TABLE_FORMAT_VERSION); - this.fileFormat = fileFormat; - this.partitioned = partitioned; - } - protected FileFormat format() { return fileFormat; } @Override - @Before + @BeforeEach public void setupTable() throws Exception { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -89,7 +89,7 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @Test + @TestTemplate public void testRollingDataWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingDataWriter writer = @@ -103,7 +103,7 @@ public void testRollingDataWriterNoRecords() throws IOException { Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testRollingDataWriterSplitData() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingDataWriter writer = @@ -125,7 +125,7 @@ public void testRollingDataWriterSplitData() throws IOException { Assert.assertEquals(4, writer.result().dataFiles().size()); } - @Test + @TestTemplate public void testRollingEqualityDeleteWriterNoRecords() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -146,7 +146,7 @@ public void testRollingEqualityDeleteWriterNoRecords() throws IOException { Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testRollingEqualityDeleteWriterSplitDeletes() throws IOException { List equalityFieldIds = ImmutableList.of(table.schema().findField("id").fieldId()); Schema equalityDeleteRowSchema = table.schema().select("id"); @@ -174,7 +174,7 @@ public void testRollingEqualityDeleteWriterSplitDeletes() throws IOException { Assert.assertFalse(result.referencesDataFiles()); } - @Test + @TestTemplate public void testRollingPositionDeleteWriterNoRecords() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingPositionDeleteWriter writer = @@ -192,7 +192,7 @@ public void testRollingPositionDeleteWriterNoRecords() throws IOException { Assert.assertFalse(writer.result().referencesDataFiles()); } - @Test + @TestTemplate public void testRollingPositionDeleteWriterSplitDeletes() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table.schema()); RollingPositionDeleteWriter writer = diff --git a/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java b/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java index b8e99515598a..52e052b711e5 100644 --- a/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java +++ b/data/src/test/java/org/apache/iceberg/io/WriterTestBase.java @@ -25,7 +25,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.IcebergGenerics; import org.apache.iceberg.data.Record; @@ -33,11 +33,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.util.StructLikeSet; -public abstract class WriterTestBase extends TableTestBase { - - public WriterTestBase(int formatVersion) { - super(formatVersion); - } +public abstract class WriterTestBase extends TestBase { protected abstract FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 4ecbd1c12921..f780c6135bee 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -45,12 +45,15 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; 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.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.TestTables; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -65,31 +68,28 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -170,13 +170,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitioned() throws IOException { createAndInitTable(false); testCdcEvents(false); } - @Test + @TestTemplate public void testPartitioned() throws IOException { createAndInitTable(true); testCdcEvents(true); @@ -201,12 +201,12 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(false); } - @Test + @TestTemplate public void testPartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(true); } @@ -243,17 +243,17 @@ private void testAbort(boolean partitioned) throws IOException { } } - @Test + @TestTemplate public void testUnpartitionedAbort() throws IOException { testAbort(false); } - @Test + @TestTemplate public void testPartitionedAbort() throws IOException { testAbort(true); } - @Test + @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); @@ -298,7 +298,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); @@ -323,7 +323,7 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } - @Test + @TestTemplate public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { Schema tableSchema = new Schema( diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java index da45241256f5..414ee40d1357 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index 3951c2e70f65..f79a3e634071 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java index 9e846efe6fc9..3050752d1c24 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java index 07716b9c3e60..03051b69cf87 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.FileWriterFactory; @@ -29,10 +28,6 @@ public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index a2fe092b7cf6..4459f224afe6 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableMap; @@ -56,10 +57,13 @@ import org.apache.iceberg.GenericManifestFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -73,44 +77,39 @@ import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { private static final Configuration CONF = new Configuration(); private File flinkManifestFolder; - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); + flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -125,7 +124,7 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -156,7 +155,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { } } - @Test + @TestTemplate public void testMaxContinuousEmptyCommits() throws Exception { table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); @@ -182,7 +181,7 @@ private WriteResult of(DataFile dataFile) { return WriteResult.builder().addDataFiles(dataFile).build(); } - @Test + @TestTemplate public void testCommitTxn() throws Exception { // Test with 3 continues checkpoints: // 1. snapshotState for checkpoint#1 @@ -225,7 +224,7 @@ public void testCommitTxn() throws Exception { } } - @Test + @TestTemplate public void testOrderedEventsBetweenCheckpoints() throws Exception { // It's possible that two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -278,7 +277,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testDisorderedEventsBetweenCheckpoints() throws Exception { // It's possible that the two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -331,7 +330,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -392,7 +391,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's // possible that we @@ -490,7 +489,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except } } - @Test + @TestTemplate public void testStartAnotherJobToWriteSameTable() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -557,7 +556,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleJobsWriteSameTable() throws Exception { long timestamp = 0; List tableRows = Lists.newArrayList(); @@ -595,7 +594,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -693,7 +692,7 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; @@ -722,7 +721,7 @@ public void testBoundedStream() throws Exception { } } - @Test + @TestTemplate public void testFlinkManifests() throws Exception { long timestamp = 0; final long checkpoint = 10; @@ -766,7 +765,7 @@ public void testFlinkManifests() throws Exception { } } - @Test + @TestTemplate public void testDeleteFiles() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -835,7 +834,7 @@ public void testDeleteFiles() throws Exception { } } - @Test + @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -888,7 +887,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - @Test + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; int checkpointId = 0; diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 6d1891baf538..a612617835b0 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -22,7 +22,9 @@ 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; import java.util.concurrent.CountDownLatch; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -33,9 +35,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -50,13 +54,12 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { private static final Schema SCHEMA = new Schema( @@ -65,19 +68,15 @@ public class TestStreamingMonitorFunction extends TableTestBase { private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; private static final long WAIT_TIME_MILLIS = 10 * 1000L; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -99,7 +98,7 @@ private void runSourceFunctionInTask( task.start(); } - @Test + @TestTemplate public void testConsumeWithoutStartSnapshotId() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -122,7 +121,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartSnapshotId() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -155,7 +154,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartTag() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -187,7 +186,7 @@ public void testConsumeFromStartTag() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -243,7 +242,7 @@ private void awaitExpectedSplits(TestSourceContext sourceContext) { }); } - @Test + @TestTemplate public void testInvalidMaxPlanningSnapshotCount() { ScanContext scanContext1 = ScanContext.builder() @@ -265,7 +264,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .hasMessage("The max-planning-snapshot-count must be greater than zero"); } - @Test + @TestTemplate public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { generateRecordsAndCommitTxn(10); diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index e258a197edf3..f96426a59a2d 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -20,6 +20,8 @@ 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 org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -32,28 +34,28 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.TestTableLoader; -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.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { private static final Schema SCHEMA = new Schema( @@ -61,19 +63,15 @@ public class TestStreamingReaderOperator extends TableTestBase { Types.NestedField.required(2, "data", Types.StringType.get())); private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -81,7 +79,7 @@ public void setupTable() throws IOException { table = create(SCHEMA, PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); @@ -109,7 +107,7 @@ public void testProcessAllRecords() throws Exception { } } - @Test + @TestTemplate public void testTriggerCheckpoint() throws Exception { // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading // records from @@ -150,7 +148,7 @@ public void testTriggerCheckpoint() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index 4ecbd1c12921..f780c6135bee 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -45,12 +45,15 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; 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.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.TestTables; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -65,31 +68,28 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -170,13 +170,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitioned() throws IOException { createAndInitTable(false); testCdcEvents(false); } - @Test + @TestTemplate public void testPartitioned() throws IOException { createAndInitTable(true); testCdcEvents(true); @@ -201,12 +201,12 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(false); } - @Test + @TestTemplate public void testPartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(true); } @@ -243,17 +243,17 @@ private void testAbort(boolean partitioned) throws IOException { } } - @Test + @TestTemplate public void testUnpartitionedAbort() throws IOException { testAbort(false); } - @Test + @TestTemplate public void testPartitionedAbort() throws IOException { testAbort(true); } - @Test + @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); @@ -298,7 +298,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); @@ -323,7 +323,7 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } - @Test + @TestTemplate public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { Schema tableSchema = new Schema( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java index da45241256f5..414ee40d1357 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index 3951c2e70f65..f79a3e634071 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java index 9e846efe6fc9..3050752d1c24 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java index 07716b9c3e60..03051b69cf87 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.FileWriterFactory; @@ -29,10 +28,6 @@ public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index a2fe092b7cf6..4459f224afe6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableMap; @@ -56,10 +57,13 @@ import org.apache.iceberg.GenericManifestFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -73,44 +77,39 @@ import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { private static final Configuration CONF = new Configuration(); private File flinkManifestFolder; - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); + flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -125,7 +124,7 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -156,7 +155,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { } } - @Test + @TestTemplate public void testMaxContinuousEmptyCommits() throws Exception { table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); @@ -182,7 +181,7 @@ private WriteResult of(DataFile dataFile) { return WriteResult.builder().addDataFiles(dataFile).build(); } - @Test + @TestTemplate public void testCommitTxn() throws Exception { // Test with 3 continues checkpoints: // 1. snapshotState for checkpoint#1 @@ -225,7 +224,7 @@ public void testCommitTxn() throws Exception { } } - @Test + @TestTemplate public void testOrderedEventsBetweenCheckpoints() throws Exception { // It's possible that two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -278,7 +277,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testDisorderedEventsBetweenCheckpoints() throws Exception { // It's possible that the two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -331,7 +330,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -392,7 +391,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's // possible that we @@ -490,7 +489,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except } } - @Test + @TestTemplate public void testStartAnotherJobToWriteSameTable() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -557,7 +556,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleJobsWriteSameTable() throws Exception { long timestamp = 0; List tableRows = Lists.newArrayList(); @@ -595,7 +594,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -693,7 +692,7 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; @@ -722,7 +721,7 @@ public void testBoundedStream() throws Exception { } } - @Test + @TestTemplate public void testFlinkManifests() throws Exception { long timestamp = 0; final long checkpoint = 10; @@ -766,7 +765,7 @@ public void testFlinkManifests() throws Exception { } } - @Test + @TestTemplate public void testDeleteFiles() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -835,7 +834,7 @@ public void testDeleteFiles() throws Exception { } } - @Test + @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -888,7 +887,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - @Test + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; int checkpointId = 0; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 494c633088d9..451e3552b9ac 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -22,7 +22,9 @@ 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; import java.util.concurrent.CountDownLatch; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -33,9 +35,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -50,13 +54,12 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { private static final Schema SCHEMA = new Schema( @@ -65,19 +68,15 @@ public class TestStreamingMonitorFunction extends TableTestBase { private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; private static final long WAIT_TIME_MILLIS = 10 * 1000L; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -99,7 +98,7 @@ private void runSourceFunctionInTask( task.start(); } - @Test + @TestTemplate public void testConsumeWithoutStartSnapshotId() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -122,7 +121,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartSnapshotId() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -155,7 +154,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartTag() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -187,7 +186,7 @@ public void testConsumeFromStartTag() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -243,7 +242,7 @@ private void awaitExpectedSplits(TestSourceContext sourceContext) { }); } - @Test + @TestTemplate public void testInvalidMaxPlanningSnapshotCount() { ScanContext scanContext1 = ScanContext.builder() @@ -266,7 +265,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .hasMessage("The max-planning-snapshot-count must be greater than zero"); } - @Test + @TestTemplate public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { generateRecordsAndCommitTxn(10); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index e258a197edf3..f96426a59a2d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -20,6 +20,8 @@ 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 org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -32,28 +34,28 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.TestTableLoader; -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.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { private static final Schema SCHEMA = new Schema( @@ -61,19 +63,15 @@ public class TestStreamingReaderOperator extends TableTestBase { Types.NestedField.required(2, "data", Types.StringType.get())); private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -81,7 +79,7 @@ public void setupTable() throws IOException { table = create(SCHEMA, PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); @@ -109,7 +107,7 @@ public void testProcessAllRecords() throws Exception { } } - @Test + @TestTemplate public void testTriggerCheckpoint() throws Exception { // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading // records from @@ -150,7 +148,7 @@ public void testTriggerCheckpoint() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); 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 4ecbd1c12921..f780c6135bee 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 @@ -45,12 +45,15 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileContent; 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.RowDelta; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.TestTables; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; @@ -65,31 +68,28 @@ import org.apache.iceberg.util.StructLikeSet; import org.assertj.core.api.Assertions; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestDeltaTaskWriter extends TableTestBase { - private static final int FORMAT_V2 = 2; - - private final FileFormat format; - - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{"avro"}, {"orc"}, {"parquet"}}; - } - - public TestDeltaTaskWriter(String fileFormat) { - super(FORMAT_V2); - this.format = FileFormat.fromString(fileFormat); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); Assert.assertTrue(tableDir.delete()); // created by table create this.metadataDir = new File(tableDir, "metadata"); @@ -170,13 +170,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitioned() throws IOException { createAndInitTable(false); testCdcEvents(false); } - @Test + @TestTemplate public void testPartitioned() throws IOException { createAndInitTable(true); testCdcEvents(true); @@ -201,12 +201,12 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); } - @Test + @TestTemplate public void testUnpartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(false); } - @Test + @TestTemplate public void testPartitionedPureEqDeletes() throws IOException { testWritePureEqDeletes(true); } @@ -243,17 +243,17 @@ private void testAbort(boolean partitioned) throws IOException { } } - @Test + @TestTemplate public void testUnpartitionedAbort() throws IOException { testAbort(false); } - @Test + @TestTemplate public void testPartitionedAbort() throws IOException { testAbort(true); } - @Test + @TestTemplate public void testPartitionedTableWithDataAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId()); @@ -298,7 +298,7 @@ public void testPartitionedTableWithDataAsKey() throws IOException { actualRowSet("*")); } - @Test + @TestTemplate public void testPartitionedTableWithDataAndIdAsKey() throws IOException { createAndInitTable(true); List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); @@ -323,7 +323,7 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); } - @Test + @TestTemplate public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { Schema tableSchema = new Schema( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java index da45241256f5..414ee40d1357 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - public TestFlinkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java index 3951c2e70f65..939ed2be7dbc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Arrays; import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,8 +35,12 @@ public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - public TestFlinkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); } @Override diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java index 9e846efe6fc9..3050752d1c24 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -21,7 +21,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; @@ -33,10 +32,6 @@ public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestFlinkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java index 07716b9c3e60..03051b69cf87 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -20,7 +20,6 @@ import java.util.List; import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.FileWriterFactory; @@ -29,10 +28,6 @@ public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - public TestFlinkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, 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 a2fe092b7cf6..4459f224afe6 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 @@ -27,6 +27,7 @@ import java.io.IOException; import java.nio.file.Files; import java.nio.file.Path; +import java.util.Arrays; import java.util.Collections; import java.util.List; import java.util.NavigableMap; @@ -56,10 +57,13 @@ import org.apache.iceberg.GenericManifestFile; import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionData; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestHelpers; @@ -73,44 +77,39 @@ import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; import org.junit.Assume; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergFilesCommitter extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { private static final Configuration CONF = new Configuration(); private File flinkManifestFolder; - private final FileFormat format; - private final String branch; - - @Parameterized.Parameters(name = "FileFormat = {0}, FormatVersion = {1}, branch = {2}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, "main"}, - new Object[] {"avro", 2, "test-branch"}, - new Object[] {"parquet", 1, "main"}, - new Object[] {"parquet", 2, "test-branch"}, - new Object[] {"orc", 1, "main"}, - new Object[] {"orc", 2, "test-branch"} - }; - } - - public TestIcebergFilesCommitter(String format, int formatVersion, String branch) { - super(formatVersion); - this.format = FileFormat.fromString(format); - this.branch = branch; + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); } @Override - @Before + @BeforeEach public void setupTable() throws IOException { - flinkManifestFolder = temp.newFolder(); + flinkManifestFolder = Files.createTempDirectory(temp, "flink").toFile(); - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -125,7 +124,7 @@ public void setupTable() throws IOException { .commit(); } - @Test + @TestTemplate public void testCommitTxnWithoutDataFiles() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -156,7 +155,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { } } - @Test + @TestTemplate public void testMaxContinuousEmptyCommits() throws Exception { table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); @@ -182,7 +181,7 @@ private WriteResult of(DataFile dataFile) { return WriteResult.builder().addDataFiles(dataFile).build(); } - @Test + @TestTemplate public void testCommitTxn() throws Exception { // Test with 3 continues checkpoints: // 1. snapshotState for checkpoint#1 @@ -225,7 +224,7 @@ public void testCommitTxn() throws Exception { } } - @Test + @TestTemplate public void testOrderedEventsBetweenCheckpoints() throws Exception { // It's possible that two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -278,7 +277,7 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testDisorderedEventsBetweenCheckpoints() throws Exception { // It's possible that the two checkpoints happen in the following orders: // 1. snapshotState for checkpoint#1; @@ -331,7 +330,7 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -392,7 +391,7 @@ public void testRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's // possible that we @@ -490,7 +489,7 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except } } - @Test + @TestTemplate public void testStartAnotherJobToWriteSameTable() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -557,7 +556,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleJobsWriteSameTable() throws Exception { long timestamp = 0; List tableRows = Lists.newArrayList(); @@ -595,7 +594,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { } } - @Test + @TestTemplate public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { long checkpointId = 0; long timestamp = 0; @@ -693,7 +692,7 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } } - @Test + @TestTemplate public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; @@ -722,7 +721,7 @@ public void testBoundedStream() throws Exception { } } - @Test + @TestTemplate public void testFlinkManifests() throws Exception { long timestamp = 0; final long checkpoint = 10; @@ -766,7 +765,7 @@ public void testFlinkManifests() throws Exception { } } - @Test + @TestTemplate public void testDeleteFiles() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -835,7 +834,7 @@ public void testDeleteFiles() throws Exception { } } - @Test + @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); @@ -888,7 +887,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } - @Test + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; int checkpointId = 0; 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 494c633088d9..451e3552b9ac 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 @@ -22,7 +22,9 @@ 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; import java.util.concurrent.CountDownLatch; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -33,9 +35,11 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; @@ -50,13 +54,12 @@ import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingMonitorFunction extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { private static final Schema SCHEMA = new Schema( @@ -65,19 +68,15 @@ public class TestStreamingMonitorFunction extends TableTestBase { private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; private static final long WAIT_TIME_MILLIS = 10 * 1000L; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingMonitorFunction(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -99,7 +98,7 @@ private void runSourceFunctionInTask( task.start(); } - @Test + @TestTemplate public void testConsumeWithoutStartSnapshotId() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -122,7 +121,7 @@ public void testConsumeWithoutStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartSnapshotId() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -155,7 +154,7 @@ public void testConsumeFromStartSnapshotId() throws Exception { } } - @Test + @TestTemplate public void testConsumeFromStartTag() throws Exception { // Commit the first five transactions. generateRecordsAndCommitTxn(5); @@ -187,7 +186,7 @@ public void testConsumeFromStartTag() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> recordsList = generateRecordsAndCommitTxn(10); ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); @@ -243,7 +242,7 @@ private void awaitExpectedSplits(TestSourceContext sourceContext) { }); } - @Test + @TestTemplate public void testInvalidMaxPlanningSnapshotCount() { ScanContext scanContext1 = ScanContext.builder() @@ -266,7 +265,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .hasMessage("The max-planning-snapshot-count must be greater than zero"); } - @Test + @TestTemplate public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { generateRecordsAndCommitTxn(10); 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 e258a197edf3..f96426a59a2d 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 @@ -20,6 +20,8 @@ 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 org.apache.flink.runtime.checkpoint.OperatorSubtaskState; @@ -32,28 +34,28 @@ import org.apache.flink.table.data.RowData; import org.apache.flink.types.Row; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.TableTestBase; +import org.apache.iceberg.TestBase; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.TestTableLoader; -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.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestStreamingReaderOperator extends TableTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { private static final Schema SCHEMA = new Schema( @@ -61,19 +63,15 @@ public class TestStreamingReaderOperator extends TableTestBase { Types.NestedField.required(2, "data", Types.StringType.get())); private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - @Parameterized.Parameters(name = "FormatVersion={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {1}, new Object[] {2}); + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); } - public TestStreamingReaderOperator(int formatVersion) { - super(formatVersion); - } - - @Before + @BeforeEach @Override public void setupTable() throws IOException { - this.tableDir = temp.newFolder(); + this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); Assert.assertTrue(tableDir.delete()); @@ -81,7 +79,7 @@ public void setupTable() throws IOException { table = create(SCHEMA, PartitionSpec.unpartitioned()); } - @Test + @TestTemplate public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); @@ -109,7 +107,7 @@ public void testProcessAllRecords() throws Exception { } } - @Test + @TestTemplate public void testTriggerCheckpoint() throws Exception { // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading // records from @@ -150,7 +148,7 @@ public void testTriggerCheckpoint() throws Exception { } } - @Test + @TestTemplate public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 4a3263e368c0..8f4b3042b1c5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -33,10 +32,6 @@ public class TestSparkFileWriterFactory extends TestFileWriterFactory { - public TestSparkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 276d8c632fc0..5e8107806a25 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPartitioningWriters; @@ -33,10 +32,6 @@ public class TestSparkPartitioningWriters extends TestPartitioningWriters { - public TestSparkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index 245c392774f5..baac1efe0e40 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPositionDeltaWriters; @@ -33,10 +32,6 @@ public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestSparkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java index dcf9140a8885..5ebeafcb8cef 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestRollingFileWriters; @@ -30,10 +29,6 @@ public class TestSparkRollingFileWriters extends TestRollingFileWriters { - public TestSparkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 4a3263e368c0..8f4b3042b1c5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -33,10 +32,6 @@ public class TestSparkFileWriterFactory extends TestFileWriterFactory { - public TestSparkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 276d8c632fc0..5e8107806a25 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPartitioningWriters; @@ -33,10 +32,6 @@ public class TestSparkPartitioningWriters extends TestPartitioningWriters { - public TestSparkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index 245c392774f5..baac1efe0e40 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPositionDeltaWriters; @@ -33,10 +32,6 @@ public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestSparkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java index dcf9140a8885..5ebeafcb8cef 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestRollingFileWriters; @@ -30,10 +29,6 @@ public class TestSparkRollingFileWriters extends TestRollingFileWriters { - public TestSparkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 4a3263e368c0..8f4b3042b1c5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestFileWriterFactory; @@ -33,10 +32,6 @@ public class TestSparkFileWriterFactory extends TestFileWriterFactory { - public TestSparkFileWriterFactory(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 276d8c632fc0..5e8107806a25 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPartitioningWriters; @@ -33,10 +32,6 @@ public class TestSparkPartitioningWriters extends TestPartitioningWriters { - public TestSparkPartitioningWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index 245c392774f5..baac1efe0e40 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestPositionDeltaWriters; @@ -33,10 +32,6 @@ public class TestSparkPositionDeltaWriters extends TestPositionDeltaWriters { - public TestSparkPositionDeltaWriters(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java index dcf9140a8885..5ebeafcb8cef 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkRollingFileWriters.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.io.FileWriterFactory; import org.apache.iceberg.io.TestRollingFileWriters; @@ -30,10 +29,6 @@ public class TestSparkRollingFileWriters extends TestRollingFileWriters { - public TestSparkRollingFileWriters(FileFormat fileFormat, boolean partitioned) { - super(fileFormat, partitioned); - } - @Override protected FileWriterFactory newWriterFactory( Schema dataSchema, From d21f721b1fe2229811e104fdb87993cd1200cac1 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 3 Apr 2024 09:59:13 +0200 Subject: [PATCH 0190/1019] Build: Ignore link-checking for Blogs / https://search.maven.org/ (#10081) --- site/docs/blogs.md | 71 +++++++++++++++++++++++++++++++++++ site/link-checker-config.json | 3 ++ 2 files changed, 74 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index e7cda2f997a6..4d327596129e 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -28,16 +28,19 @@ 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/) + ### [Data Lakehouse Versioning Comparison: (Nessie, Apache Iceberg, LakeFS)](https://www.dremio.com/blog/data-lakehouse-versioning-comparison-nessie-apache-iceberg-lakefs/) **Date**: March 5th, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [What is Lakehouse Management?: Git-for-Data, Automated Apache Iceberg Table Maintenance and more](https://www.dremio.com/blog/what-is-lakehouse-management-git-for-data-automated-apache-iceberg-table-maintenance-and-more/) **Date**: February 23rd, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [What is DataOps? Automating Data Management on the Apache Iceberg Lakehouse](https://www.dremio.com/blog/what-is-dataops-automating-data-management-on-the-apache-iceberg-lakehouse/) **Date**: February 23rd, 2024, **Company**: Dremio @@ -49,11 +52,13 @@ 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/) + ### [Ingesting Data Into Apache Iceberg Tables with Dremio: A Unified Path to Iceberg](https://www.dremio.com/blog/ingesting-data-into-apache-iceberg-tables-with-dremio-a-unified-path-to-iceberg/) **Date**: February 1st, 2024, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Open Source and the Data Lakehouse: Apache Arrow, Apache Iceberg, Nessie and Dremio](https://www.dremio.com/blog/open-source-and-the-data-lakehouse-apache-arrow-apache-iceberg-nessie-and-dremio/) **Date**: February 1st, 2024, **Company**: Dremio @@ -77,26 +82,31 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ayush Saxena](https://www.linkedin.com/in/ayush151/) + ### [Getting Started with Flink SQL and Apache Iceberg](https://www.dremio.com/blog/getting-started-with-flink-sql-and-apache-iceberg/) **Date**: August 8th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [Using Flink with Apache Iceberg and Nessie](https://www.dremio.com/blog/using-flink-with-apache-iceberg-and-nessie/) **Date**: July 28th, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [From Hive Tables to Iceberg Tables: Hassle-Free](https://blog.cloudera.com/from-hive-tables-to-iceberg-tables-hassle-free/) **Date**: July 14th, 2023, **Company**: Cloudera **Authors**: [Srinivas Rishindra Pothireddi](https://www.linkedin.com/in/srinivas-rishindra/) + ### [From Hive Tables to Iceberg Tables: Hassle-Free](https://blog.cloudera.com/from-hive-tables-to-iceberg-tables-hassle-free/) **Date**: July 14th, 2023, **Company**: Cloudera **Authors**: [Srinivas Rishindra Pothireddi](https://www.linkedin.com/in/srinivas-rishindra/) + ### [12 Times Faster Query Planning With Iceberg Manifest Caching in Impala](https://blog.cloudera.com/12-times-faster-query-planning-with-iceberg-manifest-caching-in-impala/) **Date**: July 13th, 2023, **Company**: Cloudera @@ -108,46 +118,55 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Rui Li](https://www.linkedin.com/in/rui-li-19282979/) + ### [How to Convert JSON Files Into an Apache Iceberg Table with Dremio](https://www.dremio.com/blog/how-to-convert-json-files-into-an-apache-iceberg-table-with-dremio/) **Date**: May 31st, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Deep Dive Into Configuring Your Apache Iceberg Catalog with Apache Spark](https://www.dremio.com/blog/deep-dive-into-configuring-your-apache-iceberg-catalog-with-apache-spark/) **Date**: May 31st, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Streamlining Data Quality in Apache Iceberg with write-audit-publish & branching](https://www.dremio.com/blog/streamlining-data-quality-in-apache-iceberg-with-write-audit-publish-branching/) **Date**: May 19th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [Introducing the Apache Iceberg Catalog Migration Tool](https://www.dremio.com/blog/introducing-the-apache-iceberg-catalog-migration-tool/) **Date**: May 12th, 2023, **Company**: Dremio **Authors**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) & [Ajantha Bhat](https://www.linkedin.com/in/ajanthabhat/) + ### [3 Ways to Use Python with Apache Iceberg](https://www.dremio.com/blog/3-ways-to-use-python-with-apache-iceberg/) **Date**: April 12th, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [3 Ways to Convert a Delta Lake Table Into an Apache Iceberg Table](https://www.dremio.com/blog/3-ways-to-convert-a-delta-lake-table-into-an-apache-iceberg-table/) **Date**: April 3rd, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How to Convert CSV Files into an Apache Iceberg table with Dremio](https://www.dremio.com/blog/how-to-convert-csv-files-into-an-apache-iceberg-table-with-dremio/) **Date**: April 3rd, 2023, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Open Data Lakehouse powered by Iceberg for all your Data Warehouse needs](https://blog.cloudera.com/open-data-lakehouse-powered-by-iceberg-for-all-your-data-warehouse-needs/) **Date**: April 3rd, 2023, **Company**: Cloudera **Authors**: [Zoltan Borok-Nagy](https://www.linkedin.com/in/zoltán-borók-nagy-7370a65b/), [Ayush Saxena](https://www.linkedin.com/in/ayush-saxena151/), [Tamas Mate](https://www.linkedin.com/in/tmater/), [Simhadri Govindappa](https://www.linkedin.com/in/simhadri-govindappa-1a7788148/) + ### [Exploring Branch & Tags in Apache Iceberg using Spark](https://www.dremio.com/blog/exploring-branch-tags-in-apache-iceberg-using-spark/) **Date**: March 29th, 2022, **Company**: Dremio @@ -159,11 +178,13 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ron Ortloff](https://www.linkedin.com/in/ron-ortloff/), [Dennis Huo](https://www.linkedin.com/in/dennis-huo-2aaba92a/) + ### [Dealing with Data Incidents Using the Rollback Feature in Apache Iceberg](https://www.dremio.com/blog/dealing-with-data-incidents-using-the-rollback-feature-in-apache-iceberg/) **Date**: February 24th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Partition and File Pruning for Dremio’s Apache Iceberg-backed Reflections](https://www.dremio.com/blog/partition-and-file-pruning-for-dremios-apache-iceberg-backed-reflections/) **Date**: February 8th, 2022, **Company**: Dremio @@ -193,181 +214,217 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Sumeet Tandure](https://www.linkedin.com/in/sumeettandure/) + ### [Multi-Cloud Open Lakehouse with Apache Iceberg in Cloudera Data Platform](https://blog.cloudera.com/implement-a-multi-cloud-open-lakehouse-with-apache-iceberg-in-cloudera-data-platform/) **Date**: December 15th, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Shaun Ahmadian](https://www.linkedin.com/in/ssahmadian/), [Zoltan Borok-Nagy](https://www.linkedin.com/in/zoltán-borók-nagy-7370a65b/), [Vincent Kulandaisamy](https://www.linkedin.com/in/vincentkulandaisamy/) + ### [Connecting Tableau to Apache Iceberg Tables with Dremio](https://www.dremio.com/blog/connecting-tableau-to-apache-iceberg-tables-with-dremio/) **Date**: December 15th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Getting Started with Project Nessie, Apache Iceberg, and Apache Spark Using Docker](https://www.dremio.com/blog/getting-started-with-project-nessie-apache-iceberg-and-apache-spark-using-docker/) **Date**: December 15th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Apache Iceberg FAQ](https://www.dremio.com/blog/apache-iceberg-faq/) **Date**: December 14th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [A Notebook for getting started with Project Nessie, Apache Iceberg, and Apache Spark](https://www.dremio.com/blog/a-notebook-for-getting-started-with-project-nessie-apache-iceberg-and-apache-spark/) **Date**: December 5th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Time Travel with Dremio and Apache Iceberg](https://www.dremio.com/blog/time-travel-with-dremio-and-apache-iceberg/) **Date**: November 29th, 2022, **Company**: Dremio **Author**: [Michael Flower](https://www.linkedin.com/in/michael-flower-b0a3474/) + ### [Compaction in Apache Iceberg: Fine-Tuning Your Iceberg Table's Data Files](https://www.dremio.com/subsurface/compaction-in-apache-iceberg-fine-tuning-your-iceberg-tables-data-files/) **Date**: November 9th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [The Life of a Read Query for Apache Iceberg Tables](https://www.dremio.com/subsurface/the-life-of-a-read-query-for-apache-iceberg-tables/) **Date**: October 31st, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Puffins and Icebergs: Additional Stats for Apache Iceberg Tables](https://www.dremio.com/subsurface/puffins-and-icebergs-additional-stats-for-apache-iceberg-tables/) **Date**: October 17th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Apache Iceberg and the Right to be Forgotten](https://www.dremio.com/subsurface/apache-iceberg-and-the-right-to-be-forgotten/) **Date**: September 30th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Streaming Data into Apache Iceberg tables using AWS Kinesis and AWS Glue](https://www.dremio.com/subsurface/streaming-data-into-apache-iceberg-tables-using-aws-kinesis-and-aws-glue/) **Date**: September 26th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Iceberg Flink Sink: Stream Directly into your Data Warehouse Tables](https://tabular.io/blog/flink-sink/) **Date**: October 12, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Partitioning for Correctness (and Performance)](https://tabular.io/blog/partitioning/) **Date**: September 28, 2022, **Company**: Tabular **Author**: [Jason Reid](https://www.linkedin.com/in/jasonreid/) + ### [Ensuring High Performance at Any Scale with Apache Iceberg’s Object Store File Layout](https://www.dremio.com/subsurface/ensuring-high-performance-at-any-scale-with-apache-icebergs-object-store-file-layout/) **Date**: September 20, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Introduction to Apache Iceberg Using Spark](https://www.dremio.com/subsurface/introduction-to-apache-iceberg-using-spark/) **Date**: September 15, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How Z-Ordering in Apache Iceberg Helps Improve Performance](https://www.dremio.com/subsurface/how-z-ordering-in-apache-iceberg-helps-improve-performance/) **Date**: September 13th, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Apache Iceberg 101 – Your Guide to Learning Apache Iceberg Concepts and Practices](https://www.dremio.com/subsurface/apache-iceberg-101-your-guide-to-learning-apache-iceberg-concepts-and-practices/) **Date**: September 12th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [A Hands-On Look at the Structure of an Apache Iceberg Table](https://www.dremio.com/subsurface/a-hands-on-look-at-the-structure-of-an-apache-iceberg-table/) **Date**: August 24, 2022, **Company**: Dremio **Author**: [Dipankar Mazumdar](https://www.linkedin.com/in/dipankar-mazumdar/) + ### [Future-Proof Partitioning and Fewer Table Rewrites with Apache Iceberg](https://www.dremio.com/subsurface/future-proof-partitioning-and-fewer-table-rewrites-with-apache-iceberg/) **Date**: August 18, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [How to use Apache Iceberg in CDP's Open Lakehouse](https://blog.cloudera.com/how-to-use-apache-iceberg-in-cdps-open-lakehouse/) **Date**: August 8th, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Peter Ableda](https://www.linkedin.com/in/peterableda), [Shaun Ahmadian](https://www.linkedin.com/in/ssahmadian/), [Manish Maheshwari](https://www.linkedin.com/in/mmaheshwari/) + ### [Near Real-Time Ingestion For Trino](https://www.starburst.io/blog/near-real-time-ingestion-for-trino/) **Date**: August 4th, 2022, **Company**: Starburst **Authors**: [Eric Hwang](https://www.linkedin.com/in/ericwhwang), [Monica Miller](https://www.linkedin.com/in/monica-d-miller), [Brian Zhan](https://www.linkedin.com/in/bzhan) + ### [How to implement Apache Iceberg in AWS Athena](https://big-data-demystified.ninja/2022/07/28/how-to-implement-apache-iceberg-in-aws-athena/) **Date**: July 28th, 2022 **Author**: [Shneior Dicastro] + ### [Supercharge your Data Lakehouse with Apache Iceberg in Cloudera Data Platform](https://blog.cloudera.com/supercharge-your-data-lakehouse-with-apache-iceberg-in-cloudera-data-platform/) **Date**: June 30th, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Shaun Ahmadian](https://www.linkedin.com/in/ssahmadian/) + ### [Migrating a Hive Table to an Iceberg Table Hands-on Tutorial](https://www.dremio.com/subsurface/migrating-a-hive-table-to-an-iceberg-table-hands-on-tutorial/) **Date**: June 6th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Fewer Accidental Full Table Scans Brought to You by Apache Iceberg’s Hidden Partitioning](https://www.dremio.com/subsurface/fewer-accidental-full-table-scans-brought-to-you-by-apache-icebergs-hidden-partitioning/) **Date**: May 21st, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [An Introduction To The Iceberg Java API Part 2 - Table Scans](https://tabular.io/blog/java-api-part-2/) **Date**: May 11th, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Iceberg's Guiding Light: The Iceberg Open Table Format Specification](https://tabular.io/blog/iceberg-format-version/) **Date**: April 26th, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [How to Migrate a Hive Table to an Iceberg Table](https://www.dremio.com/subsurface/how-to-migrate-a-hive-table-to-an-iceberg-table/) **Date**: April 15th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [Using Iceberg's S3FileIO Implementation To Store Your Data In MinIO](https://tabular.io/blog/minio/) **Date**: April 14th, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Maintaining Iceberg Tables – Compaction, Expiring Snapshots, and More](https://www.dremio.com/subsurface/maintaining-iceberg-tables-compaction-expiring-snapshots-and-more/) **Date**: April 7th, 2022, **Company**: Dremio **Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [An Introduction To The Iceberg Java API - Part 1](https://tabular.io/blog/java-api-part-1/) **Date**: April 1st, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Integrated Audits: Streamlined Data Observability With Apache Iceberg](https://tabular.io/blog/integrated-audits/) **Date**: March 2nd, 2022, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Introducing Apache Iceberg in Cloudera Data Platform](https://blog.cloudera.com/introducing-apache-iceberg-in-cloudera-data-platform/) **Date**: February 23rd, 2022, **Company**: Cloudera **Authors**: [Bill Zhang](https://www.linkedin.com/in/billzhang01/), [Peter Vary](https://www.linkedin.com/in/peter-vary/), [Marton Bod](https://www.linkedin.com/in/martonbod/), [Wing Yew Poon](https://github.com/wypoon) + ### [What's new in Iceberg 0.13](https://tabular.io/blog/whats-new-in-iceberg-0.13/) **Date**: February 22nd, 2022, **Company**: Tabular **Author**: [Ryan Blue](https://www.linkedin.com/in/rdblue/) + ### [Apache Iceberg Becomes Industry Open Standard with Ecosystem Adoption](https://www.dremio.com/apache-iceberg-becomes-industry-open-standard-with-ecosystem-adoption/) **Date**: February 3rd, 2022, **Company**: Dremio **Author**: [Mark Lyons](https://www.linkedin.com/in/markclyons/) + ### [Docker, Spark, and Iceberg: The Fastest Way to Try Iceberg!](https://tabular.io/blog/docker-spark-and-iceberg/) **Date**: February 2nd, 2022, **Company**: Tabular @@ -379,31 +436,37 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [James Malone](https://www.linkedin.com/in/jamesamalone/) + ### [Iceberg FileIO: Cloud Native Tables](https://tabular.io/blog/iceberg-fileio/) **Date**: December 16th, 2021, **Company**: Tabular **Author**: [Daniel Weeks](https://www.linkedin.com/in/daniel-weeks-a1946860/) + ### [Using Spark in EMR with Apache Iceberg](https://tabular.io/blog/emr-spark-and-iceberg/) **Date**: December 10th, 2021, **Company**: Tabular **Author**: [Sam Redai](https://www.linkedin.com/in/sredai/) + ### [Metadata Indexing in Iceberg](https://tabular.io/blog/iceberg-metadata-indexing/) **Date**: October 10th, 2021, **Company**: Tabular **Author**: [Ryan Blue](https://www.linkedin.com/in/rdblue/) + ### [Using Debezium to Create a Data Lake with Apache Iceberg](https://debezium.io/blog/2021/10/20/using-debezium-create-data-lake-with-apache-iceberg/) **Date**: October 20th, 2021, **Company**: Memiiso Community **Author**: [Ismail Simsek](https://www.linkedin.com/in/ismailsimsek/) + ### [How to Analyze CDC Data in Iceberg Data Lake Using Flink](https://www.alibabacloud.com/blog/how-to-analyze-cdc-data-in-iceberg-data-lake-using-flink_597838) **Date**: June 15th, 2021, **Company**: Alibaba Cloud Community **Author**: [Li Jinsong](https://www.linkedin.com/in/%E5%8A%B2%E6%9D%BE-%E6%9D%8E-48b54b101/), [Hu Zheng](https://www.linkedin.com/in/zheng-hu-37017683/), [Yang Weihai](https://www.linkedin.com/in/weihai-yang-697a16224/), [Peidan Li](https://www.linkedin.com/in/peidian-li-18938820a/) + ### [Apache Iceberg: An Architectural Look Under the Covers](https://www.dremio.com/apache-iceberg-an-architectural-look-under-the-covers/) **Date**: July 6th, 2021, **Company**: Dremio @@ -415,26 +478,31 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Romin Parekh](https://www.linkedin.com/in/rominparekh/), [Miao Wang](https://www.linkedin.com/in/miao-wang-0406a74/), [Shone Sadler](https://www.linkedin.com/in/shonesadler/) + ### [Flink + Iceberg: How to Construct a Whole-scenario Real-time Data Warehouse](https://www.alibabacloud.com/blog/flink-%2B-iceberg-how-to-construct-a-whole-scenario-real-time-data-warehouse_597824) **Date**: Jun 8th, 2021, **Company**: Tencent **Author** [Shu (Simon Su) Su](https://www.linkedin.com/in/shu-su-62944994/) + ### [Trino on Ice III: Iceberg Concurrency Model, Snapshots, and the Iceberg Spec](https://blog.starburst.io/trino-on-ice-iii-iceberg-concurrency-model-snapshots-and-the-iceberg-spec) **Date**: May 25th, 2021, **Company**: Starburst **Author**: [Brian Olsen](https://www.linkedin.com/in/bitsondatadev) + ### [Trino on Ice II: In-Place Table Evolution and Cloud Compatibility with Iceberg](https://blog.starburst.io/trino-on-ice-ii-in-place-table-evolution-and-cloud-compatibility-with-iceberg) **Date**: May 11th, 2021, **Company**: Starburst **Author**: [Brian Olsen](https://www.linkedin.com/in/bitsondatadev) + ### [Trino On Ice I: A Gentle Introduction To Iceberg](https://blog.starburst.io/trino-on-ice-i-a-gentle-introduction-to-iceberg) **Date**: Apr 27th, 2021, **Company**: Starburst **Author**: [Brian Olsen](https://www.linkedin.com/in/bitsondatadev) + ### [Apache Iceberg: A Different Table Design for Big Data](https://thenewstack.io/apache-iceberg-a-different-table-design-for-big-data/) **Date**: Feb 1st, 2021, **Company**: thenewstack.io @@ -452,6 +520,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Gautam Kowshik](https://www.linkedin.com/in/gautamk/), [Xabriel J. Collazo Mojica](https://www.linkedin.com/in/xabriel/) + ### [FastIngest: Low-latency Gobblin with Apache Iceberg and ORC format](https://engineering.linkedin.com/blog/2021/fastingest-low-latency-gobblin) **Date**: Jan 6th, 2021, **Company**: Linkedin @@ -463,6 +532,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Andrei Ionescu](https://www.linkedin.com/in/andreiionescu), [Shone Sadler](https://www.linkedin.com/in/shonesadler/), [Anil Malkani](https://www.linkedin.com/in/anil-malkani-52861a/) + ### [Optimizing data warehouse storage](https://netflixtechblog.com/optimizing-data-warehouse-storage-7b94a48fdcbe) **Date**: Dec 21st, 2020, **Company**: Netflix @@ -474,6 +544,7 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Author**: [Shone Sadler](https://www.linkedin.com/in/shonesadler/), [Romin Parekh](https://www.linkedin.com/in/rominparekh/), [Anil Malkani](https://www.linkedin.com/in/anil-malkani-52861a/) + ### [Bulldozer: Batch Data Moving from Data Warehouse to Online Key-Value Stores](https://netflixtechblog.com/bulldozer-batch-data-moving-from-data-warehouse-to-online-key-value-stores-41bac13863f8) **Date**: Oct 27th, 2020, **Company**: Netflix diff --git a/site/link-checker-config.json b/site/link-checker-config.json index 8eed0c163404..54ff94c0145c 100644 --- a/site/link-checker-config.json +++ b/site/link-checker-config.json @@ -8,6 +8,9 @@ }, { "pattern": "^../../javadoc" + }, + { + "pattern": "^https://search.maven.org/" } ], "replacementPatterns": [ From e1cfddc9553c7b1a3a43dd4ae95ce55cd08820b0 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 3 Apr 2024 19:28:29 +0900 Subject: [PATCH 0191/1019] Core, Data, Flink: Migrate TableTestBase related classes to JUnit5 (#10080) --- .../iceberg/util/TestTableScanUtil.java | 8 +- .../iceberg/io/TestPartitioningWriters.java | 95 +++++++++--------- .../iceberg/io/TestPositionDeltaWriters.java | 36 +++---- .../iceberg/io/TestRollingFileWriters.java | 49 +++++----- .../flink/sink/TestDeltaTaskWriter.java | 97 ++++++++----------- .../flink/sink/TestIcebergFilesCommitter.java | 55 +++++------ .../source/TestStreamingMonitorFunction.java | 16 ++- .../source/TestStreamingReaderOperator.java | 36 ++++--- .../flink/sink/TestDeltaTaskWriter.java | 97 ++++++++----------- .../flink/sink/TestIcebergFilesCommitter.java | 55 +++++------ .../source/TestStreamingMonitorFunction.java | 16 ++- .../source/TestStreamingReaderOperator.java | 36 ++++--- .../flink/sink/TestDeltaTaskWriter.java | 97 ++++++++----------- .../flink/sink/TestIcebergFilesCommitter.java | 55 +++++------ .../source/TestStreamingMonitorFunction.java | 16 ++- .../source/TestStreamingReaderOperator.java | 36 ++++--- 16 files changed, 382 insertions(+), 418 deletions(-) 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 d868fba90190..eb713a4d2e0b 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTableScanUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.util; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.Collections; @@ -49,7 +50,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -163,13 +163,13 @@ public void testTaskGroupPlanningCorruptedOffset() { TableScanUtil.planTaskGroups(CloseableIterable.withNoopClose(baseFileScanTasks), 1, 1, 0)) { for (FileScanTask fileScanTask : task.tasks()) { DataFile taskDataFile = fileScanTask.file(); - Assertions.assertThat(taskDataFile.splitOffsets()).isNull(); + assertThat(taskDataFile.splitOffsets()).isNull(); taskCount++; } } // 10 tasks since the split offsets are ignored and there are 1 byte splits for a 10 byte file - Assertions.assertThat(taskCount).isEqualTo(10); + assertThat(taskCount).isEqualTo(10); } @Test @@ -280,7 +280,7 @@ public void testTaskGroupPlanningByPartition() { ImmutableList.of( taskWithPartition(SPEC1, PARTITION1, 128), taskWithPartition(SPEC2, PARTITION2, 128)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TableScanUtil.planTaskGroups(tasks2, 128, 10, 4, SPEC2.partitionType())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find field"); 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 88d3c16c2dca..8dc031314eda 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -19,6 +19,7 @@ package org.apache.iceberg.io; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -38,8 +39,6 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -72,7 +71,7 @@ protected FileFormat format() { @BeforeEach public void setupTable() throws Exception { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created during table creation + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -86,10 +85,10 @@ public void testClusteredDataWriterNoRecords() throws IOException { new ClusteredDataWriter<>(writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); } @TestTemplate @@ -111,7 +110,7 @@ public void testClusteredDataWriterMultiplePartitions() throws IOException { writer.close(); DataWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 data files", 3, result.dataFiles().size()); + assertThat(result.dataFiles()).hasSize(3); RowDelta rowDelta = table.newRowDelta(); result.dataFiles().forEach(rowDelta::addRows); @@ -120,7 +119,7 @@ public void testClusteredDataWriterMultiplePartitions() throws IOException { List expectedRows = ImmutableList.of( toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb"), toRow(4, "bbb"), toRow(5, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -139,8 +138,7 @@ public void testClusteredDataWriterOutOfOrderPartitions() throws IOException { writer.write(toRow(4, "bbb"), spec, partitionKey(spec, "bbb")); writer.write(toRow(5, "ccc"), spec, partitionKey(spec, "ccc")); - Assertions.assertThatThrownBy( - () -> writer.write(toRow(6, "aaa"), spec, partitionKey(spec, "aaa"))) + assertThatThrownBy(() -> writer.write(toRow(6, "aaa"), spec, partitionKey(spec, "aaa"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("partition 'data=aaa' in spec " + spec); @@ -159,14 +157,14 @@ public void testClusteredEqualityDeleteWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -222,17 +220,16 @@ public void testClusteredEqualityDeleteWriterMultipleSpecs() throws IOException writer.close(); DeleteWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 delete files", 3, result.deleteFiles().size()); - Assert.assertEquals( - "Must not reference data files", 0, writer.result().referencedDataFiles().size()); - Assert.assertFalse("Must not reference data files", writer.result().referencesDataFiles()); + assertThat(result.deleteFiles()).hasSize(3); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); RowDelta rowDelta = table.newRowDelta(); result.deleteFiles().forEach(rowDelta::addDeletes); rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(11, "aaa"), toRow(12, "bbb"), toRow(13, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -265,13 +262,13 @@ public void testClusteredEqualityDeleteWriterOutOfOrderSpecsAndPartitions() thro writer.write(toRow(5, "ccc"), identitySpec, partitionKey(identitySpec, "ccc")); writer.write(toRow(6, "ddd"), identitySpec, partitionKey(identitySpec, "ddd")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> writer.write(toRow(7, "ccc"), identitySpec, partitionKey(identitySpec, "ccc"))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("partition 'data=ccc' in spec " + identitySpec); - Assertions.assertThatThrownBy(() -> writer.write(toRow(7, "aaa"), unpartitionedSpec, null)) + assertThatThrownBy(() -> writer.write(toRow(7, "aaa"), unpartitionedSpec, null)) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("spec []"); @@ -297,14 +294,14 @@ private void checkClusteredPositionDeleteWriterNoRecords(DeleteGranularity delet writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE, deleteGranularity); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -373,17 +370,16 @@ private void checkClusteredPositionDeleteWriterMultipleSpecs(DeleteGranularity d writer.close(); DeleteWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 delete files", 3, result.deleteFiles().size()); - Assert.assertEquals( - "Must reference 3 data files", 3, writer.result().referencedDataFiles().size()); - Assert.assertTrue("Must reference data files", writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(3); + assertThat(writer.result().referencedDataFiles()).hasSize(3); + assertThat(writer.result().referencesDataFiles()).isTrue(); RowDelta rowDelta = table.newRowDelta(); result.deleteFiles().forEach(rowDelta::addDeletes); rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(11, "aaa"), toRow(12, "bbb"), toRow(13, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -433,7 +429,7 @@ private void checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions( identitySpec, partitionKey(identitySpec, "ddd")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { PositionDelete positionDelete = positionDelete("file-5.parquet", 1L, null); writer.write(positionDelete, identitySpec, partitionKey(identitySpec, "ccc")); @@ -442,7 +438,7 @@ private void checkClusteredPositionDeleteWriterOutOfOrderSpecsAndPartitions( .hasMessageContaining("Encountered records that belong to already closed files") .hasMessageEndingWith("partition 'data=ccc' in spec " + identitySpec); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { PositionDelete positionDelete = positionDelete("file-1.parquet", 3L, null); writer.write(positionDelete, unpartitionedSpec, null); @@ -516,10 +512,10 @@ public void testFanoutDataWriterNoRecords() throws IOException { new FanoutDataWriter<>(writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); } @TestTemplate @@ -541,7 +537,7 @@ public void testFanoutDataWriterMultiplePartitions() throws IOException { writer.close(); DataWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 data files", 3, result.dataFiles().size()); + assertThat(result.dataFiles()).hasSize(3); RowDelta rowDelta = table.newRowDelta(); result.dataFiles().forEach(rowDelta::addRows); @@ -550,7 +546,7 @@ public void testFanoutDataWriterMultiplePartitions() throws IOException { List expectedRows = ImmutableList.of( toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb"), toRow(4, "bbb"), toRow(5, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -571,14 +567,14 @@ private void checkFanoutPositionOnlyDeleteWriterNoRecords(DeleteGranularity dele writerFactory, fileFactory, table.io(), TARGET_FILE_SIZE, deleteGranularity); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -657,17 +653,16 @@ private void checkFanoutPositionOnlyDeleteWriterOutOfOrderRecords( writer.close(); DeleteWriteResult result = writer.result(); - Assert.assertEquals("Must be 3 delete files", 3, result.deleteFiles().size()); - Assert.assertEquals( - "Must reference 3 data files", 3, writer.result().referencedDataFiles().size()); - Assert.assertTrue("Must reference data files", writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(3); + assertThat(writer.result().referencedDataFiles()).hasSize(3); + assertThat(writer.result().referencesDataFiles()).isTrue(); RowDelta rowDelta = table.newRowDelta(); result.deleteFiles().forEach(rowDelta::addDeletes); rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(12, "bbb")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate 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 2de7bf7324c0..177982a59cb3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPositionDeltaWriters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -34,7 +36,6 @@ import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -67,7 +68,7 @@ protected FileFormat format() { @BeforeEach public void setupTable() throws Exception { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created during table creation + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); this.table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -94,9 +95,9 @@ public void testPositionDeltaWithOneDataWriter() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 1 data files", 1, dataFiles.length); - Assert.assertEquals("Must be no delete files", 0, deleteFiles.length); - Assert.assertEquals("Must not reference data files", 0, referencedDataFiles.length); + assertThat(dataFiles).hasSize(1); + assertThat(deleteFiles).isEmpty(); + assertThat(referencedDataFiles).isEmpty(); } @TestTemplate @@ -121,9 +122,9 @@ public void testPositionDeltaInsertOnly() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 1 data files", 1, dataFiles.length); - Assert.assertEquals("Must be no delete files", 0, deleteFiles.length); - Assert.assertEquals("Must not reference data files", 0, referencedDataFiles.length); + assertThat(dataFiles).hasSize(1); + assertThat(deleteFiles).isEmpty(); + assertThat(referencedDataFiles).isEmpty(); RowDelta rowDelta = table.newRowDelta(); for (DataFile dataFile : dataFiles) { @@ -132,7 +133,7 @@ public void testPositionDeltaInsertOnly() throws IOException { rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(1, "aaa")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -177,10 +178,9 @@ public void testPositionDeltaDeleteOnly() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 0 data files", 0, dataFiles.length); - Assert.assertEquals("Must be 2 delete files", 2, deleteFiles.length); - Assert.assertEquals("Must reference 2 data files", 2, referencedDataFiles.length); - + assertThat(dataFiles).isEmpty(); + assertThat(deleteFiles).hasSize(2); + assertThat(referencedDataFiles).hasSize(2); RowDelta rowDelta = table.newRowDelta(); for (DeleteFile deleteFile : deleteFiles) { rowDelta.addDeletes(deleteFile); @@ -188,7 +188,7 @@ public void testPositionDeltaDeleteOnly() throws IOException { rowDelta.commit(); List expectedRows = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } @TestTemplate @@ -234,9 +234,9 @@ public void testPositionDeltaMultipleSpecs() throws IOException { DeleteFile[] deleteFiles = result.deleteFiles(); CharSequence[] referencedDataFiles = result.referencedDataFiles(); - Assert.assertEquals("Must be 1 data files", 1, dataFiles.length); - Assert.assertEquals("Must be 2 delete files", 2, deleteFiles.length); - Assert.assertEquals("Must reference 2 data files", 2, referencedDataFiles.length); + assertThat(dataFiles).hasSize(1); + assertThat(deleteFiles).hasSize(2); + assertThat(referencedDataFiles).hasSize(2); RowDelta rowDelta = table.newRowDelta(); for (DataFile dataFile : dataFiles) { @@ -249,6 +249,6 @@ public void testPositionDeltaMultipleSpecs() throws IOException { List expectedRows = ImmutableList.of(toRow(1, "aaa"), toRow(2, "aaa"), toRow(3, "bbb"), toRow(10, "ccc")); - Assert.assertEquals("Records should match", toSet(expectedRows), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } } 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 f6d5d41b72b3..6407fd0cbf70 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestRollingFileWriters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -33,7 +35,6 @@ import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -74,7 +75,7 @@ protected FileFormat format() { @BeforeEach public void setupTable() throws Exception { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created during table creation + assertThat(tableDir.delete()).isTrue(); // created during table creation this.metadataDir = new File(tableDir, "metadata"); @@ -97,10 +98,10 @@ public void testRollingDataWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), DEFAULT_FILE_SIZE, table.spec(), partition); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); writer.close(); - Assert.assertEquals("Must be no data files", 0, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).isEmpty(); } @TestTemplate @@ -122,7 +123,7 @@ public void testRollingDataWriterSplitData() throws IOException { // call close again to ensure it is idempotent writer.close(); - Assert.assertEquals(4, writer.result().dataFiles().size()); + assertThat(writer.result().dataFiles()).hasSize(4); } @TestTemplate @@ -136,14 +137,14 @@ public void testRollingEqualityDeleteWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), DEFAULT_FILE_SIZE, table.spec(), partition); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -168,10 +169,9 @@ public void testRollingEqualityDeleteWriterSplitDeletes() throws IOException { // call close again to ensure it is idempotent writer.close(); - DeleteWriteResult result = writer.result(); - Assert.assertEquals(4, result.deleteFiles().size()); - Assert.assertEquals(0, result.referencedDataFiles().size()); - Assert.assertFalse(result.referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(4); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -182,14 +182,14 @@ public void testRollingPositionDeleteWriterNoRecords() throws IOException { writerFactory, fileFactory, table.io(), DEFAULT_FILE_SIZE, table.spec(), partition); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); writer.close(); - Assert.assertEquals(0, writer.result().deleteFiles().size()); - Assert.assertEquals(0, writer.result().referencedDataFiles().size()); - Assert.assertFalse(writer.result().referencesDataFiles()); + assertThat(writer.result().deleteFiles()).isEmpty(); + assertThat(writer.result().referencedDataFiles()).isEmpty(); + assertThat(writer.result().referencesDataFiles()).isFalse(); } @TestTemplate @@ -212,9 +212,8 @@ public void testRollingPositionDeleteWriterSplitDeletes() throws IOException { // call close again to ensure it is idempotent writer.close(); - DeleteWriteResult result = writer.result(); - Assert.assertEquals(4, result.deleteFiles().size()); - Assert.assertEquals(1, result.referencedDataFiles().size()); - Assert.assertTrue(result.referencesDataFiles()); + assertThat(writer.result().deleteFiles()).hasSize(4); + assertThat(writer.result().referencedDataFiles()).hasSize(1); + assertThat(writer.result().referencesDataFiles()).isTrue(); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index f780c6135bee..21f3ee2c655a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; 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; @@ -66,8 +67,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,7 +89,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created by table create + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); } @@ -132,18 +131,17 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -160,14 +158,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(4, "fff")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); } @TestTemplate @@ -194,11 +191,11 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); } @TestTemplate @@ -232,14 +229,11 @@ private void testAbort(boolean partitioned) throws IOException { .filter(p -> p.toFile().isFile()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); + assertThat(files).hasSize(partitioned ? 4 : 2); writer.abort(); for (Path file : files) { - Assert.assertFalse(Files.exists(file)); + assertThat(file).doesNotExist(); } } @@ -268,14 +262,13 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createInsert(4, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -284,18 +277,17 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createDelete(7, "ccc")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); } @TestTemplate @@ -312,15 +304,12 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { writer.write(createDelete(2, "aaa")); // 1 pos-delete. WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); } @TestTemplate @@ -361,10 +350,10 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { WriteResult result = writer.complete(); // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + assertThat(result.dataFiles()).hasSize(1); // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( + assertThat(result.deleteFiles()).hasSize(2); + assertThat( Arrays.stream(result.deleteFiles()) .map(ContentFile::content) .collect(Collectors.toSet())) @@ -376,7 +365,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { int cutPrecisionNano = start.getNano() / 1000000 * 1000000; expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); } private void commitTransaction(WriteResult result) { diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 4459f224afe6..06b6bc9a977b 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -75,8 +76,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -111,7 +110,7 @@ public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); @@ -217,9 +216,8 @@ public void testCommitTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); assertSnapshotSize(i); assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } } @@ -715,9 +713,8 @@ public void testBoundedStream() throws Exception { SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } @@ -745,16 +742,16 @@ public void testFlinkManifests() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -767,7 +764,9 @@ public void testFlinkManifests() throws Exception { @TestTemplate public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -792,16 +791,16 @@ public void testDeleteFiles() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -836,7 +835,9 @@ public void testDeleteFiles() throws Exception { @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -882,8 +883,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(2); } } @@ -1047,10 +1047,7 @@ private List assertFlinkManifests(int expectedCount) throws IOException { Files.list(flinkManifestFolder.toPath()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); + assertThat(manifests).hasSize(expectedCount); return manifests; } @@ -1084,12 +1081,12 @@ private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId( table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); + assertThat(actualId).isEqualTo(expectedId); } private void assertSnapshotSize(int expectedSnapshotSize) { table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index a612617835b0..f7b13598bc2a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,9 +52,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -78,7 +77,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -249,7 +248,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .monitorInterval(Duration.ofMillis(100)) .maxPlanningSnapshotCount(0) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -259,7 +258,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(-10) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); } @@ -282,7 +281,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { FlinkInputSplit[] expectedSplits = FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + assertThat(expectedSplits).hasSize(9); // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the // total splits number @@ -306,10 +305,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { function.monitorAndForwardSplits(); if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); } } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index f96426a59a2d..1606ee9f9648 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -49,7 +51,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +74,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -84,7 +85,7 @@ public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); + assertThat(splits).hasSize(10); try (OneInputStreamOperatorTestHarness harness = createReader()) { harness.setup(); @@ -98,7 +99,7 @@ public void testProcessAllRecords() throws Exception { harness.processElement(splits.get(i), -1); // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); // Assert the output has expected elements. expected.addAll(expectedRecords.get(i)); @@ -115,7 +116,7 @@ public void testTriggerCheckpoint() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(3); List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); + assertThat(splits).hasSize(3); long timestamp = 0; try (OneInputStreamOperatorTestHarness harness = createReader()) { @@ -131,17 +132,18 @@ public void testTriggerCheckpoint() throws Exception { // Trigger snapshot state, it will start to work once all records from split0 are read. processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); TestHelpers.assertRecords( readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); @@ -153,7 +155,7 @@ public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); + assertThat(splits).hasSize(15); OperatorSubtaskState state; List expected = Lists.newArrayList(); @@ -170,7 +172,9 @@ public void testCheckpointRestore() throws Exception { SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); for (int i = 0; i < 5; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -190,7 +194,9 @@ public void testCheckpointRestore() throws Exception { for (int i = 5; i < 10; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -200,7 +206,9 @@ public void testCheckpointRestore() throws Exception { expected.addAll(expectedRecords.get(i)); harness.processElement(splits.get(i), 1); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java index f780c6135bee..21f3ee2c655a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; 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; @@ -66,8 +67,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,7 +89,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created by table create + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); } @@ -132,18 +131,17 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -160,14 +158,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(4, "fff")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); } @TestTemplate @@ -194,11 +191,11 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); } @TestTemplate @@ -232,14 +229,11 @@ private void testAbort(boolean partitioned) throws IOException { .filter(p -> p.toFile().isFile()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); + assertThat(files).hasSize(partitioned ? 4 : 2); writer.abort(); for (Path file : files) { - Assert.assertFalse(Files.exists(file)); + assertThat(file).doesNotExist(); } } @@ -268,14 +262,13 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createInsert(4, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -284,18 +277,17 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createDelete(7, "ccc")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); } @TestTemplate @@ -312,15 +304,12 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { writer.write(createDelete(2, "aaa")); // 1 pos-delete. WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); } @TestTemplate @@ -361,10 +350,10 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { WriteResult result = writer.complete(); // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + assertThat(result.dataFiles()).hasSize(1); // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( + assertThat(result.deleteFiles()).hasSize(2); + assertThat( Arrays.stream(result.deleteFiles()) .map(ContentFile::content) .collect(Collectors.toSet())) @@ -376,7 +365,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { int cutPrecisionNano = start.getNano() / 1000000 * 1000000; expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); } private void commitTransaction(WriteResult result) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 4459f224afe6..06b6bc9a977b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -75,8 +76,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -111,7 +110,7 @@ public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); @@ -217,9 +216,8 @@ public void testCommitTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); assertSnapshotSize(i); assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } } @@ -715,9 +713,8 @@ public void testBoundedStream() throws Exception { SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } @@ -745,16 +742,16 @@ public void testFlinkManifests() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -767,7 +764,9 @@ public void testFlinkManifests() throws Exception { @TestTemplate public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -792,16 +791,16 @@ public void testDeleteFiles() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -836,7 +835,9 @@ public void testDeleteFiles() throws Exception { @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -882,8 +883,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(2); } } @@ -1047,10 +1047,7 @@ private List assertFlinkManifests(int expectedCount) throws IOException { Files.list(flinkManifestFolder.toPath()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); + assertThat(manifests).hasSize(expectedCount); return manifests; } @@ -1084,12 +1081,12 @@ private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId( table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); + assertThat(actualId).isEqualTo(expectedId); } private void assertSnapshotSize(int expectedSnapshotSize) { table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java index 451e3552b9ac..9c4f476b02b4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,9 +52,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -78,7 +77,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -250,7 +249,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(0) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -260,7 +259,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(-10) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); } @@ -283,7 +282,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { FlinkInputSplit[] expectedSplits = FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + assertThat(expectedSplits).hasSize(9); // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the // total splits number @@ -307,10 +306,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { function.monitorAndForwardSplits(); if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); } } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java index f96426a59a2d..1606ee9f9648 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -49,7 +51,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +74,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -84,7 +85,7 @@ public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); + assertThat(splits).hasSize(10); try (OneInputStreamOperatorTestHarness harness = createReader()) { harness.setup(); @@ -98,7 +99,7 @@ public void testProcessAllRecords() throws Exception { harness.processElement(splits.get(i), -1); // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); // Assert the output has expected elements. expected.addAll(expectedRecords.get(i)); @@ -115,7 +116,7 @@ public void testTriggerCheckpoint() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(3); List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); + assertThat(splits).hasSize(3); long timestamp = 0; try (OneInputStreamOperatorTestHarness harness = createReader()) { @@ -131,17 +132,18 @@ public void testTriggerCheckpoint() throws Exception { // Trigger snapshot state, it will start to work once all records from split0 are read. processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); TestHelpers.assertRecords( readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); @@ -153,7 +155,7 @@ public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); + assertThat(splits).hasSize(15); OperatorSubtaskState state; List expected = Lists.newArrayList(); @@ -170,7 +172,9 @@ public void testCheckpointRestore() throws Exception { SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); for (int i = 0; i < 5; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -190,7 +194,9 @@ public void testCheckpointRestore() throws Exception { for (int i = 5; i < 10; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -200,7 +206,9 @@ public void testCheckpointRestore() throws Exception { expected.addAll(expectedRecords.get(i)); harness.processElement(splits.get(i), 1); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } } 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 f780c6135bee..21f3ee2c655a 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 @@ -24,6 +24,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; 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; @@ -66,8 +67,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -90,7 +89,7 @@ protected static List parameters() { @BeforeEach public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); - Assert.assertTrue(tableDir.delete()); // created by table create + assertThat(tableDir.delete()).isTrue(); // created by table create this.metadataDir = new File(tableDir, "metadata"); } @@ -132,18 +131,17 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. WriteResult result = writer.complete(); - Assert.assertEquals(partitioned ? 7 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records.", - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -160,14 +158,13 @@ private void testCdcEvents(boolean partitioned) throws IOException { writer.write(createDelete(4, "fff")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(partitioned ? 2 : 1, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); } @TestTemplate @@ -194,11 +191,11 @@ private void testWritePureEqDeletes(boolean partitioned) throws IOException { writer.write(createDelete(3, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(0, result.dataFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.deleteFiles().length); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); commitTransaction(result); - Assert.assertEquals("Should have no record", expectedRowSet(), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); } @TestTemplate @@ -232,14 +229,11 @@ private void testAbort(boolean partitioned) throws IOException { .filter(p -> p.toFile().isFile()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - "Should have expected file count, but files are: " + files, - partitioned ? 4 : 2, - files.size()); + assertThat(files).hasSize(partitioned ? 4 : 2); writer.abort(); for (Path file : files) { - Assert.assertFalse(Files.exists(file)); + assertThat(file).doesNotExist(); } } @@ -268,14 +262,13 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createInsert(4, "ccc")); WriteResult result = writer.complete(); - Assert.assertEquals(3, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); // Start the 2nd transaction. writer = taskWriterFactory.create(); @@ -284,18 +277,17 @@ public void testPartitionedTableWithDataAsKey() throws IOException { writer.write(createDelete(7, "ccc")); // 1 eq-delete. result = writer.complete(); - Assert.assertEquals(2, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb")), - actualRowSet("*")); + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); } @TestTemplate @@ -312,15 +304,12 @@ public void testPartitionedTableWithDataAndIdAsKey() throws IOException { writer.write(createDelete(2, "aaa")); // 1 pos-delete. WriteResult result = writer.complete(); - Assert.assertEquals(1, result.dataFiles().length); - Assert.assertEquals(1, result.deleteFiles().length); - Assert.assertEquals( - Sets.newHashSet(FileContent.POSITION_DELETES), - Sets.newHashSet(result.deleteFiles()[0].content())); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); commitTransaction(result); - Assert.assertEquals( - "Should have expected records", expectedRowSet(createRecord(1, "aaa")), actualRowSet("*")); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); } @TestTemplate @@ -361,10 +350,10 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { WriteResult result = writer.complete(); // One data file - Assertions.assertThat(result.dataFiles().length).isEqualTo(1); + assertThat(result.dataFiles()).hasSize(1); // One eq delete file + one pos delete file - Assertions.assertThat(result.deleteFiles().length).isEqualTo(2); - Assertions.assertThat( + assertThat(result.deleteFiles()).hasSize(2); + assertThat( Arrays.stream(result.deleteFiles()) .map(ContentFile::content) .collect(Collectors.toSet())) @@ -376,7 +365,7 @@ public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { int cutPrecisionNano = start.getNano() / 1000000 * 1000000; expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - Assertions.assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); } private void commitTransaction(WriteResult result) { 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 4459f224afe6..06b6bc9a977b 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 @@ -22,6 +22,7 @@ import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -75,8 +76,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.Assume; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -111,7 +110,7 @@ public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SimpleDataUtil.SCHEMA, PartitionSpec.unpartitioned()); @@ -217,9 +216,8 @@ public void testCommitTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); assertSnapshotSize(i); assertMaxCommittedCheckpointId(jobID, operatorId, i); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } } @@ -715,9 +713,8 @@ public void testBoundedStream() throws Exception { SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - Assert.assertEquals( - TestIcebergFilesCommitter.class.getName(), - SimpleDataUtil.latestSnapshot(table, branch).summary().get("flink.test")); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } } @@ -745,16 +742,16 @@ public void testFlinkManifests() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -767,7 +764,9 @@ public void testFlinkManifests() throws Exception { @TestTemplate public void testDeleteFiles() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -792,16 +791,16 @@ public void testDeleteFiles() throws Exception { harness.snapshot(checkpoint, ++timestamp); List manifestPaths = assertFlinkManifests(1); Path manifestPath = manifestPaths.get(0); - Assert.assertEquals( - "File name should have the expected pattern.", - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1), - manifestPath.getFileName().toString()); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); // 2. Read the data files from manifests and assert. List dataFiles = FlinkManifestUtil.readDataFiles( createTestingManifestFile(manifestPath), table.io(), table.specs()); - Assert.assertEquals(1, dataFiles.size()); + assertThat(dataFiles).hasSize(1); TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); // 3. notifyCheckpointComplete for checkpoint#1 @@ -836,7 +835,9 @@ public void testDeleteFiles() throws Exception { @TestTemplate public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - Assume.assumeFalse("Only support equality-delete in format v2.", formatVersion < 2); + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); long timestamp = 0; long checkpoint = 10; @@ -882,8 +883,7 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); assertFlinkManifests(0); - Assert.assertEquals( - "Should have committed 2 txn.", 2, ImmutableList.copyOf(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(2); } } @@ -1047,10 +1047,7 @@ private List assertFlinkManifests(int expectedCount) throws IOException { Files.list(flinkManifestFolder.toPath()) .filter(p -> !p.toString().endsWith(".crc")) .collect(Collectors.toList()); - Assert.assertEquals( - String.format("Expected %s flink manifests, but the list is: %s", expectedCount, manifests), - expectedCount, - manifests.size()); + assertThat(manifests).hasSize(expectedCount); return manifests; } @@ -1084,12 +1081,12 @@ private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long actualId = IcebergFilesCommitter.getMaxCommittedCheckpointId( table, jobID.toString(), operatorID.toHexString(), branch); - Assert.assertEquals(expectedId, actualId); + assertThat(actualId).isEqualTo(expectedId); } private void assertSnapshotSize(int expectedSnapshotSize) { table.refresh(); - Assert.assertEquals(expectedSnapshotSize, Lists.newArrayList(table.snapshots()).size()); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) 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 451e3552b9ac..9c4f476b02b4 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,9 +52,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -78,7 +77,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -250,7 +249,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(0) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext1)) + assertThatThrownBy(() -> createFunction(scanContext1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); @@ -260,7 +259,7 @@ public void testInvalidMaxPlanningSnapshotCount() { .maxPlanningSnapshotCount(-10) .build(); - Assertions.assertThatThrownBy(() -> createFunction(scanContext2)) + assertThatThrownBy(() -> createFunction(scanContext2)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("The max-planning-snapshot-count must be greater than zero"); } @@ -283,7 +282,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { FlinkInputSplit[] expectedSplits = FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - Assert.assertEquals("should produce 9 splits", 9, expectedSplits.length); + assertThat(expectedSplits).hasSize(9); // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the // total splits number @@ -307,10 +306,7 @@ public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { function.monitorAndForwardSplits(); if (maxPlanningSnapshotCount < 10) { - Assert.assertEquals( - "Should produce same splits as max-planning-snapshot-count", - maxPlanningSnapshotCount, - sourceContext.splits.size()); + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); } } } 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 f96426a59a2d..1606ee9f9648 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.nio.file.Files; @@ -49,7 +51,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +74,7 @@ protected static List parameters() { public void setupTable() throws IOException { this.tableDir = Files.createTempDirectory(temp, "junit").toFile(); this.metadataDir = new File(tableDir, "metadata"); - Assert.assertTrue(tableDir.delete()); + assertThat(tableDir.delete()).isTrue(); // Construct the iceberg table. table = create(SCHEMA, PartitionSpec.unpartitioned()); @@ -84,7 +85,7 @@ public void testProcessAllRecords() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(10); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 10, splits.size()); + assertThat(splits).hasSize(10); try (OneInputStreamOperatorTestHarness harness = createReader()) { harness.setup(); @@ -98,7 +99,7 @@ public void testProcessAllRecords() throws Exception { harness.processElement(splits.get(i), -1); // Run the mail-box once to read all records from the given split. - Assert.assertTrue("Should processed 1 split", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); // Assert the output has expected elements. expected.addAll(expectedRecords.get(i)); @@ -115,7 +116,7 @@ public void testTriggerCheckpoint() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(3); List splits = generateSplits(); - Assert.assertEquals("Should have 3 splits", 3, splits.size()); + assertThat(splits).hasSize(3); long timestamp = 0; try (OneInputStreamOperatorTestHarness harness = createReader()) { @@ -131,17 +132,18 @@ public void testTriggerCheckpoint() throws Exception { // Trigger snapshot state, it will start to work once all records from split0 are read. processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - Assert.assertTrue("Should have processed the split0", processor.runMailboxStep()); - Assert.assertTrue( - "Should have processed the snapshot state action", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); // Read records from split1. - Assert.assertTrue("Should have processed the split1", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); // Read records from split2. - Assert.assertTrue("Should have processed the split2", processor.runMailboxStep()); + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); TestHelpers.assertRecords( readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); @@ -153,7 +155,7 @@ public void testCheckpointRestore() throws Exception { List> expectedRecords = generateRecordsAndCommitTxn(15); List splits = generateSplits(); - Assert.assertEquals("Should have 10 splits", 15, splits.size()); + assertThat(splits).hasSize(15); OperatorSubtaskState state; List expected = Lists.newArrayList(); @@ -170,7 +172,9 @@ public void testCheckpointRestore() throws Exception { SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); for (int i = 0; i < 5; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -190,7 +194,9 @@ public void testCheckpointRestore() throws Exception { for (int i = 5; i < 10; i++) { expected.addAll(expectedRecords.get(i)); - Assert.assertTrue("Should have processed one split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } @@ -200,7 +206,9 @@ public void testCheckpointRestore() throws Exception { expected.addAll(expectedRecords.get(i)); harness.processElement(splits.get(i), 1); - Assert.assertTrue("Should have processed the split#" + i, localMailbox.runMailboxStep()); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); } } From d2544d5dce0c5b698997d805d6432e6992d8caa4 Mon Sep 17 00:00:00 2001 From: lurnagao-dahua <91278331+lurnagao-dahua@users.noreply.github.com> Date: Wed, 3 Apr 2024 21:27:39 +0800 Subject: [PATCH 0192/1019] Hive: Avoid NPE on Throwables without error msg (#10069) --- .../iceberg/hive/HiveTableOperations.java | 11 ++++++----- .../apache/iceberg/hive/TestHiveCommits.java | 18 ++++++++++++++++++ 2 files changed, 24 insertions(+), 5 deletions(-) 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 bae074d55d24..75d59de75d4d 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 @@ -263,11 +263,12 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { throw e; } catch (Throwable e) { - if (e.getMessage() - .contains( - "The table has been modified. The parameter value for key '" - + HiveTableOperations.METADATA_LOCATION_PROP - + "' is")) { + if (e.getMessage() != null + && e.getMessage() + .contains( + "The table has been modified. The parameter value for key '" + + HiveTableOperations.METADATA_LOCATION_PROP + + "' is")) { throw new CommitFailedException( e, "The table %s.%s has been modified concurrently", database, tableName); } 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 aaa659042118..acf4f8dc5cd3 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 @@ -397,6 +397,24 @@ public void testLockExceptionUnknownSuccessCommit() throws TException, Interrupt .isTrue(); } + @Test + public void testCommitExceptionWithoutMessage() 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(); + + HiveTableOperations spyOps = spy(ops); + doThrow(new RuntimeException()).when(spyOps).persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(ops.current(), metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("null\nCannot determine whether the commit was successful or not"); + } + private void commitAndThrowException( HiveTableOperations realOperations, HiveTableOperations spyOperations) throws TException, InterruptedException { From 597fa0d464aff9f7a79e691edfc13dc4fcda084d Mon Sep 17 00:00:00 2001 From: Alex Merced Date: Wed, 3 Apr 2024 09:22:11 -0500 Subject: [PATCH 0193/1019] Docs: Add 5 dremio blogs (#10067) --- site/docs/blogs.md | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 4d327596129e..73eac3cf710a 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -22,6 +22,43 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. + + +### [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) +**Date**: April 1st, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From MongoDB to Dashboards with Dremio and Apache Iceberg](https://www.dremio.com/blog/from-mongodb-to-dashboards-with-dremio-and-apache-iceberg/) +**Date**: March 29th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From SQLServer to Dashboards with Dremio and Apache Iceberg](https://www.dremio.com/blog/from-sqlserver-to-dashboards-with-dremio-and-apache-iceberg/) +**Date**: March 29th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [BI Dashboards with Apache Iceberg Using AWS Glue and Apache Superset](https://www.dremio.com/blog/bi-dashboards-with-apache-iceberg-using-aws-glue-and-apache-superset/) +**Date**: March 29th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [From Postgres to Dashboards with Dremio and Apache Iceberg](https://www.dremio.com/blog/from-postgres-to-dashboards-with-dremio-and-apache-iceberg/) +**Date**: March 28th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + + +### [Run Graph Queries on Apache Iceberg Tables with Dremio & Puppygraph](https://www.dremio.com/blog/run-graph-queries-on-apache-iceberg-tables-with-dremio-puppygraph/) +**Date**: March 27th, 2024, **Company**: Dremio + +**Author**: [Alex Merced](https://www.linkedin.com/in/alexmerced/) + ### [The Apache Iceberg Lakehouse: The Great Data Equalizer](https://amdatalakehouse.substack.com/p/the-apache-iceberg-lakehouse-the) **Date**: March 6th, 2024, **Company**: Dremio From eb6ba70062900560bf37ee28afd1589b7a6419db Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Thu, 4 Apr 2024 21:36:24 +0200 Subject: [PATCH 0194/1019] REST: Fix spurious warning when shutting down refresh executor (#10087) --- .../main/java/org/apache/iceberg/rest/RESTSessionCatalog.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 96aa14b128da..f8a47f19fdb9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -602,7 +602,7 @@ private void shutdownRefreshExecutor() { }); try { - if (service.awaitTermination(1, TimeUnit.MINUTES)) { + if (!service.awaitTermination(1, TimeUnit.MINUTES)) { LOG.warn("Timed out waiting for refresh executor to terminate"); } } catch (InterruptedException e) { From fb838360af9fb547c2a12258c17bfca13a8303c3 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 4 Apr 2024 14:02:50 -0600 Subject: [PATCH 0195/1019] API: Fix default FileIO#newInputFile ManifestFile, DataFile and DeleteFile implementations (#9953) --- .../java/org/apache/iceberg/io/FileIO.java | 12 ++-- .../apache/iceberg/aws/s3/TestS3FileIO.java | 58 +++++++++++++++++++ 2 files changed, 64 insertions(+), 6 deletions(-) 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 fc6a53367f21..a521cbf79d7f 100644 --- a/api/src/main/java/org/apache/iceberg/io/FileIO.java +++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java @@ -49,25 +49,25 @@ default InputFile newInputFile(String path, long length) { default InputFile newInputFile(DataFile file) { Preconditions.checkArgument( file.keyMetadata() == null, - "Cannot decrypt data file: {} (use EncryptingFileIO)", + "Cannot decrypt data file: %s (use EncryptingFileIO)", file.path()); - return newInputFile(file.path().toString()); + return newInputFile(file.path().toString(), file.fileSizeInBytes()); } default InputFile newInputFile(DeleteFile file) { Preconditions.checkArgument( file.keyMetadata() == null, - "Cannot decrypt delete file: {} (use EncryptingFileIO)", + "Cannot decrypt delete file: %s (use EncryptingFileIO)", file.path()); - return newInputFile(file.path().toString()); + return newInputFile(file.path().toString(), file.fileSizeInBytes()); } default InputFile newInputFile(ManifestFile manifest) { Preconditions.checkArgument( manifest.keyMetadata() == null, - "Cannot decrypt manifest: {} (use EncryptingFileIO)", + "Cannot decrypt manifest: %s (use EncryptingFileIO)", manifest.path()); - return newInputFile(manifest.path()); + return newInputFile(manifest.path(), manifest.length()); } /** Get a {@link OutputFile} instance to write bytes to the file at the given path. */ 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 a74e574c9751..26c9bc133b13 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 @@ -22,6 +22,8 @@ import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.never; +import static org.mockito.Mockito.reset; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; @@ -38,6 +40,13 @@ import org.apache.hadoop.conf.Configuration; 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.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.aws.AwsProperties; @@ -74,6 +83,7 @@ import software.amazon.awssdk.services.s3.model.CreateBucketRequest; 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.S3Error; @ExtendWith(S3MockExtension.class) @@ -377,6 +387,54 @@ public void testResolvingFileIOLoad() { Assertions.assertThat(result).isInstanceOf(S3FileIO.class); } + @Test + public void testInputFileWithDataFile() throws IOException { + String location = "s3://bucket/path/to/data-file.parquet"; + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(location) + .withFileSizeInBytes(123L) + .withFormat(FileFormat.PARQUET) + .withRecordCount(123L) + .build(); + OutputStream outputStream = s3FileIO.newOutputFile(location).create(); + byte[] data = "testing".getBytes(); + outputStream.write(data); + outputStream.close(); + + InputFile inputFile = s3FileIO.newInputFile(dataFile); + reset(s3mock); + + Assertions.assertThat(inputFile.getLength()) + .as("Data file length should be determined from the file size stats") + .isEqualTo(123L); + verify(s3mock, never()).headObject(any(HeadObjectRequest.class)); + } + + @Test + public void testInputFileWithManifest() throws IOException { + String dataFileLocation = "s3://bucket/path/to/data-file-2.parquet"; + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(dataFileLocation) + .withFileSizeInBytes(123L) + .withFormat(FileFormat.PARQUET) + .withRecordCount(123L) + .build(); + String manifestLocation = "s3://bucket/path/to/manifest.avro"; + OutputFile outputFile = s3FileIO.newOutputFile(manifestLocation); + ManifestWriter writer = + ManifestFiles.write(PartitionSpec.unpartitioned(), outputFile); + writer.add(dataFile); + writer.close(); + ManifestFile manifest = writer.toManifestFile(); + InputFile inputFile = s3FileIO.newInputFile(manifest); + reset(s3mock); + + Assertions.assertThat(inputFile.getLength()).isEqualTo(manifest.length()); + verify(s3mock, never()).headObject(any(HeadObjectRequest.class)); + } + private void createRandomObjects(String prefix, int count) { S3URI s3URI = new S3URI(prefix); From aa4d27b57ff7efa4fff187dad9e451cdd67b4db6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 4 Apr 2024 22:22:08 +0200 Subject: [PATCH 0196/1019] Build: Bump org.testcontainers:testcontainers from 1.19.5 to 1.19.7 (#9912) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.5 to 1.19.7. - [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.19.5...1.19.7) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 41b87a22faf4..fa5b8f268d05 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.33" sqlite-jdbc = "3.45.2.0" -testcontainers = "1.19.5" +testcontainers = "1.19.7" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 99712f9b53cc727a5d8b50fefefa821f16a3d566 Mon Sep 17 00:00:00 2001 From: Haizhou Zhao Date: Thu, 4 Apr 2024 14:32:51 -0700 Subject: [PATCH 0197/1019] OpenAPI: Fix additionalProperties for SnapshotSummary (#9838) * [ISSUE-9837] Correct additionalProperties for SnapshotSummary Model * Include minimal change to generated python files --------- Co-authored-by: Haizhou Zhao Co-authored-by: Steve Zhang --- open-api/rest-catalog-open-api.py | 1 - open-api/rest-catalog-open-api.yaml | 4 ++-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 7bd97b69885f..e13aeeeec5bf 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -171,7 +171,6 @@ class SortOrder(BaseModel): class Summary(BaseModel): operation: Literal['append', 'replace', 'overwrite', 'delete'] - additionalProperties: Optional[str] = None class Snapshot(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 161d5e0fcff8..67dc235d3b20 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -1986,8 +1986,8 @@ components: operation: type: string enum: ["append", "replace", "overwrite", "delete"] - additionalProperties: - type: string + additionalProperties: + type: string schema-id: type: integer From ffb8ebca2a55051b1b06ae4ee3689c3425ab6289 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Fri, 5 Apr 2024 17:34:03 +0530 Subject: [PATCH 0198/1019] Hive, JDBC: Avoid NPE on Throwables without error msg (#10082) --- .../iceberg/jdbc/JdbcTableOperations.java | 2 +- .../iceberg/jdbc/JdbcViewOperations.java | 2 +- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 45 +++++++++++++ .../iceberg/jdbc/TestJdbcViewCatalog.java | 67 +++++++++++++++++++ .../apache/iceberg/hive/HiveClientPool.java | 3 +- .../iceberg/hive/TestHiveClientPool.java | 45 +++++++++++++ 6 files changed, 161 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java index 68d75b8e4f5f..619296ad3336 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcTableOperations.java @@ -138,7 +138,7 @@ public void doCommit(TableMetadata base, TableMetadata metadata) { throw new UncheckedSQLException(e, "Database warning"); } catch (SQLException e) { // SQLite doesn't set SQLState or throw SQLIntegrityConstraintViolationException - if (e.getMessage().contains("constraint failed")) { + if (e.getMessage() != null && e.getMessage().contains("constraint failed")) { throw new AlreadyExistsException("Table already exists: %s", tableIdentifier); } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java index 2ded12b102ae..10f46941d694 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcViewOperations.java @@ -129,7 +129,7 @@ protected void doCommit(ViewMetadata base, ViewMetadata metadata) { throw new UncheckedSQLException(e, "Database warning"); } catch (SQLException e) { // SQLite doesn't set SQLState or throw SQLIntegrityConstraintViolationException - if (e.getMessage().contains("constraint failed")) { + if (e.getMessage() != null && e.getMessage().contains("constraint failed")) { throw new AlreadyExistsException("View already exists: %s", viewIdentifier); } 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 48ad71734799..d8553a1858f5 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -23,6 +23,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.any; import java.io.File; import java.io.IOException; @@ -41,6 +42,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.DataFile; @@ -51,6 +53,7 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; import org.apache.iceberg.TableProperties; import org.apache.iceberg.Transaction; @@ -79,6 +82,8 @@ import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.mockito.MockedStatic; +import org.mockito.Mockito; import org.sqlite.SQLiteDataSource; public class TestJdbcCatalog extends CatalogTests { @@ -970,6 +975,46 @@ public void testCatalogWithCustomMetricsReporter() throws IOException { Assertions.assertThat(CustomMetricsReporter.COUNTER.get()).isEqualTo(2); } + @Test + public void testCommitExceptionWithoutMessage() { + TableIdentifier tableIdent = TableIdentifier.of("db", "tbl"); + BaseTable table = (BaseTable) catalog.buildTable(tableIdent, SCHEMA).create(); + TableOperations ops = table.operations(); + TableMetadata metadataV1 = ops.current(); + + table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); + ops.refresh(); + + try (MockedStatic mockedStatic = Mockito.mockStatic(JdbcUtil.class)) { + mockedStatic + .when(() -> JdbcUtil.loadTable(any(), any(), any(), any())) + .thenThrow(new SQLException()); + assertThatThrownBy(() -> ops.commit(ops.current(), metadataV1)) + .isInstanceOf(UncheckedSQLException.class) + .hasMessageStartingWith("Unknown failure"); + } + } + + @Test + public void testCommitExceptionWithMessage() { + TableIdentifier tableIdent = TableIdentifier.of("db", "tbl"); + BaseTable table = (BaseTable) catalog.buildTable(tableIdent, SCHEMA).create(); + TableOperations ops = table.operations(); + TableMetadata metadataV1 = ops.current(); + + table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); + ops.refresh(); + + try (MockedStatic mockedStatic = Mockito.mockStatic(JdbcUtil.class)) { + mockedStatic + .when(() -> JdbcUtil.loadTable(any(), any(), any(), any())) + .thenThrow(new SQLException("constraint failed")); + assertThatThrownBy(() -> ops.commit(ops.current(), metadataV1)) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("Table already exists: " + tableIdent); + } + } + public static class CustomMetricsReporter implements MetricsReporter { static final AtomicInteger COUNTER = new AtomicInteger(0); diff --git a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java index 8c0255764251..a66532d90f63 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcViewCatalog.java @@ -18,15 +18,28 @@ */ package org.apache.iceberg.jdbc; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; + +import java.sql.SQLException; import java.util.Map; import java.util.UUID; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.view.BaseView; import org.apache.iceberg.view.ViewCatalogTests; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewOperations; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import org.mockito.MockedStatic; +import org.mockito.Mockito; public class TestJdbcViewCatalog extends ViewCatalogTests { @@ -64,4 +77,58 @@ protected Catalog tableCatalog() { protected boolean requiresNamespaceCreate() { return true; } + + @Test + public void testCommitExceptionWithoutMessage() { + TableIdentifier identifier = TableIdentifier.of("namespace1", "view"); + BaseView view = + (BaseView) + catalog + .buildView(identifier) + .withQuery("spark", "select * from tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(Namespace.of("namespace1")) + .create(); + ViewOperations ops = view.operations(); + ViewMetadata metadataV1 = ops.current(); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + + try (MockedStatic mockedStatic = Mockito.mockStatic(JdbcUtil.class)) { + mockedStatic + .when(() -> JdbcUtil.loadView(any(), any(), any(), any())) + .thenThrow(new SQLException()); + assertThatThrownBy(() -> ops.commit(ops.current(), metadataV1)) + .isInstanceOf(UncheckedSQLException.class) + .hasMessageStartingWith("Unknown failure"); + } + } + + @Test + public void testCommitExceptionWithMessage() { + TableIdentifier identifier = TableIdentifier.of("namespace1", "view"); + BaseView view = + (BaseView) + catalog + .buildView(identifier) + .withQuery("spark", "select * from tbl") + .withSchema(SCHEMA) + .withDefaultNamespace(Namespace.of("namespace1")) + .create(); + ViewOperations ops = view.operations(); + ViewMetadata metadataV1 = ops.current(); + + view.updateProperties().set("k1", "v1").commit(); + ops.refresh(); + + try (MockedStatic mockedStatic = Mockito.mockStatic(JdbcUtil.class)) { + mockedStatic + .when(() -> JdbcUtil.loadView(any(), any(), any(), any())) + .thenThrow(new SQLException("constraint failed")); + assertThatThrownBy(() -> ops.commit(ops.current(), metadataV1)) + .isInstanceOf(AlreadyExistsException.class) + .hasMessageStartingWith("View already exists: " + identifier); + } + } } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java index 9bc232043a5d..b0ecb0ceffaa 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java @@ -73,7 +73,8 @@ protected IMetaStoreClient newClient() { } catch (MetaException e) { throw new RuntimeMetaException(e, "Failed to connect to Hive Metastore"); } catch (Throwable t) { - if (t.getMessage().contains("Another instance of Derby may have already booted")) { + if (t.getMessage() != null + && t.getMessage().contains("Another instance of Derby may have already booted")) { throw new RuntimeMetaException( t, "Failed to start an embedded metastore because embedded " diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java index 5a565d0e983c..2fe1bacf9dd1 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveClientPool.java @@ -20,6 +20,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -29,6 +30,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.MetaStoreUtils; import org.apache.hadoop.hive.metastore.api.Function; import org.apache.hadoop.hive.metastore.api.FunctionType; import org.apache.hadoop.hive.metastore.api.GetAllFunctionsResponse; @@ -39,6 +41,7 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; import org.mockito.Mockito; public class TestHiveClientPool { @@ -116,6 +119,48 @@ public void testGetTablesFailsForNonReconnectableException() throws Exception { .hasMessage("Another meta exception"); } + @Test + public void testExceptionMessages() { + try (MockedStatic mockedStatic = Mockito.mockStatic(MetaStoreUtils.class)) { + mockedStatic + .when(() -> MetaStoreUtils.newInstance(any(), any(), any())) + .thenThrow(new RuntimeException(new MetaException("Another meta exception"))); + assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) + .isInstanceOf(RuntimeMetaException.class) + .hasMessage("Failed to connect to Hive Metastore"); + } + + try (MockedStatic mockedStatic = Mockito.mockStatic(MetaStoreUtils.class)) { + mockedStatic + .when(() -> MetaStoreUtils.newInstance(any(), any(), any())) + .thenThrow(new RuntimeException(new MetaException())); + assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) + .isInstanceOf(RuntimeMetaException.class) + .hasMessage("Failed to connect to Hive Metastore"); + } + + try (MockedStatic mockedStatic = Mockito.mockStatic(MetaStoreUtils.class)) { + mockedStatic + .when(() -> MetaStoreUtils.newInstance(any(), any(), any())) + .thenThrow(new RuntimeException()); + assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) + .isInstanceOf(RuntimeMetaException.class) + .hasMessage("Failed to connect to Hive Metastore"); + } + + try (MockedStatic mockedStatic = Mockito.mockStatic(MetaStoreUtils.class)) { + mockedStatic + .when(() -> MetaStoreUtils.newInstance(any(), any(), any())) + .thenThrow(new RuntimeException("Another instance of Derby may have already booted")); + assertThatThrownBy(() -> clients.run(client -> client.getTables("default", "t"))) + .isInstanceOf(RuntimeMetaException.class) + .hasMessage( + "Failed to start an embedded metastore because embedded " + + "Derby supports only one client at a time. To fix this, use a metastore that supports " + + "multiple clients."); + } + } + @Test public void testConnectionFailureRestoreForMetaException() throws Exception { HiveMetaStoreClient hmsClient = newClient(); From efc272729abdcdf33049bb7f887f5601732a046b Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 5 Apr 2024 18:14:05 +0200 Subject: [PATCH 0199/1019] Core: Introduce ConfigResponseParser (#9952) --- .../apache/iceberg/rest/RESTSerializers.java | 22 ++- .../rest/responses/ConfigResponseParser.java | 72 +++++++++ .../org/apache/iceberg/util/JsonUtil.java | 20 +++ .../rest/responses/TestConfigResponse.java | 9 +- .../responses/TestConfigResponseParser.java | 138 ++++++++++++++++++ .../org/apache/iceberg/util/TestJsonUtil.java | 47 ++++++ 6 files changed, 303 insertions(+), 5 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.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 4311b9aa7701..341dda0e3f6c 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java @@ -55,6 +55,8 @@ import org.apache.iceberg.rest.requests.ReportMetricsRequestParser; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.requests.UpdateTableRequestParser; +import org.apache.iceberg.rest.responses.ConfigResponse; +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.ImmutableLoadViewResponse; @@ -111,7 +113,9 @@ public static void registerAll(ObjectMapper mapper) { .addSerializer(LoadViewResponse.class, new LoadViewResponseSerializer<>()) .addSerializer(ImmutableLoadViewResponse.class, new LoadViewResponseSerializer<>()) .addDeserializer(LoadViewResponse.class, new LoadViewResponseDeserializer<>()) - .addDeserializer(ImmutableLoadViewResponse.class, new LoadViewResponseDeserializer<>()); + .addDeserializer(ImmutableLoadViewResponse.class, new LoadViewResponseDeserializer<>()) + .addSerializer(ConfigResponse.class, new ConfigResponseSerializer<>()) + .addDeserializer(ConfigResponse.class, new ConfigResponseDeserializer<>()); mapper.registerModule(module); } @@ -402,4 +406,20 @@ public T deserialize(JsonParser p, DeserializationContext context) throws IOExce return (T) LoadViewResponseParser.fromJson(jsonNode); } } + + static class ConfigResponseSerializer extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + ConfigResponseParser.toJson(request, gen); + } + } + + static class ConfigResponseDeserializer extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) ConfigResponseParser.fromJson(jsonNode); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java new file mode 100644 index 000000000000..3240840e3e93 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.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.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.util.JsonUtil; + +public class ConfigResponseParser { + + private static final String DEFAULTS = "defaults"; + private static final String OVERRIDES = "overrides"; + + private ConfigResponseParser() {} + + public static String toJson(ConfigResponse response) { + return toJson(response, false); + } + + public static String toJson(ConfigResponse response, boolean pretty) { + return JsonUtil.generate(gen -> toJson(response, gen), pretty); + } + + public static void toJson(ConfigResponse response, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != response, "Invalid config response: null"); + + gen.writeStartObject(); + + JsonUtil.writeStringMap(DEFAULTS, response.defaults(), gen); + JsonUtil.writeStringMap(OVERRIDES, response.overrides(), gen); + + gen.writeEndObject(); + } + + public static ConfigResponse fromJson(String json) { + return JsonUtil.parse(json, ConfigResponseParser::fromJson); + } + + public static ConfigResponse fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse config response from null object"); + + ConfigResponse.Builder builder = ConfigResponse.builder(); + + if (json.hasNonNull(DEFAULTS)) { + builder.withDefaults(JsonUtil.getStringMapNullableValues(DEFAULTS, json)); + } + + if (json.hasNonNull(OVERRIDES)) { + builder.withOverrides(JsonUtil.getStringMapNullableValues(OVERRIDES, json)); + } + + return builder.build(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java index aa90c63f80da..2810ff5f23c0 100644 --- a/core/src/main/java/org/apache/iceberg/util/JsonUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/JsonUtil.java @@ -36,6 +36,7 @@ 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.Maps; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; public class JsonUtil { @@ -206,6 +207,25 @@ public static Map getStringMap(String property, JsonNode node) { return builder.build(); } + public static Map getStringMapNullableValues(String property, JsonNode node) { + Preconditions.checkArgument(node.has(property), "Cannot parse missing map: %s", property); + JsonNode pNode = node.get(property); + Preconditions.checkArgument( + pNode != null && !pNode.isNull() && pNode.isObject(), + "Cannot parse string map from non-object value: %s: %s", + property, + pNode); + + Map map = Maps.newHashMap(); + Iterator fields = pNode.fieldNames(); + while (fields.hasNext()) { + String field = fields.next(); + map.put(field, getStringOrNull(field, pNode)); + } + + return map; + } + public static String[] getStringArray(JsonNode node) { Preconditions.checkArgument( node != null && !node.isNull() && node.isArray(), diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java index 298ebc3cf5bb..273fe48e2dcb 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java @@ -145,15 +145,16 @@ public void testDeserializeInvalidResponse() { String jsonDefaultsHasWrongType = "{\"defaults\":[\"warehouse\",\"s3://bucket/warehouse\"],\"overrides\":{\"clients\":\"5\"}}"; Assertions.assertThatThrownBy(() -> deserialize(jsonDefaultsHasWrongType)) - .isInstanceOf(JsonProcessingException.class) + .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( - "Cannot deserialize value of type `java.util.LinkedHashMap`"); + "Cannot parse string map from non-object value: defaults: [\"warehouse\",\"s3://bucket/warehouse\"]"); String jsonOverridesHasWrongType = "{\"defaults\":{\"warehouse\":\"s3://bucket/warehouse\"},\"overrides\":\"clients\"}"; Assertions.assertThatThrownBy(() -> deserialize(jsonOverridesHasWrongType)) - .isInstanceOf(JsonProcessingException.class) - .hasMessageContaining("Cannot construct instance of `java.util.LinkedHashMap`"); + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot parse string map from non-object value: overrides: \"clients\""); Assertions.assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java new file mode 100644 index 000000000000..ec4c793c279f --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.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.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 java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +public class TestConfigResponseParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> ConfigResponseParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid config response: null"); + + assertThatThrownBy(() -> ConfigResponseParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse config response from null object"); + + ConfigResponse actual = ConfigResponseParser.fromJson("{}"); + ConfigResponse expected = ConfigResponse.builder().build(); + // ConfigResponse doesn't implement hashCode/equals + assertThat(actual.defaults()).isEqualTo(expected.defaults()).isEmpty(); + assertThat(actual.overrides()).isEqualTo(expected.overrides()).isEmpty(); + } + + @Test + public void unknownFields() { + ConfigResponse actual = ConfigResponseParser.fromJson("{\"x\": \"val\", \"y\": \"val2\"}"); + ConfigResponse expected = ConfigResponse.builder().build(); + // ConfigResponse doesn't implement hashCode/equals + assertThat(actual.defaults()).isEqualTo(expected.defaults()).isEmpty(); + assertThat(actual.overrides()).isEqualTo(expected.overrides()).isEmpty(); + } + + @Test + public void defaultsOnly() { + Map defaults = Maps.newHashMap(); + defaults.put("a", "1"); + defaults.put("b", null); + defaults.put("c", "2"); + defaults.put("d", null); + + ConfigResponse response = ConfigResponse.builder().withDefaults(defaults).build(); + String expectedJson = + "{\n" + + " \"defaults\" : {\n" + + " \"a\" : \"1\",\n" + + " \"b\" : null,\n" + + " \"c\" : \"2\",\n" + + " \"d\" : null\n" + + " },\n" + + " \"overrides\" : { }\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void overridesOnly() { + Map overrides = Maps.newHashMap(); + overrides.put("a", "1"); + overrides.put("b", null); + overrides.put("c", "2"); + overrides.put("d", null); + + ConfigResponse response = ConfigResponse.builder().withOverrides(overrides).build(); + String expectedJson = + "{\n" + + " \"defaults\" : { },\n" + + " \"overrides\" : {\n" + + " \"a\" : \"1\",\n" + + " \"b\" : null,\n" + + " \"c\" : \"2\",\n" + + " \"d\" : null\n" + + " }\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } + + @Test + public void roundTripSerde() { + Map defaults = Maps.newHashMap(); + defaults.put("key1", "1"); + defaults.put("key2", null); + + Map overrides = Maps.newHashMap(); + overrides.put("key3", "23"); + overrides.put("key4", null); + + ConfigResponse response = + ConfigResponse.builder().withDefaults(defaults).withOverrides(overrides).build(); + String expectedJson = + "{\n" + + " \"defaults\" : {\n" + + " \"key1\" : \"1\",\n" + + " \"key2\" : null\n" + + " },\n" + + " \"overrides\" : {\n" + + " \"key3\" : \"23\",\n" + + " \"key4\" : null\n" + + " }\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index 6b558599ac46..f5d92129fb3d 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -24,6 +24,7 @@ import java.util.List; import java.util.Map; 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.io.BaseEncoding; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -543,4 +544,50 @@ public void getStringMap() throws JsonProcessingException { Assertions.assertThat(JsonUtil.getStringMap("items", JsonUtil.mapper().readTree(json))) .isEqualTo(items); } + + @Test + public void getStringMapNullableValues() throws JsonProcessingException { + Assertions.assertThatThrownBy( + () -> JsonUtil.getStringMapNullableValues("items", JsonUtil.mapper().readTree("{}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing map: items"); + + Assertions.assertThatThrownBy( + () -> + JsonUtil.getStringMapNullableValues( + "items", JsonUtil.mapper().readTree("{\"items\": null}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse string map from non-object value: items: null"); + + Assertions.assertThatThrownBy( + () -> + JsonUtil.getStringMapNullableValues( + "items", JsonUtil.mapper().readTree("{\"items\": {\"a\":\"23\", \"b\":45}}"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse to a string value: b: 45"); + + Map itemsWithNullableValues = Maps.newHashMap(); + itemsWithNullableValues.put("a", null); + itemsWithNullableValues.put("b", null); + itemsWithNullableValues.put("c", "23"); + Assertions.assertThat( + JsonUtil.getStringMapNullableValues( + "items", + JsonUtil.mapper() + .readTree("{\"items\": {\"a\": null, \"b\": null, \"c\": \"23\"}}"))) + .isEqualTo(itemsWithNullableValues); + + String json = + JsonUtil.generate( + gen -> { + gen.writeStartObject(); + JsonUtil.writeStringMap("items", itemsWithNullableValues, gen); + gen.writeEndObject(); + }, + false); + + Assertions.assertThat( + JsonUtil.getStringMapNullableValues("items", JsonUtil.mapper().readTree(json))) + .isEqualTo(itemsWithNullableValues); + } } From 45cadb0c6a2869a3312dd0e2125c5a9e94ef5577 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 7 Apr 2024 13:20:22 +0200 Subject: [PATCH 0200/1019] Build: Bump mkdocs-material from 9.5.15 to 9.5.17 (#10092) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.15 to 9.5.17. - [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.15...9.5.17) --- 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 e170b507fb44..5132cdf017cc 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.15 +mkdocs-material==9.5.17 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 5398f2d0b8f193a5a2523a53dd2d4c12494b0178 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Mon, 8 Apr 2024 19:54:50 +0900 Subject: [PATCH 0201/1019] AWS: Migrate tests to JUnit5 (#10086) --- .baseline/checkstyle/checkstyle.xml | 4 +- .../aws/TestAssumeRoleAwsClientFactory.java | 23 +- .../aws/TestDefaultAwsClientFactory.java | 13 +- .../aws/dynamodb/TestDynamoDbCatalog.java | 127 +++++----- .../aws/dynamodb/TestDynamoDbLockManager.java | 74 +++--- .../apache/iceberg/aws/glue/GlueTestBase.java | 8 +- .../glue/TestGlueCatalogCommitFailure.java | 162 +++++++------ .../iceberg/aws/glue/TestGlueCatalogLock.java | 26 +-- .../aws/glue/TestGlueCatalogNamespace.java | 68 +++--- .../aws/glue/TestGlueCatalogTable.java | 217 ++++++++---------- .../lakeformation/LakeFormationTestBase.java | 15 +- .../TestLakeFormationAwsClientFactory.java | 26 +-- .../TestLakeFormationDataOperations.java | 19 +- .../TestLakeFormationMetadataOperations.java | 33 ++- .../aws/s3/TestS3FileIOIntegration.java | 73 +++--- .../iceberg/aws/s3/TestS3MultipartUpload.java | 35 ++- build.gradle | 1 + 17 files changed, 445 insertions(+), 479 deletions(-) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index ab7e66d71480..16b06c6bb1a5 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -145,7 +145,9 @@ org.apache.spark.sql.functions.*, org.apache.spark.sql.connector.iceberg.write.RowLevelOperation.Command.*, org.apache.spark.sql.connector.write.RowLevelOperation.Command.*, - org.junit.Assert.*"/> + org.junit.Assert.*, + org.assertj.core.api.Assertions.*, + org.assertj.core.api.Assumptions.*"/> diff --git a/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java index 99687777f67b..9845d31021d8 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.time.Duration; import java.util.Map; import java.util.UUID; @@ -26,13 +29,11 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.assertj.core.api.InstanceOfAssertFactories; import org.awaitility.Awaitility; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkServiceException; @@ -57,7 +58,7 @@ public class TestAssumeRoleAwsClientFactory { private Map assumeRoleProperties; private String policyName; - @Before + @BeforeEach public void before() { roleName = UUID.randomUUID().toString(); iam = @@ -95,7 +96,7 @@ public void before() { policyName = UUID.randomUUID().toString(); } - @After + @AfterEach public void after() { iam.deleteRolePolicy( DeleteRolePolicyRequest.builder().roleName(roleName).policyName(policyName).build()); @@ -134,7 +135,7 @@ public void testAssumeRoleGlueCatalog() { GlueCatalog glueCatalog = new GlueCatalog(); assumeRoleProperties.put("warehouse", "s3://path"); glueCatalog.initialize("test", assumeRoleProperties); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalog.createNamespace( Namespace.of("denied_" + UUID.randomUUID().toString().replace("-", "")))) @@ -177,7 +178,7 @@ public void testAssumeRoleS3FileIO() throws Exception { S3FileIO s3FileIO = new S3FileIO(); s3FileIO.initialize(assumeRoleProperties); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> s3FileIO .newInputFile("s3://" + AwsIntegTestUtil.testBucketName() + "/denied/file") @@ -189,7 +190,7 @@ public void testAssumeRoleS3FileIO() throws Exception { InputFile inputFile = s3FileIO.newInputFile("s3://" + AwsIntegTestUtil.testBucketName() + "/allowed/file"); - Assert.assertFalse("should be able to access file", inputFile.exists()); + assertThat(inputFile.exists()).isFalse(); } private void waitForIamConsistency() { @@ -199,7 +200,7 @@ private void waitForIamConsistency() { .ignoreExceptions() .untilAsserted( () -> - Assertions.assertThat( + assertThat( iam.getRolePolicy( GetRolePolicyRequest.builder() .roleName(roleName) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/TestDefaultAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/TestDefaultAwsClientFactory.java index 8e750a0280a4..28fd17234a92 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/TestDefaultAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/TestDefaultAwsClientFactory.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.aws.s3.S3FileIOProperties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.core.exception.SdkClientException; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.glue.GlueClient; @@ -39,7 +40,7 @@ public void testGlueEndpointOverride() { properties.put(AwsProperties.GLUE_CATALOG_ENDPOINT, "https://unknown:1234"); AwsClientFactory factory = AwsClientFactories.from(properties); GlueClient glueClient = factory.glue(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueClient.getDatabase(GetDatabaseRequest.builder().name("TEST").build())) .cause() .isInstanceOf(SdkClientException.class) @@ -52,7 +53,7 @@ public void testS3FileIoEndpointOverride() { properties.put(S3FileIOProperties.ENDPOINT, "https://unknown:1234"); AwsClientFactory factory = AwsClientFactories.from(properties); S3Client s3Client = factory.s3(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> s3Client.getObject(GetObjectRequest.builder().bucket("bucket").key("key").build())) .cause() @@ -67,7 +68,7 @@ public void testS3FileIoCredentialsOverride() { properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, "unknown"); AwsClientFactory factory = AwsClientFactories.from(properties); S3Client s3Client = factory.s3(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> s3Client.getObject( GetObjectRequest.builder() @@ -84,7 +85,7 @@ public void testDynamoDbEndpointOverride() { properties.put(AwsProperties.DYNAMODB_ENDPOINT, "https://unknown:1234"); AwsClientFactory factory = AwsClientFactories.from(properties); DynamoDbClient dynamoDbClient = factory.dynamo(); - Assertions.assertThatThrownBy(dynamoDbClient::listTables) + assertThatThrownBy(dynamoDbClient::listTables) .cause() .isInstanceOf(SdkClientException.class) .hasMessageContaining("Unable to execute HTTP request: unknown"); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java b/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java index 49ba0d6ee260..5ee6b3e1cf34 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.dynamodb; +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 java.util.UUID; @@ -42,11 +45,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest; import software.amazon.awssdk.services.dynamodb.model.GetItemRequest; @@ -67,7 +68,7 @@ public class TestDynamoDbCatalog { private static DynamoDbCatalog catalog; private static String testBucket; - @BeforeClass + @BeforeAll public static void beforeClass() { catalogTableName = genRandomName(); AwsClientFactory clientFactory = AwsClientFactories.defaultFactory(); @@ -84,7 +85,7 @@ public static void beforeClass() { "s3://" + testBucket + "/" + genRandomName())); } - @AfterClass + @AfterAll public static void afterClass() { dynamo.deleteTable(DeleteTableRequest.builder().tableName(catalogTableName).build()); } @@ -99,22 +100,23 @@ public void testCreateNamespace() { .tableName(catalogTableName) .key(DynamoDbCatalog.namespacePrimaryKey(namespace)) .build()); - Assert.assertTrue("namespace must exist", response.hasItem()); - Assert.assertEquals( - "namespace must be stored in DynamoDB", - namespace.toString(), - response.item().get("namespace").s()); - Assertions.assertThatThrownBy(() -> catalog.createNamespace(namespace)) + assertThat(response.hasItem()).as("namespace must exist").isTrue(); + assertThat(response.item()) + .as("namespace must be stored in DynamoDB") + .hasEntrySatisfying( + "namespace", + attributeValue -> assertThat(attributeValue.s()).isEqualTo(namespace.toString())); + assertThatThrownBy(() -> catalog.createNamespace(namespace)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("already exists"); } @Test public void testCreateNamespaceBadName() { - Assertions.assertThatThrownBy(() -> catalog.createNamespace(Namespace.of("a", "", "b"))) + assertThatThrownBy(() -> catalog.createNamespace(Namespace.of("a", "", "b"))) .isInstanceOf(ValidationException.class) .hasMessageContaining("must not be empty"); - Assertions.assertThatThrownBy(() -> catalog.createNamespace(Namespace.of("a", "b.c"))) + assertThatThrownBy(() -> catalog.createNamespace(Namespace.of("a", "b.c"))) .isInstanceOf(ValidationException.class) .hasMessageContaining("must not contain dot"); } @@ -128,7 +130,7 @@ public void testListSubNamespaces() { .collect(Collectors.toList()); catalog.createNamespace(parent); namespaceList.forEach(ns -> catalog.createNamespace(ns)); - Assert.assertEquals(4, catalog.listNamespaces(parent).size()); + assertThat(catalog.listNamespaces(parent)).hasSize(4); } @Test @@ -138,16 +140,16 @@ public void testNamespaceProperties() { properties.put("key1", "val1"); properties.put("key2", "val2"); catalog.createNamespace(namespace, properties); - Assert.assertEquals(properties, catalog.loadNamespaceMetadata(namespace)); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); properties.put("key3", "val3"); properties.put("key2", "val2-1"); catalog.setProperties(namespace, properties); - Assert.assertEquals(properties, catalog.loadNamespaceMetadata(namespace)); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); properties.remove("key3"); catalog.removeProperties(namespace, Sets.newHashSet("key3")); - Assert.assertEquals(properties, catalog.loadNamespaceMetadata(namespace)); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); } @Test @@ -162,16 +164,17 @@ public void testCreateTable() { .tableName(catalogTableName) .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)) .build()); - Assert.assertTrue("table must exist", response.hasItem()); - Assert.assertEquals( - "table must be stored in DynamoDB with table identifier as partition key", - tableIdentifier.toString(), - response.item().get("identifier").s()); - Assert.assertEquals( - "table must be stored in DynamoDB with namespace as sort key", - namespace.toString(), - response.item().get("namespace").s()); - Assertions.assertThatThrownBy(() -> catalog.createTable(tableIdentifier, SCHEMA)) + assertThat(response.hasItem()).as("table must exist").isTrue(); + assertThat(response.item()) + .as("table must be stored in DynamoDB with table identifier as partition key") + .hasEntrySatisfying( + "identifier", + attributeValue -> assertThat(attributeValue.s()).isEqualTo(tableIdentifier.toString())) + .as("table must be stored in DynamoDB with namespace as sort key") + .hasEntrySatisfying( + "namespace", + attributeValue -> assertThat(attributeValue.s()).isEqualTo(namespace.toString())); + assertThatThrownBy(() -> catalog.createTable(tableIdentifier, SCHEMA)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("already exists"); } @@ -180,12 +183,11 @@ public void testCreateTable() { public void testCreateTableBadName() { Namespace namespace = Namespace.of(genRandomName()); catalog.createNamespace(namespace); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.createTable(TableIdentifier.of(Namespace.empty(), "a"), SCHEMA)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Table namespace must not be empty"); - Assertions.assertThatThrownBy( - () -> catalog.createTable(TableIdentifier.of(namespace, "a.b"), SCHEMA)) + assertThatThrownBy(() -> catalog.createTable(TableIdentifier.of(namespace, "a.b"), SCHEMA)) .isInstanceOf(ValidationException.class) .hasMessageContaining("must not contain dot"); } @@ -199,7 +201,7 @@ public void testListTable() { .mapToObj(i -> TableIdentifier.of(namespace, genRandomName())) .collect(Collectors.toList()); tableIdentifiers.forEach(id -> catalog.createTable(id, SCHEMA)); - Assert.assertEquals(3, catalog.listTables(namespace).size()); + assertThat(catalog.listTables(namespace)).hasSize(3); } @Test @@ -219,16 +221,17 @@ public void testDropTable() { .get("p.metadata_location") .s(); catalog.dropTable(tableIdentifier, true); - Assert.assertFalse( - "table entry should not exist in dynamo", - dynamo - .getItem( - GetItemRequest.builder() - .tableName(catalogTableName) - .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)) - .build()) - .hasItem()); - Assertions.assertThatThrownBy( + assertThat( + dynamo + .getItem( + GetItemRequest.builder() + .tableName(catalogTableName) + .key(DynamoDbCatalog.tablePrimaryKey(tableIdentifier)) + .build()) + .hasItem()) + .as("table entry should not exist in dynamo") + .isFalse(); + assertThatThrownBy( () -> s3.headObject( HeadObjectRequest.builder() @@ -251,12 +254,12 @@ public void testRenameTable() { TableIdentifier tableIdentifier = TableIdentifier.of(namespace, genRandomName()); catalog.createTable(tableIdentifier, SCHEMA); TableIdentifier tableIdentifier2 = TableIdentifier.of(namespace2, genRandomName()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.renameTable(TableIdentifier.of(namespace, "a"), tableIdentifier2)) .isInstanceOf(NoSuchTableException.class) .hasMessageContaining("does not exist"); - Assertions.assertThatThrownBy(() -> catalog.renameTable(tableIdentifier, tableIdentifier)) + assertThatThrownBy(() -> catalog.renameTable(tableIdentifier, tableIdentifier)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("already exists"); @@ -284,10 +287,9 @@ public void testRenameTable() { .get("p.metadata_location") .s(); - Assert.assertEquals( - "metadata location should be copied to new table entry", - metadataLocation, - metadataLocation2); + assertThat(metadataLocation2) + .as("metadata location should be copied to new table entry") + .isEqualTo(metadataLocation); } @Test @@ -299,7 +301,7 @@ public void testUpdateTable() { Table table = catalog.loadTable(tableIdentifier); table.updateSchema().addColumn("data", Types.StringType.get()).commit(); table.refresh(); - Assert.assertEquals(2, table.schema().columns().size()); + assertThat(table.schema().columns()).hasSize(2); } @Test @@ -326,7 +328,7 @@ public void testConcurrentCommits() throws Exception { })) .get(); - Assert.assertEquals(2, table.schema().columns().size()); + assertThat(table.schema().columns()).hasSize(2); } @Test @@ -340,7 +342,7 @@ public void testDropNamespace() { .tableName(catalogTableName) .key(DynamoDbCatalog.namespacePrimaryKey(namespace)) .build()); - Assert.assertFalse("namespace must not exist", response.hasItem()); + assertThat(response.hasItem()).as("namespace must not exist").isFalse(); } @Test @@ -350,17 +352,17 @@ public void testRegisterTable() { TableIdentifier identifier = TableIdentifier.of(namespace, catalogTableName); catalog.createTable(identifier, SCHEMA); Table registeringTable = catalog.loadTable(identifier); - Assertions.assertThat(catalog.dropTable(identifier, false)).isTrue(); + assertThat(catalog.dropTable(identifier, false)).isTrue(); TableOperations ops = ((HasTableOperations) registeringTable).operations(); String metadataLocation = ((DynamoDbTableOperations) ops).currentMetadataLocation(); Table registeredTable = catalog.registerTable(identifier, metadataLocation); - Assertions.assertThat(registeredTable).isNotNull(); + assertThat(registeredTable).isNotNull(); String expectedMetadataLocation = ((HasTableOperations) registeredTable).operations().current().metadataFileLocation(); - Assertions.assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); - Assertions.assertThat(catalog.loadTable(identifier)).isNotNull(); - Assertions.assertThat(catalog.dropTable(identifier, true)).isTrue(); - Assertions.assertThat(catalog.dropNamespace(namespace)).isTrue(); + assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); + assertThat(catalog.loadTable(identifier)).isNotNull(); + assertThat(catalog.dropTable(identifier, true)).isTrue(); + assertThat(catalog.dropNamespace(namespace)).isTrue(); } @Test @@ -373,8 +375,7 @@ public void testDefaultWarehousePathWithLocation() { properties.put(DynamoDbCatalog.defaultLocationProperty(), defaultLocation); catalog.createNamespace(namespace, properties); String tableName = genRandomName(); - Assertions.assertThat( - catalog.defaultWarehouseLocation(TableIdentifier.of(namespaceName, tableName))) + assertThat(catalog.defaultWarehouseLocation(TableIdentifier.of(namespaceName, tableName))) .isEqualTo(defaultLocation + "/" + tableName); } @@ -387,11 +388,11 @@ public void testRegisterExistingTable() { Table registeringTable = catalog.loadTable(identifier); TableOperations ops = ((HasTableOperations) registeringTable).operations(); String metadataLocation = ((DynamoDbTableOperations) ops).currentMetadataLocation(); - Assertions.assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) + assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("already exists"); - Assertions.assertThat(catalog.dropTable(identifier, true)).isTrue(); - Assertions.assertThat(catalog.dropNamespace(namespace)).isTrue(); + assertThat(catalog.dropTable(identifier, true)).isTrue(); + assertThat(catalog.dropNamespace(namespace)).isTrue(); } private static String genRandomName() { diff --git a/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbLockManager.java b/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbLockManager.java index eade5713bc7b..120a4d702681 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbLockManager.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbLockManager.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.dynamodb; +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 java.util.UUID; @@ -29,12 +32,10 @@ import org.apache.iceberg.aws.AwsClientFactories; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +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.mockito.Mockito; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.dynamodb.model.AttributeValue; @@ -56,27 +57,27 @@ public class TestDynamoDbLockManager { private String entityId; private String ownerId; - @BeforeClass + @BeforeAll public static void beforeClass() { lockTableName = genTableName(); dynamo = AwsClientFactories.defaultFactory().dynamo(); } - @Before + @BeforeEach public void before() { lockManager = new DynamoDbLockManager(dynamo, lockTableName); entityId = UUID.randomUUID().toString(); ownerId = UUID.randomUUID().toString(); } - @AfterClass + @AfterAll public static void afterClass() { dynamo.deleteTable(DeleteTableRequest.builder().tableName(lockTableName).build()); } @Test public void testTableCreation() { - Assert.assertTrue(lockManager.tableExists(lockTableName)); + assertThat(lockManager.tableExists(lockTableName)).isTrue(); } @Test @@ -86,11 +87,15 @@ public void testAcquireOnceSingleProcess() { key.put("entityId", AttributeValue.builder().s(entityId).build()); GetItemResponse response = dynamo.getItem(GetItemRequest.builder().tableName(lockTableName).key(key).build()); - Assert.assertTrue("should have item in dynamo after acquire", response.hasItem()); - Assert.assertEquals(entityId, response.item().get("entityId").s()); - Assert.assertEquals(ownerId, response.item().get("ownerId").s()); - Assert.assertNotNull(response.item().get("version")); - Assert.assertNotNull(response.item().get("leaseDurationMs")); + assertThat(response.hasItem()).as("should have item in dynamo after acquire").isTrue(); + assertThat(response.item()) + .hasEntrySatisfying( + "entityId", attributeValue -> assertThat(attributeValue.s()).isEqualTo(entityId)) + .hasEntrySatisfying( + "ownerId", attributeValue -> assertThat(attributeValue.s()).isEqualTo(ownerId)) + .hasEntrySatisfying("version", attributeValue -> assertThat(attributeValue).isNotNull()) + .hasEntrySatisfying( + "leaseDurationMs", attributeValue -> assertThat(attributeValue).isNotNull()); } @Test @@ -114,29 +119,26 @@ public void testAcquireOnceMultiProcesses() throws Exception { }) .collect(Collectors.toList())) .get(); - Assert.assertEquals( - "should have only 1 process succeeded in acquisition", - 1, - results.stream().filter(s -> s).count()); + assertThat(results).as("should have only 1 process succeeded in acquisition").hasSize(1); } @Test public void testReleaseAndAcquire() { - Assert.assertTrue(lockManager.acquire(entityId, ownerId)); - Assert.assertTrue(lockManager.release(entityId, ownerId)); - Assert.assertTrue(lockManager.acquire(entityId, ownerId)); + assertThat(lockManager.acquire(entityId, ownerId)).isTrue(); + assertThat(lockManager.release(entityId, ownerId)).isTrue(); + assertThat(lockManager.acquire(entityId, ownerId)).isTrue(); } @Test public void testReleaseWithWrongOwner() { - Assert.assertTrue(lockManager.acquire(entityId, ownerId)); - Assert.assertFalse(lockManager.release(entityId, UUID.randomUUID().toString())); + assertThat(lockManager.acquire(entityId, ownerId)).isTrue(); + assertThat(lockManager.release(entityId, UUID.randomUUID().toString())).isFalse(); } @Test @SuppressWarnings({"DangerousCompletableFutureUsage", "FutureReturnValueIgnored"}) public void testAcquireSingleProcess() throws Exception { - Assert.assertTrue(lockManager.acquire(entityId, ownerId)); + assertThat(lockManager.acquire(entityId, ownerId)).isTrue(); String oldOwner = ownerId; CompletableFuture.supplyAsync( @@ -146,14 +148,14 @@ public void testAcquireSingleProcess() throws Exception { } catch (InterruptedException e) { throw new RuntimeException(e); } - Assert.assertTrue(lockManager.release(entityId, oldOwner)); + assertThat(lockManager.release(entityId, oldOwner)).isTrue(); return null; }); ownerId = UUID.randomUUID().toString(); long start = System.currentTimeMillis(); - Assert.assertTrue(lockManager.acquire(entityId, ownerId)); - Assert.assertTrue("should succeed after 5 seconds", System.currentTimeMillis() - start >= 5000); + assertThat(lockManager.acquire(entityId, ownerId)).isTrue(); + assertThat(System.currentTimeMillis() - start).isGreaterThanOrEqualTo(5000); } @Test @@ -181,18 +183,15 @@ public void testAcquireMultiProcessAllSucceed() throws Exception { } catch (InterruptedException e) { throw new RuntimeException(e); } - Assert.assertTrue(threadLocalLockManager.release(entityId, owner)); + assertThat(threadLocalLockManager.release(entityId, owner)) + .isTrue(); } return succeeded; }) .collect(Collectors.toList())) .get(); - Assert.assertEquals( - "all lock acquire should succeed sequentially", - 16, - results.stream().filter(s -> s).count()); - Assert.assertTrue( - "must take more than 16 seconds", System.currentTimeMillis() - start >= 16000); + assertThat(results).as("all lock acquire should succeed sequentially").hasSize(16); + assertThat(System.currentTimeMillis() - start).isGreaterThanOrEqualTo(16000); } @Test @@ -217,8 +216,7 @@ public void testAcquireMultiProcessOnlyOneSucceed() throws Exception { }) .collect(Collectors.toList())) .get(); - Assert.assertEquals( - "only 1 thread should have acquired the lock", 1, results.stream().filter(s -> s).count()); + assertThat(results).as("only 1 thread should have acquired the lock").hasSize(1); } @Test @@ -227,7 +225,7 @@ public void testTableCreationFailure() { Mockito.doThrow(ResourceNotFoundException.class) .when(dynamo2) .describeTable(Mockito.any(DescribeTableRequest.class)); - Assertions.assertThatThrownBy(() -> new DynamoDbLockManager(dynamo2, lockTableName)) + assertThatThrownBy(() -> new DynamoDbLockManager(dynamo2, lockTableName)) .as("should fail to initialize the lock manager") .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Cannot find Dynamo table"); 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 d900f133bd4b..2a810f06508f 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 @@ -34,8 +34,8 @@ 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.AfterClass; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; @@ -75,7 +75,7 @@ public class GlueTestBase { static final String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; - @BeforeClass + @BeforeAll public static void beforeClass() { glueCatalog = new GlueCatalog(); AwsProperties awsProperties = new AwsProperties(); @@ -103,7 +103,7 @@ public static void beforeClass() { ImmutableMap.of()); } - @AfterClass + @AfterAll public static void afterClass() { AwsIntegTestUtil.cleanGlueCatalog(glue, namespaces); AwsIntegTestUtil.cleanS3Bucket(s3, testBucketName, testPathPrefix); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java index 079423cd1245..f1748737874e 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.glue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.File; import java.util.Map; import org.apache.iceberg.BaseMetastoreTableOperations; @@ -32,9 +35,7 @@ import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.mockito.Mockito; import software.amazon.awssdk.core.metrics.CoreMetric; import software.amazon.awssdk.metrics.MetricCollector; @@ -60,14 +61,16 @@ public void testFailedCommit() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, new CommitFailedException("Datacenter on fire")); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessageContaining("Datacenter on fire"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -80,18 +83,17 @@ public void testFailedCommitThrowsUnknownException() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageContaining("Datacenter on fire"); Mockito.verify(spyOps, Mockito.times(1)).refresh(); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals( - "Client could not determine outcome so new metadata file should also exist", - 3, - metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Client could not determine outcome so new metadata file should also exist") + .isEqualTo(3); } @Test @@ -104,7 +106,7 @@ public void testConcurrentModificationExceptionDoesNotCheckCommitStatus() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, ConcurrentModificationException.builder().build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessageContaining("Glue detected concurrent update") .cause() @@ -112,9 +114,11 @@ public void testConcurrentModificationExceptionDoesNotCheckCommitStatus() { Mockito.verify(spyOps, Mockito.times(0)).refresh(); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -133,10 +137,13 @@ public void testCheckCommitStatusAfterRetries() { simulateRetriedCommit(spyOps, true /* report retry */); updateTable(table, spyOps); - Assert.assertNotEquals("Current metadata should have changed", metadataV1, spyOps.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(spyOps.current())); - Assert.assertEquals( - "No new metadata files should exist", 2, metadataFileCount(spyOps.current())); + assertThat(spyOps.current()) + .as("Current metadata should have changed") + .isNotEqualTo(metadataV1); + assertThat(metadataFileExists(spyOps.current())).isTrue(); + assertThat(metadataFileCount(spyOps.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -158,14 +165,14 @@ public void testNoRetryAwarenessCorruptsTable() { // still work. If or when that happens, we can re-evaluate whether the mechanism is still // necessary. simulateRetriedCommit(spyOps, false /* hide retry */); - Assertions.assertThatThrownBy(() -> updateTable(table, spyOps)) + assertThatThrownBy(() -> updateTable(table, spyOps)) .as("Hidden retry causes writer to conflict with itself") .isInstanceOf(CommitFailedException.class) .hasMessageContaining("Glue detected concurrent update") .cause() .isInstanceOf(ConcurrentModificationException.class); - Assertions.assertThatThrownBy(() -> glueCatalog.loadTable(tableId)) + assertThatThrownBy(() -> glueCatalog.loadTable(tableId)) .as("Table still accessible despite hidden retry, underlying assumptions may have changed") .isInstanceOf(NotFoundException.class) .hasMessageContaining("Location does not exist"); @@ -206,13 +213,11 @@ public void testCommitThrowsExceptionWhileSucceeded() { spyOps.commit(metadataV2, metadataV1); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current()); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals( - "Commit should have been successful and new metadata file should be made", - 3, - metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Commit should have been successful and new metadata file should be made") + .isEqualTo(3); } @Test @@ -226,19 +231,17 @@ public void testFailedCommitThrowsUnknownExceptionWhenStatusCheckFails() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps); breakFallbackCatalogCommitCheck(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageContaining("Datacenter on fire"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals( - "Client could not determine outcome so new metadata file should also exist", - 3, - metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("Client could not determine outcome so new metadata file should also exist") + .isEqualTo(3); } @Test @@ -252,14 +255,13 @@ public void testSucceededCommitThrowsUnknownException() { GlueTableOperations spyOps = Mockito.spy(ops); commitAndThrowException(ops, spyOps); breakFallbackCatalogCommitCheck(spyOps); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitStateUnknownException.class) .hasMessageContaining("Datacenter on fire"); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", ops.current(), metadataV2); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())).isTrue(); } /** @@ -293,13 +295,11 @@ public void testExceptionThrownInConcurrentCommit() { spyOps.commit(metadataV2, metadataV1); ops.refresh(); - Assert.assertNotEquals("Current metadata should have changed", metadataV2, ops.current()); - Assert.assertTrue( - "Current metadata file should still exist", metadataFileExists(ops.current())); - Assert.assertEquals( - "The column addition from the concurrent commit should have been successful", - 2, - ops.current().schema().columns().size()); + assertThat(ops.current()).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(ops.current())).isTrue(); + assertThat(ops.current().schema().columns()) + .as("The column addition from the concurrent commit should have been successful") + .hasSize(2); } @SuppressWarnings("unchecked") @@ -340,14 +340,16 @@ public void testCreateTableWithInvalidDB() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, EntityNotFoundException.builder().build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(NotFoundException.class) .hasMessageContaining("because Glue cannot find the requested entity"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -360,13 +362,15 @@ public void testGlueAccessDeniedException() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, AccessDeniedException.builder().build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(ForbiddenException.class) .hasMessageContaining("because Glue cannot access the requested resources"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -379,15 +383,17 @@ public void testGlueValidationException() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, ValidationException.builder().build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(org.apache.iceberg.exceptions.ValidationException.class) .hasMessageContaining( "because Glue encountered a validation exception while accessing requested resources"); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -400,13 +406,15 @@ public void testS3Exception() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, S3Exception.builder().statusCode(300).build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(S3Exception.class) .hasMessage(null); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -419,14 +427,16 @@ public void testOtherGlueException() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, GlueException.builder().statusCode(300).build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(GlueException.class) .hasMessage(null); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } @Test @@ -439,14 +449,16 @@ public void testInternalServerErrorRetryCommit() { GlueTableOperations spyOps = Mockito.spy(ops); failCommitAndThrowException(spyOps, GlueException.builder().statusCode(500).build()); - Assertions.assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) .isInstanceOf(CommitFailedException.class) .hasMessage(null); ops.refresh(); - Assert.assertEquals("Current metadata should not have changed", metadataV2, ops.current()); - Assert.assertTrue("Current metadata should still exist", metadataFileExists(metadataV2)); - Assert.assertEquals("No new metadata files should exist", 2, metadataFileCount(ops.current())); + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).isTrue(); + assertThat(metadataFileCount(ops.current())) + .as("No new metadata files should exist") + .isEqualTo(2); } private Table setupTable() { @@ -462,7 +474,7 @@ private TableMetadata updateTable(Table table, GlueTableOperations ops) { TableMetadata metadataV2 = ops.current(); - Assert.assertEquals(2, metadataV2.schema().columns().size()); + assertThat(metadataV2.schema().columns()).hasSize(2); return metadataV2; } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java index 825f2a330533..53ec2a252fea 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aws.glue; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.Duration; import java.util.List; import java.util.UUID; @@ -40,10 +42,9 @@ import org.apache.iceberg.relocated.com.google.common.util.concurrent.MoreExecutors; import org.apache.iceberg.util.Tasks; import org.awaitility.Awaitility; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.dynamodb.DynamoDbClient; import software.amazon.awssdk.services.dynamodb.model.DeleteTableRequest; @@ -52,7 +53,7 @@ public class TestGlueCatalogLock extends GlueTestBase { private static String lockTableName; private static DynamoDbClient dynamo; - @BeforeClass + @BeforeAll public static void beforeClass() { GlueTestBase.beforeClass(); String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; @@ -71,7 +72,7 @@ public static void beforeClass() { ImmutableMap.of()); } - @AfterClass + @AfterAll public static void afterClass() { GlueTestBase.afterClass(); dynamo.deleteTable(DeleteTableRequest.builder().tableName(lockTableName).build()); @@ -107,12 +108,8 @@ public void testParallelCommitMultiThreadSingleCommit() { .run(i -> pendingCommits.get(i).commit()); table.refresh(); - Assert.assertEquals( - "Commits should all succeed sequentially", nThreads, table.history().size()); - Assert.assertEquals( - "Should have all manifests", - nThreads, - table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.history()).as("Commits should all succeed sequentially").hasSize(nThreads); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(nThreads); } @Test @@ -153,8 +150,7 @@ public void testParallelCommitMultiThreadMultiCommit() { }); table.refresh(); - Assert.assertEquals("Commits should all succeed sequentially", 20, table.history().size()); - Assert.assertEquals( - "should have 20 manifests", 20, table.currentSnapshot().allManifests(table.io()).size()); + assertThat(table.history()).as("Commits should all succeed sequentially").hasSize(20); + assertThat(table.currentSnapshot().allManifests(table.io())).hasSize(20); } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java index 2c821f749c0c..f36207005116 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.glue; +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 java.util.UUID; @@ -29,9 +32,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.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.CreateTableRequest; import software.amazon.awssdk.services.glue.model.Database; import software.amazon.awssdk.services.glue.model.EntityNotFoundException; @@ -44,8 +45,7 @@ public class TestGlueCatalogNamespace extends GlueTestBase { public void testCreateNamespace() { String namespace = getRandomName(); namespaces.add(namespace); - Assertions.assertThatThrownBy( - () -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) + assertThatThrownBy(() -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) .as("namespace does not exist before create") .isInstanceOf(EntityNotFoundException.class) .hasMessageContaining("not found"); @@ -61,20 +61,17 @@ public void testCreateNamespace() { glueCatalog.createNamespace(ns, properties); Database database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); - Assert.assertEquals("namespace must equal database name", namespace, database.name()); - Assert.assertEquals( - "namespace description should be set", "description", database.description()); - Assert.assertEquals( - "namespace location should be set", "s3://location", database.locationUri()); - Assert.assertEquals( - "namespace parameters should be set", ImmutableMap.of("key", "val"), database.parameters()); - Assert.assertEquals(properties, glueCatalog.loadNamespaceMetadata(ns)); + assertThat(database.name()).isEqualTo(namespace); + assertThat(database.description()).isEqualTo("description"); + assertThat(database.locationUri()).isEqualTo("s3://location"); + assertThat(database.parameters()).containsEntry("key", "val"); + assertThat(glueCatalog.loadNamespaceMetadata(ns)).isEqualTo(properties); } @Test public void testCreateDuplicate() { String namespace = createNamespace(); - Assertions.assertThatThrownBy(() -> glueCatalog.createNamespace(Namespace.of(namespace))) + assertThatThrownBy(() -> glueCatalog.createNamespace(Namespace.of(namespace))) .as("should not create namespace with the same name") .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("it already exists in Glue"); @@ -86,7 +83,7 @@ public void testCreateBadName() { Lists.newArrayList(Namespace.of("db-1"), Namespace.of("db", "db2")); for (Namespace namespace : invalidNamespaces) { - Assertions.assertThatThrownBy(() -> glueCatalog.createNamespace(namespace)) + assertThatThrownBy(() -> glueCatalog.createNamespace(namespace)) .as("should not create namespace with invalid or nested names") .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot convert namespace"); @@ -96,17 +93,17 @@ public void testCreateBadName() { @Test public void testNamespaceExists() { String namespace = createNamespace(); - Assert.assertTrue(glueCatalog.namespaceExists(Namespace.of(namespace))); + assertThat(glueCatalog.namespaceExists(Namespace.of(namespace))).isTrue(); } @Test public void testListNamespace() { String namespace = createNamespace(); List namespaceList = glueCatalog.listNamespaces(); - Assert.assertFalse(namespaceList.isEmpty()); - Assert.assertTrue(namespaceList.contains(Namespace.of(namespace))); + assertThat(namespaceList).isNotEmpty(); + assertThat(namespaceList).contains(Namespace.of(namespace)); namespaceList = glueCatalog.listNamespaces(Namespace.of(namespace)); - Assert.assertTrue(namespaceList.isEmpty()); + assertThat(namespaceList).isEmpty(); } @Test @@ -121,12 +118,9 @@ public void testNamespaceProperties() { glueCatalog.setProperties(Namespace.of(namespace), properties); Database database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); - Assert.assertTrue(database.parameters().containsKey("key")); - Assert.assertEquals("val", database.parameters().get("key")); - Assert.assertTrue(database.parameters().containsKey("key2")); - Assert.assertEquals("val2", database.parameters().get("key2")); - Assert.assertEquals("s3://test", database.locationUri()); - Assert.assertEquals("description", database.description()); + assertThat(database.parameters()).containsEntry("key", "val").containsEntry("key2", "val2"); + assertThat(database.locationUri()).isEqualTo("s3://test"); + assertThat(database.description()).isEqualTo("description"); // remove properties glueCatalog.removeProperties( Namespace.of(namespace), @@ -135,11 +129,9 @@ public void testNamespaceProperties() { IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, IcebergToGlueConverter.GLUE_DESCRIPTION_KEY)); database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); - Assert.assertFalse(database.parameters().containsKey("key")); - Assert.assertTrue(database.parameters().containsKey("key2")); - Assert.assertEquals("val2", database.parameters().get("key2")); - Assert.assertNull(database.locationUri()); - Assert.assertNull(database.description()); + assertThat(database.parameters()).doesNotContainKey("key").containsEntry("key2", "val2"); + assertThat(database.locationUri()).isNull(); + assertThat(database.description()).isNull(); // add back properties = Maps.newHashMap(); properties.put("key", "val"); @@ -147,20 +139,16 @@ public void testNamespaceProperties() { properties.put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, "description2"); glueCatalog.setProperties(Namespace.of(namespace), properties); database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); - Assert.assertTrue(database.parameters().containsKey("key")); - Assert.assertEquals("val", database.parameters().get("key")); - Assert.assertTrue(database.parameters().containsKey("key2")); - Assert.assertEquals("val2", database.parameters().get("key2")); - Assert.assertEquals("s3://test2", database.locationUri()); - Assert.assertEquals("description2", database.description()); + assertThat(database.parameters()).containsEntry("key", "val").containsEntry("key2", "val2"); + assertThat(database.locationUri()).isEqualTo("s3://test2"); + assertThat(database.description()).isEqualTo("description2"); } @Test public void testDropNamespace() { String namespace = createNamespace(); glueCatalog.dropNamespace(Namespace.of(namespace)); - Assertions.assertThatThrownBy( - () -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) + assertThatThrownBy(() -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) .as("namespace should not exist after deletion") .isInstanceOf(EntityNotFoundException.class) .hasMessageContaining("not found"); @@ -170,7 +158,7 @@ public void testDropNamespace() { public void testDropNamespaceThatContainsOnlyIcebergTable() { String namespace = createNamespace(); createTable(namespace); - Assertions.assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of(namespace))) + assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of(namespace))) .as("namespace should not be dropped when still has Iceberg table") .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageContaining("still contains Iceberg tables"); @@ -184,7 +172,7 @@ public void testDropNamespaceThatContainsNonIcebergTable() { .databaseName(namespace) .tableInput(TableInput.builder().name(UUID.randomUUID().toString()).build()) .build()); - Assertions.assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of(namespace))) + assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of(namespace))) .as("namespace should not be dropped when still has non-Iceberg table") .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageContaining("still contains non-Iceberg tables"); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index dc39d59e731e..7fb1f4ed1987 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -19,6 +19,8 @@ package org.apache.iceberg.aws.glue; import static org.apache.iceberg.expressions.Expressions.truncate; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Locale; @@ -49,9 +51,7 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.util.LockManagers; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.CreateTableRequest; import software.amazon.awssdk.services.glue.model.EntityNotFoundException; @@ -84,43 +84,35 @@ public void testCreateTable() { // verify table exists in Glue GetTableResponse response = glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); - Assert.assertEquals(namespace, response.table().databaseName()); - Assert.assertEquals(tableName, response.table().name()); - Assert.assertEquals( - BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH), - response.table().parameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP)); - Assert.assertTrue( - response - .table() - .parameters() - .containsKey(BaseMetastoreTableOperations.METADATA_LOCATION_PROP)); - Assert.assertEquals( - schema.columns().size(), response.table().storageDescriptor().columns().size()); - Assert.assertEquals(partitionSpec.fields().size(), response.table().partitionKeys().size()); - Assert.assertEquals( - "additionalLocations should match", - tableLocationProperties.values().stream().sorted().collect(Collectors.toList()), - response.table().storageDescriptor().additionalLocations().stream() - .sorted() - .collect(Collectors.toList())); + assertThat(response.table().databaseName()).isEqualTo(namespace); + assertThat(response.table().name()).isEqualTo(tableName); + assertThat(response.table().parameters()) + .containsEntry( + BaseMetastoreTableOperations.TABLE_TYPE_PROP, + BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ENGLISH)) + .containsKey(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); + assertThat(response.table().storageDescriptor().columns()).hasSameSizeAs(schema.columns()); + assertThat(response.table().partitionKeys()).hasSameSizeAs(partitionSpec.fields()); + assertThat(response.table().storageDescriptor().additionalLocations()) + .isEqualTo(tableLocationProperties.values()); // verify metadata file exists in S3 String metaLocation = response.table().parameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); String key = metaLocation.split(testBucketName, -1)[1].substring(1); s3.headObject(HeadObjectRequest.builder().bucket(testBucketName).key(key).build()); Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - Assert.assertEquals(partitionSpec, table.spec()); - Assert.assertEquals(schema.toString(), table.schema().toString()); - Assert.assertEquals( - tableDescription, table.properties().get(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY)); - Assert.assertEquals(tableDescription, response.table().description()); + assertThat(table.spec()).isEqualTo(partitionSpec); + assertThat(table.schema()).asString().isEqualTo(schema.toString()); + assertThat(table.properties()) + .containsEntry(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, tableDescription); + assertThat(response.table().description()).isEqualTo(tableDescription); } @Test public void testCreateTableDuplicate() { String namespace = createNamespace(); String tableName = createTable(namespace); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalog.createTable( TableIdentifier.of(namespace, tableName), schema, partitionSpec)) @@ -132,7 +124,7 @@ public void testCreateTableDuplicate() { @Test public void testCreateTableBadName() { String namespace = createNamespace(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalog.createTable( TableIdentifier.of(namespace, "table-1"), schema, partitionSpec)) @@ -167,20 +159,17 @@ public void testCreateAndLoadTableWithoutWarehouseLocation() { @Test public void testListTables() { String namespace = createNamespace(); - Assert.assertTrue( - "list namespace should have nothing before table creation", - glueCatalog.listTables(Namespace.of(namespace)).isEmpty()); + assertThat(glueCatalog.listTables(Namespace.of(namespace))).isEmpty(); String tableName = createTable(namespace); List tables = glueCatalog.listTables(Namespace.of(namespace)); - Assert.assertEquals(1, tables.size()); - Assert.assertEquals(TableIdentifier.of(namespace, tableName), tables.get(0)); + assertThat(tables).hasSize(1).first().isEqualTo(TableIdentifier.of(namespace, tableName)); } @Test public void testTableExists() { String namespace = createNamespace(); String tableName = createTable(namespace); - Assert.assertTrue(glueCatalog.tableExists(TableIdentifier.of(namespace, tableName))); + assertThat(glueCatalog.tableExists(TableIdentifier.of(namespace, tableName))).isTrue(); } @Test @@ -190,14 +179,14 @@ public void testUpdateTable() { // current should be null TableOperations ops = glueCatalog.newTableOps(TableIdentifier.of(namespace, tableName)); TableMetadata current = ops.current(); - Assert.assertNull(current); + assertThat(current).isNull(); // create table, refresh should update createTable(namespace, tableName); current = ops.refresh(); - Assert.assertEquals(schema.toString(), current.schema().toString()); - Assert.assertEquals(partitionSpec, current.spec()); + assertThat(current.schema()).asString().isEqualTo(schema.toString()); + assertThat(current.spec()).isEqualTo(partitionSpec); Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - Assert.assertTrue("initial table history should be empty", table.history().isEmpty()); + assertThat(table.history()).isEmpty(); // commit new version, should create a new snapshot table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); DataFile dataFile = @@ -208,15 +197,15 @@ public void testUpdateTable() { .build(); table.newAppend().appendFile(dataFile).commit(); table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); - Assert.assertEquals("commit should create a new table version", 1, table.history().size()); + assertThat(table.history()).hasSize(1); // check table in Glue GetTableResponse response = glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); - Assert.assertEquals( - "external table type is set after update", "EXTERNAL_TABLE", response.table().tableType()); - Assert.assertEquals( - schema.columns().size(), response.table().storageDescriptor().columns().size()); - Assert.assertEquals(partitionSpec.fields().size(), response.table().partitionKeys().size()); + assertThat(response.table().tableType()) + .as("external table type is set after update") + .isEqualTo("EXTERNAL_TABLE"); + assertThat(response.table().storageDescriptor().columns()).hasSameSizeAs(schema.columns()); + assertThat(response.table().partitionKeys()).hasSameSizeAs(partitionSpec.fields()); } @Test @@ -229,10 +218,10 @@ public void testRenameTable() { glueCatalog.renameTable( TableIdentifier.of(namespace, tableName), TableIdentifier.of(namespace, newTableName)); Table renamedTable = glueCatalog.loadTable(TableIdentifier.of(namespace, newTableName)); - Assert.assertEquals(table.location(), renamedTable.location()); - Assert.assertEquals(table.schema().toString(), renamedTable.schema().toString()); - Assert.assertEquals(table.spec(), renamedTable.spec()); - Assert.assertEquals(table.currentSnapshot(), renamedTable.currentSnapshot()); + assertThat(renamedTable.location()).isEqualTo(table.location()); + assertThat(renamedTable.schema()).asString().isEqualTo(table.schema().toString()); + assertThat(renamedTable.spec()).isEqualTo(table.spec()); + assertThat(renamedTable.currentSnapshot()).isEqualTo(table.currentSnapshot()); } @Test @@ -248,7 +237,7 @@ public void testRenameTableFailsToCreateNewTable() { .databaseName(namespace) .tableInput(TableInput.builder().name(newTableName).build()) .build()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalog.renameTable( TableIdentifier.of(namespace, tableName), @@ -258,10 +247,10 @@ public void testRenameTableFailsToCreateNewTable() { .hasMessageContaining("Table already exists"); // old table can still be read with same metadata Table oldTable = glueCatalog.loadTable(id); - Assert.assertEquals(table.location(), oldTable.location()); - Assert.assertEquals(table.schema().toString(), oldTable.schema().toString()); - Assert.assertEquals(table.spec(), oldTable.spec()); - Assert.assertEquals(table.currentSnapshot(), oldTable.currentSnapshot()); + assertThat(oldTable.location()).isEqualTo(table.location()); + assertThat(oldTable.schema()).asString().isEqualTo(table.schema().toString()); + assertThat(oldTable.spec()).isEqualTo(table.spec()); + assertThat(oldTable.currentSnapshot()).isEqualTo(table.currentSnapshot()); } @Test @@ -277,7 +266,7 @@ public void testRenameTableFailsToDeleteOldTable() { .databaseName(namespace) .tableInput(TableInput.builder().name(tableName).parameters(Maps.newHashMap()).build()) .build()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalog.renameTable( TableIdentifier.of(namespace, tableName), @@ -285,7 +274,7 @@ public void testRenameTableFailsToDeleteOldTable() { .isInstanceOf(ValidationException.class) .as("should fail to rename") .hasMessageContaining("Input Glue table is not an iceberg table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glue.getTable( GetTableRequest.builder().databaseName(namespace).name(newTableName).build())) @@ -299,8 +288,7 @@ public void testDeleteTableWithoutPurge() { String namespace = createNamespace(); String tableName = createTable(namespace); glueCatalog.dropTable(TableIdentifier.of(namespace, tableName), false); - Assertions.assertThatThrownBy( - () -> glueCatalog.loadTable(TableIdentifier.of(namespace, tableName))) + assertThatThrownBy(() -> glueCatalog.loadTable(TableIdentifier.of(namespace, tableName))) .isInstanceOf(NoSuchTableException.class) .as("should not have table") .hasMessageContaining("Table does not exist"); @@ -313,7 +301,7 @@ public void testDeleteTableWithoutPurge() { .bucket(testBucketName) .prefix(prefix + "/metadata/") .build()); - Assert.assertTrue(response.hasContents()); + assertThat(response.hasContents()).isTrue(); boolean hasMetaFile = false; for (S3Object s3Object : response.contents()) { if (s3Object.key().contains(".json")) { @@ -321,7 +309,7 @@ public void testDeleteTableWithoutPurge() { break; } } - Assert.assertTrue("metadata json file exists after delete without purge", hasMetaFile); + assertThat(hasMetaFile).as("metadata json file exists after delete without purge").isTrue(); } @Test @@ -353,8 +341,7 @@ public void testDeleteTableWithPurge() { txn.commitTransaction(); glueCatalog.dropTable(TableIdentifier.of(namespace, tableName)); - Assertions.assertThatThrownBy( - () -> glueCatalog.loadTable(TableIdentifier.of(namespace, tableName))) + assertThatThrownBy(() -> glueCatalog.loadTable(TableIdentifier.of(namespace, tableName))) .isInstanceOf(NoSuchTableException.class) .as("should not have table") .hasMessageContaining("Table does not exist"); @@ -368,8 +355,8 @@ public void testDeleteTableWithPurge() { // might have directory markers left for (S3Object s3Object : response.contents()) { Optional size = s3Object.getValueForField("Size", Long.class); - Assert.assertTrue(size.isPresent()); - Assert.assertEquals(0L, (long) size.get()); + assertThat(size.isPresent()).isTrue(); + assertThat(size.get()).isEqualTo(0); } } } @@ -403,31 +390,29 @@ public void testCommitTableSkipArchive() { .withRecordCount(1) .build(); table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals( - 2, - glue.getTableVersions( - GetTableVersionsRequest.builder() - .databaseName(namespace) - .tableName(tableName) - .build()) - .tableVersions() - .size()); + assertThat( + glue.getTableVersions( + GetTableVersionsRequest.builder() + .databaseName(namespace) + .tableName(tableName) + .build()) + .tableVersions()) + .hasSize(2); // create table and commit with skip tableName = getRandomName(); glueCatalog.initialize(catalogName, ImmutableMap.of()); glueCatalog.createTable(TableIdentifier.of(namespace, tableName), schema, partitionSpec); table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); table.newAppend().appendFile(dataFile).commit(); - Assert.assertEquals( - "skipArchive should not create new version", - 1, - glue.getTableVersions( - GetTableVersionsRequest.builder() - .databaseName(namespace) - .tableName(tableName) - .build()) - .tableVersions() - .size()); + assertThat( + glue.getTableVersions( + GetTableVersionsRequest.builder() + .databaseName(namespace) + .tableName(tableName) + .build()) + .tableVersions()) + .as("skipArchive should not create new version") + .hasSize(1); } @Test @@ -440,8 +425,8 @@ public void testCommitTableSkipNameValidation() { TableIdentifier.of(namespace, tableName), schema, partitionSpec, tableLocationProperties); GetTableResponse response = glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); - Assert.assertEquals(namespace, response.table().databaseName()); - Assert.assertEquals(tableName, response.table().name()); + assertThat(response.table().databaseName()).isEqualTo(namespace); + assertThat(response.table().name()).isEqualTo(tableName); } @Test @@ -513,7 +498,7 @@ public void testColumnCommentsAndParameters() { IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "true", IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "false")) .build()); - Assert.assertEquals("Columns do not match", expectedColumns, actualColumns); + assertThat(actualColumns).isEqualTo(expectedColumns); } @Test @@ -545,28 +530,19 @@ public void testTablePropsDefinedAtCatalogLevel() { .withProperty("key5", "table-key5") .create(); - Assert.assertEquals( - "Table defaults set for the catalog must be added to the table properties.", - "catalog-default-key1", - table.properties().get("key1")); - Assert.assertEquals( - "Table property must override table default properties set at catalog level.", - "table-key2", - table.properties().get("key2")); - Assert.assertEquals( - "Table property override set at catalog level must override table default" - + " properties set at catalog level and table property specified.", - "catalog-override-key3", - table.properties().get("key3")); - Assert.assertEquals( - "Table override not in table props or defaults should be added to table properties", - "catalog-override-key4", - table.properties().get("key4")); - Assert.assertEquals( - "Table properties without any catalog level default or override should be added to table" - + " properties.", - "table-key5", - table.properties().get("key5")); + assertThat(table.properties()) + .as("Table defaults set for the catalog must be added to the table properties.") + .containsEntry("key1", "catalog-default-key1") + .as("Table property must override table default properties set at catalog level.") + .containsEntry("key2", "table-key2") + .as( + "Table property override set at catalog level must override table default properties set at catalog level and table property specified.") + .containsEntry("key3", "catalog-override-key3") + .as("Table override not in table props or defaults should be added to table properties") + .containsEntry("key4", "catalog-override-key4") + .as( + "Table properties without any catalog level default or override should be added to table properties") + .containsEntry("key5", "table-key5"); } @Test @@ -577,15 +553,15 @@ public void testRegisterTable() { TableIdentifier identifier = TableIdentifier.of(namespace, tableName); Table table = glueCatalog.loadTable(identifier); String metadataLocation = ((BaseTable) table).operations().current().metadataFileLocation(); - Assertions.assertThat(glueCatalog.dropTable(identifier, false)).isTrue(); + assertThat(glueCatalog.dropTable(identifier, false)).isTrue(); Table registeredTable = glueCatalog.registerTable(identifier, metadataLocation); - Assertions.assertThat(registeredTable).isNotNull(); + assertThat(registeredTable).isNotNull(); String expectedMetadataLocation = ((BaseTable) table).operations().current().metadataFileLocation(); - Assertions.assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); - Assertions.assertThat(glueCatalog.loadTable(identifier)).isNotNull(); - Assertions.assertThat(glueCatalog.dropTable(identifier, true)).isTrue(); - Assertions.assertThat(glueCatalog.dropNamespace(Namespace.of(namespace))).isTrue(); + assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); + assertThat(glueCatalog.loadTable(identifier)).isNotNull(); + assertThat(glueCatalog.dropTable(identifier, true)).isTrue(); + assertThat(glueCatalog.dropNamespace(Namespace.of(namespace))).isTrue(); } @Test @@ -596,10 +572,10 @@ public void testRegisterTableAlreadyExists() { TableIdentifier identifier = TableIdentifier.of(namespace, tableName); Table table = glueCatalog.loadTable(identifier); String metadataLocation = ((BaseTable) table).operations().current().metadataFileLocation(); - Assertions.assertThatThrownBy(() -> glueCatalog.registerTable(identifier, metadataLocation)) + assertThatThrownBy(() -> glueCatalog.registerTable(identifier, metadataLocation)) .isInstanceOf(AlreadyExistsException.class); - Assertions.assertThat(glueCatalog.dropTable(identifier, true)).isTrue(); - Assertions.assertThat(glueCatalog.dropNamespace(Namespace.of(namespace))).isTrue(); + assertThat(glueCatalog.dropTable(identifier, true)).isTrue(); + assertThat(glueCatalog.dropNamespace(Namespace.of(namespace))).isTrue(); } @Test @@ -634,9 +610,8 @@ public void testTableLevelS3Tags() { .tagSet(); Map tagMap = tags.stream().collect(Collectors.toMap(Tag::key, Tag::value)); - Assert.assertTrue(tagMap.containsKey(S3FileIOProperties.S3_TAG_ICEBERG_TABLE)); - Assert.assertEquals(tableName, tagMap.get(S3FileIOProperties.S3_TAG_ICEBERG_TABLE)); - Assert.assertTrue(tagMap.containsKey(S3FileIOProperties.S3_TAG_ICEBERG_NAMESPACE)); - Assert.assertEquals(namespace, tagMap.get(S3FileIOProperties.S3_TAG_ICEBERG_NAMESPACE)); + assertThat(tagMap) + .containsEntry(S3FileIOProperties.S3_TAG_ICEBERG_TABLE, tableName) + .containsEntry(S3FileIOProperties.S3_TAG_ICEBERG_NAMESPACE, namespace); } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java index 67586943c4d7..5e0f66610c42 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aws.lakeformation; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.UnsupportedEncodingException; import java.net.URLDecoder; import java.nio.charset.StandardCharsets; @@ -38,10 +40,9 @@ 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.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.AfterClass; -import org.junit.BeforeClass; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; @@ -125,7 +126,7 @@ public class LakeFormationTestBase { static LakeFormationClient lakeformation; static GlueClient glue; - @BeforeClass + @BeforeAll public static void beforeClass() throws Exception { lfRegisterPathRoleName = LF_REGISTER_PATH_ROLE_PREFIX + UUID.randomUUID().toString(); lfPrivilegedRoleName = LF_PRIVILEGED_ROLE_PREFIX + UUID.randomUUID().toString(); @@ -256,7 +257,7 @@ public static void beforeClass() throws Exception { registerResource(testBucketPath); } - @AfterClass + @AfterAll public static void afterClass() { GetDataLakeSettingsResponse getDataLakeSettingsResponse = lakeformation.getDataLakeSettings(GetDataLakeSettingsRequest.builder().build()); @@ -367,7 +368,7 @@ private static void waitForIamConsistency(String roleName, String policyName) { .atMost(Duration.ofSeconds(10)) .untilAsserted( () -> - Assertions.assertThat( + assertThat( iam.getRolePolicy( GetRolePolicyRequest.builder() .roleName(roleName) @@ -438,7 +439,7 @@ private static void registerResource(String s3Location) { .ignoreExceptions() .untilAsserted( () -> - Assertions.assertThat( + assertThat( lakeformation .describeResource( DescribeResourceRequest.builder().resourceArn(arn).build()) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java index f8d88901b9f8..9af1e3dc0331 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.lakeformation; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.time.Duration; import java.util.Map; import java.util.UUID; @@ -29,16 +32,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.glue.model.AccessDeniedException; -import software.amazon.awssdk.services.glue.model.GlueException; import software.amazon.awssdk.services.iam.IamClient; import software.amazon.awssdk.services.iam.model.CreateRoleRequest; import software.amazon.awssdk.services.iam.model.CreateRoleResponse; @@ -59,7 +60,7 @@ public class TestLakeFormationAwsClientFactory { private Map assumeRoleProperties; private String policyName; - @Before + @BeforeEach public void before() { roleName = UUID.randomUUID().toString(); iam = @@ -97,7 +98,7 @@ public void before() { policyName = UUID.randomUUID().toString(); } - @After + @AfterEach public void after() { iam.deleteRolePolicy( DeleteRolePolicyRequest.builder().roleName(roleName).policyName(policyName).build()); @@ -150,10 +151,8 @@ public void testLakeFormationEnabledGlueCatalog() throws Exception { Namespace deniedNamespace = Namespace.of("denied_" + UUID.randomUUID().toString().replace("-", "")); try { - glueCatalog.createNamespace(deniedNamespace); - Assert.fail("Access to Glue should be denied"); - } catch (GlueException e) { - Assert.assertEquals(AccessDeniedException.class, e.getClass()); + assertThatThrownBy(() -> glueCatalog.createNamespace(deniedNamespace)) + .isInstanceOf(AccessDeniedException.class); } catch (AssertionError e) { glueCatalog.dropNamespace(deniedNamespace); throw e; @@ -162,10 +161,7 @@ public void testLakeFormationEnabledGlueCatalog() throws Exception { Namespace allowedNamespace = Namespace.of("allowed_" + UUID.randomUUID().toString().replace("-", "")); try { - glueCatalog.createNamespace(allowedNamespace); - } catch (GlueException e) { - LOG.error("fail to create Glue database", e); - Assert.fail("create namespace should succeed"); + assertThatNoException().isThrownBy(() -> glueCatalog.createNamespace(allowedNamespace)); } finally { glueCatalog.dropNamespace(allowedNamespace); try { diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationDataOperations.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationDataOperations.java index 9b7db24324b9..f42db1ef3f3d 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationDataOperations.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationDataOperations.java @@ -18,16 +18,17 @@ */ package org.apache.iceberg.aws.lakeformation; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ForbiddenException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.AccessDeniedException; import software.amazon.awssdk.services.lakeformation.model.Permission; import software.amazon.awssdk.services.s3.model.S3Exception; @@ -37,7 +38,7 @@ public class TestLakeFormationDataOperations extends LakeFormationTestBase { private static String testDbName; private static String testTableName; - @Before + @BeforeEach public void before() { testDbName = getRandomDbName(); testTableName = getRandomTableName(); @@ -45,7 +46,7 @@ public void before() { lfRegisterPathRoleCreateTable(testDbName, testTableName); } - @After + @AfterEach public void after() { lfRegisterPathRoleDeleteTable(testDbName, testTableName); lfRegisterPathRoleDeleteDb(testDbName); @@ -53,7 +54,7 @@ public void after() { @Test public void testLoadTableWithNoTableAccess() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalogPrivilegedRole.loadTable( TableIdentifier.of(Namespace.of(testDbName), testTableName))) @@ -81,7 +82,7 @@ public void testUpdateTableWithNoInsertAccess() { .withFileSizeInBytes(10) .withRecordCount(1) .build(); - Assertions.assertThatThrownBy(() -> table.newAppend().appendFile(dataFile).commit()) + assertThatThrownBy(() -> table.newAppend().appendFile(dataFile).commit()) .as("attempt to insert to a table without INSERT permission should fail") .isInstanceOf(S3Exception.class) .hasMessageContaining("Access Denied"); @@ -117,7 +118,7 @@ public void testDeleteWithNoDataPathAccess() { .withFileSizeInBytes(10) .withRecordCount(1) .build(); - Assertions.assertThatThrownBy(() -> table.newDelete().deleteFile(dataFile).commit()) + assertThatThrownBy(() -> table.newDelete().deleteFile(dataFile).commit()) .as("attempt to delete without DATA_LOCATION_ACCESS permission should fail") .isInstanceOf(ForbiddenException.class) .hasMessageContaining("Glue cannot access the requested resources"); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationMetadataOperations.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationMetadataOperations.java index 4f247755cda6..37465575c0e2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationMetadataOperations.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationMetadataOperations.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.lakeformation; +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.Table; @@ -27,9 +30,7 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.AccessDeniedException; import software.amazon.awssdk.services.lakeformation.model.Permission; @@ -48,8 +49,7 @@ public void testCreateAndDropDatabaseSuccessful() { @Test public void testCreateDatabaseNoPrivileges() { String testDbName = getRandomDbName(); - Assertions.assertThatThrownBy( - () -> glueCatalogPrivilegedRole.createNamespace(Namespace.of(testDbName))) + assertThatThrownBy(() -> glueCatalogPrivilegedRole.createNamespace(Namespace.of(testDbName))) .as("attempt to create a database without CREATE_DATABASE permission should fail") .isInstanceOf(AccessDeniedException.class) .hasMessageContaining("Insufficient Lake Formation permission(s)"); @@ -60,8 +60,7 @@ public void testDropDatabaseNoPrivileges() { String testDbName = getRandomDbName(); lfRegisterPathRoleCreateDb(testDbName); try { - Assertions.assertThatThrownBy( - () -> glueCatalogPrivilegedRole.dropNamespace(Namespace.of(testDbName))) + assertThatThrownBy(() -> glueCatalogPrivilegedRole.dropNamespace(Namespace.of(testDbName))) .as("attempt to drop a database without DROP permission should fail") .isInstanceOf(AccessDeniedException.class) .hasMessageContaining("Insufficient Lake Formation permission(s)"); @@ -77,7 +76,7 @@ public void testShowDatabasesSuccessful() { grantDatabasePrivileges(testDbName, Permission.ALTER); try { List namespaces = glueCatalogPrivilegedRole.listNamespaces(); - Assert.assertTrue(namespaces.contains(Namespace.of(testDbName))); + assertThat(namespaces).contains(Namespace.of(testDbName)); } finally { lfRegisterPathRoleDeleteDb(testDbName); } @@ -92,7 +91,7 @@ public void testCreateTableNoCreateTablePermission() { String tableLocation = getTableLocation(testTableName); grantDataPathPrivileges(tableLocation); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalogPrivilegedRole.createTable( TableIdentifier.of(testDbName, testTableName), @@ -117,8 +116,7 @@ public void testShowTablesSuccessful() { grantTablePrivileges(testDbName, testTableName, Permission.ALTER); try { List tables = glueCatalogPrivilegedRole.listTables(Namespace.of(testDbName)); - Assert.assertTrue( - tables.contains(TableIdentifier.of(Namespace.of(testDbName), testTableName))); + assertThat(tables).contains(TableIdentifier.of(Namespace.of(testDbName), testTableName)); } finally { lfRegisterPathRoleDeleteTable(testDbName, testTableName); lfRegisterPathRoleDeleteDb(testDbName); @@ -132,8 +130,7 @@ public void testShowTablesNoPrivileges() { lfRegisterPathRoleCreateDb(testDbName); lfRegisterPathRoleCreateTable(testDbName, testTableName); try { - Assertions.assertThatThrownBy( - () -> glueCatalogPrivilegedRole.listTables(Namespace.of(testDbName))) + assertThatThrownBy(() -> glueCatalogPrivilegedRole.listTables(Namespace.of(testDbName))) .as("attempt to show tables without any permissions should fail") .isInstanceOf(AccessDeniedException.class) .hasMessageContaining("Insufficient Lake Formation permission(s)"); @@ -150,7 +147,7 @@ public void testCreateTableNoDataPathPermission() { lfRegisterPathRoleCreateDb(testDbName); grantDatabasePrivileges(testDbName, Permission.CREATE_TABLE); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalogPrivilegedRole.createTable( TableIdentifier.of(testDbName, testTableName), @@ -210,7 +207,7 @@ public void testDropTableNoDropPermission() { lfRegisterPathRoleCreateTable(testDbName, testTableName); grantTablePrivileges(testDbName, testTableName, Permission.SELECT); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalogPrivilegedRole.dropTable( TableIdentifier.of(testDbName, testTableName), false)) @@ -265,7 +262,7 @@ public void testAlterTableSetPropertiesNoDataPathAccess() { TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); UpdateProperties updateProperties = table.updateProperties(); properties.forEach(updateProperties::set); - Assertions.assertThatThrownBy(updateProperties::commit) + assertThatThrownBy(updateProperties::commit) .as("attempt to alter a table without ALTER permission should fail") .isInstanceOf(ForbiddenException.class) .hasMessageContaining("Glue cannot access the requested resources"); @@ -283,7 +280,7 @@ public void testAlterTableSetPropertiesNoPrivileges() { lfRegisterPathRoleCreateTable(testDbName, testTableName); grantDataPathPrivileges(getTableLocation(testTableName)); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> glueCatalogPrivilegedRole.loadTable( TableIdentifier.of(Namespace.of(testDbName), testTableName))) @@ -313,7 +310,7 @@ public void testAlterTableSetPropertiesNoAlterPermission() { TableProperties.DEFAULT_FILE_FORMAT, TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); UpdateProperties updateProperties = table.updateProperties(); properties.forEach(updateProperties::set); - Assertions.assertThatThrownBy(updateProperties::commit) + assertThatThrownBy(updateProperties::commit) .as("attempt to alter a table without ALTER privileges should fail") .isInstanceOf(ForbiddenException.class) .hasMessageContaining("Glue cannot access the requested resources"); 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 389e5c82e3f6..244f10dca397 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 @@ -18,7 +18,7 @@ */ package org.apache.iceberg.aws.s3; -import static org.junit.Assert.assertEquals; +import static org.assertj.core.api.Assertions.assertThat; import java.io.ByteArrayInputStream; import java.io.InputStream; @@ -41,14 +41,10 @@ import org.apache.iceberg.io.OutputFile; 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.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.jupiter.api.Assertions; +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 software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.regions.PartitionMetadata; import software.amazon.awssdk.regions.Region; @@ -89,7 +85,7 @@ public class TestS3FileIOIntegration { private String objectKey; private String objectUri; - @BeforeClass + @BeforeAll public static void beforeClass() { clientFactory = AwsClientFactories.defaultFactory(); s3 = clientFactory.s3(); @@ -112,7 +108,7 @@ public static void beforeClass() { crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); } - @AfterClass + @AfterAll public static void afterClass() { AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix); AwsIntegTestUtil.deleteAccessPoint(s3Control, accessPointName); @@ -121,14 +117,10 @@ public static void afterClass() { ScheduleKeyDeletionRequest.builder().keyId(kmsKeyArn).pendingWindowInDays(7).build()); } - @Before - public void before() { - objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString()); - objectUri = String.format("s3://%s/%s", bucketName, objectKey); - } - @BeforeEach public void beforeEach() { + objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString()); + objectUri = String.format("s3://%s/%s", bucketName, objectKey); clientFactory.initialize(Maps.newHashMap()); } @@ -209,7 +201,7 @@ public void testNewOutputStream() throws Exception { s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); String result = IoUtils.toUtf8String(stream); stream.close(); - Assert.assertEquals(content, result); + assertThat(result).isEqualTo(content); } @Test @@ -224,7 +216,7 @@ public void testNewOutputStreamWithAccessPoint() throws Exception { s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); String result = IoUtils.toUtf8String(stream); stream.close(); - Assert.assertEquals(content, result); + assertThat(result).isEqualTo(content); } @Test @@ -247,7 +239,7 @@ public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception { .build()); String result = IoUtils.toUtf8String(stream); stream.close(); - Assert.assertEquals(content, result); + assertThat(result).isEqualTo(content); } @Test @@ -260,7 +252,7 @@ public void testServerSideS3Encryption() throws Exception { GetObjectResponse response = s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()) .response(); - Assert.assertEquals(ServerSideEncryption.AES256, response.serverSideEncryption()); + assertThat(response.serverSideEncryption()).isEqualTo(ServerSideEncryption.AES256); } @Test @@ -274,8 +266,8 @@ public void testServerSideKmsEncryption() throws Exception { GetObjectResponse response = s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()) .response(); - Assert.assertEquals(ServerSideEncryption.AWS_KMS, response.serverSideEncryption()); - Assert.assertEquals(response.ssekmsKeyId(), kmsKeyArn); + assertThat(response.serverSideEncryption()).isEqualTo(ServerSideEncryption.AWS_KMS); + assertThat(kmsKeyArn).isEqualTo(response.ssekmsKeyId()); } @Test @@ -288,12 +280,15 @@ public void testServerSideKmsEncryptionWithDefaultKey() throws Exception { GetObjectResponse response = s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()) .response(); - Assert.assertEquals(ServerSideEncryption.AWS_KMS, response.serverSideEncryption()); + assertThat(response.serverSideEncryption()).isEqualTo(ServerSideEncryption.AWS_KMS); ListAliasesResponse listAliasesResponse = kms.listAliases(ListAliasesRequest.builder().keyId(response.ssekmsKeyId()).build()); - Assert.assertTrue(listAliasesResponse.hasAliases()); - Assert.assertEquals(1, listAliasesResponse.aliases().size()); - Assert.assertEquals("alias/aws/s3", listAliasesResponse.aliases().get(0).aliasName()); + assertThat(listAliasesResponse.hasAliases()).isTrue(); + assertThat(listAliasesResponse.aliases()) + .hasSize(1) + .first() + .satisfies( + aliasListEntry -> assertThat(aliasListEntry.aliasName()).isEqualTo("alias/aws/s3")); } @Test @@ -326,9 +321,9 @@ public void testServerSideCustomEncryption() throws Exception { .sseCustomerKeyMD5(md5) .build()) .response(); - Assert.assertNull(response.serverSideEncryption()); - Assert.assertEquals(ServerSideEncryption.AES256.name(), response.sseCustomerAlgorithm()); - Assert.assertEquals(md5, response.sseCustomerKeyMD5()); + assertThat(response.serverSideEncryption()).isNull(); + assertThat(response.sseCustomerAlgorithm()).isEqualTo(ServerSideEncryption.AES256.toString()); + assertThat(response.sseCustomerKeyMD5()).isEqualTo(md5); } @Test @@ -340,9 +335,11 @@ public void testACL() throws Exception { validateRead(s3FileIO); GetObjectAclResponse response = s3.getObjectAcl(GetObjectAclRequest.builder().bucket(bucketName).key(objectKey).build()); - Assert.assertTrue(response.hasGrants()); - Assert.assertEquals(1, response.grants().size()); - Assert.assertEquals(Permission.FULL_CONTROL, response.grants().get(0).permission()); + assertThat(response.hasGrants()).isTrue(); + assertThat(response.grants()) + .hasSize(1) + .first() + .satisfies(grant -> assertThat(grant.permission()).isEqualTo(Permission.FULL_CONTROL)); } @Test @@ -406,11 +403,11 @@ public void testPrefixList() { scale -> { String scalePrefix = String.format("%s/%s/", listPrefix, scale); createRandomObjects(scalePrefix, scale); - assertEquals((long) scale, Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()); + assertThat(s3FileIO.listPrefix(scalePrefix)).hasSize(scale); }); long totalFiles = scaleSizes.stream().mapToLong(Integer::longValue).sum(); - Assertions.assertEquals(totalFiles, Streams.stream(s3FileIO.listPrefix(listPrefix)).count()); + assertThat(s3FileIO.listPrefix(listPrefix)).hasSize((int) totalFiles); } @SuppressWarnings("DangerousParallelStreamUsage") @@ -429,7 +426,7 @@ public void testPrefixDelete() { String scalePrefix = String.format("%s/%s/", deletePrefix, scale); createRandomObjects(scalePrefix, scale); s3FileIO.deletePrefix(scalePrefix); - assertEquals(0L, Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()); + assertThat(s3FileIO.listPrefix(scalePrefix)).isEmpty(); }); } @@ -448,7 +445,7 @@ private void testDeleteFiles(int numObjects, S3FileIO s3FileIO) throws Exception } s3FileIO.deleteFiles(paths); for (String path : paths) { - Assert.assertFalse(s3FileIO.newInputFile(path).exists()); + assertThat(s3FileIO.newInputFile(path).exists()).isFalse(); } } @@ -465,11 +462,11 @@ private void write(S3FileIO s3FileIO, String uri) throws Exception { private void validateRead(S3FileIO s3FileIO) throws Exception { InputFile file = s3FileIO.newInputFile(objectUri); - Assert.assertEquals(contentBytes.length, file.getLength()); + assertThat(file.getLength()).isEqualTo(contentBytes.length); InputStream stream = file.newStream(); String result = IoUtils.toUtf8String(stream); stream.close(); - Assert.assertEquals(content, result); + assertThat(result).isEqualTo(content); } private String testAccessPointARN(String region, String accessPoint) { 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 3ebc51aab823..ac34807db676 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aws.s3; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.Random; import java.util.UUID; @@ -27,11 +29,10 @@ import org.apache.iceberg.aws.AwsIntegTestUtil; import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.io.SeekableInputStream; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +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 software.amazon.awssdk.services.s3.S3Client; /** Long-running tests to ensure multipart upload logic is resilient */ @@ -45,7 +46,7 @@ public class TestS3MultipartUpload { private static S3FileIO io; private String objectUri; - @BeforeClass + @BeforeAll public static void beforeClass() { s3 = AwsClientFactories.defaultFactory().s3(); bucketName = AwsIntegTestUtil.testBucketName(); @@ -56,28 +57,27 @@ public static void beforeClass() { io = new S3FileIO(() -> s3, properties); } - @AfterClass + @AfterAll public static void afterClass() { AwsIntegTestUtil.cleanS3Bucket(s3, bucketName, prefix); } - @Before + @BeforeEach public void before() { String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString()); objectUri = String.format("s3://%s/%s", bucketName, objectKey); } @Test - public void testManyPartsWriteWithInt() throws IOException { + public void testManyPartsWriteWithInt() { int parts = 200; writeInts(objectUri, parts, random::nextInt); - Assert.assertEquals( - parts * (long) S3FileIOProperties.MULTIPART_SIZE_MIN, - io.newInputFile(objectUri).getLength()); + assertThat(io.newInputFile(objectUri).getLength()) + .isEqualTo(parts * (long) S3FileIOProperties.MULTIPART_SIZE_MIN); } @Test - public void testManyPartsWriteWithBytes() throws IOException { + public void testManyPartsWriteWithBytes() { int parts = 200; byte[] bytes = new byte[S3FileIOProperties.MULTIPART_SIZE_MIN]; writeBytes( @@ -87,9 +87,8 @@ public void testManyPartsWriteWithBytes() throws IOException { random.nextBytes(bytes); return bytes; }); - Assert.assertEquals( - parts * (long) S3FileIOProperties.MULTIPART_SIZE_MIN, - io.newInputFile(objectUri).getLength()); + assertThat(io.newInputFile(objectUri).getLength()) + .isEqualTo(parts * (long) S3FileIOProperties.MULTIPART_SIZE_MIN); } @Test @@ -112,7 +111,7 @@ public void testContentsWriteWithBytes() throws IOException { public void testUploadRemainder() throws IOException { long length = 3 * S3FileIOProperties.MULTIPART_SIZE_MIN + 2 * 1024 * 1024; writeInts(objectUri, 1, length, random::nextInt); - Assert.assertEquals(length, io.newInputFile(objectUri).getLength()); + assertThat(io.newInputFile(objectUri).getLength()).isEqualTo(length); } @Test @@ -146,7 +145,7 @@ private void verifyInts(String fileUri, Supplier verifier) { try (SeekableInputStream inputStream = io.newInputFile(fileUri).newStream()) { int cur; while ((cur = inputStream.read()) != -1) { - Assert.assertEquals(verifier.get().intValue(), cur); + assertThat(cur).isEqualTo(verifier.get()); } } catch (IOException e) { throw new RuntimeException(e); diff --git a/build.gradle b/build.gradle index c21813a4d4ae..138f9d4f8bb8 100644 --- a/build.gradle +++ b/build.gradle @@ -529,6 +529,7 @@ project(':iceberg-aws') { } task integrationTest(type: Test) { + useJUnitPlatform() testClassesDirs = sourceSets.integration.output.classesDirs classpath = sourceSets.integration.runtimeClasspath jvmArgs += project.property('extraJvmArgs') From 66f89432c8193e2edbd8935122c69b219b1cf5c1 Mon Sep 17 00:00:00 2001 From: Brian Hulette Date: Mon, 8 Apr 2024 14:12:57 -0700 Subject: [PATCH 0202/1019] Spec: Document support for binary in truncate transform (#10079) --- format/spec.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index ab6f3494830c..aa905e7032bc 100644 --- a/format/spec.md +++ b/format/spec.md @@ -314,7 +314,7 @@ Partition field IDs must be reused if an existing partition spec contains an equ |-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------| | **`identity`** | Source value, unmodified | Any | Source type | | **`bucket[N]`** | Hash of value, mod `N` (see below) | `int`, `long`, `decimal`, `date`, `time`, `timestamp`, `timestamptz`, `timestamp_ns`, `timestamptz_ns`, `string`, `uuid`, `fixed`, `binary` | `int` | -| **`truncate[W]`** | Value truncated to width `W` (see below) | `int`, `long`, `decimal`, `string` | Source type | +| **`truncate[W]`** | Value truncated to width `W` (see below) | `int`, `long`, `decimal`, `string`, `binary` | Source type | | **`year`** | Extract a date or timestamp year, as years from 1970 | `date`, `timestamp`, `timestamptz`, `timestamp_ns`, `timestamptz_ns` | `int` | | **`month`** | Extract a date or timestamp month, as months from 1970-01-01 | `date`, `timestamp`, `timestamptz`, `timestamp_ns`, `timestamptz_ns` | `int` | | **`day`** | Extract a date or timestamp day, as days from 1970-01-01 | `date`, `timestamp`, `timestamptz`, `timestamp_ns`, `timestamptz_ns` | `int` | @@ -351,12 +351,14 @@ For hash function details by type, see Appendix B. | **`long`** | `W`, width | `v - (v % W)` remainders must be positive [1] | `W=10`: `1` → `0`, `-1` → `-10` | | **`decimal`** | `W`, width (no scale) | `scaled_W = decimal(W, scale(v))` `v - (v % scaled_W)` [1, 2] | `W=50`, `s=2`: `10.65` → `10.50` | | **`string`** | `L`, length | Substring of length `L`: `v.substring(0, L)` [3] | `L=3`: `iceberg` → `ice` | +| **`binary`** | `L`, length | Sub array of length `L`: `v.subarray(0, L)` [4] | `L=3`: `\x01\x02\x03\x04\x05` → `\x01\x02\x03` | Notes: 1. The remainder, `v % W`, must be positive. For languages where `%` can produce negative values, the correct truncate function is: `v - (((v % W) + W) % W)` 2. The width, `W`, used to truncate decimal values is applied using the scale of the decimal column to avoid additional (and potentially conflicting) parameters. 3. Strings are truncated to a valid UTF-8 string with no more than `L` code points. +4. In contrast to strings, binary values do not have an assumed encoding and are truncated to `L` bytes. #### Partition Evolution From cd9ca0799c828a46ae2c44e5e2419cd86c7f7ed5 Mon Sep 17 00:00:00 2001 From: Jason Date: Tue, 9 Apr 2024 10:33:31 +0300 Subject: [PATCH 0203/1019] Docs: Add Upsolver to vendor list (#10096) --- site/docs/vendors.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/site/docs/vendors.md b/site/docs/vendors.md index d549219e5c10..5b8e1a76a19c 100644 --- a/site/docs/vendors.md +++ b/site/docs/vendors.md @@ -71,3 +71,7 @@ Starburst is a commercial offering for the [Trino query engine](https://trino.io ### [Tabular](https://tabular.io) [Tabular](https://tabular.io/product/) is a managed warehouse and automation platform. Tabular offers a central store for analytic data that can be used with any query engine or processing framework that supports Iceberg. Tabular warehouses add role-based access control and automatic optimization, clustering, and compaction to Iceberg tables. + +### [Upsolver](https://upsolver.com) + +[Upsolver](https://upsolver.com) is a streaming data ingestion and table management solution for Apache Iceberg. With Upsolver, users can easily ingest batch and streaming data from files, streams and databases (CDC) into [Iceberg tables](https://docs.upsolver.com/reference/sql-commands/iceberg-tables/upsolver-managed-tables). In addition, Upsolver connects to your existing REST and Hive catalogs, and [analyzes the health](https://docs.upsolver.com/how-to-guides/apache-iceberg/optimize-your-iceberg-tables) of your tables. Use Upsolver to continuously optimize tables by compacting small files, sorting and compressing, repartitioning, and cleaning up dangling files and expired manifests. Upsolver is available from the [Upsolver Cloud](https://www.upsolver.com/sqlake-signup-wp) or can be deployed in your AWS VPC. From 91375fc656afd8f4ae3b56dbeb689ee8c8cd2ef8 Mon Sep 17 00:00:00 2001 From: liko Date: Tue, 9 Apr 2024 02:55:39 -0500 Subject: [PATCH 0204/1019] Docs: Update releases.md for Spark scala versions (#10104) --- site/docs/releases.md | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 09b7f439385b..5a829b593bf3 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -25,9 +25,12 @@ title: "Releases" The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apache/iceberg/releases/tag/apache-iceberg-{{ icebergVersion }}). * [{{ icebergVersion }} source tar.gz](https://www.apache.org/dyn/closer.cgi/iceberg/apache-iceberg-{{ icebergVersion }}/apache-iceberg-{{ icebergVersion }}.tar.gz) -- [signature](https://downloads.apache.org/iceberg/apache-iceberg-{{ icebergVersion }}/apache-iceberg-{{ icebergVersion }}.tar.gz.asc) -- [sha512](https://downloads.apache.org/iceberg/apache-iceberg-{{ icebergVersion }}/apache-iceberg-{{ icebergVersion }}.tar.gz.sha512) -* [{{ icebergVersion }} Spark 3.5\_2.12 runtime Jar](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) -- [3.5\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.5_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.5_2.13-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Spark 3.4\_2.12 runtime Jar](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.4\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.4_2.13-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Spark 3.3\_2.12 runtime Jar](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\_2.13](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.13-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Spark 3.5\_with Scala 2.12 runtime Jar](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) +* [{{ icebergVersion }} Spark 3.5\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.5_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.5_2.13-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Spark 3.4\_with Scala 2.12 runtime Jar](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) +* [{{ icebergVersion }} Spark 3.4\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.4_2.13-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Spark 3.3\_with Scala 2.12 runtime Jar](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) +* [{{ icebergVersion }} Spark 3.3\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.13-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.18 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.17 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.16 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) From 80d54f7ca1a93028ffe432a60aa50de8aed99a1e Mon Sep 17 00:00:00 2001 From: bering <70102274+lawofcycles@users.noreply.github.com> Date: Tue, 9 Apr 2024 17:16:45 +0900 Subject: [PATCH 0205/1019] Docs: Fix spacing/descriptions on Branching and Tagging DDL (#10091) --- docs/docs/spark-ddl.md | 12 +++++++----- 1 file changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/docs/spark-ddl.md b/docs/docs/spark-ddl.md index 8b30710997c8..1d1dc695d105 100644 --- a/docs/docs/spark-ddl.md +++ b/docs/docs/spark-ddl.md @@ -478,10 +478,11 @@ Note that although the identifier is removed, the column will still exist in the #### `ALTER TABLE ... CREATE BRANCH` Branches can be created via the `CREATE BRANCH` statement with the following options: + * Do not fail if the branch already exists with `IF NOT EXISTS` * Update the branch if it already exists with `CREATE OR REPLACE` -* Create at a snapshot -* Create with retention +* Create a branch at a specific snapshot +* Create a branch with a specified retention period ```sql -- CREATE audit-branch at current snapshot with default retention. @@ -497,7 +498,7 @@ ALTER TABLE prod.db.sample CREATE OR REPLACE BRANCH `audit-branch` ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch` AS OF VERSION 1234 --- CREATE audit-branch at snapshot 1234, retain audit-branch for 31 days, and retain the latest 31 days. The latest 3 snapshot snapshots, and 2 days worth of snapshots. +-- CREATE audit-branch at snapshot 1234, retain audit-branch for 30 days, and retain the latest 30 days. The latest 3 snapshot snapshots, and 2 days worth of snapshots. ALTER TABLE prod.db.sample CREATE BRANCH `audit-branch` AS OF VERSION 1234 RETAIN 30 DAYS WITH SNAPSHOT RETENTION 3 SNAPSHOTS 2 DAYS @@ -506,10 +507,11 @@ WITH SNAPSHOT RETENTION 3 SNAPSHOTS 2 DAYS #### `ALTER TABLE ... CREATE TAG` Tags can be created via the `CREATE TAG` statement with the following options: + * Do not fail if the tag already exists with `IF NOT EXISTS` * Update the tag if it already exists with `CREATE OR REPLACE` -* Create at a snapshot -* Create with retention +* Create a tag at a specific snapshot +* Create a tag with a specified retention period ```sql -- CREATE historical-tag at current snapshot with default retention. From 7fb1519c105affa0255bab89b966feec6909d47a Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 9 Apr 2024 23:22:32 +0800 Subject: [PATCH 0206/1019] Core: Add EnvironmentContext to commit summary (#9273) --- .../org/apache/iceberg/SnapshotProducer.java | 1 + .../apache/iceberg/TestSnapshotSummary.java | 7 ++++++ .../TestRewriteDataFilesProcedure.java | 17 ++++++++++++++ ...stRewritePositionDeleteFilesProcedure.java | 22 +++++++++++++++++++ 4 files changed, 47 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 1c4491a6841a..218daca64dbd 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -352,6 +352,7 @@ private Map summary(TableMetadata previous) { SnapshotSummary.ADDED_EQ_DELETES_PROP, SnapshotSummary.REMOVED_EQ_DELETES_PROP); + builder.putAll(EnvironmentContext.get()); return builder.build(); } diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 75b98bd4dca1..23982c510dfa 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -90,4 +90,11 @@ public void testFileSizeSummaryWithDeletes() { .containsEntry(SnapshotSummary.ADD_EQ_DELETE_FILES_PROP, "1") .containsEntry(SnapshotSummary.ADD_POS_DELETE_FILES_PROP, "1"); } + + @TestTemplate + public void testIcebergVersionInSummary() { + table.newFastAppend().appendFile(FILE_A).commit(); + Map summary = table.currentSnapshot().summary(); + assertThat(summary).containsKey("iceberg-version"); + } } 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 9ba886db4516..b7ae0ace5ceb 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 @@ -25,6 +25,8 @@ import java.util.List; import java.util.Map; import java.util.stream.IntStream; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.TableProperties; @@ -881,6 +883,21 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() { .hasMessageContaining("Cannot convert Spark filter"); } + @TestTemplate + public void testRewriteDataFilesSummary() { + createTable(); + // create 10 files under non-partitioned table + insertData(10); + sql("CALL %s.system.rewrite_data_files(table => '%s')", catalogName, tableIdent); + + Map summary = snapshotSummary(); + assertThat(summary) + .containsKey(CatalogProperties.APP_ID) + .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") + .hasEntrySatisfying( + EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("3.5")); + } + private void createTable() { sql("CREATE TABLE %s (c1 int, c2 string, c3 string) USING iceberg", tableName); } 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 585db39a3c44..bb82b63d208d 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 @@ -25,6 +25,8 @@ import java.util.List; import java.util.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -222,6 +224,26 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { .hasMessageContaining("Cannot convert Spark filter"); } + @TestTemplate + public void testRewriteSummary() throws Exception { + createTable(); + sql("DELETE FROM %s WHERE id=1", tableName); + + sql( + "CALL %s.system.rewrite_position_delete_files(" + + "table => '%s'," + + "options => map(" + + "'rewrite-all','true'))", + catalogName, tableIdent); + + Map summary = snapshotSummary(); + assertThat(summary) + .containsKey(CatalogProperties.APP_ID) + .containsEntry(EnvironmentContext.ENGINE_NAME, "spark") + .hasEntrySatisfying( + EnvironmentContext.ENGINE_VERSION, v -> assertThat(v).startsWith("3.5")); + } + private Map snapshotSummary() { return validationCatalog.loadTable(tableIdent).currentSnapshot().summary(); } From 2acad6399496fd27f07f17e8ce158d23de2ad6e0 Mon Sep 17 00:00:00 2001 From: Wei Guo Date: Thu, 11 Apr 2024 04:35:09 +0800 Subject: [PATCH 0207/1019] docs: Fix links of `Get Started` and `Community` parts in footer (#10098) --- site/overrides/partials/footer.html | 24 ++++++++++++------------ 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/site/overrides/partials/footer.html b/site/overrides/partials/footer.html index adb4acc0b0b2..0a55cc32c360 100644 --- a/site/overrides/partials/footer.html +++ b/site/overrides/partials/footer.html @@ -48,12 +48,12 @@

Features

@@ -61,14 +61,14 @@

Get Started

From e137faeddec71e8cca06598c1db80b393a1d9460 Mon Sep 17 00:00:00 2001 From: Harish Chandrasekaran <32119899+harishch1998@users.noreply.github.com> Date: Thu, 11 Apr 2024 03:31:41 -0700 Subject: [PATCH 0208/1019] Core: Allow configuring socket/connection timeout in HTTPClient (#10053) --- .../org/apache/iceberg/rest/HTTPClient.java | 69 ++++++++++++++---- .../apache/iceberg/rest/TestHTTPClient.java | 70 +++++++++++++++++++ 2 files changed, 127 insertions(+), 12 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index 936663095492..13f96a6b2179 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -26,10 +26,12 @@ import java.net.URISyntaxException; import java.nio.charset.StandardCharsets; import java.util.Map; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.stream.Collectors; import org.apache.hc.client5.http.classic.methods.HttpUriRequest; import org.apache.hc.client5.http.classic.methods.HttpUriRequestBase; +import org.apache.hc.client5.http.config.ConnectionConfig; import org.apache.hc.client5.http.impl.classic.CloseableHttpClient; import org.apache.hc.client5.http.impl.classic.CloseableHttpResponse; import org.apache.hc.client5.http.impl.classic.HttpClientBuilder; @@ -79,6 +81,11 @@ public class HTTPClient implements RESTClient { private static final String REST_MAX_CONNECTIONS_PER_ROUTE = "rest.client.connections-per-route"; private static final int REST_MAX_CONNECTIONS_PER_ROUTE_DEFAULT = 100; + @VisibleForTesting + static final String REST_CONNECTION_TIMEOUT_MS = "rest.client.connection-timeout-ms"; + + @VisibleForTesting static final String REST_SOCKET_TIMEOUT_MS = "rest.client.socket-timeout-ms"; + private final String uri; private final CloseableHttpClient httpClient; private final ObjectMapper mapper; @@ -88,22 +95,13 @@ private HTTPClient( Map baseHeaders, ObjectMapper objectMapper, HttpRequestInterceptor requestInterceptor, - Map properties) { + Map properties, + HttpClientConnectionManager connectionManager) { this.uri = uri; this.mapper = objectMapper; HttpClientBuilder clientBuilder = HttpClients.custom(); - HttpClientConnectionManager connectionManager = - PoolingHttpClientConnectionManagerBuilder.create() - .useSystemProperties() - .setMaxConnTotal(Integer.getInteger(REST_MAX_CONNECTIONS, REST_MAX_CONNECTIONS_DEFAULT)) - .setMaxConnPerRoute( - PropertyUtil.propertyAsInt( - properties, - REST_MAX_CONNECTIONS_PER_ROUTE, - REST_MAX_CONNECTIONS_PER_ROUTE_DEFAULT)) - .build(); clientBuilder.setConnectionManager(connectionManager); if (baseHeaders != null) { @@ -448,6 +446,47 @@ static HttpRequestInterceptor loadInterceptorDynamically( return instance; } + static HttpClientConnectionManager configureConnectionManager(Map properties) { + PoolingHttpClientConnectionManagerBuilder connectionManagerBuilder = + PoolingHttpClientConnectionManagerBuilder.create(); + ConnectionConfig connectionConfig = configureConnectionConfig(properties); + if (connectionConfig != null) { + connectionManagerBuilder.setDefaultConnectionConfig(connectionConfig); + } + + return connectionManagerBuilder + .useSystemProperties() + .setMaxConnTotal(Integer.getInteger(REST_MAX_CONNECTIONS, REST_MAX_CONNECTIONS_DEFAULT)) + .setMaxConnPerRoute( + PropertyUtil.propertyAsInt( + properties, REST_MAX_CONNECTIONS_PER_ROUTE, REST_MAX_CONNECTIONS_PER_ROUTE_DEFAULT)) + .build(); + } + + @VisibleForTesting + static ConnectionConfig configureConnectionConfig(Map properties) { + Long connectionTimeoutMillis = + PropertyUtil.propertyAsNullableLong(properties, REST_CONNECTION_TIMEOUT_MS); + Integer socketTimeoutMillis = + PropertyUtil.propertyAsNullableInt(properties, REST_SOCKET_TIMEOUT_MS); + + if (connectionTimeoutMillis == null && socketTimeoutMillis == null) { + return null; + } + + ConnectionConfig.Builder connConfigBuilder = ConnectionConfig.custom(); + + if (connectionTimeoutMillis != null) { + connConfigBuilder.setConnectTimeout(connectionTimeoutMillis, TimeUnit.MILLISECONDS); + } + + if (socketTimeoutMillis != null) { + connConfigBuilder.setSocketTimeout(socketTimeoutMillis, TimeUnit.MILLISECONDS); + } + + return connConfigBuilder.build(); + } + public static Builder builder(Map properties) { return new Builder(properties); } @@ -493,7 +532,13 @@ public HTTPClient build() { interceptor = loadInterceptorDynamically(SIGV4_REQUEST_INTERCEPTOR_IMPL, properties); } - return new HTTPClient(uri, baseHeaders, mapper, interceptor, properties); + return new HTTPClient( + uri, + baseHeaders, + mapper, + interceptor, + properties, + configureConnectionManager(properties)); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java index e596df43e6f5..93585cdbb52e 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java @@ -31,10 +31,13 @@ import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.IOException; +import java.net.SocketTimeoutException; import java.util.Locale; import java.util.Map; import java.util.Objects; +import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import org.apache.hc.client5.http.config.ConnectionConfig; import org.apache.hc.core5.http.EntityDetails; import org.apache.hc.core5.http.HttpException; import org.apache.hc.core5.http.HttpRequestInterceptor; @@ -47,6 +50,8 @@ import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; import org.mockserver.integration.ClientAndServer; import org.mockserver.model.HttpRequest; import org.mockserver.model.HttpResponse; @@ -133,6 +138,71 @@ public void testDynamicHttpRequestInterceptorLoading() { assertThat(((TestHttpRequestInterceptor) interceptor).properties).isEqualTo(properties); } + @Test + public void testSocketAndConnectionTimeoutSet() { + long connectionTimeoutMs = 10L; + int socketTimeoutMs = 10; + Map properties = + ImmutableMap.of( + HTTPClient.REST_CONNECTION_TIMEOUT_MS, String.valueOf(connectionTimeoutMs), + HTTPClient.REST_SOCKET_TIMEOUT_MS, String.valueOf(socketTimeoutMs)); + + ConnectionConfig connectionConfig = HTTPClient.configureConnectionConfig(properties); + assertThat(connectionConfig).isNotNull(); + assertThat(connectionConfig.getConnectTimeout().getDuration()).isEqualTo(connectionTimeoutMs); + assertThat(connectionConfig.getSocketTimeout().getDuration()).isEqualTo(socketTimeoutMs); + } + + @Test + public void testSocketTimeout() throws IOException { + long socketTimeoutMs = 2000L; + Map properties = + ImmutableMap.of(HTTPClient.REST_SOCKET_TIMEOUT_MS, String.valueOf(socketTimeoutMs)); + String path = "socket/timeout/path"; + + try (HTTPClient client = HTTPClient.builder(properties).uri(URI).build()) { + HttpRequest mockRequest = + request() + .withPath("/" + path) + .withMethod(HttpMethod.HEAD.name().toUpperCase(Locale.ROOT)); + // Setting a response delay of 5 seconds to simulate hitting the configured socket timeout of + // 2 seconds + HttpResponse mockResponse = + response() + .withStatusCode(200) + .withBody("Delayed response") + .withDelay(TimeUnit.MILLISECONDS, 5000); + mockServer.when(mockRequest).respond(mockResponse); + + Assertions.assertThatThrownBy(() -> client.head(path, ImmutableMap.of(), (unused) -> {})) + .cause() + .isInstanceOf(SocketTimeoutException.class) + .hasMessage("Read timed out"); + } + } + + @ParameterizedTest + @ValueSource(strings = {HTTPClient.REST_CONNECTION_TIMEOUT_MS, HTTPClient.REST_SOCKET_TIMEOUT_MS}) + public void testInvalidTimeout(String timeoutMsType) { + String invalidTimeoutMs = "invalidMs"; + Assertions.assertThatThrownBy( + () -> + HTTPClient.builder(ImmutableMap.of(timeoutMsType, invalidTimeoutMs)) + .uri(URI) + .build()) + .isInstanceOf(NumberFormatException.class) + .hasMessage(String.format("For input string: \"%s\"", invalidTimeoutMs)); + + String invalidNegativeTimeoutMs = "-1"; + Assertions.assertThatThrownBy( + () -> + HTTPClient.builder(ImmutableMap.of(timeoutMsType, invalidNegativeTimeoutMs)) + .uri(URI) + .build()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage(String.format("duration must not be negative: %s", invalidNegativeTimeoutMs)); + } + public static void testHttpMethodOnSuccess(HttpMethod method) throws JsonProcessingException { Item body = new Item(0L, "hank"); int statusCode = 200; From b54dd4bf488a9980e081cc74c86eb988f52a7b3e Mon Sep 17 00:00:00 2001 From: Harish Chandrasekaran <32119899+harishch1998@users.noreply.github.com> Date: Thu, 11 Apr 2024 13:58:44 -0700 Subject: [PATCH 0209/1019] Core: Extend HTTPClient Builder to allow setting a proxy server (#10052) --- .../org/apache/iceberg/rest/HTTPClient.java | 34 +++++++ .../apache/iceberg/rest/TestHTTPClient.java | 96 +++++++++++++++++++ 2 files changed, 130 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java index 13f96a6b2179..e83ee650cf99 100644 --- a/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java +++ b/core/src/main/java/org/apache/iceberg/rest/HTTPClient.java @@ -29,6 +29,7 @@ import java.util.concurrent.TimeUnit; import java.util.function.Consumer; import java.util.stream.Collectors; +import org.apache.hc.client5.http.auth.CredentialsProvider; import org.apache.hc.client5.http.classic.methods.HttpUriRequest; import org.apache.hc.client5.http.classic.methods.HttpUriRequestBase; import org.apache.hc.client5.http.config.ConnectionConfig; @@ -41,6 +42,7 @@ import org.apache.hc.core5.http.ContentType; import org.apache.hc.core5.http.Header; import org.apache.hc.core5.http.HttpHeaders; +import org.apache.hc.core5.http.HttpHost; import org.apache.hc.core5.http.HttpRequestInterceptor; import org.apache.hc.core5.http.HttpStatus; import org.apache.hc.core5.http.Method; @@ -92,6 +94,8 @@ public class HTTPClient implements RESTClient { private HTTPClient( String uri, + HttpHost proxy, + CredentialsProvider proxyCredsProvider, Map baseHeaders, ObjectMapper objectMapper, HttpRequestInterceptor requestInterceptor, @@ -118,6 +122,14 @@ private HTTPClient( int maxRetries = PropertyUtil.propertyAsInt(properties, REST_MAX_RETRIES, 5); clientBuilder.setRetryStrategy(new ExponentialHttpRequestRetryStrategy(maxRetries)); + if (proxy != null) { + if (proxyCredsProvider != null) { + clientBuilder.setDefaultCredentialsProvider(proxyCredsProvider); + } + + clientBuilder.setProxy(proxy); + } + this.httpClient = clientBuilder.build(); } @@ -496,6 +508,8 @@ public static class Builder { private final Map baseHeaders = Maps.newHashMap(); private String uri; private ObjectMapper mapper = RESTObjectMapper.mapper(); + private HttpHost proxy; + private CredentialsProvider proxyCredentialsProvider; private Builder(Map properties) { this.properties = properties; @@ -507,6 +521,19 @@ public Builder uri(String path) { return this; } + public Builder withProxy(String hostname, int port) { + Preconditions.checkNotNull(hostname, "Invalid hostname for http client proxy: null"); + this.proxy = new HttpHost(hostname, port); + return this; + } + + public Builder withProxyCredentialsProvider(CredentialsProvider credentialsProvider) { + Preconditions.checkNotNull( + credentialsProvider, "Invalid credentials provider for http client proxy: null"); + this.proxyCredentialsProvider = credentialsProvider; + return this; + } + public Builder withHeader(String key, String value) { baseHeaders.put(key, value); return this; @@ -532,8 +559,15 @@ public HTTPClient build() { interceptor = loadInterceptorDynamically(SIGV4_REQUEST_INTERCEPTOR_IMPL, properties); } + if (this.proxyCredentialsProvider != null) { + Preconditions.checkNotNull( + proxy, "Invalid http client proxy for proxy credentials provider: null"); + } + return new HTTPClient( uri, + proxy, + proxyCredentialsProvider, baseHeaders, mapper, interceptor, diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java index 93585cdbb52e..ffd10dff932c 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java @@ -37,10 +37,15 @@ import java.util.Objects; import java.util.concurrent.TimeUnit; import java.util.function.Consumer; +import org.apache.hc.client5.http.auth.AuthScope; +import org.apache.hc.client5.http.auth.UsernamePasswordCredentials; import org.apache.hc.client5.http.config.ConnectionConfig; +import org.apache.hc.client5.http.impl.auth.BasicCredentialsProvider; import org.apache.hc.core5.http.EntityDetails; import org.apache.hc.core5.http.HttpException; +import org.apache.hc.core5.http.HttpHost; import org.apache.hc.core5.http.HttpRequestInterceptor; +import org.apache.hc.core5.http.HttpStatus; import org.apache.hc.core5.http.protocol.HttpContext; import org.apache.iceberg.IcebergBuild; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -52,9 +57,11 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import org.mockserver.configuration.Configuration; import org.mockserver.integration.ClientAndServer; import org.mockserver.model.HttpRequest; import org.mockserver.model.HttpResponse; +import org.mockserver.verify.VerificationTimes; /** * * Exercises the RESTClient interface, specifically over a mocked-server using the actual @@ -126,6 +133,95 @@ public void testHeadFailure() throws JsonProcessingException { testHttpMethodOnFailure(HttpMethod.HEAD); } + @Test + public void testProxyServer() throws IOException { + int proxyPort = 1070; + try (ClientAndServer proxyServer = startClientAndServer(proxyPort); + RESTClient clientWithProxy = + HTTPClient.builder(ImmutableMap.of()) + .uri(URI) + .withProxy("localhost", proxyPort) + .build()) { + String path = "v1/config"; + HttpRequest mockRequest = + request("/" + path).withMethod(HttpMethod.HEAD.name().toUpperCase(Locale.ROOT)); + HttpResponse mockResponse = response().withStatusCode(200); + proxyServer.when(mockRequest).respond(mockResponse); + clientWithProxy.head(path, ImmutableMap.of(), (onError) -> {}); + proxyServer.verify(mockRequest, VerificationTimes.exactly(1)); + } + } + + @Test + public void testProxyCredentialProviderWithoutProxyServer() { + Assertions.assertThatThrownBy( + () -> + HTTPClient.builder(ImmutableMap.of()) + .uri(URI) + .withProxyCredentialsProvider(new BasicCredentialsProvider()) + .build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid http client proxy for proxy credentials provider: null"); + } + + @Test + public void testProxyServerWithNullHostname() { + Assertions.assertThatThrownBy( + () -> HTTPClient.builder(ImmutableMap.of()).uri(URI).withProxy(null, 1070).build()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid hostname for http client proxy: null"); + } + + @Test + public void testProxyAuthenticationFailure() throws IOException { + int proxyPort = 1050; + String proxyHostName = "localhost"; + String authorizedUsername = "test-username"; + String authorizedPassword = "test-password"; + String invalidPassword = "invalid-password"; + + HttpHost proxy = new HttpHost(proxyHostName, proxyPort); + BasicCredentialsProvider credentialsProvider = new BasicCredentialsProvider(); + credentialsProvider.setCredentials( + new AuthScope(proxy), + new UsernamePasswordCredentials(authorizedUsername, invalidPassword.toCharArray())); + + try (ClientAndServer proxyServer = + startClientAndServer( + new Configuration() + .proxyAuthenticationUsername(authorizedUsername) + .proxyAuthenticationPassword(authorizedPassword), + proxyPort); + RESTClient clientWithProxy = + HTTPClient.builder(ImmutableMap.of()) + .uri(URI) + .withProxy(proxyHostName, proxyPort) + .withProxyCredentialsProvider(credentialsProvider) + .build()) { + + ErrorHandler onError = + new ErrorHandler() { + @Override + public ErrorResponse parseResponse(int code, String responseBody) { + return null; + } + + @Override + public void accept(ErrorResponse errorResponse) { + throw new RuntimeException(errorResponse.message() + " - " + errorResponse.code()); + } + }; + + Assertions.assertThatThrownBy( + () -> clientWithProxy.get("v1/config", Item.class, ImmutableMap.of(), onError)) + .isInstanceOf(RuntimeException.class) + .hasMessage( + String.format( + "%s - %s", + "Proxy Authentication Required", HttpStatus.SC_PROXY_AUTHENTICATION_REQUIRED)); + } + } + @Test public void testDynamicHttpRequestInterceptorLoading() { Map properties = ImmutableMap.of("key", "val"); From 3f69fa00d6278248bff60a251019c5dae0bd544a Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 11 Apr 2024 14:04:13 -0700 Subject: [PATCH 0210/1019] Spark 3.4: Fix system function pushdown in CoW row-level commands (#10119) --- .../optimizer/ReplaceStaticInvoke.scala | 49 ++- .../spark/extensions/SparkPlanUtil.java | 48 +++ ...mFunctionPushDownInRowLevelOperations.java | 354 ++++++++++++++++++ .../spark/functions/BaseScalarFunction.java | 40 ++ .../spark/functions/BucketFunction.java | 3 +- .../iceberg/spark/functions/DaysFunction.java | 3 +- .../spark/functions/HoursFunction.java | 5 +- .../functions/IcebergVersionFunction.java | 3 +- .../spark/functions/MonthsFunction.java | 3 +- .../spark/functions/TruncateFunction.java | 3 +- .../spark/functions/YearsFunction.java | 3 +- 11 files changed, 486 insertions(+), 28 deletions(-) create mode 100644 spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala index 1f0e164d8467..655a93a7db8b 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceStaticInvoke.scala @@ -22,12 +22,20 @@ import org.apache.iceberg.spark.functions.SparkFunctions import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression import org.apache.spark.sql.catalyst.expressions.BinaryComparison import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.In +import org.apache.spark.sql.catalyst.expressions.InSet import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.catalyst.plans.logical.Join import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.ReplaceData import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.trees.TreePattern.BINARY_COMPARISON +import org.apache.spark.sql.catalyst.trees.TreePattern.COMMAND import org.apache.spark.sql.catalyst.trees.TreePattern.FILTER +import org.apache.spark.sql.catalyst.trees.TreePattern.IN +import org.apache.spark.sql.catalyst.trees.TreePattern.INSET +import org.apache.spark.sql.catalyst.trees.TreePattern.JOIN import org.apache.spark.sql.connector.catalog.functions.ScalarFunction import org.apache.spark.sql.types.StructField import org.apache.spark.sql.types.StructType @@ -40,21 +48,36 @@ import org.apache.spark.sql.types.StructType object ReplaceStaticInvoke extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = - plan.transformWithPruning (_.containsAllPatterns(BINARY_COMPARISON, FILTER)) { - case filter @ Filter(condition, _) => - val newCondition = condition.transformWithPruning(_.containsPattern(BINARY_COMPARISON)) { - case c @ BinaryComparison(left: StaticInvoke, right) if canReplace(left) && right.foldable => - c.withNewChildren(Seq(replaceStaticInvoke(left), right)) + plan.transformWithPruning (_.containsAnyPattern(COMMAND, FILTER, JOIN)) { + case join @ Join(_, _, _, Some(cond), _) => + replaceStaticInvoke(join, cond, newCond => join.copy(condition = Some(newCond))) - case c @ BinaryComparison(left, right: StaticInvoke) if canReplace(right) && left.foldable => - c.withNewChildren(Seq(left, replaceStaticInvoke(right))) - } + case filter @ Filter(cond, _) => + replaceStaticInvoke(filter, cond, newCond => filter.copy(condition = newCond)) + } + + private def replaceStaticInvoke[T <: LogicalPlan]( + node: T, + condition: Expression, + copy: Expression => T): T = { + val newCondition = replaceStaticInvoke(condition) + if (newCondition fastEquals condition) node else copy(newCondition) + } + + private def replaceStaticInvoke(condition: Expression): Expression = { + condition.transformWithPruning(_.containsAnyPattern(BINARY_COMPARISON, IN, INSET)) { + case in @ In(value: StaticInvoke, _) if canReplace(value) => + in.copy(value = replaceStaticInvoke(value)) - if (newCondition fastEquals condition) { - filter - } else { - filter.copy(condition = newCondition) - } + case in @ InSet(value: StaticInvoke, _) if canReplace(value) => + in.copy(child = replaceStaticInvoke(value)) + + case c @ BinaryComparison(left: StaticInvoke, right) if canReplace(left) && right.foldable => + c.withNewChildren(Seq(replaceStaticInvoke(left), right)) + + case c @ BinaryComparison(left, right: StaticInvoke) if canReplace(right) && left.foldable => + c.withNewChildren(Seq(left, replaceStaticInvoke(right))) + } } private def replaceStaticInvoke(invoke: StaticInvoke): Expression = { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java index 4f7c3ebadbc5..830d07d86eab 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkPlanUtil.java @@ -20,12 +20,17 @@ import static scala.collection.JavaConverters.seqAsJavaListConverter; +import java.util.Collection; import java.util.List; +import java.util.function.Predicate; import java.util.stream.Collectors; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.execution.CommandResultExec; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanHelper; import org.apache.spark.sql.execution.datasources.v2.BatchScanExec; +import scala.PartialFunction; import scala.collection.Seq; public class SparkPlanUtil { @@ -53,6 +58,49 @@ private static SparkPlan actualPlan(SparkPlan plan) { } } + public static List collectExprs( + SparkPlan sparkPlan, Predicate predicate) { + Seq> seq = + SPARK_HELPER.collect( + sparkPlan, + new PartialFunction>() { + @Override + public List apply(SparkPlan plan) { + List exprs = Lists.newArrayList(); + + for (Expression expr : toJavaList(plan.expressions())) { + exprs.addAll(collectExprs(expr, predicate)); + } + + return exprs; + } + + @Override + public boolean isDefinedAt(SparkPlan plan) { + return true; + } + }); + return toJavaList(seq).stream().flatMap(Collection::stream).collect(Collectors.toList()); + } + + private static List collectExprs( + Expression expression, Predicate predicate) { + Seq seq = + expression.collect( + new PartialFunction() { + @Override + public Expression apply(Expression expr) { + return expr; + } + + @Override + public boolean isDefinedAt(Expression expr) { + return predicate.test(expr); + } + }); + return toJavaList(seq); + } + private static List toJavaList(Seq seq) { return seqAsJavaListConverter(seq).asJava(); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java new file mode 100644 index 000000000000..db4d10645b99 --- /dev/null +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java @@ -0,0 +1,354 @@ +/* + * 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.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.RowLevelOperationMode; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.expressions.ApplyFunctionExpression; +import org.apache.spark.sql.catalyst.expressions.Expression; +import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; +import org.apache.spark.sql.execution.CommandResultExec; +import org.apache.spark.sql.execution.datasources.v2.V2TableWriteExec; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.runners.Parameterized.Parameters; + +public class TestSystemFunctionPushDownInRowLevelOperations extends SparkExtensionsTestBase { + + private static final String CHANGES_TABLE_NAME = "changes"; + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + public static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + SparkCatalogConfig.HIVE.properties() + } + }; + } + + public TestSystemFunctionPushDownInRowLevelOperations( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Before + public void beforeEach() { + sql("USE %s", catalogName); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s PURGE", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", tableName(CHANGES_TABLE_NAME)); + } + + @Test + public void testCopyOnWriteDeleteBucketTransformInPredicate() { + initTable("bucket(4, dep)"); + checkDelete(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @Test + public void testMergeOnReadDeleteBucketTransformInPredicate() { + initTable("bucket(4, dep)"); + checkDelete(MERGE_ON_READ, "system.bucket(4, dep) IN (2, 3)"); + } + + @Test + public void testCopyOnWriteDeleteBucketTransformEqPredicate() { + initTable("bucket(4, dep)"); + checkDelete(COPY_ON_WRITE, "system.bucket(4, dep) = 2"); + } + + @Test + public void testMergeOnReadDeleteBucketTransformEqPredicate() { + initTable("bucket(4, dep)"); + checkDelete(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @Test + public void testCopyOnWriteDeleteYearsTransform() { + initTable("years(ts)"); + checkDelete(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @Test + public void testMergeOnReadDeleteYearsTransform() { + initTable("years(ts)"); + checkDelete(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @Test + public void testCopyOnWriteDeleteMonthsTransform() { + initTable("months(ts)"); + checkDelete(COPY_ON_WRITE, "system.months(ts) <= 250"); + } + + @Test + public void testMergeOnReadDeleteMonthsTransform() { + initTable("months(ts)"); + checkDelete(MERGE_ON_READ, "system.months(ts) > 250"); + } + + @Test + public void testCopyOnWriteDeleteDaysTransform() { + initTable("days(ts)"); + checkDelete(COPY_ON_WRITE, "system.days(ts) <= date('2000-01-03 00:00:00')"); + } + + @Test + public void testMergeOnReadDeleteDaysTransform() { + initTable("days(ts)"); + checkDelete(MERGE_ON_READ, "system.days(ts) > date('2000-01-03 00:00:00')"); + } + + @Test + public void testCopyOnWriteDeleteHoursTransform() { + initTable("hours(ts)"); + checkDelete(COPY_ON_WRITE, "system.hours(ts) <= 100000"); + } + + @Test + public void testMergeOnReadDeleteHoursTransform() { + initTable("hours(ts)"); + checkDelete(MERGE_ON_READ, "system.hours(ts) > 100000"); + } + + @Test + public void testCopyOnWriteDeleteTruncateTransform() { + initTable("truncate(1, dep)"); + checkDelete(COPY_ON_WRITE, "system.truncate(1, dep) = 'i'"); + } + + @Test + public void testMergeOnReadDeleteTruncateTransform() { + initTable("truncate(1, dep)"); + checkDelete(MERGE_ON_READ, "system.truncate(1, dep) = 'i'"); + } + + @Test + public void testCopyOnWriteUpdateBucketTransform() { + initTable("bucket(4, dep)"); + checkUpdate(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @Test + public void testMergeOnReadUpdateBucketTransform() { + initTable("bucket(4, dep)"); + checkUpdate(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @Test + public void testCopyOnWriteUpdateYearsTransform() { + initTable("years(ts)"); + checkUpdate(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @Test + public void testMergeOnReadUpdateYearsTransform() { + initTable("years(ts)"); + checkUpdate(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @Test + public void testCopyOnWriteMergeBucketTransform() { + initTable("bucket(4, dep)"); + checkMerge(COPY_ON_WRITE, "system.bucket(4, dep) IN (2, 3)"); + } + + @Test + public void testMergeOnReadMergeBucketTransform() { + initTable("bucket(4, dep)"); + checkMerge(MERGE_ON_READ, "system.bucket(4, dep) = 2"); + } + + @Test + public void testCopyOnWriteMergeYearsTransform() { + initTable("years(ts)"); + checkMerge(COPY_ON_WRITE, "system.years(ts) > 30"); + } + + @Test + public void testMergeOnReadMergeYearsTransform() { + initTable("years(ts)"); + checkMerge(MERGE_ON_READ, "system.years(ts) <= 30"); + } + + @Test + public void testCopyOnWriteMergeTruncateTransform() { + initTable("truncate(1, dep)"); + checkMerge(COPY_ON_WRITE, "system.truncate(1, dep) = 'i'"); + } + + @Test + public void testMergeOnReadMergeTruncateTransform() { + initTable("truncate(1, dep)"); + checkMerge(MERGE_ON_READ, "system.truncate(1, dep) = 'i'"); + } + + private void checkDelete(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.DELETE_MODE, + mode.modeName(), + TableProperties.DELETE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = spark.table(tableName).where(cond).limit(2).select("id"); + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + + List calls = + executeAndCollectFunctionCalls( + "DELETE FROM %s t WHERE %s AND t.id IN (SELECT id FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME)); + // CoW planning currently does not optimize post-scan filters in DELETE + int expectedCallCount = mode == COPY_ON_WRITE ? 1 : 0; + assertThat(calls).hasSize(expectedCallCount); + + assertEquals( + "Should have no matching rows", + ImmutableList.of(), + sql( + "SELECT * FROM %s WHERE %s AND id IN (SELECT * FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME))); + }); + } + + private void checkUpdate(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.UPDATE_MODE, + mode.modeName(), + TableProperties.UPDATE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = spark.table(tableName).where(cond).limit(2).select("id"); + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + + List calls = + executeAndCollectFunctionCalls( + "UPDATE %s t SET t.salary = -1 WHERE %s AND t.id IN (SELECT id FROM %s)", + tableName, cond, tableName(CHANGES_TABLE_NAME)); + // CoW planning currently does not optimize post-scan filters in UPDATE + int expectedCallCount = mode == COPY_ON_WRITE ? 2 : 0; + assertThat(calls).hasSize(expectedCallCount); + + assertEquals( + "Should have correct updates", + sql("SELECT id FROM %s", tableName(CHANGES_TABLE_NAME)), + sql("SELECT id FROM %s WHERE %s AND salary = -1", tableName, cond)); + }); + } + + private void checkMerge(RowLevelOperationMode mode, String cond) { + withUnavailableLocations( + findIrrelevantFileLocations(cond), + () -> { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s', '%s' '%s')", + tableName, + TableProperties.MERGE_MODE, + mode.modeName(), + TableProperties.MERGE_DISTRIBUTION_MODE, + DistributionMode.NONE.modeName()); + + Dataset changeDF = + spark.table(tableName).where(cond).limit(2).selectExpr("id + 1 as id"); + changeDF.coalesce(1).writeTo(tableName(CHANGES_TABLE_NAME)).create(); + + List calls = + executeAndCollectFunctionCalls( + "MERGE INTO %s t USING %s s " + + "ON t.id == s.id AND %s " + + "WHEN MATCHED THEN " + + " UPDATE SET salary = -1 " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT (id, salary, dep, ts) VALUES (100, -1, 'hr', null)", + tableName, tableName(CHANGES_TABLE_NAME), cond); + assertThat(calls).isEmpty(); + + assertEquals( + "Should have correct updates", + sql("SELECT id FROM %s", tableName(CHANGES_TABLE_NAME)), + sql("SELECT id FROM %s WHERE %s AND salary = -1", tableName, cond)); + }); + } + + private List executeAndCollectFunctionCalls(String query, Object... args) { + CommandResultExec command = (CommandResultExec) executeAndKeepPlan(query, args); + V2TableWriteExec write = (V2TableWriteExec) command.commandPhysicalPlan(); + System.out.println("!!! WRITE PLAN !!!"); + System.out.println(write.toString()); + return SparkPlanUtil.collectExprs( + write.query(), + expr -> expr instanceof StaticInvoke || expr instanceof ApplyFunctionExpression); + } + + private List findIrrelevantFileLocations(String cond) { + return spark + .table(tableName) + .where("NOT " + cond) + .select(MetadataColumns.FILE_PATH.name()) + .distinct() + .as(Encoders.STRING()) + .collectAsList(); + } + + private void initTable(String transform) { + sql( + "CREATE TABLE %s (id BIGINT, salary INT, dep STRING, ts TIMESTAMP)" + + "USING iceberg " + + "PARTITIONED BY (%s) " + + "TBLPROPERTIES ('%s' 'true')", + tableName, transform, TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED); + + append( + tableName, + "{ \"id\": 1, \"salary\": 100, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 2, \"salary\": 200, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 3, \"salary\": 300, \"dep\": \"hr\", \"ts\": \"1975-01-01 06:00:00\" }", + "{ \"id\": 4, \"salary\": 400, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }", + "{ \"id\": 5, \"salary\": 500, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }", + "{ \"id\": 6, \"salary\": 600, \"dep\": \"it\", \"ts\": \"2020-01-01 10:00:00\" }"); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.java new file mode 100644 index 000000000000..5ec44f314180 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BaseScalarFunction.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.spark.functions; + +import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; + +abstract class BaseScalarFunction implements ScalarFunction { + @Override + public int hashCode() { + return canonicalName().hashCode(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof ScalarFunction)) { + return false; + } + + ScalarFunction that = (ScalarFunction) other; + return canonicalName().equals(that.canonicalName()); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java index af3c67a4bb63..c3de3d48dbcc 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/BucketFunction.java @@ -25,7 +25,6 @@ import org.apache.iceberg.util.BucketUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.ByteType; @@ -115,7 +114,7 @@ public String name() { return "bucket"; } - public abstract static class BucketBase implements ScalarFunction { + public abstract static class BucketBase extends BaseScalarFunction { public static int apply(int numBuckets, int hashedValue) { return (hashedValue & Integer.MAX_VALUE) % numBuckets; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java index b8d28b73f42f..f52edd9b208f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/DaysFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DateType; @@ -61,7 +60,7 @@ public String name() { return "days"; } - private abstract static class BaseToDaysFunction implements ScalarFunction { + private abstract static class BaseToDaysFunction extends BaseScalarFunction { @Override public String name() { return "days"; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java index 18697e1c16fb..660a182f0e78 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/HoursFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.TimestampNTZType; @@ -58,7 +57,7 @@ public String name() { return "hours"; } - public static class TimestampToHoursFunction implements ScalarFunction { + public static class TimestampToHoursFunction extends BaseScalarFunction { // magic method used in codegen public static int invoke(long micros) { return DateTimeUtil.microsToHours(micros); @@ -91,7 +90,7 @@ public Integer produceResult(InternalRow input) { } } - public static class TimestampNtzToHoursFunction implements ScalarFunction { + public static class TimestampNtzToHoursFunction extends BaseScalarFunction { // magic method used in codegen public static int invoke(long micros) { return DateTimeUtil.microsToHours(micros); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java index 9cd059377ce3..689a0f4cb4df 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/IcebergVersionFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.IcebergBuild; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; @@ -55,7 +54,7 @@ public String name() { // Implementing class cannot be private, otherwise Spark is unable to access the static invoke // function during code-gen and calling the function fails - static class IcebergVersionFunctionImpl implements ScalarFunction { + static class IcebergVersionFunctionImpl extends BaseScalarFunction { private static final UTF8String VERSION = UTF8String.fromString(IcebergBuild.version()); // magic function used in code-gen. must be named `invoke`. diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java index 1d38014461c1..353d850f86e2 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/MonthsFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DateType; @@ -61,7 +60,7 @@ public String name() { return "months"; } - private abstract static class BaseToMonthsFunction implements ScalarFunction { + private abstract static class BaseToMonthsFunction extends BaseScalarFunction { @Override public String name() { return "months"; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java index 8cfb529e1028..fac90c9efee6 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/TruncateFunction.java @@ -27,7 +27,6 @@ import org.apache.iceberg.util.TruncateUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.ByteType; @@ -108,7 +107,7 @@ public String name() { return "truncate"; } - public abstract static class TruncateBase implements ScalarFunction { + public abstract static class TruncateBase extends BaseScalarFunction { @Override public String name() { return "truncate"; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java index 02642e657d76..cfd1b0e8d002 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/functions/YearsFunction.java @@ -21,7 +21,6 @@ import org.apache.iceberg.util.DateTimeUtil; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.connector.catalog.functions.BoundFunction; -import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DateType; @@ -61,7 +60,7 @@ public String name() { return "years"; } - private abstract static class BaseToYearsFunction implements ScalarFunction { + private abstract static class BaseToYearsFunction extends BaseScalarFunction { @Override public String name() { return "years"; From 90d276994efc4d7ee604a9aa6b95771b533b0608 Mon Sep 17 00:00:00 2001 From: sullis Date: Fri, 12 Apr 2024 08:51:09 -0700 Subject: [PATCH 0211/1019] API, Core, Kafka, Spark: Reduce enum array allocation (#10126) --- api/src/main/java/org/apache/iceberg/FileFormat.java | 4 +++- core/src/main/java/org/apache/iceberg/BaseFile.java | 3 ++- core/src/main/java/org/apache/iceberg/CachingCatalog.java | 3 ++- .../main/java/org/apache/iceberg/GenericManifestEntry.java | 3 ++- .../src/main/java/org/apache/iceberg/GenericManifestFile.java | 3 ++- .../java/org/apache/iceberg/hadoop/HadoopTableOperations.java | 4 +++- .../main/java/org/apache/iceberg/connect/events/Event.java | 3 ++- .../main/java/org/apache/iceberg/spark/SparkContentFile.java | 4 +++- .../main/java/org/apache/iceberg/spark/SparkContentFile.java | 4 +++- 9 files changed, 22 insertions(+), 9 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/FileFormat.java b/api/src/main/java/org/apache/iceberg/FileFormat.java index 892475780bbf..d662437d5ddb 100644 --- a/api/src/main/java/org/apache/iceberg/FileFormat.java +++ b/api/src/main/java/org/apache/iceberg/FileFormat.java @@ -32,6 +32,8 @@ public enum FileFormat { private final String ext; private final boolean splittable; + private static final FileFormat[] VALUES = values(); + FileFormat(String ext, boolean splittable) { this.ext = "." + ext; this.splittable = splittable; @@ -55,7 +57,7 @@ public String addExtension(String filename) { } public static FileFormat fromFileName(CharSequence filename) { - for (FileFormat format : FileFormat.values()) { + for (FileFormat format : VALUES) { int extStart = filename.length() - format.ext.length(); if (Comparators.charSequences() .compare(format.ext, filename.subSequence(extStart, filename.length())) diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index 6fe2fcc149cb..d4583b31c7a1 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -45,6 +45,7 @@ abstract class BaseFile StructLike, SpecificData.SchemaConstructable, Serializable { + private static final FileContent[] FILE_CONTENT_VALUES = FileContent.values(); static final Types.StructType EMPTY_STRUCT_TYPE = Types.StructType.of(); static final PartitionData EMPTY_PARTITION_DATA = new PartitionData(EMPTY_STRUCT_TYPE) { @@ -268,7 +269,7 @@ public void put(int i, Object value) { } switch (pos) { case 0: - this.content = value != null ? FileContent.values()[(Integer) value] : FileContent.DATA; + this.content = value != null ? FILE_CONTENT_VALUES[(Integer) value] : FileContent.DATA; return; case 1: // always coerce to String for Serializable diff --git a/core/src/main/java/org/apache/iceberg/CachingCatalog.java b/core/src/main/java/org/apache/iceberg/CachingCatalog.java index f6c3657ff16b..1043e3e7205c 100644 --- a/core/src/main/java/org/apache/iceberg/CachingCatalog.java +++ b/core/src/main/java/org/apache/iceberg/CachingCatalog.java @@ -45,6 +45,7 @@ */ public class CachingCatalog implements Catalog { private static final Logger LOG = LoggerFactory.getLogger(CachingCatalog.class); + private static final MetadataTableType[] METADATA_TABLE_TYPE_VALUES = MetadataTableType.values(); public static Catalog wrap(Catalog catalog) { return wrap(catalog, CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); @@ -197,7 +198,7 @@ public Table registerTable(TableIdentifier identifier, String metadataFileLocati private Iterable metadataTableIdentifiers(TableIdentifier ident) { ImmutableList.Builder builder = ImmutableList.builder(); - for (MetadataTableType type : MetadataTableType.values()) { + for (MetadataTableType type : METADATA_TABLE_TYPE_VALUES) { // metadata table resolution is case insensitive right now builder.add(TableIdentifier.parse(ident + "." + type.name())); builder.add(TableIdentifier.parse(ident + "." + type.name().toLowerCase(Locale.ROOT))); diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java index 96a475881dab..959e2446c710 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java @@ -26,6 +26,7 @@ class GenericManifestEntry> implements ManifestEntry, IndexedRecord, SpecificData.SchemaConstructable, StructLike { + private static final Status[] STATUS_VALUES = Status.values(); private final org.apache.avro.Schema schema; private Status status = Status.EXISTING; private Long snapshotId = null; @@ -151,7 +152,7 @@ public void setFileSequenceNumber(long newFileSequenceNumber) { public void put(int i, Object v) { switch (i) { case 0: - this.status = Status.values()[(Integer) v]; + this.status = STATUS_VALUES[(Integer) v]; return; case 1: this.snapshotId = (Long) v; diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index d6b7636d3a55..ed94da5e1791 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -39,6 +39,7 @@ public class GenericManifestFile implements ManifestFile, StructLike, IndexedRecord, SchemaConstructable, Serializable { private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ManifestFile.schema(), "manifest_file"); + private static final ManifestContent[] MANIFEST_CONTENT_VALUES = ManifestContent.values(); private transient Schema avroSchema; // not final for Java serialization private int[] fromProjectionPos; @@ -339,7 +340,7 @@ public void set(int i, T value) { return; case 3: this.content = - value != null ? ManifestContent.values()[(Integer) value] : ManifestContent.DATA; + value != null ? MANIFEST_CONTENT_VALUES[(Integer) value] : ManifestContent.DATA; return; case 4: this.sequenceNumber = value != null ? (Long) value : 0; 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 9ef2c63e265c..4e815ceff59a 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -60,6 +60,8 @@ public class HadoopTableOperations implements TableOperations { private static final Logger LOG = LoggerFactory.getLogger(HadoopTableOperations.class); private static final Pattern VERSION_PATTERN = Pattern.compile("v([^\\.]*)\\..*"); + private static final TableMetadataParser.Codec[] TABLE_METADATA_PARSER_CODEC_VALUES = + TableMetadataParser.Codec.values(); private final Configuration conf; private final Path location; @@ -235,7 +237,7 @@ public long newSnapshotId() { @VisibleForTesting Path getMetadataFile(int metadataVersion) throws IOException { - for (TableMetadataParser.Codec codec : TableMetadataParser.Codec.values()) { + for (TableMetadataParser.Codec codec : TABLE_METADATA_PARSER_CODEC_VALUES) { Path metadataFile = metadataFilePath(metadataVersion, codec); FileSystem fs = getFileSystem(metadataFile, conf); if (fs.exists(metadataFile)) { diff --git a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/Event.java b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/Event.java index 8b6dbc0f45a6..72d7e22d1858 100644 --- a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/Event.java +++ b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/Event.java @@ -41,6 +41,7 @@ */ public class Event implements IndexedRecord { + private static final PayloadType[] PAYLOAD_TYPE_VALUES = PayloadType.values(); private UUID id; private PayloadType type; private OffsetDateTime timestamp; @@ -115,7 +116,7 @@ public void put(int i, Object v) { this.id = (UUID) v; return; case TYPE: - this.type = v == null ? null : PayloadType.values()[(Integer) v]; + this.type = v == null ? null : PAYLOAD_TYPE_VALUES[(Integer) v]; return; case TIMESTAMP: this.timestamp = v == null ? null : DateTimeUtil.timestamptzFromMicros((Long) v); 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 3dd8049c13f6..f756c4cde015 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 @@ -35,6 +35,8 @@ public abstract class SparkContentFile implements ContentFile { + private static final FileContent[] FILE_CONTENT_VALUES = FileContent.values(); + private final int fileContentPosition; private final int filePathPosition; private final int fileFormatPosition; @@ -126,7 +128,7 @@ public FileContent content() { if (wrapped.isNullAt(fileContentPosition)) { return null; } - return FileContent.values()[wrapped.getInt(fileContentPosition)]; + return FILE_CONTENT_VALUES[wrapped.getInt(fileContentPosition)]; } @Override 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 3dd8049c13f6..f756c4cde015 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 @@ -35,6 +35,8 @@ public abstract class SparkContentFile implements ContentFile { + private static final FileContent[] FILE_CONTENT_VALUES = FileContent.values(); + private final int fileContentPosition; private final int filePathPosition; private final int fileFormatPosition; @@ -126,7 +128,7 @@ public FileContent content() { if (wrapped.isNullAt(fileContentPosition)) { return null; } - return FileContent.values()[wrapped.getInt(fileContentPosition)]; + return FILE_CONTENT_VALUES[wrapped.getInt(fileContentPosition)]; } @Override From 6ea8e2b2114ea99e6a09368c2acb2edc024b0370 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 12 Apr 2024 19:20:22 +0200 Subject: [PATCH 0212/1019] Spark: Test initialization improvements (#10131) --- .../apache/iceberg/spark/extensions/ExtensionsTestBase.java | 3 +++ .../test/java/org/apache/iceberg/spark/CatalogTestBase.java | 4 ---- .../java/org/apache/iceberg/spark/TestBaseWithCatalog.java | 2 +- .../java/org/apache/iceberg/spark/TestSparkExecutorCache.java | 4 ++-- 4 files changed, 6 insertions(+), 7 deletions(-) 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 8e167b7f7320..005d5e29d5ff 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 @@ -28,6 +28,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.TestBase; +import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.internal.SQLConf; import org.junit.jupiter.api.BeforeAll; @@ -57,6 +58,8 @@ public static void startMetastoreAndSpark() { .enableHiveSupport() .getOrCreate(); + TestBase.sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); + TestBase.catalog = (HiveCatalog) CatalogUtil.loadCatalog( 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 6203bf89bfa1..ba864bf89e33 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,11 +18,9 @@ */ package org.apache.iceberg.spark; -import java.nio.file.Path; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; @ExtendWith(ParameterizedTestExtension.class) public abstract class CatalogTestBase extends TestBaseWithCatalog { @@ -48,6 +46,4 @@ protected static Object[][] parameters() { } }; } - - @TempDir protected Path temp; } 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 c3c958abf0fd..c869c4a30a19 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 @@ -73,7 +73,7 @@ public static void dropWarehouse() throws IOException { } } - @TempDir protected File temp; + @TempDir protected java.nio.file.Path temp; @Parameter(index = 0) protected String catalogName; 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 189c125b4e3b..c5463b4249d6 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 @@ -410,13 +410,13 @@ private DeleteFile writeEqDeletes(Table table, String col, Object... values) thr deletes.add(delete.copy(col, value)); } - OutputFile out = Files.localOutput(new File(temp, "eq-deletes-" + UUID.randomUUID())); + OutputFile out = Files.localOutput(new File(temp.toFile(), "eq-deletes-" + UUID.randomUUID())); return FileHelpers.writeDeleteFile(table, out, null, deletes, deleteSchema); } private Pair writePosDeletes( Table table, List> deletes) throws IOException { - OutputFile out = Files.localOutput(new File(temp, "pos-deletes-" + UUID.randomUUID())); + OutputFile out = Files.localOutput(new File(temp.toFile(), "pos-deletes-" + UUID.randomUUID())); return FileHelpers.writeDeleteFile(table, out, null, deletes); } From fb2d0d8600037b50aab8ad97b2bda42f297f39ad Mon Sep 17 00:00:00 2001 From: westse <10818305+westse@users.noreply.github.com> Date: Fri, 12 Apr 2024 11:28:59 -0600 Subject: [PATCH 0213/1019] Spec: Make request bodies required (#10125) --- open-api/rest-catalog-open-api.yaml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 67dc235d3b20..9c2300dcf6a4 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -170,6 +170,7 @@ paths: sending a token exchange request (3). The request's "subject" token should be the expiring token. This request should use the subject token in the "Authorization" header. requestBody: + required: true content: application/x-www-form-urlencoded: schema: @@ -248,6 +249,7 @@ paths: The server might also add properties, such as `last_modified_time` etc. operationId: createNamespace requestBody: + required: true content: application/json: schema: @@ -396,6 +398,7 @@ paths: Server implementations are not required to support namespace properties. requestBody: + required: true content: application/json: schema: @@ -499,6 +502,7 @@ paths: parameters: - $ref: '#/components/parameters/data-access' requestBody: + required: true content: application/json: schema: @@ -551,6 +555,7 @@ paths: operationId: registerTable requestBody: + required: true content: application/json: schema: @@ -680,6 +685,7 @@ paths: initialization, like AddSchemaUpdate and SetCurrentSchemaUpdate. The `assert-create` requirement is used to ensure that the table was not created concurrently. requestBody: + required: true content: application/json: schema: @@ -1111,6 +1117,7 @@ paths: Create a view in the given namespace. operationId: createView requestBody: + required: true content: application/json: schema: @@ -1209,6 +1216,7 @@ paths: description: Commit updates to a view. requestBody: + required: true content: application/json: schema: From 9a057d896d48bde77c8dee1cc35ab29b40c8c154 Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Sat, 13 Apr 2024 05:12:28 +0800 Subject: [PATCH 0214/1019] Spark 3.5: Support preserving schema nullability in CTAS and RTAS (#10074) --- docs/docs/spark-configuration.md | 1 + .../org/apache/iceberg/spark/BaseCatalog.java | 20 ++++ .../apache/iceberg/spark/SparkCatalog.java | 2 + .../iceberg/spark/SparkSessionCatalog.java | 2 + .../spark/TestSparkCatalogOperations.java | 102 ++++++++++++++++++ 5 files changed, 127 insertions(+) diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 6ac4f1e9c82a..9ff739649839 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -77,6 +77,7 @@ Both catalogs are configured using properties nested under the catalog name. Com | 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_.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). 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 38f15a42958c..2082c0584608 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 @@ -21,12 +21,14 @@ import org.apache.iceberg.spark.procedures.SparkProcedures; import org.apache.iceberg.spark.procedures.SparkProcedures.ProcedureBuilder; import org.apache.iceberg.spark.source.HasIcebergCatalog; +import org.apache.iceberg.util.PropertyUtil; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; 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.iceberg.catalog.Procedure; import org.apache.spark.sql.connector.iceberg.catalog.ProcedureCatalog; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; abstract class BaseCatalog implements StagingTableCatalog, @@ -34,6 +36,10 @@ abstract class BaseCatalog SupportsNamespaces, HasIcebergCatalog, SupportsFunctions { + 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; + + private boolean useNullableQuerySchema = USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS_DEFAULT; @Override public Procedure loadProcedure(Identifier ident) throws NoSuchProcedureException { @@ -66,6 +72,20 @@ public boolean isExistingNamespace(String[] namespace) { return namespaceExists(namespace); } + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + this.useNullableQuerySchema = + PropertyUtil.propertyAsBoolean( + options, + USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS, + USE_NULLABLE_QUERY_SCHEMA_CTAS_RTAS_DEFAULT); + } + + @Override + public boolean useNullableQuerySchema() { + return useNullableQuerySchema; + } + private static boolean isSystemNamespace(String[] namespace) { return namespace.length == 1 && namespace[0].equalsIgnoreCase("system"); } 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 0be9fd948466..0c361598623e 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 @@ -726,6 +726,8 @@ public void renameView(Identifier fromIdentifier, Identifier toIdentifier) @Override public final void initialize(String name, CaseInsensitiveStringMap options) { + super.initialize(name, options); + this.cacheEnabled = PropertyUtil.propertyAsBoolean( options, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); 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 33384e3eff08..fa3f1fbe4b2a 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 @@ -304,6 +304,8 @@ public void renameTable(Identifier from, Identifier to) @Override public final void initialize(String name, CaseInsensitiveStringMap options) { + super.initialize(name, options); + if (options.containsKey(CatalogUtil.ICEBERG_CATALOG_TYPE) && options .get(CatalogUtil.ICEBERG_CATALOG_TYPE) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java index 0f29faf274dd..d0860ff01451 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java @@ -20,8 +20,13 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Schema; import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.Table; @@ -33,6 +38,47 @@ import org.junit.jupiter.api.TestTemplate; public class TestSparkCatalogOperations extends CatalogTestBase { + private static boolean useNullableQuerySchema = ThreadLocalRandom.current().nextBoolean(); + + @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") + protected static Object[][] parameters() { + return new Object[][] { + { + SparkCatalogConfig.HIVE.catalogName(), + SparkCatalogConfig.HIVE.implementation(), + ImmutableMap.of( + "type", "hive", + "default-namespace", "default", + "use-nullable-query-schema", Boolean.toString(useNullableQuerySchema)) + }, + { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + ImmutableMap.of( + "type", + "hadoop", + "cache-enabled", + "false", + "use-nullable-query-schema", + Boolean.toString(useNullableQuerySchema)) + }, + { + SparkCatalogConfig.SPARK.catalogName(), + SparkCatalogConfig.SPARK.implementation(), + ImmutableMap.of( + "type", + "hive", + "default-namespace", + "default", + "parquet-enabled", + "true", + "cache-enabled", + "false", // Spark will delete tables using v1, leaving the cache out of sync + "use-nullable-query-schema", + Boolean.toString(useNullableQuerySchema)), + } + }; + } @BeforeEach public void createTable() { @@ -86,4 +132,60 @@ public void testInvalidateTable() { sql("REFRESH TABLE %s", tableName); sql("SELECT count(1) FROM %s", tableName); } + + @TestTemplate + public void testCTASUseNullableQuerySchema() { + sql("INSERT INTO %s VALUES(1, 'abc'), (2, null)", tableName); + + String ctasTableName = tableName("ctas_table"); + + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", ctasTableName, tableName); + + org.apache.iceberg.Table ctasTable = + validationCatalog.loadTable(TableIdentifier.parse("default.ctas_table")); + + Schema expectedSchema = + new Schema( + useNullableQuerySchema + ? Types.NestedField.optional(1, "id", Types.LongType.get()) + : Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(ctasTable.schema().asStruct()) + .as("Should have expected schema") + .isEqualTo(expectedSchema.asStruct()); + + sql("DROP TABLE IF EXISTS %s", ctasTableName); + } + + @TestTemplate + public void testRTASUseNullableQuerySchema() { + sql("INSERT INTO %s VALUES(1, 'abc'), (2, null)", tableName); + + String rtasTableName = tableName("rtas_table"); + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", rtasTableName); + + sql("REPLACE TABLE %s USING iceberg AS SELECT * FROM %s", rtasTableName, tableName); + + org.apache.iceberg.Table rtasTable = + validationCatalog.loadTable(TableIdentifier.parse("default.rtas_table")); + + Schema expectedSchema = + new Schema( + useNullableQuerySchema + ? Types.NestedField.optional(1, "id", Types.LongType.get()) + : Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + assertThat(rtasTable.schema().asStruct()) + .as("Should have expected schema") + .isEqualTo(expectedSchema.asStruct()); + + assertEquals( + "Should have rows matching the source table", + sql("SELECT * FROM %s ORDER BY id", tableName), + sql("SELECT * FROM %s ORDER BY id", rtasTableName)); + + sql("DROP TABLE IF EXISTS %s", rtasTableName); + } } From b721253cae645d6f625778ed56b296fc12eec1d6 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 14 Apr 2024 05:19:55 +0200 Subject: [PATCH 0215/1019] Flink, Spark: Replace Boolean.getBoolean() with Boolean.parseBoolean() (#10136) --- .../java/org/apache/iceberg/flink/source/TestFlinkSource.java | 2 +- .../java/org/apache/iceberg/flink/source/TestFlinkSource.java | 2 +- .../java/org/apache/iceberg/flink/source/TestFlinkSource.java | 2 +- .../src/main/java/org/apache/iceberg/spark/SparkTableUtil.java | 2 +- .../src/main/java/org/apache/iceberg/spark/SparkTableUtil.java | 2 +- .../src/main/java/org/apache/iceberg/spark/SparkTableUtil.java | 2 +- 6 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index 86c7e8991db8..ff7892071231 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -60,7 +60,7 @@ protected List runWithFilter(Expression filter, String sqlFilter, boolean c protected List runWithOptions(Map options) throws Exception { FlinkSource.Builder builder = FlinkSource.forRowData(); Optional.ofNullable(options.get("case-sensitive")) - .ifPresent(value -> builder.caseSensitive(Boolean.getBoolean(value))); + .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); Optional.ofNullable(options.get("snapshot-id")) .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index 86c7e8991db8..ff7892071231 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -60,7 +60,7 @@ protected List runWithFilter(Expression filter, String sqlFilter, boolean c protected List runWithOptions(Map options) throws Exception { FlinkSource.Builder builder = FlinkSource.forRowData(); Optional.ofNullable(options.get("case-sensitive")) - .ifPresent(value -> builder.caseSensitive(Boolean.getBoolean(value))); + .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); Optional.ofNullable(options.get("snapshot-id")) .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index 86c7e8991db8..ff7892071231 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -60,7 +60,7 @@ protected List runWithFilter(Expression filter, String sqlFilter, boolean c protected List runWithOptions(Map options) throws Exception { FlinkSource.Builder builder = FlinkSource.forRowData(); Optional.ofNullable(options.get("case-sensitive")) - .ifPresent(value -> builder.caseSensitive(Boolean.getBoolean(value))); + .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); Optional.ofNullable(options.get("snapshot-id")) .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); 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 6cb31a3b8830..2da0cf70992b 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 @@ -754,7 +754,7 @@ public static boolean wapEnabled(Table table) { return PropertyUtil.propertyAsBoolean( table.properties(), TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, - Boolean.getBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); + Boolean.parseBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); } /** Class representing a table partition. */ 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 51df02d56959..5af6b46b3178 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 @@ -705,7 +705,7 @@ public static boolean wapEnabled(Table table) { return PropertyUtil.propertyAsBoolean( table.properties(), TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, - Boolean.getBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); + Boolean.parseBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); } /** Class representing a table partition. */ 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 3a2324d89184..eb27e1483d13 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 @@ -770,7 +770,7 @@ public static boolean wapEnabled(Table table) { return PropertyUtil.propertyAsBoolean( table.properties(), TableProperties.WRITE_AUDIT_PUBLISH_ENABLED, - Boolean.getBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); + Boolean.parseBoolean(TableProperties.WRITE_AUDIT_PUBLISH_ENABLED_DEFAULT)); } /** Class representing a table partition. */ From b9f5d1887f2ddfc8ae9a38a99f3bda252e0dec83 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Apr 2024 07:16:47 +0200 Subject: [PATCH 0216/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.14.5 to 3.15.1 (#10095) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.14.5 to 3.15.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.14.5...v3.15.1) --- 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 fa5b8f268d05..130d19433396 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -76,7 +76,7 @@ roaringbitmap = "1.0.5" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.14.5" +snowflake-jdbc = "3.15.1" spark-hive33 = "3.3.4" spark-hive34 = "3.4.2" spark-hive35 = "3.5.1" From ee4ab20d00c350ffc48ed656493e1a56393e5539 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Apr 2024 09:27:53 +0200 Subject: [PATCH 0217/1019] Build: Bump software.amazon.awssdk:bom from 2.25.21 to 2.25.31 (#10138) Bumps software.amazon.awssdk:bom from 2.25.21 to 2.25.31. --- 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 130d19433396..a3cca47cd0e6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.21" +awssdk-bom = "2.25.31" azuresdk-bom = "1.2.22" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From a6b25ba533d7c9b9585d06a9c65ee5ca6daf3e87 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Apr 2024 09:28:02 +0200 Subject: [PATCH 0218/1019] Build: Bump org.springframework:spring-web from 5.3.33 to 5.3.34 (#10139) Bumps [org.springframework:spring-web](https://github.com/spring-projects/spring-framework) from 5.3.33 to 5.3.34. - [Release notes](https://github.com/spring-projects/spring-framework/releases) - [Commits](https://github.com/spring-projects/spring-framework/compare/v5.3.33...v5.3.34) --- updated-dependencies: - dependency-name: org.springframework:spring-web 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 a3cca47cd0e6..23628d26fd4f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -81,7 +81,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.2" spark-hive35 = "3.5.1" spring-boot = "2.7.18" -spring-web = "5.3.33" +spring-web = "5.3.34" sqlite-jdbc = "3.45.2.0" testcontainers = "1.19.7" tez010 = "0.10.3" From 9337d0f825db816abc4c51dc5d0cd4e8f064cd13 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Sun, 14 Apr 2024 07:44:28 -0600 Subject: [PATCH 0219/1019] Spark: Simplify SparkSchemaUtil#schemaForTable (#10137) --- .../main/java/org/apache/iceberg/spark/SparkSchemaUtil.java | 4 +--- .../main/java/org/apache/iceberg/spark/SparkSchemaUtil.java | 4 +--- .../main/java/org/apache/iceberg/spark/SparkSchemaUtil.java | 4 +--- 3 files changed, 3 insertions(+), 9 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index 6075aba7ac5f..b1cc0ac620cb 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -59,9 +59,7 @@ private SparkSchemaUtil() {} * @return a Schema for the table, if found */ public static Schema schemaForTable(SparkSession spark, String name) { - StructType sparkType = spark.table(name).schema(); - Type converted = SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)); - return new Schema(converted.asNestedType().asStructType().fields()); + return convert(spark.table(name).schema()); } /** diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index 21a1d92a11a9..d0f77bcdd9cc 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -59,9 +59,7 @@ private SparkSchemaUtil() {} * @return a Schema for the table, if found */ public static Schema schemaForTable(SparkSession spark, String name) { - StructType sparkType = spark.table(name).schema(); - Type converted = SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)); - return new Schema(converted.asNestedType().asStructType().fields()); + return convert(spark.table(name).schema()); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java index 21a1d92a11a9..d0f77bcdd9cc 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSchemaUtil.java @@ -59,9 +59,7 @@ private SparkSchemaUtil() {} * @return a Schema for the table, if found */ public static Schema schemaForTable(SparkSession spark, String name) { - StructType sparkType = spark.table(name).schema(); - Type converted = SparkTypeVisitor.visit(sparkType, new SparkTypeToType(sparkType)); - return new Schema(converted.asNestedType().asStructType().fields()); + return convert(spark.table(name).schema()); } /** From dab55b34b7711dc91983a5032e787746a1292d02 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Mon, 15 Apr 2024 15:45:45 +0900 Subject: [PATCH 0220/1019] Flink: Migrate tests to JUnit5 (#10130) --- .../org/apache/iceberg/data/DataTest.java | 8 +- .../iceberg/data/avro/TestGenericData.java | 7 +- .../iceberg/data/orc/TestGenericData.java | 81 +++-- .../iceberg/data/parquet/TestGenericData.java | 20 +- ...TestParquetEncryptionWithWriteSupport.java | 41 +-- .../flink/data/TestFlinkAvroReaderWriter.java | 21 +- .../flink/data/TestFlinkOrcReaderWriter.java | 19 +- .../flink/data/TestFlinkParquetReader.java | 24 +- .../flink/data/TestFlinkParquetWriter.java | 17 +- .../flink/data/TestFlinkAvroReaderWriter.java | 21 +- .../flink/data/TestFlinkOrcReaderWriter.java | 19 +- .../flink/data/TestFlinkParquetReader.java | 24 +- .../flink/data/TestFlinkParquetWriter.java | 17 +- .../flink/AvroGenericRecordConverterBase.java | 2 +- .../flink/TestDataFileSerialization.java | 14 +- .../flink/TestFlinkCatalogFactory.java | 20 +- .../iceberg/flink/TestFlinkFilters.java | 87 +++--- .../iceberg/flink/TestFlinkSchemaUtil.java | 40 +-- .../flink/TestManifestFileSerialization.java | 17 +- .../iceberg/flink/TestRowDataWrapper.java | 18 +- .../iceberg/flink/TestTableSerialization.java | 18 +- .../flink/data/TestFlinkAvroReaderWriter.java | 21 +- .../flink/data/TestFlinkOrcReaderWriter.java | 19 +- .../flink/data/TestFlinkParquetReader.java | 24 +- .../flink/data/TestFlinkParquetWriter.java | 17 +- .../iceberg/flink/data/TestRowProjection.java | 280 +++++++++--------- .../iceberg/flink/data/TestStructRowData.java | 2 +- .../TestAvroGenericRecordToRowDataMapper.java | 5 +- ...stRowDataToAvroGenericRecordConverter.java | 5 +- .../iceberg/flink/util/TestFlinkPackage.java | 11 +- 30 files changed, 483 insertions(+), 436 deletions(-) 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 7e32da4c6edf..5ea742e45188 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTest.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTest.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import java.io.IOException; +import java.nio.file.Path; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.Schema; import org.apache.iceberg.types.TypeUtil; @@ -30,9 +31,8 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.MapType; import org.apache.iceberg.types.Types.StructType; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class DataTest { @@ -58,7 +58,7 @@ public abstract class DataTest { required(116, "dec_38_10", Types.DecimalType.of(38, 10)), // maximum precision required(117, "time", Types.TimeType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir protected Path temp; @Test public void testSimpleStruct() throws IOException { 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 ef690919ae07..83e8c09449e4 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,6 +18,8 @@ */ package org.apache.iceberg.data.avro; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -31,15 +33,14 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; public class TestGenericData extends DataTest { @Override protected void writeAndValidate(Schema schema) throws IOException { List expected = RandomGenericData.generate(schema, 100, 0L); - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender writer = Avro.write(Files.localOutput(testFile)) diff --git a/data/src/test/java/org/apache/iceberg/data/orc/TestGenericData.java b/data/src/test/java/org/apache/iceberg/data/orc/TestGenericData.java index 180f7a6ad0fc..5147fd377c62 100644 --- a/data/src/test/java/org/apache/iceberg/data/orc/TestGenericData.java +++ b/data/src/test/java/org/apache/iceberg/data/orc/TestGenericData.java @@ -20,6 +20,7 @@ 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; @@ -49,8 +50,7 @@ import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; import org.apache.orc.storage.ql.exec.vector.LongColumnVector; import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestGenericData extends DataTest { @@ -97,8 +97,8 @@ public void writeAndValidateTimestamps() throws IOException { record4.setField("tsTzCol", OffsetDateTime.parse("1935-05-16T17:10:34-08:00")); record4.setField("tsCol", LocalDateTime.parse("1935-05-01T00:01:00")); - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender writer = ORC.write(Files.localOutput(testFile)) @@ -123,22 +123,42 @@ public void writeAndValidateTimestamps() throws IOException { rows = Lists.newArrayList(reader); } - Assert.assertEquals( - OffsetDateTime.parse("2017-01-17T01:10:34Z"), rows.get(0).getField("tsTzCol")); - Assert.assertEquals( - LocalDateTime.parse("1970-01-01T00:01:00"), rows.get(0).getField("tsCol")); - Assert.assertEquals( - OffsetDateTime.parse("2017-05-17T01:10:34Z"), rows.get(1).getField("tsTzCol")); - Assert.assertEquals( - LocalDateTime.parse("1970-05-01T00:01:00"), rows.get(1).getField("tsCol")); - Assert.assertEquals( - OffsetDateTime.parse("1935-01-17T01:10:34Z"), rows.get(2).getField("tsTzCol")); - Assert.assertEquals( - LocalDateTime.parse("1935-01-01T00:01:00"), rows.get(2).getField("tsCol")); - Assert.assertEquals( - OffsetDateTime.parse("1935-05-17T01:10:34Z"), rows.get(3).getField("tsTzCol")); - Assert.assertEquals( - LocalDateTime.parse("1935-05-01T00:01:00"), rows.get(3).getField("tsCol")); + assertThat(rows) + .element(0) + .satisfies( + record -> { + assertThat(record.getField("tsTzCol")) + .isEqualTo(OffsetDateTime.parse("2017-01-17T01:10:34Z")); + assertThat(record.getField("tsCol")) + .isEqualTo(LocalDateTime.parse("1970-01-01T00:01:00")); + }); + assertThat(rows) + .element(1) + .satisfies( + record -> { + assertThat(record.getField("tsTzCol")) + .isEqualTo(OffsetDateTime.parse("2017-05-17T01:10:34Z")); + assertThat(record.getField("tsCol")) + .isEqualTo(LocalDateTime.parse("1970-05-01T00:01:00")); + }); + assertThat(rows) + .element(2) + .satisfies( + record -> { + assertThat(record.getField("tsTzCol")) + .isEqualTo(OffsetDateTime.parse("1935-01-17T01:10:34Z")); + assertThat(record.getField("tsCol")) + .isEqualTo(LocalDateTime.parse("1935-01-01T00:01:00")); + }); + assertThat(rows) + .element(3) + .satisfies( + record -> { + assertThat(record.getField("tsTzCol")) + .isEqualTo(OffsetDateTime.parse("1935-05-17T01:10:34Z")); + assertThat(record.getField("tsCol")) + .isEqualTo(LocalDateTime.parse("1935-05-01T00:01:00")); + }); } finally { TimeZone.setDefault(currentTz); } @@ -146,8 +166,8 @@ public void writeAndValidateTimestamps() throws IOException { @Test public void writeAndValidateExternalData() throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); Configuration conf = new Configuration(); TypeDescription writerSchema = @@ -179,15 +199,20 @@ public void writeAndValidateExternalData() throws IOException { .build()) { rows = Lists.newArrayList(reader); } - Assert.assertEquals(1, rows.get(0).getField("a")); - Assert.assertEquals(123, rows.get(0).getField("b")); - Assert.assertEquals("1", rows.get(0).getField("c")); - Assert.assertEquals("123", rows.get(0).getField("d")); + assertThat(rows) + .first() + .satisfies( + record -> { + assertThat(record.getField("a")).isEqualTo(1); + assertThat(record.getField("b")).isEqualTo(123); + assertThat(record.getField("c")).isEqualTo("1"); + assertThat(record.getField("d")).isEqualTo("123"); + }); } private void writeAndValidateRecords(Schema schema, List expected) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender writer = ORC.write(Files.localOutput(testFile)) 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 71dd16d584a2..6de56570589c 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,6 +19,7 @@ package org.apache.iceberg.data.parquet; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -44,16 +45,15 @@ import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.Assert; -import org.junit.Test; +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); - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender appender = Parquet.write(Files.localOutput(testFile)) @@ -101,8 +101,8 @@ public void testTwoLevelList() throws IOException { optional(2, "topbytes", Types.BinaryType.get())); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); ParquetWriter writer = AvroParquetWriter.builder(new Path(testFile.toURI())) @@ -132,12 +132,12 @@ public void testTwoLevelList() throws IOException { .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { CloseableIterator it = reader.iterator(); - Assert.assertTrue("Should have at least one row", it.hasNext()); + assertThat(it).hasNext(); while (it.hasNext()) { GenericRecord actualRecord = (GenericRecord) it.next(); - Assert.assertEquals(actualRecord.get(0, ArrayList.class).get(0), expectedBinary); - Assert.assertEquals(actualRecord.get(1, ByteBuffer.class), expectedBinary); - Assert.assertFalse("Should not have more than one row", it.hasNext()); + assertThat(actualRecord.get(0, ArrayList.class)).first().isEqualTo(expectedBinary); + assertThat(actualRecord.get(1, ByteBuffer.class)).isEqualTo(expectedBinary); + assertThat(it).isExhausted(); } } } diff --git a/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java index ba382e1c8ab2..c6a5ed9f6d86 100644 --- a/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java @@ -20,6 +20,8 @@ import static org.apache.iceberg.Files.localInput; 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 java.io.File; import java.io.IOException; @@ -48,8 +50,7 @@ import org.apache.parquet.crypto.FileEncryptionProperties; import org.apache.parquet.crypto.ParquetCryptoRuntimeException; import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestParquetEncryptionWithWriteSupport extends DataTest { private static final ByteBuffer fileDek = ByteBuffer.allocate(16); @@ -59,8 +60,8 @@ public class TestParquetEncryptionWithWriteSupport extends DataTest { protected void writeAndValidate(Schema schema) throws IOException { List expected = RandomGenericData.generate(schema, 100, 0L); - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); SecureRandom rand = new SecureRandom(); rand.nextBytes(fileDek.array()); @@ -76,16 +77,16 @@ protected void writeAndValidate(Schema schema) throws IOException { appender.addAll(expected); } - Assert.assertThrows( - "Decrypted without keys", - ParquetCryptoRuntimeException.class, - () -> - Parquet.read(localInput(testFile)) - .project(schema) - .createReaderFunc( - fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) - .build() - .iterator()); + assertThatThrownBy( + () -> + Parquet.read(localInput(testFile)) + .project(schema) + .createReaderFunc( + fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) + .build() + .iterator()) + .hasMessage("Trying to read file with encrypted footer. No keys available") + .isInstanceOf(ParquetCryptoRuntimeException.class); List rows; try (CloseableIterable reader = @@ -129,8 +130,8 @@ public void testTwoLevelList() throws IOException { optional(2, "topbytes", Types.BinaryType.get())); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); SecureRandom rand = new SecureRandom(); rand.nextBytes(fileDek.array()); @@ -169,12 +170,12 @@ public void testTwoLevelList() throws IOException { .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { CloseableIterator it = reader.iterator(); - Assert.assertTrue("Should have at least one row", it.hasNext()); + assertThat(it).hasNext(); while (it.hasNext()) { GenericRecord actualRecord = (GenericRecord) it.next(); - Assert.assertEquals(actualRecord.get(0, ArrayList.class).get(0), expectedBinary); - Assert.assertEquals(actualRecord.get(1, ByteBuffer.class), expectedBinary); - Assert.assertFalse("Should not have more than one row", it.hasNext()); + assertThat(actualRecord.get(0, ArrayList.class)).first().isEqualTo(expectedBinary); + assertThat(actualRecord.get(1, ByteBuffer.class)).isEqualTo(expectedBinary); + assertThat(it).isExhausted(); } } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java index e8aab824ea2d..a1039d27d888 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.math.BigDecimal; @@ -44,8 +46,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkAvroReaderWriter extends DataTest { @@ -74,8 +75,8 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n RowType flinkSchema = FlinkSchemaUtil.convert(schema); List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); // Write the expected records into AVRO file, then read them into RowData and assert with the // expected Record list. @@ -95,14 +96,14 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < numRecord; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra records", rows.hasNext()); + assertThat(rows).isExhausted(); } - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); // Write the expected RowData into AVRO file, then read them into Record and assert with the // expected RowData list. @@ -122,10 +123,10 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < numRecord; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); + assertThat(records).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } - Assert.assertFalse("Should not have extra records", records.hasNext()); + assertThat(records).isExhausted(); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java index fdffc0e01c20..72f2ce4f4bce 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Iterator; @@ -37,7 +39,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; public class TestFlinkOrcReaderWriter extends DataTest { private static final int NUM_RECORDS = 100; @@ -48,8 +49,8 @@ protected void writeAndValidate(Schema schema) throws IOException { List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); // Write the expected records into ORC file, then read them into RowData and assert with the // expected Record list. @@ -69,14 +70,14 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra records", rows.hasNext()); + assertThat(rows).isExhausted(); } - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); // Write the expected RowData into ORC file, then read them into Record and assert with the // expected RowData list. @@ -97,10 +98,10 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); + assertThat(records.hasNext()).isTrue(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } - Assert.assertFalse("Should not have extra records", records.hasNext()); + assertThat(records).isExhausted(); } } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java index 30a2a7bb51ce..1fdc4cf8381a 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -47,8 +48,7 @@ import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkParquetReader extends DataTest { private static final int NUM_RECORDS = 100; @@ -61,8 +61,8 @@ public void testTwoLevelList() throws IOException { optional(2, "topbytes", Types.BinaryType.get())); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); ParquetWriter writer = AvroParquetWriter.builder(new Path(testFile.toURI())) @@ -90,17 +90,17 @@ public void testTwoLevelList() throws IOException { .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) .build()) { Iterator rows = reader.iterator(); - Assert.assertTrue("Should have at least one row", rows.hasNext()); + assertThat(rows).hasNext(); RowData rowData = rows.next(); - Assert.assertArrayEquals(rowData.getArray(0).getBinary(0), expectedByte); - Assert.assertArrayEquals(rowData.getBinary(1), expectedByte); - Assert.assertFalse("Should not have more than one row", rows.hasNext()); + assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); + assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); + assertThat(rows).isExhausted(); } } private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender writer = Parquet.write(Files.localOutput(testFile)) @@ -119,10 +119,10 @@ private void writeAndValidate(Iterable iterable, Schema schema) throws I Iterator rows = reader.iterator(); LogicalType rowType = FlinkSchemaUtil.convert(schema); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra rows", rows.hasNext()); + assertThat(rows).isExhausted(); } } diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java index 7b868eafc311..b1e6f5aa00ff 100644 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java +++ b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.data; +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 org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.LogicalType; @@ -34,18 +37,16 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkParquetWriter extends DataTest { private static final int NUM_RECORDS = 100; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); LogicalType logicalType = FlinkSchemaUtil.convert(schema); @@ -66,10 +67,10 @@ private void writeAndValidate(Iterable iterable, Schema schema) throws Iterator actual = reader.iterator(); LogicalType rowType = FlinkSchemaUtil.convert(schema); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", actual.hasNext()); + assertThat(actual).hasNext(); TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); } - Assert.assertFalse("Should not have extra rows", actual.hasNext()); + assertThat(actual).isExhausted(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java index e8aab824ea2d..a1039d27d888 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.math.BigDecimal; @@ -44,8 +46,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkAvroReaderWriter extends DataTest { @@ -74,8 +75,8 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n RowType flinkSchema = FlinkSchemaUtil.convert(schema); List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); // Write the expected records into AVRO file, then read them into RowData and assert with the // expected Record list. @@ -95,14 +96,14 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < numRecord; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra records", rows.hasNext()); + assertThat(rows).isExhausted(); } - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); // Write the expected RowData into AVRO file, then read them into Record and assert with the // expected RowData list. @@ -122,10 +123,10 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < numRecord; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); + assertThat(records).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } - Assert.assertFalse("Should not have extra records", records.hasNext()); + assertThat(records).isExhausted(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java index fdffc0e01c20..91ee017238ac 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Iterator; @@ -37,7 +39,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; public class TestFlinkOrcReaderWriter extends DataTest { private static final int NUM_RECORDS = 100; @@ -48,8 +49,8 @@ protected void writeAndValidate(Schema schema) throws IOException { List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); // Write the expected records into ORC file, then read them into RowData and assert with the // expected Record list. @@ -69,14 +70,14 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); + assertThat(rows.hasNext()).isTrue(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra records", rows.hasNext()); + assertThat(rows).isExhausted(); } - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); // Write the expected RowData into ORC file, then read them into Record and assert with the // expected RowData list. @@ -97,10 +98,10 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); + assertThat(records).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } - Assert.assertFalse("Should not have extra records", records.hasNext()); + assertThat(records).isExhausted(); } } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java index 30a2a7bb51ce..1fdc4cf8381a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -47,8 +48,7 @@ import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkParquetReader extends DataTest { private static final int NUM_RECORDS = 100; @@ -61,8 +61,8 @@ public void testTwoLevelList() throws IOException { optional(2, "topbytes", Types.BinaryType.get())); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); ParquetWriter writer = AvroParquetWriter.builder(new Path(testFile.toURI())) @@ -90,17 +90,17 @@ public void testTwoLevelList() throws IOException { .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) .build()) { Iterator rows = reader.iterator(); - Assert.assertTrue("Should have at least one row", rows.hasNext()); + assertThat(rows).hasNext(); RowData rowData = rows.next(); - Assert.assertArrayEquals(rowData.getArray(0).getBinary(0), expectedByte); - Assert.assertArrayEquals(rowData.getBinary(1), expectedByte); - Assert.assertFalse("Should not have more than one row", rows.hasNext()); + assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); + assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); + assertThat(rows).isExhausted(); } } private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender writer = Parquet.write(Files.localOutput(testFile)) @@ -119,10 +119,10 @@ private void writeAndValidate(Iterable iterable, Schema schema) throws I Iterator rows = reader.iterator(); LogicalType rowType = FlinkSchemaUtil.convert(schema); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra rows", rows.hasNext()); + assertThat(rows).isExhausted(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java index 7b868eafc311..b1e6f5aa00ff 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.data; +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 org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.LogicalType; @@ -34,18 +37,16 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkParquetWriter extends DataTest { private static final int NUM_RECORDS = 100; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); LogicalType logicalType = FlinkSchemaUtil.convert(schema); @@ -66,10 +67,10 @@ private void writeAndValidate(Iterable iterable, Schema schema) throws Iterator actual = reader.iterator(); LogicalType rowType = FlinkSchemaUtil.convert(schema); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", actual.hasNext()); + assertThat(actual).hasNext(); TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); } - Assert.assertFalse("Should not have extra rows", actual.hasNext()); + assertThat(actual).isExhausted(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java index 47319ec9bc60..4184526a6a1a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink; -import org.junit.Test; +import org.junit.jupiter.api.Test; public abstract class AvroGenericRecordConverterBase { protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java index e9372adda4c1..8992cbd75187 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java @@ -20,6 +20,7 @@ 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.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -44,8 +45,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDataFileSerialization { @@ -135,23 +135,19 @@ public void testJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); } for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a position DeleteFile") - .isInstanceOf(DeleteFile.class); + assertThat(obj).as("Should be a position DeleteFile").isInstanceOf(DeleteFile.class); TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); } for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a equality DeleteFile") - .isInstanceOf(DeleteFile.class); + assertThat(obj).as("Should be a equality DeleteFile").isInstanceOf(DeleteFile.class); TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java index ba08b76dd59d..4c9e95b8fa82 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; @@ -26,15 +29,14 @@ import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestFlinkCatalogFactory { private Map props; - @Before + @BeforeEach public void before() { props = Maps.newHashMap(); props.put("type", "iceberg"); @@ -51,7 +53,7 @@ public void testCreateCatalogHive() { FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) .loadCatalog(); - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); + assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); } @Test @@ -64,7 +66,7 @@ public void testCreateCatalogHadoop() { FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) .loadCatalog(); - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); + assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); } @Test @@ -76,7 +78,7 @@ public void testCreateCatalogCustom() { FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) .loadCatalog(); - Assertions.assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); + assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); } @Test @@ -86,7 +88,7 @@ public void testCreateCatalogCustomWithHiveCatalogTypeSet() { props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith( @@ -98,7 +100,7 @@ public void testLoadCatalogUnknown() { String catalogName = "unknownCatalog"; props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(UnsupportedOperationException.class) .hasMessageStartingWith("Unknown catalog-type: fooType"); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java index c89ea4f53054..838b0ea0e1a9 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.time.Instant; @@ -49,9 +51,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkFilters { @@ -121,13 +121,13 @@ public void testEquals() { Optional actual = FlinkFilters.convert( resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert( resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } } @@ -138,12 +138,12 @@ public void testEqualsNaN() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -156,13 +156,13 @@ public void testNotEquals() { Optional actual = FlinkFilters.convert( resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert( resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } } @@ -174,13 +174,13 @@ public void testNotEqualsNaN() { Optional actual = FlinkFilters.convert( resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert( resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -191,12 +191,12 @@ public void testGreaterThan() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -207,12 +207,12 @@ public void testGreaterThanEquals() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -223,12 +223,12 @@ public void testLessThan() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -239,12 +239,12 @@ public void testLessThanEquals() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -252,7 +252,7 @@ public void testLessThanEquals() { public void testIsNull() { Expression expr = resolve(Expressions.$("field1").isNull()); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); assertPredicatesMatch(expected, actual.get()); } @@ -261,7 +261,7 @@ public void testIsNull() { public void testIsNotNull() { Expression expr = resolve(Expressions.$("field1").isNotNull()); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNull("field1"); assertPredicatesMatch(expected, actual.get()); @@ -275,7 +275,7 @@ public void testAnd() { .isEqual(Expressions.lit(1)) .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); And and = (And) actual.get(); And expected = (And) @@ -295,7 +295,7 @@ public void testOr() { .isEqual(Expressions.lit(1)) .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); Or or = (Or) actual.get(); Or expected = (Or) @@ -315,14 +315,14 @@ public void testNot() { BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1)))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); Not not = (Not) actual.get(); Not expected = (Not) org.apache.iceberg.expressions.Expressions.not( org.apache.iceberg.expressions.Expressions.equal("field1", 1)); - Assert.assertEquals("Predicate operation should match", expected.op(), not.op()); + assertThat(not.op()).as("Predicate operation should match").isEqualTo(expected.op()); assertPredicatesMatch(expected.child(), not.child()); } @@ -335,7 +335,7 @@ public void testLike() { ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); expr = @@ -343,7 +343,7 @@ public void testLike() { ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( @@ -352,7 +352,7 @@ public void testLike() { Expressions.$("field5"), Expressions.lit("%abc%"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( @@ -361,49 +361,49 @@ public void testLike() { Expressions.$("field5"), Expressions.lit("abc%d"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); } @SuppressWarnings("unchecked") private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); org.apache.iceberg.expressions.Expression expression = actual.get(); - Assertions.assertThat(expression) + assertThat(expression) .as("The expression should be a UnboundPredicate") .isInstanceOf(UnboundPredicate.class); UnboundPredicate unboundPredicate = (UnboundPredicate) expression; org.apache.iceberg.expressions.Expression expression1 = unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); - Assertions.assertThat(expression1) + assertThat(expression1) .as("The expression should be a BoundLiteralPredicate") .isInstanceOf(BoundLiteralPredicate.class); BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; - Assert.assertTrue("Should match the literal", predicate.test(icebergLiteral)); + assertThat(predicate.test(icebergLiteral)).isTrue(); } private static Expression resolve(Expression originalExpression) { @@ -447,21 +447,16 @@ protected Expression defaultMethod(Expression expression) { private void assertPredicatesMatch( org.apache.iceberg.expressions.Expression expected, org.apache.iceberg.expressions.Expression actual) { - Assertions.assertThat(expected) + assertThat(expected) .as("The expected expression should be a UnboundPredicate") .isInstanceOf(UnboundPredicate.class); - Assertions.assertThat(actual) + assertThat(actual) .as("The actual expression should be a UnboundPredicate") .isInstanceOf(UnboundPredicate.class); UnboundPredicate predicateExpected = (UnboundPredicate) expected; UnboundPredicate predicateActual = (UnboundPredicate) actual; - Assert.assertEquals( - "Predicate operation should match", predicateExpected.op(), predicateActual.op()); - Assert.assertEquals( - "Predicate literal should match", predicateExpected.literal(), predicateActual.literal()); - Assert.assertEquals( - "Predicate name should match", - predicateExpected.ref().name(), - predicateActual.ref().name()); + assertThat(predicateActual.op()).isEqualTo(predicateExpected.op()); + assertThat(predicateActual.literal()).isEqualTo(predicateExpected.literal()); + assertThat(predicateActual.ref().name()).isEqualTo(predicateExpected.ref().name()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java index 4ac32c08ebef..eab60d886ada 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; @@ -31,14 +34,11 @@ import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.iceberg.Schema; -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.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkSchemaUtil { @@ -313,12 +313,13 @@ public void testListField() { } private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { - Assert.assertEquals(icebergSchema.asStruct(), FlinkSchemaUtil.convert(flinkSchema).asStruct()); + assertThat(FlinkSchemaUtil.convert(flinkSchema).asStruct()).isEqualTo(icebergSchema.asStruct()); // The conversion is not a 1:1 mapping, so we just check iceberg types. - Assert.assertEquals( - icebergSchema.asStruct(), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) - .asStruct()); + assertThat( + FlinkSchemaUtil.convert( + FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) + .asStruct()) + .isEqualTo(icebergSchema.asStruct()); } @Test @@ -354,10 +355,9 @@ private void checkInconsistentType( LogicalType flinkExpectedType, LogicalType flinkType, Type icebergExpectedType) { - Assert.assertEquals(flinkExpectedType, FlinkSchemaUtil.convert(icebergType)); - Assert.assertEquals( - Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType)), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()); + assertThat(FlinkSchemaUtil.convert(icebergType)).isEqualTo(flinkExpectedType); + assertThat(FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()) + .isEqualTo(Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType))); } @Test @@ -376,8 +376,8 @@ public void testConvertFlinkSchemaBaseOnIcebergSchema() { .primaryKey("int") .build(); Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); - Assert.assertEquals(baseSchema.asStruct(), convertedSchema.asStruct()); - Assert.assertEquals(ImmutableSet.of(101), convertedSchema.identifierFieldIds()); + assertThat(convertedSchema.asStruct()).isEqualTo(baseSchema.asStruct()); + assertThat(convertedSchema.identifierFieldIds()).containsExactly(101); } @Test @@ -390,10 +390,10 @@ public void testConvertFlinkSchemaWithPrimaryKeys() { Sets.newHashSet(1, 2)); TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); - Assert.assertTrue(tableSchema.getPrimaryKey().isPresent()); - Assert.assertEquals( - ImmutableSet.of("int", "string"), - ImmutableSet.copyOf(tableSchema.getPrimaryKey().get().getColumns())); + assertThat(tableSchema.getPrimaryKey()) + .isPresent() + .get() + .satisfies(k -> assertThat(k.getColumns()).containsExactly("int", "string")); } @Test @@ -408,7 +408,7 @@ public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { Types.NestedField.required(2, "inner", Types.IntegerType.get())))), Sets.newHashSet(2)); - Assertions.assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) + assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Could not create a PRIMARY KEY") .hasMessageContaining("Column 'struct.inner' does not exist."); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java index 6bd94e9ca61c..8f1f129e183b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -20,6 +20,7 @@ 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.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -29,6 +30,7 @@ import java.io.ObjectOutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.Path; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputDeserializer; @@ -48,11 +50,8 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestManifestFileSerialization { @@ -104,7 +103,7 @@ public class TestManifestFileSerialization { private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testKryoSerialization() throws IOException { @@ -145,15 +144,15 @@ public void testJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 3; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); TestHelpers.assertEquals(manifest, (ManifestFile) obj); } } } private ManifestFile writeManifest(DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); - Assert.assertTrue(manifestFile.delete()); + File manifestFile = File.createTempFile("input", "m0.avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index c78fa51215dd..caefbb5a5429 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Iterator; import org.apache.flink.table.data.RowData; import org.apache.iceberg.RecordWrapperTest; @@ -28,8 +30,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.util.StructLikeWrapper; -import org.assertj.core.api.Assertions; -import org.junit.Assert; public class TestRowDataWrapper extends RecordWrapperTest { @@ -49,12 +49,12 @@ public void testTime() { return; } - Assertions.assertThat(actual).isNotNull(); - Assertions.assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); int expectedMilliseconds = (int) ((long) expected / 1000_000); int actualMilliseconds = (int) ((long) actual / 1000_000); - Assert.assertEquals(message, expectedMilliseconds, actualMilliseconds); + assertThat(actualMilliseconds).as(message).isEqualTo(expectedMilliseconds); } }); } @@ -75,8 +75,8 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); for (int i = 0; i < numRecords; i++) { - Assert.assertTrue("Should have more records", actual.hasNext()); - Assert.assertTrue("Should have more RowData", expected.hasNext()); + assertThat(actual).hasNext(); + assertThat(expected).hasNext(); StructLike recordStructLike = recordWrapper.wrap(actual.next()); StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); @@ -87,7 +87,7 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod expectedWrapper.set(rowDataStructLike)); } - Assert.assertFalse("Shouldn't have more record", actual.hasNext()); - Assert.assertFalse("Shouldn't have more RowData", expected.hasNext()); + assertThat(actual).isExhausted(); + assertThat(expected).isExhausted(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java index 27124d93fef4..7f0e7acaa822 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java @@ -21,9 +21,11 @@ import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; 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.Map; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; @@ -39,11 +41,9 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestTableSerialization { private static final HadoopTables TABLES = new HadoopTables(); @@ -60,15 +60,15 @@ public class TestTableSerialization { private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private Table table; - @Before + @BeforeEach public void initTable() throws IOException { Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - File tableLocation = temp.newFolder(); - Assert.assertTrue(tableLocation.delete()); + File tableLocation = File.createTempFile("junit", null, temp.toFile()); + assertThat(tableLocation.delete()).isTrue(); this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); } 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/TestFlinkAvroReaderWriter.java index e8aab824ea2d..a1039d27d888 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/TestFlinkAvroReaderWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.math.BigDecimal; @@ -44,8 +46,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkAvroReaderWriter extends DataTest { @@ -74,8 +75,8 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n RowType flinkSchema = FlinkSchemaUtil.convert(schema); List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); // Write the expected records into AVRO file, then read them into RowData and assert with the // expected Record list. @@ -95,14 +96,14 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < numRecord; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra records", rows.hasNext()); + assertThat(rows).isExhausted(); } - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); // Write the expected RowData into AVRO file, then read them into Record and assert with the // expected RowData list. @@ -122,10 +123,10 @@ private void writeAndValidate(Schema schema, List expectedRecords, int n Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < numRecord; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); + assertThat(records).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } - Assert.assertFalse("Should not have extra records", records.hasNext()); + assertThat(records).isExhausted(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java index fdffc0e01c20..72f2ce4f4bce 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Iterator; @@ -37,7 +39,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; public class TestFlinkOrcReaderWriter extends DataTest { private static final int NUM_RECORDS = 100; @@ -48,8 +49,8 @@ protected void writeAndValidate(Schema schema) throws IOException { List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - File recordsFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", recordsFile.delete()); + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); // Write the expected records into ORC file, then read them into RowData and assert with the // expected Record list. @@ -69,14 +70,14 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRecords.iterator(); Iterator rows = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i++) { - Assert.assertTrue("Should have expected number of records", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra records", rows.hasNext()); + assertThat(rows).isExhausted(); } - File rowDataFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", rowDataFile.delete()); + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); // Write the expected RowData into ORC file, then read them into Record and assert with the // expected RowData list. @@ -97,10 +98,10 @@ protected void writeAndValidate(Schema schema) throws IOException { Iterator expected = expectedRows.iterator(); Iterator records = reader.iterator(); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of records", records.hasNext()); + assertThat(records.hasNext()).isTrue(); TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); } - Assert.assertFalse("Should not have extra records", records.hasNext()); + assertThat(records).isExhausted(); } } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java index 30a2a7bb51ce..1fdc4cf8381a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -47,8 +48,7 @@ import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkParquetReader extends DataTest { private static final int NUM_RECORDS = 100; @@ -61,8 +61,8 @@ public void testTwoLevelList() throws IOException { optional(2, "topbytes", Types.BinaryType.get())); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - File testFile = temp.newFile(); - Assert.assertTrue(testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); ParquetWriter writer = AvroParquetWriter.builder(new Path(testFile.toURI())) @@ -90,17 +90,17 @@ public void testTwoLevelList() throws IOException { .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) .build()) { Iterator rows = reader.iterator(); - Assert.assertTrue("Should have at least one row", rows.hasNext()); + assertThat(rows).hasNext(); RowData rowData = rows.next(); - Assert.assertArrayEquals(rowData.getArray(0).getBinary(0), expectedByte); - Assert.assertArrayEquals(rowData.getBinary(1), expectedByte); - Assert.assertFalse("Should not have more than one row", rows.hasNext()); + assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); + assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); + assertThat(rows).isExhausted(); } } private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); try (FileAppender writer = Parquet.write(Files.localOutput(testFile)) @@ -119,10 +119,10 @@ private void writeAndValidate(Iterable iterable, Schema schema) throws I Iterator rows = reader.iterator(); LogicalType rowType = FlinkSchemaUtil.convert(schema); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", rows.hasNext()); + assertThat(rows).hasNext(); TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); } - Assert.assertFalse("Should not have extra rows", rows.hasNext()); + assertThat(rows).isExhausted(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java index 7b868eafc311..b1e6f5aa00ff 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.data; +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 org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.LogicalType; @@ -34,18 +37,16 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkParquetWriter extends DataTest { private static final int NUM_RECORDS = 100; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); LogicalType logicalType = FlinkSchemaUtil.convert(schema); @@ -66,10 +67,10 @@ private void writeAndValidate(Iterable iterable, Schema schema) throws Iterator actual = reader.iterator(); LogicalType rowType = FlinkSchemaUtil.convert(schema); for (int i = 0; i < NUM_RECORDS; i += 1) { - Assert.assertTrue("Should have expected number of rows", actual.hasNext()); + assertThat(actual).hasNext(); TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); } - Assert.assertFalse("Should not have extra rows", actual.hasNext()); + assertThat(actual).isExhausted(); } } 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 df2e6ae21c7e..7dd4e8759c0e 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 @@ -18,8 +18,12 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.withPrecision; + import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; @@ -37,19 +41,17 @@ 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.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestRowProjection { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { - File file = temp.newFile(desc + ".avro"); - Assert.assertTrue(file.delete()); + File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); + assertThat(file.delete()).isTrue(); try (FileAppender appender = Avro.write(Files.localOutput(file)) @@ -79,10 +81,8 @@ public void testFullProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, schema, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data value", cmp, 0); + assertThat(projected.getLong(0)).isEqualTo(34); + assertThat(projected.getString(1)).asString().isEqualTo("test"); } @Test @@ -96,19 +96,13 @@ public void testSpecialCharacterProjection() throws Exception { RowData full = writeAndRead("special_chars", schema, schema, row); - Assert.assertEquals("Should contain the correct id value", 34L, full.getLong(0)); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", full.getString(1).toString())); + assertThat(full.getLong(0)).isEqualTo(34L); + assertThat(full.getString(1)).asString().isEqualTo("test"); RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); - Assert.assertEquals("Should not contain id value", 1, projected.getArity()); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", projected.getString(0).toString())); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getString(0)).asString().isEqualTo("test"); } @Test @@ -127,9 +121,8 @@ public void testReorderedFullProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, reordered, row); - Assert.assertEquals( - "Should contain the correct 0 value", "test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.getLong(1)); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + assertThat(projected.getLong(1)).isEqualTo(34); } @Test @@ -149,10 +142,9 @@ public void testReorderedProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, reordered, row); - Assert.assertTrue("Should contain the correct 0 value", projected.isNullAt(0)); - Assert.assertEquals( - "Should contain the correct 1 value", "test", projected.getString(1).toString()); - Assert.assertTrue("Should contain the correct 2 value", projected.isNullAt(2)); + assertThat(projected.isNullAt(0)).isTrue(); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + assertThat(projected.isNullAt(2)).isTrue(); } @Test @@ -173,10 +165,16 @@ public void testRenamedAddedField() throws Exception { Types.NestedField.optional(4, "d", Types.LongType.get())); RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertEquals("Should contain the correct value in column 2", projected.getLong(1), 200L); - Assert.assertEquals("Should contain the correct value in column 3", projected.getLong(2), 300L); - Assert.assertTrue("Should contain empty value on new column 4", projected.isNullAt(3)); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.getLong(1)) + .as("Should contain the correct value in column 2") + .isEqualTo(200L); + assertThat(projected.getLong(2)) + .as("Should contain the correct value in column 1") + .isEqualTo(300L); + assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } @Test @@ -190,8 +188,8 @@ public void testEmptyProjection() throws Exception { RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); - Assert.assertNotNull("Should read a non-null record", projected); - Assert.assertEquals(0, projected.getArity()); + assertThat(projected).isNotNull(); + assertThat(projected.getArity()).isEqualTo(0); } @Test @@ -206,16 +204,16 @@ public void testBasicProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); - Assert.assertEquals("Should not project data", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + assertThat(projected.getArity()).as("Should not project data").isEqualTo(1); + assertThat(projected.getLong(0)).isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(projected.getString(0)).asString().isEqualTo("test"); } @Test @@ -234,9 +232,11 @@ public void testRename() throws Exception { RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getString(1)) + .as("Should contain the correct data/renamed value") + .asString() + .isEqualTo("test"); } @Test @@ -257,8 +257,8 @@ public void testNestedStructProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should not project location", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getLong(0)).as("Should contain the correct id value").isEqualTo(34L); Schema latOnly = new Schema( @@ -269,11 +269,12 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); RowData projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project longitude", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project longitude").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); Schema longOnly = new Schema( @@ -284,21 +285,24 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project latitutde", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(0), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project latitutde").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(1), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedLocation.getFloat(1)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); } @Test @@ -324,23 +328,23 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project properties map", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project properties map").isEqualTo(1); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); } private Map toStringMap(Map map) { @@ -381,42 +385,50 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project locations map", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project locations map").isEqualTo(1); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project locations map", row.getMap(1), projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(row.getMap(1)); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); GenericMapData locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); + assertThat(locations).isNotNull(); GenericArrayData l1l2Array = new GenericArrayData( new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should contain lat", 53.992811f, projectedL1.getFloat(0), 0.000001); - Assert.assertEquals("L1 should not contain long", 1, projectedL1.getArity()); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain lat") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + assertThat(projectedL1.getArity()).as("L1 should not contain long").isEqualTo(1); RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should contain lat", 52.995143f, projectedL2.getFloat(0), 0.000001); - Assert.assertEquals("L2 should not contain long", 1, projectedL2.getArity()); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain lat") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedL2.getArity()).as("L2 should not contain long").isEqualTo(1); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should not contain lat", 1, projectedL1.getArity()); - Assert.assertEquals("L1 should contain long", -1.542616f, projectedL1.getFloat(0), 0.000001); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getArity()).as("L1 should not contain lat").isEqualTo(1); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain long") + .isEqualTo(-1.542616f, withPrecision(0.000001f)); projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should not contain lat", 1, projectedL2.getArity()); - Assert.assertEquals("L2 should contain long", -1.539054f, projectedL2.getFloat(0), 0.000001); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getArity()).as("L2 should not contain lat").isEqualTo(1); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain long") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -431,18 +443,20 @@ public void testMapOfStructsProjection() throws IOException { Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", 53.992811f, projectedL1.getFloat(0), 0.000001); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain latitude") + .isEqualTo(53.992811f, withPrecision(0.000001f)); projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", 52.995143f, projectedL2.getFloat(0), 0.000001); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); } @Test @@ -460,18 +474,18 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project values list", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project values list").isEqualTo(1); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); } @Test @@ -497,36 +511,36 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project points list", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).isEqualTo(1); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project points list", row.getArray(1), projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(row.getArray(1)); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); ArrayData points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); RowData projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should project x", 1, projectedP1.getInt(0)); - Assert.assertEquals("Should not project y", 1, projectedP1.getArity()); + assertThat(projectedP1.getInt(0)).as("Should project x").isEqualTo(1); + assertThat(projectedP1.getArity()).as("Should not project y").isEqualTo(1); RowData projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project y", 1, projectedP2.getArity()); - Assert.assertEquals("Should project x", 3, projectedP2.getInt(0)); + assertThat(projectedP2.getArity()).as("Should not project y").isEqualTo(1); + assertThat(projectedP2.getInt(0)).as("Should project x").isEqualTo(3); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x", 1, projectedP1.getArity()); - Assert.assertEquals("Should project y", 2, projectedP1.getInt(0)); + assertThat(projectedP1.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project y").isEqualTo(2); projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null y", projectedP2.isNullAt(0)); + assertThat(projectedP2.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null y").isTrue(); Schema yRenamed = new Schema( @@ -539,16 +553,16 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP1.getArity()); - Assert.assertEquals("Should project z", 2, projectedP1.getInt(0)); + assertThat(projectedP1.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project z").isEqualTo(2); projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null z", projectedP2.isNullAt(0)); + assertThat(projectedP2.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } @Test @@ -572,9 +586,11 @@ public void testAddedFieldsWithRequiredChildren() throws Exception { RowData projected = writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertTrue("Should contain empty value in new column 2", projected.isNullAt(1)); - Assert.assertTrue("Should contain empty value in new column 4", projected.isNullAt(2)); - Assert.assertTrue("Should contain empty value in new column 6", projected.isNullAt(3)); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.isNullAt(1)).as("Should contain empty value in new column 2").isTrue(); + assertThat(projected.isNullAt(2)).as("Should contain empty value in new column 4").isTrue(); + assertThat(projected.isNullAt(3)).as("Should contain empty value in new column 6").isTrue(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java index e0340e0743b0..eccab20e04fc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java @@ -22,7 +22,7 @@ import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; import org.apache.iceberg.flink.TestHelpers; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStructRowData { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java index 6a493692c20d..44eb907a17aa 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.flink.table.data.RowData; import org.apache.iceberg.flink.AvroGenericRecordConverterBase; import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { @Override @@ -32,6 +33,6 @@ protected void testConverter(DataGenerator dataGenerator) throws Exception { AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); RowData expected = dataGenerator.generateFlinkRowData(); RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java index 485035787d6d..6ef40693827e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.avro.generic.GenericRecord; import org.apache.iceberg.flink.AvroGenericRecordConverterBase; import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { @Override @@ -30,6 +31,6 @@ protected void testConverter(DataGenerator dataGenerator) { RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); GenericRecord expected = dataGenerator.generateAvroGenericRecord(); GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index 6cef0d122857..08bbc4fc802b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.flink.util; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -28,7 +29,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - Assert.assertEquals("1.18.1", FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo("1.18.1"); } @Test @@ -41,14 +42,14 @@ public void testDefaultVersion() { try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class); mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); - Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); } FlinkPackage.setVersion(null); try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null); mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); FlinkPackage.setVersion(null); - Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); } } } From bf6d3c54a014300d660cb7fa36d19d752ef27654 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 15 Apr 2024 19:15:37 +0800 Subject: [PATCH 0221/1019] Core: Fix logging table name when scanning metadata table (#10141) --- .../java/org/apache/iceberg/BaseAllMetadataTableScan.java | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java b/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java index f7f05e61158e..bd06f80f873e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseAllMetadataTableScan.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Locale; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.ExpressionUtil; @@ -60,11 +61,12 @@ public TableScan asOfTime(long timestampMillis) { @Override public CloseableIterable planFiles() { + String metadataTableName = table().name() + "." + tableType().name().toLowerCase(Locale.ROOT); LOG.info( "Scanning metadata table {} with filter {}.", - table(), + metadataTableName, ExpressionUtil.toSanitizedString(filter())); - Listeners.notifyAll(new ScanEvent(table().name(), 0L, filter(), schema())); + Listeners.notifyAll(new ScanEvent(metadataTableName, 0L, filter(), schema())); return doPlanFiles(); } From 71ca10dd8c900118d1b4ce9dc7f2b50bc2d12a4b Mon Sep 17 00:00:00 2001 From: Filipe Regadas Date: Mon, 15 Apr 2024 18:06:10 +0100 Subject: [PATCH 0222/1019] AWS: Close underlying scheduler for DynamoDbLockManager (#10132) --- .../org/apache/iceberg/aws/dynamodb/DynamoDbLockManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbLockManager.java b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbLockManager.java index 252ce10a7ccb..80bbfcbee798 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbLockManager.java +++ b/aws/src/main/java/org/apache/iceberg/aws/dynamodb/DynamoDbLockManager.java @@ -312,10 +312,12 @@ private static Map toLockIdValues(String entityId, Strin } @Override - public void close() { + public void close() throws Exception { dynamo.close(); heartbeats.values().forEach(DynamoDbHeartbeat::cancel); heartbeats.clear(); + + super.close(); } /** From a6db52799664e64a781329374e8152c1a5de4f8c Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 16 Apr 2024 05:06:23 +0800 Subject: [PATCH 0223/1019] Spark 3.5: Add threshold for failed commits in data rewrites (#9611) --- .../iceberg/actions/RewriteDataFiles.java | 7 +++ .../iceberg/actions/BaseCommitService.java | 6 +++ .../actions/RewriteDataFilesSparkAction.java | 36 ++++++++++---- .../actions/TestRewriteDataFilesAction.java | 48 +++++++++++++++++-- 4 files changed, 86 insertions(+), 11 deletions(-) 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 40dc24318ccf..f6ef40270852 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteDataFiles.java @@ -52,6 +52,13 @@ public interface RewriteDataFiles int PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT = 10; + /** + * The maximum amount of failed commits that this rewrite is allowed if partial progress is + * enabled. By default, all commits are allowed to fail. This setting has no effect if partial + * progress is disabled. + */ + String PARTIAL_PROGRESS_MAX_FAILED_COMMITS = "partial-progress.max-failed-commits"; + /** * The entire rewrite operation is broken down into pieces based on partitioning and within * partitions based on size into groups. These sub-units of the rewrite are referred to as file diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java b/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java index d8e5b753cbd2..d45bbfb4a667 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseCommitService.java @@ -60,6 +60,7 @@ abstract class BaseCommitService implements Closeable { private final int rewritesPerCommit; private final AtomicBoolean running = new AtomicBoolean(false); private final long timeoutInMS; + private int succeededCommits = 0; /** * Constructs a {@link BaseCommitService} @@ -227,6 +228,7 @@ private void commitReadyCommitGroups() { try { commitOrClean(batch); committedRewrites.addAll(batch); + succeededCommits++; } catch (Exception e) { LOG.error("Failure during rewrite commit process, partial progress enabled. Ignoring", e); } @@ -234,6 +236,10 @@ private void commitReadyCommitGroups() { } } + public int succeededCommits() { + return succeededCommits; + } + @VisibleForTesting boolean canCreateCommitGroup() { // Either we have a full commit group, or we have completed writing and need to commit 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 bf1a901dbd98..e301ad512c70 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 @@ -79,6 +79,7 @@ public class RewriteDataFilesSparkAction MAX_FILE_GROUP_SIZE_BYTES, PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_MAX_COMMITS, + PARTIAL_PROGRESS_MAX_FAILED_COMMITS, TARGET_FILE_SIZE_BYTES, USE_STARTING_SEQUENCE_NUMBER, REWRITE_JOB_ORDER, @@ -92,6 +93,7 @@ public class RewriteDataFilesSparkAction private Expression filter = Expressions.alwaysTrue(); private int maxConcurrentFileGroupRewrites; private int maxCommits; + private int maxFailedCommits; private boolean partialProgressEnabled; private boolean useStartingSequenceNumber; private RewriteJobOrder rewriteJobOrder; @@ -359,20 +361,31 @@ private Result doExecuteWithPartialProgress( // stop commit service commitService.close(); - List commitResults = commitService.results(); - if (commitResults.size() == 0) { - LOG.error( - "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " - + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " + + int failedCommits = maxCommits - commitService.succeededCommits(); + if (failedCommits > 0 && failedCommits <= maxFailedCommits) { + LOG.warn( + "{} is true but {} rewrite commits failed. Check the logs to determine why the individual " + + "commits failed. If this is persistent it may help to increase {} which will split the rewrite operation " + "into smaller commits.", PARTIAL_PROGRESS_ENABLED, + failedCommits, PARTIAL_PROGRESS_MAX_COMMITS); + } else if (failedCommits > maxFailedCommits) { + String errorMessage = + String.format( + "%s is true but %d rewrite commits failed. This is more than the maximum allowed failures of %d. " + + "Check the logs to determine why the individual commits failed. If this is persistent it may help to " + + "increase %s which will split the rewrite operation into smaller commits.", + PARTIAL_PROGRESS_ENABLED, + failedCommits, + maxFailedCommits, + PARTIAL_PROGRESS_MAX_COMMITS); + throw new RuntimeException(errorMessage); } - List rewriteResults = - commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); return ImmutableRewriteDataFiles.Result.builder() - .rewriteResults(rewriteResults) + .rewriteResults(toRewriteResults(commitService.results())) .rewriteFailures(rewriteFailures) .build(); } @@ -403,6 +416,10 @@ private RewriteFileGroup newRewriteGroup( return new RewriteFileGroup(info, tasks); } + private Iterable toRewriteResults(List commitResults) { + return commitResults.stream().map(RewriteFileGroup::asResult).collect(Collectors.toList()); + } + void validateAndInitOptions() { Set validOptions = Sets.newHashSet(rewriter.validOptions()); validOptions.addAll(VALID_OPTIONS); @@ -428,6 +445,9 @@ void validateAndInitOptions() { PropertyUtil.propertyAsInt( options(), PARTIAL_PROGRESS_MAX_COMMITS, PARTIAL_PROGRESS_MAX_COMMITS_DEFAULT); + maxFailedCommits = + PropertyUtil.propertyAsInt(options(), PARTIAL_PROGRESS_MAX_FAILED_COMMITS, maxCommits); + partialProgressEnabled = PropertyUtil.propertyAsBoolean( options(), PARTIAL_PROGRESS_ENABLED, PARTIAL_PROGRESS_ENABLED_DEFAULT); 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 500092c044b1..776a8890b5eb 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 @@ -857,8 +857,8 @@ public void testParallelPartialProgressWithRewriteFailure() { List postRewriteData = currentData(); assertEquals("We shouldn't have changed the data", originalData, postRewriteData); - // With 10 original groups and Max Commits of 3, we should have commits with 4, 4, and 2. - // removing 3 groups leaves us with only 2 new commits, 4 and 3 + // With 10 original groups and max commits of 3, we have 4 groups per commit. + // Removing 3 groups, we are left with 4 groups and 3 groups in two commits. shouldHaveSnapshots(table, 3); shouldHaveNoOrphans(table); shouldHaveACleanCache(table); @@ -894,7 +894,7 @@ public void testParallelPartialProgressWithCommitFailure() { RewriteDataFiles.Result result = spyRewrite.execute(); - // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups comitted + // Commit 1: 4/4 + Commit 2 failed 0/4 + Commit 3: 2/2 == 6 out of 10 total groups committed assertThat(result.rewriteResults()).as("Should have 6 fileGroups").hasSize(6); assertThat(result.rewrittenBytesCount()).isGreaterThan(0L).isLessThan(dataSizeBefore); @@ -909,6 +909,48 @@ public void testParallelPartialProgressWithCommitFailure() { shouldHaveACleanCache(table); } + @Test + public void testParallelPartialProgressWithMaxFailedCommits() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "3") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_FAILED_COMMITS, "0"); + + RewriteDataFilesSparkAction spyRewrite = Mockito.spy(realRewrite); + + // Fail groups 1, 3, and 7 during rewrite + GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); + doThrow(new RuntimeException("Rewrite Failed")) + .when(spyRewrite) + .rewriteFiles(any(), argThat(failGroup)); + + assertThatThrownBy(() -> spyRewrite.execute()) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining( + "1 rewrite commits failed. This is more than the maximum allowed failures of 0"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + // With 10 original groups and max commits of 3, we have 4 groups per commit. + // Removing 3 groups, we are left with 4 groups and 3 groups in two commits. + // Adding max allowed failed commits doesn't change the number of successful commits. + shouldHaveSnapshots(table, 3); + shouldHaveNoOrphans(table); + shouldHaveACleanCache(table); + } + @Test public void testInvalidOptions() { Table table = createTable(20); From 1c8f98faed444266758b1c632195ceb773ba0b50 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 16 Apr 2024 07:42:50 +0200 Subject: [PATCH 0224/1019] Core: Fix JDBC Catalog table commit when migrating from schema V0 to V1 (#10111) --- .../org/apache/iceberg/jdbc/JdbcUtil.java | 38 +++++-- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 62 +++++++++++- .../org/apache/iceberg/jdbc/TestJdbcUtil.java | 98 +++++++++++++++++++ 3 files changed, 190 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 077c33321be4..8f918b456088 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -54,7 +54,31 @@ enum SchemaVersion { static final String TABLE_RECORD_TYPE = "TABLE"; static final String VIEW_RECORD_TYPE = "VIEW"; - private static final String V1_DO_COMMIT_SQL = + private static final String V1_DO_COMMIT_TABLE_SQL = + "UPDATE " + + CATALOG_TABLE_VIEW_NAME + + " SET " + + JdbcTableOperations.METADATA_LOCATION_PROP + + " = ? , " + + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP + + " = ?" + + " WHERE " + + CATALOG_NAME + + " = ? AND " + + TABLE_NAMESPACE + + " = ? AND " + + TABLE_NAME + + " = ? AND " + + JdbcTableOperations.METADATA_LOCATION_PROP + + " = ? AND (" + + RECORD_TYPE + + " = '" + + TABLE_RECORD_TYPE + + "'" + + " OR " + + RECORD_TYPE + + " IS NULL)"; + private static final String V1_DO_COMMIT_VIEW_SQL = "UPDATE " + CATALOG_TABLE_VIEW_NAME + " SET " @@ -72,7 +96,10 @@ enum SchemaVersion { + JdbcTableOperations.METADATA_LOCATION_PROP + " = ? AND " + RECORD_TYPE - + " = ?"; + + " = " + + "'" + + VIEW_RECORD_TYPE + + "'"; private static final String V0_DO_COMMIT_SQL = "UPDATE " + CATALOG_TABLE_VIEW_NAME @@ -504,7 +531,9 @@ private static int update( conn -> { try (PreparedStatement sql = conn.prepareStatement( - (schemaVersion == SchemaVersion.V1) ? V1_DO_COMMIT_SQL : V0_DO_COMMIT_SQL)) { + (schemaVersion == SchemaVersion.V1) + ? (isTable ? V1_DO_COMMIT_TABLE_SQL : V1_DO_COMMIT_VIEW_SQL) + : V0_DO_COMMIT_SQL)) { // UPDATE sql.setString(1, newMetadataLocation); sql.setString(2, oldMetadataLocation); @@ -513,9 +542,6 @@ private static int update( sql.setString(4, namespaceToString(identifier.namespace())); sql.setString(5, identifier.name()); sql.setString(6, oldMetadataLocation); - if (schemaVersion == SchemaVersion.V1) { - sql.setString(7, isTable ? TABLE_RECORD_TYPE : VIEW_RECORD_TYPE); - } return sql.executeUpdate(); } 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 d8553a1858f5..c2f0869d0029 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -31,6 +31,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.sql.Connection; +import java.sql.ResultSet; import java.sql.SQLException; import java.util.List; import java.util.Map; @@ -239,6 +240,17 @@ public void testSchemaIsMigratedToAddViewSupport() throws Exception { .create(); assertThat(jdbcCatalog.listViews(Namespace.of("namespace1"))).hasSize(1).containsExactly(view); + + TableIdentifier tableThree = TableIdentifier.of("namespace2", "table3"); + jdbcCatalog.createTable(tableThree, SCHEMA); + assertThat(jdbcCatalog.tableExists(tableThree)).isTrue(); + + // testing append datafile to check commit, it should not throw an exception + jdbcCatalog.loadTable(tableOne).newAppend().appendFile(FILE_A).commit(); + jdbcCatalog.loadTable(tableTwo).newAppend().appendFile(FILE_B).commit(); + + assertThat(jdbcCatalog.tableExists(tableOne)).isTrue(); + assertThat(jdbcCatalog.tableExists(tableTwo)).isTrue(); } @ParameterizedTest @@ -1024,7 +1036,49 @@ public void report(MetricsReport report) { } } + private String createMetadataLocationViaJdbcCatalog(TableIdentifier identifier) + throws SQLException { + // temporary connection just to actually create a concrete metadata location + String jdbcUrl = null; + try { + java.nio.file.Path dbFile = Files.createTempFile("temp", "metadata"); + jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); + } catch (IOException e) { + throw new SQLException("Error while creating temp data", e); + } + + Map properties = Maps.newHashMap(); + + properties.put(CatalogProperties.URI, jdbcUrl); + + warehouseLocation = this.tableDir.toAbsolutePath().toString(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + properties.put("type", "jdbc"); + + JdbcCatalog jdbcCatalog = + (JdbcCatalog) CatalogUtil.buildIcebergCatalog("TEMP", properties, conf); + jdbcCatalog.buildTable(identifier, SCHEMA).create(); + + SQLiteDataSource dataSource = new SQLiteDataSource(); + dataSource.setUrl(jdbcUrl); + + try (Connection connection = dataSource.getConnection()) { + ResultSet result = + connection + .prepareStatement("SELECT * FROM " + JdbcUtil.CATALOG_TABLE_VIEW_NAME) + .executeQuery(); + result.next(); + return result.getString(JdbcTableOperations.METADATA_LOCATION_PROP); + } + } + private void initLegacySchema(String jdbcUrl) throws SQLException { + TableIdentifier table1 = TableIdentifier.of(Namespace.of("namespace1"), "table1"); + TableIdentifier table2 = TableIdentifier.of(Namespace.of("namespace2"), "table2"); + + String table1MetadataLocation = createMetadataLocationViaJdbcCatalog(table1); + String table2MetadataLocation = createMetadataLocationViaJdbcCatalog(table2); + SQLiteDataSource dataSource = new SQLiteDataSource(); dataSource.setUrl(jdbcUrl); @@ -1045,7 +1099,9 @@ private void initLegacySchema(String jdbcUrl) throws SQLException { + JdbcTableOperations.METADATA_LOCATION_PROP + "," + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP - + ") VALUES('TEST','namespace1','table1',null,null)") + + ") VALUES('TEST','namespace1','table1','" + + table1MetadataLocation + + "',null)") .execute(); connection .prepareStatement( @@ -1061,7 +1117,9 @@ private void initLegacySchema(String jdbcUrl) throws SQLException { + JdbcTableOperations.METADATA_LOCATION_PROP + "," + JdbcTableOperations.PREVIOUS_METADATA_LOCATION_PROP - + ") VALUES('TEST','namespace2','table2',null,null)") + + ") VALUES('TEST','namespace2','table2','" + + table2MetadataLocation + + "',null)") .execute(); } } 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 7dde37d4b510..cfea9740ebd9 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcUtil.java @@ -20,10 +20,16 @@ import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Files; +import java.sql.PreparedStatement; +import java.sql.ResultSet; import java.util.Map; import java.util.Properties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.junit.jupiter.api.Test; +import org.sqlite.SQLiteDataSource; public class TestJdbcUtil { @@ -45,4 +51,96 @@ public void testFilterAndRemovePrefix() { assertThat(expected).isEqualTo(actual); } + + @Test + public void testV0toV1SqlStatements() throws Exception { + java.nio.file.Path dbFile = Files.createTempFile("icebergSchemaUpdate", "db"); + String jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); + + SQLiteDataSource dataSource = new SQLiteDataSource(); + dataSource.setUrl(jdbcUrl); + + try (JdbcClientPool connections = new JdbcClientPool(jdbcUrl, Maps.newHashMap())) { + // create "old style" SQL schema + connections.newClient().prepareStatement(JdbcUtil.V0_CREATE_CATALOG_SQL).executeUpdate(); + + // inserting tables + JdbcUtil.doCommitCreateTable( + JdbcUtil.SchemaVersion.V0, + connections, + "TEST", + Namespace.of("namespace1"), + TableIdentifier.of(Namespace.of("namespace1"), "table1"), + "testLocation"); + JdbcUtil.doCommitCreateTable( + JdbcUtil.SchemaVersion.V0, + connections, + "TEST", + Namespace.of("namespace1"), + TableIdentifier.of(Namespace.of("namespace1"), "table2"), + "testLocation"); + + try (PreparedStatement statement = + connections.newClient().prepareStatement(JdbcUtil.V0_LIST_TABLE_SQL)) { + statement.setString(1, "TEST"); + statement.setString(2, "namespace1"); + ResultSet tables = statement.executeQuery(); + tables.next(); + assertThat(tables.getString(JdbcUtil.TABLE_NAME)).isEqualTo("table1"); + tables.next(); + assertThat(tables.getString(JdbcUtil.TABLE_NAME)).isEqualTo("table2"); + } + + // updating the schema from V0 to V1 + connections.newClient().prepareStatement(JdbcUtil.V1_UPDATE_CATALOG_SQL).execute(); + + // trying to add a table on the updated schema + JdbcUtil.doCommitCreateTable( + JdbcUtil.SchemaVersion.V1, + connections, + "TEST", + Namespace.of("namespace1"), + TableIdentifier.of(Namespace.of("namespace1"), "table3"), + "testLocation"); + + // testing the tables after migration and new table added + try (PreparedStatement statement = + connections.newClient().prepareStatement(JdbcUtil.V0_LIST_TABLE_SQL)) { + statement.setString(1, "TEST"); + statement.setString(2, "namespace1"); + ResultSet tables = statement.executeQuery(); + tables.next(); + assertThat(tables.getString(JdbcUtil.TABLE_NAME)).isEqualTo("table1"); + assertThat(tables.getString(JdbcUtil.RECORD_TYPE)).isNull(); + tables.next(); + assertThat(tables.getString(JdbcUtil.TABLE_NAME)).isEqualTo("table2"); + assertThat(tables.getString(JdbcUtil.RECORD_TYPE)).isNull(); + tables.next(); + assertThat(tables.getString(JdbcUtil.TABLE_NAME)).isEqualTo("table3"); + assertThat(tables.getString(JdbcUtil.RECORD_TYPE)).isEqualTo(JdbcUtil.TABLE_RECORD_TYPE); + } + + // update a table (commit) created on V1 schema + int updated = + JdbcUtil.updateTable( + JdbcUtil.SchemaVersion.V1, + connections, + "TEST", + TableIdentifier.of(Namespace.of("namespace1"), "table3"), + "newLocation", + "testLocation"); + assertThat(updated).isEqualTo(1); + + // update a table (commit) migrated from V0 schema + updated = + JdbcUtil.updateTable( + JdbcUtil.SchemaVersion.V1, + connections, + "TEST", + TableIdentifier.of(Namespace.of("namespace1"), "table1"), + "newLocation", + "testLocation"); + assertThat(updated).isEqualTo(1); + } + } } From 7752ab1956174bece924862b5e0f9ba258dc3e4a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 16 Apr 2024 09:56:33 +0200 Subject: [PATCH 0225/1019] Core: Use 'delete' if RowDelta only has delete files (#10123) --- .../java/org/apache/iceberg/BaseRowDelta.java | 4 +++ .../apache/iceberg/TestCommitReporting.java | 2 +- .../java/org/apache/iceberg/TestRowDelta.java | 12 +++++++++ .../SparkRowLevelOperationsTestBase.java | 14 +++++++--- .../iceberg/spark/extensions/TestDelete.java | 27 +++++++++++++++++++ .../source/TestStructuredStreamingRead3.java | 13 ++++++++- .../SparkRowLevelOperationsTestBase.java | 13 ++++++--- .../iceberg/spark/extensions/TestDelete.java | 27 +++++++++++++++++++ .../source/TestStructuredStreamingRead3.java | 13 ++++++++- .../SparkRowLevelOperationsTestBase.java | 14 ++++++---- .../iceberg/spark/extensions/TestDelete.java | 26 ++++++++++++++++++ .../source/TestStructuredStreamingRead3.java | 12 ++++++++- 12 files changed, 160 insertions(+), 17 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index a8e60045ac4f..42fd17f0320b 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -43,6 +43,10 @@ protected BaseRowDelta self() { @Override protected String operation() { + if (addsDeleteFiles() && !addsDataFiles()) { + return DataOperations.DELETE; + } + return DataOperations.OVERWRITE; } diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index 48b9c8d72bd2..bbba192fab7f 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -102,7 +102,7 @@ public void addAndDeleteDeleteFiles() { CommitReport report = reporter.lastCommitReport(); assertThat(report).isNotNull(); - assertThat(report.operation()).isEqualTo("overwrite"); + assertThat(report.operation()).isEqualTo("delete"); assertThat(report.snapshotId()).isEqualTo(1L); assertThat(report.sequenceNumber()).isEqualTo(1L); assertThat(report.tableName()).isEqualTo(tableName); diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index 867e4b062f4d..a2a043e630bb 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -55,6 +55,18 @@ protected static List parameters() { return Arrays.asList(new Object[] {2, "main"}, new Object[] {2, "testBranch"}); } + @TestTemplate + public void addOnlyDeleteFilesProducesDeleteOperation() { + SnapshotUpdate rowDelta = + table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_B_DELETES); + + commit(table, rowDelta, branch); + Snapshot snap = latestSnapshot(table, branch); + assertThat(snap.sequenceNumber()).isEqualTo(1); + assertThat(snap.operation()).isEqualTo(DataOperations.DELETE); + assertThat(snap.deleteManifests(table.io())).hasSize(1); + } + @TestTemplate public void testAddDeleteFile() { SnapshotUpdate rowDelta = diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 9a1257ba39bf..7398e3830020 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -30,6 +30,7 @@ import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_HASH; import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_NONE; import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE_RANGE; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.io.UncheckedIOException; @@ -255,8 +256,9 @@ protected void validateMergeOnRead( String changedPartitionCount, String addedDeleteFiles, String addedDataFiles) { + String operation = null == addedDataFiles && null != addedDeleteFiles ? DELETE : OVERWRITE; validateSnapshot( - snapshot, OVERWRITE, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); + snapshot, operation, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); } protected void validateSnapshot( @@ -286,9 +288,13 @@ protected void validateProperty(Snapshot snapshot, String property, Set } protected void validateProperty(Snapshot snapshot, String property, String expectedValue) { - String actual = snapshot.summary().get(property); - Assert.assertEquals( - "Snapshot property " + property + " has unexpected value.", expectedValue, actual); + if (null == expectedValue) { + assertThat(snapshot.summary()).doesNotContainKey(property); + } else { + assertThat(snapshot.summary()) + .as("Snapshot property " + property + " has unexpected value.") + .containsEntry(property, expectedValue); + } } protected void sleep(long millis) { diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 6020e00b3235..cdc508f9859d 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -19,12 +19,14 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_MODE; import static org.apache.iceberg.TableProperties.DELETE_MODE_DEFAULT; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; @@ -334,6 +336,31 @@ public void testDeleteNonExistingRecords() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } + @Test + public void deleteSingleRecordProducesDeleteOperation() throws NoSuchTableException { + createAndInitPartitionedTable(); + append(tableName, new Employee(1, "eng"), new Employee(2, "eng"), new Employee(3, "eng")); + + sql("DELETE FROM %s WHERE id = 2", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(2); + + Snapshot currentSnapshot = table.currentSnapshot(); + + if (mode(table) == COPY_ON_WRITE) { + // this is an OverwriteFiles and produces "overwrite" + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } 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"); + } + + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactlyInAnyOrder(row(1, "eng"), row(3, "eng")); + } + @Test public void testDeleteWithoutCondition() { createAndInitPartitionedTable(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index a2d0c9acaf48..c706603d06c7 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -31,8 +31,11 @@ import java.util.stream.IntStream; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.DataOperations; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -476,7 +479,15 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception dataDeletes, deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(temp.newFile().toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + + table.newRowDelta().addRows(dataFile).addDeletes(eqDeletes).commit(); // check pre-condition - that the above Delete file write - actually resulted in snapshot of // type OVERWRITE diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index e4605a765823..463cf2a47d8b 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -292,8 +292,9 @@ protected void validateMergeOnRead( String changedPartitionCount, String addedDeleteFiles, String addedDataFiles) { + String operation = null == addedDataFiles && null != addedDeleteFiles ? DELETE : OVERWRITE; validateSnapshot( - snapshot, OVERWRITE, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); + snapshot, operation, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); } protected void validateSnapshot( @@ -323,9 +324,13 @@ protected void validateProperty(Snapshot snapshot, String property, Set } protected void validateProperty(Snapshot snapshot, String property, String expectedValue) { - String actual = snapshot.summary().get(property); - Assert.assertEquals( - "Snapshot property " + property + " has unexpected value.", expectedValue, actual); + if (null == expectedValue) { + assertThat(snapshot.summary()).doesNotContainKey(property); + } else { + assertThat(snapshot.summary()) + .as("Snapshot property " + property + " has unexpected value.") + .containsEntry(property, expectedValue); + } } protected void sleep(long millis) { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index b901d567f646..731dedbd48d2 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.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.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; 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; @@ -27,6 +28,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_OPEN_FILE_COST; import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; @@ -502,6 +504,31 @@ public void testDeleteNonExistingRecords() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } + @Test + public void deleteSingleRecordProducesDeleteOperation() throws NoSuchTableException { + createAndInitPartitionedTable(); + append(tableName, new Employee(1, "eng"), new Employee(2, "eng"), new Employee(3, "eng")); + + sql("DELETE FROM %s WHERE id = 2", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(2); + + Snapshot currentSnapshot = table.currentSnapshot(); + + if (mode(table) == COPY_ON_WRITE) { + // this is an OverwriteFiles and produces "overwrite" + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } 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"); + } + + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactlyInAnyOrder(row(1, "eng"), row(3, "eng")); + } + @Test public void testDeleteWithoutCondition() { createAndInitPartitionedTable(); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 1c08744f5662..c1bbf304fac4 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -30,8 +30,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.DataOperations; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -483,7 +486,15 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception dataDeletes, deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(temp.newFile().toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + + table.newRowDelta().addRows(dataFile).addDeletes(eqDeletes).commit(); // check pre-condition - that the above Delete file write - actually resulted in snapshot of // type OVERWRITE 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 ea1040dcf0ac..a926388e4aa8 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 @@ -285,8 +285,9 @@ protected void validateMergeOnRead( String changedPartitionCount, String addedDeleteFiles, String addedDataFiles) { + String operation = null == addedDataFiles && null != addedDeleteFiles ? DELETE : OVERWRITE; validateSnapshot( - snapshot, OVERWRITE, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); + snapshot, operation, changedPartitionCount, null, addedDeleteFiles, addedDataFiles); } protected void validateSnapshot( @@ -317,10 +318,13 @@ protected void validateProperty(Snapshot snapshot, String property, Set } protected void validateProperty(Snapshot snapshot, String property, String expectedValue) { - String actual = snapshot.summary().get(property); - assertThat(actual) - .as("Snapshot property " + property + " has unexpected value.") - .isEqualTo(expectedValue); + if (null == expectedValue) { + assertThat(snapshot.summary()).doesNotContainKey(property); + } else { + assertThat(snapshot.summary()) + .as("Snapshot property " + property + " has unexpected value.") + .containsEntry(property, expectedValue); + } } protected void sleep(long millis) { 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 e9a8c13be56a..05887d2a8b1b 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; +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; import static org.apache.iceberg.TableProperties.DELETE_MODE; @@ -501,6 +502,31 @@ public void testDeleteNonExistingRecords() { sql("SELECT * FROM %s ORDER BY id ASC NULLS LAST", selectTarget())); } + @TestTemplate + public void deleteSingleRecordProducesDeleteOperation() throws NoSuchTableException { + createAndInitPartitionedTable(); + append(tableName, new Employee(1, "eng"), new Employee(2, "eng"), new Employee(3, "eng")); + + sql("DELETE FROM %s WHERE id = 2", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + assertThat(table.snapshots()).hasSize(2); + + Snapshot currentSnapshot = table.currentSnapshot(); + + if (mode(table) == COPY_ON_WRITE) { + // this is an OverwriteFiles and produces "overwrite" + validateCopyOnWrite(currentSnapshot, "1", "1", "1"); + } 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"); + } + + assertThat(sql("SELECT * FROM %s", tableName)) + .containsExactlyInAnyOrder(row(1, "eng"), row(3, "eng")); + } + @TestTemplate public void testDeleteWithoutCondition() { createAndInitPartitionedTable(); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 22e7df0f4e17..a5bcf53bd72c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -32,8 +32,10 @@ import java.util.stream.IntStream; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; import org.apache.iceberg.DataOperations; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.Files; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.RewriteFiles; @@ -484,7 +486,15 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception dataDeletes, deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(File.createTempFile("junit", null, temp.toFile()).getPath()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + + table.newRowDelta().addRows(dataFile).addDeletes(eqDeletes).commit(); // check pre-condition - that the above Delete file write - actually resulted in snapshot of // type OVERWRITE From c3017e842047961ad23b3eda625949b34b318ad9 Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Tue, 16 Apr 2024 07:43:58 -0700 Subject: [PATCH 0226/1019] Flink: Move flink/v1.18 to flink/v1.19 --- flink/{v1.18 => v1.19}/build.gradle | 0 flink/{v1.18 => v1.19}/flink-runtime/LICENSE | 0 flink/{v1.18 => v1.19}/flink-runtime/NOTICE | 0 .../java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java | 0 .../iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java | 0 .../src/main/java/org/apache/iceberg/flink/CatalogLoader.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkCatalog.java | 0 .../main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkConfParser.java | 0 .../main/java/org/apache/iceberg/flink/FlinkConfigOptions.java | 0 .../java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java | 0 .../java/org/apache/iceberg/flink/FlinkEnvironmentContext.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkFilters.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkReadConf.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java | 0 .../src/main/java/org/apache/iceberg/flink/IcebergTableSink.java | 0 .../src/main/java/org/apache/iceberg/flink/RowDataWrapper.java | 0 .../flink/src/main/java/org/apache/iceberg/flink/TableLoader.java | 0 .../src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java | 0 .../src/main/java/org/apache/iceberg/flink/actions/Actions.java | 0 .../org/apache/iceberg/flink/actions/RewriteDataFilesAction.java | 0 .../org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java | 0 .../java/org/apache/iceberg/flink/data/FlinkParquetReaders.java | 0 .../java/org/apache/iceberg/flink/data/FlinkParquetWriters.java | 0 .../java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java | 0 .../java/org/apache/iceberg/flink/data/FlinkValueReaders.java | 0 .../java/org/apache/iceberg/flink/data/FlinkValueWriters.java | 0 .../apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java | 0 .../java/org/apache/iceberg/flink/data/RowDataProjection.java | 0 .../src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java | 0 .../main/java/org/apache/iceberg/flink/data/StructRowData.java | 0 .../iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java | 0 .../java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java | 0 .../org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/BucketPartitioner.java | 0 .../java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java | 0 .../java/org/apache/iceberg/flink/sink/CachingTableSupplier.java | 0 .../main/java/org/apache/iceberg/flink/sink/CommitSummary.java | 0 .../main/java/org/apache/iceberg/flink/sink/DeltaManifests.java | 0 .../org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java | 0 .../org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java | 0 .../org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java | 0 .../java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java | 0 .../apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java | 0 .../java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java | 0 .../org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java | 0 .../org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java | 0 .../java/org/apache/iceberg/flink/sink/PartitionKeySelector.java | 0 .../org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java | 0 .../org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/TaskWriterFactory.java | 0 .../org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java | 0 .../apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java | 0 .../iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java | 0 .../org/apache/iceberg/flink/sink/shuffle/DataStatistics.java | 0 .../iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java | 0 .../flink/sink/shuffle/DataStatisticsCoordinatorProvider.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java | 0 .../flink/sink/shuffle/DataStatisticsOrRecordSerializer.java | 0 .../org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java | 0 .../org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java | 0 .../iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java | 0 .../apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java | 0 .../org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java | 0 .../iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java | 0 .../main/java/org/apache/iceberg/flink/source/DataIterator.java | 0 .../main/java/org/apache/iceberg/flink/source/DataTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/FileScanTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/FlinkInputFormat.java | 0 .../java/org/apache/iceberg/flink/source/FlinkInputSplit.java | 0 .../main/java/org/apache/iceberg/flink/source/FlinkSource.java | 0 .../java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java | 0 .../main/java/org/apache/iceberg/flink/source/IcebergSource.java | 0 .../java/org/apache/iceberg/flink/source/IcebergTableSource.java | 0 .../apache/iceberg/flink/source/RowDataFileScanTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/RowDataRewriter.java | 0 .../iceberg/flink/source/RowDataToAvroGenericRecordConverter.java | 0 .../main/java/org/apache/iceberg/flink/source/ScanContext.java | 0 .../src/main/java/org/apache/iceberg/flink/source/SourceUtil.java | 0 .../org/apache/iceberg/flink/source/StreamingMonitorFunction.java | 0 .../org/apache/iceberg/flink/source/StreamingReaderOperator.java | 0 .../apache/iceberg/flink/source/StreamingStartingStrategy.java | 0 .../iceberg/flink/source/assigner/DefaultSplitAssigner.java | 0 .../org/apache/iceberg/flink/source/assigner/GetSplitResult.java | 0 .../flink/source/assigner/OrderedSplitAssignerFactory.java | 0 .../iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java | 0 .../org/apache/iceberg/flink/source/assigner/SplitAssigner.java | 0 .../iceberg/flink/source/assigner/SplitAssignerFactory.java | 0 .../apache/iceberg/flink/source/assigner/SplitAssignerType.java | 0 .../flink/source/enumerator/AbstractIcebergEnumerator.java | 0 .../flink/source/enumerator/ContinuousEnumerationResult.java | 0 .../flink/source/enumerator/ContinuousIcebergEnumerator.java | 0 .../iceberg/flink/source/enumerator/ContinuousSplitPlanner.java | 0 .../flink/source/enumerator/ContinuousSplitPlannerImpl.java | 0 .../iceberg/flink/source/enumerator/EnumerationHistory.java | 0 .../flink/source/enumerator/IcebergEnumeratorPosition.java | 0 .../source/enumerator/IcebergEnumeratorPositionSerializer.java | 0 .../iceberg/flink/source/enumerator/IcebergEnumeratorState.java | 0 .../flink/source/enumerator/IcebergEnumeratorStateSerializer.java | 0 .../iceberg/flink/source/enumerator/StaticIcebergEnumerator.java | 0 .../org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java | 0 .../iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java | 0 .../flink/source/reader/AvroGenericRecordReaderFunction.java | 0 .../flink/source/reader/ColumnStatsWatermarkExtractor.java | 0 .../apache/iceberg/flink/source/reader/DataIteratorBatcher.java | 0 .../iceberg/flink/source/reader/DataIteratorReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/IcebergSourceReader.java | 0 .../iceberg/flink/source/reader/IcebergSourceReaderMetrics.java | 0 .../iceberg/flink/source/reader/IcebergSourceSplitReader.java | 0 .../org/apache/iceberg/flink/source/reader/ListBatchRecords.java | 0 .../iceberg/flink/source/reader/ListDataIteratorBatcher.java | 0 .../iceberg/flink/source/reader/MetaDataReaderFunction.java | 0 .../org/apache/iceberg/flink/source/reader/ReaderFunction.java | 0 .../org/apache/iceberg/flink/source/reader/RecordAndPosition.java | 0 .../org/apache/iceberg/flink/source/reader/RecordFactory.java | 0 .../apache/iceberg/flink/source/reader/RowDataReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/RowDataRecordFactory.java | 0 .../iceberg/flink/source/reader/SerializableRecordEmitter.java | 0 .../iceberg/flink/source/reader/SplitWatermarkExtractor.java | 0 .../flink/source/reader/WatermarkExtractorRecordEmitter.java | 0 .../org/apache/iceberg/flink/source/split/IcebergSourceSplit.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitSerializer.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitState.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitStatus.java | 0 .../apache/iceberg/flink/source/split/SerializableComparator.java | 0 .../org/apache/iceberg/flink/source/split/SplitComparators.java | 0 .../org/apache/iceberg/flink/source/split/SplitRequestEvent.java | 0 .../main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java | 0 .../java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java | 0 .../org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 .../services/org.apache.flink.table.factories.TableFactory | 0 .../org/apache/iceberg/flink/AvroGenericRecordConverterBase.java | 0 .../src/test/java/org/apache/iceberg/flink/CatalogTestBase.java | 0 .../src/test/java/org/apache/iceberg/flink/DataGenerator.java | 0 .../src/test/java/org/apache/iceberg/flink/DataGenerators.java | 0 .../src/test/java/org/apache/iceberg/flink/FlinkTestBase.java | 0 .../java/org/apache/iceberg/flink/HadoopCatalogExtension.java | 0 .../test/java/org/apache/iceberg/flink/HadoopCatalogResource.java | 0 .../test/java/org/apache/iceberg/flink/HadoopTableResource.java | 0 .../test/java/org/apache/iceberg/flink/MiniClusterResource.java | 0 .../java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java | 0 .../src/test/java/org/apache/iceberg/flink/RowDataConverter.java | 0 .../src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/TestBase.java | 0 .../src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java | 0 .../java/org/apache/iceberg/flink/TestCatalogTableLoader.java | 0 .../test/java/org/apache/iceberg/flink/TestChangeLogTable.java | 0 .../java/org/apache/iceberg/flink/TestDataFileSerialization.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFixtures.java | 0 .../java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java | 0 .../java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java | 0 .../java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java | 0 .../org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkConfParser.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkTableSink.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java | 0 .../test/java/org/apache/iceberg/flink/TestIcebergConnector.java | 0 .../org/apache/iceberg/flink/TestManifestFileSerialization.java | 0 .../test/java/org/apache/iceberg/flink/TestRowDataWrapper.java | 0 .../src/test/java/org/apache/iceberg/flink/TestTableLoader.java | 0 .../java/org/apache/iceberg/flink/TestTableSerialization.java | 0 .../apache/iceberg/flink/actions/TestRewriteDataFilesAction.java | 0 .../test/java/org/apache/iceberg/flink/data/RandomRowData.java | 0 .../java/org/apache/iceberg/flink/data/RowDataToRowMapper.java | 0 .../org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java | 0 .../org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java | 0 .../org/apache/iceberg/flink/data/TestFlinkParquetReader.java | 0 .../org/apache/iceberg/flink/data/TestFlinkParquetWriter.java | 0 .../java/org/apache/iceberg/flink/data/TestRowDataProjection.java | 0 .../java/org/apache/iceberg/flink/data/TestRowProjection.java | 0 .../java/org/apache/iceberg/flink/data/TestStructRowData.java | 0 .../iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java | 0 .../apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java | 0 .../iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java | 0 .../org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java | 0 .../org/apache/iceberg/flink/sink/TestCachingTableSupplier.java | 0 .../org/apache/iceberg/flink/sink/TestCompressionSettings.java | 0 .../java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java | 0 .../apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java | 0 .../java/org/apache/iceberg/flink/sink/TestFlinkManifest.java | 0 .../apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java | 0 .../apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java | 0 .../apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java | 0 .../org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java | 0 .../org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java | 0 .../org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java | 0 .../test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java | 0 .../iceberg/flink/sink/shuffle/TestAggregatedStatistics.java | 0 .../flink/sink/shuffle/TestAggregatedStatisticsTracker.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java | 0 .../flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java | 0 .../apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java | 0 .../iceberg/flink/sink/shuffle/TestMapRangePartitioner.java | 0 .../iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java | 0 .../flink/sink/shuffle/TestSortKeySerializerNestedStruct.java | 0 .../flink/sink/shuffle/TestSortKeySerializerPrimitives.java | 0 .../java/org/apache/iceberg/flink/source/BoundedTableFactory.java | 0 .../java/org/apache/iceberg/flink/source/BoundedTestSource.java | 0 .../org/apache/iceberg/flink/source/ChangeLogTableTestBase.java | 0 .../test/java/org/apache/iceberg/flink/source/SplitHelpers.java | 0 .../src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java | 0 .../org/apache/iceberg/flink/source/TestBoundedTableFactory.java | 0 .../org/apache/iceberg/flink/source/TestFlinkInputFormat.java | 0 .../iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java | 0 .../org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java | 0 .../org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java | 0 .../apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java | 0 .../test/java/org/apache/iceberg/flink/source/TestFlinkScan.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkScanSql.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkSource.java | 0 .../org/apache/iceberg/flink/source/TestFlinkSourceConfig.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java | 0 .../org/apache/iceberg/flink/source/TestFlinkTableSource.java | 0 .../org/apache/iceberg/flink/source/TestIcebergSourceBounded.java | 0 .../flink/source/TestIcebergSourceBoundedGenericRecord.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceContinuous.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceFailover.java | 0 .../source/TestIcebergSourceFailoverWithWatermarkExtractor.java | 0 .../iceberg/flink/source/TestIcebergSourceReaderDeletes.java | 0 .../org/apache/iceberg/flink/source/TestIcebergSourceSql.java | 0 .../flink/source/TestIcebergSourceWithWatermarkExtractor.java | 0 .../iceberg/flink/source/TestMetadataTableReadableMetrics.java | 0 .../org/apache/iceberg/flink/source/TestProjectMetaColumn.java | 0 .../flink/source/TestRowDataToAvroGenericRecordConverter.java | 0 .../test/java/org/apache/iceberg/flink/source/TestSourceUtil.java | 0 .../test/java/org/apache/iceberg/flink/source/TestSqlBase.java | 0 .../java/org/apache/iceberg/flink/source/TestStreamScanSql.java | 0 .../apache/iceberg/flink/source/TestStreamingMonitorFunction.java | 0 .../apache/iceberg/flink/source/TestStreamingReaderOperator.java | 0 .../iceberg/flink/source/assigner/SplitAssignerTestBase.java | 0 .../iceberg/flink/source/assigner/TestDefaultSplitAssigner.java | 0 .../source/assigner/TestFileSequenceNumberBasedSplitAssigner.java | 0 .../flink/source/assigner/TestWatermarkBasedSplitAssigner.java | 0 .../flink/source/enumerator/ManualContinuousSplitPlanner.java | 0 .../flink/source/enumerator/TestContinuousIcebergEnumerator.java | 0 .../flink/source/enumerator/TestContinuousSplitPlannerImpl.java | 0 .../enumerator/TestContinuousSplitPlannerImplStartStrategy.java | 0 .../iceberg/flink/source/enumerator/TestEnumerationHistory.java | 0 .../source/enumerator/TestIcebergEnumeratorStateSerializer.java | 0 .../iceberg/flink/source/reader/ReaderFunctionTestBase.java | 0 .../java/org/apache/iceberg/flink/source/reader/ReaderUtil.java | 0 .../apache/iceberg/flink/source/reader/TestArrayBatchRecords.java | 0 .../source/reader/TestArrayPoolDataIteratorBatcherRowData.java | 0 .../flink/source/reader/TestColumnStatsWatermarkExtractor.java | 0 .../iceberg/flink/source/reader/TestIcebergSourceReader.java | 0 .../iceberg/flink/source/reader/TestRowDataReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/TestingMetricGroup.java | 0 .../flink/source/split/TestIcebergSourceSplitSerializer.java | 0 .../test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 286 files changed, 0 insertions(+), 0 deletions(-) rename flink/{v1.18 => v1.19}/build.gradle (100%) rename flink/{v1.18 => v1.19}/flink-runtime/LICENSE (100%) rename flink/{v1.18 => v1.19}/flink-runtime/NOTICE (100%) rename flink/{v1.18 => v1.19}/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java (100%) rename flink/{v1.18 => v1.19}/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java (100%) rename flink/{v1.18 => v1.19}/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename flink/{v1.18 => v1.19}/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java (100%) rename flink/{v1.18 => v1.19}/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) diff --git a/flink/v1.18/build.gradle b/flink/v1.19/build.gradle similarity index 100% rename from flink/v1.18/build.gradle rename to flink/v1.19/build.gradle diff --git a/flink/v1.18/flink-runtime/LICENSE b/flink/v1.19/flink-runtime/LICENSE similarity index 100% rename from flink/v1.18/flink-runtime/LICENSE rename to flink/v1.19/flink-runtime/LICENSE diff --git a/flink/v1.18/flink-runtime/NOTICE b/flink/v1.19/flink-runtime/NOTICE similarity index 100% rename from flink/v1.18/flink-runtime/NOTICE rename to flink/v1.19/flink-runtime/NOTICE diff --git a/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java similarity index 100% rename from flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java rename to flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java similarity index 100% rename from flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java rename to flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java similarity index 100% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java diff --git a/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java similarity index 100% rename from flink/v1.18/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/TestFlinkAvroReaderWriter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java diff --git a/flink/v1.18/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 similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java similarity index 100% rename from flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java diff --git a/flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory From 352fee647b2f454bda17d562512d57480a999e6a Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Tue, 16 Apr 2024 07:44:22 -0700 Subject: [PATCH 0227/1019] Flink: Recover flink/1.18 files from history --- flink/v1.18/build.gradle | 266 ++++ flink/v1.18/flink-runtime/LICENSE | 502 +++++++ flink/v1.18/flink-runtime/NOTICE | 91 ++ .../flink/IcebergConnectorSmokeTest.java | 29 + .../shuffle/MapRangePartitionerBenchmark.java | 199 +++ .../apache/iceberg/flink/CatalogLoader.java | 215 +++ .../apache/iceberg/flink/FlinkCatalog.java | 833 ++++++++++++ .../iceberg/flink/FlinkCatalogFactory.java | 213 +++ .../apache/iceberg/flink/FlinkConfParser.java | 261 ++++ .../iceberg/flink/FlinkConfigOptions.java | 107 ++ .../flink/FlinkDynamicTableFactory.java | 208 +++ .../flink/FlinkEnvironmentContext.java | 31 + .../apache/iceberg/flink/FlinkFilters.java | 266 ++++ .../apache/iceberg/flink/FlinkFixupTypes.java | 50 + .../apache/iceberg/flink/FlinkReadConf.java | 213 +++ .../iceberg/flink/FlinkReadOptions.java | 123 ++ .../apache/iceberg/flink/FlinkSchemaUtil.java | 232 ++++ .../iceberg/flink/FlinkSourceFilter.java | 49 + .../apache/iceberg/flink/FlinkTypeToType.java | 203 +++ .../iceberg/flink/FlinkTypeVisitor.java | 80 ++ .../apache/iceberg/flink/FlinkWriteConf.java | 205 +++ .../iceberg/flink/FlinkWriteOptions.java | 73 + .../iceberg/flink/IcebergTableSink.java | 121 ++ .../apache/iceberg/flink/RowDataWrapper.java | 142 ++ .../org/apache/iceberg/flink/TableLoader.java | 159 +++ .../apache/iceberg/flink/TypeToFlinkType.java | 134 ++ .../apache/iceberg/flink/actions/Actions.java | 52 + .../flink/actions/RewriteDataFilesAction.java | 72 + .../data/AvroWithFlinkSchemaVisitor.java | 75 ++ .../iceberg/flink/data/FlinkAvroReader.java | 169 +++ .../iceberg/flink/data/FlinkAvroWriter.java | 165 +++ .../iceberg/flink/data/FlinkOrcReader.java | 131 ++ .../iceberg/flink/data/FlinkOrcReaders.java | 283 ++++ .../iceberg/flink/data/FlinkOrcWriter.java | 163 +++ .../iceberg/flink/data/FlinkOrcWriters.java | 317 +++++ .../flink/data/FlinkParquetReaders.java | 832 ++++++++++++ .../flink/data/FlinkParquetWriters.java | 504 +++++++ .../flink/data/FlinkSchemaVisitor.java | 161 +++ .../iceberg/flink/data/FlinkValueReaders.java | 312 +++++ .../iceberg/flink/data/FlinkValueWriters.java | 253 ++++ .../data/ParquetWithFlinkSchemaVisitor.java | 222 ++++ .../iceberg/flink/data/RowDataProjection.java | 341 +++++ .../iceberg/flink/data/RowDataUtil.java | 100 ++ .../iceberg/flink/data/StructRowData.java | 300 +++++ .../AvroGenericRecordToRowDataMapper.java | 61 + .../flink/sink/BaseDeltaTaskWriter.java | 125 ++ .../sink/BucketPartitionKeySelector.java | 70 + .../iceberg/flink/sink/BucketPartitioner.java | 103 ++ .../flink/sink/BucketPartitionerUtil.java | 125 ++ .../flink/sink/CachingTableSupplier.java | 91 ++ .../iceberg/flink/sink/CommitSummary.java | 93 ++ .../iceberg/flink/sink/DeltaManifests.java | 71 + .../flink/sink/DeltaManifestsSerializer.java | 122 ++ .../flink/sink/EqualityFieldKeySelector.java | 86 ++ .../flink/sink/FlinkAppenderFactory.java | 274 ++++ .../flink/sink/FlinkFileWriterFactory.java | 293 +++++ .../iceberg/flink/sink/FlinkManifestUtil.java | 132 ++ .../apache/iceberg/flink/sink/FlinkSink.java | 654 +++++++++ .../flink/sink/IcebergFilesCommitter.java | 516 ++++++++ .../sink/IcebergFilesCommitterMetrics.java | 73 + .../flink/sink/IcebergStreamWriter.java | 120 ++ .../sink/IcebergStreamWriterMetrics.java | 89 ++ .../flink/sink/ManifestOutputFileFactory.java | 94 ++ .../flink/sink/PartitionKeySelector.java | 64 + .../flink/sink/PartitionedDeltaWriter.java | 97 ++ .../flink/sink/RowDataTaskWriterFactory.java | 244 ++++ .../iceberg/flink/sink/TaskWriterFactory.java | 45 + .../flink/sink/UnpartitionedDeltaWriter.java | 69 + .../sink/shuffle/AggregatedStatistics.java | 71 + .../shuffle/AggregatedStatisticsTracker.java | 133 ++ .../flink/sink/shuffle/DataStatistics.java | 57 + .../shuffle/DataStatisticsCoordinator.java | 396 ++++++ .../DataStatisticsCoordinatorProvider.java | 51 + .../sink/shuffle/DataStatisticsEvent.java | 57 + .../sink/shuffle/DataStatisticsOperator.java | 197 +++ .../sink/shuffle/DataStatisticsOrRecord.java | 113 ++ .../DataStatisticsOrRecordSerializer.java | 219 +++ .../sink/shuffle/DataStatisticsUtil.java | 96 ++ .../flink/sink/shuffle/MapDataStatistics.java | 70 + .../shuffle/MapDataStatisticsSerializer.java | 187 +++ .../sink/shuffle/MapRangePartitioner.java | 381 ++++++ .../flink/sink/shuffle/SortKeySerializer.java | 353 +++++ .../AvroGenericRecordFileScanTaskReader.java | 42 + .../iceberg/flink/source/DataIterator.java | 156 +++ .../iceberg/flink/source/DataTaskReader.java | 47 + .../flink/source/FileScanTaskReader.java | 35 + .../flink/source/FlinkInputFormat.java | 141 ++ .../iceberg/flink/source/FlinkInputSplit.java | 48 + .../iceberg/flink/source/FlinkSource.java | 310 +++++ .../flink/source/FlinkSplitPlanner.java | 189 +++ .../iceberg/flink/source/IcebergSource.java | 543 ++++++++ .../flink/source/IcebergTableSource.java | 229 ++++ .../source/RowDataFileScanTaskReader.java | 243 ++++ .../iceberg/flink/source/RowDataRewriter.java | 172 +++ .../RowDataToAvroGenericRecordConverter.java | 70 + .../iceberg/flink/source/ScanContext.java | 592 +++++++++ .../iceberg/flink/source/SourceUtil.java | 77 ++ .../source/StreamingMonitorFunction.java | 269 ++++ .../flink/source/StreamingReaderOperator.java | 246 ++++ .../source/StreamingStartingStrategy.java | 54 + .../source/assigner/DefaultSplitAssigner.java | 119 ++ .../flink/source/assigner/GetSplitResult.java | 77 ++ .../assigner/OrderedSplitAssignerFactory.java | 46 + .../assigner/SimpleSplitAssignerFactory.java | 37 + .../flink/source/assigner/SplitAssigner.java | 124 ++ .../source/assigner/SplitAssignerFactory.java | 30 + .../source/assigner/SplitAssignerType.java | 33 + .../enumerator/AbstractIcebergEnumerator.java | 175 +++ .../ContinuousEnumerationResult.java | 57 + .../ContinuousIcebergEnumerator.java | 187 +++ .../enumerator/ContinuousSplitPlanner.java | 30 + .../ContinuousSplitPlannerImpl.java | 240 ++++ .../source/enumerator/EnumerationHistory.java | 96 ++ .../enumerator/IcebergEnumeratorPosition.java | 79 ++ .../IcebergEnumeratorPositionSerializer.java | 90 ++ .../enumerator/IcebergEnumeratorState.java | 65 + .../IcebergEnumeratorStateSerializer.java | 196 +++ .../enumerator/StaticIcebergEnumerator.java | 51 + .../source/reader/ArrayBatchRecords.java | 171 +++ .../reader/ArrayPoolDataIteratorBatcher.java | 130 ++ .../AvroGenericRecordReaderFunction.java | 102 ++ .../reader/ColumnStatsWatermarkExtractor.java | 98 ++ .../source/reader/DataIteratorBatcher.java | 36 + .../reader/DataIteratorReaderFunction.java | 43 + .../source/reader/IcebergSourceReader.java | 77 ++ .../reader/IcebergSourceReaderMetrics.java | 61 + .../reader/IcebergSourceSplitReader.java | 167 +++ .../flink/source/reader/ListBatchRecords.java | 85 ++ .../reader/ListDataIteratorBatcher.java | 94 ++ .../source/reader/MetaDataReaderFunction.java | 65 + .../flink/source/reader/ReaderFunction.java | 31 + .../source/reader/RecordAndPosition.java | 78 ++ .../flink/source/reader/RecordFactory.java | 34 + .../source/reader/RowDataReaderFunction.java | 79 ++ .../source/reader/RowDataRecordFactory.java | 61 + .../reader/SerializableRecordEmitter.java | 40 + .../reader/SplitWatermarkExtractor.java | 28 + .../WatermarkExtractorRecordEmitter.java | 67 + .../source/split/IcebergSourceSplit.java | 177 +++ .../split/IcebergSourceSplitSerializer.java | 60 + .../source/split/IcebergSourceSplitState.java | 37 + .../split/IcebergSourceSplitStatus.java | 25 + .../source/split/SerializableComparator.java | 24 + .../flink/source/split/SplitComparators.java | 76 ++ .../flink/source/split/SplitRequestEvent.java | 54 + .../iceberg/flink/util/ElapsedTimeGauge.java | 47 + .../flink/util/FlinkAlterTableUtil.java | 248 ++++ .../flink/util/FlinkCompatibilityUtil.java | 47 + .../iceberg/flink/util/FlinkPackage.java | 61 + .../org.apache.flink.table.factories.Factory | 16 + ....apache.flink.table.factories.TableFactory | 16 + .../flink/AvroGenericRecordConverterBase.java | 90 ++ .../apache/iceberg/flink/CatalogTestBase.java | 143 ++ .../apache/iceberg/flink/DataGenerator.java | 42 + .../apache/iceberg/flink/DataGenerators.java | 1172 +++++++++++++++++ .../apache/iceberg/flink/FlinkTestBase.java | 129 ++ .../iceberg/flink/HadoopCatalogExtension.java | 104 ++ .../iceberg/flink/HadoopCatalogResource.java | 90 ++ .../iceberg/flink/HadoopTableResource.java | 64 + .../iceberg/flink/MiniClusterResource.java | 68 + .../flink/MiniFlinkClusterExtension.java | 53 + .../iceberg/flink/RowDataConverter.java | 135 ++ .../apache/iceberg/flink/SimpleDataUtil.java | 443 +++++++ .../org/apache/iceberg/flink/TestBase.java | 130 ++ .../iceberg/flink/TestCatalogLoader.java | 118 ++ .../iceberg/flink/TestCatalogTableLoader.java | 116 ++ .../iceberg/flink/TestChangeLogTable.java | 305 +++++ .../flink/TestDataFileSerialization.java | 203 +++ .../apache/iceberg/flink/TestFixtures.java | 59 + .../flink/TestFlinkAnonymousTable.java | 64 + .../flink/TestFlinkCatalogDatabase.java | 254 ++++ .../flink/TestFlinkCatalogFactory.java | 119 ++ .../iceberg/flink/TestFlinkCatalogTable.java | 672 ++++++++++ .../TestFlinkCatalogTablePartitions.java | 119 ++ .../iceberg/flink/TestFlinkConfParser.java | 61 + .../iceberg/flink/TestFlinkFilters.java | 462 +++++++ .../iceberg/flink/TestFlinkHiveCatalog.java | 103 ++ .../iceberg/flink/TestFlinkSchemaUtil.java | 416 ++++++ .../iceberg/flink/TestFlinkTableSink.java | 358 +++++ .../apache/iceberg/flink/TestFlinkUpsert.java | 334 +++++ .../org/apache/iceberg/flink/TestHelpers.java | 628 +++++++++ .../iceberg/flink/TestIcebergConnector.java | 350 +++++ .../flink/TestManifestFileSerialization.java | 173 +++ .../iceberg/flink/TestRowDataWrapper.java | 93 ++ .../apache/iceberg/flink/TestTableLoader.java | 57 + .../iceberg/flink/TestTableSerialization.java | 110 ++ .../actions/TestRewriteDataFilesAction.java | 481 +++++++ .../iceberg/flink/data/RandomRowData.java | 38 + .../flink/data/RowDataToRowMapper.java | 50 + .../flink/data/TestFlinkAvroReaderWriter.java | 185 +++ .../flink/data/TestFlinkOrcReaderWriter.java | 107 ++ .../flink/data/TestFlinkParquetReader.java | 138 ++ .../flink/data/TestFlinkParquetWriter.java | 94 ++ .../flink/data/TestRowDataProjection.java | 593 +++++++++ .../iceberg/flink/data/TestRowProjection.java | 596 +++++++++ .../iceberg/flink/data/TestStructRowData.java | 100 ++ .../TestAvroGenericRecordToRowDataMapper.java | 38 + .../sink/TestBucketPartitionKeySelector.java | 65 + .../flink/sink/TestBucketPartitioner.java | 107 ++ ...TestBucketPartitionerFlinkIcebergSink.java | 227 ++++ .../flink/sink/TestBucketPartitionerUtil.java | 126 ++ .../flink/sink/TestCachingTableSupplier.java | 81 ++ .../flink/sink/TestCompressionSettings.java | 254 ++++ .../flink/sink/TestDeltaTaskWriter.java | 429 ++++++ .../flink/sink/TestFlinkAppenderFactory.java | 65 + .../sink/TestFlinkFileWriterFactory.java | 66 + .../flink/sink/TestFlinkIcebergSink.java | 397 ++++++ .../flink/sink/TestFlinkIcebergSinkBase.java | 64 + .../sink/TestFlinkIcebergSinkBranch.java | 137 ++ .../flink/sink/TestFlinkIcebergSinkV2.java | 236 ++++ .../sink/TestFlinkIcebergSinkV2Base.java | 362 +++++ .../sink/TestFlinkIcebergSinkV2Branch.java | 134 ++ .../iceberg/flink/sink/TestFlinkManifest.java | 315 +++++ .../sink/TestFlinkPartitioningWriters.java | 77 ++ .../sink/TestFlinkPositionDeltaWriters.java | 66 + .../sink/TestFlinkRollingFileWriters.java | 51 + .../flink/sink/TestFlinkWriterMetrics.java | 60 + .../flink/sink/TestIcebergFilesCommitter.java | 1148 ++++++++++++++++ .../flink/sink/TestIcebergStreamWriter.java | 388 ++++++ .../flink/sink/TestRowDataPartitionKey.java | 252 ++++ .../iceberg/flink/sink/TestTaskWriters.java | 243 ++++ .../shuffle/TestAggregatedStatistics.java | 63 + .../TestAggregatedStatisticsTracker.java | 178 +++ .../TestDataStatisticsCoordinator.java | 178 +++ ...TestDataStatisticsCoordinatorProvider.java | 143 ++ .../shuffle/TestDataStatisticsOperator.java | 255 ++++ .../sink/shuffle/TestMapDataStatistics.java | 90 ++ .../sink/shuffle/TestMapRangePartitioner.java | 448 +++++++ .../shuffle/TestSortKeySerializerBase.java | 65 + .../TestSortKeySerializerNestedStruct.java | 55 + .../TestSortKeySerializerPrimitives.java | 57 + .../flink/source/BoundedTableFactory.java | 170 +++ .../flink/source/BoundedTestSource.java | 108 ++ .../flink/source/ChangeLogTableTestBase.java | 87 ++ .../iceberg/flink/source/SplitHelpers.java | 132 ++ .../iceberg/flink/source/SqlHelpers.java | 60 + .../flink/source/TestBoundedTableFactory.java | 87 ++ .../flink/source/TestFlinkInputFormat.java | 191 +++ .../TestFlinkInputFormatReaderDeletes.java | 71 + .../flink/source/TestFlinkMergingMetrics.java | 71 + .../flink/source/TestFlinkMetaDataTable.java | 813 ++++++++++++ .../source/TestFlinkReaderDeletesBase.java | 90 ++ .../iceberg/flink/source/TestFlinkScan.java | 540 ++++++++ .../flink/source/TestFlinkScanSql.java | 69 + .../iceberg/flink/source/TestFlinkSource.java | 90 ++ .../flink/source/TestFlinkSourceConfig.java | 58 + .../flink/source/TestFlinkSourceSql.java | 85 ++ .../flink/source/TestFlinkTableSource.java | 614 +++++++++ .../source/TestIcebergSourceBounded.java | 127 ++ ...TestIcebergSourceBoundedGenericRecord.java | 203 +++ .../source/TestIcebergSourceBoundedSql.java | 76 ++ .../source/TestIcebergSourceContinuous.java | 543 ++++++++ .../source/TestIcebergSourceFailover.java | 297 +++++ ...gSourceFailoverWithWatermarkExtractor.java | 112 ++ .../TestIcebergSourceReaderDeletes.java | 102 ++ .../flink/source/TestIcebergSourceSql.java | 158 +++ ...stIcebergSourceWithWatermarkExtractor.java | 408 ++++++ .../TestMetadataTableReadableMetrics.java | 299 +++++ .../flink/source/TestProjectMetaColumn.java | 186 +++ ...stRowDataToAvroGenericRecordConverter.java | 36 + .../iceberg/flink/source/TestSourceUtil.java | 60 + .../iceberg/flink/source/TestSqlBase.java | 158 +++ .../flink/source/TestStreamScanSql.java | 430 ++++++ .../source/TestStreamingMonitorFunction.java | 402 ++++++ .../source/TestStreamingReaderOperator.java | 293 +++++ .../assigner/SplitAssignerTestBase.java | 130 ++ .../assigner/TestDefaultSplitAssigner.java | 43 + ...tFileSequenceNumberBasedSplitAssigner.java | 80 ++ .../TestWatermarkBasedSplitAssigner.java | 146 ++ .../ManualContinuousSplitPlanner.java | 97 ++ .../TestContinuousIcebergEnumerator.java | 352 +++++ .../TestContinuousSplitPlannerImpl.java | 692 ++++++++++ ...ntinuousSplitPlannerImplStartStrategy.java | 205 +++ .../enumerator/TestEnumerationHistory.java | 134 ++ .../TestIcebergEnumeratorStateSerializer.java | 145 ++ .../source/reader/ReaderFunctionTestBase.java | 219 +++ .../flink/source/reader/ReaderUtil.java | 125 ++ .../source/reader/TestArrayBatchRecords.java | 68 + ...stArrayPoolDataIteratorBatcherRowData.java | 358 +++++ .../TestColumnStatsWatermarkExtractor.java | 178 +++ .../reader/TestIcebergSourceReader.java | 184 +++ .../reader/TestRowDataReaderFunction.java | 74 ++ .../source/reader/TestingMetricGroup.java | 102 ++ .../TestIcebergSourceSplitSerializer.java | 162 +++ .../iceberg/flink/util/TestFlinkPackage.java | 55 + .../org.apache.flink.table.factories.Factory | 16 + 286 files changed, 51109 insertions(+) create mode 100644 flink/v1.18/build.gradle create mode 100644 flink/v1.18/flink-runtime/LICENSE create mode 100644 flink/v1.18/flink-runtime/NOTICE create mode 100644 flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java create mode 100644 flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java create mode 100644 flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java create mode 100644 flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle new file mode 100644 index 000000000000..c08ae5d8cc1f --- /dev/null +++ b/flink/v1.18/build.gradle @@ -0,0 +1,266 @@ +/* + * 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. + */ + +String flinkMajorVersion = '1.18' +String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") + +project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { + + dependencies { + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + api project(':iceberg-api') + implementation project(':iceberg-common') + implementation project(':iceberg-core') + api project(':iceberg-data') + implementation project(':iceberg-orc') + implementation project(':iceberg-parquet') + implementation project(':iceberg-hive-metastore') + + compileOnly libs.flink118.avro + // for dropwizard histogram metrics implementation + compileOnly libs.flink118.metrics.dropwizard + compileOnly libs.flink118.streaming.java + compileOnly "${libs.flink118.streaming.java.get().module}:${libs.flink118.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink118.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" + compileOnly libs.flink118.connector.base + compileOnly libs.flink118.connector.files + + compileOnly libs.hadoop2.hdfs + compileOnly libs.hadoop2.common + compileOnly(libs.hadoop2.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + } + + 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.avro.avro + + implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { + exclude group: 'org.apache.hadoop' + exclude group: 'commons-lang' + // These artifacts are shaded and included in the orc-core fat jar + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.hive', module: 'hive-storage-api' + exclude group: 'org.slf4j' + } + + testImplementation libs.flink118.connector.test.utils + testImplementation libs.flink118.core + testImplementation libs.flink118.runtime + testImplementation(libs.flink118.test.utilsjunit) { + exclude group: 'junit' + } + testImplementation(libs.flink118.test.utils) { + exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' + } + + testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + + // By default, hive-exec is a fat/uber jar and it exports a guava library + // that's really old. We use the core classifier to be able to override our guava + // version. Luckily, hive-exec seems to work okay so far with this version of guava + // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. + testImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + testImplementation(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + exclude group: 'org.slf4j' + } + + testImplementation libs.awaitility + testImplementation libs.assertj.core + } + + test { + useJUnitPlatform() + } +} + +project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { + apply plugin: 'com.github.johnrengelman.shadow' + + tasks.jar.dependsOn tasks.shadowJar + + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + } + } + + configurations { + implementation { + // included in Flink + exclude group: 'org.slf4j' + exclude group: 'org.apache.commons' + exclude group: 'commons-pool' + exclude group: 'commons-codec' + exclude group: 'org.xerial.snappy' + exclude group: 'javax.xml.bind' + exclude group: 'javax.annotation' + } + } + + dependencies { + implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) { + exclude group: 'org.apache.flink' + } + implementation project(':iceberg-aws') + implementation project(':iceberg-azure') + implementation(project(':iceberg-aliyun')) { + exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' + exclude group: 'org.apache.httpcomponents', module: 'httpclient' + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation project(':iceberg-gcp') + implementation(project(':iceberg-nessie')) { + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + // for dropwizard histogram metrics implementation + implementation libs.flink118.metrics.dropwizard + + // for integration testing with the flink-runtime-jar + // all of those dependencies are required because the integration test extends FlinkTestBase + integrationCompileOnly project(':iceberg-api') + integrationImplementation libs.junit.vintage.engine + integrationImplementation libs.assertj.core + integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") + integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation(libs.flink118.test.utils) { + exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' + } + + integrationImplementation libs.flink118.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" + + integrationImplementation libs.hadoop2.common + integrationImplementation libs.hadoop2.hdfs + integrationImplementation(libs.hadoop2.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + } + + integrationImplementation(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + exclude group: 'org.slf4j' + } + + integrationImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + } + + shadowJar { + configurations = [project.configurations.runtimeClasspath] + + zip64 true + + // include the LICENSE and NOTICE files for the shaded Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + // Relocate dependencies to avoid conflicts + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' + relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' + relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' + relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' + relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' + relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' + relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' + relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' + relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' + + archiveClassifier.set(null) + } + + task integrationTest(type: Test) { + description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" + group = "verification" + jvmArgs += project.property('extraJvmArgs') + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) + inputs.file(shadowJar.archiveFile.get().asFile.path) + } + integrationTest.dependsOn shadowJar + check.dependsOn integrationTest + + jar { + enabled = false + } +} diff --git a/flink/v1.18/flink-runtime/LICENSE b/flink/v1.18/flink-runtime/LICENSE new file mode 100644 index 000000000000..8ab53469eb87 --- /dev/null +++ b/flink/v1.18/flink-runtime/LICENSE @@ -0,0 +1,502 @@ + + 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 Apache Avro. + +Copyright: 2014-2020 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains the Jackson JSON processor. + +Copyright: 2007-2020 Tatu Saloranta and other contributors +Home page: http://jackson.codehaus.org/ +License: http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Parquet. + +Copyright: 2014-2020 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Thrift. + +Copyright: 2006-2010 The Apache Software Foundation. +Home page: https://thrift.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains fastutil. + +Copyright: 2002-2014 Sebastiano Vigna +Home page: http://fastutil.di.unimi.it/ +License: http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache ORC. + +Copyright: 2013-2020 The Apache Software Foundation. +Home page: https://orc.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Hive's storage API via ORC. + +Copyright: 2013-2020 The Apache Software Foundation. +Home page: https://hive.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Aircompressor. + +Copyright: 2011-2020 Aircompressor authors. +Home page: https://github.com/airlift/aircompressor +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Slice. + +Copyright: 2013-2020 Slice authors. +Home page: https://github.com/airlift/slice +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains JetBrains annotations. + +Copyright: 2000-2020 JetBrains s.r.o. +Home page: https://github.com/JetBrains/java-annotations +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Guava. + +Copyright: 2006-2020 The Guava Authors +Home page: https://github.com/google/guava +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Error Prone Annotations. + +Copyright: Copyright 2011-2019 The Error Prone Authors +Home page: https://github.com/google/error-prone +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains checkerframework checker-qual Annotations. + +Copyright: 2004-2020 the Checker Framework developers +Home page: https://github.com/typetools/checker-framework +License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) + +License text: +| The annotations are licensed under the MIT License. (The text of this +| license appears below.) More specifically, all the parts of the Checker +| Framework that you might want to include with your own program use the +| MIT License. This is the checker-qual.jar file and all the files that +| appear in it: every file in a qual/ directory, plus utility files such +| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. +| In addition, the cleanroom implementations of third-party annotations, +| which the Checker Framework recognizes as aliases for its own +| annotations, are licensed under the MIT License. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Animal Sniffer Annotations. + +Copyright: 2009-2018 codehaus.org +Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/ +License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license) + +License text: +| The MIT License +| +| Copyright (c) 2009 codehaus.org. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Caffeine by Ben Manes. + +Copyright: 2014-2020 Ben Manes and contributors +Home page: https://github.com/ben-manes/caffeine +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Yetus audience annotations. + +Copyright: 2008-2020 The Apache Software Foundation. +Home page: https://yetus.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google protobuf. + +Copyright: 2008 Google Inc. +Home page: https://developers.google.com/protocol-buffers +License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) + +License text: + +| Copyright 2008 Google Inc. All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +| +| Code generated by the Protocol Buffer compiler is owned by the owner +| of the input file used when generating it. This code is not +| standalone and requires a support library to be linked with it. This +| support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +This binary artifact contains ThreeTen. + +Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. +Home page: https://www.threeten.org/threeten-extra/ +License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) + +License text: + +| All rights reserved. +| +| * Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are met: +| +| * Redistributions of source code must retain the above copyright notice, +| this list of conditions and the following disclaimer. +| +| * Redistributions in binary form must reproduce the above copyright notice, +| this list of conditions and the following disclaimer in the documentation +| and/or other materials provided with the distribution. +| +| * Neither the name of JSR-310 nor the names of its contributors +| may be used to endorse or promote products derived from this software +| without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR +| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Dremio +| Copyright 2015-2017 Dremio Corporation +| +| This product includes software developed at +| The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary 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 binary artifact contains Apache HttpComponents Client. + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: http://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 diff --git a/flink/v1.18/flink-runtime/NOTICE b/flink/v1.18/flink-runtime/NOTICE new file mode 100644 index 000000000000..81aa1660456a --- /dev/null +++ b/flink/v1.18/flink-runtime/NOTICE @@ -0,0 +1,91 @@ + +Apache Iceberg +Copyright 2017-2022 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary artifact includes Apache ORC with the following in its NOTICE file: + +| Apache ORC +| Copyright 2013-2019 The Apache Software Foundation +| +| This product includes software developed by The Apache Software +| Foundation (http://www.apache.org/). +| +| This product includes software developed by Hewlett-Packard: +| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + +-------------------------------------------------------------------------------- + +This binary artifact includes Airlift Aircompressor with the following in its +NOTICE file: + +| Snappy Copyright Notices +| ========================= +| +| * Copyright 2011 Dain Sundstrom +| * Copyright 2011, Google Inc. +| +| +| Snappy License +| =============== +| Copyright 2011, Google Inc. +| All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Apache Yetus with the following in its NOTICE +file: + +| Apache Yetus +| Copyright 2008-2020 The Apache Software Foundation +| +| This product includes software developed at +| The Apache Software Foundation (https://www.apache.org/). +| +| --- +| Additional licenses for the Apache Yetus Source/Website: +| --- +| +| +| See LICENSE for terms. + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Dremio +| Copyright 2015-2017 Dremio Corporation +| +| This product includes software developed at +| The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java new file mode 100644 index 000000000000..65cfba1ec876 --- /dev/null +++ b/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.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.flink; + +import java.util.Map; + +public class IcebergConnectorSmokeTest extends TestIcebergConnector { + + public IcebergConnectorSmokeTest( + String catalogName, Map properties, boolean isStreaming) { + super(catalogName, properties, isStreaming); + } +} diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +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.apache.iceberg.types.Types; +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.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java new file mode 100644 index 000000000000..18473bf4f190 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java @@ -0,0 +1,215 @@ +/* + * 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; + +import java.io.Serializable; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.hive.HiveCatalog; +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.Maps; +import org.apache.iceberg.rest.RESTCatalog; + +/** Serializable loader to load an Iceberg {@link Catalog}. */ +public interface CatalogLoader extends Serializable, Cloneable { + + /** + * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the + * {@link CatalogLoader} at flink sql client side or job manager side, and then serialize this + * catalog loader to task manager, finally deserialize it and create a new catalog at task manager + * side. + * + * @return a newly created {@link Catalog} + */ + Catalog loadCatalog(); + + /** Clone a CatalogLoader. */ + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + CatalogLoader clone(); + + static CatalogLoader hadoop( + String name, Configuration hadoopConf, Map properties) { + return new HadoopCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader hive(String name, Configuration hadoopConf, Map properties) { + return new HiveCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader rest(String name, Configuration hadoopConf, Map properties) { + return new RESTCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader custom( + String name, Map properties, Configuration hadoopConf, String impl) { + return new CustomCatalogLoader(name, properties, hadoopConf, impl); + } + + class HadoopCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final String warehouseLocation; + private final Map properties; + + private HadoopCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new HadoopCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("warehouseLocation", warehouseLocation) + .toString(); + } + } + + class HiveCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final String uri; + private final String warehouse; + private final int clientPoolSize; + private final Map properties; + + private HiveCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.uri = properties.get(CatalogProperties.URI); + this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + this.clientPoolSize = + properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) + ? Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) + : CatalogProperties.CLIENT_POOL_SIZE_DEFAULT; + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new HiveCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("uri", uri) + .add("warehouse", warehouse) + .add("clientPoolSize", clientPoolSize) + .toString(); + } + } + + class RESTCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final Map properties; + + private RESTCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + RESTCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new RESTCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("properties", properties) + .toString(); + } + } + + class CustomCatalogLoader implements CatalogLoader { + + private final SerializableConfiguration hadoopConf; + private final Map properties; + private final String name; + private final String impl; + + private CustomCatalogLoader( + String name, Map properties, Configuration conf, String impl) { + this.hadoopConf = new SerializableConfiguration(conf); + this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization + this.name = name; + this.impl = + Preconditions.checkNotNull( + impl, "Cannot initialize custom Catalog, impl class name is null"); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new CustomCatalogLoader(name, properties, new Configuration(hadoopConf.get()), impl); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("name", name).add("impl", impl).toString(); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java new file mode 100644 index 000000000000..86295d78cc13 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -0,0 +1,833 @@ +/* + * 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; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +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.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.flink.util.FlinkAlterTableUtil; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +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.relocated.com.google.common.collect.Sets; + +/** + * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}. + * + *

The mapping between Flink database and Iceberg namespace: Supplying a base namespace for a + * given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the + * first level in the catalog configuration and the second level would be exposed as Flink + * databases. + * + *

The Iceberg table manages its partitions by itself. The partition of the Iceberg table is + * independent of the partition of Flink. + */ +public class FlinkCatalog extends AbstractCatalog { + private final CatalogLoader catalogLoader; + private final Catalog icebergCatalog; + private final Namespace baseNamespace; + private final SupportsNamespaces asNamespaceCatalog; + private final Closeable closeable; + private final boolean cacheEnabled; + + public FlinkCatalog( + String catalogName, + String defaultDatabase, + Namespace baseNamespace, + CatalogLoader catalogLoader, + boolean cacheEnabled, + long cacheExpirationIntervalMs) { + super(catalogName, defaultDatabase); + this.catalogLoader = catalogLoader; + this.baseNamespace = baseNamespace; + this.cacheEnabled = cacheEnabled; + + Catalog originalCatalog = catalogLoader.loadCatalog(); + icebergCatalog = + cacheEnabled + ? CachingCatalog.wrap(originalCatalog, cacheExpirationIntervalMs) + : originalCatalog; + asNamespaceCatalog = + originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null; + closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null; + + FlinkEnvironmentContext.init(); + } + + @Override + public void open() throws CatalogException {} + + @Override + public void close() throws CatalogException { + if (closeable != null) { + try { + closeable.close(); + } catch (IOException e) { + throw new CatalogException(e); + } + } + } + + public Catalog catalog() { + return icebergCatalog; + } + + /** Append a new level to the base namespace */ + private static Namespace appendLevel(Namespace baseNamespace, String newLevel) { + String[] namespace = new String[baseNamespace.levels().length + 1]; + System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length); + namespace[baseNamespace.levels().length] = newLevel; + return Namespace.of(namespace); + } + + TableIdentifier toIdentifier(ObjectPath path) { + String objectName = path.getObjectName(); + List tableName = Splitter.on('$').splitToList(objectName); + + if (tableName.size() == 1) { + return TableIdentifier.of( + appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName()); + } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) { + return TableIdentifier.of( + appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)), + tableName.get(1)); + } else { + throw new IllegalArgumentException("Illegal table name:" + objectName); + } + } + + @Override + public List listDatabases() throws CatalogException { + if (asNamespaceCatalog == null) { + return Collections.singletonList(getDefaultDatabase()); + } + + return asNamespaceCatalog.listNamespaces(baseNamespace).stream() + .map(n -> n.level(n.levels().length - 1)) + .collect(Collectors.toList()); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + if (asNamespaceCatalog == null) { + if (!getDefaultDatabase().equals(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } else { + return new CatalogDatabaseImpl(Maps.newHashMap(), ""); + } + } else { + try { + Map metadata = + Maps.newHashMap( + asNamespaceCatalog.loadNamespaceMetadata(appendLevel(baseNamespace, databaseName))); + String comment = metadata.remove("comment"); + return new CatalogDatabaseImpl(metadata, comment); + } catch (NoSuchNamespaceException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + getDatabase(databaseName); + return true; + } catch (DatabaseNotExistException ignore) { + return false; + } + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + createDatabase( + name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists); + } + + private void createDatabase( + String databaseName, Map metadata, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + if (asNamespaceCatalog != null) { + try { + asNamespaceCatalog.createNamespace(appendLevel(baseNamespace, databaseName), metadata); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), databaseName, e); + } + } + } else { + throw new UnsupportedOperationException( + "Namespaces are not supported by catalog: " + getName()); + } + } + + private Map mergeComment(Map metadata, String comment) { + Map ret = Maps.newHashMap(metadata); + if (metadata.containsKey("comment")) { + throw new CatalogException("Database properties should not contain key: 'comment'."); + } + + if (!StringUtils.isNullOrWhitespaceOnly(comment)) { + ret.put("comment", comment); + } + return ret; + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + if (asNamespaceCatalog != null) { + try { + boolean success = asNamespaceCatalog.dropNamespace(appendLevel(baseNamespace, name)); + if (!success && !ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (NoSuchNamespaceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name, e); + } + } catch (NamespaceNotEmptyException e) { + throw new DatabaseNotEmptyException(getName(), name, e); + } + } else { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + if (asNamespaceCatalog != null) { + Namespace namespace = appendLevel(baseNamespace, name); + Map updates = Maps.newHashMap(); + Set removals = Sets.newHashSet(); + + try { + Map oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace); + Map newProperties = + mergeComment(newDatabase.getProperties(), newDatabase.getComment()); + + for (String key : oldProperties.keySet()) { + if (!newProperties.containsKey(key)) { + removals.add(key); + } + } + + for (Map.Entry entry : newProperties.entrySet()) { + if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) { + updates.put(entry.getKey(), entry.getValue()); + } + } + + if (!updates.isEmpty()) { + asNamespaceCatalog.setProperties(namespace, updates); + } + + if (!removals.isEmpty()) { + asNamespaceCatalog.removeProperties(namespace, removals); + } + + } catch (NoSuchNamespaceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name, e); + } + } + } else { + if (getDefaultDatabase().equals(name)) { + throw new CatalogException( + "Can not alter the default database when the iceberg catalog doesn't support namespaces."); + } + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + try { + return icebergCatalog.listTables(appendLevel(baseNamespace, databaseName)).stream() + .map(TableIdentifier::name) + .collect(Collectors.toList()); + } catch (NoSuchNamespaceException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } + } + + @Override + public CatalogTable getTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + Table table = loadIcebergTable(tablePath); + return toCatalogTable(table); + } + + private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { + try { + Table table = icebergCatalog.loadTable(toIdentifier(tablePath)); + if (cacheEnabled) { + table.refresh(); + } + + return table; + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new TableNotExistException(getName(), tablePath, e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + return icebergCatalog.tableExists(toIdentifier(tablePath)); + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + try { + icebergCatalog.dropTable(toIdentifier(tablePath)); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + try { + icebergCatalog.renameTable( + toIdentifier(tablePath), + toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName))); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + if (Objects.equals( + table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { + throw new IllegalArgumentException( + "Cannot create the table with 'connector'='iceberg' table property in " + + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + + "create table without 'connector'='iceberg' related properties in an iceberg table."); + } + Preconditions.checkArgument(table instanceof ResolvedCatalogTable, "table should be resolved"); + createIcebergTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists); + } + + void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + validateFlinkTable(table); + + Schema icebergSchema = FlinkSchemaUtil.convert(table.getResolvedSchema()); + PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); + ImmutableMap.Builder properties = ImmutableMap.builder(); + String location = null; + for (Map.Entry entry : table.getOptions().entrySet()) { + if ("location".equalsIgnoreCase(entry.getKey())) { + location = entry.getValue(); + } else { + properties.put(entry.getKey(), entry.getValue()); + } + } + + try { + icebergCatalog.createTable( + toIdentifier(tablePath), icebergSchema, spec, location, properties.build()); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } + } + + private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) { + TableSchema ts1 = ct1.getSchema(); + TableSchema ts2 = ct2.getSchema(); + boolean equalsPrimary = false; + + if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { + equalsPrimary = + Objects.equals(ts1.getPrimaryKey().get().getType(), ts2.getPrimaryKey().get().getType()) + && Objects.equals( + ts1.getPrimaryKey().get().getColumns(), ts2.getPrimaryKey().get().getColumns()); + } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { + equalsPrimary = true; + } + + if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) + && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) + && equalsPrimary)) { + throw new UnsupportedOperationException( + "Altering schema is not supported in the old alterTable API. " + + "To alter schema, use the other alterTable API and provide a list of TableChange's."); + } + + validateTablePartition(ct1, ct2); + } + + private static void validateTablePartition(CatalogTable ct1, CatalogTable ct2) { + if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { + throw new UnsupportedOperationException("Altering partition keys is not supported yet."); + } + } + + /** + * This alterTable API only supports altering table properties. + * + *

Support for adding/removing/renaming columns cannot be done by comparing CatalogTable + * instances, unless the Flink schema contains Iceberg column IDs. + * + *

To alter columns, use the other alterTable API and provide a list of TableChange's. + * + * @param tablePath path of the table or view to be modified + * @param newTable the new table definition + * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if set + * to false, throw an exception, if set to true, do nothing. + * @throws CatalogException in case of any runtime exception + * @throws TableNotExistException if the table does not exist + */ + @Override + public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws CatalogException, TableNotExistException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + CatalogTable table = toCatalogTable(icebergTable); + validateTableSchemaAndPartition(table, (CatalogTable) newTable); + + Map oldProperties = table.getOptions(); + Map setProperties = Maps.newHashMap(); + + String setLocation = null; + String setSnapshotId = null; + String pickSnapshotId = null; + + for (Map.Entry entry : newTable.getOptions().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (Objects.equals(value, oldProperties.get(key))) { + continue; + } + + if ("location".equalsIgnoreCase(key)) { + setLocation = value; + } else if ("current-snapshot-id".equalsIgnoreCase(key)) { + setSnapshotId = value; + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) { + pickSnapshotId = value; + } else { + setProperties.put(key, value); + } + } + + oldProperties + .keySet() + .forEach( + k -> { + if (!newTable.getOptions().containsKey(k)) { + setProperties.put(k, null); + } + }); + + FlinkAlterTableUtil.commitChanges( + icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); + } + + @Override + public void alterTable( + ObjectPath tablePath, + CatalogBaseTable newTable, + List tableChanges, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + // Does not support altering partition yet. + validateTablePartition(toCatalogTable(icebergTable), (CatalogTable) newTable); + + String setLocation = null; + String setSnapshotId = null; + String cherrypickSnapshotId = null; + + List propertyChanges = Lists.newArrayList(); + List schemaChanges = Lists.newArrayList(); + for (TableChange change : tableChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption set = (TableChange.SetOption) change; + + if ("location".equalsIgnoreCase(set.getKey())) { + setLocation = set.getValue(); + } else if ("current-snapshot-id".equalsIgnoreCase(set.getKey())) { + setSnapshotId = set.getValue(); + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.getKey())) { + cherrypickSnapshotId = set.getValue(); + } else { + propertyChanges.add(change); + } + } else if (change instanceof TableChange.ResetOption) { + propertyChanges.add(change); + } else { + schemaChanges.add(change); + } + } + + FlinkAlterTableUtil.commitChanges( + icebergTable, + setLocation, + setSnapshotId, + cherrypickSnapshotId, + schemaChanges, + propertyChanges); + } + + private static void validateFlinkTable(CatalogBaseTable table) { + Preconditions.checkArgument( + table instanceof CatalogTable, "The Table should be a CatalogTable."); + + TableSchema schema = table.getSchema(); + schema + .getTableColumns() + .forEach( + column -> { + if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) { + throw new UnsupportedOperationException( + "Creating table with computed columns is not supported yet."); + } + }); + + if (!schema.getWatermarkSpecs().isEmpty()) { + throw new UnsupportedOperationException( + "Creating table with watermark specs is not supported yet."); + } + } + + private static PartitionSpec toPartitionSpec(List partitionKeys, Schema icebergSchema) { + PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema); + partitionKeys.forEach(builder::identity); + return builder.build(); + } + + private static List toPartitionKeys(PartitionSpec spec, Schema icebergSchema) { + ImmutableList.Builder partitionKeysBuilder = ImmutableList.builder(); + for (PartitionField field : spec.fields()) { + if (field.transform().isIdentity()) { + partitionKeysBuilder.add(icebergSchema.findColumnName(field.sourceId())); + } else { + // Not created by Flink SQL. + // For compatibility with iceberg tables, return empty. + // TODO modify this after Flink support partition transform. + return Collections.emptyList(); + } + } + return partitionKeysBuilder.build(); + } + + static CatalogTable toCatalogTable(Table table) { + TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); + List partitionKeys = toPartitionKeys(table.spec(), table.schema()); + + // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer + // may use + // CatalogTableImpl to copy a new catalog table. + // Let's re-loading table from Iceberg catalog when creating source/sink operators. + // Iceberg does not have Table comment, so pass a null (Default comment value in Flink). + return new CatalogTableImpl(schema, partitionKeys, table.properties(), null); + } + + @Override + public Optional getFactory() { + return Optional.of(new FlinkDynamicTableFactory(this)); + } + + CatalogLoader getCatalogLoader() { + return catalogLoader; + } + + // ------------------------------ Unsupported methods + // --------------------------------------------- + + @Override + public List listViews(String databaseName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listFunctions(String dbName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } + + @Override + public void createFunction( + ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterFunction( + ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + Table table = loadIcebergTable(tablePath); + + if (table.spec().isUnpartitioned()) { + throw new TableNotPartitionedException(icebergCatalog.name(), tablePath); + } + + Set set = Sets.newHashSet(); + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) { + Map map = Maps.newHashMap(); + StructLike structLike = dataFile.partition(); + PartitionSpec spec = table.specs().get(dataFile.specId()); + for (int i = 0; i < structLike.size(); i++) { + map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class))); + } + set.add(new CatalogPartitionSpec(map)); + } + } catch (IOException e) { + throw new CatalogException( + String.format("Failed to list partitions of table %s", tablePath), e); + } + + return Lists.newArrayList(set); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List filters) throws CatalogException { + throw new UnsupportedOperationException(); + } + + // After partition pruning and filter push down, the statistics have become very inaccurate, so + // the statistics from + // here are of little significance. + // Flink will support something like SupportsReportStatistics in future. + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java new file mode 100644 index 000000000000..1453753849ec --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -0,0 +1,213 @@ +/* + * 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; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; + +/** + * A Flink Catalog factory implementation that creates {@link FlinkCatalog}. + * + *

This supports the following catalog configuration options: + * + *

    + *
  • type - Flink catalog factory key, should be "iceberg" + *
  • catalog-type - iceberg catalog type, "hive", "hadoop" or "rest" + *
  • uri - the Hive Metastore URI (Hive catalog only) + *
  • clients - the Hive Client Pool Size (Hive catalog only) + *
  • warehouse - the warehouse path (Hadoop catalog only) + *
  • default-database - a database name to use as the default + *
  • base-namespace - a base namespace as the prefix for all databases (Hadoop + * catalog only) + *
  • cache-enabled - whether to enable catalog cache + *
+ * + *

To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override + * {@link #createCatalogLoader(String, Map, Configuration)}. + */ +public class FlinkCatalogFactory implements CatalogFactory { + + // Can not just use "type", it conflicts with CATALOG_TYPE. + public static final String ICEBERG_CATALOG_TYPE = "catalog-type"; + public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop"; + public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive"; + public static final String ICEBERG_CATALOG_TYPE_REST = "rest"; + + public static final String HIVE_CONF_DIR = "hive-conf-dir"; + public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; + public static final String DEFAULT_DATABASE = "default-database"; + public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String BASE_NAMESPACE = "base-namespace"; + + public static final String TYPE = "type"; + public static final String PROPERTY_VERSION = "property-version"; + + /** + * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink + * catalog adapter. + * + * @param name Flink's catalog name + * @param properties Flink's catalog properties + * @param hadoopConf Hadoop configuration for catalog + * @return an Iceberg catalog loader + */ + static CatalogLoader createCatalogLoader( + String name, Map properties, Configuration hadoopConf) { + String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); + if (catalogImpl != null) { + String catalogType = properties.get(ICEBERG_CATALOG_TYPE); + Preconditions.checkArgument( + catalogType == null, + "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s", + name, + catalogType, + catalogImpl); + return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl); + } + + String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE); + switch (catalogType.toLowerCase(Locale.ENGLISH)) { + case ICEBERG_CATALOG_TYPE_HIVE: + // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in + // that case it will + // fallback to parse those values from hadoop configuration which is loaded from classpath. + String hiveConfDir = properties.get(HIVE_CONF_DIR); + String hadoopConfDir = properties.get(HADOOP_CONF_DIR); + Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); + return CatalogLoader.hive(name, newHadoopConf, properties); + + case ICEBERG_CATALOG_TYPE_HADOOP: + return CatalogLoader.hadoop(name, hadoopConf, properties); + + case ICEBERG_CATALOG_TYPE_REST: + return CatalogLoader.rest(name, hadoopConf, properties); + + default: + throw new UnsupportedOperationException( + "Unknown catalog-type: " + catalogType + " (Must be 'hive', 'hadoop' or 'rest')"); + } + } + + @Override + public Map requiredContext() { + Map context = Maps.newHashMap(); + context.put(TYPE, "iceberg"); + context.put(PROPERTY_VERSION, "1"); + return context; + } + + @Override + public List supportedProperties() { + return ImmutableList.of("*"); + } + + @Override + public Catalog createCatalog(String name, Map properties) { + return createCatalog(name, properties, clusterHadoopConf()); + } + + protected Catalog createCatalog( + String name, Map properties, Configuration hadoopConf) { + CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); + String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); + + Namespace baseNamespace = Namespace.empty(); + if (properties.containsKey(BASE_NAMESPACE)) { + baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\.")); + } + + boolean cacheEnabled = + PropertyUtil.propertyAsBoolean( + properties, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); + + long cacheExpirationIntervalMs = + PropertyUtil.propertyAsLong( + properties, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); + Preconditions.checkArgument( + cacheExpirationIntervalMs != 0, + "%s is not allowed to be 0.", + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS); + + return new FlinkCatalog( + name, + defaultDatabase, + baseNamespace, + catalogLoader, + cacheEnabled, + cacheExpirationIntervalMs); + } + + private static Configuration mergeHiveConf( + Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { + Configuration newConf = new Configuration(hadoopConf); + if (!Strings.isNullOrEmpty(hiveConfDir)) { + Preconditions.checkState( + Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), + "There should be a hive-site.xml file under the directory %s", + hiveConfDir); + newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); + } else { + // If don't provide the hive-site.xml path explicitly, it will try to load resource from + // classpath. If still + // couldn't load the configuration file, then it will throw exception in HiveCatalog. + URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); + if (configFile != null) { + newConf.addResource(configFile); + } + } + + if (!Strings.isNullOrEmpty(hadoopConfDir)) { + Preconditions.checkState( + Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), + "Failed to load Hadoop configuration: missing %s", + Paths.get(hadoopConfDir, "hdfs-site.xml")); + newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); + Preconditions.checkState( + Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), + "Failed to load Hadoop configuration: missing %s", + Paths.get(hadoopConfDir, "core-site.xml")); + newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); + } + + return newConf; + } + + public static Configuration clusterHadoopConf() { + return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java new file mode 100644 index 000000000000..7167859e600c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -0,0 +1,261 @@ +/* + * 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; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class FlinkConfParser { + + private final Map tableProperties; + private final Map options; + private final ReadableConfig readableConfig; + + FlinkConfParser(Table table, Map options, ReadableConfig readableConfig) { + this.tableProperties = table.properties(); + this.options = options; + this.readableConfig = readableConfig; + } + + public BooleanConfParser booleanConf() { + return new BooleanConfParser(); + } + + public IntConfParser intConf() { + return new IntConfParser(); + } + + public LongConfParser longConf() { + return new LongConfParser(); + } + + public > EnumConfParser enumConfParser(Class enumClass) { + return new EnumConfParser<>(enumClass); + } + + public StringConfParser stringConf() { + return new StringConfParser(); + } + + public DurationConfParser durationConf() { + return new DurationConfParser(); + } + + class BooleanConfParser extends ConfParser { + private Boolean defaultValue; + + @Override + protected BooleanConfParser self() { + return this; + } + + public BooleanConfParser defaultValue(boolean value) { + this.defaultValue = value; + return self(); + } + + public BooleanConfParser defaultValue(String value) { + this.defaultValue = Boolean.parseBoolean(value); + return self(); + } + + public boolean parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Boolean::parseBoolean, defaultValue); + } + } + + class IntConfParser extends ConfParser { + private Integer defaultValue; + + @Override + protected IntConfParser self() { + return this; + } + + public IntConfParser defaultValue(int value) { + this.defaultValue = value; + return self(); + } + + public int parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Integer::parseInt, defaultValue); + } + + public Integer parseOptional() { + return parse(Integer::parseInt, null); + } + } + + class LongConfParser extends ConfParser { + private Long defaultValue; + + @Override + protected LongConfParser self() { + return this; + } + + public LongConfParser defaultValue(long value) { + this.defaultValue = value; + return self(); + } + + public long parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Long::parseLong, defaultValue); + } + + public Long parseOptional() { + return parse(Long::parseLong, null); + } + } + + class StringConfParser extends ConfParser { + private String defaultValue; + + @Override + protected StringConfParser self() { + return this; + } + + public StringConfParser defaultValue(String value) { + this.defaultValue = value; + return self(); + } + + public String parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Function.identity(), defaultValue); + } + + public String parseOptional() { + return parse(Function.identity(), null); + } + } + + class EnumConfParser> extends ConfParser, E> { + private E defaultValue; + private final Class enumClass; + + EnumConfParser(Class enumClass) { + this.enumClass = enumClass; + } + + @Override + protected EnumConfParser self() { + return this; + } + + public EnumConfParser defaultValue(E value) { + this.defaultValue = value; + return self(); + } + + public E parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(s -> Enum.valueOf(enumClass, s), defaultValue); + } + + public E parseOptional() { + return parse(s -> Enum.valueOf(enumClass, s), null); + } + } + + class DurationConfParser extends ConfParser { + private Duration defaultValue; + + @Override + protected DurationConfParser self() { + return this; + } + + public DurationConfParser defaultValue(Duration value) { + this.defaultValue = value; + return self(); + } + + public Duration parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(TimeUtils::parseDuration, defaultValue); + } + + public Duration parseOptional() { + return parse(TimeUtils::parseDuration, null); + } + } + + abstract class ConfParser { + private final List optionNames = Lists.newArrayList(); + private String tablePropertyName; + private ConfigOption configOption; + + protected abstract ThisT self(); + + public ThisT option(String name) { + this.optionNames.add(name); + return self(); + } + + public ThisT flinkConfig(ConfigOption newConfigOption) { + this.configOption = newConfigOption; + return self(); + } + + public ThisT tableProperty(String name) { + this.tablePropertyName = name; + return self(); + } + + protected T parse(Function conversion, T defaultValue) { + if (!optionNames.isEmpty()) { + for (String optionName : optionNames) { + String optionValue = options.get(optionName); + if (optionValue != null) { + return conversion.apply(optionValue); + } + } + } + + if (configOption != null) { + T propertyValue = readableConfig.get(configOption); + if (propertyValue != null) { + return propertyValue; + } + } + + if (tablePropertyName != null) { + String propertyValue = tableProperties.get(tablePropertyName); + if (propertyValue != null) { + return conversion.apply(propertyValue); + } + } + + return defaultValue; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java new file mode 100644 index 000000000000..7c7afd24ed8e --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.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.flink; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.TextElement; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.flink.source.assigner.SplitAssignerType; +import org.apache.iceberg.util.ThreadPools; + +/** + * When constructing Flink Iceberg source via Java API, configs can be set in {@link Configuration} + * passed to source builder. E.g. + * + *

+ *   configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
+ *   FlinkSource.forRowData()
+ *       .flinkConf(configuration)
+ *       ...
+ * 
+ * + *

When using Flink SQL/table API, connector options can be set in Flink's {@link + * TableEnvironment}. + * + *

+ *   TableEnvironment tEnv = createTableEnv();
+ *   tEnv.getConfig()
+ *        .getConfiguration()
+ *        .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
+ * 
+ */ +public class FlinkConfigOptions { + + private FlinkConfigOptions() {} + + public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = + ConfigOptions.key("table.exec.iceberg.infer-source-parallelism") + .booleanType() + .defaultValue(true) + .withDescription( + "If is false, parallelism of source are set by config.\n" + + "If is true, source parallelism is inferred according to splits number.\n"); + + public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX = + ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max") + .intType() + .defaultValue(100) + .withDescription("Sets max infer parallelism for source operator."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = + ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") + .booleanType() + .noDefaultValue() + .withDescription( + "Expose split host information to use Flink's locality aware split assigner."); + + public static final ConfigOption SOURCE_READER_FETCH_BATCH_RECORD_COUNT = + ConfigOptions.key("table.exec.iceberg.fetch-batch-record-count") + .intType() + .defaultValue(2048) + .withDescription("The target number of records for Iceberg reader fetch batch."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE = + ConfigOptions.key("table.exec.iceberg.worker-pool-size") + .intType() + .defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE) + .withDescription("The size of workers pool used to plan or scan manifests."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = + ConfigOptions.key("table.exec.iceberg.use-flip27-source") + .booleanType() + .defaultValue(false) + .withDescription("Use the FLIP-27 based Iceberg source implementation."); + + public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = + ConfigOptions.key("table.exec.iceberg.split-assigner-type") + .enumType(SplitAssignerType.class) + .defaultValue(SplitAssignerType.SIMPLE) + .withDescription( + Description.builder() + .text("Split assigner type that determine how splits are assigned to readers.") + .linebreak() + .list( + TextElement.text( + SplitAssignerType.SIMPLE + + ": simple assigner that doesn't provide any guarantee on order or locality.")) + .build()); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java new file mode 100644 index 000000000000..b7f1be4b93fb --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.util.Map; +import java.util.Set; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.flink.source.IcebergTableSource; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +public class FlinkDynamicTableFactory + implements DynamicTableSinkFactory, DynamicTableSourceFactory { + static final String FACTORY_IDENTIFIER = "iceberg"; + + private static final ConfigOption CATALOG_NAME = + ConfigOptions.key("catalog-name") + .stringType() + .noDefaultValue() + .withDescription("Catalog name"); + + private static final ConfigOption CATALOG_TYPE = + ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) + .stringType() + .noDefaultValue() + .withDescription("Catalog type, the optional types are: custom, hadoop, hive."); + + private static final ConfigOption CATALOG_DATABASE = + ConfigOptions.key("catalog-database") + .stringType() + .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) + .withDescription("Database name managed in the iceberg catalog."); + + private static final ConfigOption CATALOG_TABLE = + ConfigOptions.key("catalog-table") + .stringType() + .noDefaultValue() + .withDescription("Table name managed in the underlying iceberg catalog and database."); + + private final FlinkCatalog catalog; + + public FlinkDynamicTableFactory() { + this.catalog = null; + } + + public FlinkDynamicTableFactory(FlinkCatalog catalog) { + this.catalog = catalog; + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map tableProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = + createTableLoader( + resolvedCatalogTable, + tableProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } + + return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map writeProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = + createTableLoader( + resolvedCatalogTable, + writeProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } + + return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); + } + + @Override + public Set> requiredOptions() { + Set> options = Sets.newHashSet(); + options.add(CATALOG_TYPE); + options.add(CATALOG_NAME); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = Sets.newHashSet(); + options.add(CATALOG_DATABASE); + options.add(CATALOG_TABLE); + return options; + } + + @Override + public String factoryIdentifier() { + return FACTORY_IDENTIFIER; + } + + private static TableLoader createTableLoader( + ResolvedCatalogTable resolvedCatalogTable, + Map tableProps, + String databaseName, + String tableName) { + Configuration flinkConf = new Configuration(); + tableProps.forEach(flinkConf::setString); + + String catalogName = flinkConf.getString(CATALOG_NAME); + Preconditions.checkNotNull( + catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); + + String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); + Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); + + String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); + Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); + + org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); + FlinkCatalogFactory factory = new FlinkCatalogFactory(); + FlinkCatalog flinkCatalog = + (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); + ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); + + // Create database if not exists in the external catalog. + if (!flinkCatalog.databaseExists(catalogDatabase)) { + try { + flinkCatalog.createDatabase( + catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true); + } catch (DatabaseAlreadyExistException e) { + throw new AlreadyExistsException( + e, + "Database %s already exists in the iceberg catalog %s.", + catalogName, + catalogDatabase); + } + } + + // Create table if not exists in the external catalog. + if (!flinkCatalog.tableExists(objectPath)) { + try { + flinkCatalog.createIcebergTable(objectPath, resolvedCatalogTable, true); + } catch (TableAlreadyExistException e) { + throw new AlreadyExistsException( + e, + "Table %s already exists in the database %s and catalog %s", + catalogTable, + catalogDatabase, + catalogName); + } + } + + return TableLoader.fromCatalog( + flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); + } + + private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { + Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); + return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java new file mode 100644 index 000000000000..f35bb577fbba --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java @@ -0,0 +1,31 @@ +/* + * 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; + +import org.apache.iceberg.EnvironmentContext; +import org.apache.iceberg.flink.util.FlinkPackage; + +class FlinkEnvironmentContext { + private FlinkEnvironmentContext() {} + + public static void init() { + EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "flink"); + EnvironmentContext.put(EnvironmentContext.ENGINE_VERSION, FlinkPackage.version()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java new file mode 100644 index 000000000000..f2244d5137a1 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java @@ -0,0 +1,266 @@ +/* + * 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; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.NaNUtil; + +public class FlinkFilters { + private FlinkFilters() {} + + private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); + + private static final Map FILTERS = + ImmutableMap.builder() + .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) + .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) + .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) + .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) + .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) + .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) + .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) + .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) + .put(BuiltInFunctionDefinitions.AND, Operation.AND) + .put(BuiltInFunctionDefinitions.OR, Operation.OR) + .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) + .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) + .buildOrThrow(); + + /** + * Convert flink expression to iceberg expression. + * + *

the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the + * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR + * GT_EQ), the IN will be converted to OR, so we do not add the conversion here + * + * @param flinkExpression the flink expression + * @return the iceberg expression + */ + public static Optional convert( + org.apache.flink.table.expressions.Expression flinkExpression) { + if (!(flinkExpression instanceof CallExpression)) { + return Optional.empty(); + } + + CallExpression call = (CallExpression) flinkExpression; + Operation op = FILTERS.get(call.getFunctionDefinition()); + if (op != null) { + switch (op) { + case IS_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(Expressions::isNull); + + case NOT_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(Expressions::notNull); + + case LT: + return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call); + + case LT_EQ: + return convertFieldAndLiteral( + Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call); + + case GT: + return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call); + + case GT_EQ: + return convertFieldAndLiteral( + Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call); + + case EQ: + return convertFieldAndLiteral( + (ref, lit) -> { + if (NaNUtil.isNaN(lit)) { + return Expressions.isNaN(ref); + } else { + return Expressions.equal(ref, lit); + } + }, + call); + + case NOT_EQ: + return convertFieldAndLiteral( + (ref, lit) -> { + if (NaNUtil.isNaN(lit)) { + return Expressions.notNaN(ref); + } else { + return Expressions.notEqual(ref, lit); + } + }, + call); + + case NOT: + return onlyChildAs(call, CallExpression.class) + .flatMap(FlinkFilters::convert) + .map(Expressions::not); + + case AND: + return convertLogicExpression(Expressions::and, call); + + case OR: + return convertLogicExpression(Expressions::or, call); + + case STARTS_WITH: + return convertLike(call); + } + } + + return Optional.empty(); + } + + private static Optional onlyChildAs( + CallExpression call, Class expectedChildClass) { + List children = call.getResolvedChildren(); + if (children.size() != 1) { + return Optional.empty(); + } + + ResolvedExpression child = children.get(0); + if (!expectedChildClass.isInstance(child)) { + return Optional.empty(); + } + + return Optional.of(expectedChildClass.cast(child)); + } + + private static Optional convertLike(CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + org.apache.flink.table.expressions.Expression left = args.get(0); + org.apache.flink.table.expressions.Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + return convertLiteral((ValueLiteralExpression) right) + .flatMap( + lit -> { + if (lit instanceof String) { + String pattern = (String) lit; + Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); + // exclude special char of LIKE + // '_' is the wildcard of the SQL LIKE + if (!pattern.contains("_") && matcher.matches()) { + return Optional.of(Expressions.startsWith(name, matcher.group(1))); + } + } + + return Optional.empty(); + }); + } + + return Optional.empty(); + } + + private static Optional convertLogicExpression( + BiFunction function, CallExpression call) { + List args = call.getResolvedChildren(); + if (args == null || args.size() != 2) { + return Optional.empty(); + } + + Optional left = convert(args.get(0)); + Optional right = convert(args.get(1)); + if (left.isPresent() && right.isPresent()) { + return Optional.of(function.apply(left.get(), right.get())); + } + + return Optional.empty(); + } + + private static Optional convertLiteral(ValueLiteralExpression expression) { + Optional value = + expression.getValueAs( + expression.getOutputDataType().getLogicalType().getDefaultConversion()); + return value.map( + o -> { + if (o instanceof LocalDateTime) { + return DateTimeUtil.microsFromTimestamp((LocalDateTime) o); + } else if (o instanceof Instant) { + return DateTimeUtil.microsFromInstant((Instant) o); + } else if (o instanceof LocalTime) { + return DateTimeUtil.microsFromTime((LocalTime) o); + } else if (o instanceof LocalDate) { + return DateTimeUtil.daysFromDate((LocalDate) o); + } + + return o; + }); + } + + private static Optional convertFieldAndLiteral( + BiFunction expr, CallExpression call) { + return convertFieldAndLiteral(expr, expr, call); + } + + private static Optional convertFieldAndLiteral( + BiFunction convertLR, + BiFunction convertRL, + CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + org.apache.flink.table.expressions.Expression left = args.get(0); + org.apache.flink.table.expressions.Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + Optional lit = convertLiteral((ValueLiteralExpression) right); + if (lit.isPresent()) { + return Optional.of(convertLR.apply(name, lit.get())); + } + } else if (left instanceof ValueLiteralExpression + && right instanceof FieldReferenceExpression) { + Optional lit = convertLiteral((ValueLiteralExpression) left); + String name = ((FieldReferenceExpression) right).getName(); + if (lit.isPresent()) { + return Optional.of(convertRL.apply(name, lit.get())); + } + } + + return Optional.empty(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java new file mode 100644 index 000000000000..767d4497ac91 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.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.flink; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.FixupTypes; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, + * which may not be correct. + */ +class FlinkFixupTypes extends FixupTypes { + + private FlinkFixupTypes(Schema referenceSchema) { + super(referenceSchema); + } + + static Schema fixup(Schema schema, Schema referenceSchema) { + return new Schema( + TypeUtil.visit(schema, new FlinkFixupTypes(referenceSchema)).asStructType().fields()); + } + + @Override + protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { + if (type instanceof Types.FixedType) { + int length = ((Types.FixedType) type).length(); + return source.typeId() == Type.TypeID.UUID && length == 16; + } + return false; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java new file mode 100644 index 000000000000..804a956ec9b9 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java @@ -0,0 +1,213 @@ +/* + * 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; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; + +public class FlinkReadConf { + + private final FlinkConfParser confParser; + + public FlinkReadConf( + Table table, Map readOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(table, readOptions, readableConfig); + } + + public Long snapshotId() { + return confParser.longConf().option(FlinkReadOptions.SNAPSHOT_ID.key()).parseOptional(); + } + + public String tag() { + return confParser.stringConf().option(FlinkReadOptions.TAG.key()).parseOptional(); + } + + public String startTag() { + return confParser.stringConf().option(FlinkReadOptions.START_TAG.key()).parseOptional(); + } + + public String endTag() { + return confParser.stringConf().option(FlinkReadOptions.END_TAG.key()).parseOptional(); + } + + public String branch() { + return confParser.stringConf().option(FlinkReadOptions.BRANCH.key()).parseOptional(); + } + + public boolean caseSensitive() { + return confParser + .booleanConf() + .option(FlinkReadOptions.CASE_SENSITIVE) + .flinkConfig(FlinkReadOptions.CASE_SENSITIVE_OPTION) + .defaultValue(FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue()) + .parse(); + } + + public Long asOfTimestamp() { + return confParser.longConf().option(FlinkReadOptions.AS_OF_TIMESTAMP.key()).parseOptional(); + } + + public StreamingStartingStrategy startingStrategy() { + return confParser + .enumConfParser(StreamingStartingStrategy.class) + .option(FlinkReadOptions.STARTING_STRATEGY) + .flinkConfig(FlinkReadOptions.STARTING_STRATEGY_OPTION) + .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .parse(); + } + + public Long startSnapshotTimestamp() { + return confParser + .longConf() + .option(FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key()) + .parseOptional(); + } + + public Long startSnapshotId() { + return confParser.longConf().option(FlinkReadOptions.START_SNAPSHOT_ID.key()).parseOptional(); + } + + public Long endSnapshotId() { + return confParser.longConf().option(FlinkReadOptions.END_SNAPSHOT_ID.key()).parseOptional(); + } + + public long splitSize() { + return confParser + .longConf() + .option(FlinkReadOptions.SPLIT_SIZE) + .flinkConfig(FlinkReadOptions.SPLIT_SIZE_OPTION) + .tableProperty(TableProperties.SPLIT_SIZE) + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) + .parse(); + } + + public int splitLookback() { + return confParser + .intConf() + .option(FlinkReadOptions.SPLIT_LOOKBACK) + .flinkConfig(FlinkReadOptions.SPLIT_LOOKBACK_OPTION) + .tableProperty(TableProperties.SPLIT_LOOKBACK) + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) + .parse(); + } + + public long splitFileOpenCost() { + return confParser + .longConf() + .option(FlinkReadOptions.SPLIT_FILE_OPEN_COST) + .flinkConfig(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION) + .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) + .parse(); + } + + public boolean streaming() { + return confParser + .booleanConf() + .option(FlinkReadOptions.STREAMING) + .flinkConfig(FlinkReadOptions.STREAMING_OPTION) + .defaultValue(FlinkReadOptions.STREAMING_OPTION.defaultValue()) + .parse(); + } + + public Duration monitorInterval() { + String duration = + confParser + .stringConf() + .option(FlinkReadOptions.MONITOR_INTERVAL) + .flinkConfig(FlinkReadOptions.MONITOR_INTERVAL_OPTION) + .defaultValue(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()) + .parse(); + + return TimeUtils.parseDuration(duration); + } + + public boolean includeColumnStats() { + return confParser + .booleanConf() + .option(FlinkReadOptions.INCLUDE_COLUMN_STATS) + .flinkConfig(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION) + .defaultValue(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue()) + .parse(); + } + + public int maxPlanningSnapshotCount() { + return confParser + .intConf() + .option(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT) + .flinkConfig(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION) + .defaultValue(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue()) + .parse(); + } + + public String nameMapping() { + return confParser.stringConf().option(TableProperties.DEFAULT_NAME_MAPPING).parseOptional(); + } + + public long limit() { + return confParser + .longConf() + .option(FlinkReadOptions.LIMIT) + .flinkConfig(FlinkReadOptions.LIMIT_OPTION) + .defaultValue(FlinkReadOptions.LIMIT_OPTION.defaultValue()) + .parse(); + } + + public int workerPoolSize() { + return confParser + .intConf() + .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) + .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) + .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) + .parse(); + } + + public int maxAllowedPlanningFailures() { + return confParser + .intConf() + .option(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES) + .flinkConfig(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION) + .defaultValue(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue()) + .parse(); + } + + public String watermarkColumn() { + return confParser + .stringConf() + .option(FlinkReadOptions.WATERMARK_COLUMN) + .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_OPTION) + .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue()) + .parseOptional(); + } + + public TimeUnit watermarkColumnTimeUnit() { + return confParser + .enumConfParser(TimeUnit.class) + .option(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT) + .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION) + .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue()) + .parse(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java new file mode 100644 index 000000000000..1bbd88146c8f --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.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.flink; + +import java.util.concurrent.TimeUnit; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; + +/** Flink source read options */ +public class FlinkReadOptions { + private static final String PREFIX = "connector.iceberg."; + + private FlinkReadOptions() {} + + public static final ConfigOption SNAPSHOT_ID = + ConfigOptions.key("snapshot-id").longType().defaultValue(null); + + public static final ConfigOption TAG = + ConfigOptions.key("tag").stringType().defaultValue(null); + + public static final ConfigOption BRANCH = + ConfigOptions.key("branch").stringType().defaultValue(null); + + public static final ConfigOption START_TAG = + ConfigOptions.key("start-tag").stringType().defaultValue(null); + + public static final ConfigOption END_TAG = + ConfigOptions.key("end-tag").stringType().defaultValue(null); + + public static final String CASE_SENSITIVE = "case-sensitive"; + public static final ConfigOption CASE_SENSITIVE_OPTION = + ConfigOptions.key(PREFIX + CASE_SENSITIVE).booleanType().defaultValue(false); + + public static final ConfigOption AS_OF_TIMESTAMP = + ConfigOptions.key("as-of-timestamp").longType().defaultValue(null); + + public static final String STARTING_STRATEGY = "starting-strategy"; + public static final ConfigOption STARTING_STRATEGY_OPTION = + ConfigOptions.key(PREFIX + STARTING_STRATEGY) + .enumType(StreamingStartingStrategy.class) + .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT); + + public static final ConfigOption START_SNAPSHOT_TIMESTAMP = + ConfigOptions.key("start-snapshot-timestamp").longType().defaultValue(null); + + public static final ConfigOption START_SNAPSHOT_ID = + ConfigOptions.key("start-snapshot-id").longType().defaultValue(null); + + public static final ConfigOption END_SNAPSHOT_ID = + ConfigOptions.key("end-snapshot-id").longType().defaultValue(null); + + public static final String SPLIT_SIZE = "split-size"; + public static final ConfigOption SPLIT_SIZE_OPTION = + ConfigOptions.key(PREFIX + SPLIT_SIZE) + .longType() + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT); + + public static final String SPLIT_LOOKBACK = "split-lookback"; + public static final ConfigOption SPLIT_LOOKBACK_OPTION = + ConfigOptions.key(PREFIX + SPLIT_LOOKBACK) + .intType() + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT); + + public static final String SPLIT_FILE_OPEN_COST = "split-file-open-cost"; + public static final ConfigOption SPLIT_FILE_OPEN_COST_OPTION = + ConfigOptions.key(PREFIX + SPLIT_FILE_OPEN_COST) + .longType() + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + + public static final String STREAMING = "streaming"; + public static final ConfigOption STREAMING_OPTION = + ConfigOptions.key(PREFIX + STREAMING).booleanType().defaultValue(false); + + public static final String MONITOR_INTERVAL = "monitor-interval"; + public static final ConfigOption MONITOR_INTERVAL_OPTION = + ConfigOptions.key(PREFIX + MONITOR_INTERVAL).stringType().defaultValue("60s"); + + public static final String INCLUDE_COLUMN_STATS = "include-column-stats"; + public static final ConfigOption INCLUDE_COLUMN_STATS_OPTION = + ConfigOptions.key(PREFIX + INCLUDE_COLUMN_STATS).booleanType().defaultValue(false); + + public static final String MAX_PLANNING_SNAPSHOT_COUNT = "max-planning-snapshot-count"; + public static final ConfigOption MAX_PLANNING_SNAPSHOT_COUNT_OPTION = + ConfigOptions.key(PREFIX + MAX_PLANNING_SNAPSHOT_COUNT) + .intType() + .defaultValue(Integer.MAX_VALUE); + + public static final String LIMIT = "limit"; + public static final ConfigOption LIMIT_OPTION = + ConfigOptions.key(PREFIX + LIMIT).longType().defaultValue(-1L); + + public static final String MAX_ALLOWED_PLANNING_FAILURES = "max-allowed-planning-failures"; + public static final ConfigOption MAX_ALLOWED_PLANNING_FAILURES_OPTION = + ConfigOptions.key(PREFIX + MAX_ALLOWED_PLANNING_FAILURES).intType().defaultValue(3); + + public static final String WATERMARK_COLUMN = "watermark-column"; + public static final ConfigOption WATERMARK_COLUMN_OPTION = + ConfigOptions.key(PREFIX + WATERMARK_COLUMN).stringType().noDefaultValue(); + + public static final String WATERMARK_COLUMN_TIME_UNIT = "watermark-column-time-unit"; + public static final ConfigOption WATERMARK_COLUMN_TIME_UNIT_OPTION = + ConfigOptions.key(PREFIX + WATERMARK_COLUMN_TIME_UNIT) + .enumType(TimeUnit.class) + .defaultValue(TimeUnit.MICROSECONDS); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java new file mode 100644 index 000000000000..4790dc85bf28 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -0,0 +1,232 @@ +/* + * 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; + +import java.util.List; +import java.util.Set; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.Schema; +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.Sets; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * Converter between Flink types and Iceberg type. The conversion is not a 1:1 mapping that not + * allows back-and-forth conversion. So some information might get lost during the back-and-forth + * conversion. + * + *

This inconsistent types: + * + *

    + *
  • map Iceberg UUID type to Flink BinaryType(16) + *
  • map Flink VarCharType(_) and CharType(_) to Iceberg String type + *
  • map Flink VarBinaryType(_) to Iceberg Binary type + *
  • map Flink TimeType(_) to Iceberg Time type (microseconds) + *
  • map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds) + *
  • map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds) + *
  • map Flink MultiSetType to Iceberg Map type(element, int) + *
+ * + *

+ */ +public class FlinkSchemaUtil { + + private FlinkSchemaUtil() {} + + /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + @Deprecated + public static Schema convert(TableSchema schema) { + LogicalType schemaType = schema.toRowDataType().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be row type."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (schema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, schema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } + } + + /** Convert the flink table schema to apache iceberg schema with column comment. */ + public static Schema convert(ResolvedSchema flinkSchema) { + List tableColumns = flinkSchema.getColumns(); + // copy from org.apache.flink.table.api.Schema#toRowDataType + DataTypes.Field[] fields = + tableColumns.stream() + .map( + column -> { + if (column.getComment().isPresent()) { + return DataTypes.FIELD( + column.getName(), column.getDataType(), column.getComment().get()); + } else { + return DataTypes.FIELD(column.getName(), column.getDataType()); + } + }) + .toArray(DataTypes.Field[]::new); + + LogicalType schemaType = DataTypes.ROW(fields).notNull().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be row type."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } + } + + private static Schema freshIdentifierFieldIds(Schema icebergSchema, List primaryKeys) { + // Locate the identifier field id list. + Set identifierFieldIds = Sets.newHashSet(); + for (String primaryKey : primaryKeys) { + Types.NestedField field = icebergSchema.findField(primaryKey); + Preconditions.checkNotNull( + field, + "Cannot find field ID for the primary key column %s in schema %s", + primaryKey, + icebergSchema); + identifierFieldIds.add(field.fieldId()); + } + return new Schema( + icebergSchema.schemaId(), icebergSchema.asStruct().fields(), identifierFieldIds); + } + + /** + * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema. + * + *

This conversion does not assign new ids; it uses ids from the base schema. + * + *

Data types, field order, and nullability will match the Flink type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param flinkSchema a Flink TableSchema + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + */ + public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { + // convert to a type with fresh ids + Types.StructType struct = convert(flinkSchema).asStruct(); + // reassign ids to match the base schema + Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema); + // reassign doc to match the base schema + schema = TypeUtil.reassignDoc(schema, baseSchema); + + // fix types that can't be represented in Flink (UUID) + Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(fixedSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return fixedSchema; + } + } + + /** + * Convert a {@link Schema} to a {@link RowType Flink type}. + * + * @param schema a Schema + * @return the equivalent Flink type + * @throws IllegalArgumentException if the type cannot be converted to Flink + */ + public static RowType convert(Schema schema) { + return (RowType) TypeUtil.visit(schema, new TypeToFlinkType()); + } + + /** + * Convert a {@link Type} to a {@link LogicalType Flink type}. + * + * @param type a Type + * @return the equivalent Flink type + * @throws IllegalArgumentException if the type cannot be converted to Flink + */ + public static LogicalType convert(Type type) { + return TypeUtil.visit(type, new TypeToFlinkType()); + } + + /** + * Convert a {@link LogicalType Flink type} to a {@link Type}. + * + * @param flinkType a FlinkType + * @return the equivalent Iceberg type + */ + public static Type convert(LogicalType flinkType) { + return flinkType.accept(new FlinkTypeToType()); + } + + /** + * Convert a {@link RowType} to a {@link TableSchema}. + * + * @param rowType a RowType + * @return Flink TableSchema + */ + public static TableSchema toSchema(RowType rowType) { + TableSchema.Builder builder = TableSchema.builder(); + for (RowType.RowField field : rowType.getFields()) { + builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); + } + return builder.build(); + } + + /** + * Convert a {@link Schema} to a {@link TableSchema}. + * + * @param schema iceberg schema to convert. + * @return Flink TableSchema. + */ + public static TableSchema toSchema(Schema schema) { + TableSchema.Builder builder = TableSchema.builder(); + + // Add columns. + for (RowType.RowField field : convert(schema).getFields()) { + builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); + } + + // Add primary key. + Set identifierFieldIds = schema.identifierFieldIds(); + if (!identifierFieldIds.isEmpty()) { + List columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size()); + for (Integer identifierFieldId : identifierFieldIds) { + String columnName = schema.findColumnName(identifierFieldId); + Preconditions.checkNotNull( + columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema); + + columns.add(columnName); + } + builder.primaryKey(columns.toArray(new String[0])); + } + + return builder.build(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java new file mode 100644 index 000000000000..5fbd84909d69 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java @@ -0,0 +1,49 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Types; + +public class FlinkSourceFilter implements FilterFunction { + + private final RowType rowType; + private final Evaluator evaluator; + private final Types.StructType struct; + private volatile RowDataWrapper wrapper; + + public FlinkSourceFilter(Schema schema, Expression expr, boolean caseSensitive) { + this.rowType = FlinkSchemaUtil.convert(schema); + this.struct = schema.asStruct(); + this.evaluator = new Evaluator(struct, expr, caseSensitive); + } + + @Override + public boolean filter(RowData value) { + if (wrapper == null) { + this.wrapper = new RowDataWrapper(rowType, struct); + } + return evaluator.eval(wrapper.wrap(value)); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java new file mode 100644 index 000000000000..408065f06057 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java @@ -0,0 +1,203 @@ +/* + * 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; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class FlinkTypeToType extends FlinkTypeVisitor { + + private final RowType root; + private int nextId; + + FlinkTypeToType() { + this.root = null; + } + + FlinkTypeToType(RowType root) { + this.root = root; + // the root struct's fields use the first ids + this.nextId = root.getFieldCount(); + } + + private int getNextId() { + int next = nextId; + nextId += 1; + return next; + } + + @Override + public Type visit(CharType charType) { + return Types.StringType.get(); + } + + @Override + public Type visit(VarCharType varCharType) { + return Types.StringType.get(); + } + + @Override + public Type visit(BooleanType booleanType) { + return Types.BooleanType.get(); + } + + @Override + public Type visit(BinaryType binaryType) { + return Types.FixedType.ofLength(binaryType.getLength()); + } + + @Override + public Type visit(VarBinaryType varBinaryType) { + return Types.BinaryType.get(); + } + + @Override + public Type visit(DecimalType decimalType) { + return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public Type visit(TinyIntType tinyIntType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(SmallIntType smallIntType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(IntType intType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(BigIntType bigIntType) { + return Types.LongType.get(); + } + + @Override + public Type visit(FloatType floatType) { + return Types.FloatType.get(); + } + + @Override + public Type visit(DoubleType doubleType) { + return Types.DoubleType.get(); + } + + @Override + public Type visit(DateType dateType) { + return Types.DateType.get(); + } + + @Override + public Type visit(TimeType timeType) { + return Types.TimeType.get(); + } + + @Override + public Type visit(TimestampType timestampType) { + return Types.TimestampType.withoutZone(); + } + + @Override + public Type visit(LocalZonedTimestampType localZonedTimestampType) { + return Types.TimestampType.withZone(); + } + + @Override + public Type visit(ArrayType arrayType) { + Type elementType = arrayType.getElementType().accept(this); + if (arrayType.getElementType().isNullable()) { + return Types.ListType.ofOptional(getNextId(), elementType); + } else { + return Types.ListType.ofRequired(getNextId(), elementType); + } + } + + @Override + public Type visit(MultisetType multisetType) { + Type elementType = multisetType.getElementType().accept(this); + return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get()); + } + + @Override + public Type visit(MapType mapType) { + // keys in map are not allowed to be null. + Type keyType = mapType.getKeyType().accept(this); + Type valueType = mapType.getValueType().accept(this); + if (mapType.getValueType().isNullable()) { + return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); + } else { + return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); + } + } + + @Override + @SuppressWarnings("ReferenceEquality") + public Type visit(RowType rowType) { + List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); + boolean isRoot = root == rowType; + + List types = + rowType.getFields().stream() + .map(f -> f.getType().accept(this)) + .collect(Collectors.toList()); + + for (int i = 0; i < rowType.getFieldCount(); i++) { + int id = isRoot ? i : getNextId(); + + RowType.RowField field = rowType.getFields().get(i); + String name = field.getName(); + String comment = field.getDescription().orElse(null); + + if (field.getType().isNullable()) { + newFields.add(Types.NestedField.optional(id, name, types.get(i), comment)); + } else { + newFields.add(Types.NestedField.required(id, name, types.get(i), comment)); + } + } + + return Types.StructType.of(newFields); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java new file mode 100644 index 000000000000..f3de2416088c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.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; + +import org.apache.flink.table.types.logical.DayTimeIntervalType; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeVisitor; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RawType; +import org.apache.flink.table.types.logical.StructuredType; +import org.apache.flink.table.types.logical.SymbolType; +import org.apache.flink.table.types.logical.YearMonthIntervalType; +import org.apache.flink.table.types.logical.ZonedTimestampType; + +public abstract class FlinkTypeVisitor implements LogicalTypeVisitor { + + // ------------------------- Unsupported types ------------------------------ + + @Override + public T visit(ZonedTimestampType zonedTimestampType) { + throw new UnsupportedOperationException("Unsupported ZonedTimestampType."); + } + + @Override + public T visit(YearMonthIntervalType yearMonthIntervalType) { + throw new UnsupportedOperationException("Unsupported YearMonthIntervalType."); + } + + @Override + public T visit(DayTimeIntervalType dayTimeIntervalType) { + throw new UnsupportedOperationException("Unsupported DayTimeIntervalType."); + } + + @Override + public T visit(DistinctType distinctType) { + throw new UnsupportedOperationException("Unsupported DistinctType."); + } + + @Override + public T visit(StructuredType structuredType) { + throw new UnsupportedOperationException("Unsupported StructuredType."); + } + + @Override + public T visit(NullType nullType) { + throw new UnsupportedOperationException("Unsupported NullType."); + } + + @Override + public T visit(RawType rawType) { + throw new UnsupportedOperationException("Unsupported RawType."); + } + + @Override + public T visit(SymbolType symbolType) { + throw new UnsupportedOperationException("Unsupported SymbolType."); + } + + @Override + public T visit(LogicalType other) { + throw new UnsupportedOperationException("Unsupported type: " + other); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java new file mode 100644 index 000000000000..ca7b1120bc81 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -0,0 +1,205 @@ +/* + * 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; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +/** + * A class for common Iceberg configs for Flink writes. + * + *

If a config is set at multiple levels, the following order of precedence is used (top to + * bottom): + * + *

    + *
  1. Write options + *
  2. flink ReadableConfig + *
  3. Table metadata + *
+ * + * The most specific value is set in write options and takes precedence over all other configs. If + * no write option is provided, this class checks the flink configuration for any overrides. If no + * applicable value is found in the write options, this class uses the table metadata. + * + *

Note this class is NOT meant to be serialized. + */ +public class FlinkWriteConf { + + private final FlinkConfParser confParser; + + public FlinkWriteConf( + Table table, Map writeOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); + } + + public boolean overwriteMode() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.OVERWRITE_MODE.key()) + .flinkConfig(FlinkWriteOptions.OVERWRITE_MODE) + .defaultValue(FlinkWriteOptions.OVERWRITE_MODE.defaultValue()) + .parse(); + } + + public boolean upsertMode() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key()) + .flinkConfig(FlinkWriteOptions.WRITE_UPSERT_ENABLED) + .tableProperty(TableProperties.UPSERT_ENABLED) + .defaultValue(TableProperties.UPSERT_ENABLED_DEFAULT) + .parse(); + } + + public FileFormat dataFileFormat() { + String valueAsString = + confParser + .stringConf() + .option(FlinkWriteOptions.WRITE_FORMAT.key()) + .flinkConfig(FlinkWriteOptions.WRITE_FORMAT) + .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) + .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) + .parse(); + return FileFormat.fromString(valueAsString); + } + + public long targetDataFileSize() { + return confParser + .longConf() + .option(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES.key()) + .flinkConfig(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES) + .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) + .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) + .parse(); + } + + public String parquetCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.PARQUET_COMPRESSION) + .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) + .parse(); + } + + public String parquetCompressionLevel() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) + .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) + .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + public String avroCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.AVRO_COMPRESSION) + .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) + .parse(); + } + + public String avroCompressionLevel() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) + .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) + .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + public String orcCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.ORC_COMPRESSION) + .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) + .parse(); + } + + public String orcCompressionStrategy() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_STRATEGY.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_STRATEGY) + .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) + .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) + .parse(); + } + + public DistributionMode distributionMode() { + String modeName = + confParser + .stringConf() + .option(FlinkWriteOptions.DISTRIBUTION_MODE.key()) + .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE) + .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) + .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE) + .parse(); + return DistributionMode.fromName(modeName); + } + + public int workerPoolSize() { + return confParser + .intConf() + .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) + .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) + .parse(); + } + + public String branch() { + return confParser + .stringConf() + .option(FlinkWriteOptions.BRANCH.key()) + .defaultValue(FlinkWriteOptions.BRANCH.defaultValue()) + .parse(); + } + + public Integer writeParallelism() { + return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional(); + } + + /** + * NOTE: This may be removed or changed in a future release. This value specifies the interval for + * refreshing the table instances in sink writer subtasks. If not specified then the default + * behavior is to not refresh the table. + * + * @return the interval for refreshing the table in sink writer subtasks + */ + @Experimental + public Duration tableRefreshInterval() { + return confParser + .durationConf() + .option(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key()) + .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) + .parseOptional(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java new file mode 100644 index 000000000000..df73f2e09cac --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -0,0 +1,73 @@ +/* + * 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; + +import java.time.Duration; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.iceberg.SnapshotRef; + +/** Flink sink write options */ +public class FlinkWriteOptions { + + private FlinkWriteOptions() {} + + // File format for write operations(default: Table write.format.default ) + public static final ConfigOption WRITE_FORMAT = + ConfigOptions.key("write-format").stringType().noDefaultValue(); + + // Overrides this table's write.target-file-size-bytes + public static final ConfigOption TARGET_FILE_SIZE_BYTES = + ConfigOptions.key("target-file-size-bytes").longType().noDefaultValue(); + + // Overrides this table's write..compression-codec + public static final ConfigOption COMPRESSION_CODEC = + ConfigOptions.key("compression-codec").stringType().noDefaultValue(); + + // Overrides this table's write..compression-level + public static final ConfigOption COMPRESSION_LEVEL = + ConfigOptions.key("compression-level").stringType().noDefaultValue(); + + // Overrides this table's write..compression-strategy + public static final ConfigOption COMPRESSION_STRATEGY = + ConfigOptions.key("compression-strategy").stringType().noDefaultValue(); + + // Overrides this table's write.upsert.enabled + public static final ConfigOption WRITE_UPSERT_ENABLED = + ConfigOptions.key("upsert-enabled").booleanType().noDefaultValue(); + + public static final ConfigOption OVERWRITE_MODE = + ConfigOptions.key("overwrite-enabled").booleanType().defaultValue(false); + + // Overrides the table's write.distribution-mode + public static final ConfigOption DISTRIBUTION_MODE = + ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + + // Branch to write to + public static final ConfigOption BRANCH = + ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); + + public static final ConfigOption WRITE_PARALLELISM = + ConfigOptions.key("write-parallelism").intType().noDefaultValue(); + + @Experimental + public static final ConfigOption TABLE_REFRESH_INTERVAL = + ConfigOptions.key("table-refresh-interval").durationType().noDefaultValue(); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java new file mode 100644 index 000000000000..1b9268569d9a --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java @@ -0,0 +1,121 @@ +/* + * 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; + +import java.util.List; +import java.util.Map; +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.api.constraints.UniqueConstraint; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; + +public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { + private final TableLoader tableLoader; + private final TableSchema tableSchema; + private final ReadableConfig readableConfig; + private final Map writeProps; + + private boolean overwrite = false; + + private IcebergTableSink(IcebergTableSink toCopy) { + this.tableLoader = toCopy.tableLoader; + this.tableSchema = toCopy.tableSchema; + this.overwrite = toCopy.overwrite; + this.readableConfig = toCopy.readableConfig; + this.writeProps = toCopy.writeProps; + } + + public IcebergTableSink( + TableLoader tableLoader, + TableSchema tableSchema, + ReadableConfig readableConfig, + Map writeProps) { + this.tableLoader = tableLoader; + this.tableSchema = tableSchema; + this.readableConfig = readableConfig; + this.writeProps = writeProps; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + Preconditions.checkState( + !overwrite || context.isBounded(), + "Unbounded data stream doesn't support overwrite operation."); + + List equalityColumns = + tableSchema.getPrimaryKey().map(UniqueConstraint::getColumns).orElseGet(ImmutableList::of); + + return new DataStreamSinkProvider() { + @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(); + } + }; + } + + @Override + public void applyStaticPartition(Map partition) { + // The flink's PartitionFanoutWriter will handle the static partition write policy + // automatically. + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + builder.addContainedKind(kind); + } + return builder.build(); + } + + @Override + public DynamicTableSink copy() { + return new IcebergTableSink(this); + } + + @Override + public String asSummaryString() { + return "Iceberg table sink"; + } + + @Override + public void applyOverwrite(boolean newOverwrite) { + this.overwrite = newOverwrite; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java new file mode 100644 index 000000000000..d4cec7a3e80b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java @@ -0,0 +1,142 @@ +/* + * 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; + +import java.lang.reflect.Array; +import java.nio.ByteBuffer; +import java.time.LocalDateTime; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.UUIDUtil; + +public class RowDataWrapper implements StructLike { + + private final LogicalType[] types; + private final PositionalGetter[] getters; + private RowData rowData = null; + + public RowDataWrapper(RowType rowType, Types.StructType struct) { + int size = rowType.getFieldCount(); + + types = (LogicalType[]) Array.newInstance(LogicalType.class, size); + getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); + + for (int i = 0; i < size; i++) { + types[i] = rowType.getTypeAt(i); + getters[i] = buildGetter(types[i], struct.fields().get(i).type()); + } + } + + public RowDataWrapper wrap(RowData data) { + this.rowData = data; + return this; + } + + @Override + public int size() { + return types.length; + } + + @Override + public T get(int pos, Class javaClass) { + if (rowData.isNullAt(pos)) { + return null; + } else if (getters[pos] != null) { + return javaClass.cast(getters[pos].get(rowData, pos)); + } + + Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); + return javaClass.cast(value); + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException( + "Could not set a field in the RowDataWrapper because rowData is read-only"); + } + + private interface PositionalGetter { + T get(RowData data, int pos); + } + + private static PositionalGetter buildGetter(LogicalType logicalType, Type type) { + switch (logicalType.getTypeRoot()) { + case TINYINT: + return (row, pos) -> (int) row.getByte(pos); + case SMALLINT: + return (row, pos) -> (int) row.getShort(pos); + case CHAR: + case VARCHAR: + return (row, pos) -> row.getString(pos).toString(); + + case BINARY: + case VARBINARY: + if (Type.TypeID.UUID == type.typeId()) { + return (row, pos) -> UUIDUtil.convert(row.getBinary(pos)); + } else { + return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); + } + + case DECIMAL: + DecimalType decimalType = (DecimalType) logicalType; + return (row, pos) -> + row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); + + case TIME_WITHOUT_TIME_ZONE: + // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds + // (Long). + return (row, pos) -> ((long) row.getInt(pos)) * 1_000; + + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) logicalType; + return (row, pos) -> { + LocalDateTime localDateTime = + row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); + return DateTimeUtil.microsFromTimestamp(localDateTime); + }; + + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; + return (row, pos) -> { + TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); + return timestampData.getMillisecond() * 1000 + + timestampData.getNanoOfMillisecond() / 1000; + }; + + case ROW: + RowType rowType = (RowType) logicalType; + Types.StructType structType = (Types.StructType) type; + + RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); + return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); + + default: + return null; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java new file mode 100644 index 000000000000..da509451fee7 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java @@ -0,0 +1,159 @@ +/* + * 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; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * Serializable loader to load an Iceberg {@link Table}. Flink needs to get {@link Table} objects in + * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg + * table loader to get the {@link Table} object. + */ +public interface TableLoader extends Closeable, Serializable, Cloneable { + + void open(); + + boolean isOpen(); + + Table loadTable(); + + /** Clone a TableLoader */ + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + TableLoader clone(); + + static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { + return new CatalogTableLoader(catalogLoader, identifier); + } + + static TableLoader fromHadoopTable(String location) { + return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf()); + } + + static TableLoader fromHadoopTable(String location, Configuration hadoopConf) { + return new HadoopTableLoader(location, hadoopConf); + } + + class HadoopTableLoader implements TableLoader { + + private static final long serialVersionUID = 1L; + + private final String location; + private final SerializableConfiguration hadoopConf; + + private transient HadoopTables tables; + + private HadoopTableLoader(String location, Configuration conf) { + this.location = location; + this.hadoopConf = new SerializableConfiguration(conf); + } + + @Override + public void open() { + tables = new HadoopTables(hadoopConf.get()); + } + + @Override + public boolean isOpen() { + return tables != null; + } + + @Override + public Table loadTable() { + FlinkEnvironmentContext.init(); + return tables.load(location); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new HadoopTableLoader(location, new Configuration(hadoopConf.get())); + } + + @Override + public void close() {} + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("location", location).toString(); + } + } + + class CatalogTableLoader implements TableLoader { + + private static final long serialVersionUID = 1L; + + private final CatalogLoader catalogLoader; + private final String identifier; + + private transient Catalog catalog; + + private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { + this.catalogLoader = catalogLoader; + this.identifier = tableIdentifier.toString(); + } + + @Override + public void open() { + catalog = catalogLoader.loadCatalog(); + } + + @Override + public boolean isOpen() { + return catalog != null; + } + + @Override + public Table loadTable() { + FlinkEnvironmentContext.init(); + return catalog.loadTable(TableIdentifier.parse(identifier)); + } + + @Override + public void close() throws IOException { + if (catalog instanceof Closeable) { + ((Closeable) catalog).close(); + } + + catalog = null; + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier)); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableIdentifier", identifier) + .add("catalogLoader", catalogLoader) + .toString(); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java new file mode 100644 index 000000000000..f8f1b74b1ceb --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java @@ -0,0 +1,134 @@ +/* + * 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; + +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +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.TypeUtil; +import org.apache.iceberg.types.Types; + +class TypeToFlinkType extends TypeUtil.SchemaVisitor { + TypeToFlinkType() {} + + @Override + public LogicalType schema(Schema schema, LogicalType structType) { + return structType; + } + + @Override + public LogicalType struct(Types.StructType struct, List fieldResults) { + List fields = struct.fields(); + + List flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + LogicalType type = fieldResults.get(i); + RowType.RowField flinkField = + new RowType.RowField(field.name(), type.copy(field.isOptional()), field.doc()); + flinkFields.add(flinkField); + } + + return new RowType(flinkFields); + } + + @Override + public LogicalType field(Types.NestedField field, LogicalType fieldResult) { + return fieldResult; + } + + @Override + public LogicalType list(Types.ListType list, LogicalType elementResult) { + return new ArrayType(elementResult.copy(list.isElementOptional())); + } + + @Override + public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) { + // keys in map are not allowed to be null. + return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional())); + } + + @Override + public LogicalType primitive(Type.PrimitiveType primitive) { + switch (primitive.typeId()) { + case BOOLEAN: + return new BooleanType(); + case INTEGER: + return new IntType(); + case LONG: + return new BigIntType(); + case FLOAT: + return new FloatType(); + case DOUBLE: + return new DoubleType(); + case DATE: + return new DateType(); + case TIME: + // For the type: Flink only support TimeType with default precision (second) now. The + // precision of time is + // not supported in Flink, so we can think of it as a simple time type directly. + // For the data: Flink uses int that support mills to represent time data, so it supports + // mills precision. + return new TimeType(); + case TIMESTAMP: + Types.TimestampType timestamp = (Types.TimestampType) primitive; + if (timestamp.shouldAdjustToUTC()) { + // MICROS + return new LocalZonedTimestampType(6); + } else { + // MICROS + return new TimestampType(6); + } + case STRING: + return new VarCharType(VarCharType.MAX_LENGTH); + case UUID: + // UUID length is 16 + return new BinaryType(16); + case FIXED: + Types.FixedType fixedType = (Types.FixedType) primitive; + return new BinaryType(fixedType.length()); + case BINARY: + return new VarBinaryType(VarBinaryType.MAX_LENGTH); + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + return new DecimalType(decimal.precision(), decimal.scale()); + default: + throw new UnsupportedOperationException( + "Cannot convert unknown type to Flink: " + primitive); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java new file mode 100644 index 000000000000..06ac54617ae6 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.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.actions; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.Table; + +public class Actions { + + public static final Configuration CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private StreamExecutionEnvironment env; + private Table table; + + private Actions(StreamExecutionEnvironment env, Table table) { + this.env = env; + this.table = table; + } + + public static Actions forTable(StreamExecutionEnvironment env, Table table) { + return new Actions(env, table); + } + + public static Actions forTable(Table table) { + return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table); + } + + public RewriteDataFilesAction rewriteDataFiles() { + return new RewriteDataFilesAction(env, table); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java new file mode 100644 index 000000000000..9876bb3861c4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.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.flink.actions; + +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BaseRewriteDataFilesAction; +import org.apache.iceberg.flink.source.RowDataRewriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { + + private StreamExecutionEnvironment env; + private int maxParallelism; + + public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { + super(table); + this.env = env; + this.maxParallelism = env.getParallelism(); + } + + @Override + protected FileIO fileIO() { + return table().io(); + } + + @Override + protected List rewriteDataForTasks(List combinedScanTasks) { + int size = combinedScanTasks.size(); + int parallelism = Math.min(size, maxParallelism); + DataStream dataStream = env.fromCollection(combinedScanTasks); + RowDataRewriter rowDataRewriter = + new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); + try { + return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); + } catch (Exception e) { + throw new RuntimeException("Rewrite data file error.", e); + } + } + + @Override + protected RewriteDataFilesAction self() { + return this; + } + + public RewriteDataFilesAction maxParallelism(int parallelism) { + Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %s", parallelism); + this.maxParallelism = parallelism; + return this; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java new file mode 100644 index 000000000000..8103224a0b6c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.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.flink.data; + +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.Pair; + +public abstract class AvroWithFlinkSchemaVisitor + extends AvroWithPartnerByStructureVisitor { + + @Override + protected boolean isStringType(LogicalType logicalType) { + return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING); + } + + @Override + protected boolean isMapType(LogicalType logicalType) { + return logicalType instanceof MapType; + } + + @Override + protected LogicalType arrayElementType(LogicalType arrayType) { + Preconditions.checkArgument( + arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); + return ((ArrayType) arrayType).getElementType(); + } + + @Override + protected LogicalType mapKeyType(LogicalType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).getKeyType(); + } + + @Override + protected LogicalType mapValueType(LogicalType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).getValueType(); + } + + @Override + protected Pair fieldNameAndType(LogicalType structType, int pos) { + Preconditions.checkArgument( + structType instanceof RowType, "Invalid struct: %s is not a struct", structType); + RowType.RowField field = ((RowType) structType).getFields().get(pos); + return Pair.of(field.getName(), field.getType()); + } + + @Override + protected LogicalType nullType() { + return new NullType(); + } +} 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 new file mode 100644 index 000000000000..86404959735a --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java @@ -0,0 +1,169 @@ +/* + * 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; + +public class FlinkAvroReader implements DatumReader, SupportsRowPosition { + + private final Schema readSchema; + private final ValueReader reader; + private Schema fileSchema = null; + + public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { + this(expectedSchema, readSchema, ImmutableMap.of()); + } + + @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/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java new file mode 100644 index 000000000000..873e65783119 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java @@ -0,0 +1,165 @@ +/* + * 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.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.Encoder; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.avro.MetricsAwareDatumWriter; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class FlinkAvroWriter implements MetricsAwareDatumWriter { + private final RowType rowType; + private ValueWriter writer = null; + + public FlinkAvroWriter(RowType rowType) { + this.rowType = rowType; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema schema) { + this.writer = + (ValueWriter) + AvroWithFlinkSchemaVisitor.visit(rowType, schema, new WriteBuilder()); + } + + @Override + public void write(RowData datum, Encoder out) throws IOException { + writer.write(datum, out); + } + + @Override + public Stream metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends AvroWithFlinkSchemaVisitor> { + @Override + public ValueWriter record( + LogicalType struct, Schema record, List names, List> fields) { + return FlinkValueWriters.row( + fields, + IntStream.range(0, names.size()) + .mapToObj(i -> fieldNameAndType(struct, i).second()) + .collect(Collectors.toList())); + } + + @Override + public ValueWriter union(LogicalType type, Schema union, List> options) { + Preconditions.checkArgument( + options.contains(ValueWriters.nulls()), + "Cannot create writer for non-option union: %s", + union); + Preconditions.checkArgument( + options.size() == 2, "Cannot create writer for non-option union: %s", union); + if (union.getTypes().get(0).getType() == Schema.Type.NULL) { + return ValueWriters.option(0, options.get(1)); + } else { + return ValueWriters.option(1, options.get(0)); + } + } + + @Override + public ValueWriter array(LogicalType sArray, Schema array, ValueWriter elementWriter) { + return FlinkValueWriters.array(elementWriter, arrayElementType(sArray)); + } + + @Override + public ValueWriter map(LogicalType sMap, Schema map, ValueWriter valueReader) { + return FlinkValueWriters.map( + FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); + } + + @Override + public ValueWriter map( + LogicalType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { + return FlinkValueWriters.arrayMap( + keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); + } + + @Override + public ValueWriter primitive(LogicalType type, Schema primitive) { + org.apache.avro.LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return ValueWriters.ints(); + + case "time-micros": + return FlinkValueWriters.timeMicros(); + + case "timestamp-micros": + return FlinkValueWriters.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); + + case "uuid": + return ValueWriters.uuids(); + + default: + throw new IllegalArgumentException("Unsupported logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueWriters.nulls(); + case BOOLEAN: + return ValueWriters.booleans(); + case INT: + switch (type.getTypeRoot()) { + case TINYINT: + return ValueWriters.tinyints(); + case SMALLINT: + return ValueWriters.shorts(); + default: + return ValueWriters.ints(); + } + case LONG: + return ValueWriters.longs(); + case FLOAT: + return ValueWriters.floats(); + case DOUBLE: + return ValueWriters.doubles(); + case STRING: + return FlinkValueWriters.strings(); + case FIXED: + return ValueWriters.fixed(primitive.getFixedSize()); + case BYTES: + return ValueWriters.bytes(); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java new file mode 100644 index 000000000000..65b9d44ad4b8 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java @@ -0,0 +1,131 @@ +/* + * 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.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.orc.OrcRowReader; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +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.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +public class FlinkOrcReader implements OrcRowReader { + private final OrcValueReader reader; + + public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) { + this(iSchema, readSchema, ImmutableMap.of()); + } + + public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map idToConstant) { + this.reader = + OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant)); + } + + @Override + public RowData read(VectorizedRowBatch batch, int row) { + return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + reader.setBatchContext(batchOffsetInFile); + } + + private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public OrcValueReader record( + Types.StructType iStruct, + TypeDescription record, + List names, + List> fields) { + return FlinkOrcReaders.struct(fields, iStruct, idToConstant); + } + + @Override + public OrcValueReader list( + Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { + return FlinkOrcReaders.array(elementReader); + } + + @Override + public OrcValueReader map( + Types.MapType iMap, + TypeDescription map, + OrcValueReader keyReader, + OrcValueReader valueReader) { + return FlinkOrcReaders.map(keyReader, valueReader); + } + + @Override + public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { + switch (iPrimitive.typeId()) { + case BOOLEAN: + return OrcValueReaders.booleans(); + case INTEGER: + return OrcValueReaders.ints(); + case LONG: + return OrcValueReaders.longs(); + case FLOAT: + return OrcValueReaders.floats(); + case DOUBLE: + return OrcValueReaders.doubles(); + case DATE: + return FlinkOrcReaders.dates(); + case TIME: + return FlinkOrcReaders.times(); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; + if (timestampType.shouldAdjustToUTC()) { + return FlinkOrcReaders.timestampTzs(); + } else { + return FlinkOrcReaders.timestamps(); + } + case STRING: + return FlinkOrcReaders.strings(); + case UUID: + case FIXED: + case BINARY: + return OrcValueReaders.bytes(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; + return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale()); + default: + throw new IllegalArgumentException( + String.format( + "Invalid iceberg type %s corresponding to ORC type %s", iPrimitive, primitive)); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java new file mode 100644 index 000000000000..7a4a15c7e600 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java @@ -0,0 +1,283 @@ +/* + * 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.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +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.apache.iceberg.types.Types; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.serde2.io.HiveDecimalWritable; + +class FlinkOrcReaders { + private FlinkOrcReaders() {} + + static OrcValueReader strings() { + return StringReader.INSTANCE; + } + + static OrcValueReader dates() { + return DateReader.INSTANCE; + } + + static OrcValueReader decimals(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Reader(precision, scale); + } else if (precision <= 38) { + return new Decimal38Reader(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueReader times() { + return TimeReader.INSTANCE; + } + + static OrcValueReader timestamps() { + return TimestampReader.INSTANCE; + } + + static OrcValueReader timestampTzs() { + return TimestampTzReader.INSTANCE; + } + + static OrcValueReader array(OrcValueReader elementReader) { + return new ArrayReader<>(elementReader); + } + + public static OrcValueReader map( + OrcValueReader keyReader, OrcValueReader valueReader) { + return new MapReader<>(keyReader, valueReader); + } + + public static OrcValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements OrcValueReader { + private static final StringReader INSTANCE = new StringReader(); + + @Override + public StringData nonNullRead(ColumnVector vector, int row) { + BytesColumnVector bytesVector = (BytesColumnVector) vector; + return StringData.fromBytes( + bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); + } + } + + private static class DateReader implements OrcValueReader { + private static final DateReader INSTANCE = new DateReader(); + + @Override + public Integer nonNullRead(ColumnVector vector, int row) { + return (int) ((LongColumnVector) vector).vector[row]; + } + } + + private static class Decimal18Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal18Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData nonNullRead(ColumnVector vector, int row) { + HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; + + // The hive ORC writer may will adjust the scale of decimal data. + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale); + } + } + + private static class Decimal38Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal38Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData nonNullRead(ColumnVector vector, int row) { + BigDecimal value = + ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); + + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return DecimalData.fromBigDecimal(value, precision, scale); + } + } + + private static class TimeReader implements OrcValueReader { + private static final TimeReader INSTANCE = new TimeReader(); + + @Override + public Integer nonNullRead(ColumnVector vector, int row) { + long micros = ((LongColumnVector) vector).vector[row]; + // Flink only support time mills, just erase micros. + return (int) (micros / 1000); + } + } + + private static class TimestampReader implements OrcValueReader { + private static final TimestampReader INSTANCE = new TimestampReader(); + + @Override + public TimestampData nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + LocalDateTime localDate = + Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) + .atOffset(ZoneOffset.UTC) + .toLocalDateTime(); + return TimestampData.fromLocalDateTime(localDate); + } + } + + private static class TimestampTzReader implements OrcValueReader { + private static final TimestampTzReader INSTANCE = new TimestampTzReader(); + + @Override + public TimestampData nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + Instant instant = + Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) + .atOffset(ZoneOffset.UTC) + .toInstant(); + return TimestampData.fromInstant(instant); + } + } + + private static class ArrayReader implements OrcValueReader { + private final OrcValueReader elementReader; + + private ArrayReader(OrcValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public ArrayData nonNullRead(ColumnVector vector, int row) { + ListColumnVector listVector = (ListColumnVector) vector; + int offset = (int) listVector.offsets[row]; + int length = (int) listVector.lengths[row]; + List elements = Lists.newArrayListWithExpectedSize(length); + for (int c = 0; c < length; ++c) { + elements.add(elementReader.read(listVector.child, offset + c)); + } + return new GenericArrayData(elements.toArray()); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + elementReader.setBatchContext(batchOffsetInFile); + } + } + + private static class MapReader implements OrcValueReader { + private final OrcValueReader keyReader; + private final OrcValueReader valueReader; + + private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData nonNullRead(ColumnVector vector, int row) { + MapColumnVector mapVector = (MapColumnVector) vector; + int offset = (int) mapVector.offsets[row]; + long length = mapVector.lengths[row]; + + Map map = Maps.newHashMap(); + for (int c = 0; c < length; c++) { + K key = keyReader.read(mapVector.keys, offset + c); + V value = valueReader.read(mapVector.values, offset + c); + map.put(key, value); + } + + return new GenericMapData(map); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + keyReader.setBatchContext(batchOffsetInFile); + valueReader.setBatchContext(batchOffsetInFile); + } + } + + private static class StructReader extends OrcValueReaders.StructReader { + private final int numFields; + + StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = struct.fields().size(); + } + + @Override + protected RowData create() { + return new GenericRowData(numFields); + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java new file mode 100644 index 000000000000..6a31accffd22 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java @@ -0,0 +1,163 @@ +/* + * 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.util.Deque; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.OrcRowWriter; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.orc.storage.ql.exec.vector.VectorizedRowBatch; + +public class FlinkOrcWriter implements OrcRowWriter { + private final FlinkOrcWriters.RowDataWriter writer; + + private FlinkOrcWriter(RowType rowType, Schema iSchema) { + this.writer = + (FlinkOrcWriters.RowDataWriter) + FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder()); + } + + public static OrcRowWriter buildWriter(RowType rowType, Schema iSchema) { + return new FlinkOrcWriter(rowType, iSchema); + } + + @Override + public void write(RowData row, VectorizedRowBatch output) { + Preconditions.checkArgument(row != null, "value must not be null"); + writer.writeRow(row, output); + } + + @Override + public List> writers() { + return writer.writers(); + } + + @Override + public Stream> metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends FlinkSchemaVisitor> { + private final Deque fieldIds = Lists.newLinkedList(); + + private WriteBuilder() {} + + @Override + public void beforeField(Types.NestedField field) { + fieldIds.push(field.fieldId()); + } + + @Override + public void afterField(Types.NestedField field) { + fieldIds.pop(); + } + + @Override + public OrcValueWriter record( + Types.StructType iStruct, List> results, List fieldType) { + return FlinkOrcWriters.struct(results, fieldType); + } + + @Override + public OrcValueWriter map( + Types.MapType iMap, + OrcValueWriter key, + OrcValueWriter value, + LogicalType keyType, + LogicalType valueType) { + return FlinkOrcWriters.map(key, value, keyType, valueType); + } + + @Override + public OrcValueWriter list( + Types.ListType iList, OrcValueWriter element, LogicalType elementType) { + return FlinkOrcWriters.list(element, elementType); + } + + @Override + public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { + switch (iPrimitive.typeId()) { + case BOOLEAN: + return GenericOrcWriters.booleans(); + case INTEGER: + switch (flinkPrimitive.getTypeRoot()) { + case TINYINT: + return GenericOrcWriters.bytes(); + case SMALLINT: + return GenericOrcWriters.shorts(); + } + return GenericOrcWriters.ints(); + case LONG: + return GenericOrcWriters.longs(); + case FLOAT: + Preconditions.checkArgument( + fieldIds.peek() != null, + String.format( + "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " + + "information is not properly pushed during schema visiting.", + iPrimitive)); + return GenericOrcWriters.floats(fieldIds.peek()); + case DOUBLE: + Preconditions.checkArgument( + fieldIds.peek() != null, + String.format( + "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " + + "information is not properly pushed during schema visiting.", + iPrimitive)); + return GenericOrcWriters.doubles(fieldIds.peek()); + case DATE: + return FlinkOrcWriters.dates(); + case TIME: + return FlinkOrcWriters.times(); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; + if (timestampType.shouldAdjustToUTC()) { + return FlinkOrcWriters.timestampTzs(); + } else { + return FlinkOrcWriters.timestamps(); + } + case STRING: + return FlinkOrcWriters.strings(); + case UUID: + case FIXED: + case BINARY: + return GenericOrcWriters.byteArrays(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; + return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale()); + default: + throw new IllegalArgumentException( + String.format( + "Invalid iceberg type %s corresponding to Flink logical type %s", + iPrimitive, flinkPrimitive)); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java new file mode 100644 index 000000000000..da2f95cf822f --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -0,0 +1,317 @@ +/* + * 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.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.orc.storage.common.type.HiveDecimal; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; + +class FlinkOrcWriters { + + private FlinkOrcWriters() {} + + static OrcValueWriter strings() { + return StringWriter.INSTANCE; + } + + static OrcValueWriter dates() { + return DateWriter.INSTANCE; + } + + static OrcValueWriter times() { + return TimeWriter.INSTANCE; + } + + static OrcValueWriter timestamps() { + return TimestampWriter.INSTANCE; + } + + static OrcValueWriter timestampTzs() { + return TimestampTzWriter.INSTANCE; + } + + static OrcValueWriter decimals(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Writer(precision, scale); + } else if (precision <= 38) { + return new Decimal38Writer(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueWriter list( + OrcValueWriter elementWriter, LogicalType elementType) { + return new ListWriter<>(elementWriter, elementType); + } + + static OrcValueWriter map( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + return new MapWriter<>(keyWriter, valueWriter, keyType, valueType); + } + + static OrcValueWriter struct(List> writers, List types) { + return new RowDataWriter(writers, types); + } + + private static class StringWriter implements OrcValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + @Override + public void nonNullWrite(int rowId, StringData data, ColumnVector output) { + byte[] value = data.toBytes(); + ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); + } + } + + private static class DateWriter implements OrcValueWriter { + private static final DateWriter INSTANCE = new DateWriter(); + + @Override + public void nonNullWrite(int rowId, Integer data, ColumnVector output) { + ((LongColumnVector) output).vector[rowId] = data; + } + } + + private static class TimeWriter implements OrcValueWriter { + private static final TimeWriter INSTANCE = new TimeWriter(); + + @Override + public void nonNullWrite(int rowId, Integer millis, ColumnVector output) { + // The time in flink is in millisecond, while the standard time in iceberg is microsecond. + // So we need to transform it to microsecond. + ((LongColumnVector) output).vector[rowId] = millis * 1000L; + } + } + + private static class TimestampWriter implements OrcValueWriter { + private static final TimestampWriter INSTANCE = new TimestampWriter(); + + @Override + public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + cv.setIsUTC(true); + // millis + OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC); + cv.time[rowId] = + offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000; + // truncate nanos to only keep microsecond precision. + cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000; + } + } + + private static class TimestampTzWriter implements OrcValueWriter { + private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); + + @SuppressWarnings("JavaInstantGetSecondsGetNano") + @Override + public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + // millis + Instant instant = data.toInstant(); + cv.time[rowId] = instant.toEpochMilli(); + // truncate nanos to only keep microsecond precision. + cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000; + } + } + + private static class Decimal18Writer implements OrcValueWriter { + private final int precision; + private final int scale; + + Decimal18Writer(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { + Preconditions.checkArgument( + scale == data.scale(), + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + data); + Preconditions.checkArgument( + data.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + data); + + ((DecimalColumnVector) output) + .vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale()); + } + } + + private static class Decimal38Writer implements OrcValueWriter { + private final int precision; + private final int scale; + + Decimal38Writer(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { + Preconditions.checkArgument( + scale == data.scale(), + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + data); + Preconditions.checkArgument( + data.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + data); + + ((DecimalColumnVector) output) + .vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false)); + } + } + + static class ListWriter implements OrcValueWriter { + private final OrcValueWriter elementWriter; + private final ArrayData.ElementGetter elementGetter; + + ListWriter(OrcValueWriter elementWriter, LogicalType elementType) { + this.elementWriter = elementWriter; + this.elementGetter = ArrayData.createElementGetter(elementType); + } + + @Override + @SuppressWarnings("unchecked") + public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { + ListColumnVector cv = (ListColumnVector) output; + cv.lengths[rowId] = data.size(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough. + growColumnVector(cv.child, cv.childCount); + + for (int e = 0; e < cv.lengths[rowId]; ++e) { + Object value = elementGetter.getElementOrNull(data, e); + elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child); + } + } + + @Override + public Stream> metrics() { + return elementWriter.metrics(); + } + } + + static class MapWriter implements OrcValueWriter { + private final OrcValueWriter keyWriter; + private final OrcValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + MapWriter( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.valueWriter = valueWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void nonNullWrite(int rowId, MapData data, ColumnVector output) { + MapColumnVector cv = (MapColumnVector) output; + ArrayData keyArray = data.keyArray(); + ArrayData valArray = data.valueArray(); + + // record the length and start of the list elements + cv.lengths[rowId] = data.size(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough + growColumnVector(cv.keys, cv.childCount); + growColumnVector(cv.values, cv.childCount); + // Add each element + for (int e = 0; e < cv.lengths[rowId]; ++e) { + int pos = (int) (e + cv.offsets[rowId]); + keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys); + valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values); + } + } + + @Override + public Stream> metrics() { + return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); + } + } + + static class RowDataWriter extends GenericOrcWriters.StructWriter { + private final List fieldGetters; + + RowDataWriter(List> writers, List types) { + super(writers); + + this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); + for (int i = 0; i < types.size(); i++) { + fieldGetters.add(RowData.createFieldGetter(types.get(i), i)); + } + } + + @Override + protected Object get(RowData struct, int index) { + return fieldGetters.get(index).getFieldOrNull(struct); + } + } + + private static void growColumnVector(ColumnVector cv, int requestedSize) { + if (cv.isNull.length < requestedSize) { + // Use growth factor of 3 to avoid frequent array allocations + cv.ensureSize(requestedSize * 3, true); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java new file mode 100644 index 000000000000..ab7b1174c9f3 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -0,0 +1,832 @@ +/* + * 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.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.ZoneOffset; +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.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.MetadataColumns; +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.TypeWithSchemaVisitor; +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.types.Types; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +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; + +public class FlinkParquetReaders { + private FlinkParquetReaders() {} + + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema) { + return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); + } + + @SuppressWarnings("unchecked") + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return (ParquetValueReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); + } + + private static class ReadBuilder extends TypeWithSchemaVisitor> { + private final MessageType type; + private final Map idToConstant; + + ReadBuilder(MessageType type, Map idToConstant) { + this.type = type; + this.idToConstant = idToConstant; + } + + @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); + if (fieldReaders.get(i) != null) { + int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; + if (fieldType.getId() != null) { + int id = fieldType.getId().intValue(); + readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); + typesById.put(id, fieldType); + if (idToConstant.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 (idToConstant.containsKey(id)) { + // containsKey is used because the constant may be null + int fieldMaxDefinitionLevel = + maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); + reorderedFields.add( + ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel)); + types.add(null); + } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { + reorderedFields.add(ParquetValueReaders.position()); + types.add(null); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(ParquetValueReaders.constant(false)); + 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 RowDataReader(types, reorderedFields); + } + + @Override + public ParquetValueReader list( + Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { + if (expectedList == null) { + return null; + } + + 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) { + if (expectedMap == null) { + return null; + } + + 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 + @SuppressWarnings("CyclomaticComplexity") + public ParquetValueReader primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { + if (expected == null) { + return null; + } + + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + + if (primitive.getOriginalType() != null) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + return new StringReader(desc); + case INT_8: + case INT_16: + case INT_32: + if (expected.typeId() == Types.LongType.get().typeId()) { + return new ParquetValueReaders.IntAsLongReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case TIME_MICROS: + return new LossyMicrosToMillisTimeReader(desc); + case TIME_MILLIS: + return new MillisTimeReader(desc); + case DATE: + case INT_64: + return new ParquetValueReaders.UnboxedReader<>(desc); + case TIMESTAMP_MICROS: + if (((Types.TimestampType) expected).shouldAdjustToUTC()) { + return new MicrosToTimestampTzReader(desc); + } else { + return new MicrosToTimestampReader(desc); + } + case TIMESTAMP_MILLIS: + if (((Types.TimestampType) expected).shouldAdjustToUTC()) { + return new MillisToTimestampTzReader(desc); + } else { + return new MillisToTimestampReader(desc); + } + case DECIMAL: + DecimalLogicalTypeAnnotation decimal = + (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + case INT64: + return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + case INT32: + return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); + default: + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); + } + case BSON: + return new ParquetValueReaders.ByteArrayReader(desc); + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return new ParquetValueReaders.ByteArrayReader(desc); + case INT32: + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { + return new ParquetValueReaders.IntAsLongReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case FLOAT: + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { + return new ParquetValueReaders.FloatAsDoubleReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case BOOLEAN: + case INT64: + case DOUBLE: + return new ParquetValueReaders.UnboxedReader<>(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static class BinaryDecimalReader + extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + Binary binary = column.nextBinary(); + BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale); + // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader + return DecimalData.fromBigDecimal(bigDecimal, precision, scale); + } + } + + private static class IntegerDecimalReader + extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale); + } + } + + private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale); + } + } + + private static class MicrosToTimestampTzReader + extends ParquetValueReaders.UnboxedReader { + MicrosToTimestampTzReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long value = readLong(); + return TimestampData.fromLocalDateTime( + Instant.ofEpochSecond( + Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) + .atOffset(ZoneOffset.UTC) + .toLocalDateTime()); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MicrosToTimestampReader + extends ParquetValueReaders.UnboxedReader { + MicrosToTimestampReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long value = readLong(); + return TimestampData.fromInstant( + Instant.ofEpochSecond( + Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MillisToTimestampReader + extends ParquetValueReaders.UnboxedReader { + MillisToTimestampReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long millis = readLong(); + return TimestampData.fromEpochMillis(millis); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MillisToTimestampTzReader + extends ParquetValueReaders.UnboxedReader { + MillisToTimestampTzReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long millis = readLong(); + return TimestampData.fromLocalDateTime( + Instant.ofEpochMilli(millis).atOffset(ZoneOffset.UTC).toLocalDateTime()); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class StringReader extends ParquetValueReaders.PrimitiveReader { + StringReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public StringData read(StringData ignored) { + Binary binary = column.nextBinary(); + ByteBuffer buffer = binary.toByteBuffer(); + if (buffer.hasArray()) { + return StringData.fromBytes( + buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + } else { + return StringData.fromBytes(binary.getBytes()); + } + } + } + + private static class LossyMicrosToMillisTimeReader + extends ParquetValueReaders.PrimitiveReader { + LossyMicrosToMillisTimeReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Integer read(Integer reuse) { + // Discard microseconds since Flink uses millisecond unit for TIME type. + return (int) Math.floorDiv(column.nextLong(), 1000); + } + } + + private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader { + MillisTimeReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Integer read(Integer reuse) { + return (int) column.nextLong(); + } + } + + private static class ArrayReader + extends ParquetValueReaders.RepeatedReader { + private int readPos = 0; + private int writePos = 0; + + ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { + super(definitionLevel, repetitionLevel, reader); + } + + @Override + protected ReusableArrayData newListData(ArrayData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableArrayData) { + return (ReusableArrayData) reuse; + } else { + return new ReusableArrayData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected E getElement(ReusableArrayData list) { + E value = null; + if (readPos < list.capacity()) { + value = (E) list.values[readPos]; + } + + readPos += 1; + + return value; + } + + @Override + protected void addElement(ReusableArrayData reused, E element) { + if (writePos >= reused.capacity()) { + reused.grow(); + } + + reused.values[writePos] = element; + + writePos += 1; + } + + @Override + protected ArrayData buildList(ReusableArrayData list) { + // Since ReusableArrayData is not accepted by Flink, use GenericArrayData temporarily to walk + // around it. + // Revert this to use ReusableArrayData once it is fixed in Flink. + // For your reference, https://issues.apache.org/jira/browse/FLINK-25238. + return new GenericArrayData(Arrays.copyOf(list.values, writePos)); + } + } + + private static class MapReader + extends ParquetValueReaders.RepeatedKeyValueReader { + private int readPos = 0; + private int writePos = 0; + + private final ParquetValueReaders.ReusableEntry entry = + new ParquetValueReaders.ReusableEntry<>(); + private final ParquetValueReaders.ReusableEntry nullEntry = + new ParquetValueReaders.ReusableEntry<>(); + + MapReader( + int definitionLevel, + int repetitionLevel, + ParquetValueReader keyReader, + ParquetValueReader valueReader) { + super(definitionLevel, repetitionLevel, keyReader, valueReader); + } + + @Override + protected ReusableMapData newMapData(MapData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableMapData) { + return (ReusableMapData) reuse; + } else { + return new ReusableMapData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected Map.Entry getPair(ReusableMapData map) { + Map.Entry kv = nullEntry; + if (readPos < map.capacity()) { + entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); + kv = entry; + } + + readPos += 1; + + return kv; + } + + @Override + protected void addPair(ReusableMapData map, K key, V value) { + if (writePos >= map.capacity()) { + map.grow(); + } + + map.keys.values[writePos] = key; + map.values.values[writePos] = value; + + writePos += 1; + } + + @Override + protected MapData buildMap(ReusableMapData map) { + map.setNumElements(writePos); + return map; + } + } + + private static class RowDataReader + extends ParquetValueReaders.StructReader { + private final int numFields; + + RowDataReader(List types, List> readers) { + super(types, readers); + this.numFields = readers.size(); + } + + @Override + protected GenericRowData newStructData(RowData reuse) { + if (reuse instanceof GenericRowData) { + return (GenericRowData) reuse; + } else { + return new GenericRowData(numFields); + } + } + + @Override + protected Object getField(GenericRowData intermediate, int pos) { + return intermediate.getField(pos); + } + + @Override + protected RowData buildStruct(GenericRowData struct) { + return struct; + } + + @Override + protected void set(GenericRowData row, int pos, Object value) { + row.setField(pos, value); + } + + @Override + protected void setNull(GenericRowData row, int pos) { + row.setField(pos, null); + } + + @Override + protected void setBoolean(GenericRowData row, int pos, boolean value) { + row.setField(pos, value); + } + + @Override + protected void setInteger(GenericRowData row, int pos, int value) { + row.setField(pos, value); + } + + @Override + protected void setLong(GenericRowData row, int pos, long value) { + row.setField(pos, value); + } + + @Override + protected void setFloat(GenericRowData row, int pos, float value) { + row.setField(pos, value); + } + + @Override + protected void setDouble(GenericRowData row, int pos, double value) { + row.setField(pos, value); + } + } + + private static class ReusableMapData implements MapData { + private final ReusableArrayData keys; + private final ReusableArrayData values; + + private int numElements; + + private ReusableMapData() { + this.keys = new ReusableArrayData(); + this.values = new ReusableArrayData(); + } + + private void grow() { + keys.grow(); + values.grow(); + } + + private int capacity() { + return keys.capacity(); + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + keys.setNumElements(numElements); + values.setNumElements(numElements); + } + + @Override + public int size() { + return numElements; + } + + @Override + public ReusableArrayData keyArray() { + return keys; + } + + @Override + public ReusableArrayData valueArray() { + return values; + } + } + + private static class ReusableArrayData implements ArrayData { + private static final Object[] EMPTY = new Object[0]; + + private Object[] values = EMPTY; + private int numElements = 0; + + private void grow() { + if (values.length == 0) { + this.values = new Object[20]; + } else { + Object[] old = values; + this.values = new Object[old.length << 1]; + // copy the old array in case it has values that can be reused + System.arraycopy(old, 0, values, 0, old.length); + } + } + + private int capacity() { + return values.length; + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public boolean isNullAt(int ordinal) { + return null == values[ordinal]; + } + + @Override + public boolean getBoolean(int ordinal) { + return (boolean) values[ordinal]; + } + + @Override + public byte getByte(int ordinal) { + return (byte) values[ordinal]; + } + + @Override + public short getShort(int ordinal) { + return (short) values[ordinal]; + } + + @Override + public int getInt(int ordinal) { + return (int) values[ordinal]; + } + + @Override + public long getLong(int ordinal) { + return (long) values[ordinal]; + } + + @Override + public float getFloat(int ordinal) { + return (float) values[ordinal]; + } + + @Override + public double getDouble(int ordinal) { + return (double) values[ordinal]; + } + + @Override + public StringData getString(int pos) { + return (StringData) values[pos]; + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return (DecimalData) values[pos]; + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return (TimestampData) values[pos]; + } + + @SuppressWarnings("unchecked") + @Override + public RawValueData getRawValue(int pos) { + return (RawValueData) values[pos]; + } + + @Override + public byte[] getBinary(int ordinal) { + return (byte[]) values[ordinal]; + } + + @Override + public ArrayData getArray(int ordinal) { + return (ArrayData) values[ordinal]; + } + + @Override + public MapData getMap(int ordinal) { + return (MapData) values[ordinal]; + } + + @Override + public RowData getRow(int pos, int numFields) { + return (RowData) values[pos]; + } + + @Override + public boolean[] toBooleanArray() { + return ArrayUtil.toPrimitive((Boolean[]) values); + } + + @Override + public byte[] toByteArray() { + return ArrayUtil.toPrimitive((Byte[]) values); + } + + @Override + public short[] toShortArray() { + return ArrayUtil.toPrimitive((Short[]) values); + } + + @Override + public int[] toIntArray() { + return ArrayUtil.toPrimitive((Integer[]) values); + } + + @Override + public long[] toLongArray() { + return ArrayUtil.toPrimitive((Long[]) values); + } + + @Override + public float[] toFloatArray() { + return ArrayUtil.toPrimitive((Float[]) values); + } + + @Override + public double[] toDoubleArray() { + return ArrayUtil.toPrimitive((Double[]) values); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java new file mode 100644 index 000000000000..db4f1730a134 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java @@ -0,0 +1,504 @@ +/* + * 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.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +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; + +public class FlinkParquetWriters { + private FlinkParquetWriters() {} + + @SuppressWarnings("unchecked") + public static ParquetValueWriter buildWriter(LogicalType schema, MessageType type) { + return (ParquetValueWriter) + ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type)); + } + + private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor> { + private final MessageType type; + + WriteBuilder(MessageType type) { + this.type = type; + } + + @Override + public ParquetValueWriter message( + RowType sStruct, MessageType message, List> fields) { + return struct(sStruct, message.asGroupType(), fields); + } + + @Override + public ParquetValueWriter struct( + RowType sStruct, GroupType struct, List> fieldWriters) { + List fields = struct.getFields(); + List flinkFields = sStruct.getFields(); + List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); + List flinkTypes = Lists.newArrayList(); + for (int i = 0; i < fields.size(); i += 1) { + writers.add(newOption(struct.getType(i), fieldWriters.get(i))); + flinkTypes.add(flinkFields.get(i).getType()); + } + + return new RowDataWriter(writers, flinkTypes); + } + + @Override + public ParquetValueWriter list( + ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { + GroupType repeated = array.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new ArrayDataWriter<>( + repeatedD, + repeatedR, + newOption(repeated.getType(0), elementWriter), + sArray.getElementType()); + } + + @Override + public ParquetValueWriter map( + MapType sMap, + GroupType map, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter) { + GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new MapDataWriter<>( + repeatedD, + repeatedR, + newOption(repeatedKeyValue.getType(0), keyWriter), + newOption(repeatedKeyValue.getType(1), valueWriter), + sMap.getKeyType(), + sMap.getValueType()); + } + + private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { + int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); + return ParquetValueWriters.option(fieldType, maxD, writer); + } + + @Override + public ParquetValueWriter primitive(LogicalType fType, PrimitiveType primitive) { + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + + if (primitive.getOriginalType() != null) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + return strings(desc); + case DATE: + case INT_8: + case INT_16: + case INT_32: + return ints(fType, desc); + case INT_64: + return ParquetValueWriters.longs(desc); + case TIME_MICROS: + return timeMicros(desc); + case TIMESTAMP_MICROS: + return timestamps(desc); + case DECIMAL: + DecimalLogicalTypeAnnotation decimal = + (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); + case INT64: + return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); + default: + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); + } + case BSON: + return byteArrays(desc); + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return byteArrays(desc); + case BOOLEAN: + return ParquetValueWriters.booleans(desc); + case INT32: + return ints(fType, desc); + case INT64: + return ParquetValueWriters.longs(desc); + case FLOAT: + return ParquetValueWriters.floats(desc); + case DOUBLE: + return ParquetValueWriters.doubles(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static ParquetValueWriters.PrimitiveWriter ints( + LogicalType type, ColumnDescriptor desc) { + if (type instanceof TinyIntType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof SmallIntType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + + private static ParquetValueWriters.PrimitiveWriter strings(ColumnDescriptor desc) { + return new StringDataWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter timeMicros(ColumnDescriptor desc) { + return new TimeMicrosWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( + ColumnDescriptor desc, int precision, int scale) { + Preconditions.checkArgument( + precision <= 9, + "Cannot write decimal value as integer with precision larger than 9," + + " wrong precision %s", + precision); + return new IntegerDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsLong( + ColumnDescriptor desc, int precision, int scale) { + Preconditions.checkArgument( + precision <= 18, + "Cannot write decimal value as long with precision larger than 18, " + + " wrong precision %s", + precision); + return new LongDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( + ColumnDescriptor desc, int precision, int scale) { + return new FixedDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter timestamps( + ColumnDescriptor desc) { + return new TimestampDataWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { + return new ByteArrayWriter(desc); + } + + private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter { + private StringDataWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, StringData value) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes())); + } + } + + private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter { + private TimeMicrosWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, Integer value) { + long micros = value.longValue() * 1000; + column.writeLong(repetitionLevel, micros); + } + } + + private static class IntegerDecimalWriter + extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + + private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); + } + } + + private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + + private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeLong(repetitionLevel, decimal.toUnscaledLong()); + } + } + + private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + byte[] binary = + DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get()); + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); + } + } + + private static class TimestampDataWriter + extends ParquetValueWriters.PrimitiveWriter { + private TimestampDataWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, TimestampData value) { + column.writeLong( + repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000); + } + } + + private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { + private ByteArrayWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, byte[] bytes) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); + } + } + + private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { + private final LogicalType elementType; + + private ArrayDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter writer, + LogicalType elementType) { + super(definitionLevel, repetitionLevel, writer); + this.elementType = elementType; + } + + @Override + protected Iterator elements(ArrayData list) { + return new ElementIterator<>(list); + } + + private class ElementIterator implements Iterator { + private final int size; + private final ArrayData list; + private final ArrayData.ElementGetter getter; + private int index; + + private ElementIterator(ArrayData list) { + this.list = list; + size = list.size(); + getter = ArrayData.createElementGetter(elementType); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public E next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + E element = (E) getter.getElementOrNull(list, index); + index += 1; + + return element; + } + } + } + + private static class MapDataWriter + extends ParquetValueWriters.RepeatedKeyValueWriter { + private final LogicalType keyType; + private final LogicalType valueType; + + private MapDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + super(definitionLevel, repetitionLevel, keyWriter, valueWriter); + this.keyType = keyType; + this.valueType = valueType; + } + + @Override + protected Iterator> pairs(MapData map) { + return new EntryIterator<>(map); + } + + private class EntryIterator implements Iterator> { + private final int size; + private final ArrayData keys; + private final ArrayData values; + private final ParquetValueReaders.ReusableEntry entry; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + private int index; + + private EntryIterator(MapData map) { + size = map.size(); + keys = map.keyArray(); + values = map.valueArray(); + entry = new ParquetValueReaders.ReusableEntry<>(); + keyGetter = ArrayData.createElementGetter(keyType); + valueGetter = ArrayData.createElementGetter(valueType); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public Map.Entry next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + entry.set( + (K) keyGetter.getElementOrNull(keys, index), + (V) valueGetter.getElementOrNull(values, index)); + index += 1; + + return entry; + } + } + } + + private static class RowDataWriter extends ParquetValueWriters.StructWriter { + private final RowData.FieldGetter[] fieldGetter; + + RowDataWriter(List> writers, List types) { + super(writers); + fieldGetter = new RowData.FieldGetter[types.size()]; + for (int i = 0; i < types.size(); i += 1) { + fieldGetter[i] = RowData.createFieldGetter(types.get(i), i); + } + } + + @Override + protected Object get(RowData struct, int index) { + return fieldGetter[index].getFieldOrNull(struct); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java new file mode 100644 index 000000000000..ba4e1a7a7aec --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -0,0 +1,161 @@ +/* + * 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.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +abstract class FlinkSchemaVisitor { + + static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { + return visit(flinkType, schema.asStruct(), visitor); + } + + private static T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor visitor) { + switch (iType.typeId()) { + case STRUCT: + return visitRecord(flinkType, iType.asStructType(), visitor); + + case MAP: + MapType mapType = (MapType) flinkType; + Types.MapType iMapType = iType.asMapType(); + T key; + T value; + + Types.NestedField keyField = iMapType.field(iMapType.keyId()); + visitor.beforeMapKey(keyField); + try { + key = visit(mapType.getKeyType(), iMapType.keyType(), visitor); + } finally { + visitor.afterMapKey(keyField); + } + + Types.NestedField valueField = iMapType.field(iMapType.valueId()); + visitor.beforeMapValue(valueField); + try { + value = visit(mapType.getValueType(), iMapType.valueType(), visitor); + } finally { + visitor.afterMapValue(valueField); + } + + return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType()); + + case LIST: + ArrayType listType = (ArrayType) flinkType; + Types.ListType iListType = iType.asListType(); + T element; + + Types.NestedField elementField = iListType.field(iListType.elementId()); + visitor.beforeListElement(elementField); + try { + element = visit(listType.getElementType(), iListType.elementType(), visitor); + } finally { + visitor.afterListElement(elementField); + } + + return visitor.list(iListType, element, listType.getElementType()); + + default: + return visitor.primitive(iType.asPrimitiveType(), flinkType); + } + } + + private static T visitRecord( + LogicalType flinkType, Types.StructType struct, FlinkSchemaVisitor visitor) { + Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType); + RowType rowType = (RowType) flinkType; + + int fieldSize = struct.fields().size(); + List results = Lists.newArrayListWithExpectedSize(fieldSize); + List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); + List nestedFields = struct.fields(); + + for (int i = 0; i < fieldSize; i++) { + Types.NestedField iField = nestedFields.get(i); + int fieldIndex = rowType.getFieldIndex(iField.name()); + Preconditions.checkArgument( + fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); + + LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); + + fieldTypes.add(fieldFlinkType); + + visitor.beforeField(iField); + try { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } finally { + visitor.afterField(iField); + } + } + + return visitor.record(struct, results, fieldTypes); + } + + public T record(Types.StructType iStruct, List results, List fieldTypes) { + return null; + } + + public T list(Types.ListType iList, T element, LogicalType elementType) { + return null; + } + + public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) { + return null; + } + + public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { + return null; + } + + public void beforeField(Types.NestedField field) {} + + public void afterField(Types.NestedField field) {} + + public void beforeListElement(Types.NestedField elementField) { + beforeField(elementField); + } + + public void afterListElement(Types.NestedField elementField) { + afterField(elementField); + } + + public void beforeMapKey(Types.NestedField keyField) { + beforeField(keyField); + } + + public void afterMapKey(Types.NestedField keyField) { + afterField(keyField); + } + + public void beforeMapValue(Types.NestedField valueField) { + beforeField(valueField); + } + + public void afterMapValue(Types.NestedField valueField) { + afterField(valueField); + } +} 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 new file mode 100644 index 000000000000..32f6c3a2ccfd --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.avro.io.Decoder; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +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; + +public class FlinkValueReaders { + + private FlinkValueReaders() {} + + static ValueReader strings() { + return StringReader.INSTANCE; + } + + static ValueReader enums(List symbols) { + return new EnumReader(symbols); + } + + static ValueReader uuids() { + return ValueReaders.fixed(16); + } + + static ValueReader timeMicros() { + return TimeMicrosReader.INSTANCE; + } + + static ValueReader timestampMills() { + return TimestampMillsReader.INSTANCE; + } + + static ValueReader timestampMicros() { + return TimestampMicrosReader.INSTANCE; + } + + static ValueReader decimal( + ValueReader unscaledReader, int precision, int scale) { + return new DecimalReader(unscaledReader, precision, scale); + } + + static ValueReader array(ValueReader elementReader) { + return new ArrayReader(elementReader); + } + + static ValueReader arrayMap(ValueReader keyReader, ValueReader valueReader) { + return new ArrayMapReader(keyReader, valueReader); + } + + static ValueReader map(ValueReader keyReader, ValueReader valueReader) { + return new MapReader(keyReader, valueReader); + } + + static ValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements ValueReader { + private static final StringReader INSTANCE = new StringReader(); + + private StringReader() {} + + @Override + public StringData read(Decoder decoder, Object reuse) throws IOException { + // use the decoder's readString(Utf8) method because it may be a resolving decoder + Utf8 utf8 = null; + if (reuse instanceof StringData) { + utf8 = new Utf8(((StringData) reuse).toBytes()); + } + + Utf8 string = decoder.readString(utf8); + return StringData.fromBytes(string.getBytes(), 0, string.getByteLength()); + } + } + + private static class EnumReader implements ValueReader { + private final StringData[] symbols; + + private EnumReader(List symbols) { + this.symbols = new StringData[symbols.size()]; + for (int i = 0; i < this.symbols.length; i += 1) { + this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); + } + } + + @Override + public StringData read(Decoder decoder, Object ignore) throws IOException { + int index = decoder.readEnum(); + return symbols[index]; + } + } + + private static class DecimalReader implements ValueReader { + private final ValueReader bytesReader; + private final int precision; + private final int scale; + + private DecimalReader(ValueReader bytesReader, int precision, int scale) { + this.bytesReader = bytesReader; + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(Decoder decoder, Object reuse) throws IOException { + byte[] bytes = bytesReader.read(decoder, null); + return DecimalData.fromBigDecimal( + new BigDecimal(new BigInteger(bytes), scale), precision, scale); + } + } + + private static class TimeMicrosReader implements ValueReader { + private static final TimeMicrosReader INSTANCE = new TimeMicrosReader(); + + @Override + public Integer read(Decoder decoder, Object reuse) throws IOException { + long micros = decoder.readLong(); + // Flink only support time mills, just erase micros. + return (int) (micros / 1000); + } + } + + private static class TimestampMillsReader implements ValueReader { + private static final TimestampMillsReader INSTANCE = new TimestampMillsReader(); + + @Override + public TimestampData read(Decoder decoder, Object reuse) throws IOException { + return TimestampData.fromEpochMillis(decoder.readLong()); + } + } + + private static class TimestampMicrosReader implements ValueReader { + private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader(); + + @Override + public TimestampData read(Decoder decoder, Object reuse) throws IOException { + long micros = decoder.readLong(); + long mills = micros / 1000; + int nanos = ((int) (micros % 1000)) * 1000; + if (nanos < 0) { + nanos += 1_000_000; + mills -= 1; + } + return TimestampData.fromEpochMillis(mills, nanos); + } + } + + private static class ArrayReader implements ValueReader { + private final ValueReader elementReader; + private final List reusedList = Lists.newArrayList(); + + private ArrayReader(ValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { + reusedList.clear(); + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedList.add(elementReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + // this will convert the list to an array so it is okay to reuse the list + return new GenericArrayData(reusedList.toArray()); + } + } + + private static MapData kvArrayToMap(List keyList, List valueList) { + Map map = Maps.newHashMap(); + Object[] keys = keyList.toArray(); + Object[] values = valueList.toArray(); + for (int i = 0; i < keys.length; i++) { + map.put(keys[i], values[i]); + } + + return new GenericMapData(map); + } + + private static class ArrayMapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + return kvArrayToMap(reusedKeyList, reusedValueList); + } + } + + private static class MapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private MapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readMapStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.mapNext(); + } + + return kvArrayToMap(reusedKeyList, reusedValueList); + } + } + + private static class StructReader extends ValueReaders.StructReader { + private final int numFields; + + private StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = readers.size(); + } + + @Override + protected RowData reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { + return (GenericRowData) 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); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java new file mode 100644 index 000000000000..4e86ecce28b5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java @@ -0,0 +1,253 @@ +/* + * 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.lang.reflect.Array; +import java.util.List; +import org.apache.avro.io.Encoder; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; + +public class FlinkValueWriters { + + private FlinkValueWriters() {} + + static ValueWriter strings() { + return StringWriter.INSTANCE; + } + + static ValueWriter timeMicros() { + return TimeMicrosWriter.INSTANCE; + } + + static ValueWriter timestampMicros() { + return TimestampMicrosWriter.INSTANCE; + } + + static ValueWriter decimal(int precision, int scale) { + return new DecimalWriter(precision, scale); + } + + static ValueWriter array(ValueWriter elementWriter, LogicalType elementType) { + return new ArrayWriter<>(elementWriter, elementType); + } + + static ValueWriter arrayMap( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter map( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter row(List> writers, List types) { + return new RowWriter(writers, types); + } + + private static class StringWriter implements ValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + private StringWriter() {} + + @Override + public void write(StringData s, Encoder encoder) throws IOException { + // toBytes is cheaper than Avro calling toString, which incurs encoding costs + encoder.writeString(new Utf8(s.toBytes())); + } + } + + private static class DecimalWriter implements ValueWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private DecimalWriter(int precision, int scale) { + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(DecimalData d, Encoder encoder) throws IOException { + encoder.writeFixed( + DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get())); + } + } + + private static class TimeMicrosWriter implements ValueWriter { + private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter(); + + @Override + public void write(Integer timeMills, Encoder encoder) throws IOException { + encoder.writeLong(timeMills * 1000L); + } + } + + private static class TimestampMicrosWriter implements ValueWriter { + private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter(); + + @Override + public void write(TimestampData timestampData, Encoder encoder) throws IOException { + long micros = + timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; + encoder.writeLong(micros); + } + } + + private static class ArrayWriter implements ValueWriter { + private final ValueWriter elementWriter; + private final ArrayData.ElementGetter elementGetter; + + private ArrayWriter(ValueWriter elementWriter, LogicalType elementType) { + this.elementWriter = elementWriter; + this.elementGetter = ArrayData.createElementGetter(elementType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(ArrayData array, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = array.size(); + encoder.setItemCount(numElements); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class ArrayMapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private ArrayMapWriter( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueWriter = valueWriter; + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = map.size(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); + valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class MapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private MapWriter( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueWriter = valueWriter; + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeMapStart(); + int numElements = map.size(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); + valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); + } + encoder.writeMapEnd(); + } + } + + static class RowWriter implements ValueWriter { + private final ValueWriter[] writers; + private final RowData.FieldGetter[] getters; + + private RowWriter(List> writers, List types) { + this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); + this.getters = new RowData.FieldGetter[writers.size()]; + for (int i = 0; i < writers.size(); i += 1) { + this.writers[i] = writers.get(i); + this.getters[i] = RowData.createFieldGetter(types.get(i), i); + } + } + + @Override + public void write(RowData row, Encoder encoder) throws IOException { + for (int i = 0; i < writers.length; i += 1) { + if (row.isNullAt(i)) { + writers[i].write(null, encoder); + } else { + write(row, i, writers[i], encoder); + } + } + } + + @SuppressWarnings("unchecked") + private void write(RowData row, int pos, ValueWriter writer, Encoder encoder) + throws IOException { + writer.write((T) getters[pos].getFieldOrNull(row), encoder); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java new file mode 100644 index 000000000000..33feb2e32118 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java @@ -0,0 +1,222 @@ +/* + * 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.util.Deque; +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class ParquetWithFlinkSchemaVisitor { + private final Deque fieldNames = Lists.newLinkedList(); + + public static T visit( + LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor visitor) { + Preconditions.checkArgument(sType != null, "Invalid DataType: null"); + if (type instanceof MessageType) { + Preconditions.checkArgument( + sType instanceof RowType, "Invalid struct: %s is not a struct", sType); + RowType struct = (RowType) sType; + return visitor.message( + struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); + } else if (type.isPrimitive()) { + return visitor.primitive(sType, type.asPrimitiveType()); + } else { + // if not a primitive, the typeId must be a group + GroupType group = type.asGroupType(); + OriginalType annotation = group.getOriginalType(); + if (annotation != null) { + switch (annotation) { + case LIST: + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid list: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid list: does not contain single repeated field: %s", + group); + + GroupType repeatedElement = group.getFields().get(0).asGroupType(); + Preconditions.checkArgument( + repeatedElement.isRepetition(Type.Repetition.REPEATED), + "Invalid list: inner group is not repeated"); + Preconditions.checkArgument( + repeatedElement.getFieldCount() <= 1, + "Invalid list: repeated group is not a single field: %s", + group); + + Preconditions.checkArgument( + sType instanceof ArrayType, "Invalid list: %s is not an array", sType); + ArrayType array = (ArrayType) sType; + RowType.RowField element = + new RowField( + "element", array.getElementType(), "element of " + array.asSummaryString()); + + visitor.fieldNames.push(repeatedElement.getName()); + try { + T elementResult = null; + if (repeatedElement.getFieldCount() > 0) { + elementResult = visitField(element, repeatedElement.getType(0), visitor); + } + + return visitor.list(array, group, elementResult); + + } finally { + visitor.fieldNames.pop(); + } + + case MAP: + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid map: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid map: does not contain single repeated field: %s", + group); + + GroupType repeatedKeyValue = group.getType(0).asGroupType(); + Preconditions.checkArgument( + repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), + "Invalid map: inner group is not repeated"); + Preconditions.checkArgument( + repeatedKeyValue.getFieldCount() <= 2, + "Invalid map: repeated group does not have 2 fields"); + + Preconditions.checkArgument( + sType instanceof MapType, "Invalid map: %s is not a map", sType); + MapType map = (MapType) sType; + RowField keyField = + new RowField("key", map.getKeyType(), "key of " + map.asSummaryString()); + RowField valueField = + new RowField("value", map.getValueType(), "value of " + map.asSummaryString()); + + visitor.fieldNames.push(repeatedKeyValue.getName()); + try { + T keyResult = null; + T valueResult = null; + switch (repeatedKeyValue.getFieldCount()) { + case 2: + // if there are 2 fields, both key and value are projected + keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); + valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); + break; + case 1: + // if there is just one, use the name to determine what it is + Type keyOrValue = repeatedKeyValue.getType(0); + if (keyOrValue.getName().equalsIgnoreCase("key")) { + keyResult = visitField(keyField, keyOrValue, visitor); + // value result remains null + } else { + valueResult = visitField(valueField, keyOrValue, visitor); + // key result remains null + } + break; + default: + // both results will remain null + } + + return visitor.map(map, group, keyResult, valueResult); + + } finally { + visitor.fieldNames.pop(); + } + + default: + } + } + Preconditions.checkArgument( + sType instanceof RowType, "Invalid struct: %s is not a struct", sType); + RowType struct = (RowType) sType; + return visitor.struct(struct, group, visitFields(struct, group, visitor)); + } + } + + private static T visitField( + RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor visitor) { + visitor.fieldNames.push(field.getName()); + try { + return visit(sField.getType(), field, visitor); + } finally { + visitor.fieldNames.pop(); + } + } + + private static List visitFields( + RowType struct, GroupType group, ParquetWithFlinkSchemaVisitor visitor) { + List sFields = struct.getFields(); + Preconditions.checkArgument( + sFields.size() == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); + List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); + for (int i = 0; i < sFields.size(); i += 1) { + Type field = group.getFields().get(i); + RowType.RowField sField = sFields.get(i); + Preconditions.checkArgument( + field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())), + "Structs do not match: field %s != %s", + field.getName(), + sField.getName()); + results.add(visitField(sField, field, visitor)); + } + + return results; + } + + public T message(RowType sStruct, MessageType message, List fields) { + return null; + } + + public T struct(RowType sStruct, GroupType struct, List fields) { + return null; + } + + public T list(ArrayType sArray, GroupType array, T element) { + return null; + } + + public T map(MapType sMap, GroupType map, T key, T value) { + return null; + } + + public T primitive(LogicalType sPrimitive, PrimitiveType primitive) { + return null; + } + + protected String[] currentPath() { + return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); + } + + protected String[] path(String name) { + List list = Lists.newArrayList(fieldNames.descendingIterator()); + list.add(name); + return list.toArray(new String[0]); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java new file mode 100644 index 000000000000..33816c97ac29 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java @@ -0,0 +1,341 @@ +/* + * 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.util.Arrays; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public class RowDataProjection implements RowData { + /** + * Creates a projecting wrapper for {@link RowData} rows. + * + *

This projection will not project the nested children types of repeated types like lists and + * maps. + * + * @param schema schema of rows wrapped by this projection + * @param projectedSchema result schema of the projected rows + * @return a wrapper to project rows + */ + public static RowDataProjection create(Schema schema, Schema projectedSchema) { + return RowDataProjection.create( + FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct()); + } + + /** + * Creates a projecting wrapper for {@link RowData} rows. + * + *

This projection will not project the nested children types of repeated types like lists and + * maps. + * + * @param rowType flink row type of rows wrapped by this projection + * @param schema schema of rows wrapped by this projection + * @param projectedSchema result schema of the projected rows + * @return a wrapper to project rows + */ + public static RowDataProjection create( + RowType rowType, Types.StructType schema, Types.StructType projectedSchema) { + return new RowDataProjection(rowType, schema, projectedSchema); + } + + private final RowData.FieldGetter[] getters; + private RowData rowData; + + private RowDataProjection( + RowType rowType, Types.StructType rowStruct, Types.StructType projectType) { + Map fieldIdToPosition = Maps.newHashMap(); + for (int i = 0; i < rowStruct.fields().size(); i++) { + fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i); + } + + this.getters = new RowData.FieldGetter[projectType.fields().size()]; + for (int i = 0; i < getters.length; i++) { + Types.NestedField projectField = projectType.fields().get(i); + Types.NestedField rowField = rowStruct.field(projectField.fieldId()); + + Preconditions.checkNotNull( + rowField, + "Cannot locate the project field <%s> in the iceberg struct <%s>", + projectField, + rowStruct); + + getters[i] = + createFieldGetter( + rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField); + } + } + + private static RowData.FieldGetter createFieldGetter( + RowType rowType, int position, Types.NestedField rowField, Types.NestedField projectField) { + Preconditions.checkArgument( + rowField.type().typeId() == projectField.type().typeId(), + "Different iceberg type between row field <%s> and project field <%s>", + rowField, + projectField); + + switch (projectField.type().typeId()) { + case STRUCT: + RowType nestedRowType = (RowType) rowType.getTypeAt(position); + return row -> { + // null nested struct value + if (row.isNullAt(position)) { + return null; + } + + RowData nestedRow = row.getRow(position, nestedRowType.getFieldCount()); + return RowDataProjection.create( + nestedRowType, rowField.type().asStructType(), projectField.type().asStructType()) + .wrap(nestedRow); + }; + + case MAP: + Types.MapType projectedMap = projectField.type().asMapType(); + Types.MapType originalMap = rowField.type().asMapType(); + + boolean keyProjectable = + !projectedMap.keyType().isNestedType() + || projectedMap.keyType().equals(originalMap.keyType()); + boolean valueProjectable = + !projectedMap.valueType().isNestedType() + || projectedMap.valueType().equals(originalMap.valueType()); + Preconditions.checkArgument( + keyProjectable && valueProjectable, + "Cannot project a partial map key or value with non-primitive type. Trying to project <%s> out of <%s>", + projectField, + rowField); + + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + + case LIST: + Types.ListType projectedList = projectField.type().asListType(); + Types.ListType originalList = rowField.type().asListType(); + + boolean elementProjectable = + !projectedList.elementType().isNestedType() + || projectedList.elementType().equals(originalList.elementType()); + Preconditions.checkArgument( + elementProjectable, + "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>", + projectField, + rowField); + + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + + default: + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + } + } + + public RowData wrap(RowData row) { + // StructProjection allow wrapping null root struct object. + // See more discussions in https://github.com/apache/iceberg/pull/7517. + // RowDataProjection never allowed null root object to be wrapped. + // Hence, it is fine to enforce strict Preconditions check here. + Preconditions.checkArgument(row != null, "Invalid row data: null"); + this.rowData = row; + return this; + } + + private Object getValue(int pos) { + Preconditions.checkState(rowData != null, "Row data not wrapped"); + return getters[pos].getFieldOrNull(rowData); + } + + @Override + public int getArity() { + return getters.length; + } + + @Override + public RowKind getRowKind() { + Preconditions.checkState(rowData != null, "Row data not wrapped"); + return rowData.getRowKind(); + } + + @Override + public void setRowKind(RowKind kind) { + throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection"); + } + + @Override + public boolean isNullAt(int pos) { + return getValue(pos) == null; + } + + @Override + public boolean getBoolean(int pos) { + return (boolean) getValue(pos); + } + + @Override + public byte getByte(int pos) { + return (byte) getValue(pos); + } + + @Override + public short getShort(int pos) { + return (short) getValue(pos); + } + + @Override + public int getInt(int pos) { + return (int) getValue(pos); + } + + @Override + public long getLong(int pos) { + return (long) getValue(pos); + } + + @Override + public float getFloat(int pos) { + return (float) getValue(pos); + } + + @Override + public double getDouble(int pos) { + return (double) getValue(pos); + } + + @Override + public StringData getString(int pos) { + return (StringData) getValue(pos); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return (DecimalData) getValue(pos); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return (TimestampData) getValue(pos); + } + + @Override + @SuppressWarnings("unchecked") + public RawValueData getRawValue(int pos) { + return (RawValueData) getValue(pos); + } + + @Override + public byte[] getBinary(int pos) { + return (byte[]) getValue(pos); + } + + @Override + public ArrayData getArray(int pos) { + return (ArrayData) getValue(pos); + } + + @Override + public MapData getMap(int pos) { + return (MapData) getValue(pos); + } + + @Override + public RowData getRow(int pos, int numFields) { + return (RowData) getValue(pos); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof RowDataProjection)) { + return false; + } + + RowDataProjection that = (RowDataProjection) o; + return deepEquals(that); + } + + @Override + public int hashCode() { + int result = Objects.hashCode(getRowKind()); + for (int pos = 0; pos < getArity(); pos++) { + if (!isNullAt(pos)) { + // Arrays.deepHashCode handles array object properly + result = 31 * result + Arrays.deepHashCode(new Object[] {getValue(pos)}); + } + } + + return result; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getRowKind().shortString()).append("("); + for (int pos = 0; pos < getArity(); pos++) { + if (pos != 0) { + sb.append(","); + } + // copied the behavior from Flink GenericRowData + sb.append(StringUtils.arrayAwareToString(getValue(pos))); + } + + sb.append(")"); + return sb.toString(); + } + + private boolean deepEquals(RowDataProjection other) { + if (getRowKind() != other.getRowKind()) { + return false; + } + + if (getArity() != other.getArity()) { + return false; + } + + for (int pos = 0; pos < getArity(); ++pos) { + if (isNullAt(pos) && other.isNullAt(pos)) { + continue; + } + + if ((isNullAt(pos) && !other.isNullAt(pos)) || (!isNullAt(pos) && other.isNullAt(pos))) { + return false; + } + + // Objects.deepEquals handles array object properly + if (!Objects.deepEquals(getValue(pos), other.getValue(pos))) { + return false; + } + } + + return true; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java new file mode 100644 index 000000000000..c5cb51b7eae4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -0,0 +1,100 @@ +/* + * 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.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; + +public class RowDataUtil { + + private RowDataUtil() {} + + public static Object convertConstant(Type type, Object value) { + if (value == null) { + return null; + } + + switch (type.typeId()) { + case DECIMAL: // DecimalData + Types.DecimalType decimal = (Types.DecimalType) type; + return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale()); + case STRING: // StringData + if (value instanceof Utf8) { + Utf8 utf8 = (Utf8) value; + return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); + } + return StringData.fromString(value.toString()); + case FIXED: // byte[] + if (value instanceof byte[]) { + return value; + } else if (value instanceof GenericData.Fixed) { + return ((GenericData.Fixed) value).bytes(); + } + return ByteBuffers.toByteArray((ByteBuffer) value); + case BINARY: // byte[] + return ByteBuffers.toByteArray((ByteBuffer) value); + case TIME: // int mills instead of long + return (int) ((Long) value / 1000); + case TIMESTAMP: // TimestampData + return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value)); + default: + } + return value; + } + + /** + * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method. This + * skips the check the arity of rowType and from, because the from RowData may contains additional + * column for position deletes. Using {@link RowDataSerializer#copy(RowData, RowData)} will fail + * the arity check. + */ + public static RowData clone( + RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + GenericRowData ret; + if (reuse instanceof GenericRowData) { + ret = (GenericRowData) reuse; + } else { + ret = new GenericRowData(from.getArity()); + } + ret.setRowKind(from.getRowKind()); + for (int i = 0; i < rowType.getFieldCount(); i++) { + if (!from.isNullAt(i)) { + RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); + ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); + } else { + ret.setField(i, null); + } + } + return ret; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java new file mode 100644 index 000000000000..1019285018d0 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -0,0 +1,300 @@ +/* + * 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.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.ByteBuffers; + +@Internal +public class StructRowData implements RowData { + private final Types.StructType type; + private RowKind kind; + private StructLike struct; + + public StructRowData(Types.StructType type) { + this(type, RowKind.INSERT); + } + + public StructRowData(Types.StructType type, RowKind kind) { + this(type, null, kind); + } + + private StructRowData(Types.StructType type, StructLike struct) { + this(type, struct, RowKind.INSERT); + } + + private StructRowData(Types.StructType type, StructLike struct, RowKind kind) { + this.type = type; + this.struct = struct; + this.kind = kind; + } + + public StructRowData setStruct(StructLike newStruct) { + this.struct = newStruct; + return this; + } + + @Override + public int getArity() { + return struct.size(); + } + + @Override + public RowKind getRowKind() { + return kind; + } + + @Override + public void setRowKind(RowKind newKind) { + Preconditions.checkNotNull(newKind, "kind can not be null"); + this.kind = newKind; + } + + @Override + public boolean isNullAt(int pos) { + return struct.get(pos, Object.class) == null; + } + + @Override + public boolean getBoolean(int pos) { + return struct.get(pos, Boolean.class); + } + + @Override + public byte getByte(int pos) { + return (byte) (int) struct.get(pos, Integer.class); + } + + @Override + public short getShort(int pos) { + return (short) (int) struct.get(pos, Integer.class); + } + + @Override + public int getInt(int pos) { + Object integer = struct.get(pos, Object.class); + + if (integer instanceof Integer) { + return (int) integer; + } else if (integer instanceof LocalDate) { + return (int) ((LocalDate) integer).toEpochDay(); + } else if (integer instanceof LocalTime) { + return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000); + } else { + throw new IllegalStateException( + "Unknown type for int field. Type name: " + integer.getClass().getName()); + } + } + + @Override + public long getLong(int pos) { + Object longVal = struct.get(pos, Object.class); + + if (longVal instanceof Long) { + return (long) longVal; + } else if (longVal instanceof OffsetDateTime) { + return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; + } else if (longVal instanceof LocalDate) { + return ((LocalDate) longVal).toEpochDay(); + } else if (longVal instanceof LocalTime) { + return ((LocalTime) longVal).toNanoOfDay(); + } else if (longVal instanceof LocalDateTime) { + return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC)) + .toNanos() + / 1000; + } else { + throw new IllegalStateException( + "Unknown type for long field. Type name: " + longVal.getClass().getName()); + } + } + + @Override + public float getFloat(int pos) { + return struct.get(pos, Float.class); + } + + @Override + public double getDouble(int pos) { + return struct.get(pos, Double.class); + } + + @Override + public StringData getString(int pos) { + return isNullAt(pos) ? null : getStringDataInternal(pos); + } + + private StringData getStringDataInternal(int pos) { + CharSequence seq = struct.get(pos, CharSequence.class); + return StringData.fromString(seq.toString()); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return isNullAt(pos) + ? null + : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale); + } + + private BigDecimal getDecimalInternal(int pos) { + return struct.get(pos, BigDecimal.class); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + long timeLong = getLong(pos); + return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public byte[] getBinary(int pos) { + return isNullAt(pos) ? null : getBinaryInternal(pos); + } + + private byte[] getBinaryInternal(int pos) { + Object bytes = struct.get(pos, Object.class); + + // should only be either ByteBuffer or byte[] + if (bytes instanceof ByteBuffer) { + return ByteBuffers.toByteArray((ByteBuffer) bytes); + } else if (bytes instanceof byte[]) { + return (byte[]) bytes; + } else if (bytes instanceof UUID) { + UUID uuid = (UUID) bytes; + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + return bb.array(); + } else { + throw new IllegalStateException( + "Unknown type for binary field. Type name: " + bytes.getClass().getName()); + } + } + + @Override + public ArrayData getArray(int pos) { + return isNullAt(pos) + ? null + : (ArrayData) + convertValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class)); + } + + @Override + public MapData getMap(int pos) { + return isNullAt(pos) + ? null + : (MapData) + convertValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class)); + } + + @Override + public RowData getRow(int pos, int numFields) { + return isNullAt(pos) ? null : getStructRowData(pos, numFields); + } + + private StructRowData getStructRowData(int pos, int numFields) { + return new StructRowData( + type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); + } + + private Object convertValue(Type elementType, Object value) { + switch (elementType.typeId()) { + case BOOLEAN: + case INTEGER: + case DATE: + case TIME: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + return value; + case TIMESTAMP: + long millisecond = (long) value / 1000; + int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000; + return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); + case STRING: + return StringData.fromString(value.toString()); + case FIXED: + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) value); + case STRUCT: + return new StructRowData(elementType.asStructType(), (StructLike) value); + case LIST: + List list = (List) value; + Object[] array = new Object[list.size()]; + + int index = 0; + for (Object element : list) { + if (element == null) { + array[index] = null; + } else { + array[index] = convertValue(elementType.asListType().elementType(), element); + } + + index += 1; + } + return new GenericArrayData(array); + case MAP: + Types.MapType mapType = elementType.asMapType(); + Set> entries = ((Map) value).entrySet(); + Map result = Maps.newHashMap(); + for (Map.Entry entry : entries) { + final Object keyValue = convertValue(mapType.keyType(), entry.getKey()); + final Object valueValue = convertValue(mapType.valueType(), entry.getValue()); + result.put(keyValue, valueValue); + } + + return new GenericMapData(result); + default: + throw new UnsupportedOperationException("Unsupported element type: " + elementType); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java new file mode 100644 index 000000000000..04e168385a36 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java @@ -0,0 +1,61 @@ +/* + * 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 org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; + +/** + * This util class converts Avro GenericRecord to Flink RowData.
+ *
+ * Internally it uses Flink {@link AvroToRowDataConverters}. Because of the precision difference + * between how Iceberg schema (micro) and Flink {@link AvroToRowDataConverters} (milli) deal with + * time type, we can't directly use the Avro Schema converted from Iceberg schema via {@link + * AvroSchemaUtil#convert(org.apache.iceberg.Schema, String)}. + */ +public class AvroGenericRecordToRowDataMapper implements MapFunction { + + private final AvroToRowDataConverters.AvroToRowDataConverter converter; + + AvroGenericRecordToRowDataMapper(RowType rowType) { + this.converter = AvroToRowDataConverters.createRowConverter(rowType); + } + + @Override + public RowData map(GenericRecord genericRecord) throws Exception { + return (RowData) converter.convert(genericRecord); + } + + /** Create a mapper based on Avro schema. */ + public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + return new AvroGenericRecordToRowDataMapper(rowType); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java new file mode 100644 index 000000000000..40e0b5f2a34e --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.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.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RowDataProjection; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; + +abstract class BaseDeltaTaskWriter extends BaseTaskWriter { + + private final Schema schema; + private final Schema deleteSchema; + private final RowDataWrapper wrapper; + private final RowDataWrapper keyWrapper; + private final RowDataProjection keyProjection; + private final boolean upsert; + + BaseDeltaTaskWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.schema = schema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + this.keyWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); + this.keyProjection = + RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); + this.upsert = upsert; + } + + abstract RowDataDeltaWriter route(RowData row); + + RowDataWrapper wrapper() { + return wrapper; + } + + @Override + public void write(RowData row) throws IOException { + RowDataDeltaWriter writer = route(row); + + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + if (upsert) { + writer.deleteKey(keyProjection.wrap(row)); + } + writer.write(row); + break; + + case UPDATE_BEFORE: + if (upsert) { + break; // UPDATE_BEFORE is not necessary for UPSERT, we do nothing to prevent delete one + // row twice + } + writer.delete(row); + break; + case DELETE: + if (upsert) { + writer.deleteKey(keyProjection.wrap(row)); + } else { + writer.delete(row); + } + break; + + default: + throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind()); + } + } + + protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { + RowDataDeltaWriter(PartitionKey partition) { + super(partition, schema, deleteSchema); + } + + @Override + protected StructLike asStructLike(RowData data) { + return wrapper.wrap(data); + } + + @Override + protected StructLike asStructLikeKey(RowData data) { + return keyWrapper.wrap(data); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java new file mode 100644 index 000000000000..1cb6e013bd2c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -0,0 +1,70 @@ +/* + * 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.stream.IntStream; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; + +/** + * A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key. + * To be used with the {@link BucketPartitioner}. + */ +class BucketPartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + private final int bucketFieldPosition; + + private transient RowDataWrapper rowDataWrapper; + + BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { + this.schema = schema; + this.partitionKey = new PartitionKey(partitionSpec, schema); + this.flinkSchema = flinkSchema; + this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); + } + + private int getBucketFieldPosition(PartitionSpec partitionSpec) { + int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); + return IntStream.range(0, partitionSpec.fields().size()) + .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) + .toArray()[0]; + } + + private RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + return rowDataWrapper; + } + + @Override + public Integer getKey(RowData rowData) { + partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); + return partitionKey.get(bucketFieldPosition, Integer.class); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java new file mode 100644 index 000000000000..9c9a117906e2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.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.flink.sink; + +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This partitioner will redirect records to writers deterministically based on the Bucket partition + * spec. It'll attempt to optimize the file size written depending on whether numPartitions is + * greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE + * bucket in the partition spec. + */ +class BucketPartitioner implements Partitioner { + + static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null"; + static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE = + "Invalid bucket ID %s: must be non-negative."; + static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE = + "Invalid bucket ID %s: must be less than bucket limit: %s."; + + private final int maxNumBuckets; + + // To hold the OFFSET of the next writer to use for any bucket, only used when writers > the + // number of buckets + private final int[] currentBucketWriterOffset; + + BucketPartitioner(PartitionSpec partitionSpec) { + this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); + this.currentBucketWriterOffset = new int[maxNumBuckets]; + } + + /** + * Determine the partition id based on the following criteria: If the number of writers <= the + * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one + * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic + * is handled by the {@link #getPartitionWithMoreWritersThanBuckets + * getPartitionWritersGreaterThanBuckets} method. + * + * @param bucketId the bucketId for each request + * @param numPartitions the total number of partitions + * @return the partition id (writer) to use for each request + */ + @Override + public int partition(Integer bucketId, int numPartitions) { + Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); + Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); + Preconditions.checkArgument( + bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); + + if (numPartitions <= maxNumBuckets) { + return bucketId % numPartitions; + } else { + return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); + } + } + + /*- + * If the number of writers > the number of buckets each partitioner will keep a state of multiple + * writers per bucket as evenly as possible, and will round-robin the requests across them, in this + * case each writer will target only one bucket at all times (many writers -> one bucket). Example: + * Configuration: numPartitions (writers) = 5, maxBuckets = 2 + * Expected behavior: + * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 + * - Records for Bucket 1 will always use Writer 1 and 3 + * Notes: + * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId + * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). + * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. + * + * @return the destination partition index (writer subtask id) + */ + private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { + int currentOffset = currentBucketWriterOffset[bucketId]; + // Determine if this bucket requires an "extra writer" + int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; + // The max number of writers this bucket can have + int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; + + // Increment the writer offset or reset if it's reached the max for this bucket + int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; + currentBucketWriterOffset[bucketId] = nextOffset; + + return bucketId + (maxNumBuckets * currentOffset); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java new file mode 100644 index 000000000000..c33207728d3e --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.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.sink; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.transforms.PartitionSpecVisitor; + +final class BucketPartitionerUtil { + static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = + "Invalid number of buckets: %s (must be 1)"; + + private BucketPartitionerUtil() {} + + /** + * Determines whether the PartitionSpec has one and only one Bucket definition + * + * @param partitionSpec the partition spec in question + * @return whether the PartitionSpec has only one Bucket + */ + static boolean hasOneBucketField(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + return bucketFields != null && bucketFields.size() == 1; + } + + /** + * Extracts the Bucket definition from a PartitionSpec. + * + * @param partitionSpec the partition spec in question + * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) + */ + private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + Preconditions.checkArgument( + bucketFields.size() == 1, + BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, + bucketFields.size()); + return bucketFields.get(0); + } + + static int getBucketFieldId(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f0; + } + + static int getMaxNumBuckets(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f1; + } + + private static List> getBucketFields(PartitionSpec spec) { + return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + private static class BucketPartitionSpecVisitor + implements PartitionSpecVisitor> { + @Override + public Tuple2 identity(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 bucket( + int fieldId, String sourceName, int sourceId, int numBuckets) { + return new Tuple2<>(fieldId, numBuckets); + } + + @Override + public Tuple2 truncate( + int fieldId, String sourceName, int sourceId, int width) { + return null; + } + + @Override + public Tuple2 year(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 month(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 day(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 hour(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 unknown( + int fieldId, String sourceName, int sourceId, String transform) { + return null; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java new file mode 100644 index 000000000000..e9f9786f9190 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java @@ -0,0 +1,91 @@ +/* + * 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.time.Duration; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A table loader that will only reload a table after a certain interval has passed. WARNING: This + * table loader should be used carefully when used with writer tasks. It could result in heavy load + * on a catalog for jobs with many writers. + */ +class CachingTableSupplier implements SerializableSupplier

{ + + private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); + + private final Table initialTable; + private final TableLoader tableLoader; + private final Duration tableRefreshInterval; + private long lastLoadTimeMillis; + private transient Table table; + + CachingTableSupplier( + SerializableTable initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { + Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); + Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); + Preconditions.checkArgument( + tableRefreshInterval != null, "tableRefreshInterval cannot be null"); + this.initialTable = initialTable; + this.table = initialTable; + this.tableLoader = tableLoader; + this.tableRefreshInterval = tableRefreshInterval; + this.lastLoadTimeMillis = System.currentTimeMillis(); + } + + @Override + public Table get() { + if (table == null) { + this.table = initialTable; + } + return table; + } + + Table initialTable() { + return initialTable; + } + + void refreshTable() { + if (System.currentTimeMillis() > lastLoadTimeMillis + tableRefreshInterval.toMillis()) { + try { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.lastLoadTimeMillis = System.currentTimeMillis(); + + LOG.info( + "Table {} reloaded, next min load time threshold is {}", + table.name(), + DateTimeUtil.formatTimestampMillis( + lastLoadTimeMillis + tableRefreshInterval.toMillis())); + } catch (Exception e) { + LOG.warn("An error occurred reloading table {}, table was not reloaded", table.name(), e); + } + } + } +} 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 new file mode 100644 index 000000000000..9a2f57181708 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -0,0 +1,93 @@ +/* + * 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.Arrays; +import java.util.NavigableMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class CommitSummary { + + private final AtomicLong dataFilesCount = new AtomicLong(); + private final AtomicLong dataFilesRecordCount = new AtomicLong(); + private final AtomicLong dataFilesByteCount = new AtomicLong(); + private final AtomicLong deleteFilesCount = new AtomicLong(); + private final AtomicLong deleteFilesRecordCount = new AtomicLong(); + private final AtomicLong deleteFilesByteCount = new AtomicLong(); + + CommitSummary(NavigableMap pendingResults) { + pendingResults + .values() + .forEach( + writeResult -> { + dataFilesCount.addAndGet(writeResult.dataFiles().length); + Arrays.stream(writeResult.dataFiles()) + .forEach( + dataFile -> { + dataFilesRecordCount.addAndGet(dataFile.recordCount()); + dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); + }); + deleteFilesCount.addAndGet(writeResult.deleteFiles().length); + Arrays.stream(writeResult.deleteFiles()) + .forEach( + deleteFile -> { + deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); + deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); + }); + }); + } + + long dataFilesCount() { + return dataFilesCount.get(); + } + + long dataFilesRecordCount() { + return dataFilesRecordCount.get(); + } + + long dataFilesByteCount() { + return dataFilesByteCount.get(); + } + + long deleteFilesCount() { + return deleteFilesCount.get(); + } + + long deleteFilesRecordCount() { + return deleteFilesRecordCount.get(); + } + + long deleteFilesByteCount() { + return deleteFilesByteCount.get(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFilesCount", dataFilesCount) + .add("dataFilesRecordCount", dataFilesRecordCount) + .add("dataFilesByteCount", dataFilesByteCount) + .add("deleteFilesCount", deleteFilesCount) + .add("deleteFilesRecordCount", deleteFilesRecordCount) + .add("deleteFilesByteCount", deleteFilesByteCount) + .toString(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java new file mode 100644 index 000000000000..036970c06d5b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java @@ -0,0 +1,71 @@ +/* + * 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 org.apache.iceberg.ManifestFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class DeltaManifests { + + private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; + + private final ManifestFile dataManifest; + private final ManifestFile deleteManifest; + private final CharSequence[] referencedDataFiles; + + DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { + this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); + } + + DeltaManifests( + ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { + Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); + + this.dataManifest = dataManifest; + this.deleteManifest = deleteManifest; + this.referencedDataFiles = referencedDataFiles; + } + + ManifestFile dataManifest() { + return dataManifest; + } + + ManifestFile deleteManifest() { + return deleteManifest; + } + + CharSequence[] referencedDataFiles() { + return referencedDataFiles; + } + + List manifests() { + List manifests = Lists.newArrayListWithCapacity(2); + if (dataManifest != null) { + manifests.add(dataManifest); + } + + if (deleteManifest != null) { + manifests.add(deleteManifest); + } + + return manifests; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java new file mode 100644 index 000000000000..92ca284b12ba --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.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.flink.sink; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class DeltaManifestsSerializer implements SimpleVersionedSerializer { + private static final int VERSION_1 = 1; + private static final int VERSION_2 = 2; + private static final byte[] EMPTY_BINARY = new byte[0]; + + static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); + + @Override + public int getVersion() { + return VERSION_2; + } + + @Override + public byte[] serialize(DeltaManifests deltaManifests) throws IOException { + Preconditions.checkNotNull( + deltaManifests, "DeltaManifests to be serialized should not be null"); + + ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(binaryOut); + + byte[] dataManifestBinary = EMPTY_BINARY; + if (deltaManifests.dataManifest() != null) { + dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); + } + + out.writeInt(dataManifestBinary.length); + out.write(dataManifestBinary); + + byte[] deleteManifestBinary = EMPTY_BINARY; + if (deltaManifests.deleteManifest() != null) { + deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); + } + + out.writeInt(deleteManifestBinary.length); + out.write(deleteManifestBinary); + + CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); + out.writeInt(referencedDataFiles.length); + for (CharSequence referencedDataFile : referencedDataFiles) { + out.writeUTF(referencedDataFile.toString()); + } + + return binaryOut.toByteArray(); + } + + @Override + public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { + if (version == VERSION_1) { + return deserializeV1(serialized); + } else if (version == VERSION_2) { + return deserializeV2(serialized); + } else { + throw new RuntimeException("Unknown serialize version: " + version); + } + } + + private DeltaManifests deserializeV1(byte[] serialized) throws IOException { + return new DeltaManifests(ManifestFiles.decode(serialized), null); + } + + private DeltaManifests deserializeV2(byte[] serialized) throws IOException { + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(binaryIn); + + int dataManifestSize = in.readInt(); + if (dataManifestSize > 0) { + byte[] dataManifestBinary = new byte[dataManifestSize]; + Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); + + dataManifest = ManifestFiles.decode(dataManifestBinary); + } + + int deleteManifestSize = in.readInt(); + if (deleteManifestSize > 0) { + byte[] deleteManifestBinary = new byte[deleteManifestSize]; + Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); + + deleteManifest = ManifestFiles.decode(deleteManifestBinary); + } + + int referenceDataFileNum = in.readInt(); + CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; + for (int i = 0; i < referenceDataFileNum; i++) { + referencedDataFiles[i] = in.readUTF(); + } + + return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java new file mode 100644 index 000000000000..18b269d6c3e9 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -0,0 +1,86 @@ +/* + * 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 org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.iceberg.util.StructProjection; + +/** + * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record + * will be emitted to same writer in order. + */ +class EqualityFieldKeySelector implements KeySelector { + + private final Schema schema; + private final RowType flinkSchema; + private final Schema deleteSchema; + + private transient RowDataWrapper rowDataWrapper; + private transient StructProjection structProjection; + private transient StructLikeWrapper structLikeWrapper; + + EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { + this.schema = schema; + this.flinkSchema = flinkSchema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + } + + /** + * Construct the {@link RowDataWrapper} lazily here because few members in it are not + * serializable. In this way, we don't have to serialize them with forcing. + */ + protected RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + return rowDataWrapper; + } + + /** Construct the {@link StructProjection} lazily because it is not serializable. */ + protected StructProjection lazyStructProjection() { + if (structProjection == null) { + structProjection = StructProjection.create(schema, deleteSchema); + } + return structProjection; + } + + /** Construct the {@link StructLikeWrapper} lazily because it is not serializable. */ + protected StructLikeWrapper lazyStructLikeWrapper() { + if (structLikeWrapper == null) { + structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct()); + } + return structLikeWrapper; + } + + @Override + public Integer getKey(RowData row) { + RowDataWrapper wrappedRowData = lazyRowDataWrapper().wrap(row); + StructProjection projectedRowData = lazyStructProjection().wrap(wrappedRowData); + StructLikeWrapper wrapper = lazyStructLikeWrapper().set(projectedRowData); + return wrapper.hashCode(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java new file mode 100644 index 000000000000..b6f1392d1562 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java @@ -0,0 +1,274 @@ +/* + * 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.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkOrcWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +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; + +public class FlinkAppenderFactory implements FileAppenderFactory, Serializable { + private final Schema schema; + private final RowType flinkSchema; + private final Map props; + private final PartitionSpec spec; + private final int[] equalityFieldIds; + private final Schema eqDeleteRowSchema; + private final Schema posDeleteRowSchema; + private final Table table; + + private RowType eqDeleteFlinkSchema = null; + private RowType posDeleteFlinkSchema = null; + + public FlinkAppenderFactory( + Table table, + Schema schema, + RowType flinkSchema, + Map props, + PartitionSpec spec, + int[] equalityFieldIds, + Schema eqDeleteRowSchema, + Schema posDeleteRowSchema) { + Preconditions.checkNotNull(table, "Table shouldn't be null"); + this.table = table; + this.schema = schema; + this.flinkSchema = flinkSchema; + this.props = props; + this.spec = spec; + this.equalityFieldIds = equalityFieldIds; + this.eqDeleteRowSchema = eqDeleteRowSchema; + this.posDeleteRowSchema = posDeleteRowSchema; + } + + private RowType lazyEqDeleteFlinkSchema() { + if (eqDeleteFlinkSchema == null) { + Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); + this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema); + } + return eqDeleteFlinkSchema; + } + + private RowType lazyPosDeleteFlinkSchema() { + if (posDeleteFlinkSchema == null) { + Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null"); + this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema); + } + return this.posDeleteFlinkSchema; + } + + @Override + public FileAppender newAppender(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + try { + switch (format) { + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .setAll(props) + .schema(schema) + .metricsConfig(metricsConfig) + .overwrite() + .build(); + + case ORC: + return ORC.write(outputFile) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(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 outputFile, 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"); + + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case ORC: + return ORC.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write equality-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public PositionDeleteWriter newPosDeleteWriter( + EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { + MetricsConfig metricsConfig = MetricsConfig.forPositionDelete(table); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case ORC: + RowType orcPosDeleteSchema = + FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); + return ORC.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .transformPaths(path -> StringData.fromString(path.toString())) + .buildPositionWriter(); + + case PARQUET: + RowType flinkPosDeleteSchema = + FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .transformPaths(path -> StringData.fromString(path.toString())) + .buildPositionWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write pos-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java new file mode 100644 index 000000000000..2183fe062af4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java @@ -0,0 +1,293 @@ +/* + * 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 static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; + +import java.io.Serializable; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkOrcWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { + private RowType dataFlinkType; + private RowType equalityDeleteFlinkType; + private RowType positionDeleteFlinkType; + + FlinkFileWriterFactory( + Table table, + FileFormat dataFileFormat, + Schema dataSchema, + RowType dataFlinkType, + SortOrder dataSortOrder, + FileFormat deleteFileFormat, + int[] equalityFieldIds, + Schema equalityDeleteRowSchema, + RowType equalityDeleteFlinkType, + SortOrder equalityDeleteSortOrder, + Schema positionDeleteRowSchema, + RowType positionDeleteFlinkType) { + + super( + table, + dataFileFormat, + dataSchema, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteSortOrder, + positionDeleteRowSchema); + + this.dataFlinkType = dataFlinkType; + this.equalityDeleteFlinkType = equalityDeleteFlinkType; + this.positionDeleteFlinkType = positionDeleteFlinkType; + } + + static Builder builderFor(Table table) { + return new Builder(table); + } + + @Override + protected void configureDataWrite(Avro.DataWriteBuilder builder) { + builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); + } + + @Override + protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { + builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); + } + + @Override + protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { + int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME); + if (rowFieldIndex >= 0) { + // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos + RowType positionDeleteRowFlinkType = + (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex); + builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType)); + } + } + + @Override + protected void configureDataWrite(Parquet.DataWriteBuilder builder) { + builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); + } + + @Override + protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); + } + + @Override + protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType)); + builder.transformPaths(path -> StringData.fromString(path.toString())); + } + + @Override + protected void configureDataWrite(ORC.DataWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); + } + + @Override + protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); + } + + @Override + protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema)); + builder.transformPaths(path -> StringData.fromString(path.toString())); + } + + private RowType dataFlinkType() { + if (dataFlinkType == null) { + Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); + this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); + } + + return dataFlinkType; + } + + private RowType equalityDeleteFlinkType() { + if (equalityDeleteFlinkType == null) { + Preconditions.checkNotNull( + equalityDeleteRowSchema(), "Equality delete schema must not be null"); + this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); + } + + return equalityDeleteFlinkType; + } + + private RowType positionDeleteFlinkType() { + if (positionDeleteFlinkType == null) { + // wrap the optional row schema into the position delete schema that contains path and + // position + Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); + this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema); + } + + return positionDeleteFlinkType; + } + + static class Builder { + private final Table table; + private FileFormat dataFileFormat; + private Schema dataSchema; + private RowType dataFlinkType; + private SortOrder dataSortOrder; + private FileFormat deleteFileFormat; + private int[] equalityFieldIds; + private Schema equalityDeleteRowSchema; + private RowType equalityDeleteFlinkType; + private SortOrder equalityDeleteSortOrder; + private Schema positionDeleteRowSchema; + private RowType positionDeleteFlinkType; + + Builder(Table table) { + this.table = table; + + Map properties = table.properties(); + + String dataFileFormatName = + properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + this.dataFileFormat = FileFormat.fromString(dataFileFormatName); + + String deleteFileFormatName = + properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); + this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); + } + + Builder dataFileFormat(FileFormat newDataFileFormat) { + this.dataFileFormat = newDataFileFormat; + return this; + } + + Builder dataSchema(Schema newDataSchema) { + this.dataSchema = newDataSchema; + return this; + } + + /** + * Sets a Flink type for data. + * + *

If not set, the value is derived from the provided Iceberg schema. + */ + Builder dataFlinkType(RowType newDataFlinkType) { + this.dataFlinkType = newDataFlinkType; + return this; + } + + Builder dataSortOrder(SortOrder newDataSortOrder) { + this.dataSortOrder = newDataSortOrder; + return this; + } + + Builder deleteFileFormat(FileFormat newDeleteFileFormat) { + this.deleteFileFormat = newDeleteFileFormat; + return this; + } + + Builder equalityFieldIds(int[] newEqualityFieldIds) { + this.equalityFieldIds = newEqualityFieldIds; + return this; + } + + Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { + this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; + return this; + } + + /** + * Sets a Flink type for equality deletes. + * + *

If not set, the value is derived from the provided Iceberg schema. + */ + Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) { + this.equalityDeleteFlinkType = newEqualityDeleteFlinkType; + return this; + } + + Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { + this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; + return this; + } + + Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { + this.positionDeleteRowSchema = newPositionDeleteRowSchema; + return this; + } + + /** + * Sets a Flink type for position deletes. + * + *

If not set, the value is derived from the provided Iceberg schema. + */ + Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) { + this.positionDeleteFlinkType = newPositionDeleteFlinkType; + return this; + } + + FlinkFileWriterFactory build() { + boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; + boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; + Preconditions.checkArgument( + noEqualityDeleteConf || fullEqualityDeleteConf, + "Equality field IDs and equality delete row schema must be set together"); + + return new FlinkFileWriterFactory( + table, + dataFileFormat, + dataSchema, + dataFlinkType, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteFlinkType, + equalityDeleteSortOrder, + positionDeleteRowSchema, + positionDeleteFlinkType); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java new file mode 100644 index 000000000000..c7e8a2dea7cb --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -0,0 +1,132 @@ +/* + * 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.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class FlinkManifestUtil { + private static final int FORMAT_V2 = 2; + private static final Long DUMMY_SNAPSHOT_ID = 0L; + + private FlinkManifestUtil() {} + + static ManifestFile writeDataFiles( + OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { + ManifestWriter writer = + ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); + + try (ManifestWriter closeableWriter = writer) { + closeableWriter.addAll(dataFiles); + } + + return writer.toManifestFile(); + } + + static List readDataFiles( + ManifestFile manifestFile, FileIO io, Map specsById) + throws IOException { + try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io, specsById)) { + return Lists.newArrayList(dataFiles); + } + } + + static ManifestOutputFileFactory createOutputFileFactory( + Supplier

tableSupplier, + Map tableProps, + String flinkJobId, + String operatorUniqueId, + int subTaskId, + long attemptNumber) { + return new ManifestOutputFileFactory( + tableSupplier, tableProps, flinkJobId, operatorUniqueId, subTaskId, attemptNumber); + } + + /** + * Write the {@link WriteResult} to temporary manifest files. + * + * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same + * partition spec + */ + static DeltaManifests writeCompletedFiles( + WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) + throws IOException { + + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + // Write the completed data files into a newly created data manifest file. + if (result.dataFiles() != null && result.dataFiles().length > 0) { + dataManifest = + writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); + } + + // Write the completed delete files into a newly created delete manifest file. + if (result.deleteFiles() != null && result.deleteFiles().length > 0) { + OutputFile deleteManifestFile = outputFileSupplier.get(); + + ManifestWriter deleteManifestWriter = + ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); + try (ManifestWriter writer = deleteManifestWriter) { + for (DeleteFile deleteFile : result.deleteFiles()) { + writer.add(deleteFile); + } + } + + deleteManifest = deleteManifestWriter.toManifestFile(); + } + + return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); + } + + static WriteResult readCompletedFiles( + DeltaManifests deltaManifests, FileIO io, Map specsById) + throws IOException { + WriteResult.Builder builder = WriteResult.builder(); + + // Read the completed data files from persisted data manifest file. + if (deltaManifests.dataManifest() != null) { + builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io, specsById)); + } + + // Read the completed delete files from persisted delete manifests file. + if (deltaManifests.deleteManifest() != null) { + try (CloseableIterable deleteFiles = + ManifestFiles.readDeleteManifest(deltaManifests.deleteManifest(), io, specsById)) { + builder.addDeleteFiles(deleteFiles); + } + } + + return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java new file mode 100644 index 000000000000..769af7d77140 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -0,0 +1,654 @@ +/* + * 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 static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +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.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +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.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FlinkSink { + private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class); + + private static final String ICEBERG_STREAM_WRITER_NAME = + IcebergStreamWriter.class.getSimpleName(); + private static final String ICEBERG_FILES_COMMITTER_NAME = + IcebergFilesCommitter.class.getSimpleName(); + + private FlinkSink() {} + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } + + public static class Builder { + private Function> inputCreator = null; + private TableLoader tableLoader; + private Table table; + private TableSchema tableSchema; + private List equalityFieldColumns = null; + private String uidPrefix = null; + private final Map snapshotProperties = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private final Map writeOptions = Maps.newHashMap(); + private FlinkWriteConf flinkWriteConf = null; + + private Builder() {} + + private Builder forRowData(DataStream newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forMapperOutputType( + DataStream input, MapFunction mapper, TypeInformation outputType) { + this.inputCreator = + newUidPrefix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism + // of map operator same as its input to keep map operator chaining its input, and avoid + // rebalanced by default. + SingleOutputStreamOperator inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidPrefix != null) { + inputStream.name(operatorName(newUidPrefix)).uid(newUidPrefix + "-mapper"); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} + * which will write all the records into {@link DataFile}s and emit them to downstream operator. + * Providing a table would avoid so many table loading from each separate task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + /** + * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need + * this loader because {@link Table} is not serializable and could not just use the loaded table + * from Builder#table in the remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + /** + * Set the write properties for Flink sink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for Flink sink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of + * multiple operators (like writer, committer, dummy sink etc.) Actually operator uid will be + * appended with a suffix like "uidPrefix-writer".
+ *
+ * If provided, this prefix is also applied to operator names.
+ *
+ * Flink auto generates operator uid if not set explicitly. It is a recommended + * best-practice to set uid for all operators before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid.
+ *
+ * Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous Flink sink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore Flink sink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newPrefix prefix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidPrefix(String newPrefix) { + this.uidPrefix = newPrefix; + return this; + } + + public Builder setSnapshotProperties(Map properties) { + snapshotProperties.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotProperties.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + private DataStreamSink chainIcebergOperators() { + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); + + DataStream rowDataInput = inputCreator.apply(uidPrefix); + + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + this.table = loader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + // Find out the equality field id list based on the user-provided equality field column names. + List equalityFieldIds = checkAndGetEqualityFieldIds(); + + // Convert the requested flink table schema to flink row type. + RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + + // Distribute the records from input data stream based on the write.distribution-mode and + // equality fields. + DataStream distributeStream = + distributeDataStream( + rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + + // Add parallel writers that append rows to files + SingleOutputStreamOperator writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds); + + // Add single-parallelism committer that commits files + // after successful checkpoint or end of input + SingleOutputStreamOperator committerStream = appendCommitter(writerStream); + + // Add dummy discard sink + return appendDummySink(committerStream); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + return chainIcebergOperators(); + } + + private String operatorName(String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + @VisibleForTesting + List checkAndGetEqualityFieldIds() { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = + Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + @SuppressWarnings("unchecked") + private DataStreamSink appendDummySink( + SingleOutputStreamOperator committerStream) { + DataStreamSink resultStream = + committerStream + .addSink(new DiscardingSink()) + .name(operatorName(String.format("IcebergSink %s", this.table.name()))) + .setParallelism(1); + if (uidPrefix != null) { + resultStream = resultStream.uid(uidPrefix + "-dummysink"); + } + return resultStream; + } + + private SingleOutputStreamOperator appendCommitter( + SingleOutputStreamOperator writerStream) { + IcebergFilesCommitter filesCommitter = + new IcebergFilesCommitter( + tableLoader, + flinkWriteConf.overwriteMode(), + snapshotProperties, + flinkWriteConf.workerPoolSize(), + flinkWriteConf.branch(), + table.spec()); + SingleOutputStreamOperator committerStream = + writerStream + .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) + .setParallelism(1) + .setMaxParallelism(1); + if (uidPrefix != null) { + committerStream = committerStream.uid(uidPrefix + "-committer"); + } + return committerStream; + } + + private SingleOutputStreamOperator appendWriter( + DataStream input, RowType flinkRowType, List equalityFieldIds) { + // Validate the equality fields and partition fields if we enable the upsert mode. + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !flinkWriteConf.overwriteMode(), + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + + SerializableSupplier
tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = + new CachingTableSupplier(serializableTable, tableLoader, tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + IcebergStreamWriter streamWriter = + createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); + + int parallelism = + flinkWriteConf.writeParallelism() == null + ? input.getParallelism() + : flinkWriteConf.writeParallelism(); + SingleOutputStreamOperator writerStream = + input + .transform( + operatorName(ICEBERG_STREAM_WRITER_NAME), + TypeInformation.of(WriteResult.class), + streamWriter) + .setParallelism(parallelism); + if (uidPrefix != null) { + writerStream = writerStream.uid(uidPrefix + "-writer"); + } + return writerStream; + } + + private DataStream distributeDataStream( + DataStream input, + List equalityFieldIds, + PartitionSpec partitionSpec, + Schema iSchema, + RowType flinkRowType) { + DistributionMode writeMode = flinkWriteConf.distributionMode(); + + LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + switch (writeMode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (partitionSpec.isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } + } else { + if (partitionSpec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : partitionSpec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); + } + } + } + + static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to + // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 + // 'byte'), we will + // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here + // we must use flink + // schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + static IcebergStreamWriter createStreamWriter( + SerializableSupplier
tableSupplier, + FlinkWriteConf flinkWriteConf, + RowType flinkRowType, + List equalityFieldIds) { + Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); + + Table initTable = tableSupplier.get(); + FileFormat format = flinkWriteConf.dataFileFormat(); + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + flinkWriteConf.targetDataFileSize(), + format, + writeProperties(initTable, format, flinkWriteConf), + equalityFieldIds, + flinkWriteConf.upsertMode()); + + return new IcebergStreamWriter<>(initTable.name(), taskWriterFactory); + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java new file mode 100644 index 000000000000..b9bceaa9311d --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -0,0 +1,516 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +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; +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.types.Comparators; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergFilesCommitter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + private static final long INITIAL_CHECKPOINT_ID = -1L; + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class); + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + + // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always + // increasing, so we could correctly commit all the data files whose checkpoint id is greater than + // the max committed one to iceberg table, for avoiding committing the same data files twice. This + // id will be attached to iceberg's meta when committing the iceberg transaction. + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + // TableLoader to load iceberg table lazily. + private final TableLoader tableLoader; + private final boolean replacePartitions; + private final Map snapshotProperties; + + // A sorted map to maintain the completed data files for each pending checkpointId (which have not + // been committed to iceberg table). We need a sorted map here because there's possible that few + // checkpoints snapshot failed, for example: the 1st checkpoint have 2 data files <1, >, the 2st checkpoint have 1 data files <2, >. Snapshot for checkpoint#1 + // interrupted because of network/disk failure etc, while we don't expect any data loss in iceberg + // table. So we keep the finished files <1, > in memory and retry to commit iceberg + // table when the next checkpoint happen. + private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); + + // The completed files cache for current checkpoint. Once the snapshot barrier received, it will + // be flushed to the 'dataFilesPerCheckpoint'. + private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final String branch; + + // It will have an unique identifier for one job. + private transient String flinkJobId; + private transient String operatorUniqueId; + private transient Table table; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient ManifestOutputFileFactory manifestOutputFileFactory; + private transient long maxCommittedCheckpointId; + private transient int continuousEmptyCheckpoints; + private transient int maxContinuousEmptyCommits; + // There're two cases that we restore from flink checkpoints: the first case is restoring from + // snapshot created by the same flink job; another case is restoring from snapshot created by + // another different job. For the second case, we need to maintain the old flink job's id in flink + // state backend to find the max-committed-checkpoint-id when traversing iceberg table's + // snapshots. + private static final ListStateDescriptor JOB_ID_DESCRIPTOR = + new ListStateDescriptor<>("iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); + private transient ListState jobIdState; + // All pending checkpoints states for this function. + private static final ListStateDescriptor> STATE_DESCRIPTOR = + buildStateDescriptor(); + private transient ListState> checkpointsState; + + private final Integer workerPoolSize; + private final PartitionSpec spec; + private transient ExecutorService workerPool; + + IcebergFilesCommitter( + TableLoader tableLoader, + boolean replacePartitions, + Map snapshotProperties, + Integer workerPoolSize, + String branch, + PartitionSpec spec) { + this.tableLoader = tableLoader; + this.replacePartitions = replacePartitions; + this.snapshotProperties = snapshotProperties; + this.workerPoolSize = workerPoolSize; + this.branch = branch; + this.spec = spec; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + this.operatorUniqueId = getRuntimeContext().getOperatorUniqueID(); + + // Open the table loader and load the table. + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + this.committerMetrics = new IcebergFilesCommitterMetrics(super.metrics, table.name()); + + maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + int attemptId = getRuntimeContext().getAttemptNumber(); + this.manifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); + this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); + this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); + if (context.isRestored()) { + Iterable jobIdIterable = jobIdState.get(); + if (jobIdIterable == null || !jobIdIterable.iterator().hasNext()) { + LOG.warn( + "Failed to restore committer state. This can happen when operator uid changed and Flink " + + "allowNonRestoredState is enabled. Best practice is to explicitly set the operator id " + + "via FlinkSink#Builder#uidPrefix() so that the committer operator uid is stable. " + + "Otherwise, Flink auto generate an operator uid based on job topology." + + "With that, operator uid is subjective to change upon topology change."); + return; + } + + String restoredFlinkJobId = jobIdIterable.iterator().next(); + Preconditions.checkState( + !Strings.isNullOrEmpty(restoredFlinkJobId), + "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); + + // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new + // flink job even if it's restored from a snapshot created by another different flink job, so + // it's safe to assign the max committed checkpoint id from restored flink job to the current + // flink job. + this.maxCommittedCheckpointId = + getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + + NavigableMap uncommittedDataFiles = + Maps.newTreeMap(checkpointsState.get().iterator().next()) + .tailMap(maxCommittedCheckpointId, false); + if (!uncommittedDataFiles.isEmpty()) { + // Committed all uncommitted data files from the old flink job to iceberg table. + long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey(); + commitUpToCheckpoint( + uncommittedDataFiles, restoredFlinkJobId, operatorUniqueId, maxUncommittedCheckpointId); + } + } + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + long checkpointId = context.getCheckpointId(); + LOG.info( + "Start to flush snapshot state to state backend, table: {}, checkpointId: {}", + table, + checkpointId); + + // Update the checkpoint state. + long startNano = System.nanoTime(); + dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + // Reset the snapshot state to the latest state. + checkpointsState.clear(); + checkpointsState.add(dataFilesPerCheckpoint); + + jobIdState.clear(); + jobIdState.add(flinkJobId); + + // Clear the local buffer for current checkpoint. + writeResultsOfCurrentCkpt.clear(); + committerMetrics.checkpointDuration( + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + super.notifyCheckpointComplete(checkpointId); + // It's possible that we have the following events: + // 1. snapshotState(ckpId); + // 2. snapshotState(ckpId+1); + // 3. notifyCheckpointComplete(ckpId+1); + // 4. notifyCheckpointComplete(ckpId); + // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all + // the files, + // Besides, we need to maintain the max-committed-checkpoint-id to be increasing. + if (checkpointId > maxCommittedCheckpointId) { + LOG.info("Checkpoint {} completed. Attempting commit.", checkpointId); + commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, checkpointId); + this.maxCommittedCheckpointId = checkpointId; + } else { + LOG.info( + "Skipping committing checkpoint {}. {} is already committed.", + checkpointId, + maxCommittedCheckpointId); + } + + // reload the table in case new configuration is needed + this.table = tableLoader.loadTable(); + } + + private void commitUpToCheckpoint( + NavigableMap deltaManifestsMap, + String newFlinkJobId, + String operatorId, + long checkpointId) + throws IOException { + NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); + List manifests = Lists.newArrayList(); + NavigableMap pendingResults = Maps.newTreeMap(); + for (Map.Entry e : pendingMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { + // Skip the empty flink manifest. + continue; + } + + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + committerMetrics.updateCommitSummary(summary); + pendingMap.clear(); + deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + } + + private void commitPendingResult( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } + continuousEmptyCheckpoints = 0; + } else { + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void deleteCommittedManifests( + List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(this) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } + + private void replacePartitions( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + + commitOperation( + dynamicOverwrite, + summary, + "dynamic partition overwrite", + newFlinkJobId, + operatorId, + checkpointId); + } + + private void commitDeltaTxn( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + commitOperation(appendFiles, summary, "append", newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + commitOperation(rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate operation, + CommitSummary summary, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + LOG.info( + "Committing {} for checkpoint {} to table {} branch {} with summary: {}", + description, + checkpointId, + table.name(), + branch, + summary); + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(FLINK_JOB_ID, newFlinkJobId); + operation.set(OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + committerMetrics.commitDuration(durationMs); + } + + @Override + public void processElement(StreamRecord element) { + this.writeResultsOfCurrentCkpt.add(element.getValue()); + } + + @Override + public void endInput() throws IOException { + // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. + long currentCheckpointId = Long.MAX_VALUE; + dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); + writeResultsOfCurrentCkpt.clear(); + + commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); + } + + /** + * Write all the complete data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + private byte[] writeToManifest(long checkpointId) throws IOException { + if (writeResultsOfCurrentCkpt.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> manifestOutputFileFactory.create(checkpointId), spec); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void open() throws Exception { + super.open(); + + final String operatorID = getRuntimeContext().getOperatorUniqueID(); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); + } + + @Override + public void close() throws Exception { + if (tableLoader != null) { + tableLoader.close(); + } + + if (workerPool != null) { + workerPool.shutdown(); + } + } + + @VisibleForTesting + static ListStateDescriptor> buildStateDescriptor() { + Comparator longComparator = Comparators.forType(Types.LongType.get()); + // Construct a SortedMapTypeInfo. + SortedMapTypeInfo sortedMapTypeInfo = + new SortedMapTypeInfo<>( + BasicTypeInfo.LONG_TYPE_INFO, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + longComparator); + return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java new file mode 100644 index 000000000000..5b28c4acb1c5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java @@ -0,0 +1,73 @@ +/* + * 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.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.flink.util.ElapsedTimeGauge; + +class IcebergFilesCommitterMetrics { + private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); + private final AtomicLong lastCommitDurationMs = new AtomicLong(); + private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; + private final Counter committedDataFilesCount; + private final Counter committedDataFilesRecordCount; + private final Counter committedDataFilesByteCount; + private final Counter committedDeleteFilesCount; + private final Counter committedDeleteFilesRecordCount; + private final Counter committedDeleteFilesByteCount; + + IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup committerMetrics = + metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); + committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); + committerMetrics.gauge("lastCommitDurationMs", lastCommitDurationMs::get); + this.elapsedSecondsSinceLastSuccessfulCommit = new ElapsedTimeGauge(TimeUnit.SECONDS); + committerMetrics.gauge( + "elapsedSecondsSinceLastSuccessfulCommit", elapsedSecondsSinceLastSuccessfulCommit); + this.committedDataFilesCount = committerMetrics.counter("committedDataFilesCount"); + this.committedDataFilesRecordCount = committerMetrics.counter("committedDataFilesRecordCount"); + this.committedDataFilesByteCount = committerMetrics.counter("committedDataFilesByteCount"); + this.committedDeleteFilesCount = committerMetrics.counter("committedDeleteFilesCount"); + this.committedDeleteFilesRecordCount = + committerMetrics.counter("committedDeleteFilesRecordCount"); + this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); + } + + void checkpointDuration(long checkpointDurationMs) { + lastCheckpointDurationMs.set(checkpointDurationMs); + } + + void commitDuration(long commitDurationMs) { + lastCommitDurationMs.set(commitDurationMs); + } + + /** This is called upon a successful commit. */ + void updateCommitSummary(CommitSummary stats) { + elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); + committedDataFilesCount.inc(stats.dataFilesCount()); + committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); + committedDataFilesByteCount.inc(stats.dataFilesByteCount()); + committedDeleteFilesCount.inc(stats.deleteFilesCount()); + committedDeleteFilesRecordCount.inc(stats.deleteFilesRecordCount()); + committedDeleteFilesByteCount.inc(stats.deleteFilesByteCount()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java new file mode 100644 index 000000000000..9ea0349fb057 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -0,0 +1,120 @@ +/* + * 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.io.IOException; +import java.util.concurrent.TimeUnit; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + + private transient TaskWriter writer; + private transient int subTaskId; + private transient int attemptId; + private transient IcebergStreamWriterMetrics writerMetrics; + + IcebergStreamWriter(String fullTableName, TaskWriterFactory taskWriterFactory) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + setChainingStrategy(ChainingStrategy.ALWAYS); + } + + @Override + public void open() { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); + + // Initialize the task writer factory. + this.taskWriterFactory.initialize(subTaskId, attemptId); + + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + flush(); + this.writer = taskWriterFactory.create(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + writer.write(element.getValue()); + } + + @Override + public void close() throws Exception { + super.close(); + if (writer != null) { + writer.close(); + writer = null; + } + } + + @Override + public void endInput() throws IOException { + // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the + // remaining completed files to downstream before closing the writer so that we won't miss any + // of them. + // Note that if the task is not closed after calling endInput, checkpoint may be triggered again + // causing files to be sent repeatedly, the writer is marked as null after the last file is sent + // to guard against duplicated writes. + flush(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("table_name", fullTableName) + .add("subtask_id", subTaskId) + .add("attempt_id", attemptId) + .toString(); + } + + /** close all open files and emit files to downstream committer operator */ + private void flush() throws IOException { + if (writer == null) { + return; + } + + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + writerMetrics.updateFlushResult(result); + output.collect(new StreamRecord<>(result)); + writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + + // Set writer to null to prevent duplicate flushes in the corner case of + // prepareSnapshotPreBarrier happening after endInput. + writer = null; + } +} 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 new file mode 100644 index 000000000000..ce2a6c583fdf --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -0,0 +1,89 @@ +/* + * 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 com.codahale.metrics.SlidingWindowReservoir; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.io.WriteResult; + +class IcebergStreamWriterMetrics { + // 1,024 reservoir size should cost about 8KB, which is quite small. + // It should also produce good accuracy for histogram distribution (like percentiles). + private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; + + private final Counter flushedDataFiles; + private final Counter flushedDeleteFiles; + private final Counter flushedReferencedDataFiles; + private final AtomicLong lastFlushDurationMs; + private final Histogram dataFilesSizeHistogram; + private final Histogram deleteFilesSizeHistogram; + + IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup writerMetrics = + metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); + this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); + this.flushedDeleteFiles = writerMetrics.counter("flushedDeleteFiles"); + this.flushedReferencedDataFiles = writerMetrics.counter("flushedReferencedDataFiles"); + this.lastFlushDurationMs = new AtomicLong(); + writerMetrics.gauge("lastFlushDurationMs", lastFlushDurationMs::get); + + com.codahale.metrics.Histogram dropwizardDataFilesSizeHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); + this.dataFilesSizeHistogram = + writerMetrics.histogram( + "dataFilesSizeHistogram", + new DropwizardHistogramWrapper(dropwizardDataFilesSizeHistogram)); + com.codahale.metrics.Histogram dropwizardDeleteFilesSizeHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); + this.deleteFilesSizeHistogram = + writerMetrics.histogram( + "deleteFilesSizeHistogram", + new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); + } + + void updateFlushResult(WriteResult result) { + flushedDataFiles.inc(result.dataFiles().length); + flushedDeleteFiles.inc(result.deleteFiles().length); + flushedReferencedDataFiles.inc(result.referencedDataFiles().length); + + // For file size distribution histogram, we don't have to update them after successful commits. + // This should works equally well and we avoided the overhead of tracking the list of file sizes + // in the {@link CommitSummary}, which currently stores simple stats for counters and gauges + // metrics. + Arrays.stream(result.dataFiles()) + .forEach( + dataFile -> { + dataFilesSizeHistogram.update(dataFile.fileSizeInBytes()); + }); + Arrays.stream(result.deleteFiles()) + .forEach( + deleteFile -> { + deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); + }); + } + + void flushDuration(long flushDurationMs) { + lastFlushDurationMs.set(flushDurationMs); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java new file mode 100644 index 000000000000..da5e6e7627ae --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -0,0 +1,94 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Strings; + +class ManifestOutputFileFactory { + // Users could define their own flink manifests directory by setting this value in table + // properties. + static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; + + private final Supplier
tableSupplier; + private final Map props; + private final String flinkJobId; + private final String operatorUniqueId; + private final int subTaskId; + private final long attemptNumber; + private final AtomicInteger fileCount = new AtomicInteger(0); + + ManifestOutputFileFactory( + Supplier
tableSupplier, + Map props, + String flinkJobId, + String operatorUniqueId, + int subTaskId, + long attemptNumber) { + this.tableSupplier = tableSupplier; + this.props = props; + this.flinkJobId = flinkJobId; + this.operatorUniqueId = operatorUniqueId; + this.subTaskId = subTaskId; + this.attemptNumber = attemptNumber; + } + + private String generatePath(long checkpointId) { + return FileFormat.AVRO.addExtension( + String.format( + "%s-%s-%05d-%d-%d-%05d", + flinkJobId, + operatorUniqueId, + subTaskId, + attemptNumber, + checkpointId, + fileCount.incrementAndGet())); + } + + OutputFile create(long checkpointId) { + String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); + TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); + + String newManifestFullPath; + if (Strings.isNullOrEmpty(flinkManifestDir)) { + // User don't specify any flink manifest directory, so just use the default metadata path. + newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); + } else { + newManifestFullPath = + String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); + } + + return tableSupplier.get().io().newOutputFile(newManifestFullPath); + } + + private static String stripTrailingSlash(String path) { + String result = path; + while (result.endsWith("/")) { + result = result.substring(0, result.length() - 1); + } + return result; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java new file mode 100644 index 000000000000..df951684b446 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.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.sink; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; + +/** + * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be + * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy + * for {@link FlinkSink}. + */ +class PartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + + private transient RowDataWrapper rowDataWrapper; + + PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { + this.schema = schema; + this.partitionKey = new PartitionKey(spec, schema); + this.flinkSchema = flinkSchema; + } + + /** + * Construct the {@link RowDataWrapper} lazily here because few members in it are not + * serializable. In this way, we don't have to serialize them with forcing. + */ + private RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + return rowDataWrapper; + } + + @Override + public String getKey(RowData row) { + partitionKey.partition(lazyRowDataWrapper().wrap(row)); + return partitionKey.toPath(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java new file mode 100644 index 000000000000..38062dd1a2c4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java @@ -0,0 +1,97 @@ +/* + * 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.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Tasks; + +class PartitionedDeltaWriter extends BaseDeltaTaskWriter { + + private final PartitionKey partitionKey; + + private final Map writers = Maps.newHashMap(); + + PartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + flinkSchema, + equalityFieldIds, + upsert); + this.partitionKey = new PartitionKey(spec, schema); + } + + @Override + RowDataDeltaWriter route(RowData row) { + partitionKey.partition(wrapper().wrap(row)); + + RowDataDeltaWriter writer = writers.get(partitionKey); + if (writer == null) { + // NOTICE: we need to copy a new partition key here, in case of messing up the keys in + // writers. + PartitionKey copiedKey = partitionKey.copy(); + writer = new RowDataDeltaWriter(copiedKey); + writers.put(copiedKey, writer); + } + + return writer; + } + + @Override + public void close() { + try { + Tasks.foreach(writers.values()) + .throwFailureWhenFinished() + .noRetry() + .run(RowDataDeltaWriter::close, IOException.class); + + writers.clear(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close equality delta writer", e); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java new file mode 100644 index 000000000000..67422a1afeb1 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -0,0 +1,244 @@ +/* + * 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 java.util.function.Supplier; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitionedFanoutWriter; +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.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.SerializableSupplier; + +public class RowDataTaskWriterFactory implements TaskWriterFactory { + private final Supplier
tableSupplier; + private final Schema schema; + private final RowType flinkSchema; + private final PartitionSpec spec; + private final long targetFileSizeBytes; + private final FileFormat format; + private final List equalityFieldIds; + private final boolean upsert; + private final FileAppenderFactory appenderFactory; + + private transient OutputFileFactory outputFileFactory; + + public RowDataTaskWriterFactory( + Table table, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert) { + this( + () -> table, + flinkSchema, + targetFileSizeBytes, + format, + writeProperties, + equalityFieldIds, + upsert); + } + + public RowDataTaskWriterFactory( + SerializableSupplier
tableSupplier, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert) { + this.tableSupplier = tableSupplier; + + Table table; + if (tableSupplier instanceof CachingTableSupplier) { + // rely on the initial table metadata for schema, etc., until schema evolution is supported + table = ((CachingTableSupplier) tableSupplier).initialTable(); + } else { + table = tableSupplier.get(); + } + + this.schema = table.schema(); + this.flinkSchema = flinkSchema; + this.spec = table.spec(); + this.targetFileSizeBytes = targetFileSizeBytes; + this.format = format; + this.equalityFieldIds = equalityFieldIds; + this.upsert = upsert; + + if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { + this.appenderFactory = + new FlinkAppenderFactory( + table, schema, flinkSchema, writeProperties, spec, null, null, null); + } else if (upsert) { + // In upsert mode, only the new row is emitted using INSERT row kind. Therefore, any column of + // the inserted row + // may differ from the deleted row other than the primary key fields, and the delete file must + // contain values + // that are correct for the deleted row. Therefore, only write the equality delete fields. + this.appenderFactory = + new FlinkAppenderFactory( + table, + schema, + flinkSchema, + writeProperties, + spec, + ArrayUtil.toIntArray(equalityFieldIds), + TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), + null); + } else { + this.appenderFactory = + new FlinkAppenderFactory( + table, + schema, + flinkSchema, + writeProperties, + spec, + ArrayUtil.toIntArray(equalityFieldIds), + schema, + null); + } + } + + @Override + public void initialize(int taskId, int attemptId) { + Table table; + if (tableSupplier instanceof CachingTableSupplier) { + // rely on the initial table metadata for schema, etc., until schema evolution is supported + table = ((CachingTableSupplier) tableSupplier).initialTable(); + } else { + table = tableSupplier.get(); + } + + refreshTable(); + + this.outputFileFactory = + OutputFileFactory.builderFor(table, taskId, attemptId) + .format(format) + .ioSupplier(() -> tableSupplier.get().io()) + .build(); + } + + @Override + public TaskWriter create() { + Preconditions.checkNotNull( + outputFileFactory, + "The outputFileFactory shouldn't be null if we have invoked the initialize()."); + + refreshTable(); + + if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { + // Initialize a task writer to write INSERT only. + if (spec.isUnpartitioned()) { + return new UnpartitionedWriter<>( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes); + } else { + return new RowDataPartitionedFanoutWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema); + } + } else { + // Initialize a task writer to write both INSERT and equality DELETE. + if (spec.isUnpartitioned()) { + return new UnpartitionedDeltaWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema, + equalityFieldIds, + upsert); + } else { + return new PartitionedDeltaWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema, + equalityFieldIds, + upsert); + } + } + } + + void refreshTable() { + if (tableSupplier instanceof CachingTableSupplier) { + ((CachingTableSupplier) tableSupplier).refreshTable(); + } + } + + private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { + + private final PartitionKey partitionKey; + private final RowDataWrapper rowDataWrapper; + + RowDataPartitionedFanoutWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.partitionKey = new PartitionKey(spec, schema); + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + @Override + protected PartitionKey partition(RowData row) { + partitionKey.partition(rowDataWrapper.wrap(row)); + return partitionKey; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java new file mode 100644 index 000000000000..e3a1245e8cbd --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -0,0 +1,45 @@ +/* + * 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.io.Serializable; +import org.apache.iceberg.io.TaskWriter; + +/** + * Factory to create {@link TaskWriter} + * + * @param data type of record. + */ +public interface TaskWriterFactory extends Serializable { + + /** + * Initialize the factory with a given taskId and attemptId. + * + * @param taskId the identifier of task. + * @param attemptId the attempt id of this task. + */ + void initialize(int taskId, int attemptId); + + /** + * Initialize a {@link TaskWriter} with given task id and attempt id. + * + * @return a newly created task writer. + */ + TaskWriter create(); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java new file mode 100644 index 000000000000..7680fb933b20 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.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.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { + private final RowDataDeltaWriter writer; + + UnpartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + flinkSchema, + equalityFieldIds, + upsert); + this.writer = new RowDataDeltaWriter(null); + } + + @Override + RowDataDeltaWriter route(RowData row) { + return writer; + } + + @Override + public void close() throws IOException { + writer.close(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java new file mode 100644 index 000000000000..157f04b8b0ed --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java @@ -0,0 +1,71 @@ +/* + * 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.io.Serializable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link + * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores + * the merged {@link DataStatistics} result from all reported subtasks. + */ +class AggregatedStatistics, S> implements Serializable { + + private final long checkpointId; + private final DataStatistics dataStatistics; + + AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { + this.checkpointId = checkpoint; + this.dataStatistics = statisticsSerializer.createInstance(); + } + + AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { + this.checkpointId = checkpoint; + this.dataStatistics = dataStatistics; + } + + long checkpointId() { + return checkpointId; + } + + DataStatistics dataStatistics() { + return dataStatistics; + } + + void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { + Preconditions.checkArgument( + checkpointId == eventCheckpointId, + "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", + operatorName, + eventCheckpointId, + checkpointId); + dataStatistics.merge(eventDataStatistics); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("dataStatistics", dataStatistics) + .toString(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java new file mode 100644 index 000000000000..e8ff61dbeb27 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -0,0 +1,133 @@ +/* + * 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.Set; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress + * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific + * checkpoint. + */ +class AggregatedStatisticsTracker, S> { + private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); + private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; + private final String operatorName; + private final TypeSerializer> statisticsSerializer; + private final int parallelism; + private final Set inProgressSubtaskSet; + private volatile AggregatedStatistics inProgressStatistics; + + AggregatedStatisticsTracker( + String operatorName, + TypeSerializer> statisticsSerializer, + int parallelism) { + this.operatorName = operatorName; + this.statisticsSerializer = statisticsSerializer; + this.parallelism = parallelism; + this.inProgressSubtaskSet = Sets.newHashSet(); + } + + AggregatedStatistics updateAndCheckCompletion( + int subtask, DataStatisticsEvent event) { + long checkpointId = event.checkpointId(); + + if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { + LOG.info( + "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + operatorName, + inProgressStatistics.checkpointId(), + checkpointId); + return null; + } + + AggregatedStatistics completedStatistics = null; + if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { + if ((double) inProgressSubtaskSet.size() / parallelism * 100 + >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { + completedStatistics = inProgressStatistics; + LOG.info( + "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " + + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", + inProgressSubtaskSet.size(), + parallelism, + operatorName, + checkpointId, + inProgressStatistics.checkpointId(), + ACCEPT_PARTIAL_AGGR_THRESHOLD); + } else { + LOG.info( + "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " + + "Aborting the incomplete aggregation for checkpoint {}", + inProgressSubtaskSet.size(), + parallelism, + operatorName, + checkpointId, + inProgressStatistics.checkpointId()); + } + + inProgressStatistics = null; + inProgressSubtaskSet.clear(); + } + + if (inProgressStatistics == null) { + LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); + inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); + inProgressSubtaskSet.clear(); + } + + if (!inProgressSubtaskSet.add(subtask)) { + LOG.debug( + "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } else { + inProgressStatistics.mergeDataStatistic( + operatorName, + event.checkpointId(), + DataStatisticsUtil.deserializeDataStatistics( + event.statisticsBytes(), statisticsSerializer)); + } + + if (inProgressSubtaskSet.size() == parallelism) { + completedStatistics = inProgressStatistics; + LOG.info( + "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", + parallelism, + operatorName, + inProgressStatistics.checkpointId(), + completedStatistics.dataStatistics()); + inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); + inProgressSubtaskSet.clear(); + } + + return completedStatistics; + } + + @VisibleForTesting + AggregatedStatistics inProgressStatistics() { + return inProgressStatistics; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java new file mode 100644 index 000000000000..9d7cf179ab1c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.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.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SortKey; + +/** + * DataStatistics defines the interface to collect data distribution information. + * + *

Data statistics tracks traffic volume distribution across data keys. For low-cardinality key, + * a simple map of (key, count) can be used. For high-cardinality key, probabilistic data structures + * (sketching) can be used. + */ +@Internal +interface DataStatistics, S> { + + /** + * Check if data statistics contains any statistics information. + * + * @return true if data statistics doesn't contain any statistics information + */ + boolean isEmpty(); + + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); + + /** + * Merge current statistics with other statistics. + * + * @param otherStatistics the statistics to be merged + */ + void merge(D otherStatistics); + + /** + * Get the underline statistics. + * + * @return the underline statistics + */ + S statistics(); +} 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 new file mode 100644 index 000000000000..c8ac79c61bf6 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -0,0 +1,396 @@ +/* + * 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.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ThrowableCatchingRunnable; +import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link + * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all + * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data + * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will + * distribute traffic based on the aggregated data statistics to improve data clustering. + */ +@Internal +class DataStatisticsCoordinator, S> implements OperatorCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); + + private final String operatorName; + private final ExecutorService coordinatorExecutor; + private final OperatorCoordinator.Context operatorCoordinatorContext; + private final SubtaskGateways subtaskGateways; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private final TypeSerializer> statisticsSerializer; + private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private volatile AggregatedStatistics completedStatistics; + private volatile boolean started; + + DataStatisticsCoordinator( + String operatorName, + OperatorCoordinator.Context context, + TypeSerializer> statisticsSerializer) { + this.operatorName = operatorName; + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.operatorCoordinatorContext = context; + this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); + this.statisticsSerializer = statisticsSerializer; + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + } + + @Override + public void start() throws Exception { + LOG.info("Starting data statistics coordinator: {}.", operatorName); + started = true; + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + LOG.info("Closed data statistics coordinator: {}.", operatorName); + } + + @VisibleForTesting + void callInCoordinatorThread(Callable callable, String errorMessage) { + ensureStarted(); + // Ensure the task is done by the coordinator executor. + if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { + try { + Callable guardedCallable = + () -> { + try { + return callable.call(); + } catch (Throwable t) { + LOG.error( + "Uncaught Exception in data statistics coordinator: {} executor", + operatorName, + t); + ExceptionUtils.rethrowException(t); + return null; + } + }; + + coordinatorExecutor.submit(guardedCallable).get(); + } catch (InterruptedException | ExecutionException e) { + throw new FlinkRuntimeException(errorMessage, e); + } + } else { + try { + callable.call(); + } catch (Throwable t) { + LOG.error( + "Uncaught Exception in data statistics coordinator: {} executor", operatorName, t); + throw new FlinkRuntimeException(errorMessage, t); + } + } + } + + public void runInCoordinatorThread(Runnable runnable) { + this.coordinatorExecutor.execute( + new ThrowableCatchingRunnable( + throwable -> + this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable), + runnable)); + } + + private void runInCoordinatorThread(ThrowingRunnable action, String actionString) { + ensureStarted(); + runInCoordinatorThread( + () -> { + try { + action.run(); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + LOG.error( + "Uncaught exception in the data statistics coordinator: {} while {}. Triggering job failover", + operatorName, + actionString, + t); + operatorCoordinatorContext.failJob(t); + } + }); + } + + private void ensureStarted() { + Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); + } + + private int parallelism() { + return operatorCoordinatorContext.currentParallelism(); + } + + private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { + AggregatedStatistics aggregatedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); + + if (aggregatedStatistics != null) { + completedStatistics = aggregatedStatistics; + sendDataStatisticsToSubtasks( + completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + } + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void sendDataStatisticsToSubtasks( + long checkpointId, DataStatistics globalDataStatistics) { + callInCoordinatorThread( + () -> { + DataStatisticsEvent dataStatisticsEvent = + DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); + int parallelism = parallelism(); + for (int i = 0; i < parallelism; ++i) { + subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + } + + return null; + }, + String.format( + "Failed to send operator %s coordinator global data statistics for checkpoint %d", + operatorName, checkpointId)); + } + + @Override + @SuppressWarnings("unchecked") + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + Preconditions.checkArgument(event instanceof DataStatisticsEvent); + handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); + }, + String.format( + "handling operator event %s from subtask %d (#%d)", + event.getClass(), subtask, attemptNumber)); + } + + @Override + public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Snapshotting data statistics coordinator {} for checkpoint {}", + operatorName, + checkpointId); + resultFuture.complete( + DataStatisticsUtil.serializeAggregatedStatistics( + completedStatistics, statisticsSerializer)); + }, + String.format("taking checkpoint %d", checkpointId)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} + + @Override + public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) + throws Exception { + Preconditions.checkState( + !started, "The coordinator %s can only be reset if it was not yet started", operatorName); + + if (checkpointData == null) { + LOG.info( + "Data statistic coordinator {} has nothing to restore from checkpoint {}", + operatorName, + checkpointId); + return; + } + + LOG.info( + "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); + completedStatistics = + DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + } + + @Override + public void subtaskReset(int subtask, long checkpointId) { + runInCoordinatorThread( + () -> { + LOG.info( + "Operator {} subtask {} is reset to checkpoint {}", + operatorName, + subtask, + checkpointId); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.reset(subtask); + }, + String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + } + + @Override + public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Throwable reason) { + runInCoordinatorThread( + () -> { + LOG.info( + "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + subtask, + attemptNumber, + operatorName); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); + }, + String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + } + + @Override + public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { + Preconditions.checkArgument(subtask == gateway.getSubtask()); + Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); + runInCoordinatorThread( + () -> { + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.registerSubtaskGateway(gateway); + }, + String.format( + "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + } + + @VisibleForTesting + AggregatedStatistics completedStatistics() { + return completedStatistics; + } + + private static class SubtaskGateways { + private final String operatorName; + private final Map[] gateways; + + private SubtaskGateways(String operatorName, int parallelism) { + this.operatorName = operatorName; + gateways = new Map[parallelism]; + + for (int i = 0; i < parallelism; ++i) { + gateways[i] = Maps.newHashMap(); + } + } + + private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway gateway) { + int subtaskIndex = gateway.getSubtask(); + int attemptNumber = gateway.getExecution().getAttemptNumber(); + Preconditions.checkState( + !gateways[subtaskIndex].containsKey(attemptNumber), + "Coordinator of %s already has a subtask gateway for %d (#%d)", + operatorName, + subtaskIndex, + attemptNumber); + LOG.debug( + "Coordinator of {} registers gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].put(attemptNumber, gateway); + } + + private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { + LOG.debug( + "Coordinator of {} unregisters gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].remove(attemptNumber); + } + + private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { + Preconditions.checkState( + !gateways[subtaskIndex].isEmpty(), + "Coordinator of %s subtask %d is not ready yet to receive events", + operatorName, + subtaskIndex); + return Iterables.getOnlyElement(gateways[subtaskIndex].values()); + } + + private void reset(int subtaskIndex) { + gateways[subtaskIndex].clear(); + } + } + + private static class CoordinatorExecutorThreadFactory + implements ThreadFactory, Thread.UncaughtExceptionHandler { + + private final String coordinatorThreadName; + private final ClassLoader classLoader; + private final Thread.UncaughtExceptionHandler errorHandler; + + @javax.annotation.Nullable private Thread thread; + + CoordinatorExecutorThreadFactory( + final String coordinatorThreadName, final ClassLoader contextClassLoader) { + this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE); + } + + @org.apache.flink.annotation.VisibleForTesting + CoordinatorExecutorThreadFactory( + final String coordinatorThreadName, + final ClassLoader contextClassLoader, + final Thread.UncaughtExceptionHandler errorHandler) { + this.coordinatorThreadName = coordinatorThreadName; + this.classLoader = contextClassLoader; + this.errorHandler = errorHandler; + } + + @Override + public synchronized Thread newThread(@NotNull Runnable runnable) { + thread = new Thread(runnable, coordinatorThreadName); + thread.setContextClassLoader(classLoader); + thread.setUncaughtExceptionHandler(this); + return thread; + } + + @Override + public synchronized void uncaughtException(Thread t, Throwable e) { + errorHandler.uncaughtException(t, e); + } + + boolean isCurrentThreadCoordinatorThread() { + return Thread.currentThread() == thread; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..47dbfc3cfbe1 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -0,0 +1,51 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; + +/** + * DataStatisticsCoordinatorProvider provides the method to create new {@link + * DataStatisticsCoordinator} + */ +@Internal +public class DataStatisticsCoordinatorProvider, S> + extends RecreateOnResetOperatorCoordinator.Provider { + + private final String operatorName; + private final TypeSerializer> statisticsSerializer; + + public DataStatisticsCoordinatorProvider( + String operatorName, + OperatorID operatorID, + TypeSerializer> statisticsSerializer) { + super(operatorID); + this.operatorName = operatorName; + this.statisticsSerializer = statisticsSerializer; + } + + @Override + public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { + return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java new file mode 100644 index 000000000000..852d2157b8cb --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.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.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +/** + * DataStatisticsEvent is sent between data statistics coordinator and operator to transmit data + * statistics in bytes + */ +@Internal +class DataStatisticsEvent, S> implements OperatorEvent { + + private static final long serialVersionUID = 1L; + private final long checkpointId; + private final byte[] statisticsBytes; + + private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + this.checkpointId = checkpointId; + this.statisticsBytes = statisticsBytes; + } + + static , S> DataStatisticsEvent create( + long checkpointId, + DataStatistics dataStatistics, + TypeSerializer> statisticsSerializer) { + return new DataStatisticsEvent<>( + checkpointId, + DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + } + + long checkpointId() { + return checkpointId; + } + + byte[] statisticsBytes() { + return statisticsBytes; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java new file mode 100644 index 000000000000..5157a37cf2cd --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -0,0 +1,197 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +@Internal +class DataStatisticsOperator, S> + extends AbstractStreamOperator> + implements OneInputStreamOperator>, OperatorEventHandler { + + private static final long serialVersionUID = 1L; + + private final String operatorName; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final OperatorEventGateway operatorEventGateway; + private final TypeSerializer> statisticsSerializer; + private transient volatile DataStatistics localStatistics; + private transient volatile DataStatistics globalStatistics; + private transient ListState> globalStatisticsState; + + DataStatisticsOperator( + String operatorName, + Schema schema, + SortOrder sortOrder, + OperatorEventGateway operatorEventGateway, + TypeSerializer> statisticsSerializer) { + this.operatorName = operatorName; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.operatorEventGateway = operatorEventGateway; + this.statisticsSerializer = statisticsSerializer; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + localStatistics = statisticsSerializer.createInstance(); + globalStatisticsState = + context + .getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); + + if (context.isRestored()) { + int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + if (globalStatisticsState.get() == null + || !globalStatisticsState.get().iterator().hasNext()) { + LOG.warn( + "Operator {} subtask {} doesn't have global statistics state to restore", + operatorName, + subtaskIndex); + globalStatistics = statisticsSerializer.createInstance(); + } else { + LOG.info( + "Restoring operator {} global statistics state for subtask {}", + operatorName, + subtaskIndex); + globalStatistics = globalStatisticsState.get().iterator().next(); + } + } else { + globalStatistics = statisticsSerializer.createInstance(); + } + } + + @Override + public void open() throws Exception { + if (!globalStatistics.isEmpty()) { + output.collect( + new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + } + + @Override + @SuppressWarnings("unchecked") + public void handleOperatorEvent(OperatorEvent event) { + int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + Preconditions.checkArgument( + event instanceof DataStatisticsEvent, + String.format( + "Operator %s subtask %s received unexpected operator event %s", + operatorName, subtaskIndex, event.getClass())); + DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; + LOG.info( + "Operator {} received global data event from coordinator checkpoint {}", + operatorName, + statisticsEvent.checkpointId()); + globalStatistics = + DataStatisticsUtil.deserializeDataStatistics( + statisticsEvent.statisticsBytes(), statisticsSerializer); + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + + @Override + public void processElement(StreamRecord streamRecord) { + RowData record = streamRecord.getValue(); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); + output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + long checkpointId = context.getCheckpointId(); + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + LOG.info( + "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", + operatorName, + checkpointId, + subTaskId); + + // Pass global statistics to partitioners so that all the operators refresh statistics + // at same checkpoint barrier + if (!globalStatistics.isEmpty()) { + output.collect( + new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + } + + // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores + // an exact copy of globalStatistics + if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { + globalStatisticsState.clear(); + LOG.info( + "Saving operator {} global statistics {} to state in subtask {}", + operatorName, + globalStatistics, + subTaskId); + globalStatisticsState.add(globalStatistics); + } + + // For now, local statistics are sent to coordinator at checkpoint + operatorEventGateway.sendEventToCoordinator( + DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); + LOG.debug( + "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", + subTaskId, + operatorName, + checkpointId, + localStatistics); + + // Recreate the local statistics + localStatistics = statisticsSerializer.createInstance(); + } + + @VisibleForTesting + DataStatistics localDataStatistics() { + return localStatistics; + } + + @VisibleForTesting + DataStatistics globalDataStatistics() { + return globalStatistics; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java new file mode 100644 index 000000000000..889e85112e16 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.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.flink.sink.shuffle; + +import java.io.Serializable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * The wrapper class for data statistics and record. It is the only way for data statistics operator + * to send global data statistics to custom partitioner to distribute data based on statistics + * + *

DataStatisticsOrRecord contains either data statistics(globally aggregated) or a record. It is + * sent from {@link DataStatisticsOperator} to partitioner. Once partitioner receives the data + * statistics, it will use that to decide the coming record should send to which writer subtask. + * After shuffling, a filter and mapper are required to filter out the data distribution weight, + * unwrap the object and extract the original record type T. + */ +class DataStatisticsOrRecord, S> implements Serializable { + + private static final long serialVersionUID = 1L; + + private DataStatistics statistics; + private RowData record; + + private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { + Preconditions.checkArgument( + record != null ^ statistics != null, "DataStatistics or record, not neither or both"); + this.statistics = statistics; + this.record = record; + } + + static , S> DataStatisticsOrRecord fromRecord( + RowData record) { + return new DataStatisticsOrRecord<>(null, record); + } + + static , S> DataStatisticsOrRecord fromDataStatistics( + DataStatistics statistics) { + return new DataStatisticsOrRecord<>(statistics, null); + } + + static , S> DataStatisticsOrRecord reuseRecord( + DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { + if (reuse.hasRecord()) { + return reuse; + } else { + // not reusable + return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + } + } + + static , S> DataStatisticsOrRecord reuseStatistics( + DataStatisticsOrRecord reuse, + TypeSerializer> statisticsSerializer) { + if (reuse.hasDataStatistics()) { + return reuse; + } else { + // not reusable + return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + } + } + + boolean hasDataStatistics() { + return statistics != null; + } + + boolean hasRecord() { + return record != null; + } + + DataStatistics dataStatistics() { + return statistics; + } + + void dataStatistics(DataStatistics newStatistics) { + this.statistics = newStatistics; + } + + RowData record() { + return record; + } + + void record(RowData newRecord) { + this.record = newRecord; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("statistics", statistics) + .add("record", record) + .toString(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java new file mode 100644 index 000000000000..e9a6fa0cbfc5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; + +@Internal +class DataStatisticsOrRecordSerializer, S> + extends TypeSerializer> { + private final TypeSerializer> statisticsSerializer; + private final TypeSerializer recordSerializer; + + DataStatisticsOrRecordSerializer( + TypeSerializer> statisticsSerializer, + TypeSerializer recordSerializer) { + this.statisticsSerializer = statisticsSerializer; + this.recordSerializer = recordSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer> duplicate() { + TypeSerializer> duplicateStatisticsSerializer = + statisticsSerializer.duplicate(); + TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); + if ((statisticsSerializer != duplicateStatisticsSerializer) + || (recordSerializer != duplicateRowDataSerializer)) { + return new DataStatisticsOrRecordSerializer<>( + duplicateStatisticsSerializer, duplicateRowDataSerializer); + } else { + return this; + } + } + + @Override + public DataStatisticsOrRecord createInstance() { + // arbitrarily always create RowData value instance + return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + } + + @Override + public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { + if (from.hasRecord()) { + return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + } else { + return DataStatisticsOrRecord.fromDataStatistics( + statisticsSerializer.copy(from.dataStatistics())); + } + } + + @Override + public DataStatisticsOrRecord copy( + DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { + DataStatisticsOrRecord to; + if (from.hasRecord()) { + to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + RowData record = recordSerializer.copy(from.record(), to.record()); + to.record(record); + } else { + to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + DataStatistics statistics = + statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); + to.dataStatistics(statistics); + } + + return to; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) + throws IOException { + if (statisticsOrRecord.hasRecord()) { + target.writeBoolean(true); + recordSerializer.serialize(statisticsOrRecord.record(), target); + } else { + target.writeBoolean(false); + statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + } + } + + @Override + public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { + boolean isRecord = source.readBoolean(); + if (isRecord) { + return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + } else { + return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + } + } + + @Override + public DataStatisticsOrRecord deserialize( + DataStatisticsOrRecord reuse, DataInputView source) throws IOException { + DataStatisticsOrRecord to; + boolean isRecord = source.readBoolean(); + if (isRecord) { + to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + RowData record = recordSerializer.deserialize(to.record(), source); + to.record(record); + } else { + to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + DataStatistics statistics = + statisticsSerializer.deserialize(to.dataStatistics(), source); + to.dataStatistics(statistics); + } + + return to; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean hasRecord = source.readBoolean(); + target.writeBoolean(hasRecord); + if (hasRecord) { + recordSerializer.copy(source, target); + } else { + statisticsSerializer.copy(source, target); + } + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + return false; + } + + @SuppressWarnings("unchecked") + DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; + return Objects.equals(statisticsSerializer, other.statisticsSerializer) + && Objects.equals(recordSerializer, other.recordSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(statisticsSerializer, recordSerializer); + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return new DataStatisticsOrRecordSerializerSnapshot<>(this); + } + + public static class DataStatisticsOrRecordSerializerSnapshot, S> + extends CompositeTypeSerializerSnapshot< + DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { + private static final int CURRENT_VERSION = 1; + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsOrRecordSerializerSnapshot() { + super(DataStatisticsOrRecordSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsOrRecordSerializerSnapshot( + DataStatisticsOrRecordSerializer serializer) { + super(serializer); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + DataStatisticsOrRecordSerializer outerSerializer) { + return new TypeSerializer[] { + outerSerializer.statisticsSerializer, outerSerializer.recordSerializer + }; + } + + @SuppressWarnings("unchecked") + @Override + protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + TypeSerializer> statisticsSerializer = + (TypeSerializer>) nestedSerializers[0]; + TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; + return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java new file mode 100644 index 000000000000..8716cb872d0e --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java @@ -0,0 +1,96 @@ +/* + * 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.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +/** + * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link + * AggregatedStatistics} + */ +class DataStatisticsUtil { + + private DataStatisticsUtil() {} + + static , S> byte[] serializeDataStatistics( + DataStatistics dataStatistics, + TypeSerializer> statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new IllegalStateException("Fail to serialize data statistics", e); + } + } + + @SuppressWarnings("unchecked") + static , S> D deserializeDataStatistics( + byte[] bytes, TypeSerializer> statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return (D) statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new IllegalStateException("Fail to deserialize data statistics", e); + } + } + + static , S> byte[] serializeAggregatedStatistics( + AggregatedStatistics aggregatedStatistics, + TypeSerializer> statisticsSerializer) + throws IOException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + ObjectOutputStream out = new ObjectOutputStream(bytes); + + DataOutputSerializer outSerializer = new DataOutputSerializer(64); + out.writeLong(aggregatedStatistics.checkpointId()); + statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); + byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); + out.writeInt(statisticsBytes.length); + out.write(statisticsBytes); + out.flush(); + + return bytes.toByteArray(); + } + + static , S> + AggregatedStatistics deserializeAggregatedStatistics( + byte[] bytes, TypeSerializer> statisticsSerializer) + throws IOException { + ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); + ObjectInputStream in = new ObjectInputStream(bytesIn); + + long completedCheckpointId = in.readLong(); + int statisticsBytesLength = in.readInt(); + byte[] statisticsBytes = new byte[statisticsBytesLength]; + in.readFully(statisticsBytes); + DataInputDeserializer input = + new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); + DataStatistics dataStatistics = statisticsSerializer.deserialize(input); + + return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java new file mode 100644 index 000000000000..0ffffd9cf49f --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -0,0 +1,70 @@ +/* + * 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.Map; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** MapDataStatistics uses map to count key frequency */ +@Internal +class MapDataStatistics implements DataStatistics> { + private final Map statistics; + + MapDataStatistics() { + this.statistics = Maps.newHashMap(); + } + + MapDataStatistics(Map statistics) { + this.statistics = statistics; + } + + @Override + public boolean isEmpty() { + return statistics.size() == 0; + } + + @Override + public void add(SortKey sortKey) { + if (statistics.containsKey(sortKey)) { + statistics.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + statistics.put(copiedKey, 1L); + } + } + + @Override + public void merge(MapDataStatistics otherStatistics) { + otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + } + + @Override + public Map statistics() { + return statistics; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java new file mode 100644 index 000000000000..b6cccd0566fc --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class MapDataStatisticsSerializer + extends TypeSerializer>> { + private final MapSerializer mapSerializer; + + static MapDataStatisticsSerializer fromSortKeySerializer( + TypeSerializer sortKeySerializer) { + return new MapDataStatisticsSerializer( + new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); + } + + MapDataStatisticsSerializer(MapSerializer mapSerializer) { + this.mapSerializer = mapSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer>> duplicate() { + MapSerializer duplicateMapSerializer = + (MapSerializer) mapSerializer.duplicate(); + return (duplicateMapSerializer == mapSerializer) + ? this + : new MapDataStatisticsSerializer(duplicateMapSerializer); + } + + @Override + public MapDataStatistics createInstance() { + return new MapDataStatistics(); + } + + @Override + public MapDataStatistics copy(DataStatistics> obj) { + Preconditions.checkArgument( + obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); + MapDataStatistics from = (MapDataStatistics) obj; + TypeSerializer keySerializer = mapSerializer.getKeySerializer(); + Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); + for (Map.Entry entry : from.statistics().entrySet()) { + SortKey newKey = keySerializer.copy(entry.getKey()); + // no need to copy value since it is just a Long + newMap.put(newKey, entry.getValue()); + } + + return new MapDataStatistics(newMap); + } + + @Override + public DataStatistics> copy( + DataStatistics> from, + DataStatistics> reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize( + DataStatistics> obj, DataOutputView target) + throws IOException { + Preconditions.checkArgument( + obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); + MapDataStatistics mapStatistics = (MapDataStatistics) obj; + mapSerializer.serialize(mapStatistics.statistics(), target); + } + + @Override + public DataStatistics> deserialize(DataInputView source) + throws IOException { + return new MapDataStatistics(mapSerializer.deserialize(source)); + } + + @Override + public DataStatistics> deserialize( + DataStatistics> reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + mapSerializer.copy(source, target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MapDataStatisticsSerializer)) { + return false; + } + + MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; + return Objects.equals(mapSerializer, other.mapSerializer); + } + + @Override + public int hashCode() { + return mapSerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot>> + snapshotConfiguration() { + return new MapDataStatisticsSerializerSnapshot(this); + } + + public static class MapDataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot< + DataStatistics>, MapDataStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public MapDataStatisticsSerializerSnapshot() { + super(MapDataStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + MapDataStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.mapSerializer}; + } + + @Override + protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + @SuppressWarnings("unchecked") + MapSerializer mapSerializer = + (MapSerializer) nestedSerializers[0]; + return new MapDataStatisticsSerializer(mapSerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..fb1a8f03a65c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,381 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +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; +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.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final Comparator comparator; + private final Map mapStatistics; + private final double closeFileCostInWeightPercentage; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + // lazily computed due to the need of numPartitions + private Map assignment; + private NavigableMap sortedStatsWithCloseFileCost; + + MapRangePartitioner( + Schema schema, + SortOrder sortOrder, + MapDataStatistics dataStatistics, + double closeFileCostInWeightPercentage) { + dataStatistics + .statistics() + .entrySet() + .forEach( + entry -> + Preconditions.checkArgument( + entry.getValue() > 0, + "Invalid statistics: weight is 0 for key %s", + entry.getKey())); + + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.mapStatistics = dataStatistics.statistics(); + this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // assignment table can only be built lazily when first referenced here, + // because number of partitions (downstream subtasks) is needed. + // the numPartitions is not available in the constructor. + Map assignmentMap = assignment(numPartitions); + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = assignmentMap.get(sortKey); + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + } + return (int) (newSortKeyCounter % numPartitions); + } + + return keyAssignment.select(); + } + + @VisibleForTesting + Map assignment(int numPartitions) { + if (assignment == null) { + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostInWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); + + this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + this.assignment = + buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostInWeight); + } + + return assignment; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + assignment.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { + int subtaskId = keyAssignment.assignedSubtasks[i]; + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + private Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostInWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostInWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostInWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks.clear(); + subtaskWeights.clear(); + currentKey = null; + } + } + + return assignmentMap; + } + + /** Subtask assignment for a key */ + @VisibleForTesting + static class KeyAssignment { + private final int[] assignedSubtasks; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It + * could also be multiple subtasks if the key has heavy weight that should be handled by + * multiple subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the + * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain + * values as [10, 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostInWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostInWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostInWeight)); + + this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.length == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks[0]; + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.length, + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks[position]; + } + } + + @Override + public int hashCode() { + return 31 * Arrays.hashCode(assignedSubtasks) + + Arrays.hashCode(subtaskWeightsExcludingCloseCost); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Arrays.equals(assignedSubtasks, that.assignedSubtasks) + && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) + .toString(); + } + } +} 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 new file mode 100644 index 000000000000..d03409f2a430 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,353 @@ +/* + * 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.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.UUID; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + // constructors need to public. Otherwise, Flink state restore would complain + // "The class has no (implicit) public nullary constructor". + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @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); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializer newSerializer) { + if (!(newSerializer instanceof SortKeySerializer)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; + return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + + @VisibleForTesting + static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + Schema readSchema, Schema writeSchema) { + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java new file mode 100644 index 000000000000..796434c45136 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import org.apache.avro.generic.GenericRecord; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; + +public class AvroGenericRecordFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataToAvroGenericRecordConverter converter; + + public AvroGenericRecordFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataToAvroGenericRecordConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java new file mode 100644 index 000000000000..91d975349b19 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -0,0 +1,156 @@ +/* + * 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.io.UncheckedIOException; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Flink data iterator that reads {@link CombinedScanTask} into a {@link CloseableIterator} + * + * @param is the output data type returned by this iterator. + */ +@Internal +public class DataIterator implements CloseableIterator { + + private final FileScanTaskReader fileScanTaskReader; + + private final InputFilesDecryptor inputFilesDecryptor; + private final CombinedScanTask combinedTask; + + private Iterator tasks; + private CloseableIterator currentIterator; + private int fileOffset; + private long recordOffset; + + public DataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption) { + this.fileScanTaskReader = fileScanTaskReader; + + this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption); + this.combinedTask = task; + + this.tasks = task.files().iterator(); + this.currentIterator = CloseableIterator.empty(); + + // fileOffset starts at -1 because we started + // from an empty iterator that is not from the split files. + this.fileOffset = -1; + // record offset points to the record that next() should return when called + this.recordOffset = 0L; + } + + /** + * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume + * from. E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the + * 2nd row in file 0. When next() is called after seek, 2nd row from file 0 should be returned. + */ + public void seek(int startingFileOffset, long startingRecordOffset) { + Preconditions.checkState( + fileOffset == -1, "Seek should be called before any other iterator actions"); + // skip files + Preconditions.checkState( + startingFileOffset < combinedTask.files().size(), + "Invalid starting file offset %s for combined scan task with %s files: %s", + startingFileOffset, + combinedTask.files().size(), + combinedTask); + for (long i = 0L; i < startingFileOffset; ++i) { + tasks.next(); + } + + updateCurrentIterator(); + // skip records within the file + for (long i = 0; i < startingRecordOffset; ++i) { + if (currentFileHasNext() && hasNext()) { + next(); + } else { + throw new IllegalStateException( + String.format( + "Invalid starting record offset %d for file %d from CombinedScanTask: %s", + startingRecordOffset, startingFileOffset, combinedTask)); + } + } + + fileOffset = startingFileOffset; + recordOffset = startingRecordOffset; + } + + @Override + public boolean hasNext() { + updateCurrentIterator(); + return currentIterator.hasNext(); + } + + @Override + public T next() { + updateCurrentIterator(); + recordOffset += 1; + return currentIterator.next(); + } + + public boolean currentFileHasNext() { + return currentIterator.hasNext(); + } + + /** Updates the current iterator field to ensure that the current Iterator is not exhausted. */ + private void updateCurrentIterator() { + try { + while (!currentIterator.hasNext() && tasks.hasNext()) { + currentIterator.close(); + currentIterator = openTaskIterator(tasks.next()); + fileOffset += 1; + recordOffset = 0L; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private CloseableIterator openTaskIterator(FileScanTask scanTask) { + return fileScanTaskReader.open(scanTask, inputFilesDecryptor); + } + + @Override + public void close() throws IOException { + // close the current iterator + currentIterator.close(); + tasks = null; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java new file mode 100644 index 000000000000..4394dab4d4cc --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java @@ -0,0 +1,47 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.flink.data.StructRowData; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; + +@Internal +public class DataTaskReader implements FileScanTaskReader { + + private final Schema readSchema; + + public DataTaskReader(Schema readSchema) { + this.readSchema = readSchema; + } + + @Override + public CloseableIterator open( + FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { + StructRowData row = new StructRowData(readSchema.asStruct()); + CloseableIterable iterable = + CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); + return iterable.iterator(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java new file mode 100644 index 000000000000..927a804a4792 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.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.flink.source; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Read a {@link FileScanTask} into a {@link CloseableIterator} + * + * @param is the output data type returned by this iterator. + */ +@Internal +public interface FileScanTaskReader extends Serializable { + CloseableIterator open(FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java new file mode 100644 index 000000000000..9a5123dc489e --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java @@ -0,0 +1,141 @@ +/* + * 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.util.concurrent.ExecutorService; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.LocatableInputSplitAssigner; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.util.ThreadPools; + +/** Flink {@link InputFormat} for Iceberg. */ +public class FlinkInputFormat extends RichInputFormat { + + private static final long serialVersionUID = 1L; + + private final TableLoader tableLoader; + private final FileIO io; + private final EncryptionManager encryption; + private final ScanContext context; + private final FileScanTaskReader rowDataReader; + + private transient DataIterator iterator; + private transient long currentReadCount = 0L; + + FlinkInputFormat( + TableLoader tableLoader, + Schema tableSchema, + FileIO io, + EncryptionManager encryption, + ScanContext context) { + this.tableLoader = tableLoader; + this.io = io; + this.encryption = encryption; + this.context = context; + + tableLoader.open(); + Table table = tableLoader.loadTable(); + if (table instanceof BaseMetadataTable) { + this.rowDataReader = new DataTaskReader(context.project()); + } else { + this.rowDataReader = + new RowDataFileScanTaskReader( + tableSchema, + context.project(), + context.nameMapping(), + context.caseSensitive(), + context.filters()); + } + } + + @VisibleForTesting + Schema projectedSchema() { + return context.project(); + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { + // Legacy method, not be used. + return null; + } + + @Override + public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { + // Called in Job manager, so it is OK to load table from catalog. + tableLoader.open(); + final ExecutorService workerPool = + ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); + try (TableLoader loader = tableLoader) { + Table table = loader.loadTable(); + return FlinkSplitPlanner.planInputSplits(table, context, workerPool); + } finally { + workerPool.shutdown(); + } + } + + @Override + public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { + return context.exposeLocality() + ? new LocatableInputSplitAssigner(inputSplits) + : new DefaultInputSplitAssigner(inputSplits); + } + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(FlinkInputSplit split) { + this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption); + } + + @Override + public boolean reachedEnd() { + if (context.limit() > 0 && currentReadCount >= context.limit()) { + return true; + } else { + return !iterator.hasNext(); + } + } + + @Override + public RowData nextRecord(RowData reuse) { + currentReadCount++; + return iterator.next(); + } + + @Override + public void close() throws IOException { + if (iterator != null) { + iterator.close(); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java new file mode 100644 index 000000000000..16fd4f39596c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java @@ -0,0 +1,48 @@ +/* + * 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.util.Arrays; +import javax.annotation.Nullable; +import org.apache.flink.core.io.LocatableInputSplit; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +public class FlinkInputSplit extends LocatableInputSplit { + + private final CombinedScanTask task; + + FlinkInputSplit(int splitNumber, CombinedScanTask task, @Nullable String[] hostnames) { + super(splitNumber, hostnames); + this.task = task; + } + + CombinedScanTask getTask() { + return task; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("splitNumber", getSplitNumber()) + .add("task", task) + .add("hosts", Arrays.toString(getHostnames())) + .toString(); + } +} 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 new file mode 100644 index 000000000000..fa1656c55278 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -0,0 +1,310 @@ +/* + * 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.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +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.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.FileIO; +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.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FlinkSource { + private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); + + private FlinkSource() {} + + /** + * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link + * TableScan}. See more options in {@link ScanContext}. + * + *

The Source can be read static data in bounded mode. It can also continuously check the + * arrival of new data and read records incrementally. + * + *

    + *
  • Without startSnapshotId: Bounded + *
  • With startSnapshotId and with endSnapshotId: Bounded + *
  • With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded + *
+ * + *

+ * + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData() { + return new Builder(); + } + + /** Source builder to build {@link DataStream}. */ + public static class Builder { + private StreamExecutionEnvironment env; + private Table table; + private TableLoader tableLoader; + private TableSchema projectedSchema; + private ReadableConfig readableConfig = new Configuration(); + private final ScanContext.Builder contextBuilder = ScanContext.builder(); + private Boolean exposeLocality; + + private final Map readOptions = Maps.newHashMap(); + + public Builder tableLoader(TableLoader newLoader) { + this.tableLoader = newLoader; + return this; + } + + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + public Builder env(StreamExecutionEnvironment newEnv) { + this.env = newEnv; + return this; + } + + public Builder filters(List filters) { + contextBuilder.filters(filters); + return this; + } + + public Builder project(TableSchema schema) { + this.projectedSchema = schema; + return this; + } + + public Builder limit(Long newLimit) { + if (newLimit != null) { + readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); + } + return this; + } + + public Builder set(String property, String value) { + readOptions.put(property, value); + return this; + } + + public Builder setAll(Map properties) { + readOptions.putAll(properties); + return this; + } + + /** @deprecated Use {@link #setAll} instead. */ + @Deprecated + public Builder properties(Map properties) { + readOptions.putAll(properties); + return this; + } + + public Builder caseSensitive(boolean caseSensitive) { + readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(caseSensitive)); + return this; + } + + public Builder snapshotId(Long snapshotId) { + readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(snapshotId)); + return this; + } + + public Builder branch(String branch) { + readOptions.put(FlinkReadOptions.BRANCH.key(), branch); + return this; + } + + public Builder tag(String tag) { + readOptions.put(FlinkReadOptions.TAG.key(), tag); + return this; + } + + public Builder startSnapshotId(Long startSnapshotId) { + readOptions.put(FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(startSnapshotId)); + return this; + } + + public Builder endSnapshotId(Long endSnapshotId) { + readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(endSnapshotId)); + return this; + } + + public Builder startTag(String startTag) { + readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); + return this; + } + + public Builder endTag(String endTag) { + readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); + return this; + } + + public Builder asOfTimestamp(Long asOfTimestamp) { + readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(asOfTimestamp)); + return this; + } + + public Builder splitSize(Long splitSize) { + readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(splitSize)); + return this; + } + + public Builder splitLookback(Integer splitLookback) { + readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(splitLookback)); + return this; + } + + public Builder splitOpenFileCost(Long splitOpenFileCost) { + readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(splitOpenFileCost)); + return this; + } + + public Builder streaming(boolean streaming) { + readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder nameMapping(String nameMapping) { + readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, nameMapping); + return this; + } + + public Builder monitorInterval(Duration interval) { + readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, interval.toNanos() + " ns"); + return this; + } + + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + readOptions.put( + FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT, + Integer.toString(newMaxPlanningSnapshotCount)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + public FlinkInputFormat buildFormat() { + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + + Schema icebergSchema; + FileIO io; + EncryptionManager encryption; + if (table == null) { + // load required fields by table loader. + tableLoader.open(); + try (TableLoader loader = tableLoader) { + table = loader.loadTable(); + icebergSchema = table.schema(); + io = table.io(); + encryption = table.encryption(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + icebergSchema = table.schema(); + io = table.io(); + encryption = table.encryption(); + } + + if (projectedSchema == null) { + contextBuilder.project(icebergSchema); + } else { + contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); + } + + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, readableConfig, exposeLocality)); + contextBuilder.planParallelism( + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); + + contextBuilder.resolveConfig(table, readOptions, readableConfig); + + return new FlinkInputFormat( + tableLoader, icebergSchema, io, encryption, contextBuilder.build()); + } + + public DataStream build() { + Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); + FlinkInputFormat format = buildFormat(); + + ScanContext context = contextBuilder.build(); + TypeInformation typeInfo = + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project())); + + if (!context.isStreaming()) { + int parallelism = + SourceUtil.inferParallelism( + readableConfig, + context.limit(), + () -> { + try { + return format.createInputSplits(0).length; + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to create iceberg input splits for table: " + table, e); + } + }); + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + return env.createInput(format, typeInfo).setParallelism(parallelism); + } else { + StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context); + + String monitorFunctionName = String.format("Iceberg table (%s) monitor", table); + String readerOperatorName = String.format("Iceberg table (%s) reader", table); + + return env.addSource(function, monitorFunctionName) + .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format)); + } + } + } + + public static boolean isBounded(Map properties) { + return !PropertyUtil.propertyAsBoolean(properties, FlinkReadOptions.STREAMING, false); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java new file mode 100644 index 000000000000..15078809714f --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java @@ -0,0 +1,189 @@ +/* + * 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.io.UncheckedIOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.IncrementalAppendScan; +import org.apache.iceberg.Scan; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.CloseableIterable; +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.apache.iceberg.util.Tasks; + +@Internal +public class FlinkSplitPlanner { + private FlinkSplitPlanner() {} + + static FlinkInputSplit[] planInputSplits( + Table table, ScanContext context, ExecutorService workerPool) { + try (CloseableIterable tasksIterable = + planTasks(table, context, workerPool)) { + List tasks = Lists.newArrayList(tasksIterable); + FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()]; + boolean exposeLocality = context.exposeLocality(); + + Tasks.range(tasks.size()) + .stopOnFailure() + .executeWith(exposeLocality ? workerPool : null) + .run( + index -> { + CombinedScanTask task = tasks.get(index); + String[] hostnames = null; + if (exposeLocality) { + hostnames = Util.blockLocations(table.io(), task); + } + splits[index] = new FlinkInputSplit(index, task, hostnames); + }); + return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to process tasks iterable", e); + } + } + + /** This returns splits for the FLIP-27 source */ + public static List planIcebergSourceSplits( + Table table, ScanContext context, ExecutorService workerPool) { + try (CloseableIterable tasksIterable = + planTasks(table, context, workerPool)) { + return Lists.newArrayList( + CloseableIterable.transform(tasksIterable, IcebergSourceSplit::fromCombinedScanTask)); + } catch (IOException e) { + throw new UncheckedIOException("Failed to process task iterable: ", e); + } + } + + static CloseableIterable planTasks( + Table table, ScanContext context, ExecutorService workerPool) { + ScanMode scanMode = checkScanMode(context); + if (scanMode == ScanMode.INCREMENTAL_APPEND_SCAN) { + IncrementalAppendScan scan = table.newIncrementalAppendScan(); + scan = refineScanWithBaseConfigs(scan, context, workerPool); + + if (context.startTag() != null) { + Preconditions.checkArgument( + table.snapshot(context.startTag()) != null, + "Cannot find snapshot with tag %s", + context.startTag()); + scan = scan.fromSnapshotExclusive(table.snapshot(context.startTag()).snapshotId()); + } + + if (context.startSnapshotId() != null) { + Preconditions.checkArgument( + context.startTag() == null, "START_SNAPSHOT_ID and START_TAG cannot both be set"); + scan = scan.fromSnapshotExclusive(context.startSnapshotId()); + } + + if (context.endTag() != null) { + Preconditions.checkArgument( + table.snapshot(context.endTag()) != null, + "Cannot find snapshot with tag %s", + context.endTag()); + scan = scan.toSnapshot(table.snapshot(context.endTag()).snapshotId()); + } + + if (context.endSnapshotId() != null) { + Preconditions.checkArgument( + context.endTag() == null, "END_SNAPSHOT_ID and END_TAG cannot both be set"); + scan = scan.toSnapshot(context.endSnapshotId()); + } + + return scan.planTasks(); + } else { + TableScan scan = table.newScan(); + scan = refineScanWithBaseConfigs(scan, context, workerPool); + + if (context.snapshotId() != null) { + scan = scan.useSnapshot(context.snapshotId()); + } else if (context.tag() != null) { + scan = scan.useRef(context.tag()); + } else if (context.branch() != null) { + scan = scan.useRef(context.branch()); + } + + if (context.asOfTimestamp() != null) { + scan = scan.asOfTime(context.asOfTimestamp()); + } + + return scan.planTasks(); + } + } + + @VisibleForTesting + enum ScanMode { + BATCH, + INCREMENTAL_APPEND_SCAN + } + + @VisibleForTesting + static ScanMode checkScanMode(ScanContext context) { + if (context.startSnapshotId() != null + || context.endSnapshotId() != null + || context.startTag() != null + || context.endTag() != null) { + return ScanMode.INCREMENTAL_APPEND_SCAN; + } else { + return ScanMode.BATCH; + } + } + + /** refine scan with common configs */ + private static > T refineScanWithBaseConfigs( + T scan, ScanContext context, ExecutorService workerPool) { + T refinedScan = + scan.caseSensitive(context.caseSensitive()).project(context.project()).planWith(workerPool); + + if (context.includeColumnStats()) { + refinedScan = refinedScan.includeColumnStats(); + } + + if (context.includeStatsForColumns() != null) { + refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); + } + + refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); + + refinedScan = + refinedScan.option(TableProperties.SPLIT_LOOKBACK, context.splitLookback().toString()); + + refinedScan = + refinedScan.option( + TableProperties.SPLIT_OPEN_FILE_COST, context.splitOpenFileCost().toString()); + + if (context.filters() != null) { + for (Expression filter : context.filters()) { + refinedScan = refinedScan.filter(filter); + } + } + + return refinedScan; + } +} 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 new file mode 100644 index 000000000000..0655cf87a996 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -0,0 +1,543 @@ +/* + * 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.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +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.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadConf; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; +import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; +import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner; +import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl; +import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; +import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; +import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.IcebergSourceReader; +import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; +import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; +import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +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.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +public class IcebergSource implements Source { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); + + // This table loader can be closed, and it is only safe to use this instance for resource + // independent information (e.g. a table name). Copies of this are required to avoid lifecycle + // management conflicts with the user provided table loader. e.g. a copy of this is required for + // split planning, which uses the underlying io, and should be closed after split planning is + // complete. + private final TableLoader tableLoader; + private final ScanContext scanContext; + private final ReaderFunction readerFunction; + private final SplitAssignerFactory assignerFactory; + private final SerializableComparator splitComparator; + private final SerializableRecordEmitter emitter; + private final String tableName; + + IcebergSource( + TableLoader tableLoader, + ScanContext scanContext, + ReaderFunction readerFunction, + SplitAssignerFactory assignerFactory, + SerializableComparator splitComparator, + Table table, + SerializableRecordEmitter emitter) { + Preconditions.checkNotNull(tableLoader, "tableLoader is required."); + Preconditions.checkNotNull(readerFunction, "readerFunction is required."); + Preconditions.checkNotNull(assignerFactory, "assignerFactory is required."); + Preconditions.checkNotNull(table, "table is required."); + this.tableLoader = tableLoader; + this.scanContext = scanContext; + this.readerFunction = readerFunction; + this.assignerFactory = assignerFactory; + this.splitComparator = splitComparator; + this.emitter = emitter; + this.tableName = table.name(); + } + + String name() { + return "IcebergSource-" + tableName; + } + + private String planningThreadName() { + // Ideally, operatorId should be used as the threadPoolName as Flink guarantees its uniqueness + // within a job. SplitEnumeratorContext doesn't expose the OperatorCoordinator.Context, which + // would contain the OperatorID. Need to discuss with Flink community whether it is ok to expose + // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" + // from SourceCoordinatorContext implementation. For now,

- is used as + // the unique thread pool name. + return tableName + "-" + UUID.randomUUID(); + } + + private List planSplitsForBatch(String threadName) { + ExecutorService workerPool = + ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); + try (TableLoader loader = tableLoader.clone()) { + loader.open(); + List splits = + FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); + LOG.info( + "Discovered {} splits from table {} during job initialization", splits.size(), tableName); + return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to close table loader", e); + } finally { + workerPool.shutdown(); + } + } + + @Override + public Boundedness getBoundedness() { + return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) { + IcebergSourceReaderMetrics metrics = + new IcebergSourceReaderMetrics(readerContext.metricGroup(), tableName); + return new IcebergSourceReader<>( + emitter, metrics, readerFunction, splitComparator, readerContext); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return createEnumerator(enumContext, null); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, IcebergEnumeratorState enumState) { + return createEnumerator(enumContext, enumState); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); + } + + private SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext, + @Nullable IcebergEnumeratorState enumState) { + SplitAssigner assigner; + if (enumState == null) { + assigner = assignerFactory.createAssigner(); + } else { + LOG.info( + "Iceberg source restored {} splits from state for table {}", + enumState.pendingSplits().size(), + tableName); + assigner = assignerFactory.createAssigner(enumState.pendingSplits()); + } + if (scanContext.isStreaming()) { + ContinuousSplitPlanner splitPlanner = + new ContinuousSplitPlannerImpl(tableLoader, scanContext, planningThreadName()); + return new ContinuousIcebergEnumerator( + enumContext, assigner, scanContext, splitPlanner, enumState); + } else { + List splits = planSplitsForBatch(planningThreadName()); + assigner.onDiscoveredSplits(splits); + return new StaticIcebergEnumerator(enumContext, assigner); + } + } + + public static Builder builder() { + return new Builder<>(); + } + + public static Builder forRowData() { + return new Builder<>(); + } + + public static class Builder { + private TableLoader tableLoader; + private Table table; + private SplitAssignerFactory splitAssignerFactory; + private SerializableComparator splitComparator; + private ReaderFunction readerFunction; + private ReadableConfig flinkConfig = new Configuration(); + private final ScanContext.Builder contextBuilder = ScanContext.builder(); + private TableSchema projectedFlinkSchema; + private Boolean exposeLocality; + + private final Map readOptions = Maps.newHashMap(); + + Builder() {} + + public Builder tableLoader(TableLoader loader) { + this.tableLoader = loader; + return this; + } + + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + this.splitAssignerFactory = assignerFactory; + return this; + } + + public Builder splitComparator( + SerializableComparator newSplitComparator) { + this.splitComparator = newSplitComparator; + return this; + } + + public Builder readerFunction(ReaderFunction newReaderFunction) { + this.readerFunction = newReaderFunction; + return this; + } + + public Builder flinkConfig(ReadableConfig config) { + this.flinkConfig = config; + return this; + } + + public Builder caseSensitive(boolean newCaseSensitive) { + readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(newCaseSensitive)); + return this; + } + + public Builder useSnapshotId(Long newSnapshotId) { + if (newSnapshotId != null) { + readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(newSnapshotId)); + } + return this; + } + + public Builder streamingStartingStrategy(StreamingStartingStrategy newStartingStrategy) { + readOptions.put(FlinkReadOptions.STARTING_STRATEGY, newStartingStrategy.name()); + return this; + } + + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + if (newStartSnapshotTimestamp != null) { + readOptions.put( + FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key(), + Long.toString(newStartSnapshotTimestamp)); + } + return this; + } + + public Builder startSnapshotId(Long newStartSnapshotId) { + if (newStartSnapshotId != null) { + readOptions.put( + FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(newStartSnapshotId)); + } + return this; + } + + public Builder tag(String tag) { + readOptions.put(FlinkReadOptions.TAG.key(), tag); + return this; + } + + public Builder branch(String branch) { + readOptions.put(FlinkReadOptions.BRANCH.key(), branch); + return this; + } + + public Builder startTag(String startTag) { + readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); + return this; + } + + public Builder endTag(String endTag) { + readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); + return this; + } + + public Builder endSnapshotId(Long newEndSnapshotId) { + if (newEndSnapshotId != null) { + readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(newEndSnapshotId)); + } + return this; + } + + public Builder asOfTimestamp(Long newAsOfTimestamp) { + if (newAsOfTimestamp != null) { + readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(newAsOfTimestamp)); + } + return this; + } + + public Builder splitSize(Long newSplitSize) { + if (newSplitSize != null) { + readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(newSplitSize)); + } + return this; + } + + public Builder splitLookback(Integer newSplitLookback) { + if (newSplitLookback != null) { + readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(newSplitLookback)); + } + return this; + } + + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { + if (newSplitOpenFileCost != null) { + readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(newSplitOpenFileCost)); + } + + return this; + } + + public Builder streaming(boolean streaming) { + readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); + return this; + } + + public Builder monitorInterval(Duration newMonitorInterval) { + if (newMonitorInterval != null) { + readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, newMonitorInterval.toNanos() + " ns"); + } + return this; + } + + public Builder nameMapping(String newNameMapping) { + readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, newNameMapping); + return this; + } + + public Builder project(Schema newProjectedSchema) { + this.contextBuilder.project(newProjectedSchema); + return this; + } + + public Builder project(TableSchema newProjectedFlinkSchema) { + this.projectedFlinkSchema = newProjectedFlinkSchema; + return this; + } + + public Builder filters(List newFilters) { + this.contextBuilder.filters(newFilters); + return this; + } + + public Builder limit(Long newLimit) { + if (newLimit != null) { + readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); + } + return this; + } + + public Builder includeColumnStats(boolean newIncludeColumnStats) { + readOptions.put( + FlinkReadOptions.INCLUDE_COLUMN_STATS, Boolean.toString(newIncludeColumnStats)); + return this; + } + + public Builder planParallelism(int planParallelism) { + readOptions.put( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key(), + Integer.toString(planParallelism)); + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder maxAllowedPlanningFailures(int maxAllowedPlanningFailures) { + readOptions.put( + FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.key(), + Integer.toString(maxAllowedPlanningFailures)); + return this; + } + + /** + * Set the read properties for Flink source. View the supported properties in {@link + * FlinkReadOptions} + */ + public Builder set(String property, String value) { + readOptions.put(property, value); + return this; + } + + /** + * Set the read properties for Flink source. View the supported properties in {@link + * FlinkReadOptions} + */ + public Builder setAll(Map properties) { + readOptions.putAll(properties); + return this; + } + + /** + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkColumnTimeUnit(TimeUnit)}. + * + *

Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. + */ + public Builder watermarkColumn(String columnName) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "Watermark column and SplitAssigner should not be set in the same source"); + readOptions.put(FlinkReadOptions.WATERMARK_COLUMN, columnName); + return this; + } + + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { + readOptions.put(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT, timeUnit.name()); + return this; + } + + /** @deprecated Use {@link #setAll} instead. */ + @Deprecated + public Builder properties(Map properties) { + readOptions.putAll(properties); + return this; + } + + public IcebergSource build() { + if (table == null) { + try (TableLoader loader = tableLoader) { + loader.open(); + this.table = tableLoader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + contextBuilder.resolveConfig(table, readOptions, flinkConfig); + Schema icebergSchema = table.schema(); + if (projectedFlinkSchema != null) { + contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); + } + + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, flinkConfig); + String watermarkColumn = flinkReadConf.watermarkColumn(); + TimeUnit watermarkTimeUnit = flinkReadConf.watermarkColumnTimeUnit(); + + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); + } + + ScanContext context = contextBuilder.build(); + if (readerFunction == null) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + this.readerFunction = (ReaderFunction) rowDataReaderFunction; + } else { + RowDataReaderFunction rowDataReaderFunction = + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters()); + this.readerFunction = (ReaderFunction) rowDataReaderFunction; + } + } + + if (splitAssignerFactory == null) { + if (splitComparator == null) { + splitAssignerFactory = new SimpleSplitAssignerFactory(); + } else { + splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); + } + } + + // Since builder already load the table, pass it to the source to avoid double loading + return new IcebergSource<>( + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitter); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java new file mode 100644 index 000000000000..610657e8d47b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.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.flink.source; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.types.DataType; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkFilters; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.SplitAssignerType; +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; + +/** Flink Iceberg table source. */ +@Internal +public class IcebergTableSource + implements ScanTableSource, + SupportsProjectionPushDown, + SupportsFilterPushDown, + SupportsLimitPushDown { + + private int[] projectedFields; + private Long limit; + private List filters; + + private final TableLoader loader; + private final TableSchema schema; + private final Map properties; + private final boolean isLimitPushDown; + private final ReadableConfig readableConfig; + + private IcebergTableSource(IcebergTableSource toCopy) { + this.loader = toCopy.loader; + this.schema = toCopy.schema; + this.properties = toCopy.properties; + this.projectedFields = toCopy.projectedFields; + this.isLimitPushDown = toCopy.isLimitPushDown; + this.limit = toCopy.limit; + this.filters = toCopy.filters; + this.readableConfig = toCopy.readableConfig; + } + + public IcebergTableSource( + TableLoader loader, + TableSchema schema, + Map properties, + ReadableConfig readableConfig) { + this(loader, schema, properties, null, false, null, ImmutableList.of(), readableConfig); + } + + private IcebergTableSource( + TableLoader loader, + TableSchema schema, + Map properties, + int[] projectedFields, + boolean isLimitPushDown, + Long limit, + List filters, + ReadableConfig readableConfig) { + this.loader = loader; + this.schema = schema; + this.properties = properties; + this.projectedFields = projectedFields; + this.isLimitPushDown = isLimitPushDown; + this.limit = limit; + this.filters = filters; + this.readableConfig = readableConfig; + } + + @Override + public void applyProjection(int[][] projectFields) { + this.projectedFields = new int[projectFields.length]; + for (int i = 0; i < projectFields.length; i++) { + Preconditions.checkArgument( + projectFields[i].length == 1, "Don't support nested projection in iceberg source now."); + this.projectedFields[i] = projectFields[i][0]; + } + } + + private DataStream createDataStream(StreamExecutionEnvironment execEnv) { + return FlinkSource.forRowData() + .env(execEnv) + .tableLoader(loader) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConf(readableConfig) + .build(); + } + + private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + SplitAssignerType assignerType = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); + IcebergSource source = + IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .build(); + DataStreamSource stream = + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + source.name(), + TypeInformation.of(RowData.class)); + return stream; + } + + private TableSchema getProjectedSchema() { + if (projectedFields == null) { + return schema; + } else { + String[] fullNames = schema.getFieldNames(); + DataType[] fullTypes = schema.getFieldDataTypes(); + return TableSchema.builder() + .fields( + Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), + Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)) + .build(); + } + } + + @Override + public void applyLimit(long newLimit) { + this.limit = newLimit; + } + + @Override + public Result applyFilters(List flinkFilters) { + List acceptedFilters = Lists.newArrayList(); + List expressions = Lists.newArrayList(); + + for (ResolvedExpression resolvedExpression : flinkFilters) { + Optional icebergExpression = FlinkFilters.convert(resolvedExpression); + if (icebergExpression.isPresent()) { + expressions.add(icebergExpression.get()); + acceptedFilters.add(resolvedExpression); + } + } + + this.filters = expressions; + return Result.of(acceptedFilters, flinkFilters); + } + + @Override + public boolean supportsNestedProjection() { + // TODO: support nested projection + return false; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment execEnv) { + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { + return createFLIP27Stream(execEnv); + } else { + return createDataStream(execEnv); + } + } + + @Override + public boolean isBounded() { + return FlinkSource.isBounded(properties); + } + }; + } + + @Override + public DynamicTableSource copy() { + return new IcebergTableSource(this); + } + + @Override + public String asSummaryString() { + return "Iceberg table source"; + } +} 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 new file mode 100644 index 000000000000..88364f4e87b1 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -0,0 +1,243 @@ +/* + * 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.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +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.RowDataProjection; +import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; + +@Internal +public class RowDataFileScanTaskReader implements FileScanTaskReader { + + private final Schema tableSchema; + private final Schema projectedSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FlinkSourceFilter rowFilter; + + public RowDataFileScanTaskReader( + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + List filters) { + this.tableSchema = tableSchema; + this.projectedSchema = projectedSchema; + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + + if (filters != null && !filters.isEmpty()) { + Expression combinedExpression = + filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); + this.rowFilter = + new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); + } else { + this.rowFilter = null; + } + } + + @Override + public CloseableIterator open( + FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { + Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds()); + + Map idToConstant = + partitionSchema.columns().isEmpty() + ? ImmutableMap.of() + : PartitionUtil.constantsMap(task, RowDataUtil::convertConstant); + + FlinkDeleteFilter deletes = + new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor); + CloseableIterable iterable = + deletes.filter( + newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor)); + + // Project the RowData to remove the extra meta columns. + if (!projectedSchema.sameSchema(deletes.requiredSchema())) { + RowDataProjection rowDataProjection = + RowDataProjection.create( + deletes.requiredRowType(), + deletes.requiredSchema().asStruct(), + projectedSchema.asStruct()); + iterable = CloseableIterable.transform(iterable, rowDataProjection::wrap); + } + + return iterable.iterator(); + } + + private CloseableIterable newIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + CloseableIterable iter; + if (task.isDataTask()) { + throw new UnsupportedOperationException("Cannot read data task."); + } else { + switch (task.file().format()) { + case PARQUET: + iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + case AVRO: + iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + case ORC: + iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + default: + throw new UnsupportedOperationException( + "Cannot read unknown format: " + task.file().format()); + } + } + + if (rowFilter != null) { + return CloseableIterable.filter(iter, rowFilter::filter); + } + return iter; + } + + private CloseableIterable newAvroIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Avro.ReadBuilder builder = + Avro.read(inputFilesDecryptor.getInputFile(task)) + .reuseContainers() + .project(schema) + .split(task.start(), task.length()) + .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private CloseableIterable newParquetIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Parquet.ReadBuilder builder = + Parquet.read(inputFilesDecryptor.getInputFile(task)) + .split(task.start(), task.length()) + .project(schema) + .createReaderFunc( + fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) + .filter(task.residual()) + .caseSensitive(caseSensitive) + .reuseContainers(); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private CloseableIterable newOrcIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Schema readSchemaWithoutConstantAndMetadataFields = + TypeUtil.selectNot( + schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); + + ORC.ReadBuilder builder = + ORC.read(inputFilesDecryptor.getInputFile(task)) + .project(readSchemaWithoutConstantAndMetadataFields) + .split(task.start(), task.length()) + .createReaderFunc( + readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) + .filter(task.residual()) + .caseSensitive(caseSensitive); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private static class FlinkDeleteFilter extends DeleteFilter { + private final RowType requiredRowType; + private final RowDataWrapper asStructLike; + private final InputFilesDecryptor inputFilesDecryptor; + + FlinkDeleteFilter( + FileScanTask task, + Schema tableSchema, + Schema requestedSchema, + InputFilesDecryptor inputFilesDecryptor) { + super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); + this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); + this.inputFilesDecryptor = inputFilesDecryptor; + } + + public RowType requiredRowType() { + return requiredRowType; + } + + @Override + protected StructLike asStructLike(RowData row) { + return asStructLike.wrap(row); + } + + @Override + protected InputFile getInputFile(String location) { + return inputFilesDecryptor.getInputFile(location); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java new file mode 100644 index 000000000000..c958604c004a --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -0,0 +1,172 @@ +/* + * 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 static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RowDataRewriter { + + private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); + + private final Schema schema; + private final String nameMapping; + private final FileIO io; + private final boolean caseSensitive; + private final EncryptionManager encryptionManager; + private final TaskWriterFactory taskWriterFactory; + private final String tableName; + + public RowDataRewriter( + Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { + this.schema = table.schema(); + this.caseSensitive = caseSensitive; + this.io = io; + this.encryptionManager = encryptionManager; + this.nameMapping = + PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); + this.tableName = table.name(); + + String formatString = + PropertyUtil.propertyAsString( + table.properties(), + TableProperties.DEFAULT_FILE_FORMAT, + TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); + FileFormat format = FileFormat.fromString(formatString); + RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); + this.taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkSchema, + Long.MAX_VALUE, + format, + table.properties(), + null, + false); + } + + public List rewriteDataForTasks( + DataStream dataStream, int parallelism) throws Exception { + RewriteMap map = + new RewriteMap( + schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); + DataStream> ds = dataStream.map(map).setParallelism(parallelism); + return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + public static class RewriteMap extends RichMapFunction> { + + private TaskWriter writer; + private int subTaskId; + private int attemptId; + + private final Schema schema; + private final String nameMapping; + private final FileIO io; + private final boolean caseSensitive; + private final EncryptionManager encryptionManager; + private final TaskWriterFactory taskWriterFactory; + private final RowDataFileScanTaskReader rowDataReader; + + public RewriteMap( + Schema schema, + String nameMapping, + FileIO io, + boolean caseSensitive, + EncryptionManager encryptionManager, + TaskWriterFactory taskWriterFactory) { + this.schema = schema; + this.nameMapping = nameMapping; + this.io = io; + this.caseSensitive = caseSensitive; + this.encryptionManager = encryptionManager; + this.taskWriterFactory = taskWriterFactory; + this.rowDataReader = + new RowDataFileScanTaskReader( + schema, schema, nameMapping, caseSensitive, Collections.emptyList()); + } + + @Override + public void open(Configuration parameters) { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + // Initialize the task writer factory. + this.taskWriterFactory.initialize(subTaskId, attemptId); + } + + @Override + public List map(CombinedScanTask task) throws Exception { + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + try (DataIterator iterator = + new DataIterator<>(rowDataReader, task, io, encryptionManager)) { + while (iterator.hasNext()) { + RowData rowData = iterator.next(); + writer.write(rowData); + } + return Lists.newArrayList(writer.dataFiles()); + } catch (Throwable originalThrowable) { + try { + LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); + writer.abort(); + LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); + } catch (Throwable inner) { + if (originalThrowable != inner) { + originalThrowable.addSuppressed(inner); + LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); + } + } + + if (originalThrowable instanceof Exception) { + throw originalThrowable; + } else { + throw new RuntimeException(originalThrowable); + } + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java new file mode 100644 index 000000000000..8f95e3e554a0 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java @@ -0,0 +1,70 @@ +/* + * 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.Serializable; +import java.util.function.Function; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +/** + * This is not serializable because Avro {@link Schema} is not actually serializable, even though it + * implements {@link Serializable} interface. + */ +@Internal +public class RowDataToAvroGenericRecordConverter implements Function { + private final RowDataToAvroConverters.RowDataToAvroConverter converter; + private final Schema avroSchema; + + private RowDataToAvroGenericRecordConverter(RowType rowType, Schema avroSchema) { + this.converter = RowDataToAvroConverters.createConverter(rowType); + this.avroSchema = avroSchema; + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) converter.convert(avroSchema, rowData); + } + + /** Create a converter based on Iceberg schema */ + public static RowDataToAvroGenericRecordConverter fromIcebergSchema( + String tableName, org.apache.iceberg.Schema icebergSchema) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); + } + + /** Create a mapper based on Avro schema */ + public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java new file mode 100644 index 000000000000..cf57a126ae59 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -0,0 +1,592 @@ +/* + * 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.Serializable; +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadConf; +import org.apache.iceberg.flink.FlinkReadOptions; + +/** Context object with optional arguments for a Flink Scan. */ +@Internal +public class ScanContext implements Serializable { + + private static final long serialVersionUID = 1L; + + private final boolean caseSensitive; + private final boolean exposeLocality; + private final Long snapshotId; + private final String branch; + private final String tag; + private final StreamingStartingStrategy startingStrategy; + private final Long startSnapshotId; + private final Long startSnapshotTimestamp; + private final Long endSnapshotId; + private final Long asOfTimestamp; + private final String startTag; + private final String endTag; + private final Long splitSize; + private final Integer splitLookback; + private final Long splitOpenFileCost; + private final boolean isStreaming; + private final Duration monitorInterval; + + private final String nameMapping; + private final Schema schema; + private final List filters; + private final long limit; + private final boolean includeColumnStats; + private final Collection includeStatsForColumns; + private final Integer planParallelism; + private final int maxPlanningSnapshotCount; + private final int maxAllowedPlanningFailures; + private final String watermarkColumn; + private final TimeUnit watermarkColumnTimeUnit; + + private ScanContext( + boolean caseSensitive, + Long snapshotId, + StreamingStartingStrategy startingStrategy, + Long startSnapshotTimestamp, + Long startSnapshotId, + Long endSnapshotId, + Long asOfTimestamp, + Long splitSize, + Integer splitLookback, + Long splitOpenFileCost, + boolean isStreaming, + Duration monitorInterval, + String nameMapping, + Schema schema, + List filters, + long limit, + boolean includeColumnStats, + Collection includeStatsForColumns, + boolean exposeLocality, + Integer planParallelism, + int maxPlanningSnapshotCount, + int maxAllowedPlanningFailures, + String watermarkColumn, + TimeUnit watermarkColumnTimeUnit, + String branch, + String tag, + String startTag, + String endTag) { + this.caseSensitive = caseSensitive; + this.snapshotId = snapshotId; + this.tag = tag; + this.branch = branch; + this.startingStrategy = startingStrategy; + this.startSnapshotTimestamp = startSnapshotTimestamp; + this.startSnapshotId = startSnapshotId; + this.endSnapshotId = endSnapshotId; + this.asOfTimestamp = asOfTimestamp; + this.startTag = startTag; + this.endTag = endTag; + this.splitSize = splitSize; + this.splitLookback = splitLookback; + this.splitOpenFileCost = splitOpenFileCost; + this.isStreaming = isStreaming; + this.monitorInterval = monitorInterval; + + this.nameMapping = nameMapping; + this.schema = schema; + this.filters = filters; + this.limit = limit; + this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; + this.exposeLocality = exposeLocality; + this.planParallelism = planParallelism; + this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; + this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; + this.watermarkColumn = watermarkColumn; + this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; + + validate(); + } + + private void validate() { + if (isStreaming) { + if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { + Preconditions.checkArgument( + startSnapshotId != null, + "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + Preconditions.checkArgument( + startSnapshotTimestamp == null, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) { + Preconditions.checkArgument( + startSnapshotTimestamp != null, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + Preconditions.checkArgument( + startSnapshotId == null, + "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + Preconditions.checkArgument( + tag == null, + String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + } + + Preconditions.checkArgument( + !(startTag != null && startSnapshotId() != null), + "START_SNAPSHOT_ID and START_TAG cannot both be set."); + + Preconditions.checkArgument( + !(endTag != null && endSnapshotId() != null), + "END_SNAPSHOT_ID and END_TAG cannot both be set."); + + Preconditions.checkArgument( + maxAllowedPlanningFailures >= -1, + "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + public boolean caseSensitive() { + return caseSensitive; + } + + public Long snapshotId() { + return snapshotId; + } + + public String branch() { + return branch; + } + + public String tag() { + return tag; + } + + public String startTag() { + return startTag; + } + + public String endTag() { + return endTag; + } + + public StreamingStartingStrategy streamingStartingStrategy() { + return startingStrategy; + } + + public Long startSnapshotTimestamp() { + return startSnapshotTimestamp; + } + + public Long startSnapshotId() { + return startSnapshotId; + } + + public Long endSnapshotId() { + return endSnapshotId; + } + + public Long asOfTimestamp() { + return asOfTimestamp; + } + + public Long splitSize() { + return splitSize; + } + + public Integer splitLookback() { + return splitLookback; + } + + public Long splitOpenFileCost() { + return splitOpenFileCost; + } + + public boolean isStreaming() { + return isStreaming; + } + + public Duration monitorInterval() { + return monitorInterval; + } + + public String nameMapping() { + return nameMapping; + } + + public Schema project() { + return schema; + } + + public List filters() { + return filters; + } + + public long limit() { + return limit; + } + + public boolean includeColumnStats() { + return includeColumnStats; + } + + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + + public boolean exposeLocality() { + return exposeLocality; + } + + public Integer planParallelism() { + return planParallelism; + } + + public int maxPlanningSnapshotCount() { + return maxPlanningSnapshotCount; + } + + public int maxAllowedPlanningFailures() { + return maxAllowedPlanningFailures; + } + + public String watermarkColumn() { + return watermarkColumn; + } + + public TimeUnit watermarkColumnTimeUnit() { + return watermarkColumnTimeUnit; + } + + public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { + return ScanContext.builder() + .caseSensitive(caseSensitive) + .useSnapshotId(null) + .useBranch(branch) + .useTag(null) + .startSnapshotId(newStartSnapshotId) + .endSnapshotId(newEndSnapshotId) + .startTag(null) + .endTag(null) + .asOfTimestamp(null) + .splitSize(splitSize) + .splitLookback(splitLookback) + .splitOpenFileCost(splitOpenFileCost) + .streaming(isStreaming) + .monitorInterval(monitorInterval) + .nameMapping(nameMapping) + .project(schema) + .filters(filters) + .limit(limit) + .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) + .exposeLocality(exposeLocality) + .planParallelism(planParallelism) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit) + .build(); + } + + public ScanContext copyWithSnapshotId(long newSnapshotId) { + return ScanContext.builder() + .caseSensitive(caseSensitive) + .useSnapshotId(newSnapshotId) + .useBranch(branch) + .useTag(tag) + .startSnapshotId(null) + .endSnapshotId(null) + .startTag(null) + .endTag(null) + .asOfTimestamp(null) + .splitSize(splitSize) + .splitLookback(splitLookback) + .splitOpenFileCost(splitOpenFileCost) + .streaming(isStreaming) + .monitorInterval(monitorInterval) + .nameMapping(nameMapping) + .project(schema) + .filters(filters) + .limit(limit) + .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) + .exposeLocality(exposeLocality) + .planParallelism(planParallelism) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit) + .build(); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); + private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); + private String branch = FlinkReadOptions.BRANCH.defaultValue(); + private String tag = FlinkReadOptions.TAG.defaultValue(); + private String startTag = FlinkReadOptions.START_TAG.defaultValue(); + private String endTag = FlinkReadOptions.END_TAG.defaultValue(); + private StreamingStartingStrategy startingStrategy = + FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue(); + private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue(); + private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue(); + private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue(); + private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue(); + private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue(); + private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue(); + private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue(); + private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue(); + private Duration monitorInterval = + TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()); + private String nameMapping; + private Schema projectedSchema; + private List filters; + private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); + private boolean includeColumnStats = + FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; + private boolean exposeLocality; + private Integer planParallelism = + FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); + private int maxPlanningSnapshotCount = + FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); + private int maxAllowedPlanningFailures = + FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); + private String watermarkColumn = FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue(); + private TimeUnit watermarkColumnTimeUnit = + FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue(); + + private Builder() {} + + public Builder caseSensitive(boolean newCaseSensitive) { + this.caseSensitive = newCaseSensitive; + return this; + } + + public Builder useSnapshotId(Long newSnapshotId) { + this.snapshotId = newSnapshotId; + return this; + } + + public Builder useTag(String newTag) { + this.tag = newTag; + return this; + } + + public Builder useBranch(String newBranch) { + this.branch = newBranch; + return this; + } + + public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { + this.startingStrategy = newStartingStrategy; + return this; + } + + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + this.startSnapshotTimestamp = newStartSnapshotTimestamp; + return this; + } + + public Builder startSnapshotId(Long newStartSnapshotId) { + this.startSnapshotId = newStartSnapshotId; + return this; + } + + public Builder endSnapshotId(Long newEndSnapshotId) { + this.endSnapshotId = newEndSnapshotId; + return this; + } + + public Builder startTag(String newStartTag) { + this.startTag = newStartTag; + return this; + } + + public Builder endTag(String newEndTag) { + this.endTag = newEndTag; + return this; + } + + public Builder asOfTimestamp(Long newAsOfTimestamp) { + this.asOfTimestamp = newAsOfTimestamp; + return this; + } + + public Builder splitSize(Long newSplitSize) { + this.splitSize = newSplitSize; + return this; + } + + public Builder splitLookback(Integer newSplitLookback) { + this.splitLookback = newSplitLookback; + return this; + } + + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { + this.splitOpenFileCost = newSplitOpenFileCost; + return this; + } + + public Builder streaming(boolean streaming) { + this.isStreaming = streaming; + return this; + } + + public Builder monitorInterval(Duration newMonitorInterval) { + this.monitorInterval = newMonitorInterval; + return this; + } + + public Builder nameMapping(String newNameMapping) { + this.nameMapping = newNameMapping; + return this; + } + + public Builder project(Schema newProjectedSchema) { + this.projectedSchema = newProjectedSchema; + return this; + } + + public Builder filters(List newFilters) { + this.filters = newFilters; + return this; + } + + public Builder limit(long newLimit) { + this.limit = newLimit; + return this; + } + + public Builder includeColumnStats(boolean newIncludeColumnStats) { + this.includeColumnStats = newIncludeColumnStats; + return this; + } + + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder planParallelism(Integer parallelism) { + this.planParallelism = parallelism; + return this; + } + + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; + return this; + } + + public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { + this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; + return this; + } + + public Builder watermarkColumn(String newWatermarkColumn) { + this.watermarkColumn = newWatermarkColumn; + return this; + } + + public Builder watermarkColumnTimeUnit(TimeUnit newWatermarkTimeUnit) { + this.watermarkColumnTimeUnit = newWatermarkTimeUnit; + return this; + } + + public Builder resolveConfig( + Table table, Map readOptions, ReadableConfig readableConfig) { + FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); + + return this.useSnapshotId(flinkReadConf.snapshotId()) + .useTag(flinkReadConf.tag()) + .useBranch(flinkReadConf.branch()) + .startTag(flinkReadConf.startTag()) + .endTag(flinkReadConf.endTag()) + .caseSensitive(flinkReadConf.caseSensitive()) + .asOfTimestamp(flinkReadConf.asOfTimestamp()) + .startingStrategy(flinkReadConf.startingStrategy()) + .startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp()) + .startSnapshotId(flinkReadConf.startSnapshotId()) + .endSnapshotId(flinkReadConf.endSnapshotId()) + .splitSize(flinkReadConf.splitSize()) + .splitLookback(flinkReadConf.splitLookback()) + .splitOpenFileCost(flinkReadConf.splitFileOpenCost()) + .streaming(flinkReadConf.streaming()) + .monitorInterval(flinkReadConf.monitorInterval()) + .nameMapping(flinkReadConf.nameMapping()) + .limit(flinkReadConf.limit()) + .planParallelism(flinkReadConf.workerPoolSize()) + .includeColumnStats(flinkReadConf.includeColumnStats()) + .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(flinkReadConf.watermarkColumn()) + .watermarkColumnTimeUnit(flinkReadConf.watermarkColumnTimeUnit()); + } + + public ScanContext build() { + return new ScanContext( + caseSensitive, + snapshotId, + startingStrategy, + startSnapshotTimestamp, + startSnapshotId, + endSnapshotId, + asOfTimestamp, + splitSize, + splitLookback, + splitOpenFileCost, + isStreaming, + monitorInterval, + nameMapping, + projectedSchema, + filters, + limit, + includeColumnStats, + includeStatsForColumns, + exposeLocality, + planParallelism, + maxPlanningSnapshotCount, + maxAllowedPlanningFailures, + watermarkColumn, + watermarkColumnTimeUnit, + branch, + tag, + startTag, + endTag); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java new file mode 100644 index 000000000000..7c3a69dbc141 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.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.flink.source; + +import java.util.function.Supplier; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SourceUtil { + private SourceUtil() {} + + static boolean isLocalityEnabled( + Table table, ReadableConfig readableConfig, Boolean exposeLocality) { + Boolean localityEnabled = + exposeLocality != null + ? exposeLocality + : readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO); + + if (localityEnabled != null && !localityEnabled) { + return false; + } + + return Util.mayHaveBlockLocations(table.io(), table.location()); + } + + /** + * Infer source parallelism. + * + * @param readableConfig Flink config. + * @param splitCountProvider Split count supplier. As the computation may involve expensive split + * discover, lazy evaluation is performed if inferring parallelism is enabled. + * @param limitCount limited output count. + */ + static int inferParallelism( + ReadableConfig readableConfig, long limitCount, Supplier splitCountProvider) { + int parallelism = + readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { + int maxInferParallelism = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); + Preconditions.checkState( + maxInferParallelism >= 1, + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() + + " cannot be less than 1"); + parallelism = Math.min(splitCountProvider.get(), maxInferParallelism); + } + + if (limitCount > 0) { + int limit = limitCount >= Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limitCount; + parallelism = Math.min(parallelism, limit); + } + + // parallelism must be positive. + parallelism = Math.max(1, parallelism); + return parallelism; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java new file mode 100644 index 000000000000..a07613aee59b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java @@ -0,0 +1,269 @@ +/* + * 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.io.UncheckedIOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the single (non-parallel) monitoring task which takes a {@link FlinkInputFormat}, it is + * responsible for: + * + *

    + *
  1. Monitoring snapshots of the Iceberg table. + *
  2. Creating the {@link FlinkInputSplit splits} corresponding to the incremental files + *
  3. Assigning them to downstream tasks for further processing. + *
+ * + *

The splits to be read are forwarded to the downstream {@link StreamingReaderOperator} which + * can have parallelism greater than one. + */ +public class StreamingMonitorFunction extends RichSourceFunction + implements CheckpointedFunction { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingMonitorFunction.class); + + private static final long INIT_LAST_SNAPSHOT_ID = -1L; + + private final TableLoader tableLoader; + private final ScanContext scanContext; + + private volatile boolean isRunning = true; + + // The checkpoint thread is not the same thread that running the function for SourceStreamTask + // now. It's necessary to + // mark this as volatile. + private volatile long lastSnapshotId = INIT_LAST_SNAPSHOT_ID; + + private transient SourceContext sourceContext; + private transient Table table; + private transient ListState lastSnapshotIdState; + private transient ExecutorService workerPool; + + public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext) { + Preconditions.checkArgument( + scanContext.snapshotId() == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + scanContext.asOfTimestamp() == null, + "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + scanContext.endSnapshotId() == null, + "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument( + scanContext.endTag() == null, "Cannot set end-tag option for streaming reader"); + Preconditions.checkArgument( + scanContext.maxPlanningSnapshotCount() > 0, + "The max-planning-snapshot-count must be greater than zero"); + this.tableLoader = tableLoader; + this.scanContext = scanContext; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + final RuntimeContext runtimeContext = getRuntimeContext(); + ValidationException.check( + runtimeContext instanceof StreamingRuntimeContext, + "context should be instance of StreamingRuntimeContext"); + final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // Load iceberg table from table loader. + tableLoader.open(); + table = tableLoader.loadTable(); + + // Initialize the flink state for last snapshot id. + lastSnapshotIdState = + context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("snapshot-id-state", LongSerializer.INSTANCE)); + + // Restore the last-snapshot-id from flink's state if possible. + if (context.isRestored()) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + lastSnapshotId = lastSnapshotIdState.get().iterator().next(); + } else if (scanContext.startTag() != null || scanContext.startSnapshotId() != null) { + Preconditions.checkArgument( + !(scanContext.startTag() != null && scanContext.startSnapshotId() != null), + "START_SNAPSHOT_ID and START_TAG cannot both be set."); + Preconditions.checkNotNull( + table.currentSnapshot(), "Don't have any available snapshot in table."); + + long startSnapshotId; + if (scanContext.startTag() != null) { + Preconditions.checkArgument( + table.snapshot(scanContext.startTag()) != null, + "Cannot find snapshot with tag %s in table.", + scanContext.startTag()); + startSnapshotId = table.snapshot(scanContext.startTag()).snapshotId(); + } else { + startSnapshotId = scanContext.startSnapshotId(); + } + + long currentSnapshotId = table.currentSnapshot().snapshotId(); + Preconditions.checkState( + SnapshotUtil.isAncestorOf(table, currentSnapshotId, startSnapshotId), + "The option start-snapshot-id %s is not an ancestor of the current snapshot.", + startSnapshotId); + + lastSnapshotId = startSnapshotId; + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + lastSnapshotIdState.clear(); + lastSnapshotIdState.add(lastSnapshotId); + } + + @Override + public void run(SourceContext ctx) throws Exception { + this.sourceContext = ctx; + while (isRunning) { + monitorAndForwardSplits(); + Thread.sleep(scanContext.monitorInterval().toMillis()); + } + } + + private long toSnapshotIdInclusive( + long lastConsumedSnapshotId, long currentSnapshotId, int maxPlanningSnapshotCount) { + List snapshotIds = + SnapshotUtil.snapshotIdsBetween(table, lastConsumedSnapshotId, currentSnapshotId); + if (snapshotIds.size() <= maxPlanningSnapshotCount) { + return currentSnapshotId; + } else { + // It uses reverted index since snapshotIdsBetween returns Ids that are ordered by committed + // time descending. + return snapshotIds.get(snapshotIds.size() - maxPlanningSnapshotCount); + } + } + + @VisibleForTesting + void sourceContext(SourceContext ctx) { + this.sourceContext = ctx; + } + + @VisibleForTesting + void monitorAndForwardSplits() { + // Refresh the table to get the latest committed snapshot. + table.refresh(); + + Snapshot snapshot = + scanContext.branch() != null + ? table.snapshot(scanContext.branch()) + : table.currentSnapshot(); + if (snapshot != null && snapshot.snapshotId() != lastSnapshotId) { + long snapshotId = snapshot.snapshotId(); + + ScanContext newScanContext; + if (lastSnapshotId == INIT_LAST_SNAPSHOT_ID) { + newScanContext = scanContext.copyWithSnapshotId(snapshotId); + } else { + snapshotId = + toSnapshotIdInclusive( + lastSnapshotId, snapshotId, scanContext.maxPlanningSnapshotCount()); + newScanContext = scanContext.copyWithAppendsBetween(lastSnapshotId, snapshotId); + } + + LOG.debug( + "Start discovering splits from {} (exclusive) to {} (inclusive)", + lastSnapshotId, + snapshotId); + long start = System.currentTimeMillis(); + FlinkInputSplit[] splits = + FlinkSplitPlanner.planInputSplits(table, newScanContext, workerPool); + LOG.debug( + "Discovered {} splits, time elapsed {}ms", + splits.length, + System.currentTimeMillis() - start); + + // only need to hold the checkpoint lock when emitting the splits and updating lastSnapshotId + start = System.currentTimeMillis(); + synchronized (sourceContext.getCheckpointLock()) { + for (FlinkInputSplit split : splits) { + sourceContext.collect(split); + } + + lastSnapshotId = snapshotId; + } + LOG.debug( + "Forwarded {} splits, time elapsed {}ms", + splits.length, + System.currentTimeMillis() - start); + } + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (sourceContext != null) { + synchronized (sourceContext.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + + // Release all the resources here. + if (tableLoader != null) { + try { + tableLoader.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + @Override + public void close() { + cancel(); + + if (workerPool != null) { + workerPool.shutdown(); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java new file mode 100644 index 000000000000..ee6f7b63988d --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java @@ -0,0 +1,246 @@ +/* + * 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.util.Queue; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.JavaSerializer; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The operator that reads the {@link FlinkInputSplit splits} received from the preceding {@link + * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} which has a + * parallelism of 1, this operator can have multiple parallelism. + * + *

As soon as a split descriptor is received, it is put in a queue, and use {@link + * MailboxExecutor} read the actual data of the split. This architecture allows the separation of + * the reading thread from the one split processing the checkpoint barriers, thus removing any + * potential back-pressure. + */ +public class StreamingReaderOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingReaderOperator.class); + + // It's the same thread that is running this operator and checkpoint actions. we use this executor + // to schedule only + // one split for future reading, so that a new checkpoint could be triggered without blocking long + // time for exhausting + // all scheduled splits. + private final MailboxExecutor executor; + private FlinkInputFormat format; + + private transient SourceFunction.SourceContext sourceContext; + + private transient ListState inputSplitsState; + private transient Queue splits; + + // Splits are read by the same thread that calls processElement. Each read task is submitted to + // that thread by adding + // them to the executor. This state is used to ensure that only one read task is in that queue at + // a time, so that read + // tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this + // is set to RUNNING. + // When there are no more files to read, this will be set to IDLE. + private transient SplitState currentSplitState; + + private StreamingReaderOperator( + FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { + this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); + this.processingTimeService = timeService; + this.executor = + Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + // TODO Replace Java serialization with Avro approach to keep state compatibility. + // See issue: https://github.com/apache/iceberg/issues/1698 + inputSplitsState = + context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("splits", new JavaSerializer<>())); + + // Initialize the current split state to IDLE. + currentSplitState = SplitState.IDLE; + + // Recover splits state from flink state backend if possible. + splits = Lists.newLinkedList(); + if (context.isRestored()) { + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); + + for (FlinkInputSplit split : inputSplitsState.get()) { + splits.add(split); + } + } + + this.sourceContext = + StreamSourceContexts.getSourceContext( + getOperatorConfig().getTimeCharacteristic(), + getProcessingTimeService(), + new Object(), // no actual locking needed + output, + getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), + -1, + true); + + // Enqueue to process the recovered input splits. + enqueueProcessSplits(); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + + inputSplitsState.clear(); + inputSplitsState.addAll(Lists.newArrayList(splits)); + } + + @Override + public void processElement(StreamRecord element) { + splits.add(element.getValue()); + enqueueProcessSplits(); + } + + private void enqueueProcessSplits() { + if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) { + currentSplitState = SplitState.RUNNING; + executor.execute(this::processSplits, this.getClass().getSimpleName()); + } + } + + private void processSplits() throws IOException { + FlinkInputSplit split = splits.poll(); + if (split == null) { + currentSplitState = SplitState.IDLE; + return; + } + + format.open(split); + try { + RowData nextElement = null; + while (!format.reachedEnd()) { + nextElement = format.nextRecord(nextElement); + sourceContext.collect(nextElement); + } + } finally { + currentSplitState = SplitState.IDLE; + format.close(); + } + + // Re-schedule to process the next split. + enqueueProcessSplits(); + } + + @Override + public void processWatermark(Watermark mark) { + // we do nothing because we emit our own watermarks if needed. + } + + @Override + public void close() throws Exception { + super.close(); + + if (format != null) { + format.close(); + format.closeInputFormat(); + format = null; + } + + sourceContext = null; + } + + @Override + public void finish() throws Exception { + super.finish(); + output.close(); + if (sourceContext != null) { + sourceContext.emitWatermark(Watermark.MAX_WATERMARK); + sourceContext.close(); + sourceContext = null; + } + } + + static OneInputStreamOperatorFactory factory(FlinkInputFormat format) { + return new OperatorFactory(format); + } + + private enum SplitState { + IDLE, + RUNNING + } + + private static class OperatorFactory extends AbstractStreamOperatorFactory + implements YieldingOperatorFactory, + OneInputStreamOperatorFactory { + + private final FlinkInputFormat format; + + private transient MailboxExecutor mailboxExecutor; + + private OperatorFactory(FlinkInputFormat format) { + this.format = format; + } + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + @SuppressWarnings("unchecked") + @Override + public > O createStreamOperator( + StreamOperatorParameters parameters) { + StreamingReaderOperator operator = + new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); + operator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + return (O) operator; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return StreamingReaderOperator.class; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java new file mode 100644 index 000000000000..11707bf82a0f --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java @@ -0,0 +1,54 @@ +/* + * 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; + +/** Starting strategy for streaming execution. */ +public enum StreamingStartingStrategy { + /** + * Do a regular table scan then switch to the incremental mode. + * + *

The incremental mode starts from the current snapshot exclusive. + */ + TABLE_SCAN_THEN_INCREMENTAL, + + /** + * Start incremental mode from the latest snapshot inclusive. + * + *

If it is an empty map, all future append snapshots should be discovered. + */ + INCREMENTAL_FROM_LATEST_SNAPSHOT, + + /** + * Start incremental mode from the earliest snapshot inclusive. + * + *

If it is an empty map, all future append snapshots should be discovered. + */ + INCREMENTAL_FROM_EARLIEST_SNAPSHOT, + + /** Start incremental mode from a snapshot with a specific id inclusive. */ + INCREMENTAL_FROM_SNAPSHOT_ID, + + /** + * Start incremental mode from a snapshot with a specific timestamp inclusive. + * + *

If the timestamp is between two snapshots, it should start from the snapshot after the + * timestamp. + */ + INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java new file mode 100644 index 000000000000..e7447d08c985 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java @@ -0,0 +1,119 @@ +/* + * 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.assigner; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Since all methods are called in the source coordinator thread by enumerator, there is no need for + * locking. + */ +@Internal +public class DefaultSplitAssigner implements SplitAssigner { + + private final Queue pendingSplits; + private CompletableFuture availableFuture; + + public DefaultSplitAssigner(SerializableComparator comparator) { + this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); + } + + public DefaultSplitAssigner( + SerializableComparator comparator, + Collection assignerState) { + this(comparator); + // Because default assigner only tracks unassigned splits, + // there is no need to filter splits based on status (unassigned) here. + assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); + } + + @Override + public synchronized GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } else { + IcebergSourceSplit split = pendingSplits.poll(); + return GetSplitResult.forSplit(split); + } + } + + @Override + public void onDiscoveredSplits(Collection splits) { + addSplits(splits); + } + + @Override + public void onUnassignedSplits(Collection splits) { + addSplits(splits); + } + + private synchronized void addSplits(Collection splits) { + if (!splits.isEmpty()) { + pendingSplits.addAll(splits); + // only complete pending future if new splits are discovered + completeAvailableFuturesIfNeeded(); + } + } + + /** Simple assigner only tracks unassigned splits */ + @Override + public synchronized Collection state() { + return pendingSplits.stream() + .map(split -> new IcebergSourceSplitState(split, IcebergSourceSplitStatus.UNASSIGNED)) + .collect(Collectors.toList()); + } + + @Override + public synchronized CompletableFuture isAvailable() { + if (availableFuture == null) { + availableFuture = new CompletableFuture<>(); + } + return availableFuture; + } + + @Override + public synchronized int pendingSplitCount() { + return pendingSplits.size(); + } + + @Override + public long pendingRecords() { + return pendingSplits.stream() + .map(split -> split.task().estimatedRowsCount()) + .reduce(0L, Long::sum); + } + + private synchronized void completeAvailableFuturesIfNeeded() { + if (availableFuture != null && !pendingSplits.isEmpty()) { + availableFuture.complete(null); + } + availableFuture = null; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java new file mode 100644 index 000000000000..72deaeb890f3 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.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.flink.source.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +public class GetSplitResult { + + public enum Status { + AVAILABLE, + + /** + * There are pending splits. But they can't be assigned due to constraints (like event time + * alignment) + */ + CONSTRAINED, + + /** Assigner doesn't have pending splits. */ + UNAVAILABLE + } + + private final Status status; + private final IcebergSourceSplit split; + + private GetSplitResult(Status status) { + this.status = status; + this.split = null; + } + + private GetSplitResult(IcebergSourceSplit split) { + Preconditions.checkNotNull(split, "Split cannot be null"); + this.status = Status.AVAILABLE; + this.split = split; + } + + public Status status() { + return status; + } + + public IcebergSourceSplit split() { + return split; + } + + private static final GetSplitResult UNAVAILABLE = new GetSplitResult(Status.UNAVAILABLE); + private static final GetSplitResult CONSTRAINED = new GetSplitResult(Status.CONSTRAINED); + + public static GetSplitResult unavailable() { + return UNAVAILABLE; + } + + public static GetSplitResult constrained() { + return CONSTRAINED; + } + + public static GetSplitResult forSplit(IcebergSourceSplit split) { + return new GetSplitResult(split); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java new file mode 100644 index 000000000000..e58478897aef --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.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.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Create default assigner with a comparator that hands out splits where the order of the splits + * will be defined by the {@link SerializableComparator}. + */ +public class OrderedSplitAssignerFactory implements SplitAssignerFactory { + private final SerializableComparator comparator; + + public OrderedSplitAssignerFactory(SerializableComparator comparator) { + this.comparator = comparator; + } + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(comparator); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(comparator, assignerState); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java new file mode 100644 index 000000000000..a2e2ff364d46 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.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.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** Create simple assigner that hands out splits without any guarantee in order or locality. */ +public class SimpleSplitAssignerFactory implements SplitAssignerFactory { + public SimpleSplitAssignerFactory() {} + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(null); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(null, assignerState); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java new file mode 100644 index 000000000000..dae7c8cca70c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.io.Closeable; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** + * SplitAssigner interface is extracted out as a separate component so that we can plug in different + * split assignment strategy for different requirements. E.g. + * + *

    + *
  • Simple assigner with no ordering guarantee or locality aware optimization. + *
  • Locality aware assigner that prefer splits that are local. + *
  • Snapshot aware assigner that assign splits based on the order they are committed. + *
  • Event time alignment assigner that assign splits satisfying certain time ordering within a + * single source or across sources. + *
+ * + *

Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from + * the coordinator thread. But enumerator may call the {@link SplitAssigner#pendingSplitCount()} + * from the I/O threads. + */ +public interface SplitAssigner extends Closeable { + + /** + * Some assigners may need to start background threads or perform other activity such as + * registering as listeners to updates from other event sources e.g., watermark tracker. + */ + default void start() {} + + /** + * Some assigners may need to perform certain actions when their corresponding enumerators are + * closed + */ + @Override + default void close() {} + + /** + * Request a new split from the assigner when enumerator trying to assign splits to awaiting + * readers. + * + *

If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should + * call {@link SplitAssigner#onUnassignedSplits} to return the split. + */ + GetSplitResult getNext(@Nullable String hostname); + + /** Add new splits discovered by enumerator */ + void onDiscoveredSplits(Collection splits); + + /** Forward addSplitsBack event (for failed reader) to assigner */ + void onUnassignedSplits(Collection splits); + + /** + * Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon + * completed splits + */ + default void onCompletedSplits(Collection completedSplitIds) {} + + /** + * Get assigner state for checkpointing. This is a super-set API that works for all currently + * imagined assigners. + */ + Collection state(); + + /** + * Enumerator can get a notification via CompletableFuture when the assigner has more splits + * available later. Enumerator should schedule assignment in the thenAccept action of the future. + * + *

Assigner will return the same future if this method is called again before the previous + * future is completed. + * + *

The future can be completed from other thread, e.g. the coordinator thread from another + * thread for event time alignment. + * + *

If enumerator need to trigger action upon the future completion, it may want to run it in + * the coordinator thread using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}. + */ + CompletableFuture isAvailable(); + + /** + * Return the number of pending splits that haven't been assigned yet. + * + *

The enumerator can poll this API to publish a metric on the number of pending splits. + * + *

The enumerator can also use this information to throttle split discovery for streaming read. + * If there are already many pending splits tracked by the assigner, it is undesirable to discover + * more splits and track them in the assigner. That will increase the memory footprint and + * enumerator checkpoint size. + * + *

Throttling works better together with {@link ScanContext#maxPlanningSnapshotCount()}. + * Otherwise, the next split discovery after throttling will just discover all non-enumerated + * snapshots and splits, which defeats the purpose of throttling. + */ + int pendingSplitCount(); + + /** + * Return the number of pending records, which can act as a measure of the source lag. This value + * could be an estimation if the exact number of records cannot be accurately computed. + */ + long pendingRecords(); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java new file mode 100644 index 000000000000..6e02a556ffcd --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java @@ -0,0 +1,30 @@ +/* + * 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.assigner; + +import java.io.Serializable; +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +public interface SplitAssignerFactory extends Serializable { + + SplitAssigner createAssigner(); + + SplitAssigner createAssigner(Collection assignerState); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java new file mode 100644 index 000000000000..03ba67a554f9 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.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.flink.source.assigner; + +import org.apache.flink.annotation.Internal; + +@Internal +public enum SplitAssignerType { + SIMPLE { + @Override + public SplitAssignerFactory factory() { + return new SimpleSplitAssignerFactory(); + } + }; + + public abstract SplitAssignerFactory factory(); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java new file mode 100644 index 000000000000..6c9a855bc149 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -0,0 +1,175 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.assigner.GetSplitResult; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class AbstractIcebergEnumerator + implements SplitEnumerator { + private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); + + private final SplitEnumeratorContext enumeratorContext; + private final SplitAssigner assigner; + private final Map readersAwaitingSplit; + private final AtomicReference> availableFuture; + + AbstractIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { + this.enumeratorContext = enumeratorContext; + this.assigner = assigner; + this.readersAwaitingSplit = new LinkedHashMap<>(); + this.availableFuture = new AtomicReference<>(); + this.enumeratorContext + .metricGroup() + // This number may not capture the entire backlog due to split discovery throttling to avoid + // excessive memory footprint. Some pending splits may not have been discovered yet. + .setUnassignedSplitsGauge(() -> Long.valueOf(assigner.pendingSplitCount())); + this.enumeratorContext.metricGroup().gauge("pendingRecords", assigner::pendingRecords); + } + + @Override + public void start() { + assigner.start(); + } + + @Override + public void close() throws IOException { + assigner.close(); + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // Iceberg source uses custom split request event to piggyback finished split ids. + throw new UnsupportedOperationException( + String.format( + "Received invalid default split request event " + + "from subtask %d as Iceberg source uses custom split request event", + subtaskId)); + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + if (sourceEvent instanceof SplitRequestEvent) { + SplitRequestEvent splitRequestEvent = (SplitRequestEvent) sourceEvent; + LOG.info("Received request split event from subtask {}", subtaskId); + assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds()); + readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname()); + assignSplits(); + } else { + throw new IllegalArgumentException( + String.format( + "Received unknown event from subtask %d: %s", + subtaskId, sourceEvent.getClass().getCanonicalName())); + } + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); + assigner.onUnassignedSplits(splits); + assignSplits(); + } + + @Override + public void addReader(int subtaskId) { + LOG.info("Added reader: {}", subtaskId); + } + + private void assignSplits() { + LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size()); + Iterator> awaitingReader = + readersAwaitingSplit.entrySet().iterator(); + while (awaitingReader.hasNext()) { + Map.Entry nextAwaiting = awaitingReader.next(); + // if the reader that requested another split has failed in the meantime, remove + // it from the list of waiting readers + if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) { + awaitingReader.remove(); + continue; + } + + int awaitingSubtask = nextAwaiting.getKey(); + String hostname = nextAwaiting.getValue(); + GetSplitResult getResult = assigner.getNext(hostname); + if (getResult.status() == GetSplitResult.Status.AVAILABLE) { + LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split()); + enumeratorContext.assignSplit(getResult.split(), awaitingSubtask); + awaitingReader.remove(); + } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) { + getAvailableFutureIfNeeded(); + break; + } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) { + if (shouldWaitForMoreSplits()) { + getAvailableFutureIfNeeded(); + break; + } else { + LOG.info("No more splits available for subtask {}", awaitingSubtask); + enumeratorContext.signalNoMoreSplits(awaitingSubtask); + awaitingReader.remove(); + } + } else { + throw new IllegalArgumentException("Unsupported status: " + getResult.status()); + } + } + } + + /** return true if enumerator should wait for splits like in the continuous enumerator case */ + protected abstract boolean shouldWaitForMoreSplits(); + + private synchronized void getAvailableFutureIfNeeded() { + if (availableFuture.get() != null) { + return; + } + + CompletableFuture future = + assigner + .isAvailable() + .thenAccept( + ignore -> + // Must run assignSplits in coordinator thread + // because the future may be completed from other threads. + // E.g., in event time alignment assigner, + // watermark advancement from another source may + // cause the available future to be completed + enumeratorContext.runInCoordinatorThread( + () -> { + LOG.debug("Executing callback of assignSplits"); + availableFuture.set(null); + assignSplits(); + })); + availableFuture.set(future); + LOG.debug("Registered callback for future available splits"); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java new file mode 100644 index 000000000000..41863ffee60b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.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.flink.source.enumerator; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class ContinuousEnumerationResult { + private final Collection splits; + private final IcebergEnumeratorPosition fromPosition; + private final IcebergEnumeratorPosition toPosition; + + /** + * @param splits should never be null. But it can be an empty collection + * @param fromPosition can be null + * @param toPosition should never be null. But it can have null snapshotId and snapshotTimestampMs + */ + ContinuousEnumerationResult( + Collection splits, + IcebergEnumeratorPosition fromPosition, + IcebergEnumeratorPosition toPosition) { + Preconditions.checkArgument(splits != null, "Invalid to splits collection: null"); + Preconditions.checkArgument(toPosition != null, "Invalid end position: null"); + this.splits = splits; + this.fromPosition = fromPosition; + this.toPosition = toPosition; + } + + public Collection splits() { + return splits; + } + + public IcebergEnumeratorPosition fromPosition() { + return fromPosition; + } + + public IcebergEnumeratorPosition toPosition() { + return toPosition; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java new file mode 100644 index 000000000000..c7021b9c6847 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.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.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collections; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.util.ElapsedTimeGauge; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { + + private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** + * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to + * control the total number of snapshots worth of splits tracked by assigner. + */ + private static final int ENUMERATION_SPLIT_COUNT_HISTORY_SIZE = 3; + + private final SplitEnumeratorContext enumeratorContext; + private final SplitAssigner assigner; + private final ScanContext scanContext; + private final ContinuousSplitPlanner splitPlanner; + + /** + * snapshotId for the last enumerated snapshot. next incremental enumeration should be based off + * this as the starting position. + */ + private final AtomicReference enumeratorPosition; + + /** Track enumeration result history for split discovery throttling. */ + private final EnumerationHistory enumerationHistory; + + /** Count the consecutive failures and throw exception if the max allowed failres are reached */ + private transient int consecutiveFailures = 0; + + private final ElapsedTimeGauge elapsedSecondsSinceLastSplitDiscovery; + + public ContinuousIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, + SplitAssigner assigner, + ScanContext scanContext, + ContinuousSplitPlanner splitPlanner, + @Nullable IcebergEnumeratorState enumState) { + super(enumeratorContext, assigner); + + this.enumeratorContext = enumeratorContext; + this.assigner = assigner; + this.scanContext = scanContext; + this.splitPlanner = splitPlanner; + this.enumeratorPosition = new AtomicReference<>(); + this.enumerationHistory = new EnumerationHistory(ENUMERATION_SPLIT_COUNT_HISTORY_SIZE); + this.elapsedSecondsSinceLastSplitDiscovery = new ElapsedTimeGauge(TimeUnit.SECONDS); + this.enumeratorContext + .metricGroup() + .gauge("elapsedSecondsSinceLastSplitDiscovery", elapsedSecondsSinceLastSplitDiscovery); + + if (enumState != null) { + this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); + this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); + } + } + + @Override + public void start() { + super.start(); + enumeratorContext.callAsync( + this::discoverSplits, + this::processDiscoveredSplits, + 0L, + scanContext.monitorInterval().toMillis()); + } + + @Override + public void close() throws IOException { + splitPlanner.close(); + super.close(); + } + + @Override + protected boolean shouldWaitForMoreSplits() { + return true; + } + + @Override + public IcebergEnumeratorState snapshotState(long checkpointId) { + return new IcebergEnumeratorState( + enumeratorPosition.get(), assigner.state(), enumerationHistory.snapshot()); + } + + /** This method is executed in an IO thread pool. */ + private ContinuousEnumerationResult discoverSplits() { + int pendingSplitCountFromAssigner = assigner.pendingSplitCount(); + if (enumerationHistory.shouldPauseSplitDiscovery(pendingSplitCountFromAssigner)) { + // If the assigner already has many pending splits, it is better to pause split discovery. + // Otherwise, eagerly discovering more splits will just increase assigner memory footprint + // and enumerator checkpoint state size. + LOG.info( + "Pause split discovery as the assigner already has too many pending splits: {}", + pendingSplitCountFromAssigner); + return new ContinuousEnumerationResult( + Collections.emptyList(), enumeratorPosition.get(), enumeratorPosition.get()); + } else { + return splitPlanner.planSplits(enumeratorPosition.get()); + } + } + + /** This method is executed in a single coordinator thread. */ + private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwable error) { + if (error == null) { + consecutiveFailures = 0; + if (!Objects.equals(result.fromPosition(), enumeratorPosition.get())) { + // Multiple discoverSplits() may be triggered with the same starting snapshot to the I/O + // thread pool. E.g., the splitDiscoveryInterval is very short (like 10 ms in some unit + // tests) or the thread pool is busy and multiple discovery actions are executed + // concurrently. Discovery result should only be accepted if the starting position + // matches the enumerator position (like compare-and-swap). + LOG.info( + "Skip {} discovered splits because the scan starting position doesn't match " + + "the current enumerator position: enumerator position = {}, scan starting position = {}", + result.splits().size(), + enumeratorPosition.get(), + result.fromPosition()); + } else { + elapsedSecondsSinceLastSplitDiscovery.refreshLastRecordedTime(); + // Sometimes, enumeration may yield no splits for a few reasons. + // - upstream paused or delayed streaming writes to the Iceberg table. + // - enumeration frequency is higher than the upstream write frequency. + if (!result.splits().isEmpty()) { + assigner.onDiscoveredSplits(result.splits()); + // EnumerationHistory makes throttling decision on split discovery + // based on the total number of splits discovered in the last a few cycles. + // Only update enumeration history when there are some discovered splits. + enumerationHistory.add(result.splits().size()); + LOG.info( + "Added {} splits discovered between ({}, {}] to the assigner", + result.splits().size(), + result.fromPosition(), + result.toPosition()); + } else { + LOG.info( + "No new splits discovered between ({}, {}]", + result.fromPosition(), + result.toPosition()); + } + // update the enumerator position even if there is no split discovered + // or the toPosition is empty (e.g. for empty table). + enumeratorPosition.set(result.toPosition()); + LOG.info("Update enumerator position to {}", result.toPosition()); + } + } else { + consecutiveFailures++; + if (scanContext.maxAllowedPlanningFailures() < 0 + || consecutiveFailures <= scanContext.maxAllowedPlanningFailures()) { + LOG.error("Failed to discover new splits", error); + } else { + throw new RuntimeException("Failed to discover new splits", error); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java new file mode 100644 index 000000000000..2a1325178873 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java @@ -0,0 +1,30 @@ +/* + * 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.enumerator; + +import java.io.Closeable; +import org.apache.flink.annotation.Internal; + +/** This interface is introduced so that we can plug in different split planner for unit test */ +@Internal +public interface ContinuousSplitPlanner extends Closeable { + + /** Discover the files appended between {@code lastPosition} and current table snapshot */ + ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java new file mode 100644 index 000000000000..e9e3c159b07b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.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.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.FlinkSplitPlanner; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { + private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); + + private final Table table; + private final ScanContext scanContext; + private final boolean isSharedPool; + private final ExecutorService workerPool; + private final TableLoader tableLoader; + + /** + * @param tableLoader A cloned tableLoader. + * @param threadName thread name prefix for worker pool to run the split planning. If null, a + * shared worker pool will be used. + */ + public ContinuousSplitPlannerImpl( + TableLoader tableLoader, ScanContext scanContext, String threadName) { + this.tableLoader = tableLoader.clone(); + this.tableLoader.open(); + this.table = this.tableLoader.loadTable(); + this.scanContext = scanContext; + this.isSharedPool = threadName == null; + this.workerPool = + isSharedPool + ? ThreadPools.getWorkerPool() + : ThreadPools.newWorkerPool( + "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); + } + + @Override + public void close() throws IOException { + if (!isSharedPool) { + workerPool.shutdown(); + } + tableLoader.close(); + } + + @Override + public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { + table.refresh(); + if (lastPosition != null) { + return discoverIncrementalSplits(lastPosition); + } else { + return discoverInitialSplits(); + } + } + + private Snapshot toSnapshotInclusive( + Long lastConsumedSnapshotId, Snapshot currentSnapshot, int maxPlanningSnapshotCount) { + // snapshots are in reverse order (latest snapshot first) + List snapshots = + Lists.newArrayList( + SnapshotUtil.ancestorsBetween( + table, currentSnapshot.snapshotId(), lastConsumedSnapshotId)); + if (snapshots.size() <= maxPlanningSnapshotCount) { + return currentSnapshot; + } else { + // Because snapshots are in reverse order of commit history, this index returns + // the max allowed number of snapshots from the lastConsumedSnapshotId. + return snapshots.get(snapshots.size() - maxPlanningSnapshotCount); + } + } + + private ContinuousEnumerationResult discoverIncrementalSplits( + IcebergEnumeratorPosition lastPosition) { + Snapshot currentSnapshot = + scanContext.branch() != null + ? table.snapshot(scanContext.branch()) + : table.currentSnapshot(); + + if (currentSnapshot == null) { + // empty table + Preconditions.checkArgument( + lastPosition.snapshotId() == null, + "Invalid last enumerated position for an empty table: not null"); + LOG.info("Skip incremental scan because table is empty"); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else if (lastPosition.snapshotId() != null + && currentSnapshot.snapshotId() == lastPosition.snapshotId()) { + LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else { + Long lastConsumedSnapshotId = lastPosition != null ? lastPosition.snapshotId() : null; + Snapshot toSnapshotInclusive = + toSnapshotInclusive( + lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); + IcebergEnumeratorPosition newPosition = + IcebergEnumeratorPosition.of( + toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis()); + ScanContext incrementalScan = + scanContext.copyWithAppendsBetween( + lastPosition.snapshotId(), toSnapshotInclusive.snapshotId()); + List splits = + FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); + LOG.info( + "Discovered {} splits from incremental scan: " + + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", + splits.size(), + lastPosition, + newPosition); + return new ContinuousEnumerationResult(splits, lastPosition, newPosition); + } + } + + /** + * Discovery initial set of splits based on {@link StreamingStartingStrategy}. + *

  • {@link ContinuousEnumerationResult#splits()} should contain initial splits discovered from + * table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. For all other + * strategies, splits collection should be empty. + *
  • {@link ContinuousEnumerationResult#toPosition()} points to the starting position for the + * next incremental split discovery with exclusive behavior. Meaning files committed by the + * snapshot from the position in {@code ContinuousEnumerationResult} won't be included in the + * next incremental scan. + */ + private ContinuousEnumerationResult discoverInitialSplits() { + Optional startSnapshotOptional = startSnapshot(table, scanContext); + if (!startSnapshotOptional.isPresent()) { + return new ContinuousEnumerationResult( + Collections.emptyList(), null, IcebergEnumeratorPosition.empty()); + } + + Snapshot startSnapshot = startSnapshotOptional.get(); + LOG.info( + "Get starting snapshot id {} based on strategy {}", + startSnapshot.snapshotId(), + scanContext.streamingStartingStrategy()); + List splits; + IcebergEnumeratorPosition toPosition; + if (scanContext.streamingStartingStrategy() + == StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) { + // do a batch table scan first + splits = + FlinkSplitPlanner.planIcebergSourceSplits( + table, scanContext.copyWithSnapshotId(startSnapshot.snapshotId()), workerPool); + LOG.info( + "Discovered {} splits from initial batch table scan with snapshot Id {}", + splits.size(), + startSnapshot.snapshotId()); + // For TABLE_SCAN_THEN_INCREMENTAL, incremental mode starts exclusive from the startSnapshot + toPosition = + IcebergEnumeratorPosition.of(startSnapshot.snapshotId(), startSnapshot.timestampMillis()); + } else { + // For all other modes, starting snapshot should be consumed inclusively. + // Use parentId to achieve the inclusive behavior. It is fine if parentId is null. + splits = Collections.emptyList(); + Long parentSnapshotId = startSnapshot.parentId(); + if (parentSnapshotId != null) { + Snapshot parentSnapshot = table.snapshot(parentSnapshotId); + Long parentSnapshotTimestampMs = + parentSnapshot != null ? parentSnapshot.timestampMillis() : null; + toPosition = IcebergEnumeratorPosition.of(parentSnapshotId, parentSnapshotTimestampMs); + } else { + toPosition = IcebergEnumeratorPosition.empty(); + } + + LOG.info( + "Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", + startSnapshot.snapshotId(), + startSnapshot.timestampMillis()); + } + + return new ContinuousEnumerationResult(splits, null, toPosition); + } + + /** + * Calculate the starting snapshot based on the {@link StreamingStartingStrategy} defined in + * {@code ScanContext}. + * + *

    If the {@link StreamingStartingStrategy} is not {@link + * StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}, the start snapshot should be consumed + * inclusively. + */ + @VisibleForTesting + static Optional startSnapshot(Table table, ScanContext scanContext) { + switch (scanContext.streamingStartingStrategy()) { + case TABLE_SCAN_THEN_INCREMENTAL: + case INCREMENTAL_FROM_LATEST_SNAPSHOT: + return Optional.ofNullable(table.currentSnapshot()); + case INCREMENTAL_FROM_EARLIEST_SNAPSHOT: + return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); + case INCREMENTAL_FROM_SNAPSHOT_ID: + Snapshot matchedSnapshotById = table.snapshot(scanContext.startSnapshotId()); + Preconditions.checkArgument( + matchedSnapshotById != null, + "Start snapshot id not found in history: " + scanContext.startSnapshotId()); + return Optional.of(matchedSnapshotById); + case INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: + Snapshot matchedSnapshotByTimestamp = + SnapshotUtil.oldestAncestorAfter(table, scanContext.startSnapshotTimestamp()); + Preconditions.checkArgument( + matchedSnapshotByTimestamp != null, + "Cannot find a snapshot after: " + scanContext.startSnapshotTimestamp()); + return Optional.of(matchedSnapshotByTimestamp); + default: + throw new IllegalArgumentException( + "Unknown starting strategy: " + scanContext.streamingStartingStrategy()); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java new file mode 100644 index 000000000000..ef21dad0199d --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java @@ -0,0 +1,96 @@ +/* + * 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.enumerator; + +import java.util.Arrays; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.flink.annotation.VisibleForTesting; + +/** + * This enumeration history is used for split discovery throttling. It tracks the discovered split + * count per every non-empty enumeration. + */ +@ThreadSafe +class EnumerationHistory { + + private final int[] history; + // int (2B) should be enough without overflow for enumeration history + private int count; + + EnumerationHistory(int maxHistorySize) { + this.history = new int[maxHistorySize]; + } + + synchronized void restore(int[] restoredHistory) { + int startingOffset = 0; + int restoreSize = restoredHistory.length; + + if (restoredHistory.length > history.length) { + // keep the newest history + startingOffset = restoredHistory.length - history.length; + // only restore the latest history up to maxHistorySize + restoreSize = history.length; + } + + System.arraycopy(restoredHistory, startingOffset, history, 0, restoreSize); + count = restoreSize; + } + + synchronized int[] snapshot() { + int len = history.length; + if (count > len) { + int[] copy = new int[len]; + // this is like a circular buffer + int indexForOldest = count % len; + System.arraycopy(history, indexForOldest, copy, 0, len - indexForOldest); + System.arraycopy(history, 0, copy, len - indexForOldest, indexForOldest); + return copy; + } else { + return Arrays.copyOfRange(history, 0, count); + } + } + + /** Add the split count from the last enumeration result. */ + synchronized void add(int splitCount) { + int pos = count % history.length; + history[pos] = splitCount; + count += 1; + } + + @VisibleForTesting + synchronized boolean hasFullHistory() { + return count >= history.length; + } + + /** @return true if split discovery should pause because assigner has too many splits already. */ + synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { + if (count < history.length) { + // only check throttling when full history is obtained. + return false; + } else { + // if ScanContext#maxPlanningSnapshotCount() is 10, each split enumeration can + // discovery splits up to 10 snapshots. if maxHistorySize is 3, the max number of + // splits tracked in assigner shouldn't be more than 10 * (3 + 1) snapshots + // worth of splits. +1 because there could be another enumeration when the + // pending splits fall just below the 10 * 3. + int totalSplitCountFromRecentDiscovery = Arrays.stream(history).reduce(0, Integer::sum); + return pendingSplitCountFromAssigner >= totalSplitCountFromRecentDiscovery; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java new file mode 100644 index 000000000000..96aba296f8cf --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java @@ -0,0 +1,79 @@ +/* + * 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.enumerator; + +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +class IcebergEnumeratorPosition { + private final Long snapshotId; + // Track snapshot timestamp mainly for info logging + private final Long snapshotTimestampMs; + + static IcebergEnumeratorPosition empty() { + return new IcebergEnumeratorPosition(null, null); + } + + static IcebergEnumeratorPosition of(long snapshotId, Long snapshotTimestampMs) { + return new IcebergEnumeratorPosition(snapshotId, snapshotTimestampMs); + } + + private IcebergEnumeratorPosition(Long snapshotId, Long snapshotTimestampMs) { + this.snapshotId = snapshotId; + this.snapshotTimestampMs = snapshotTimestampMs; + } + + boolean isEmpty() { + return snapshotId == null; + } + + Long snapshotId() { + return snapshotId; + } + + Long snapshotTimestampMs() { + return snapshotTimestampMs; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("snapshotId", snapshotId) + .add("snapshotTimestampMs", snapshotTimestampMs) + .toString(); + } + + @Override + public int hashCode() { + return Objects.hashCode(snapshotId, snapshotTimestampMs); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IcebergEnumeratorPosition other = (IcebergEnumeratorPosition) o; + return Objects.equal(snapshotId, other.snapshotId()) + && Objects.equal(snapshotTimestampMs, other.snapshotTimestampMs()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java new file mode 100644 index 000000000000..1c63807361c5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java @@ -0,0 +1,90 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class IcebergEnumeratorPositionSerializer + implements SimpleVersionedSerializer { + + public static final IcebergEnumeratorPositionSerializer INSTANCE = + new IcebergEnumeratorPositionSerializer(); + + private static final int VERSION = 1; + + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(128)); + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergEnumeratorPosition position) throws IOException { + return serializeV1(position); + } + + @Override + public IcebergEnumeratorPosition deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IOException("Unknown version: " + version); + } + } + + private byte[] serializeV1(IcebergEnumeratorPosition position) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + out.writeBoolean(position.snapshotId() != null); + if (position.snapshotId() != null) { + out.writeLong(position.snapshotId()); + } + out.writeBoolean(position.snapshotTimestampMs() != null); + if (position.snapshotTimestampMs() != null) { + out.writeLong(position.snapshotTimestampMs()); + } + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + private IcebergEnumeratorPosition deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + Long snapshotId = null; + if (in.readBoolean()) { + snapshotId = in.readLong(); + } + + Long snapshotTimestampMs = null; + if (in.readBoolean()) { + snapshotTimestampMs = in.readLong(); + } + + if (snapshotId != null) { + return IcebergEnumeratorPosition.of(snapshotId, snapshotTimestampMs); + } else { + return IcebergEnumeratorPosition.empty(); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java new file mode 100644 index 000000000000..024d0b101165 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.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.flink.source.enumerator; + +import java.io.Serializable; +import java.util.Collection; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** Enumerator state for checkpointing */ +@Internal +public class IcebergEnumeratorState implements Serializable { + @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; + private final Collection pendingSplits; + private int[] enumerationSplitCountHistory; + + public IcebergEnumeratorState(Collection pendingSplits) { + this(null, pendingSplits); + } + + public IcebergEnumeratorState( + @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, + Collection pendingSplits) { + this(lastEnumeratedPosition, pendingSplits, new int[0]); + } + + public IcebergEnumeratorState( + @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, + Collection pendingSplits, + int[] enumerationSplitCountHistory) { + this.lastEnumeratedPosition = lastEnumeratedPosition; + this.pendingSplits = pendingSplits; + this.enumerationSplitCountHistory = enumerationSplitCountHistory; + } + + @Nullable + public IcebergEnumeratorPosition lastEnumeratedPosition() { + return lastEnumeratedPosition; + } + + public Collection pendingSplits() { + return pendingSplits; + } + + public int[] enumerationSplitCountHistory() { + return enumerationSplitCountHistory; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java new file mode 100644 index 000000000000..95d6db2cfbc4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java @@ -0,0 +1,196 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergEnumeratorStateSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION = 2; + + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); + + private final IcebergEnumeratorPositionSerializer positionSerializer = + IcebergEnumeratorPositionSerializer.INSTANCE; + private final IcebergSourceSplitSerializer splitSerializer; + + public IcebergEnumeratorStateSerializer(boolean caseSensitive) { + this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergEnumeratorState enumState) throws IOException { + return serializeV2(enumState); + } + + @Override + public IcebergEnumeratorState deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + case 2: + return deserializeV2(serialized); + default: + throw new IOException("Unknown version: " + version); + } + } + + @VisibleForTesting + byte[] serializeV1(IcebergEnumeratorState enumState) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); + serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + @VisibleForTesting + IcebergEnumeratorState deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + IcebergEnumeratorPosition enumeratorPosition = + deserializeEnumeratorPosition(in, positionSerializer); + Collection pendingSplits = + deserializePendingSplits(in, splitSerializer); + return new IcebergEnumeratorState(enumeratorPosition, pendingSplits); + } + + @VisibleForTesting + byte[] serializeV2(IcebergEnumeratorState enumState) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); + serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); + serializeEnumerationSplitCountHistory(out, enumState.enumerationSplitCountHistory()); + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + @VisibleForTesting + IcebergEnumeratorState deserializeV2(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + IcebergEnumeratorPosition enumeratorPosition = + deserializeEnumeratorPosition(in, positionSerializer); + Collection pendingSplits = + deserializePendingSplits(in, splitSerializer); + int[] enumerationSplitCountHistory = deserializeEnumerationSplitCountHistory(in); + return new IcebergEnumeratorState( + enumeratorPosition, pendingSplits, enumerationSplitCountHistory); + } + + private static void serializeEnumeratorPosition( + DataOutputSerializer out, + IcebergEnumeratorPosition enumeratorPosition, + IcebergEnumeratorPositionSerializer positionSerializer) + throws IOException { + out.writeBoolean(enumeratorPosition != null); + if (enumeratorPosition != null) { + out.writeInt(positionSerializer.getVersion()); + byte[] positionBytes = positionSerializer.serialize(enumeratorPosition); + out.writeInt(positionBytes.length); + out.write(positionBytes); + } + } + + private static IcebergEnumeratorPosition deserializeEnumeratorPosition( + DataInputDeserializer in, IcebergEnumeratorPositionSerializer positionSerializer) + throws IOException { + IcebergEnumeratorPosition enumeratorPosition = null; + if (in.readBoolean()) { + int version = in.readInt(); + byte[] positionBytes = new byte[in.readInt()]; + in.read(positionBytes); + enumeratorPosition = positionSerializer.deserialize(version, positionBytes); + } + return enumeratorPosition; + } + + private static void serializePendingSplits( + DataOutputSerializer out, + Collection pendingSplits, + IcebergSourceSplitSerializer splitSerializer) + throws IOException { + out.writeInt(splitSerializer.getVersion()); + out.writeInt(pendingSplits.size()); + for (IcebergSourceSplitState splitState : pendingSplits) { + byte[] splitBytes = splitSerializer.serialize(splitState.split()); + out.writeInt(splitBytes.length); + out.write(splitBytes); + out.writeUTF(splitState.status().name()); + } + } + + private static Collection deserializePendingSplits( + DataInputDeserializer in, IcebergSourceSplitSerializer splitSerializer) throws IOException { + int splitSerializerVersion = in.readInt(); + int splitCount = in.readInt(); + Collection pendingSplits = Lists.newArrayListWithCapacity(splitCount); + for (int i = 0; i < splitCount; ++i) { + byte[] splitBytes = new byte[in.readInt()]; + in.read(splitBytes); + IcebergSourceSplit split = splitSerializer.deserialize(splitSerializerVersion, splitBytes); + String statusName = in.readUTF(); + pendingSplits.add( + new IcebergSourceSplitState(split, IcebergSourceSplitStatus.valueOf(statusName))); + } + return pendingSplits; + } + + private static void serializeEnumerationSplitCountHistory( + DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { + out.writeInt(enumerationSplitCountHistory.length); + if (enumerationSplitCountHistory.length > 0) { + for (int enumerationSplitCount : enumerationSplitCountHistory) { + out.writeInt(enumerationSplitCount); + } + } + } + + private static int[] deserializeEnumerationSplitCountHistory(DataInputDeserializer in) + throws IOException { + int historySize = in.readInt(); + int[] history = new int[historySize]; + if (historySize > 0) { + for (int i = 0; i < historySize; ++i) { + history[i] = in.readInt(); + } + } + + return history; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java new file mode 100644 index 000000000000..4e55ea5d5fd6 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java @@ -0,0 +1,51 @@ +/* + * 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.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** One-time split enumeration at the start-up for batch execution */ +@Internal +public class StaticIcebergEnumerator extends AbstractIcebergEnumerator { + private final SplitAssigner assigner; + + public StaticIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { + super(enumeratorContext, assigner); + this.assigner = assigner; + } + + @Override + public void start() { + super.start(); + } + + @Override + protected boolean shouldWaitForMoreSplits() { + return false; + } + + @Override + public IcebergEnumeratorState snapshotState(long checkpointId) { + return new IcebergEnumeratorState(null, assigner.state(), new int[0]); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java new file mode 100644 index 000000000000..7b94c364c976 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.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.flink.source.reader; + +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * {@link RecordsWithSplitIds} is used to pass a batch of records from fetcher to source reader. + * Batching is to improve the efficiency for records handover. + * + *

    {@link RecordsWithSplitIds} interface can encapsulate batches from multiple splits. This is + * the case for Kafka source where fetchers can retrieve records from multiple Kafka partitions at + * the same time. + * + *

    For file-based sources like Iceberg, readers always read one split/file at a time. Hence, we + * will only have a batch of records for one split here. + * + *

    This class uses array to store a batch of records from the same file (with the same + * fileOffset). + */ +class ArrayBatchRecords implements RecordsWithSplitIds> { + @Nullable private String splitId; + @Nullable private final Pool.Recycler recycler; + @Nullable private final T[] records; + private final int numberOfRecords; + private final Set finishedSplits; + private final RecordAndPosition recordAndPosition; + + // point to current read position within the records array + private int position; + + private ArrayBatchRecords( + @Nullable String splitId, + @Nullable Pool.Recycler recycler, + @Nullable T[] records, + int numberOfRecords, + int fileOffset, + long startingRecordOffset, + Set finishedSplits) { + Preconditions.checkArgument(numberOfRecords >= 0, "numberOfRecords can't be negative"); + Preconditions.checkArgument(fileOffset >= 0, "fileOffset can't be negative"); + Preconditions.checkArgument(startingRecordOffset >= 0, "numberOfRecords can't be negative"); + + this.splitId = splitId; + this.recycler = recycler; + this.records = records; + this.numberOfRecords = numberOfRecords; + this.finishedSplits = + Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); + this.recordAndPosition = new RecordAndPosition<>(); + + recordAndPosition.set(null, fileOffset, startingRecordOffset); + this.position = 0; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = this.splitId; + // set the splitId to null to indicate no more splits + // this class only contains record for one split + this.splitId = null; + return nextSplit; + } + + @Nullable + @Override + public RecordAndPosition nextRecordFromSplit() { + if (position < numberOfRecords) { + recordAndPosition.record(records[position]); + position++; + return recordAndPosition; + } else { + return null; + } + } + + /** + * This method is called when all records from this batch has been emitted. If recycler is set, it + * should be called to return the records array back to pool. + */ + @Override + public void recycle() { + if (recycler != null) { + recycler.recycle(records); + } + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + @VisibleForTesting + T[] records() { + return records; + } + + @VisibleForTesting + int numberOfRecords() { + return numberOfRecords; + } + + /** + * Create a ArrayBatchRecords backed up an array with records from the same file + * + * @param splitId Iceberg source only read from one split a time. We never have multiple records + * from multiple splits. + * @param recycler Because {@link DataIterator} with {@link RowData} returns an iterator of reused + * RowData object, we need to clone RowData eagerly when constructing a batch of records. We + * can use object pool to reuse the RowData array object which can be expensive to create. + * This recycler can be provided to recycle the array object back to pool after read is + * exhausted. If the {@link DataIterator} returns an iterator of non-reused objects, we don't + * need to clone objects. It is cheap to just create the batch array. Hence, we don't need + * object pool and recycler can be set to null. + * @param records an array (maybe reused) holding a batch of records + * @param numberOfRecords actual number of records in the array + * @param fileOffset fileOffset for all records in this batch + * @param startingRecordOffset starting recordOffset + * @param record type + */ + public static ArrayBatchRecords forRecords( + String splitId, + Pool.Recycler recycler, + T[] records, + int numberOfRecords, + int fileOffset, + long startingRecordOffset) { + return new ArrayBatchRecords<>( + splitId, + recycler, + records, + numberOfRecords, + fileOffset, + startingRecordOffset, + Collections.emptySet()); + } + + /** + * Create ab ArrayBatchRecords with only finished split id + * + * @param splitId for the split that is just exhausted + */ + public static ArrayBatchRecords finishedSplit(String splitId) { + return new ArrayBatchRecords<>(null, null, null, 0, 0, 0, Collections.singleton(splitId)); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java new file mode 100644 index 000000000000..306afd1811be --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.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.source.reader; + +import java.io.IOException; +import java.util.NoSuchElementException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderOptions; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** This implementation stores record batch in array from recyclable pool */ +class ArrayPoolDataIteratorBatcher implements DataIteratorBatcher { + private final int batchSize; + private final int handoverQueueSize; + private final RecordFactory recordFactory; + + private transient Pool pool; + + ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory recordFactory) { + this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); + this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY); + this.recordFactory = recordFactory; + } + + @Override + public CloseableIterator>> batch( + String splitId, DataIterator inputIterator) { + Preconditions.checkArgument(inputIterator != null, "Input data iterator can't be null"); + // lazily create pool as it is not serializable + if (pool == null) { + this.pool = createPoolOfBatches(handoverQueueSize); + } + return new ArrayPoolBatchIterator(splitId, inputIterator, pool); + } + + private Pool createPoolOfBatches(int numBatches) { + Pool poolOfBatches = new Pool<>(numBatches); + for (int batchId = 0; batchId < numBatches; batchId++) { + T[] batch = recordFactory.createBatch(batchSize); + poolOfBatches.add(batch); + } + + return poolOfBatches; + } + + private class ArrayPoolBatchIterator + implements CloseableIterator>> { + + private final String splitId; + private final DataIterator inputIterator; + private final Pool pool; + + ArrayPoolBatchIterator(String splitId, DataIterator inputIterator, Pool pool) { + this.splitId = splitId; + this.inputIterator = inputIterator; + this.pool = pool; + } + + @Override + public boolean hasNext() { + return inputIterator.hasNext(); + } + + @Override + public RecordsWithSplitIds> next() { + if (!inputIterator.hasNext()) { + throw new NoSuchElementException(); + } + + T[] batch = getCachedEntry(); + int recordCount = 0; + while (inputIterator.hasNext() && recordCount < batchSize) { + // The record produced by inputIterator can be reused like for the RowData case. + // inputIterator.next() can't be called again until the copy is made + // since the record is not consumed immediately. + T nextRecord = inputIterator.next(); + recordFactory.clone(nextRecord, batch, recordCount); + recordCount++; + if (!inputIterator.currentFileHasNext()) { + // break early so that records in the ArrayResultIterator + // have the same fileOffset. + break; + } + } + + return ArrayBatchRecords.forRecords( + splitId, + pool.recycler(), + batch, + recordCount, + inputIterator.fileOffset(), + inputIterator.recordOffset() - recordCount); + } + + @Override + public void close() throws IOException { + inputIterator.close(); + } + + private T[] getCachedEntry() { + try { + return pool.pollEntry(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for array pool entry", e); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java new file mode 100644 index 000000000000..66e59633fff2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -0,0 +1,102 @@ +/* + * 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 java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Read Iceberg rows as {@link GenericRecord}. */ +public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { + private final String tableName; + private final Schema readSchema; + private final FileIO io; + private final EncryptionManager encryption; + private final RowDataFileScanTaskReader rowDataReader; + + private transient RowDataToAvroGenericRecordConverter converter; + + /** + * Create a reader function without projection and name mapping. Column name is case-insensitive. + */ + public static AvroGenericRecordReaderFunction fromTable(Table table) { + return new AvroGenericRecordReaderFunction( + table.name(), + new Configuration(), + table.schema(), + null, + null, + false, + table.io(), + table.encryption(), + null); + } + + public AvroGenericRecordReaderFunction( + String tableName, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters) { + super(new ListDataIteratorBatcher<>(config)); + this.tableName = tableName; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.io = io; + this.encryption = encryption; + this.rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>( + new AvroGenericRecordFileScanTaskReader(rowDataReader, lazyConverter()), + split.task(), + io, + encryption); + } + + private RowDataToAvroGenericRecordConverter lazyConverter() { + if (converter == null) { + this.converter = RowDataToAvroGenericRecordConverter.fromIcebergSchema(tableName, readSchema); + } + return converter; + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..4bb6f0a98c4c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -0,0 +1,98 @@ +/* + * 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 java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for column name = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) + .min(Comparator.comparingLong(l -> l)) + .get(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java new file mode 100644 index 000000000000..c376e359c600 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.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.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Batcher converts iterator of T into iterator of batched {@code + * RecordsWithSplitIds>}, as FLIP-27's {@link SplitReader#fetch()} returns + * batched records. + */ +@FunctionalInterface +public interface DataIteratorBatcher extends Serializable { + CloseableIterator>> batch( + String splitId, DataIterator inputIterator); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java new file mode 100644 index 000000000000..bbf797ef4aa8 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java @@ -0,0 +1,43 @@ +/* + * 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.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +/** A {@link ReaderFunction} implementation that uses {@link DataIterator}. */ +public abstract class DataIteratorReaderFunction implements ReaderFunction { + private final DataIteratorBatcher batcher; + + public DataIteratorReaderFunction(DataIteratorBatcher batcher) { + this.batcher = batcher; + } + + protected abstract DataIterator createDataIterator(IcebergSourceSplit split); + + @Override + public CloseableIterator>> apply( + IcebergSourceSplit split) { + DataIterator inputIterator = createDataIterator(split); + inputIterator.seek(split.fileOffset(), split.recordOffset()); + return batcher.batch(split.splitId(), inputIterator); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java new file mode 100644 index 000000000000..f143b8d2df2e --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.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.flink.source.reader; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergSourceReader + extends SingleThreadMultiplexSourceReaderBase< + RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { + + public IcebergSourceReader( + SerializableRecordEmitter emitter, + IcebergSourceReaderMetrics metrics, + ReaderFunction readerFunction, + SerializableComparator splitComparator, + SourceReaderContext context) { + super( + () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), + emitter, + context.getConfiguration(), + context); + } + + @Override + public void start() { + // We request a split only if we did not get splits during the checkpoint restore. + // Otherwise, reader restarts will keep requesting more and more splits. + if (getNumberOfCurrentlyAssignedSplits() == 0) { + requestSplit(Collections.emptyList()); + } + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + requestSplit(Lists.newArrayList(finishedSplitIds.keySet())); + } + + @Override + protected IcebergSourceSplit initializedState(IcebergSourceSplit split) { + return split; + } + + @Override + protected IcebergSourceSplit toSplitType(String splitId, IcebergSourceSplit splitState) { + return splitState; + } + + private void requestSplit(Collection finishedSplitIds) { + context.sendSourceEventToCoordinator(new SplitRequestEvent(finishedSplitIds)); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java new file mode 100644 index 000000000000..2a3e1dd86b95 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java @@ -0,0 +1,61 @@ +/* + * 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.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; + +public class IcebergSourceReaderMetrics { + private final Counter assignedSplits; + private final Counter assignedBytes; + private final Counter finishedSplits; + private final Counter finishedBytes; + private final Counter splitReaderFetchCalls; + + public IcebergSourceReaderMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup readerMetrics = + metrics.addGroup("IcebergSourceReader").addGroup("table", fullTableName); + + this.assignedSplits = readerMetrics.counter("assignedSplits"); + this.assignedBytes = readerMetrics.counter("assignedBytes"); + this.finishedSplits = readerMetrics.counter("finishedSplits"); + this.finishedBytes = readerMetrics.counter("finishedBytes"); + this.splitReaderFetchCalls = readerMetrics.counter("splitReaderFetchCalls"); + } + + public void incrementAssignedSplits(long count) { + assignedSplits.inc(count); + } + + public void incrementAssignedBytes(long count) { + assignedBytes.inc(count); + } + + public void incrementFinishedSplits(long count) { + finishedSplits.inc(count); + } + + public void incrementFinishedBytes(long count) { + finishedBytes.inc(count); + } + + public void incrementSplitReaderFetchCalls(long count) { + splitReaderFetchCalls.inc(count); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java new file mode 100644 index 000000000000..9c20494fdbcd --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -0,0 +1,167 @@ +/* + * 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 java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergSourceSplitReader implements SplitReader, IcebergSourceSplit> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceSplitReader.class); + + private final IcebergSourceReaderMetrics metrics; + private final ReaderFunction openSplitFunction; + private final SerializableComparator splitComparator; + private final int indexOfSubtask; + private final Queue splits; + + private CloseableIterator>> currentReader; + private IcebergSourceSplit currentSplit; + private String currentSplitId; + + IcebergSourceSplitReader( + IcebergSourceReaderMetrics metrics, + ReaderFunction openSplitFunction, + SerializableComparator splitComparator, + SourceReaderContext context) { + this.metrics = metrics; + this.openSplitFunction = openSplitFunction; + this.splitComparator = splitComparator; + this.indexOfSubtask = context.getIndexOfSubtask(); + this.splits = Queues.newArrayDeque(); + } + + /** + * The method reads a batch of records from the assigned splits. If all the records from the + * current split are returned then it will emit a {@link ArrayBatchRecords#finishedSplit(String)} + * batch to signal this event. In the next fetch loop the reader will continue with the next split + * (if any). + * + * @return The fetched records + * @throws IOException If there is an error during reading + */ + @Override + public RecordsWithSplitIds> fetch() throws IOException { + metrics.incrementSplitReaderFetchCalls(1); + if (currentReader == null) { + IcebergSourceSplit nextSplit = splits.poll(); + if (nextSplit != null) { + currentSplit = nextSplit; + currentSplitId = nextSplit.splitId(); + currentReader = openSplitFunction.apply(currentSplit); + } else { + // return an empty result, which will lead to split fetch to be idle. + // SplitFetcherManager will then close idle fetcher. + return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + } + } + + if (currentReader.hasNext()) { + // Because Iterator#next() doesn't support checked exception, + // we need to wrap and unwrap the checked IOException with UncheckedIOException + try { + return currentReader.next(); + } catch (UncheckedIOException e) { + throw e.getCause(); + } + } else { + return finishSplit(); + } + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChange) { + if (!(splitsChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format("Unsupported split change: %s", splitsChange.getClass())); + } + + if (splitComparator != null) { + List newSplits = Lists.newArrayList(splitsChange.splits()); + newSplits.sort(splitComparator); + LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); + splits.addAll(newSplits); + } else { + LOG.info("Add {} splits to reader", splitsChange.splits().size()); + splits.addAll(splitsChange.splits()); + } + metrics.incrementAssignedSplits(splitsChange.splits().size()); + metrics.incrementAssignedBytes(calculateBytes(splitsChange)); + } + + @Override + public void wakeUp() {} + + @Override + public void close() throws Exception { + currentSplitId = null; + if (currentReader != null) { + currentReader.close(); + } + } + + @Override + public void pauseOrResumeSplits( + Collection splitsToPause, Collection splitsToResume) { + // IcebergSourceSplitReader only reads splits sequentially. When waiting for watermark alignment + // the SourceOperator will stop processing and recycling the fetched batches. This exhausts the + // {@link ArrayPoolDataIteratorBatcher#pool} and the `currentReader.next()` call will be + // blocked even without split-level watermark alignment. Based on this the + // `pauseOrResumeSplits` and the `wakeUp` are left empty. + } + + private long calculateBytes(IcebergSourceSplit split) { + return split.task().files().stream().map(FileScanTask::length).reduce(0L, Long::sum); + } + + private long calculateBytes(SplitsChange splitsChanges) { + return splitsChanges.splits().stream().map(this::calculateBytes).reduce(0L, Long::sum); + } + + private ArrayBatchRecords finishSplit() throws IOException { + if (currentReader != null) { + currentReader.close(); + currentReader = null; + } + + ArrayBatchRecords finishRecords = ArrayBatchRecords.finishedSplit(currentSplitId); + LOG.info("Split reader {} finished split: {}", indexOfSubtask, currentSplitId); + metrics.incrementFinishedSplits(1); + metrics.incrementFinishedBytes(calculateBytes(currentSplit)); + currentSplitId = null; + return finishRecords; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java new file mode 100644 index 000000000000..1acb3df76102 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java @@ -0,0 +1,85 @@ +/* + * 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 java.util.Collections; +import java.util.List; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class ListBatchRecords implements RecordsWithSplitIds> { + private String splitId; + private final List records; + private final Set finishedSplits; + private final RecordAndPosition recordAndPosition; + + // point to current read position within the records list + private int position; + + ListBatchRecords( + String splitId, + List records, + int fileOffset, + long startingRecordOffset, + Set finishedSplits) { + this.splitId = splitId; + this.records = records; + this.finishedSplits = + Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); + this.recordAndPosition = new RecordAndPosition<>(); + this.recordAndPosition.set(null, fileOffset, startingRecordOffset); + + this.position = 0; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = this.splitId; + // set the splitId to null to indicate no more splits + // this class only contains record for one split + this.splitId = null; + return nextSplit; + } + + @Nullable + @Override + public RecordAndPosition nextRecordFromSplit() { + if (position < records.size()) { + recordAndPosition.record(records.get(position)); + position++; + return recordAndPosition; + } else { + return null; + } + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + public static ListBatchRecords forRecords( + String splitId, List records, int fileOffset, long startingRecordOffset) { + return new ListBatchRecords<>( + splitId, records, fileOffset, startingRecordOffset, Collections.emptySet()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java new file mode 100644 index 000000000000..365416239d37 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java @@ -0,0 +1,94 @@ +/* + * 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 java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * FlinkRecordReaderFunction essentially cloned objects already. So there is no need to use array + * pool to clone objects. Simply create a new ArrayList for each batch. + */ +class ListDataIteratorBatcher implements DataIteratorBatcher { + + private final int batchSize; + + ListDataIteratorBatcher(ReadableConfig config) { + this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); + } + + @Override + public CloseableIterator>> batch( + String splitId, DataIterator dataIterator) { + return new ListBatchIterator(splitId, dataIterator); + } + + private class ListBatchIterator + implements CloseableIterator>> { + + private final String splitId; + private final DataIterator inputIterator; + + ListBatchIterator(String splitId, DataIterator inputIterator) { + this.splitId = splitId; + this.inputIterator = inputIterator; + } + + @Override + public boolean hasNext() { + return inputIterator.hasNext(); + } + + @Override + public RecordsWithSplitIds> next() { + if (!inputIterator.hasNext()) { + throw new NoSuchElementException(); + } + + final List batch = Lists.newArrayListWithCapacity(batchSize); + int recordCount = 0; + while (inputIterator.hasNext() && recordCount < batchSize) { + T nextRecord = inputIterator.next(); + batch.add(nextRecord); + recordCount++; + if (!inputIterator.currentFileHasNext()) { + // break early so that records have the same fileOffset. + break; + } + } + + return ListBatchRecords.forRecords( + splitId, batch, inputIterator.fileOffset(), inputIterator.recordOffset() - recordCount); + } + + @Override + public void close() throws IOException { + if (inputIterator != null) { + inputIterator.close(); + } + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java new file mode 100644 index 000000000000..fb4466913b90 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.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.flink.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.DataTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Reading metadata tables (like snapshots, manifests, etc.) */ +@Internal +public class MetaDataReaderFunction extends DataIteratorReaderFunction { + private final Schema readSchema; + private final FileIO io; + private final EncryptionManager encryption; + + public MetaDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + FileIO io, + EncryptionManager encryption) { + super( + new ArrayPoolDataIteratorBatcher<>( + config, + new RowDataRecordFactory( + FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); + this.readSchema = readSchema(tableSchema, projectedSchema); + this.io = io; + this.encryption = encryption; + } + + @Override + public DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>(new DataTaskReader(readSchema), split.task(), io, encryption); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java new file mode 100644 index 000000000000..1ea91f10b4e7 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java @@ -0,0 +1,31 @@ +/* + * 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 java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +@FunctionalInterface +public interface ReaderFunction + extends Serializable, + Function< + IcebergSourceSplit, CloseableIterator>>> {} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java new file mode 100644 index 000000000000..6ac92592b6aa --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.annotation.Internal; + +/** + * A record along with the reader position to be stored in the checkpoint. + * + *

    The position defines the point in the reader AFTER the record. Record processing and updating + * checkpointed state happens atomically. The position points to where the reader should resume + * after this record is processed. + * + *

    This mutable object is useful in cases where only one instance of a {@code RecordAndPosition} + * is needed at a time. Then the same instance of RecordAndPosition can be reused. + */ +@Internal +public class RecordAndPosition { + private T record; + private int fileOffset; + private long recordOffset; + + public RecordAndPosition(T record, int fileOffset, long recordOffset) { + this.record = record; + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + public RecordAndPosition() {} + + // ------------------------------------------------------------------------ + + public T record() { + return record; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + /** Updates the record and position in this object. */ + public void set(T newRecord, int newFileOffset, long newRecordOffset) { + this.record = newRecord; + this.fileOffset = newFileOffset; + this.recordOffset = newRecordOffset; + } + + /** Sets the next record of a sequence. This increments the {@code recordOffset} by one. */ + public void record(T nextRecord) { + this.record = nextRecord; + this.recordOffset++; + } + + @Override + public String toString() { + return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java new file mode 100644 index 000000000000..ef92e2e6b81f --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.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.source.reader; + +import java.io.Serializable; + +/** + * In FLIP-27 source, SplitReader#fetch() returns a batch of records. Since DataIterator for RowData + * returns an iterator of reused RowData objects, RecordFactory is needed to (1) create object array + * that is recyclable via pool. (2) clone RowData element from DataIterator to the batch array. + */ +interface RecordFactory extends Serializable { + /** Create a batch of records */ + T[] createBatch(int batchSize); + + /** Clone record into the specified position of the batch array */ + void clone(T from, T[] batch, int position); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java new file mode 100644 index 000000000000..5d0a00954e7a --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -0,0 +1,79 @@ +/* + * 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 java.util.List; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class RowDataReaderFunction extends DataIteratorReaderFunction { + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters) { + super( + new ArrayPoolDataIteratorBatcher<>( + config, + new RowDataRecordFactory( + FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + } + + @Override + public DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>( + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), + split.task(), + io, + encryption); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java new file mode 100644 index 000000000000..1e265b2663ce --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java @@ -0,0 +1,61 @@ +/* + * 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.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.flink.data.RowDataUtil; + +class RowDataRecordFactory implements RecordFactory { + private final RowType rowType; + private final TypeSerializer[] fieldSerializers; + + RowDataRecordFactory(RowType rowType) { + this.rowType = rowType; + this.fieldSerializers = createFieldSerializers(rowType); + } + + static TypeSerializer[] createFieldSerializers(RowType rowType) { + return rowType.getChildren().stream() + .map(InternalSerializers::create) + .toArray(TypeSerializer[]::new); + } + + @Override + public RowData[] createBatch(int batchSize) { + RowData[] arr = new RowData[batchSize]; + for (int i = 0; i < batchSize; ++i) { + arr[i] = new GenericRowData(rowType.getFieldCount()); + } + return arr; + } + + @Override + public void clone(RowData from, RowData[] batch, int position) { + // Set the return value from RowDataUtil.clone back to the array. + // Clone method returns same clone target object (reused) if it is a GenericRowData. + // Clone method will allocate a new GenericRowData object + // if the target object is NOT a GenericRowData. + // So we should always set the clone return value back to the array. + batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java new file mode 100644 index 000000000000..a6e2c1dae243 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.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.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +@FunctionalInterface +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java new file mode 100644 index 000000000000..d1c50ac8ca52 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.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.flink.source.reader; + +import java.io.Serializable; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** The interface used to extract watermarks from splits. */ +public interface SplitWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java new file mode 100644 index 000000000000..02ef57d344b1 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.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.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + *

    The Emitter emits watermarks at the beginning of every split provided by the {@link + * SplitWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; + private long watermark; + + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplitId)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + watermark, + newWatermark, + lastSplitId, + split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); + } + + lastSplitId = split.splitId(); + } + + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} 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 new file mode 100644 index 000000000000..e4bfbf1452e2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -0,0 +1,177 @@ +/* + * 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.split; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.FileScanTaskParser; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergSourceSplit implements SourceSplit, Serializable { + private static final long serialVersionUID = 1L; + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); + + private final CombinedScanTask task; + + private int fileOffset; + private long recordOffset; + + // The splits are frequently serialized into checkpoints. + // Caching the byte representation makes repeated serialization cheap. + @Nullable private transient byte[] serializedBytesCache; + + private IcebergSourceSplit(CombinedScanTask task, int fileOffset, long recordOffset) { + this.task = task; + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) { + return fromCombinedScanTask(combinedScanTask, 0, 0L); + } + + public static IcebergSourceSplit fromCombinedScanTask( + CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) { + return new IcebergSourceSplit(combinedScanTask, fileOffset, recordOffset); + } + + public CombinedScanTask task() { + return task; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + @Override + public String splitId() { + return MoreObjects.toStringHelper(this).add("files", toString(task.files())).toString(); + } + + public void updatePosition(int newFileOffset, long newRecordOffset) { + // invalidate the cache after position change + serializedBytesCache = null; + fileOffset = newFileOffset; + recordOffset = newRecordOffset; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("files", toString(task.files())) + .add("fileOffset", fileOffset) + .add("recordOffset", recordOffset) + .toString(); + } + + private String toString(Collection files) { + return Iterables.toString( + files.stream() + .map( + fileScanTask -> + MoreObjects.toStringHelper(fileScanTask) + .add("file", fileScanTask.file().path().toString()) + .add("start", fileScanTask.start()) + .add("length", fileScanTask.length()) + .toString()) + .collect(Collectors.toList())); + } + + byte[] serializeV1() throws IOException { + if (serializedBytesCache == null) { + serializedBytesCache = InstantiationUtil.serializeObject(this); + } + + return serializedBytesCache; + } + + static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { + try { + return InstantiationUtil.deserializeObject( + serialized, IcebergSourceSplit.class.getClassLoader()); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Failed to deserialize the split.", e); + } + } + + byte[] serializeV2() throws IOException { + if (serializedBytesCache == null) { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + Collection fileScanTasks = task.tasks(); + Preconditions.checkArgument( + fileOffset >= 0 && fileOffset < fileScanTasks.size(), + "Invalid file offset: %s. Should be within the range of [0, %s)", + fileOffset, + fileScanTasks.size()); + + out.writeInt(fileOffset); + out.writeLong(recordOffset); + out.writeInt(fileScanTasks.size()); + + for (FileScanTask fileScanTask : fileScanTasks) { + String taskJson = FileScanTaskParser.toJson(fileScanTask); + out.writeUTF(taskJson); + } + + serializedBytesCache = out.getCopyOfBuffer(); + out.clear(); + } + + return serializedBytesCache; + } + + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) + throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + int fileOffset = in.readInt(); + long recordOffset = in.readLong(); + int taskCount = in.readInt(); + + List tasks = Lists.newArrayListWithCapacity(taskCount); + for (int i = 0; i < taskCount; ++i) { + String taskJson = in.readUTF(); + FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); + tasks.add(task); + } + + CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); + return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java new file mode 100644 index 000000000000..8c089819e731 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.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.flink.source.split; + +import java.io.IOException; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +@Internal +public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 2; + + private final boolean caseSensitive; + + public IcebergSourceSplitSerializer(boolean caseSensitive) { + this.caseSensitive = caseSensitive; + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergSourceSplit split) throws IOException { + return split.serializeV2(); + } + + @Override + public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return IcebergSourceSplit.deserializeV1(serialized); + case 2: + return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); + default: + throw new IOException( + String.format( + "Failed to deserialize IcebergSourceSplit. " + + "Encountered unsupported version: %d. Supported version are [1]", + version)); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java new file mode 100644 index 000000000000..d9061e049e00 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.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.flink.source.split; + +public class IcebergSourceSplitState { + private final IcebergSourceSplit split; + private final IcebergSourceSplitStatus status; + + public IcebergSourceSplitState(IcebergSourceSplit split, IcebergSourceSplitStatus status) { + this.split = split; + this.status = status; + } + + public IcebergSourceSplit split() { + return split; + } + + public IcebergSourceSplitStatus status() { + return status; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java new file mode 100644 index 000000000000..d4a84a165e1a --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java @@ -0,0 +1,25 @@ +/* + * 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.split; + +public enum IcebergSourceSplitStatus { + UNASSIGNED, + ASSIGNED, + COMPLETED +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java new file mode 100644 index 000000000000..319648ca275c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * 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.split; + +import java.io.Serializable; +import java.util.Comparator; + +public interface SerializableComparator extends Comparator, Serializable {} 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 new file mode 100644 index 000000000000..56ee92014d12 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} + * which could be used for ordering splits. These are used by the {@link + * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link + * org.apache.iceberg.flink.source.reader.IcebergSourceReader} + */ +public class SplitComparators { + private SplitComparators() {} + + /** Comparator which orders the splits based on the file sequence number of the data files */ + 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"); + + Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); + Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); + + Preconditions.checkNotNull( + seq1, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o1); + Preconditions.checkNotNull( + seq2, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); + + int temp = Long.compare(watermark1, watermark2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java new file mode 100644 index 000000000000..eabd757aa638 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java @@ -0,0 +1,54 @@ +/* + * 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.split; + +import java.util.Collection; +import java.util.Collections; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; + +/** We can remove this class once FLINK-21364 is resolved. */ +@Internal +public class SplitRequestEvent implements SourceEvent { + private static final long serialVersionUID = 1L; + + private final Collection finishedSplitIds; + private final String requesterHostname; + + public SplitRequestEvent() { + this(Collections.emptyList()); + } + + public SplitRequestEvent(Collection finishedSplitIds) { + this(finishedSplitIds, null); + } + + public SplitRequestEvent(Collection finishedSplitIds, String requesterHostname) { + this.finishedSplitIds = finishedSplitIds; + this.requesterHostname = requesterHostname; + } + + public Collection finishedSplitIds() { + return finishedSplitIds; + } + + public String requesterHostname() { + return requesterHostname; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java new file mode 100644 index 000000000000..6306e82d5729 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java @@ -0,0 +1,47 @@ +/* + * 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.util; + +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Gauge; + +/** + * This gauge measures the elapsed time between now and last recorded time set by {@link + * ElapsedTimeGauge#refreshLastRecordedTime()}. + */ +@Internal +public class ElapsedTimeGauge implements Gauge { + private final TimeUnit reportUnit; + private volatile long lastRecordedTimeNano; + + public ElapsedTimeGauge(TimeUnit timeUnit) { + this.reportUnit = timeUnit; + refreshLastRecordedTime(); + } + + public void refreshLastRecordedTime() { + this.lastRecordedTimeNano = System.nanoTime(); + } + + @Override + public Long getValue() { + return reportUnit.convert(System.nanoTime() - lastRecordedTimeNano, TimeUnit.NANOSECONDS); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java new file mode 100644 index 000000000000..2bbc9cf208fe --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java @@ -0,0 +1,248 @@ +/* + * 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.util; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; + +public class FlinkAlterTableUtil { + private FlinkAlterTableUtil() {} + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + Map setProperties) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!setProperties.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + setProperties.forEach( + (k, v) -> { + if (v == null) { + updateProperties.remove(k); + } else { + updateProperties.set(k, v); + } + }); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + List schemaChanges, + List propertyChanges) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!schemaChanges.isEmpty()) { + UpdateSchema updateSchema = transaction.updateSchema(); + FlinkAlterTableUtil.applySchemaChanges(updateSchema, schemaChanges); + updateSchema.commit(); + } + + if (!propertyChanges.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + FlinkAlterTableUtil.applyPropertyChanges(updateProperties, propertyChanges); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitManageSnapshots( + Table table, String setSnapshotId, String cherrypickSnapshotId) { + // don't allow setting the snapshot and picking a commit at the same time because order is + // ambiguous and choosing one order leads to different results + Preconditions.checkArgument( + setSnapshotId == null || cherrypickSnapshotId == null, + "Cannot set the current snapshot ID and cherry-pick snapshot changes"); + + if (setSnapshotId != null) { + long newSnapshotId = Long.parseLong(setSnapshotId); + table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); + } + + // if updating the table snapshot, perform that update first in case it fails + if (cherrypickSnapshotId != null) { + long newSnapshotId = Long.parseLong(cherrypickSnapshotId); + table.manageSnapshots().cherrypick(newSnapshotId).commit(); + } + } + + /** + * Applies a list of Flink table changes to an {@link UpdateSchema} operation. + * + * @param pendingUpdate an uncommitted UpdateSchema operation to configure + * @param schemaChanges a list of Flink table changes + */ + public static void applySchemaChanges( + UpdateSchema pendingUpdate, List schemaChanges) { + for (TableChange change : schemaChanges) { + if (change instanceof TableChange.AddColumn) { + TableChange.AddColumn addColumn = (TableChange.AddColumn) change; + Column flinkColumn = addColumn.getColumn(); + Preconditions.checkArgument( + FlinkCompatibilityUtil.isPhysicalColumn(flinkColumn), + "Unsupported table change: Adding computed column %s.", + flinkColumn.getName()); + Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); + if (flinkColumn.getDataType().getLogicalType().isNullable()) { + pendingUpdate.addColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); + } else { + pendingUpdate.addRequiredColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); + } + } else if (change instanceof TableChange.ModifyColumn) { + TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; + applyModifyColumn(pendingUpdate, modifyColumn); + } else if (change instanceof TableChange.DropColumn) { + TableChange.DropColumn dropColumn = (TableChange.DropColumn) change; + pendingUpdate.deleteColumn(dropColumn.getColumnName()); + } else if (change instanceof TableChange.AddWatermark) { + throw new UnsupportedOperationException("Unsupported table change: AddWatermark."); + } else if (change instanceof TableChange.ModifyWatermark) { + throw new UnsupportedOperationException("Unsupported table change: ModifyWatermark."); + } else if (change instanceof TableChange.DropWatermark) { + throw new UnsupportedOperationException("Unsupported table change: DropWatermark."); + } else if (change instanceof TableChange.AddUniqueConstraint) { + TableChange.AddUniqueConstraint addPk = (TableChange.AddUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, addPk.getConstraint()); + } else if (change instanceof TableChange.ModifyUniqueConstraint) { + TableChange.ModifyUniqueConstraint modifyPk = (TableChange.ModifyUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, modifyPk.getNewConstraint()); + } else if (change instanceof TableChange.DropConstraint) { + throw new UnsupportedOperationException("Unsupported table change: DropConstraint."); + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + } + + /** + * Applies a list of Flink table property changes to an {@link UpdateProperties} operation. + * + * @param pendingUpdate an uncommitted UpdateProperty operation to configure + * @param propertyChanges a list of Flink table changes + */ + public static void applyPropertyChanges( + UpdateProperties pendingUpdate, List propertyChanges) { + for (TableChange change : propertyChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption setOption = (TableChange.SetOption) change; + pendingUpdate.set(setOption.getKey(), setOption.getValue()); + } else if (change instanceof TableChange.ResetOption) { + TableChange.ResetOption resetOption = (TableChange.ResetOption) change; + pendingUpdate.remove(resetOption.getKey()); + } else { + throw new UnsupportedOperationException( + "The given table change is not a property change: " + change); + } + } + } + + private static void applyModifyColumn( + UpdateSchema pendingUpdate, TableChange.ModifyColumn modifyColumn) { + if (modifyColumn instanceof TableChange.ModifyColumnName) { + TableChange.ModifyColumnName modifyName = (TableChange.ModifyColumnName) modifyColumn; + pendingUpdate.renameColumn(modifyName.getOldColumnName(), modifyName.getNewColumnName()); + } else if (modifyColumn instanceof TableChange.ModifyColumnPosition) { + TableChange.ModifyColumnPosition modifyPosition = + (TableChange.ModifyColumnPosition) modifyColumn; + applyModifyColumnPosition(pendingUpdate, modifyPosition); + } else if (modifyColumn instanceof TableChange.ModifyPhysicalColumnType) { + TableChange.ModifyPhysicalColumnType modifyType = + (TableChange.ModifyPhysicalColumnType) modifyColumn; + Type type = FlinkSchemaUtil.convert(modifyType.getNewType().getLogicalType()); + String columnName = modifyType.getOldColumn().getName(); + pendingUpdate.updateColumn(columnName, type.asPrimitiveType()); + if (modifyType.getNewColumn().getDataType().getLogicalType().isNullable()) { + pendingUpdate.makeColumnOptional(columnName); + } else { + pendingUpdate.requireColumn(columnName); + } + } else if (modifyColumn instanceof TableChange.ModifyColumnComment) { + TableChange.ModifyColumnComment modifyComment = + (TableChange.ModifyColumnComment) modifyColumn; + pendingUpdate.updateColumnDoc( + modifyComment.getOldColumn().getName(), modifyComment.getNewComment()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column change: " + modifyColumn); + } + } + + private static void applyModifyColumnPosition( + UpdateSchema pendingUpdate, TableChange.ModifyColumnPosition modifyColumnPosition) { + TableChange.ColumnPosition newPosition = modifyColumnPosition.getNewPosition(); + if (newPosition instanceof TableChange.First) { + pendingUpdate.moveFirst(modifyColumnPosition.getOldColumn().getName()); + } else if (newPosition instanceof TableChange.After) { + TableChange.After after = (TableChange.After) newPosition; + pendingUpdate.moveAfter(modifyColumnPosition.getOldColumn().getName(), after.column()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column-position change: " + modifyColumnPosition); + } + } + + private static void applyUniqueConstraint( + UpdateSchema pendingUpdate, UniqueConstraint constraint) { + switch (constraint.getType()) { + case PRIMARY_KEY: + pendingUpdate.setIdentifierFields(constraint.getColumns()); + break; + case UNIQUE_KEY: + throw new UnsupportedOperationException( + "Unsupported table change: setting unique key constraints."); + default: + throw new UnsupportedOperationException( + "Cannot apply unknown unique constraint: " + constraint.getType().name()); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java new file mode 100644 index 000000000000..f02af894e82b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java @@ -0,0 +1,47 @@ +/* + * 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.util; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +/** + * This is a small util class that try to hide calls to Flink Internal or PublicEvolve interfaces as + * Flink can change those APIs during minor version release. + */ +public class FlinkCompatibilityUtil { + + private FlinkCompatibilityUtil() {} + + public static TypeInformation toTypeInfo(RowType rowType) { + return InternalTypeInfo.of(rowType); + } + + public static boolean isPhysicalColumn(TableColumn column) { + return column.isPhysical(); + } + + public static boolean isPhysicalColumn(Column column) { + return column.isPhysical(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java new file mode 100644 index 000000000000..353cee56bebb --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java @@ -0,0 +1,61 @@ +/* + * 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.util; + +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; + +public class FlinkPackage { + + private static final AtomicReference VERSION = new AtomicReference<>(); + public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION"; + + private FlinkPackage() {} + + /** Returns Flink version string like x.y.z */ + public static String version() { + if (null == VERSION.get()) { + String detectedVersion = null; + try { + detectedVersion = versionFromJar(); + // use unknown version in case exact implementation version can't be found from the jar + // (this can happen if the DataStream class appears multiple times in the same classpath + // such as with shading) + detectedVersion = detectedVersion != null ? detectedVersion : FLINK_UNKNOWN_VERSION; + } catch (Exception e) { + detectedVersion = FLINK_UNKNOWN_VERSION; + } + VERSION.set(detectedVersion); + } + + return VERSION.get(); + } + + @VisibleForTesting + static String versionFromJar() { + // Choose {@link DataStream} class because it is one of the core Flink API + return DataStream.class.getPackage().getImplementationVersion(); + } + + @VisibleForTesting + static void setVersion(String version) { + VERSION.set(version); + } +} diff --git a/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000000..29a9955a7e20 --- /dev/null +++ b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -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.flink.FlinkDynamicTableFactory diff --git a/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 000000000000..2b6bfa3cd579 --- /dev/null +++ b/flink/v1.18/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -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.flink.FlinkCatalogFactory diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java new file mode 100644 index 000000000000..4184526a6a1a --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java @@ -0,0 +1,90 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; + +public abstract class AvroGenericRecordConverterBase { + protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; + + @Test + public void testPrimitiveTypes() throws Exception { + testConverter(new DataGenerators.Primitives()); + } + + @Test + public void testStructOfPrimitive() throws Exception { + testConverter(new DataGenerators.StructOfPrimitive()); + } + + @Test + public void testStructOfArray() throws Exception { + testConverter(new DataGenerators.StructOfArray()); + } + + @Test + public void testStructOfMap() throws Exception { + testConverter(new DataGenerators.StructOfMap()); + } + + @Test + public void testStructOfStruct() throws Exception { + testConverter(new DataGenerators.StructOfStruct()); + } + + @Test + public void testArrayOfPrimitive() throws Exception { + testConverter(new DataGenerators.ArrayOfPrimitive()); + } + + @Test + public void testArrayOfArray() throws Exception { + testConverter(new DataGenerators.ArrayOfArray()); + } + + @Test + public void testArrayOfMap() throws Exception { + testConverter(new DataGenerators.ArrayOfMap()); + } + + @Test + public void testArrayOfStruct() throws Exception { + testConverter(new DataGenerators.ArrayOfStruct()); + } + + @Test + public void testMapOfPrimitives() throws Exception { + testConverter(new DataGenerators.MapOfPrimitives()); + } + + @Test + public void testMapOfArray() throws Exception { + testConverter(new DataGenerators.MapOfArray()); + } + + @Test + public void testMapOfMap() throws Exception { + testConverter(new DataGenerators.MapOfMap()); + } + + @Test + public void testMapOfStruct() throws Exception { + testConverter(new DataGenerators.MapOfStruct()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java new file mode 100644 index 000000000000..91ed3c4adea3 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.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.flink; + +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.flink.util.ArrayUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +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.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class CatalogTestBase extends TestBase { + + protected static final String DATABASE = "db"; + @TempDir protected File hiveWarehouse; + @TempDir protected File hadoopWarehouse; + + @Parameter(index = 0) + protected String catalogName; + + @Parameter(index = 1) + protected Namespace baseNamespace; + + protected Catalog validationCatalog; + protected SupportsNamespaces validationNamespaceCatalog; + protected Map config = Maps.newHashMap(); + + protected String flinkDatabase; + protected Namespace icebergNamespace; + protected boolean isHadoopCatalog; + + @Parameters(name = "catalogName={0}, baseNamespace={1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {"testhive", Namespace.empty()}, + new Object[] {"testhadoop", Namespace.empty()}, + new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); + } + + @BeforeEach + public void before() { + this.isHadoopCatalog = catalogName.startsWith("testhadoop"); + this.validationCatalog = + isHadoopCatalog + ? new HadoopCatalog(hiveConf, "file:" + hadoopWarehouse.getPath()) + : catalog; + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + + config.put("type", "iceberg"); + if (!baseNamespace.isEmpty()) { + config.put(FlinkCatalogFactory.BASE_NAMESPACE, baseNamespace.toString()); + } + if (isHadoopCatalog) { + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"); + } else { + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + config.put(CatalogProperties.URI, getURI(hiveConf)); + } + config.put(CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouseRoot())); + + this.flinkDatabase = catalogName + "." + DATABASE; + this.icebergNamespace = + Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE})); + sql("CREATE CATALOG %s WITH %s", catalogName, toWithClause(config)); + } + + @AfterEach + public void clean() { + dropCatalog(catalogName, true); + } + + protected String warehouseRoot() { + if (isHadoopCatalog) { + return hadoopWarehouse.getAbsolutePath(); + } else { + return hiveWarehouse.getAbsolutePath(); + } + } + + protected String getFullQualifiedTableName(String tableName) { + final List levels = Lists.newArrayList(icebergNamespace.levels()); + levels.add(tableName); + return Joiner.on('.').join(levels); + } + + static String getURI(HiveConf conf) { + return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); + } + + static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java new file mode 100644 index 000000000000..b1e3b20ff7ac --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; + +/** + * This interface defines test data generator. Different implementations for primitive and complex + * nested fields are defined in {@link DataGenerators}. + */ +public interface DataGenerator { + Schema icebergSchema(); + + RowType flinkRowType(); + + org.apache.avro.Schema avroSchema(); + + GenericRecord generateIcebergGenericRecord(); + + GenericRowData generateFlinkRowData(); + + org.apache.avro.generic.GenericRecord generateAvroGenericRecord(); +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java new file mode 100644 index 000000000000..e2cd411d7069 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java @@ -0,0 +1,1172 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import com.fasterxml.jackson.databind.node.IntNode; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.avro.LogicalTypes; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Days; + +/** + * Util class to generate test data with extensive coverage different field types: from primitives + * to complex nested types. + */ +public class DataGenerators { + + public static class Primitives implements DataGenerator { + private static final DateTime JODA_DATETIME_EPOC = + new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC); + private static final DateTime JODA_DATETIME_20220110 = + new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); + private static final int DAYS_BTW_EPOC_AND_20220110 = + Days.daysBetween(JODA_DATETIME_EPOC, JODA_DATETIME_20220110).getDays(); + private static final int HOUR_8_IN_MILLI = (int) TimeUnit.HOURS.toMillis(8); + + private static final LocalDate JAVA_LOCAL_DATE_20220110 = LocalDate.of(2022, 1, 10); + private static final LocalTime JAVA_LOCAL_TIME_HOUR8 = LocalTime.of(8, 0); + private static final OffsetDateTime JAVA_OFFSET_DATE_TIME_20220110 = + OffsetDateTime.of(2022, 1, 10, 0, 0, 0, 0, ZoneOffset.UTC); + private static final LocalDateTime JAVA_LOCAL_DATE_TIME_20220110 = + LocalDateTime.of(2022, 1, 10, 0, 0, 0); + private static final BigDecimal BIG_DECIMAL_NEGATIVE = new BigDecimal("-1.50"); + private static final byte[] FIXED_BYTES = "012345689012345".getBytes(StandardCharsets.UTF_8); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + // primitive types + Types.NestedField.optional(2, "boolean_field", Types.BooleanType.get()), + Types.NestedField.optional(3, "int_field", Types.IntegerType.get()), + Types.NestedField.optional(4, "long_field", Types.LongType.get()), + Types.NestedField.optional(5, "float_field", Types.FloatType.get()), + Types.NestedField.optional(6, "double_field", Types.DoubleType.get()), + Types.NestedField.required(7, "string_field", Types.StringType.get()), + Types.NestedField.required(8, "date_field", Types.DateType.get()), + Types.NestedField.required(9, "time_field", Types.TimeType.get()), + Types.NestedField.required(10, "ts_with_zone_field", Types.TimestampType.withZone()), + Types.NestedField.required( + 11, "ts_without_zone_field", Types.TimestampType.withoutZone()), + Types.NestedField.required(12, "uuid_field", Types.UUIDType.get()), + Types.NestedField.required(13, "binary_field", Types.BinaryType.get()), + Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2)), + Types.NestedField.required(15, "fixed_field", Types.FixedType.ofLength(16))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + /** + * Fix up Avro Schema that is converted from Iceberg Schema. + * + * @param schemaConvertedFromIceberg Avro Schema converted from Iceberg schema via {@link + * AvroSchemaUtil#convert(Schema, String)} + */ + private org.apache.avro.Schema fixupAvroSchemaConvertedFromIcebergSchema( + org.apache.avro.Schema schemaConvertedFromIceberg) { + List fixedFields = + schemaConvertedFromIceberg.getFields().stream() + .map( + field -> { + org.apache.avro.Schema.Field updatedField = field; + if (field.name().equals("time_field")) { + // Iceberg's AvroSchemaUtil uses timestamp-micros with Long value for time + // field, while AvroToRowDataConverters#convertToTime() always looks for + // Integer value assuming millis. The root problem is that + // AvroToRowDataConverters#createConverter() uses LogicalTypeRoot to + // determine converter and LogicalTypeRoot lost the timestamp precision + // carried by LogicalType like Time(6). + org.apache.avro.Schema fieldSchema = + LogicalTypes.timeMillis() + .addToSchema( + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); + updatedField = new org.apache.avro.Schema.Field("time_field", fieldSchema); + } + + return new org.apache.avro.Schema.Field(updatedField, updatedField.schema()); + }) + .collect(Collectors.toList()); + return org.apache.avro.Schema.createRecord( + schemaConvertedFromIceberg.getName(), + schemaConvertedFromIceberg.getDoc(), + schemaConvertedFromIceberg.getNamespace(), + schemaConvertedFromIceberg.isError(), + fixedFields); + } + + private final org.apache.avro.Schema avroSchema = + fixupAvroSchemaConvertedFromIcebergSchema(AvroSchemaUtil.convert(icebergSchema, "table")); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("boolean_field", false); + genericRecord.setField("int_field", Integer.MAX_VALUE); + genericRecord.setField("long_field", Long.MAX_VALUE); + genericRecord.setField("float_field", Float.MAX_VALUE); + genericRecord.setField("double_field", Double.MAX_VALUE); + genericRecord.setField("string_field", "str"); + + genericRecord.setField("date_field", JAVA_LOCAL_DATE_20220110); + genericRecord.setField("time_field", JAVA_LOCAL_TIME_HOUR8); + genericRecord.setField("ts_with_zone_field", JAVA_OFFSET_DATE_TIME_20220110); + genericRecord.setField("ts_without_zone_field", JAVA_LOCAL_DATE_TIME_20220110); + + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + + genericRecord.setField("uuid_field", UUID.nameUUIDFromBytes(uuidBytes)); + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + genericRecord.setField("binary_field", ByteBuffer.wrap(binaryBytes)); + + genericRecord.setField("decimal_field", BIG_DECIMAL_NEGATIVE); + genericRecord.setField("fixed_field", FIXED_BYTES); + + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + + return GenericRowData.of( + StringData.fromString("row_id_value"), + false, + Integer.MAX_VALUE, + Long.MAX_VALUE, + Float.MAX_VALUE, + Double.MAX_VALUE, + StringData.fromString("str"), + DAYS_BTW_EPOC_AND_20220110, + HOUR_8_IN_MILLI, + // Although Avro logical type for timestamp fields are in micro seconds, + // AvroToRowDataConverters only looks for long value in milliseconds. + TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), + TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), + uuidBytes, + binaryBytes, + DecimalData.fromBigDecimal(BIG_DECIMAL_NEGATIVE, 9, 2), + FIXED_BYTES); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", new Utf8("row_id_value")); + genericRecord.put("boolean_field", false); + genericRecord.put("int_field", Integer.MAX_VALUE); + genericRecord.put("long_field", Long.MAX_VALUE); + genericRecord.put("float_field", Float.MAX_VALUE); + genericRecord.put("double_field", Double.MAX_VALUE); + genericRecord.put("string_field", new Utf8("str")); + + genericRecord.put("date_field", DAYS_BTW_EPOC_AND_20220110); + genericRecord.put("time_field", HOUR_8_IN_MILLI); + // Although Avro logical type for timestamp fields are in micro seconds, + // AvroToRowDataConverters only looks for long value in milliseconds. + genericRecord.put("ts_with_zone_field", JODA_DATETIME_20220110.getMillis()); + genericRecord.put("ts_without_zone_field", JODA_DATETIME_20220110.getMillis()); + + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + genericRecord.put("uuid_field", ByteBuffer.wrap(uuidBytes)); + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + genericRecord.put("binary_field", ByteBuffer.wrap(binaryBytes)); + + BigDecimal bigDecimal = new BigDecimal("-1.50"); + // unscaledValue().toByteArray() is to match the behavior of RowDataToAvroConverters from + // Flink for decimal type + genericRecord.put("decimal_field", ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray())); + + genericRecord.put("fixed_field", ByteBuffer.wrap(FIXED_BYTES)); + + return genericRecord; + } + } + + public static class StructOfPrimitive implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_primitive", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required(102, "name", Types.StringType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_primitive").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("name", "Jane"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_primitive", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of(1, StringData.fromString("Jane"))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_primitive").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("name", "Jane"); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_primitive", struct); + return genericRecord; + } + } + + public static class StructOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_array", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, "names", Types.ListType.ofRequired(201, Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_array").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("names", Arrays.asList("Jane", "Joe")); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_array", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + StringData[] names = {StringData.fromString("Jane"), StringData.fromString("Joe")}; + return GenericRowData.of( + StringData.fromString("row_id_value"), GenericRowData.of(1, new GenericArrayData(names))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_array").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("names", Arrays.asList("Jane", "Joe")); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_array", struct); + return genericRecord; + } + } + + public static class StructOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_map", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, + "names", + Types.MapType.ofRequired( + 201, 202, Types.StringType.get(), Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_map").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("names", ImmutableMap.of("Jane", "female", "Joe", "male")); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_map", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of( + 1, + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), + StringData.fromString("female"), + StringData.fromString("Joe"), + StringData.fromString("male"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_map").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("names", ImmutableMap.of("Jane", new Utf8("female"), "Joe", new Utf8("male"))); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_map", struct); + return genericRecord; + } + } + + public static class StructOfStruct implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_struct", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, + "person_struct", + Types.StructType.of( + Types.NestedField.required(201, "name", Types.StringType.get()), + Types.NestedField.required(202, "address", Types.StringType.get())))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_struct").type().asStructType().fields()); + Schema personSchema = + new Schema(structSchema.findField("person_struct").type().asStructType().fields()); + GenericRecord person = GenericRecord.create(personSchema); + person.setField("name", "Jane"); + person.setField("address", "Apple Park"); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("person_struct", person); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_struct", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of( + 1, + GenericRowData.of( + StringData.fromString("Jane"), StringData.fromString("Apple Park")))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_struct").schema(); + org.apache.avro.Schema personSchema = structSchema.getField("person_struct").schema(); + org.apache.avro.generic.GenericRecord person = new GenericData.Record(personSchema); + person.put("name", "Jane"); + person.put("address", "Apple Park"); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("person_struct", person); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_struct", struct); + return genericRecord; + } + } + + public static class ArrayOfPrimitive implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, "array_of_int", Types.ListType.ofOptional(101, Types.IntegerType.get()))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("array_of_int", Arrays.asList(1, 2, 3)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + Integer[] arr = {1, 2, 3}; + return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(arr)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("array_of_int", Arrays.asList(1, 2, 3)); + return genericRecord; + } + } + + public static class ArrayOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "array_of_array", + Types.ListType.ofRequired( + 101, Types.ListType.ofRequired(201, Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + // non-primitive + Integer[] array1 = {1, 2, 3}; + Integer[] array2 = {4, 5, 6}; + GenericArrayData[] arrayOfArrays = { + new GenericArrayData(array1), new GenericArrayData(array2) + }; + return GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(arrayOfArrays)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); + return genericRecord; + } + } + + public static class ArrayOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "array_of_map", + Types.ListType.ofRequired( + 101, + Types.MapType.ofRequired( + 201, 202, Types.StringType.get(), Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "array_of_map", + Arrays.asList( + ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + GenericMapData[] array = { + new GenericMapData( + ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2)), + new GenericMapData( + ImmutableMap.of(StringData.fromString("Alice"), 3, StringData.fromString("Bob"), 4)) + }; + return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(array)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "array_of_map", + Arrays.asList( + ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); + return genericRecord; + } + } + + public static class ArrayOfStruct implements DataGenerator { + private final Types.StructType structType = + Types.StructType.of( + required(201, "id", Types.IntegerType.get()), + required(202, "name", Types.StringType.get())); + private final Schema structIcebergSchema = new Schema(structType.fields()); + private final org.apache.avro.Schema structAvroSchema = + AvroSchemaUtil.convert(structIcebergSchema, "struct"); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, "array_of_struct", Types.ListType.ofRequired(101, structType))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord struct1 = GenericRecord.create(structIcebergSchema); + struct1.setField("id", 1); + struct1.setField("name", "Jane"); + GenericRecord struct2 = GenericRecord.create(structIcebergSchema); + struct2.setField("id", 2); + struct2.setField("name", "Joe"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("array_of_struct", Arrays.asList(struct1, struct2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + GenericRowData[] structArray = { + GenericRowData.of(1, StringData.fromString("Jane")), + GenericRowData.of(2, StringData.fromString("Joe")) + }; + return GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(structArray)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); + struct1.put("id", 1); + struct1.put("name", "Jane"); + org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); + struct2.put("id", 2); + struct2.put("name", "Joe"); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("array_of_struct", Arrays.asList(struct1, struct2)); + return genericRecord; + } + } + + public static class MapOfPrimitives implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "map_of_primitives", + Types.MapType.ofRequired( + 101, 102, Types.StringType.get(), Types.IntegerType.get()))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); + return genericRecord; + } + } + + public static class MapOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_array", + Types.MapType.ofRequired( + 101, + 102, + Types.StringType.get(), + Types.ListType.ofRequired(201, Types.IntegerType.get())))); + + private final RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return rowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_array", + ImmutableMap.of( + "Jane", Arrays.asList(1, 2, 3), + "Joe", Arrays.asList(4, 5, 6))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + Integer[] janeArray = {1, 2, 3}; + Integer[] joeArray = {4, 5, 6}; + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), + new GenericArrayData(janeArray), + StringData.fromString("Joe"), + new GenericArrayData(joeArray)))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "map_of_array", + ImmutableMap.of( + "Jane", Arrays.asList(1, 2, 3), + "Joe", Arrays.asList(4, 5, 6))); + return genericRecord; + } + } + + public static class MapOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_map", + Types.MapType.ofRequired( + 101, + 102, + Types.StringType.get(), + Types.MapType.ofRequired( + 301, 302, Types.StringType.get(), Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_map", + ImmutableMap.of( + "female", ImmutableMap.of("Jane", 1, "Alice", 2), + "male", ImmutableMap.of("Joe", 3, "Bob", 4))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("female"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), 1, StringData.fromString("Alice"), 2)), + StringData.fromString("male"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Joe"), 3, StringData.fromString("Bob"), 4))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "map_of_map", + ImmutableMap.of( + "female", ImmutableMap.of("Jane", 1, "Alice", 2), + "male", ImmutableMap.of("Joe", 3, "Bob", 4))); + return genericRecord; + } + } + + public static class MapOfStruct implements DataGenerator { + private org.apache.avro.Schema createAvroSchemaIdField() { + org.apache.avro.Schema schema = SchemaBuilder.builder().intType(); + // this is needed to match the converter generated schema props + schema.addProp("field-id", IntNode.valueOf(201)); + return schema; + } + + private org.apache.avro.Schema createAvroSchemaNameField() { + org.apache.avro.Schema schema = SchemaBuilder.builder().stringType(); + // this is needed to match the converter generated schema props + schema.addProp("field-id", IntNode.valueOf(202)); + return schema; + } + + private final Types.StructType structType = + Types.StructType.of( + required(201, "id", Types.IntegerType.get()), + required(202, "name", Types.StringType.get())); + private final Schema structIcebergSchema = new Schema(structType.fields()); + + private final org.apache.avro.Schema structAvroSchema = + SchemaBuilder.builder() + .record("struct") + .fields() + .name("id") + .type(createAvroSchemaIdField()) + .noDefault() + .name("name") + .type(createAvroSchemaNameField()) + .noDefault() + .endRecord(); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_struct", + Types.MapType.ofRequired(101, 102, Types.StringType.get(), structType))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + // Can't use AvroSchemaUtil.convert otherwise the nested schema will have generated name like + // `r102` not the specified name like `struct`. + org.apache.avro.Schema avroSchema = + SchemaBuilder.builder() + .record("table") + .fields() + .requiredString("row_id") + .name("map_of_struct") + .type(SchemaBuilder.builder().map().values(structAvroSchema)) + .noDefault() + .endRecord(); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord struct1 = GenericRecord.create(structIcebergSchema); + struct1.setField("id", 1); + struct1.setField("name", "Jane"); + GenericRecord struct2 = GenericRecord.create(structIcebergSchema); + struct2.setField("id", 2); + struct2.setField("name", "Joe"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("struct1"), + GenericRowData.of(1, StringData.fromString("Jane")), + StringData.fromString("struct2"), + GenericRowData.of(2, StringData.fromString("Joe"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); + struct1.put("id", 1); + struct1.put("name", new Utf8("Jane")); + org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); + struct2.put("id", 2); + struct2.put("name", new Utf8("Joe")); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", new Utf8("row_id_value")); + genericRecord.put("map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); + return genericRecord; + } + } + + public static class MapOfStructStruct implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get()), + Types.NestedField.optional(202, "keyData", Types.StringType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()), + Types.NestedField.optional(204, "valueData", Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + throw new UnsupportedOperationException( + "Not applicable as Avro Map only support string key type"); + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + GenericRowData.of(1L, StringData.fromString("key_data")), + GenericRowData.of(1L, StringData.fromString("value_data"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + throw new UnsupportedOperationException("Avro Map only support string key type"); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java new file mode 100644 index 000000000000..8076e0ec76f8 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.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; + +import java.util.List; +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.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.rules.TemporaryFolder; + +public abstract class FlinkTestBase extends TestBaseUtils { + + @ClassRule + public static MiniClusterWithClientResource miniClusterResource = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeClass + public static void startMetastore() { + FlinkTestBase.metastore = new TestHiveMetastore(); + metastore.start(); + FlinkTestBase.hiveConf = metastore.hiveConf(); + FlinkTestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterClass + public static void stopMetastore() throws Exception { + metastore.stop(); + FlinkTestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java new file mode 100644 index 000000000000..d8e1325254d9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -0,0 +1,104 @@ +/* + * 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; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.UUID; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopCatalogExtension + implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, AfterEachCallback { + protected final String database; + protected final String tableName; + + protected Path temporaryFolder; + protected Catalog catalog; + protected CatalogLoader catalogLoader; + protected String warehouse; + protected TableLoader tableLoader; + + public HadoopCatalogExtension(String database, String tableName) { + this.database = database; + this.tableName = tableName; + } + + @Override + public void beforeAll(ExtensionContext context) throws Exception { + this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-"); + } + + @Override + public void afterAll(ExtensionContext context) throws Exception { + FileUtils.deleteDirectory(temporaryFolder.toFile()); + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + Assertions.assertThat(temporaryFolder).exists().isDirectory(); + this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); + this.catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + this.catalog = catalogLoader.loadCatalog(); + this.tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); + } + + @Override + public void afterEach(ExtensionContext context) throws Exception { + try { + catalog.dropTable(TableIdentifier.of(database, tableName)); + ((HadoopCatalog) catalog).close(); + tableLoader.close(); + } catch (Exception e) { + throw new RuntimeException("Failed to close catalog resource"); + } + } + + public TableLoader tableLoader() { + return tableLoader; + } + + public Catalog catalog() { + return catalog; + } + + public CatalogLoader catalogLoader() { + return catalogLoader; + } + + public String warehouse() { + return warehouse; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java new file mode 100644 index 000000000000..2b4694d3c268 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java @@ -0,0 +1,90 @@ +/* + * 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; + +import java.io.File; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.rules.ExternalResource; +import org.junit.rules.TemporaryFolder; + +public class HadoopCatalogResource extends ExternalResource { + protected final TemporaryFolder temporaryFolder; + protected final String database; + protected final String tableName; + + protected Catalog catalog; + protected CatalogLoader catalogLoader; + protected String warehouse; + protected TableLoader tableLoader; + + public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { + this.temporaryFolder = temporaryFolder; + this.database = database; + this.tableName = tableName; + } + + @Override + protected void before() throws Throwable { + File warehouseFile = temporaryFolder.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + // before variables + this.warehouse = "file:" + warehouseFile; + this.catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + this.catalog = catalogLoader.loadCatalog(); + this.tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); + } + + @Override + protected void after() { + try { + catalog.dropTable(TableIdentifier.of(database, tableName)); + ((HadoopCatalog) catalog).close(); + tableLoader.close(); + } catch (Exception e) { + throw new RuntimeException("Failed to close catalog resource"); + } + } + + public TableLoader tableLoader() { + return tableLoader; + } + + public Catalog catalog() { + return catalog; + } + + public CatalogLoader catalogLoader() { + return catalogLoader; + } + + public String warehouse() { + return warehouse; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java new file mode 100644 index 000000000000..2da6f74b91d8 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.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; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.rules.TemporaryFolder; + +public class HadoopTableResource extends HadoopCatalogResource { + private final Schema schema; + private final PartitionSpec partitionSpec; + + private Table table; + + public HadoopTableResource( + TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { + this(temporaryFolder, database, tableName, schema, null); + } + + public HadoopTableResource( + TemporaryFolder temporaryFolder, + String database, + String tableName, + Schema schema, + PartitionSpec partitionSpec) { + super(temporaryFolder, database, tableName); + this.schema = schema; + this.partitionSpec = partitionSpec; + } + + @Override + protected void before() throws Throwable { + super.before(); + if (partitionSpec == null) { + this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); + } else { + this.table = + catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); + } + tableLoader.open(); + } + + public Table table() { + return table; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java new file mode 100644 index 000000000000..399d7aaff64c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java @@ -0,0 +1,68 @@ +/* + * 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; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.util.MiniClusterWithClientResource; + +public class MiniClusterResource { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniClusterResource() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { + return new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } + + public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = + new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java new file mode 100644 index 000000000000..9a73b80e077d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.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.flink; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +public class MiniFlinkClusterExtension { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniFlinkClusterExtension() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterExtension createWithClassloaderCheckDisabled() { + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java new file mode 100644 index 000000000000..e532fb62615c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java @@ -0,0 +1,135 @@ +/* + * 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; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class RowDataConverter { + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); + + private RowDataConverter() {} + + public static RowData convert(Schema iSchema, Record record) { + return convert(iSchema.asStruct(), record); + } + + private static RowData convert(Types.StructType struct, Record record) { + GenericRowData rowData = new GenericRowData(struct.fields().size()); + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + + Type fieldType = field.type(); + rowData.setField(i, convert(fieldType, record.get(i))); + } + return rowData; + } + + private static Object convert(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + case FIXED: + return object; + case DATE: + return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); + case TIME: + // Iceberg's time is in microseconds, while flink's time is in milliseconds. + LocalTime localTime = (LocalTime) object; + return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); + } else { + return TimestampData.fromLocalDateTime((LocalDateTime) object); + } + case STRING: + return StringData.fromString((String) object); + case UUID: + UUID uuid = (UUID) object; + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + return bb.array(); + case BINARY: + ByteBuffer buffer = (ByteBuffer) object; + return Arrays.copyOfRange( + buffer.array(), + buffer.arrayOffset() + buffer.position(), + buffer.arrayOffset() + buffer.remaining()); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) type; + return DecimalData.fromBigDecimal( + (BigDecimal) object, decimalType.precision(), decimalType.scale()); + case STRUCT: + return convert(type.asStructType(), (Record) object); + case LIST: + List list = (List) object; + Object[] convertedArray = new Object[list.size()]; + for (int i = 0; i < convertedArray.length; i++) { + convertedArray[i] = convert(type.asListType().elementType(), list.get(i)); + } + return new GenericArrayData(convertedArray); + case MAP: + Map convertedMap = Maps.newLinkedHashMap(); + Map map = (Map) object; + for (Map.Entry entry : map.entrySet()) { + convertedMap.put( + convert(type.asMapType().keyType(), entry.getKey()), + convert(type.asMapType().valueType(), entry.getValue())); + } + return new GenericMapData(convertedMap); + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java new file mode 100644 index 000000000000..f48764f772b4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -0,0 +1,443 @@ +/* + * 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; + +import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +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.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; +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.EncryptedOutputFile; +import org.apache.iceberg.flink.sink.FlinkAppenderFactory; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; +import org.junit.Assert; + +public class SimpleDataUtil { + + private SimpleDataUtil() {} + + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + public static final TableSchema FLINK_SCHEMA = + TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); + + public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); + + public static final Record RECORD = GenericRecord.create(SCHEMA); + + public static Table createTable( + String path, Map properties, boolean partitioned) { + PartitionSpec spec; + if (partitioned) { + spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + return new HadoopTables().create(SCHEMA, spec, properties, path); + } + + public static Record createRecord(Integer id, String data) { + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + return record; + } + + public static RowData createRowData(Integer id, String data) { + return GenericRowData.of(id, StringData.fromString(data)); + } + + public static RowData createInsert(Integer id, String data) { + return GenericRowData.ofKind(RowKind.INSERT, id, StringData.fromString(data)); + } + + public static RowData createDelete(Integer id, String data) { + return GenericRowData.ofKind(RowKind.DELETE, id, StringData.fromString(data)); + } + + public static RowData createUpdateBefore(Integer id, String data) { + return GenericRowData.ofKind(RowKind.UPDATE_BEFORE, id, StringData.fromString(data)); + } + + public static RowData createUpdateAfter(Integer id, String data) { + return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); + } + + public static DataFile writeFile( + Table table, + Schema schema, + PartitionSpec spec, + Configuration conf, + String location, + String filename, + List rows) + throws IOException { + return writeFile(table, schema, spec, conf, location, filename, rows, null); + } + + /** Write the list of {@link RowData} to the given path and with the given partition data */ + public static DataFile writeFile( + Table table, + Schema schema, + PartitionSpec spec, + Configuration conf, + String location, + String filename, + List rows, + StructLike partition) + throws IOException { + Path path = new Path(location, filename); + FileFormat fileFormat = FileFormat.fromFileName(filename); + Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); + + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, schema, flinkSchema, ImmutableMap.of(), spec, null, null, null); + + FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); + try (FileAppender closeableAppender = appender) { + closeableAppender.addAll(rows); + } + + DataFiles.Builder builder = + DataFiles.builder(spec) + .withInputFile(HadoopInputFile.fromPath(path, conf)) + .withMetrics(appender.metrics()); + + if (partition != null) { + builder = builder.withPartition(partition); + } + + return builder.build(); + } + + public static DeleteFile writeEqDeleteFile( + Table table, + FileFormat format, + String filename, + FileAppenderFactory appenderFactory, + List deletes) + throws IOException { + EncryptedOutputFile outputFile = + table + .encryption() + .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); + + EqualityDeleteWriter eqWriter = + appenderFactory.newEqDeleteWriter(outputFile, format, null); + try (EqualityDeleteWriter writer = eqWriter) { + writer.write(deletes); + } + return eqWriter.toDeleteFile(); + } + + public static DeleteFile writePosDeleteFile( + Table table, + FileFormat format, + String filename, + FileAppenderFactory appenderFactory, + List> positions) + throws IOException { + EncryptedOutputFile outputFile = + table + .encryption() + .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); + + PositionDeleteWriter posWriter = + appenderFactory.newPosDeleteWriter(outputFile, format, null); + PositionDelete posDelete = PositionDelete.create(); + try (PositionDeleteWriter writer = posWriter) { + for (Pair p : positions) { + writer.write(posDelete.set(p.first(), p.second(), null)); + } + } + return posWriter.toDeleteFile(); + } + + private static List convertToRecords(List rows) { + List records = Lists.newArrayList(); + for (RowData row : rows) { + Integer id = row.isNullAt(0) ? null : row.getInt(0); + String data = row.isNullAt(1) ? null : row.getString(1).toString(); + records.add(createRecord(id, data)); + } + return records; + } + + public static void assertTableRows(String tablePath, List expected, String branch) + throws IOException { + assertTableRecords(tablePath, convertToRecords(expected), branch); + } + + public static void assertTableRows(Table table, List expected) throws IOException { + assertTableRecords(table, convertToRecords(expected), SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRows(Table table, List expected, String branch) + throws IOException { + assertTableRecords(table, convertToRecords(expected), branch); + } + + /** Get all rows for a table */ + public static List tableRecords(Table table) throws IOException { + table.refresh(); + List records = Lists.newArrayList(); + try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { + for (Record record : iterable) { + records.add(record); + } + } + return records; + } + + public static boolean equalsRecords(List expected, List actual, Schema schema) { + if (expected.size() != actual.size()) { + return false; + } + Types.StructType type = schema.asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + StructLikeSet actualSet = StructLikeSet.create(type); + actualSet.addAll(actual); + return expectedSet.equals(actualSet); + } + + public static void assertRecordsEqual(List expected, List actual, Schema schema) { + Assert.assertEquals(expected.size(), actual.size()); + Types.StructType type = schema.asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + StructLikeSet actualSet = StructLikeSet.create(type); + actualSet.addAll(actual); + Assert.assertEquals(expectedSet, actualSet); + } + + /** + * Assert table contains the expected list of records after waiting up to the configured {@code + * timeout} + */ + public static void assertTableRecords(Table table, List expected, Duration timeout) { + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + equalsRecords(expected, tableRecords(table), table.schema()); + assertRecordsEqual(expected, tableRecords(table), table.schema()); + }); + } + + public static void assertTableRecords(Table table, List expected) throws IOException { + assertTableRecords(table, expected, SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRecords(Table table, List expected, String branch) + throws IOException { + table.refresh(); + Snapshot snapshot = latestSnapshot(table, branch); + + if (snapshot == null) { + Assert.assertEquals(expected, ImmutableList.of()); + return; + } + + Types.StructType type = table.schema().asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + + try (CloseableIterable iterable = + IcebergGenerics.read(table).useSnapshot(snapshot.snapshotId()).build()) { + StructLikeSet actualSet = StructLikeSet.create(type); + + for (Record record : iterable) { + actualSet.add(record); + } + + Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + } + } + + // Returns the latest snapshot of the given branch in the table + public static Snapshot latestSnapshot(Table table, String branch) { + // For the main branch, currentSnapshot() is used to validate that the API behavior has + // not changed since that was the API used for validation prior to addition of branches. + if (branch.equals(SnapshotRef.MAIN_BRANCH)) { + return table.currentSnapshot(); + } + + return table.snapshot(branch); + } + + public static void assertTableRecords(String tablePath, List expected) + throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + assertTableRecords(new HadoopTables().load(tablePath), expected, SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRecords(String tablePath, List expected, String branch) + throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + assertTableRecords(new HadoopTables().load(tablePath), expected, branch); + } + + public static StructLikeSet expectedRowSet(Table table, Record... records) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + for (Record record : records) { + set.add(wrapper.copyFor(record)); + } + return set; + } + + public static StructLikeSet actualRowSet(Table table, String... columns) throws IOException { + return actualRowSet(table, null, columns); + } + + public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... columns) + throws IOException { + table.refresh(); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table) + .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) + .select(columns) + .build()) { + reader.forEach(record -> set.add(wrapper.copyFor(record))); + } + return set; + } + + public static List partitionDataFiles(Table table, Map partitionValues) + throws IOException { + table.refresh(); + Types.StructType partitionType = table.spec().partitionType(); + + Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); + StructLikeWrapper expectedWrapper = + StructLikeWrapper.forType(partitionType).set(partitionRecord); + + List dataFiles = Lists.newArrayList(); + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + StructLikeWrapper wrapper = + StructLikeWrapper.forType(partitionType).set(scanTask.file().partition()); + + if (expectedWrapper.equals(wrapper)) { + dataFiles.add(scanTask.file()); + } + } + } + + return dataFiles; + } + + public static Map> snapshotToDataFiles(Table table) throws IOException { + table.refresh(); + + Map> result = Maps.newHashMap(); + Snapshot current = table.currentSnapshot(); + while (current != null) { + TableScan tableScan = table.newScan(); + if (current.parentId() != null) { + // Collect the data files that was added only in current snapshot. + tableScan = tableScan.appendsBetween(current.parentId(), current.snapshotId()); + } else { + // Collect the data files that was added in the oldest snapshot. + tableScan = tableScan.useSnapshot(current.snapshotId()); + } + try (CloseableIterable scanTasks = tableScan.planFiles()) { + result.put( + current.snapshotId(), + ImmutableList.copyOf(Iterables.transform(scanTasks, FileScanTask::file))); + } + + // Continue to traverse the parent snapshot if exists. + if (current.parentId() == null) { + break; + } + // Iterate to the parent snapshot. + current = table.snapshot(current.parentId()); + } + return result; + } + + public static List matchingPartitions( + List dataFiles, PartitionSpec partitionSpec, Map partitionValues) { + Types.StructType partitionType = partitionSpec.partitionType(); + Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); + StructLikeWrapper expected = StructLikeWrapper.forType(partitionType).set(partitionRecord); + return dataFiles.stream() + .filter( + df -> { + StructLikeWrapper wrapper = + StructLikeWrapper.forType(partitionType).set(df.partition()); + return wrapper.equals(expected); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java new file mode 100644 index 000000000000..3986f1a796a5 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.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; + +import java.nio.file.Path; +import java.util.List; +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.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.TestBaseUtils; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBase extends TestBaseUtils { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir protected Path temporaryDirectory; + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeAll + public static void startMetastore() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + TestBase.catalog = null; + } + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + Assertions.assertThat(actual) + .isNotNull() + .as(message) + .containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG default_catalog"); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java new file mode 100644 index 000000000000..384ac5c52d00 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java @@ -0,0 +1,118 @@ +/* + * 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; + +import static org.apache.iceberg.CatalogProperties.URI; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +/** Test for {@link CatalogLoader}. */ +public class TestCatalogLoader extends FlinkTestBase { + + private static File warehouse = null; + private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); + private static final Schema SCHEMA = + new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); + + @BeforeClass + public static void createWarehouse() throws IOException { + warehouse = File.createTempFile("warehouse", null); + Assert.assertTrue(warehouse.delete()); + hiveConf.set("my_key", "my_value"); + } + + @AfterClass + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + } + } + + @Test + public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse); + CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties); + validateCatalogLoader(loader); + } + + @Test + public void testHiveCatalogLoader() throws IOException, ClassNotFoundException { + CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + validateCatalogLoader(loader); + } + + @Test + public void testRESTCatalogLoader() { + Map properties = Maps.newHashMap(); + properties.put(URI, "http://localhost/"); + CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap()); + } + + private static void validateCatalogLoader(CatalogLoader loader) + throws IOException, ClassNotFoundException { + Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); + validateHadoopConf(table); + } + + private static void validateHadoopConf(Table table) { + FileIO io = table.io(); + Assertions.assertThat(io) + .as("FileIO should be a HadoopFileIO") + .isInstanceOf(HadoopFileIO.class); + HadoopFileIO hadoopIO = (HadoopFileIO) io; + Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + } + + @SuppressWarnings("unchecked") + private static T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(object); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + return (T) in.readObject(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java new file mode 100644 index 000000000000..147d2a173ddc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.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; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +/** Test for {@link TableLoader}. */ +public class TestCatalogTableLoader extends TestBase { + + private static File warehouse = null; + private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); + private static final Schema SCHEMA = + new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); + + @BeforeAll + public static void createWarehouse() throws IOException { + warehouse = File.createTempFile("warehouse", null); + Assertions.assertThat(warehouse.delete()).isTrue(); + hiveConf.set("my_key", "my_value"); + } + + @AfterAll + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + Assertions.assertThat(fs.delete(warehousePath, true)) + .as("Failed to delete " + warehousePath) + .isTrue(); + } + } + + @Test + public void testHadoopTableLoader() throws IOException, ClassNotFoundException { + String location = "file:" + warehouse + "/my_table"; + new HadoopTables(hiveConf).create(SCHEMA, location); + validateTableLoader(TableLoader.fromHadoopTable(location, hiveConf)); + } + + @Test + public void testHiveCatalogTableLoader() throws IOException, ClassNotFoundException { + CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + javaSerdes(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); + + CatalogLoader catalogLoader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + validateTableLoader(TableLoader.fromCatalog(catalogLoader, IDENTIFIER)); + } + + private static void validateTableLoader(TableLoader loader) + throws IOException, ClassNotFoundException { + TableLoader copied = javaSerdes(loader); + copied.open(); + try { + validateHadoopConf(copied.loadTable()); + } finally { + copied.close(); + } + } + + private static void validateHadoopConf(Table table) { + FileIO io = table.io(); + Assertions.assertThat(io) + .as("FileIO should be a HadoopFileIO") + .isInstanceOf(HadoopFileIO.class); + HadoopFileIO hadoopIO = (HadoopFileIO) io; + Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); + } + + @SuppressWarnings("unchecked") + private static T javaSerdes(T object) throws IOException, ClassNotFoundException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(object); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + return (T) in.readObject(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java new file mode 100644 index 000000000000..8e9066e391c9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -0,0 +1,305 @@ +/* + * 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; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.flink.source.ChangeLogTableTestBase; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +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.Sets; +import org.apache.iceberg.util.StructLikeSet; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +/** + * In this test case, we mainly cover the impact of primary key selection, multiple operations + * within a single transaction, and multiple operations between different txn on the correctness of + * the data. + */ +@RunWith(Parameterized.class) +public class TestChangeLogTable extends ChangeLogTableTestBase { + private static final Configuration CONF = new Configuration(); + private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; + + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String TABLE_NAME = "test_table"; + private static String warehouse; + + private final boolean partitioned; + + @Parameterized.Parameters(name = "PartitionedTable={0}") + public static Iterable parameters() { + return ImmutableList.of(new Object[] {true}, new Object[] {false}); + } + + public TestChangeLogTable(boolean partitioned) { + this.partitioned = partitioned; + } + + @BeforeClass + public static void createWarehouse() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + warehouse = String.format("file:%s", warehouseFile); + } + + @Before + public void before() { + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + // Set the table.exec.sink.upsert-materialize=NONE, so that downstream operators will receive + // the + // records with the same order as the source operator, bypassing Flink's inferred shuffle. + getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); + } + + @After + @Override + public void clean() { + sql("DROP TABLE IF EXISTS %s", TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + BoundedTableFactory.clearDataSets(); + } + + @Test + public void testSqlChangeLogOnIdKey() throws Exception { + List> inputRowsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(1, "bbb"), + insertRow(2, "aaa"), + deleteRow(2, "aaa"), + insertRow(2, "bbb")), + ImmutableList.of( + updateBeforeRow(2, "bbb"), + updateAfterRow(2, "ccc"), + deleteRow(2, "ccc"), + insertRow(2, "ddd")), + ImmutableList.of( + deleteRow(1, "bbb"), + insertRow(1, "ccc"), + deleteRow(1, "ccc"), + insertRow(1, "ddd"))); + + List> expectedRecordsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "bbb")), + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "ddd")), + ImmutableList.of(insertRow(1, "ddd"), insertRow(2, "ddd"))); + + testSqlChangeLog( + TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); + } + + @Test + public void testChangeLogOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(1, "bbb"), + insertRow(2, "aaa")), + ImmutableList.of( + updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), + ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "ccc"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa")), + ImmutableList.of(insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(1, "ccc"), + insertRow(2, "aaa"), + insertRow(2, "ccc"))); + + testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); + } + + @Test + public void testChangeLogOnIdDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(1, "bbb"), + insertRow(2, "aaa")), + ImmutableList.of( + updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), + ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(1, "ccc"), + insertRow(2, "aaa"), + insertRow(2, "bbb"))); + + testSqlChangeLog( + TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); + } + + @Test + public void testPureInsertOnIdKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), + ImmutableList.of(insertRow(3, "ccc"), insertRow(4, "ddd")), + ImmutableList.of(insertRow(5, "eee"), insertRow(6, "fff"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), insertRow(2, "bbb"), insertRow(3, "ccc"), insertRow(4, "ddd")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(3, "ccc"), + insertRow(4, "ddd"), + insertRow(5, "eee"), + insertRow(6, "fff"))); + + testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); + } + + private static Record record(int id, String data) { + return SimpleDataUtil.createRecord(id, data); + } + + private Table createTable(String tableName, List key, boolean isPartitioned) { + String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : ""; + sql( + "CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) %s", + tableName, Joiner.on(',').join(key), partitionByCause); + + // Upgrade the iceberg table to format v2. + CatalogLoader loader = + CatalogLoader.hadoop( + "my_catalog", CONF, ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + Table table = loader.loadCatalog().loadTable(TableIdentifier.of(DATABASE_NAME, TABLE_NAME)); + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + return table; + } + + private void testSqlChangeLog( + String tableName, + List key, + List> inputRowsPerCheckpoint, + List> expectedRecordsPerCheckpoint) + throws Exception { + String dataId = BoundedTableFactory.registerDataSet(inputRowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + Assert.assertEquals( + "Should have the expected rows", + listJoin(inputRowsPerCheckpoint), + sql("SELECT * FROM %s", SOURCE_TABLE)); + + Table table = createTable(tableName, key, partitioned); + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + table.refresh(); + List snapshots = findValidSnapshots(table); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + Assert.assertEquals( + "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRows = expectedRecordsPerCheckpoint.get(i); + Assert.assertEquals( + "Should have the expected records for the checkpoint#" + i, + expectedRowSet(table, expectedRows), + actualRowSet(table, snapshotId)); + } + + if (expectedSnapshotNum > 0) { + Assert.assertEquals( + "Should have the expected rows in the final table", + Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), + Sets.newHashSet(sql("SELECT * FROM %s", tableName))); + } + } + + private List findValidSnapshots(Table table) { + List validSnapshots = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.allManifests(table.io()).stream() + .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + validSnapshots.add(snapshot); + } + } + return validSnapshots; + } + + private static StructLikeSet expectedRowSet(Table table, List rows) { + Record[] records = new Record[rows.size()]; + for (int i = 0; i < records.length; i++) { + records[i] = record((int) rows.get(i).getField(0), (String) rows.get(i).getField(1)); + } + return SimpleDataUtil.expectedRowSet(table, records); + } + + private static StructLikeSet actualRowSet(Table table, long snapshotId) throws IOException { + return SimpleDataUtil.actualRowSet(table, snapshotId, "*"); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java new file mode 100644 index 000000000000..8992cbd75187 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java @@ -0,0 +1,203 @@ +/* + * 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; + +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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestDataFileSerialization { + + private static final Schema DATE_SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec PARTITION_SPEC = + PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); + + private static final Map COLUMN_SIZES = Maps.newHashMap(); + private static final Map VALUE_COUNTS = Maps.newHashMap(); + private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); + private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); + private static final Map LOWER_BOUNDS = Maps.newHashMap(); + private static final Map UPPER_BOUNDS = Maps.newHashMap(); + + static { + COLUMN_SIZES.put(1, 2L); + COLUMN_SIZES.put(2, 3L); + VALUE_COUNTS.put(1, 5L); + VALUE_COUNTS.put(2, 3L); + VALUE_COUNTS.put(4, 2L); + NULL_VALUE_COUNTS.put(1, 0L); + NULL_VALUE_COUNTS.put(2, 2L); + NAN_VALUE_COUNTS.put(4, 1L); + LOWER_BOUNDS.put(1, longToBuffer(0L)); + UPPER_BOUNDS.put(1, longToBuffer(4L)); + } + + private static final Metrics METRICS = + new Metrics( + 5L, null, VALUE_COUNTS, NULL_VALUE_COUNTS, NAN_VALUE_COUNTS, LOWER_BOUNDS, UPPER_BOUNDS); + + private static final DataFile DATA_FILE = + DataFiles.builder(PARTITION_SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(1234) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withSplitOffsets(ImmutableList.of(4L)) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) + .withSortOrder(SortOrder.unsorted()) + .build(); + + private static final DeleteFile POS_DELETE_FILE = + FileMetadata.deleteFileBuilder(PARTITION_SPEC) + .ofPositionDeletes() + .withPath("/path/to/pos-delete.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .withRecordCount(23) + .build(); + + private static final DeleteFile EQ_DELETE_FILE = + FileMetadata.deleteFileBuilder(PARTITION_SPEC) + .ofEqualityDeletes(2, 3) + .withPath("/path/to/equality-delete.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .withRecordCount(23) + .withSortOrder(SortOrder.unsorted()) + .build(); + + @Test + public void testJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(DATA_FILE); + out.writeObject(DATA_FILE.copy()); + + out.writeObject(POS_DELETE_FILE); + out.writeObject(POS_DELETE_FILE.copy()); + + out.writeObject(EQ_DELETE_FILE); + out.writeObject(EQ_DELETE_FILE.copy()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); + } + + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a position DeleteFile").isInstanceOf(DeleteFile.class); + TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); + } + + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a equality DeleteFile").isInstanceOf(DeleteFile.class); + TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); + } + } + } + + @Test + public void testDataFileKryoSerialization() throws IOException { + KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + kryo.serialize(DATA_FILE, outputView); + kryo.serialize(DATA_FILE.copy(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + DataFile dataFile1 = kryo.deserialize(inputView); + DataFile dataFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(DATA_FILE, dataFile1); + TestHelpers.assertEquals(DATA_FILE, dataFile2); + } + + @Test + public void testDeleteFileKryoSerialization() throws IOException { + KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + kryo.serialize(POS_DELETE_FILE, outputView); + kryo.serialize(POS_DELETE_FILE.copy(), outputView); + + kryo.serialize(EQ_DELETE_FILE, outputView); + kryo.serialize(EQ_DELETE_FILE.copy(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + + DeleteFile posDeleteFile1 = kryo.deserialize(inputView); + DeleteFile posDeleteFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile1); + TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile2); + + DeleteFile eqDeleteFile1 = kryo.deserialize(inputView); + DeleteFile eqDeleteFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile1); + TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile2); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java new file mode 100644 index 000000000000..884ea2d1d3b1 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; + +public class TestFixtures { + + private TestFixtures() {} + + public static final Schema SCHEMA = + new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + public static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dt").bucket("id", 1).build(); + + public static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + public static final String DATABASE = "default"; + public static final String TABLE = "t"; + public static final String SINK_TABLE = "t_sink"; + + public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + + public static final Schema TS_SCHEMA = + new Schema( + required(1, "ts", Types.TimestampType.withoutZone()), + required(2, "str", Types.StringType.get())); + + public static final PartitionSpec TS_SPEC = + PartitionSpec.builderFor(TS_SCHEMA).hour("ts").build(); + + public static final RowType TS_ROW_TYPE = FlinkSchemaUtil.convert(TS_SCHEMA); +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java new file mode 100644 index 000000000000..2fbd7627efab --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.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; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.assertj.core.api.Assertions; +import org.awaitility.Awaitility; +import org.junit.Test; + +public class TestFlinkAnonymousTable extends FlinkTestBase { + + @Test + public void testWriteAnonymousTable() throws Exception { + File warehouseDir = TEMPORARY_FOLDER.newFolder(); + TableEnvironment tEnv = getTableEnv(); + Table table = + tEnv.from( + TableDescriptor.forConnector("datagen") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("number-of-rows", "3") + .build()); + + TableDescriptor descriptor = + TableDescriptor.forConnector("iceberg") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("catalog-name", "hadoop_test") + .option("catalog-type", "hadoop") + .option("catalog-database", "test_db") + .option("catalog-table", "test") + .option("warehouse", warehouseDir.getAbsolutePath()) + .build(); + + table.insertInto(descriptor).execute(); + Awaitility.await() + .atMost(3, TimeUnit.SECONDS) + .untilAsserted( + () -> + Assertions.assertThat( + warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + .exists()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java new file mode 100644 index 000000000000..f46d50a5f0ab --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.types.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkCatalogDatabase extends CatalogTestBase { + + @AfterEach + @Override + public void clean() { + sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @TestTemplate + public void testCreateNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should not already exist") + .isFalse(); + + sql("CREATE DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should exist") + .isTrue(); + + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should still exist") + .isTrue(); + + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should be dropped") + .isFalse(); + + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should be created") + .isTrue(); + } + + @TestTemplate + public void testDropEmptyDatabase() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + sql("DROP DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should have been dropped") + .isFalse(); + } + + @TestTemplate + public void testDropNonEmptyNamespace() { + assumeThat(isHadoopCatalog) + .as("Hadoop catalog throws IOException: Directory is not empty.") + .isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) + .as("Table should exist") + .isTrue(); + Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + .cause() + .isInstanceOf(DatabaseNotEmptyException.class) + .hasMessage( + String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); + sql("DROP TABLE %s.tl", flinkDatabase); + } + + @TestTemplate + public void testListTables() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + assertThat(sql("SHOW TABLES")).isEmpty(); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); + + List tables = sql("SHOW TABLES"); + assertThat(tables).hasSize(1); + assertThat("tl").as("Table name should match").isEqualTo(tables.get(0).getField(0)); + } + + @TestTemplate + public void testListNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + + List databases = sql("SHOW DATABASES"); + + if (isHadoopCatalog) { + assertThat(databases).hasSize(1); + assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); + if (!baseNamespace.isEmpty()) { + // test namespace not belongs to this catalog + validationNamespaceCatalog.createNamespace( + Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); + databases = sql("SHOW DATABASES"); + assertThat(databases).hasSize(1); + assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); + } + } else { + // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the + // creation for default + // database. See HiveMetaStore.HMSHandler.init. + assertThat(databases) + .as("Should have db database") + .anyMatch(d -> Objects.equals(d.getField(0), "db")); + } + } + + @TestTemplate + public void testCreateNamespaceWithMetadata() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("prop", "value"); + } + + @TestTemplate + public void testCreateNamespaceWithComment() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("comment", "namespace doc"); + } + + @TestTemplate + public void testCreateNamespaceWithLocation() throws Exception { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + + Path location = temporaryDirectory.getRoot(); + sql("CREATE DATABASE %s WITH ('location'='%s')", flinkDatabase, location); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("location", "file:" + location.getRoot()); + } + + @TestTemplate + public void testSetProperties() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + + Map defaultMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(defaultMetadata).doesNotContainKey("prop"); + sql("ALTER DATABASE %s SET ('prop'='value')", flinkDatabase); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("prop", "value"); + } + + @TestTemplate + public void testHadoopNotSupportMeta() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isTrue(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + Assertions.assertThatThrownBy( + () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + .cause() + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + String.format( + "Cannot create namespace %s: metadata is not supported", icebergNamespace)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java new file mode 100644 index 000000000000..4c9e95b8fa82 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -0,0 +1,119 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +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.Test; + +public class TestFlinkCatalogFactory { + + private Map props; + + @BeforeEach + public void before() { + props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "/tmp/location"); + } + + @Test + public void testCreateCatalogHive() { + String catalogName = "hiveCatalog"; + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); + } + + @Test + public void testCreateCatalogHadoop() { + String catalogName = "hadoopCatalog"; + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); + } + + @Test + public void testCreateCatalogCustom() { + String catalogName = "customCatalog"; + props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); + } + + @Test + public void testCreateCatalogCustomWithHiveCatalogTypeSet() { + String catalogName = "customCatalog"; + props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + + assertThatThrownBy( + () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot create catalog customCatalog, both catalog-type and catalog-impl are set"); + } + + @Test + public void testLoadCatalogUnknown() { + String catalogName = "unknownCatalog"; + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); + + assertThatThrownBy( + () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith("Unknown catalog-type: fooType"); + } + + public static class CustomHadoopCatalog extends HadoopCatalog { + + public CustomHadoopCatalog() {} + + public CustomHadoopCatalog(Configuration conf, String warehouseLocation) { + setConf(conf); + initialize( + "custom", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation)); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java new file mode 100644 index 000000000000..eaa92e32c49d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -0,0 +1,672 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.constraints.UniqueConstraint; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkCatalogTable extends CatalogTestBase { + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @AfterEach + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); + sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @TestTemplate + public void testGetTable() { + sql("CREATE TABLE tl(id BIGINT, strV STRING)"); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, "tl")); + Schema iSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "strV", Types.StringType.get())); + assertThat(table.schema().toString()) + .as("Should load the expected iceberg schema") + .isEqualTo(iSchema.toString()); + } + + @TestTemplate + public void testRenameTable() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support rename table").isFalse(); + final Schema tableSchema = + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); + validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); + sql("ALTER TABLE tl RENAME TO tl2"); + + assertThatThrownBy(() -> getTableEnv().from("tl")) + .isInstanceOf(ValidationException.class) + .hasMessage("Table `tl` was not found."); + + Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); + assertThat(tableSchema.asStruct()).isEqualTo(actualSchema.asStruct()); + } + + @TestTemplate + public void testCreateTable() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT)"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + CatalogTable catalogTable = catalogTable("tl"); + assertThat(catalogTable.getSchema()) + .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); + } + + @TestTemplate + public void testCreateTableWithPrimaryKey() throws Exception { + sql("CREATE TABLE tl(id BIGINT, data STRING, key STRING PRIMARY KEY NOT ENFORCED)"); + + Table table = table("tl"); + assertThat(table.schema().identifierFieldIds()) + .as("Should have the expected row key.") + .isEqualTo(Sets.newHashSet(table.schema().findField("key").fieldId())); + CatalogTable catalogTable = catalogTable("tl"); + Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); + assertThat(uniqueConstraintOptional).isPresent(); + assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("key"); + } + + @TestTemplate + public void testCreateTableWithMultiColumnsInPrimaryKey() throws Exception { + sql( + "CREATE TABLE tl(id BIGINT, data STRING, CONSTRAINT pk_constraint PRIMARY KEY(data, id) NOT ENFORCED)"); + + Table table = table("tl"); + assertThat(table.schema().identifierFieldIds()) + .as("Should have the expected RowKey") + .isEqualTo( + Sets.newHashSet( + table.schema().findField("id").fieldId(), + table.schema().findField("data").fieldId())); + CatalogTable catalogTable = catalogTable("tl"); + Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); + assertThat(uniqueConstraintOptional).isPresent(); + assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("id", "data"); + } + + @TestTemplate + public void testCreateTableIfNotExists() { + sql("CREATE TABLE tl(id BIGINT)"); + + // Assert that table does exist. + assertThat(table("tl")).isNotNull(); + + sql("DROP TABLE tl"); + Assertions.assertThatThrownBy(() -> table("tl")) + .isInstanceOf(NoSuchTableException.class) + .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); + + sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); + assertThat(table("tl").properties()).doesNotContainKey("key"); + + table("tl").updateProperties().set("key", "value").commit(); + assertThat(table("tl").properties()).containsEntry("key", "value"); + + sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); + assertThat(table("tl").properties()).containsEntry("key", "value"); + } + + @TestTemplate + public void testCreateTableLike() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT)"); + sql("CREATE TABLE tl2 LIKE tl"); + + Table table = table("tl2"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + CatalogTable catalogTable = catalogTable("tl2"); + assertThat(catalogTable.getSchema()) + .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); + } + + @TestTemplate + public void testCreateTableLocation() { + assumeThat(isHadoopCatalog) + .as("HadoopCatalog does not support creating table with location") + .isFalse(); + sql("CREATE TABLE tl(id BIGINT) WITH ('location'='file:///tmp/location')"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + assertThat(table.location()).isEqualTo("file:///tmp/location"); + } + + @TestTemplate + public void testCreatePartitionTable() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT, dt STRING) PARTITIONED BY(dt)"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + assertThat(table.spec()) + .isEqualTo(PartitionSpec.builderFor(table.schema()).identity("dt").build()); + CatalogTable catalogTable = catalogTable("tl"); + assertThat(catalogTable.getSchema()) + .isEqualTo( + TableSchema.builder() + .field("id", DataTypes.BIGINT()) + .field("dt", DataTypes.STRING()) + .build()); + assertThat(catalogTable.getPartitionKeys()).isEqualTo(Collections.singletonList("dt")); + } + + @TestTemplate + public void testCreateTableWithColumnComment() { + sql("CREATE TABLE tl(id BIGINT COMMENT 'comment - id', data STRING COMMENT 'comment - data')"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get(), "comment - id"), + Types.NestedField.optional(2, "data", Types.StringType.get(), "comment - data")) + .asStruct()); + } + + @TestTemplate + public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); + + Table table = table("tl"); + assertThat(((BaseTable) table).operations().current().formatVersion()).isEqualTo(2); + } + + @TestTemplate + public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); + + Table table = table("tl"); + TableOperations ops = ((BaseTable) table).operations(); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v1") + .isEqualTo(1); + sql("ALTER TABLE tl SET('format-version'='2')"); + assertThat(ops.refresh().formatVersion()) + .as("should update table to use format v2") + .isEqualTo(2); + } + + @TestTemplate + public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); + + Table table = table("tl"); + TableOperations ops = ((BaseTable) table).operations(); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) + .rootCause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot downgrade v2 table to v1"); + } + + @TestTemplate + public void testLoadTransformPartitionTable() throws TableNotExistException { + Schema schema = new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + schema, + PartitionSpec.builderFor(schema).bucket("id", 100).build()); + + CatalogTable catalogTable = catalogTable("tl"); + assertThat(catalogTable.getSchema()) + .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); + assertThat(catalogTable.getPartitionKeys()).isEmpty(); + } + + @TestTemplate + public void testAlterTableProperties() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); + Map properties = Maps.newHashMap(); + properties.put("oldK", "oldV"); + + // new + sql("ALTER TABLE tl SET('newK'='newV')"); + properties.put("newK", "newV"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + + // update old + sql("ALTER TABLE tl SET('oldK'='oldV2')"); + properties.put("oldK", "oldV2"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + + // remove property + sql("ALTER TABLE tl RESET('oldK')"); + properties.remove("oldK"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + } + + @TestTemplate + public void testAlterTableAddColumn() { + sql("CREATE TABLE tl(id BIGINT)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + sql("ALTER TABLE tl ADD (dt STRING)"); + Schema schemaAfter1 = table("tl").schema(); + assertThat(schemaAfter1.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Add multiple columns + sql("ALTER TABLE tl ADD (col1 STRING COMMENT 'comment for col1', col2 BIGINT)"); + Schema schemaAfter2 = table("tl").schema(); + assertThat(schemaAfter2.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional( + 3, "col1", Types.StringType.get(), "comment for col1"), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct()); + // Adding a required field should fail because Iceberg's SchemaUpdate does not allow + // incompatible changes. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); + + // Adding an existing field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Try to add a column `id` which already exists in the table."); + } + + @TestTemplate + public void testAlterTableDropColumn() { + sql("CREATE TABLE tl(id BIGINT, dt STRING, col1 STRING, col2 BIGINT)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct()); + sql("ALTER TABLE tl DROP (dt)"); + Schema schemaAfter1 = table("tl").schema(); + assertThat(schemaAfter1.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct()); + // Drop multiple columns + sql("ALTER TABLE tl DROP (col1, col2)"); + Schema schemaAfter2 = table("tl").schema(); + assertThat(schemaAfter2.asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + // Dropping an non-existing field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `foo` does not exist in the base table."); + + // Dropping an already-deleted field should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `dt` does not exist in the base table."); + } + + @TestTemplate + public void testAlterTableModifyColumnName() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + sql("ALTER TABLE tl RENAME dt TO data"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())) + .asStruct()); + } + + @TestTemplate + public void testAlterTableModifyColumnType() { + sql("CREATE TABLE tl(id INTEGER, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Promote type from Integer to Long + sql("ALTER TABLE tl MODIFY (id BIGINT)"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's + // validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column type: dt: string -> int"); + } + + @TestTemplate + public void testAlterTableModifyColumnNullability() { + sql("CREATE TABLE tl(id INTEGER NOT NULL, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Changing nullability from optional to required should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); + + // Set nullability from required to optional + sql("ALTER TABLE tl MODIFY (id INTEGER)"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + } + + @TestTemplate + public void testAlterTableModifyColumnPosition() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING FIRST)"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(1, "id", Types.LongType.get())) + .asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING AFTER id)"); + Schema schemaAfterAfter = table("tl").schema(); + assertThat(schemaAfterAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Modifying the position of a non-existing column should fail due to Flink's internal + // validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Try to modify a column `non_existing` which does not exist in the table."); + + // Moving a column after a non-existing column should fail due to Flink's internal validation. + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Referenced column `non_existing` by 'AFTER' does not exist in the table."); + } + + @TestTemplate + public void testAlterTableModifyColumnComment() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING COMMENT 'comment for dt field')"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional( + 2, "dt", Types.StringType.get(), "comment for dt field")) + .asStruct()); + } + + @TestTemplate + public void testAlterTableConstraint() { + sql("CREATE TABLE tl(id BIGINT NOT NULL, dt STRING NOT NULL, col1 STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct()); + assertThat(schemaBefore.identifierFieldNames()).isEmpty(); + sql("ALTER TABLE tl ADD (PRIMARY KEY (id) NOT ENFORCED)"); + Schema schemaAfterAdd = table("tl").schema(); + assertThat(schemaAfterAdd.identifierFieldNames()).containsExactly("id"); + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (dt) NOT ENFORCED)"); + Schema schemaAfterModify = table("tl").schema(); + assertThat(schemaAfterModify.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct()); + assertThat(schemaAfterModify.identifierFieldNames()).containsExactly("dt"); + // Composite primary key + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, dt) NOT ENFORCED)"); + Schema schemaAfterComposite = table("tl").schema(); + assertThat(schemaAfterComposite.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct()); + assertThat(schemaAfterComposite.identifierFieldNames()).containsExactlyInAnyOrder("id", "dt"); + // Setting an optional field as primary key should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Setting a composite key containing an optional field should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Dropping constraints is not supported yet + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(UnsupportedOperationException.class) + .hasRootCauseMessage("Unsupported table change: DropConstraint."); + } + + @TestTemplate + public void testRelocateTable() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support relocate table").isFalse(); + sql("CREATE TABLE tl(id BIGINT)"); + sql("ALTER TABLE tl SET('location'='file:///tmp/location')"); + assertThat(table("tl").location()).isEqualTo("file:///tmp/location"); + } + + @TestTemplate + public void testSetCurrentAndCherryPickSnapshotId() { + sql("CREATE TABLE tl(c1 INT, c2 STRING, c3 STRING) PARTITIONED BY (c1)"); + + Table table = table("tl"); + + DataFile fileA = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + DataFile fileB = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") // easy way to set partition data for now + .withRecordCount(1) + .build(); + DataFile replacementFile = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-a-replacement.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + table.newAppend().appendFile(fileA).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + + // stage an overwrite that replaces FILE_A + table.newReplacePartitions().addFile(replacementFile).stageOnly().commit(); + + Snapshot staged = Iterables.getLast(table.snapshots()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); + // add another append so that the original commit can't be fast-forwarded + table.newAppend().appendFile(fileB).commit(); + + // test cherry pick + sql("ALTER TABLE tl SET('cherry-pick-snapshot-id'='%s')", staged.snapshotId()); + validateTableFiles(table, fileB, replacementFile); + + // test set current snapshot + sql("ALTER TABLE tl SET('current-snapshot-id'='%s')", snapshotId); + validateTableFiles(table, fileA); + } + + private void validateTableFiles(Table tbl, DataFile... expectedFiles) { + tbl.refresh(); + Set expectedFilePaths = + Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Set actualFilePaths = + StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) + .map(FileScanTask::file) + .map(ContentFile::path) + .collect(Collectors.toSet()); + assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); + } + + private Table table(String name) { + return validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, name)); + } + + private CatalogTable catalogTable(String name) throws TableNotExistException { + return (CatalogTable) + getTableEnv() + .getCatalog(getTableEnv().getCurrentCatalog()) + .get() + .getTable(new ObjectPath(DATABASE, name)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java new file mode 100644 index 000000000000..b32be379caeb --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -0,0 +1,119 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkCatalogTablePartitions extends CatalogTestBase { + + private String tableName = "test_table"; + + @Parameter(index = 2) + private FileFormat format; + + @Parameter(index = 3) + private Boolean cacheEnabled; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, cacheEnabled={3}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean cacheEnabled : new Boolean[] {true, false}) { + for (Object[] catalogParams : CatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, cacheEnabled}); + } + } + } + return parameters; + } + + @Override + @BeforeEach + public void before() { + super.before(); + config.put(CatalogProperties.CACHE_ENABLED, String.valueOf(cacheEnabled)); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @AfterEach + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @TestTemplate + public void testListPartitionsWithUnpartitionedTable() { + sql( + "CREATE TABLE %s (id INT, data VARCHAR) with ('write.format.default'='%s')", + tableName, format.name()); + sql("INSERT INTO %s SELECT 1,'a'", tableName); + + ObjectPath objectPath = new ObjectPath(DATABASE, tableName); + FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); + Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) + .isInstanceOf(TableNotPartitionedException.class) + .hasMessageStartingWith("Table db.test_table in catalog") + .hasMessageEndingWith("is not partitioned."); + } + + @TestTemplate + public void testListPartitionsWithPartitionedTable() + throws TableNotExistException, TableNotPartitionedException { + sql( + "CREATE TABLE %s (id INT, data VARCHAR) PARTITIONED BY (data) " + + "with ('write.format.default'='%s')", + tableName, format.name()); + sql("INSERT INTO %s SELECT 1,'a'", tableName); + sql("INSERT INTO %s SELECT 2,'b'", tableName); + + ObjectPath objectPath = new ObjectPath(DATABASE, tableName); + FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); + List list = flinkCatalog.listPartitions(objectPath); + assertThat(list).hasSize(2); + List expected = Lists.newArrayList(); + CatalogPartitionSpec partitionSpec1 = new CatalogPartitionSpec(ImmutableMap.of("data", "a")); + CatalogPartitionSpec partitionSpec2 = new CatalogPartitionSpec(ImmutableMap.of("data", "b")); + expected.add(partitionSpec1); + expected.add(partitionSpec2); + assertThat(list).as("Should produce the expected catalog partition specs.").isEqualTo(expected); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java new file mode 100644 index 000000000000..4b6ac25ab8e3 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java @@ -0,0 +1,61 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestFlinkConfParser { + + @Test + public void testDurationConf() { + Map writeOptions = ImmutableMap.of("write-prop", "111s"); + + ConfigOption configOption = + ConfigOptions.key("conf-prop").durationType().noDefaultValue(); + Configuration flinkConf = new Configuration(); + flinkConf.setString(configOption.key(), "222s"); + + Table table = mock(Table.class); + when(table.properties()).thenReturn(ImmutableMap.of("table-prop", "333s")); + + FlinkConfParser confParser = new FlinkConfParser(table, writeOptions, flinkConf); + Duration defaultVal = Duration.ofMillis(999); + + Duration result = + confParser.durationConf().option("write-prop").defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(111)); + + result = confParser.durationConf().flinkConfig(configOption).defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(222)); + + result = confParser.durationConf().tableProperty("table-prop").defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(333)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java new file mode 100644 index 000000000000..838b0ea0e1a9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java @@ -0,0 +1,462 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.UnresolvedCallExpression; +import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.iceberg.expressions.And; +import org.apache.iceberg.expressions.BoundLiteralPredicate; +import org.apache.iceberg.expressions.Not; +import org.apache.iceberg.expressions.Or; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.Test; + +public class TestFlinkFilters { + + private static final TableSchema TABLE_SCHEMA = + TableSchema.builder() + .field("field1", DataTypes.INT()) + .field("field2", DataTypes.BIGINT()) + .field("field3", DataTypes.FLOAT()) + .field("field4", DataTypes.DOUBLE()) + .field("field5", DataTypes.STRING()) + .field("field6", DataTypes.BOOLEAN()) + .field("field7", DataTypes.BINARY(2)) + .field("field8", DataTypes.DECIMAL(10, 2)) + .field("field9", DataTypes.DATE()) + .field("field10", DataTypes.TIME()) + .field("field11", DataTypes.TIMESTAMP()) + .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .build(); + + // A map list of fields and values used to verify the conversion of flink expression to iceberg + // expression + private static final List> FIELD_VALUE_LIST = + ImmutableList.of( + Pair.of("field1", 1), + Pair.of("field2", 2L), + Pair.of("field3", 3F), + Pair.of("field4", 4D), + Pair.of("field5", "iceberg"), + Pair.of("field6", true), + Pair.of("field7", new byte[] {'a', 'b'}), + Pair.of("field8", BigDecimal.valueOf(10.12)), + Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())), + Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())), + Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())), + Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))); + + @Test + public void testFlinkDataTypeEqual() { + matchLiteral("field1", 1, 1); + matchLiteral("field2", 10L, 10L); + matchLiteral("field3", 1.2F, 1.2F); + matchLiteral("field4", 3.4D, 3.4D); + matchLiteral("field5", "abcd", "abcd"); + matchLiteral("field6", true, true); + matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'})); + matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12)); + + LocalDate date = LocalDate.parse("2020-12-23"); + matchLiteral("field9", date, DateTimeUtil.daysFromDate(date)); + + LocalTime time = LocalTime.parse("12:13:14"); + matchLiteral("field10", time, DateTimeUtil.microsFromTime(time)); + + LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14"); + matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime)); + + Instant instant = Instant.parse("2020-12-23T12:13:14.00Z"); + matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant)); + } + + @Test + public void testEquals() { + for (Pair pair : FIELD_VALUE_LIST) { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second()); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + } + + @Test + public void testEqualsNaN() { + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNaN("field3"); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testNotEquals() { + for (Pair pair : FIELD_VALUE_LIST) { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second()); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + } + + @Test + public void testNotEqualsNaN() { + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNaN("field3"); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testGreaterThan() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testGreaterThanEquals() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testLessThan() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.lessThan("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testLessThanEquals() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testIsNull() { + Expression expr = resolve(Expressions.$("field1").isNull()); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); + assertPredicatesMatch(expected, actual.get()); + } + + @Test + public void testIsNotNull() { + Expression expr = resolve(Expressions.$("field1").isNotNull()); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.notNull("field1"); + assertPredicatesMatch(expected, actual.get()); + } + + @Test + public void testAnd() { + Expression expr = + resolve( + Expressions.$("field1") + .isEqual(Expressions.lit(1)) + .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + And and = (And) actual.get(); + And expected = + (And) + org.apache.iceberg.expressions.Expressions.and( + org.apache.iceberg.expressions.Expressions.equal("field1", 1), + org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); + + assertPredicatesMatch(expected.left(), and.left()); + assertPredicatesMatch(expected.right(), and.right()); + } + + @Test + public void testOr() { + Expression expr = + resolve( + Expressions.$("field1") + .isEqual(Expressions.lit(1)) + .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + Or or = (Or) actual.get(); + Or expected = + (Or) + org.apache.iceberg.expressions.Expressions.or( + org.apache.iceberg.expressions.Expressions.equal("field1", 1), + org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); + + assertPredicatesMatch(expected.left(), or.left()); + assertPredicatesMatch(expected.right(), or.right()); + } + + @Test + public void testNot() { + Expression expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.NOT, + Expressions.$("field1").isEqual(Expressions.lit(1)))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + Not not = (Not) actual.get(); + Not expected = + (Not) + org.apache.iceberg.expressions.Expressions.not( + org.apache.iceberg.expressions.Expressions.equal("field1", 1)); + + assertThat(not.op()).as("Predicate operation should match").isEqualTo(expected.op()); + assertPredicatesMatch(expected.child(), not.child()); + } + + @Test + public void testLike() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc"); + Expression expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, + Expressions.$("field5"), + Expressions.lit("%abc%"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, + Expressions.$("field5"), + Expressions.lit("abc%d"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + } + + @SuppressWarnings("unchecked") + private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { + Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + org.apache.iceberg.expressions.Expression expression = actual.get(); + assertThat(expression) + .as("The expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + UnboundPredicate unboundPredicate = (UnboundPredicate) expression; + + org.apache.iceberg.expressions.Expression expression1 = + unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); + assertThat(expression1) + .as("The expression should be a BoundLiteralPredicate") + .isInstanceOf(BoundLiteralPredicate.class); + + BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; + assertThat(predicate.test(icebergLiteral)).isTrue(); + } + + private static Expression resolve(Expression originalExpression) { + return originalExpression.accept( + new ApiExpressionDefaultVisitor() { + @Override + public Expression visit(UnresolvedReferenceExpression unresolvedReference) { + String name = unresolvedReference.getName(); + Optional field = TABLE_SCHEMA.getTableColumn(name); + if (field.isPresent()) { + int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get()); + return new FieldReferenceExpression(name, field.get().getType(), 0, index); + } else { + return null; + } + } + + @Override + public Expression visit(UnresolvedCallExpression unresolvedCall) { + List children = + unresolvedCall.getChildren().stream() + .map(e -> (ResolvedExpression) e.accept(this)) + .collect(Collectors.toList()); + return new CallExpression( + unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); + } + + @Override + public Expression visit(ValueLiteralExpression valueLiteral) { + return valueLiteral; + } + + @Override + protected Expression defaultMethod(Expression expression) { + throw new UnsupportedOperationException( + String.format("unsupported expression: %s", expression)); + } + }); + } + + private void assertPredicatesMatch( + org.apache.iceberg.expressions.Expression expected, + org.apache.iceberg.expressions.Expression actual) { + assertThat(expected) + .as("The expected expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + assertThat(actual) + .as("The actual expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + UnboundPredicate predicateExpected = (UnboundPredicate) expected; + UnboundPredicate predicateActual = (UnboundPredicate) actual; + assertThat(predicateActual.op()).isEqualTo(predicateExpected.op()); + assertThat(predicateActual.literal()).isEqualTo(predicateExpected.literal()); + assertThat(predicateActual.ref().name()).isEqualTo(predicateExpected.ref().name()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java new file mode 100644 index 000000000000..47ee2afceb02 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.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.flink; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkHiveCatalog extends FlinkTestBase { + + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void testCreateCatalogWithWarehouseLocation() throws IOException { + Map props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + + File warehouseDir = tempFolder.newFolder(); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); + + checkSQLQuery(props, warehouseDir); + } + + @Test + public void testCreateCatalogWithHiveConfDir() throws IOException { + // Dump the hive conf into a local file. + File hiveConfDir = tempFolder.newFolder(); + File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); + File warehouseDir = tempFolder.newFolder(); + try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { + Configuration newConf = new Configuration(hiveConf); + // Set another new directory which is different with the hive metastore's warehouse path. + newConf.set( + HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); + newConf.writeXml(fos); + } + Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); + + // Construct the catalog attributions. + Map props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + // Set the 'hive-conf-dir' instead of 'warehouse' + props.put(FlinkCatalogFactory.HIVE_CONF_DIR, hiveConfDir.getAbsolutePath()); + + checkSQLQuery(props, warehouseDir); + } + + private void checkSQLQuery(Map catalogProperties, File warehouseDir) + throws IOException { + sql("CREATE CATALOG test_catalog WITH %s", CatalogTestBase.toWithClause(catalogProperties)); + sql("USE CATALOG test_catalog"); + sql("CREATE DATABASE test_db"); + sql("USE test_db"); + sql("CREATE TABLE test_table(c1 INT, c2 STRING)"); + sql("INSERT INTO test_table SELECT 1, 'a'"); + + Path databasePath = warehouseDir.toPath().resolve("test_db.db"); + Assert.assertTrue("Database path should exist", Files.exists(databasePath)); + + Path tablePath = databasePath.resolve("test_table"); + Assert.assertTrue("Table path should exist", Files.exists(tablePath)); + + Path dataPath = tablePath.resolve("data"); + Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); + Assert.assertEquals( + "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); + + sql("DROP TABLE test_table"); + sql("DROP DATABASE test_db"); + dropCatalog("test_catalog", false); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java new file mode 100644 index 000000000000..eab60d886ada --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -0,0 +1,416 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +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.junit.jupiter.api.Test; + +public class TestFlinkSchemaUtil { + + @Test + public void testConvertFlinkSchemaToIcebergSchema() { + TableSchema flinkSchema = + TableSchema.builder() + .field("id", DataTypes.INT().notNull()) + .field("name", DataTypes.STRING()) /* optional by default */ + .field("salary", DataTypes.DOUBLE().notNull()) + .field( + "locations", + DataTypes.MAP( + DataTypes.STRING(), + DataTypes.ROW( + DataTypes.FIELD("posX", DataTypes.DOUBLE().notNull(), "X field"), + DataTypes.FIELD("posY", DataTypes.DOUBLE().notNull(), "Y field")))) + .field("strArray", DataTypes.ARRAY(DataTypes.STRING()).nullable()) + .field("intArray", DataTypes.ARRAY(DataTypes.INT()).nullable()) + .field("char", DataTypes.CHAR(10).notNull()) + .field("varchar", DataTypes.VARCHAR(10).notNull()) + .field("boolean", DataTypes.BOOLEAN().nullable()) + .field("tinyint", DataTypes.TINYINT()) + .field("smallint", DataTypes.SMALLINT()) + .field("bigint", DataTypes.BIGINT()) + .field("varbinary", DataTypes.VARBINARY(10)) + .field("binary", DataTypes.BINARY(10)) + .field("time", DataTypes.TIME()) + .field("timestampWithoutZone", DataTypes.TIMESTAMP()) + .field("timestampWithZone", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("date", DataTypes.DATE()) + .field("decimal", DataTypes.DECIMAL(2, 2)) + .field("decimal2", DataTypes.DECIMAL(38, 2)) + .field("decimal3", DataTypes.DECIMAL(10, 1)) + .field("multiset", DataTypes.MULTISET(DataTypes.STRING().notNull())) + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get(), null), + Types.NestedField.optional(1, "name", Types.StringType.get(), null), + Types.NestedField.required(2, "salary", Types.DoubleType.get(), null), + Types.NestedField.optional( + 3, + "locations", + Types.MapType.ofOptional( + 24, + 25, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(22, "posX", Types.DoubleType.get(), "X field"), + Types.NestedField.required( + 23, "posY", Types.DoubleType.get(), "Y field")))), + Types.NestedField.optional( + 4, "strArray", Types.ListType.ofOptional(26, Types.StringType.get())), + Types.NestedField.optional( + 5, "intArray", Types.ListType.ofOptional(27, Types.IntegerType.get())), + Types.NestedField.required(6, "char", Types.StringType.get()), + Types.NestedField.required(7, "varchar", Types.StringType.get()), + Types.NestedField.optional(8, "boolean", Types.BooleanType.get()), + Types.NestedField.optional(9, "tinyint", Types.IntegerType.get()), + Types.NestedField.optional(10, "smallint", Types.IntegerType.get()), + Types.NestedField.optional(11, "bigint", Types.LongType.get()), + Types.NestedField.optional(12, "varbinary", Types.BinaryType.get()), + Types.NestedField.optional(13, "binary", Types.FixedType.ofLength(10)), + Types.NestedField.optional(14, "time", Types.TimeType.get()), + Types.NestedField.optional( + 15, "timestampWithoutZone", Types.TimestampType.withoutZone()), + Types.NestedField.optional(16, "timestampWithZone", Types.TimestampType.withZone()), + Types.NestedField.optional(17, "date", Types.DateType.get()), + Types.NestedField.optional(18, "decimal", Types.DecimalType.of(2, 2)), + Types.NestedField.optional(19, "decimal2", Types.DecimalType.of(38, 2)), + Types.NestedField.optional(20, "decimal3", Types.DecimalType.of(10, 1)), + Types.NestedField.optional( + 21, + "multiset", + Types.MapType.ofRequired(28, 29, Types.StringType.get(), Types.IntegerType.get()))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testMapField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "map_int_long", + DataTypes.MAP(DataTypes.INT(), DataTypes.BIGINT()).notNull()) /* Required */ + .field( + "map_int_array_string", + DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING())) + .field( + "map_decimal_string", DataTypes.MAP(DataTypes.DECIMAL(10, 2), DataTypes.STRING())) + .field( + "map_fields_fields", + DataTypes.MAP( + DataTypes.ROW( + DataTypes.FIELD("field_int", DataTypes.INT(), "doc - int"), + DataTypes.FIELD("field_string", DataTypes.STRING(), "doc - string")) + .notNull(), /* Required */ + DataTypes.ROW( + DataTypes.FIELD( + "field_array", + DataTypes.ARRAY(DataTypes.STRING()), + "doc - array")) + .notNull() /* Required */) + .notNull() /* Required */) + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "map_int_long", + Types.MapType.ofOptional(4, 5, Types.IntegerType.get(), Types.LongType.get()), + null), + Types.NestedField.optional( + 1, + "map_int_array_string", + Types.MapType.ofOptional( + 7, + 8, + Types.ListType.ofOptional(6, Types.IntegerType.get()), + Types.StringType.get()), + null), + Types.NestedField.optional( + 2, + "map_decimal_string", + Types.MapType.ofOptional( + 9, 10, Types.DecimalType.of(10, 2), Types.StringType.get())), + Types.NestedField.required( + 3, + "map_fields_fields", + Types.MapType.ofRequired( + 15, + 16, + Types.StructType.of( + Types.NestedField.optional( + 11, "field_int", Types.IntegerType.get(), "doc - int"), + Types.NestedField.optional( + 12, "field_string", Types.StringType.get(), "doc - string")), + Types.StructType.of( + Types.NestedField.optional( + 14, + "field_array", + Types.ListType.ofOptional(13, Types.StringType.get()), + "doc - array"))))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testStructField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "struct_int_string_decimal", + DataTypes.ROW( + DataTypes.FIELD("field_int", DataTypes.INT()), + DataTypes.FIELD("field_string", DataTypes.STRING()), + DataTypes.FIELD("field_decimal", DataTypes.DECIMAL(19, 2)), + DataTypes.FIELD( + "field_struct", + DataTypes.ROW( + DataTypes.FIELD("inner_struct_int", DataTypes.INT()), + DataTypes.FIELD( + "inner_struct_float_array", + DataTypes.ARRAY(DataTypes.FLOAT()))) + .notNull()) /* Row is required */) + .notNull()) /* Required */ + .field( + "struct_map_int_int", + DataTypes.ROW( + DataTypes.FIELD( + "field_map", DataTypes.MAP(DataTypes.INT(), DataTypes.INT()))) + .nullable()) /* Optional */ + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "struct_int_string_decimal", + Types.StructType.of( + Types.NestedField.optional(5, "field_int", Types.IntegerType.get()), + Types.NestedField.optional(6, "field_string", Types.StringType.get()), + Types.NestedField.optional(7, "field_decimal", Types.DecimalType.of(19, 2)), + Types.NestedField.required( + 8, + "field_struct", + Types.StructType.of( + Types.NestedField.optional( + 3, "inner_struct_int", Types.IntegerType.get()), + Types.NestedField.optional( + 4, + "inner_struct_float_array", + Types.ListType.ofOptional(2, Types.FloatType.get())))))), + Types.NestedField.optional( + 1, + "struct_map_int_int", + Types.StructType.of( + Types.NestedField.optional( + 11, + "field_map", + Types.MapType.ofOptional( + 9, 10, Types.IntegerType.get(), Types.IntegerType.get()))))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testListField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "list_struct_fields", + DataTypes.ARRAY(DataTypes.ROW(DataTypes.FIELD("field_int", DataTypes.INT()))) + .notNull()) /* Required */ + .field( + "list_optional_struct_fields", + DataTypes.ARRAY( + DataTypes.ROW( + DataTypes.FIELD( + "field_timestamp_with_local_time_zone", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()))) + .nullable()) /* Optional */ + .field( + "list_map_fields", + DataTypes.ARRAY( + DataTypes.MAP( + DataTypes.ARRAY( + DataTypes.INT().notNull()), /* Key of map must be required */ + DataTypes.ROW( + DataTypes.FIELD("field_0", DataTypes.INT(), "doc - int"))) + .notNull()) + .notNull()) /* Required */ + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "list_struct_fields", + Types.ListType.ofOptional( + 4, + Types.StructType.of( + Types.NestedField.optional(3, "field_int", Types.IntegerType.get())))), + Types.NestedField.optional( + 1, + "list_optional_struct_fields", + Types.ListType.ofOptional( + 6, + Types.StructType.of( + Types.NestedField.optional( + 5, + "field_timestamp_with_local_time_zone", + Types.TimestampType.withZone())))), + Types.NestedField.required( + 2, + "list_map_fields", + Types.ListType.ofRequired( + 11, + Types.MapType.ofOptional( + 9, + 10, + Types.ListType.ofRequired(7, Types.IntegerType.get()), + Types.StructType.of( + Types.NestedField.optional( + 8, "field_0", Types.IntegerType.get(), "doc - int")))))); + + checkSchema(flinkSchema, icebergSchema); + } + + private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { + assertThat(FlinkSchemaUtil.convert(flinkSchema).asStruct()).isEqualTo(icebergSchema.asStruct()); + // The conversion is not a 1:1 mapping, so we just check iceberg types. + assertThat( + FlinkSchemaUtil.convert( + FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) + .asStruct()) + .isEqualTo(icebergSchema.asStruct()); + } + + @Test + public void testInconsistentTypes() { + checkInconsistentType( + Types.UUIDType.get(), new BinaryType(16), new BinaryType(16), Types.FixedType.ofLength(16)); + checkInconsistentType( + Types.StringType.get(), + new VarCharType(VarCharType.MAX_LENGTH), + new CharType(100), + Types.StringType.get()); + checkInconsistentType( + Types.BinaryType.get(), + new VarBinaryType(VarBinaryType.MAX_LENGTH), + new VarBinaryType(100), + Types.BinaryType.get()); + checkInconsistentType( + Types.TimeType.get(), new TimeType(), new TimeType(3), Types.TimeType.get()); + checkInconsistentType( + Types.TimestampType.withoutZone(), + new TimestampType(6), + new TimestampType(3), + Types.TimestampType.withoutZone()); + checkInconsistentType( + Types.TimestampType.withZone(), + new LocalZonedTimestampType(6), + new LocalZonedTimestampType(3), + Types.TimestampType.withZone()); + } + + private void checkInconsistentType( + Type icebergType, + LogicalType flinkExpectedType, + LogicalType flinkType, + Type icebergExpectedType) { + assertThat(FlinkSchemaUtil.convert(icebergType)).isEqualTo(flinkExpectedType); + assertThat(FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()) + .isEqualTo(Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType))); + } + + @Test + public void testConvertFlinkSchemaBaseOnIcebergSchema() { + Schema baseSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required(101, "int", Types.IntegerType.get()), + Types.NestedField.optional(102, "string", Types.StringType.get())), + Sets.newHashSet(101)); + + TableSchema flinkSchema = + TableSchema.builder() + .field("int", DataTypes.INT().notNull()) + .field("string", DataTypes.STRING().nullable()) + .primaryKey("int") + .build(); + Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); + assertThat(convertedSchema.asStruct()).isEqualTo(baseSchema.asStruct()); + assertThat(convertedSchema.identifierFieldIds()).containsExactly(101); + } + + @Test + public void testConvertFlinkSchemaWithPrimaryKeys() { + Schema icebergSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "int", Types.IntegerType.get()), + Types.NestedField.required(2, "string", Types.StringType.get())), + Sets.newHashSet(1, 2)); + + TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); + assertThat(tableSchema.getPrimaryKey()) + .isPresent() + .get() + .satisfies(k -> assertThat(k.getColumns()).containsExactly("int", "string")); + } + + @Test + public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { + Schema icebergSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required( + 1, + "struct", + Types.StructType.of( + Types.NestedField.required(2, "inner", Types.IntegerType.get())))), + Sets.newHashSet(2)); + + assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Could not create a PRIMARY KEY") + .hasMessageContaining("Column 'struct.inner' does not exist."); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java new file mode 100644 index 000000000000..b7fce104f490 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -0,0 +1,358 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +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.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +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; + + @Parameter(index = 2) + private FileFormat format; + + @Parameter(index = 3) + private boolean isStreamingJob; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + public static List parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + for (Object[] catalogParams : CatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + } + } + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + } + return tEnv; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE_NAME, format.name()); + icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + BoundedTableFactory.clearDataSets(); + super.clean(); + } + + @TestTemplate + public void testInsertFromSourceTable() throws Exception { + // Register the rows into a temporary table. + getTableEnv() + .createTemporaryView( + "sourceTable", + getTableEnv() + .fromValues( + SimpleDataUtil.FLINK_SCHEMA.toRowDataType(), + Expressions.row(1, "hello"), + Expressions.row(2, "world"), + Expressions.row(3, (String) null), + Expressions.row(null, "bar"))); + + // Redirect the records from source table to destination table. + sql("INSERT INTO %s SELECT id,data from sourceTable", TABLE_NAME); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), + SimpleDataUtil.createRecord(2, "world"), + SimpleDataUtil.createRecord(3, null), + SimpleDataUtil.createRecord(null, "bar"))); + } + + @TestTemplate + public void testOverwriteTable() throws Exception { + assumeThat(isStreamingJob) + .as("Flink unbounded streaming does not support overwrite operation") + .isFalse(); + + sql("INSERT INTO %s SELECT 1, 'a'", TABLE_NAME); + SimpleDataUtil.assertTableRecords( + icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(1, "a"))); + + sql("INSERT OVERWRITE %s SELECT 2, 'b'", TABLE_NAME); + SimpleDataUtil.assertTableRecords( + icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); + } + + @TestTemplate + public void testWriteParallelism() throws Exception { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + 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 /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", + TABLE_NAME, 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)); + } + + @TestTemplate + public void testReplacePartitions() throws Exception { + assumeThat(isStreamingJob) + .as("Flink unbounded streaming does not support overwrite operation") + .isFalse(); + String tableName = "test_partition"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", + tableName, format.name()); + + try { + Table partitionedTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + + sql("INSERT INTO %s SELECT 1, 'a'", tableName); + sql("INSERT INTO %s SELECT 2, 'b'", tableName); + sql("INSERT INTO %s SELECT 3, 'c'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"))); + + sql("INSERT OVERWRITE %s SELECT 4, 'b'", tableName); + sql("INSERT OVERWRITE %s SELECT 5, 'a'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(5, "a"), + SimpleDataUtil.createRecord(4, "b"), + SimpleDataUtil.createRecord(3, "c"))); + + sql("INSERT OVERWRITE %s PARTITION (data='a') SELECT 6", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(6, "a"), + SimpleDataUtil.createRecord(4, "b"), + SimpleDataUtil.createRecord(3, "c"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testInsertIntoPartition() throws Exception { + String tableName = "test_insert_into_partition"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", + tableName, format.name()); + + try { + Table partitionedTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + + // Full partition. + sql("INSERT INTO %s PARTITION (data='a') SELECT 1", tableName); + sql("INSERT INTO %s PARTITION (data='a') SELECT 2", tableName); + sql("INSERT INTO %s PARTITION (data='b') SELECT 3", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "a"), + SimpleDataUtil.createRecord(3, "b"))); + + // Partial partition. + sql("INSERT INTO %s SELECT 4, 'c'", tableName); + sql("INSERT INTO %s SELECT 5, 'd'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "a"), + SimpleDataUtil.createRecord(3, "b"), + SimpleDataUtil.createRecord(4, "c"), + SimpleDataUtil.createRecord(5, "d"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + String tableName = "test_hash_distribution_mode"; + Map tableProps = + ImmutableMap.of( + "write.format.default", + format.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java new file mode 100644 index 000000000000..5674c83e40b8 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -0,0 +1,334 @@ +/* + * 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; + +import java.time.LocalDate; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +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.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkUpsert extends CatalogTestBase { + + @Parameter(index = 2) + private FileFormat format; + + @Parameter(index = 3) + private boolean isStreamingJob; + + private final Map tableUpsertProps = Maps.newHashMap(); + private TableEnvironment tEnv; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + public static List parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.PARQUET, FileFormat.AVRO, FileFormat.ORC}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + // Only test with one catalog as this is a file operation concern. + // FlinkCatalogTestBase requires the catalog name start with testhadoop if using hadoop + // catalog. + String catalogName = "testhadoop"; + Namespace baseNamespace = Namespace.of("default"); + parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + } + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + } + return tEnv; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + tableUpsertProps.put(TableProperties.FORMAT_VERSION, "2"); + tableUpsertProps.put(TableProperties.UPSERT_ENABLED, "true"); + tableUpsertProps.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + } + + @Override + @AfterEach + public void clean() { + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @TestTemplate + public void testUpsertAndQuery() { + String tableName = "test_upsert_query"; + LocalDate dt20220301 = LocalDate.of(2022, 3, 1); + LocalDate dt20220302 = LocalDate.of(2022, 3, 2); + + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + try { + sql( + "INSERT INTO %s VALUES " + + "(1, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-01')," + + "(2, 'Jane', DATE '2022-03-01')", + tableName); + + sql( + "INSERT INTO %s VALUES " + + "(2, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-02')," + + "(2, 'Jane', DATE '2022-03-02')", + tableName); + + List rowsOn20220301 = + Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); + + List rowsOn20220302 = + Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testUpsertOptions() { + String tableName = "test_upsert_options"; + LocalDate dt20220301 = LocalDate.of(2022, 3, 1); + LocalDate dt20220302 = LocalDate.of(2022, 3, 2); + + Map optionsUpsertProps = Maps.newHashMap(tableUpsertProps); + optionsUpsertProps.remove(TableProperties.UPSERT_ENABLED); + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(optionsUpsertProps)); + + try { + sql( + "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " + + "(1, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-01')," + + "(2, 'Jane', DATE '2022-03-01')", + tableName); + + sql( + "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " + + "(2, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-02')," + + "(2, 'Jane', DATE '2022-03-02')", + tableName); + + List rowsOn20220301 = + Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); + + List rowsOn20220302 = + Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testPrimaryKeyEqualToPartitionKey() { + // This is an SQL based reproduction of TestFlinkIcebergSinkV2#testUpsertOnDataKey + String tableName = "upsert_on_id_key"; + try { + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, PRIMARY KEY(id) NOT ENFORCED) " + + "PARTITIONED BY (id) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(1, 'Jane')," + "(2, 'Bill')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, "Jane"), Row.of(2, "Bill"))); + + sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(2, 'Jane')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, "Bill"), Row.of(2, "Jane"))); + + sql("INSERT INTO %s VALUES " + "(3, 'Bill')," + "(4, 'Jane')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of(1, "Bill"), Row.of(2, "Jane"), Row.of(3, "Bill"), Row.of(4, "Jane"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testPrimaryKeyFieldsAtBeginningOfSchema() { + String tableName = "upsert_on_pk_at_schema_start"; + LocalDate dt = LocalDate.of(2022, 3, 1); + try { + sql( + "CREATE TABLE %s(id INT, dt DATE NOT NULL, name STRING NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql( + "INSERT INTO %s VALUES " + + "(1, DATE '2022-03-01', 'Andy')," + + "(1, DATE '2022-03-01', 'Bill')," + + "(2, DATE '2022-03-01', 'Jane')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, dt, "Bill"), Row.of(2, dt, "Jane"))); + + sql( + "INSERT INTO %s VALUES " + + "(1, DATE '2022-03-01', 'Jane')," + + "(2, DATE '2022-03-01', 'Bill')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, dt, "Jane"), Row.of(2, dt, "Bill"))); + + sql( + "INSERT INTO %s VALUES " + + "(3, DATE '2022-03-01', 'Duke')," + + "(4, DATE '2022-03-01', 'Leon')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of(1, dt, "Jane"), + Row.of(2, dt, "Bill"), + Row.of(3, dt, "Duke"), + Row.of(4, dt, "Leon"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testPrimaryKeyFieldsAtEndOfTableSchema() { + // This is the same test case as testPrimaryKeyFieldsAtBeginningOfSchema, but the primary key + // fields + // are located at the end of the flink schema. + String tableName = "upsert_on_pk_at_schema_end"; + LocalDate dt = LocalDate.of(2022, 3, 1); + try { + sql( + "CREATE TABLE %s(name STRING NOT NULL, id INT, dt DATE NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql( + "INSERT INTO %s VALUES " + + "('Andy', 1, DATE '2022-03-01')," + + "('Bill', 1, DATE '2022-03-01')," + + "('Jane', 2, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of("Bill", 1, dt), Row.of("Jane", 2, dt))); + + sql( + "INSERT INTO %s VALUES " + + "('Jane', 1, DATE '2022-03-01')," + + "('Bill', 2, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of("Jane", 1, dt), Row.of("Bill", 2, dt))); + + sql( + "INSERT INTO %s VALUES " + + "('Duke', 3, DATE '2022-03-01')," + + "('Leon', 4, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of("Jane", 1, dt), + Row.of("Bill", 2, dt), + Row.of("Duke", 3, dt), + Row.of("Leon", 4, dt))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java new file mode 100644 index 000000000000..80e5ddd24fba --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -0,0 +1,628 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.avro.generic.GenericData; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +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.ContentFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.flink.source.FlinkInputFormat; +import org.apache.iceberg.flink.source.FlinkInputSplit; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.assertj.core.api.Assertions; + +public class TestHelpers { + private TestHelpers() {} + + public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { + KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + kryo.serialize(table, outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + return kryo.deserialize(inputView); + } + + public static RowData copyRowData(RowData from, RowType rowType) { + TypeSerializer[] fieldSerializers = + rowType.getChildren().stream() + .map((LogicalType type) -> InternalSerializers.create(type)) + .toArray(TypeSerializer[]::new); + return RowDataUtil.clone(from, null, rowType, fieldSerializers); + } + + public static void readRowData(FlinkInputFormat input, Consumer visitor) + throws IOException { + for (FlinkInputSplit s : input.createInputSplits(0)) { + input.open(s); + try { + while (!input.reachedEnd()) { + RowData row = input.nextRecord(null); + visitor.accept(row); + } + } finally { + input.close(); + } + } + } + + public static List readRowData(FlinkInputFormat inputFormat, RowType rowType) + throws IOException { + List results = Lists.newArrayList(); + readRowData(inputFormat, row -> results.add(copyRowData(row, rowType))); + return results; + } + + public static List readRows(FlinkInputFormat inputFormat, RowType rowType) + throws IOException { + return convertRowDataToRow(readRowData(inputFormat, rowType), rowType); + } + + public static List convertRowDataToRow(List rowDataList, RowType rowType) { + DataStructureConverter converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + return rowDataList.stream() + .map(converter::toExternal) + .map(Row.class::cast) + .collect(Collectors.toList()); + } + + private static List convertRecordToRow(List expectedRecords, Schema schema) { + List expected = Lists.newArrayList(); + @SuppressWarnings("unchecked") + DataStructureConverter converter = + (DataStructureConverter) + DataStructureConverters.getConverter( + TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(schema))); + expectedRecords.forEach( + r -> expected.add(converter.toExternal(RowDataConverter.convert(schema, r)))); + return expected; + } + + public static void assertRecordsWithOrder( + List results, List expectedRecords, Schema schema) { + List expected = convertRecordToRow(expectedRecords, schema); + assertRowsWithOrder(results, expected); + } + + public static void assertRecords(List results, List expectedRecords, Schema schema) { + List expected = convertRecordToRow(expectedRecords, schema); + assertRows(results, expected); + } + + public static void assertRows(List results, List expected, RowType rowType) { + assertRows(convertRowDataToRow(results, rowType), convertRowDataToRow(expected, rowType)); + } + + public static void assertRows(List results, List expected) { + assertThat(results).containsExactlyInAnyOrderElementsOf(expected); + } + + public static void assertRowsWithOrder(List results, List expected) { + Assertions.assertThat(results).containsExactlyElementsOf(expected); + } + + public static void assertRowData(Schema schema, StructLike expected, RowData actual) { + assertRowData(schema.asStruct(), FlinkSchemaUtil.convert(schema), expected, actual); + } + + public static void assertRowData( + Types.StructType structType, + LogicalType rowType, + StructLike expectedRecord, + RowData actualRowData) { + if (expectedRecord == null && actualRowData == null) { + return; + } + + assertThat(expectedRecord).isNotNull(); + assertThat(actualRowData).isNotNull(); + + List types = Lists.newArrayList(); + for (Types.NestedField field : structType.fields()) { + types.add(field.type()); + } + + for (int i = 0; i < types.size(); i += 1) { + LogicalType logicalType = ((RowType) rowType).getTypeAt(i); + Object expected = expectedRecord.get(i, Object.class); + // The RowData.createFieldGetter won't return null for the required field. But in the + // projection case, if we are + // projecting a nested required field from an optional struct, then we should give a null for + // the projected field + // if the outer struct value is null. So we need to check the nullable for actualRowData here. + // For more details + // please see issue #2738. + Object actual = + actualRowData.isNullAt(i) + ? null + : RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData); + assertEquals(types.get(i), logicalType, expected, actual); + } + } + + private static void assertEquals( + Type type, LogicalType logicalType, Object expected, Object actual) { + + if (expected == null && actual == null) { + return; + } + + assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + + switch (type.typeId()) { + case BOOLEAN: + assertThat(actual).as("boolean value should be equal").isEqualTo(expected); + break; + case INTEGER: + assertThat(actual).as("int value should be equal").isEqualTo(expected); + break; + case LONG: + assertThat(actual).as("long value should be equal").isEqualTo(expected); + break; + case FLOAT: + assertThat(actual).as("float value should be equal").isEqualTo(expected); + break; + case DOUBLE: + assertThat(actual).as("double value should be equal").isEqualTo(expected); + break; + case STRING: + assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); + assertThat(actual.toString()) + .as("string should be equal") + .isEqualTo(String.valueOf(expected)); + break; + case DATE: + assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); + LocalDate date = DateTimeUtil.dateFromDays((int) actual); + assertThat(date).as("date should be equal").isEqualTo(expected); + break; + case TIME: + assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); + int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); + assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); + break; + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + assertThat(expected) + .as("Should expect a OffsetDataTime") + .isInstanceOf(OffsetDateTime.class); + OffsetDateTime ts = (OffsetDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("OffsetDataTime should be equal") + .isEqualTo(ts.toLocalDateTime()); + } else { + assertThat(expected) + .as("Should expect a LocalDataTime") + .isInstanceOf(LocalDateTime.class); + LocalDateTime ts = (LocalDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("LocalDataTime should be equal") + .isEqualTo(ts); + } + break; + case BINARY: + assertThat(ByteBuffer.wrap((byte[]) actual)) + .as("Should expect a ByteBuffer") + .isInstanceOf(ByteBuffer.class) + .isEqualTo(expected); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + BigDecimal bd = (BigDecimal) expected; + assertThat(((DecimalData) actual).toBigDecimal()) + .as("decimal value should be equal") + .isEqualTo(bd); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + Collection expectedArrayData = (Collection) expected; + ArrayData actualArrayData = (ArrayData) actual; + LogicalType elementType = ((ArrayType) logicalType).getElementType(); + assertThat(actualArrayData.size()) + .as("array length should be equal") + .isEqualTo(expectedArrayData.size()); + assertArrayValues( + type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); + break; + case MAP: + assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + assertMapValues(type.asMapType(), logicalType, (Map) expected, (MapData) actual); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(StructLike.class); + assertRowData(type.asStructType(), logicalType, (StructLike) expected, (RowData) actual); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); + long firstLong = bb.getLong(); + long secondLong = bb.getLong(); + assertThat(new UUID(firstLong, secondLong).toString()) + .as("UUID should be equal") + .isEqualTo(expected.toString()); + break; + case FIXED: + assertThat(actual) + .as("Should expect byte[]") + .isInstanceOf(byte[].class) + .isEqualTo(expected); + break; + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + public static void assertEquals(Schema schema, List records, List rows) { + Streams.forEachPair( + records.stream(), rows.stream(), (record, row) -> assertEquals(schema, record, row)); + } + + public static void assertEquals(Schema schema, GenericData.Record record, Row row) { + List fields = schema.asStruct().fields(); + assertThat(fields).hasSameSizeAs(record.getSchema().getFields()); + assertThat(fields).hasSize(row.getArity()); + + RowType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < fields.size(); ++i) { + Type fieldType = fields.get(i).type(); + Object expectedValue = record.get(i); + Object actualValue = row.getField(i); + LogicalType logicalType = rowType.getTypeAt(i); + assertAvroEquals(fieldType, logicalType, expectedValue, actualValue); + } + } + + private static void assertEquals(Types.StructType struct, GenericData.Record record, Row row) { + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i).type(); + Object expectedValue = record.get(i); + Object actualValue = row.getField(i); + assertAvroEquals(fieldType, null, expectedValue, actualValue); + } + } + + private static void assertAvroEquals( + Type type, LogicalType logicalType, Object expected, Object actual) { + + if (expected == null && actual == null) { + return; + } + assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + assertThat(expected) + .as("Should expect a " + type.typeId().javaClass()) + .isInstanceOf(type.typeId().javaClass()); + assertThat(actual) + .as("Should expect a " + type.typeId().javaClass()) + .isInstanceOf(type.typeId().javaClass()); + assertThat(actual).as(type.typeId() + " value should be equal").isEqualTo(expected); + break; + case STRING: + assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); + assertThat(actual).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); + assertThat(actual.toString()).as("string should be equal").isEqualTo(expected.toString()); + break; + case DATE: + assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); + LocalDate date = DateTimeUtil.dateFromDays((int) actual); + assertThat(date).as("date should be equal").isEqualTo(expected); + break; + case TIME: + assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); + int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); + assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); + break; + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + assertThat(expected) + .as("Should expect a OffsetDataTime") + .isInstanceOf(OffsetDateTime.class); + OffsetDateTime ts = (OffsetDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("OffsetDataTime should be equal") + .isEqualTo(ts.toLocalDateTime()); + } else { + assertThat(expected) + .as("Should expect a LocalDataTime") + .isInstanceOf(LocalDateTime.class); + LocalDateTime ts = (LocalDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("LocalDataTime should be equal") + .isEqualTo(ts); + } + break; + case BINARY: + assertThat(ByteBuffer.wrap((byte[]) actual)) + .as("Should expect a ByteBuffer") + .isInstanceOf(ByteBuffer.class) + .isEqualTo(expected); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + BigDecimal bd = (BigDecimal) expected; + assertThat(((DecimalData) actual).toBigDecimal()) + .as("decimal value should be equal") + .isEqualTo(bd); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + Collection expectedArrayData = (Collection) expected; + ArrayData actualArrayData; + try { + actualArrayData = (ArrayData) actual; + } catch (ClassCastException e) { + actualArrayData = new GenericArrayData((Object[]) actual); + } + LogicalType elementType = ((ArrayType) logicalType).getElementType(); + assertThat(actualArrayData.size()) + .as("array length should be equal") + .isEqualTo(expectedArrayData.size()); + assertArrayValues( + type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); + break; + case MAP: + assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + MapData actualMap; + try { + actualMap = (MapData) actual; + } catch (ClassCastException e) { + actualMap = new GenericMapData((Map) actual); + } + assertMapValues(type.asMapType(), logicalType, (Map) expected, actualMap); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(GenericData.Record.class); + assertEquals( + type.asNestedType().asStructType(), (GenericData.Record) expected, (Row) actual); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); + long firstLong = bb.getLong(); + long secondLong = bb.getLong(); + assertThat(new UUID(firstLong, secondLong).toString()) + .as("UUID should be equal") + .isEqualTo(expected.toString()); + break; + case FIXED: + assertThat(actual) + .as("Should expect byte[]") + .isInstanceOf(byte[].class) + .isEqualTo(expected); + break; + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + private static void assertArrayValues( + Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { + List expectedElements = Lists.newArrayList(expectedArray); + for (int i = 0; i < expectedArray.size(); i += 1) { + if (expectedElements.get(i) == null) { + assertThat(actualArray.isNullAt(i)).isTrue(); + continue; + } + + Object expected = expectedElements.get(i); + + assertEquals( + type, + logicalType, + expected, + ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, i)); + } + } + + private static void assertMapValues( + Types.MapType mapType, LogicalType type, Map expected, MapData actual) { + assertThat(actual.size()).as("map size should be equal").isEqualTo(expected.size()); + + ArrayData actualKeyArrayData = actual.keyArray(); + ArrayData actualValueArrayData = actual.valueArray(); + LogicalType actualKeyType = ((MapType) type).getKeyType(); + LogicalType actualValueType = ((MapType) type).getValueType(); + Type keyType = mapType.keyType(); + Type valueType = mapType.valueType(); + + ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(actualKeyType); + ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(actualValueType); + + for (Map.Entry entry : expected.entrySet()) { + Object matchedActualKey = null; + int matchedKeyIndex = 0; + for (int i = 0; i < actual.size(); i += 1) { + try { + Object key = keyGetter.getElementOrNull(actualKeyArrayData, i); + assertEquals(keyType, actualKeyType, entry.getKey(), key); + matchedActualKey = key; + matchedKeyIndex = i; + break; + } catch (AssertionError e) { + // not found + } + } + assertThat(matchedActualKey).as("Should have a matching key").isNotNull(); + final int valueIndex = matchedKeyIndex; + assertEquals( + valueType, + actualValueType, + entry.getValue(), + valueGetter.getElementOrNull(actualValueArrayData, valueIndex)); + } + } + + public static void assertEquals(ManifestFile expected, ManifestFile actual) { + if (expected == actual) { + return; + } + assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + assertThat(actual.path()).as("Path must match").isEqualTo(expected.path()); + assertThat(actual.length()).as("Length must match").isEqualTo(expected.length()); + assertThat(actual.partitionSpecId()) + .as("Spec id must match") + .isEqualTo(expected.partitionSpecId()); + assertThat(actual.content()).as("ManifestContent must match").isEqualTo(expected.content()); + assertThat(actual.sequenceNumber()) + .as("SequenceNumber must match") + .isEqualTo(expected.sequenceNumber()); + assertThat(actual.minSequenceNumber()) + .as("MinSequenceNumber must match") + .isEqualTo(expected.minSequenceNumber()); + assertThat(actual.snapshotId()).as("Snapshot id must match").isEqualTo(expected.snapshotId()); + assertThat(actual.hasAddedFiles()) + .as("Added files flag must match") + .isEqualTo(expected.hasAddedFiles()); + assertThat(actual.addedFilesCount()) + .as("Added files count must match") + .isEqualTo(expected.addedFilesCount()); + assertThat(actual.addedRowsCount()) + .as("Added rows count must match") + .isEqualTo(expected.addedRowsCount()); + assertThat(actual.hasExistingFiles()) + .as("Existing files flag must match") + .isEqualTo(expected.hasExistingFiles()); + assertThat(actual.existingFilesCount()) + .as("Existing files count must match") + .isEqualTo(expected.existingFilesCount()); + assertThat(actual.existingRowsCount()) + .as("Existing rows count must match") + .isEqualTo(expected.existingRowsCount()); + assertThat(actual.hasDeletedFiles()) + .as("Deleted files flag must match") + .isEqualTo(expected.hasDeletedFiles()); + assertThat(actual.deletedFilesCount()) + .as("Deleted files count must match") + .isEqualTo(expected.deletedFilesCount()); + assertThat(actual.deletedRowsCount()) + .as("Deleted rows count must match") + .isEqualTo(expected.deletedRowsCount()); + + List expectedSummaries = expected.partitions(); + List actualSummaries = actual.partitions(); + assertThat(actualSummaries) + .as("PartitionFieldSummary size does not match") + .hasSameSizeAs(expectedSummaries); + for (int i = 0; i < expectedSummaries.size(); i++) { + assertThat(actualSummaries.get(i).containsNull()) + .as("Null flag in partition must match") + .isEqualTo(expectedSummaries.get(i).containsNull()); + assertThat(actualSummaries.get(i).containsNaN()) + .as("NaN flag in partition must match") + .isEqualTo(expectedSummaries.get(i).containsNaN()); + assertThat(actualSummaries.get(i).lowerBound()) + .as("Lower bounds in partition must match") + .isEqualTo(expectedSummaries.get(i).lowerBound()); + assertThat(actualSummaries.get(i).upperBound()) + .as("Upper bounds in partition must match") + .isEqualTo(expectedSummaries.get(i).upperBound()); + } + } + + public static void assertEquals(ContentFile expected, ContentFile actual) { + if (expected == actual) { + return; + } + assertThat(expected).isNotNull(); + 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.format()).as("Format").isEqualTo(expected.format()); + assertThat(actual.partition().size()) + .as("Partition size") + .isEqualTo(expected.partition().size()); + for (int i = 0; i < expected.partition().size(); i++) { + assertThat(actual.partition().get(i, Object.class)) + .as("Partition data at index " + i) + .isEqualTo(expected.partition().get(i, Object.class)); + } + assertThat(actual.recordCount()).as("Record count").isEqualTo(expected.recordCount()); + assertThat(actual.fileSizeInBytes()) + .as("File size in bytes") + .isEqualTo(expected.fileSizeInBytes()); + assertThat(actual.columnSizes()).as("Column sizes").isEqualTo(expected.columnSizes()); + assertThat(actual.valueCounts()).as("Value counts").isEqualTo(expected.valueCounts()); + assertThat(actual.nullValueCounts()) + .as("Null value counts") + .isEqualTo(expected.nullValueCounts()); + assertThat(actual.lowerBounds()).as("Lower bounds").isEqualTo(expected.lowerBounds()); + assertThat(actual.upperBounds()).as("Upper bounds").isEqualTo(expected.upperBounds()); + assertThat(actual.keyMetadata()).as("Key metadata").isEqualTo(expected.keyMetadata()); + assertThat(actual.splitOffsets()).as("Split offsets").isEqualTo(expected.splitOffsets()); + assertThat(actual.equalityFieldIds()) + .as("Equality field id list") + .isEqualTo(expected.equalityFieldIds()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java new file mode 100644 index 000000000000..cb409b784383 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -0,0 +1,350 @@ +/* + * 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; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.iceberg.CatalogProperties; +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.relocated.com.google.common.collect.Sets; +import org.apache.thrift.TException; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergConnector extends FlinkTestBase { + + private static final String TABLE_NAME = "test_table"; + + @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); + + private final String catalogName; + private final Map properties; + private final boolean isStreaming; + private volatile TableEnvironment tEnv; + + @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") + public static Iterable parameters() { + return Lists.newArrayList( + // Create iceberg table in the hadoop catalog and default database. + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop"), + false + }, + // Create iceberg table in the hadoop catalog and not_existing_db. + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db"), + false + }, + // Create iceberg table in the hive catalog and default database. + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive"), + false + }, + // Create iceberg table in the hive catalog and not_existing_db. + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db"), + false + }); + } + + public TestIcebergConnector( + String catalogName, Map properties, boolean isStreaming) { + this.catalogName = catalogName; + this.properties = properties; + this.isStreaming = isStreaming; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreaming) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + // Set only one parallelism. + tEnv.getConfig() + .getConfiguration() + .set(CoreOptions.DEFAULT_PARALLELISM, 1) + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + } + } + } + return tEnv; + } + + @After + public void after() throws TException { + sql("DROP TABLE IF EXISTS %s", TABLE_NAME); + + // Clean the created orphan databases and tables from hive-metastore. + if (isHiveCatalog()) { + HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf); + try { + metaStoreClient.dropTable(databaseName(), tableName()); + if (!isDefaultDatabaseName()) { + try { + metaStoreClient.dropDatabase(databaseName()); + } catch (Exception ignored) { + // Ignore + } + } + } finally { + metaStoreClient.close(); + } + } + } + + private void testCreateConnectorTable() { + Map tableProps = createTableProps(); + + // Create table under the flink's current database. + sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); + sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); + Assert.assertEquals( + "Should have expected rows", + Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + + FlinkCatalogFactory factory = new FlinkCatalogFactory(); + Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); + Assert.assertTrue( + "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); + Assert.assertTrue( + "Should have created the expected table", + flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); + + // Drop and create it again. + sql("DROP TABLE %s", TABLE_NAME); + sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); + Assert.assertEquals( + "Should have expected rows", + Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), + Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + } + + @Test + public void testCreateTableUnderDefaultDatabase() { + testCreateConnectorTable(); + } + + @Test + public void testCatalogDatabaseConflictWithFlinkDatabase() { + sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); + sql("USE `%s`", databaseName()); + + try { + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + Assertions.assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); + } finally { + sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); + if (!isDefaultDatabaseName()) { + sql("DROP DATABASE `%s`", databaseName()); + } + } + } + + @Test + public void testConnectorTableInIcebergCatalog() { + // Create the catalog properties + Map catalogProps = Maps.newHashMap(); + catalogProps.put("type", "iceberg"); + if (isHiveCatalog()) { + catalogProps.put("catalog-type", "hive"); + catalogProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + } else { + catalogProps.put("catalog-type", "hadoop"); + } + catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); + + // Create the table properties + Map tableProps = createTableProps(); + + // Create a connector table in an iceberg catalog. + sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); + try { + Assertions.assertThatThrownBy( + () -> + sql( + "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", + FlinkCatalogFactory.DEFAULT_DATABASE_NAME, + TABLE_NAME, + toWithClause(tableProps))) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " + + "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + + "create table without 'connector'='iceberg' related properties in an iceberg table."); + } finally { + sql("DROP CATALOG IF EXISTS `test_catalog`"); + } + } + + private Map createTableProps() { + Map tableProps = Maps.newHashMap(properties); + tableProps.put("catalog-name", catalogName); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); + if (isHiveCatalog()) { + tableProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + } + return tableProps; + } + + private boolean isHiveCatalog() { + return "testhive".equalsIgnoreCase(catalogName); + } + + private boolean isDefaultDatabaseName() { + return FlinkCatalogFactory.DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName()); + } + + private String tableName() { + return properties.getOrDefault("catalog-table", TABLE_NAME); + } + + private String databaseName() { + return properties.getOrDefault("catalog-database", "default_database"); + } + + private String toWithClause(Map props) { + return CatalogTestBase.toWithClause(props); + } + + private static String createWarehouse() { + try { + return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java new file mode 100644 index 000000000000..8f1f129e183b --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -0,0 +1,173 @@ +/* + * 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; + +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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Path; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestManifestFileSerialization { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + required(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("double").build(); + + private static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withPartition(org.apache.iceberg.TestHelpers.Row.of(1D)) + .withPartitionPath("double=1") + .withMetrics( + new Metrics( + 5L, + null, // no column sizes + ImmutableMap.of(1, 5L, 2, 3L), // value count + ImmutableMap.of(1, 0L, 2, 2L), // null count + ImmutableMap.of(), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withPartition(org.apache.iceberg.TestHelpers.Row.of(Double.NaN)) + .withPartitionPath("double=NaN") + .withMetrics( + new Metrics( + 1L, + null, // no column sizes + ImmutableMap.of(1, 1L, 4, 1L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count + ImmutableMap.of(4, 1L), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(1L)) // upper bounds + )) + .build(); + + private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); + + @TempDir private Path temp; + + @Test + public void testKryoSerialization() throws IOException { + KryoSerializer kryo = + new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + kryo.serialize(manifest, outputView); + kryo.serialize(manifest.copy(), outputView); + kryo.serialize(GenericManifestFile.copyOf(manifest).build(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + ManifestFile m1 = kryo.deserialize(inputView); + ManifestFile m2 = kryo.deserialize(inputView); + ManifestFile m3 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(manifest, m1); + TestHelpers.assertEquals(manifest, m2); + TestHelpers.assertEquals(manifest, m3); + } + + @Test + public void testJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(manifest); + out.writeObject(manifest.copy()); + out.writeObject(GenericManifestFile.copyOf(manifest).build()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 3; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + TestHelpers.assertEquals(manifest, (ManifestFile) obj); + } + } + } + + private ManifestFile writeManifest(DataFile... files) throws IOException { + File manifestFile = File.createTempFile("input", "m0.avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); + OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); + + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); + try { + for (DataFile file : files) { + writer.add(file); + } + } finally { + writer.close(); + } + + return writer.toManifestFile(); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java new file mode 100644 index 000000000000..caefbb5a5429 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -0,0 +1,93 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Iterator; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.RecordWrapperTest; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.util.StructLikeWrapper; + +public class TestRowDataWrapper extends RecordWrapperTest { + + /** + * Flink's time type has been truncated to millis seconds, so we need a customized assert method + * to check the values. + */ + @Override + public void testTime() { + generateAndValidate( + new Schema(TIME.fields()), + (message, expectedWrapper, actualWrapper) -> { + for (int pos = 0; pos < TIME.fields().size(); pos++) { + Object expected = expectedWrapper.get().get(pos, Object.class); + Object actual = actualWrapper.get().get(pos, Object.class); + if (expected == actual) { + return; + } + + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); + + int expectedMilliseconds = (int) ((long) expected / 1000_000); + int actualMilliseconds = (int) ((long) actual / 1000_000); + assertThat(actualMilliseconds).as(message).isEqualTo(expectedMilliseconds); + } + }); + } + + @Override + protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { + int numRecords = 100; + Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); + Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); + + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + Iterator actual = recordList.iterator(); + Iterator expected = rowDataList.iterator(); + + StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); + StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); + for (int i = 0; i < numRecords; i++) { + assertThat(actual).hasNext(); + assertThat(expected).hasNext(); + + StructLike recordStructLike = recordWrapper.wrap(actual.next()); + StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); + + assertMethod.assertEquals( + "Should have expected StructLike values", + actualWrapper.set(recordStructLike), + expectedWrapper.set(rowDataStructLike)); + } + + assertThat(actual).isExhausted(); + assertThat(expected).isExhausted(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java new file mode 100644 index 000000000000..4ad302dde436 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.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.flink; + +import java.io.File; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestTables; + +public class TestTableLoader implements TableLoader { + private File dir; + + public static TableLoader of(String dir) { + return new TestTableLoader(dir); + } + + public TestTableLoader(String dir) { + this.dir = new File(dir); + } + + @Override + public void open() {} + + @Override + public boolean isOpen() { + return true; + } + + @Override + public Table loadTable() { + return TestTables.load(dir, "test"); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new TestTableLoader(dir.getAbsolutePath()); + } + + @Override + public void close() {} +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java new file mode 100644 index 000000000000..7f0e7acaa822 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.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.flink; + +import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; +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.Map; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.hadoop.HadoopTables; +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; + +public class TestTableSerialization { + private static final HadoopTables TABLES = new HadoopTables(); + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("date").build(); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @TempDir private Path temp; + private Table table; + + @BeforeEach + public void initTable() throws IOException { + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + + File tableLocation = File.createTempFile("junit", null, temp.toFile()); + assertThat(tableLocation.delete()).isTrue(); + + this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); + } + + @Test + public void testSerializableTableKryoSerialization() throws IOException { + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + TestHelpers.assertSerializedAndLoadedMetadata( + table, roundTripKryoSerialize(SerializableTable.class, serializableTable)); + } + + @Test + public void testSerializableMetadataTableKryoSerialization() throws IOException { + for (MetadataTableType type : MetadataTableType.values()) { + TableOperations ops = ((HasTableOperations) table).operations(); + Table metadataTable = + MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); + SerializableTable serializableMetadataTable = + (SerializableTable) SerializableTable.copyOf(metadataTable); + + TestHelpers.assertSerializedAndLoadedMetadata( + metadataTable, + roundTripKryoSerialize(SerializableTable.class, serializableMetadataTable)); + } + } + + @Test + public void testSerializableTransactionTableKryoSerialization() throws IOException { + Transaction txn = table.newTransaction(); + + txn.updateProperties().set("k1", "v1").commit(); + + Table txnTable = txn.table(); + SerializableTable serializableTxnTable = (SerializableTable) SerializableTable.copyOf(txnTable); + + TestHelpers.assertSerializedMetadata( + txnTable, roundTripKryoSerialize(SerializableTable.class, serializableTxnTable)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java new file mode 100644 index 000000000000..4220775f41fa --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -0,0 +1,481 @@ +/* + * 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.actions; + +import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFilesActionResult; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +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.types.Types; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +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; + +public class TestRewriteDataFilesAction extends CatalogTestBase { + + private static final String TABLE_NAME_UNPARTITIONED = "test_table_unpartitioned"; + private static final String TABLE_NAME_PARTITIONED = "test_table_partitioned"; + private static final String TABLE_NAME_WITH_PK = "test_table_with_pk"; + + @Parameter(index = 2) + private FileFormat format; + + private Table icebergTableUnPartitioned; + private Table icebergTablePartitioned; + private Table icebergTableWithPk; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") + 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}); + } + } + return parameters; + } + + private @TempDir Path temp; + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + 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()); + 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()); + 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()); + icebergTableWithPk = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @TestTemplate + public void testRewriteDataFilesEmptyTable() throws Exception { + assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); + Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); + assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); + } + + @TestTemplate + public void testRewriteDataFilesUnpartitionedTable() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(2); + RewriteDataFilesActionResult result = + Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); + + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFiles1).hasSize(1); + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTableUnPartitioned, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), SimpleDataUtil.createRecord(2, "world"))); + } + + @TestTemplate + public void testRewriteDataFilesPartitionedTable() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 3, 'world' ,'b'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(4); + RewriteDataFilesActionResult result = + Actions.forTable(icebergTablePartitioned).rewriteDataFiles().execute(); + + assertThat(result.deletedDataFiles()).hasSize(4); + assertThat(result.addedDataFiles()).hasSize(2); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFiles1).hasSize(2); + // Assert the table records as expected. + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "spec", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + SimpleDataUtil.assertTableRecords( + icebergTablePartitioned, + Lists.newArrayList( + record.copy("id", 1, "data", "hello", "spec", "a"), + record.copy("id", 2, "data", "hello", "spec", "a"), + record.copy("id", 3, "data", "world", "spec", "b"), + record.copy("id", 4, "data", "world", "spec", "b"))); + } + + @TestTemplate + public void testRewriteDataFilesWithFilter() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 3, 'world' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 5, 'world' ,'b'", TABLE_NAME_PARTITIONED); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(5); + RewriteDataFilesActionResult result = + Actions.forTable(icebergTablePartitioned) + .rewriteDataFiles() + .filter(Expressions.equal("spec", "a")) + .filter(Expressions.startsWith("data", "he")) + .execute(); + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFiles1).hasSize(4); + // Assert the table records as expected. + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "spec", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + SimpleDataUtil.assertTableRecords( + icebergTablePartitioned, + Lists.newArrayList( + record.copy("id", 1, "data", "hello", "spec", "a"), + record.copy("id", 2, "data", "hello", "spec", "a"), + record.copy("id", 3, "data", "world", "spec", "a"), + record.copy("id", 4, "data", "world", "spec", "b"), + record.copy("id", 5, "data", "world", "spec", "b"))); + } + + @TestTemplate + public void testRewriteLargeTableHasResiduals() throws IOException { + // all records belong to the same partition + List records1 = Lists.newArrayList(); + List records2 = Lists.newArrayList(); + List expected = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + int id = i; + String data = String.valueOf(i % 3); + if (i % 2 == 0) { + records1.add("(" + id + ",'" + data + "')"); + } else { + records2.add("(" + id + ",'" + data + "')"); + } + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + expected.add(record); + } + + sql("INSERT INTO %s values " + StringUtils.join(records1, ","), TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s values " + StringUtils.join(records2, ","), TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = + icebergTableUnPartitioned + .newScan() + .ignoreResiduals() + .filter(Expressions.equal("data", "0")) + .planFiles(); + for (FileScanTask task : tasks) { + assertThat(task.residual()) + .as("Residuals must be ignored") + .isEqualTo(Expressions.alwaysTrue()); + } + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(2); + Actions actions = Actions.forTable(icebergTableUnPartitioned); + + RewriteDataFilesActionResult result = + actions.rewriteDataFiles().filter(Expressions.equal("data", "0")).execute(); + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); + } + + /** + * a test case to test avoid repeate compress + * + *

    If datafile cannot be combined to CombinedScanTask with other DataFiles, the size of the + * CombinedScanTask list size is 1, so we remove these CombinedScanTasks to avoid compressed + * repeatedly. + * + *

    In this test case,we generated 3 data files and set targetSizeInBytes greater than the + * largest file size so that it cannot be combined a CombinedScanTask with other datafiles. The + * datafile with the largest file size will not be compressed. + * + * @throws IOException IOException + */ + @TestTemplate + public void testRewriteAvoidRepeateCompress() throws IOException { + List expected = Lists.newArrayList(); + Schema schema = icebergTableUnPartitioned.schema(); + GenericAppenderFactory genericAppenderFactory = new GenericAppenderFactory(schema); + File file = File.createTempFile("junit", null, temp.toFile()); + int count = 0; + try (FileAppender fileAppender = + genericAppenderFactory.newAppender(Files.localOutput(file), format)) { + long filesize = 20000; + for (; fileAppender.length() < filesize; count++) { + Record record = SimpleDataUtil.createRecord(count, UUID.randomUUID().toString()); + fileAppender.add(record); + expected.add(record); + } + } + + DataFile dataFile = + DataFiles.builder(icebergTableUnPartitioned.spec()) + .withPath(file.getAbsolutePath()) + .withFileSizeInBytes(file.length()) + .withFormat(format) + .withRecordCount(count) + .build(); + + icebergTableUnPartitioned.newAppend().appendFile(dataFile).commit(); + + sql("INSERT INTO %s SELECT 1,'a' ", TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s SELECT 2,'b' ", TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(3); + Actions actions = Actions.forTable(icebergTableUnPartitioned); + + long targetSizeInBytes = file.length() + 10; + RewriteDataFilesActionResult result = + actions + .rewriteDataFiles() + .targetSizeInBytes(targetSizeInBytes) + .splitOpenFileCost(1) + .execute(); + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); + List dataFilesRewrote = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFilesRewrote).hasSize(2); + // the biggest file do not be rewrote + List rewroteDataFileNames = + dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); + + // Assert the table records as expected. + expected.add(SimpleDataUtil.createRecord(1, "a")); + expected.add(SimpleDataUtil.createRecord(2, "b")); + SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); + } + + @TestTemplate + public void testRewriteNoConflictWithEqualityDeletes() throws IOException { + // Add 2 data files + sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_WITH_PK); + sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_WITH_PK); + + // Load 2 stale tables to pass to rewrite actions + // Since the first rewrite will refresh stale1, we need another stale2 for the second rewrite + Table stale1 = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + Table stale2 = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + + // Add 1 data file and 1 equality-delete file + sql("INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ SELECT 1, 'hi'", TABLE_NAME_WITH_PK); + + icebergTableWithPk.refresh(); + assertThat(icebergTableWithPk.currentSnapshot().sequenceNumber()) + .as("The latest sequence number should be greater than that of the stale snapshot") + .isEqualTo(stale1.currentSnapshot().sequenceNumber() + 1); + CloseableIterable tasks = icebergTableWithPk.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Set deleteFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::deletes)).stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + assertThat(dataFiles).hasSize(3); + assertThat(deleteFiles).hasSize(1); + assertThat(Iterables.getOnlyElement(deleteFiles).content()) + .isEqualTo(FileContent.EQUALITY_DELETES); + shouldHaveDataAndFileSequenceNumbers( + TABLE_NAME_WITH_PK, + ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); + + Assertions.assertThatThrownBy( + () -> + Actions.forTable(stale1) + .rewriteDataFiles() + .useStartingSequenceNumber(false) + .execute(), + "Rewrite using new sequence number should fail") + .isInstanceOf(ValidationException.class); + + // Rewrite using the starting sequence number should succeed + RewriteDataFilesActionResult result = + Actions.forTable(stale2).rewriteDataFiles().useStartingSequenceNumber(true).execute(); + + // Should not rewrite files from the new commit + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + // The 2 older files with file-sequence-number <= 2 should be rewritten into a new file. + // The new file is the one with file-sequence-number == 4. + // The new file should use rewrite's starting-sequence-number 2 as its data-sequence-number. + shouldHaveDataAndFileSequenceNumbers( + TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(3L, 3L), Pair.of(3L, 3L), Pair.of(2L, 4L))); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTableWithPk, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hi"), SimpleDataUtil.createRecord(2, "world"))); + } + + /** + * Assert that data files and delete files in the table should have expected data sequence numbers + * and file sequence numbers + * + * @param tableName table name + * @param expectedSequenceNumbers list of {@link Pair}'s. Each {@link Pair} contains + * (expectedDataSequenceNumber, expectedFileSequenceNumber) of a file. + */ + private void shouldHaveDataAndFileSequenceNumbers( + String tableName, List> expectedSequenceNumbers) { + // "status < 2" for added or existing entries + List liveEntries = sql("SELECT * FROM %s$entries WHERE status < 2", tableName); + + List> actualSequenceNumbers = + liveEntries.stream() + .map( + row -> + Pair.of( + row.getFieldAs("sequence_number"), row.getFieldAs("file_sequence_number"))) + .collect(Collectors.toList()); + assertThat(actualSequenceNumbers).hasSameElementsAs(expectedSequenceNumbers); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java new file mode 100644 index 000000000000..cc58d9817ac6 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.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 org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class RandomRowData { + private RandomRowData() {} + + public static Iterable generate(Schema schema, int numRecords, long seed) { + return convert(schema, RandomGenericData.generate(schema, numRecords, seed)); + } + + public static Iterable convert(Schema schema, Iterable records) { + return Iterables.transform(records, record -> RowDataConverter.convert(schema, record)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java new file mode 100644 index 000000000000..74b1da6007e6 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.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.flink.data; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +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; + +public class RowDataToRowMapper extends RichMapFunction { + + private final RowType rowType; + + private transient DataStructureConverter converter; + + public RowDataToRowMapper(RowType rowType) { + this.rowType = rowType; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + } + + @Override + public Row map(RowData value) throws Exception { + return (Row) converter.toExternal(value); + } +} 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/TestFlinkAvroReaderWriter.java new file mode 100644 index 000000000000..a1039d27d888 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java @@ -0,0 +1,185 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DataTest; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +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.iceberg.util.DateTimeUtil; +import org.junit.jupiter.api.Test; + +public class TestFlinkAvroReaderWriter extends DataTest { + + private static final int NUM_RECORDS = 100; + + private static final Schema SCHEMA_NUM_TYPE = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "int", Types.IntegerType.get()), + Types.NestedField.optional(3, "float", Types.FloatType.get()), + Types.NestedField.optional(4, "double", Types.DoubleType.get()), + Types.NestedField.optional(5, "date", Types.DateType.get()), + Types.NestedField.optional(6, "time", Types.TimeType.get()), + Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone()), + Types.NestedField.optional(8, "bigint", Types.LongType.get()), + Types.NestedField.optional(9, "decimal", Types.DecimalType.of(4, 2))); + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L); + writeAndValidate(schema, expectedRecords, NUM_RECORDS); + } + + private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) + throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); + + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); + + // Write the expected records into AVRO file, then read them into RowData and assert with the + // expected Record list. + try (FileAppender writer = + Avro.write(Files.localOutput(recordsFile)) + .schema(schema) + .createWriterFunc(DataWriter::create) + .build()) { + writer.addAll(expectedRecords); + } + + try (CloseableIterable reader = + Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(FlinkAvroReader::new) + .build()) { + Iterator expected = expectedRecords.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < numRecord; i++) { + assertThat(rows).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); + } + assertThat(rows).isExhausted(); + } + + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); + + // Write the expected RowData into AVRO file, then read them into Record and assert with the + // expected RowData list. + try (FileAppender writer = + Avro.write(Files.localOutput(rowDataFile)) + .schema(schema) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .build()) { + writer.addAll(expectedRows); + } + + try (CloseableIterable reader = + Avro.read(Files.localInput(rowDataFile)) + .project(schema) + .createReaderFunc(DataReader::create) + .build()) { + Iterator expected = expectedRows.iterator(); + Iterator records = reader.iterator(); + for (int i = 0; i < numRecord; i += 1) { + assertThat(records).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); + } + assertThat(records).isExhausted(); + } + } + + private Record recordNumType( + int id, + int intV, + float floatV, + double doubleV, + long date, + long time, + long timestamp, + long bigint, + double decimal) { + Record record = GenericRecord.create(SCHEMA_NUM_TYPE); + record.setField("id", id); + record.setField("int", intV); + record.setField("float", floatV); + record.setField("double", doubleV); + record.setField( + "date", DateTimeUtil.dateFromDays((int) new Date(date).toLocalDate().toEpochDay())); + record.setField("time", new Time(time).toLocalTime()); + record.setField("timestamp", DateTimeUtil.timestampFromMicros(timestamp * 1000)); + record.setField("bigint", bigint); + record.setField("decimal", BigDecimal.valueOf(decimal)); + return record; + } + + @Test + public void testNumericTypes() throws IOException { + + List expected = + ImmutableList.of( + recordNumType( + 2, + Integer.MAX_VALUE, + Float.MAX_VALUE, + Double.MAX_VALUE, + Long.MAX_VALUE, + 1643811742000L, + 1643811742000L, + 1643811742000L, + 10.24d), + recordNumType( + 2, + Integer.MIN_VALUE, + Float.MIN_VALUE, + Double.MIN_VALUE, + Long.MIN_VALUE, + 1643811742000L, + 1643811742000L, + 1643811742000L, + 10.24d)); + + writeAndValidate(SCHEMA_NUM_TYPE, expected, 2); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java new file mode 100644 index 000000000000..72f2ce4f4bce --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.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.flink.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class TestFlinkOrcReaderWriter extends DataTest { + private static final int NUM_RECORDS = 100; + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); + List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); + + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); + + // Write the expected records into ORC file, then read them into RowData and assert with the + // expected Record list. + try (FileAppender writer = + ORC.write(Files.localOutput(recordsFile)) + .schema(schema) + .createWriterFunc(GenericOrcWriter::buildWriter) + .build()) { + writer.addAll(expectedRecords); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(type -> new FlinkOrcReader(schema, type)) + .build()) { + Iterator expected = expectedRecords.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < NUM_RECORDS; i++) { + assertThat(rows).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); + } + assertThat(rows).isExhausted(); + } + + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); + + // Write the expected RowData into ORC file, then read them into Record and assert with the + // expected RowData list. + RowType rowType = FlinkSchemaUtil.convert(schema); + try (FileAppender writer = + ORC.write(Files.localOutput(rowDataFile)) + .schema(schema) + .createWriterFunc((iSchema, typeDesc) -> FlinkOrcWriter.buildWriter(rowType, iSchema)) + .build()) { + writer.addAll(expectedRows); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(rowDataFile)) + .project(schema) + .createReaderFunc(type -> GenericOrcReader.buildReader(schema, type)) + .build()) { + Iterator expected = expectedRows.iterator(); + Iterator records = reader.iterator(); + for (int i = 0; i < NUM_RECORDS; i += 1) { + assertThat(records.hasNext()).isTrue(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); + } + assertThat(records).isExhausted(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java new file mode 100644 index 000000000000..1fdc4cf8381a --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.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.flink.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +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.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.junit.jupiter.api.Test; + +public class TestFlinkParquetReader extends DataTest { + private static final int NUM_RECORDS = 100; + + @Test + public void testTwoLevelList() throws IOException { + Schema schema = + new Schema( + optional(1, "arraybytes", Types.ListType.ofRequired(3, Types.BinaryType.get())), + optional(2, "topbytes", Types.BinaryType.get())); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); + + ParquetWriter writer = + AvroParquetWriter.builder(new Path(testFile.toURI())) + .withDataModel(GenericData.get()) + .withSchema(avroSchema) + .config("parquet.avro.add-list-element-records", "true") + .config("parquet.avro.write-old-list-structure", "true") + .build(); + + GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema); + List expectedByteList = Lists.newArrayList(); + byte[] expectedByte = {0x00, 0x01}; + ByteBuffer expectedBinary = ByteBuffer.wrap(expectedByte); + expectedByteList.add(expectedBinary); + recordBuilder.set("arraybytes", expectedByteList); + recordBuilder.set("topbytes", expectedBinary); + GenericData.Record expectedRecord = recordBuilder.build(); + + writer.write(expectedRecord); + writer.close(); + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) + .build()) { + Iterator rows = reader.iterator(); + assertThat(rows).hasNext(); + RowData rowData = rows.next(); + assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); + assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); + assertThat(rows).isExhausted(); + } + } + + private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::buildWriter) + .build()) { + writer.addAll(iterable); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) + .build()) { + Iterator expected = iterable.iterator(); + Iterator rows = reader.iterator(); + LogicalType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < NUM_RECORDS; i += 1) { + assertThat(rows).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); + } + assertThat(rows).isExhausted(); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(RandomGenericData.generate(schema, NUM_RECORDS, 19981), schema); + writeAndValidate( + RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124), schema); + writeAndValidate( + RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), + schema); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java new file mode 100644 index 000000000000..b1e6f5aa00ff --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java @@ -0,0 +1,94 @@ +/* + * 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 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 org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.junit.jupiter.api.io.TempDir; + +public class TestFlinkParquetWriter extends DataTest { + private static final int NUM_RECORDS = 100; + + @TempDir private Path temp; + + private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); + + LogicalType logicalType = FlinkSchemaUtil.convert(schema); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(logicalType, msgType)) + .build()) { + writer.addAll(iterable); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) + .build()) { + Iterator expected = iterable.iterator(); + Iterator actual = reader.iterator(); + LogicalType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < NUM_RECORDS; i += 1) { + assertThat(actual).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); + } + assertThat(actual).isExhausted(); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(RandomRowData.generate(schema, NUM_RECORDS, 19981), schema); + + writeAndValidate( + RandomRowData.convert( + schema, + RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124)), + schema); + + writeAndValidate( + RandomRowData.convert( + schema, + RandomGenericData.generateFallbackRecords( + schema, NUM_RECORDS, 21124, NUM_RECORDS / 20)), + schema); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java new file mode 100644 index 000000000000..3cd25c8fa983 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java @@ -0,0 +1,593 @@ +/* + * 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.util.List; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.TestHelpers; +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.apache.iceberg.util.StructProjection; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestRowDataProjection { + @Test + public void testNullRootRowData() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); + + Assertions.assertThatThrownBy(() -> projection.wrap(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid row data: null"); + } + + @Test + public void testFullProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + generateAndValidate(schema, schema); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + } + + @Test + public void testReorderedFullProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + generateAndValidate(schema, reordered); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, reordered, rowData, copyRowData, otherRowData); + } + + @Test + public void testBasicProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + generateAndValidate(schema, idOnly); + generateAndValidate(schema, dataOnly); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, dataOnly, rowData, copyRowData, otherRowData); + } + + @Test + public void testEmptyProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + generateAndValidate(schema, schema.select()); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, schema.select(), rowData, copyRowData, otherRowData, true); + } + + @Test + public void testRename() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Schema renamed = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + generateAndValidate(schema, renamed); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, renamed, rowData, copyRowData, otherRowData); + } + + @Test + public void testNestedProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + GenericRowData rowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); + GenericRowData copyRowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); + GenericRowData otherRowData = GenericRowData.of(2L, GenericRowData.of(2.0f, 2.0f)); + + GenericRowData rowDataNullStruct = GenericRowData.of(1L, null); + GenericRowData copyRowDataNullStruct = GenericRowData.of(1L, null); + GenericRowData otherRowDataNullStruct = GenericRowData.of(2L, null); + + // Project id only. + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, idOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct); + + // Project lat only. + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, latOnly); + testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, latOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); + + // Project long only. + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, longOnly); + testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, longOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); + + // Project location. + Schema locationOnly = schema.select("location"); + Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, locationOnly); + testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, + locationOnly, + rowDataNullStruct, + copyRowDataNullStruct, + otherRowDataNullStruct, + true); + } + + @Test + public void testPrimitivesFullProjection() { + DataGenerator dataGenerator = new DataGenerators.Primitives(); + Schema schema = dataGenerator.icebergSchema(); + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + GenericRowData otherRowData = dataGenerator.generateFlinkRowData(); + // modify the string field value (position 6) + otherRowData.setField(6, StringData.fromString("foo_bar")); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + setOptionalFieldsNullForPrimitives(rowDataNullOptionalFields); + GenericRowData copyRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + setOptionalFieldsNullForPrimitives(copyRowDataNullOptionalFields); + GenericRowData otherRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + // modify the string field value (position 6) + otherRowDataNullOptionalFields.setField(6, StringData.fromString("foo_bar")); + setOptionalFieldsNullForPrimitives(otherRowData); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + private void setOptionalFieldsNullForPrimitives(GenericRowData rowData) { + // fields from [1, 5] range are optional + for (int pos = 1; pos <= 5; ++pos) { + rowData.setField(pos, null); + } + } + + @Test + public void testMapOfPrimitivesProjection() { + DataGenerator dataGenerator = new DataGenerators.MapOfPrimitives(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project map only. + Schema mapOnly = schema.select("map_of_primitives"); + Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, mapOnly); + + // Project all. + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(StringData.fromString("foo"), 1, StringData.fromString("bar"), 2))); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData, true); + testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("row_id_value"), null); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("row_id_value"), null); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("other_row_id_value"), null); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + mapOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields, + true); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testMapOfStructStructProjection() { + DataGenerator dataGenerator = new DataGenerators.MapOfStructStruct(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project map only. + Schema mapOnly = schema.select("map"); + Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, mapOnly); + + // Project all. + generateAndValidate(schema, schema); + + // Project partial map key. + Schema partialMapKey = + new Schema( + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()), + Types.NestedField.required(204, "valueData", Types.StringType.get()))))); + Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial map key or value struct."); + + // Project partial map key. + Schema partialMapValue = + new Schema( + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get()), + Types.NestedField.required(202, "keyData", Types.StringType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()))))); + Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial map key or value struct."); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericMapData( + ImmutableMap.of( + GenericRowData.of(1L, StringData.fromString("other_key_data")), + GenericRowData.of(1L, StringData.fromString("other_value_data"))))); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(2L, null), GenericRowData.of(2L, null)))); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + mapOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testArrayOfPrimitiveProjection() { + DataGenerator dataGenerator = new DataGenerators.ArrayOfPrimitive(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project list only. + Schema arrayOnly = schema.select("array_of_int"); + Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, arrayOnly); + + // Project all. + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericArrayData(new Integer[] {4, 5, 6})); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, arrayOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericArrayData(new Integer[] {4, null, 6})); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + arrayOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testArrayOfStructProjection() { + DataGenerator dataGenerator = new DataGenerators.ArrayOfStruct(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project list only. + Schema arrayOnly = schema.select("array_of_struct"); + Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, arrayOnly); + + // Project all. + generateAndValidate(schema, schema); + + // Project partial list value. + Schema partialList = + new Schema( + Types.NestedField.optional( + 2, + "array_of_struct", + Types.ListType.ofOptional( + 101, + Types.StructType.of( + Types.NestedField.required(202, "name", Types.StringType.get()))))); + + Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial list element struct."); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(new Integer[] {4, 5, 6})); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {4, null, 6})); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + private void generateAndValidate(Schema schema, Schema projectSchema) { + int numRecords = 100; + List recordList = RandomGenericData.generate(schema, numRecords, 102L); + List rowDataList = + Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); + Assertions.assertThat(rowDataList).hasSize(recordList.size()); + + StructProjection structProjection = StructProjection.create(schema, projectSchema); + RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); + + for (int i = 0; i < numRecords; i++) { + StructLike expected = structProjection.wrap(recordList.get(i)); + RowData projected = rowDataProjection.wrap(rowDataList.get(i)); + TestHelpers.assertRowData(projectSchema, expected, projected); + + Assertions.assertThat(projected).isEqualTo(projected); + Assertions.assertThat(projected).hasSameHashCodeAs(projected); + // make sure toString doesn't throw NPE for null values + Assertions.assertThatNoException().isThrownBy(projected::toString); + } + } + + private void testEqualsAndHashCode( + Schema schema, + Schema projectionSchema, + RowData rowData, + RowData copyRowData, + RowData otherRowData) { + testEqualsAndHashCode(schema, projectionSchema, rowData, copyRowData, otherRowData, false); + } + + /** + * @param isOtherRowDataSameAsRowData sometimes projection on otherRowData can result in the same + * RowData, e.g. due to empty projection or null struct + */ + private void testEqualsAndHashCode( + Schema schema, + Schema projectionSchema, + RowData rowData, + RowData copyRowData, + RowData otherRowData, + boolean isOtherRowDataSameAsRowData) { + RowDataProjection projection = RowDataProjection.create(schema, projectionSchema); + RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); + RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); + + Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); + Assertions.assertThat(projection.wrap(rowData)) + .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); + + if (isOtherRowDataSameAsRowData) { + Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); + Assertions.assertThat(projection.wrap(rowData)) + .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); + } else { + Assertions.assertThat(projection.wrap(rowData)) + .isNotEqualTo(otherProjection.wrap(otherRowData)); + Assertions.assertThat(projection.wrap(rowData)) + .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); + } + } +} 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 new file mode 100644 index 000000000000..7dd4e8759c0e --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -0,0 +1,596 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.withPrecision; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +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; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Files; +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.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestRowProjection { + + @TempDir private Path temp; + + private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) + throws IOException { + File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); + assertThat(file.delete()).isTrue(); + + try (FileAppender appender = + Avro.write(Files.localOutput(file)) + .schema(writeSchema) + .createWriterFunc(ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(writeSchema))) + .build()) { + appender.add(row); + } + + Iterable records = + Avro.read(Files.localInput(file)) + .project(readSchema) + .createReaderFunc(FlinkAvroReader::new) + .build(); + + return Iterables.getOnlyElement(records); + } + + @Test + public void testFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData projected = writeAndRead("full_projection", schema, schema, row); + + assertThat(projected.getLong(0)).isEqualTo(34); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + } + + @Test + public void testSpecialCharacterProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "user id", Types.LongType.get()), + Types.NestedField.optional(1, "data%0", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData full = writeAndRead("special_chars", schema, schema, row); + + assertThat(full.getLong(0)).isEqualTo(34L); + assertThat(full.getString(1)).asString().isEqualTo("test"); + + RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); + + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + } + + @Test + public void testReorderedFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("full_projection", schema, reordered, row); + + assertThat(projected.getString(0)).asString().isEqualTo("test"); + assertThat(projected.getLong(1)).isEqualTo(34); + } + + @Test + public void testReorderedProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema reordered = + new Schema( + Types.NestedField.optional(2, "missing_1", Types.StringType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(3, "missing_2", Types.LongType.get())); + + RowData projected = writeAndRead("full_projection", schema, reordered, row); + + assertThat(projected.isNullAt(0)).isTrue(); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + assertThat(projected.isNullAt(2)).isTrue(); + } + + @Test + public void testRenamedAddedField() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(1, "a", Types.LongType.get()), + Types.NestedField.required(2, "b", Types.LongType.get()), + Types.NestedField.required(3, "d", Types.LongType.get())); + + RowData row = GenericRowData.of(100L, 200L, 300L); + + Schema renamedAdded = + new Schema( + Types.NestedField.optional(1, "a", Types.LongType.get()), + Types.NestedField.optional(2, "b", Types.LongType.get()), + Types.NestedField.optional(3, "c", Types.LongType.get()), + Types.NestedField.optional(4, "d", Types.LongType.get())); + + RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.getLong(1)) + .as("Should contain the correct value in column 2") + .isEqualTo(200L); + assertThat(projected.getLong(2)) + .as("Should contain the correct value in column 1") + .isEqualTo(300L); + assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); + } + + @Test + public void testEmptyProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); + + assertThat(projected).isNotNull(); + assertThat(projected.getArity()).isEqualTo(0); + } + + @Test + public void testBasicProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); + assertThat(projected.getArity()).as("Should not project data").isEqualTo(1); + assertThat(projected.getLong(0)).isEqualTo(34L); + + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + + projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); + + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + } + + @Test + public void testRename() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema readSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + + RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); + + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getString(1)) + .as("Should contain the correct data/renamed value") + .asString() + .isEqualTo("test"); + } + + @Test + public void testNestedStructProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + RowData location = GenericRowData.of(52.995143f, -1.539054f); + RowData record = GenericRowData.of(34L, location); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getLong(0)).as("Should contain the correct id value").isEqualTo(34L); + + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + + projected = writeAndRead("latitude_only", writeSchema, latOnly, record); + RowData projectedLocation = projected.getRow(0, 1); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project longitude").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + + projected = writeAndRead("longitude_only", writeSchema, longOnly, record); + projectedLocation = projected.getRow(0, 1); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project latitutde").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); + + Schema locationOnly = writeSchema.select("location"); + projected = writeAndRead("location_only", writeSchema, locationOnly, record); + projectedLocation = projected.getRow(0, 1); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedLocation.getFloat(1)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); + } + + @Test + public void testMapProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "properties", + Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); + + GenericMapData properties = + new GenericMapData( + ImmutableMap.of( + StringData.fromString("a"), + StringData.fromString("A"), + StringData.fromString("b"), + StringData.fromString("B"))); + + RowData row = GenericRowData.of(34L, properties); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project properties map").isEqualTo(1); + + Schema keyOnly = writeSchema.select("properties.key"); + projected = writeAndRead("key_only", writeSchema, keyOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); + + Schema valueOnly = writeSchema.select("properties.value"); + projected = writeAndRead("value_only", writeSchema, valueOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); + + Schema mapOnly = writeSchema.select("properties"); + projected = writeAndRead("map_only", writeSchema, mapOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + 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; + } + + @Test + public void testMapOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get()))))); + + RowData l1 = GenericRowData.of(53.992811f, -1.542616f); + RowData l2 = GenericRowData.of(52.995143f, -1.539054f); + GenericMapData map = + new GenericMapData( + ImmutableMap.of(StringData.fromString("L1"), l1, StringData.fromString("L2"), l2)); + RowData row = GenericRowData.of(34L, map); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project locations map").isEqualTo(1); + + projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(row.getMap(1)); + + projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); + GenericMapData locations = (GenericMapData) projected.getMap(0); + assertThat(locations).isNotNull(); + GenericArrayData l1l2Array = + new GenericArrayData( + new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); + RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain lat") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + assertThat(projectedL1.getArity()).as("L1 should not contain long").isEqualTo(1); + RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain lat") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedL2.getArity()).as("L2 should not contain long").isEqualTo(1); + + projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + locations = (GenericMapData) projected.getMap(0); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); + projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getArity()).as("L1 should not contain lat").isEqualTo(1); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain long") + .isEqualTo(-1.542616f, withPrecision(0.000001f)); + projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getArity()).as("L2 should not contain lat").isEqualTo(1); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain long") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); + + Schema latitiudeRenamed = + new Schema( + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); + + projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + locations = (GenericMapData) projected.getMap(0); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); + projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain latitude") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + } + + @Test + public void testListProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); + + GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L}); + + RowData row = GenericRowData.of(34L, values); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project values list").isEqualTo(1); + + Schema elementOnly = writeSchema.select("values.element"); + projected = writeAndRead("element_only", writeSchema, elementOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); + + Schema listOnly = writeSchema.select("values"); + projected = writeAndRead("list_only", writeSchema, listOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); + } + + @Test + @SuppressWarnings("unchecked") + public void testListOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.required(19, "x", Types.IntegerType.get()), + Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); + + RowData p1 = GenericRowData.of(1, 2); + RowData p2 = GenericRowData.of(3, null); + GenericArrayData arrayData = new GenericArrayData(new RowData[] {p1, p2}); + RowData row = GenericRowData.of(34L, arrayData); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).isEqualTo(1); + + projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(row.getArray(1)); + + projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); + ArrayData points = projected.getArray(0); + assertThat(points.size()).isEqualTo(2); + RowData projectedP1 = points.getRow(0, 2); + assertThat(projectedP1.getInt(0)).as("Should project x").isEqualTo(1); + assertThat(projectedP1.getArity()).as("Should not project y").isEqualTo(1); + RowData projectedP2 = points.getRow(1, 2); + assertThat(projectedP2.getArity()).as("Should not project y").isEqualTo(1); + assertThat(projectedP2.getInt(0)).as("Should project x").isEqualTo(3); + + projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); + points = projected.getArray(0); + assertThat(points.size()).isEqualTo(2); + projectedP1 = points.getRow(0, 2); + assertThat(projectedP1.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project y").isEqualTo(2); + projectedP2 = points.getRow(1, 2); + assertThat(projectedP2.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null y").isTrue(); + + Schema yRenamed = + new Schema( + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); + + projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); + points = projected.getArray(0); + assertThat(points.size()).isEqualTo(2); + projectedP1 = points.getRow(0, 2); + assertThat(projectedP1.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project z").isEqualTo(2); + projectedP2 = points.getRow(1, 2); + assertThat(projectedP2.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); + } + + @Test + public void testAddedFieldsWithRequiredChildren() throws Exception { + Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); + + RowData row = GenericRowData.of(100L); + + Schema addedFields = + new Schema( + Types.NestedField.optional(1, "a", Types.LongType.get()), + Types.NestedField.optional( + 2, + "b", + Types.StructType.of(Types.NestedField.required(3, "c", Types.LongType.get()))), + Types.NestedField.optional(4, "d", Types.ListType.ofRequired(5, Types.LongType.get())), + Types.NestedField.optional( + 6, + "e", + Types.MapType.ofRequired(7, 8, Types.LongType.get(), Types.LongType.get()))); + + RowData projected = + writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.isNullAt(1)).as("Should contain empty value in new column 2").isTrue(); + assertThat(projected.isNullAt(2)).as("Should contain empty value in new column 4").isTrue(); + assertThat(projected.isNullAt(3)).as("Should contain empty value in new column 6").isTrue(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java new file mode 100644 index 000000000000..eccab20e04fc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java @@ -0,0 +1,100 @@ +/* + * 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 org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.TestHelpers; +import org.junit.jupiter.api.Test; + +public class TestStructRowData { + + protected void testConverter(DataGenerator dataGenerator) { + StructRowData converter = new StructRowData(dataGenerator.icebergSchema().asStruct()); + GenericRecord expected = dataGenerator.generateIcebergGenericRecord(); + StructRowData actual = converter.setStruct(expected); + TestHelpers.assertRowData(dataGenerator.icebergSchema(), expected, actual); + } + + @Test + public void testPrimitiveTypes() { + testConverter(new DataGenerators.Primitives()); + } + + @Test + public void testStructOfPrimitive() { + testConverter(new DataGenerators.StructOfPrimitive()); + } + + @Test + public void testStructOfArray() { + testConverter(new DataGenerators.StructOfArray()); + } + + @Test + public void testStructOfMap() { + testConverter(new DataGenerators.StructOfMap()); + } + + @Test + public void testStructOfStruct() { + testConverter(new DataGenerators.StructOfStruct()); + } + + @Test + public void testArrayOfPrimitive() { + testConverter(new DataGenerators.ArrayOfPrimitive()); + } + + @Test + public void testArrayOfArray() { + testConverter(new DataGenerators.ArrayOfArray()); + } + + @Test + public void testArrayOfMap() { + testConverter(new DataGenerators.ArrayOfMap()); + } + + @Test + public void testArrayOfStruct() { + testConverter(new DataGenerators.ArrayOfStruct()); + } + + @Test + public void testMapOfPrimitives() { + testConverter(new DataGenerators.MapOfPrimitives()); + } + + @Test + public void testMapOfArray() { + testConverter(new DataGenerators.MapOfArray()); + } + + @Test + public void testMapOfMap() { + testConverter(new DataGenerators.MapOfMap()); + } + + @Test + public void testMapOfStruct() { + testConverter(new DataGenerators.MapOfStruct()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java new file mode 100644 index 000000000000..44eb907a17aa --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.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.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.flink.AvroGenericRecordConverterBase; +import org.apache.iceberg.flink.DataGenerator; + +public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { + @Override + protected void testConverter(DataGenerator dataGenerator) throws Exception { + // Need to use avroSchema from DataGenerator because some primitive types have special Avro + // type handling. Hence the Avro schema converted from Iceberg schema won't work. + AvroGenericRecordToRowDataMapper mapper = + AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); + RowData expected = dataGenerator.generateFlinkRowData(); + RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java new file mode 100644 index 000000000000..5ebcc6361c7b --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.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.flink.sink; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionKeySelector { + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testCorrectKeySelection(TableSchemaType tableSchemaType) { + int numBuckets = 60; + + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitionKeySelector keySelector = + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); + + TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) + .forEach( + rowData -> { + int expectedBucketId = + TestBucketPartitionerUtil.computeBucketId( + numBuckets, rowData.getString(1).toString()); + Integer key = keySelector.getKey(rowData); + Assertions.assertThat(key).isEqualTo(expectedBucketId); + }); + } + + @Test + public void testKeySelectorMultipleBucketsFail() { + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy( + () -> + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java new file mode 100644 index 000000000000..835713e6b417 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.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.flink.sink; + +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +public class TestBucketPartitioner { + + static final int DEFAULT_NUM_BUCKETS = 60; + + @ParameterizedTest + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) + public void testPartitioningParallelismGreaterThanBuckets( + String schemaTypeStr, String numBucketsStr) { + int numPartitions = 500; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + int bucketId = 0; + for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); + bucketId++; + if (bucketId == numBuckets) { + bucketId = 0; + } + } + } + + @ParameterizedTest + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) + public void testPartitioningParallelismEqualLessThanBuckets( + String schemaTypeStr, String numBucketsStr) { + int numPartitions = 30; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); + } + } + + @Test + public void testPartitionerBucketIdNullFail() { + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) + .withMessage(BUCKET_NULL_MESSAGE); + } + + @Test + public void testPartitionerMultipleBucketsFail() { + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); + + Assertions.assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitioner(partitionSpec)) + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } + + @Test + public void testPartitionerBucketIdOutOfRangeFail() { + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + int negativeBucketId = -1; + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) + .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); + + int tooBigBucketId = DEFAULT_NUM_BUCKETS; + Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) + .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); + } +} 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 new file mode 100644 index 000000000000..9dae43ce5e58 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.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.sink; + +import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +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.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.CloseableIterable; +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.assertj.core.api.Assertions; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionerFlinkIcebergSink { + + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension catalogExtension = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + private static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + // Parallelism = 8 (parallelism > numBuckets) throughout the test suite + private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; + private final FileFormat format = FileFormat.PARQUET; + private final int numBuckets = 4; + + private Table table; + private StreamExecutionEnvironment env; + private TableLoader tableLoader; + + private void setupEnvironment(TableSchemaType tableSchemaType) { + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + table = + catalogExtension + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitionSpec, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + env = + StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism * 2); + tableLoader = catalogExtension.tableLoader(); + } + + private void appendRowsToTable(List allRows) throws Exception { + DataFormatConverters.RowConverter converter = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + DataStream dataStream = + env.addSource( + new BoundedTestSource<>( + allRows.stream().map(converter::toExternal).toArray(Row[]::new)), + ROW_TYPE_INFO) + .map(converter::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)) + .partitionCustom( + new BucketPartitioner(table.spec()), + new BucketPartitionKeySelector( + table.spec(), + table.schema(), + FlinkSink.toFlinkRowType(table.schema(), SimpleDataUtil.FLINK_SCHEMA))); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.NONE) + .append(); + + env.execute("Test Iceberg DataStream"); + + SimpleDataUtil.assertTableRows(table, allRows); + } + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { + setupEnvironment(tableSchemaType); + List rows = generateTestDataRows(); + + appendRowsToTable(rows); + TableTestStats stats = extractPartitionResults(tableSchemaType); + + Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); + // All 4 buckets should've been written to + Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); + Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); + // Writer expectation (2 writers per bucket): + // - Bucket0 -> Writers [0, 4] + // - Bucket1 -> Writers [1, 5] + // - Bucket2 -> Writers [2, 6] + // - Bucket3 -> Writers [3, 7] + for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { + Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); + // 2 files per bucket (one file is created by each writer) + Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); + // 2 rows per file (total of 16 rows across 8 files) + Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); + } + } + + /** + * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 + * buckets) + */ + private List generateTestDataRows() { + int totalNumRows = parallelism * 2; + int numRowsPerBucket = totalNumRows / numBuckets; + return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); + } + + private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) + throws IOException { + int totalRecordCount = 0; + Map> writersPerBucket = Maps.newHashMap(); // > + Map filesPerBucket = Maps.newHashMap(); // + Map rowsPerWriter = Maps.newHashMap(); // + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + long recordCountInFile = scanTask.file().recordCount(); + + String[] splitFilePath = scanTask.file().path().toString().split("/"); + // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet + // Writer ID: .......^^^^^ + String filename = splitFilePath[splitFilePath.length - 1]; + int writerId = Integer.parseInt(filename.split("-")[0]); + + totalRecordCount += recordCountInFile; + int bucketId = + scanTask + .file() + .partition() + .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); + writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); + writersPerBucket.get(bucketId).add(writerId); + filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); + rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); + } + } + + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); + } + + /** DTO to hold Test Stats */ + private static class TableTestStats { + final int totalRowCount; + final Map> writersPerBucket; + final Map numFilesPerBucket; + final Map rowsPerWriter; + + TableTestStats( + int totalRecordCount, + Map> writersPerBucket, + Map numFilesPerBucket, + Map rowsPerWriter) { + this.totalRowCount = totalRecordCount; + this.writersPerBucket = writersPerBucket; + this.numFilesPerBucket = numFilesPerBucket; + this.rowsPerWriter = rowsPerWriter; + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java new file mode 100644 index 000000000000..e1309bfac6d5 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.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.flink.sink; + +import java.util.List; +import java.util.UUID; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.BucketUtil; + +final class TestBucketPartitionerUtil { + + enum TableSchemaType { + ONE_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 0; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); + } + }, + IDENTITY_AND_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .identity("id") + .bucket("data", numBuckets) + .build(); + } + }, + TWO_BUCKETS { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .bucket("id", numBuckets) + .bucket("data", numBuckets) + .build(); + } + }; + + public abstract int bucketPartitionColumnPosition(); + + public abstract PartitionSpec getPartitionSpec(int numBuckets); + } + + private TestBucketPartitionerUtil() {} + + /** + * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to + * numBuckets - 1) + * + * @param numRowsPerBucket how many different rows should be generated per bucket + * @param numBuckets max number of buckets to consider + * @return the list of rows whose data "hashes" to the desired bucketId + */ + static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { + List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); + // For some of our tests, this order of the generated rows matters + for (int i = 0; i < numRowsPerBucket; i++) { + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + String value = generateValueForBucketId(bucketId, numBuckets); + rows.add(GenericRowData.of(1, StringData.fromString(value))); + } + } + return rows; + } + + /** + * Utility method to generate a UUID string that will "hash" to a desired bucketId + * + * @param bucketId the desired bucketId + * @return the string data that "hashes" to the desired bucketId + */ + private static String generateValueForBucketId(int bucketId, int numBuckets) { + while (true) { + String uuid = UUID.randomUUID().toString(); + if (computeBucketId(numBuckets, uuid) == bucketId) { + return uuid; + } + } + } + + /** + * Utility that performs the same hashing/bucketing mechanism used by Bucket.java + * + * @param numBuckets max number of buckets to consider + * @param value the string to compute the bucketId from + * @return the computed bucketId + */ + static int computeBucketId(int numBuckets, String value) { + return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java new file mode 100644 index 000000000000..360db658cd2f --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java @@ -0,0 +1,81 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; + +public class TestCachingTableSupplier { + + @Test + public void testCheckArguments() { + SerializableTable initialTable = mock(SerializableTable.class); + + Table loadedTable = mock(Table.class); + TableLoader tableLoader = mock(TableLoader.class); + when(tableLoader.loadTable()).thenReturn(loadedTable); + + new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); + + assertThatThrownBy(() -> new CachingTableSupplier(initialTable, tableLoader, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("tableRefreshInterval cannot be null"); + assertThatThrownBy(() -> new CachingTableSupplier(null, tableLoader, Duration.ofMillis(100))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("initialTable cannot be null"); + assertThatThrownBy(() -> new CachingTableSupplier(initialTable, null, Duration.ofMillis(100))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("tableLoader cannot be null"); + } + + @Test + public void testTableReload() { + SerializableTable initialTable = mock(SerializableTable.class); + + Table loadedTable = mock(Table.class); + TableLoader tableLoader = mock(TableLoader.class); + when(tableLoader.loadTable()).thenReturn(loadedTable); + + CachingTableSupplier cachingTableSupplier = + new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); + + // refresh shouldn't do anything as the min reload interval hasn't passed + cachingTableSupplier.refreshTable(); + assertThat(cachingTableSupplier.get()).isEqualTo(initialTable); + + // refresh after waiting past the min reload interval + Awaitility.await() + .atLeast(100, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + cachingTableSupplier.refreshTable(); + assertThat(cachingTableSupplier.get()).isEqualTo(loadedTable); + }); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java new file mode 100644 index 000000000000..14f12422da96 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.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.sink; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestCompressionSettings { + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + private Table table; + + private final Map initProperties; + + @Parameterized.Parameters(name = "tableProperties = {0}") + public static Object[] parameters() { + return new Object[] { + ImmutableMap.of(), + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + }; + } + + public TestCompressionSettings(Map initProperties) { + this.initProperties = initProperties; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); + } + + @Test + public void testCompressionAvro() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); + + if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { + Assert.assertEquals( + TableProperties.AVRO_COMPRESSION_DEFAULT, + resultProperties.get(TableProperties.AVRO_COMPRESSION)); + Assert.assertEquals( + TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, + resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } else { + Assert.assertEquals( + initProperties.get(TableProperties.AVRO_COMPRESSION), + resultProperties.get(TableProperties.AVRO_COMPRESSION)); + Assert.assertEquals( + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), + resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } + + // Override compression to snappy and some random level + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "AVRO", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_LEVEL.key(), + "6")); + + Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); + Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } + + @Test + public void testCompressionParquet() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); + + if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { + Assert.assertEquals( + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, + resultProperties.get(TableProperties.PARQUET_COMPRESSION)); + Assert.assertEquals( + TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, + resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + } else { + Assert.assertEquals( + initProperties.get(TableProperties.PARQUET_COMPRESSION), + resultProperties.get(TableProperties.PARQUET_COMPRESSION)); + Assert.assertEquals( + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), + resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + } + + // Override compression to snappy and some random level + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "PARQUET", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_LEVEL.key(), + "6")); + + Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); + Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + } + + @Test + public void testCompressionOrc() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); + + if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { + Assert.assertEquals( + TableProperties.ORC_COMPRESSION_DEFAULT, + resultProperties.get(TableProperties.ORC_COMPRESSION)); + Assert.assertEquals( + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, + resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } else { + Assert.assertEquals( + initProperties.get(TableProperties.ORC_COMPRESSION), + resultProperties.get(TableProperties.ORC_COMPRESSION)); + Assert.assertEquals( + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), + resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } + + // Override compression to snappy and a different strategy + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "ORC", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_STRATEGY.key(), + "speed")); + + Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); + Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } + + private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); + FlinkWriteConf flinkWriteConfig = + new FlinkWriteConf( + icebergTable, override, new org.apache.flink.configuration.Configuration()); + + IcebergStreamWriter streamWriter = + FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); + + harness.setup(); + harness.open(); + + return harness; + } + + private static Map appenderProperties( + Table table, TableSchema schema, Map override) throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter(table, schema, override)) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + + testHarness.prepareSnapshotPreBarrier(1L); + DynFields.BoundField operatorField = + DynFields.builder() + .hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator") + .build(testHarness.getOperatorFactory()); + DynFields.BoundField writerField = + DynFields.builder() + .hiddenImpl(IcebergStreamWriter.class, "writer") + .build(operatorField.get()); + DynFields.BoundField appenderField = + DynFields.builder() + .hiddenImpl(BaseTaskWriter.class, "appenderFactory") + .build(writerField.get()); + DynFields.BoundField> propsField = + DynFields.builder() + .hiddenImpl(FlinkAppenderFactory.class, "props") + .build(appenderField.get()); + return propsField.get(); + } + } +} 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 new file mode 100644 index 000000000000..21f3ee2c655a --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -0,0 +1,429 @@ +/* + * 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 static org.apache.iceberg.flink.SimpleDataUtil.createDelete; +import static org.apache.iceberg.flink.SimpleDataUtil.createInsert; +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; +import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; +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.nio.file.Path; +import java.nio.file.Paths; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.FileContent; +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.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +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.types.Types; +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 class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); + } + + @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"); + } + + private int idFieldId() { + return table.schema().findField("id").fieldId(); + } + + private int dataFieldId() { + return table.schema().findField("data").fieldId(); + } + + private void testCdcEvents(boolean partitioned) throws IOException { + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + // Start the 1th transaction. + TaskWriter writer = taskWriterFactory.create(); + + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "bbb")); + writer.write(createInsert(3, "ccc")); + + // Update <2, 'bbb'> to <2, 'ddd'> + writer.write(createUpdateBefore(2, "bbb")); // 1 pos-delete and 1 eq-delete. + writer.write(createUpdateAfter(2, "ddd")); + + // Update <1, 'aaa'> to <1, 'eee'> + writer.write(createUpdateBefore(1, "aaa")); // 1 pos-delete and 1 eq-delete. + writer.write(createUpdateAfter(1, "eee")); + + // Insert <4, 'fff'> + writer.write(createInsert(4, "fff")); + // Insert <5, 'ggg'> + writer.write(createInsert(5, "ggg")); + + // Delete <3, 'ccc'> + writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); + + // Start the 2nd transaction. + writer = taskWriterFactory.create(); + + // Update <2, 'ddd'> to <6, 'hhh'> - (Update both key and value) + writer.write(createUpdateBefore(2, "ddd")); // 1 eq-delete + writer.write(createUpdateAfter(6, "hhh")); + + // Update <5, 'ggg'> to <5, 'iii'> + writer.write(createUpdateBefore(5, "ggg")); // 1 eq-delete + writer.write(createUpdateAfter(5, "iii")); + + // Delete <4, 'fff'> + writer.write(createDelete(4, "fff")); // 1 eq-delete. + + result = writer.complete(); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); + } + + @TestTemplate + public void testUnpartitioned() throws IOException { + createAndInitTable(false); + testCdcEvents(false); + } + + @TestTemplate + public void testPartitioned() throws IOException { + createAndInitTable(true); + testCdcEvents(true); + } + + private void testWritePureEqDeletes(boolean partitioned) throws IOException { + createAndInitTable(partitioned); + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + writer.write(createDelete(1, "aaa")); + writer.write(createDelete(2, "bbb")); + writer.write(createDelete(3, "ccc")); + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); + commitTransaction(result); + + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); + } + + @TestTemplate + public void testUnpartitionedPureEqDeletes() throws IOException { + testWritePureEqDeletes(false); + } + + @TestTemplate + public void testPartitionedPureEqDeletes() throws IOException { + testWritePureEqDeletes(true); + } + + private void testAbort(boolean partitioned) throws IOException { + createAndInitTable(partitioned); + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + for (int i = 0; i < 8_000; i += 2) { + writer.write(createUpdateBefore(i + 1, "aaa")); + writer.write(createUpdateAfter(i + 1, "aaa")); + + writer.write(createUpdateBefore(i + 2, "bbb")); + writer.write(createUpdateAfter(i + 2, "bbb")); + } + + // Assert the current data/delete file count. + List files = + Files.walk(Paths.get(tableDir.getPath(), "data")) + .filter(p -> p.toFile().isFile()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(files).hasSize(partitioned ? 4 : 2); + + writer.abort(); + for (Path file : files) { + assertThat(file).doesNotExist(); + } + } + + @TestTemplate + public void testUnpartitionedAbort() throws IOException { + testAbort(false); + } + + @TestTemplate + public void testPartitionedAbort() throws IOException { + testAbort(true); + } + + @TestTemplate + public void testPartitionedTableWithDataAsKey() throws IOException { + createAndInitTable(true); + List equalityFieldIds = Lists.newArrayList(dataFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + // Start the 1th transaction. + TaskWriter writer = taskWriterFactory.create(); + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "aaa")); + writer.write(createInsert(3, "bbb")); + writer.write(createInsert(4, "ccc")); + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); + + // Start the 2nd transaction. + writer = taskWriterFactory.create(); + writer.write(createInsert(5, "aaa")); + writer.write(createInsert(6, "bbb")); + writer.write(createDelete(7, "ccc")); // 1 eq-delete. + + result = writer.complete(); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); + } + + @TestTemplate + public void testPartitionedTableWithDataAndIdAsKey() throws IOException { + createAndInitTable(true); + List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "aaa")); + + writer.write(createDelete(2, "aaa")); // 1 pos-delete. + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); + commitTransaction(result); + + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); + } + + @TestTemplate + public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { + Schema tableSchema = + new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "ts", Types.TimestampType.withZone())); + RowType flinkType = + new RowType( + false, + ImmutableList.of( + new RowType.RowField("id", new IntType()), + new RowType.RowField("ts", new LocalZonedTimestampType(3)))); + + this.table = create(tableSchema, PartitionSpec.unpartitioned()); + initTable(table); + + List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + RowDataSerializer serializer = new RowDataSerializer(flinkType); + OffsetDateTime start = OffsetDateTime.now(); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + + WriteResult result = writer.complete(); + // One data file + assertThat(result.dataFiles()).hasSize(1); + // One eq delete file + one pos delete file + assertThat(result.deleteFiles()).hasSize(2); + assertThat( + Arrays.stream(result.deleteFiles()) + .map(ContentFile::content) + .collect(Collectors.toSet())) + .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); + commitTransaction(result); + + Record expectedRecord = GenericRecord.create(tableSchema); + expectedRecord.setField("id", 1); + int cutPrecisionNano = start.getNano() / 1000000 * 1000000; + expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); + + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + } + + private void commitTransaction(WriteResult result) { + RowDelta rowDelta = table.newRowDelta(); + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + rowDelta + .validateDeletedFiles() + .validateDataFilesExist(Lists.newArrayList(result.referencedDataFiles())) + .commit(); + } + + private StructLikeSet expectedRowSet(Record... records) { + return SimpleDataUtil.expectedRowSet(table, records); + } + + private StructLikeSet actualRowSet(String... columns) throws IOException { + return SimpleDataUtil.actualRowSet(table, columns); + } + + private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + FlinkSchemaUtil.convert(table.schema()), + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + false); + } + + private TaskWriterFactory createTaskWriterFactory( + RowType flinkType, List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkType, + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + true); + } + + private void createAndInitTable(boolean partitioned) { + if (partitioned) { + this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); + } else { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + initTable(table); + } + + private void initTable(TestTables.TestTable testTable) { + testTable + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) + .defaultFormat(format) + .commit(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java new file mode 100644 index 000000000000..dd89f43483b0 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.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.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.TestAppenderFactory; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkAppenderFactory extends TestAppenderFactory { + + private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); + + @Override + protected FileAppenderFactory createAppenderFactory( + List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { + return new FlinkAppenderFactory( + table, + table.schema(), + rowType, + table.properties(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteSchema, + posDeleteRowSchema); + } + + @Override + protected RowData createRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet expectedRowSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(rowType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java new file mode 100644 index 000000000000..414ee40d1357 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestFileWriterFactory; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkFileWriterFactory extends TestFileWriterFactory { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java new file mode 100644 index 000000000000..11a73d2cc144 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -0,0 +1,397 @@ +/* + * 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.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +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.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private TableLoader tableLoader; + + private final FileFormat format; + private final int parallelism; + private final boolean partitioned; + + @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", 1, true}, + {"avro", 1, false}, + {"avro", 2, true}, + {"avro", 2, false}, + {"orc", 1, true}, + {"orc", 1, false}, + {"orc", 2, true}, + {"orc", 2, false}, + {"parquet", 1, true}, + {"parquet", 1, false}, + {"parquet", 2, true}, + {"parquet", 2, false} + }; + } + + public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { + this.format = FileFormat.fromString(format); + this.parallelism = parallelism; + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + private int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } + + @Test + public void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @Test + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @Test + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); + } + } + } + + @Test + public void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @Test + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + Assert.assertEquals( + "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); + Assert.assertEquals( + "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); + Assert.assertEquals( + "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + } + } + + @Test + public void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + Assert.assertEquals( + "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); + Assert.assertEquals( + "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); + Assert.assertEquals( + "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + } + } + + @Test + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + Assert.assertEquals( + "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); + Assert.assertEquals( + "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); + Assert.assertEquals( + "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + } + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + catalogResource + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + catalogResource + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); + Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); + rightTable.refresh(); + Assert.assertEquals( + TestFlinkIcebergSink.class.getName(), + rightTable.currentSnapshot().summary().get("flink.test")); + Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); + } + + @Test + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + Assertions.assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + Assertions.assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java new file mode 100644 index 000000000000..b38aa6b50ce6 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.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.sink; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.types.Row; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class TestFlinkIcebergSinkBase { + + protected Table table; + protected StreamExecutionEnvironment env; + protected static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + protected static final DataFormatConverters.RowConverter CONVERTER = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + protected BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(rows.toArray(new Row[0])); + } + + protected List createRows(String prefix) { + return Lists.newArrayList( + Row.of(1, prefix + "aaa"), + Row.of(1, prefix + "bbb"), + Row.of(1, prefix + "ccc"), + Row.of(2, prefix + "aaa"), + Row.of(2, prefix + "bbb"), + Row.of(2, prefix + "ccc"), + Row.of(3, prefix + "aaa"), + Row.of(3, prefix + "bbb"), + Row.of(3, prefix + "ccc")); + } + + protected List convertToRowData(List rows) { + return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java new file mode 100644 index 000000000000..16b4542b00d3 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.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.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private final String branch; + private TableLoader tableLoader; + + @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") + public static Object[] parameters() { + return new Object[] {"main", "testBranch"}; + } + + public TestFlinkIcebergSinkBranch(String branch) { + this.branch = branch; + } + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + Assert.assertNull(table.currentSnapshot()); + } + + Assert.assertTrue(table.snapshot(otherBranch) == null); + } +} 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 new file mode 100644 index 000000000000..b5c3bcf41734 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -0,0 +1,236 @@ +/* + * 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 org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.Sets; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameterized.Parameters( + name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} + }; + } + + public TestFlinkIcebergSinkV2( + String format, int parallelism, boolean partitioned, String writeDistributionMode) { + this.format = FileFormat.fromString(format); + this.parallelism = parallelism; + this.partitioned = partitioned; + this.writeDistributionMode = writeDistributionMode; + } + + @Before + public void setupTable() { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use schema identifier field IDs as equality field id list by default + Assert.assertEquals( + table.schema().identifierFieldIds(), + Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + Assert.assertEquals( + Sets.newHashSet(table.schema().findField("id").fieldId()), + Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + + builder.equalityFieldColumns(Lists.newArrayList("type")); + Assert.assertEquals( + Sets.newHashSet(table.schema().findField("type").fieldId()), + Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + } + + @Test + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + Assertions.assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + Assertions.assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @Test + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @Test + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java new file mode 100644 index 000000000000..0b403756cefe --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -0,0 +1,362 @@ +/* + * 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.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +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.types.RowKind; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.BoundedTestSource; +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.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.StructLikeSet; +import org.assertj.core.api.Assertions; +import org.junit.Assert; + +public class TestFlinkIcebergSinkV2Base { + + protected static final int FORMAT_V2 = 2; + protected static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + protected static final int ROW_ID_POS = 0; + protected static final int ROW_DATA_POS = 1; + + protected int parallelism = 1; + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected FileFormat format; + protected boolean partitioned; + protected String writeDistributionMode; + + protected static final Map ROW_KIND_MAP = + ImmutableMap.of( + "+I", RowKind.INSERT, + "-D", RowKind.DELETE, + "-U", RowKind.UPDATE_BEFORE, + "+U", RowKind.UPDATE_AFTER); + + protected Row row(String rowKind, int id, String data) { + RowKind kind = ROW_KIND_MAP.get(rowKind); + if (kind == null) { + throw new IllegalArgumentException("Unknown row kind: " + rowKind); + } + + return Row.ofKind(kind, id, data); + } + + protected void testUpsertOnIdDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 2, "bbb"), row("+I", 2, "ccc")), + ImmutableList.of(row("+U", 1, "bbb"), row("-U", 1, "ccc"), row("-D", 1, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "aaa"), record(2, "bbb")), + ImmutableList.of(record(1, "aaa"), record(2, "ccc")), + ImmutableList.of(record(1, "bbb"), record(2, "ccc"))); + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnIdDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 2, "bbb"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa")), + ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "aaa"), record(2, "bbb")), + ImmutableList.of( + record(1, "aaa"), record(1, "bbb"), record(1, "ccc"), record(2, "bbb")), + ImmutableList.of( + record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "bbb"))); + + testChangeLogs( + ImmutableList.of("data", "id"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnSameKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa")), + // Checkpoint #2 + ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa")), + // Checkpoint #3 + ImmutableList.of(row("-D", 1, "aaa"), row("+I", 1, "aaa")), + // Checkpoint #4 + ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa"), record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 2, "bbb"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa")), + ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"), row("+I", 2, "ccc"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "aaa")), + ImmutableList.of(record(1, "aaa"), record(1, "bbb"), record(1, "ccc")), + ImmutableList.of( + record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "ccc"))); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testUpsertOnDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), + ImmutableList.of(row("+U", 4, "aaa"), row("-U", 3, "bbb"), row("+U", 5, "bbb")), + ImmutableList.of(row("+I", 6, "aaa"), row("+U", 7, "bbb"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(2, "aaa"), record(3, "bbb")), + ImmutableList.of(record(4, "aaa"), record(5, "bbb")), + ImmutableList.of(record(6, "aaa"), record(7, "bbb"))); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnIdKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa"), + row("-D", 2, "aaa"), + row("+I", 2, "bbb")), + ImmutableList.of( + row("-U", 2, "bbb"), row("+U", 2, "ccc"), row("-D", 2, "ccc"), row("+I", 2, "ddd")), + ImmutableList.of( + row("-D", 1, "bbb"), + row("+I", 1, "ccc"), + row("-D", 1, "ccc"), + row("+I", 1, "ddd"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "bbb")), + ImmutableList.of(record(1, "bbb"), record(2, "ddd")), + ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); + + if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { + Assertions.assertThatThrownBy( + () -> + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "In 'hash' distribution mode with equality fields set, partition field") + .hasMessageContaining("should be included in equality fields:"); + + } else { + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + } + + protected void testUpsertOnIdKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), + ImmutableList.of(row("+I", 1, "ccc")), + ImmutableList.of(row("+U", 1, "ddd"), row("+I", 1, "eee"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb")), + ImmutableList.of(record(1, "ccc")), + ImmutableList.of(record(1, "eee"))); + + if (!partitioned) { + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } else { + Assertions.assertThatThrownBy( + () -> + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("should be included in equality fields:"); + } + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + Assert.assertEquals( + "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + Assert.assertEquals( + "Should have the expected records for the checkpoint#" + i, + expectedRowSet(expectedRecords.toArray(new Record[0])), + actualRowSet(snapshotId, "*")); + } + } + + protected Record record(int id, String data) { + return SimpleDataUtil.createRecord(id, data); + } + + private List findValidSnapshots() { + List validSnapshots = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.allManifests(table.io()).stream() + .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + validSnapshots.add(snapshot); + } + } + return validSnapshots; + } + + private StructLikeSet expectedRowSet(Record... records) { + return SimpleDataUtil.expectedRowSet(table, records); + } + + private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + table.refresh(); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table).useSnapshot(snapshotId).select(columns).build()) { + reader.forEach(set::add); + } + return set; + } +} 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 new file mode 100644 index 000000000000..fed333848279 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -0,0 +1,134 @@ +/* + * 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.io.IOException; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private final String branch; + + @Parameterized.Parameters(name = "branch = {0}") + public static Object[] parameters() { + return new Object[] {"main", "testBranch"}; + } + + public TestFlinkIcebergSinkV2Branch(String branch) { + this.branch = branch; + } + + @Before + public void before() throws IOException { + table = + catalogResource + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = catalogResource.tableLoader(); + } + + @Test + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @Test + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + Assert.assertNull(table.currentSnapshot()); + } + + Assert.assertTrue(table.snapshot(otherBranch) == null); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java new file mode 100644 index 000000000000..ce1f208a4b07 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -0,0 +1,315 @@ +/* + * 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 static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Pair; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkManifest { + private static final Configuration CONF = new Configuration(); + + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + private Table table; + private FileAppenderFactory appenderFactory; + private final AtomicInteger fileCount = new AtomicInteger(0); + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + String warehouse = folder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + + // Construct the iceberg table. + table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); + + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + this.appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + @Test + public void testIO() throws IOException { + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + for (long checkpointId = 1; checkpointId <= 3; checkpointId++) { + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + final long curCkpId = checkpointId; + + List dataFiles = generateDataFiles(10); + List eqDeleteFiles = generateEqDeleteFiles(5); + List posDeleteFiles = generatePosDeleteFiles(5); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(eqDeleteFiles) + .addDeleteFiles(posDeleteFiles) + .build(), + () -> factory.create(curCkpId), + table.spec()); + + WriteResult result = + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); + Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + for (int i = 0; i < dataFiles.size(); i++) { + TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); + } + Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + for (int i = 0; i < 5; i++) { + TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); + } + for (int i = 0; i < 5; i++) { + TestHelpers.assertEquals(posDeleteFiles.get(i), result.deleteFiles()[5 + i]); + } + } + } + + @Test + public void testUserProvidedManifestLocation() throws IOException { + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + File userProvidedFolder = tempFolder.newFolder(); + Map props = + ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ManifestOutputFileFactory factory = + new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(5); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder().addDataFiles(dataFiles).build(), + () -> factory.create(checkpointId), + table.spec()); + + Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); + Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); + Assert.assertEquals( + "The newly created manifest file should be located under the user provided directory", + userProvidedFolder.toPath(), + Paths.get(deltaManifests.dataManifest().path()).getParent()); + + WriteResult result = + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); + + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(5, result.dataFiles().length); + + Assert.assertEquals( + "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + for (int i = 0; i < dataFiles.size(); i++) { + TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); + } + } + + @Test + public void testVersionedSerializer() throws IOException { + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(10); + List eqDeleteFiles = generateEqDeleteFiles(10); + List posDeleteFiles = generatePosDeleteFiles(10); + DeltaManifests expected = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(eqDeleteFiles) + .addDeleteFiles(posDeleteFiles) + .build(), + () -> factory.create(checkpointId), + table.spec()); + + byte[] versionedSerializeData = + SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, expected); + DeltaManifests actual = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, versionedSerializeData); + TestHelpers.assertEquals(expected.dataManifest(), actual.dataManifest()); + TestHelpers.assertEquals(expected.deleteManifest(), actual.deleteManifest()); + + byte[] versionedSerializeData2 = + SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, actual); + Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + } + + @Test + public void testCompatibility() throws IOException { + // The v2 deserializer should be able to deserialize the v1 binary. + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(10); + ManifestFile manifest = + FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); + byte[] dataV1 = + SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); + + DeltaManifests delta = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, dataV1); + Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); + Assert.assertNotNull( + "Serialization v1 should not have null data manifest.", delta.dataManifest()); + TestHelpers.assertEquals(manifest, delta.dataManifest()); + + List actualFiles = + FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); + Assert.assertEquals(10, actualFiles.size()); + for (int i = 0; i < 10; i++) { + TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); + } + } + + private static class V1Serializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(ManifestFile m) throws IOException { + return ManifestFiles.encode(m); + } + + @Override + public ManifestFile deserialize(int version, byte[] serialized) throws IOException { + return ManifestFiles.decode(serialized); + } + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + CONF, + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private DeleteFile writePosDeleteFile(String filename, List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, positions); + } + + private List generateDataFiles(int fileNum) throws IOException { + List rowDataList = Lists.newArrayList(); + List dataFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + rowDataList.add(SimpleDataUtil.createRowData(i, "a" + i)); + dataFiles.add(writeDataFile("data-file-" + fileCount.incrementAndGet(), rowDataList)); + } + return dataFiles; + } + + private List generateEqDeleteFiles(int fileNum) throws IOException { + List rowDataList = Lists.newArrayList(); + List deleteFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + rowDataList.add(SimpleDataUtil.createDelete(i, "a" + i)); + deleteFiles.add( + writeEqDeleteFile("eq-delete-file-" + fileCount.incrementAndGet(), rowDataList)); + } + return deleteFiles; + } + + private List generatePosDeleteFiles(int fileNum) throws IOException { + List> positions = Lists.newArrayList(); + List deleteFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + positions.add(Pair.of("data-file-1", (long) i)); + deleteFiles.add( + writePosDeleteFile("pos-delete-file-" + fileCount.incrementAndGet(), positions)); + } + return deleteFiles; + } + + private static String newFlinkJobId() { + return UUID.randomUUID().toString(); + } + + private static String newOperatorUniqueId() { + return UUID.randomUUID().toString(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java new file mode 100644 index 000000000000..939ed2be7dbc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.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.flink.sink; + +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPartitioningWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkPartitioningWriters extends TestPartitioningWriters { + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); + } + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java new file mode 100644 index 000000000000..3050752d1c24 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPositionDeltaWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java new file mode 100644 index 000000000000..03051b69cf87 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -0,0 +1,51 @@ +/* + * 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 org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestRollingFileWriters; +import org.apache.iceberg.util.ArrayUtil; + +public class TestFlinkRollingFileWriters extends TestRollingFileWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java new file mode 100644 index 000000000000..e6d64ef2c720 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.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.flink.sink; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestWriterMetrics; + +public class TestFlinkWriterMetrics extends TestWriterMetrics { + + public TestFlinkWriterMetrics(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileWriterFactory newWriterFactory(Table sourceTable) { + return FlinkFileWriterFactory.builderFor(sourceTable) + .dataSchema(sourceTable.schema()) + .dataFileFormat(fileFormat) + .deleteFileFormat(fileFormat) + .positionDeleteRowSchema(sourceTable.schema()) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data, boolean boolValue, Long longValue) { + GenericRowData nested = GenericRowData.of(boolValue, longValue); + GenericRowData row = GenericRowData.of(id, StringData.fromString(data), nested); + return row; + } + + @Override + public RowData toGenericRow(int value, int repeated) { + GenericRowData row = new GenericRowData(repeated); + for (int i = 0; i < repeated; i++) { + row.setField(i, value); + } + return row; + } +} 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 new file mode 100644 index 000000000000..06b6bc9a977b --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -0,0 +1,1148 @@ +/* + * 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 static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +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.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +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.util.Pair; +import org.apache.iceberg.util.ThreadPools; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { + private static final Configuration CONF = new Configuration(); + + private File flinkManifestFolder; + + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); + } + + @Override + @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()); + + table + .updateProperties() + .set(DEFAULT_FILE_FORMAT, format.name()) + .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) + .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") + .commit(); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + long checkpointId = 0; + long timestamp = 0; + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the + // future flink job + // failover won't fail. + for (int i = 1; i <= 3; i++) { + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + } + + @TestTemplate + public void testMaxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + + JobID jobId = new JobID(); + long checkpointId = 0; + long timestamp = 0; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + + assertSnapshotSize(0); + + for (int i = 1; i <= 9; i++) { + harness.snapshot(++checkpointId, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(i / 3); + } + } + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + @TestTemplate + public void testCommitTxn() throws Exception { + // Test with 3 continues checkpoints: + // 1. snapshotState for checkpoint#1 + // 2. notifyCheckpointComplete for checkpoint#1 + // 3. snapshotState for checkpoint#2 + // 4. notifyCheckpointComplete for checkpoint#2 + // 5. snapshotState for checkpoint#3 + // 6. notifyCheckpointComplete for checkpoint#3 + long timestamp = 0; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + + harness.snapshot(i, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(i); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobID, operatorId, i); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); + } + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + long timestamp = 0; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + harness.processElement(of(dataFile2), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that the two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + long timestamp = 0; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + harness.processElement(of(dataFile2), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testRecoveryFromValidSnapshot() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); + + harness.processElement(of(dataFile1), ++timestamp); + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's + // possible that we + // flink job will restore from a checkpoint with only step#1 finished. + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(newJobId)) { + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId, operatorId, -1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId, operatorId, checkpointId); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(oldJobId)) { + harness.setup(); + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 0; + timestamp = 0; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(newJobId)) { + harness.setup(); + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, 3); + assertMaxCommittedCheckpointId(newJobId, newOperatorId, -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId, newOperatorId, checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobId = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId == 0 ? -1 : checkpointId); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(checkpointId + 1, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId + 1); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId + 1); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobId = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId1, -1L); + assertMaxCommittedCheckpointId(jobId, operatorId2, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + + harness1.processElement(of(dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + + harness2.processElement(of(dataFile2), ++timestamp); + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, -1); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + harness1.processElement(of(dataFile1), ++timestamp); + harness1.snapshot(++checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + harness2.processElement(of(dataFile2), ++timestamp); + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); + } + } + + @TestTemplate + public void testBoundedStream() throws Exception { + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertFlinkManifests(0); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); + + DataFile dataFile = writeDataFile("data-1", tableRows); + harness.processElement(of(dataFile), 1); + ((BoundedOneInput) harness.getOneInputOperator()).endInput(); + + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + final long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); + + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); + + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + + // 4. process both data files and delete files. + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + + // 5. snapshotState for checkpoint#2 + harness.snapshot(++checkpoint, ++timestamp); + assertFlinkManifests(2); + + // 6. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testCommitTwoCheckpointsInSingleTxn() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(2); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); + RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); + RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + ++timestamp); + + // The 1th snapshotState. + harness.snapshot(checkpoint, ++timestamp); + + RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); + RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); + DeleteFile deleteFile2 = + writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + ++timestamp); + + // The 2nd snapshotState. + harness.snapshot(++checkpoint, ++timestamp); + + // Notify the 2nd snapshot to complete. + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(2); + } + } + + @TestTemplate + public void testSpecEvolution() throws Exception { + long timestamp = 0; + int checkpointId = 0; + List rows = Lists.newArrayList(); + JobID jobId = new JobID(); + + OperatorID operatorId; + OperatorSubtaskState snapshot; + DataFile dataFile; + int specId; + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + + checkpointId++; + RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); + // table unpartitioned + dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + harness.snapshot(checkpointId, ++timestamp); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(table.spec().specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + // Change partition spec + table.refresh(); + PartitionSpec oldSpec = table.spec(); + table.updateSpec().addField("id").commit(); + + checkpointId++; + rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); + // write data with old partition spec + dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + snapshot = harness.snapshot(checkpointId, ++timestamp); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(oldSpec.specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + SimpleDataUtil.assertTableRows(table, rows, branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + checkpointId++; + RowData row = SimpleDataUtil.createRowData(checkpointId, "world" + checkpointId); + StructLike partition = new PartitionData(table.spec().partitionType()); + partition.set(0, checkpointId); + dataFile = + writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); + harness.processElement(of(dataFile), ++timestamp); + rows.add(row); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(table.spec().specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, rows, branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + + private int getStagingManifestSpecId(OperatorStateStore operatorStateStore, long checkPointId) + throws Exception { + ListState> checkpointsState = + operatorStateStore.getListState(IcebergFilesCommitter.buildStateDescriptor()); + NavigableMap statedDataFiles = + Maps.newTreeMap(checkpointsState.get().iterator().next()); + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, statedDataFiles.get(checkPointId)); + return deltaManifests.dataManifest().partitionSpecId(); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, deletes); + } + + private DeleteFile writePosDeleteFile( + FileAppenderFactory appenderFactory, + String filename, + List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, positions); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + CONF, + table.location(), + format.addExtension(filename), + rows); + } + + private DataFile writeDataFile( + String filename, List rows, PartitionSpec spec, StructLike partition) + throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + spec, + CONF, + table.location(), + format.addExtension(filename), + rows, + partition); + } + + private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { + table.refresh(); + long actualId = + IcebergFilesCommitter.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toHexString(), branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + throws Exception { + TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); + return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); + } + + private static MockEnvironment createEnvironment(JobID jobID) { + return new MockEnvironmentBuilder() + .setTaskName("test task") + .setManagedMemorySize(32 * 1024) + .setInputSplitProvider(new MockInputSplitProvider()) + .setBufferSize(256) + .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) + .setExecutionConfig(new ExecutionConfig()) + .setMaxParallelism(16) + .setJobID(jobID) + .build(); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final String tablePath; + private final String branch; + private final PartitionSpec spec; + + private TestOperatorFactory(String tablePath, String branch, PartitionSpec spec) { + this.tablePath = tablePath; + this.branch = branch; + this.spec = spec; + } + + private static TestOperatorFactory of(String tablePath, String branch, PartitionSpec spec) { + return new TestOperatorFactory(tablePath, branch, spec); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters param) { + IcebergFilesCommitter committer = + new IcebergFilesCommitter( + new TestTableLoader(tablePath), + false, + Collections.singletonMap("flink.test", TestIcebergFilesCommitter.class.getName()), + ThreadPools.WORKER_THREAD_POOL_SIZE, + branch, + spec); + committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); + return (T) committer; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IcebergFilesCommitter.class; + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java new file mode 100644 index 000000000000..0968f89f55e0 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -0,0 +1,388 @@ +/* + * 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.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.WriteResult; +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.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergStreamWriter { + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + private Table table; + + private final FileFormat format; + private final boolean partitioned; + + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", true}, + {"avro", false}, + {"orc", true}, + {"orc", false}, + {"parquet", true}, + {"parquet", false} + }; + } + + public TestIcebergStreamWriter(String format, boolean partitioned) { + this.format = FileFormat.fromString(format); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + // Construct the iceberg table. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); + } + + @Test + public void testWritingTable() throws Exception { + long checkpointId = 1L; + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + // The first checkpoint + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + checkpointId = checkpointId + 1; + + // The second checkpoint + testHarness.processElement(SimpleDataUtil.createRowData(4, "foo"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(5, "bar"), 2); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + expectedDataFiles = partitioned ? 4 : 2; + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + // Commit the iceberg transaction. + AppendFiles appendFiles = table.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + + // Assert the table records. + SimpleDataUtil.assertTableRecords( + table, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), + SimpleDataUtil.createRecord(2, "world"), + SimpleDataUtil.createRecord(3, "hello"), + SimpleDataUtil.createRecord(4, "foo"), + SimpleDataUtil.createRecord(5, "bar"))); + } + } + + @Test + public void testSnapshotTwice() throws Exception { + long checkpointId = 1; + long timestamp = 1; + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); + + testHarness.prepareSnapshotPreBarrier(checkpointId++); + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + // snapshot again immediately. + for (int i = 0; i < 5; i++) { + testHarness.prepareSnapshotPreBarrier(checkpointId++); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + } + } + } + + @Test + public void testTableWithoutSnapshot() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + Assert.assertEquals(0, testHarness.extractOutputValues().size()); + } + // Even if we closed the iceberg stream writer, there's no orphan data file. + Assert.assertEquals(0, scanDataFiles().size()); + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + // Still not emit the data file yet, because there is no checkpoint. + Assert.assertEquals(0, testHarness.extractOutputValues().size()); + } + // Once we closed the iceberg stream writer, there will left an orphan data file. + Assert.assertEquals(1, scanDataFiles().size()); + } + + private Set scanDataFiles() throws IOException { + Path dataDir = new Path(table.location(), "data"); + FileSystem fs = FileSystem.get(new Configuration()); + if (!fs.exists(dataDir)) { + return ImmutableSet.of(); + } else { + Set paths = Sets.newHashSet(); + RemoteIterator iterators = fs.listFiles(dataDir, true); + while (iterators.hasNext()) { + LocatedFileStatus status = iterators.next(); + if (status.isFile()) { + Path path = status.getPath(); + if (path.getName().endsWith("." + format.toString().toLowerCase())) { + paths.add(path.toString()); + } + } + } + return paths; + } + } + + @Test + public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); + + Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); + ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); + + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + // Datafiles should not be sent again + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + } + } + + @Test + public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); + + testHarness.endInput(); + + long expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + + testHarness.prepareSnapshotPreBarrier(1L); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier + // is triggered, write should only send WriteResult once + Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + } + } + + @Test + public void testTableWithTargetFileSize() throws Exception { + // Adjust the target-file-size in table properties. + table + .updateProperties() + .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger + .commit(); + + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(SimpleDataUtil.createRowData(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + for (RowData row : rows) { + testHarness.processElement(row, 1); + } + + // snapshot the operator. + testHarness.prepareSnapshotPreBarrier(1); + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(8, result.dataFiles().length); + + // Assert that the data file have the expected records. + for (DataFile dataFile : result.dataFiles()) { + Assert.assertEquals(1000, dataFile.recordCount()); + } + + // Commit the iceberg transaction. + AppendFiles appendFiles = table.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + // Assert the table records. + SimpleDataUtil.assertTableRecords(table, records); + } + + @Test + public void testPromotedFlinkDataType() throws Exception { + Schema iSchema = + new Schema( + Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), + Types.NestedField.required(2, "smallint", Types.IntegerType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + TableSchema flinkSchema = + TableSchema.builder() + .field("tinyint", DataTypes.TINYINT().notNull()) + .field("smallint", DataTypes.SMALLINT().notNull()) + .field("int", DataTypes.INT().nullable()) + .build(); + + PartitionSpec spec; + if (partitioned) { + spec = + PartitionSpec.builderFor(iSchema) + .identity("smallint") + .identity("tinyint") + .identity("int") + .build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + + String location = tempFolder.newFolder().getAbsolutePath(); + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); + + List rows = + Lists.newArrayList( + GenericRowData.of((byte) 0x01, (short) -32768, 101), + GenericRowData.of((byte) 0x02, (short) 0, 102), + GenericRowData.of((byte) 0x03, (short) 32767, 103)); + + Record record = GenericRecord.create(iSchema); + List expected = + Lists.newArrayList( + record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), + record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), + record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter(icebergTable, flinkSchema)) { + for (RowData row : rows) { + testHarness.processElement(row, 1); + } + testHarness.prepareSnapshotPreBarrier(1); + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + Assert.assertEquals(0, result.deleteFiles().length); + Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + + // Commit the iceberg transaction. + AppendFiles appendFiles = icebergTable.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + SimpleDataUtil.assertTableRecords(location, expected); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + throws Exception { + return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema) throws Exception { + RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); + FlinkWriteConf flinkWriteConfig = + new FlinkWriteConf( + icebergTable, Maps.newHashMap(), new org.apache.flink.configuration.Configuration()); + + IcebergStreamWriter streamWriter = + FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); + + harness.setup(); + harness.open(); + + return harness; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java new file mode 100644 index 000000000000..79be96d20902 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -0,0 +1,252 @@ +/* + * 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.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.Assert; +import org.junit.Test; + +public class TestRowDataPartitionKey { + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(0, "boolType", Types.BooleanType.get()), + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "longType", Types.LongType.get()), + Types.NestedField.required(3, "dateType", Types.DateType.get()), + Types.NestedField.required(4, "timeType", Types.TimeType.get()), + Types.NestedField.required(5, "stringType", Types.StringType.get()), + Types.NestedField.required(6, "timestampWithoutZone", Types.TimestampType.withoutZone()), + Types.NestedField.required(7, "timestampWithZone", Types.TimestampType.withZone()), + Types.NestedField.required(8, "fixedType", Types.FixedType.ofLength(5)), + Types.NestedField.required(9, "uuidType", Types.UUIDType.get()), + Types.NestedField.required(10, "binaryType", Types.BinaryType.get()), + Types.NestedField.required(11, "decimalType1", Types.DecimalType.of(18, 3)), + Types.NestedField.required(12, "decimalType2", Types.DecimalType.of(10, 5)), + Types.NestedField.required(13, "decimalType3", Types.DecimalType.of(38, 19)), + Types.NestedField.required(14, "floatType", Types.FloatType.get()), + Types.NestedField.required(15, "doubleType", Types.DoubleType.get())); + + private static final List SUPPORTED_PRIMITIVES = + SCHEMA.asStruct().fields().stream().map(Types.NestedField::name).collect(Collectors.toList()); + + private static final Schema NESTED_SCHEMA = + new Schema( + Types.NestedField.required( + 1, + "structType", + Types.StructType.of( + Types.NestedField.optional(2, "innerStringType", Types.StringType.get()), + Types.NestedField.optional(3, "innerIntegerType", Types.IntegerType.get())))); + + @Test + public void testNullPartitionValue() { + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); + + List rows = + Lists.newArrayList( + GenericRowData.of(1, StringData.fromString("a")), + GenericRowData.of(2, StringData.fromString("b")), + GenericRowData.of(3, null)); + + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + for (RowData row : rows) { + PartitionKey partitionKey = new PartitionKey(spec, schema); + partitionKey.partition(rowWrapper.wrap(row)); + Assert.assertEquals(partitionKey.size(), 1); + + String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); + Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + } + } + + @Test + public void testPartitionWithOneNestedField() { + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); + List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1991); + List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); + + PartitionSpec spec1 = + PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerStringType").build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerIntegerType").build(); + + for (int i = 0; i < rows.size(); i++) { + RowData row = rows.get(i); + Record record = (Record) records.get(i).get(0); + + PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); + partitionKey1.partition(rowWrapper.wrap(row)); + Assert.assertEquals(partitionKey1.size(), 1); + + Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + + PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); + partitionKey2.partition(rowWrapper.wrap(row)); + Assert.assertEquals(partitionKey2.size(), 1); + + Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + } + } + + @Test + public void testPartitionMultipleNestedField() { + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); + List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1992); + List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); + + PartitionSpec spec1 = + PartitionSpec.builderFor(NESTED_SCHEMA) + .identity("structType.innerIntegerType") + .identity("structType.innerStringType") + .build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(NESTED_SCHEMA) + .identity("structType.innerStringType") + .identity("structType.innerIntegerType") + .build(); + + PartitionKey pk1 = new PartitionKey(spec1, NESTED_SCHEMA); + PartitionKey pk2 = new PartitionKey(spec2, NESTED_SCHEMA); + + for (int i = 0; i < rows.size(); i++) { + RowData row = rows.get(i); + Record record = (Record) records.get(i).get(0); + + pk1.partition(rowWrapper.wrap(row)); + Assert.assertEquals(2, pk1.size()); + + Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); + Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + + pk2.partition(rowWrapper.wrap(row)); + Assert.assertEquals(2, pk2.size()); + + Assert.assertEquals(record.get(0), pk2.get(0, String.class)); + Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + } + } + + @Test + public void testPartitionValueTypes() { + RowType rowType = FlinkSchemaUtil.convert(SCHEMA); + RowDataWrapper rowWrapper = new RowDataWrapper(rowType, SCHEMA.asStruct()); + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(SCHEMA.asStruct()); + + List records = RandomGenericData.generate(SCHEMA, 10, 1993); + List rows = Lists.newArrayList(RandomRowData.convert(SCHEMA, records)); + + for (String column : SUPPORTED_PRIMITIVES) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity(column).build(); + Class[] javaClasses = spec.javaClasses(); + + PartitionKey pk = new PartitionKey(spec, SCHEMA); + PartitionKey expectedPK = new PartitionKey(spec, SCHEMA); + + for (int j = 0; j < rows.size(); j++) { + RowData row = rows.get(j); + Record record = records.get(j); + + pk.partition(rowWrapper.wrap(row)); + expectedPK.partition(recordWrapper.wrap(record)); + + Assert.assertEquals( + "Partition with column " + column + " should have one field.", 1, pk.size()); + + if (column.equals("timeType")) { + Assert.assertEquals( + "Partition with column " + column + " should have the expected values", + expectedPK.get(0, Long.class) / 1000, + pk.get(0, Long.class) / 1000); + } else { + Assert.assertEquals( + "Partition with column " + column + " should have the expected values", + expectedPK.get(0, javaClasses[0]), + pk.get(0, javaClasses[0])); + } + } + } + } + + @Test + public void testNestedPartitionValues() { + Schema nestedSchema = new Schema(Types.NestedField.optional(1001, "nested", SCHEMA.asStruct())); + RowType rowType = FlinkSchemaUtil.convert(nestedSchema); + + RowDataWrapper rowWrapper = new RowDataWrapper(rowType, nestedSchema.asStruct()); + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(nestedSchema.asStruct()); + + List records = RandomGenericData.generate(nestedSchema, 10, 1994); + List rows = Lists.newArrayList(RandomRowData.convert(nestedSchema, records)); + + for (String supportedPrimitive : SUPPORTED_PRIMITIVES) { + String column = String.format("nested.%s", supportedPrimitive); + + PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity(column).build(); + Class[] javaClasses = spec.javaClasses(); + + PartitionKey pk = new PartitionKey(spec, nestedSchema); + PartitionKey expectedPK = new PartitionKey(spec, nestedSchema); + + for (int j = 0; j < rows.size(); j++) { + pk.partition(rowWrapper.wrap(rows.get(j))); + expectedPK.partition(recordWrapper.wrap(records.get(j))); + + Assert.assertEquals( + "Partition with nested column " + column + " should have one field.", 1, pk.size()); + + if (column.equals("nested.timeType")) { + Assert.assertEquals( + "Partition with nested column " + column + " should have the expected values.", + expectedPK.get(0, Long.class) / 1000, + pk.get(0, Long.class) / 1000); + } else { + Assert.assertEquals( + "Partition with nested column " + column + " should have the expected values.", + expectedPK.get(0, javaClasses[0]), + pk.get(0, javaClasses[0])); + } + } + } + } +} 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 new file mode 100644 index 000000000000..112dbb511310 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -0,0 +1,243 @@ +/* + * 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.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestTaskWriters { + private static final Configuration CONF = new Configuration(); + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + + @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + + @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", true}, + {"avro", false}, + {"orc", true}, + {"orc", false}, + {"parquet", true}, + {"parquet", false} + }; + } + + private final FileFormat format; + private final boolean partitioned; + + private Table table; + + public TestTaskWriters(String format, boolean partitioned) { + this.format = FileFormat.fromString(format); + this.partitioned = partitioned; + } + + @Before + public void before() throws IOException { + File folder = tempFolder.newFolder(); + // Construct the iceberg table with the specified file format. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); + } + + @Test + public void testWriteZeroRecord() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.close(); + + DataFile[] dataFiles = taskWriter.dataFiles(); + Assert.assertNotNull(dataFiles); + Assert.assertEquals(0, dataFiles.length); + + // Close again. + taskWriter.close(); + dataFiles = taskWriter.dataFiles(); + Assert.assertNotNull(dataFiles); + Assert.assertEquals(0, dataFiles.length); + } + } + + @Test + public void testCloseTwice() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + taskWriter.close(); // The first close + taskWriter.close(); // The second close + + int expectedFiles = partitioned ? 2 : 1; + DataFile[] dataFiles = taskWriter.dataFiles(); + Assert.assertEquals(expectedFiles, dataFiles.length); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + } + } + } + + @Test + public void testAbort() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + + taskWriter.abort(); + DataFile[] dataFiles = taskWriter.dataFiles(); + + int expectedFiles = partitioned ? 2 : 1; + Assert.assertEquals(expectedFiles, dataFiles.length); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + } + } + } + + @Test + public void testCompleteFiles() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "a")); + taskWriter.write(SimpleDataUtil.createRowData(2, "b")); + taskWriter.write(SimpleDataUtil.createRowData(3, "c")); + taskWriter.write(SimpleDataUtil.createRowData(4, "d")); + + DataFile[] dataFiles = taskWriter.dataFiles(); + int expectedFiles = partitioned ? 4 : 1; + Assert.assertEquals(expectedFiles, dataFiles.length); + + dataFiles = taskWriter.dataFiles(); + Assert.assertEquals(expectedFiles, dataFiles.length); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + } + + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords( + table, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"), + SimpleDataUtil.createRecord(4, "d"))); + } + } + + @Test + public void testRollingWithTargetFileSize() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(4)) { + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(SimpleDataUtil.createRowData(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + for (RowData row : rows) { + taskWriter.write(row); + } + + DataFile[] dataFiles = taskWriter.dataFiles(); + Assert.assertEquals(8, dataFiles.length); + + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords(table, records); + } + } + + @Test + public void testRandomData() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); + for (RowData row : rows) { + taskWriter.write(row); + } + + taskWriter.close(); + DataFile[] dataFiles = taskWriter.dataFiles(); + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(rows)); + } + } + + private TaskWriter createTaskWriter(long targetFileSize) { + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), + targetFileSize, + format, + table.properties(), + null, + false); + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java new file mode 100644 index 000000000000..890cc361b246 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.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.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; +import org.junit.Test; + +public class TestAggregatedStatistics { + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + @Test + public void mergeDataStatisticTest() { + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + + AggregatedStatistics> aggregatedStatistics = + new AggregatedStatistics<>(1, statisticsSerializer); + MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyA); + mapDataStatistics1.add(keyB); + aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); + MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); + mapDataStatistics2.add(keyA); + aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) + .isEqualTo( + mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); + assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) + .isEqualTo( + mapDataStatistics1.statistics().get(keyB) + + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java new file mode 100644 index 000000000000..4c64ce522201 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -0,0 +1,178 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; +import org.junit.Before; +import org.junit.Test; + +public class TestAggregatedStatisticsTracker { + private static final int NUM_SUBTASKS = 2; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + private final SortKey keyA = sortKey.copy(); + private final SortKey keyB = sortKey.copy(); + + private AggregatedStatisticsTracker> + aggregatedStatisticsTracker; + + public TestAggregatedStatisticsTracker() { + keyA.set(0, "a"); + keyB.set(0, "b"); + } + + @Before + public void before() throws Exception { + aggregatedStatisticsTracker = + new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); + } + + @Test + public void receiveNewerDataStatisticEvent() { + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint1Subtask0DataStatisticEvent)) + .isNull(); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint2Subtask0DataStatisticEvent)) + .isNull(); + // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + } + + @Test + public void receiveOlderDataStatisticEventTest() { + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyA); + checkpoint2Subtask0DataStatistic.add(keyB); + checkpoint2Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> + checkpoint3Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint3Subtask0DataStatisticEvent)) + .isNull(); + + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> + checkpoint1Subtask1DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return + // completed statistics and in progress statistics won't be updated + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 1, checkpoint1Subtask1DataStatisticEvent)) + .isNull(); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + } + + @Test + public void receiveCompletedDataStatisticEvent() { + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyB); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint1Subtask0DataStatisticEvent)) + .isNull(); + + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyA); + checkpoint1Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> + checkpoint1Subtask1DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + // Receive data statistics from all subtasks at checkpoint 1 + AggregatedStatistics> completedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion( + 1, checkpoint1Subtask1DataStatisticEvent); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1); + MapDataStatistics globalDataStatistics = + (MapDataStatistics) completedStatistics.dataStatistics(); + assertThat((long) globalDataStatistics.statistics().get(keyA)) + .isEqualTo( + checkpoint1Subtask0DataStatistic.statistics().get(keyA) + + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); + assertThat((long) globalDataStatistics.statistics().get(keyB)) + .isEqualTo( + checkpoint1Subtask0DataStatistic.statistics().get(keyB) + + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) + .isEqualTo(completedStatistics.checkpointId() + 1); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyA); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + assertThat( + aggregatedStatisticsTracker.updateAndCheckCompletion( + 0, checkpoint2Subtask0DataStatisticEvent)) + .isNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(1); + + MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); + checkpoint2Subtask1DataStatistic.add(keyB); + DataStatisticsEvent> + checkpoint2Subtask1DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); + // Receive data statistics from all subtasks at checkpoint 2 + completedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion( + 1, checkpoint2Subtask1DataStatisticEvent); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2); + assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) + .isEqualTo(completedStatistics.checkpointId() + 1); + } +} 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 new file mode 100644 index 000000000000..3df714059c37 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -0,0 +1,178 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsCoordinator { + private static final String OPERATOR_NAME = "TestCoordinator"; + private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); + private static final int NUM_SUBTASKS = 2; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private EventReceivingTasks receivingTasks; + private DataStatisticsCoordinator> + dataStatisticsCoordinator; + + @Before + public void before() throws Exception { + receivingTasks = EventReceivingTasks.createForRunningTasks(); + dataStatisticsCoordinator = + new DataStatisticsCoordinator<>( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + statisticsSerializer); + } + + private void tasksReady() throws Exception { + dataStatisticsCoordinator.start(); + setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + } + + @Test + public void testThrowExceptionWhenNotStarted() { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + + @Test + public void testDataStatisticsEventHandling() throws Exception { + tasksReady(); + SortKey key = sortKey.copy(); + + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + key.set(0, "a"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask0DataStatistic.add(key); + + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + + MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); + key.set(0, "a"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "b"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + key.set(0, "c"); + checkpoint1Subtask1DataStatistic.add(key); + + DataStatisticsEvent> + checkpoint1Subtask1DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + // Verify global data statistics is the aggregation of all subtasks data statistics + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + MapDataStatistics globalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(globalDataStatistics.statistics()) + .containsExactlyInAnyOrderEntriesOf( + ImmutableMap.of( + keyA, 2L, + keyB, 3L, + keyC, 5L)); + } + + static void setAllTasksReady( + int subtasks, + DataStatisticsCoordinator> dataStatisticsCoordinator, + EventReceivingTasks receivingTasks) { + for (int i = 0; i < subtasks; i++) { + dataStatisticsCoordinator.executionAttemptReady( + i, 0, receivingTasks.createGatewayForSubtask(i, 0)); + } + } + + static void waitForCoordinatorToProcessActions( + DataStatisticsCoordinator> coordinator) { + CompletableFuture future = new CompletableFuture<>(); + coordinator.callInCoordinatorThread( + () -> { + future.complete(null); + return null; + }, + "Coordinator fails to process action"); + + try { + future.get(); + } catch (InterruptedException e) { + throw new AssertionError("test interrupted"); + } catch (ExecutionException e) { + ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..5e0a752be506 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.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.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.types.Types; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsCoordinatorProvider { + private static final OperatorID OPERATOR_ID = new OperatorID(); + private static final int NUM_SUBTASKS = 1; + + private final Schema schema = + new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final MapDataStatisticsSerializer statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsCoordinatorProvider> provider; + private EventReceivingTasks receivingTasks; + + @Before + public void before() { + provider = + new DataStatisticsCoordinatorProvider<>( + "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + @Test + @SuppressWarnings("unchecked") + public void testCheckpointAndReset() throws Exception { + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + SortKey keyD = sortKey.copy(); + keyD.set(0, "c"); + SortKey keyE = sortKey.copy(); + keyE.set(0, "c"); + + try (RecreateOnResetOperatorCoordinator coordinator = + (RecreateOnResetOperatorCoordinator) + provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { + DataStatisticsCoordinator> dataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); + checkpoint1Subtask0DataStatistic.add(keyA); + checkpoint1Subtask0DataStatistic.add(keyB); + checkpoint1Subtask0DataStatistic.add(keyC); + DataStatisticsEvent> + checkpoint1Subtask0DataStatisticEvent = + DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + + // Handle events from operators for checkpoint 1 + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics + MapDataStatistics checkpoint1GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint1GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + + MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); + checkpoint2Subtask0DataStatistic.add(keyD); + checkpoint2Subtask0DataStatistic.add(keyE); + checkpoint2Subtask0DataStatistic.add(keyE); + DataStatisticsEvent> + checkpoint2Subtask0DataStatisticEvent = + DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); + // Handle events from operators for checkpoint 2 + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics + MapDataStatistics checkpoint2GlobalDataStatistics = + (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(checkpoint2GlobalDataStatistics.statistics()) + .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + waitForCheckpoint(2L, dataStatisticsCoordinator); + + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator> + restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator>) + coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + // Verify restored data statistics + MapDataStatistics restoredAggregateDataStatistics = + (MapDataStatistics) + restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); + assertThat(restoredAggregateDataStatistics.statistics()) + .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + } + } + + private byte[] waitForCheckpoint( + long checkpointId, + DataStatisticsCoordinator> coordinator) + throws InterruptedException, ExecutionException { + CompletableFuture future = new CompletableFuture<>(); + coordinator.checkpointCoordinator(checkpointId, future); + return future.get(); + } +} 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 new file mode 100644 index 000000000000..0e99a2d74ccb --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -0,0 +1,255 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.fs.CloseableRegistry; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +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.types.Types; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestDataStatisticsOperator { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); + private final SortKey sortKey = new SortKey(schema, sortOrder); + private final RowType rowType = RowType.of(new VarCharType(), new IntType()); + private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); + private final TypeSerializer>> + statisticsSerializer = + MapDataStatisticsSerializer.fromSortKeySerializer( + new SortKeySerializer(schema, sortOrder)); + + private DataStatisticsOperator> operator; + + private Environment getTestingEnvironment() { + return new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); + } + + @Before + public void before() throws Exception { + this.operator = createOperator(); + Environment env = getTestingEnvironment(); + this.operator.setup( + new OneInputStreamTask(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + } + + private DataStatisticsOperator> createOperator() { + MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + return new DataStatisticsOperator<>( + "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); + } + + @After + public void clean() throws Exception { + operator.close(); + } + + @Test + public void testProcessElement() throws Exception { + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness = createHarness(this.operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); + assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); + + MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); + Map statsMap = mapDataStatistics.statistics(); + assertThat(statsMap).hasSize(2); + assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + + testHarness.endInput(); + } + } + + @Test + public void testOperatorOutput() throws Exception { + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness = createHarness(this.operator)) { + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); + + List recordsOutput = + testHarness.extractOutputValues().stream() + .filter(DataStatisticsOrRecord::hasRecord) + .map(DataStatisticsOrRecord::record) + .collect(Collectors.toList()); + assertThat(recordsOutput) + .containsExactlyInAnyOrderElementsOf( + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); + } + } + + @Test + public void testRestoreState() throws Exception { + OperatorSubtaskState snapshot; + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness1 = createHarness(this.operator)) { + MapDataStatistics mapDataStatistics = new MapDataStatistics(); + + SortKey key = sortKey.copy(); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "a"); + mapDataStatistics.add(key); + key.set(0, "b"); + mapDataStatistics.add(key); + key.set(0, "c"); + mapDataStatistics.add(key); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + DataStatisticsEvent> event = + DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + operator.handleOperatorEvent(event); + assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); + assertThat(operator.globalDataStatistics().statistics()) + .containsExactlyInAnyOrderEntriesOf(expectedMap); + snapshot = testHarness1.snapshot(1L, 0); + } + + // Use the snapshot to initialize state for another new operator and then verify that the global + // statistics for the new operator is same as before + DataStatisticsOperator> restoredOperator = + createOperator(); + try (OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + testHarness2.setup(); + testHarness2.initializeState(snapshot); + assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); + + // restored RowData is BinaryRowData. convert to GenericRowData for comparison + Map restoredStatistics = Maps.newHashMap(); + restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + + SortKey keyA = sortKey.copy(); + keyA.set(0, "a"); + SortKey keyB = sortKey.copy(); + keyB.set(0, "b"); + SortKey keyC = sortKey.copy(); + keyC.set(0, "c"); + Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + + assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); + } + } + + private StateInitializationContext getStateContext() throws Exception { + MockEnvironment env = new MockEnvironmentBuilder().build(); + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + env, "test-operator", Collections.emptyList(), cancelStreamRegistry); + return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + } + + private OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + createHarness( + final DataStatisticsOperator> + dataStatisticsOperator) + throws Exception { + + OneInputStreamOperatorTestHarness< + RowData, DataStatisticsOrRecord>> + harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); + harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); + harness.open(); + return harness; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..a07808e935d9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -0,0 +1,90 @@ +/* + * 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.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); + private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private final RowDataWrapper rowWrapper = + new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); + + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("c")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("a")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + reusedRow.setField(0, StringData.fromString("b")); + sortKey.wrap(rowWrapper.wrap(reusedRow)); + dataStatistics.add(sortKey); + + Map actual = dataStatistics.statistics(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyA = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyB = sortKey.copy(); + + rowWrapper.wrap( + GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); + sortKey.wrap(rowWrapper); + SortKey keyC = sortKey.copy(); + + Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Assertions.assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..92eb71acc834 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.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.flink.sink.shuffle; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final MapDataStatistics mapDataStatistics = + new MapDataStatistics( + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L)); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 8; + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 8; + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + int numPartitions = 9; + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + int numPartitions = 9; + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new MapRangePartitioner.KeyAssignment( + ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + Map actualAssignment = + partitioner.assignment(numPartitions); + Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + Map> actualAssignmentInfo = partitioner.assignmentInfo(); + Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + + Map>> partitionResults = + runPartitioner(partitioner, numPartitions); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + partitioner + .mapStatistics() + .forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + Assertions.assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.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.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} 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 new file mode 100644 index 000000000000..291302aef486 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.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.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java new file mode 100644 index 000000000000..a08578a4c106 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java @@ -0,0 +1,170 @@ +/* + * 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.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.stream.Stream; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +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.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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; + +public class BoundedTableFactory implements DynamicTableSourceFactory { + private static final AtomicInteger DATA_SET_ID = new AtomicInteger(0); + private static final Map>> DATA_SETS = Maps.newHashMap(); + + private static final ConfigOption DATA_ID = + ConfigOptions.key("data-id").stringType().noDefaultValue(); + + public static String registerDataSet(List> dataSet) { + String dataSetId = String.valueOf(DATA_SET_ID.incrementAndGet()); + DATA_SETS.put(dataSetId, dataSet); + return dataSetId; + } + + public static void clearDataSets() { + DATA_SETS.clear(); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + TableSchema tableSchema = + TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + + Configuration configuration = Configuration.fromMap(context.getCatalogTable().getOptions()); + String dataId = configuration.getString(DATA_ID); + Preconditions.checkArgument( + DATA_SETS.containsKey(dataId), "data-id %s does not found in registered data set.", dataId); + + return new BoundedTableSource(DATA_SETS.get(dataId), tableSchema); + } + + @Override + public String factoryIdentifier() { + return "BoundedSource"; + } + + @Override + public Set> requiredOptions() { + return ImmutableSet.of(); + } + + @Override + public Set> optionalOptions() { + return ImmutableSet.of(DATA_ID); + } + + private static class BoundedTableSource implements ScanTableSource { + + private final List> elementsPerCheckpoint; + private final TableSchema tableSchema; + + private BoundedTableSource(List> elementsPerCheckpoint, TableSchema tableSchema) { + this.elementsPerCheckpoint = elementsPerCheckpoint; + this.tableSchema = tableSchema; + } + + private BoundedTableSource(BoundedTableSource toCopy) { + this.elementsPerCheckpoint = toCopy.elementsPerCheckpoint; + this.tableSchema = toCopy.tableSchema; + } + + @Override + public ChangelogMode getChangelogMode() { + Supplier> supplier = () -> elementsPerCheckpoint.stream().flatMap(List::stream); + + // Add the INSERT row kind by default. + ChangelogMode.Builder builder = ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT); + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.DELETE)) { + builder.addContainedKind(RowKind.DELETE); + } + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_BEFORE)) { + builder.addContainedKind(RowKind.UPDATE_BEFORE); + } + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_AFTER)) { + builder.addContainedKind(RowKind.UPDATE_AFTER); + } + + return builder.build(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment env) { + boolean checkpointEnabled = env.getCheckpointConfig().isCheckpointingEnabled(); + SourceFunction source = + new BoundedTestSource<>(elementsPerCheckpoint, checkpointEnabled); + + RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + // Converter to convert the Row to RowData. + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); + + return env.addSource(source, new RowTypeInfo(tableSchema.getFieldTypes())) + .map(rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)); + } + + @Override + public boolean isBounded() { + return true; + } + }; + } + + @Override + public DynamicTableSource copy() { + return new BoundedTableSource(this); + } + + @Override + public String asSummaryString() { + return "Bounded test table source"; + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java new file mode 100644 index 000000000000..7b435d059845 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java @@ -0,0 +1,108 @@ +/* + * 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.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * A stream source that: 1) emits the elements from elementsPerCheckpoint.get(0) without allowing + * checkpoints. 2) then waits for the checkpoint to complete. 3) emits the elements from + * elementsPerCheckpoint.get(1) without allowing checkpoints. 4) then waits for the checkpoint to + * complete. 5) ... + * + *

    Util all the list from elementsPerCheckpoint are exhausted. + */ +public final class BoundedTestSource implements SourceFunction, CheckpointListener { + + private final List> elementsPerCheckpoint; + private final boolean checkpointEnabled; + private volatile boolean running = true; + + private final AtomicInteger numCheckpointsComplete = new AtomicInteger(0); + + /** Emits all those elements in several checkpoints. */ + public BoundedTestSource(List> elementsPerCheckpoint, boolean checkpointEnabled) { + this.elementsPerCheckpoint = elementsPerCheckpoint; + this.checkpointEnabled = checkpointEnabled; + } + + public BoundedTestSource(List> elementsPerCheckpoint) { + this(elementsPerCheckpoint, true); + } + + /** Emits all those elements in a single checkpoint. */ + public BoundedTestSource(T... elements) { + this(Collections.singletonList(Arrays.asList(elements))); + } + + @Override + public void run(SourceContext ctx) throws Exception { + if (!checkpointEnabled) { + Preconditions.checkArgument( + elementsPerCheckpoint.size() <= 1, + "There should be at most one list in the elementsPerCheckpoint when checkpoint is disabled."); + elementsPerCheckpoint.stream().flatMap(List::stream).forEach(ctx::collect); + return; + } + + for (List elements : elementsPerCheckpoint) { + + final int checkpointToAwait; + synchronized (ctx.getCheckpointLock()) { + // Let's say checkpointToAwait = numCheckpointsComplete.get() + delta, in fact the value of + // delta should not + // affect the final table records because we only need to make sure that there will be + // exactly + // elementsPerCheckpoint.size() checkpoints to emit each records buffer from the original + // elementsPerCheckpoint. + // Even if the checkpoints that emitted results are not continuous, the correctness of the + // data should not be + // affected in the end. Setting the delta to be 2 is introducing the variable that produce + // un-continuous + // checkpoints that emit the records buffer from elementsPerCheckpoints. + checkpointToAwait = numCheckpointsComplete.get() + 2; + for (T element : elements) { + ctx.collect(element); + } + } + + synchronized (ctx.getCheckpointLock()) { + while (running && numCheckpointsComplete.get() < checkpointToAwait) { + ctx.getCheckpointLock().wait(1); + } + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + numCheckpointsComplete.incrementAndGet(); + } + + @Override + public void cancel() { + running = false; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java new file mode 100644 index 000000000000..7aa2b8034bc5 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -0,0 +1,87 @@ +/* + * 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.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.flink.MiniClusterResource; +import org.junit.After; +import org.junit.Rule; +import org.junit.rules.TestName; + +public class ChangeLogTableTestBase extends FlinkTestBase { + private volatile TableEnvironment tEnv = null; + + @Rule public TestName name = new TestName(); + + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s", name.getMethodName()); + BoundedTableFactory.clearDataSets(); + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = + EnvironmentSettings.newInstance().inStreamingMode().build(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(400) + .setMaxParallelism(1) + .setParallelism(1); + + tEnv = StreamTableEnvironment.create(env, settings); + } + } + } + return tEnv; + } + + protected static Row insertRow(Object... values) { + return Row.ofKind(RowKind.INSERT, values); + } + + protected static Row deleteRow(Object... values) { + return Row.ofKind(RowKind.DELETE, values); + } + + protected static Row updateBeforeRow(Object... values) { + return Row.ofKind(RowKind.UPDATE_BEFORE, values); + } + + protected static Row updateAfterRow(Object... values) { + return Row.ofKind(RowKind.UPDATE_AFTER, values); + } + + protected static List listJoin(List> lists) { + return lists.stream().flatMap(List::stream).collect(Collectors.toList()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java new file mode 100644 index 000000000000..3a8071523b7c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -0,0 +1,132 @@ +/* + * 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.File; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; +import org.junit.Assert; +import org.junit.rules.TemporaryFolder; + +public class SplitHelpers { + + private SplitHelpers() {} + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + *

    By default, v1 Iceberg table is created. For v2 table use {@link + * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); + } + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + * @param version The table version to create + */ + public static List createSplitsFromTransientHadoopTable( + TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) + throws Exception { + final File warehouseFile = temporaryFolder.newFolder(); + Assert.assertTrue(warehouseFile.delete()); + final String warehouse = "file:" + warehouseFile; + Configuration hadoopConf = new Configuration(); + final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); + ImmutableMap properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, version); + try { + final Table table = + catalog.createTable( + TestFixtures.TABLE_IDENTIFIER, + TestFixtures.SCHEMA, + PartitionSpec.unpartitioned(), + null, + properties); + final GenericAppenderHelper dataAppender = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + for (int i = 0; i < fileCount; ++i) { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); + dataAppender.appendToTable(records); + } + + final ScanContext scanContext = ScanContext.builder().build(); + final List splits = + FlinkSplitPlanner.planIcebergSourceSplits( + table, scanContext, ThreadPools.getWorkerPool()); + return splits.stream() + .flatMap( + split -> { + List> filesList = + Lists.partition(Lists.newArrayList(split.task().files()), filesPerSplit); + return filesList.stream() + .map(files -> new BaseCombinedScanTask(files)) + .map( + combinedScanTask -> + IcebergSourceSplit.fromCombinedScanTask(combinedScanTask)); + }) + .collect(Collectors.toList()); + } finally { + catalog.dropTable(TestFixtures.TABLE_IDENTIFIER); + catalog.close(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java new file mode 100644 index 000000000000..e4e48ca67f66 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.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.flink.source; + +import java.util.List; +import java.util.Map; +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.iceberg.relocated.com.google.common.collect.Lists; + +public class SqlHelpers { + private SqlHelpers() {} + + public static List sql(TableEnvironment tableEnv, String query, Object... args) { + TableResult tableResult = tableEnv.executeSql(String.format(query, args)); + try (CloseableIterator iter = tableResult.collect()) { + List results = Lists.newArrayList(iter); + return results; + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + public static String sqlOptionsToString(Map sqlOptions) { + StringBuilder builder = new StringBuilder(); + sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(",")); + String optionStr = builder.toString(); + if (optionStr.endsWith(",")) { + optionStr = optionStr.substring(0, optionStr.length() - 1); + } + + if (!optionStr.isEmpty()) { + optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr); + } + + return optionStr; + } + + private static String optionToKv(String key, Object value) { + return "'" + key + "'='" + value + "'"; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java new file mode 100644 index 000000000000..7b5f9328694c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java @@ -0,0 +1,87 @@ +/* + * 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.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.flink.types.Row; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.junit.Assert; +import org.junit.Test; + +public class TestBoundedTableFactory extends ChangeLogTableTestBase { + + @Test + public void testEmptyDataSet() { + String table = name.getMethodName(); + List> emptyDataSet = ImmutableList.of(); + + String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); + sql( + "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", + table, dataId); + + Assert.assertEquals( + "Should have caught empty change log set.", + ImmutableList.of(), + sql("SELECT * FROM %s", table)); + } + + @Test + public void testBoundedTableFactory() { + String table = name.getMethodName(); + List> dataSet = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(1, "bbb"), + insertRow(2, "aaa"), + deleteRow(2, "aaa"), + insertRow(2, "bbb")), + ImmutableList.of( + updateBeforeRow(2, "bbb"), + updateAfterRow(2, "ccc"), + deleteRow(2, "ccc"), + insertRow(2, "ddd")), + ImmutableList.of( + deleteRow(1, "bbb"), + insertRow(1, "ccc"), + deleteRow(1, "ccc"), + insertRow(1, "ddd"))); + + String dataId = BoundedTableFactory.registerDataSet(dataSet); + sql( + "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", + table, dataId); + + List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); + Assert.assertEquals( + "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); + + Assert.assertEquals( + "Should have the expected change log events", + rowSet.stream() + .filter(r -> Objects.equals(r.getField(1), "aaa")) + .collect(Collectors.toList()), + sql("SELECT * FROM %s WHERE data='aaa'", table)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java new file mode 100644 index 000000000000..ed3f54bec642 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -0,0 +1,191 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.TestTemplate; + +/** Test {@link FlinkInputFormat}. */ +public class TestFlinkInputFormat extends TestFlinkSource { + + @Override + protected List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + return runFormat(formatBuilder.tableLoader(tableLoader()).buildFormat()); + } + + @TestTemplate + public void testNestedProjection() throws Exception { + Schema schema = + new Schema( + required(1, "data", Types.StringType.get()), + required( + 2, + "nested", + Types.StructType.of( + Types.NestedField.required(3, "f1", Types.StringType.get()), + Types.NestedField.required(4, "f2", Types.StringType.get()), + Types.NestedField.required(5, "f3", Types.LongType.get()))), + required(6, "id", Types.LongType.get())); + + Table table = + catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), schema); + + List writeRecords = RandomGenericData.generate(schema, 2, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); + + // Schema: [data, nested[f1, f2, f3], id] + // Projection: [nested.f2, data] + // The Flink SQL output: [f2, data] + // The FlinkInputFormat output: [nested[f2], data] + + TableSchema projectedSchema = + TableSchema.builder() + .field("nested", DataTypes.ROW(DataTypes.FIELD("f2", DataTypes.STRING()))) + .field("data", DataTypes.STRING()) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : writeRecords) { + Row nested = Row.of(((Record) record.get(1)).get(1)); + expected.add(Row.of(nested, record.get(0))); + } + + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testBasicProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); + + Table table = + catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); + + List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); + + TableSchema projectedSchema = + TableSchema.builder() + .field("id", DataTypes.BIGINT()) + .field("data", DataTypes.STRING()) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : writeRecords) { + expected.add(Row.of(record.get(0), record.get(1))); + } + + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testReadPartitionColumn() throws Exception { + assumeThat(fileFormat).as("Temporary skip ORC").isNotEqualTo(FileFormat.ORC); + + Schema nestedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional( + 2, + "struct", + Types.StructType.of( + Types.NestedField.optional(3, "innerId", Types.LongType.get()), + Types.NestedField.optional(4, "innerName", Types.StringType.get())))); + PartitionSpec spec = + PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); + + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); + List records = RandomGenericData.generate(nestedSchema, 10, 0L); + GenericAppenderHelper appender = + new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + for (Record record : records) { + org.apache.iceberg.TestHelpers.Row partition = + org.apache.iceberg.TestHelpers.Row.of(record.get(1, Record.class).get(1)); + appender.appendToTable(partition, Collections.singletonList(record)); + } + + TableSchema projectedSchema = + TableSchema.builder() + .field("struct", DataTypes.ROW(DataTypes.FIELD("innerName", DataTypes.STRING()))) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : records) { + Row nested = Row.of(((Record) record.get(1)).get(1)); + expected.add(Row.of(nested)); + } + + TestHelpers.assertRows(result, expected); + } + + private List runFormat(FlinkInputFormat inputFormat) throws IOException { + RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); + return TestHelpers.readRows(inputFormat, rowType); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java new file mode 100644 index 000000000000..1b4fc863631f --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java @@ -0,0 +1,71 @@ +/* + * 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.util.Map; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { + + @Override + protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) + throws IOException { + Schema projected = testTable.schema().select(columns); + RowType rowType = FlinkSchemaUtil.convert(projected); + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.WAREHOUSE_LOCATION, + hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + properties.put( + CatalogProperties.CLIENT_POOL_SIZE, + Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); + CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); + FlinkInputFormat inputFormat = + FlinkSource.forRowData() + .tableLoader( + TableLoader.fromCatalog( + hiveCatalogLoader, TableIdentifier.of("default", tableName))) + .project(FlinkSchemaUtil.toSchema(rowType)) + .buildFormat(); + + StructLikeSet set = StructLikeSet.create(projected.asStruct()); + TestHelpers.readRowData(inputFormat, rowType) + .forEach( + rowData -> { + RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); + set.add(wrapper.wrap(rowData)); + }); + + return set; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java new file mode 100644 index 000000000000..1d52acb2fe7b --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java @@ -0,0 +1,71 @@ +/* + * 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.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TestMergingMetrics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.sink.FlinkAppenderFactory; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.rules.TemporaryFolder; + +public class TestFlinkMergingMetrics extends TestMergingMetrics { + + @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource tableResource = + new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); + + public TestFlinkMergingMetrics(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileAppender writeAndGetAppender(List records) throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); + + FileAppender appender = + new FlinkAppenderFactory( + tableResource.table(), + SCHEMA, + flinkSchema, + ImmutableMap.of(), + PartitionSpec.unpartitioned(), + null, + null, + null) + .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + try (FileAppender fileAppender = appender) { + records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); + } + return appender; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java new file mode 100644 index 000000000000..f58cc87c6a29 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -0,0 +1,813 @@ +/* + * 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 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.time.Instant; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.avro.generic.GenericData; +import org.apache.commons.collections.ListUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.MetricsUtil; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.catalog.Namespace; +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.expressions.Expressions; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +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.SnapshotUtil; +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; + +public class TestFlinkMetaDataTable extends CatalogTestBase { + private static final String TABLE_NAME = "test_table"; + private final FileFormat format = FileFormat.AVRO; + private @TempDir Path temp; + + @Parameter(index = 2) + private Boolean isPartition; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + + for (Boolean isPartition : new Boolean[] {true, false}) { + String catalogName = "testhadoop"; + Namespace baseNamespace = Namespace.of("default"); + parameters.add(new Object[] {catalogName, baseNamespace, isPartition}); + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeEach + public void before() { + super.before(); + sql("USE CATALOG %s", catalogName); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE %s", DATABASE); + if (isPartition) { + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')", + TABLE_NAME, format.name()); + sql("INSERT INTO %s VALUES (1,'a',10),(2,'a',20)", TABLE_NAME); + sql("INSERT INTO %s VALUES (1,'b',10),(2,'b',20)", TABLE_NAME); + } else { + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + sql("INSERT INTO %s VALUES (4,'iceberg',10)", TABLE_NAME); + } + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + @TestTemplate + public void testSnapshots() { + String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + Iterator snapshots = table.snapshots().iterator(); + for (Row row : result) { + Snapshot next = snapshots.next(); + assertThat(((Instant) row.getField(0)).toEpochMilli()) + .as("Should have expected timestamp") + .isEqualTo(next.timestampMillis()); + assertThat(next.snapshotId()) + .as("Should have expected snapshot id") + .isEqualTo(next.snapshotId()); + assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); + assertThat(row.getField(3)).as("Should have expected operation").isEqualTo(next.operation()); + assertThat(row.getField(4)) + .as("Should have expected manifest list location") + .isEqualTo(next.manifestListLocation()); + assertThat(row.getField(5)).as("Should have expected summary").isEqualTo(next.summary()); + } + } + + @TestTemplate + public void testHistory() { + String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + Iterator snapshots = table.snapshots().iterator(); + for (Row row : result) { + Snapshot next = snapshots.next(); + assertThat(((Instant) row.getField(0)).toEpochMilli()) + .as("Should have expected made_current_at") + .isEqualTo(next.timestampMillis()); + assertThat(row.getField(1)) + .as("Should have expected snapshot id") + .isEqualTo(next.snapshotId()); + assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); + assertThat(row.getField(3)) + .as("Should have expected is current ancestor") + .isEqualTo( + SnapshotUtil.isAncestorOf( + table, table.currentSnapshot().snapshotId(), next.snapshotId())); + } + } + + @TestTemplate + public void testManifests() { + String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + List expectedDataManifests = dataManifests(table); + + for (int i = 0; i < result.size(); i++) { + Row row = result.get(i); + ManifestFile manifestFile = expectedDataManifests.get(i); + assertThat(row.getField(0)) + .as("Should have expected content") + .isEqualTo(manifestFile.content().id()); + assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); + assertThat(row.getField(2)) + .as("Should have expected length") + .isEqualTo(manifestFile.length()); + assertThat(row.getField(3)) + .as("Should have expected partition_spec_id") + .isEqualTo(manifestFile.partitionSpecId()); + assertThat(row.getField(4)) + .as("Should have expected added_snapshot_id") + .isEqualTo(manifestFile.snapshotId()); + assertThat(row.getField(5)) + .as("Should have expected added_data_files_count") + .isEqualTo(manifestFile.addedFilesCount()); + assertThat(row.getField(6)) + .as("Should have expected existing_data_files_count") + .isEqualTo(manifestFile.existingFilesCount()); + assertThat(row.getField(7)) + .as("Should have expected deleted_data_files_count") + .isEqualTo(manifestFile.deletedFilesCount()); + } + } + + @TestTemplate + public void testAllManifests() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + String sql = String.format("SELECT * FROM %s$all_manifests ", TABLE_NAME); + List result = sql(sql); + + List expectedDataManifests = allDataManifests(table); + + assertThat(expectedDataManifests).hasSize(result.size()); + for (int i = 0; i < result.size(); i++) { + Row row = result.get(i); + ManifestFile manifestFile = expectedDataManifests.get(i); + assertThat(row.getField(0)) + .as("Should have expected content") + .isEqualTo(manifestFile.content().id()); + assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); + assertThat(row.getField(2)) + .as("Should have expected length") + .isEqualTo(manifestFile.length()); + assertThat(row.getField(3)) + .as("Should have expected partition_spec_id") + .isEqualTo(manifestFile.partitionSpecId()); + assertThat(row.getField(4)) + .as("Should have expected added_snapshot_id") + .isEqualTo(manifestFile.snapshotId()); + assertThat(row.getField(5)) + .as("Should have expected added_data_files_count") + .isEqualTo(manifestFile.addedFilesCount()); + assertThat(row.getField(6)) + .as("Should have expected existing_data_files_count") + .isEqualTo(manifestFile.existingFilesCount()); + assertThat(row.getField(7)) + .as("Should have expected deleted_data_files_count") + .isEqualTo(manifestFile.deletedFilesCount()); + } + } + + @TestTemplate + public void testUnPartitionedTable() throws IOException { + assumeThat(isPartition).isFalse(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = Lists.newArrayList(dataDelete.copy("id", 1)); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, Files.localOutput(testFile), dataDeletes, deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + List expectedDataManifests = dataManifests(table); + List expectedDeleteManifests = deleteManifests(table); + + assertThat(expectedDataManifests).hasSize(2); + assertThat(expectedDeleteManifests).hasSize(1); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + + // check delete files table + Schema deleteFilesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("delete_files")) + .schema(); + + List deleteColumns = + deleteFilesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String deleteNames = + deleteColumns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + deleteFilesTableSchema = deleteFilesTableSchema.select(deleteColumns); + + List actualDeleteFiles = sql("SELECT %s FROM %s$delete_files", deleteNames, TABLE_NAME); + assertThat(actualDeleteFiles).hasSize(1); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); + + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); + assertThat(expectedDeleteFiles).as("Should be 1 delete file manifest entry").hasSize(1); + TestHelpers.assertEquals( + deleteFilesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check data files table + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + List actualDataFiles = sql("SELECT %s FROM %s$data_files", names, TABLE_NAME); + assertThat(actualDataFiles).as("Metadata table should return 2 data file").hasSize(2); + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + assertThat(expectedDataFiles).as("Should be 2 data file manifest entry").hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + // check all files table + List actualFiles = sql("SELECT %s FROM %s$files ORDER BY content", names, TABLE_NAME); + assertThat(actualFiles).as("Metadata table should return 3 files").hasSize(3); + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + assertThat(expectedFiles).as("Should have 3 files manifest entriess").hasSize(3); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + assumeThat(isPartition).isTrue(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id", "data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + deleteRow.put("data", "a"); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile), + org.apache.iceberg.TestHelpers.Row.of("a"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + deleteRow.put("data", "b"); + File testFile2 = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile2), + org.apache.iceberg.TestHelpers.Row.of("b"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes2).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + + List expectedDataManifests = dataManifests(table); + List expectedDeleteManifests = deleteManifests(table); + + assertThat(expectedDataManifests).hasSize(2); + assertThat(expectedDeleteManifests).hasSize(2); + Table deleteFilesTable = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("delete_files")); + Schema filesTableSchema = deleteFilesTable.schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check delete files table + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + assertThat(expectedDeleteFiles).hasSize(1); + List actualDeleteFiles = + sql("SELECT %s FROM %s$delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + + assertThat(actualDeleteFiles).hasSize(1); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check data files table + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + assertThat(expectedDataFiles).hasSize(1); + List actualDataFiles = + sql("SELECT %s FROM %s$data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + assertThat(actualDataFiles).hasSize(1); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + List actualPartitionsWithProjection = + sql("SELECT file_count FROM %s$partitions ", TABLE_NAME); + assertThat(actualPartitionsWithProjection).hasSize(2); + for (int i = 0; i < 2; ++i) { + assertThat(actualPartitionsWithProjection.get(i).getField(0)).isEqualTo(1); + } + + // Check files table + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + assertThat(expectedFiles).hasSize(2); + List actualFiles = + sql( + "SELECT %s FROM %s$files WHERE `partition`.`data`='a' ORDER BY content", + names, TABLE_NAME); + assertThat(actualFiles).hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); + } + + @TestTemplate + public void testAllFilesUnpartitioned() throws Exception { + assumeThat(isPartition).isFalse(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id", "data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + List expectedDataManifests = dataManifests(table); + assertThat(expectedDataManifests).hasSize(2); + List expectedDeleteManifests = deleteManifests(table); + assertThat(expectedDeleteManifests).hasSize(1); + + // Clear table to test whether 'all_files' can read past files + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("all_data_files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check all data files table + List actualDataFiles = + sql("SELECT %s FROM %s$all_data_files order by record_count ", names, TABLE_NAME); + + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + assertThat(expectedDataFiles).hasSize(2); + assertThat(actualDataFiles).hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles, actualDataFiles); + + // Check all delete files table + List actualDeleteFiles = sql("SELECT %s FROM %s$all_delete_files", names, TABLE_NAME); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); + assertThat(expectedDeleteFiles).hasSize(1); + assertThat(actualDeleteFiles).hasSize(1); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check all files table + List actualFiles = + sql("SELECT %s FROM %s$all_files ORDER BY content, record_count asc", names, TABLE_NAME); + List expectedFiles = + ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + assertThat(actualFiles).hasSize(3); + TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); + } + + @TestTemplate + public void testAllFilesPartitioned() throws Exception { + assumeThat(!isPartition).isFalse(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + // Create delete file + Schema deleteRowSchema = table.schema().select("id"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile), + org.apache.iceberg.TestHelpers.Row.of("a"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + File testFile2 = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile2), + org.apache.iceberg.TestHelpers.Row.of("b"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).addDeletes(eqDeletes2).commit(); + + List expectedDataManifests = dataManifests(table); + assertThat(expectedDataManifests).hasSize(2); + List expectedDeleteManifests = deleteManifests(table); + assertThat(expectedDeleteManifests).hasSize(1); + // Clear table to test whether 'all_files' can read past files + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("all_data_files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check all data files table + List actualDataFiles = + sql("SELECT %s FROM %s$all_data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + assertThat(expectedDataFiles).hasSize(1); + assertThat(actualDataFiles).hasSize(1); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + // Check all delete files table + List actualDeleteFiles = + sql("SELECT %s FROM %s$all_delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + assertThat(expectedDeleteFiles).hasSize(1); + assertThat(actualDeleteFiles).hasSize(1); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check all files table + List actualFiles = + sql( + "SELECT %s FROM %s$all_files WHERE `partition`.`data`='a' ORDER BY content", + names, TABLE_NAME); + List expectedFiles = + ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + assertThat(actualFiles).hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); + } + + @TestTemplate + public void testMetadataLogEntries() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + Snapshot currentSnapshot = tableMetadata.currentSnapshot(); + Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); + List metadataLogEntries = + Lists.newArrayList(tableMetadata.previousFiles()); + + // Check metadataLog table + List metadataLogs = sql("SELECT * FROM %s$metadata_log_entries", TABLE_NAME); + + assertThat(metadataLogs).hasSize(3); + Row metadataLog = metadataLogs.get(0); + assertThat(metadataLog.getField("timestamp")) + .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(0).timestampMillis())); + assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(0).file()); + assertThat(metadataLog.getField("latest_snapshot_id")).isNull(); + assertThat(metadataLog.getField("latest_schema_id")).isNull(); + assertThat(metadataLog.getField("latest_sequence_number")).isNull(); + + metadataLog = metadataLogs.get(1); + assertThat(metadataLog.getField("timestamp")) + .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(1).timestampMillis())); + assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(1).file()); + assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); + assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(parentSnapshot.schemaId()); + assertThat(metadataLog.getField("latest_sequence_number")) + .isEqualTo(parentSnapshot.sequenceNumber()); + assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); + + metadataLog = metadataLogs.get(2); + assertThat(metadataLog.getField("timestamp")) + .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); + assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); + assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); + assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); + assertThat(metadataLog.getField("latest_sequence_number")) + .isEqualTo(currentSnapshot.sequenceNumber()); + + // test filtering + List metadataLogWithFilters = + sql( + "SELECT * FROM %s$metadata_log_entries WHERE latest_snapshot_id = %s", + TABLE_NAME, currentSnapshotId); + assertThat(metadataLogWithFilters).hasSize(1); + metadataLog = metadataLogWithFilters.get(0); + assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) + .isEqualTo(metadataLog.getField("timestamp")); + + assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); + assertThat(metadataLog.getField("latest_snapshot_id")) + .isEqualTo(tableMetadata.currentSnapshot().snapshotId()); + assertThat(metadataLog.getField("latest_schema_id")) + .isEqualTo(tableMetadata.currentSnapshot().schemaId()); + assertThat(metadataLog.getField("latest_sequence_number")) + .isEqualTo(tableMetadata.currentSnapshot().sequenceNumber()); + + // test projection + List metadataFiles = + metadataLogEntries.stream() + .map(TableMetadata.MetadataLogEntry::file) + .collect(Collectors.toList()); + metadataFiles.add(tableMetadata.metadataFileLocation()); + List metadataLogWithProjection = + sql("SELECT file FROM %s$metadata_log_entries", TABLE_NAME); + assertThat(metadataLogWithProjection).hasSize(3); + for (int i = 0; i < metadataFiles.size(); i++) { + assertThat(metadataLogWithProjection.get(i).getField("file")).isEqualTo(metadataFiles.get(i)); + } + } + + @TestTemplate + public void testSnapshotReferencesMetatable() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + + // Create branch + table + .manageSnapshots() + .createBranch("testBranch", currentSnapshotId) + .setMaxRefAgeMs("testBranch", 10) + .setMinSnapshotsToKeep("testBranch", 20) + .setMaxSnapshotAgeMs("testBranch", 30) + .commit(); + // Create Tag + table + .manageSnapshots() + .createTag("testTag", currentSnapshotId) + .setMaxRefAgeMs("testTag", 50) + .commit(); + // Check refs table + List references = sql("SELECT * FROM %s$refs", TABLE_NAME); + List branches = sql("SELECT * FROM %s$refs WHERE type='BRANCH'", TABLE_NAME); + assertThat(references).hasSize(3); + assertThat(branches).hasSize(2); + List tags = sql("SELECT * FROM %s$refs WHERE type='TAG'", TABLE_NAME); + assertThat(tags).hasSize(1); + // Check branch entries in refs table + List mainBranch = + sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); + assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); + List testBranch = + sql("SELECT * FROM %s$refs WHERE name='testBranch' AND type='BRANCH'", TABLE_NAME); + assertThat((String) testBranch.get(0).getFieldAs("name")).isEqualTo("testBranch"); + assertThat((String) testBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + assertThat((Long) testBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); + assertThat((Long) testBranch.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(10)); + assertThat((Integer) testBranch.get(0).getFieldAs("min_snapshots_to_keep")) + .isEqualTo(Integer.valueOf(20)); + assertThat((Long) testBranch.get(0).getFieldAs("max_snapshot_age_in_ms")) + .isEqualTo(Long.valueOf(30)); + + // Check tag entries in refs table + List testTag = + sql("SELECT * FROM %s$refs WHERE name='testTag' AND type='TAG'", TABLE_NAME); + assertThat((String) testTag.get(0).getFieldAs("name")).isEqualTo("testTag"); + assertThat((String) testTag.get(0).getFieldAs("type")).isEqualTo("TAG"); + assertThat((Long) testTag.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); + assertThat((Long) testTag.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(50)); + // Check projection in refs table + List testTagProjection = + sql( + "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM %s$refs where type='TAG'", + TABLE_NAME); + assertThat((String) testTagProjection.get(0).getFieldAs("name")).isEqualTo("testTag"); + assertThat((String) testTagProjection.get(0).getFieldAs("type")).isEqualTo("TAG"); + assertThat((Long) testTagProjection.get(0).getFieldAs("snapshot_id")) + .isEqualTo(currentSnapshotId); + assertThat((Long) testTagProjection.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(50)); + assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); + List mainBranchProjection = + sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); + assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + List testBranchProjection = + sql( + "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM %s$refs WHERE name='testBranch' AND type = 'BRANCH'", + TABLE_NAME); + assertThat((String) testBranchProjection.get(0).getFieldAs("name")).isEqualTo("testBranch"); + assertThat((String) testBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + assertThat((Long) testBranchProjection.get(0).getFieldAs("snapshot_id")) + .isEqualTo(currentSnapshotId); + assertThat((Long) testBranchProjection.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(10)); + } + + /** + * Find matching manifest entries of an Iceberg table + * + * @param table iceberg table + * @param expectedContent file content to populate on entries + * @param entriesTableSchema schema of Manifest entries + * @param manifestsToExplore manifests to explore of the table + * @param partValue partition value that manifest entries must match, or null to skip filtering + */ + private List expectedEntries( + Table table, + FileContent expectedContent, + Schema entriesTableSchema, + List manifestsToExplore, + String partValue) + throws IOException { + List expected = Lists.newArrayList(); + for (ManifestFile manifest : manifestsToExplore) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTableSchema).build()) { + for (GenericData.Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + if (partitionMatch(file, partValue)) { + asMetadataRecord(file, expectedContent); + expected.add(file); + } + } + } + } + } + return expected; + } + + // Populate certain fields derived in the metadata tables + private void asMetadataRecord(GenericData.Record file, FileContent content) { + file.put(0, content.id()); + file.put(3, 0); // specId + } + + private boolean partitionMatch(GenericData.Record file, String partValue) { + if (partValue == null) { + return true; + } + GenericData.Record partition = (GenericData.Record) file.get(4); + return partValue.equals(partition.get(0).toString()); + } + + private List dataManifests(Table table) { + return table.currentSnapshot().dataManifests(table.io()); + } + + private List allDataManifests(Table table) { + List manifests = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + manifests.addAll(snapshot.dataManifests(table.io())); + } + return manifests; + } + + private List deleteManifests(Table table) { + return table.currentSnapshot().deleteManifests(table.io()); + } +} 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 new file mode 100644 index 000000000000..0b5a8011ad3f --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java @@ -0,0 +1,90 @@ +/* + * 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.util.Map; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ParameterizedTestExtension; +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.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.DeleteReadTests; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +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.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { + + protected static String databaseName = "default"; + + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + private static TestHiveMetastore metastore = null; + + @BeforeAll + public static void startMetastore() { + metastore = new TestHiveMetastore(); + metastore.start(); + hiveConf = metastore.hiveConf(); + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + catalog = null; + } + + @Override + protected Table createTable(String name, Schema schema, PartitionSpec spec) { + Map props = Maps.newHashMap(); + props.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + 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)); + + return table; + } + + @Override + protected void dropTable(String name) { + catalog.dropTable(TableIdentifier.of(databaseName, name)); + } + + @Override + protected boolean expectPruned() { + return false; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java new file mode 100644 index 000000000000..428da49f1de6 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -0,0 +1,540 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.nio.file.Path; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +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.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +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.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +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 TestFlinkScan { + @RegisterExtension + protected static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir protected Path temporaryDirectory; + + @RegisterExtension + protected static final HadoopCatalogExtension catalogExtension = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameter protected FileFormat fileFormat; + + @Parameters(name = "format={0}") + public static Collection fileFormat() { + return Arrays.asList(FileFormat.AVRO, FileFormat.PARQUET, FileFormat.ORC); + } + + protected TableLoader tableLoader() { + return catalogExtension.tableLoader(); + } + + protected abstract List runWithProjection(String... projected) throws Exception; + + protected abstract List runWithFilter( + Expression filter, String sqlFilter, boolean caseSensitive) throws Exception; + + protected List runWithFilter(Expression filter, String sqlFilter) throws Exception { + return runWithFilter(filter, sqlFilter, true); + } + + protected abstract List runWithOptions(Map options) throws Exception; + + protected abstract List run() throws Exception; + + @TestTemplate + public void testUnpartitionedTable() throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + Table table = + catalogExtension + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testProjection() throws Exception { + Table table = + catalogExtension + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); + assertRows(runWithProjection("data"), Row.of(inputRecords.get(0).get(0))); + } + + @TestTemplate + public void testIdentityPartitionProjections() throws Exception { + Schema logSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "level", Types.StringType.get()), + Types.NestedField.optional(4, "message", Types.StringType.get())); + PartitionSpec spec = + PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); + + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); + List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); + + int idx = 0; + AppendFiles append = table.newAppend(); + for (Record record : inputRecords) { + record.set(1, "2020-03-2" + idx); + record.set(2, Integer.toString(idx)); + append.appendFile( + new GenericAppenderHelper(table, fileFormat, temporaryDirectory) + .writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), + ImmutableList.of(record))); + idx += 1; + } + append.commit(); + + // individual fields + validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); + // field pairs + validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); + // out-of-order pairs + validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); + // out-of-order triplets + validateIdentityPartitionProjections( + table, Arrays.asList("dt", "level", "message"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("level", "dt", "message"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("dt", "message", "level"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("level", "message", "dt"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("message", "dt", "level"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("message", "level", "dt"), inputRecords); + } + + private void validateIdentityPartitionProjections( + Table table, List projectedFields, List inputRecords) throws Exception { + List rows = runWithProjection(projectedFields.toArray(new String[0])); + + for (int pos = 0; pos < inputRecords.size(); pos++) { + Record inputRecord = inputRecords.get(pos); + Row actualRecord = rows.get(pos); + + for (int i = 0; i < projectedFields.size(); i++) { + String name = projectedFields.get(i); + assertThat(inputRecord.getField(name)) + .as("Projected field " + name + " should match") + .isEqualTo(actualRecord.getField(i)); + } + } + } + + @TestTemplate + public void testSnapshotReads() throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords); + long snapshotId = table.currentSnapshot().snapshotId(); + + long timestampMillis = table.currentSnapshot().timestampMillis(); + + // produce another timestamp + waitUntilAfter(timestampMillis); + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L)); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("snapshot-id", Long.toString(snapshotId))), + expectedRecords, + TestFixtures.SCHEMA); + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("as-of-timestamp", Long.toString(timestampMillis))), + expectedRecords, + TestFixtures.SCHEMA); + } + + @TestTemplate + public void testTagReads() throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List expectedRecords1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords1); + long snapshotId = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().createTag("t1", snapshotId).commit(); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords1, TestFixtures.SCHEMA); + + List expectedRecords2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords2); + snapshotId = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().replaceTag("t1", snapshotId).commit(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(expectedRecords1); + expectedRecords.addAll(expectedRecords2); + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testBranchReads() throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List expectedRecordsBase = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecordsBase); + long snapshotId = table.currentSnapshot().snapshotId(); + + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName, snapshotId).commit(); + + List expectedRecordsForBranch = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(branchName, expectedRecordsForBranch); + + List expectedRecordsForMain = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecordsForMain); + + List branchExpectedRecords = Lists.newArrayList(); + branchExpectedRecords.addAll(expectedRecordsBase); + branchExpectedRecords.addAll(expectedRecordsForBranch); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("branch", branchName)), + branchExpectedRecords, + TestFixtures.SCHEMA); + + List mainExpectedRecords = Lists.newArrayList(); + mainExpectedRecords.addAll(expectedRecordsBase); + mainExpectedRecords.addAll(expectedRecordsForMain); + + TestHelpers.assertRecords(run(), mainExpectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testIncrementalReadViaTag() throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(records1); + long snapshotId1 = table.currentSnapshot().snapshotId(); + String startTag = "t1"; + table.manageSnapshots().createTag(startTag, snapshotId1).commit(); + + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); + helper.appendToTable(records2); + + List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); + helper.appendToTable(records3); + long snapshotId3 = table.currentSnapshot().snapshotId(); + String endTag = "t2"; + table.manageSnapshots().createTag(endTag, snapshotId3).commit(); + + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); + + List expected = Lists.newArrayList(); + expected.addAll(records2); + expected.addAll(records3); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-snapshot-id", Long.toString(snapshotId1)) + .put("end-tag", endTag) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + assertThatThrownBy( + () -> + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .put("start-snapshot-id", Long.toString(snapshotId1)) + .buildOrThrow())) + .isInstanceOf(Exception.class) + .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + + assertThatThrownBy( + () -> + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow())) + .isInstanceOf(Exception.class) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @TestTemplate + public void testIncrementalRead() throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(records1); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + // snapshot 2 + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); + helper.appendToTable(records2); + + List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); + helper.appendToTable(records3); + long snapshotId3 = table.currentSnapshot().snapshotId(); + + // snapshot 4 + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); + + List expected2 = Lists.newArrayList(); + expected2.addAll(records2); + expected2.addAll(records3); + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-snapshot-id", Long.toString(snapshotId1)) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow()), + expected2, + TestFixtures.SCHEMA); + } + + @TestTemplate + public void testFilterExpPartition() throws Exception { + Table table = + catalogExtension + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + expectedRecords.get(1).set(2, "2020-03-20"); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + DataFile dataFile1 = + helper.writeFile(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + DataFile dataFile2 = + helper.writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-21", 0), + RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); + helper.appendToTable(dataFile1, dataFile2); + TestHelpers.assertRecords( + runWithFilter(Expressions.equal("dt", "2020-03-20"), "where dt='2020-03-20'", true), + expectedRecords, + TestFixtures.SCHEMA); + } + + private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); + expectedRecords.get(0).set(0, "a"); + expectedRecords.get(1).set(0, "b"); + expectedRecords.get(2).set(0, "c"); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + DataFile dataFile = helper.writeFile(expectedRecords); + helper.appendToTable(dataFile); + + List actual = + runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); + + TestHelpers.assertRecords(actual, expectedRecords.subList(1, 3), TestFixtures.SCHEMA); + } + + @TestTemplate + public void testFilterExp() throws Exception { + testFilterExp(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); + } + + @TestTemplate + public void testFilterExpCaseInsensitive() throws Exception { + // sqlFilter does not support case-insensitive filtering: + // https://issues.apache.org/jira/browse/FLINK-16175 + testFilterExp(Expressions.greaterThanOrEqual("DATA", "b"), "where data>='b'", false); + } + + @TestTemplate + public void testPartitionTypes() throws Exception { + Schema typesSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "decimal", Types.DecimalType.of(38, 18)), + Types.NestedField.optional(3, "str", Types.StringType.get()), + Types.NestedField.optional(4, "binary", Types.BinaryType.get()), + Types.NestedField.optional(5, "date", Types.DateType.get()), + Types.NestedField.optional(6, "time", Types.TimeType.get()), + Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone())); + PartitionSpec spec = + PartitionSpec.builderFor(typesSchema) + .identity("decimal") + .identity("str") + .identity("binary") + .identity("date") + .identity("time") + .identity("timestamp") + .build(); + + Table table = + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); + List records = RandomGenericData.generate(typesSchema, 10, 0L); + GenericAppenderHelper appender = + new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + for (Record record : records) { + org.apache.iceberg.TestHelpers.Row partition = + org.apache.iceberg.TestHelpers.Row.of( + record.get(1), + record.get(2), + record.get(3), + record.get(4) == null ? null : DateTimeUtil.daysFromDate((LocalDate) record.get(4)), + record.get(5) == null ? null : DateTimeUtil.microsFromTime((LocalTime) record.get(5)), + record.get(6) == null + ? null + : DateTimeUtil.microsFromTimestamp((LocalDateTime) record.get(6))); + appender.appendToTable(partition, Collections.singletonList(record)); + } + + TestHelpers.assertRecords(run(), records, typesSchema); + } + + @TestTemplate + public void testCustomizedFlinkDataTypes() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required( + 1, + "map", + Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), + Types.NestedField.required( + 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); + Table table = catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); + List records = RandomGenericData.generate(schema, 10, 0L); + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + helper.appendToTable(records); + TestHelpers.assertRecords(run(), records, schema); + } + + private static void assertRows(List results, Row... expected) { + TestHelpers.assertRows(results, Arrays.asList(expected)); + } + + private static void waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java new file mode 100644 index 000000000000..b5bddc7767fb --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.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.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.junit.jupiter.api.BeforeEach; + +/** Test Flink SELECT SQLs. */ +public class TestFlinkScanSql extends TestFlinkSource { + private volatile TableEnvironment tEnv; + + @BeforeEach + public void before() throws IOException { + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogExtension.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Override + protected List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(sqlOptions); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java new file mode 100644 index 000000000000..ff7892071231 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -0,0 +1,90 @@ +/* + * 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.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public abstract class TestFlinkSource extends TestFlinkScan { + + @Override + protected List runWithProjection(String... projected) throws Exception { + TableSchema.Builder builder = TableSchema.builder(); + TableSchema schema = + FlinkSchemaUtil.toSchema( + FlinkSchemaUtil.convert( + catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); + for (String field : projected) { + TableColumn column = schema.getTableColumn(field).get(); + builder.field(column.getName(), column.getType()); + } + return run(FlinkSource.forRowData().project(builder.build()), Maps.newHashMap(), "", projected); + } + + @Override + protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + FlinkSource.Builder builder = + FlinkSource.forRowData().filters(Collections.singletonList(filter)); + Map options = Maps.newHashMap(); + options.put("case-sensitive", Boolean.toString(caseSensitive)); + return run(builder, options, sqlFilter, "*"); + } + + @Override + protected List runWithOptions(Map options) throws Exception { + FlinkSource.Builder builder = FlinkSource.forRowData(); + Optional.ofNullable(options.get("case-sensitive")) + .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); + Optional.ofNullable(options.get("snapshot-id")) + .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); + Optional.ofNullable(options.get("branch")).ifPresent(value -> builder.branch(value)); + Optional.ofNullable(options.get("start-tag")).ifPresent(value -> builder.startTag(value)); + Optional.ofNullable(options.get("end-tag")).ifPresent(value -> builder.endTag(value)); + Optional.ofNullable(options.get("start-snapshot-id")) + .ifPresent(value -> builder.startSnapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("end-snapshot-id")) + .ifPresent(value -> builder.endSnapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("as-of-timestamp")) + .ifPresent(value -> builder.asOfTimestamp(Long.parseLong(value))); + return run(builder, options, "", "*"); + } + + @Override + protected List run() throws Exception { + return run(FlinkSource.forRowData(), Maps.newHashMap(), "", "*"); + } + + protected abstract List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) + throws Exception; +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java new file mode 100644 index 000000000000..1814ff8f8542 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -0,0 +1,58 @@ +/* + * 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.util.List; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFlinkSourceConfig extends TestFlinkTableSource { + private static final String TABLE = "test_table"; + + @Test + public void testFlinkSessionConfig() { + getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); + Assertions.assertThatThrownBy( + () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set as-of-timestamp option for streaming reader"); + } + + @Test + public void testFlinkHintConfig() { + List result = + sql( + "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", + TABLE, System.currentTimeMillis()); + Assert.assertEquals(3, result.size()); + } + + @Test + public void testReadOptionHierarchy() { + getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); + List result = sql("SELECT * FROM %s", TABLE); + Assert.assertEquals(1, result.size()); + + result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); + Assert.assertEquals(3, result.size()); + } +} 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 new file mode 100644 index 000000000000..affd90c347dd --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -0,0 +1,85 @@ +/* + * 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.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +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.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.Test; + +/** Use the FlinkSource */ +public class TestFlinkSourceSql extends TestSqlBase { + @Override + public void before() throws IOException { + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogResource.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + @Test + public void testInferParallelismWithGlobalSetting() throws IOException { + Configuration cfg = getTableEnv().getConfig().getConfiguration(); + cfg.set(PipelineOptions.MAX_PARALLELISM, 1); + + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + List expectedRecords = Lists.newArrayList(); + long maxFileLen = 0; + for (int i = 0; i < 5; i++) { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); + DataFile dataFile = helper.writeFile(null, records); + helper.appendToTable(dataFile); + expectedRecords.addAll(records); + maxFileLen = Math.max(dataFile.fileSizeInBytes(), maxFileLen); + } + + // Make sure to generate multiple CombinedScanTasks + SqlHelpers.sql( + getTableEnv(), + "ALTER TABLE t SET ('read.split.open-file-cost'='1', 'read.split.target-size'='%s')", + maxFileLen); + + List results = run(Maps.newHashMap(), "", "*"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java new file mode 100644 index 000000000000..ff14bc406276 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -0,0 +1,614 @@ +/* + * 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.File; +import java.io.IOException; +import java.util.List; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.SqlParserException; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestFlinkTableSource extends FlinkTestBase { + + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String TABLE_NAME = "test_table"; + private final FileFormat format = FileFormat.AVRO; + private static String warehouse; + + private int scanEventCount = 0; + private ScanEvent lastScanEvent = null; + + public TestFlinkTableSource() { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + } + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeClass + public static void createWarehouse() throws IOException { + File warehouseFile = TEMPORARY_FOLDER.newFolder(); + Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + // before variables + warehouse = "file:" + warehouseFile; + } + + @Before + public void before() { + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @After + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testLimitPushDown() { + + Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) + .isInstanceOf(SqlParserException.class) + .hasMessageStartingWith("SQL parse failed."); + + Assert.assertEquals( + "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); + + String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); + List resultExceed = sql(sqlLimitExceed); + Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); + List expectedList = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedList, resultExceed); + + String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); + String explain = getTableEnv().explainSql(querySql); + String expectedExplain = "limit=[1]"; + Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); + List result = sql(querySql); + Assert.assertEquals("Should have 1 record", 1, result.size()); + Assertions.assertThat(result).containsAnyElementsOf(expectedList); + + String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); + List mixedResult = sql(sqlMixed); + Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); + Assert.assertEquals( + "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); + } + + @Test + public void testNoFilterPushDown() { + String sql = String.format("SELECT * FROM %s ", TABLE_NAME); + List result = sql(sql); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedRecords, result); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + } + + @Test + public void testFilterPushDownEqual() { + String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") == 1"; + + List result = sql(sqlLiteralRight); + Assert.assertEquals("Should have 1 record", 1, result.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownEqualNull() { + String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); + + List result = sql(sqlEqualNull); + Assert.assertEquals("Should have 0 record", 0, result.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownEqualLiteralOnLeft() { + String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") == 1"; + + List resultLeft = sql(sqlLiteralLeft); + Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNoEqual() { + String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") != 1"; + + List resultNE = sql(sqlNE); + Assert.assertEquals("Should have 2 records", 2, resultNE.size()); + + List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedNE, resultNE); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNoEqualNull() { + String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); + + List resultNE = sql(sqlNotEqualNull); + Assert.assertEquals("Should have 0 records", 0, resultNE.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownAnd() { + String sqlAnd = + String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); + + List resultAnd = sql(sqlAnd); + Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; + Assert.assertEquals( + "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownOr() { + String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; + + List resultOr = sql(sqlOr); + Assert.assertEquals("Should have 2 record", 2, resultOr.size()); + + List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedOR, resultOr); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThan() { + String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") > 1"; + + List resultGT = sql(sqlGT); + Assert.assertEquals("Should have 2 record", 2, resultGT.size()); + + List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedGT, resultGT); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThanNull() { + String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); + + List resultGT = sql(sqlGT); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownGreaterThanLiteralOnLeft() { + String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") < 3"; + + List resultGT = sql(sqlGT); + Assert.assertEquals("Should have 2 records", 2, resultGT.size()); + + List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedGT, resultGT); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThanEqual() { + String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") >= 2"; + + List resultGTE = sql(sqlGTE); + Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + + List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedGTE, resultGTE); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownGreaterThanEqualNull() { + String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); + + List resultGT = sql(sqlGTE); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { + String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") <= 2"; + + List resultGTE = sql(sqlGTE); + Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + + List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedGTE, resultGTE); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThan() { + String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") < 2"; + + List resultLT = sql(sqlLT); + Assert.assertEquals("Should have 1 record", 1, resultLT.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThanNull() { + String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); + + List resultGT = sql(sqlLT); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownLessThanLiteralOnLeft() { + String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") > 2"; + + List resultLT = sql(sqlLT); + Assert.assertEquals("Should have 1 record", 1, resultLT.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThanEqual() { + String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") <= 1"; + + List resultLTE = sql(sqlLTE); + Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLessThanEqualNull() { + String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); + + List resultGT = sql(sqlLTE); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull("Should not push down a filter", lastScanEvent); + } + + @Test + public void testFilterPushDownLessThanEqualLiteralOnLeft() { + String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") >= 3"; + + List resultLTE = sql(sqlLTE); + Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownIn() { + String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; + List resultIN = sql(sqlIN); + Assert.assertEquals("Should have 2 records", 2, resultIN.size()); + + List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedIN, resultIN); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownInNull() { + String sqlInNull = + String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); + + List result = sql(sqlInNull); + Assert.assertEquals("Should have 1 record", 1, result.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + + // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it + // and push the rest down. + String expectedScan = "ref(name=\"data\") == \"iceberg\""; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNotIn() { + String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); + + List resultNotIn = sql(sqlNotIn); + Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNotInNull() { + String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); + List resultGT = sql(sqlNotInNull); + Assert.assertEquals("Should have 0 record", 0, resultGT.size()); + Assert.assertNull( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.", + lastScanEvent); + } + + @Test + public void testFilterPushDownIsNotNull() { + String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); + String expectedFilter = "not_null(ref(name=\"data\"))"; + + List resultNotNull = sql(sqlNotNull); + Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); + + List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expected, resultNotNull); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownIsNull() { + String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); + String expectedFilter = "is_null(ref(name=\"data\"))"; + + List resultNull = sql(sqlNull); + Assert.assertEquals("Should have 1 record", 1, resultNull.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNot() { + String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); + + List resultNot = sql(sqlNot); + Assert.assertEquals("Should have 1 record", 1, resultNot.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownBetween() { + String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); + + List resultBetween = sql(sqlBetween); + Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); + + List expectedBetween = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedBetween, resultBetween); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; + Assert.assertEquals( + "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownNotBetween() { + String sqlNotBetween = + String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; + + List resultNotBetween = sql(sqlNotBetween); + Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); + Assert.assertEquals( + "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); + + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterPushDownLike() { + String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; + + String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; + List resultLike = sql(sqlLike); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals( + "The like result should produce the expected record", + Row.of(1, "iceberg", 10.0), + resultLike.get(0)); + Assert.assertEquals("Should create only one scan", 1, scanEventCount); + Assert.assertEquals( + "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + + // %% won't match the row with null value + sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; + resultLike = sql(sqlLike); + Assert.assertEquals("Should have 2 records", 2, resultLike.size()); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedRecords, resultLike); + String expectedScan = "not_null(ref(name=\"data\"))"; + Assert.assertEquals( + "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + } + + @Test + public void testFilterNotPushDownLike() { + Row expectRecord = Row.of(1, "iceberg", 10.0); + String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; + List resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 0 record", 0, resultLike.size()); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; + resultLike = sql(sqlNoPushDown); + Assert.assertEquals("Should have 1 record", 1, resultLike.size()); + Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + } + + @Test + public void testFilterPushDown2Literal() { + String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); + List result = sql(sql2Literal); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedRecords, result); + Assert.assertEquals( + "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + } + + @Test + public void testSqlParseNaN() { + // todo add some test case to test NaN + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java new file mode 100644 index 000000000000..3c0c38e1115d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -0,0 +1,127 @@ +/* + * 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.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.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class TestIcebergSourceBounded extends TestFlinkScan { + + @Override + protected List runWithProjection(String... projected) throws Exception { + Schema icebergTableSchema = + catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); + TableSchema.Builder builder = TableSchema.builder(); + TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); + for (String field : projected) { + TableColumn column = schema.getTableColumn(field).get(); + builder.field(column.getName(), column.getType()); + } + TableSchema flinkSchema = builder.build(); + Schema projectedSchema = FlinkSchemaUtil.convert(icebergTableSchema, flinkSchema); + return run(projectedSchema, Lists.newArrayList(), Maps.newHashMap(), "", projected); + } + + @Override + protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + Map options = Maps.newHashMap(); + options.put("case-sensitive", Boolean.toString(caseSensitive)); + return run(null, Collections.singletonList(filter), options, sqlFilter, "*"); + } + + @Override + protected List runWithOptions(Map options) throws Exception { + return run(null, Lists.newArrayList(), options, "", "*"); + } + + @Override + protected List run() throws Exception { + return run(null, Lists.newArrayList(), Maps.newHashMap(), "", "*"); + } + + protected List run( + Schema projectedSchema, + List filters, + Map options, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + 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(); + } + + IcebergSource.Builder sourceBuilder = + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .table(table) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.properties(options); + + DataStream stream = + env.fromSource( + sourceBuilder.build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + TypeInformation.of(RowData.class)) + .map( + new RowDataToRowMapper( + FlinkSchemaUtil.convert( + projectedSchema == null ? table.schema() : projectedSchema))); + + try (CloseableIterator iter = stream.executeAndCollect()) { + return Lists.newArrayList(iter); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java new file mode 100644 index 000000000000..0337f3597053 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -0,0 +1,203 @@ +/* + * 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.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +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.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +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.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +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.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.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergSourceBoundedGenericRecord { + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"avro", 2}, + {"parquet", 2}, + {"orc", 2} + }; + } + + private final FileFormat fileFormat; + private final int parallelism; + + public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { + this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); + this.parallelism = parallelism; + } + + @Test + public void testUnpartitionedTable() throws Exception { + Table table = + catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testPartitionedTable() throws Exception { + String dateStr = "2020-03-20"; + Table table = + catalogResource + .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, TEMPORARY_FOLDER) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @Test + public void testProjection() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + .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 = catalogResource.tableLoader()) { + tableLoader.open(); + table = tableLoader.loadTable(); + } + + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + null, + null, + false, + table.io(), + table.encryption(), + filters); + + IcebergSource.Builder sourceBuilder = + IcebergSource.builder() + .tableLoader(catalogResource.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.setAll(options); + + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + RowType rowType = FlinkSchemaUtil.convert(readSchema); + 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); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java new file mode 100644 index 000000000000..ff3348bbc3a3 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.junit.jupiter.api.BeforeEach; + +public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { + private volatile TableEnvironment tEnv; + + @BeforeEach + public void before() throws IOException { + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogExtension.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Override + protected List run( + Schema projectedSchema, + List filters, + Map options, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(options); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java new file mode 100644 index 000000000000..61e05e99e14f --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -0,0 +1,543 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceContinuous { + + public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); + + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(METRIC_REPORTER); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource tableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + private final AtomicLong randomSeed = new AtomicLong(0L); + + @Test + public void testTableScanThenIncremental() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + tableResource.table().currentSnapshot().snapshotId(); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + tableResource.table().currentSnapshot().snapshotId(); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testTableScanThenIncrementalAfterExpiration() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + + // snapshot2 + List batch2 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + + Assert.assertEquals(1, tableResource.table().history().size()); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + Assert.assertEquals( + FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 4); + List initialRecords = Lists.newArrayList(); + initialRecords.addAll(batch1); + initialRecords.addAll(batch2); + TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + tableResource.table().currentSnapshot().snapshotId(); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testEarliestSnapshot() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 4); + List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); + combinedBatch0AndBatch1.addAll(batch1); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testLatestSnapshot() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + // we want to make sure job is running first so that enumerator can + // start from the latest snapshot before inserting the next batch2 below. + waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + + // inclusive behavior for starting snapshot + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testSpecificSnapshotId() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot1) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testSpecificSnapshotTimestamp() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + // snapshot0 + List batch0 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + + // sleep for 2 ms to make sure snapshot1 has a higher timestamp value + Thread.sleep(2); + + // snapshot1 + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot1Timestamp) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + // consume data from snapshot1 + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testReadingFromBranch() throws Exception { + String branch = "b1"; + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + + List batchBase = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batchBase); + + // create branch + tableResource + .table() + .manageSnapshots() + .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .commit(); + + // snapshot1 to branch + List batch1 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch1); + + // snapshot2 to branch + List batch2 = + RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch2); + + List branchExpectedRecords = Lists.newArrayList(); + branchExpectedRecords.addAll(batchBase); + branchExpectedRecords.addAll(batch1); + branchExpectedRecords.addAll(batch2); + // reads from branch: it should contain the first snapshot (before the branch creation) followed + // by the next 2 snapshots added + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .useBranch(branch) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List resultMain = waitForResult(iter, 6); + TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + + // snapshot3 to branch + List batch3 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + + // snapshot4 to branch + List batch4 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch4); + + List result4 = waitForResult(iter, 2); + TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + } + + // read only from main branch. Should contain only the first snapshot + scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List resultMain = waitForResult(iter, 2); + TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + + List batchMain2 = + RandomGenericData.generate( + tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batchMain2); + resultMain = waitForResult(iter, 2); + TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + } + } + + private DataStream createStream(ScanContext scanContext) throws Exception { + // start the source and collect output + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream stream = + env.fromSource( + IcebergSource.forRowData() + .tableLoader(tableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(scanContext.isStreaming()) + .streamingStartingStrategy(scanContext.streamingStartingStrategy()) + .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) + .startSnapshotId(scanContext.startSnapshotId()) + .monitorInterval(Duration.ofMillis(10L)) + .branch(scanContext.branch()) + .build(), + WatermarkStrategy.noWatermarks(), + "icebergSource", + TypeInformation.of(RowData.class)) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + return stream; + } + + public static List waitForResult(CloseableIterator iter, int limit) { + List results = Lists.newArrayListWithCapacity(limit); + while (results.size() < limit) { + if (iter.hasNext()) { + results.add(iter.next()); + } else { + break; + } + } + return results; + } + + public static void waitUntilJobIsRunning(ClusterClient client) { + Awaitility.await("job should be running") + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofMillis(10)) + .untilAsserted(() -> assertThat(getRunningJobs(client)).isNotEmpty()); + } + + public static List getRunningJobs(ClusterClient client) throws Exception { + Collection statusMessages = client.listJobs().get(); + return statusMessages.stream() + .filter(status -> status.getJobState() == JobStatus.RUNNING) + .map(JobStatusMessage::getJobId) + .collect(Collectors.toList()); + } + + private static void assertThatIcebergEnumeratorMetricsExist() { + assertThatIcebergSourceMetricExists( + "enumerator", "coordinator.enumerator.elapsedSecondsSinceLastSplitDiscovery"); + assertThatIcebergSourceMetricExists("enumerator", "coordinator.enumerator.unassignedSplits"); + assertThatIcebergSourceMetricExists("enumerator", "coordinator.enumerator.pendingRecords"); + } + + private static void assertThatIcebergSourceMetricExists( + String metricGroupPattern, String metricName) { + Optional groups = METRIC_REPORTER.findGroup(metricGroupPattern); + assertThat(groups).isPresent(); + assertThat( + METRIC_REPORTER.getMetricsByGroup(groups.get()).keySet().stream() + .map(name -> groups.get().getMetricIdentifier(name))) + .satisfiesOnlyOnce( + fullMetricName -> assertThat(fullMetricName).containsSubsequence(metricName)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java new file mode 100644 index 000000000000..7d991ee603c9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -0,0 +1,297 @@ +/* + * 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.time.Duration; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.iceberg.FileFormat; +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.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceFailover { + + private static final int PARALLELISM = 4; + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); + + @Rule + public final HadoopTableResource sinkTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); + + protected IcebergSource.Builder sourceBuilder() { + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + return IcebergSource.forRowData() + .tableLoader(sourceTableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + } + + protected Schema schema() { + return TestFixtures.SCHEMA; + } + + protected List generateRecords(int numRecords, long seed) { + return RandomGenericData.generate(schema(), numRecords, seed); + } + + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + + @Test + public void testBoundedWithTaskManagerFailover() throws Exception { + testBoundedIcebergSource(FailoverType.TM); + } + + @Test + public void testBoundedWithJobManagerFailover() throws Exception { + testBoundedIcebergSource(FailoverType.JM); + } + + private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + List expectedRecords = Lists.newArrayList(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + + DataStream stream = + env.fromSource( + sourceBuilder().build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + DataStream streamFailingInTheMiddleOfReading = + RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(streamFailingInTheMiddleOfReading) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); + JobID jobId = jobClient.getJobID(); + + RecordCounterToFail.waitToFail(); + triggerFailover( + failoverType, + jobId, + RecordCounterToFail::continueProcessing, + miniClusterResource.getMiniCluster()); + + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + } + + @Test + public void testContinuousWithTaskManagerFailover() throws Exception { + testContinuousIcebergSource(FailoverType.TM); + } + + @Test + public void testContinuousWithJobManagerFailover() throws Exception { + testContinuousIcebergSource(FailoverType.JM); + } + + private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + List expectedRecords = Lists.newArrayList(); + + List batch = generateRecords(2, 0); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(10L); + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + + DataStream stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(stream) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + + JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); + JobID jobId = jobClient.getJobID(); + + for (int i = 1; i < 5; i++) { + Thread.sleep(10); + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + if (i == 2) { + triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + } + } + + // wait longer for continuous source to reduce flakiness + // because CI servers tend to be overloaded. + assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + } + + // ------------------------------------------------------------------------ + // test utilities copied from Flink's FileSourceTextLinesITCase + // ------------------------------------------------------------------------ + + private enum FailoverType { + NONE, + TM, + JM + } + + private static void triggerFailover( + FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + switch (type) { + case NONE: + afterFailAction.run(); + break; + case TM: + restartTaskManager(afterFailAction, miniCluster); + break; + case JM: + triggerJobManagerFailover(jobId, afterFailAction, miniCluster); + break; + } + } + + private static void triggerJobManagerFailover( + JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { + HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + private static class RecordCounterToFail { + + private static AtomicInteger records; + private static CompletableFuture fail; + private static CompletableFuture continueProcessing; + + private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { + + records = new AtomicInteger(); + fail = new CompletableFuture<>(); + continueProcessing = new CompletableFuture<>(); + return stream.map( + record -> { + boolean reachedFailPoint = records.incrementAndGet() > failAfter; + boolean notFailedYet = !fail.isDone(); + if (notFailedYet && reachedFailPoint) { + fail.complete(null); + continueProcessing.get(); + } + return record; + }); + } + + private static void waitToFail() throws ExecutionException, InterruptedException { + fail.get(); + } + + private static void continueProcessing() { + continueProcessing.complete(null); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java new file mode 100644 index 000000000000..f7dc931c506c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.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.flink.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + protected IcebergSource.Builder sourceBuilder() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> { + SimpleDataUtil.equalsRecords( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema()); + }); + } + + private List convertLocalDateTimeToMilli(List records) { + return records.stream() + .peek( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java new file mode 100644 index 000000000000..df148c212ebd --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java @@ -0,0 +1,102 @@ +/* + * 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.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.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +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.StructLikeSet; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSourceReaderDeletes extends TestFlinkReaderDeletesBase { + + private static final int PARALLELISM = 4; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @Override + protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) + throws IOException { + Schema projected = testTable.schema().select(columns); + RowType rowType = FlinkSchemaUtil.convert(projected); + + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.WAREHOUSE_LOCATION, + hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + properties.put( + CatalogProperties.CLIENT_POOL_SIZE, + Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); + CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); + TableLoader hiveTableLoader = + TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName)); + hiveTableLoader.open(); + try (TableLoader tableLoader = hiveTableLoader) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream stream = + env.fromSource( + IcebergSource.builder() + .tableLoader(tableLoader) + .assignerFactory(new SimpleSplitAssignerFactory()) + .project(projected) + .build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + TypeInformation.of(RowData.class)); + + try (CloseableIterator iter = stream.executeAndCollect()) { + List rowDataList = Lists.newArrayList(iter); + StructLikeSet set = StructLikeSet.create(projected.asStruct()); + rowDataList.forEach( + rowData -> { + RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); + set.add(wrapper.wrap(rowData)); + }); + return set; + } catch (Exception e) { + throw new IOException("Failed to collect result", e); + } + } + } +} 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 new file mode 100644 index 000000000000..4250460d278d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.ZoneId; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +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.Test; + +/** Use the IcebergSource (FLIP-27) */ +public class TestIcebergSourceSql extends TestSqlBase { + private static final Schema SCHEMA_TS = + new Schema( + required(1, "t1", Types.TimestampType.withoutZone()), + required(2, "t2", Types.LongType.get())); + + @Override + public void before() throws IOException { + TableEnvironment tableEnvironment = getTableEnv(); + Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); + tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + + tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); + SqlHelpers.sql( + tableEnvironment, + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + catalogResource.warehouse()); + SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); + + tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private Record generateRecord(Instant t1, long t2) { + Record record = GenericRecord.create(SCHEMA_TS); + record.setField("t1", t1.atZone(ZoneId.systemDefault()).toLocalDateTime()); + record.setField("t2", t2); + return record; + } + + /** Generates the records in the expected order, with respect to their datafile */ + private List generateExpectedRecords(boolean ascending) throws Exception { + Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); + long baseTime = 1702382109000L; + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + + Record file1Record1 = + generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); + Record file1Record2 = + generateRecord( + Instant.ofEpochMilli(baseTime - 10 * 1000L), baseTime + (1000 * 60 * 60 * 24 * 35L)); + + List recordsDataFile1 = Lists.newArrayList(); + recordsDataFile1.add(file1Record1); + recordsDataFile1.add(file1Record2); + DataFile dataFile1 = helper.writeFile(recordsDataFile1); + + Record file2Record1 = + generateRecord( + Instant.ofEpochMilli(baseTime + 14 * 1000L), baseTime - (1000 * 60 * 60 * 24 * 30L)); + Record file2Record2 = + generateRecord( + Instant.ofEpochMilli(baseTime + 12 * 1000L), baseTime - (1000 * 60 * 61 * 24 * 35L)); + + List recordsDataFile2 = Lists.newArrayList(); + recordsDataFile2.add(file2Record1); + recordsDataFile2.add(file2Record2); + + DataFile dataFile2 = helper.writeFile(recordsDataFile2); + helper.appendToTable(dataFile1, dataFile2); + + // Expected records if the splits are ordered + // - ascending (watermark from t1) - records from the split with early timestamps, then + // records from the split with late timestamps + // - descending (watermark from t2) - records from the split with old longs, then records + // from the split with new longs + List expected = Lists.newArrayList(); + if (ascending) { + expected.addAll(recordsDataFile1); + expected.addAll(recordsDataFile2); + } else { + expected.addAll(recordsDataFile2); + expected.addAll(recordsDataFile1); + } + return expected; + } + + /** Tests the order of splits returned when setting the watermark-column options */ + @Test + public void testWatermarkOptionsAscending() throws Exception { + List expected = generateExpectedRecords(true); + TestHelpers.assertRecordsWithOrder( + run( + ImmutableMap.of("watermark-column", "t1", "split-file-open-cost", "128000000"), + "", + "*"), + expected, + SCHEMA_TS); + } + + /** + * Tests the order of splits returned when setting the watermark-column and + * watermark-column-time-unit" options + */ + @Test + public void testWatermarkOptionsDescending() throws Exception { + List expected = generateExpectedRecords(false); + TestHelpers.assertRecordsWithOrder( + run( + ImmutableMap.of( + "watermark-column", + "t2", + "watermark-column-time-unit", + "MILLISECONDS", + "split-file-open-cost", + "128000000"), + "", + "*"), + expected, + SCHEMA_TS); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java new file mode 100644 index 000000000000..bc1808ee77da --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -0,0 +1,408 @@ +/* + * 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 static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; + +import java.io.Serializable; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +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.awaitility.Awaitility; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap windows = Maps.newConcurrentMap(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + + @Rule + public final MiniClusterWithClientResource miniClusterResource = + new MiniClusterWithClientResource( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .withHaLeadershipControl() + .build()); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *

      + *
    • - Ordering of the splits + *
    • - Emitting of watermarks + *
    • - Firing windows based on watermarks + *
    + * + *

    The test generates 4 splits + * + *

      + *
    • - Split 1 - Watermark 100 min + *
    • - Split 2, 3 - Watermark 0 min + *
    • - Split 4 - Watermark 6 min + *
    + * + *

    Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

    Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

      + *
    • - Split 2, 3 + *
    • - Split 4 + *
    • - Split 1 + *
    + * + *

    As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

    Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + windows.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + windows.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + windows.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

      + *
    • - Emitting of watermarks + *
    • - Watermark alignment + *
    + * + *

    The test generates 3 splits + * + *

      + *
    • - Split 1 - Watermark 100 min + *
    • - Split 2, 3 - Watermark 0 min + *
    + * + * The splits are read in the following order: + * + *
      + *
    • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
    • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
    + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

    The status of the watermark alignment is checked by the alignment related metrics. + * + *

    Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

    After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling() throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records in advance + + // File 1 - Later records (Watermark 6.000.000 - 100 min) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + List batch1 = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + + // File 2 - First records (Watermark 0 - 0 min) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List batch2 = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch2.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + // File 3 - Some records will be blocked (Watermark 900.000 - 15 min) + List batch3 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + + // File 4 - Some records will be blocked (Watermark 900.000 - 15 min) + List batch4 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + + // File 5 - Records which will remove the block (Watermark 5.400.000 - 90 min) + List batch5 = + ImmutableList.of( + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + CommonTestUtils.waitForAllTaskRunning( + miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + + // Insert the first data into the table + dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + dataAppender.appendToTable(dataAppender.writeFile(batch3), dataAppender.writeFile(batch4)); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + dataAppender.appendToTable(batch5); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); + } + } + + protected IcebergSource source() { + return IcebergSource.builder() + .tableLoader(sourceTableResource.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return reporter.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + } + + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } +} 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 new file mode 100644 index 000000000000..40dfda723749 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -0,0 +1,299 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.util.Base64; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameters; +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.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.TestHelpers; +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.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.io.TempDir; + +public class TestMetadataTableReadableMetrics extends CatalogTestBase { + private static final String TABLE_NAME = "test_table"; + + @Parameters(name = "catalogName={0}, baseNamespace={1}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + String catalogName = "testhive"; + Namespace baseNamespace = Namespace.empty(); + parameters.add(new Object[] {catalogName, baseNamespace}); + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + private @TempDir Path temp; + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema PRIMITIVE_SCHEMA = + new Schema( + required(1, "booleanCol", Types.BooleanType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "longCol", Types.LongType.get()), + required(4, "floatCol", Types.FloatType.get()), + required(5, "doubleCol", Types.DoubleType.get()), + optional(6, "decimalCol", Types.DecimalType.of(10, 2)), + optional(7, "stringCol", Types.StringType.get()), + optional(8, "fixedCol", Types.FixedType.ofLength(3)), + optional(9, "binaryCol", Types.BinaryType.get())); + + private Table createPrimitiveTable() throws IOException { + Table table = + catalog.createTable( + TableIdentifier.of(DATABASE, TABLE_NAME), + PRIMITIVE_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + List records = + Lists.newArrayList( + createPrimitiveRecord( + false, + 1, + 1L, + 0, + 1.0D, + new BigDecimal("1.00"), + "1", + Base64.getDecoder().decode("1111"), + ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), + createPrimitiveRecord( + true, + 2, + 2L, + 0, + 2.0D, + new BigDecimal("2.00"), + "2", + Base64.getDecoder().decode("2222"), + ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), + createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), + createPrimitiveRecord( + false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); + table.newAppend().appendFile(dataFile).commit(); + return table; + } + + private void createNestedTable() throws IOException { + Table table = + validationCatalog.createTable( + TableIdentifier.of(DATABASE, TABLE_NAME), + NESTED_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + + List records = + Lists.newArrayList( + createNestedRecord(0L, 0.0), + createNestedRecord(1L, Double.NaN), + createNestedRecord(null, null)); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); + table.newAppend().appendFile(dataFile).commit(); + } + + @BeforeEach + public void before() { + super.before(); + sql("USE CATALOG %s", catalogName); + sql("CREATE DATABASE %s", DATABASE); + sql("USE %s", DATABASE); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + protected GenericRecord createPrimitiveRecord( + boolean booleanCol, + int intCol, + long longCol, + float floatCol, + double doubleCol, + BigDecimal decimalCol, + String stringCol, + byte[] fixedCol, + ByteBuffer binaryCol) { + GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); + record.set(0, booleanCol); + record.set(1, intCol); + record.set(2, longCol); + record.set(3, floatCol); + record.set(4, doubleCol); + record.set(5, decimalCol); + record.set(6, stringCol); + record.set(7, fixedCol); + record.set(8, binaryCol); + return record; + } + + private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { + GenericRecord record = GenericRecord.create(NESTED_SCHEMA); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, longCol); + leaf.set(1, doubleCol); + nested.set(0, leaf); + record.set(0, nested); + return record; + } + + protected Object[] row(Object... values) { + return values; + } + + @TestTemplate + public void testPrimitiveColumns() throws Exception { + createPrimitiveTable(); + List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + + Row binaryCol = + Row.of( + 52L, + 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 fixedCol = + Row.of( + 44L, + 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"); + + List expected = + Lists.newArrayList( + Row.of( + Row.of( + binaryCol, + booleanCol, + decimalCol, + doubleCol, + fixedCol, + floatCol, + intCol, + longCol, + stringCol))); + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testSelectPrimitiveValues() throws Exception { + createPrimitiveTable(); + + TestHelpers.assertRows( + sql( + "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s$files", + TABLE_NAME), + ImmutableList.of(Row.of(1, true))); + + TestHelpers.assertRows( + sql("SELECT content, readable_metrics.longCol.value_count FROM %s$files", TABLE_NAME), + ImmutableList.of(Row.of(0, 4L))); + + TestHelpers.assertRows( + sql("SELECT readable_metrics.longCol.value_count, content FROM %s$files", TABLE_NAME), + ImmutableList.of(Row.of(4L, 0))); + } + + @TestTemplate + public void testSelectNestedValues() throws Exception { + createNestedTable(); + TestHelpers.assertRows( + sql( + "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " + + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s$files", + TABLE_NAME), + ImmutableList.of(Row.of(0L, 3L))); + } + + @TestTemplate + 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 metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); + + TestHelpers.assertRows( + sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java new file mode 100644 index 000000000000..25ecec23d216 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -0,0 +1,186 @@ +/* + * 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.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataProjection; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +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.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestProjectMetaColumn { + + @Rule public final TemporaryFolder folder = new TemporaryFolder(); + private final FileFormat format; + + @Parameterized.Parameters(name = "fileFormat={0}") + public static Iterable parameters() { + return Lists.newArrayList( + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.ORC}, + new Object[] {FileFormat.AVRO}); + } + + public TestProjectMetaColumn(FileFormat format) { + this.format = format; + } + + private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { + // Create the table with given format version. + String location = folder.getRoot().getAbsolutePath(); + Table table = + SimpleDataUtil.createTable( + location, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + false); + + List rows = + Lists.newArrayList( + SimpleDataUtil.createInsert(1, "AAA"), + SimpleDataUtil.createInsert(2, "BBB"), + SimpleDataUtil.createInsert(3, "CCC")); + writeAndCommit(table, ImmutableList.of(), false, rows); + + FlinkInputFormat input = + FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); + + List results = Lists.newArrayList(); + TestHelpers.readRowData( + input, + rowData -> { + // If project to remove the meta columns, it will get a RowDataProjection. + Assert.assertTrue(rowData instanceof GenericRowData); + results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); + }); + + // Assert the results. + TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); + } + + @Test + public void testV1SkipToRemoveMetaColumn() throws IOException { + testSkipToRemoveMetaColumn(1); + } + + @Test + public void testV2SkipToRemoveMetaColumn() throws IOException { + testSkipToRemoveMetaColumn(2); + } + + @Test + public void testV2RemoveMetaColumn() throws Exception { + // Create the v2 table. + String location = folder.getRoot().getAbsolutePath(); + Table table = + SimpleDataUtil.createTable( + location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); + + List rows = + Lists.newArrayList( + SimpleDataUtil.createInsert(1, "AAA"), + SimpleDataUtil.createDelete(1, "AAA"), + SimpleDataUtil.createInsert(2, "AAA"), + SimpleDataUtil.createInsert(2, "BBB")); + int eqFieldId = table.schema().findField("data").fieldId(); + writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); + + FlinkInputFormat input = + FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); + + List results = Lists.newArrayList(); + TestHelpers.readRowData( + input, + rowData -> { + // If project to remove the meta columns, it will get a RowDataProjection. + Assert.assertTrue(rowData instanceof RowDataProjection); + results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); + }); + + // Assert the results. + TestHelpers.assertRows( + ImmutableList.of( + SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")), + results, + SimpleDataUtil.ROW_TYPE); + } + + private void writeAndCommit( + Table table, List eqFieldIds, boolean upsert, List rows) + throws IOException { + TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); + try (TaskWriter io = writer) { + for (RowData row : rows) { + io.write(row); + } + } + + RowDelta delta = table.newRowDelta(); + WriteResult result = writer.complete(); + + for (DataFile dataFile : result.dataFiles()) { + delta.addRows(dataFile); + } + + for (DeleteFile deleteFile : result.deleteFiles()) { + delta.addDeletes(deleteFile); + } + + delta.commit(); + } + + private TaskWriter createTaskWriter( + Table table, List equalityFieldIds, boolean upsert) { + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + SimpleDataUtil.ROW_TYPE, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, + format, + table.properties(), + equalityFieldIds, + upsert); + + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java new file mode 100644 index 000000000000..6ef40693827e --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.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.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.avro.generic.GenericRecord; +import org.apache.iceberg.flink.AvroGenericRecordConverterBase; +import org.apache.iceberg.flink.DataGenerator; + +public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { + @Override + protected void testConverter(DataGenerator dataGenerator) { + RowDataToAvroGenericRecordConverter converter = + RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); + GenericRecord expected = dataGenerator.generateAvroGenericRecord(); + GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java new file mode 100644 index 000000000000..317301260f66 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.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.flink.source; + +import java.io.IOException; +import org.apache.flink.configuration.Configuration; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.junit.Assert; +import org.junit.Test; + +public class TestSourceUtil { + @Test + public void testInferedParallelism() throws IOException { + Configuration configuration = new Configuration(); + // Empty table, infer parallelism should be at least 1 + int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits + // num : 2 + parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + + // 2 splits and limit is 1 , max infer parallelism is default 100, + // which is greater than splits num and limit, the parallelism is the limit value : 1 + parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 + configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); + parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : + // 1 + parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + + // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 + configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); + Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java new file mode 100644 index 000000000000..dda46033143e --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.TableLoader; +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.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +/** Test other more advanced usage of SQL. They don't need to run for every file format. */ +public abstract class TestSqlBase { + @ClassRule + public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = + MiniClusterResource.createWithClassloaderCheckDisabled(); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopCatalogResource catalogResource = + new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + + private volatile TableEnvironment tEnv; + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Before + public abstract void before() throws IOException; + + @Test + public void testResiduals() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + List writeRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + writeRecords.get(0).set(1, 123L); + writeRecords.get(0).set(2, "2020-03-20"); + writeRecords.get(1).set(1, 456L); + writeRecords.get(1).set(2, "2020-03-20"); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.add(writeRecords.get(0)); + + DataFile dataFile1 = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), writeRecords); + DataFile dataFile2 = + helper.writeFile( + TestHelpers.Row.of("2020-03-21", 0), + RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); + helper.appendToTable(dataFile1, dataFile2); + + org.apache.iceberg.flink.TestHelpers.assertRecords( + run(Maps.newHashMap(), "where dt='2020-03-20' and id=123", "*"), + expectedRecords, + TestFixtures.SCHEMA); + } + + @Test + public void testExposeLocality() throws Exception { + Table table = + catalogResource + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + TableLoader tableLoader = TableLoader.fromHadoopTable(table.location()); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 10, 0L); + expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + helper.appendToTable(dataFile); + + // test sql api + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); + + List results = SqlHelpers.sql(getTableEnv(), "select * from t"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + + // test table api + tableConf.setBoolean( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), true); + FlinkSource.Builder builder = FlinkSource.forRowData().tableLoader(tableLoader).table(table); + + // When running with CI or local, `localityEnabled` will be false even if this configuration is + // enabled + Assert.assertFalse( + "Expose split locality info should be false.", + SourceUtil.isLocalityEnabled(table, tableConf, true)); + + results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + } + + protected List run( + Map options, String sqlFilter, String... sqlSelectedFields) { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(options); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java new file mode 100644 index 000000000000..9e043bbbbbd2 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -0,0 +1,430 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestStreamScanSql extends CatalogTestBase { + private static final String TABLE = "test_table"; + private static final FileFormat FORMAT = FileFormat.PARQUET; + + private TableEnvironment tEnv; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = + EnvironmentSettings.newInstance().inStreamingMode(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + + StreamTableEnvironment streamTableEnv = + StreamTableEnvironment.create(env, settingsBuilder.build()); + streamTableEnv + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + tEnv = streamTableEnv; + } + } + } + return tEnv; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); + sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + super.clean(); + } + + private void insertRows(String partition, Table table, Row... rows) throws IOException { + insertRows(partition, SnapshotRef.MAIN_BRANCH, table, rows); + } + + private void insertRows(String partition, String branch, Table table, Row... rows) + throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, FORMAT, temporaryDirectory); + + GenericRecord gRecord = GenericRecord.create(table.schema()); + List records = Lists.newArrayList(); + for (Row row : rows) { + records.add( + gRecord.copy( + "id", row.getField(0), + "data", row.getField(1), + "dt", row.getField(2))); + } + + if (partition != null) { + appender.appendToTable(TestHelpers.Row.of(partition, 0), branch, records); + } else { + appender.appendToTable(branch, records); + } + } + + private void insertRowsInBranch(String branch, Table table, Row... rows) throws IOException { + insertRows(null, branch, table, rows); + } + + private void insertRows(Table table, Row... rows) throws IOException { + insertRows(null, table, rows); + } + + private void assertRows(List expectedRows, Iterator iterator) { + for (Row expectedRow : expectedRows) { + assertThat(iterator).hasNext(); + Row actualRow = iterator.next(); + assertThat(actualRow.getArity()).isEqualTo(3); + assertThat(actualRow.getField(0)).isEqualTo(expectedRow.getField(0)); + assertThat(actualRow.getField(1)).isEqualTo(expectedRow.getField(1)); + assertThat(actualRow.getField(2)).isEqualTo(expectedRow.getField(2)); + } + } + + @TestTemplate + public void testUnPartitionedTable() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + + Row row1 = Row.of(1, "aaa", "2021-01-01"); + insertRows(table, row1); + assertRows(ImmutableList.of(row1), iterator); + + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row2); + assertRows(ImmutableList.of(row2), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR) PARTITIONED BY (dt)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + Row row1 = Row.of(1, "aaa", "2021-01-01"); + insertRows("2021-01-01", table, row1); + assertRows(ImmutableList.of(row1), iterator); + + Row row2 = Row.of(2, "bbb", "2021-01-02"); + insertRows("2021-01-02", table, row2); + assertRows(ImmutableList.of(row2), iterator); + + Row row3 = Row.of(1, "aaa", "2021-01-02"); + insertRows("2021-01-02", table, row3); + assertRows(ImmutableList.of(row3), iterator); + + Row row4 = Row.of(2, "bbb", "2021-01-01"); + insertRows("2021-01-01", table, row4); + assertRows(ImmutableList.of(row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testConsumeFromBeginning() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1, row2); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + assertRows(ImmutableList.of(row1, row2), iterator); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + insertRows(table, row3); + assertRows(ImmutableList.of(row3), iterator); + + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row4); + assertRows(ImmutableList.of(row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + /** + * Insert records on the main branch. Then, insert in a named branch. Reads from the main branch + * and assert that the only records from main are returned + */ + public void testConsumeFilesFromMainBranch() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots on main branch + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + + insertRows(table, row1, row2); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName).commit(); + + // insert on the 'b1' branch + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + + insertRowsInBranch(branchName, table, row3, row4); + + // read from main + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row1, row2), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + /** + * Insert records on the main branch. Creates a named branch. Insert record on named branch. Then + * select from the named branch and assert all the records are returned. + */ + public void testConsumeFilesFromBranch() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots on main branch + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + + insertRows(table, row1, row2); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName).commit(); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", + TABLE, branchName); + + try (CloseableIterator iterator = result.collect()) { + assertRows(ImmutableList.of(row1, row2), iterator); + // insert on the 'b1' branch + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRowsInBranch(branchName, table, row3, row4); + assertRows(ImmutableList.of(row3, row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + /** + * Insert records on branch b1. Then insert record on b2. Then select from each branch and assert + * the correct records are returned + */ + public void testConsumeFilesFromTwoBranches() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + String branch1 = "b1"; + String branch2 = "b2"; + table.manageSnapshots().createBranch(branch1).commit(); + table.manageSnapshots().createBranch(branch2).commit(); + + // Produce two snapshots on main branch + Row row1Branch1 = Row.of(1, "b1", "2021-01-01"); + Row row2Branch1 = Row.of(2, "b1", "2021-01-01"); + + Row row1Branch2 = Row.of(2, "b2", "2021-01-01"); + Row row2Branch2 = Row.of(3, "b3", "2021-01-01"); + + insertRowsInBranch(branch1, table, row1Branch1, row2Branch1); + insertRowsInBranch(branch2, table, row1Branch2, row2Branch2); + + TableResult resultBranch1 = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", + TABLE, branch1); + + try (CloseableIterator iterator = resultBranch1.collect()) { + assertRows(ImmutableList.of(row1Branch1, row2Branch1), iterator); + Row another = Row.of(4, "ccc", "2021-01-01"); + insertRowsInBranch(branch1, table, another); + assertRows(ImmutableList.of(another), iterator); + } + + TableResult resultBranch2 = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", + TABLE, branch2); + try (CloseableIterator iterator = resultBranch2.collect()) { + assertRows(ImmutableList.of(row1Branch2, row2Branch2), iterator); + Row another = Row.of(4, "ccc", "2021-01-01"); + insertRowsInBranch(branch2, table, another); + assertRows(ImmutableList.of(another), iterator); + } + + resultBranch1.getJobClient().ifPresent(JobClient::cancel); + resultBranch2.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testConsumeFromStartSnapshotId() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots. + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1); + insertRows(table, row2); + + long startSnapshotId = table.currentSnapshot().snapshotId(); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row3, row4); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " + + "'start-snapshot-id'='%d')*/", + TABLE, startSnapshotId); + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row3, row4), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testConsumeFromStartTag() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots. + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1); + insertRows(table, row2); + + String tagName = "t1"; + long startSnapshotId = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row3, row4); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " + + "'start-tag'='%s')*/", + TABLE, tagName); + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row3, row4), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + + Assertions.assertThatThrownBy( + () -> + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " + + "'start-snapshot-id'='%d' )*/", + TABLE, tagName, startSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + } +} 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 new file mode 100644 index 000000000000..9c4f476b02b4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -0,0 +1,402 @@ +/* + * 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 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.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +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.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; + private static final long WAIT_TIME_MILLIS = 10 * 1000L; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); + } + + @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()); + } + + private void runSourceFunctionInTask( + TestSourceContext sourceContext, StreamingMonitorFunction function) { + Thread task = + new Thread( + () -> { + try { + function.run(sourceContext); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + task.start(); + } + + @TestTemplate + public void testConsumeWithoutStartSnapshotId() throws Exception { + List> recordsList = generateRecordsAndCommitTxn(10); + ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @TestTemplate + public void testConsumeFromStartSnapshotId() throws Exception { + // Commit the first five transactions. + generateRecordsAndCommitTxn(5); + long startSnapshotId = table.currentSnapshot().snapshotId(); + + // Commit the next five transactions. + List> recordsList = generateRecordsAndCommitTxn(5); + + ScanContext scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .startSnapshotId(startSnapshotId) + .build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @TestTemplate + public void testConsumeFromStartTag() throws Exception { + // Commit the first five transactions. + generateRecordsAndCommitTxn(5); + long startSnapshotId = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); + + // Commit the next five transactions. + List> recordsList = generateRecordsAndCommitTxn(5); + + ScanContext scanContext = + ScanContext.builder().monitorInterval(Duration.ofMillis(100)).startTag(tagName).build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @TestTemplate + public void testCheckpointRestore() throws Exception { + List> recordsList = generateRecordsAndCommitTxn(10); + ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); + + StreamingMonitorFunction func = createFunction(scanContext); + OperatorSubtaskState state; + try (AbstractStreamOperatorTestHarness harness = createHarness(func)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, func); + + awaitExpectedSplits(sourceContext); + + state = harness.snapshot(1, 1); + + // Stop the stream task. + func.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + + List> newRecordsList = generateRecordsAndCommitTxn(10); + StreamingMonitorFunction newFunc = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(newFunc)) { + harness.setup(); + // Recover to process the remaining snapshots. + harness.initializeState(state); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, newFunc); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + newFunc.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(newRecordsList)), SCHEMA); + } + } + + private void awaitExpectedSplits(TestSourceContext sourceContext) { + Awaitility.await("expected splits should be produced") + .atMost(Duration.ofMillis(WAIT_TIME_MILLIS)) + .untilAsserted( + () -> { + assertThat(sourceContext.latch.getCount()).isEqualTo(0); + assertThat(sourceContext.splits).as("Should produce the expected splits").hasSize(1); + }); + } + + @TestTemplate + public void testInvalidMaxPlanningSnapshotCount() { + ScanContext scanContext1 = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .maxPlanningSnapshotCount(0) + .build(); + + assertThatThrownBy(() -> createFunction(scanContext1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("The max-planning-snapshot-count must be greater than zero"); + + ScanContext scanContext2 = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .maxPlanningSnapshotCount(-10) + .build(); + + assertThatThrownBy(() -> createFunction(scanContext2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("The max-planning-snapshot-count must be greater than zero"); + } + + @TestTemplate + public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { + generateRecordsAndCommitTxn(10); + + // Use the oldest snapshot as starting to avoid the initial case. + long oldestSnapshotId = SnapshotUtil.oldestAncestor(table).snapshotId(); + + ScanContext scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .splitSize(1000L) + .startSnapshotId(oldestSnapshotId) + .maxPlanningSnapshotCount(Integer.MAX_VALUE) + .build(); + + FlinkInputSplit[] expectedSplits = + FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); + + assertThat(expectedSplits).hasSize(9); + + // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the + // total splits number + for (int maxPlanningSnapshotCount : ImmutableList.of(1, 9, 15)) { + scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(500)) + .startSnapshotId(oldestSnapshotId) + .splitSize(1000L) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + CountDownLatch latch = new CountDownLatch(1); + TestSourceContext sourceContext = new TestSourceContext(latch); + function.sourceContext(sourceContext); + function.monitorAndForwardSplits(); + + if (maxPlanningSnapshotCount < 10) { + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); + } + } + } + } + + private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { + List> expectedRecords = Lists.newArrayList(); + for (int i = 0; i < commitTimes; i++) { + List records = RandomGenericData.generate(SCHEMA, 100, 0L); + expectedRecords.add(records); + + // Commit those records to iceberg table. + writeRecords(records); + } + return expectedRecords; + } + + private void writeRecords(List records) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); + appender.appendToTable(records); + } + + private StreamingMonitorFunction createFunction(ScanContext scanContext) { + return new StreamingMonitorFunction( + TestTableLoader.of(tableDir.getAbsolutePath()), scanContext); + } + + private AbstractStreamOperatorTestHarness createHarness( + StreamingMonitorFunction function) throws Exception { + StreamSource streamSource = + new StreamSource<>(function); + return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0); + } + + private class TestSourceContext implements SourceFunction.SourceContext { + private final List splits = Lists.newArrayList(); + private final Object checkpointLock = new Object(); + private final CountDownLatch latch; + + TestSourceContext(CountDownLatch latch) { + this.latch = latch; + } + + @Override + public void collect(FlinkInputSplit element) { + splits.add(element); + latch.countDown(); + } + + @Override + public void collectWithTimestamp(FlinkInputSplit element, long timestamp) { + collect(element); + } + + @Override + public void emitWatermark(Watermark mark) {} + + @Override + public void markAsTemporarilyIdle() {} + + @Override + public Object getCheckpointLock() { + return checkpointLock; + } + + @Override + public void close() {} + + private List toRows() throws IOException { + FlinkInputFormat format = + FlinkSource.forRowData() + .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) + .buildFormat(); + + List rows = Lists.newArrayList(); + for (FlinkInputSplit split : splits) { + format.open(split); + + RowData element = null; + try { + while (!format.reachedEnd()) { + element = format.nextRecord(element); + rows.add(Row.of(element.getInt(0), element.getString(1).toString())); + } + } finally { + format.close(); + } + } + + return rows; + } + } +} 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 new file mode 100644 index 000000000000..1606ee9f9648 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -0,0 +1,293 @@ +/* + * 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 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.Collections; +import java.util.List; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; +import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +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.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); + } + + @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()); + } + + @TestTemplate + public void testProcessAllRecords() throws Exception { + List> expectedRecords = generateRecordsAndCommitTxn(10); + + List splits = generateSplits(); + assertThat(splits).hasSize(10); + + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + SteppingMailboxProcessor processor = createLocalMailbox(harness); + + List expected = Lists.newArrayList(); + for (int i = 0; i < splits.size(); i++) { + // Process this element to enqueue to mail-box. + harness.processElement(splits.get(i), -1); + + // Run the mail-box once to read all records from the given split. + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); + + // Assert the output has expected elements. + expected.addAll(expectedRecords.get(i)); + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + } + } + + @TestTemplate + public void testTriggerCheckpoint() throws Exception { + // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading + // records from + // split1. + List> expectedRecords = generateRecordsAndCommitTxn(3); + + List splits = generateSplits(); + assertThat(splits).hasSize(3); + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + SteppingMailboxProcessor processor = createLocalMailbox(harness); + + harness.processElement(splits.get(0), ++timestamp); + harness.processElement(splits.get(1), ++timestamp); + harness.processElement(splits.get(2), ++timestamp); + + // Trigger snapshot state, it will start to work once all records from split0 are read. + processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); + + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); + + TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); + + // Read records from split1. + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); + + // Read records from split2. + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); + + TestHelpers.assertRecords( + readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); + } + } + + @TestTemplate + public void testCheckpointRestore() throws Exception { + List> expectedRecords = generateRecordsAndCommitTxn(15); + + List splits = generateSplits(); + assertThat(splits).hasSize(15); + + OperatorSubtaskState state; + List expected = Lists.newArrayList(); + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + // Enqueue all the splits. + for (FlinkInputSplit split : splits) { + harness.processElement(split, -1); + } + + // Read all records from the first five splits. + SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); + for (int i = 0; i < 5; i++) { + expected.addAll(expectedRecords.get(i)); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); + + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + + // Snapshot state now, there're 10 splits left in the state. + state = harness.snapshot(1, 1); + } + + expected.clear(); + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + // Recover to process the remaining splits. + harness.initializeState(state); + harness.open(); + + SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); + + for (int i = 5; i < 10; i++) { + expected.addAll(expectedRecords.get(i)); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); + + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + + // Let's process the final 5 splits now. + for (int i = 10; i < 15; i++) { + expected.addAll(expectedRecords.get(i)); + harness.processElement(splits.get(i), 1); + + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + } + } + + private List readOutputValues( + OneInputStreamOperatorTestHarness harness) { + List results = Lists.newArrayList(); + for (RowData rowData : harness.extractOutputValues()) { + results.add(Row.of(rowData.getInt(0), rowData.getString(1).toString())); + } + return results; + } + + private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { + List> expectedRecords = Lists.newArrayList(); + for (int i = 0; i < commitTimes; i++) { + List records = RandomGenericData.generate(SCHEMA, 100, 0L); + expectedRecords.add(records); + + // Commit those records to iceberg table. + writeRecords(records); + } + return expectedRecords; + } + + private void writeRecords(List records) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); + appender.appendToTable(records); + } + + private List generateSplits() { + List inputSplits = Lists.newArrayList(); + + List snapshotIds = SnapshotUtil.currentAncestorIds(table); + for (int i = snapshotIds.size() - 1; i >= 0; i--) { + ScanContext scanContext; + if (i == snapshotIds.size() - 1) { + // Generate the splits from the first snapshot. + scanContext = ScanContext.builder().useSnapshotId(snapshotIds.get(i)).build(); + } else { + // Generate the splits between the previous snapshot and current snapshot. + scanContext = + ScanContext.builder() + .startSnapshotId(snapshotIds.get(i + 1)) + .endSnapshotId(snapshotIds.get(i)) + .build(); + } + + Collections.addAll( + inputSplits, + FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool())); + } + + return inputSplits; + } + + private OneInputStreamOperatorTestHarness createReader() + throws Exception { + // This input format is used to opening the emitted split. + FlinkInputFormat inputFormat = + FlinkSource.forRowData() + .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) + .buildFormat(); + + OneInputStreamOperatorFactory factory = + StreamingReaderOperator.factory(inputFormat); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); + harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + return harness; + } + + private SteppingMailboxProcessor createLocalMailbox( + OneInputStreamOperatorTestHarness harness) { + return new SteppingMailboxProcessor( + MailboxDefaultAction.Controller::suspendDefaultAction, + harness.getTaskMailbox(), + StreamTaskActionExecutor.IMMEDIATE); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java new file mode 100644 index 000000000000..090b304942c6 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.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.source.assigner; + +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public abstract class SplitAssignerTestBase { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Test + public void testEmptyInitialization() { + SplitAssigner assigner = splitAssigner(); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test a sequence of interactions for StaticEnumerator */ + @Test + public void testStaticEnumeratorSequence() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); + assertSnapshot(assigner, 2); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + /** Test a sequence of interactions for ContinuousEnumerator */ + @Test + public void testContinuousEnumeratorSequence() throws Exception { + SplitAssigner assigner = splitAssigner(); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + + List splits1 = createSplits(1, 1, "1"); + assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); + List splits2 = createSplits(1, 1, "1"); + assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); + + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); + assertSnapshot(assigner, 2); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + private void assertAvailableFuture( + SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { + // register callback + AtomicBoolean futureCompleted = new AtomicBoolean(); + CompletableFuture future = assigner.isAvailable(); + future.thenAccept(ignored -> futureCompleted.set(true)); + // calling isAvailable again should return the same object reference + // note that thenAccept will return a new future. + // we want to assert the same instance on the assigner returned future + Assert.assertSame(future, assigner.isAvailable()); + + // now add some splits + addSplitsRunnable.run(); + Assert.assertEquals(true, futureCompleted.get()); + + for (int i = 0; i < splitCount; ++i) { + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + } + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(expectedStatus, result.status()); + switch (expectedStatus) { + case AVAILABLE: + Assert.assertNotNull(result.split()); + break; + case CONSTRAINED: + case UNAVAILABLE: + Assert.assertNull(result.split()); + break; + default: + Assert.fail("Unknown status: " + expectedStatus); + } + } + + protected void assertSnapshot(SplitAssigner assigner, int splitCount) { + Collection stateBeforeGet = assigner.state(); + Assert.assertEquals(splitCount, stateBeforeGet.size()); + } + + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + } + + protected abstract SplitAssigner splitAssigner(); +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java new file mode 100644 index 000000000000..8994f3054abe --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -0,0 +1,43 @@ +/* + * 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.assigner; + +import org.apache.iceberg.flink.source.SplitHelpers; +import org.junit.Test; + +public class TestDefaultSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new DefaultSplitAssigner(null); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInASplit() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } +} 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 new file mode 100644 index 000000000000..e78634e6b873 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.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.source.assigner; + +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.util.SerializationUtil; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Test; + +public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + Assertions.assertThatThrownBy( + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), + "Multiple files in a split is not allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Please use 'split-open-file-cost'"); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() throws Exception { + SplitAssigner assigner = splitAssigner(); + List splits = createSplits(5, 1, "2"); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, 1L); + assertGetNext(assigner, 2L); + assertGetNext(assigner, 3L); + assertGetNext(assigner, 4L); + assertGetNext(assigner, 5L); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + GetSplitResult result = assigner.getNext(null); + ContentFile file = result.split().task().files().iterator().next().file(); + Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..e1fc63fda918 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * 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.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.Assert; +import org.junit.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + Assert.assertNotNull(comparator); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + Assert.assertEquals(result.split(), split); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java new file mode 100644 index 000000000000..ebc92df02360 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java @@ -0,0 +1,97 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.List; +import java.util.NavigableMap; +import java.util.TreeMap; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class ManualContinuousSplitPlanner implements ContinuousSplitPlanner { + private final int maxPlanningSnapshotCount; + // track splits per snapshot + private final NavigableMap> splits; + private long latestSnapshotId; + private int remainingFailures; + + ManualContinuousSplitPlanner(ScanContext scanContext, int expectedFailures) { + this.maxPlanningSnapshotCount = scanContext.maxPlanningSnapshotCount(); + this.splits = new TreeMap<>(); + this.latestSnapshotId = 0L; + this.remainingFailures = expectedFailures; + } + + @Override + public synchronized ContinuousEnumerationResult planSplits( + IcebergEnumeratorPosition lastPosition) { + if (remainingFailures > 0) { + remainingFailures--; + throw new RuntimeException("Expected failure at planning"); + } + + long fromSnapshotIdExclusive = 0; + if (lastPosition != null && lastPosition.snapshotId() != null) { + fromSnapshotIdExclusive = lastPosition.snapshotId(); + } + + Preconditions.checkArgument( + fromSnapshotIdExclusive <= latestSnapshotId, + "last enumerated snapshotId is greater than the latestSnapshotId"); + if (fromSnapshotIdExclusive == latestSnapshotId) { + // already discovered everything. + return new ContinuousEnumerationResult(Lists.newArrayList(), lastPosition, lastPosition); + } + + // find the subset of snapshots to return discovered splits + long toSnapshotIdInclusive; + if (latestSnapshotId - fromSnapshotIdExclusive > maxPlanningSnapshotCount) { + toSnapshotIdInclusive = fromSnapshotIdExclusive + maxPlanningSnapshotCount; + } else { + toSnapshotIdInclusive = latestSnapshotId; + } + + List discoveredSplits = Lists.newArrayList(); + NavigableMap> discoveredView = + splits.subMap(fromSnapshotIdExclusive, false, toSnapshotIdInclusive, true); + discoveredView.forEach((snapshotId, snapshotSplits) -> discoveredSplits.addAll(snapshotSplits)); + ContinuousEnumerationResult result = + new ContinuousEnumerationResult( + discoveredSplits, + lastPosition, + // use the snapshot Id as snapshot timestamp. + IcebergEnumeratorPosition.of(toSnapshotIdInclusive, toSnapshotIdInclusive)); + return result; + } + + /** + * Add a collection of new splits. A monotonically increased snapshotId is assigned to each batch + * of splits added by this method. + */ + public synchronized void addSplits(List newSplits) { + latestSnapshotId += 1; + splits.put(latestSnapshotId, newSplits); + } + + @Override + public void close() throws IOException {} +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java new file mode 100644 index 000000000000..349eb11cf549 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -0,0 +1,352 @@ +/* + * 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.enumerator; + +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestContinuousIcebergEnumerator { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Test + public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + Collection pendingSplitsEmpty = + enumerator.snapshotState(1).pendingSplits(); + Assert.assertEquals(0, pendingSplitsEmpty.size()); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); + Assert.assertEquals(1, pendingSplits.size()); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); + Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + } + + @Test + public void testDiscoverWhenReaderRegistered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register one reader, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); + Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .contains(splits.get(0)); + } + + @Test + public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register one reader, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // remove the reader (like in a failure) + enumeratorContext.registeredReaders().remove(2); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + Assert.assertEquals(1, splits.size()); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); + List pendingSplitIds = + enumerator.snapshotState(1).pendingSplits().stream() + .map(IcebergSourceSplitState::split) + .map(IcebergSourceSplit::splitId) + .collect(Collectors.toList()); + Assert.assertEquals(splits.size(), pendingSplitIds.size()); + Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); + + // register the reader again, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); + Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .contains(splits.get(0)); + } + + @Test + public void testThrottlingDiscovery() throws Exception { + // create 10 splits + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); + + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + // discover one snapshot at a time + .maxPlanningSnapshotCount(1) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register reader-2, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // add splits[0] to the planner for next discovery + splitPlanner.addSplits(Arrays.asList(splits.get(0))); + enumeratorContext.triggerAllActions(); + + // because discovered split was assigned to reader, pending splits should be empty + Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); + // split assignment to reader-2 should contain splits[0, 1) + Assert.assertEquals( + splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // add the remaining 9 splits (one for every snapshot) + // run discovery cycles while reader-2 still processing the splits[0] + for (int i = 1; i < 10; ++i) { + splitPlanner.addSplits(Arrays.asList(splits.get(i))); + enumeratorContext.triggerAllActions(); + } + + // can only discover up to 3 snapshots/splits + Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); + // split assignment to reader-2 should be splits[0, 1) + Assert.assertEquals( + splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // now reader-2 finished splits[0] + enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); + enumeratorContext.triggerAllActions(); + // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was + // discovered and added. + Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); + // split assignment to reader-2 should be splits[0, 2) + Assert.assertEquals( + splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // run 3 more split discovery cycles + for (int i = 0; i < 3; ++i) { + enumeratorContext.triggerAllActions(); + } + + // no more splits are discovered due to throttling + Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); + // split assignment to reader-2 should still be splits[0, 2) + Assert.assertEquals( + splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + + // now reader-2 finished splits[1] + enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); + enumeratorContext.triggerAllActions(); + // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was + // discovered and added. + Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); + // split assignment to reader-2 should be splits[0, 3) + Assert.assertEquals( + splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + } + + @Test + public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(2) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 1); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + + // Trigger a planning and check that no splits returned due to the planning error + enumeratorContext.triggerAllActions(); + Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); + + // Second scan planning should succeed and discover the expected splits + enumeratorContext.triggerAllActions(); + Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); + Assert.assertEquals(1, pendingSplits.size()); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); + Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + } + + @Test + public void testOverMaxAllowedPlanningErrors() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(1) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 2); + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + + // Check that the scheduler response ignores the current error and continues to run until the + // failure limit is reached + enumeratorContext.triggerAllActions(); + Assert.assertFalse( + enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + + // Check that the task has failed with the expected exception after the failure limit is reached + enumeratorContext.triggerAllActions(); + Assert.assertTrue( + enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + Assertions.assertThatThrownBy( + () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) + .hasCauseInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to discover new split"); + } + + @Test + public void testPlanningIgnoringErrors() throws Exception { + int expectedFailures = 3; + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(-1) + .build(); + ManualContinuousSplitPlanner splitPlanner = + new ManualContinuousSplitPlanner(scanContext, expectedFailures); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + splitPlanner.addSplits(splits); + + Collection pendingSplits; + // Can not discover the new split with planning failures + for (int i = 0; i < expectedFailures; ++i) { + enumeratorContext.triggerAllActions(); + pendingSplits = enumerator.snapshotState(i).pendingSplits(); + Assert.assertEquals(0, pendingSplits.size()); + } + + // Discovered the new split after a successful scan planning + enumeratorContext.triggerAllActions(); + pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); + Assert.assertEquals(1, pendingSplits.size()); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); + Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + } + + private static ContinuousIcebergEnumerator createEnumerator( + SplitEnumeratorContext context, + ScanContext scanContext, + ContinuousSplitPlanner splitPlanner) { + + ContinuousIcebergEnumerator enumerator = + new ContinuousIcebergEnumerator( + context, + new DefaultSplitAssigner(null, Collections.emptyList()), + scanContext, + splitPlanner, + null); + enumerator.start(); + return enumerator; + } +} 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 new file mode 100644 index 000000000000..1bb2b1c6bf85 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -0,0 +1,692 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestName; + +public class TestContinuousSplitPlannerImpl { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private static final FileFormat fileFormat = FileFormat.PARQUET; + private static final AtomicLong randomSeed = new AtomicLong(); + + @Rule + public final HadoopTableResource tableResource = + new HadoopTableResource( + TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + @Rule public TestName testName = new TestName(); + + private GenericAppenderHelper dataAppender; + private DataFile dataFile1; + private Snapshot snapshot1; + private DataFile dataFile2; + private Snapshot snapshot2; + + @Before + public void before() throws IOException { + dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); + } + + private void appendTwoSnapshots() throws IOException { + // snapshot1 + List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + dataFile1 = dataAppender.writeFile(null, batch1); + dataAppender.appendToTable(dataFile1); + snapshot1 = tableResource.table().currentSnapshot(); + + // snapshot2 + List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); + dataFile2 = dataAppender.writeFile(null, batch2); + dataAppender.appendToTable(dataFile2); + snapshot2 = tableResource.table().currentSnapshot(); + } + + /** @return the last enumerated snapshot id */ + private CycleResult verifyOneCycle( + ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) + throws Exception { + List batch = + RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); + DataFile dataFile = dataAppender.writeFile(null, batch); + dataAppender.appendToTable(dataFile); + Snapshot snapshot = tableResource.table().currentSnapshot(); + + ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); + Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); + Assert.assertEquals( + lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); + Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(1, result.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); + Assert.assertEquals(1, split.task().files().size()); + Assert.assertEquals( + dataFile.path().toString(), + Iterables.getOnlyElement(split.task().files()).file().path().toString()); + return new CycleResult(result.toPosition(), split); + } + + @Test + public void testTableScanThenIncrementalWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); + Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + Assert.assertEquals( + snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + Set expectedFiles = + ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .splitSize(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); + Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + + // latest mode should discover both snapshots, as latest position is marked by when job starts + appendTwoSnapshots(); + ContinuousEnumerationResult afterTwoSnapshotsAppended = + splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); + Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1 + // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) + Assert.assertEquals( + snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertEquals( + snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); + Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); + Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); + Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1's parent, + // which leads to null snapshotId and snapshotTimestampMs. + Assert.assertNull(initialResult.toPosition().snapshotId()); + Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertNull(secondResult.fromPosition().snapshotId()); + Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover files appended in both snapshot1 and snapshot2 + Set expectedFiles = + ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromSnapshotIdWithEmptyTable() { + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: 1"); + } + + @Test + public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { + appendTwoSnapshots(); + + // find an invalid snapshotId + long invalidSnapshotId = 0L; + while (invalidSnapshotId == snapshot1.snapshotId() + || invalidSnapshotId == snapshot2.snapshotId()) { + invalidSnapshotId++; + } + + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(invalidSnapshotId) + .build(); + + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); + } + + @Test + public void testIncrementalFromSnapshotId() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot2.snapshotId()) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as + // snapshot2's parent) + Assert.assertEquals( + snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertEquals( + snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromSnapshotTimestampWithEmptyTable() { + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find a snapshot after: 1"); + } + + @Test + public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exception { + appendTwoSnapshots(); + + long invalidSnapshotTimestampMs = snapshot2.timestampMillis() + 1000L; + + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(invalidSnapshotTimestampMs) + .build(); + + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a snapshot after:"); + } + + @Test + public void testIncrementalFromSnapshotTimestamp() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis()) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). + Assert.assertEquals( + snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + Assert.assertEquals( + snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); + Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); + Assert.assertEquals( + snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + Assert.assertEquals(expectedFiles, discoveredFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testMaxPlanningSnapshotCount() throws Exception { + appendTwoSnapshots(); + // append 3 more snapshots + for (int i = 2; i < 5; ++i) { + appendSnapshot(i, 2); + } + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + // limit to 1 snapshot per discovery + .maxPlanningSnapshotCount(1) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertNull(initialResult.fromPosition()); + // For inclusive behavior, the initial result should point to snapshot1's parent, + // which leads to null snapshotId and snapshotTimestampMs. + Assert.assertNull(initialResult.toPosition().snapshotId()); + Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); + Assert.assertEquals(0, initialResult.splits().size()); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + // should discover dataFile1 appended in snapshot1 + verifyMaxPlanningSnapshotCountResult( + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + + ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); + // should discover dataFile2 appended in snapshot2 + verifyMaxPlanningSnapshotCountResult( + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + } + + @Test + public void testTableScanNoStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(false) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 0); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 0); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanAllStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 3); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 3); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanSingleStat() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(ImmutableSet.of("data")) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + Assert.assertEquals(1, initialResult.splits().size()); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + Assert.assertEquals(2, split.task().files().size()); + verifyStatCount(split, 1); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 1); + lastPosition = result.lastPosition; + } + } + + private void verifyStatCount(IcebergSourceSplit split, int expected) { + if (expected == 0) { + split + .task() + .files() + .forEach( + f -> { + Assert.assertNull(f.file().valueCounts()); + Assert.assertNull(f.file().columnSizes()); + Assert.assertNull(f.file().lowerBounds()); + Assert.assertNull(f.file().upperBounds()); + Assert.assertNull(f.file().nanValueCounts()); + Assert.assertNull(f.file().nullValueCounts()); + }); + } else { + split + .task() + .files() + .forEach( + f -> { + Assert.assertEquals(expected, f.file().valueCounts().size()); + Assert.assertEquals(expected, f.file().columnSizes().size()); + Assert.assertEquals(expected, f.file().lowerBounds().size()); + Assert.assertEquals(expected, f.file().upperBounds().size()); + Assert.assertEquals(expected, f.file().nullValueCounts().size()); + // The nanValue is not counted for long and string fields + Assert.assertEquals(0, f.file().nanValueCounts().size()); + }); + } + } + + private void verifyMaxPlanningSnapshotCountResult( + ContinuousEnumerationResult result, + Snapshot fromSnapshotExclusive, + Snapshot toSnapshotInclusive, + Set expectedFiles) { + if (fromSnapshotExclusive == null) { + Assert.assertNull(result.fromPosition().snapshotId()); + Assert.assertNull(result.fromPosition().snapshotTimestampMs()); + } else { + Assert.assertEquals( + fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); + Assert.assertEquals( + fromSnapshotExclusive.timestampMillis(), + result.fromPosition().snapshotTimestampMs().longValue()); + } + Assert.assertEquals( + toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); + Assert.assertEquals( + toSnapshotInclusive.timestampMillis(), + result.toPosition().snapshotTimestampMs().longValue()); + // should only have one split with one data file, because split discover is limited to + // one snapshot and each snapshot has only one data file appended. + IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); + Assert.assertEquals(1, split.task().files().size()); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + Assert.assertEquals(expectedFiles, discoveredFiles); + } + + private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { + List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); + DataFile dataFile = dataAppender.writeFile(null, batch); + dataAppender.appendToTable(dataFile); + return tableResource.table().currentSnapshot(); + } + + private static class CycleResult { + IcebergEnumeratorPosition lastPosition; + IcebergSourceSplit split; + + CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { + this.lastPosition = lastPosition; + this.split = split; + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java new file mode 100644 index 000000000000..2c94f215908a --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -0,0 +1,205 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.RuleChain; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.TestRule; + +public class TestContinuousSplitPlannerImplStartStrategy { + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + + public final TemporaryFolder temporaryFolder = new TemporaryFolder(); + public final HadoopTableResource tableResource = + new HadoopTableResource( + temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); + + private GenericAppenderHelper dataAppender; + private Snapshot snapshot1; + private Snapshot snapshot2; + private Snapshot snapshot3; + + @Before + public void before() throws IOException { + dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); + } + + private void appendThreeSnapshots() throws IOException { + List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + dataAppender.appendToTable(batch1); + snapshot1 = tableResource.table().currentSnapshot(); + + List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); + dataAppender.appendToTable(batch2); + snapshot2 = tableResource.table().currentSnapshot(); + + List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); + dataAppender.appendToTable(batch3); + snapshot3 = tableResource.table().currentSnapshot(); + } + + @Test + public void testTableScanThenIncrementalStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + // empty table + Assert.assertFalse( + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForLatestSnapshotStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + + // empty table + Assert.assertFalse( + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForEarliestSnapshotStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + + // empty table + Assert.assertFalse( + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForSpecificSnapshotIdStrategy() throws IOException { + ScanContext scanContextInvalidSnapshotId = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .build(); + + // empty table + Assertions.assertThatThrownBy( + () -> + ContinuousSplitPlannerImpl.startSnapshot( + tableResource.table(), scanContextInvalidSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: 1"); + + appendThreeSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot2.snapshotId()) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOException { + ScanContext scanContextInvalidSnapshotTimestamp = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(1L) + .build(); + + // empty table + Assertions.assertThatThrownBy( + () -> + ContinuousSplitPlannerImpl.startSnapshot( + tableResource.table(), scanContextInvalidSnapshotTimestamp)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a snapshot after: "); + + appendThreeSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis()) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); + Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + } + + @Test + public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOException { + appendThreeSnapshots(); + + ScanContext config = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis() - 1L) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); + Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java new file mode 100644 index 000000000000..e2be0b4b0398 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java @@ -0,0 +1,134 @@ +/* + * 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.enumerator; + +import org.junit.Assert; +import org.junit.Test; + +public class TestEnumerationHistory { + private static final int MAX_HISTORY_SIZE = 3; + private static final int FEW_PENDING_SPLITS = 2; + private static final int TOO_MANY_PENDING_SPLITS = 100; + + @Test + public void testEmptyHistory() { + EnumerationHistory history = new EnumerationHistory(MAX_HISTORY_SIZE); + int[] expectedHistorySnapshot = new int[0]; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testNotFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + int[] expectedHistorySnapshot = {1, 2}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testExactFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + int[] expectedHistorySnapshot = {1, 2, 3}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testOneMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + int[] expectedHistorySnapshot = {2, 3, 4}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testTwoMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + history.add(5); + int[] expectedHistorySnapshot = {3, 4, 5}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testThreeMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + history.add(5); + history.add(6); + int[] expectedHistorySnapshot = {4, 5, 6}; + testHistory(history, expectedHistorySnapshot); + } + + private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { + Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + if (history.hasFullHistory()) { + // throttle because pending split count is more than the sum of enumeration history + Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + } else { + // skipped throttling check because there is not enough history + Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + } + + int[] historySnapshot = history.snapshot(); + Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); + + EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); + restoredHistory.restore(historySnapshot); + + Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + if (history.hasFullHistory()) { + // throttle because pending split count is more than the sum of enumeration history + Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + } else { + // skipped throttling check because there is not enough history + Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); + } + } + + @Test + public void testRestoreDifferentSize() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + int[] historySnapshot = history.snapshot(); + + EnumerationHistory smallerHistory = new EnumerationHistory(2); + smallerHistory.restore(historySnapshot); + int[] expectedRestoredHistorySnapshot = {2, 3}; + Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); + + EnumerationHistory largerHisotry = new EnumerationHistory(4); + largerHisotry.restore(historySnapshot); + Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java new file mode 100644 index 000000000000..1d12d9f66a8a --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java @@ -0,0 +1,145 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestIcebergEnumeratorStateSerializer { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final IcebergEnumeratorStateSerializer serializer = + new IcebergEnumeratorStateSerializer(true); + + protected final int version; + + @Parameterized.Parameters(name = "version={0}") + public static Object[] parameters() { + return new Object[] {1, 2}; + } + + public TestIcebergEnumeratorStateSerializer(int version) { + this.version = version; + } + + @Test + public void testEmptySnapshotIdAndPendingSplits() throws Exception { + IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); + testSerializer(enumeratorState); + } + + @Test + public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); + + IcebergEnumeratorState enumeratorState = + new IcebergEnumeratorState(position, Collections.emptyList()); + testSerializer(enumeratorState); + } + + @Test + public void testSomeSnapshotIdAndPendingSplits() throws Exception { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + Collection pendingSplits = Lists.newArrayList(); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); + + IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(position, pendingSplits); + testSerializer(enumeratorState); + } + + @Test + public void testEnumerationSplitCountHistory() throws Exception { + if (version == 2) { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + Collection pendingSplits = Lists.newArrayList(); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); + int[] enumerationSplitCountHistory = {1, 2, 3}; + + IcebergEnumeratorState enumeratorState = + new IcebergEnumeratorState(position, pendingSplits, enumerationSplitCountHistory); + testSerializer(enumeratorState); + } + } + + private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOException { + byte[] result; + if (version == 1) { + result = serializer.serializeV1(enumeratorState); + } else { + result = serializer.serialize(enumeratorState); + } + + IcebergEnumeratorState deserialized = serializer.deserialize(version, result); + assertEnumeratorStateEquals(enumeratorState, deserialized); + } + + private void assertEnumeratorStateEquals( + IcebergEnumeratorState expected, IcebergEnumeratorState actual) { + Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); + + Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); + Iterator expectedIterator = expected.pendingSplits().iterator(); + Iterator actualIterator = actual.pendingSplits().iterator(); + for (int i = 0; i < expected.pendingSplits().size(); ++i) { + IcebergSourceSplitState expectedSplitState = expectedIterator.next(); + IcebergSourceSplitState actualSplitState = actualIterator.next(); + Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); + Assert.assertEquals( + expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); + Assert.assertEquals( + expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); + Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); + } + + Assert.assertArrayEquals( + expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java new file mode 100644 index 000000000000..0d33e4ed08ad --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.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.flink.source.reader; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public abstract class ReaderFunctionTestBase { + + @Parameterized.Parameters(name = "fileFormat={0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC}, + new Object[] {FileFormat.PARQUET} + }; + } + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + protected abstract ReaderFunction readerFunction(); + + protected abstract void assertRecords(List expected, List actual, Schema schema); + + private final FileFormat fileFormat; + private final GenericAppenderFactory appenderFactory; + + public ReaderFunctionTestBase(FileFormat fileFormat) { + this.fileFormat = fileFormat; + this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + } + + private void assertRecordsAndPosition( + List expectedRecords, + int expectedFileOffset, + long startRecordOffset, + RecordsWithSplitIds> batch) { + batch.nextSplit(); + List actualRecords = Lists.newArrayList(); + long recordOffset = startRecordOffset; + RecordAndPosition recordAndPosition; + while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { + actualRecords.add(recordAndPosition.record()); + Assert.assertEquals( + "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); + Assert.assertEquals( + "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); + recordOffset++; + } + + Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); + assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); + } + + @Test + public void testNoCheckpointedPosition() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionBeforeFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionMiddleFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0).subList(1, 2), 0, 1L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionAfterFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionBeforeSecondFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @Test + public void testCheckpointedPositionMidSecondFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1).subList(1, 2), 1, 1L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java new file mode 100644 index 000000000000..2a2503ef2478 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.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.source.reader; + +import java.io.File; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.rules.TemporaryFolder; + +public class ReaderUtil { + + private ReaderUtil() {} + + public static FileScanTask createFileTask( + List records, + File file, + FileFormat fileFormat, + FileAppenderFactory appenderFactory) + throws IOException { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { + appender.addAll(records); + } finally { + appender.close(); + } + + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(records.size()) + .withFileSizeInBytes(file.length()) + .withPath(file.toString()) + .withFormat(fileFormat) + .withMetrics(appender.metrics()) + .build(); + + ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); + return new BaseFileScanTask( + dataFile, + null, + SchemaParser.toJson(TestFixtures.SCHEMA), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + residuals); + } + + public static DataIterator createDataIterator(CombinedScanTask combinedTask) { + return new DataIterator<>( + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), + combinedTask, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new PlaintextEncryptionManager()); + } + + public static List> createRecordBatchList( + Schema schema, int listSize, int batchCount) { + return createRecordBatchList(0L, schema, listSize, batchCount); + } + + public static List> createRecordBatchList( + long seed, Schema schema, int listSize, int batchCount) { + List records = RandomGenericData.generate(schema, listSize * batchCount, seed); + return Lists.partition(records, batchCount); + } + + public static CombinedScanTask createCombinedScanTask( + List> recordBatchList, + TemporaryFolder temporaryFolder, + FileFormat fileFormat, + GenericAppenderFactory appenderFactory) + throws IOException { + List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); + for (List recordBatch : recordBatchList) { + FileScanTask fileTask = + ReaderUtil.createFileTask( + recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); + fileTasks.add(fileTask); + } + + return new BaseCombinedScanTask(fileTasks); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java new file mode 100644 index 000000000000..644ac2bad6b8 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java @@ -0,0 +1,68 @@ +/* + * 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 java.util.concurrent.atomic.AtomicBoolean; +import org.junit.Assert; +import org.junit.Test; + +public class TestArrayBatchRecords { + + @Test + public void testFullRange() { + String[] elements = new String[] {"0", "1", "2", "3"}; + testArray(elements, elements.length, 2, 119); + } + + @Test + public void testSubRange() { + String[] elements = new String[] {"0", "1", "2", "3"}; + testArray(elements, 2, 0, 0); + } + + private void testArray( + String[] elements, int numberOfRecords, int fileOffset, long startingRecordOffset) { + String splitId = "iceberg_split_1"; + AtomicBoolean recycled = new AtomicBoolean(); + + ArrayBatchRecords recordsWithSplitIds = + ArrayBatchRecords.forRecords( + splitId, + ignored -> recycled.set(true), + elements, + numberOfRecords, + fileOffset, + startingRecordOffset); + + Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); + + for (int i = 0; i < numberOfRecords; i++) { + RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); + Assert.assertEquals(elements[i], recAndPos.record()); + Assert.assertEquals(fileOffset, recAndPos.fileOffset()); + // recordOffset points to the position after this one + Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); + } + + Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); + Assert.assertNull(recordsWithSplitIds.nextSplit()); + recordsWithSplitIds.recycle(); + Assert.assertTrue(recycled.get()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java new file mode 100644 index 000000000000..f964a7707689 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -0,0 +1,358 @@ +/* + * 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 java.util.Arrays; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderOptions; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestArrayPoolDataIteratorBatcherRowData { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + private static final FileFormat fileFormat = FileFormat.PARQUET; + + private final GenericAppenderFactory appenderFactory; + private final DataIteratorBatcher batcher; + + public TestArrayPoolDataIteratorBatcherRowData() { + Configuration config = new Configuration(); + // set array pool size to 1 + config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); + // set batch array size to 2 + config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); + this.batcher = + new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); + this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + } + + /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ + @Test + public void testSingleFileLessThanOneFullBatch() throws Exception { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); + FileScanTask fileTask = + ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); + Assert.assertTrue(batch.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch.numberOfRecords()); + + RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); + + /////////////////////////////// + // assert first record + + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(1, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); + + Assert.assertNull(batch.nextRecordFromSplit()); + Assert.assertNull(batch.nextSplit()); + batch.recycle(); + + // assert end of input + Assert.assertFalse(recordBatchIterator.hasNext()); + } + + /** + * Read a CombinedScanTask that contains a single file with multiple batches. + * + *

    Insert 5 records in a single file that should result in 3 batches + */ + @Test + public void testSingleFileWithMultipleBatches() throws Exception { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); + FileScanTask fileTask = + ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + /////////////////////////////// + // assert first batch with full batch of 2 records + + ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); + Assert.assertTrue(batch0.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch0.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch0.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch0.numberOfRecords()); + + RecordAndPosition recordAndPosition; + + // assert first record + recordAndPosition = batch0.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(1, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); + + // assert second record + recordAndPosition = batch0.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(2, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); + + Assert.assertNull(batch0.nextRecordFromSplit()); + Assert.assertNull(batch0.nextSplit()); + batch0.recycle(); + + /////////////////////////////// + // assert second batch with full batch of 2 records + + ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch0.records(), batch1.records()); + Assert.assertTrue(batch1.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch1.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch1.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch1.numberOfRecords()); + + // assert third record + recordAndPosition = batch1.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(3, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); + + // assert fourth record + recordAndPosition = batch1.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(4, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); + + Assert.assertNull(batch1.nextRecordFromSplit()); + Assert.assertNull(batch1.nextSplit()); + batch1.recycle(); + + /////////////////////////////// + // assert third batch with partial batch of 1 record + + ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch0.records(), batch2.records()); + Assert.assertTrue(batch2.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch2.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch2.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch2.numberOfRecords()); + + // assert fifth record + recordAndPosition = batch2.nextRecordFromSplit(); + Assert.assertEquals(0, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(5, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); + + Assert.assertNull(batch2.nextRecordFromSplit()); + Assert.assertNull(batch2.nextSplit()); + batch2.recycle(); + + // assert end of input + Assert.assertFalse(recordBatchIterator.hasNext()); + } + + /** + * Read a CombinedScanTask that contains with multiple files. + * + *

    In this test, we also seek the iterator to starting position (1, 1). + */ + @Test + public void testMultipleFilesWithSeekPosition() throws Exception { + List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); + FileScanTask fileTask0 = + ReaderUtil.createFileTask( + records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); + FileScanTask fileTask1 = + ReaderUtil.createFileTask( + records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); + FileScanTask fileTask2 = + ReaderUtil.createFileTask( + records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + CombinedScanTask combinedTask = + new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); + + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + // seek to file1 and after record 1 + dataIterator.seek(1, 1); + + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + /////////////////////////////// + // file0 is skipped by seek + + /////////////////////////////// + // file1 has 4 records. because the seek position, first record is skipped. + // we should read 3 remaining records in 2 batches: + // batch10 with 2 records and batch11 with 1 records. + + // assert first batch from file1 with full batch of 2 records + + // variable naming convention: batch + ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); + Assert.assertTrue(batch10.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch10.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch10.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch10.numberOfRecords()); + + RecordAndPosition recordAndPosition; + + recordAndPosition = batch10.nextRecordFromSplit(); + Assert.assertEquals(1, recordAndPosition.fileOffset()); + // seek should skip the first record in file1. starting from the second record + Assert.assertEquals(2, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); + + recordAndPosition = batch10.nextRecordFromSplit(); + Assert.assertEquals(1, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(3, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); + + Assert.assertNull(batch10.nextRecordFromSplit()); + Assert.assertNull(batch10.nextSplit()); + batch10.recycle(); + + // assert second batch from file1 with partial batch of 1 record + + // variable naming convention: batch__ + ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch10.records(), batch11.records()); + Assert.assertTrue(batch11.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch11.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch11.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch11.numberOfRecords()); + + recordAndPosition = batch11.nextRecordFromSplit(); + Assert.assertEquals(1, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(4, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); + + Assert.assertNull(batch11.nextRecordFromSplit()); + Assert.assertNull(batch11.nextSplit()); + batch11.recycle(); + + /////////////////////////////// + // file2 has 3 records. + // we should read 3 records in 2 batches: + // batch20 with 2 records and batch21 with 1 records + + // assert first batch from file2 with full batch of 2 records + + // variable naming convention: batch__ + ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch10.records(), batch20.records()); + Assert.assertTrue(batch20.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch20.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch20.records().length); + // assert actual number of records in the array + Assert.assertEquals(2, batch20.numberOfRecords()); + + recordAndPosition = batch20.nextRecordFromSplit(); + Assert.assertEquals(2, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(1, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); + + recordAndPosition = batch20.nextRecordFromSplit(); + Assert.assertEquals(2, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(2, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); + + Assert.assertNull(batch20.nextRecordFromSplit()); + Assert.assertNull(batch20.nextSplit()); + batch20.recycle(); + + /////////////////////////////// + // assert second batch from file2 with partial batch of 1 record + + // variable naming convention: batch__ + ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); + // assert array is reused + Assert.assertSame(batch10.records(), batch21.records()); + Assert.assertTrue(batch21.finishedSplits().isEmpty()); + Assert.assertEquals(splitId, batch21.nextSplit()); + // reusable array size should be the configured value of 2 + Assert.assertEquals(2, batch21.records().length); + // assert actual number of records in the array + Assert.assertEquals(1, batch21.numberOfRecords()); + + recordAndPosition = batch21.nextRecordFromSplit(); + Assert.assertEquals(2, recordAndPosition.fileOffset()); + // The position points to where the reader should resume after this record is processed. + Assert.assertEquals(3, recordAndPosition.recordOffset()); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); + + Assert.assertNull(batch21.nextRecordFromSplit()); + Assert.assertNull(batch21.nextSplit()); + batch21.recycle(); + + // assert end of input + Assert.assertFalse(recordBatchIterator.hasNext()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..afe8a5d0152c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,178 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + @Rule + public final HadoopTableResource sourceTableResource = + new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + + private final String columnName; + + @BeforeClass + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); + + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); + + minValues.merge("long_column", (Long) r.get(2), Math::min); + } + } + } + + @Parameterized.Parameters(name = "{0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } + + public TestColumnStatsWatermarkExtractor(String columnName) { + this.columnName = columnName; + } + + @Test + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + } + + @Test + public void testTimeUnit() throws IOException { + Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue() / 1000L, + extractor.extractWatermark(split(0))); + } + + @Test + public void testMultipleFiles() throws IOException { + Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); + + Assert.assertEquals( + MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + Assert.assertEquals( + MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); + Assert.assertEquals( + Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), + extractor.extractWatermark(combinedSplit)); + } + + @Test + public void testWrongColumn() { + Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + @Test + public void testEmptyStatistics() throws IOException { + Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); + Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics for column"); + } + + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + TEMPORARY_FOLDER, + FileFormat.PARQUET, + APPENDER_FACTORY)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java new file mode 100644 index 000000000000..f19d57083b89 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -0,0 +1,184 @@ +/* + * 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 java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceReader { + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final GenericAppenderFactory appenderFactory; + + public TestIcebergSourceReader() { + this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + } + + @Test + public void testReaderMetrics() throws Exception { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + IcebergSourceReader reader = createReader(metricGroup, readerContext, null); + reader.start(); + + testOneSplitFetcher(reader, readerOutput, metricGroup, 1); + testOneSplitFetcher(reader, readerOutput, metricGroup, 2); + } + + @Test + public void testReaderOrder() throws Exception { + // Create 2 splits + List> recordBatchList1 = + ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task1 = + ReaderUtil.createCombinedScanTask( + recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + List> recordBatchList2 = + ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task2 = + ReaderUtil.createCombinedScanTask( + recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + + // Sort the splits in one way + List rowDataList1 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task1), + IcebergSourceSplit.fromCombinedScanTask(task2)), + 2); + + // Reverse the splits + List rowDataList2 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task2), + IcebergSourceSplit.fromCombinedScanTask(task1)), + 2); + + // Check that the order of the elements is not changed + Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); + Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + } + + private List read(List splits, long expected) throws Exception { + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + // Using IdBasedComparator, so we can have a deterministic order of the splits + IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); + reader.start(); + + reader.addSplits(splits); + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + while (readerOutput.getEmittedRecords().size() < expected) { + reader.pollNext(readerOutput); + } + + reader.pollNext(readerOutput); + + Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + return readerOutput.getEmittedRecords(); + } + + private void testOneSplitFetcher( + IcebergSourceReader reader, + TestingReaderOutput readerOutput, + TestingMetricGroup metricGroup, + int expectedCount) + throws Exception { + long seed = expectedCount; + // Each split should contain only one file with one record + List> recordBatchList = + ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task = + ReaderUtil.createCombinedScanTask( + recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); + reader.addSplits(Collections.singletonList(split)); + + while (readerOutput.getEmittedRecords().size() < expectedCount) { + reader.pollNext(readerOutput); + } + + Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); + TestHelpers.assertRowData( + TestFixtures.SCHEMA, + recordBatchList.get(0).get(0), + readerOutput.getEmittedRecords().get(expectedCount - 1)); + Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); + + // One more poll will get null record batch. + // That will finish the split and cause split fetcher to be closed due to idleness. + // Then next split will create a new split reader. + reader.pollNext(readerOutput); + } + + private IcebergSourceReader createReader( + MetricGroup metricGroup, + SourceReaderContext readerContext, + SerializableComparator splitComparator) { + IcebergSourceReaderMetrics readerMetrics = + new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); + RowDataReaderFunction readerFunction = + new RowDataReaderFunction( + new Configuration(), + TestFixtures.SCHEMA, + TestFixtures.SCHEMA, + null, + true, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new PlaintextEncryptionManager(), + Collections.emptyList()); + return new IcebergSourceReader<>( + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); + } + + private static class IdBasedComparator implements SerializableComparator { + @Override + public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { + return o1.splitId().compareTo(o2.splitId()); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java new file mode 100644 index 000000000000..d063ad7f4a80 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -0,0 +1,74 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; +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.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.hadoop.HadoopFileIO; + +public class TestRowDataReaderFunction extends ReaderFunctionTestBase { + + protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final DataStructureConverter rowDataConverter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + + public TestRowDataReaderFunction(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected ReaderFunction readerFunction() { + return new RowDataReaderFunction( + new Configuration(), + TestFixtures.SCHEMA, + TestFixtures.SCHEMA, + null, + true, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + new PlaintextEncryptionManager(), + Collections.emptyList()); + } + + @Override + protected void assertRecords(List expected, List actual, Schema schema) { + List rows = toRows(actual); + TestHelpers.assertRecords(rows, expected, TestFixtures.SCHEMA); + } + + private List toRows(List actual) { + return actual.stream() + .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java new file mode 100644 index 000000000000..290628c5fc90 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java @@ -0,0 +1,102 @@ +/* + * 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 java.util.Map; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class TestingMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { + private final Map counters; + + TestingMetricGroup() { + this.counters = Maps.newHashMap(); + } + + /** Pass along the reference to share the map for child metric groups. */ + private TestingMetricGroup(Map counters) { + this.counters = counters; + } + + Map counters() { + return counters; + } + + @Override + public Counter counter(String name) { + Counter counter = new SimpleCounter(); + counters.put(name, counter); + return counter; + } + + @Override + public MetricGroup addGroup(String name) { + return new TestingMetricGroup(counters); + } + + @Override + public MetricGroup addGroup(String key, String value) { + return new TestingMetricGroup(counters); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return new TestingOperatorIOMetricGroup(); + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + return new SimpleCounter(); + } + + @Override + public void setPendingBytesGauge(Gauge pendingBytesGauge) {} + + @Override + public void setPendingRecordsGauge(Gauge pendingRecordsGauge) {} + + private static class TestingOperatorIOMetricGroup extends UnregisteredMetricsGroup + implements OperatorIOMetricGroup { + @Override + public Counter getNumRecordsInCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumRecordsOutCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumBytesInCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumBytesOutCounter() { + return new SimpleCounter(); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java new file mode 100644 index 000000000000..cd778309f90d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.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.flink.source.split; + +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.Assert; +import org.junit.ClassRule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestIcebergSourceSplitSerializer { + + @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + + private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); + + @Test + public void testLatestVersion() throws Exception { + serializeAndDeserialize(1, 1); + serializeAndDeserialize(10, 2); + } + + private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = serializer.serialize(split); + IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); + assertSplitEquals(split, deserialized); + + byte[] cachedResult = serializer.serialize(split); + Assert.assertSame(result, cachedResult); + IcebergSourceSplit deserialized2 = + serializer.deserialize(serializer.getVersion(), cachedResult); + assertSplitEquals(split, deserialized2); + + split.updatePosition(0, 100); + byte[] resultAfterUpdatePosition = serializer.serialize(split); + // after position change, serialized bytes should have changed + Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); + IcebergSourceSplit deserialized3 = + serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); + assertSplitEquals(split, deserialized3); + } + } + + @Test + public void testV1() throws Exception { + serializeAndDeserializeV1(1, 1); + serializeAndDeserializeV1(10, 2); + } + + private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testV2() throws Exception { + serializeAndDeserializeV2(1, 1); + serializeAndDeserializeV2(10, 2); + } + + private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV2(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testDeserializeV1() throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = serializer.deserialize(1, result); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testCheckpointedPosition() throws Exception { + final AtomicInteger index = new AtomicInteger(); + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() + .map( + split -> { + IcebergSourceSplit result; + if (index.get() % 2 == 0) { + result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); + } else { + result = split; + } + index.incrementAndGet(); + return result; + }) + .collect(Collectors.toList()); + + for (IcebergSourceSplit split : splits) { + byte[] result = serializer.serialize(split); + IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); + assertSplitEquals(split, deserialized); + + byte[] cachedResult = serializer.serialize(split); + Assert.assertSame(result, cachedResult); + IcebergSourceSplit deserialized2 = + serializer.deserialize(serializer.getVersion(), cachedResult); + assertSplitEquals(split, deserialized2); + } + } + + private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { + List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); + List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); + Assert.assertEquals(expectedTasks.size(), actualTasks.size()); + for (int i = 0; i < expectedTasks.size(); ++i) { + FileScanTask expectedTask = expectedTasks.get(i); + FileScanTask actualTask = actualTasks.get(i); + Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); + Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); + Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); + Assert.assertEquals(expectedTask.start(), actualTask.start()); + Assert.assertEquals(expectedTask.length(), actualTask.length()); + } + + Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); + Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java new file mode 100644 index 000000000000..08bbc4fc802b --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -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. + */ +package org.apache.iceberg.flink.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +public class TestFlinkPackage { + + /** This unit test would need to be adjusted as new Flink version is supported. */ + @Test + public void testVersion() { + assertThat(FlinkPackage.version()).isEqualTo("1.18.1"); + } + + @Test + public void testDefaultVersion() { + // It's difficult to reproduce a reflection error in a unit test, so we just inject a mocked + // fault to test the default logic + + // First make sure we're not caching a version result from a previous test + FlinkPackage.setVersion(null); + try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { + mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class); + mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); + } + FlinkPackage.setVersion(null); + try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { + mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null); + mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); + FlinkPackage.setVersion(null); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); + } + } +} diff --git a/flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000000..47a3c94aa991 --- /dev/null +++ b/flink/v1.18/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -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.flink.source.BoundedTableFactory From 31968f5ac30daba39115bacb7abdbec9112e78f6 Mon Sep 17 00:00:00 2001 From: Rodrigo Meneses Date: Mon, 15 Apr 2024 10:03:00 -0700 Subject: [PATCH 0228/1019] Flink: Refactoring code and properties to make Flink 1.19 to work --- .github/workflows/flink-ci.yml | 2 +- dev/stage-binaries.sh | 2 +- flink/build.gradle | 7 ++-- flink/v1.19/build.gradle | 36 +++++++++---------- .../iceberg/flink/FlinkCatalogFactory.java | 2 +- .../apache/iceberg/flink/FlinkTestBase.java | 18 ++++++++++ .../org/apache/iceberg/flink/TestBase.java | 20 ++++++++++- .../iceberg/flink/TestChangeLogTable.java | 2 +- .../flink/TestFlinkCatalogDatabase.java | 8 ++--- .../iceberg/flink/TestFlinkCatalogTable.java | 2 +- .../TestFlinkCatalogTablePartitions.java | 2 +- .../iceberg/flink/TestFlinkHiveCatalog.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 20 ++++------- .../actions/TestRewriteDataFilesAction.java | 2 +- .../shuffle/TestDataStatisticsOperator.java | 4 ++- .../flink/source/TestFlinkMetaDataTable.java | 2 +- .../flink/source/TestFlinkTableSource.java | 2 +- .../TestMetadataTableReadableMetrics.java | 2 +- .../flink/source/TestStreamScanSql.java | 2 +- .../iceberg/flink/util/TestFlinkPackage.java | 2 +- gradle.properties | 4 +-- gradle/libs.versions.toml | 24 ++++++------- settings.gradle | 18 +++++----- 25 files changed, 110 insertions(+), 79 deletions(-) diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index af1c650f3085..0791f5b73337 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -72,7 +72,7 @@ jobs: strategy: matrix: jvm: [8, 11] - flink: ['1.16', '1.17', '1.18'] + flink: ['1.17', '1.18', '1.19'] env: SPARK_LOCAL_IP: localhost steps: diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index b7cd1a37ace9..05bf3c4253f2 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -19,7 +19,7 @@ # SCALA_VERSION=2.12 -FLINK_VERSIONS=1.16,1.17,1.18 +FLINK_VERSIONS=1.17,1.18,1.19 SPARK_VERSIONS=3.3,3.4,3.5 HIVE_VERSIONS=2,3 diff --git a/flink/build.gradle b/flink/build.gradle index a33fc84e5727..f049ff69b059 100644 --- a/flink/build.gradle +++ b/flink/build.gradle @@ -19,9 +19,6 @@ def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") -if (flinkVersions.contains("1.16")) { - apply from: file("$projectDir/v1.16/build.gradle") -} if (flinkVersions.contains("1.17")) { apply from: file("$projectDir/v1.17/build.gradle") @@ -30,3 +27,7 @@ if (flinkVersions.contains("1.17")) { if (flinkVersions.contains("1.18")) { apply from: file("$projectDir/v1.18/build.gradle") } + +if (flinkVersions.contains("1.19")) { + apply from: file("$projectDir/v1.19/build.gradle") +} diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index c08ae5d8cc1f..f70c7a4d32fe 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.18' +String flinkMajorVersion = '1.19' String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { @@ -32,15 +32,15 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') - compileOnly libs.flink118.avro + compileOnly libs.flink119.avro // for dropwizard histogram metrics implementation - compileOnly libs.flink118.metrics.dropwizard - compileOnly libs.flink118.streaming.java - compileOnly "${libs.flink118.streaming.java.get().module}:${libs.flink118.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink118.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" - compileOnly libs.flink118.connector.base - compileOnly libs.flink118.connector.files + compileOnly libs.flink119.metrics.dropwizard + compileOnly libs.flink119.streaming.java + compileOnly "${libs.flink119.streaming.java.get().module}:${libs.flink119.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink119.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" + compileOnly libs.flink119.connector.base + compileOnly libs.flink119.connector.files compileOnly libs.hadoop2.hdfs compileOnly libs.hadoop2.common @@ -66,13 +66,13 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } - testImplementation libs.flink118.connector.test.utils - testImplementation libs.flink118.core - testImplementation libs.flink118.runtime - testImplementation(libs.flink118.test.utilsjunit) { + testImplementation libs.flink119.connector.test.utils + testImplementation libs.flink119.core + testImplementation libs.flink119.runtime + testImplementation(libs.flink119.test.utilsjunit) { exclude group: 'junit' } - testImplementation(libs.flink118.test.utils) { + testImplementation(libs.flink119.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -166,7 +166,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink118.metrics.dropwizard + implementation libs.flink119.metrics.dropwizard // for integration testing with the flink-runtime-jar // all of those dependencies are required because the integration test extends FlinkTestBase @@ -176,13 +176,13 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink118.test.utils) { + integrationImplementation(libs.flink119.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } - integrationImplementation libs.flink118.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink118.get()}" + integrationImplementation libs.flink119.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" integrationImplementation libs.hadoop2.common integrationImplementation libs.hadoop2.hdfs diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index 1453753849ec..fe4008a13ce5 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -70,8 +70,8 @@ public class FlinkCatalogFactory implements CatalogFactory { public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; public static final String DEFAULT_DATABASE = "default-database"; public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String DEFAULT_CATALOG_NAME = "default_catalog"; public static final String BASE_NAMESPACE = "base-namespace"; - public static final String TYPE = "type"; public static final String PROPERTY_VERSION = "property-version"; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java index 8076e0ec76f8..0b7d19f27cc0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; + import java.util.List; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; @@ -126,4 +128,20 @@ protected void dropCatalog(String catalogName, boolean ifExists) { sql("USE CATALOG default_catalog"); sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 3986f1a796a5..e0b429b31b64 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; + import java.nio.file.Path; import java.util.List; import org.apache.flink.table.api.EnvironmentSettings; @@ -124,7 +126,23 @@ protected void assertSameElements(String message, Iterable expected, Iterab * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog */ protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java index 8e9066e391c9..d9f01796fb44 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -98,7 +98,7 @@ public void before() { @Override public void clean() { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); BoundedTableFactory.clearDataSets(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index f46d50a5f0ab..38379967044a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -41,7 +41,7 @@ public class TestFlinkCatalogDatabase extends CatalogTestBase { @Override public void clean() { sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } @@ -61,7 +61,7 @@ public void testCreateNamespace() { .as("Database should still exist") .isTrue(); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) .as("Database should be dropped") .isFalse(); @@ -81,7 +81,7 @@ public void testDropEmptyDatabase() { assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) .as("Namespace should exist") .isTrue(); - sql("DROP DATABASE %s", flinkDatabase); + dropDatabase(flinkDatabase, true); assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) .as("Namespace should have been dropped") .isFalse(); @@ -105,7 +105,7 @@ public void testDropNonEmptyNamespace() { assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) .as("Table should exist") .isTrue(); - Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + Assertions.assertThatThrownBy(() -> dropDatabase(flinkDatabase, true)) .cause() .isInstanceOf(DatabaseNotEmptyException.class) .hasMessage( 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 eaa92e32c49d..f3af2c3cbe98 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 @@ -74,7 +74,7 @@ public void before() { public void cleanNamespaces() { sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index b32be379caeb..a6feb2607774 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -76,7 +76,7 @@ public void before() { @AfterEach public void cleanNamespaces() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java index 47ee2afceb02..dd643520136c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java @@ -97,7 +97,7 @@ private void checkSQLQuery(Map catalogProperties, File warehouse "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); sql("DROP TABLE test_table"); - sql("DROP DATABASE test_db"); + dropDatabase("test_db", false); dropCatalog("test_catalog", false); } } 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 b7fce104f490..3f66174049a4 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 @@ -122,7 +122,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); BoundedTableFactory.clearDataSets(); super.clean(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..baf13017ff99 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -104,7 +104,7 @@ public void before() { @Override @AfterEach public void clean() { - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index cb409b784383..632997e357a9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -257,20 +257,12 @@ public void testCreateTableUnderDefaultDatabase() { public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - Assertions.assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + Assertions.assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); } @Test 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 4220775f41fa..5d8cce07330a 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 @@ -136,7 +136,7 @@ public void clean() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } 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 0e99a2d74ccb..8ddb147e07c1 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 @@ -36,6 +36,7 @@ import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; import org.apache.flink.runtime.state.StateInitializationContext; import org.apache.flink.runtime.state.StateInitializationContextImpl; import org.apache.flink.runtime.state.TestTaskStateManager; @@ -234,7 +235,8 @@ private StateInitializationContext getStateContext() throws Exception { CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); OperatorStateStore operatorStateStore = abstractStateBackend.createOperatorStateBackend( - env, "test-operator", Collections.emptyList(), cancelStreamRegistry); + new OperatorStateBackendParametersImpl( + env, "test-operator", Collections.emptyList(), cancelStreamRegistry)); return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java index f58cc87c6a29..8352924d042a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -126,7 +126,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(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index ff14bc406276..e8ec482520ca 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -95,7 +95,7 @@ public void before() { @After public void clean() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); } 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 40dfda723749..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 @@ -168,7 +168,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(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 9e043bbbbbd2..d86927999760 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -94,7 +94,7 @@ public void before() { @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); + dropDatabase(flinkDatabase, true); super.clean(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index 08bbc4fc802b..4ba4f9d983dc 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -29,7 +29,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - assertThat(FlinkPackage.version()).isEqualTo("1.18.1"); + assertThat(FlinkPackage.version()).isEqualTo("1.19.0"); } @Test diff --git a/gradle.properties b/gradle.properties index ea857e7f275e..f593e700fcd1 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,8 +16,8 @@ jmhOutputPath=build/reports/jmh/human-readable-output.txt jmhJsonOutputPath=build/reports/jmh/results.json jmhIncludeRegex=.* -systemProp.defaultFlinkVersions=1.18 -systemProp.knownFlinkVersions=1.16,1.17,1.18 +systemProp.defaultFlinkVersions=1.19 +systemProp.knownFlinkVersions=1.17,1.18,1.19 systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 23628d26fd4f..3f2f2591e36e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -38,9 +38,9 @@ delta-spark = "3.1.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.26.1" findbugs-jsr305 = "3.0.2" -flink116 = { strictly = "1.16.3"} flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} +flink119 = { strictly = "1.19.0"} google-libraries-bom = "26.28.0" guava = "33.1.0-jre" hadoop2 = "2.7.3" @@ -105,12 +105,6 @@ calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "ca delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } -flink116-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink116" } -flink116-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink116" } -flink116-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink116" } -flink116-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink116" } -flink116-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink116" } -flink116-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink116" } flink117-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink117" } flink117-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink117" } flink117-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink117" } @@ -123,6 +117,12 @@ flink118-connector-files = { module = "org.apache.flink:flink-connector-files", flink118-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink118" } flink118-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink118" } flink118-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink118" } +flink119-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink119" } +flink119-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink119" } +flink119-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink119" } +flink119-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink119" } +flink119-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink119" } +flink119-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink119" } google-libraries-bom = { module = "com.google.cloud:libraries-bom", version.ref = "google-libraries-bom" } guava-guava = { module = "com.google.guava:guava", version.ref = "guava" } hadoop2-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop2" } @@ -175,11 +175,6 @@ assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-cor awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } -flink116-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink116" } -flink116-core = { module = "org.apache.flink:flink-core", version.ref = "flink116" } -flink116-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink116" } -flink116-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink116" } -flink116-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink116" } flink117-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink117" } flink117-core = { module = "org.apache.flink:flink-core", version.ref = "flink117" } flink117-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink117" } @@ -190,6 +185,11 @@ flink118-core = { module = "org.apache.flink:flink-core", version.ref = "flink11 flink118-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink118" } flink118-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink118" } flink118-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink118" } +flink119-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink119" } +flink119-core = { module = "org.apache.flink:flink-core", version.ref = "flink119" } +flink119-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink119" } +flink119-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink119" } +flink119-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink119" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jakarta-el-api = { module = "jakarta.el:jakarta.el-api", version.ref = "jakarta-el-api" } jetty-server = { module = "org.eclipse.jetty:jetty-server", version.ref = "jetty" } diff --git a/settings.gradle b/settings.gradle index 15bb83754e14..4f42d24c32cf 100644 --- a/settings.gradle +++ b/settings.gradle @@ -112,15 +112,6 @@ if (!flinkVersions.isEmpty()) { project(':flink').name = 'iceberg-flink' } -if (flinkVersions.contains("1.16")) { - include ":iceberg-flink:flink-1.16" - include ":iceberg-flink:flink-runtime-1.16" - project(":iceberg-flink:flink-1.16").projectDir = file('flink/v1.16/flink') - project(":iceberg-flink:flink-1.16").name = "iceberg-flink-1.16" - project(":iceberg-flink:flink-runtime-1.16").projectDir = file('flink/v1.16/flink-runtime') - project(":iceberg-flink:flink-runtime-1.16").name = "iceberg-flink-runtime-1.16" -} - if (flinkVersions.contains("1.17")) { include ":iceberg-flink:flink-1.17" include ":iceberg-flink:flink-runtime-1.17" @@ -139,6 +130,15 @@ if (flinkVersions.contains("1.18")) { project(":iceberg-flink:flink-runtime-1.18").name = "iceberg-flink-runtime-1.18" } +if (flinkVersions.contains("1.19")) { + include ":iceberg-flink:flink-1.19" + include ":iceberg-flink:flink-runtime-1.19" + project(":iceberg-flink:flink-1.19").projectDir = file('flink/v1.19/flink') + project(":iceberg-flink:flink-1.19").name = "iceberg-flink-1.19" + project(":iceberg-flink:flink-runtime-1.19").projectDir = file('flink/v1.19/flink-runtime') + project(":iceberg-flink:flink-runtime-1.19").name = "iceberg-flink-runtime-1.19" +} + if (sparkVersions.contains("3.3")) { include ":iceberg-spark:spark-3.3_${scalaVersion}" include ":iceberg-spark:spark-extensions-3.3_${scalaVersion}" From 16ead67b90d681f3ae598ad99a307b4d1f464b8b Mon Sep 17 00:00:00 2001 From: Rodrigo Date: Tue, 16 Apr 2024 09:10:27 -0700 Subject: [PATCH 0229/1019] Flink: Removes Flink version 1.16 (#10154) --- flink/v1.16/build.gradle | 266 ---- flink/v1.16/flink-runtime/LICENSE | 502 ------- flink/v1.16/flink-runtime/NOTICE | 91 -- .../flink/IcebergConnectorSmokeTest.java | 29 - .../shuffle/MapRangePartitionerBenchmark.java | 199 --- .../apache/iceberg/flink/CatalogLoader.java | 215 --- .../apache/iceberg/flink/FlinkCatalog.java | 806 ------------ .../iceberg/flink/FlinkCatalogFactory.java | 213 --- .../apache/iceberg/flink/FlinkConfParser.java | 261 ---- .../iceberg/flink/FlinkConfigOptions.java | 107 -- .../flink/FlinkDynamicTableFactory.java | 209 --- .../flink/FlinkEnvironmentContext.java | 31 - .../apache/iceberg/flink/FlinkFilters.java | 266 ---- .../apache/iceberg/flink/FlinkFixupTypes.java | 50 - .../apache/iceberg/flink/FlinkReadConf.java | 213 --- .../iceberg/flink/FlinkReadOptions.java | 123 -- .../apache/iceberg/flink/FlinkSchemaUtil.java | 181 --- .../iceberg/flink/FlinkSourceFilter.java | 49 - .../apache/iceberg/flink/FlinkTypeToType.java | 199 --- .../iceberg/flink/FlinkTypeVisitor.java | 80 -- .../apache/iceberg/flink/FlinkWriteConf.java | 205 --- .../iceberg/flink/FlinkWriteOptions.java | 73 - .../iceberg/flink/IcebergTableSink.java | 121 -- .../apache/iceberg/flink/RowDataWrapper.java | 142 -- .../org/apache/iceberg/flink/TableLoader.java | 159 --- .../apache/iceberg/flink/TypeToFlinkType.java | 134 -- .../apache/iceberg/flink/actions/Actions.java | 52 - .../flink/actions/RewriteDataFilesAction.java | 72 - .../data/AvroWithFlinkSchemaVisitor.java | 75 -- .../iceberg/flink/data/FlinkAvroReader.java | 169 --- .../iceberg/flink/data/FlinkAvroWriter.java | 165 --- .../iceberg/flink/data/FlinkOrcReader.java | 131 -- .../iceberg/flink/data/FlinkOrcReaders.java | 283 ---- .../iceberg/flink/data/FlinkOrcWriter.java | 163 --- .../iceberg/flink/data/FlinkOrcWriters.java | 317 ----- .../flink/data/FlinkParquetReaders.java | 832 ------------ .../flink/data/FlinkParquetWriters.java | 504 ------- .../flink/data/FlinkSchemaVisitor.java | 161 --- .../iceberg/flink/data/FlinkValueReaders.java | 312 ----- .../iceberg/flink/data/FlinkValueWriters.java | 253 ---- .../data/ParquetWithFlinkSchemaVisitor.java | 222 ---- .../iceberg/flink/data/RowDataProjection.java | 341 ----- .../iceberg/flink/data/RowDataUtil.java | 100 -- .../iceberg/flink/data/StructRowData.java | 300 ----- .../AvroGenericRecordToRowDataMapper.java | 61 - .../flink/sink/BaseDeltaTaskWriter.java | 125 -- .../sink/BucketPartitionKeySelector.java | 70 - .../iceberg/flink/sink/BucketPartitioner.java | 103 -- .../flink/sink/BucketPartitionerUtil.java | 125 -- .../flink/sink/CachingTableSupplier.java | 91 -- .../iceberg/flink/sink/CommitSummary.java | 93 -- .../iceberg/flink/sink/DeltaManifests.java | 71 - .../flink/sink/DeltaManifestsSerializer.java | 122 -- .../flink/sink/EqualityFieldKeySelector.java | 86 -- .../flink/sink/FlinkAppenderFactory.java | 274 ---- .../flink/sink/FlinkFileWriterFactory.java | 293 ----- .../iceberg/flink/sink/FlinkManifestUtil.java | 132 -- .../apache/iceberg/flink/sink/FlinkSink.java | 654 --------- .../flink/sink/IcebergFilesCommitter.java | 516 -------- .../sink/IcebergFilesCommitterMetrics.java | 96 -- .../flink/sink/IcebergStreamWriter.java | 120 -- .../sink/IcebergStreamWriterMetrics.java | 89 -- .../flink/sink/ManifestOutputFileFactory.java | 94 -- .../flink/sink/PartitionKeySelector.java | 64 - .../flink/sink/PartitionedDeltaWriter.java | 97 -- .../flink/sink/RowDataTaskWriterFactory.java | 244 ---- .../iceberg/flink/sink/TaskWriterFactory.java | 45 - .../flink/sink/UnpartitionedDeltaWriter.java | 69 - .../sink/shuffle/AggregatedStatistics.java | 71 - .../shuffle/AggregatedStatisticsTracker.java | 133 -- .../flink/sink/shuffle/DataStatistics.java | 57 - .../shuffle/DataStatisticsCoordinator.java | 396 ------ .../DataStatisticsCoordinatorProvider.java | 51 - .../sink/shuffle/DataStatisticsEvent.java | 57 - .../sink/shuffle/DataStatisticsOperator.java | 197 --- .../sink/shuffle/DataStatisticsOrRecord.java | 113 -- .../DataStatisticsOrRecordSerializer.java | 219 --- .../sink/shuffle/DataStatisticsUtil.java | 96 -- .../flink/sink/shuffle/MapDataStatistics.java | 70 - .../shuffle/MapDataStatisticsSerializer.java | 187 --- .../sink/shuffle/MapRangePartitioner.java | 381 ------ .../flink/sink/shuffle/SortKeySerializer.java | 353 ----- .../AvroGenericRecordFileScanTaskReader.java | 42 - .../iceberg/flink/source/DataIterator.java | 156 --- .../iceberg/flink/source/DataTaskReader.java | 47 - .../flink/source/FileScanTaskReader.java | 35 - .../flink/source/FlinkInputFormat.java | 141 -- .../iceberg/flink/source/FlinkInputSplit.java | 48 - .../iceberg/flink/source/FlinkSource.java | 310 ----- .../flink/source/FlinkSplitPlanner.java | 189 --- .../iceberg/flink/source/IcebergSource.java | 543 -------- .../flink/source/IcebergTableSource.java | 229 ---- .../source/RowDataFileScanTaskReader.java | 243 ---- .../iceberg/flink/source/RowDataRewriter.java | 172 --- .../RowDataToAvroGenericRecordConverter.java | 70 - .../iceberg/flink/source/ScanContext.java | 592 --------- .../iceberg/flink/source/SourceUtil.java | 77 -- .../source/StreamingMonitorFunction.java | 269 ---- .../flink/source/StreamingReaderOperator.java | 246 ---- .../source/StreamingStartingStrategy.java | 54 - .../source/assigner/DefaultSplitAssigner.java | 112 -- .../flink/source/assigner/GetSplitResult.java | 77 -- .../assigner/OrderedSplitAssignerFactory.java | 46 - .../assigner/SimpleSplitAssignerFactory.java | 37 - .../flink/source/assigner/SplitAssigner.java | 118 -- .../source/assigner/SplitAssignerFactory.java | 30 - .../source/assigner/SplitAssignerType.java | 33 - .../enumerator/AbstractIcebergEnumerator.java | 173 --- .../ContinuousEnumerationResult.java | 57 - .../ContinuousIcebergEnumerator.java | 178 --- .../enumerator/ContinuousSplitPlanner.java | 30 - .../ContinuousSplitPlannerImpl.java | 240 ---- .../source/enumerator/EnumerationHistory.java | 96 -- .../enumerator/IcebergEnumeratorPosition.java | 79 -- .../IcebergEnumeratorPositionSerializer.java | 90 -- .../enumerator/IcebergEnumeratorState.java | 65 - .../IcebergEnumeratorStateSerializer.java | 196 --- .../enumerator/StaticIcebergEnumerator.java | 51 - .../source/reader/ArrayBatchRecords.java | 171 --- .../reader/ArrayPoolDataIteratorBatcher.java | 130 -- .../AvroGenericRecordReaderFunction.java | 102 -- .../reader/ColumnStatsWatermarkExtractor.java | 98 -- .../source/reader/DataIteratorBatcher.java | 36 - .../reader/DataIteratorReaderFunction.java | 43 - .../source/reader/IcebergSourceReader.java | 77 -- .../reader/IcebergSourceReaderMetrics.java | 61 - .../reader/IcebergSourceSplitReader.java | 156 --- .../flink/source/reader/ListBatchRecords.java | 85 -- .../reader/ListDataIteratorBatcher.java | 94 -- .../source/reader/MetaDataReaderFunction.java | 65 - .../flink/source/reader/ReaderFunction.java | 31 - .../source/reader/RecordAndPosition.java | 78 -- .../flink/source/reader/RecordFactory.java | 34 - .../source/reader/RowDataReaderFunction.java | 79 -- .../source/reader/RowDataRecordFactory.java | 61 - .../reader/SerializableRecordEmitter.java | 40 - .../reader/SplitWatermarkExtractor.java | 28 - .../WatermarkExtractorRecordEmitter.java | 67 - .../source/split/IcebergSourceSplit.java | 177 --- .../split/IcebergSourceSplitSerializer.java | 60 - .../source/split/IcebergSourceSplitState.java | 37 - .../split/IcebergSourceSplitStatus.java | 25 - .../source/split/SerializableComparator.java | 24 - .../flink/source/split/SplitComparators.java | 76 -- .../flink/source/split/SplitRequestEvent.java | 54 - .../flink/util/FlinkCompatibilityUtil.java | 42 - .../iceberg/flink/util/FlinkPackage.java | 61 - .../org.apache.flink.table.factories.Factory | 16 - ....apache.flink.table.factories.TableFactory | 16 - .../flink/AvroGenericRecordConverterBase.java | 90 -- .../apache/iceberg/flink/CatalogTestBase.java | 143 -- .../apache/iceberg/flink/DataGenerator.java | 42 - .../apache/iceberg/flink/DataGenerators.java | 1172 ----------------- .../apache/iceberg/flink/FlinkTestBase.java | 129 -- .../iceberg/flink/HadoopCatalogExtension.java | 104 -- .../iceberg/flink/HadoopCatalogResource.java | 90 -- .../iceberg/flink/HadoopTableResource.java | 64 - .../iceberg/flink/MiniClusterResource.java | 53 - .../flink/MiniFlinkClusterExtension.java | 53 - .../iceberg/flink/RowDataConverter.java | 135 -- .../apache/iceberg/flink/SimpleDataUtil.java | 443 ------- .../org/apache/iceberg/flink/TestBase.java | 130 -- .../iceberg/flink/TestCatalogLoader.java | 118 -- .../iceberg/flink/TestCatalogTableLoader.java | 116 -- .../iceberg/flink/TestChangeLogTable.java | 305 ----- .../flink/TestDataFileSerialization.java | 207 --- .../apache/iceberg/flink/TestFixtures.java | 59 - .../flink/TestFlinkAnonymousTable.java | 64 - .../flink/TestFlinkCatalogDatabase.java | 254 ---- .../flink/TestFlinkCatalogFactory.java | 115 -- .../iceberg/flink/TestFlinkCatalogTable.java | 413 ------ .../TestFlinkCatalogTablePartitions.java | 118 -- .../iceberg/flink/TestFlinkConfParser.java | 61 - .../iceberg/flink/TestFlinkFilters.java | 467 ------- .../iceberg/flink/TestFlinkHiveCatalog.java | 103 -- .../iceberg/flink/TestFlinkSchemaUtil.java | 415 ------ .../iceberg/flink/TestFlinkTableSink.java | 358 ----- .../apache/iceberg/flink/TestFlinkUpsert.java | 334 ----- .../org/apache/iceberg/flink/TestHelpers.java | 628 --------- .../iceberg/flink/TestIcebergConnector.java | 350 ----- .../flink/TestManifestFileSerialization.java | 174 --- .../iceberg/flink/TestRowDataWrapper.java | 93 -- .../apache/iceberg/flink/TestTableLoader.java | 57 - .../iceberg/flink/TestTableSerialization.java | 110 -- .../actions/TestRewriteDataFilesAction.java | 481 ------- .../iceberg/flink/data/RandomRowData.java | 38 - .../flink/data/RowDataToRowMapper.java | 50 - .../flink/data/TestFlinkAvroReaderWriter.java | 185 --- .../flink/data/TestFlinkOrcReaderWriter.java | 107 -- .../flink/data/TestFlinkParquetReader.java | 138 -- .../flink/data/TestFlinkParquetWriter.java | 94 -- .../flink/data/TestRowDataProjection.java | 593 --------- .../iceberg/flink/data/TestRowProjection.java | 580 -------- .../iceberg/flink/data/TestStructRowData.java | 100 -- .../TestAvroGenericRecordToRowDataMapper.java | 37 - .../sink/TestBucketPartitionKeySelector.java | 65 - .../flink/sink/TestBucketPartitioner.java | 107 -- ...TestBucketPartitionerFlinkIcebergSink.java | 227 ---- .../flink/sink/TestBucketPartitionerUtil.java | 126 -- .../flink/sink/TestCachingTableSupplier.java | 81 -- .../flink/sink/TestCompressionSettings.java | 254 ---- .../flink/sink/TestDeltaTaskWriter.java | 429 ------ .../flink/sink/TestFlinkAppenderFactory.java | 65 - .../sink/TestFlinkFileWriterFactory.java | 66 - .../flink/sink/TestFlinkIcebergSink.java | 397 ------ .../flink/sink/TestFlinkIcebergSinkBase.java | 64 - .../sink/TestFlinkIcebergSinkBranch.java | 137 -- .../flink/sink/TestFlinkIcebergSinkV2.java | 236 ---- .../sink/TestFlinkIcebergSinkV2Base.java | 361 ----- .../sink/TestFlinkIcebergSinkV2Branch.java | 134 -- .../iceberg/flink/sink/TestFlinkManifest.java | 315 ----- .../sink/TestFlinkPartitioningWriters.java | 66 - .../sink/TestFlinkPositionDeltaWriters.java | 66 - .../sink/TestFlinkRollingFileWriters.java | 51 - .../flink/sink/TestFlinkWriterMetrics.java | 60 - .../flink/sink/TestIcebergFilesCommitter.java | 1148 ---------------- .../flink/sink/TestIcebergStreamWriter.java | 388 ------ .../flink/sink/TestRowDataPartitionKey.java | 252 ---- .../iceberg/flink/sink/TestTaskWriters.java | 243 ---- .../shuffle/TestAggregatedStatistics.java | 63 - .../TestAggregatedStatisticsTracker.java | 178 --- .../TestDataStatisticsCoordinator.java | 178 --- ...TestDataStatisticsCoordinatorProvider.java | 143 -- .../shuffle/TestDataStatisticsOperator.java | 255 ---- .../sink/shuffle/TestMapDataStatistics.java | 90 -- .../sink/shuffle/TestMapRangePartitioner.java | 448 ------- .../shuffle/TestSortKeySerializerBase.java | 65 - .../TestSortKeySerializerNestedStruct.java | 55 - .../TestSortKeySerializerPrimitives.java | 57 - .../flink/source/BoundedTableFactory.java | 170 --- .../flink/source/BoundedTestSource.java | 108 -- .../flink/source/ChangeLogTableTestBase.java | 87 -- .../iceberg/flink/source/SplitHelpers.java | 132 -- .../iceberg/flink/source/SqlHelpers.java | 60 - .../flink/source/TestBoundedTableFactory.java | 87 -- .../flink/source/TestFlinkInputFormat.java | 191 --- .../TestFlinkInputFormatReaderDeletes.java | 70 - .../flink/source/TestFlinkMergingMetrics.java | 71 - .../flink/source/TestFlinkMetaDataTable.java | 813 ------------ .../source/TestFlinkReaderDeletesBase.java | 89 -- .../iceberg/flink/source/TestFlinkScan.java | 540 -------- .../flink/source/TestFlinkScanSql.java | 69 - .../iceberg/flink/source/TestFlinkSource.java | 90 -- .../flink/source/TestFlinkSourceConfig.java | 58 - .../flink/source/TestFlinkSourceSql.java | 85 -- .../flink/source/TestFlinkTableSource.java | 607 --------- .../source/TestIcebergSourceBounded.java | 127 -- ...TestIcebergSourceBoundedGenericRecord.java | 203 --- .../source/TestIcebergSourceBoundedSql.java | 76 -- .../source/TestIcebergSourceContinuous.java | 508 ------- .../source/TestIcebergSourceFailover.java | 297 ----- ...gSourceFailoverWithWatermarkExtractor.java | 112 -- .../TestIcebergSourceReaderDeletes.java | 102 -- .../flink/source/TestIcebergSourceSql.java | 158 --- ...stIcebergSourceWithWatermarkExtractor.java | 408 ------ .../TestMetadataTableReadableMetrics.java | 299 ----- .../flink/source/TestProjectMetaColumn.java | 186 --- ...stRowDataToAvroGenericRecordConverter.java | 35 - .../iceberg/flink/source/TestSourceUtil.java | 60 - .../iceberg/flink/source/TestSqlBase.java | 158 --- .../flink/source/TestStreamScanSql.java | 430 ------ .../source/TestStreamingMonitorFunction.java | 401 ------ .../source/TestStreamingReaderOperator.java | 293 ----- .../assigner/SplitAssignerTestBase.java | 130 -- .../assigner/TestDefaultSplitAssigner.java | 43 - ...tFileSequenceNumberBasedSplitAssigner.java | 80 -- .../TestWatermarkBasedSplitAssigner.java | 146 -- .../ManualContinuousSplitPlanner.java | 97 -- .../TestContinuousIcebergEnumerator.java | 352 ----- .../TestContinuousSplitPlannerImpl.java | 688 ---------- ...ntinuousSplitPlannerImplStartStrategy.java | 205 --- .../enumerator/TestEnumerationHistory.java | 134 -- .../TestIcebergEnumeratorStateSerializer.java | 145 -- .../source/reader/ReaderFunctionTestBase.java | 219 --- .../flink/source/reader/ReaderUtil.java | 125 -- .../source/reader/TestArrayBatchRecords.java | 68 - ...stArrayPoolDataIteratorBatcherRowData.java | 358 ----- .../TestColumnStatsWatermarkExtractor.java | 178 --- .../reader/TestIcebergSourceReader.java | 184 --- .../reader/TestRowDataReaderFunction.java | 74 -- .../source/reader/TestingMetricGroup.java | 102 -- .../TestIcebergSourceSplitSerializer.java | 162 --- .../iceberg/flink/util/TestFlinkPackage.java | 54 - .../org.apache.flink.table.factories.Factory | 16 - 284 files changed, 50366 deletions(-) delete mode 100644 flink/v1.16/build.gradle delete mode 100644 flink/v1.16/flink-runtime/LICENSE delete mode 100644 flink/v1.16/flink-runtime/NOTICE delete mode 100644 flink/v1.16/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java delete mode 100644 flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java delete mode 100644 flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java delete mode 100644 flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java delete mode 100644 flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java delete mode 100644 flink/v1.16/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.16/build.gradle b/flink/v1.16/build.gradle deleted file mode 100644 index 335a471e2455..000000000000 --- a/flink/v1.16/build.gradle +++ /dev/null @@ -1,266 +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. - */ - -String flinkMajorVersion = '1.16' -String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") - -project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { - - dependencies { - implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') - api project(':iceberg-api') - implementation project(':iceberg-common') - implementation project(':iceberg-core') - api project(':iceberg-data') - implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') - implementation project(':iceberg-hive-metastore') - - compileOnly libs.flink116.avro - // for dropwizard histogram metrics implementation - compileOnly libs.flink116.metrics.dropwizard - compileOnly libs.flink116.streaming.java - compileOnly "${libs.flink116.streaming.java.get().module}:${libs.flink116.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink116.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink116.get()}" - compileOnly libs.flink116.connector.base - compileOnly libs.flink116.connector.files - - compileOnly libs.hadoop2.hdfs - compileOnly libs.hadoop2.common - compileOnly(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - - 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.avro.avro - - implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { - exclude group: 'org.apache.hadoop' - exclude group: 'commons-lang' - // These artifacts are shaded and included in the orc-core fat jar - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.hive', module: 'hive-storage-api' - exclude group: 'org.slf4j' - } - - testImplementation libs.flink116.connector.test.utils - testImplementation libs.flink116.core - testImplementation libs.flink116.runtime - testImplementation (libs.flink116.test.utilsjunit) { - exclude group: 'junit' - } - testImplementation(libs.flink116.test.utils) { - exclude group: "org.apache.curator", module: 'curator-test' - exclude group: 'junit' - } - - testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') - - // By default, hive-exec is a fat/uber jar and it exports a guava library - // that's really old. We use the core classifier to be able to override our guava - // version. Luckily, hive-exec seems to work okay so far with this version of guava - // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. - testImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hive', module: 'hive-llap-tez' - exclude group: 'org.apache.logging.log4j' - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.calcite' - exclude group: 'org.apache.calcite.avatica' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - - testImplementation(libs.hive2.metastore) { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hbase' - exclude group: 'org.apache.logging.log4j' - exclude group: 'co.cask.tephra' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' - exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' - exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' - exclude group: 'com.tdunning', module: 'json' - exclude group: 'javax.transaction', module: 'transaction-api' - exclude group: 'com.zaxxer', module: 'HikariCP' - exclude group: 'org.slf4j' - } - - testImplementation libs.awaitility - testImplementation libs.assertj.core - } - - test { - useJUnitPlatform() - } -} - -project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' - - tasks.jar.dependsOn tasks.shadowJar - - sourceSets { - integration { - java.srcDir "$projectDir/src/integration/java" - resources.srcDir "$projectDir/src/integration/resources" - } - } - - configurations { - implementation { - // included in Flink - exclude group: 'org.slf4j' - exclude group: 'org.apache.commons' - exclude group: 'commons-pool' - exclude group: 'commons-codec' - exclude group: 'org.xerial.snappy' - exclude group: 'javax.xml.bind' - exclude group: 'javax.annotation' - } - } - - dependencies { - implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) { - exclude group: 'org.apache.flink' - } - implementation project(':iceberg-aws') - implementation project(':iceberg-azure') - implementation(project(':iceberg-aliyun')) { - exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' - exclude group: 'org.apache.httpcomponents', module: 'httpclient' - exclude group: 'commons-logging', module: 'commons-logging' - } - implementation project(':iceberg-gcp') - implementation(project(':iceberg-nessie')) { - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - - // for dropwizard histogram metrics implementation - implementation libs.flink116.metrics.dropwizard - - // for integration testing with the flink-runtime-jar - // all of those dependencies are required because the integration test extends FlinkTestBase - integrationCompileOnly project(':iceberg-api') - integrationImplementation libs.junit.vintage.engine - integrationImplementation libs.assertj.core - integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") - integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink116.test.utils) { - exclude group: "org.apache.curator", module: 'curator-test' - exclude group: 'junit' - } - - integrationImplementation libs.flink116.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink116.get()}" - - integrationImplementation libs.hadoop2.common - integrationImplementation libs.hadoop2.hdfs - integrationImplementation(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - - integrationImplementation(libs.hive2.metastore) { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hbase' - exclude group: 'org.apache.logging.log4j' - exclude group: 'co.cask.tephra' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' - exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' - exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' - exclude group: 'com.tdunning', module: 'json' - exclude group: 'javax.transaction', module: 'transaction-api' - exclude group: 'com.zaxxer', module: 'HikariCP' - exclude group: 'org.slf4j' - } - - integrationImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hive', module: 'hive-llap-tez' - exclude group: 'org.apache.logging.log4j' - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.calcite' - exclude group: 'org.apache.calcite.avatica' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - } - - shadowJar { - configurations = [project.configurations.runtimeClasspath] - - zip64 true - - // include the LICENSE and NOTICE files for the shaded Jar - from(projectDir) { - include 'LICENSE' - include 'NOTICE' - } - - // Relocate dependencies to avoid conflicts - relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' - relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' - relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' - relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' - relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' - relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' - relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' - relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' - relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' - relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' - relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' - relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' - relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' - - archiveClassifier.set(null) - } - - task integrationTest(type: Test) { - description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" - group = "verification" - jvmArgs += project.property('extraJvmArgs') - testClassesDirs = sourceSets.integration.output.classesDirs - classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) - inputs.file(shadowJar.archiveFile.get().asFile.path) - } - integrationTest.dependsOn shadowJar - check.dependsOn integrationTest - - jar { - enabled = false - } -} diff --git a/flink/v1.16/flink-runtime/LICENSE b/flink/v1.16/flink-runtime/LICENSE deleted file mode 100644 index 8ab53469eb87..000000000000 --- a/flink/v1.16/flink-runtime/LICENSE +++ /dev/null @@ -1,502 +0,0 @@ - - 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 Apache Avro. - -Copyright: 2014-2020 The Apache Software Foundation. -Home page: https://parquet.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains the Jackson JSON processor. - -Copyright: 2007-2020 Tatu Saloranta and other contributors -Home page: http://jackson.codehaus.org/ -License: http://www.apache.org/licenses/LICENSE-2.0.txt - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Parquet. - -Copyright: 2014-2020 The Apache Software Foundation. -Home page: https://parquet.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Thrift. - -Copyright: 2006-2010 The Apache Software Foundation. -Home page: https://thrift.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains fastutil. - -Copyright: 2002-2014 Sebastiano Vigna -Home page: http://fastutil.di.unimi.it/ -License: http://www.apache.org/licenses/LICENSE-2.0.html - --------------------------------------------------------------------------------- - -This binary artifact contains Apache ORC. - -Copyright: 2013-2020 The Apache Software Foundation. -Home page: https://orc.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Hive's storage API via ORC. - -Copyright: 2013-2020 The Apache Software Foundation. -Home page: https://hive.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Airlift Aircompressor. - -Copyright: 2011-2020 Aircompressor authors. -Home page: https://github.com/airlift/aircompressor -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Airlift Slice. - -Copyright: 2013-2020 Slice authors. -Home page: https://github.com/airlift/slice -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains JetBrains annotations. - -Copyright: 2000-2020 JetBrains s.r.o. -Home page: https://github.com/JetBrains/java-annotations -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google Guava. - -Copyright: 2006-2020 The Guava Authors -Home page: https://github.com/google/guava -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google Error Prone Annotations. - -Copyright: Copyright 2011-2019 The Error Prone Authors -Home page: https://github.com/google/error-prone -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains checkerframework checker-qual Annotations. - -Copyright: 2004-2020 the Checker Framework developers -Home page: https://github.com/typetools/checker-framework -License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) - -License text: -| The annotations are licensed under the MIT License. (The text of this -| license appears below.) More specifically, all the parts of the Checker -| Framework that you might want to include with your own program use the -| MIT License. This is the checker-qual.jar file and all the files that -| appear in it: every file in a qual/ directory, plus utility files such -| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. -| In addition, the cleanroom implementations of third-party annotations, -| which the Checker Framework recognizes as aliases for its own -| annotations, are licensed under the MIT License. -| -| Permission is hereby granted, free of charge, to any person obtaining a copy -| of this software and associated documentation files (the "Software"), to deal -| in the Software without restriction, including without limitation the rights -| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -| copies of the Software, and to permit persons to whom the Software is -| furnished to do so, subject to the following conditions: -| -| The above copyright notice and this permission notice shall be included in -| all copies or substantial portions of the Software. -| -| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -| THE SOFTWARE. - --------------------------------------------------------------------------------- - -This binary artifact contains Animal Sniffer Annotations. - -Copyright: 2009-2018 codehaus.org -Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/ -License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license) - -License text: -| The MIT License -| -| Copyright (c) 2009 codehaus.org. -| -| Permission is hereby granted, free of charge, to any person obtaining a copy -| of this software and associated documentation files (the "Software"), to deal -| in the Software without restriction, including without limitation the rights -| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -| copies of the Software, and to permit persons to whom the Software is -| furnished to do so, subject to the following conditions: -| -| The above copyright notice and this permission notice shall be included in -| all copies or substantial portions of the Software. -| -| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -| THE SOFTWARE. - --------------------------------------------------------------------------------- - -This binary artifact contains Caffeine by Ben Manes. - -Copyright: 2014-2020 Ben Manes and contributors -Home page: https://github.com/ben-manes/caffeine -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Yetus audience annotations. - -Copyright: 2008-2020 The Apache Software Foundation. -Home page: https://yetus.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google protobuf. - -Copyright: 2008 Google Inc. -Home page: https://developers.google.com/protocol-buffers -License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) - -License text: - -| Copyright 2008 Google Inc. All rights reserved. -| -| Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are -| met: -| -| * Redistributions of source code must retain the above copyright -| notice, this list of conditions and the following disclaimer. -| * Redistributions in binary form must reproduce the above -| copyright notice, this list of conditions and the following disclaimer -| in the documentation and/or other materials provided with the -| distribution. -| * Neither the name of Google Inc. nor the names of its -| contributors may be used to endorse or promote products derived from -| this software without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -| -| Code generated by the Protocol Buffer compiler is owned by the owner -| of the input file used when generating it. This code is not -| standalone and requires a support library to be linked with it. This -| support library is itself covered by the above license. - --------------------------------------------------------------------------------- - -This binary artifact contains ThreeTen. - -Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. -Home page: https://www.threeten.org/threeten-extra/ -License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) - -License text: - -| All rights reserved. -| -| * Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are met: -| -| * Redistributions of source code must retain the above copyright notice, -| this list of conditions and the following disclaimer. -| -| * Redistributions in binary form must reproduce the above copyright notice, -| this list of conditions and the following disclaimer in the documentation -| and/or other materials provided with the distribution. -| -| * Neither the name of JSR-310 nor the names of its contributors -| may be used to endorse or promote products derived from this software -| without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR -| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, -| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - -This binary artifact includes Project Nessie with the following in its NOTICE -file: - -| Dremio -| Copyright 2015-2017 Dremio Corporation -| -| This product includes software developed at -| The Apache Software Foundation (http://www.apache.org/). - --------------------------------------------------------------------------------- - -This binary 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 binary artifact contains Apache HttpComponents Client. - -Copyright: 1999-2022 The Apache Software Foundation. -Home page: https://hc.apache.org/ -License: http://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 diff --git a/flink/v1.16/flink-runtime/NOTICE b/flink/v1.16/flink-runtime/NOTICE deleted file mode 100644 index 81aa1660456a..000000000000 --- a/flink/v1.16/flink-runtime/NOTICE +++ /dev/null @@ -1,91 +0,0 @@ - -Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - --------------------------------------------------------------------------------- - -This binary artifact includes Apache ORC with the following in its NOTICE file: - -| Apache ORC -| Copyright 2013-2019 The Apache Software Foundation -| -| This product includes software developed by The Apache Software -| Foundation (http://www.apache.org/). -| -| This product includes software developed by Hewlett-Packard: -| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P - --------------------------------------------------------------------------------- - -This binary artifact includes Airlift Aircompressor with the following in its -NOTICE file: - -| Snappy Copyright Notices -| ========================= -| -| * Copyright 2011 Dain Sundstrom -| * Copyright 2011, Google Inc. -| -| -| Snappy License -| =============== -| Copyright 2011, Google Inc. -| All rights reserved. -| -| Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are -| met: -| -| * Redistributions of source code must retain the above copyright -| notice, this list of conditions and the following disclaimer. -| * Redistributions in binary form must reproduce the above -| copyright notice, this list of conditions and the following disclaimer -| in the documentation and/or other materials provided with the -| distribution. -| * Neither the name of Google Inc. nor the names of its -| contributors may be used to endorse or promote products derived from -| this software without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - -This binary artifact includes Apache Yetus with the following in its NOTICE -file: - -| Apache Yetus -| Copyright 2008-2020 The Apache Software Foundation -| -| This product includes software developed at -| The Apache Software Foundation (https://www.apache.org/). -| -| --- -| Additional licenses for the Apache Yetus Source/Website: -| --- -| -| -| See LICENSE for terms. - --------------------------------------------------------------------------------- - -This binary artifact includes Project Nessie with the following in its NOTICE -file: - -| Dremio -| Copyright 2015-2017 Dremio Corporation -| -| This product includes software developed at -| The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.16/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.16/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java deleted file mode 100644 index 65cfba1ec876..000000000000 --- a/flink/v1.16/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.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; - -import java.util.Map; - -public class IcebergConnectorSmokeTest extends TestIcebergConnector { - - public IcebergConnectorSmokeTest( - String catalogName, Map properties, boolean isStreaming) { - super(catalogName, properties, isStreaming); - } -} diff --git a/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java deleted file mode 100644 index c3917165753d..000000000000 --- a/flink/v1.16/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ /dev/null @@ -1,199 +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.sink.shuffle; - -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -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.apache.iceberg.types.Types; -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.TearDown; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -@Fork(1) -@State(Scope.Benchmark) -@Warmup(iterations = 3) -@Measurement(iterations = 5) -@BenchmarkMode(Mode.SingleShotTime) -public class MapRangePartitionerBenchmark { - private static final String CHARS = - "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; - private static final int SAMPLE_SIZE = 100_000; - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "name2", Types.StringType.get()), - Types.NestedField.required(3, "name3", Types.StringType.get()), - Types.NestedField.required(4, "name4", Types.StringType.get()), - Types.NestedField.required(5, "name5", Types.StringType.get()), - Types.NestedField.required(6, "name6", Types.StringType.get()), - Types.NestedField.required(7, "name7", Types.StringType.get()), - Types.NestedField.required(8, "name8", Types.StringType.get()), - Types.NestedField.required(9, "name9", Types.StringType.get())); - - private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); - - private MapRangePartitioner partitioner; - private RowData[] rows; - - @Setup - public void setupBenchmark() { - NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); - Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); - weights.forEach( - (id, weight) -> { - SortKey sortKey = SORT_KEY.copy(); - sortKey.set(0, id); - mapStatistics.put(sortKey, weight); - }); - - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); - this.partitioner = - new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); - - List keys = Lists.newArrayList(weights.keySet().iterator()); - long[] weightsCDF = new long[keys.size()]; - long totalWeight = 0; - for (int i = 0; i < keys.size(); ++i) { - totalWeight += weights.get(keys.get(i)); - weightsCDF[i] = totalWeight; - } - - // pre-calculate the samples for benchmark run - this.rows = new GenericRowData[SAMPLE_SIZE]; - for (int i = 0; i < SAMPLE_SIZE; ++i) { - long weight = ThreadLocalRandom.current().nextLong(totalWeight); - int index = binarySearchIndex(weightsCDF, weight); - rows[i] = - GenericRowData.of( - keys.get(index), - randomString("name2-"), - randomString("name3-"), - randomString("name4-"), - randomString("name5-"), - randomString("name6-"), - randomString("name7-"), - randomString("name8-"), - randomString("name9-")); - } - } - - @TearDown - public void tearDownBenchmark() {} - - @Benchmark - @Threads(1) - public void testPartitionerLongTailDistribution(Blackhole blackhole) { - for (int i = 0; i < SAMPLE_SIZE; ++i) { - blackhole.consume(partitioner.partition(rows[i], 128)); - } - } - - private static String randomString(String prefix) { - int length = ThreadLocalRandom.current().nextInt(200); - byte[] buffer = new byte[length]; - - for (int i = 0; i < length; i += 1) { - buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); - } - - return prefix + new String(buffer); - } - - /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ - private static int binarySearchIndex(long[] weightsUDF, long target) { - Preconditions.checkArgument( - target < weightsUDF[weightsUDF.length - 1], - "weight is out of range: total weight = %s, search target = %s", - weightsUDF[weightsUDF.length - 1], - target); - int start = 0; - int end = weightsUDF.length - 1; - while (start < end) { - int mid = (start + end) / 2; - if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { - return mid; - } - - if (weightsUDF[mid] >= target) { - end = mid - 1; - } else if (weightsUDF[mid + 1] < target) { - start = mid + 1; - } - } - return start; - } - - /** Key is the id string and value is the weight in long value. */ - private static NavigableMap longTailDistribution( - long startingWeight, - int longTailStartingIndex, - int longTailLength, - long longTailBaseWeight, - double weightRandomJitterPercentage) { - - NavigableMap weights = Maps.newTreeMap(); - - // first part just decays the weight by half - long currentWeight = startingWeight; - for (int index = 0; index < longTailStartingIndex; ++index) { - double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); - long weight = (long) (currentWeight * (1.0 + jitter)); - weight = weight > 0 ? weight : 1; - weights.put(index, weight); - if (currentWeight > longTailBaseWeight) { - currentWeight = currentWeight / 2; - } - } - - // long tail part - for (int index = longTailStartingIndex; - index < longTailStartingIndex + longTailLength; - ++index) { - long longTailWeight = - (long) - (longTailBaseWeight - * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); - longTailWeight = longTailWeight > 0 ? longTailWeight : 1; - weights.put(index, longTailWeight); - } - - return weights; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java deleted file mode 100644 index 18473bf4f190..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java +++ /dev/null @@ -1,215 +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; - -import java.io.Serializable; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hadoop.SerializableConfiguration; -import org.apache.iceberg.hive.HiveCatalog; -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.Maps; -import org.apache.iceberg.rest.RESTCatalog; - -/** Serializable loader to load an Iceberg {@link Catalog}. */ -public interface CatalogLoader extends Serializable, Cloneable { - - /** - * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the - * {@link CatalogLoader} at flink sql client side or job manager side, and then serialize this - * catalog loader to task manager, finally deserialize it and create a new catalog at task manager - * side. - * - * @return a newly created {@link Catalog} - */ - Catalog loadCatalog(); - - /** Clone a CatalogLoader. */ - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - CatalogLoader clone(); - - static CatalogLoader hadoop( - String name, Configuration hadoopConf, Map properties) { - return new HadoopCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader hive(String name, Configuration hadoopConf, Map properties) { - return new HiveCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader rest(String name, Configuration hadoopConf, Map properties) { - return new RESTCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader custom( - String name, Map properties, Configuration hadoopConf, String impl) { - return new CustomCatalogLoader(name, properties, hadoopConf, impl); - } - - class HadoopCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final String warehouseLocation; - private final Map properties; - - private HadoopCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new HadoopCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("warehouseLocation", warehouseLocation) - .toString(); - } - } - - class HiveCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final String uri; - private final String warehouse; - private final int clientPoolSize; - private final Map properties; - - private HiveCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.uri = properties.get(CatalogProperties.URI); - this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); - this.clientPoolSize = - properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) - ? Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) - : CatalogProperties.CLIENT_POOL_SIZE_DEFAULT; - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new HiveCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("uri", uri) - .add("warehouse", warehouse) - .add("clientPoolSize", clientPoolSize) - .toString(); - } - } - - class RESTCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final Map properties; - - private RESTCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - RESTCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new RESTCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("properties", properties) - .toString(); - } - } - - class CustomCatalogLoader implements CatalogLoader { - - private final SerializableConfiguration hadoopConf; - private final Map properties; - private final String name; - private final String impl; - - private CustomCatalogLoader( - String name, Map properties, Configuration conf, String impl) { - this.hadoopConf = new SerializableConfiguration(conf); - this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization - this.name = name; - this.impl = - Preconditions.checkNotNull( - impl, "Cannot initialize custom Catalog, impl class name is null"); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new CustomCatalogLoader(name, properties, new Configuration(hadoopConf.get()), impl); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("name", name).add("impl", impl).toString(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java deleted file mode 100644 index 825816fdf416..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ /dev/null @@ -1,806 +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; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.AbstractCatalog; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogDatabase; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogFunction; -import org.apache.flink.table.catalog.CatalogPartition; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; -import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; -import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; -import org.apache.flink.table.catalog.stats.CatalogTableStatistics; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.factories.Factory; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.CachingCatalog; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.UpdateProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.exceptions.NamespaceNotEmptyException; -import org.apache.iceberg.exceptions.NoSuchNamespaceException; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Splitter; -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.relocated.com.google.common.collect.Sets; - -/** - * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}. - * - *

    The mapping between Flink database and Iceberg namespace: Supplying a base namespace for a - * given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the - * first level in the catalog configuration and the second level would be exposed as Flink - * databases. - * - *

    The Iceberg table manages its partitions by itself. The partition of the Iceberg table is - * independent of the partition of Flink. - */ -public class FlinkCatalog extends AbstractCatalog { - - private final CatalogLoader catalogLoader; - private final Catalog icebergCatalog; - private final Namespace baseNamespace; - private final SupportsNamespaces asNamespaceCatalog; - private final Closeable closeable; - private final boolean cacheEnabled; - - public FlinkCatalog( - String catalogName, - String defaultDatabase, - Namespace baseNamespace, - CatalogLoader catalogLoader, - boolean cacheEnabled, - long cacheExpirationIntervalMs) { - super(catalogName, defaultDatabase); - this.catalogLoader = catalogLoader; - this.baseNamespace = baseNamespace; - this.cacheEnabled = cacheEnabled; - - Catalog originalCatalog = catalogLoader.loadCatalog(); - icebergCatalog = - cacheEnabled - ? CachingCatalog.wrap(originalCatalog, cacheExpirationIntervalMs) - : originalCatalog; - asNamespaceCatalog = - originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null; - closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null; - - FlinkEnvironmentContext.init(); - } - - @Override - public void open() throws CatalogException {} - - @Override - public void close() throws CatalogException { - if (closeable != null) { - try { - closeable.close(); - } catch (IOException e) { - throw new CatalogException(e); - } - } - } - - public Catalog catalog() { - return icebergCatalog; - } - - /** Append a new level to the base namespace */ - private static Namespace appendLevel(Namespace baseNamespace, String newLevel) { - String[] namespace = new String[baseNamespace.levels().length + 1]; - System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length); - namespace[baseNamespace.levels().length] = newLevel; - return Namespace.of(namespace); - } - - TableIdentifier toIdentifier(ObjectPath path) { - String objectName = path.getObjectName(); - List tableName = Splitter.on('$').splitToList(objectName); - - if (tableName.size() == 1) { - return TableIdentifier.of( - appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName()); - } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) { - return TableIdentifier.of( - appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)), - tableName.get(1)); - } else { - throw new IllegalArgumentException("Illegal table name:" + objectName); - } - } - - @Override - public List listDatabases() throws CatalogException { - if (asNamespaceCatalog == null) { - return Collections.singletonList(getDefaultDatabase()); - } - - return asNamespaceCatalog.listNamespaces(baseNamespace).stream() - .map(n -> n.level(n.levels().length - 1)) - .collect(Collectors.toList()); - } - - @Override - public CatalogDatabase getDatabase(String databaseName) - throws DatabaseNotExistException, CatalogException { - if (asNamespaceCatalog == null) { - if (!getDefaultDatabase().equals(databaseName)) { - throw new DatabaseNotExistException(getName(), databaseName); - } else { - return new CatalogDatabaseImpl(Maps.newHashMap(), ""); - } - } else { - try { - Map metadata = - Maps.newHashMap( - asNamespaceCatalog.loadNamespaceMetadata(appendLevel(baseNamespace, databaseName))); - String comment = metadata.remove("comment"); - return new CatalogDatabaseImpl(metadata, comment); - } catch (NoSuchNamespaceException e) { - throw new DatabaseNotExistException(getName(), databaseName, e); - } - } - } - - @Override - public boolean databaseExists(String databaseName) throws CatalogException { - try { - getDatabase(databaseName); - return true; - } catch (DatabaseNotExistException ignore) { - return false; - } - } - - @Override - public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - createDatabase( - name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists); - } - - private void createDatabase( - String databaseName, Map metadata, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - if (asNamespaceCatalog != null) { - try { - asNamespaceCatalog.createNamespace(appendLevel(baseNamespace, databaseName), metadata); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new DatabaseAlreadyExistException(getName(), databaseName, e); - } - } - } else { - throw new UnsupportedOperationException( - "Namespaces are not supported by catalog: " + getName()); - } - } - - private Map mergeComment(Map metadata, String comment) { - Map ret = Maps.newHashMap(metadata); - if (metadata.containsKey("comment")) { - throw new CatalogException("Database properties should not contain key: 'comment'."); - } - - if (!StringUtils.isNullOrWhitespaceOnly(comment)) { - ret.put("comment", comment); - } - return ret; - } - - @Override - public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) - throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { - if (asNamespaceCatalog != null) { - try { - boolean success = asNamespaceCatalog.dropNamespace(appendLevel(baseNamespace, name)); - if (!success && !ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } catch (NoSuchNamespaceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name, e); - } - } catch (NamespaceNotEmptyException e) { - throw new DatabaseNotEmptyException(getName(), name, e); - } - } else { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } - } - - @Override - public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) - throws DatabaseNotExistException, CatalogException { - if (asNamespaceCatalog != null) { - Namespace namespace = appendLevel(baseNamespace, name); - Map updates = Maps.newHashMap(); - Set removals = Sets.newHashSet(); - - try { - Map oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace); - Map newProperties = - mergeComment(newDatabase.getProperties(), newDatabase.getComment()); - - for (String key : oldProperties.keySet()) { - if (!newProperties.containsKey(key)) { - removals.add(key); - } - } - - for (Map.Entry entry : newProperties.entrySet()) { - if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) { - updates.put(entry.getKey(), entry.getValue()); - } - } - - if (!updates.isEmpty()) { - asNamespaceCatalog.setProperties(namespace, updates); - } - - if (!removals.isEmpty()) { - asNamespaceCatalog.removeProperties(namespace, removals); - } - - } catch (NoSuchNamespaceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name, e); - } - } - } else { - if (getDefaultDatabase().equals(name)) { - throw new CatalogException( - "Can not alter the default database when the iceberg catalog doesn't support namespaces."); - } - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } - } - - @Override - public List listTables(String databaseName) - throws DatabaseNotExistException, CatalogException { - try { - return icebergCatalog.listTables(appendLevel(baseNamespace, databaseName)).stream() - .map(TableIdentifier::name) - .collect(Collectors.toList()); - } catch (NoSuchNamespaceException e) { - throw new DatabaseNotExistException(getName(), databaseName, e); - } - } - - @Override - public CatalogTable getTable(ObjectPath tablePath) - throws TableNotExistException, CatalogException { - Table table = loadIcebergTable(tablePath); - return toCatalogTable(table); - } - - private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { - try { - Table table = icebergCatalog.loadTable(toIdentifier(tablePath)); - if (cacheEnabled) { - table.refresh(); - } - - return table; - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new TableNotExistException(getName(), tablePath, e); - } - } - - @Override - public boolean tableExists(ObjectPath tablePath) throws CatalogException { - return icebergCatalog.tableExists(toIdentifier(tablePath)); - } - - @Override - public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) - throws TableNotExistException, CatalogException { - try { - icebergCatalog.dropTable(toIdentifier(tablePath)); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - if (!ignoreIfNotExists) { - throw new TableNotExistException(getName(), tablePath, e); - } - } - } - - @Override - public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) - throws TableNotExistException, TableAlreadyExistException, CatalogException { - try { - icebergCatalog.renameTable( - toIdentifier(tablePath), - toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName))); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - if (!ignoreIfNotExists) { - throw new TableNotExistException(getName(), tablePath, e); - } - } catch (AlreadyExistsException e) { - throw new TableAlreadyExistException(getName(), tablePath, e); - } - } - - @Override - public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) - throws CatalogException, TableAlreadyExistException { - if (Objects.equals( - table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { - throw new IllegalArgumentException( - "Cannot create the table with 'connector'='iceberg' table property in " - + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " - + "create table without 'connector'='iceberg' related properties in an iceberg table."); - } - - createIcebergTable(tablePath, table, ignoreIfExists); - } - - void createIcebergTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) - throws CatalogException, TableAlreadyExistException { - validateFlinkTable(table); - - Schema icebergSchema = FlinkSchemaUtil.convert(table.getSchema()); - PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); - - ImmutableMap.Builder properties = ImmutableMap.builder(); - String location = null; - for (Map.Entry entry : table.getOptions().entrySet()) { - if ("location".equalsIgnoreCase(entry.getKey())) { - location = entry.getValue(); - } else { - properties.put(entry.getKey(), entry.getValue()); - } - } - - try { - icebergCatalog.createTable( - toIdentifier(tablePath), icebergSchema, spec, location, properties.build()); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new TableAlreadyExistException(getName(), tablePath, e); - } - } - } - - private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) { - TableSchema ts1 = ct1.getSchema(); - TableSchema ts2 = ct2.getSchema(); - boolean equalsPrimary = false; - - if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { - equalsPrimary = - Objects.equals(ts1.getPrimaryKey().get().getType(), ts2.getPrimaryKey().get().getType()) - && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), ts2.getPrimaryKey().get().getColumns()); - } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { - equalsPrimary = true; - } - - if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) - && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) - && equalsPrimary)) { - throw new UnsupportedOperationException("Altering schema is not supported yet."); - } - - if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { - throw new UnsupportedOperationException("Altering partition keys is not supported yet."); - } - } - - @Override - public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) - throws CatalogException, TableNotExistException { - validateFlinkTable(newTable); - - Table icebergTable; - try { - icebergTable = loadIcebergTable(tablePath); - } catch (TableNotExistException e) { - if (!ignoreIfNotExists) { - throw e; - } else { - return; - } - } - - CatalogTable table = toCatalogTable(icebergTable); - - // Currently, Flink SQL only support altering table properties. - - // For current Flink Catalog API, support for adding/removing/renaming columns cannot be done by - // comparing - // CatalogTable instances, unless the Flink schema contains Iceberg column IDs. - validateTableSchemaAndPartition(table, (CatalogTable) newTable); - - Map oldProperties = table.getOptions(); - Map setProperties = Maps.newHashMap(); - - String setLocation = null; - String setSnapshotId = null; - String pickSnapshotId = null; - - for (Map.Entry entry : newTable.getOptions().entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - - if (Objects.equals(value, oldProperties.get(key))) { - continue; - } - - if ("location".equalsIgnoreCase(key)) { - setLocation = value; - } else if ("current-snapshot-id".equalsIgnoreCase(key)) { - setSnapshotId = value; - } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) { - pickSnapshotId = value; - } else { - setProperties.put(key, value); - } - } - - oldProperties - .keySet() - .forEach( - k -> { - if (!newTable.getOptions().containsKey(k)) { - setProperties.put(k, null); - } - }); - - commitChanges(icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); - } - - private static void validateFlinkTable(CatalogBaseTable table) { - Preconditions.checkArgument( - table instanceof CatalogTable, "The Table should be a CatalogTable."); - - TableSchema schema = table.getSchema(); - schema - .getTableColumns() - .forEach( - column -> { - if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) { - throw new UnsupportedOperationException( - "Creating table with computed columns is not supported yet."); - } - }); - - if (!schema.getWatermarkSpecs().isEmpty()) { - throw new UnsupportedOperationException( - "Creating table with watermark specs is not supported yet."); - } - } - - private static PartitionSpec toPartitionSpec(List partitionKeys, Schema icebergSchema) { - PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema); - partitionKeys.forEach(builder::identity); - return builder.build(); - } - - private static List toPartitionKeys(PartitionSpec spec, Schema icebergSchema) { - ImmutableList.Builder partitionKeysBuilder = ImmutableList.builder(); - for (PartitionField field : spec.fields()) { - if (field.transform().isIdentity()) { - partitionKeysBuilder.add(icebergSchema.findColumnName(field.sourceId())); - } else { - // Not created by Flink SQL. - // For compatibility with iceberg tables, return empty. - // TODO modify this after Flink support partition transform. - return Collections.emptyList(); - } - } - return partitionKeysBuilder.build(); - } - - private static void commitChanges( - Table table, - String setLocation, - String setSnapshotId, - String pickSnapshotId, - Map setProperties) { - // don't allow setting the snapshot and picking a commit at the same time because order is - // ambiguous and choosing - // one order leads to different results - Preconditions.checkArgument( - setSnapshotId == null || pickSnapshotId == null, - "Cannot set the current snapshot ID and cherry-pick snapshot changes"); - - if (setSnapshotId != null) { - long newSnapshotId = Long.parseLong(setSnapshotId); - table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); - } - - // if updating the table snapshot, perform that update first in case it fails - if (pickSnapshotId != null) { - long newSnapshotId = Long.parseLong(pickSnapshotId); - table.manageSnapshots().cherrypick(newSnapshotId).commit(); - } - - Transaction transaction = table.newTransaction(); - - if (setLocation != null) { - transaction.updateLocation().setLocation(setLocation).commit(); - } - - if (!setProperties.isEmpty()) { - UpdateProperties updateProperties = transaction.updateProperties(); - setProperties.forEach( - (k, v) -> { - if (v == null) { - updateProperties.remove(k); - } else { - updateProperties.set(k, v); - } - }); - updateProperties.commit(); - } - - transaction.commitTransaction(); - } - - static CatalogTable toCatalogTable(Table table) { - TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); - List partitionKeys = toPartitionKeys(table.spec(), table.schema()); - - // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer - // may use - // CatalogTableImpl to copy a new catalog table. - // Let's re-loading table from Iceberg catalog when creating source/sink operators. - // Iceberg does not have Table comment, so pass a null (Default comment value in Flink). - return new CatalogTableImpl(schema, partitionKeys, table.properties(), null); - } - - @Override - public Optional getFactory() { - return Optional.of(new FlinkDynamicTableFactory(this)); - } - - CatalogLoader getCatalogLoader() { - return catalogLoader; - } - - // ------------------------------ Unsupported methods - // --------------------------------------------- - - @Override - public List listViews(String databaseName) throws CatalogException { - return Collections.emptyList(); - } - - @Override - public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void createPartition( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogPartition partition, - boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropPartition( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartition( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogPartition newPartition, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listFunctions(String dbName) throws CatalogException { - return Collections.emptyList(); - } - - @Override - public CatalogFunction getFunction(ObjectPath functionPath) - throws FunctionNotExistException, CatalogException { - throw new FunctionNotExistException(getName(), functionPath); - } - - @Override - public boolean functionExists(ObjectPath functionPath) throws CatalogException { - return false; - } - - @Override - public void createFunction( - ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterFunction( - ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTableStatistics( - ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTableColumnStatistics( - ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartitionStatistics( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogTableStatistics partitionStatistics, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartitionColumnStatistics( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogColumnStatistics columnStatistics, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listPartitions(ObjectPath tablePath) - throws TableNotExistException, TableNotPartitionedException, CatalogException { - Table table = loadIcebergTable(tablePath); - - if (table.spec().isUnpartitioned()) { - throw new TableNotPartitionedException(icebergCatalog.name(), tablePath); - } - - Set set = Sets.newHashSet(); - try (CloseableIterable tasks = table.newScan().planFiles()) { - for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) { - Map map = Maps.newHashMap(); - StructLike structLike = dataFile.partition(); - PartitionSpec spec = table.specs().get(dataFile.specId()); - for (int i = 0; i < structLike.size(); i++) { - map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class))); - } - set.add(new CatalogPartitionSpec(map)); - } - } catch (IOException e) { - throw new CatalogException( - String.format("Failed to list partitions of table %s", tablePath), e); - } - - return Lists.newArrayList(set); - } - - @Override - public List listPartitions( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listPartitionsByFilter( - ObjectPath tablePath, List filters) throws CatalogException { - throw new UnsupportedOperationException(); - } - - // After partition pruning and filter push down, the statistics have become very inaccurate, so - // the statistics from - // here are of little significance. - // Flink will support something like SupportsReportStatistics in future. - - @Override - public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { - return CatalogTableStatistics.UNKNOWN; - } - - @Override - public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) - throws CatalogException { - return CatalogColumnStatistics.UNKNOWN; - } - - @Override - public CatalogTableStatistics getPartitionStatistics( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - return CatalogTableStatistics.UNKNOWN; - } - - @Override - public CatalogColumnStatistics getPartitionColumnStatistics( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - return CatalogColumnStatistics.UNKNOWN; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java deleted file mode 100644 index 1453753849ec..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ /dev/null @@ -1,213 +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; - -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.util.HadoopUtils; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.factories.CatalogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Strings; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.PropertyUtil; - -/** - * A Flink Catalog factory implementation that creates {@link FlinkCatalog}. - * - *

    This supports the following catalog configuration options: - * - *

      - *
    • type - Flink catalog factory key, should be "iceberg" - *
    • catalog-type - iceberg catalog type, "hive", "hadoop" or "rest" - *
    • uri - the Hive Metastore URI (Hive catalog only) - *
    • clients - the Hive Client Pool Size (Hive catalog only) - *
    • warehouse - the warehouse path (Hadoop catalog only) - *
    • default-database - a database name to use as the default - *
    • base-namespace - a base namespace as the prefix for all databases (Hadoop - * catalog only) - *
    • cache-enabled - whether to enable catalog cache - *
    - * - *

    To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override - * {@link #createCatalogLoader(String, Map, Configuration)}. - */ -public class FlinkCatalogFactory implements CatalogFactory { - - // Can not just use "type", it conflicts with CATALOG_TYPE. - public static final String ICEBERG_CATALOG_TYPE = "catalog-type"; - public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop"; - public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive"; - public static final String ICEBERG_CATALOG_TYPE_REST = "rest"; - - public static final String HIVE_CONF_DIR = "hive-conf-dir"; - public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; - public static final String DEFAULT_DATABASE = "default-database"; - public static final String DEFAULT_DATABASE_NAME = "default"; - public static final String BASE_NAMESPACE = "base-namespace"; - - public static final String TYPE = "type"; - public static final String PROPERTY_VERSION = "property-version"; - - /** - * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink - * catalog adapter. - * - * @param name Flink's catalog name - * @param properties Flink's catalog properties - * @param hadoopConf Hadoop configuration for catalog - * @return an Iceberg catalog loader - */ - static CatalogLoader createCatalogLoader( - String name, Map properties, Configuration hadoopConf) { - String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); - if (catalogImpl != null) { - String catalogType = properties.get(ICEBERG_CATALOG_TYPE); - Preconditions.checkArgument( - catalogType == null, - "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s", - name, - catalogType, - catalogImpl); - return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl); - } - - String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE); - switch (catalogType.toLowerCase(Locale.ENGLISH)) { - case ICEBERG_CATALOG_TYPE_HIVE: - // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in - // that case it will - // fallback to parse those values from hadoop configuration which is loaded from classpath. - String hiveConfDir = properties.get(HIVE_CONF_DIR); - String hadoopConfDir = properties.get(HADOOP_CONF_DIR); - Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); - return CatalogLoader.hive(name, newHadoopConf, properties); - - case ICEBERG_CATALOG_TYPE_HADOOP: - return CatalogLoader.hadoop(name, hadoopConf, properties); - - case ICEBERG_CATALOG_TYPE_REST: - return CatalogLoader.rest(name, hadoopConf, properties); - - default: - throw new UnsupportedOperationException( - "Unknown catalog-type: " + catalogType + " (Must be 'hive', 'hadoop' or 'rest')"); - } - } - - @Override - public Map requiredContext() { - Map context = Maps.newHashMap(); - context.put(TYPE, "iceberg"); - context.put(PROPERTY_VERSION, "1"); - return context; - } - - @Override - public List supportedProperties() { - return ImmutableList.of("*"); - } - - @Override - public Catalog createCatalog(String name, Map properties) { - return createCatalog(name, properties, clusterHadoopConf()); - } - - protected Catalog createCatalog( - String name, Map properties, Configuration hadoopConf) { - CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); - String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); - - Namespace baseNamespace = Namespace.empty(); - if (properties.containsKey(BASE_NAMESPACE)) { - baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\.")); - } - - boolean cacheEnabled = - PropertyUtil.propertyAsBoolean( - properties, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); - - long cacheExpirationIntervalMs = - PropertyUtil.propertyAsLong( - properties, - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); - Preconditions.checkArgument( - cacheExpirationIntervalMs != 0, - "%s is not allowed to be 0.", - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS); - - return new FlinkCatalog( - name, - defaultDatabase, - baseNamespace, - catalogLoader, - cacheEnabled, - cacheExpirationIntervalMs); - } - - private static Configuration mergeHiveConf( - Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { - Configuration newConf = new Configuration(hadoopConf); - if (!Strings.isNullOrEmpty(hiveConfDir)) { - Preconditions.checkState( - Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), - "There should be a hive-site.xml file under the directory %s", - hiveConfDir); - newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); - } else { - // If don't provide the hive-site.xml path explicitly, it will try to load resource from - // classpath. If still - // couldn't load the configuration file, then it will throw exception in HiveCatalog. - URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); - if (configFile != null) { - newConf.addResource(configFile); - } - } - - if (!Strings.isNullOrEmpty(hadoopConfDir)) { - Preconditions.checkState( - Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), - "Failed to load Hadoop configuration: missing %s", - Paths.get(hadoopConfDir, "hdfs-site.xml")); - newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); - Preconditions.checkState( - Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), - "Failed to load Hadoop configuration: missing %s", - Paths.get(hadoopConfDir, "core-site.xml")); - newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); - } - - return newConf; - } - - public static Configuration clusterHadoopConf() { - return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java deleted file mode 100644 index 7167859e600c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ /dev/null @@ -1,261 +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; - -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class FlinkConfParser { - - private final Map tableProperties; - private final Map options; - private final ReadableConfig readableConfig; - - FlinkConfParser(Table table, Map options, ReadableConfig readableConfig) { - this.tableProperties = table.properties(); - this.options = options; - this.readableConfig = readableConfig; - } - - public BooleanConfParser booleanConf() { - return new BooleanConfParser(); - } - - public IntConfParser intConf() { - return new IntConfParser(); - } - - public LongConfParser longConf() { - return new LongConfParser(); - } - - public > EnumConfParser enumConfParser(Class enumClass) { - return new EnumConfParser<>(enumClass); - } - - public StringConfParser stringConf() { - return new StringConfParser(); - } - - public DurationConfParser durationConf() { - return new DurationConfParser(); - } - - class BooleanConfParser extends ConfParser { - private Boolean defaultValue; - - @Override - protected BooleanConfParser self() { - return this; - } - - public BooleanConfParser defaultValue(boolean value) { - this.defaultValue = value; - return self(); - } - - public BooleanConfParser defaultValue(String value) { - this.defaultValue = Boolean.parseBoolean(value); - return self(); - } - - public boolean parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Boolean::parseBoolean, defaultValue); - } - } - - class IntConfParser extends ConfParser { - private Integer defaultValue; - - @Override - protected IntConfParser self() { - return this; - } - - public IntConfParser defaultValue(int value) { - this.defaultValue = value; - return self(); - } - - public int parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Integer::parseInt, defaultValue); - } - - public Integer parseOptional() { - return parse(Integer::parseInt, null); - } - } - - class LongConfParser extends ConfParser { - private Long defaultValue; - - @Override - protected LongConfParser self() { - return this; - } - - public LongConfParser defaultValue(long value) { - this.defaultValue = value; - return self(); - } - - public long parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Long::parseLong, defaultValue); - } - - public Long parseOptional() { - return parse(Long::parseLong, null); - } - } - - class StringConfParser extends ConfParser { - private String defaultValue; - - @Override - protected StringConfParser self() { - return this; - } - - public StringConfParser defaultValue(String value) { - this.defaultValue = value; - return self(); - } - - public String parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Function.identity(), defaultValue); - } - - public String parseOptional() { - return parse(Function.identity(), null); - } - } - - class EnumConfParser> extends ConfParser, E> { - private E defaultValue; - private final Class enumClass; - - EnumConfParser(Class enumClass) { - this.enumClass = enumClass; - } - - @Override - protected EnumConfParser self() { - return this; - } - - public EnumConfParser defaultValue(E value) { - this.defaultValue = value; - return self(); - } - - public E parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(s -> Enum.valueOf(enumClass, s), defaultValue); - } - - public E parseOptional() { - return parse(s -> Enum.valueOf(enumClass, s), null); - } - } - - class DurationConfParser extends ConfParser { - private Duration defaultValue; - - @Override - protected DurationConfParser self() { - return this; - } - - public DurationConfParser defaultValue(Duration value) { - this.defaultValue = value; - return self(); - } - - public Duration parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(TimeUtils::parseDuration, defaultValue); - } - - public Duration parseOptional() { - return parse(TimeUtils::parseDuration, null); - } - } - - abstract class ConfParser { - private final List optionNames = Lists.newArrayList(); - private String tablePropertyName; - private ConfigOption configOption; - - protected abstract ThisT self(); - - public ThisT option(String name) { - this.optionNames.add(name); - return self(); - } - - public ThisT flinkConfig(ConfigOption newConfigOption) { - this.configOption = newConfigOption; - return self(); - } - - public ThisT tableProperty(String name) { - this.tablePropertyName = name; - return self(); - } - - protected T parse(Function conversion, T defaultValue) { - if (!optionNames.isEmpty()) { - for (String optionName : optionNames) { - String optionValue = options.get(optionName); - if (optionValue != null) { - return conversion.apply(optionValue); - } - } - } - - if (configOption != null) { - T propertyValue = readableConfig.get(configOption); - if (propertyValue != null) { - return propertyValue; - } - } - - if (tablePropertyName != null) { - String propertyValue = tableProperties.get(tablePropertyName); - if (propertyValue != null) { - return conversion.apply(propertyValue); - } - } - - return defaultValue; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java deleted file mode 100644 index 7c7afd24ed8e..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ /dev/null @@ -1,107 +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; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.description.Description; -import org.apache.flink.configuration.description.TextElement; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.iceberg.flink.source.assigner.SplitAssignerType; -import org.apache.iceberg.util.ThreadPools; - -/** - * When constructing Flink Iceberg source via Java API, configs can be set in {@link Configuration} - * passed to source builder. E.g. - * - *

    - *   configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    - *   FlinkSource.forRowData()
    - *       .flinkConf(configuration)
    - *       ...
    - * 
    - * - *

    When using Flink SQL/table API, connector options can be set in Flink's {@link - * TableEnvironment}. - * - *

    - *   TableEnvironment tEnv = createTableEnv();
    - *   tEnv.getConfig()
    - *        .getConfiguration()
    - *        .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    - * 
    - */ -public class FlinkConfigOptions { - - private FlinkConfigOptions() {} - - public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = - ConfigOptions.key("table.exec.iceberg.infer-source-parallelism") - .booleanType() - .defaultValue(true) - .withDescription( - "If is false, parallelism of source are set by config.\n" - + "If is true, source parallelism is inferred according to splits number.\n"); - - public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX = - ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max") - .intType() - .defaultValue(100) - .withDescription("Sets max infer parallelism for source operator."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = - ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") - .booleanType() - .noDefaultValue() - .withDescription( - "Expose split host information to use Flink's locality aware split assigner."); - - public static final ConfigOption SOURCE_READER_FETCH_BATCH_RECORD_COUNT = - ConfigOptions.key("table.exec.iceberg.fetch-batch-record-count") - .intType() - .defaultValue(2048) - .withDescription("The target number of records for Iceberg reader fetch batch."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE = - ConfigOptions.key("table.exec.iceberg.worker-pool-size") - .intType() - .defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE) - .withDescription("The size of workers pool used to plan or scan manifests."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = - ConfigOptions.key("table.exec.iceberg.use-flip27-source") - .booleanType() - .defaultValue(false) - .withDescription("Use the FLIP-27 based Iceberg source implementation."); - - public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = - ConfigOptions.key("table.exec.iceberg.split-assigner-type") - .enumType(SplitAssignerType.class) - .defaultValue(SplitAssignerType.SIMPLE) - .withDescription( - Description.builder() - .text("Split assigner type that determine how splits are assigned to readers.") - .linebreak() - .list( - TextElement.text( - SplitAssignerType.SIMPLE - + ": simple assigner that doesn't provide any guarantee on order or locality.")) - .build()); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java deleted file mode 100644 index 8e1f420b722d..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ /dev/null @@ -1,209 +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; - -import java.util.Map; -import java.util.Set; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.flink.source.IcebergTableSource; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; - -public class FlinkDynamicTableFactory - implements DynamicTableSinkFactory, DynamicTableSourceFactory { - static final String FACTORY_IDENTIFIER = "iceberg"; - - private static final ConfigOption CATALOG_NAME = - ConfigOptions.key("catalog-name") - .stringType() - .noDefaultValue() - .withDescription("Catalog name"); - - private static final ConfigOption CATALOG_TYPE = - ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) - .stringType() - .noDefaultValue() - .withDescription("Catalog type, the optional types are: custom, hadoop, hive."); - - private static final ConfigOption CATALOG_DATABASE = - ConfigOptions.key("catalog-database") - .stringType() - .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) - .withDescription("Database name managed in the iceberg catalog."); - - private static final ConfigOption CATALOG_TABLE = - ConfigOptions.key("catalog-table") - .stringType() - .noDefaultValue() - .withDescription("Table name managed in the underlying iceberg catalog and database."); - - private final FlinkCatalog catalog; - - public FlinkDynamicTableFactory() { - this.catalog = null; - } - - public FlinkDynamicTableFactory(FlinkCatalog catalog) { - this.catalog = catalog; - } - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map tableProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); - - TableLoader tableLoader; - if (catalog != null) { - tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); - } else { - tableLoader = - createTableLoader( - catalogTable, - tableProps, - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - } - - return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); - } - - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - CatalogTable catalogTable = context.getCatalogTable(); - Map writeProps = catalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(catalogTable.getSchema()); - - TableLoader tableLoader; - if (catalog != null) { - tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); - } else { - tableLoader = - createTableLoader( - catalogTable, - writeProps, - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - } - - return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); - } - - @Override - public Set> requiredOptions() { - Set> options = Sets.newHashSet(); - options.add(CATALOG_TYPE); - options.add(CATALOG_NAME); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = Sets.newHashSet(); - options.add(CATALOG_DATABASE); - options.add(CATALOG_TABLE); - return options; - } - - @Override - public String factoryIdentifier() { - return FACTORY_IDENTIFIER; - } - - private static TableLoader createTableLoader( - CatalogBaseTable catalogBaseTable, - Map tableProps, - String databaseName, - String tableName) { - Configuration flinkConf = new Configuration(); - tableProps.forEach(flinkConf::setString); - - String catalogName = flinkConf.getString(CATALOG_NAME); - Preconditions.checkNotNull( - catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); - - String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); - Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); - - String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); - Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); - - org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); - FlinkCatalogFactory factory = new FlinkCatalogFactory(); - FlinkCatalog flinkCatalog = - (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); - ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); - - // Create database if not exists in the external catalog. - if (!flinkCatalog.databaseExists(catalogDatabase)) { - try { - flinkCatalog.createDatabase( - catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true); - } catch (DatabaseAlreadyExistException e) { - throw new AlreadyExistsException( - e, - "Database %s already exists in the iceberg catalog %s.", - catalogName, - catalogDatabase); - } - } - - // Create table if not exists in the external catalog. - if (!flinkCatalog.tableExists(objectPath)) { - try { - flinkCatalog.createIcebergTable(objectPath, catalogBaseTable, true); - } catch (TableAlreadyExistException e) { - throw new AlreadyExistsException( - e, - "Table %s already exists in the database %s and catalog %s", - catalogTable, - catalogDatabase, - catalogName); - } - } - - return TableLoader.fromCatalog( - flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); - } - - private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { - Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); - return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java deleted file mode 100644 index f35bb577fbba..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java +++ /dev/null @@ -1,31 +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; - -import org.apache.iceberg.EnvironmentContext; -import org.apache.iceberg.flink.util.FlinkPackage; - -class FlinkEnvironmentContext { - private FlinkEnvironmentContext() {} - - public static void init() { - EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "flink"); - EnvironmentContext.put(EnvironmentContext.ENGINE_VERSION, FlinkPackage.version()); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java deleted file mode 100644 index f2244d5137a1..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java +++ /dev/null @@ -1,266 +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; - -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.BiFunction; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.FieldReferenceExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expression.Operation; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.NaNUtil; - -public class FlinkFilters { - private FlinkFilters() {} - - private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); - - private static final Map FILTERS = - ImmutableMap.builder() - .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) - .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) - .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) - .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) - .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) - .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) - .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) - .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) - .put(BuiltInFunctionDefinitions.AND, Operation.AND) - .put(BuiltInFunctionDefinitions.OR, Operation.OR) - .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) - .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) - .buildOrThrow(); - - /** - * Convert flink expression to iceberg expression. - * - *

    the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the - * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR - * GT_EQ), the IN will be converted to OR, so we do not add the conversion here - * - * @param flinkExpression the flink expression - * @return the iceberg expression - */ - public static Optional convert( - org.apache.flink.table.expressions.Expression flinkExpression) { - if (!(flinkExpression instanceof CallExpression)) { - return Optional.empty(); - } - - CallExpression call = (CallExpression) flinkExpression; - Operation op = FILTERS.get(call.getFunctionDefinition()); - if (op != null) { - switch (op) { - case IS_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(Expressions::isNull); - - case NOT_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(Expressions::notNull); - - case LT: - return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call); - - case LT_EQ: - return convertFieldAndLiteral( - Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call); - - case GT: - return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call); - - case GT_EQ: - return convertFieldAndLiteral( - Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call); - - case EQ: - return convertFieldAndLiteral( - (ref, lit) -> { - if (NaNUtil.isNaN(lit)) { - return Expressions.isNaN(ref); - } else { - return Expressions.equal(ref, lit); - } - }, - call); - - case NOT_EQ: - return convertFieldAndLiteral( - (ref, lit) -> { - if (NaNUtil.isNaN(lit)) { - return Expressions.notNaN(ref); - } else { - return Expressions.notEqual(ref, lit); - } - }, - call); - - case NOT: - return onlyChildAs(call, CallExpression.class) - .flatMap(FlinkFilters::convert) - .map(Expressions::not); - - case AND: - return convertLogicExpression(Expressions::and, call); - - case OR: - return convertLogicExpression(Expressions::or, call); - - case STARTS_WITH: - return convertLike(call); - } - } - - return Optional.empty(); - } - - private static Optional onlyChildAs( - CallExpression call, Class expectedChildClass) { - List children = call.getResolvedChildren(); - if (children.size() != 1) { - return Optional.empty(); - } - - ResolvedExpression child = children.get(0); - if (!expectedChildClass.isInstance(child)) { - return Optional.empty(); - } - - return Optional.of(expectedChildClass.cast(child)); - } - - private static Optional convertLike(CallExpression call) { - List args = call.getResolvedChildren(); - if (args.size() != 2) { - return Optional.empty(); - } - - org.apache.flink.table.expressions.Expression left = args.get(0); - org.apache.flink.table.expressions.Expression right = args.get(1); - - if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - return convertLiteral((ValueLiteralExpression) right) - .flatMap( - lit -> { - if (lit instanceof String) { - String pattern = (String) lit; - Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); - // exclude special char of LIKE - // '_' is the wildcard of the SQL LIKE - if (!pattern.contains("_") && matcher.matches()) { - return Optional.of(Expressions.startsWith(name, matcher.group(1))); - } - } - - return Optional.empty(); - }); - } - - return Optional.empty(); - } - - private static Optional convertLogicExpression( - BiFunction function, CallExpression call) { - List args = call.getResolvedChildren(); - if (args == null || args.size() != 2) { - return Optional.empty(); - } - - Optional left = convert(args.get(0)); - Optional right = convert(args.get(1)); - if (left.isPresent() && right.isPresent()) { - return Optional.of(function.apply(left.get(), right.get())); - } - - return Optional.empty(); - } - - private static Optional convertLiteral(ValueLiteralExpression expression) { - Optional value = - expression.getValueAs( - expression.getOutputDataType().getLogicalType().getDefaultConversion()); - return value.map( - o -> { - if (o instanceof LocalDateTime) { - return DateTimeUtil.microsFromTimestamp((LocalDateTime) o); - } else if (o instanceof Instant) { - return DateTimeUtil.microsFromInstant((Instant) o); - } else if (o instanceof LocalTime) { - return DateTimeUtil.microsFromTime((LocalTime) o); - } else if (o instanceof LocalDate) { - return DateTimeUtil.daysFromDate((LocalDate) o); - } - - return o; - }); - } - - private static Optional convertFieldAndLiteral( - BiFunction expr, CallExpression call) { - return convertFieldAndLiteral(expr, expr, call); - } - - private static Optional convertFieldAndLiteral( - BiFunction convertLR, - BiFunction convertRL, - CallExpression call) { - List args = call.getResolvedChildren(); - if (args.size() != 2) { - return Optional.empty(); - } - - org.apache.flink.table.expressions.Expression left = args.get(0); - org.apache.flink.table.expressions.Expression right = args.get(1); - - if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - Optional lit = convertLiteral((ValueLiteralExpression) right); - if (lit.isPresent()) { - return Optional.of(convertLR.apply(name, lit.get())); - } - } else if (left instanceof ValueLiteralExpression - && right instanceof FieldReferenceExpression) { - Optional lit = convertLiteral((ValueLiteralExpression) left); - String name = ((FieldReferenceExpression) right).getName(); - if (lit.isPresent()) { - return Optional.of(convertRL.apply(name, lit.get())); - } - } - - return Optional.empty(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java deleted file mode 100644 index 767d4497ac91..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java +++ /dev/null @@ -1,50 +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; - -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.FixupTypes; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -/** - * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, - * which may not be correct. - */ -class FlinkFixupTypes extends FixupTypes { - - private FlinkFixupTypes(Schema referenceSchema) { - super(referenceSchema); - } - - static Schema fixup(Schema schema, Schema referenceSchema) { - return new Schema( - TypeUtil.visit(schema, new FlinkFixupTypes(referenceSchema)).asStructType().fields()); - } - - @Override - protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { - if (type instanceof Types.FixedType) { - int length = ((Types.FixedType) type).length(); - return source.typeId() == Type.TypeID.UUID && length == 16; - } - return false; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java deleted file mode 100644 index 804a956ec9b9..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java +++ /dev/null @@ -1,213 +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; - -import java.time.Duration; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; - -public class FlinkReadConf { - - private final FlinkConfParser confParser; - - public FlinkReadConf( - Table table, Map readOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(table, readOptions, readableConfig); - } - - public Long snapshotId() { - return confParser.longConf().option(FlinkReadOptions.SNAPSHOT_ID.key()).parseOptional(); - } - - public String tag() { - return confParser.stringConf().option(FlinkReadOptions.TAG.key()).parseOptional(); - } - - public String startTag() { - return confParser.stringConf().option(FlinkReadOptions.START_TAG.key()).parseOptional(); - } - - public String endTag() { - return confParser.stringConf().option(FlinkReadOptions.END_TAG.key()).parseOptional(); - } - - public String branch() { - return confParser.stringConf().option(FlinkReadOptions.BRANCH.key()).parseOptional(); - } - - public boolean caseSensitive() { - return confParser - .booleanConf() - .option(FlinkReadOptions.CASE_SENSITIVE) - .flinkConfig(FlinkReadOptions.CASE_SENSITIVE_OPTION) - .defaultValue(FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue()) - .parse(); - } - - public Long asOfTimestamp() { - return confParser.longConf().option(FlinkReadOptions.AS_OF_TIMESTAMP.key()).parseOptional(); - } - - public StreamingStartingStrategy startingStrategy() { - return confParser - .enumConfParser(StreamingStartingStrategy.class) - .option(FlinkReadOptions.STARTING_STRATEGY) - .flinkConfig(FlinkReadOptions.STARTING_STRATEGY_OPTION) - .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .parse(); - } - - public Long startSnapshotTimestamp() { - return confParser - .longConf() - .option(FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key()) - .parseOptional(); - } - - public Long startSnapshotId() { - return confParser.longConf().option(FlinkReadOptions.START_SNAPSHOT_ID.key()).parseOptional(); - } - - public Long endSnapshotId() { - return confParser.longConf().option(FlinkReadOptions.END_SNAPSHOT_ID.key()).parseOptional(); - } - - public long splitSize() { - return confParser - .longConf() - .option(FlinkReadOptions.SPLIT_SIZE) - .flinkConfig(FlinkReadOptions.SPLIT_SIZE_OPTION) - .tableProperty(TableProperties.SPLIT_SIZE) - .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) - .parse(); - } - - public int splitLookback() { - return confParser - .intConf() - .option(FlinkReadOptions.SPLIT_LOOKBACK) - .flinkConfig(FlinkReadOptions.SPLIT_LOOKBACK_OPTION) - .tableProperty(TableProperties.SPLIT_LOOKBACK) - .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) - .parse(); - } - - public long splitFileOpenCost() { - return confParser - .longConf() - .option(FlinkReadOptions.SPLIT_FILE_OPEN_COST) - .flinkConfig(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION) - .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) - .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) - .parse(); - } - - public boolean streaming() { - return confParser - .booleanConf() - .option(FlinkReadOptions.STREAMING) - .flinkConfig(FlinkReadOptions.STREAMING_OPTION) - .defaultValue(FlinkReadOptions.STREAMING_OPTION.defaultValue()) - .parse(); - } - - public Duration monitorInterval() { - String duration = - confParser - .stringConf() - .option(FlinkReadOptions.MONITOR_INTERVAL) - .flinkConfig(FlinkReadOptions.MONITOR_INTERVAL_OPTION) - .defaultValue(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()) - .parse(); - - return TimeUtils.parseDuration(duration); - } - - public boolean includeColumnStats() { - return confParser - .booleanConf() - .option(FlinkReadOptions.INCLUDE_COLUMN_STATS) - .flinkConfig(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION) - .defaultValue(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue()) - .parse(); - } - - public int maxPlanningSnapshotCount() { - return confParser - .intConf() - .option(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT) - .flinkConfig(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION) - .defaultValue(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue()) - .parse(); - } - - public String nameMapping() { - return confParser.stringConf().option(TableProperties.DEFAULT_NAME_MAPPING).parseOptional(); - } - - public long limit() { - return confParser - .longConf() - .option(FlinkReadOptions.LIMIT) - .flinkConfig(FlinkReadOptions.LIMIT_OPTION) - .defaultValue(FlinkReadOptions.LIMIT_OPTION.defaultValue()) - .parse(); - } - - public int workerPoolSize() { - return confParser - .intConf() - .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) - .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) - .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) - .parse(); - } - - public int maxAllowedPlanningFailures() { - return confParser - .intConf() - .option(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES) - .flinkConfig(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION) - .defaultValue(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue()) - .parse(); - } - - public String watermarkColumn() { - return confParser - .stringConf() - .option(FlinkReadOptions.WATERMARK_COLUMN) - .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_OPTION) - .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue()) - .parseOptional(); - } - - public TimeUnit watermarkColumnTimeUnit() { - return confParser - .enumConfParser(TimeUnit.class) - .option(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT) - .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION) - .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue()) - .parse(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java deleted file mode 100644 index 1bbd88146c8f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java +++ /dev/null @@ -1,123 +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; - -import java.util.concurrent.TimeUnit; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; - -/** Flink source read options */ -public class FlinkReadOptions { - private static final String PREFIX = "connector.iceberg."; - - private FlinkReadOptions() {} - - public static final ConfigOption SNAPSHOT_ID = - ConfigOptions.key("snapshot-id").longType().defaultValue(null); - - public static final ConfigOption TAG = - ConfigOptions.key("tag").stringType().defaultValue(null); - - public static final ConfigOption BRANCH = - ConfigOptions.key("branch").stringType().defaultValue(null); - - public static final ConfigOption START_TAG = - ConfigOptions.key("start-tag").stringType().defaultValue(null); - - public static final ConfigOption END_TAG = - ConfigOptions.key("end-tag").stringType().defaultValue(null); - - public static final String CASE_SENSITIVE = "case-sensitive"; - public static final ConfigOption CASE_SENSITIVE_OPTION = - ConfigOptions.key(PREFIX + CASE_SENSITIVE).booleanType().defaultValue(false); - - public static final ConfigOption AS_OF_TIMESTAMP = - ConfigOptions.key("as-of-timestamp").longType().defaultValue(null); - - public static final String STARTING_STRATEGY = "starting-strategy"; - public static final ConfigOption STARTING_STRATEGY_OPTION = - ConfigOptions.key(PREFIX + STARTING_STRATEGY) - .enumType(StreamingStartingStrategy.class) - .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT); - - public static final ConfigOption START_SNAPSHOT_TIMESTAMP = - ConfigOptions.key("start-snapshot-timestamp").longType().defaultValue(null); - - public static final ConfigOption START_SNAPSHOT_ID = - ConfigOptions.key("start-snapshot-id").longType().defaultValue(null); - - public static final ConfigOption END_SNAPSHOT_ID = - ConfigOptions.key("end-snapshot-id").longType().defaultValue(null); - - public static final String SPLIT_SIZE = "split-size"; - public static final ConfigOption SPLIT_SIZE_OPTION = - ConfigOptions.key(PREFIX + SPLIT_SIZE) - .longType() - .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT); - - public static final String SPLIT_LOOKBACK = "split-lookback"; - public static final ConfigOption SPLIT_LOOKBACK_OPTION = - ConfigOptions.key(PREFIX + SPLIT_LOOKBACK) - .intType() - .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT); - - public static final String SPLIT_FILE_OPEN_COST = "split-file-open-cost"; - public static final ConfigOption SPLIT_FILE_OPEN_COST_OPTION = - ConfigOptions.key(PREFIX + SPLIT_FILE_OPEN_COST) - .longType() - .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); - - public static final String STREAMING = "streaming"; - public static final ConfigOption STREAMING_OPTION = - ConfigOptions.key(PREFIX + STREAMING).booleanType().defaultValue(false); - - public static final String MONITOR_INTERVAL = "monitor-interval"; - public static final ConfigOption MONITOR_INTERVAL_OPTION = - ConfigOptions.key(PREFIX + MONITOR_INTERVAL).stringType().defaultValue("60s"); - - public static final String INCLUDE_COLUMN_STATS = "include-column-stats"; - public static final ConfigOption INCLUDE_COLUMN_STATS_OPTION = - ConfigOptions.key(PREFIX + INCLUDE_COLUMN_STATS).booleanType().defaultValue(false); - - public static final String MAX_PLANNING_SNAPSHOT_COUNT = "max-planning-snapshot-count"; - public static final ConfigOption MAX_PLANNING_SNAPSHOT_COUNT_OPTION = - ConfigOptions.key(PREFIX + MAX_PLANNING_SNAPSHOT_COUNT) - .intType() - .defaultValue(Integer.MAX_VALUE); - - public static final String LIMIT = "limit"; - public static final ConfigOption LIMIT_OPTION = - ConfigOptions.key(PREFIX + LIMIT).longType().defaultValue(-1L); - - public static final String MAX_ALLOWED_PLANNING_FAILURES = "max-allowed-planning-failures"; - public static final ConfigOption MAX_ALLOWED_PLANNING_FAILURES_OPTION = - ConfigOptions.key(PREFIX + MAX_ALLOWED_PLANNING_FAILURES).intType().defaultValue(3); - - public static final String WATERMARK_COLUMN = "watermark-column"; - public static final ConfigOption WATERMARK_COLUMN_OPTION = - ConfigOptions.key(PREFIX + WATERMARK_COLUMN).stringType().noDefaultValue(); - - public static final String WATERMARK_COLUMN_TIME_UNIT = "watermark-column-time-unit"; - public static final ConfigOption WATERMARK_COLUMN_TIME_UNIT_OPTION = - ConfigOptions.key(PREFIX + WATERMARK_COLUMN_TIME_UNIT) - .enumType(TimeUnit.class) - .defaultValue(TimeUnit.MICROSECONDS); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java deleted file mode 100644 index 25725639c330..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.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; - -import java.util.List; -import java.util.Set; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.Schema; -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.Sets; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -/** - * Converter between Flink types and Iceberg type. The conversion is not a 1:1 mapping that not - * allows back-and-forth conversion. So some information might get lost during the back-and-forth - * conversion. - * - *

    This inconsistent types: - * - *

      - *
    • map Iceberg UUID type to Flink BinaryType(16) - *
    • map Flink VarCharType(_) and CharType(_) to Iceberg String type - *
    • map Flink VarBinaryType(_) to Iceberg Binary type - *
    • map Flink TimeType(_) to Iceberg Time type (microseconds) - *
    • map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds) - *
    • map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds) - *
    • map Flink MultiSetType to Iceberg Map type(element, int) - *
    - * - *

    - */ -public class FlinkSchemaUtil { - - private FlinkSchemaUtil() {} - - /** Convert the flink table schema to apache iceberg schema. */ - public static Schema convert(TableSchema schema) { - LogicalType schemaType = schema.toRowDataType().getLogicalType(); - Preconditions.checkArgument( - schemaType instanceof RowType, "Schema logical type should be RowType."); - - RowType root = (RowType) schemaType; - Type converted = root.accept(new FlinkTypeToType(root)); - - Schema iSchema = new Schema(converted.asStructType().fields()); - return freshIdentifierFieldIds(iSchema, schema); - } - - private static Schema freshIdentifierFieldIds(Schema iSchema, TableSchema schema) { - // Locate the identifier field id list. - Set identifierFieldIds = Sets.newHashSet(); - if (schema.getPrimaryKey().isPresent()) { - for (String column : schema.getPrimaryKey().get().getColumns()) { - Types.NestedField field = iSchema.findField(column); - Preconditions.checkNotNull( - field, - "Cannot find field ID for the primary key column %s in schema %s", - column, - iSchema); - identifierFieldIds.add(field.fieldId()); - } - } - - return new Schema(iSchema.schemaId(), iSchema.asStruct().fields(), identifierFieldIds); - } - - /** - * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema. - * - *

    This conversion does not assign new ids; it uses ids from the base schema. - * - *

    Data types, field order, and nullability will match the Flink type. This conversion may - * return a schema that is not compatible with base schema. - * - * @param baseSchema a Schema on which conversion is based - * @param flinkSchema a Flink TableSchema - * @return the equivalent Schema - * @throws IllegalArgumentException if the type cannot be converted or there are missing ids - */ - public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { - // convert to a type with fresh ids - Types.StructType struct = convert(flinkSchema).asStruct(); - // reassign ids to match the base schema - Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema); - // reassign doc to match the base schema - schema = TypeUtil.reassignDoc(schema, baseSchema); - - // fix types that can't be represented in Flink (UUID) - Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); - return freshIdentifierFieldIds(fixedSchema, flinkSchema); - } - - /** - * Convert a {@link Schema} to a {@link RowType Flink type}. - * - * @param schema a Schema - * @return the equivalent Flink type - * @throws IllegalArgumentException if the type cannot be converted to Flink - */ - public static RowType convert(Schema schema) { - return (RowType) TypeUtil.visit(schema, new TypeToFlinkType()); - } - - /** - * Convert a {@link Type} to a {@link LogicalType Flink type}. - * - * @param type a Type - * @return the equivalent Flink type - * @throws IllegalArgumentException if the type cannot be converted to Flink - */ - public static LogicalType convert(Type type) { - return TypeUtil.visit(type, new TypeToFlinkType()); - } - - /** - * Convert a {@link RowType} to a {@link TableSchema}. - * - * @param rowType a RowType - * @return Flink TableSchema - */ - public static TableSchema toSchema(RowType rowType) { - TableSchema.Builder builder = TableSchema.builder(); - for (RowType.RowField field : rowType.getFields()) { - builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); - } - return builder.build(); - } - - /** - * Convert a {@link Schema} to a {@link TableSchema}. - * - * @param schema iceberg schema to convert. - * @return Flink TableSchema. - */ - public static TableSchema toSchema(Schema schema) { - TableSchema.Builder builder = TableSchema.builder(); - - // Add columns. - for (RowType.RowField field : convert(schema).getFields()) { - builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); - } - - // Add primary key. - Set identifierFieldIds = schema.identifierFieldIds(); - if (!identifierFieldIds.isEmpty()) { - List columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size()); - for (Integer identifierFieldId : identifierFieldIds) { - String columnName = schema.findColumnName(identifierFieldId); - Preconditions.checkNotNull( - columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema); - - columns.add(columnName); - } - builder.primaryKey(columns.toArray(new String[0])); - } - - return builder.build(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java deleted file mode 100644 index 5fbd84909d69..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java +++ /dev/null @@ -1,49 +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; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Evaluator; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.types.Types; - -public class FlinkSourceFilter implements FilterFunction { - - private final RowType rowType; - private final Evaluator evaluator; - private final Types.StructType struct; - private volatile RowDataWrapper wrapper; - - public FlinkSourceFilter(Schema schema, Expression expr, boolean caseSensitive) { - this.rowType = FlinkSchemaUtil.convert(schema); - this.struct = schema.asStruct(); - this.evaluator = new Evaluator(struct, expr, caseSensitive); - } - - @Override - public boolean filter(RowData value) { - if (wrapper == null) { - this.wrapper = new RowDataWrapper(rowType, struct); - } - return evaluator.eval(wrapper.wrap(value)); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java deleted file mode 100644 index 6f8bfef2ef44..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java +++ /dev/null @@ -1,199 +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; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -class FlinkTypeToType extends FlinkTypeVisitor { - - private final RowType root; - private int nextId; - - FlinkTypeToType(RowType root) { - this.root = root; - // the root struct's fields use the first ids - this.nextId = root.getFieldCount(); - } - - private int getNextId() { - int next = nextId; - nextId += 1; - return next; - } - - @Override - public Type visit(CharType charType) { - return Types.StringType.get(); - } - - @Override - public Type visit(VarCharType varCharType) { - return Types.StringType.get(); - } - - @Override - public Type visit(BooleanType booleanType) { - return Types.BooleanType.get(); - } - - @Override - public Type visit(BinaryType binaryType) { - return Types.FixedType.ofLength(binaryType.getLength()); - } - - @Override - public Type visit(VarBinaryType varBinaryType) { - return Types.BinaryType.get(); - } - - @Override - public Type visit(DecimalType decimalType) { - return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); - } - - @Override - public Type visit(TinyIntType tinyIntType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(SmallIntType smallIntType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(IntType intType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(BigIntType bigIntType) { - return Types.LongType.get(); - } - - @Override - public Type visit(FloatType floatType) { - return Types.FloatType.get(); - } - - @Override - public Type visit(DoubleType doubleType) { - return Types.DoubleType.get(); - } - - @Override - public Type visit(DateType dateType) { - return Types.DateType.get(); - } - - @Override - public Type visit(TimeType timeType) { - return Types.TimeType.get(); - } - - @Override - public Type visit(TimestampType timestampType) { - return Types.TimestampType.withoutZone(); - } - - @Override - public Type visit(LocalZonedTimestampType localZonedTimestampType) { - return Types.TimestampType.withZone(); - } - - @Override - public Type visit(ArrayType arrayType) { - Type elementType = arrayType.getElementType().accept(this); - if (arrayType.getElementType().isNullable()) { - return Types.ListType.ofOptional(getNextId(), elementType); - } else { - return Types.ListType.ofRequired(getNextId(), elementType); - } - } - - @Override - public Type visit(MultisetType multisetType) { - Type elementType = multisetType.getElementType().accept(this); - return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get()); - } - - @Override - public Type visit(MapType mapType) { - // keys in map are not allowed to be null. - Type keyType = mapType.getKeyType().accept(this); - Type valueType = mapType.getValueType().accept(this); - if (mapType.getValueType().isNullable()) { - return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); - } else { - return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); - } - } - - @Override - @SuppressWarnings("ReferenceEquality") - public Type visit(RowType rowType) { - List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); - boolean isRoot = root == rowType; - - List types = - rowType.getFields().stream() - .map(f -> f.getType().accept(this)) - .collect(Collectors.toList()); - - for (int i = 0; i < rowType.getFieldCount(); i++) { - int id = isRoot ? i : getNextId(); - - RowType.RowField field = rowType.getFields().get(i); - String name = field.getName(); - String comment = field.getDescription().orElse(null); - - if (field.getType().isNullable()) { - newFields.add(Types.NestedField.optional(id, name, types.get(i), comment)); - } else { - newFields.add(Types.NestedField.required(id, name, types.get(i), comment)); - } - } - - return Types.StructType.of(newFields); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java deleted file mode 100644 index f3de2416088c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java +++ /dev/null @@ -1,80 +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; - -import org.apache.flink.table.types.logical.DayTimeIntervalType; -import org.apache.flink.table.types.logical.DistinctType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeVisitor; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RawType; -import org.apache.flink.table.types.logical.StructuredType; -import org.apache.flink.table.types.logical.SymbolType; -import org.apache.flink.table.types.logical.YearMonthIntervalType; -import org.apache.flink.table.types.logical.ZonedTimestampType; - -public abstract class FlinkTypeVisitor implements LogicalTypeVisitor { - - // ------------------------- Unsupported types ------------------------------ - - @Override - public T visit(ZonedTimestampType zonedTimestampType) { - throw new UnsupportedOperationException("Unsupported ZonedTimestampType."); - } - - @Override - public T visit(YearMonthIntervalType yearMonthIntervalType) { - throw new UnsupportedOperationException("Unsupported YearMonthIntervalType."); - } - - @Override - public T visit(DayTimeIntervalType dayTimeIntervalType) { - throw new UnsupportedOperationException("Unsupported DayTimeIntervalType."); - } - - @Override - public T visit(DistinctType distinctType) { - throw new UnsupportedOperationException("Unsupported DistinctType."); - } - - @Override - public T visit(StructuredType structuredType) { - throw new UnsupportedOperationException("Unsupported StructuredType."); - } - - @Override - public T visit(NullType nullType) { - throw new UnsupportedOperationException("Unsupported NullType."); - } - - @Override - public T visit(RawType rawType) { - throw new UnsupportedOperationException("Unsupported RawType."); - } - - @Override - public T visit(SymbolType symbolType) { - throw new UnsupportedOperationException("Unsupported SymbolType."); - } - - @Override - public T visit(LogicalType other) { - throw new UnsupportedOperationException("Unsupported type: " + other); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java deleted file mode 100644 index ca7b1120bc81..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ /dev/null @@ -1,205 +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; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; - -/** - * A class for common Iceberg configs for Flink writes. - * - *

    If a config is set at multiple levels, the following order of precedence is used (top to - * bottom): - * - *

      - *
    1. Write options - *
    2. flink ReadableConfig - *
    3. Table metadata - *
    - * - * The most specific value is set in write options and takes precedence over all other configs. If - * no write option is provided, this class checks the flink configuration for any overrides. If no - * applicable value is found in the write options, this class uses the table metadata. - * - *

    Note this class is NOT meant to be serialized. - */ -public class FlinkWriteConf { - - private final FlinkConfParser confParser; - - public FlinkWriteConf( - Table table, Map writeOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); - } - - public boolean overwriteMode() { - return confParser - .booleanConf() - .option(FlinkWriteOptions.OVERWRITE_MODE.key()) - .flinkConfig(FlinkWriteOptions.OVERWRITE_MODE) - .defaultValue(FlinkWriteOptions.OVERWRITE_MODE.defaultValue()) - .parse(); - } - - public boolean upsertMode() { - return confParser - .booleanConf() - .option(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key()) - .flinkConfig(FlinkWriteOptions.WRITE_UPSERT_ENABLED) - .tableProperty(TableProperties.UPSERT_ENABLED) - .defaultValue(TableProperties.UPSERT_ENABLED_DEFAULT) - .parse(); - } - - public FileFormat dataFileFormat() { - String valueAsString = - confParser - .stringConf() - .option(FlinkWriteOptions.WRITE_FORMAT.key()) - .flinkConfig(FlinkWriteOptions.WRITE_FORMAT) - .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) - .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) - .parse(); - return FileFormat.fromString(valueAsString); - } - - public long targetDataFileSize() { - return confParser - .longConf() - .option(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES.key()) - .flinkConfig(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES) - .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) - .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) - .parse(); - } - - public String parquetCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.PARQUET_COMPRESSION) - .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) - .parse(); - } - - public String parquetCompressionLevel() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) - .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) - .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) - .parseOptional(); - } - - public String avroCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.AVRO_COMPRESSION) - .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) - .parse(); - } - - public String avroCompressionLevel() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) - .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) - .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) - .parseOptional(); - } - - public String orcCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.ORC_COMPRESSION) - .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) - .parse(); - } - - public String orcCompressionStrategy() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_STRATEGY.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_STRATEGY) - .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) - .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) - .parse(); - } - - public DistributionMode distributionMode() { - String modeName = - confParser - .stringConf() - .option(FlinkWriteOptions.DISTRIBUTION_MODE.key()) - .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE) - .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) - .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE) - .parse(); - return DistributionMode.fromName(modeName); - } - - public int workerPoolSize() { - return confParser - .intConf() - .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) - .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) - .parse(); - } - - public String branch() { - return confParser - .stringConf() - .option(FlinkWriteOptions.BRANCH.key()) - .defaultValue(FlinkWriteOptions.BRANCH.defaultValue()) - .parse(); - } - - public Integer writeParallelism() { - return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional(); - } - - /** - * NOTE: This may be removed or changed in a future release. This value specifies the interval for - * refreshing the table instances in sink writer subtasks. If not specified then the default - * behavior is to not refresh the table. - * - * @return the interval for refreshing the table in sink writer subtasks - */ - @Experimental - public Duration tableRefreshInterval() { - return confParser - .durationConf() - .option(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key()) - .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) - .parseOptional(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java deleted file mode 100644 index df73f2e09cac..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.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; - -import java.time.Duration; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.iceberg.SnapshotRef; - -/** Flink sink write options */ -public class FlinkWriteOptions { - - private FlinkWriteOptions() {} - - // File format for write operations(default: Table write.format.default ) - public static final ConfigOption WRITE_FORMAT = - ConfigOptions.key("write-format").stringType().noDefaultValue(); - - // Overrides this table's write.target-file-size-bytes - public static final ConfigOption TARGET_FILE_SIZE_BYTES = - ConfigOptions.key("target-file-size-bytes").longType().noDefaultValue(); - - // Overrides this table's write..compression-codec - public static final ConfigOption COMPRESSION_CODEC = - ConfigOptions.key("compression-codec").stringType().noDefaultValue(); - - // Overrides this table's write..compression-level - public static final ConfigOption COMPRESSION_LEVEL = - ConfigOptions.key("compression-level").stringType().noDefaultValue(); - - // Overrides this table's write..compression-strategy - public static final ConfigOption COMPRESSION_STRATEGY = - ConfigOptions.key("compression-strategy").stringType().noDefaultValue(); - - // Overrides this table's write.upsert.enabled - public static final ConfigOption WRITE_UPSERT_ENABLED = - ConfigOptions.key("upsert-enabled").booleanType().noDefaultValue(); - - public static final ConfigOption OVERWRITE_MODE = - ConfigOptions.key("overwrite-enabled").booleanType().defaultValue(false); - - // Overrides the table's write.distribution-mode - public static final ConfigOption DISTRIBUTION_MODE = - ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); - - // Branch to write to - public static final ConfigOption BRANCH = - ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); - - public static final ConfigOption WRITE_PARALLELISM = - ConfigOptions.key("write-parallelism").intType().noDefaultValue(); - - @Experimental - public static final ConfigOption TABLE_REFRESH_INTERVAL = - ConfigOptions.key("table-refresh-interval").durationType().noDefaultValue(); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java deleted file mode 100644 index 1b9268569d9a..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java +++ /dev/null @@ -1,121 +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; - -import java.util.List; -import java.util.Map; -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.api.constraints.UniqueConstraint; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.sink.DataStreamSinkProvider; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; -import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.flink.sink.FlinkSink; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; - -public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { - private final TableLoader tableLoader; - private final TableSchema tableSchema; - private final ReadableConfig readableConfig; - private final Map writeProps; - - private boolean overwrite = false; - - private IcebergTableSink(IcebergTableSink toCopy) { - this.tableLoader = toCopy.tableLoader; - this.tableSchema = toCopy.tableSchema; - this.overwrite = toCopy.overwrite; - this.readableConfig = toCopy.readableConfig; - this.writeProps = toCopy.writeProps; - } - - public IcebergTableSink( - TableLoader tableLoader, - TableSchema tableSchema, - ReadableConfig readableConfig, - Map writeProps) { - this.tableLoader = tableLoader; - this.tableSchema = tableSchema; - this.readableConfig = readableConfig; - this.writeProps = writeProps; - } - - @Override - public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { - Preconditions.checkState( - !overwrite || context.isBounded(), - "Unbounded data stream doesn't support overwrite operation."); - - List equalityColumns = - tableSchema.getPrimaryKey().map(UniqueConstraint::getColumns).orElseGet(ImmutableList::of); - - return new DataStreamSinkProvider() { - @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(); - } - }; - } - - @Override - public void applyStaticPartition(Map partition) { - // The flink's PartitionFanoutWriter will handle the static partition write policy - // automatically. - } - - @Override - public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - builder.addContainedKind(kind); - } - return builder.build(); - } - - @Override - public DynamicTableSink copy() { - return new IcebergTableSink(this); - } - - @Override - public String asSummaryString() { - return "Iceberg table sink"; - } - - @Override - public void applyOverwrite(boolean newOverwrite) { - this.overwrite = newOverwrite; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java deleted file mode 100644 index d4cec7a3e80b..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java +++ /dev/null @@ -1,142 +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; - -import java.lang.reflect.Array; -import java.nio.ByteBuffer; -import java.time.LocalDateTime; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.UUIDUtil; - -public class RowDataWrapper implements StructLike { - - private final LogicalType[] types; - private final PositionalGetter[] getters; - private RowData rowData = null; - - public RowDataWrapper(RowType rowType, Types.StructType struct) { - int size = rowType.getFieldCount(); - - types = (LogicalType[]) Array.newInstance(LogicalType.class, size); - getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); - - for (int i = 0; i < size; i++) { - types[i] = rowType.getTypeAt(i); - getters[i] = buildGetter(types[i], struct.fields().get(i).type()); - } - } - - public RowDataWrapper wrap(RowData data) { - this.rowData = data; - return this; - } - - @Override - public int size() { - return types.length; - } - - @Override - public T get(int pos, Class javaClass) { - if (rowData.isNullAt(pos)) { - return null; - } else if (getters[pos] != null) { - return javaClass.cast(getters[pos].get(rowData, pos)); - } - - Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); - return javaClass.cast(value); - } - - @Override - public void set(int pos, T value) { - throw new UnsupportedOperationException( - "Could not set a field in the RowDataWrapper because rowData is read-only"); - } - - private interface PositionalGetter { - T get(RowData data, int pos); - } - - private static PositionalGetter buildGetter(LogicalType logicalType, Type type) { - switch (logicalType.getTypeRoot()) { - case TINYINT: - return (row, pos) -> (int) row.getByte(pos); - case SMALLINT: - return (row, pos) -> (int) row.getShort(pos); - case CHAR: - case VARCHAR: - return (row, pos) -> row.getString(pos).toString(); - - case BINARY: - case VARBINARY: - if (Type.TypeID.UUID == type.typeId()) { - return (row, pos) -> UUIDUtil.convert(row.getBinary(pos)); - } else { - return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); - } - - case DECIMAL: - DecimalType decimalType = (DecimalType) logicalType; - return (row, pos) -> - row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); - - case TIME_WITHOUT_TIME_ZONE: - // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds - // (Long). - return (row, pos) -> ((long) row.getInt(pos)) * 1_000; - - case TIMESTAMP_WITHOUT_TIME_ZONE: - TimestampType timestampType = (TimestampType) logicalType; - return (row, pos) -> { - LocalDateTime localDateTime = - row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); - return DateTimeUtil.microsFromTimestamp(localDateTime); - }; - - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; - return (row, pos) -> { - TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); - return timestampData.getMillisecond() * 1000 - + timestampData.getNanoOfMillisecond() / 1000; - }; - - case ROW: - RowType rowType = (RowType) logicalType; - Types.StructType structType = (Types.StructType) type; - - RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); - return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); - - default: - return null; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java deleted file mode 100644 index da509451fee7..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java +++ /dev/null @@ -1,159 +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; - -import java.io.Closeable; -import java.io.IOException; -import java.io.Serializable; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.hadoop.SerializableConfiguration; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -/** - * Serializable loader to load an Iceberg {@link Table}. Flink needs to get {@link Table} objects in - * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg - * table loader to get the {@link Table} object. - */ -public interface TableLoader extends Closeable, Serializable, Cloneable { - - void open(); - - boolean isOpen(); - - Table loadTable(); - - /** Clone a TableLoader */ - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - TableLoader clone(); - - static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { - return new CatalogTableLoader(catalogLoader, identifier); - } - - static TableLoader fromHadoopTable(String location) { - return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf()); - } - - static TableLoader fromHadoopTable(String location, Configuration hadoopConf) { - return new HadoopTableLoader(location, hadoopConf); - } - - class HadoopTableLoader implements TableLoader { - - private static final long serialVersionUID = 1L; - - private final String location; - private final SerializableConfiguration hadoopConf; - - private transient HadoopTables tables; - - private HadoopTableLoader(String location, Configuration conf) { - this.location = location; - this.hadoopConf = new SerializableConfiguration(conf); - } - - @Override - public void open() { - tables = new HadoopTables(hadoopConf.get()); - } - - @Override - public boolean isOpen() { - return tables != null; - } - - @Override - public Table loadTable() { - FlinkEnvironmentContext.init(); - return tables.load(location); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new HadoopTableLoader(location, new Configuration(hadoopConf.get())); - } - - @Override - public void close() {} - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("location", location).toString(); - } - } - - class CatalogTableLoader implements TableLoader { - - private static final long serialVersionUID = 1L; - - private final CatalogLoader catalogLoader; - private final String identifier; - - private transient Catalog catalog; - - private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { - this.catalogLoader = catalogLoader; - this.identifier = tableIdentifier.toString(); - } - - @Override - public void open() { - catalog = catalogLoader.loadCatalog(); - } - - @Override - public boolean isOpen() { - return catalog != null; - } - - @Override - public Table loadTable() { - FlinkEnvironmentContext.init(); - return catalog.loadTable(TableIdentifier.parse(identifier)); - } - - @Override - public void close() throws IOException { - if (catalog instanceof Closeable) { - ((Closeable) catalog).close(); - } - - catalog = null; - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier)); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("tableIdentifier", identifier) - .add("catalogLoader", catalogLoader) - .toString(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java deleted file mode 100644 index f8f1b74b1ceb..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java +++ /dev/null @@ -1,134 +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; - -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -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.TypeUtil; -import org.apache.iceberg.types.Types; - -class TypeToFlinkType extends TypeUtil.SchemaVisitor { - TypeToFlinkType() {} - - @Override - public LogicalType schema(Schema schema, LogicalType structType) { - return structType; - } - - @Override - public LogicalType struct(Types.StructType struct, List fieldResults) { - List fields = struct.fields(); - - List flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); - for (int i = 0; i < fields.size(); i += 1) { - Types.NestedField field = fields.get(i); - LogicalType type = fieldResults.get(i); - RowType.RowField flinkField = - new RowType.RowField(field.name(), type.copy(field.isOptional()), field.doc()); - flinkFields.add(flinkField); - } - - return new RowType(flinkFields); - } - - @Override - public LogicalType field(Types.NestedField field, LogicalType fieldResult) { - return fieldResult; - } - - @Override - public LogicalType list(Types.ListType list, LogicalType elementResult) { - return new ArrayType(elementResult.copy(list.isElementOptional())); - } - - @Override - public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) { - // keys in map are not allowed to be null. - return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional())); - } - - @Override - public LogicalType primitive(Type.PrimitiveType primitive) { - switch (primitive.typeId()) { - case BOOLEAN: - return new BooleanType(); - case INTEGER: - return new IntType(); - case LONG: - return new BigIntType(); - case FLOAT: - return new FloatType(); - case DOUBLE: - return new DoubleType(); - case DATE: - return new DateType(); - case TIME: - // For the type: Flink only support TimeType with default precision (second) now. The - // precision of time is - // not supported in Flink, so we can think of it as a simple time type directly. - // For the data: Flink uses int that support mills to represent time data, so it supports - // mills precision. - return new TimeType(); - case TIMESTAMP: - Types.TimestampType timestamp = (Types.TimestampType) primitive; - if (timestamp.shouldAdjustToUTC()) { - // MICROS - return new LocalZonedTimestampType(6); - } else { - // MICROS - return new TimestampType(6); - } - case STRING: - return new VarCharType(VarCharType.MAX_LENGTH); - case UUID: - // UUID length is 16 - return new BinaryType(16); - case FIXED: - Types.FixedType fixedType = (Types.FixedType) primitive; - return new BinaryType(fixedType.length()); - case BINARY: - return new VarBinaryType(VarBinaryType.MAX_LENGTH); - case DECIMAL: - Types.DecimalType decimal = (Types.DecimalType) primitive; - return new DecimalType(decimal.precision(), decimal.scale()); - default: - throw new UnsupportedOperationException( - "Cannot convert unknown type to Flink: " + primitive); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java deleted file mode 100644 index 06ac54617ae6..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java +++ /dev/null @@ -1,52 +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.actions; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.Table; - -public class Actions { - - public static final Configuration CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private StreamExecutionEnvironment env; - private Table table; - - private Actions(StreamExecutionEnvironment env, Table table) { - this.env = env; - this.table = table; - } - - public static Actions forTable(StreamExecutionEnvironment env, Table table) { - return new Actions(env, table); - } - - public static Actions forTable(Table table) { - return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table); - } - - public RewriteDataFilesAction rewriteDataFiles() { - return new RewriteDataFilesAction(env, table); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java deleted file mode 100644 index 9876bb3861c4..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java +++ /dev/null @@ -1,72 +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.actions; - -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.BaseRewriteDataFilesAction; -import org.apache.iceberg.flink.source.RowDataRewriter; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { - - private StreamExecutionEnvironment env; - private int maxParallelism; - - public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { - super(table); - this.env = env; - this.maxParallelism = env.getParallelism(); - } - - @Override - protected FileIO fileIO() { - return table().io(); - } - - @Override - protected List rewriteDataForTasks(List combinedScanTasks) { - int size = combinedScanTasks.size(); - int parallelism = Math.min(size, maxParallelism); - DataStream dataStream = env.fromCollection(combinedScanTasks); - RowDataRewriter rowDataRewriter = - new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); - try { - return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); - } catch (Exception e) { - throw new RuntimeException("Rewrite data file error.", e); - } - } - - @Override - protected RewriteDataFilesAction self() { - return this; - } - - public RewriteDataFilesAction maxParallelism(int parallelism) { - Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %s", parallelism); - this.maxParallelism = parallelism; - return this; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java deleted file mode 100644 index 8103224a0b6c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java +++ /dev/null @@ -1,75 +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 org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.Pair; - -public abstract class AvroWithFlinkSchemaVisitor - extends AvroWithPartnerByStructureVisitor { - - @Override - protected boolean isStringType(LogicalType logicalType) { - return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING); - } - - @Override - protected boolean isMapType(LogicalType logicalType) { - return logicalType instanceof MapType; - } - - @Override - protected LogicalType arrayElementType(LogicalType arrayType) { - Preconditions.checkArgument( - arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); - return ((ArrayType) arrayType).getElementType(); - } - - @Override - protected LogicalType mapKeyType(LogicalType mapType) { - Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); - return ((MapType) mapType).getKeyType(); - } - - @Override - protected LogicalType mapValueType(LogicalType mapType) { - Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); - return ((MapType) mapType).getValueType(); - } - - @Override - protected Pair fieldNameAndType(LogicalType structType, int pos) { - Preconditions.checkArgument( - structType instanceof RowType, "Invalid struct: %s is not a struct", structType); - RowType.RowField field = ((RowType) structType).getFields().get(pos); - return Pair.of(field.getName(), field.getType()); - } - - @Override - protected LogicalType nullType() { - return new NullType(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java deleted file mode 100644 index 86404959735a..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ /dev/null @@ -1,169 +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; - -public class FlinkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - @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.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java deleted file mode 100644 index 873e65783119..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java +++ /dev/null @@ -1,165 +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.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.Encoder; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.avro.MetricsAwareDatumWriter; -import org.apache.iceberg.avro.ValueWriter; -import org.apache.iceberg.avro.ValueWriters; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class FlinkAvroWriter implements MetricsAwareDatumWriter { - private final RowType rowType; - private ValueWriter writer = null; - - public FlinkAvroWriter(RowType rowType) { - this.rowType = rowType; - } - - @Override - @SuppressWarnings("unchecked") - public void setSchema(Schema schema) { - this.writer = - (ValueWriter) - AvroWithFlinkSchemaVisitor.visit(rowType, schema, new WriteBuilder()); - } - - @Override - public void write(RowData datum, Encoder out) throws IOException { - writer.write(datum, out); - } - - @Override - public Stream metrics() { - return writer.metrics(); - } - - private static class WriteBuilder extends AvroWithFlinkSchemaVisitor> { - @Override - public ValueWriter record( - LogicalType struct, Schema record, List names, List> fields) { - return FlinkValueWriters.row( - fields, - IntStream.range(0, names.size()) - .mapToObj(i -> fieldNameAndType(struct, i).second()) - .collect(Collectors.toList())); - } - - @Override - public ValueWriter union(LogicalType type, Schema union, List> options) { - Preconditions.checkArgument( - options.contains(ValueWriters.nulls()), - "Cannot create writer for non-option union: %s", - union); - Preconditions.checkArgument( - options.size() == 2, "Cannot create writer for non-option union: %s", union); - if (union.getTypes().get(0).getType() == Schema.Type.NULL) { - return ValueWriters.option(0, options.get(1)); - } else { - return ValueWriters.option(1, options.get(0)); - } - } - - @Override - public ValueWriter array(LogicalType sArray, Schema array, ValueWriter elementWriter) { - return FlinkValueWriters.array(elementWriter, arrayElementType(sArray)); - } - - @Override - public ValueWriter map(LogicalType sMap, Schema map, ValueWriter valueReader) { - return FlinkValueWriters.map( - FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); - } - - @Override - public ValueWriter map( - LogicalType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { - return FlinkValueWriters.arrayMap( - keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); - } - - @Override - public ValueWriter primitive(LogicalType type, Schema primitive) { - org.apache.avro.LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueWriters.ints(); - - case "time-micros": - return FlinkValueWriters.timeMicros(); - - case "timestamp-micros": - return FlinkValueWriters.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); - - case "uuid": - return ValueWriters.uuids(); - - default: - throw new IllegalArgumentException("Unsupported logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueWriters.nulls(); - case BOOLEAN: - return ValueWriters.booleans(); - case INT: - switch (type.getTypeRoot()) { - case TINYINT: - return ValueWriters.tinyints(); - case SMALLINT: - return ValueWriters.shorts(); - default: - return ValueWriters.ints(); - } - case LONG: - return ValueWriters.longs(); - case FLOAT: - return ValueWriters.floats(); - case DOUBLE: - return ValueWriters.doubles(); - case STRING: - return FlinkValueWriters.strings(); - case FIXED: - return ValueWriters.fixed(primitive.getFixedSize()); - case BYTES: - return ValueWriters.bytes(); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java deleted file mode 100644 index 65b9d44ad4b8..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java +++ /dev/null @@ -1,131 +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.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.orc.OrcRowReader; -import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; -import org.apache.iceberg.orc.OrcValueReader; -import org.apache.iceberg.orc.OrcValueReaders; -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.orc.TypeDescription; -import org.apache.orc.storage.ql.exec.vector.StructColumnVector; -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; - -public class FlinkOrcReader implements OrcRowReader { - private final OrcValueReader reader; - - public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) { - this(iSchema, readSchema, ImmutableMap.of()); - } - - public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map idToConstant) { - this.reader = - OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant)); - } - - @Override - public RowData read(VectorizedRowBatch batch, int row) { - return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - reader.setBatchContext(batchOffsetInFile); - } - - private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public OrcValueReader record( - Types.StructType iStruct, - TypeDescription record, - List names, - List> fields) { - return FlinkOrcReaders.struct(fields, iStruct, idToConstant); - } - - @Override - public OrcValueReader list( - Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { - return FlinkOrcReaders.array(elementReader); - } - - @Override - public OrcValueReader map( - Types.MapType iMap, - TypeDescription map, - OrcValueReader keyReader, - OrcValueReader valueReader) { - return FlinkOrcReaders.map(keyReader, valueReader); - } - - @Override - public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { - switch (iPrimitive.typeId()) { - case BOOLEAN: - return OrcValueReaders.booleans(); - case INTEGER: - return OrcValueReaders.ints(); - case LONG: - return OrcValueReaders.longs(); - case FLOAT: - return OrcValueReaders.floats(); - case DOUBLE: - return OrcValueReaders.doubles(); - case DATE: - return FlinkOrcReaders.dates(); - case TIME: - return FlinkOrcReaders.times(); - case TIMESTAMP: - Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; - if (timestampType.shouldAdjustToUTC()) { - return FlinkOrcReaders.timestampTzs(); - } else { - return FlinkOrcReaders.timestamps(); - } - case STRING: - return FlinkOrcReaders.strings(); - case UUID: - case FIXED: - case BINARY: - return OrcValueReaders.bytes(); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; - return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale()); - default: - throw new IllegalArgumentException( - String.format( - "Invalid iceberg type %s corresponding to ORC type %s", iPrimitive, primitive)); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java deleted file mode 100644 index 7a4a15c7e600..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java +++ /dev/null @@ -1,283 +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.math.BigDecimal; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.orc.OrcValueReader; -import org.apache.iceberg.orc.OrcValueReaders; -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.apache.iceberg.types.Types; -import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; -import org.apache.orc.storage.ql.exec.vector.ColumnVector; -import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; -import org.apache.orc.storage.ql.exec.vector.ListColumnVector; -import org.apache.orc.storage.ql.exec.vector.LongColumnVector; -import org.apache.orc.storage.ql.exec.vector.MapColumnVector; -import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; -import org.apache.orc.storage.serde2.io.HiveDecimalWritable; - -class FlinkOrcReaders { - private FlinkOrcReaders() {} - - static OrcValueReader strings() { - return StringReader.INSTANCE; - } - - static OrcValueReader dates() { - return DateReader.INSTANCE; - } - - static OrcValueReader decimals(int precision, int scale) { - if (precision <= 18) { - return new Decimal18Reader(precision, scale); - } else if (precision <= 38) { - return new Decimal38Reader(precision, scale); - } else { - throw new IllegalArgumentException("Invalid precision: " + precision); - } - } - - static OrcValueReader times() { - return TimeReader.INSTANCE; - } - - static OrcValueReader timestamps() { - return TimestampReader.INSTANCE; - } - - static OrcValueReader timestampTzs() { - return TimestampTzReader.INSTANCE; - } - - static OrcValueReader array(OrcValueReader elementReader) { - return new ArrayReader<>(elementReader); - } - - public static OrcValueReader map( - OrcValueReader keyReader, OrcValueReader valueReader) { - return new MapReader<>(keyReader, valueReader); - } - - public static OrcValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); - } - - private static class StringReader implements OrcValueReader { - private static final StringReader INSTANCE = new StringReader(); - - @Override - public StringData nonNullRead(ColumnVector vector, int row) { - BytesColumnVector bytesVector = (BytesColumnVector) vector; - return StringData.fromBytes( - bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); - } - } - - private static class DateReader implements OrcValueReader { - private static final DateReader INSTANCE = new DateReader(); - - @Override - public Integer nonNullRead(ColumnVector vector, int row) { - return (int) ((LongColumnVector) vector).vector[row]; - } - } - - private static class Decimal18Reader implements OrcValueReader { - private final int precision; - private final int scale; - - Decimal18Reader(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData nonNullRead(ColumnVector vector, int row) { - HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; - - // The hive ORC writer may will adjust the scale of decimal data. - Preconditions.checkArgument( - value.precision() <= precision, - "Cannot read value as decimal(%s,%s), too large: %s", - precision, - scale, - value); - - return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale); - } - } - - private static class Decimal38Reader implements OrcValueReader { - private final int precision; - private final int scale; - - Decimal38Reader(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData nonNullRead(ColumnVector vector, int row) { - BigDecimal value = - ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); - - Preconditions.checkArgument( - value.precision() <= precision, - "Cannot read value as decimal(%s,%s), too large: %s", - precision, - scale, - value); - - return DecimalData.fromBigDecimal(value, precision, scale); - } - } - - private static class TimeReader implements OrcValueReader { - private static final TimeReader INSTANCE = new TimeReader(); - - @Override - public Integer nonNullRead(ColumnVector vector, int row) { - long micros = ((LongColumnVector) vector).vector[row]; - // Flink only support time mills, just erase micros. - return (int) (micros / 1000); - } - } - - private static class TimestampReader implements OrcValueReader { - private static final TimestampReader INSTANCE = new TimestampReader(); - - @Override - public TimestampData nonNullRead(ColumnVector vector, int row) { - TimestampColumnVector tcv = (TimestampColumnVector) vector; - LocalDateTime localDate = - Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) - .atOffset(ZoneOffset.UTC) - .toLocalDateTime(); - return TimestampData.fromLocalDateTime(localDate); - } - } - - private static class TimestampTzReader implements OrcValueReader { - private static final TimestampTzReader INSTANCE = new TimestampTzReader(); - - @Override - public TimestampData nonNullRead(ColumnVector vector, int row) { - TimestampColumnVector tcv = (TimestampColumnVector) vector; - Instant instant = - Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) - .atOffset(ZoneOffset.UTC) - .toInstant(); - return TimestampData.fromInstant(instant); - } - } - - private static class ArrayReader implements OrcValueReader { - private final OrcValueReader elementReader; - - private ArrayReader(OrcValueReader elementReader) { - this.elementReader = elementReader; - } - - @Override - public ArrayData nonNullRead(ColumnVector vector, int row) { - ListColumnVector listVector = (ListColumnVector) vector; - int offset = (int) listVector.offsets[row]; - int length = (int) listVector.lengths[row]; - List elements = Lists.newArrayListWithExpectedSize(length); - for (int c = 0; c < length; ++c) { - elements.add(elementReader.read(listVector.child, offset + c)); - } - return new GenericArrayData(elements.toArray()); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - elementReader.setBatchContext(batchOffsetInFile); - } - } - - private static class MapReader implements OrcValueReader { - private final OrcValueReader keyReader; - private final OrcValueReader valueReader; - - private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData nonNullRead(ColumnVector vector, int row) { - MapColumnVector mapVector = (MapColumnVector) vector; - int offset = (int) mapVector.offsets[row]; - long length = mapVector.lengths[row]; - - Map map = Maps.newHashMap(); - for (int c = 0; c < length; c++) { - K key = keyReader.read(mapVector.keys, offset + c); - V value = valueReader.read(mapVector.values, offset + c); - map.put(key, value); - } - - return new GenericMapData(map); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - keyReader.setBatchContext(batchOffsetInFile); - valueReader.setBatchContext(batchOffsetInFile); - } - } - - private static class StructReader extends OrcValueReaders.StructReader { - private final int numFields; - - StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); - this.numFields = struct.fields().size(); - } - - @Override - protected RowData create() { - return new GenericRowData(numFields); - } - - @Override - protected void set(RowData struct, int pos, Object value) { - ((GenericRowData) struct).setField(pos, value); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java deleted file mode 100644 index 6a31accffd22..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java +++ /dev/null @@ -1,163 +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.util.Deque; -import java.util.List; -import java.util.stream.Stream; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.orc.GenericOrcWriters; -import org.apache.iceberg.orc.OrcRowWriter; -import org.apache.iceberg.orc.OrcValueWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -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.orc.storage.ql.exec.vector.VectorizedRowBatch; - -public class FlinkOrcWriter implements OrcRowWriter { - private final FlinkOrcWriters.RowDataWriter writer; - - private FlinkOrcWriter(RowType rowType, Schema iSchema) { - this.writer = - (FlinkOrcWriters.RowDataWriter) - FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder()); - } - - public static OrcRowWriter buildWriter(RowType rowType, Schema iSchema) { - return new FlinkOrcWriter(rowType, iSchema); - } - - @Override - public void write(RowData row, VectorizedRowBatch output) { - Preconditions.checkArgument(row != null, "value must not be null"); - writer.writeRow(row, output); - } - - @Override - public List> writers() { - return writer.writers(); - } - - @Override - public Stream> metrics() { - return writer.metrics(); - } - - private static class WriteBuilder extends FlinkSchemaVisitor> { - private final Deque fieldIds = Lists.newLinkedList(); - - private WriteBuilder() {} - - @Override - public void beforeField(Types.NestedField field) { - fieldIds.push(field.fieldId()); - } - - @Override - public void afterField(Types.NestedField field) { - fieldIds.pop(); - } - - @Override - public OrcValueWriter record( - Types.StructType iStruct, List> results, List fieldType) { - return FlinkOrcWriters.struct(results, fieldType); - } - - @Override - public OrcValueWriter map( - Types.MapType iMap, - OrcValueWriter key, - OrcValueWriter value, - LogicalType keyType, - LogicalType valueType) { - return FlinkOrcWriters.map(key, value, keyType, valueType); - } - - @Override - public OrcValueWriter list( - Types.ListType iList, OrcValueWriter element, LogicalType elementType) { - return FlinkOrcWriters.list(element, elementType); - } - - @Override - public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { - switch (iPrimitive.typeId()) { - case BOOLEAN: - return GenericOrcWriters.booleans(); - case INTEGER: - switch (flinkPrimitive.getTypeRoot()) { - case TINYINT: - return GenericOrcWriters.bytes(); - case SMALLINT: - return GenericOrcWriters.shorts(); - } - return GenericOrcWriters.ints(); - case LONG: - return GenericOrcWriters.longs(); - case FLOAT: - Preconditions.checkArgument( - fieldIds.peek() != null, - String.format( - "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " - + "information is not properly pushed during schema visiting.", - iPrimitive)); - return GenericOrcWriters.floats(fieldIds.peek()); - case DOUBLE: - Preconditions.checkArgument( - fieldIds.peek() != null, - String.format( - "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " - + "information is not properly pushed during schema visiting.", - iPrimitive)); - return GenericOrcWriters.doubles(fieldIds.peek()); - case DATE: - return FlinkOrcWriters.dates(); - case TIME: - return FlinkOrcWriters.times(); - case TIMESTAMP: - Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; - if (timestampType.shouldAdjustToUTC()) { - return FlinkOrcWriters.timestampTzs(); - } else { - return FlinkOrcWriters.timestamps(); - } - case STRING: - return FlinkOrcWriters.strings(); - case UUID: - case FIXED: - case BINARY: - return GenericOrcWriters.byteArrays(); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; - return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale()); - default: - throw new IllegalArgumentException( - String.format( - "Invalid iceberg type %s corresponding to Flink logical type %s", - iPrimitive, flinkPrimitive)); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java deleted file mode 100644 index da2f95cf822f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ /dev/null @@ -1,317 +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.time.Instant; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.stream.Stream; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.data.orc.GenericOrcWriters; -import org.apache.iceberg.orc.OrcValueWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.orc.storage.common.type.HiveDecimal; -import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; -import org.apache.orc.storage.ql.exec.vector.ColumnVector; -import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; -import org.apache.orc.storage.ql.exec.vector.ListColumnVector; -import org.apache.orc.storage.ql.exec.vector.LongColumnVector; -import org.apache.orc.storage.ql.exec.vector.MapColumnVector; -import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; - -class FlinkOrcWriters { - - private FlinkOrcWriters() {} - - static OrcValueWriter strings() { - return StringWriter.INSTANCE; - } - - static OrcValueWriter dates() { - return DateWriter.INSTANCE; - } - - static OrcValueWriter times() { - return TimeWriter.INSTANCE; - } - - static OrcValueWriter timestamps() { - return TimestampWriter.INSTANCE; - } - - static OrcValueWriter timestampTzs() { - return TimestampTzWriter.INSTANCE; - } - - static OrcValueWriter decimals(int precision, int scale) { - if (precision <= 18) { - return new Decimal18Writer(precision, scale); - } else if (precision <= 38) { - return new Decimal38Writer(precision, scale); - } else { - throw new IllegalArgumentException("Invalid precision: " + precision); - } - } - - static OrcValueWriter list( - OrcValueWriter elementWriter, LogicalType elementType) { - return new ListWriter<>(elementWriter, elementType); - } - - static OrcValueWriter map( - OrcValueWriter keyWriter, - OrcValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - return new MapWriter<>(keyWriter, valueWriter, keyType, valueType); - } - - static OrcValueWriter struct(List> writers, List types) { - return new RowDataWriter(writers, types); - } - - private static class StringWriter implements OrcValueWriter { - private static final StringWriter INSTANCE = new StringWriter(); - - @Override - public void nonNullWrite(int rowId, StringData data, ColumnVector output) { - byte[] value = data.toBytes(); - ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); - } - } - - private static class DateWriter implements OrcValueWriter { - private static final DateWriter INSTANCE = new DateWriter(); - - @Override - public void nonNullWrite(int rowId, Integer data, ColumnVector output) { - ((LongColumnVector) output).vector[rowId] = data; - } - } - - private static class TimeWriter implements OrcValueWriter { - private static final TimeWriter INSTANCE = new TimeWriter(); - - @Override - public void nonNullWrite(int rowId, Integer millis, ColumnVector output) { - // The time in flink is in millisecond, while the standard time in iceberg is microsecond. - // So we need to transform it to microsecond. - ((LongColumnVector) output).vector[rowId] = millis * 1000L; - } - } - - private static class TimestampWriter implements OrcValueWriter { - private static final TimestampWriter INSTANCE = new TimestampWriter(); - - @Override - public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { - TimestampColumnVector cv = (TimestampColumnVector) output; - cv.setIsUTC(true); - // millis - OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC); - cv.time[rowId] = - offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000; - // truncate nanos to only keep microsecond precision. - cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000; - } - } - - private static class TimestampTzWriter implements OrcValueWriter { - private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); - - @SuppressWarnings("JavaInstantGetSecondsGetNano") - @Override - public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { - TimestampColumnVector cv = (TimestampColumnVector) output; - // millis - Instant instant = data.toInstant(); - cv.time[rowId] = instant.toEpochMilli(); - // truncate nanos to only keep microsecond precision. - cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000; - } - } - - private static class Decimal18Writer implements OrcValueWriter { - private final int precision; - private final int scale; - - Decimal18Writer(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { - Preconditions.checkArgument( - scale == data.scale(), - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - data); - Preconditions.checkArgument( - data.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - data); - - ((DecimalColumnVector) output) - .vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale()); - } - } - - private static class Decimal38Writer implements OrcValueWriter { - private final int precision; - private final int scale; - - Decimal38Writer(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { - Preconditions.checkArgument( - scale == data.scale(), - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - data); - Preconditions.checkArgument( - data.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - data); - - ((DecimalColumnVector) output) - .vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false)); - } - } - - static class ListWriter implements OrcValueWriter { - private final OrcValueWriter elementWriter; - private final ArrayData.ElementGetter elementGetter; - - ListWriter(OrcValueWriter elementWriter, LogicalType elementType) { - this.elementWriter = elementWriter; - this.elementGetter = ArrayData.createElementGetter(elementType); - } - - @Override - @SuppressWarnings("unchecked") - public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { - ListColumnVector cv = (ListColumnVector) output; - cv.lengths[rowId] = data.size(); - cv.offsets[rowId] = cv.childCount; - cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); - // make sure the child is big enough. - growColumnVector(cv.child, cv.childCount); - - for (int e = 0; e < cv.lengths[rowId]; ++e) { - Object value = elementGetter.getElementOrNull(data, e); - elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child); - } - } - - @Override - public Stream> metrics() { - return elementWriter.metrics(); - } - } - - static class MapWriter implements OrcValueWriter { - private final OrcValueWriter keyWriter; - private final OrcValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - MapWriter( - OrcValueWriter keyWriter, - OrcValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.valueWriter = valueWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void nonNullWrite(int rowId, MapData data, ColumnVector output) { - MapColumnVector cv = (MapColumnVector) output; - ArrayData keyArray = data.keyArray(); - ArrayData valArray = data.valueArray(); - - // record the length and start of the list elements - cv.lengths[rowId] = data.size(); - cv.offsets[rowId] = cv.childCount; - cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); - // make sure the child is big enough - growColumnVector(cv.keys, cv.childCount); - growColumnVector(cv.values, cv.childCount); - // Add each element - for (int e = 0; e < cv.lengths[rowId]; ++e) { - int pos = (int) (e + cv.offsets[rowId]); - keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys); - valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values); - } - } - - @Override - public Stream> metrics() { - return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); - } - } - - static class RowDataWriter extends GenericOrcWriters.StructWriter { - private final List fieldGetters; - - RowDataWriter(List> writers, List types) { - super(writers); - - this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); - for (int i = 0; i < types.size(); i++) { - fieldGetters.add(RowData.createFieldGetter(types.get(i), i)); - } - } - - @Override - protected Object get(RowData struct, int index) { - return fieldGetters.get(index).getFieldOrNull(struct); - } - } - - private static void growColumnVector(ColumnVector cv, int requestedSize) { - if (cv.isNull.length < requestedSize) { - // Use growth factor of 3 to avoid frequent array allocations - cv.ensureSize(requestedSize * 3, true); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java deleted file mode 100644 index ab7b1174c9f3..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ /dev/null @@ -1,832 +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.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.ZoneOffset; -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.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.MetadataColumns; -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.TypeWithSchemaVisitor; -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.types.Types; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -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; - -public class FlinkParquetReaders { - private FlinkParquetReaders() {} - - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema) { - return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); - } - - @SuppressWarnings("unchecked") - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); - } - - private static class ReadBuilder extends TypeWithSchemaVisitor> { - private final MessageType type; - private final Map idToConstant; - - ReadBuilder(MessageType type, Map idToConstant) { - this.type = type; - this.idToConstant = idToConstant; - } - - @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); - if (fieldReaders.get(i) != null) { - int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; - if (fieldType.getId() != null) { - int id = fieldType.getId().intValue(); - readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - typesById.put(id, fieldType); - if (idToConstant.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 (idToConstant.containsKey(id)) { - // containsKey is used because the constant may be null - int fieldMaxDefinitionLevel = - maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); - reorderedFields.add( - ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel)); - types.add(null); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(ParquetValueReaders.position()); - types.add(null); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - reorderedFields.add(ParquetValueReaders.constant(false)); - 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 RowDataReader(types, reorderedFields); - } - - @Override - public ParquetValueReader list( - Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { - if (expectedList == null) { - return null; - } - - 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) { - if (expectedMap == null) { - return null; - } - - 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 - @SuppressWarnings("CyclomaticComplexity") - public ParquetValueReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - if (expected == null) { - return null; - } - - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected.typeId() == Types.LongType.get().typeId()) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case TIME_MICROS: - return new LossyMicrosToMillisTimeReader(desc); - case TIME_MILLIS: - return new MillisTimeReader(desc); - case DATE: - case INT_64: - return new ParquetValueReaders.UnboxedReader<>(desc); - case TIMESTAMP_MICROS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MicrosToTimestampTzReader(desc); - } else { - return new MicrosToTimestampReader(desc); - } - case TIMESTAMP_MILLIS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MillisToTimestampTzReader(desc); - } else { - return new MillisToTimestampReader(desc); - } - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT32: - return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return new ParquetValueReaders.ByteArrayReader(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return new ParquetValueReaders.ByteArrayReader(desc); - case INT32: - if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case FLOAT: - if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { - return new ParquetValueReaders.FloatAsDoubleReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case BOOLEAN: - case INT64: - case DOUBLE: - return new ParquetValueReaders.UnboxedReader<>(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static class BinaryDecimalReader - extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - Binary binary = column.nextBinary(); - BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale); - // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - } - } - - private static class IntegerDecimalReader - extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale); - } - } - - private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale); - } - } - - private static class MicrosToTimestampTzReader - extends ParquetValueReaders.UnboxedReader { - MicrosToTimestampTzReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long value = readLong(); - return TimestampData.fromLocalDateTime( - Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) - .atOffset(ZoneOffset.UTC) - .toLocalDateTime()); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MicrosToTimestampReader - extends ParquetValueReaders.UnboxedReader { - MicrosToTimestampReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long value = readLong(); - return TimestampData.fromInstant( - Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MillisToTimestampReader - extends ParquetValueReaders.UnboxedReader { - MillisToTimestampReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long millis = readLong(); - return TimestampData.fromEpochMillis(millis); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MillisToTimestampTzReader - extends ParquetValueReaders.UnboxedReader { - MillisToTimestampTzReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long millis = readLong(); - return TimestampData.fromLocalDateTime( - Instant.ofEpochMilli(millis).atOffset(ZoneOffset.UTC).toLocalDateTime()); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class StringReader extends ParquetValueReaders.PrimitiveReader { - StringReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public StringData read(StringData ignored) { - Binary binary = column.nextBinary(); - ByteBuffer buffer = binary.toByteBuffer(); - if (buffer.hasArray()) { - return StringData.fromBytes( - buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); - } else { - return StringData.fromBytes(binary.getBytes()); - } - } - } - - private static class LossyMicrosToMillisTimeReader - extends ParquetValueReaders.PrimitiveReader { - LossyMicrosToMillisTimeReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Integer read(Integer reuse) { - // Discard microseconds since Flink uses millisecond unit for TIME type. - return (int) Math.floorDiv(column.nextLong(), 1000); - } - } - - private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader { - MillisTimeReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Integer read(Integer reuse) { - return (int) column.nextLong(); - } - } - - private static class ArrayReader - extends ParquetValueReaders.RepeatedReader { - private int readPos = 0; - private int writePos = 0; - - ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { - super(definitionLevel, repetitionLevel, reader); - } - - @Override - protected ReusableArrayData newListData(ArrayData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableArrayData) { - return (ReusableArrayData) reuse; - } else { - return new ReusableArrayData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected E getElement(ReusableArrayData list) { - E value = null; - if (readPos < list.capacity()) { - value = (E) list.values[readPos]; - } - - readPos += 1; - - return value; - } - - @Override - protected void addElement(ReusableArrayData reused, E element) { - if (writePos >= reused.capacity()) { - reused.grow(); - } - - reused.values[writePos] = element; - - writePos += 1; - } - - @Override - protected ArrayData buildList(ReusableArrayData list) { - // Since ReusableArrayData is not accepted by Flink, use GenericArrayData temporarily to walk - // around it. - // Revert this to use ReusableArrayData once it is fixed in Flink. - // For your reference, https://issues.apache.org/jira/browse/FLINK-25238. - return new GenericArrayData(Arrays.copyOf(list.values, writePos)); - } - } - - private static class MapReader - extends ParquetValueReaders.RepeatedKeyValueReader { - private int readPos = 0; - private int writePos = 0; - - private final ParquetValueReaders.ReusableEntry entry = - new ParquetValueReaders.ReusableEntry<>(); - private final ParquetValueReaders.ReusableEntry nullEntry = - new ParquetValueReaders.ReusableEntry<>(); - - MapReader( - int definitionLevel, - int repetitionLevel, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - super(definitionLevel, repetitionLevel, keyReader, valueReader); - } - - @Override - protected ReusableMapData newMapData(MapData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableMapData) { - return (ReusableMapData) reuse; - } else { - return new ReusableMapData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected Map.Entry getPair(ReusableMapData map) { - Map.Entry kv = nullEntry; - if (readPos < map.capacity()) { - entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); - kv = entry; - } - - readPos += 1; - - return kv; - } - - @Override - protected void addPair(ReusableMapData map, K key, V value) { - if (writePos >= map.capacity()) { - map.grow(); - } - - map.keys.values[writePos] = key; - map.values.values[writePos] = value; - - writePos += 1; - } - - @Override - protected MapData buildMap(ReusableMapData map) { - map.setNumElements(writePos); - return map; - } - } - - private static class RowDataReader - extends ParquetValueReaders.StructReader { - private final int numFields; - - RowDataReader(List types, List> readers) { - super(types, readers); - this.numFields = readers.size(); - } - - @Override - protected GenericRowData newStructData(RowData reuse) { - if (reuse instanceof GenericRowData) { - return (GenericRowData) reuse; - } else { - return new GenericRowData(numFields); - } - } - - @Override - protected Object getField(GenericRowData intermediate, int pos) { - return intermediate.getField(pos); - } - - @Override - protected RowData buildStruct(GenericRowData struct) { - return struct; - } - - @Override - protected void set(GenericRowData row, int pos, Object value) { - row.setField(pos, value); - } - - @Override - protected void setNull(GenericRowData row, int pos) { - row.setField(pos, null); - } - - @Override - protected void setBoolean(GenericRowData row, int pos, boolean value) { - row.setField(pos, value); - } - - @Override - protected void setInteger(GenericRowData row, int pos, int value) { - row.setField(pos, value); - } - - @Override - protected void setLong(GenericRowData row, int pos, long value) { - row.setField(pos, value); - } - - @Override - protected void setFloat(GenericRowData row, int pos, float value) { - row.setField(pos, value); - } - - @Override - protected void setDouble(GenericRowData row, int pos, double value) { - row.setField(pos, value); - } - } - - private static class ReusableMapData implements MapData { - private final ReusableArrayData keys; - private final ReusableArrayData values; - - private int numElements; - - private ReusableMapData() { - this.keys = new ReusableArrayData(); - this.values = new ReusableArrayData(); - } - - private void grow() { - keys.grow(); - values.grow(); - } - - private int capacity() { - return keys.capacity(); - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - keys.setNumElements(numElements); - values.setNumElements(numElements); - } - - @Override - public int size() { - return numElements; - } - - @Override - public ReusableArrayData keyArray() { - return keys; - } - - @Override - public ReusableArrayData valueArray() { - return values; - } - } - - private static class ReusableArrayData implements ArrayData { - private static final Object[] EMPTY = new Object[0]; - - private Object[] values = EMPTY; - private int numElements = 0; - - private void grow() { - if (values.length == 0) { - this.values = new Object[20]; - } else { - Object[] old = values; - this.values = new Object[old.length << 1]; - // copy the old array in case it has values that can be reused - System.arraycopy(old, 0, values, 0, old.length); - } - } - - private int capacity() { - return values.length; - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - } - - @Override - public int size() { - return numElements; - } - - @Override - public boolean isNullAt(int ordinal) { - return null == values[ordinal]; - } - - @Override - public boolean getBoolean(int ordinal) { - return (boolean) values[ordinal]; - } - - @Override - public byte getByte(int ordinal) { - return (byte) values[ordinal]; - } - - @Override - public short getShort(int ordinal) { - return (short) values[ordinal]; - } - - @Override - public int getInt(int ordinal) { - return (int) values[ordinal]; - } - - @Override - public long getLong(int ordinal) { - return (long) values[ordinal]; - } - - @Override - public float getFloat(int ordinal) { - return (float) values[ordinal]; - } - - @Override - public double getDouble(int ordinal) { - return (double) values[ordinal]; - } - - @Override - public StringData getString(int pos) { - return (StringData) values[pos]; - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return (DecimalData) values[pos]; - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return (TimestampData) values[pos]; - } - - @SuppressWarnings("unchecked") - @Override - public RawValueData getRawValue(int pos) { - return (RawValueData) values[pos]; - } - - @Override - public byte[] getBinary(int ordinal) { - return (byte[]) values[ordinal]; - } - - @Override - public ArrayData getArray(int ordinal) { - return (ArrayData) values[ordinal]; - } - - @Override - public MapData getMap(int ordinal) { - return (MapData) values[ordinal]; - } - - @Override - public RowData getRow(int pos, int numFields) { - return (RowData) values[pos]; - } - - @Override - public boolean[] toBooleanArray() { - return ArrayUtil.toPrimitive((Boolean[]) values); - } - - @Override - public byte[] toByteArray() { - return ArrayUtil.toPrimitive((Byte[]) values); - } - - @Override - public short[] toShortArray() { - return ArrayUtil.toPrimitive((Short[]) values); - } - - @Override - public int[] toIntArray() { - return ArrayUtil.toPrimitive((Integer[]) values); - } - - @Override - public long[] toLongArray() { - return ArrayUtil.toPrimitive((Long[]) values); - } - - @Override - public float[] toFloatArray() { - return ArrayUtil.toPrimitive((Float[]) values); - } - - @Override - public double[] toDoubleArray() { - return ArrayUtil.toPrimitive((Double[]) values); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java deleted file mode 100644 index db4f1730a134..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java +++ /dev/null @@ -1,504 +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.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.RowType.RowField; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.ParquetValueWriter; -import org.apache.iceberg.parquet.ParquetValueWriters; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -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; - -public class FlinkParquetWriters { - private FlinkParquetWriters() {} - - @SuppressWarnings("unchecked") - public static ParquetValueWriter buildWriter(LogicalType schema, MessageType type) { - return (ParquetValueWriter) - ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type)); - } - - private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor> { - private final MessageType type; - - WriteBuilder(MessageType type) { - this.type = type; - } - - @Override - public ParquetValueWriter message( - RowType sStruct, MessageType message, List> fields) { - return struct(sStruct, message.asGroupType(), fields); - } - - @Override - public ParquetValueWriter struct( - RowType sStruct, GroupType struct, List> fieldWriters) { - List fields = struct.getFields(); - List flinkFields = sStruct.getFields(); - List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); - List flinkTypes = Lists.newArrayList(); - for (int i = 0; i < fields.size(); i += 1) { - writers.add(newOption(struct.getType(i), fieldWriters.get(i))); - flinkTypes.add(flinkFields.get(i).getType()); - } - - return new RowDataWriter(writers, flinkTypes); - } - - @Override - public ParquetValueWriter list( - ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { - GroupType repeated = array.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new ArrayDataWriter<>( - repeatedD, - repeatedR, - newOption(repeated.getType(0), elementWriter), - sArray.getElementType()); - } - - @Override - public ParquetValueWriter map( - MapType sMap, - GroupType map, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter) { - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new MapDataWriter<>( - repeatedD, - repeatedR, - newOption(repeatedKeyValue.getType(0), keyWriter), - newOption(repeatedKeyValue.getType(1), valueWriter), - sMap.getKeyType(), - sMap.getValueType()); - } - - private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { - int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); - return ParquetValueWriters.option(fieldType, maxD, writer); - } - - @Override - public ParquetValueWriter primitive(LogicalType fType, PrimitiveType primitive) { - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(fType, desc); - case INT_64: - return ParquetValueWriters.longs(desc); - case TIME_MICROS: - return timeMicros(desc); - case TIMESTAMP_MICROS: - return timestamps(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return byteArrays(desc); - case BOOLEAN: - return ParquetValueWriters.booleans(desc); - case INT32: - return ints(fType, desc); - case INT64: - return ParquetValueWriters.longs(desc); - case FLOAT: - return ParquetValueWriters.floats(desc); - case DOUBLE: - return ParquetValueWriters.doubles(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static ParquetValueWriters.PrimitiveWriter ints( - LogicalType type, ColumnDescriptor desc) { - if (type instanceof TinyIntType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof SmallIntType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - - private static ParquetValueWriters.PrimitiveWriter strings(ColumnDescriptor desc) { - return new StringDataWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter timeMicros(ColumnDescriptor desc) { - return new TimeMicrosWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( - ColumnDescriptor desc, int precision, int scale) { - Preconditions.checkArgument( - precision <= 9, - "Cannot write decimal value as integer with precision larger than 9," - + " wrong precision %s", - precision); - return new IntegerDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsLong( - ColumnDescriptor desc, int precision, int scale) { - Preconditions.checkArgument( - precision <= 18, - "Cannot write decimal value as long with precision larger than 18, " - + " wrong precision %s", - precision); - return new LongDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( - ColumnDescriptor desc, int precision, int scale) { - return new FixedDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter timestamps( - ColumnDescriptor desc) { - return new TimestampDataWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { - return new ByteArrayWriter(desc); - } - - private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter { - private StringDataWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, StringData value) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes())); - } - } - - private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter { - private TimeMicrosWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, Integer value) { - long micros = value.longValue() * 1000; - column.writeLong(repetitionLevel, micros); - } - } - - private static class IntegerDecimalWriter - extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); - } - } - - private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeLong(repetitionLevel, decimal.toUnscaledLong()); - } - } - - private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - byte[] binary = - DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get()); - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); - } - } - - private static class TimestampDataWriter - extends ParquetValueWriters.PrimitiveWriter { - private TimestampDataWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, TimestampData value) { - column.writeLong( - repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000); - } - } - - private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { - private ByteArrayWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, byte[] bytes) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); - } - } - - private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { - private final LogicalType elementType; - - private ArrayDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter writer, - LogicalType elementType) { - super(definitionLevel, repetitionLevel, writer); - this.elementType = elementType; - } - - @Override - protected Iterator elements(ArrayData list) { - return new ElementIterator<>(list); - } - - private class ElementIterator implements Iterator { - private final int size; - private final ArrayData list; - private final ArrayData.ElementGetter getter; - private int index; - - private ElementIterator(ArrayData list) { - this.list = list; - size = list.size(); - getter = ArrayData.createElementGetter(elementType); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public E next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - E element = (E) getter.getElementOrNull(list, index); - index += 1; - - return element; - } - } - } - - private static class MapDataWriter - extends ParquetValueWriters.RepeatedKeyValueWriter { - private final LogicalType keyType; - private final LogicalType valueType; - - private MapDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - super(definitionLevel, repetitionLevel, keyWriter, valueWriter); - this.keyType = keyType; - this.valueType = valueType; - } - - @Override - protected Iterator> pairs(MapData map) { - return new EntryIterator<>(map); - } - - private class EntryIterator implements Iterator> { - private final int size; - private final ArrayData keys; - private final ArrayData values; - private final ParquetValueReaders.ReusableEntry entry; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - private int index; - - private EntryIterator(MapData map) { - size = map.size(); - keys = map.keyArray(); - values = map.valueArray(); - entry = new ParquetValueReaders.ReusableEntry<>(); - keyGetter = ArrayData.createElementGetter(keyType); - valueGetter = ArrayData.createElementGetter(valueType); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public Map.Entry next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - entry.set( - (K) keyGetter.getElementOrNull(keys, index), - (V) valueGetter.getElementOrNull(values, index)); - index += 1; - - return entry; - } - } - } - - private static class RowDataWriter extends ParquetValueWriters.StructWriter { - private final RowData.FieldGetter[] fieldGetter; - - RowDataWriter(List> writers, List types) { - super(writers); - fieldGetter = new RowData.FieldGetter[types.size()]; - for (int i = 0; i < types.size(); i += 1) { - fieldGetter[i] = RowData.createFieldGetter(types.get(i), i); - } - } - - @Override - protected Object get(RowData struct, int index) { - return fieldGetter[index].getFieldOrNull(struct); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java deleted file mode 100644 index ba4e1a7a7aec..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ /dev/null @@ -1,161 +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.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -abstract class FlinkSchemaVisitor { - - static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { - return visit(flinkType, schema.asStruct(), visitor); - } - - private static T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor visitor) { - switch (iType.typeId()) { - case STRUCT: - return visitRecord(flinkType, iType.asStructType(), visitor); - - case MAP: - MapType mapType = (MapType) flinkType; - Types.MapType iMapType = iType.asMapType(); - T key; - T value; - - Types.NestedField keyField = iMapType.field(iMapType.keyId()); - visitor.beforeMapKey(keyField); - try { - key = visit(mapType.getKeyType(), iMapType.keyType(), visitor); - } finally { - visitor.afterMapKey(keyField); - } - - Types.NestedField valueField = iMapType.field(iMapType.valueId()); - visitor.beforeMapValue(valueField); - try { - value = visit(mapType.getValueType(), iMapType.valueType(), visitor); - } finally { - visitor.afterMapValue(valueField); - } - - return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType()); - - case LIST: - ArrayType listType = (ArrayType) flinkType; - Types.ListType iListType = iType.asListType(); - T element; - - Types.NestedField elementField = iListType.field(iListType.elementId()); - visitor.beforeListElement(elementField); - try { - element = visit(listType.getElementType(), iListType.elementType(), visitor); - } finally { - visitor.afterListElement(elementField); - } - - return visitor.list(iListType, element, listType.getElementType()); - - default: - return visitor.primitive(iType.asPrimitiveType(), flinkType); - } - } - - private static T visitRecord( - LogicalType flinkType, Types.StructType struct, FlinkSchemaVisitor visitor) { - Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType); - RowType rowType = (RowType) flinkType; - - int fieldSize = struct.fields().size(); - List results = Lists.newArrayListWithExpectedSize(fieldSize); - List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); - List nestedFields = struct.fields(); - - for (int i = 0; i < fieldSize; i++) { - Types.NestedField iField = nestedFields.get(i); - int fieldIndex = rowType.getFieldIndex(iField.name()); - Preconditions.checkArgument( - fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); - - LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); - - fieldTypes.add(fieldFlinkType); - - visitor.beforeField(iField); - try { - results.add(visit(fieldFlinkType, iField.type(), visitor)); - } finally { - visitor.afterField(iField); - } - } - - return visitor.record(struct, results, fieldTypes); - } - - public T record(Types.StructType iStruct, List results, List fieldTypes) { - return null; - } - - public T list(Types.ListType iList, T element, LogicalType elementType) { - return null; - } - - public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) { - return null; - } - - public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { - return null; - } - - public void beforeField(Types.NestedField field) {} - - public void afterField(Types.NestedField field) {} - - public void beforeListElement(Types.NestedField elementField) { - beforeField(elementField); - } - - public void afterListElement(Types.NestedField elementField) { - afterField(elementField); - } - - public void beforeMapKey(Types.NestedField keyField) { - beforeField(keyField); - } - - public void afterMapKey(Types.NestedField keyField) { - afterField(keyField); - } - - public void beforeMapValue(Types.NestedField valueField) { - beforeField(valueField); - } - - public void afterMapValue(Types.NestedField valueField) { - afterField(valueField); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java deleted file mode 100644 index 32f6c3a2ccfd..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java +++ /dev/null @@ -1,312 +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.math.BigDecimal; -import java.math.BigInteger; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import org.apache.avro.io.Decoder; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -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; - -public class FlinkValueReaders { - - private FlinkValueReaders() {} - - static ValueReader strings() { - return StringReader.INSTANCE; - } - - static ValueReader enums(List symbols) { - return new EnumReader(symbols); - } - - static ValueReader uuids() { - return ValueReaders.fixed(16); - } - - static ValueReader timeMicros() { - return TimeMicrosReader.INSTANCE; - } - - static ValueReader timestampMills() { - return TimestampMillsReader.INSTANCE; - } - - static ValueReader timestampMicros() { - return TimestampMicrosReader.INSTANCE; - } - - static ValueReader decimal( - ValueReader unscaledReader, int precision, int scale) { - return new DecimalReader(unscaledReader, precision, scale); - } - - static ValueReader array(ValueReader elementReader) { - return new ArrayReader(elementReader); - } - - static ValueReader arrayMap(ValueReader keyReader, ValueReader valueReader) { - return new ArrayMapReader(keyReader, valueReader); - } - - static ValueReader map(ValueReader keyReader, ValueReader valueReader) { - return new MapReader(keyReader, valueReader); - } - - static ValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); - } - - private static class StringReader implements ValueReader { - private static final StringReader INSTANCE = new StringReader(); - - private StringReader() {} - - @Override - public StringData read(Decoder decoder, Object reuse) throws IOException { - // use the decoder's readString(Utf8) method because it may be a resolving decoder - Utf8 utf8 = null; - if (reuse instanceof StringData) { - utf8 = new Utf8(((StringData) reuse).toBytes()); - } - - Utf8 string = decoder.readString(utf8); - return StringData.fromBytes(string.getBytes(), 0, string.getByteLength()); - } - } - - private static class EnumReader implements ValueReader { - private final StringData[] symbols; - - private EnumReader(List symbols) { - this.symbols = new StringData[symbols.size()]; - for (int i = 0; i < this.symbols.length; i += 1) { - this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); - } - } - - @Override - public StringData read(Decoder decoder, Object ignore) throws IOException { - int index = decoder.readEnum(); - return symbols[index]; - } - } - - private static class DecimalReader implements ValueReader { - private final ValueReader bytesReader; - private final int precision; - private final int scale; - - private DecimalReader(ValueReader bytesReader, int precision, int scale) { - this.bytesReader = bytesReader; - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(Decoder decoder, Object reuse) throws IOException { - byte[] bytes = bytesReader.read(decoder, null); - return DecimalData.fromBigDecimal( - new BigDecimal(new BigInteger(bytes), scale), precision, scale); - } - } - - private static class TimeMicrosReader implements ValueReader { - private static final TimeMicrosReader INSTANCE = new TimeMicrosReader(); - - @Override - public Integer read(Decoder decoder, Object reuse) throws IOException { - long micros = decoder.readLong(); - // Flink only support time mills, just erase micros. - return (int) (micros / 1000); - } - } - - private static class TimestampMillsReader implements ValueReader { - private static final TimestampMillsReader INSTANCE = new TimestampMillsReader(); - - @Override - public TimestampData read(Decoder decoder, Object reuse) throws IOException { - return TimestampData.fromEpochMillis(decoder.readLong()); - } - } - - private static class TimestampMicrosReader implements ValueReader { - private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader(); - - @Override - public TimestampData read(Decoder decoder, Object reuse) throws IOException { - long micros = decoder.readLong(); - long mills = micros / 1000; - int nanos = ((int) (micros % 1000)) * 1000; - if (nanos < 0) { - nanos += 1_000_000; - mills -= 1; - } - return TimestampData.fromEpochMillis(mills, nanos); - } - } - - private static class ArrayReader implements ValueReader { - private final ValueReader elementReader; - private final List reusedList = Lists.newArrayList(); - - private ArrayReader(ValueReader elementReader) { - this.elementReader = elementReader; - } - - @Override - public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { - reusedList.clear(); - long chunkLength = decoder.readArrayStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedList.add(elementReader.read(decoder, null)); - } - - chunkLength = decoder.arrayNext(); - } - - // this will convert the list to an array so it is okay to reuse the list - return new GenericArrayData(reusedList.toArray()); - } - } - - private static MapData kvArrayToMap(List keyList, List valueList) { - Map map = Maps.newHashMap(); - Object[] keys = keyList.toArray(); - Object[] values = valueList.toArray(); - for (int i = 0; i < keys.length; i++) { - map.put(keys[i], values[i]); - } - - return new GenericMapData(map); - } - - private static class ArrayMapReader implements ValueReader { - private final ValueReader keyReader; - private final ValueReader valueReader; - - private final List reusedKeyList = Lists.newArrayList(); - private final List reusedValueList = Lists.newArrayList(); - - private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData read(Decoder decoder, Object reuse) throws IOException { - reusedKeyList.clear(); - reusedValueList.clear(); - - long chunkLength = decoder.readArrayStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedKeyList.add(keyReader.read(decoder, null)); - reusedValueList.add(valueReader.read(decoder, null)); - } - - chunkLength = decoder.arrayNext(); - } - - return kvArrayToMap(reusedKeyList, reusedValueList); - } - } - - private static class MapReader implements ValueReader { - private final ValueReader keyReader; - private final ValueReader valueReader; - - private final List reusedKeyList = Lists.newArrayList(); - private final List reusedValueList = Lists.newArrayList(); - - private MapReader(ValueReader keyReader, ValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData read(Decoder decoder, Object reuse) throws IOException { - reusedKeyList.clear(); - reusedValueList.clear(); - - long chunkLength = decoder.readMapStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedKeyList.add(keyReader.read(decoder, null)); - reusedValueList.add(valueReader.read(decoder, null)); - } - - chunkLength = decoder.mapNext(); - } - - return kvArrayToMap(reusedKeyList, reusedValueList); - } - } - - private static class StructReader extends ValueReaders.StructReader { - private final int numFields; - - private StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); - this.numFields = readers.size(); - } - - @Override - protected RowData reuseOrCreate(Object reuse) { - if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { - return (GenericRowData) 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); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java deleted file mode 100644 index 4e86ecce28b5..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java +++ /dev/null @@ -1,253 +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.lang.reflect.Array; -import java.util.List; -import org.apache.avro.io.Encoder; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.avro.ValueWriter; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; - -public class FlinkValueWriters { - - private FlinkValueWriters() {} - - static ValueWriter strings() { - return StringWriter.INSTANCE; - } - - static ValueWriter timeMicros() { - return TimeMicrosWriter.INSTANCE; - } - - static ValueWriter timestampMicros() { - return TimestampMicrosWriter.INSTANCE; - } - - static ValueWriter decimal(int precision, int scale) { - return new DecimalWriter(precision, scale); - } - - static ValueWriter array(ValueWriter elementWriter, LogicalType elementType) { - return new ArrayWriter<>(elementWriter, elementType); - } - - static ValueWriter arrayMap( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); - } - - static ValueWriter map( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); - } - - static ValueWriter row(List> writers, List types) { - return new RowWriter(writers, types); - } - - private static class StringWriter implements ValueWriter { - private static final StringWriter INSTANCE = new StringWriter(); - - private StringWriter() {} - - @Override - public void write(StringData s, Encoder encoder) throws IOException { - // toBytes is cheaper than Avro calling toString, which incurs encoding costs - encoder.writeString(new Utf8(s.toBytes())); - } - } - - private static class DecimalWriter implements ValueWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private DecimalWriter(int precision, int scale) { - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(DecimalData d, Encoder encoder) throws IOException { - encoder.writeFixed( - DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get())); - } - } - - private static class TimeMicrosWriter implements ValueWriter { - private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter(); - - @Override - public void write(Integer timeMills, Encoder encoder) throws IOException { - encoder.writeLong(timeMills * 1000L); - } - } - - private static class TimestampMicrosWriter implements ValueWriter { - private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter(); - - @Override - public void write(TimestampData timestampData, Encoder encoder) throws IOException { - long micros = - timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; - encoder.writeLong(micros); - } - } - - private static class ArrayWriter implements ValueWriter { - private final ValueWriter elementWriter; - private final ArrayData.ElementGetter elementGetter; - - private ArrayWriter(ValueWriter elementWriter, LogicalType elementType) { - this.elementWriter = elementWriter; - this.elementGetter = ArrayData.createElementGetter(elementType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(ArrayData array, Encoder encoder) throws IOException { - encoder.writeArrayStart(); - int numElements = array.size(); - encoder.setItemCount(numElements); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder); - } - encoder.writeArrayEnd(); - } - } - - private static class ArrayMapWriter implements ValueWriter { - private final ValueWriter keyWriter; - private final ValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - private ArrayMapWriter( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueWriter = valueWriter; - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(MapData map, Encoder encoder) throws IOException { - encoder.writeArrayStart(); - int numElements = map.size(); - encoder.setItemCount(numElements); - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); - valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); - } - encoder.writeArrayEnd(); - } - } - - private static class MapWriter implements ValueWriter { - private final ValueWriter keyWriter; - private final ValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - private MapWriter( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueWriter = valueWriter; - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(MapData map, Encoder encoder) throws IOException { - encoder.writeMapStart(); - int numElements = map.size(); - encoder.setItemCount(numElements); - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); - valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); - } - encoder.writeMapEnd(); - } - } - - static class RowWriter implements ValueWriter { - private final ValueWriter[] writers; - private final RowData.FieldGetter[] getters; - - private RowWriter(List> writers, List types) { - this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); - this.getters = new RowData.FieldGetter[writers.size()]; - for (int i = 0; i < writers.size(); i += 1) { - this.writers[i] = writers.get(i); - this.getters[i] = RowData.createFieldGetter(types.get(i), i); - } - } - - @Override - public void write(RowData row, Encoder encoder) throws IOException { - for (int i = 0; i < writers.length; i += 1) { - if (row.isNullAt(i)) { - writers[i].write(null, encoder); - } else { - write(row, i, writers[i], encoder); - } - } - } - - @SuppressWarnings("unchecked") - private void write(RowData row, int pos, ValueWriter writer, Encoder encoder) - throws IOException { - writer.write((T) getters[pos].getFieldOrNull(row), encoder); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java deleted file mode 100644 index 33feb2e32118..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java +++ /dev/null @@ -1,222 +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.util.Deque; -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.RowType.RowField; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -public class ParquetWithFlinkSchemaVisitor { - private final Deque fieldNames = Lists.newLinkedList(); - - public static T visit( - LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor visitor) { - Preconditions.checkArgument(sType != null, "Invalid DataType: null"); - if (type instanceof MessageType) { - Preconditions.checkArgument( - sType instanceof RowType, "Invalid struct: %s is not a struct", sType); - RowType struct = (RowType) sType; - return visitor.message( - struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); - } else if (type.isPrimitive()) { - return visitor.primitive(sType, type.asPrimitiveType()); - } else { - // if not a primitive, the typeId must be a group - GroupType group = type.asGroupType(); - OriginalType annotation = group.getOriginalType(); - if (annotation != null) { - switch (annotation) { - case LIST: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid list: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid list: does not contain single repeated field: %s", - group); - - GroupType repeatedElement = group.getFields().get(0).asGroupType(); - Preconditions.checkArgument( - repeatedElement.isRepetition(Type.Repetition.REPEATED), - "Invalid list: inner group is not repeated"); - Preconditions.checkArgument( - repeatedElement.getFieldCount() <= 1, - "Invalid list: repeated group is not a single field: %s", - group); - - Preconditions.checkArgument( - sType instanceof ArrayType, "Invalid list: %s is not an array", sType); - ArrayType array = (ArrayType) sType; - RowType.RowField element = - new RowField( - "element", array.getElementType(), "element of " + array.asSummaryString()); - - visitor.fieldNames.push(repeatedElement.getName()); - try { - T elementResult = null; - if (repeatedElement.getFieldCount() > 0) { - elementResult = visitField(element, repeatedElement.getType(0), visitor); - } - - return visitor.list(array, group, elementResult); - - } finally { - visitor.fieldNames.pop(); - } - - case MAP: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid map: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid map: does not contain single repeated field: %s", - group); - - GroupType repeatedKeyValue = group.getType(0).asGroupType(); - Preconditions.checkArgument( - repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), - "Invalid map: inner group is not repeated"); - Preconditions.checkArgument( - repeatedKeyValue.getFieldCount() <= 2, - "Invalid map: repeated group does not have 2 fields"); - - Preconditions.checkArgument( - sType instanceof MapType, "Invalid map: %s is not a map", sType); - MapType map = (MapType) sType; - RowField keyField = - new RowField("key", map.getKeyType(), "key of " + map.asSummaryString()); - RowField valueField = - new RowField("value", map.getValueType(), "value of " + map.asSummaryString()); - - visitor.fieldNames.push(repeatedKeyValue.getName()); - try { - T keyResult = null; - T valueResult = null; - switch (repeatedKeyValue.getFieldCount()) { - case 2: - // if there are 2 fields, both key and value are projected - keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); - valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); - break; - case 1: - // if there is just one, use the name to determine what it is - Type keyOrValue = repeatedKeyValue.getType(0); - if (keyOrValue.getName().equalsIgnoreCase("key")) { - keyResult = visitField(keyField, keyOrValue, visitor); - // value result remains null - } else { - valueResult = visitField(valueField, keyOrValue, visitor); - // key result remains null - } - break; - default: - // both results will remain null - } - - return visitor.map(map, group, keyResult, valueResult); - - } finally { - visitor.fieldNames.pop(); - } - - default: - } - } - Preconditions.checkArgument( - sType instanceof RowType, "Invalid struct: %s is not a struct", sType); - RowType struct = (RowType) sType; - return visitor.struct(struct, group, visitFields(struct, group, visitor)); - } - } - - private static T visitField( - RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor visitor) { - visitor.fieldNames.push(field.getName()); - try { - return visit(sField.getType(), field, visitor); - } finally { - visitor.fieldNames.pop(); - } - } - - private static List visitFields( - RowType struct, GroupType group, ParquetWithFlinkSchemaVisitor visitor) { - List sFields = struct.getFields(); - Preconditions.checkArgument( - sFields.size() == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); - List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); - for (int i = 0; i < sFields.size(); i += 1) { - Type field = group.getFields().get(i); - RowType.RowField sField = sFields.get(i); - Preconditions.checkArgument( - field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())), - "Structs do not match: field %s != %s", - field.getName(), - sField.getName()); - results.add(visitField(sField, field, visitor)); - } - - return results; - } - - public T message(RowType sStruct, MessageType message, List fields) { - return null; - } - - public T struct(RowType sStruct, GroupType struct, List fields) { - return null; - } - - public T list(ArrayType sArray, GroupType array, T element) { - return null; - } - - public T map(MapType sMap, GroupType map, T key, T value) { - return null; - } - - public T primitive(LogicalType sPrimitive, PrimitiveType primitive) { - return null; - } - - protected String[] currentPath() { - return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); - } - - protected String[] path(String name) { - List list = Lists.newArrayList(fieldNames.descendingIterator()); - list.add(name); - return list.toArray(new String[0]); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java deleted file mode 100644 index 33816c97ac29..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java +++ /dev/null @@ -1,341 +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.util.Arrays; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; - -public class RowDataProjection implements RowData { - /** - * Creates a projecting wrapper for {@link RowData} rows. - * - *

    This projection will not project the nested children types of repeated types like lists and - * maps. - * - * @param schema schema of rows wrapped by this projection - * @param projectedSchema result schema of the projected rows - * @return a wrapper to project rows - */ - public static RowDataProjection create(Schema schema, Schema projectedSchema) { - return RowDataProjection.create( - FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct()); - } - - /** - * Creates a projecting wrapper for {@link RowData} rows. - * - *

    This projection will not project the nested children types of repeated types like lists and - * maps. - * - * @param rowType flink row type of rows wrapped by this projection - * @param schema schema of rows wrapped by this projection - * @param projectedSchema result schema of the projected rows - * @return a wrapper to project rows - */ - public static RowDataProjection create( - RowType rowType, Types.StructType schema, Types.StructType projectedSchema) { - return new RowDataProjection(rowType, schema, projectedSchema); - } - - private final RowData.FieldGetter[] getters; - private RowData rowData; - - private RowDataProjection( - RowType rowType, Types.StructType rowStruct, Types.StructType projectType) { - Map fieldIdToPosition = Maps.newHashMap(); - for (int i = 0; i < rowStruct.fields().size(); i++) { - fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i); - } - - this.getters = new RowData.FieldGetter[projectType.fields().size()]; - for (int i = 0; i < getters.length; i++) { - Types.NestedField projectField = projectType.fields().get(i); - Types.NestedField rowField = rowStruct.field(projectField.fieldId()); - - Preconditions.checkNotNull( - rowField, - "Cannot locate the project field <%s> in the iceberg struct <%s>", - projectField, - rowStruct); - - getters[i] = - createFieldGetter( - rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField); - } - } - - private static RowData.FieldGetter createFieldGetter( - RowType rowType, int position, Types.NestedField rowField, Types.NestedField projectField) { - Preconditions.checkArgument( - rowField.type().typeId() == projectField.type().typeId(), - "Different iceberg type between row field <%s> and project field <%s>", - rowField, - projectField); - - switch (projectField.type().typeId()) { - case STRUCT: - RowType nestedRowType = (RowType) rowType.getTypeAt(position); - return row -> { - // null nested struct value - if (row.isNullAt(position)) { - return null; - } - - RowData nestedRow = row.getRow(position, nestedRowType.getFieldCount()); - return RowDataProjection.create( - nestedRowType, rowField.type().asStructType(), projectField.type().asStructType()) - .wrap(nestedRow); - }; - - case MAP: - Types.MapType projectedMap = projectField.type().asMapType(); - Types.MapType originalMap = rowField.type().asMapType(); - - boolean keyProjectable = - !projectedMap.keyType().isNestedType() - || projectedMap.keyType().equals(originalMap.keyType()); - boolean valueProjectable = - !projectedMap.valueType().isNestedType() - || projectedMap.valueType().equals(originalMap.valueType()); - Preconditions.checkArgument( - keyProjectable && valueProjectable, - "Cannot project a partial map key or value with non-primitive type. Trying to project <%s> out of <%s>", - projectField, - rowField); - - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - - case LIST: - Types.ListType projectedList = projectField.type().asListType(); - Types.ListType originalList = rowField.type().asListType(); - - boolean elementProjectable = - !projectedList.elementType().isNestedType() - || projectedList.elementType().equals(originalList.elementType()); - Preconditions.checkArgument( - elementProjectable, - "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>", - projectField, - rowField); - - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - - default: - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - } - } - - public RowData wrap(RowData row) { - // StructProjection allow wrapping null root struct object. - // See more discussions in https://github.com/apache/iceberg/pull/7517. - // RowDataProjection never allowed null root object to be wrapped. - // Hence, it is fine to enforce strict Preconditions check here. - Preconditions.checkArgument(row != null, "Invalid row data: null"); - this.rowData = row; - return this; - } - - private Object getValue(int pos) { - Preconditions.checkState(rowData != null, "Row data not wrapped"); - return getters[pos].getFieldOrNull(rowData); - } - - @Override - public int getArity() { - return getters.length; - } - - @Override - public RowKind getRowKind() { - Preconditions.checkState(rowData != null, "Row data not wrapped"); - return rowData.getRowKind(); - } - - @Override - public void setRowKind(RowKind kind) { - throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection"); - } - - @Override - public boolean isNullAt(int pos) { - return getValue(pos) == null; - } - - @Override - public boolean getBoolean(int pos) { - return (boolean) getValue(pos); - } - - @Override - public byte getByte(int pos) { - return (byte) getValue(pos); - } - - @Override - public short getShort(int pos) { - return (short) getValue(pos); - } - - @Override - public int getInt(int pos) { - return (int) getValue(pos); - } - - @Override - public long getLong(int pos) { - return (long) getValue(pos); - } - - @Override - public float getFloat(int pos) { - return (float) getValue(pos); - } - - @Override - public double getDouble(int pos) { - return (double) getValue(pos); - } - - @Override - public StringData getString(int pos) { - return (StringData) getValue(pos); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return (DecimalData) getValue(pos); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return (TimestampData) getValue(pos); - } - - @Override - @SuppressWarnings("unchecked") - public RawValueData getRawValue(int pos) { - return (RawValueData) getValue(pos); - } - - @Override - public byte[] getBinary(int pos) { - return (byte[]) getValue(pos); - } - - @Override - public ArrayData getArray(int pos) { - return (ArrayData) getValue(pos); - } - - @Override - public MapData getMap(int pos) { - return (MapData) getValue(pos); - } - - @Override - public RowData getRow(int pos, int numFields) { - return (RowData) getValue(pos); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof RowDataProjection)) { - return false; - } - - RowDataProjection that = (RowDataProjection) o; - return deepEquals(that); - } - - @Override - public int hashCode() { - int result = Objects.hashCode(getRowKind()); - for (int pos = 0; pos < getArity(); pos++) { - if (!isNullAt(pos)) { - // Arrays.deepHashCode handles array object properly - result = 31 * result + Arrays.deepHashCode(new Object[] {getValue(pos)}); - } - } - - return result; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(getRowKind().shortString()).append("("); - for (int pos = 0; pos < getArity(); pos++) { - if (pos != 0) { - sb.append(","); - } - // copied the behavior from Flink GenericRowData - sb.append(StringUtils.arrayAwareToString(getValue(pos))); - } - - sb.append(")"); - return sb.toString(); - } - - private boolean deepEquals(RowDataProjection other) { - if (getRowKind() != other.getRowKind()) { - return false; - } - - if (getArity() != other.getArity()) { - return false; - } - - for (int pos = 0; pos < getArity(); ++pos) { - if (isNullAt(pos) && other.isNullAt(pos)) { - continue; - } - - if ((isNullAt(pos) && !other.isNullAt(pos)) || (!isNullAt(pos) && other.isNullAt(pos))) { - return false; - } - - // Objects.deepEquals handles array object properly - if (!Objects.deepEquals(getValue(pos), other.getValue(pos))) { - return false; - } - } - - return true; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java deleted file mode 100644 index c5cb51b7eae4..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ /dev/null @@ -1,100 +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.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ByteBuffers; -import org.apache.iceberg.util.DateTimeUtil; - -public class RowDataUtil { - - private RowDataUtil() {} - - public static Object convertConstant(Type type, Object value) { - if (value == null) { - return null; - } - - switch (type.typeId()) { - case DECIMAL: // DecimalData - Types.DecimalType decimal = (Types.DecimalType) type; - return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale()); - case STRING: // StringData - if (value instanceof Utf8) { - Utf8 utf8 = (Utf8) value; - return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); - } - return StringData.fromString(value.toString()); - case FIXED: // byte[] - if (value instanceof byte[]) { - return value; - } else if (value instanceof GenericData.Fixed) { - return ((GenericData.Fixed) value).bytes(); - } - return ByteBuffers.toByteArray((ByteBuffer) value); - case BINARY: // byte[] - return ByteBuffers.toByteArray((ByteBuffer) value); - case TIME: // int mills instead of long - return (int) ((Long) value / 1000); - case TIMESTAMP: // TimestampData - return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value)); - default: - } - return value; - } - - /** - * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method. This - * skips the check the arity of rowType and from, because the from RowData may contains additional - * column for position deletes. Using {@link RowDataSerializer#copy(RowData, RowData)} will fail - * the arity check. - */ - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - GenericRowData ret; - if (reuse instanceof GenericRowData) { - ret = (GenericRowData) reuse; - } else { - ret = new GenericRowData(from.getArity()); - } - ret.setRowKind(from.getRowKind()); - for (int i = 0; i < rowType.getFieldCount(); i++) { - if (!from.isNullAt(i)) { - RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); - ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); - } else { - ret.setField(i, null); - } - } - return ret; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java deleted file mode 100644 index 1019285018d0..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ /dev/null @@ -1,300 +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.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -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.ByteBuffers; - -@Internal -public class StructRowData implements RowData { - private final Types.StructType type; - private RowKind kind; - private StructLike struct; - - public StructRowData(Types.StructType type) { - this(type, RowKind.INSERT); - } - - public StructRowData(Types.StructType type, RowKind kind) { - this(type, null, kind); - } - - private StructRowData(Types.StructType type, StructLike struct) { - this(type, struct, RowKind.INSERT); - } - - private StructRowData(Types.StructType type, StructLike struct, RowKind kind) { - this.type = type; - this.struct = struct; - this.kind = kind; - } - - public StructRowData setStruct(StructLike newStruct) { - this.struct = newStruct; - return this; - } - - @Override - public int getArity() { - return struct.size(); - } - - @Override - public RowKind getRowKind() { - return kind; - } - - @Override - public void setRowKind(RowKind newKind) { - Preconditions.checkNotNull(newKind, "kind can not be null"); - this.kind = newKind; - } - - @Override - public boolean isNullAt(int pos) { - return struct.get(pos, Object.class) == null; - } - - @Override - public boolean getBoolean(int pos) { - return struct.get(pos, Boolean.class); - } - - @Override - public byte getByte(int pos) { - return (byte) (int) struct.get(pos, Integer.class); - } - - @Override - public short getShort(int pos) { - return (short) (int) struct.get(pos, Integer.class); - } - - @Override - public int getInt(int pos) { - Object integer = struct.get(pos, Object.class); - - if (integer instanceof Integer) { - return (int) integer; - } else if (integer instanceof LocalDate) { - return (int) ((LocalDate) integer).toEpochDay(); - } else if (integer instanceof LocalTime) { - return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000); - } else { - throw new IllegalStateException( - "Unknown type for int field. Type name: " + integer.getClass().getName()); - } - } - - @Override - public long getLong(int pos) { - Object longVal = struct.get(pos, Object.class); - - if (longVal instanceof Long) { - return (long) longVal; - } else if (longVal instanceof OffsetDateTime) { - return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; - } else if (longVal instanceof LocalDate) { - return ((LocalDate) longVal).toEpochDay(); - } else if (longVal instanceof LocalTime) { - return ((LocalTime) longVal).toNanoOfDay(); - } else if (longVal instanceof LocalDateTime) { - return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC)) - .toNanos() - / 1000; - } else { - throw new IllegalStateException( - "Unknown type for long field. Type name: " + longVal.getClass().getName()); - } - } - - @Override - public float getFloat(int pos) { - return struct.get(pos, Float.class); - } - - @Override - public double getDouble(int pos) { - return struct.get(pos, Double.class); - } - - @Override - public StringData getString(int pos) { - return isNullAt(pos) ? null : getStringDataInternal(pos); - } - - private StringData getStringDataInternal(int pos) { - CharSequence seq = struct.get(pos, CharSequence.class); - return StringData.fromString(seq.toString()); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return isNullAt(pos) - ? null - : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale); - } - - private BigDecimal getDecimalInternal(int pos) { - return struct.get(pos, BigDecimal.class); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - long timeLong = getLong(pos); - return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000); - } - - @Override - public RawValueData getRawValue(int pos) { - throw new UnsupportedOperationException("Not supported yet."); - } - - @Override - public byte[] getBinary(int pos) { - return isNullAt(pos) ? null : getBinaryInternal(pos); - } - - private byte[] getBinaryInternal(int pos) { - Object bytes = struct.get(pos, Object.class); - - // should only be either ByteBuffer or byte[] - if (bytes instanceof ByteBuffer) { - return ByteBuffers.toByteArray((ByteBuffer) bytes); - } else if (bytes instanceof byte[]) { - return (byte[]) bytes; - } else if (bytes instanceof UUID) { - UUID uuid = (UUID) bytes; - ByteBuffer bb = ByteBuffer.allocate(16); - bb.putLong(uuid.getMostSignificantBits()); - bb.putLong(uuid.getLeastSignificantBits()); - return bb.array(); - } else { - throw new IllegalStateException( - "Unknown type for binary field. Type name: " + bytes.getClass().getName()); - } - } - - @Override - public ArrayData getArray(int pos) { - return isNullAt(pos) - ? null - : (ArrayData) - convertValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class)); - } - - @Override - public MapData getMap(int pos) { - return isNullAt(pos) - ? null - : (MapData) - convertValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class)); - } - - @Override - public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); - } - - private StructRowData getStructRowData(int pos, int numFields) { - return new StructRowData( - type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); - } - - private Object convertValue(Type elementType, Object value) { - switch (elementType.typeId()) { - case BOOLEAN: - case INTEGER: - case DATE: - case TIME: - case LONG: - case FLOAT: - case DOUBLE: - case DECIMAL: - return value; - case TIMESTAMP: - long millisecond = (long) value / 1000; - int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000; - return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); - case STRING: - return StringData.fromString(value.toString()); - case FIXED: - case BINARY: - return ByteBuffers.toByteArray((ByteBuffer) value); - case STRUCT: - return new StructRowData(elementType.asStructType(), (StructLike) value); - case LIST: - List list = (List) value; - Object[] array = new Object[list.size()]; - - int index = 0; - for (Object element : list) { - if (element == null) { - array[index] = null; - } else { - array[index] = convertValue(elementType.asListType().elementType(), element); - } - - index += 1; - } - return new GenericArrayData(array); - case MAP: - Types.MapType mapType = elementType.asMapType(); - Set> entries = ((Map) value).entrySet(); - Map result = Maps.newHashMap(); - for (Map.Entry entry : entries) { - final Object keyValue = convertValue(mapType.keyType(), entry.getKey()); - final Object valueValue = convertValue(mapType.valueType(), entry.getValue()); - result.put(keyValue, valueValue); - } - - return new GenericMapData(result); - default: - throw new UnsupportedOperationException("Unsupported element type: " + elementType); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java deleted file mode 100644 index 04e168385a36..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java +++ /dev/null @@ -1,61 +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.sink; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.avro.AvroSchemaUtil; - -/** - * This util class converts Avro GenericRecord to Flink RowData.
    - *
    - * Internally it uses Flink {@link AvroToRowDataConverters}. Because of the precision difference - * between how Iceberg schema (micro) and Flink {@link AvroToRowDataConverters} (milli) deal with - * time type, we can't directly use the Avro Schema converted from Iceberg schema via {@link - * AvroSchemaUtil#convert(org.apache.iceberg.Schema, String)}. - */ -public class AvroGenericRecordToRowDataMapper implements MapFunction { - - private final AvroToRowDataConverters.AvroToRowDataConverter converter; - - AvroGenericRecordToRowDataMapper(RowType rowType) { - this.converter = AvroToRowDataConverters.createRowConverter(rowType); - } - - @Override - public RowData map(GenericRecord genericRecord) throws Exception { - return (RowData) converter.convert(genericRecord); - } - - /** Create a mapper based on Avro schema. */ - public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { - DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); - return new AvroGenericRecordToRowDataMapper(rowType); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java deleted file mode 100644 index 40e0b5f2a34e..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ /dev/null @@ -1,125 +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.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.RowDataProjection; -import org.apache.iceberg.io.BaseTaskWriter; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; - -abstract class BaseDeltaTaskWriter extends BaseTaskWriter { - - private final Schema schema; - private final Schema deleteSchema; - private final RowDataWrapper wrapper; - private final RowDataWrapper keyWrapper; - private final RowDataProjection keyProjection; - private final boolean upsert; - - BaseDeltaTaskWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.schema = schema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); - this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - this.keyWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); - this.keyProjection = - RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); - this.upsert = upsert; - } - - abstract RowDataDeltaWriter route(RowData row); - - RowDataWrapper wrapper() { - return wrapper; - } - - @Override - public void write(RowData row) throws IOException { - RowDataDeltaWriter writer = route(row); - - switch (row.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - if (upsert) { - writer.deleteKey(keyProjection.wrap(row)); - } - writer.write(row); - break; - - case UPDATE_BEFORE: - if (upsert) { - break; // UPDATE_BEFORE is not necessary for UPSERT, we do nothing to prevent delete one - // row twice - } - writer.delete(row); - break; - case DELETE: - if (upsert) { - writer.deleteKey(keyProjection.wrap(row)); - } else { - writer.delete(row); - } - break; - - default: - throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind()); - } - } - - protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { - RowDataDeltaWriter(PartitionKey partition) { - super(partition, schema, deleteSchema); - } - - @Override - protected StructLike asStructLike(RowData data) { - return wrapper.wrap(data); - } - - @Override - protected StructLike asStructLikeKey(RowData data) { - return keyWrapper.wrap(data); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java deleted file mode 100644 index 1cb6e013bd2c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.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.flink.sink; - -import java.util.stream.IntStream; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; - -/** - * A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key. - * To be used with the {@link BucketPartitioner}. - */ -class BucketPartitionKeySelector implements KeySelector { - - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - private final int bucketFieldPosition; - - private transient RowDataWrapper rowDataWrapper; - - BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(partitionSpec, schema); - this.flinkSchema = flinkSchema; - this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); - } - - private int getBucketFieldPosition(PartitionSpec partitionSpec) { - int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); - return IntStream.range(0, partitionSpec.fields().size()) - .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) - .toArray()[0]; - } - - private RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - return rowDataWrapper; - } - - @Override - public Integer getKey(RowData rowData) { - partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); - return partitionKey.get(bucketFieldPosition, Integer.class); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java deleted file mode 100644 index 9c9a117906e2..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ /dev/null @@ -1,103 +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.sink; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * This partitioner will redirect records to writers deterministically based on the Bucket partition - * spec. It'll attempt to optimize the file size written depending on whether numPartitions is - * greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE - * bucket in the partition spec. - */ -class BucketPartitioner implements Partitioner { - - static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null"; - static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE = - "Invalid bucket ID %s: must be non-negative."; - static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE = - "Invalid bucket ID %s: must be less than bucket limit: %s."; - - private final int maxNumBuckets; - - // To hold the OFFSET of the next writer to use for any bucket, only used when writers > the - // number of buckets - private final int[] currentBucketWriterOffset; - - BucketPartitioner(PartitionSpec partitionSpec) { - this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); - this.currentBucketWriterOffset = new int[maxNumBuckets]; - } - - /** - * Determine the partition id based on the following criteria: If the number of writers <= the - * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one - * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic - * is handled by the {@link #getPartitionWithMoreWritersThanBuckets - * getPartitionWritersGreaterThanBuckets} method. - * - * @param bucketId the bucketId for each request - * @param numPartitions the total number of partitions - * @return the partition id (writer) to use for each request - */ - @Override - public int partition(Integer bucketId, int numPartitions) { - Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); - Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); - Preconditions.checkArgument( - bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); - - if (numPartitions <= maxNumBuckets) { - return bucketId % numPartitions; - } else { - return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); - } - } - - /*- - * If the number of writers > the number of buckets each partitioner will keep a state of multiple - * writers per bucket as evenly as possible, and will round-robin the requests across them, in this - * case each writer will target only one bucket at all times (many writers -> one bucket). Example: - * Configuration: numPartitions (writers) = 5, maxBuckets = 2 - * Expected behavior: - * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 - * - Records for Bucket 1 will always use Writer 1 and 3 - * Notes: - * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId - * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). - * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. - * - * @return the destination partition index (writer subtask id) - */ - private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { - int currentOffset = currentBucketWriterOffset[bucketId]; - // Determine if this bucket requires an "extra writer" - int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; - // The max number of writers this bucket can have - int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; - - // Increment the writer offset or reset if it's reached the max for this bucket - int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; - currentBucketWriterOffset[bucketId] = nextOffset; - - return bucketId + (maxNumBuckets * currentOffset); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java deleted file mode 100644 index c33207728d3e..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ /dev/null @@ -1,125 +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.sink; - -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.transforms.PartitionSpecVisitor; - -final class BucketPartitionerUtil { - static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = - "Invalid number of buckets: %s (must be 1)"; - - private BucketPartitionerUtil() {} - - /** - * Determines whether the PartitionSpec has one and only one Bucket definition - * - * @param partitionSpec the partition spec in question - * @return whether the PartitionSpec has only one Bucket - */ - static boolean hasOneBucketField(PartitionSpec partitionSpec) { - List> bucketFields = getBucketFields(partitionSpec); - return bucketFields != null && bucketFields.size() == 1; - } - - /** - * Extracts the Bucket definition from a PartitionSpec. - * - * @param partitionSpec the partition spec in question - * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) - */ - private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { - List> bucketFields = getBucketFields(partitionSpec); - Preconditions.checkArgument( - bucketFields.size() == 1, - BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, - bucketFields.size()); - return bucketFields.get(0); - } - - static int getBucketFieldId(PartitionSpec partitionSpec) { - return getBucketFieldInfo(partitionSpec).f0; - } - - static int getMaxNumBuckets(PartitionSpec partitionSpec) { - return getBucketFieldInfo(partitionSpec).f1; - } - - private static List> getBucketFields(PartitionSpec spec) { - return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() - .filter(Objects::nonNull) - .collect(Collectors.toList()); - } - - private static class BucketPartitionSpecVisitor - implements PartitionSpecVisitor> { - @Override - public Tuple2 identity(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 bucket( - int fieldId, String sourceName, int sourceId, int numBuckets) { - return new Tuple2<>(fieldId, numBuckets); - } - - @Override - public Tuple2 truncate( - int fieldId, String sourceName, int sourceId, int width) { - return null; - } - - @Override - public Tuple2 year(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 month(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 day(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 hour(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 unknown( - int fieldId, String sourceName, int sourceId, String transform) { - return null; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java deleted file mode 100644 index e9f9786f9190..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink; - -import java.time.Duration; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.SerializableSupplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A table loader that will only reload a table after a certain interval has passed. WARNING: This - * table loader should be used carefully when used with writer tasks. It could result in heavy load - * on a catalog for jobs with many writers. - */ -class CachingTableSupplier implements SerializableSupplier

  • { - - private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); - - private final Table initialTable; - private final TableLoader tableLoader; - private final Duration tableRefreshInterval; - private long lastLoadTimeMillis; - private transient Table table; - - CachingTableSupplier( - SerializableTable initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { - Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); - Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); - Preconditions.checkArgument( - tableRefreshInterval != null, "tableRefreshInterval cannot be null"); - this.initialTable = initialTable; - this.table = initialTable; - this.tableLoader = tableLoader; - this.tableRefreshInterval = tableRefreshInterval; - this.lastLoadTimeMillis = System.currentTimeMillis(); - } - - @Override - public Table get() { - if (table == null) { - this.table = initialTable; - } - return table; - } - - Table initialTable() { - return initialTable; - } - - void refreshTable() { - if (System.currentTimeMillis() > lastLoadTimeMillis + tableRefreshInterval.toMillis()) { - try { - if (!tableLoader.isOpen()) { - tableLoader.open(); - } - - this.table = tableLoader.loadTable(); - this.lastLoadTimeMillis = System.currentTimeMillis(); - - LOG.info( - "Table {} reloaded, next min load time threshold is {}", - table.name(), - DateTimeUtil.formatTimestampMillis( - lastLoadTimeMillis + tableRefreshInterval.toMillis())); - } catch (Exception e) { - LOG.warn("An error occurred reloading table {}, table was not reloaded", table.name(), e); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java deleted file mode 100644 index 9a2f57181708..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ /dev/null @@ -1,93 +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.sink; - -import java.util.Arrays; -import java.util.NavigableMap; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -class CommitSummary { - - private final AtomicLong dataFilesCount = new AtomicLong(); - private final AtomicLong dataFilesRecordCount = new AtomicLong(); - private final AtomicLong dataFilesByteCount = new AtomicLong(); - private final AtomicLong deleteFilesCount = new AtomicLong(); - private final AtomicLong deleteFilesRecordCount = new AtomicLong(); - private final AtomicLong deleteFilesByteCount = new AtomicLong(); - - CommitSummary(NavigableMap pendingResults) { - pendingResults - .values() - .forEach( - writeResult -> { - dataFilesCount.addAndGet(writeResult.dataFiles().length); - Arrays.stream(writeResult.dataFiles()) - .forEach( - dataFile -> { - dataFilesRecordCount.addAndGet(dataFile.recordCount()); - dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); - }); - deleteFilesCount.addAndGet(writeResult.deleteFiles().length); - Arrays.stream(writeResult.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); - }); - }); - } - - long dataFilesCount() { - return dataFilesCount.get(); - } - - long dataFilesRecordCount() { - return dataFilesRecordCount.get(); - } - - long dataFilesByteCount() { - return dataFilesByteCount.get(); - } - - long deleteFilesCount() { - return deleteFilesCount.get(); - } - - long deleteFilesRecordCount() { - return deleteFilesRecordCount.get(); - } - - long deleteFilesByteCount() { - return deleteFilesByteCount.get(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("dataFilesCount", dataFilesCount) - .add("dataFilesRecordCount", dataFilesRecordCount) - .add("dataFilesByteCount", dataFilesByteCount) - .add("deleteFilesCount", deleteFilesCount) - .add("deleteFilesRecordCount", deleteFilesRecordCount) - .add("deleteFilesByteCount", deleteFilesByteCount) - .toString(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java deleted file mode 100644 index 036970c06d5b..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.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.flink.sink; - -import java.util.List; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class DeltaManifests { - - private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; - - private final ManifestFile dataManifest; - private final ManifestFile deleteManifest; - private final CharSequence[] referencedDataFiles; - - DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { - this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); - } - - DeltaManifests( - ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { - Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); - - this.dataManifest = dataManifest; - this.deleteManifest = deleteManifest; - this.referencedDataFiles = referencedDataFiles; - } - - ManifestFile dataManifest() { - return dataManifest; - } - - ManifestFile deleteManifest() { - return deleteManifest; - } - - CharSequence[] referencedDataFiles() { - return referencedDataFiles; - } - - List manifests() { - List manifests = Lists.newArrayListWithCapacity(2); - if (dataManifest != null) { - manifests.add(dataManifest); - } - - if (deleteManifest != null) { - manifests.add(deleteManifest); - } - - return manifests; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java deleted file mode 100644 index 92ca284b12ba..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ /dev/null @@ -1,122 +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.sink; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class DeltaManifestsSerializer implements SimpleVersionedSerializer { - private static final int VERSION_1 = 1; - private static final int VERSION_2 = 2; - private static final byte[] EMPTY_BINARY = new byte[0]; - - static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); - - @Override - public int getVersion() { - return VERSION_2; - } - - @Override - public byte[] serialize(DeltaManifests deltaManifests) throws IOException { - Preconditions.checkNotNull( - deltaManifests, "DeltaManifests to be serialized should not be null"); - - ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(binaryOut); - - byte[] dataManifestBinary = EMPTY_BINARY; - if (deltaManifests.dataManifest() != null) { - dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); - } - - out.writeInt(dataManifestBinary.length); - out.write(dataManifestBinary); - - byte[] deleteManifestBinary = EMPTY_BINARY; - if (deltaManifests.deleteManifest() != null) { - deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); - } - - out.writeInt(deleteManifestBinary.length); - out.write(deleteManifestBinary); - - CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); - out.writeInt(referencedDataFiles.length); - for (CharSequence referencedDataFile : referencedDataFiles) { - out.writeUTF(referencedDataFile.toString()); - } - - return binaryOut.toByteArray(); - } - - @Override - public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { - if (version == VERSION_1) { - return deserializeV1(serialized); - } else if (version == VERSION_2) { - return deserializeV2(serialized); - } else { - throw new RuntimeException("Unknown serialize version: " + version); - } - } - - private DeltaManifests deserializeV1(byte[] serialized) throws IOException { - return new DeltaManifests(ManifestFiles.decode(serialized), null); - } - - private DeltaManifests deserializeV2(byte[] serialized) throws IOException { - ManifestFile dataManifest = null; - ManifestFile deleteManifest = null; - - ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); - DataInputStream in = new DataInputStream(binaryIn); - - int dataManifestSize = in.readInt(); - if (dataManifestSize > 0) { - byte[] dataManifestBinary = new byte[dataManifestSize]; - Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); - - dataManifest = ManifestFiles.decode(dataManifestBinary); - } - - int deleteManifestSize = in.readInt(); - if (deleteManifestSize > 0) { - byte[] deleteManifestBinary = new byte[deleteManifestSize]; - Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); - - deleteManifest = ManifestFiles.decode(deleteManifestBinary); - } - - int referenceDataFileNum = in.readInt(); - CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; - for (int i = 0; i < referenceDataFileNum; i++) { - referencedDataFiles[i] = in.readUTF(); - } - - return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java deleted file mode 100644 index 18b269d6c3e9..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ /dev/null @@ -1,86 +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.sink; - -import java.util.List; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.StructLikeWrapper; -import org.apache.iceberg.util.StructProjection; - -/** - * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record - * will be emitted to same writer in order. - */ -class EqualityFieldKeySelector implements KeySelector { - - private final Schema schema; - private final RowType flinkSchema; - private final Schema deleteSchema; - - private transient RowDataWrapper rowDataWrapper; - private transient StructProjection structProjection; - private transient StructLikeWrapper structLikeWrapper; - - EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { - this.schema = schema; - this.flinkSchema = flinkSchema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); - } - - /** - * Construct the {@link RowDataWrapper} lazily here because few members in it are not - * serializable. In this way, we don't have to serialize them with forcing. - */ - protected RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - return rowDataWrapper; - } - - /** Construct the {@link StructProjection} lazily because it is not serializable. */ - protected StructProjection lazyStructProjection() { - if (structProjection == null) { - structProjection = StructProjection.create(schema, deleteSchema); - } - return structProjection; - } - - /** Construct the {@link StructLikeWrapper} lazily because it is not serializable. */ - protected StructLikeWrapper lazyStructLikeWrapper() { - if (structLikeWrapper == null) { - structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct()); - } - return structLikeWrapper; - } - - @Override - public Integer getKey(RowData row) { - RowDataWrapper wrappedRowData = lazyRowDataWrapper().wrap(row); - StructProjection projectedRowData = lazyStructProjection().wrap(wrappedRowData); - StructLikeWrapper wrapper = lazyStructLikeWrapper().set(projectedRowData); - return wrapper.hashCode(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java deleted file mode 100644 index b6f1392d1562..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java +++ /dev/null @@ -1,274 +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.sink; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -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.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -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; - -public class FlinkAppenderFactory implements FileAppenderFactory, Serializable { - private final Schema schema; - private final RowType flinkSchema; - private final Map props; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - private final Table table; - - private RowType eqDeleteFlinkSchema = null; - private RowType posDeleteFlinkSchema = null; - - public FlinkAppenderFactory( - Table table, - Schema schema, - RowType flinkSchema, - Map props, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema) { - Preconditions.checkNotNull(table, "Table shouldn't be null"); - this.table = table; - this.schema = schema; - this.flinkSchema = flinkSchema; - this.props = props; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - } - - private RowType lazyEqDeleteFlinkSchema() { - if (eqDeleteFlinkSchema == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteFlinkSchema; - } - - private RowType lazyPosDeleteFlinkSchema() { - if (posDeleteFlinkSchema == null) { - Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null"); - this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema); - } - return this.posDeleteFlinkSchema; - } - - @Override - public FileAppender newAppender(OutputFile outputFile, FileFormat format) { - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - try { - switch (format) { - case AVRO: - return Avro.write(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .setAll(props) - .schema(schema) - .metricsConfig(metricsConfig) - .overwrite() - .build(); - - case ORC: - return ORC.write(outputFile) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - case PARQUET: - return Parquet.write(outputFile) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(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 outputFile, 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"); - - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - try { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case PARQUET: - return Parquet.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { - MetricsConfig metricsConfig = MetricsConfig.forPositionDelete(table); - try { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .buildPositionWriter(); - - case ORC: - RowType orcPosDeleteSchema = - FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return ORC.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .transformPaths(path -> StringData.fromString(path.toString())) - .buildPositionWriter(); - - case PARQUET: - RowType flinkPosDeleteSchema = - FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(outputFile.encryptingOutputFile()) - .createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .transformPaths(path -> StringData.fromString(path.toString())) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java deleted file mode 100644 index 2183fe062af4..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java +++ /dev/null @@ -1,293 +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.sink; - -import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; -import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; - -import java.io.Serializable; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.BaseFileWriterFactory; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { - private RowType dataFlinkType; - private RowType equalityDeleteFlinkType; - private RowType positionDeleteFlinkType; - - FlinkFileWriterFactory( - Table table, - FileFormat dataFileFormat, - Schema dataSchema, - RowType dataFlinkType, - SortOrder dataSortOrder, - FileFormat deleteFileFormat, - int[] equalityFieldIds, - Schema equalityDeleteRowSchema, - RowType equalityDeleteFlinkType, - SortOrder equalityDeleteSortOrder, - Schema positionDeleteRowSchema, - RowType positionDeleteFlinkType) { - - super( - table, - dataFileFormat, - dataSchema, - dataSortOrder, - deleteFileFormat, - equalityFieldIds, - equalityDeleteRowSchema, - equalityDeleteSortOrder, - positionDeleteRowSchema); - - this.dataFlinkType = dataFlinkType; - this.equalityDeleteFlinkType = equalityDeleteFlinkType; - this.positionDeleteFlinkType = positionDeleteFlinkType; - } - - static Builder builderFor(Table table) { - return new Builder(table); - } - - @Override - protected void configureDataWrite(Avro.DataWriteBuilder builder) { - builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); - } - - @Override - protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); - } - - @Override - protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { - int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME); - if (rowFieldIndex >= 0) { - // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos - RowType positionDeleteRowFlinkType = - (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex); - builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType)); - } - } - - @Override - protected void configureDataWrite(Parquet.DataWriteBuilder builder) { - builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); - } - - @Override - protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); - } - - @Override - protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType)); - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - @Override - protected void configureDataWrite(ORC.DataWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); - } - - @Override - protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); - } - - @Override - protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema)); - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - private RowType dataFlinkType() { - if (dataFlinkType == null) { - Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); - this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); - } - - return dataFlinkType; - } - - private RowType equalityDeleteFlinkType() { - if (equalityDeleteFlinkType == null) { - Preconditions.checkNotNull( - equalityDeleteRowSchema(), "Equality delete schema must not be null"); - this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); - } - - return equalityDeleteFlinkType; - } - - private RowType positionDeleteFlinkType() { - if (positionDeleteFlinkType == null) { - // wrap the optional row schema into the position delete schema that contains path and - // position - Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); - this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema); - } - - return positionDeleteFlinkType; - } - - static class Builder { - private final Table table; - private FileFormat dataFileFormat; - private Schema dataSchema; - private RowType dataFlinkType; - private SortOrder dataSortOrder; - private FileFormat deleteFileFormat; - private int[] equalityFieldIds; - private Schema equalityDeleteRowSchema; - private RowType equalityDeleteFlinkType; - private SortOrder equalityDeleteSortOrder; - private Schema positionDeleteRowSchema; - private RowType positionDeleteFlinkType; - - Builder(Table table) { - this.table = table; - - Map properties = table.properties(); - - String dataFileFormatName = - properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); - this.dataFileFormat = FileFormat.fromString(dataFileFormatName); - - String deleteFileFormatName = - properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); - this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); - } - - Builder dataFileFormat(FileFormat newDataFileFormat) { - this.dataFileFormat = newDataFileFormat; - return this; - } - - Builder dataSchema(Schema newDataSchema) { - this.dataSchema = newDataSchema; - return this; - } - - /** - * Sets a Flink type for data. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder dataFlinkType(RowType newDataFlinkType) { - this.dataFlinkType = newDataFlinkType; - return this; - } - - Builder dataSortOrder(SortOrder newDataSortOrder) { - this.dataSortOrder = newDataSortOrder; - return this; - } - - Builder deleteFileFormat(FileFormat newDeleteFileFormat) { - this.deleteFileFormat = newDeleteFileFormat; - return this; - } - - Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { - this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; - return this; - } - - /** - * Sets a Flink type for equality deletes. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) { - this.equalityDeleteFlinkType = newEqualityDeleteFlinkType; - return this; - } - - Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { - this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; - return this; - } - - Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { - this.positionDeleteRowSchema = newPositionDeleteRowSchema; - return this; - } - - /** - * Sets a Flink type for position deletes. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) { - this.positionDeleteFlinkType = newPositionDeleteFlinkType; - return this; - } - - FlinkFileWriterFactory build() { - boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; - boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; - Preconditions.checkArgument( - noEqualityDeleteConf || fullEqualityDeleteConf, - "Equality field IDs and equality delete row schema must be set together"); - - return new FlinkFileWriterFactory( - table, - dataFileFormat, - dataSchema, - dataFlinkType, - dataSortOrder, - deleteFileFormat, - equalityFieldIds, - equalityDeleteRowSchema, - equalityDeleteFlinkType, - equalityDeleteSortOrder, - positionDeleteRowSchema, - positionDeleteFlinkType); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java deleted file mode 100644 index c7e8a2dea7cb..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ /dev/null @@ -1,132 +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.sink; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class FlinkManifestUtil { - private static final int FORMAT_V2 = 2; - private static final Long DUMMY_SNAPSHOT_ID = 0L; - - private FlinkManifestUtil() {} - - static ManifestFile writeDataFiles( - OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { - ManifestWriter writer = - ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); - - try (ManifestWriter closeableWriter = writer) { - closeableWriter.addAll(dataFiles); - } - - return writer.toManifestFile(); - } - - static List readDataFiles( - ManifestFile manifestFile, FileIO io, Map specsById) - throws IOException { - try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io, specsById)) { - return Lists.newArrayList(dataFiles); - } - } - - static ManifestOutputFileFactory createOutputFileFactory( - Supplier

    tableSupplier, - Map tableProps, - String flinkJobId, - String operatorUniqueId, - int subTaskId, - long attemptNumber) { - return new ManifestOutputFileFactory( - tableSupplier, tableProps, flinkJobId, operatorUniqueId, subTaskId, attemptNumber); - } - - /** - * Write the {@link WriteResult} to temporary manifest files. - * - * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same - * partition spec - */ - static DeltaManifests writeCompletedFiles( - WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) - throws IOException { - - ManifestFile dataManifest = null; - ManifestFile deleteManifest = null; - - // Write the completed data files into a newly created data manifest file. - if (result.dataFiles() != null && result.dataFiles().length > 0) { - dataManifest = - writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); - } - - // Write the completed delete files into a newly created delete manifest file. - if (result.deleteFiles() != null && result.deleteFiles().length > 0) { - OutputFile deleteManifestFile = outputFileSupplier.get(); - - ManifestWriter deleteManifestWriter = - ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); - try (ManifestWriter writer = deleteManifestWriter) { - for (DeleteFile deleteFile : result.deleteFiles()) { - writer.add(deleteFile); - } - } - - deleteManifest = deleteManifestWriter.toManifestFile(); - } - - return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); - } - - static WriteResult readCompletedFiles( - DeltaManifests deltaManifests, FileIO io, Map specsById) - throws IOException { - WriteResult.Builder builder = WriteResult.builder(); - - // Read the completed data files from persisted data manifest file. - if (deltaManifests.dataManifest() != null) { - builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io, specsById)); - } - - // Read the completed delete files from persisted delete manifests file. - if (deltaManifests.deleteManifest() != null) { - try (CloseableIterable deleteFiles = - ManifestFiles.readDeleteManifest(deltaManifests.deleteManifest(), io, specsById)) { - builder.addDeleteFiles(deleteFiles); - } - } - - return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java deleted file mode 100644 index 769af7d77140..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ /dev/null @@ -1,654 +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.sink; - -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.Configuration; -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.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; -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.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.SerializableSupplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class FlinkSink { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class); - - private static final String ICEBERG_STREAM_WRITER_NAME = - IcebergStreamWriter.class.getSimpleName(); - private static final String ICEBERG_FILES_COMMITTER_NAME = - IcebergFilesCommitter.class.getSimpleName(); - - private FlinkSink() {} - - /** - * Initialize a {@link Builder} to export the data from generic input data stream into iceberg - * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper - * function and a {@link TypeInformation} to convert those generic records to a RowData - * DataStream. - * - * @param input the generic source input data stream. - * @param mapper function to convert the generic data to {@link RowData} - * @param outputType to define the {@link TypeInformation} for the input data. - * @param the data type of records. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder builderFor( - DataStream input, MapFunction mapper, TypeInformation outputType) { - return new Builder().forMapperOutputType(input, mapper, outputType); - } - - /** - * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into - * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a - * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. - * - * @param input the source input data stream with {@link Row}s. - * @param tableSchema defines the {@link TypeInformation} for input data. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRow(DataStream input, TableSchema tableSchema) { - RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); - - DataFormatConverters.RowConverter rowConverter = - new DataFormatConverters.RowConverter(fieldDataTypes); - return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) - .tableSchema(tableSchema); - } - - /** - * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s - * into iceberg table. - * - * @param input the source input data stream with {@link RowData}s. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRowData(DataStream input) { - return new Builder().forRowData(input); - } - - public static class Builder { - private Function> inputCreator = null; - private TableLoader tableLoader; - private Table table; - private TableSchema tableSchema; - private List equalityFieldColumns = null; - private String uidPrefix = null; - private final Map snapshotProperties = Maps.newHashMap(); - private ReadableConfig readableConfig = new Configuration(); - private final Map writeOptions = Maps.newHashMap(); - private FlinkWriteConf flinkWriteConf = null; - - private Builder() {} - - private Builder forRowData(DataStream newRowDataInput) { - this.inputCreator = ignored -> newRowDataInput; - return this; - } - - private Builder forMapperOutputType( - DataStream input, MapFunction mapper, TypeInformation outputType) { - this.inputCreator = - newUidPrefix -> { - // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we - // need to set the parallelism - // of map operator same as its input to keep map operator chaining its input, and avoid - // rebalanced by default. - SingleOutputStreamOperator inputStream = - input.map(mapper, outputType).setParallelism(input.getParallelism()); - if (newUidPrefix != null) { - inputStream.name(operatorName(newUidPrefix)).uid(newUidPrefix + "-mapper"); - } - return inputStream; - }; - return this; - } - - /** - * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} - * which will write all the records into {@link DataFile}s and emit them to downstream operator. - * Providing a table would avoid so many table loading from each separate task. - * - * @param newTable the loaded iceberg table instance. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - /** - * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need - * this loader because {@link Table} is not serializable and could not just use the loaded table - * from Builder#table in the remote task manager. - * - * @param newTableLoader to load iceberg table inside tasks. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder tableLoader(TableLoader newTableLoader) { - this.tableLoader = newTableLoader; - return this; - } - - /** - * Set the write properties for Flink sink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder set(String property, String value) { - writeOptions.put(property, value); - return this; - } - - /** - * Set the write properties for Flink sink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder setAll(Map properties) { - writeOptions.putAll(properties); - return this; - } - - public Builder tableSchema(TableSchema newTableSchema) { - this.tableSchema = newTableSchema; - return this; - } - - public Builder overwrite(boolean newOverwrite) { - writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - /** - * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink - * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. - * - * @param mode to specify the write distribution mode. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); - if (mode != null) { - writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); - } - return this; - } - - /** - * Configuring the write parallel number for iceberg stream writer. - * - * @param newWriteParallelism the number of parallel iceberg stream writer. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder writeParallelism(int newWriteParallelism) { - writeOptions.put( - FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); - return this; - } - - /** - * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which - * means it will DELETE the old records and then INSERT the new records. In partitioned table, - * the partition fields should be a subset of equality fields, otherwise the old row that - * located in partition-A could not be deleted by the new row that located in partition-B. - * - * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder upsert(boolean enabled) { - writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); - return this; - } - - /** - * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. - * - * @param columns defines the iceberg table's key. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder equalityFieldColumns(List columns) { - this.equalityFieldColumns = columns; - return this; - } - - /** - * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of - * multiple operators (like writer, committer, dummy sink etc.) Actually operator uid will be - * appended with a suffix like "uidPrefix-writer".
    - *
    - * If provided, this prefix is also applied to operator names.
    - *
    - * Flink auto generates operator uid if not set explicitly. It is a recommended - * best-practice to set uid for all operators before deploying to production. Flink has an - * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force - * explicit setting of all operator uid.
    - *
    - * Be careful with setting this for an existing job, because now we are changing the operator - * uid from an auto-generated one to this new value. When deploying the change with a - * checkpoint, Flink won't be able to restore the previous Flink sink operator state (more - * specifically the committer operator state). You need to use {@code --allowNonRestoredState} - * to ignore the previous sink state. During restore Flink sink state is used to check if last - * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss - * if the Iceberg commit failed in the last completed checkpoint. - * - * @param newPrefix prefix for Flink sink operator uid and name - * @return {@link Builder} to connect the iceberg table. - */ - public Builder uidPrefix(String newPrefix) { - this.uidPrefix = newPrefix; - return this; - } - - public Builder setSnapshotProperties(Map properties) { - snapshotProperties.putAll(properties); - return this; - } - - public Builder setSnapshotProperty(String property, String value) { - snapshotProperties.put(property, value); - return this; - } - - public Builder toBranch(String branch) { - writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); - return this; - } - - private DataStreamSink chainIcebergOperators() { - Preconditions.checkArgument( - inputCreator != null, - "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); - Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); - - DataStream rowDataInput = inputCreator.apply(uidPrefix); - - if (table == null) { - if (!tableLoader.isOpen()) { - tableLoader.open(); - } - - try (TableLoader loader = tableLoader) { - this.table = loader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to load iceberg table from table loader: " + tableLoader, e); - } - } - - flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); - - // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); - - // Convert the requested flink table schema to flink row type. - RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); - - // Distribute the records from input data stream based on the write.distribution-mode and - // equality fields. - DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); - - // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); - - // Add single-parallelism committer that commits files - // after successful checkpoint or end of input - SingleOutputStreamOperator committerStream = appendCommitter(writerStream); - - // Add dummy discard sink - return appendDummySink(committerStream); - } - - /** - * Append the iceberg sink operators to write records to iceberg table. - * - * @return {@link DataStreamSink} for sink. - */ - public DataStreamSink append() { - return chainIcebergOperators(); - } - - private String operatorName(String suffix) { - return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; - } - - @VisibleForTesting - List checkAndGetEqualityFieldIds() { - List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); - if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { - Set equalityFieldSet = - Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); - for (String column : equalityFieldColumns) { - org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); - Preconditions.checkNotNull( - field, - "Missing required equality field column '%s' in table schema %s", - column, - table.schema()); - equalityFieldSet.add(field.fieldId()); - } - - if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { - LOG.warn( - "The configured equality field column IDs {} are not matched with the schema identifier field IDs" - + " {}, use job specified equality field columns as the equality fields by default.", - equalityFieldSet, - table.schema().identifierFieldIds()); - } - equalityFieldIds = Lists.newArrayList(equalityFieldSet); - } - return equalityFieldIds; - } - - @SuppressWarnings("unchecked") - private DataStreamSink appendDummySink( - SingleOutputStreamOperator committerStream) { - DataStreamSink resultStream = - committerStream - .addSink(new DiscardingSink()) - .name(operatorName(String.format("IcebergSink %s", this.table.name()))) - .setParallelism(1); - if (uidPrefix != null) { - resultStream = resultStream.uid(uidPrefix + "-dummysink"); - } - return resultStream; - } - - private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { - IcebergFilesCommitter filesCommitter = - new IcebergFilesCommitter( - tableLoader, - flinkWriteConf.overwriteMode(), - snapshotProperties, - flinkWriteConf.workerPoolSize(), - flinkWriteConf.branch(), - table.spec()); - SingleOutputStreamOperator committerStream = - writerStream - .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) - .setParallelism(1) - .setMaxParallelism(1); - if (uidPrefix != null) { - committerStream = committerStream.uid(uidPrefix + "-committer"); - } - return committerStream; - } - - private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { - // Validate the equality fields and partition fields if we enable the upsert mode. - if (flinkWriteConf.upsertMode()) { - Preconditions.checkState( - !flinkWriteConf.overwriteMode(), - "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - Preconditions.checkState( - !equalityFieldIds.isEmpty(), - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); - if (!table.spec().isUnpartitioned()) { - for (PartitionField partitionField : table.spec().fields()) { - Preconditions.checkState( - equalityFieldIds.contains(partitionField.sourceId()), - "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", - partitionField, - equalityFieldColumns); - } - } - } - - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); - - SerializableSupplier
    tableSupplier; - if (tableRefreshInterval != null) { - tableSupplier = - new CachingTableSupplier(serializableTable, tableLoader, tableRefreshInterval); - } else { - tableSupplier = () -> serializableTable; - } - - IcebergStreamWriter streamWriter = - createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator writerStream = - input - .transform( - operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), - streamWriter) - .setParallelism(parallelism); - if (uidPrefix != null) { - writerStream = writerStream.uid(uidPrefix + "-writer"); - } - return writerStream; - } - - private DataStream distributeDataStream( - DataStream input, - List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { - DistributionMode writeMode = flinkWriteConf.distributionMode(); - - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); - switch (writeMode) { - case NONE: - if (equalityFieldIds.isEmpty()) { - return input; - } else { - LOG.info("Distribute rows by equality fields, because there are equality fields set"); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } - - case HASH: - if (equalityFieldIds.isEmpty()) { - if (partitionSpec.isUnpartitioned()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and table is unpartitioned"); - return input; - } else { - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); - } - } else { - if (partitionSpec.isUnpartitioned()) { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and table is unpartitioned"); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } else { - for (PartitionField partitionField : partitionSpec.fields()) { - Preconditions.checkState( - equalityFieldIds.contains(partitionField.sourceId()), - "In 'hash' distribution mode with equality fields set, partition field '%s' " - + "should be included in equality fields: '%s'", - partitionField, - equalityFieldColumns); - } - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); - } - } - - case RANGE: - if (equalityFieldIds.isEmpty()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } - - default: - throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); - } - } - } - - static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { - if (requestedSchema != null) { - // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing - // iceberg schema. - Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); - TypeUtil.validateWriteSchema(schema, writeSchema, true, true); - - // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. - return (RowType) requestedSchema.toRowDataType().getLogicalType(); - } else { - return FlinkSchemaUtil.convert(schema); - } - } - - static IcebergStreamWriter createStreamWriter( - SerializableSupplier
    tableSupplier, - FlinkWriteConf flinkWriteConf, - RowType flinkRowType, - List equalityFieldIds) { - Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); - - Table initTable = tableSupplier.get(); - FileFormat format = flinkWriteConf.dataFileFormat(); - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - tableSupplier, - flinkRowType, - flinkWriteConf.targetDataFileSize(), - format, - writeProperties(initTable, format, flinkWriteConf), - equalityFieldIds, - flinkWriteConf.upsertMode()); - - return new IcebergStreamWriter<>(initTable.name(), taskWriterFactory); - } - - /** - * Based on the {@link FileFormat} overwrites the table level compression properties for the table - * write. - * - * @param table The table to get the table level settings - * @param format The FileFormat to use - * @param conf The write configuration - * @return The properties to use for writing - */ - private static Map writeProperties( - Table table, FileFormat format, FlinkWriteConf conf) { - Map writeProperties = Maps.newHashMap(table.properties()); - - switch (format) { - case PARQUET: - writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); - String parquetCompressionLevel = conf.parquetCompressionLevel(); - if (parquetCompressionLevel != null) { - writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); - } - - break; - case AVRO: - writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); - String avroCompressionLevel = conf.avroCompressionLevel(); - if (avroCompressionLevel != null) { - writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); - } - - break; - case ORC: - writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); - writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); - break; - default: - throw new IllegalArgumentException(String.format("Unknown file format %s", format)); - } - - return writeProperties; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java deleted file mode 100644 index b9bceaa9311d..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ /dev/null @@ -1,516 +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.sink; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.SortedMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.ReplacePartitions; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotUpdate; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.WriteResult; -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; -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.types.Comparators; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private static final long serialVersionUID = 1L; - private static final long INITIAL_CHECKPOINT_ID = -1L; - private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; - - private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class); - private static final String FLINK_JOB_ID = "flink.job-id"; - private static final String OPERATOR_ID = "flink.operator-id"; - - // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always - // increasing, so we could correctly commit all the data files whose checkpoint id is greater than - // the max committed one to iceberg table, for avoiding committing the same data files twice. This - // id will be attached to iceberg's meta when committing the iceberg transaction. - private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; - - // TableLoader to load iceberg table lazily. - private final TableLoader tableLoader; - private final boolean replacePartitions; - private final Map snapshotProperties; - - // A sorted map to maintain the completed data files for each pending checkpointId (which have not - // been committed to iceberg table). We need a sorted map here because there's possible that few - // checkpoints snapshot failed, for example: the 1st checkpoint have 2 data files <1, >, the 2st checkpoint have 1 data files <2, >. Snapshot for checkpoint#1 - // interrupted because of network/disk failure etc, while we don't expect any data loss in iceberg - // table. So we keep the finished files <1, > in memory and retry to commit iceberg - // table when the next checkpoint happen. - private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); - - // The completed files cache for current checkpoint. Once the snapshot barrier received, it will - // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); - private final String branch; - - // It will have an unique identifier for one job. - private transient String flinkJobId; - private transient String operatorUniqueId; - private transient Table table; - private transient IcebergFilesCommitterMetrics committerMetrics; - private transient ManifestOutputFileFactory manifestOutputFileFactory; - private transient long maxCommittedCheckpointId; - private transient int continuousEmptyCheckpoints; - private transient int maxContinuousEmptyCommits; - // There're two cases that we restore from flink checkpoints: the first case is restoring from - // snapshot created by the same flink job; another case is restoring from snapshot created by - // another different job. For the second case, we need to maintain the old flink job's id in flink - // state backend to find the max-committed-checkpoint-id when traversing iceberg table's - // snapshots. - private static final ListStateDescriptor JOB_ID_DESCRIPTOR = - new ListStateDescriptor<>("iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); - private transient ListState jobIdState; - // All pending checkpoints states for this function. - private static final ListStateDescriptor> STATE_DESCRIPTOR = - buildStateDescriptor(); - private transient ListState> checkpointsState; - - private final Integer workerPoolSize; - private final PartitionSpec spec; - private transient ExecutorService workerPool; - - IcebergFilesCommitter( - TableLoader tableLoader, - boolean replacePartitions, - Map snapshotProperties, - Integer workerPoolSize, - String branch, - PartitionSpec spec) { - this.tableLoader = tableLoader; - this.replacePartitions = replacePartitions; - this.snapshotProperties = snapshotProperties; - this.workerPoolSize = workerPoolSize; - this.branch = branch; - this.spec = spec; - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); - this.operatorUniqueId = getRuntimeContext().getOperatorUniqueID(); - - // Open the table loader and load the table. - this.tableLoader.open(); - this.table = tableLoader.loadTable(); - this.committerMetrics = new IcebergFilesCommitterMetrics(super.metrics, table.name()); - - maxContinuousEmptyCommits = - PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); - Preconditions.checkArgument( - maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); - - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - int attemptId = getRuntimeContext().getAttemptNumber(); - this.manifestOutputFileFactory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); - this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); - this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); - if (context.isRestored()) { - Iterable jobIdIterable = jobIdState.get(); - if (jobIdIterable == null || !jobIdIterable.iterator().hasNext()) { - LOG.warn( - "Failed to restore committer state. This can happen when operator uid changed and Flink " - + "allowNonRestoredState is enabled. Best practice is to explicitly set the operator id " - + "via FlinkSink#Builder#uidPrefix() so that the committer operator uid is stable. " - + "Otherwise, Flink auto generate an operator uid based on job topology." - + "With that, operator uid is subjective to change upon topology change."); - return; - } - - String restoredFlinkJobId = jobIdIterable.iterator().next(); - Preconditions.checkState( - !Strings.isNullOrEmpty(restoredFlinkJobId), - "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); - - // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new - // flink job even if it's restored from a snapshot created by another different flink job, so - // it's safe to assign the max committed checkpoint id from restored flink job to the current - // flink job. - this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); - - NavigableMap uncommittedDataFiles = - Maps.newTreeMap(checkpointsState.get().iterator().next()) - .tailMap(maxCommittedCheckpointId, false); - if (!uncommittedDataFiles.isEmpty()) { - // Committed all uncommitted data files from the old flink job to iceberg table. - long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey(); - commitUpToCheckpoint( - uncommittedDataFiles, restoredFlinkJobId, operatorUniqueId, maxUncommittedCheckpointId); - } - } - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - long checkpointId = context.getCheckpointId(); - LOG.info( - "Start to flush snapshot state to state backend, table: {}, checkpointId: {}", - table, - checkpointId); - - // Update the checkpoint state. - long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); - // Reset the snapshot state to the latest state. - checkpointsState.clear(); - checkpointsState.add(dataFilesPerCheckpoint); - - jobIdState.clear(); - jobIdState.add(flinkJobId); - - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); - committerMetrics.checkpointDuration( - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - super.notifyCheckpointComplete(checkpointId); - // It's possible that we have the following events: - // 1. snapshotState(ckpId); - // 2. snapshotState(ckpId+1); - // 3. notifyCheckpointComplete(ckpId+1); - // 4. notifyCheckpointComplete(ckpId); - // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all - // the files, - // Besides, we need to maintain the max-committed-checkpoint-id to be increasing. - if (checkpointId > maxCommittedCheckpointId) { - LOG.info("Checkpoint {} completed. Attempting commit.", checkpointId); - commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, checkpointId); - this.maxCommittedCheckpointId = checkpointId; - } else { - LOG.info( - "Skipping committing checkpoint {}. {} is already committed.", - checkpointId, - maxCommittedCheckpointId); - } - - // reload the table in case new configuration is needed - this.table = tableLoader.loadTable(); - } - - private void commitUpToCheckpoint( - NavigableMap deltaManifestsMap, - String newFlinkJobId, - String operatorId, - long checkpointId) - throws IOException { - NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); - List manifests = Lists.newArrayList(); - NavigableMap pendingResults = Maps.newTreeMap(); - for (Map.Entry e : pendingMap.entrySet()) { - if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { - // Skip the empty flink manifest. - continue; - } - - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, e.getValue()); - pendingResults.put( - e.getKey(), - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); - manifests.addAll(deltaManifests.manifests()); - } - - CommitSummary summary = new CommitSummary(pendingResults); - commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - committerMetrics.updateCommitSummary(summary); - pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); - } - - private void commitPendingResult( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); - continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; - if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { - if (replacePartitions) { - replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - } else { - commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - } - continuousEmptyCheckpoints = 0; - } else { - LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); - } - } - - private void deleteCommittedManifests( - List manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - - private void replacePartitions( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - Preconditions.checkState( - summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); - // Commit the overwrite transaction. - ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); - for (WriteResult result : pendingResults.values()) { - Preconditions.checkState( - result.referencedDataFiles().length == 0, "Should have no referenced data files."); - Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); - } - - commitOperation( - dynamicOverwrite, - summary, - "dynamic partition overwrite", - newFlinkJobId, - operatorId, - checkpointId); - } - - private void commitDeltaTxn( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - if (summary.deleteFilesCount() == 0) { - // To be compatible with iceberg format V1. - AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); - for (WriteResult result : pendingResults.values()) { - Preconditions.checkState( - result.referencedDataFiles().length == 0, - "Should have no referenced data files for append."); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - } - commitOperation(appendFiles, summary, "append", newFlinkJobId, operatorId, checkpointId); - } else { - // To be compatible with iceberg format V2. - for (Map.Entry e : pendingResults.entrySet()) { - // We don't commit the merged result into a single transaction because for the sequential - // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied - // to data files from txn1. Committing the merged one will lead to the incorrect delete - // semantic. - WriteResult result = e.getValue(); - - // Row delta validations are not needed for streaming changes that write equality deletes. - // Equality deletes are applied to data in all previous sequence numbers, so retries may - // push deletes further in the future, but do not affect correctness. Position deletes - // committed to the table in this path are used only to delete rows from data files that are - // being added in this commit. There is no way for data files added along with the delete - // files to be concurrently removed, so there is no need to validate the files referenced by - // the position delete files that are being committed. - RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); - - Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); - Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); - commitOperation(rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); - } - } - } - - private void commitOperation( - SnapshotUpdate operation, - CommitSummary summary, - String description, - String newFlinkJobId, - String operatorId, - long checkpointId) { - LOG.info( - "Committing {} for checkpoint {} to table {} branch {} with summary: {}", - description, - checkpointId, - table.name(), - branch, - summary); - snapshotProperties.forEach(operation::set); - // custom snapshot metadata properties will be overridden if they conflict with internal ones - // used by the sink. - operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); - operation.set(FLINK_JOB_ID, newFlinkJobId); - operation.set(OPERATOR_ID, operatorId); - operation.toBranch(branch); - - long startNano = System.nanoTime(); - operation.commit(); // abort is automatically called if this fails. - long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); - LOG.info( - "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", - description, - table.name(), - branch, - checkpointId, - durationMs); - committerMetrics.commitDuration(durationMs); - } - - @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); - } - - @Override - public void endInput() throws IOException { - // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - - commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); - } - - /** - * Write all the complete data files to a newly created manifest file and return the manifest's - * avro serialized bytes. - */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - result, () -> manifestOutputFileFactory.create(checkpointId), spec); - - return SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, deltaManifests); - } - - @Override - public void open() throws Exception { - super.open(); - - final String operatorID = getRuntimeContext().getOperatorUniqueID(); - this.workerPool = - ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); - } - - @Override - public void close() throws Exception { - if (tableLoader != null) { - tableLoader.close(); - } - - if (workerPool != null) { - workerPool.shutdown(); - } - } - - @VisibleForTesting - static ListStateDescriptor> buildStateDescriptor() { - Comparator longComparator = Comparators.forType(Types.LongType.get()); - // Construct a SortedMapTypeInfo. - SortedMapTypeInfo sortedMapTypeInfo = - new SortedMapTypeInfo<>( - BasicTypeInfo.LONG_TYPE_INFO, - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, - longComparator); - return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); - } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java deleted file mode 100644 index 9de0d6aaa551..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ /dev/null @@ -1,96 +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.sink; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; - -class IcebergFilesCommitterMetrics { - private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); - private final AtomicLong lastCommitDurationMs = new AtomicLong(); - private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; - private final Counter committedDataFilesCount; - private final Counter committedDataFilesRecordCount; - private final Counter committedDataFilesByteCount; - private final Counter committedDeleteFilesCount; - private final Counter committedDeleteFilesRecordCount; - private final Counter committedDeleteFilesByteCount; - - IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup committerMetrics = - metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); - committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); - committerMetrics.gauge("lastCommitDurationMs", lastCommitDurationMs::get); - this.elapsedSecondsSinceLastSuccessfulCommit = new ElapsedTimeGauge(TimeUnit.SECONDS); - committerMetrics.gauge( - "elapsedSecondsSinceLastSuccessfulCommit", elapsedSecondsSinceLastSuccessfulCommit); - this.committedDataFilesCount = committerMetrics.counter("committedDataFilesCount"); - this.committedDataFilesRecordCount = committerMetrics.counter("committedDataFilesRecordCount"); - this.committedDataFilesByteCount = committerMetrics.counter("committedDataFilesByteCount"); - this.committedDeleteFilesCount = committerMetrics.counter("committedDeleteFilesCount"); - this.committedDeleteFilesRecordCount = - committerMetrics.counter("committedDeleteFilesRecordCount"); - this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); - } - - void checkpointDuration(long checkpointDurationMs) { - lastCheckpointDurationMs.set(checkpointDurationMs); - } - - void commitDuration(long commitDurationMs) { - lastCommitDurationMs.set(commitDurationMs); - } - - /** This is called upon a successful commit. */ - void updateCommitSummary(CommitSummary stats) { - elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); - committedDataFilesCount.inc(stats.dataFilesCount()); - committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); - committedDataFilesByteCount.inc(stats.dataFilesByteCount()); - committedDeleteFilesCount.inc(stats.deleteFilesCount()); - committedDeleteFilesRecordCount.inc(stats.deleteFilesRecordCount()); - committedDeleteFilesByteCount.inc(stats.deleteFilesByteCount()); - } - - /** - * This gauge measures the elapsed time between now and last recorded time set by {@link - * ElapsedTimeGauge#refreshLastRecordedTime()}. - */ - private static class ElapsedTimeGauge implements Gauge { - private final TimeUnit reportUnit; - private volatile long lastRecordedTimeNano; - - ElapsedTimeGauge(TimeUnit timeUnit) { - this.reportUnit = timeUnit; - this.lastRecordedTimeNano = System.nanoTime(); - } - - void refreshLastRecordedTime() { - this.lastRecordedTimeNano = System.nanoTime(); - } - - @Override - public Long getValue() { - return reportUnit.convert(System.nanoTime() - lastRecordedTimeNano, TimeUnit.NANOSECONDS); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java deleted file mode 100644 index 9ea0349fb057..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ /dev/null @@ -1,120 +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.sink; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private static final long serialVersionUID = 1L; - - private final String fullTableName; - private final TaskWriterFactory taskWriterFactory; - - private transient TaskWriter writer; - private transient int subTaskId; - private transient int attemptId; - private transient IcebergStreamWriterMetrics writerMetrics; - - IcebergStreamWriter(String fullTableName, TaskWriterFactory taskWriterFactory) { - this.fullTableName = fullTableName; - this.taskWriterFactory = taskWriterFactory; - setChainingStrategy(ChainingStrategy.ALWAYS); - } - - @Override - public void open() { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); - this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); - - // Initialize the task writer factory. - this.taskWriterFactory.initialize(subTaskId, attemptId); - - // Initialize the task writer. - this.writer = taskWriterFactory.create(); - } - - @Override - public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); - this.writer = taskWriterFactory.create(); - } - - @Override - public void processElement(StreamRecord element) throws Exception { - writer.write(element.getValue()); - } - - @Override - public void close() throws Exception { - super.close(); - if (writer != null) { - writer.close(); - writer = null; - } - } - - @Override - public void endInput() throws IOException { - // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the - // remaining completed files to downstream before closing the writer so that we won't miss any - // of them. - // Note that if the task is not closed after calling endInput, checkpoint may be triggered again - // causing files to be sent repeatedly, the writer is marked as null after the last file is sent - // to guard against duplicated writes. - flush(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) - .toString(); - } - - /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { - if (writer == null) { - return; - } - - long startNano = System.nanoTime(); - WriteResult result = writer.complete(); - writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); - writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); - - // Set writer to null to prevent duplicate flushes in the corner case of - // prepareSnapshotPreBarrier happening after endInput. - writer = null; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java deleted file mode 100644 index ce2a6c583fdf..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ /dev/null @@ -1,89 +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.sink; - -import com.codahale.metrics.SlidingWindowReservoir; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Histogram; -import org.apache.flink.metrics.MetricGroup; -import org.apache.iceberg.io.WriteResult; - -class IcebergStreamWriterMetrics { - // 1,024 reservoir size should cost about 8KB, which is quite small. - // It should also produce good accuracy for histogram distribution (like percentiles). - private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; - - private final Counter flushedDataFiles; - private final Counter flushedDeleteFiles; - private final Counter flushedReferencedDataFiles; - private final AtomicLong lastFlushDurationMs; - private final Histogram dataFilesSizeHistogram; - private final Histogram deleteFilesSizeHistogram; - - IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup writerMetrics = - metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); - this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); - this.flushedDeleteFiles = writerMetrics.counter("flushedDeleteFiles"); - this.flushedReferencedDataFiles = writerMetrics.counter("flushedReferencedDataFiles"); - this.lastFlushDurationMs = new AtomicLong(); - writerMetrics.gauge("lastFlushDurationMs", lastFlushDurationMs::get); - - com.codahale.metrics.Histogram dropwizardDataFilesSizeHistogram = - new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); - this.dataFilesSizeHistogram = - writerMetrics.histogram( - "dataFilesSizeHistogram", - new DropwizardHistogramWrapper(dropwizardDataFilesSizeHistogram)); - com.codahale.metrics.Histogram dropwizardDeleteFilesSizeHistogram = - new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); - this.deleteFilesSizeHistogram = - writerMetrics.histogram( - "deleteFilesSizeHistogram", - new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); - } - - void updateFlushResult(WriteResult result) { - flushedDataFiles.inc(result.dataFiles().length); - flushedDeleteFiles.inc(result.deleteFiles().length); - flushedReferencedDataFiles.inc(result.referencedDataFiles().length); - - // For file size distribution histogram, we don't have to update them after successful commits. - // This should works equally well and we avoided the overhead of tracking the list of file sizes - // in the {@link CommitSummary}, which currently stores simple stats for counters and gauges - // metrics. - Arrays.stream(result.dataFiles()) - .forEach( - dataFile -> { - dataFilesSizeHistogram.update(dataFile.fileSizeInBytes()); - }); - Arrays.stream(result.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); - }); - } - - void flushDuration(long flushDurationMs) { - lastFlushDurationMs.set(flushDurationMs); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java deleted file mode 100644 index da5e6e7627ae..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ /dev/null @@ -1,94 +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.sink; - -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.base.Strings; - -class ManifestOutputFileFactory { - // Users could define their own flink manifests directory by setting this value in table - // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - - private final Supplier
    tableSupplier; - private final Map props; - private final String flinkJobId; - private final String operatorUniqueId; - private final int subTaskId; - private final long attemptNumber; - private final AtomicInteger fileCount = new AtomicInteger(0); - - ManifestOutputFileFactory( - Supplier
    tableSupplier, - Map props, - String flinkJobId, - String operatorUniqueId, - int subTaskId, - long attemptNumber) { - this.tableSupplier = tableSupplier; - this.props = props; - this.flinkJobId = flinkJobId; - this.operatorUniqueId = operatorUniqueId; - this.subTaskId = subTaskId; - this.attemptNumber = attemptNumber; - } - - private String generatePath(long checkpointId) { - return FileFormat.AVRO.addExtension( - String.format( - "%s-%s-%05d-%d-%d-%05d", - flinkJobId, - operatorUniqueId, - subTaskId, - attemptNumber, - checkpointId, - fileCount.incrementAndGet())); - } - - OutputFile create(long checkpointId) { - String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); - TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); - - String newManifestFullPath; - if (Strings.isNullOrEmpty(flinkManifestDir)) { - // User don't specify any flink manifest directory, so just use the default metadata path. - newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); - } else { - newManifestFullPath = - String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); - } - - return tableSupplier.get().io().newOutputFile(newManifestFullPath); - } - - private static String stripTrailingSlash(String path) { - String result = path; - while (result.endsWith("/")) { - result = result.substring(0, result.length() - 1); - } - return result; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java deleted file mode 100644 index df951684b446..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.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.flink.sink; - -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; - -/** - * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be - * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy - * for {@link FlinkSink}. - */ -class PartitionKeySelector implements KeySelector { - - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - - private transient RowDataWrapper rowDataWrapper; - - PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(spec, schema); - this.flinkSchema = flinkSchema; - } - - /** - * Construct the {@link RowDataWrapper} lazily here because few members in it are not - * serializable. In this way, we don't have to serialize them with forcing. - */ - private RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - return rowDataWrapper; - } - - @Override - public String getKey(RowData row) { - partitionKey.partition(lazyRowDataWrapper().wrap(row)); - return partitionKey.toPath(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java deleted file mode 100644 index 38062dd1a2c4..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java +++ /dev/null @@ -1,97 +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.sink; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Tasks; - -class PartitionedDeltaWriter extends BaseDeltaTaskWriter { - - private final PartitionKey partitionKey; - - private final Map writers = Maps.newHashMap(); - - PartitionedDeltaWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super( - spec, - format, - appenderFactory, - fileFactory, - io, - targetFileSize, - schema, - flinkSchema, - equalityFieldIds, - upsert); - this.partitionKey = new PartitionKey(spec, schema); - } - - @Override - RowDataDeltaWriter route(RowData row) { - partitionKey.partition(wrapper().wrap(row)); - - RowDataDeltaWriter writer = writers.get(partitionKey); - if (writer == null) { - // NOTICE: we need to copy a new partition key here, in case of messing up the keys in - // writers. - PartitionKey copiedKey = partitionKey.copy(); - writer = new RowDataDeltaWriter(copiedKey); - writers.put(copiedKey, writer); - } - - return writer; - } - - @Override - public void close() { - try { - Tasks.foreach(writers.values()) - .throwFailureWhenFinished() - .noRetry() - .run(RowDataDeltaWriter::close, IOException.class); - - writers.clear(); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close equality delta writer", e); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java deleted file mode 100644 index 67422a1afeb1..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ /dev/null @@ -1,244 +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.sink; - -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.PartitionedFanoutWriter; -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.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.SerializableSupplier; - -public class RowDataTaskWriterFactory implements TaskWriterFactory { - private final Supplier
    tableSupplier; - private final Schema schema; - private final RowType flinkSchema; - private final PartitionSpec spec; - private final long targetFileSizeBytes; - private final FileFormat format; - private final List equalityFieldIds; - private final boolean upsert; - private final FileAppenderFactory appenderFactory; - - private transient OutputFileFactory outputFileFactory; - - public RowDataTaskWriterFactory( - Table table, - RowType flinkSchema, - long targetFileSizeBytes, - FileFormat format, - Map writeProperties, - List equalityFieldIds, - boolean upsert) { - this( - () -> table, - flinkSchema, - targetFileSizeBytes, - format, - writeProperties, - equalityFieldIds, - upsert); - } - - public RowDataTaskWriterFactory( - SerializableSupplier
    tableSupplier, - RowType flinkSchema, - long targetFileSizeBytes, - FileFormat format, - Map writeProperties, - List equalityFieldIds, - boolean upsert) { - this.tableSupplier = tableSupplier; - - Table table; - if (tableSupplier instanceof CachingTableSupplier) { - // rely on the initial table metadata for schema, etc., until schema evolution is supported - table = ((CachingTableSupplier) tableSupplier).initialTable(); - } else { - table = tableSupplier.get(); - } - - this.schema = table.schema(); - this.flinkSchema = flinkSchema; - this.spec = table.spec(); - this.targetFileSizeBytes = targetFileSizeBytes; - this.format = format; - this.equalityFieldIds = equalityFieldIds; - this.upsert = upsert; - - if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { - this.appenderFactory = - new FlinkAppenderFactory( - table, schema, flinkSchema, writeProperties, spec, null, null, null); - } else if (upsert) { - // In upsert mode, only the new row is emitted using INSERT row kind. Therefore, any column of - // the inserted row - // may differ from the deleted row other than the primary key fields, and the delete file must - // contain values - // that are correct for the deleted row. Therefore, only write the equality delete fields. - this.appenderFactory = - new FlinkAppenderFactory( - table, - schema, - flinkSchema, - writeProperties, - spec, - ArrayUtil.toIntArray(equalityFieldIds), - TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), - null); - } else { - this.appenderFactory = - new FlinkAppenderFactory( - table, - schema, - flinkSchema, - writeProperties, - spec, - ArrayUtil.toIntArray(equalityFieldIds), - schema, - null); - } - } - - @Override - public void initialize(int taskId, int attemptId) { - Table table; - if (tableSupplier instanceof CachingTableSupplier) { - // rely on the initial table metadata for schema, etc., until schema evolution is supported - table = ((CachingTableSupplier) tableSupplier).initialTable(); - } else { - table = tableSupplier.get(); - } - - refreshTable(); - - this.outputFileFactory = - OutputFileFactory.builderFor(table, taskId, attemptId) - .format(format) - .ioSupplier(() -> tableSupplier.get().io()) - .build(); - } - - @Override - public TaskWriter create() { - Preconditions.checkNotNull( - outputFileFactory, - "The outputFileFactory shouldn't be null if we have invoked the initialize()."); - - refreshTable(); - - if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { - // Initialize a task writer to write INSERT only. - if (spec.isUnpartitioned()) { - return new UnpartitionedWriter<>( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes); - } else { - return new RowDataPartitionedFanoutWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema); - } - } else { - // Initialize a task writer to write both INSERT and equality DELETE. - if (spec.isUnpartitioned()) { - return new UnpartitionedDeltaWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema, - equalityFieldIds, - upsert); - } else { - return new PartitionedDeltaWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema, - equalityFieldIds, - upsert); - } - } - } - - void refreshTable() { - if (tableSupplier instanceof CachingTableSupplier) { - ((CachingTableSupplier) tableSupplier).refreshTable(); - } - } - - private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { - - private final PartitionKey partitionKey; - private final RowDataWrapper rowDataWrapper; - - RowDataPartitionedFanoutWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.partitionKey = new PartitionKey(spec, schema); - this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - @Override - protected PartitionKey partition(RowData row) { - partitionKey.partition(rowDataWrapper.wrap(row)); - return partitionKey; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java deleted file mode 100644 index e3a1245e8cbd..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ /dev/null @@ -1,45 +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.sink; - -import java.io.Serializable; -import org.apache.iceberg.io.TaskWriter; - -/** - * Factory to create {@link TaskWriter} - * - * @param data type of record. - */ -public interface TaskWriterFactory extends Serializable { - - /** - * Initialize the factory with a given taskId and attemptId. - * - * @param taskId the identifier of task. - * @param attemptId the attempt id of this task. - */ - void initialize(int taskId, int attemptId); - - /** - * Initialize a {@link TaskWriter} with given task id and attempt id. - * - * @return a newly created task writer. - */ - TaskWriter create(); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java deleted file mode 100644 index 7680fb933b20..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; - -class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { - private final RowDataDeltaWriter writer; - - UnpartitionedDeltaWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super( - spec, - format, - appenderFactory, - fileFactory, - io, - targetFileSize, - schema, - flinkSchema, - equalityFieldIds, - upsert); - this.writer = new RowDataDeltaWriter(null); - } - - @Override - RowDataDeltaWriter route(RowData row) { - return writer; - } - - @Override - public void close() throws IOException { - writer.close(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java deleted file mode 100644 index 157f04b8b0ed..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.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.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. - */ -class AggregatedStatistics, S> implements Serializable { - - private final long checkpointId; - private final DataStatistics dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } - - AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; - } - - DataStatistics dataStatistics() { - return dataStatistics; - } - - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) - .toString(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java deleted file mode 100644 index e8ff61dbeb27..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ /dev/null @@ -1,133 +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.sink.shuffle; - -import java.util.Set; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. - */ -class AggregatedStatisticsTracker, S> { - private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; - private final String operatorName; - private final TypeSerializer> statisticsSerializer; - private final int parallelism; - private final Set inProgressSubtaskSet; - private volatile AggregatedStatistics inProgressStatistics; - - AggregatedStatisticsTracker( - String operatorName, - TypeSerializer> statisticsSerializer, - int parallelism) { - this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; - this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); - } - - AggregatedStatistics updateAndCheckCompletion( - int subtask, DataStatisticsEvent event) { - long checkpointId = event.checkpointId(); - - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { - LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", - operatorName, - inProgressStatistics.checkpointId(), - checkpointId); - return null; - } - - AggregatedStatistics completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); - } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); - } - - inProgressStatistics = null; - inProgressSubtaskSet.clear(); - } - - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); - } - - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); - } - - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); - } - - return completedStatistics; - } - - @VisibleForTesting - AggregatedStatistics inProgressStatistics() { - return inProgressStatistics; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java deleted file mode 100644 index 9d7cf179ab1c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.SortKey; - -/** - * DataStatistics defines the interface to collect data distribution information. - * - *

    Data statistics tracks traffic volume distribution across data keys. For low-cardinality key, - * a simple map of (key, count) can be used. For high-cardinality key, probabilistic data structures - * (sketching) can be used. - */ -@Internal -interface DataStatistics, S> { - - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ - boolean isEmpty(); - - /** Add row sortKey to data statistics. */ - void add(SortKey sortKey); - - /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics - */ - S statistics(); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java deleted file mode 100644 index c8ac79c61bf6..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ /dev/null @@ -1,396 +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.sink.shuffle; - -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FatalExitExceptionHandler; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.ThrowableCatchingRunnable; -import org.apache.flink.util.function.ThrowingRunnable; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. - */ -@Internal -class DataStatisticsCoordinator, S> implements OperatorCoordinator { - private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); - - private final String operatorName; - private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; - private final SubtaskGateways subtaskGateways; - private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer> statisticsSerializer; - private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private volatile AggregatedStatistics completedStatistics; - private volatile boolean started; - - DataStatisticsCoordinator( - String operatorName, - OperatorCoordinator.Context context, - TypeSerializer> statisticsSerializer) { - this.operatorName = operatorName; - this.coordinatorThreadFactory = - new CoordinatorExecutorThreadFactory( - "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); - this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); - } - - @Override - public void start() throws Exception { - LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; - } - - @Override - public void close() throws Exception { - coordinatorExecutor.shutdown(); - LOG.info("Closed data statistics coordinator: {}.", operatorName); - } - - @VisibleForTesting - void callInCoordinatorThread(Callable callable, String errorMessage) { - ensureStarted(); - // Ensure the task is done by the coordinator executor. - if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { - try { - Callable guardedCallable = - () -> { - try { - return callable.call(); - } catch (Throwable t) { - LOG.error( - "Uncaught Exception in data statistics coordinator: {} executor", - operatorName, - t); - ExceptionUtils.rethrowException(t); - return null; - } - }; - - coordinatorExecutor.submit(guardedCallable).get(); - } catch (InterruptedException | ExecutionException e) { - throw new FlinkRuntimeException(errorMessage, e); - } - } else { - try { - callable.call(); - } catch (Throwable t) { - LOG.error( - "Uncaught Exception in data statistics coordinator: {} executor", operatorName, t); - throw new FlinkRuntimeException(errorMessage, t); - } - } - } - - public void runInCoordinatorThread(Runnable runnable) { - this.coordinatorExecutor.execute( - new ThrowableCatchingRunnable( - throwable -> - this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable), - runnable)); - } - - private void runInCoordinatorThread(ThrowingRunnable action, String actionString) { - ensureStarted(); - runInCoordinatorThread( - () -> { - try { - action.run(); - } catch (Throwable t) { - ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - LOG.error( - "Uncaught exception in the data statistics coordinator: {} while {}. Triggering job failover", - operatorName, - actionString, - t); - operatorCoordinatorContext.failJob(t); - } - }); - } - - private void ensureStarted() { - Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); - } - - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); - } - } - - @SuppressWarnings("FutureReturnValueIgnored") - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics globalDataStatistics) { - callInCoordinatorThread( - () -> { - DataStatisticsEvent dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); - } - - return null; - }, - String.format( - "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); - } - - @Override - @SuppressWarnings("unchecked") - public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { - runInCoordinatorThread( - () -> { - LOG.debug( - "Handling event from subtask {} (#{}) of {}: {}", - subtask, - attemptNumber, - operatorName, - event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); - }, - String.format( - "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); - } - - @Override - public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) { - runInCoordinatorThread( - () -> { - LOG.debug( - "Snapshotting data statistics coordinator {} for checkpoint {}", - operatorName, - checkpointId); - resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); - }, - String.format("taking checkpoint %d", checkpointId)); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) {} - - @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { - Preconditions.checkState( - !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - - if (checkpointData == null) { - LOG.info( - "Data statistic coordinator {} has nothing to restore from checkpoint {}", - operatorName, - checkpointId); - return; - } - - LOG.info( - "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); - } - - @Override - public void subtaskReset(int subtask, long checkpointId) { - runInCoordinatorThread( - () -> { - LOG.info( - "Operator {} subtask {} is reset to checkpoint {}", - operatorName, - subtask, - checkpointId); - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways.reset(subtask); - }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); - } - - @Override - public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Throwable reason) { - runInCoordinatorThread( - () -> { - LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", - subtask, - attemptNumber, - operatorName); - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); - }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); - } - - @Override - public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { - Preconditions.checkArgument(subtask == gateway.getSubtask()); - Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); - runInCoordinatorThread( - () -> { - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways.registerSubtaskGateway(gateway); - }, - String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); - } - - @VisibleForTesting - AggregatedStatistics completedStatistics() { - return completedStatistics; - } - - private static class SubtaskGateways { - private final String operatorName; - private final Map[] gateways; - - private SubtaskGateways(String operatorName, int parallelism) { - this.operatorName = operatorName; - gateways = new Map[parallelism]; - - for (int i = 0; i < parallelism; ++i) { - gateways[i] = Maps.newHashMap(); - } - } - - private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway gateway) { - int subtaskIndex = gateway.getSubtask(); - int attemptNumber = gateway.getExecution().getAttemptNumber(); - Preconditions.checkState( - !gateways[subtaskIndex].containsKey(attemptNumber), - "Coordinator of %s already has a subtask gateway for %d (#%d)", - operatorName, - subtaskIndex, - attemptNumber); - LOG.debug( - "Coordinator of {} registers gateway for subtask {} attempt {}", - operatorName, - subtaskIndex, - attemptNumber); - gateways[subtaskIndex].put(attemptNumber, gateway); - } - - private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { - LOG.debug( - "Coordinator of {} unregisters gateway for subtask {} attempt {}", - operatorName, - subtaskIndex, - attemptNumber); - gateways[subtaskIndex].remove(attemptNumber); - } - - private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { - Preconditions.checkState( - !gateways[subtaskIndex].isEmpty(), - "Coordinator of %s subtask %d is not ready yet to receive events", - operatorName, - subtaskIndex); - return Iterables.getOnlyElement(gateways[subtaskIndex].values()); - } - - private void reset(int subtaskIndex) { - gateways[subtaskIndex].clear(); - } - } - - private static class CoordinatorExecutorThreadFactory - implements ThreadFactory, Thread.UncaughtExceptionHandler { - - private final String coordinatorThreadName; - private final ClassLoader classLoader; - private final Thread.UncaughtExceptionHandler errorHandler; - - @javax.annotation.Nullable private Thread thread; - - CoordinatorExecutorThreadFactory( - final String coordinatorThreadName, final ClassLoader contextClassLoader) { - this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE); - } - - @org.apache.flink.annotation.VisibleForTesting - CoordinatorExecutorThreadFactory( - final String coordinatorThreadName, - final ClassLoader contextClassLoader, - final Thread.UncaughtExceptionHandler errorHandler) { - this.coordinatorThreadName = coordinatorThreadName; - this.classLoader = contextClassLoader; - this.errorHandler = errorHandler; - } - - @Override - public synchronized Thread newThread(@NotNull Runnable runnable) { - thread = new Thread(runnable, coordinatorThreadName); - thread.setContextClassLoader(classLoader); - thread.setUncaughtExceptionHandler(this); - return thread; - } - - @Override - public synchronized void uncaughtException(Thread t, Throwable e) { - errorHandler.uncaughtException(t, e); - } - - boolean isCurrentThreadCoordinatorThread() { - return Thread.currentThread() == thread; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java deleted file mode 100644 index 47dbfc3cfbe1..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,51 +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.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; - -/** - * DataStatisticsCoordinatorProvider provides the method to create new {@link - * DataStatisticsCoordinator} - */ -@Internal -public class DataStatisticsCoordinatorProvider, S> - extends RecreateOnResetOperatorCoordinator.Provider { - - private final String operatorName; - private final TypeSerializer> statisticsSerializer; - - public DataStatisticsCoordinatorProvider( - String operatorName, - OperatorID operatorID, - TypeSerializer> statisticsSerializer) { - super(operatorID); - this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; - } - - @Override - public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java deleted file mode 100644 index 852d2157b8cb..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; - -/** - * DataStatisticsEvent is sent between data statistics coordinator and operator to transmit data - * statistics in bytes - */ -@Internal -class DataStatisticsEvent, S> implements OperatorEvent { - - private static final long serialVersionUID = 1L; - private final long checkpointId; - private final byte[] statisticsBytes; - - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { - this.checkpointId = checkpointId; - this.statisticsBytes = statisticsBytes; - } - - static , S> DataStatisticsEvent create( - long checkpointId, - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - return new DataStatisticsEvent<>( - checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); - } - - long checkpointId() { - return checkpointId; - } - - byte[] statisticsBytes() { - return statisticsBytes; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java deleted file mode 100644 index 5157a37cf2cd..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ /dev/null @@ -1,197 +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.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; -import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be - * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to - * shuffle record to improve data clustering while maintaining relative balanced traffic - * distribution to downstream subtasks. - */ -@Internal -class DataStatisticsOperator, S> - extends AbstractStreamOperator> - implements OneInputStreamOperator>, OperatorEventHandler { - - private static final long serialVersionUID = 1L; - - private final String operatorName; - private final RowDataWrapper rowDataWrapper; - private final SortKey sortKey; - private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer> statisticsSerializer; - private transient volatile DataStatistics localStatistics; - private transient volatile DataStatistics globalStatistics; - private transient ListState> globalStatisticsState; - - DataStatisticsOperator( - String operatorName, - Schema schema, - SortOrder sortOrder, - OperatorEventGateway operatorEventGateway, - TypeSerializer> statisticsSerializer) { - this.operatorName = operatorName; - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - this.sortKey = new SortKey(schema, sortOrder); - this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = - context - .getOperatorStateStore() - .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); - - if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - if (globalStatisticsState.get() == null - || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( - "Operator {} subtask {} doesn't have global statistics state to restore", - operatorName, - subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); - } else { - LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); - } - } else { - globalStatistics = statisticsSerializer.createInstance(); - } - } - - @Override - public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - } - - @Override - @SuppressWarnings("unchecked") - public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - Preconditions.checkArgument( - event instanceof DataStatisticsEvent, - String.format( - "Operator %s subtask %s received unexpected operator event %s", - operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; - LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", - operatorName, - statisticsEvent.checkpointId()); - globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - - @Override - public void processElement(StreamRecord streamRecord) { - RowData record = streamRecord.getValue(); - StructLike struct = rowDataWrapper.wrap(record); - sortKey.wrap(struct); - localStatistics.add(sortKey); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", - operatorName, - checkpointId, - subTaskId); - - // Pass global statistics to partitioners so that all the operators refresh statistics - // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); - } - - // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores - // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { - globalStatisticsState.clear(); - LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); - globalStatisticsState.add(globalStatistics); - } - - // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, - operatorName, - checkpointId, - localStatistics); - - // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); - } - - @VisibleForTesting - DataStatistics localDataStatistics() { - return localStatistics; - } - - @VisibleForTesting - DataStatistics globalDataStatistics() { - return globalStatistics; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java deleted file mode 100644 index 889e85112e16..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ /dev/null @@ -1,113 +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.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * The wrapper class for data statistics and record. It is the only way for data statistics operator - * to send global data statistics to custom partitioner to distribute data based on statistics - * - *

    DataStatisticsOrRecord contains either data statistics(globally aggregated) or a record. It is - * sent from {@link DataStatisticsOperator} to partitioner. Once partitioner receives the data - * statistics, it will use that to decide the coming record should send to which writer subtask. - * After shuffling, a filter and mapper are required to filter out the data distribution weight, - * unwrap the object and extract the original record type T. - */ -class DataStatisticsOrRecord, S> implements Serializable { - - private static final long serialVersionUID = 1L; - - private DataStatistics statistics; - private RowData record; - - private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { - Preconditions.checkArgument( - record != null ^ statistics != null, "DataStatistics or record, not neither or both"); - this.statistics = statistics; - this.record = record; - } - - static , S> DataStatisticsOrRecord fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); - } - - static , S> DataStatisticsOrRecord fromDataStatistics( - DataStatistics statistics) { - return new DataStatisticsOrRecord<>(statistics, null); - } - - static , S> DataStatisticsOrRecord reuseRecord( - DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { - if (reuse.hasRecord()) { - return reuse; - } else { - // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); - } - } - - static , S> DataStatisticsOrRecord reuseStatistics( - DataStatisticsOrRecord reuse, - TypeSerializer> statisticsSerializer) { - if (reuse.hasDataStatistics()) { - return reuse; - } else { - // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); - } - } - - boolean hasDataStatistics() { - return statistics != null; - } - - boolean hasRecord() { - return record != null; - } - - DataStatistics dataStatistics() { - return statistics; - } - - void dataStatistics(DataStatistics newStatistics) { - this.statistics = newStatistics; - } - - RowData record() { - return record; - } - - void record(RowData newRecord) { - this.record = newRecord; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("statistics", statistics) - .add("record", record) - .toString(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java deleted file mode 100644 index e9a6fa0cbfc5..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ /dev/null @@ -1,219 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; - -@Internal -class DataStatisticsOrRecordSerializer, S> - extends TypeSerializer> { - private final TypeSerializer> statisticsSerializer; - private final TypeSerializer recordSerializer; - - DataStatisticsOrRecordSerializer( - TypeSerializer> statisticsSerializer, - TypeSerializer recordSerializer) { - this.statisticsSerializer = statisticsSerializer; - this.recordSerializer = recordSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer> duplicate() { - TypeSerializer> duplicateStatisticsSerializer = - statisticsSerializer.duplicate(); - TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); - if ((statisticsSerializer != duplicateStatisticsSerializer) - || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( - duplicateStatisticsSerializer, duplicateRowDataSerializer); - } else { - return this; - } - } - - @Override - public DataStatisticsOrRecord createInstance() { - // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); - } - - @Override - public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { - if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); - } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); - } - } - - @Override - public DataStatisticsOrRecord copy( - DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { - DataStatisticsOrRecord to; - if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); - RowData record = recordSerializer.copy(from.record(), to.record()); - to.record(record); - } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); - } - - return to; - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) - throws IOException { - if (statisticsOrRecord.hasRecord()) { - target.writeBoolean(true); - recordSerializer.serialize(statisticsOrRecord.record(), target); - } else { - target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); - } - } - - @Override - public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { - boolean isRecord = source.readBoolean(); - if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); - } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); - } - } - - @Override - public DataStatisticsOrRecord deserialize( - DataStatisticsOrRecord reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord to; - boolean isRecord = source.readBoolean(); - if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); - RowData record = recordSerializer.deserialize(to.record(), source); - to.record(record); - } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); - } - - return to; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - boolean hasRecord = source.readBoolean(); - target.writeBoolean(hasRecord); - if (hasRecord) { - recordSerializer.copy(source, target); - } else { - statisticsSerializer.copy(source, target); - } - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { - return false; - } - - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; - return Objects.equals(statisticsSerializer, other.statisticsSerializer) - && Objects.equals(recordSerializer, other.recordSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(statisticsSerializer, recordSerializer); - } - - @Override - public TypeSerializerSnapshot> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); - } - - public static class DataStatisticsOrRecordSerializerSnapshot, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer serializer) { - super(serializer); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - DataStatisticsOrRecordSerializer outerSerializer) { - return new TypeSerializer[] { - outerSerializer.statisticsSerializer, outerSerializer.recordSerializer - }; - } - - @SuppressWarnings("unchecked") - @Override - protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - TypeSerializer> statisticsSerializer = - (TypeSerializer>) nestedSerializers[0]; - TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 8716cb872d0e..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,96 +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.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static , S> byte[] serializeDataStatistics( - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static , S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static , S> byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - static , S> - AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java deleted file mode 100644 index 0ffffd9cf49f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.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.flink.sink.shuffle; - -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -/** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; - - MapDataStatistics() { - this.statistics = Maps.newHashMap(); - } - - MapDataStatistics(Map statistics) { - this.statistics = statistics; - } - - @Override - public boolean isEmpty() { - return statistics.size() == 0; - } - - @Override - public void add(SortKey sortKey) { - if (statistics.containsKey(sortKey)) { - statistics.merge(sortKey, 1L, Long::sum); - } else { - // clone the sort key before adding to map because input sortKey object can be reused - SortKey copiedKey = sortKey.copy(); - statistics.put(copiedKey, 1L); - } - } - - @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); - } - - @Override - public Map statistics() { - return statistics; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index b6cccd0566fc..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,187 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; - - static MapDataStatisticsSerializer fromSortKeySerializer( - TypeSerializer sortKeySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public MapDataStatistics createInstance() { - return new MapDataStatistics(); - } - - @Override - public MapDataStatistics copy(DataStatistics> obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - SortKey newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics> copy( - DataStatistics> from, - DataStatistics> reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize( - DataStatistics> obj, DataOutputView target) - throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics> deserialize( - DataStatistics> reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java deleted file mode 100644 index fb1a8f03a65c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ /dev/null @@ -1,381 +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.sink.shuffle; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; -import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -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; -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.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Internal partitioner implementation that supports MapDataStatistics, which is typically used for - * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used - * for high-cardinality use cases. Otherwise, the memory footprint is too high. - * - *

    It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always - * precise when calculating close cost for every file, target weight per subtask, padding residual - * weight, assigned weight without close cost. - * - *

    All actions should be executed in a single Flink mailbox thread. So there is no need to make - * it thread safe. - */ -class MapRangePartitioner implements Partitioner { - private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); - - private final RowDataWrapper rowDataWrapper; - private final SortKey sortKey; - private final Comparator comparator; - private final Map mapStatistics; - private final double closeFileCostInWeightPercentage; - - // Counter that tracks how many times a new key encountered - // where there is no traffic statistics learned about it. - private long newSortKeyCounter; - private long lastNewSortKeyLogTimeMilli; - - // lazily computed due to the need of numPartitions - private Map assignment; - private NavigableMap sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - MapDataStatistics dataStatistics, - double closeFileCostInWeightPercentage) { - dataStatistics - .statistics() - .entrySet() - .forEach( - entry -> - Preconditions.checkArgument( - entry.getValue() > 0, - "Invalid statistics: weight is 0 for key %s", - entry.getKey())); - - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = dataStatistics.statistics(); - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; - this.newSortKeyCounter = 0; - this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); - } - - @Override - public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map assignmentMap = assignment(numPartitions); - // reuse the sortKey and rowDataWrapper - sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); - if (keyAssignment == null) { - LOG.trace( - "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", - sortKey); - // Ideally unknownKeyCounter should be published as a counter metric. - // It seems difficult to pass in MetricGroup into the partitioner. - // Just log an INFO message every minute. - newSortKeyCounter += 1; - long now = System.currentTimeMillis(); - if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); - lastNewSortKeyLogTimeMilli = now; - } - return (int) (newSortKeyCounter % numPartitions); - } - - return keyAssignment.select(); - } - - @VisibleForTesting - Map assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map mapStatistics() { - return mapStatistics; - } - - /** - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map> assignmentInfo() { - Map> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map buildAssignment( - int numPartitions, - NavigableMap sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List assignedSubtasks = Lists.newArrayList(); - List subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List assignedSubtasks, - List subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** @return subtask id */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java deleted file mode 100644 index d03409f2a430..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ /dev/null @@ -1,353 +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.sink.shuffle; - -import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Objects; -import java.util.UUID; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.SortField; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderParser; -import org.apache.iceberg.types.CheckCompatibility; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -class SortKeySerializer extends TypeSerializer { - private final Schema schema; - private final SortOrder sortOrder; - private final int size; - private final Types.NestedField[] transformedFields; - - private transient SortKey sortKey; - - SortKeySerializer(Schema schema, SortOrder sortOrder) { - this.schema = schema; - this.sortOrder = sortOrder; - this.size = sortOrder.fields().size(); - - this.transformedFields = new Types.NestedField[size]; - for (int i = 0; i < size; ++i) { - SortField sortField = sortOrder.fields().get(i); - Types.NestedField sourceField = schema.findField(sortField.sourceId()); - Type resultType = sortField.transform().getResultType(sourceField.type()); - Types.NestedField transformedField = - Types.NestedField.of( - sourceField.fieldId(), - sourceField.isOptional(), - sourceField.name(), - resultType, - sourceField.doc()); - transformedFields[i] = transformedField; - } - } - - private SortKey lazySortKey() { - if (sortKey == null) { - this.sortKey = new SortKey(schema, sortOrder); - } - - return sortKey; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public TypeSerializer duplicate() { - return new SortKeySerializer(schema, sortOrder); - } - - @Override - public SortKey createInstance() { - return new SortKey(schema, sortOrder); - } - - @Override - public SortKey copy(SortKey from) { - return from.copy(); - } - - @Override - public SortKey copy(SortKey from, SortKey reuse) { - // no benefit of reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(SortKey record, DataOutputView target) throws IOException { - Preconditions.checkArgument( - record.size() == size, - "Invalid size of the sort key object: %s. Expected %s", - record.size(), - size); - for (int i = 0; i < size; ++i) { - int fieldId = transformedFields[i].fieldId(); - Type.TypeID typeId = transformedFields[i].type().typeId(); - switch (typeId) { - case BOOLEAN: - target.writeBoolean(record.get(i, Boolean.class)); - break; - case INTEGER: - case DATE: - target.writeInt(record.get(i, Integer.class)); - break; - case LONG: - case TIME: - case TIMESTAMP: - target.writeLong(record.get(i, Long.class)); - break; - case FLOAT: - target.writeFloat(record.get(i, Float.class)); - break; - case DOUBLE: - target.writeDouble(record.get(i, Double.class)); - break; - case STRING: - target.writeUTF(record.get(i, CharSequence.class).toString()); - break; - case UUID: - UUID uuid = record.get(i, UUID.class); - target.writeLong(uuid.getMostSignificantBits()); - target.writeLong(uuid.getLeastSignificantBits()); - break; - case FIXED: - case BINARY: - byte[] bytes = record.get(i, ByteBuffer.class).array(); - target.writeInt(bytes.length); - target.write(bytes); - break; - case DECIMAL: - BigDecimal decimal = record.get(i, BigDecimal.class); - byte[] decimalBytes = decimal.unscaledValue().toByteArray(); - target.writeInt(decimalBytes.length); - target.write(decimalBytes); - target.writeInt(decimal.scale()); - break; - case STRUCT: - case MAP: - case LIST: - default: - // SortKey transformation is a flattened struct without list and map - throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); - } - } - } - - @Override - public SortKey deserialize(DataInputView source) throws IOException { - // copying is a little faster than constructing a new SortKey object - SortKey deserialized = lazySortKey().copy(); - deserialize(deserialized, source); - return deserialized; - } - - @Override - public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { - Preconditions.checkArgument( - reuse.size() == size, - "Invalid size of the sort key object: %s. Expected %s", - reuse.size(), - size); - for (int i = 0; i < size; ++i) { - int fieldId = transformedFields[i].fieldId(); - Type.TypeID typeId = transformedFields[i].type().typeId(); - switch (typeId) { - case BOOLEAN: - reuse.set(i, source.readBoolean()); - break; - case INTEGER: - case DATE: - reuse.set(i, source.readInt()); - break; - case LONG: - case TIME: - case TIMESTAMP: - reuse.set(i, source.readLong()); - break; - case FLOAT: - reuse.set(i, source.readFloat()); - break; - case DOUBLE: - reuse.set(i, source.readDouble()); - break; - case STRING: - reuse.set(i, source.readUTF()); - break; - case UUID: - long mostSignificantBits = source.readLong(); - long leastSignificantBits = source.readLong(); - reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); - break; - case FIXED: - case BINARY: - byte[] bytes = new byte[source.readInt()]; - source.read(bytes); - reuse.set(i, ByteBuffer.wrap(bytes)); - break; - case DECIMAL: - byte[] unscaledBytes = new byte[source.readInt()]; - source.read(unscaledBytes); - int scale = source.readInt(); - BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); - reuse.set(i, decimal); - break; - case STRUCT: - case MAP: - case LIST: - default: - // SortKey transformation is a flattened struct without list and map - throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); - } - } - - return reuse; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - // no optimization here - serialize(deserialize(source), target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof SortKeySerializer)) { - return false; - } - - SortKeySerializer other = (SortKeySerializer) obj; - return Objects.equals(schema.asStruct(), other.schema.asStruct()) - && Objects.equals(sortOrder, other.sortOrder); - } - - @Override - public int hashCode() { - return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new SortKeySerializerSnapshot(schema, sortOrder); - } - - public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; - - private Schema schema; - private SortOrder sortOrder; - - @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) - public SortKeySerializerSnapshot() { - // this constructor is used when restoring from a checkpoint. - } - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { - this.schema = schema; - this.sortOrder = sortOrder; - } - - @Override - public int getCurrentVersion() { - return CURRENT_VERSION; - } - - @Override - public void writeSnapshot(DataOutputView out) throws IOException { - Preconditions.checkState(schema != null, "Invalid schema: null"); - Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - - StringUtils.writeString(SchemaParser.toJson(schema), out); - StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); - } - - @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); - } - } - - @Override - public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - TypeSerializer newSerializer) { - if (!(newSerializer instanceof SortKeySerializer)) { - return TypeSerializerSchemaCompatibility.incompatible(); - } - - SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; - return resolveSchemaCompatibility(newAvroSerializer.schema, schema); - } - - @Override - public TypeSerializer restoreSerializer() { - Preconditions.checkState(schema != null, "Invalid schema: null"); - Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - return new SortKeySerializer(schema, sortOrder); - } - - private void readV1(DataInputView in) throws IOException { - String schemaJson = StringUtils.readString(in); - String sortOrderJson = StringUtils.readString(in); - this.schema = SchemaParser.fromJson(schemaJson); - this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); - } - - @VisibleForTesting - static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java deleted file mode 100644 index 796434c45136..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source; - -import org.apache.avro.generic.GenericRecord; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; - -public class AvroGenericRecordFileScanTaskReader implements FileScanTaskReader { - private final RowDataFileScanTaskReader rowDataReader; - private final RowDataToAvroGenericRecordConverter converter; - - public AvroGenericRecordFileScanTaskReader( - RowDataFileScanTaskReader rowDataReader, RowDataToAvroGenericRecordConverter converter) { - this.rowDataReader = rowDataReader; - this.converter = converter; - } - - @Override - public CloseableIterator open( - FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { - return CloseableIterator.transform( - rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java deleted file mode 100644 index 91d975349b19..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ /dev/null @@ -1,156 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Iterator; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Flink data iterator that reads {@link CombinedScanTask} into a {@link CloseableIterator} - * - * @param is the output data type returned by this iterator. - */ -@Internal -public class DataIterator implements CloseableIterator { - - private final FileScanTaskReader fileScanTaskReader; - - private final InputFilesDecryptor inputFilesDecryptor; - private final CombinedScanTask combinedTask; - - private Iterator tasks; - private CloseableIterator currentIterator; - private int fileOffset; - private long recordOffset; - - public DataIterator( - FileScanTaskReader fileScanTaskReader, - CombinedScanTask task, - FileIO io, - EncryptionManager encryption) { - this.fileScanTaskReader = fileScanTaskReader; - - this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption); - this.combinedTask = task; - - this.tasks = task.files().iterator(); - this.currentIterator = CloseableIterator.empty(); - - // fileOffset starts at -1 because we started - // from an empty iterator that is not from the split files. - this.fileOffset = -1; - // record offset points to the record that next() should return when called - this.recordOffset = 0L; - } - - /** - * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume - * from. E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the - * 2nd row in file 0. When next() is called after seek, 2nd row from file 0 should be returned. - */ - public void seek(int startingFileOffset, long startingRecordOffset) { - Preconditions.checkState( - fileOffset == -1, "Seek should be called before any other iterator actions"); - // skip files - Preconditions.checkState( - startingFileOffset < combinedTask.files().size(), - "Invalid starting file offset %s for combined scan task with %s files: %s", - startingFileOffset, - combinedTask.files().size(), - combinedTask); - for (long i = 0L; i < startingFileOffset; ++i) { - tasks.next(); - } - - updateCurrentIterator(); - // skip records within the file - for (long i = 0; i < startingRecordOffset; ++i) { - if (currentFileHasNext() && hasNext()) { - next(); - } else { - throw new IllegalStateException( - String.format( - "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); - } - } - - fileOffset = startingFileOffset; - recordOffset = startingRecordOffset; - } - - @Override - public boolean hasNext() { - updateCurrentIterator(); - return currentIterator.hasNext(); - } - - @Override - public T next() { - updateCurrentIterator(); - recordOffset += 1; - return currentIterator.next(); - } - - public boolean currentFileHasNext() { - return currentIterator.hasNext(); - } - - /** Updates the current iterator field to ensure that the current Iterator is not exhausted. */ - private void updateCurrentIterator() { - try { - while (!currentIterator.hasNext() && tasks.hasNext()) { - currentIterator.close(); - currentIterator = openTaskIterator(tasks.next()); - fileOffset += 1; - recordOffset = 0L; - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private CloseableIterator openTaskIterator(FileScanTask scanTask) { - return fileScanTaskReader.open(scanTask, inputFilesDecryptor); - } - - @Override - public void close() throws IOException { - // close the current iterator - currentIterator.close(); - tasks = null; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java deleted file mode 100644 index 4394dab4d4cc..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java +++ /dev/null @@ -1,47 +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.source; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.flink.data.StructRowData; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; - -@Internal -public class DataTaskReader implements FileScanTaskReader { - - private final Schema readSchema; - - public DataTaskReader(Schema readSchema) { - this.readSchema = readSchema; - } - - @Override - public CloseableIterator open( - FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { - StructRowData row = new StructRowData(readSchema.asStruct()); - CloseableIterable iterable = - CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); - return iterable.iterator(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java deleted file mode 100644 index 927a804a4792..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java +++ /dev/null @@ -1,35 +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.source; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; - -/** - * Read a {@link FileScanTask} into a {@link CloseableIterator} - * - * @param is the output data type returned by this iterator. - */ -@Internal -public interface FileScanTaskReader extends Serializable { - CloseableIterator open(FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java deleted file mode 100644 index 9a5123dc489e..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java +++ /dev/null @@ -1,141 +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.source; - -import java.io.IOException; -import java.util.concurrent.ExecutorService; -import org.apache.flink.api.common.io.DefaultInputSplitAssigner; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.io.LocatableInputSplitAssigner; -import org.apache.flink.api.common.io.RichInputFormat; -import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.util.ThreadPools; - -/** Flink {@link InputFormat} for Iceberg. */ -public class FlinkInputFormat extends RichInputFormat { - - private static final long serialVersionUID = 1L; - - private final TableLoader tableLoader; - private final FileIO io; - private final EncryptionManager encryption; - private final ScanContext context; - private final FileScanTaskReader rowDataReader; - - private transient DataIterator iterator; - private transient long currentReadCount = 0L; - - FlinkInputFormat( - TableLoader tableLoader, - Schema tableSchema, - FileIO io, - EncryptionManager encryption, - ScanContext context) { - this.tableLoader = tableLoader; - this.io = io; - this.encryption = encryption; - this.context = context; - - tableLoader.open(); - Table table = tableLoader.loadTable(); - if (table instanceof BaseMetadataTable) { - this.rowDataReader = new DataTaskReader(context.project()); - } else { - this.rowDataReader = - new RowDataFileScanTaskReader( - tableSchema, - context.project(), - context.nameMapping(), - context.caseSensitive(), - context.filters()); - } - } - - @VisibleForTesting - Schema projectedSchema() { - return context.project(); - } - - @Override - public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { - // Legacy method, not be used. - return null; - } - - @Override - public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { - // Called in Job manager, so it is OK to load table from catalog. - tableLoader.open(); - final ExecutorService workerPool = - ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); - try (TableLoader loader = tableLoader) { - Table table = loader.loadTable(); - return FlinkSplitPlanner.planInputSplits(table, context, workerPool); - } finally { - workerPool.shutdown(); - } - } - - @Override - public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { - return context.exposeLocality() - ? new LocatableInputSplitAssigner(inputSplits) - : new DefaultInputSplitAssigner(inputSplits); - } - - @Override - public void configure(Configuration parameters) {} - - @Override - public void open(FlinkInputSplit split) { - this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption); - } - - @Override - public boolean reachedEnd() { - if (context.limit() > 0 && currentReadCount >= context.limit()) { - return true; - } else { - return !iterator.hasNext(); - } - } - - @Override - public RowData nextRecord(RowData reuse) { - currentReadCount++; - return iterator.next(); - } - - @Override - public void close() throws IOException { - if (iterator != null) { - iterator.close(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java deleted file mode 100644 index 16fd4f39596c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java +++ /dev/null @@ -1,48 +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.source; - -import java.util.Arrays; -import javax.annotation.Nullable; -import org.apache.flink.core.io.LocatableInputSplit; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -public class FlinkInputSplit extends LocatableInputSplit { - - private final CombinedScanTask task; - - FlinkInputSplit(int splitNumber, CombinedScanTask task, @Nullable String[] hostnames) { - super(splitNumber, hostnames); - this.task = task; - } - - CombinedScanTask getTask() { - return task; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("splitNumber", getSplitNumber()) - .add("task", task) - .add("hosts", Arrays.toString(getHostnames())) - .toString(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java deleted file mode 100644 index fa1656c55278..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ /dev/null @@ -1,310 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -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.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.FileIO; -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.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class FlinkSource { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); - - private FlinkSource() {} - - /** - * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link - * TableScan}. See more options in {@link ScanContext}. - * - *

    The Source can be read static data in bounded mode. It can also continuously check the - * arrival of new data and read records incrementally. - * - *

      - *
    • Without startSnapshotId: Bounded - *
    • With startSnapshotId and with endSnapshotId: Bounded - *
    • With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded - *
    - * - *

    - * - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRowData() { - return new Builder(); - } - - /** Source builder to build {@link DataStream}. */ - public static class Builder { - private StreamExecutionEnvironment env; - private Table table; - private TableLoader tableLoader; - private TableSchema projectedSchema; - private ReadableConfig readableConfig = new Configuration(); - private final ScanContext.Builder contextBuilder = ScanContext.builder(); - private Boolean exposeLocality; - - private final Map readOptions = Maps.newHashMap(); - - public Builder tableLoader(TableLoader newLoader) { - this.tableLoader = newLoader; - return this; - } - - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - public Builder env(StreamExecutionEnvironment newEnv) { - this.env = newEnv; - return this; - } - - public Builder filters(List filters) { - contextBuilder.filters(filters); - return this; - } - - public Builder project(TableSchema schema) { - this.projectedSchema = schema; - return this; - } - - public Builder limit(Long newLimit) { - if (newLimit != null) { - readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); - } - return this; - } - - public Builder set(String property, String value) { - readOptions.put(property, value); - return this; - } - - public Builder setAll(Map properties) { - readOptions.putAll(properties); - return this; - } - - /** @deprecated Use {@link #setAll} instead. */ - @Deprecated - public Builder properties(Map properties) { - readOptions.putAll(properties); - return this; - } - - public Builder caseSensitive(boolean caseSensitive) { - readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(caseSensitive)); - return this; - } - - public Builder snapshotId(Long snapshotId) { - readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(snapshotId)); - return this; - } - - public Builder branch(String branch) { - readOptions.put(FlinkReadOptions.BRANCH.key(), branch); - return this; - } - - public Builder tag(String tag) { - readOptions.put(FlinkReadOptions.TAG.key(), tag); - return this; - } - - public Builder startSnapshotId(Long startSnapshotId) { - readOptions.put(FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(startSnapshotId)); - return this; - } - - public Builder endSnapshotId(Long endSnapshotId) { - readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(endSnapshotId)); - return this; - } - - public Builder startTag(String startTag) { - readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); - return this; - } - - public Builder endTag(String endTag) { - readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); - return this; - } - - public Builder asOfTimestamp(Long asOfTimestamp) { - readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(asOfTimestamp)); - return this; - } - - public Builder splitSize(Long splitSize) { - readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(splitSize)); - return this; - } - - public Builder splitLookback(Integer splitLookback) { - readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(splitLookback)); - return this; - } - - public Builder splitOpenFileCost(Long splitOpenFileCost) { - readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(splitOpenFileCost)); - return this; - } - - public Builder streaming(boolean streaming) { - readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder nameMapping(String nameMapping) { - readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, nameMapping); - return this; - } - - public Builder monitorInterval(Duration interval) { - readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, interval.toNanos() + " ns"); - return this; - } - - public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { - readOptions.put( - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT, - Integer.toString(newMaxPlanningSnapshotCount)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - public FlinkInputFormat buildFormat() { - Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); - - Schema icebergSchema; - FileIO io; - EncryptionManager encryption; - if (table == null) { - // load required fields by table loader. - tableLoader.open(); - try (TableLoader loader = tableLoader) { - table = loader.loadTable(); - icebergSchema = table.schema(); - io = table.io(); - encryption = table.encryption(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } else { - icebergSchema = table.schema(); - io = table.io(); - encryption = table.encryption(); - } - - if (projectedSchema == null) { - contextBuilder.project(icebergSchema); - } else { - contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); - } - - contextBuilder.exposeLocality( - SourceUtil.isLocalityEnabled(table, readableConfig, exposeLocality)); - contextBuilder.planParallelism( - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); - - contextBuilder.resolveConfig(table, readOptions, readableConfig); - - return new FlinkInputFormat( - tableLoader, icebergSchema, io, encryption, contextBuilder.build()); - } - - public DataStream build() { - Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); - FlinkInputFormat format = buildFormat(); - - ScanContext context = contextBuilder.build(); - TypeInformation typeInfo = - FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project())); - - if (!context.isStreaming()) { - int parallelism = - SourceUtil.inferParallelism( - readableConfig, - context.limit(), - () -> { - try { - return format.createInputSplits(0).length; - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to create iceberg input splits for table: " + table, e); - } - }); - if (env.getMaxParallelism() > 0) { - parallelism = Math.min(parallelism, env.getMaxParallelism()); - } - return env.createInput(format, typeInfo).setParallelism(parallelism); - } else { - StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context); - - String monitorFunctionName = String.format("Iceberg table (%s) monitor", table); - String readerOperatorName = String.format("Iceberg table (%s) reader", table); - - return env.addSource(function, monitorFunctionName) - .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format)); - } - } - } - - public static boolean isBounded(Map properties) { - return !PropertyUtil.propertyAsBoolean(properties, FlinkReadOptions.STREAMING, false); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java deleted file mode 100644 index 15078809714f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java +++ /dev/null @@ -1,189 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.IncrementalAppendScan; -import org.apache.iceberg.Scan; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.io.CloseableIterable; -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.apache.iceberg.util.Tasks; - -@Internal -public class FlinkSplitPlanner { - private FlinkSplitPlanner() {} - - static FlinkInputSplit[] planInputSplits( - Table table, ScanContext context, ExecutorService workerPool) { - try (CloseableIterable tasksIterable = - planTasks(table, context, workerPool)) { - List tasks = Lists.newArrayList(tasksIterable); - FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()]; - boolean exposeLocality = context.exposeLocality(); - - Tasks.range(tasks.size()) - .stopOnFailure() - .executeWith(exposeLocality ? workerPool : null) - .run( - index -> { - CombinedScanTask task = tasks.get(index); - String[] hostnames = null; - if (exposeLocality) { - hostnames = Util.blockLocations(table.io(), task); - } - splits[index] = new FlinkInputSplit(index, task, hostnames); - }); - return splits; - } catch (IOException e) { - throw new UncheckedIOException("Failed to process tasks iterable", e); - } - } - - /** This returns splits for the FLIP-27 source */ - public static List planIcebergSourceSplits( - Table table, ScanContext context, ExecutorService workerPool) { - try (CloseableIterable tasksIterable = - planTasks(table, context, workerPool)) { - return Lists.newArrayList( - CloseableIterable.transform(tasksIterable, IcebergSourceSplit::fromCombinedScanTask)); - } catch (IOException e) { - throw new UncheckedIOException("Failed to process task iterable: ", e); - } - } - - static CloseableIterable planTasks( - Table table, ScanContext context, ExecutorService workerPool) { - ScanMode scanMode = checkScanMode(context); - if (scanMode == ScanMode.INCREMENTAL_APPEND_SCAN) { - IncrementalAppendScan scan = table.newIncrementalAppendScan(); - scan = refineScanWithBaseConfigs(scan, context, workerPool); - - if (context.startTag() != null) { - Preconditions.checkArgument( - table.snapshot(context.startTag()) != null, - "Cannot find snapshot with tag %s", - context.startTag()); - scan = scan.fromSnapshotExclusive(table.snapshot(context.startTag()).snapshotId()); - } - - if (context.startSnapshotId() != null) { - Preconditions.checkArgument( - context.startTag() == null, "START_SNAPSHOT_ID and START_TAG cannot both be set"); - scan = scan.fromSnapshotExclusive(context.startSnapshotId()); - } - - if (context.endTag() != null) { - Preconditions.checkArgument( - table.snapshot(context.endTag()) != null, - "Cannot find snapshot with tag %s", - context.endTag()); - scan = scan.toSnapshot(table.snapshot(context.endTag()).snapshotId()); - } - - if (context.endSnapshotId() != null) { - Preconditions.checkArgument( - context.endTag() == null, "END_SNAPSHOT_ID and END_TAG cannot both be set"); - scan = scan.toSnapshot(context.endSnapshotId()); - } - - return scan.planTasks(); - } else { - TableScan scan = table.newScan(); - scan = refineScanWithBaseConfigs(scan, context, workerPool); - - if (context.snapshotId() != null) { - scan = scan.useSnapshot(context.snapshotId()); - } else if (context.tag() != null) { - scan = scan.useRef(context.tag()); - } else if (context.branch() != null) { - scan = scan.useRef(context.branch()); - } - - if (context.asOfTimestamp() != null) { - scan = scan.asOfTime(context.asOfTimestamp()); - } - - return scan.planTasks(); - } - } - - @VisibleForTesting - enum ScanMode { - BATCH, - INCREMENTAL_APPEND_SCAN - } - - @VisibleForTesting - static ScanMode checkScanMode(ScanContext context) { - if (context.startSnapshotId() != null - || context.endSnapshotId() != null - || context.startTag() != null - || context.endTag() != null) { - return ScanMode.INCREMENTAL_APPEND_SCAN; - } else { - return ScanMode.BATCH; - } - } - - /** refine scan with common configs */ - private static > T refineScanWithBaseConfigs( - T scan, ScanContext context, ExecutorService workerPool) { - T refinedScan = - scan.caseSensitive(context.caseSensitive()).project(context.project()).planWith(workerPool); - - if (context.includeColumnStats()) { - refinedScan = refinedScan.includeColumnStats(); - } - - if (context.includeStatsForColumns() != null) { - refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); - } - - refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); - - refinedScan = - refinedScan.option(TableProperties.SPLIT_LOOKBACK, context.splitLookback().toString()); - - refinedScan = - refinedScan.option( - TableProperties.SPLIT_OPEN_FILE_COST, context.splitOpenFileCost().toString()); - - if (context.filters() != null) { - for (Expression filter : context.filters()) { - refinedScan = refinedScan.filter(filter); - } - } - - return refinedScan; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java deleted file mode 100644 index 0655cf87a996..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ /dev/null @@ -1,543 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.UUID; -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.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.SourceReader; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadConf; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; -import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; -import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner; -import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl; -import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; -import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; -import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; -import org.apache.iceberg.flink.source.reader.IcebergSourceReader; -import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; -import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; -import org.apache.iceberg.flink.source.reader.ReaderFunction; -import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; -import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -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.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Experimental -public class IcebergSource implements Source { - private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); - - // This table loader can be closed, and it is only safe to use this instance for resource - // independent information (e.g. a table name). Copies of this are required to avoid lifecycle - // management conflicts with the user provided table loader. e.g. a copy of this is required for - // split planning, which uses the underlying io, and should be closed after split planning is - // complete. - private final TableLoader tableLoader; - private final ScanContext scanContext; - private final ReaderFunction readerFunction; - private final SplitAssignerFactory assignerFactory; - private final SerializableComparator splitComparator; - private final SerializableRecordEmitter emitter; - private final String tableName; - - IcebergSource( - TableLoader tableLoader, - ScanContext scanContext, - ReaderFunction readerFunction, - SplitAssignerFactory assignerFactory, - SerializableComparator splitComparator, - Table table, - SerializableRecordEmitter emitter) { - Preconditions.checkNotNull(tableLoader, "tableLoader is required."); - Preconditions.checkNotNull(readerFunction, "readerFunction is required."); - Preconditions.checkNotNull(assignerFactory, "assignerFactory is required."); - Preconditions.checkNotNull(table, "table is required."); - this.tableLoader = tableLoader; - this.scanContext = scanContext; - this.readerFunction = readerFunction; - this.assignerFactory = assignerFactory; - this.splitComparator = splitComparator; - this.emitter = emitter; - this.tableName = table.name(); - } - - String name() { - return "IcebergSource-" + tableName; - } - - private String planningThreadName() { - // Ideally, operatorId should be used as the threadPoolName as Flink guarantees its uniqueness - // within a job. SplitEnumeratorContext doesn't expose the OperatorCoordinator.Context, which - // would contain the OperatorID. Need to discuss with Flink community whether it is ok to expose - // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" - // from SourceCoordinatorContext implementation. For now,

    - is used as - // the unique thread pool name. - return tableName + "-" + UUID.randomUUID(); - } - - private List planSplitsForBatch(String threadName) { - ExecutorService workerPool = - ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); - try (TableLoader loader = tableLoader.clone()) { - loader.open(); - List splits = - FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); - LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table loader", e); - } finally { - workerPool.shutdown(); - } - } - - @Override - public Boundedness getBoundedness() { - return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; - } - - @Override - public SourceReader createReader(SourceReaderContext readerContext) { - IcebergSourceReaderMetrics metrics = - new IcebergSourceReaderMetrics(readerContext.metricGroup(), tableName); - return new IcebergSourceReader<>( - emitter, metrics, readerFunction, splitComparator, readerContext); - } - - @Override - public SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext) { - return createEnumerator(enumContext, null); - } - - @Override - public SplitEnumerator restoreEnumerator( - SplitEnumeratorContext enumContext, IcebergEnumeratorState enumState) { - return createEnumerator(enumContext, enumState); - } - - @Override - public SimpleVersionedSerializer getSplitSerializer() { - return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); - } - - @Override - public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { - return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); - } - - private SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext, - @Nullable IcebergEnumeratorState enumState) { - SplitAssigner assigner; - if (enumState == null) { - assigner = assignerFactory.createAssigner(); - } else { - LOG.info( - "Iceberg source restored {} splits from state for table {}", - enumState.pendingSplits().size(), - tableName); - assigner = assignerFactory.createAssigner(enumState.pendingSplits()); - } - if (scanContext.isStreaming()) { - ContinuousSplitPlanner splitPlanner = - new ContinuousSplitPlannerImpl(tableLoader, scanContext, planningThreadName()); - return new ContinuousIcebergEnumerator( - enumContext, assigner, scanContext, splitPlanner, enumState); - } else { - List splits = planSplitsForBatch(planningThreadName()); - assigner.onDiscoveredSplits(splits); - return new StaticIcebergEnumerator(enumContext, assigner); - } - } - - public static Builder builder() { - return new Builder<>(); - } - - public static Builder forRowData() { - return new Builder<>(); - } - - public static class Builder { - private TableLoader tableLoader; - private Table table; - private SplitAssignerFactory splitAssignerFactory; - private SerializableComparator splitComparator; - private ReaderFunction readerFunction; - private ReadableConfig flinkConfig = new Configuration(); - private final ScanContext.Builder contextBuilder = ScanContext.builder(); - private TableSchema projectedFlinkSchema; - private Boolean exposeLocality; - - private final Map readOptions = Maps.newHashMap(); - - Builder() {} - - public Builder tableLoader(TableLoader loader) { - this.tableLoader = loader; - return this; - } - - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - public Builder assignerFactory(SplitAssignerFactory assignerFactory) { - this.splitAssignerFactory = assignerFactory; - return this; - } - - public Builder splitComparator( - SerializableComparator newSplitComparator) { - this.splitComparator = newSplitComparator; - return this; - } - - public Builder readerFunction(ReaderFunction newReaderFunction) { - this.readerFunction = newReaderFunction; - return this; - } - - public Builder flinkConfig(ReadableConfig config) { - this.flinkConfig = config; - return this; - } - - public Builder caseSensitive(boolean newCaseSensitive) { - readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(newCaseSensitive)); - return this; - } - - public Builder useSnapshotId(Long newSnapshotId) { - if (newSnapshotId != null) { - readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(newSnapshotId)); - } - return this; - } - - public Builder streamingStartingStrategy(StreamingStartingStrategy newStartingStrategy) { - readOptions.put(FlinkReadOptions.STARTING_STRATEGY, newStartingStrategy.name()); - return this; - } - - public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { - if (newStartSnapshotTimestamp != null) { - readOptions.put( - FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key(), - Long.toString(newStartSnapshotTimestamp)); - } - return this; - } - - public Builder startSnapshotId(Long newStartSnapshotId) { - if (newStartSnapshotId != null) { - readOptions.put( - FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(newStartSnapshotId)); - } - return this; - } - - public Builder tag(String tag) { - readOptions.put(FlinkReadOptions.TAG.key(), tag); - return this; - } - - public Builder branch(String branch) { - readOptions.put(FlinkReadOptions.BRANCH.key(), branch); - return this; - } - - public Builder startTag(String startTag) { - readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); - return this; - } - - public Builder endTag(String endTag) { - readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); - return this; - } - - public Builder endSnapshotId(Long newEndSnapshotId) { - if (newEndSnapshotId != null) { - readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(newEndSnapshotId)); - } - return this; - } - - public Builder asOfTimestamp(Long newAsOfTimestamp) { - if (newAsOfTimestamp != null) { - readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(newAsOfTimestamp)); - } - return this; - } - - public Builder splitSize(Long newSplitSize) { - if (newSplitSize != null) { - readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(newSplitSize)); - } - return this; - } - - public Builder splitLookback(Integer newSplitLookback) { - if (newSplitLookback != null) { - readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(newSplitLookback)); - } - return this; - } - - public Builder splitOpenFileCost(Long newSplitOpenFileCost) { - if (newSplitOpenFileCost != null) { - readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(newSplitOpenFileCost)); - } - - return this; - } - - public Builder streaming(boolean streaming) { - readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); - return this; - } - - public Builder monitorInterval(Duration newMonitorInterval) { - if (newMonitorInterval != null) { - readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, newMonitorInterval.toNanos() + " ns"); - } - return this; - } - - public Builder nameMapping(String newNameMapping) { - readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, newNameMapping); - return this; - } - - public Builder project(Schema newProjectedSchema) { - this.contextBuilder.project(newProjectedSchema); - return this; - } - - public Builder project(TableSchema newProjectedFlinkSchema) { - this.projectedFlinkSchema = newProjectedFlinkSchema; - return this; - } - - public Builder filters(List newFilters) { - this.contextBuilder.filters(newFilters); - return this; - } - - public Builder limit(Long newLimit) { - if (newLimit != null) { - readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); - } - return this; - } - - public Builder includeColumnStats(boolean newIncludeColumnStats) { - readOptions.put( - FlinkReadOptions.INCLUDE_COLUMN_STATS, Boolean.toString(newIncludeColumnStats)); - return this; - } - - public Builder planParallelism(int planParallelism) { - readOptions.put( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key(), - Integer.toString(planParallelism)); - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder maxAllowedPlanningFailures(int maxAllowedPlanningFailures) { - readOptions.put( - FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.key(), - Integer.toString(maxAllowedPlanningFailures)); - return this; - } - - /** - * Set the read properties for Flink source. View the supported properties in {@link - * FlinkReadOptions} - */ - public Builder set(String property, String value) { - readOptions.put(property, value); - return this; - } - - /** - * Set the read properties for Flink source. View the supported properties in {@link - * FlinkReadOptions} - */ - public Builder setAll(Map properties) { - readOptions.putAll(properties); - return this; - } - - /** - * Emits watermarks once per split based on the min value of column statistics from files - * metadata in the given split. The generated watermarks are also used for ordering the splits - * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider - * setting {@link #watermarkColumnTimeUnit(TimeUnit)}. - * - *

    Consider setting `read.split.open-file-cost` to prevent combining small files to a single - * split when the watermark is used for watermark alignment. - */ - public Builder watermarkColumn(String columnName) { - Preconditions.checkArgument( - splitAssignerFactory == null, - "Watermark column and SplitAssigner should not be set in the same source"); - readOptions.put(FlinkReadOptions.WATERMARK_COLUMN, columnName); - return this; - } - - /** - * When the type of the {@link #watermarkColumn} is {@link - * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the - * value. The default value is {@link TimeUnit#MICROSECONDS}. - */ - public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { - readOptions.put(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT, timeUnit.name()); - return this; - } - - /** @deprecated Use {@link #setAll} instead. */ - @Deprecated - public Builder properties(Map properties) { - readOptions.putAll(properties); - return this; - } - - public IcebergSource build() { - if (table == null) { - try (TableLoader loader = tableLoader) { - loader.open(); - this.table = tableLoader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - contextBuilder.resolveConfig(table, readOptions, flinkConfig); - Schema icebergSchema = table.schema(); - if (projectedFlinkSchema != null) { - contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); - } - - SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); - FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, flinkConfig); - String watermarkColumn = flinkReadConf.watermarkColumn(); - TimeUnit watermarkTimeUnit = flinkReadConf.watermarkColumnTimeUnit(); - - if (watermarkColumn != null) { - // Column statistics is needed for watermark generation - contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); - - SplitWatermarkExtractor watermarkExtractor = - new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); - emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); - splitAssignerFactory = - new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); - } - - ScanContext context = contextBuilder.build(); - if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } - } - - if (splitAssignerFactory == null) { - if (splitComparator == null) { - splitAssignerFactory = new SimpleSplitAssignerFactory(); - } else { - splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); - } - } - - // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource<>( - tableLoader, - context, - readerFunction, - splitAssignerFactory, - splitComparator, - table, - emitter); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java deleted file mode 100644 index 610657e8d47b..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ /dev/null @@ -1,229 +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.source; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.types.DataType; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkFilters; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.SplitAssignerType; -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; - -/** Flink Iceberg table source. */ -@Internal -public class IcebergTableSource - implements ScanTableSource, - SupportsProjectionPushDown, - SupportsFilterPushDown, - SupportsLimitPushDown { - - private int[] projectedFields; - private Long limit; - private List filters; - - private final TableLoader loader; - private final TableSchema schema; - private final Map properties; - private final boolean isLimitPushDown; - private final ReadableConfig readableConfig; - - private IcebergTableSource(IcebergTableSource toCopy) { - this.loader = toCopy.loader; - this.schema = toCopy.schema; - this.properties = toCopy.properties; - this.projectedFields = toCopy.projectedFields; - this.isLimitPushDown = toCopy.isLimitPushDown; - this.limit = toCopy.limit; - this.filters = toCopy.filters; - this.readableConfig = toCopy.readableConfig; - } - - public IcebergTableSource( - TableLoader loader, - TableSchema schema, - Map properties, - ReadableConfig readableConfig) { - this(loader, schema, properties, null, false, null, ImmutableList.of(), readableConfig); - } - - private IcebergTableSource( - TableLoader loader, - TableSchema schema, - Map properties, - int[] projectedFields, - boolean isLimitPushDown, - Long limit, - List filters, - ReadableConfig readableConfig) { - this.loader = loader; - this.schema = schema; - this.properties = properties; - this.projectedFields = projectedFields; - this.isLimitPushDown = isLimitPushDown; - this.limit = limit; - this.filters = filters; - this.readableConfig = readableConfig; - } - - @Override - public void applyProjection(int[][] projectFields) { - this.projectedFields = new int[projectFields.length]; - for (int i = 0; i < projectFields.length; i++) { - Preconditions.checkArgument( - projectFields[i].length == 1, "Don't support nested projection in iceberg source now."); - this.projectedFields[i] = projectFields[i][0]; - } - } - - private DataStream createDataStream(StreamExecutionEnvironment execEnv) { - return FlinkSource.forRowData() - .env(execEnv) - .tableLoader(loader) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConf(readableConfig) - .build(); - } - - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { - SplitAssignerType assignerType = - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; - } - - private TableSchema getProjectedSchema() { - if (projectedFields == null) { - return schema; - } else { - String[] fullNames = schema.getFieldNames(); - DataType[] fullTypes = schema.getFieldDataTypes(); - return TableSchema.builder() - .fields( - Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), - Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)) - .build(); - } - } - - @Override - public void applyLimit(long newLimit) { - this.limit = newLimit; - } - - @Override - public Result applyFilters(List flinkFilters) { - List acceptedFilters = Lists.newArrayList(); - List expressions = Lists.newArrayList(); - - for (ResolvedExpression resolvedExpression : flinkFilters) { - Optional icebergExpression = FlinkFilters.convert(resolvedExpression); - if (icebergExpression.isPresent()) { - expressions.add(icebergExpression.get()); - acceptedFilters.add(resolvedExpression); - } - } - - this.filters = expressions; - return Result.of(acceptedFilters, flinkFilters); - } - - @Override - public boolean supportsNestedProjection() { - // TODO: support nested projection - return false; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return new DataStreamScanProvider() { - @Override - public DataStream produceDataStream( - ProviderContext providerContext, StreamExecutionEnvironment execEnv) { - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { - return createFLIP27Stream(execEnv); - } else { - return createDataStream(execEnv); - } - } - - @Override - public boolean isBounded() { - return FlinkSource.isBounded(properties); - } - }; - } - - @Override - public DynamicTableSource copy() { - return new IcebergTableSource(this); - } - - @Override - public String asSummaryString() { - return "Iceberg table source"; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java deleted file mode 100644 index 88364f4e87b1..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ /dev/null @@ -1,243 +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.source; - -import java.util.List; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.DeleteFilter; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -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.RowDataProjection; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.PartitionUtil; - -@Internal -public class RowDataFileScanTaskReader implements FileScanTaskReader { - - private final Schema tableSchema; - private final Schema projectedSchema; - private final String nameMapping; - private final boolean caseSensitive; - private final FlinkSourceFilter rowFilter; - - public RowDataFileScanTaskReader( - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - List filters) { - this.tableSchema = tableSchema; - this.projectedSchema = projectedSchema; - this.nameMapping = nameMapping; - this.caseSensitive = caseSensitive; - - if (filters != null && !filters.isEmpty()) { - Expression combinedExpression = - filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); - this.rowFilter = - new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); - } else { - this.rowFilter = null; - } - } - - @Override - public CloseableIterator open( - FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { - Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds()); - - Map idToConstant = - partitionSchema.columns().isEmpty() - ? ImmutableMap.of() - : PartitionUtil.constantsMap(task, RowDataUtil::convertConstant); - - FlinkDeleteFilter deletes = - new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor); - CloseableIterable iterable = - deletes.filter( - newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor)); - - // Project the RowData to remove the extra meta columns. - if (!projectedSchema.sameSchema(deletes.requiredSchema())) { - RowDataProjection rowDataProjection = - RowDataProjection.create( - deletes.requiredRowType(), - deletes.requiredSchema().asStruct(), - projectedSchema.asStruct()); - iterable = CloseableIterable.transform(iterable, rowDataProjection::wrap); - } - - return iterable.iterator(); - } - - private CloseableIterable newIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - CloseableIterable iter; - if (task.isDataTask()) { - throw new UnsupportedOperationException("Cannot read data task."); - } else { - switch (task.file().format()) { - case PARQUET: - iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case AVRO: - iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case ORC: - iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - default: - throw new UnsupportedOperationException( - "Cannot read unknown format: " + task.file().format()); - } - } - - if (rowFilter != null) { - return CloseableIterable.filter(iter, rowFilter::filter); - } - return iter; - } - - private CloseableIterable newAvroIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Avro.ReadBuilder builder = - Avro.read(inputFilesDecryptor.getInputFile(task)) - .reuseContainers() - .project(schema) - .split(task.start(), task.length()) - .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newParquetIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Parquet.ReadBuilder builder = - Parquet.read(inputFilesDecryptor.getInputFile(task)) - .split(task.start(), task.length()) - .project(schema) - .createReaderFunc( - fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive) - .reuseContainers(); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newOrcIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Schema readSchemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); - - ORC.ReadBuilder builder = - ORC.read(inputFilesDecryptor.getInputFile(task)) - .project(readSchemaWithoutConstantAndMetadataFields) - .split(task.start(), task.length()) - .createReaderFunc( - readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private static class FlinkDeleteFilter extends DeleteFilter { - private final RowType requiredRowType; - private final RowDataWrapper asStructLike; - private final InputFilesDecryptor inputFilesDecryptor; - - FlinkDeleteFilter( - FileScanTask task, - Schema tableSchema, - Schema requestedSchema, - InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); - this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); - this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); - this.inputFilesDecryptor = inputFilesDecryptor; - } - - public RowType requiredRowType() { - return requiredRowType; - } - - @Override - protected StructLike asStructLike(RowData row) { - return asStructLike.wrap(row); - } - - @Override - protected InputFile getInputFile(String location) { - return inputFilesDecryptor.getInputFile(location); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java deleted file mode 100644 index c958604c004a..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ /dev/null @@ -1,172 +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.source; - -import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; - -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; -import org.apache.iceberg.flink.sink.TaskWriterFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class RowDataRewriter { - - private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); - - private final Schema schema; - private final String nameMapping; - private final FileIO io; - private final boolean caseSensitive; - private final EncryptionManager encryptionManager; - private final TaskWriterFactory taskWriterFactory; - private final String tableName; - - public RowDataRewriter( - Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { - this.schema = table.schema(); - this.caseSensitive = caseSensitive; - this.io = io; - this.encryptionManager = encryptionManager; - this.nameMapping = - PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); - this.tableName = table.name(); - - String formatString = - PropertyUtil.propertyAsString( - table.properties(), - TableProperties.DEFAULT_FILE_FORMAT, - TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); - FileFormat format = FileFormat.fromString(formatString); - RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); - this.taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - flinkSchema, - Long.MAX_VALUE, - format, - table.properties(), - null, - false); - } - - public List rewriteDataForTasks( - DataStream dataStream, int parallelism) throws Exception { - RewriteMap map = - new RewriteMap( - schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); - DataStream> ds = dataStream.map(map).setParallelism(parallelism); - return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() - .flatMap(Collection::stream) - .collect(Collectors.toList()); - } - - public static class RewriteMap extends RichMapFunction> { - - private TaskWriter writer; - private int subTaskId; - private int attemptId; - - private final Schema schema; - private final String nameMapping; - private final FileIO io; - private final boolean caseSensitive; - private final EncryptionManager encryptionManager; - private final TaskWriterFactory taskWriterFactory; - private final RowDataFileScanTaskReader rowDataReader; - - public RewriteMap( - Schema schema, - String nameMapping, - FileIO io, - boolean caseSensitive, - EncryptionManager encryptionManager, - TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; - this.io = io; - this.caseSensitive = caseSensitive; - this.encryptionManager = encryptionManager; - this.taskWriterFactory = taskWriterFactory; - this.rowDataReader = - new RowDataFileScanTaskReader( - schema, schema, nameMapping, caseSensitive, Collections.emptyList()); - } - - @Override - public void open(Configuration parameters) { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); - // Initialize the task writer factory. - this.taskWriterFactory.initialize(subTaskId, attemptId); - } - - @Override - public List map(CombinedScanTask task) throws Exception { - // Initialize the task writer. - this.writer = taskWriterFactory.create(); - try (DataIterator iterator = - new DataIterator<>(rowDataReader, task, io, encryptionManager)) { - while (iterator.hasNext()) { - RowData rowData = iterator.next(); - writer.write(rowData); - } - return Lists.newArrayList(writer.dataFiles()); - } catch (Throwable originalThrowable) { - try { - LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); - writer.abort(); - LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); - } catch (Throwable inner) { - if (originalThrowable != inner) { - originalThrowable.addSuppressed(inner); - LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); - } - } - - if (originalThrowable instanceof Exception) { - throw originalThrowable; - } else { - throw new RuntimeException(originalThrowable); - } - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java deleted file mode 100644 index 8f95e3e554a0..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.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.flink.source; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.annotation.Internal; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.flink.FlinkSchemaUtil; - -/** - * This is not serializable because Avro {@link Schema} is not actually serializable, even though it - * implements {@link Serializable} interface. - */ -@Internal -public class RowDataToAvroGenericRecordConverter implements Function { - private final RowDataToAvroConverters.RowDataToAvroConverter converter; - private final Schema avroSchema; - - private RowDataToAvroGenericRecordConverter(RowType rowType, Schema avroSchema) { - this.converter = RowDataToAvroConverters.createConverter(rowType); - this.avroSchema = avroSchema; - } - - @Override - public GenericRecord apply(RowData rowData) { - return (GenericRecord) converter.convert(avroSchema, rowData); - } - - /** Create a converter based on Iceberg schema */ - public static RowDataToAvroGenericRecordConverter fromIcebergSchema( - String tableName, org.apache.iceberg.Schema icebergSchema) { - RowType rowType = FlinkSchemaUtil.convert(icebergSchema); - Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); - return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); - } - - /** Create a mapper based on Avro schema */ - public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { - DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); - return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java deleted file mode 100644 index cf57a126ae59..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ /dev/null @@ -1,592 +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.source; - -import java.io.Serializable; -import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadConf; -import org.apache.iceberg.flink.FlinkReadOptions; - -/** Context object with optional arguments for a Flink Scan. */ -@Internal -public class ScanContext implements Serializable { - - private static final long serialVersionUID = 1L; - - private final boolean caseSensitive; - private final boolean exposeLocality; - private final Long snapshotId; - private final String branch; - private final String tag; - private final StreamingStartingStrategy startingStrategy; - private final Long startSnapshotId; - private final Long startSnapshotTimestamp; - private final Long endSnapshotId; - private final Long asOfTimestamp; - private final String startTag; - private final String endTag; - private final Long splitSize; - private final Integer splitLookback; - private final Long splitOpenFileCost; - private final boolean isStreaming; - private final Duration monitorInterval; - - private final String nameMapping; - private final Schema schema; - private final List filters; - private final long limit; - private final boolean includeColumnStats; - private final Collection includeStatsForColumns; - private final Integer planParallelism; - private final int maxPlanningSnapshotCount; - private final int maxAllowedPlanningFailures; - private final String watermarkColumn; - private final TimeUnit watermarkColumnTimeUnit; - - private ScanContext( - boolean caseSensitive, - Long snapshotId, - StreamingStartingStrategy startingStrategy, - Long startSnapshotTimestamp, - Long startSnapshotId, - Long endSnapshotId, - Long asOfTimestamp, - Long splitSize, - Integer splitLookback, - Long splitOpenFileCost, - boolean isStreaming, - Duration monitorInterval, - String nameMapping, - Schema schema, - List filters, - long limit, - boolean includeColumnStats, - Collection includeStatsForColumns, - boolean exposeLocality, - Integer planParallelism, - int maxPlanningSnapshotCount, - int maxAllowedPlanningFailures, - String watermarkColumn, - TimeUnit watermarkColumnTimeUnit, - String branch, - String tag, - String startTag, - String endTag) { - this.caseSensitive = caseSensitive; - this.snapshotId = snapshotId; - this.tag = tag; - this.branch = branch; - this.startingStrategy = startingStrategy; - this.startSnapshotTimestamp = startSnapshotTimestamp; - this.startSnapshotId = startSnapshotId; - this.endSnapshotId = endSnapshotId; - this.asOfTimestamp = asOfTimestamp; - this.startTag = startTag; - this.endTag = endTag; - this.splitSize = splitSize; - this.splitLookback = splitLookback; - this.splitOpenFileCost = splitOpenFileCost; - this.isStreaming = isStreaming; - this.monitorInterval = monitorInterval; - - this.nameMapping = nameMapping; - this.schema = schema; - this.filters = filters; - this.limit = limit; - this.includeColumnStats = includeColumnStats; - this.includeStatsForColumns = includeStatsForColumns; - this.exposeLocality = exposeLocality; - this.planParallelism = planParallelism; - this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; - this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; - this.watermarkColumn = watermarkColumn; - this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; - - validate(); - } - - private void validate() { - if (isStreaming) { - if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { - Preconditions.checkArgument( - startSnapshotId != null, - "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); - Preconditions.checkArgument( - startSnapshotTimestamp == null, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) { - Preconditions.checkArgument( - startSnapshotTimestamp != null, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); - Preconditions.checkArgument( - startSnapshotId == null, - "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - - Preconditions.checkArgument( - tag == null, - String.format("Cannot scan table using ref %s configured for streaming reader", tag)); - } - - Preconditions.checkArgument( - !(startTag != null && startSnapshotId() != null), - "START_SNAPSHOT_ID and START_TAG cannot both be set."); - - Preconditions.checkArgument( - !(endTag != null && endSnapshotId() != null), - "END_SNAPSHOT_ID and END_TAG cannot both be set."); - - Preconditions.checkArgument( - maxAllowedPlanningFailures >= -1, - "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); - } - - public boolean caseSensitive() { - return caseSensitive; - } - - public Long snapshotId() { - return snapshotId; - } - - public String branch() { - return branch; - } - - public String tag() { - return tag; - } - - public String startTag() { - return startTag; - } - - public String endTag() { - return endTag; - } - - public StreamingStartingStrategy streamingStartingStrategy() { - return startingStrategy; - } - - public Long startSnapshotTimestamp() { - return startSnapshotTimestamp; - } - - public Long startSnapshotId() { - return startSnapshotId; - } - - public Long endSnapshotId() { - return endSnapshotId; - } - - public Long asOfTimestamp() { - return asOfTimestamp; - } - - public Long splitSize() { - return splitSize; - } - - public Integer splitLookback() { - return splitLookback; - } - - public Long splitOpenFileCost() { - return splitOpenFileCost; - } - - public boolean isStreaming() { - return isStreaming; - } - - public Duration monitorInterval() { - return monitorInterval; - } - - public String nameMapping() { - return nameMapping; - } - - public Schema project() { - return schema; - } - - public List filters() { - return filters; - } - - public long limit() { - return limit; - } - - public boolean includeColumnStats() { - return includeColumnStats; - } - - public Collection includeStatsForColumns() { - return includeStatsForColumns; - } - - public boolean exposeLocality() { - return exposeLocality; - } - - public Integer planParallelism() { - return planParallelism; - } - - public int maxPlanningSnapshotCount() { - return maxPlanningSnapshotCount; - } - - public int maxAllowedPlanningFailures() { - return maxAllowedPlanningFailures; - } - - public String watermarkColumn() { - return watermarkColumn; - } - - public TimeUnit watermarkColumnTimeUnit() { - return watermarkColumnTimeUnit; - } - - public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { - return ScanContext.builder() - .caseSensitive(caseSensitive) - .useSnapshotId(null) - .useBranch(branch) - .useTag(null) - .startSnapshotId(newStartSnapshotId) - .endSnapshotId(newEndSnapshotId) - .startTag(null) - .endTag(null) - .asOfTimestamp(null) - .splitSize(splitSize) - .splitLookback(splitLookback) - .splitOpenFileCost(splitOpenFileCost) - .streaming(isStreaming) - .monitorInterval(monitorInterval) - .nameMapping(nameMapping) - .project(schema) - .filters(filters) - .limit(limit) - .includeColumnStats(includeColumnStats) - .includeColumnStats(includeStatsForColumns) - .exposeLocality(exposeLocality) - .planParallelism(planParallelism) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .watermarkColumn(watermarkColumn) - .watermarkColumnTimeUnit(watermarkColumnTimeUnit) - .build(); - } - - public ScanContext copyWithSnapshotId(long newSnapshotId) { - return ScanContext.builder() - .caseSensitive(caseSensitive) - .useSnapshotId(newSnapshotId) - .useBranch(branch) - .useTag(tag) - .startSnapshotId(null) - .endSnapshotId(null) - .startTag(null) - .endTag(null) - .asOfTimestamp(null) - .splitSize(splitSize) - .splitLookback(splitLookback) - .splitOpenFileCost(splitOpenFileCost) - .streaming(isStreaming) - .monitorInterval(monitorInterval) - .nameMapping(nameMapping) - .project(schema) - .filters(filters) - .limit(limit) - .includeColumnStats(includeColumnStats) - .includeColumnStats(includeStatsForColumns) - .exposeLocality(exposeLocality) - .planParallelism(planParallelism) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .watermarkColumn(watermarkColumn) - .watermarkColumnTimeUnit(watermarkColumnTimeUnit) - .build(); - } - - public static Builder builder() { - return new Builder(); - } - - public static class Builder { - private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); - private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); - private String branch = FlinkReadOptions.BRANCH.defaultValue(); - private String tag = FlinkReadOptions.TAG.defaultValue(); - private String startTag = FlinkReadOptions.START_TAG.defaultValue(); - private String endTag = FlinkReadOptions.END_TAG.defaultValue(); - private StreamingStartingStrategy startingStrategy = - FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue(); - private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue(); - private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue(); - private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue(); - private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue(); - private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue(); - private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue(); - private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue(); - private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue(); - private Duration monitorInterval = - TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()); - private String nameMapping; - private Schema projectedSchema; - private List filters; - private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); - private boolean includeColumnStats = - FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); - private Collection includeStatsForColumns = null; - private boolean exposeLocality; - private Integer planParallelism = - FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); - private int maxPlanningSnapshotCount = - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); - private int maxAllowedPlanningFailures = - FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); - private String watermarkColumn = FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue(); - private TimeUnit watermarkColumnTimeUnit = - FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue(); - - private Builder() {} - - public Builder caseSensitive(boolean newCaseSensitive) { - this.caseSensitive = newCaseSensitive; - return this; - } - - public Builder useSnapshotId(Long newSnapshotId) { - this.snapshotId = newSnapshotId; - return this; - } - - public Builder useTag(String newTag) { - this.tag = newTag; - return this; - } - - public Builder useBranch(String newBranch) { - this.branch = newBranch; - return this; - } - - public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { - this.startingStrategy = newStartingStrategy; - return this; - } - - public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { - this.startSnapshotTimestamp = newStartSnapshotTimestamp; - return this; - } - - public Builder startSnapshotId(Long newStartSnapshotId) { - this.startSnapshotId = newStartSnapshotId; - return this; - } - - public Builder endSnapshotId(Long newEndSnapshotId) { - this.endSnapshotId = newEndSnapshotId; - return this; - } - - public Builder startTag(String newStartTag) { - this.startTag = newStartTag; - return this; - } - - public Builder endTag(String newEndTag) { - this.endTag = newEndTag; - return this; - } - - public Builder asOfTimestamp(Long newAsOfTimestamp) { - this.asOfTimestamp = newAsOfTimestamp; - return this; - } - - public Builder splitSize(Long newSplitSize) { - this.splitSize = newSplitSize; - return this; - } - - public Builder splitLookback(Integer newSplitLookback) { - this.splitLookback = newSplitLookback; - return this; - } - - public Builder splitOpenFileCost(Long newSplitOpenFileCost) { - this.splitOpenFileCost = newSplitOpenFileCost; - return this; - } - - public Builder streaming(boolean streaming) { - this.isStreaming = streaming; - return this; - } - - public Builder monitorInterval(Duration newMonitorInterval) { - this.monitorInterval = newMonitorInterval; - return this; - } - - public Builder nameMapping(String newNameMapping) { - this.nameMapping = newNameMapping; - return this; - } - - public Builder project(Schema newProjectedSchema) { - this.projectedSchema = newProjectedSchema; - return this; - } - - public Builder filters(List newFilters) { - this.filters = newFilters; - return this; - } - - public Builder limit(long newLimit) { - this.limit = newLimit; - return this; - } - - public Builder includeColumnStats(boolean newIncludeColumnStats) { - this.includeColumnStats = newIncludeColumnStats; - return this; - } - - public Builder includeColumnStats(Collection newIncludeStatsForColumns) { - this.includeStatsForColumns = newIncludeStatsForColumns; - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder planParallelism(Integer parallelism) { - this.planParallelism = parallelism; - return this; - } - - public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { - this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; - return this; - } - - public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { - this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; - return this; - } - - public Builder watermarkColumn(String newWatermarkColumn) { - this.watermarkColumn = newWatermarkColumn; - return this; - } - - public Builder watermarkColumnTimeUnit(TimeUnit newWatermarkTimeUnit) { - this.watermarkColumnTimeUnit = newWatermarkTimeUnit; - return this; - } - - public Builder resolveConfig( - Table table, Map readOptions, ReadableConfig readableConfig) { - FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); - - return this.useSnapshotId(flinkReadConf.snapshotId()) - .useTag(flinkReadConf.tag()) - .useBranch(flinkReadConf.branch()) - .startTag(flinkReadConf.startTag()) - .endTag(flinkReadConf.endTag()) - .caseSensitive(flinkReadConf.caseSensitive()) - .asOfTimestamp(flinkReadConf.asOfTimestamp()) - .startingStrategy(flinkReadConf.startingStrategy()) - .startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp()) - .startSnapshotId(flinkReadConf.startSnapshotId()) - .endSnapshotId(flinkReadConf.endSnapshotId()) - .splitSize(flinkReadConf.splitSize()) - .splitLookback(flinkReadConf.splitLookback()) - .splitOpenFileCost(flinkReadConf.splitFileOpenCost()) - .streaming(flinkReadConf.streaming()) - .monitorInterval(flinkReadConf.monitorInterval()) - .nameMapping(flinkReadConf.nameMapping()) - .limit(flinkReadConf.limit()) - .planParallelism(flinkReadConf.workerPoolSize()) - .includeColumnStats(flinkReadConf.includeColumnStats()) - .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .watermarkColumn(flinkReadConf.watermarkColumn()) - .watermarkColumnTimeUnit(flinkReadConf.watermarkColumnTimeUnit()); - } - - public ScanContext build() { - return new ScanContext( - caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - projectedSchema, - filters, - limit, - includeColumnStats, - includeStatsForColumns, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - maxAllowedPlanningFailures, - watermarkColumn, - watermarkColumnTimeUnit, - branch, - tag, - startTag, - endTag); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java deleted file mode 100644 index 7c3a69dbc141..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java +++ /dev/null @@ -1,77 +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.source; - -import java.util.function.Supplier; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class SourceUtil { - private SourceUtil() {} - - static boolean isLocalityEnabled( - Table table, ReadableConfig readableConfig, Boolean exposeLocality) { - Boolean localityEnabled = - exposeLocality != null - ? exposeLocality - : readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO); - - if (localityEnabled != null && !localityEnabled) { - return false; - } - - return Util.mayHaveBlockLocations(table.io(), table.location()); - } - - /** - * Infer source parallelism. - * - * @param readableConfig Flink config. - * @param splitCountProvider Split count supplier. As the computation may involve expensive split - * discover, lazy evaluation is performed if inferring parallelism is enabled. - * @param limitCount limited output count. - */ - static int inferParallelism( - ReadableConfig readableConfig, long limitCount, Supplier splitCountProvider) { - int parallelism = - readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { - int maxInferParallelism = - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); - Preconditions.checkState( - maxInferParallelism >= 1, - FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() - + " cannot be less than 1"); - parallelism = Math.min(splitCountProvider.get(), maxInferParallelism); - } - - if (limitCount > 0) { - int limit = limitCount >= Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limitCount; - parallelism = Math.min(parallelism, limit); - } - - // parallelism must be positive. - parallelism = Math.max(1, parallelism); - return parallelism; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java deleted file mode 100644 index a07613aee59b..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java +++ /dev/null @@ -1,269 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This is the single (non-parallel) monitoring task which takes a {@link FlinkInputFormat}, it is - * responsible for: - * - *

      - *
    1. Monitoring snapshots of the Iceberg table. - *
    2. Creating the {@link FlinkInputSplit splits} corresponding to the incremental files - *
    3. Assigning them to downstream tasks for further processing. - *
    - * - *

    The splits to be read are forwarded to the downstream {@link StreamingReaderOperator} which - * can have parallelism greater than one. - */ -public class StreamingMonitorFunction extends RichSourceFunction - implements CheckpointedFunction { - - private static final Logger LOG = LoggerFactory.getLogger(StreamingMonitorFunction.class); - - private static final long INIT_LAST_SNAPSHOT_ID = -1L; - - private final TableLoader tableLoader; - private final ScanContext scanContext; - - private volatile boolean isRunning = true; - - // The checkpoint thread is not the same thread that running the function for SourceStreamTask - // now. It's necessary to - // mark this as volatile. - private volatile long lastSnapshotId = INIT_LAST_SNAPSHOT_ID; - - private transient SourceContext sourceContext; - private transient Table table; - private transient ListState lastSnapshotIdState; - private transient ExecutorService workerPool; - - public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext) { - Preconditions.checkArgument( - scanContext.snapshotId() == null, "Cannot set snapshot-id option for streaming reader"); - Preconditions.checkArgument( - scanContext.asOfTimestamp() == null, - "Cannot set as-of-timestamp option for streaming reader"); - Preconditions.checkArgument( - scanContext.endSnapshotId() == null, - "Cannot set end-snapshot-id option for streaming reader"); - Preconditions.checkArgument( - scanContext.endTag() == null, "Cannot set end-tag option for streaming reader"); - Preconditions.checkArgument( - scanContext.maxPlanningSnapshotCount() > 0, - "The max-planning-snapshot-count must be greater than zero"); - this.tableLoader = tableLoader; - this.scanContext = scanContext; - } - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - - final RuntimeContext runtimeContext = getRuntimeContext(); - ValidationException.check( - runtimeContext instanceof StreamingRuntimeContext, - "context should be instance of StreamingRuntimeContext"); - final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); - this.workerPool = - ThreadPools.newWorkerPool( - "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - // Load iceberg table from table loader. - tableLoader.open(); - table = tableLoader.loadTable(); - - // Initialize the flink state for last snapshot id. - lastSnapshotIdState = - context - .getOperatorStateStore() - .getListState(new ListStateDescriptor<>("snapshot-id-state", LongSerializer.INSTANCE)); - - // Restore the last-snapshot-id from flink's state if possible. - if (context.isRestored()) { - LOG.info("Restoring state for the {}.", getClass().getSimpleName()); - lastSnapshotId = lastSnapshotIdState.get().iterator().next(); - } else if (scanContext.startTag() != null || scanContext.startSnapshotId() != null) { - Preconditions.checkArgument( - !(scanContext.startTag() != null && scanContext.startSnapshotId() != null), - "START_SNAPSHOT_ID and START_TAG cannot both be set."); - Preconditions.checkNotNull( - table.currentSnapshot(), "Don't have any available snapshot in table."); - - long startSnapshotId; - if (scanContext.startTag() != null) { - Preconditions.checkArgument( - table.snapshot(scanContext.startTag()) != null, - "Cannot find snapshot with tag %s in table.", - scanContext.startTag()); - startSnapshotId = table.snapshot(scanContext.startTag()).snapshotId(); - } else { - startSnapshotId = scanContext.startSnapshotId(); - } - - long currentSnapshotId = table.currentSnapshot().snapshotId(); - Preconditions.checkState( - SnapshotUtil.isAncestorOf(table, currentSnapshotId, startSnapshotId), - "The option start-snapshot-id %s is not an ancestor of the current snapshot.", - startSnapshotId); - - lastSnapshotId = startSnapshotId; - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - lastSnapshotIdState.clear(); - lastSnapshotIdState.add(lastSnapshotId); - } - - @Override - public void run(SourceContext ctx) throws Exception { - this.sourceContext = ctx; - while (isRunning) { - monitorAndForwardSplits(); - Thread.sleep(scanContext.monitorInterval().toMillis()); - } - } - - private long toSnapshotIdInclusive( - long lastConsumedSnapshotId, long currentSnapshotId, int maxPlanningSnapshotCount) { - List snapshotIds = - SnapshotUtil.snapshotIdsBetween(table, lastConsumedSnapshotId, currentSnapshotId); - if (snapshotIds.size() <= maxPlanningSnapshotCount) { - return currentSnapshotId; - } else { - // It uses reverted index since snapshotIdsBetween returns Ids that are ordered by committed - // time descending. - return snapshotIds.get(snapshotIds.size() - maxPlanningSnapshotCount); - } - } - - @VisibleForTesting - void sourceContext(SourceContext ctx) { - this.sourceContext = ctx; - } - - @VisibleForTesting - void monitorAndForwardSplits() { - // Refresh the table to get the latest committed snapshot. - table.refresh(); - - Snapshot snapshot = - scanContext.branch() != null - ? table.snapshot(scanContext.branch()) - : table.currentSnapshot(); - if (snapshot != null && snapshot.snapshotId() != lastSnapshotId) { - long snapshotId = snapshot.snapshotId(); - - ScanContext newScanContext; - if (lastSnapshotId == INIT_LAST_SNAPSHOT_ID) { - newScanContext = scanContext.copyWithSnapshotId(snapshotId); - } else { - snapshotId = - toSnapshotIdInclusive( - lastSnapshotId, snapshotId, scanContext.maxPlanningSnapshotCount()); - newScanContext = scanContext.copyWithAppendsBetween(lastSnapshotId, snapshotId); - } - - LOG.debug( - "Start discovering splits from {} (exclusive) to {} (inclusive)", - lastSnapshotId, - snapshotId); - long start = System.currentTimeMillis(); - FlinkInputSplit[] splits = - FlinkSplitPlanner.planInputSplits(table, newScanContext, workerPool); - LOG.debug( - "Discovered {} splits, time elapsed {}ms", - splits.length, - System.currentTimeMillis() - start); - - // only need to hold the checkpoint lock when emitting the splits and updating lastSnapshotId - start = System.currentTimeMillis(); - synchronized (sourceContext.getCheckpointLock()) { - for (FlinkInputSplit split : splits) { - sourceContext.collect(split); - } - - lastSnapshotId = snapshotId; - } - LOG.debug( - "Forwarded {} splits, time elapsed {}ms", - splits.length, - System.currentTimeMillis() - start); - } - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (sourceContext != null) { - synchronized (sourceContext.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; - } - - // Release all the resources here. - if (tableLoader != null) { - try { - tableLoader.close(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - } - - @Override - public void close() { - cancel(); - - if (workerPool != null) { - workerPool.shutdown(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java deleted file mode 100644 index ee6f7b63988d..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java +++ /dev/null @@ -1,246 +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.source; - -import java.io.IOException; -import java.util.Queue; -import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.state.JavaSerializer; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.StreamSourceContexts; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The operator that reads the {@link FlinkInputSplit splits} received from the preceding {@link - * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} which has a - * parallelism of 1, this operator can have multiple parallelism. - * - *

    As soon as a split descriptor is received, it is put in a queue, and use {@link - * MailboxExecutor} read the actual data of the split. This architecture allows the separation of - * the reading thread from the one split processing the checkpoint barriers, thus removing any - * potential back-pressure. - */ -public class StreamingReaderOperator extends AbstractStreamOperator - implements OneInputStreamOperator { - - private static final Logger LOG = LoggerFactory.getLogger(StreamingReaderOperator.class); - - // It's the same thread that is running this operator and checkpoint actions. we use this executor - // to schedule only - // one split for future reading, so that a new checkpoint could be triggered without blocking long - // time for exhausting - // all scheduled splits. - private final MailboxExecutor executor; - private FlinkInputFormat format; - - private transient SourceFunction.SourceContext sourceContext; - - private transient ListState inputSplitsState; - private transient Queue splits; - - // Splits are read by the same thread that calls processElement. Each read task is submitted to - // that thread by adding - // them to the executor. This state is used to ensure that only one read task is in that queue at - // a time, so that read - // tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this - // is set to RUNNING. - // When there are no more files to read, this will be set to IDLE. - private transient SplitState currentSplitState; - - private StreamingReaderOperator( - FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { - this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); - this.processingTimeService = timeService; - this.executor = - Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - - // TODO Replace Java serialization with Avro approach to keep state compatibility. - // See issue: https://github.com/apache/iceberg/issues/1698 - inputSplitsState = - context - .getOperatorStateStore() - .getListState(new ListStateDescriptor<>("splits", new JavaSerializer<>())); - - // Initialize the current split state to IDLE. - currentSplitState = SplitState.IDLE; - - // Recover splits state from flink state backend if possible. - splits = Lists.newLinkedList(); - if (context.isRestored()) { - int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); - LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); - - for (FlinkInputSplit split : inputSplitsState.get()) { - splits.add(split); - } - } - - this.sourceContext = - StreamSourceContexts.getSourceContext( - getOperatorConfig().getTimeCharacteristic(), - getProcessingTimeService(), - new Object(), // no actual locking needed - output, - getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), - -1, - true); - - // Enqueue to process the recovered input splits. - enqueueProcessSplits(); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - - inputSplitsState.clear(); - inputSplitsState.addAll(Lists.newArrayList(splits)); - } - - @Override - public void processElement(StreamRecord element) { - splits.add(element.getValue()); - enqueueProcessSplits(); - } - - private void enqueueProcessSplits() { - if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) { - currentSplitState = SplitState.RUNNING; - executor.execute(this::processSplits, this.getClass().getSimpleName()); - } - } - - private void processSplits() throws IOException { - FlinkInputSplit split = splits.poll(); - if (split == null) { - currentSplitState = SplitState.IDLE; - return; - } - - format.open(split); - try { - RowData nextElement = null; - while (!format.reachedEnd()) { - nextElement = format.nextRecord(nextElement); - sourceContext.collect(nextElement); - } - } finally { - currentSplitState = SplitState.IDLE; - format.close(); - } - - // Re-schedule to process the next split. - enqueueProcessSplits(); - } - - @Override - public void processWatermark(Watermark mark) { - // we do nothing because we emit our own watermarks if needed. - } - - @Override - public void close() throws Exception { - super.close(); - - if (format != null) { - format.close(); - format.closeInputFormat(); - format = null; - } - - sourceContext = null; - } - - @Override - public void finish() throws Exception { - super.finish(); - output.close(); - if (sourceContext != null) { - sourceContext.emitWatermark(Watermark.MAX_WATERMARK); - sourceContext.close(); - sourceContext = null; - } - } - - static OneInputStreamOperatorFactory factory(FlinkInputFormat format) { - return new OperatorFactory(format); - } - - private enum SplitState { - IDLE, - RUNNING - } - - private static class OperatorFactory extends AbstractStreamOperatorFactory - implements YieldingOperatorFactory, - OneInputStreamOperatorFactory { - - private final FlinkInputFormat format; - - private transient MailboxExecutor mailboxExecutor; - - private OperatorFactory(FlinkInputFormat format) { - this.format = format; - } - - @Override - public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { - this.mailboxExecutor = mailboxExecutor; - } - - @SuppressWarnings("unchecked") - @Override - public > O createStreamOperator( - StreamOperatorParameters parameters) { - StreamingReaderOperator operator = - new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); - operator.setup( - parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); - return (O) operator; - } - - @Override - public Class getStreamOperatorClass(ClassLoader classLoader) { - return StreamingReaderOperator.class; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java deleted file mode 100644 index 11707bf82a0f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java +++ /dev/null @@ -1,54 +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.source; - -/** Starting strategy for streaming execution. */ -public enum StreamingStartingStrategy { - /** - * Do a regular table scan then switch to the incremental mode. - * - *

    The incremental mode starts from the current snapshot exclusive. - */ - TABLE_SCAN_THEN_INCREMENTAL, - - /** - * Start incremental mode from the latest snapshot inclusive. - * - *

    If it is an empty map, all future append snapshots should be discovered. - */ - INCREMENTAL_FROM_LATEST_SNAPSHOT, - - /** - * Start incremental mode from the earliest snapshot inclusive. - * - *

    If it is an empty map, all future append snapshots should be discovered. - */ - INCREMENTAL_FROM_EARLIEST_SNAPSHOT, - - /** Start incremental mode from a snapshot with a specific id inclusive. */ - INCREMENTAL_FROM_SNAPSHOT_ID, - - /** - * Start incremental mode from a snapshot with a specific timestamp inclusive. - * - *

    If the timestamp is between two snapshots, it should start from the snapshot after the - * timestamp. - */ - INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java deleted file mode 100644 index 37a0f1a6055f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java +++ /dev/null @@ -1,112 +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.source.assigner; - -import java.util.ArrayDeque; -import java.util.Collection; -import java.util.PriorityQueue; -import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.flink.source.split.SerializableComparator; - -/** - * Since all methods are called in the source coordinator thread by enumerator, there is no need for - * locking. - */ -@Internal -public class DefaultSplitAssigner implements SplitAssigner { - - private final Queue pendingSplits; - private CompletableFuture availableFuture; - - public DefaultSplitAssigner(SerializableComparator comparator) { - this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); - } - - public DefaultSplitAssigner( - SerializableComparator comparator, - Collection assignerState) { - this(comparator); - // Because default assigner only tracks unassigned splits, - // there is no need to filter splits based on status (unassigned) here. - assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); - } - - @Override - public synchronized GetSplitResult getNext(@Nullable String hostname) { - if (pendingSplits.isEmpty()) { - return GetSplitResult.unavailable(); - } else { - IcebergSourceSplit split = pendingSplits.poll(); - return GetSplitResult.forSplit(split); - } - } - - @Override - public void onDiscoveredSplits(Collection splits) { - addSplits(splits); - } - - @Override - public void onUnassignedSplits(Collection splits) { - addSplits(splits); - } - - private synchronized void addSplits(Collection splits) { - if (!splits.isEmpty()) { - pendingSplits.addAll(splits); - // only complete pending future if new splits are discovered - completeAvailableFuturesIfNeeded(); - } - } - - /** Simple assigner only tracks unassigned splits */ - @Override - public synchronized Collection state() { - return pendingSplits.stream() - .map(split -> new IcebergSourceSplitState(split, IcebergSourceSplitStatus.UNASSIGNED)) - .collect(Collectors.toList()); - } - - @Override - public synchronized CompletableFuture isAvailable() { - if (availableFuture == null) { - availableFuture = new CompletableFuture<>(); - } - return availableFuture; - } - - @Override - public synchronized int pendingSplitCount() { - return pendingSplits.size(); - } - - private synchronized void completeAvailableFuturesIfNeeded() { - if (availableFuture != null && !pendingSplits.isEmpty()) { - availableFuture.complete(null); - } - availableFuture = null; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java deleted file mode 100644 index 72deaeb890f3..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java +++ /dev/null @@ -1,77 +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.source.assigner; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -@Internal -public class GetSplitResult { - - public enum Status { - AVAILABLE, - - /** - * There are pending splits. But they can't be assigned due to constraints (like event time - * alignment) - */ - CONSTRAINED, - - /** Assigner doesn't have pending splits. */ - UNAVAILABLE - } - - private final Status status; - private final IcebergSourceSplit split; - - private GetSplitResult(Status status) { - this.status = status; - this.split = null; - } - - private GetSplitResult(IcebergSourceSplit split) { - Preconditions.checkNotNull(split, "Split cannot be null"); - this.status = Status.AVAILABLE; - this.split = split; - } - - public Status status() { - return status; - } - - public IcebergSourceSplit split() { - return split; - } - - private static final GetSplitResult UNAVAILABLE = new GetSplitResult(Status.UNAVAILABLE); - private static final GetSplitResult CONSTRAINED = new GetSplitResult(Status.CONSTRAINED); - - public static GetSplitResult unavailable() { - return UNAVAILABLE; - } - - public static GetSplitResult constrained() { - return CONSTRAINED; - } - - public static GetSplitResult forSplit(IcebergSourceSplit split) { - return new GetSplitResult(split); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java deleted file mode 100644 index e58478897aef..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java +++ /dev/null @@ -1,46 +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.source.assigner; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.SerializableComparator; - -/** - * Create default assigner with a comparator that hands out splits where the order of the splits - * will be defined by the {@link SerializableComparator}. - */ -public class OrderedSplitAssignerFactory implements SplitAssignerFactory { - private final SerializableComparator comparator; - - public OrderedSplitAssignerFactory(SerializableComparator comparator) { - this.comparator = comparator; - } - - @Override - public SplitAssigner createAssigner() { - return new DefaultSplitAssigner(comparator); - } - - @Override - public SplitAssigner createAssigner(Collection assignerState) { - return new DefaultSplitAssigner(comparator, assignerState); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java deleted file mode 100644 index a2e2ff364d46..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java +++ /dev/null @@ -1,37 +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.source.assigner; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** Create simple assigner that hands out splits without any guarantee in order or locality. */ -public class SimpleSplitAssignerFactory implements SplitAssignerFactory { - public SimpleSplitAssignerFactory() {} - - @Override - public SplitAssigner createAssigner() { - return new DefaultSplitAssigner(null); - } - - @Override - public SplitAssigner createAssigner(Collection assignerState) { - return new DefaultSplitAssigner(null, assignerState); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java deleted file mode 100644 index ca60612f0ec9..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java +++ /dev/null @@ -1,118 +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.source.assigner; - -import java.io.Closeable; -import java.util.Collection; -import java.util.concurrent.CompletableFuture; -import javax.annotation.Nullable; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** - * SplitAssigner interface is extracted out as a separate component so that we can plug in different - * split assignment strategy for different requirements. E.g. - * - *

      - *
    • Simple assigner with no ordering guarantee or locality aware optimization. - *
    • Locality aware assigner that prefer splits that are local. - *
    • Snapshot aware assigner that assign splits based on the order they are committed. - *
    • Event time alignment assigner that assign splits satisfying certain time ordering within a - * single source or across sources. - *
    - * - *

    Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from - * the coordinator thread. But enumerator may call the {@link SplitAssigner#pendingSplitCount()} - * from the I/O threads. - */ -public interface SplitAssigner extends Closeable { - - /** - * Some assigners may need to start background threads or perform other activity such as - * registering as listeners to updates from other event sources e.g., watermark tracker. - */ - default void start() {} - - /** - * Some assigners may need to perform certain actions when their corresponding enumerators are - * closed - */ - @Override - default void close() {} - - /** - * Request a new split from the assigner when enumerator trying to assign splits to awaiting - * readers. - * - *

    If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should - * call {@link SplitAssigner#onUnassignedSplits} to return the split. - */ - GetSplitResult getNext(@Nullable String hostname); - - /** Add new splits discovered by enumerator */ - void onDiscoveredSplits(Collection splits); - - /** Forward addSplitsBack event (for failed reader) to assigner */ - void onUnassignedSplits(Collection splits); - - /** - * Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon - * completed splits - */ - default void onCompletedSplits(Collection completedSplitIds) {} - - /** - * Get assigner state for checkpointing. This is a super-set API that works for all currently - * imagined assigners. - */ - Collection state(); - - /** - * Enumerator can get a notification via CompletableFuture when the assigner has more splits - * available later. Enumerator should schedule assignment in the thenAccept action of the future. - * - *

    Assigner will return the same future if this method is called again before the previous - * future is completed. - * - *

    The future can be completed from other thread, e.g. the coordinator thread from another - * thread for event time alignment. - * - *

    If enumerator need to trigger action upon the future completion, it may want to run it in - * the coordinator thread using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}. - */ - CompletableFuture isAvailable(); - - /** - * Return the number of pending splits that haven't been assigned yet. - * - *

    The enumerator can poll this API to publish a metric on the number of pending splits. - * - *

    The enumerator can also use this information to throttle split discovery for streaming read. - * If there are already many pending splits tracked by the assigner, it is undesirable to discover - * more splits and track them in the assigner. That will increase the memory footprint and - * enumerator checkpoint size. - * - *

    Throttling works better together with {@link ScanContext#maxPlanningSnapshotCount()}. - * Otherwise, the next split discovery after throttling will just discover all non-enumerated - * snapshots and splits, which defeats the purpose of throttling. - */ - int pendingSplitCount(); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java deleted file mode 100644 index 6e02a556ffcd..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java +++ /dev/null @@ -1,30 +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.source.assigner; - -import java.io.Serializable; -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -public interface SplitAssignerFactory extends Serializable { - - SplitAssigner createAssigner(); - - SplitAssigner createAssigner(Collection assignerState); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java deleted file mode 100644 index 03ba67a554f9..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java +++ /dev/null @@ -1,33 +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.source.assigner; - -import org.apache.flink.annotation.Internal; - -@Internal -public enum SplitAssignerType { - SIMPLE { - @Override - public SplitAssignerFactory factory() { - return new SimpleSplitAssignerFactory(); - } - }; - - public abstract SplitAssignerFactory factory(); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java deleted file mode 100644 index 3aca390755ed..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ /dev/null @@ -1,173 +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.source.enumerator; - -import java.io.IOException; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.assigner.GetSplitResult; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * TODO: publish enumerator monitor metrics like number of pending metrics after FLINK-21000 is - * resolved - */ -abstract class AbstractIcebergEnumerator - implements SplitEnumerator { - private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); - - private final SplitEnumeratorContext enumeratorContext; - private final SplitAssigner assigner; - private final Map readersAwaitingSplit; - private final AtomicReference> availableFuture; - - AbstractIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { - this.enumeratorContext = enumeratorContext; - this.assigner = assigner; - this.readersAwaitingSplit = new LinkedHashMap<>(); - this.availableFuture = new AtomicReference<>(); - } - - @Override - public void start() { - assigner.start(); - } - - @Override - public void close() throws IOException { - assigner.close(); - } - - @Override - public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { - // Iceberg source uses custom split request event to piggyback finished split ids. - throw new UnsupportedOperationException( - String.format( - "Received invalid default split request event " - + "from subtask %d as Iceberg source uses custom split request event", - subtaskId)); - } - - @Override - public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { - if (sourceEvent instanceof SplitRequestEvent) { - SplitRequestEvent splitRequestEvent = (SplitRequestEvent) sourceEvent; - LOG.info("Received request split event from subtask {}", subtaskId); - assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds()); - readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname()); - assignSplits(); - } else { - throw new IllegalArgumentException( - String.format( - "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); - } - } - - @Override - public void addSplitsBack(List splits, int subtaskId) { - LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); - assigner.onUnassignedSplits(splits); - assignSplits(); - } - - @Override - public void addReader(int subtaskId) { - LOG.info("Added reader: {}", subtaskId); - } - - private void assignSplits() { - LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size()); - Iterator> awaitingReader = - readersAwaitingSplit.entrySet().iterator(); - while (awaitingReader.hasNext()) { - Map.Entry nextAwaiting = awaitingReader.next(); - // if the reader that requested another split has failed in the meantime, remove - // it from the list of waiting readers - if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) { - awaitingReader.remove(); - continue; - } - - int awaitingSubtask = nextAwaiting.getKey(); - String hostname = nextAwaiting.getValue(); - GetSplitResult getResult = assigner.getNext(hostname); - if (getResult.status() == GetSplitResult.Status.AVAILABLE) { - LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split()); - enumeratorContext.assignSplit(getResult.split(), awaitingSubtask); - awaitingReader.remove(); - } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) { - getAvailableFutureIfNeeded(); - break; - } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) { - if (shouldWaitForMoreSplits()) { - getAvailableFutureIfNeeded(); - break; - } else { - LOG.info("No more splits available for subtask {}", awaitingSubtask); - enumeratorContext.signalNoMoreSplits(awaitingSubtask); - awaitingReader.remove(); - } - } else { - throw new IllegalArgumentException("Unsupported status: " + getResult.status()); - } - } - } - - /** return true if enumerator should wait for splits like in the continuous enumerator case */ - protected abstract boolean shouldWaitForMoreSplits(); - - private synchronized void getAvailableFutureIfNeeded() { - if (availableFuture.get() != null) { - return; - } - - CompletableFuture future = - assigner - .isAvailable() - .thenAccept( - ignore -> - // Must run assignSplits in coordinator thread - // because the future may be completed from other threads. - // E.g., in event time alignment assigner, - // watermark advancement from another source may - // cause the available future to be completed - enumeratorContext.runInCoordinatorThread( - () -> { - LOG.debug("Executing callback of assignSplits"); - availableFuture.set(null); - assignSplits(); - })); - availableFuture.set(future); - LOG.debug("Registered callback for future available splits"); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java deleted file mode 100644 index 41863ffee60b..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.enumerator; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class ContinuousEnumerationResult { - private final Collection splits; - private final IcebergEnumeratorPosition fromPosition; - private final IcebergEnumeratorPosition toPosition; - - /** - * @param splits should never be null. But it can be an empty collection - * @param fromPosition can be null - * @param toPosition should never be null. But it can have null snapshotId and snapshotTimestampMs - */ - ContinuousEnumerationResult( - Collection splits, - IcebergEnumeratorPosition fromPosition, - IcebergEnumeratorPosition toPosition) { - Preconditions.checkArgument(splits != null, "Invalid to splits collection: null"); - Preconditions.checkArgument(toPosition != null, "Invalid end position: null"); - this.splits = splits; - this.fromPosition = fromPosition; - this.toPosition = toPosition; - } - - public Collection splits() { - return splits; - } - - public IcebergEnumeratorPosition fromPosition() { - return fromPosition; - } - - public IcebergEnumeratorPosition toPosition() { - return toPosition; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java deleted file mode 100644 index 55451b105885..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ /dev/null @@ -1,178 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collections; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { - - private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); - /** - * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to - * control the total number of snapshots worth of splits tracked by assigner. - */ - private static final int ENUMERATION_SPLIT_COUNT_HISTORY_SIZE = 3; - - private final SplitEnumeratorContext enumeratorContext; - private final SplitAssigner assigner; - private final ScanContext scanContext; - private final ContinuousSplitPlanner splitPlanner; - - /** - * snapshotId for the last enumerated snapshot. next incremental enumeration should be based off - * this as the starting position. - */ - private final AtomicReference enumeratorPosition; - - /** Track enumeration result history for split discovery throttling. */ - private final EnumerationHistory enumerationHistory; - - /** Count the consecutive failures and throw exception if the max allowed failres are reached */ - private transient int consecutiveFailures = 0; - - public ContinuousIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, - SplitAssigner assigner, - ScanContext scanContext, - ContinuousSplitPlanner splitPlanner, - @Nullable IcebergEnumeratorState enumState) { - super(enumeratorContext, assigner); - - this.enumeratorContext = enumeratorContext; - this.assigner = assigner; - this.scanContext = scanContext; - this.splitPlanner = splitPlanner; - this.enumeratorPosition = new AtomicReference<>(); - this.enumerationHistory = new EnumerationHistory(ENUMERATION_SPLIT_COUNT_HISTORY_SIZE); - - if (enumState != null) { - this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); - this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); - } - } - - @Override - public void start() { - super.start(); - enumeratorContext.callAsync( - this::discoverSplits, - this::processDiscoveredSplits, - 0L, - scanContext.monitorInterval().toMillis()); - } - - @Override - public void close() throws IOException { - splitPlanner.close(); - super.close(); - } - - @Override - protected boolean shouldWaitForMoreSplits() { - return true; - } - - @Override - public IcebergEnumeratorState snapshotState(long checkpointId) { - return new IcebergEnumeratorState( - enumeratorPosition.get(), assigner.state(), enumerationHistory.snapshot()); - } - - /** This method is executed in an IO thread pool. */ - private ContinuousEnumerationResult discoverSplits() { - int pendingSplitCountFromAssigner = assigner.pendingSplitCount(); - if (enumerationHistory.shouldPauseSplitDiscovery(pendingSplitCountFromAssigner)) { - // If the assigner already has many pending splits, it is better to pause split discovery. - // Otherwise, eagerly discovering more splits will just increase assigner memory footprint - // and enumerator checkpoint state size. - LOG.info( - "Pause split discovery as the assigner already has too many pending splits: {}", - pendingSplitCountFromAssigner); - return new ContinuousEnumerationResult( - Collections.emptyList(), enumeratorPosition.get(), enumeratorPosition.get()); - } else { - return splitPlanner.planSplits(enumeratorPosition.get()); - } - } - - /** This method is executed in a single coordinator thread. */ - private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwable error) { - if (error == null) { - consecutiveFailures = 0; - if (!Objects.equals(result.fromPosition(), enumeratorPosition.get())) { - // Multiple discoverSplits() may be triggered with the same starting snapshot to the I/O - // thread pool. E.g., the splitDiscoveryInterval is very short (like 10 ms in some unit - // tests) or the thread pool is busy and multiple discovery actions are executed - // concurrently. Discovery result should only be accepted if the starting position - // matches the enumerator position (like compare-and-swap). - LOG.info( - "Skip {} discovered splits because the scan starting position doesn't match " - + "the current enumerator position: enumerator position = {}, scan starting position = {}", - result.splits().size(), - enumeratorPosition.get(), - result.fromPosition()); - } else { - // Sometimes, enumeration may yield no splits for a few reasons. - // - upstream paused or delayed streaming writes to the Iceberg table. - // - enumeration frequency is higher than the upstream write frequency. - if (!result.splits().isEmpty()) { - assigner.onDiscoveredSplits(result.splits()); - // EnumerationHistory makes throttling decision on split discovery - // based on the total number of splits discovered in the last a few cycles. - // Only update enumeration history when there are some discovered splits. - enumerationHistory.add(result.splits().size()); - LOG.info( - "Added {} splits discovered between ({}, {}] to the assigner", - result.splits().size(), - result.fromPosition(), - result.toPosition()); - } else { - LOG.info( - "No new splits discovered between ({}, {}]", - result.fromPosition(), - result.toPosition()); - } - // update the enumerator position even if there is no split discovered - // or the toPosition is empty (e.g. for empty table). - enumeratorPosition.set(result.toPosition()); - LOG.info("Update enumerator position to {}", result.toPosition()); - } - } else { - consecutiveFailures++; - if (scanContext.maxAllowedPlanningFailures() < 0 - || consecutiveFailures <= scanContext.maxAllowedPlanningFailures()) { - LOG.error("Failed to discover new splits", error); - } else { - throw new RuntimeException("Failed to discover new splits", error); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java deleted file mode 100644 index 2a1325178873..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java +++ /dev/null @@ -1,30 +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.source.enumerator; - -import java.io.Closeable; -import org.apache.flink.annotation.Internal; - -/** This interface is introduced so that we can plug in different split planner for unit test */ -@Internal -public interface ContinuousSplitPlanner extends Closeable { - - /** Discover the files appended between {@code lastPosition} and current table snapshot */ - ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java deleted file mode 100644 index e9e3c159b07b..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ /dev/null @@ -1,240 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.FlinkSplitPlanner; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { - private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); - - private final Table table; - private final ScanContext scanContext; - private final boolean isSharedPool; - private final ExecutorService workerPool; - private final TableLoader tableLoader; - - /** - * @param tableLoader A cloned tableLoader. - * @param threadName thread name prefix for worker pool to run the split planning. If null, a - * shared worker pool will be used. - */ - public ContinuousSplitPlannerImpl( - TableLoader tableLoader, ScanContext scanContext, String threadName) { - this.tableLoader = tableLoader.clone(); - this.tableLoader.open(); - this.table = this.tableLoader.loadTable(); - this.scanContext = scanContext; - this.isSharedPool = threadName == null; - this.workerPool = - isSharedPool - ? ThreadPools.getWorkerPool() - : ThreadPools.newWorkerPool( - "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); - } - - @Override - public void close() throws IOException { - if (!isSharedPool) { - workerPool.shutdown(); - } - tableLoader.close(); - } - - @Override - public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { - table.refresh(); - if (lastPosition != null) { - return discoverIncrementalSplits(lastPosition); - } else { - return discoverInitialSplits(); - } - } - - private Snapshot toSnapshotInclusive( - Long lastConsumedSnapshotId, Snapshot currentSnapshot, int maxPlanningSnapshotCount) { - // snapshots are in reverse order (latest snapshot first) - List snapshots = - Lists.newArrayList( - SnapshotUtil.ancestorsBetween( - table, currentSnapshot.snapshotId(), lastConsumedSnapshotId)); - if (snapshots.size() <= maxPlanningSnapshotCount) { - return currentSnapshot; - } else { - // Because snapshots are in reverse order of commit history, this index returns - // the max allowed number of snapshots from the lastConsumedSnapshotId. - return snapshots.get(snapshots.size() - maxPlanningSnapshotCount); - } - } - - private ContinuousEnumerationResult discoverIncrementalSplits( - IcebergEnumeratorPosition lastPosition) { - Snapshot currentSnapshot = - scanContext.branch() != null - ? table.snapshot(scanContext.branch()) - : table.currentSnapshot(); - - if (currentSnapshot == null) { - // empty table - Preconditions.checkArgument( - lastPosition.snapshotId() == null, - "Invalid last enumerated position for an empty table: not null"); - LOG.info("Skip incremental scan because table is empty"); - return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); - } else if (lastPosition.snapshotId() != null - && currentSnapshot.snapshotId() == lastPosition.snapshotId()) { - LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); - return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); - } else { - Long lastConsumedSnapshotId = lastPosition != null ? lastPosition.snapshotId() : null; - Snapshot toSnapshotInclusive = - toSnapshotInclusive( - lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); - IcebergEnumeratorPosition newPosition = - IcebergEnumeratorPosition.of( - toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis()); - ScanContext incrementalScan = - scanContext.copyWithAppendsBetween( - lastPosition.snapshotId(), toSnapshotInclusive.snapshotId()); - List splits = - FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); - LOG.info( - "Discovered {} splits from incremental scan: " - + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", - splits.size(), - lastPosition, - newPosition); - return new ContinuousEnumerationResult(splits, lastPosition, newPosition); - } - } - - /** - * Discovery initial set of splits based on {@link StreamingStartingStrategy}. - *

  • {@link ContinuousEnumerationResult#splits()} should contain initial splits discovered from - * table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. For all other - * strategies, splits collection should be empty. - *
  • {@link ContinuousEnumerationResult#toPosition()} points to the starting position for the - * next incremental split discovery with exclusive behavior. Meaning files committed by the - * snapshot from the position in {@code ContinuousEnumerationResult} won't be included in the - * next incremental scan. - */ - private ContinuousEnumerationResult discoverInitialSplits() { - Optional startSnapshotOptional = startSnapshot(table, scanContext); - if (!startSnapshotOptional.isPresent()) { - return new ContinuousEnumerationResult( - Collections.emptyList(), null, IcebergEnumeratorPosition.empty()); - } - - Snapshot startSnapshot = startSnapshotOptional.get(); - LOG.info( - "Get starting snapshot id {} based on strategy {}", - startSnapshot.snapshotId(), - scanContext.streamingStartingStrategy()); - List splits; - IcebergEnumeratorPosition toPosition; - if (scanContext.streamingStartingStrategy() - == StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) { - // do a batch table scan first - splits = - FlinkSplitPlanner.planIcebergSourceSplits( - table, scanContext.copyWithSnapshotId(startSnapshot.snapshotId()), workerPool); - LOG.info( - "Discovered {} splits from initial batch table scan with snapshot Id {}", - splits.size(), - startSnapshot.snapshotId()); - // For TABLE_SCAN_THEN_INCREMENTAL, incremental mode starts exclusive from the startSnapshot - toPosition = - IcebergEnumeratorPosition.of(startSnapshot.snapshotId(), startSnapshot.timestampMillis()); - } else { - // For all other modes, starting snapshot should be consumed inclusively. - // Use parentId to achieve the inclusive behavior. It is fine if parentId is null. - splits = Collections.emptyList(); - Long parentSnapshotId = startSnapshot.parentId(); - if (parentSnapshotId != null) { - Snapshot parentSnapshot = table.snapshot(parentSnapshotId); - Long parentSnapshotTimestampMs = - parentSnapshot != null ? parentSnapshot.timestampMillis() : null; - toPosition = IcebergEnumeratorPosition.of(parentSnapshotId, parentSnapshotTimestampMs); - } else { - toPosition = IcebergEnumeratorPosition.empty(); - } - - LOG.info( - "Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", - startSnapshot.snapshotId(), - startSnapshot.timestampMillis()); - } - - return new ContinuousEnumerationResult(splits, null, toPosition); - } - - /** - * Calculate the starting snapshot based on the {@link StreamingStartingStrategy} defined in - * {@code ScanContext}. - * - *

    If the {@link StreamingStartingStrategy} is not {@link - * StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}, the start snapshot should be consumed - * inclusively. - */ - @VisibleForTesting - static Optional startSnapshot(Table table, ScanContext scanContext) { - switch (scanContext.streamingStartingStrategy()) { - case TABLE_SCAN_THEN_INCREMENTAL: - case INCREMENTAL_FROM_LATEST_SNAPSHOT: - return Optional.ofNullable(table.currentSnapshot()); - case INCREMENTAL_FROM_EARLIEST_SNAPSHOT: - return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); - case INCREMENTAL_FROM_SNAPSHOT_ID: - Snapshot matchedSnapshotById = table.snapshot(scanContext.startSnapshotId()); - Preconditions.checkArgument( - matchedSnapshotById != null, - "Start snapshot id not found in history: " + scanContext.startSnapshotId()); - return Optional.of(matchedSnapshotById); - case INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: - Snapshot matchedSnapshotByTimestamp = - SnapshotUtil.oldestAncestorAfter(table, scanContext.startSnapshotTimestamp()); - Preconditions.checkArgument( - matchedSnapshotByTimestamp != null, - "Cannot find a snapshot after: " + scanContext.startSnapshotTimestamp()); - return Optional.of(matchedSnapshotByTimestamp); - default: - throw new IllegalArgumentException( - "Unknown starting strategy: " + scanContext.streamingStartingStrategy()); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java deleted file mode 100644 index ef21dad0199d..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java +++ /dev/null @@ -1,96 +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.source.enumerator; - -import java.util.Arrays; -import javax.annotation.concurrent.ThreadSafe; -import org.apache.flink.annotation.VisibleForTesting; - -/** - * This enumeration history is used for split discovery throttling. It tracks the discovered split - * count per every non-empty enumeration. - */ -@ThreadSafe -class EnumerationHistory { - - private final int[] history; - // int (2B) should be enough without overflow for enumeration history - private int count; - - EnumerationHistory(int maxHistorySize) { - this.history = new int[maxHistorySize]; - } - - synchronized void restore(int[] restoredHistory) { - int startingOffset = 0; - int restoreSize = restoredHistory.length; - - if (restoredHistory.length > history.length) { - // keep the newest history - startingOffset = restoredHistory.length - history.length; - // only restore the latest history up to maxHistorySize - restoreSize = history.length; - } - - System.arraycopy(restoredHistory, startingOffset, history, 0, restoreSize); - count = restoreSize; - } - - synchronized int[] snapshot() { - int len = history.length; - if (count > len) { - int[] copy = new int[len]; - // this is like a circular buffer - int indexForOldest = count % len; - System.arraycopy(history, indexForOldest, copy, 0, len - indexForOldest); - System.arraycopy(history, 0, copy, len - indexForOldest, indexForOldest); - return copy; - } else { - return Arrays.copyOfRange(history, 0, count); - } - } - - /** Add the split count from the last enumeration result. */ - synchronized void add(int splitCount) { - int pos = count % history.length; - history[pos] = splitCount; - count += 1; - } - - @VisibleForTesting - synchronized boolean hasFullHistory() { - return count >= history.length; - } - - /** @return true if split discovery should pause because assigner has too many splits already. */ - synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { - if (count < history.length) { - // only check throttling when full history is obtained. - return false; - } else { - // if ScanContext#maxPlanningSnapshotCount() is 10, each split enumeration can - // discovery splits up to 10 snapshots. if maxHistorySize is 3, the max number of - // splits tracked in assigner shouldn't be more than 10 * (3 + 1) snapshots - // worth of splits. +1 because there could be another enumeration when the - // pending splits fall just below the 10 * 3. - int totalSplitCountFromRecentDiscovery = Arrays.stream(history).reduce(0, Integer::sum); - return pendingSplitCountFromAssigner >= totalSplitCountFromRecentDiscovery; - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java deleted file mode 100644 index 96aba296f8cf..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java +++ /dev/null @@ -1,79 +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.source.enumerator; - -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; - -class IcebergEnumeratorPosition { - private final Long snapshotId; - // Track snapshot timestamp mainly for info logging - private final Long snapshotTimestampMs; - - static IcebergEnumeratorPosition empty() { - return new IcebergEnumeratorPosition(null, null); - } - - static IcebergEnumeratorPosition of(long snapshotId, Long snapshotTimestampMs) { - return new IcebergEnumeratorPosition(snapshotId, snapshotTimestampMs); - } - - private IcebergEnumeratorPosition(Long snapshotId, Long snapshotTimestampMs) { - this.snapshotId = snapshotId; - this.snapshotTimestampMs = snapshotTimestampMs; - } - - boolean isEmpty() { - return snapshotId == null; - } - - Long snapshotId() { - return snapshotId; - } - - Long snapshotTimestampMs() { - return snapshotTimestampMs; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("snapshotId", snapshotId) - .add("snapshotTimestampMs", snapshotTimestampMs) - .toString(); - } - - @Override - public int hashCode() { - return Objects.hashCode(snapshotId, snapshotTimestampMs); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IcebergEnumeratorPosition other = (IcebergEnumeratorPosition) o; - return Objects.equal(snapshotId, other.snapshotId()) - && Objects.equal(snapshotTimestampMs, other.snapshotTimestampMs()); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java deleted file mode 100644 index 1c63807361c5..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java +++ /dev/null @@ -1,90 +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.source.enumerator; - -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -class IcebergEnumeratorPositionSerializer - implements SimpleVersionedSerializer { - - public static final IcebergEnumeratorPositionSerializer INSTANCE = - new IcebergEnumeratorPositionSerializer(); - - private static final int VERSION = 1; - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(128)); - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergEnumeratorPosition position) throws IOException { - return serializeV1(position); - } - - @Override - public IcebergEnumeratorPosition deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return deserializeV1(serialized); - default: - throw new IOException("Unknown version: " + version); - } - } - - private byte[] serializeV1(IcebergEnumeratorPosition position) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - out.writeBoolean(position.snapshotId() != null); - if (position.snapshotId() != null) { - out.writeLong(position.snapshotId()); - } - out.writeBoolean(position.snapshotTimestampMs() != null); - if (position.snapshotTimestampMs() != null) { - out.writeLong(position.snapshotTimestampMs()); - } - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - private IcebergEnumeratorPosition deserializeV1(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - Long snapshotId = null; - if (in.readBoolean()) { - snapshotId = in.readLong(); - } - - Long snapshotTimestampMs = null; - if (in.readBoolean()) { - snapshotTimestampMs = in.readLong(); - } - - if (snapshotId != null) { - return IcebergEnumeratorPosition.of(snapshotId, snapshotTimestampMs); - } else { - return IcebergEnumeratorPosition.empty(); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java deleted file mode 100644 index 024d0b101165..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java +++ /dev/null @@ -1,65 +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.source.enumerator; - -import java.io.Serializable; -import java.util.Collection; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** Enumerator state for checkpointing */ -@Internal -public class IcebergEnumeratorState implements Serializable { - @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; - private final Collection pendingSplits; - private int[] enumerationSplitCountHistory; - - public IcebergEnumeratorState(Collection pendingSplits) { - this(null, pendingSplits); - } - - public IcebergEnumeratorState( - @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, - Collection pendingSplits) { - this(lastEnumeratedPosition, pendingSplits, new int[0]); - } - - public IcebergEnumeratorState( - @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, - Collection pendingSplits, - int[] enumerationSplitCountHistory) { - this.lastEnumeratedPosition = lastEnumeratedPosition; - this.pendingSplits = pendingSplits; - this.enumerationSplitCountHistory = enumerationSplitCountHistory; - } - - @Nullable - public IcebergEnumeratorPosition lastEnumeratedPosition() { - return lastEnumeratedPosition; - } - - public Collection pendingSplits() { - return pendingSplits; - } - - public int[] enumerationSplitCountHistory() { - return enumerationSplitCountHistory; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java deleted file mode 100644 index 95d6db2cfbc4..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ /dev/null @@ -1,196 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collection; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergEnumeratorStateSerializer - implements SimpleVersionedSerializer { - - private static final int VERSION = 2; - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); - - private final IcebergEnumeratorPositionSerializer positionSerializer = - IcebergEnumeratorPositionSerializer.INSTANCE; - private final IcebergSourceSplitSerializer splitSerializer; - - public IcebergEnumeratorStateSerializer(boolean caseSensitive) { - this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); - } - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergEnumeratorState enumState) throws IOException { - return serializeV2(enumState); - } - - @Override - public IcebergEnumeratorState deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return deserializeV1(serialized); - case 2: - return deserializeV2(serialized); - default: - throw new IOException("Unknown version: " + version); - } - } - - @VisibleForTesting - byte[] serializeV1(IcebergEnumeratorState enumState) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); - serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - @VisibleForTesting - IcebergEnumeratorState deserializeV1(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - IcebergEnumeratorPosition enumeratorPosition = - deserializeEnumeratorPosition(in, positionSerializer); - Collection pendingSplits = - deserializePendingSplits(in, splitSerializer); - return new IcebergEnumeratorState(enumeratorPosition, pendingSplits); - } - - @VisibleForTesting - byte[] serializeV2(IcebergEnumeratorState enumState) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); - serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); - serializeEnumerationSplitCountHistory(out, enumState.enumerationSplitCountHistory()); - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - @VisibleForTesting - IcebergEnumeratorState deserializeV2(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - IcebergEnumeratorPosition enumeratorPosition = - deserializeEnumeratorPosition(in, positionSerializer); - Collection pendingSplits = - deserializePendingSplits(in, splitSerializer); - int[] enumerationSplitCountHistory = deserializeEnumerationSplitCountHistory(in); - return new IcebergEnumeratorState( - enumeratorPosition, pendingSplits, enumerationSplitCountHistory); - } - - private static void serializeEnumeratorPosition( - DataOutputSerializer out, - IcebergEnumeratorPosition enumeratorPosition, - IcebergEnumeratorPositionSerializer positionSerializer) - throws IOException { - out.writeBoolean(enumeratorPosition != null); - if (enumeratorPosition != null) { - out.writeInt(positionSerializer.getVersion()); - byte[] positionBytes = positionSerializer.serialize(enumeratorPosition); - out.writeInt(positionBytes.length); - out.write(positionBytes); - } - } - - private static IcebergEnumeratorPosition deserializeEnumeratorPosition( - DataInputDeserializer in, IcebergEnumeratorPositionSerializer positionSerializer) - throws IOException { - IcebergEnumeratorPosition enumeratorPosition = null; - if (in.readBoolean()) { - int version = in.readInt(); - byte[] positionBytes = new byte[in.readInt()]; - in.read(positionBytes); - enumeratorPosition = positionSerializer.deserialize(version, positionBytes); - } - return enumeratorPosition; - } - - private static void serializePendingSplits( - DataOutputSerializer out, - Collection pendingSplits, - IcebergSourceSplitSerializer splitSerializer) - throws IOException { - out.writeInt(splitSerializer.getVersion()); - out.writeInt(pendingSplits.size()); - for (IcebergSourceSplitState splitState : pendingSplits) { - byte[] splitBytes = splitSerializer.serialize(splitState.split()); - out.writeInt(splitBytes.length); - out.write(splitBytes); - out.writeUTF(splitState.status().name()); - } - } - - private static Collection deserializePendingSplits( - DataInputDeserializer in, IcebergSourceSplitSerializer splitSerializer) throws IOException { - int splitSerializerVersion = in.readInt(); - int splitCount = in.readInt(); - Collection pendingSplits = Lists.newArrayListWithCapacity(splitCount); - for (int i = 0; i < splitCount; ++i) { - byte[] splitBytes = new byte[in.readInt()]; - in.read(splitBytes); - IcebergSourceSplit split = splitSerializer.deserialize(splitSerializerVersion, splitBytes); - String statusName = in.readUTF(); - pendingSplits.add( - new IcebergSourceSplitState(split, IcebergSourceSplitStatus.valueOf(statusName))); - } - return pendingSplits; - } - - private static void serializeEnumerationSplitCountHistory( - DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { - out.writeInt(enumerationSplitCountHistory.length); - if (enumerationSplitCountHistory.length > 0) { - for (int enumerationSplitCount : enumerationSplitCountHistory) { - out.writeInt(enumerationSplitCount); - } - } - } - - private static int[] deserializeEnumerationSplitCountHistory(DataInputDeserializer in) - throws IOException { - int historySize = in.readInt(); - int[] history = new int[historySize]; - if (historySize > 0) { - for (int i = 0; i < historySize; ++i) { - history[i] = in.readInt(); - } - } - - return history; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java deleted file mode 100644 index 4e55ea5d5fd6..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java +++ /dev/null @@ -1,51 +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.source.enumerator; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** One-time split enumeration at the start-up for batch execution */ -@Internal -public class StaticIcebergEnumerator extends AbstractIcebergEnumerator { - private final SplitAssigner assigner; - - public StaticIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { - super(enumeratorContext, assigner); - this.assigner = assigner; - } - - @Override - public void start() { - super.start(); - } - - @Override - protected boolean shouldWaitForMoreSplits() { - return false; - } - - @Override - public IcebergEnumeratorState snapshotState(long checkpointId) { - return new IcebergEnumeratorState(null, assigner.state(), new int[0]); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java deleted file mode 100644 index 7b94c364c976..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.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.flink.source.reader; - -import java.util.Collections; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.file.src.util.Pool; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * {@link RecordsWithSplitIds} is used to pass a batch of records from fetcher to source reader. - * Batching is to improve the efficiency for records handover. - * - *

    {@link RecordsWithSplitIds} interface can encapsulate batches from multiple splits. This is - * the case for Kafka source where fetchers can retrieve records from multiple Kafka partitions at - * the same time. - * - *

    For file-based sources like Iceberg, readers always read one split/file at a time. Hence, we - * will only have a batch of records for one split here. - * - *

    This class uses array to store a batch of records from the same file (with the same - * fileOffset). - */ -class ArrayBatchRecords implements RecordsWithSplitIds> { - @Nullable private String splitId; - @Nullable private final Pool.Recycler recycler; - @Nullable private final T[] records; - private final int numberOfRecords; - private final Set finishedSplits; - private final RecordAndPosition recordAndPosition; - - // point to current read position within the records array - private int position; - - private ArrayBatchRecords( - @Nullable String splitId, - @Nullable Pool.Recycler recycler, - @Nullable T[] records, - int numberOfRecords, - int fileOffset, - long startingRecordOffset, - Set finishedSplits) { - Preconditions.checkArgument(numberOfRecords >= 0, "numberOfRecords can't be negative"); - Preconditions.checkArgument(fileOffset >= 0, "fileOffset can't be negative"); - Preconditions.checkArgument(startingRecordOffset >= 0, "numberOfRecords can't be negative"); - - this.splitId = splitId; - this.recycler = recycler; - this.records = records; - this.numberOfRecords = numberOfRecords; - this.finishedSplits = - Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); - this.recordAndPosition = new RecordAndPosition<>(); - - recordAndPosition.set(null, fileOffset, startingRecordOffset); - this.position = 0; - } - - @Nullable - @Override - public String nextSplit() { - String nextSplit = this.splitId; - // set the splitId to null to indicate no more splits - // this class only contains record for one split - this.splitId = null; - return nextSplit; - } - - @Nullable - @Override - public RecordAndPosition nextRecordFromSplit() { - if (position < numberOfRecords) { - recordAndPosition.record(records[position]); - position++; - return recordAndPosition; - } else { - return null; - } - } - - /** - * This method is called when all records from this batch has been emitted. If recycler is set, it - * should be called to return the records array back to pool. - */ - @Override - public void recycle() { - if (recycler != null) { - recycler.recycle(records); - } - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - - @VisibleForTesting - T[] records() { - return records; - } - - @VisibleForTesting - int numberOfRecords() { - return numberOfRecords; - } - - /** - * Create a ArrayBatchRecords backed up an array with records from the same file - * - * @param splitId Iceberg source only read from one split a time. We never have multiple records - * from multiple splits. - * @param recycler Because {@link DataIterator} with {@link RowData} returns an iterator of reused - * RowData object, we need to clone RowData eagerly when constructing a batch of records. We - * can use object pool to reuse the RowData array object which can be expensive to create. - * This recycler can be provided to recycle the array object back to pool after read is - * exhausted. If the {@link DataIterator} returns an iterator of non-reused objects, we don't - * need to clone objects. It is cheap to just create the batch array. Hence, we don't need - * object pool and recycler can be set to null. - * @param records an array (maybe reused) holding a batch of records - * @param numberOfRecords actual number of records in the array - * @param fileOffset fileOffset for all records in this batch - * @param startingRecordOffset starting recordOffset - * @param record type - */ - public static ArrayBatchRecords forRecords( - String splitId, - Pool.Recycler recycler, - T[] records, - int numberOfRecords, - int fileOffset, - long startingRecordOffset) { - return new ArrayBatchRecords<>( - splitId, - recycler, - records, - numberOfRecords, - fileOffset, - startingRecordOffset, - Collections.emptySet()); - } - - /** - * Create ab ArrayBatchRecords with only finished split id - * - * @param splitId for the split that is just exhausted - */ - public static ArrayBatchRecords finishedSplit(String splitId) { - return new ArrayBatchRecords<>(null, null, null, 0, 0, 0, Collections.singleton(splitId)); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java deleted file mode 100644 index 306afd1811be..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java +++ /dev/null @@ -1,130 +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.source.reader; - -import java.io.IOException; -import java.util.NoSuchElementException; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SourceReaderOptions; -import org.apache.flink.connector.file.src.util.Pool; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** This implementation stores record batch in array from recyclable pool */ -class ArrayPoolDataIteratorBatcher implements DataIteratorBatcher { - private final int batchSize; - private final int handoverQueueSize; - private final RecordFactory recordFactory; - - private transient Pool pool; - - ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory recordFactory) { - this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); - this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY); - this.recordFactory = recordFactory; - } - - @Override - public CloseableIterator>> batch( - String splitId, DataIterator inputIterator) { - Preconditions.checkArgument(inputIterator != null, "Input data iterator can't be null"); - // lazily create pool as it is not serializable - if (pool == null) { - this.pool = createPoolOfBatches(handoverQueueSize); - } - return new ArrayPoolBatchIterator(splitId, inputIterator, pool); - } - - private Pool createPoolOfBatches(int numBatches) { - Pool poolOfBatches = new Pool<>(numBatches); - for (int batchId = 0; batchId < numBatches; batchId++) { - T[] batch = recordFactory.createBatch(batchSize); - poolOfBatches.add(batch); - } - - return poolOfBatches; - } - - private class ArrayPoolBatchIterator - implements CloseableIterator>> { - - private final String splitId; - private final DataIterator inputIterator; - private final Pool pool; - - ArrayPoolBatchIterator(String splitId, DataIterator inputIterator, Pool pool) { - this.splitId = splitId; - this.inputIterator = inputIterator; - this.pool = pool; - } - - @Override - public boolean hasNext() { - return inputIterator.hasNext(); - } - - @Override - public RecordsWithSplitIds> next() { - if (!inputIterator.hasNext()) { - throw new NoSuchElementException(); - } - - T[] batch = getCachedEntry(); - int recordCount = 0; - while (inputIterator.hasNext() && recordCount < batchSize) { - // The record produced by inputIterator can be reused like for the RowData case. - // inputIterator.next() can't be called again until the copy is made - // since the record is not consumed immediately. - T nextRecord = inputIterator.next(); - recordFactory.clone(nextRecord, batch, recordCount); - recordCount++; - if (!inputIterator.currentFileHasNext()) { - // break early so that records in the ArrayResultIterator - // have the same fileOffset. - break; - } - } - - return ArrayBatchRecords.forRecords( - splitId, - pool.recycler(), - batch, - recordCount, - inputIterator.fileOffset(), - inputIterator.recordOffset() - recordCount); - } - - @Override - public void close() throws IOException { - inputIterator.close(); - } - - private T[] getCachedEntry() { - try { - return pool.pollEntry(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting for array pool entry", e); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java deleted file mode 100644 index 66e59633fff2..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ /dev/null @@ -1,102 +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.source.reader; - -import java.util.List; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Read Iceberg rows as {@link GenericRecord}. */ -public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { - private final String tableName; - private final Schema readSchema; - private final FileIO io; - private final EncryptionManager encryption; - private final RowDataFileScanTaskReader rowDataReader; - - private transient RowDataToAvroGenericRecordConverter converter; - - /** - * Create a reader function without projection and name mapping. Column name is case-insensitive. - */ - public static AvroGenericRecordReaderFunction fromTable(Table table) { - return new AvroGenericRecordReaderFunction( - table.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - null); - } - - public AvroGenericRecordReaderFunction( - String tableName, - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters) { - super(new ListDataIteratorBatcher<>(config)); - this.tableName = tableName; - this.readSchema = readSchema(tableSchema, projectedSchema); - this.io = io; - this.encryption = encryption; - this.rowDataReader = - new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); - } - - @Override - protected DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( - new AvroGenericRecordFileScanTaskReader(rowDataReader, lazyConverter()), - split.task(), - io, - encryption); - } - - private RowDataToAvroGenericRecordConverter lazyConverter() { - if (converter == null) { - this.converter = RowDataToAvroGenericRecordConverter.fromIcebergSchema(tableName, readSchema); - } - return converter; - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java deleted file mode 100644 index 4bb6f0a98c4c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ /dev/null @@ -1,98 +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.source.reader; - -import java.io.Serializable; -import java.util.Comparator; -import java.util.concurrent.TimeUnit; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Type.TypeID; -import org.apache.iceberg.types.Types; - -/** - * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics - * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link - * WatermarkExtractorRecordEmitter} along with the actual records. - */ -@Internal -public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { - private final int eventTimeFieldId; - private final String eventTimeFieldName; - private final TimeUnit timeUnit; - - /** - * Creates the extractor. - * - * @param schema The schema of the Table - * @param eventTimeFieldName The column which should be used as an event time - * @param timeUnit Used for converting the long value to epoch milliseconds - */ - public ColumnStatsWatermarkExtractor( - Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { - Types.NestedField field = schema.findField(eventTimeFieldName); - TypeID typeID = field.type().typeId(); - Preconditions.checkArgument( - typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), - "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", - typeID); - this.eventTimeFieldId = field.fieldId(); - this.eventTimeFieldName = eventTimeFieldName; - // Use the timeUnit only for Long columns. - this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; - } - - @VisibleForTesting - ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { - this.eventTimeFieldId = eventTimeFieldId; - this.eventTimeFieldName = eventTimeFieldName; - this.timeUnit = TimeUnit.MICROSECONDS; - } - - /** - * Get the watermark for a split using column statistics. - * - * @param split The split - * @return The watermark - * @throws IllegalArgumentException if there is no statistics for the column - */ - @Override - public long extractWatermark(IcebergSourceSplit split) { - return split.task().files().stream() - .map( - scanTask -> { - Preconditions.checkArgument( - scanTask.file().lowerBounds() != null - && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, - "Missing statistics for column name = %s in file = %s", - eventTimeFieldName, - eventTimeFieldId, - scanTask.file()); - return timeUnit.toMillis( - Conversions.fromByteBuffer( - Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); - }) - .min(Comparator.comparingLong(l -> l)) - .get(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java deleted file mode 100644 index c376e359c600..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java +++ /dev/null @@ -1,36 +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.source.reader; - -import java.io.Serializable; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; - -/** - * Batcher converts iterator of T into iterator of batched {@code - * RecordsWithSplitIds>}, as FLIP-27's {@link SplitReader#fetch()} returns - * batched records. - */ -@FunctionalInterface -public interface DataIteratorBatcher extends Serializable { - CloseableIterator>> batch( - String splitId, DataIterator inputIterator); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java deleted file mode 100644 index bbf797ef4aa8..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.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.flink.source.reader; - -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; - -/** A {@link ReaderFunction} implementation that uses {@link DataIterator}. */ -public abstract class DataIteratorReaderFunction implements ReaderFunction { - private final DataIteratorBatcher batcher; - - public DataIteratorReaderFunction(DataIteratorBatcher batcher) { - this.batcher = batcher; - } - - protected abstract DataIterator createDataIterator(IcebergSourceSplit split); - - @Override - public CloseableIterator>> apply( - IcebergSourceSplit split) { - DataIterator inputIterator = createDataIterator(split); - inputIterator.seek(split.fileOffset(), split.recordOffset()); - return batcher.batch(split.splitId(), inputIterator); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java deleted file mode 100644 index f143b8d2df2e..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ /dev/null @@ -1,77 +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.source.reader; - -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergSourceReader - extends SingleThreadMultiplexSourceReaderBase< - RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { - - public IcebergSourceReader( - SerializableRecordEmitter emitter, - IcebergSourceReaderMetrics metrics, - ReaderFunction readerFunction, - SerializableComparator splitComparator, - SourceReaderContext context) { - super( - () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - emitter, - context.getConfiguration(), - context); - } - - @Override - public void start() { - // We request a split only if we did not get splits during the checkpoint restore. - // Otherwise, reader restarts will keep requesting more and more splits. - if (getNumberOfCurrentlyAssignedSplits() == 0) { - requestSplit(Collections.emptyList()); - } - } - - @Override - protected void onSplitFinished(Map finishedSplitIds) { - requestSplit(Lists.newArrayList(finishedSplitIds.keySet())); - } - - @Override - protected IcebergSourceSplit initializedState(IcebergSourceSplit split) { - return split; - } - - @Override - protected IcebergSourceSplit toSplitType(String splitId, IcebergSourceSplit splitState) { - return splitState; - } - - private void requestSplit(Collection finishedSplitIds) { - context.sendSourceEventToCoordinator(new SplitRequestEvent(finishedSplitIds)); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java deleted file mode 100644 index 2a3e1dd86b95..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java +++ /dev/null @@ -1,61 +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.source.reader; - -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.MetricGroup; - -public class IcebergSourceReaderMetrics { - private final Counter assignedSplits; - private final Counter assignedBytes; - private final Counter finishedSplits; - private final Counter finishedBytes; - private final Counter splitReaderFetchCalls; - - public IcebergSourceReaderMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup readerMetrics = - metrics.addGroup("IcebergSourceReader").addGroup("table", fullTableName); - - this.assignedSplits = readerMetrics.counter("assignedSplits"); - this.assignedBytes = readerMetrics.counter("assignedBytes"); - this.finishedSplits = readerMetrics.counter("finishedSplits"); - this.finishedBytes = readerMetrics.counter("finishedBytes"); - this.splitReaderFetchCalls = readerMetrics.counter("splitReaderFetchCalls"); - } - - public void incrementAssignedSplits(long count) { - assignedSplits.inc(count); - } - - public void incrementAssignedBytes(long count) { - assignedBytes.inc(count); - } - - public void incrementFinishedSplits(long count) { - finishedSplits.inc(count); - } - - public void incrementFinishedBytes(long count) { - finishedBytes.inc(count); - } - - public void incrementSplitReaderFetchCalls(long count) { - splitReaderFetchCalls.inc(count); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java deleted file mode 100644 index 7ee75d2ade01..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ /dev/null @@ -1,156 +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.source.reader; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Collections; -import java.util.List; -import java.util.Queue; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.RecordsBySplits; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Queues; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class IcebergSourceSplitReader implements SplitReader, IcebergSourceSplit> { - private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceSplitReader.class); - - private final IcebergSourceReaderMetrics metrics; - private final ReaderFunction openSplitFunction; - private final SerializableComparator splitComparator; - private final int indexOfSubtask; - private final Queue splits; - - private CloseableIterator>> currentReader; - private IcebergSourceSplit currentSplit; - private String currentSplitId; - - IcebergSourceSplitReader( - IcebergSourceReaderMetrics metrics, - ReaderFunction openSplitFunction, - SerializableComparator splitComparator, - SourceReaderContext context) { - this.metrics = metrics; - this.openSplitFunction = openSplitFunction; - this.splitComparator = splitComparator; - this.indexOfSubtask = context.getIndexOfSubtask(); - this.splits = Queues.newArrayDeque(); - } - - /** - * The method reads a batch of records from the assigned splits. If all the records from the - * current split are returned then it will emit a {@link ArrayBatchRecords#finishedSplit(String)} - * batch to signal this event. In the next fetch loop the reader will continue with the next split - * (if any). - * - * @return The fetched records - * @throws IOException If there is an error during reading - */ - @Override - public RecordsWithSplitIds> fetch() throws IOException { - metrics.incrementSplitReaderFetchCalls(1); - if (currentReader == null) { - IcebergSourceSplit nextSplit = splits.poll(); - if (nextSplit != null) { - currentSplit = nextSplit; - currentSplitId = nextSplit.splitId(); - currentReader = openSplitFunction.apply(currentSplit); - } else { - // return an empty result, which will lead to split fetch to be idle. - // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); - } - } - - if (currentReader.hasNext()) { - // Because Iterator#next() doesn't support checked exception, - // we need to wrap and unwrap the checked IOException with UncheckedIOException - try { - return currentReader.next(); - } catch (UncheckedIOException e) { - throw e.getCause(); - } - } else { - return finishSplit(); - } - } - - @Override - public void handleSplitsChanges(SplitsChange splitsChange) { - if (!(splitsChange instanceof SplitsAddition)) { - throw new UnsupportedOperationException( - String.format("Unsupported split change: %s", splitsChange.getClass())); - } - - if (splitComparator != null) { - List newSplits = Lists.newArrayList(splitsChange.splits()); - newSplits.sort(splitComparator); - LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); - splits.addAll(newSplits); - } else { - LOG.info("Add {} splits to reader", splitsChange.splits().size()); - splits.addAll(splitsChange.splits()); - } - metrics.incrementAssignedSplits(splitsChange.splits().size()); - metrics.incrementAssignedBytes(calculateBytes(splitsChange)); - } - - @Override - public void wakeUp() {} - - @Override - public void close() throws Exception { - currentSplitId = null; - if (currentReader != null) { - currentReader.close(); - } - } - - private long calculateBytes(IcebergSourceSplit split) { - return split.task().files().stream().map(FileScanTask::length).reduce(0L, Long::sum); - } - - private long calculateBytes(SplitsChange splitsChanges) { - return splitsChanges.splits().stream().map(this::calculateBytes).reduce(0L, Long::sum); - } - - private ArrayBatchRecords finishSplit() throws IOException { - if (currentReader != null) { - currentReader.close(); - currentReader = null; - } - - ArrayBatchRecords finishRecords = ArrayBatchRecords.finishedSplit(currentSplitId); - LOG.info("Split reader {} finished split: {}", indexOfSubtask, currentSplitId); - metrics.incrementFinishedSplits(1); - metrics.incrementFinishedBytes(calculateBytes(currentSplit)); - currentSplitId = null; - return finishRecords; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java deleted file mode 100644 index 1acb3df76102..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java +++ /dev/null @@ -1,85 +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.source.reader; - -import java.util.Collections; -import java.util.List; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class ListBatchRecords implements RecordsWithSplitIds> { - private String splitId; - private final List records; - private final Set finishedSplits; - private final RecordAndPosition recordAndPosition; - - // point to current read position within the records list - private int position; - - ListBatchRecords( - String splitId, - List records, - int fileOffset, - long startingRecordOffset, - Set finishedSplits) { - this.splitId = splitId; - this.records = records; - this.finishedSplits = - Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); - this.recordAndPosition = new RecordAndPosition<>(); - this.recordAndPosition.set(null, fileOffset, startingRecordOffset); - - this.position = 0; - } - - @Nullable - @Override - public String nextSplit() { - String nextSplit = this.splitId; - // set the splitId to null to indicate no more splits - // this class only contains record for one split - this.splitId = null; - return nextSplit; - } - - @Nullable - @Override - public RecordAndPosition nextRecordFromSplit() { - if (position < records.size()) { - recordAndPosition.record(records.get(position)); - position++; - return recordAndPosition; - } else { - return null; - } - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - - public static ListBatchRecords forRecords( - String splitId, List records, int fileOffset, long startingRecordOffset) { - return new ListBatchRecords<>( - splitId, records, fileOffset, startingRecordOffset, Collections.emptySet()); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java deleted file mode 100644 index 365416239d37..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java +++ /dev/null @@ -1,94 +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.source.reader; - -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -/** - * FlinkRecordReaderFunction essentially cloned objects already. So there is no need to use array - * pool to clone objects. Simply create a new ArrayList for each batch. - */ -class ListDataIteratorBatcher implements DataIteratorBatcher { - - private final int batchSize; - - ListDataIteratorBatcher(ReadableConfig config) { - this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); - } - - @Override - public CloseableIterator>> batch( - String splitId, DataIterator dataIterator) { - return new ListBatchIterator(splitId, dataIterator); - } - - private class ListBatchIterator - implements CloseableIterator>> { - - private final String splitId; - private final DataIterator inputIterator; - - ListBatchIterator(String splitId, DataIterator inputIterator) { - this.splitId = splitId; - this.inputIterator = inputIterator; - } - - @Override - public boolean hasNext() { - return inputIterator.hasNext(); - } - - @Override - public RecordsWithSplitIds> next() { - if (!inputIterator.hasNext()) { - throw new NoSuchElementException(); - } - - final List batch = Lists.newArrayListWithCapacity(batchSize); - int recordCount = 0; - while (inputIterator.hasNext() && recordCount < batchSize) { - T nextRecord = inputIterator.next(); - batch.add(nextRecord); - recordCount++; - if (!inputIterator.currentFileHasNext()) { - // break early so that records have the same fileOffset. - break; - } - } - - return ListBatchRecords.forRecords( - splitId, batch, inputIterator.fileOffset(), inputIterator.recordOffset() - recordCount); - } - - @Override - public void close() throws IOException { - if (inputIterator != null) { - inputIterator.close(); - } - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java deleted file mode 100644 index fb4466913b90..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java +++ /dev/null @@ -1,65 +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.source.reader; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.DataTaskReader; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Reading metadata tables (like snapshots, manifests, etc.) */ -@Internal -public class MetaDataReaderFunction extends DataIteratorReaderFunction { - private final Schema readSchema; - private final FileIO io; - private final EncryptionManager encryption; - - public MetaDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - FileIO io, - EncryptionManager encryption) { - super( - new ArrayPoolDataIteratorBatcher<>( - config, - new RowDataRecordFactory( - FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); - this.readSchema = readSchema(tableSchema, projectedSchema); - this.io = io; - this.encryption = encryption; - } - - @Override - public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>(new DataTaskReader(readSchema), split.task(), io, encryption); - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java deleted file mode 100644 index 1ea91f10b4e7..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java +++ /dev/null @@ -1,31 +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.source.reader; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; - -@FunctionalInterface -public interface ReaderFunction - extends Serializable, - Function< - IcebergSourceSplit, CloseableIterator>>> {} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java deleted file mode 100644 index 6ac92592b6aa..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.reader; - -import org.apache.flink.annotation.Internal; - -/** - * A record along with the reader position to be stored in the checkpoint. - * - *

    The position defines the point in the reader AFTER the record. Record processing and updating - * checkpointed state happens atomically. The position points to where the reader should resume - * after this record is processed. - * - *

    This mutable object is useful in cases where only one instance of a {@code RecordAndPosition} - * is needed at a time. Then the same instance of RecordAndPosition can be reused. - */ -@Internal -public class RecordAndPosition { - private T record; - private int fileOffset; - private long recordOffset; - - public RecordAndPosition(T record, int fileOffset, long recordOffset) { - this.record = record; - this.fileOffset = fileOffset; - this.recordOffset = recordOffset; - } - - public RecordAndPosition() {} - - // ------------------------------------------------------------------------ - - public T record() { - return record; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } - - /** Updates the record and position in this object. */ - public void set(T newRecord, int newFileOffset, long newRecordOffset) { - this.record = newRecord; - this.fileOffset = newFileOffset; - this.recordOffset = newRecordOffset; - } - - /** Sets the next record of a sequence. This increments the {@code recordOffset} by one. */ - public void record(T nextRecord) { - this.record = nextRecord; - this.recordOffset++; - } - - @Override - public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java deleted file mode 100644 index ef92e2e6b81f..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java +++ /dev/null @@ -1,34 +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.source.reader; - -import java.io.Serializable; - -/** - * In FLIP-27 source, SplitReader#fetch() returns a batch of records. Since DataIterator for RowData - * returns an iterator of reused RowData objects, RecordFactory is needed to (1) create object array - * that is recyclable via pool. (2) clone RowData element from DataIterator to the batch array. - */ -interface RecordFactory extends Serializable { - /** Create a batch of records */ - T[] createBatch(int batchSize); - - /** Clone record into the specified position of the batch array */ - void clone(T from, T[] batch, int position); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java deleted file mode 100644 index 5d0a00954e7a..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ /dev/null @@ -1,79 +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.source.reader; - -import java.util.List; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class RowDataReaderFunction extends DataIteratorReaderFunction { - private final Schema tableSchema; - private final Schema readSchema; - private final String nameMapping; - private final boolean caseSensitive; - private final FileIO io; - private final EncryptionManager encryption; - private final List filters; - - public RowDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters) { - super( - new ArrayPoolDataIteratorBatcher<>( - config, - new RowDataRecordFactory( - FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); - this.tableSchema = tableSchema; - this.readSchema = readSchema(tableSchema, projectedSchema); - this.nameMapping = nameMapping; - this.caseSensitive = caseSensitive; - this.io = io; - this.encryption = encryption; - this.filters = filters; - } - - @Override - public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( - new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), - split.task(), - io, - encryption); - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java deleted file mode 100644 index 1e265b2663ce..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java +++ /dev/null @@ -1,61 +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.source.reader; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.flink.data.RowDataUtil; - -class RowDataRecordFactory implements RecordFactory { - private final RowType rowType; - private final TypeSerializer[] fieldSerializers; - - RowDataRecordFactory(RowType rowType) { - this.rowType = rowType; - this.fieldSerializers = createFieldSerializers(rowType); - } - - static TypeSerializer[] createFieldSerializers(RowType rowType) { - return rowType.getChildren().stream() - .map(InternalSerializers::create) - .toArray(TypeSerializer[]::new); - } - - @Override - public RowData[] createBatch(int batchSize) { - RowData[] arr = new RowData[batchSize]; - for (int i = 0; i < batchSize; ++i) { - arr[i] = new GenericRowData(rowType.getFieldCount()); - } - return arr; - } - - @Override - public void clone(RowData from, RowData[] batch, int position) { - // Set the return value from RowDataUtil.clone back to the array. - // Clone method returns same clone target object (reused) if it is a GenericRowData. - // Clone method will allocate a new GenericRowData object - // if the target object is NOT a GenericRowData. - // So we should always set the clone return value back to the array. - batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java deleted file mode 100644 index a6e2c1dae243..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ /dev/null @@ -1,40 +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.source.reader; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -@Internal -@FunctionalInterface -public interface SerializableRecordEmitter - extends RecordEmitter, T, IcebergSourceSplit>, Serializable { - static SerializableRecordEmitter defaultEmitter() { - return (element, output, split) -> { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - }; - } - - static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { - return new WatermarkExtractorRecordEmitter<>(extractor); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java deleted file mode 100644 index d1c50ac8ca52..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java +++ /dev/null @@ -1,28 +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.source.reader; - -import java.io.Serializable; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** The interface used to extract watermarks from splits. */ -public interface SplitWatermarkExtractor extends Serializable { - /** Get the watermark for a split. */ - long extractWatermark(IcebergSourceSplit split); -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java deleted file mode 100644 index 02ef57d344b1..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ /dev/null @@ -1,67 +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.source.reader; - -import org.apache.flink.api.common.eventtime.Watermark; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Emitter which emits the watermarks, records and updates the split position. - * - *

    The Emitter emits watermarks at the beginning of every split provided by the {@link - * SplitWatermarkExtractor}. - */ -class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { - private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); - private final SplitWatermarkExtractor timeExtractor; - private String lastSplitId = null; - private long watermark; - - WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { - this.timeExtractor = timeExtractor; - } - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - if (!split.splitId().equals(lastSplitId)) { - long newWatermark = timeExtractor.extractWatermark(split); - if (newWatermark < watermark) { - LOG.info( - "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", - watermark, - newWatermark, - lastSplitId, - split.splitId()); - } else { - watermark = newWatermark; - output.emitWatermark(new Watermark(watermark)); - LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); - } - - lastSplitId = split.splitId(); - } - - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java deleted file mode 100644 index e4bfbf1452e2..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ /dev/null @@ -1,177 +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.source.split; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceSplit; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.util.InstantiationUtil; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.FileScanTaskParser; -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.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergSourceSplit implements SourceSplit, Serializable { - private static final long serialVersionUID = 1L; - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); - - private final CombinedScanTask task; - - private int fileOffset; - private long recordOffset; - - // The splits are frequently serialized into checkpoints. - // Caching the byte representation makes repeated serialization cheap. - @Nullable private transient byte[] serializedBytesCache; - - private IcebergSourceSplit(CombinedScanTask task, int fileOffset, long recordOffset) { - this.task = task; - this.fileOffset = fileOffset; - this.recordOffset = recordOffset; - } - - public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) { - return fromCombinedScanTask(combinedScanTask, 0, 0L); - } - - public static IcebergSourceSplit fromCombinedScanTask( - CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) { - return new IcebergSourceSplit(combinedScanTask, fileOffset, recordOffset); - } - - public CombinedScanTask task() { - return task; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } - - @Override - public String splitId() { - return MoreObjects.toStringHelper(this).add("files", toString(task.files())).toString(); - } - - public void updatePosition(int newFileOffset, long newRecordOffset) { - // invalidate the cache after position change - serializedBytesCache = null; - fileOffset = newFileOffset; - recordOffset = newRecordOffset; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("files", toString(task.files())) - .add("fileOffset", fileOffset) - .add("recordOffset", recordOffset) - .toString(); - } - - private String toString(Collection files) { - return Iterables.toString( - files.stream() - .map( - fileScanTask -> - MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) - .add("start", fileScanTask.start()) - .add("length", fileScanTask.length()) - .toString()) - .collect(Collectors.toList())); - } - - byte[] serializeV1() throws IOException { - if (serializedBytesCache == null) { - serializedBytesCache = InstantiationUtil.serializeObject(this); - } - - return serializedBytesCache; - } - - static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { - try { - return InstantiationUtil.deserializeObject( - serialized, IcebergSourceSplit.class.getClassLoader()); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Failed to deserialize the split.", e); - } - } - - byte[] serializeV2() throws IOException { - if (serializedBytesCache == null) { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - Collection fileScanTasks = task.tasks(); - Preconditions.checkArgument( - fileOffset >= 0 && fileOffset < fileScanTasks.size(), - "Invalid file offset: %s. Should be within the range of [0, %s)", - fileOffset, - fileScanTasks.size()); - - out.writeInt(fileOffset); - out.writeLong(recordOffset); - out.writeInt(fileScanTasks.size()); - - for (FileScanTask fileScanTask : fileScanTasks) { - String taskJson = FileScanTaskParser.toJson(fileScanTask); - out.writeUTF(taskJson); - } - - serializedBytesCache = out.getCopyOfBuffer(); - out.clear(); - } - - return serializedBytesCache; - } - - static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) - throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - int fileOffset = in.readInt(); - long recordOffset = in.readLong(); - int taskCount = in.readInt(); - - List tasks = Lists.newArrayListWithCapacity(taskCount); - for (int i = 0; i < taskCount; ++i) { - String taskJson = in.readUTF(); - FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); - tasks.add(task); - } - - CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); - return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java deleted file mode 100644 index 8c089819e731..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ /dev/null @@ -1,60 +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.source.split; - -import java.io.IOException; -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.io.SimpleVersionedSerializer; - -@Internal -public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 2; - - private final boolean caseSensitive; - - public IcebergSourceSplitSerializer(boolean caseSensitive) { - this.caseSensitive = caseSensitive; - } - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV2(); - } - - @Override - public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return IcebergSourceSplit.deserializeV1(serialized); - case 2: - return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); - default: - throw new IOException( - String.format( - "Failed to deserialize IcebergSourceSplit. " - + "Encountered unsupported version: %d. Supported version are [1]", - version)); - } - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java deleted file mode 100644 index d9061e049e00..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java +++ /dev/null @@ -1,37 +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.source.split; - -public class IcebergSourceSplitState { - private final IcebergSourceSplit split; - private final IcebergSourceSplitStatus status; - - public IcebergSourceSplitState(IcebergSourceSplit split, IcebergSourceSplitStatus status) { - this.split = split; - this.status = status; - } - - public IcebergSourceSplit split() { - return split; - } - - public IcebergSourceSplitStatus status() { - return status; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java deleted file mode 100644 index d4a84a165e1a..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java +++ /dev/null @@ -1,25 +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.source.split; - -public enum IcebergSourceSplitStatus { - UNASSIGNED, - ASSIGNED, - COMPLETED -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java deleted file mode 100644 index 319648ca275c..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java +++ /dev/null @@ -1,24 +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.source.split; - -import java.io.Serializable; -import java.util.Comparator; - -public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java deleted file mode 100644 index 56ee92014d12..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ /dev/null @@ -1,76 +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.source.split; - -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} - * which could be used for ordering splits. These are used by the {@link - * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link - * org.apache.iceberg.flink.source.reader.IcebergSourceReader} - */ -public class SplitComparators { - private SplitComparators() {} - - /** Comparator which orders the splits based on the file sequence number of the data files */ - 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"); - - Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); - Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); - - Preconditions.checkNotNull( - seq1, - "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", - o1); - Preconditions.checkNotNull( - seq2, - "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", - o2); - - int temp = Long.compare(seq1, seq2); - if (temp != 0) { - return temp; - } else { - return o1.splitId().compareTo(o2.splitId()); - } - }; - } - - /** Comparator which orders the splits based on watermark of the splits */ - public static SerializableComparator watermark( - SplitWatermarkExtractor watermarkExtractor) { - return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { - long watermark1 = watermarkExtractor.extractWatermark(o1); - long watermark2 = watermarkExtractor.extractWatermark(o2); - - int temp = Long.compare(watermark1, watermark2); - if (temp != 0) { - return temp; - } else { - return o1.splitId().compareTo(o2.splitId()); - } - }; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java deleted file mode 100644 index eabd757aa638..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java +++ /dev/null @@ -1,54 +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.source.split; - -import java.util.Collection; -import java.util.Collections; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceEvent; - -/** We can remove this class once FLINK-21364 is resolved. */ -@Internal -public class SplitRequestEvent implements SourceEvent { - private static final long serialVersionUID = 1L; - - private final Collection finishedSplitIds; - private final String requesterHostname; - - public SplitRequestEvent() { - this(Collections.emptyList()); - } - - public SplitRequestEvent(Collection finishedSplitIds) { - this(finishedSplitIds, null); - } - - public SplitRequestEvent(Collection finishedSplitIds, String requesterHostname) { - this.finishedSplitIds = finishedSplitIds; - this.requesterHostname = requesterHostname; - } - - public Collection finishedSplitIds() { - return finishedSplitIds; - } - - public String requesterHostname() { - return requesterHostname; - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java deleted file mode 100644 index 2c5c587f4ebf..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.util; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; - -/** - * This is a small util class that try to hide calls to Flink Internal or PublicEvolve interfaces as - * Flink can change those APIs during minor version release. - */ -public class FlinkCompatibilityUtil { - - private FlinkCompatibilityUtil() {} - - public static TypeInformation toTypeInfo(RowType rowType) { - return InternalTypeInfo.of(rowType); - } - - public static boolean isPhysicalColumn(TableColumn column) { - return column.isPhysical(); - } -} diff --git a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java deleted file mode 100644 index 353cee56bebb..000000000000 --- a/flink/v1.16/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java +++ /dev/null @@ -1,61 +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.util; - -import java.util.concurrent.atomic.AtomicReference; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; - -public class FlinkPackage { - - private static final AtomicReference VERSION = new AtomicReference<>(); - public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION"; - - private FlinkPackage() {} - - /** Returns Flink version string like x.y.z */ - public static String version() { - if (null == VERSION.get()) { - String detectedVersion = null; - try { - detectedVersion = versionFromJar(); - // use unknown version in case exact implementation version can't be found from the jar - // (this can happen if the DataStream class appears multiple times in the same classpath - // such as with shading) - detectedVersion = detectedVersion != null ? detectedVersion : FLINK_UNKNOWN_VERSION; - } catch (Exception e) { - detectedVersion = FLINK_UNKNOWN_VERSION; - } - VERSION.set(detectedVersion); - } - - return VERSION.get(); - } - - @VisibleForTesting - static String versionFromJar() { - // Choose {@link DataStream} class because it is one of the core Flink API - return DataStream.class.getPackage().getImplementationVersion(); - } - - @VisibleForTesting - static void setVersion(String version) { - VERSION.set(version); - } -} diff --git a/flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 29a9955a7e20..000000000000 --- a/flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.FlinkDynamicTableFactory diff --git a/flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 2b6bfa3cd579..000000000000 --- a/flink/v1.16/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.FlinkCatalogFactory diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java deleted file mode 100644 index 47319ec9bc60..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java +++ /dev/null @@ -1,90 +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; - -import org.junit.Test; - -public abstract class AvroGenericRecordConverterBase { - protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; - - @Test - public void testPrimitiveTypes() throws Exception { - testConverter(new DataGenerators.Primitives()); - } - - @Test - public void testStructOfPrimitive() throws Exception { - testConverter(new DataGenerators.StructOfPrimitive()); - } - - @Test - public void testStructOfArray() throws Exception { - testConverter(new DataGenerators.StructOfArray()); - } - - @Test - public void testStructOfMap() throws Exception { - testConverter(new DataGenerators.StructOfMap()); - } - - @Test - public void testStructOfStruct() throws Exception { - testConverter(new DataGenerators.StructOfStruct()); - } - - @Test - public void testArrayOfPrimitive() throws Exception { - testConverter(new DataGenerators.ArrayOfPrimitive()); - } - - @Test - public void testArrayOfArray() throws Exception { - testConverter(new DataGenerators.ArrayOfArray()); - } - - @Test - public void testArrayOfMap() throws Exception { - testConverter(new DataGenerators.ArrayOfMap()); - } - - @Test - public void testArrayOfStruct() throws Exception { - testConverter(new DataGenerators.ArrayOfStruct()); - } - - @Test - public void testMapOfPrimitives() throws Exception { - testConverter(new DataGenerators.MapOfPrimitives()); - } - - @Test - public void testMapOfArray() throws Exception { - testConverter(new DataGenerators.MapOfArray()); - } - - @Test - public void testMapOfMap() throws Exception { - testConverter(new DataGenerators.MapOfMap()); - } - - @Test - public void testMapOfStruct() throws Exception { - testConverter(new DataGenerators.MapOfStruct()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java deleted file mode 100644 index 91ed3c4adea3..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ /dev/null @@ -1,143 +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; - -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.flink.util.ArrayUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; -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.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; - -@ExtendWith(ParameterizedTestExtension.class) -public abstract class CatalogTestBase extends TestBase { - - protected static final String DATABASE = "db"; - @TempDir protected File hiveWarehouse; - @TempDir protected File hadoopWarehouse; - - @Parameter(index = 0) - protected String catalogName; - - @Parameter(index = 1) - protected Namespace baseNamespace; - - protected Catalog validationCatalog; - protected SupportsNamespaces validationNamespaceCatalog; - protected Map config = Maps.newHashMap(); - - protected String flinkDatabase; - protected Namespace icebergNamespace; - protected boolean isHadoopCatalog; - - @Parameters(name = "catalogName={0}, baseNamespace={1}") - protected static List parameters() { - return Arrays.asList( - new Object[] {"testhive", Namespace.empty()}, - new Object[] {"testhadoop", Namespace.empty()}, - new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); - } - - @BeforeEach - public void before() { - this.isHadoopCatalog = catalogName.startsWith("testhadoop"); - this.validationCatalog = - isHadoopCatalog - ? new HadoopCatalog(hiveConf, "file:" + hadoopWarehouse.getPath()) - : catalog; - this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; - - config.put("type", "iceberg"); - if (!baseNamespace.isEmpty()) { - config.put(FlinkCatalogFactory.BASE_NAMESPACE, baseNamespace.toString()); - } - if (isHadoopCatalog) { - config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"); - } else { - config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - config.put(CatalogProperties.URI, getURI(hiveConf)); - } - config.put(CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouseRoot())); - - this.flinkDatabase = catalogName + "." + DATABASE; - this.icebergNamespace = - Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE})); - sql("CREATE CATALOG %s WITH %s", catalogName, toWithClause(config)); - } - - @AfterEach - public void clean() { - dropCatalog(catalogName, true); - } - - protected String warehouseRoot() { - if (isHadoopCatalog) { - return hadoopWarehouse.getAbsolutePath(); - } else { - return hiveWarehouse.getAbsolutePath(); - } - } - - protected String getFullQualifiedTableName(String tableName) { - final List levels = Lists.newArrayList(icebergNamespace.levels()); - levels.add(tableName); - return Joiner.on('.').join(levels); - } - - static String getURI(HiveConf conf) { - return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); - } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java deleted file mode 100644 index b1e3b20ff7ac..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericRecord; - -/** - * This interface defines test data generator. Different implementations for primitive and complex - * nested fields are defined in {@link DataGenerators}. - */ -public interface DataGenerator { - Schema icebergSchema(); - - RowType flinkRowType(); - - org.apache.avro.Schema avroSchema(); - - GenericRecord generateIcebergGenericRecord(); - - GenericRowData generateFlinkRowData(); - - org.apache.avro.generic.GenericRecord generateAvroGenericRecord(); -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java deleted file mode 100644 index e2cd411d7069..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java +++ /dev/null @@ -1,1172 +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; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import com.fasterxml.jackson.databind.node.IntNode; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import org.apache.avro.LogicalTypes; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Days; - -/** - * Util class to generate test data with extensive coverage different field types: from primitives - * to complex nested types. - */ -public class DataGenerators { - - public static class Primitives implements DataGenerator { - private static final DateTime JODA_DATETIME_EPOC = - new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC); - private static final DateTime JODA_DATETIME_20220110 = - new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); - private static final int DAYS_BTW_EPOC_AND_20220110 = - Days.daysBetween(JODA_DATETIME_EPOC, JODA_DATETIME_20220110).getDays(); - private static final int HOUR_8_IN_MILLI = (int) TimeUnit.HOURS.toMillis(8); - - private static final LocalDate JAVA_LOCAL_DATE_20220110 = LocalDate.of(2022, 1, 10); - private static final LocalTime JAVA_LOCAL_TIME_HOUR8 = LocalTime.of(8, 0); - private static final OffsetDateTime JAVA_OFFSET_DATE_TIME_20220110 = - OffsetDateTime.of(2022, 1, 10, 0, 0, 0, 0, ZoneOffset.UTC); - private static final LocalDateTime JAVA_LOCAL_DATE_TIME_20220110 = - LocalDateTime.of(2022, 1, 10, 0, 0, 0); - private static final BigDecimal BIG_DECIMAL_NEGATIVE = new BigDecimal("-1.50"); - private static final byte[] FIXED_BYTES = "012345689012345".getBytes(StandardCharsets.UTF_8); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - // primitive types - Types.NestedField.optional(2, "boolean_field", Types.BooleanType.get()), - Types.NestedField.optional(3, "int_field", Types.IntegerType.get()), - Types.NestedField.optional(4, "long_field", Types.LongType.get()), - Types.NestedField.optional(5, "float_field", Types.FloatType.get()), - Types.NestedField.optional(6, "double_field", Types.DoubleType.get()), - Types.NestedField.required(7, "string_field", Types.StringType.get()), - Types.NestedField.required(8, "date_field", Types.DateType.get()), - Types.NestedField.required(9, "time_field", Types.TimeType.get()), - Types.NestedField.required(10, "ts_with_zone_field", Types.TimestampType.withZone()), - Types.NestedField.required( - 11, "ts_without_zone_field", Types.TimestampType.withoutZone()), - Types.NestedField.required(12, "uuid_field", Types.UUIDType.get()), - Types.NestedField.required(13, "binary_field", Types.BinaryType.get()), - Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2)), - Types.NestedField.required(15, "fixed_field", Types.FixedType.ofLength(16))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - /** - * Fix up Avro Schema that is converted from Iceberg Schema. - * - * @param schemaConvertedFromIceberg Avro Schema converted from Iceberg schema via {@link - * AvroSchemaUtil#convert(Schema, String)} - */ - private org.apache.avro.Schema fixupAvroSchemaConvertedFromIcebergSchema( - org.apache.avro.Schema schemaConvertedFromIceberg) { - List fixedFields = - schemaConvertedFromIceberg.getFields().stream() - .map( - field -> { - org.apache.avro.Schema.Field updatedField = field; - if (field.name().equals("time_field")) { - // Iceberg's AvroSchemaUtil uses timestamp-micros with Long value for time - // field, while AvroToRowDataConverters#convertToTime() always looks for - // Integer value assuming millis. The root problem is that - // AvroToRowDataConverters#createConverter() uses LogicalTypeRoot to - // determine converter and LogicalTypeRoot lost the timestamp precision - // carried by LogicalType like Time(6). - org.apache.avro.Schema fieldSchema = - LogicalTypes.timeMillis() - .addToSchema( - org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); - updatedField = new org.apache.avro.Schema.Field("time_field", fieldSchema); - } - - return new org.apache.avro.Schema.Field(updatedField, updatedField.schema()); - }) - .collect(Collectors.toList()); - return org.apache.avro.Schema.createRecord( - schemaConvertedFromIceberg.getName(), - schemaConvertedFromIceberg.getDoc(), - schemaConvertedFromIceberg.getNamespace(), - schemaConvertedFromIceberg.isError(), - fixedFields); - } - - private final org.apache.avro.Schema avroSchema = - fixupAvroSchemaConvertedFromIcebergSchema(AvroSchemaUtil.convert(icebergSchema, "table")); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("boolean_field", false); - genericRecord.setField("int_field", Integer.MAX_VALUE); - genericRecord.setField("long_field", Long.MAX_VALUE); - genericRecord.setField("float_field", Float.MAX_VALUE); - genericRecord.setField("double_field", Double.MAX_VALUE); - genericRecord.setField("string_field", "str"); - - genericRecord.setField("date_field", JAVA_LOCAL_DATE_20220110); - genericRecord.setField("time_field", JAVA_LOCAL_TIME_HOUR8); - genericRecord.setField("ts_with_zone_field", JAVA_OFFSET_DATE_TIME_20220110); - genericRecord.setField("ts_without_zone_field", JAVA_LOCAL_DATE_TIME_20220110); - - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - - genericRecord.setField("uuid_field", UUID.nameUUIDFromBytes(uuidBytes)); - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - genericRecord.setField("binary_field", ByteBuffer.wrap(binaryBytes)); - - genericRecord.setField("decimal_field", BIG_DECIMAL_NEGATIVE); - genericRecord.setField("fixed_field", FIXED_BYTES); - - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - - return GenericRowData.of( - StringData.fromString("row_id_value"), - false, - Integer.MAX_VALUE, - Long.MAX_VALUE, - Float.MAX_VALUE, - Double.MAX_VALUE, - StringData.fromString("str"), - DAYS_BTW_EPOC_AND_20220110, - HOUR_8_IN_MILLI, - // Although Avro logical type for timestamp fields are in micro seconds, - // AvroToRowDataConverters only looks for long value in milliseconds. - TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), - TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), - uuidBytes, - binaryBytes, - DecimalData.fromBigDecimal(BIG_DECIMAL_NEGATIVE, 9, 2), - FIXED_BYTES); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", new Utf8("row_id_value")); - genericRecord.put("boolean_field", false); - genericRecord.put("int_field", Integer.MAX_VALUE); - genericRecord.put("long_field", Long.MAX_VALUE); - genericRecord.put("float_field", Float.MAX_VALUE); - genericRecord.put("double_field", Double.MAX_VALUE); - genericRecord.put("string_field", new Utf8("str")); - - genericRecord.put("date_field", DAYS_BTW_EPOC_AND_20220110); - genericRecord.put("time_field", HOUR_8_IN_MILLI); - // Although Avro logical type for timestamp fields are in micro seconds, - // AvroToRowDataConverters only looks for long value in milliseconds. - genericRecord.put("ts_with_zone_field", JODA_DATETIME_20220110.getMillis()); - genericRecord.put("ts_without_zone_field", JODA_DATETIME_20220110.getMillis()); - - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - genericRecord.put("uuid_field", ByteBuffer.wrap(uuidBytes)); - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - genericRecord.put("binary_field", ByteBuffer.wrap(binaryBytes)); - - BigDecimal bigDecimal = new BigDecimal("-1.50"); - // unscaledValue().toByteArray() is to match the behavior of RowDataToAvroConverters from - // Flink for decimal type - genericRecord.put("decimal_field", ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray())); - - genericRecord.put("fixed_field", ByteBuffer.wrap(FIXED_BYTES)); - - return genericRecord; - } - } - - public static class StructOfPrimitive implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_primitive", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required(102, "name", Types.StringType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_primitive").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("name", "Jane"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_primitive", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of(1, StringData.fromString("Jane"))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_primitive").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("name", "Jane"); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_primitive", struct); - return genericRecord; - } - } - - public static class StructOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_array", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, "names", Types.ListType.ofRequired(201, Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_array").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("names", Arrays.asList("Jane", "Joe")); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_array", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - StringData[] names = {StringData.fromString("Jane"), StringData.fromString("Joe")}; - return GenericRowData.of( - StringData.fromString("row_id_value"), GenericRowData.of(1, new GenericArrayData(names))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_array").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("names", Arrays.asList("Jane", "Joe")); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_array", struct); - return genericRecord; - } - } - - public static class StructOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_map", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, - "names", - Types.MapType.ofRequired( - 201, 202, Types.StringType.get(), Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_map").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("names", ImmutableMap.of("Jane", "female", "Joe", "male")); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_map", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of( - 1, - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), - StringData.fromString("female"), - StringData.fromString("Joe"), - StringData.fromString("male"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_map").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("names", ImmutableMap.of("Jane", new Utf8("female"), "Joe", new Utf8("male"))); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_map", struct); - return genericRecord; - } - } - - public static class StructOfStruct implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_struct", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, - "person_struct", - Types.StructType.of( - Types.NestedField.required(201, "name", Types.StringType.get()), - Types.NestedField.required(202, "address", Types.StringType.get())))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_struct").type().asStructType().fields()); - Schema personSchema = - new Schema(structSchema.findField("person_struct").type().asStructType().fields()); - GenericRecord person = GenericRecord.create(personSchema); - person.setField("name", "Jane"); - person.setField("address", "Apple Park"); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("person_struct", person); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_struct", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of( - 1, - GenericRowData.of( - StringData.fromString("Jane"), StringData.fromString("Apple Park")))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_struct").schema(); - org.apache.avro.Schema personSchema = structSchema.getField("person_struct").schema(); - org.apache.avro.generic.GenericRecord person = new GenericData.Record(personSchema); - person.put("name", "Jane"); - person.put("address", "Apple Park"); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("person_struct", person); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_struct", struct); - return genericRecord; - } - } - - public static class ArrayOfPrimitive implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, "array_of_int", Types.ListType.ofOptional(101, Types.IntegerType.get()))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("array_of_int", Arrays.asList(1, 2, 3)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - Integer[] arr = {1, 2, 3}; - return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(arr)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("array_of_int", Arrays.asList(1, 2, 3)); - return genericRecord; - } - } - - public static class ArrayOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "array_of_array", - Types.ListType.ofRequired( - 101, Types.ListType.ofRequired(201, Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - // non-primitive - Integer[] array1 = {1, 2, 3}; - Integer[] array2 = {4, 5, 6}; - GenericArrayData[] arrayOfArrays = { - new GenericArrayData(array1), new GenericArrayData(array2) - }; - return GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(arrayOfArrays)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - return genericRecord; - } - } - - public static class ArrayOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "array_of_map", - Types.ListType.ofRequired( - 101, - Types.MapType.ofRequired( - 201, 202, Types.StringType.get(), Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "array_of_map", - Arrays.asList( - ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - GenericMapData[] array = { - new GenericMapData( - ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2)), - new GenericMapData( - ImmutableMap.of(StringData.fromString("Alice"), 3, StringData.fromString("Bob"), 4)) - }; - return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(array)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "array_of_map", - Arrays.asList( - ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); - return genericRecord; - } - } - - public static class ArrayOfStruct implements DataGenerator { - private final Types.StructType structType = - Types.StructType.of( - required(201, "id", Types.IntegerType.get()), - required(202, "name", Types.StringType.get())); - private final Schema structIcebergSchema = new Schema(structType.fields()); - private final org.apache.avro.Schema structAvroSchema = - AvroSchemaUtil.convert(structIcebergSchema, "struct"); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, "array_of_struct", Types.ListType.ofRequired(101, structType))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord struct1 = GenericRecord.create(structIcebergSchema); - struct1.setField("id", 1); - struct1.setField("name", "Jane"); - GenericRecord struct2 = GenericRecord.create(structIcebergSchema); - struct2.setField("id", 2); - struct2.setField("name", "Joe"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("array_of_struct", Arrays.asList(struct1, struct2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - GenericRowData[] structArray = { - GenericRowData.of(1, StringData.fromString("Jane")), - GenericRowData.of(2, StringData.fromString("Joe")) - }; - return GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(structArray)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); - struct1.put("id", 1); - struct1.put("name", "Jane"); - org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); - struct2.put("id", 2); - struct2.put("name", "Joe"); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("array_of_struct", Arrays.asList(struct1, struct2)); - return genericRecord; - } - } - - public static class MapOfPrimitives implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, - "map_of_primitives", - Types.MapType.ofRequired( - 101, 102, Types.StringType.get(), Types.IntegerType.get()))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); - return genericRecord; - } - } - - public static class MapOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_array", - Types.MapType.ofRequired( - 101, - 102, - Types.StringType.get(), - Types.ListType.ofRequired(201, Types.IntegerType.get())))); - - private final RowType rowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return rowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_array", - ImmutableMap.of( - "Jane", Arrays.asList(1, 2, 3), - "Joe", Arrays.asList(4, 5, 6))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - Integer[] janeArray = {1, 2, 3}; - Integer[] joeArray = {4, 5, 6}; - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), - new GenericArrayData(janeArray), - StringData.fromString("Joe"), - new GenericArrayData(joeArray)))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "map_of_array", - ImmutableMap.of( - "Jane", Arrays.asList(1, 2, 3), - "Joe", Arrays.asList(4, 5, 6))); - return genericRecord; - } - } - - public static class MapOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_map", - Types.MapType.ofRequired( - 101, - 102, - Types.StringType.get(), - Types.MapType.ofRequired( - 301, 302, Types.StringType.get(), Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_map", - ImmutableMap.of( - "female", ImmutableMap.of("Jane", 1, "Alice", 2), - "male", ImmutableMap.of("Joe", 3, "Bob", 4))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("female"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), 1, StringData.fromString("Alice"), 2)), - StringData.fromString("male"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Joe"), 3, StringData.fromString("Bob"), 4))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "map_of_map", - ImmutableMap.of( - "female", ImmutableMap.of("Jane", 1, "Alice", 2), - "male", ImmutableMap.of("Joe", 3, "Bob", 4))); - return genericRecord; - } - } - - public static class MapOfStruct implements DataGenerator { - private org.apache.avro.Schema createAvroSchemaIdField() { - org.apache.avro.Schema schema = SchemaBuilder.builder().intType(); - // this is needed to match the converter generated schema props - schema.addProp("field-id", IntNode.valueOf(201)); - return schema; - } - - private org.apache.avro.Schema createAvroSchemaNameField() { - org.apache.avro.Schema schema = SchemaBuilder.builder().stringType(); - // this is needed to match the converter generated schema props - schema.addProp("field-id", IntNode.valueOf(202)); - return schema; - } - - private final Types.StructType structType = - Types.StructType.of( - required(201, "id", Types.IntegerType.get()), - required(202, "name", Types.StringType.get())); - private final Schema structIcebergSchema = new Schema(structType.fields()); - - private final org.apache.avro.Schema structAvroSchema = - SchemaBuilder.builder() - .record("struct") - .fields() - .name("id") - .type(createAvroSchemaIdField()) - .noDefault() - .name("name") - .type(createAvroSchemaNameField()) - .noDefault() - .endRecord(); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_struct", - Types.MapType.ofRequired(101, 102, Types.StringType.get(), structType))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - // Can't use AvroSchemaUtil.convert otherwise the nested schema will have generated name like - // `r102` not the specified name like `struct`. - org.apache.avro.Schema avroSchema = - SchemaBuilder.builder() - .record("table") - .fields() - .requiredString("row_id") - .name("map_of_struct") - .type(SchemaBuilder.builder().map().values(structAvroSchema)) - .noDefault() - .endRecord(); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord struct1 = GenericRecord.create(structIcebergSchema); - struct1.setField("id", 1); - struct1.setField("name", "Jane"); - GenericRecord struct2 = GenericRecord.create(structIcebergSchema); - struct2.setField("id", 2); - struct2.setField("name", "Joe"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("struct1"), - GenericRowData.of(1, StringData.fromString("Jane")), - StringData.fromString("struct2"), - GenericRowData.of(2, StringData.fromString("Joe"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); - struct1.put("id", 1); - struct1.put("name", new Utf8("Jane")); - org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); - struct2.put("id", 2); - struct2.put("name", new Utf8("Joe")); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", new Utf8("row_id_value")); - genericRecord.put("map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); - return genericRecord; - } - } - - public static class MapOfStructStruct implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get()), - Types.NestedField.optional(202, "keyData", Types.StringType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()), - Types.NestedField.optional(204, "valueData", Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - throw new UnsupportedOperationException( - "Not applicable as Avro Map only support string key type"); - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - GenericRowData.of(1L, StringData.fromString("key_data")), - GenericRowData.of(1L, StringData.fromString("value_data"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - throw new UnsupportedOperationException("Avro Map only support string key type"); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java deleted file mode 100644 index 8076e0ec76f8..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ /dev/null @@ -1,129 +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; - -import java.util.List; -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.test.util.MiniClusterWithClientResource; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; - -public abstract class FlinkTestBase extends TestBaseUtils { - - @ClassRule - public static MiniClusterWithClientResource miniClusterResource = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeClass - public static void startMetastore() { - FlinkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - FlinkTestBase.hiveConf = metastore.hiveConf(); - FlinkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterClass - public static void stopMetastore() throws Exception { - metastore.stop(); - FlinkTestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java deleted file mode 100644 index d8e1325254d9..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ /dev/null @@ -1,104 +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; - -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.UUID; -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.extension.AfterAllCallback; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeAllCallback; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -public class HadoopCatalogExtension - implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, AfterEachCallback { - protected final String database; - protected final String tableName; - - protected Path temporaryFolder; - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogExtension(String database, String tableName) { - this.database = database; - this.tableName = tableName; - } - - @Override - public void beforeAll(ExtensionContext context) throws Exception { - this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-"); - } - - @Override - public void afterAll(ExtensionContext context) throws Exception { - FileUtils.deleteDirectory(temporaryFolder.toFile()); - } - - @Override - public void beforeEach(ExtensionContext context) throws Exception { - Assertions.assertThat(temporaryFolder).exists().isDirectory(); - this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - public void afterEach(ExtensionContext context) throws Exception { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +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; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.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.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 45af9241b743..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,53 +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; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java deleted file mode 100644 index 9a73b80e077d..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ /dev/null @@ -1,53 +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; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.junit5.MiniClusterExtension; - -public class MiniFlinkClusterExtension { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniFlinkClusterExtension() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterExtension createWithClassloaderCheckDisabled() { - return new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java deleted file mode 100644 index e532fb62615c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.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; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -public class RowDataConverter { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); - - private RowDataConverter() {} - - public static RowData convert(Schema iSchema, Record record) { - return convert(iSchema.asStruct(), record); - } - - private static RowData convert(Types.StructType struct, Record record) { - GenericRowData rowData = new GenericRowData(struct.fields().size()); - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Types.NestedField field = fields.get(i); - - Type fieldType = field.type(); - rowData.setField(i, convert(fieldType, record.get(i))); - } - return rowData; - } - - private static Object convert(Type type, Object object) { - if (object == null) { - return null; - } - - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - case FIXED: - return object; - case DATE: - return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); - case TIME: - // Iceberg's time is in microseconds, while flink's time is in milliseconds. - LocalTime localTime = (LocalTime) object; - return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); - } else { - return TimestampData.fromLocalDateTime((LocalDateTime) object); - } - case STRING: - return StringData.fromString((String) object); - case UUID: - UUID uuid = (UUID) object; - ByteBuffer bb = ByteBuffer.allocate(16); - bb.putLong(uuid.getMostSignificantBits()); - bb.putLong(uuid.getLeastSignificantBits()); - return bb.array(); - case BINARY: - ByteBuffer buffer = (ByteBuffer) object; - return Arrays.copyOfRange( - buffer.array(), - buffer.arrayOffset() + buffer.position(), - buffer.arrayOffset() + buffer.remaining()); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) type; - return DecimalData.fromBigDecimal( - (BigDecimal) object, decimalType.precision(), decimalType.scale()); - case STRUCT: - return convert(type.asStructType(), (Record) object); - case LIST: - List list = (List) object; - Object[] convertedArray = new Object[list.size()]; - for (int i = 0; i < convertedArray.length; i++) { - convertedArray[i] = convert(type.asListType().elementType(), list.get(i)); - } - return new GenericArrayData(convertedArray); - case MAP: - Map convertedMap = Maps.newLinkedHashMap(); - Map map = (Map) object; - for (Map.Entry entry : map.entrySet()) { - convertedMap.put( - convert(type.asMapType().keyType(), entry.getKey()), - convert(type.asMapType().valueType(), entry.getValue())); - } - return new GenericMapData(convertedMap); - default: - throw new UnsupportedOperationException("Not a supported type: " + type); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java deleted file mode 100644 index f48764f772b4..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ /dev/null @@ -1,443 +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; - -import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; - -import java.io.IOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -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.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.InternalRecordWrapper; -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.EncryptedOutputFile; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.hadoop.HadoopInputFile; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -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.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.iceberg.util.StructLikeWrapper; -import org.awaitility.Awaitility; -import org.junit.Assert; - -public class SimpleDataUtil { - - private SimpleDataUtil() {} - - public static final Schema SCHEMA = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())); - - public static final TableSchema FLINK_SCHEMA = - TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); - - public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); - - public static final Record RECORD = GenericRecord.create(SCHEMA); - - public static Table createTable( - String path, Map properties, boolean partitioned) { - PartitionSpec spec; - if (partitioned) { - spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); - } else { - spec = PartitionSpec.unpartitioned(); - } - return new HadoopTables().create(SCHEMA, spec, properties, path); - } - - public static Record createRecord(Integer id, String data) { - Record record = RECORD.copy(); - record.setField("id", id); - record.setField("data", data); - return record; - } - - public static RowData createRowData(Integer id, String data) { - return GenericRowData.of(id, StringData.fromString(data)); - } - - public static RowData createInsert(Integer id, String data) { - return GenericRowData.ofKind(RowKind.INSERT, id, StringData.fromString(data)); - } - - public static RowData createDelete(Integer id, String data) { - return GenericRowData.ofKind(RowKind.DELETE, id, StringData.fromString(data)); - } - - public static RowData createUpdateBefore(Integer id, String data) { - return GenericRowData.ofKind(RowKind.UPDATE_BEFORE, id, StringData.fromString(data)); - } - - public static RowData createUpdateAfter(Integer id, String data) { - return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); - } - - public static DataFile writeFile( - Table table, - Schema schema, - PartitionSpec spec, - Configuration conf, - String location, - String filename, - List rows) - throws IOException { - return writeFile(table, schema, spec, conf, location, filename, rows, null); - } - - /** Write the list of {@link RowData} to the given path and with the given partition data */ - public static DataFile writeFile( - Table table, - Schema schema, - PartitionSpec spec, - Configuration conf, - String location, - String filename, - List rows, - StructLike partition) - throws IOException { - Path path = new Path(location, filename); - FileFormat fileFormat = FileFormat.fromFileName(filename); - Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); - - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - FileAppenderFactory appenderFactory = - new FlinkAppenderFactory( - table, schema, flinkSchema, ImmutableMap.of(), spec, null, null, null); - - FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); - try (FileAppender closeableAppender = appender) { - closeableAppender.addAll(rows); - } - - DataFiles.Builder builder = - DataFiles.builder(spec) - .withInputFile(HadoopInputFile.fromPath(path, conf)) - .withMetrics(appender.metrics()); - - if (partition != null) { - builder = builder.withPartition(partition); - } - - return builder.build(); - } - - public static DeleteFile writeEqDeleteFile( - Table table, - FileFormat format, - String filename, - FileAppenderFactory appenderFactory, - List deletes) - throws IOException { - EncryptedOutputFile outputFile = - table - .encryption() - .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); - - EqualityDeleteWriter eqWriter = - appenderFactory.newEqDeleteWriter(outputFile, format, null); - try (EqualityDeleteWriter writer = eqWriter) { - writer.write(deletes); - } - return eqWriter.toDeleteFile(); - } - - public static DeleteFile writePosDeleteFile( - Table table, - FileFormat format, - String filename, - FileAppenderFactory appenderFactory, - List> positions) - throws IOException { - EncryptedOutputFile outputFile = - table - .encryption() - .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); - - PositionDeleteWriter posWriter = - appenderFactory.newPosDeleteWriter(outputFile, format, null); - PositionDelete posDelete = PositionDelete.create(); - try (PositionDeleteWriter writer = posWriter) { - for (Pair p : positions) { - writer.write(posDelete.set(p.first(), p.second(), null)); - } - } - return posWriter.toDeleteFile(); - } - - private static List convertToRecords(List rows) { - List records = Lists.newArrayList(); - for (RowData row : rows) { - Integer id = row.isNullAt(0) ? null : row.getInt(0); - String data = row.isNullAt(1) ? null : row.getString(1).toString(); - records.add(createRecord(id, data)); - } - return records; - } - - public static void assertTableRows(String tablePath, List expected, String branch) - throws IOException { - assertTableRecords(tablePath, convertToRecords(expected), branch); - } - - public static void assertTableRows(Table table, List expected) throws IOException { - assertTableRecords(table, convertToRecords(expected), SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRows(Table table, List expected, String branch) - throws IOException { - assertTableRecords(table, convertToRecords(expected), branch); - } - - /** Get all rows for a table */ - public static List tableRecords(Table table) throws IOException { - table.refresh(); - List records = Lists.newArrayList(); - try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { - for (Record record : iterable) { - records.add(record); - } - } - return records; - } - - public static boolean equalsRecords(List expected, List actual, Schema schema) { - if (expected.size() != actual.size()) { - return false; - } - Types.StructType type = schema.asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - StructLikeSet actualSet = StructLikeSet.create(type); - actualSet.addAll(actual); - return expectedSet.equals(actualSet); - } - - public static void assertRecordsEqual(List expected, List actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); - Types.StructType type = schema.asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - StructLikeSet actualSet = StructLikeSet.create(type); - actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); - } - - /** - * Assert table contains the expected list of records after waiting up to the configured {@code - * timeout} - */ - public static void assertTableRecords(Table table, List expected, Duration timeout) { - Awaitility.await("expected list of records should be produced") - .atMost(timeout) - .untilAsserted( - () -> { - equalsRecords(expected, tableRecords(table), table.schema()); - assertRecordsEqual(expected, tableRecords(table), table.schema()); - }); - } - - public static void assertTableRecords(Table table, List expected) throws IOException { - assertTableRecords(table, expected, SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRecords(Table table, List expected, String branch) - throws IOException { - table.refresh(); - Snapshot snapshot = latestSnapshot(table, branch); - - if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); - return; - } - - Types.StructType type = table.schema().asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - - try (CloseableIterable iterable = - IcebergGenerics.read(table).useSnapshot(snapshot.snapshotId()).build()) { - StructLikeSet actualSet = StructLikeSet.create(type); - - for (Record record : iterable) { - actualSet.add(record); - } - - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); - } - } - - // Returns the latest snapshot of the given branch in the table - public static Snapshot latestSnapshot(Table table, String branch) { - // For the main branch, currentSnapshot() is used to validate that the API behavior has - // not changed since that was the API used for validation prior to addition of branches. - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return table.currentSnapshot(); - } - - return table.snapshot(branch); - } - - public static void assertTableRecords(String tablePath, List expected) - throws IOException { - Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); - assertTableRecords(new HadoopTables().load(tablePath), expected, SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRecords(String tablePath, List expected, String branch) - throws IOException { - Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); - assertTableRecords(new HadoopTables().load(tablePath), expected, branch); - } - - public static StructLikeSet expectedRowSet(Table table, Record... records) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); - for (Record record : records) { - set.add(wrapper.copyFor(record)); - } - return set; - } - - public static StructLikeSet actualRowSet(Table table, String... columns) throws IOException { - return actualRowSet(table, null, columns); - } - - public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... columns) - throws IOException { - table.refresh(); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table) - .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) - .select(columns) - .build()) { - reader.forEach(record -> set.add(wrapper.copyFor(record))); - } - return set; - } - - public static List partitionDataFiles(Table table, Map partitionValues) - throws IOException { - table.refresh(); - Types.StructType partitionType = table.spec().partitionType(); - - Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); - StructLikeWrapper expectedWrapper = - StructLikeWrapper.forType(partitionType).set(partitionRecord); - - List dataFiles = Lists.newArrayList(); - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - StructLikeWrapper wrapper = - StructLikeWrapper.forType(partitionType).set(scanTask.file().partition()); - - if (expectedWrapper.equals(wrapper)) { - dataFiles.add(scanTask.file()); - } - } - } - - return dataFiles; - } - - public static Map> snapshotToDataFiles(Table table) throws IOException { - table.refresh(); - - Map> result = Maps.newHashMap(); - Snapshot current = table.currentSnapshot(); - while (current != null) { - TableScan tableScan = table.newScan(); - if (current.parentId() != null) { - // Collect the data files that was added only in current snapshot. - tableScan = tableScan.appendsBetween(current.parentId(), current.snapshotId()); - } else { - // Collect the data files that was added in the oldest snapshot. - tableScan = tableScan.useSnapshot(current.snapshotId()); - } - try (CloseableIterable scanTasks = tableScan.planFiles()) { - result.put( - current.snapshotId(), - ImmutableList.copyOf(Iterables.transform(scanTasks, FileScanTask::file))); - } - - // Continue to traverse the parent snapshot if exists. - if (current.parentId() == null) { - break; - } - // Iterate to the parent snapshot. - current = table.snapshot(current.parentId()); - } - return result; - } - - public static List matchingPartitions( - List dataFiles, PartitionSpec partitionSpec, Map partitionValues) { - Types.StructType partitionType = partitionSpec.partitionType(); - Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); - StructLikeWrapper expected = StructLikeWrapper.forType(partitionType).set(partitionRecord); - return dataFiles.stream() - .filter( - df -> { - StructLikeWrapper wrapper = - StructLikeWrapper.forType(partitionType).set(df.partition()); - return wrapper.equals(expected); - }) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java deleted file mode 100644 index 3986f1a796a5..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ /dev/null @@ -1,130 +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; - -import java.nio.file.Path; -import java.util.List; -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.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public abstract class TestBase extends TestBaseUtils { - - @RegisterExtension - public static MiniClusterExtension miniClusterResource = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @TempDir protected Path temporaryDirectory; - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeAll - public static void startMetastore() { - TestBase.metastore = new TestHiveMetastore(); - metastore.start(); - TestBase.hiveConf = metastore.hiveConf(); - TestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterAll - public static void stopMetastore() throws Exception { - metastore.stop(); - TestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java deleted file mode 100644 index 384ac5c52d00..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java +++ /dev/null @@ -1,118 +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; - -import static org.apache.iceberg.CatalogProperties.URI; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.Map; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -/** Test for {@link CatalogLoader}. */ -public class TestCatalogLoader extends FlinkTestBase { - - private static File warehouse = null; - private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); - private static final Schema SCHEMA = - new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - - @BeforeClass - public static void createWarehouse() throws IOException { - warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); - hiveConf.set("my_key", "my_value"); - } - - @AfterClass - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); - } - } - - @Test - public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException { - Map properties = Maps.newHashMap(); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse); - CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties); - validateCatalogLoader(loader); - } - - @Test - public void testHiveCatalogLoader() throws IOException, ClassNotFoundException { - CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - validateCatalogLoader(loader); - } - - @Test - public void testRESTCatalogLoader() { - Map properties = Maps.newHashMap(); - properties.put(URI, "http://localhost/"); - CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap()); - } - - private static void validateCatalogLoader(CatalogLoader loader) - throws IOException, ClassNotFoundException { - Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); - validateHadoopConf(table); - } - - private static void validateHadoopConf(Table table) { - FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); - HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); - } - - @SuppressWarnings("unchecked") - private static T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(object); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - return (T) in.readObject(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java deleted file mode 100644 index 147d2a173ddc..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ /dev/null @@ -1,116 +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; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -/** Test for {@link TableLoader}. */ -public class TestCatalogTableLoader extends TestBase { - - private static File warehouse = null; - private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); - private static final Schema SCHEMA = - new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - - @BeforeAll - public static void createWarehouse() throws IOException { - warehouse = File.createTempFile("warehouse", null); - Assertions.assertThat(warehouse.delete()).isTrue(); - hiveConf.set("my_key", "my_value"); - } - - @AfterAll - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assertions.assertThat(fs.delete(warehousePath, true)) - .as("Failed to delete " + warehousePath) - .isTrue(); - } - } - - @Test - public void testHadoopTableLoader() throws IOException, ClassNotFoundException { - String location = "file:" + warehouse + "/my_table"; - new HadoopTables(hiveConf).create(SCHEMA, location); - validateTableLoader(TableLoader.fromHadoopTable(location, hiveConf)); - } - - @Test - public void testHiveCatalogTableLoader() throws IOException, ClassNotFoundException { - CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - javaSerdes(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); - - CatalogLoader catalogLoader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - validateTableLoader(TableLoader.fromCatalog(catalogLoader, IDENTIFIER)); - } - - private static void validateTableLoader(TableLoader loader) - throws IOException, ClassNotFoundException { - TableLoader copied = javaSerdes(loader); - copied.open(); - try { - validateHadoopConf(copied.loadTable()); - } finally { - copied.close(); - } - } - - private static void validateHadoopConf(Table table) { - FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); - HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); - } - - @SuppressWarnings("unchecked") - private static T javaSerdes(T object) throws IOException, ClassNotFoundException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(object); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - return (T) in.readObject(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java deleted file mode 100644 index 8e9066e391c9..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ /dev/null @@ -1,305 +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; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import org.apache.flink.types.Row; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.flink.source.ChangeLogTableTestBase; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; -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.Sets; -import org.apache.iceberg.util.StructLikeSet; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -/** - * In this test case, we mainly cover the impact of primary key selection, multiple operations - * within a single transaction, and multiple operations between different txn on the correctness of - * the data. - */ -@RunWith(Parameterized.class) -public class TestChangeLogTable extends ChangeLogTableTestBase { - private static final Configuration CONF = new Configuration(); - private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private static String warehouse; - - private final boolean partitioned; - - @Parameterized.Parameters(name = "PartitionedTable={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {true}, new Object[] {false}); - } - - public TestChangeLogTable(boolean partitioned) { - this.partitioned = partitioned; - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - warehouse = String.format("file:%s", warehouseFile); - } - - @Before - public void before() { - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - // Set the table.exec.sink.upsert-materialize=NONE, so that downstream operators will receive - // the - // records with the same order as the source operator, bypassing Flink's inferred shuffle. - getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); - } - - @After - @Override - public void clean() { - sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - BoundedTableFactory.clearDataSets(); - } - - @Test - public void testSqlChangeLogOnIdKey() throws Exception { - List> inputRowsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(1, "bbb"), - insertRow(2, "aaa"), - deleteRow(2, "aaa"), - insertRow(2, "bbb")), - ImmutableList.of( - updateBeforeRow(2, "bbb"), - updateAfterRow(2, "ccc"), - deleteRow(2, "ccc"), - insertRow(2, "ddd")), - ImmutableList.of( - deleteRow(1, "bbb"), - insertRow(1, "ccc"), - deleteRow(1, "ccc"), - insertRow(1, "ddd"))); - - List> expectedRecordsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "bbb")), - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "ddd")), - ImmutableList.of(insertRow(1, "ddd"), insertRow(2, "ddd"))); - - testSqlChangeLog( - TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); - } - - @Test - public void testChangeLogOnDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(1, "bbb"), - insertRow(2, "aaa")), - ImmutableList.of( - updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), - ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "ccc"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa")), - ImmutableList.of(insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(1, "ccc"), - insertRow(2, "aaa"), - insertRow(2, "ccc"))); - - testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); - } - - @Test - public void testChangeLogOnIdDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(1, "bbb"), - insertRow(2, "aaa")), - ImmutableList.of( - updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), - ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(1, "ccc"), - insertRow(2, "aaa"), - insertRow(2, "bbb"))); - - testSqlChangeLog( - TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); - } - - @Test - public void testPureInsertOnIdKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), - ImmutableList.of(insertRow(3, "ccc"), insertRow(4, "ddd")), - ImmutableList.of(insertRow(5, "eee"), insertRow(6, "fff"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), insertRow(2, "bbb"), insertRow(3, "ccc"), insertRow(4, "ddd")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(3, "ccc"), - insertRow(4, "ddd"), - insertRow(5, "eee"), - insertRow(6, "fff"))); - - testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); - } - - private static Record record(int id, String data) { - return SimpleDataUtil.createRecord(id, data); - } - - private Table createTable(String tableName, List key, boolean isPartitioned) { - String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : ""; - sql( - "CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) %s", - tableName, Joiner.on(',').join(key), partitionByCause); - - // Upgrade the iceberg table to format v2. - CatalogLoader loader = - CatalogLoader.hadoop( - "my_catalog", CONF, ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - Table table = loader.loadCatalog().loadTable(TableIdentifier.of(DATABASE_NAME, TABLE_NAME)); - TableOperations ops = ((BaseTable) table).operations(); - TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); - - return table; - } - - private void testSqlChangeLog( - String tableName, - List key, - List> inputRowsPerCheckpoint, - List> expectedRecordsPerCheckpoint) - throws Exception { - String dataId = BoundedTableFactory.registerDataSet(inputRowsPerCheckpoint); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - Assert.assertEquals( - "Should have the expected rows", - listJoin(inputRowsPerCheckpoint), - sql("SELECT * FROM %s", SOURCE_TABLE)); - - Table table = createTable(tableName, key, partitioned); - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - table.refresh(); - List snapshots = findValidSnapshots(table); - int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); - - for (int i = 0; i < expectedSnapshotNum; i++) { - long snapshotId = snapshots.get(i).snapshotId(); - List expectedRows = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(table, expectedRows), - actualRowSet(table, snapshotId)); - } - - if (expectedSnapshotNum > 0) { - Assert.assertEquals( - "Should have the expected rows in the final table", - Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), - Sets.newHashSet(sql("SELECT * FROM %s", tableName))); - } - } - - private List findValidSnapshots(Table table) { - List validSnapshots = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests(table.io()).stream() - .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { - validSnapshots.add(snapshot); - } - } - return validSnapshots; - } - - private static StructLikeSet expectedRowSet(Table table, List rows) { - Record[] records = new Record[rows.size()]; - for (int i = 0; i < records.length; i++) { - records[i] = record((int) rows.get(i).getField(0), (String) rows.get(i).getField(1)); - } - return SimpleDataUtil.expectedRowSet(table, records); - } - - private static StructLikeSet actualRowSet(Table table, long snapshotId) throws IOException { - return SimpleDataUtil.actualRowSet(table, snapshotId, "*"); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java deleted file mode 100644 index e9372adda4c1..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ /dev/null @@ -1,207 +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; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Map; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Test; - -public class TestDataFileSerialization { - - private static final Schema DATE_SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - optional(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec PARTITION_SPEC = - PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); - - private static final Map COLUMN_SIZES = Maps.newHashMap(); - private static final Map VALUE_COUNTS = Maps.newHashMap(); - private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); - private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); - private static final Map LOWER_BOUNDS = Maps.newHashMap(); - private static final Map UPPER_BOUNDS = Maps.newHashMap(); - - static { - COLUMN_SIZES.put(1, 2L); - COLUMN_SIZES.put(2, 3L); - VALUE_COUNTS.put(1, 5L); - VALUE_COUNTS.put(2, 3L); - VALUE_COUNTS.put(4, 2L); - NULL_VALUE_COUNTS.put(1, 0L); - NULL_VALUE_COUNTS.put(2, 2L); - NAN_VALUE_COUNTS.put(4, 1L); - LOWER_BOUNDS.put(1, longToBuffer(0L)); - UPPER_BOUNDS.put(1, longToBuffer(4L)); - } - - private static final Metrics METRICS = - new Metrics( - 5L, null, VALUE_COUNTS, NULL_VALUE_COUNTS, NAN_VALUE_COUNTS, LOWER_BOUNDS, UPPER_BOUNDS); - - private static final DataFile DATA_FILE = - DataFiles.builder(PARTITION_SPEC) - .withPath("/path/to/data-1.parquet") - .withFileSizeInBytes(1234) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withSplitOffsets(ImmutableList.of(4L)) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) - .withSortOrder(SortOrder.unsorted()) - .build(); - - private static final DeleteFile POS_DELETE_FILE = - FileMetadata.deleteFileBuilder(PARTITION_SPEC) - .ofPositionDeletes() - .withPath("/path/to/pos-delete.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) - .withRecordCount(23) - .build(); - - private static final DeleteFile EQ_DELETE_FILE = - FileMetadata.deleteFileBuilder(PARTITION_SPEC) - .ofEqualityDeletes(2, 3) - .withPath("/path/to/equality-delete.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) - .withRecordCount(23) - .withSortOrder(SortOrder.unsorted()) - .build(); - - @Test - public void testJavaSerialization() throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(DATA_FILE); - out.writeObject(DATA_FILE.copy()); - - out.writeObject(POS_DELETE_FILE); - out.writeObject(POS_DELETE_FILE.copy()); - - out.writeObject(EQ_DELETE_FILE); - out.writeObject(EQ_DELETE_FILE.copy()); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); - TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); - } - - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a position DeleteFile") - .isInstanceOf(DeleteFile.class); - TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); - } - - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a equality DeleteFile") - .isInstanceOf(DeleteFile.class); - TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); - } - } - } - - @Test - public void testDataFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - kryo.serialize(DATA_FILE, outputView); - kryo.serialize(DATA_FILE.copy(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - DataFile dataFile1 = kryo.deserialize(inputView); - DataFile dataFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(DATA_FILE, dataFile1); - TestHelpers.assertEquals(DATA_FILE, dataFile2); - } - - @Test - public void testDeleteFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - kryo.serialize(POS_DELETE_FILE, outputView); - kryo.serialize(POS_DELETE_FILE.copy(), outputView); - - kryo.serialize(EQ_DELETE_FILE, outputView); - kryo.serialize(EQ_DELETE_FILE.copy(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - - DeleteFile posDeleteFile1 = kryo.deserialize(inputView); - DeleteFile posDeleteFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile1); - TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile2); - - DeleteFile eqDeleteFile1 = kryo.deserialize(inputView); - DeleteFile eqDeleteFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile1); - TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile2); - } - - private static ByteBuffer longToBuffer(long value) { - return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java deleted file mode 100644 index 884ea2d1d3b1..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ /dev/null @@ -1,59 +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; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; - -public class TestFixtures { - - private TestFixtures() {} - - public static final Schema SCHEMA = - new Schema( - required(1, "data", Types.StringType.get()), - required(2, "id", Types.LongType.get()), - required(3, "dt", Types.StringType.get())); - - public static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("dt").bucket("id", 1).build(); - - public static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); - - public static final String DATABASE = "default"; - public static final String TABLE = "t"; - public static final String SINK_TABLE = "t_sink"; - - public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); - - public static final Schema TS_SCHEMA = - new Schema( - required(1, "ts", Types.TimestampType.withoutZone()), - required(2, "str", Types.StringType.get())); - - public static final PartitionSpec TS_SPEC = - PartitionSpec.builderFor(TS_SCHEMA).hour("ts").build(); - - public static final RowType TS_ROW_TYPE = FlinkSchemaUtil.convert(TS_SCHEMA); -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java deleted file mode 100644 index 2fbd7627efab..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.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.flink; - -import java.io.File; -import java.util.concurrent.TimeUnit; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableDescriptor; -import org.apache.flink.table.api.TableEnvironment; -import org.assertj.core.api.Assertions; -import org.awaitility.Awaitility; -import org.junit.Test; - -public class TestFlinkAnonymousTable extends FlinkTestBase { - - @Test - public void testWriteAnonymousTable() throws Exception { - File warehouseDir = TEMPORARY_FOLDER.newFolder(); - TableEnvironment tEnv = getTableEnv(); - Table table = - tEnv.from( - TableDescriptor.forConnector("datagen") - .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) - .option("number-of-rows", "3") - .build()); - - TableDescriptor descriptor = - TableDescriptor.forConnector("iceberg") - .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) - .option("catalog-name", "hadoop_test") - .option("catalog-type", "hadoop") - .option("catalog-database", "test_db") - .option("catalog-table", "test") - .option("warehouse", warehouseDir.getAbsolutePath()) - .build(); - - table.insertInto(descriptor).execute(); - Awaitility.await() - .atMost(3, TimeUnit.SECONDS) - .untilAsserted( - () -> - Assertions.assertThat( - warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) - .exists()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java deleted file mode 100644 index f46d50a5f0ab..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ /dev/null @@ -1,254 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.nio.file.Path; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; -import org.apache.flink.types.Row; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkCatalogDatabase extends CatalogTestBase { - - @AfterEach - @Override - public void clean() { - sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testCreateNamespace() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should not already exist") - .isFalse(); - - sql("CREATE DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should exist") - .isTrue(); - - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should still exist") - .isTrue(); - - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should be dropped") - .isFalse(); - - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should be created") - .isTrue(); - } - - @TestTemplate - public void testDropEmptyDatabase() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - sql("DROP DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should have been dropped") - .isFalse(); - } - - @TestTemplate - public void testDropNonEmptyNamespace() { - assumeThat(isHadoopCatalog) - .as("Hadoop catalog throws IOException: Directory is not empty.") - .isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) - .as("Table should exist") - .isTrue(); - Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) - .cause() - .isInstanceOf(DatabaseNotEmptyException.class) - .hasMessage( - String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); - sql("DROP TABLE %s.tl", flinkDatabase); - } - - @TestTemplate - public void testListTables() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - assertThat(sql("SHOW TABLES")).isEmpty(); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); - - List tables = sql("SHOW TABLES"); - assertThat(tables).hasSize(1); - assertThat("tl").as("Table name should match").isEqualTo(tables.get(0).getField(0)); - } - - @TestTemplate - public void testListNamespace() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - - List databases = sql("SHOW DATABASES"); - - if (isHadoopCatalog) { - assertThat(databases).hasSize(1); - assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); - if (!baseNamespace.isEmpty()) { - // test namespace not belongs to this catalog - validationNamespaceCatalog.createNamespace( - Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); - databases = sql("SHOW DATABASES"); - assertThat(databases).hasSize(1); - assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); - } - } else { - // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the - // creation for default - // database. See HiveMetaStore.HMSHandler.init. - assertThat(databases) - .as("Should have db database") - .anyMatch(d -> Objects.equals(d.getField(0), "db")); - } - } - - @TestTemplate - public void testCreateNamespaceWithMetadata() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("prop", "value"); - } - - @TestTemplate - public void testCreateNamespaceWithComment() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - - sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("comment", "namespace doc"); - } - - @TestTemplate - public void testCreateNamespaceWithLocation() throws Exception { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - - Path location = temporaryDirectory.getRoot(); - sql("CREATE DATABASE %s WITH ('location'='%s')", flinkDatabase, location); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("location", "file:" + location.getRoot()); - } - - @TestTemplate - public void testSetProperties() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - - sql("CREATE DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - - Map defaultMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(defaultMetadata).doesNotContainKey("prop"); - sql("ALTER DATABASE %s SET ('prop'='value')", flinkDatabase); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("prop", "value"); - } - - @TestTemplate - public void testHadoopNotSupportMeta() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isTrue(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - Assertions.assertThatThrownBy( - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) - .cause() - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage( - String.format( - "Cannot create namespace %s: metadata is not supported", icebergNamespace)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java deleted file mode 100644 index c3f8bf92e47a..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ /dev/null @@ -1,115 +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; - -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; - -public class TestFlinkCatalogFactory { - - private Map props; - - @Before - public void before() { - props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(CatalogProperties.WAREHOUSE_LOCATION, "/tmp/location"); - } - - @Test - public void testCreateCatalogHive() { - String catalogName = "hiveCatalog"; - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); - } - - @Test - public void testCreateCatalogHadoop() { - String catalogName = "hadoopCatalog"; - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); - } - - @Test - public void testCreateCatalogCustom() { - String catalogName = "customCatalog"; - props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - Assertions.assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); - } - - @Test - public void testCreateCatalogCustomWithHiveCatalogTypeSet() { - String catalogName = "customCatalog"; - props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - Assertions.assertThatThrownBy( - () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith( - "Cannot create catalog customCatalog, both catalog-type and catalog-impl are set"); - } - - @Test - public void testLoadCatalogUnknown() { - String catalogName = "unknownCatalog"; - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); - Assertions.assertThatThrownBy( - () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessageStartingWith("Unknown catalog-type: fooType"); - } - - public static class CustomHadoopCatalog extends HadoopCatalog { - - public CustomHadoopCatalog() {} - - public CustomHadoopCatalog(Configuration conf, String warehouseLocation) { - setConf(conf); - initialize( - "custom", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation)); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java deleted file mode 100644 index 3442ae8f9b16..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ /dev/null @@ -1,413 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.util.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.constraints.UniqueConstraint; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DataOperations; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkCatalogTable extends CatalogTestBase { - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @AfterEach - public void cleanNamespaces() { - sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testGetTable() { - sql("CREATE TABLE tl(id BIGINT, strV STRING)"); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, "tl")); - Schema iSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "strV", Types.StringType.get())); - assertThat(table.schema().toString()) - .as("Should load the expected iceberg schema") - .isEqualTo(iSchema.toString()); - } - - @TestTemplate - public void testRenameTable() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support rename table").isFalse(); - final Schema tableSchema = - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); - validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); - sql("ALTER TABLE tl RENAME TO tl2"); - - assertThatThrownBy(() -> getTableEnv().from("tl")) - .isInstanceOf(ValidationException.class) - .hasMessage("Table `tl` was not found."); - - Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); - assertThat(tableSchema.asStruct()).isEqualTo(actualSchema.asStruct()); - } - - @TestTemplate - public void testCreateTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT)"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - CatalogTable catalogTable = catalogTable("tl"); - assertThat(catalogTable.getSchema()) - .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); - } - - @TestTemplate - public void testCreateTableWithPrimaryKey() throws Exception { - sql("CREATE TABLE tl(id BIGINT, data STRING, key STRING PRIMARY KEY NOT ENFORCED)"); - - Table table = table("tl"); - assertThat(table.schema().identifierFieldIds()) - .as("Should have the expected row key.") - .isEqualTo(Sets.newHashSet(table.schema().findField("key").fieldId())); - CatalogTable catalogTable = catalogTable("tl"); - Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); - assertThat(uniqueConstraintOptional).isPresent(); - assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("key"); - } - - @TestTemplate - public void testCreateTableWithMultiColumnsInPrimaryKey() throws Exception { - sql( - "CREATE TABLE tl(id BIGINT, data STRING, CONSTRAINT pk_constraint PRIMARY KEY(data, id) NOT ENFORCED)"); - - Table table = table("tl"); - assertThat(table.schema().identifierFieldIds()) - .as("Should have the expected RowKey") - .isEqualTo( - Sets.newHashSet( - table.schema().findField("id").fieldId(), - table.schema().findField("data").fieldId())); - CatalogTable catalogTable = catalogTable("tl"); - Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); - assertThat(uniqueConstraintOptional).isPresent(); - assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("id", "data"); - } - - @TestTemplate - public void testCreateTableIfNotExists() { - sql("CREATE TABLE tl(id BIGINT)"); - - // Assert that table does exist. - assertThat(table("tl")).isNotNull(); - - sql("DROP TABLE tl"); - Assertions.assertThatThrownBy(() -> table("tl")) - .isInstanceOf(NoSuchTableException.class) - .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); - - sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); - assertThat(table("tl").properties()).doesNotContainKey("key"); - - table("tl").updateProperties().set("key", "value").commit(); - assertThat(table("tl").properties()).containsEntry("key", "value"); - - sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); - assertThat(table("tl").properties()).containsEntry("key", "value"); - } - - @TestTemplate - public void testCreateTableLike() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT)"); - sql("CREATE TABLE tl2 LIKE tl"); - - Table table = table("tl2"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - CatalogTable catalogTable = catalogTable("tl2"); - assertThat(catalogTable.getSchema()) - .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); - } - - @TestTemplate - public void testCreateTableLocation() { - assumeThat(isHadoopCatalog) - .as("HadoopCatalog does not support creating table with location") - .isFalse(); - sql("CREATE TABLE tl(id BIGINT) WITH ('location'='file:///tmp/location')"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - assertThat(table.location()).isEqualTo("file:///tmp/location"); - } - - @TestTemplate - public void testCreatePartitionTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT, dt STRING) PARTITIONED BY(dt)"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - assertThat(table.spec()) - .isEqualTo(PartitionSpec.builderFor(table.schema()).identity("dt").build()); - CatalogTable catalogTable = catalogTable("tl"); - assertThat(catalogTable.getSchema()) - .isEqualTo( - TableSchema.builder() - .field("id", DataTypes.BIGINT()) - .field("dt", DataTypes.STRING()) - .build()); - assertThat(catalogTable.getPartitionKeys()).isEqualTo(Collections.singletonList("dt")); - } - - @TestTemplate - public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); - - Table table = table("tl"); - assertThat(((BaseTable) table).operations().current().formatVersion()).isEqualTo(2); - } - - @TestTemplate - public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); - - Table table = table("tl"); - TableOperations ops = ((BaseTable) table).operations(); - assertThat(ops.refresh().formatVersion()) - .as("should create table using format v1") - .isEqualTo(1); - sql("ALTER TABLE tl SET('format-version'='2')"); - assertThat(ops.refresh().formatVersion()) - .as("should update table to use format v2") - .isEqualTo(2); - } - - @TestTemplate - public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); - - Table table = table("tl"); - TableOperations ops = ((BaseTable) table).operations(); - assertThat(ops.refresh().formatVersion()) - .as("should create table using format v2") - .isEqualTo(2); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) - .rootCause() - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot downgrade v2 table to v1"); - } - - @TestTemplate - public void testLoadTransformPartitionTable() throws TableNotExistException { - Schema schema = new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - schema, - PartitionSpec.builderFor(schema).bucket("id", 100).build()); - - CatalogTable catalogTable = catalogTable("tl"); - assertThat(catalogTable.getSchema()) - .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); - assertThat(catalogTable.getPartitionKeys()).isEmpty(); - } - - @TestTemplate - public void testAlterTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); - Map properties = Maps.newHashMap(); - properties.put("oldK", "oldV"); - - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // remove property - CatalogTable catalogTable = catalogTable("tl"); - properties.remove("oldK"); - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .alterTable(new ObjectPath(DATABASE, "tl"), catalogTable.copy(properties), false); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - } - - @TestTemplate - public void testAlterTableWithPrimaryKey() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT, PRIMARY KEY(id) NOT ENFORCED) WITH ('oldK'='oldV')"); - Map properties = Maps.newHashMap(); - properties.put("oldK", "oldV"); - - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // remove property - CatalogTable catalogTable = catalogTable("tl"); - properties.remove("oldK"); - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .alterTable(new ObjectPath(DATABASE, "tl"), catalogTable.copy(properties), false); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - } - - @TestTemplate - public void testRelocateTable() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support relocate table").isFalse(); - sql("CREATE TABLE tl(id BIGINT)"); - sql("ALTER TABLE tl SET('location'='file:///tmp/location')"); - assertThat(table("tl").location()).isEqualTo("file:///tmp/location"); - } - - @TestTemplate - public void testSetCurrentAndCherryPickSnapshotId() { - sql("CREATE TABLE tl(c1 INT, c2 STRING, c3 STRING) PARTITIONED BY (c1)"); - - Table table = table("tl"); - - DataFile fileA = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - DataFile fileB = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-b.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=1") // easy way to set partition data for now - .withRecordCount(1) - .build(); - DataFile replacementFile = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-a-replacement.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - - table.newAppend().appendFile(fileA).commit(); - long snapshotId = table.currentSnapshot().snapshotId(); - - // stage an overwrite that replaces FILE_A - table.newReplacePartitions().addFile(replacementFile).stageOnly().commit(); - - Snapshot staged = Iterables.getLast(table.snapshots()); - assertThat(staged.operation()) - .as("Should find the staged overwrite snapshot") - .isEqualTo(DataOperations.OVERWRITE); - // add another append so that the original commit can't be fast-forwarded - table.newAppend().appendFile(fileB).commit(); - - // test cherry pick - sql("ALTER TABLE tl SET('cherry-pick-snapshot-id'='%s')", staged.snapshotId()); - validateTableFiles(table, fileB, replacementFile); - - // test set current snapshot - sql("ALTER TABLE tl SET('current-snapshot-id'='%s')", snapshotId); - validateTableFiles(table, fileA); - } - - private void validateTableFiles(Table tbl, DataFile... expectedFiles) { - tbl.refresh(); - Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); - Set actualFilePaths = - StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) - .map(FileScanTask::file) - .map(ContentFile::path) - .collect(Collectors.toSet()); - assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); - } - - private Table table(String name) { - return validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, name)); - } - - private CatalogTable catalogTable(String name) throws TableNotExistException { - return (CatalogTable) - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .getTable(new ObjectPath(DATABASE, name)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java deleted file mode 100644 index 8fe406747169..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ /dev/null @@ -1,118 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.List; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkCatalogTablePartitions extends CatalogTestBase { - - private String tableName = "test_table"; - - @Parameter(index = 2) - private FileFormat format; - - @Parameter(index = 3) - private Boolean cacheEnabled; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, cacheEnabled={3}") - protected static List parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { - for (Boolean cacheEnabled : new Boolean[] {true, false}) { - for (Object[] catalogParams : CatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, cacheEnabled}); - } - } - } - return parameters; - } - - @Override - @BeforeEach - public void before() { - super.before(); - config.put(CatalogProperties.CACHE_ENABLED, String.valueOf(cacheEnabled)); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @AfterEach - public void cleanNamespaces() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testListPartitionsWithUnpartitionedTable() { - sql( - "CREATE TABLE %s (id INT, data VARCHAR) with ('write.format.default'='%s')", - tableName, format.name()); - sql("INSERT INTO %s SELECT 1,'a'", tableName); - - ObjectPath objectPath = new ObjectPath(DATABASE, tableName); - FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) - .isInstanceOf(TableNotPartitionedException.class) - .hasMessage("Table " + objectPath + " in catalog " + catalogName + " is not partitioned."); - } - - @TestTemplate - public void testListPartitionsWithPartitionedTable() - throws TableNotExistException, TableNotPartitionedException { - sql( - "CREATE TABLE %s (id INT, data VARCHAR) PARTITIONED BY (data) " - + "with ('write.format.default'='%s')", - tableName, format.name()); - sql("INSERT INTO %s SELECT 1,'a'", tableName); - sql("INSERT INTO %s SELECT 2,'b'", tableName); - - ObjectPath objectPath = new ObjectPath(DATABASE, tableName); - FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - List list = flinkCatalog.listPartitions(objectPath); - assertThat(list).hasSize(2); - List expected = Lists.newArrayList(); - CatalogPartitionSpec partitionSpec1 = new CatalogPartitionSpec(ImmutableMap.of("data", "a")); - CatalogPartitionSpec partitionSpec2 = new CatalogPartitionSpec(ImmutableMap.of("data", "b")); - expected.add(partitionSpec1); - expected.add(partitionSpec2); - assertThat(list).as("Should produce the expected catalog partition specs.").isEqualTo(expected); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java deleted file mode 100644 index 4b6ac25ab8e3..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java +++ /dev/null @@ -1,61 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; - -public class TestFlinkConfParser { - - @Test - public void testDurationConf() { - Map writeOptions = ImmutableMap.of("write-prop", "111s"); - - ConfigOption configOption = - ConfigOptions.key("conf-prop").durationType().noDefaultValue(); - Configuration flinkConf = new Configuration(); - flinkConf.setString(configOption.key(), "222s"); - - Table table = mock(Table.class); - when(table.properties()).thenReturn(ImmutableMap.of("table-prop", "333s")); - - FlinkConfParser confParser = new FlinkConfParser(table, writeOptions, flinkConf); - Duration defaultVal = Duration.ofMillis(999); - - Duration result = - confParser.durationConf().option("write-prop").defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(111)); - - result = confParser.durationConf().flinkConfig(configOption).defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(222)); - - result = confParser.durationConf().tableProperty("table-prop").defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(333)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java deleted file mode 100644 index c89ea4f53054..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java +++ /dev/null @@ -1,467 +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; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Expressions; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.expressions.ApiExpressionUtils; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.expressions.FieldReferenceExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.UnresolvedCallExpression; -import org.apache.flink.table.expressions.UnresolvedReferenceExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.iceberg.expressions.And; -import org.apache.iceberg.expressions.BoundLiteralPredicate; -import org.apache.iceberg.expressions.Not; -import org.apache.iceberg.expressions.Or; -import org.apache.iceberg.expressions.UnboundPredicate; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkFilters { - - private static final TableSchema TABLE_SCHEMA = - TableSchema.builder() - .field("field1", DataTypes.INT()) - .field("field2", DataTypes.BIGINT()) - .field("field3", DataTypes.FLOAT()) - .field("field4", DataTypes.DOUBLE()) - .field("field5", DataTypes.STRING()) - .field("field6", DataTypes.BOOLEAN()) - .field("field7", DataTypes.BINARY(2)) - .field("field8", DataTypes.DECIMAL(10, 2)) - .field("field9", DataTypes.DATE()) - .field("field10", DataTypes.TIME()) - .field("field11", DataTypes.TIMESTAMP()) - .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .build(); - - // A map list of fields and values used to verify the conversion of flink expression to iceberg - // expression - private static final List> FIELD_VALUE_LIST = - ImmutableList.of( - Pair.of("field1", 1), - Pair.of("field2", 2L), - Pair.of("field3", 3F), - Pair.of("field4", 4D), - Pair.of("field5", "iceberg"), - Pair.of("field6", true), - Pair.of("field7", new byte[] {'a', 'b'}), - Pair.of("field8", BigDecimal.valueOf(10.12)), - Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())), - Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())), - Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())), - Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))); - - @Test - public void testFlinkDataTypeEqual() { - matchLiteral("field1", 1, 1); - matchLiteral("field2", 10L, 10L); - matchLiteral("field3", 1.2F, 1.2F); - matchLiteral("field4", 3.4D, 3.4D); - matchLiteral("field5", "abcd", "abcd"); - matchLiteral("field6", true, true); - matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'})); - matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12)); - - LocalDate date = LocalDate.parse("2020-12-23"); - matchLiteral("field9", date, DateTimeUtil.daysFromDate(date)); - - LocalTime time = LocalTime.parse("12:13:14"); - matchLiteral("field10", time, DateTimeUtil.microsFromTime(time)); - - LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14"); - matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime)); - - Instant instant = Instant.parse("2020-12-23T12:13:14.00Z"); - matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant)); - } - - @Test - public void testEquals() { - for (Pair pair : FIELD_VALUE_LIST) { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second()); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - } - - @Test - public void testEqualsNaN() { - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNaN("field3"); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testNotEquals() { - for (Pair pair : FIELD_VALUE_LIST) { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second()); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - } - - @Test - public void testNotEqualsNaN() { - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNaN("field3"); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testGreaterThan() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testGreaterThanEquals() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testLessThan() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.lessThan("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testLessThanEquals() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testIsNull() { - Expression expr = resolve(Expressions.$("field1").isNull()); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); - assertPredicatesMatch(expected, actual.get()); - } - - @Test - public void testIsNotNull() { - Expression expr = resolve(Expressions.$("field1").isNotNull()); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.notNull("field1"); - assertPredicatesMatch(expected, actual.get()); - } - - @Test - public void testAnd() { - Expression expr = - resolve( - Expressions.$("field1") - .isEqual(Expressions.lit(1)) - .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - And and = (And) actual.get(); - And expected = - (And) - org.apache.iceberg.expressions.Expressions.and( - org.apache.iceberg.expressions.Expressions.equal("field1", 1), - org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); - - assertPredicatesMatch(expected.left(), and.left()); - assertPredicatesMatch(expected.right(), and.right()); - } - - @Test - public void testOr() { - Expression expr = - resolve( - Expressions.$("field1") - .isEqual(Expressions.lit(1)) - .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - Or or = (Or) actual.get(); - Or expected = - (Or) - org.apache.iceberg.expressions.Expressions.or( - org.apache.iceberg.expressions.Expressions.equal("field1", 1), - org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); - - assertPredicatesMatch(expected.left(), or.left()); - assertPredicatesMatch(expected.right(), or.right()); - } - - @Test - public void testNot() { - Expression expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.NOT, - Expressions.$("field1").isEqual(Expressions.lit(1)))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - Not not = (Not) actual.get(); - Not expected = - (Not) - org.apache.iceberg.expressions.Expressions.not( - org.apache.iceberg.expressions.Expressions.equal("field1", 1)); - - Assert.assertEquals("Predicate operation should match", expected.op(), not.op()); - assertPredicatesMatch(expected.child(), not.child()); - } - - @Test - public void testLike() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc"); - Expression expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - assertPredicatesMatch(expected, actual.get()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, - Expressions.$("field5"), - Expressions.lit("%abc%"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, - Expressions.$("field5"), - Expressions.lit("abc%d"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); - actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); - } - - @SuppressWarnings("unchecked") - private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { - Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); - Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); - org.apache.iceberg.expressions.Expression expression = actual.get(); - Assertions.assertThat(expression) - .as("The expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - UnboundPredicate unboundPredicate = (UnboundPredicate) expression; - - org.apache.iceberg.expressions.Expression expression1 = - unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); - Assertions.assertThat(expression1) - .as("The expression should be a BoundLiteralPredicate") - .isInstanceOf(BoundLiteralPredicate.class); - - BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; - Assert.assertTrue("Should match the literal", predicate.test(icebergLiteral)); - } - - private static Expression resolve(Expression originalExpression) { - return originalExpression.accept( - new ApiExpressionDefaultVisitor() { - @Override - public Expression visit(UnresolvedReferenceExpression unresolvedReference) { - String name = unresolvedReference.getName(); - Optional field = TABLE_SCHEMA.getTableColumn(name); - if (field.isPresent()) { - int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get()); - return new FieldReferenceExpression(name, field.get().getType(), 0, index); - } else { - return null; - } - } - - @Override - public Expression visit(UnresolvedCallExpression unresolvedCall) { - List children = - unresolvedCall.getChildren().stream() - .map(e -> (ResolvedExpression) e.accept(this)) - .collect(Collectors.toList()); - return new CallExpression( - unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); - } - - @Override - public Expression visit(ValueLiteralExpression valueLiteral) { - return valueLiteral; - } - - @Override - protected Expression defaultMethod(Expression expression) { - throw new UnsupportedOperationException( - String.format("unsupported expression: %s", expression)); - } - }); - } - - private void assertPredicatesMatch( - org.apache.iceberg.expressions.Expression expected, - org.apache.iceberg.expressions.Expression actual) { - Assertions.assertThat(expected) - .as("The expected expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - Assertions.assertThat(actual) - .as("The actual expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - UnboundPredicate predicateExpected = (UnboundPredicate) expected; - UnboundPredicate predicateActual = (UnboundPredicate) actual; - Assert.assertEquals( - "Predicate operation should match", predicateExpected.op(), predicateActual.op()); - Assert.assertEquals( - "Predicate literal should match", predicateExpected.literal(), predicateActual.literal()); - Assert.assertEquals( - "Predicate name should match", - predicateExpected.ref().name(), - predicateActual.ref().name()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java deleted file mode 100644 index 47ee2afceb02..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ /dev/null @@ -1,103 +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; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkHiveCatalog extends FlinkTestBase { - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - @Test - public void testCreateCatalogWithWarehouseLocation() throws IOException { - Map props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - - File warehouseDir = tempFolder.newFolder(); - props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); - - checkSQLQuery(props, warehouseDir); - } - - @Test - public void testCreateCatalogWithHiveConfDir() throws IOException { - // Dump the hive conf into a local file. - File hiveConfDir = tempFolder.newFolder(); - File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); - File warehouseDir = tempFolder.newFolder(); - try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { - Configuration newConf = new Configuration(hiveConf); - // Set another new directory which is different with the hive metastore's warehouse path. - newConf.set( - HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); - newConf.writeXml(fos); - } - Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); - - // Construct the catalog attributions. - Map props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - // Set the 'hive-conf-dir' instead of 'warehouse' - props.put(FlinkCatalogFactory.HIVE_CONF_DIR, hiveConfDir.getAbsolutePath()); - - checkSQLQuery(props, warehouseDir); - } - - private void checkSQLQuery(Map catalogProperties, File warehouseDir) - throws IOException { - sql("CREATE CATALOG test_catalog WITH %s", CatalogTestBase.toWithClause(catalogProperties)); - sql("USE CATALOG test_catalog"); - sql("CREATE DATABASE test_db"); - sql("USE test_db"); - sql("CREATE TABLE test_table(c1 INT, c2 STRING)"); - sql("INSERT INTO test_table SELECT 1, 'a'"); - - Path databasePath = warehouseDir.toPath().resolve("test_db.db"); - Assert.assertTrue("Database path should exist", Files.exists(databasePath)); - - Path tablePath = databasePath.resolve("test_table"); - Assert.assertTrue("Table path should exist", Files.exists(tablePath)); - - Path dataPath = tablePath.resolve("data"); - Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); - Assert.assertEquals( - "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); - - sql("DROP TABLE test_table"); - sql("DROP DATABASE test_db"); - dropCatalog("test_catalog", false); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java deleted file mode 100644 index 16b220ba6715..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ /dev/null @@ -1,415 +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; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -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.types.Type; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkSchemaUtil { - - @Test - public void testConvertFlinkSchemaToIcebergSchema() { - TableSchema flinkSchema = - TableSchema.builder() - .field("id", DataTypes.INT().notNull()) - .field("name", DataTypes.STRING()) /* optional by default */ - .field("salary", DataTypes.DOUBLE().notNull()) - .field( - "locations", - DataTypes.MAP( - DataTypes.STRING(), - DataTypes.ROW( - DataTypes.FIELD("posX", DataTypes.DOUBLE().notNull(), "X field"), - DataTypes.FIELD("posY", DataTypes.DOUBLE().notNull(), "Y field")))) - .field("strArray", DataTypes.ARRAY(DataTypes.STRING()).nullable()) - .field("intArray", DataTypes.ARRAY(DataTypes.INT()).nullable()) - .field("char", DataTypes.CHAR(10).notNull()) - .field("varchar", DataTypes.VARCHAR(10).notNull()) - .field("boolean", DataTypes.BOOLEAN().nullable()) - .field("tinyint", DataTypes.TINYINT()) - .field("smallint", DataTypes.SMALLINT()) - .field("bigint", DataTypes.BIGINT()) - .field("varbinary", DataTypes.VARBINARY(10)) - .field("binary", DataTypes.BINARY(10)) - .field("time", DataTypes.TIME()) - .field("timestampWithoutZone", DataTypes.TIMESTAMP()) - .field("timestampWithZone", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .field("date", DataTypes.DATE()) - .field("decimal", DataTypes.DECIMAL(2, 2)) - .field("decimal2", DataTypes.DECIMAL(38, 2)) - .field("decimal3", DataTypes.DECIMAL(10, 1)) - .field("multiset", DataTypes.MULTISET(DataTypes.STRING().notNull())) - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get(), null), - Types.NestedField.optional(1, "name", Types.StringType.get(), null), - Types.NestedField.required(2, "salary", Types.DoubleType.get(), null), - Types.NestedField.optional( - 3, - "locations", - Types.MapType.ofOptional( - 24, - 25, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(22, "posX", Types.DoubleType.get(), "X field"), - Types.NestedField.required( - 23, "posY", Types.DoubleType.get(), "Y field")))), - Types.NestedField.optional( - 4, "strArray", Types.ListType.ofOptional(26, Types.StringType.get())), - Types.NestedField.optional( - 5, "intArray", Types.ListType.ofOptional(27, Types.IntegerType.get())), - Types.NestedField.required(6, "char", Types.StringType.get()), - Types.NestedField.required(7, "varchar", Types.StringType.get()), - Types.NestedField.optional(8, "boolean", Types.BooleanType.get()), - Types.NestedField.optional(9, "tinyint", Types.IntegerType.get()), - Types.NestedField.optional(10, "smallint", Types.IntegerType.get()), - Types.NestedField.optional(11, "bigint", Types.LongType.get()), - Types.NestedField.optional(12, "varbinary", Types.BinaryType.get()), - Types.NestedField.optional(13, "binary", Types.FixedType.ofLength(10)), - Types.NestedField.optional(14, "time", Types.TimeType.get()), - Types.NestedField.optional( - 15, "timestampWithoutZone", Types.TimestampType.withoutZone()), - Types.NestedField.optional(16, "timestampWithZone", Types.TimestampType.withZone()), - Types.NestedField.optional(17, "date", Types.DateType.get()), - Types.NestedField.optional(18, "decimal", Types.DecimalType.of(2, 2)), - Types.NestedField.optional(19, "decimal2", Types.DecimalType.of(38, 2)), - Types.NestedField.optional(20, "decimal3", Types.DecimalType.of(10, 1)), - Types.NestedField.optional( - 21, - "multiset", - Types.MapType.ofRequired(28, 29, Types.StringType.get(), Types.IntegerType.get()))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testMapField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "map_int_long", - DataTypes.MAP(DataTypes.INT(), DataTypes.BIGINT()).notNull()) /* Required */ - .field( - "map_int_array_string", - DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING())) - .field( - "map_decimal_string", DataTypes.MAP(DataTypes.DECIMAL(10, 2), DataTypes.STRING())) - .field( - "map_fields_fields", - DataTypes.MAP( - DataTypes.ROW( - DataTypes.FIELD("field_int", DataTypes.INT(), "doc - int"), - DataTypes.FIELD("field_string", DataTypes.STRING(), "doc - string")) - .notNull(), /* Required */ - DataTypes.ROW( - DataTypes.FIELD( - "field_array", - DataTypes.ARRAY(DataTypes.STRING()), - "doc - array")) - .notNull() /* Required */) - .notNull() /* Required */) - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "map_int_long", - Types.MapType.ofOptional(4, 5, Types.IntegerType.get(), Types.LongType.get()), - null), - Types.NestedField.optional( - 1, - "map_int_array_string", - Types.MapType.ofOptional( - 7, - 8, - Types.ListType.ofOptional(6, Types.IntegerType.get()), - Types.StringType.get()), - null), - Types.NestedField.optional( - 2, - "map_decimal_string", - Types.MapType.ofOptional( - 9, 10, Types.DecimalType.of(10, 2), Types.StringType.get())), - Types.NestedField.required( - 3, - "map_fields_fields", - Types.MapType.ofRequired( - 15, - 16, - Types.StructType.of( - Types.NestedField.optional( - 11, "field_int", Types.IntegerType.get(), "doc - int"), - Types.NestedField.optional( - 12, "field_string", Types.StringType.get(), "doc - string")), - Types.StructType.of( - Types.NestedField.optional( - 14, - "field_array", - Types.ListType.ofOptional(13, Types.StringType.get()), - "doc - array"))))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testStructField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "struct_int_string_decimal", - DataTypes.ROW( - DataTypes.FIELD("field_int", DataTypes.INT()), - DataTypes.FIELD("field_string", DataTypes.STRING()), - DataTypes.FIELD("field_decimal", DataTypes.DECIMAL(19, 2)), - DataTypes.FIELD( - "field_struct", - DataTypes.ROW( - DataTypes.FIELD("inner_struct_int", DataTypes.INT()), - DataTypes.FIELD( - "inner_struct_float_array", - DataTypes.ARRAY(DataTypes.FLOAT()))) - .notNull()) /* Row is required */) - .notNull()) /* Required */ - .field( - "struct_map_int_int", - DataTypes.ROW( - DataTypes.FIELD( - "field_map", DataTypes.MAP(DataTypes.INT(), DataTypes.INT()))) - .nullable()) /* Optional */ - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "struct_int_string_decimal", - Types.StructType.of( - Types.NestedField.optional(5, "field_int", Types.IntegerType.get()), - Types.NestedField.optional(6, "field_string", Types.StringType.get()), - Types.NestedField.optional(7, "field_decimal", Types.DecimalType.of(19, 2)), - Types.NestedField.required( - 8, - "field_struct", - Types.StructType.of( - Types.NestedField.optional( - 3, "inner_struct_int", Types.IntegerType.get()), - Types.NestedField.optional( - 4, - "inner_struct_float_array", - Types.ListType.ofOptional(2, Types.FloatType.get())))))), - Types.NestedField.optional( - 1, - "struct_map_int_int", - Types.StructType.of( - Types.NestedField.optional( - 11, - "field_map", - Types.MapType.ofOptional( - 9, 10, Types.IntegerType.get(), Types.IntegerType.get()))))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testListField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "list_struct_fields", - DataTypes.ARRAY(DataTypes.ROW(DataTypes.FIELD("field_int", DataTypes.INT()))) - .notNull()) /* Required */ - .field( - "list_optional_struct_fields", - DataTypes.ARRAY( - DataTypes.ROW( - DataTypes.FIELD( - "field_timestamp_with_local_time_zone", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()))) - .nullable()) /* Optional */ - .field( - "list_map_fields", - DataTypes.ARRAY( - DataTypes.MAP( - DataTypes.ARRAY( - DataTypes.INT().notNull()), /* Key of map must be required */ - DataTypes.ROW( - DataTypes.FIELD("field_0", DataTypes.INT(), "doc - int"))) - .notNull()) - .notNull()) /* Required */ - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "list_struct_fields", - Types.ListType.ofOptional( - 4, - Types.StructType.of( - Types.NestedField.optional(3, "field_int", Types.IntegerType.get())))), - Types.NestedField.optional( - 1, - "list_optional_struct_fields", - Types.ListType.ofOptional( - 6, - Types.StructType.of( - Types.NestedField.optional( - 5, - "field_timestamp_with_local_time_zone", - Types.TimestampType.withZone())))), - Types.NestedField.required( - 2, - "list_map_fields", - Types.ListType.ofRequired( - 11, - Types.MapType.ofOptional( - 9, - 10, - Types.ListType.ofRequired(7, Types.IntegerType.get()), - Types.StructType.of( - Types.NestedField.optional( - 8, "field_0", Types.IntegerType.get(), "doc - int")))))); - - checkSchema(flinkSchema, icebergSchema); - } - - private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { - Assert.assertEquals(icebergSchema.asStruct(), FlinkSchemaUtil.convert(flinkSchema).asStruct()); - // The conversion is not a 1:1 mapping, so we just check iceberg types. - Assert.assertEquals( - icebergSchema.asStruct(), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) - .asStruct()); - } - - @Test - public void testInconsistentTypes() { - checkInconsistentType( - Types.UUIDType.get(), new BinaryType(16), new BinaryType(16), Types.FixedType.ofLength(16)); - checkInconsistentType( - Types.StringType.get(), - new VarCharType(VarCharType.MAX_LENGTH), - new CharType(100), - Types.StringType.get()); - checkInconsistentType( - Types.BinaryType.get(), - new VarBinaryType(VarBinaryType.MAX_LENGTH), - new VarBinaryType(100), - Types.BinaryType.get()); - checkInconsistentType( - Types.TimeType.get(), new TimeType(), new TimeType(3), Types.TimeType.get()); - checkInconsistentType( - Types.TimestampType.withoutZone(), - new TimestampType(6), - new TimestampType(3), - Types.TimestampType.withoutZone()); - checkInconsistentType( - Types.TimestampType.withZone(), - new LocalZonedTimestampType(6), - new LocalZonedTimestampType(3), - Types.TimestampType.withZone()); - } - - private void checkInconsistentType( - Type icebergType, - LogicalType flinkExpectedType, - LogicalType flinkType, - Type icebergExpectedType) { - Assert.assertEquals(flinkExpectedType, FlinkSchemaUtil.convert(icebergType)); - Assert.assertEquals( - Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType)), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()); - } - - @Test - public void testConvertFlinkSchemaBaseOnIcebergSchema() { - Schema baseSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required(101, "int", Types.IntegerType.get()), - Types.NestedField.optional(102, "string", Types.StringType.get())), - Sets.newHashSet(101)); - - TableSchema flinkSchema = - TableSchema.builder() - .field("int", DataTypes.INT().notNull()) - .field("string", DataTypes.STRING().nullable()) - .primaryKey("int") - .build(); - Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); - Assert.assertEquals(baseSchema.asStruct(), convertedSchema.asStruct()); - Assert.assertEquals(ImmutableSet.of(101), convertedSchema.identifierFieldIds()); - } - - @Test - public void testConvertFlinkSchemaWithPrimaryKeys() { - Schema icebergSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required(1, "int", Types.IntegerType.get()), - Types.NestedField.required(2, "string", Types.StringType.get())), - Sets.newHashSet(1, 2)); - - TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); - Assert.assertTrue(tableSchema.getPrimaryKey().isPresent()); - Assert.assertEquals( - ImmutableSet.of("int", "string"), - ImmutableSet.copyOf(tableSchema.getPrimaryKey().get().getColumns())); - } - - @Test - public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { - Schema icebergSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required( - 1, - "struct", - Types.StructType.of( - Types.NestedField.required(2, "inner", Types.IntegerType.get())))), - Sets.newHashSet(2)); - Assertions.assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) - .isInstanceOf(ValidationException.class) - .hasMessageStartingWith("Could not create a PRIMARY KEY") - .hasMessageContaining("Column 'struct.inner' does not exist"); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java deleted file mode 100644 index b7fce104f490..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ /dev/null @@ -1,358 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.Expressions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.source.BoundedTableFactory; -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.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -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; - - @Parameter(index = 2) - private FileFormat format; - - @Parameter(index = 3) - private boolean isStreamingJob; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") - public static List parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { - for (Boolean isStreaming : new Boolean[] {true, false}) { - for (Object[] catalogParams : CatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); - } - } - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreamingJob) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - } - } - return tEnv; - } - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - sql( - "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", - TABLE_NAME, format.name()); - icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - BoundedTableFactory.clearDataSets(); - super.clean(); - } - - @TestTemplate - public void testInsertFromSourceTable() throws Exception { - // Register the rows into a temporary table. - getTableEnv() - .createTemporaryView( - "sourceTable", - getTableEnv() - .fromValues( - SimpleDataUtil.FLINK_SCHEMA.toRowDataType(), - Expressions.row(1, "hello"), - Expressions.row(2, "world"), - Expressions.row(3, (String) null), - Expressions.row(null, "bar"))); - - // Redirect the records from source table to destination table. - sql("INSERT INTO %s SELECT id,data from sourceTable", TABLE_NAME); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), - SimpleDataUtil.createRecord(2, "world"), - SimpleDataUtil.createRecord(3, null), - SimpleDataUtil.createRecord(null, "bar"))); - } - - @TestTemplate - public void testOverwriteTable() throws Exception { - assumeThat(isStreamingJob) - .as("Flink unbounded streaming does not support overwrite operation") - .isFalse(); - - sql("INSERT INTO %s SELECT 1, 'a'", TABLE_NAME); - SimpleDataUtil.assertTableRecords( - icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(1, "a"))); - - sql("INSERT OVERWRITE %s SELECT 2, 'b'", TABLE_NAME); - SimpleDataUtil.assertTableRecords( - icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); - } - - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - 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 /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, 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)); - } - - @TestTemplate - public void testReplacePartitions() throws Exception { - assumeThat(isStreamingJob) - .as("Flink unbounded streaming does not support overwrite operation") - .isFalse(); - String tableName = "test_partition"; - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", - tableName, format.name()); - - try { - Table partitionedTable = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - - sql("INSERT INTO %s SELECT 1, 'a'", tableName); - sql("INSERT INTO %s SELECT 2, 'b'", tableName); - sql("INSERT INTO %s SELECT 3, 'c'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "b"), - SimpleDataUtil.createRecord(3, "c"))); - - sql("INSERT OVERWRITE %s SELECT 4, 'b'", tableName); - sql("INSERT OVERWRITE %s SELECT 5, 'a'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(5, "a"), - SimpleDataUtil.createRecord(4, "b"), - SimpleDataUtil.createRecord(3, "c"))); - - sql("INSERT OVERWRITE %s PARTITION (data='a') SELECT 6", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(6, "a"), - SimpleDataUtil.createRecord(4, "b"), - SimpleDataUtil.createRecord(3, "c"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testInsertIntoPartition() throws Exception { - String tableName = "test_insert_into_partition"; - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", - tableName, format.name()); - - try { - Table partitionedTable = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - - // Full partition. - sql("INSERT INTO %s PARTITION (data='a') SELECT 1", tableName); - sql("INSERT INTO %s PARTITION (data='a') SELECT 2", tableName); - sql("INSERT INTO %s PARTITION (data='b') SELECT 3", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "a"), - SimpleDataUtil.createRecord(3, "b"))); - - // Partial partition. - sql("INSERT INTO %s SELECT 4, 'c'", tableName); - sql("INSERT INTO %s SELECT 5, 'd'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "a"), - SimpleDataUtil.createRecord(3, "b"), - SimpleDataUtil.createRecord(4, "c"), - SimpleDataUtil.createRecord(5, "d"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java deleted file mode 100644 index 5674c83e40b8..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ /dev/null @@ -1,334 +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; - -import java.time.LocalDate; -import java.util.List; -import java.util.Map; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Namespace; -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.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkUpsert extends CatalogTestBase { - - @Parameter(index = 2) - private FileFormat format; - - @Parameter(index = 3) - private boolean isStreamingJob; - - private final Map tableUpsertProps = Maps.newHashMap(); - private TableEnvironment tEnv; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") - public static List parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.PARQUET, FileFormat.AVRO, FileFormat.ORC}) { - for (Boolean isStreaming : new Boolean[] {true, false}) { - // Only test with one catalog as this is a file operation concern. - // FlinkCatalogTestBase requires the catalog name start with testhadoop if using hadoop - // catalog. - String catalogName = "testhadoop"; - Namespace baseNamespace = Namespace.of("default"); - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); - } - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreamingJob) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - } - } - return tEnv; - } - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - tableUpsertProps.put(TableProperties.FORMAT_VERSION, "2"); - tableUpsertProps.put(TableProperties.UPSERT_ENABLED, "true"); - tableUpsertProps.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - } - - @Override - @AfterEach - public void clean() { - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testUpsertAndQuery() { - String tableName = "test_upsert_query"; - LocalDate dt20220301 = LocalDate.of(2022, 3, 1); - LocalDate dt20220302 = LocalDate.of(2022, 3, 2); - - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - try { - sql( - "INSERT INTO %s VALUES " - + "(1, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-01')," - + "(2, 'Jane', DATE '2022-03-01')", - tableName); - - sql( - "INSERT INTO %s VALUES " - + "(2, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-02')," - + "(2, 'Jane', DATE '2022-03-02')", - tableName); - - List rowsOn20220301 = - Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); - - List rowsOn20220302 = - Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testUpsertOptions() { - String tableName = "test_upsert_options"; - LocalDate dt20220301 = LocalDate.of(2022, 3, 1); - LocalDate dt20220302 = LocalDate.of(2022, 3, 2); - - Map optionsUpsertProps = Maps.newHashMap(tableUpsertProps); - optionsUpsertProps.remove(TableProperties.UPSERT_ENABLED); - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(optionsUpsertProps)); - - try { - sql( - "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " - + "(1, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-01')," - + "(2, 'Jane', DATE '2022-03-01')", - tableName); - - sql( - "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " - + "(2, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-02')," - + "(2, 'Jane', DATE '2022-03-02')", - tableName); - - List rowsOn20220301 = - Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); - - List rowsOn20220302 = - Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testPrimaryKeyEqualToPartitionKey() { - // This is an SQL based reproduction of TestFlinkIcebergSinkV2#testUpsertOnDataKey - String tableName = "upsert_on_id_key"; - try { - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, PRIMARY KEY(id) NOT ENFORCED) " - + "PARTITIONED BY (id) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(1, 'Jane')," + "(2, 'Bill')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, "Jane"), Row.of(2, "Bill"))); - - sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(2, 'Jane')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, "Bill"), Row.of(2, "Jane"))); - - sql("INSERT INTO %s VALUES " + "(3, 'Bill')," + "(4, 'Jane')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of(1, "Bill"), Row.of(2, "Jane"), Row.of(3, "Bill"), Row.of(4, "Jane"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testPrimaryKeyFieldsAtBeginningOfSchema() { - String tableName = "upsert_on_pk_at_schema_start"; - LocalDate dt = LocalDate.of(2022, 3, 1); - try { - sql( - "CREATE TABLE %s(id INT, dt DATE NOT NULL, name STRING NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql( - "INSERT INTO %s VALUES " - + "(1, DATE '2022-03-01', 'Andy')," - + "(1, DATE '2022-03-01', 'Bill')," - + "(2, DATE '2022-03-01', 'Jane')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, dt, "Bill"), Row.of(2, dt, "Jane"))); - - sql( - "INSERT INTO %s VALUES " - + "(1, DATE '2022-03-01', 'Jane')," - + "(2, DATE '2022-03-01', 'Bill')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, dt, "Jane"), Row.of(2, dt, "Bill"))); - - sql( - "INSERT INTO %s VALUES " - + "(3, DATE '2022-03-01', 'Duke')," - + "(4, DATE '2022-03-01', 'Leon')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of(1, dt, "Jane"), - Row.of(2, dt, "Bill"), - Row.of(3, dt, "Duke"), - Row.of(4, dt, "Leon"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testPrimaryKeyFieldsAtEndOfTableSchema() { - // This is the same test case as testPrimaryKeyFieldsAtBeginningOfSchema, but the primary key - // fields - // are located at the end of the flink schema. - String tableName = "upsert_on_pk_at_schema_end"; - LocalDate dt = LocalDate.of(2022, 3, 1); - try { - sql( - "CREATE TABLE %s(name STRING NOT NULL, id INT, dt DATE NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql( - "INSERT INTO %s VALUES " - + "('Andy', 1, DATE '2022-03-01')," - + "('Bill', 1, DATE '2022-03-01')," - + "('Jane', 2, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of("Bill", 1, dt), Row.of("Jane", 2, dt))); - - sql( - "INSERT INTO %s VALUES " - + "('Jane', 1, DATE '2022-03-01')," - + "('Bill', 2, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of("Jane", 1, dt), Row.of("Bill", 2, dt))); - - sql( - "INSERT INTO %s VALUES " - + "('Duke', 3, DATE '2022-03-01')," - + "('Leon', 4, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of("Jane", 1, dt), - Row.of("Bill", 2, dt), - Row.of("Duke", 3, dt), - Row.of("Leon", 4, dt))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java deleted file mode 100644 index 80e5ddd24fba..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ /dev/null @@ -1,628 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import org.apache.avro.generic.GenericData; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -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.ContentFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.flink.source.FlinkInputFormat; -import org.apache.iceberg.flink.source.FlinkInputSplit; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; - -public class TestHelpers { - private TestHelpers() {} - - public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { - KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - kryo.serialize(table, outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - return kryo.deserialize(inputView); - } - - public static RowData copyRowData(RowData from, RowType rowType) { - TypeSerializer[] fieldSerializers = - rowType.getChildren().stream() - .map((LogicalType type) -> InternalSerializers.create(type)) - .toArray(TypeSerializer[]::new); - return RowDataUtil.clone(from, null, rowType, fieldSerializers); - } - - public static void readRowData(FlinkInputFormat input, Consumer visitor) - throws IOException { - for (FlinkInputSplit s : input.createInputSplits(0)) { - input.open(s); - try { - while (!input.reachedEnd()) { - RowData row = input.nextRecord(null); - visitor.accept(row); - } - } finally { - input.close(); - } - } - } - - public static List readRowData(FlinkInputFormat inputFormat, RowType rowType) - throws IOException { - List results = Lists.newArrayList(); - readRowData(inputFormat, row -> results.add(copyRowData(row, rowType))); - return results; - } - - public static List readRows(FlinkInputFormat inputFormat, RowType rowType) - throws IOException { - return convertRowDataToRow(readRowData(inputFormat, rowType), rowType); - } - - public static List convertRowDataToRow(List rowDataList, RowType rowType) { - DataStructureConverter converter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - return rowDataList.stream() - .map(converter::toExternal) - .map(Row.class::cast) - .collect(Collectors.toList()); - } - - private static List convertRecordToRow(List expectedRecords, Schema schema) { - List expected = Lists.newArrayList(); - @SuppressWarnings("unchecked") - DataStructureConverter converter = - (DataStructureConverter) - DataStructureConverters.getConverter( - TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(schema))); - expectedRecords.forEach( - r -> expected.add(converter.toExternal(RowDataConverter.convert(schema, r)))); - return expected; - } - - public static void assertRecordsWithOrder( - List results, List expectedRecords, Schema schema) { - List expected = convertRecordToRow(expectedRecords, schema); - assertRowsWithOrder(results, expected); - } - - public static void assertRecords(List results, List expectedRecords, Schema schema) { - List expected = convertRecordToRow(expectedRecords, schema); - assertRows(results, expected); - } - - public static void assertRows(List results, List expected, RowType rowType) { - assertRows(convertRowDataToRow(results, rowType), convertRowDataToRow(expected, rowType)); - } - - public static void assertRows(List results, List expected) { - assertThat(results).containsExactlyInAnyOrderElementsOf(expected); - } - - public static void assertRowsWithOrder(List results, List expected) { - Assertions.assertThat(results).containsExactlyElementsOf(expected); - } - - public static void assertRowData(Schema schema, StructLike expected, RowData actual) { - assertRowData(schema.asStruct(), FlinkSchemaUtil.convert(schema), expected, actual); - } - - public static void assertRowData( - Types.StructType structType, - LogicalType rowType, - StructLike expectedRecord, - RowData actualRowData) { - if (expectedRecord == null && actualRowData == null) { - return; - } - - assertThat(expectedRecord).isNotNull(); - assertThat(actualRowData).isNotNull(); - - List types = Lists.newArrayList(); - for (Types.NestedField field : structType.fields()) { - types.add(field.type()); - } - - for (int i = 0; i < types.size(); i += 1) { - LogicalType logicalType = ((RowType) rowType).getTypeAt(i); - Object expected = expectedRecord.get(i, Object.class); - // The RowData.createFieldGetter won't return null for the required field. But in the - // projection case, if we are - // projecting a nested required field from an optional struct, then we should give a null for - // the projected field - // if the outer struct value is null. So we need to check the nullable for actualRowData here. - // For more details - // please see issue #2738. - Object actual = - actualRowData.isNullAt(i) - ? null - : RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData); - assertEquals(types.get(i), logicalType, expected, actual); - } - } - - private static void assertEquals( - Type type, LogicalType logicalType, Object expected, Object actual) { - - if (expected == null && actual == null) { - return; - } - - assertThat(expected).isNotNull(); - assertThat(actual).isNotNull(); - - switch (type.typeId()) { - case BOOLEAN: - assertThat(actual).as("boolean value should be equal").isEqualTo(expected); - break; - case INTEGER: - assertThat(actual).as("int value should be equal").isEqualTo(expected); - break; - case LONG: - assertThat(actual).as("long value should be equal").isEqualTo(expected); - break; - case FLOAT: - assertThat(actual).as("float value should be equal").isEqualTo(expected); - break; - case DOUBLE: - assertThat(actual).as("double value should be equal").isEqualTo(expected); - break; - case STRING: - assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); - assertThat(actual.toString()) - .as("string should be equal") - .isEqualTo(String.valueOf(expected)); - break; - case DATE: - assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); - LocalDate date = DateTimeUtil.dateFromDays((int) actual); - assertThat(date).as("date should be equal").isEqualTo(expected); - break; - case TIME: - assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); - int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); - assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); - break; - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - assertThat(expected) - .as("Should expect a OffsetDataTime") - .isInstanceOf(OffsetDateTime.class); - OffsetDateTime ts = (OffsetDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("OffsetDataTime should be equal") - .isEqualTo(ts.toLocalDateTime()); - } else { - assertThat(expected) - .as("Should expect a LocalDataTime") - .isInstanceOf(LocalDateTime.class); - LocalDateTime ts = (LocalDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("LocalDataTime should be equal") - .isEqualTo(ts); - } - break; - case BINARY: - assertThat(ByteBuffer.wrap((byte[]) actual)) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class) - .isEqualTo(expected); - break; - case DECIMAL: - assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); - BigDecimal bd = (BigDecimal) expected; - assertThat(((DecimalData) actual).toBigDecimal()) - .as("decimal value should be equal") - .isEqualTo(bd); - break; - case LIST: - assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); - Collection expectedArrayData = (Collection) expected; - ArrayData actualArrayData = (ArrayData) actual; - LogicalType elementType = ((ArrayType) logicalType).getElementType(); - assertThat(actualArrayData.size()) - .as("array length should be equal") - .isEqualTo(expectedArrayData.size()); - assertArrayValues( - type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); - break; - case MAP: - assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - assertMapValues(type.asMapType(), logicalType, (Map) expected, (MapData) actual); - break; - case STRUCT: - assertThat(expected).as("Should expect a Record").isInstanceOf(StructLike.class); - assertRowData(type.asStructType(), logicalType, (StructLike) expected, (RowData) actual); - break; - case UUID: - assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); - long firstLong = bb.getLong(); - long secondLong = bb.getLong(); - assertThat(new UUID(firstLong, secondLong).toString()) - .as("UUID should be equal") - .isEqualTo(expected.toString()); - break; - case FIXED: - assertThat(actual) - .as("Should expect byte[]") - .isInstanceOf(byte[].class) - .isEqualTo(expected); - break; - default: - throw new IllegalArgumentException("Not a supported type: " + type); - } - } - - public static void assertEquals(Schema schema, List records, List rows) { - Streams.forEachPair( - records.stream(), rows.stream(), (record, row) -> assertEquals(schema, record, row)); - } - - public static void assertEquals(Schema schema, GenericData.Record record, Row row) { - List fields = schema.asStruct().fields(); - assertThat(fields).hasSameSizeAs(record.getSchema().getFields()); - assertThat(fields).hasSize(row.getArity()); - - RowType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < fields.size(); ++i) { - Type fieldType = fields.get(i).type(); - Object expectedValue = record.get(i); - Object actualValue = row.getField(i); - LogicalType logicalType = rowType.getTypeAt(i); - assertAvroEquals(fieldType, logicalType, expectedValue, actualValue); - } - } - - private static void assertEquals(Types.StructType struct, GenericData.Record record, Row row) { - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); - Object expectedValue = record.get(i); - Object actualValue = row.getField(i); - assertAvroEquals(fieldType, null, expectedValue, actualValue); - } - } - - private static void assertAvroEquals( - Type type, LogicalType logicalType, Object expected, Object actual) { - - if (expected == null && actual == null) { - return; - } - assertThat(expected).isNotNull(); - assertThat(actual).isNotNull(); - - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - assertThat(expected) - .as("Should expect a " + type.typeId().javaClass()) - .isInstanceOf(type.typeId().javaClass()); - assertThat(actual) - .as("Should expect a " + type.typeId().javaClass()) - .isInstanceOf(type.typeId().javaClass()); - assertThat(actual).as(type.typeId() + " value should be equal").isEqualTo(expected); - break; - case STRING: - assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); - assertThat(actual).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); - assertThat(actual.toString()).as("string should be equal").isEqualTo(expected.toString()); - break; - case DATE: - assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); - LocalDate date = DateTimeUtil.dateFromDays((int) actual); - assertThat(date).as("date should be equal").isEqualTo(expected); - break; - case TIME: - assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); - int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); - assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); - break; - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - assertThat(expected) - .as("Should expect a OffsetDataTime") - .isInstanceOf(OffsetDateTime.class); - OffsetDateTime ts = (OffsetDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("OffsetDataTime should be equal") - .isEqualTo(ts.toLocalDateTime()); - } else { - assertThat(expected) - .as("Should expect a LocalDataTime") - .isInstanceOf(LocalDateTime.class); - LocalDateTime ts = (LocalDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("LocalDataTime should be equal") - .isEqualTo(ts); - } - break; - case BINARY: - assertThat(ByteBuffer.wrap((byte[]) actual)) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class) - .isEqualTo(expected); - break; - case DECIMAL: - assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); - BigDecimal bd = (BigDecimal) expected; - assertThat(((DecimalData) actual).toBigDecimal()) - .as("decimal value should be equal") - .isEqualTo(bd); - break; - case LIST: - assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); - Collection expectedArrayData = (Collection) expected; - ArrayData actualArrayData; - try { - actualArrayData = (ArrayData) actual; - } catch (ClassCastException e) { - actualArrayData = new GenericArrayData((Object[]) actual); - } - LogicalType elementType = ((ArrayType) logicalType).getElementType(); - assertThat(actualArrayData.size()) - .as("array length should be equal") - .isEqualTo(expectedArrayData.size()); - assertArrayValues( - type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); - break; - case MAP: - assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - MapData actualMap; - try { - actualMap = (MapData) actual; - } catch (ClassCastException e) { - actualMap = new GenericMapData((Map) actual); - } - assertMapValues(type.asMapType(), logicalType, (Map) expected, actualMap); - break; - case STRUCT: - assertThat(expected).as("Should expect a Record").isInstanceOf(GenericData.Record.class); - assertEquals( - type.asNestedType().asStructType(), (GenericData.Record) expected, (Row) actual); - break; - case UUID: - assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); - long firstLong = bb.getLong(); - long secondLong = bb.getLong(); - assertThat(new UUID(firstLong, secondLong).toString()) - .as("UUID should be equal") - .isEqualTo(expected.toString()); - break; - case FIXED: - assertThat(actual) - .as("Should expect byte[]") - .isInstanceOf(byte[].class) - .isEqualTo(expected); - break; - default: - throw new IllegalArgumentException("Not a supported type: " + type); - } - } - - private static void assertArrayValues( - Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { - List expectedElements = Lists.newArrayList(expectedArray); - for (int i = 0; i < expectedArray.size(); i += 1) { - if (expectedElements.get(i) == null) { - assertThat(actualArray.isNullAt(i)).isTrue(); - continue; - } - - Object expected = expectedElements.get(i); - - assertEquals( - type, - logicalType, - expected, - ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, i)); - } - } - - private static void assertMapValues( - Types.MapType mapType, LogicalType type, Map expected, MapData actual) { - assertThat(actual.size()).as("map size should be equal").isEqualTo(expected.size()); - - ArrayData actualKeyArrayData = actual.keyArray(); - ArrayData actualValueArrayData = actual.valueArray(); - LogicalType actualKeyType = ((MapType) type).getKeyType(); - LogicalType actualValueType = ((MapType) type).getValueType(); - Type keyType = mapType.keyType(); - Type valueType = mapType.valueType(); - - ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(actualKeyType); - ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(actualValueType); - - for (Map.Entry entry : expected.entrySet()) { - Object matchedActualKey = null; - int matchedKeyIndex = 0; - for (int i = 0; i < actual.size(); i += 1) { - try { - Object key = keyGetter.getElementOrNull(actualKeyArrayData, i); - assertEquals(keyType, actualKeyType, entry.getKey(), key); - matchedActualKey = key; - matchedKeyIndex = i; - break; - } catch (AssertionError e) { - // not found - } - } - assertThat(matchedActualKey).as("Should have a matching key").isNotNull(); - final int valueIndex = matchedKeyIndex; - assertEquals( - valueType, - actualValueType, - entry.getValue(), - valueGetter.getElementOrNull(actualValueArrayData, valueIndex)); - } - } - - public static void assertEquals(ManifestFile expected, ManifestFile actual) { - if (expected == actual) { - return; - } - assertThat(expected).isNotNull(); - assertThat(actual).isNotNull(); - assertThat(actual.path()).as("Path must match").isEqualTo(expected.path()); - assertThat(actual.length()).as("Length must match").isEqualTo(expected.length()); - assertThat(actual.partitionSpecId()) - .as("Spec id must match") - .isEqualTo(expected.partitionSpecId()); - assertThat(actual.content()).as("ManifestContent must match").isEqualTo(expected.content()); - assertThat(actual.sequenceNumber()) - .as("SequenceNumber must match") - .isEqualTo(expected.sequenceNumber()); - assertThat(actual.minSequenceNumber()) - .as("MinSequenceNumber must match") - .isEqualTo(expected.minSequenceNumber()); - assertThat(actual.snapshotId()).as("Snapshot id must match").isEqualTo(expected.snapshotId()); - assertThat(actual.hasAddedFiles()) - .as("Added files flag must match") - .isEqualTo(expected.hasAddedFiles()); - assertThat(actual.addedFilesCount()) - .as("Added files count must match") - .isEqualTo(expected.addedFilesCount()); - assertThat(actual.addedRowsCount()) - .as("Added rows count must match") - .isEqualTo(expected.addedRowsCount()); - assertThat(actual.hasExistingFiles()) - .as("Existing files flag must match") - .isEqualTo(expected.hasExistingFiles()); - assertThat(actual.existingFilesCount()) - .as("Existing files count must match") - .isEqualTo(expected.existingFilesCount()); - assertThat(actual.existingRowsCount()) - .as("Existing rows count must match") - .isEqualTo(expected.existingRowsCount()); - assertThat(actual.hasDeletedFiles()) - .as("Deleted files flag must match") - .isEqualTo(expected.hasDeletedFiles()); - assertThat(actual.deletedFilesCount()) - .as("Deleted files count must match") - .isEqualTo(expected.deletedFilesCount()); - assertThat(actual.deletedRowsCount()) - .as("Deleted rows count must match") - .isEqualTo(expected.deletedRowsCount()); - - List expectedSummaries = expected.partitions(); - List actualSummaries = actual.partitions(); - assertThat(actualSummaries) - .as("PartitionFieldSummary size does not match") - .hasSameSizeAs(expectedSummaries); - for (int i = 0; i < expectedSummaries.size(); i++) { - assertThat(actualSummaries.get(i).containsNull()) - .as("Null flag in partition must match") - .isEqualTo(expectedSummaries.get(i).containsNull()); - assertThat(actualSummaries.get(i).containsNaN()) - .as("NaN flag in partition must match") - .isEqualTo(expectedSummaries.get(i).containsNaN()); - assertThat(actualSummaries.get(i).lowerBound()) - .as("Lower bounds in partition must match") - .isEqualTo(expectedSummaries.get(i).lowerBound()); - assertThat(actualSummaries.get(i).upperBound()) - .as("Upper bounds in partition must match") - .isEqualTo(expectedSummaries.get(i).upperBound()); - } - } - - public static void assertEquals(ContentFile expected, ContentFile actual) { - if (expected == actual) { - return; - } - assertThat(expected).isNotNull(); - 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.format()).as("Format").isEqualTo(expected.format()); - assertThat(actual.partition().size()) - .as("Partition size") - .isEqualTo(expected.partition().size()); - for (int i = 0; i < expected.partition().size(); i++) { - assertThat(actual.partition().get(i, Object.class)) - .as("Partition data at index " + i) - .isEqualTo(expected.partition().get(i, Object.class)); - } - assertThat(actual.recordCount()).as("Record count").isEqualTo(expected.recordCount()); - assertThat(actual.fileSizeInBytes()) - .as("File size in bytes") - .isEqualTo(expected.fileSizeInBytes()); - assertThat(actual.columnSizes()).as("Column sizes").isEqualTo(expected.columnSizes()); - assertThat(actual.valueCounts()).as("Value counts").isEqualTo(expected.valueCounts()); - assertThat(actual.nullValueCounts()) - .as("Null value counts") - .isEqualTo(expected.nullValueCounts()); - assertThat(actual.lowerBounds()).as("Lower bounds").isEqualTo(expected.lowerBounds()); - assertThat(actual.upperBounds()).as("Upper bounds").isEqualTo(expected.upperBounds()); - assertThat(actual.keyMetadata()).as("Key metadata").isEqualTo(expected.keyMetadata()); - assertThat(actual.splitOffsets()).as("Split offsets").isEqualTo(expected.splitOffsets()); - assertThat(actual.equalityFieldIds()) - .as("Equality field id list") - .isEqualTo(expected.equalityFieldIds()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java deleted file mode 100644 index cb409b784383..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ /dev/null @@ -1,350 +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; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.types.Row; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.iceberg.CatalogProperties; -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.relocated.com.google.common.collect.Sets; -import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergConnector extends FlinkTestBase { - - private static final String TABLE_NAME = "test_table"; - - @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); - - private final String catalogName; - private final Map properties; - private final boolean isStreaming; - private volatile TableEnvironment tEnv; - - @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") - public static Iterable parameters() { - return Lists.newArrayList( - // Create iceberg table in the hadoop catalog and default database. - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop"), - false - }, - // Create iceberg table in the hadoop catalog and not_existing_db. - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db"), - false - }, - // Create iceberg table in the hive catalog and default database. - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive"), - false - }, - // Create iceberg table in the hive catalog and not_existing_db. - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db"), - false - }); - } - - public TestIcebergConnector( - String catalogName, Map properties, boolean isStreaming) { - this.catalogName = catalogName; - this.properties = properties; - this.isStreaming = isStreaming; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreaming) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - // Set only one parallelism. - tEnv.getConfig() - .getConfiguration() - .set(CoreOptions.DEFAULT_PARALLELISM, 1) - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - } - } - } - return tEnv; - } - - @After - public void after() throws TException { - sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - - // Clean the created orphan databases and tables from hive-metastore. - if (isHiveCatalog()) { - HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf); - try { - metaStoreClient.dropTable(databaseName(), tableName()); - if (!isDefaultDatabaseName()) { - try { - metaStoreClient.dropDatabase(databaseName()); - } catch (Exception ignored) { - // Ignore - } - } - } finally { - metaStoreClient.close(); - } - } - } - - private void testCreateConnectorTable() { - Map tableProps = createTableProps(); - - // Create table under the flink's current database. - sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); - - FlinkCatalogFactory factory = new FlinkCatalogFactory(); - Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); - Assert.assertTrue( - "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); - Assert.assertTrue( - "Should have created the expected table", - flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); - - // Drop and create it again. - sql("DROP TABLE %s", TABLE_NAME); - sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); - } - - @Test - public void testCreateTableUnderDefaultDatabase() { - testCreateConnectorTable(); - } - - @Test - public void testCatalogDatabaseConflictWithFlinkDatabase() { - sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); - sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - Assertions.assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } - } - - @Test - public void testConnectorTableInIcebergCatalog() { - // Create the catalog properties - Map catalogProps = Maps.newHashMap(); - catalogProps.put("type", "iceberg"); - if (isHiveCatalog()) { - catalogProps.put("catalog-type", "hive"); - catalogProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - } else { - catalogProps.put("catalog-type", "hadoop"); - } - catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); - - // Create the table properties - Map tableProps = createTableProps(); - - // Create a connector table in an iceberg catalog. - sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); - try { - Assertions.assertThatThrownBy( - () -> - sql( - "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", - FlinkCatalogFactory.DEFAULT_DATABASE_NAME, - TABLE_NAME, - toWithClause(tableProps))) - .cause() - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " - + "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " - + "create table without 'connector'='iceberg' related properties in an iceberg table."); - } finally { - sql("DROP CATALOG IF EXISTS `test_catalog`"); - } - } - - private Map createTableProps() { - Map tableProps = Maps.newHashMap(properties); - tableProps.put("catalog-name", catalogName); - tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); - if (isHiveCatalog()) { - tableProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - } - return tableProps; - } - - private boolean isHiveCatalog() { - return "testhive".equalsIgnoreCase(catalogName); - } - - private boolean isDefaultDatabaseName() { - return FlinkCatalogFactory.DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName()); - } - - private String tableName() { - return properties.getOrDefault("catalog-table", TABLE_NAME); - } - - private String databaseName() { - return properties.getOrDefault("catalog-database", "default_database"); - } - - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - - private static String createWarehouse() { - try { - return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java deleted file mode 100644 index 6bd94e9ca61c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ /dev/null @@ -1,174 +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; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.GenericManifestFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestManifestFileSerialization { - - private static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - required(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("double").build(); - - private static final DataFile FILE_A = - DataFiles.builder(SPEC) - .withPath("/path/to/data-1.parquet") - .withFileSizeInBytes(0) - .withPartition(org.apache.iceberg.TestHelpers.Row.of(1D)) - .withPartitionPath("double=1") - .withMetrics( - new Metrics( - 5L, - null, // no column sizes - ImmutableMap.of(1, 5L, 2, 3L), // value count - ImmutableMap.of(1, 0L, 2, 2L), // null count - ImmutableMap.of(), // nan count - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(4L)) // upper bounds - )) - .build(); - - private static final DataFile FILE_B = - DataFiles.builder(SPEC) - .withPath("/path/to/data-2.parquet") - .withFileSizeInBytes(0) - .withPartition(org.apache.iceberg.TestHelpers.Row.of(Double.NaN)) - .withPartitionPath("double=NaN") - .withMetrics( - new Metrics( - 1L, - null, // no column sizes - ImmutableMap.of(1, 1L, 4, 1L), // value count - ImmutableMap.of(1, 0L, 2, 0L), // null count - ImmutableMap.of(4, 1L), // nan count - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(1L)) // upper bounds - )) - .build(); - - private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - @Test - public void testKryoSerialization() throws IOException { - KryoSerializer kryo = - new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - ManifestFile manifest = writeManifest(FILE_A, FILE_B); - - kryo.serialize(manifest, outputView); - kryo.serialize(manifest.copy(), outputView); - kryo.serialize(GenericManifestFile.copyOf(manifest).build(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - ManifestFile m1 = kryo.deserialize(inputView); - ManifestFile m2 = kryo.deserialize(inputView); - ManifestFile m3 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(manifest, m1); - TestHelpers.assertEquals(manifest, m2); - TestHelpers.assertEquals(manifest, m3); - } - - @Test - public void testJavaSerialization() throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - - ManifestFile manifest = writeManifest(FILE_A, FILE_B); - - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(manifest); - out.writeObject(manifest.copy()); - out.writeObject(GenericManifestFile.copyOf(manifest).build()); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - for (int i = 0; i < 3; i += 1) { - Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); - TestHelpers.assertEquals(manifest, (ManifestFile) obj); - } - } - } - - private ManifestFile writeManifest(DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); - Assert.assertTrue(manifestFile.delete()); - OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - private static ByteBuffer longToBuffer(long value) { - return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java deleted file mode 100644 index c78fa51215dd..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ /dev/null @@ -1,93 +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; - -import java.util.Iterator; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.RecordWrapperTest; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.util.StructLikeWrapper; -import org.assertj.core.api.Assertions; -import org.junit.Assert; - -public class TestRowDataWrapper extends RecordWrapperTest { - - /** - * Flink's time type has been truncated to millis seconds, so we need a customized assert method - * to check the values. - */ - @Override - public void testTime() { - generateAndValidate( - new Schema(TIME.fields()), - (message, expectedWrapper, actualWrapper) -> { - for (int pos = 0; pos < TIME.fields().size(); pos++) { - Object expected = expectedWrapper.get().get(pos, Object.class); - Object actual = actualWrapper.get().get(pos, Object.class); - if (expected == actual) { - return; - } - - Assertions.assertThat(actual).isNotNull(); - Assertions.assertThat(expected).isNotNull(); - - int expectedMilliseconds = (int) ((long) expected / 1000_000); - int actualMilliseconds = (int) ((long) actual / 1000_000); - Assert.assertEquals(message, expectedMilliseconds, actualMilliseconds); - } - }); - } - - @Override - protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { - int numRecords = 100; - Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); - Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); - - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); - RowDataWrapper rowDataWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - - Iterator actual = recordList.iterator(); - Iterator expected = rowDataList.iterator(); - - StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); - StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); - for (int i = 0; i < numRecords; i++) { - Assert.assertTrue("Should have more records", actual.hasNext()); - Assert.assertTrue("Should have more RowData", expected.hasNext()); - - StructLike recordStructLike = recordWrapper.wrap(actual.next()); - StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); - - assertMethod.assertEquals( - "Should have expected StructLike values", - actualWrapper.set(recordStructLike), - expectedWrapper.set(rowDataStructLike)); - } - - Assert.assertFalse("Shouldn't have more record", actual.hasNext()); - Assert.assertFalse("Shouldn't have more RowData", expected.hasNext()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java deleted file mode 100644 index 4ad302dde436..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import java.io.File; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestTables; - -public class TestTableLoader implements TableLoader { - private File dir; - - public static TableLoader of(String dir) { - return new TestTableLoader(dir); - } - - public TestTableLoader(String dir) { - this.dir = new File(dir); - } - - @Override - public void open() {} - - @Override - public boolean isOpen() { - return true; - } - - @Override - public Table loadTable() { - return TestTables.load(dir, "test"); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new TestTableLoader(dir.getAbsolutePath()); - } - - @Override - public void close() {} -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java deleted file mode 100644 index 27124d93fef4..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java +++ /dev/null @@ -1,110 +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; - -import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.util.Map; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestTableSerialization { - private static final HadoopTables TABLES = new HadoopTables(); - - private static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - optional(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("date").build(); - - private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - private Table table; - - @Before - public void initTable() throws IOException { - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - - File tableLocation = temp.newFolder(); - Assert.assertTrue(tableLocation.delete()); - - this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); - } - - @Test - public void testSerializableTableKryoSerialization() throws IOException { - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - TestHelpers.assertSerializedAndLoadedMetadata( - table, roundTripKryoSerialize(SerializableTable.class, serializableTable)); - } - - @Test - public void testSerializableMetadataTableKryoSerialization() throws IOException { - for (MetadataTableType type : MetadataTableType.values()) { - TableOperations ops = ((HasTableOperations) table).operations(); - Table metadataTable = - MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); - SerializableTable serializableMetadataTable = - (SerializableTable) SerializableTable.copyOf(metadataTable); - - TestHelpers.assertSerializedAndLoadedMetadata( - metadataTable, - roundTripKryoSerialize(SerializableTable.class, serializableMetadataTable)); - } - } - - @Test - public void testSerializableTransactionTableKryoSerialization() throws IOException { - Transaction txn = table.newTransaction(); - - txn.updateProperties().set("k1", "v1").commit(); - - Table txnTable = txn.table(); - SerializableTable serializableTxnTable = (SerializableTable) SerializableTable.copyOf(txnTable); - - TestHelpers.assertSerializedMetadata( - txnTable, roundTripKryoSerialize(SerializableTable.class, serializableTxnTable)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java deleted file mode 100644 index 4220775f41fa..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ /dev/null @@ -1,481 +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.actions; - -import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.util.Collection; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Files; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.RewriteDataFilesActionResult; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -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.types.Types; -import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -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; - -public class TestRewriteDataFilesAction extends CatalogTestBase { - - private static final String TABLE_NAME_UNPARTITIONED = "test_table_unpartitioned"; - private static final String TABLE_NAME_PARTITIONED = "test_table_partitioned"; - private static final String TABLE_NAME_WITH_PK = "test_table_with_pk"; - - @Parameter(index = 2) - private FileFormat format; - - private Table icebergTableUnPartitioned; - private Table icebergTablePartitioned; - private Table icebergTableWithPk; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") - 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}); - } - } - return parameters; - } - - private @TempDir Path temp; - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - 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()); - 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()); - 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()); - icebergTableWithPk = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testRewriteDataFilesEmptyTable() throws Exception { - assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); - Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); - assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); - } - - @TestTemplate - public void testRewriteDataFilesUnpartitionedTable() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(2); - RewriteDataFilesActionResult result = - Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); - - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFiles1).hasSize(1); - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTableUnPartitioned, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), SimpleDataUtil.createRecord(2, "world"))); - } - - @TestTemplate - public void testRewriteDataFilesPartitionedTable() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 3, 'world' ,'b'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(4); - RewriteDataFilesActionResult result = - Actions.forTable(icebergTablePartitioned).rewriteDataFiles().execute(); - - assertThat(result.deletedDataFiles()).hasSize(4); - assertThat(result.addedDataFiles()).hasSize(2); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFiles1).hasSize(2); - // Assert the table records as expected. - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "spec", Types.StringType.get())); - - Record record = GenericRecord.create(schema); - SimpleDataUtil.assertTableRecords( - icebergTablePartitioned, - Lists.newArrayList( - record.copy("id", 1, "data", "hello", "spec", "a"), - record.copy("id", 2, "data", "hello", "spec", "a"), - record.copy("id", 3, "data", "world", "spec", "b"), - record.copy("id", 4, "data", "world", "spec", "b"))); - } - - @TestTemplate - public void testRewriteDataFilesWithFilter() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 3, 'world' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 5, 'world' ,'b'", TABLE_NAME_PARTITIONED); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(5); - RewriteDataFilesActionResult result = - Actions.forTable(icebergTablePartitioned) - .rewriteDataFiles() - .filter(Expressions.equal("spec", "a")) - .filter(Expressions.startsWith("data", "he")) - .execute(); - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFiles1).hasSize(4); - // Assert the table records as expected. - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "spec", Types.StringType.get())); - - Record record = GenericRecord.create(schema); - SimpleDataUtil.assertTableRecords( - icebergTablePartitioned, - Lists.newArrayList( - record.copy("id", 1, "data", "hello", "spec", "a"), - record.copy("id", 2, "data", "hello", "spec", "a"), - record.copy("id", 3, "data", "world", "spec", "a"), - record.copy("id", 4, "data", "world", "spec", "b"), - record.copy("id", 5, "data", "world", "spec", "b"))); - } - - @TestTemplate - public void testRewriteLargeTableHasResiduals() throws IOException { - // all records belong to the same partition - List records1 = Lists.newArrayList(); - List records2 = Lists.newArrayList(); - List expected = Lists.newArrayList(); - for (int i = 0; i < 100; i++) { - int id = i; - String data = String.valueOf(i % 3); - if (i % 2 == 0) { - records1.add("(" + id + ",'" + data + "')"); - } else { - records2.add("(" + id + ",'" + data + "')"); - } - Record record = RECORD.copy(); - record.setField("id", id); - record.setField("data", data); - expected.add(record); - } - - sql("INSERT INTO %s values " + StringUtils.join(records1, ","), TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s values " + StringUtils.join(records2, ","), TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = - icebergTableUnPartitioned - .newScan() - .ignoreResiduals() - .filter(Expressions.equal("data", "0")) - .planFiles(); - for (FileScanTask task : tasks) { - assertThat(task.residual()) - .as("Residuals must be ignored") - .isEqualTo(Expressions.alwaysTrue()); - } - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(2); - Actions actions = Actions.forTable(icebergTableUnPartitioned); - - RewriteDataFilesActionResult result = - actions.rewriteDataFiles().filter(Expressions.equal("data", "0")).execute(); - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); - } - - /** - * a test case to test avoid repeate compress - * - *

    If datafile cannot be combined to CombinedScanTask with other DataFiles, the size of the - * CombinedScanTask list size is 1, so we remove these CombinedScanTasks to avoid compressed - * repeatedly. - * - *

    In this test case,we generated 3 data files and set targetSizeInBytes greater than the - * largest file size so that it cannot be combined a CombinedScanTask with other datafiles. The - * datafile with the largest file size will not be compressed. - * - * @throws IOException IOException - */ - @TestTemplate - public void testRewriteAvoidRepeateCompress() throws IOException { - List expected = Lists.newArrayList(); - Schema schema = icebergTableUnPartitioned.schema(); - GenericAppenderFactory genericAppenderFactory = new GenericAppenderFactory(schema); - File file = File.createTempFile("junit", null, temp.toFile()); - int count = 0; - try (FileAppender fileAppender = - genericAppenderFactory.newAppender(Files.localOutput(file), format)) { - long filesize = 20000; - for (; fileAppender.length() < filesize; count++) { - Record record = SimpleDataUtil.createRecord(count, UUID.randomUUID().toString()); - fileAppender.add(record); - expected.add(record); - } - } - - DataFile dataFile = - DataFiles.builder(icebergTableUnPartitioned.spec()) - .withPath(file.getAbsolutePath()) - .withFileSizeInBytes(file.length()) - .withFormat(format) - .withRecordCount(count) - .build(); - - icebergTableUnPartitioned.newAppend().appendFile(dataFile).commit(); - - sql("INSERT INTO %s SELECT 1,'a' ", TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s SELECT 2,'b' ", TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(3); - Actions actions = Actions.forTable(icebergTableUnPartitioned); - - long targetSizeInBytes = file.length() + 10; - RewriteDataFilesActionResult result = - actions - .rewriteDataFiles() - .targetSizeInBytes(targetSizeInBytes) - .splitOpenFileCost(1) - .execute(); - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); - List dataFilesRewrote = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFilesRewrote).hasSize(2); - // the biggest file do not be rewrote - List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); - assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); - - // Assert the table records as expected. - expected.add(SimpleDataUtil.createRecord(1, "a")); - expected.add(SimpleDataUtil.createRecord(2, "b")); - SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); - } - - @TestTemplate - public void testRewriteNoConflictWithEqualityDeletes() throws IOException { - // Add 2 data files - sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_WITH_PK); - sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_WITH_PK); - - // Load 2 stale tables to pass to rewrite actions - // Since the first rewrite will refresh stale1, we need another stale2 for the second rewrite - Table stale1 = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - Table stale2 = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - - // Add 1 data file and 1 equality-delete file - sql("INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ SELECT 1, 'hi'", TABLE_NAME_WITH_PK); - - icebergTableWithPk.refresh(); - assertThat(icebergTableWithPk.currentSnapshot().sequenceNumber()) - .as("The latest sequence number should be greater than that of the stale snapshot") - .isEqualTo(stale1.currentSnapshot().sequenceNumber() + 1); - CloseableIterable tasks = icebergTableWithPk.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Set deleteFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::deletes)).stream() - .flatMap(Collection::stream) - .collect(Collectors.toSet()); - assertThat(dataFiles).hasSize(3); - assertThat(deleteFiles).hasSize(1); - assertThat(Iterables.getOnlyElement(deleteFiles).content()) - .isEqualTo(FileContent.EQUALITY_DELETES); - shouldHaveDataAndFileSequenceNumbers( - TABLE_NAME_WITH_PK, - ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); - - Assertions.assertThatThrownBy( - () -> - Actions.forTable(stale1) - .rewriteDataFiles() - .useStartingSequenceNumber(false) - .execute(), - "Rewrite using new sequence number should fail") - .isInstanceOf(ValidationException.class); - - // Rewrite using the starting sequence number should succeed - RewriteDataFilesActionResult result = - Actions.forTable(stale2).rewriteDataFiles().useStartingSequenceNumber(true).execute(); - - // Should not rewrite files from the new commit - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - // The 2 older files with file-sequence-number <= 2 should be rewritten into a new file. - // The new file is the one with file-sequence-number == 4. - // The new file should use rewrite's starting-sequence-number 2 as its data-sequence-number. - shouldHaveDataAndFileSequenceNumbers( - TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(3L, 3L), Pair.of(3L, 3L), Pair.of(2L, 4L))); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTableWithPk, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hi"), SimpleDataUtil.createRecord(2, "world"))); - } - - /** - * Assert that data files and delete files in the table should have expected data sequence numbers - * and file sequence numbers - * - * @param tableName table name - * @param expectedSequenceNumbers list of {@link Pair}'s. Each {@link Pair} contains - * (expectedDataSequenceNumber, expectedFileSequenceNumber) of a file. - */ - private void shouldHaveDataAndFileSequenceNumbers( - String tableName, List> expectedSequenceNumbers) { - // "status < 2" for added or existing entries - List liveEntries = sql("SELECT * FROM %s$entries WHERE status < 2", tableName); - - List> actualSequenceNumbers = - liveEntries.stream() - .map( - row -> - Pair.of( - row.getFieldAs("sequence_number"), row.getFieldAs("file_sequence_number"))) - .collect(Collectors.toList()); - assertThat(actualSequenceNumbers).hasSameElementsAs(expectedSequenceNumbers); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java deleted file mode 100644 index cc58d9817ac6..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.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 org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; - -public class RandomRowData { - private RandomRowData() {} - - public static Iterable generate(Schema schema, int numRecords, long seed) { - return convert(schema, RandomGenericData.generate(schema, numRecords, seed)); - } - - public static Iterable convert(Schema schema, Iterable records) { - return Iterables.transform(records, record -> RowDataConverter.convert(schema, record)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java deleted file mode 100644 index 74b1da6007e6..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java +++ /dev/null @@ -1,50 +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 org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; -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; - -public class RowDataToRowMapper extends RichMapFunction { - - private final RowType rowType; - - private transient DataStructureConverter converter; - - public RowDataToRowMapper(RowType rowType) { - this.rowType = rowType; - } - - @Override - public void open(Configuration parameters) throws Exception { - this.converter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - } - - @Override - public Row map(RowData value) throws Exception { - return (Row) converter.toExternal(value); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java deleted file mode 100644 index a1039d27d888..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ /dev/null @@ -1,185 +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 static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.DataTest; -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.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -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.iceberg.util.DateTimeUtil; -import org.junit.jupiter.api.Test; - -public class TestFlinkAvroReaderWriter extends DataTest { - - private static final int NUM_RECORDS = 100; - - private static final Schema SCHEMA_NUM_TYPE = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "int", Types.IntegerType.get()), - Types.NestedField.optional(3, "float", Types.FloatType.get()), - Types.NestedField.optional(4, "double", Types.DoubleType.get()), - Types.NestedField.optional(5, "date", Types.DateType.get()), - Types.NestedField.optional(6, "time", Types.TimeType.get()), - Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone()), - Types.NestedField.optional(8, "bigint", Types.LongType.get()), - Types.NestedField.optional(9, "decimal", Types.DecimalType.of(4, 2))); - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L); - writeAndValidate(schema, expectedRecords, NUM_RECORDS); - } - - private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) - throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - - File recordsFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(recordsFile.delete()).isTrue(); - - // Write the expected records into AVRO file, then read them into RowData and assert with the - // expected Record list. - try (FileAppender writer = - Avro.write(Files.localOutput(recordsFile)) - .schema(schema) - .createWriterFunc(DataWriter::create) - .build()) { - writer.addAll(expectedRecords); - } - - try (CloseableIterable reader = - Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new) - .build()) { - Iterator expected = expectedRecords.iterator(); - Iterator rows = reader.iterator(); - for (int i = 0; i < numRecord; i++) { - assertThat(rows).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); - } - assertThat(rows).isExhausted(); - } - - File rowDataFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(rowDataFile.delete()).isTrue(); - - // Write the expected RowData into AVRO file, then read them into Record and assert with the - // expected RowData list. - try (FileAppender writer = - Avro.write(Files.localOutput(rowDataFile)) - .schema(schema) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .build()) { - writer.addAll(expectedRows); - } - - try (CloseableIterable reader = - Avro.read(Files.localInput(rowDataFile)) - .project(schema) - .createReaderFunc(DataReader::create) - .build()) { - Iterator expected = expectedRows.iterator(); - Iterator records = reader.iterator(); - for (int i = 0; i < numRecord; i += 1) { - assertThat(records).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); - } - assertThat(records).isExhausted(); - } - } - - private Record recordNumType( - int id, - int intV, - float floatV, - double doubleV, - long date, - long time, - long timestamp, - long bigint, - double decimal) { - Record record = GenericRecord.create(SCHEMA_NUM_TYPE); - record.setField("id", id); - record.setField("int", intV); - record.setField("float", floatV); - record.setField("double", doubleV); - record.setField( - "date", DateTimeUtil.dateFromDays((int) new Date(date).toLocalDate().toEpochDay())); - record.setField("time", new Time(time).toLocalTime()); - record.setField("timestamp", DateTimeUtil.timestampFromMicros(timestamp * 1000)); - record.setField("bigint", bigint); - record.setField("decimal", BigDecimal.valueOf(decimal)); - return record; - } - - @Test - public void testNumericTypes() throws IOException { - - List expected = - ImmutableList.of( - recordNumType( - 2, - Integer.MAX_VALUE, - Float.MAX_VALUE, - Double.MAX_VALUE, - Long.MAX_VALUE, - 1643811742000L, - 1643811742000L, - 1643811742000L, - 10.24d), - recordNumType( - 2, - Integer.MIN_VALUE, - Float.MIN_VALUE, - Double.MIN_VALUE, - Long.MIN_VALUE, - 1643811742000L, - 1643811742000L, - 1643811742000L, - 10.24d)); - - writeAndValidate(SCHEMA_NUM_TYPE, expected, 2); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java deleted file mode 100644 index 72f2ce4f4bce..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ /dev/null @@ -1,107 +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 static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.orc.GenericOrcWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class TestFlinkOrcReaderWriter extends DataTest { - private static final int NUM_RECORDS = 100; - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); - List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - - File recordsFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(recordsFile.delete()).isTrue(); - - // Write the expected records into ORC file, then read them into RowData and assert with the - // expected Record list. - try (FileAppender writer = - ORC.write(Files.localOutput(recordsFile)) - .schema(schema) - .createWriterFunc(GenericOrcWriter::buildWriter) - .build()) { - writer.addAll(expectedRecords); - } - - try (CloseableIterable reader = - ORC.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(type -> new FlinkOrcReader(schema, type)) - .build()) { - Iterator expected = expectedRecords.iterator(); - Iterator rows = reader.iterator(); - for (int i = 0; i < NUM_RECORDS; i++) { - assertThat(rows).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); - } - assertThat(rows).isExhausted(); - } - - File rowDataFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(rowDataFile.delete()).isTrue(); - - // Write the expected RowData into ORC file, then read them into Record and assert with the - // expected RowData list. - RowType rowType = FlinkSchemaUtil.convert(schema); - try (FileAppender writer = - ORC.write(Files.localOutput(rowDataFile)) - .schema(schema) - .createWriterFunc((iSchema, typeDesc) -> FlinkOrcWriter.buildWriter(rowType, iSchema)) - .build()) { - writer.addAll(expectedRows); - } - - try (CloseableIterable reader = - ORC.read(Files.localInput(rowDataFile)) - .project(schema) - .createReaderFunc(type -> GenericOrcReader.buildReader(schema, type)) - .build()) { - Iterator expected = expectedRows.iterator(); - Iterator records = reader.iterator(); - for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(records.hasNext()).isTrue(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); - } - assertThat(records).isExhausted(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java deleted file mode 100644 index 1fdc4cf8381a..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ /dev/null @@ -1,138 +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 static org.apache.iceberg.types.Types.NestedField.optional; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.GenericParquetWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -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.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.hadoop.ParquetWriter; -import org.junit.jupiter.api.Test; - -public class TestFlinkParquetReader extends DataTest { - private static final int NUM_RECORDS = 100; - - @Test - public void testTwoLevelList() throws IOException { - Schema schema = - new Schema( - optional(1, "arraybytes", Types.ListType.ofRequired(3, Types.BinaryType.get())), - optional(2, "topbytes", Types.BinaryType.get())); - org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).isTrue(); - - ParquetWriter writer = - AvroParquetWriter.builder(new Path(testFile.toURI())) - .withDataModel(GenericData.get()) - .withSchema(avroSchema) - .config("parquet.avro.add-list-element-records", "true") - .config("parquet.avro.write-old-list-structure", "true") - .build(); - - GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema); - List expectedByteList = Lists.newArrayList(); - byte[] expectedByte = {0x00, 0x01}; - ByteBuffer expectedBinary = ByteBuffer.wrap(expectedByte); - expectedByteList.add(expectedBinary); - recordBuilder.set("arraybytes", expectedByteList); - recordBuilder.set("topbytes", expectedBinary); - GenericData.Record expectedRecord = recordBuilder.build(); - - writer.write(expectedRecord); - writer.close(); - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) - .build()) { - Iterator rows = reader.iterator(); - assertThat(rows).hasNext(); - RowData rowData = rows.next(); - assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); - assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); - assertThat(rows).isExhausted(); - } - } - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).isTrue(); - - try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(GenericParquetWriter::buildWriter) - .build()) { - writer.addAll(iterable); - } - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) - .build()) { - Iterator expected = iterable.iterator(); - Iterator rows = reader.iterator(); - LogicalType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(rows).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); - } - assertThat(rows).isExhausted(); - } - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(RandomGenericData.generate(schema, NUM_RECORDS, 19981), schema); - writeAndValidate( - RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124), schema); - writeAndValidate( - RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), - schema); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java deleted file mode 100644 index b1e6f5aa00ff..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java +++ /dev/null @@ -1,94 +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 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 org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.GenericParquetReaders; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.parquet.Parquet; -import org.junit.jupiter.api.io.TempDir; - -public class TestFlinkParquetWriter extends DataTest { - private static final int NUM_RECORDS = 100; - - @TempDir private Path temp; - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).isTrue(); - - LogicalType logicalType = FlinkSchemaUtil.convert(schema); - - try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(logicalType, msgType)) - .build()) { - writer.addAll(iterable); - } - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) - .build()) { - Iterator expected = iterable.iterator(); - Iterator actual = reader.iterator(); - LogicalType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(actual).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); - } - assertThat(actual).isExhausted(); - } - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(RandomRowData.generate(schema, NUM_RECORDS, 19981), schema); - - writeAndValidate( - RandomRowData.convert( - schema, - RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124)), - schema); - - writeAndValidate( - RandomRowData.convert( - schema, - RandomGenericData.generateFallbackRecords( - schema, NUM_RECORDS, 21124, NUM_RECORDS / 20)), - schema); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java deleted file mode 100644 index 3cd25c8fa983..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java +++ /dev/null @@ -1,593 +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.util.List; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.TestHelpers; -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.apache.iceberg.util.StructProjection; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; - -public class TestRowDataProjection { - @Test - public void testNullRootRowData() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); - - Assertions.assertThatThrownBy(() -> projection.wrap(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid row data: null"); - } - - @Test - public void testFullProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - generateAndValidate(schema, schema); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - } - - @Test - public void testReorderedFullProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - generateAndValidate(schema, reordered); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, reordered, rowData, copyRowData, otherRowData); - } - - @Test - public void testBasicProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - generateAndValidate(schema, idOnly); - generateAndValidate(schema, dataOnly); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, dataOnly, rowData, copyRowData, otherRowData); - } - - @Test - public void testEmptyProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - generateAndValidate(schema, schema.select()); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, schema.select(), rowData, copyRowData, otherRowData, true); - } - - @Test - public void testRename() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Schema renamed = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - generateAndValidate(schema, renamed); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, renamed, rowData, copyRowData, otherRowData); - } - - @Test - public void testNestedProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - GenericRowData rowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); - GenericRowData copyRowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); - GenericRowData otherRowData = GenericRowData.of(2L, GenericRowData.of(2.0f, 2.0f)); - - GenericRowData rowDataNullStruct = GenericRowData.of(1L, null); - GenericRowData copyRowDataNullStruct = GenericRowData.of(1L, null); - GenericRowData otherRowDataNullStruct = GenericRowData.of(2L, null); - - // Project id only. - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, idOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct); - - // Project lat only. - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, latOnly); - testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, latOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); - - // Project long only. - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, longOnly); - testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, longOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); - - // Project location. - Schema locationOnly = schema.select("location"); - Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, locationOnly); - testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, - locationOnly, - rowDataNullStruct, - copyRowDataNullStruct, - otherRowDataNullStruct, - true); - } - - @Test - public void testPrimitivesFullProjection() { - DataGenerator dataGenerator = new DataGenerators.Primitives(); - Schema schema = dataGenerator.icebergSchema(); - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - GenericRowData otherRowData = dataGenerator.generateFlinkRowData(); - // modify the string field value (position 6) - otherRowData.setField(6, StringData.fromString("foo_bar")); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - setOptionalFieldsNullForPrimitives(rowDataNullOptionalFields); - GenericRowData copyRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - setOptionalFieldsNullForPrimitives(copyRowDataNullOptionalFields); - GenericRowData otherRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - // modify the string field value (position 6) - otherRowDataNullOptionalFields.setField(6, StringData.fromString("foo_bar")); - setOptionalFieldsNullForPrimitives(otherRowData); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - private void setOptionalFieldsNullForPrimitives(GenericRowData rowData) { - // fields from [1, 5] range are optional - for (int pos = 1; pos <= 5; ++pos) { - rowData.setField(pos, null); - } - } - - @Test - public void testMapOfPrimitivesProjection() { - DataGenerator dataGenerator = new DataGenerators.MapOfPrimitives(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project map only. - Schema mapOnly = schema.select("map_of_primitives"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, mapOnly); - - // Project all. - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(StringData.fromString("foo"), 1, StringData.fromString("bar"), 2))); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData, true); - testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("row_id_value"), null); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("row_id_value"), null); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("other_row_id_value"), null); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - mapOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields, - true); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testMapOfStructStructProjection() { - DataGenerator dataGenerator = new DataGenerators.MapOfStructStruct(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project map only. - Schema mapOnly = schema.select("map"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, mapOnly); - - // Project all. - generateAndValidate(schema, schema); - - // Project partial map key. - Schema partialMapKey = - new Schema( - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()), - Types.NestedField.required(204, "valueData", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial map key or value struct."); - - // Project partial map key. - Schema partialMapValue = - new Schema( - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get()), - Types.NestedField.required(202, "keyData", Types.StringType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial map key or value struct."); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericMapData( - ImmutableMap.of( - GenericRowData.of(1L, StringData.fromString("other_key_data")), - GenericRowData.of(1L, StringData.fromString("other_value_data"))))); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(2L, null), GenericRowData.of(2L, null)))); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - mapOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testArrayOfPrimitiveProjection() { - DataGenerator dataGenerator = new DataGenerators.ArrayOfPrimitive(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project list only. - Schema arrayOnly = schema.select("array_of_int"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, arrayOnly); - - // Project all. - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericArrayData(new Integer[] {4, 5, 6})); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, arrayOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericArrayData(new Integer[] {4, null, 6})); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - arrayOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testArrayOfStructProjection() { - DataGenerator dataGenerator = new DataGenerators.ArrayOfStruct(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project list only. - Schema arrayOnly = schema.select("array_of_struct"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, arrayOnly); - - // Project all. - generateAndValidate(schema, schema); - - // Project partial list value. - Schema partialList = - new Schema( - Types.NestedField.optional( - 2, - "array_of_struct", - Types.ListType.ofOptional( - 101, - Types.StructType.of( - Types.NestedField.required(202, "name", Types.StringType.get()))))); - - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial list element struct."); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(new Integer[] {4, 5, 6})); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {4, null, 6})); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - private void generateAndValidate(Schema schema, Schema projectSchema) { - int numRecords = 100; - List recordList = RandomGenericData.generate(schema, numRecords, 102L); - List rowDataList = - Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); - Assertions.assertThat(rowDataList).hasSize(recordList.size()); - - StructProjection structProjection = StructProjection.create(schema, projectSchema); - RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); - - for (int i = 0; i < numRecords; i++) { - StructLike expected = structProjection.wrap(recordList.get(i)); - RowData projected = rowDataProjection.wrap(rowDataList.get(i)); - TestHelpers.assertRowData(projectSchema, expected, projected); - - Assertions.assertThat(projected).isEqualTo(projected); - Assertions.assertThat(projected).hasSameHashCodeAs(projected); - // make sure toString doesn't throw NPE for null values - Assertions.assertThatNoException().isThrownBy(projected::toString); - } - } - - private void testEqualsAndHashCode( - Schema schema, - Schema projectionSchema, - RowData rowData, - RowData copyRowData, - RowData otherRowData) { - testEqualsAndHashCode(schema, projectionSchema, rowData, copyRowData, otherRowData, false); - } - - /** - * @param isOtherRowDataSameAsRowData sometimes projection on otherRowData can result in the same - * RowData, e.g. due to empty projection or null struct - */ - private void testEqualsAndHashCode( - Schema schema, - Schema projectionSchema, - RowData rowData, - RowData copyRowData, - RowData otherRowData, - boolean isOtherRowDataSameAsRowData) { - RowDataProjection projection = RowDataProjection.create(schema, projectionSchema); - RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); - RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); - - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); - - if (isOtherRowDataSameAsRowData) { - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); - } else { - Assertions.assertThat(projection.wrap(rowData)) - .isNotEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java deleted file mode 100644 index df2e6ae21c7e..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ /dev/null @@ -1,580 +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 java.io.IOException; -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; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Files; -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.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestRowProjection { - - @Rule public TemporaryFolder temp = new TemporaryFolder(); - - private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) - throws IOException { - File file = temp.newFile(desc + ".avro"); - Assert.assertTrue(file.delete()); - - try (FileAppender appender = - Avro.write(Files.localOutput(file)) - .schema(writeSchema) - .createWriterFunc(ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(writeSchema))) - .build()) { - appender.add(row); - } - - Iterable records = - Avro.read(Files.localInput(file)) - .project(readSchema) - .createReaderFunc(FlinkAvroReader::new) - .build(); - - return Iterables.getOnlyElement(records); - } - - @Test - public void testFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData projected = writeAndRead("full_projection", schema, schema, row); - - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data value", cmp, 0); - } - - @Test - public void testSpecialCharacterProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "user id", Types.LongType.get()), - Types.NestedField.optional(1, "data%0", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData full = writeAndRead("special_chars", schema, schema, row); - - Assert.assertEquals("Should contain the correct id value", 34L, full.getLong(0)); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", full.getString(1).toString())); - - RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); - - Assert.assertEquals("Should not contain id value", 1, projected.getArity()); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", projected.getString(0).toString())); - } - - @Test - public void testReorderedFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("full_projection", schema, reordered, row); - - Assert.assertEquals( - "Should contain the correct 0 value", "test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.getLong(1)); - } - - @Test - public void testReorderedProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema reordered = - new Schema( - Types.NestedField.optional(2, "missing_1", Types.StringType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(3, "missing_2", Types.LongType.get())); - - RowData projected = writeAndRead("full_projection", schema, reordered, row); - - Assert.assertTrue("Should contain the correct 0 value", projected.isNullAt(0)); - Assert.assertEquals( - "Should contain the correct 1 value", "test", projected.getString(1).toString()); - Assert.assertTrue("Should contain the correct 2 value", projected.isNullAt(2)); - } - - @Test - public void testRenamedAddedField() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(1, "a", Types.LongType.get()), - Types.NestedField.required(2, "b", Types.LongType.get()), - Types.NestedField.required(3, "d", Types.LongType.get())); - - RowData row = GenericRowData.of(100L, 200L, 300L); - - Schema renamedAdded = - new Schema( - Types.NestedField.optional(1, "a", Types.LongType.get()), - Types.NestedField.optional(2, "b", Types.LongType.get()), - Types.NestedField.optional(3, "c", Types.LongType.get()), - Types.NestedField.optional(4, "d", Types.LongType.get())); - - RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertEquals("Should contain the correct value in column 2", projected.getLong(1), 200L); - Assert.assertEquals("Should contain the correct value in column 3", projected.getLong(2), 300L); - Assert.assertTrue("Should contain empty value on new column 4", projected.isNullAt(3)); - } - - @Test - public void testEmptyProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); - - Assert.assertNotNull("Should read a non-null record", projected); - Assert.assertEquals(0, projected.getArity()); - } - - @Test - public void testBasicProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); - Assert.assertEquals("Should not project data", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - - projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); - - Assert.assertEquals("Should not project id", 1, projected.getArity()); - int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct data value", 0, cmp); - } - - @Test - public void testRename() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema readSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - - RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); - - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); - } - - @Test - public void testNestedStructProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - RowData location = GenericRowData.of(52.995143f, -1.539054f); - RowData record = GenericRowData.of(34L, location); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should not project location", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - - projected = writeAndRead("latitude_only", writeSchema, latOnly, record); - RowData projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project longitude", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - - projected = writeAndRead("longitude_only", writeSchema, longOnly, record); - projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project latitutde", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(0), 0.000001f); - - Schema locationOnly = writeSchema.select("location"); - projected = writeAndRead("location_only", writeSchema, locationOnly, record); - projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(1), 0.000001f); - } - - @Test - public void testMapProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "properties", - Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); - - GenericMapData properties = - new GenericMapData( - ImmutableMap.of( - StringData.fromString("a"), - StringData.fromString("A"), - StringData.fromString("b"), - StringData.fromString("B"))); - - RowData row = GenericRowData.of(34L, properties); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project properties map", 1, projected.getArity()); - - Schema keyOnly = writeSchema.select("properties.key"); - projected = writeAndRead("key_only", writeSchema, keyOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); - - Schema valueOnly = writeSchema.select("properties.value"); - projected = writeAndRead("value_only", writeSchema, valueOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); - - Schema mapOnly = writeSchema.select("properties"); - projected = writeAndRead("map_only", writeSchema, mapOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); - } - - 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; - } - - @Test - public void testMapOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get()))))); - - RowData l1 = GenericRowData.of(53.992811f, -1.542616f); - RowData l2 = GenericRowData.of(52.995143f, -1.539054f); - GenericMapData map = - new GenericMapData( - ImmutableMap.of(StringData.fromString("L1"), l1, StringData.fromString("L2"), l2)); - RowData row = GenericRowData.of(34L, map); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project locations map", 1, projected.getArity()); - - projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project locations map", row.getMap(1), projected.getMap(0)); - - projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); - GenericMapData locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - GenericArrayData l1l2Array = - new GenericArrayData( - new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); - RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should contain lat", 53.992811f, projectedL1.getFloat(0), 0.000001); - Assert.assertEquals("L1 should not contain long", 1, projectedL1.getArity()); - RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should contain lat", 52.995143f, projectedL2.getFloat(0), 0.000001); - Assert.assertEquals("L2 should not contain long", 1, projectedL2.getArity()); - - projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); - projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should not contain lat", 1, projectedL1.getArity()); - Assert.assertEquals("L1 should contain long", -1.542616f, projectedL1.getFloat(0), 0.000001); - projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should not contain lat", 1, projectedL2.getArity()); - Assert.assertEquals("L2 should contain long", -1.539054f, projectedL2.getFloat(0), 0.000001); - - Schema latitiudeRenamed = - new Schema( - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); - - projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); - projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", 53.992811f, projectedL1.getFloat(0), 0.000001); - projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", 52.995143f, projectedL2.getFloat(0), 0.000001); - } - - @Test - public void testListProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); - - GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L}); - - RowData row = GenericRowData.of(34L, values); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project values list", 1, projected.getArity()); - - Schema elementOnly = writeSchema.select("values.element"); - projected = writeAndRead("element_only", writeSchema, elementOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); - - Schema listOnly = writeSchema.select("values"); - projected = writeAndRead("list_only", writeSchema, listOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); - } - - @Test - @SuppressWarnings("unchecked") - public void testListOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.required(19, "x", Types.IntegerType.get()), - Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); - - RowData p1 = GenericRowData.of(1, 2); - RowData p2 = GenericRowData.of(3, null); - GenericArrayData arrayData = new GenericArrayData(new RowData[] {p1, p2}); - RowData row = GenericRowData.of(34L, arrayData); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project points list", 1, projected.getArity()); - - projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project points list", row.getArray(1), projected.getArray(0)); - - projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); - ArrayData points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); - RowData projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should project x", 1, projectedP1.getInt(0)); - Assert.assertEquals("Should not project y", 1, projectedP1.getArity()); - RowData projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project y", 1, projectedP2.getArity()); - Assert.assertEquals("Should project x", 3, projectedP2.getInt(0)); - - projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); - points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); - projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x", 1, projectedP1.getArity()); - Assert.assertEquals("Should project y", 2, projectedP1.getInt(0)); - projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null y", projectedP2.isNullAt(0)); - - Schema yRenamed = - new Schema( - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); - - projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); - points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); - projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP1.getArity()); - Assert.assertEquals("Should project z", 2, projectedP1.getInt(0)); - projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null z", projectedP2.isNullAt(0)); - } - - @Test - public void testAddedFieldsWithRequiredChildren() throws Exception { - Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); - - RowData row = GenericRowData.of(100L); - - Schema addedFields = - new Schema( - Types.NestedField.optional(1, "a", Types.LongType.get()), - Types.NestedField.optional( - 2, - "b", - Types.StructType.of(Types.NestedField.required(3, "c", Types.LongType.get()))), - Types.NestedField.optional(4, "d", Types.ListType.ofRequired(5, Types.LongType.get())), - Types.NestedField.optional( - 6, - "e", - Types.MapType.ofRequired(7, 8, Types.LongType.get(), Types.LongType.get()))); - - RowData projected = - writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertTrue("Should contain empty value in new column 2", projected.isNullAt(1)); - Assert.assertTrue("Should contain empty value in new column 4", projected.isNullAt(2)); - Assert.assertTrue("Should contain empty value in new column 6", projected.isNullAt(3)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java deleted file mode 100644 index e0340e0743b0..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java +++ /dev/null @@ -1,100 +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 org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.TestHelpers; -import org.junit.Test; - -public class TestStructRowData { - - protected void testConverter(DataGenerator dataGenerator) { - StructRowData converter = new StructRowData(dataGenerator.icebergSchema().asStruct()); - GenericRecord expected = dataGenerator.generateIcebergGenericRecord(); - StructRowData actual = converter.setStruct(expected); - TestHelpers.assertRowData(dataGenerator.icebergSchema(), expected, actual); - } - - @Test - public void testPrimitiveTypes() { - testConverter(new DataGenerators.Primitives()); - } - - @Test - public void testStructOfPrimitive() { - testConverter(new DataGenerators.StructOfPrimitive()); - } - - @Test - public void testStructOfArray() { - testConverter(new DataGenerators.StructOfArray()); - } - - @Test - public void testStructOfMap() { - testConverter(new DataGenerators.StructOfMap()); - } - - @Test - public void testStructOfStruct() { - testConverter(new DataGenerators.StructOfStruct()); - } - - @Test - public void testArrayOfPrimitive() { - testConverter(new DataGenerators.ArrayOfPrimitive()); - } - - @Test - public void testArrayOfArray() { - testConverter(new DataGenerators.ArrayOfArray()); - } - - @Test - public void testArrayOfMap() { - testConverter(new DataGenerators.ArrayOfMap()); - } - - @Test - public void testArrayOfStruct() { - testConverter(new DataGenerators.ArrayOfStruct()); - } - - @Test - public void testMapOfPrimitives() { - testConverter(new DataGenerators.MapOfPrimitives()); - } - - @Test - public void testMapOfArray() { - testConverter(new DataGenerators.MapOfArray()); - } - - @Test - public void testMapOfMap() { - testConverter(new DataGenerators.MapOfMap()); - } - - @Test - public void testMapOfStruct() { - testConverter(new DataGenerators.MapOfStruct()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java deleted file mode 100644 index 6a493692c20d..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java +++ /dev/null @@ -1,37 +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.sink; - -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.flink.AvroGenericRecordConverterBase; -import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; - -public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { - @Override - protected void testConverter(DataGenerator dataGenerator) throws Exception { - // Need to use avroSchema from DataGenerator because some primitive types have special Avro - // type handling. Hence the Avro schema converted from Iceberg schema won't work. - AvroGenericRecordToRowDataMapper mapper = - AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); - RowData expected = dataGenerator.generateFlinkRowData(); - RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); - Assert.assertEquals(expected, actual); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java deleted file mode 100644 index 5ebcc6361c7b..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ /dev/null @@ -1,65 +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.sink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestBucketPartitionKeySelector { - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testCorrectKeySelection(TableSchemaType tableSchemaType) { - int numBuckets = 60; - - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitionKeySelector keySelector = - new BucketPartitionKeySelector( - partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); - - TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) - .forEach( - rowData -> { - int expectedBucketId = - TestBucketPartitionerUtil.computeBucketId( - numBuckets, rowData.getString(1).toString()); - Integer key = keySelector.getKey(rowData); - Assertions.assertThat(key).isEqualTo(expectedBucketId); - }); - } - - @Test - public void testKeySelectorMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy( - () -> - new BucketPartitionKeySelector( - partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) - .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java deleted file mode 100644 index 835713e6b417..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ /dev/null @@ -1,107 +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.sink; - -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; - -public class TestBucketPartitioner { - - static final int DEFAULT_NUM_BUCKETS = 60; - - @ParameterizedTest - @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) - public void testPartitioningParallelismGreaterThanBuckets( - String schemaTypeStr, String numBucketsStr) { - int numPartitions = 500; - TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - int bucketId = 0; - for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { - int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); - bucketId++; - if (bucketId == numBuckets) { - bucketId = 0; - } - } - } - - @ParameterizedTest - @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) - public void testPartitioningParallelismEqualLessThanBuckets( - String schemaTypeStr, String numBucketsStr) { - int numPartitions = 30; - TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - for (int bucketId = 0; bucketId < numBuckets; bucketId++) { - int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); - } - } - - @Test - public void testPartitionerBucketIdNullFail() { - PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) - .withMessage(BUCKET_NULL_MESSAGE); - } - - @Test - public void testPartitionerMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); - - Assertions.assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> new BucketPartitioner(partitionSpec)) - .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); - } - - @Test - public void testPartitionerBucketIdOutOfRangeFail() { - PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - int negativeBucketId = -1; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) - .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); - - int tooBigBucketId = DEFAULT_NUM_BUCKETS; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) - .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java deleted file mode 100644 index 9dae43ce5e58..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ /dev/null @@ -1,227 +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.sink; - -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -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.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.CloseableIterable; -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.assertj.core.api.Assertions; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestBucketPartitionerFlinkIcebergSink { - - private static final int NUMBER_TASK_MANAGERS = 1; - private static final int SLOTS_PER_TASK_MANAGER = 8; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(NUMBER_TASK_MANAGERS) - .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - - @RegisterExtension - private static final HadoopCatalogExtension catalogExtension = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - // Parallelism = 8 (parallelism > numBuckets) throughout the test suite - private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; - private final FileFormat format = FileFormat.PARQUET; - private final int numBuckets = 4; - - private Table table; - private StreamExecutionEnvironment env; - private TableLoader tableLoader; - - private void setupEnvironment(TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - table = - catalogExtension - .catalog() - .createTable( - TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitionSpec, - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = - StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism * 2); - tableLoader = catalogExtension.tableLoader(); - } - - private void appendRowsToTable(List allRows) throws Exception { - DataFormatConverters.RowConverter converter = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - DataStream dataStream = - env.addSource( - new BoundedTestSource<>( - allRows.stream().map(converter::toExternal).toArray(Row[]::new)), - ROW_TYPE_INFO) - .map(converter::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)) - .partitionCustom( - new BucketPartitioner(table.spec()), - new BucketPartitionKeySelector( - table.spec(), - table.schema(), - FlinkSink.toFlinkRowType(table.schema(), SimpleDataUtil.FLINK_SCHEMA))); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.NONE) - .append(); - - env.execute("Test Iceberg DataStream"); - - SimpleDataUtil.assertTableRows(table, allRows); - } - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { - setupEnvironment(tableSchemaType); - List rows = generateTestDataRows(); - - appendRowsToTable(rows); - TableTestStats stats = extractPartitionResults(tableSchemaType); - - Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); - // All 4 buckets should've been written to - Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); - Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); - // Writer expectation (2 writers per bucket): - // - Bucket0 -> Writers [0, 4] - // - Bucket1 -> Writers [1, 5] - // - Bucket2 -> Writers [2, 6] - // - Bucket3 -> Writers [3, 7] - for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { - Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); - // 2 files per bucket (one file is created by each writer) - Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); - // 2 rows per file (total of 16 rows across 8 files) - Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); - } - } - - /** - * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 - * buckets) - */ - private List generateTestDataRows() { - int totalNumRows = parallelism * 2; - int numRowsPerBucket = totalNumRows / numBuckets; - return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); - } - - private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) - throws IOException { - int totalRecordCount = 0; - Map> writersPerBucket = Maps.newHashMap(); // > - Map filesPerBucket = Maps.newHashMap(); // - Map rowsPerWriter = Maps.newHashMap(); // - - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - long recordCountInFile = scanTask.file().recordCount(); - - String[] splitFilePath = scanTask.file().path().toString().split("/"); - // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet - // Writer ID: .......^^^^^ - String filename = splitFilePath[splitFilePath.length - 1]; - int writerId = Integer.parseInt(filename.split("-")[0]); - - totalRecordCount += recordCountInFile; - int bucketId = - scanTask - .file() - .partition() - .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); - writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); - writersPerBucket.get(bucketId).add(writerId); - filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); - rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); - } - } - - return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); - } - - /** DTO to hold Test Stats */ - private static class TableTestStats { - final int totalRowCount; - final Map> writersPerBucket; - final Map numFilesPerBucket; - final Map rowsPerWriter; - - TableTestStats( - int totalRecordCount, - Map> writersPerBucket, - Map numFilesPerBucket, - Map rowsPerWriter) { - this.totalRowCount = totalRecordCount; - this.writersPerBucket = writersPerBucket; - this.numFilesPerBucket = numFilesPerBucket; - this.rowsPerWriter = rowsPerWriter; - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java deleted file mode 100644 index e1309bfac6d5..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java +++ /dev/null @@ -1,126 +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.sink; - -import java.util.List; -import java.util.UUID; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.BucketUtil; - -final class TestBucketPartitionerUtil { - - enum TableSchemaType { - ONE_BUCKET { - @Override - public int bucketPartitionColumnPosition() { - return 0; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); - } - }, - IDENTITY_AND_BUCKET { - @Override - public int bucketPartitionColumnPosition() { - return 1; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .identity("id") - .bucket("data", numBuckets) - .build(); - } - }, - TWO_BUCKETS { - @Override - public int bucketPartitionColumnPosition() { - return 1; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("id", numBuckets) - .bucket("data", numBuckets) - .build(); - } - }; - - public abstract int bucketPartitionColumnPosition(); - - public abstract PartitionSpec getPartitionSpec(int numBuckets); - } - - private TestBucketPartitionerUtil() {} - - /** - * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to - * numBuckets - 1) - * - * @param numRowsPerBucket how many different rows should be generated per bucket - * @param numBuckets max number of buckets to consider - * @return the list of rows whose data "hashes" to the desired bucketId - */ - static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { - List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); - // For some of our tests, this order of the generated rows matters - for (int i = 0; i < numRowsPerBucket; i++) { - for (int bucketId = 0; bucketId < numBuckets; bucketId++) { - String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(GenericRowData.of(1, StringData.fromString(value))); - } - } - return rows; - } - - /** - * Utility method to generate a UUID string that will "hash" to a desired bucketId - * - * @param bucketId the desired bucketId - * @return the string data that "hashes" to the desired bucketId - */ - private static String generateValueForBucketId(int bucketId, int numBuckets) { - while (true) { - String uuid = UUID.randomUUID().toString(); - if (computeBucketId(numBuckets, uuid) == bucketId) { - return uuid; - } - } - } - - /** - * Utility that performs the same hashing/bucketing mechanism used by Bucket.java - * - * @param numBuckets max number of buckets to consider - * @param value the string to compute the bucketId from - * @return the computed bucketId - */ - static int computeBucketId(int numBuckets, String value) { - return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java deleted file mode 100644 index 360db658cd2f..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java +++ /dev/null @@ -1,81 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.concurrent.TimeUnit; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; - -public class TestCachingTableSupplier { - - @Test - public void testCheckArguments() { - SerializableTable initialTable = mock(SerializableTable.class); - - Table loadedTable = mock(Table.class); - TableLoader tableLoader = mock(TableLoader.class); - when(tableLoader.loadTable()).thenReturn(loadedTable); - - new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); - - assertThatThrownBy(() -> new CachingTableSupplier(initialTable, tableLoader, null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("tableRefreshInterval cannot be null"); - assertThatThrownBy(() -> new CachingTableSupplier(null, tableLoader, Duration.ofMillis(100))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("initialTable cannot be null"); - assertThatThrownBy(() -> new CachingTableSupplier(initialTable, null, Duration.ofMillis(100))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("tableLoader cannot be null"); - } - - @Test - public void testTableReload() { - SerializableTable initialTable = mock(SerializableTable.class); - - Table loadedTable = mock(Table.class); - TableLoader tableLoader = mock(TableLoader.class); - when(tableLoader.loadTable()).thenReturn(loadedTable); - - CachingTableSupplier cachingTableSupplier = - new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); - - // refresh shouldn't do anything as the min reload interval hasn't passed - cachingTableSupplier.refreshTable(); - assertThat(cachingTableSupplier.get()).isEqualTo(initialTable); - - // refresh after waiting past the min reload interval - Awaitility.await() - .atLeast(100, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - cachingTableSupplier.refreshTable(); - assertThat(cachingTableSupplier.get()).isEqualTo(loadedTable); - }); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java deleted file mode 100644 index 214c3de1063e..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ /dev/null @@ -1,254 +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.sink; - -import java.io.File; -import java.io.IOException; -import java.util.Map; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.common.DynFields; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.BaseTaskWriter; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private Table table; - - private final Map initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } - - public TestCompressionSettings(Map initProperties) { - this.initProperties = initProperties; - } - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); - } - - @Test - public void testCompressionAvro() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); - - if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } - - // Override compression to snappy and some random level - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "AVRO", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_LEVEL.key(), - "6")); - - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } - - @Test - public void testCompressionParquet() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); - - if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); - } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); - } - - // Override compression to snappy and some random level - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "PARQUET", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_LEVEL.key(), - "6")); - - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); - } - - @Test - public void testCompressionOrc() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); - - if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } - - // Override compression to snappy and a different strategy - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "ORC", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_STRATEGY.key(), - "speed")); - - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } - - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { - RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); - FlinkWriteConf flinkWriteConfig = - new FlinkWriteConf( - icebergTable, override, new org.apache.flink.configuration.Configuration()); - - IcebergStreamWriter streamWriter = - FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); - - harness.setup(); - harness.open(); - - return harness; - } - - private static Map appenderProperties( - Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter(table, schema, override)) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - - testHarness.prepareSnapshotPreBarrier(1L); - DynFields.BoundField operatorField = - DynFields.builder() - .hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator") - .build(testHarness.getOperatorFactory()); - DynFields.BoundField writerField = - DynFields.builder() - .hiddenImpl(IcebergStreamWriter.class, "writer") - .build(operatorField.get()); - DynFields.BoundField appenderField = - DynFields.builder() - .hiddenImpl(BaseTaskWriter.class, "appenderFactory") - .build(writerField.get()); - DynFields.BoundField> propsField = - DynFields.builder() - .hiddenImpl(FlinkAppenderFactory.class, "props") - .build(appenderField.get()); - return propsField.get(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java deleted file mode 100644 index 21f3ee2c655a..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ /dev/null @@ -1,429 +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.sink; - -import static org.apache.iceberg.flink.SimpleDataUtil.createDelete; -import static org.apache.iceberg.flink.SimpleDataUtil.createInsert; -import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; -import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; -import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; -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.nio.file.Path; -import java.nio.file.Paths; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.FileContent; -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.RowDelta; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.TestTables; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -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.types.Types; -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 class TestDeltaTaskWriter extends TestBase { - - @Parameter(index = 1) - private FileFormat format; - - @Parameters(name = "formatVersion = {0}, fileFormat = {1}") - protected static List parameters() { - return Arrays.asList( - new Object[] {2, FileFormat.AVRO}, - new Object[] {2, FileFormat.ORC}, - new Object[] {2, FileFormat.PARQUET}); - } - - @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"); - } - - private int idFieldId() { - return table.schema().findField("id").fieldId(); - } - - private int dataFieldId() { - return table.schema().findField("data").fieldId(); - } - - private void testCdcEvents(boolean partitioned) throws IOException { - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - // Start the 1th transaction. - TaskWriter writer = taskWriterFactory.create(); - - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "bbb")); - writer.write(createInsert(3, "ccc")); - - // Update <2, 'bbb'> to <2, 'ddd'> - writer.write(createUpdateBefore(2, "bbb")); // 1 pos-delete and 1 eq-delete. - writer.write(createUpdateAfter(2, "ddd")); - - // Update <1, 'aaa'> to <1, 'eee'> - writer.write(createUpdateBefore(1, "aaa")); // 1 pos-delete and 1 eq-delete. - writer.write(createUpdateAfter(1, "eee")); - - // Insert <4, 'fff'> - writer.write(createInsert(4, "fff")); - // Insert <5, 'ggg'> - writer.write(createInsert(5, "ggg")); - - // Delete <3, 'ccc'> - writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); - assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg"))); - - // Start the 2nd transaction. - writer = taskWriterFactory.create(); - - // Update <2, 'ddd'> to <6, 'hhh'> - (Update both key and value) - writer.write(createUpdateBefore(2, "ddd")); // 1 eq-delete - writer.write(createUpdateAfter(6, "hhh")); - - // Update <5, 'ggg'> to <5, 'iii'> - writer.write(createUpdateBefore(5, "ggg")); // 1 eq-delete - writer.write(createUpdateAfter(5, "iii")); - - // Delete <4, 'fff'> - writer.write(createDelete(4, "fff")); // 1 eq-delete. - - result = writer.complete(); - assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); - assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); - } - - @TestTemplate - public void testUnpartitioned() throws IOException { - createAndInitTable(false); - testCdcEvents(false); - } - - @TestTemplate - public void testPartitioned() throws IOException { - createAndInitTable(true); - testCdcEvents(true); - } - - private void testWritePureEqDeletes(boolean partitioned) throws IOException { - createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - writer.write(createDelete(1, "aaa")); - writer.write(createDelete(2, "bbb")); - writer.write(createDelete(3, "ccc")); - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).isEmpty(); - assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); - commitTransaction(result); - - assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); - } - - @TestTemplate - public void testUnpartitionedPureEqDeletes() throws IOException { - testWritePureEqDeletes(false); - } - - @TestTemplate - public void testPartitionedPureEqDeletes() throws IOException { - testWritePureEqDeletes(true); - } - - private void testAbort(boolean partitioned) throws IOException { - createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - for (int i = 0; i < 8_000; i += 2) { - writer.write(createUpdateBefore(i + 1, "aaa")); - writer.write(createUpdateAfter(i + 1, "aaa")); - - writer.write(createUpdateBefore(i + 2, "bbb")); - writer.write(createUpdateAfter(i + 2, "bbb")); - } - - // Assert the current data/delete file count. - List files = - Files.walk(Paths.get(tableDir.getPath(), "data")) - .filter(p -> p.toFile().isFile()) - .filter(p -> !p.toString().endsWith(".crc")) - .collect(Collectors.toList()); - assertThat(files).hasSize(partitioned ? 4 : 2); - - writer.abort(); - for (Path file : files) { - assertThat(file).doesNotExist(); - } - } - - @TestTemplate - public void testUnpartitionedAbort() throws IOException { - testAbort(false); - } - - @TestTemplate - public void testPartitionedAbort() throws IOException { - testAbort(true); - } - - @TestTemplate - public void testPartitionedTableWithDataAsKey() throws IOException { - createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - // Start the 1th transaction. - TaskWriter writer = taskWriterFactory.create(); - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "aaa")); - writer.write(createInsert(3, "bbb")); - writer.write(createInsert(4, "ccc")); - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).hasSize(3); - assertThat(result.deleteFiles()).hasSize(1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); - - // Start the 2nd transaction. - writer = taskWriterFactory.create(); - writer.write(createInsert(5, "aaa")); - writer.write(createInsert(6, "bbb")); - writer.write(createDelete(7, "ccc")); // 1 eq-delete. - - result = writer.complete(); - assertThat(result.dataFiles()).hasSize(2); - assertThat(result.deleteFiles()).hasSize(1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb"))); - } - - @TestTemplate - public void testPartitionedTableWithDataAndIdAsKey() throws IOException { - createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "aaa")); - - writer.write(createDelete(2, "aaa")); // 1 pos-delete. - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).hasSize(1); - assertThat(result.deleteFiles()).hasSize(1); - assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); - commitTransaction(result); - - assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); - } - - @TestTemplate - public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { - Schema tableSchema = - new Schema( - required(3, "id", Types.IntegerType.get()), - required(4, "ts", Types.TimestampType.withZone())); - RowType flinkType = - new RowType( - false, - ImmutableList.of( - new RowType.RowField("id", new IntType()), - new RowType.RowField("ts", new LocalZonedTimestampType(3)))); - - this.table = create(tableSchema, PartitionSpec.unpartitioned()); - initTable(table); - - List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - RowDataSerializer serializer = new RowDataSerializer(flinkType); - OffsetDateTime start = OffsetDateTime.now(); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); - - WriteResult result = writer.complete(); - // One data file - assertThat(result.dataFiles()).hasSize(1); - // One eq delete file + one pos delete file - assertThat(result.deleteFiles()).hasSize(2); - assertThat( - Arrays.stream(result.deleteFiles()) - .map(ContentFile::content) - .collect(Collectors.toSet())) - .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); - commitTransaction(result); - - Record expectedRecord = GenericRecord.create(tableSchema); - expectedRecord.setField("id", 1); - int cutPrecisionNano = start.getNano() / 1000000 * 1000000; - expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - - assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); - } - - private void commitTransaction(WriteResult result) { - RowDelta rowDelta = table.newRowDelta(); - Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); - Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); - rowDelta - .validateDeletedFiles() - .validateDataFilesExist(Lists.newArrayList(result.referencedDataFiles())) - .commit(); - } - - private StructLikeSet expectedRowSet(Record... records) { - return SimpleDataUtil.expectedRowSet(table, records); - } - - private StructLikeSet actualRowSet(String... columns) throws IOException { - return SimpleDataUtil.actualRowSet(table, columns); - } - - private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { - return new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - FlinkSchemaUtil.convert(table.schema()), - 128 * 1024 * 1024, - format, - table.properties(), - equalityFieldIds, - false); - } - - private TaskWriterFactory createTaskWriterFactory( - RowType flinkType, List equalityFieldIds) { - return new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - flinkType, - 128 * 1024 * 1024, - format, - table.properties(), - equalityFieldIds, - true); - } - - private void createAndInitTable(boolean partitioned) { - if (partitioned) { - this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); - } else { - this.table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - initTable(table); - } - - private void initTable(TestTables.TestTable testTable) { - testTable - .updateProperties() - .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) - .defaultFormat(format) - .commit(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java deleted file mode 100644 index dd89f43483b0..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java +++ /dev/null @@ -1,65 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TestAppenderFactory; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkAppenderFactory extends TestAppenderFactory { - - private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); - - @Override - protected FileAppenderFactory createAppenderFactory( - List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { - return new FlinkAppenderFactory( - table, - table.schema(), - rowType, - table.properties(), - table.spec(), - ArrayUtil.toIntArray(equalityFieldIds), - eqDeleteSchema, - posDeleteRowSchema); - } - - @Override - protected RowData createRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet expectedRowSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(rowType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java deleted file mode 100644 index 414ee40d1357..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ /dev/null @@ -1,66 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestFileWriterFactory; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java deleted file mode 100644 index 11a73d2cc144..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ /dev/null @@ -1,397 +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.sink; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -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.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -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.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - - private final FileFormat format; - private final int parallelism; - private final boolean partitioned; - - @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", 1, true}, - {"avro", 1, false}, - {"avro", 2, true}, - {"avro", 2, false}, - {"orc", 1, true}, - {"orc", 1, false}, - {"orc", 2, true}, - {"orc", 2, false}, - {"parquet", 1, true}, - {"parquet", 1, false}, - {"parquet", 2, true}, - {"parquet", 2, false} - }; - } - - public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - } - - @Before - public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testWriteRowData() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - - @Test - public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); - } - - @Test - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @Test - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); - } - } - } - - @Test - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @Test - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); - } - } - - @Test - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); - } - } - - @Test - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); - } - } - - @Test - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - catalogResource - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - catalogResource - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); - Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); - rightTable.refresh(); - Assert.assertEquals( - TestFlinkIcebergSink.class.getName(), - rightTable.currentSnapshot().summary().get("flink.test")); - Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); - } - - @Test - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - Assertions.assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @Test - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - Assertions.assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @Test - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java deleted file mode 100644 index b38aa6b50ce6..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.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.flink.sink; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.types.Row; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class TestFlinkIcebergSinkBase { - - protected Table table; - protected StreamExecutionEnvironment env; - protected static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - protected static final DataFormatConverters.RowConverter CONVERTER = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - protected BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); - } - - protected List createRows(String prefix) { - return Lists.newArrayList( - Row.of(1, prefix + "aaa"), - Row.of(1, prefix + "bbb"), - Row.of(1, prefix + "ccc"), - Row.of(2, prefix + "aaa"), - Row.of(2, prefix + "bbb"), - Row.of(2, prefix + "ccc"), - Row.of(3, prefix + "aaa"), - Row.of(3, prefix + "bbb"), - Row.of(3, prefix + "ccc")); - } - - protected List convertToRowData(List rows) { - return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java deleted file mode 100644 index 16b4542b00d3..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ /dev/null @@ -1,137 +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.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; - private TableLoader tableLoader; - - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkBranch(String branch) { - this.branch = branch; - } - - @Before - public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - FileFormat.AVRO.name(), - TableProperties.FORMAT_VERSION, - "1")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - verifyOtherBranchUnmodified(); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .toBranch(branch) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); - SimpleDataUtil.assertTableRows( - table, - ImmutableList.of(), - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); - - verifyOtherBranchUnmodified(); - } - - private void verifyOtherBranchUnmodified() { - String otherBranch = - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; - if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); - } - - Assert.assertTrue(table.snapshot(otherBranch) == null); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java deleted file mode 100644 index b5c3bcf41734..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ /dev/null @@ -1,236 +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.sink; - -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -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.Sets; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Parameterized.Parameters( - name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} - }; - } - - public TestFlinkIcebergSinkV2( - String format, int parallelism, boolean partitioned, String writeDistributionMode) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - this.writeDistributionMode = writeDistributionMode; - } - - @Before - public void setupTable() { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - format.name(), - TableProperties.FORMAT_VERSION, - String.valueOf(FORMAT_V2))); - - table - .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) - .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) - .commit(); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100L) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testCheckAndGetEqualityFieldIds() { - table - .updateSchema() - .allowIncompatibleChanges() - .addRequiredColumn("type", Types.StringType.get()) - .setIdentifierFields("type") - .commit(); - - DataStream dataStream = - env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); - - // Use schema identifier field IDs as equality field id list by default - Assert.assertEquals( - table.schema().identifierFieldIds(), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); - - // Use user-provided equality field column as equality field id list - builder.equalityFieldColumns(Lists.newArrayList("id")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("id").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); - - builder.equalityFieldColumns(Lists.newArrayList("type")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("type").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); - } - - @Test - public void testChangeLogOnIdKey() throws Exception { - testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertOnlyDeletesOnDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); - - List> expectedRecords = - ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - true, - elementsPerCheckpoint, - expectedRecords, - SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testChangeLogOnDataKey() throws Exception { - testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testChangeLogOnIdDataKey() throws Exception { - testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testChangeLogOnSameKey() throws Exception { - testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertModeCheck() throws Exception { - DataStream dataStream = - env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .tableLoader(tableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .upsert(true); - - Assertions.assertThatThrownBy( - () -> - builder - .equalityFieldColumns(ImmutableList.of("id", "data")) - .overwrite(true) - .append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - - Assertions.assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); - } - - @Test - public void testUpsertOnIdKey() throws Exception { - testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertOnDataKey() throws Exception { - testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); - } - - @Test - public void testUpsertOnIdDataKey() throws Exception { - testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java deleted file mode 100644 index 507df9e35215..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ /dev/null @@ -1,361 +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.sink; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -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.types.RowKind; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.BoundedTestSource; -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.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; -import org.junit.Assert; - -public class TestFlinkIcebergSinkV2Base { - - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; - - protected int parallelism = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; - protected FileFormat format; - protected boolean partitioned; - protected String writeDistributionMode; - - protected static final Map ROW_KIND_MAP = - ImmutableMap.of( - "+I", RowKind.INSERT, - "-D", RowKind.DELETE, - "-U", RowKind.UPDATE_BEFORE, - "+U", RowKind.UPDATE_AFTER); - - protected Row row(String rowKind, int id, String data) { - RowKind kind = ROW_KIND_MAP.get(rowKind); - if (kind == null) { - throw new IllegalArgumentException("Unknown row kind: " + rowKind); - } - - return Row.ofKind(kind, id, data); - } - - protected void testUpsertOnIdDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 2, "bbb"), row("+I", 2, "ccc")), - ImmutableList.of(row("+U", 1, "bbb"), row("-U", 1, "ccc"), row("-D", 1, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "aaa"), record(2, "bbb")), - ImmutableList.of(record(1, "aaa"), record(2, "ccc")), - ImmutableList.of(record(1, "bbb"), record(2, "ccc"))); - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnIdDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 2, "bbb"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa")), - ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "aaa"), record(2, "bbb")), - ImmutableList.of( - record(1, "aaa"), record(1, "bbb"), record(1, "ccc"), record(2, "bbb")), - ImmutableList.of( - record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "bbb"))); - - testChangeLogs( - ImmutableList.of("data", "id"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnSameKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - // Checkpoint #1 - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa")), - // Checkpoint #2 - ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa")), - // Checkpoint #3 - ImmutableList.of(row("-D", 1, "aaa"), row("+I", 1, "aaa")), - // Checkpoint #4 - ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 1, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa"), record(1, "aaa"))); - - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 2, "bbb"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa")), - ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"), row("+I", 2, "ccc"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "aaa")), - ImmutableList.of(record(1, "aaa"), record(1, "bbb"), record(1, "ccc")), - ImmutableList.of( - record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "ccc"))); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testUpsertOnDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), - ImmutableList.of(row("+U", 4, "aaa"), row("-U", 3, "bbb"), row("+U", 5, "bbb")), - ImmutableList.of(row("+I", 6, "aaa"), row("+U", 7, "bbb"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(2, "aaa"), record(3, "bbb")), - ImmutableList.of(record(4, "aaa"), record(5, "bbb")), - ImmutableList.of(record(6, "aaa"), record(7, "bbb"))); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnIdKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa"), - row("-D", 2, "aaa"), - row("+I", 2, "bbb")), - ImmutableList.of( - row("-U", 2, "bbb"), row("+U", 2, "ccc"), row("-D", 2, "ccc"), row("+I", 2, "ddd")), - ImmutableList.of( - row("-D", 1, "bbb"), - row("+I", 1, "ccc"), - row("-D", 1, "ccc"), - row("+I", 1, "ddd"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "bbb")), - ImmutableList.of(record(1, "bbb"), record(2, "ddd")), - ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); - - if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { - Assertions.assertThatThrownBy( - () -> - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch)) - .isInstanceOf(IllegalStateException.class) - .hasMessageStartingWith( - "In 'hash' distribution mode with equality fields set, partition field") - .hasMessageContaining("should be included in equality fields:"); - } else { - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - } - - protected void testUpsertOnIdKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), - ImmutableList.of(row("+I", 1, "ccc")), - ImmutableList.of(row("+U", 1, "ddd"), row("+I", 1, "eee"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb")), - ImmutableList.of(record(1, "ccc")), - ImmutableList.of(record(1, "eee"))); - - if (!partitioned) { - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } else { - Assertions.assertThatThrownBy( - () -> - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch)) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("should be included in equality fields"); - } - } - - protected void testChangeLogs( - List equalityFieldColumns, - KeySelector keySelector, - boolean insertAsUpsert, - List> elementsPerCheckpoint, - List> expectedRecordsPerCheckpoint, - String branch) - throws Exception { - DataStream dataStream = - env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .tableLoader(tableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .equalityFieldColumns(equalityFieldColumns) - .upsert(insertAsUpsert) - .toBranch(branch) - .append(); - - // Execute the program. - env.execute("Test Iceberg Change-Log DataStream."); - - table.refresh(); - List snapshots = findValidSnapshots(); - int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); - - for (int i = 0; i < expectedSnapshotNum; i++) { - long snapshotId = snapshots.get(i).snapshotId(); - List expectedRecords = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(expectedRecords.toArray(new Record[0])), - actualRowSet(snapshotId, "*")); - } - } - - protected Record record(int id, String data) { - return SimpleDataUtil.createRecord(id, data); - } - - private List findValidSnapshots() { - List validSnapshots = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests(table.io()).stream() - .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { - validSnapshots.add(snapshot); - } - } - return validSnapshots; - } - - private StructLikeSet expectedRowSet(Record... records) { - return SimpleDataUtil.expectedRowSet(table, records); - } - - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { - table.refresh(); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table).useSnapshot(snapshotId).select(columns).build()) { - reader.forEach(set::add); - } - return set; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java deleted file mode 100644 index fed333848279..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ /dev/null @@ -1,134 +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.sink; - -import java.io.IOException; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; - - @Parameterized.Parameters(name = "branch = {0}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkV2Branch(String branch) { - this.branch = branch; - } - - @Before - public void before() throws IOException { - table = - catalogResource - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - FileFormat.AVRO.name(), - TableProperties.FORMAT_VERSION, - "2")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100); - - tableLoader = catalogResource.tableLoader(); - } - - @Test - public void testChangeLogOnIdKey() throws Exception { - testChangeLogOnIdKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testChangeLogOnDataKey() throws Exception { - testChangeLogOnDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testChangeLogOnIdDataKey() throws Exception { - testChangeLogOnIdDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testUpsertOnIdKey() throws Exception { - testUpsertOnIdKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testUpsertOnDataKey() throws Exception { - testUpsertOnDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @Test - public void testUpsertOnIdDataKey() throws Exception { - testUpsertOnIdDataKey(branch); - verifyOtherBranchUnmodified(); - } - - private void verifyOtherBranchUnmodified() { - String otherBranch = - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; - if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); - } - - Assert.assertTrue(table.snapshot(otherBranch) == null); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java deleted file mode 100644 index ce1f208a4b07..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ /dev/null @@ -1,315 +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.sink; - -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Paths; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkManifest { - private static final Configuration CONF = new Configuration(); - - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private Table table; - private FileAppenderFactory appenderFactory; - private final AtomicInteger fileCount = new AtomicInteger(0); - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - String warehouse = folder.getAbsolutePath(); - - String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); - - // Construct the iceberg table. - table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); - - int[] equalityFieldIds = - new int[] { - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() - }; - this.appenderFactory = - new FlinkAppenderFactory( - table, - table.schema(), - FlinkSchemaUtil.convert(table.schema()), - table.properties(), - table.spec(), - equalityFieldIds, - table.schema(), - null); - } - - @Test - public void testIO() throws IOException { - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - for (long checkpointId = 1; checkpointId <= 3; checkpointId++) { - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - final long curCkpId = checkpointId; - - List dataFiles = generateDataFiles(10); - List eqDeleteFiles = generateEqDeleteFiles(5); - List posDeleteFiles = generatePosDeleteFiles(5); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder() - .addDataFiles(dataFiles) - .addDeleteFiles(eqDeleteFiles) - .addDeleteFiles(posDeleteFiles) - .build(), - () -> factory.create(curCkpId), - table.spec()); - - WriteResult result = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); - for (int i = 0; i < dataFiles.size(); i++) { - TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); - } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); - for (int i = 0; i < 5; i++) { - TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); - } - for (int i = 0; i < 5; i++) { - TestHelpers.assertEquals(posDeleteFiles.get(i), result.deleteFiles()[5 + i]); - } - } - } - - @Test - public void testUserProvidedManifestLocation() throws IOException { - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); - Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); - ManifestOutputFileFactory factory = - new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(5); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder().addDataFiles(dataFiles).build(), - () -> factory.create(checkpointId), - table.spec()); - - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); - - WriteResult result = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); - - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); - for (int i = 0; i < dataFiles.size(); i++) { - TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); - } - } - - @Test - public void testVersionedSerializer() throws IOException { - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(10); - List eqDeleteFiles = generateEqDeleteFiles(10); - List posDeleteFiles = generatePosDeleteFiles(10); - DeltaManifests expected = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder() - .addDataFiles(dataFiles) - .addDeleteFiles(eqDeleteFiles) - .addDeleteFiles(posDeleteFiles) - .build(), - () -> factory.create(checkpointId), - table.spec()); - - byte[] versionedSerializeData = - SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, expected); - DeltaManifests actual = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, versionedSerializeData); - TestHelpers.assertEquals(expected.dataManifest(), actual.dataManifest()); - TestHelpers.assertEquals(expected.deleteManifest(), actual.deleteManifest()); - - byte[] versionedSerializeData2 = - SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); - } - - @Test - public void testCompatibility() throws IOException { - // The v2 deserializer should be able to deserialize the v1 binary. - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(10); - ManifestFile manifest = - FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); - byte[] dataV1 = - SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); - - DeltaManifests delta = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); - TestHelpers.assertEquals(manifest, delta.dataManifest()); - - List actualFiles = - FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); - for (int i = 0; i < 10; i++) { - TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); - } - } - - private static class V1Serializer implements SimpleVersionedSerializer { - - @Override - public int getVersion() { - return 1; - } - - @Override - public byte[] serialize(ManifestFile m) throws IOException { - return ManifestFiles.encode(m); - } - - @Override - public ManifestFile deserialize(int version, byte[] serialized) throws IOException { - return ManifestFiles.decode(serialized); - } - } - - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - table.spec(), - CONF, - table.location(), - FileFormat.PARQUET.addExtension(filename), - rows); - } - - private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { - return SimpleDataUtil.writeEqDeleteFile( - table, FileFormat.PARQUET, filename, appenderFactory, deletes); - } - - private DeleteFile writePosDeleteFile(String filename, List> positions) - throws IOException { - return SimpleDataUtil.writePosDeleteFile( - table, FileFormat.PARQUET, filename, appenderFactory, positions); - } - - private List generateDataFiles(int fileNum) throws IOException { - List rowDataList = Lists.newArrayList(); - List dataFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - rowDataList.add(SimpleDataUtil.createRowData(i, "a" + i)); - dataFiles.add(writeDataFile("data-file-" + fileCount.incrementAndGet(), rowDataList)); - } - return dataFiles; - } - - private List generateEqDeleteFiles(int fileNum) throws IOException { - List rowDataList = Lists.newArrayList(); - List deleteFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - rowDataList.add(SimpleDataUtil.createDelete(i, "a" + i)); - deleteFiles.add( - writeEqDeleteFile("eq-delete-file-" + fileCount.incrementAndGet(), rowDataList)); - } - return deleteFiles; - } - - private List generatePosDeleteFiles(int fileNum) throws IOException { - List> positions = Lists.newArrayList(); - List deleteFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - positions.add(Pair.of("data-file-1", (long) i)); - deleteFiles.add( - writePosDeleteFile("pos-delete-file-" + fileCount.incrementAndGet(), positions)); - } - return deleteFiles; - } - - private static String newFlinkJobId() { - return UUID.randomUUID().toString(); - } - - private static String newOperatorUniqueId() { - return UUID.randomUUID().toString(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java deleted file mode 100644 index f79a3e634071..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ /dev/null @@ -1,66 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestPartitioningWriters; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java deleted file mode 100644 index 3050752d1c24..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ /dev/null @@ -1,66 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestPositionDeltaWriters; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java deleted file mode 100644 index 03051b69cf87..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ /dev/null @@ -1,51 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestRollingFileWriters; -import org.apache.iceberg.util.ArrayUtil; - -public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java deleted file mode 100644 index e6d64ef2c720..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java +++ /dev/null @@ -1,60 +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.sink; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestWriterMetrics; - -public class TestFlinkWriterMetrics extends TestWriterMetrics { - - public TestFlinkWriterMetrics(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileWriterFactory newWriterFactory(Table sourceTable) { - return FlinkFileWriterFactory.builderFor(sourceTable) - .dataSchema(sourceTable.schema()) - .dataFileFormat(fileFormat) - .deleteFileFormat(fileFormat) - .positionDeleteRowSchema(sourceTable.schema()) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data, boolean boolValue, Long longValue) { - GenericRowData nested = GenericRowData.of(boolValue, longValue); - GenericRowData row = GenericRowData.of(id, StringData.fromString(data), nested); - return row; - } - - @Override - public RowData toGenericRow(int value, int repeated) { - GenericRowData row = new GenericRowData(repeated); - for (int i = 0; i < repeated; i++) { - row.setField(i, value); - } - return row; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java deleted file mode 100644 index 06b6bc9a977b..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ /dev/null @@ -1,1148 +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.sink; - -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; -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.Files; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.NavigableMap; -import java.util.SortedMap; -import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.GenericManifestFile; -import org.apache.iceberg.ManifestContent; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.WriteResult; -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.util.Pair; -import org.apache.iceberg.util.ThreadPools; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestIcebergFilesCommitter extends TestBase { - private static final Configuration CONF = new Configuration(); - - private File flinkManifestFolder; - - @Parameter(index = 1) - private FileFormat format; - - @Parameter(index = 2) - private String branch; - - @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") - protected static List parameters() { - return Arrays.asList( - new Object[] {1, FileFormat.AVRO, "main"}, - new Object[] {2, FileFormat.AVRO, "test-branch"}, - new Object[] {1, FileFormat.PARQUET, "main"}, - new Object[] {2, FileFormat.PARQUET, "test-branch"}, - new Object[] {1, FileFormat.ORC, "main"}, - new Object[] {2, FileFormat.ORC, "test-branch"}); - } - - @Override - @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()); - - table - .updateProperties() - .set(DEFAULT_FILE_FORMAT, format.name()) - .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) - .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") - .commit(); - } - - @TestTemplate - public void testCommitTxnWithoutDataFiles() throws Exception { - long checkpointId = 0; - long timestamp = 0; - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. - for (int i = 1; i <= 3; i++) { - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(0); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - } - - @TestTemplate - public void testMaxContinuousEmptyCommits() throws Exception { - table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); - - JobID jobId = new JobID(); - long checkpointId = 0; - long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - - assertSnapshotSize(0); - - for (int i = 1; i <= 9; i++) { - harness.snapshot(++checkpointId, ++timestamp); - harness.notifyOfCompletedCheckpoint(checkpointId); - - assertSnapshotSize(i / 3); - } - } - } - - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); - } - - @TestTemplate - public void testCommitTxn() throws Exception { - // Test with 3 continues checkpoints: - // 1. snapshotState for checkpoint#1 - // 2. notifyCheckpointComplete for checkpoint#1 - // 3. snapshotState for checkpoint#2 - // 4. notifyCheckpointComplete for checkpoint#2 - // 5. snapshotState for checkpoint#3 - // 6. notifyCheckpointComplete for checkpoint#3 - long timestamp = 0; - - JobID jobID = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - - List rows = Lists.newArrayListWithExpectedSize(3); - for (int i = 1; i <= 3; i++) { - RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); - DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - - harness.snapshot(i, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(i); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobID, operatorId, i); - assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) - .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); - } - } - } - - @TestTemplate - public void testOrderedEventsBetweenCheckpoints() throws Exception { - // It's possible that two checkpoints happen in the following orders: - // 1. snapshotState for checkpoint#1; - // 2. snapshotState for checkpoint#2; - // 3. notifyCheckpointComplete for checkpoint#1; - // 4. notifyCheckpointComplete for checkpoint#2; - long timestamp = 0; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; - harness.snapshot(firstCheckpointId, ++timestamp); - assertFlinkManifests(1); - - RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; - harness.snapshot(secondCheckpointId, ++timestamp); - assertFlinkManifests(2); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(firstCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, firstCheckpointId); - assertFlinkManifests(1); - - // 4. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(secondCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testDisorderedEventsBetweenCheckpoints() throws Exception { - // It's possible that the two checkpoints happen in the following orders: - // 1. snapshotState for checkpoint#1; - // 2. snapshotState for checkpoint#2; - // 3. notifyCheckpointComplete for checkpoint#2; - // 4. notifyCheckpointComplete for checkpoint#1; - long timestamp = 0; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; - harness.snapshot(firstCheckpointId, ++timestamp); - assertFlinkManifests(1); - - RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; - harness.snapshot(secondCheckpointId, ++timestamp); - assertFlinkManifests(2); - - // 3. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(secondCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - - // 4. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(firstCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testRecoveryFromValidSnapshot() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List expectedRows = Lists.newArrayList(); - OperatorSubtaskState snapshot; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row = SimpleDataUtil.createRowData(1, "hello"); - expectedRows.add(row); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row), branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - RowData row = SimpleDataUtil.createRowData(2, "world"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - - @TestTemplate - public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { - // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's - // possible that we - // flink job will restore from a checkpoint with only step#1 finished. - long checkpointId = 0; - long timestamp = 0; - OperatorSubtaskState snapshot; - List expectedRows = Lists.newArrayList(); - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row = SimpleDataUtil.createRowData(1, "hello"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - snapshot = harness.snapshot(++checkpointId, ++timestamp); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - assertFlinkManifests(1); - } - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - harness.snapshot(++checkpointId, ++timestamp); - // Did not write any new record, so it won't generate new manifest. - assertFlinkManifests(0); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - RowData row = SimpleDataUtil.createRowData(2, "world"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - } - - // Redeploying flink job from external checkpoint. - JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(newJobId)) { - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - assertMaxCommittedCheckpointId(newJobId, operatorId, -1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(3); - - RowData row = SimpleDataUtil.createRowData(3, "foo"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(newJobId, operatorId, checkpointId); - } - } - - @TestTemplate - public void testStartAnotherJobToWriteSameTable() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List rows = Lists.newArrayList(); - List tableRows = Lists.newArrayList(); - - JobID oldJobId = new JobID(); - OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(oldJobId)) { - harness.setup(); - harness.open(); - oldOperatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, -1L); - - for (int i = 1; i <= 3; i++) { - rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, checkpointId); - } - } - - // The new started job will start with checkpoint = 1 again. - checkpointId = 0; - timestamp = 0; - JobID newJobId = new JobID(); - OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(newJobId)) { - harness.setup(); - harness.open(); - newOperatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(3); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, 3); - assertMaxCommittedCheckpointId(newJobId, newOperatorId, -1); - - rows.add(SimpleDataUtil.createRowData(2, "world")); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(newJobId, newOperatorId, checkpointId); - } - } - - @TestTemplate - public void testMultipleJobsWriteSameTable() throws Exception { - long timestamp = 0; - List tableRows = Lists.newArrayList(); - - JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; - OperatorID[] operatorIds = - new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; - for (int i = 0; i < 20; i++) { - int jobIndex = i % 3; - int checkpointId = i / 3; - JobID jobId = jobs[jobIndex]; - OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.open(); - - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId == 0 ? -1 : checkpointId); - - List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(checkpointId + 1, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId + 1); - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(i + 1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId + 1); - } - } - } - - @TestTemplate - public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List expectedRows = Lists.newArrayList(); - OperatorSubtaskState snapshot1; - OperatorSubtaskState snapshot2; - - JobID jobId = new JobID(); - OperatorID operatorId1 = new OperatorID(); - OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { - harness1.getStreamConfig().setOperatorID(operatorId1); - harness1.setup(); - harness1.open(); - harness2.getStreamConfig().setOperatorID(operatorId2); - harness2.setup(); - harness2.open(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId1, -1L); - assertMaxCommittedCheckpointId(jobId, operatorId2, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); - expectedRows.add(row1); - DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); - - RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); - expectedRows.add(row2); - DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - - harness2.processElement(of(dataFile2), ++timestamp); - snapshot2 = harness2.snapshot(checkpointId, ++timestamp); - assertFlinkManifests(2); - - // Only notify one of the committers - harness1.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(1); - - // Only the first row is committed at this point - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, -1); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { - harness1.getStreamConfig().setOperatorID(operatorId1); - harness1.setup(); - harness1.initializeState(snapshot1); - harness1.open(); - - harness2.getStreamConfig().setOperatorID(operatorId2); - harness2.setup(); - harness2.initializeState(snapshot2); - harness2.open(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); - - RowData row1 = SimpleDataUtil.createRowData(2, "world1"); - expectedRows.add(row1); - DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); - - RowData row2 = SimpleDataUtil.createRowData(2, "world2"); - expectedRows.add(row2); - DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); - harness2.snapshot(checkpointId, ++timestamp); - - assertFlinkManifests(2); - - harness1.notifyOfCompletedCheckpoint(checkpointId); - harness2.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); - } - } - - @TestTemplate - public void testBoundedStream() throws Exception { - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertFlinkManifests(0); - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); - - DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); - ((BoundedOneInput) harness.getOneInputOperator()).endInput(); - - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) - .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); - } - } - - @TestTemplate - public void testFlinkManifests() throws Exception { - long timestamp = 0; - final long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - harness.snapshot(checkpoint, ++timestamp); - List manifestPaths = assertFlinkManifests(1); - Path manifestPath = manifestPaths.get(0); - assertThat(manifestPath.getFileName()) - .asString() - .isEqualTo( - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); - - // 2. Read the data files from manifests and assert. - List dataFiles = - FlinkManifestUtil.readDataFiles( - createTestingManifestFile(manifestPath), table.io(), table.specs()); - assertThat(dataFiles).hasSize(1); - TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testDeleteFiles() throws Exception { - assumeThat(formatVersion) - .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); - - long timestamp = 0; - long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - harness.snapshot(checkpoint, ++timestamp); - List manifestPaths = assertFlinkManifests(1); - Path manifestPath = manifestPaths.get(0); - assertThat(manifestPath.getFileName()) - .asString() - .isEqualTo( - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); - - // 2. Read the data files from manifests and assert. - List dataFiles = - FlinkManifestUtil.readDataFiles( - createTestingManifestFile(manifestPath), table.io(), table.specs()); - assertThat(dataFiles).hasSize(1); - TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - - // 4. process both data files and delete files. - RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); - - RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); - DeleteFile deleteFile1 = - writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), - ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - - // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); - assertFlinkManifests(2); - - // 6. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - assumeThat(formatVersion) - .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); - - long timestamp = 0; - long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); - RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); - RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); - DeleteFile deleteFile1 = - writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), - ++timestamp); - - // The 1th snapshotState. - harness.snapshot(checkpoint, ++timestamp); - - RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); - RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); - DeleteFile deleteFile2 = - writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), - ++timestamp); - - // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); - - // Notify the 2nd snapshot to complete. - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - assertThat(table.snapshots()).hasSize(2); - } - } - - @TestTemplate - public void testSpecEvolution() throws Exception { - long timestamp = 0; - int checkpointId = 0; - List rows = Lists.newArrayList(); - JobID jobId = new JobID(); - - OperatorID operatorId; - OperatorSubtaskState snapshot; - DataFile dataFile; - int specId; - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - - checkpointId++; - RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); - // table unpartitioned - dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - harness.snapshot(checkpointId, ++timestamp); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(table.spec().specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - - // Change partition spec - table.refresh(); - PartitionSpec oldSpec = table.spec(); - table.updateSpec().addField("id").commit(); - - checkpointId++; - rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); - // write data with old partition spec - dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - snapshot = harness.snapshot(checkpointId, ++timestamp); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(oldSpec.specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - SimpleDataUtil.assertTableRows(table, rows, branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - checkpointId++; - RowData row = SimpleDataUtil.createRowData(checkpointId, "world" + checkpointId); - StructLike partition = new PartitionData(table.spec().partitionType()); - partition.set(0, checkpointId); - dataFile = - writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); - rows.add(row); - harness.snapshot(checkpointId, ++timestamp); - assertFlinkManifests(1); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(table.spec().specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, rows, branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - - private int getStagingManifestSpecId(OperatorStateStore operatorStateStore, long checkPointId) - throws Exception { - ListState> checkpointsState = - operatorStateStore.getListState(IcebergFilesCommitter.buildStateDescriptor()); - NavigableMap statedDataFiles = - Maps.newTreeMap(checkpointsState.get().iterator().next()); - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, statedDataFiles.get(checkPointId)); - return deltaManifests.dataManifest().partitionSpecId(); - } - - private DeleteFile writeEqDeleteFile( - FileAppenderFactory appenderFactory, String filename, List deletes) - throws IOException { - return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, deletes); - } - - private DeleteFile writePosDeleteFile( - FileAppenderFactory appenderFactory, - String filename, - List> positions) - throws IOException { - return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, positions); - } - - private FileAppenderFactory createDeletableAppenderFactory() { - int[] equalityFieldIds = - new int[] { - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() - }; - return new FlinkAppenderFactory( - table, - table.schema(), - FlinkSchemaUtil.convert(table.schema()), - table.properties(), - table.spec(), - equalityFieldIds, - table.schema(), - null); - } - - private ManifestFile createTestingManifestFile(Path manifestPath) { - return new GenericManifestFile( - manifestPath.toAbsolutePath().toString(), - manifestPath.toFile().length(), - 0, - ManifestContent.DATA, - 0, - 0, - 0L, - 0, - 0, - 0, - 0, - 0, - 0, - null, - null); - } - - private List assertFlinkManifests(int expectedCount) throws IOException { - List manifests = - Files.list(flinkManifestFolder.toPath()) - .filter(p -> !p.toString().endsWith(".crc")) - .collect(Collectors.toList()); - assertThat(manifests).hasSize(expectedCount); - return manifests; - } - - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - table.spec(), - CONF, - table.location(), - format.addExtension(filename), - rows); - } - - private DataFile writeDataFile( - String filename, List rows, PartitionSpec spec, StructLike partition) - throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - spec, - CONF, - table.location(), - format.addExtension(filename), - rows, - partition); - } - - private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { - table.refresh(); - long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); - assertThat(actualId).isEqualTo(expectedId); - } - - private void assertSnapshotSize(int expectedSnapshotSize) { - table.refresh(); - assertThat(table.snapshots()).hasSize(expectedSnapshotSize); - } - - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) - throws Exception { - TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); - return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); - } - - private static MockEnvironment createEnvironment(JobID jobID) { - return new MockEnvironmentBuilder() - .setTaskName("test task") - .setManagedMemorySize(32 * 1024) - .setInputSplitProvider(new MockInputSplitProvider()) - .setBufferSize(256) - .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) - .setExecutionConfig(new ExecutionConfig()) - .setMaxParallelism(16) - .setJobID(jobID) - .build(); - } - - private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { - private final String tablePath; - private final String branch; - private final PartitionSpec spec; - - private TestOperatorFactory(String tablePath, String branch, PartitionSpec spec) { - this.tablePath = tablePath; - this.branch = branch; - this.spec = spec; - } - - private static TestOperatorFactory of(String tablePath, String branch, PartitionSpec spec) { - return new TestOperatorFactory(tablePath, branch, spec); - } - - @Override - @SuppressWarnings("unchecked") - public > T createStreamOperator( - StreamOperatorParameters param) { - IcebergFilesCommitter committer = - new IcebergFilesCommitter( - new TestTableLoader(tablePath), - false, - Collections.singletonMap("flink.test", TestIcebergFilesCommitter.class.getName()), - ThreadPools.WORKER_THREAD_POOL_SIZE, - branch, - spec); - committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); - return (T) committer; - } - - @Override - public Class getStreamOperatorClass(ClassLoader classLoader) { - return IcebergFilesCommitter.class; - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java deleted file mode 100644 index 0968f89f55e0..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ /dev/null @@ -1,388 +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.sink; - -import java.io.File; -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Set; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.WriteResult; -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.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); - - private Table table; - - private final FileFormat format; - private final boolean partitioned; - - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} - }; - } - - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - // Construct the iceberg table. - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); - } - - @Test - public void testWritingTable() throws Exception { - long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - // The first checkpoint - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); - - testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - checkpointId = checkpointId + 1; - - // The second checkpoint - testHarness.processElement(SimpleDataUtil.createRowData(4, "foo"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(5, "bar"), 2); - - testHarness.prepareSnapshotPreBarrier(checkpointId); - expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - // Commit the iceberg transaction. - AppendFiles appendFiles = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - - // Assert the table records. - SimpleDataUtil.assertTableRecords( - table, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), - SimpleDataUtil.createRecord(2, "world"), - SimpleDataUtil.createRecord(3, "hello"), - SimpleDataUtil.createRecord(4, "foo"), - SimpleDataUtil.createRecord(5, "bar"))); - } - } - - @Test - public void testSnapshotTwice() throws Exception { - long checkpointId = 1; - long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); - - testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - // snapshot again immediately. - for (int i = 0; i < 5; i++) { - testHarness.prepareSnapshotPreBarrier(checkpointId++); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - } - } - } - - @Test - public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); - } - // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); - } - // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); - } - - private Set scanDataFiles() throws IOException { - Path dataDir = new Path(table.location(), "data"); - FileSystem fs = FileSystem.get(new Configuration()); - if (!fs.exists(dataDir)) { - return ImmutableSet.of(); - } else { - Set paths = Sets.newHashSet(); - RemoteIterator iterators = fs.listFiles(dataDir, true); - while (iterators.hasNext()) { - LocatedFileStatus status = iterators.next(); - if (status.isFile()) { - Path path = status.getPath(); - if (path.getName().endsWith("." + format.toString().toLowerCase())) { - paths.add(path.toString()); - } - } - } - return paths; - } - } - - @Test - public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - - Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); - ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - } - } - - @Test - public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - - testHarness.endInput(); - - long expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - - testHarness.prepareSnapshotPreBarrier(1L); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier - // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); - } - } - - @Test - public void testTableWithTargetFileSize() throws Exception { - // Adjust the target-file-size in table properties. - table - .updateProperties() - .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger - .commit(); - - List rows = Lists.newArrayListWithCapacity(8000); - List records = Lists.newArrayListWithCapacity(8000); - for (int i = 0; i < 2000; i++) { - for (String data : new String[] {"a", "b", "c", "d"}) { - rows.add(SimpleDataUtil.createRowData(i, data)); - records.add(SimpleDataUtil.createRecord(i, data)); - } - } - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - for (RowData row : rows) { - testHarness.processElement(row, 1); - } - - // snapshot the operator. - testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); - - // Assert that the data file have the expected records. - for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); - } - - // Commit the iceberg transaction. - AppendFiles appendFiles = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - } - - // Assert the table records. - SimpleDataUtil.assertTableRecords(table, records); - } - - @Test - public void testPromotedFlinkDataType() throws Exception { - Schema iSchema = - new Schema( - Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), - Types.NestedField.required(2, "smallint", Types.IntegerType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get())); - TableSchema flinkSchema = - TableSchema.builder() - .field("tinyint", DataTypes.TINYINT().notNull()) - .field("smallint", DataTypes.SMALLINT().notNull()) - .field("int", DataTypes.INT().nullable()) - .build(); - - PartitionSpec spec; - if (partitioned) { - spec = - PartitionSpec.builderFor(iSchema) - .identity("smallint") - .identity("tinyint") - .identity("int") - .build(); - } else { - spec = PartitionSpec.unpartitioned(); - } - - String location = tempFolder.newFolder().getAbsolutePath(); - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); - - List rows = - Lists.newArrayList( - GenericRowData.of((byte) 0x01, (short) -32768, 101), - GenericRowData.of((byte) 0x02, (short) 0, 102), - GenericRowData.of((byte) 0x03, (short) 32767, 103)); - - Record record = GenericRecord.create(iSchema); - List expected = - Lists.newArrayList( - record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), - record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), - record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter(icebergTable, flinkSchema)) { - for (RowData row : rows) { - testHarness.processElement(row, 1); - } - testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); - - // Commit the iceberg transaction. - AppendFiles appendFiles = icebergTable.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - } - - SimpleDataUtil.assertTableRecords(location, expected); - } - - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() - throws Exception { - return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); - } - - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema) throws Exception { - RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); - FlinkWriteConf flinkWriteConfig = - new FlinkWriteConf( - icebergTable, Maps.newHashMap(), new org.apache.flink.configuration.Configuration()); - - IcebergStreamWriter streamWriter = - FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); - - harness.setup(); - harness.open(); - - return harness; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java deleted file mode 100644 index 79be96d20902..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ /dev/null @@ -1,252 +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.sink; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; - -public class TestRowDataPartitionKey { - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(0, "boolType", Types.BooleanType.get()), - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "longType", Types.LongType.get()), - Types.NestedField.required(3, "dateType", Types.DateType.get()), - Types.NestedField.required(4, "timeType", Types.TimeType.get()), - Types.NestedField.required(5, "stringType", Types.StringType.get()), - Types.NestedField.required(6, "timestampWithoutZone", Types.TimestampType.withoutZone()), - Types.NestedField.required(7, "timestampWithZone", Types.TimestampType.withZone()), - Types.NestedField.required(8, "fixedType", Types.FixedType.ofLength(5)), - Types.NestedField.required(9, "uuidType", Types.UUIDType.get()), - Types.NestedField.required(10, "binaryType", Types.BinaryType.get()), - Types.NestedField.required(11, "decimalType1", Types.DecimalType.of(18, 3)), - Types.NestedField.required(12, "decimalType2", Types.DecimalType.of(10, 5)), - Types.NestedField.required(13, "decimalType3", Types.DecimalType.of(38, 19)), - Types.NestedField.required(14, "floatType", Types.FloatType.get()), - Types.NestedField.required(15, "doubleType", Types.DoubleType.get())); - - private static final List SUPPORTED_PRIMITIVES = - SCHEMA.asStruct().fields().stream().map(Types.NestedField::name).collect(Collectors.toList()); - - private static final Schema NESTED_SCHEMA = - new Schema( - Types.NestedField.required( - 1, - "structType", - Types.StructType.of( - Types.NestedField.optional(2, "innerStringType", Types.StringType.get()), - Types.NestedField.optional(3, "innerIntegerType", Types.IntegerType.get())))); - - @Test - public void testNullPartitionValue() { - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); - - List rows = - Lists.newArrayList( - GenericRowData.of(1, StringData.fromString("a")), - GenericRowData.of(2, StringData.fromString("b")), - GenericRowData.of(3, null)); - - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - - for (RowData row : rows) { - PartitionKey partitionKey = new PartitionKey(spec, schema); - partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); - - String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); - } - } - - @Test - public void testPartitionWithOneNestedField() { - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); - List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1991); - List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); - - PartitionSpec spec1 = - PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerStringType").build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerIntegerType").build(); - - for (int i = 0; i < rows.size(); i++) { - RowData row = rows.get(i); - Record record = (Record) records.get(i).get(0); - - PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); - partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); - - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); - - PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); - partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); - - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); - } - } - - @Test - public void testPartitionMultipleNestedField() { - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); - List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1992); - List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); - - PartitionSpec spec1 = - PartitionSpec.builderFor(NESTED_SCHEMA) - .identity("structType.innerIntegerType") - .identity("structType.innerStringType") - .build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(NESTED_SCHEMA) - .identity("structType.innerStringType") - .identity("structType.innerIntegerType") - .build(); - - PartitionKey pk1 = new PartitionKey(spec1, NESTED_SCHEMA); - PartitionKey pk2 = new PartitionKey(spec2, NESTED_SCHEMA); - - for (int i = 0; i < rows.size(); i++) { - RowData row = rows.get(i); - Record record = (Record) records.get(i).get(0); - - pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); - - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); - - pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); - - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); - } - } - - @Test - public void testPartitionValueTypes() { - RowType rowType = FlinkSchemaUtil.convert(SCHEMA); - RowDataWrapper rowWrapper = new RowDataWrapper(rowType, SCHEMA.asStruct()); - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(SCHEMA.asStruct()); - - List records = RandomGenericData.generate(SCHEMA, 10, 1993); - List rows = Lists.newArrayList(RandomRowData.convert(SCHEMA, records)); - - for (String column : SUPPORTED_PRIMITIVES) { - PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity(column).build(); - Class[] javaClasses = spec.javaClasses(); - - PartitionKey pk = new PartitionKey(spec, SCHEMA); - PartitionKey expectedPK = new PartitionKey(spec, SCHEMA); - - for (int j = 0; j < rows.size(); j++) { - RowData row = rows.get(j); - Record record = records.get(j); - - pk.partition(rowWrapper.wrap(row)); - expectedPK.partition(recordWrapper.wrap(record)); - - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); - - if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); - } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); - } - } - } - } - - @Test - public void testNestedPartitionValues() { - Schema nestedSchema = new Schema(Types.NestedField.optional(1001, "nested", SCHEMA.asStruct())); - RowType rowType = FlinkSchemaUtil.convert(nestedSchema); - - RowDataWrapper rowWrapper = new RowDataWrapper(rowType, nestedSchema.asStruct()); - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(nestedSchema.asStruct()); - - List records = RandomGenericData.generate(nestedSchema, 10, 1994); - List rows = Lists.newArrayList(RandomRowData.convert(nestedSchema, records)); - - for (String supportedPrimitive : SUPPORTED_PRIMITIVES) { - String column = String.format("nested.%s", supportedPrimitive); - - PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity(column).build(); - Class[] javaClasses = spec.javaClasses(); - - PartitionKey pk = new PartitionKey(spec, nestedSchema); - PartitionKey expectedPK = new PartitionKey(spec, nestedSchema); - - for (int j = 0; j < rows.size(); j++) { - pk.partition(rowWrapper.wrap(rows.get(j))); - expectedPK.partition(recordWrapper.wrap(records.get(j))); - - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); - - if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); - } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); - } - } - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java deleted file mode 100644 index 112dbb511310..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ /dev/null @@ -1,243 +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.sink; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestTaskWriters { - private static final Configuration CONF = new Configuration(); - private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); - - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} - }; - } - - private final FileFormat format; - private final boolean partitioned; - - private Table table; - - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before - public void before() throws IOException { - File folder = tempFolder.newFolder(); - // Construct the iceberg table with the specified file format. - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); - } - - @Test - public void testWriteZeroRecord() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.close(); - - DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); - - // Close again. - taskWriter.close(); - dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); - } - } - - @Test - public void testCloseTwice() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - taskWriter.close(); // The first close - taskWriter.close(); // The second close - - int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); - } - } - } - - @Test - public void testAbort() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - - taskWriter.abort(); - DataFile[] dataFiles = taskWriter.dataFiles(); - - int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); - } - } - } - - @Test - public void testCompleteFiles() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "a")); - taskWriter.write(SimpleDataUtil.createRowData(2, "b")); - taskWriter.write(SimpleDataUtil.createRowData(3, "c")); - taskWriter.write(SimpleDataUtil.createRowData(4, "d")); - - DataFile[] dataFiles = taskWriter.dataFiles(); - int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); - - dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); - } - - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRecords( - table, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "b"), - SimpleDataUtil.createRecord(3, "c"), - SimpleDataUtil.createRecord(4, "d"))); - } - } - - @Test - public void testRollingWithTargetFileSize() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(4)) { - List rows = Lists.newArrayListWithCapacity(8000); - List records = Lists.newArrayListWithCapacity(8000); - for (int i = 0; i < 2000; i++) { - for (String data : new String[] {"a", "b", "c", "d"}) { - rows.add(SimpleDataUtil.createRowData(i, data)); - records.add(SimpleDataUtil.createRecord(i, data)); - } - } - - for (RowData row : rows) { - taskWriter.write(row); - } - - DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); - - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRecords(table, records); - } - } - - @Test - public void testRandomData() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); - for (RowData row : rows) { - taskWriter.write(row); - } - - taskWriter.close(); - DataFile[] dataFiles = taskWriter.dataFiles(); - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRows(table, Lists.newArrayList(rows)); - } - } - - private TaskWriter createTaskWriter(long targetFileSize) { - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), - targetFileSize, - format, - table.properties(), - null, - false); - taskWriterFactory.initialize(1, 1); - return taskWriterFactory.create(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index 890cc361b246..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,63 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Test; - -public class TestAggregatedStatistics { - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - @Test - public void mergeDataStatisticTest() { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>(1, statisticsSerializer); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(keyA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyB) - + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java deleted file mode 100644 index 4c64ce522201..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ /dev/null @@ -1,178 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; - -public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - private final SortKey keyA = sortKey.copy(); - private final SortKey keyB = sortKey.copy(); - - private AggregatedStatisticsTracker> - aggregatedStatisticsTracker; - - public TestAggregatedStatisticsTracker() { - keyA.set(0, "a"); - keyB.set(0, "b"); - } - - @Before - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } - - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); - } - - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - checkpoint2Subtask0DataStatistic.add(keyB); - checkpoint2Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); - } - - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(keyA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyA) - + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); - assertThat((long) globalDataStatistics.statistics().get(keyB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyB) - + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); - // Receive data statistics from all subtasks at checkpoint 2 - completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java deleted file mode 100644 index 3df714059c37..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ /dev/null @@ -1,178 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsCoordinator { - private static final String OPERATOR_NAME = "TestCoordinator"; - private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> - dataStatisticsCoordinator; - - @Before - public void before() throws Exception { - receivingTasks = EventReceivingTasks.createForRunningTasks(); - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); - } - - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - } - - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - } - - @Test - public void testDataStatisticsEventHandling() throws Exception { - tasksReady(); - SortKey key = sortKey.copy(); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify global data statistics is the aggregation of all subtasks data statistics - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - keyA, 2L, - keyB, 3L, - keyC, 5L)); - } - - static void setAllTasksReady( - int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, - EventReceivingTasks receivingTasks) { - for (int i = 0; i < subtasks; i++) { - dataStatisticsCoordinator.executionAttemptReady( - i, 0, receivingTasks.createGatewayForSubtask(i, 0)); - } - } - - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { - CompletableFuture future = new CompletableFuture<>(); - coordinator.callInCoordinatorThread( - () -> { - future.complete(null); - return null; - }, - "Coordinator fails to process action"); - - try { - future.get(); - } catch (InterruptedException e) { - throw new AssertionError("test interrupted"); - } catch (ExecutionException e) { - ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java deleted file mode 100644 index 5e0a752be506..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,143 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsCoordinatorProvider { - private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsCoordinatorProvider> provider; - private EventReceivingTasks receivingTasks; - - @Before - public void before() { - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); - receivingTasks = EventReceivingTasks.createForRunningTasks(); - } - - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - SortKey keyD = sortKey.copy(); - keyD.set(0, "c"); - SortKey keyE = sortKey.copy(); - keyE.set(0, "c"); - - try (RecreateOnResetOperatorCoordinator coordinator = - (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyD); - checkpoint2Subtask0DataStatistic.add(keyE); - checkpoint2Subtask0DataStatistic.add(keyE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); - waitForCheckpoint(2L, dataStatisticsCoordinator); - - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); - } - } - - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator> coordinator) - throws InterruptedException, ExecutionException { - CompletableFuture future = new CompletableFuture<>(); - coordinator.checkpointCoordinator(checkpointId, future); - return future.get(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java deleted file mode 100644 index 0e99a2d74ccb..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ /dev/null @@ -1,255 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.fs.CloseableRegistry; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; -import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; -import org.apache.flink.streaming.util.MockOutput; -import org.apache.flink.streaming.util.MockStreamConfig; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -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.types.Types; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; - -public class TestDataStatisticsOperator { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final RowType rowType = RowType.of(new VarCharType(), new IntType()); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer( - new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsOperator> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } - - @Before - public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); - } - - private DataStatisticsOperator> createOperator() { - MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return new DataStatisticsOperator<>( - "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); - } - - @After - public void clean() throws Exception { - operator.close(); - } - - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { - StateInitializationContext stateContext = getStateContext(); - operator.initializeState(stateContext); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); - - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); - - testHarness.endInput(); - } - } - - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - - List recordsOutput = - testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) - .collect(Collectors.toList()); - assertThat(recordsOutput) - .containsExactlyInAnyOrderElementsOf( - ImmutableList.of( - GenericRowData.of(StringData.fromString("a"), 2), - GenericRowData.of(StringData.fromString("b"), 3), - GenericRowData.of(StringData.fromString("b"), 1))); - } - } - - @Test - public void testRestoreState() throws Exception { - OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness1 = createHarness(this.operator)) { - MapDataStatistics mapDataStatistics = new MapDataStatistics(); - - SortKey key = sortKey.copy(); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "b"); - mapDataStatistics.add(key); - key.set(0, "c"); - mapDataStatistics.add(key); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - DataStatisticsEvent> event = - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); - operator.handleOperatorEvent(event); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf(expectedMap); - snapshot = testHarness1.snapshot(1L, 0); - } - - // Use the snapshot to initialize state for another new operator and then verify that the global - // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { - testHarness2.setup(); - testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - - // restored RowData is BinaryRowData. convert to GenericRowData for comparison - Map restoredStatistics = Maps.newHashMap(); - restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); - } - } - - private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); - AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); - CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); - OperatorStateStore operatorStateStore = - abstractStateBackend.createOperatorStateBackend( - env, "test-operator", Collections.emptyList(), cancelStreamRegistry); - return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); - } - - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - createHarness( - final DataStatisticsOperator> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); - harness.open(); - return harness; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java deleted file mode 100644 index a07808e935d9..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ /dev/null @@ -1,90 +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.sink.shuffle; - -import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; - -public class TestMapDataStatistics { - private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); - private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private final RowDataWrapper rowWrapper = - new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); - - @Test - public void testAddsAndGet() { - MapDataStatistics dataStatistics = new MapDataStatistics(); - - GenericRowData reusedRow = - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("c")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("a")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - Map actual = dataStatistics.statistics(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyA = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyB = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyC = sortKey.copy(); - - Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); - Assertions.assertThat(actual).isEqualTo(expected); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java deleted file mode 100644 index 92eb71acc834..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ /dev/null @@ -1,448 +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.sink.shuffle; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; -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.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.Test; - -public class TestMapRangePartitioner { - private static final SortOrder SORT_ORDER = - SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - - private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); - private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final SortKey[] SORT_KEYS = initSortKeys(); - - private static SortKey[] initSortKeys() { - SortKey[] sortKeys = new SortKey[10]; - for (int i = 0; i < 10; ++i) { - RowData rowData = - GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); - RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); - keyWrapper.wrap(rowData); - SortKey sortKey = SORT_KEY.copy(); - sortKey.wrap(keyWrapper); - sortKeys[i] = sortKey; - } - return sortKeys; - } - - // Total weight is 800 - private final MapDataStatistics mapDataStatistics = - new MapDataStatistics( - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L)); - - @Test - public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); - int numPartitions = 8; - - // each task should get targeted weight of 100 (=800/8) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), - SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), - SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), - SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), - SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight, second is the number of assigned keys - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(100L, 1), - 1, - Pair.of(100L, 1), - 2, - Pair.of(100L, 1), - 3, - Pair.of(100L, 2), - 4, - Pair.of(100L, 1), - 5, - Pair.of(100L, 2), - 6, - Pair.of(100L, 1), - 7, - Pair.of(100L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); - - Map>> partitionResults = - runPartitioner(partitioner, numPartitions); - validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); - } - - @Test - public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); - int numPartitions = 8; - - // target subtask weight is 100 before close file cost factored in. - // close file cost is 5 = 5% * 100. - // key weights before and after close file cost factored in - // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 - // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 - // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 - // target subtask weight with close cost per subtask is 110 (880/8) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), - SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), - SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), - SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), - SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight (excluding close file cost) for the subtask, - // second is the number of keys assigned to the subtask - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(105L, 1), - 1, - Pair.of(105L, 1), - 2, - Pair.of(105L, 1), - 3, - Pair.of(100L, 2), - 4, - Pair.of(105L, 1), - 5, - Pair.of(100L, 2), - 6, - Pair.of(100L, 2), - 7, - Pair.of(75L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); - - Map>> partitionResults = - runPartitioner(partitioner, numPartitions); - validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); - } - - @Test - public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); - int numPartitions = 9; - - // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 - // each task should get targeted weight of 89 = ceiling(800/9) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), - SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), - SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), - SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), - SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight, second is the number of assigned keys - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(89L, 1), - 1, - Pair.of(89L, 1), - 2, - Pair.of(89L, 1), - 3, - Pair.of(89L, 2), - 4, - Pair.of(89L, 1), - 5, - Pair.of(89L, 1), - 6, - Pair.of(89L, 2), - 7, - Pair.of(89L, 2), - 8, - Pair.of(88L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); - - Map>> partitionResults = - runPartitioner(partitioner, numPartitions); - validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); - } - - @Test - public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); - int numPartitions = 9; - - // target subtask weight is 89 before close file cost factored in. - // close file cost is 5 (= 5% * 89) per file. - // key weights before and after close file cost factored in - // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 - // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 - // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 - // target subtask weight per subtask is 98 ceiling(880/9) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), - SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), - SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), - SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), - SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight for the subtask, second is the number of keys - // assigned to the subtask - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(93L, 1), - 1, - Pair.of(93L, 1), - 2, - Pair.of(93L, 1), - 3, - Pair.of(88L, 2), - 4, - Pair.of(93L, 1), - 5, - Pair.of(93L, 1), - 6, - Pair.of(88L, 2), - 7, - Pair.of(88L, 2), - 8, - Pair.of(61L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); - - Map>> partitionResults = - runPartitioner(partitioner, numPartitions); - // drift threshold is high for non-dividable scenario with close cost - validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); - } - - private static Map>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { - // The Map key is the subtaskId. - // For the map value pair, the first element is the count of assigned and - // the second element of Set is for the set of assigned keys. - Map>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); - return partitionResults; - } - - /** @param expectedAssignmentInfo excluding closing cost */ - private void validatePartitionResults( - Map> expectedAssignmentInfo, - Map>> partitionResults, - double maxDriftPercentage) { - - Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); - - List expectedAssignedKeyCounts = - Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); - List actualAssignedKeyCounts = - Lists.newArrayListWithExpectedSize(partitionResults.size()); - List expectedNormalizedWeights = - Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); - List actualNormalizedWeights = - Lists.newArrayListWithExpectedSize(partitionResults.size()); - - long expectedTotalWeight = - expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); - expectedAssignmentInfo.forEach( - (subtaskId, pair) -> { - expectedAssignedKeyCounts.add(pair.second()); - expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); - }); - - long actualTotalWeight = - partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); - partitionResults.forEach( - (subtaskId, pair) -> { - actualAssignedKeyCounts.add(pair.second().size()); - actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); - }); - - // number of assigned keys should match exactly - Assertions.assertThat(actualAssignedKeyCounts) - .as("the number of assigned keys should match for every subtask") - .isEqualTo(expectedAssignedKeyCounts); - - // weight for every subtask shouldn't differ for more than some threshold relative to the - // expected weight - for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { - double expectedWeight = expectedNormalizedWeights.get(subtaskId); - double min = expectedWeight * (1 - maxDriftPercentage / 100); - double max = expectedWeight * (1 + maxDriftPercentage / 100); - Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) - .as( - "Subtask %d weight should within %.1f percent of the expected range %s", - subtaskId, maxDriftPercentage, expectedWeight) - .isBetween(min, max); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java deleted file mode 100644 index c7fea015142c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java +++ /dev/null @@ -1,65 +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.sink.shuffle; - -import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; - -public abstract class TestSortKeySerializerBase extends SerializerTestBase { - - protected abstract Schema schema(); - - protected abstract SortOrder sortOrder(); - - protected abstract GenericRowData rowData(); - - @Override - protected TypeSerializer createSerializer() { - return new SortKeySerializer(schema(), sortOrder()); - } - - @Override - protected int getLength() { - return -1; - } - - @Override - protected Class getTypeClass() { - return SortKey.class; - } - - @Override - protected SortKey[] getTestData() { - return new SortKey[] {sortKey()}; - } - - private SortKey sortKey() { - RowDataWrapper rowDataWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); - SortKey sortKey = new SortKey(schema(), sortOrder()); - sortKey.wrap(rowDataWrapper.wrap(rowData())); - return sortKey; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java deleted file mode 100644 index 0000688a8b55..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java +++ /dev/null @@ -1,55 +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.sink.shuffle; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.iceberg.NullOrder; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortDirection; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; - -public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { - private final DataGenerator generator = new DataGenerators.StructOfStruct(); - - @Override - protected Schema schema() { - return generator.icebergSchema(); - } - - @Override - protected SortOrder sortOrder() { - return SortOrder.builderFor(schema()) - .asc("row_id") - .sortBy( - Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) - .sortBy( - Expressions.truncate("struct_of_struct.person_struct.name", 16), - SortDirection.ASC, - NullOrder.NULLS_FIRST) - .build(); - } - - @Override - protected GenericRowData rowData() { - return generator.generateFlinkRowData(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java deleted file mode 100644 index 291302aef486..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink.shuffle; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.iceberg.NullOrder; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortDirection; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; - -public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { - private final DataGenerator generator = new DataGenerators.Primitives(); - - @Override - protected Schema schema() { - return generator.icebergSchema(); - } - - @Override - protected SortOrder sortOrder() { - return SortOrder.builderFor(schema()) - .asc("boolean_field") - .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) - .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) - // can not test HeapByteBuffer due to equality test inside SerializerTestBase - // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, - // NullOrder.NULLS_FIRST) - .build(); - } - - @Override - protected GenericRowData rowData() { - return generator.generateFlinkRowData(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java deleted file mode 100644 index a08578a4c106..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java +++ /dev/null @@ -1,170 +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.source; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import java.util.stream.Stream; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -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.streaming.api.functions.source.SourceFunction; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -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; - -public class BoundedTableFactory implements DynamicTableSourceFactory { - private static final AtomicInteger DATA_SET_ID = new AtomicInteger(0); - private static final Map>> DATA_SETS = Maps.newHashMap(); - - private static final ConfigOption DATA_ID = - ConfigOptions.key("data-id").stringType().noDefaultValue(); - - public static String registerDataSet(List> dataSet) { - String dataSetId = String.valueOf(DATA_SET_ID.incrementAndGet()); - DATA_SETS.put(dataSetId, dataSet); - return dataSetId; - } - - public static void clearDataSets() { - DATA_SETS.clear(); - } - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); - - Configuration configuration = Configuration.fromMap(context.getCatalogTable().getOptions()); - String dataId = configuration.getString(DATA_ID); - Preconditions.checkArgument( - DATA_SETS.containsKey(dataId), "data-id %s does not found in registered data set.", dataId); - - return new BoundedTableSource(DATA_SETS.get(dataId), tableSchema); - } - - @Override - public String factoryIdentifier() { - return "BoundedSource"; - } - - @Override - public Set> requiredOptions() { - return ImmutableSet.of(); - } - - @Override - public Set> optionalOptions() { - return ImmutableSet.of(DATA_ID); - } - - private static class BoundedTableSource implements ScanTableSource { - - private final List> elementsPerCheckpoint; - private final TableSchema tableSchema; - - private BoundedTableSource(List> elementsPerCheckpoint, TableSchema tableSchema) { - this.elementsPerCheckpoint = elementsPerCheckpoint; - this.tableSchema = tableSchema; - } - - private BoundedTableSource(BoundedTableSource toCopy) { - this.elementsPerCheckpoint = toCopy.elementsPerCheckpoint; - this.tableSchema = toCopy.tableSchema; - } - - @Override - public ChangelogMode getChangelogMode() { - Supplier> supplier = () -> elementsPerCheckpoint.stream().flatMap(List::stream); - - // Add the INSERT row kind by default. - ChangelogMode.Builder builder = ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT); - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.DELETE)) { - builder.addContainedKind(RowKind.DELETE); - } - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_BEFORE)) { - builder.addContainedKind(RowKind.UPDATE_BEFORE); - } - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_AFTER)) { - builder.addContainedKind(RowKind.UPDATE_AFTER); - } - - return builder.build(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return new DataStreamScanProvider() { - @Override - public DataStream produceDataStream( - ProviderContext providerContext, StreamExecutionEnvironment env) { - boolean checkpointEnabled = env.getCheckpointConfig().isCheckpointingEnabled(); - SourceFunction source = - new BoundedTestSource<>(elementsPerCheckpoint, checkpointEnabled); - - RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - // Converter to convert the Row to RowData. - DataFormatConverters.RowConverter rowConverter = - new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); - - return env.addSource(source, new RowTypeInfo(tableSchema.getFieldTypes())) - .map(rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)); - } - - @Override - public boolean isBounded() { - return true; - } - }; - } - - @Override - public DynamicTableSource copy() { - return new BoundedTableSource(this); - } - - @Override - public String asSummaryString() { - return "Bounded test table source"; - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java deleted file mode 100644 index 7b435d059845..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java +++ /dev/null @@ -1,108 +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.source; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * A stream source that: 1) emits the elements from elementsPerCheckpoint.get(0) without allowing - * checkpoints. 2) then waits for the checkpoint to complete. 3) emits the elements from - * elementsPerCheckpoint.get(1) without allowing checkpoints. 4) then waits for the checkpoint to - * complete. 5) ... - * - *

    Util all the list from elementsPerCheckpoint are exhausted. - */ -public final class BoundedTestSource implements SourceFunction, CheckpointListener { - - private final List> elementsPerCheckpoint; - private final boolean checkpointEnabled; - private volatile boolean running = true; - - private final AtomicInteger numCheckpointsComplete = new AtomicInteger(0); - - /** Emits all those elements in several checkpoints. */ - public BoundedTestSource(List> elementsPerCheckpoint, boolean checkpointEnabled) { - this.elementsPerCheckpoint = elementsPerCheckpoint; - this.checkpointEnabled = checkpointEnabled; - } - - public BoundedTestSource(List> elementsPerCheckpoint) { - this(elementsPerCheckpoint, true); - } - - /** Emits all those elements in a single checkpoint. */ - public BoundedTestSource(T... elements) { - this(Collections.singletonList(Arrays.asList(elements))); - } - - @Override - public void run(SourceContext ctx) throws Exception { - if (!checkpointEnabled) { - Preconditions.checkArgument( - elementsPerCheckpoint.size() <= 1, - "There should be at most one list in the elementsPerCheckpoint when checkpoint is disabled."); - elementsPerCheckpoint.stream().flatMap(List::stream).forEach(ctx::collect); - return; - } - - for (List elements : elementsPerCheckpoint) { - - final int checkpointToAwait; - synchronized (ctx.getCheckpointLock()) { - // Let's say checkpointToAwait = numCheckpointsComplete.get() + delta, in fact the value of - // delta should not - // affect the final table records because we only need to make sure that there will be - // exactly - // elementsPerCheckpoint.size() checkpoints to emit each records buffer from the original - // elementsPerCheckpoint. - // Even if the checkpoints that emitted results are not continuous, the correctness of the - // data should not be - // affected in the end. Setting the delta to be 2 is introducing the variable that produce - // un-continuous - // checkpoints that emit the records buffer from elementsPerCheckpoints. - checkpointToAwait = numCheckpointsComplete.get() + 2; - for (T element : elements) { - ctx.collect(element); - } - } - - synchronized (ctx.getCheckpointLock()) { - while (running && numCheckpointsComplete.get() < checkpointToAwait) { - ctx.getCheckpointLock().wait(1); - } - } - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - numCheckpointsComplete.incrementAndGet(); - } - - @Override - public void cancel() { - running = false; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java deleted file mode 100644 index 7aa2b8034bc5..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ /dev/null @@ -1,87 +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.source; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.FlinkTestBase; -import org.apache.iceberg.flink.MiniClusterResource; -import org.junit.After; -import org.junit.Rule; -import org.junit.rules.TestName; - -public class ChangeLogTableTestBase extends FlinkTestBase { - private volatile TableEnvironment tEnv = null; - - @Rule public TestName name = new TestName(); - - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s", name.getMethodName()); - BoundedTableFactory.clearDataSets(); - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = - EnvironmentSettings.newInstance().inStreamingMode().build(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(400) - .setMaxParallelism(1) - .setParallelism(1); - - tEnv = StreamTableEnvironment.create(env, settings); - } - } - } - return tEnv; - } - - protected static Row insertRow(Object... values) { - return Row.ofKind(RowKind.INSERT, values); - } - - protected static Row deleteRow(Object... values) { - return Row.ofKind(RowKind.DELETE, values); - } - - protected static Row updateBeforeRow(Object... values) { - return Row.ofKind(RowKind.UPDATE_BEFORE, values); - } - - protected static Row updateAfterRow(Object... values) { - return Row.ofKind(RowKind.UPDATE_AFTER, values); - } - - protected static List listJoin(List> lists) { - return lists.stream().flatMap(List::stream).collect(Collectors.toList()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java deleted file mode 100644 index 3a8071523b7c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ /dev/null @@ -1,132 +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.source; - -import java.io.File; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; - -public class SplitHelpers { - - private SplitHelpers() {} - - /** - * This create a list of IcebergSourceSplit from real files - *

  • Create a new Hadoop table under the {@code temporaryFolder} - *
  • write {@code fileCount} number of files to the new Iceberg table - *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} - * limit - *
  • Delete the Hadoop table - * - *

    Since the table and data files are deleted before this method return, caller shouldn't - * attempt to read the data files. - * - *

    By default, v1 Iceberg table is created. For v2 table use {@link - * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} - * - * @param temporaryFolder Folder to place the data to - * @param fileCount The number of files to create and add to the table - * @param filesPerSplit The number of files used for a split - */ - public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { - return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); - } - - /** - * This create a list of IcebergSourceSplit from real files - *

  • Create a new Hadoop table under the {@code temporaryFolder} - *
  • write {@code fileCount} number of files to the new Iceberg table - *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} - * limit - *
  • Delete the Hadoop table - * - *

    Since the table and data files are deleted before this method return, caller shouldn't - * attempt to read the data files. - * - * @param temporaryFolder Folder to place the data to - * @param fileCount The number of files to create and add to the table - * @param filesPerSplit The number of files used for a split - * @param version The table version to create - */ - public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) - throws Exception { - final File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - final String warehouse = "file:" + warehouseFile; - Configuration hadoopConf = new Configuration(); - final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); - ImmutableMap properties = - ImmutableMap.of(TableProperties.FORMAT_VERSION, version); - try { - final Table table = - catalog.createTable( - TestFixtures.TABLE_IDENTIFIER, - TestFixtures.SCHEMA, - PartitionSpec.unpartitioned(), - null, - properties); - final GenericAppenderHelper dataAppender = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - for (int i = 0; i < fileCount; ++i) { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); - dataAppender.appendToTable(records); - } - - final ScanContext scanContext = ScanContext.builder().build(); - final List splits = - FlinkSplitPlanner.planIcebergSourceSplits( - table, scanContext, ThreadPools.getWorkerPool()); - return splits.stream() - .flatMap( - split -> { - List> filesList = - Lists.partition(Lists.newArrayList(split.task().files()), filesPerSplit); - return filesList.stream() - .map(files -> new BaseCombinedScanTask(files)) - .map( - combinedScanTask -> - IcebergSourceSplit.fromCombinedScanTask(combinedScanTask)); - }) - .collect(Collectors.toList()); - } finally { - catalog.dropTable(TestFixtures.TABLE_IDENTIFIER); - catalog.close(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java deleted file mode 100644 index e4e48ca67f66..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java +++ /dev/null @@ -1,60 +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.source; - -import java.util.List; -import java.util.Map; -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.iceberg.relocated.com.google.common.collect.Lists; - -public class SqlHelpers { - private SqlHelpers() {} - - public static List sql(TableEnvironment tableEnv, String query, Object... args) { - TableResult tableResult = tableEnv.executeSql(String.format(query, args)); - try (CloseableIterator iter = tableResult.collect()) { - List results = Lists.newArrayList(iter); - return results; - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - public static String sqlOptionsToString(Map sqlOptions) { - StringBuilder builder = new StringBuilder(); - sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(",")); - String optionStr = builder.toString(); - if (optionStr.endsWith(",")) { - optionStr = optionStr.substring(0, optionStr.length() - 1); - } - - if (!optionStr.isEmpty()) { - optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr); - } - - return optionStr; - } - - private static String optionToKv(String key, Object value) { - return "'" + key + "'='" + value + "'"; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java deleted file mode 100644 index 7b5f9328694c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java +++ /dev/null @@ -1,87 +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.source; - -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import org.apache.flink.types.Row; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.Assert; -import org.junit.Test; - -public class TestBoundedTableFactory extends ChangeLogTableTestBase { - - @Test - public void testEmptyDataSet() { - String table = name.getMethodName(); - List> emptyDataSet = ImmutableList.of(); - - String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); - sql( - "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); - - Assert.assertEquals( - "Should have caught empty change log set.", - ImmutableList.of(), - sql("SELECT * FROM %s", table)); - } - - @Test - public void testBoundedTableFactory() { - String table = name.getMethodName(); - List> dataSet = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(1, "bbb"), - insertRow(2, "aaa"), - deleteRow(2, "aaa"), - insertRow(2, "bbb")), - ImmutableList.of( - updateBeforeRow(2, "bbb"), - updateAfterRow(2, "ccc"), - deleteRow(2, "ccc"), - insertRow(2, "ddd")), - ImmutableList.of( - deleteRow(1, "bbb"), - insertRow(1, "ccc"), - deleteRow(1, "ccc"), - insertRow(1, "ddd"))); - - String dataId = BoundedTableFactory.registerDataSet(dataSet); - sql( - "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); - - List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); - Assert.assertEquals( - "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); - - Assert.assertEquals( - "Should have the expected change log events", - rowSet.stream() - .filter(r -> Objects.equals(r.getField(1), "aaa")) - .collect(Collectors.toList()), - sql("SELECT * FROM %s WHERE data='aaa'", table)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java deleted file mode 100644 index ed3f54bec642..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ /dev/null @@ -1,191 +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.source; - -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.TestTemplate; - -/** Test {@link FlinkInputFormat}. */ -public class TestFlinkInputFormat extends TestFlinkSource { - - @Override - protected List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - return runFormat(formatBuilder.tableLoader(tableLoader()).buildFormat()); - } - - @TestTemplate - public void testNestedProjection() throws Exception { - Schema schema = - new Schema( - required(1, "data", Types.StringType.get()), - required( - 2, - "nested", - Types.StructType.of( - Types.NestedField.required(3, "f1", Types.StringType.get()), - Types.NestedField.required(4, "f2", Types.StringType.get()), - Types.NestedField.required(5, "f3", Types.LongType.get()))), - required(6, "id", Types.LongType.get())); - - Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), schema); - - List writeRecords = RandomGenericData.generate(schema, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); - - // Schema: [data, nested[f1, f2, f3], id] - // Projection: [nested.f2, data] - // The Flink SQL output: [f2, data] - // The FlinkInputFormat output: [nested[f2], data] - - TableSchema projectedSchema = - TableSchema.builder() - .field("nested", DataTypes.ROW(DataTypes.FIELD("f2", DataTypes.STRING()))) - .field("data", DataTypes.STRING()) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : writeRecords) { - Row nested = Row.of(((Record) record.get(1)).get(1)); - expected.add(Row.of(nested, record.get(0))); - } - - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testBasicProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); - - Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); - - List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); - - TableSchema projectedSchema = - TableSchema.builder() - .field("id", DataTypes.BIGINT()) - .field("data", DataTypes.STRING()) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : writeRecords) { - expected.add(Row.of(record.get(0), record.get(1))); - } - - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testReadPartitionColumn() throws Exception { - assumeThat(fileFormat).as("Temporary skip ORC").isNotEqualTo(FileFormat.ORC); - - Schema nestedSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional( - 2, - "struct", - Types.StructType.of( - Types.NestedField.optional(3, "innerId", Types.LongType.get()), - Types.NestedField.optional(4, "innerName", Types.StringType.get())))); - PartitionSpec spec = - PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); - - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); - List records = RandomGenericData.generate(nestedSchema, 10, 0L); - GenericAppenderHelper appender = - new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - for (Record record : records) { - org.apache.iceberg.TestHelpers.Row partition = - org.apache.iceberg.TestHelpers.Row.of(record.get(1, Record.class).get(1)); - appender.appendToTable(partition, Collections.singletonList(record)); - } - - TableSchema projectedSchema = - TableSchema.builder() - .field("struct", DataTypes.ROW(DataTypes.FIELD("innerName", DataTypes.STRING()))) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : records) { - Row nested = Row.of(((Record) record.get(1)).get(1)); - expected.add(Row.of(nested)); - } - - TestHelpers.assertRows(result, expected); - } - - private List runFormat(FlinkInputFormat inputFormat) throws IOException { - RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); - return TestHelpers.readRows(inputFormat, rowType); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java deleted file mode 100644 index 226da5813ad8..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.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.flink.source; - -import java.io.IOException; -import java.util.Map; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { - @Override - protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) - throws IOException { - Schema projected = testTable.schema().select(columns); - RowType rowType = FlinkSchemaUtil.convert(projected); - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.WAREHOUSE_LOCATION, - hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); - properties.put( - CatalogProperties.CLIENT_POOL_SIZE, - Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); - CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); - FlinkInputFormat inputFormat = - FlinkSource.forRowData() - .tableLoader( - TableLoader.fromCatalog( - hiveCatalogLoader, TableIdentifier.of("default", tableName))) - .project(FlinkSchemaUtil.toSchema(rowType)) - .buildFormat(); - - StructLikeSet set = StructLikeSet.create(projected.asStruct()); - TestHelpers.readRowData(inputFormat, rowType) - .forEach( - rowData -> { - RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); - set.add(wrapper.wrap(rowData)); - }); - - return set; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java deleted file mode 100644 index 1d52acb2fe7b..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.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.flink.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.TestMergingMetrics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkMergingMetrics extends TestMergingMetrics { - - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); - - public TestFlinkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileAppender writeAndGetAppender(List records) throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - - FileAppender appender = - new FlinkAppenderFactory( - tableResource.table(), - SCHEMA, - flinkSchema, - ImmutableMap.of(), - PartitionSpec.unpartitioned(), - null, - null, - null) - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); - try (FileAppender fileAppender = appender) { - records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); - } - return appender; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java deleted file mode 100644 index f58cc87c6a29..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ /dev/null @@ -1,813 +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.source; - -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.time.Instant; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.avro.generic.GenericData; -import org.apache.commons.collections.ListUtils; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Files; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.MetricsUtil; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.catalog.Namespace; -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.expressions.Expressions; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -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.SnapshotUtil; -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; - -public class TestFlinkMetaDataTable extends CatalogTestBase { - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - private @TempDir Path temp; - - @Parameter(index = 2) - private Boolean isPartition; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}") - protected static List parameters() { - List parameters = Lists.newArrayList(); - - for (Boolean isPartition : new Boolean[] {true, false}) { - String catalogName = "testhadoop"; - Namespace baseNamespace = Namespace.of("default"); - parameters.add(new Object[] {catalogName, baseNamespace, isPartition}); - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); - configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() { - super.before(); - sql("USE CATALOG %s", catalogName); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE %s", DATABASE); - if (isPartition) { - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')", - TABLE_NAME, format.name()); - sql("INSERT INTO %s VALUES (1,'a',10),(2,'a',20)", TABLE_NAME); - sql("INSERT INTO %s VALUES (1,'b',10),(2,'b',20)", TABLE_NAME); - } else { - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - sql("INSERT INTO %s VALUES (4,'iceberg',10)", TABLE_NAME); - } - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testSnapshots() { - String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - Iterator snapshots = table.snapshots().iterator(); - for (Row row : result) { - Snapshot next = snapshots.next(); - assertThat(((Instant) row.getField(0)).toEpochMilli()) - .as("Should have expected timestamp") - .isEqualTo(next.timestampMillis()); - assertThat(next.snapshotId()) - .as("Should have expected snapshot id") - .isEqualTo(next.snapshotId()); - assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); - assertThat(row.getField(3)).as("Should have expected operation").isEqualTo(next.operation()); - assertThat(row.getField(4)) - .as("Should have expected manifest list location") - .isEqualTo(next.manifestListLocation()); - assertThat(row.getField(5)).as("Should have expected summary").isEqualTo(next.summary()); - } - } - - @TestTemplate - public void testHistory() { - String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - Iterator snapshots = table.snapshots().iterator(); - for (Row row : result) { - Snapshot next = snapshots.next(); - assertThat(((Instant) row.getField(0)).toEpochMilli()) - .as("Should have expected made_current_at") - .isEqualTo(next.timestampMillis()); - assertThat(row.getField(1)) - .as("Should have expected snapshot id") - .isEqualTo(next.snapshotId()); - assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); - assertThat(row.getField(3)) - .as("Should have expected is current ancestor") - .isEqualTo( - SnapshotUtil.isAncestorOf( - table, table.currentSnapshot().snapshotId(), next.snapshotId())); - } - } - - @TestTemplate - public void testManifests() { - String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - List expectedDataManifests = dataManifests(table); - - for (int i = 0; i < result.size(); i++) { - Row row = result.get(i); - ManifestFile manifestFile = expectedDataManifests.get(i); - assertThat(row.getField(0)) - .as("Should have expected content") - .isEqualTo(manifestFile.content().id()); - assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); - assertThat(row.getField(2)) - .as("Should have expected length") - .isEqualTo(manifestFile.length()); - assertThat(row.getField(3)) - .as("Should have expected partition_spec_id") - .isEqualTo(manifestFile.partitionSpecId()); - assertThat(row.getField(4)) - .as("Should have expected added_snapshot_id") - .isEqualTo(manifestFile.snapshotId()); - assertThat(row.getField(5)) - .as("Should have expected added_data_files_count") - .isEqualTo(manifestFile.addedFilesCount()); - assertThat(row.getField(6)) - .as("Should have expected existing_data_files_count") - .isEqualTo(manifestFile.existingFilesCount()); - assertThat(row.getField(7)) - .as("Should have expected deleted_data_files_count") - .isEqualTo(manifestFile.deletedFilesCount()); - } - } - - @TestTemplate - public void testAllManifests() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - String sql = String.format("SELECT * FROM %s$all_manifests ", TABLE_NAME); - List result = sql(sql); - - List expectedDataManifests = allDataManifests(table); - - assertThat(expectedDataManifests).hasSize(result.size()); - for (int i = 0; i < result.size(); i++) { - Row row = result.get(i); - ManifestFile manifestFile = expectedDataManifests.get(i); - assertThat(row.getField(0)) - .as("Should have expected content") - .isEqualTo(manifestFile.content().id()); - assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); - assertThat(row.getField(2)) - .as("Should have expected length") - .isEqualTo(manifestFile.length()); - assertThat(row.getField(3)) - .as("Should have expected partition_spec_id") - .isEqualTo(manifestFile.partitionSpecId()); - assertThat(row.getField(4)) - .as("Should have expected added_snapshot_id") - .isEqualTo(manifestFile.snapshotId()); - assertThat(row.getField(5)) - .as("Should have expected added_data_files_count") - .isEqualTo(manifestFile.addedFilesCount()); - assertThat(row.getField(6)) - .as("Should have expected existing_data_files_count") - .isEqualTo(manifestFile.existingFilesCount()); - assertThat(row.getField(7)) - .as("Should have expected deleted_data_files_count") - .isEqualTo(manifestFile.deletedFilesCount()); - } - } - - @TestTemplate - public void testUnPartitionedTable() throws IOException { - assumeThat(isPartition).isFalse(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - List dataDeletes = Lists.newArrayList(dataDelete.copy("id", 1)); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, Files.localOutput(testFile), dataDeletes, deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - List expectedDataManifests = dataManifests(table); - List expectedDeleteManifests = deleteManifests(table); - - assertThat(expectedDataManifests).hasSize(2); - assertThat(expectedDeleteManifests).hasSize(1); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - - // check delete files table - Schema deleteFilesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("delete_files")) - .schema(); - - List deleteColumns = - deleteFilesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String deleteNames = - deleteColumns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - deleteFilesTableSchema = deleteFilesTableSchema.select(deleteColumns); - - List actualDeleteFiles = sql("SELECT %s FROM %s$delete_files", deleteNames, TABLE_NAME); - assertThat(actualDeleteFiles).hasSize(1); - assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); - - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); - assertThat(expectedDeleteFiles).as("Should be 1 delete file manifest entry").hasSize(1); - TestHelpers.assertEquals( - deleteFilesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check data files table - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - List actualDataFiles = sql("SELECT %s FROM %s$data_files", names, TABLE_NAME); - assertThat(actualDataFiles).as("Metadata table should return 2 data file").hasSize(2); - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - assertThat(expectedDataFiles).as("Should be 2 data file manifest entry").hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - // check all files table - List actualFiles = sql("SELECT %s FROM %s$files ORDER BY content", names, TABLE_NAME); - assertThat(actualFiles).as("Metadata table should return 3 files").hasSize(3); - List expectedFiles = - Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) - .collect(Collectors.toList()); - assertThat(expectedFiles).as("Should have 3 files manifest entriess").hasSize(3); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); - } - - @TestTemplate - public void testPartitionedTable() throws Exception { - assumeThat(isPartition).isTrue(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id", "data"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - deleteRow.put("data", "a"); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile), - org.apache.iceberg.TestHelpers.Row.of("a"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - deleteRow.put("data", "b"); - File testFile2 = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes2 = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile2), - org.apache.iceberg.TestHelpers.Row.of("b"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes2).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - - List expectedDataManifests = dataManifests(table); - List expectedDeleteManifests = deleteManifests(table); - - assertThat(expectedDataManifests).hasSize(2); - assertThat(expectedDeleteManifests).hasSize(2); - Table deleteFilesTable = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("delete_files")); - Schema filesTableSchema = deleteFilesTable.schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check delete files table - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - assertThat(expectedDeleteFiles).hasSize(1); - List actualDeleteFiles = - sql("SELECT %s FROM %s$delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - - assertThat(actualDeleteFiles).hasSize(1); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check data files table - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - assertThat(expectedDataFiles).hasSize(1); - List actualDataFiles = - sql("SELECT %s FROM %s$data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - assertThat(actualDataFiles).hasSize(1); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - List actualPartitionsWithProjection = - sql("SELECT file_count FROM %s$partitions ", TABLE_NAME); - assertThat(actualPartitionsWithProjection).hasSize(2); - for (int i = 0; i < 2; ++i) { - assertThat(actualPartitionsWithProjection.get(i).getField(0)).isEqualTo(1); - } - - // Check files table - List expectedFiles = - Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) - .collect(Collectors.toList()); - assertThat(expectedFiles).hasSize(2); - List actualFiles = - sql( - "SELECT %s FROM %s$files WHERE `partition`.`data`='a' ORDER BY content", - names, TABLE_NAME); - assertThat(actualFiles).hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); - } - - @TestTemplate - public void testAllFilesUnpartitioned() throws Exception { - assumeThat(isPartition).isFalse(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id", "data"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - List expectedDataManifests = dataManifests(table); - assertThat(expectedDataManifests).hasSize(2); - List expectedDeleteManifests = deleteManifests(table); - assertThat(expectedDeleteManifests).hasSize(1); - - // Clear table to test whether 'all_files' can read past files - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("all_data_files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check all data files table - List actualDataFiles = - sql("SELECT %s FROM %s$all_data_files order by record_count ", names, TABLE_NAME); - - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - assertThat(expectedDataFiles).hasSize(2); - assertThat(actualDataFiles).hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles, actualDataFiles); - - // Check all delete files table - List actualDeleteFiles = sql("SELECT %s FROM %s$all_delete_files", names, TABLE_NAME); - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); - assertThat(expectedDeleteFiles).hasSize(1); - assertThat(actualDeleteFiles).hasSize(1); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check all files table - List actualFiles = - sql("SELECT %s FROM %s$all_files ORDER BY content, record_count asc", names, TABLE_NAME); - List expectedFiles = - ListUtils.union(expectedDataFiles, expectedDeleteFiles); - expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - assertThat(actualFiles).hasSize(3); - TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); - } - - @TestTemplate - public void testAllFilesPartitioned() throws Exception { - assumeThat(!isPartition).isFalse(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - // Create delete file - Schema deleteRowSchema = table.schema().select("id"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile), - org.apache.iceberg.TestHelpers.Row.of("a"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - File testFile2 = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes2 = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile2), - org.apache.iceberg.TestHelpers.Row.of("b"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).addDeletes(eqDeletes2).commit(); - - List expectedDataManifests = dataManifests(table); - assertThat(expectedDataManifests).hasSize(2); - List expectedDeleteManifests = deleteManifests(table); - assertThat(expectedDeleteManifests).hasSize(1); - // Clear table to test whether 'all_files' can read past files - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("all_data_files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check all data files table - List actualDataFiles = - sql("SELECT %s FROM %s$all_data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - assertThat(expectedDataFiles).hasSize(1); - assertThat(actualDataFiles).hasSize(1); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - // Check all delete files table - List actualDeleteFiles = - sql("SELECT %s FROM %s$all_delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - assertThat(expectedDeleteFiles).hasSize(1); - assertThat(actualDeleteFiles).hasSize(1); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check all files table - List actualFiles = - sql( - "SELECT %s FROM %s$all_files WHERE `partition`.`data`='a' ORDER BY content", - names, TABLE_NAME); - List expectedFiles = - ListUtils.union(expectedDataFiles, expectedDeleteFiles); - expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - assertThat(actualFiles).hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); - } - - @TestTemplate - public void testMetadataLogEntries() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Long currentSnapshotId = table.currentSnapshot().snapshotId(); - TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); - Snapshot currentSnapshot = tableMetadata.currentSnapshot(); - Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); - List metadataLogEntries = - Lists.newArrayList(tableMetadata.previousFiles()); - - // Check metadataLog table - List metadataLogs = sql("SELECT * FROM %s$metadata_log_entries", TABLE_NAME); - - assertThat(metadataLogs).hasSize(3); - Row metadataLog = metadataLogs.get(0); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(0).timestampMillis())); - assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(0).file()); - assertThat(metadataLog.getField("latest_snapshot_id")).isNull(); - assertThat(metadataLog.getField("latest_schema_id")).isNull(); - assertThat(metadataLog.getField("latest_sequence_number")).isNull(); - - metadataLog = metadataLogs.get(1); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(1).timestampMillis())); - assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(1).file()); - assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); - assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(parentSnapshot.schemaId()); - assertThat(metadataLog.getField("latest_sequence_number")) - .isEqualTo(parentSnapshot.sequenceNumber()); - assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); - - metadataLog = metadataLogs.get(2); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); - assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); - assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); - assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); - assertThat(metadataLog.getField("latest_sequence_number")) - .isEqualTo(currentSnapshot.sequenceNumber()); - - // test filtering - List metadataLogWithFilters = - sql( - "SELECT * FROM %s$metadata_log_entries WHERE latest_snapshot_id = %s", - TABLE_NAME, currentSnapshotId); - assertThat(metadataLogWithFilters).hasSize(1); - metadataLog = metadataLogWithFilters.get(0); - assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) - .isEqualTo(metadataLog.getField("timestamp")); - - assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); - assertThat(metadataLog.getField("latest_snapshot_id")) - .isEqualTo(tableMetadata.currentSnapshot().snapshotId()); - assertThat(metadataLog.getField("latest_schema_id")) - .isEqualTo(tableMetadata.currentSnapshot().schemaId()); - assertThat(metadataLog.getField("latest_sequence_number")) - .isEqualTo(tableMetadata.currentSnapshot().sequenceNumber()); - - // test projection - List metadataFiles = - metadataLogEntries.stream() - .map(TableMetadata.MetadataLogEntry::file) - .collect(Collectors.toList()); - metadataFiles.add(tableMetadata.metadataFileLocation()); - List metadataLogWithProjection = - sql("SELECT file FROM %s$metadata_log_entries", TABLE_NAME); - assertThat(metadataLogWithProjection).hasSize(3); - for (int i = 0; i < metadataFiles.size(); i++) { - assertThat(metadataLogWithProjection.get(i).getField("file")).isEqualTo(metadataFiles.get(i)); - } - } - - @TestTemplate - public void testSnapshotReferencesMetatable() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Long currentSnapshotId = table.currentSnapshot().snapshotId(); - - // Create branch - table - .manageSnapshots() - .createBranch("testBranch", currentSnapshotId) - .setMaxRefAgeMs("testBranch", 10) - .setMinSnapshotsToKeep("testBranch", 20) - .setMaxSnapshotAgeMs("testBranch", 30) - .commit(); - // Create Tag - table - .manageSnapshots() - .createTag("testTag", currentSnapshotId) - .setMaxRefAgeMs("testTag", 50) - .commit(); - // Check refs table - List references = sql("SELECT * FROM %s$refs", TABLE_NAME); - List branches = sql("SELECT * FROM %s$refs WHERE type='BRANCH'", TABLE_NAME); - assertThat(references).hasSize(3); - assertThat(branches).hasSize(2); - List tags = sql("SELECT * FROM %s$refs WHERE type='TAG'", TABLE_NAME); - assertThat(tags).hasSize(1); - // Check branch entries in refs table - List mainBranch = - sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); - assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); - assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); - List testBranch = - sql("SELECT * FROM %s$refs WHERE name='testBranch' AND type='BRANCH'", TABLE_NAME); - assertThat((String) testBranch.get(0).getFieldAs("name")).isEqualTo("testBranch"); - assertThat((String) testBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - assertThat((Long) testBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); - assertThat((Long) testBranch.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(10)); - assertThat((Integer) testBranch.get(0).getFieldAs("min_snapshots_to_keep")) - .isEqualTo(Integer.valueOf(20)); - assertThat((Long) testBranch.get(0).getFieldAs("max_snapshot_age_in_ms")) - .isEqualTo(Long.valueOf(30)); - - // Check tag entries in refs table - List testTag = - sql("SELECT * FROM %s$refs WHERE name='testTag' AND type='TAG'", TABLE_NAME); - assertThat((String) testTag.get(0).getFieldAs("name")).isEqualTo("testTag"); - assertThat((String) testTag.get(0).getFieldAs("type")).isEqualTo("TAG"); - assertThat((Long) testTag.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); - assertThat((Long) testTag.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(50)); - // Check projection in refs table - List testTagProjection = - sql( - "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM %s$refs where type='TAG'", - TABLE_NAME); - assertThat((String) testTagProjection.get(0).getFieldAs("name")).isEqualTo("testTag"); - assertThat((String) testTagProjection.get(0).getFieldAs("type")).isEqualTo("TAG"); - assertThat((Long) testTagProjection.get(0).getFieldAs("snapshot_id")) - .isEqualTo(currentSnapshotId); - assertThat((Long) testTagProjection.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(50)); - assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); - List mainBranchProjection = - sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); - assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); - assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - List testBranchProjection = - sql( - "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM %s$refs WHERE name='testBranch' AND type = 'BRANCH'", - TABLE_NAME); - assertThat((String) testBranchProjection.get(0).getFieldAs("name")).isEqualTo("testBranch"); - assertThat((String) testBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - assertThat((Long) testBranchProjection.get(0).getFieldAs("snapshot_id")) - .isEqualTo(currentSnapshotId); - assertThat((Long) testBranchProjection.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(10)); - } - - /** - * Find matching manifest entries of an Iceberg table - * - * @param table iceberg table - * @param expectedContent file content to populate on entries - * @param entriesTableSchema schema of Manifest entries - * @param manifestsToExplore manifests to explore of the table - * @param partValue partition value that manifest entries must match, or null to skip filtering - */ - private List expectedEntries( - Table table, - FileContent expectedContent, - Schema entriesTableSchema, - List manifestsToExplore, - String partValue) - throws IOException { - List expected = Lists.newArrayList(); - for (ManifestFile manifest : manifestsToExplore) { - InputFile in = table.io().newInputFile(manifest.path()); - try (CloseableIterable rows = - Avro.read(in).project(entriesTableSchema).build()) { - for (GenericData.Record record : rows) { - if ((Integer) record.get("status") < 2 /* added or existing */) { - GenericData.Record file = (GenericData.Record) record.get("data_file"); - if (partitionMatch(file, partValue)) { - asMetadataRecord(file, expectedContent); - expected.add(file); - } - } - } - } - } - return expected; - } - - // Populate certain fields derived in the metadata tables - private void asMetadataRecord(GenericData.Record file, FileContent content) { - file.put(0, content.id()); - file.put(3, 0); // specId - } - - private boolean partitionMatch(GenericData.Record file, String partValue) { - if (partValue == null) { - return true; - } - GenericData.Record partition = (GenericData.Record) file.get(4); - return partValue.equals(partition.get(0).toString()); - } - - private List dataManifests(Table table) { - return table.currentSnapshot().dataManifests(table.io()); - } - - private List allDataManifests(Table table) { - List manifests = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - manifests.addAll(snapshot.dataManifests(table.io())); - } - return manifests; - } - - private List deleteManifests(Table table) { - return table.currentSnapshot().deleteManifests(table.io()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java deleted file mode 100644 index 2f0af1c3ba0c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java +++ /dev/null @@ -1,89 +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.source; - -import java.util.Map; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.ParameterizedTestExtension; -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.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.DeleteReadTests; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -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.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { - protected static String databaseName = "default"; - - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - private static TestHiveMetastore metastore = null; - - @BeforeAll - public static void startMetastore() { - metastore = new TestHiveMetastore(); - metastore.start(); - hiveConf = metastore.hiveConf(); - catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterAll - public static void stopMetastore() throws Exception { - metastore.stop(); - catalog = null; - } - - @Override - protected Table createTable(String name, Schema schema, PartitionSpec spec) { - Map props = Maps.newHashMap(); - props.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - 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)); - - return table; - } - - @Override - protected void dropTable(String name) { - catalog.dropTable(TableIdentifier.of(databaseName, name)); - } - - @Override - protected boolean expectPruned() { - return false; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java deleted file mode 100644 index 428da49f1de6..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ /dev/null @@ -1,540 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.nio.file.Path; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -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.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -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.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -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 TestFlinkScan { - @RegisterExtension - protected static MiniClusterExtension miniClusterResource = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @TempDir protected Path temporaryDirectory; - - @RegisterExtension - protected static final HadoopCatalogExtension catalogExtension = - new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - - @Parameter protected FileFormat fileFormat; - - @Parameters(name = "format={0}") - public static Collection fileFormat() { - return Arrays.asList(FileFormat.AVRO, FileFormat.PARQUET, FileFormat.ORC); - } - - protected TableLoader tableLoader() { - return catalogExtension.tableLoader(); - } - - protected abstract List runWithProjection(String... projected) throws Exception; - - protected abstract List runWithFilter( - Expression filter, String sqlFilter, boolean caseSensitive) throws Exception; - - protected List runWithFilter(Expression filter, String sqlFilter) throws Exception { - return runWithFilter(filter, sqlFilter, true); - } - - protected abstract List runWithOptions(Map options) throws Exception; - - protected abstract List run() throws Exception; - - @TestTemplate - public void testUnpartitionedTable() throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testPartitionedTable() throws Exception { - Table table = - catalogExtension - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - expectedRecords.get(0).set(2, "2020-03-20"); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testProjection() throws Exception { - Table table = - catalogExtension - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); - assertRows(runWithProjection("data"), Row.of(inputRecords.get(0).get(0))); - } - - @TestTemplate - public void testIdentityPartitionProjections() throws Exception { - Schema logSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "level", Types.StringType.get()), - Types.NestedField.optional(4, "message", Types.StringType.get())); - PartitionSpec spec = - PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); - - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); - List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); - - int idx = 0; - AppendFiles append = table.newAppend(); - for (Record record : inputRecords) { - record.set(1, "2020-03-2" + idx); - record.set(2, Integer.toString(idx)); - append.appendFile( - new GenericAppenderHelper(table, fileFormat, temporaryDirectory) - .writeFile( - org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), - ImmutableList.of(record))); - idx += 1; - } - append.commit(); - - // individual fields - validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); - // field pairs - validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); - // out-of-order pairs - validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); - // out-of-order triplets - validateIdentityPartitionProjections( - table, Arrays.asList("dt", "level", "message"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("level", "dt", "message"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("dt", "message", "level"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("level", "message", "dt"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("message", "dt", "level"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("message", "level", "dt"), inputRecords); - } - - private void validateIdentityPartitionProjections( - Table table, List projectedFields, List inputRecords) throws Exception { - List rows = runWithProjection(projectedFields.toArray(new String[0])); - - for (int pos = 0; pos < inputRecords.size(); pos++) { - Record inputRecord = inputRecords.get(pos); - Row actualRecord = rows.get(pos); - - for (int i = 0; i < projectedFields.size(); i++) { - String name = projectedFields.get(i); - assertThat(inputRecord.getField(name)) - .as("Projected field " + name + " should match") - .isEqualTo(actualRecord.getField(i)); - } - } - } - - @TestTemplate - public void testSnapshotReads() throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords); - long snapshotId = table.currentSnapshot().snapshotId(); - - long timestampMillis = table.currentSnapshot().timestampMillis(); - - // produce another timestamp - waitUntilAfter(timestampMillis); - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L)); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("snapshot-id", Long.toString(snapshotId))), - expectedRecords, - TestFixtures.SCHEMA); - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("as-of-timestamp", Long.toString(timestampMillis))), - expectedRecords, - TestFixtures.SCHEMA); - } - - @TestTemplate - public void testTagReads() throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List expectedRecords1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords1); - long snapshotId = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().createTag("t1", snapshotId).commit(); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords1, TestFixtures.SCHEMA); - - List expectedRecords2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords2); - snapshotId = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().replaceTag("t1", snapshotId).commit(); - - List expectedRecords = Lists.newArrayList(); - expectedRecords.addAll(expectedRecords1); - expectedRecords.addAll(expectedRecords2); - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testBranchReads() throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List expectedRecordsBase = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecordsBase); - long snapshotId = table.currentSnapshot().snapshotId(); - - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName, snapshotId).commit(); - - List expectedRecordsForBranch = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(branchName, expectedRecordsForBranch); - - List expectedRecordsForMain = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecordsForMain); - - List branchExpectedRecords = Lists.newArrayList(); - branchExpectedRecords.addAll(expectedRecordsBase); - branchExpectedRecords.addAll(expectedRecordsForBranch); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("branch", branchName)), - branchExpectedRecords, - TestFixtures.SCHEMA); - - List mainExpectedRecords = Lists.newArrayList(); - mainExpectedRecords.addAll(expectedRecordsBase); - mainExpectedRecords.addAll(expectedRecordsForMain); - - TestHelpers.assertRecords(run(), mainExpectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testIncrementalReadViaTag() throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(records1); - long snapshotId1 = table.currentSnapshot().snapshotId(); - String startTag = "t1"; - table.manageSnapshots().createTag(startTag, snapshotId1).commit(); - - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); - helper.appendToTable(records2); - - List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); - helper.appendToTable(records3); - long snapshotId3 = table.currentSnapshot().snapshotId(); - String endTag = "t2"; - table.manageSnapshots().createTag(endTag, snapshotId3).commit(); - - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); - - List expected = Lists.newArrayList(); - expected.addAll(records2); - expected.addAll(records3); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-snapshot-id", Long.toString(snapshotId1)) - .put("end-tag", endTag) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - assertThatThrownBy( - () -> - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .put("start-snapshot-id", Long.toString(snapshotId1)) - .buildOrThrow())) - .isInstanceOf(Exception.class) - .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); - - assertThatThrownBy( - () -> - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow())) - .isInstanceOf(Exception.class) - .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set."); - } - - @TestTemplate - public void testIncrementalRead() throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(records1); - long snapshotId1 = table.currentSnapshot().snapshotId(); - - // snapshot 2 - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); - helper.appendToTable(records2); - - List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); - helper.appendToTable(records3); - long snapshotId3 = table.currentSnapshot().snapshotId(); - - // snapshot 4 - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); - - List expected2 = Lists.newArrayList(); - expected2.addAll(records2); - expected2.addAll(records3); - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-snapshot-id", Long.toString(snapshotId1)) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow()), - expected2, - TestFixtures.SCHEMA); - } - - @TestTemplate - public void testFilterExpPartition() throws Exception { - Table table = - catalogExtension - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - expectedRecords.get(0).set(2, "2020-03-20"); - expectedRecords.get(1).set(2, "2020-03-20"); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - DataFile dataFile1 = - helper.writeFile(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - DataFile dataFile2 = - helper.writeFile( - org.apache.iceberg.TestHelpers.Row.of("2020-03-21", 0), - RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); - helper.appendToTable(dataFile1, dataFile2); - TestHelpers.assertRecords( - runWithFilter(Expressions.equal("dt", "2020-03-20"), "where dt='2020-03-20'", true), - expectedRecords, - TestFixtures.SCHEMA); - } - - private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); - expectedRecords.get(0).set(0, "a"); - expectedRecords.get(1).set(0, "b"); - expectedRecords.get(2).set(0, "c"); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - DataFile dataFile = helper.writeFile(expectedRecords); - helper.appendToTable(dataFile); - - List actual = - runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); - - TestHelpers.assertRecords(actual, expectedRecords.subList(1, 3), TestFixtures.SCHEMA); - } - - @TestTemplate - public void testFilterExp() throws Exception { - testFilterExp(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); - } - - @TestTemplate - public void testFilterExpCaseInsensitive() throws Exception { - // sqlFilter does not support case-insensitive filtering: - // https://issues.apache.org/jira/browse/FLINK-16175 - testFilterExp(Expressions.greaterThanOrEqual("DATA", "b"), "where data>='b'", false); - } - - @TestTemplate - public void testPartitionTypes() throws Exception { - Schema typesSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "decimal", Types.DecimalType.of(38, 18)), - Types.NestedField.optional(3, "str", Types.StringType.get()), - Types.NestedField.optional(4, "binary", Types.BinaryType.get()), - Types.NestedField.optional(5, "date", Types.DateType.get()), - Types.NestedField.optional(6, "time", Types.TimeType.get()), - Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone())); - PartitionSpec spec = - PartitionSpec.builderFor(typesSchema) - .identity("decimal") - .identity("str") - .identity("binary") - .identity("date") - .identity("time") - .identity("timestamp") - .build(); - - Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); - List records = RandomGenericData.generate(typesSchema, 10, 0L); - GenericAppenderHelper appender = - new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - for (Record record : records) { - org.apache.iceberg.TestHelpers.Row partition = - org.apache.iceberg.TestHelpers.Row.of( - record.get(1), - record.get(2), - record.get(3), - record.get(4) == null ? null : DateTimeUtil.daysFromDate((LocalDate) record.get(4)), - record.get(5) == null ? null : DateTimeUtil.microsFromTime((LocalTime) record.get(5)), - record.get(6) == null - ? null - : DateTimeUtil.microsFromTimestamp((LocalDateTime) record.get(6))); - appender.appendToTable(partition, Collections.singletonList(record)); - } - - TestHelpers.assertRecords(run(), records, typesSchema); - } - - @TestTemplate - public void testCustomizedFlinkDataTypes() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required( - 1, - "map", - Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), - Types.NestedField.required( - 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); - Table table = catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); - List records = RandomGenericData.generate(schema, 10, 0L); - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - helper.appendToTable(records); - TestHelpers.assertRecords(run(), records, schema); - } - - private static void assertRows(List results, Row... expected) { - TestHelpers.assertRows(results, Arrays.asList(expected)); - } - - private static void waitUntilAfter(long timestampMillis) { - long current = System.currentTimeMillis(); - while (current <= timestampMillis) { - current = System.currentTimeMillis(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java deleted file mode 100644 index b5bddc7767fb..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.junit.jupiter.api.BeforeEach; - -/** Test Flink SELECT SQLs. */ -public class TestFlinkScanSql extends TestFlinkSource { - private volatile TableEnvironment tEnv; - - @BeforeEach - public void before() throws IOException { - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Override - protected List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(sqlOptions); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java deleted file mode 100644 index ff7892071231..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ /dev/null @@ -1,90 +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.source; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -public abstract class TestFlinkSource extends TestFlinkScan { - - @Override - protected List runWithProjection(String... projected) throws Exception { - TableSchema.Builder builder = TableSchema.builder(); - TableSchema schema = - FlinkSchemaUtil.toSchema( - FlinkSchemaUtil.convert( - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); - for (String field : projected) { - TableColumn column = schema.getTableColumn(field).get(); - builder.field(column.getName(), column.getType()); - } - return run(FlinkSource.forRowData().project(builder.build()), Maps.newHashMap(), "", projected); - } - - @Override - protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - FlinkSource.Builder builder = - FlinkSource.forRowData().filters(Collections.singletonList(filter)); - Map options = Maps.newHashMap(); - options.put("case-sensitive", Boolean.toString(caseSensitive)); - return run(builder, options, sqlFilter, "*"); - } - - @Override - protected List runWithOptions(Map options) throws Exception { - FlinkSource.Builder builder = FlinkSource.forRowData(); - Optional.ofNullable(options.get("case-sensitive")) - .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); - Optional.ofNullable(options.get("snapshot-id")) - .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); - Optional.ofNullable(options.get("branch")).ifPresent(value -> builder.branch(value)); - Optional.ofNullable(options.get("start-tag")).ifPresent(value -> builder.startTag(value)); - Optional.ofNullable(options.get("end-tag")).ifPresent(value -> builder.endTag(value)); - Optional.ofNullable(options.get("start-snapshot-id")) - .ifPresent(value -> builder.startSnapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("end-snapshot-id")) - .ifPresent(value -> builder.endSnapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("as-of-timestamp")) - .ifPresent(value -> builder.asOfTimestamp(Long.parseLong(value))); - return run(builder, options, "", "*"); - } - - @Override - protected List run() throws Exception { - return run(FlinkSource.forRowData(), Maps.newHashMap(), "", "*"); - } - - protected abstract List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) - throws Exception; -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java deleted file mode 100644 index 1814ff8f8542..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ /dev/null @@ -1,58 +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.source; - -import java.util.List; -import org.apache.flink.types.Row; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFlinkSourceConfig extends TestFlinkTableSource { - private static final String TABLE = "test_table"; - - @Test - public void testFlinkSessionConfig() { - getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot set as-of-timestamp option for streaming reader"); - } - - @Test - public void testFlinkHintConfig() { - List result = - sql( - "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", - TABLE, System.currentTimeMillis()); - Assert.assertEquals(3, result.size()); - } - - @Test - public void testReadOptionHierarchy() { - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); - List result = sql("SELECT * FROM %s", TABLE); - Assert.assertEquals(1, result.size()); - - result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); - Assert.assertEquals(3, result.size()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java deleted file mode 100644 index affd90c347dd..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ /dev/null @@ -1,85 +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.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -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.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.Test; - -/** Use the FlinkSource */ -public class TestFlinkSourceSql extends TestSqlBase { - @Override - public void before() throws IOException { - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - @Test - public void testInferParallelismWithGlobalSetting() throws IOException { - Configuration cfg = getTableEnv().getConfig().getConfiguration(); - cfg.set(PipelineOptions.MAX_PARALLELISM, 1); - - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - List expectedRecords = Lists.newArrayList(); - long maxFileLen = 0; - for (int i = 0; i < 5; i++) { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); - DataFile dataFile = helper.writeFile(null, records); - helper.appendToTable(dataFile); - expectedRecords.addAll(records); - maxFileLen = Math.max(dataFile.fileSizeInBytes(), maxFileLen); - } - - // Make sure to generate multiple CombinedScanTasks - SqlHelpers.sql( - getTableEnv(), - "ALTER TABLE t SET ('read.split.open-file-cost'='1', 'read.split.target-size'='%s')", - maxFileLen); - - List results = run(Maps.newHashMap(), "", "*"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java deleted file mode 100644 index b56e804d1400..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ /dev/null @@ -1,607 +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.source; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkTestBase; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; - -public class TestFlinkTableSource extends FlinkTestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - private static String warehouse; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - public TestFlinkTableSource() { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - } - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - // before variables - warehouse = "file:" + warehouseFile; - } - - @Before - public void before() { - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @After - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - } - - @Test - public void testLimitPushDown() { - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) - .as("Invalid limit number: -1 ") - .isInstanceOf(SqlParserException.class) - .hasMessageContaining("SQL parse failed. Encountered \"-\""); - - Assert.assertEquals( - "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); - - String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); - List resultExceed = sql(sqlLimitExceed); - Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); - List expectedList = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedList, resultExceed); - - String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); - String explain = getTableEnv().explainSql(querySql); - String expectedExplain = "limit=[1]"; - Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); - List result = sql(querySql); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assertions.assertThat(result).containsAnyElementsOf(expectedList); - - String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); - List mixedResult = sql(sqlMixed); - Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); - Assert.assertEquals( - "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); - } - - @Test - public void testNoFilterPushDown() { - String sql = String.format("SELECT * FROM %s ", TABLE_NAME); - List result = sql(sql); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDownEqual() { - String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") == 1"; - - List result = sql(sqlLiteralRight); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownEqualNull() { - String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); - - List result = sql(sqlEqualNull); - Assert.assertEquals("Should have 0 record", 0, result.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownEqualLiteralOnLeft() { - String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") == 1"; - - List resultLeft = sql(sqlLiteralLeft); - Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNoEqual() { - String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") != 1"; - - List resultNE = sql(sqlNE); - Assert.assertEquals("Should have 2 records", 2, resultNE.size()); - - List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedNE, resultNE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNoEqualNull() { - String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); - - List resultNE = sql(sqlNotEqualNull); - Assert.assertEquals("Should have 0 records", 0, resultNE.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownAnd() { - String sqlAnd = - String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); - - List resultAnd = sql(sqlAnd); - Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownOr() { - String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; - - List resultOr = sql(sqlOr); - Assert.assertEquals("Should have 2 record", 2, resultOr.size()); - - List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedOR, resultOr); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThan() { - String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") > 1"; - - List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 record", 2, resultGT.size()); - - List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedGT, resultGT); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThanNull() { - String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); - - List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownGreaterThanLiteralOnLeft() { - String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") < 3"; - - List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 records", 2, resultGT.size()); - - List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedGT, resultGT); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThanEqual() { - String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") >= 2"; - - List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); - - List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedGTE, resultGTE); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownGreaterThanEqualNull() { - String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); - - List resultGT = sql(sqlGTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { - String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") <= 2"; - - List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); - - List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedGTE, resultGTE); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThan() { - String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") < 2"; - - List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThanNull() { - String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); - - List resultGT = sql(sqlLT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownLessThanLiteralOnLeft() { - String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") > 2"; - - List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThanEqual() { - String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") <= 1"; - - List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLessThanEqualNull() { - String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); - - List resultGT = sql(sqlLTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); - } - - @Test - public void testFilterPushDownLessThanEqualLiteralOnLeft() { - String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") >= 3"; - - List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownIn() { - String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; - List resultIN = sql(sqlIN); - Assert.assertEquals("Should have 2 records", 2, resultIN.size()); - - List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedIN, resultIN); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownInNull() { - String sqlInNull = - String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); - - List result = sql(sqlInNull); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDownNotIn() { - String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); - - List resultNotIn = sql(sqlNotIn); - Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNotInNull() { - String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); - List resultGT = sql(sqlNotInNull); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDownIsNotNull() { - String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); - String expectedFilter = "not_null(ref(name=\"data\"))"; - - List resultNotNull = sql(sqlNotNull); - Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); - - List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expected, resultNotNull); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownIsNull() { - String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); - String expectedFilter = "is_null(ref(name=\"data\"))"; - - List resultNull = sql(sqlNull); - Assert.assertEquals("Should have 1 record", 1, resultNull.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNot() { - String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); - - List resultNot = sql(sqlNot); - Assert.assertEquals("Should have 1 record", 1, resultNot.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownBetween() { - String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); - - List resultBetween = sql(sqlBetween); - Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); - - List expectedBetween = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedBetween, resultBetween); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownNotBetween() { - String sqlNotBetween = - String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; - - List resultNotBetween = sql(sqlNotBetween); - Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterPushDownLike() { - String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; - - String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; - List resultLike = sql(sqlLike); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals( - "The like result should produce the expected record", - Row.of(1, "iceberg", 10.0), - resultLike.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); - } - - @Test - public void testFilterNotPushDownLike() { - Row expectRecord = Row.of(1, "iceberg", 10.0); - String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; - List resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 0, resultLike.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 3 records", 3, resultLike.size()); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, resultLike); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; - resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testFilterPushDown2Literal() { - String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); - List result = sql(sql2Literal); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); - } - - @Test - public void testSqlParseNaN() { - // todo add some test case to test NaN - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java deleted file mode 100644 index 3c0c38e1115d..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ /dev/null @@ -1,127 +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.source; - -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.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -public class TestIcebergSourceBounded extends TestFlinkScan { - - @Override - protected List runWithProjection(String... projected) throws Exception { - Schema icebergTableSchema = - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); - TableSchema.Builder builder = TableSchema.builder(); - TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); - for (String field : projected) { - TableColumn column = schema.getTableColumn(field).get(); - builder.field(column.getName(), column.getType()); - } - TableSchema flinkSchema = builder.build(); - Schema projectedSchema = FlinkSchemaUtil.convert(icebergTableSchema, flinkSchema); - return run(projectedSchema, Lists.newArrayList(), Maps.newHashMap(), "", projected); - } - - @Override - protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - Map options = Maps.newHashMap(); - options.put("case-sensitive", Boolean.toString(caseSensitive)); - return run(null, Collections.singletonList(filter), options, sqlFilter, "*"); - } - - @Override - protected List runWithOptions(Map options) throws Exception { - return run(null, Lists.newArrayList(), options, "", "*"); - } - - @Override - protected List run() throws Exception { - return run(null, Lists.newArrayList(), Maps.newHashMap(), "", "*"); - } - - protected List run( - Schema projectedSchema, - List filters, - Map options, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - 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(); - } - - IcebergSource.Builder sourceBuilder = - IcebergSource.forRowData() - .tableLoader(tableLoader()) - .table(table) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.properties(options); - - DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) - .map( - new RowDataToRowMapper( - FlinkSchemaUtil.convert( - projectedSchema == null ? table.schema() : projectedSchema))); - - try (CloseableIterator iter = stream.executeAndCollect()) { - return Lists.newArrayList(iter); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java deleted file mode 100644 index 0337f3597053..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ /dev/null @@ -1,203 +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.source; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -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.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -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.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -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.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.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} - }; - } - - private final FileFormat fileFormat; - private final int parallelism; - - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } - - @Test - public void testUnpartitionedTable() throws Exception { - Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testPartitionedTable() throws Exception { - String dateStr = "2020-03-20"; - Table table = - catalogResource - .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, TEMPORARY_FOLDER) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @Test - public void testProjection() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) - .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 = catalogResource.tableLoader()) { - tableLoader.open(); - table = tableLoader.loadTable(); - } - - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); - - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(catalogResource.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.setAll(options); - - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); - RowType rowType = FlinkSchemaUtil.convert(readSchema); - 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); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java deleted file mode 100644 index ff3348bbc3a3..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ /dev/null @@ -1,76 +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.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.jupiter.api.BeforeEach; - -public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { - private volatile TableEnvironment tEnv; - - @BeforeEach - public void before() throws IOException { - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Override - protected List run( - Schema projectedSchema, - List filters, - Map options, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(options); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java deleted file mode 100644 index bfd7fa5758e3..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ /dev/null @@ -1,508 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceContinuous { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - private final AtomicLong randomSeed = new AtomicLong(0L); - - @Test - public void testTableScanThenIncremental() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testTableScanThenIncrementalAfterExpiration() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); - - // snapshot2 - List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - - Assert.assertEquals(1, tableResource.table().history().size()); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 4); - List initialRecords = Lists.newArrayList(); - initialRecords.addAll(batch1); - initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testEarliestSnapshot() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 4); - List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); - combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testLatestSnapshot() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - // we want to make sure job is running first so that enumerator can - // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); - - // inclusive behavior for starting snapshot - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testSpecificSnapshotId() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot1) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testSpecificSnapshotTimestamp() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - // snapshot0 - List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); - - // sleep for 2 ms to make sure snapshot1 has a higher timestamp value - Thread.sleep(2); - - // snapshot1 - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot1Timestamp) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - // consume data from snapshot1 - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - } - } - - @Test - public void testReadingFromBranch() throws Exception { - String branch = "b1"; - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - - List batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batchBase); - - // create branch - tableResource - .table() - .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) - .commit(); - - // snapshot1 to branch - List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch1); - - // snapshot2 to branch - List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch2); - - List branchExpectedRecords = Lists.newArrayList(); - branchExpectedRecords.addAll(batchBase); - branchExpectedRecords.addAll(batch1); - branchExpectedRecords.addAll(batch2); - // reads from branch: it should contain the first snapshot (before the branch creation) followed - // by the next 2 snapshots added - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .useBranch(branch) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); - - // snapshot3 to branch - List batch3 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); - - // snapshot4 to branch - List batch4 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch4); - - List result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); - } - - // read only from main branch. Should contain only the first snapshot - scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); - - List batchMain2 = - RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batchMain2); - resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); - } - } - - private DataStream createStream(ScanContext scanContext) throws Exception { - // start the source and collect output - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - DataStream stream = - env.fromSource( - IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .streaming(scanContext.isStreaming()) - .streamingStartingStrategy(scanContext.streamingStartingStrategy()) - .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) - .startSnapshotId(scanContext.startSnapshotId()) - .monitorInterval(Duration.ofMillis(10L)) - .branch(scanContext.branch()) - .build(), - WatermarkStrategy.noWatermarks(), - "icebergSource", - TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); - return stream; - } - - public static List waitForResult(CloseableIterator iter, int limit) { - List results = Lists.newArrayListWithCapacity(limit); - while (results.size() < limit) { - if (iter.hasNext()) { - results.add(iter.next()); - } else { - break; - } - } - return results; - } - - public static void waitUntilJobIsRunning(ClusterClient client) { - Awaitility.await("job should be running") - .atMost(Duration.ofSeconds(30)) - .pollInterval(Duration.ofMillis(10)) - .untilAsserted(() -> assertThat(getRunningJobs(client)).isNotEmpty()); - } - - public static List getRunningJobs(ClusterClient client) throws Exception { - Collection statusMessages = client.listJobs().get(); - return statusMessages.stream() - .filter(status -> status.getJobState() == JobStatus.RUNNING) - .map(JobStatusMessage::getJobId) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java deleted file mode 100644 index 7d991ee603c9..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ /dev/null @@ -1,297 +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.source; - -import java.time.Duration; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.runtime.highavailability.nonha.embedded.HaLeadershipControl; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.iceberg.FileFormat; -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.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.sink.FlinkSink; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceFailover { - - private static final int PARALLELISM = 4; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - protected IcebergSource.Builder sourceBuilder() { - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - } - - protected Schema schema() { - return TestFixtures.SCHEMA; - } - - protected List generateRecords(int numRecords, long seed) { - return RandomGenericData.generate(schema(), numRecords, seed); - } - - protected void assertRecords(Table table, List expectedRecords, Duration timeout) - throws Exception { - SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); - } - - @Test - public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); - } - - @Test - public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); - } - - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { - List expectedRecords = Lists.newArrayList(); - GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - for (int i = 0; i < 4; ++i) { - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - } - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - - DataStream stream = - env.fromSource( - sourceBuilder().build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); - - JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); - JobID jobId = jobClient.getJobID(); - - RecordCounterToFail.waitToFail(); - triggerFailover( - failoverType, - jobId, - RecordCounterToFail::continueProcessing, - miniClusterResource.getMiniCluster()); - - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); - } - - @Test - public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); - } - - @Test - public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); - } - - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); - List expectedRecords = Lists.newArrayList(); - - List batch = generateRecords(2, 0); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(10L); - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - - DataStream stream = - env.fromSource( - sourceBuilder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); - - JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); - JobID jobId = jobClient.getJobID(); - - for (int i = 1; i < 5; i++) { - Thread.sleep(10); - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); - } - } - - // wait longer for continuous source to reduce flakiness - // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); - } - - // ------------------------------------------------------------------------ - // test utilities copied from Flink's FileSourceTextLinesITCase - // ------------------------------------------------------------------------ - - private enum FailoverType { - NONE, - TM, - JM - } - - private static void triggerFailover( - FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) - throws Exception { - switch (type) { - case NONE: - afterFailAction.run(); - break; - case TM: - restartTaskManager(afterFailAction, miniCluster); - break; - case JM: - triggerJobManagerFailover(jobId, afterFailAction, miniCluster); - break; - } - } - - private static void triggerJobManagerFailover( - JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { - HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); - haLeadershipControl.revokeJobMasterLeadership(jobId).get(); - afterFailAction.run(); - haLeadershipControl.grantJobMasterLeadership(jobId).get(); - } - - private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) - throws Exception { - miniCluster.terminateTaskManager(0).get(); - afterFailAction.run(); - miniCluster.startTaskManager(); - } - - private static class RecordCounterToFail { - - private static AtomicInteger records; - private static CompletableFuture fail; - private static CompletableFuture continueProcessing; - - private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { - - records = new AtomicInteger(); - fail = new CompletableFuture<>(); - continueProcessing = new CompletableFuture<>(); - return stream.map( - record -> { - boolean reachedFailPoint = records.incrementAndGet() > failAfter; - boolean notFailedYet = !fail.isDone(); - if (notFailedYet && reachedFailPoint) { - fail.complete(null); - continueProcessing.get(); - } - return record; - }); - } - - private static void waitToFail() throws ExecutionException, InterruptedException { - fail.get(); - } - - private static void continueProcessing() { - continueProcessing.complete(null); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java deleted file mode 100644 index f7dc931c506c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ /dev/null @@ -1,112 +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.source; - -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.util.StructLikeWrapper; -import org.awaitility.Awaitility; - -public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { - // Increment ts by 15 minutes for each generateRecords batch - private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); - // Within a batch, increment ts by 1 second - private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); - - private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); - - @Override - protected IcebergSource.Builder sourceBuilder() { - return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) - .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA); - } - - @Override - protected Schema schema() { - return TestFixtures.TS_SCHEMA; - } - - @Override - protected List generateRecords(int numRecords, long seed) { - // Override the ts field to create a more realistic situation for event time alignment - tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); - return RandomGenericData.generate(schema(), numRecords, seed).stream() - .peek( - record -> { - LocalDateTime ts = - LocalDateTime.ofInstant( - Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), - ZoneId.of("Z")); - record.setField("ts", ts); - }) - .collect(Collectors.toList()); - } - - /** - * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves - * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates - * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the - * {@link LocalDateTime} to a Long type so that Comparators can continue to work. - */ - @Override - protected void assertRecords(Table table, List expectedRecords, Duration timeout) - throws Exception { - List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); - Awaitility.await("expected list of records should be produced") - .atMost(timeout) - .untilAsserted( - () -> { - SimpleDataUtil.equalsRecords( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - }); - } - - private List convertLocalDateTimeToMilli(List records) { - return records.stream() - .peek( - r -> { - LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); - r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); - }) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java deleted file mode 100644 index df148c212ebd..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java +++ /dev/null @@ -1,102 +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.source; - -import java.io.IOException; -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.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -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.StructLikeSet; -import org.junit.jupiter.api.extension.RegisterExtension; - -public class TestIcebergSourceReaderDeletes extends TestFlinkReaderDeletesBase { - - private static final int PARALLELISM = 4; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @Override - protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) - throws IOException { - Schema projected = testTable.schema().select(columns); - RowType rowType = FlinkSchemaUtil.convert(projected); - - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.WAREHOUSE_LOCATION, - hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); - properties.put( - CatalogProperties.CLIENT_POOL_SIZE, - Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); - CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); - TableLoader hiveTableLoader = - TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName)); - hiveTableLoader.open(); - try (TableLoader tableLoader = hiveTableLoader) { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - DataStream stream = - env.fromSource( - IcebergSource.builder() - .tableLoader(tableLoader) - .assignerFactory(new SimpleSplitAssignerFactory()) - .project(projected) - .build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)); - - try (CloseableIterator iter = stream.executeAndCollect()) { - List rowDataList = Lists.newArrayList(iter); - StructLikeSet set = StructLikeSet.create(projected.asStruct()); - rowDataList.forEach( - rowData -> { - RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); - set.add(wrapper.wrap(rowData)); - }); - return set; - } catch (Exception e) { - throw new IOException("Failed to collect result", e); - } - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java deleted file mode 100644 index 4250460d278d..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.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.flink.source; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.time.Instant; -import java.time.ZoneId; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -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.Test; - -/** Use the IcebergSource (FLIP-27) */ -public class TestIcebergSourceSql extends TestSqlBase { - private static final Schema SCHEMA_TS = - new Schema( - required(1, "t1", Types.TimestampType.withoutZone()), - required(2, "t2", Types.LongType.get())); - - @Override - public void before() throws IOException { - TableEnvironment tableEnvironment = getTableEnv(); - Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); - - tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); - SqlHelpers.sql( - tableEnvironment, - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); - SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); - - tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private Record generateRecord(Instant t1, long t2) { - Record record = GenericRecord.create(SCHEMA_TS); - record.setField("t1", t1.atZone(ZoneId.systemDefault()).toLocalDateTime()); - record.setField("t2", t2); - return record; - } - - /** Generates the records in the expected order, with respect to their datafile */ - private List generateExpectedRecords(boolean ascending) throws Exception { - Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); - long baseTime = 1702382109000L; - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - - Record file1Record1 = - generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); - Record file1Record2 = - generateRecord( - Instant.ofEpochMilli(baseTime - 10 * 1000L), baseTime + (1000 * 60 * 60 * 24 * 35L)); - - List recordsDataFile1 = Lists.newArrayList(); - recordsDataFile1.add(file1Record1); - recordsDataFile1.add(file1Record2); - DataFile dataFile1 = helper.writeFile(recordsDataFile1); - - Record file2Record1 = - generateRecord( - Instant.ofEpochMilli(baseTime + 14 * 1000L), baseTime - (1000 * 60 * 60 * 24 * 30L)); - Record file2Record2 = - generateRecord( - Instant.ofEpochMilli(baseTime + 12 * 1000L), baseTime - (1000 * 60 * 61 * 24 * 35L)); - - List recordsDataFile2 = Lists.newArrayList(); - recordsDataFile2.add(file2Record1); - recordsDataFile2.add(file2Record2); - - DataFile dataFile2 = helper.writeFile(recordsDataFile2); - helper.appendToTable(dataFile1, dataFile2); - - // Expected records if the splits are ordered - // - ascending (watermark from t1) - records from the split with early timestamps, then - // records from the split with late timestamps - // - descending (watermark from t2) - records from the split with old longs, then records - // from the split with new longs - List expected = Lists.newArrayList(); - if (ascending) { - expected.addAll(recordsDataFile1); - expected.addAll(recordsDataFile2); - } else { - expected.addAll(recordsDataFile2); - expected.addAll(recordsDataFile1); - } - return expected; - } - - /** Tests the order of splits returned when setting the watermark-column options */ - @Test - public void testWatermarkOptionsAscending() throws Exception { - List expected = generateExpectedRecords(true); - TestHelpers.assertRecordsWithOrder( - run( - ImmutableMap.of("watermark-column", "t1", "split-file-open-cost", "128000000"), - "", - "*"), - expected, - SCHEMA_TS); - } - - /** - * Tests the order of splits returned when setting the watermark-column and - * watermark-column-time-unit" options - */ - @Test - public void testWatermarkOptionsDescending() throws Exception { - List expected = generateExpectedRecords(false); - TestHelpers.assertRecordsWithOrder( - run( - ImmutableMap.of( - "watermark-column", - "t2", - "watermark-column-time-unit", - "MILLISECONDS", - "split-file-open-cost", - "128000000"), - "", - "*"), - expected, - SCHEMA_TS); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java deleted file mode 100644 index bc1808ee77da..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ /dev/null @@ -1,408 +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.source; - -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; - -import java.io.Serializable; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.runtime.metrics.MetricNames; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; -import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; -import org.apache.flink.streaming.api.windowing.time.Time; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.Collector; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -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.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceWithWatermarkExtractor implements Serializable { - private static final int PARALLELISM = 4; - private static final String SOURCE_NAME = "IcebergSource"; - private static final int RECORD_NUM_FOR_2_SPLITS = 200; - private static final ConcurrentMap windows = Maps.newConcurrentMap(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); - - /** - * This is an integration test for watermark handling and windowing. Integration testing the - * following features: - * - *

      - *
    • - Ordering of the splits - *
    • - Emitting of watermarks - *
    • - Firing windows based on watermarks - *
    - * - *

    The test generates 4 splits - * - *

      - *
    • - Split 1 - Watermark 100 min - *
    • - Split 2, 3 - Watermark 0 min - *
    • - Split 4 - Watermark 6 min - *
    - * - *

    Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency - * issues). - * - *

    Checks that windows are handled correctly based on the emitted watermarks, and splits are - * read in the following order: - * - *

      - *
    • - Split 2, 3 - *
    • - Split 4 - *
    • - Split 1 - *
    - * - *

    As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. - * - *

    Add 2 more splits, so the task manager close the windows for the original 4 splits and emit - * the appropriate aggregated records. - */ - @Test - public void testWindowing() throws Exception { - GenericAppenderHelper dataAppender = appender(); - List expectedRecords = Lists.newArrayList(); - - // Generate records with the following pattern: - // - File 1 - Later records (Watermark 6000000) - // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") - // - File 2 - First records (Watermark 0) - // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - File 3 - Parallel write for the first records (Watermark 360000) - // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") - List batch = - ImmutableList.of( - generateRecord(100, "file_1-recordTs_100"), - generateRecord(101, "file_1-recordTs_101"), - generateRecord(103, "file_1-recordTs_103")); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - batch = Lists.newArrayListWithCapacity(100); - for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - // Generate records where the timestamps are out of order, but still between 0-5 minutes - batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); - } - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - batch = - ImmutableList.of( - generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataStream stream = - env.fromSource( - source(), - WatermarkStrategy.noWatermarks() - .withTimestampAssigner(new RowDataTimestampAssigner()), - SOURCE_NAME, - TypeInformation.of(RowData.class)); - - stream - .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) - .apply( - new AllWindowFunction() { - @Override - public void apply( - TimeWindow window, Iterable values, Collector out) { - // Emit RowData which contains the window start time, and the record count in - // that window - AtomicInteger count = new AtomicInteger(0); - values.forEach(a -> count.incrementAndGet()); - out.collect(row(window.getStart(), count.get())); - windows.put(window.getStart(), count.get()); - } - }); - - // Use static variable to collect the windows, since other solutions were flaky - windows.clear(); - env.executeAsync("Iceberg Source Windowing Test"); - - // Wait for the 2 first windows from File 2 and File 3 - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - windows.equals( - ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); - - // Write data so the windows containing test data are closed - dataAppender.appendToTable( - dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); - - // Wait for last test record window from File 1 - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - windows.equals( - ImmutableMap.of( - 0L, - RECORD_NUM_FOR_2_SPLITS, - TimeUnit.MINUTES.toMillis(5), - 2, - TimeUnit.MINUTES.toMillis(100), - 3))); - } - - /** - * This is an integration test for watermark handling and throttling. Integration testing the - * following: - * - *

      - *
    • - Emitting of watermarks - *
    • - Watermark alignment - *
    - * - *

    The test generates 3 splits - * - *

      - *
    • - Split 1 - Watermark 100 min - *
    • - Split 2, 3 - Watermark 0 min - *
    - * - * The splits are read in the following order: - * - *
      - *
    • - Split 2, 3 (Task Manager 1, Task Manager 2) - *
    • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) - *
    - * - * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. - * - *

    The status of the watermark alignment is checked by the alignment related metrics. - * - *

    Adding new records with old timestamps to the table will enable the running reader to - * continue reading the files, but the watermark alignment will still prevent the paused reader to - * continue. - * - *

    After adding some records with new timestamps the blocked reader is un-paused, and both ot - * the readers continue reading. - */ - @Test - public void testThrottling() throws Exception { - GenericAppenderHelper dataAppender = appender(); - - // Generate records in advance - - // File 1 - Later records (Watermark 6.000.000 - 100 min) - // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") - List batch1 = - ImmutableList.of( - generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); - - // File 2 - First records (Watermark 0 - 0 min) - // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - List batch2 = Lists.newArrayListWithCapacity(100); - for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch2.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); - } - - // File 3 - Some records will be blocked (Watermark 900.000 - 15 min) - List batch3 = - ImmutableList.of( - generateRecord(15, "file_3-recordTs_15"), - generateRecord(16, "file_3-recordTs_16"), - generateRecord(17, "file_3-recordTs_17")); - - // File 4 - Some records will be blocked (Watermark 900.000 - 15 min) - List batch4 = - ImmutableList.of( - generateRecord(15, "file_4-recordTs_15"), - generateRecord(16, "file_4-recordTs_16"), - generateRecord(17, "file_4-recordTs_17")); - - // File 5 - Records which will remove the block (Watermark 5.400.000 - 90 min) - List batch5 = - ImmutableList.of( - generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - DataStream stream = - env.fromSource( - source(), - WatermarkStrategy.noWatermarks() - .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), - SOURCE_NAME, - TypeInformation.of(RowData.class)); - - try (CloseableIterator resultIterator = stream.collectAsync()) { - JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); - - // Insert the first data into the table - dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); - - // Get the drift metric, wait for it to be created and reach the expected state - // (100 min - 20 min - 0 min) - // Also this validates that the WatermarkAlignment is working - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) - .isPresent()); - Gauge drift = - findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); - - // Add some old records with 2 splits, so even if the blocked gets one split, the other reader - // one gets one as well - dataAppender.appendToTable(dataAppender.writeFile(batch3), dataAppender.writeFile(batch4)); - - // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 - // min - 15 min) - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); - - // Add some new records which should unblock the throttled reader - dataAppender.appendToTable(batch5); - - // Wait for the new drift to decrease below the allowed drift to signal the normal state - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); - } - } - - protected IcebergSource source() { - return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) - .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA) - .splitSize(100L) - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - } - - protected Record generateRecord(int minutes, String str) { - // Override the ts field to create a more realistic situation for event time alignment - Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); - LocalDateTime ts = - LocalDateTime.ofInstant( - Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), - ZoneId.of("Z")); - record.setField("ts", ts); - record.setField("str", str); - return record; - } - - private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { - String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() - .map(m -> (Gauge) m) - .filter(m -> m.getValue() == withValue) - .findFirst(); - } - - private GenericAppenderHelper appender() { - // We need to create multiple splits, so we need to generate parquet files with multiple offsets - org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); - hadoopConf.set("write.parquet.page-size-bytes", "64"); - hadoopConf.set("write.parquet.row-group-size-bytes", "64"); - return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); - } - - private static RowData row(long time, long count) { - GenericRowData result = new GenericRowData(2); - result.setField(0, time); - result.setField(1, String.valueOf(count)); - return result; - } - - private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { - @Override - public long extractTimestamp(RowData element, long recordTimestamp) { - return element.getTimestamp(0, 0).getMillisecond(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java deleted file mode 100644 index 40dfda723749..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ /dev/null @@ -1,299 +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.source; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.file.Path; -import java.util.Base64; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Files; -import org.apache.iceberg.Parameters; -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.TableIdentifier; -import org.apache.iceberg.data.FileHelpers; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.TestHelpers; -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.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.io.TempDir; - -public class TestMetadataTableReadableMetrics extends CatalogTestBase { - private static final String TABLE_NAME = "test_table"; - - @Parameters(name = "catalogName={0}, baseNamespace={1}") - protected static List parameters() { - List parameters = Lists.newArrayList(); - String catalogName = "testhive"; - Namespace baseNamespace = Namespace.empty(); - parameters.add(new Object[] {catalogName, baseNamespace}); - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); - configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - private @TempDir Path temp; - - private static final Types.StructType LEAF_STRUCT_TYPE = - Types.StructType.of( - optional(1, "leafLongCol", Types.LongType.get()), - optional(2, "leafDoubleCol", Types.DoubleType.get())); - - private static final Types.StructType NESTED_STRUCT_TYPE = - Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); - - private static final Schema NESTED_SCHEMA = - new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); - - private static final Schema PRIMITIVE_SCHEMA = - new Schema( - required(1, "booleanCol", Types.BooleanType.get()), - required(2, "intCol", Types.IntegerType.get()), - required(3, "longCol", Types.LongType.get()), - required(4, "floatCol", Types.FloatType.get()), - required(5, "doubleCol", Types.DoubleType.get()), - optional(6, "decimalCol", Types.DecimalType.of(10, 2)), - optional(7, "stringCol", Types.StringType.get()), - optional(8, "fixedCol", Types.FixedType.ofLength(3)), - optional(9, "binaryCol", Types.BinaryType.get())); - - private Table createPrimitiveTable() throws IOException { - Table table = - catalog.createTable( - TableIdentifier.of(DATABASE, TABLE_NAME), - PRIMITIVE_SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of()); - List records = - Lists.newArrayList( - createPrimitiveRecord( - false, - 1, - 1L, - 0, - 1.0D, - new BigDecimal("1.00"), - "1", - Base64.getDecoder().decode("1111"), - ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), - createPrimitiveRecord( - true, - 2, - 2L, - 0, - 2.0D, - new BigDecimal("2.00"), - "2", - Base64.getDecoder().decode("2222"), - ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), - createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), - createPrimitiveRecord( - false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); - - File testFile = File.createTempFile("junit", null, temp.toFile()); - DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); - table.newAppend().appendFile(dataFile).commit(); - return table; - } - - private void createNestedTable() throws IOException { - Table table = - validationCatalog.createTable( - TableIdentifier.of(DATABASE, TABLE_NAME), - NESTED_SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of()); - - List records = - Lists.newArrayList( - createNestedRecord(0L, 0.0), - createNestedRecord(1L, Double.NaN), - createNestedRecord(null, null)); - - File testFile = File.createTempFile("junit", null, temp.toFile()); - DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); - table.newAppend().appendFile(dataFile).commit(); - } - - @BeforeEach - public void before() { - super.before(); - sql("USE CATALOG %s", catalogName); - sql("CREATE DATABASE %s", DATABASE); - sql("USE %s", DATABASE); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - protected GenericRecord createPrimitiveRecord( - boolean booleanCol, - int intCol, - long longCol, - float floatCol, - double doubleCol, - BigDecimal decimalCol, - String stringCol, - byte[] fixedCol, - ByteBuffer binaryCol) { - GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); - record.set(0, booleanCol); - record.set(1, intCol); - record.set(2, longCol); - record.set(3, floatCol); - record.set(4, doubleCol); - record.set(5, decimalCol); - record.set(6, stringCol); - record.set(7, fixedCol); - record.set(8, binaryCol); - return record; - } - - private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { - GenericRecord record = GenericRecord.create(NESTED_SCHEMA); - GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); - GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); - leaf.set(0, longCol); - leaf.set(1, doubleCol); - nested.set(0, leaf); - record.set(0, nested); - return record; - } - - protected Object[] row(Object... values) { - return values; - } - - @TestTemplate - public void testPrimitiveColumns() throws Exception { - createPrimitiveTable(); - List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); - - Row binaryCol = - Row.of( - 52L, - 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 fixedCol = - Row.of( - 44L, - 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"); - - List expected = - Lists.newArrayList( - Row.of( - Row.of( - binaryCol, - booleanCol, - decimalCol, - doubleCol, - fixedCol, - floatCol, - intCol, - longCol, - stringCol))); - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testSelectPrimitiveValues() throws Exception { - createPrimitiveTable(); - - TestHelpers.assertRows( - sql( - "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s$files", - TABLE_NAME), - ImmutableList.of(Row.of(1, true))); - - TestHelpers.assertRows( - sql("SELECT content, readable_metrics.longCol.value_count FROM %s$files", TABLE_NAME), - ImmutableList.of(Row.of(0, 4L))); - - TestHelpers.assertRows( - sql("SELECT readable_metrics.longCol.value_count, content FROM %s$files", TABLE_NAME), - ImmutableList.of(Row.of(4L, 0))); - } - - @TestTemplate - public void testSelectNestedValues() throws Exception { - createNestedTable(); - TestHelpers.assertRows( - sql( - "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " - + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s$files", - TABLE_NAME), - ImmutableList.of(Row.of(0L, 3L))); - } - - @TestTemplate - 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 metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); - - TestHelpers.assertRows( - sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java deleted file mode 100644 index 25ecec23d216..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ /dev/null @@ -1,186 +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.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataProjection; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; -import org.apache.iceberg.flink.sink.TaskWriterFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -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.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestProjectMetaColumn { - - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; - - @Parameterized.Parameters(name = "fileFormat={0}") - public static Iterable parameters() { - return Lists.newArrayList( - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.AVRO}); - } - - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { - // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); - Table table = - SimpleDataUtil.createTable( - location, - ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), - false); - - List rows = - Lists.newArrayList( - SimpleDataUtil.createInsert(1, "AAA"), - SimpleDataUtil.createInsert(2, "BBB"), - SimpleDataUtil.createInsert(3, "CCC")); - writeAndCommit(table, ImmutableList.of(), false, rows); - - FlinkInputFormat input = - FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); - - List results = Lists.newArrayList(); - TestHelpers.readRowData( - input, - rowData -> { - // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); - results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); - }); - - // Assert the results. - TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); - } - - @Test - public void testV1SkipToRemoveMetaColumn() throws IOException { - testSkipToRemoveMetaColumn(1); - } - - @Test - public void testV2SkipToRemoveMetaColumn() throws IOException { - testSkipToRemoveMetaColumn(2); - } - - @Test - public void testV2RemoveMetaColumn() throws Exception { - // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); - Table table = - SimpleDataUtil.createTable( - location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); - - List rows = - Lists.newArrayList( - SimpleDataUtil.createInsert(1, "AAA"), - SimpleDataUtil.createDelete(1, "AAA"), - SimpleDataUtil.createInsert(2, "AAA"), - SimpleDataUtil.createInsert(2, "BBB")); - int eqFieldId = table.schema().findField("data").fieldId(); - writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); - - FlinkInputFormat input = - FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); - - List results = Lists.newArrayList(); - TestHelpers.readRowData( - input, - rowData -> { - // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); - results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); - }); - - // Assert the results. - TestHelpers.assertRows( - ImmutableList.of( - SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")), - results, - SimpleDataUtil.ROW_TYPE); - } - - private void writeAndCommit( - Table table, List eqFieldIds, boolean upsert, List rows) - throws IOException { - TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); - try (TaskWriter io = writer) { - for (RowData row : rows) { - io.write(row); - } - } - - RowDelta delta = table.newRowDelta(); - WriteResult result = writer.complete(); - - for (DataFile dataFile : result.dataFiles()) { - delta.addRows(dataFile); - } - - for (DeleteFile deleteFile : result.deleteFiles()) { - delta.addDeletes(deleteFile); - } - - delta.commit(); - } - - private TaskWriter createTaskWriter( - Table table, List equalityFieldIds, boolean upsert) { - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - SimpleDataUtil.ROW_TYPE, - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, - format, - table.properties(), - equalityFieldIds, - upsert); - - taskWriterFactory.initialize(1, 1); - return taskWriterFactory.create(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java deleted file mode 100644 index 485035787d6d..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java +++ /dev/null @@ -1,35 +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.source; - -import org.apache.avro.generic.GenericRecord; -import org.apache.iceberg.flink.AvroGenericRecordConverterBase; -import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; - -public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { - @Override - protected void testConverter(DataGenerator dataGenerator) { - RowDataToAvroGenericRecordConverter converter = - RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); - GenericRecord expected = dataGenerator.generateAvroGenericRecord(); - GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); - Assert.assertEquals(expected, actual); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java deleted file mode 100644 index 317301260f66..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ /dev/null @@ -1,60 +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.source; - -import java.io.IOException; -import org.apache.flink.configuration.Configuration; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; - -public class TestSourceUtil { - @Test - public void testInferedParallelism() throws IOException { - Configuration configuration = new Configuration(); - // Empty table, infer parallelism should be at least 1 - int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits - // num : 2 - parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); - - // 2 splits and limit is 1 , max infer parallelism is default 100, - // which is greater than splits num and limit, the parallelism is the limit value : 1 - parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 - configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); - parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : - // 1 - parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - - // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 - configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java deleted file mode 100644 index dda46033143e..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.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.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.flink.TableLoader; -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.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -/** Test other more advanced usage of SQL. They don't need to run for every file format. */ -public abstract class TestSqlBase { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private volatile TableEnvironment tEnv; - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Before - public abstract void before() throws IOException; - - @Test - public void testResiduals() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - List writeRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - writeRecords.get(0).set(1, 123L); - writeRecords.get(0).set(2, "2020-03-20"); - writeRecords.get(1).set(1, 456L); - writeRecords.get(1).set(2, "2020-03-20"); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - - List expectedRecords = Lists.newArrayList(); - expectedRecords.add(writeRecords.get(0)); - - DataFile dataFile1 = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), writeRecords); - DataFile dataFile2 = - helper.writeFile( - TestHelpers.Row.of("2020-03-21", 0), - RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); - helper.appendToTable(dataFile1, dataFile2); - - org.apache.iceberg.flink.TestHelpers.assertRecords( - run(Maps.newHashMap(), "where dt='2020-03-20' and id=123", "*"), - expectedRecords, - TestFixtures.SCHEMA); - } - - @Test - public void testExposeLocality() throws Exception { - Table table = - catalogResource - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - TableLoader tableLoader = TableLoader.fromHadoopTable(table.location()); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 10, 0L); - expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); - DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - helper.appendToTable(dataFile); - - // test sql api - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); - - List results = SqlHelpers.sql(getTableEnv(), "select * from t"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - - // test table api - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), true); - FlinkSource.Builder builder = FlinkSource.forRowData().tableLoader(tableLoader).table(table); - - // When running with CI or local, `localityEnabled` will be false even if this configuration is - // enabled - Assert.assertFalse( - "Expose split locality info should be false.", - SourceUtil.isLocalityEnabled(table, tableConf, true)); - - results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - } - - protected List run( - Map options, String sqlFilter, String... sqlSelectedFields) { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(options); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java deleted file mode 100644 index 9e043bbbbbd2..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ /dev/null @@ -1,430 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -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.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestStreamScanSql extends CatalogTestBase { - private static final String TABLE = "test_table"; - private static final FileFormat FORMAT = FileFormat.PARQUET; - - private TableEnvironment tEnv; - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = - EnvironmentSettings.newInstance().inStreamingMode(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - - StreamTableEnvironment streamTableEnv = - StreamTableEnvironment.create(env, settingsBuilder.build()); - streamTableEnv - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - tEnv = streamTableEnv; - } - } - } - return tEnv; - } - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - private void insertRows(String partition, Table table, Row... rows) throws IOException { - insertRows(partition, SnapshotRef.MAIN_BRANCH, table, rows); - } - - private void insertRows(String partition, String branch, Table table, Row... rows) - throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, FORMAT, temporaryDirectory); - - GenericRecord gRecord = GenericRecord.create(table.schema()); - List records = Lists.newArrayList(); - for (Row row : rows) { - records.add( - gRecord.copy( - "id", row.getField(0), - "data", row.getField(1), - "dt", row.getField(2))); - } - - if (partition != null) { - appender.appendToTable(TestHelpers.Row.of(partition, 0), branch, records); - } else { - appender.appendToTable(branch, records); - } - } - - private void insertRowsInBranch(String branch, Table table, Row... rows) throws IOException { - insertRows(null, branch, table, rows); - } - - private void insertRows(Table table, Row... rows) throws IOException { - insertRows(null, table, rows); - } - - private void assertRows(List expectedRows, Iterator iterator) { - for (Row expectedRow : expectedRows) { - assertThat(iterator).hasNext(); - Row actualRow = iterator.next(); - assertThat(actualRow.getArity()).isEqualTo(3); - assertThat(actualRow.getField(0)).isEqualTo(expectedRow.getField(0)); - assertThat(actualRow.getField(1)).isEqualTo(expectedRow.getField(1)); - assertThat(actualRow.getField(2)).isEqualTo(expectedRow.getField(2)); - } - } - - @TestTemplate - public void testUnPartitionedTable() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - - Row row1 = Row.of(1, "aaa", "2021-01-01"); - insertRows(table, row1); - assertRows(ImmutableList.of(row1), iterator); - - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row2); - assertRows(ImmutableList.of(row2), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testPartitionedTable() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR) PARTITIONED BY (dt)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - Row row1 = Row.of(1, "aaa", "2021-01-01"); - insertRows("2021-01-01", table, row1); - assertRows(ImmutableList.of(row1), iterator); - - Row row2 = Row.of(2, "bbb", "2021-01-02"); - insertRows("2021-01-02", table, row2); - assertRows(ImmutableList.of(row2), iterator); - - Row row3 = Row.of(1, "aaa", "2021-01-02"); - insertRows("2021-01-02", table, row3); - assertRows(ImmutableList.of(row3), iterator); - - Row row4 = Row.of(2, "bbb", "2021-01-01"); - insertRows("2021-01-01", table, row4); - assertRows(ImmutableList.of(row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testConsumeFromBeginning() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1, row2); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - assertRows(ImmutableList.of(row1, row2), iterator); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - insertRows(table, row3); - assertRows(ImmutableList.of(row3), iterator); - - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row4); - assertRows(ImmutableList.of(row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - /** - * Insert records on the main branch. Then, insert in a named branch. Reads from the main branch - * and assert that the only records from main are returned - */ - public void testConsumeFilesFromMainBranch() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots on main branch - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - - insertRows(table, row1, row2); - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName).commit(); - - // insert on the 'b1' branch - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - - insertRowsInBranch(branchName, table, row3, row4); - - // read from main - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row1, row2), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - /** - * Insert records on the main branch. Creates a named branch. Insert record on named branch. Then - * select from the named branch and assert all the records are returned. - */ - public void testConsumeFilesFromBranch() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots on main branch - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - - insertRows(table, row1, row2); - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName).commit(); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", - TABLE, branchName); - - try (CloseableIterator iterator = result.collect()) { - assertRows(ImmutableList.of(row1, row2), iterator); - // insert on the 'b1' branch - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRowsInBranch(branchName, table, row3, row4); - assertRows(ImmutableList.of(row3, row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - /** - * Insert records on branch b1. Then insert record on b2. Then select from each branch and assert - * the correct records are returned - */ - public void testConsumeFilesFromTwoBranches() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - String branch1 = "b1"; - String branch2 = "b2"; - table.manageSnapshots().createBranch(branch1).commit(); - table.manageSnapshots().createBranch(branch2).commit(); - - // Produce two snapshots on main branch - Row row1Branch1 = Row.of(1, "b1", "2021-01-01"); - Row row2Branch1 = Row.of(2, "b1", "2021-01-01"); - - Row row1Branch2 = Row.of(2, "b2", "2021-01-01"); - Row row2Branch2 = Row.of(3, "b3", "2021-01-01"); - - insertRowsInBranch(branch1, table, row1Branch1, row2Branch1); - insertRowsInBranch(branch2, table, row1Branch2, row2Branch2); - - TableResult resultBranch1 = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", - TABLE, branch1); - - try (CloseableIterator iterator = resultBranch1.collect()) { - assertRows(ImmutableList.of(row1Branch1, row2Branch1), iterator); - Row another = Row.of(4, "ccc", "2021-01-01"); - insertRowsInBranch(branch1, table, another); - assertRows(ImmutableList.of(another), iterator); - } - - TableResult resultBranch2 = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", - TABLE, branch2); - try (CloseableIterator iterator = resultBranch2.collect()) { - assertRows(ImmutableList.of(row1Branch2, row2Branch2), iterator); - Row another = Row.of(4, "ccc", "2021-01-01"); - insertRowsInBranch(branch2, table, another); - assertRows(ImmutableList.of(another), iterator); - } - - resultBranch1.getJobClient().ifPresent(JobClient::cancel); - resultBranch2.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testConsumeFromStartSnapshotId() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots. - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1); - insertRows(table, row2); - - long startSnapshotId = table.currentSnapshot().snapshotId(); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row3, row4); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " - + "'start-snapshot-id'='%d')*/", - TABLE, startSnapshotId); - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row3, row4), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testConsumeFromStartTag() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots. - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1); - insertRows(table, row2); - - String tagName = "t1"; - long startSnapshotId = table.currentSnapshot().snapshotId(); - table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row3, row4); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " - + "'start-tag'='%s')*/", - TABLE, tagName); - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row3, row4), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - - Assertions.assertThatThrownBy( - () -> - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " - + "'start-snapshot-id'='%d' )*/", - TABLE, tagName, startSnapshotId)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java deleted file mode 100644 index f7b13598bc2a..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ /dev/null @@ -1,401 +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.source; - -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.time.Duration; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -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.types.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestStreamingMonitorFunction extends TestBase { - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - private static final long WAIT_TIME_MILLIS = 10 * 1000L; - - @Parameters(name = "formatVersion = {0}") - protected static List parameters() { - return Arrays.asList(1, 2); - } - - @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()); - } - - private void runSourceFunctionInTask( - TestSourceContext sourceContext, StreamingMonitorFunction function) { - Thread task = - new Thread( - () -> { - try { - function.run(sourceContext); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - task.start(); - } - - @TestTemplate - public void testConsumeWithoutStartSnapshotId() throws Exception { - List> recordsList = generateRecordsAndCommitTxn(10); - ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @TestTemplate - public void testConsumeFromStartSnapshotId() throws Exception { - // Commit the first five transactions. - generateRecordsAndCommitTxn(5); - long startSnapshotId = table.currentSnapshot().snapshotId(); - - // Commit the next five transactions. - List> recordsList = generateRecordsAndCommitTxn(5); - - ScanContext scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .startSnapshotId(startSnapshotId) - .build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @TestTemplate - public void testConsumeFromStartTag() throws Exception { - // Commit the first five transactions. - generateRecordsAndCommitTxn(5); - long startSnapshotId = table.currentSnapshot().snapshotId(); - String tagName = "t1"; - table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); - - // Commit the next five transactions. - List> recordsList = generateRecordsAndCommitTxn(5); - - ScanContext scanContext = - ScanContext.builder().monitorInterval(Duration.ofMillis(100)).startTag(tagName).build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @TestTemplate - public void testCheckpointRestore() throws Exception { - List> recordsList = generateRecordsAndCommitTxn(10); - ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); - - StreamingMonitorFunction func = createFunction(scanContext); - OperatorSubtaskState state; - try (AbstractStreamOperatorTestHarness harness = createHarness(func)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, func); - - awaitExpectedSplits(sourceContext); - - state = harness.snapshot(1, 1); - - // Stop the stream task. - func.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - - List> newRecordsList = generateRecordsAndCommitTxn(10); - StreamingMonitorFunction newFunc = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(newFunc)) { - harness.setup(); - // Recover to process the remaining snapshots. - harness.initializeState(state); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, newFunc); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - newFunc.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(newRecordsList)), SCHEMA); - } - } - - private void awaitExpectedSplits(TestSourceContext sourceContext) { - Awaitility.await("expected splits should be produced") - .atMost(Duration.ofMillis(WAIT_TIME_MILLIS)) - .untilAsserted( - () -> { - assertThat(sourceContext.latch.getCount()).isEqualTo(0); - assertThat(sourceContext.splits).as("Should produce the expected splits").hasSize(1); - }); - } - - @TestTemplate - public void testInvalidMaxPlanningSnapshotCount() { - ScanContext scanContext1 = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .maxPlanningSnapshotCount(0) - .build(); - assertThatThrownBy(() -> createFunction(scanContext1)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("The max-planning-snapshot-count must be greater than zero"); - - ScanContext scanContext2 = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .maxPlanningSnapshotCount(-10) - .build(); - - assertThatThrownBy(() -> createFunction(scanContext2)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("The max-planning-snapshot-count must be greater than zero"); - } - - @TestTemplate - public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { - generateRecordsAndCommitTxn(10); - - // Use the oldest snapshot as starting to avoid the initial case. - long oldestSnapshotId = SnapshotUtil.oldestAncestor(table).snapshotId(); - - ScanContext scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .splitSize(1000L) - .startSnapshotId(oldestSnapshotId) - .maxPlanningSnapshotCount(Integer.MAX_VALUE) - .build(); - - FlinkInputSplit[] expectedSplits = - FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - - assertThat(expectedSplits).hasSize(9); - - // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the - // total splits number - for (int maxPlanningSnapshotCount : ImmutableList.of(1, 9, 15)) { - scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(500)) - .startSnapshotId(oldestSnapshotId) - .splitSize(1000L) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); - function.sourceContext(sourceContext); - function.monitorAndForwardSplits(); - - if (maxPlanningSnapshotCount < 10) { - assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); - } - } - } - } - - private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { - List> expectedRecords = Lists.newArrayList(); - for (int i = 0; i < commitTimes; i++) { - List records = RandomGenericData.generate(SCHEMA, 100, 0L); - expectedRecords.add(records); - - // Commit those records to iceberg table. - writeRecords(records); - } - return expectedRecords; - } - - private void writeRecords(List records) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); - appender.appendToTable(records); - } - - private StreamingMonitorFunction createFunction(ScanContext scanContext) { - return new StreamingMonitorFunction( - TestTableLoader.of(tableDir.getAbsolutePath()), scanContext); - } - - private AbstractStreamOperatorTestHarness createHarness( - StreamingMonitorFunction function) throws Exception { - StreamSource streamSource = - new StreamSource<>(function); - return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0); - } - - private class TestSourceContext implements SourceFunction.SourceContext { - private final List splits = Lists.newArrayList(); - private final Object checkpointLock = new Object(); - private final CountDownLatch latch; - - TestSourceContext(CountDownLatch latch) { - this.latch = latch; - } - - @Override - public void collect(FlinkInputSplit element) { - splits.add(element); - latch.countDown(); - } - - @Override - public void collectWithTimestamp(FlinkInputSplit element, long timestamp) { - collect(element); - } - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return checkpointLock; - } - - @Override - public void close() {} - - private List toRows() throws IOException { - FlinkInputFormat format = - FlinkSource.forRowData() - .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) - .buildFormat(); - - List rows = Lists.newArrayList(); - for (FlinkInputSplit split : splits) { - format.open(split); - - RowData element = null; - try { - while (!format.reachedEnd()) { - element = format.nextRecord(element); - rows.add(Row.of(element.getInt(0), element.getString(1).toString())); - } - } finally { - format.close(); - } - } - - return rows; - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java deleted file mode 100644 index 1606ee9f9648..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ /dev/null @@ -1,293 +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.source; - -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.Collections; -import java.util.List; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; -import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -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.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestStreamingReaderOperator extends TestBase { - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - - @Parameters(name = "formatVersion = {0}") - protected static List parameters() { - return Arrays.asList(1, 2); - } - - @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()); - } - - @TestTemplate - public void testProcessAllRecords() throws Exception { - List> expectedRecords = generateRecordsAndCommitTxn(10); - - List splits = generateSplits(); - assertThat(splits).hasSize(10); - - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - SteppingMailboxProcessor processor = createLocalMailbox(harness); - - List expected = Lists.newArrayList(); - for (int i = 0; i < splits.size(); i++) { - // Process this element to enqueue to mail-box. - harness.processElement(splits.get(i), -1); - - // Run the mail-box once to read all records from the given split. - assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); - - // Assert the output has expected elements. - expected.addAll(expectedRecords.get(i)); - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - } - } - - @TestTemplate - public void testTriggerCheckpoint() throws Exception { - // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading - // records from - // split1. - List> expectedRecords = generateRecordsAndCommitTxn(3); - - List splits = generateSplits(); - assertThat(splits).hasSize(3); - - long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - SteppingMailboxProcessor processor = createLocalMailbox(harness); - - harness.processElement(splits.get(0), ++timestamp); - harness.processElement(splits.get(1), ++timestamp); - harness.processElement(splits.get(2), ++timestamp); - - // Trigger snapshot state, it will start to work once all records from split0 are read. - processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - - assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); - assertThat(processor.runMailboxStep()) - .as("Should have processed the snapshot state action") - .isTrue(); - - TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); - - // Read records from split1. - assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); - - // Read records from split2. - assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); - - TestHelpers.assertRecords( - readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); - } - } - - @TestTemplate - public void testCheckpointRestore() throws Exception { - List> expectedRecords = generateRecordsAndCommitTxn(15); - - List splits = generateSplits(); - assertThat(splits).hasSize(15); - - OperatorSubtaskState state; - List expected = Lists.newArrayList(); - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - // Enqueue all the splits. - for (FlinkInputSplit split : splits) { - harness.processElement(split, -1); - } - - // Read all records from the first five splits. - SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); - for (int i = 0; i < 5; i++) { - expected.addAll(expectedRecords.get(i)); - assertThat(localMailbox.runMailboxStep()) - .as("Should have processed the split#" + i) - .isTrue(); - - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - - // Snapshot state now, there're 10 splits left in the state. - state = harness.snapshot(1, 1); - } - - expected.clear(); - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - // Recover to process the remaining splits. - harness.initializeState(state); - harness.open(); - - SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); - - for (int i = 5; i < 10; i++) { - expected.addAll(expectedRecords.get(i)); - assertThat(localMailbox.runMailboxStep()) - .as("Should have processed the split#" + i) - .isTrue(); - - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - - // Let's process the final 5 splits now. - for (int i = 10; i < 15; i++) { - expected.addAll(expectedRecords.get(i)); - harness.processElement(splits.get(i), 1); - - assertThat(localMailbox.runMailboxStep()) - .as("Should have processed the split#" + i) - .isTrue(); - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - } - } - - private List readOutputValues( - OneInputStreamOperatorTestHarness harness) { - List results = Lists.newArrayList(); - for (RowData rowData : harness.extractOutputValues()) { - results.add(Row.of(rowData.getInt(0), rowData.getString(1).toString())); - } - return results; - } - - private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { - List> expectedRecords = Lists.newArrayList(); - for (int i = 0; i < commitTimes; i++) { - List records = RandomGenericData.generate(SCHEMA, 100, 0L); - expectedRecords.add(records); - - // Commit those records to iceberg table. - writeRecords(records); - } - return expectedRecords; - } - - private void writeRecords(List records) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); - appender.appendToTable(records); - } - - private List generateSplits() { - List inputSplits = Lists.newArrayList(); - - List snapshotIds = SnapshotUtil.currentAncestorIds(table); - for (int i = snapshotIds.size() - 1; i >= 0; i--) { - ScanContext scanContext; - if (i == snapshotIds.size() - 1) { - // Generate the splits from the first snapshot. - scanContext = ScanContext.builder().useSnapshotId(snapshotIds.get(i)).build(); - } else { - // Generate the splits between the previous snapshot and current snapshot. - scanContext = - ScanContext.builder() - .startSnapshotId(snapshotIds.get(i + 1)) - .endSnapshotId(snapshotIds.get(i)) - .build(); - } - - Collections.addAll( - inputSplits, - FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool())); - } - - return inputSplits; - } - - private OneInputStreamOperatorTestHarness createReader() - throws Exception { - // This input format is used to opening the emitted split. - FlinkInputFormat inputFormat = - FlinkSource.forRowData() - .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) - .buildFormat(); - - OneInputStreamOperatorFactory factory = - StreamingReaderOperator.factory(inputFormat); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); - harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - - return harness; - } - - private SteppingMailboxProcessor createLocalMailbox( - OneInputStreamOperatorTestHarness harness) { - return new SteppingMailboxProcessor( - MailboxDefaultAction.Controller::suspendDefaultAction, - harness.getTaskMailbox(), - StreamTaskActionExecutor.IMMEDIATE); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java deleted file mode 100644 index 090b304942c6..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ /dev/null @@ -1,130 +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.source.assigner; - -import java.util.Collection; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public abstract class SplitAssignerTestBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Test - public void testEmptyInitialization() { - SplitAssigner assigner = splitAssigner(); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - /** Test a sequence of interactions for StaticEnumerator */ - @Test - public void testStaticEnumeratorSequence() throws Exception { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits(createSplits(4, 1, "1")); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertSnapshot(assigner, 1); - assigner.onUnassignedSplits(createSplits(1, 1, "1")); - assertSnapshot(assigner, 2); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - /** Test a sequence of interactions for ContinuousEnumerator */ - @Test - public void testContinuousEnumeratorSequence() throws Exception { - SplitAssigner assigner = splitAssigner(); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - - List splits1 = createSplits(1, 1, "1"); - assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = createSplits(1, 1, "1"); - assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - - assigner.onDiscoveredSplits(createSplits(2, 1, "1")); - assertSnapshot(assigner, 2); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - private void assertAvailableFuture( - SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { - // register callback - AtomicBoolean futureCompleted = new AtomicBoolean(); - CompletableFuture future = assigner.isAvailable(); - future.thenAccept(ignored -> futureCompleted.set(true)); - // calling isAvailable again should return the same object reference - // note that thenAccept will return a new future. - // we want to assert the same instance on the assigner returned future - Assert.assertSame(future, assigner.isAvailable()); - - // now add some splits - addSplitsRunnable.run(); - Assert.assertEquals(true, futureCompleted.get()); - - for (int i = 0; i < splitCount; ++i) { - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - } - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { - GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(expectedStatus, result.status()); - switch (expectedStatus) { - case AVAILABLE: - Assert.assertNotNull(result.split()); - break; - case CONSTRAINED: - case UNAVAILABLE: - Assert.assertNull(result.split()); - break; - default: - Assert.fail("Unknown status: " + expectedStatus); - } - } - - protected void assertSnapshot(SplitAssigner assigner, int splitCount) { - Collection stateBeforeGet = assigner.state(); - Assert.assertEquals(splitCount, stateBeforeGet.size()); - } - - protected List createSplits(int fileCount, int filesPerSplit, String version) - throws Exception { - return SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, fileCount, filesPerSplit, version); - } - - protected abstract SplitAssigner splitAssigner(); -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java deleted file mode 100644 index 8994f3054abe..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.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.flink.source.assigner; - -import org.apache.iceberg.flink.source.SplitHelpers; -import org.junit.Test; - -public class TestDefaultSplitAssigner extends SplitAssignerTestBase { - @Override - protected SplitAssigner splitAssigner() { - return new DefaultSplitAssigner(null); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInASplit() throws Exception { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertSnapshot(assigner, 1); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java deleted file mode 100644 index e78634e6b873..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ /dev/null @@ -1,80 +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.source.assigner; - -import java.util.List; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; - -public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { - @Override - protected SplitAssigner splitAssigner() { - return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInAnIcebergSplit() { - SplitAssigner assigner = splitAssigner(); - Assertions.assertThatThrownBy( - () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), - "Multiple files in a split is not allowed") - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Please use 'split-open-file-cost'"); - } - - /** Test sorted splits */ - @Test - public void testSplitSort() throws Exception { - SplitAssigner assigner = splitAssigner(); - List splits = createSplits(5, 1, "2"); - - assigner.onDiscoveredSplits(splits.subList(3, 5)); - assigner.onDiscoveredSplits(splits.subList(0, 1)); - assigner.onDiscoveredSplits(splits.subList(1, 3)); - - assertGetNext(assigner, 1L); - assertGetNext(assigner, 2L); - assertGetNext(assigner, 3L); - assertGetNext(assigner, 4L); - assertGetNext(assigner, 5L); - - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - @Test - public void testSerializable() { - byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); - SerializableComparator comparator = - SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); - } - - private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { - GetSplitResult result = assigner.getNext(null); - ContentFile file = result.split().task().files().iterator().next().file(); - Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java deleted file mode 100644 index e1fc63fda918..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ /dev/null @@ -1,146 +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.source.assigner; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; -import org.apache.iceberg.flink.source.reader.ReaderUtil; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; - -public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { - public static final Schema SCHEMA = - new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); - private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); - - @Override - protected SplitAssigner splitAssigner() { - return new OrderedSplitAssignerFactory( - SplitComparators.watermark( - new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) - .createAssigner(); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInAnIcebergSplit() { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits(createSplits(4, 2, "2")); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - /** Test sorted splits */ - @Test - public void testSplitSort() { - SplitAssigner assigner = splitAssigner(); - - Instant now = Instant.now(); - List splits = - IntStream.range(0, 5) - .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) - .collect(Collectors.toList()); - - assigner.onDiscoveredSplits(splits.subList(3, 5)); - assigner.onDiscoveredSplits(splits.subList(0, 1)); - assigner.onDiscoveredSplits(splits.subList(1, 3)); - - assertGetNext(assigner, splits.get(0)); - assertGetNext(assigner, splits.get(1)); - assertGetNext(assigner, splits.get(2)); - assertGetNext(assigner, splits.get(3)); - assertGetNext(assigner, splits.get(4)); - - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - @Test - public void testSerializable() { - byte[] bytes = - SerializationUtil.serializeToBytes( - SplitComparators.watermark( - new ColumnStatsWatermarkExtractor( - TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); - SerializableComparator comparator = - SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); - } - - private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { - GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(result.split(), split); - } - - @Override - protected List createSplits( - int fileCount, int filesPerSplit, String version) { - return IntStream.range(0, fileCount / filesPerSplit) - .mapToObj( - splitNum -> - splitFromRecords( - IntStream.range(0, filesPerSplit) - .mapToObj( - fileNum -> - RandomGenericData.generate( - SCHEMA, 2, splitNum * filesPerSplit + fileNum)) - .collect(Collectors.toList()))) - .collect(Collectors.toList()); - } - - private IcebergSourceSplit splitFromInstant(Instant instant) { - Record record = GenericRecord.create(SCHEMA); - record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); - return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); - } - - private IcebergSourceSplit splitFromRecords(List> records) { - try { - return IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); - } catch (IOException e) { - throw new RuntimeException("Split creation exception", e); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java deleted file mode 100644 index ebc92df02360..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java +++ /dev/null @@ -1,97 +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.source.enumerator; - -import java.io.IOException; -import java.util.List; -import java.util.NavigableMap; -import java.util.TreeMap; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class ManualContinuousSplitPlanner implements ContinuousSplitPlanner { - private final int maxPlanningSnapshotCount; - // track splits per snapshot - private final NavigableMap> splits; - private long latestSnapshotId; - private int remainingFailures; - - ManualContinuousSplitPlanner(ScanContext scanContext, int expectedFailures) { - this.maxPlanningSnapshotCount = scanContext.maxPlanningSnapshotCount(); - this.splits = new TreeMap<>(); - this.latestSnapshotId = 0L; - this.remainingFailures = expectedFailures; - } - - @Override - public synchronized ContinuousEnumerationResult planSplits( - IcebergEnumeratorPosition lastPosition) { - if (remainingFailures > 0) { - remainingFailures--; - throw new RuntimeException("Expected failure at planning"); - } - - long fromSnapshotIdExclusive = 0; - if (lastPosition != null && lastPosition.snapshotId() != null) { - fromSnapshotIdExclusive = lastPosition.snapshotId(); - } - - Preconditions.checkArgument( - fromSnapshotIdExclusive <= latestSnapshotId, - "last enumerated snapshotId is greater than the latestSnapshotId"); - if (fromSnapshotIdExclusive == latestSnapshotId) { - // already discovered everything. - return new ContinuousEnumerationResult(Lists.newArrayList(), lastPosition, lastPosition); - } - - // find the subset of snapshots to return discovered splits - long toSnapshotIdInclusive; - if (latestSnapshotId - fromSnapshotIdExclusive > maxPlanningSnapshotCount) { - toSnapshotIdInclusive = fromSnapshotIdExclusive + maxPlanningSnapshotCount; - } else { - toSnapshotIdInclusive = latestSnapshotId; - } - - List discoveredSplits = Lists.newArrayList(); - NavigableMap> discoveredView = - splits.subMap(fromSnapshotIdExclusive, false, toSnapshotIdInclusive, true); - discoveredView.forEach((snapshotId, snapshotSplits) -> discoveredSplits.addAll(snapshotSplits)); - ContinuousEnumerationResult result = - new ContinuousEnumerationResult( - discoveredSplits, - lastPosition, - // use the snapshot Id as snapshot timestamp. - IcebergEnumeratorPosition.of(toSnapshotIdInclusive, toSnapshotIdInclusive)); - return result; - } - - /** - * Add a collection of new splits. A monotonically increased snapshotId is assigned to each batch - * of splits added by this method. - */ - public synchronized void addSplits(List newSplits) { - latestSnapshotId += 1; - splits.put(latestSnapshotId, newSplits); - } - - @Override - public void close() throws IOException {} -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java deleted file mode 100644 index 349eb11cf549..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ /dev/null @@ -1,352 +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.source.enumerator; - -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestContinuousIcebergEnumerator { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Test - public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - Collection pendingSplitsEmpty = - enumerator.snapshotState(1).pendingSplits(); - Assert.assertEquals(0, pendingSplitsEmpty.size()); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); - } - - @Test - public void testDiscoverWhenReaderRegistered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register one reader, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .contains(splits.get(0)); - } - - @Test - public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register one reader, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // remove the reader (like in a failure) - enumeratorContext.registeredReaders().remove(2); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - Assert.assertEquals(1, splits.size()); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); - List pendingSplitIds = - enumerator.snapshotState(1).pendingSplits().stream() - .map(IcebergSourceSplitState::split) - .map(IcebergSourceSplit::splitId) - .collect(Collectors.toList()); - Assert.assertEquals(splits.size(), pendingSplitIds.size()); - Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); - - // register the reader again, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .contains(splits.get(0)); - } - - @Test - public void testThrottlingDiscovery() throws Exception { - // create 10 splits - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); - - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - // discover one snapshot at a time - .maxPlanningSnapshotCount(1) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register reader-2, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // add splits[0] to the planner for next discovery - splitPlanner.addSplits(Arrays.asList(splits.get(0))); - enumeratorContext.triggerAllActions(); - - // because discovered split was assigned to reader, pending splits should be empty - Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); - // split assignment to reader-2 should contain splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // add the remaining 9 splits (one for every snapshot) - // run discovery cycles while reader-2 still processing the splits[0] - for (int i = 1; i < 10; ++i) { - splitPlanner.addSplits(Arrays.asList(splits.get(i))); - enumeratorContext.triggerAllActions(); - } - - // can only discover up to 3 snapshots/splits - Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); - // split assignment to reader-2 should be splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // now reader-2 finished splits[0] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); - enumeratorContext.triggerAllActions(); - // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was - // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); - // split assignment to reader-2 should be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // run 3 more split discovery cycles - for (int i = 0; i < 3; ++i) { - enumeratorContext.triggerAllActions(); - } - - // no more splits are discovered due to throttling - Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); - // split assignment to reader-2 should still be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - - // now reader-2 finished splits[1] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); - enumeratorContext.triggerAllActions(); - // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was - // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); - // split assignment to reader-2 should be splits[0, 3) - Assert.assertEquals( - splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); - } - - @Test - public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(2) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 1); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - - // Trigger a planning and check that no splits returned due to the planning error - enumeratorContext.triggerAllActions(); - Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); - - // Second scan planning should succeed and discover the expected splits - enumeratorContext.triggerAllActions(); - Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); - } - - @Test - public void testOverMaxAllowedPlanningErrors() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(1) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 2); - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - - // Check that the scheduler response ignores the current error and continues to run until the - // failure limit is reached - enumeratorContext.triggerAllActions(); - Assert.assertFalse( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - - // Check that the task has failed with the expected exception after the failure limit is reached - enumeratorContext.triggerAllActions(); - Assert.assertTrue( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - Assertions.assertThatThrownBy( - () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) - .hasCauseInstanceOf(RuntimeException.class) - .hasMessageContaining("Failed to discover new split"); - } - - @Test - public void testPlanningIgnoringErrors() throws Exception { - int expectedFailures = 3; - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(-1) - .build(); - ManualContinuousSplitPlanner splitPlanner = - new ManualContinuousSplitPlanner(scanContext, expectedFailures); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - splitPlanner.addSplits(splits); - - Collection pendingSplits; - // Can not discover the new split with planning failures - for (int i = 0; i < expectedFailures; ++i) { - enumeratorContext.triggerAllActions(); - pendingSplits = enumerator.snapshotState(i).pendingSplits(); - Assert.assertEquals(0, pendingSplits.size()); - } - - // Discovered the new split after a successful scan planning - enumeratorContext.triggerAllActions(); - pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); - } - - private static ContinuousIcebergEnumerator createEnumerator( - SplitEnumeratorContext context, - ScanContext scanContext, - ContinuousSplitPlanner splitPlanner) { - - ContinuousIcebergEnumerator enumerator = - new ContinuousIcebergEnumerator( - context, - new DefaultSplitAssigner(null, Collections.emptyList()), - scanContext, - splitPlanner, - null); - enumerator.start(); - return enumerator; - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java deleted file mode 100644 index bb747caae589..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ /dev/null @@ -1,688 +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.source.enumerator; - -import java.io.IOException; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; - -public class TestContinuousSplitPlannerImpl { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static final FileFormat fileFormat = FileFormat.PARQUET; - private static final AtomicLong randomSeed = new AtomicLong(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - @Rule public TestName testName = new TestName(); - - private GenericAppenderHelper dataAppender; - private DataFile dataFile1; - private Snapshot snapshot1; - private DataFile dataFile2; - private Snapshot snapshot2; - - @Before - public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); - } - - private void appendTwoSnapshots() throws IOException { - // snapshot1 - List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - dataFile1 = dataAppender.writeFile(null, batch1); - dataAppender.appendToTable(dataFile1); - snapshot1 = tableResource.table().currentSnapshot(); - - // snapshot2 - List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); - dataFile2 = dataAppender.writeFile(null, batch2); - dataAppender.appendToTable(dataFile2); - snapshot2 = tableResource.table().currentSnapshot(); - } - - /** @return the last enumerated snapshot id */ - private CycleResult verifyOneCycle( - ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) - throws Exception { - List batch = - RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); - DataFile dataFile = dataAppender.writeFile(null, batch); - dataAppender.appendToTable(dataFile); - Snapshot snapshot = tableResource.table().currentSnapshot(); - - ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); - Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); - Assert.assertEquals( - lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, result.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Assert.assertEquals( - dataFile.path().toString(), - Iterables.getOnlyElement(split.task().files()).file().path().toString()); - return new CycleResult(result.toPosition(), split); - } - - @Test - public void testTableScanThenIncrementalWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - Assert.assertEquals( - snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .splitSize(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); - - // latest mode should discover both snapshots, as latest position is marked by when job starts - appendTwoSnapshots(); - ContinuousEnumerationResult afterTwoSnapshotsAppended = - splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); - Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1 - // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1's parent, - // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertNull(secondResult.fromPosition().snapshotId()); - Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromSnapshotIdWithEmptyTable() { - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: 1"); - } - - @Test - public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { - appendTwoSnapshots(); - - // find an invalid snapshotId - long invalidSnapshotId = 0L; - while (invalidSnapshotId == snapshot1.snapshotId() - || invalidSnapshotId == snapshot2.snapshotId()) { - invalidSnapshotId++; - } - - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(invalidSnapshotId) - .build(); - - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); - } - - @Test - public void testIncrementalFromSnapshotId() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot2.snapshotId()) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as - // snapshot2's parent) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromSnapshotTimestampWithEmptyTable() { - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot find a snapshot after: 1"); - } - - @Test - public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exception { - appendTwoSnapshots(); - - long invalidSnapshotTimestampMs = snapshot2.timestampMillis() + 1000L; - - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(invalidSnapshotTimestampMs) - .build(); - - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); - } - - @Test - public void testIncrementalFromSnapshotTimestamp() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis()) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testMaxPlanningSnapshotCount() throws Exception { - appendTwoSnapshots(); - // append 3 more snapshots - for (int i = 2; i < 5; ++i) { - appendSnapshot(i, 2); - } - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - // limit to 1 snapshot per discovery - .maxPlanningSnapshotCount(1) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - // For inclusive behavior, the initial result should point to snapshot1's parent, - // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - // should discover dataFile1 appended in snapshot1 - verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); - - ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); - // should discover dataFile2 appended in snapshot2 - verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); - } - - @Test - public void testTableScanNoStats() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(false) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - verifyStatCount(split, 0); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 0); - lastPosition = result.lastPosition; - } - } - - @Test - public void testTableScanAllStats() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - verifyStatCount(split, 3); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 3); - lastPosition = result.lastPosition; - } - } - - @Test - public void testTableScanSingleStat() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(ImmutableSet.of("data")) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); - verifyStatCount(split, 1); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 1); - lastPosition = result.lastPosition; - } - } - - private void verifyStatCount(IcebergSourceSplit split, int expected) { - if (expected == 0) { - split - .task() - .files() - .forEach( - f -> { - Assert.assertNull(f.file().valueCounts()); - Assert.assertNull(f.file().columnSizes()); - Assert.assertNull(f.file().lowerBounds()); - Assert.assertNull(f.file().upperBounds()); - Assert.assertNull(f.file().nanValueCounts()); - Assert.assertNull(f.file().nullValueCounts()); - }); - } else { - split - .task() - .files() - .forEach( - f -> { - Assert.assertEquals(expected, f.file().valueCounts().size()); - Assert.assertEquals(expected, f.file().columnSizes().size()); - Assert.assertEquals(expected, f.file().lowerBounds().size()); - Assert.assertEquals(expected, f.file().upperBounds().size()); - Assert.assertEquals(expected, f.file().nullValueCounts().size()); - // The nanValue is not counted for long and string fields - Assert.assertEquals(0, f.file().nanValueCounts().size()); - }); - } - } - - private void verifyMaxPlanningSnapshotCountResult( - ContinuousEnumerationResult result, - Snapshot fromSnapshotExclusive, - Snapshot toSnapshotInclusive, - Set expectedFiles) { - if (fromSnapshotExclusive == null) { - Assert.assertNull(result.fromPosition().snapshotId()); - Assert.assertNull(result.fromPosition().snapshotTimestampMs()); - } else { - Assert.assertEquals( - fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - fromSnapshotExclusive.timestampMillis(), - result.fromPosition().snapshotTimestampMs().longValue()); - } - Assert.assertEquals( - toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - toSnapshotInclusive.timestampMillis(), - result.toPosition().snapshotTimestampMs().longValue()); - // should only have one split with one data file, because split discover is limited to - // one snapshot and each snapshot has only one data file appended. - IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - Assert.assertEquals(expectedFiles, discoveredFiles); - } - - private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { - List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); - DataFile dataFile = dataAppender.writeFile(null, batch); - dataAppender.appendToTable(dataFile); - return tableResource.table().currentSnapshot(); - } - - private static class CycleResult { - IcebergEnumeratorPosition lastPosition; - IcebergSourceSplit split; - - CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { - this.lastPosition = lastPosition; - this.split = split; - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java deleted file mode 100644 index 2c94f215908a..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ /dev/null @@ -1,205 +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.source.enumerator; - -import java.io.IOException; -import java.util.List; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.RuleChain; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestRule; - -public class TestContinuousSplitPlannerImplStartStrategy { - private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - public final HadoopTableResource tableResource = - new HadoopTableResource( - temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); - - private GenericAppenderHelper dataAppender; - private Snapshot snapshot1; - private Snapshot snapshot2; - private Snapshot snapshot3; - - @Before - public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); - } - - private void appendThreeSnapshots() throws IOException { - List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - dataAppender.appendToTable(batch1); - snapshot1 = tableResource.table().currentSnapshot(); - - List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); - dataAppender.appendToTable(batch2); - snapshot2 = tableResource.table().currentSnapshot(); - - List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); - dataAppender.appendToTable(batch3); - snapshot3 = tableResource.table().currentSnapshot(); - } - - @Test - public void testTableScanThenIncrementalStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForLatestSnapshotStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForEarliestSnapshotStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForSpecificSnapshotIdStrategy() throws IOException { - ScanContext scanContextInvalidSnapshotId = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .build(); - - // empty table - Assertions.assertThatThrownBy( - () -> - ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotId)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: 1"); - - appendThreeSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot2.snapshotId()) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOException { - ScanContext scanContextInvalidSnapshotTimestamp = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(1L) - .build(); - - // empty table - Assertions.assertThatThrownBy( - () -> - ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotTimestamp)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); - - appendThreeSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis()) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); - } - - @Test - public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOException { - appendThreeSnapshots(); - - ScanContext config = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis() - 1L) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java deleted file mode 100644 index e2be0b4b0398..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java +++ /dev/null @@ -1,134 +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.source.enumerator; - -import org.junit.Assert; -import org.junit.Test; - -public class TestEnumerationHistory { - private static final int MAX_HISTORY_SIZE = 3; - private static final int FEW_PENDING_SPLITS = 2; - private static final int TOO_MANY_PENDING_SPLITS = 100; - - @Test - public void testEmptyHistory() { - EnumerationHistory history = new EnumerationHistory(MAX_HISTORY_SIZE); - int[] expectedHistorySnapshot = new int[0]; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testNotFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - int[] expectedHistorySnapshot = {1, 2}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testExactFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - int[] expectedHistorySnapshot = {1, 2, 3}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testOneMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - int[] expectedHistorySnapshot = {2, 3, 4}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testTwoMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - history.add(5); - int[] expectedHistorySnapshot = {3, 4, 5}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testThreeMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - history.add(5); - history.add(6); - int[] expectedHistorySnapshot = {4, 5, 6}; - testHistory(history, expectedHistorySnapshot); - } - - private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); - if (history.hasFullHistory()) { - // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); - } else { - // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); - } - - int[] historySnapshot = history.snapshot(); - Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); - - EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); - restoredHistory.restore(historySnapshot); - - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); - if (history.hasFullHistory()) { - // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); - } else { - // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); - } - } - - @Test - public void testRestoreDifferentSize() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - int[] historySnapshot = history.snapshot(); - - EnumerationHistory smallerHistory = new EnumerationHistory(2); - smallerHistory.restore(historySnapshot); - int[] expectedRestoredHistorySnapshot = {2, 3}; - Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); - - EnumerationHistory largerHisotry = new EnumerationHistory(4); - largerHisotry.restore(historySnapshot); - Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java deleted file mode 100644 index 1d12d9f66a8a..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ /dev/null @@ -1,145 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergEnumeratorStateSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final IcebergEnumeratorStateSerializer serializer = - new IcebergEnumeratorStateSerializer(true); - - protected final int version; - - @Parameterized.Parameters(name = "version={0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } - - public TestIcebergEnumeratorStateSerializer(int version) { - this.version = version; - } - - @Test - public void testEmptySnapshotIdAndPendingSplits() throws Exception { - IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); - testSerializer(enumeratorState); - } - - @Test - public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); - - IcebergEnumeratorState enumeratorState = - new IcebergEnumeratorState(position, Collections.emptyList()); - testSerializer(enumeratorState); - } - - @Test - public void testSomeSnapshotIdAndPendingSplits() throws Exception { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); - Collection pendingSplits = Lists.newArrayList(); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); - - IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(position, pendingSplits); - testSerializer(enumeratorState); - } - - @Test - public void testEnumerationSplitCountHistory() throws Exception { - if (version == 2) { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); - Collection pendingSplits = Lists.newArrayList(); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); - int[] enumerationSplitCountHistory = {1, 2, 3}; - - IcebergEnumeratorState enumeratorState = - new IcebergEnumeratorState(position, pendingSplits, enumerationSplitCountHistory); - testSerializer(enumeratorState); - } - } - - private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOException { - byte[] result; - if (version == 1) { - result = serializer.serializeV1(enumeratorState); - } else { - result = serializer.serialize(enumeratorState); - } - - IcebergEnumeratorState deserialized = serializer.deserialize(version, result); - assertEnumeratorStateEquals(enumeratorState, deserialized); - } - - private void assertEnumeratorStateEquals( - IcebergEnumeratorState expected, IcebergEnumeratorState actual) { - Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); - - Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); - Iterator expectedIterator = expected.pendingSplits().iterator(); - Iterator actualIterator = actual.pendingSplits().iterator(); - for (int i = 0; i < expected.pendingSplits().size(); ++i) { - IcebergSourceSplitState expectedSplitState = expectedIterator.next(); - IcebergSourceSplitState actualSplitState = actualIterator.next(); - Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); - Assert.assertEquals( - expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); - Assert.assertEquals( - expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); - Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); - } - - Assert.assertArrayEquals( - expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java deleted file mode 100644 index 0d33e4ed08ad..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ /dev/null @@ -1,219 +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.source.reader; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public abstract class ReaderFunctionTestBase { - - @Parameterized.Parameters(name = "fileFormat={0}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.PARQUET} - }; - } - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - protected abstract ReaderFunction readerFunction(); - - protected abstract void assertRecords(List expected, List actual, Schema schema); - - private final FileFormat fileFormat; - private final GenericAppenderFactory appenderFactory; - - public ReaderFunctionTestBase(FileFormat fileFormat) { - this.fileFormat = fileFormat; - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } - - private void assertRecordsAndPosition( - List expectedRecords, - int expectedFileOffset, - long startRecordOffset, - RecordsWithSplitIds> batch) { - batch.nextSplit(); - List actualRecords = Lists.newArrayList(); - long recordOffset = startRecordOffset; - RecordAndPosition recordAndPosition; - while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { - actualRecords.add(recordAndPosition.record()); - Assert.assertEquals( - "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); - Assert.assertEquals( - "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); - recordOffset++; - } - - Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); - assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); - } - - @Test - public void testNoCheckpointedPosition() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionBeforeFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionMiddleFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0).subList(1, 2), 0, 1L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionAfterFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionBeforeSecondFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @Test - public void testCheckpointedPositionMidSecondFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1).subList(1, 2), 1, 1L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java deleted file mode 100644 index 2a2503ef2478..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ /dev/null @@ -1,125 +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.source.reader; - -import java.io.File; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.BaseFileScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Files; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.expressions.ResidualEvaluator; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.rules.TemporaryFolder; - -public class ReaderUtil { - - private ReaderUtil() {} - - public static FileScanTask createFileTask( - List records, - File file, - FileFormat fileFormat, - FileAppenderFactory appenderFactory) - throws IOException { - FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat); - try { - appender.addAll(records); - } finally { - appender.close(); - } - - DataFile dataFile = - DataFiles.builder(PartitionSpec.unpartitioned()) - .withRecordCount(records.size()) - .withFileSizeInBytes(file.length()) - .withPath(file.toString()) - .withFormat(fileFormat) - .withMetrics(appender.metrics()) - .build(); - - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); - return new BaseFileScanTask( - dataFile, - null, - SchemaParser.toJson(TestFixtures.SCHEMA), - PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), - residuals); - } - - public static DataIterator createDataIterator(CombinedScanTask combinedTask) { - return new DataIterator<>( - new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), - combinedTask, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager()); - } - - public static List> createRecordBatchList( - Schema schema, int listSize, int batchCount) { - return createRecordBatchList(0L, schema, listSize, batchCount); - } - - public static List> createRecordBatchList( - long seed, Schema schema, int listSize, int batchCount) { - List records = RandomGenericData.generate(schema, listSize * batchCount, seed); - return Lists.partition(records, batchCount); - } - - public static CombinedScanTask createCombinedScanTask( - List> recordBatchList, - TemporaryFolder temporaryFolder, - FileFormat fileFormat, - GenericAppenderFactory appenderFactory) - throws IOException { - List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); - for (List recordBatch : recordBatchList) { - FileScanTask fileTask = - ReaderUtil.createFileTask( - recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); - fileTasks.add(fileTask); - } - - return new BaseCombinedScanTask(fileTasks); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java deleted file mode 100644 index 644ac2bad6b8..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.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.flink.source.reader; - -import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Assert; -import org.junit.Test; - -public class TestArrayBatchRecords { - - @Test - public void testFullRange() { - String[] elements = new String[] {"0", "1", "2", "3"}; - testArray(elements, elements.length, 2, 119); - } - - @Test - public void testSubRange() { - String[] elements = new String[] {"0", "1", "2", "3"}; - testArray(elements, 2, 0, 0); - } - - private void testArray( - String[] elements, int numberOfRecords, int fileOffset, long startingRecordOffset) { - String splitId = "iceberg_split_1"; - AtomicBoolean recycled = new AtomicBoolean(); - - ArrayBatchRecords recordsWithSplitIds = - ArrayBatchRecords.forRecords( - splitId, - ignored -> recycled.set(true), - elements, - numberOfRecords, - fileOffset, - startingRecordOffset); - - Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); - - for (int i = 0; i < numberOfRecords; i++) { - RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); - Assert.assertEquals(elements[i], recAndPos.record()); - Assert.assertEquals(fileOffset, recAndPos.fileOffset()); - // recordOffset points to the position after this one - Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); - } - - Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); - Assert.assertNull(recordsWithSplitIds.nextSplit()); - recordsWithSplitIds.recycle(); - Assert.assertTrue(recycled.get()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java deleted file mode 100644 index f964a7707689..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ /dev/null @@ -1,358 +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.source.reader; - -import java.util.Arrays; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SourceReaderOptions; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestArrayPoolDataIteratorBatcherRowData { - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private static final FileFormat fileFormat = FileFormat.PARQUET; - - private final GenericAppenderFactory appenderFactory; - private final DataIteratorBatcher batcher; - - public TestArrayPoolDataIteratorBatcherRowData() { - Configuration config = new Configuration(); - // set array pool size to 1 - config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); - // set batch array size to 2 - config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - this.batcher = - new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } - - /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ - @Test - public void testSingleFileLessThanOneFullBatch() throws Exception { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); - FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch.numberOfRecords()); - - RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); - - /////////////////////////////// - // assert first record - - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - - Assert.assertNull(batch.nextRecordFromSplit()); - Assert.assertNull(batch.nextSplit()); - batch.recycle(); - - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); - } - - /** - * Read a CombinedScanTask that contains a single file with multiple batches. - * - *

    Insert 5 records in a single file that should result in 3 batches - */ - @Test - public void testSingleFileWithMultipleBatches() throws Exception { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); - FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - /////////////////////////////// - // assert first batch with full batch of 2 records - - ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch0.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch0.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch0.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch0.numberOfRecords()); - - RecordAndPosition recordAndPosition; - - // assert first record - recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - - // assert second record - recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); - - Assert.assertNull(batch0.nextRecordFromSplit()); - Assert.assertNull(batch0.nextSplit()); - batch0.recycle(); - - /////////////////////////////// - // assert second batch with full batch of 2 records - - ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch1.records()); - Assert.assertTrue(batch1.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch1.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch1.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch1.numberOfRecords()); - - // assert third record - recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); - - // assert fourth record - recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); - - Assert.assertNull(batch1.nextRecordFromSplit()); - Assert.assertNull(batch1.nextSplit()); - batch1.recycle(); - - /////////////////////////////// - // assert third batch with partial batch of 1 record - - ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch2.records()); - Assert.assertTrue(batch2.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch2.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch2.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch2.numberOfRecords()); - - // assert fifth record - recordAndPosition = batch2.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(5, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); - - Assert.assertNull(batch2.nextRecordFromSplit()); - Assert.assertNull(batch2.nextSplit()); - batch2.recycle(); - - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); - } - - /** - * Read a CombinedScanTask that contains with multiple files. - * - *

    In this test, we also seek the iterator to starting position (1, 1). - */ - @Test - public void testMultipleFilesWithSeekPosition() throws Exception { - List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); - FileScanTask fileTask0 = - ReaderUtil.createFileTask( - records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); - FileScanTask fileTask1 = - ReaderUtil.createFileTask( - records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); - FileScanTask fileTask2 = - ReaderUtil.createFileTask( - records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); - CombinedScanTask combinedTask = - new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); - - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - // seek to file1 and after record 1 - dataIterator.seek(1, 1); - - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - /////////////////////////////// - // file0 is skipped by seek - - /////////////////////////////// - // file1 has 4 records. because the seek position, first record is skipped. - // we should read 3 remaining records in 2 batches: - // batch10 with 2 records and batch11 with 1 records. - - // assert first batch from file1 with full batch of 2 records - - // variable naming convention: batch - ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch10.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch10.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch10.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch10.numberOfRecords()); - - RecordAndPosition recordAndPosition; - - recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // seek should skip the first record in file1. starting from the second record - Assert.assertEquals(2, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); - - recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); - - Assert.assertNull(batch10.nextRecordFromSplit()); - Assert.assertNull(batch10.nextSplit()); - batch10.recycle(); - - // assert second batch from file1 with partial batch of 1 record - - // variable naming convention: batch__ - ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch11.records()); - Assert.assertTrue(batch11.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch11.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch11.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch11.numberOfRecords()); - - recordAndPosition = batch11.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); - - Assert.assertNull(batch11.nextRecordFromSplit()); - Assert.assertNull(batch11.nextSplit()); - batch11.recycle(); - - /////////////////////////////// - // file2 has 3 records. - // we should read 3 records in 2 batches: - // batch20 with 2 records and batch21 with 1 records - - // assert first batch from file2 with full batch of 2 records - - // variable naming convention: batch__ - ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch20.records()); - Assert.assertTrue(batch20.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch20.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch20.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch20.numberOfRecords()); - - recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); - - recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); - - Assert.assertNull(batch20.nextRecordFromSplit()); - Assert.assertNull(batch20.nextSplit()); - batch20.recycle(); - - /////////////////////////////// - // assert second batch from file2 with partial batch of 1 record - - // variable naming convention: batch__ - ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch21.records()); - Assert.assertTrue(batch21.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch21.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch21.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch21.numberOfRecords()); - - recordAndPosition = batch21.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); - - Assert.assertNull(batch21.nextRecordFromSplit()); - Assert.assertNull(batch21.nextSplit()); - batch21.recycle(); - - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java deleted file mode 100644 index afe8a5d0152c..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ /dev/null @@ -1,178 +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.source.reader; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.time.LocalDateTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestColumnStatsWatermarkExtractor { - public static final Schema SCHEMA = - new Schema( - required(1, "timestamp_column", Types.TimestampType.withoutZone()), - required(2, "timestamptz_column", Types.TimestampType.withZone()), - required(3, "long_column", Types.LongType.get()), - required(4, "string_column", Types.StringType.get())); - - private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); - - private static final List> TEST_RECORDS = - ImmutableList.of( - RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); - - private static final List> MIN_VALUES = - ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); - - private final String columnName; - - @BeforeClass - public static void updateMinValue() { - for (int i = 0; i < TEST_RECORDS.size(); ++i) { - for (Record r : TEST_RECORDS.get(i)) { - Map minValues = MIN_VALUES.get(i); - - LocalDateTime localDateTime = (LocalDateTime) r.get(0); - minValues.merge( - "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); - - OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); - minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); - - minValues.merge("long_column", (Long) r.get(2), Math::min); - } - } - } - - @Parameterized.Parameters(name = "{0}") - public static Collection data() { - return ImmutableList.of( - new Object[] {"timestamp_column"}, - new Object[] {"timestamptz_column"}, - new Object[] {"long_column"}); - } - - public TestColumnStatsWatermarkExtractor(String columnName) { - this.columnName = columnName; - } - - @Test - public void testSingle() throws IOException { - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); - - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - } - - @Test - public void testTimeUnit() throws IOException { - Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); - - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue() / 1000L, - extractor.extractWatermark(split(0))); - } - - @Test - public void testMultipleFiles() throws IOException { - Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); - IcebergSourceSplit combinedSplit = - IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); - - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); - - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - Assert.assertEquals( - MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); - Assert.assertEquals( - Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), - extractor.extractWatermark(combinedSplit)); - } - - @Test - public void testWrongColumn() { - Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); - Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining( - "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); - } - - @Test - public void testEmptyStatistics() throws IOException { - Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); - - // Create an extractor for a column we do not have statistics - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(10, "missing_field"); - Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Missing statistics for column"); - } - - private IcebergSourceSplit split(int id) throws IOException { - return IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - ImmutableList.of(TEST_RECORDS.get(id)), - TEMPORARY_FOLDER, - FileFormat.PARQUET, - APPENDER_FACTORY)); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java deleted file mode 100644 index f19d57083b89..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ /dev/null @@ -1,184 +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.source.reader; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; -import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceReader { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final GenericAppenderFactory appenderFactory; - - public TestIcebergSourceReader() { - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } - - @Test - public void testReaderMetrics() throws Exception { - TestingReaderOutput readerOutput = new TestingReaderOutput<>(); - TestingMetricGroup metricGroup = new TestingMetricGroup(); - TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - IcebergSourceReader reader = createReader(metricGroup, readerContext, null); - reader.start(); - - testOneSplitFetcher(reader, readerOutput, metricGroup, 1); - testOneSplitFetcher(reader, readerOutput, metricGroup, 2); - } - - @Test - public void testReaderOrder() throws Exception { - // Create 2 splits - List> recordBatchList1 = - ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task1 = - ReaderUtil.createCombinedScanTask( - recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); - - List> recordBatchList2 = - ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task2 = - ReaderUtil.createCombinedScanTask( - recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); - - // Sort the splits in one way - List rowDataList1 = - read( - Arrays.asList( - IcebergSourceSplit.fromCombinedScanTask(task1), - IcebergSourceSplit.fromCombinedScanTask(task2)), - 2); - - // Reverse the splits - List rowDataList2 = - read( - Arrays.asList( - IcebergSourceSplit.fromCombinedScanTask(task2), - IcebergSourceSplit.fromCombinedScanTask(task1)), - 2); - - // Check that the order of the elements is not changed - Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); - Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); - } - - private List read(List splits, long expected) throws Exception { - TestingMetricGroup metricGroup = new TestingMetricGroup(); - TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - // Using IdBasedComparator, so we can have a deterministic order of the splits - IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); - reader.start(); - - reader.addSplits(splits); - TestingReaderOutput readerOutput = new TestingReaderOutput<>(); - while (readerOutput.getEmittedRecords().size() < expected) { - reader.pollNext(readerOutput); - } - - reader.pollNext(readerOutput); - - Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); - return readerOutput.getEmittedRecords(); - } - - private void testOneSplitFetcher( - IcebergSourceReader reader, - TestingReaderOutput readerOutput, - TestingMetricGroup metricGroup, - int expectedCount) - throws Exception { - long seed = expectedCount; - // Each split should contain only one file with one record - List> recordBatchList = - ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task = - ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Collections.singletonList(split)); - - while (readerOutput.getEmittedRecords().size() < expectedCount) { - reader.pollNext(readerOutput); - } - - Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); - TestHelpers.assertRowData( - TestFixtures.SCHEMA, - recordBatchList.get(0).get(0), - readerOutput.getEmittedRecords().get(expectedCount - 1)); - Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); - - // One more poll will get null record batch. - // That will finish the split and cause split fetcher to be closed due to idleness. - // Then next split will create a new split reader. - reader.pollNext(readerOutput); - } - - private IcebergSourceReader createReader( - MetricGroup metricGroup, - SourceReaderContext readerContext, - SerializableComparator splitComparator) { - IcebergSourceReaderMetrics readerMetrics = - new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); - RowDataReaderFunction readerFunction = - new RowDataReaderFunction( - new Configuration(), - TestFixtures.SCHEMA, - TestFixtures.SCHEMA, - null, - true, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), - Collections.emptyList()); - return new IcebergSourceReader<>( - SerializableRecordEmitter.defaultEmitter(), - readerMetrics, - readerFunction, - splitComparator, - readerContext); - } - - private static class IdBasedComparator implements SerializableComparator { - @Override - public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { - return o1.splitId().compareTo(o2.splitId()); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java deleted file mode 100644 index d063ad7f4a80..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ /dev/null @@ -1,74 +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.source.reader; - -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.configuration.Configuration; -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.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.hadoop.HadoopFileIO; - -public class TestRowDataReaderFunction extends ReaderFunctionTestBase { - - protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final DataStructureConverter rowDataConverter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - - public TestRowDataReaderFunction(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected ReaderFunction readerFunction() { - return new RowDataReaderFunction( - new Configuration(), - TestFixtures.SCHEMA, - TestFixtures.SCHEMA, - null, - true, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), - Collections.emptyList()); - } - - @Override - protected void assertRecords(List expected, List actual, Schema schema) { - List rows = toRows(actual); - TestHelpers.assertRecords(rows, expected, TestFixtures.SCHEMA); - } - - private List toRows(List actual) { - return actual.stream() - .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java deleted file mode 100644 index 290628c5fc90..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java +++ /dev/null @@ -1,102 +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.source.reader; - -import java.util.Map; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.SimpleCounter; -import org.apache.flink.metrics.groups.OperatorIOMetricGroup; -import org.apache.flink.metrics.groups.SourceReaderMetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -class TestingMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { - private final Map counters; - - TestingMetricGroup() { - this.counters = Maps.newHashMap(); - } - - /** Pass along the reference to share the map for child metric groups. */ - private TestingMetricGroup(Map counters) { - this.counters = counters; - } - - Map counters() { - return counters; - } - - @Override - public Counter counter(String name) { - Counter counter = new SimpleCounter(); - counters.put(name, counter); - return counter; - } - - @Override - public MetricGroup addGroup(String name) { - return new TestingMetricGroup(counters); - } - - @Override - public MetricGroup addGroup(String key, String value) { - return new TestingMetricGroup(counters); - } - - @Override - public OperatorIOMetricGroup getIOMetricGroup() { - return new TestingOperatorIOMetricGroup(); - } - - @Override - public Counter getNumRecordsInErrorsCounter() { - return new SimpleCounter(); - } - - @Override - public void setPendingBytesGauge(Gauge pendingBytesGauge) {} - - @Override - public void setPendingRecordsGauge(Gauge pendingRecordsGauge) {} - - private static class TestingOperatorIOMetricGroup extends UnregisteredMetricsGroup - implements OperatorIOMetricGroup { - @Override - public Counter getNumRecordsInCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumRecordsOutCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumBytesInCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumBytesOutCounter() { - return new SimpleCounter(); - } - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java deleted file mode 100644 index cd778309f90d..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ /dev/null @@ -1,162 +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.source.split; - -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestIcebergSourceSplitSerializer { - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); - - @Test - public void testLatestVersion() throws Exception { - serializeAndDeserialize(1, 1); - serializeAndDeserialize(10, 2); - } - - private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = serializer.serialize(split); - IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); - assertSplitEquals(split, deserialized); - - byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); - IcebergSourceSplit deserialized2 = - serializer.deserialize(serializer.getVersion(), cachedResult); - assertSplitEquals(split, deserialized2); - - split.updatePosition(0, 100); - byte[] resultAfterUpdatePosition = serializer.serialize(split); - // after position change, serialized bytes should have changed - Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); - IcebergSourceSplit deserialized3 = - serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); - assertSplitEquals(split, deserialized3); - } - } - - @Test - public void testV1() throws Exception { - serializeAndDeserializeV1(1, 1); - serializeAndDeserializeV1(10, 2); - } - - private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV1(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testV2() throws Exception { - serializeAndDeserializeV2(1, 1); - serializeAndDeserializeV2(10, 2); - } - - private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV2(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testDeserializeV1() throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV1(); - IcebergSourceSplit deserialized = serializer.deserialize(1, result); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testCheckpointedPosition() throws Exception { - final AtomicInteger index = new AtomicInteger(); - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() - .map( - split -> { - IcebergSourceSplit result; - if (index.get() % 2 == 0) { - result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); - } else { - result = split; - } - index.incrementAndGet(); - return result; - }) - .collect(Collectors.toList()); - - for (IcebergSourceSplit split : splits) { - byte[] result = serializer.serialize(split); - IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); - assertSplitEquals(split, deserialized); - - byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); - IcebergSourceSplit deserialized2 = - serializer.deserialize(serializer.getVersion(), cachedResult); - assertSplitEquals(split, deserialized2); - } - } - - private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { - List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); - List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); - Assert.assertEquals(expectedTasks.size(), actualTasks.size()); - for (int i = 0; i < expectedTasks.size(); ++i) { - FileScanTask expectedTask = expectedTasks.get(i); - FileScanTask actualTask = actualTasks.get(i); - Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); - Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); - Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); - Assert.assertEquals(expectedTask.start(), actualTask.start()); - Assert.assertEquals(expectedTask.length(), actualTask.length()); - } - - Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); - Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); - } -} diff --git a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java deleted file mode 100644 index 9d99193fb1be..000000000000 --- a/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ /dev/null @@ -1,54 +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.util; - -import org.junit.Assert; -import org.junit.Test; -import org.mockito.MockedStatic; -import org.mockito.Mockito; - -public class TestFlinkPackage { - - /** This unit test would need to be adjusted as new Flink version is supported. */ - @Test - public void testVersion() { - Assert.assertEquals("1.16.3", FlinkPackage.version()); - } - - @Test - public void testDefaultVersion() { - // It's difficult to reproduce a reflection error in a unit test, so we just inject a mocked - // fault to test the default logic - - // First make sure we're not caching a version result from a previous test - FlinkPackage.setVersion(null); - try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { - mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class); - mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); - Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, FlinkPackage.version()); - } - FlinkPackage.setVersion(null); - try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { - mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null); - mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); - FlinkPackage.setVersion(null); - Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, FlinkPackage.version()); - } - } -} diff --git a/flink/v1.16/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.16/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 47a3c94aa991..000000000000 --- a/flink/v1.16/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.source.BoundedTableFactory From 5b9110192243447d7126f80fec323462815befe9 Mon Sep 17 00:00:00 2001 From: Rodrigo Date: Tue, 16 Apr 2024 22:11:54 -0700 Subject: [PATCH 0230/1019] Docs: Updates flink versioning information in our docs (#10155) --- docs/docs/flink-writes.md | 2 +- site/docs/multi-engine-support.md | 5 +++-- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index ef1e602c8212..93ef1cf1b7de 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -103,7 +103,7 @@ FlinkSink.forRowData(input) env.execute("Test Iceberg DataStream"); ``` -The iceberg API also allows users to write generic `DataStream` to iceberg table, more example could be found in this [unit test](https://github.com/apache/iceberg/blob/main/flink/v1.16/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java). +The iceberg API also allows users to write generic `DataStream` to iceberg table, more example could be found in this [unit test](https://github.com/apache/iceberg/blob/main/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java). ### Overwrite data diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index ce4de4bdc1b8..e4123cc579b3 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -89,9 +89,10 @@ Users should continuously upgrade their Flink version to stay up-to-date. | 1.13 | End of Life | 0.13.0 | 1.0.0 | [iceberg-flink-runtime-1.13](https://repo1.maven.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.13/1.2.0/iceberg-flink-runtime-1.13-1.0.0.jar) | | 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 | Deprecated | 1.1.0 | {{ icebergVersion }} | [iceberg-flink-runtime-1.16](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) | -| 1.17 | Maintained | 1.3.0 | {{ icebergVersion }} | [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.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 | {{ icebergVersion }} | [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.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) | From ba32988a95bb38e62128234dfa30aee55ec573f0 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 17 Apr 2024 20:43:23 +0900 Subject: [PATCH 0231/1019] Flink: Backport Flink 1.18 JUnit5 migration to Flink 1.17 (#10163) --- .../flink/AvroGenericRecordConverterBase.java | 2 +- .../flink/TestDataFileSerialization.java | 14 ++- .../flink/TestFlinkCatalogFactory.java | 20 +++-- .../iceberg/flink/TestFlinkFilters.java | 87 +++++++++---------- .../iceberg/flink/TestFlinkSchemaUtil.java | 40 ++++----- .../flink/TestManifestFileSerialization.java | 17 ++-- .../iceberg/flink/TestRowDataWrapper.java | 18 ++-- .../iceberg/flink/TestTableSerialization.java | 18 ++-- .../TestAvroGenericRecordToRowDataMapper.java | 5 +- ...stRowDataToAvroGenericRecordConverter.java | 5 +- .../iceberg/flink/util/TestFlinkPackage.java | 11 +-- 11 files changed, 116 insertions(+), 121 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java index 47319ec9bc60..4184526a6a1a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink; -import org.junit.Test; +import org.junit.jupiter.api.Test; public abstract class AvroGenericRecordConverterBase { protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java index e9372adda4c1..8992cbd75187 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java @@ -20,6 +20,7 @@ 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.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -44,8 +45,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDataFileSerialization { @@ -135,23 +135,19 @@ public void testJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); } for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a position DeleteFile") - .isInstanceOf(DeleteFile.class); + assertThat(obj).as("Should be a position DeleteFile").isInstanceOf(DeleteFile.class); TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); } for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("Should be a equality DeleteFile") - .isInstanceOf(DeleteFile.class); + assertThat(obj).as("Should be a equality DeleteFile").isInstanceOf(DeleteFile.class); TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java index ba08b76dd59d..4c9e95b8fa82 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogProperties; @@ -26,15 +29,14 @@ import org.apache.iceberg.hive.HiveCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestFlinkCatalogFactory { private Map props; - @Before + @BeforeEach public void before() { props = Maps.newHashMap(); props.put("type", "iceberg"); @@ -51,7 +53,7 @@ public void testCreateCatalogHive() { FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) .loadCatalog(); - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); + assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); } @Test @@ -64,7 +66,7 @@ public void testCreateCatalogHadoop() { FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) .loadCatalog(); - Assertions.assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); + assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); } @Test @@ -76,7 +78,7 @@ public void testCreateCatalogCustom() { FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) .loadCatalog(); - Assertions.assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); + assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); } @Test @@ -86,7 +88,7 @@ public void testCreateCatalogCustomWithHiveCatalogTypeSet() { props.put( FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith( @@ -98,7 +100,7 @@ public void testLoadCatalogUnknown() { String catalogName = "unknownCatalog"; props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) .isInstanceOf(UnsupportedOperationException.class) .hasMessageStartingWith("Unknown catalog-type: fooType"); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java index c89ea4f53054..838b0ea0e1a9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.time.Instant; @@ -49,9 +51,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkFilters { @@ -121,13 +121,13 @@ public void testEquals() { Optional actual = FlinkFilters.convert( resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert( resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } } @@ -138,12 +138,12 @@ public void testEqualsNaN() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -156,13 +156,13 @@ public void testNotEquals() { Optional actual = FlinkFilters.convert( resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert( resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } } @@ -174,13 +174,13 @@ public void testNotEqualsNaN() { Optional actual = FlinkFilters.convert( resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert( resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -191,12 +191,12 @@ public void testGreaterThan() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -207,12 +207,12 @@ public void testGreaterThanEquals() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -223,12 +223,12 @@ public void testLessThan() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -239,12 +239,12 @@ public void testLessThanEquals() { Optional actual = FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); Optional actual1 = FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); - Assert.assertTrue("Conversion should succeed", actual1.isPresent()); + assertThat(actual1).isPresent(); assertPredicatesMatch(expected, actual1.get()); } @@ -252,7 +252,7 @@ public void testLessThanEquals() { public void testIsNull() { Expression expr = resolve(Expressions.$("field1").isNull()); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); assertPredicatesMatch(expected, actual.get()); } @@ -261,7 +261,7 @@ public void testIsNull() { public void testIsNotNull() { Expression expr = resolve(Expressions.$("field1").isNotNull()); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNull("field1"); assertPredicatesMatch(expected, actual.get()); @@ -275,7 +275,7 @@ public void testAnd() { .isEqual(Expressions.lit(1)) .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); And and = (And) actual.get(); And expected = (And) @@ -295,7 +295,7 @@ public void testOr() { .isEqual(Expressions.lit(1)) .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); Or or = (Or) actual.get(); Or expected = (Or) @@ -315,14 +315,14 @@ public void testNot() { BuiltInFunctionDefinitions.NOT, Expressions.$("field1").isEqual(Expressions.lit(1)))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); Not not = (Not) actual.get(); Not expected = (Not) org.apache.iceberg.expressions.Expressions.not( org.apache.iceberg.expressions.Expressions.equal("field1", 1)); - Assert.assertEquals("Predicate operation should match", expected.op(), not.op()); + assertThat(not.op()).as("Predicate operation should match").isEqualTo(expected.op()); assertPredicatesMatch(expected.child(), not.child()); } @@ -335,7 +335,7 @@ public void testLike() { ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); assertPredicatesMatch(expected, actual.get()); expr = @@ -343,7 +343,7 @@ public void testLike() { ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( @@ -352,7 +352,7 @@ public void testLike() { Expressions.$("field5"), Expressions.lit("%abc%"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( @@ -361,49 +361,49 @@ public void testLike() { Expressions.$("field5"), Expressions.lit("abc%d"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); expr = resolve( ApiExpressionUtils.unresolvedCall( BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); actual = FlinkFilters.convert(expr); - Assert.assertFalse("Conversion should failed", actual.isPresent()); + assertThat(actual).isNotPresent(); } @SuppressWarnings("unchecked") private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); Optional actual = FlinkFilters.convert(expr); - Assert.assertTrue("Conversion should succeed", actual.isPresent()); + assertThat(actual).isPresent(); org.apache.iceberg.expressions.Expression expression = actual.get(); - Assertions.assertThat(expression) + assertThat(expression) .as("The expression should be a UnboundPredicate") .isInstanceOf(UnboundPredicate.class); UnboundPredicate unboundPredicate = (UnboundPredicate) expression; org.apache.iceberg.expressions.Expression expression1 = unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); - Assertions.assertThat(expression1) + assertThat(expression1) .as("The expression should be a BoundLiteralPredicate") .isInstanceOf(BoundLiteralPredicate.class); BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; - Assert.assertTrue("Should match the literal", predicate.test(icebergLiteral)); + assertThat(predicate.test(icebergLiteral)).isTrue(); } private static Expression resolve(Expression originalExpression) { @@ -447,21 +447,16 @@ protected Expression defaultMethod(Expression expression) { private void assertPredicatesMatch( org.apache.iceberg.expressions.Expression expected, org.apache.iceberg.expressions.Expression actual) { - Assertions.assertThat(expected) + assertThat(expected) .as("The expected expression should be a UnboundPredicate") .isInstanceOf(UnboundPredicate.class); - Assertions.assertThat(actual) + assertThat(actual) .as("The actual expression should be a UnboundPredicate") .isInstanceOf(UnboundPredicate.class); UnboundPredicate predicateExpected = (UnboundPredicate) expected; UnboundPredicate predicateActual = (UnboundPredicate) actual; - Assert.assertEquals( - "Predicate operation should match", predicateExpected.op(), predicateActual.op()); - Assert.assertEquals( - "Predicate literal should match", predicateExpected.literal(), predicateActual.literal()); - Assert.assertEquals( - "Predicate name should match", - predicateExpected.ref().name(), - predicateActual.ref().name()); + assertThat(predicateActual.op()).isEqualTo(predicateExpected.op()); + assertThat(predicateActual.literal()).isEqualTo(predicateExpected.literal()); + assertThat(predicateActual.ref().name()).isEqualTo(predicateExpected.ref().name()); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java index 4ac32c08ebef..eab60d886ada 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.api.ValidationException; @@ -31,14 +34,11 @@ import org.apache.flink.table.types.logical.VarBinaryType; import org.apache.flink.table.types.logical.VarCharType; import org.apache.iceberg.Schema; -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.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkSchemaUtil { @@ -313,12 +313,13 @@ public void testListField() { } private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { - Assert.assertEquals(icebergSchema.asStruct(), FlinkSchemaUtil.convert(flinkSchema).asStruct()); + assertThat(FlinkSchemaUtil.convert(flinkSchema).asStruct()).isEqualTo(icebergSchema.asStruct()); // The conversion is not a 1:1 mapping, so we just check iceberg types. - Assert.assertEquals( - icebergSchema.asStruct(), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) - .asStruct()); + assertThat( + FlinkSchemaUtil.convert( + FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) + .asStruct()) + .isEqualTo(icebergSchema.asStruct()); } @Test @@ -354,10 +355,9 @@ private void checkInconsistentType( LogicalType flinkExpectedType, LogicalType flinkType, Type icebergExpectedType) { - Assert.assertEquals(flinkExpectedType, FlinkSchemaUtil.convert(icebergType)); - Assert.assertEquals( - Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType)), - FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()); + assertThat(FlinkSchemaUtil.convert(icebergType)).isEqualTo(flinkExpectedType); + assertThat(FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()) + .isEqualTo(Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType))); } @Test @@ -376,8 +376,8 @@ public void testConvertFlinkSchemaBaseOnIcebergSchema() { .primaryKey("int") .build(); Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); - Assert.assertEquals(baseSchema.asStruct(), convertedSchema.asStruct()); - Assert.assertEquals(ImmutableSet.of(101), convertedSchema.identifierFieldIds()); + assertThat(convertedSchema.asStruct()).isEqualTo(baseSchema.asStruct()); + assertThat(convertedSchema.identifierFieldIds()).containsExactly(101); } @Test @@ -390,10 +390,10 @@ public void testConvertFlinkSchemaWithPrimaryKeys() { Sets.newHashSet(1, 2)); TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); - Assert.assertTrue(tableSchema.getPrimaryKey().isPresent()); - Assert.assertEquals( - ImmutableSet.of("int", "string"), - ImmutableSet.copyOf(tableSchema.getPrimaryKey().get().getColumns())); + assertThat(tableSchema.getPrimaryKey()) + .isPresent() + .get() + .satisfies(k -> assertThat(k.getColumns()).containsExactly("int", "string")); } @Test @@ -408,7 +408,7 @@ public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { Types.NestedField.required(2, "inner", Types.IntegerType.get())))), Sets.newHashSet(2)); - Assertions.assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) + assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Could not create a PRIMARY KEY") .hasMessageContaining("Column 'struct.inner' does not exist."); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java index 6bd94e9ca61c..8f1f129e183b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -20,6 +20,7 @@ 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.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -29,6 +30,7 @@ import java.io.ObjectOutputStream; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.Path; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; import org.apache.flink.core.memory.DataInputDeserializer; @@ -48,11 +50,8 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestManifestFileSerialization { @@ -104,7 +103,7 @@ public class TestManifestFileSerialization { private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; @Test public void testKryoSerialization() throws IOException { @@ -145,15 +144,15 @@ public void testJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 3; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); TestHelpers.assertEquals(manifest, (ManifestFile) obj); } } } private ManifestFile writeManifest(DataFile... files) throws IOException { - File manifestFile = temp.newFile("input.m0.avro"); - Assert.assertTrue(manifestFile.delete()); + File manifestFile = File.createTempFile("input", "m0.avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index c78fa51215dd..caefbb5a5429 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Iterator; import org.apache.flink.table.data.RowData; import org.apache.iceberg.RecordWrapperTest; @@ -28,8 +30,6 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.util.StructLikeWrapper; -import org.assertj.core.api.Assertions; -import org.junit.Assert; public class TestRowDataWrapper extends RecordWrapperTest { @@ -49,12 +49,12 @@ public void testTime() { return; } - Assertions.assertThat(actual).isNotNull(); - Assertions.assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); int expectedMilliseconds = (int) ((long) expected / 1000_000); int actualMilliseconds = (int) ((long) actual / 1000_000); - Assert.assertEquals(message, expectedMilliseconds, actualMilliseconds); + assertThat(actualMilliseconds).as(message).isEqualTo(expectedMilliseconds); } }); } @@ -75,8 +75,8 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); for (int i = 0; i < numRecords; i++) { - Assert.assertTrue("Should have more records", actual.hasNext()); - Assert.assertTrue("Should have more RowData", expected.hasNext()); + assertThat(actual).hasNext(); + assertThat(expected).hasNext(); StructLike recordStructLike = recordWrapper.wrap(actual.next()); StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); @@ -87,7 +87,7 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod expectedWrapper.set(rowDataStructLike)); } - Assert.assertFalse("Shouldn't have more record", actual.hasNext()); - Assert.assertFalse("Shouldn't have more RowData", expected.hasNext()); + assertThat(actual).isExhausted(); + assertThat(expected).isExhausted(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java index 27124d93fef4..7f0e7acaa822 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java @@ -21,9 +21,11 @@ import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; 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.Map; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; @@ -39,11 +41,9 @@ import org.apache.iceberg.hadoop.HadoopTables; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestTableSerialization { private static final HadoopTables TABLES = new HadoopTables(); @@ -60,15 +60,15 @@ public class TestTableSerialization { private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private Table table; - @Before + @BeforeEach public void initTable() throws IOException { Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - File tableLocation = temp.newFolder(); - Assert.assertTrue(tableLocation.delete()); + File tableLocation = File.createTempFile("junit", null, temp.toFile()); + assertThat(tableLocation.delete()).isTrue(); this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java index 6a493692c20d..44eb907a17aa 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.flink.table.data.RowData; import org.apache.iceberg.flink.AvroGenericRecordConverterBase; import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { @Override @@ -32,6 +33,6 @@ protected void testConverter(DataGenerator dataGenerator) throws Exception { AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); RowData expected = dataGenerator.generateFlinkRowData(); RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java index 485035787d6d..6ef40693827e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.avro.generic.GenericRecord; import org.apache.iceberg.flink.AvroGenericRecordConverterBase; import org.apache.iceberg.flink.DataGenerator; -import org.junit.Assert; public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { @Override @@ -30,6 +31,6 @@ protected void testConverter(DataGenerator dataGenerator) { RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); GenericRecord expected = dataGenerator.generateAvroGenericRecord(); GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); - Assert.assertEquals(expected, actual); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index eda340dedaf5..079c70bae070 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.flink.util; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; import org.mockito.MockedStatic; import org.mockito.Mockito; @@ -28,7 +29,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - Assert.assertEquals("1.17.2", FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo("1.17.2"); } @Test @@ -41,14 +42,14 @@ public void testDefaultVersion() { try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class); mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); - Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); } FlinkPackage.setVersion(null); try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null); mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); FlinkPackage.setVersion(null); - Assert.assertEquals(FlinkPackage.FLINK_UNKNOWN_VERSION, FlinkPackage.version()); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); } } } From f5135c1f6e14d439d3d5c6e89c8729f8835d7f92 Mon Sep 17 00:00:00 2001 From: c-thiel Date: Wed, 17 Apr 2024 18:32:10 +0200 Subject: [PATCH 0232/1019] OpenAPI: Renaming views should return 204 (#10166) --- open-api/rest-catalog-open-api.yaml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 9c2300dcf6a4..4883af09d581 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -1386,8 +1386,8 @@ paths: $ref: '#/components/examples/RenameViewSameNamespace' required: true responses: - 200: - description: OK + 204: + description: Success, no content 400: $ref: '#/components/responses/BadRequestErrorResponse' 401: From f342d4793af297bc62dcddb8ad2a576274111068 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Wed, 17 Apr 2024 20:01:24 +0200 Subject: [PATCH 0233/1019] Core: Fix namespace SQL statement using ESCAPE character that works with MySQL/PostgreSQL (#10167) Co-authored-by: Chauncy --- core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 8f918b456088..ffa606a94f29 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -335,7 +335,7 @@ enum SchemaVersion { + TABLE_NAMESPACE + " = ? OR " + TABLE_NAMESPACE - + " LIKE ? ESCAPE '\\')" + + " LIKE ? ESCAPE '!')" + " LIMIT 1"; static final String LIST_NAMESPACES_SQL = "SELECT DISTINCT " @@ -426,7 +426,7 @@ enum SchemaVersion { + NAMESPACE_NAME + " = ? OR " + NAMESPACE_NAME - + " LIKE ? ESCAPE '\\' " + + " LIKE ? ESCAPE '!' " + " ) "; static final String INSERT_NAMESPACE_PROPERTIES_SQL = "INSERT INTO " @@ -783,7 +783,7 @@ static boolean namespaceExists( // when namespace has sub-namespace then additionally checking it with LIKE statement. // catalog.db can exists as: catalog.db.ns1 or catalog.db.ns1.ns2 String namespaceStartsWith = - namespaceEquals.replace("\\", "\\\\").replace("_", "\\_").replace("%", "\\%") + ".%"; + namespaceEquals.replace("!", "!!").replace("_", "!_").replace("%", "!%") + ".%"; if (exists(connections, GET_NAMESPACE_SQL, catalogName, namespaceEquals, namespaceStartsWith)) { return true; } From 08f309ac4a2b7f5d4406597f826dbe5abadbbff8 Mon Sep 17 00:00:00 2001 From: Ahmet DAL Date: Thu, 18 Apr 2024 11:23:57 +0200 Subject: [PATCH 0234/1019] Flink: Don't fail to serialize IcebergSourceSplit when there is too many delete files (#9464) --- .../source/split/IcebergSourceSplit.java | 47 +++- .../split/IcebergSourceSplitSerializer.java | 6 +- .../flink/source/split/SerializerHelper.java | 206 ++++++++++++++++++ .../iceberg/flink/source/SplitHelpers.java | 71 ++++++ .../TestIcebergSourceSplitSerializer.java | 20 ++ 5 files changed, 346 insertions(+), 4 deletions(-) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java 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 e4bfbf1452e2..44e37afcfc60 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 @@ -132,6 +132,14 @@ static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { } byte[] serializeV2() throws IOException { + return serialize(2); + } + + byte[] serializeV3() throws IOException { + return serialize(3); + } + + private byte[] serialize(int version) throws IOException { if (serializedBytesCache == null) { DataOutputSerializer out = SERIALIZER_CACHE.get(); Collection fileScanTasks = task.tasks(); @@ -147,7 +155,7 @@ byte[] serializeV2() throws IOException { for (FileScanTask fileScanTask : fileScanTasks) { String taskJson = FileScanTaskParser.toJson(fileScanTask); - out.writeUTF(taskJson); + writeTaskJson(out, taskJson, version); } serializedBytesCache = out.getCopyOfBuffer(); @@ -157,8 +165,32 @@ byte[] serializeV2() throws IOException { return serializedBytesCache; } + private static void writeTaskJson(DataOutputSerializer out, String taskJson, int version) + throws IOException { + switch (version) { + case 2: + out.writeUTF(taskJson); + break; + case 3: + SerializerHelper.writeLongUTF(out, taskJson); + break; + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) throws IOException { + return deserialize(serialized, caseSensitive, 2); + } + + static IcebergSourceSplit deserializeV3(byte[] serialized, boolean caseSensitive) + throws IOException { + return deserialize(serialized, caseSensitive, 3); + } + + private static IcebergSourceSplit deserialize( + byte[] serialized, boolean caseSensitive, int version) throws IOException { DataInputDeserializer in = new DataInputDeserializer(serialized); int fileOffset = in.readInt(); long recordOffset = in.readLong(); @@ -166,7 +198,7 @@ static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive List tasks = Lists.newArrayListWithCapacity(taskCount); for (int i = 0; i < taskCount; ++i) { - String taskJson = in.readUTF(); + String taskJson = readTaskJson(in, version); FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); tasks.add(task); } @@ -174,4 +206,15 @@ static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); } + + private static String readTaskJson(DataInputDeserializer in, int version) throws IOException { + switch (version) { + case 2: + return in.readUTF(); + case 3: + return SerializerHelper.readLongUTF(in); + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index 8c089819e731..d4b0f9e1977d 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -24,7 +24,7 @@ @Internal public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 2; + private static final int VERSION = 3; private final boolean caseSensitive; @@ -39,7 +39,7 @@ public int getVersion() { @Override public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV2(); + return split.serializeV3(); } @Override @@ -49,6 +49,8 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE return IcebergSourceSplit.deserializeV1(serialized); case 2: return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); + case 3: + return IcebergSourceSplit.deserializeV3(serialized, caseSensitive); default: throw new IOException( String.format( diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java new file mode 100644 index 000000000000..a0395f29ac5b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.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.flink.source.split; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UTFDataFormatException; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +/** + * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly + * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and + * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. + */ +class SerializerHelper implements Serializable { + + private SerializerHelper() {} + + /** + * Similar to {@link DataOutputSerializer#writeUTF(String)}. Except this supports larger payloads + * which is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param out the output stream to write the string to. + * @param str the string value to be written. + */ + public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + int strlen = str.length(); + long utflen = 0; + int ch; + + /* use charAt instead of copying String to char array */ + for (int i = 0; i < strlen; i++) { + ch = str.charAt(i); + utflen += getUTFBytesSize(ch); + + if (utflen > Integer.MAX_VALUE) { + throw new UTFDataFormatException("Encoded string reached maximum length: " + utflen); + } + } + + if (utflen > Integer.MAX_VALUE - 4) { + throw new UTFDataFormatException("Encoded string is too long: " + utflen); + } + + out.writeInt((int) utflen); + writeUTFBytes(out, str, (int) utflen); + } + + /** + * Similar to {@link DataInputDeserializer#readUTF()}. Except this supports larger payloads which + * is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param in the input stream to read the string from. + * @return the string value read from the input stream. + * @throws IOException if an I/O error occurs when reading from the input stream. + */ + public static String readLongUTF(DataInputDeserializer in) throws IOException { + int utflen = in.readInt(); + byte[] bytearr = new byte[utflen]; + char[] chararr = new char[utflen]; + + int ch; + int char2; + int char3; + int count = 0; + int chararrCount = 0; + + in.readFully(bytearr, 0, utflen); + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + if (ch > 127) { + break; + } + count++; + chararr[chararrCount++] = (char) ch; + } + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + switch (ch >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + /* 0xxxxxxx */ + count++; + chararr[chararrCount++] = (char) ch; + break; + case 12: + case 13: + /* 110x xxxx 10xx xxxx */ + count += 2; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = (int) bytearr[count - 1]; + if ((char2 & 0xC0) != 0x80) { + throw new UTFDataFormatException("malformed input around byte " + count); + } + chararr[chararrCount++] = (char) (((ch & 0x1F) << 6) | (char2 & 0x3F)); + break; + case 14: + /* 1110 xxxx 10xx xxxx 10xx xxxx */ + count += 3; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = (int) bytearr[count - 2]; + char3 = (int) bytearr[count - 1]; + if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { + throw new UTFDataFormatException("malformed input around byte " + (count - 1)); + } + chararr[chararrCount++] = + (char) (((ch & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F)); + break; + default: + /* 10xx xxxx, 1111 xxxx */ + throw new UTFDataFormatException("malformed input around byte " + count); + } + } + // The number of chars produced may be less than utflen + return new String(chararr, 0, chararrCount); + } + + private static int getUTFBytesSize(int ch) { + if ((ch >= 0x0001) && (ch <= 0x007F)) { + return 1; + } else if (ch > 0x07FF) { + return 3; + } else { + return 2; + } + } + + private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) + throws IOException { + int strlen = str.length(); + int ch; + + int len = Math.max(1024, utflen); + + byte[] bytearr = new byte[len]; + int count = 0; + + int index; + for (index = 0; index < strlen; index++) { + ch = str.charAt(index); + if (!((ch >= 0x0001) && (ch <= 0x007F))) { + break; + } + bytearr[count++] = (byte) ch; + } + + for (; index < strlen; index++) { + ch = str.charAt(index); + if ((ch >= 0x0001) && (ch <= 0x007F)) { + bytearr[count++] = (byte) ch; + } else if (ch > 0x07FF) { + bytearr[count++] = (byte) (0xE0 | ((ch >> 12) & 0x0F)); + bytearr[count++] = (byte) (0x80 | ((ch >> 6) & 0x3F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } else { + bytearr[count++] = (byte) (0xC0 | ((ch >> 6) & 0x1F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } + } + + out.write(bytearr, 0, count); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index 3a8071523b7c..ebd220b00dba 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -18,19 +18,30 @@ */ package org.apache.iceberg.flink.source; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + import java.io.File; +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.hadoop.HadoopCatalog; @@ -129,4 +140,64 @@ public static List createSplitsFromTransientHadoopTable( catalog.close(); } } + + /** + * This method will equip the {@code icebergSourceSplits} with mock delete files. + *

  • For each split, create {@code deleteFilesPerSplit} number of delete files + *
  • Replace the original {@code FileScanTask} with the new {@code FileScanTask} with mock + *
  • Caller should not attempt to read the deleted files since they are created as mock, and + * they are not real files + * + * @param icebergSourceSplits The real splits to equip with mock delete files + * @param temporaryFolder The temporary folder to create the mock delete files with + * @param deleteFilesPerSplit The number of delete files to create for each split + * @return The list of re-created splits with mock delete files + * @throws IOException If there is any error creating the mock delete files + */ + public static List equipSplitsWithMockDeleteFiles( + List icebergSourceSplits, + TemporaryFolder temporaryFolder, + int deleteFilesPerSplit) + throws IOException { + List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); + for (IcebergSourceSplit split : icebergSourceSplits) { + final CombinedScanTask combinedScanTask = spy(split.task()); + + final List deleteFiles = Lists.newArrayList(); + final PartitionSpec spec = + PartitionSpec.builderFor(TestFixtures.SCHEMA).withSpecId(0).build(); + + for (int i = 0; i < deleteFilesPerSplit; ++i) { + final DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(spec) + .withFormat(FileFormat.PARQUET) + .withPath(temporaryFolder.newFile().getPath()) + .ofPositionDeletes() + .withFileSizeInBytes(1000) + .withRecordCount(1000) + .build(); + deleteFiles.add(deleteFile); + } + + List newFileScanTasks = Lists.newArrayList(); + for (FileScanTask task : combinedScanTask.tasks()) { + String schemaString = SchemaParser.toJson(task.schema()); + String specString = PartitionSpecParser.toJson(task.spec()); + + BaseFileScanTask baseFileScanTask = + new BaseFileScanTask( + task.file(), + deleteFiles.toArray(new DeleteFile[] {}), + schemaString, + specString, + ResidualEvaluator.unpartitioned(task.residual())); + newFileScanTasks.add(baseFileScanTask); + } + doReturn(newFileScanTasks).when(combinedScanTask).tasks(); + icebergSourceSplitsWithMockDeleteFiles.add( + IcebergSourceSplit.fromCombinedScanTask( + combinedScanTask, split.fileOffset(), split.recordOffset())); + } + return icebergSourceSplitsWithMockDeleteFiles; + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index cd778309f90d..c72d622f86ba 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -100,6 +100,26 @@ private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws } } + @Test + public void testV3WithTooManyDeleteFiles() throws Exception { + serializeAndDeserializeV3(1, 1, 5000); + } + + private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mockDeletesPerSplit) + throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + final List splitsWithMockDeleteFiles = + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, TEMPORARY_FOLDER, mockDeletesPerSplit); + + for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { + byte[] result = split.serializeV3(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV3(result, true); + assertSplitEquals(split, deserialized); + } + } + @Test public void testDeserializeV1() throws Exception { final List splits = From 7cb189e683e9c19f0d0baeb907bff8fc3bd94315 Mon Sep 17 00:00:00 2001 From: Elkhan Dadash Date: Thu, 18 Apr 2024 12:46:28 -0700 Subject: [PATCH 0235/1019] Flink: port #9464 to v1.17 and v1.19 (#10177) Co-authored-by: Elkhan Dadashov --- .../source/split/IcebergSourceSplit.java | 47 +++- .../split/IcebergSourceSplitSerializer.java | 6 +- .../flink/source/split/SerializerHelper.java | 206 ++++++++++++++++++ .../iceberg/flink/source/SplitHelpers.java | 71 ++++++ .../TestIcebergSourceSplitSerializer.java | 20 ++ .../source/split/IcebergSourceSplit.java | 47 +++- .../split/IcebergSourceSplitSerializer.java | 6 +- .../flink/source/split/SerializerHelper.java | 206 ++++++++++++++++++ .../iceberg/flink/source/SplitHelpers.java | 71 ++++++ .../TestIcebergSourceSplitSerializer.java | 20 ++ 10 files changed, 692 insertions(+), 8 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index e4bfbf1452e2..44e37afcfc60 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -132,6 +132,14 @@ static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { } byte[] serializeV2() throws IOException { + return serialize(2); + } + + byte[] serializeV3() throws IOException { + return serialize(3); + } + + private byte[] serialize(int version) throws IOException { if (serializedBytesCache == null) { DataOutputSerializer out = SERIALIZER_CACHE.get(); Collection fileScanTasks = task.tasks(); @@ -147,7 +155,7 @@ byte[] serializeV2() throws IOException { for (FileScanTask fileScanTask : fileScanTasks) { String taskJson = FileScanTaskParser.toJson(fileScanTask); - out.writeUTF(taskJson); + writeTaskJson(out, taskJson, version); } serializedBytesCache = out.getCopyOfBuffer(); @@ -157,8 +165,32 @@ byte[] serializeV2() throws IOException { return serializedBytesCache; } + private static void writeTaskJson(DataOutputSerializer out, String taskJson, int version) + throws IOException { + switch (version) { + case 2: + out.writeUTF(taskJson); + break; + case 3: + SerializerHelper.writeLongUTF(out, taskJson); + break; + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) throws IOException { + return deserialize(serialized, caseSensitive, 2); + } + + static IcebergSourceSplit deserializeV3(byte[] serialized, boolean caseSensitive) + throws IOException { + return deserialize(serialized, caseSensitive, 3); + } + + private static IcebergSourceSplit deserialize( + byte[] serialized, boolean caseSensitive, int version) throws IOException { DataInputDeserializer in = new DataInputDeserializer(serialized); int fileOffset = in.readInt(); long recordOffset = in.readLong(); @@ -166,7 +198,7 @@ static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive List tasks = Lists.newArrayListWithCapacity(taskCount); for (int i = 0; i < taskCount; ++i) { - String taskJson = in.readUTF(); + String taskJson = readTaskJson(in, version); FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); tasks.add(task); } @@ -174,4 +206,15 @@ static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); } + + private static String readTaskJson(DataInputDeserializer in, int version) throws IOException { + switch (version) { + case 2: + return in.readUTF(); + case 3: + return SerializerHelper.readLongUTF(in); + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index 8c089819e731..d4b0f9e1977d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -24,7 +24,7 @@ @Internal public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 2; + private static final int VERSION = 3; private final boolean caseSensitive; @@ -39,7 +39,7 @@ public int getVersion() { @Override public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV2(); + return split.serializeV3(); } @Override @@ -49,6 +49,8 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE return IcebergSourceSplit.deserializeV1(serialized); case 2: return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); + case 3: + return IcebergSourceSplit.deserializeV3(serialized, caseSensitive); default: throw new IOException( String.format( diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java new file mode 100644 index 000000000000..a0395f29ac5b --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.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.flink.source.split; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UTFDataFormatException; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +/** + * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly + * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and + * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. + */ +class SerializerHelper implements Serializable { + + private SerializerHelper() {} + + /** + * Similar to {@link DataOutputSerializer#writeUTF(String)}. Except this supports larger payloads + * which is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param out the output stream to write the string to. + * @param str the string value to be written. + */ + public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + int strlen = str.length(); + long utflen = 0; + int ch; + + /* use charAt instead of copying String to char array */ + for (int i = 0; i < strlen; i++) { + ch = str.charAt(i); + utflen += getUTFBytesSize(ch); + + if (utflen > Integer.MAX_VALUE) { + throw new UTFDataFormatException("Encoded string reached maximum length: " + utflen); + } + } + + if (utflen > Integer.MAX_VALUE - 4) { + throw new UTFDataFormatException("Encoded string is too long: " + utflen); + } + + out.writeInt((int) utflen); + writeUTFBytes(out, str, (int) utflen); + } + + /** + * Similar to {@link DataInputDeserializer#readUTF()}. Except this supports larger payloads which + * is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param in the input stream to read the string from. + * @return the string value read from the input stream. + * @throws IOException if an I/O error occurs when reading from the input stream. + */ + public static String readLongUTF(DataInputDeserializer in) throws IOException { + int utflen = in.readInt(); + byte[] bytearr = new byte[utflen]; + char[] chararr = new char[utflen]; + + int ch; + int char2; + int char3; + int count = 0; + int chararrCount = 0; + + in.readFully(bytearr, 0, utflen); + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + if (ch > 127) { + break; + } + count++; + chararr[chararrCount++] = (char) ch; + } + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + switch (ch >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + /* 0xxxxxxx */ + count++; + chararr[chararrCount++] = (char) ch; + break; + case 12: + case 13: + /* 110x xxxx 10xx xxxx */ + count += 2; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = (int) bytearr[count - 1]; + if ((char2 & 0xC0) != 0x80) { + throw new UTFDataFormatException("malformed input around byte " + count); + } + chararr[chararrCount++] = (char) (((ch & 0x1F) << 6) | (char2 & 0x3F)); + break; + case 14: + /* 1110 xxxx 10xx xxxx 10xx xxxx */ + count += 3; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = (int) bytearr[count - 2]; + char3 = (int) bytearr[count - 1]; + if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { + throw new UTFDataFormatException("malformed input around byte " + (count - 1)); + } + chararr[chararrCount++] = + (char) (((ch & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F)); + break; + default: + /* 10xx xxxx, 1111 xxxx */ + throw new UTFDataFormatException("malformed input around byte " + count); + } + } + // The number of chars produced may be less than utflen + return new String(chararr, 0, chararrCount); + } + + private static int getUTFBytesSize(int ch) { + if ((ch >= 0x0001) && (ch <= 0x007F)) { + return 1; + } else if (ch > 0x07FF) { + return 3; + } else { + return 2; + } + } + + private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) + throws IOException { + int strlen = str.length(); + int ch; + + int len = Math.max(1024, utflen); + + byte[] bytearr = new byte[len]; + int count = 0; + + int index; + for (index = 0; index < strlen; index++) { + ch = str.charAt(index); + if (!((ch >= 0x0001) && (ch <= 0x007F))) { + break; + } + bytearr[count++] = (byte) ch; + } + + for (; index < strlen; index++) { + ch = str.charAt(index); + if ((ch >= 0x0001) && (ch <= 0x007F)) { + bytearr[count++] = (byte) ch; + } else if (ch > 0x07FF) { + bytearr[count++] = (byte) (0xE0 | ((ch >> 12) & 0x0F)); + bytearr[count++] = (byte) (0x80 | ((ch >> 6) & 0x3F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } else { + bytearr[count++] = (byte) (0xC0 | ((ch >> 6) & 0x1F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } + } + + out.write(bytearr, 0, count); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index 3a8071523b7c..ebd220b00dba 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -18,19 +18,30 @@ */ package org.apache.iceberg.flink.source; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + import java.io.File; +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.hadoop.HadoopCatalog; @@ -129,4 +140,64 @@ public static List createSplitsFromTransientHadoopTable( catalog.close(); } } + + /** + * This method will equip the {@code icebergSourceSplits} with mock delete files. + *

  • For each split, create {@code deleteFilesPerSplit} number of delete files + *
  • Replace the original {@code FileScanTask} with the new {@code FileScanTask} with mock + *
  • Caller should not attempt to read the deleted files since they are created as mock, and + * they are not real files + * + * @param icebergSourceSplits The real splits to equip with mock delete files + * @param temporaryFolder The temporary folder to create the mock delete files with + * @param deleteFilesPerSplit The number of delete files to create for each split + * @return The list of re-created splits with mock delete files + * @throws IOException If there is any error creating the mock delete files + */ + public static List equipSplitsWithMockDeleteFiles( + List icebergSourceSplits, + TemporaryFolder temporaryFolder, + int deleteFilesPerSplit) + throws IOException { + List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); + for (IcebergSourceSplit split : icebergSourceSplits) { + final CombinedScanTask combinedScanTask = spy(split.task()); + + final List deleteFiles = Lists.newArrayList(); + final PartitionSpec spec = + PartitionSpec.builderFor(TestFixtures.SCHEMA).withSpecId(0).build(); + + for (int i = 0; i < deleteFilesPerSplit; ++i) { + final DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(spec) + .withFormat(FileFormat.PARQUET) + .withPath(temporaryFolder.newFile().getPath()) + .ofPositionDeletes() + .withFileSizeInBytes(1000) + .withRecordCount(1000) + .build(); + deleteFiles.add(deleteFile); + } + + List newFileScanTasks = Lists.newArrayList(); + for (FileScanTask task : combinedScanTask.tasks()) { + String schemaString = SchemaParser.toJson(task.schema()); + String specString = PartitionSpecParser.toJson(task.spec()); + + BaseFileScanTask baseFileScanTask = + new BaseFileScanTask( + task.file(), + deleteFiles.toArray(new DeleteFile[] {}), + schemaString, + specString, + ResidualEvaluator.unpartitioned(task.residual())); + newFileScanTasks.add(baseFileScanTask); + } + doReturn(newFileScanTasks).when(combinedScanTask).tasks(); + icebergSourceSplitsWithMockDeleteFiles.add( + IcebergSourceSplit.fromCombinedScanTask( + combinedScanTask, split.fileOffset(), split.recordOffset())); + } + return icebergSourceSplitsWithMockDeleteFiles; + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index cd778309f90d..c72d622f86ba 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -100,6 +100,26 @@ private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws } } + @Test + public void testV3WithTooManyDeleteFiles() throws Exception { + serializeAndDeserializeV3(1, 1, 5000); + } + + private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mockDeletesPerSplit) + throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + final List splitsWithMockDeleteFiles = + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, TEMPORARY_FOLDER, mockDeletesPerSplit); + + for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { + byte[] result = split.serializeV3(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV3(result, true); + assertSplitEquals(split, deserialized); + } + } + @Test public void testDeserializeV1() throws Exception { final List splits = 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 e4bfbf1452e2..44e37afcfc60 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 @@ -132,6 +132,14 @@ static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { } byte[] serializeV2() throws IOException { + return serialize(2); + } + + byte[] serializeV3() throws IOException { + return serialize(3); + } + + private byte[] serialize(int version) throws IOException { if (serializedBytesCache == null) { DataOutputSerializer out = SERIALIZER_CACHE.get(); Collection fileScanTasks = task.tasks(); @@ -147,7 +155,7 @@ byte[] serializeV2() throws IOException { for (FileScanTask fileScanTask : fileScanTasks) { String taskJson = FileScanTaskParser.toJson(fileScanTask); - out.writeUTF(taskJson); + writeTaskJson(out, taskJson, version); } serializedBytesCache = out.getCopyOfBuffer(); @@ -157,8 +165,32 @@ byte[] serializeV2() throws IOException { return serializedBytesCache; } + private static void writeTaskJson(DataOutputSerializer out, String taskJson, int version) + throws IOException { + switch (version) { + case 2: + out.writeUTF(taskJson); + break; + case 3: + SerializerHelper.writeLongUTF(out, taskJson); + break; + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) throws IOException { + return deserialize(serialized, caseSensitive, 2); + } + + static IcebergSourceSplit deserializeV3(byte[] serialized, boolean caseSensitive) + throws IOException { + return deserialize(serialized, caseSensitive, 3); + } + + private static IcebergSourceSplit deserialize( + byte[] serialized, boolean caseSensitive, int version) throws IOException { DataInputDeserializer in = new DataInputDeserializer(serialized); int fileOffset = in.readInt(); long recordOffset = in.readLong(); @@ -166,7 +198,7 @@ static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive List tasks = Lists.newArrayListWithCapacity(taskCount); for (int i = 0; i < taskCount; ++i) { - String taskJson = in.readUTF(); + String taskJson = readTaskJson(in, version); FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); tasks.add(task); } @@ -174,4 +206,15 @@ static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); } + + private static String readTaskJson(DataInputDeserializer in, int version) throws IOException { + switch (version) { + case 2: + return in.readUTF(); + case 3: + return SerializerHelper.readLongUTF(in); + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index 8c089819e731..d4b0f9e1977d 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -24,7 +24,7 @@ @Internal public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 2; + private static final int VERSION = 3; private final boolean caseSensitive; @@ -39,7 +39,7 @@ public int getVersion() { @Override public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV2(); + return split.serializeV3(); } @Override @@ -49,6 +49,8 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE return IcebergSourceSplit.deserializeV1(serialized); case 2: return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); + case 3: + return IcebergSourceSplit.deserializeV3(serialized, caseSensitive); default: throw new IOException( String.format( diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java new file mode 100644 index 000000000000..a0395f29ac5b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.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.flink.source.split; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UTFDataFormatException; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +/** + * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly + * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and + * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. + */ +class SerializerHelper implements Serializable { + + private SerializerHelper() {} + + /** + * Similar to {@link DataOutputSerializer#writeUTF(String)}. Except this supports larger payloads + * which is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param out the output stream to write the string to. + * @param str the string value to be written. + */ + public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + int strlen = str.length(); + long utflen = 0; + int ch; + + /* use charAt instead of copying String to char array */ + for (int i = 0; i < strlen; i++) { + ch = str.charAt(i); + utflen += getUTFBytesSize(ch); + + if (utflen > Integer.MAX_VALUE) { + throw new UTFDataFormatException("Encoded string reached maximum length: " + utflen); + } + } + + if (utflen > Integer.MAX_VALUE - 4) { + throw new UTFDataFormatException("Encoded string is too long: " + utflen); + } + + out.writeInt((int) utflen); + writeUTFBytes(out, str, (int) utflen); + } + + /** + * Similar to {@link DataInputDeserializer#readUTF()}. Except this supports larger payloads which + * is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param in the input stream to read the string from. + * @return the string value read from the input stream. + * @throws IOException if an I/O error occurs when reading from the input stream. + */ + public static String readLongUTF(DataInputDeserializer in) throws IOException { + int utflen = in.readInt(); + byte[] bytearr = new byte[utflen]; + char[] chararr = new char[utflen]; + + int ch; + int char2; + int char3; + int count = 0; + int chararrCount = 0; + + in.readFully(bytearr, 0, utflen); + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + if (ch > 127) { + break; + } + count++; + chararr[chararrCount++] = (char) ch; + } + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + switch (ch >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + /* 0xxxxxxx */ + count++; + chararr[chararrCount++] = (char) ch; + break; + case 12: + case 13: + /* 110x xxxx 10xx xxxx */ + count += 2; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = (int) bytearr[count - 1]; + if ((char2 & 0xC0) != 0x80) { + throw new UTFDataFormatException("malformed input around byte " + count); + } + chararr[chararrCount++] = (char) (((ch & 0x1F) << 6) | (char2 & 0x3F)); + break; + case 14: + /* 1110 xxxx 10xx xxxx 10xx xxxx */ + count += 3; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = (int) bytearr[count - 2]; + char3 = (int) bytearr[count - 1]; + if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { + throw new UTFDataFormatException("malformed input around byte " + (count - 1)); + } + chararr[chararrCount++] = + (char) (((ch & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F)); + break; + default: + /* 10xx xxxx, 1111 xxxx */ + throw new UTFDataFormatException("malformed input around byte " + count); + } + } + // The number of chars produced may be less than utflen + return new String(chararr, 0, chararrCount); + } + + private static int getUTFBytesSize(int ch) { + if ((ch >= 0x0001) && (ch <= 0x007F)) { + return 1; + } else if (ch > 0x07FF) { + return 3; + } else { + return 2; + } + } + + private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) + throws IOException { + int strlen = str.length(); + int ch; + + int len = Math.max(1024, utflen); + + byte[] bytearr = new byte[len]; + int count = 0; + + int index; + for (index = 0; index < strlen; index++) { + ch = str.charAt(index); + if (!((ch >= 0x0001) && (ch <= 0x007F))) { + break; + } + bytearr[count++] = (byte) ch; + } + + for (; index < strlen; index++) { + ch = str.charAt(index); + if ((ch >= 0x0001) && (ch <= 0x007F)) { + bytearr[count++] = (byte) ch; + } else if (ch > 0x07FF) { + bytearr[count++] = (byte) (0xE0 | ((ch >> 12) & 0x0F)); + bytearr[count++] = (byte) (0x80 | ((ch >> 6) & 0x3F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } else { + bytearr[count++] = (byte) (0xC0 | ((ch >> 6) & 0x1F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } + } + + out.write(bytearr, 0, count); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index 3a8071523b7c..ebd220b00dba 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -18,19 +18,30 @@ */ package org.apache.iceberg.flink.source; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + import java.io.File; +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.SchemaParser; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.hadoop.HadoopCatalog; @@ -129,4 +140,64 @@ public static List createSplitsFromTransientHadoopTable( catalog.close(); } } + + /** + * This method will equip the {@code icebergSourceSplits} with mock delete files. + *

  • For each split, create {@code deleteFilesPerSplit} number of delete files + *
  • Replace the original {@code FileScanTask} with the new {@code FileScanTask} with mock + *
  • Caller should not attempt to read the deleted files since they are created as mock, and + * they are not real files + * + * @param icebergSourceSplits The real splits to equip with mock delete files + * @param temporaryFolder The temporary folder to create the mock delete files with + * @param deleteFilesPerSplit The number of delete files to create for each split + * @return The list of re-created splits with mock delete files + * @throws IOException If there is any error creating the mock delete files + */ + public static List equipSplitsWithMockDeleteFiles( + List icebergSourceSplits, + TemporaryFolder temporaryFolder, + int deleteFilesPerSplit) + throws IOException { + List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); + for (IcebergSourceSplit split : icebergSourceSplits) { + final CombinedScanTask combinedScanTask = spy(split.task()); + + final List deleteFiles = Lists.newArrayList(); + final PartitionSpec spec = + PartitionSpec.builderFor(TestFixtures.SCHEMA).withSpecId(0).build(); + + for (int i = 0; i < deleteFilesPerSplit; ++i) { + final DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(spec) + .withFormat(FileFormat.PARQUET) + .withPath(temporaryFolder.newFile().getPath()) + .ofPositionDeletes() + .withFileSizeInBytes(1000) + .withRecordCount(1000) + .build(); + deleteFiles.add(deleteFile); + } + + List newFileScanTasks = Lists.newArrayList(); + for (FileScanTask task : combinedScanTask.tasks()) { + String schemaString = SchemaParser.toJson(task.schema()); + String specString = PartitionSpecParser.toJson(task.spec()); + + BaseFileScanTask baseFileScanTask = + new BaseFileScanTask( + task.file(), + deleteFiles.toArray(new DeleteFile[] {}), + schemaString, + specString, + ResidualEvaluator.unpartitioned(task.residual())); + newFileScanTasks.add(baseFileScanTask); + } + doReturn(newFileScanTasks).when(combinedScanTask).tasks(); + icebergSourceSplitsWithMockDeleteFiles.add( + IcebergSourceSplit.fromCombinedScanTask( + combinedScanTask, split.fileOffset(), split.recordOffset())); + } + return icebergSourceSplitsWithMockDeleteFiles; + } } 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 cd778309f90d..c72d622f86ba 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 @@ -100,6 +100,26 @@ private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws } } + @Test + public void testV3WithTooManyDeleteFiles() throws Exception { + serializeAndDeserializeV3(1, 1, 5000); + } + + private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mockDeletesPerSplit) + throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + TEMPORARY_FOLDER, splitCount, filesPerSplit); + final List splitsWithMockDeleteFiles = + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, TEMPORARY_FOLDER, mockDeletesPerSplit); + + for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { + byte[] result = split.serializeV3(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV3(result, true); + assertSplitEquals(split, deserialized); + } + } + @Test public void testDeserializeV1() throws Exception { final List splits = From dc75f0ff491b131c08d40070c09e16d6e3b08d20 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 19 Apr 2024 15:20:44 +0200 Subject: [PATCH 0236/1019] Core: Improve size check in CatalogTests (#10182) --- .../java/org/apache/iceberg/catalog/CatalogTests.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) 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 6904449cc4c3..ff01d80679f1 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -2746,9 +2746,9 @@ public void assertUUIDsMatch(Table expected, Table actual) { public void assertPreviousMetadataFileCount(Table table, int metadataFileCount) { TableOperations ops = ((BaseTable) table).operations(); - Assertions.assertThat(ops.current().previousFiles().size()) + Assertions.assertThat(ops.current().previousFiles()) .as("Table should have correct number of previous metadata locations") - .isEqualTo(metadataFileCount); + .hasSize(metadataFileCount); } public void assertNoFiles(Table table) { @@ -2766,9 +2766,9 @@ public void assertFiles(Table table, DataFile... files) { .map(FileScanTask::file) .map(DataFile::path) .collect(Collectors.toList()); - Assertions.assertThat(paths.size()) + Assertions.assertThat(paths) .as("Should contain expected number of data files") - .isEqualTo(files.length); + .hasSize(files.length); Assertions.assertThat(CharSequenceSet.of(paths)) .as("Should contain correct file paths") .isEqualTo(CharSequenceSet.of(Iterables.transform(Arrays.asList(files), DataFile::path))); From 8dce583aa06c2b64306162c3c298c8bd8e3dba48 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Fri, 19 Apr 2024 22:13:46 +0530 Subject: [PATCH 0237/1019] Kafka-connect: Update iceberg.hadoop-conf-dir config description (#10184) --- .../org/apache/iceberg/connect/IcebergSinkConfig.java | 8 ++++---- 1 file changed, 4 insertions(+), 4 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 d1572fbff37b..e64e183089cf 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 @@ -87,7 +87,7 @@ public class IcebergSinkConfig extends AbstractConfig { private static final int COMMIT_TIMEOUT_MS_DEFAULT = 30_000; private static final String COMMIT_THREADS_PROP = "iceberg.control.commit.threads"; private static final String CONNECT_GROUP_ID_PROP = "iceberg.connect.group-id"; - private static final String HADDOP_CONF_DIR_PROP = "iceberg.hadoop-conf-dir"; + private static final String HADOOP_CONF_DIR_PROP = "iceberg.hadoop-conf-dir"; private static final String NAME_PROP = "name"; private static final String BOOTSTRAP_SERVERS_PROP = "bootstrap.servers"; @@ -216,11 +216,11 @@ private static ConfigDef newConfigDef() { Importance.MEDIUM, "Coordinator threads to use for table commits, default is (cores * 2)"); configDef.define( - HADDOP_CONF_DIR_PROP, + HADOOP_CONF_DIR_PROP, ConfigDef.Type.STRING, null, Importance.MEDIUM, - "Coordinator threads to use for table commits, default is (cores * 2)"); + "If specified, Hadoop config files in this directory will be loaded"); return configDef; } @@ -404,7 +404,7 @@ public int commitThreads() { } public String hadoopConfDir() { - return getString(HADDOP_CONF_DIR_PROP); + return getString(HADOOP_CONF_DIR_PROP); } public boolean autoCreateEnabled() { From 4b74efc7383b62e86799647c70bfb3862fe4f30e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Apr 2024 17:09:48 +0200 Subject: [PATCH 0238/1019] Build: Bump software.amazon.awssdk:bom from 2.25.31 to 2.25.35 (#10192) Bumps software.amazon.awssdk:bom from 2.25.31 to 2.25.35. --- 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 3f2f2591e36e..8cb3e9329d86 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.31" +awssdk-bom = "2.25.35" azuresdk-bom = "1.2.22" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 203d8f1505dc949d21db08f24f817c616633683b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Apr 2024 17:10:08 +0200 Subject: [PATCH 0239/1019] Build: Bump io.netty:netty-buffer from 4.1.108.Final to 4.1.109.Final (#10191) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.108.Final to 4.1.109.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.108.Final...netty-4.1.109.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 8cb3e9329d86..aaf8f03fd9e6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,8 +66,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.79.0" -netty-buffer = "4.1.108.Final" -netty-buffer-compat = "4.1.108.Final" +netty-buffer = "4.1.109.Final" +netty-buffer-compat = "4.1.109.Final" object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" From 9c62c5caa53bb3151073e708d3671e6d9ca80575 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Apr 2024 17:10:26 +0200 Subject: [PATCH 0240/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.0.5 to 1.0.6 (#10190) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.0.5 to 1.0.6. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.0.5...1.0.6) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 aaf8f03fd9e6..b561f66e0c34 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -72,7 +72,7 @@ object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.0.5" +roaringbitmap = "1.0.6" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" From 07fc6ecb3ad3d51e17ab17b9c04b6413967bd6ac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Apr 2024 17:10:37 +0200 Subject: [PATCH 0241/1019] Build: Bump mkdocs-material from 9.5.17 to 9.5.18 (#10189) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.17 to 9.5.18. - [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.17...9.5.18) --- 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 5132cdf017cc..f238604eb00a 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.17 +mkdocs-material==9.5.18 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 8a52df981e39f9c57f2be6edf0df53b578b4bf37 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Apr 2024 17:10:52 +0200 Subject: [PATCH 0242/1019] Build: Bump gradle.plugin.io.morethan.jmhreport:gradle-jmh-report (#10193) Bumps gradle.plugin.io.morethan.jmhreport:gradle-jmh-report from 0.9.0 to 0.9.6. --- updated-dependencies: - dependency-name: gradle.plugin.io.morethan.jmhreport:gradle-jmh-report 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 138f9d4f8bb8..b30e4550cc60 100644 --- a/build.gradle +++ b/build.gradle @@ -36,7 +36,7 @@ buildscript { classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' - classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.0' + classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.5.0" classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.1' From e486ba096b8549afe5f122babe05f6d2ebc5e47c Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 22 Apr 2024 09:42:18 +0200 Subject: [PATCH 0243/1019] Core: Lazily compute & cache hashCode in CharSequenceWrapper (#10023) --- .../org/apache/iceberg/types/JavaHashes.java | 4 + .../iceberg/util/CharSequenceWrapper.java | 25 ++++- .../iceberg/util/TestCharSequenceMap.java | 7 ++ .../iceberg/util/TestCharSequenceSet.java | 6 ++ .../iceberg/util/TestCharSequenceWrapper.java | 93 +++++++++++++++++++ 5 files changed, 134 insertions(+), 1 deletion(-) create mode 100644 api/src/test/java/org/apache/iceberg/util/TestCharSequenceWrapper.java diff --git a/api/src/main/java/org/apache/iceberg/types/JavaHashes.java b/api/src/main/java/org/apache/iceberg/types/JavaHashes.java index c25198990013..9a14f7639f07 100644 --- a/api/src/main/java/org/apache/iceberg/types/JavaHashes.java +++ b/api/src/main/java/org/apache/iceberg/types/JavaHashes.java @@ -26,6 +26,10 @@ public class JavaHashes { private JavaHashes() {} public static int hashCode(CharSequence str) { + if (null == str) { + return 0; + } + int result = 177; for (int i = 0; i < str.length(); i += 1) { char ch = str.charAt(i); diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java index fcccb9eac090..854264c1ae21 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceWrapper.java @@ -29,6 +29,11 @@ public static CharSequenceWrapper wrap(CharSequence seq) { } private CharSequence wrapped; + // lazily computed & cached hashCode + private transient int hashCode = 0; + // tracks if the hash has been calculated as actually being zero to avoid re-calculating the hash. + // this follows the hashCode() implementation from java.lang.String + private transient boolean hashIsZero = false; private CharSequenceWrapper(CharSequence wrapped) { this.wrapped = wrapped; @@ -36,6 +41,8 @@ private CharSequenceWrapper(CharSequence wrapped) { public CharSequenceWrapper set(CharSequence newWrapped) { this.wrapped = newWrapped; + this.hashCode = 0; + this.hashIsZero = false; return this; } @@ -58,6 +65,10 @@ public boolean equals(Object other) { return wrapped.equals(that.wrapped); } + if (null == wrapped && null == that.wrapped) { + return true; + } + if (length() != that.length()) { return false; } @@ -67,7 +78,19 @@ public boolean equals(Object other) { @Override public int hashCode() { - return JavaHashes.hashCode(wrapped); + int hash = hashCode; + + // don't recalculate if the hash is actually 0 + if (hash == 0 && !hashIsZero) { + hash = JavaHashes.hashCode(wrapped); + if (hash == 0) { + hashIsZero = true; + } else { + this.hashCode = hash; + } + } + + return hash; } @Override diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java index 47d686d3abbe..8ca7889b4717 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java @@ -25,10 +25,17 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCharSequenceMap { + @Test + public void nullString() { + Assertions.assertThat(CharSequenceMap.create()).doesNotContainKey((String) null); + Assertions.assertThat(CharSequenceMap.create()).doesNotContainValue((String) null); + } + @Test public void testEmptyMap() { CharSequenceMap map = CharSequenceMap.create(); diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java index 9420548ca9aa..b0f242c177f6 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java @@ -39,6 +39,12 @@ public void testSearchingInCharSequenceCollection() { Assertions.assertThat(set.contains("def")).isTrue(); } + @Test + public void nullString() { + Assertions.assertThat(CharSequenceSet.of(Arrays.asList((String) null))).contains((String) null); + Assertions.assertThat(CharSequenceSet.empty()).doesNotContain((String) null); + } + @Test public void testRetainAll() { CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("123", "456")); diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceWrapper.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceWrapper.java new file mode 100644 index 000000000000..cdf46ee8d7fa --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceWrapper.java @@ -0,0 +1,93 @@ +/* + * 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.junit.jupiter.api.Test; + +public class TestCharSequenceWrapper { + + @Test + public void nullWrapper() { + CharSequenceWrapper one = CharSequenceWrapper.wrap(null); + CharSequenceWrapper two = CharSequenceWrapper.wrap(null); + + // at this point hashCode is not computed yet + assertThat(one).isEqualTo(two); + + // hashCode is lazily computed and stored + assertThat(one.hashCode()).isEqualTo(two.hashCode()).isEqualTo(0); + + assertThat(one).isEqualTo(two); + } + + @Test + public void equalsWithLazyHashCode() { + CharSequenceWrapper string = CharSequenceWrapper.wrap("v1"); + CharSequenceWrapper buffer = CharSequenceWrapper.wrap(new StringBuffer("v1")); + CharSequenceWrapper builder = CharSequenceWrapper.wrap(new StringBuilder("v1")); + + // at this point hashCode is 0 for all + assertThat(string).isEqualTo(buffer).isEqualTo(builder); + + // hashCode is lazily computed and stored + assertThat(string.hashCode()).isEqualTo(buffer.hashCode()).isEqualTo(builder.hashCode()); + + assertThat(string).isEqualTo(buffer).isEqualTo(builder); + } + + @Test + public void notEqualsWithLazyHashCode() { + CharSequenceWrapper v1 = CharSequenceWrapper.wrap("v1"); + CharSequenceWrapper v2 = CharSequenceWrapper.wrap("v2"); + + // at this point hashCode is 0 for all + assertThat(v1).isNotEqualTo(v2); + + // hashCode is lazily computed and stored + assertThat(v1.hashCode()).isNotEqualTo(v2.hashCode()); + + assertThat(v1).isNotEqualTo(v2); + } + + @Test + public void hashCodeIsRecomputed() { + CharSequenceWrapper wrapper = CharSequenceWrapper.wrap("v1"); + assertThat(wrapper.hashCode()).isEqualTo(173804); + + wrapper.set("v2"); + assertThat(wrapper.hashCode()).isEqualTo(173805); + + wrapper.set(new StringBuffer("v2")); + assertThat(wrapper.hashCode()).isEqualTo(173805); + + wrapper.set(new StringBuilder("v2")); + assertThat(wrapper.hashCode()).isEqualTo(173805); + + wrapper.set("v3"); + assertThat(wrapper.hashCode()).isEqualTo(173806); + + wrapper.set(null); + assertThat(wrapper.hashCode()).isEqualTo(0); + + wrapper.set("v2"); + assertThat(wrapper.hashCode()).isEqualTo(173805); + } +} From d2c3ff5b0f550388229764bcf2ea5e55a26419f7 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 22 Apr 2024 16:09:16 +0200 Subject: [PATCH 0244/1019] AWS: Make sure Signer + User Agent config are both applied (#10198) --- .../iceberg/aws/s3/S3FileIOProperties.java | 20 +++++++++++--- .../iceberg/aws/TestS3FileIOProperties.java | 27 +++++++++++++++++++ .../aws/s3/TestS3FileIOProperties.java | 18 +++++++++---- 3 files changed, 56 insertions(+), 9 deletions(-) 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 857f35e710ab..b3801d3f3621 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 @@ -36,6 +36,7 @@ import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SerializableMap; import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; import software.amazon.awssdk.services.s3.S3ClientBuilder; import software.amazon.awssdk.services.s3.S3Configuration; @@ -788,10 +789,15 @@ public void applyServiceConfigurations(T builder) { */ public void applySignerConfiguration(T builder) { if (isRemoteSigningEnabled) { + ClientOverrideConfiguration.Builder configBuilder = + null != builder.overrideConfiguration() + ? builder.overrideConfiguration().toBuilder() + : ClientOverrideConfiguration.builder(); builder.overrideConfiguration( - c -> - c.putAdvancedOption( - SdkAdvancedClientOption.SIGNER, S3V4RestSignerClient.create(allProperties))); + configBuilder + .putAdvancedOption( + SdkAdvancedClientOption.SIGNER, S3V4RestSignerClient.create(allProperties)) + .build()); } } @@ -829,8 +835,14 @@ public void applyS3AccessGrantsConfigurations(T buil } public void applyUserAgentConfigurations(T builder) { + ClientOverrideConfiguration.Builder configBuilder = + null != builder.overrideConfiguration() + ? builder.overrideConfiguration().toBuilder() + : ClientOverrideConfiguration.builder(); builder.overrideConfiguration( - c -> c.putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, S3_FILE_IO_USER_AGENT)); + configBuilder + .putAdvancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX, S3_FILE_IO_USER_AGENT) + .build()); } /** 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 83234dc09e6a..b7a3f6048991 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java @@ -236,6 +236,33 @@ public void testS3RemoteSigningEnabled() { Assertions.assertThat(signerClient.properties()).isEqualTo(properties); } + @Test + public void s3RemoteSigningEnabledWithUserAgent() { + String uri = "http://localhost:12345"; + Map properties = + ImmutableMap.of( + S3FileIOProperties.REMOTE_SIGNING_ENABLED, "true", CatalogProperties.URI, uri); + S3FileIOProperties s3Properties = new S3FileIOProperties(properties); + S3ClientBuilder builder = S3Client.builder(); + + s3Properties.applySignerConfiguration(builder); + s3Properties.applyUserAgentConfigurations(builder); + + Optional userAgent = + builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX); + Assertions.assertThat(userAgent) + .isPresent() + .get() + .satisfies(x -> Assertions.assertThat(x).startsWith("s3fileio")); + + Optional signer = + builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.SIGNER); + Assertions.assertThat(signer).isPresent().get().isInstanceOf(S3V4RestSignerClient.class); + S3V4RestSignerClient signerClient = (S3V4RestSignerClient) signer.get(); + Assertions.assertThat(signerClient.baseSignerUri()).isEqualTo(uri); + Assertions.assertThat(signerClient.properties()).isEqualTo(properties); + } + @Test public void testS3RemoteSigningDisabled() { Map properties = diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index 658b5b781969..c6d3776b9b0e 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -22,16 +22,18 @@ import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.function.Consumer; import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.aws.AwsClientProperties; 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.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; +import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.services.s3.S3ClientBuilder; import software.amazon.awssdk.services.s3.S3Configuration; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; @@ -459,13 +461,18 @@ public void testApplyS3ServiceConfigurations() { @Test public void testApplySignerConfiguration() { - Map properties = Maps.newHashMap(); - properties.put(S3FileIOProperties.REMOTE_SIGNING_ENABLED, "true"); + Map properties = + ImmutableMap.of( + S3FileIOProperties.REMOTE_SIGNING_ENABLED, + "true", + CatalogProperties.URI, + "http://localhost:12345"); S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(properties); S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); s3FileIOProperties.applySignerConfiguration(mockS3ClientBuilder); - Mockito.verify(mockS3ClientBuilder).overrideConfiguration(Mockito.any(Consumer.class)); + Mockito.verify(mockS3ClientBuilder) + .overrideConfiguration(Mockito.any(ClientOverrideConfiguration.class)); } @Test @@ -486,6 +493,7 @@ public void testApplyUserAgentConfigurations() { S3ClientBuilder mockS3ClientBuilder = Mockito.mock(S3ClientBuilder.class); s3FileIOProperties.applyUserAgentConfigurations(mockS3ClientBuilder); - Mockito.verify(mockS3ClientBuilder).overrideConfiguration(Mockito.any(Consumer.class)); + Mockito.verify(mockS3ClientBuilder) + .overrideConfiguration(Mockito.any(ClientOverrideConfiguration.class)); } } From 5bde8e78e6ab2a1c1341a81942f114b8fc629d4e Mon Sep 17 00:00:00 2001 From: Hanzhi Wang Date: Tue, 23 Apr 2024 03:21:20 -0700 Subject: [PATCH 0245/1019] Hive: turn off the stats gathering when iceberg.hive.keep.stats is false (#10148) --- .../iceberg/hive/HiveTableOperations.java | 1 + .../TestHiveIcebergStorageHandlerNoScan.java | 4 +- ...stHiveIcebergWithHiveAutogatherEnable.java | 185 ++++++++++++++++++ 3 files changed, 188 insertions(+), 2 deletions(-) create mode 100644 mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWithHiveAutogatherEnable.java 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 75d59de75d4d..5293f915407e 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 @@ -235,6 +235,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { if (!keepHiveStats) { tbl.getParameters().remove(StatsSetupConst.COLUMN_STATS_ACCURATE); + tbl.getParameters().put(StatsSetupConst.DO_NOT_UPDATE_STATS, StatsSetupConst.TRUE); } lock.ensureActive(); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 534cc7d7476c..328b9f3b5b95 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -776,7 +776,7 @@ public void testIcebergAndHmsTableProperties() throws Exception { if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { assertThat(hmsParams) - .hasSize(14) + .hasSize(15) .containsEntry("custom_property", "initial_val") .containsEntry(InputFormatConfig.EXTERNAL_TABLE_PURGE, "TRUE") .containsEntry("EXTERNAL", "TRUE") @@ -819,7 +819,7 @@ public void testIcebergAndHmsTableProperties() throws Exception { if (Catalogs.hiveCatalog(shell.getHiveConf(), tableProperties)) { assertThat(hmsParams) - .hasSize(17) + .hasSize(18) .containsEntry("new_prop_1", "true") .containsEntry("new_prop_2", "false") .containsEntry("custom_property", "new_val"); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWithHiveAutogatherEnable.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWithHiveAutogatherEnable.java new file mode 100644 index 000000000000..6b3bddd637c2 --- /dev/null +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergWithHiveAutogatherEnable.java @@ -0,0 +1,185 @@ +/* + * 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.mr.hive; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collection; +import org.apache.hadoop.hive.common.StatsSetupConst; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.ConfigProperties; +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.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; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestHiveIcebergWithHiveAutogatherEnable { + + @Parameters(name = "fileFormat={0}, catalog={1}") + public static Collection parameters() { + Collection testParams = Lists.newArrayList(); + // Run tests with every FileFormat for a single Catalog (HiveCatalog) + for (FileFormat fileFormat : HiveIcebergStorageHandlerTestUtils.FILE_FORMATS) { + testParams.add(new Object[] {fileFormat, TestTables.TestTableType.HIVE_CATALOG}); + } + return testParams; + } + + private static TestHiveShell shell; + + private TestTables testTables; + + @Parameter(index = 0) + private FileFormat fileFormat; + + @Parameter(index = 1) + private TestTables.TestTableType testTableType; + + @TempDir private Path temp; + + @BeforeAll + public static void beforeClass() { + // The hive configuration HIVESTATSAUTOGATHER must be set to true from hive engine + shell = + HiveIcebergStorageHandlerTestUtils.shell( + ImmutableMap.of(HiveConf.ConfVars.HIVESTATSAUTOGATHER.varname, "true")); + } + + @AfterAll + public static void afterClass() throws Exception { + shell.stop(); + } + + @BeforeEach + public void before() throws IOException { + testTables = HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); + HiveIcebergStorageHandlerTestUtils.init(shell, testTables, temp, "mr"); + } + + @AfterEach + public void after() throws Exception { + HiveIcebergStorageHandlerTestUtils.close(shell); + } + + @TestTemplate + public void testHiveStatsAutogatherWhenCreateNewTable() throws Exception { + // Create a Catalog where the KEEP_HIVE_STATS is false + shell.metastore().hiveConf().set(ConfigProperties.KEEP_HIVE_STATS, StatsSetupConst.FALSE); + TestTables hiveStatsDisabledTestTables = + HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); + + TableIdentifier identifierWithoutStats = + TableIdentifier.of("default", "customers_without_stats"); + + // To validate the stats augother is disabled from Hive engine, the creation of iceberg table + // cannot have any records. Otherwise, the table parameters TOTAL_SIZE and NUM_FILES are + // added by Iceberg when inserting records. + hiveStatsDisabledTestTables.createTable( + shell, + identifierWithoutStats.name(), + HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, + ImmutableList.of()); + + // The table parameter TOTAL_SIZE is removed from hive engine + String totalSize = + shell + .metastore() + .getTable(identifierWithoutStats) + .getParameters() + .get(StatsSetupConst.TOTAL_SIZE); + assertThat(totalSize).isNull(); + + // The table parameter NUM_FILES is removed from hive engine + String numFiles = + shell + .metastore() + .getTable(identifierWithoutStats) + .getParameters() + .get(StatsSetupConst.NUM_FILES); + assertThat(numFiles).isNull(); + + // The table parameter DO_NOT_UPDATE_STATS is removed from hive engine + String stats = + shell + .metastore() + .getTable(identifierWithoutStats) + .getParameters() + .get(StatsSetupConst.DO_NOT_UPDATE_STATS); + assertThat(stats).isNull(); + + // Create a Catalog where the KEEP_HIVE_STATS is true + shell.metastore().hiveConf().set(ConfigProperties.KEEP_HIVE_STATS, StatsSetupConst.TRUE); + TestTables keepHiveStatsTestTables = + HiveIcebergStorageHandlerTestUtils.testTables(shell, testTableType, temp); + + TableIdentifier identifierWithStats = TableIdentifier.of("default", "customers_with_stats"); + + // To validate the stats augother is enabled from Hive engine, the creation of iceberg table + // cannot have any records. Otherwise, the table parameters TOTAL_SIZE and NUM_FILES are + // added by Iceberg when inserting records. + keepHiveStatsTestTables.createTable( + shell, + identifierWithStats.name(), + HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA, + fileFormat, + ImmutableList.of()); + + // The table parameter DO_NOT_UPDATE_STATS doesn't exist + stats = + shell + .metastore() + .getTable(identifierWithStats) + .getParameters() + .get(StatsSetupConst.DO_NOT_UPDATE_STATS); + assertThat(stats).isNull(); + + // The table parameter NUM_FILES is gathered from hive engine + numFiles = + shell + .metastore() + .getTable(identifierWithStats) + .getParameters() + .get(StatsSetupConst.NUM_FILES); + assertThat(numFiles).isEqualTo("1"); + + // The table parameter TOTAL_SIZE is gathered from hive engine + numFiles = + shell + .metastore() + .getTable(identifierWithStats) + .getParameters() + .get(StatsSetupConst.TOTAL_SIZE); + assertThat(numFiles).isNotNull(); + } +} From ac57b0da26cb06a07c542984352e8849dae9dd21 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 24 Apr 2024 17:24:52 +0200 Subject: [PATCH 0246/1019] Docs: Don't check links on Release page (#10212) --- site/docs/releases.md | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 5a829b593bf3..d43032a107cb 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -39,7 +39,6 @@ The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apach * [{{ icebergVersion }} gcp-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-gcp-bundle/{{ icebergVersion }}/iceberg-gcp-bundle-{{ icebergVersion }}.jar) * [{{ icebergVersion }} azure-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-azure-bundle/{{ icebergVersion }}/iceberg-azure-bundle-{{ icebergVersion }}.jar) - To use Iceberg in Spark or Flink, download the runtime JAR for your engine version and add it to the jars folder of your installation. @@ -977,3 +976,5 @@ A more exhaustive list of changes is available under the [0.10.0 release milesto ### 0.7.0 * Git tag: [apache-iceberg-0.7.0-incubating](https://github.com/apache/iceberg/releases/tag/apache-iceberg-0.7.0-incubating) + + From d47b8efec8597fc3d17140aa5b4c32e9005ea624 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 24 Apr 2024 18:24:12 +0200 Subject: [PATCH 0247/1019] Docs: Use `svn mv` when releasing the binaries (#9926) --- site/docs/how-to-release.md | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index f65b9d6c3941..b3b5b241d556 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -34,7 +34,7 @@ This page describes the procedures that the release manager and voting PMC membe To create a release candidate, you will need: -* Apache LDAP credentals for Nexus and SVN +* 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) If you have not published your GPG key yet, you must publish it before sending the vote email by doing: @@ -256,14 +256,7 @@ are prepared when going through the below steps. First, copy the source release directory to releases: ```bash -mkdir iceberg -cd iceberg -svn co https://dist.apache.org/repos/dist/dev/iceberg candidates -svn co https://dist.apache.org/repos/dist/release/iceberg releases -cp -r candidates/apache-iceberg--rcN/ releases/apache-iceberg- -cd releases -svn add apache-iceberg- -svn ci -m 'Iceberg: Add release ' +svn mv https://dist.apache.org/repos/dist/dev/iceberg/apache-iceberg--rcN https://dist.apache.org/repos/dist/release/iceberg/apache-iceberg- -m "Iceberg: Add release " ``` !!! Note From 59a99261818645e0d30f1b6e30cd94cb674d497f Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 24 Apr 2024 13:11:21 -0600 Subject: [PATCH 0248/1019] Infra: Add 1.5.1 to issue template (#10214) --- .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 f2ea3f8aab4c..7c584618cb69 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.5.0 (latest release)" + - "1.5.1 (latest release)" + - "1.5.0" - "1.4.3" - "1.4.2" - "1.4.1" From 0bdae7ff8f7df721a114ba0e428b0ba698d38be3 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 24 Apr 2024 13:37:03 -0600 Subject: [PATCH 0249/1019] Update site to 1.5.1 docs (#10218) --- site/mkdocs.yml | 2 +- site/nav.yml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 45df8d93bd39..db3e1aef9709 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.5.0' + icebergVersion: '1.5.1' nessieVersion: '0.77.1' social: - icon: fontawesome/regular/comments diff --git a/site/nav.yml b/site/nav.yml index 4a3548433636..2de2913d53cf 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.5.1: '!include docs/docs/1.5.1/mkdocs.yml' - 1.5.0: '!include docs/docs/1.5.0/mkdocs.yml' - 1.4.3: '!include docs/docs/1.4.3/mkdocs.yml' - 1.4.2: '!include docs/docs/1.4.2/mkdocs.yml' From 8d1db047ac9181f35ecc96919b425cfedc4011d9 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 25 Apr 2024 11:21:28 +0200 Subject: [PATCH 0250/1019] Core: Use 'delete' / 'append' if OverwriteFiles only deletes/appends data files (#10150) --- .../apache/iceberg/BaseOverwriteFiles.java | 8 ++++ .../org/apache/iceberg/TestOverwrite.java | 46 ++++++++++++++++++- .../SparkRowLevelOperationsTestBase.java | 3 +- .../iceberg/spark/extensions/TestDelete.java | 5 +- .../source/TestStructuredStreamingRead3.java | 14 +++++- .../SparkRowLevelOperationsTestBase.java | 3 +- .../iceberg/spark/extensions/TestDelete.java | 5 +- .../source/TestStructuredStreamingRead3.java | 14 +++++- .../SparkRowLevelOperationsTestBase.java | 3 +- .../iceberg/spark/extensions/TestDelete.java | 5 +- .../source/TestStructuredStreamingRead3.java | 14 +++++- 11 files changed, 109 insertions(+), 11 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java index a994eaf44d9a..d929bc068ec2 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -48,6 +48,14 @@ protected OverwriteFiles self() { @Override protected String operation() { + if (deletesDataFiles() && !addsDataFiles()) { + return DataOperations.DELETE; + } + + if (addsDataFiles() && !deletesDataFiles()) { + return DataOperations.APPEND; + } + return DataOperations.OVERWRITE; } diff --git a/core/src/test/java/org/apache/iceberg/TestOverwrite.java b/core/src/test/java/org/apache/iceberg/TestOverwrite.java index 15b5547cc456..d4c886cefc4e 100644 --- a/core/src/test/java/org/apache/iceberg/TestOverwrite.java +++ b/core/src/test/java/org/apache/iceberg/TestOverwrite.java @@ -101,8 +101,8 @@ public class TestOverwrite extends TestBase { ImmutableMap.of(1, 5L, 2, 3L), // value count ImmutableMap.of(1, 0L, 2, 2L), // null count null, - ImmutableMap.of(1, longToBuffer(5L)), // lower bounds - ImmutableMap.of(1, longToBuffer(9L)) // upper bounds + ImmutableMap.of(1, longToBuffer(10L)), // lower bounds + ImmutableMap.of(1, longToBuffer(14L)) // upper bounds )) .build(); @@ -135,6 +135,43 @@ public void createTestTable() throws IOException { commit(table, table.newAppend().appendFile(FILE_0_TO_4).appendFile(FILE_5_TO_9), branch); } + @TestTemplate + public void deleteDataFilesProducesDeleteOperation() { + commit(table, table.newOverwrite().deleteFile(FILE_A).deleteFile(FILE_B), branch); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.DELETE); + } + + @TestTemplate + public void addAndDeleteDataFilesProducesOverwriteOperation() { + commit(table, table.newOverwrite().addFile(FILE_10_TO_14).deleteFile(FILE_B), branch); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.OVERWRITE); + } + + @TestTemplate + public void overwriteByRowFilterProducesDeleteOperation() { + commit(table, table.newOverwrite().overwriteByRowFilter(equal("date", "2018-06-08")), branch); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.DELETE); + } + + @TestTemplate + public void addAndOverwriteByRowFilterProducesOverwriteOperation() { + commit( + table, + table + .newOverwrite() + .addFile(FILE_10_TO_14) + .overwriteByRowFilter(equal("date", "2018-06-08")), + branch); + + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.OVERWRITE); + } + + @TestTemplate + public void addFilesProducesAppendOperation() { + commit(table, table.newOverwrite().addFile(FILE_10_TO_14).addFile(FILE_5_TO_9), branch); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.APPEND); + } + @TestTemplate public void testOverwriteWithoutAppend() { TableMetadata base = TestTables.readMetadata(TABLE_NAME); @@ -145,6 +182,7 @@ public void testOverwriteWithoutAppend() { long overwriteId = latestSnapshot(table, branch).snapshotId(); assertThat(overwriteId).isNotEqualTo(baseId); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.DELETE); assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); validateManifestEntries( @@ -188,6 +226,7 @@ public void testOverwriteWithAppendOutsideOfDelete() { long overwriteId = latestSnapshot(table, branch).snapshotId(); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.OVERWRITE); assertThat(overwriteId).isNotEqualTo(baseId); assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(2); @@ -224,6 +263,7 @@ public void testOverwriteWithMergedAppendOutsideOfDelete() { long overwriteId = latestSnapshot(table, branch).snapshotId(); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.OVERWRITE); assertThat(overwriteId).isNotEqualTo(baseId); assertThat(latestSnapshot(table, branch).allManifests(table.io())).hasSize(1); @@ -255,6 +295,7 @@ public void testValidatedOverwriteWithAppendOutsideOfDelete() { .hasMessageStartingWith("Cannot append file with rows that do not match filter"); assertThat(latestSnapshot(table, branch).snapshotId()).isEqualTo(baseId); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.APPEND); } @TestTemplate @@ -275,6 +316,7 @@ public void testValidatedOverwriteWithAppendOutsideOfDeleteMetrics() { .hasMessageStartingWith("Cannot append file with rows that do not match filter"); assertThat(latestSnapshot(base, branch).snapshotId()).isEqualTo(baseId); + assertThat(latestSnapshot(table, branch).operation()).isEqualTo(DataOperations.APPEND); } @TestTemplate diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 7398e3830020..6aae08434638 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -247,8 +247,9 @@ protected void validateCopyOnWrite( String changedPartitionCount, String deletedDataFiles, String addedDataFiles) { + String operation = null == addedDataFiles && null != deletedDataFiles ? DELETE : OVERWRITE; validateSnapshot( - snapshot, OVERWRITE, changedPartitionCount, deletedDataFiles, null, addedDataFiles); + snapshot, operation, changedPartitionCount, deletedDataFiles, null, addedDataFiles); } protected void validateMergeOnRead( diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index cdc508f9859d..e5ea378e5aab 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static org.apache.iceberg.DataOperations.DELETE; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; @@ -424,8 +425,10 @@ public void testDeleteWithArbitraryPartitionPredicates() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); - // should be an overwrite since cannot be executed using a metadata operation + // should be a "delete" instead of an "overwrite" as only data files have been removed (COW) / + // delete files have been added (MOR) Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + assertThat(currentSnapshot.operation()).isEqualTo(DELETE); if (mode(table) == COPY_ON_WRITE) { validateCopyOnWrite(currentSnapshot, "1", "1", null); } else { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index c706603d06c7..de94a7c8bf8b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -571,8 +571,20 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(dataAcrossSnapshots); + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(temp.newFile().toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + // this should create a snapshot with type overwrite. - table.newOverwrite().overwriteByRowFilter(Expressions.greaterThan("id", 4)).commit(); + table + .newOverwrite() + .addFile(dataFile) + .overwriteByRowFilter(Expressions.greaterThan("id", 4)) + .commit(); // check pre-condition - that the above delete operation on table resulted in Snapshot of Type // OVERWRITE. diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 463cf2a47d8b..5a1cc6343424 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -283,8 +283,9 @@ protected void validateCopyOnWrite( String changedPartitionCount, String deletedDataFiles, String addedDataFiles) { + String operation = null == addedDataFiles && null != deletedDataFiles ? DELETE : OVERWRITE; validateSnapshot( - snapshot, OVERWRITE, changedPartitionCount, deletedDataFiles, null, addedDataFiles); + snapshot, operation, changedPartitionCount, deletedDataFiles, null, addedDataFiles); } protected void validateMergeOnRead( diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index 731dedbd48d2..e6114d4abcca 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static org.apache.iceberg.DataOperations.DELETE; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; @@ -592,8 +593,10 @@ public void testDeleteWithArbitraryPartitionPredicates() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertEquals("Should have 4 snapshots", 4, Iterables.size(table.snapshots())); - // should be an overwrite since cannot be executed using a metadata operation + // should be a "delete" instead of an "overwrite" as only data files have been removed (COW) / + // delete files have been added (MOR) Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + assertThat(currentSnapshot.operation()).isEqualTo(DELETE); if (mode(table) == COPY_ON_WRITE) { validateCopyOnWrite(currentSnapshot, "1", "1", null); } else { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index c1bbf304fac4..47e933360156 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -576,8 +576,20 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(dataAcrossSnapshots); + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(temp.newFile().toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + // this should create a snapshot with type overwrite. - table.newOverwrite().overwriteByRowFilter(Expressions.greaterThan("id", 4)).commit(); + table + .newOverwrite() + .addFile(dataFile) + .overwriteByRowFilter(Expressions.greaterThan("id", 4)) + .commit(); // check pre-condition - that the above delete operation on table resulted in Snapshot of Type // OVERWRITE. 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 a926388e4aa8..f00b94219078 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 @@ -276,8 +276,9 @@ protected void validateCopyOnWrite( String changedPartitionCount, String deletedDataFiles, String addedDataFiles) { + String operation = null == addedDataFiles && null != deletedDataFiles ? DELETE : OVERWRITE; validateSnapshot( - snapshot, OVERWRITE, changedPartitionCount, deletedDataFiles, null, addedDataFiles); + snapshot, operation, changedPartitionCount, deletedDataFiles, null, addedDataFiles); } protected void validateMergeOnRead( 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 05887d2a8b1b..b88df19ea9ba 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static org.apache.iceberg.DataOperations.DELETE; import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; import static org.apache.iceberg.SnapshotSummary.ADD_POS_DELETE_FILES_PROP; import static org.apache.iceberg.TableProperties.DELETE_DISTRIBUTION_MODE; @@ -590,8 +591,10 @@ public void testDeleteWithArbitraryPartitionPredicates() { Table table = validationCatalog.loadTable(tableIdent); assertThat(table.snapshots()).as("Should have 4 snapshots").hasSize(4); - // should be an overwrite since cannot be executed using a metadata operation + // should be a "delete" instead of an "overwrite" as only data files have been removed (COW) / + // delete files have been added (MOR) Snapshot currentSnapshot = SnapshotUtil.latestSnapshot(table, branch); + assertThat(currentSnapshot.operation()).isEqualTo(DELETE); if (mode(table) == COPY_ON_WRITE) { validateCopyOnWrite(currentSnapshot, "1", "1", null); } else { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index a5bcf53bd72c..5a8d4deec730 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -656,8 +656,20 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws List> dataAcrossSnapshots = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(dataAcrossSnapshots); + DataFile dataFile = + DataFiles.builder(table.spec()) + .withPath(File.createTempFile("junit", null, temp.toFile()).getPath()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .withFormat(FileFormat.PARQUET) + .build(); + // this should create a snapshot with type overwrite. - table.newOverwrite().overwriteByRowFilter(Expressions.greaterThan("id", 4)).commit(); + table + .newOverwrite() + .addFile(dataFile) + .overwriteByRowFilter(Expressions.greaterThan("id", 4)) + .commit(); // check pre-condition - that the above delete operation on table resulted in Snapshot of Type // OVERWRITE. From 0a39e754f4e4509ee237f4b2349cf651c53252bf Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Thu, 25 Apr 2024 20:38:23 +0900 Subject: [PATCH 0251/1019] AWS: Fix TestGlueCatalogTable#testCreateTable (#10221) --- .../java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 7fb1f4ed1987..f6bdd89707dc 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -94,7 +94,7 @@ public void testCreateTable() { assertThat(response.table().storageDescriptor().columns()).hasSameSizeAs(schema.columns()); assertThat(response.table().partitionKeys()).hasSameSizeAs(partitionSpec.fields()); assertThat(response.table().storageDescriptor().additionalLocations()) - .isEqualTo(tableLocationProperties.values()); + .containsExactlyInAnyOrderElementsOf(tableLocationProperties.values()); // verify metadata file exists in S3 String metaLocation = response.table().parameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); From ba0f65490af2c8f8fe119346e1edbcdc51bf01db Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 25 Apr 2024 10:09:29 -0600 Subject: [PATCH 0252/1019] Docs: Add 1.5.1 release notes (#10224) --- site/docs/releases.md | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/site/docs/releases.md b/site/docs/releases.md index d43032a107cb..1c00d0e2eee1 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -72,6 +72,23 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ``` +### 1.5.1 release +Apache Iceberg 1.5.1 was released on April 25, 2024. + +The 1.5.1 patch release contains fixes for JDBC Catalog, fixing a FileIO regression +where an extra head request was performed when reading manifests and REST client retries +for 5xx failures. The release also includes fixes for system function pushdown for CoW tables +in Spark 3.4 and 3.5. + +* Core + - Fix FileIO regression where extra head request was performed when reading manifests ([\#10114](https://github.com/apache/iceberg/pull/10114)) + - Mark 502 and 504 HTTP status codes as retryable in REST Client ([\#10113](https://github.com/apache/iceberg/pull/10113)) + - Fix JDBC Catalog table commits when migrating from V0 to V1 schema ([\#10152](https://github.com/apache/iceberg/pull/10152)) + - Fix JDBC Catalog namespaces SQL to use the proper escape character which generalizes to different database backends like Postgres and MySQL ([\#10167](https://github.com/apache/iceberg/pull/10167)) +* Spark + - Fix system function pushdown in CoW row level commands for Spark 3.5 ([\#9873](https://github.com/apache/iceberg/pull/9873)) + - Fix system function pushdown in CoW row level commands for Spark 3.4 ([\#10119](https://github.com/apache/iceberg/pull/10119)) + ### 1.5.0 release Apache Iceberg 1.5.0 was released on March 11, 2024. From 1d85611717d4fc9f704b4ddf163de499d6b7d92d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 26 Apr 2024 08:50:30 +0200 Subject: [PATCH 0253/1019] Spec: Clarify missing fields when writing (#8672) * Spec: Carify missing fields when writing Jan raised a point on slack of the symantic meaning of a field that can be written: https://apache-iceberg.slack.com/archives/C03LG1D563F/p1695834739711569 There are two options: - The field is not part of the schema, and omitted from the file - The field is part of the schema, but the value is not written (nullable) My personal take on this is that we should use static schema's when writing Avro files, so that all the fields that are either optional or required are in the schema. I'm well aware of that this doesn't impose any issues if you dogfood the Iceberg Avro reader, where you can add required fields, for example the `134: content` field in the manifest. However, I think we should try to stick to the concept of write strict, read permissive where we try to encourage people to write all the fields that are in the spec (even they if the value itself is all null). * Add manifest-list explicitly Co-authored-by: JFinis * Update wording * Comments * Retain formatting * Thanks Steven --------- Co-authored-by: JFinis --- format/spec.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index aa905e7032bc..b00c63256a4e 100644 --- a/format/spec.md +++ b/format/spec.md @@ -127,12 +127,12 @@ Tables do not require rename, except for tables that use atomic rename to implem #### Writer requirements -Some tables in this spec have columns that specify requirements for v1 and v2 tables. These requirements are intended for writers when adding metadata files to a table with the given version. +Some tables in this spec have columns that specify requirements for v1 and v2 tables. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. | Requirement | Write behavior | |-------------|----------------| | (blank) | The field should be omitted | -| _optional_ | The field can be written | +| _optional_ | The field can be written or omitted | | _required_ | The field must be written | Readers should be more permissive because v1 metadata files are allowed in v2 tables so that tables can be upgraded to v2 without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected v2 read behavior: From acfff296474bee8adb4a03bab12322a8c76ef7b4 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Fri, 26 Apr 2024 08:50:48 +0200 Subject: [PATCH 0254/1019] Flink: Move ParquetReader to LogicalTypeAnnotationVisitor (#9719) * Flink: Move ParquetReader to LogicalTypeAnnotationVisitor This will enable nanosecond timestamp support, since the OriginalType does not represent the nanosecond timestamp. * Add a test * Update gradle.properties * Add other versions as well --- .../org/apache/iceberg/data/DataTest.java | 2 +- .../flink/data/FlinkParquetReaders.java | 183 ++++++++++++------ .../flink/data/TestFlinkParquetReader.java | 101 ++++++++++ .../flink/data/FlinkParquetReaders.java | 183 ++++++++++++------ .../flink/data/TestFlinkParquetReader.java | 101 ++++++++++ .../flink/data/FlinkParquetReaders.java | 183 ++++++++++++------ .../flink/data/TestFlinkParquetReader.java | 101 ++++++++++ 7 files changed, 688 insertions(+), 166 deletions(-) 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 5ea742e45188..638a344cd2bc 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTest.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTest.java @@ -38,7 +38,7 @@ public abstract class DataTest { protected abstract void writeAndValidate(Schema schema) throws IOException; - private static final StructType SUPPORTED_PRIMITIVES = + protected static final StructType SUPPORTED_PRIMITIVES = StructType.of( required(100, "id", LongType.get()), optional(101, "data", Types.StringType.get()), diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index ab7b1174c9f3..ad4310a6d195 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; @@ -50,6 +51,7 @@ import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -193,6 +195,124 @@ public ParquetValueReader map( ParquetValueReaders.option(valueType, valueD, valueReader)); } + private static class LogicalTypeAnnotationParquetValueReaderVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final PrimitiveType primitive; + private final ColumnDescriptor desc; + private final org.apache.iceberg.types.Type.PrimitiveType expected; + + LogicalTypeAnnotationParquetValueReaderVisitor( + PrimitiveType primitive, + ColumnDescriptor desc, + org.apache.iceberg.types.Type.PrimitiveType expected) { + this.primitive = primitive; + this.desc = desc; + this.expected = expected; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + DecimalLogicalTypeAnnotation decimalLogicalType) { + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of( + new BinaryDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT64: + return Optional.of( + new LongDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT32: + return Optional.of( + new IntegerDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(decimalLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + return Optional.of(new MillisTimeReader(desc)); + } else if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(new LossyMicrosToMillisTimeReader(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timeLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MillisToTimestampTzReader(desc)); + } else { + return Optional.of(new MillisToTimestampReader(desc)); + } + } else if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MicrosToTimestampTzReader(desc)); + } else { + return Optional.of(new MicrosToTimestampReader(desc)); + } + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timestampLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int width = intLogicalType.getBitWidth(); + if (width <= 32) { + if (expected.typeId() == Types.LongType.get().typeId()) { + return Optional.of(new ParquetValueReaders.IntAsLongReader(desc)); + } else { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + } else if (width <= 64) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(intLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(new ParquetValueReaders.ByteArrayReader(desc)); + } + } + @Override @SuppressWarnings("CyclomaticComplexity") public ParquetValueReader primitive( @@ -202,61 +322,14 @@ public ParquetValueReader primitive( } ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected.typeId() == Types.LongType.get().typeId()) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case TIME_MICROS: - return new LossyMicrosToMillisTimeReader(desc); - case TIME_MILLIS: - return new MillisTimeReader(desc); - case DATE: - case INT_64: - return new ParquetValueReaders.UnboxedReader<>(desc); - case TIMESTAMP_MICROS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MicrosToTimestampTzReader(desc); - } else { - return new MicrosToTimestampReader(desc); - } - case TIMESTAMP_MILLIS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MillisToTimestampTzReader(desc); - } else { - return new MillisToTimestampReader(desc); - } - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT32: - return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return new ParquetValueReaders.ByteArrayReader(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueReaderVisitor(primitive, desc, expected)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); } switch (primitive.getPrimitiveTypeName()) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java index 1fdc4cf8381a..4cfb24f62921 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.parquet.schema.Types.primitive; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -44,15 +45,115 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetValueReader; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; import org.junit.jupiter.api.Test; public class TestFlinkParquetReader extends DataTest { private static final int NUM_RECORDS = 100; + @Test + public void testBuildReader() { + MessageType fileSchema = + new MessageType( + "test", + // 0: required(100, "id", LongType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(100) + .named("id"), + // 1: optional(101, "data", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(101) + .named("data"), + // 2: required(102, "b", Types.BooleanType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, Type.Repetition.REQUIRED) + .id(102) + .named("b"), + // 3: optional(103, "i", Types.IntegerType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(103) + .named("i"), + // 4: optional(105, "f", Types.FloatType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(104) + .named("l"), + // 5: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.FLOAT, Type.Repetition.OPTIONAL) + .id(105) + .named("f"), + // 6: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, Type.Repetition.REQUIRED) + .id(106) + .named("d"), + // 7: optional(107, "date", Types.DateType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(107) + .as(LogicalTypeAnnotation.dateType()) + .named("date"), + // 8: required(108, "ts_tz", Types.TimestampType.withZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(108) + .as( + LogicalTypeAnnotation.timestampType( + true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts_tz"), + // 9: required(109, "ts", Types.TimestampType.withoutZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(109) + .as( + LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts"), + // 10: required(110, "s", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .id(110) + .as(LogicalTypeAnnotation.stringType()) + .named("s"), + // 11: required(112, "fixed", Types.FixedType.ofLength(7)) + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(112) + .length(7) + .named("f"), + // 12: optional(113, "bytes", Types.BinaryType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(113) + .named("bytes"), + // 13: required(114, "dec_9_0", Types.DecimalType.of(9, 0)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(114) + .as(LogicalTypeAnnotation.decimalType(0, 9)) + .named("dec_9_0"), + // 14: required(115, "dec_11_2", Types.DecimalType.of(11, 2)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(115) + .as(LogicalTypeAnnotation.decimalType(2, 11)) + .named("dec_11_2"), + // 15: required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(116) + .length(16) + .as(LogicalTypeAnnotation.decimalType(10, 38)) + .named("dec_38_10"), + // 16: required(117, "time", Types.TimeType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .id(117) + .as(LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("time")); + ParquetValueReader reader = + FlinkParquetReaders.buildReader(new Schema(SUPPORTED_PRIMITIVES.fields()), fileSchema); + + assertThat(reader.columns().size()).isEqualTo(SUPPORTED_PRIMITIVES.fields().size()); + } + @Test public void testTwoLevelList() throws IOException { Schema schema = diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index ab7b1174c9f3..ad4310a6d195 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; @@ -50,6 +51,7 @@ import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -193,6 +195,124 @@ public ParquetValueReader map( ParquetValueReaders.option(valueType, valueD, valueReader)); } + private static class LogicalTypeAnnotationParquetValueReaderVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final PrimitiveType primitive; + private final ColumnDescriptor desc; + private final org.apache.iceberg.types.Type.PrimitiveType expected; + + LogicalTypeAnnotationParquetValueReaderVisitor( + PrimitiveType primitive, + ColumnDescriptor desc, + org.apache.iceberg.types.Type.PrimitiveType expected) { + this.primitive = primitive; + this.desc = desc; + this.expected = expected; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + DecimalLogicalTypeAnnotation decimalLogicalType) { + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of( + new BinaryDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT64: + return Optional.of( + new LongDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT32: + return Optional.of( + new IntegerDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(decimalLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + return Optional.of(new MillisTimeReader(desc)); + } else if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(new LossyMicrosToMillisTimeReader(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timeLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MillisToTimestampTzReader(desc)); + } else { + return Optional.of(new MillisToTimestampReader(desc)); + } + } else if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MicrosToTimestampTzReader(desc)); + } else { + return Optional.of(new MicrosToTimestampReader(desc)); + } + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timestampLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int width = intLogicalType.getBitWidth(); + if (width <= 32) { + if (expected.typeId() == Types.LongType.get().typeId()) { + return Optional.of(new ParquetValueReaders.IntAsLongReader(desc)); + } else { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + } else if (width <= 64) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(intLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(new ParquetValueReaders.ByteArrayReader(desc)); + } + } + @Override @SuppressWarnings("CyclomaticComplexity") public ParquetValueReader primitive( @@ -202,61 +322,14 @@ public ParquetValueReader primitive( } ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected.typeId() == Types.LongType.get().typeId()) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case TIME_MICROS: - return new LossyMicrosToMillisTimeReader(desc); - case TIME_MILLIS: - return new MillisTimeReader(desc); - case DATE: - case INT_64: - return new ParquetValueReaders.UnboxedReader<>(desc); - case TIMESTAMP_MICROS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MicrosToTimestampTzReader(desc); - } else { - return new MicrosToTimestampReader(desc); - } - case TIMESTAMP_MILLIS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MillisToTimestampTzReader(desc); - } else { - return new MillisToTimestampReader(desc); - } - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT32: - return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return new ParquetValueReaders.ByteArrayReader(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueReaderVisitor(primitive, desc, expected)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); } switch (primitive.getPrimitiveTypeName()) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java index 1fdc4cf8381a..4cfb24f62921 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.parquet.schema.Types.primitive; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -44,15 +45,115 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetValueReader; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; import org.junit.jupiter.api.Test; public class TestFlinkParquetReader extends DataTest { private static final int NUM_RECORDS = 100; + @Test + public void testBuildReader() { + MessageType fileSchema = + new MessageType( + "test", + // 0: required(100, "id", LongType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(100) + .named("id"), + // 1: optional(101, "data", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(101) + .named("data"), + // 2: required(102, "b", Types.BooleanType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, Type.Repetition.REQUIRED) + .id(102) + .named("b"), + // 3: optional(103, "i", Types.IntegerType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(103) + .named("i"), + // 4: optional(105, "f", Types.FloatType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(104) + .named("l"), + // 5: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.FLOAT, Type.Repetition.OPTIONAL) + .id(105) + .named("f"), + // 6: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, Type.Repetition.REQUIRED) + .id(106) + .named("d"), + // 7: optional(107, "date", Types.DateType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(107) + .as(LogicalTypeAnnotation.dateType()) + .named("date"), + // 8: required(108, "ts_tz", Types.TimestampType.withZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(108) + .as( + LogicalTypeAnnotation.timestampType( + true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts_tz"), + // 9: required(109, "ts", Types.TimestampType.withoutZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(109) + .as( + LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts"), + // 10: required(110, "s", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .id(110) + .as(LogicalTypeAnnotation.stringType()) + .named("s"), + // 11: required(112, "fixed", Types.FixedType.ofLength(7)) + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(112) + .length(7) + .named("f"), + // 12: optional(113, "bytes", Types.BinaryType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(113) + .named("bytes"), + // 13: required(114, "dec_9_0", Types.DecimalType.of(9, 0)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(114) + .as(LogicalTypeAnnotation.decimalType(0, 9)) + .named("dec_9_0"), + // 14: required(115, "dec_11_2", Types.DecimalType.of(11, 2)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(115) + .as(LogicalTypeAnnotation.decimalType(2, 11)) + .named("dec_11_2"), + // 15: required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(116) + .length(16) + .as(LogicalTypeAnnotation.decimalType(10, 38)) + .named("dec_38_10"), + // 16: required(117, "time", Types.TimeType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .id(117) + .as(LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("time")); + ParquetValueReader reader = + FlinkParquetReaders.buildReader(new Schema(SUPPORTED_PRIMITIVES.fields()), fileSchema); + + assertThat(reader.columns().size()).isEqualTo(SUPPORTED_PRIMITIVES.fields().size()); + } + @Test public void testTwoLevelList() throws IOException { Schema schema = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index ab7b1174c9f3..ad4310a6d195 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -26,6 +26,7 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.DecimalData; import org.apache.flink.table.data.GenericArrayData; @@ -50,6 +51,7 @@ import org.apache.parquet.column.ColumnDescriptor; import org.apache.parquet.io.api.Binary; import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.LogicalTypeAnnotation.DecimalLogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -193,6 +195,124 @@ public ParquetValueReader map( ParquetValueReaders.option(valueType, valueD, valueReader)); } + private static class LogicalTypeAnnotationParquetValueReaderVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final PrimitiveType primitive; + private final ColumnDescriptor desc; + private final org.apache.iceberg.types.Type.PrimitiveType expected; + + LogicalTypeAnnotationParquetValueReaderVisitor( + PrimitiveType primitive, + ColumnDescriptor desc, + org.apache.iceberg.types.Type.PrimitiveType expected) { + this.primitive = primitive; + this.desc = desc; + this.expected = expected; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + DecimalLogicalTypeAnnotation decimalLogicalType) { + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of( + new BinaryDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT64: + return Optional.of( + new LongDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT32: + return Optional.of( + new IntegerDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(decimalLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + return Optional.of(new MillisTimeReader(desc)); + } else if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(new LossyMicrosToMillisTimeReader(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timeLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MillisToTimestampTzReader(desc)); + } else { + return Optional.of(new MillisToTimestampReader(desc)); + } + } else if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MicrosToTimestampTzReader(desc)); + } else { + return Optional.of(new MicrosToTimestampReader(desc)); + } + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timestampLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int width = intLogicalType.getBitWidth(); + if (width <= 32) { + if (expected.typeId() == Types.LongType.get().typeId()) { + return Optional.of(new ParquetValueReaders.IntAsLongReader(desc)); + } else { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + } else if (width <= 64) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(intLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(new ParquetValueReaders.ByteArrayReader(desc)); + } + } + @Override @SuppressWarnings("CyclomaticComplexity") public ParquetValueReader primitive( @@ -202,61 +322,14 @@ public ParquetValueReader primitive( } ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return new StringReader(desc); - case INT_8: - case INT_16: - case INT_32: - if (expected.typeId() == Types.LongType.get().typeId()) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case TIME_MICROS: - return new LossyMicrosToMillisTimeReader(desc); - case TIME_MILLIS: - return new MillisTimeReader(desc); - case DATE: - case INT_64: - return new ParquetValueReaders.UnboxedReader<>(desc); - case TIMESTAMP_MICROS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MicrosToTimestampTzReader(desc); - } else { - return new MicrosToTimestampReader(desc); - } - case TIMESTAMP_MILLIS: - if (((Types.TimestampType) expected).shouldAdjustToUTC()) { - return new MillisToTimestampTzReader(desc); - } else { - return new MillisToTimestampReader(desc); - } - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return new BinaryDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return new LongDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - case INT32: - return new IntegerDecimalReader(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return new ParquetValueReaders.ByteArrayReader(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueReaderVisitor(primitive, desc, expected)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); } switch (primitive.getPrimitiveTypeName()) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java index 1fdc4cf8381a..4cfb24f62921 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.parquet.schema.Types.primitive; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -44,15 +45,115 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetValueReader; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.parquet.avro.AvroParquetWriter; import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; import org.junit.jupiter.api.Test; public class TestFlinkParquetReader extends DataTest { private static final int NUM_RECORDS = 100; + @Test + public void testBuildReader() { + MessageType fileSchema = + new MessageType( + "test", + // 0: required(100, "id", LongType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(100) + .named("id"), + // 1: optional(101, "data", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(101) + .named("data"), + // 2: required(102, "b", Types.BooleanType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, Type.Repetition.REQUIRED) + .id(102) + .named("b"), + // 3: optional(103, "i", Types.IntegerType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(103) + .named("i"), + // 4: optional(105, "f", Types.FloatType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(104) + .named("l"), + // 5: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.FLOAT, Type.Repetition.OPTIONAL) + .id(105) + .named("f"), + // 6: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, Type.Repetition.REQUIRED) + .id(106) + .named("d"), + // 7: optional(107, "date", Types.DateType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(107) + .as(LogicalTypeAnnotation.dateType()) + .named("date"), + // 8: required(108, "ts_tz", Types.TimestampType.withZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(108) + .as( + LogicalTypeAnnotation.timestampType( + true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts_tz"), + // 9: required(109, "ts", Types.TimestampType.withoutZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(109) + .as( + LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts"), + // 10: required(110, "s", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .id(110) + .as(LogicalTypeAnnotation.stringType()) + .named("s"), + // 11: required(112, "fixed", Types.FixedType.ofLength(7)) + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(112) + .length(7) + .named("f"), + // 12: optional(113, "bytes", Types.BinaryType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(113) + .named("bytes"), + // 13: required(114, "dec_9_0", Types.DecimalType.of(9, 0)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(114) + .as(LogicalTypeAnnotation.decimalType(0, 9)) + .named("dec_9_0"), + // 14: required(115, "dec_11_2", Types.DecimalType.of(11, 2)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(115) + .as(LogicalTypeAnnotation.decimalType(2, 11)) + .named("dec_11_2"), + // 15: required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(116) + .length(16) + .as(LogicalTypeAnnotation.decimalType(10, 38)) + .named("dec_38_10"), + // 16: required(117, "time", Types.TimeType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .id(117) + .as(LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("time")); + ParquetValueReader reader = + FlinkParquetReaders.buildReader(new Schema(SUPPORTED_PRIMITIVES.fields()), fileSchema); + + assertThat(reader.columns().size()).isEqualTo(SUPPORTED_PRIMITIVES.fields().size()); + } + @Test public void testTwoLevelList() throws IOException { Schema schema = From d62136008e3af0cca7a89fa10aadcfb1bae19a00 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 26 Apr 2024 12:30:16 +0200 Subject: [PATCH 0255/1019] Flink: Fix bounded source state restore record duplication (#10208) --- .../iceberg/flink/source/IcebergSource.java | 8 +- .../apache/iceberg/flink/SimpleDataUtil.java | 6 +- .../source/TestIcebergSourceFailover.java | 107 ++++++++++++++---- ...gSourceFailoverWithWatermarkExtractor.java | 15 +-- 4 files changed, 99 insertions(+), 37 deletions(-) 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 0655cf87a996..8ec92c8c2c1d 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 @@ -201,8 +201,12 @@ private SplitEnumerator createEnumer return new ContinuousIcebergEnumerator( enumContext, assigner, scanContext, splitPlanner, enumState); } else { - List splits = planSplitsForBatch(planningThreadName()); - assigner.onDiscoveredSplits(splits); + if (enumState == null) { + // Only do scan planning if nothing is restored from checkpoint state + List splits = planSplitsForBatch(planningThreadName()); + assigner.onDiscoveredSplits(splits); + } + return new StaticIcebergEnumerator(enumContext, assigner); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index f48764f772b4..ce6caca12158 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -284,11 +284,7 @@ public static void assertRecordsEqual(List expected, List actual public static void assertTableRecords(Table table, List expected, Duration timeout) { Awaitility.await("expected list of records should be produced") .atMost(timeout) - .untilAsserted( - () -> { - equalsRecords(expected, tableRecords(table), table.schema()); - assertRecordsEqual(expected, tableRecords(table), table.schema()); - }); + .untilAsserted(() -> assertRecordsEqual(expected, tableRecords(table), table.schema())); } public static void assertTableRecords(Table table, List expected) throws IOException { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7d991ee603c9..7047a62a2c32 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -29,7 +32,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; 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.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -40,10 +45,12 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +64,10 @@ public class TestIcebergSourceFailover { - private static final int PARALLELISM = 4; + // Parallelism higher than 1, but lower than the number of splits used by some of our tests + // The goal is to allow some splits to remain in the enumerator when restoring the state + private static final int PARALLELISM = 2; + private static final int DO_NOT_FAIL = Integer.MAX_VALUE; @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -87,6 +97,10 @@ protected IcebergSource.Builder sourceBuilder() { return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) .flinkConfig(config); } @@ -103,6 +117,55 @@ protected void assertRecords(Table table, List expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Test + public void testBoundedWithSavepoint() throws Exception { + List expectedRecords = Lists.newArrayList(); + Table sinkTable = sinkTableResource.table(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + createBoundedStreams(env, 2); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Savepoint Test"); + JobID jobId = jobClient.getJobID(); + + // Write something, but do not finish before checkpoint is created + RecordCounterToFail.waitToFail(); + CompletableFuture savepoint = + miniClusterResource + .getClusterClient() + .stopWithSavepoint( + jobId, + false, + TEMPORARY_FOLDER.newFolder().toPath().toString(), + SavepointFormatType.CANONICAL); + RecordCounterToFail.continueProcessing(); + + // Wait for the job to stop with the savepoint + String savepointPath = savepoint.get(); + + // We expect that at least a few records has written + assertThat(tableRecords(sinkTable)).hasSizeGreaterThan(0); + + // New env from the savepoint + Configuration conf = new Configuration(); + conf.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + createBoundedStreams(env, DO_NOT_FAIL); + + env.execute("Bounded Iceberg Source Savepoint Test"); + + // We expect no duplications + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -125,26 +188,8 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio } StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - - DataStream stream = - env.fromSource( - sourceBuilder().build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); + createBoundedStreams(env, expectedRecords.size() / 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); JobID jobId = jobClient.getJobID(); @@ -222,6 +267,28 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } + private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { + env.setParallelism(PARALLELISM); + + DataStream stream = + env.fromSource( + sourceBuilder().build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + DataStream streamFailingInTheMiddleOfReading = + RecordCounterToFail.wrapWithFailureAfter(stream, failAfter); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(streamFailingInTheMiddleOfReading) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + } + // ------------------------------------------------------------------------ // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index f7dc931c506c..27a8894ad494 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -88,16 +88,11 @@ protected void assertRecords(Table table, List expectedRecords, Duration Awaitility.await("expected list of records should be produced") .atMost(timeout) .untilAsserted( - () -> { - SimpleDataUtil.equalsRecords( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - }); + () -> + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema())); } private List convertLocalDateTimeToMilli(List records) { From 4f9d41a2a1b2239900915842736faa600db3d0f2 Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Fri, 26 Apr 2024 15:46:34 +0200 Subject: [PATCH 0256/1019] REST: fix incorrect token refresh thread name (#10223) The thread currently is always called "null-token-refresh" because tokenRefreshExecutor() is called before the catalog name is set. --- .../apache/iceberg/rest/RESTSessionCatalog.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 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 f8a47f19fdb9..94f3057f9fe7 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -221,11 +221,11 @@ public void initialize(String name, Map unresolved) { if (authResponse != null) { this.catalogAuth = AuthSession.fromTokenResponse( - client, tokenRefreshExecutor(), authResponse, startTimeMillis, catalogAuth); + client, tokenRefreshExecutor(name), authResponse, startTimeMillis, catalogAuth); } else if (token != null) { this.catalogAuth = AuthSession.fromAccessToken( - client, tokenRefreshExecutor(), token, expiresAtMillis(mergedProps), catalogAuth); + client, tokenRefreshExecutor(name), token, expiresAtMillis(mergedProps), catalogAuth); } this.io = newFileIO(SessionContext.createEmpty(), mergedProps); @@ -558,7 +558,7 @@ public boolean updateNamespaceMetadata( return !response.updated().isEmpty(); } - private ScheduledExecutorService tokenRefreshExecutor() { + private ScheduledExecutorService tokenRefreshExecutor(String catalogName) { if (!keepTokenRefreshed) { return null; } @@ -566,7 +566,7 @@ private ScheduledExecutorService tokenRefreshExecutor() { if (refreshExecutor == null) { synchronized (this) { if (refreshExecutor == null) { - this.refreshExecutor = ThreadPools.newScheduledPool(name() + "-token-refresh", 1); + this.refreshExecutor = ThreadPools.newScheduledPool(catalogName + "-token-refresh", 1); } } } @@ -930,7 +930,7 @@ private Pair> newSession( () -> AuthSession.fromAccessToken( client, - tokenRefreshExecutor(), + tokenRefreshExecutor(name()), credentials.get(OAuth2Properties.TOKEN), expiresAtMillis(properties), parent)); @@ -943,7 +943,7 @@ private Pair> newSession( () -> AuthSession.fromCredential( client, - tokenRefreshExecutor(), + tokenRefreshExecutor(name()), credentials.get(OAuth2Properties.CREDENTIAL), parent)); } @@ -956,7 +956,7 @@ private Pair> newSession( () -> AuthSession.fromTokenExchange( client, - tokenRefreshExecutor(), + tokenRefreshExecutor(name()), credentials.get(tokenType), tokenType, parent)); From f499b0bd891d305d60bc4b10026a22202c2b98e3 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 26 Apr 2024 15:51:37 +0200 Subject: [PATCH 0257/1019] Flink: Backport #10208 to v1.18 and v1.17 (#10230) --- .../iceberg/flink/source/IcebergSource.java | 8 +- .../apache/iceberg/flink/SimpleDataUtil.java | 6 +- .../source/TestIcebergSourceFailover.java | 107 ++++++++++++++---- ...gSourceFailoverWithWatermarkExtractor.java | 15 +-- .../iceberg/flink/source/IcebergSource.java | 8 +- .../apache/iceberg/flink/SimpleDataUtil.java | 6 +- .../source/TestIcebergSourceFailover.java | 107 ++++++++++++++---- ...gSourceFailoverWithWatermarkExtractor.java | 15 +-- 8 files changed, 198 insertions(+), 74 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 0655cf87a996..8ec92c8c2c1d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -201,8 +201,12 @@ private SplitEnumerator createEnumer return new ContinuousIcebergEnumerator( enumContext, assigner, scanContext, splitPlanner, enumState); } else { - List splits = planSplitsForBatch(planningThreadName()); - assigner.onDiscoveredSplits(splits); + if (enumState == null) { + // Only do scan planning if nothing is restored from checkpoint state + List splits = planSplitsForBatch(planningThreadName()); + assigner.onDiscoveredSplits(splits); + } + return new StaticIcebergEnumerator(enumContext, assigner); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index f48764f772b4..ce6caca12158 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -284,11 +284,7 @@ public static void assertRecordsEqual(List expected, List actual public static void assertTableRecords(Table table, List expected, Duration timeout) { Awaitility.await("expected list of records should be produced") .atMost(timeout) - .untilAsserted( - () -> { - equalsRecords(expected, tableRecords(table), table.schema()); - assertRecordsEqual(expected, tableRecords(table), table.schema()); - }); + .untilAsserted(() -> assertRecordsEqual(expected, tableRecords(table), table.schema())); } public static void assertTableRecords(Table table, List expected) throws IOException { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7d991ee603c9..7047a62a2c32 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -29,7 +32,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; 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.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -40,10 +45,12 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +64,10 @@ public class TestIcebergSourceFailover { - private static final int PARALLELISM = 4; + // Parallelism higher than 1, but lower than the number of splits used by some of our tests + // The goal is to allow some splits to remain in the enumerator when restoring the state + private static final int PARALLELISM = 2; + private static final int DO_NOT_FAIL = Integer.MAX_VALUE; @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -87,6 +97,10 @@ protected IcebergSource.Builder sourceBuilder() { return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) .flinkConfig(config); } @@ -103,6 +117,55 @@ protected void assertRecords(Table table, List expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Test + public void testBoundedWithSavepoint() throws Exception { + List expectedRecords = Lists.newArrayList(); + Table sinkTable = sinkTableResource.table(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + createBoundedStreams(env, 2); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Savepoint Test"); + JobID jobId = jobClient.getJobID(); + + // Write something, but do not finish before checkpoint is created + RecordCounterToFail.waitToFail(); + CompletableFuture savepoint = + miniClusterResource + .getClusterClient() + .stopWithSavepoint( + jobId, + false, + TEMPORARY_FOLDER.newFolder().toPath().toString(), + SavepointFormatType.CANONICAL); + RecordCounterToFail.continueProcessing(); + + // Wait for the job to stop with the savepoint + String savepointPath = savepoint.get(); + + // We expect that at least a few records has written + assertThat(tableRecords(sinkTable)).hasSizeGreaterThan(0); + + // New env from the savepoint + Configuration conf = new Configuration(); + conf.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + createBoundedStreams(env, DO_NOT_FAIL); + + env.execute("Bounded Iceberg Source Savepoint Test"); + + // We expect no duplications + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -125,26 +188,8 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio } StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - - DataStream stream = - env.fromSource( - sourceBuilder().build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); + createBoundedStreams(env, expectedRecords.size() / 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); JobID jobId = jobClient.getJobID(); @@ -222,6 +267,28 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } + private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { + env.setParallelism(PARALLELISM); + + DataStream stream = + env.fromSource( + sourceBuilder().build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + DataStream streamFailingInTheMiddleOfReading = + RecordCounterToFail.wrapWithFailureAfter(stream, failAfter); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(streamFailingInTheMiddleOfReading) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + } + // ------------------------------------------------------------------------ // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index f7dc931c506c..27a8894ad494 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -88,16 +88,11 @@ protected void assertRecords(Table table, List expectedRecords, Duration Awaitility.await("expected list of records should be produced") .atMost(timeout) .untilAsserted( - () -> { - SimpleDataUtil.equalsRecords( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - }); + () -> + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema())); } private List convertLocalDateTimeToMilli(List records) { 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 0655cf87a996..8ec92c8c2c1d 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 @@ -201,8 +201,12 @@ private SplitEnumerator createEnumer return new ContinuousIcebergEnumerator( enumContext, assigner, scanContext, splitPlanner, enumState); } else { - List splits = planSplitsForBatch(planningThreadName()); - assigner.onDiscoveredSplits(splits); + if (enumState == null) { + // Only do scan planning if nothing is restored from checkpoint state + List splits = planSplitsForBatch(planningThreadName()); + assigner.onDiscoveredSplits(splits); + } + return new StaticIcebergEnumerator(enumContext, assigner); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index f48764f772b4..ce6caca12158 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -284,11 +284,7 @@ public static void assertRecordsEqual(List expected, List actual public static void assertTableRecords(Table table, List expected, Duration timeout) { Awaitility.await("expected list of records should be produced") .atMost(timeout) - .untilAsserted( - () -> { - equalsRecords(expected, tableRecords(table), table.schema()); - assertRecordsEqual(expected, tableRecords(table), table.schema()); - }); + .untilAsserted(() -> assertRecordsEqual(expected, tableRecords(table), table.schema())); } public static void assertTableRecords(Table table, List expected) throws IOException { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7d991ee603c9..7047a62a2c32 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -29,7 +32,9 @@ import org.apache.flink.api.common.typeinfo.TypeInformation; 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.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -40,10 +45,12 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.HadoopTableResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +64,10 @@ public class TestIcebergSourceFailover { - private static final int PARALLELISM = 4; + // Parallelism higher than 1, but lower than the number of splits used by some of our tests + // The goal is to allow some splits to remain in the enumerator when restoring the state + private static final int PARALLELISM = 2; + private static final int DO_NOT_FAIL = Integer.MAX_VALUE; @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -87,6 +97,10 @@ protected IcebergSource.Builder sourceBuilder() { return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) .flinkConfig(config); } @@ -103,6 +117,55 @@ protected void assertRecords(Table table, List expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Test + public void testBoundedWithSavepoint() throws Exception { + List expectedRecords = Lists.newArrayList(); + Table sinkTable = sinkTableResource.table(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper( + sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + createBoundedStreams(env, 2); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Savepoint Test"); + JobID jobId = jobClient.getJobID(); + + // Write something, but do not finish before checkpoint is created + RecordCounterToFail.waitToFail(); + CompletableFuture savepoint = + miniClusterResource + .getClusterClient() + .stopWithSavepoint( + jobId, + false, + TEMPORARY_FOLDER.newFolder().toPath().toString(), + SavepointFormatType.CANONICAL); + RecordCounterToFail.continueProcessing(); + + // Wait for the job to stop with the savepoint + String savepointPath = savepoint.get(); + + // We expect that at least a few records has written + assertThat(tableRecords(sinkTable)).hasSizeGreaterThan(0); + + // New env from the savepoint + Configuration conf = new Configuration(); + conf.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + createBoundedStreams(env, DO_NOT_FAIL); + + env.execute("Bounded Iceberg Source Savepoint Test"); + + // We expect no duplications + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); + } + @Test public void testBoundedWithTaskManagerFailover() throws Exception { testBoundedIcebergSource(FailoverType.TM); @@ -125,26 +188,8 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio } StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - - DataStream stream = - env.fromSource( - sourceBuilder().build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, expectedRecords.size() / 2); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) - .append(); + createBoundedStreams(env, expectedRecords.size() / 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); JobID jobId = jobClient.getJobID(); @@ -222,6 +267,28 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); } + private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { + env.setParallelism(PARALLELISM); + + DataStream stream = + env.fromSource( + sourceBuilder().build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + DataStream streamFailingInTheMiddleOfReading = + RecordCounterToFail.wrapWithFailureAfter(stream, failAfter); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(streamFailingInTheMiddleOfReading) + .table(sinkTableResource.table()) + .tableLoader(sinkTableResource.tableLoader()) + .append(); + } + // ------------------------------------------------------------------------ // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index f7dc931c506c..27a8894ad494 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -88,16 +88,11 @@ protected void assertRecords(Table table, List expectedRecords, Duration Awaitility.await("expected list of records should be produced") .atMost(timeout) .untilAsserted( - () -> { - SimpleDataUtil.equalsRecords( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema()); - }); + () -> + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema())); } private List convertLocalDateTimeToMilli(List records) { From 982fad1b8e729e079c0d0769033fad696fec0f84 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 26 Apr 2024 19:19:13 +0200 Subject: [PATCH 0258/1019] Flink: Prevent setting endTag/endSnapshotId for streaming source (#10207) --- .../iceberg/flink/source/FlinkSource.java | 9 +- .../iceberg/flink/source/IcebergSource.java | 1 + .../iceberg/flink/source/ScanContext.java | 11 +- .../flink/source/TestFlinkInputFormat.java | 20 ++++ .../source/TestIcebergSourceBounded.java | 20 ++++ .../source/TestIcebergSourceContinuous.java | 15 +++ .../iceberg/flink/source/TestScanContext.java | 111 ++++++++++++++++++ .../flink/source/TestStreamScanSql.java | 2 + 8 files changed, 180 insertions(+), 9 deletions(-) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java 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 fa1656c55278..b1431a32dd20 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 @@ -45,12 +45,8 @@ 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.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class FlinkSource { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); - private FlinkSource() {} /** @@ -263,8 +259,9 @@ public FlinkInputFormat buildFormat() { contextBuilder.resolveConfig(table, readOptions, readableConfig); - return new FlinkInputFormat( - tableLoader, icebergSchema, io, encryption, contextBuilder.build()); + ScanContext context = contextBuilder.build(); + context.validate(); + return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption, context); } public DataStream build() { 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 8ec92c8c2c1d..48201ea09359 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 @@ -504,6 +504,7 @@ public IcebergSource build() { } ScanContext context = contextBuilder.build(); + context.validate(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { MetaDataReaderFunction rowDataReaderFunction = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index cf57a126ae59..ab79a3173933 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -129,11 +129,9 @@ private ScanContext( this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; this.watermarkColumn = watermarkColumn; this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; - - validate(); } - private void validate() { + void validate() { if (isStreaming) { if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { Preconditions.checkArgument( @@ -155,6 +153,13 @@ private void validate() { Preconditions.checkArgument( tag == null, String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + Preconditions.checkArgument( + snapshotId == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + asOfTimestamp == null, "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + endSnapshotId == null, "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument(endTag == null, "Cannot set end-tag option for streaming reader"); } Preconditions.checkArgument( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index ed3f54bec642..366a3c2b5087 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -18,13 +18,16 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.types.logical.RowType; @@ -184,6 +187,23 @@ public void testReadPartitionColumn() throws Exception { TestHelpers.assertRows(result, expected); } + @TestTemplate + public void testValidation() { + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + FlinkSource.forRowData() + .env(StreamExecutionEnvironment.getExecutionEnvironment()) + .tableLoader(tableLoader()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } + private List runFormat(FlinkInputFormat inputFormat) throws IOException { RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); return TestHelpers.readRows(inputFormat, rowType); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index 3c0c38e1115d..0bf09b1643be 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Collections; import java.util.List; import java.util.Map; @@ -42,8 +45,25 @@ import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; 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.TestTemplate; public class TestIcebergSourceBounded extends TestFlinkScan { + @TestTemplate + public void testValidation() { + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } @Override protected List runWithProjection(String... projected) throws Exception { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 61e05e99e14f..5765b73a1f63 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; import java.util.Collection; @@ -473,6 +474,20 @@ public void testReadingFromBranch() throws Exception { } } + @Test + public void testValidation() { + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(true) + .endTag("tag") + .build()) + .hasMessage("Cannot set end-tag option for streaming reader") + .isInstanceOf(IllegalArgumentException.class); + } + private DataStream createStream(ScanContext scanContext) throws Exception { // start the source and collect output StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java new file mode 100644 index 000000000000..146f1926257a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.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.flink.source; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +class TestScanContext { + @Test + void testIncrementalFromSnapshotId() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testIncrementalFromSnapshotTimestamp() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testStreaming() { + ScanContext context = ScanContext.builder().streaming(true).useTag("tag").build(); + assertException(context, "Cannot scan table using ref tag configured for streaming reader"); + + context = ScanContext.builder().streaming(true).useSnapshotId(1L).build(); + assertException(context, "Cannot set snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build(); + assertException(context, "Cannot set as-of-timestamp option for streaming reader"); + + context = ScanContext.builder().streaming(true).endSnapshotId(1L).build(); + assertException(context, "Cannot set end-snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).endTag("tag").build(); + assertException(context, "Cannot set end-tag option for streaming reader"); + } + + @Test + void testStartConflict() { + ScanContext context = ScanContext.builder().startTag("tag").startSnapshotId(1L).build(); + assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be set."); + } + + @Test + void testEndConflict() { + ScanContext context = ScanContext.builder().endTag("tag").endSnapshotId(1L).build(); + assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @Test + void testMaxAllowedPlanningFailures() { + ScanContext context = ScanContext.builder().maxAllowedPlanningFailures(-2).build(); + assertException( + context, "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + private void assertException(ScanContext context, String message) { + Assertions.assertThatThrownBy(() -> context.validate()) + .hasMessage(message) + .isInstanceOf(IllegalArgumentException.class); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d86927999760..420bcd52bfda 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -48,7 +48,9 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +@Timeout(60) public class TestStreamScanSql extends CatalogTestBase { private static final String TABLE = "test_table"; private static final FileFormat FORMAT = FileFormat.PARQUET; From 7487ba76ed88df11aa60b2544172f4561bc9c727 Mon Sep 17 00:00:00 2001 From: pvary Date: Sat, 27 Apr 2024 15:44:39 +0200 Subject: [PATCH 0259/1019] Flink: Backport #10207 to v1.18 and v1.17 (#10235) Co-authored-by: Peter Vary --- .../iceberg/flink/source/FlinkSource.java | 9 +- .../iceberg/flink/source/IcebergSource.java | 1 + .../iceberg/flink/source/ScanContext.java | 11 +- .../flink/source/TestFlinkInputFormat.java | 20 ++++ .../source/TestIcebergSourceBounded.java | 20 ++++ .../source/TestIcebergSourceContinuous.java | 15 +++ .../iceberg/flink/source/TestScanContext.java | 111 ++++++++++++++++++ .../flink/source/TestStreamScanSql.java | 2 + .../iceberg/flink/source/FlinkSource.java | 9 +- .../iceberg/flink/source/IcebergSource.java | 1 + .../iceberg/flink/source/ScanContext.java | 11 +- .../flink/source/TestFlinkInputFormat.java | 20 ++++ .../source/TestIcebergSourceBounded.java | 20 ++++ .../source/TestIcebergSourceContinuous.java | 15 +++ .../iceberg/flink/source/TestScanContext.java | 111 ++++++++++++++++++ .../flink/source/TestStreamScanSql.java | 2 + 16 files changed, 360 insertions(+), 18 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java index fa1656c55278..b1431a32dd20 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -45,12 +45,8 @@ 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.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class FlinkSource { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); - private FlinkSource() {} /** @@ -263,8 +259,9 @@ public FlinkInputFormat buildFormat() { contextBuilder.resolveConfig(table, readOptions, readableConfig); - return new FlinkInputFormat( - tableLoader, icebergSchema, io, encryption, contextBuilder.build()); + ScanContext context = contextBuilder.build(); + context.validate(); + return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption, context); } public DataStream build() { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 8ec92c8c2c1d..48201ea09359 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -504,6 +504,7 @@ public IcebergSource build() { } ScanContext context = contextBuilder.build(); + context.validate(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { MetaDataReaderFunction rowDataReaderFunction = diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index cf57a126ae59..ab79a3173933 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -129,11 +129,9 @@ private ScanContext( this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; this.watermarkColumn = watermarkColumn; this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; - - validate(); } - private void validate() { + void validate() { if (isStreaming) { if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { Preconditions.checkArgument( @@ -155,6 +153,13 @@ private void validate() { Preconditions.checkArgument( tag == null, String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + Preconditions.checkArgument( + snapshotId == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + asOfTimestamp == null, "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + endSnapshotId == null, "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument(endTag == null, "Cannot set end-tag option for streaming reader"); } Preconditions.checkArgument( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index ed3f54bec642..366a3c2b5087 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -18,13 +18,16 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.types.logical.RowType; @@ -184,6 +187,23 @@ public void testReadPartitionColumn() throws Exception { TestHelpers.assertRows(result, expected); } + @TestTemplate + public void testValidation() { + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + FlinkSource.forRowData() + .env(StreamExecutionEnvironment.getExecutionEnvironment()) + .tableLoader(tableLoader()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } + private List runFormat(FlinkInputFormat inputFormat) throws IOException { RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); return TestHelpers.readRows(inputFormat, rowType); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index 3c0c38e1115d..0bf09b1643be 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Collections; import java.util.List; import java.util.Map; @@ -42,8 +45,25 @@ import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; 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.TestTemplate; public class TestIcebergSourceBounded extends TestFlinkScan { + @TestTemplate + public void testValidation() { + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } @Override protected List runWithProjection(String... projected) throws Exception { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index bfd7fa5758e3..3f5af78704ca 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; import java.util.Collection; @@ -456,6 +457,20 @@ public void testReadingFromBranch() throws Exception { } } + @Test + public void testValidation() { + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(true) + .endTag("tag") + .build()) + .hasMessage("Cannot set end-tag option for streaming reader") + .isInstanceOf(IllegalArgumentException.class); + } + private DataStream createStream(ScanContext scanContext) throws Exception { // start the source and collect output StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java new file mode 100644 index 000000000000..146f1926257a --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.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.flink.source; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +class TestScanContext { + @Test + void testIncrementalFromSnapshotId() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testIncrementalFromSnapshotTimestamp() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testStreaming() { + ScanContext context = ScanContext.builder().streaming(true).useTag("tag").build(); + assertException(context, "Cannot scan table using ref tag configured for streaming reader"); + + context = ScanContext.builder().streaming(true).useSnapshotId(1L).build(); + assertException(context, "Cannot set snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build(); + assertException(context, "Cannot set as-of-timestamp option for streaming reader"); + + context = ScanContext.builder().streaming(true).endSnapshotId(1L).build(); + assertException(context, "Cannot set end-snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).endTag("tag").build(); + assertException(context, "Cannot set end-tag option for streaming reader"); + } + + @Test + void testStartConflict() { + ScanContext context = ScanContext.builder().startTag("tag").startSnapshotId(1L).build(); + assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be set."); + } + + @Test + void testEndConflict() { + ScanContext context = ScanContext.builder().endTag("tag").endSnapshotId(1L).build(); + assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @Test + void testMaxAllowedPlanningFailures() { + ScanContext context = ScanContext.builder().maxAllowedPlanningFailures(-2).build(); + assertException( + context, "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + private void assertException(ScanContext context, String message) { + Assertions.assertThatThrownBy(() -> context.validate()) + .hasMessage(message) + .isInstanceOf(IllegalArgumentException.class); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 9e043bbbbbd2..421498464602 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -48,7 +48,9 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +@Timeout(60) public class TestStreamScanSql extends CatalogTestBase { private static final String TABLE = "test_table"; private static final FileFormat FORMAT = FileFormat.PARQUET; 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 fa1656c55278..b1431a32dd20 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 @@ -45,12 +45,8 @@ 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.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class FlinkSource { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSource.class); - private FlinkSource() {} /** @@ -263,8 +259,9 @@ public FlinkInputFormat buildFormat() { contextBuilder.resolveConfig(table, readOptions, readableConfig); - return new FlinkInputFormat( - tableLoader, icebergSchema, io, encryption, contextBuilder.build()); + ScanContext context = contextBuilder.build(); + context.validate(); + return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption, context); } public DataStream build() { 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 8ec92c8c2c1d..48201ea09359 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 @@ -504,6 +504,7 @@ public IcebergSource build() { } ScanContext context = contextBuilder.build(); + context.validate(); if (readerFunction == null) { if (table instanceof BaseMetadataTable) { MetaDataReaderFunction rowDataReaderFunction = diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java index cf57a126ae59..ab79a3173933 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -129,11 +129,9 @@ private ScanContext( this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; this.watermarkColumn = watermarkColumn; this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; - - validate(); } - private void validate() { + void validate() { if (isStreaming) { if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { Preconditions.checkArgument( @@ -155,6 +153,13 @@ private void validate() { Preconditions.checkArgument( tag == null, String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + Preconditions.checkArgument( + snapshotId == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + asOfTimestamp == null, "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + endSnapshotId == null, "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument(endTag == null, "Cannot set end-tag option for streaming reader"); } Preconditions.checkArgument( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index ed3f54bec642..366a3c2b5087 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -18,13 +18,16 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; import java.util.Collections; import java.util.List; import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.types.logical.RowType; @@ -184,6 +187,23 @@ public void testReadPartitionColumn() throws Exception { TestHelpers.assertRows(result, expected); } + @TestTemplate + public void testValidation() { + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + FlinkSource.forRowData() + .env(StreamExecutionEnvironment.getExecutionEnvironment()) + .tableLoader(tableLoader()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } + private List runFormat(FlinkInputFormat inputFormat) throws IOException { RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); return TestHelpers.readRows(inputFormat, rowType); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index 3c0c38e1115d..0bf09b1643be 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Collections; import java.util.List; import java.util.Map; @@ -42,8 +45,25 @@ import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; 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.TestTemplate; public class TestIcebergSourceBounded extends TestFlinkScan { + @TestTemplate + public void testValidation() { + catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } @Override protected List runWithProjection(String... projected) throws Exception { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 61e05e99e14f..5765b73a1f63 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; import java.util.Collection; @@ -473,6 +474,20 @@ public void testReadingFromBranch() throws Exception { } } + @Test + public void testValidation() { + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableResource.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(true) + .endTag("tag") + .build()) + .hasMessage("Cannot set end-tag option for streaming reader") + .isInstanceOf(IllegalArgumentException.class); + } + private DataStream createStream(ScanContext scanContext) throws Exception { // start the source and collect output StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java new file mode 100644 index 000000000000..146f1926257a --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.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.flink.source; + +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +class TestScanContext { + @Test + void testIncrementalFromSnapshotId() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testIncrementalFromSnapshotTimestamp() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testStreaming() { + ScanContext context = ScanContext.builder().streaming(true).useTag("tag").build(); + assertException(context, "Cannot scan table using ref tag configured for streaming reader"); + + context = ScanContext.builder().streaming(true).useSnapshotId(1L).build(); + assertException(context, "Cannot set snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build(); + assertException(context, "Cannot set as-of-timestamp option for streaming reader"); + + context = ScanContext.builder().streaming(true).endSnapshotId(1L).build(); + assertException(context, "Cannot set end-snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).endTag("tag").build(); + assertException(context, "Cannot set end-tag option for streaming reader"); + } + + @Test + void testStartConflict() { + ScanContext context = ScanContext.builder().startTag("tag").startSnapshotId(1L).build(); + assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be set."); + } + + @Test + void testEndConflict() { + ScanContext context = ScanContext.builder().endTag("tag").endSnapshotId(1L).build(); + assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @Test + void testMaxAllowedPlanningFailures() { + ScanContext context = ScanContext.builder().maxAllowedPlanningFailures(-2).build(); + assertException( + context, "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + private void assertException(ScanContext context, String message) { + Assertions.assertThatThrownBy(() -> context.validate()) + .hasMessage(message) + .isInstanceOf(IllegalArgumentException.class); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 9e043bbbbbd2..421498464602 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -48,7 +48,9 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +@Timeout(60) public class TestStreamScanSql extends CatalogTestBase { private static final String TABLE = "test_table"; private static final FileFormat FORMAT = FileFormat.PARQUET; From 60391c8dbe54fd45ba131d7fe78fb7e07b5a89a0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 28 Apr 2024 08:04:31 +0200 Subject: [PATCH 0260/1019] Build: Bump nessie from 0.79.0 to 0.80.0 (#10237) --- 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 b561f66e0c34..3bdb406b0c01 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.79.0" +nessie = "0.80.0" netty-buffer = "4.1.109.Final" netty-buffer-compat = "4.1.109.Final" object-client-bundle = "3.3.2" From 1c1861aaf30bbc92c76a9c94e254962b55b7df35 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Apr 2024 08:45:08 +0200 Subject: [PATCH 0261/1019] Build: Bump software.amazon.awssdk:bom from 2.25.35 to 2.25.40 (#10240) Bumps software.amazon.awssdk:bom from 2.25.35 to 2.25.40. --- 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 3bdb406b0c01..a7b2626fd476 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.35" +awssdk-bom = "2.25.40" azuresdk-bom = "1.2.22" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 7a70c41d8f9a8e3390e18c8927f39e80e7be9ebd Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 29 Apr 2024 12:15:27 +0530 Subject: [PATCH 0262/1019] Spark: Bump minor version for Spark-3.4 (#10243) release notes: https://spark.apache.org/releases/spark-release-3-4-3.html --- 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 a7b2626fd476..17d093e0b975 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -78,7 +78,7 @@ scala-collection-compat = "2.11.0" slf4j = "1.7.36" snowflake-jdbc = "3.15.1" spark-hive33 = "3.3.4" -spark-hive34 = "3.4.2" +spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.34" From a9571b6be5667196904a94f830eb9b5bdaec2482 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Apr 2024 08:45:52 +0200 Subject: [PATCH 0263/1019] Build: Bump com.gorylenko.gradle-git-properties:gradle-git-properties (#10239) Bumps com.gorylenko.gradle-git-properties:gradle-git-properties from 2.4.1 to 2.4.2. --- updated-dependencies: - dependency-name: com.gorylenko.gradle-git-properties:gradle-git-properties 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 b30e4550cc60..2b95fe291790 100644 --- a/build.gradle +++ b/build.gradle @@ -39,7 +39,7 @@ buildscript { classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.5.0" classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' - classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.1' + classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.2' classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.0.0' classpath 'org.openapitools:openapi-generator-gradle-plugin:6.6.0' } From 496cf457bea80f21b223ea8ca12e9b5af1867137 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Apr 2024 08:46:13 +0200 Subject: [PATCH 0264/1019] Build: Bump com.google.errorprone:error_prone_annotations (#10236) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.26.1 to 2.27.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.26.1...v2.27.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 17d093e0b975..ba4e0129129b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -36,7 +36,7 @@ calcite = "1.10.0" delta-standalone = "3.1.0" delta-spark = "3.1.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.26.1" +errorprone-annotations = "2.27.0" findbugs-jsr305 = "3.0.2" flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} From fe8a756fd1ebb22363b8b6cb077a5ed0077722d9 Mon Sep 17 00:00:00 2001 From: Marc Cenac <547446+mrcnc@users.noreply.github.com> Date: Mon, 29 Apr 2024 05:20:15 -0500 Subject: [PATCH 0265/1019] Core: Add property to disable table initialization for JdbcCatalog (#10124) --- .../org/apache/iceberg/jdbc/JdbcCatalog.java | 5 +- .../org/apache/iceberg/jdbc/JdbcUtil.java | 3 + .../apache/iceberg/jdbc/TestJdbcCatalog.java | 73 +++++++++++++++++++ 3 files changed, 80 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index dc7352743e88..71590e7618a2 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -86,7 +86,7 @@ public class JdbcCatalog extends BaseMetastoreViewCatalog private Map catalogProperties; private final Function, FileIO> ioBuilder; private final Function, JdbcClientPool> clientPoolBuilder; - private final boolean initializeCatalogTables; + private boolean initializeCatalogTables; private CloseableGroup closeableGroup; private JdbcUtil.SchemaVersion schemaVersion = JdbcUtil.SchemaVersion.V0; @@ -137,6 +137,9 @@ public void initialize(String name, Map properties) { this.connections = new JdbcClientPool(uri, properties); } + this.initializeCatalogTables = + PropertyUtil.propertyAsBoolean( + properties, JdbcUtil.INIT_CATALOG_TABLES_PROPERTY, initializeCatalogTables); if (initializeCatalogTables) { initializeCatalogTables(); } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index ffa606a94f29..749c2d485f0f 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -39,6 +39,9 @@ final class JdbcUtil { static final String STRICT_MODE_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "strict-mode"; // property to control if view support is added to the existing database static final String SCHEMA_VERSION_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "schema-version"; + // property to control if catalog tables are created during initialization + static final String INIT_CATALOG_TABLES_PROPERTY = + JdbcCatalog.PROPERTY_PREFIX + "init-catalog-tables"; enum SchemaVersion { V0, 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 c2f0869d0029..985c84f0dc32 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -31,6 +31,7 @@ import java.nio.file.Files; import java.nio.file.Paths; import java.sql.Connection; +import java.sql.DatabaseMetaData; import java.sql.ResultSet; import java.sql.SQLException; import java.util.List; @@ -167,6 +168,46 @@ public void testInitialize() { jdbcCatalog.initialize("test_jdbc_catalog", properties); } + @Test + public void testDisableInitCatalogTablesOverridesDefault() throws Exception { + // as this test uses different connections, we can't use memory database (as it's per + // connection), but a file database instead + java.nio.file.Path dbFile = Files.createTempFile("icebergInitCatalogTables", "db"); + String jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); + + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, jdbcUrl); + properties.put(JdbcUtil.INIT_CATALOG_TABLES_PROPERTY, "false"); + + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.initialize("test_jdbc_catalog", properties); + + assertThat(catalogTablesExist(jdbcUrl)).isFalse(); + + assertThatThrownBy(() -> jdbcCatalog.listNamespaces()) + .isInstanceOf(UncheckedSQLException.class) + .hasMessage(String.format("Failed to execute query: %s", JdbcUtil.LIST_ALL_NAMESPACES_SQL)); + } + + @Test + public void testEnableInitCatalogTablesOverridesDefault() throws Exception { + // as this test uses different connections, we can't use memory database (as it's per + // connection), but a file database instead + java.nio.file.Path dbFile = Files.createTempFile("icebergInitCatalogTables", "db"); + String jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); + + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, jdbcUrl); + properties.put(JdbcUtil.INIT_CATALOG_TABLES_PROPERTY, "true"); + + JdbcCatalog jdbcCatalog = new JdbcCatalog(null, null, false); + jdbcCatalog.initialize("test_jdbc_catalog", properties); + + assertThat(catalogTablesExist(jdbcUrl)).isTrue(); + } + @Test public void testInitSchemaV0() { Map properties = Maps.newHashMap(); @@ -1123,4 +1164,36 @@ private void initLegacySchema(String jdbcUrl) throws SQLException { .execute(); } } + + private boolean catalogTablesExist(String jdbcUrl) throws SQLException { + SQLiteDataSource dataSource = new SQLiteDataSource(); + dataSource.setUrl(jdbcUrl); + + boolean catalogTableExists = false; + boolean namespacePropertiesTableExists = false; + + try (Connection connection = dataSource.getConnection()) { + DatabaseMetaData metadata = connection.getMetaData(); + if (tableExists(metadata, JdbcUtil.CATALOG_TABLE_VIEW_NAME)) { + catalogTableExists = true; + } + if (tableExists(metadata, JdbcUtil.NAMESPACE_PROPERTIES_TABLE_NAME)) { + namespacePropertiesTableExists = true; + } + } + + return catalogTableExists && namespacePropertiesTableExists; + } + + private boolean tableExists(DatabaseMetaData metadata, String tableName) throws SQLException { + ResultSet resultSet = metadata.getTables(null, null, tableName, new String[] {"TABLE"}); + + while (resultSet.next()) { + if (tableName.equals(resultSet.getString("TABLE_NAME"))) { + return true; + } + } + + return false; + } } From 33fa01cffec2b48187cb44e296053a4df9b09adf Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 30 Apr 2024 19:23:55 +0900 Subject: [PATCH 0266/1019] Flink: Migrate tests to JUnit5 (#10232) --- .../flink/IcebergConnectorSmokeTest.java | 10 +- .../apache/iceberg/flink/FlinkTestBase.java | 129 ------ .../iceberg/flink/TestCatalogLoader.java | 26 +- .../iceberg/flink/TestChangeLogTable.java | 73 ++-- .../flink/TestFlinkAnonymousTable.java | 13 +- .../iceberg/flink/TestFlinkHiveCatalog.java | 30 +- .../iceberg/flink/TestIcebergConnector.java | 81 ++-- .../flink/source/ChangeLogTableTestBase.java | 24 +- .../flink/source/TestBoundedTableFactory.java | 30 +- .../flink/source/TestFlinkSourceConfig.java | 16 +- .../flink/source/TestFlinkTableSource.java | 403 +++++++++--------- .../flink/IcebergConnectorSmokeTest.java | 10 +- .../apache/iceberg/flink/FlinkTestBase.java | 129 ------ .../iceberg/flink/TestCatalogLoader.java | 26 +- .../iceberg/flink/TestChangeLogTable.java | 73 ++-- .../flink/TestFlinkAnonymousTable.java | 13 +- .../iceberg/flink/TestFlinkHiveCatalog.java | 30 +- .../iceberg/flink/TestIcebergConnector.java | 81 ++-- .../flink/source/ChangeLogTableTestBase.java | 24 +- .../flink/source/TestBoundedTableFactory.java | 30 +- .../flink/source/TestFlinkSourceConfig.java | 16 +- .../flink/source/TestFlinkTableSource.java | 403 +++++++++--------- .../flink/IcebergConnectorSmokeTest.java | 10 +- .../apache/iceberg/flink/FlinkTestBase.java | 147 ------- .../iceberg/flink/TestCatalogLoader.java | 26 +- .../iceberg/flink/TestChangeLogTable.java | 75 ++-- .../flink/TestFlinkAnonymousTable.java | 13 +- .../iceberg/flink/TestFlinkHiveCatalog.java | 30 +- .../iceberg/flink/TestIcebergConnector.java | 81 ++-- .../flink/source/ChangeLogTableTestBase.java | 24 +- .../flink/source/TestBoundedTableFactory.java | 30 +- .../flink/source/TestFlinkSourceConfig.java | 16 +- .../flink/source/TestFlinkTableSource.java | 403 +++++++++--------- 33 files changed, 1036 insertions(+), 1489 deletions(-) delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java diff --git a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java index 65cfba1ec876..0d9bbf9d3601 100644 --- a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java +++ b/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java @@ -18,12 +18,4 @@ */ package org.apache.iceberg.flink; -import java.util.Map; - -public class IcebergConnectorSmokeTest extends TestIcebergConnector { - - public IcebergConnectorSmokeTest( - String catalogName, Map properties, boolean isStreaming) { - super(catalogName, properties, isStreaming); - } -} +public class IcebergConnectorSmokeTest extends TestIcebergConnector {} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java deleted file mode 100644 index 8076e0ec76f8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ /dev/null @@ -1,129 +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; - -import java.util.List; -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.test.util.MiniClusterWithClientResource; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; - -public abstract class FlinkTestBase extends TestBaseUtils { - - @ClassRule - public static MiniClusterWithClientResource miniClusterResource = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeClass - public static void startMetastore() { - FlinkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - FlinkTestBase.hiveConf = metastore.hiveConf(); - FlinkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterClass - public static void stopMetastore() throws Exception { - metastore.stop(); - FlinkTestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java index 384ac5c52d00..e8f65921c19a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java @@ -19,6 +19,8 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.CatalogProperties.URI; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.entry; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -37,33 +39,31 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** Test for {@link CatalogLoader}. */ -public class TestCatalogLoader extends FlinkTestBase { +public class TestCatalogLoader extends TestBase { private static File warehouse = null; private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - @BeforeClass + @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); + assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } - @AfterClass + @AfterAll public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); } } @@ -96,11 +96,9 @@ private static void validateCatalogLoader(CatalogLoader loader) private static void validateHadoopConf(Table table) { FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + assertThat(hadoopIO.conf()).contains(entry("my_key", "my_value")); } @SuppressWarnings("unchecked") diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java index 8e9066e391c9..7fc6ab82490d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -25,6 +27,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -37,22 +42,18 @@ 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.Sets; import org.apache.iceberg.util.StructLikeSet; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +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; /** * In this test case, we mainly cover the impact of primary key selection, multiple operations * within a single transaction, and multiple operations between different txn on the correctness of * the data. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestChangeLogTable extends ChangeLogTableTestBase { private static final Configuration CONF = new Configuration(); private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; @@ -62,26 +63,19 @@ public class TestChangeLogTable extends ChangeLogTableTestBase { private static final String TABLE_NAME = "test_table"; private static String warehouse; - private final boolean partitioned; + @Parameter private boolean partitioned; - @Parameterized.Parameters(name = "PartitionedTable={0}") + @Parameters(name = "PartitionedTable={0}") public static Iterable parameters() { return ImmutableList.of(new Object[] {true}, new Object[] {false}); } - public TestChangeLogTable(boolean partitioned) { - this.partitioned = partitioned; - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + @BeforeEach + public void before() throws IOException { + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); warehouse = String.format("file:%s", warehouseFile); - } - @Before - public void before() { sql( "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", CATALOG_NAME, warehouse); @@ -94,7 +88,7 @@ public void before() { getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); } - @After + @AfterEach @Override public void clean() { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); @@ -103,7 +97,7 @@ public void clean() { BoundedTableFactory.clearDataSets(); } - @Test + @TestTemplate public void testSqlChangeLogOnIdKey() throws Exception { List> inputRowsPerCheckpoint = ImmutableList.of( @@ -135,7 +129,7 @@ public void testSqlChangeLogOnIdKey() throws Exception { TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -162,7 +156,7 @@ public void testChangeLogOnDataKey() throws Exception { testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -191,7 +185,7 @@ public void testChangeLogOnIdDataKey() throws Exception { TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); } - @Test + @TestTemplate public void testPureInsertOnIdKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -249,10 +243,7 @@ private void testSqlChangeLog( + " WITH ('connector'='BoundedSource', 'data-id'='%s')", SOURCE_TABLE, dataId); - Assert.assertEquals( - "Should have the expected rows", - listJoin(inputRowsPerCheckpoint), - sql("SELECT * FROM %s", SOURCE_TABLE)); + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)).isEqualTo(listJoin(inputRowsPerCheckpoint)); Table table = createTable(tableName, key, partitioned); sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); @@ -260,23 +251,23 @@ private void testSqlChangeLog( table.refresh(); List snapshots = findValidSnapshots(table); int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + assertThat(snapshots) + .as("Should have the expected snapshot number") + .hasSameSizeAs(expectedRecordsPerCheckpoint); for (int i = 0; i < expectedSnapshotNum; i++) { long snapshotId = snapshots.get(i).snapshotId(); List expectedRows = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(table, expectedRows), - actualRowSet(table, snapshotId)); + assertThat(actualRowSet(table, snapshotId)) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(table, expectedRows)); } if (expectedSnapshotNum > 0) { - Assert.assertEquals( - "Should have the expected rows in the final table", - Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), - Sets.newHashSet(sql("SELECT * FROM %s", tableName))); + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in the final table") + .containsExactlyInAnyOrderElementsOf( + expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java index 2fbd7627efab..70c8043f8fbb 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -18,22 +18,24 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; +import java.nio.file.Files; import java.util.concurrent.TimeUnit; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableDescriptor; import org.apache.flink.table.api.TableEnvironment; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Test; +import org.junit.jupiter.api.Test; -public class TestFlinkAnonymousTable extends FlinkTestBase { +public class TestFlinkAnonymousTable extends TestBase { @Test public void testWriteAnonymousTable() throws Exception { - File warehouseDir = TEMPORARY_FOLDER.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); TableEnvironment tEnv = getTableEnv(); Table table = tEnv.from( @@ -57,8 +59,7 @@ public void testWriteAnonymousTable() throws Exception { .atMost(3, TimeUnit.SECONDS) .untilAsserted( () -> - Assertions.assertThat( - warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + assertThat(warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) .exists()); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java index 47ee2afceb02..f1de267cf29b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -28,14 +30,9 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkHiveCatalog extends FlinkTestBase { +import org.junit.jupiter.api.Test; - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); +public class TestFlinkHiveCatalog extends TestBase { @Test public void testCreateCatalogWithWarehouseLocation() throws IOException { @@ -44,7 +41,7 @@ public void testCreateCatalogWithWarehouseLocation() throws IOException { props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - File warehouseDir = tempFolder.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); checkSQLQuery(props, warehouseDir); @@ -53,9 +50,9 @@ public void testCreateCatalogWithWarehouseLocation() throws IOException { @Test public void testCreateCatalogWithHiveConfDir() throws IOException { // Dump the hive conf into a local file. - File hiveConfDir = tempFolder.newFolder(); + File hiveConfDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); - File warehouseDir = tempFolder.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { Configuration newConf = new Configuration(hiveConf); // Set another new directory which is different with the hive metastore's warehouse path. @@ -63,7 +60,7 @@ public void testCreateCatalogWithHiveConfDir() throws IOException { HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); newConf.writeXml(fos); } - Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); + assertThat(hiveSiteXML.toPath()).exists(); // Construct the catalog attributions. Map props = Maps.newHashMap(); @@ -86,15 +83,16 @@ private void checkSQLQuery(Map catalogProperties, File warehouse sql("INSERT INTO test_table SELECT 1, 'a'"); Path databasePath = warehouseDir.toPath().resolve("test_db.db"); - Assert.assertTrue("Database path should exist", Files.exists(databasePath)); + assertThat(databasePath).exists(); Path tablePath = databasePath.resolve("test_table"); - Assert.assertTrue("Table path should exist", Files.exists(tablePath)); + assertThat(tablePath).exists(); Path dataPath = tablePath.resolve("data"); - Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); - Assert.assertEquals( - "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); + assertThat(dataPath).exists(); + assertThat(Files.list(dataPath).count()) + .as("Should have a .crc file and a .parquet file") + .isEqualTo(2); sql("DROP TABLE test_table"); sql("DROP DATABASE test_db"); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index cb409b784383..013b98e3b82b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -18,8 +18,12 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Files; import java.util.Map; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -32,33 +36,34 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; 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.relocated.com.google.common.collect.Sets; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergConnector extends FlinkTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergConnector extends TestBase { private static final String TABLE_NAME = "test_table"; - @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); + @Parameter(index = 0) + private String catalogName; + + @Parameter(index = 1) + private Map properties; + + @Parameter(index = 2) + private boolean isStreaming; - private final String catalogName; - private final Map properties; - private final boolean isStreaming; private volatile TableEnvironment tEnv; - @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") + @Parameters(name = "catalogName = {0}, properties = {1}, isStreaming = {2}") public static Iterable parameters() { return Lists.newArrayList( // Create iceberg table in the hadoop catalog and default database. @@ -161,13 +166,6 @@ public static Iterable parameters() { }); } - public TestIcebergConnector( - String catalogName, Map properties, boolean isStreaming) { - this.catalogName = catalogName; - this.properties = properties; - this.isStreaming = isStreaming; - } - @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { @@ -198,7 +196,7 @@ protected TableEnvironment getTableEnv() { return tEnv; } - @After + @AfterEach public void after() throws TException { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); @@ -226,34 +224,27 @@ private void testCreateConnectorTable() { // Create table under the flink's current database. sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); FlinkCatalogFactory factory = new FlinkCatalogFactory(); Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); - Assert.assertTrue( - "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); - Assert.assertTrue( - "Should have created the expected table", - flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); + assertThat(flinkCatalog.databaseExists(databaseName())).isTrue(); + assertThat(flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))).isTrue(); // Drop and create it again. sql("DROP TABLE %s", TABLE_NAME); sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); } - @Test + @TestTemplate public void testCreateTableUnderDefaultDatabase() { testCreateConnectorTable(); } - @Test + @TestTemplate public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); @@ -261,7 +252,7 @@ public void testCatalogDatabaseConflictWithFlinkDatabase() { try { testCreateConnectorTable(); // Ensure that the table was created under the specific database. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) .isInstanceOf(org.apache.flink.table.api.TableException.class) .hasMessageStartingWith("Could not execute CreateTable in path"); @@ -273,7 +264,7 @@ public void testCatalogDatabaseConflictWithFlinkDatabase() { } } - @Test + @TestTemplate public void testConnectorTableInIcebergCatalog() { // Create the catalog properties Map catalogProps = Maps.newHashMap(); @@ -292,7 +283,7 @@ public void testConnectorTableInIcebergCatalog() { // Create a connector table in an iceberg catalog. sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", @@ -340,9 +331,11 @@ private String toWithClause(Map props) { return CatalogTestBase.toWithClause(props); } - private static String createWarehouse() { + private String createWarehouse() { try { - return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); + return String.format( + "file://%s", + Files.createTempDirectory(temporaryDirectory, "junit").toFile().getAbsolutePath()); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index 7aa2b8034bc5..d3748e008b8e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -26,20 +28,26 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.FlinkTestBase; import org.apache.iceberg.flink.MiniClusterResource; -import org.junit.After; -import org.junit.Rule; -import org.junit.rules.TestName; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; -public class ChangeLogTableTestBase extends FlinkTestBase { +public class ChangeLogTableTestBase extends TestBase { private volatile TableEnvironment tEnv = null; - @Rule public TestName name = new TestName(); + protected String tableName; + + @BeforeEach + public void setup(TestInfo testInfo) { + assertThat(testInfo.getTestMethod()).isPresent(); + this.tableName = testInfo.getTestMethod().get().getName(); + } - @After + @AfterEach public void clean() { - sql("DROP TABLE IF EXISTS %s", name.getMethodName()); + sql("DROP TABLE IF EXISTS %s", tableName); BoundedTableFactory.clearDataSets(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java index 7b5f9328694c..bde751e1f87f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java @@ -18,36 +18,32 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Objects; import java.util.stream.Collectors; import org.apache.flink.types.Row; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestBoundedTableFactory extends ChangeLogTableTestBase { @Test public void testEmptyDataSet() { - String table = name.getMethodName(); List> emptyDataSet = ImmutableList.of(); String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); sql( "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); + tableName, dataId); - Assert.assertEquals( - "Should have caught empty change log set.", - ImmutableList.of(), - sql("SELECT * FROM %s", table)); + assertThat(sql("SELECT * FROM %s", tableName)).isEmpty(); } @Test public void testBoundedTableFactory() { - String table = name.getMethodName(); List> dataSet = ImmutableList.of( ImmutableList.of( @@ -71,17 +67,15 @@ public void testBoundedTableFactory() { String dataId = BoundedTableFactory.registerDataSet(dataSet); sql( "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); + tableName, dataId); List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); - Assert.assertEquals( - "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); + assertThat(sql("SELECT * FROM %s", tableName)).isEqualTo(rowSet); - Assert.assertEquals( - "Should have the expected change log events", - rowSet.stream() - .filter(r -> Objects.equals(r.getField(1), "aaa")) - .collect(Collectors.toList()), - sql("SELECT * FROM %s WHERE data='aaa'", table)); + assertThat(sql("SELECT * FROM %s WHERE data='aaa'", tableName)) + .isEqualTo( + rowSet.stream() + .filter(r -> Objects.equals(r.getField(1), "aaa")) + .collect(Collectors.toList())); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 1814ff8f8542..bc7194e38088 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkSourceConfig extends TestFlinkTableSource { private static final String TABLE = "test_table"; @@ -31,8 +32,7 @@ public class TestFlinkSourceConfig extends TestFlinkTableSource { @Test public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) + assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } @@ -43,16 +43,16 @@ public void testFlinkHintConfig() { sql( "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", TABLE, System.currentTimeMillis()); - Assert.assertEquals(3, result.size()); + assertThat(result).hasSize(3); } @Test public void testReadOptionHierarchy() { getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); - Assert.assertEquals(1, result.size()); + assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); - Assert.assertEquals(3, result.size()); + assertThat(result).hasSize(3); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index ff14bc406276..01bab6d063fd 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +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.util.List; @@ -29,27 +32,30 @@ import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -public class TestFlinkTableSource extends FlinkTestBase { +public class TestFlinkTableSource extends TestBase { private static final String CATALOG_NAME = "test_catalog"; private static final String DATABASE_NAME = "test_db"; private static final String TABLE_NAME = "test_table"; private final FileFormat format = FileFormat.AVRO; - private static String warehouse; private int scanEventCount = 0; private ScanEvent lastScanEvent = null; - public TestFlinkTableSource() { + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { // register a scan event listener to validate pushdown Listeners.register( event -> { @@ -57,24 +63,11 @@ public TestFlinkTableSource() { lastScanEvent = event; }, ScanEvent.class); - } - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - // before variables - warehouse = "file:" + warehouseFile; - } + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); - @Before - public void before() { sql( "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", CATALOG_NAME, warehouse); @@ -92,7 +85,7 @@ public void before() { this.lastScanEvent = null; } - @After + @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); @@ -102,16 +95,15 @@ public void clean() { @Test public void testLimitPushDown() { - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) .isInstanceOf(SqlParserException.class) .hasMessageStartingWith("SQL parse failed."); - Assert.assertEquals( - "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); + assertThat(sql("SELECT * FROM %s LIMIT 0", TABLE_NAME)).isEmpty(); String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); List resultExceed = sql(sqlLimitExceed); - Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); + assertThat(resultExceed).hasSize(3); List expectedList = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedList, resultExceed); @@ -119,16 +111,14 @@ public void testLimitPushDown() { String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); String explain = getTableEnv().explainSql(querySql); String expectedExplain = "limit=[1]"; - Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); + assertThat(explain).as("Explain should contain LimitPushDown").contains(expectedExplain); List result = sql(querySql); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assertions.assertThat(result).containsAnyElementsOf(expectedList); + assertThat(result).hasSize(1); + assertThat(result).containsAnyElementsOf(expectedList); String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); List mixedResult = sql(sqlMixed); - Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); - Assert.assertEquals( - "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); + assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } @Test @@ -138,8 +128,9 @@ public void testNoFilterPushDown() { List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test @@ -148,13 +139,12 @@ public void testFilterPushDownEqual() { String expectedFilter = "ref(name=\"id\") == 1"; List result = sql(sqlLiteralRight); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -162,8 +152,8 @@ public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); List result = sql(sqlEqualNull); - Assert.assertEquals("Should have 0 record", 0, result.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(result).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -172,13 +162,12 @@ public void testFilterPushDownEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") == 1"; List resultLeft = sql(sqlLiteralLeft); - Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(resultLeft).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -187,13 +176,15 @@ public void testFilterPushDownNoEqual() { String expectedFilter = "ref(name=\"id\") != 1"; List resultNE = sql(sqlNE); - Assert.assertEquals("Should have 2 records", 2, resultNE.size()); + assertThat(resultNE).hasSize(2); List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedNE, resultNE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -201,8 +192,8 @@ public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); List resultNE = sql(sqlNotEqualNull); - Assert.assertEquals("Should have 0 records", 0, resultNE.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultNE).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -211,14 +202,13 @@ public void testFilterPushDownAnd() { String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); List resultAnd = sql(sqlAnd); - Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(resultAnd).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); } @Test @@ -227,14 +217,16 @@ public void testFilterPushDownOr() { String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; List resultOr = sql(sqlOr); - Assert.assertEquals("Should have 2 record", 2, resultOr.size()); + assertThat(resultOr).hasSize(2); List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedOR, resultOr); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -243,14 +235,16 @@ public void testFilterPushDownGreaterThan() { String expectedFilter = "ref(name=\"id\") > 1"; List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 record", 2, resultGT.size()); + assertThat(resultGT).hasSize(2); List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedGT, resultGT); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -258,8 +252,8 @@ public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -268,14 +262,16 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") < 3"; List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 records", 2, resultGT.size()); + assertThat(resultGT).hasSize(2); List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedGT, resultGT); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -284,14 +280,16 @@ public void testFilterPushDownGreaterThanEqual() { String expectedFilter = "ref(name=\"id\") >= 2"; List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + assertThat(resultGTE).hasSize(2); List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedGTE, resultGTE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -299,8 +297,8 @@ public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); List resultGT = sql(sqlGTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -309,14 +307,16 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") <= 2"; List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + assertThat(resultGTE).hasSize(2); List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedGTE, resultGTE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -325,13 +325,13 @@ public void testFilterPushDownLessThan() { String expectedFilter = "ref(name=\"id\") < 2"; List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -339,8 +339,8 @@ public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); List resultGT = sql(sqlLT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -349,13 +349,13 @@ public void testFilterPushDownLessThanLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") > 2"; List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -364,13 +364,13 @@ public void testFilterPushDownLessThanEqual() { String expectedFilter = "ref(name=\"id\") <= 1"; List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -378,8 +378,8 @@ public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); List resultGT = sql(sqlLTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -388,13 +388,13 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") >= 3"; List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -402,13 +402,15 @@ public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; List resultIN = sql(sqlIN); - Assert.assertEquals("Should have 2 records", 2, resultIN.size()); + assertThat(resultIN).hasSize(2); List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedIN, resultIN); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -417,15 +419,15 @@ public void testFilterPushDownInNull() { String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); List result = sql(sqlInNull); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it // and push the rest down. String expectedScan = "ref(name=\"data\") == \"iceberg\""; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test @@ -433,23 +435,24 @@ public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); List resultNotIn = sql(sqlNotIn); - Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(resultNotIn).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull( - "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.", - lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent) + .as( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.") + .isNull(); } @Test @@ -458,14 +461,16 @@ public void testFilterPushDownIsNotNull() { String expectedFilter = "not_null(ref(name=\"data\"))"; List resultNotNull = sql(sqlNotNull); - Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); + assertThat(resultNotNull).hasSize(2); List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expected, resultNotNull); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -474,13 +479,13 @@ public void testFilterPushDownIsNull() { String expectedFilter = "is_null(ref(name=\"data\"))"; List resultNull = sql(sqlNull); - Assert.assertEquals("Should have 1 record", 1, resultNull.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); + assertThat(resultNull).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -488,14 +493,14 @@ public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); List resultNot = sql(sqlNot); - Assert.assertEquals("Should have 1 record", 1, resultNot.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); + assertThat(resultNot).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(scanEventCount).isEqualTo(1); String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -503,16 +508,18 @@ public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); List resultBetween = sql(sqlBetween); - Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); + assertThat(resultBetween).hasSize(2); List expectedBetween = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedBetween, resultBetween); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(scanEventCount).isEqualTo(1); String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); } @Test @@ -522,13 +529,13 @@ public void testFilterPushDownNotBetween() { String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; List resultNotBetween = sql(sqlNotBetween); - Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); + assertThat(resultNotBetween).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -537,25 +544,25 @@ public void testFilterPushDownLike() { String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; List resultLike = sql(sqlLike); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals( - "The like result should produce the expected record", - Row.of(1, "iceberg", 10.0), - resultLike.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(resultLike).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); // %% won't match the row with null value sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; resultLike = sql(sqlLike); - Assert.assertEquals("Should have 2 records", 2, resultLike.size()); + assertThat(resultLike).hasSize(2); List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedRecords, resultLike); String expectedScan = "not_null(ref(name=\"data\"))"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test @@ -563,37 +570,38 @@ public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; List resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 0 record", 0, resultLike.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).isEmpty(); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test @@ -603,8 +611,9 @@ public void testFilterPushDown2Literal() { List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test diff --git a/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java index 65cfba1ec876..0d9bbf9d3601 100644 --- a/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java +++ b/flink/v1.18/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java @@ -18,12 +18,4 @@ */ package org.apache.iceberg.flink; -import java.util.Map; - -public class IcebergConnectorSmokeTest extends TestIcebergConnector { - - public IcebergConnectorSmokeTest( - String catalogName, Map properties, boolean isStreaming) { - super(catalogName, properties, isStreaming); - } -} +public class IcebergConnectorSmokeTest extends TestIcebergConnector {} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java deleted file mode 100644 index 8076e0ec76f8..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ /dev/null @@ -1,129 +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; - -import java.util.List; -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.test.util.MiniClusterWithClientResource; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; - -public abstract class FlinkTestBase extends TestBaseUtils { - - @ClassRule - public static MiniClusterWithClientResource miniClusterResource = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeClass - public static void startMetastore() { - FlinkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - FlinkTestBase.hiveConf = metastore.hiveConf(); - FlinkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterClass - public static void stopMetastore() throws Exception { - metastore.stop(); - FlinkTestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java index 384ac5c52d00..e8f65921c19a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java @@ -19,6 +19,8 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.CatalogProperties.URI; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.entry; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -37,33 +39,31 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** Test for {@link CatalogLoader}. */ -public class TestCatalogLoader extends FlinkTestBase { +public class TestCatalogLoader extends TestBase { private static File warehouse = null; private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - @BeforeClass + @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); + assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } - @AfterClass + @AfterAll public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); } } @@ -96,11 +96,9 @@ private static void validateCatalogLoader(CatalogLoader loader) private static void validateHadoopConf(Table table) { FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + assertThat(hadoopIO.conf()).contains(entry("my_key", "my_value")); } @SuppressWarnings("unchecked") diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java index 8e9066e391c9..7fc6ab82490d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -25,6 +27,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -37,22 +42,18 @@ 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.Sets; import org.apache.iceberg.util.StructLikeSet; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +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; /** * In this test case, we mainly cover the impact of primary key selection, multiple operations * within a single transaction, and multiple operations between different txn on the correctness of * the data. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestChangeLogTable extends ChangeLogTableTestBase { private static final Configuration CONF = new Configuration(); private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; @@ -62,26 +63,19 @@ public class TestChangeLogTable extends ChangeLogTableTestBase { private static final String TABLE_NAME = "test_table"; private static String warehouse; - private final boolean partitioned; + @Parameter private boolean partitioned; - @Parameterized.Parameters(name = "PartitionedTable={0}") + @Parameters(name = "PartitionedTable={0}") public static Iterable parameters() { return ImmutableList.of(new Object[] {true}, new Object[] {false}); } - public TestChangeLogTable(boolean partitioned) { - this.partitioned = partitioned; - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + @BeforeEach + public void before() throws IOException { + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); warehouse = String.format("file:%s", warehouseFile); - } - @Before - public void before() { sql( "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", CATALOG_NAME, warehouse); @@ -94,7 +88,7 @@ public void before() { getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); } - @After + @AfterEach @Override public void clean() { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); @@ -103,7 +97,7 @@ public void clean() { BoundedTableFactory.clearDataSets(); } - @Test + @TestTemplate public void testSqlChangeLogOnIdKey() throws Exception { List> inputRowsPerCheckpoint = ImmutableList.of( @@ -135,7 +129,7 @@ public void testSqlChangeLogOnIdKey() throws Exception { TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -162,7 +156,7 @@ public void testChangeLogOnDataKey() throws Exception { testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -191,7 +185,7 @@ public void testChangeLogOnIdDataKey() throws Exception { TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); } - @Test + @TestTemplate public void testPureInsertOnIdKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -249,10 +243,7 @@ private void testSqlChangeLog( + " WITH ('connector'='BoundedSource', 'data-id'='%s')", SOURCE_TABLE, dataId); - Assert.assertEquals( - "Should have the expected rows", - listJoin(inputRowsPerCheckpoint), - sql("SELECT * FROM %s", SOURCE_TABLE)); + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)).isEqualTo(listJoin(inputRowsPerCheckpoint)); Table table = createTable(tableName, key, partitioned); sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); @@ -260,23 +251,23 @@ private void testSqlChangeLog( table.refresh(); List snapshots = findValidSnapshots(table); int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + assertThat(snapshots) + .as("Should have the expected snapshot number") + .hasSameSizeAs(expectedRecordsPerCheckpoint); for (int i = 0; i < expectedSnapshotNum; i++) { long snapshotId = snapshots.get(i).snapshotId(); List expectedRows = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(table, expectedRows), - actualRowSet(table, snapshotId)); + assertThat(actualRowSet(table, snapshotId)) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(table, expectedRows)); } if (expectedSnapshotNum > 0) { - Assert.assertEquals( - "Should have the expected rows in the final table", - Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), - Sets.newHashSet(sql("SELECT * FROM %s", tableName))); + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in the final table") + .containsExactlyInAnyOrderElementsOf( + expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java index 2fbd7627efab..70c8043f8fbb 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -18,22 +18,24 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; +import java.nio.file.Files; import java.util.concurrent.TimeUnit; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableDescriptor; import org.apache.flink.table.api.TableEnvironment; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Test; +import org.junit.jupiter.api.Test; -public class TestFlinkAnonymousTable extends FlinkTestBase { +public class TestFlinkAnonymousTable extends TestBase { @Test public void testWriteAnonymousTable() throws Exception { - File warehouseDir = TEMPORARY_FOLDER.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); TableEnvironment tEnv = getTableEnv(); Table table = tEnv.from( @@ -57,8 +59,7 @@ public void testWriteAnonymousTable() throws Exception { .atMost(3, TimeUnit.SECONDS) .untilAsserted( () -> - Assertions.assertThat( - warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + assertThat(warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) .exists()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java index 47ee2afceb02..f1de267cf29b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -28,14 +30,9 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkHiveCatalog extends FlinkTestBase { +import org.junit.jupiter.api.Test; - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); +public class TestFlinkHiveCatalog extends TestBase { @Test public void testCreateCatalogWithWarehouseLocation() throws IOException { @@ -44,7 +41,7 @@ public void testCreateCatalogWithWarehouseLocation() throws IOException { props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - File warehouseDir = tempFolder.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); checkSQLQuery(props, warehouseDir); @@ -53,9 +50,9 @@ public void testCreateCatalogWithWarehouseLocation() throws IOException { @Test public void testCreateCatalogWithHiveConfDir() throws IOException { // Dump the hive conf into a local file. - File hiveConfDir = tempFolder.newFolder(); + File hiveConfDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); - File warehouseDir = tempFolder.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { Configuration newConf = new Configuration(hiveConf); // Set another new directory which is different with the hive metastore's warehouse path. @@ -63,7 +60,7 @@ public void testCreateCatalogWithHiveConfDir() throws IOException { HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); newConf.writeXml(fos); } - Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); + assertThat(hiveSiteXML.toPath()).exists(); // Construct the catalog attributions. Map props = Maps.newHashMap(); @@ -86,15 +83,16 @@ private void checkSQLQuery(Map catalogProperties, File warehouse sql("INSERT INTO test_table SELECT 1, 'a'"); Path databasePath = warehouseDir.toPath().resolve("test_db.db"); - Assert.assertTrue("Database path should exist", Files.exists(databasePath)); + assertThat(databasePath).exists(); Path tablePath = databasePath.resolve("test_table"); - Assert.assertTrue("Table path should exist", Files.exists(tablePath)); + assertThat(tablePath).exists(); Path dataPath = tablePath.resolve("data"); - Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); - Assert.assertEquals( - "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); + assertThat(dataPath).exists(); + assertThat(Files.list(dataPath).count()) + .as("Should have a .crc file and a .parquet file") + .isEqualTo(2); sql("DROP TABLE test_table"); sql("DROP DATABASE test_db"); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index cb409b784383..013b98e3b82b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -18,8 +18,12 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Files; import java.util.Map; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -32,33 +36,34 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; 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.relocated.com.google.common.collect.Sets; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergConnector extends FlinkTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergConnector extends TestBase { private static final String TABLE_NAME = "test_table"; - @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); + @Parameter(index = 0) + private String catalogName; + + @Parameter(index = 1) + private Map properties; + + @Parameter(index = 2) + private boolean isStreaming; - private final String catalogName; - private final Map properties; - private final boolean isStreaming; private volatile TableEnvironment tEnv; - @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") + @Parameters(name = "catalogName = {0}, properties = {1}, isStreaming = {2}") public static Iterable parameters() { return Lists.newArrayList( // Create iceberg table in the hadoop catalog and default database. @@ -161,13 +166,6 @@ public static Iterable parameters() { }); } - public TestIcebergConnector( - String catalogName, Map properties, boolean isStreaming) { - this.catalogName = catalogName; - this.properties = properties; - this.isStreaming = isStreaming; - } - @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { @@ -198,7 +196,7 @@ protected TableEnvironment getTableEnv() { return tEnv; } - @After + @AfterEach public void after() throws TException { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); @@ -226,34 +224,27 @@ private void testCreateConnectorTable() { // Create table under the flink's current database. sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); FlinkCatalogFactory factory = new FlinkCatalogFactory(); Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); - Assert.assertTrue( - "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); - Assert.assertTrue( - "Should have created the expected table", - flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); + assertThat(flinkCatalog.databaseExists(databaseName())).isTrue(); + assertThat(flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))).isTrue(); // Drop and create it again. sql("DROP TABLE %s", TABLE_NAME); sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); } - @Test + @TestTemplate public void testCreateTableUnderDefaultDatabase() { testCreateConnectorTable(); } - @Test + @TestTemplate public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); @@ -261,7 +252,7 @@ public void testCatalogDatabaseConflictWithFlinkDatabase() { try { testCreateConnectorTable(); // Ensure that the table was created under the specific database. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) .isInstanceOf(org.apache.flink.table.api.TableException.class) .hasMessageStartingWith("Could not execute CreateTable in path"); @@ -273,7 +264,7 @@ public void testCatalogDatabaseConflictWithFlinkDatabase() { } } - @Test + @TestTemplate public void testConnectorTableInIcebergCatalog() { // Create the catalog properties Map catalogProps = Maps.newHashMap(); @@ -292,7 +283,7 @@ public void testConnectorTableInIcebergCatalog() { // Create a connector table in an iceberg catalog. sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", @@ -340,9 +331,11 @@ private String toWithClause(Map props) { return CatalogTestBase.toWithClause(props); } - private static String createWarehouse() { + private String createWarehouse() { try { - return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); + return String.format( + "file://%s", + Files.createTempDirectory(temporaryDirectory, "junit").toFile().getAbsolutePath()); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index 7aa2b8034bc5..d3748e008b8e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -26,20 +28,26 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.FlinkTestBase; import org.apache.iceberg.flink.MiniClusterResource; -import org.junit.After; -import org.junit.Rule; -import org.junit.rules.TestName; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; -public class ChangeLogTableTestBase extends FlinkTestBase { +public class ChangeLogTableTestBase extends TestBase { private volatile TableEnvironment tEnv = null; - @Rule public TestName name = new TestName(); + protected String tableName; + + @BeforeEach + public void setup(TestInfo testInfo) { + assertThat(testInfo.getTestMethod()).isPresent(); + this.tableName = testInfo.getTestMethod().get().getName(); + } - @After + @AfterEach public void clean() { - sql("DROP TABLE IF EXISTS %s", name.getMethodName()); + sql("DROP TABLE IF EXISTS %s", tableName); BoundedTableFactory.clearDataSets(); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java index 7b5f9328694c..bde751e1f87f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java @@ -18,36 +18,32 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Objects; import java.util.stream.Collectors; import org.apache.flink.types.Row; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestBoundedTableFactory extends ChangeLogTableTestBase { @Test public void testEmptyDataSet() { - String table = name.getMethodName(); List> emptyDataSet = ImmutableList.of(); String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); sql( "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); + tableName, dataId); - Assert.assertEquals( - "Should have caught empty change log set.", - ImmutableList.of(), - sql("SELECT * FROM %s", table)); + assertThat(sql("SELECT * FROM %s", tableName)).isEmpty(); } @Test public void testBoundedTableFactory() { - String table = name.getMethodName(); List> dataSet = ImmutableList.of( ImmutableList.of( @@ -71,17 +67,15 @@ public void testBoundedTableFactory() { String dataId = BoundedTableFactory.registerDataSet(dataSet); sql( "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); + tableName, dataId); List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); - Assert.assertEquals( - "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); + assertThat(sql("SELECT * FROM %s", tableName)).isEqualTo(rowSet); - Assert.assertEquals( - "Should have the expected change log events", - rowSet.stream() - .filter(r -> Objects.equals(r.getField(1), "aaa")) - .collect(Collectors.toList()), - sql("SELECT * FROM %s WHERE data='aaa'", table)); + assertThat(sql("SELECT * FROM %s WHERE data='aaa'", tableName)) + .isEqualTo( + rowSet.stream() + .filter(r -> Objects.equals(r.getField(1), "aaa")) + .collect(Collectors.toList())); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 1814ff8f8542..bc7194e38088 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkSourceConfig extends TestFlinkTableSource { private static final String TABLE = "test_table"; @@ -31,8 +32,7 @@ public class TestFlinkSourceConfig extends TestFlinkTableSource { @Test public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) + assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } @@ -43,16 +43,16 @@ public void testFlinkHintConfig() { sql( "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", TABLE, System.currentTimeMillis()); - Assert.assertEquals(3, result.size()); + assertThat(result).hasSize(3); } @Test public void testReadOptionHierarchy() { getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); - Assert.assertEquals(1, result.size()); + assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); - Assert.assertEquals(3, result.size()); + assertThat(result).hasSize(3); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index ff14bc406276..01bab6d063fd 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +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.util.List; @@ -29,27 +32,30 @@ import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -public class TestFlinkTableSource extends FlinkTestBase { +public class TestFlinkTableSource extends TestBase { private static final String CATALOG_NAME = "test_catalog"; private static final String DATABASE_NAME = "test_db"; private static final String TABLE_NAME = "test_table"; private final FileFormat format = FileFormat.AVRO; - private static String warehouse; private int scanEventCount = 0; private ScanEvent lastScanEvent = null; - public TestFlinkTableSource() { + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { // register a scan event listener to validate pushdown Listeners.register( event -> { @@ -57,24 +63,11 @@ public TestFlinkTableSource() { lastScanEvent = event; }, ScanEvent.class); - } - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - // before variables - warehouse = "file:" + warehouseFile; - } + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); - @Before - public void before() { sql( "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", CATALOG_NAME, warehouse); @@ -92,7 +85,7 @@ public void before() { this.lastScanEvent = null; } - @After + @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); @@ -102,16 +95,15 @@ public void clean() { @Test public void testLimitPushDown() { - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) .isInstanceOf(SqlParserException.class) .hasMessageStartingWith("SQL parse failed."); - Assert.assertEquals( - "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); + assertThat(sql("SELECT * FROM %s LIMIT 0", TABLE_NAME)).isEmpty(); String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); List resultExceed = sql(sqlLimitExceed); - Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); + assertThat(resultExceed).hasSize(3); List expectedList = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedList, resultExceed); @@ -119,16 +111,14 @@ public void testLimitPushDown() { String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); String explain = getTableEnv().explainSql(querySql); String expectedExplain = "limit=[1]"; - Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); + assertThat(explain).as("Explain should contain LimitPushDown").contains(expectedExplain); List result = sql(querySql); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assertions.assertThat(result).containsAnyElementsOf(expectedList); + assertThat(result).hasSize(1); + assertThat(result).containsAnyElementsOf(expectedList); String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); List mixedResult = sql(sqlMixed); - Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); - Assert.assertEquals( - "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); + assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } @Test @@ -138,8 +128,9 @@ public void testNoFilterPushDown() { List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test @@ -148,13 +139,12 @@ public void testFilterPushDownEqual() { String expectedFilter = "ref(name=\"id\") == 1"; List result = sql(sqlLiteralRight); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -162,8 +152,8 @@ public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); List result = sql(sqlEqualNull); - Assert.assertEquals("Should have 0 record", 0, result.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(result).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -172,13 +162,12 @@ public void testFilterPushDownEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") == 1"; List resultLeft = sql(sqlLiteralLeft); - Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(resultLeft).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -187,13 +176,15 @@ public void testFilterPushDownNoEqual() { String expectedFilter = "ref(name=\"id\") != 1"; List resultNE = sql(sqlNE); - Assert.assertEquals("Should have 2 records", 2, resultNE.size()); + assertThat(resultNE).hasSize(2); List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedNE, resultNE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -201,8 +192,8 @@ public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); List resultNE = sql(sqlNotEqualNull); - Assert.assertEquals("Should have 0 records", 0, resultNE.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultNE).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -211,14 +202,13 @@ public void testFilterPushDownAnd() { String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); List resultAnd = sql(sqlAnd); - Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(resultAnd).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); } @Test @@ -227,14 +217,16 @@ public void testFilterPushDownOr() { String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; List resultOr = sql(sqlOr); - Assert.assertEquals("Should have 2 record", 2, resultOr.size()); + assertThat(resultOr).hasSize(2); List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedOR, resultOr); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -243,14 +235,16 @@ public void testFilterPushDownGreaterThan() { String expectedFilter = "ref(name=\"id\") > 1"; List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 record", 2, resultGT.size()); + assertThat(resultGT).hasSize(2); List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedGT, resultGT); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -258,8 +252,8 @@ public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -268,14 +262,16 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") < 3"; List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 records", 2, resultGT.size()); + assertThat(resultGT).hasSize(2); List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedGT, resultGT); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -284,14 +280,16 @@ public void testFilterPushDownGreaterThanEqual() { String expectedFilter = "ref(name=\"id\") >= 2"; List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + assertThat(resultGTE).hasSize(2); List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedGTE, resultGTE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -299,8 +297,8 @@ public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); List resultGT = sql(sqlGTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -309,14 +307,16 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") <= 2"; List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + assertThat(resultGTE).hasSize(2); List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedGTE, resultGTE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -325,13 +325,13 @@ public void testFilterPushDownLessThan() { String expectedFilter = "ref(name=\"id\") < 2"; List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -339,8 +339,8 @@ public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); List resultGT = sql(sqlLT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -349,13 +349,13 @@ public void testFilterPushDownLessThanLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") > 2"; List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -364,13 +364,13 @@ public void testFilterPushDownLessThanEqual() { String expectedFilter = "ref(name=\"id\") <= 1"; List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -378,8 +378,8 @@ public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); List resultGT = sql(sqlLTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -388,13 +388,13 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") >= 3"; List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -402,13 +402,15 @@ public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; List resultIN = sql(sqlIN); - Assert.assertEquals("Should have 2 records", 2, resultIN.size()); + assertThat(resultIN).hasSize(2); List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedIN, resultIN); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -417,15 +419,15 @@ public void testFilterPushDownInNull() { String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); List result = sql(sqlInNull); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it // and push the rest down. String expectedScan = "ref(name=\"data\") == \"iceberg\""; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test @@ -433,23 +435,24 @@ public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); List resultNotIn = sql(sqlNotIn); - Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(resultNotIn).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull( - "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.", - lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent) + .as( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.") + .isNull(); } @Test @@ -458,14 +461,16 @@ public void testFilterPushDownIsNotNull() { String expectedFilter = "not_null(ref(name=\"data\"))"; List resultNotNull = sql(sqlNotNull); - Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); + assertThat(resultNotNull).hasSize(2); List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expected, resultNotNull); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -474,13 +479,13 @@ public void testFilterPushDownIsNull() { String expectedFilter = "is_null(ref(name=\"data\"))"; List resultNull = sql(sqlNull); - Assert.assertEquals("Should have 1 record", 1, resultNull.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); + assertThat(resultNull).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -488,14 +493,14 @@ public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); List resultNot = sql(sqlNot); - Assert.assertEquals("Should have 1 record", 1, resultNot.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); + assertThat(resultNot).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(scanEventCount).isEqualTo(1); String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -503,16 +508,18 @@ public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); List resultBetween = sql(sqlBetween); - Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); + assertThat(resultBetween).hasSize(2); List expectedBetween = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedBetween, resultBetween); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(scanEventCount).isEqualTo(1); String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); } @Test @@ -522,13 +529,13 @@ public void testFilterPushDownNotBetween() { String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; List resultNotBetween = sql(sqlNotBetween); - Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); + assertThat(resultNotBetween).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -537,25 +544,25 @@ public void testFilterPushDownLike() { String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; List resultLike = sql(sqlLike); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals( - "The like result should produce the expected record", - Row.of(1, "iceberg", 10.0), - resultLike.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(resultLike).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); // %% won't match the row with null value sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; resultLike = sql(sqlLike); - Assert.assertEquals("Should have 2 records", 2, resultLike.size()); + assertThat(resultLike).hasSize(2); List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedRecords, resultLike); String expectedScan = "not_null(ref(name=\"data\"))"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test @@ -563,37 +570,38 @@ public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; List resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 0 record", 0, resultLike.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).isEmpty(); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test @@ -603,8 +611,9 @@ public void testFilterPushDown2Literal() { List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test diff --git a/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java index 65cfba1ec876..0d9bbf9d3601 100644 --- a/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java +++ b/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java @@ -18,12 +18,4 @@ */ package org.apache.iceberg.flink; -import java.util.Map; - -public class IcebergConnectorSmokeTest extends TestIcebergConnector { - - public IcebergConnectorSmokeTest( - String catalogName, Map properties, boolean isStreaming) { - super(catalogName, properties, isStreaming); - } -} +public class IcebergConnectorSmokeTest extends TestIcebergConnector {} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java deleted file mode 100644 index 0b7d19f27cc0..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/FlinkTestBase.java +++ /dev/null @@ -1,147 +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; - -import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; - -import java.util.List; -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.test.util.MiniClusterWithClientResource; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.rules.TemporaryFolder; - -public abstract class FlinkTestBase extends TestBaseUtils { - - @ClassRule - public static MiniClusterWithClientResource miniClusterResource = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeClass - public static void startMetastore() { - FlinkTestBase.metastore = new TestHiveMetastore(); - metastore.start(); - FlinkTestBase.hiveConf = metastore.hiveConf(); - FlinkTestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterClass - public static void stopMetastore() throws Exception { - metastore.stop(); - FlinkTestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } - - /** - * We can not drop currently used database after FLINK-33226, so we have make sure that we do not - * use the current database before dropping it. This method switches to the default database in - * the default catalog, and then it and drops the one requested. - * - * @param database The database to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the database - */ - protected void dropDatabase(String database, boolean ifExists) { - String currentCatalog = getTableEnv().getCurrentCatalog(); - sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); - sql("USE %s", getTableEnv().listDatabases()[0]); - sql("USE CATALOG %s", currentCatalog); - sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java index 384ac5c52d00..e8f65921c19a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java @@ -19,6 +19,8 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.CatalogProperties.URI; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.entry; import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; @@ -37,33 +39,31 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; -import org.junit.AfterClass; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; /** Test for {@link CatalogLoader}. */ -public class TestCatalogLoader extends FlinkTestBase { +public class TestCatalogLoader extends TestBase { private static File warehouse = null; private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); private static final Schema SCHEMA = new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - @BeforeClass + @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assert.assertTrue(warehouse.delete()); + assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } - @AfterClass + @AfterAll public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assert.assertTrue("Failed to delete " + warehousePath, fs.delete(warehousePath, true)); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); } } @@ -96,11 +96,9 @@ private static void validateCatalogLoader(CatalogLoader loader) private static void validateHadoopConf(Table table) { FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assert.assertEquals("my_value", hadoopIO.conf().get("my_key")); + assertThat(hadoopIO.conf()).contains(entry("my_key", "my_value")); } @SuppressWarnings("unchecked") diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java index d9f01796fb44..1997ef6998a2 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -25,6 +27,9 @@ import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.BaseTable; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; @@ -37,22 +42,18 @@ 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.Sets; import org.apache.iceberg.util.StructLikeSet; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +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; /** * In this test case, we mainly cover the impact of primary key selection, multiple operations * within a single transaction, and multiple operations between different txn on the correctness of * the data. */ -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestChangeLogTable extends ChangeLogTableTestBase { private static final Configuration CONF = new Configuration(); private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; @@ -60,28 +61,21 @@ public class TestChangeLogTable extends ChangeLogTableTestBase { private static final String CATALOG_NAME = "test_catalog"; private static final String DATABASE_NAME = "test_db"; private static final String TABLE_NAME = "test_table"; - private static String warehouse; + private String warehouse; - private final boolean partitioned; + @Parameter private boolean partitioned; - @Parameterized.Parameters(name = "PartitionedTable={0}") + @Parameters(name = "PartitionedTable={0}") public static Iterable parameters() { return ImmutableList.of(new Object[] {true}, new Object[] {false}); } - public TestChangeLogTable(boolean partitioned) { - this.partitioned = partitioned; - } - - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); + @BeforeEach + public void before() throws IOException { + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); warehouse = String.format("file:%s", warehouseFile); - } - @Before - public void before() { sql( "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", CATALOG_NAME, warehouse); @@ -94,7 +88,7 @@ public void before() { getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); } - @After + @AfterEach @Override public void clean() { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); @@ -103,7 +97,7 @@ public void clean() { BoundedTableFactory.clearDataSets(); } - @Test + @TestTemplate public void testSqlChangeLogOnIdKey() throws Exception { List> inputRowsPerCheckpoint = ImmutableList.of( @@ -135,7 +129,7 @@ public void testSqlChangeLogOnIdKey() throws Exception { TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -162,7 +156,7 @@ public void testChangeLogOnDataKey() throws Exception { testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -191,7 +185,7 @@ public void testChangeLogOnIdDataKey() throws Exception { TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); } - @Test + @TestTemplate public void testPureInsertOnIdKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -249,10 +243,7 @@ private void testSqlChangeLog( + " WITH ('connector'='BoundedSource', 'data-id'='%s')", SOURCE_TABLE, dataId); - Assert.assertEquals( - "Should have the expected rows", - listJoin(inputRowsPerCheckpoint), - sql("SELECT * FROM %s", SOURCE_TABLE)); + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)).isEqualTo(listJoin(inputRowsPerCheckpoint)); Table table = createTable(tableName, key, partitioned); sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); @@ -260,23 +251,23 @@ private void testSqlChangeLog( table.refresh(); List snapshots = findValidSnapshots(table); int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + assertThat(snapshots) + .as("Should have the expected snapshot number") + .hasSameSizeAs(expectedRecordsPerCheckpoint); for (int i = 0; i < expectedSnapshotNum; i++) { long snapshotId = snapshots.get(i).snapshotId(); List expectedRows = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(table, expectedRows), - actualRowSet(table, snapshotId)); + assertThat(actualRowSet(table, snapshotId)) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(table, expectedRows)); } if (expectedSnapshotNum > 0) { - Assert.assertEquals( - "Should have the expected rows in the final table", - Sets.newHashSet(expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)), - Sets.newHashSet(sql("SELECT * FROM %s", tableName))); + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in the final table") + .containsExactlyInAnyOrderElementsOf( + expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java index 2fbd7627efab..70c8043f8fbb 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java @@ -18,22 +18,24 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; +import java.nio.file.Files; import java.util.concurrent.TimeUnit; import org.apache.flink.table.api.DataTypes; import org.apache.flink.table.api.Schema; import org.apache.flink.table.api.Table; import org.apache.flink.table.api.TableDescriptor; import org.apache.flink.table.api.TableEnvironment; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; -import org.junit.Test; +import org.junit.jupiter.api.Test; -public class TestFlinkAnonymousTable extends FlinkTestBase { +public class TestFlinkAnonymousTable extends TestBase { @Test public void testWriteAnonymousTable() throws Exception { - File warehouseDir = TEMPORARY_FOLDER.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); TableEnvironment tEnv = getTableEnv(); Table table = tEnv.from( @@ -57,8 +59,7 @@ public void testWriteAnonymousTable() throws Exception { .atMost(3, TimeUnit.SECONDS) .untilAsserted( () -> - Assertions.assertThat( - warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + assertThat(warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) .exists()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java index dd643520136c..91343ab1ee72 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.FileOutputStream; import java.io.IOException; @@ -28,14 +30,9 @@ import org.apache.hadoop.hive.conf.HiveConf; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; - -public class TestFlinkHiveCatalog extends FlinkTestBase { +import org.junit.jupiter.api.Test; - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); +public class TestFlinkHiveCatalog extends TestBase { @Test public void testCreateCatalogWithWarehouseLocation() throws IOException { @@ -44,7 +41,7 @@ public void testCreateCatalogWithWarehouseLocation() throws IOException { props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - File warehouseDir = tempFolder.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); checkSQLQuery(props, warehouseDir); @@ -53,9 +50,9 @@ public void testCreateCatalogWithWarehouseLocation() throws IOException { @Test public void testCreateCatalogWithHiveConfDir() throws IOException { // Dump the hive conf into a local file. - File hiveConfDir = tempFolder.newFolder(); + File hiveConfDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); - File warehouseDir = tempFolder.newFolder(); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { Configuration newConf = new Configuration(hiveConf); // Set another new directory which is different with the hive metastore's warehouse path. @@ -63,7 +60,7 @@ public void testCreateCatalogWithHiveConfDir() throws IOException { HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); newConf.writeXml(fos); } - Assert.assertTrue("hive-site.xml should be created now.", Files.exists(hiveSiteXML.toPath())); + assertThat(hiveSiteXML.toPath()).exists(); // Construct the catalog attributions. Map props = Maps.newHashMap(); @@ -86,15 +83,16 @@ private void checkSQLQuery(Map catalogProperties, File warehouse sql("INSERT INTO test_table SELECT 1, 'a'"); Path databasePath = warehouseDir.toPath().resolve("test_db.db"); - Assert.assertTrue("Database path should exist", Files.exists(databasePath)); + assertThat(databasePath).exists(); Path tablePath = databasePath.resolve("test_table"); - Assert.assertTrue("Table path should exist", Files.exists(tablePath)); + assertThat(tablePath).exists(); Path dataPath = tablePath.resolve("data"); - Assert.assertTrue("Table data path should exist", Files.exists(dataPath)); - Assert.assertEquals( - "Should have a .crc file and a .parquet file", 2, Files.list(dataPath).count()); + assertThat(dataPath).exists(); + assertThat(Files.list(dataPath).count()) + .as("Should have a .crc file and a .parquet file") + .isEqualTo(2); sql("DROP TABLE test_table"); dropDatabase("test_db", false); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 632997e357a9..c978ec6f8bd4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -18,8 +18,12 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.file.Files; import java.util.Map; import org.apache.flink.configuration.CoreOptions; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -32,33 +36,34 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; 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.relocated.com.google.common.collect.Sets; import org.apache.thrift.TException; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) -public class TestIcebergConnector extends FlinkTestBase { +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergConnector extends TestBase { private static final String TABLE_NAME = "test_table"; - @ClassRule public static final TemporaryFolder WAREHOUSE = new TemporaryFolder(); + @Parameter(index = 0) + private String catalogName; + + @Parameter(index = 1) + private Map properties; + + @Parameter(index = 2) + private boolean isStreaming; - private final String catalogName; - private final Map properties; - private final boolean isStreaming; private volatile TableEnvironment tEnv; - @Parameterized.Parameters(name = "catalogName = {0}, properties = {1}, isStreaming={2}") + @Parameters(name = "catalogName = {0}, properties = {1}, isStreaming = {2}") public static Iterable parameters() { return Lists.newArrayList( // Create iceberg table in the hadoop catalog and default database. @@ -161,13 +166,6 @@ public static Iterable parameters() { }); } - public TestIcebergConnector( - String catalogName, Map properties, boolean isStreaming) { - this.catalogName = catalogName; - this.properties = properties; - this.isStreaming = isStreaming; - } - @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { @@ -198,7 +196,7 @@ protected TableEnvironment getTableEnv() { return tEnv; } - @After + @AfterEach public void after() throws TException { sql("DROP TABLE IF EXISTS %s", TABLE_NAME); @@ -226,46 +224,39 @@ private void testCreateConnectorTable() { // Create table under the flink's current database. sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); FlinkCatalogFactory factory = new FlinkCatalogFactory(); Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); - Assert.assertTrue( - "Should have created the expected database", flinkCatalog.databaseExists(databaseName())); - Assert.assertTrue( - "Should have created the expected table", - flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))); + assertThat(flinkCatalog.databaseExists(databaseName())).isTrue(); + assertThat(flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))).isTrue(); // Drop and create it again. sql("DROP TABLE %s", TABLE_NAME); sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - Assert.assertEquals( - "Should have expected rows", - Sets.newHashSet(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")), - Sets.newHashSet(sql("SELECT * FROM %s", TABLE_NAME))); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); } - @Test + @TestTemplate public void testCreateTableUnderDefaultDatabase() { testCreateConnectorTable(); } - @Test + @TestTemplate public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); testCreateConnectorTable(); // Ensure that the table was created under the specific database. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) .isInstanceOf(org.apache.flink.table.api.TableException.class) .hasMessageStartingWith("Could not execute CreateTable in path"); } - @Test + @TestTemplate public void testConnectorTableInIcebergCatalog() { // Create the catalog properties Map catalogProps = Maps.newHashMap(); @@ -284,7 +275,7 @@ public void testConnectorTableInIcebergCatalog() { // Create a connector table in an iceberg catalog. sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); try { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", @@ -332,9 +323,11 @@ private String toWithClause(Map props) { return CatalogTestBase.toWithClause(props); } - private static String createWarehouse() { + private String createWarehouse() { try { - return String.format("file://%s", WAREHOUSE.newFolder().getAbsolutePath()); + return String.format( + "file://%s", + Files.createTempDirectory(temporaryDirectory, "junit").toFile().getAbsolutePath()); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index 7aa2b8034bc5..d3748e008b8e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; @@ -26,20 +28,26 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.FlinkTestBase; import org.apache.iceberg.flink.MiniClusterResource; -import org.junit.After; -import org.junit.Rule; -import org.junit.rules.TestName; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; -public class ChangeLogTableTestBase extends FlinkTestBase { +public class ChangeLogTableTestBase extends TestBase { private volatile TableEnvironment tEnv = null; - @Rule public TestName name = new TestName(); + protected String tableName; + + @BeforeEach + public void setup(TestInfo testInfo) { + assertThat(testInfo.getTestMethod()).isPresent(); + this.tableName = testInfo.getTestMethod().get().getName(); + } - @After + @AfterEach public void clean() { - sql("DROP TABLE IF EXISTS %s", name.getMethodName()); + sql("DROP TABLE IF EXISTS %s", tableName); BoundedTableFactory.clearDataSets(); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java index 7b5f9328694c..bde751e1f87f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java @@ -18,36 +18,32 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Objects; import java.util.stream.Collectors; import org.apache.flink.types.Row; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestBoundedTableFactory extends ChangeLogTableTestBase { @Test public void testEmptyDataSet() { - String table = name.getMethodName(); List> emptyDataSet = ImmutableList.of(); String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); sql( "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); + tableName, dataId); - Assert.assertEquals( - "Should have caught empty change log set.", - ImmutableList.of(), - sql("SELECT * FROM %s", table)); + assertThat(sql("SELECT * FROM %s", tableName)).isEmpty(); } @Test public void testBoundedTableFactory() { - String table = name.getMethodName(); List> dataSet = ImmutableList.of( ImmutableList.of( @@ -71,17 +67,15 @@ public void testBoundedTableFactory() { String dataId = BoundedTableFactory.registerDataSet(dataSet); sql( "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - table, dataId); + tableName, dataId); List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); - Assert.assertEquals( - "Should have the expected change log events.", rowSet, sql("SELECT * FROM %s", table)); + assertThat(sql("SELECT * FROM %s", tableName)).isEqualTo(rowSet); - Assert.assertEquals( - "Should have the expected change log events", - rowSet.stream() - .filter(r -> Objects.equals(r.getField(1), "aaa")) - .collect(Collectors.toList()), - sql("SELECT * FROM %s WHERE data='aaa'", table)); + assertThat(sql("SELECT * FROM %s WHERE data='aaa'", tableName)) + .isEqualTo( + rowSet.stream() + .filter(r -> Objects.equals(r.getField(1), "aaa")) + .collect(Collectors.toList())); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 1814ff8f8542..bc7194e38088 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.assertj.core.api.Assertions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFlinkSourceConfig extends TestFlinkTableSource { private static final String TABLE = "test_table"; @@ -31,8 +32,7 @@ public class TestFlinkSourceConfig extends TestFlinkTableSource { @Test public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) + assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } @@ -43,16 +43,16 @@ public void testFlinkHintConfig() { sql( "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", TABLE, System.currentTimeMillis()); - Assert.assertEquals(3, result.size()); + assertThat(result).hasSize(3); } @Test public void testReadOptionHierarchy() { getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); - Assert.assertEquals(1, result.size()); + assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); - Assert.assertEquals(3, result.size()); + assertThat(result).hasSize(3); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index e8ec482520ca..facbdcaaa533 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source; +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.util.List; @@ -29,27 +32,30 @@ import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.FlinkTestBase; +import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; -public class TestFlinkTableSource extends FlinkTestBase { +public class TestFlinkTableSource extends TestBase { private static final String CATALOG_NAME = "test_catalog"; private static final String DATABASE_NAME = "test_db"; private static final String TABLE_NAME = "test_table"; private final FileFormat format = FileFormat.AVRO; - private static String warehouse; private int scanEventCount = 0; private ScanEvent lastScanEvent = null; - public TestFlinkTableSource() { + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { // register a scan event listener to validate pushdown Listeners.register( event -> { @@ -57,24 +63,11 @@ public TestFlinkTableSource() { lastScanEvent = event; }, ScanEvent.class); - } - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - @BeforeClass - public static void createWarehouse() throws IOException { - File warehouseFile = TEMPORARY_FOLDER.newFolder(); - Assert.assertTrue("The warehouse should be deleted", warehouseFile.delete()); - // before variables - warehouse = "file:" + warehouseFile; - } + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); - @Before - public void before() { sql( "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", CATALOG_NAME, warehouse); @@ -92,7 +85,7 @@ public void before() { this.lastScanEvent = null; } - @After + @AfterEach public void clean() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); dropDatabase(DATABASE_NAME, true); @@ -102,16 +95,15 @@ public void clean() { @Test public void testLimitPushDown() { - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) .isInstanceOf(SqlParserException.class) .hasMessageStartingWith("SQL parse failed."); - Assert.assertEquals( - "Should have 0 record", 0, sql("SELECT * FROM %s LIMIT 0", TABLE_NAME).size()); + assertThat(sql("SELECT * FROM %s LIMIT 0", TABLE_NAME)).isEmpty(); String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); List resultExceed = sql(sqlLimitExceed); - Assert.assertEquals("Should have 3 records", 3, resultExceed.size()); + assertThat(resultExceed).hasSize(3); List expectedList = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedList, resultExceed); @@ -119,16 +111,14 @@ public void testLimitPushDown() { String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); String explain = getTableEnv().explainSql(querySql); String expectedExplain = "limit=[1]"; - Assert.assertTrue("Explain should contain LimitPushDown", explain.contains(expectedExplain)); + assertThat(explain).as("Explain should contain LimitPushDown").contains(expectedExplain); List result = sql(querySql); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assertions.assertThat(result).containsAnyElementsOf(expectedList); + assertThat(result).hasSize(1); + assertThat(result).containsAnyElementsOf(expectedList); String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); List mixedResult = sql(sqlMixed); - Assert.assertEquals("Should have 1 record", 1, mixedResult.size()); - Assert.assertEquals( - "Should produce the expected records", Row.of(1, "iceberg", 10.0), mixedResult.get(0)); + assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } @Test @@ -138,8 +128,9 @@ public void testNoFilterPushDown() { List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test @@ -148,13 +139,12 @@ public void testFilterPushDownEqual() { String expectedFilter = "ref(name=\"id\") == 1"; List result = sql(sqlLiteralRight); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -162,8 +152,8 @@ public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); List result = sql(sqlEqualNull); - Assert.assertEquals("Should have 0 record", 0, result.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(result).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -172,13 +162,12 @@ public void testFilterPushDownEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") == 1"; List resultLeft = sql(sqlLiteralLeft); - Assert.assertEquals("Should have 1 record", 1, resultLeft.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLeft.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(resultLeft).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -187,13 +176,15 @@ public void testFilterPushDownNoEqual() { String expectedFilter = "ref(name=\"id\") != 1"; List resultNE = sql(sqlNE); - Assert.assertEquals("Should have 2 records", 2, resultNE.size()); + assertThat(resultNE).hasSize(2); List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedNE, resultNE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -201,8 +192,8 @@ public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); List resultNE = sql(sqlNotEqualNull); - Assert.assertEquals("Should have 0 records", 0, resultNE.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultNE).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -211,14 +202,13 @@ public void testFilterPushDownAnd() { String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); List resultAnd = sql(sqlAnd); - Assert.assertEquals("Should have 1 record", 1, resultAnd.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultAnd.get(0)); - - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(resultAnd).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); } @Test @@ -227,14 +217,16 @@ public void testFilterPushDownOr() { String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; List resultOr = sql(sqlOr); - Assert.assertEquals("Should have 2 record", 2, resultOr.size()); + assertThat(resultOr).hasSize(2); List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedOR, resultOr); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -243,14 +235,16 @@ public void testFilterPushDownGreaterThan() { String expectedFilter = "ref(name=\"id\") > 1"; List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 record", 2, resultGT.size()); + assertThat(resultGT).hasSize(2); List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedGT, resultGT); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -258,8 +252,8 @@ public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -268,14 +262,16 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") < 3"; List resultGT = sql(sqlGT); - Assert.assertEquals("Should have 2 records", 2, resultGT.size()); + assertThat(resultGT).hasSize(2); List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedGT, resultGT); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -284,14 +280,16 @@ public void testFilterPushDownGreaterThanEqual() { String expectedFilter = "ref(name=\"id\") >= 2"; List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + assertThat(resultGTE).hasSize(2); List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedGTE, resultGTE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -299,8 +297,8 @@ public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); List resultGT = sql(sqlGTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -309,14 +307,16 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") <= 2"; List resultGTE = sql(sqlGTE); - Assert.assertEquals("Should have 2 records", 2, resultGTE.size()); + assertThat(resultGTE).hasSize(2); List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedGTE, resultGTE); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -325,13 +325,13 @@ public void testFilterPushDownLessThan() { String expectedFilter = "ref(name=\"id\") < 2"; List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLT.get(0)); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -339,8 +339,8 @@ public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); List resultGT = sql(sqlLT); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -349,13 +349,13 @@ public void testFilterPushDownLessThanLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") > 2"; List resultLT = sql(sqlLT); - Assert.assertEquals("Should have 1 record", 1, resultLT.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLT.get(0)); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -364,13 +364,13 @@ public void testFilterPushDownLessThanEqual() { String expectedFilter = "ref(name=\"id\") <= 1"; List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultLTE.get(0)); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -378,8 +378,8 @@ public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); List resultGT = sql(sqlLTE); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull("Should not push down a filter", lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } @Test @@ -388,13 +388,13 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { String expectedFilter = "ref(name=\"id\") >= 3"; List resultLTE = sql(sqlLTE); - Assert.assertEquals("Should have 1 record", 1, resultLTE.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultLTE.get(0)); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -402,13 +402,15 @@ public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; List resultIN = sql(sqlIN); - Assert.assertEquals("Should have 2 records", 2, resultIN.size()); + assertThat(resultIN).hasSize(2); List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedIN, resultIN); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -417,15 +419,15 @@ public void testFilterPushDownInNull() { String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); List result = sql(sqlInNull); - Assert.assertEquals("Should have 1 record", 1, result.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), result.get(0)); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it // and push the rest down. String expectedScan = "ref(name=\"data\") == \"iceberg\""; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test @@ -433,23 +435,24 @@ public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); List resultNotIn = sql(sqlNotIn); - Assert.assertEquals("Should have 1 record", 1, resultNotIn.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotIn.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(resultNotIn).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); - Assert.assertEquals("Should have 0 record", 0, resultGT.size()); - Assert.assertNull( - "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.", - lastScanEvent); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent) + .as( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.") + .isNull(); } @Test @@ -458,14 +461,16 @@ public void testFilterPushDownIsNotNull() { String expectedFilter = "not_null(ref(name=\"data\"))"; List resultNotNull = sql(sqlNotNull); - Assert.assertEquals("Should have 2 record", 2, resultNotNull.size()); + assertThat(resultNotNull).hasSize(2); List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expected, resultNotNull); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -474,13 +479,13 @@ public void testFilterPushDownIsNull() { String expectedFilter = "is_null(ref(name=\"data\"))"; List resultNull = sql(sqlNull); - Assert.assertEquals("Should have 1 record", 1, resultNull.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNull.get(0)); + assertThat(resultNull).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -488,14 +493,14 @@ public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); List resultNot = sql(sqlNot); - Assert.assertEquals("Should have 1 record", 1, resultNot.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(3, null, 30.0), resultNot.get(0)); + assertThat(resultNot).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(scanEventCount).isEqualTo(1); String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -503,16 +508,18 @@ public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); List resultBetween = sql(sqlBetween); - Assert.assertEquals("Should have 2 record", 2, resultBetween.size()); + assertThat(resultBetween).hasSize(2); List expectedBetween = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedBetween, resultBetween); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); + assertThat(scanEventCount).isEqualTo(1); String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; - Assert.assertEquals( - "Should contain the push down filter", expected, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); } @Test @@ -522,13 +529,13 @@ public void testFilterPushDownNotBetween() { String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; List resultNotBetween = sql(sqlNotBetween); - Assert.assertEquals("Should have 1 record", 1, resultNotBetween.size()); - Assert.assertEquals( - "Should produce the expected record", Row.of(1, "iceberg", 10.0), resultNotBetween.get(0)); + assertThat(resultNotBetween).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); } @Test @@ -537,25 +544,25 @@ public void testFilterPushDownLike() { String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; List resultLike = sql(sqlLike); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals( - "The like result should produce the expected record", - Row.of(1, "iceberg", 10.0), - resultLike.get(0)); - Assert.assertEquals("Should create only one scan", 1, scanEventCount); - Assert.assertEquals( - "Should contain the push down filter", expectedFilter, lastScanEvent.filter().toString()); + assertThat(resultLike).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); // %% won't match the row with null value sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; resultLike = sql(sqlLike); - Assert.assertEquals("Should have 2 records", 2, resultLike.size()); + assertThat(resultLike).hasSize(2); List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); assertSameElements(expectedRecords, resultLike); String expectedScan = "not_null(ref(name=\"data\"))"; - Assert.assertEquals( - "Should contain the push down filter", expectedScan, lastScanEvent.filter().toString()); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); } @Test @@ -563,37 +570,38 @@ public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; List resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 0 record", 0, resultLike.size()); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).isEmpty(); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; resultLike = sql(sqlNoPushDown); - Assert.assertEquals("Should have 1 record", 1, resultLike.size()); - Assert.assertEquals("Should produce the expected record", expectRecord, resultLike.get(0)); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test @@ -603,8 +611,9 @@ public void testFilterPushDown2Literal() { List expectedRecords = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); assertSameElements(expectedRecords, result); - Assert.assertEquals( - "Should not push down a filter", Expressions.alwaysTrue(), lastScanEvent.filter()); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); } @Test From 4fd635e109e03bdd75049538aabeeb89252f4979 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 30 Apr 2024 14:45:37 +0200 Subject: [PATCH 0267/1019] Release: add instruction to update doap.rdf file as part of release process (#9655) --- site/docs/how-to-release.md | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index b3b5b241d556..5d240b9d03eb 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -303,6 +303,20 @@ Create a PR in the `iceberg` repo to make revapi run on the new release. For an - Create a PR in the `iceberg` repo to add the new version to the github issue template. For an example see [this PR](https://github.com/apache/iceberg/pull/6287). - Draft [a new release to update Github](https://github.com/apache/iceberg/releases/new) to show the latest release. A changelog can be generated automatically using Github. +#### Update DOAP (ASF Project Description) + +- Create a PR to update the release version in [doap.rdf](https://github.com/apache/iceberg/blob/main/doap.rdf) file, in the `` section: + +```xml + + + x.y.z + yyyy-mm-dd + x.y.z + + +``` + ### Documentation Release Documentation needs to be updated as a part of an Iceberg release after a release candidate is passed. From 5afe4dce1e64b249e7c3565fff595aac8737b42d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 30 Apr 2024 16:14:39 +0200 Subject: [PATCH 0268/1019] Add stale PRs management (#10134) --- .github/workflows/stale.yml | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index f981d2cc631a..0f599e5582cc 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -17,7 +17,7 @@ # under the License. # -name: "Close Stale Issues" +name: "Close Stale Issues and PRs" on: schedule: - cron: '0 0 * * *' @@ -33,12 +33,11 @@ jobs: steps: - uses: actions/stale@v9.0.0 with: + # stale issues stale-issue-label: 'stale' exempt-issue-labels: 'not-stale' days-before-issue-stale: 180 days-before-issue-close: 14 - # Only close stale issues, leave PRs alone - days-before-pr-stale: -1 stale-issue-message: > This issue has been automatically marked as stale because it has been open for 180 days with no activity. It will be closed in next 14 days if no further activity occurs. To @@ -47,5 +46,12 @@ jobs: close-issue-message: > This issue has been closed because it has not received any activity in the last 14 days since being marked as 'stale' + # stale PRs + stale-pr-label: 'stale' + exempt-pr-labels: 'not-stale,security' + stale-pr-message: 'This pull request has been marked as stale due to 30 days of inactivity. It will be closed in 1 week if no further activity occurs. If you think that’s incorrect or this pull request requires a review, please simply write any comment. If closed, you can revive the PR at any time and @mention a reviewer or discuss it on the dev@iceberg.apache.org list. Thank you for your contributions.' + close-pr-message: 'This pull request has been closed due to lack of activity. This is not a judgement on the merit of the PR in any way. It is just a way of keeping the PR queue manageable. If you think that is incorrect, or the pull request requires review, you can revive the PR at any time.' + days-before-pr-stale: 30 + days-before-pr-close: 7 ascending: true operations-per-run: 100 From f7196e4d2362c2e8c640500a0b725438f53f9a58 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 30 Apr 2024 20:13:04 +0530 Subject: [PATCH 0269/1019] Docs: Update doap.rdf (#10255) --- doap.rdf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doap.rdf b/doap.rdf index d3ef2e34497c..8ee99ef757a5 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.5.0 - 2024-03-11 - 1.5.0 + 1.5.1 + 2024-04-25 + 1.5.1 From 01acc6d531d80abd54cff99bcacb7ceb9a2c7a0b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Apr 2024 21:35:12 +0200 Subject: [PATCH 0270/1019] Build: Bump com.azure:azure-sdk-bom from 1.2.22 to 1.2.23 (#10238) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.22 to 1.2.23. - [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.22...azure-sdk-bom_1.2.23) --- 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 ba4e0129129b..cfade621366a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" awssdk-bom = "2.25.40" -azuresdk-bom = "1.2.22" +azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From 087c09296481bed47747305efacd6f992932ec6e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Apr 2024 21:35:27 +0200 Subject: [PATCH 0271/1019] Build: Bump mkdocs-material from 9.5.18 to 9.5.19 (#10241) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.18 to 9.5.19. - [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.18...9.5.19) --- 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 f238604eb00a..8fdef1d27c37 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.18 +mkdocs-material==9.5.19 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 5d66fbcf3b0167b831ff005ec3f3db6011faa6e2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 30 Apr 2024 21:35:35 +0200 Subject: [PATCH 0272/1019] Build: Bump datamodel-code-generator from 0.25.5 to 0.25.6 (#10242) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.5 to 0.25.6. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.5...0.25.6) --- 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 5b819b796841..4619cc35e336 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.25.5 +datamodel-code-generator==0.25.6 From 8e0d2a42203fc0080b96319aa6b30036f3b867d1 Mon Sep 17 00:00:00 2001 From: pvary Date: Tue, 30 Apr 2024 22:52:48 +0200 Subject: [PATCH 0273/1019] Flink: Apply DeleteGranularity for writes (#10200) --- .../SortingPositionOnlyDeleteWriter.java | 4 + .../org/apache/iceberg/io/BaseTaskWriter.java | 42 ++++++- .../io/TestTaskEqualityDeltaWriter.java | 108 +++++++++++++++--- .../flink/sink/BaseDeltaTaskWriter.java | 3 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 35 ++++++ 5 files changed, 169 insertions(+), 23 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 c63f3130109d..1d4d131dfe6f 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java @@ -118,6 +118,10 @@ private DeleteWriteResult writeFileDeletes() throws IOException { @SuppressWarnings("CollectionUndefinedEquality") private DeleteWriteResult writeDeletes(Collection paths) throws IOException { + if (paths.isEmpty()) { + return new DeleteWriteResult(Lists.newArrayList(), CharSequenceSet.empty()); + } + FileWriter, DeleteWriteResult> writer = writers.get(); try { 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 bba2c5355a98..471dc3e56035 100644 --- a/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BaseTaskWriter.java @@ -29,7 +29,10 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.SortingPositionOnlyDeleteWriter; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -107,20 +110,32 @@ public WriteResult complete() throws IOException { /** Base equality delta writer to write both insert records and equality-deletes. */ protected abstract class BaseEqualityDeltaWriter implements Closeable { private final StructProjection structProjection; + private final PositionDelete positionDelete; private RollingFileWriter dataWriter; private RollingEqDeleteWriter eqDeleteWriter; - private SortedPosDeleteWriter posDeleteWriter; + private FileWriter, DeleteWriteResult> posDeleteWriter; private Map insertedRowMap; protected BaseEqualityDeltaWriter(StructLike partition, Schema schema, Schema deleteSchema) { + this(partition, schema, deleteSchema, DeleteGranularity.PARTITION); + } + + protected BaseEqualityDeltaWriter( + StructLike partition, + Schema schema, + Schema deleteSchema, + DeleteGranularity deleteGranularity) { Preconditions.checkNotNull(schema, "Iceberg table schema cannot be null."); Preconditions.checkNotNull(deleteSchema, "Equality-delete schema cannot be null."); this.structProjection = StructProjection.create(schema, deleteSchema); + this.positionDelete = PositionDelete.create(); this.dataWriter = new RollingFileWriter(partition); this.eqDeleteWriter = new RollingEqDeleteWriter(partition); this.posDeleteWriter = - new SortedPosDeleteWriter<>(appenderFactory, fileFactory, format, partition); + new SortingPositionOnlyDeleteWriter<>( + () -> appenderFactory.newPosDeleteWriter(newOutputFile(partition), format, partition), + deleteGranularity); this.insertedRowMap = StructLikeMap.create(deleteSchema.asStruct()); } @@ -140,12 +155,25 @@ public void write(T row) throws IOException { PathOffset previous = insertedRowMap.put(copiedKey, pathOffset); if (previous != null) { // TODO attach the previous row if has a positional-delete row schema in appender factory. - posDeleteWriter.delete(previous.path, previous.rowOffset, null); + writePosDelete(previous); } dataWriter.write(row); } + private EncryptedOutputFile newOutputFile(StructLike partition) { + if (spec.isUnpartitioned() || partition == null) { + return fileFactory.newOutputFile(); + } else { + return fileFactory.newOutputFile(spec, partition); + } + } + + private void writePosDelete(PathOffset pathOffset) { + positionDelete.set(pathOffset.path, pathOffset.rowOffset, null); + posDeleteWriter.write(positionDelete); + } + /** * Write the pos-delete if there's an existing row matching the given key. * @@ -156,7 +184,7 @@ private boolean internalPosDelete(StructLike key) { if (previous != null) { // TODO attach the previous row if has a positional-delete row schema in appender factory. - posDeleteWriter.delete(previous.path, previous.rowOffset, null); + writePosDelete(previous); return true; } @@ -217,8 +245,10 @@ public void close() throws IOException { if (posDeleteWriter != null) { try { // complete will call close - completedDeleteFiles.addAll(posDeleteWriter.complete()); - referencedDataFiles.addAll(posDeleteWriter.referencedDataFiles()); + posDeleteWriter.close(); + DeleteWriteResult result = posDeleteWriter.result(); + completedDeleteFiles.addAll(result.deleteFiles()); + referencedDataFiles.addAll(result.referencedDataFiles()); } finally { posDeleteWriter = null; } 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 4910b74a2abe..aed549bbf6ca 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestTaskEqualityDeltaWriter.java @@ -24,6 +24,7 @@ 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; @@ -47,10 +48,12 @@ import org.apache.iceberg.data.avro.DataReader; import org.apache.iceberg.data.orc.GenericOrcReader; import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.orc.ORC; import org.apache.iceberg.parquet.Parquet; 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.util.ArrayUtil; import org.apache.iceberg.util.StructLikeSet; import org.junit.jupiter.api.BeforeEach; @@ -60,7 +63,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestTaskEqualityDeltaWriter extends TestBase { private static final int FORMAT_V2 = 2; - private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024L; + private static final long TARGET_FILE_SIZE = 128L; private final GenericRecord gRecord = GenericRecord.create(SCHEMA); private final GenericRecord posRecord = GenericRecord.create(DeleteSchemaUtil.pathPosSchema()); @@ -105,7 +108,8 @@ public void testPureInsert() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(idFieldId, dataFieldId); Schema eqDeleteRowSchema = table.schema(); - GenericTaskDeltaWriter deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + GenericTaskDeltaWriter deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); List expected = Lists.newArrayList(); for (int i = 0; i < 20; i++) { Record record = createRecord(i, String.format("val-%d", i)); @@ -122,7 +126,8 @@ public void testPureInsert() throws IOException { .as("Should have expected records") .isEqualTo(actualRowSet("*")); - deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); for (int i = 20; i < 30; i++) { Record record = createRecord(i, String.format("val-%d", i)); expected.add(record); @@ -143,7 +148,8 @@ public void testInsertDuplicatedKey() throws IOException { List equalityFieldIds = Lists.newArrayList(idFieldId); Schema eqDeleteRowSchema = table.schema(); - GenericTaskDeltaWriter deltaWriter = createTaskWriter(equalityFieldIds, eqDeleteRowSchema); + GenericTaskDeltaWriter deltaWriter = + createTaskWriter(equalityFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); deltaWriter.write(createRecord(1, "aaa")); deltaWriter.write(createRecord(2, "bbb")); deltaWriter.write(createRecord(3, "ccc")); @@ -203,7 +209,8 @@ public void testUpsertSameRow() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(idFieldId, dataFieldId); Schema eqDeleteRowSchema = table.schema(); - GenericTaskDeltaWriter deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + GenericTaskDeltaWriter deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); Record record = createRecord(1, "aaa"); deltaWriter.write(record); @@ -230,7 +237,8 @@ public void testUpsertSameRow() throws IOException { assertThat(readRecordsAsList(DeleteSchemaUtil.pathPosSchema(), posDeleteFile.path())) .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); - deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); deltaWriter.delete(record); result = deltaWriter.complete(); assertThat(result.dataFiles()).as("Should have 0 data file.").hasSize(0); @@ -246,7 +254,8 @@ public void testUpsertData() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(dataFieldId); Schema eqDeleteRowSchema = table.schema().select("data"); - GenericTaskDeltaWriter deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + GenericTaskDeltaWriter deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); deltaWriter.write(createRecord(1, "aaa")); deltaWriter.write(createRecord(2, "bbb")); deltaWriter.write(createRecord(3, "aaa")); @@ -273,7 +282,8 @@ public void testUpsertData() throws IOException { createRecord(2, "bbb"), createRecord(3, "aaa"), createRecord(4, "ccc")))); // Start the 2nd transaction. - deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); GenericRecord keyRecord = GenericRecord.create(eqDeleteRowSchema); Function keyFunc = data -> keyRecord.copy("data", data); @@ -330,7 +340,8 @@ public void testUpsertDataWithFullRowSchema() throws IOException { List eqDeleteFieldIds = Lists.newArrayList(dataFieldId); Schema eqDeleteRowSchema = table.schema(); - GenericTaskDeltaWriter deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + GenericTaskDeltaWriter deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); deltaWriter.write(createRecord(1, "aaa")); deltaWriter.write(createRecord(2, "bbb")); deltaWriter.write(createRecord(3, "aaa")); @@ -357,7 +368,8 @@ public void testUpsertDataWithFullRowSchema() throws IOException { createRecord(2, "bbb"), createRecord(3, "aaa"), createRecord(4, "ccc")))); // Start the 2nd transaction. - deltaWriter = createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema); + deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, DeleteGranularity.PARTITION); // UPSERT <3,'aaa'> to <5,'aaa'> - (by delete the entire row) deltaWriter.delete(createRecord(3, "aaa")); @@ -409,6 +421,62 @@ public void testUpsertDataWithFullRowSchema() throws IOException { .isEqualTo(ImmutableList.of(posRecord.copy("file_path", dataFile.path(), "pos", 0L))); } + @TestTemplate + public void testDeleteFileGranularity() throws IOException { + withGranularity(DeleteGranularity.FILE); + } + + @TestTemplate + public void testDeletePartitionGranularity() throws IOException { + withGranularity(DeleteGranularity.PARTITION); + } + + private void withGranularity(DeleteGranularity granularity) throws IOException { + List eqDeleteFieldIds = Lists.newArrayList(idFieldId, dataFieldId); + Schema eqDeleteRowSchema = table.schema(); + + GenericTaskDeltaWriter deltaWriter = + createTaskWriter(eqDeleteFieldIds, eqDeleteRowSchema, granularity); + + Map expected = Maps.newHashMapWithExpectedSize(2000); + int expectedDeleteCount = 0; + // Create enough records, so we have multiple files + for (int i = 0; i < 2000; ++i) { + Record record = createRecord(i, "aaa" + i); + deltaWriter.write(record); + if (i % 5 == 0) { + deltaWriter.delete(record); + ++expectedDeleteCount; + } else { + expected.put(i, record); + } + } + + // Add some deletes in the end + for (int i = 0; i < 199; ++i) { + int id = i * 10 + 1; + Record record = createRecord(id, "aaa" + id); + deltaWriter.delete(record); + ++expectedDeleteCount; + expected.remove(id); + } + + WriteResult result = deltaWriter.complete(); + + // Should have 2 files, as BaseRollingWriter checks the size on every 1000 rows (ROWS_DIVISOR) + assertThat(result.dataFiles()).as("Should have 2 data files.").hasSize(2); + assertThat(result.deleteFiles()) + .as("Should have correct number of pos-delete files") + .hasSize(granularity.equals(DeleteGranularity.FILE) ? 2 : 1); + assertThat(Arrays.stream(result.deleteFiles()).mapToLong(delete -> delete.recordCount()).sum()) + .isEqualTo(expectedDeleteCount); + + commitTransaction(result); + assertThat(actualRowSet("*")) + .as("Should have expected record") + .isEqualTo(expectedRowSet(expected.values())); + } + private void commitTransaction(WriteResult result) { RowDelta rowDelta = table.newRowDelta(); Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); @@ -442,7 +510,9 @@ private StructLikeSet actualRowSet(String... columns) throws IOException { * be the entire fields of the table schema. */ private GenericTaskDeltaWriter createTaskWriter( - List equalityFieldIds, Schema eqDeleteRowSchema) { + List equalityFieldIds, + Schema eqDeleteRowSchema, + DeleteGranularity deleteGranularity) { FileAppenderFactory appenderFactory = new GenericAppenderFactory( table.schema(), @@ -465,7 +535,8 @@ private GenericTaskDeltaWriter createTaskWriter( appenderFactory, fileFactory, table.io(), - TARGET_FILE_SIZE); + TARGET_FILE_SIZE, + deleteGranularity); } private static class GenericTaskDeltaWriter extends BaseTaskWriter { @@ -479,9 +550,11 @@ private GenericTaskDeltaWriter( FileAppenderFactory appenderFactory, OutputFileFactory fileFactory, FileIO io, - long targetFileSize) { + long targetFileSize, + DeleteGranularity deleteGranularity) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.deltaWriter = new GenericEqualityDeltaWriter(null, schema, deleteSchema); + this.deltaWriter = + new GenericEqualityDeltaWriter(null, schema, deleteSchema, deleteGranularity); } @Override @@ -505,8 +578,11 @@ public void close() throws IOException { private class GenericEqualityDeltaWriter extends BaseEqualityDeltaWriter { private GenericEqualityDeltaWriter( - PartitionKey partition, Schema schema, Schema eqDeleteSchema) { - super(partition, schema, eqDeleteSchema); + PartitionKey partition, + Schema schema, + Schema eqDeleteSchema, + DeleteGranularity deleteGranularity) { + super(partition, schema, eqDeleteSchema, deleteGranularity); } @Override diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index 40e0b5f2a34e..e8a46c5becd7 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -27,6 +27,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.data.RowDataProjection; @@ -109,7 +110,7 @@ public void write(RowData row) throws IOException { protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { RowDataDeltaWriter(PartitionKey partition) { - super(partition, schema, deleteSchema); + super(partition, schema, deleteSchema, DeleteGranularity.FILE); } @Override 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 b5c3bcf41734..5e81c279b69b 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 @@ -18,12 +18,17 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.util.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; @@ -45,6 +50,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,6 +67,8 @@ public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { public final HadoopCatalogResource catalogResource = new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @Rule public final Timeout globalTimeout = Timeout.seconds(60); + @Parameterized.Parameters( name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") public static Object[][] parameters() { @@ -233,4 +241,31 @@ public void testUpsertOnDataKey() throws Exception { public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); } + + @Test + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + 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()); + } } From 990d988549ee09135bfd8cb3835f96f847b60bfa Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 1 May 2024 17:11:29 +0200 Subject: [PATCH 0274/1019] Hive: Remove deprecated `setSchema(TableMetadata, Map)` (#10257) --- .../java/org/apache/iceberg/hive/HiveOperationsBase.java | 8 -------- .../java/org/apache/iceberg/hive/TestHiveCatalog.java | 2 +- 2 files changed, 1 insertion(+), 9 deletions(-) diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java index a93577a35c73..055a14246e77 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -103,14 +103,6 @@ default boolean exposeInHmsProperties() { return maxHiveTablePropertySize() > 0; } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link #setSchema(Schema, Map)} instead - */ - @Deprecated - default void setSchema(TableMetadata metadata, Map parameters) { - setSchema(metadata.schema(), parameters); - } - default void setSchema(Schema schema, Map parameters) { parameters.remove(TableProperties.CURRENT_SCHEMA); if (exposeInHmsProperties() && schema != null) { 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 369ad46c8e49..95bf6c697c32 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 @@ -1041,7 +1041,7 @@ public void testNotExposeTableProperties() { .doesNotContainKey(CURRENT_SNAPSHOT_ID) .doesNotContainKey(CURRENT_SNAPSHOT_TIMESTAMP); - ops.setSchema(metadata, parameters); + ops.setSchema(metadata.schema(), parameters); assertThat(parameters).doesNotContainKey(CURRENT_SCHEMA); ops.setPartitionSpec(metadata, parameters); From b96a9e47700eda56e85f6e09ea943d26fdf3aa60 Mon Sep 17 00:00:00 2001 From: pvary Date: Wed, 1 May 2024 20:46:38 +0200 Subject: [PATCH 0275/1019] Flink: Backport #10200 to v1.19 and v1.17 (#10259) --- .../flink/sink/BaseDeltaTaskWriter.java | 3 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 35 +++++++++++++++++++ .../flink/sink/BaseDeltaTaskWriter.java | 3 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 35 +++++++++++++++++++ 4 files changed, 74 insertions(+), 2 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index 40e0b5f2a34e..e8a46c5becd7 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -27,6 +27,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.data.RowDataProjection; @@ -109,7 +110,7 @@ public void write(RowData row) throws IOException { protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { RowDataDeltaWriter(PartitionKey partition) { - super(partition, schema, deleteSchema); + super(partition, schema, deleteSchema, DeleteGranularity.FILE); } @Override diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index b5c3bcf41734..5e81c279b69b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -18,12 +18,17 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.util.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; @@ -45,6 +50,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,6 +67,8 @@ public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { public final HadoopCatalogResource catalogResource = new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @Rule public final Timeout globalTimeout = Timeout.seconds(60); + @Parameterized.Parameters( name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") public static Object[][] parameters() { @@ -233,4 +241,31 @@ public void testUpsertOnDataKey() throws Exception { public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); } + + @Test + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + 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()); + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java index 40e0b5f2a34e..e8a46c5becd7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java @@ -27,6 +27,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.data.RowDataProjection; @@ -109,7 +110,7 @@ public void write(RowData row) throws IOException { protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { RowDataDeltaWriter(PartitionKey partition) { - super(partition, schema, deleteSchema); + super(partition, schema, deleteSchema, DeleteGranularity.FILE); } @Override 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 b5c3bcf41734..5e81c279b69b 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 @@ -18,12 +18,17 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assumptions.assumeThat; + import java.util.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; @@ -45,6 +50,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -61,6 +67,8 @@ public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { public final HadoopCatalogResource catalogResource = new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @Rule public final Timeout globalTimeout = Timeout.seconds(60); + @Parameterized.Parameters( name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") public static Object[][] parameters() { @@ -233,4 +241,31 @@ public void testUpsertOnDataKey() throws Exception { public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); } + + @Test + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + 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()); + } } From d6d1f97b22cfabb9dd726e0a2d40526c4608ddd3 Mon Sep 17 00:00:00 2001 From: Rahil C <32500120+rahil-c@users.noreply.github.com> Date: Thu, 2 May 2024 23:35:38 -0700 Subject: [PATCH 0276/1019] Core: Add pagination when listing namespaces/tables/views (#9782) --- .../apache/iceberg/rest/CatalogHandlers.java | 58 +++++++ .../iceberg/rest/RESTSessionCatalog.java | 99 ++++++++---- .../responses/ListNamespacesResponse.java | 21 ++- .../rest/responses/ListTablesResponse.java | 21 ++- .../iceberg/rest/RESTCatalogAdapter.java | 32 +++- .../apache/iceberg/rest/TestRESTCatalog.java | 145 ++++++++++++++++++ .../iceberg/rest/TestRESTViewCatalog.java | 85 ++++++++++ .../responses/TestListNamespacesResponse.java | 29 +++- .../responses/TestListTablesResponse.java | 29 +++- 9 files changed, 476 insertions(+), 43 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 e4e3c065fb55..746da5ffcab0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -80,6 +80,7 @@ public class CatalogHandlers { private static final Schema EMPTY_SCHEMA = new Schema(); + private static final String INTIAL_PAGE_TOKEN = ""; private CatalogHandlers() {} @@ -117,6 +118,29 @@ public static ListNamespacesResponse listNamespaces( return ListNamespacesResponse.builder().addAll(results).build(); } + public static ListNamespacesResponse listNamespaces( + SupportsNamespaces catalog, Namespace parent, String pageToken, String pageSize) { + List results; + List subResults; + + if (parent.isEmpty()) { + results = catalog.listNamespaces(); + } else { + results = catalog.listNamespaces(parent); + } + + int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); + int end = start + Integer.parseInt(pageSize); + subResults = results.subList(start, end); + String nextToken = String.valueOf(end); + + if (end >= results.size()) { + nextToken = null; + } + + return ListNamespacesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + } + public static CreateNamespaceResponse createNamespace( SupportsNamespaces catalog, CreateNamespaceRequest request) { Namespace namespace = request.namespace(); @@ -174,6 +198,23 @@ public static ListTablesResponse listTables(Catalog catalog, Namespace namespace return ListTablesResponse.builder().addAll(idents).build(); } + public static ListTablesResponse listTables( + Catalog catalog, Namespace namespace, String pageToken, String pageSize) { + List results = catalog.listTables(namespace); + List subResults; + + int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); + int end = start + Integer.parseInt(pageSize); + subResults = results.subList(start, end); + String nextToken = String.valueOf(end); + + if (end >= results.size()) { + nextToken = null; + } + + return ListTablesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + } + public static LoadTableResponse stageTableCreate( Catalog catalog, Namespace namespace, CreateTableRequest request) { request.validate(); @@ -397,6 +438,23 @@ public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namesp return ListTablesResponse.builder().addAll(catalog.listViews(namespace)).build(); } + public static ListTablesResponse listViews( + ViewCatalog catalog, Namespace namespace, String pageToken, String pageSize) { + List results = catalog.listViews(namespace); + List subResults; + + int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); + int end = start + Integer.parseInt(pageSize); + subResults = results.subList(start, end); + String nextToken = String.valueOf(end); + + if (end >= results.size()) { + nextToken = null; + } + + return ListTablesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + } + public static LoadViewResponse createView( ViewCatalog catalog, Namespace namespace, CreateViewRequest request) { request.validate(); 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 94f3057f9fe7..dcf92289df2e 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -114,6 +114,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO"; private static final String REST_METRICS_REPORTING_ENABLED = "rest-metrics-reporting-enabled"; private static final String REST_SNAPSHOT_LOADING_MODE = "snapshot-loading-mode"; + public static final String REST_PAGE_SIZE = "rest-page-size"; private static final List TOKEN_PREFERENCE_ORDER = ImmutableList.of( OAuth2Properties.ID_TOKEN_TYPE, @@ -136,6 +137,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private FileIO io = null; private MetricsReporter reporter = null; private boolean reportingViaRestEnabled; + private Integer pageSize = null; private CloseableGroup closeables = null; // a lazy thread pool for token refresh @@ -228,6 +230,12 @@ public void initialize(String name, Map unresolved) { client, tokenRefreshExecutor(name), token, expiresAtMillis(mergedProps), catalogAuth); } + this.pageSize = PropertyUtil.propertyAsNullableInt(mergedProps, REST_PAGE_SIZE); + if (pageSize != null) { + Preconditions.checkArgument( + pageSize > 0, "Invalid value for %s, must be a positive integer", REST_PAGE_SIZE); + } + this.io = newFileIO(SessionContext.createEmpty(), mergedProps); this.fileIOCloser = newFileIOCloser(); @@ -278,14 +286,27 @@ public void setConf(Object newConf) { @Override public List listTables(SessionContext context, Namespace ns) { checkNamespaceIsValid(ns); + Map queryParams = Maps.newHashMap(); + ImmutableList.Builder tables = ImmutableList.builder(); + String pageToken = ""; + if (pageSize != null) { + queryParams.put("pageSize", String.valueOf(pageSize)); + } - ListTablesResponse response = - client.get( - paths.tables(ns), - ListTablesResponse.class, - headers(context), - ErrorHandlers.namespaceErrorHandler()); - return response.identifiers(); + do { + queryParams.put("pageToken", pageToken); + ListTablesResponse response = + client.get( + paths.tables(ns), + queryParams, + ListTablesResponse.class, + headers(context), + ErrorHandlers.namespaceErrorHandler()); + pageToken = response.nextPageToken(); + tables.addAll(response.identifiers()); + } while (pageToken != null); + + return tables.build(); } @Override @@ -494,22 +515,31 @@ public void createNamespace( @Override public List listNamespaces(SessionContext context, Namespace namespace) { - Map queryParams; - if (namespace.isEmpty()) { - queryParams = ImmutableMap.of(); - } else { - // query params should be unescaped - queryParams = ImmutableMap.of("parent", RESTUtil.NAMESPACE_JOINER.join(namespace.levels())); + Map queryParams = Maps.newHashMap(); + if (!namespace.isEmpty()) { + queryParams.put("parent", RESTUtil.NAMESPACE_JOINER.join(namespace.levels())); } - ListNamespacesResponse response = - client.get( - paths.namespaces(), - queryParams, - ListNamespacesResponse.class, - headers(context), - ErrorHandlers.namespaceErrorHandler()); - return response.namespaces(); + ImmutableList.Builder namespaces = ImmutableList.builder(); + String pageToken = ""; + if (pageSize != null) { + queryParams.put("pageSize", String.valueOf(pageSize)); + } + + do { + queryParams.put("pageToken", pageToken); + ListNamespacesResponse response = + client.get( + paths.namespaces(), + queryParams, + ListNamespacesResponse.class, + headers(context), + ErrorHandlers.namespaceErrorHandler()); + pageToken = response.nextPageToken(); + namespaces.addAll(response.namespaces()); + } while (pageToken != null); + + return namespaces.build(); } @Override @@ -1048,14 +1078,27 @@ public void commitTransaction(SessionContext context, List commits) @Override public List listViews(SessionContext context, Namespace namespace) { checkNamespaceIsValid(namespace); + Map queryParams = Maps.newHashMap(); + ImmutableList.Builder views = ImmutableList.builder(); + String pageToken = ""; + if (pageSize != null) { + queryParams.put("pageSize", String.valueOf(pageSize)); + } - ListTablesResponse response = - client.get( - paths.views(namespace), - ListTablesResponse.class, - headers(context), - ErrorHandlers.namespaceErrorHandler()); - return response.identifiers(); + do { + queryParams.put("pageToken", pageToken); + ListTablesResponse response = + client.get( + paths.views(namespace), + queryParams, + ListTablesResponse.class, + headers(context), + ErrorHandlers.namespaceErrorHandler()); + pageToken = response.nextPageToken(); + views.addAll(response.identifiers()); + } while (pageToken != null); + + return views.build(); } @Override diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ListNamespacesResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ListNamespacesResponse.java index 13a599e1a76c..8feeda6f2b8b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ListNamespacesResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ListNamespacesResponse.java @@ -29,13 +29,15 @@ public class ListNamespacesResponse implements RESTResponse { private List namespaces; + private String nextPageToken; public ListNamespacesResponse() { // Required for Jackson deserialization } - private ListNamespacesResponse(List namespaces) { + private ListNamespacesResponse(List namespaces, String nextPageToken) { this.namespaces = namespaces; + this.nextPageToken = nextPageToken; validate(); } @@ -48,9 +50,16 @@ public List namespaces() { return namespaces != null ? namespaces : ImmutableList.of(); } + public String nextPageToken() { + return nextPageToken; + } + @Override public String toString() { - return MoreObjects.toStringHelper(this).add("namespaces", namespaces()).toString(); + return MoreObjects.toStringHelper(this) + .add("namespaces", namespaces()) + .add("next-page-token", nextPageToken()) + .toString(); } public static Builder builder() { @@ -59,6 +68,7 @@ public static Builder builder() { public static class Builder { private final ImmutableList.Builder namespaces = ImmutableList.builder(); + private String nextPageToken; private Builder() {} @@ -75,8 +85,13 @@ public Builder addAll(Collection toAdd) { return this; } + public Builder nextPageToken(String pageToken) { + nextPageToken = pageToken; + return this; + } + public ListNamespacesResponse build() { - return new ListNamespacesResponse(namespaces.build()); + return new ListNamespacesResponse(namespaces.build(), nextPageToken); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ListTablesResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ListTablesResponse.java index 3c99c12c9023..1db05709b470 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ListTablesResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ListTablesResponse.java @@ -30,13 +30,15 @@ public class ListTablesResponse implements RESTResponse { private List identifiers; + private String nextPageToken; public ListTablesResponse() { // Required for Jackson deserialization } - private ListTablesResponse(List identifiers) { + private ListTablesResponse(List identifiers, String nextPageToken) { this.identifiers = identifiers; + this.nextPageToken = nextPageToken; validate(); } @@ -49,9 +51,16 @@ public List identifiers() { return identifiers != null ? identifiers : ImmutableList.of(); } + public String nextPageToken() { + return nextPageToken; + } + @Override public String toString() { - return MoreObjects.toStringHelper(this).add("identifiers", identifiers).toString(); + return MoreObjects.toStringHelper(this) + .add("identifiers", identifiers) + .add("next-page-token", nextPageToken()) + .toString(); } public static Builder builder() { @@ -60,6 +69,7 @@ public static Builder builder() { public static class Builder { private final ImmutableList.Builder identifiers = ImmutableList.builder(); + private String nextPageToken; private Builder() {} @@ -76,8 +86,13 @@ public Builder addAll(Collection toAdd) { return this; } + public Builder nextPageToken(String pageToken) { + nextPageToken = pageToken; + return this; + } + public ListTablesResponse build() { - return new ListTablesResponse(identifiers.build()); + return new ListTablesResponse(identifiers.build(), nextPageToken); } } } 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 7fccc4e974e3..357b05e85c20 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,17 @@ public T handleRequest( ns = Namespace.empty(); } - return castResponse(responseType, CatalogHandlers.listNamespaces(asNamespaceCatalog, ns)); + String pageToken = PropertyUtil.propertyAsString(vars, "pageToken", null); + String pageSize = PropertyUtil.propertyAsString(vars, "pageSize", null); + + if (pageSize != null) { + return castResponse( + responseType, + CatalogHandlers.listNamespaces(asNamespaceCatalog, ns, pageToken, pageSize)); + } else { + return castResponse( + responseType, CatalogHandlers.listNamespaces(asNamespaceCatalog, ns)); + } } break; @@ -339,7 +349,14 @@ public T handleRequest( case LIST_TABLES: { Namespace namespace = namespaceFromPathVars(vars); - return castResponse(responseType, CatalogHandlers.listTables(catalog, namespace)); + String pageToken = PropertyUtil.propertyAsString(vars, "pageToken", null); + String pageSize = PropertyUtil.propertyAsString(vars, "pageSize", null); + if (pageSize != null) { + return castResponse( + responseType, CatalogHandlers.listTables(catalog, namespace, pageToken, pageSize)); + } else { + return castResponse(responseType, CatalogHandlers.listTables(catalog, namespace)); + } } case CREATE_TABLE: @@ -412,7 +429,16 @@ public T handleRequest( { if (null != asViewCatalog) { Namespace namespace = namespaceFromPathVars(vars); - return castResponse(responseType, CatalogHandlers.listViews(asViewCatalog, namespace)); + String pageToken = PropertyUtil.propertyAsString(vars, "pageToken", null); + String pageSize = PropertyUtil.propertyAsString(vars, "pageSize", null); + if (pageSize != null) { + return castResponse( + responseType, + CatalogHandlers.listViews(asViewCatalog, namespace, pageToken, pageSize)); + } else { + return castResponse( + responseType, CatalogHandlers.listViews(asViewCatalog, namespace)); + } } break; } 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 18d832b3cd46..95380424e7ad 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -77,7 +77,10 @@ import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.ConfigResponse; +import org.apache.iceberg.rest.responses.CreateNamespaceResponse; import org.apache.iceberg.rest.responses.ErrorResponse; +import org.apache.iceberg.rest.responses.ListNamespacesResponse; +import org.apache.iceberg.rest.responses.ListTablesResponse; import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.types.Types; @@ -2329,6 +2332,148 @@ public void multipleDiffsAgainstMultipleTablesLastFails() { assertThat(schema2.columns()).hasSize(1); } + @Test + public void testInvalidPageSize() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + Assertions.assertThatThrownBy( + () -> + catalog.initialize( + "test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "-1"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + String.format( + "Invalid value for %s, must be a positive integer", + RESTSessionCatalog.REST_PAGE_SIZE)); + } + + @Test + public void testPaginationForListNamespaces() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); + int numberOfItems = 30; + String namespaceName = "newdb"; + + // create several namespaces for listing and verify + for (int i = 0; i < numberOfItems; i++) { + String nameSpaceName = namespaceName + i; + catalog.createNamespace(Namespace.of(nameSpaceName)); + } + + assertThat(catalog.listNamespaces()).hasSize(numberOfItems); + + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/config"), + any(), + any(), + eq(ConfigResponse.class), + any(), + any()); + + Mockito.verify(adapter, times(numberOfItems)) + .execute( + eq(HTTPMethod.POST), + eq("v1/namespaces"), + any(), + any(), + eq(CreateNamespaceResponse.class), + any(), + any()); + + // verify initial request with empty pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_NAMESPACES), + eq(ImmutableMap.of("pageToken", "", "pageSize", "10")), + any(), + eq(ListNamespacesResponse.class)); + + // verify second request with updated pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_NAMESPACES), + eq(ImmutableMap.of("pageToken", "10", "pageSize", "10")), + any(), + eq(ListNamespacesResponse.class)); + + // verify third request with update pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_NAMESPACES), + eq(ImmutableMap.of("pageToken", "20", "pageSize", "10")), + any(), + eq(ListNamespacesResponse.class)); + } + + @Test + public void testPaginationForListTables() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); + int numberOfItems = 30; + String namespaceName = "newdb"; + String tableName = "newtable"; + catalog.createNamespace(Namespace.of(namespaceName)); + + // create several tables under namespace for listing and verify + for (int i = 0; i < numberOfItems; i++) { + TableIdentifier tableIdentifier = TableIdentifier.of(namespaceName, tableName + i); + catalog.createTable(tableIdentifier, SCHEMA); + } + + assertThat(catalog.listTables(Namespace.of(namespaceName))).hasSize(numberOfItems); + + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/config"), + any(), + any(), + eq(ConfigResponse.class), + any(), + any()); + + Mockito.verify(adapter, times(numberOfItems)) + .execute( + eq(HTTPMethod.POST), + eq(String.format("v1/namespaces/%s/tables", namespaceName)), + any(), + any(), + eq(LoadTableResponse.class), + any(), + any()); + + // verify initial request with empty pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_TABLES), + eq(ImmutableMap.of("pageToken", "", "pageSize", "10", "namespace", namespaceName)), + any(), + eq(ListTablesResponse.class)); + + // verify second request with updated pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_TABLES), + eq(ImmutableMap.of("pageToken", "10", "pageSize", "10", "namespace", namespaceName)), + any(), + eq(ListTablesResponse.class)); + + // verify third request with update pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_TABLES), + eq(ImmutableMap.of("pageToken", "20", "pageSize", "10", "namespace", namespaceName)), + any(), + eq(ListTablesResponse.class)); + } + @Test public void testCleanupUncommitedFilesForCleanableFailures() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); 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 0b29da70426b..f67c4b078e5d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -18,20 +18,31 @@ */ package org.apache.iceberg.rest; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.eq; +import static org.mockito.Mockito.times; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; import java.nio.file.Path; +import java.util.List; import java.util.Map; import java.util.UUID; import java.util.function.Consumer; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SessionCatalog; +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.rest.RESTCatalogAdapter.HTTPMethod; +import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.rest.responses.ErrorResponse; +import org.apache.iceberg.rest.responses.ListTablesResponse; +import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.view.ViewCatalogTests; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.server.handler.gzip.GzipHandler; @@ -39,7 +50,9 @@ 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.mockito.Mockito; public class TestRESTViewCatalog extends ViewCatalogTests { private static final ObjectMapper MAPPER = RESTObjectMapper.mapper(); @@ -144,6 +157,78 @@ public void closeCatalog() throws Exception { } } + @Test + public void testPaginationForListViews() { + RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); + RESTCatalog catalog = + new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); + catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); + + int numberOfItems = 30; + String namespaceName = "newdb"; + String viewName = "newview"; + + // create initial namespace + catalog().createNamespace(Namespace.of(namespaceName)); + + // create several views under namespace, based off a table for listing and verify + for (int i = 0; i < numberOfItems; i++) { + TableIdentifier viewIndentifier = TableIdentifier.of(namespaceName, viewName + i); + catalog + .buildView(viewIndentifier) + .withSchema(SCHEMA) + .withDefaultNamespace(viewIndentifier.namespace()) + .withQuery("spark", "select * from ns.tbl") + .create(); + } + List views = catalog.listViews(Namespace.of(namespaceName)); + assertThat(views).hasSize(numberOfItems); + + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/config"), + any(), + any(), + eq(ConfigResponse.class), + any(), + any()); + + Mockito.verify(adapter, times(numberOfItems)) + .execute( + eq(HTTPMethod.POST), + eq(String.format("v1/namespaces/%s/views", namespaceName)), + any(), + any(), + eq(LoadViewResponse.class), + any(), + any()); + + // verify initial request with empty pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_VIEWS), + eq(ImmutableMap.of("pageToken", "", "pageSize", "10", "namespace", namespaceName)), + any(), + eq(ListTablesResponse.class)); + + // verify second request with update pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_VIEWS), + eq(ImmutableMap.of("pageToken", "10", "pageSize", "10", "namespace", namespaceName)), + any(), + eq(ListTablesResponse.class)); + + // verify third request with update pageToken + Mockito.verify(adapter) + .handleRequest( + eq(RESTCatalogAdapter.Route.LIST_VIEWS), + eq(ImmutableMap.of("pageToken", "20", "pageSize", "10", "namespace", namespaceName)), + any(), + eq(ListTablesResponse.class)); + } + @Override protected RESTCatalog catalog() { return restCatalog; diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java index bfe5a662b219..d9ed801de0cb 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java @@ -34,11 +34,11 @@ public class TestListNamespacesResponse extends RequestResponseTestBase Date: Fri, 3 May 2024 15:24:24 +0530 Subject: [PATCH 0277/1019] Docs: Update features for Hive 4.0 (#10162) Co-authored-by: Sourabh Badhya --- docs/docs/hive.md | 297 ++++++++++++++++++++++++++++++++++++++++++---- 1 file changed, 276 insertions(+), 21 deletions(-) diff --git a/docs/docs/hive.md b/docs/docs/hive.md index a8df15440538..69c0f6279538 100644 --- a/docs/docs/hive.md +++ b/docs/docs/hive.md @@ -24,6 +24,21 @@ Iceberg supports reading and writing Iceberg tables through [Hive](https://hive. a [StorageHandler](https://cwiki.apache.org/confluence/display/Hive/StorageHandlers). ## Feature support +The following features matrix illustrates the support for different features across Hive releases for Iceberg tables - + +| Feature support | Hive 2 / 3 | Hive 4 | +|-----------------------------------------------------------------|------------|--------| +| [SQL create table](#create-table) | ✔️ | ✔️ | +| [SQL create table as select (CTAS)](#create-table-as-select) | ✔️ | ✔️ | +| [SQL create table like table (CTLT)](#create-table-like-table) | ✔️ | ✔️ | +| [SQL drop table](#drop-table) | ✔️ | ✔️ | +| [SQL insert into](#insert-into) | ✔️ | ✔️ | +| [SQL insert overwrite](#insert-overwrite) | ✔️ | ✔️ | +| [SQL delete from](#delete-from) | | ✔️ | +| [SQL update](#update) | | ✔️ | +| [SQL merge into](#merge-into) | | ✔️ | +| [Branches and tags](#branches-and-tags) | | ✔️ | + Iceberg compatibility with Hive 2.x and Hive 3.1.2/3 supports the following features: * Creating a table @@ -34,16 +49,7 @@ Iceberg compatibility with Hive 2.x and Hive 3.1.2/3 supports the following feat !!! warning DML operations work only with MapReduce execution engine. -The HiveCatalog supports the following additional features with Hive version 4.0.0-alpha-2 and above: - -* Altering a table with expiring snapshots. -* Create a table like an existing table (CTLT table) -* Support adding parquet compression type via Table properties [Compression types](https://spark.apache.org/docs/2.4.3/sql-data-sources-parquet.html#configuration) -* Altering a table metadata location -* Supporting table rollback -* Honors sort orders on existing tables when writing a table [Sort orders specification](../../spec.md#sort-orders) - -With Hive version 4.0.0-alpha-1 and above, the Iceberg integration when using HiveCatalog supports the following additional features: +Hive supports the following additional features with Hive version 4.0.0 and above: * Creating an Iceberg identity-partitioned table * Creating an Iceberg table with any partition spec, including the various transforms supported by Iceberg @@ -51,13 +57,31 @@ With Hive version 4.0.0-alpha-1 and above, the Iceberg integration when using Hi * Altering a table while keeping Iceberg and Hive schemas in sync * Altering the partition schema (updating columns) * Altering the partition schema by specifying partition transforms -* Truncating a table +* Truncating a table / partition, dropping a partition. * Migrating tables in Avro, Parquet, or ORC (Non-ACID) format to Iceberg -* Reading the schema of a table -* Querying Iceberg metadata tables -* Time travel applications -* Inserting into a table (INSERT INTO) -* Inserting data overwriting existing data (INSERT OVERWRITE) +* Reading the schema of a table. +* Querying Iceberg metadata tables. +* Time travel applications. +* Inserting into a table / partition (INSERT INTO). +* Inserting data overwriting existing data (INSERT OVERWRITE) in a table / partition. +* Copy-on-write support for delete, update and merge queries, CRUD support for Iceberg V1 tables. +* Altering a table with expiring snapshots. +* Create a table like an existing table (CTLT table) +* Support adding parquet compression type via Table properties [Compression types](https://spark.apache.org/docs/2.4.3/sql-data-sources-parquet.html#configuration) +* Altering a table metadata location. +* Supporting table rollback. +* Honors sort orders on existing tables when writing a table [Sort orders specification](../../spec.md#sort-orders) +* Creating, writing to and dropping an Iceberg branch / tag. +* Allowing expire snapshots by Snapshot ID, by time range, by retention of last N snapshots and using table properties. +* Set current snapshot using snapshot ID for an Iceberg table. +* Support for renaming an Iceberg table. +* Altering a table to convert to an Iceberg table. +* Fast forwarding, cherry-picking commit to an Iceberg branch. +* Creating a branch from an Iceberg tag. +* Set current snapshot using branch/tag for an Iceberg table. +* Delete orphan files for an Iceberg table. +* Allow full table compaction of Iceberg tables. +* Support of showing partition information for Iceberg tables (SHOW PARTITIONS). !!! warning DML operations work only with Tez execution engine. @@ -67,6 +91,10 @@ With Hive version 4.0.0-alpha-1 and above, the Iceberg integration when using Hi Hive 4 comes with `hive-iceberg` that ships Iceberg, so no additional downloads or jars are needed. For older versions of Hive a runtime jar has to be added. +### Hive 4.0.0 + +Hive 4.0.0 comes with the Iceberg 1.4.3 included. + ### Hive 4.0.0-beta-1 Hive 4.0.0-beta-1 comes with the Iceberg 1.3.0 included. @@ -388,6 +416,11 @@ ALTER TABLE t SET TBLPROPERTIES('...'='...'); The Hive table schema is kept in sync with the Iceberg table. If an outside source (Impala/Spark/Java API/etc) changes the schema, the Hive table immediately reflects the changes. You alter the table schema using Hive commands: +* Rename a table +```sql +ALTER TABLE orders RENAME TO renamed_orders; +``` + * Add a column ```sql ALTER TABLE orders ADD COLUMNS (nickname string); @@ -431,12 +464,121 @@ ALTER TABLE t SET TBLPROPERTIES ('storage_handler'='org.apache.iceberg.mr.hive.H During the migration the data files are not changed, only the appropriate Iceberg metadata files are created. After the migration, handle the table as a normal Iceberg table. +#### Drop partitions +You can drop partitions based on a single / multiple partition specification using the following commands: +```sql +ALTER TABLE orders DROP PARTITION (buy_date == '2023-01-01', market_price > 1000), PARTITION (buy_date == '2024-01-01', market_price <= 2000); +``` +The partition specification supports only identity-partition columns. Transform columns in partition specification are not supported. + +#### Branches and tags + +`ALTER TABLE ... CREATE BRANCH` + +Branches can be created via the CREATE BRANCH statement with the following options: + +* Create a branch using default properties. +* Create a branch at a specific snapshot ID. +* Create a branch using system time. +* Create a branch with a specified number of snapshot retentions. +* Create a branch using specific tag. + +```sql +-- CREATE branch1 with default properties. +ALTER TABLE test CREATE BRANCH branch1; + +-- CREATE branch1 at a specific snapshot ID. +ALTER TABLE test CREATE BRANCH branch1 FOR SYSTEM_VERSION AS OF 3369973735913135680; + +-- CREATE branch1 using system time. +ALTER TABLE test CREATE BRANCH branch1 FOR SYSTEM_TIME AS OF '2023-09-16 09:46:38.939 Etc/UTC'; + +-- CREATE branch1 with a specified number of snapshot retentions. +ALTER TABLE test CREATE BRANCH branch1 FOR SYSTEM_VERSION AS OF 3369973735913135680 WITH SNAPSHOT RETENTION 5 SNAPSHOTS; + +-- CREATE branch1 using a specific tag. +ALTER TABLE test CREATE BRANCH branch1 FOR TAG AS OF tag1; +``` + +`ALTER TABLE ... CREATE TAG` + +Tags can be created via the CREATE TAG statement with the following options: + +* Create a tag using default properties. +* Create a tag at a specific snapshot ID. +* Create a tag using system time. + +```sql +-- CREATE tag1 with default properties. +ALTER TABLE test CREATE TAG tag1; + +-- CREATE tag1 at a specific snapshot ID. +ALTER TABLE test CREATE TAG tag1 FOR SYSTEM_VERSION AS OF 3369973735913135680; + +-- CREATE tag1 using system time. +ALTER TABLE test CREATE TAG tag1 FOR SYSTEM_TIME AS OF '2023-09-16 09:46:38.939 Etc/UTC'; +``` + +`ALTER TABLE ... DROP BRANCH` + +Branches can be dropped via the DROP BRANCH statement with the following options: + +* Do not fail if the branch does not exist with IF EXISTS + +```sql +-- DROP branch1 +ALTER TABLE test DROP BRANCH branch1; + +-- DROP branch1 IF EXISTS +ALTER TABLE test DROP BRANCH IF EXISTS branch1; +``` + +`ALTER TABLE ... DROP TAG` + +Tags can be dropped via the DROP TAG statement with the following options: + +* Do not fail if the tag does not exist with IF EXISTS + +```sql +-- DROP tag1 +ALTER TABLE test DROP TAG tag1; + +-- DROP tag1 IF EXISTS +ALTER TABLE test DROP TAG IF EXISTS tag1; +``` + +`ALTER TABLE ... EXECUTE FAST-FORWARD` + +An iceberg branch which is an ancestor of another branch can be fast-forwarded to the state of the other branch. + +```sql +-- This fast-forwards the branch1 to the state of main branch of the Iceberg table. +ALTER table test EXECUTE FAST-FORWARD 'branch1' 'main'; + +-- This fast-forwards the branch1 to the state of branch2. +ALTER table test EXECUTE FAST-FORWARD 'branch1' 'branch2'; +``` + +#### `ALTER TABLE ... EXECUTE CHERRY-PICK` + +Cherry-pick of a snapshot requires the ID of the snapshot. Cherry-pick of snapshots as of now is supported only on the main branch of an Iceberg table. + +```sql + ALTER table test EXECUTE CHERRY-PICK 8602659039622823857; +``` + ### TRUNCATE TABLE The following command truncates the Iceberg table: ```sql TRUNCATE TABLE t; ``` -Using a partition specification is not allowed. + +#### TRUNCATE TABLE ... PARTITION +The following command truncates the partition in an Iceberg table: +```sql +TRUNCATE TABLE orders PARTITION (customer_id = 1, first_name = 'John'); +``` +The partition specification supports only identity-partition columns. Transform columns in partition specification are not supported. ### DROP TABLE @@ -476,6 +618,12 @@ Here are the features highlights for Iceberg Hive read support: Some of the advanced / little used optimizations are not yet implemented for Iceberg tables, so you should check your individual queries. Also currently the statistics stored in the MetaStore are used for query planning. This is something we are planning to improve in the future. +Hive 4 supports select operations on branches which also work similar to the table level select operations. However, the branch must be provided as follows - +```sql +-- Branches should be specified as ..branch_ +SELECT * FROM default.test.branch_branch1; +``` + ### INSERT INTO Hive supports the standard single-table INSERT INTO operation: @@ -491,6 +639,15 @@ Multi-table insert is also supported, but it will not be atomic. Commits occur o Partial changes will be visible during the commit process and failures can leave partial changes committed. Changes within a single table will remain atomic. +Insert-into operations on branches also work similar to the table level select operations. However, the branch must be provided as follows - +```sql +-- Branches should be specified as ..branch_ +INSERT INTO default.test.branch_branch1 +VALUES ('a', 1); +INSERT INTO default.test.branch_branch1 +SELECT...; +``` + Here is an example of inserting into multiple tables at once in Hive SQL: ```sql @@ -499,6 +656,18 @@ FROM customers INSERT INTO target2 SELECT last_name, customer_id; ``` +#### INSERT INTO ... PARTITION + +Hive 4 supports partition-level INSERT INTO operation: + +```sql +INSERT INTO table_a PARTITION (customer_id = 1, first_name = 'John') +VALUES (1,2); +INSERT INTO table_a PARTITION (customer_id = 1, first_name = 'John') +SELECT...; +``` +The partition specification supports only identity-partition columns. Transform columns in partition specification are not supported. + ### INSERT OVERWRITE INSERT OVERWRITE can replace data in the table with the result of a query. Overwrites are atomic operations for Iceberg tables. For nonpartitioned tables the content of the table is always removed. For partitioned tables the partitions @@ -507,6 +676,69 @@ that have rows produced by the SELECT query will be replaced. INSERT OVERWRITE TABLE target SELECT * FROM source; ``` +#### INSERT OVERWRITE ... PARTITION + +Hive 4 supports partition-level INSERT OVERWRITE operation: + +```sql +INSERT OVERWRITE TABLE target PARTITION (customer_id = 1, first_name = 'John') SELECT * FROM source; +``` +The partition specification supports only identity-partition columns. Transform columns in partition specification are not supported. + +### DELETE FROM + +Hive 4 supports DELETE FROM queries to remove data from tables. + +Delete queries accept a filter to match rows to delete. + +```sql +DELETE FROM target WHERE id > 1 AND id < 10; + +DELETE FROM target WHERE id IN (SELECT id FROM source); + +DELETE FROM target WHERE id IN (SELECT min(customer_id) FROM source); +``` +If the delete filter matches entire partitions of the table, Iceberg will perform a metadata-only delete. If the filter matches individual rows of a table, then Iceberg will rewrite only the affected data files. + +### UPDATE + +Hive 4 supports UPDATE queries which accept a filter to match rows to update. + +```sql +UPDATE target SET first_name = 'Raj' WHERE id > 1 AND id < 10; + +UPDATE target SET first_name = 'Raj' WHERE id IN (SELECT id FROM source); + +UPDATE target SET first_name = 'Raj' WHERE id IN (SELECT min(customer_id) FROM source); +``` +For more complex row-level updates based on incoming data, see the section on MERGE INTO. + +### MERGE INTO + +Hive 4 added support for MERGE INTO queries that can express row-level updates. + +MERGE INTO updates a table, called the target table, using a set of updates from another query, called the source. The update for a row in the target table is found using the ON clause that is like a join condition. + +```sql +MERGE INTO target AS t -- a target table +USING source s -- the source updates +ON t.id = s.id -- condition to find updates for target rows +WHEN ... -- updates +``` + +Updates to rows in the target table are listed using WHEN MATCHED ... THEN .... Multiple MATCHED clauses can be added with conditions that determine when each match should be applied. The first matching expression is used. +```sql +WHEN MATCHED AND s.op = 'delete' THEN DELETE +WHEN MATCHED AND t.count IS NULL AND s.op = 'increment' THEN UPDATE SET t.count = 0 +WHEN MATCHED AND s.op = 'increment' THEN UPDATE SET t.count = t.count + 1 +``` + +Source rows (updates) that do not match can be inserted: +```sql +WHEN NOT MATCHED THEN INSERT VALUES (s.a, s.b, s.c) +``` +Only one record in the source data can update any given row of the target table, or else an error will be thrown. + ### QUERYING METADATA TABLES Hive supports querying of the Iceberg Metadata tables. The tables could be used as normal Hive tables, so it is possible to use projections / joins / filters / etc. @@ -514,11 +746,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: -* files -* entries + +* all_data_files +* all_delete_files +* all_entries all_files +* all_manifests +* data_files +* delete_files +* entries +* files +* manifests +* metadata_log_entries +* partitions +* refs * snapshots -* manifests -* partitions ```sql SELECT * FROM default.table_a.files; @@ -594,3 +835,17 @@ Rollback to a specific snapshot ID ```sql 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'; + +-- Using the OPTIMIZE TABLE ... REWRITE DATA syntax +OPTIMIZE TABLE t REWRITE DATA; +``` +Both these syntax have the same effect of performing full table compaction on an Iceberg table. From 1cc74ffe00f188b78631255d9383e48eca88aec8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 3 May 2024 12:27:50 +0200 Subject: [PATCH 0278/1019] Build: Bump org.xerial:sqlite-jdbc from 3.45.2.0 to 3.45.3.0 (#10194) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.45.2.0 to 3.45.3.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.45.2.0...3.45.3.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 cfade621366a..bed5dfeb8e09 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.34" -sqlite-jdbc = "3.45.2.0" +sqlite-jdbc = "3.45.3.0" testcontainers = "1.19.7" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 82ecaa401e718f72d1078038ff892f825aa43cfa Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 5 May 2024 11:46:16 +0200 Subject: [PATCH 0279/1019] Build: Bump mkdocs-material from 9.5.19 to 9.5.21 (#10272) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.19 to 9.5.21. - [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.19...9.5.21) --- 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 8fdef1d27c37..db6ddb394d22 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.19 +mkdocs-material==9.5.21 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 9eccb6f5c98e97fb20bf2bfd882fe849301175a1 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 6 May 2024 16:41:09 +0800 Subject: [PATCH 0280/1019] docs: Remove link to Flink unit test (#10160) --- .gitignore | 1 + docs/docs/flink-writes.md | 2 -- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/.gitignore b/.gitignore index 5d3147535b87..6740a061e346 100644 --- a/.gitignore +++ b/.gitignore @@ -24,6 +24,7 @@ gradle/wrapper/gradle-wrapper.jar lib/ # web site build +docs/site/ site/site/ site/docs/docs/ site/docs/.asf.yaml diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index 93ef1cf1b7de..b916a5f9b7b0 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -103,8 +103,6 @@ FlinkSink.forRowData(input) env.execute("Test Iceberg DataStream"); ``` -The iceberg API also allows users to write generic `DataStream` to iceberg table, more example could be found in this [unit test](https://github.com/apache/iceberg/blob/main/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java). - ### Overwrite data Set the `overwrite` flag in FlinkSink builder to overwrite the data in existing iceberg tables: From 2202a79ee5b5fd4ff53b394b1c988843c4d7d110 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 6 May 2024 10:42:01 +0200 Subject: [PATCH 0281/1019] Build: Bump nessie from 0.80.0 to 0.81.1 (#10267) Bumps `nessie` from 0.80.0 to 0.81.1. Updates `org.projectnessie.nessie:nessie-client` from 0.80.0 to 0.81.1 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.80.0 to 0.81.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.80.0 to 0.81.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.80.0 to 0.81.1 --- 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 bed5dfeb8e09..ec240fba8339 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.80.0" +nessie = "0.81.1" netty-buffer = "4.1.109.Final" netty-buffer-compat = "4.1.109.Final" object-client-bundle = "3.3.2" From b3112a949bbaf5729233cd8b5848781b0279fafd Mon Sep 17 00:00:00 2001 From: lurnagao-dahua <91278331+lurnagao-dahua@users.noreply.github.com> Date: Tue, 7 May 2024 21:31:22 +0800 Subject: [PATCH 0282/1019] MR: Fix using Date type as partition field (#10210) --- .../mr/hive/HiveIcebergRecordWriter.java | 5 +- ...stHiveIcebergStorageHandlerWithEngine.java | 51 +++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) 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 793b9c5e6448..f87d79b5537e 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 @@ -30,6 +30,7 @@ import org.apache.iceberg.PartitionKey; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; +import org.apache.iceberg.data.InternalRecordWrapper; import org.apache.iceberg.data.Record; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.io.FileIO; @@ -50,6 +51,7 @@ class HiveIcebergRecordWriter extends PartitionedFanoutWriter // The current key is reused at every write to avoid unnecessary object creation private final PartitionKey currentKey; private final FileIO io; + private final InternalRecordWrapper wrapper; // > map to store the active writers // Stored in concurrent map, since some executor engines can share containers @@ -77,13 +79,14 @@ static Map getWriters(TaskAttemptID taskAttempt super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.io = io; this.currentKey = new PartitionKey(spec, schema); + this.wrapper = new InternalRecordWrapper(schema.asStruct()); writers.putIfAbsent(taskAttemptID, Maps.newConcurrentMap()); writers.get(taskAttemptID).put(tableName, this); } @Override protected PartitionKey partition(Record row) { - currentKey.partition(row); + currentKey.partition(wrapper.wrap(row)); return currentKey; } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java index c8e91de9b859..b080f4bd49f4 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java @@ -25,6 +25,8 @@ import java.io.IOException; import java.nio.file.Path; +import java.time.LocalDate; +import java.time.LocalDateTime; import java.util.Collection; import java.util.List; import java.util.Map; @@ -1206,6 +1208,55 @@ public void testRemoveAndAddBackColumnFromIcebergTable() throws IOException { 0); } + @TestTemplate + public void testWriteWithDatePartition() { + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); + + Schema dateSchema = + new Schema( + optional(1, "id", Types.LongType.get()), + optional(2, "part_field", Types.DateType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(dateSchema).identity("part_field").build(); + List records = + TestHelper.RecordsBuilder.newInstance(dateSchema) + .add(1L, LocalDate.of(2023, 1, 21)) + .add(2L, LocalDate.of(2023, 1, 22)) + .add(3L, LocalDate.of(2022, 1, 21)) + .build(); + testTables.createTable(shell, "part_test", dateSchema, spec, FileFormat.PARQUET, records); + List result = shell.executeStatement("SELECT * from part_test order by id"); + + assertThat(result).hasSameSizeAs(records); + assertThat(result.get(0)[1]).isEqualTo("2023-01-21"); + assertThat(result.get(1)[1]).isEqualTo("2023-01-22"); + assertThat(result.get(2)[1]).isEqualTo("2022-01-21"); + } + + @TestTemplate + public void testWriteWithTimestampPartition() throws IOException { + assumeThat(executionEngine).as("Tez write is not implemented yet").isEqualTo("mr"); + + Schema dateSchema = + new Schema( + optional(1, "id", Types.LongType.get()), + optional(2, "part_field", Types.TimestampType.withoutZone())); + PartitionSpec spec = PartitionSpec.builderFor(dateSchema).identity("part_field").build(); + List records = + TestHelper.RecordsBuilder.newInstance(dateSchema) + .add(1L, LocalDateTime.of(2023, 1, 21, 21, 10, 10, 100000000)) + .add(2L, LocalDateTime.of(2023, 1, 21, 22, 10, 10, 200000000)) + .add(3L, LocalDateTime.of(2023, 1, 22, 21, 10, 10, 300000000)) + .build(); + testTables.createTable(shell, "part_test", dateSchema, spec, FileFormat.PARQUET, records); + List result = shell.executeStatement("SELECT * from part_test order by id"); + + assertThat(result).hasSameSizeAs(records); + assertThat(result.get(0)[1]).isEqualTo("2023-01-21 21:10:10.1"); + assertThat(result.get(1)[1]).isEqualTo("2023-01-21 22:10:10.2"); + assertThat(result.get(2)[1]).isEqualTo("2023-01-22 21:10:10.3"); + } + /** * Checks if the certain type is an unsupported vectorized types in Hive 3.1.2 * From f2efc7015a860a7205147fb574de8a267ac27f69 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 9 May 2024 07:01:18 -0600 Subject: [PATCH 0283/1019] Docs: Update site to 1.5.2 docs (#10291) --- site/mkdocs.yml | 2 +- site/nav.yml | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index db3e1aef9709..899851984cad 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.5.1' + icebergVersion: '1.5.2' nessieVersion: '0.77.1' social: - icon: fontawesome/regular/comments diff --git a/site/nav.yml b/site/nav.yml index 2de2913d53cf..014e47b3a015 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.5.2: '!include docs/docs/1.5.2/mkdocs.yml' - 1.5.1: '!include docs/docs/1.5.1/mkdocs.yml' - 1.5.0: '!include docs/docs/1.5.0/mkdocs.yml' - 1.4.3: '!include docs/docs/1.4.3/mkdocs.yml' From 2ae71e8d3a03a563848030429ebb6bc91e994d41 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 9 May 2024 15:21:39 +0200 Subject: [PATCH 0284/1019] Build: Bump software.amazon.awssdk:bom from 2.25.40 to 2.25.45 (#10266) Bumps software.amazon.awssdk:bom from 2.25.40 to 2.25.45. --- 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 ec240fba8339..20d3a8926a5c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.40" +awssdk-bom = "2.25.45" azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 69cfd8db7ec2cf038b4b804f53917159151f992e Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 9 May 2024 11:42:37 -0600 Subject: [PATCH 0285/1019] Infra: Add Iceberg 1.5.2 to issue template (#10296) --- .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 7c584618cb69..a6d4fc7e5c2e 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.5.1 (latest release)" + - "1.5.2 (latest release)" + - "1.5.1" - "1.5.0" - "1.4.3" - "1.4.2" From c829fb050beb157232cdf0bc7fcfba9ed6a11fd0 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 9 May 2024 11:43:35 -0600 Subject: [PATCH 0286/1019] Update doap.rdf for 1.5.2 release (#10297) --- doap.rdf | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/doap.rdf b/doap.rdf index 8ee99ef757a5..7f49fc42f793 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.5.1 - 2024-04-25 - 1.5.1 + 1.5.2 + 2024-05-09 + 1.5.2 From 959cc70ba2db4a8d72160ed4bbc137e8237c19dd Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 9 May 2024 14:06:31 -0600 Subject: [PATCH 0287/1019] Docs: Add release notes for 1.5.2 (#10295) --- site/docs/releases.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/site/docs/releases.md b/site/docs/releases.md index 1c00d0e2eee1..6ba0b481fce9 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -71,6 +71,12 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ... ``` +### 1.5.2 release +Apache Iceberg 1.5.2 was released on May 9, 2024. + +The 1.5.2 release has the same changes that the 1.5.1 release (see directly below) +has. The 1.5.1 release had issues with the spark runtime artifacts; specifically certain artifacts were built with the wrong Scala version. +It is strongly recommended to upgrade to 1.5.2 for any systems that are using 1.5.1. ### 1.5.1 release Apache Iceberg 1.5.1 was released on April 25, 2024. From 78a9fa6db5286bdf49006dd85989be8098f3c883 Mon Sep 17 00:00:00 2001 From: Dustin Metzgar Date: Thu, 9 May 2024 17:11:54 -0700 Subject: [PATCH 0288/1019] Spec: Fix markdown for struct evolution default value rules (#10290) --- format/spec.md | 1 + 1 file changed, 1 insertion(+) diff --git a/format/spec.md b/format/spec.md index b00c63256a4e..6e23b2c5834b 100644 --- a/format/spec.md +++ b/format/spec.md @@ -228,6 +228,7 @@ Any struct, including a top-level schema, can evolve through deleting fields, ad Grouping a subset of a struct’s fields into a nested struct is **not** allowed, nor is moving fields from a nested struct into its immediate parent struct (`struct ↔ struct>`). Evolving primitive types to structs is **not** allowed, nor is evolving a single-field struct to a primitive (`map ↔ map>`). Struct evolution requires the following rules for default values: + * The `initial-default` must be set when a field is added and cannot change * The `write-default` must be set when a field is added and may change * When a required field is added, both defaults must be set to a non-null value From 22c130e540855c8426641898221c6bdbc24615b7 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 10 May 2024 11:18:46 -0600 Subject: [PATCH 0289/1019] Core: Retry connections in JDBC catalog with user configured error code list (#10140) --- .../org/apache/iceberg/ClientPoolImpl.java | 47 ++++-- .../org/apache/iceberg/jdbc/JdbcCatalog.java | 6 + .../apache/iceberg/jdbc/JdbcClientPool.java | 44 +++++- .../org/apache/iceberg/jdbc/JdbcUtil.java | 2 + .../apache/iceberg/TestClientPoolImpl.java | 141 ++++++++++++++++++ .../apache/iceberg/jdbc/TestJdbcCatalog.java | 55 +++++++ 6 files changed, 282 insertions(+), 13 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java diff --git a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java index e8ab57fed3e2..4c4469544816 100644 --- a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java +++ b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java @@ -34,16 +34,29 @@ public abstract class ClientPoolImpl private final Class reconnectExc; private final Object signal = new Object(); private final boolean retryByDefault; + private final int maxRetries; + private volatile int currentSize; private boolean closed; + private int connectionRetryWaitPeriodMs = 1000; + public ClientPoolImpl(int poolSize, Class reconnectExc, boolean retryByDefault) { + this(poolSize, reconnectExc, retryByDefault, 1); + } + + public ClientPoolImpl( + int poolSize, + Class reconnectExc, + boolean retryByDefault, + int maxConnectionRetries) { this.poolSize = poolSize; this.reconnectExc = reconnectExc; this.clients = new ArrayDeque<>(poolSize); this.currentSize = 0; this.closed = false; this.retryByDefault = retryByDefault; + this.maxRetries = maxConnectionRetries; } @Override @@ -56,26 +69,38 @@ public R run(Action action, boolean retry) throws E, InterruptedExc C client = get(); try { return action.run(client); - } catch (Exception exc) { - if (retry && isConnectionException(exc)) { - try { - client = reconnect(client); - } catch (Exception ignored) { - // if reconnection throws any exception, rethrow the original failure - throw reconnectExc.cast(exc); - } - - return action.run(client); + if (!retry || !isConnectionException(exc)) { + throw exc; } - throw exc; + return retryAction(action, exc, client); } finally { release(client); } } + private R retryAction(Action action, Exception originalFailure, C client) + throws E, InterruptedException { + int retryAttempts = 0; + while (retryAttempts < maxRetries) { + try { + C reconnectedClient = reconnect(client); + return action.run(reconnectedClient); + } catch (Exception exc) { + if (isConnectionException(exc)) { + retryAttempts++; + Thread.sleep(connectionRetryWaitPeriodMs); + } else { + throw reconnectExc.cast(originalFailure); + } + } + } + + throw reconnectExc.cast(originalFailure); + } + protected abstract C newClient(); protected abstract C reconnect(C client); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 71590e7618a2..4e10ee96d125 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -55,6 +55,7 @@ import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; +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.base.Strings; @@ -689,6 +690,11 @@ public void renameView(TableIdentifier from, TableIdentifier to) { } } + @VisibleForTesting + JdbcClientPool connectionPool() { + return connections; + } + private int execute(String sql, String... args) { return execute(err -> {}, sql, args); } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java index 60e5eb49a4f8..487b8409b1cc 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java @@ -21,17 +21,40 @@ import java.sql.Connection; import java.sql.DriverManager; import java.sql.SQLException; -import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTransientException; +import java.util.Arrays; import java.util.Map; import java.util.Properties; +import java.util.Set; +import java.util.stream.Collectors; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.ClientPoolImpl; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; public class JdbcClientPool extends ClientPoolImpl { + /** + * The following are common retryable SQLSTATEs error codes which are generic across vendors. + * + *
      + *
    • 08000: Generic Connection Exception + *
    • 08003: Connection does not exist + *
    • 08006: Connection failure + *
    • 08007: Transaction resolution unknown + *
    • 40001: Serialization failure due to deadlock + *
    + * + * See https://en.wikipedia.org/wiki/SQLSTATE for more details. + */ + static final Set COMMON_RETRYABLE_CONNECTION_SQL_STATES = + ImmutableSet.of("08000", "08003", "08006", "08007", "40001"); + private final String dbUrl; private final Map properties; + private Set retryableStatusCodes; + public JdbcClientPool(String dbUrl, Map props) { this( Integer.parseInt( @@ -43,8 +66,18 @@ public JdbcClientPool(String dbUrl, Map props) { } public JdbcClientPool(int poolSize, String dbUrl, Map props) { - super(poolSize, SQLNonTransientConnectionException.class, true); + super(poolSize, SQLTransientException.class, true); properties = props; + retryableStatusCodes = Sets.newHashSet(); + retryableStatusCodes.addAll(COMMON_RETRYABLE_CONNECTION_SQL_STATES); + String configuredRetryableStatuses = props.get(JdbcUtil.RETRYABLE_STATUS_CODES); + if (configuredRetryableStatuses != null) { + retryableStatusCodes.addAll( + Arrays.stream(configuredRetryableStatuses.split(",")) + .map(status -> status.replaceAll("\\s+", "")) + .collect(Collectors.toSet())); + } + this.dbUrl = dbUrl; } @@ -72,4 +105,11 @@ protected void close(Connection client) { throw new UncheckedSQLException(e, "Failed to close connection"); } } + + @Override + protected boolean isConnectionException(Exception e) { + return super.isConnectionException(e) + || (e instanceof SQLException + && retryableStatusCodes.contains(((SQLException) e).getSQLState())); + } } diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java index 749c2d485f0f..c9bd2b78a628 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcUtil.java @@ -43,6 +43,8 @@ final class JdbcUtil { static final String INIT_CATALOG_TABLES_PROPERTY = JdbcCatalog.PROPERTY_PREFIX + "init-catalog-tables"; + static final String RETRYABLE_STATUS_CODES = "retryable_status_codes"; + enum SchemaVersion { V0, V1 diff --git a/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java new file mode 100644 index 000000000000..8d62afa17678 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java @@ -0,0 +1,141 @@ +/* + * 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.AssertionsForClassTypes.assertThatThrownBy; + +import org.junit.jupiter.api.Test; + +public class TestClientPoolImpl { + + @Test + public void testRetrySucceedsWithinMaxAttempts() throws Exception { + int maxRetries = 5; + int succeedAfterAttempts = 3; + try (MockClientPoolImpl mockClientPool = + new MockClientPoolImpl(2, RetryableException.class, true, maxRetries)) { + int actions = mockClientPool.run(client -> client.succeedAfter(succeedAfterAttempts)); + assertThat(actions) + .as("There should be exactly one successful action invocation") + .isEqualTo(1); + assertThat(mockClientPool.reconnectionAttempts()).isEqualTo(succeedAfterAttempts - 1); + } + } + + @Test + public void testRetriesExhaustedAndSurfacesFailure() { + int maxRetries = 3; + int succeedAfterAttempts = 5; + try (MockClientPoolImpl mockClientPool = + new MockClientPoolImpl(2, RetryableException.class, true, maxRetries)) { + assertThatThrownBy( + () -> mockClientPool.run(client -> client.succeedAfter(succeedAfterAttempts))) + .isInstanceOf(RetryableException.class); + assertThat(mockClientPool.reconnectionAttempts()).isEqualTo(maxRetries); + } + } + + @Test + public void testNoRetryingNonRetryableException() { + try (MockClientPoolImpl mockClientPool = + new MockClientPoolImpl(2, RetryableException.class, true, 3)) { + assertThatThrownBy(() -> mockClientPool.run(MockClient::failWithNonRetryable, true)) + .isInstanceOf(NonRetryableException.class); + assertThat(mockClientPool.reconnectionAttempts()).isEqualTo(0); + } + } + + @Test + public void testNoRetryingWhenDisabled() { + try (MockClientPoolImpl mockClientPool = + new MockClientPoolImpl(2, RetryableException.class, false, 3)) { + assertThatThrownBy(() -> mockClientPool.run(client -> client.succeedAfter(3))) + .isInstanceOf(RetryableException.class); + assertThat(mockClientPool.reconnectionAttempts()).isEqualTo(0); + } + } + + static class RetryableException extends RuntimeException {} + + static class NonRetryableException extends RuntimeException {} + + static class MockClient { + boolean closed = false; + + int actions = 0; + + int retryableFailures = 0; + + public void close() { + closed = true; + } + + public int successfulAction() { + actions++; + return actions; + } + + int succeedAfter(int succeedAfterAttempts) { + if (retryableFailures == succeedAfterAttempts - 1) { + return successfulAction(); + } + + retryableFailures++; + throw new RetryableException(); + } + + int failWithNonRetryable() { + throw new NonRetryableException(); + } + } + + static class MockClientPoolImpl extends ClientPoolImpl { + + private int reconnectionAttempts; + + MockClientPoolImpl( + int poolSize, + Class reconnectExc, + boolean retryByDefault, + int numRetries) { + super(poolSize, reconnectExc, retryByDefault, numRetries); + } + + @Override + protected MockClient newClient() { + return new MockClient(); + } + + @Override + protected MockClient reconnect(MockClient client) { + reconnectionAttempts++; + return client; + } + + @Override + protected void close(MockClient client) { + client.close(); + } + + int reconnectionAttempts() { + return reconnectionAttempts; + } + } +} 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 985c84f0dc32..90492b51097a 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -34,6 +34,7 @@ import java.sql.DatabaseMetaData; import java.sql.ResultSet; import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; import java.util.List; import java.util.Map; import java.util.Set; @@ -208,6 +209,60 @@ public void testEnableInitCatalogTablesOverridesDefault() throws Exception { assertThat(catalogTablesExist(jdbcUrl)).isTrue(); } + @Test + public void testRetryingErrorCodesProperty() { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, "jdbc:sqlite:file::memory:?icebergDB"); + properties.put(JdbcUtil.RETRYABLE_STATUS_CODES, "57000,57P03,57P04"); + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.setConf(conf); + jdbcCatalog.initialize("test_catalog_with_retryable_status_codes", properties); + JdbcClientPool jdbcClientPool = jdbcCatalog.connectionPool(); + List expectedRetryableExceptions = + Lists.newArrayList( + new SQLException("operator_intervention", "57000"), + new SQLException("cannot_connect_now", "57P03"), + new SQLException("database_dropped", "57P04")); + JdbcClientPool.COMMON_RETRYABLE_CONNECTION_SQL_STATES.forEach( + code -> expectedRetryableExceptions.add(new SQLException("some failure", code))); + + expectedRetryableExceptions.forEach( + exception -> { + assertThat(jdbcClientPool.isConnectionException(exception)) + .as(String.format("%s status should be retryable", exception.getSQLState())) + .isTrue(); + }); + + // Test the same retryable status codes but with spaces in the configuration + properties.put(JdbcUtil.RETRYABLE_STATUS_CODES, "57000, 57P03, 57P04"); + jdbcCatalog.initialize("test_catalog_with_retryable_status_codes_with_spaces", properties); + JdbcClientPool updatedClientPool = jdbcCatalog.connectionPool(); + expectedRetryableExceptions.forEach( + exception -> { + assertThat(updatedClientPool.isConnectionException(exception)) + .as(String.format("%s status should be retryable", exception.getSQLState())) + .isTrue(); + }); + } + + @Test + public void testSqlNonTransientExceptionNotRetryable() { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, this.tableDir.toAbsolutePath().toString()); + properties.put(CatalogProperties.URI, "jdbc:sqlite:file::memory:?icebergDB"); + properties.put(JdbcUtil.RETRYABLE_STATUS_CODES, "57000,57P03,57P04"); + JdbcCatalog jdbcCatalog = new JdbcCatalog(); + jdbcCatalog.setConf(conf); + jdbcCatalog.initialize("test_catalog_with_retryable_status_codes", properties); + JdbcClientPool jdbcClientPool = jdbcCatalog.connectionPool(); + Assertions.assertThat( + jdbcClientPool.isConnectionException( + new SQLNonTransientConnectionException("Failed to authenticate"))) + .as("SQL Non Transient exception is not retryable") + .isFalse(); + } + @Test public void testInitSchemaV0() { Map properties = Maps.newHashMap(); From 17c1094033d67b985f52d646e0ca6f41d0834083 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 11 May 2024 10:05:23 +0200 Subject: [PATCH 0290/1019] Build: Bump guava from 33.1.0-jre to 33.2.0-jre (#10271) Bumps `guava` from 33.1.0-jre to 33.2.0-jre. Updates `com.google.guava:guava` from 33.1.0-jre to 33.2.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.1.0-jre to 33.2.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 20d3a8926a5c..3bb887845bba 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -42,7 +42,7 @@ flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} google-libraries-bom = "26.28.0" -guava = "33.1.0-jre" +guava = "33.2.0-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" httpcomponents-httpclient5 = "5.3.1" From cbac143535cadbc07b90a902756bee810d40abcc Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sat, 11 May 2024 09:55:09 -0700 Subject: [PATCH 0291/1019] Spark 3.5: Remove obsolete conf parsing logic (#10309) --- .../apache/iceberg/spark/SparkConfParser.java | 28 +++++++++++-------- .../apache/iceberg/spark/SparkReadConf.java | 2 -- .../iceberg/spark/TestSparkWriteConf.java | 9 ++++++ 3 files changed, 25 insertions(+), 14 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java index e3b01b8375b6..bdb66bbe3c61 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java @@ -20,7 +20,6 @@ import java.time.Duration; import java.util.List; -import java.util.Locale; import java.util.Map; import java.util.function.Function; import org.apache.iceberg.Table; @@ -31,23 +30,24 @@ import org.apache.spark.sql.RuntimeConfig; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; class SparkConfParser { private final Map properties; private final RuntimeConfig sessionConf; - private final Map options; + private final CaseInsensitiveStringMap options; SparkConfParser() { this.properties = ImmutableMap.of(); this.sessionConf = new RuntimeConfig(SQLConf.get()); - this.options = ImmutableMap.of(); + this.options = CaseInsensitiveStringMap.empty(); } SparkConfParser(SparkSession spark, Table table, Map options) { this.properties = table.properties(); this.sessionConf = spark.conf(); - this.options = options; + this.options = asCaseInsensitiveStringMap(options); } public BooleanConfParser booleanConf() { @@ -70,6 +70,14 @@ public DurationConfParser durationConf() { return new DurationConfParser(); } + private static CaseInsensitiveStringMap asCaseInsensitiveStringMap(Map map) { + if (map instanceof CaseInsensitiveStringMap) { + return (CaseInsensitiveStringMap) map; + } else { + return new CaseInsensitiveStringMap(map); + } + } + class BooleanConfParser extends ConfParser { private Boolean defaultValue; private boolean negate = false; @@ -220,14 +228,10 @@ public ThisT tableProperty(String name) { } protected T parse(Function conversion, T defaultValue) { - if (!optionNames.isEmpty()) { - for (String optionName : optionNames) { - // use lower case comparison as DataSourceOptions.asMap() in Spark 2 returns a lower case - // map - String optionValue = options.get(optionName.toLowerCase(Locale.ROOT)); - if (optionValue != null) { - return conversion.apply(optionValue); - } + for (String optionName : optionNames) { + String optionValue = options.get(optionName); + if (optionValue != null) { + return conversion.apply(optionValue); } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index 2990d981d009..f5127e9cda20 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -57,7 +57,6 @@ public class SparkReadConf { private final SparkSession spark; private final Table table; private final String branch; - private final Map readOptions; private final SparkConfParser confParser; public SparkReadConf(SparkSession spark, Table table, Map readOptions) { @@ -69,7 +68,6 @@ public SparkReadConf( this.spark = spark; this.table = table; this.branch = branch; - this.readOptions = readOptions; this.confParser = new SparkConfParser(spark, table, readOptions); } 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 9f4a4f47bf00..bf2f59f1a4e1 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 @@ -79,6 +79,15 @@ public void after() { sql("DROP TABLE IF EXISTS %s", tableName); } + @TestTemplate + public void testOptionCaseInsensitive() { + Table table = validationCatalog.loadTable(tableIdent); + Map options = ImmutableMap.of("option", "value"); + SparkConfParser parser = new SparkConfParser(spark, table, options); + String parsedValue = parser.stringConf().option("oPtIoN").parseOptional(); + assertThat(parsedValue).isEqualTo("value"); + } + @TestTemplate public void testDurationConf() { Table table = validationCatalog.loadTable(tableIdent); From 4fd645b522b63c1b16a207b7dd0a2652bc1d1be0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 May 2024 04:41:07 +0200 Subject: [PATCH 0292/1019] Build: Bump org.testcontainers:testcontainers from 1.19.7 to 1.19.8 (#10322) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.7 to 1.19.8. - [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.19.7...1.19.8) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 3bb887845bba..cfcfb9a59ee2 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.34" sqlite-jdbc = "3.45.3.0" -testcontainers = "1.19.7" +testcontainers = "1.19.8" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From b5ba1949680dc9560621160f6d06b6d59fec3120 Mon Sep 17 00:00:00 2001 From: 911432 <30204381+911432@users.noreply.github.com> Date: Mon, 13 May 2024 15:54:32 +0900 Subject: [PATCH 0293/1019] docs: Update Quickstart to Hive 4.0.0 (#10325) --- site/docs/hive-quickstart.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/site/docs/hive-quickstart.md b/site/docs/hive-quickstart.md index 80247525f7d0..c110d56f6310 100644 --- a/site/docs/hive-quickstart.md +++ b/site/docs/hive-quickstart.md @@ -35,11 +35,11 @@ highlight some powerful features. You can learn more about Iceberg's Hive runtim The fastest way to get started is to use [Apache Hive images](https://hub.docker.com/r/apache/hive) which provides a SQL-like interface to create and query Iceberg tables from your laptop. You need to install the [Docker Desktop](https://www.docker.com/products/docker-desktop/). -Take a look at the Tags tab in [Apache Hive docker images](https://hub.docker.com/r/apache/hive/tags?page=1&ordering=-last_updated) to see the available Hive versions. +Take a look at the Tags tab in [Apache Hive docker images](https://hub.docker.com/r/apache/hive/tags?ordering=last_updated) to see the available Hive versions. Set the version variable. ```sh -export HIVE_VERSION=4.0.0-beta-1 +export HIVE_VERSION=4.0.0 ``` Start the container, using the option `--platform linux/amd64` for a Mac with an M-Series chip: @@ -108,7 +108,7 @@ SELECT * FROM nyc.taxis; #### Adding Iceberg to Hive -If you already have a Hive 4.0.0-alpha-1, or later, environment, it comes with the Iceberg 0.13.1 included. No additional downloads or jars are needed. If you have a Hive 2.3.x or Hive 3.1.x environment see [Enabling Iceberg support in Hive](docs/latest/hive.md#enabling-iceberg-support-in-hive). +If you already have a Hive 4.0.0, or later, environment, it comes with the Iceberg 1.4.3 included. No additional downloads or jars are needed. If you have a Hive 2.3.x or Hive 3.1.x environment see [Enabling Iceberg support in Hive](docs/latest/hive.md#hive-23x-hive-31x). #### Learn More From fb7ef9a9f8147be4347d37a6ef9f130a7a3017f0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 13 May 2024 14:29:06 +0200 Subject: [PATCH 0294/1019] Build: Bump software.amazon.awssdk:bom from 2.25.45 to 2.25.50 (#10323) Bumps software.amazon.awssdk:bom from 2.25.45 to 2.25.50. --- 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 cfcfb9a59ee2..0dc28eab1d90 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.45" +awssdk-bom = "2.25.50" azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 3144497fb563bfbfda3800ff665d6c3ffe892517 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 13 May 2024 12:47:51 -0700 Subject: [PATCH 0295/1019] Parquet: Add Bloom filter FPP config (#10149) --- .../org/apache/iceberg/TableProperties.java | 4 +++ docs/docs/configuration.md | 3 +- .../org/apache/iceberg/parquet/Parquet.java | 26 ++++++++++++++ .../spark/data/TestSparkParquetWriter.java | 35 +++++++++++++++++++ 4 files changed, 67 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index 2267ba03fd7b..e41230081615 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -167,6 +167,10 @@ private TableProperties() {} "write.parquet.bloom-filter-max-bytes"; public static final int PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT = 1024 * 1024; + public static final String PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX = + "write.parquet.bloom-filter-fpp.column."; + public static final double PARQUET_BLOOM_FILTER_COLUMN_FPP_DEFAULT = 0.01; + public static final String PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX = "write.parquet.bloom-filter-enabled.column."; diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index ec7af06ca7f9..117adca09f68 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -49,8 +49,9 @@ Iceberg tables support table properties to configure table behavior, like the de | write.parquet.dict-size-bytes | 2097152 (2 MB) | Parquet dictionary page size | | write.parquet.compression-codec | zstd | Parquet compression codec: zstd, brotli, lz4, gzip, snappy, uncompressed | | write.parquet.compression-level | null | Parquet compression level | -| write.parquet.bloom-filter-enabled.column.col1 | (not set) | Hint to parquet to write a bloom filter for the column: col1 | +| write.parquet.bloom-filter-enabled.column.col1 | (not set) | Hint to parquet to write a bloom filter for the column: 'col1' | | write.parquet.bloom-filter-max-bytes | 1048576 (1 MB) | The maximum number of bytes for a bloom filter bitset | +| write.parquet.bloom-filter-fpp.column.col1 | 0.01 | The false positive probability for a bloom filter applied to 'col1' (must > 0.0 and < 1.0) | | write.avro.compression-codec | gzip | Avro compression codec: gzip(deflate with 9 level), zstd, snappy, uncompressed | | write.avro.compression-level | null | Avro compression level | | write.orc.stripe-size-bytes | 67108864 (64 MB) | Define the default ORC stripe size, in bytes | diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index d591041d19c3..c97512a17d87 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -27,6 +27,7 @@ import static org.apache.iceberg.TableProperties.DELETE_PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT; import static org.apache.iceberg.TableProperties.DELETE_PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX; import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_MAX_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT; import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; @@ -284,6 +285,7 @@ public FileAppender build() throws IOException { int rowGroupCheckMinRecordCount = context.rowGroupCheckMinRecordCount(); int rowGroupCheckMaxRecordCount = context.rowGroupCheckMaxRecordCount(); int bloomFilterMaxBytes = context.bloomFilterMaxBytes(); + Map columnBloomFilterFpp = context.columnBloomFilterFpp(); Map columnBloomFilterEnabled = context.columnBloomFilterEnabled(); boolean dictionaryEnabled = context.dictionaryEnabled(); @@ -347,6 +349,12 @@ public FileAppender build() throws IOException { propsBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled)); } + for (Map.Entry entry : columnBloomFilterFpp.entrySet()) { + String colPath = entry.getKey(); + String fpp = entry.getValue(); + propsBuilder.withBloomFilterFPP(colPath, Double.parseDouble(fpp)); + } + ParquetProperties parquetProperties = propsBuilder.build(); return new org.apache.iceberg.parquet.ParquetWriter<>( @@ -384,6 +392,12 @@ public FileAppender build() throws IOException { parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled)); } + for (Map.Entry entry : columnBloomFilterFpp.entrySet()) { + String colPath = entry.getKey(); + String fpp = entry.getValue(); + parquetWriteBuilder.withBloomFilterFPP(colPath, Double.parseDouble(fpp)); + } + return new ParquetWriteAdapter<>(parquetWriteBuilder.build(), metricsConfig); } } @@ -398,6 +412,7 @@ private static class Context { private final int rowGroupCheckMinRecordCount; private final int rowGroupCheckMaxRecordCount; private final int bloomFilterMaxBytes; + private final Map columnBloomFilterFpp; private final Map columnBloomFilterEnabled; private final boolean dictionaryEnabled; @@ -411,6 +426,7 @@ private Context( int rowGroupCheckMinRecordCount, int rowGroupCheckMaxRecordCount, int bloomFilterMaxBytes, + Map columnBloomFilterFpp, Map columnBloomFilterEnabled, boolean dictionaryEnabled) { this.rowGroupSize = rowGroupSize; @@ -422,6 +438,7 @@ private Context( this.rowGroupCheckMinRecordCount = rowGroupCheckMinRecordCount; this.rowGroupCheckMaxRecordCount = rowGroupCheckMaxRecordCount; this.bloomFilterMaxBytes = bloomFilterMaxBytes; + this.columnBloomFilterFpp = columnBloomFilterFpp; this.columnBloomFilterEnabled = columnBloomFilterEnabled; this.dictionaryEnabled = dictionaryEnabled; } @@ -478,6 +495,9 @@ static Context dataContext(Map config) { config, PARQUET_BLOOM_FILTER_MAX_BYTES, PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT); Preconditions.checkArgument(bloomFilterMaxBytes > 0, "bloom Filter Max Bytes must be > 0"); + Map columnBloomFilterFpp = + PropertyUtil.propertiesWithPrefix(config, PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX); + Map columnBloomFilterEnabled = PropertyUtil.propertiesWithPrefix(config, PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX); @@ -494,6 +514,7 @@ static Context dataContext(Map config) { rowGroupCheckMinRecordCount, rowGroupCheckMaxRecordCount, bloomFilterMaxBytes, + columnBloomFilterFpp, columnBloomFilterEnabled, dictionaryEnabled); } @@ -562,6 +583,7 @@ static Context deleteContext(Map config) { rowGroupCheckMaxRecordCount, PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT, ImmutableMap.of(), + ImmutableMap.of(), dictionaryEnabled); } @@ -609,6 +631,10 @@ int bloomFilterMaxBytes() { return bloomFilterMaxBytes; } + Map columnBloomFilterFpp() { + return columnBloomFilterFpp; + } + Map columnBloomFilterEnabled() { return columnBloomFilterEnabled; } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java index fa62d6228118..73800d3cf3e0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetWriter.java @@ -18,12 +18,15 @@ */ package org.apache.iceberg.spark.data; +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX; +import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX; 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.lang.reflect.Field; import java.nio.file.Path; import java.util.Iterator; import org.apache.iceberg.Files; @@ -31,8 +34,12 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetSchemaUtil; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Types; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.column.ParquetProperties; +import org.apache.parquet.schema.MessageType; import org.apache.spark.sql.catalyst.InternalRow; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -40,6 +47,11 @@ public class TestSparkParquetWriter { @TempDir private Path temp; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "id_long", Types.LongType.get())); + private static final Schema COMPLEX_SCHEMA = new Schema( required(1, "roots", Types.LongType.get()), @@ -116,4 +128,27 @@ public void testCorrectness() throws IOException { assertThat(rows).as("Should not have extra rows").isExhausted(); } } + + @Test + public void testFpp() throws IOException, NoSuchFieldException, IllegalAccessException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(SCHEMA) + .set(PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX + "id", "true") + .set(PARQUET_BLOOM_FILTER_COLUMN_FPP_PREFIX + "id", "0.05") + .createWriterFunc( + msgType -> + SparkParquetWriters.buildWriter(SparkSchemaUtil.convert(SCHEMA), msgType)) + .build()) { + // Using reflection to access the private 'props' field in ParquetWriter + Field propsField = writer.getClass().getDeclaredField("props"); + propsField.setAccessible(true); + ParquetProperties props = (ParquetProperties) propsField.get(writer); + MessageType parquetSchema = ParquetSchemaUtil.convert(SCHEMA, "test"); + ColumnDescriptor descriptor = parquetSchema.getColumnDescription(new String[] {"id"}); + double fpp = props.getBloomFilterFPP(descriptor).getAsDouble(); + assertThat(fpp).isEqualTo(0.05); + } + } } From 34656459ae86da6f0811b65688ce42f22fea7f45 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 13 May 2024 14:42:01 -0700 Subject: [PATCH 0296/1019] Spark 3.5: Add support for enums in SparkConfParser (#10311) --- .../apache/iceberg/spark/SparkConfParser.java | 37 +++++++++++++++++++ .../apache/iceberg/spark/SparkReadConf.java | 14 +++---- 2 files changed, 43 insertions(+), 8 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java index bdb66bbe3c61..09ddc1a6d2ba 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java @@ -70,6 +70,10 @@ public DurationConfParser durationConf() { return new DurationConfParser(); } + public > EnumConfParser enumConf(Function toEnum) { + return new EnumConfParser<>(toEnum); + } + private static CaseInsensitiveStringMap asCaseInsensitiveStringMap(Map map) { if (map instanceof CaseInsensitiveStringMap) { return (CaseInsensitiveStringMap) map; @@ -205,6 +209,39 @@ private Duration toDuration(String time) { } } + class EnumConfParser> extends ConfParser, T> { + private final Function toEnum; + private T defaultValue; + + EnumConfParser(Function toEnum) { + this.toEnum = toEnum; + } + + @Override + protected EnumConfParser self() { + return this; + } + + public EnumConfParser defaultValue(T value) { + this.defaultValue = value; + return self(); + } + + public EnumConfParser defaultValue(String value) { + this.defaultValue = toEnum.apply(value); + return self(); + } + + public T parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(toEnum, defaultValue); + } + + public T parseOptional() { + return parse(toEnum, defaultValue); + } + } + abstract class ConfParser { private final List optionNames = Lists.newArrayList(); private String sessionConfName; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index f5127e9cda20..bd29fb0d6d42 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -300,14 +300,12 @@ public PlanningMode dataPlanningMode() { return LOCAL; } - String modeName = - confParser - .stringConf() - .sessionConf(SparkSQLProperties.DATA_PLANNING_MODE) - .tableProperty(TableProperties.DATA_PLANNING_MODE) - .defaultValue(TableProperties.PLANNING_MODE_DEFAULT) - .parse(); - return PlanningMode.fromName(modeName); + return confParser + .enumConf(PlanningMode::fromName) + .sessionConf(SparkSQLProperties.DATA_PLANNING_MODE) + .tableProperty(TableProperties.DATA_PLANNING_MODE) + .defaultValue(TableProperties.PLANNING_MODE_DEFAULT) + .parse(); } public PlanningMode deletePlanningMode() { From 163811bb610590709e05d99c275174a14d79fe57 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 14 May 2024 01:10:43 -0600 Subject: [PATCH 0297/1019] Spark: Backport tests for struct aggregation pushdown to 3.3/3.4, cleanup assertion (#10333) --- .../spark/sql/TestAggregatePushDown.java | 122 ++++++++++++++++++ .../spark/sql/TestAggregatePushDown.java | 122 ++++++++++++++++++ .../spark/sql/TestAggregatePushDown.java | 6 +- 3 files changed, 247 insertions(+), 3 deletions(-) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 37ae96a248ef..9ea1a563efa8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; @@ -35,6 +36,7 @@ import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkTestBase; import org.apache.spark.sql.SparkSession; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.BeforeClass; @@ -470,6 +472,126 @@ public void testAggregateWithComplexType() { Assert.assertFalse("max not pushed down for complex types", explainContainsPushDownAggregates); } + @Test + public void testAggregationPushdownStructInteger() { + sql("CREATE TABLE %s (id BIGINT, struct_with_int STRUCT) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2))", tableName); + sql("INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", 3))", tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_int.c1"; + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 3L, 2L); + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_int.c1)", + "max(struct_with_int.c1)", + "min(struct_with_int.c1)"); + } + + @Test + public void testAggregationPushdownNestedStruct() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_int STRUCT>>>) USING iceberg", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", NULL)))))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", 2)))))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", 3)))))", + tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_int.c1.c2.c3.c4"; + + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 3L, 2L); + + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_int.c1.c2.c3.c4)", + "max(struct_with_int.c1.c2.c3.c4)", + "min(struct_with_int.c1.c2.c3.c4)"); + } + + @Test + public void testAggregationPushdownStructTimestamp() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_ts STRUCT) USING iceberg", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", timestamp('2023-01-30T22:22:22Z')))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", timestamp('2023-01-30T22:23:23Z')))", + tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_ts.c1"; + + assertAggregates( + sql(query, aggField, aggField, aggField, tableName), + 2L, + new Timestamp(1675117403000L), + new Timestamp(1675117342000L)); + + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_ts.c1)", + "max(struct_with_ts.c1)", + "min(struct_with_ts.c1)"); + } + + @Test + public void testAggregationPushdownOnBucketedColumn() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_int STRUCT) USING iceberg PARTITIONED BY (bucket(8, id))", + tableName); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql("INSERT INTO TABLE %s VALUES (null, named_struct(\"c1\", 2))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 3))", tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "id"; + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 2L, 1L); + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(id)", + "max(id)", + "min(id)"); + } + + private void assertAggregates( + List actual, Object expectedCount, Object expectedMax, Object expectedMin) { + Object actualCount = actual.get(0)[0]; + Object actualMax = actual.get(0)[1]; + Object actualMin = actual.get(0)[2]; + + Assertions.assertThat(actualCount) + .as("Expected and actual count should equal") + .isEqualTo(expectedCount); + Assertions.assertThat(actualMax) + .as("Expected and actual max should equal") + .isEqualTo(expectedMax); + Assertions.assertThat(actualMin) + .as("Expected and actual min should equal") + .isEqualTo(expectedMin); + } + + private void assertExplainContains(List explain, String... expectedFragments) { + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + Arrays.stream(expectedFragments) + .forEach( + fragment -> + Assertions.assertThat(explainString) + .as("Expected to find plan fragment in explain plan") + .contains(fragment)); + } + @Test public void testAggregatePushDownInDeleteCopyOnWrite() { sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 37ae96a248ef..9ea1a563efa8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -21,6 +21,7 @@ import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; +import java.util.Arrays; import java.util.List; import java.util.Locale; import java.util.Map; @@ -35,6 +36,7 @@ import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkTestBase; import org.apache.spark.sql.SparkSession; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.BeforeClass; @@ -470,6 +472,126 @@ public void testAggregateWithComplexType() { Assert.assertFalse("max not pushed down for complex types", explainContainsPushDownAggregates); } + @Test + public void testAggregationPushdownStructInteger() { + sql("CREATE TABLE %s (id BIGINT, struct_with_int STRUCT) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2))", tableName); + sql("INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", 3))", tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_int.c1"; + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 3L, 2L); + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_int.c1)", + "max(struct_with_int.c1)", + "min(struct_with_int.c1)"); + } + + @Test + public void testAggregationPushdownNestedStruct() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_int STRUCT>>>) USING iceberg", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", NULL)))))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", 2)))))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", named_struct(\"c2\", named_struct(\"c3\", named_struct(\"c4\", 3)))))", + tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_int.c1.c2.c3.c4"; + + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 3L, 2L); + + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_int.c1.c2.c3.c4)", + "max(struct_with_int.c1.c2.c3.c4)", + "min(struct_with_int.c1.c2.c3.c4)"); + } + + @Test + public void testAggregationPushdownStructTimestamp() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_ts STRUCT) USING iceberg", + tableName); + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql( + "INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", timestamp('2023-01-30T22:22:22Z')))", + tableName); + sql( + "INSERT INTO TABLE %s VALUES (3, named_struct(\"c1\", timestamp('2023-01-30T22:23:23Z')))", + tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "struct_with_ts.c1"; + + assertAggregates( + sql(query, aggField, aggField, aggField, tableName), + 2L, + new Timestamp(1675117403000L), + new Timestamp(1675117342000L)); + + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(struct_with_ts.c1)", + "max(struct_with_ts.c1)", + "min(struct_with_ts.c1)"); + } + + @Test + public void testAggregationPushdownOnBucketedColumn() { + sql( + "CREATE TABLE %s (id BIGINT, struct_with_int STRUCT) USING iceberg PARTITIONED BY (bucket(8, id))", + tableName); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", NULL))", tableName); + sql("INSERT INTO TABLE %s VALUES (null, named_struct(\"c1\", 2))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 3))", tableName); + + String query = "SELECT COUNT(%s), MAX(%s), MIN(%s) FROM %s"; + String aggField = "id"; + assertAggregates(sql(query, aggField, aggField, aggField, tableName), 2L, 2L, 1L); + assertExplainContains( + sql("EXPLAIN " + query, aggField, aggField, aggField, tableName), + "count(id)", + "max(id)", + "min(id)"); + } + + private void assertAggregates( + List actual, Object expectedCount, Object expectedMax, Object expectedMin) { + Object actualCount = actual.get(0)[0]; + Object actualMax = actual.get(0)[1]; + Object actualMin = actual.get(0)[2]; + + Assertions.assertThat(actualCount) + .as("Expected and actual count should equal") + .isEqualTo(expectedCount); + Assertions.assertThat(actualMax) + .as("Expected and actual max should equal") + .isEqualTo(expectedMax); + Assertions.assertThat(actualMin) + .as("Expected and actual min should equal") + .isEqualTo(expectedMin); + } + + private void assertExplainContains(List explain, String... expectedFragments) { + String explainString = explain.get(0)[0].toString().toLowerCase(Locale.ROOT); + Arrays.stream(expectedFragments) + .forEach( + fragment -> + Assertions.assertThat(explainString) + .as("Expected to find plan fragment in explain plan") + .contains(fragment)); + } + @Test public void testAggregatePushDownInDeleteCopyOnWrite() { sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); 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 603b1a1cfb5f..7fdd5163f11b 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 @@ -595,9 +595,9 @@ private void assertExplainContains(List explain, String... expectedFra Arrays.stream(expectedFragments) .forEach( fragment -> - Assertions.assertThat(explainString.contains(fragment)) - .isTrue() - .as("Expected to find plan fragment in explain plan")); + Assertions.assertThat(explainString) + .as("Expected to find plan fragment in explain plan") + .contains(fragment)); } @TestTemplate From 4592dbaf43ed9c630cb807e58ed12d78df553897 Mon Sep 17 00:00:00 2001 From: Andrew Sherman Date: Tue, 14 May 2024 00:40:09 -0700 Subject: [PATCH 0298/1019] Docs: Update vendor information for Cloudera (#10278) --- site/docs/vendors.md | 25 ++++++++++++++++--------- 1 file changed, 16 insertions(+), 9 deletions(-) diff --git a/site/docs/vendors.md b/site/docs/vendors.md index 5b8e1a76a19c..dc20783da5ba 100644 --- a/site/docs/vendors.md +++ b/site/docs/vendors.md @@ -33,15 +33,22 @@ ClickHouse is a column-oriented database that enables its users to generate powe ### [Cloudera](http://cloudera.com) -Cloudera Data Platform integrates Apache Iceberg to the following components: -* Apache Hive, Apache Impala, and Apache Spark to query Apache Iceberg tables -* Cloudera Data Warehouse service providing access to Apache Iceberg tables through Apache Hive and Apache Impala -* Cloudera Data Engineering service providing access to Apache Iceberg tables through Apache Spark -* The CDP Shared Data Experience (SDX) provides compliance and self-service data access for Apache Iceberg tables -* Hive metastore, which plays a lightweight role in providing the Iceberg Catalog -* Data Visualization to visualize data stored in Apache Iceberg - -https://docs.cloudera.com/cdp-public-cloud/cloud/cdp-iceberg/topics/iceberg-in-cdp.html +[Cloudera's data lakehouse](https://www.cloudera.com/products/open-data-lakehouse.html) +enables customers to store and manage their data in open table +formats like Apache Iceberg for running large scale multi-function analytics and AI. +Organizations rely on Cloudera's Iceberg support because it is easy to use, easy +to integrate into any data ecosystem and easy to run multiple engines - both Cloudera +and non-Cloudera, regardless of where the data resides. +It provides a common standard for all data with unified security, governance, metadata +management, and fine-grained access control across the data. + +[Cloudera](https://www.cloudera.com/) provides an integrated end to end open data lakehouse with the ability +to ingest batch and streaming data using NiFi, Flink and Kafka, then process +the same copy of data using Spark and run analytics or AI with our +[Data Visualization](https://www.cloudera.com/products/cloudera-data-platform/data-visualization.html), +[Data warehouse](https://www.cloudera.com/products/data-warehouse.html) and +[Machine Learning](https://www.cloudera.com/products/machine-learning.html) tools on private +or any public cloud. ### [Dremio](https://www.dremio.com/) From 00fbb8b47fd77359757fbc5d915fd806d993131c Mon Sep 17 00:00:00 2001 From: Yufei Gu Date: Tue, 14 May 2024 10:29:24 -0700 Subject: [PATCH 0299/1019] Make proxy endpoint configurable for s3 Http clients (#10332) --- .../iceberg/aws/ApacheHttpClientConfigurations.java | 10 ++++++++++ .../apache/iceberg/aws/HttpClientProperties.java | 7 +++++++ .../aws/UrlConnectionHttpClientConfigurations.java | 10 ++++++++++ .../iceberg/aws/TestHttpClientConfigurations.java | 13 +++++++++++++ docs/docs/aws.md | 9 +++++---- 5 files changed, 45 insertions(+), 4 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java index 04f5b911ff58..95fe34b742ff 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/ApacheHttpClientConfigurations.java @@ -18,12 +18,14 @@ */ package org.apache.iceberg.aws; +import java.net.URI; import java.time.Duration; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.util.PropertyUtil; import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder; import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; class ApacheHttpClientConfigurations { private Long connectionTimeoutMs; @@ -35,6 +37,7 @@ class ApacheHttpClientConfigurations { private Integer maxConnections; private Boolean tcpKeepAliveEnabled; private Boolean useIdleConnectionReaperEnabled; + private String proxyEndpoint; private ApacheHttpClientConfigurations() {} @@ -72,6 +75,9 @@ private void initialize(Map httpClientProperties) { this.useIdleConnectionReaperEnabled = PropertyUtil.propertyAsNullableBoolean( httpClientProperties, HttpClientProperties.APACHE_USE_IDLE_CONNECTION_REAPER_ENABLED); + this.proxyEndpoint = + PropertyUtil.propertyAsString( + httpClientProperties, HttpClientProperties.PROXY_ENDPOINT, null); } @VisibleForTesting @@ -103,6 +109,10 @@ void configureApacheHttpClientBuilder(ApacheHttpClient.Builder apacheHttpClientB if (useIdleConnectionReaperEnabled != null) { apacheHttpClientBuilder.useIdleConnectionReaper(useIdleConnectionReaperEnabled); } + if (proxyEndpoint != null) { + apacheHttpClientBuilder.proxyConfiguration( + ProxyConfiguration.builder().endpoint(URI.create(proxyEndpoint)).build()); + } } public static ApacheHttpClientConfigurations create(Map properties) { diff --git a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java index 2a5ca2ece8e0..3e615cad7024 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java @@ -52,6 +52,13 @@ public class HttpClientProperties implements Serializable { public static final String CLIENT_TYPE_URLCONNECTION = "urlconnection"; public static final String CLIENT_TYPE_DEFAULT = CLIENT_TYPE_APACHE; + + /** + * Used to configure the proxy endpoint. Used by both {@link + * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder} and {@link + * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder} + */ + public static final String PROXY_ENDPOINT = "http-client.proxy-endpoint"; /** * Used to configure the connection timeout in milliseconds for {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only diff --git a/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java b/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java index e71b02caef7d..ff8dafcf0645 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/UrlConnectionHttpClientConfigurations.java @@ -18,17 +18,20 @@ */ package org.apache.iceberg.aws; +import java.net.URI; import java.time.Duration; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.util.PropertyUtil; import software.amazon.awssdk.awscore.client.builder.AwsSyncClientBuilder; +import software.amazon.awssdk.http.urlconnection.ProxyConfiguration; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; class UrlConnectionHttpClientConfigurations { private Long httpClientUrlConnectionConnectionTimeoutMs; private Long httpClientUrlConnectionSocketTimeoutMs; + private String proxyEndpoint; private UrlConnectionHttpClientConfigurations() {} @@ -46,6 +49,9 @@ private void initialize(Map httpClientProperties) { this.httpClientUrlConnectionSocketTimeoutMs = PropertyUtil.propertyAsNullableLong( httpClientProperties, HttpClientProperties.URLCONNECTION_SOCKET_TIMEOUT_MS); + this.proxyEndpoint = + PropertyUtil.propertyAsString( + httpClientProperties, HttpClientProperties.PROXY_ENDPOINT, null); } @VisibleForTesting @@ -59,6 +65,10 @@ void configureUrlConnectionHttpClientBuilder( urlConnectionHttpClientBuilder.socketTimeout( Duration.ofMillis(httpClientUrlConnectionSocketTimeoutMs)); } + if (proxyEndpoint != null) { + urlConnectionHttpClientBuilder.proxyConfiguration( + ProxyConfiguration.builder().endpoint(URI.create(proxyEndpoint)).build()); + } } public static UrlConnectionHttpClientConfigurations create( diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java index 17ac7ca72828..0f96ac0f6c82 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientConfigurations.java @@ -24,6 +24,7 @@ import org.junit.jupiter.api.Test; import org.mockito.Mockito; import software.amazon.awssdk.http.apache.ApacheHttpClient; +import software.amazon.awssdk.http.apache.ProxyConfiguration; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; public class TestHttpClientConfigurations { @@ -34,6 +35,7 @@ public void testUrlConnectionOverrideConfigurations() { properties.put(HttpClientProperties.URLCONNECTION_CONNECTION_TIMEOUT_MS, "80"); properties.put(HttpClientProperties.APACHE_SOCKET_TIMEOUT_MS, "100"); properties.put(HttpClientProperties.APACHE_CONNECTION_TIMEOUT_MS, "200"); + properties.put(HttpClientProperties.PROXY_ENDPOINT, "http://proxy:8080"); UrlConnectionHttpClientConfigurations urlConnectionHttpClientConfigurations = UrlConnectionHttpClientConfigurations.create(properties); UrlConnectionHttpClient.Builder urlConnectionHttpClientBuilder = @@ -46,6 +48,9 @@ public void testUrlConnectionOverrideConfigurations() { Mockito.verify(spyUrlConnectionHttpClientBuilder).socketTimeout(Duration.ofMillis(90)); Mockito.verify(spyUrlConnectionHttpClientBuilder).connectionTimeout(Duration.ofMillis(80)); + Mockito.verify(spyUrlConnectionHttpClientBuilder) + .proxyConfiguration( + Mockito.any(software.amazon.awssdk.http.urlconnection.ProxyConfiguration.class)); } @Test @@ -64,6 +69,9 @@ public void testUrlConnectionDefaultConfigurations() { .connectionTimeout(Mockito.any(Duration.class)); Mockito.verify(spyUrlConnectionHttpClientBuilder, Mockito.never()) .socketTimeout(Mockito.any(Duration.class)); + Mockito.verify(spyUrlConnectionHttpClientBuilder, Mockito.never()) + .proxyConfiguration( + Mockito.any(software.amazon.awssdk.http.urlconnection.ProxyConfiguration.class)); } @Test @@ -80,6 +88,7 @@ public void testApacheOverrideConfigurations() { properties.put(HttpClientProperties.APACHE_MAX_CONNECTIONS, "104"); properties.put(HttpClientProperties.APACHE_TCP_KEEP_ALIVE_ENABLED, "true"); properties.put(HttpClientProperties.APACHE_USE_IDLE_CONNECTION_REAPER_ENABLED, "false"); + properties.put(HttpClientProperties.PROXY_ENDPOINT, "http://proxy:8080"); ApacheHttpClientConfigurations apacheHttpClientConfigurations = ApacheHttpClientConfigurations.create(properties); ApacheHttpClient.Builder apacheHttpClientBuilder = ApacheHttpClient.builder(); @@ -96,6 +105,8 @@ public void testApacheOverrideConfigurations() { Mockito.verify(spyApacheHttpClientBuilder).maxConnections(104); Mockito.verify(spyApacheHttpClientBuilder).tcpKeepAlive(true); Mockito.verify(spyApacheHttpClientBuilder).useIdleConnectionReaper(false); + Mockito.verify(spyApacheHttpClientBuilder) + .proxyConfiguration(Mockito.any(ProxyConfiguration.class)); } @Test @@ -123,5 +134,7 @@ public void testApacheDefaultConfigurations() { Mockito.verify(spyApacheHttpClientBuilder, Mockito.never()).tcpKeepAlive(Mockito.anyBoolean()); Mockito.verify(spyApacheHttpClientBuilder, Mockito.never()) .useIdleConnectionReaper(Mockito.anyBoolean()); + Mockito.verify(spyApacheHttpClientBuilder, Mockito.never()) + .proxyConfiguration(Mockito.any(ProxyConfiguration.class)); } } diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 2bd6636670ee..b2f3241654de 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -578,11 +578,12 @@ In contrast, URL Connection HTTP Client optimizes for minimum dependencies and s For more details of configuration, see sections [URL Connection HTTP Client Configurations](#url-connection-http-client-configurations) and [Apache HTTP Client Configurations](#apache-http-client-configurations). -Configure the following property to set the type of HTTP client: +Configurations for the HTTP client can be set via catalog properties. Below is an overview of available configurations: -| Property | Default | Description | -|------------------|---------|------------------------------------------------------------------------------------------------------------| -| http-client.type | apache | Types of HTTP Client.
    `urlconnection`: URL Connection HTTP Client
    `apache`: Apache HTTP Client | +| Property | Default | Description | +|----------------------------|---------|------------------------------------------------------------------------------------------------------------| +| http-client.type | apache | Types of HTTP Client.
    `urlconnection`: URL Connection HTTP Client
    `apache`: Apache HTTP Client | +| http-client.proxy-endpoint | null | An optional proxy endpoint to use for the HTTP client. | #### URL Connection HTTP Client Configurations From ce9d856dc2e280f14d6da21dc87e4766d5c76dba Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 14 May 2024 11:00:17 -0700 Subject: [PATCH 0300/1019] Spark 3.4: Add support for enums in SparkConfParser (#10330) Co-authored-by: Huaxin Gao --- .../apache/iceberg/spark/SparkConfParser.java | 37 +++++++++++++++++++ .../apache/iceberg/spark/SparkReadConf.java | 14 +++---- 2 files changed, 43 insertions(+), 8 deletions(-) diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java index d666c9c35b96..5ebcabe815cc 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java @@ -69,6 +69,10 @@ public DurationConfParser durationConf() { return new DurationConfParser(); } + public > EnumConfParser enumConf(Function toEnum) { + return new EnumConfParser<>(toEnum); + } + class BooleanConfParser extends ConfParser { private Boolean defaultValue; private boolean negate = false; @@ -196,6 +200,39 @@ private Duration toDuration(String time) { } } + class EnumConfParser> extends ConfParser, T> { + private final Function toEnum; + private T defaultValue; + + EnumConfParser(Function toEnum) { + this.toEnum = toEnum; + } + + @Override + protected EnumConfParser self() { + return this; + } + + public EnumConfParser defaultValue(T value) { + this.defaultValue = value; + return self(); + } + + public EnumConfParser defaultValue(String value) { + this.defaultValue = toEnum.apply(value); + return self(); + } + + public T parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(toEnum, defaultValue); + } + + public T parseOptional() { + return parse(toEnum, defaultValue); + } + } + abstract class ConfParser { private final List optionNames = Lists.newArrayList(); private String sessionConfName; 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 f3ad91554ec6..f0c6b9115c7b 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 @@ -304,14 +304,12 @@ public PlanningMode dataPlanningMode() { return LOCAL; } - String modeName = - confParser - .stringConf() - .sessionConf(SparkSQLProperties.DATA_PLANNING_MODE) - .tableProperty(TableProperties.DATA_PLANNING_MODE) - .defaultValue(TableProperties.PLANNING_MODE_DEFAULT) - .parse(); - return PlanningMode.fromName(modeName); + return confParser + .enumConf(PlanningMode::fromName) + .sessionConf(SparkSQLProperties.DATA_PLANNING_MODE) + .tableProperty(TableProperties.DATA_PLANNING_MODE) + .defaultValue(TableProperties.PLANNING_MODE_DEFAULT) + .parse(); } public PlanningMode deletePlanningMode() { From e6711935ee5d2d7e669ffc10683ff1ac7d5d25d9 Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Wed, 15 May 2024 09:08:47 +0900 Subject: [PATCH 0301/1019] AWS: Retain Glue Catalog table description after updating Iceberg table (#10199) --- .../apache/iceberg/aws/glue/GlueTestBase.java | 29 +++++++++++++++++++ .../aws/glue/TestGlueCatalogTable.java | 13 +++++++++ .../iceberg/aws/glue/GlueTableOperations.java | 3 ++ 3 files changed, 45 insertions(+) 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 2a810f06508f..ed3a235eb0c8 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 @@ -39,6 +39,11 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.GetTableRequest; +import software.amazon.awssdk.services.glue.model.GetTableResponse; +import software.amazon.awssdk.services.glue.model.Table; +import software.amazon.awssdk.services.glue.model.TableInput; +import software.amazon.awssdk.services.glue.model.UpdateTableRequest; import software.amazon.awssdk.services.s3.S3Client; @SuppressWarnings({"VisibilityModifier", "HideUtilityClassConstructor"}) @@ -129,4 +134,28 @@ public static String createTable(String namespace, String tableName) { glueCatalog.createTable(TableIdentifier.of(namespace, tableName), schema, partitionSpec); return tableName; } + + // Directly call Glue API to update table description + public static void updateTableDescription( + String namespace, String tableName, String description) { + GetTableResponse response = + glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + Table table = response.table(); + UpdateTableRequest request = + UpdateTableRequest.builder() + .catalogId(table.catalogId()) + .databaseName(table.databaseName()) + .tableInput( + TableInput.builder() + .description(description) + .name(table.name()) + .partitionKeys(table.partitionKeys()) + .tableType(table.tableType()) + .owner(table.owner()) + .parameters(table.parameters()) + .storageDescriptor(table.storageDescriptor()) + .build()) + .build(); + glue.updateTable(request); + } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index f6bdd89707dc..6dffdb5b9253 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -182,6 +182,8 @@ public void testUpdateTable() { assertThat(current).isNull(); // create table, refresh should update createTable(namespace, tableName); + String description = "test description"; + updateTableDescription(namespace, tableName, description); current = ops.refresh(); assertThat(current.schema()).asString().isEqualTo(schema.toString()); assertThat(current.spec()).isEqualTo(partitionSpec); @@ -206,6 +208,17 @@ public void testUpdateTable() { .isEqualTo("EXTERNAL_TABLE"); assertThat(response.table().storageDescriptor().columns()).hasSameSizeAs(schema.columns()); assertThat(response.table().partitionKeys()).hasSameSizeAs(partitionSpec.fields()); + assertThat(response.table().description()).isEqualTo(description); + + String updatedComment = "test updated comment"; + table + .updateProperties() + .set(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, updatedComment) + .commit(); + // check table in Glue + response = + glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + assertThat(response.table().description()).isEqualTo(updatedComment); } @Test diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java index 6e53e707aa09..aedf78523485 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java @@ -316,6 +316,9 @@ void persistGlueTable( .skipArchive(awsProperties.glueCatalogSkipArchive()) .tableInput( TableInput.builder() + // Call description before applyMutation so that applyMutation overwrites the + // description with the comment specified in the query + .description(glueTable.description()) .applyMutation( builder -> IcebergToGlueConverter.setTableInputInformation(builder, metadata)) From 5dc5b40b38d48af1414a16df7882e7778d62859f Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 16 May 2024 00:44:20 +0530 Subject: [PATCH 0302/1019] Kafka-connect: Handle namespace creation for auto table creation (#10186) --- .../connect/data/IcebergWriterFactory.java | 24 +++++++++++++++++++ .../data/IcebergWriterFactoryTest.java | 19 ++++++++++++--- 2 files changed, 40 insertions(+), 3 deletions(-) diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java index 67d0e850e62e..47dcddcb9925 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java @@ -18,14 +18,18 @@ */ package org.apache.iceberg.connect.data; +import java.util.Arrays; import java.util.List; import java.util.concurrent.atomic.AtomicReference; import org.apache.iceberg.PartitionSpec; 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.connect.IcebergSinkConfig; import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.ForbiddenException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.types.Type; @@ -83,6 +87,8 @@ Table autoCreateTable(String tableName, SinkRecord sample) { org.apache.iceberg.Schema schema = new org.apache.iceberg.Schema(structType.fields()); TableIdentifier identifier = TableIdentifier.parse(tableName); + createNamespaceIfNotExist(catalog, identifier.namespace()); + List partitionBy = config.tableConfig(tableName).partitionBy(); PartitionSpec spec; try { @@ -112,4 +118,22 @@ Table autoCreateTable(String tableName, SinkRecord sample) { }); return result.get(); } + + @VisibleForTesting + static void createNamespaceIfNotExist(Catalog catalog, Namespace identifierNamespace) { + if (!(catalog instanceof SupportsNamespaces)) { + return; + } + + String[] levels = identifierNamespace.levels(); + for (int index = 0; index < levels.length; index++) { + Namespace namespace = Namespace.of(Arrays.copyOfRange(levels, 0, index + 1)); + try { + ((SupportsNamespaces) catalog).createNamespace(namespace); + } catch (AlreadyExistsException | ForbiddenException ex) { + // Ignoring the error as forcefully creating the namespace even if it exists + // to avoid double namespaceExists() check. + } + } + } } diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/IcebergWriterFactoryTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/IcebergWriterFactoryTest.java index 93d1d2fa6bea..ab8bbdd02c23 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/IcebergWriterFactoryTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/IcebergWriterFactoryTest.java @@ -21,13 +21,18 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; +import static org.mockito.Mockito.withSettings; +import java.util.List; import java.util.Map; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; 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.connect.IcebergSinkConfig; import org.apache.iceberg.connect.TableSinkConfig; @@ -47,7 +52,7 @@ public class IcebergWriterFactoryTest { @ValueSource(booleans = {true, false}) @SuppressWarnings("unchecked") public void testAutoCreateTable(boolean partitioned) { - Catalog catalog = mock(Catalog.class); + Catalog catalog = mock(Catalog.class, withSettings().extraInterfaces(SupportsNamespaces.class)); when(catalog.loadTable(any())).thenThrow(new NoSuchTableException("no such table")); TableSinkConfig tableConfig = mock(TableSinkConfig.class); @@ -63,7 +68,7 @@ public void testAutoCreateTable(boolean partitioned) { when(record.value()).thenReturn(ImmutableMap.of("id", 123, "data", "foo2")); IcebergWriterFactory factory = new IcebergWriterFactory(catalog, config); - factory.autoCreateTable("db.tbl", record); + factory.autoCreateTable("foo1.foo2.foo3.bar", record); ArgumentCaptor identCaptor = ArgumentCaptor.forClass(TableIdentifier.class); ArgumentCaptor schemaCaptor = ArgumentCaptor.forClass(Schema.class); @@ -77,10 +82,18 @@ public void testAutoCreateTable(boolean partitioned) { specCaptor.capture(), propsCaptor.capture()); - assertThat(identCaptor.getValue()).isEqualTo(TableIdentifier.of("db", "tbl")); + assertThat(identCaptor.getValue()) + .isEqualTo(TableIdentifier.of(Namespace.of("foo1", "foo2", "foo3"), "bar")); assertThat(schemaCaptor.getValue().findField("id").type()).isEqualTo(LongType.get()); assertThat(schemaCaptor.getValue().findField("data").type()).isEqualTo(StringType.get()); assertThat(specCaptor.getValue().isPartitioned()).isEqualTo(partitioned); assertThat(propsCaptor.getValue()).containsKey("test-prop"); + + ArgumentCaptor namespaceCaptor = ArgumentCaptor.forClass(Namespace.class); + verify((SupportsNamespaces) catalog, times(3)).createNamespace(namespaceCaptor.capture()); + List capturedArguments = namespaceCaptor.getAllValues(); + assertThat(capturedArguments.get(0)).isEqualTo(Namespace.of("foo1")); + assertThat(capturedArguments.get(1)).isEqualTo(Namespace.of("foo1", "foo2")); + assertThat(capturedArguments.get(2)).isEqualTo(Namespace.of("foo1", "foo2", "foo3")); } } From 79e0571f496a8bf58b15323e3b02d5b84fa29217 Mon Sep 17 00:00:00 2001 From: Yufei Gu Date: Wed, 15 May 2024 14:11:53 -0700 Subject: [PATCH 0303/1019] Avoid adding a closed client to the pool (#10337) --- .../org/apache/iceberg/ClientPoolImpl.java | 46 +++++++++---------- .../apache/iceberg/TestClientPoolImpl.java | 15 ++++-- 2 files changed, 34 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java index 4c4469544816..c3534fa22a4a 100644 --- a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java +++ b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java @@ -21,6 +21,7 @@ import java.io.Closeable; import java.util.ArrayDeque; import java.util.Deque; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -70,37 +71,29 @@ public R run(Action action, boolean retry) throws E, InterruptedExc try { return action.run(client); } catch (Exception exc) { - if (!retry || !isConnectionException(exc)) { - throw exc; + if (retry && isConnectionException(exc)) { + int retryAttempts = 0; + while (retryAttempts < maxRetries) { + try { + client = reconnect(client); + return action.run(client); + } catch (Exception e) { + if (isConnectionException(e)) { + retryAttempts++; + Thread.sleep(connectionRetryWaitPeriodMs); + } else { + throw reconnectExc.cast(exc); + } + } + } } - return retryAction(action, exc, client); - + throw exc; } finally { release(client); } } - private R retryAction(Action action, Exception originalFailure, C client) - throws E, InterruptedException { - int retryAttempts = 0; - while (retryAttempts < maxRetries) { - try { - C reconnectedClient = reconnect(client); - return action.run(reconnectedClient); - } catch (Exception exc) { - if (isConnectionException(exc)) { - retryAttempts++; - Thread.sleep(connectionRetryWaitPeriodMs); - } else { - throw reconnectExc.cast(originalFailure); - } - } - } - - throw reconnectExc.cast(originalFailure); - } - protected abstract C newClient(); protected abstract C reconnect(C client); @@ -169,6 +162,11 @@ private void release(C client) { } } + @VisibleForTesting + Deque clients() { + return clients; + } + public int poolSize() { return poolSize; } diff --git a/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java index 8d62afa17678..3a6666bac38b 100644 --- a/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java +++ b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java @@ -31,11 +31,16 @@ public void testRetrySucceedsWithinMaxAttempts() throws Exception { int succeedAfterAttempts = 3; try (MockClientPoolImpl mockClientPool = new MockClientPoolImpl(2, RetryableException.class, true, maxRetries)) { + // initial the client pool with a client, so that we can verify the client is replaced + MockClient firstClient = mockClientPool.newClient(); + mockClientPool.clients().add(firstClient); + int actions = mockClientPool.run(client -> client.succeedAfter(succeedAfterAttempts)); assertThat(actions) .as("There should be exactly one successful action invocation") .isEqualTo(1); assertThat(mockClientPool.reconnectionAttempts()).isEqualTo(succeedAfterAttempts - 1); + assertThat(mockClientPool.clients().peekFirst().equals(firstClient)).isFalse(); } } @@ -78,11 +83,15 @@ static class NonRetryableException extends RuntimeException {} static class MockClient { boolean closed = false; - int actions = 0; - int retryableFailures = 0; + MockClient() {} + + MockClient(int retryableFailures) { + this.retryableFailures = retryableFailures; + } + public void close() { closed = true; } @@ -126,7 +135,7 @@ protected MockClient newClient() { @Override protected MockClient reconnect(MockClient client) { reconnectionAttempts++; - return client; + return new MockClient(reconnectionAttempts); } @Override From f09b2069362917ecb466c789442650cb29518ed0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 16 May 2024 14:05:19 +0200 Subject: [PATCH 0304/1019] Build: Bump nessie from 0.81.1 to 0.82.0 (#10318) Bumps `nessie` from 0.81.1 to 0.82.0. Updates `org.projectnessie.nessie:nessie-client` from 0.81.1 to 0.82.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.81.1 to 0.82.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.81.1 to 0.82.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.81.1 to 0.82.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 0dc28eab1d90..67dc52ba82ee 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.81.1" +nessie = "0.82.0" netty-buffer = "4.1.109.Final" netty-buffer-compat = "4.1.109.Final" object-client-bundle = "3.3.2" From 16a6342899007bc7e51c6d41360694d73505b70b Mon Sep 17 00:00:00 2001 From: dongwang Date: Thu, 16 May 2024 20:05:38 +0800 Subject: [PATCH 0305/1019] Spark 3.5: Fix the setting of equalAuthorities in RemoveOrphanFilesProcedure (#10334) --- .../TestRemoveOrphanFilesProcedure.java | 83 +++++++++++++++++++ .../RemoveOrphanFilesProcedure.java | 2 +- 2 files changed, 84 insertions(+), 1 deletion(-) 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 5d48daa74f40..76eef6a73b98 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 @@ -666,4 +666,87 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() // Dropping the table here sql("DROP TABLE %s", tableName); } + + @TestTemplate + public void testRemoveOrphanFilesProcedureWithEqualAuthorities() + throws NoSuchTableException, ParseException, IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, java.nio.file.Files.createTempDirectory(temp, "junit")); + } + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Path originalPath = new Path(table.location()); + + URI uri = originalPath.toUri(); + String originalAuthority = uri.getAuthority() == null ? "" : uri.getAuthority(); + Path newParentPath = new Path(uri.getScheme(), "localhost", uri.getPath()); + + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-a.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-b.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + table.newFastAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Timestamp lastModifiedTimestamp = new Timestamp(10000); + + List allFiles = + Lists.newArrayList( + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-a.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-b.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + ReachableFileUtil.versionHintLocation(table), lastModifiedTimestamp)); + + for (String file : ReachableFileUtil.metadataFileLocations(table, true)) { + allFiles.add(new FilePathLastModifiedRecord(file, lastModifiedTimestamp)); + } + + for (ManifestFile manifest : TestHelpers.dataManifests(table)) { + allFiles.add(new FilePathLastModifiedRecord(manifest.path(), lastModifiedTimestamp)); + } + + Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + String fileListViewName = "files_view"; + compareToFileList.createOrReplaceTempView(fileListViewName); + List orphanFiles = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "equal_authorities => map('localhost', '%s')," + + "file_list_view => '%s')", + catalogName, tableIdent, originalAuthority, fileListViewName); + assertThat(orphanFiles).isEmpty(); + + // Test with no equal authorities + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("Conflicting authorities/schemes: [(localhost, null)]."); + + // Drop table in afterEach has purge and fails due to invalid authority "localhost" + // Dropping the table here + sql("DROP TABLE %s", tableName); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java index 6e66ea2629b8..6609efa95eb1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -128,7 +128,7 @@ public InternalRow[] call(InternalRow args) { DataTypes.StringType, DataTypes.StringType, (k, v) -> { - equalSchemes.put(k.toString(), v.toString()); + equalAuthorities.put(k.toString(), v.toString()); return BoxedUnit.UNIT; }); } From 4838639c04c95f817ca09c6ff78f85e784f8bf07 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Marcos=20Vin=C3=ADcius=20da=20Silva?= Date: Thu, 16 May 2024 11:18:59 -0300 Subject: [PATCH 0306/1019] Docs: Fix Apache Doris documentation link (#10263) Fixes Apache Doris documentation link. --- docs/mkdocs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index a803f4585e41..9037cf174564 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -60,7 +60,7 @@ nav: - Amazon EMR: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - - Doris: https://doris.apache.org/docs/dev/lakehouse/multi-catalog/iceberg + - Doris: https://doris.apache.org/docs/dev/lakehouse/datalake-analytics/iceberg - Integrations: - aws.md - dell.md From dafa608a7917f053ec364014bb39d74aff5e2c14 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 16 May 2024 17:18:00 +0200 Subject: [PATCH 0307/1019] Build: Bump io.delta:delta-spark_2.12 from 3.1.0 to 3.2.0 (#10320) Bumps [io.delta:delta-spark_2.12](https://github.com/delta-io/delta) from 3.1.0 to 3.2.0. - [Release notes](https://github.com/delta-io/delta/releases) - [Commits](https://github.com/delta-io/delta/compare/v3.1.0...v3.2.0) --- updated-dependencies: - dependency-name: io.delta:delta-spark_2.12 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 67dc52ba82ee..0ad9469ed8da 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -34,7 +34,7 @@ awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" delta-standalone = "3.1.0" -delta-spark = "3.1.0" +delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.27.0" findbugs-jsr305 = "3.0.2" From b3e0eca5dafefc66e203a33564c56c8540f95822 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 16 May 2024 10:21:49 -0600 Subject: [PATCH 0308/1019] Remove unused manifest predicate (#10339) --- .../main/java/org/apache/iceberg/ManifestGroup.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/ManifestGroup.java b/core/src/main/java/org/apache/iceberg/ManifestGroup.java index 97e480837c7b..38ad2661a8a3 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestGroup.java +++ b/core/src/main/java/org/apache/iceberg/ManifestGroup.java @@ -51,7 +51,6 @@ class ManifestGroup { private final FileIO io; private final Set dataManifests; private final DeleteFileIndex.Builder deleteIndexBuilder; - private Predicate manifestPredicate; private Predicate> manifestEntryPredicate; private Map specsById; private Expression dataFilter; @@ -86,7 +85,6 @@ class ManifestGroup { this.ignoreResiduals = false; this.columns = ManifestReader.ALL_COLUMNS; this.caseSensitive = true; - this.manifestPredicate = m -> true; this.manifestEntryPredicate = e -> true; this.scanMetrics = ScanMetrics.noop(); } @@ -114,11 +112,6 @@ ManifestGroup filterPartitions(Expression newPartitionFilter) { return this; } - ManifestGroup filterManifests(Predicate newManifestPredicate) { - this.manifestPredicate = manifestPredicate.and(newManifestPredicate); - return this; - } - ManifestGroup filterManifestEntries( Predicate> newManifestEntryPredicate) { this.manifestEntryPredicate = manifestEntryPredicate.and(newManifestEntryPredicate); @@ -302,9 +295,6 @@ private Iterable> entries( manifest -> manifest.hasAddedFiles() || manifest.hasDeletedFiles()); } - matchingManifests = - CloseableIterable.filter( - scanMetrics.skippedDataManifests(), matchingManifests, manifestPredicate); matchingManifests = CloseableIterable.count(scanMetrics.scannedDataManifests(), matchingManifests); From 7ac42c1070e25ec372c5c2d590f518bf98931e1a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 16 May 2024 18:48:58 +0200 Subject: [PATCH 0309/1019] Spark: Fix issue when partitioning by UUID (#8250) --- .../org/apache/iceberg/RecordWrapperTest.java | 4 +-- .../spark/source/WritersBenchmark.java | 9 ++++--- .../iceberg/spark/source/BaseReader.java | 4 ++- .../spark/source/InternalRowWrapper.java | 26 +++++++++++++++--- .../source/SparkPartitionedFanoutWriter.java | 2 +- .../spark/source/SparkPartitionedWriter.java | 2 +- .../spark/source/SparkPositionDeltaWrite.java | 5 ++-- .../iceberg/spark/source/SparkWrite.java | 2 +- .../spark/source/TestInternalRowWrapper.java | 3 ++- .../source/TestSparkAppenderFactory.java | 2 +- .../source/TestSparkFileWriterFactory.java | 2 +- .../source/TestSparkPartitioningWriters.java | 2 +- .../source/TestSparkPositionDeltaWriters.java | 2 +- .../spark/source/TestSparkReaderDeletes.java | 3 ++- .../iceberg/spark/sql/TestCreateTable.java | 27 +++++++++++++++++++ .../spark/source/WritersBenchmark.java | 9 ++++--- .../iceberg/spark/source/BaseReader.java | 4 ++- .../spark/source/InternalRowWrapper.java | 26 +++++++++++++++--- .../source/SparkPartitionedFanoutWriter.java | 2 +- .../spark/source/SparkPartitionedWriter.java | 2 +- .../spark/source/SparkPositionDeltaWrite.java | 5 ++-- .../iceberg/spark/source/SparkWrite.java | 2 +- .../spark/source/TestInternalRowWrapper.java | 3 ++- .../source/TestSparkAppenderFactory.java | 2 +- .../source/TestSparkFileWriterFactory.java | 2 +- .../source/TestSparkPartitioningWriters.java | 2 +- .../source/TestSparkPositionDeltaWriters.java | 2 +- .../spark/source/TestSparkReaderDeletes.java | 3 ++- .../iceberg/spark/sql/TestCreateTable.java | 26 ++++++++++++++++++ .../spark/source/WritersBenchmark.java | 9 ++++--- .../iceberg/spark/source/BaseReader.java | 4 ++- .../spark/source/InternalRowWrapper.java | 26 +++++++++++++++--- .../source/SparkPartitionedFanoutWriter.java | 2 +- .../spark/source/SparkPartitionedWriter.java | 2 +- .../spark/source/SparkPositionDeltaWrite.java | 5 ++-- .../iceberg/spark/source/SparkWrite.java | 2 +- .../spark/source/TestInternalRowWrapper.java | 3 ++- .../source/TestSparkAppenderFactory.java | 2 +- .../source/TestSparkFileWriterFactory.java | 2 +- .../source/TestSparkPartitioningWriters.java | 2 +- .../source/TestSparkPositionDeltaWriters.java | 2 +- .../spark/source/TestSparkReaderDeletes.java | 3 ++- .../iceberg/spark/sql/TestCreateTable.java | 26 ++++++++++++++++++ 43 files changed, 216 insertions(+), 59 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java index 22b928d23883..94e9825fa5f3 100644 --- a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java +++ b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java @@ -44,8 +44,8 @@ public abstract class RecordWrapperTest { optional(113, "bytes", Types.BinaryType.get()), required(114, "dec_9_0", Types.DecimalType.of(9, 0)), required(115, "dec_11_2", Types.DecimalType.of(11, 2)), - required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision - ); + required(116, "dec_38_10", Types.DecimalType.of(38, 10)), // maximum precision + optional(117, "uuid", Types.UUIDType.get())); private static final Types.StructType TIMESTAMP_WITHOUT_ZONE = Types.StructType.of( 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 13ff034e4bf5..dd2cf80716f6 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 @@ -193,7 +193,8 @@ public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOEx PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(dataSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); try (ClusteredDataWriter closeableWriter = writer) { for (InternalRow row : rows) { @@ -256,7 +257,8 @@ public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOExcep PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(dataSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); try (FanoutDataWriter closeableWriter = writer) { for (InternalRow row : rows) { @@ -324,7 +326,8 @@ public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType deleteSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(deleteSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(deleteSparkType, table().schema().asStruct()); try (ClusteredEqualityDeleteWriter closeableWriter = writer) { for (InternalRow row : rows) { 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 4fb838202c88..0882edcb7c4a 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 @@ -259,7 +259,9 @@ protected class SparkDeleteFilter extends DeleteFilter { SparkDeleteFilter(String filePath, List deletes, DeleteCounter counter) { super(filePath, deletes, tableSchema, expectedSchema, counter); - this.asStructLike = new InternalRowWrapper(SparkSchemaUtil.convert(requiredSchema())); + this.asStructLike = + new InternalRowWrapper( + SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); } @Override diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java index 524266f6f83a..d1682b8c85c1 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java @@ -19,9 +19,13 @@ package org.apache.iceberg.spark.source; import java.nio.ByteBuffer; +import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Stream; import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.DataType; @@ -40,9 +44,17 @@ class InternalRowWrapper implements StructLike { private InternalRow row = null; @SuppressWarnings("unchecked") - InternalRowWrapper(StructType rowType) { + InternalRowWrapper(StructType rowType, Types.StructType icebergSchema) { this.types = Stream.of(rowType.fields()).map(StructField::dataType).toArray(DataType[]::new); - this.getters = Stream.of(types).map(InternalRowWrapper::getter).toArray(BiFunction[]::new); + Preconditions.checkArgument( + types.length == icebergSchema.fields().size(), + "Invalid length: Spark struct type (%s) != Iceberg struct type (%s)", + types.length, + icebergSchema.fields().size()); + this.getters = new BiFunction[types.length]; + for (int i = 0; i < types.length; i++) { + getters[i] = getter(icebergSchema.fields().get(i).type(), types[i]); + } } InternalRowWrapper wrap(InternalRow internalRow) { @@ -71,8 +83,13 @@ public void set(int pos, T value) { row.update(pos, value); } - private static BiFunction getter(DataType type) { + private static BiFunction getter(Type icebergType, DataType type) { if (type instanceof StringType) { + // Spark represents UUIDs as strings + if (Type.TypeID.UUID == icebergType.typeId()) { + return (row, pos) -> UUID.fromString(row.getUTF8String(pos).toString()); + } + return (row, pos) -> row.getUTF8String(pos).toString(); } else if (type instanceof DecimalType) { DecimalType decimal = (DecimalType) type; @@ -82,7 +99,8 @@ public void set(int pos, T value) { return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); } else if (type instanceof StructType) { StructType structType = (StructType) type; - InternalRowWrapper nestedWrapper = new InternalRowWrapper(structType); + InternalRowWrapper nestedWrapper = + new InternalRowWrapper(structType, icebergType.asStructType()); return (row, pos) -> nestedWrapper.wrap(row.getStruct(pos, structType.size())); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java index f17cd260f928..c34ad2f3ad4a 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java @@ -44,7 +44,7 @@ public SparkPartitionedFanoutWriter( StructType sparkSchema) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); - this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema, schema.asStruct()); } @Override diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java index a86091644360..6904446829e4 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java @@ -44,7 +44,7 @@ public SparkPartitionedWriter( StructType sparkSchema) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); - this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema, schema.asStruct()); } @Override diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index ce4b248e0f54..2d3829243990 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -375,7 +375,7 @@ private abstract static class BaseDeltaWriter implements DeltaWriter buildPartitionProjections( @@ -645,7 +645,8 @@ private static class PartitionedDeltaWriter extends DeleteAndDataDeltaWriter { this.dataSpec = table.spec(); this.dataPartitionKey = new PartitionKey(dataSpec, context.dataSchema()); - this.internalRowDataWrapper = new InternalRowWrapper(context.dataSparkType()); + this.internalRowDataWrapper = + new InternalRowWrapper(context.dataSparkType(), context.dataSchema().asStruct()); } @Override diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index 59fecd45cc2c..c26b7f5f389b 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -741,7 +741,7 @@ private PartitionedDataWriter( this.io = io; this.spec = spec; this.partitionKey = new PartitionKey(spec, dataSchema); - this.internalRowWrapper = new InternalRowWrapper(dataSparkType); + this.internalRowWrapper = new InternalRowWrapper(dataSparkType, dataSchema.asStruct()); } @Override diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 9e75145faff9..1b4698fe5b7a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -53,7 +53,8 @@ protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { Iterable rowList = RandomData.generateSpark(schema, numRecords, 101L); InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); - InternalRowWrapper rowWrapper = new InternalRowWrapper(SparkSchemaUtil.convert(schema)); + InternalRowWrapper rowWrapper = + new InternalRowWrapper(SparkSchemaUtil.convert(schema), schema.asStruct()); Iterator actual = recordList.iterator(); Iterator expected = rowList.iterator(); 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 index 1f4c613f749e..0664400c7911 100644 --- 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 @@ -56,7 +56,7 @@ protected InternalRow createRow(Integer id, String data) { protected StructLikeSet expectedRowSet(Iterable rows) { StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); for (InternalRow row : rows) { - InternalRowWrapper wrapper = new InternalRowWrapper(sparkType); + 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/TestSparkFileWriterFactory.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 8f4b3042b1c5..575e6658db22 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkPartitioningWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 5e8107806a25..979abd21e7f7 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkPositionDeltaWriters.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index baac1efe0e40..9dc56abf9fb6 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkReaderDeletes.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index 88fd3d96dd58..76a4143fcb2b 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 @@ -315,7 +315,8 @@ public void testReadEqualityDeleteRows() throws IOException { new EqualityDeleteRowReader(task, table, null, table.schema(), false)) { while (reader.next()) { actualRowSet.add( - new InternalRowWrapper(SparkSchemaUtil.convert(table.schema())) + new InternalRowWrapper( + SparkSchemaUtil.convert(table.schema()), table.schema().asStruct()) .wrap(reader.get().copy())); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index a6256afcdf65..3a62361590d1 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -20,6 +20,7 @@ import java.io.File; import java.util.Map; +import java.util.UUID; import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.BaseTable; import org.apache.iceberg.PartitionSpec; @@ -33,6 +34,7 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.spark.sql.connector.catalog.TableCatalog; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -104,6 +106,31 @@ public void testCreateTable() { table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); } + @Test + public void testCreateTablePartitionedByUUID() { + Assertions.assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); + validationCatalog.createTable(tableIdent, schema, spec); + + Table table = validationCatalog.loadTable(tableIdent); + Assertions.assertThat(table).isNotNull(); + + StructType expectedSchema = + StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + Assertions.assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + Assertions.assertThat(table.spec().fields()).hasSize(1); + + String uuid = UUID.randomUUID().toString(); + + sql("INSERT INTO %s VALUES('%s')", tableName, uuid); + + Assertions.assertThat(sql("SELECT uuid FROM %s", tableName)) + .hasSize(1) + .element(0) + .isEqualTo(row(uuid)); + } + @Test public void testCreateTableInRootNamespace() { Assume.assumeTrue( 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 71813c5a63a6..963159fe4ee9 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 @@ -220,7 +220,8 @@ public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOEx PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(dataSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); try (ClusteredDataWriter closeableWriter = writer) { for (InternalRow row : rows) { @@ -283,7 +284,8 @@ public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOExcep PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(dataSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); try (FanoutDataWriter closeableWriter = writer) { for (InternalRow row : rows) { @@ -351,7 +353,8 @@ public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType deleteSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(deleteSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(deleteSparkType, table().schema().asStruct()); try (ClusteredEqualityDeleteWriter closeableWriter = writer) { for (InternalRow row : rows) { 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 c2b3e7c2dc56..238069e1c9a5 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 @@ -264,7 +264,9 @@ protected class SparkDeleteFilter extends DeleteFilter { SparkDeleteFilter(String filePath, List deletes, DeleteCounter counter) { super(filePath, deletes, tableSchema, expectedSchema, counter); - this.asStructLike = new InternalRowWrapper(SparkSchemaUtil.convert(requiredSchema())); + this.asStructLike = + new InternalRowWrapper( + SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java index 524266f6f83a..d1682b8c85c1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java @@ -19,9 +19,13 @@ package org.apache.iceberg.spark.source; import java.nio.ByteBuffer; +import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Stream; import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.DataType; @@ -40,9 +44,17 @@ class InternalRowWrapper implements StructLike { private InternalRow row = null; @SuppressWarnings("unchecked") - InternalRowWrapper(StructType rowType) { + InternalRowWrapper(StructType rowType, Types.StructType icebergSchema) { this.types = Stream.of(rowType.fields()).map(StructField::dataType).toArray(DataType[]::new); - this.getters = Stream.of(types).map(InternalRowWrapper::getter).toArray(BiFunction[]::new); + Preconditions.checkArgument( + types.length == icebergSchema.fields().size(), + "Invalid length: Spark struct type (%s) != Iceberg struct type (%s)", + types.length, + icebergSchema.fields().size()); + this.getters = new BiFunction[types.length]; + for (int i = 0; i < types.length; i++) { + getters[i] = getter(icebergSchema.fields().get(i).type(), types[i]); + } } InternalRowWrapper wrap(InternalRow internalRow) { @@ -71,8 +83,13 @@ public void set(int pos, T value) { row.update(pos, value); } - private static BiFunction getter(DataType type) { + private static BiFunction getter(Type icebergType, DataType type) { if (type instanceof StringType) { + // Spark represents UUIDs as strings + if (Type.TypeID.UUID == icebergType.typeId()) { + return (row, pos) -> UUID.fromString(row.getUTF8String(pos).toString()); + } + return (row, pos) -> row.getUTF8String(pos).toString(); } else if (type instanceof DecimalType) { DecimalType decimal = (DecimalType) type; @@ -82,7 +99,8 @@ public void set(int pos, T value) { return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); } else if (type instanceof StructType) { StructType structType = (StructType) type; - InternalRowWrapper nestedWrapper = new InternalRowWrapper(structType); + InternalRowWrapper nestedWrapper = + new InternalRowWrapper(structType, icebergType.asStructType()); return (row, pos) -> nestedWrapper.wrap(row.getStruct(pos, structType.size())); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java index f17cd260f928..c34ad2f3ad4a 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java @@ -44,7 +44,7 @@ public SparkPartitionedFanoutWriter( StructType sparkSchema) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); - this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema, schema.asStruct()); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java index a86091644360..6904446829e4 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java @@ -44,7 +44,7 @@ public SparkPartitionedWriter( StructType sparkSchema) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); - this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema, schema.asStruct()); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 5c6243bbb06f..c5fc8e0b0fe1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -391,7 +391,7 @@ private abstract static class BaseDeltaWriter implements DeltaWriter buildPartitionProjections( @@ -653,7 +653,8 @@ private static class PartitionedDeltaWriter extends DeleteAndDataDeltaWriter { this.dataSpec = table.spec(); this.dataPartitionKey = new PartitionKey(dataSpec, context.dataSchema()); - this.internalRowDataWrapper = new InternalRowWrapper(context.dataSparkType()); + this.internalRowDataWrapper = + new InternalRowWrapper(context.dataSparkType(), context.dataSchema().asStruct()); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index fb49d0bff5f4..d879a1f961e5 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -756,7 +756,7 @@ private PartitionedDataWriter( this.io = io; this.spec = spec; this.partitionKey = new PartitionKey(spec, dataSchema); - this.internalRowWrapper = new InternalRowWrapper(dataSparkType); + this.internalRowWrapper = new InternalRowWrapper(dataSparkType, dataSchema.asStruct()); } @Override diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index 9e75145faff9..1b4698fe5b7a 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -53,7 +53,8 @@ protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { Iterable rowList = RandomData.generateSpark(schema, numRecords, 101L); InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); - InternalRowWrapper rowWrapper = new InternalRowWrapper(SparkSchemaUtil.convert(schema)); + InternalRowWrapper rowWrapper = + new InternalRowWrapper(SparkSchemaUtil.convert(schema), schema.asStruct()); Iterator actual = recordList.iterator(); Iterator expected = rowList.iterator(); 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 index 1f4c613f749e..0664400c7911 100644 --- 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 @@ -56,7 +56,7 @@ protected InternalRow createRow(Integer id, String data) { protected StructLikeSet expectedRowSet(Iterable rows) { StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); for (InternalRow row : rows) { - InternalRowWrapper wrapper = new InternalRowWrapper(sparkType); + 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/TestSparkFileWriterFactory.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 8f4b3042b1c5..575e6658db22 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkPartitioningWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 5e8107806a25..979abd21e7f7 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkPositionDeltaWriters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index baac1efe0e40..9dc56abf9fb6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkReaderDeletes.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index b3edb0e70858..4643836542aa 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 @@ -324,7 +324,8 @@ public void testReadEqualityDeleteRows() throws IOException { new EqualityDeleteRowReader(task, table, null, table.schema(), false)) { while (reader.next()) { actualRowSet.add( - new InternalRowWrapper(SparkSchemaUtil.convert(table.schema())) + new InternalRowWrapper( + SparkSchemaUtil.convert(table.schema()), table.schema().asStruct()) .wrap(reader.get().copy())); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index ecfd6759b900..b4dde2f94500 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -20,6 +20,7 @@ import java.io.File; import java.util.Map; +import java.util.UUID; import org.apache.iceberg.BaseTable; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -104,6 +105,31 @@ public void testCreateTable() { table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); } + @Test + public void testCreateTablePartitionedByUUID() { + Assertions.assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); + validationCatalog.createTable(tableIdent, schema, spec); + + Table table = validationCatalog.loadTable(tableIdent); + Assertions.assertThat(table).isNotNull(); + + StructType expectedSchema = + StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + Assertions.assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + Assertions.assertThat(table.spec().fields()).hasSize(1); + + String uuid = UUID.randomUUID().toString(); + + sql("INSERT INTO %s VALUES('%s')", tableName, uuid); + + Assertions.assertThat(sql("SELECT uuid FROM %s", tableName)) + .hasSize(1) + .element(0) + .isEqualTo(row(uuid)); + } + @Test public void testCreateTableInRootNamespace() { Assume.assumeTrue( 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 71813c5a63a6..963159fe4ee9 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 @@ -220,7 +220,8 @@ public void writePartitionedClusteredDataWriter(Blackhole blackhole) throws IOEx PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(dataSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); try (ClusteredDataWriter closeableWriter = writer) { for (InternalRow row : rows) { @@ -283,7 +284,8 @@ public void writePartitionedFanoutDataWriter(Blackhole blackhole) throws IOExcep PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType dataSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(dataSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(dataSparkType, table().schema().asStruct()); try (FanoutDataWriter closeableWriter = writer) { for (InternalRow row : rows) { @@ -351,7 +353,8 @@ public void writePartitionedClusteredEqualityDeleteWriter(Blackhole blackhole) PartitionKey partitionKey = new PartitionKey(partitionedSpec, table().schema()); StructType deleteSparkType = SparkSchemaUtil.convert(table().schema()); - InternalRowWrapper internalRowWrapper = new InternalRowWrapper(deleteSparkType); + InternalRowWrapper internalRowWrapper = + new InternalRowWrapper(deleteSparkType, table().schema().asStruct()); try (ClusteredEqualityDeleteWriter closeableWriter = writer) { for (InternalRow row : rows) { 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 a982309ec16d..3c9438480d0d 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 @@ -251,7 +251,9 @@ protected class SparkDeleteFilter extends DeleteFilter { SparkDeleteFilter(String filePath, List deletes, DeleteCounter counter) { super(filePath, deletes, tableSchema, expectedSchema, counter); - this.asStructLike = new InternalRowWrapper(SparkSchemaUtil.convert(requiredSchema())); + this.asStructLike = + new InternalRowWrapper( + SparkSchemaUtil.convert(requiredSchema()), requiredSchema().asStruct()); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java index 524266f6f83a..d1682b8c85c1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/InternalRowWrapper.java @@ -19,9 +19,13 @@ package org.apache.iceberg.spark.source; import java.nio.ByteBuffer; +import java.util.UUID; import java.util.function.BiFunction; import java.util.stream.Stream; import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.types.BinaryType; import org.apache.spark.sql.types.DataType; @@ -40,9 +44,17 @@ class InternalRowWrapper implements StructLike { private InternalRow row = null; @SuppressWarnings("unchecked") - InternalRowWrapper(StructType rowType) { + InternalRowWrapper(StructType rowType, Types.StructType icebergSchema) { this.types = Stream.of(rowType.fields()).map(StructField::dataType).toArray(DataType[]::new); - this.getters = Stream.of(types).map(InternalRowWrapper::getter).toArray(BiFunction[]::new); + Preconditions.checkArgument( + types.length == icebergSchema.fields().size(), + "Invalid length: Spark struct type (%s) != Iceberg struct type (%s)", + types.length, + icebergSchema.fields().size()); + this.getters = new BiFunction[types.length]; + for (int i = 0; i < types.length; i++) { + getters[i] = getter(icebergSchema.fields().get(i).type(), types[i]); + } } InternalRowWrapper wrap(InternalRow internalRow) { @@ -71,8 +83,13 @@ public void set(int pos, T value) { row.update(pos, value); } - private static BiFunction getter(DataType type) { + private static BiFunction getter(Type icebergType, DataType type) { if (type instanceof StringType) { + // Spark represents UUIDs as strings + if (Type.TypeID.UUID == icebergType.typeId()) { + return (row, pos) -> UUID.fromString(row.getUTF8String(pos).toString()); + } + return (row, pos) -> row.getUTF8String(pos).toString(); } else if (type instanceof DecimalType) { DecimalType decimal = (DecimalType) type; @@ -82,7 +99,8 @@ public void set(int pos, T value) { return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); } else if (type instanceof StructType) { StructType structType = (StructType) type; - InternalRowWrapper nestedWrapper = new InternalRowWrapper(structType); + InternalRowWrapper nestedWrapper = + new InternalRowWrapper(structType, icebergType.asStructType()); return (row, pos) -> nestedWrapper.wrap(row.getStruct(pos, structType.size())); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java index f17cd260f928..c34ad2f3ad4a 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedFanoutWriter.java @@ -44,7 +44,7 @@ public SparkPartitionedFanoutWriter( StructType sparkSchema) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); - this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema, schema.asStruct()); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java index a86091644360..6904446829e4 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitionedWriter.java @@ -44,7 +44,7 @@ public SparkPartitionedWriter( StructType sparkSchema) { super(spec, format, appenderFactory, fileFactory, io, targetFileSize); this.partitionKey = new PartitionKey(spec, schema); - this.internalRowWrapper = new InternalRowWrapper(sparkSchema); + this.internalRowWrapper = new InternalRowWrapper(sparkSchema, schema.asStruct()); } @Override 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 022283631fd8..a964f7686394 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 @@ -402,7 +402,7 @@ private abstract static class BaseDeltaWriter implements DeltaWriter buildPartitionProjections( @@ -663,7 +663,8 @@ private static class PartitionedDeltaWriter extends DeleteAndDataDeltaWriter { this.dataSpec = table.spec(); this.dataPartitionKey = new PartitionKey(dataSpec, context.dataSchema()); - this.internalRowDataWrapper = new InternalRowWrapper(context.dataSparkType()); + this.internalRowDataWrapper = + new InternalRowWrapper(context.dataSparkType(), context.dataSchema().asStruct()); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index 32f560a7bd2f..d23c473bb46b 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -772,7 +772,7 @@ private PartitionedDataWriter( this.io = io; this.spec = spec; this.partitionKey = new PartitionKey(spec, dataSchema); - this.internalRowWrapper = new InternalRowWrapper(dataSparkType); + this.internalRowWrapper = new InternalRowWrapper(dataSparkType, dataSchema.asStruct()); } @Override diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java index a0bc98200281..0c869aa8e7e0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestInternalRowWrapper.java @@ -54,7 +54,8 @@ protected void generateAndValidate(Schema schema, AssertMethod assertMethod) { Iterable rowList = RandomData.generateSpark(schema, numRecords, 101L); InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); - InternalRowWrapper rowWrapper = new InternalRowWrapper(SparkSchemaUtil.convert(schema)); + InternalRowWrapper rowWrapper = + new InternalRowWrapper(SparkSchemaUtil.convert(schema), schema.asStruct()); Iterator actual = recordList.iterator(); Iterator expected = rowList.iterator(); 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 index 1f4c613f749e..0664400c7911 100644 --- 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 @@ -56,7 +56,7 @@ protected InternalRow createRow(Integer id, String data) { protected StructLikeSet expectedRowSet(Iterable rows) { StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); for (InternalRow row : rows) { - InternalRowWrapper wrapper = new InternalRowWrapper(sparkType); + 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/TestSparkFileWriterFactory.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java index 8f4b3042b1c5..575e6658db22 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkFileWriterFactory.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkPartitioningWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java index 5e8107806a25..979abd21e7f7 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPartitioningWriters.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkPositionDeltaWriters.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java index baac1efe0e40..9dc56abf9fb6 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkPositionDeltaWriters.java @@ -61,7 +61,7 @@ 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); + 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/TestSparkReaderDeletes.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReaderDeletes.java index b3edb0e70858..4643836542aa 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 @@ -324,7 +324,8 @@ public void testReadEqualityDeleteRows() throws IOException { new EqualityDeleteRowReader(task, table, null, table.schema(), false)) { while (reader.next()) { actualRowSet.add( - new InternalRowWrapper(SparkSchemaUtil.convert(table.schema())) + new InternalRowWrapper( + SparkSchemaUtil.convert(table.schema()), table.schema().asStruct()) .wrap(reader.get().copy())); } } 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 718233c697a1..73be40e050dd 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 @@ -23,6 +23,7 @@ import java.io.File; import java.nio.file.Files; +import java.util.UUID; import org.apache.iceberg.BaseTable; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -111,6 +112,31 @@ public void testCreateTable() { .isNull(); } + @TestTemplate + public void testCreateTablePartitionedByUUID() { + Assertions.assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); + validationCatalog.createTable(tableIdent, schema, spec); + + Table table = validationCatalog.loadTable(tableIdent); + Assertions.assertThat(table).isNotNull(); + + StructType expectedSchema = + StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); + Assertions.assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + Assertions.assertThat(table.spec().fields()).hasSize(1); + + String uuid = UUID.randomUUID().toString(); + + sql("INSERT INTO %s VALUES('%s')", tableName, uuid); + + Assertions.assertThat(sql("SELECT uuid FROM %s", tableName)) + .hasSize(1) + .element(0) + .isEqualTo(row(uuid)); + } + @TestTemplate public void testCreateTableInRootNamespace() { assumeThat(catalogName) From 73375cbf691c5e6d84c94c73f0c25c05de10daa6 Mon Sep 17 00:00:00 2001 From: dongwang Date: Fri, 17 May 2024 15:30:28 +0800 Subject: [PATCH 0310/1019] Spark 3.4, 3.3: Fix the setting of equalAuthorities in RemoveOrphanFilesProcedure (#10342) --- .../TestRemoveOrphanFilesProcedure.java | 83 +++++++++++++++++++ .../RemoveOrphanFilesProcedure.java | 2 +- .../TestRemoveOrphanFilesProcedure.java | 83 +++++++++++++++++++ .../RemoveOrphanFilesProcedure.java | 2 +- 4 files changed, 168 insertions(+), 2 deletions(-) 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 05eb7a6f80d1..670c4e565760 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 @@ -694,4 +694,87 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() // Dropping the table here sql("DROP TABLE %s", tableName); } + + @Test + public void testRemoveOrphanFilesProcedureWithEqualAuthorities() + throws NoSuchTableException, ParseException, IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, temp.newFolder().toURI().toString()); + } + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Path originalPath = new Path(table.location()); + + URI uri = originalPath.toUri(); + String originalAuthority = uri.getAuthority() == null ? "" : uri.getAuthority(); + Path newParentPath = new Path(uri.getScheme(), "localhost", uri.getPath()); + + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-a.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-b.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + table.newFastAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Timestamp lastModifiedTimestamp = new Timestamp(10000); + + List allFiles = + Lists.newArrayList( + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-a.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-b.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + ReachableFileUtil.versionHintLocation(table), lastModifiedTimestamp)); + + for (String file : ReachableFileUtil.metadataFileLocations(table, true)) { + allFiles.add(new FilePathLastModifiedRecord(file, lastModifiedTimestamp)); + } + + for (ManifestFile manifest : TestHelpers.dataManifests(table)) { + allFiles.add(new FilePathLastModifiedRecord(manifest.path(), lastModifiedTimestamp)); + } + + Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + String fileListViewName = "files_view"; + compareToFileList.createOrReplaceTempView(fileListViewName); + List orphanFiles = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "equal_authorities => map('localhost', '%s')," + + "file_list_view => '%s')", + catalogName, tableIdent, originalAuthority, fileListViewName); + Assert.assertEquals(0, orphanFiles.size()); + + // Test with no equal authorities + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("Conflicting authorities/schemes: [(localhost, null)]."); + + // Drop table in afterEach has purge and fails due to invalid authority "localhost" + // Dropping the table here + sql("DROP TABLE %s", tableName); + } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java index 6e66ea2629b8..6609efa95eb1 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -128,7 +128,7 @@ public InternalRow[] call(InternalRow args) { DataTypes.StringType, DataTypes.StringType, (k, v) -> { - equalSchemes.put(k.toString(), v.toString()); + equalAuthorities.put(k.toString(), v.toString()); return BoxedUnit.UNIT; }); } 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 80b515d34417..56854c561f5b 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 @@ -680,4 +680,87 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() // Dropping the table here sql("DROP TABLE %s", tableName); } + + @Test + public void testRemoveOrphanFilesProcedureWithEqualAuthorities() + throws NoSuchTableException, ParseException, IOException { + if (catalogName.equals("testhadoop")) { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + } else { + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg LOCATION '%s'", + tableName, temp.newFolder().toURI().toString()); + } + Table table = Spark3Util.loadIcebergTable(spark, tableName); + Path originalPath = new Path(table.location()); + + URI uri = originalPath.toUri(); + String originalAuthority = uri.getAuthority() == null ? "" : uri.getAuthority(); + Path newParentPath = new Path(uri.getScheme(), "localhost", uri.getPath()); + + DataFile dataFile1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-a.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(new Path(newParentPath, "path/to/data-b.parquet").toString()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + table.newFastAppend().appendFile(dataFile1).appendFile(dataFile2).commit(); + + Timestamp lastModifiedTimestamp = new Timestamp(10000); + + List allFiles = + Lists.newArrayList( + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-a.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + new Path(originalPath, "path/to/data-b.parquet").toString(), lastModifiedTimestamp), + new FilePathLastModifiedRecord( + ReachableFileUtil.versionHintLocation(table), lastModifiedTimestamp)); + + for (String file : ReachableFileUtil.metadataFileLocations(table, true)) { + allFiles.add(new FilePathLastModifiedRecord(file, lastModifiedTimestamp)); + } + + for (ManifestFile manifest : TestHelpers.dataManifests(table)) { + allFiles.add(new FilePathLastModifiedRecord(manifest.path(), lastModifiedTimestamp)); + } + + Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); + String fileListViewName = "files_view"; + compareToFileList.createOrReplaceTempView(fileListViewName); + List orphanFiles = + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "equal_authorities => map('localhost', '%s')," + + "file_list_view => '%s')", + catalogName, tableIdent, originalAuthority, fileListViewName); + Assert.assertEquals(0, orphanFiles.size()); + + // Test with no equal authorities + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .isInstanceOf(ValidationException.class) + .hasMessageEndingWith("Conflicting authorities/schemes: [(localhost, null)]."); + + // Drop table in afterEach has purge and fails due to invalid authority "localhost" + // Dropping the table here + sql("DROP TABLE %s", tableName); + } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java index 6e66ea2629b8..6609efa95eb1 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/RemoveOrphanFilesProcedure.java @@ -128,7 +128,7 @@ public InternalRow[] call(InternalRow args) { DataTypes.StringType, DataTypes.StringType, (k, v) -> { - equalSchemes.put(k.toString(), v.toString()); + equalAuthorities.put(k.toString(), v.toString()); return BoxedUnit.UNIT; }); } From b29e74358d7c3000f071860f3c90160c4640454f Mon Sep 17 00:00:00 2001 From: Farooq Qaiser Date: Fri, 17 May 2024 12:32:00 -0400 Subject: [PATCH 0311/1019] Use a unique field-id for delete files elements (#10347) --- .../java/org/apache/iceberg/connect/events/DataWritten.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/DataWritten.java b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/DataWritten.java index c6815925a904..d4abf9381154 100644 --- a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/DataWritten.java +++ b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/DataWritten.java @@ -49,7 +49,7 @@ public class DataWritten implements Payload { static final int DATA_FILES = 10_302; static final int DATA_FILES_ELEMENT = 10_303; static final int DELETE_FILES = 10_304; - static final int DELETE_FILES_ELEMENT = 10_304; + static final int DELETE_FILES_ELEMENT = 10_305; // Used by Avro reflection to instantiate this class when reading events, note that this does not // set the partition type so the instance cannot be re-serialized From 8f3c97196dd713cc46cd80d5efd65fef88910bcc Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 17 May 2024 10:20:57 -0700 Subject: [PATCH 0312/1019] Core, Spark 3.4: Remove redundant output in tests (#10348) --- .../apache/iceberg/metrics/TestCommitMetricsResultParser.java | 1 - .../org/apache/iceberg/metrics/TestScanMetricsResultParser.java | 1 - .../TestSystemFunctionPushDownInRowLevelOperations.java | 2 -- 3 files changed, 4 deletions(-) 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 7598555c6b25..5aa2660143a4 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCommitMetricsResultParser.java @@ -231,7 +231,6 @@ public void roundTripSerdeNoopCommitMetrics() { String expectedJson = "{ }"; String json = CommitMetricsResultParser.toJson(commitMetricsResult, true); - System.out.println(json); assertThat(json).isEqualTo(expectedJson); assertThat(CommitMetricsResultParser.fromJson(json)) .isEqualTo(ImmutableCommitMetricsResult.builder().build()); 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 47b67cd96104..2daa2b459220 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java @@ -324,7 +324,6 @@ public void roundTripSerdeNoopScanMetrics() { String expectedJson = "{ }"; String json = ScanMetricsResultParser.toJson(scanMetricsResult, true); - System.out.println(json); Assertions.assertThat(json).isEqualTo(expectedJson); Assertions.assertThat(ScanMetricsResultParser.fromJson(json)) .isEqualTo(ImmutableScanMetricsResult.builder().build()); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java index db4d10645b99..49119d319c40 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownInRowLevelOperations.java @@ -317,8 +317,6 @@ private void checkMerge(RowLevelOperationMode mode, String cond) { private List executeAndCollectFunctionCalls(String query, Object... args) { CommandResultExec command = (CommandResultExec) executeAndKeepPlan(query, args); V2TableWriteExec write = (V2TableWriteExec) command.commandPhysicalPlan(); - System.out.println("!!! WRITE PLAN !!!"); - System.out.println(write.toString()); return SparkPlanUtil.collectExprs( write.query(), expr -> expr instanceof StaticInvoke || expr instanceof ApplyFunctionExpression); From d3db4cfdd3614b522a64fce5c8dd4059d98cef49 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Sat, 18 May 2024 10:50:04 -0600 Subject: [PATCH 0313/1019] Core: Replace deprecated Roaring64Bitmap#add call with addRange (#10350) --- .../org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 97699c1c9113..72f1e00e498a 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -38,7 +38,7 @@ public void delete(long position) { @Override public void delete(long posStart, long posEnd) { - roaring64Bitmap.add(posStart, posEnd); + roaring64Bitmap.addRange(posStart, posEnd); } @Override From 0b325da234f3a257a3547cff0550b625d8dbedbb Mon Sep 17 00:00:00 2001 From: Shardul Mahadik Date: Sun, 19 May 2024 18:17:36 -0700 Subject: [PATCH 0314/1019] Spark: Coerce shorts and bytes into ints in Parquet Writer (#10349) --- .../spark/data/SparkParquetWriters.java | 13 ++- .../source/TestDataFrameWriterV2Coercion.java | 69 ++++++++++++++++ .../spark/data/SparkParquetWriters.java | 13 ++- .../source/TestDataFrameWriterV2Coercion.java | 69 ++++++++++++++++ .../spark/data/SparkParquetWriters.java | 13 ++- .../source/TestDataFrameWriterV2Coercion.java | 81 +++++++++++++++++++ 6 files changed, 255 insertions(+), 3 deletions(-) create mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index 8baea6c5ab59..1a4f7052de39 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -49,9 +49,11 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.ByteType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; @@ -267,7 +269,7 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) case BOOLEAN: return ParquetValueWriters.booleans(desc); case INT32: - return ParquetValueWriters.ints(desc); + return ints(sType, desc); case INT64: return ParquetValueWriters.longs(desc); case FLOAT: @@ -280,6 +282,15 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) } } + private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { + if (type instanceof ByteType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof ShortType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { return new UTF8StringWriter(desc); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java new file mode 100644 index 000000000000..efb6352ce8ba --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.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 org.apache.iceberg.FileFormat; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestDataFrameWriterV2Coercion extends SparkTestBaseWithCatalog { + + private final FileFormat format; + private final String dataType; + + public TestDataFrameWriterV2Coercion(FileFormat format, String dataType) { + this.format = format; + this.dataType = dataType; + } + + @Parameterized.Parameters(name = "format = {0}, dataType = {1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO, "byte"}, + new Object[] {FileFormat.ORC, "byte"}, + new Object[] {FileFormat.PARQUET, "byte"}, + new Object[] {FileFormat.AVRO, "short"}, + new Object[] {FileFormat.ORC, "short"}, + new Object[] {FileFormat.PARQUET, "short"} + }; + } + + @Test + public void testByteAndShortCoercion() { + + Dataset df = + jsonToDF( + "id " + dataType + ", data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + df.writeTo(tableName).option("write-format", format.name()).createOrReplace(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("select * from %s order by id", tableName)); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index 8baea6c5ab59..1a4f7052de39 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -49,9 +49,11 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.ByteType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; @@ -267,7 +269,7 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) case BOOLEAN: return ParquetValueWriters.booleans(desc); case INT32: - return ParquetValueWriters.ints(desc); + return ints(sType, desc); case INT64: return ParquetValueWriters.longs(desc); case FLOAT: @@ -280,6 +282,15 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) } } + private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { + if (type instanceof ByteType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof ShortType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { return new UTF8StringWriter(desc); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java new file mode 100644 index 000000000000..efb6352ce8ba --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.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 org.apache.iceberg.FileFormat; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkTestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +@RunWith(Parameterized.class) +public class TestDataFrameWriterV2Coercion extends SparkTestBaseWithCatalog { + + private final FileFormat format; + private final String dataType; + + public TestDataFrameWriterV2Coercion(FileFormat format, String dataType) { + this.format = format; + this.dataType = dataType; + } + + @Parameterized.Parameters(name = "format = {0}, dataType = {1}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO, "byte"}, + new Object[] {FileFormat.ORC, "byte"}, + new Object[] {FileFormat.PARQUET, "byte"}, + new Object[] {FileFormat.AVRO, "short"}, + new Object[] {FileFormat.ORC, "short"}, + new Object[] {FileFormat.PARQUET, "short"} + }; + } + + @Test + public void testByteAndShortCoercion() { + + Dataset df = + jsonToDF( + "id " + dataType + ", data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + df.writeTo(tableName).option("write-format", format.name()).createOrReplace(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("select * from %s order by id", tableName)); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index b2e95df59eb8..209c06bacb3e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -49,9 +49,11 @@ import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.ArrayType; +import org.apache.spark.sql.types.ByteType; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.sql.types.MapType; +import org.apache.spark.sql.types.ShortType; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; @@ -266,7 +268,7 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) case BOOLEAN: return ParquetValueWriters.booleans(desc); case INT32: - return ParquetValueWriters.ints(desc); + return ints(sType, desc); case INT64: return ParquetValueWriters.longs(desc); case FLOAT: @@ -279,6 +281,15 @@ public ParquetValueWriter primitive(DataType sType, PrimitiveType primitive) } } + private static PrimitiveWriter ints(DataType type, ColumnDescriptor desc) { + if (type instanceof ByteType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof ShortType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + private static PrimitiveWriter utf8Strings(ColumnDescriptor desc) { return new UTF8StringWriter(desc); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java new file mode 100644 index 000000000000..f51a06853a69 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestDataFrameWriterV2Coercion.java @@ -0,0 +1,81 @@ +/* + * 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.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.spark.SparkCatalogConfig; +import org.apache.iceberg.spark.TestBaseWithCatalog; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestDataFrameWriterV2Coercion extends TestBaseWithCatalog { + + @Parameters( + name = "catalogName = {0}, implementation = {1}, config = {2}, format = {3}, dataType = {4}") + public static Object[][] parameters() { + return new Object[][] { + parameter(FileFormat.AVRO, "byte"), + parameter(FileFormat.ORC, "byte"), + parameter(FileFormat.PARQUET, "byte"), + parameter(FileFormat.AVRO, "short"), + parameter(FileFormat.ORC, "short"), + parameter(FileFormat.PARQUET, "short") + }; + } + + private static Object[] parameter(FileFormat fileFormat, String dataType) { + return new Object[] { + SparkCatalogConfig.HADOOP.catalogName(), + SparkCatalogConfig.HADOOP.implementation(), + SparkCatalogConfig.HADOOP.properties(), + fileFormat, + dataType + }; + } + + @Parameter(index = 3) + private FileFormat format; + + @Parameter(index = 4) + private String dataType; + + @TestTemplate + public void testByteAndShortCoercion() { + + Dataset df = + jsonToDF( + "id " + dataType + ", data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + df.writeTo(tableName).option("write-format", format.name()).createOrReplace(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1, "a"), row(2, "b")), + sql("select * from %s order by id", tableName)); + } +} From 6d346509303dd6a938dc375d5321c2b9a0afb922 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 23 May 2024 09:17:55 +0200 Subject: [PATCH 0315/1019] Build: Bump mkdocs-material from 9.5.21 to 9.5.23 (#10353) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.21 to 9.5.23. - [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.21...9.5.23) --- 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 db6ddb394d22..4ba0bc972df6 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.21 +mkdocs-material==9.5.23 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 7d464fba3499869b1e3bb12852cdf7312bec21de Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 23 May 2024 09:18:11 +0200 Subject: [PATCH 0316/1019] Build: Bump org.springframework:spring-web from 5.3.34 to 5.3.35 (#10354) Bumps [org.springframework:spring-web](https://github.com/spring-projects/spring-framework) from 5.3.34 to 5.3.35. - [Release notes](https://github.com/spring-projects/spring-framework/releases) - [Commits](https://github.com/spring-projects/spring-framework/compare/v5.3.34...v5.3.35) --- updated-dependencies: - dependency-name: org.springframework:spring-web 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 0ad9469ed8da..738178ca2eab 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -81,7 +81,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" -spring-web = "5.3.34" +spring-web = "5.3.35" sqlite-jdbc = "3.45.3.0" testcontainers = "1.19.8" tez010 = "0.10.3" From 251edfa885af8d80284fe389fda127c382d3b4e4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 23 May 2024 09:18:20 +0200 Subject: [PATCH 0317/1019] Build: Bump software.amazon.awssdk:bom from 2.25.50 to 2.25.57 (#10367) Bumps software.amazon.awssdk:bom from 2.25.50 to 2.25.57. --- 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 738178ca2eab..68d94aaf3b42 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.50" +awssdk-bom = "2.25.57" azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From e94e3784c252c7bac058fc530a3a20aaeabb0d31 Mon Sep 17 00:00:00 2001 From: Rui Li Date: Fri, 24 May 2024 03:23:24 +0800 Subject: [PATCH 0318/1019] Hive: Use base table metadata to create HiveLock (#10016) --- .../iceberg/hive/HiveTableOperations.java | 6 +-- .../apache/iceberg/hive/TestHiveCommits.java | 39 +++++++++++++++++-- 2 files changed, 39 insertions(+), 6 deletions(-) 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 5293f915407e..64f091385297 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 @@ -179,7 +179,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { BaseMetastoreOperations.CommitStatus.FAILURE; boolean updateHiveTable = false; - HiveLock lock = lockObject(metadata); + HiveLock lock = lockObject(base); try { lock.lock(); @@ -242,7 +242,7 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { try { persistTable( - tbl, updateHiveTable, hiveLockEnabled(metadata, conf) ? null : baseMetadataLocation); + tbl, updateHiveTable, hiveLockEnabled(base, conf) ? null : baseMetadataLocation); lock.ensureActive(); commitStatus = BaseMetastoreOperations.CommitStatus.SUCCESS; @@ -510,7 +510,7 @@ private static boolean hiveEngineEnabled(TableMetadata metadata, Configuration c * @return if the hive engine related values should be enabled or not */ private static boolean hiveLockEnabled(TableMetadata metadata, Configuration conf) { - if (metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) { + if (metadata != null && metadata.properties().get(TableProperties.HIVE_LOCK_ENABLED) != null) { // We know that the property is set, so default value will not be used, return metadata.propertyAsBoolean(TableProperties.HIVE_LOCK_ENABLED, false); } 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 acf4f8dc5cd3..b3bbde460667 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.hive; +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.Mockito.any; @@ -39,6 +40,7 @@ import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; import org.apache.thrift.TException; import org.junit.jupiter.api.Test; @@ -64,7 +66,7 @@ public void testSuppressUnlockExceptions() { AtomicReference lockRef = new AtomicReference<>(); - when(spyOps.lockObject(metadataV1)) + when(spyOps.lockObject(metadataV2)) .thenAnswer( i -> { HiveLock lock = (HiveLock) i.callRealMethod(); @@ -273,11 +275,11 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted AtomicReference lock = new AtomicReference<>(); doAnswer( l -> { - lock.set(ops.lockObject(metadataV1)); + lock.set(ops.lockObject(metadataV2)); return lock.get(); }) .when(spyOps) - .lockObject(metadataV1); + .lockObject(metadataV2); concurrentCommitAndThrowException(ops, spyOps, table, lock); @@ -415,6 +417,37 @@ public void testCommitExceptionWithoutMessage() throws TException, InterruptedEx .hasMessageStartingWith("null\nCannot determine whether the commit was successful or not"); } + @Test + public void testChangeLockWithAlterTable() throws Exception { + Table table = catalog.loadTable(TABLE_IDENTIFIER); + HiveTableOperations ops = (HiveTableOperations) ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + final HiveLock initialLock = ops.lockObject(base); + + AtomicReference lockRef = new AtomicReference<>(); + HiveTableOperations spyOps = spy(ops); + doAnswer( + i -> { + lockRef.set(ops.lockObject(i.getArgument(0))); + return lockRef.get(); + }) + .when(spyOps) + .lockObject(base); + + TableMetadata newMetadata = + TableMetadata.buildFrom(base) + .setProperties( + ImmutableMap.of( + HIVE_LOCK_ENABLED, initialLock instanceof NoLock ? "true" : "false")) + .build(); + spyOps.commit(base, newMetadata); + + assertThat(lockRef).as("Lock not captured by the stub").doesNotHaveNullValue(); + assertThat(lockRef.get()) + .as("New lock mechanism shouldn't take effect before the commit completes") + .hasSameClassAs(initialLock); + } + private void commitAndThrowException( HiveTableOperations realOperations, HiveTableOperations spyOperations) throws TException, InterruptedException { From 1da3c43a80d9919bade0d6f45d05ce4a42436daa Mon Sep 17 00:00:00 2001 From: Joshua Kolash Date: Thu, 23 May 2024 15:48:33 -0400 Subject: [PATCH 0319/1019] API: Fix aggregate pushdown when optional DataFile stats are null (#10273) --- .../iceberg/expressions/BoundAggregate.java | 7 +++ .../iceberg/expressions/CountNonNull.java | 3 +- .../iceberg/expressions/MaxAggregate.java | 2 +- .../iceberg/expressions/MinAggregate.java | 2 +- .../expressions/TestAggregateEvaluator.java | 52 +++++++++++++++++++ 5 files changed, 63 insertions(+), 3 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/BoundAggregate.java b/api/src/main/java/org/apache/iceberg/expressions/BoundAggregate.java index f8db6eac2022..4be4154c38a2 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/BoundAggregate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundAggregate.java @@ -87,6 +87,13 @@ public String describe() { } } + boolean safeContainsKey(Map map, int key) { + if (map == null) { + return false; + } + return map.containsKey(key); + } + V safeGet(Map map, int key) { return safeGet(map, key, null); } diff --git a/api/src/main/java/org/apache/iceberg/expressions/CountNonNull.java b/api/src/main/java/org/apache/iceberg/expressions/CountNonNull.java index 10afd72e2e36..ecc83c9ef680 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/CountNonNull.java +++ b/api/src/main/java/org/apache/iceberg/expressions/CountNonNull.java @@ -39,7 +39,8 @@ protected Long countFor(StructLike row) { @Override protected boolean hasValue(DataFile file) { - return file.valueCounts().containsKey(fieldId) && file.nullValueCounts().containsKey(fieldId); + return safeContainsKey(file.valueCounts(), fieldId) + && file.nullValueCounts().containsKey(fieldId); } @Override diff --git a/api/src/main/java/org/apache/iceberg/expressions/MaxAggregate.java b/api/src/main/java/org/apache/iceberg/expressions/MaxAggregate.java index 754da9046f5b..d37af7470df2 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/MaxAggregate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/MaxAggregate.java @@ -40,7 +40,7 @@ protected MaxAggregate(BoundTerm term) { @Override protected boolean hasValue(DataFile file) { - boolean hasBound = file.upperBounds().containsKey(fieldId); + boolean hasBound = safeContainsKey(file.upperBounds(), fieldId); Long valueCount = safeGet(file.valueCounts(), fieldId); Long nullCount = safeGet(file.nullValueCounts(), fieldId); boolean boundAllNull = diff --git a/api/src/main/java/org/apache/iceberg/expressions/MinAggregate.java b/api/src/main/java/org/apache/iceberg/expressions/MinAggregate.java index a6bcea4145c3..667b66d6500d 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/MinAggregate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/MinAggregate.java @@ -40,7 +40,7 @@ protected MinAggregate(BoundTerm term) { @Override protected boolean hasValue(DataFile file) { - boolean hasBound = file.lowerBounds().containsKey(fieldId); + boolean hasBound = safeContainsKey(file.lowerBounds(), fieldId); Long valueCount = safeGet(file.valueCounts(), fieldId); Long nullCount = safeGet(file.nullValueCounts(), fieldId); boolean boundAllNull = diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java index 7230e721aeaa..b418dede8610 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java @@ -95,6 +95,22 @@ public class TestAggregateEvaluator { FILE, MISSING_SOME_NULLS_STATS_1, MISSING_SOME_NULLS_STATS_2 }; + private static final DataFile MISSING_ALL_OPTIONAL_STATS = + new TestDataFile( + "file_null_stats.avro", + Row.of(), + 20, + // any value counts, including nulls + null, + // null value counts + null, + // nan value counts + null, + // lower bounds + null, + // upper bounds + null); + @Test public void testIntAggregate() { List list = @@ -173,6 +189,42 @@ public void testNoStats() { assertEvaluatorResult(result, expected); } + @Test + public void testIntAggregateAllMissingStats() { + List list = + ImmutableList.of( + Expressions.countStar(), + Expressions.count("id"), + Expressions.max("id"), + Expressions.min("id")); + AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); + + aggregateEvaluator.update(MISSING_ALL_OPTIONAL_STATS); + + assertThat(aggregateEvaluator.allAggregatorsValid()).isFalse(); + StructLike result = aggregateEvaluator.result(); + Object[] expected = {20L, null, null, null}; + assertEvaluatorResult(result, expected); + } + + @Test + public void testOptionalColAllMissingStats() { + List list = + ImmutableList.of( + Expressions.countStar(), + Expressions.count("no_stats"), + Expressions.max("no_stats"), + Expressions.min("no_stats")); + AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); + + aggregateEvaluator.update(MISSING_ALL_OPTIONAL_STATS); + + assertThat(aggregateEvaluator.allAggregatorsValid()).isFalse(); + StructLike result = aggregateEvaluator.result(); + Object[] expected = {20L, null, null, null}; + assertEvaluatorResult(result, expected); + } + private void assertEvaluatorResult(StructLike result, Object[] expected) { Object[] actual = new Object[result.size()]; for (int i = 0; i < result.size(); i++) { From 67c70ea9bd77f3c01cf0c0de4879436f03d56322 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 24 May 2024 08:59:02 -0700 Subject: [PATCH 0320/1019] Spark 3.5: Support camel case session configs and options (#10310) --- .../apache/iceberg/spark/SparkConfParser.java | 28 +++++++++++++++++++ .../iceberg/spark/TestSparkWriteConf.java | 26 +++++++++++++++++ 2 files changed, 54 insertions(+) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java index 09ddc1a6d2ba..896d77a76029 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkConfParser.java @@ -270,6 +270,11 @@ protected T parse(Function conversion, T defaultValue) { if (optionValue != null) { return conversion.apply(optionValue); } + + String sparkOptionValue = options.get(toCamelCase(optionName)); + if (sparkOptionValue != null) { + return conversion.apply(sparkOptionValue); + } } if (sessionConfName != null) { @@ -277,6 +282,11 @@ protected T parse(Function conversion, T defaultValue) { if (sessionConfValue != null) { return conversion.apply(sessionConfValue); } + + String sparkSessionConfValue = sessionConf.get(toCamelCase(sessionConfName), null); + if (sparkSessionConfValue != null) { + return conversion.apply(sparkSessionConfValue); + } } if (tablePropertyName != null) { @@ -288,5 +298,23 @@ protected T parse(Function conversion, T defaultValue) { return defaultValue; } + + private String toCamelCase(String key) { + StringBuilder transformedKey = new StringBuilder(); + boolean capitalizeNext = false; + + for (char character : key.toCharArray()) { + if (character == '-') { + capitalizeNext = true; + } else if (capitalizeNext) { + transformedKey.append(Character.toUpperCase(character)); + capitalizeNext = false; + } else { + transformedKey.append(character); + } + } + + return transformedKey.toString(); + } } } 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 bf2f59f1a4e1..c2df62697882 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 @@ -88,6 +88,32 @@ public void testOptionCaseInsensitive() { assertThat(parsedValue).isEqualTo("value"); } + @TestTemplate + public void testCamelCaseSparkSessionConf() { + Table table = validationCatalog.loadTable(tableIdent); + String confName = "spark.sql.iceberg.some-int-conf"; + String sparkConfName = "spark.sql.iceberg.someIntConf"; + + withSQLConf( + ImmutableMap.of(sparkConfName, "1"), + () -> { + SparkConfParser parser = new SparkConfParser(spark, table, ImmutableMap.of()); + Integer value = parser.intConf().sessionConf(confName).parseOptional(); + assertThat(value).isEqualTo(1); + }); + } + + @TestTemplate + public void testCamelCaseSparkOption() { + Table table = validationCatalog.loadTable(tableIdent); + String option = "some-int-option"; + String sparkOption = "someIntOption"; + Map options = ImmutableMap.of(sparkOption, "1"); + SparkConfParser parser = new SparkConfParser(spark, table, options); + Integer value = parser.intConf().option(option).parseOptional(); + assertThat(value).isEqualTo(1); + } + @TestTemplate public void testDurationConf() { Table table = validationCatalog.loadTable(tableIdent); From 295aa8e49325456cf12c5774d039ff637b2e41ba Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Sat, 25 May 2024 01:23:02 +0900 Subject: [PATCH 0321/1019] AWS: Support S3 DSSE-KMS encryption (#8370) --- .../aws/s3/TestS3FileIOIntegration.java | 15 +++++++++++++++ .../iceberg/aws/s3/S3FileIOProperties.java | 14 +++++++++++--- .../apache/iceberg/aws/s3/S3RequestUtil.java | 5 +++++ .../iceberg/aws/s3/TestS3RequestUtil.java | 19 +++++++++++++++++++ docs/docs/aws.md | 5 +++-- 5 files changed, 53 insertions(+), 5 deletions(-) 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 244f10dca397..9a52ae5ebfd8 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 @@ -291,6 +291,21 @@ public void testServerSideKmsEncryptionWithDefaultKey() throws Exception { aliasListEntry -> assertThat(aliasListEntry.aliasName()).isEqualTo("alias/aws/s3")); } + @Test + public void testDualLayerServerSideKmsEncryption() throws Exception { + S3FileIOProperties properties = new S3FileIOProperties(); + properties.setSseType(S3FileIOProperties.DSSE_TYPE_KMS); + properties.setSseKey(kmsKeyArn); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, properties); + write(s3FileIO); + validateRead(s3FileIO); + GetObjectResponse response = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()) + .response(); + assertThat(response.serverSideEncryption()).isEqualTo(ServerSideEncryption.AWS_KMS_DSSE); + assertThat(response.ssekmsKeyId()).isEqualTo(kmsKeyArn); + } + @Test public void testServerSideCustomEncryption() throws Exception { // generate key 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 b3801d3f3621..1bcbdf7ed36b 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 @@ -92,6 +92,14 @@ public class S3FileIOProperties implements Serializable { */ public static final String SSE_TYPE_KMS = "kms"; + /** + * S3 DSSE-KMS encryption. + * + *

    For more details: + * https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingDSSEncryption.html + */ + public static final String DSSE_TYPE_KMS = "dsse-kms"; + /** * S3 SSE-S3 encryption. * @@ -109,9 +117,9 @@ public class S3FileIOProperties implements Serializable { public static final String SSE_TYPE_CUSTOM = "custom"; /** - * If S3 encryption type is SSE-KMS, input is a KMS Key ID or ARN. In case this property is not - * set, default key "aws/s3" is used. If encryption type is SSE-C, input is a custom base-64 - * AES256 symmetric key. + * If S3 encryption type is SSE-KMS or DSSE-KMS, input is a KMS Key ID or ARN. In case this + * property is not set, default key "aws/s3" is used. If encryption type is SSE-C, input is a + * custom base-64 AES256 symmetric key. */ public static final String SSE_KEY = "s3.sse.key"; diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java index 62738e636bfe..235a3532a45b 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3RequestUtil.java @@ -111,6 +111,11 @@ static void configureEncryption( kmsKeySetter.apply(s3FileIOProperties.sseKey()); break; + case S3FileIOProperties.DSSE_TYPE_KMS: + encryptionSetter.apply(ServerSideEncryption.AWS_KMS_DSSE); + kmsKeySetter.apply(s3FileIOProperties.sseKey()); + break; + case S3FileIOProperties.SSE_TYPE_S3: encryptionSetter.apply(ServerSideEncryption.AES256); break; diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java index 17e3e238e0fd..379fde9244b4 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java @@ -88,6 +88,25 @@ public void testConfigureServerSideKmsEncryption() { Assertions.assertThat(customMd5).isNull(); } + @Test + public void testConfigureDualLayerServerSideKmsEncryption() { + S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); + s3FileIOProperties.setSseType(S3FileIOProperties.DSSE_TYPE_KMS); + s3FileIOProperties.setSseKey("key"); + S3RequestUtil.configureEncryption( + s3FileIOProperties, + this::setServerSideEncryption, + this::setKmsKeyId, + this::setCustomAlgorithm, + this::setCustomKey, + this::setCustomMd5); + Assertions.assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AWS_KMS_DSSE); + Assertions.assertThat(kmsKeyId).isEqualTo("key"); + Assertions.assertThat(customAlgorithm).isNull(); + Assertions.assertThat(customKey).isNull(); + Assertions.assertThat(customMd5).isNull(); + } + @Test public void testConfigureEncryptionSkipNullSetters() { S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); diff --git a/docs/docs/aws.md b/docs/docs/aws.md index b2f3241654de..bba968fa5586 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -319,14 +319,15 @@ Here are the configurations that users can tune related to this feature: * [SSE-S3](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingServerSideEncryption.html): When you use Server-Side Encryption with Amazon S3-Managed Keys (SSE-S3), each object is encrypted with a unique key. As an additional safeguard, it encrypts the key itself with a master key that it regularly rotates. Amazon S3 server-side encryption uses one of the strongest block ciphers available, 256-bit Advanced Encryption Standard (AES-256), to encrypt your data. * [SSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/dev/UsingKMSEncryption.html): Server-Side Encryption with Customer Master Keys (CMKs) Stored in AWS Key Management Service (SSE-KMS) is similar to SSE-S3, but with some additional benefits and charges for using this service. There are separate permissions for the use of a CMK that provides added protection against unauthorized access of your objects in Amazon S3. SSE-KMS also provides you with an audit trail that shows when your CMK was used and by whom. Additionally, you can create and manage customer managed CMKs or use AWS managed CMKs that are unique to you, your service, and your Region. +* [DSSE-KMS](https://docs.aws.amazon.com/AmazonS3/latest/userguide/UsingDSSEncryption.html): Dual-layer Server-Side Encryption with AWS Key Management Service keys (DSSE-KMS) is similar to SSE-KMS, but applies two layers of encryption to objects when they are uploaded to Amazon S3. DSSE-KMS can be used to fulfill compliance standards that require you to apply multilayer encryption to your data and have full control of your encryption keys. * [SSE-C](https://docs.aws.amazon.com/AmazonS3/latest/dev/ServerSideEncryptionCustomerKeys.html): With Server-Side Encryption with Customer-Provided Keys (SSE-C), you manage the encryption keys and Amazon S3 manages the encryption, as it writes to disks, and decryption when you access your objects. To enable server side encryption, use the following configuration properties: | Property | Default | Description | | --------------------------------- | ---------------------------------------- | ------------------------------------------------------ | -| s3.sse.type | `none` | `none`, `s3`, `kms` or `custom` | -| s3.sse.key | `aws/s3` for `kms` type, null otherwise | A KMS Key ID or ARN for `kms` type, or a custom base-64 AES256 symmetric key for `custom` type. | +| s3.sse.type | `none` | `none`, `s3`, `kms`, `dsse-kms` or `custom` | +| s3.sse.key | `aws/s3` for `kms` and `dsse-kms` types, null otherwise | A KMS Key ID or ARN for `kms` and `dsse-kms` types, or a custom base-64 AES256 symmetric key for `custom` type. | | s3.sse.md5 | null | If SSE type is `custom`, this value must be set as the base-64 MD5 digest of the symmetric key to ensure integrity. | ### S3 Access Control List From 10bfbd29b47864857ce13a70c3077ff049c8701a Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 27 May 2024 15:44:49 +0800 Subject: [PATCH 0322/1019] Docs: add metrics-reporting back (#10377) --- docs/mkdocs.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 9037cf174564..10479067770f 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -27,6 +27,7 @@ nav: - configuration.md - evolution.md - maintenance.md + - metrics-reporting.md - partitioning.md - performance.md - reliability.md From d2443ca310e5ae0b3b5cf01e86863e6ef4b21e35 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 09:48:58 +0200 Subject: [PATCH 0323/1019] Build: Bump software.amazon.awssdk:bom from 2.25.57 to 2.25.60 (#10385) --- 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 68d94aaf3b42..317e0631ae10 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" awaitility = "4.2.1" -awssdk-bom = "2.25.57" +awssdk-bom = "2.25.60" azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 64d64137048dfbea99435afdcb89b8d9b60ee4e2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 12:30:43 +0200 Subject: [PATCH 0324/1019] Build: Bump io.netty:netty-buffer from 4.1.109.Final to 4.1.110.Final (#10384) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.109.Final to 4.1.110.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.109.Final...netty-4.1.110.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 317e0631ae10..626ba9f01fbd 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,8 +66,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.82.0" -netty-buffer = "4.1.109.Final" -netty-buffer-compat = "4.1.109.Final" +netty-buffer = "4.1.110.Final" +netty-buffer-compat = "4.1.110.Final" object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" From 6e0da97f21f2d1c30e411720673c7b33c21bc228 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 12:31:07 +0200 Subject: [PATCH 0325/1019] Build: Bump io.airlift:aircompressor from 0.26 to 0.27 (#10383) Bumps [io.airlift:aircompressor](https://github.com/airlift/aircompressor) from 0.26 to 0.27. - [Commits](https://github.com/airlift/aircompressor/compare/0.26...0.27) --- updated-dependencies: - dependency-name: io.airlift:aircompressor 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 626ba9f01fbd..202f95ec3f32 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -23,7 +23,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" -aircompressor = "0.26" +aircompressor = "0.27" arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.25.3" From 29b6dd25918ae0b4abd8fa10e5ae4cf70b8db22f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 12:31:27 +0200 Subject: [PATCH 0326/1019] Build: Bump org.springframework:spring-web from 5.3.35 to 5.3.36 (#10382) Bumps [org.springframework:spring-web](https://github.com/spring-projects/spring-framework) from 5.3.35 to 5.3.36. - [Release notes](https://github.com/spring-projects/spring-framework/releases) - [Commits](https://github.com/spring-projects/spring-framework/compare/v5.3.35...v5.3.36) --- updated-dependencies: - dependency-name: org.springframework:spring-web 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 202f95ec3f32..bc92ae82a5bc 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -81,7 +81,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" -spring-web = "5.3.35" +spring-web = "5.3.36" sqlite-jdbc = "3.45.3.0" testcontainers = "1.19.8" tez010 = "0.10.3" From fa0dcf4e188e8125392abbbc624dacf42a3250fd Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Mon, 27 May 2024 16:21:56 +0200 Subject: [PATCH 0327/1019] Prevent deadlock in Jackson (#10379) Following warning is always printed when using Iceberg REST clients: ``` PropertyNamingStrategy.KebabCaseStrategy is used but it has been deprecated due to risk of deadlock. Consider using PropertyNamingStrategies.KebabCaseStrategy instead. See https://github.com/FasterXML/jackson-databind/issues/2715 for more details. ``` --- .../java/org/apache/iceberg/aws/s3/signer/S3ObjectMapper.java | 4 ++-- .../main/java/org/apache/iceberg/rest/RESTObjectMapper.java | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3ObjectMapper.java b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3ObjectMapper.java index b763c9acebd7..89145b2465e5 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3ObjectMapper.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3ObjectMapper.java @@ -29,7 +29,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.JsonSerializer; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.PropertyNamingStrategy; +import com.fasterxml.jackson.databind.PropertyNamingStrategies; import com.fasterxml.jackson.databind.SerializerProvider; import com.fasterxml.jackson.databind.module.SimpleModule; import java.io.IOException; @@ -57,7 +57,7 @@ static ObjectMapper mapper() { // even though using new PropertyNamingStrategy.KebabCaseStrategy() is deprecated // and PropertyNamingStrategies.KebabCaseStrategy.INSTANCE (introduced in jackson 2.14) is // recommended, we can't use it because Spark still relies on jackson 2.13.x stuff - MAPPER.setPropertyNamingStrategy(new PropertyNamingStrategy.KebabCaseStrategy()); + MAPPER.setPropertyNamingStrategy(new PropertyNamingStrategies.KebabCaseStrategy()); MAPPER.registerModule(initModule()); isInitialized = true; } diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTObjectMapper.java b/core/src/main/java/org/apache/iceberg/rest/RESTObjectMapper.java index f9572b227d10..36ed10916ab1 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTObjectMapper.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTObjectMapper.java @@ -23,7 +23,7 @@ import com.fasterxml.jackson.core.JsonFactory; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; -import com.fasterxml.jackson.databind.PropertyNamingStrategy; +import com.fasterxml.jackson.databind.PropertyNamingStrategies; class RESTObjectMapper { private static final JsonFactory FACTORY = new JsonFactory(); @@ -38,7 +38,7 @@ static ObjectMapper mapper() { if (!isInitialized) { MAPPER.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); MAPPER.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); - MAPPER.setPropertyNamingStrategy(new PropertyNamingStrategy.KebabCaseStrategy()); + MAPPER.setPropertyNamingStrategy(new PropertyNamingStrategies.KebabCaseStrategy()); RESTSerializers.registerAll(MAPPER); isInitialized = true; } From a6b859f86d1f8b739ef6b5abe43e85f70a353eaf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 27 May 2024 16:22:12 +0200 Subject: [PATCH 0328/1019] Build: Bump nessie from 0.82.0 to 0.83.2 (#10381) Bumps `nessie` from 0.82.0 to 0.83.2. Updates `org.projectnessie.nessie:nessie-client` from 0.82.0 to 0.83.2 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.82.0 to 0.83.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.82.0 to 0.83.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.82.0 to 0.83.2 --- 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 bc92ae82a5bc..aaac1258ea22 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -65,7 +65,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.82.0" +nessie = "0.83.2" netty-buffer = "4.1.110.Final" netty-buffer-compat = "4.1.110.Final" object-client-bundle = "3.3.2" From 4046d5aa9fdc4be47860ca3465566248f3fb79ef Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Mon, 27 May 2024 14:52:29 -0700 Subject: [PATCH 0329/1019] Url encode field names for partition paths (#10329) * Url encode field ids for partition paths * Cleanup tests * Add test for partition paths --- .../org/apache/iceberg/PartitionSpec.java | 2 +- .../apache/iceberg/TestPartitionPaths.java | 12 +++++++++++- .../apache/iceberg/TestLocationProvider.java | 19 +++++++++++++++++++ 3 files changed, 31 insertions(+), 2 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 0c29edea364f..4fcb110db87c 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -189,7 +189,7 @@ public String partitionToPath(StructLike data) { if (i > 0) { sb.append("/"); } - sb.append(field.name()).append("=").append(escape(valueString)); + sb.append(escape(field.name())).append("=").append(escape(valueString)); } return sb.toString(); } diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java index 2fda247a33c8..5455415da015 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java @@ -32,7 +32,8 @@ public class TestPartitionPaths { new Schema( Types.NestedField.required(1, "id", Types.IntegerType.get()), Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "ts", Types.TimestampType.withoutZone())); + Types.NestedField.optional(3, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(4, "\"esc\"#1", Types.StringType.get())); @Test public void testPartitionPath() { @@ -62,4 +63,13 @@ public void testEscapedStrings() { .as("Should escape / as %2F") .isEqualTo("data=a%2Fb%2Fc%2Fd/data_trunc=a%2Fb%2Fc%2Fd"); } + + @Test + public void testEscapedFieldNames() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("\"esc\"#1").build(); + + assertThat(spec.partitionToPath(Row.of("a/b/c/d"))) + .as("Should escape \" as %22 and # as %23") + .isEqualTo("%22esc%22%231=a%2Fb%2Fc%2Fd"); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 1b9c6581200a..93dd1a8555ce 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -26,6 +26,7 @@ import java.util.Map; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.base.Splitter; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -285,4 +286,22 @@ public void testObjectStorageWithinTableLocation() { assertThat(parts).element(2).asString().isNotEmpty(); assertThat(parts).element(3).asString().isEqualTo("test.parquet"); } + + @TestTemplate + public void testEncodedFieldNameInPartitionPath() { + // 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.updateSchema().addColumn("data#1", Types.StringType.get()).commit(); + table.updateSpec().addField("data#1").commit(); + + // Use a partition value that has a special character + StructLike partitionData = TestHelpers.CustomRow.of(0, "val#1"); + + String fileLocation = + table.locationProvider().newDataLocation(table.spec(), partitionData, "test.parquet"); + List parts = Splitter.on("/").splitToList(fileLocation); + String partitionString = parts.get(parts.size() - 2); + + assertThat(partitionString).isEqualTo("data%231=val%231"); + } } From 6725cf7a711478c476e39d2148aa1c2449ea56d0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 28 May 2024 09:55:57 +0200 Subject: [PATCH 0330/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.15.1 to 3.16.0 (#10269) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.15.1 to 3.16.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.15.1...v3.16.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 aaac1258ea22..2ad34002d1ed 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -76,7 +76,7 @@ roaringbitmap = "1.0.6" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.15.1" +snowflake-jdbc = "3.16.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" From 8b05d68495cb2dcdb81225edc98065dd3de48440 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 30 May 2024 14:20:28 +0800 Subject: [PATCH 0331/1019] Spark 3.5: Only traverse ancestors of current snapshot when building changelog scan (#10252) --- .../spark/extensions/TestChangelogTable.java | 63 +++++++++++++++++++ .../spark/source/SparkScanBuilder.java | 13 +++- 2 files changed, 75 insertions(+), 1 deletion(-) 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 daf45626753b..79c52f25e31f 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 @@ -291,6 +291,69 @@ public void testMetadataColumns() { rows); } + @TestTemplate + public void testQueryWithRollback() { + createTable(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + long rightAfterSnap1 = waitUntilAfter(snap1.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + long rightAfterSnap2 = waitUntilAfter(snap2.timestampMillis()); + + sql( + "CALL %s.system.rollback_to_snapshot('%s', %d)", + catalogName, tableIdent, snap1.snapshotId()); + table.refresh(); + assertThat(table.currentSnapshot()).isEqualTo(snap1); + + sql("INSERT OVERWRITE %s VALUES (-2, 'a')", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); + + assertEquals( + "Should have expected changed rows up to snapshot 3", + ImmutableList.of( + row(1, "a", "INSERT", 0, snap1.snapshotId()), + row(1, "a", "DELETE", 1, snap3.snapshotId()), + row(-2, "a", "INSERT", 1, snap3.snapshotId())), + changelogRecords(null, rightAfterSnap3)); + + assertEquals( + "Should have expected changed rows up to snapshot 2", + ImmutableList.of(row(1, "a", "INSERT", 0, snap1.snapshotId())), + changelogRecords(null, rightAfterSnap2)); + + assertEquals( + "Should have expected changed rows from snapshot 3 only since snapshot 2 is on a different branch.", + ImmutableList.of( + row(1, "a", "DELETE", 0, snap3.snapshotId()), + row(-2, "a", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap1, snap3.timestampMillis())); + + assertEquals( + "Should have expected changed rows from snapshot 3", + ImmutableList.of( + row(1, "a", "DELETE", 0, snap3.snapshotId()), + row(-2, "a", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap2, null)); + + sql( + "CALL %s.system.set_current_snapshot('%s', %d)", + catalogName, tableIdent, snap2.snapshotId()); + table.refresh(); + assertThat(table.currentSnapshot()).isEqualTo(snap2); + assertEquals( + "Should have expected changed rows from snapshot 2 only since snapshot 3 is on a different branch.", + ImmutableList.of(row(2, "b", "INSERT", 0, snap2.snapshotId())), + changelogRecords(rightAfterSnap1, null)); + } + private void createTableWithDefaultRows() { createTable(); insertDefaultRows(); 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 09c09c6caad7..d6f34231ae75 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 @@ -546,7 +546,7 @@ public Scan buildChangelogScan() { } if (endTimestamp != null) { - endSnapshotId = SnapshotUtil.nullableSnapshotIdAsOfTime(table, endTimestamp); + endSnapshotId = getEndSnapshotId(endTimestamp); if ((startSnapshotId == null && endSnapshotId == null) || (startSnapshotId != null && startSnapshotId.equals(endSnapshotId))) { emptyScan = true; @@ -589,6 +589,17 @@ private Long getStartSnapshotId(Long startTimestamp) { } } + private Long getEndSnapshotId(Long endTimestamp) { + Long endSnapshotId = null; + for (Snapshot snapshot : SnapshotUtil.currentAncestors(table)) { + if (snapshot.timestampMillis() <= endTimestamp) { + endSnapshotId = snapshot.snapshotId(); + break; + } + } + return endSnapshotId; + } + public Scan buildMergeOnReadScan() { Preconditions.checkArgument( readConf.snapshotId() == null && readConf.asOfTimestamp() == null && readConf.tag() == null, From fd26d47289c0e18408ef2e9e23469fc33836e16b Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 30 May 2024 16:27:11 +0800 Subject: [PATCH 0332/1019] docs: Add archive for documentations older than 1.4.0 (#10374) * Docs: Add archive for documentations older than 1.4.0 * Update archive.md --- site/docs/archive.md | 34 ++++++++++++++++++++++++++++++++++ site/nav.yml | 1 + 2 files changed, 35 insertions(+) create mode 100644 site/docs/archive.md diff --git a/site/docs/archive.md b/site/docs/archive.md new file mode 100644 index 000000000000..0767c0a11c63 --- /dev/null +++ b/site/docs/archive.md @@ -0,0 +1,34 @@ +--- +title: "Archive" +--- + + +Documentations of Iceberg versions older than 1.4.0 are no longer maintained. Here are the links to web archive. + +- [1.3.1](https://web.archive.org/web/20231210001939/https://iceberg.apache.org/docs/1.3.1/) +- [1.3.0](https://web.archive.org/web/20230923205931/https://iceberg.apache.org/docs/1.3.0/) +- [1.2.1](https://web.archive.org/web/20230923215523/https://iceberg.apache.org/docs/1.2.1/) +- [1.2.0](https://web.archive.org/web/20230611013035/https://iceberg.apache.org/docs/1.2.0/) +- [1.1.0](https://web.archive.org/web/20230611011106/https://iceberg.apache.org/docs/1.1.0/) +- [1.0.0](https://web.archive.org/web/20230510171002/https://iceberg.apache.org/docs/1.0.0/) +- [0.14.1](https://web.archive.org/web/20230507183526/https://iceberg.apache.org/docs/0.14.1/) +- [0.14.0](https://web.archive.org/web/20230510192858/https://iceberg.apache.org/docs/0.14.0/) +- [0.13.2](https://web.archive.org/web/20230508105038/https://iceberg.apache.org/docs/0.13.2/) +- [0.13.1](https://web.archive.org/web/20230508052534/https://iceberg.apache.org/docs/0.13.1/) +- [0.13.0](https://web.archive.org/web/20230510202955/https://iceberg.apache.org/docs/0.13.0/) +- [0.12.1](https://web.archive.org/web/20230514073135/https://iceberg.apache.org/docs/0.12.1/) diff --git a/site/nav.yml b/site/nav.yml index 014e47b3a015..3fe17a94953f 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -30,6 +30,7 @@ nav: - 1.4.2: '!include docs/docs/1.4.2/mkdocs.yml' - 1.4.1: '!include docs/docs/1.4.1/mkdocs.yml' - 1.4.0: '!include docs/docs/1.4.0/mkdocs.yml' + - archive: archive.md - Releases: releases.md - Blogs: blogs.md - Talks: talks.md From 329b9db928139ed522916e8554c231d156af6de1 Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 30 May 2024 17:41:40 +0200 Subject: [PATCH 0333/1019] Flink 1.19: Fix flaky TestIcebergSourceFailover > testBoundedWithSavepoint (#10393) --- .../source/TestIcebergSourceFailover.java | 36 ++++++++++--------- ...gSourceFailoverWithWatermarkExtractor.java | 13 +++++-- .../source/TestIcebergSourceFailover.java | 36 ++++++++++--------- ...gSourceFailoverWithWatermarkExtractor.java | 13 +++++-- .../source/TestIcebergSourceFailover.java | 36 ++++++++++--------- ...gSourceFailoverWithWatermarkExtractor.java | 13 +++++-- 6 files changed, 90 insertions(+), 57 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7047a62a2c32..a378a0b93be3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -24,7 +24,7 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -61,6 +61,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; public class TestIcebergSourceFailover { @@ -91,9 +92,10 @@ public class TestIcebergSourceFailover { new HadoopTableResource( TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); + @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) @@ -137,7 +139,7 @@ public void testBoundedWithSavepoint() throws Exception { JobID jobId = jobClient.getJobID(); // Write something, but do not finish before checkpoint is created - RecordCounterToFail.waitToFail(); + RecordCounterToWait.waitForCondition(); CompletableFuture savepoint = miniClusterResource .getClusterClient() @@ -146,7 +148,7 @@ public void testBoundedWithSavepoint() throws Exception { false, TEMPORARY_FOLDER.newFolder().toPath().toString(), SavepointFormatType.CANONICAL); - RecordCounterToFail.continueProcessing(); + RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint String savepointPath = savepoint.get(); @@ -189,16 +191,16 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - createBoundedStreams(env, expectedRecords.size() / 2); + createBoundedStreams(env, 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); JobID jobId = jobClient.getJobID(); - RecordCounterToFail.waitToFail(); + RecordCounterToWait.waitForCondition(); triggerFailover( failoverType, jobId, - RecordCounterToFail::continueProcessing, + RecordCounterToWait::continueProcessing, miniClusterResource.getMiniCluster()); assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); @@ -278,7 +280,7 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) TypeInformation.of(RowData.class)); DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, failAfter); + RecordCounterToWait.wrapWithFailureAfter(stream, failAfter); // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee // exactly-once behavior. When Iceberg sink, we can verify end-to-end @@ -330,31 +332,31 @@ private static void restartTaskManager(Runnable afterFailAction, MiniCluster min miniCluster.startTaskManager(); } - private static class RecordCounterToFail { + private static class RecordCounterToWait { private static AtomicInteger records; - private static CompletableFuture fail; + private static CountDownLatch countDownLatch; private static CompletableFuture continueProcessing; - private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { + private static DataStream wrapWithFailureAfter(DataStream stream, int condition) { records = new AtomicInteger(); - fail = new CompletableFuture<>(); continueProcessing = new CompletableFuture<>(); + countDownLatch = new CountDownLatch(stream.getParallelism()); return stream.map( record -> { - boolean reachedFailPoint = records.incrementAndGet() > failAfter; - boolean notFailedYet = !fail.isDone(); + boolean reachedFailPoint = records.incrementAndGet() > condition; + boolean notFailedYet = countDownLatch.getCount() != 0; if (notFailedYet && reachedFailPoint) { - fail.complete(null); + countDownLatch.countDown(); continueProcessing.get(); } return record; }); } - private static void waitToFail() throws ExecutionException, InterruptedException { - fail.get(); + private static void waitForCondition() throws InterruptedException { + countDownLatch.await(); } private static void continueProcessing() { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index 27a8894ad494..cde39ec98d61 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -27,11 +27,14 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.types.Comparators; @@ -48,10 +51,16 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg @Override protected IcebergSource.Builder sourceBuilder() { - return IcebergSource.builder() + Configuration config = new Configuration(); + return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA); + .project(TestFixtures.TS_SCHEMA) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) + .flinkConfig(config); } @Override diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7047a62a2c32..a378a0b93be3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -24,7 +24,7 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -61,6 +61,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; public class TestIcebergSourceFailover { @@ -91,9 +92,10 @@ public class TestIcebergSourceFailover { new HadoopTableResource( TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); + @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) @@ -137,7 +139,7 @@ public void testBoundedWithSavepoint() throws Exception { JobID jobId = jobClient.getJobID(); // Write something, but do not finish before checkpoint is created - RecordCounterToFail.waitToFail(); + RecordCounterToWait.waitForCondition(); CompletableFuture savepoint = miniClusterResource .getClusterClient() @@ -146,7 +148,7 @@ public void testBoundedWithSavepoint() throws Exception { false, TEMPORARY_FOLDER.newFolder().toPath().toString(), SavepointFormatType.CANONICAL); - RecordCounterToFail.continueProcessing(); + RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint String savepointPath = savepoint.get(); @@ -189,16 +191,16 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - createBoundedStreams(env, expectedRecords.size() / 2); + createBoundedStreams(env, 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); JobID jobId = jobClient.getJobID(); - RecordCounterToFail.waitToFail(); + RecordCounterToWait.waitForCondition(); triggerFailover( failoverType, jobId, - RecordCounterToFail::continueProcessing, + RecordCounterToWait::continueProcessing, miniClusterResource.getMiniCluster()); assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); @@ -278,7 +280,7 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) TypeInformation.of(RowData.class)); DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, failAfter); + RecordCounterToWait.wrapWithFailureAfter(stream, failAfter); // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee // exactly-once behavior. When Iceberg sink, we can verify end-to-end @@ -330,31 +332,31 @@ private static void restartTaskManager(Runnable afterFailAction, MiniCluster min miniCluster.startTaskManager(); } - private static class RecordCounterToFail { + private static class RecordCounterToWait { private static AtomicInteger records; - private static CompletableFuture fail; + private static CountDownLatch countDownLatch; private static CompletableFuture continueProcessing; - private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { + private static DataStream wrapWithFailureAfter(DataStream stream, int condition) { records = new AtomicInteger(); - fail = new CompletableFuture<>(); continueProcessing = new CompletableFuture<>(); + countDownLatch = new CountDownLatch(stream.getParallelism()); return stream.map( record -> { - boolean reachedFailPoint = records.incrementAndGet() > failAfter; - boolean notFailedYet = !fail.isDone(); + boolean reachedFailPoint = records.incrementAndGet() > condition; + boolean notFailedYet = countDownLatch.getCount() != 0; if (notFailedYet && reachedFailPoint) { - fail.complete(null); + countDownLatch.countDown(); continueProcessing.get(); } return record; }); } - private static void waitToFail() throws ExecutionException, InterruptedException { - fail.get(); + private static void waitForCondition() throws InterruptedException { + countDownLatch.await(); } private static void continueProcessing() { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index 27a8894ad494..cde39ec98d61 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -27,11 +27,14 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.types.Comparators; @@ -48,10 +51,16 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg @Override protected IcebergSource.Builder sourceBuilder() { - return IcebergSource.builder() + Configuration config = new Configuration(); + return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA); + .project(TestFixtures.TS_SCHEMA) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) + .flinkConfig(config); } @Override diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 7047a62a2c32..a378a0b93be3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -24,7 +24,7 @@ import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; +import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; import org.apache.flink.api.common.JobID; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -61,6 +61,7 @@ import org.junit.Rule; import org.junit.Test; import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; public class TestIcebergSourceFailover { @@ -91,9 +92,10 @@ public class TestIcebergSourceFailover { new HadoopTableResource( TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); + @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) @@ -137,7 +139,7 @@ public void testBoundedWithSavepoint() throws Exception { JobID jobId = jobClient.getJobID(); // Write something, but do not finish before checkpoint is created - RecordCounterToFail.waitToFail(); + RecordCounterToWait.waitForCondition(); CompletableFuture savepoint = miniClusterResource .getClusterClient() @@ -146,7 +148,7 @@ public void testBoundedWithSavepoint() throws Exception { false, TEMPORARY_FOLDER.newFolder().toPath().toString(), SavepointFormatType.CANONICAL); - RecordCounterToFail.continueProcessing(); + RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint String savepointPath = savepoint.get(); @@ -189,16 +191,16 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - createBoundedStreams(env, expectedRecords.size() / 2); + createBoundedStreams(env, 2); JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); JobID jobId = jobClient.getJobID(); - RecordCounterToFail.waitToFail(); + RecordCounterToWait.waitForCondition(); triggerFailover( failoverType, jobId, - RecordCounterToFail::continueProcessing, + RecordCounterToWait::continueProcessing, miniClusterResource.getMiniCluster()); assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); @@ -278,7 +280,7 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) TypeInformation.of(RowData.class)); DataStream streamFailingInTheMiddleOfReading = - RecordCounterToFail.wrapWithFailureAfter(stream, failAfter); + RecordCounterToWait.wrapWithFailureAfter(stream, failAfter); // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee // exactly-once behavior. When Iceberg sink, we can verify end-to-end @@ -330,31 +332,31 @@ private static void restartTaskManager(Runnable afterFailAction, MiniCluster min miniCluster.startTaskManager(); } - private static class RecordCounterToFail { + private static class RecordCounterToWait { private static AtomicInteger records; - private static CompletableFuture fail; + private static CountDownLatch countDownLatch; private static CompletableFuture continueProcessing; - private static DataStream wrapWithFailureAfter(DataStream stream, int failAfter) { + private static DataStream wrapWithFailureAfter(DataStream stream, int condition) { records = new AtomicInteger(); - fail = new CompletableFuture<>(); continueProcessing = new CompletableFuture<>(); + countDownLatch = new CountDownLatch(stream.getParallelism()); return stream.map( record -> { - boolean reachedFailPoint = records.incrementAndGet() > failAfter; - boolean notFailedYet = !fail.isDone(); + boolean reachedFailPoint = records.incrementAndGet() > condition; + boolean notFailedYet = countDownLatch.getCount() != 0; if (notFailedYet && reachedFailPoint) { - fail.complete(null); + countDownLatch.countDown(); continueProcessing.get(); } return record; }); } - private static void waitToFail() throws ExecutionException, InterruptedException { - fail.get(); + private static void waitForCondition() throws InterruptedException { + countDownLatch.await(); } private static void continueProcessing() { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index 27a8894ad494..cde39ec98d61 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -27,11 +27,14 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkReadOptions; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.types.Comparators; @@ -48,10 +51,16 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg @Override protected IcebergSource.Builder sourceBuilder() { - return IcebergSource.builder() + Configuration config = new Configuration(); + return IcebergSource.forRowData() .tableLoader(sourceTableResource.tableLoader()) .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA); + .project(TestFixtures.TS_SCHEMA) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) + .flinkConfig(config); } @Override From 9d9514e3cafcc144de9599156a5b56f023eb54a4 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 31 May 2024 02:50:47 +0800 Subject: [PATCH 0334/1019] docs: deploy on changes in `docs/` (#10394) --- .github/workflows/site-ci.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/site-ci.yml b/.github/workflows/site-ci.yml index b0a375a37cfa..c085009a3997 100644 --- a/.github/workflows/site-ci.yml +++ b/.github/workflows/site-ci.yml @@ -22,6 +22,7 @@ on: branches: - main paths: + - docs/** - site/** workflow_dispatch: jobs: From 55d895a464bd03ad6ebcc87a88a40e56e64c7916 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Sun, 2 Jun 2024 00:12:54 +0800 Subject: [PATCH 0335/1019] Spark 3.4: Only traverse ancestors of current snapshot when building changelog scan (#10405) --- .../spark/extensions/TestChangelogTable.java | 63 +++++++++++++++++++ .../spark/source/SparkScanBuilder.java | 13 +++- 2 files changed, 75 insertions(+), 1 deletion(-) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java index ab22eee00605..d82ed114556a 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -295,6 +295,69 @@ public void testMetadataColumns() { rows); } + @Test + public void testQueryWithRollback() { + createTable(); + + sql("INSERT INTO %s VALUES (1, 'a')", tableName); + Table table = validationCatalog.loadTable(tableIdent); + Snapshot snap1 = table.currentSnapshot(); + long rightAfterSnap1 = waitUntilAfter(snap1.timestampMillis()); + + sql("INSERT INTO %s VALUES (2, 'b')", tableName); + table.refresh(); + Snapshot snap2 = table.currentSnapshot(); + long rightAfterSnap2 = waitUntilAfter(snap2.timestampMillis()); + + sql( + "CALL %s.system.rollback_to_snapshot('%s', %d)", + catalogName, tableIdent, snap1.snapshotId()); + table.refresh(); + Assert.assertEquals("Snapshot should match after rollback", table.currentSnapshot(), snap1); + + sql("INSERT OVERWRITE %s VALUES (-2, 'a')", tableName); + table.refresh(); + Snapshot snap3 = table.currentSnapshot(); + long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); + + assertEquals( + "Should have expected changed rows up to snapshot 3", + ImmutableList.of( + row(1, "a", "INSERT", 0, snap1.snapshotId()), + row(1, "a", "DELETE", 1, snap3.snapshotId()), + row(-2, "a", "INSERT", 1, snap3.snapshotId())), + changelogRecords(null, rightAfterSnap3)); + + assertEquals( + "Should have expected changed rows up to snapshot 2", + ImmutableList.of(row(1, "a", "INSERT", 0, snap1.snapshotId())), + changelogRecords(null, rightAfterSnap2)); + + assertEquals( + "Should have expected changed rows from snapshot 3 only since snapshot 2 is on a different branch.", + ImmutableList.of( + row(1, "a", "DELETE", 0, snap3.snapshotId()), + row(-2, "a", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap1, snap3.timestampMillis())); + + assertEquals( + "Should have expected changed rows from snapshot 3", + ImmutableList.of( + row(1, "a", "DELETE", 0, snap3.snapshotId()), + row(-2, "a", "INSERT", 0, snap3.snapshotId())), + changelogRecords(rightAfterSnap2, null)); + + sql( + "CALL %s.system.set_current_snapshot('%s', %d)", + catalogName, tableIdent, snap2.snapshotId()); + table.refresh(); + Assert.assertEquals("Snapshot should match after reset", table.currentSnapshot(), snap2); + assertEquals( + "Should have expected changed rows from snapshot 2 only since snapshot 3 is on a different branch.", + ImmutableList.of(row(2, "b", "INSERT", 0, snap2.snapshotId())), + changelogRecords(rightAfterSnap1, null)); + } + private void createTableWithDefaultRows() { createTable(); insertDefaultRows(); 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 09c09c6caad7..d6f34231ae75 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 @@ -546,7 +546,7 @@ public Scan buildChangelogScan() { } if (endTimestamp != null) { - endSnapshotId = SnapshotUtil.nullableSnapshotIdAsOfTime(table, endTimestamp); + endSnapshotId = getEndSnapshotId(endTimestamp); if ((startSnapshotId == null && endSnapshotId == null) || (startSnapshotId != null && startSnapshotId.equals(endSnapshotId))) { emptyScan = true; @@ -589,6 +589,17 @@ private Long getStartSnapshotId(Long startTimestamp) { } } + private Long getEndSnapshotId(Long endTimestamp) { + Long endSnapshotId = null; + for (Snapshot snapshot : SnapshotUtil.currentAncestors(table)) { + if (snapshot.timestampMillis() <= endTimestamp) { + endSnapshotId = snapshot.snapshotId(); + break; + } + } + return endSnapshotId; + } + public Scan buildMergeOnReadScan() { Preconditions.checkArgument( readConf.snapshotId() == null && readConf.asOfTimestamp() == null && readConf.tag() == null, From 6bbd6138e99b35e046c0c42b6caaf0351912d709 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 2 Jun 2024 21:44:00 +0200 Subject: [PATCH 0336/1019] Bump Azurite test-container to `3.30.0` --- .../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 ff2b9cbc5d0c..887758af4b28 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.29.0"; + private static final String DEFAULT_TAG = "3.30.0"; private static final String LOG_WAIT_REGEX = "Azurite Blob service is successfully listening at .*"; From 0675b7c441e335d3f1e52801f10b3d12f99fbfbd Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 2 Jun 2024 21:45:01 +0200 Subject: [PATCH 0337/1019] Build: Bump mkdocs-material from 9.5.23 to 9.5.25 (#10413) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.23 to 9.5.25. - [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.23...9.5.25) --- 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 4ba0bc972df6..ec9f4f07d3bf 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.23 +mkdocs-material==9.5.25 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 5895d4e300a7f0b770e87142454810db56724c2e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 08:03:11 +0200 Subject: [PATCH 0338/1019] Build: Bump org.assertj:assertj-core from 3.25.3 to 3.26.0 (#10416) Bumps [org.assertj:assertj-core](https://github.com/assertj/assertj) from 3.25.3 to 3.26.0. - [Release notes](https://github.com/assertj/assertj/releases) - [Commits](https://github.com/assertj/assertj/compare/assertj-build-3.25.3...assertj-build-3.26.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 2ad34002d1ed..bfb6fb3765dc 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -26,7 +26,7 @@ antlr = "4.9.3" aircompressor = "0.27" arrow = "15.0.2" avro = "1.11.3" -assertj-core = "3.25.3" +assertj-core = "3.26.0" awaitility = "4.2.1" awssdk-bom = "2.25.60" azuresdk-bom = "1.2.23" From 66a05a32c9230991a5208b5a6edd738c975451fb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 08:09:13 +0200 Subject: [PATCH 0339/1019] Build: Bump guava from 33.2.0-jre to 33.2.1-jre (#10414) Bumps `guava` from 33.2.0-jre to 33.2.1-jre. Updates `com.google.guava:guava` from 33.2.0-jre to 33.2.1-jre - [Release notes](https://github.com/google/guava/releases) - [Commits](https://github.com/google/guava/commits) Updates `com.google.guava:guava-testlib` from 33.2.0-jre to 33.2.1-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-patch - dependency-name: com.google.guava:guava-testlib 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 bfb6fb3765dc..b6017bfc83b0 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -42,7 +42,7 @@ flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} google-libraries-bom = "26.28.0" -guava = "33.2.0-jre" +guava = "33.2.1-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" httpcomponents-httpclient5 = "5.3.1" From 818d25b7db0cacca1910fa3c30edba2e64552399 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 3 Jun 2024 08:09:39 +0200 Subject: [PATCH 0340/1019] Build: Bump org.xerial:sqlite-jdbc from 3.45.3.0 to 3.46.0.0 (#10415) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.45.3.0 to 3.46.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.45.3.0...3.46.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 b6017bfc83b0..4f876497811a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.36" -sqlite-jdbc = "3.45.3.0" +sqlite-jdbc = "3.46.0.0" testcontainers = "1.19.8" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From d155009f82716d0e245c083891be0eafdc35fdaa Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 3 Jun 2024 08:11:33 +0200 Subject: [PATCH 0341/1019] Docs: Refer to the README.md in `site/` for the docs (#10402) --- site/README.md | 18 +++++----- site/docs/how-to-release.md | 70 +------------------------------------ 2 files changed, 10 insertions(+), 78 deletions(-) diff --git a/site/README.md b/site/README.md index b1f9310e8bf8..39bc1d931b81 100644 --- a/site/README.md +++ b/site/README.md @@ -24,7 +24,6 @@ This subproject contains the [MkDocs projects](https://www.mkdocs.org/) that def * Python >=3.9 * pip - ## Usage The directory structure in this repository aims to mimic the sitemap hierarchy of the website. This helps contributors find the source files needed when updating or adding new documentation. It's helpful to have some basic understanding of the MkDocs framework defaults. @@ -35,7 +34,7 @@ In MkDocs, the [`docs_dir`](https://www.mkdocs.org/user-guide/configuration/#doc ### Iceberg docs layout -The static Iceberg website lives under the `/site` directory, while the versioned documentation lives under the `/docs` of the main Iceberg repository. The `/site/docs` directory is named that way to follow the [MkDocs convention](https://www.mkdocs.org/user-guide/configuration/#docs_dir). The `/docs` directory contains the current state of the versioned documentation with local revisions. Notice that the root `/site` and `/docs` just happened to share the same naming convention as MkDocs but does not correlate to the mkdocs +The static Iceberg website lives under the `/site` directory, while the versioned documentation lives under the `/docs` of the main Iceberg repository. The `/site/docs` directory is named that way to follow the [MkDocs convention](https://www.mkdocs.org/user-guide/configuration/#docs_dir). The `/docs` directory contains the current state of the versioned documentation with local revisions. Notice that the root `/site` and `/docs` just happened to share the same naming convention as MkDocs but does not correlate to the mkdocs. The static Iceberg site pages are Markdown files that live at `/site/docs/*.md`. The versioned documentation are Markdown files that live at `/docs/docs/*.md` files. You may ask where the older versions of the docs and javadocs are, which is covered later in the build section. @@ -58,11 +57,12 @@ The static Iceberg site pages are Markdown files that live at `/site/docs/*.md`. ├── mkdocs.yml └── requirements.txt ``` + ### Building the versioned docs The Iceberg versioned docs are committed in two [orphan](https://git-scm.com/docs/gitglossary#Documentation/gitglossary.txt-aiddeforphanaorphan) branches and mounted using [git worktree](https://git-scm.com/docs/git-worktree) at build time: - 1. [`docs`](https://github.com/apache/iceberg/tree/docs) - contains the state of the documenation source files (`/docs`) during release. These versions are mounted at the `/site/docs/docs/` directory at build time. + 1. [`docs`](https://github.com/apache/iceberg/tree/docs) - contains the state of the documentation source files (`/docs`) during release. These versions are mounted at the `/site/docs/docs/` directory at build time. 1. [`javadoc`](https://github.com/apache/iceberg/tree/javadoc) - contains prior statically generated versions of the javadocs mounted at `/site/docs/javadoc/` directory at build time. The `latest` version, is a soft link to the most recent [semver version](https://semver.org/) in the `docs` branch. The `nightly` version, is a soft link to the current local state of the `/docs` markdown files. @@ -104,12 +104,12 @@ This step will generate the staged source code which blends into the original so To run this, run the `serve` recipe, which runs the `build` recipe and calls `mkdocs serve`. This will run locally at . -``` +```sh make serve ``` To clear all build files, run `clean`. -``` +```sh make clean ``` @@ -117,7 +117,7 @@ make clean One of the great advantages to the MkDocs material plugin is the [offline feature](https://squidfunk.github.io/mkdocs-material/plugins/offline). You can view the Iceberg docs without the need of a server. To enable OFFLINE builds, add theOFFLINE environment variable to either `build` or `serve` recipes. -``` +```sh make build OFFLINE=true ``` @@ -132,11 +132,11 @@ Deploying the docs is a two step process: > The `make release` directive is currently unavailable as we wanted to discuss the best way forward on how or if we should automate the release. It involves taking an existing snapshot of the versioned documentation, and potentially automerging the [`docs` branch](https://github.com/apache/iceberg/tree/docs) and the [`javadoc` branch](https://github.com/apache/iceberg/tree/javadoc) which are independent from the `main` branch. Once this is complete, we can create a pull request with an offline build of the documentation to verify everything renders correctly, and then have the release manager merge that PR to finalize the docs release. So the real process would be manually invoking a docs release action, then merging a pull request. 1. Release a new version by copying the current `/docs` directory to a new version directory in the `docs` branch and a new javadoc build in the `javadoc` branch. - ``` + ```sh make release ICEBERG_VERSION=${ICEBERG_VERSION} ``` 1. Build and push the generated site to `asf-site`. - ``` + ```sh make deploy ``` @@ -166,7 +166,7 @@ As mentioned in the MkDocs section, when you build MkDocs `mkdocs build`, MkDocs └── mkdocs.yml ``` -Since there are multiple MkDocs projects that build independently, links between them will initially cause a warning when building. This occurs when `mkdocs-monorepo-plugin` compiles, it must first build the versioned documentation sites before aggregating the top-level site with the generated. Due to the delayed aggregation of subdocs of `mkdocs-monorepo-plugin` there may be warnings that display for the versioned docs that compile without being able to reference documentation it expects outside of the immediate poject due to being off by one or more directories. In other words, if the relative linking required doesn't mirror the directory layout on disk, these errors will occur. The only place this occurs now is with the nav link to javadoc. For more information, refer to: +Since there are multiple MkDocs projects that build independently, links between them will initially cause a warning when building. This occurs when `mkdocs-monorepo-plugin` compiles, it must first build the versioned documentation sites before aggregating the top-level site with the generated. Due to the delayed aggregation of subdocs of `mkdocs-monorepo-plugin` there may be warnings that display for the versioned docs that compile without being able to reference documentation it expects outside the immediate project due to being off by one or more directories. In other words, if the relative linking required doesn't mirror the directory layout on disk, these errors will occur. The only place this occurs now is with the nav link to javadoc. For more information, refer to: To ensure the links work, you may use linkchecker to traverse the links on the livesite when you're running locally. This may eventually be used as part of the build unless a more suitable static solution is found. diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index 5d240b9d03eb..0d2bc48464a6 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -319,75 +319,7 @@ Create a PR in the `iceberg` repo to make revapi run on the new release. For an ### Documentation Release -Documentation needs to be updated as a part of an Iceberg release after a release candidate is passed. -The commands described below assume you are in a directory containing a local clone of the `iceberg-docs` -repository and `iceberg` repository. Adjust the commands accordingly if it is not the case. Note that all -changes in `iceberg` need to happen against the `master` branch and changes in `iceberg-docs` need to happen -against the `main` branch. - -#### Common documentation update - -1. To start the release process, run the following steps in the `iceberg-docs` repository to copy docs over: -```shell -cp -r ../iceberg/format/* ../iceberg-docs/landing-page/content/common/ -``` -2. Change into the `iceberg-docs` repository and create a branch. -```shell -cd ../iceberg-docs -git checkout -b -``` -3. Commit, push, and open a PR against the `iceberg-docs` repo (`` -> `main`) - -#### Versioned documentation update - -Once the common docs changes have been merged into `main`, the next step is to update the versioned docs. - -1. In the `iceberg-docs` repository, cut a new branch using the version number as the branch name -```shell -cd ../iceberg-docs -git checkout -b -git push --set-upstream apache -``` -2. Copy the versioned docs from the `iceberg` repo into the `iceberg-docs` repo -```shell -rm -rf ../iceberg-docs/docs/content -cp -r ../iceberg/docs ../iceberg-docs/docs/content -``` -3. Commit the changes and open a PR against the `` branch in the `iceberg-docs` repo - -#### Javadoc update - -In the `iceberg` repository, generate the javadoc for your release and copy it to the `javadoc` folder in `iceberg-docs` repo: -```shell -cd ../iceberg -./gradlew refreshJavadoc -rm -rf ../iceberg-docs/javadoc -cp -r site/docs/javadoc/ ../iceberg-docs/javadoc -``` - -This resulted changes in `iceberg-docs` should be approved in a separate PR. - -#### Update the latest branch - -Since `main` is currently the same as the version branch, one needs to rebase `latest` branch against `main`: - -```shell -git checkout latest -git rebase main -git push apache latest -``` - -#### Set latest version in iceberg-docs repo - -The last step is to update the `main` branch in `iceberg-docs` to set the latest version. -A PR needs to be published in the `iceberg-docs` repository with the following changes: -1. Update variable `latestVersions.iceberg` to the new release version in `landing-page/config.toml` -2. Update variable `latestVersions.iceberg` to the new release version and -`versions.nessie` to the version of `org.projectnessie.nessie:*` from [mkdocs.yml](https://github.com/apache/iceberg/blob/main/site/mkdocs.yml) in `docs/config.toml` -3. Update list `versions` with the new release in `landing-page/config.toml` -4. Update list `versions` with the new release in `docs/config.toml` -5. Mark the current latest release notes to past releases under `landing-page/content/common/release-notes.md` -6. Add release notes for the new release version in `landing-page/content/common/release-notes.md` +Please follow the instructions on the GitHub repository in the [`README.md` in the `site/`](https://github.com/apache/iceberg/tree/main/site) directory. # How to Verify a Release From 581aac0a13f068445d36d20fe5a805e5bc6b3328 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 3 Jun 2024 14:36:06 +0200 Subject: [PATCH 0342/1019] Build: Require approving review (#10424) --- .asf.yaml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.asf.yaml b/.asf.yaml index ba8be89ccdcf..260641b35819 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -35,6 +35,9 @@ github: protected_branches: main: + required_pull_request_reviews: + required_approving_review_count: 1 + required_linear_history: true features: From cf41f2580feeb4f94fbf52f6140ddf1dc19e159f Mon Sep 17 00:00:00 2001 From: advancedxy Date: Mon, 3 Jun 2024 22:49:40 +0800 Subject: [PATCH 0343/1019] Parquet: Remove TestHelpers in parquet module (#10428) --- .../java/org/apache/iceberg/TestHelpers.java | 85 ------------------- .../parquet/TestBloomRowGroupFilter.java | 36 ++++---- .../parquet/TestParquetEncryption.java | 37 ++++---- 3 files changed, 37 insertions(+), 121 deletions(-) delete mode 100644 parquet/src/test/java/org/apache/iceberg/TestHelpers.java diff --git a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java b/parquet/src/test/java/org/apache/iceberg/TestHelpers.java deleted file mode 100644 index 0e7627cab1f5..000000000000 --- a/parquet/src/test/java/org/apache/iceberg/TestHelpers.java +++ /dev/null @@ -1,85 +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; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.concurrent.Callable; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.generic.GenericRecord; -import org.assertj.core.api.AbstractThrowableAssert; - -public class TestHelpers { - - private TestHelpers() {} - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param containedInMessage A String that should be contained by the thrown exception's message - * @param callable A Callable that is expected to throw the exception - */ - public static void assertThrows( - String message, - Class expected, - String containedInMessage, - Callable callable) { - AbstractThrowableAssert check = - assertThatThrownBy(callable::call).as(message).isInstanceOf(expected); - if (null != containedInMessage) { - check.hasMessageContaining(containedInMessage); - } - } - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param containedInMessage A String that should be contained by the thrown exception's message - * @param runnable A Runnable that is expected to throw the runtime exception - */ - public static void assertThrows( - String message, - Class expected, - String containedInMessage, - Runnable runnable) { - AbstractThrowableAssert check = - assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); - if (null != containedInMessage) { - check.hasMessageContaining(containedInMessage); - } - } - - /** - * A convenience method to assert if an Avro field is empty - * - * @param record The record to read from - * @param field The name of the field - */ - public static void assertEmptyAvroField(GenericRecord record, String field) { - TestHelpers.assertThrows( - "Not a valid schema field: " + field, - AvroRuntimeException.class, - "Not a valid schema field: " + field, - () -> record.get(field)); - } -} diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java index 56736ae42f91..5f64c7230601 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java @@ -40,6 +40,7 @@ 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; @@ -54,7 +55,6 @@ import org.apache.avro.generic.GenericRecordBuilder; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; -import org.apache.iceberg.TestHelpers; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; @@ -413,13 +413,13 @@ public void testStartsWith() { @Test public void testMissingColumn() { - TestHelpers.assertThrows( - "Should complain about missing column in expression", - ValidationException.class, - "Cannot find field 'missing'", - () -> - new ParquetBloomRowGroupFilter(SCHEMA, lessThan("missing", 5)) - .shouldRead(parquetSchema, rowGroupMetadata, bloomStore)); + assertThatThrownBy( + () -> + new ParquetBloomRowGroupFilter(SCHEMA, equal("missing", 5)) + .shouldRead(parquetSchema, rowGroupMetadata, bloomStore)) + .as("Should complain about missing column in expression") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot find field 'missing'"); } @Test @@ -973,16 +973,16 @@ public void testCaseInsensitive() { @Test public void testMissingBloomFilterForColumn() { - TestHelpers.assertThrows( - "Should complain about missing bloom filter", - IllegalStateException.class, - "Failed to read required bloom filter for id: 10", - () -> - new ParquetBloomRowGroupFilter(SCHEMA, equal("some_nulls", "some")) - .shouldRead( - parquetSchema, - rowGroupMetadata, - new DummyBloomFilterReader(null, rowGroupMetadata))); + assertThatThrownBy( + () -> + new ParquetBloomRowGroupFilter(SCHEMA, equal("some_nulls", "some")) + .shouldRead( + parquetSchema, + rowGroupMetadata, + new DummyBloomFilterReader(null, rowGroupMetadata))) + .as("Should complain about missing bloom filter") + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Failed to read required bloom filter for id: 10"); } private static class DummyBloomFilterReader extends BloomFilterReader { diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java index c68a79c68977..ea47ecb1c268 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java @@ -23,6 +23,7 @@ import static org.apache.iceberg.parquet.ParquetWritingTestUtils.createTempFile; 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 java.io.Closeable; import java.io.File; @@ -33,7 +34,6 @@ import java.util.List; import org.apache.avro.generic.GenericData; import org.apache.iceberg.Schema; -import org.apache.iceberg.TestHelpers; import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.io.FileAppender; @@ -85,27 +85,28 @@ public void writeEncryptedFile() throws IOException { @Test public void testReadEncryptedFileWithoutKeys() throws IOException { - TestHelpers.assertThrows( - "Decrypted without keys", - ParquetCryptoRuntimeException.class, - "Trying to read file with encrypted footer. No keys available", - () -> Parquet.read(localInput(file)).project(schema).callInit().build().iterator()); + assertThatThrownBy( + () -> Parquet.read(localInput(file)).project(schema).callInit().build().iterator()) + .as("Decrypted without keys") + .isInstanceOf(ParquetCryptoRuntimeException.class) + .hasMessage("Trying to read file with encrypted footer. No keys available"); } @Test public void testReadEncryptedFileWithoutAADPrefix() throws IOException { - TestHelpers.assertThrows( - "Decrypted without AAD prefix", - ParquetCryptoRuntimeException.class, - "AAD prefix used for file encryption, " - + "but not stored in file and not supplied in decryption properties", - () -> - Parquet.read(localInput(file)) - .project(schema) - .withFileEncryptionKey(fileDek) - .callInit() - .build() - .iterator()); + assertThatThrownBy( + () -> + Parquet.read(localInput(file)) + .project(schema) + .withFileEncryptionKey(fileDek) + .callInit() + .build() + .iterator()) + .as("Decrypted without AAD prefix") + .isInstanceOf(ParquetCryptoRuntimeException.class) + .hasMessage( + "AAD prefix used for file encryption, " + + "but not stored in file and not supplied in decryption properties"); } @Test From 0220b06d8edef60837ec08a8f9ad0ea7a17e58e9 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 3 Jun 2024 16:58:16 +0200 Subject: [PATCH 0344/1019] Core: Introduce AuthConfig (#10161) --- .../aws/s3/signer/S3V4RestSignerClient.java | 26 +-- .../iceberg/rest/RESTSessionCatalog.java | 9 +- .../apache/iceberg/rest/auth/AuthConfig.java | 72 +++++++++ .../apache/iceberg/rest/auth/OAuth2Util.java | 151 +++++++++--------- 4 files changed, 166 insertions(+), 92 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/auth/AuthConfig.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java index cdbdfb3d869e..806c52420f89 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/signer/S3V4RestSignerClient.java @@ -42,6 +42,7 @@ import org.apache.iceberg.rest.HTTPClient; import org.apache.iceberg.rest.RESTClient; import org.apache.iceberg.rest.ResourcePaths; +import org.apache.iceberg.rest.auth.AuthConfig; import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession; @@ -213,12 +214,13 @@ private AuthSession authSession() { expiresAtMillis(properties()), new AuthSession( ImmutableMap.of(), - token, - null, - credential(), - SCOPE, - oauth2ServerUri(), - optionalOAuthParams()))); + AuthConfig.builder() + .token(token) + .credential(credential()) + .scope(SCOPE) + .oauth2ServerUri(oauth2ServerUri()) + .optionalOAuthParams(optionalOAuthParams()) + .build()))); } if (credentialProvided()) { @@ -229,12 +231,12 @@ private AuthSession authSession() { AuthSession session = new AuthSession( ImmutableMap.of(), - null, - null, - credential(), - SCOPE, - oauth2ServerUri(), - optionalOAuthParams()); + AuthConfig.builder() + .credential(credential()) + .scope(SCOPE) + .oauth2ServerUri(oauth2ServerUri()) + .optionalOAuthParams(optionalOAuthParams()) + .build()); long startTimeMillis = System.currentTimeMillis(); OAuthTokenResponse authResponse = OAuth2Util.fetchToken( 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 dcf92289df2e..da40d4c3ae31 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -70,6 +70,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.rest.auth.AuthConfig; import org.apache.iceberg.rest.auth.OAuth2Properties; import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.auth.OAuth2Util.AuthSession; @@ -219,7 +220,13 @@ public void initialize(String name, Map unresolved) { String token = mergedProps.get(OAuth2Properties.TOKEN); this.catalogAuth = new AuthSession( - baseHeaders, null, null, credential, scope, oauth2ServerUri, optionalOAuthParams); + baseHeaders, + AuthConfig.builder() + .credential(credential) + .scope(scope) + .oauth2ServerUri(oauth2ServerUri) + .optionalOAuthParams(optionalOAuthParams) + .build()); if (authResponse != null) { this.catalogAuth = AuthSession.fromTokenResponse( diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/AuthConfig.java b/core/src/main/java/org/apache/iceberg/rest/auth/AuthConfig.java new file mode 100644 index 000000000000..275884e1184a --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/auth/AuthConfig.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 java.util.Map; +import javax.annotation.Nullable; +import org.apache.iceberg.rest.ResourcePaths; +import org.immutables.value.Value; + +/** + * The purpose of this class is to hold configuration options for {@link + * org.apache.iceberg.rest.auth.OAuth2Util.AuthSession}. + */ +@Value.Style(redactedMask = "****") +@SuppressWarnings("ImmutablesStyle") +@Value.Immutable +public interface AuthConfig { + @Nullable + @Value.Redacted + String token(); + + @Nullable + String tokenType(); + + @Nullable + @Value.Redacted + String credential(); + + @Value.Default + default String scope() { + return OAuth2Properties.CATALOG_SCOPE; + } + + @Value.Lazy + @Nullable + default Long expiresAtMillis() { + return OAuth2Util.expiresAtMillis(token()); + } + + @Value.Default + default boolean keepRefreshed() { + return true; + } + + @Nullable + @Value.Default + default String oauth2ServerUri() { + return ResourcePaths.tokens(); + } + + Map optionalOAuthParams(); + + static ImmutableAuthConfig.Builder builder() { + return ImmutableAuthConfig.builder(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 9e36694508d9..2283aba7d02d 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -458,32 +458,11 @@ public static class AuthSession { private static final long MAX_REFRESH_WINDOW_MILLIS = 300_000; // 5 minutes private static final long MIN_REFRESH_WAIT_MILLIS = 10; private volatile Map headers; - private volatile String token; - private volatile String tokenType; - private volatile Long expiresAtMillis; - private final String credential; - private final String scope; - private volatile boolean keepRefreshed = true; - private final String oauth2ServerUri; + private volatile AuthConfig config; - private Map optionalOAuthParams = ImmutableMap.of(); - - public AuthSession( - Map baseHeaders, - String token, - String tokenType, - String credential, - String scope, - String oauth2ServerUri, - Map optionalOAuthParams) { - this.headers = RESTUtil.merge(baseHeaders, authHeaders(token)); - this.token = token; - this.tokenType = tokenType; - this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); - this.credential = credential; - this.scope = scope; - this.oauth2ServerUri = oauth2ServerUri; - this.optionalOAuthParams = optionalOAuthParams; + public AuthSession(Map baseHeaders, AuthConfig config) { + this.headers = RESTUtil.merge(baseHeaders, authHeaders(config.token())); + this.config = config; } /** @deprecated since 1.5.0, will be removed in 1.6.0 */ @@ -494,13 +473,14 @@ public AuthSession( String tokenType, String credential, String scope) { - this.headers = RESTUtil.merge(baseHeaders, authHeaders(token)); - this.token = token; - this.tokenType = tokenType; - this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); - this.credential = credential; - this.scope = scope; - this.oauth2ServerUri = ResourcePaths.tokens(); + this( + baseHeaders, + AuthConfig.builder() + .token(token) + .tokenType(tokenType) + .credential(credential) + .scope(scope) + .build()); } /** @deprecated since 1.6.0, will be removed in 1.7.0 */ @@ -512,14 +492,15 @@ public AuthSession( String credential, String scope, String oauth2ServerUri) { - this.headers = RESTUtil.merge(baseHeaders, authHeaders(token)); - this.token = token; - this.tokenType = tokenType; - this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); - this.credential = credential; - this.scope = scope; - this.oauth2ServerUri = oauth2ServerUri; - this.optionalOAuthParams = ImmutableMap.of(); + this( + baseHeaders, + AuthConfig.builder() + .token(token) + .tokenType(tokenType) + .credential(credential) + .scope(scope) + .oauth2ServerUri(oauth2ServerUri) + .build()); } public Map headers() { @@ -527,35 +508,39 @@ public Map headers() { } public String token() { - return token; + return config.token(); } public String tokenType() { - return tokenType; + return config.tokenType(); } public Long expiresAtMillis() { - return expiresAtMillis; + return config.expiresAtMillis(); } public String scope() { - return scope; + return config.scope(); } - public void stopRefreshing() { - this.keepRefreshed = false; + public synchronized void stopRefreshing() { + this.config = ImmutableAuthConfig.copyOf(config).withKeepRefreshed(false); } public String credential() { - return credential; + return config.credential(); } public String oauth2ServerUri() { - return oauth2ServerUri; + return config.oauth2ServerUri(); } public Map optionalOAuthParams() { - return optionalOAuthParams; + return config.optionalOAuthParams(); + } + + public AuthConfig config() { + return config; } @VisibleForTesting @@ -569,14 +554,7 @@ static void setTokenRefreshNumRetries(int retries) { * @return A new {@link AuthSession} with empty headers. */ public static AuthSession empty() { - return new AuthSession( - ImmutableMap.of(), - null, - null, - null, - OAuth2Properties.CATALOG_SCOPE, - null, - ImmutableMap.of()); + return new AuthSession(ImmutableMap.of(), AuthConfig.builder().build()); } /** @@ -586,7 +564,7 @@ public static AuthSession empty() { * @return interval to wait before calling refresh again, or null if no refresh is needed */ public Pair refresh(RESTClient client) { - if (token != null && keepRefreshed) { + if (token() != null && config.keepRefreshed()) { AtomicReference ref = new AtomicReference<>(null); boolean isSuccessful = Tasks.foreach(ref) @@ -612,10 +590,13 @@ public Pair refresh(RESTClient client) { } OAuthTokenResponse response = ref.get(); - this.token = response.token(); - this.tokenType = response.issuedTokenType(); - this.expiresAtMillis = OAuth2Util.expiresAtMillis(token); - this.headers = RESTUtil.merge(headers, authHeaders(token)); + this.config = + AuthConfig.builder() + .from(config()) + .token(response.token()) + .tokenType(response.issuedTokenType()) + .build(); + this.headers = RESTUtil.merge(headers, authHeaders(config.token())); if (response.expiresInSeconds() != null) { return Pair.of(response.expiresInSeconds(), TimeUnit.SECONDS); @@ -626,21 +607,34 @@ public Pair refresh(RESTClient client) { } private OAuthTokenResponse refreshCurrentToken(RESTClient client) { - if (null != expiresAtMillis && expiresAtMillis <= System.currentTimeMillis()) { + if (null != expiresAtMillis() && expiresAtMillis() <= System.currentTimeMillis()) { // the token has already expired, attempt to refresh using the credential return refreshExpiredToken(client); } else { // attempt a normal refresh return refreshToken( - client, headers(), token, tokenType, scope, oauth2ServerUri, optionalOAuthParams); + client, + headers(), + token(), + tokenType(), + scope(), + oauth2ServerUri(), + optionalOAuthParams()); } } private OAuthTokenResponse refreshExpiredToken(RESTClient client) { - if (credential != null) { - Map basicHeaders = RESTUtil.merge(headers(), basicAuthHeaders(credential)); + if (credential() != null) { + Map basicHeaders = + RESTUtil.merge(headers(), basicAuthHeaders(credential())); return refreshToken( - client, basicHeaders, token, tokenType, scope, oauth2ServerUri, optionalOAuthParams); + client, + basicHeaders, + token(), + tokenType(), + scope(), + oauth2ServerUri(), + optionalOAuthParams()); } return null; @@ -693,12 +687,11 @@ public static AuthSession fromAccessToken( AuthSession session = new AuthSession( parent.headers(), - token, - OAuth2Properties.ACCESS_TOKEN_TYPE, - parent.credential(), - parent.scope(), - parent.oauth2ServerUri(), - parent.optionalOAuthParams()); + AuthConfig.builder() + .from(parent.config()) + .token(token) + .tokenType(OAuth2Properties.ACCESS_TOKEN_TYPE) + .build()); long startTimeMillis = System.currentTimeMillis(); Long expiresAtMillis = session.expiresAtMillis(); @@ -766,12 +759,12 @@ private static AuthSession fromTokenResponse( AuthSession session = new AuthSession( parent.headers(), - response.token(), - response.issuedTokenType(), - credential, - parent.scope(), - parent.oauth2ServerUri(), - parent.optionalOAuthParams()); + AuthConfig.builder() + .from(parent.config()) + .token(response.token()) + .tokenType(response.issuedTokenType()) + .credential(credential) + .build()); Long expiresAtMillis = session.expiresAtMillis(); if (null == expiresAtMillis && response.expiresInSeconds() != null) { From c4da0c7b9a14c81f87655ed64fbe40d7de18a546 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 4 Jun 2024 08:22:28 +0200 Subject: [PATCH 0345/1019] Build: Bump software.amazon.awssdk:bom from 2.25.60 to 2.25.64 (#10421) Bumps software.amazon.awssdk:bom from 2.25.60 to 2.25.64. --- 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 4f876497811a..d9e600e7bc43 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.25.60" +awssdk-bom = "2.25.64" azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From a1de2ae5c46836775396985077552f26a62b9872 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 4 Jun 2024 08:23:04 +0200 Subject: [PATCH 0346/1019] Build: Bump com.google.errorprone:error_prone_annotations (#10418) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.27.0 to 2.28.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.27.0...v2.28.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 d9e600e7bc43..2784a1b6137f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -36,7 +36,7 @@ calcite = "1.10.0" delta-standalone = "3.1.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.27.0" +errorprone-annotations = "2.28.0" findbugs-jsr305 = "3.0.2" flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} From 0e027ced274098cee766b3d896bfe13bbd9b3f06 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 4 Jun 2024 12:32:07 +0200 Subject: [PATCH 0347/1019] Core: Use TestTemplate instead of Test annotation in TestPartitionSpecParser/Info (#10435) I've noticed that the `formatVersion` parameter wasn't properly initialized for this test class because the tests weren't annotated with `@TestTemplate` --- .../org/apache/iceberg/TestPartitionSpecInfo.java | 12 ++++++------ .../org/apache/iceberg/TestPartitionSpecParser.java | 10 +++++----- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index a82ac6636d4a..ff54929504ee 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -31,7 +31,7 @@ import org.apache.iceberg.types.Types; 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; @@ -62,7 +62,7 @@ public void cleanupTables() { TestTables.clearTables(); } - @Test + @TestTemplate public void testSpecIsUnpartitionedForVoidTranforms() { PartitionSpec spec = PartitionSpec.builderFor(schema).alwaysNull("id").alwaysNull("data").build(); @@ -70,7 +70,7 @@ public void testSpecIsUnpartitionedForVoidTranforms() { assertThat(spec.isUnpartitioned()).isTrue(); } - @Test + @TestTemplate public void testSpecInfoUnpartitionedTable() { PartitionSpec spec = PartitionSpec.unpartitioned(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); @@ -83,7 +83,7 @@ public void testSpecInfoUnpartitionedTable() { .doesNotContainKey(Integer.MAX_VALUE); } - @Test + @TestTemplate public void testSpecInfoPartitionedTable() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); @@ -95,7 +95,7 @@ public void testSpecInfoPartitionedTable() { .doesNotContainKey(Integer.MAX_VALUE); } - @Test + @TestTemplate public void testColumnDropWithPartitionSpecEvolution() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); @@ -119,7 +119,7 @@ public void testColumnDropWithPartitionSpecEvolution() { assertThat(table.schema().asStruct()).isEqualTo(expectedSchema.asStruct()); } - @Test + @TestTemplate public void testSpecInfoPartitionSpecEvolutionForV1Table() { PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("data", 4).build(); TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java index ad8861f53685..e97d2f98b416 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecParser.java @@ -22,7 +22,7 @@ import java.util.Arrays; import java.util.List; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @ExtendWith(ParameterizedTestExtension.class) @@ -32,7 +32,7 @@ protected static List parameters() { return Arrays.asList(1); } - @Test + @TestTemplate public void testToJsonForV1Table() { String expected = "{\n" @@ -69,7 +69,7 @@ public void testToJsonForV1Table() { assertThat(PartitionSpecParser.toJson(table.spec(), true)).isEqualTo(expected); } - @Test + @TestTemplate public void testFromJsonWithFieldId() { String specString = "{\n" @@ -95,7 +95,7 @@ public void testFromJsonWithFieldId() { assertThat(spec.fields().get(1).fieldId()).isEqualTo(1000); } - @Test + @TestTemplate public void testFromJsonWithoutFieldId() { String specString = "{\n" @@ -119,7 +119,7 @@ public void testFromJsonWithoutFieldId() { assertThat(spec.fields().get(1).fieldId()).isEqualTo(1001); } - @Test + @TestTemplate public void testTransforms() { for (PartitionSpec spec : PartitionSpecTestBase.SPECS) { assertThat(roundTripJSON(spec)).isEqualTo(spec); From ab85abdddf1d5f225849e15068bc17637fddc3be Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 4 Jun 2024 18:52:10 +0800 Subject: [PATCH 0348/1019] Docs: Point links in metrics-reporting.md to GitHub Java source (#10397) --- .../iceberg/rest/RESTMetricsReporter.java | 4 ++++ docs/docs/metrics-reporting.md | 18 +++++++++--------- 2 files changed, 13 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTMetricsReporter.java b/core/src/main/java/org/apache/iceberg/rest/RESTMetricsReporter.java index 4ceb86f32c4a..bb3f6e683be1 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTMetricsReporter.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTMetricsReporter.java @@ -26,6 +26,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * A {@link MetricsReporter} implementation that reports the {@link MetricsReport} to a REST + * endpoint. This is the default metrics reporter when using {@link RESTCatalog}. + */ class RESTMetricsReporter implements MetricsReporter { private static final Logger LOG = LoggerFactory.getLogger(RESTMetricsReporter.class); diff --git a/docs/docs/metrics-reporting.md b/docs/docs/metrics-reporting.md index 3a83e1baecde..48ee94f85942 100644 --- a/docs/docs/metrics-reporting.md +++ b/docs/docs/metrics-reporting.md @@ -20,12 +20,12 @@ title: "Metrics Reporting" # Metrics Reporting -As of 1.1.0 Iceberg supports the [`MetricsReporter`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/MetricsReporter.html) and the [`MetricsReport`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/MetricsReport.html) APIs. These two APIs allow expressing different metrics reports while supporting a pluggable way of reporting these reports. +As of 1.1.0 Iceberg supports the [`MetricsReporter`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/MetricsReporter.java) and the [`MetricsReport`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/MetricsReport.java) APIs. These two APIs allow expressing different metrics reports while supporting a pluggable way of reporting these reports. ## Type of Reports ### ScanReport -A [`ScanReport`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/ScanReport.html) carries metrics being collected during scan planning against a given table. Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like: +A [`ScanReport`](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/metrics/ScanReport.java) carries metrics being collected during scan planning against a given table. Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like: * total scan planning duration * number of data/delete files included in the result @@ -35,7 +35,7 @@ A [`ScanReport`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/S ### CommitReport -A [`CommitReport`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/CommitReport.html) carries metrics being collected after committing changes to a table (aka producing a snapshot). Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like: +A [`CommitReport`](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/metrics/CommitReport.java) carries metrics being collected after committing changes to a table (aka producing a snapshot). Amongst some general information about the involved table, such as the snapshot id or the table name, it includes metrics like: * total duration * number of attempts required for the commit to succeed @@ -46,7 +46,7 @@ A [`CommitReport`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics ## Available Metrics Reporters -### [`LoggingMetricsReporter`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/LoggingMetricsReporter.html) +### [`LoggingMetricsReporter`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/LoggingMetricsReporter.java) This is the default metrics reporter when nothing else is configured and its purpose is to log results to the log file. Example output would look as shown below: @@ -117,16 +117,16 @@ CommitReport{ ``` -### [`RESTMetricsReporter`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/rest/RESTMetricsReporter.html) +### [`RESTMetricsReporter`](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/rest/RESTMetricsReporter.java) -This is the default when using the [`RESTCatalog`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/rest/RESTCatalog.html) and its purpose is to send metrics to a REST server at the `/v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics` endpoint as defined in the [REST OpenAPI spec](https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml). +This is the default when using the [`RESTCatalog`](https://github.com/apache/iceberg/blob/main/core/src/main/java/org/apache/iceberg/rest/RESTCatalog.java) and its purpose is to send metrics to a REST server at the `/v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics` endpoint as defined in the [REST OpenAPI spec](https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml). Sending metrics via REST can be controlled with the `rest-metrics-reporting-enabled` (defaults to `true`) property. ## Implementing a custom Metrics Reporter -Implementing the [`MetricsReporter`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/MetricsReporter.html) API gives full flexibility in dealing with incoming [`MetricsReport`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/MetricsReport.html) instances. For example, it would be possible to send results to a Prometheus endpoint or any other observability framework/system. +Implementing the [`MetricsReporter`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/MetricsReporter.java) API gives full flexibility in dealing with incoming [`MetricsReport`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/MetricsReport.java) instances. For example, it would be possible to send results to a Prometheus endpoint or any other observability framework/system. Below is a short example illustrating an `InMemoryMetricsReporter` that stores reports in a list and makes them available: ```java @@ -149,11 +149,11 @@ public class InMemoryMetricsReporter implements MetricsReporter { ### Via Catalog Configuration -The [catalog property](configuration.md#catalog-properties) `metrics-reporter-impl` allows registering a given [`MetricsReporter`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/MetricsReporter.html) by specifying its fully-qualified class name, e.g. `metrics-reporter-impl=org.apache.iceberg.metrics.InMemoryMetricsReporter`. +The [catalog property](configuration.md#catalog-properties) `metrics-reporter-impl` allows registering a given [`MetricsReporter`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/MetricsReporter.java) by specifying its fully-qualified class name, e.g. `metrics-reporter-impl=org.apache.iceberg.metrics.InMemoryMetricsReporter`. ### Via the Java API during Scan planning -Independently of the [`MetricsReporter`](../../javadoc/{{ icebergVersion }}/org/apache/iceberg/metrics/MetricsReporter.html) being registered at the catalog level via the `metrics-reporter-impl` property, it is also possible to supply additional reporters during scan planning as shown below: +Independently of the [`MetricsReporter`](https://github.com/apache/iceberg/blob/main/api/src/main/java/org/apache/iceberg/metrics/MetricsReporter.java) being registered at the catalog level via the `metrics-reporter-impl` property, it is also possible to supply additional reporters during scan planning as shown below: ```java TableScan tableScan = From 2fe835c5c4695f78db772016e7e3760b05fb0a24 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 5 Jun 2024 17:22:22 +0200 Subject: [PATCH 0349/1019] Build: Clean up Jackson dependency usages (#10448) --- build.gradle | 18 +++++++++--------- gradle/libs.versions.toml | 5 +++-- kafka-connect/build.gradle | 4 ++-- mr/build.gradle | 2 +- 4 files changed, 15 insertions(+), 14 deletions(-) diff --git a/build.gradle b/build.gradle index 2b95fe291790..314393dfd902 100644 --- a/build.gradle +++ b/build.gradle @@ -352,8 +352,8 @@ project(':iceberg-core') { implementation libs.aircompressor implementation libs.httpcomponents.httpclient5 implementation platform(libs.jackson.bom) - implementation "com.fasterxml.jackson.core:jackson-core" - implementation "com.fasterxml.jackson.core:jackson-databind" + implementation libs.jackson.core + implementation libs.jackson.databind implementation libs.caffeine implementation libs.roaringbitmap compileOnly(libs.hadoop2.client) { @@ -469,8 +469,8 @@ project(':iceberg-aws') { compileOnly libs.immutables.value implementation libs.caffeine implementation platform(libs.jackson.bom) - implementation "com.fasterxml.jackson.core:jackson-databind" - implementation "com.fasterxml.jackson.core:jackson-core" + implementation libs.jackson.core + implementation libs.jackson.databind compileOnly(platform(libs.awssdk.bom)) compileOnly(libs.awssdk.s3accessgrants) @@ -582,7 +582,7 @@ project(':iceberg-delta-lake') { implementation project(':iceberg-core') implementation project(':iceberg-parquet') implementation platform(libs.jackson.bom) - implementation "com.fasterxml.jackson.core:jackson-databind" + implementation libs.jackson.databind annotationProcessor libs.immutables.value compileOnly libs.immutables.value @@ -912,8 +912,8 @@ project(':iceberg-nessie') { exclude group: 'com.fasterxml.jackson' } implementation platform(libs.jackson.bom) - implementation "com.fasterxml.jackson.core:jackson-databind" - implementation "com.fasterxml.jackson.core:jackson-core" + implementation libs.jackson.core + implementation libs.jackson.databind compileOnly libs.hadoop2.common // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages @@ -964,8 +964,8 @@ project(':iceberg-snowflake') { implementation project(':iceberg-common') implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') implementation platform(libs.jackson.bom) - implementation "com.fasterxml.jackson.core:jackson-databind" - implementation "com.fasterxml.jackson.core:jackson-core" + implementation libs.jackson.core + implementation libs.jackson.databind runtimeOnly libs.snowflake.jdbc diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 2784a1b6137f..1ad8ee69147d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -142,8 +142,9 @@ hive3-service = { module = "org.apache.hive:hive-service", version.ref = "hive3" httpcomponents-httpclient5 = { module = "org.apache.httpcomponents.client5:httpclient5", version.ref = "httpcomponents-httpclient5" } immutables-value = { module = "org.immutables:value", version.ref = "immutables-value" } jackson-bom = { module = "com.fasterxml.jackson:jackson-bom", version.ref = "jackson-bom" } -jackson-core = { module = "com.fasterxml.jackson:jackson-core", version.ref = "jackson-bom" } -jackson-databind = { module = "com.fasterxml.jackson:jackson-databind", version.ref = "jackson-bom" } +jackson-core = { module = "com.fasterxml.jackson.core:jackson-core", version.ref = "jackson-bom" } +jackson-databind = { module = "com.fasterxml.jackson.core:jackson-databind", version.ref = "jackson-bom" } +jackson-annotations = { module = "com.fasterxml.jackson.core:jackson-annotations", version.ref = "jackson-bom" } jackson211-bom = { module = "com.fasterxml.jackson:jackson-bom", version.ref = "jackson211" } jackson212-bom = { module = "com.fasterxml.jackson:jackson-bom", version.ref = "jackson212" } jackson213-bom = { module = "com.fasterxml.jackson:jackson-bom", version.ref = "jackson213" } diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 08f044c233e9..1fdd6bc6ea4c 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -40,8 +40,8 @@ project(":iceberg-kafka-connect:iceberg-kafka-connect") { implementation project(':iceberg-data') implementation project(':iceberg-kafka-connect:iceberg-kafka-connect-events') implementation platform(libs.jackson.bom) - implementation "com.fasterxml.jackson.core:jackson-core" - implementation "com.fasterxml.jackson.core:jackson-databind" + implementation libs.jackson.core + implementation libs.jackson.databind implementation libs.avro.avro compileOnly libs.kafka.clients diff --git a/mr/build.gradle b/mr/build.gradle index d2bf9290d8ae..bf8f9ee943f7 100644 --- a/mr/build.gradle +++ b/mr/build.gradle @@ -69,7 +69,7 @@ project(':iceberg-mr') { testImplementation libs.calcite.core testImplementation libs.kryo.shaded testImplementation platform(libs.jackson.bom) - testImplementation "com.fasterxml.jackson.core:jackson-annotations" + testImplementation libs.jackson.annotations testImplementation(libs.hive2.service) { exclude group: 'org.apache.hive', module: 'hive-exec' } From 2a5c91f240b55b9b9dc15483ba21101097a544b4 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 5 Jun 2024 10:00:45 -0700 Subject: [PATCH 0350/1019] Flink: refactor sink shuffling statistics collection (#10331) refactor sink shuffling statistics collection to support sketch statistics and auto migration from Map stats to reservoir sampling sketch if cardinality is detected high --- flink/v1.19/build.gradle | 2 + .../sink/shuffle/AggregatedStatistics.java | 93 ++- .../AggregatedStatisticsSerializer.java | 175 ++++++ .../shuffle/AggregatedStatisticsTracker.java | 271 ++++++--- .../flink/sink/shuffle/DataStatistics.java | 25 +- .../shuffle/DataStatisticsCoordinator.java | 110 ++-- .../DataStatisticsCoordinatorProvider.java | 24 +- .../sink/shuffle/DataStatisticsOperator.java | 150 +++-- .../shuffle/DataStatisticsSerializer.java | 204 +++++++ .../sink/shuffle/DataStatisticsUtil.java | 96 ---- .../flink/sink/shuffle/MapDataStatistics.java | 52 +- .../shuffle/MapDataStatisticsSerializer.java | 187 ------ .../sink/shuffle/MapRangePartitioner.java | 22 +- .../sink/shuffle/SketchDataStatistics.java | 87 +++ .../flink/sink/shuffle/SketchUtil.java | 148 +++++ .../flink/sink/shuffle/SortKeySerializer.java | 17 +- .../sink/shuffle/SortKeySketchSerializer.java | 143 +++++ ...tisticsEvent.java => StatisticsEvent.java} | 22 +- ...sOrRecord.java => StatisticsOrRecord.java} | 43 +- ...java => StatisticsOrRecordSerializer.java} | 102 ++-- .../flink/sink/shuffle/StatisticsType.java | 55 ++ .../flink/sink/shuffle/StatisticsUtil.java | 97 ++++ .../iceberg/flink/sink/shuffle/Fixtures.java | 98 ++++ .../shuffle/TestAggregatedStatistics.java | 63 --- .../TestAggregatedStatisticsSerializer.java | 54 ++ .../TestAggregatedStatisticsTracker.java | 535 +++++++++++++----- .../TestDataStatisticsCoordinator.java | 195 +++---- ...TestDataStatisticsCoordinatorProvider.java | 182 +++--- .../shuffle/TestDataStatisticsOperator.java | 332 ++++++----- .../shuffle/TestDataStatisticsSerializer.java | 53 ++ .../sink/shuffle/TestMapDataStatistics.java | 63 +-- .../sink/shuffle/TestMapRangePartitioner.java | 53 +- .../shuffle/TestSketchDataStatistics.java | 60 ++ .../flink/sink/shuffle/TestSketchUtil.java | 133 +++++ .../TestSortKeySerializerPrimitives.java | 33 ++ gradle/libs.versions.toml | 2 + 36 files changed, 2770 insertions(+), 1211 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java delete mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{DataStatisticsEvent.java => StatisticsEvent.java} (68%) rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{DataStatisticsOrRecord.java => StatisticsOrRecord.java} (66%) rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{DataStatisticsOrRecordSerializer.java => StatisticsOrRecordSerializer.java} (54%) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index f70c7a4d32fe..a200a33890cc 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -66,6 +66,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } + implementation libs.datasketches + testImplementation libs.flink119.connector.test.utils testImplementation libs.flink119.core testImplementation libs.flink119.runtime diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java index 157f04b8b0ed..8a91411c1d21 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java @@ -19,8 +19,11 @@ package org.apache.iceberg.flink.sink.shuffle; import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** @@ -28,44 +31,80 @@ * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores * the merged {@link DataStatistics} result from all reported subtasks. */ -class AggregatedStatistics, S> implements Serializable { - +class AggregatedStatistics implements Serializable { private final long checkpointId; - private final DataStatistics dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } + private final StatisticsType type; + private final Map keyFrequency; + private final SortKey[] rangeBounds; - AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; + AggregatedStatistics( + long checkpointId, + StatisticsType type, + Map keyFrequency, + SortKey[] rangeBounds) { + Preconditions.checkArgument( + (keyFrequency != null && rangeBounds == null) + || (keyFrequency == null && rangeBounds != null), + "Invalid key frequency or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.keyFrequency = keyFrequency; + this.rangeBounds = rangeBounds; } - DataStatistics dataStatistics() { - return dataStatistics; + static AggregatedStatistics fromKeyFrequency(long checkpointId, Map stats) { + return new AggregatedStatistics(checkpointId, StatisticsType.Map, stats, null); } - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); + static AggregatedStatistics fromRangeBounds(long checkpointId, SortKey[] stats) { + return new AggregatedStatistics(checkpointId, StatisticsType.Sketch, null, stats); } @Override public String toString() { return MoreObjects.toStringHelper(this) .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) + .add("type", type) + .add("keyFrequency", keyFrequency) + .add("rangeBounds", rangeBounds) .toString(); } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof AggregatedStatistics)) { + return false; + } + + AggregatedStatistics other = (AggregatedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId()) + && Objects.equal(type, other.type()) + && Objects.equal(keyFrequency, other.keyFrequency()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + return Objects.hashCode(checkpointId, type, keyFrequency, rangeBounds); + } + + StatisticsType type() { + return type; + } + + Map keyFrequency() { + return keyFrequency; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } + + long checkpointId() { + return checkpointId; + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java new file mode 100644 index 000000000000..56ba5e04f05a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java @@ -0,0 +1,175 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; + +public class AggregatedStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer keyFrequencySerializer; + private final ListSerializer rangeBoundsSerializer; + + AggregatedStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new AggregatedStatisticsSerializer(sortKeySerializer); + } + + @Override + public AggregatedStatistics createInstance() { + return new AggregatedStatistics(0, StatisticsType.Map, Collections.emptyMap(), null); + } + + @Override + public AggregatedStatistics copy(AggregatedStatistics from) { + return new AggregatedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.rangeBounds()); + } + + @Override + public AggregatedStatistics copy(AggregatedStatistics from, AggregatedStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(AggregatedStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + keyFrequencySerializer.serialize(record.keyFrequency(), target); + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public AggregatedStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + Map keyFrequency = null; + SortKey[] rangeBounds = null; + if (type == StatisticsType.Map) { + keyFrequency = keyFrequencySerializer.deserialize(source); + } else { + List sortKeys = rangeBoundsSerializer.deserialize(source); + rangeBounds = new SortKey[sortKeys.size()]; + rangeBounds = sortKeys.toArray(rangeBounds); + } + + return new AggregatedStatistics(checkpointId, type, keyFrequency, rangeBounds); + } + + @Override + public AggregatedStatistics deserialize(AggregatedStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof AggregatedStatisticsSerializer)) { + return false; + } + + AggregatedStatisticsSerializer other = (AggregatedStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new AggregatedStatisticsSerializerSnapshot(this); + } + + public static class AggregatedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot< + AggregatedStatistics, AggregatedStatisticsSerializer> { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public AggregatedStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public AggregatedStatisticsSerializerSnapshot(AggregatedStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + AggregatedStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected AggregatedStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new AggregatedStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index e8ff61dbeb27..52d8a2f16f99 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -18,9 +18,21 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; +import java.util.Map; +import java.util.NavigableMap; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.datasketches.sampling.ReservoirItemsUnion; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,104 +42,225 @@ * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific * checkpoint. */ -class AggregatedStatisticsTracker, S> { +class AggregatedStatisticsTracker { private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; + private final String operatorName; - private final TypeSerializer> statisticsSerializer; private final int parallelism; - private final Set inProgressSubtaskSet; - private volatile AggregatedStatistics inProgressStatistics; + private final TypeSerializer statisticsSerializer; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final int switchToSketchThreshold; + private final Comparator comparator; + private final NavigableMap aggregationsPerCheckpoint; + + private AggregatedStatistics completedStatistics; AggregatedStatisticsTracker( String operatorName, - TypeSerializer> statisticsSerializer, - int parallelism) { + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable AggregatedStatistics restoredStatistics) { this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); + this.statisticsSerializer = + new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.switchToSketchThreshold = switchToSketchThreshold; + this.completedStatistics = restoredStatistics; + + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.aggregationsPerCheckpoint = Maps.newTreeMap(); } - AggregatedStatistics updateAndCheckCompletion( - int subtask, DataStatisticsEvent event) { + AggregatedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { long checkpointId = event.checkpointId(); + LOG.debug( + "Handling statistics event from subtask {} of operator {} for checkpoint {}", + subtask, + operatorName, + checkpointId); - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { + if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " + + "Was expecting data statistics from checkpoint higher than {}", operatorName, - inProgressStatistics.checkpointId(), - checkpointId); + subtask, + checkpointId, + completedStatistics.checkpointId()); return null; } - AggregatedStatistics completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, + Aggregation aggregation = + aggregationsPerCheckpoint.computeIfAbsent( checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); + ignored -> + new Aggregation( + parallelism, + downstreamParallelism, + switchToSketchThreshold, + comparator, + statisticsType, + StatisticsUtil.collectType(statisticsType, completedStatistics))); + DataStatistics dataStatistics = + StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); + if (!aggregation.merge(subtask, dataStatistics)) { + LOG.debug( + "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } + + if (aggregation.isComplete()) { + this.completedStatistics = aggregation.completedStatistics(checkpointId); + // clean up aggregations up to the completed checkpoint id + aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); + return completedStatistics; + } + + return null; + } + + @VisibleForTesting + NavigableMap aggregationsPerCheckpoint() { + return aggregationsPerCheckpoint; + } + + static class Aggregation { + private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); + + private final Set subtaskSet; + private final int parallelism; + private final int downstreamParallelism; + private final int switchToSketchThreshold; + private final Comparator comparator; + private final StatisticsType configuredType; + private StatisticsType currentType; + private Map mapStatistics; + private ReservoirItemsUnion sketchStatistics; + + Aggregation( + int parallelism, + int downstreamParallelism, + int switchToSketchThreshold, + Comparator comparator, + StatisticsType configuredType, + StatisticsType currentType) { + this.subtaskSet = Sets.newHashSet(); + this.parallelism = parallelism; + this.downstreamParallelism = downstreamParallelism; + this.switchToSketchThreshold = switchToSketchThreshold; + this.comparator = comparator; + this.configuredType = configuredType; + this.currentType = currentType; + + if (currentType == StatisticsType.Map) { + this.mapStatistics = Maps.newHashMap(); + this.sketchStatistics = null; } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); + this.mapStatistics = null; + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); } + } - inProgressStatistics = null; - inProgressSubtaskSet.clear(); + @VisibleForTesting + Set subtaskSet() { + return subtaskSet; } - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + StatisticsType currentType() { + return currentType; } - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; } - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + ReservoirItemsUnion sketchStatistics() { + return sketchStatistics; } - return completedStatistics; - } + private boolean isComplete() { + return subtaskSet.size() == parallelism; + } - @VisibleForTesting - AggregatedStatistics inProgressStatistics() { - return inProgressStatistics; + /** @return false if duplicate */ + private boolean merge(int subtask, DataStatistics taskStatistics) { + if (subtaskSet.contains(subtask)) { + return false; + } + + subtaskSet.add(subtask); + merge(taskStatistics); + return true; + } + + @SuppressWarnings("unchecked") + private void merge(DataStatistics taskStatistics) { + if (taskStatistics.type() == StatisticsType.Map) { + Map taskMapStats = (Map) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); + if (configuredType == StatisticsType.Auto + && mapStatistics.size() > switchToSketchThreshold) { + convertCoordinatorToSketch(); + } + } else { + // convert task stats to sketch first + ReservoirItemsSketch taskSketch = + ReservoirItemsSketch.newInstance( + SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); + SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); + sketchStatistics.update(taskSketch); + } + } else { + ReservoirItemsSketch taskSketch = + (ReservoirItemsSketch) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + // convert global stats to sketch first + convertCoordinatorToSketch(); + } + + sketchStatistics.update(taskSketch); + } + } + + private void convertCoordinatorToSketch() { + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); + this.currentType = StatisticsType.Sketch; + this.mapStatistics = null; + } + + private AggregatedStatistics completedStatistics(long checkpointId) { + if (currentType == StatisticsType.Map) { + LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); + return AggregatedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + } else { + ReservoirItemsSketch sketch = sketchStatistics.getResult(); + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return AggregatedStatistics.fromRangeBounds( + checkpointId, SketchUtil.rangeBounds(downstreamParallelism, comparator, sketch)); + } + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 9d7cf179ab1c..76c59cd5f4b8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; @@ -29,29 +31,18 @@ * (sketching) can be used. */ @Internal -interface DataStatistics, S> { +interface DataStatistics { + + StatisticsType type(); - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ boolean isEmpty(); /** Add row sortKey to data statistics. */ void add(SortKey sortKey); /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics + * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link + * ReservoirItemsSketch} (high cardinality) */ - S statistics(); + Object result(); } 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 c8ac79c61bf6..6e9e4f6fa6c2 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 @@ -35,6 +35,8 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.ThrowableCatchingRunnable; import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -44,51 +46,76 @@ import org.slf4j.LoggerFactory; /** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. + * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} + * every subtask and then merge them together. Once aggregation for all subtasks data statistics + * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link + * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the + * aggregated data statistics to improve data clustering. */ @Internal -class DataStatisticsCoordinator, S> implements OperatorCoordinator { +class DataStatisticsCoordinator implements OperatorCoordinator { private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); private final String operatorName; + private final OperatorCoordinator.Context context; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer> statisticsSerializer; - private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private volatile AggregatedStatistics completedStatistics; - private volatile boolean started; + private final TypeSerializer aggregatedStatisticsSerializer; + + private transient boolean started; + private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private transient AggregatedStatistics completedStatistics; DataStatisticsCoordinator( String operatorName, OperatorCoordinator.Context context, - TypeSerializer> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType) { this.operatorName = operatorName; + this.context = context; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.aggregatedStatisticsSerializer = new AggregatedStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; + this.started = true; + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker( + operatorName, + context.currentParallelism(), + schema, + sortOrder, + downstreamParallelism, + statisticsType, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + completedStatistics); } @Override public void close() throws Exception { coordinatorExecutor.shutdown(); + this.aggregatedStatisticsTracker = null; + this.started = false; LOG.info("Closed data statistics coordinator: {}.", operatorName); } @@ -148,7 +175,7 @@ private void runInCoordinatorThread(ThrowingRunnable action, String a operatorName, actionString, t); - operatorCoordinatorContext.failJob(t); + context.failJob(t); } }); } @@ -157,31 +184,26 @@ private void ensureStarted() { Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); } - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = + private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { + AggregatedStatistics aggregatedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (aggregatedStatistics != null) { completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + sendAggregatedStatisticsToSubtasks(completedStatistics.checkpointId(), completedStatistics); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics globalDataStatistics) { + private void sendAggregatedStatisticsToSubtasks( + long checkpointId, AggregatedStatistics globalStatistics) { callInCoordinatorThread( () -> { - DataStatisticsEvent dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + StatisticsEvent statisticsEvent = + StatisticsEvent.createAggregatedStatisticsEvent( + checkpointId, globalStatistics, aggregatedStatisticsSerializer); + for (int i = 0; i < context.currentParallelism(); ++i) { + subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } return null; @@ -192,7 +214,6 @@ private void sendDataStatisticsToSubtasks( } @Override - @SuppressWarnings("unchecked") public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { runInCoordinatorThread( () -> { @@ -202,8 +223,8 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); + Preconditions.checkArgument(event instanceof StatisticsEvent); + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -219,8 +240,8 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r operatorName, checkpointId); resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); + StatisticsUtil.serializeAggregatedStatistics( + completedStatistics, aggregatedStatisticsSerializer)); }, String.format("taking checkpoint %d", checkpointId)); } @@ -229,8 +250,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r public void notifyCheckpointComplete(long checkpointId) {} @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); @@ -244,8 +264,9 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + this.completedStatistics = + StatisticsUtil.deserializeAggregatedStatistics( + checkpointData, aggregatedStatisticsSerializer); } @Override @@ -295,7 +316,7 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics completedStatistics() { + AggregatedStatistics completedStatistics() { return completedStatistics; } @@ -303,6 +324,7 @@ private static class SubtaskGateways { private final String operatorName; private final Map[] gateways; + @SuppressWarnings("unchecked") private SubtaskGateways(String operatorName, int parallelism) { this.operatorName = operatorName; gateways = new Map[parallelism]; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java index 47dbfc3cfbe1..ffb428283785 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -19,33 +19,43 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; /** * DataStatisticsCoordinatorProvider provides the method to create new {@link * DataStatisticsCoordinator} */ @Internal -public class DataStatisticsCoordinatorProvider, S> - extends RecreateOnResetOperatorCoordinator.Provider { +public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { private final String operatorName; - private final TypeSerializer> statisticsSerializer; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; public DataStatisticsCoordinatorProvider( String operatorName, OperatorID operatorID, - TypeSerializer> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type) { super(operatorID); this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + return new DataStatisticsCoordinator( + operatorName, context, schema, sortOrder, downstreamParallelism, type); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 5157a37cf2cd..2910471762cd 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -47,9 +48,8 @@ * distribution to downstream subtasks. */ @Internal -class DataStatisticsOperator, S> - extends AbstractStreamOperator> - implements OneInputStreamOperator>, OperatorEventHandler { +public class DataStatisticsOperator extends AbstractStreamOperator + implements OneInputStreamOperator, OperatorEventHandler { private static final long serialVersionUID = 1L; @@ -57,141 +57,181 @@ class DataStatisticsOperator, S> private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer> statisticsSerializer; - private transient volatile DataStatistics localStatistics; - private transient volatile DataStatistics globalStatistics; - private transient ListState> globalStatisticsState; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final TypeSerializer taskStatisticsSerializer; + private final TypeSerializer aggregatedStatisticsSerializer; + + private transient int parallelism; + private transient int subtaskIndex; + private transient ListState globalStatisticsState; + // current statistics type may be different from the config due to possible + // migration from Map statistics to Sketch statistics when high cardinality detected + private transient volatile StatisticsType taskStatisticsType; + private transient volatile DataStatistics localStatistics; + private transient volatile AggregatedStatistics globalStatistics; DataStatisticsOperator( String operatorName, Schema schema, SortOrder sortOrder, OperatorEventGateway operatorEventGateway, - TypeSerializer> statisticsSerializer) { + int downstreamParallelism, + StatisticsType statisticsType) { this.operatorName = operatorName; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); + this.aggregatedStatisticsSerializer = new AggregatedStatisticsSerializer(sortKeySerializer); } @Override public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = + this.parallelism = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); + this.subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + this.globalStatisticsState = context .getOperatorStateStore() .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); + new ListStateDescriptor<>("globalStatisticsState", aggregatedStatisticsSerializer)); if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); if (globalStatisticsState.get() == null || !globalStatisticsState.get().iterator().hasNext()) { LOG.warn( "Operator {} subtask {} doesn't have global statistics state to restore", operatorName, subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); } else { LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); + "Operator {} subtask {} restoring global statistics state", operatorName, subtaskIndex); + this.globalStatistics = globalStatisticsState.get().iterator().next(); } - } else { - globalStatistics = statisticsSerializer.createInstance(); } + + this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); + this.localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); } @Override public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } } @Override - @SuppressWarnings("unchecked") public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); Preconditions.checkArgument( - event instanceof DataStatisticsEvent, + event instanceof StatisticsEvent, String.format( "Operator %s subtask %s received unexpected operator event %s", operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; + StatisticsEvent statisticsEvent = (StatisticsEvent) event; LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", + "Operator {} subtask {} received global data event from coordinator checkpoint {}", operatorName, + subtaskIndex, statisticsEvent.checkpointId()); globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + StatisticsUtil.deserializeAggregatedStatistics( + statisticsEvent.statisticsBytes(), aggregatedStatisticsSerializer); + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } + @SuppressWarnings("unchecked") @Override public void processElement(StreamRecord streamRecord) { + // collect data statistics RowData record = streamRecord.getValue(); StructLike struct = rowDataWrapper.wrap(record); sortKey.wrap(struct); localStatistics.add(sortKey); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); + + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", + "Operator {} subtask {} snapshotting data statistics for checkpoint {}", operatorName, - checkpointId, - subTaskId); + subtaskIndex, + checkpointId); - // Pass global statistics to partitioners so that all the operators refresh statistics + // Pass global statistics to partitioner so that all the operators refresh statistics // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { + if (globalStatistics != null + && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { globalStatisticsState.clear(); LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); + "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); globalStatisticsState.add(globalStatistics); + LOG.debug( + "Operator {} subtask {} saved global statistics to state: {}", + operatorName, + subtaskIndex, + globalStatistics); } // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, + LOG.info( + "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", operatorName, - checkpointId, - localStatistics); + subtaskIndex, + checkpointId); + operatorEventGateway.sendEventToCoordinator( + StatisticsEvent.createTaskStatisticsEvent( + checkpointId, localStatistics, taskStatisticsSerializer)); // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); + localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + private void checkStatisticsTypeMigration() { + // only check if the statisticsType config is Auto and localStatistics is currently Map type + if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { + Map mapStatistics = (Map) localStatistics.result(); + // convert if local statistics has cardinality over the threshold or + // if received global statistics is already sketch type + if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { + LOG.info( + "Operator {} subtask {} switched local statistics from Map to Sketch.", + operatorName, + subtaskIndex); + this.taskStatisticsType = StatisticsType.Sketch; + this.localStatistics = + StatisticsUtil.createTaskStatistics( + taskStatisticsType, parallelism, downstreamParallelism); + SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); + } + } } @VisibleForTesting - DataStatistics localDataStatistics() { + DataStatistics localStatistics() { return localStatistics; } @VisibleForTesting - DataStatistics globalDataStatistics() { + AggregatedStatistics globalStatistics() { return globalStatistics; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java new file mode 100644 index 000000000000..8ce99073836d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java @@ -0,0 +1,204 @@ +/* + * 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.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class DataStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer mapSerializer; + private final SortKeySketchSerializer sketchSerializer; + + DataStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer duplicate() { + TypeSerializer duplicateSortKeySerializer = sortKeySerializer.duplicate(); + return (duplicateSortKeySerializer == sortKeySerializer) + ? this + : new DataStatisticsSerializer(duplicateSortKeySerializer); + } + + @Override + public DataStatistics createInstance() { + return new MapDataStatistics(); + } + + @SuppressWarnings("unchecked") + @Override + public DataStatistics copy(DataStatistics obj) { + StatisticsType statisticsType = obj.type(); + if (statisticsType == StatisticsType.Map) { + MapDataStatistics from = (MapDataStatistics) obj; + Map fromStats = (Map) from.result(); + Map toStats = Maps.newHashMap(fromStats); + return new MapDataStatistics(toStats); + } else if (statisticsType == StatisticsType.Sketch) { + // because ReservoirItemsSketch doesn't expose enough public methods for cloning, + // this implementation adopted the less efficient serialization and deserialization. + SketchDataStatistics from = (SketchDataStatistics) obj; + ReservoirItemsSketch fromStats = (ReservoirItemsSketch) from.result(); + byte[] bytes = fromStats.toByteArray(sketchSerializer); + Memory memory = Memory.wrap(bytes); + ReservoirItemsSketch toStats = + ReservoirItemsSketch.heapify(memory, sketchSerializer); + return new SketchDataStatistics(toStats); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics copy(DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @SuppressWarnings("unchecked") + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + StatisticsType statisticsType = obj.type(); + statisticsTypeSerializer.serialize(obj.type(), target); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = (Map) obj.result(); + mapSerializer.serialize(mapStatistics, target); + } else if (statisticsType == StatisticsType.Sketch) { + ReservoirItemsSketch sketch = (ReservoirItemsSketch) obj.result(); + byte[] sketchBytes = sketch.toByteArray(sketchSerializer); + target.writeInt(sketchBytes.length); + target.write(sketchBytes); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataInputView source) throws IOException { + StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = mapSerializer.deserialize(source); + return new MapDataStatistics(mapStatistics); + } else if (statisticsType == StatisticsType.Sketch) { + int numBytes = source.readInt(); + byte[] sketchBytes = new byte[numBytes]; + source.read(sketchBytes); + Memory sketchMemory = Memory.wrap(sketchBytes); + ReservoirItemsSketch sketch = + ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); + return new SketchDataStatistics(sketch); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsSerializer)) { + return false; + } + + DataStatisticsSerializer other = (DataStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DataStatisticsSerializerSnapshot(this); + } + + public static class DataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DataStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new DataStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 8716cb872d0e..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,96 +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.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static , S> byte[] serializeDataStatistics( - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static , S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static , S> byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - static , S> - AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 0ffffd9cf49f..22dd2388cc3e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -19,52 +19,70 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics { + private final Map keyFrequency; MapDataStatistics() { - this.statistics = Maps.newHashMap(); + this.keyFrequency = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { - this.statistics = statistics; + MapDataStatistics(Map keyFrequency) { + this.keyFrequency = keyFrequency; + } + + @Override + public StatisticsType type() { + return StatisticsType.Map; } @Override public boolean isEmpty() { - return statistics.size() == 0; + return keyFrequency.size() == 0; } @Override public void add(SortKey sortKey) { - if (statistics.containsKey(sortKey)) { - statistics.merge(sortKey, 1L, Long::sum); + if (keyFrequency.containsKey(sortKey)) { + keyFrequency.merge(sortKey, 1L, Long::sum); } else { // clone the sort key before adding to map because input sortKey object can be reused SortKey copiedKey = sortKey.copy(); - statistics.put(copiedKey, 1L); + keyFrequency.put(copiedKey, 1L); } } @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + public Object result() { + return keyFrequency; } @Override - public Map statistics() { - return statistics; + public String toString() { + return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); } @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof MapDataStatistics)) { + return false; + } + + MapDataStatistics other = (MapDataStatistics) o; + return Objects.equal(keyFrequency, other.keyFrequency); + } + + @Override + public int hashCode() { + return Objects.hashCode(keyFrequency); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index b6cccd0566fc..000000000000 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,187 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; - - static MapDataStatisticsSerializer fromSortKeySerializer( - TypeSerializer sortKeySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public MapDataStatistics createInstance() { - return new MapDataStatistics(); - } - - @Override - public MapDataStatistics copy(DataStatistics> obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - SortKey newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics> copy( - DataStatistics> from, - DataStatistics> reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize( - DataStatistics> obj, DataOutputView target) - throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics> deserialize( - DataStatistics> reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index fb1a8f03a65c..4f52915a925e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -77,22 +77,17 @@ class MapRangePartitioner implements Partitioner { MapRangePartitioner( Schema schema, SortOrder sortOrder, - MapDataStatistics dataStatistics, + Map mapStatistics, double closeFileCostInWeightPercentage) { - dataStatistics - .statistics() - .entrySet() - .forEach( - entry -> - Preconditions.checkArgument( - entry.getValue() > 0, - "Invalid statistics: weight is 0 for key %s", - entry.getKey())); + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = dataStatistics.statistics(); + this.mapStatistics = mapStatistics; this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; this.newSortKeyCounter = 0; this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); @@ -117,8 +112,11 @@ public int partition(RowData row, int numPartitions) { newSortKeyCounter += 1; long now = System.currentTimeMillis(); if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + LOG.info( + "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", + newSortKeyCounter); lastNewSortKeyLogTimeMilli = now; + newSortKeyCounter = 0; } return (int) (newSortKeyCounter % numPartitions); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java new file mode 100644 index 000000000000..35bbb27baf16 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java @@ -0,0 +1,87 @@ +/* + * 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.Arrays; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** MapDataStatistics uses map to count key frequency */ +class SketchDataStatistics implements DataStatistics { + + private final ReservoirItemsSketch sketch; + + SketchDataStatistics(int reservoirSize) { + this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); + } + + SketchDataStatistics(ReservoirItemsSketch sketchStats) { + this.sketch = sketchStats; + } + + @Override + public StatisticsType type() { + return StatisticsType.Sketch; + } + + @Override + public boolean isEmpty() { + return sketch.getNumSamples() == 0; + } + + @Override + public void add(SortKey sortKey) { + // clone the sort key first because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + sketch.update(copiedKey); + } + + @Override + public Object result() { + return sketch; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SketchDataStatistics)) { + return false; + } + + ReservoirItemsSketch otherSketch = ((SketchDataStatistics) o).sketch; + return Objects.equal(sketch.getK(), otherSketch.getK()) + && Objects.equal(sketch.getN(), otherSketch.getN()) + && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java new file mode 100644 index 000000000000..bfd3082a5aa7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -0,0 +1,148 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; + +class SketchUtil { + static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; + static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; + static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; + static final int OPERATOR_OVER_SAMPLE_RATIO = 10; + + // switch the statistics tracking from map to sketch if the cardinality of the sort key is over + // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. + static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; + static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; + + private SketchUtil() {} + + /** + * The larger the reservoir size, the more accurate for range bounds calculation and the more + * balanced range distribution. + * + *

    Here are the heuristic rules + *

  • Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the + * range bounds + *
  • Min is 10K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 1M to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @return reservoir size + */ + static int determineCoordinatorReservoirSize(int numPartitions) { + int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; + + if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { + // adjust it up and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); + } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { + // adjust it down and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; + } + + return reservoirSize; + } + + /** + * Determine the sampling reservoir size where operator subtasks collect data statistics. + * + *

    Here are the heuristic rules + *

  • Target size is "coordinator reservoir size * over sampling ration (10) / operator + * parallelism" + *
  • Min is 1K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 100K to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @param operatorParallelism data statistics operator parallelism + * @return reservoir size + */ + static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { + int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); + int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; + return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); + } + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + *
  • age <= 15 + *
  • age > 15 && age <= 32 + *
  • age >32 && age <= 60 + *
  • age > 60 + * + * @param numPartitions number of partitions which maps to downstream operator parallelism + * @param sketch aggregated reservoir sampling sketch + * @return list of range partition bounds. It should be a sorted list (ascending). Number of items + * should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + */ + static SortKey[] rangeBounds( + int numPartitions, Comparator comparator, ReservoirItemsSketch sketch) { + SortKey[] sortKeys = sketch.getSamples(); + return determineBounds(Math.min(numPartitions, sortKeys.length), comparator, sortKeys); + } + + /** + * This assumes the sort keys have equal weight, which is usually the case for high-cardinality + * scenarios (like device_id, user_id, uuid etc.). + */ + static SortKey[] determineBounds( + int numPartitions, Comparator comparator, SortKey[] sortKeys) { + // sort the keys first + Arrays.sort(sortKeys, comparator); + int numCandidates = numPartitions - 1; + SortKey[] candidates = new SortKey[numCandidates]; + int step = (int) Math.ceil((double) sortKeys.length / numPartitions); + int position = step - 1; + int numChosen = 0; + while (position < sortKeys.length && numChosen < numCandidates) { + SortKey candidate = sortKeys[position]; + // skip duplicate values + if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { + // linear probe for the next distinct value + position += 1; + } else { + candidates[numChosen] = candidate; + position += step; + numChosen += 1; + } + } + + return candidates; + } + + /** This can be a bit expensive since it is quadratic. */ + static void convertMapToSketch( + Map taskMapStats, Consumer sketchConsumer) { + taskMapStats.forEach( + (sortKey, count) -> { + for (int i = 0; i < count; ++i) { + sketchConsumer.accept(sortKey); + } + }); + } +} 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 d03409f2a430..9c0e65a1fc49 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 @@ -276,13 +276,12 @@ public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot< private Schema schema; private SortOrder sortOrder; - @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { // this constructor is used when restoring from a checkpoint. } - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". @SuppressWarnings("checkstyle:RedundantModifier") public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { this.schema = schema; @@ -315,13 +314,17 @@ public void readSnapshot(int readVersion, DataInputView in, ClassLoader userCode @Override public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - TypeSerializer newSerializer) { - if (!(newSerializer instanceof SortKeySerializer)) { + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof SortKeySerializerSnapshot)) { return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; - return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; + if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + return resolveSchemaCompatibility(oldSnapshot.schema, schema); } @Override diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java new file mode 100644 index 000000000000..d6c23f035015 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +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.base.Preconditions; + +/** + * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link + * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The + * implementation is modeled after {@link ArrayOfStringsSerDe} + */ +class SortKeySketchSerializer extends ArrayOfItemsSerDe implements Serializable { + private static final int DEFAULT_SORT_KEY_SIZE = 128; + + private final TypeSerializer itemSerializer; + private final ListSerializer listSerializer; + private final DataInputDeserializer input; + + SortKeySketchSerializer(TypeSerializer itemSerializer) { + this.itemSerializer = itemSerializer; + this.listSerializer = new ListSerializer<>(itemSerializer); + this.input = new DataInputDeserializer(); + } + + @Override + public byte[] serializeToByteArray(SortKey item) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); + itemSerializer.serialize(item, output); + byte[] itemBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[numBytes + Integer.BYTES]; + ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); + ByteArrayUtil.putIntLE(out, 0, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public byte[] serializeToByteArray(SortKey[] items) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); + listSerializer.serialize(Arrays.asList(items), output); + byte[] itemsBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[Integer.BYTES + numBytes]; + ByteArrayUtil.putIntLE(out, 0, numBytes); + System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return new SortKey[0]; + } + + long offset = startingOffset; + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + offset += Integer.BYTES; + + Util.checkBounds(offset, numBytes, mem.getCapacity()); + byte[] sortKeyBytes = new byte[numBytes]; + mem.getByteArray(offset, sortKeyBytes, 0, numBytes); + input.setBuffer(sortKeyBytes); + + try { + List sortKeys = listSerializer.deserialize(input); + SortKey[] array = new SortKey[numItems]; + sortKeys.toArray(array); + input.releaseArrays(); + return array; + } catch (IOException e) { + throw new UncheckedIOException("Failed to deserialize sort key sketch", e); + } + } + + @Override + public int sizeOf(SortKey item) { + return serializeToByteArray(item).length; + } + + @Override + public int sizeOf(Memory mem, long offset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return 0; + } + + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + return Integer.BYTES + numBytes; + } + + @Override + public String toString(SortKey item) { + return item.toString(); + } + + @Override + public Class getClassOfT() { + return SortKey.class; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 68% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index 852d2157b8cb..f3391d72297f 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -27,24 +27,32 @@ * statistics in bytes */ @Internal -class DataStatisticsEvent, S> implements OperatorEvent { +class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; } - static , S> DataStatisticsEvent create( + static StatisticsEvent createTaskStatisticsEvent( long checkpointId, - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - return new DataStatisticsEvent<>( + DataStatistics statistics, + TypeSerializer statisticsSerializer) { + return new StatisticsEvent( + checkpointId, StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer)); + } + + static StatisticsEvent createAggregatedStatisticsEvent( + long checkpointId, + AggregatedStatistics statistics, + TypeSerializer statisticsSerializer) { + return new StatisticsEvent( checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + StatisticsUtil.serializeAggregatedStatistics(statistics, statisticsSerializer)); } long checkpointId() { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 66% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index 889e85112e16..c251ba1360fc 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.io.Serializable; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.data.RowData; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -34,68 +35,66 @@ * After shuffling, a filter and mapper are required to filter out the data distribution weight, * unwrap the object and extract the original record type T. */ -class DataStatisticsOrRecord, S> implements Serializable { +@Internal +public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private DataStatistics statistics; + private AggregatedStatistics statistics; private RowData record; - private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { + private StatisticsOrRecord(AggregatedStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; this.record = record; } - static , S> DataStatisticsOrRecord fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); + static StatisticsOrRecord fromRecord(RowData record) { + return new StatisticsOrRecord(null, record); } - static , S> DataStatisticsOrRecord fromDataStatistics( - DataStatistics statistics) { - return new DataStatisticsOrRecord<>(statistics, null); + static StatisticsOrRecord fromStatistics(AggregatedStatistics statistics) { + return new StatisticsOrRecord(statistics, null); } - static , S> DataStatisticsOrRecord reuseRecord( - DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { + static StatisticsOrRecord reuseRecord( + StatisticsOrRecord reuse, TypeSerializer recordSerializer) { if (reuse.hasRecord()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } } - static , S> DataStatisticsOrRecord reuseStatistics( - DataStatisticsOrRecord reuse, - TypeSerializer> statisticsSerializer) { - if (reuse.hasDataStatistics()) { + static StatisticsOrRecord reuseStatistics( + StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { + if (reuse.hasStatistics()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); } } - boolean hasDataStatistics() { + boolean hasStatistics() { return statistics != null; } - boolean hasRecord() { + public boolean hasRecord() { return record != null; } - DataStatistics dataStatistics() { + AggregatedStatistics statistics() { return statistics; } - void dataStatistics(DataStatistics newStatistics) { + void statistics(AggregatedStatistics newStatistics) { this.statistics = newStatistics; } - RowData record() { + public RowData record() { return record; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 54% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index e9a6fa0cbfc5..7e690f61a58b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -29,13 +29,12 @@ import org.apache.flink.table.data.RowData; @Internal -class DataStatisticsOrRecordSerializer, S> - extends TypeSerializer> { - private final TypeSerializer> statisticsSerializer; +class StatisticsOrRecordSerializer extends TypeSerializer { + private final TypeSerializer statisticsSerializer; private final TypeSerializer recordSerializer; - DataStatisticsOrRecordSerializer( - TypeSerializer> statisticsSerializer, + StatisticsOrRecordSerializer( + TypeSerializer statisticsSerializer, TypeSerializer recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,13 +47,13 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer> duplicate() { - TypeSerializer> duplicateStatisticsSerializer = + public TypeSerializer duplicate() { + TypeSerializer duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( + return new StatisticsOrRecordSerializer( duplicateStatisticsSerializer, duplicateRowDataSerializer); } else { return this; @@ -62,34 +61,32 @@ public TypeSerializer> duplicate() { } @Override - public DataStatisticsOrRecord createInstance() { + public StatisticsOrRecord createInstance() { // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } @Override - public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { + public StatisticsOrRecord copy(StatisticsOrRecord from) { if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); } } @Override - public DataStatisticsOrRecord copy( - DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { - DataStatisticsOrRecord to; + public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { + StatisticsOrRecord to; if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.copy(from.record(), to.record()); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + AggregatedStatistics statistics = + statisticsSerializer.copy(from.statistics(), to.statistics()); + to.statistics(statistics); } return to; @@ -101,41 +98,40 @@ public int getLength() { } @Override - public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) + public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) throws IOException { if (statisticsOrRecord.hasRecord()) { target.writeBoolean(true); recordSerializer.serialize(statisticsOrRecord.record(), target); } else { target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); } } @Override - public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { + public StatisticsOrRecord deserialize(DataInputView source) throws IOException { boolean isRecord = source.readBoolean(); if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); } } @Override - public DataStatisticsOrRecord deserialize( - DataStatisticsOrRecord reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord to; + public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) + throws IOException { + StatisticsOrRecord to; boolean isRecord = source.readBoolean(); if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.deserialize(to.record(), source); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + AggregatedStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + to.statistics(statistics); } return to; @@ -154,12 +150,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + if (!(obj instanceof StatisticsOrRecordSerializer)) { return false; } - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; + StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; return Objects.equals(statisticsSerializer, other.statisticsSerializer) && Objects.equals(recordSerializer, other.recordSerializer); } @@ -170,25 +165,20 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); + public TypeSerializerSnapshot snapshotConfiguration() { + return new StatisticsOrRecordSerializerSnapshot(this); } - public static class DataStatisticsOrRecordSerializerSnapshot, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { + public static class StatisticsOrRecordSerializerSnapshot + extends CompositeTypeSerializerSnapshot { private static final int CURRENT_VERSION = 1; - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); - } + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public StatisticsOrRecordSerializerSnapshot() {} @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer serializer) { + public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { super(serializer); } @@ -200,7 +190,7 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer[] getNestedSerializers( - DataStatisticsOrRecordSerializer outerSerializer) { + StatisticsOrRecordSerializer outerSerializer) { return new TypeSerializer[] { outerSerializer.statisticsSerializer, outerSerializer.recordSerializer }; @@ -208,12 +198,12 @@ protected TypeSerializer[] getNestedSerializers( @SuppressWarnings("unchecked") @Override - protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( + protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { - TypeSerializer> statisticsSerializer = - (TypeSerializer>) nestedSerializers[0]; + TypeSerializer statisticsSerializer = + (TypeSerializer) nestedSerializers[0]; TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java new file mode 100644 index 000000000000..43f72e336e06 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +/** + * Range distribution requires gathering statistics on the sort keys to determine proper range + * boundaries to distribute/cluster rows before writer operators. + */ +public enum StatisticsType { + /** + * Tracks the data statistics as {@code Map} frequency. It works better for + * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in + * hundreds or thousands. + * + *
      + *
    • Pro: accurate measurement on the statistics/weight of every key. + *
    • Con: memory footprint can be large if the key cardinality is high. + *
    + */ + Map, + + /** + * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds + * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, + * uuid etc.) where the cardinalities can be in millions or billions. + * + *
      + *
    • Pro: relatively low memory footprint for high-cardinality sort keys. + *
    • Con: non-precise approximation with potentially lower accuracy. + *
    + */ + Sketch, + + /** + * Initially use Map for statistics tracking. If key cardinality turns out to be high, + * automatically switch to sketch sampling. + */ + Auto +} 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 new file mode 100644 index 000000000000..9d3d128535fe --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java @@ -0,0 +1,97 @@ +/* + * 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.io.IOException; +import java.io.UncheckedIOException; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class StatisticsUtil { + + private StatisticsUtil() {} + + static DataStatistics createTaskStatistics( + StatisticsType type, int operatorParallelism, int numPartitions) { + if (type == StatisticsType.Map) { + return new MapDataStatistics(); + } else { + return new SketchDataStatistics( + SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); + } + } + + static byte[] serializeDataStatistics( + DataStatistics dataStatistics, TypeSerializer statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize data statistics", e); + } + } + + static DataStatistics deserializeDataStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize data statistics", e); + } + } + + static byte[] serializeAggregatedStatistics( + AggregatedStatistics aggregatedStatistics, + TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(aggregatedStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static AggregatedStatistics deserializeAggregatedStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static StatisticsType collectType(StatisticsType config) { + return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; + } + + static StatisticsType collectType( + StatisticsType config, @Nullable AggregatedStatistics restoredStatistics) { + if (restoredStatistics != null) { + return restoredStatistics.type(); + } + + return collectType(config); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java new file mode 100644 index 000000000000..c7e9f19abb02 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -0,0 +1,98 @@ +/* + * 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.Comparator; +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +class Fixtures { + private Fixtures() {} + + public static final int NUM_SUBTASKS = 2; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); + public static final TypeSerializer ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); + public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + public static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); + public static final SortKeySerializer SORT_KEY_SERIALIZER = + new SortKeySerializer(SCHEMA, SORT_ORDER); + public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = + new DataStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final AggregatedStatisticsSerializer AGGREGATED_STATISTICS_SERIALIZER = + new AggregatedStatisticsSerializer(SORT_KEY_SERIALIZER); + + public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + public static final Map CHAR_KEYS = createCharKeys(); + + public static StatisticsEvent createStatisticsEvent( + StatisticsType type, + TypeSerializer statisticsSerializer, + long checkpointId, + SortKey... keys) { + DataStatistics statistics = createTaskStatistics(type, keys); + return StatisticsEvent.createTaskStatisticsEvent( + checkpointId, statistics, statisticsSerializer); + } + + public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { + DataStatistics statistics; + if (type == StatisticsType.Sketch) { + statistics = new SketchDataStatistics(128); + } else { + statistics = new MapDataStatistics(); + } + + for (SortKey key : keys) { + statistics.add(key); + } + + return statistics; + } + + private static Map createCharKeys() { + Map keys = Maps.newHashMap(); + for (char c = 'a'; c <= 'z'; ++c) { + String key = Character.toString(c); + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, key); + keys.put(key, sortKey); + } + + return keys; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index 890cc361b246..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,63 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Test; - -public class TestAggregatedStatistics { - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - @Test - public void mergeDataStatisticTest() { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>(1, statisticsSerializer); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(keyA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyB) - + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java new file mode 100644 index 000000000000..0ce73fa4aaad --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java @@ -0,0 +1,54 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestAggregatedStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.AGGREGATED_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return AggregatedStatistics.class; + } + + @Override + protected AggregatedStatistics[] getTestData() { + return new AggregatedStatistics[] { + AggregatedStatistics.fromKeyFrequency( + 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), + AggregatedStatistics.fromRangeBounds( + 2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 4c64ce522201..52485a3c67d9 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -18,161 +18,420 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - private final SortKey keyA = sortKey.copy(); - private final SortKey keyB = sortKey.copy(); - - private AggregatedStatisticsTracker> - aggregatedStatisticsTracker; - - public TestAggregatedStatisticsTracker() { - keyA.set(0, "a"); - keyB.set(0, "b"); - } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveNewerStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); - @Before - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + AggregatedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + // checkpoint 1 is completed + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + + // checkpoint 2 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } } - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - checkpoint2Subtask0DataStatistic.add(keyB); - checkpoint2Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveOlderStatisticsEventTest(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + AggregatedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint3Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); + // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 4L)); + } else { + assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } } - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveCompletedStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + + AggregatedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(keyA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyA) - + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); - assertThat((long) globalDataStatistics.statistics().get(keyB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyB) - + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 3L, + CHAR_KEYS.get("b"), 3L)); + } else { + assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask0DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); + completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask1DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); // Receive data statistics from all subtasks at checkpoint 2 completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); + tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + } + + @Test + public void coordinatorSwitchToSketchOverThreshold() { + int parallelism = 3; + int downstreamParallelism = 3; + int switchToSketchThreshold = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + switchToSketchThreshold, + null); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + AggregatedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + // converted to sketch statistics as map size is 4 (over the switch threshold of 3) + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.rangeBounds()) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void coordinatorMapOperatorSketch() { + int parallelism = 3; + int downstreamParallelism = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + + // first operator event has map statistics + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + AggregatedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + // second operator event contains sketch statistics + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Sketch, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + // third operator event has Map statistics + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.rangeBounds()) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + private AggregatedStatisticsTracker createTracker(StatisticsType type) { + return new AggregatedStatisticsTracker( + "testOperator", + Fixtures.NUM_SUBTASKS, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); } } 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 3df714059c37..fe1d07c3286a 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 @@ -18,138 +18,114 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; -import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> - dataStatisticsCoordinator; - @Before + @BeforeEach public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); } - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + private void tasksReady(DataStatisticsCoordinator coordinator) { + setAllTasksReady(Fixtures.NUM_SUBTASKS, coordinator, receivingTasks); } - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + StatisticsEvent.createTaskStatisticsEvent( + 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } } - @Test - public void testDataStatisticsEventHandling() throws Exception { - tasksReady(); - SortKey key = sortKey.copy(); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify global data statistics is the aggregation of all subtasks data statistics - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - keyA, 2L, - keyB, 3L, - keyC, 5L)); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + 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, + CHAR_KEYS.get("a"), + 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); + + // Verify global data statistics is the aggregation of all subtasks data statistics + AggregatedStatistics aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(aggregatedStatistics.checkpointId()).isEqualTo(1L); + assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregatedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 3L, + CHAR_KEYS.get("c"), 5L)); + } else { + assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + } + } } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -157,8 +133,7 @@ static void setAllTasksReady( } } - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { @@ -175,4 +150,14 @@ static void waitForCoordinatorToProcessActions( ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); } } + + private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { + return new DataStatisticsCoordinator( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, Fixtures.NUM_SUBTASKS), + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index 5e0a752be506..966c8474b42b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -18,126 +18,156 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; -import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - @Before + @BeforeEach public void before() { - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); receivingTasks = EventReceivingTasks.createForRunningTasks(); } - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - SortKey keyD = sortKey.copy(); - keyD.set(0, "c"); - SortKey keyE = sortKey.copy(); - keyE.set(0, "c"); - + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testCheckpointAndReset(StatisticsType type) throws Exception { + DataStatisticsCoordinatorProvider provider = createProvider(type); try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create( + new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { + DataStatisticsCoordinator dataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); // Start the coordinator coordinator.start(); TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c")); + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + assertThat(dataStatisticsCoordinator.completedStatistics()).isNotNull(); + AggregatedStatistics aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregatedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), + 1L, + CHAR_KEYS.get("b"), + 1L, + CHAR_KEYS.get("c"), + 1L, + CHAR_KEYS.get("d"), + 1L, + CHAR_KEYS.get("e"), + 1L)); + } else { + assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + } + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyD); - checkpoint2Subtask0DataStatistic.add(keyE); - checkpoint2Subtask0DataStatistic.add(keyE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); + coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregatedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L)); + } else { + assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + } waitForCheckpoint(2L, dataStatisticsCoordinator); // Reset coordinator to checkpoint 1 coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + DataStatisticsCoordinator restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + aggregatedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregatedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), + 1L, + CHAR_KEYS.get("b"), + 1L, + CHAR_KEYS.get("c"), + 1L, + CHAR_KEYS.get("d"), + 1L, + CHAR_KEYS.get("e"), + 1L)); + } else { + assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + } } } - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator> coordinator) + private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); return future.get(); } + + private static DataStatisticsCoordinatorProvider createProvider(StatisticsType type) { + return new DataStatisticsCoordinatorProvider( + "DataStatisticsCoordinatorProvider", + OPERATOR_ID, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type); + } } 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 8ddb147e07c1..6c864791a1bc 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 @@ -18,22 +18,21 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; import static org.assertj.core.api.Assertions.assertThat; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; @@ -50,102 +49,85 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; 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.types.Types; -import org.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsOperator { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final RowType rowType = RowType.of(new VarCharType(), new IntType()); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer( - new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsOperator> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } - @Before + private Environment env; + + @BeforeEach public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); + this.env = + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator createOperator(StatisticsType type) throws Exception { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return new DataStatisticsOperator<>( - "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); - } - - @After - public void clean() throws Exception { - operator.close(); + DataStatisticsOperator operator = + new DataStatisticsOperator( + "testOperator", + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + mockGateway, + Fixtures.NUM_SUBTASKS, + type); + operator.setup( + new OneInputStreamTask(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + return operator; } - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { + @SuppressWarnings("unchecked") + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElement(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); operator.initializeState(stateContext); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 5))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); - - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + DataStatistics localStatistics = operator.localStatistics(); + assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + Map keyFrequency = (Map) localStatistics.result(); + assertThat(keyFrequency) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); + } else { + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) localStatistics.result(); + assertThat(sketch.getSamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } testHarness.endInput(); } } - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testOperatorOutput(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { testHarness.processElement( new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); testHarness.processElement( @@ -155,8 +137,8 @@ public void testOperatorOutput() throws Exception { List recordsOutput = testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record) .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( @@ -167,70 +149,151 @@ public void testOperatorOutput() throws Exception { } } - @Test - public void testRestoreState() throws Exception { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testRestoreState(StatisticsType type) throws Exception { + Map keyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); + SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; + DataStatisticsOperator operator = createOperator(type); OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness1 = createHarness(this.operator)) { - MapDataStatistics mapDataStatistics = new MapDataStatistics(); - - SortKey key = sortKey.copy(); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "b"); - mapDataStatistics.add(key); - key.set(0, "c"); - mapDataStatistics.add(key); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - DataStatisticsEvent> event = - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + try (OneInputStreamOperatorTestHarness testHarness1 = + createHarness(operator)) { + AggregatedStatistics statistics; + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + statistics = AggregatedStatistics.fromKeyFrequency(1L, keyFrequency); + } else { + statistics = AggregatedStatistics.fromRangeBounds(1L, rangeBounds); + } + + StatisticsEvent event = + StatisticsEvent.createAggregatedStatisticsEvent( + 1L, statistics, Fixtures.AGGREGATED_STATISTICS_SERIALIZER); operator.handleOperatorEvent(event); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf(expectedMap); + + AggregatedStatistics globalStatistics = operator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.keyFrequency()).isEqualTo(keyFrequency); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.keyFrequency()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + DataStatisticsOperator restoredOperator = createOperator(type); + try (OneInputStreamOperatorTestHarness testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - // restored RowData is BinaryRowData. convert to GenericRowData for comparison - Map restoredStatistics = Maps.newHashMap(); - restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + AggregatedStatistics globalStatistics = restoredOperator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.keyFrequency()).isEqualTo(keyFrequency); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.keyFrequency()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + } + } - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithLocalStatsOverThreshold() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // add rows with unique keys + for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()).hasSize(i + 1); + } + + // one more item should trigger the migration to sketch stats + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); + // reservoir not full yet + assertThat(sketch.getN()).isLessThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); + + // add more items to saturate the reservoir + for (int i = 0; i < reservoirSize; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + } + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + sketch = (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()) + .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); + // reservoir is full now + assertThat(sketch.getN()).isGreaterThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize(reservoirSize); + + testHarness.endInput(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithGlobalSketchStatistics() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); - assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); + // started with Map stype + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + + // received global statistics with sketch type + AggregatedStatistics globalStatistics = + AggregatedStatistics.fromRangeBounds( + 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); + operator.handleOperatorEvent( + StatisticsEvent.createAggregatedStatisticsEvent( + 1L, globalStatistics, Fixtures.AGGREGATED_STATISTICS_SERIALIZER)); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(1); + assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); + + testHarness.endInput(); } } private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); OperatorStateStore operatorStateStore = @@ -240,17 +303,14 @@ private StateInitializationContext getStateContext() throws Exception { return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); } - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - createHarness( - final DataStatisticsOperator> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); + private OneInputStreamOperatorTestHarness createHarness( + DataStatisticsOperator dataStatisticsOperator) throws Exception { + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>( + dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); + harness.setup( + new StatisticsOrRecordSerializer( + Fixtures.AGGREGATED_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); harness.open(); return harness; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java new file mode 100644 index 000000000000..59ce6df05d9d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class TestDataStatisticsSerializer extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return Fixtures.TASK_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return DataStatistics.class; + } + + @Override + protected DataStatistics[] getTestData() { + return new DataStatistics[] { + new MapDataStatistics(), + Fixtures.createTaskStatistics( + StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), + new SketchDataStatistics(128), + Fixtures.createTaskStatistics( + StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index a07808e935d9..0ab7d7c3b2ba 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -18,73 +18,50 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; + import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { - private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); - private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private final RowDataWrapper rowWrapper = - new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); - + @SuppressWarnings("unchecked") @Test public void testAddsAndGet() { MapDataStatistics dataStatistics = new MapDataStatistics(); - GenericRowData reusedRow = - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("c")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("a")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - Map actual = dataStatistics.statistics(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyA = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyB = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyC = sortKey.copy(); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); - Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Map actual = (Map) dataStatistics.result(); + Map expected = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); Assertions.assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index 92eb71acc834..dc3bad83947a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -63,34 +63,33 @@ private static SortKey[] initSortKeys() { } // Total weight is 800 - private final MapDataStatistics mapDataStatistics = - new MapDataStatistics( - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L)); + private final Map mapStatistics = + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L); @Test public void testEvenlyDividableNoClosingFileCost() { MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 0.0); int numPartitions = 8; // each task should get targeted weight of 100 (=800/8) @@ -154,7 +153,7 @@ public void testEvenlyDividableNoClosingFileCost() { @Test public void testEvenlyDividableWithClosingFileCost() { MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 5.0); int numPartitions = 8; // target subtask weight is 100 before close file cost factored in. @@ -226,7 +225,7 @@ public void testEvenlyDividableWithClosingFileCost() { @Test public void testNonDividableNoClosingFileCost() { MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 0.0); int numPartitions = 9; // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 @@ -294,7 +293,7 @@ public void testNonDividableNoClosingFileCost() { @Test public void testNonDividableWithClosingFileCost() { MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 5.0); int numPartitions = 9; // target subtask weight is 89 before close file cost factored in. diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java new file mode 100644 index 000000000000..658d31c01143 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; + +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +public class TestSketchDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + SketchDataStatistics dataStatistics = new SketchDataStatistics(128); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + ReservoirItemsSketch actual = (ReservoirItemsSketch) dataStatistics.result(); + Assertions.assertThat(actual.getSamples()) + .isEqualTo( + new SortKey[] { + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") + }); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java new file mode 100644 index 000000000000..435748281f83 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -0,0 +1,133 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchUtil { + @Test + public void testCoordinatorReservoirSize() { + // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); + // adjust to multiplier of 100 + assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); + // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) + .isEqualTo(1_000_000 - (1_000_000 % 10_123)); + } + + @Test + public void testOperatorReservoirSize() { + assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) + .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); + assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) + .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); + assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) + .isEqualTo( + (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); + assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) + .isEqualTo( + (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); + } + + @Test + public void testRangeBoundsOneChannel() { + assertThat( + SketchUtil.determineBounds( + 1, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .isEmpty(); + } + + @Test + public void testRangeBoundsDivisible() { + assertThat( + SketchUtil.determineBounds( + 3, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void testRangeBoundsNonDivisible() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.determineBounds( + 4, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); + } + + @Test + public void testRangeBoundsSkipDuplicates() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.determineBounds( + 4, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + // skipped duplicate c's + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); + } +} 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 291302aef486..54cceae6e55b 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 @@ -18,14 +18,24 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.iceberg.NullOrder; import org.apache.iceberg.Schema; import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.RowDataWrapper; +import org.junit.jupiter.api.Test; public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { private final DataGenerator generator = new DataGenerators.Primitives(); @@ -54,4 +64,27 @@ protected SortOrder sortOrder() { protected GenericRowData rowData() { return generator.generateFlinkRowData(); } + + @Test + public void testSerializationSize() throws Exception { + RowData rowData = + GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = Fixtures.SORT_KEY.copy(); + sortKey.wrap(struct); + SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataOutputSerializer output = new DataOutputSerializer(1024); + serializer.serialize(sortKey, output); + 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); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = serializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 1ad8ee69147d..a5875a75ecf6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -33,6 +33,7 @@ azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" +datasketches = "6.0.0" delta-standalone = "3.1.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" @@ -102,6 +103,7 @@ azuresdk-bom = { module = "com.azure:azure-sdk-bom", version.ref = "azuresdk-bom caffeine = { module = "com.github.ben-manes.caffeine:caffeine", version.ref = "caffeine" } calcite-core = { module = "org.apache.calcite:calcite-core", version.ref = "calcite" } calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "calcite" } +datasketches = { module = "org.apache.datasketches:datasketches-java", version.ref = "datasketches" } delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } From 7832d3cb789aa542787f19ed0a59e5dcc5d2d366 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 5 Jun 2024 10:31:49 -0700 Subject: [PATCH 0351/1019] Spark 3.4, 3.5: SHOW VIEWS failed with AssertionError (#10442) --- .../sql/catalyst/analysis/RewriteViewCommands.scala | 11 +++++++---- .../apache/iceberg/spark/extensions/TestViews.java | 5 +++++ .../java/org/apache/iceberg/spark/SmokeTest.java | 9 +++++++++ .../sql/catalyst/analysis/RewriteViewCommands.scala | 11 +++++++---- .../apache/iceberg/spark/extensions/TestViews.java | 5 +++++ .../java/org/apache/iceberg/spark/SmokeTest.java | 9 +++++++++ 6 files changed, 42 insertions(+), 8 deletions(-) diff --git a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 377025f92644..3e854dfcb966 100644 --- a/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.4/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -66,10 +66,13 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi allowExisting = allowExisting, replace = replace) - case ShowViews(UnresolvedNamespace(Seq()), pattern, output) - if ViewUtil.isViewCatalog(catalogManager.currentCatalog) => - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), - pattern, output) + case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) => + if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, output) + } else { + view + } case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)), pattern, output) if ViewUtil.isViewCatalog(catalog) => 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 624b4e354937..65b8669c35a0 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 @@ -1463,6 +1463,11 @@ public void showViews() throws NoSuchTableException { .contains( // spark stores temp views case-insensitive by default row("global_temp", "globalviewforlisting", true), tempView); + + sql("USE spark_catalog"); + assertThat(sql("SHOW VIEWS")).contains(tempView); + + assertThat(sql("SHOW VIEWS IN default")).contains(tempView); } @Test diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 25d7e7471588..59bc70854cbe 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; @@ -165,6 +167,13 @@ public void testCreateTable() { Assert.assertEquals("Should be partitioned on 3 columns", 3, third.spec().fields().size()); } + @Test + public void showView() { + sql("DROP VIEW IF EXISTS %s", "test"); + sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); + assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + } + private Table getTable(String name) { return validationCatalog.loadTable(TableIdentifier.of("default", name)); } diff --git a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala index 011626c280b6..c5bd648f2e4e 100644 --- a/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala +++ b/spark/v3.5/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteViewCommands.scala @@ -66,10 +66,13 @@ case class RewriteViewCommands(spark: SparkSession) extends Rule[LogicalPlan] wi allowExisting = allowExisting, replace = replace) - case ShowViews(UnresolvedNamespace(Seq()), pattern, output) - if ViewUtil.isViewCatalog(catalogManager.currentCatalog) => - ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), - pattern, output) + case view @ ShowViews(UnresolvedNamespace(Seq()), pattern, output) => + if (ViewUtil.isViewCatalog(catalogManager.currentCatalog)) { + ShowIcebergViews(ResolvedNamespace(catalogManager.currentCatalog, catalogManager.currentNamespace), + pattern, output) + } else { + view + } case ShowViews(UnresolvedNamespace(CatalogAndNamespace(catalog, ns)), pattern, output) if ViewUtil.isViewCatalog(catalog) => 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 3cc1e32d00a1..d521cc72cb31 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 @@ -1462,6 +1462,11 @@ public void showViews() throws NoSuchTableException { .contains( // spark stores temp views case-insensitive by default row("global_temp", "globalviewforlisting", true), tempView); + + sql("USE spark_catalog"); + assertThat(sql("SHOW VIEWS")).contains(tempView); + + assertThat(sql("SHOW VIEWS IN default")).contains(tempView); } @TestTemplate diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 207fca3cc216..4fb1a8ad0628 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.nio.file.Files; import org.apache.iceberg.ParameterizedTestExtension; @@ -174,6 +176,13 @@ public void testCreateTable() { .hasSize(3); } + @TestTemplate + public void showView() { + sql("DROP VIEW IF EXISTS %s", "test"); + sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); + assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + } + private Table getTable(String name) { return validationCatalog.loadTable(TableIdentifier.of("default", name)); } From 82a2b1975175f0ed380fd469fcf095ca1ecda61e Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 5 Jun 2024 12:14:00 -0600 Subject: [PATCH 0352/1019] Core, Parquet, Orc: Don't write column sizes when metrics mode is None (#10440) --- core/src/test/java/org/apache/iceberg/TestMetrics.java | 6 +++++- orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java | 3 ++- .../main/java/org/apache/iceberg/parquet/ParquetUtil.java | 4 ++-- 3 files changed, 9 insertions(+), 4 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index 424e0e0a7b93..b95b92979f91 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -561,7 +561,7 @@ public void testNoneMetricsMode() throws IOException { MetricsConfig.fromProperties(ImmutableMap.of("write.metadata.metrics.default", "none")), buildNestedTestRecord()); assertThat(metrics.recordCount()).isEqualTo(1L); - assertThat(metrics.columnSizes()).doesNotContainValue(null); + assertThat(metrics.columnSizes()).isEmpty(); assertCounts(1, null, null, metrics); assertBounds(1, Types.IntegerType.get(), null, null, metrics); assertCounts(3, null, null, metrics); @@ -584,6 +584,7 @@ public void testCountsMetricsMode() throws IOException { buildNestedTestRecord()); assertThat(metrics.recordCount()).isEqualTo(1L); assertThat(metrics.columnSizes()).doesNotContainValue(null); + assertThat(metrics.columnSizes()).isNotEmpty(); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.IntegerType.get(), null, null, metrics); assertCounts(3, 1L, 0L, metrics); @@ -605,6 +606,7 @@ public void testFullMetricsMode() throws IOException { buildNestedTestRecord()); assertThat(metrics.recordCount()).isEqualTo(1L); assertThat(metrics.columnSizes()).doesNotContainValue(null); + assertThat(metrics.columnSizes()).isNotEmpty(); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.IntegerType.get(), Integer.MAX_VALUE, Integer.MAX_VALUE, metrics); assertCounts(3, 1L, 0L, metrics); @@ -642,6 +644,7 @@ public void testTruncateStringMetricsMode() throws IOException { CharBuffer expectedMaxBound = CharBuffer.wrap("Lorem ipsv"); assertThat(metrics.recordCount()).isEqualTo(1L); assertThat(metrics.columnSizes()).doesNotContainValue(null); + assertThat(metrics.columnSizes()).isNotEmpty(); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.StringType.get(), expectedMinBound, expectedMaxBound, metrics); } @@ -666,6 +669,7 @@ public void testTruncateBinaryMetricsMode() throws IOException { ByteBuffer expectedMaxBounds = ByteBuffer.wrap(new byte[] {0x1, 0x2, 0x3, 0x4, 0x6}); assertThat(metrics.recordCount()).isEqualTo(1L); assertThat(metrics.columnSizes()).doesNotContainValue(null); + assertThat(metrics.columnSizes()).isNotEmpty(); assertCounts(1, 1L, 0L, metrics); assertBounds(1, Types.BinaryType.get(), expectedMinBounds, expectedMaxBounds, metrics); } diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java b/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java index 972591d53d03..b057b265dfc3 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcMetrics.java @@ -160,12 +160,13 @@ private static Metrics buildOrcMetrics( final MetricsMode metricsMode = MetricsUtil.metricsMode(schema, effectiveMetricsConfig, icebergCol.fieldId()); - columnSizes.put(fieldId, colStat.getBytesOnDisk()); if (metricsMode == MetricsModes.None.get()) { continue; } + columnSizes.put(fieldId, colStat.getBytesOnDisk()); + if (statsColumns.contains(fieldId)) { // Since ORC does not track null values nor repeated ones, the value count for columns in // containers (maps, list) may be larger than what it actually is, however these are not diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java index 2de423146ac6..63d5adbbd201 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetUtil.java @@ -124,12 +124,12 @@ public static Metrics footerMetrics( continue; } - increment(columnSizes, fieldId, column.getTotalSize()); - MetricsMode metricsMode = MetricsUtil.metricsMode(fileSchema, metricsConfig, fieldId); if (metricsMode == MetricsModes.None.get()) { continue; } + + increment(columnSizes, fieldId, column.getTotalSize()); increment(valueCounts, fieldId, column.getValueCount()); Statistics stats = column.getStatistics(); From 6b90dda59d859f8d4cfd117fcdf5175982fc2e72 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 5 Jun 2024 18:32:45 -0700 Subject: [PATCH 0353/1019] Spark 3.4, 3.5: Follow-up for #10442, Remove static test import (#10451) --- .../integration/java/org/apache/iceberg/spark/SmokeTest.java | 5 ++--- .../integration/java/org/apache/iceberg/spark/SmokeTest.java | 4 +--- 2 files changed, 3 insertions(+), 6 deletions(-) diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 59bc70854cbe..4704bcbbdac5 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -18,14 +18,13 @@ */ package org.apache.iceberg.spark; -import static org.assertj.core.api.Assertions.assertThat; - import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.spark.extensions.SparkExtensionsTestBase; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -171,7 +170,7 @@ public void testCreateTable() { public void showView() { sql("DROP VIEW IF EXISTS %s", "test"); sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); - assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + Assertions.assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); } private Table getTable(String name) { diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 4fb1a8ad0628..ee735a4f84a8 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg.spark; -import static org.assertj.core.api.Assertions.assertThat; - import java.io.IOException; import java.nio.file.Files; import org.apache.iceberg.ParameterizedTestExtension; @@ -180,7 +178,7 @@ public void testCreateTable() { public void showView() { sql("DROP VIEW IF EXISTS %s", "test"); sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); - assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + Assertions.assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); } private Table getTable(String name) { From 1a1a9e48f6fe65fbfa3a9910a8099520b9a00d9c Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 6 Jun 2024 09:51:25 +0200 Subject: [PATCH 0354/1019] Flink: Maintenance - MonitorSource (#10308) --- .../maintenance/operator/MonitorSource.java | 206 ++++++++++ .../SingleThreadedIteratorSource.java | 197 ++++++++++ .../maintenance/operator/TableChange.java | 133 +++++++ .../maintenance/operator/CollectingSink.java | 115 ++++++ .../operator/FlinkSqlExtension.java | 132 +++++++ .../operator/FlinkStreamingTestUtils.java | 73 ++++ .../maintenance/operator/ManualSource.java | 316 +++++++++++++++ .../operator/OperatorTestBase.java | 51 +++ .../operator/TestMonitorSource.java | 362 ++++++++++++++++++ 9 files changed, 1585 insertions(+) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java create 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/ManualSource.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java 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 new file mode 100644 index 000000000000..d74b2349b1de --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.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.flink.maintenance.operator; + +import java.io.IOException; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiter; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Monitors an Iceberg table for changes */ +@Internal +public class MonitorSource extends SingleThreadedIteratorSource { + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final TableLoader tableLoader; + private final RateLimiterStrategy rateLimiterStrategy; + private final long maxReadBack; + + /** + * Creates a {@link org.apache.flink.api.connector.source.Source} which monitors an Iceberg table + * for changes. + * + * @param tableLoader used for accessing the table + * @param rateLimiterStrategy limits the frequency the table is checked + * @param maxReadBack sets the number of snapshots read before stopping change collection + */ + 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"); + Preconditions.checkArgument(maxReadBack > 0, "Need to read at least 1 snapshot to work"); + + this.tableLoader = tableLoader; + this.rateLimiterStrategy = rateLimiterStrategy; + this.maxReadBack = maxReadBack; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(TableChange.class); + } + + @Override + Iterator createIterator() { + return new TableChangeIterator(tableLoader, null, maxReadBack); + } + + @Override + SimpleVersionedSerializer> iteratorSerializer() { + return new TableChangeIteratorSerializer(tableLoader, maxReadBack); + } + + @Override + public SourceReader> createReader( + SourceReaderContext readerContext) throws Exception { + RateLimiter rateLimiter = rateLimiterStrategy.createRateLimiter(1); + return new RateLimitedSourceReader<>(super.createReader(readerContext), rateLimiter); + } + + /** The Iterator which returns the latest changes on an Iceberg table. */ + @VisibleForTesting + static class TableChangeIterator implements Iterator { + private Long lastSnapshotId; + private final long maxReadBack; + private final Table table; + + TableChangeIterator(TableLoader tableLoader, Long lastSnapshotId, long maxReadBack) { + this.lastSnapshotId = lastSnapshotId; + this.maxReadBack = maxReadBack; + tableLoader.open(); + this.table = tableLoader.loadTable(); + } + + @Override + public boolean hasNext() { + return true; + } + + @Override + public TableChange next() { + try { + table.refresh(); + Snapshot currentSnapshot = table.currentSnapshot(); + Long current = currentSnapshot != null ? currentSnapshot.snapshotId() : null; + Long checking = current; + TableChange event = TableChange.empty(); + long readBack = 0; + while (checking != null && !checking.equals(lastSnapshotId) && ++readBack <= maxReadBack) { + Snapshot snapshot = table.snapshot(checking); + if (snapshot != null) { + if (!DataOperations.REPLACE.equals(snapshot.operation())) { + LOG.debug("Reading snapshot {}", snapshot.snapshotId()); + event.merge(new TableChange(snapshot, table.io())); + } else { + LOG.debug("Skipping replace snapshot {}", snapshot.snapshotId()); + } + + checking = snapshot.parentId(); + } else { + // If the last snapshot has been removed from the history + checking = null; + } + } + + lastSnapshotId = current; + return event; + } catch (Exception e) { + LOG.warn("Failed to fetch table changes for {}", table, e); + return TableChange.empty(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("lastSnapshotId", lastSnapshotId) + .add("maxReadBack", maxReadBack) + .add("table", table) + .toString(); + } + } + + private static final class TableChangeIteratorSerializer + implements SimpleVersionedSerializer> { + + private static final int CURRENT_VERSION = 1; + private final TableLoader tableLoader; + private final long maxReadBack; + + TableChangeIteratorSerializer(TableLoader tableLoader, long maxReadBack) { + this.tableLoader = tableLoader; + this.maxReadBack = maxReadBack; + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(Iterator iterator) throws IOException { + Preconditions.checkArgument( + iterator instanceof TableChangeIterator, + "Use TableChangeIterator iterator. Found incompatible type: %s", + iterator.getClass()); + + TableChangeIterator tableChangeIterator = (TableChangeIterator) iterator; + DataOutputSerializer out = new DataOutputSerializer(8); + long toStore = + tableChangeIterator.lastSnapshotId != null ? tableChangeIterator.lastSnapshotId : -1L; + out.writeLong(toStore); + return out.getCopyOfBuffer(); + } + + @Override + public TableChangeIterator deserialize(int version, byte[] serialized) throws IOException { + if (version == CURRENT_VERSION) { + DataInputDeserializer in = new DataInputDeserializer(serialized); + long fromStore = in.readLong(); + return new TableChangeIterator( + tableLoader, fromStore != -1 ? fromStore : null, maxReadBack); + } else { + throw new IOException("Unrecognized version or corrupt state: " + version); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java new file mode 100644 index 000000000000..20c7684d9700 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java @@ -0,0 +1,197 @@ +/* + * 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.IOException; +import java.util.Collection; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; +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; + +/** + * Implementation of the Source V2 API which uses an iterator to read the elements, and uses a + * single thread to do so. + * + * @param The return type of the source + */ +@Internal +public abstract class SingleThreadedIteratorSource + implements Source< + T, + SingleThreadedIteratorSource.GlobalSplit, + Collection>>, + ResultTypeQueryable { + private static final String PARALLELISM_ERROR = "Parallelism should be set to 1"; + + /** + * Creates the iterator to return the elements which then emitted by the source. + * + * @return iterator for the elements + */ + abstract Iterator createIterator(); + + /** + * Serializes the iterator, which is used to save and restore the state of the source. + * + * @return serializer for the iterator + */ + abstract SimpleVersionedSerializer> iteratorSerializer(); + + @Override + public SplitEnumerator, Collection>> createEnumerator( + SplitEnumeratorContext> enumContext) { + Preconditions.checkArgument(enumContext.currentParallelism() == 1, PARALLELISM_ERROR); + return new IteratorSourceEnumerator<>( + enumContext, ImmutableList.of(new GlobalSplit<>(createIterator()))); + } + + @Override + public SplitEnumerator, Collection>> restoreEnumerator( + SplitEnumeratorContext> enumContext, Collection> checkpoint) { + Preconditions.checkArgument(enumContext.currentParallelism() == 1, PARALLELISM_ERROR); + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer> getSplitSerializer() { + return new SplitSerializer<>(iteratorSerializer()); + } + + @Override + public SimpleVersionedSerializer>> getEnumeratorCheckpointSerializer() { + return new EnumeratorSerializer<>(iteratorSerializer()); + } + + @Override + public SourceReader> createReader(SourceReaderContext readerContext) + throws Exception { + Preconditions.checkArgument(readerContext.getIndexOfSubtask() == 0, PARALLELISM_ERROR); + return new IteratorSourceReader<>(readerContext); + } + + /** The single split of the {@link SingleThreadedIteratorSource}. */ + static class GlobalSplit implements IteratorSourceSplit> { + private final Iterator iterator; + + GlobalSplit(Iterator iterator) { + this.iterator = iterator; + } + + @Override + public String splitId() { + return "1"; + } + + @Override + public Iterator getIterator() { + return iterator; + } + + @Override + public IteratorSourceSplit> getUpdatedSplitForIterator( + final Iterator newIterator) { + return new GlobalSplit<>(newIterator); + } + + @Override + public String toString() { + return String.format("GlobalSplit (%s)", iterator); + } + } + + private static final class SplitSerializer + implements SimpleVersionedSerializer> { + private final SimpleVersionedSerializer> iteratorSerializer; + + SplitSerializer(SimpleVersionedSerializer> iteratorSerializer) { + this.iteratorSerializer = iteratorSerializer; + } + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(GlobalSplit split) throws IOException { + return iteratorSerializer.serialize(split.iterator); + } + + @Override + public GlobalSplit deserialize(int version, byte[] serialized) throws IOException { + return new GlobalSplit<>(iteratorSerializer.deserialize(version, serialized)); + } + } + + private static final class EnumeratorSerializer + implements SimpleVersionedSerializer>> { + private static final int CURRENT_VERSION = 1; + private final SimpleVersionedSerializer> iteratorSerializer; + + EnumeratorSerializer(SimpleVersionedSerializer> iteratorSerializer) { + this.iteratorSerializer = iteratorSerializer; + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(Collection> checkpoint) throws IOException { + Preconditions.checkArgument(checkpoint.size() < 2, PARALLELISM_ERROR); + if (checkpoint.isEmpty()) { + return new byte[] {0}; + } else { + byte[] iterator = iteratorSerializer.serialize(checkpoint.iterator().next().getIterator()); + byte[] result = new byte[iterator.length + 1]; + result[0] = 1; + System.arraycopy(iterator, 0, result, 1, iterator.length); + return result; + } + } + + @Override + public Collection> deserialize(int version, byte[] serialized) + throws IOException { + if (serialized[0] == 0) { + return Lists.newArrayList(); + } else { + byte[] iterator = new byte[serialized.length - 1]; + System.arraycopy(serialized, 1, iterator, 0, serialized.length - 1); + return Lists.newArrayList( + new GlobalSplit<>(iteratorSerializer.deserialize(version, iterator))); + } + } + } +} 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 new file mode 100644 index 000000000000..452ed80ed0e5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -0,0 +1,133 @@ +/* + * 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.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** Event describing changes in an Iceberg table */ +@Internal +class TableChange { + private int dataFileNum; + private int deleteFileNum; + private long dataFileSize; + private long deleteFileSize; + private int commitNum; + + TableChange( + int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { + this.dataFileNum = dataFileNum; + this.deleteFileNum = deleteFileNum; + this.dataFileSize = dataFileSize; + this.deleteFileSize = deleteFileSize; + this.commitNum = commitNum; + } + + TableChange(Snapshot snapshot, FileIO io) { + Iterable dataFiles = snapshot.addedDataFiles(io); + Iterable deleteFiles = snapshot.addedDeleteFiles(io); + + dataFiles.forEach( + dataFile -> { + this.dataFileNum++; + this.dataFileSize += dataFile.fileSizeInBytes(); + }); + + deleteFiles.forEach( + deleteFile -> { + this.deleteFileNum++; + this.deleteFileSize += deleteFile.fileSizeInBytes(); + }); + + this.commitNum = 1; + } + + static TableChange empty() { + return new TableChange(0, 0, 0L, 0L, 0); + } + + int dataFileNum() { + return dataFileNum; + } + + int deleteFileNum() { + return deleteFileNum; + } + + long dataFileSize() { + return dataFileSize; + } + + long deleteFileSize() { + return deleteFileSize; + } + + public int commitNum() { + return commitNum; + } + + public void merge(TableChange other) { + this.dataFileNum += other.dataFileNum; + this.deleteFileNum += other.deleteFileNum; + this.dataFileSize += other.dataFileSize; + this.deleteFileSize += other.deleteFileSize; + this.commitNum += other.commitNum; + } + + TableChange copy() { + return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFileNum", dataFileNum) + .add("deleteFileNum", deleteFileNum) + .add("dataFileSize", dataFileSize) + .add("deleteFileSize", deleteFileSize) + .add("commitNum", commitNum) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + TableChange that = (TableChange) other; + return dataFileNum == that.dataFileNum + && deleteFileNum == that.deleteFileNum + && dataFileSize == that.dataFileSize + && deleteFileSize == that.deleteFileSize + && commitNum == that.commitNum; + } + + @Override + public int hashCode() { + return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + } +} 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 new file mode 100644 index 000000000000..a49459d61a54 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.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.maintenance.operator; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Sink for collecting output during testing. */ +class CollectingSink implements Sink { + private static final long serialVersionUID = 1L; + private static final List> queues = + Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); + private static final AtomicInteger numSinks = new AtomicInteger(-1); + private final int index; + + /** Creates a new sink which collects the elements received. */ + CollectingSink() { + this.index = numSinks.incrementAndGet(); + queues.add(new LinkedBlockingQueue<>()); + } + + /** + * Gets all the remaining output received by this {@link Sink}. + * + * @return all the remaining output + */ + List remainingOutput() { + return Lists.newArrayList((BlockingQueue) queues.get(this.index)); + } + + /** + * Check if there is no remaining output received by this {@link Sink}. + * + * @return true if there is no remaining output + */ + boolean isEmpty() { + return queues.get(this.index).isEmpty(); + } + + /** + * Wait until the next element received by the {@link Sink}. + * + * @param timeout for the poll + * @return The first element received by this {@link Sink} + * @throws TimeoutException if no element received until the timeout + */ + T poll(Duration timeout) throws TimeoutException { + Object element; + + try { + element = queues.get(this.index).poll(timeout.toMillis(), TimeUnit.MILLISECONDS); + } catch (InterruptedException var4) { + throw new RuntimeException(var4); + } + + if (element == null) { + throw new TimeoutException(); + } else { + return (T) element; + } + } + + @Override + public SinkWriter createWriter(InitContext context) { + return new CollectingWriter<>(index); + } + + private static class CollectingWriter implements SinkWriter { + private final int index; + + CollectingWriter(int index) { + this.index = index; + } + + @Override + public void write(T element, Context context) { + queues.get(index).add(element); + } + + @Override + public void flush(boolean endOfInput) { + // Nothing to do here + } + + @Override + public void close() { + // Nothing to do here + } + } +} 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 new file mode 100644 index 000000000000..90790b373d5f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java @@ -0,0 +1,132 @@ +/* + * 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 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); + Files.walk(warehouse).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 new file mode 100644 index 000000000000..9cdc55cb0cce --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java @@ -0,0 +1,73 @@ +/* + * 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 new file mode 100644 index 000000000000..e08742a89ddb --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -0,0 +1,316 @@ +/* + * 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.ArrayDeque; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.jetbrains.annotations.Nullable; + +/** Testing source implementation for Flink sources which can be triggered manually. */ +class ManualSource + implements Source, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + private static final List>> queues = + Collections.synchronizedList(Lists.newArrayList()); + private static final List> availabilities = + Collections.synchronizedList(Lists.newArrayList()); + private static int numSources = 0; + private final TypeInformation type; + private final int index; + private transient DataStream stream; + private final transient StreamExecutionEnvironment env; + + /** + * Creates a new source for testing. + * + * @param env to register the source + * @param type of the events returned by the source + */ + ManualSource(StreamExecutionEnvironment env, TypeInformation type) { + this.type = type; + this.env = env; + this.index = numSources++; + queues.add(Queues.newArrayDeque()); + availabilities.add(new CompletableFuture<>()); + } + + /** + * Emit a new record from the source. + * + * @param event to emit + */ + void sendRecord(T event) { + this.sendInternal(Tuple2.of(event, null)); + } + + /** + * Emit a new record with the given event time from the source. + * + * @param event to emit + * @param eventTime of the event + */ + void sendRecord(T event, long eventTime) { + this.sendInternal(Tuple2.of(event, eventTime)); + } + + /** + * Emit a watermark from the source. + * + * @param timeStamp of the watermark + */ + void sendWatermark(long timeStamp) { + this.sendInternal(Tuple2.of(null, timeStamp)); + } + + /** Mark the source as finished. */ + void markFinished() { + this.sendWatermark(Long.MAX_VALUE); + this.sendInternal(Tuple2.of(null, null)); + } + + /** + * Get the {@link DataStream} for this source. + * + * @return the stream emitted by this source + */ + DataStream dataStream() { + if (this.stream == null) { + this.stream = + this.env + .fromSource(this, WatermarkStrategy.noWatermarks(), "ManualSource-" + index, type) + .forceNonParallel(); + } + + return this.stream; + } + + private void sendInternal(Tuple2 tuple) { + queues.get(index).offer(tuple); + availabilities.get(index).complete(null); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new DummyCheckpointEnumerator(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, DummyCheckpoint checkpoint) { + return new DummyCheckpointEnumerator(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new NoOpDummySplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new NoOpDummyCheckpointSerializer(); + } + + @Override + public SourceReader createReader(SourceReaderContext sourceReaderContext) { + return new SourceReader() { + @Override + public void start() { + // Do nothing + } + + @Override + public InputStatus pollNext(ReaderOutput output) { + Tuple2 next = (Tuple2) queues.get(index).poll(); + + if (next != null) { + if (next.f0 == null) { + // No more input + return InputStatus.END_OF_INPUT; + } + + if (next.f1 == null) { + // No event time set + output.collect(next.f0); + } else { + // With event time + output.collect(next.f0, next.f1); + } + } + + availabilities.set(index, new CompletableFuture<>()); + return queues.get(index).isEmpty() + ? InputStatus.NOTHING_AVAILABLE + : InputStatus.MORE_AVAILABLE; + } + + @Override + public List snapshotState(long checkpointId) { + return Lists.newArrayList(new DummySplit()); + } + + @Override + public CompletableFuture isAvailable() { + return availabilities.get(index); + } + + @Override + public void addSplits(List splits) { + // do nothing + } + + @Override + public void notifyNoMoreSplits() { + // do nothing + } + + @Override + public void close() { + // do nothing + } + }; + } + + @Override + public TypeInformation getProducedType() { + return this.type; + } + + /** + * Placeholder because the ManualSource itself implicitly represents the only split and does not + * require an actual split object. + */ + public static class DummySplit implements SourceSplit { + @Override + public String splitId() { + return "dummy"; + } + } + + /** + * Placeholder because the ManualSource does not support fault-tolerance and thus does not require + * actual checkpointing. + */ + public static class DummyCheckpoint {} + + /** Placeholder because the ManualSource does not need enumeration, but checkpointing needs it. */ + private static class DummyCheckpointEnumerator + implements SplitEnumerator { + + @Override + public void start() { + // do nothing + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // do nothing + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + // do nothing + } + + @Override + public void addReader(int subtaskId) { + // do nothing + } + + @Override + public DummyCheckpoint snapshotState(long checkpointId) { + return new DummyCheckpoint(); + } + + @Override + public void close() { + // do nothing + } + } + + /** + * Not used - only required to avoid NullPointerException. The split is not transferred from the + * enumerator, it is implicitly represented by the ManualSource. + */ + private static class NoOpDummySplitSerializer implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(DummySplit split) { + return new byte[0]; + } + + @Override + public DummySplit deserialize(int version, byte[] serialized) { + return new DummySplit(); + } + } + + /** + * Not used - only required to avoid NullPointerException. The split is not transferred from the + * enumerator, it is implicitly represented by the ManualSource. + */ + private static class NoOpDummyCheckpointSerializer + implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(DummyCheckpoint split) { + return new byte[0]; + } + + @Override + public DummyCheckpoint deserialize(int version, byte[] serialized) { + return new DummyCheckpoint(); + } + } +} 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 new file mode 100644 index 000000000000..269ae681b02e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -0,0 +1,51 @@ +/* + * 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.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.extension.RegisterExtension; + +class OperatorTestBase { + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + static final String TABLE_NAME = "test_table"; + + @RegisterExtension + protected static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .build()); + + @RegisterExtension + final FlinkSqlExtension sql = + new FlinkSqlExtension( + "catalog", + ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), + "db"); +} 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 new file mode 100644 index 000000000000..876d64214560 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -0,0 +1,362 @@ +/* + * 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.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.RewriteFiles; +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.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +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; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = TableChange.empty(); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @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(); + + MonitorSource.TableChangeIterator iterator = + 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(); + 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(); + expected = tableChangeWithLastSnapshot(table, TableChange.empty()); + + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + table.refresh(); + expected = tableChangeWithLastSnapshot(table, expected); + + assertThat(iterator.next()).isEqualTo(expected); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + /** + * Create a table and check that the source returns the data as new commits arrive to the table. + */ + @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(); + DataStream events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink result = new CollectingSink<>(); + events.sinkTo(result); + + JobClient jobClient = null; + try { + // First result is an empty event + jobClient = env.executeAsync("Table Change Source Test"); + assertThat(result.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + + // Insert some data + File dataDir = new File(new Path(table.location(), "data").toUri().getPath()); + dataDir.mkdir(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(table, FileFormat.PARQUET, dataDir.toPath()); + List batch1 = RandomGenericData.generate(table.schema(), 2, 1); + dataAppender.appendToTable(batch1); + + // Wait until the changes are committed + Awaitility.await() + .until( + () -> { + table.refresh(); + return table.currentSnapshot() != null; + }); + + table.refresh(); + long size = firstFileLength(table); + + // Wait until the first non-empty event has arrived, and check the expected result + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + }); + } finally { + closeJobClient(jobClient); + } + } + + /** 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); + + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + 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), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink result = new CollectingSink<>(); + events.sinkTo(result); + + // Start the job + Configuration conf; + JobClient jobClient = null; + AtomicReference firstNonEmptyEvent = new AtomicReference<>(); + try { + jobClient = env.executeAsync("Table Change Source Test"); + + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + firstNonEmptyEvent.set(newEvent); + return true; + }); + } finally { + // Stop with savepoint + conf = closeJobClient(jobClient, savepointDir); + } + + // Restore from savepoint, create the same topology with a different env + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink resultWithSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithSavepoint); + + // Make sure that the job with restored source does not read new records from the table + JobClient clientWithSavepoint = null; + try { + clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); + + assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + } finally { + closeJobClient(clientWithSavepoint, null); + } + + // Restore without savepoint + env = StreamExecutionEnvironment.getExecutionEnvironment(); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink resultWithoutSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithoutSavepoint); + + // Make sure that a new job without state reads the event as expected + JobClient clientWithoutSavepoint = null; + try { + clientWithoutSavepoint = env.executeAsync("Table Change Source Test without savepoint"); + assertThat(resultWithoutSavepoint.poll(Duration.ofSeconds(5L))) + .isEqualTo(firstNonEmptyEvent.get()); + } finally { + closeJobClient(clientWithoutSavepoint); + } + } + + @Test + void testNotOneParallelismThrows() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .setParallelism(2) + .print(); + + assertThatThrownBy(env::execute) + .isInstanceOf(JobExecutionException.class) + .rootCause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be set to 1"); + } + + @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); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + MonitorSource.TableChangeIterator iterator = + new MonitorSource.TableChangeIterator(tableLoader, null, 1); + + // For a single maxReadBack we only get a single change + assertThat(iterator.next().commitNum()).isEqualTo(1); + + iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); + + // Expecting 2 commits/snapshots for maxReadBack=2 + assertThat(iterator.next().commitNum()).isEqualTo(2); + + iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + + // For maxReadBack Long.MAX_VALUE we get every change + assertThat(iterator.next().commitNum()).isEqualTo(3); + } + + @Test + void testSkipReplace() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + MonitorSource.TableChangeIterator iterator = + new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + + // Read the current snapshot + assertThat(iterator.next().commitNum()).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(); + // Replace the file with itself for testing purposes + rewrite.deleteFile(dataFile); + rewrite.addFile(dataFile); + rewrite.commit(); + + // Check that the rewrite is ignored + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + private static long firstFileLength(Table table) { + return table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + } + + private static TableChange tableChangeWithLastSnapshot(Table table, TableChange previous) { + List dataFiles = + Lists.newArrayList(table.currentSnapshot().addedDataFiles(table.io()).iterator()); + List deleteFiles = + Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); + + long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); + long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); + boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); + + return new TableChange( + previous.dataFileNum() + dataFiles.size(), + previous.deleteFileNum() + deleteFiles.size(), + previous.dataFileSize() + dataSize, + previous.deleteFileSize() + deleteSize, + previous.commitNum() + 1); + } +} From 3c002a0719c3309e825fe8e0f840edbbdf2eb944 Mon Sep 17 00:00:00 2001 From: Anurag Mantripragada Date: Thu, 6 Jun 2024 16:12:05 -0700 Subject: [PATCH 0355/1019] Open-API: Use union instead of inheritance for TableRequirements (#10434) --- open-api/rest-catalog-open-api.py | 43 +++++++++++++++++------------ open-api/rest-catalog-open-api.yaml | 43 +++++++++-------------------- 2 files changed, 39 insertions(+), 47 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index e13aeeeec5bf..c5af1940c324 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -360,11 +360,7 @@ class RemovePartitionStatisticsUpdate(BaseUpdate): snapshot_id: int = Field(..., alias='snapshot-id') -class TableRequirement(BaseModel): - type: str - - -class AssertCreate(TableRequirement): +class AssertCreate(BaseModel): """ The table must not already exist; used for create transactions """ @@ -372,7 +368,7 @@ class AssertCreate(TableRequirement): type: Literal['assert-create'] -class AssertTableUUID(TableRequirement): +class AssertTableUUID(BaseModel): """ The table UUID must match the requirement's `uuid` """ @@ -381,7 +377,7 @@ class AssertTableUUID(TableRequirement): uuid: str -class AssertRefSnapshotId(TableRequirement): +class AssertRefSnapshotId(BaseModel): """ The table branch or tag identified by the requirement's `ref` must reference the requirement's `snapshot-id`; if `snapshot-id` is `null` or missing, the ref must not already exist """ @@ -391,7 +387,7 @@ class AssertRefSnapshotId(TableRequirement): snapshot_id: int = Field(..., alias='snapshot-id') -class AssertLastAssignedFieldId(TableRequirement): +class AssertLastAssignedFieldId(BaseModel): """ The table's last assigned column id must match the requirement's `last-assigned-field-id` """ @@ -400,7 +396,7 @@ class AssertLastAssignedFieldId(TableRequirement): last_assigned_field_id: int = Field(..., alias='last-assigned-field-id') -class AssertCurrentSchemaId(TableRequirement): +class AssertCurrentSchemaId(BaseModel): """ The table's current schema id must match the requirement's `current-schema-id` """ @@ -409,7 +405,7 @@ class AssertCurrentSchemaId(TableRequirement): current_schema_id: int = Field(..., alias='current-schema-id') -class AssertLastAssignedPartitionId(TableRequirement): +class AssertLastAssignedPartitionId(BaseModel): """ The table's last assigned partition id must match the requirement's `last-assigned-partition-id` """ @@ -418,7 +414,7 @@ class AssertLastAssignedPartitionId(TableRequirement): last_assigned_partition_id: int = Field(..., alias='last-assigned-partition-id') -class AssertDefaultSpecId(TableRequirement): +class AssertDefaultSpecId(BaseModel): """ The table's default spec id must match the requirement's `default-spec-id` """ @@ -427,7 +423,7 @@ class AssertDefaultSpecId(TableRequirement): default_spec_id: int = Field(..., alias='default-spec-id') -class AssertDefaultSortOrderId(TableRequirement): +class AssertDefaultSortOrderId(BaseModel): """ The table's default sort order id must match the requirement's `default-sort-order-id` """ @@ -436,11 +432,7 @@ class AssertDefaultSortOrderId(TableRequirement): default_sort_order_id: int = Field(..., alias='default-sort-order-id') -class ViewRequirement(BaseModel): - type: str - - -class AssertViewUUID(ViewRequirement): +class AssertViewUUID(BaseModel): """ The view UUID must match the requirement's `uuid` """ @@ -843,6 +835,23 @@ class SetPartitionStatisticsUpdate(BaseUpdate): ) +class TableRequirement(BaseModel): + __root__: Union[ + AssertCreate, + AssertTableUUID, + AssertRefSnapshotId, + AssertLastAssignedFieldId, + AssertCurrentSchemaId, + AssertLastAssignedPartitionId, + AssertDefaultSpecId, + AssertDefaultSortOrderId, + ] = Field(..., discriminator='type') + + +class ViewRequirement(BaseModel): + __root__: AssertViewUUID = Field(..., discriminator='type') + + class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 4883af09d581..4bb73cd44120 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2586,6 +2586,7 @@ components: - $ref: '#/components/schemas/SetCurrentViewVersionUpdate' TableRequirement: + type: object discriminator: propertyName: type mapping: @@ -2597,16 +2598,17 @@ components: assert-last-assigned-partition-id: '#/components/schemas/AssertLastAssignedPartitionId' assert-default-spec-id: '#/components/schemas/AssertDefaultSpecId' assert-default-sort-order-id: '#/components/schemas/AssertDefaultSortOrderId' - type: object - required: - - type - properties: - type: - type: "string" + oneOf: + - $ref: '#/components/schemas/AssertCreate' + - $ref: '#/components/schemas/AssertTableUUID' + - $ref: '#/components/schemas/AssertRefSnapshotId' + - $ref: '#/components/schemas/AssertLastAssignedFieldId' + - $ref: '#/components/schemas/AssertCurrentSchemaId' + - $ref: '#/components/schemas/AssertLastAssignedPartitionId' + - $ref: '#/components/schemas/AssertDefaultSpecId' + - $ref: '#/components/schemas/AssertDefaultSortOrderId' AssertCreate: - allOf: - - $ref: "#/components/schemas/TableRequirement" type: object description: The table must not already exist; used for create transactions required: @@ -2617,8 +2619,6 @@ components: enum: ["assert-create"] AssertTableUUID: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table UUID must match the requirement's `uuid` required: - type @@ -2631,8 +2631,6 @@ components: type: string AssertRefSnapshotId: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table branch or tag identified by the requirement's `ref` must reference the requirement's `snapshot-id`; if `snapshot-id` is `null` or missing, the ref must not already exist @@ -2651,8 +2649,6 @@ components: format: int64 AssertLastAssignedFieldId: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table's last assigned column id must match the requirement's `last-assigned-field-id` required: @@ -2666,8 +2662,6 @@ components: type: integer AssertCurrentSchemaId: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table's current schema id must match the requirement's `current-schema-id` required: @@ -2681,8 +2675,6 @@ components: type: integer AssertLastAssignedPartitionId: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table's last assigned partition id must match the requirement's `last-assigned-partition-id` required: @@ -2696,8 +2688,6 @@ components: type: integer AssertDefaultSpecId: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table's default spec id must match the requirement's `default-spec-id` required: @@ -2711,8 +2701,6 @@ components: type: integer AssertDefaultSortOrderId: - allOf: - - $ref: "#/components/schemas/TableRequirement" description: The table's default sort order id must match the requirement's `default-sort-order-id` required: @@ -2726,20 +2714,15 @@ components: type: integer ViewRequirement: + type: object discriminator: propertyName: type mapping: assert-view-uuid: '#/components/schemas/AssertViewUUID' - type: object - required: - - type - properties: - type: - type: "string" + oneOf: + - $ref: '#/components/schemas/AssertViewUUID' AssertViewUUID: - allOf: - - $ref: "#/components/schemas/ViewRequirement" description: The view UUID must match the requirement's `uuid` required: - type From 9d50b11bf1c5fb34f14ed10462a002ae1aa087d7 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Fri, 7 Jun 2024 06:09:06 +0530 Subject: [PATCH 0356/1019] Core: Reword exception message in RewriteManifests validation (#10446) --- .../java/org/apache/iceberg/BaseRewriteManifests.java | 9 ++++++--- .../java/org/apache/iceberg/TestRewriteManifests.java | 10 ++++++++-- 2 files changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index e8fbfef2ca1a..f3f8e5fcd7cb 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -171,7 +171,7 @@ public List apply(TableMetadata base, Snapshot snapshot) { List currentManifests = base.currentSnapshot().allManifests(ops.io()); Set currentManifestSet = ImmutableSet.copyOf(currentManifests); - validateDeletedManifests(currentManifestSet); + validateDeletedManifests(currentManifestSet, base.currentSnapshot().snapshotId()); if (requiresRewrite(currentManifestSet)) { performRewrite(currentManifests); @@ -275,14 +275,17 @@ private boolean matchesPredicate(ManifestFile manifest) { return predicate == null || predicate.test(manifest); } - private void validateDeletedManifests(Set currentManifests) { + private void validateDeletedManifests( + Set currentManifests, long currentSnapshotID) { // directly deleted manifests must be still present in the current snapshot deletedManifests.stream() .filter(manifest -> !currentManifests.contains(manifest)) .findAny() .ifPresent( manifest -> { - throw new ValidationException("Manifest is missing: %s", manifest.path()); + throw new ValidationException( + "Deleted manifest %s could not be found in the latest snapshot %d", + manifest.path(), currentSnapshotID); }); } diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index ef42fc1793da..176f61079fc3 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -810,7 +810,10 @@ public void testManifestReplacementConcurrentConflictingDelete() throws IOExcept assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(ValidationException.class) - .hasMessageStartingWith("Manifest is missing"); + .hasMessageStartingWith( + String.format( + "Deleted manifest %s could not be found in the latest snapshot %d", + firstSnapshotManifest.path(), table.currentSnapshot().snapshotId())); } @TestTemplate @@ -1604,7 +1607,10 @@ public void testDeleteManifestReplacementConflictingDeleteFileRemoval() throws I // the rewrite must fail as the original delete manifest was replaced concurrently assertThatThrownBy(rewriteManifests::commit) .isInstanceOf(ValidationException.class) - .hasMessageStartingWith("Manifest is missing"); + .hasMessageStartingWith( + String.format( + "Deleted manifest %s could not be found in the latest snapshot %d", + originalDeleteManifest.path(), table.currentSnapshot().snapshotId())); } @TestTemplate From 91b028fee304619360fb432719b43f483e37970a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 9 Jun 2024 19:17:46 +0200 Subject: [PATCH 0357/1019] Build: Bump com.palantir.gradle.gitversion:gradle-git-version (#10468) Bumps [com.palantir.gradle.gitversion:gradle-git-version](https://github.com/palantir/gradle-git-version) from 3.0.0 to 3.1.0. - [Release notes](https://github.com/palantir/gradle-git-version/releases) - [Changelog](https://github.com/palantir/gradle-git-version/blob/develop/.changelog.yml) - [Commits](https://github.com/palantir/gradle-git-version/compare/3.0.0...3.1.0) --- updated-dependencies: - dependency-name: com.palantir.gradle.gitversion:gradle-git-version 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 314393dfd902..213109852ec4 100644 --- a/build.gradle +++ b/build.gradle @@ -40,7 +40,7 @@ buildscript { classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.5.0" classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.2' - classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.0.0' + classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.1.0' classpath 'org.openapitools:openapi-generator-gradle-plugin:6.6.0' } } From fa8e3dec5a0b2d8d0e1c6a652f9bf8306956865d Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Sun, 9 Jun 2024 20:56:32 +0200 Subject: [PATCH 0358/1019] Build: Remove links checker (#10404) Running the links checker from the CI turns out the be very flaky: https://github.com/apache/iceberg/pull/10397 Let's remove it for now: cc @manuzhang @pvary --- .github/workflows/docs-check-links.yml | 43 -------------------------- site/link-checker-config.json | 26 ---------------- 2 files changed, 69 deletions(-) delete mode 100644 .github/workflows/docs-check-links.yml delete mode 100644 site/link-checker-config.json diff --git a/.github/workflows/docs-check-links.yml b/.github/workflows/docs-check-links.yml deleted file mode 100644 index 4275a2d6efc4..000000000000 --- a/.github/workflows/docs-check-links.yml +++ /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. -# - -name: Check Markdown docs links - -on: - push: - paths: - - docs/** - - site/** - branches: - - 'main' - pull_request: - paths: - - docs/** - - site/** - workflow_dispatch: - -jobs: - markdown-link-check: - runs-on: ubuntu-latest - steps: - - uses: actions/checkout@v4 - - uses: gaurav-nelson/github-action-markdown-link-check@v1 - with: - config-file: 'site/link-checker-config.json' - use-verbose-mode: yes diff --git a/site/link-checker-config.json b/site/link-checker-config.json deleted file mode 100644 index 54ff94c0145c..000000000000 --- a/site/link-checker-config.json +++ /dev/null @@ -1,26 +0,0 @@ -{ - "ignorePatterns": [ - { - "pattern": "^https://www.linkedin.com/" - }, - { - "pattern": "^https://mvnrepository.com/" - }, - { - "pattern": "^../../javadoc" - }, - { - "pattern": "^https://search.maven.org/" - } - ], - "replacementPatterns": [ - { - "pattern": "^docs/latest/", - "replacement": "{{BASEURL}}/docs/docs/" - }, - { - "pattern": "^../../", - "replacement": "{{BASEURL}}/site/docs/" - } - ] -} \ No newline at end of file From 250c40a4a13e3095cfc3b496d2d2391a4b2c2bbf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 10 Jun 2024 07:34:28 +0200 Subject: [PATCH 0359/1019] Build: Bump software.amazon.awssdk:bom from 2.25.64 to 2.25.69 (#10466) --- 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 a5875a75ecf6..8af794c8b408 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.25.64" +awssdk-bom = "2.25.69" azuresdk-bom = "1.2.23" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From bf790bb021abe4a1ce8c590b25f1c7db62357bc0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 11 Jun 2024 03:36:47 -0700 Subject: [PATCH 0360/1019] Build: Bump com.azure:azure-sdk-bom from 1.2.23 to 1.2.24 (#10420) Bumps [com.azure:azure-sdk-bom](https://github.com/azure/azure-sdk-for-java) from 1.2.23 to 1.2.24. - [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.23...azure-sdk-bom_1.2.24) --- 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 8af794c8b408..564fc773b2ce 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" awssdk-bom = "2.25.69" -azuresdk-bom = "1.2.23" +azuresdk-bom = "1.2.24" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From 1ac53f427205353760f90a0fd3b99aba927da2cf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 11 Jun 2024 03:40:06 -0700 Subject: [PATCH 0361/1019] Build: Bump mkdocs-material from 9.5.25 to 9.5.26 (#10464) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.25 to 9.5.26. - [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.25...9.5.26) --- 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 ec9f4f07d3bf..963e97c866c9 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.25 +mkdocs-material==9.5.26 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 adace140de17dc0e0515e68322f4ad6c3154f53e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 11 Jun 2024 14:04:06 +0200 Subject: [PATCH 0362/1019] Pin 3rd party CI action version (#10481) GitHub allows to delete and re-publish a tag, so referencing 3rd party action by tag name should be discouraged. --- .github/workflows/spark-ci.yml | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 64469d3e65e4..ef028fcd7676 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -86,7 +86,7 @@ jobs: ~/.gradle/wrapper key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - - uses: jlumbroso/free-disk-space@v1.3.1 + - uses: jlumbroso/free-disk-space@54081f138730dfa15788a46383842cd2f914a1be # v1.3.1 with: tool-cache: false - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts @@ -119,7 +119,7 @@ jobs: ~/.gradle/wrapper key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - - uses: jlumbroso/free-disk-space@v1.3.1 + - uses: jlumbroso/free-disk-space@54081f138730dfa15788a46383842cd2f914a1be # v1.3.1 with: tool-cache: false - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts @@ -152,7 +152,7 @@ jobs: ~/.gradle/wrapper key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - - uses: jlumbroso/free-disk-space@v1.3.1 + - uses: jlumbroso/free-disk-space@54081f138730dfa15788a46383842cd2f914a1be # v1.3.1 with: tool-cache: false - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts From 1a843093a58ac9db0519c025f375afb7e43b1471 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 12 Jun 2024 06:14:20 -0700 Subject: [PATCH 0363/1019] Build: Bump io.delta:delta-standalone_2.12 from 3.1.0 to 3.2.0 (#10321) Bumps [io.delta:delta-standalone_2.12](https://github.com/delta-io/delta) from 3.1.0 to 3.2.0. - [Release notes](https://github.com/delta-io/delta/releases) - [Commits](https://github.com/delta-io/delta/compare/v3.1.0...v3.2.0) --- updated-dependencies: - dependency-name: io.delta:delta-standalone_2.12 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 564fc773b2ce..21c4c57b07e1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -34,7 +34,7 @@ awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" datasketches = "6.0.0" -delta-standalone = "3.1.0" +delta-standalone = "3.2.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.28.0" From 839d8bbf4d3fdc31ccdc9c4476f65447f142662f Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Wed, 12 Jun 2024 16:50:17 -0700 Subject: [PATCH 0364/1019] Core, Spark: Calling rewrite_position_delete_files fails on tables with more than 1k columns (#10020) --- .palantir/revapi.yml | 6 ++ .../org/apache/iceberg/PartitionSpec.java | 26 +++++ .../main/java/org/apache/iceberg/Schema.java | 70 ++++++++++++- .../org/apache/iceberg/types/AssignIds.java | 99 +++++++++++++++++++ .../org/apache/iceberg/types/TypeUtil.java | 16 +++ .../java/org/apache/iceberg/types/Types.java | 4 + .../org/apache/iceberg/BaseMetadataTable.java | 5 +- .../org/apache/iceberg/ManifestReader.java | 2 +- .../apache/iceberg/PositionDeletesTable.java | 37 ++++++- .../iceberg/TestMetadataTableScans.java | 71 ++++++++++++- ...adataTableScansWithPartitionEvolution.java | 9 +- ...RewritePositionDeleteFilesSparkAction.java | 11 +-- .../TestRewritePositionDeleteFilesAction.java | 66 +++++++++++++ ...RewritePositionDeleteFilesSparkAction.java | 12 +-- .../TestRewritePositionDeleteFilesAction.java | 65 ++++++++++++ ...RewritePositionDeleteFilesSparkAction.java | 11 +-- .../TestRewritePositionDeleteFilesAction.java | 65 ++++++++++++ 17 files changed, 541 insertions(+), 34 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/types/AssignIds.java diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index a41d3ddfb8df..808a19299055 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1018,6 +1018,12 @@ acceptedBreaks: old: "method void org.apache.iceberg.PositionDeletesTable.PositionDeletesBatchScan::(org.apache.iceberg.Table,\ \ org.apache.iceberg.Schema, org.apache.iceberg.TableScanContext)" justification: "Removing deprecated code" + "1.5.0": + org.apache.iceberg:iceberg-api: + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.types.Types.NestedField" + new: "class org.apache.iceberg.types.Types.NestedField" + justification: "new Constructor added" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 4fcb110db87c..8f1df794030a 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -27,6 +27,7 @@ import java.util.Map; import java.util.Set; import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; 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.ImmutableList; @@ -60,6 +61,7 @@ public class PartitionSpec implements Serializable { private transient volatile ListMultimap fieldsBySourceId = null; private transient volatile Class[] lazyJavaClasses = null; private transient volatile StructType lazyPartitionType = null; + private transient volatile StructType lazyRawPartitionType = null; private transient volatile List fieldList = null; private final int lastAssignedFieldId; @@ -140,6 +142,30 @@ public StructType partitionType() { return lazyPartitionType; } + /** + * Returns a struct matching partition information as written into manifest files. See {@link + * #partitionType()} for a struct with field ID's potentially re-assigned to avoid conflict. + */ + public StructType rawPartitionType() { + if (schema.idsToOriginal().isEmpty()) { + // not re-assigned. + return partitionType(); + } + if (lazyRawPartitionType == null) { + synchronized (this) { + if (lazyRawPartitionType == null) { + this.lazyRawPartitionType = + StructType.of( + partitionType().fields().stream() + .map(f -> f.withFieldId(schema.idsToOriginal().get(f.fieldId()))) + .collect(Collectors.toList())); + } + } + } + + return lazyRawPartitionType; + } + public Class[] javaClasses() { if (lazyJavaClasses == null) { synchronized (this) { diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 5e024b7c1c29..d5ec3f250982 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -21,6 +21,7 @@ import java.io.Serializable; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.Deque; import java.util.List; import java.util.Locale; @@ -34,6 +35,7 @@ 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.relocated.com.google.common.primitives.Ints; import org.apache.iceberg.types.Type; @@ -65,6 +67,8 @@ public class Schema implements Serializable { private transient Map> idToAccessor = null; private transient Map idToName = null; private transient Set identifierFieldIdSet = null; + private transient Map idsToReassigned; + private transient Map idsToOriginal; public Schema(List columns, Map aliases) { this(columns, aliases, ImmutableSet.of()); @@ -83,12 +87,24 @@ public Schema(List columns, Set identifierFieldIds) { this(DEFAULT_SCHEMA_ID, columns, identifierFieldIds); } + public Schema(List columns, Set identifierFieldIds, TypeUtil.GetID getId) { + this(DEFAULT_SCHEMA_ID, columns, identifierFieldIds, getId); + } + public Schema(int schemaId, List columns) { this(schemaId, columns, ImmutableSet.of()); } public Schema(int schemaId, List columns, Set identifierFieldIds) { - this(schemaId, columns, null, identifierFieldIds); + this(schemaId, columns, null, identifierFieldIds, null); + } + + public Schema( + int schemaId, + List columns, + Set identifierFieldIds, + TypeUtil.GetID getId) { + this(schemaId, columns, null, identifierFieldIds, getId); } public Schema( @@ -96,8 +112,22 @@ public Schema( List columns, Map aliases, Set identifierFieldIds) { + this(schemaId, columns, aliases, identifierFieldIds, null); + } + + public Schema( + int schemaId, + List columns, + Map aliases, + Set identifierFieldIds, + TypeUtil.GetID getID) { this.schemaId = schemaId; - this.struct = StructType.of(columns); + + this.idsToOriginal = Maps.newHashMap(); + this.idsToReassigned = Maps.newHashMap(); + List finalColumns = reassignIds(columns, getID); + + this.struct = StructType.of(finalColumns); this.aliasToId = aliases != null ? ImmutableBiMap.copyOf(aliases) : null; // validate IdentifierField @@ -507,4 +537,40 @@ public String toString() { .map(this::identifierFieldToString) .collect(Collectors.toList()))); } + + /** + * The ID's of some fields will be re-assigned if GetID is specified for the Schema. + * + * @return map of original to reassigned field ids + */ + public Map idsToReassigned() { + return idsToReassigned != null ? idsToReassigned : Collections.emptyMap(); + } + + /** + * The ID's of some fields will be re-assigned if GetID is specified for the Schema. + * + * @return map of reassigned to original field ids + */ + public Map idsToOriginal() { + return idsToOriginal != null ? idsToOriginal : Collections.emptyMap(); + } + + private List reassignIds(List columns, TypeUtil.GetID getID) { + if (getID == null) { + return columns; + } + Type res = + TypeUtil.assignIds( + StructType.of(columns), + oldId -> { + int newId = getID.get(oldId); + if (newId != oldId) { + idsToReassigned.put(oldId, newId); + idsToOriginal.put(newId, oldId); + } + return newId; + }); + return res.asStructType().fields(); + } } diff --git a/api/src/main/java/org/apache/iceberg/types/AssignIds.java b/api/src/main/java/org/apache/iceberg/types/AssignIds.java new file mode 100644 index 000000000000..68588f581adc --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/types/AssignIds.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.types; + +import java.util.Iterator; +import java.util.List; +import java.util.function.Supplier; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class AssignIds extends TypeUtil.CustomOrderSchemaVisitor { + private final TypeUtil.GetID getID; + + AssignIds(TypeUtil.GetID getID) { + this.getID = getID; + } + + private int idFor(int id) { + return getID.get(id); + } + + @Override + public Type schema(Schema schema, Supplier future) { + return future.get(); + } + + @Override + public Type struct(Types.StructType struct, Iterable futures) { + List fields = struct.fields(); + int length = struct.fields().size(); + + // assign IDs for this struct's fields first + List newIds = Lists.newArrayListWithExpectedSize(length); + for (Types.NestedField field : fields) { + newIds.add(idFor(field.fieldId())); + } + + List newFields = Lists.newArrayListWithExpectedSize(length); + Iterator types = futures.iterator(); + for (int i = 0; i < length; i += 1) { + Types.NestedField field = fields.get(i); + Type type = types.next(); + if (field.isOptional()) { + newFields.add(Types.NestedField.optional(newIds.get(i), field.name(), type, field.doc())); + } else { + newFields.add(Types.NestedField.required(newIds.get(i), field.name(), type, field.doc())); + } + } + + return Types.StructType.of(newFields); + } + + @Override + public Type field(Types.NestedField field, Supplier future) { + return future.get(); + } + + @Override + public Type list(Types.ListType list, Supplier future) { + int newId = idFor(list.elementId()); + if (list.isElementOptional()) { + return Types.ListType.ofOptional(newId, future.get()); + } else { + return Types.ListType.ofRequired(newId, future.get()); + } + } + + @Override + public Type map(Types.MapType map, Supplier keyFuture, Supplier valueFuture) { + int newKeyId = idFor(map.keyId()); + int newValueId = idFor(map.valueId()); + if (map.isValueOptional()) { + return Types.MapType.ofOptional(newKeyId, newValueId, keyFuture.get(), valueFuture.get()); + } else { + return Types.MapType.ofRequired(newKeyId, newValueId, keyFuture.get(), valueFuture.get()); + } + } + + @Override + public Type primitive(Type.PrimitiveType primitive) { + return primitive; + } +} 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 7c13d6094084..07d06dcc5a89 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -355,6 +355,17 @@ public static Schema reassignOrRefreshIds( return new Schema(struct.fields(), refreshIdentifierFields(struct, schema)); } + /** + * Assigns fresh ids from the {@link GetID getId function} for all fields in a type. + * + * @param type a type + * @param getId an id assignment function + * @return an structurally identical type with new ids assigned by the getId function + */ + public static Type assignIds(Type type, GetID getId) { + return TypeUtil.visit(type, new AssignIds(getId)); + } + public static Type find(Schema schema, Predicate predicate) { return visit(schema, new FindTypeVisitor(predicate)); } @@ -521,6 +532,11 @@ public interface NextID { int get(); } + /** Interface for passing a function that assigns column IDs from the previous Id. */ + public interface GetID { + int get(int oldId); + } + public static class SchemaVisitor { public void beforeField(Types.NestedField field) {} 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 dda842c9e161..ce6caa4721df 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -475,6 +475,10 @@ public NestedField asRequired() { return new NestedField(false, id, name, type, doc); } + public NestedField withFieldId(int newId) { + return new NestedField(isOptional, newId, name, type, doc); + } + public int fieldId() { return id; } diff --git a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java index 57a6386093d6..e1e138109f8e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetadataTable.java @@ -74,8 +74,11 @@ static PartitionSpec transformSpec(Schema metadataTableSchema, PartitionSpec spe .withSpecId(spec.specId()) .checkConflicts(false); + Map reassignedFields = metadataTableSchema.idsToReassigned(); + for (PartitionField field : spec.fields()) { - builder.add(field.fieldId(), field.fieldId(), field.name(), Transforms.identity()); + int newFieldId = reassignedFields.getOrDefault(field.fieldId(), field.fieldId()); + builder.add(newFieldId, newFieldId, field.name(), Transforms.identity()); } return builder.build(); } diff --git a/core/src/main/java/org/apache/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java index 4ee51aa60c31..b5f85813dd2f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -114,7 +114,7 @@ protected ManifestReader( this.spec = readPartitionSpec(file); } - this.fileSchema = new Schema(DataFile.getType(spec.partitionType()).fields()); + this.fileSchema = new Schema(DataFile.getType(spec.rawPartitionType()).fields()); } private > PartitionSpec readPartitionSpec(InputFile inputFile) { diff --git a/core/src/main/java/org/apache/iceberg/PositionDeletesTable.java b/core/src/main/java/org/apache/iceberg/PositionDeletesTable.java index 56270635809b..382ad663a8d1 100644 --- a/core/src/main/java/org/apache/iceberg/PositionDeletesTable.java +++ b/core/src/main/java/org/apache/iceberg/PositionDeletesTable.java @@ -24,6 +24,8 @@ import java.util.Collections; import java.util.List; import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Function; import java.util.stream.Collectors; import org.apache.iceberg.expressions.Expression; @@ -33,6 +35,8 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +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.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; @@ -107,8 +111,8 @@ public Map properties() { private Schema calculateSchema() { Types.StructType partitionType = Partitioning.partitionType(table()); - Schema result = - new Schema( + List columns = + ImmutableList.of( MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS, Types.NestedField.optional( @@ -132,6 +136,35 @@ private Schema calculateSchema() { Types.StringType.get(), MetadataColumns.FILE_PATH_COLUMN_DOC)); + // Calculate used ids (for de-conflict) + Set currentlyUsedIds = + Collections.unmodifiableSet(TypeUtil.indexById(Types.StructType.of(columns)).keySet()); + Set allUsedIds = + table().schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Calculate ids to reassign + Set idsToReassign = + partitionType.fields().stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + + // Reassign selected ids to de-conflict with used ids. + AtomicInteger nextId = new AtomicInteger(); + Schema result = + new Schema( + columns, + ImmutableSet.of(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); + if (!partitionType.fields().isEmpty()) { return result; } else { diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 4c5f1d240f57..df314f6a802f 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -42,6 +42,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterators; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeWrapper; import org.junit.jupiter.api.TestTemplate; @@ -1266,7 +1267,7 @@ public void testPositionDeletesWithFilter() { ScanTask task = tasks.get(0); assertThat(task).isInstanceOf(PositionDeletesScanTask.class); - Types.StructType partitionType = Partitioning.partitionType(table); + Types.StructType partitionType = positionDeletesTable.spec().partitionType(); PositionDeletesScanTask posDeleteTask = (PositionDeletesScanTask) task; int filePartition = posDeleteTask.file().partition().get(0, Integer.class); @@ -1333,7 +1334,7 @@ private void testPositionDeletesBaseTableFilter(boolean transactional) { ScanTask task = tasks.get(0); assertThat(task).isInstanceOf(PositionDeletesScanTask.class); - Types.StructType partitionType = Partitioning.partitionType(table); + Types.StructType partitionType = positionDeletesTable.spec().partitionType(); PositionDeletesScanTask posDeleteTask = (PositionDeletesScanTask) task; // base table filter should only be used to evaluate partitions @@ -1415,7 +1416,7 @@ public void testPositionDeletesWithBaseTableFilterNot() { ScanTask task = tasks.get(0); assertThat(task).isInstanceOf(PositionDeletesScanTask.class); - Types.StructType partitionType = Partitioning.partitionType(table); + Types.StructType partitionType = positionDeletesTable.spec().partitionType(); PositionDeletesScanTask posDeleteTask = (PositionDeletesScanTask) task; // base table filter should only be used to evaluate partitions @@ -1426,7 +1427,7 @@ public void testPositionDeletesWithBaseTableFilterNot() { (StructLike) constantsMap(posDeleteTask, partitionType).get(MetadataColumns.PARTITION_COLUMN_ID); int taskPartition = - taskPartitionStruct.get(1, Integer.class); // new partition field in position 1 + taskPartitionStruct.get(0, Integer.class); // new partition field in position 0 assertThat(filePartition).as("Expected correct partition on task's file").isEqualTo(1); assertThat(taskPartition).as("Expected correct partition on task's column").isEqualTo(1); @@ -1564,4 +1565,66 @@ public void testPositionDeletesUnpartitioned() { assertThat(scanTask1Partition).isEqualTo(expected); assertThat(scanTask2Partition).isEqualTo(expected); } + + @TestTemplate + public void testPositionDeletesManyColumns() { + assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + + UpdateSchema updateSchema = table.updateSchema(); + for (int i = 0; i <= 2000; i++) { + updateSchema.addColumn(String.valueOf(i), Types.IntegerType.get()); + } + updateSchema.commit(); + + DataFile dataFile1 = + DataFiles.builder(table.spec()) + .withPath("/path/to/data1.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + DataFile dataFile2 = + DataFiles.builder(table.spec()) + .withPath("/path/to/data2.parquet") + .withFileSizeInBytes(10) + .withRecordCount(1) + .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(); + table.newRowDelta().addDeletes(delete1).addDeletes(delete2).commit(); + + PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); + assertThat(TypeUtil.indexById(positionDeletesTable.schema().asStruct()).size()).isEqualTo(2010); + + BatchScan scan = positionDeletesTable.newBatchScan(); + assertThat(scan).isInstanceOf(PositionDeletesTable.PositionDeletesBatchScan.class); + PositionDeletesTable.PositionDeletesBatchScan deleteScan = + (PositionDeletesTable.PositionDeletesBatchScan) scan; + + List scanTasks = + Lists.newArrayList( + Iterators.transform( + deleteScan.planFiles().iterator(), + task -> { + assertThat(task).isInstanceOf(PositionDeletesScanTask.class); + 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"); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index faccdcb3dd95..a2e5386d29df 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -21,7 +21,6 @@ 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.entry; import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; @@ -186,7 +185,7 @@ public void testPositionDeletesPartitionSpecRemoval() { ScanTask task = tasks.get(0); assertThat(task).isInstanceOf(PositionDeletesScanTask.class); - Types.StructType partitionType = Partitioning.partitionType(table); + Types.StructType partitionType = positionDeletesTable.spec().partitionType(); PositionDeletesScanTask posDeleteTask = (PositionDeletesScanTask) task; int filePartition = posDeleteTask.file().partition().get(0, Integer.class); @@ -196,15 +195,13 @@ public void testPositionDeletesPartitionSpecRemoval() { int taskConstantPartition = ((StructLike) constantsMap(posDeleteTask, partitionType).get(MetadataColumns.PARTITION_COLUMN_ID)) - .get(1, Integer.class); + .get(0, Integer.class); assertThat(taskConstantPartition) .as("Expected correct partition on constant column") .isEqualTo(1); - assertThat(posDeleteTask.spec().fields().get(0).fieldId()) .as("Expected correct partition field id on task's spec") - .isEqualTo(table.ops().current().spec().partitionType().fields().get(0).fieldId()); - + .isEqualTo(partitionType.fields().get(0).fieldId()); assertThat(posDeleteTask.file().specId()) .as("Expected correct partition spec id on task") .isEqualTo(table.ops().current().spec().specId()); 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 f3dfd2dcc364..ea1c52940175 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 @@ -137,10 +137,12 @@ public RewritePositionDeleteFiles.Result execute() { } private StructLikeMap>> planFileGroups() { - CloseableIterable fileTasks = planFiles(); + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable fileTasks = planFiles(deletesTable); try { - StructType partitionType = Partitioning.partitionType(table); + StructType partitionType = Partitioning.partitionType(deletesTable); StructLikeMap> fileTasksByPartition = groupByPartition(partitionType, fileTasks); return fileGroupsByPartition(fileTasksByPartition); @@ -153,10 +155,7 @@ private StructLikeMap>> planFileGroups() { } } - private CloseableIterable planFiles() { - Table deletesTable = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); - + private CloseableIterable planFiles(Table deletesTable) { PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); return CloseableIterable.transform( scan.baseTableFilter(filter).ignoreResiduals().planFiles(), 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 77800e2ea007..aa2817e8753d 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 @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.actions; 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 java.io.IOException; import java.util.Arrays; @@ -69,6 +71,7 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -581,6 +584,69 @@ public void testSchemaEvolution() throws Exception { assertEquals("Rows must match", expectedRecords, actualRecords); } + @Test + public void testRewriteManyColumns() throws Exception { + List fields = + Lists.newArrayList(Types.NestedField.optional(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema schema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 2).build(); + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), schema, spec, tableProperties()); + + Dataset df = + spark + .range(4) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(name(table)).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(name(table)); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 1, 1, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(2); + assertThat(expectedDeletes).hasSize(2); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).hasSize(2); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 2); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + private Table createTablePartitioned(int partitions, int files, int numRecords) { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); Table table = 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 f3dfd2dcc364..bdb0ee35273f 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 @@ -137,10 +137,12 @@ public RewritePositionDeleteFiles.Result execute() { } private StructLikeMap>> planFileGroups() { - CloseableIterable fileTasks = planFiles(); + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable fileTasks = planFiles(deletesTable); try { - StructType partitionType = Partitioning.partitionType(table); + StructType partitionType = Partitioning.partitionType(deletesTable); StructLikeMap> fileTasksByPartition = groupByPartition(partitionType, fileTasks); return fileGroupsByPartition(fileTasksByPartition); @@ -153,11 +155,9 @@ private StructLikeMap>> planFileGroups() { } } - private CloseableIterable planFiles() { - Table deletesTable = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); - + private CloseableIterable planFiles(Table deletesTable) { PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); + return CloseableIterable.transform( scan.baseTableFilter(filter).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 db2d42501d04..7be300e84fc6 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; 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 java.io.IOException; @@ -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.types.StructType; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -618,6 +620,69 @@ public void testSchemaEvolution() throws Exception { assertEquals("Rows must match", expectedRecords, actualRecords); } + @Test + public void testRewriteManyColumns() throws Exception { + List fields = + Lists.newArrayList(Types.NestedField.optional(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema schema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 2).build(); + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), schema, spec, tableProperties()); + + Dataset df = + spark + .range(4) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(name(table)).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(name(table)); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 1, 1, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(2); + assertThat(expectedDeletes).hasSize(2); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).hasSize(2); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 2); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + private Table createTablePartitioned(int partitions, int files, int numRecords) { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); Table table = 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 539f6de92007..1166740f441a 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 @@ -137,10 +137,12 @@ public RewritePositionDeleteFiles.Result execute() { } private StructLikeMap>> planFileGroups() { - CloseableIterable fileTasks = planFiles(); + Table deletesTable = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); + CloseableIterable fileTasks = planFiles(deletesTable); try { - StructType partitionType = Partitioning.partitionType(table); + StructType partitionType = Partitioning.partitionType(deletesTable); StructLikeMap> fileTasksByPartition = groupByPartition(partitionType, fileTasks); return fileGroupsByPartition(fileTasksByPartition); @@ -153,10 +155,7 @@ private StructLikeMap>> planFileGroups() { } } - private CloseableIterable planFiles() { - Table deletesTable = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.POSITION_DELETES); - + private CloseableIterable planFiles(Table deletesTable) { PositionDeletesBatchScan scan = (PositionDeletesBatchScan) deletesTable.newBatchScan(); return CloseableIterable.transform( scan.baseTableFilter(filter).ignoreResiduals().planFiles(), 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 89c44dbfccf8..37b6cd86fb92 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; 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.fail; @@ -77,6 +78,7 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; +import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.io.TempDir; @@ -647,6 +649,69 @@ public void testSnapshotProperty() throws Exception { assertThat(table.currentSnapshot().summary()).containsKeys(commitMetricsKeys); } + @TestTemplate + public void testRewriteManyColumns() throws Exception { + List fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema schema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("id", 2).build(); + Table table = + validationCatalog.createTable( + TableIdentifier.of("default", TABLE_NAME), schema, spec, tableProperties()); + + Dataset df = + spark + .range(4) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(name(table)).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(name(table)); + + List dataFiles = TestHelpers.dataFiles(table); + writePosDeletesForFiles(table, 1, 1, dataFiles); + assertThat(dataFiles).hasSize(2); + + List deleteFiles = deleteFiles(table); + assertThat(deleteFiles).hasSize(2); + + List expectedRecords = records(table); + List expectedDeletes = deleteRecords(table); + assertThat(expectedRecords).hasSize(2); + assertThat(expectedDeletes).hasSize(2); + + Result result = + SparkActions.get(spark) + .rewritePositionDeletes(table) + .option(SizeBasedFileRewriter.REWRITE_ALL, "true") + .option(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, Long.toString(Long.MAX_VALUE - 1)) + .execute(); + + List newDeleteFiles = deleteFiles(table); + assertThat(newDeleteFiles).hasSize(2); + assertNotContains(deleteFiles, newDeleteFiles); + assertLocallySorted(newDeleteFiles); + checkResult(result, deleteFiles, newDeleteFiles, 2); + checkSequenceNumbers(table, deleteFiles, newDeleteFiles); + + List actualRecords = records(table); + List actualDeletes = deleteRecords(table); + assertEquals("Rows must match", expectedRecords, actualRecords); + assertEquals("Position deletes must match", expectedDeletes, actualDeletes); + } + private Table createTablePartitioned(int partitions, int files, int numRecords) { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); Table table = From 8255366b865141f2bf7cc38ed76bb56cb614c984 Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Fri, 14 Jun 2024 11:55:54 +0200 Subject: [PATCH 0365/1019] Build: Bump Nessie to 0.90.4 (#10492) --- 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 21c4c57b07e1..dbada50333b0 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.83.2" +nessie = "0.90.4" netty-buffer = "4.1.110.Final" netty-buffer-compat = "4.1.110.Final" object-client-bundle = "3.3.2" From 271bf62db1d55512c321f8394c56195c01db23fa Mon Sep 17 00:00:00 2001 From: GYoung Date: Sat, 15 Jun 2024 04:04:21 +0800 Subject: [PATCH 0366/1019] Core: Simplify `loadCatalog` method call in Iceberg (#10488) Co-authored-by: howieyang --- core/src/main/java/org/apache/iceberg/CatalogUtil.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index 4846dbb8e9a9..d4fcbda0686d 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -307,7 +307,7 @@ public static Catalog buildIcebergCatalog(String name, Map optio catalogImpl); } - return CatalogUtil.loadCatalog(catalogImpl, name, options, conf); + return loadCatalog(catalogImpl, name, options, conf); } /** From de07b92dc25773334aaf580aad14309d7d2a595c Mon Sep 17 00:00:00 2001 From: GYoung Date: Sat, 15 Jun 2024 06:03:20 +0800 Subject: [PATCH 0367/1019] MR: Optimize schema string retrieval in Iceberg (#10489) Co-authored-by: howieyang --- mr/src/main/java/org/apache/iceberg/mr/Catalogs.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java b/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java index 75797580cd9b..e8a4b5da3583 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java +++ b/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java @@ -151,7 +151,7 @@ private static Table loadTable( public static Table createTable(Configuration conf, Properties props) { String schemaString = props.getProperty(InputFormatConfig.TABLE_SCHEMA); Preconditions.checkNotNull(schemaString, "Table schema not set"); - Schema schema = SchemaParser.fromJson(props.getProperty(InputFormatConfig.TABLE_SCHEMA)); + Schema schema = SchemaParser.fromJson(schemaString); String specString = props.getProperty(InputFormatConfig.PARTITION_SPEC); PartitionSpec spec = PartitionSpec.unpartitioned(); From c1427a7e29ad53d4152d9ee183f05bf284f23a91 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Sat, 15 Jun 2024 14:19:51 +0200 Subject: [PATCH 0368/1019] Build: Rename allVersions flag to allModules (#10499) The meaning of the flag is to enable all modules. In particular, it does not enable all Scala versions for the build. The new name more directly conveys the intent. --- .github/workflows/java-ci.yml | 2 +- .github/workflows/publish-snapshot.yml | 2 +- README.md | 2 +- settings.gradle | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 7effd4ab7f53..bc4b01ed2203 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -96,7 +96,7 @@ jobs: with: distribution: zulu java-version: 8 - - run: ./gradlew -DallVersions build -x test -x javadoc -x integrationTest + - run: ./gradlew -DallModules build -x test -x javadoc -x integrationTest build-javadoc: runs-on: ubuntu-22.04 diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 0d7147b24a67..411c3a5636c9 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -40,5 +40,5 @@ jobs: java-version: 8 - run: | ./gradlew printVersion - ./gradlew -DallVersions publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} + ./gradlew -DallModules publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} ./gradlew -DflinkVersions= -DsparkVersions=3.3,3.4,3.5 -DscalaVersion=2.13 -DhiveVersions= publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} diff --git a/README.md b/README.md index 8d36b212b953..8200a23ed63b 100644 --- a/README.md +++ b/README.md @@ -56,7 +56,7 @@ Iceberg is built using Gradle with Java 8, 11, or 17. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` * To fix code style for default versions: `./gradlew spotlessApply` -* To fix code style for all versions of Spark/Hive/Flink:`./gradlew spotlessApply -DallVersions` +* To fix code style for all versions of Spark/Hive/Flink:`./gradlew spotlessApply -DallModules` Iceberg table support is organized in library modules: diff --git a/settings.gradle b/settings.gradle index 4f42d24c32cf..2fa05fb46d0d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -70,7 +70,7 @@ project(':delta-lake').name = 'iceberg-delta-lake' project(':kafka-connect').name = 'iceberg-kafka-connect' project(':open-api').name = 'iceberg-open-api' -if (null != System.getProperty("allVersions")) { +if (null != System.getProperty("allModules")) { System.setProperty("flinkVersions", System.getProperty("knownFlinkVersions")) System.setProperty("sparkVersions", System.getProperty("knownSparkVersions")) System.setProperty("hiveVersions", System.getProperty("knownHiveVersions")) From f63c381bd0e08de37b3a15546d9766c1fe4757dc Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 15 Jun 2024 21:10:05 +0200 Subject: [PATCH 0369/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.16.0 to 3.16.1 (#10419) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.16.0 to 3.16.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.16.0...v3.16.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 dbada50333b0..bba80617db67 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -77,7 +77,7 @@ roaringbitmap = "1.0.6" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.11.0" slf4j = "1.7.36" -snowflake-jdbc = "3.16.0" +snowflake-jdbc = "3.16.1" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" From 200a5e219b98d03915d5a916bb2b21e68df921d3 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 15 Jun 2024 21:10:25 +0200 Subject: [PATCH 0370/1019] Build: Bump org.scala-lang.modules:scala-collection-compat_2.13 (#10195) Bumps [org.scala-lang.modules:scala-collection-compat_2.13](https://github.com/scala/scala-collection-compat) from 2.11.0 to 2.12.0. - [Release notes](https://github.com/scala/scala-collection-compat/releases) - [Commits](https://github.com/scala/scala-collection-compat/compare/v2.11.0...v2.12.0) --- updated-dependencies: - dependency-name: org.scala-lang.modules:scala-collection-compat_2.13 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 bba80617db67..e741965626ed 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -75,7 +75,7 @@ parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.0.6" s3mock-junit5 = "2.11.0" -scala-collection-compat = "2.11.0" +scala-collection-compat = "2.12.0" slf4j = "1.7.36" snowflake-jdbc = "3.16.1" spark-hive33 = "3.3.4" From 6d21dc354473d58e5c805b2716efcfb9618f5824 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 16 Jun 2024 10:06:11 +0200 Subject: [PATCH 0371/1019] Build: Bump org.springframework:spring-web from 5.3.36 to 5.3.37 (#10503) Bumps [org.springframework:spring-web](https://github.com/spring-projects/spring-framework) from 5.3.36 to 5.3.37. - [Release notes](https://github.com/spring-projects/spring-framework/releases) - [Commits](https://github.com/spring-projects/spring-framework/compare/v5.3.36...v5.3.37) --- updated-dependencies: - dependency-name: org.springframework:spring-web 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 e741965626ed..7686dfef8aff 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -82,7 +82,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" -spring-web = "5.3.36" +spring-web = "5.3.37" sqlite-jdbc = "3.46.0.0" testcontainers = "1.19.8" tez010 = "0.10.3" From e8adfad570240670131407fd084c0422a64ca927 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 16 Jun 2024 10:06:23 +0200 Subject: [PATCH 0372/1019] Build: Bump io.netty:netty-buffer from 4.1.110.Final to 4.1.111.Final (#10504) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.110.Final to 4.1.111.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.110.Final...netty-4.1.111.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 7686dfef8aff..7324bc1c5146 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -67,8 +67,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.90.4" -netty-buffer = "4.1.110.Final" -netty-buffer-compat = "4.1.110.Final" +netty-buffer = "4.1.111.Final" +netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" From 1f80331f1db0239817f26264f048b1f50a1746c4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 16 Jun 2024 10:06:38 +0200 Subject: [PATCH 0373/1019] Build: Bump software.amazon.awssdk:bom from 2.25.69 to 2.26.3 (#10505) Bumps software.amazon.awssdk:bom from 2.25.69 to 2.26.3. --- 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 7324bc1c5146..b913be8f3b9c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.25.69" +awssdk-bom = "2.26.3" azuresdk-bom = "1.2.24" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 29e1b353adfb8bffe54b317e96ebe0e7d7c3bd10 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 16 Jun 2024 10:06:53 +0200 Subject: [PATCH 0374/1019] Build: Bump datamodel-code-generator from 0.25.6 to 0.25.7 (#10507) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.6 to 0.25.7. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.6...0.25.7) --- 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 4619cc35e336..75c821241f08 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.25.6 +datamodel-code-generator==0.25.7 From ee04b0b630a26020c0ad698b36dd6955774658c4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 17 Jun 2024 08:40:04 +0200 Subject: [PATCH 0375/1019] Core: Remove deprecated APIs scheduled for removal in 1.6.0 (#10501) --- .palantir/revapi.yml | 32 +++++++++++++++++++ .../java/org/apache/iceberg/DataFiles.java | 6 ---- .../org/apache/iceberg/ReachableFileUtil.java | 19 ----------- .../PlaintextEncryptionManager.java | 4 +-- .../apache/iceberg/rest/auth/OAuth2Util.java | 18 ----------- .../flink/source/reader/ReaderUtil.java | 2 +- .../reader/TestIcebergSourceReader.java | 2 +- .../reader/TestRowDataReaderFunction.java | 2 +- .../flink/source/reader/ReaderUtil.java | 2 +- .../reader/TestIcebergSourceReader.java | 2 +- .../reader/TestRowDataReaderFunction.java | 2 +- .../flink/source/reader/ReaderUtil.java | 2 +- .../reader/TestIcebergSourceReader.java | 2 +- .../reader/TestRowDataReaderFunction.java | 2 +- .../iceberg/nessie/NessieIcebergClient.java | 13 -------- .../apache/iceberg/spark/SparkTableUtil.java | 12 ------- 16 files changed, 42 insertions(+), 80 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 808a19299055..3018840b4513 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1024,6 +1024,38 @@ acceptedBreaks: old: "class org.apache.iceberg.types.Types.NestedField" new: "class org.apache.iceberg.types.Types.NestedField" justification: "new Constructor added" + org.apache.iceberg:iceberg-core: + - code: "java.method.visibilityReduced" + old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" + new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" + justification: "Deprecations for 1.6.0 release" + - code: "java.element.noLongerDeprecated" + old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" + new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" + justification: "Constructor became private as part of deprecations cleanup for 1.6.0 release" + - code: "java.element.noLongerDeprecated" + old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, java.lang.String, java.lang.String, java.lang.String,\ + \ java.lang.String)" + new: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, org.apache.iceberg.rest.auth.AuthConfig)" + justification: "This is actually a removal of a deprecated constructor as part\ + \ of deprecations for 1.6.0 release, apparently mis-reported by rev-api" + - code: "java.method.numberOfParametersChanged" + old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, java.lang.String, java.lang.String, java.lang.String,\ + \ java.lang.String)" + new: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, org.apache.iceberg.rest.auth.AuthConfig)" + justification: "This is actually a removal of a deprecated constructor as part\ + \ of deprecations for 1.6.0 release, apparently mis-reported by rev-api" + - code: "java.method.removed" + old: "method java.util.List org.apache.iceberg.ReachableFileUtil::statisticsFilesLocations(org.apache.iceberg.Table,\ + \ java.util.function.Predicate)" + justification: "Deprecations for 1.6.0 release" + - code: "java.method.removed" + old: "method org.apache.iceberg.DataFiles.Builder org.apache.iceberg.DataFiles.Builder::withEqualityFieldIds(java.util.List)" + justification: "Deprecations for 1.6.0 release" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/core/src/main/java/org/apache/iceberg/DataFiles.java b/core/src/main/java/org/apache/iceberg/DataFiles.java index 34f830be4daa..5c7c1a646cc9 100644 --- a/core/src/main/java/org/apache/iceberg/DataFiles.java +++ b/core/src/main/java/org/apache/iceberg/DataFiles.java @@ -299,12 +299,6 @@ public Builder withSplitOffsets(List offsets) { return this; } - /** @deprecated since 1.5.0, will be removed in 1.6.0; must not be set for data files. */ - @Deprecated - public Builder withEqualityFieldIds(List equalityIds) { - throw new UnsupportedOperationException("Equality field IDs must not be set for data files"); - } - public Builder withEncryptionKeyMetadata(ByteBuffer newKeyMetadata) { this.keyMetadata = newKeyMetadata; return this; diff --git a/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java b/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java index ee1ff28e0349..fc53d6b79542 100644 --- a/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/ReachableFileUtil.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Set; import java.util.function.Predicate; -import java.util.stream.Collectors; import org.apache.hadoop.fs.Path; import org.apache.iceberg.TableMetadata.MetadataLogEntry; import org.apache.iceberg.hadoop.Util; @@ -142,24 +141,6 @@ public static List statisticsFilesLocations(Table table) { return statisticsFilesLocationsForSnapshots(table, null); } - /** - * Returns locations of statistics files for a table matching the given predicate . - * - * @param table table for which statistics files needs to be listed - * @param predicate predicate for filtering the statistics files - * @return the location of statistics files - * @deprecated since 1.5.0, will be removed in 1.6.0; use the {@code - * statisticsFilesLocationsForSnapshots(table, snapshotIds)} instead. - */ - @Deprecated - public static List statisticsFilesLocations( - Table table, Predicate predicate) { - return table.statisticsFiles().stream() - .filter(predicate) - .map(StatisticsFile::path) - .collect(Collectors.toList()); - } - /** * Returns locations of all statistics files for a table matching the given snapshot IDs. * diff --git a/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java index 336ebe9c0af7..6a5e1877e8f2 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java +++ b/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java @@ -27,9 +27,7 @@ public class PlaintextEncryptionManager implements EncryptionManager { private static final EncryptionManager INSTANCE = new PlaintextEncryptionManager(); private static final Logger LOG = LoggerFactory.getLogger(PlaintextEncryptionManager.class); - /** @deprecated will be removed in 1.6.0. use {@link #instance()} instead. */ - @Deprecated - public PlaintextEncryptionManager() {} + private PlaintextEncryptionManager() {} public static EncryptionManager instance() { return INSTANCE; diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 2283aba7d02d..76386027c629 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -465,24 +465,6 @@ public AuthSession(Map baseHeaders, AuthConfig config) { this.config = config; } - /** @deprecated since 1.5.0, will be removed in 1.6.0 */ - @Deprecated - public AuthSession( - Map baseHeaders, - String token, - String tokenType, - String credential, - String scope) { - this( - baseHeaders, - AuthConfig.builder() - .token(token) - .tokenType(tokenType) - .credential(credential) - .scope(scope) - .build()); - } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ @Deprecated public AuthSession( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index 2a2503ef2478..f06d9b83bcd4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -92,7 +92,7 @@ public static DataIterator createDataIterator(CombinedScanTask combined TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), combinedTask, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager()); + PlaintextEncryptionManager.instance()); } public static List> createRecordBatchList( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 88234c61123f..790c3bba8a42 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -165,7 +165,7 @@ private IcebergSourceReader createReader( null, true, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), + PlaintextEncryptionManager.instance(), Collections.emptyList()); return new IcebergSourceReader<>( SerializableRecordEmitter.defaultEmitter(), diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java index d063ad7f4a80..764215e904d3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -56,7 +56,7 @@ protected ReaderFunction readerFunction() { null, true, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), + PlaintextEncryptionManager.instance(), Collections.emptyList()); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index 2a2503ef2478..f06d9b83bcd4 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -92,7 +92,7 @@ public static DataIterator createDataIterator(CombinedScanTask combined TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), combinedTask, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager()); + PlaintextEncryptionManager.instance()); } public static List> createRecordBatchList( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index f19d57083b89..b9af6aca2895 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -165,7 +165,7 @@ private IcebergSourceReader createReader( null, true, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), + PlaintextEncryptionManager.instance(), Collections.emptyList()); return new IcebergSourceReader<>( SerializableRecordEmitter.defaultEmitter(), diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java index d063ad7f4a80..764215e904d3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -56,7 +56,7 @@ protected ReaderFunction readerFunction() { null, true, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), + PlaintextEncryptionManager.instance(), Collections.emptyList()); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index 2a2503ef2478..f06d9b83bcd4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -92,7 +92,7 @@ public static DataIterator createDataIterator(CombinedScanTask combined TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), combinedTask, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager()); + PlaintextEncryptionManager.instance()); } public static List> createRecordBatchList( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index f19d57083b89..b9af6aca2895 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -165,7 +165,7 @@ private IcebergSourceReader createReader( null, true, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), + PlaintextEncryptionManager.instance(), Collections.emptyList()); return new IcebergSourceReader<>( SerializableRecordEmitter.defaultEmitter(), diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java index d063ad7f4a80..764215e904d3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -56,7 +56,7 @@ protected ReaderFunction readerFunction() { null, true, new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - new PlaintextEncryptionManager(), + PlaintextEncryptionManager.instance(), Collections.emptyList()); } diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 583e45a26316..3fc38e436fb9 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -581,19 +581,6 @@ private boolean dropContent(TableIdentifier identifier, boolean purge, Content.T return false; } - /** @deprecated will be removed after 1.5.0 */ - @Deprecated - public void commitTable( - TableMetadata base, - TableMetadata metadata, - String newMetadataLocation, - IcebergTable expectedContent, - ContentKey key) - throws NessieConflictException, NessieNotFoundException { - String contentId = expectedContent == null ? null : expectedContent.getId(); - commitTable(base, metadata, newMetadataLocation, contentId, key); - } - public void commitTable( TableMetadata base, TableMetadata metadata, 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 2da0cf70992b..47690656b41d 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 @@ -694,18 +694,6 @@ private static void deleteManifests(FileIO io, List manifests) { .run(item -> io.deleteFile(item.path())); } - /** - * Loads a metadata table. - * - * @deprecated since 0.14.0, will be removed in 0.15.0; use {@link - * #loadMetadataTable(SparkSession, Table, MetadataTableType)}. - */ - @Deprecated - public static Dataset loadCatalogMetadataTable( - SparkSession spark, Table table, MetadataTableType type) { - return loadMetadataTable(spark, table, type); - } - public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type) { return loadMetadataTable(spark, table, type, ImmutableMap.of()); From b17213333fa3cf2631726f3fe69fb7a03397b79f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 17 Jun 2024 14:59:32 +0200 Subject: [PATCH 0376/1019] Core, Flink, Spark: Import the right assertThatThrownBy method from AssertJ (#10512) --- .baseline/checkstyle/checkstyle.xml | 4 ++++ core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java | 2 +- .../test/java/org/apache/iceberg/rest/TestRESTCatalog.java | 2 +- .../java/org/apache/iceberg/flink/TestFlinkCatalogTable.java | 2 +- .../java/org/apache/iceberg/flink/TestFlinkCatalogTable.java | 2 +- .../java/org/apache/iceberg/flink/TestFlinkCatalogTable.java | 2 +- .../iceberg/spark/source/TestStructuredStreamingRead3.java | 2 +- .../iceberg/spark/source/TestStructuredStreamingRead3.java | 2 +- .../iceberg/spark/source/TestStructuredStreamingRead3.java | 2 +- 9 files changed, 12 insertions(+), 8 deletions(-) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 16b06c6bb1a5..8df46a5c688a 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -88,6 +88,10 @@ + + + + diff --git a/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java index 3a6666bac38b..ba5d5af62204 100644 --- a/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java +++ b/core/src/test/java/org/apache/iceberg/TestClientPoolImpl.java @@ -19,7 +19,7 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.junit.jupiter.api.Test; 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 95380424e7ad..34d088cb60d5 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -20,7 +20,7 @@ import static org.apache.iceberg.types.Types.NestedField.required; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.atLeastOnce; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index eaa92e32c49d..3aea32e4ecc1 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index eaa92e32c49d..3aea32e4ecc1 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; 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 f3af2c3cbe98..e5b1d61b2f21 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 @@ -19,7 +19,7 @@ package org.apache.iceberg.flink; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index de94a7c8bf8b..e5ebd327db7a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -19,7 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.expressions.Expressions.ref; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 47e933360156..190a189f8888 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -19,7 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.expressions.Expressions.ref; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 5a8d4deec730..86d65ba0e558 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -20,7 +20,7 @@ import static org.apache.iceberg.expressions.Expressions.ref; import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.AssertionsForClassTypes.assertThatThrownBy; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; From c9cf9089cd5b9b587551eb8a6cf46f70075dc78e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 17 Jun 2024 15:00:56 +0200 Subject: [PATCH 0377/1019] AWS: Rename test helper to deconflict with Assertions (#10511) The test helper `assertThatThrownBy` had same name like `Assertions.assertThatThrownBy` but different API (not assertion builder). Rename to avoid confusion. --- .../apache/iceberg/aws/TestAwsClientFactories.java | 13 ++++++++----- 1 file changed, 8 insertions(+), 5 deletions(-) diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index 01c14790a34e..9598860d6fdf 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -195,12 +195,15 @@ private void testProviderAndAssertThrownBy(String providerClassName, String cont public void assertAllClientObjectsThrownBy( AwsClientFactory defaultAwsClientFactory, String containsMessage) { // invoking sdk client apis to ensure resolveCredentials() being called - assertThatThrownBy(() -> defaultAwsClientFactory.s3().listBuckets(), containsMessage); - assertThatThrownBy( + assertIllegalArgumentException( + () -> defaultAwsClientFactory.s3().listBuckets(), containsMessage); + assertIllegalArgumentException( () -> defaultAwsClientFactory.glue().getTables(GetTablesRequest.builder().build()), containsMessage); - assertThatThrownBy(() -> defaultAwsClientFactory.dynamo().listTables(), containsMessage); - assertThatThrownBy(() -> defaultAwsClientFactory.kms().listAliases(), containsMessage); + assertIllegalArgumentException( + () -> defaultAwsClientFactory.dynamo().listTables(), containsMessage); + assertIllegalArgumentException( + () -> defaultAwsClientFactory.kms().listAliases(), containsMessage); } private void assertClientObjectsNotNull(AwsClientFactory defaultAwsClientFactory) { @@ -210,7 +213,7 @@ private void assertClientObjectsNotNull(AwsClientFactory defaultAwsClientFactory Assertions.assertThat(defaultAwsClientFactory.kms()).isNotNull(); } - private void assertThatThrownBy( + private void assertIllegalArgumentException( ThrowableAssert.ThrowingCallable shouldRaiseThrowable, String containsMessage) { Assertions.assertThatThrownBy(shouldRaiseThrowable) .isInstanceOf(IllegalArgumentException.class) From d22cfaddff0e364b98a0dc0327ed8f59fb3dacd4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 17 Jun 2024 15:11:11 +0200 Subject: [PATCH 0378/1019] Build: Merge job definitions in spark-ci.yml (#10513) Refactor to remove code duplication. Does not change test coverage nor configurations exercised on CI. --- .github/workflows/spark-ci.yml | 78 ++++------------------------------ 1 file changed, 9 insertions(+), 69 deletions(-) diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index ef028fcd7676..5ddff7a506dd 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -65,12 +65,13 @@ concurrency: cancel-in-progress: ${{ github.event_name == 'pull_request' }} jobs: - spark-3x-scala-2-12-tests: + spark-tests: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11] + jvm: [8, 11, 17] spark: ['3.3', '3.4', '3.5'] + scala: ['2.12', '2.13'] env: SPARK_LOCAL_IP: localhost steps: @@ -90,73 +91,12 @@ jobs: with: tool-cache: false - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-spark:iceberg-spark-${{ matrix.spark }}_2.12:check :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_2.12:check :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_2.12:check -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 - if: failure() - with: - name: test logs - path: | - **/build/testlogs - - spark-3x-scala-2-13-tests: - runs-on: ubuntu-22.04 - strategy: - matrix: - jvm: [8, 11] - spark: ['3.3','3.4','3.5'] - env: - SPARK_LOCAL_IP: localhost - steps: - - uses: actions/checkout@v4 - - uses: actions/setup-java@v4 - with: - distribution: zulu - java-version: ${{ matrix.jvm }} - - uses: actions/cache@v4 - with: - path: | - ~/.gradle/caches - ~/.gradle/wrapper - key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} - restore-keys: ${{ runner.os }}-gradle- - - uses: jlumbroso/free-disk-space@54081f138730dfa15788a46383842cd2f914a1be # v1.3.1 - with: - tool-cache: false - - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-spark:iceberg-spark-${{ matrix.spark }}_2.13:check :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_2.13:check :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_2.13:check -Pquick=true -x javadoc - - uses: actions/upload-artifact@v4 - if: failure() - with: - name: test logs - path: | - **/build/testlogs - - spark-3x-java-17-tests: - runs-on: ubuntu-22.04 - strategy: - matrix: - spark: ['3.3','3.4', '3.5'] - scala-version: ['2.12', '2.13'] - env: - SPARK_LOCAL_IP: localhost - steps: - - uses: actions/checkout@v4 - - uses: actions/setup-java@v4 - with: - distribution: zulu - java-version: 17 - - uses: actions/cache@v4 - with: - path: | - ~/.gradle/caches - ~/.gradle/wrapper - key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} - restore-keys: ${{ runner.os }}-gradle- - - uses: jlumbroso/free-disk-space@54081f138730dfa15788a46383842cd2f914a1be # v1.3.1 - with: - tool-cache: false - - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=${{ matrix.scala-version }} -DhiveVersions= -DflinkVersions= :iceberg-spark:iceberg-spark-${{ matrix.spark }}_${{ matrix.scala-version }}:check :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_${{ matrix.scala-version }}:check :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_${{ matrix.scala-version }}:check -Pquick=true -x javadoc + - run: | + ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=${{ matrix.scala }} -DhiveVersions= -DflinkVersions= \ + :iceberg-spark:iceberg-spark-${{ matrix.spark }}_${{ matrix.scala }}:check \ + :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_${{ matrix.scala }}:check \ + :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_${{ matrix.scala }}:check \ + -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: From 8e808de23098536c6c9708af63f0abcc8b50ddf1 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 17 Jun 2024 15:49:42 +0200 Subject: [PATCH 0379/1019] API, Spark 3.3: Remove all usages of deprecated AssertHelpers (#10500) --- LICENSE | 1 - .../org/apache/iceberg/AssertHelpers.java | 213 ---- .../extensions/TestAddFilesProcedure.java | 114 ++- .../extensions/TestAlterTableSchema.java | 55 +- .../extensions/TestAncestorsOfProcedure.java | 34 +- .../spark/extensions/TestBranchDDL.java | 103 +- .../extensions/TestCallStatementParser.java | 11 +- .../spark/extensions/TestChangelogTable.java | 11 +- .../TestCherrypickSnapshotProcedure.java | 69 +- .../extensions/TestConflictValidation.java | 291 +++--- .../iceberg/spark/extensions/TestDelete.java | 20 +- .../TestExpireSnapshotsProcedure.java | 143 ++- .../iceberg/spark/extensions/TestMerge.java | 958 +++++++++--------- .../extensions/TestMergeOnReadDelete.java | 17 +- .../extensions/TestMigrateTableProcedure.java | 50 +- .../TestPublishChangesProcedure.java | 58 +- .../TestRemoveOrphanFilesProcedure.java | 169 ++- .../spark/extensions/TestReplaceBranch.java | 31 +- .../TestRequiredDistributionAndOrdering.java | 30 +- .../TestRewriteDataFilesProcedure.java | 222 ++-- .../TestRewriteManifestsProcedure.java | 70 +- .../TestRollbackToSnapshotProcedure.java | 99 +- .../TestRollbackToTimestampProcedure.java | 97 +- .../TestSetCurrentSnapshotProcedure.java | 111 +- .../TestSnapshotTableProcedure.java | 63 +- .../iceberg/spark/extensions/TestTagDDL.java | 129 ++- .../iceberg/spark/extensions/TestUpdate.java | 145 ++- .../spark/extensions/TestWriteAborts.java | 36 +- .../iceberg/spark/TestFunctionCatalog.java | 42 +- .../TestDeleteReachableFilesAction.java | 22 +- .../actions/TestExpireSnapshotsAction.java | 23 +- .../actions/TestRemoveOrphanFilesAction.java | 58 +- .../actions/TestRewriteDataFilesAction.java | 124 +-- .../actions/TestRewriteManifestsAction.java | 13 +- .../TestParquetVectorizedReads.java | 39 +- .../spark/source/TestDataFrameWriterV2.java | 50 +- .../spark/source/TestDataSourceOptions.java | 83 +- .../source/TestForwardCompatibility.java | 31 +- .../source/TestIcebergSourceTablesBase.java | 23 +- ...tMetadataTablesWithPartitionEvolution.java | 11 +- .../TestRequiredDistributionAndOrdering.java | 52 +- .../spark/source/TestSparkDataWrite.java | 30 +- .../source/TestSparkMetadataColumns.java | 20 +- .../source/TestStructuredStreamingRead3.java | 23 +- .../source/TestTimestampWithoutZone.java | 42 +- .../spark/source/TestWriteMetricsConfig.java | 14 +- .../iceberg/spark/sql/TestAlterTable.java | 40 +- .../iceberg/spark/sql/TestCreateTable.java | 28 +- .../iceberg/spark/sql/TestDeleteFrom.java | 12 +- .../iceberg/spark/sql/TestDropTable.java | 12 +- .../iceberg/spark/sql/TestNamespaceSQL.java | 11 +- .../apache/iceberg/spark/sql/TestSelect.java | 104 +- .../spark/sql/TestSparkBucketFunction.java | 181 ++-- .../spark/sql/TestSparkDaysFunction.java | 42 +- .../spark/sql/TestSparkHoursFunction.java | 42 +- .../spark/sql/TestSparkMonthsFunction.java | 41 +- .../spark/sql/TestSparkTruncateFunction.java | 174 ++-- .../spark/sql/TestSparkYearsFunction.java | 41 +- .../spark/sql/TestTimestampWithoutZone.java | 19 +- .../sql/UnpartitionedWritesTestBase.java | 29 +- 60 files changed, 2317 insertions(+), 2509 deletions(-) delete mode 100644 api/src/test/java/org/apache/iceberg/AssertHelpers.java diff --git a/LICENSE b/LICENSE index 46167007c6de..efb46dab44da 100644 --- a/LICENSE +++ b/LICENSE @@ -227,7 +227,6 @@ This product includes code from Apache Parquet. * DynMethods.java * DynConstructors.java -* AssertHelpers.java * IOUtil.java readFully and tests * ByteBufferInputStream implementations and tests diff --git a/api/src/test/java/org/apache/iceberg/AssertHelpers.java b/api/src/test/java/org/apache/iceberg/AssertHelpers.java deleted file mode 100644 index a1cf1b20797e..000000000000 --- a/api/src/test/java/org/apache/iceberg/AssertHelpers.java +++ /dev/null @@ -1,213 +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; - -import java.util.concurrent.Callable; -import org.apache.avro.AvroRuntimeException; -import org.apache.avro.generic.GenericRecord; -import org.assertj.core.api.AbstractThrowableAssert; -import org.assertj.core.api.Assertions; -import org.assertj.core.api.ThrowableAssert; - -/** - * This class is deprecated. Please use {@link - * Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} directly as shown below: - * - *
    - * Assertions.assertThatThrownBy(() -> throwingCallable)
    - *    .isInstanceOf(ExpectedException.class)
    - *    .hasMessage(expectedErrorMsg)
    - * 
    - * - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} directly - * as it provides a more fluent way of asserting on exceptions. - */ -@Deprecated -public class AssertHelpers { - - private AssertHelpers() {} - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param containedInMessage A String that should be contained by the thrown exception's message - * @param callable A Callable that is expected to throw the exception - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrows( - String message, - Class expected, - String containedInMessage, - Callable callable) { - AbstractThrowableAssert check = - Assertions.assertThatThrownBy(callable::call).as(message).isInstanceOf(expected); - if (null != containedInMessage) { - check.hasMessageContaining(containedInMessage); - } - } - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param containedInMessage A String that should be contained by the thrown exception's message - * @param runnable A Runnable that is expected to throw the runtime exception - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrows( - String message, - Class expected, - String containedInMessage, - Runnable runnable) { - AbstractThrowableAssert check = - Assertions.assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); - if (null != containedInMessage) { - check.hasMessageContaining(containedInMessage); - } - } - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param callable A Callable that is expected to throw the exception - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrows( - String message, Class expected, Callable callable) { - assertThrows(message, expected, null, callable); - } - - /** - * A convenience method to avoid a large number of @Test(expected=...) tests - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param runnable A Runnable that is expected to throw the runtime exception - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrows( - String message, Class expected, Runnable runnable) { - assertThrows(message, expected, null, runnable); - } - - /** - * A convenience method to assert the cause of thrown exception. - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the cause of the Runnable should throw - * @param containedInMessage A String that should be contained by the cause of the thrown - * exception's message - * @param runnable A Runnable that is expected to throw the runtime exception - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrowsCause( - String message, - Class expected, - String containedInMessage, - Runnable runnable) { - Assertions.assertThatThrownBy(runnable::run) - .as(message) - .getCause() - .isInstanceOf(expected) - .hasMessageContaining(containedInMessage); - } - - /** - * A convenience method to assert both the thrown exception and the cause of thrown exception. - * - * @param message A String message to describe this assertion - * @param expected An Exception class that the Runnable should throw - * @param expectedContainedInMessage A String that should be contained by the thrown exception's - * message, will be skipped if null. - * @param cause An Exception class that the cause of the Runnable should throw - * @param causeContainedInMessage A String that should be contained by the cause of the thrown - * exception's message, will be skipped if null. - * @param runnable A Runnable that is expected to throw the runtime exception - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrowsWithCause( - String message, - Class expected, - String expectedContainedInMessage, - Class cause, - String causeContainedInMessage, - Runnable runnable) { - AbstractThrowableAssert chain = - Assertions.assertThatThrownBy(runnable::run).as(message).isInstanceOf(expected); - - if (expectedContainedInMessage != null) { - chain = chain.hasMessageContaining(expectedContainedInMessage); - } - - chain = chain.getCause().isInstanceOf(cause); - if (causeContainedInMessage != null) { - chain.hasMessageContaining(causeContainedInMessage); - } - } - - /** - * A convenience method to check if an Avro field is empty. - * - * @param record The record to read from - * @param field The name of the field - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertEmptyAvroField(GenericRecord record, String field) { - AssertHelpers.assertThrows( - "Not a valid schema field: " + field, AvroRuntimeException.class, () -> record.get(field)); - } - - /** - * Same as {@link AssertHelpers#assertThrowsCause}, but this method compares root cause. - * - * @deprecated Use {@link Assertions#assertThatThrownBy(ThrowableAssert.ThrowingCallable)} - * directly as it provides a more fluent way of asserting on exceptions. - */ - @Deprecated - public static void assertThrowsRootCause( - String message, - Class expected, - String containedInMessage, - Runnable runnable) { - Assertions.assertThatThrownBy(runnable::run) - .as(message) - .getRootCause() - .isInstanceOf(expected) - .hasMessageContaining(containedInMessage); - } -} diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index a48deae7421f..aab2ae65d010 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -34,7 +34,6 @@ import org.apache.avro.generic.GenericDatumWriter; import org.apache.avro.generic.GenericRecord; import org.apache.avro.io.DatumWriter; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -48,6 +47,7 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; +import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -688,23 +688,23 @@ public void invalidDataImport() { createIcebergTable("id Integer, name String, dept String, subdept String"); - AssertHelpers.assertThrows( - "Should forbid adding of partitioned data to unpartitioned table", - IllegalArgumentException.class, - "Cannot use partition filter with an unpartitioned table", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", - catalogName, tableName, fileTableDir.getAbsolutePath())); - - AssertHelpers.assertThrows( - "Should forbid adding of partitioned data to unpartitioned table", - IllegalArgumentException.class, - "Cannot add partitioned files to an unpartitioned table", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`')", - catalogName, tableName, fileTableDir.getAbsolutePath())); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .as("Should forbid adding of partitioned data to unpartitioned table") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot use partition filter with an unpartitioned table"); + + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`')", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .as("Should forbid adding of partitioned data to unpartitioned table") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot add partitioned files to an unpartitioned table"); } @Test @@ -714,23 +714,24 @@ public void invalidDataImportPartitioned() { createIcebergTable( "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); - AssertHelpers.assertThrows( - "Should forbid adding with a mismatching partition spec", - IllegalArgumentException.class, - "is greater than the number of partitioned columns", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", - catalogName, tableName, fileTableDir.getAbsolutePath())); - - AssertHelpers.assertThrows( - "Should forbid adding with partition spec with incorrect columns", - IllegalArgumentException.class, - "specified partition filter refers to columns that are not partitioned", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", - catalogName, tableName, fileTableDir.getAbsolutePath())); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .as("Should forbid adding with a mismatching partition spec") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("is greater than the number of partitioned columns"); + + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", + catalogName, tableName, fileTableDir.getAbsolutePath())) + .as("Should forbid adding with partition spec with incorrect columns") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "specified partition filter refers to columns that are not partitioned"); } @Test @@ -782,18 +783,19 @@ public void duplicateDataPartitioned() { + "partition_filter => map('id', 1))", catalogName, tableName, sourceTableName); - AssertHelpers.assertThrows( - "Should not allow adding duplicate files", - IllegalStateException.class, - "Cannot complete import because data files to be imported already" - + " exist within the target table", - () -> - scalarSql( - "CALL %s.system.add_files(" - + "table => '%s', " - + "source_table => '%s', " - + "partition_filter => map('id', 1))", - catalogName, tableName, sourceTableName)); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files(" + + "table => '%s', " + + "source_table => '%s', " + + "partition_filter => map('id', 1))", + catalogName, tableName, sourceTableName)) + .as("Should not allow adding duplicate files") + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "Cannot complete import because data files to be imported already" + + " exist within the target table"); } @Test @@ -841,14 +843,16 @@ public void duplicateDataUnpartitioned() { sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); - AssertHelpers.assertThrows( - "Should not allow adding duplicate files", - IllegalStateException.class, - "Cannot complete import because data files to be imported already" - + " exist within the target table", - () -> - scalarSql( - "CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName)); + Assertions.assertThatThrownBy( + () -> + scalarSql( + "CALL %s.system.add_files('%s', '%s')", + catalogName, tableName, sourceTableName)) + .as("Should not allow adding duplicate files") + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining( + "Cannot complete import because data files to be imported already" + + " exist within the target table"); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java index c993c213dc5e..acbf5c7a632d 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.extensions; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -77,17 +77,16 @@ public void testSetInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - AssertHelpers.assertThrows( - "should not allow setting unknown fields", - IllegalArgumentException.class, - "not found in current schema or added columns", - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)); - - AssertHelpers.assertThrows( - "should not allow setting optional fields", - IllegalArgumentException.class, - "not a required field", - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) + .as("should not allow setting unknown fields") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("not found in current schema or added columns"); + + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) + .as("should not allow setting optional fields") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("not a required field"); } @Test @@ -140,23 +139,23 @@ public void testDropInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - AssertHelpers.assertThrows( - "should not allow dropping unknown fields", - IllegalArgumentException.class, - "field unknown not found", - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) + .as("should not allow dropping unknown fields") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("field unknown not found"); sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); - AssertHelpers.assertThrows( - "should not allow dropping a field that is not an identifier", - IllegalArgumentException.class, - "data is not an identifier field", - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)); - - AssertHelpers.assertThrows( - "should not allow dropping a nested field that is not an identifier", - IllegalArgumentException.class, - "location.lon is not an identifier field", - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) + .as("should not allow dropping a field that is not an identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("data is not an identifier field"); + + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) + .as("should not allow dropping a nested field that is not an identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("location.lon is not an identifier field"); } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index ae591821e21a..13fa96d1e08a 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -20,10 +20,10 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -147,22 +147,20 @@ public void testAncestorOfUsingNamedArgs() { @Test public void testInvalidAncestorOfCases() { - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.ancestors_of()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier for parameter 'table'", - () -> sql("CALL %s.system.ancestors_of('')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)); + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier for parameter 'table'"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); } } 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 e6ac4cfc4dac..8f3e88f3c811 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 @@ -23,7 +23,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; @@ -86,11 +85,11 @@ public void testCreateBranch() throws NoSuchTableException { Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - AssertHelpers.assertThrows( - "Cannot create an existing branch", - IllegalArgumentException.class, - "Ref b1 already exists", - () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) + .as("Cannot create an existing branch") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref b1 already exists"); } @Test @@ -196,12 +195,14 @@ public void testCreateBranchUseCustomMinSnapshotsToKeepAndMaxSnapshotAge() Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertNull(ref.maxRefAgeMs()); - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "no viable alternative at input 'WITH SNAPSHOT RETENTION'", - () -> - sql("ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", tableName, branchName)); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", + tableName, branchName)) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("no viable alternative at input 'WITH SNAPSHOT RETENTION'"); } @Test @@ -217,26 +218,27 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { Assert.assertNull(ref.maxSnapshotAgeMs()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input", - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)); - - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input", - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")); - - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}", - () -> - sql( - "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", - tableName, branchName, maxRefAge)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input"); + + Assertions.assertThatThrownBy( + () -> + sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input"); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", + tableName, branchName, maxRefAge)) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}"); } @Test @@ -258,11 +260,11 @@ public void testDropBranch() throws NoSuchTableException { @Test public void testDropBranchDoesNotExist() { - AssertHelpers.assertThrows( - "Cannot perform drop branch on branch which does not exist", - IllegalArgumentException.class, - "Branch does not exist: nonExistingBranch", - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) + .as("Cannot perform drop branch on branch which does not exist") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Branch does not exist: nonExistingBranch"); } @Test @@ -271,29 +273,26 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); - AssertHelpers.assertThrows( - "Cannot perform drop branch on tag", - IllegalArgumentException.class, - "Ref b1 is a tag not a branch", - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) + .as("Cannot perform drop branch on tag") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref b1 is a tag not a branch"); } @Test public void testDropBranchNonConformingName() { - AssertHelpers.assertThrows( - "Non-conforming branch name", - IcebergParseException.class, - "mismatched input '123'", - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) + .as("Non-conforming branch name") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input '123'"); } @Test public void testDropMainBranchFails() { - AssertHelpers.assertThrows( - "Cannot drop the main branch", - IllegalArgumentException.class, - "Cannot remove main branch", - () -> sql("ALTER TABLE %s DROP BRANCH main", tableName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) + .as("Cannot drop the main branch") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot remove main branch"); } @Test 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 9c2233ccb791..c0f7ff1f8f9b 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 @@ -22,7 +22,6 @@ import java.sql.Timestamp; import java.time.Instant; import java.util.List; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.spark.sql.SparkSession; @@ -38,6 +37,7 @@ import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -144,11 +144,10 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - AssertHelpers.assertThrows( - "Should fail with a sensible parse error", - IcebergParseException.class, - "missing '(' at 'radish'", - () -> parser.parsePlan("CALL cat.system radish kebab")); + Assertions.assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + .as("Should fail with a sensible parse error") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("missing '(' at 'radish'"); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java index cc81b4b3d323..3b6ac57cc03e 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.extensions; -import static org.apache.iceberg.AssertHelpers.assertThrows; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; @@ -35,6 +34,7 @@ import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Row; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -202,10 +202,11 @@ public void testTimeRangeValidation() { Snapshot snap3 = table.currentSnapshot(); long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); - assertThrows( - "Should fail if start time is after end time", - IllegalArgumentException.class, - () -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())); + Assertions.assertThatThrownBy( + () -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) + .as("Should fail if start time is after end time") + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); } @Test 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 7309a176b922..cbecfcef3ed6 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 @@ -22,7 +22,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,6 +31,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -161,43 +161,42 @@ public void testCherrypickSnapshotRefreshesRelationCache() { public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should reject invalid snapshot id", - ValidationException.class, - "Cannot cherry-pick unknown snapshot ID", - () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)) + .as("Should reject invalid snapshot id") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot cherry-pick unknown snapshot ID"); } @Test public void testInvalidCherrypickSnapshotCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java index 6fcbf1f903be..7d183167572d 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -31,6 +30,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -69,21 +69,24 @@ public void testOverwriteFilterSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting new data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting new data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found conflicting files that can contain records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -111,21 +114,23 @@ public void testOverwriteFilterSerializableIsolation2() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting new delete files should throw exception", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1)); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting new delete files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -149,21 +154,24 @@ public void testOverwriteFilterSerializableIsolation3() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting deleted files that can contain records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting deleted data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found conflicting deleted files that can contain records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -184,20 +192,23 @@ public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { // Validating from no snapshot id defaults to beginning snapshot id and finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting new data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwrite(functions.col("id").equalTo(1)); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting new data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found conflicting files that can contain records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -225,21 +236,23 @@ public void testOverwriteFilterSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting new delete files should throw exception", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwrite(functions.col("id").equalTo(1)); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwrite(functions.col("id").equalTo(1)); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting new delete files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found new conflicting delete files that can apply to records matching ref(name=\"id\") == 1:"); // Validating from latest snapshot should succeed table.refresh(); @@ -278,21 +291,24 @@ public void testOverwritePartitionSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching partitions [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting deleted data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found conflicting files that can contain records matching partitions [id=1]"); // Validating from latest snapshot should succeed table.refresh(); @@ -318,21 +334,23 @@ public void testOverwritePartitionSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found new conflicting delete files that can apply to records matching [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting deleted data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found new conflicting delete files that can apply to records matching [id=1]"); // Validating from latest snapshot should succeed table.refresh(); @@ -357,21 +375,23 @@ public void testOverwritePartitionSnapshotIsolation2() throws Exception { spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting deleted files that can apply to records matching [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option(SparkWriteOptions.VALIDATE_FROM_SNAPSHOT_ID, String.valueOf(snapshotId)) + .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SNAPSHOT.toString()) + .overwritePartitions(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting deleted data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found conflicting deleted files that can apply to records matching [id=1]"); // Validating from latest snapshot should succeed table.refresh(); @@ -409,20 +429,23 @@ public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { // Validating from null snapshot is equivalent to validating from beginning Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrowsCause( - "Conflicting deleted data files should throw exception", - ValidationException.class, - "Found conflicting files that can contain records matching partitions [id=1]", - () -> { - try { - conflictingDf - .writeTo(tableName) - .option(SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) - .overwritePartitions(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + conflictingDf + .writeTo(tableName) + .option( + SparkWriteOptions.ISOLATION_LEVEL, IsolationLevel.SERIALIZABLE.toString()) + .overwritePartitions(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Conflicting deleted data files should throw exception") + .cause() + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Found conflicting files that can contain records matching partitions [id=1]"); // Validating from latest snapshot should succeed table.refresh(); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index e5ea378e5aab..af66df5a9fd4 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -44,7 +44,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.RowLevelOperationMode; @@ -448,11 +447,11 @@ public void testDeleteWithNonDeterministicCondition() { sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); createBranchIfNeeded(); - AssertHelpers.assertThrows( - "Should complain about non-deterministic expressions", - AnalysisException.class, - "nondeterministic expressions are only allowed", - () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())); + Assertions.assertThatThrownBy( + () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) + .as("Should complain about non-deterministic expressions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("nondeterministic expressions are only allowed"); } @Test @@ -737,11 +736,10 @@ public void testDeleteWithNotInSubquery() throws NoSuchTableException { public void testDeleteOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Delete is supported only for Iceberg tables", - AnalysisException.class, - "DELETE is only supported with v2 tables.", - () -> sql("DELETE FROM %s WHERE c1 = -100", "testtable")); + Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s WHERE c1 = -100", "testtable")) + .as("Delete is supported only for Iceberg tables") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("DELETE is only supported with v2 tables."); } @Test 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 6383521a44c2..f677facb9902 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 @@ -31,7 +31,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; import org.apache.iceberg.PartitionStatisticsFile; @@ -163,44 +162,42 @@ public void testExpireSnapshotsGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - AssertHelpers.assertThrows( - "Should reject call", - ValidationException.class, - "Cannot expire snapshots: GC is disabled", - () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) + .as("Should reject call") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot expire snapshots: GC is disabled"); } @Test public void testInvalidExpireSnapshotsCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.expire_snapshots()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.expire_snapshots('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) + .as("Should not resolve procedures in arbitrary namespaces") + .isInstanceOf(NoSuchProcedureException.class) + .hasMessageContaining("not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } @Test @@ -218,14 +215,14 @@ public void testResolvingTableInAnotherCatalog() throws IOException { "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", anotherCatalog, tableIdent); - AssertHelpers.assertThrows( - "Should reject calls for a table in another catalog", - IllegalArgumentException.class, - "Cannot run procedure in catalog", - () -> - sql( - "CALL %s.system.expire_snapshots('%s')", - catalogName, anotherCatalog + "." + tableName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots('%s')", + catalogName, anotherCatalog + "." + tableName)) + .as("Should reject calls for a table in another catalog") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot run procedure in catalog"); } @Test @@ -255,23 +252,23 @@ public void testConcurrentExpireSnapshots() { public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes = 0", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, 0)); - - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes < 0 ", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, -1)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, 0)) + .as("Should throw an error when max_concurrent_deletes = 0") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("max_concurrent_deletes should have value > 0"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, -1)) + .as("Should throw an error when max_concurrent_deletes < 0 ") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("max_concurrent_deletes should have value > 0"); } @Test @@ -404,19 +401,19 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); - AssertHelpers.assertThrows( - "Should reject call", - IllegalArgumentException.class, - "Cannot expire", - () -> - sql( - "CALL %s.system.expire_snapshots(" - + "table => '%s'," - + "snapshot_ids => ARRAY(%d, %d))", - catalogName, - tableIdent, - table.currentSnapshot().snapshotId(), - table.currentSnapshot().parentId())); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.expire_snapshots(" + + "table => '%s'," + + "snapshot_ids => ARRAY(%d, %d))", + catalogName, + tableIdent, + table.currentSnapshot().snapshotId(), + table.currentSnapshot().parentId())) + .as("Should reject call") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot expire"); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index e855eb49a16c..ead477d71b20 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -40,7 +40,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.RowLevelOperationMode; @@ -584,22 +583,23 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { ds.union(ds).createOrReplaceTempView("source"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id == s.value " - + "WHEN MATCHED AND t.id = 1 THEN " - + " UPDATE SET id = 10 " - + "WHEN MATCHED AND t.id = 6 THEN " - + " DELETE " - + "WHEN NOT MATCHED AND s.value = 2 THEN " - + " INSERT (id, dep) VALUES (s.value, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.value = 2 THEN " + + " INSERT (id, dep) VALUES (s.value, null)", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); assertEquals( "Target should be unchanged", @@ -626,22 +626,23 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { () -> { String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id == s.value " - + "WHEN MATCHED AND t.id = 1 THEN " - + " UPDATE SET id = 10 " - + "WHEN MATCHED AND t.id = 6 THEN " - + " DELETE " - + "WHEN NOT MATCHED AND s.value = 2 THEN " - + " INSERT (id, dep) VALUES (s.value, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.value = 2 THEN " + + " INSERT (id, dep) VALUES (s.value, null)", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); }); assertEquals( @@ -666,22 +667,23 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqua () -> { String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id > s.value " - + "WHEN MATCHED AND t.id = 1 THEN " - + " UPDATE SET id = 10 " - + "WHEN MATCHED AND t.id = 6 THEN " - + " DELETE " - + "WHEN NOT MATCHED AND s.value = 2 THEN " - + " INSERT (id, dep) VALUES (s.value, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id > s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.value = 2 THEN " + + " INSERT (id, dep) VALUES (s.value, null)", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); }); assertEquals( @@ -704,20 +706,21 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM ds.union(ds).createOrReplaceTempView("source"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id == s.value " - + "WHEN MATCHED AND t.id = 1 THEN " - + " UPDATE SET id = 10 " - + "WHEN MATCHED AND t.id = 6 THEN " - + " DELETE", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); assertEquals( "Target should be unchanged", @@ -738,20 +741,21 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM ds.union(ds).createOrReplaceTempView("source"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id > s.value " - + "WHEN MATCHED AND t.id = 1 THEN " - + " UPDATE SET id = 10 " - + "WHEN MATCHED AND t.id = 6 THEN " - + " DELETE", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id > s.value " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET id = 10 " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); assertEquals( "Target should be unchanged", @@ -774,22 +778,23 @@ public void testMergeWithMultipleUpdatesForTargetRow() { + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id == s.id " - + "WHEN MATCHED AND t.id = 1 THEN " - + " UPDATE SET * " - + "WHEN MATCHED AND t.id = 6 THEN " - + " DELETE " - + "WHEN NOT MATCHED AND s.id = 2 THEN " - + " INSERT *", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " UPDATE SET * " + + "WHEN MATCHED AND t.id = 6 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); assertEquals( "Target should be unchanged", @@ -845,20 +850,21 @@ public void testMergeWithSingleConditionalDelete() { + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - AssertHelpers.assertThrowsCause( - "Should complain about multiple matches", - SparkException.class, - errorMsg, - () -> { - sql( - "MERGE INTO %s AS t USING source AS s " - + "ON t.id == s.id " - + "WHEN MATCHED AND t.id = 1 THEN " - + " DELETE " - + "WHEN NOT MATCHED AND s.id = 2 THEN " - + " INSERT *", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s AS t USING source AS s " + + "ON t.id == s.id " + + "WHEN MATCHED AND t.id = 1 THEN " + + " DELETE " + + "WHEN NOT MATCHED AND s.id = 2 THEN " + + " INSERT *", + commitTarget()); + }) + .as("Should complain about multiple matches") + .cause() + .isInstanceOf(SparkException.class) + .hasMessageContaining(errorMsg); assertEquals( "Target should be unchanged", @@ -1985,46 +1991,46 @@ public void testMergeWithNonExistingColumns() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about the invalid top-level column", - AnalysisException.class, - "cannot resolve t.invalid_col", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.invalid_col = s.c2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.invalid_col = s.c2", + commitTarget()); + }) + .as("Should complain about the invalid top-level column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("cannot resolve t.invalid_col"); - AssertHelpers.assertThrows( - "Should complain about the invalid nested column", - AnalysisException.class, - "No such struct field invalid_col", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n2.invalid_col = s.c2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.invalid_col = s.c2", + commitTarget()); + }) + .as("Should complain about the invalid nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("No such struct field invalid_col"); - AssertHelpers.assertThrows( - "Should complain about the invalid top-level column", - AnalysisException.class, - "cannot resolve invalid_col", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n2.dn1 = s.c2 " - + "WHEN NOT MATCHED THEN " - + " INSERT (id, invalid_col) VALUES (s.c1, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.dn1 = s.c2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, invalid_col) VALUES (s.c1, null)", + commitTarget()); + }) + .as("Should complain about the invalid top-level column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("cannot resolve invalid_col"); } @Test @@ -2034,48 +2040,48 @@ public void testMergeWithInvalidColumnsInInsert() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about the nested column", - AnalysisException.class, - "Nested fields are not supported inside INSERT clauses", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n2.dn1 = s.c2 " - + "WHEN NOT MATCHED THEN " - + " INSERT (id, c.n2) VALUES (s.c1, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.dn1 = s.c2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, c.n2) VALUES (s.c1, null)", + commitTarget()); + }) + .as("Should complain about the nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Nested fields are not supported inside INSERT clauses"); - AssertHelpers.assertThrows( - "Should complain about duplicate columns", - AnalysisException.class, - "Duplicate column names inside INSERT clause", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n2.dn1 = s.c2 " - + "WHEN NOT MATCHED THEN " - + " INSERT (id, id) VALUES (s.c1, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n2.dn1 = s.c2 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id, id) VALUES (s.c1, null)", + commitTarget()); + }) + .as("Should complain about duplicate columns") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Duplicate column names inside INSERT clause"); - AssertHelpers.assertThrows( - "Should complain about missing columns", - AnalysisException.class, - "must provide values for all columns of the target table", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN NOT MATCHED THEN " - + " INSERT (id) VALUES (s.c1)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED THEN " + + " INSERT (id) VALUES (s.c1)", + commitTarget()); + }) + .as("Should complain about missing columns") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("must provide values for all columns of the target table"); } @Test @@ -2085,31 +2091,31 @@ public void testMergeWithInvalidUpdates() { "{ \"id\": 1, \"a\": [ { \"c1\": 2, \"c2\": 3 } ], \"m\": { \"k\": \"v\"} }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about updating an array column", - AnalysisException.class, - "Updating nested fields is only supported for structs", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.a.c1 = s.c2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.a.c1 = s.c2", + commitTarget()); + }) + .as("Should complain about updating an array column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for structs"); - AssertHelpers.assertThrows( - "Should complain about updating a map column", - AnalysisException.class, - "Updating nested fields is only supported for structs", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.m.key = 'new_key'", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.m.key = 'new_key'", + commitTarget()); + }) + .as("Should complain about updating a map column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for structs"); } @Test @@ -2119,44 +2125,44 @@ public void testMergeWithConflictingUpdates() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a top-level column", - AnalysisException.class, - "Updates are in conflict", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.id = 1, t.c.n1 = 2, t.id = 2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.id = 1, t.c.n1 = 2, t.id = 2", + commitTarget()); + }) + .as("Should complain about conflicting updates to a top-level column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updates are in conflict"); - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a nested column", - AnalysisException.class, - "Updates are in conflict for these columns", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", + commitTarget()); + }) + .as("Should complain about conflicting updates to a nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updates are in conflict for these columns"); - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a nested column", - AnalysisException.class, - "Updates are in conflict", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", + commitTarget()); + }) + .as("Should complain about conflicting updates to a nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updates are in conflict"); } @Test @@ -2173,70 +2179,70 @@ public void testMergeWithInvalidAssignments() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), () -> { - AssertHelpers.assertThrows( - "Should complain about writing nulls to a top-level column", - AnalysisException.class, - "Cannot write nullable values to non-null column", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.id = NULL", - commitTarget()); - }); - - AssertHelpers.assertThrows( - "Should complain about writing nulls to a nested column", - AnalysisException.class, - "Cannot write nullable values to non-null column", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s.n1 = NULL", - commitTarget()); - }); - - AssertHelpers.assertThrows( - "Should complain about writing missing fields in structs", - AnalysisException.class, - "missing fields", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s = s.c2", - commitTarget()); - }); - - AssertHelpers.assertThrows( - "Should complain about writing invalid data types", - AnalysisException.class, - "Cannot safely cast", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s.n1 = s.c3", - commitTarget()); - }); - - AssertHelpers.assertThrows( - "Should complain about writing incompatible structs", - AnalysisException.class, - "field name does not match", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.s.n2 = s.c4", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.id = NULL", + commitTarget()); + }) + .as("Should complain about writing nulls to a top-level column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot write nullable values to non-null column"); + + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = NULL", + commitTarget()); + }) + .as("Should complain about writing nulls to a nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot write nullable values to non-null column"); + + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s = s.c2", + commitTarget()); + }) + .as("Should complain about writing missing fields in structs") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("missing fields"); + + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n1 = s.c3", + commitTarget()); + }) + .as("Should complain about writing invalid data types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast"); + + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.s.n2 = s.c4", + commitTarget()); + }) + .as("Should complain about writing incompatible structs") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("field name does not match"); }); } } @@ -2248,57 +2254,57 @@ public void testMergeWithNonDeterministicConditions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about non-deterministic search conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 AND rand() > t.id " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n1 = -1", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 AND rand() > t.id " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget()); + }) + .as("Should complain about non-deterministic search conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Non-deterministic functions are not supported"); - AssertHelpers.assertThrows( - "Should complain about non-deterministic update conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND rand() > t.id THEN " - + " UPDATE SET t.c.n1 = -1", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND rand() > t.id THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget()); + }) + .as("Should complain about non-deterministic update conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Non-deterministic functions are not supported"); - AssertHelpers.assertThrows( - "Should complain about non-deterministic delete conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND rand() > t.id THEN " - + " DELETE", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND rand() > t.id THEN " + + " DELETE", + commitTarget()); + }) + .as("Should complain about non-deterministic delete conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Non-deterministic functions are not supported"); - AssertHelpers.assertThrows( - "Should complain about non-deterministic insert conditions", - AnalysisException.class, - "Non-deterministic functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN NOT MATCHED AND rand() > c1 THEN " - + " INSERT (id, c) VALUES (1, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED AND rand() > c1 THEN " + + " INSERT (id, c) VALUES (1, null)", + commitTarget()); + }) + .as("Should complain about non-deterministic insert conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Non-deterministic functions are not supported"); } @Test @@ -2308,57 +2314,57 @@ public void testMergeWithAggregateExpressions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about agg expressions in search conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 AND max(t.id) == 1 " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n1 = -1", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 AND max(t.id) == 1 " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget()); + }) + .as("Should complain about agg expressions in search conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Agg functions are not supported"); - AssertHelpers.assertThrows( - "Should complain about agg expressions in update conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND sum(t.id) < 1 THEN " - + " UPDATE SET t.c.n1 = -1", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND sum(t.id) < 1 THEN " + + " UPDATE SET t.c.n1 = -1", + commitTarget()); + }) + .as("Should complain about agg expressions in update conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Agg functions are not supported"); - AssertHelpers.assertThrows( - "Should complain about agg expressions in delete conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND sum(t.id) THEN " - + " DELETE", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND sum(t.id) THEN " + + " DELETE", + commitTarget()); + }) + .as("Should complain about agg expressions in delete conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Agg functions are not supported"); - AssertHelpers.assertThrows( - "Should complain about agg expressions in insert conditions", - AnalysisException.class, - "Agg functions are not supported", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN NOT MATCHED AND sum(c1) < 1 THEN " - + " INSERT (id, c) VALUES (1, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED AND sum(c1) < 1 THEN " + + " INSERT (id, c) VALUES (1, null)", + commitTarget()); + }) + .as("Should complain about agg expressions in insert conditions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Agg functions are not supported"); } @Test @@ -2368,57 +2374,57 @@ public void testMergeWithSubqueriesInConditions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 AND t.id < (SELECT max(c2) FROM source) " - + "WHEN MATCHED THEN " - + " UPDATE SET t.c.n1 = s.c2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 AND t.id < (SELECT max(c2) FROM source) " + + "WHEN MATCHED THEN " + + " UPDATE SET t.c.n1 = s.c2", + commitTarget()); + }) + .as("Should complain about subquery expressions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Subqueries are not supported in conditions"); - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND t.id < (SELECT max(c2) FROM source) THEN " - + " UPDATE SET t.c.n1 = s.c2", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND t.id < (SELECT max(c2) FROM source) THEN " + + " UPDATE SET t.c.n1 = s.c2", + commitTarget()); + }) + .as("Should complain about subquery expressions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Subqueries are not supported in conditions"); - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN MATCHED AND t.id NOT IN (SELECT c2 FROM source) THEN " - + " DELETE", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN MATCHED AND t.id NOT IN (SELECT c2 FROM source) THEN " + + " DELETE", + commitTarget()); + }) + .as("Should complain about subquery expressions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Subqueries are not supported in conditions"); - AssertHelpers.assertThrows( - "Should complain about subquery expressions", - AnalysisException.class, - "Subqueries are not supported in conditions", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.c1 " - + "WHEN NOT MATCHED AND s.c1 IN (SELECT c2 FROM source) THEN " - + " INSERT (id, c) VALUES (1, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.c1 " + + "WHEN NOT MATCHED AND s.c1 IN (SELECT c2 FROM source) THEN " + + " INSERT (id, c) VALUES (1, null)", + commitTarget()); + }) + .as("Should complain about subquery expressions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Subqueries are not supported in conditions"); } @Test @@ -2426,18 +2432,18 @@ public void testMergeWithTargetColumnsInInsertConditions() { createAndInitTable("id INT, c2 INT", "{ \"id\": 1, \"c2\": 2 }"); createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); - AssertHelpers.assertThrows( - "Should complain about the target column", - AnalysisException.class, - "Cannot resolve [c2]", - () -> { - sql( - "MERGE INTO %s t USING source s " - + "ON t.id == s.id " - + "WHEN NOT MATCHED AND c2 = 1 THEN " - + " INSERT (id, c2) VALUES (s.id, null)", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO %s t USING source s " + + "ON t.id == s.id " + + "WHEN NOT MATCHED AND c2 = 1 THEN " + + " INSERT (id, c2) VALUES (s.id, null)", + commitTarget()); + }) + .as("Should complain about the target column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot resolve [c2]"); } @Test @@ -2445,17 +2451,17 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "Should complain non iceberg target table", - UnsupportedOperationException.class, - "MERGE INTO TABLE is not supported temporarily.", - () -> { - sql( - "MERGE INTO target t USING source s " - + "ON t.c1 == s.c1 " - + "WHEN MATCHED THEN " - + " UPDATE SET *"); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "MERGE INTO target t USING source s " + + "ON t.c1 == s.c1 " + + "WHEN MATCHED THEN " + + " UPDATE SET *"); + }) + .as("Should complain non iceberg target table") + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("MERGE INTO TABLE is not supported temporarily."); } /** diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 6d8120127586..c156e4c77767 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -25,7 +25,6 @@ import java.util.List; import java.util.Locale; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Table; @@ -38,6 +37,7 @@ import org.apache.iceberg.spark.source.TestSparkCatalog; import org.apache.spark.SparkException; import org.apache.spark.sql.connector.catalog.Identifier; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runners.Parameterized; @@ -115,13 +115,14 @@ public void testCommitUnknownException() { TestSparkCatalog.setTable(ident, sparkTable); // Although an exception is thrown here, write and commit have succeeded - AssertHelpers.assertThrowsWithCause( - "Should throw a Commit State Unknown Exception", - SparkException.class, - "Writing job aborted", - CommitStateUnknownException.class, - "Datacenter on Fire", - () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")); + Assertions.assertThatThrownBy( + () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) + .as("Should throw a Commit State Unknown Exception") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Writing job aborted") + .cause() + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageContaining("Datacenter on Fire"); // Since write and commit succeeded, the rows should be readable assertEquals( diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index b8b5df099089..3a51e87be804 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -131,14 +131,14 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); - AssertHelpers.assertThrows( - "Should reject invalid metrics config", - ValidationException.class, - "Invalid metrics config", - () -> { - String props = "map('write.metadata.metrics.column.x', 'X')"; - sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); - }); + Assertions.assertThatThrownBy( + () -> { + String props = "map('write.metadata.metrics.column.x', 'X')"; + sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); + }) + .as("Should reject invalid metrics config") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Invalid metrics config"); } @Test @@ -166,23 +166,21 @@ public void testMigrateWithConflictingProps() throws IOException { @Test public void testInvalidMigrateCases() { - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.migrate()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.migrate('')", catalogName)); + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } @Test 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 2b74cd475fae..19a065234ed8 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 @@ -22,7 +22,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -32,6 +31,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -157,37 +157,37 @@ public void testApplyWapChangesRefreshesRelationCache() { public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should reject invalid wap id", - ValidationException.class, - "Cannot apply unknown WAP ID", - () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)) + .as("Should reject invalid wap id") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot apply unknown WAP ID"); } @Test public void testInvalidApplyWapChangesCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.publish_changes('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } } 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 670c4e565760..415141968664 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 @@ -34,7 +34,6 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.Files; @@ -227,11 +226,11 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - AssertHelpers.assertThrows( - "Should reject call", - ValidationException.class, - "Cannot delete orphan files: GC is disabled", - () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)) + .as("Should reject call") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot delete orphan files: GC is disabled"); // reset the property to enable the table purging in removeTable. sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'true')", tableName, GC_ENABLED); @@ -258,35 +257,33 @@ public void testRemoveOrphanFilesWap() { @Test public void testInvalidRemoveOrphanFilesCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.remove_orphan_files()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.remove_orphan_files('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } @Test @@ -351,63 +348,63 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes = 0", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, 0)); - - AssertHelpers.assertThrows( - "Should throw an error when max_concurrent_deletes < 0 ", - IllegalArgumentException.class, - "max_concurrent_deletes should have value > 0", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", - catalogName, tableIdent, -1)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, 0)) + .as("Should throw an error when max_concurrent_deletes = 0") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("max_concurrent_deletes should have value > 0"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", + catalogName, tableIdent, -1)) + .as("Should throw an error when max_concurrent_deletes < 0 ") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("max_concurrent_deletes should have value > 0"); String tempViewName = "file_list_test"; spark.emptyDataFrame().createOrReplaceTempView(tempViewName); - AssertHelpers.assertThrows( - "Should throw an error if file_list_view is missing required columns", - IllegalArgumentException.class, - "does not exist. Available:", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", - catalogName, tableIdent, tempViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .as("Should throw an error if file_list_view is missing required columns") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("does not exist. Available:"); spark .createDataset(Lists.newArrayList(), Encoders.tuple(Encoders.INT(), Encoders.TIMESTAMP())) .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - AssertHelpers.assertThrows( - "Should throw an error if file_path has wrong type", - IllegalArgumentException.class, - "Invalid file_path column", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", - catalogName, tableIdent, tempViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .as("Should throw an error if file_path has wrong type") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid file_path column"); spark .createDataset(Lists.newArrayList(), Encoders.tuple(Encoders.STRING(), Encoders.STRING())) .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - AssertHelpers.assertThrows( - "Should throw an error if last_modified has wrong type", - IllegalArgumentException.class, - "Invalid last_modified column", - () -> - sql( - "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", - catalogName, tableIdent, tempViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", + catalogName, tableIdent, tempViewName)) + .as("Should throw an error if last_modified has wrong type") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid last_modified column"); } @Test @@ -679,16 +676,16 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() Assert.assertEquals(0, orphanFiles.size()); // Test with no equal schemes - AssertHelpers.assertThrows( - "Should complain about removing orphan files", - ValidationException.class, - "Conflicting authorities/schemes: [(file1, file)]", - () -> - sql( - "CALL %s.system.remove_orphan_files(" - + "table => '%s'," - + "file_list_view => '%s')", - catalogName, tableIdent, fileListViewName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.remove_orphan_files(" + + "table => '%s'," + + "file_list_view => '%s')", + catalogName, tableIdent, fileListViewName)) + .as("Should complain about removing orphan files") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Conflicting authorities/schemes: [(file1, file)]"); // Drop table in afterEach has purge and fails due to invalid scheme "file1" used in this test // Dropping the table here diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java index b63826e543b8..ed428310b8f4 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -30,6 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -74,11 +74,14 @@ public void testReplaceBranchFailsForTag() throws NoSuchTableException { df.writeTo(tableName).append(); long second = table.currentSnapshot().snapshotId(); - AssertHelpers.assertThrows( - "Cannot perform replace branch on tags", - IllegalArgumentException.class, - "Ref tag1 is a tag not a branch", - () -> sql("ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", tableName, tagName, second)); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, tagName, second)) + .as("Cannot perform replace branch on tags") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref tag1 is a tag not a branch"); } @Test @@ -126,14 +129,14 @@ public void testReplaceBranchDoesNotExist() throws NoSuchTableException { df.writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - AssertHelpers.assertThrows( - "Cannot perform replace branch on branch which does not exist", - IllegalArgumentException.class, - "Branch does not exist", - () -> - sql( - "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", - tableName, "someBranch", table.currentSnapshot().snapshotId())); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", + tableName, "someBranch", table.currentSnapshot().snapshotId())) + .as("Cannot perform replace branch on branch which does not exist") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Branch does not exist"); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java index 02ae0e6ac664..b459f3c24807 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -21,7 +21,6 @@ import java.math.BigDecimal; import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkWriteOptions; import org.apache.iceberg.spark.source.ThreeColumnRecord; @@ -29,6 +28,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -187,20 +187,20 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - AssertHelpers.assertThrows( - "Should reject writes without ordering", - SparkException.class, - "Writing job aborted", - () -> { - try { - inputDF - .writeTo(tableName) - .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") - .append(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + inputDF + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .append(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Should reject writes without ordering") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Writing job aborted"); } @Test 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 80cacbf376ac..4a4e4d7dfd7b 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 @@ -24,7 +24,6 @@ import java.util.List; import java.util.Map; import java.util.stream.IntStream; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; @@ -455,136 +454,137 @@ public void testRewriteDataFilesWithInvalidInputs() { insertData(2); // Test for invalid strategy - AssertHelpers.assertThrows( - "Should reject calls with unsupported strategy error message", - IllegalArgumentException.class, - "unsupported strategy: temp. Only binpack or sort is supported", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " - + "strategy => 'temp')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " + + "strategy => 'temp')", + catalogName, tableIdent)) + .as("Should reject calls with unsupported strategy error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("unsupported strategy: temp. Only binpack or sort is supported"); // Test for sort_order with binpack strategy - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Must use only one rewriter type (bin-pack, sort, zorder)", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " - + "sort_order => 'c1 ASC NULLS FIRST')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " + + "sort_order => 'c1 ASC NULLS FIRST')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Must use only one rewriter type (bin-pack, sort, zorder)"); // Test for sort strategy without any (default/user defined) sort_order - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot sort data without a valid sort order", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot sort data without a valid sort order"); // Test for sort_order with invalid null order - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Unable to parse sortOrder:", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'c1 ASC none')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1 ASC none')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unable to parse sortOrder:"); // Test for sort_order with invalid sort direction - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Unable to parse sortOrder:", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'c1 none NULLS FIRST')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1 none NULLS FIRST')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unable to parse sortOrder:"); // Test for sort_order with invalid column name - AssertHelpers.assertThrows( - "Should reject calls with error message", - ValidationException.class, - "Cannot find field 'col1' in struct:" - + " struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'col1 DESC NULLS FIRST')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'col1 DESC NULLS FIRST')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Cannot find field 'col1' in struct:" + + " struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test with invalid filter column col1 - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot parse predicates in where option: col1 = 3", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse predicates in where option: col1 = 3"); // Test for z_order with invalid column name - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot find column 'col1' in table schema (case sensitive = false): " - + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'zorder(col1)')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'zorder(col1)')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot find column 'col1' in table schema (case sensitive = false): " + + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test for z_order with sort_order - AssertHelpers.assertThrows( - "Should reject calls with error message", - IllegalArgumentException.class, - "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)", - () -> - sql( - "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " - + "sort_order => 'c1,zorder(c2,c3)')", - catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " + + "sort_order => 'c1,zorder(c2,c3)')", + catalogName, tableIdent)) + .as("Should reject calls with error message") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot mix identity sort columns and a Zorder sort expression:" + " c1,zorder(c2,c3)"); } @Test public void testInvalidCasesForRewriteDataFiles() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rewrite_data_files()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject duplicate arg names name", - AnalysisException.class, - "Duplicate procedure argument: table", - () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.rewrite_data_files('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)) + .as("Should reject duplicate arg names name") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Duplicate procedure argument: table"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } @Test 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 2675c1010baa..3afad8a7b839 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 @@ -24,7 +24,6 @@ import java.sql.Timestamp; import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -32,6 +31,7 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -276,41 +276,39 @@ public void testRewriteManifestsCaseInsensitiveArgs() { @Test public void testInvalidRewriteManifestsCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rewrite_manifests()", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject duplicate arg names name", - AnalysisException.class, - "Duplicate procedure argument: table", - () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.rewrite_manifests('')", catalogName)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) + .as("Should not resolve procedures in arbitrary namespaces") + .isInstanceOf(NoSuchProcedureException.class) + .hasMessageContaining("not found"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) + .as("Should reject duplicate arg names name") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Duplicate procedure argument: table"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } @Test 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 af94b456d02e..95d03a0104fd 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 @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -29,6 +28,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -240,58 +240,57 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - AssertHelpers.assertThrows( - "Should reject invalid snapshot id", - ValidationException.class, - "Cannot roll back to unknown snapshot id", - () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)) + .as("Should reject invalid snapshot id") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot roll back to unknown snapshot id"); } @Test public void testInvalidRollbackToSnapshotCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> - sql( - "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", - catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", + catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } } 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 6da3853bbe24..c11c7ab7e421 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 @@ -21,7 +21,6 @@ import java.time.LocalDateTime; import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -29,6 +28,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -255,50 +255,55 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> - sql( - "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", - catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_timestamp(timestamp => %s)", catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with extra args", - AnalysisException.class, - "Too many arguments", - () -> - sql("CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", catalogName, timestamp)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for timestamp: cannot cast", - () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", + catalogName, timestamp)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => %s)", + catalogName, timestamp)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", + catalogName, timestamp)) + .as("Should reject calls with extra args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Too many arguments"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type for timestamp: cannot cast"); } } 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 8a8a974bbebe..d996e4b5c805 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 @@ -22,7 +22,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.Namespace; @@ -31,6 +30,7 @@ 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.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -193,64 +193,63 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { Namespace namespace = tableIdent.namespace(); String tableName = tableIdent.name(); - AssertHelpers.assertThrows( - "Should reject invalid snapshot id", - ValidationException.class, - "Cannot roll back to unknown snapshot id", - () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)); + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)) + .as("Should reject invalid snapshot id") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot roll back to unknown snapshot id"); } @Test public void testInvalidRollbackToSnapshotCases() { - AssertHelpers.assertThrows( - "Should not allow mixed args", - AnalysisException.class, - "Named and positional arguments cannot be mixed", - () -> - sql( - "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", - catalogName)); - - AssertHelpers.assertThrows( - "Should not resolve procedures in arbitrary namespaces", - NoSuchProcedureException.class, - "not found", - () -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type for snapshot_id: cannot cast", - () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", + catalogName)) + .as("Should not allow mixed args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Named and positional arguments cannot be mixed"); + + Assertions.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"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index d8e918d8aadd..d44f8d3721ca 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -20,11 +20,11 @@ import java.io.IOException; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -195,37 +195,34 @@ public void testInvalidSnapshotsCases() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); - AssertHelpers.assertThrows( - "Should reject calls without all required args", - AnalysisException.class, - "Missing required parameters", - () -> sql("CALL %s.system.snapshot('foo')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid arg types", - AnalysisException.class, - "Wrong arg type", - () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with invalid map args", - AnalysisException.class, - "cannot resolve 'map", - () -> - sql( - "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", - catalogName, sourceName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)); - - AssertHelpers.assertThrows( - "Should reject calls with empty table identifier", - IllegalArgumentException.class, - "Cannot handle an empty identifier", - () -> sql("CALL %s.system.snapshot('src', '')", catalogName)); + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) + .as("Should reject calls without all required args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Missing required parameters"); + + Assertions.assertThatThrownBy( + () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)) + .as("Should reject calls with invalid arg types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Wrong arg type"); + + Assertions.assertThatThrownBy( + () -> + sql( + "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", + catalogName, sourceName)) + .as("Should reject calls with invalid map args") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("cannot resolve 'map"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); + + Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) + .as("Should reject calls with empty table identifier") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot handle an empty identifier"); } } 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 866a965e33e6..3dadadd9c4de 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 @@ -24,7 +24,6 @@ import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; @@ -97,29 +96,29 @@ public void testCreateTagWithRetain() throws NoSuchTableException { } String tagName = "t1"; - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input", - () -> - sql( - "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", - tableName, tagName, firstSnapshotId, maxRefAge)); - - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input", - () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")); - - AssertHelpers.assertThrows( - "Illegal statement", - IcebergParseException.class, - "mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}", - () -> - sql( - "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d SECONDS", - tableName, tagName, firstSnapshotId, maxRefAge)); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", + tableName, tagName, firstSnapshotId, maxRefAge)) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input"); + + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input"); + + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d SECONDS", + tableName, tagName, firstSnapshotId, maxRefAge)) + .as("Illegal statement") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input 'SECONDS' expecting {'DAYS', 'HOURS', 'MINUTES'}"); } @Test @@ -137,11 +136,11 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { long snapshotId = table.currentSnapshot().snapshotId(); String tagName = "t1"; - AssertHelpers.assertThrows( - "unknown snapshot", - ValidationException.class, - "unknown snapshot: -1", - () -> sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, -1)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, -1)) + .as("unknown snapshot") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("unknown snapshot: -1"); sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName); table.refresh(); @@ -151,17 +150,15 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { Assert.assertNull( "The tag needs to have the default max ref age, which is null.", ref.maxRefAgeMs()); - AssertHelpers.assertThrows( - "Cannot create an exist tag", - IllegalArgumentException.class, - "already exists", - () -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) + .as("Cannot create an exist tag") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("already exists"); - AssertHelpers.assertThrows( - "Non-conforming tag name", - IcebergParseException.class, - "mismatched input '123'", - () -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) + .as("Non-conforming tag name") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input '123'"); table.manageSnapshots().removeTag(tagName).commit(); List records = @@ -207,11 +204,11 @@ public void testReplaceTagFailsForBranch() throws NoSuchTableException { insertRows(); long second = table.currentSnapshot().snapshotId(); - AssertHelpers.assertThrows( - "Cannot perform replace tag on branches", - IllegalArgumentException.class, - "Ref branch1 is a branch not a tag", - () -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) + .as("Cannot perform replace tag on branches") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref branch1 is a branch not a tag"); } @Test @@ -244,14 +241,14 @@ public void testReplaceTag() throws NoSuchTableException { public void testReplaceTagDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - AssertHelpers.assertThrows( - "Cannot perform replace tag on tag which does not exist", - IllegalArgumentException.class, - "Tag does not exist", - () -> - sql( - "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", - tableName, "someTag", table.currentSnapshot().snapshotId())); + Assertions.assertThatThrownBy( + () -> + sql( + "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", + tableName, "someTag", table.currentSnapshot().snapshotId())) + .as("Cannot perform replace tag on tag which does not exist") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Tag does not exist"); } @Test @@ -316,20 +313,19 @@ public void testDropTag() throws NoSuchTableException { @Test public void testDropTagNonConformingName() { - AssertHelpers.assertThrows( - "Non-conforming tag name", - IcebergParseException.class, - "mismatched input '123'", - () -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) + .as("Non-conforming tag name") + .isInstanceOf(IcebergParseException.class) + .hasMessageContaining("mismatched input '123'"); } @Test public void testDropTagDoesNotExist() { - AssertHelpers.assertThrows( - "Cannot perform drop tag on tag which does not exist", - IllegalArgumentException.class, - "Tag does not exist: nonExistingTag", - () -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) + .as("Cannot perform drop tag on tag which does not exist") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Tag does not exist: nonExistingTag"); } @Test @@ -338,11 +334,10 @@ public void testDropTagFailesForBranch() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); - AssertHelpers.assertThrows( - "Cannot perform drop tag on branch", - IllegalArgumentException.class, - "Ref b1 is a branch not a tag", - () -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) + .as("Cannot perform drop tag on branch") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Ref b1 is a branch not a tag"); } @Test 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 8093e6fc0984..76c7a171f603 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 @@ -42,7 +42,6 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.Snapshot; @@ -1149,17 +1148,15 @@ public void testUpdateWithInvalidUpdates() { "id INT, a ARRAY>, m MAP", "{ \"id\": 0, \"a\": null, \"m\": null }"); - AssertHelpers.assertThrows( - "Should complain about updating an array column", - AnalysisException.class, - "Updating nested fields is only supported for structs", - () -> sql("UPDATE %s SET a.c1 = 1", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about updating a map column", - AnalysisException.class, - "Updating nested fields is only supported for structs", - () -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())); + Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) + .as("Should complain about updating an array column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for structs"); + + Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) + .as("Should complain about updating a map column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updating nested fields is only supported for structs"); } @Test @@ -1167,27 +1164,27 @@ public void testUpdateWithConflictingAssignments() { createAndInitTable( "id INT, c STRUCT>", "{ \"id\": 0, \"s\": null }"); - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a top-level column", - AnalysisException.class, - "Updates are in conflict", - () -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a nested column", - AnalysisException.class, - "Updates are in conflict for these columns", - () -> sql("UPDATE %s t SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about conflicting updates to a nested column", - AnalysisException.class, - "Updates are in conflict", - () -> { - sql( - "UPDATE %s SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", - commitTarget()); - }); + Assertions.assertThatThrownBy( + () -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) + .as("Should complain about conflicting updates to a top-level column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updates are in conflict"); + + Assertions.assertThatThrownBy( + () -> sql("UPDATE %s t SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", commitTarget())) + .as("Should complain about conflicting updates to a nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updates are in conflict for these columns"); + + Assertions.assertThatThrownBy( + () -> { + sql( + "UPDATE %s SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", + commitTarget()); + }) + .as("Should complain about conflicting updates to a nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Updates are in conflict"); } @Test @@ -1200,38 +1197,37 @@ public void testUpdateWithInvalidAssignments() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), () -> { - AssertHelpers.assertThrows( - "Should complain about writing nulls to a top-level column", - AnalysisException.class, - "Cannot write nullable values to non-null column", - () -> sql("UPDATE %s t SET t.id = NULL", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about writing nulls to a nested column", - AnalysisException.class, - "Cannot write nullable values to non-null column", - () -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about writing missing fields in structs", - AnalysisException.class, - "missing fields", - () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about writing invalid data types", - AnalysisException.class, - "Cannot safely cast", - () -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())); - - AssertHelpers.assertThrows( - "Should complain about writing incompatible structs", - AnalysisException.class, - "field name does not match", - () -> - sql( - "UPDATE %s t SET t.s.n2 = named_struct('dn2', 1, 'dn1', 2)", - commitTarget())); + Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + .as("Should complain about writing nulls to a top-level column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot write nullable values to non-null column"); + + Assertions.assertThatThrownBy( + () -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + .as("Should complain about writing nulls to a nested column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot write nullable values to non-null column"); + + Assertions.assertThatThrownBy( + () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) + .as("Should complain about writing missing fields in structs") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("missing fields"); + + Assertions.assertThatThrownBy( + () -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + .as("Should complain about writing invalid data types") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot safely cast"); + + Assertions.assertThatThrownBy( + () -> + sql( + "UPDATE %s t SET t.s.n2 = named_struct('dn2', 1, 'dn1', 2)", + commitTarget())) + .as("Should complain about writing incompatible structs") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("field name does not match"); }); } } @@ -1240,22 +1236,21 @@ public void testUpdateWithInvalidAssignments() { public void testUpdateWithNonDeterministicCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); - AssertHelpers.assertThrows( - "Should complain about non-deterministic expressions", - AnalysisException.class, - "nondeterministic expressions are only allowed", - () -> sql("UPDATE %s SET id = -1 WHERE id = 1 AND rand() > 0.5", commitTarget())); + Assertions.assertThatThrownBy( + () -> sql("UPDATE %s SET id = -1 WHERE id = 1 AND rand() > 0.5", commitTarget())) + .as("Should complain about non-deterministic expressions") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("nondeterministic expressions are only allowed"); } @Test public void testUpdateOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - AssertHelpers.assertThrows( - "UPDATE is not supported for non iceberg table", - UnsupportedOperationException.class, - "not supported temporarily", - () -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")); + Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) + .as("UPDATE is not supported for non iceberg table") + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("not supported temporarily"); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 24a9b3f232e0..2966e95fa3de 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -21,7 +21,6 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.TableProperties; import org.apache.iceberg.hadoop.HadoopFileIO; @@ -39,6 +38,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Rule; import org.junit.Test; @@ -105,23 +105,23 @@ public void testBatchAppend() throws Exception { new SimpleRecord(4, "b")); Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); - AssertHelpers.assertThrows( - "Write must fail", - SparkException.class, - "Writing job aborted", - () -> { - try { - // incoming records are not ordered by partitions so the job must fail - inputDF - .coalesce(1) - .sortWithinPartitions("id") - .writeTo(tableName) - .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") - .append(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + // incoming records are not ordered by partitions so the job must fail + inputDF + .coalesce(1) + .sortWithinPartitions("id") + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .append(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Write must fail") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Writing job aborted"); assertEquals("Should be no records", sql("SELECT * FROM %s", tableName), ImmutableList.of()); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java index 9dcd59cb4f11..d224e175d3b0 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.IcebergBuild; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.functions.IcebergVersionFunction; @@ -68,11 +67,10 @@ public void testListFunctionsViaCatalog() throws NoSuchNamespaceException { new Identifier[0], asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE)); - AssertHelpers.assertThrows( - "Listing functions in a namespace that does not exist should throw", - NoSuchNamespaceException.class, - "Namespace 'db' not found", - () -> asFunctionCatalog.listFunctions(DB_NAMESPACE)); + Assertions.assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE)) + .as("Listing functions in a namespace that does not exist should throw") + .isInstanceOf(NoSuchNamespaceException.class) + .hasMessageContaining("Namespace 'db' not found"); } @Test @@ -87,24 +85,24 @@ public void testLoadFunctions() throws NoSuchFunctionException { .isExactlyInstanceOf(IcebergVersionFunction.class); } - AssertHelpers.assertThrows( - "Cannot load a function if it's not used with the system namespace or the empty namespace", - NoSuchFunctionException.class, - "Undefined function: default.iceberg_version", - () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))); + Assertions.assertThatThrownBy( + () -> + asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))) + .as( + "Cannot load a function if it's not used with the system namespace or the empty namespace") + .isInstanceOf(NoSuchFunctionException.class) + .hasMessageContaining("Undefined function: default.iceberg_version"); Identifier undefinedFunction = Identifier.of(SYSTEM_NAMESPACE, "undefined_function"); - AssertHelpers.assertThrows( - "Cannot load a function that does not exist", - NoSuchFunctionException.class, - "Undefined function: system.undefined_function", - () -> asFunctionCatalog.loadFunction(undefinedFunction)); - - AssertHelpers.assertThrows( - "Using an undefined function from SQL should fail analysis", - AnalysisException.class, - "Undefined function", - () -> sql("SELECT undefined_function(1, 2)")); + Assertions.assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) + .as("Cannot load a function that does not exist") + .isInstanceOf(NoSuchFunctionException.class) + .hasMessageContaining("Undefined function: system.undefined_function"); + + Assertions.assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) + .as("Using an undefined function from SQL should fail analysis") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Undefined function"); } @Test 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 0e333f216c26..5b1cf8281784 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 @@ -27,7 +27,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.StreamSupport; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.DeleteFile; @@ -51,6 +50,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -362,22 +362,22 @@ public void testIgnoreMetadataFilesNotFound() { public void testEmptyIOThrowsException() { DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(null); - AssertHelpers.assertThrows( - "FileIO can't be null in DeleteReachableFiles action", - IllegalArgumentException.class, - "File IO cannot be null", - baseRemoveFilesSparkAction::execute); + Assertions.assertThatThrownBy(baseRemoveFilesSparkAction::execute) + .as("FileIO can't be null in DeleteReachableFiles action") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("File IO cannot be null"); } @Test public void testRemoveFilesActionWhenGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - AssertHelpers.assertThrows( - "Should complain about removing files when GC is disabled", - ValidationException.class, - "Cannot delete files: GC is disabled (deleting files may corrupt other tables)", - () -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()); + Assertions.assertThatThrownBy( + () -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) + .as("Should complain about removing files when GC is disabled") + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Cannot delete files: GC is disabled (deleting files may corrupt other tables)"); } private String metadataLocation(Table tbl) { 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 c8bef4a2a973..a3580bd2d980 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 @@ -29,7 +29,6 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -57,6 +56,7 @@ import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -447,11 +447,10 @@ public void testExpireSnapshotsWithDisabledGarbageCollection() { table.newAppend().appendFile(FILE_A).commit(); - AssertHelpers.assertThrows( - "Should complain about expiring snapshots", - ValidationException.class, - "Cannot expire snapshots: GC is disabled", - () -> SparkActions.get().expireSnapshots(table)); + Assertions.assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) + .as("Should complain about expiring snapshots") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot expire snapshots: GC is disabled"); } @Test @@ -529,11 +528,13 @@ public void testRetainLastMultipleCalls() { @Test public void testRetainZeroSnapshots() { - AssertHelpers.assertThrows( - "Should fail retain 0 snapshots " + "because number of snapshots to retain cannot be zero", - IllegalArgumentException.class, - "Number of snapshots to retain must be at least 1, cannot be: 0", - () -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()); + Assertions.assertThatThrownBy( + () -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) + .as( + "Should fail retain 0 snapshots " + + "because number of snapshots to retain cannot be zero") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Number of snapshots to retain must be at least 1, cannot be: 0"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index c94aec6f5c19..10af2454fa4a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -40,7 +40,6 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Files; import org.apache.iceberg.GenericBlobMetadata; import org.apache.iceberg.GenericStatisticsFile; @@ -755,11 +754,10 @@ public void testGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - AssertHelpers.assertThrows( - "Should complain about removing orphan files", - ValidationException.class, - "Cannot delete orphan files: GC is disabled", - () -> SparkActions.get().deleteOrphanFiles(table).execute()); + Assertions.assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) + .as("Should complain about removing orphan files") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot delete orphan files: GC is disabled"); } @Test @@ -991,18 +989,18 @@ public void testPathsWithActualFileHavingNoAuthority() { public void testPathsWithEqualSchemes() { List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); - AssertHelpers.assertThrows( - "Test remove orphan files with equal schemes", - ValidationException.class, - "Conflicting authorities/schemes: [(scheme1, scheme2)]", - () -> - executeTest( - validFiles, - actualFiles, - Lists.newArrayList(), - ImmutableMap.of(), - ImmutableMap.of(), - DeleteOrphanFiles.PrefixMismatchMode.ERROR)); + Assertions.assertThatThrownBy( + () -> + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR)) + .as("Test remove orphan files with equal schemes") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Conflicting authorities/schemes: [(scheme1, scheme2)]"); Map equalSchemes = Maps.newHashMap(); equalSchemes.put("scheme1", "scheme"); @@ -1020,18 +1018,18 @@ public void testPathsWithEqualSchemes() { public void testPathsWithEqualAuthorities() { List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); - AssertHelpers.assertThrows( - "Test remove orphan files with equal authorities", - ValidationException.class, - "Conflicting authorities/schemes: [(servicename1, servicename2)]", - () -> - executeTest( - validFiles, - actualFiles, - Lists.newArrayList(), - ImmutableMap.of(), - ImmutableMap.of(), - DeleteOrphanFiles.PrefixMismatchMode.ERROR)); + Assertions.assertThatThrownBy( + () -> + executeTest( + validFiles, + actualFiles, + Lists.newArrayList(), + ImmutableMap.of(), + ImmutableMap.of(), + DeleteOrphanFiles.PrefixMismatchMode.ERROR)) + .as("Test remove orphan files with equal authorities") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Conflicting authorities/schemes: [(servicename1, servicename2)]"); Map equalAuthorities = Maps.newHashMap(); equalAuthorities.put("servicename1", "servicename"); 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 cfa1c9da951e..5e8865dff2da 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 @@ -47,7 +47,6 @@ import java.util.stream.IntStream; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; @@ -643,10 +642,10 @@ public void testSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - AssertHelpers.assertThrows( - "Should fail entire rewrite if part fails", - RuntimeException.class, - () -> spyRewrite.execute()); + Assertions.assertThatThrownBy(() -> spyRewrite.execute()) + .as("Should fail entire rewrite if part fails") + .isInstanceOf(RuntimeException.class) + .hasMessage("Rewrite Failed"); table.refresh(); @@ -678,10 +677,10 @@ public void testSingleCommitWithCommitFailure() { doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); - AssertHelpers.assertThrows( - "Should fail entire rewrite if commit fails", - RuntimeException.class, - () -> spyRewrite.execute()); + Assertions.assertThatThrownBy(() -> spyRewrite.execute()) + .as("Should fail entire rewrite if commit fails") + .isInstanceOf(RuntimeException.class) + .hasMessage("Commit Failure"); table.refresh(); @@ -714,10 +713,10 @@ public void testParallelSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - AssertHelpers.assertThrows( - "Should fail entire rewrite if part fails", - RuntimeException.class, - () -> spyRewrite.execute()); + Assertions.assertThatThrownBy(() -> spyRewrite.execute()) + .as("Should fail entire rewrite if part fails") + .isInstanceOf(RuntimeException.class) + .hasMessage("Rewrite Failed"); table.refresh(); @@ -859,32 +858,39 @@ public void testParallelPartialProgressWithCommitFailure() { public void testInvalidOptions() { Table table = createTable(20); - AssertHelpers.assertThrows( - "No negative values for partial progress max commits", - IllegalArgumentException.class, - () -> - basicRewrite(table) - .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") - .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5") - .execute()); - - AssertHelpers.assertThrows( - "No negative values for max concurrent groups", - IllegalArgumentException.class, - () -> - basicRewrite(table) - .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") - .execute()); - - AssertHelpers.assertThrows( - "No unknown options allowed", - IllegalArgumentException.class, - () -> basicRewrite(table).option("foobarity", "-5").execute()); - - AssertHelpers.assertThrows( - "Cannot set rewrite-job-order to foo", - IllegalArgumentException.class, - () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()); + Assertions.assertThatThrownBy( + () -> + basicRewrite(table) + .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") + .option(RewriteDataFiles.PARTIAL_PROGRESS_MAX_COMMITS, "-5") + .execute()) + .as("No negative values for partial progress max commits") + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set partial-progress.max-commits to -5, " + + "the value must be positive when partial-progress.enabled is true"); + + Assertions.assertThatThrownBy( + () -> + basicRewrite(table) + .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") + .execute()) + .as("No negative values for max concurrent groups") + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot set max-concurrent-file-group-rewrites to -5, the value must be positive."); + + Assertions.assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) + .as("No unknown options allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot use options [foobarity], they are not supported by the action or the rewriter BIN-PACK"); + + Assertions.assertThatThrownBy( + () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()) + .as("Cannot set rewrite-job-order to foo") + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid rewrite job order name: foo"); } @Test @@ -1120,10 +1126,11 @@ public void testCommitStateUnknownException() { doReturn(util).when(spyAction).commitManager(table.currentSnapshot().snapshotId()); - AssertHelpers.assertThrows( - "Should propagate CommitStateUnknown Exception", - CommitStateUnknownException.class, - () -> spyAction.execute()); + Assertions.assertThatThrownBy(() -> spyAction.execute()) + .as("Should propagate CommitStateUnknown Exception") + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith( + "Unknown State\n" + "Cannot determine whether the commit was successful or not"); List postRewriteData = currentData(); assertEquals("We shouldn't have changed the data", originalData, postRewriteData); @@ -1239,23 +1246,20 @@ public void testInvalidAPIUsage() { SortOrder sortOrder = SortOrder.builderFor(table.schema()).asc("c2").build(); - AssertHelpers.assertThrows( - "Should be unable to set Strategy more than once", - IllegalArgumentException.class, - "Must use only one rewriter type", - () -> actions().rewriteDataFiles(table).binPack().sort()); - - AssertHelpers.assertThrows( - "Should be unable to set Strategy more than once", - IllegalArgumentException.class, - "Must use only one rewriter type", - () -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()); - - AssertHelpers.assertThrows( - "Should be unable to set Strategy more than once", - IllegalArgumentException.class, - "Must use only one rewriter type", - () -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()); + Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) + .as("Should be unable to set Strategy more than once") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Must use only one rewriter type"); + + Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + .as("Should be unable to set Strategy more than once") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Must use only one rewriter type"); + + Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + .as("Should be unable to set Strategy more than once") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Must use only one rewriter type"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index a5dd0054da25..339bb1db328e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -35,7 +35,6 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.PartitionSpec; @@ -59,6 +58,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.TableIdentifier; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -231,11 +231,12 @@ public void testRewriteManifestsWithCommitStateUnknownException() { Table spyTable = spy(table); when(spyTable.rewriteManifests()).thenReturn(spyNewRewriteManifests); - AssertHelpers.assertThrowsCause( - "Should throw a Commit State Unknown Exception", - RuntimeException.class, - "Datacenter on Fire", - () -> actions.rewriteManifests(spyTable).rewriteIf(manifest -> true).execute()); + Assertions.assertThatThrownBy( + () -> actions.rewriteManifests(spyTable).rewriteIf(manifest -> true).execute()) + .as("Should throw a Commit State Unknown Exception") + .cause() + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Datacenter on Fire"); table.refresh(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index 53099eefa40c..866bbd240aa5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -27,7 +27,6 @@ import java.security.SecureRandom; import java.util.Iterator; import org.apache.avro.generic.GenericData; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Files; import org.apache.iceberg.Schema; import org.apache.iceberg.io.CloseableIterable; @@ -49,6 +48,7 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.apache.spark.sql.vectorized.ColumnarBatch; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Ignore; @@ -255,18 +255,18 @@ public void testMixedTypes() {} @Test @Override public void testNestedStruct() { - AssertHelpers.assertThrows( - "Vectorized reads are not supported yet for struct fields", - UnsupportedOperationException.class, - "Vectorized reads are not supported yet for struct fields", - () -> - VectorizedSparkParquetReaders.buildReader( - TypeUtil.assignIncreasingFreshIds( - new Schema(required(1, "struct", SUPPORTED_PRIMITIVES))), - new MessageType( - "struct", new GroupType(Type.Repetition.OPTIONAL, "struct").withId(1)), - Maps.newHashMap(), - null)); + Assertions.assertThatThrownBy( + () -> + VectorizedSparkParquetReaders.buildReader( + TypeUtil.assignIncreasingFreshIds( + new Schema(required(1, "struct", SUPPORTED_PRIMITIVES))), + new MessageType( + "struct", new GroupType(Type.Repetition.OPTIONAL, "struct").withId(1)), + Maps.newHashMap(), + null)) + .as("Vectorized reads are not supported yet for struct fields") + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Vectorized reads are not supported yet for struct fields"); } @Test @@ -390,13 +390,10 @@ public void testUnsupportedReadsForParquetV2() throws Exception { try (FileAppender writer = parquetV2Writer(schema, dataFile)) { writer.addAll(data); } - AssertHelpers.assertThrows( - "Vectorized reads not supported", - UnsupportedOperationException.class, - "Cannot support vectorized reads for column", - () -> { - assertRecordsMatch(schema, 30000, data, dataFile, true, BATCH_SIZE); - return null; - }); + Assertions.assertThatThrownBy( + () -> assertRecordsMatch(schema, 30000, data, dataFile, true, BATCH_SIZE)) + .as("Vectorized reads not supported") + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Cannot support vectorized reads for column"); } } 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 a9b4f0d3ad2f..f57c87e8b809 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 @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.source; import java.util.List; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.Spark3Util; @@ -33,6 +32,7 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.internal.SQLConf; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -76,18 +76,18 @@ public void testMergeSchemaFailsWithoutWriterOption() throws Exception { // this has a different error message than the case without accept-any-schema because it uses // Iceberg checks - AssertHelpers.assertThrows( - "Should fail when merge-schema is not enabled on the writer", - IllegalArgumentException.class, - "Field new_col not found in source schema", - () -> { - try { - threeColDF.writeTo(tableName).append(); - } catch (NoSuchTableException e) { - // needed because append has checked exceptions - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + threeColDF.writeTo(tableName).append(); + } catch (NoSuchTableException e) { + // needed because append has checked exceptions + throw new RuntimeException(e); + } + }) + .as("Should fail when merge-schema is not enabled on the writer") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Field new_col not found in source schema"); } @Test @@ -111,18 +111,18 @@ public void testMergeSchemaWithoutAcceptAnySchema() throws Exception { "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); - AssertHelpers.assertThrows( - "Should fail when accept-any-schema is not enabled on the table", - AnalysisException.class, - "too many data columns", - () -> { - try { - threeColDF.writeTo(tableName).option("merge-schema", "true").append(); - } catch (NoSuchTableException e) { - // needed because append has checked exceptions - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + threeColDF.writeTo(tableName).option("merge-schema", "true").append(); + } catch (NoSuchTableException e) { + // needed because append has checked exceptions + throw new RuntimeException(e); + } + }) + .as("Should fail when accept-any-schema is not enabled on the table") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("too many data columns"); } @Test 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 7e254960e759..2c3a7aba59d1 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 @@ -25,7 +25,6 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; @@ -246,50 +245,52 @@ public void testIncrementalScanOptions() throws IOException { List snapshotIds = SnapshotUtil.currentAncestorIds(table); // start-snapshot-id and snapshot-id are both configured. - AssertHelpers.assertThrows( - "Check both start-snapshot-id and snapshot-id are configured", - IllegalArgumentException.class, - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans", - () -> { - spark - .read() - .format("iceberg") - .option("snapshot-id", snapshotIds.get(3).toString()) - .option("start-snapshot-id", snapshotIds.get(3).toString()) - .load(tableLocation) - .explain(); - }); + Assertions.assertThatThrownBy( + () -> { + spark + .read() + .format("iceberg") + .option("snapshot-id", snapshotIds.get(3).toString()) + .option("start-snapshot-id", snapshotIds.get(3).toString()) + .load(tableLocation) + .explain(); + }) + .as("Check both start-snapshot-id and snapshot-id are configured") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans"); // end-snapshot-id and as-of-timestamp are both configured. - AssertHelpers.assertThrows( - "Check both start-snapshot-id and snapshot-id are configured", - IllegalArgumentException.class, - "Cannot set start-snapshot-id and end-snapshot-id for incremental scans", - () -> { - spark - .read() - .format("iceberg") - .option( - SparkReadOptions.AS_OF_TIMESTAMP, - Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) - .option("end-snapshot-id", snapshotIds.get(2).toString()) - .load(tableLocation) - .explain(); - }); + Assertions.assertThatThrownBy( + () -> { + spark + .read() + .format("iceberg") + .option( + SparkReadOptions.AS_OF_TIMESTAMP, + Long.toString(table.snapshot(snapshotIds.get(3)).timestampMillis())) + .option("end-snapshot-id", snapshotIds.get(2).toString()) + .load(tableLocation) + .explain(); + }) + .as("Check both start-snapshot-id and snapshot-id are configured") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Cannot set start-snapshot-id and end-snapshot-id for incremental scans"); // only end-snapshot-id is configured. - AssertHelpers.assertThrows( - "Check both start-snapshot-id and snapshot-id are configured", - IllegalArgumentException.class, - "Cannot set only end-snapshot-id for incremental scans", - () -> { - spark - .read() - .format("iceberg") - .option("end-snapshot-id", snapshotIds.get(2).toString()) - .load(tableLocation) - .explain(); - }); + Assertions.assertThatThrownBy( + () -> { + spark + .read() + .format("iceberg") + .option("end-snapshot-id", snapshotIds.get(2).toString()) + .load(tableLocation) + .explain(); + }) + .as("Check both start-snapshot-id and snapshot-id are configured") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot set only end-snapshot-id for incremental scans"); // test (1st snapshot, current snapshot] incremental scan. List result = diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 6ab9e57949e3..8630c50151f3 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -28,7 +28,6 @@ import java.util.concurrent.TimeoutException; import org.apache.avro.generic.GenericData; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; @@ -55,6 +54,7 @@ import org.apache.spark.sql.execution.streaming.MemoryStream; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -120,16 +120,16 @@ public void testSparkWriteFailsUnknownTransform() throws IOException { Dataset df = spark.createDataFrame(expected, SimpleRecord.class); - AssertHelpers.assertThrows( - "Should reject write with unsupported transform", - UnsupportedOperationException.class, - "Cannot write using unsupported transforms: zero", - () -> - df.select("id", "data") - .write() - .format("iceberg") - .mode("append") - .save(location.toString())); + Assertions.assertThatThrownBy( + () -> + df.select("id", "data") + .write() + .format("iceberg") + .mode("append") + .save(location.toString())) + .as("Should reject write with unsupported transform") + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining("Cannot write using unsupported transforms: zero"); } @Test @@ -159,11 +159,10 @@ public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, T List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - AssertHelpers.assertThrows( - "Should reject streaming write with unsupported transform", - StreamingQueryException.class, - "Cannot write using unsupported transforms: zero", - query::processAllAvailable); + Assertions.assertThatThrownBy(query::processAllAvailable) + .as("Should reject streaming write with unsupported transform") + .isInstanceOf(StreamingQueryException.class) + .hasMessageContaining("Cannot write using unsupported transforms: zero"); } @Test 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 111da882fe8a..079ca5f77fa4 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 @@ -38,7 +38,6 @@ import java.util.stream.StreamSupport; import org.apache.avro.generic.GenericData; import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.Files; @@ -1119,17 +1118,17 @@ public void testPruneManifestsTable() { if (!spark.version().startsWith("2")) { // Spark 2 isn't able to actually push down nested struct projections so this will not break - AssertHelpers.assertThrows( - "Can't prune struct inside list", - SparkException.class, - "Cannot project a partial list element struct", - () -> - spark - .read() - .format("iceberg") - .load(loadLocation(tableIdentifier, "manifests")) - .select("partition_spec_id", "path", "partition_summaries.contains_null") - .collectAsList()); + Assertions.assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "manifests")) + .select("partition_spec_id", "path", "partition_summaries.contains_null") + .collectAsList()) + .as("Can't prune struct inside list") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Cannot project a partial list element struct"); } Dataset actualDf = diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index 0baaef1374d4..9c5d48e1debb 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -33,7 +33,6 @@ import java.util.Arrays; import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; @@ -54,6 +53,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -641,11 +641,10 @@ public void testMetadataTablesWithUnknownTransforms() { sql("REFRESH TABLE %s", tableName); for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES, ENTRIES, ALL_ENTRIES)) { - AssertHelpers.assertThrows( - "Should complain about the partition type", - ValidationException.class, - "Cannot build table partition type, unknown transforms", - () -> loadMetadataTable(tableType)); + Assertions.assertThatThrownBy(() -> loadMetadataTable(tableType)) + .as("Should complain about the partition type") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot build table partition type, unknown transforms"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java index 8e54a23f815a..e8e402ac44cc 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -30,6 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -163,20 +163,20 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - AssertHelpers.assertThrows( - "Should reject writes without ordering", - SparkException.class, - "Writing job aborted", - () -> { - try { - inputDF - .writeTo(tableName) - .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") - .append(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + inputDF + .writeTo(tableName) + .option(SparkWriteOptions.USE_TABLE_DISTRIBUTION_AND_ORDERING, "false") + .append(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Should reject writes without ordering") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Writing job aborted"); } @Test @@ -233,17 +233,17 @@ public void testNoSortBucketTransformsWithoutExtensions() throws NoSuchTableExce Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail by default as extensions are disabled - AssertHelpers.assertThrows( - "Should reject writes without ordering", - SparkException.class, - "Writing job aborted", - () -> { - try { - inputDF.writeTo(tableName).append(); - } catch (NoSuchTableException e) { - throw new RuntimeException(e); - } - }); + Assertions.assertThatThrownBy( + () -> { + try { + inputDF.writeTo(tableName).append(); + } catch (NoSuchTableException e) { + throw new RuntimeException(e); + } + }) + .as("Should reject writes without ordering") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Writing job aborted"); inputDF.writeTo(tableName).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 551dc961e309..3d759759f608 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -30,7 +30,6 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.ManifestFile; @@ -691,20 +690,21 @@ public void testCommitUnknownException() throws IOException { ManualSource.setTable(manualTableName, sparkTable); // Although an exception is thrown here, write and commit have succeeded - AssertHelpers.assertThrowsWithCause( - "Should throw a Commit State Unknown Exception", - SparkException.class, - "Writing job aborted", - CommitStateUnknownException.class, - "Datacenter on Fire", - () -> - df2.select("id", "data") - .sort("data") - .write() - .format("org.apache.iceberg.spark.source.ManualSource") - .option(ManualSource.TABLE_NAME, manualTableName) - .mode(SaveMode.Append) - .save(targetLocation)); + Assertions.assertThatThrownBy( + () -> + df2.select("id", "data") + .sort("data") + .write() + .format("org.apache.iceberg.spark.source.ManualSource") + .option(ManualSource.TABLE_NAME, manualTableName) + .mode(SaveMode.Append) + .save(targetLocation)) + .as("Should throw a Commit State Unknown Exception") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Writing job aborted") + .cause() + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageContaining("Datacenter on Fire"); // Since write and commit succeeded, the rows should be readable Dataset result = spark.read().format("iceberg").load(targetLocation); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 5c7929112fb9..2b0033fdcbb5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -30,7 +30,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; @@ -53,6 +52,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -224,11 +224,10 @@ public void testPartitionMetadataColumnWithUnknownTransforms() { TableMetadata base = ops.current(); ops.commit(base, base.updatePartitionSpec(UNKNOWN_SPEC)); - AssertHelpers.assertThrows( - "Should fail to query the partition metadata column", - ValidationException.class, - "Cannot build table partition type, unknown transforms", - () -> sql("SELECT _partition FROM %s", TABLE_NAME)); + Assertions.assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) + .as("Should fail to query the partition metadata column") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Cannot build table partition type, unknown transforms"); } @Test @@ -246,11 +245,10 @@ public void testConflictingColumns() { ImmutableList.of(row(1L, "a1")), sql("SELECT id, category FROM %s", TABLE_NAME)); - AssertHelpers.assertThrows( - "Should fail to query conflicting columns", - ValidationException.class, - "column names conflict", - () -> sql("SELECT * FROM %s", TABLE_NAME)); + Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) + .as("Should fail to query conflicting columns") + .isInstanceOf(ValidationException.class) + .hasMessageContaining("column names conflict"); table.refresh(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index e5ebd327db7a..c8e37d932290 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -29,7 +29,6 @@ import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.IntStream; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.BaseTable; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -495,11 +494,12 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception StreamingQuery query = startStream(); - AssertHelpers.assertThrowsCause( - "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND", - IllegalStateException.class, - "Cannot process overwrite snapshot", - () -> query.processAllAvailable()); + Assertions.assertThatThrownBy(() -> query.processAllAvailable()) + .as( + "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND") + .cause() + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Cannot process overwrite snapshot"); } @Test @@ -536,11 +536,12 @@ public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { StreamingQuery query = startStream(); - AssertHelpers.assertThrowsCause( - "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND", - IllegalStateException.class, - "Cannot process delete snapshot", - () -> query.processAllAvailable()); + Assertions.assertThatThrownBy(() -> query.processAllAvailable()) + .as( + "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND") + .cause() + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("Cannot process delete snapshot"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java index 053f6dbaea46..d85e1954950d 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java @@ -28,7 +28,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; @@ -53,6 +52,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -159,21 +159,22 @@ public void testUnpartitionedTimestampWithoutZoneProjection() { @Test public void testUnpartitionedTimestampWithoutZoneError() { - AssertHelpers.assertThrows( - String.format( - "Read operation performed on a timestamp without timezone field while " - + "'%s' set to false should throw exception", - SparkReadOptions.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE), - IllegalArgumentException.class, - SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, - () -> - spark - .read() - .format("iceberg") - .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) - .option(SparkReadOptions.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "false") - .load(unpartitioned.toString()) - .collectAsList()); + Assertions.assertThatThrownBy( + () -> + spark + .read() + .format("iceberg") + .option(SparkReadOptions.VECTORIZATION_ENABLED, String.valueOf(vectorized)) + .option(SparkReadOptions.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "false") + .load(unpartitioned.toString()) + .collectAsList()) + .as( + String.format( + "Read operation performed on a timestamp without timezone field while " + + "'%s' set to false should throw exception", + SparkReadOptions.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); } @Test @@ -217,11 +218,10 @@ public void testUnpartitionedTimestampWithoutZoneWriteError() { .mode(SaveMode.Append) .save(unpartitioned.toString()); - AssertHelpers.assertThrows( - errorMessage, - IllegalArgumentException.class, - SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, - writeOperation); + Assertions.assertThatThrownBy(writeOperation::run) + .as(errorMessage) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index 9bf00f1b1365..fd88eb2c907e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -27,7 +27,6 @@ import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.PartitionSpec; @@ -49,6 +48,7 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; +import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -231,11 +231,13 @@ public void testBadCustomMetricCollectionForParquet() throws IOException { properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); properties.put("write.metadata.metrics.column.ids", "full"); - AssertHelpers.assertThrows( - "Creating a table with invalid metrics should fail", - ValidationException.class, - null, - () -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)); + Assertions.assertThatThrownBy( + () -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) + .as("Creating a table with invalid metrics should fail") + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith( + "Invalid metrics config, could not find column ids from table prop write.metadata.metrics.column.ids in " + + "schema table"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java index e347cde7ba32..901a7c15e78f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java @@ -19,7 +19,6 @@ package org.apache.iceberg.spark.sql; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; @@ -28,6 +27,7 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.spark.SparkException; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -55,11 +55,10 @@ public void removeTable() { @Test public void testAddColumnNotNull() { - AssertHelpers.assertThrows( - "Should reject adding NOT NULL column", - SparkException.class, - "Incompatible change: cannot add required column", - () -> sql("ALTER TABLE %s ADD COLUMN c3 INT NOT NULL", tableName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN c3 INT NOT NULL", tableName)) + .as("Should reject adding NOT NULL column") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Incompatible change: cannot add required column"); } @Test @@ -156,11 +155,10 @@ public void testAddColumnWithMap() { validationCatalog.loadTable(tableIdent).schema().asStruct()); // should not allow changing map key column - AssertHelpers.assertThrows( - "Should reject changing key of the map column", - SparkException.class, - "Unsupported table change: Cannot add fields to map keys:", - () -> sql("ALTER TABLE %s ADD COLUMN data2.key.y int", tableName)); + Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN data2.key.y int", tableName)) + .as("Should reject changing key of the map column") + .isInstanceOf(SparkException.class) + .hasMessageContaining("Unsupported table change: Cannot add fields to map keys:"); } @Test @@ -253,11 +251,11 @@ public void testAlterColumnSetNotNull() { expectedSchema, validationCatalog.loadTable(tableIdent).schema().asStruct()); - AssertHelpers.assertThrows( - "Should reject adding NOT NULL constraint to an optional column", - AnalysisException.class, - "Cannot change nullable column to non-nullable: data", - () -> sql("ALTER TABLE %s ALTER COLUMN data SET NOT NULL", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s ALTER COLUMN data SET NOT NULL", tableName)) + .as("Should reject adding NOT NULL constraint to an optional column") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Cannot change nullable column to non-nullable: data"); } @Test @@ -323,9 +321,11 @@ public void testSetTableProperties() { "Should not have the removed table property", validationCatalog.loadTable(tableIdent).properties().get("prop")); - AssertHelpers.assertThrows( - "Cannot specify the 'sort-order' because it's a reserved table property", - UnsupportedOperationException.class, - () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('sort-order'='value')", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('sort-order'='value')", tableName)) + .as("Cannot specify the 'sort-order' because it's a reserved table property") + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith( + "Cannot specify the 'sort-order' because it's a reserved table property"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index 3a62361590d1..f4bef9cec155 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -21,7 +21,6 @@ import java.io.File; import java.util.Map; import java.util.UUID; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.BaseTable; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -168,14 +167,14 @@ public void testCreateTableUsingParquet() { "parquet", table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); - AssertHelpers.assertThrows( - "Should reject unsupported format names", - IllegalArgumentException.class, - "Unsupported format in USING: crocodile", - () -> - sql( - "CREATE TABLE %s.default.fail (id BIGINT NOT NULL, data STRING) USING crocodile", - catalogName)); + Assertions.assertThatThrownBy( + () -> + sql( + "CREATE TABLE %s.default.fail (id BIGINT NOT NULL, data STRING) USING crocodile", + catalogName)) + .as("Should reject unsupported format names") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Unsupported format in USING: crocodile"); } @Test @@ -382,10 +381,11 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { TableOperations ops = ((BaseTable) table).operations(); Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); - AssertHelpers.assertThrowsCause( - "should fail to downgrade to v1", - IllegalArgumentException.class, - "Cannot downgrade v2 table to v1", - () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)); + Assertions.assertThatThrownBy( + () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)) + .as("should fail to downgrade to v1") + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot downgrade v2 table to v1"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index cae1901aa713..55031188453a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -30,6 +29,7 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -86,11 +86,11 @@ public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - AssertHelpers.assertThrows( - "Should not be able to delete from a table at a specific snapshot", - IllegalArgumentException.class, - "Cannot delete from table at a specific snapshot", - () -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)); + Assertions.assertThatThrownBy( + () -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) + .as("Should not be able to delete from a table at a specific snapshot") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot delete from table at a specific snapshot"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java index 2189bd0dae75..c5c39e5d782f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java @@ -24,12 +24,12 @@ import java.util.stream.Collectors; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.MetadataTableType; 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.Streams; import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -116,11 +116,11 @@ public void testPurgeTableGCDisabled() throws IOException { "There totally should have 2 files for manifests and files", 2, manifestAndFiles.size()); Assert.assertTrue("All files should be existed", checkFilesExist(manifestAndFiles, true)); - AssertHelpers.assertThrows( - "Purge table is not allowed when GC is disabled", - ValidationException.class, - "Cannot purge table: GC is disabled (deleting files may corrupt other tables", - () -> sql("DROP TABLE %s PURGE", tableName)); + Assertions.assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) + .as("Purge table is not allowed when GC is disabled") + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Cannot purge table: GC is disabled (deleting files may corrupt other tables"); Assert.assertTrue("Table should not been dropped", validationCatalog.tableExists(tableIdent)); Assert.assertTrue("All files should not be deleted", checkFilesExist(manifestAndFiles, true)); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java index 6c29ea4442ef..77b564cd001b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -23,13 +23,13 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; 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; import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -103,11 +103,10 @@ public void testDropNonEmptyNamespace() { Assert.assertTrue( "Table should exist", validationCatalog.tableExists(TableIdentifier.of(NS, "table"))); - AssertHelpers.assertThrows( - "Should fail if trying to delete a non-empty namespace", - NamespaceNotEmptyException.class, - "Namespace db is not empty.", - () -> sql("DROP NAMESPACE %s", fullNamespace)); + Assertions.assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) + .as("Should fail if trying to delete a non-empty namespace") + .isInstanceOf(NamespaceNotEmptyException.class) + .hasMessageContaining("Namespace db is not empty."); sql("DROP TABLE %s.table", fullNamespace); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 2265dec4763c..ea97949d1327 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -23,7 +23,6 @@ import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Table; import org.apache.iceberg.events.Listeners; import org.apache.iceberg.events.ScanEvent; @@ -396,48 +395,48 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using snapshot in table identifier and VERSION AS OF - AssertHelpers.assertThrows( - "Cannot do time-travel based on both table identifier and AS OF", - IllegalArgumentException.class, - "Cannot do time-travel based on both table identifier and AS OF", - () -> { - sql( - "SELECT * FROM %s.%s VERSION AS OF %s", - tableName, snapshotPrefix + snapshotId, snapshotId); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s VERSION AS OF %s", + tableName, snapshotPrefix + snapshotId, snapshotId); + }) + .as("Cannot do time-travel based on both table identifier and AS OF") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); // using snapshot in table identifier and TIMESTAMP AS OF - AssertHelpers.assertThrows( - "Cannot do time-travel based on both table identifier and AS OF", - IllegalArgumentException.class, - "Cannot do time-travel based on both table identifier and AS OF", - () -> { - sql( - "SELECT * FROM %s.%s VERSION AS OF %s", - tableName, timestampPrefix + timestamp, snapshotId); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s VERSION AS OF %s", + tableName, timestampPrefix + timestamp, snapshotId); + }) + .as("Cannot do time-travel based on both table identifier and AS OF") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and VERSION AS OF - AssertHelpers.assertThrows( - "Cannot do time-travel based on both table identifier and AS OF", - IllegalArgumentException.class, - "Cannot do time-travel based on both table identifier and AS OF", - () -> { - sql( - "SELECT * FROM %s.%s TIMESTAMP AS OF %s", - tableName, snapshotPrefix + snapshotId, timestamp); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s TIMESTAMP AS OF %s", + tableName, snapshotPrefix + snapshotId, timestamp); + }) + .as("Cannot do time-travel based on both table identifier and AS OF") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and TIMESTAMP AS OF - AssertHelpers.assertThrows( - "Cannot do time-travel based on both table identifier and AS OF", - IllegalArgumentException.class, - "Cannot do time-travel based on both table identifier and AS OF", - () -> { - sql( - "SELECT * FROM %s.%s TIMESTAMP AS OF %s", - tableName, timestampPrefix + timestamp, timestamp); - }); + Assertions.assertThatThrownBy( + () -> { + sql( + "SELECT * FROM %s.%s TIMESTAMP AS OF %s", + tableName, timestampPrefix + timestamp, timestamp); + }) + .as("Cannot do time-travel based on both table identifier and AS OF") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); } @Test @@ -472,21 +471,22 @@ public void testSpecifySnapshotAndTimestamp() { // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - AssertHelpers.assertThrows( - "Should not be able to specify both snapshot id and timestamp", - IllegalArgumentException.class, - String.format( - "Can specify only one of snapshot-id (%s), as-of-timestamp (%s)", - snapshotId, timestamp), - () -> { - spark - .read() - .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) - .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) - .load(tableName) - .collectAsList(); - }); + Assertions.assertThatThrownBy( + () -> { + spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshotId) + .option(SparkReadOptions.AS_OF_TIMESTAMP, timestamp) + .load(tableName) + .collectAsList(); + }) + .as("Should not be able to specify both snapshot id and timestamp") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + String.format( + "Can specify only one of snapshot-id (%s), as-of-timestamp (%s)", + snapshotId, timestamp)); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java index c9c8c02b417c..2cf2865fe8bb 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java @@ -21,7 +21,6 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; @@ -191,101 +190,107 @@ public void testNumBucketsAcceptsShortAndByte() { @Test public void testWrongNumberOfArguments() { - AssertHelpers.assertThrows( - "Function resolution should not work with zero arguments", - AnalysisException.class, - "Function 'bucket' cannot process input: (): Wrong number of inputs (expected numBuckets and value)", - () -> scalarSql("SELECT system.bucket()")); - - AssertHelpers.assertThrows( - "Function resolution should not work with only one argument", - AnalysisException.class, - "Function 'bucket' cannot process input: (int): Wrong number of inputs (expected numBuckets and value)", - () -> scalarSql("SELECT system.bucket(1)")); - - AssertHelpers.assertThrows( - "Function resolution should not work with more than two arguments", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, bigint, int): Wrong number of inputs (expected numBuckets and value)", - () -> scalarSql("SELECT system.bucket(1, 1L, 1)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) + .as("Function resolution should not work with zero arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (): Wrong number of inputs (expected numBuckets and value)"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) + .as("Function resolution should not work with only one argument") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (int): Wrong number of inputs (expected numBuckets and value)"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) + .as("Function resolution should not work with more than two arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (int, bigint, int): Wrong number of inputs (expected numBuckets and value)"); } @Test public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { - AssertHelpers.assertThrows( - "Decimal type should not be coercible to the number of buckets", - AnalysisException.class, - "Function 'bucket' cannot process input: (decimal(9,2), int): Expected number of buckets to be tinyint, shortint or int", - () -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")); - - AssertHelpers.assertThrows( - "Long type should not be coercible to the number of buckets", - AnalysisException.class, - "Function 'bucket' cannot process input: (bigint, int): Expected number of buckets to be tinyint, shortint or int", - () -> scalarSql("SELECT system.bucket(12L, 10)")); - - AssertHelpers.assertThrows( - "String type should not be coercible to the number of buckets", - AnalysisException.class, - "Function 'bucket' cannot process input: (string, int): Expected number of buckets to be tinyint, shortint or int", - () -> scalarSql("SELECT system.bucket('5', 10)")); - - AssertHelpers.assertThrows( - "Interval year to month type should not be coercible to the number of buckets", - AnalysisException.class, - "Function 'bucket' cannot process input: (interval year to month, int): Expected number of buckets to be tinyint, shortint or int", - () -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")); - - AssertHelpers.assertThrows( - "Interval day-time type should not be coercible to the number of buckets", - AnalysisException.class, - "Function 'bucket' cannot process input: (interval day to second, int): Expected number of buckets to be tinyint, shortint or int", - () -> scalarSql("SELECT system.bucket(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) + .as("Decimal type should not be coercible to the number of buckets") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (decimal(9,2), int): Expected number of buckets to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) + .as("Long type should not be coercible to the number of buckets") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (bigint, int): Expected number of buckets to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) + .as("String type should not be coercible to the number of buckets") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (string, int): Expected number of buckets to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) + .as("Interval year to month type should not be coercible to the number of buckets") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (interval year to month, int): Expected number of buckets to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy( + () -> + scalarSql("SELECT system.bucket(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) + .as("Interval day-time type should not be coercible to the number of buckets") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (interval day to second, int): Expected number of buckets to be tinyint, shortint or int"); } @Test public void testInvalidTypesForBucketColumn() { - AssertHelpers.assertThrows( - "Double type should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, float): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary", - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")); - - AssertHelpers.assertThrows( - "Double type should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, double): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary", - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")); - - AssertHelpers.assertThrows( - "Boolean type should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, boolean)", - () -> scalarSql("SELECT system.bucket(10, true)")); - - AssertHelpers.assertThrows( - "Map types should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, map)", - () -> scalarSql("SELECT system.bucket(10, map(1, 1))")); - - AssertHelpers.assertThrows( - "Array types should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, array)", - () -> scalarSql("SELECT system.bucket(10, array(1L))")); - - AssertHelpers.assertThrows( - "Interval year-to-month type should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, interval year to month)", - () -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")); - - AssertHelpers.assertThrows( - "Interval day-time type should not be bucketable", - AnalysisException.class, - "Function 'bucket' cannot process input: (int, interval day to second)", - () -> scalarSql("SELECT system.bucket(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) + .as("Double type should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (int, float): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); + + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) + .as("Double type should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (int, double): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) + .as("Boolean type should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'bucket' cannot process input: (int, boolean)"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) + .as("Map types should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'bucket' cannot process input: (int, map)"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) + .as("Array types should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'bucket' cannot process input: (int, array)"); + + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) + .as("Interval year-to-month type should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (int, interval year to month)"); + + Assertions.assertThatThrownBy( + () -> + scalarSql("SELECT system.bucket(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) + .as("Interval day-time type should not be bucketable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'bucket' cannot process input: (int, interval day to second)"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java index ccba28735e33..e139d3d000ef 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java @@ -19,9 +19,9 @@ package org.apache.iceberg.spark.sql; import java.sql.Date; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -69,31 +69,31 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - AssertHelpers.assertThrows( - "Function resolution should not work with zero arguments", - AnalysisException.class, - "Function 'days' cannot process input: (): Wrong number of inputs", - () -> scalarSql("SELECT system.days()")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days()")) + .as("Function resolution should not work with zero arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'days' cannot process input: (): Wrong number of inputs"); - AssertHelpers.assertThrows( - "Function resolution should not work with more than one argument", - AnalysisException.class, - "Function 'days' cannot process input: (date, date): Wrong number of inputs", - () -> scalarSql("SELECT system.days(date('1969-12-31'), date('1969-12-31'))")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.days(date('1969-12-31'), date('1969-12-31'))")) + .as("Function resolution should not work with more than one argument") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'days' cannot process input: (date, date): Wrong number of inputs"); } @Test public void testInvalidInputTypes() { - AssertHelpers.assertThrows( - "Int type should not be coercible to date/timestamp", - AnalysisException.class, - "Function 'days' cannot process input: (int): Expected value to be date or timestamp", - () -> scalarSql("SELECT system.days(1)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) + .as("Int type should not be coercible to date/timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'days' cannot process input: (int): Expected value to be date or timestamp"); - AssertHelpers.assertThrows( - "Long type should not be coercible to date/timestamp", - AnalysisException.class, - "Function 'days' cannot process input: (bigint): Expected value to be date or timestamp", - () -> scalarSql("SELECT system.days(1L)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) + .as("Long type should not be coercible to date/timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'days' cannot process input: (bigint): Expected value to be date or timestamp"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java index fc0d781318c9..312299ffee09 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java @@ -18,9 +18,9 @@ */ package org.apache.iceberg.spark.sql; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -51,31 +51,31 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - AssertHelpers.assertThrows( - "Function resolution should not work with zero arguments", - AnalysisException.class, - "Function 'hours' cannot process input: (): Wrong number of inputs", - () -> scalarSql("SELECT system.hours()")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) + .as("Function resolution should not work with zero arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'hours' cannot process input: (): Wrong number of inputs"); - AssertHelpers.assertThrows( - "Function resolution should not work with more than one argument", - AnalysisException.class, - "Function 'hours' cannot process input: (date, date): Wrong number of inputs", - () -> scalarSql("SELECT system.hours(date('1969-12-31'), date('1969-12-31'))")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.hours(date('1969-12-31'), date('1969-12-31'))")) + .as("Function resolution should not work with more than one argument") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'hours' cannot process input: (date, date): Wrong number of inputs"); } @Test public void testInvalidInputTypes() { - AssertHelpers.assertThrows( - "Int type should not be coercible to timestamp", - AnalysisException.class, - "Function 'hours' cannot process input: (int): Expected value to be timestamp", - () -> scalarSql("SELECT system.hours(1)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) + .as("Int type should not be coercible to timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'hours' cannot process input: (int): Expected value to be timestamp"); - AssertHelpers.assertThrows( - "Long type should not be coercible to timestamp", - AnalysisException.class, - "Function 'hours' cannot process input: (bigint): Expected value to be timestamp", - () -> scalarSql("SELECT system.hours(1L)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) + .as("Long type should not be coercible to timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'hours' cannot process input: (bigint): Expected value to be timestamp"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java index b88bf00256b0..c95823b889c3 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.sql; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.MonthsFunction; import org.apache.spark.sql.AnalysisException; @@ -68,32 +67,32 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - AssertHelpers.assertThrows( - "Function resolution should not work with zero arguments", - AnalysisException.class, - "Function 'months' cannot process input: (): Wrong number of inputs", - () -> scalarSql("SELECT system.months()")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months()")) + .as("Function resolution should not work with zero arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'months' cannot process input: (): Wrong number of inputs"); - AssertHelpers.assertThrows( - "Function resolution should not work with more than one argument", - AnalysisException.class, - "Function 'months' cannot process input: (date, date): Wrong number of inputs", - () -> scalarSql("SELECT system.months(date('1969-12-31'), date('1969-12-31'))")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.months(date('1969-12-31'), date('1969-12-31'))")) + .as("Function resolution should not work with more than one argument") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'months' cannot process input: (date, date): Wrong number of inputs"); } @Test public void testInvalidInputTypes() { - AssertHelpers.assertThrows( - "Int type should not be coercible to date/timestamp", - AnalysisException.class, - "Function 'months' cannot process input: (int): Expected value to be date or timestamp", - () -> scalarSql("SELECT system.months(1)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) + .as("Int type should not be coercible to date/timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'months' cannot process input: (int): Expected value to be date or timestamp"); - AssertHelpers.assertThrows( - "Long type should not be coercible to date/timestamp", - AnalysisException.class, - "Function 'months' cannot process input: (bigint): Expected value to be date or timestamp", - () -> scalarSql("SELECT system.months(1L)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) + .as("Long type should not be coercible to date/timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'months' cannot process input: (bigint): Expected value to be date or timestamp"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java index f21544fcdf7a..aaf4f569b655 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java @@ -20,7 +20,6 @@ import java.math.BigDecimal; import java.nio.charset.StandardCharsets; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; @@ -298,95 +297,106 @@ public void testWidthAcceptsShortAndByte() { @Test public void testWrongNumberOfArguments() { - AssertHelpers.assertThrows( - "Function resolution should not work with zero arguments", - AnalysisException.class, - "Function 'truncate' cannot process input: (): Wrong number of inputs (expected width and value)", - () -> scalarSql("SELECT system.truncate()")); - - AssertHelpers.assertThrows( - "Function resolution should not work with only one argument", - AnalysisException.class, - "Function 'truncate' cannot process input: (int): Wrong number of inputs (expected width and value)", - () -> scalarSql("SELECT system.truncate(1)")); - - AssertHelpers.assertThrows( - "Function resolution should not work with more than two arguments", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, bigint, int): Wrong number of inputs (expected width and value)", - () -> scalarSql("SELECT system.truncate(1, 1L, 1)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) + .as("Function resolution should not work with zero arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (): Wrong number of inputs (expected width and value)"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) + .as("Function resolution should not work with only one argument") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int): Wrong number of inputs (expected width and value)"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) + .as("Function resolution should not work with more than two arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, bigint, int): Wrong number of inputs (expected width and value)"); } @Test public void testInvalidTypesCannotBeUsedForWidth() { - AssertHelpers.assertThrows( - "Decimal type should not be coercible to the width field", - AnalysisException.class, - "Function 'truncate' cannot process input: (decimal(9,2), int): Expected truncation width to be tinyint, shortint or int", - () -> scalarSql("SELECT system.truncate(CAST('12.34' as DECIMAL(9, 2)), 10)")); - - AssertHelpers.assertThrows( - "String type should not be coercible to the width field", - AnalysisException.class, - "Function 'truncate' cannot process input: (string, int): Expected truncation width to be tinyint, shortint or int", - () -> scalarSql("SELECT system.truncate('5', 10)")); - - AssertHelpers.assertThrows( - "Interval year to month type should not be coercible to the width field", - AnalysisException.class, - "Function 'truncate' cannot process input: (interval year to month, int): Expected truncation width to be tinyint, shortint or int", - () -> scalarSql("SELECT system.truncate(INTERVAL '100-00' YEAR TO MONTH, 10)")); - - AssertHelpers.assertThrows( - "Interval day-time type should not be coercible to the width field", - AnalysisException.class, - "Function 'truncate' cannot process input: (interval day to second, int): Expected truncation width to be tinyint, shortint or int", - () -> scalarSql("SELECT system.truncate(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(CAST('12.34' as DECIMAL(9, 2)), 10)")) + .as("Decimal type should not be coercible to the width field") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (decimal(9,2), int): Expected truncation width to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) + .as("String type should not be coercible to the width field") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (string, int): Expected truncation width to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(INTERVAL '100-00' YEAR TO MONTH, 10)")) + .as("Interval year to month type should not be coercible to the width field") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (interval year to month, int): Expected truncation width to be tinyint, shortint or int"); + + Assertions.assertThatThrownBy( + () -> + scalarSql( + "SELECT system.truncate(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) + .as("Interval day-time type should not be coercible to the width field") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (interval day to second, int): Expected truncation width to be tinyint, shortint or int"); } @Test public void testInvalidTypesForTruncationColumn() { - AssertHelpers.assertThrows( - "FLoat type should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, float): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")); - - AssertHelpers.assertThrows( - "Double type should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, double): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")); - - AssertHelpers.assertThrows( - "Boolean type should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, boolean): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, true)")); - - AssertHelpers.assertThrows( - "Map types should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, map): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, map(1, 1))")); - - AssertHelpers.assertThrows( - "Array types should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, array): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, array(1L))")); - - AssertHelpers.assertThrows( - "Interval year-to-month type should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, interval year to month): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, INTERVAL '100-00' YEAR TO MONTH)")); - - AssertHelpers.assertThrows( - "Interval day-time type should not be truncatable", - AnalysisException.class, - "Function 'truncate' cannot process input: (int, interval day to second): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary", - () -> scalarSql("SELECT system.truncate(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) + .as("FLoat type should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, float): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) + .as("Double type should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, double): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) + .as("Boolean type should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, boolean): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) + .as("Map types should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, map): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) + .as("Array types should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, array): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.truncate(10, INTERVAL '100-00' YEAR TO MONTH)")) + .as("Interval year-to-month type should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, interval year to month): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); + + Assertions.assertThatThrownBy( + () -> + scalarSql( + "SELECT system.truncate(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) + .as("Interval day-time type should not be truncatable") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'truncate' cannot process input: (int, interval day to second): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java index d4676716a612..600b6ac0c1c5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.sql; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.AnalysisException; @@ -70,32 +69,32 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - AssertHelpers.assertThrows( - "Function resolution should not work with zero arguments", - AnalysisException.class, - "Function 'years' cannot process input: (): Wrong number of inputs", - () -> scalarSql("SELECT system.years()")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years()")) + .as("Function resolution should not work with zero arguments") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining("Function 'years' cannot process input: (): Wrong number of inputs"); - AssertHelpers.assertThrows( - "Function resolution should not work with more than one argument", - AnalysisException.class, - "Function 'years' cannot process input: (date, date): Wrong number of inputs", - () -> scalarSql("SELECT system.years(date('1969-12-31'), date('1969-12-31'))")); + Assertions.assertThatThrownBy( + () -> scalarSql("SELECT system.years(date('1969-12-31'), date('1969-12-31'))")) + .as("Function resolution should not work with more than one argument") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'years' cannot process input: (date, date): Wrong number of inputs"); } @Test public void testInvalidInputTypes() { - AssertHelpers.assertThrows( - "Int type should not be coercible to date/timestamp", - AnalysisException.class, - "Function 'years' cannot process input: (int): Expected value to be date or timestamp", - () -> scalarSql("SELECT system.years(1)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) + .as("Int type should not be coercible to date/timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'years' cannot process input: (int): Expected value to be date or timestamp"); - AssertHelpers.assertThrows( - "Long type should not be coercible to date/timestamp", - AnalysisException.class, - "Function 'years' cannot process input: (bigint): Expected value to be date or timestamp", - () -> scalarSql("SELECT system.years(1L)")); + Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) + .as("Long type should not be coercible to date/timestamp") + .isInstanceOf(AnalysisException.class) + .hasMessageContaining( + "Function 'years' cannot process input: (bigint): Expected value to be date or timestamp"); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index 51b8d255a99b..5a8c5736a6b4 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -23,7 +23,6 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; @@ -37,6 +36,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -95,14 +95,15 @@ public void removeTables() { @Test public void testWriteTimestampWithoutZoneError() { - AssertHelpers.assertThrows( - String.format( - "Write operation performed on a timestamp without timezone field while " - + "'%s' set to false should throw exception", - SparkSQLProperties.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE), - IllegalArgumentException.class, - SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR, - () -> sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values))); + Assertions.assertThatThrownBy( + () -> sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values))) + .as( + String.format( + "Write operation performed on a timestamp without timezone field while " + + "'%s' set to false should throw exception", + SparkSQLProperties.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining(SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java index 71089ebfd79e..813e442c6604 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -20,7 +20,6 @@ import java.util.List; import java.util.Map; -import org.apache.iceberg.AssertHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.source.SimpleRecord; @@ -28,6 +27,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; +import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -97,11 +97,12 @@ public void testInsertAppendAtSnapshot() { Assume.assumeTrue(tableName.equals(commitTarget())); long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - AssertHelpers.assertThrows( - "Should not be able to insert into a table at a specific snapshot", - IllegalArgumentException.class, - "Cannot write to table at a specific snapshot", - () -> sql("INSERT INTO %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)); + Assertions.assertThatThrownBy( + () -> + sql("INSERT INTO %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)) + .as("Should not be able to insert into a table at a specific snapshot") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot write to table at a specific snapshot"); } @Test @@ -109,14 +110,14 @@ public void testInsertOverwriteAtSnapshot() { Assume.assumeTrue(tableName.equals(commitTarget())); long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - AssertHelpers.assertThrows( - "Should not be able to insert into a table at a specific snapshot", - IllegalArgumentException.class, - "Cannot write to table at a specific snapshot", - () -> - sql( - "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", - tableName, prefix + snapshotId)); + Assertions.assertThatThrownBy( + () -> + sql( + "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", + tableName, prefix + snapshotId)) + .as("Should not be able to insert into a table at a specific snapshot") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot write to table at a specific snapshot"); } @Test From 8ae745f312545c5bcbcec330cf65323ee32b309a Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 17 Jun 2024 07:34:23 -0700 Subject: [PATCH 0380/1019] Spark: Use bulk deletes in rewrite manifests action (#10343) --- .../actions/RewriteManifestsSparkAction.java | 16 +++++++++------- .../actions/RewriteManifestsSparkAction.java | 16 +++++++++------- .../actions/RewriteManifestsSparkAction.java | 16 +++++++++------- 3 files changed, 27 insertions(+), 21 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index bc2ef2306790..288b2d417f1a 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -45,6 +45,7 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.OutputFile; +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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -54,7 +55,6 @@ import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; import org.apache.spark.broadcast.Broadcast; @@ -352,12 +352,14 @@ private void replaceManifests( } private void deleteFiles(Iterable locations) { - Tasks.foreach(locations) - .executeWith(ThreadPools.getWorkerPool()) - .noRetry() - .suppressFailureWhenFinished() - .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) - .run(location -> table.io().deleteFile(location)); + Iterable files = + Iterables.transform(locations, location -> new FileInfo(location, MANIFEST)); + if (table.io() instanceof SupportsBulkOperations) { + deleteFiles((SupportsBulkOperations) table.io(), files.iterator()); + } else { + deleteFiles( + ThreadPools.getWorkerPool(), file -> table.io().deleteFile(file), files.iterator()); + } } private static ManifestFile writeManifest( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 5b1d6165695b..8ec3b44f9284 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -49,6 +49,7 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.OutputFile; +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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -60,7 +61,6 @@ import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; import org.apache.spark.broadcast.Broadcast; @@ -362,12 +362,14 @@ private void replaceManifests( } private void deleteFiles(Iterable locations) { - Tasks.foreach(locations) - .executeWith(ThreadPools.getWorkerPool()) - .noRetry() - .suppressFailureWhenFinished() - .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) - .run(location -> table.io().deleteFile(location)); + Iterable files = + Iterables.transform(locations, location -> new FileInfo(location, MANIFEST)); + if (table.io() instanceof SupportsBulkOperations) { + deleteFiles((SupportsBulkOperations) table.io(), files.iterator()); + } else { + deleteFiles( + ThreadPools.getWorkerPool(), file -> table.io().deleteFile(file), files.iterator()); + } } private ManifestWriterFactory manifestWriters() { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 648a3197730b..8c35c9c7fe96 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -52,6 +52,7 @@ import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.OutputFile; +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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; @@ -63,7 +64,6 @@ import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; import org.apache.spark.broadcast.Broadcast; @@ -447,12 +447,14 @@ private void replaceManifests( } private void deleteFiles(Iterable locations) { - Tasks.foreach(locations) - .executeWith(ThreadPools.getWorkerPool()) - .noRetry() - .suppressFailureWhenFinished() - .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) - .run(location -> table.io().deleteFile(location)); + Iterable files = + Iterables.transform(locations, location -> new FileInfo(location, MANIFEST)); + if (table.io() instanceof SupportsBulkOperations) { + deleteFiles((SupportsBulkOperations) table.io(), files.iterator()); + } else { + deleteFiles( + ThreadPools.getWorkerPool(), file -> table.io().deleteFile(file), files.iterator()); + } } private ManifestWriterFactory manifestWriters() { From e27d07531d16b24f6f9bcf7c010ec59baf33c6b2 Mon Sep 17 00:00:00 2001 From: Cancai Cai <77189278+caicancai@users.noreply.github.com> Date: Tue, 18 Jun 2024 01:36:19 +0800 Subject: [PATCH 0381/1019] Build: Update NOTICE to include copyright for 2024 (#10471) --- NOTICE | 2 +- aws-bundle/NOTICE | 2 +- azure-bundle/NOTICE | 2 +- bundled-guava/NOTICE | 2 +- flink/v1.17/flink-runtime/NOTICE | 2 +- flink/v1.18/flink-runtime/NOTICE | 2 +- flink/v1.19/flink-runtime/NOTICE | 2 +- gcp-bundle/NOTICE | 2 +- hive-runtime/NOTICE | 2 +- spark/v3.3/spark-runtime/NOTICE | 2 +- spark/v3.4/spark-runtime/NOTICE | 2 +- spark/v3.5/spark-runtime/NOTICE | 2 +- 12 files changed, 12 insertions(+), 12 deletions(-) diff --git a/NOTICE b/NOTICE index d395c9d8efdb..f202d2ab401b 100644 --- a/NOTICE +++ b/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/aws-bundle/NOTICE b/aws-bundle/NOTICE index 3626c0e15761..ed353940ba38 100644 --- a/aws-bundle/NOTICE +++ b/aws-bundle/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2023 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/azure-bundle/NOTICE b/azure-bundle/NOTICE index 5bdb96a3ee95..d03b3a1a6cc5 100644 --- a/azure-bundle/NOTICE +++ b/azure-bundle/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2023 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/bundled-guava/NOTICE b/bundled-guava/NOTICE index 977fb2258d0d..f37028183bd2 100644 --- a/bundled-guava/NOTICE +++ b/bundled-guava/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.17/flink-runtime/NOTICE b/flink/v1.17/flink-runtime/NOTICE index 81aa1660456a..dc36f84c4ac5 100644 --- a/flink/v1.17/flink-runtime/NOTICE +++ b/flink/v1.17/flink-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.18/flink-runtime/NOTICE b/flink/v1.18/flink-runtime/NOTICE index 81aa1660456a..dc36f84c4ac5 100644 --- a/flink/v1.18/flink-runtime/NOTICE +++ b/flink/v1.18/flink-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.19/flink-runtime/NOTICE b/flink/v1.19/flink-runtime/NOTICE index 81aa1660456a..dc36f84c4ac5 100644 --- a/flink/v1.19/flink-runtime/NOTICE +++ b/flink/v1.19/flink-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/gcp-bundle/NOTICE b/gcp-bundle/NOTICE index abe35d188aee..d71592a20314 100644 --- a/gcp-bundle/NOTICE +++ b/gcp-bundle/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2023 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/hive-runtime/NOTICE b/hive-runtime/NOTICE index c85424ab80b3..774e87fedf58 100644 --- a/hive-runtime/NOTICE +++ b/hive-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/spark/v3.3/spark-runtime/NOTICE b/spark/v3.3/spark-runtime/NOTICE index 4a1f4dfde1cc..2935bfff8b80 100644 --- a/spark/v3.3/spark-runtime/NOTICE +++ b/spark/v3.3/spark-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/spark/v3.4/spark-runtime/NOTICE b/spark/v3.4/spark-runtime/NOTICE index 4a1f4dfde1cc..2935bfff8b80 100644 --- a/spark/v3.4/spark-runtime/NOTICE +++ b/spark/v3.4/spark-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). diff --git a/spark/v3.5/spark-runtime/NOTICE b/spark/v3.5/spark-runtime/NOTICE index 4a1f4dfde1cc..2935bfff8b80 100644 --- a/spark/v3.5/spark-runtime/NOTICE +++ b/spark/v3.5/spark-runtime/NOTICE @@ -1,6 +1,6 @@ Apache Iceberg -Copyright 2017-2022 The Apache Software Foundation +Copyright 2017-2024 The Apache Software Foundation This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From 047c0ec5f32e6e4f8fff3bc99f2be9af8ac73dc9 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 17 Jun 2024 21:18:31 +0200 Subject: [PATCH 0382/1019] Remove redundant `-XX:+IgnoreUnrecognizedVMOptions` (#10475) The JVM flags are set depending on Java version. We should not be setting any flags that are not expected to be supported. --- build.gradle | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index 213109852ec4..9b3c35270519 100644 --- a/build.gradle +++ b/build.gradle @@ -64,8 +64,7 @@ if (JavaVersion.current() == JavaVersion.VERSION_1_8) { project.ext.extraJvmArgs = [] } else if (JavaVersion.current() == JavaVersion.VERSION_17) { project.ext.jdkVersion = '17' - project.ext.extraJvmArgs = ["-XX:+IgnoreUnrecognizedVMOptions", - "--add-opens", "java.base/java.io=ALL-UNNAMED", + project.ext.extraJvmArgs = ["--add-opens", "java.base/java.io=ALL-UNNAMED", "--add-opens", "java.base/java.lang.invoke=ALL-UNNAMED", "--add-opens", "java.base/java.lang.reflect=ALL-UNNAMED", "--add-opens", "java.base/java.lang=ALL-UNNAMED", From b21917bacf690b8c06ea4f45d6618c93cf88817f Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 18 Jun 2024 09:16:20 +0200 Subject: [PATCH 0383/1019] spec: Fix formatting of Default values (#10525) --- format/spec.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index 6e23b2c5834b..e590a8b95116 100644 --- a/format/spec.md +++ b/format/spec.md @@ -201,8 +201,9 @@ For details on how to serialize a schema to JSON, see Appendix C. #### 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: -- `initial-default` is used to populate the field's value for all records that were written before the field was added to the schema -- `write-default` is used to populate the field's value for any records written after the field was added to the schema, if the writer does not supply the field's value + +* `initial-default` is used to populate the field's value for all records that were written before the field was added to the schema +* `write-default` is used to populate the field's value for any records written after the field was added to the schema, if the writer does not supply the field's value The `initial-default` is set only when a field is added to an existing schema. The `write-default` is initially set to the same value as `initial-default` and can be changed through schema evolution. If either default is not set for an optional field, then the default value is null for compatibility with older spec versions. From 8bb800a5b5c35425d9fdc7c9915bcba83f6de1a1 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 18 Jun 2024 15:36:11 +0800 Subject: [PATCH 0384/1019] docs: Introduce variable for setting the Flink version (#10463) --- docs/docs/flink-connector.md | 4 ++-- docs/docs/flink-ddl.md | 4 ++-- docs/docs/flink-queries.md | 4 ++-- docs/docs/flink.md | 22 +++++++++++----------- site/mkdocs.yml | 2 ++ 5 files changed, 19 insertions(+), 17 deletions(-) diff --git a/docs/docs/flink-connector.md b/docs/docs/flink-connector.md index 025e9aee92ea..d3d29ddb1360 100644 --- a/docs/docs/flink-connector.md +++ b/docs/docs/flink-connector.md @@ -19,9 +19,9 @@ title: "Flink Connector" --> # Flink Connector -Apache Flink supports creating Iceberg table directly without creating the explicit Flink catalog in Flink SQL. That means we can just create an iceberg table by specifying `'connector'='iceberg'` table option in Flink SQL which is similar to usage in the Flink official [document](https://nightlies.apache.org/flink/flink-docs-release-1.13/docs/connectors/table/overview/). +Apache Flink supports creating Iceberg table directly without creating the explicit Flink catalog in Flink SQL. That means we can just create an iceberg table by specifying `'connector'='iceberg'` table option in Flink SQL which is similar to usage in the Flink official [document](https://nightlies.apache.org/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/connectors/table/overview/). -In Flink, the SQL `CREATE TABLE test (..) WITH ('connector'='iceberg', ...)` will create a Flink table in current Flink catalog (use [GenericInMemoryCatalog](https://ci.apache.org/projects/flink/flink-docs-release-1.13/docs/dev/table/catalogs/#genericinmemorycatalog) by default), +In Flink, the SQL `CREATE TABLE test (..) WITH ('connector'='iceberg', ...)` will create a Flink table in current Flink catalog (use [GenericInMemoryCatalog](https://ci.apache.org/projects/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/table/catalogs/#genericinmemorycatalog) by default), which is just mapping to the underlying iceberg table instead of maintaining iceberg table directly in current Flink catalog. To create the table in Flink SQL by using SQL syntax `CREATE TABLE test (..) WITH ('connector'='iceberg', ...)`, Flink iceberg connector provides the following table properties: diff --git a/docs/docs/flink-ddl.md b/docs/docs/flink-ddl.md index c2b3051fde8d..6dc96e6db8d9 100644 --- a/docs/docs/flink-ddl.md +++ b/docs/docs/flink-ddl.md @@ -146,7 +146,7 @@ CREATE TABLE `hive_catalog`.`default`.`sample` ( ) WITH ('format-version'='2'); ``` -Table create commands support the commonly used [Flink create clauses](https://nightlies.apache.org/flink/flink-docs-master/docs/dev/table/sql/create/) including: +Table create commands support the commonly used [Flink create clauses](https://nightlies.apache.org/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/table/sql/create/) including: * `PARTITION BY (column1, column2, ...)` to configure partitioning, Flink does not yet support hidden partitioning. * `COMMENT 'table document'` to set a table description. @@ -195,7 +195,7 @@ CREATE TABLE `hive_catalog`.`default`.`sample` ( CREATE TABLE `hive_catalog`.`default`.`sample_like` LIKE `hive_catalog`.`default`.`sample`; ``` -For more details, refer to the [Flink `CREATE TABLE` documentation](https://nightlies.apache.org/flink/flink-docs-release-1.16/docs/dev/table/sql/create/). +For more details, refer to the [Flink `CREATE TABLE` documentation](https://nightlies.apache.org/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/table/sql/create/). ### `ALTER TABLE` diff --git a/docs/docs/flink-queries.md b/docs/docs/flink-queries.md index 431a5554f248..38263d47c199 100644 --- a/docs/docs/flink-queries.md +++ b/docs/docs/flink-queries.md @@ -271,8 +271,8 @@ DataStream stream = env.fromSource(source, WatermarkStrategy.noWatermarks() ### Emitting watermarks Emitting watermarks from the source itself could be beneficial for several purposes, like harnessing the -[Flink Watermark Alignment](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment), -or prevent triggering [windows](https://nightlies.apache.org/flink/flink-docs-stable/docs/dev/datastream/operators/windows/) +[Flink Watermark Alignment](https://nightlies.apache.org/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/datastream/event-time/generating_watermarks/#watermark-alignment), +or prevent triggering [windows](https://nightlies.apache.org/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/datastream/operators/windows/) too early when reading multiple data files concurrently. Enable watermark generation for an `IcebergSource` by setting the `watermarkColumn`. diff --git a/docs/docs/flink.md b/docs/docs/flink.md index b8ab694ad9bc..0dd427a8ff51 100644 --- a/docs/docs/flink.md +++ b/docs/docs/flink.md @@ -41,12 +41,12 @@ Apache Iceberg supports both [Apache Flink](https://flink.apache.org/)'s DataStr ## Preparation when using Flink SQL Client -To create Iceberg table in Flink, it is recommended to use [Flink SQL Client](https://ci.apache.org/projects/flink/flink-docs-stable/dev/table/sqlClient.html) as it's easier for users to understand the concepts. +To create Iceberg table in Flink, it is recommended to use [Flink SQL Client](https://ci.apache.org/projects/flink/flink-docs-release-{{ flinkVersionMajor }}/dev/table/sqlClient.html) as it's easier for users to understand the concepts. -Download Flink from the [Apache download page](https://flink.apache.org/downloads.html). Iceberg uses Scala 2.12 when compiling the Apache `iceberg-flink-runtime` jar, so it's recommended to use Flink 1.16 bundled with Scala 2.12. +Download Flink from the [Apache download page](https://flink.apache.org/downloads.html). Iceberg uses Scala 2.12 when compiling the Apache `iceberg-flink-runtime` jar, so it's recommended to use Flink {{ flinkVersionMajor }} bundled with Scala 2.12. ```bash -FLINK_VERSION=1.16.2 +FLINK_VERSION={{ flinkVersion }} SCALA_VERSION=2.12 APACHE_FLINK_URL=https://archive.apache.org/dist/flink/ wget ${APACHE_FLINK_URL}/flink-${FLINK_VERSION}/flink-${FLINK_VERSION}-bin-scala_${SCALA_VERSION}.tgz @@ -69,8 +69,7 @@ export HADOOP_CLASSPATH=`$HADOOP_HOME/bin/hadoop classpath` ./bin/start-cluster.sh ``` - -Start the Flink SQL client. There is a separate `flink-runtime` module in the Iceberg project to generate a bundled jar, which could be loaded by Flink SQL client directly. To build the `flink-runtime` bundled jar manually, build the `iceberg` project, and it will generate the jar under `/flink-runtime/build/libs`. Or download the `flink-runtime` jar from the [Apache repository](https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/). +Start the Flink SQL client. There is a separate `flink-runtime` module in the Iceberg project to generate a bundled jar, which could be loaded by Flink SQL client directly. To build the `flink-runtime` bundled jar manually, build the `iceberg` project, and it will generate the jar under `/flink-runtime/build/libs`. Or download the `flink-runtime` jar from the [Apache repository](https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-flink-runtime-{{ flinkVersionMajor }}/{{ icebergVersion }}/). ```bash # HADOOP_HOME is your hadoop root directory after unpack the binary package. @@ -84,7 +83,7 @@ put iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar in flink/lib dir ./bin/sql-client.sh embedded shell ``` -By default, Iceberg ships with Hadoop jars for Hadoop catalog. To use Hive catalog, load the Hive jars when opening the Flink SQL client. Fortunately, Flink has provided a [bundled hive jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.9_2.12/1.16.2/flink-sql-connector-hive-2.3.9_2.12-1.16.2.jar) for the SQL client. An example on how to download the dependencies and get started: +By default, Iceberg ships with Hadoop jars for Hadoop catalog. To use Hive catalog, load the Hive jars when opening the Flink SQL client. Fortunately, Flink has provided a [bundled hive jar](https://repo.maven.apache.org/maven2/org/apache/flink/flink-sql-connector-hive-2.3.9_2.12/{{ flinkVersion }}/flink-sql-connector-hive-2.3.9_2.12-{{ flinkVersion }}.jar) for the SQL client. An example on how to download the dependencies and get started: ```bash # HADOOP_HOME is your hadoop root directory after unpack the binary package. @@ -94,11 +93,12 @@ ICEBERG_VERSION={{ icebergVersion }} MAVEN_URL=https://repo1.maven.org/maven2 ICEBERG_MAVEN_URL=${MAVEN_URL}/org/apache/iceberg ICEBERG_PACKAGE=iceberg-flink-runtime +FLINK_VERSION_MAJOR={{ flinkVersionMajor }} wget ${ICEBERG_MAVEN_URL}/${ICEBERG_PACKAGE}-${FLINK_VERSION_MAJOR}/${ICEBERG_VERSION}/${ICEBERG_PACKAGE}-${FLINK_VERSION_MAJOR}-${ICEBERG_VERSION}.jar -P lib/ HIVE_VERSION=2.3.9 SCALA_VERSION=2.12 -FLINK_VERSION=1.16.2 +FLINK_VERSION={{ flinkVersion }} FLINK_CONNECTOR_URL=${MAVEN_URL}/org/apache/flink FLINK_CONNECTOR_PACKAGE=flink-sql-connector-hive wget ${FLINK_CONNECTOR_URL}/${FLINK_CONNECTOR_PACKAGE}-${HIVE_VERSION}_${SCALA_VERSION}/${FLINK_VERSION}/${FLINK_CONNECTOR_PACKAGE}-${HIVE_VERSION}_${SCALA_VERSION}-${FLINK_VERSION}.jar @@ -115,7 +115,7 @@ wget ${FLINK_CONNECTOR_URL}/${FLINK_CONNECTOR_PACKAGE}-${HIVE_VERSION}_${SCALA_V Install the Apache Flink dependency using `pip`: ```python -pip install apache-flink==1.16.2 +pip install apache-flink=={{ flinkVersion }} ``` Provide a `file://` path to the `iceberg-flink-runtime` jar, which can be obtained by building the project and looking at `/flink-runtime/build/libs`, or downloading it from the [Apache official repository](https://repo.maven.apache.org/maven2/org/apache/iceberg/iceberg-flink-runtime/). Third-party jars can be added to `pyflink` via: @@ -123,7 +123,7 @@ Provide a `file://` path to the `iceberg-flink-runtime` jar, which can be obtain - `env.add_jars("file:///my/jar/path/connector.jar")` - `table_env.get_config().get_configuration().set_string("pipeline.jars", "file:///my/jar/path/connector.jar")` -This is also mentioned in the official [docs](https://ci.apache.org/projects/flink/flink-docs-release-1.16/docs/dev/python/dependency_management/). The example below uses `env.add_jars(..)`: +This is also mentioned in the official [docs](https://ci.apache.org/projects/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/python/dependency_management/). The example below uses `env.add_jars(..)`: ```python import os @@ -131,7 +131,7 @@ import os from pyflink.datastream import StreamExecutionEnvironment env = StreamExecutionEnvironment.get_execution_environment() -iceberg_flink_runtime_jar = os.path.join(os.getcwd(), "iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar") +iceberg_flink_runtime_jar = os.path.join(os.getcwd(), "iceberg-flink-runtime-{{ flinkVersionMajor }}-{{ icebergVersion }}.jar") env.add_jars("file://{}".format(iceberg_flink_runtime_jar)) ``` @@ -172,7 +172,7 @@ Run a query: 5 rows in set ``` -For more details, please refer to the [Python Table API](https://ci.apache.org/projects/flink/flink-docs-release-1.16/docs/dev/python/table/intro_to_table_api/). +For more details, please refer to the [Python Table API](https://ci.apache.org/projects/flink/flink-docs-release-{{ flinkVersionMajor }}/docs/dev/python/table/intro_to_table_api/). ## Adding catalogs. diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 899851984cad..0642281a7c24 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -80,6 +80,8 @@ markdown_extensions: extra: icebergVersion: '1.5.2' nessieVersion: '0.77.1' + flinkVersion: '1.19.0' + flinkVersionMajor: '1.19' social: - icon: fontawesome/regular/comments link: 'https://iceberg.apache.org/community/' From 5a9aee25c15514a3e16c045ae081093d0fd18c6e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 18 Jun 2024 11:41:48 +0200 Subject: [PATCH 0385/1019] Flink: Import Assertions statically (#10532) --- .../iceberg/flink/HadoopCatalogExtension.java | 5 +- .../org/apache/iceberg/flink/TestBase.java | 10 ++-- .../iceberg/flink/TestCatalogTableLoader.java | 15 ++--- .../flink/TestFlinkCatalogDatabase.java | 7 +-- .../iceberg/flink/TestFlinkCatalogTable.java | 29 ++++----- .../TestFlinkCatalogTablePartitions.java | 4 +- .../org/apache/iceberg/flink/TestHelpers.java | 3 +- .../actions/TestRewriteDataFilesAction.java | 4 +- .../flink/data/TestRowDataProjection.java | 60 +++++++++---------- .../sink/TestBucketPartitionKeySelector.java | 8 ++- .../flink/sink/TestBucketPartitioner.java | 15 ++--- ...TestBucketPartitionerFlinkIcebergSink.java | 14 ++--- .../flink/sink/TestFlinkIcebergSink.java | 9 +-- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../sink/TestFlinkIcebergSinkV2Base.java | 7 ++- .../flink/sink/TestIcebergStreamWriter.java | 5 +- .../sink/shuffle/TestMapDataStatistics.java | 5 +- .../sink/shuffle/TestMapRangePartitioner.java | 25 ++++---- .../iceberg/flink/source/TestScanContext.java | 5 +- .../flink/source/TestStreamScanSql.java | 4 +- ...tFileSequenceNumberBasedSplitAssigner.java | 5 +- .../TestContinuousIcebergEnumerator.java | 10 ++-- .../TestContinuousSplitPlannerImpl.java | 11 ++-- ...ntinuousSplitPlannerImplStartStrategy.java | 7 ++- .../TestColumnStatsWatermarkExtractor.java | 6 +- .../iceberg/flink/HadoopCatalogExtension.java | 5 +- .../org/apache/iceberg/flink/TestBase.java | 10 ++-- .../iceberg/flink/TestCatalogTableLoader.java | 15 ++--- .../flink/TestFlinkCatalogDatabase.java | 7 +-- .../iceberg/flink/TestFlinkCatalogTable.java | 29 ++++----- .../TestFlinkCatalogTablePartitions.java | 4 +- .../org/apache/iceberg/flink/TestHelpers.java | 3 +- .../actions/TestRewriteDataFilesAction.java | 4 +- .../flink/data/TestRowDataProjection.java | 60 +++++++++---------- .../sink/TestBucketPartitionKeySelector.java | 8 ++- .../flink/sink/TestBucketPartitioner.java | 15 ++--- ...TestBucketPartitionerFlinkIcebergSink.java | 14 ++--- .../flink/sink/TestFlinkIcebergSink.java | 9 +-- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../sink/TestFlinkIcebergSinkV2Base.java | 7 ++- .../flink/sink/TestIcebergStreamWriter.java | 5 +- .../sink/shuffle/TestMapDataStatistics.java | 5 +- .../sink/shuffle/TestMapRangePartitioner.java | 25 ++++---- .../iceberg/flink/source/TestScanContext.java | 5 +- .../flink/source/TestStreamScanSql.java | 4 +- ...tFileSequenceNumberBasedSplitAssigner.java | 5 +- .../TestContinuousIcebergEnumerator.java | 10 ++-- .../TestContinuousSplitPlannerImpl.java | 11 ++-- ...ntinuousSplitPlannerImplStartStrategy.java | 7 ++- .../TestColumnStatsWatermarkExtractor.java | 6 +- .../iceberg/flink/HadoopCatalogExtension.java | 5 +- .../org/apache/iceberg/flink/TestBase.java | 9 +-- .../iceberg/flink/TestCatalogTableLoader.java | 15 ++--- .../flink/TestFlinkCatalogDatabase.java | 7 +-- .../iceberg/flink/TestFlinkCatalogTable.java | 29 ++++----- .../TestFlinkCatalogTablePartitions.java | 4 +- .../org/apache/iceberg/flink/TestHelpers.java | 3 +- .../actions/TestRewriteDataFilesAction.java | 4 +- .../flink/data/TestRowDataProjection.java | 60 +++++++++---------- .../sink/TestBucketPartitionKeySelector.java | 8 ++- .../flink/sink/TestBucketPartitioner.java | 15 ++--- ...TestBucketPartitionerFlinkIcebergSink.java | 14 ++--- .../flink/sink/TestFlinkIcebergSink.java | 9 +-- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../sink/TestFlinkIcebergSinkV2Base.java | 7 ++- .../flink/sink/TestIcebergStreamWriter.java | 5 +- .../sink/shuffle/TestMapDataStatistics.java | 4 +- .../sink/shuffle/TestMapRangePartitioner.java | 25 ++++---- .../shuffle/TestSketchDataStatistics.java | 4 +- .../iceberg/flink/source/TestScanContext.java | 5 +- .../flink/source/TestStreamScanSql.java | 4 +- ...tFileSequenceNumberBasedSplitAssigner.java | 5 +- .../TestContinuousIcebergEnumerator.java | 10 ++-- .../TestContinuousSplitPlannerImpl.java | 11 ++-- ...ntinuousSplitPlannerImplStartStrategy.java | 7 ++- .../TestColumnStatsWatermarkExtractor.java | 6 +- 76 files changed, 426 insertions(+), 413 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java index d8e1325254d9..fd5c6b76b683 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.file.Files; import java.nio.file.Path; import java.util.UUID; @@ -28,7 +30,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -63,7 +64,7 @@ public void afterAll(ExtensionContext context) throws Exception { @Override public void beforeEach(ExtensionContext context) throws Exception { - Assertions.assertThat(temporaryFolder).exists().isDirectory(); + assertThat(temporaryFolder).exists().isDirectory(); this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); this.catalogLoader = CatalogLoader.hadoop( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 3986f1a796a5..6367a064f283 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.file.Path; import java.util.List; import org.apache.flink.table.api.EnvironmentSettings; @@ -33,7 +35,6 @@ import org.apache.iceberg.hive.TestHiveMetastore; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.extension.RegisterExtension; @@ -105,14 +106,11 @@ protected List sql(String query, Object... args) { } protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); } protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); } /** diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java index 147d2a173ddc..f719c7bc0001 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; @@ -34,7 +36,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -50,7 +51,7 @@ public class TestCatalogTableLoader extends TestBase { @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assertions.assertThat(warehouse.delete()).isTrue(); + assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } @@ -59,9 +60,7 @@ public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assertions.assertThat(fs.delete(warehousePath, true)) - .as("Failed to delete " + warehousePath) - .isTrue(); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); } } @@ -94,11 +93,9 @@ private static void validateTableLoader(TableLoader loader) private static void validateHadoopConf(Table table) { FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); + assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); } @SuppressWarnings("unchecked") diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index f46d50a5f0ab..6850060e8fc8 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; 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.nio.file.Path; @@ -31,7 +32,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -105,7 +105,7 @@ public void testDropNonEmptyNamespace() { assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) .as("Table should exist") .isTrue(); - Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) .cause() .isInstanceOf(DatabaseNotEmptyException.class) .hasMessage( @@ -243,8 +243,7 @@ public void testHadoopNotSupportMeta() { assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) .as("Namespace should not already exist") .isFalse(); - Assertions.assertThatThrownBy( - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + assertThatThrownBy(() -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) .cause() .isInstanceOf(UnsupportedOperationException.class) .hasMessage( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 3aea32e4ecc1..7a364b856398 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -54,7 +54,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -161,7 +160,7 @@ public void testCreateTableIfNotExists() { assertThat(table("tl")).isNotNull(); sql("DROP TABLE tl"); - Assertions.assertThatThrownBy(() -> table("tl")) + assertThatThrownBy(() -> table("tl")) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); @@ -271,7 +270,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Excep assertThat(ops.refresh().formatVersion()) .as("should create table using format v2") .isEqualTo(2); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) + assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) .rootCause() .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot downgrade v2 table to v1"); @@ -342,12 +341,12 @@ public void testAlterTableAddColumn() { .asStruct()); // Adding a required field should fail because Iceberg's SchemaUpdate does not allow // incompatible changes. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); // Adding an existing field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("Try to add a column `id` which already exists in the table."); } @@ -380,12 +379,12 @@ public void testAlterTableDropColumn() { .isEqualTo( new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); // Dropping an non-existing field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("The column `foo` does not exist in the base table."); // Dropping an already-deleted field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("The column `dt` does not exist in the base table."); } @@ -431,7 +430,7 @@ public void testAlterTableModifyColumnType() { .asStruct()); // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's // validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot change column type: dt: string -> int"); @@ -449,7 +448,7 @@ public void testAlterTableModifyColumnNullability() { .asStruct()); // Changing nullability from optional to required should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); @@ -495,13 +494,13 @@ public void testAlterTableModifyColumnPosition() { .asStruct()); // Modifying the position of a non-existing column should fail due to Flink's internal // validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Try to modify a column `non_existing` which does not exist in the table."); // Moving a column after a non-existing column should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Referenced column `non_existing` by 'AFTER' does not exist in the table."); @@ -567,22 +566,20 @@ public void testAlterTableConstraint() { assertThat(schemaAfterComposite.identifierFieldNames()).containsExactlyInAnyOrder("id", "dt"); // Setting an optional field as primary key should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); // Setting a composite key containing an optional field should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); // Dropping constraints is not supported yet - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(UnsupportedOperationException.class) .hasRootCauseMessage("Unsupported table change: DropConstraint."); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index b32be379caeb..0bb08e4f265a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.flink.table.catalog.CatalogPartitionSpec; @@ -32,7 +33,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -89,7 +89,7 @@ public void testListPartitionsWithUnpartitionedTable() { ObjectPath objectPath = new ObjectPath(DATABASE, tableName); FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) + assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) .isInstanceOf(TableNotPartitionedException.class) .hasMessageStartingWith("Table db.test_table in catalog") .hasMessageEndingWith("is not partitioned."); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 80e5ddd24fba..a243be5834a8 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -68,7 +68,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; public class TestHelpers { private TestHelpers() {} @@ -159,7 +158,7 @@ public static void assertRows(List results, List expected) { } public static void assertRowsWithOrder(List results, List expected) { - Assertions.assertThat(results).containsExactlyElementsOf(expected); + assertThat(results).containsExactlyElementsOf(expected); } public static void assertRowData(Schema schema, StructLike expected, RowData actual) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 4220775f41fa..800cce96edac 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -62,7 +63,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -427,7 +427,7 @@ public void testRewriteNoConflictWithEqualityDeletes() throws IOException { TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> Actions.forTable(stale1) .rewriteDataFiles() diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java index 3cd25c8fa983..d078b2228456 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.data; +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 java.util.List; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -35,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructProjection; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestRowDataProjection { @@ -48,7 +51,7 @@ public void testNullRootRowData() { RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); - Assertions.assertThatThrownBy(() -> projection.wrap(null)) + assertThatThrownBy(() -> projection.wrap(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid row data: null"); } @@ -161,7 +164,7 @@ public void testNestedProjection() { // Project id only. Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -174,7 +177,7 @@ public void testNestedProjection() { 3, "location", Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); + assertThat(latOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, latOnly); testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -187,7 +190,7 @@ public void testNestedProjection() { 3, "location", Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); + assertThat(longOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, longOnly); testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -195,7 +198,7 @@ public void testNestedProjection() { // Project location. Schema locationOnly = schema.select("location"); - Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); + assertThat(locationOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, locationOnly); testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -250,12 +253,12 @@ public void testMapOfPrimitivesProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project map only. Schema mapOnly = schema.select("map_of_primitives"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + assertThat(mapOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, mapOnly); // Project all. @@ -308,12 +311,12 @@ public void testMapOfStructStructProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project map only. Schema mapOnly = schema.select("map"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + assertThat(mapOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, mapOnly); // Project all. @@ -333,7 +336,7 @@ public void testMapOfStructStructProjection() { Types.StructType.of( Types.NestedField.required(203, "value", Types.LongType.get()), Types.NestedField.required(204, "valueData", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) + assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial map key or value struct."); @@ -351,7 +354,7 @@ public void testMapOfStructStructProjection() { Types.NestedField.required(202, "keyData", Types.StringType.get())), Types.StructType.of( Types.NestedField.required(203, "value", Types.LongType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) + assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial map key or value struct."); @@ -412,12 +415,12 @@ public void testArrayOfPrimitiveProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project list only. Schema arrayOnly = schema.select("array_of_int"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, arrayOnly); // Project all. @@ -474,12 +477,12 @@ public void testArrayOfStructProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project list only. Schema arrayOnly = schema.select("array_of_struct"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, arrayOnly); // Project all. @@ -496,7 +499,7 @@ public void testArrayOfStructProjection() { Types.StructType.of( Types.NestedField.required(202, "name", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) + assertThatThrownBy(() -> generateAndValidate(schema, partialList)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial list element struct."); @@ -534,7 +537,7 @@ private void generateAndValidate(Schema schema, Schema projectSchema) { List recordList = RandomGenericData.generate(schema, numRecords, 102L); List rowDataList = Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); - Assertions.assertThat(rowDataList).hasSize(recordList.size()); + assertThat(rowDataList).hasSize(recordList.size()); StructProjection structProjection = StructProjection.create(schema, projectSchema); RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); @@ -544,10 +547,10 @@ private void generateAndValidate(Schema schema, Schema projectSchema) { RowData projected = rowDataProjection.wrap(rowDataList.get(i)); TestHelpers.assertRowData(projectSchema, expected, projected); - Assertions.assertThat(projected).isEqualTo(projected); - Assertions.assertThat(projected).hasSameHashCodeAs(projected); + assertThat(projected).isEqualTo(projected); + assertThat(projected).hasSameHashCodeAs(projected); // make sure toString doesn't throw NPE for null values - Assertions.assertThatNoException().isThrownBy(projected::toString); + assertThatNoException().isThrownBy(projected::toString); } } @@ -575,18 +578,15 @@ private void testEqualsAndHashCode( RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(copyProjection.wrap(copyRowData)); if (isOtherRowDataSameAsRowData) { - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(otherProjection.wrap(otherRowData)); } else { - Assertions.assertThat(projection.wrap(rowData)) - .isNotEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) + assertThat(projection.wrap(rowData)).isNotEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)) .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index 5ebcc6361c7b..abac605f81fd 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -47,7 +49,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { TestBucketPartitionerUtil.computeBucketId( numBuckets, rowData.getString(1).toString()); Integer key = keySelector.getKey(rowData); - Assertions.assertThat(key).isEqualTo(expectedBucketId); + assertThat(key).isEqualTo(expectedBucketId); }); } @@ -55,7 +57,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { public void testKeySelectorMultipleBucketsFail() { PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy( () -> new BucketPartitionKeySelector( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 835713e6b417..59bdba578ebb 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -21,10 +21,11 @@ import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; @@ -46,7 +47,7 @@ public void testPartitioningParallelismGreaterThanBuckets( int bucketId = 0; for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); + assertThat(actualPartitionIndex).isEqualTo(expectedIndex); bucketId++; if (bucketId == numBuckets) { bucketId = 0; @@ -66,7 +67,7 @@ public void testPartitioningParallelismEqualLessThanBuckets( for (int bucketId = 0; bucketId < numBuckets; bucketId++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); + assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); } } @@ -75,7 +76,7 @@ public void testPartitionerBucketIdNullFail() { PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) .withMessage(BUCKET_NULL_MESSAGE); } @@ -84,7 +85,7 @@ public void testPartitionerBucketIdNullFail() { public void testPartitionerMultipleBucketsFail() { PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> new BucketPartitioner(partitionSpec)) .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); } @@ -95,12 +96,12 @@ public void testPartitionerBucketIdOutOfRangeFail() { BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int negativeBucketId = -1; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); int tooBigBucketId = DEFAULT_NUM_BUCKETS; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 9dae43ce5e58..1ec0cc53b45a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.util.Arrays; @@ -52,7 +53,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -145,21 +145,21 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t appendRowsToTable(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); - Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); + assertThat(stats.totalRowCount).isEqualTo(rows.size()); // All 4 buckets should've been written to - Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); - Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); // Writer expectation (2 writers per bucket): // - Bucket0 -> Writers [0, 4] // - Bucket1 -> Writers [1, 5] // - Bucket2 -> Writers [2, 6] // - Bucket3 -> Writers [3, 7] for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { - Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); + assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); // 2 files per bucket (one file is created by each writer) - Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); + assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); // 2 rows per file (total of 16 rows across 8 files) - Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); + assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 11a73d2cc144..9ff79419b020 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Collections; import java.util.List; @@ -45,7 +47,6 @@ 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.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -198,7 +199,7 @@ public void testJobHashDistributionMode() { .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) .commit(); - Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Flink does not support 'range' write distribution mode now."); } @@ -346,7 +347,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .writeParallelism(parallelism) .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) + assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } @@ -366,7 +367,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .writeParallelism(parallelism) .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) + assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file format: UNRECOGNIZED"); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 5e81c279b69b..7712481d33d9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -43,7 +44,6 @@ 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.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -210,7 +210,7 @@ public void testUpsertModeCheck() throws Exception { .writeParallelism(parallelism) .upsert(true); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> builder .equalityFieldColumns(ImmutableList.of("id", "data")) @@ -220,7 +220,7 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) .isInstanceOf(IllegalStateException.class) .hasMessage( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 0b403756cefe..9cdf7743c485 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -42,7 +44,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; import org.junit.Assert; public class TestFlinkIcebergSinkV2Base { @@ -231,7 +232,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> testChangeLogs( ImmutableList.of("id"), @@ -278,7 +279,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { expectedRecords, branch); } else { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> testChangeLogs( ImmutableList.of("id"), diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 0968f89f55e0..98f51c9001c7 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -55,7 +57,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -216,7 +217,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); + assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); long expectedDataFiles = partitioned ? 2 : 1; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index a07808e935d9..be2beeebc93c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; @@ -28,7 +30,6 @@ import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { @@ -85,6 +86,6 @@ public void testAddsAndGet() { SortKey keyC = sortKey.copy(); Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index 92eb71acc834..e6726e7db785 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.Set; @@ -37,7 +39,6 @@ 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.Pair; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapRangePartitioner { @@ -121,7 +122,7 @@ public void testEvenlyDividableNoClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -144,7 +145,7 @@ public void testEvenlyDividableNoClosingFileCost() { 7, Pair.of(100L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -192,7 +193,7 @@ public void testEvenlyDividableWithClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -216,7 +217,7 @@ public void testEvenlyDividableWithClosingFileCost() { 7, Pair.of(75L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -259,7 +260,7 @@ public void testNonDividableNoClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -284,7 +285,7 @@ public void testNonDividableNoClosingFileCost() { 8, Pair.of(88L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -332,7 +333,7 @@ public void testNonDividableWithClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -358,7 +359,7 @@ public void testNonDividableWithClosingFileCost() { 8, Pair.of(61L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -400,7 +401,7 @@ private void validatePartitionResults( Map>> partitionResults, double maxDriftPercentage) { - Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); List expectedAssignedKeyCounts = Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); @@ -428,7 +429,7 @@ private void validatePartitionResults( }); // number of assigned keys should match exactly - Assertions.assertThat(actualAssignedKeyCounts) + assertThat(actualAssignedKeyCounts) .as("the number of assigned keys should match for every subtask") .isEqualTo(expectedAssignedKeyCounts); @@ -438,7 +439,7 @@ private void validatePartitionResults( double expectedWeight = expectedNormalizedWeights.get(subtaskId); double min = expectedWeight * (1 - maxDriftPercentage / 100); double max = expectedWeight * (1 + maxDriftPercentage / 100); - Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + assertThat(actualNormalizedWeights.get(subtaskId)) .as( "Subtask %d weight should within %.1f percent of the expected range %s", subtaskId, maxDriftPercentage, expectedWeight) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java index 146f1926257a..5dd7de545e11 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg.flink.source; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.junit.jupiter.api.Test; class TestScanContext { @@ -104,7 +105,7 @@ void testMaxAllowedPlanningFailures() { } private void assertException(ScanContext context, String message) { - Assertions.assertThatThrownBy(() -> context.validate()) + assertThatThrownBy(() -> context.validate()) .hasMessage(message) .isInstanceOf(IllegalArgumentException.class); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 421498464602..581e063bb279 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Iterator; @@ -44,7 +45,6 @@ import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -420,7 +420,7 @@ public void testConsumeFromStartTag() throws Exception { } result.getJobClient().ifPresent(JobClient::cancel); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> exec( "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index e78634e6b873..20555ad464a9 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.ContentFile; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -38,7 +39,7 @@ protected SplitAssigner splitAssigner() { @Test public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 349eb11cf549..95770bb7dc2f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -33,7 +36,6 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -97,7 +99,7 @@ public void testDiscoverWhenReaderRegistered() throws Exception { enumeratorContext.triggerAllActions(); Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -144,7 +146,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio enumerator.handleSourceEvent(2, new SplitRequestEvent()); Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -289,7 +291,7 @@ public void testOverMaxAllowedPlanningErrors() throws Exception { enumeratorContext.triggerAllActions(); Assert.assertTrue( enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) .hasCauseInstanceOf(RuntimeException.class) .hasMessageContaining("Failed to discover new split"); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 1bb2b1c6bf85..f98e93835d15 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Set; @@ -36,7 +38,6 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -338,7 +339,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); } @@ -364,7 +365,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); } @@ -426,7 +427,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find a snapshot after: 1"); } @@ -447,7 +448,7 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find a snapshot after:"); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index 2c94f215908a..ce041dbbcf59 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import org.apache.iceberg.FileFormat; @@ -29,7 +31,6 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -135,7 +136,7 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .build(); // empty table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( tableResource.table(), scanContextInvalidSnapshotId)) @@ -166,7 +167,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .build(); // empty table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( tableResource.table(), scanContextInvalidSnapshotTimestamp)) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index afe8a5d0152c..604bc09619e0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.reader; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.time.LocalDateTime; @@ -39,7 +40,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.BeforeClass; @@ -149,7 +149,7 @@ public void testMultipleFiles() throws IOException { @Test public void testWrongColumn() { Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); - Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); @@ -162,7 +162,7 @@ public void testEmptyStatistics() throws IOException { // Create an extractor for a column we do not have statistics ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(10, "missing_field"); - Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + assertThatThrownBy(() -> extractor.extractWatermark(split(0))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Missing statistics for column"); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java index d8e1325254d9..fd5c6b76b683 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.file.Files; import java.nio.file.Path; import java.util.UUID; @@ -28,7 +30,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -63,7 +64,7 @@ public void afterAll(ExtensionContext context) throws Exception { @Override public void beforeEach(ExtensionContext context) throws Exception { - Assertions.assertThat(temporaryFolder).exists().isDirectory(); + assertThat(temporaryFolder).exists().isDirectory(); this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); this.catalogLoader = CatalogLoader.hadoop( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 3986f1a796a5..6367a064f283 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.file.Path; import java.util.List; import org.apache.flink.table.api.EnvironmentSettings; @@ -33,7 +35,6 @@ import org.apache.iceberg.hive.TestHiveMetastore; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.extension.RegisterExtension; @@ -105,14 +106,11 @@ protected List sql(String query, Object... args) { } protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); } protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); } /** diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java index 147d2a173ddc..f719c7bc0001 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; @@ -34,7 +36,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -50,7 +51,7 @@ public class TestCatalogTableLoader extends TestBase { @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assertions.assertThat(warehouse.delete()).isTrue(); + assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } @@ -59,9 +60,7 @@ public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assertions.assertThat(fs.delete(warehousePath, true)) - .as("Failed to delete " + warehousePath) - .isTrue(); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); } } @@ -94,11 +93,9 @@ private static void validateTableLoader(TableLoader loader) private static void validateHadoopConf(Table table) { FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); + assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); } @SuppressWarnings("unchecked") diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index f46d50a5f0ab..6850060e8fc8 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; 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.nio.file.Path; @@ -31,7 +32,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -105,7 +105,7 @@ public void testDropNonEmptyNamespace() { assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) .as("Table should exist") .isTrue(); - Assertions.assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) + assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) .cause() .isInstanceOf(DatabaseNotEmptyException.class) .hasMessage( @@ -243,8 +243,7 @@ public void testHadoopNotSupportMeta() { assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) .as("Namespace should not already exist") .isFalse(); - Assertions.assertThatThrownBy( - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + assertThatThrownBy(() -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) .cause() .isInstanceOf(UnsupportedOperationException.class) .hasMessage( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java index 3aea32e4ecc1..7a364b856398 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -54,7 +54,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -161,7 +160,7 @@ public void testCreateTableIfNotExists() { assertThat(table("tl")).isNotNull(); sql("DROP TABLE tl"); - Assertions.assertThatThrownBy(() -> table("tl")) + assertThatThrownBy(() -> table("tl")) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); @@ -271,7 +270,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Excep assertThat(ops.refresh().formatVersion()) .as("should create table using format v2") .isEqualTo(2); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) + assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) .rootCause() .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot downgrade v2 table to v1"); @@ -342,12 +341,12 @@ public void testAlterTableAddColumn() { .asStruct()); // Adding a required field should fail because Iceberg's SchemaUpdate does not allow // incompatible changes. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); // Adding an existing field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("Try to add a column `id` which already exists in the table."); } @@ -380,12 +379,12 @@ public void testAlterTableDropColumn() { .isEqualTo( new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); // Dropping an non-existing field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("The column `foo` does not exist in the base table."); // Dropping an already-deleted field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("The column `dt` does not exist in the base table."); } @@ -431,7 +430,7 @@ public void testAlterTableModifyColumnType() { .asStruct()); // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's // validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot change column type: dt: string -> int"); @@ -449,7 +448,7 @@ public void testAlterTableModifyColumnNullability() { .asStruct()); // Changing nullability from optional to required should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); @@ -495,13 +494,13 @@ public void testAlterTableModifyColumnPosition() { .asStruct()); // Modifying the position of a non-existing column should fail due to Flink's internal // validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Try to modify a column `non_existing` which does not exist in the table."); // Moving a column after a non-existing column should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Referenced column `non_existing` by 'AFTER' does not exist in the table."); @@ -567,22 +566,20 @@ public void testAlterTableConstraint() { assertThat(schemaAfterComposite.identifierFieldNames()).containsExactlyInAnyOrder("id", "dt"); // Setting an optional field as primary key should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); // Setting a composite key containing an optional field should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); // Dropping constraints is not supported yet - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(UnsupportedOperationException.class) .hasRootCauseMessage("Unsupported table change: DropConstraint."); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index b32be379caeb..0bb08e4f265a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.flink.table.catalog.CatalogPartitionSpec; @@ -32,7 +33,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -89,7 +89,7 @@ public void testListPartitionsWithUnpartitionedTable() { ObjectPath objectPath = new ObjectPath(DATABASE, tableName); FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) + assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) .isInstanceOf(TableNotPartitionedException.class) .hasMessageStartingWith("Table db.test_table in catalog") .hasMessageEndingWith("is not partitioned."); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index 80e5ddd24fba..a243be5834a8 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -68,7 +68,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; public class TestHelpers { private TestHelpers() {} @@ -159,7 +158,7 @@ public static void assertRows(List results, List expected) { } public static void assertRowsWithOrder(List results, List expected) { - Assertions.assertThat(results).containsExactlyElementsOf(expected); + assertThat(results).containsExactlyElementsOf(expected); } public static void assertRowData(Schema schema, StructLike expected, RowData actual) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java index 4220775f41fa..800cce96edac 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -62,7 +63,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -427,7 +427,7 @@ public void testRewriteNoConflictWithEqualityDeletes() throws IOException { TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> Actions.forTable(stale1) .rewriteDataFiles() diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java index 3cd25c8fa983..d078b2228456 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.data; +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 java.util.List; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -35,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructProjection; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestRowDataProjection { @@ -48,7 +51,7 @@ public void testNullRootRowData() { RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); - Assertions.assertThatThrownBy(() -> projection.wrap(null)) + assertThatThrownBy(() -> projection.wrap(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid row data: null"); } @@ -161,7 +164,7 @@ public void testNestedProjection() { // Project id only. Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -174,7 +177,7 @@ public void testNestedProjection() { 3, "location", Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); + assertThat(latOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, latOnly); testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -187,7 +190,7 @@ public void testNestedProjection() { 3, "location", Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); + assertThat(longOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, longOnly); testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -195,7 +198,7 @@ public void testNestedProjection() { // Project location. Schema locationOnly = schema.select("location"); - Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); + assertThat(locationOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, locationOnly); testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -250,12 +253,12 @@ public void testMapOfPrimitivesProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project map only. Schema mapOnly = schema.select("map_of_primitives"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + assertThat(mapOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, mapOnly); // Project all. @@ -308,12 +311,12 @@ public void testMapOfStructStructProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project map only. Schema mapOnly = schema.select("map"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + assertThat(mapOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, mapOnly); // Project all. @@ -333,7 +336,7 @@ public void testMapOfStructStructProjection() { Types.StructType.of( Types.NestedField.required(203, "value", Types.LongType.get()), Types.NestedField.required(204, "valueData", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) + assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial map key or value struct."); @@ -351,7 +354,7 @@ public void testMapOfStructStructProjection() { Types.NestedField.required(202, "keyData", Types.StringType.get())), Types.StructType.of( Types.NestedField.required(203, "value", Types.LongType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) + assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial map key or value struct."); @@ -412,12 +415,12 @@ public void testArrayOfPrimitiveProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project list only. Schema arrayOnly = schema.select("array_of_int"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, arrayOnly); // Project all. @@ -474,12 +477,12 @@ public void testArrayOfStructProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project list only. Schema arrayOnly = schema.select("array_of_struct"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, arrayOnly); // Project all. @@ -496,7 +499,7 @@ public void testArrayOfStructProjection() { Types.StructType.of( Types.NestedField.required(202, "name", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) + assertThatThrownBy(() -> generateAndValidate(schema, partialList)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial list element struct."); @@ -534,7 +537,7 @@ private void generateAndValidate(Schema schema, Schema projectSchema) { List recordList = RandomGenericData.generate(schema, numRecords, 102L); List rowDataList = Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); - Assertions.assertThat(rowDataList).hasSize(recordList.size()); + assertThat(rowDataList).hasSize(recordList.size()); StructProjection structProjection = StructProjection.create(schema, projectSchema); RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); @@ -544,10 +547,10 @@ private void generateAndValidate(Schema schema, Schema projectSchema) { RowData projected = rowDataProjection.wrap(rowDataList.get(i)); TestHelpers.assertRowData(projectSchema, expected, projected); - Assertions.assertThat(projected).isEqualTo(projected); - Assertions.assertThat(projected).hasSameHashCodeAs(projected); + assertThat(projected).isEqualTo(projected); + assertThat(projected).hasSameHashCodeAs(projected); // make sure toString doesn't throw NPE for null values - Assertions.assertThatNoException().isThrownBy(projected::toString); + assertThatNoException().isThrownBy(projected::toString); } } @@ -575,18 +578,15 @@ private void testEqualsAndHashCode( RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(copyProjection.wrap(copyRowData)); if (isOtherRowDataSameAsRowData) { - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(otherProjection.wrap(otherRowData)); } else { - Assertions.assertThat(projection.wrap(rowData)) - .isNotEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) + assertThat(projection.wrap(rowData)).isNotEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)) .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index 5ebcc6361c7b..abac605f81fd 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -47,7 +49,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { TestBucketPartitionerUtil.computeBucketId( numBuckets, rowData.getString(1).toString()); Integer key = keySelector.getKey(rowData); - Assertions.assertThat(key).isEqualTo(expectedBucketId); + assertThat(key).isEqualTo(expectedBucketId); }); } @@ -55,7 +57,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { public void testKeySelectorMultipleBucketsFail() { PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy( () -> new BucketPartitionKeySelector( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 835713e6b417..59bdba578ebb 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -21,10 +21,11 @@ import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; @@ -46,7 +47,7 @@ public void testPartitioningParallelismGreaterThanBuckets( int bucketId = 0; for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); + assertThat(actualPartitionIndex).isEqualTo(expectedIndex); bucketId++; if (bucketId == numBuckets) { bucketId = 0; @@ -66,7 +67,7 @@ public void testPartitioningParallelismEqualLessThanBuckets( for (int bucketId = 0; bucketId < numBuckets; bucketId++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); + assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); } } @@ -75,7 +76,7 @@ public void testPartitionerBucketIdNullFail() { PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) .withMessage(BUCKET_NULL_MESSAGE); } @@ -84,7 +85,7 @@ public void testPartitionerBucketIdNullFail() { public void testPartitionerMultipleBucketsFail() { PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> new BucketPartitioner(partitionSpec)) .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); } @@ -95,12 +96,12 @@ public void testPartitionerBucketIdOutOfRangeFail() { BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int negativeBucketId = -1; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); int tooBigBucketId = DEFAULT_NUM_BUCKETS; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); } 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 9dae43ce5e58..1ec0cc53b45a 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 @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.util.Arrays; @@ -52,7 +53,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -145,21 +145,21 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t appendRowsToTable(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); - Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); + assertThat(stats.totalRowCount).isEqualTo(rows.size()); // All 4 buckets should've been written to - Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); - Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); // Writer expectation (2 writers per bucket): // - Bucket0 -> Writers [0, 4] // - Bucket1 -> Writers [1, 5] // - Bucket2 -> Writers [2, 6] // - Bucket3 -> Writers [3, 7] for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { - Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); + assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); // 2 files per bucket (one file is created by each writer) - Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); + assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); // 2 rows per file (total of 16 rows across 8 files) - Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); + assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 11a73d2cc144..9ff79419b020 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Collections; import java.util.List; @@ -45,7 +47,6 @@ 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.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -198,7 +199,7 @@ public void testJobHashDistributionMode() { .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) .commit(); - Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Flink does not support 'range' write distribution mode now."); } @@ -346,7 +347,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .writeParallelism(parallelism) .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) + assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } @@ -366,7 +367,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .writeParallelism(parallelism) .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) + assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file format: UNRECOGNIZED"); } 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 5e81c279b69b..7712481d33d9 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -43,7 +44,6 @@ 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.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -210,7 +210,7 @@ public void testUpsertModeCheck() throws Exception { .writeParallelism(parallelism) .upsert(true); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> builder .equalityFieldColumns(ImmutableList.of("id", "data")) @@ -220,7 +220,7 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) .isInstanceOf(IllegalStateException.class) .hasMessage( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 0b403756cefe..9cdf7743c485 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -42,7 +44,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; import org.junit.Assert; public class TestFlinkIcebergSinkV2Base { @@ -231,7 +232,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> testChangeLogs( ImmutableList.of("id"), @@ -278,7 +279,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { expectedRecords, branch); } else { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> testChangeLogs( ImmutableList.of("id"), diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 0968f89f55e0..98f51c9001c7 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -55,7 +57,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -216,7 +217,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); + assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); long expectedDataFiles = partitioned ? 2 : 1; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index a07808e935d9..be2beeebc93c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; @@ -28,7 +30,6 @@ import org.apache.iceberg.flink.RowDataWrapper; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { @@ -85,6 +86,6 @@ public void testAddsAndGet() { SortKey keyC = sortKey.copy(); Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index 92eb71acc834..e6726e7db785 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.Set; @@ -37,7 +39,6 @@ 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.Pair; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapRangePartitioner { @@ -121,7 +122,7 @@ public void testEvenlyDividableNoClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -144,7 +145,7 @@ public void testEvenlyDividableNoClosingFileCost() { 7, Pair.of(100L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -192,7 +193,7 @@ public void testEvenlyDividableWithClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -216,7 +217,7 @@ public void testEvenlyDividableWithClosingFileCost() { 7, Pair.of(75L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -259,7 +260,7 @@ public void testNonDividableNoClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -284,7 +285,7 @@ public void testNonDividableNoClosingFileCost() { 8, Pair.of(88L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -332,7 +333,7 @@ public void testNonDividableWithClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -358,7 +359,7 @@ public void testNonDividableWithClosingFileCost() { 8, Pair.of(61L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -400,7 +401,7 @@ private void validatePartitionResults( Map>> partitionResults, double maxDriftPercentage) { - Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); List expectedAssignedKeyCounts = Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); @@ -428,7 +429,7 @@ private void validatePartitionResults( }); // number of assigned keys should match exactly - Assertions.assertThat(actualAssignedKeyCounts) + assertThat(actualAssignedKeyCounts) .as("the number of assigned keys should match for every subtask") .isEqualTo(expectedAssignedKeyCounts); @@ -438,7 +439,7 @@ private void validatePartitionResults( double expectedWeight = expectedNormalizedWeights.get(subtaskId); double min = expectedWeight * (1 - maxDriftPercentage / 100); double max = expectedWeight * (1 + maxDriftPercentage / 100); - Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + assertThat(actualNormalizedWeights.get(subtaskId)) .as( "Subtask %d weight should within %.1f percent of the expected range %s", subtaskId, maxDriftPercentage, expectedWeight) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java index 146f1926257a..5dd7de545e11 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg.flink.source; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.junit.jupiter.api.Test; class TestScanContext { @@ -104,7 +105,7 @@ void testMaxAllowedPlanningFailures() { } private void assertException(ScanContext context, String message) { - Assertions.assertThatThrownBy(() -> context.validate()) + assertThatThrownBy(() -> context.validate()) .hasMessage(message) .isInstanceOf(IllegalArgumentException.class); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 421498464602..581e063bb279 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Iterator; @@ -44,7 +45,6 @@ import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -420,7 +420,7 @@ public void testConsumeFromStartTag() throws Exception { } result.getJobClient().ifPresent(JobClient::cancel); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> exec( "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " 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 e78634e6b873..20555ad464a9 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 @@ -18,13 +18,14 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.ContentFile; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -38,7 +39,7 @@ protected SplitAssigner splitAssigner() { @Test public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 349eb11cf549..95770bb7dc2f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -33,7 +36,6 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -97,7 +99,7 @@ public void testDiscoverWhenReaderRegistered() throws Exception { enumeratorContext.triggerAllActions(); Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -144,7 +146,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio enumerator.handleSourceEvent(2, new SplitRequestEvent()); Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -289,7 +291,7 @@ public void testOverMaxAllowedPlanningErrors() throws Exception { enumeratorContext.triggerAllActions(); Assert.assertTrue( enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) .hasCauseInstanceOf(RuntimeException.class) .hasMessageContaining("Failed to discover new split"); 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 1bb2b1c6bf85..f98e93835d15 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Set; @@ -36,7 +38,6 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -338,7 +339,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); } @@ -364,7 +365,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); } @@ -426,7 +427,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find a snapshot after: 1"); } @@ -447,7 +448,7 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find a snapshot after:"); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index 2c94f215908a..ce041dbbcf59 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import org.apache.iceberg.FileFormat; @@ -29,7 +31,6 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -135,7 +136,7 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .build(); // empty table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( tableResource.table(), scanContextInvalidSnapshotId)) @@ -166,7 +167,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .build(); // empty table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( tableResource.table(), scanContextInvalidSnapshotTimestamp)) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index afe8a5d0152c..604bc09619e0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.reader; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.time.LocalDateTime; @@ -39,7 +40,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.BeforeClass; @@ -149,7 +149,7 @@ public void testMultipleFiles() throws IOException { @Test public void testWrongColumn() { Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); - Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); @@ -162,7 +162,7 @@ public void testEmptyStatistics() throws IOException { // Create an extractor for a column we do not have statistics ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(10, "missing_field"); - Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + assertThatThrownBy(() -> extractor.extractWatermark(split(0))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Missing statistics for column"); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java index d8e1325254d9..fd5c6b76b683 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.nio.file.Files; import java.nio.file.Path; import java.util.UUID; @@ -28,7 +30,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.hadoop.HadoopCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.AfterAllCallback; import org.junit.jupiter.api.extension.AfterEachCallback; import org.junit.jupiter.api.extension.BeforeAllCallback; @@ -63,7 +64,7 @@ public void afterAll(ExtensionContext context) throws Exception { @Override public void beforeEach(ExtensionContext context) throws Exception { - Assertions.assertThat(temporaryFolder).exists().isDirectory(); + assertThat(temporaryFolder).exists().isDirectory(); this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); this.catalogLoader = CatalogLoader.hadoop( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index e0b429b31b64..773d22e19e64 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; import java.nio.file.Path; import java.util.List; @@ -35,7 +36,6 @@ import org.apache.iceberg.hive.TestHiveMetastore; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.extension.RegisterExtension; @@ -107,14 +107,11 @@ protected List sql(String query, Object... args) { } protected void assertSameElements(Iterable expected, Iterable actual) { - Assertions.assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); } protected void assertSameElements(String message, Iterable expected, Iterable actual) { - Assertions.assertThat(actual) - .isNotNull() - .as(message) - .containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); } /** diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java index 147d2a173ddc..f719c7bc0001 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; @@ -34,7 +36,6 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -50,7 +51,7 @@ public class TestCatalogTableLoader extends TestBase { @BeforeAll public static void createWarehouse() throws IOException { warehouse = File.createTempFile("warehouse", null); - Assertions.assertThat(warehouse.delete()).isTrue(); + assertThat(warehouse.delete()).isTrue(); hiveConf.set("my_key", "my_value"); } @@ -59,9 +60,7 @@ public static void dropWarehouse() throws IOException { if (warehouse != null && warehouse.exists()) { Path warehousePath = new Path(warehouse.getAbsolutePath()); FileSystem fs = warehousePath.getFileSystem(hiveConf); - Assertions.assertThat(fs.delete(warehousePath, true)) - .as("Failed to delete " + warehousePath) - .isTrue(); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); } } @@ -94,11 +93,9 @@ private static void validateTableLoader(TableLoader loader) private static void validateHadoopConf(Table table) { FileIO io = table.io(); - Assertions.assertThat(io) - .as("FileIO should be a HadoopFileIO") - .isInstanceOf(HadoopFileIO.class); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); HadoopFileIO hadoopIO = (HadoopFileIO) io; - Assertions.assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); + assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); } @SuppressWarnings("unchecked") diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java index 38379967044a..bd07087756ad 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; 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.nio.file.Path; @@ -31,7 +32,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -105,7 +105,7 @@ public void testDropNonEmptyNamespace() { assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) .as("Table should exist") .isTrue(); - Assertions.assertThatThrownBy(() -> dropDatabase(flinkDatabase, true)) + assertThatThrownBy(() -> dropDatabase(flinkDatabase, true)) .cause() .isInstanceOf(DatabaseNotEmptyException.class) .hasMessage( @@ -243,8 +243,7 @@ public void testHadoopNotSupportMeta() { assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) .as("Namespace should not already exist") .isFalse(); - Assertions.assertThatThrownBy( - () -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + assertThatThrownBy(() -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) .cause() .isInstanceOf(UnsupportedOperationException.class) .hasMessage( 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 e5b1d61b2f21..0b9c2193b4d5 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 @@ -54,7 +54,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -161,7 +160,7 @@ public void testCreateTableIfNotExists() { assertThat(table("tl")).isNotNull(); sql("DROP TABLE tl"); - Assertions.assertThatThrownBy(() -> table("tl")) + assertThatThrownBy(() -> table("tl")) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); @@ -271,7 +270,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Excep assertThat(ops.refresh().formatVersion()) .as("should create table using format v2") .isEqualTo(2); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) + assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) .rootCause() .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot downgrade v2 table to v1"); @@ -342,12 +341,12 @@ public void testAlterTableAddColumn() { .asStruct()); // Adding a required field should fail because Iceberg's SchemaUpdate does not allow // incompatible changes. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); // Adding an existing field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("Try to add a column `id` which already exists in the table."); } @@ -380,12 +379,12 @@ public void testAlterTableDropColumn() { .isEqualTo( new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); // Dropping an non-existing field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("The column `foo` does not exist in the base table."); // Dropping an already-deleted field should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) .isInstanceOf(ValidationException.class) .hasMessageContaining("The column `dt` does not exist in the base table."); } @@ -431,7 +430,7 @@ public void testAlterTableModifyColumnType() { .asStruct()); // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's // validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot change column type: dt: string -> int"); @@ -449,7 +448,7 @@ public void testAlterTableModifyColumnNullability() { .asStruct()); // Changing nullability from optional to required should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); @@ -495,13 +494,13 @@ public void testAlterTableModifyColumnPosition() { .asStruct()); // Modifying the position of a non-existing column should fail due to Flink's internal // validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Try to modify a column `non_existing` which does not exist in the table."); // Moving a column after a non-existing column should fail due to Flink's internal validation. - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Referenced column `non_existing` by 'AFTER' does not exist in the table."); @@ -567,22 +566,20 @@ public void testAlterTableConstraint() { assertThat(schemaAfterComposite.identifierFieldNames()).containsExactlyInAnyOrder("id", "dt"); // Setting an optional field as primary key should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); // Setting a composite key containing an optional field should fail // because Iceberg's SchemaUpdate does not allow incompatible changes. - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(IllegalArgumentException.class) .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); // Dropping constraints is not supported yet - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) .isInstanceOf(TableException.class) .hasRootCauseInstanceOf(UnsupportedOperationException.class) .hasRootCauseMessage("Unsupported table change: DropConstraint."); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index a6feb2607774..97ab78716b8b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.flink.table.catalog.CatalogPartitionSpec; @@ -32,7 +33,6 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -89,7 +89,7 @@ public void testListPartitionsWithUnpartitionedTable() { ObjectPath objectPath = new ObjectPath(DATABASE, tableName); FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - Assertions.assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) + assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) .isInstanceOf(TableNotPartitionedException.class) .hasMessageStartingWith("Table db.test_table in catalog") .hasMessageEndingWith("is not partitioned."); 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 80e5ddd24fba..a243be5834a8 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 @@ -68,7 +68,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; public class TestHelpers { private TestHelpers() {} @@ -159,7 +158,7 @@ public static void assertRows(List results, List expected) { } public static void assertRowsWithOrder(List results, List expected) { - Assertions.assertThat(results).containsExactlyElementsOf(expected); + assertThat(results).containsExactlyElementsOf(expected); } public static void assertRowData(Schema schema, StructLike expected, RowData actual) { 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 5d8cce07330a..6b8399f666d4 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -62,7 +63,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -427,7 +427,7 @@ public void testRewriteNoConflictWithEqualityDeletes() throws IOException { TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> Actions.forTable(stale1) .rewriteDataFiles() diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java index 3cd25c8fa983..d078b2228456 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.data; +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 java.util.List; import org.apache.flink.table.data.GenericArrayData; import org.apache.flink.table.data.GenericMapData; @@ -35,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructProjection; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestRowDataProjection { @@ -48,7 +51,7 @@ public void testNullRootRowData() { RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); - Assertions.assertThatThrownBy(() -> projection.wrap(null)) + assertThatThrownBy(() -> projection.wrap(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid row data: null"); } @@ -161,7 +164,7 @@ public void testNestedProjection() { // Project id only. Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -174,7 +177,7 @@ public void testNestedProjection() { 3, "location", Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - Assertions.assertThat(latOnly.columns().size()).isGreaterThan(0); + assertThat(latOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, latOnly); testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -187,7 +190,7 @@ public void testNestedProjection() { 3, "location", Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - Assertions.assertThat(longOnly.columns().size()).isGreaterThan(0); + assertThat(longOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, longOnly); testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -195,7 +198,7 @@ public void testNestedProjection() { // Project location. Schema locationOnly = schema.select("location"); - Assertions.assertThat(locationOnly.columns().size()).isGreaterThan(0); + assertThat(locationOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, locationOnly); testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); testEqualsAndHashCode( @@ -250,12 +253,12 @@ public void testMapOfPrimitivesProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project map only. Schema mapOnly = schema.select("map_of_primitives"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + assertThat(mapOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, mapOnly); // Project all. @@ -308,12 +311,12 @@ public void testMapOfStructStructProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project map only. Schema mapOnly = schema.select("map"); - Assertions.assertThat(mapOnly.columns().size()).isGreaterThan(0); + assertThat(mapOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, mapOnly); // Project all. @@ -333,7 +336,7 @@ public void testMapOfStructStructProjection() { Types.StructType.of( Types.NestedField.required(203, "value", Types.LongType.get()), Types.NestedField.required(204, "valueData", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) + assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial map key or value struct."); @@ -351,7 +354,7 @@ public void testMapOfStructStructProjection() { Types.NestedField.required(202, "keyData", Types.StringType.get())), Types.StructType.of( Types.NestedField.required(203, "value", Types.LongType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) + assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial map key or value struct."); @@ -412,12 +415,12 @@ public void testArrayOfPrimitiveProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project list only. Schema arrayOnly = schema.select("array_of_int"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, arrayOnly); // Project all. @@ -474,12 +477,12 @@ public void testArrayOfStructProjection() { // Project id only. Schema idOnly = schema.select("row_id"); - Assertions.assertThat(idOnly.columns().size()).isGreaterThan(0); + assertThat(idOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, idOnly); // Project list only. Schema arrayOnly = schema.select("array_of_struct"); - Assertions.assertThat(arrayOnly.columns().size()).isGreaterThan(0); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); generateAndValidate(schema, arrayOnly); // Project all. @@ -496,7 +499,7 @@ public void testArrayOfStructProjection() { Types.StructType.of( Types.NestedField.required(202, "name", Types.StringType.get()))))); - Assertions.assertThatThrownBy(() -> generateAndValidate(schema, partialList)) + assertThatThrownBy(() -> generateAndValidate(schema, partialList)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot project a partial list element struct."); @@ -534,7 +537,7 @@ private void generateAndValidate(Schema schema, Schema projectSchema) { List recordList = RandomGenericData.generate(schema, numRecords, 102L); List rowDataList = Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); - Assertions.assertThat(rowDataList).hasSize(recordList.size()); + assertThat(rowDataList).hasSize(recordList.size()); StructProjection structProjection = StructProjection.create(schema, projectSchema); RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); @@ -544,10 +547,10 @@ private void generateAndValidate(Schema schema, Schema projectSchema) { RowData projected = rowDataProjection.wrap(rowDataList.get(i)); TestHelpers.assertRowData(projectSchema, expected, projected); - Assertions.assertThat(projected).isEqualTo(projected); - Assertions.assertThat(projected).hasSameHashCodeAs(projected); + assertThat(projected).isEqualTo(projected); + assertThat(projected).hasSameHashCodeAs(projected); // make sure toString doesn't throw NPE for null values - Assertions.assertThatNoException().isThrownBy(projected::toString); + assertThatNoException().isThrownBy(projected::toString); } } @@ -575,18 +578,15 @@ private void testEqualsAndHashCode( RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(copyProjection.wrap(copyRowData)); if (isOtherRowDataSameAsRowData) { - Assertions.assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) - .hasSameHashCodeAs(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(otherProjection.wrap(otherRowData)); } else { - Assertions.assertThat(projection.wrap(rowData)) - .isNotEqualTo(otherProjection.wrap(otherRowData)); - Assertions.assertThat(projection.wrap(rowData)) + assertThat(projection.wrap(rowData)).isNotEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)) .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java index 5ebcc6361c7b..abac605f81fd 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -47,7 +49,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { TestBucketPartitionerUtil.computeBucketId( numBuckets, rowData.getString(1).toString()); Integer key = keySelector.getKey(rowData); - Assertions.assertThat(key).isEqualTo(expectedBucketId); + assertThat(key).isEqualTo(expectedBucketId); }); } @@ -55,7 +57,7 @@ public void testCorrectKeySelection(TableSchemaType tableSchemaType) { public void testKeySelectorMultipleBucketsFail() { PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy( () -> new BucketPartitionKeySelector( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java index 835713e6b417..59bdba578ebb 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -21,10 +21,11 @@ import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.CsvSource; @@ -46,7 +47,7 @@ public void testPartitioningParallelismGreaterThanBuckets( int bucketId = 0; for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(expectedIndex); + assertThat(actualPartitionIndex).isEqualTo(expectedIndex); bucketId++; if (bucketId == numBuckets) { bucketId = 0; @@ -66,7 +67,7 @@ public void testPartitioningParallelismEqualLessThanBuckets( for (int bucketId = 0; bucketId < numBuckets; bucketId++) { int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - Assertions.assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); + assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); } } @@ -75,7 +76,7 @@ public void testPartitionerBucketIdNullFail() { PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) .withMessage(BUCKET_NULL_MESSAGE); } @@ -84,7 +85,7 @@ public void testPartitionerBucketIdNullFail() { public void testPartitionerMultipleBucketsFail() { PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> new BucketPartitioner(partitionSpec)) .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); } @@ -95,12 +96,12 @@ public void testPartitionerBucketIdOutOfRangeFail() { BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); int negativeBucketId = -1; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); int tooBigBucketId = DEFAULT_NUM_BUCKETS; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); } 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 9dae43ce5e58..1ec0cc53b45a 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 @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.util.Arrays; @@ -52,7 +53,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -145,21 +145,21 @@ public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) t appendRowsToTable(rows); TableTestStats stats = extractPartitionResults(tableSchemaType); - Assertions.assertThat(stats.totalRowCount).isEqualTo(rows.size()); + assertThat(stats.totalRowCount).isEqualTo(rows.size()); // All 4 buckets should've been written to - Assertions.assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); - Assertions.assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); // Writer expectation (2 writers per bucket): // - Bucket0 -> Writers [0, 4] // - Bucket1 -> Writers [1, 5] // - Bucket2 -> Writers [2, 6] // - Bucket3 -> Writers [3, 7] for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { - Assertions.assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); + assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); // 2 files per bucket (one file is created by each writer) - Assertions.assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); + assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); // 2 rows per file (total of 16 rows across 8 files) - Assertions.assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); + assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 11a73d2cc144..9ff79419b020 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Collections; import java.util.List; @@ -45,7 +47,6 @@ 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.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -198,7 +199,7 @@ public void testJobHashDistributionMode() { .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) .commit(); - Assertions.assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Flink does not support 'range' write distribution mode now."); } @@ -346,7 +347,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .writeParallelism(parallelism) .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) + assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } @@ -366,7 +367,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .writeParallelism(parallelism) .setAll(newProps); - Assertions.assertThatThrownBy(builder::append) + assertThatThrownBy(builder::append) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file format: UNRECOGNIZED"); } 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 5e81c279b69b..7712481d33d9 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -43,7 +44,6 @@ 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.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -210,7 +210,7 @@ public void testUpsertModeCheck() throws Exception { .writeParallelism(parallelism) .upsert(true); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> builder .equalityFieldColumns(ImmutableList.of("id", "data")) @@ -220,7 +220,7 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) .isInstanceOf(IllegalStateException.class) .hasMessage( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 0b403756cefe..9cdf7743c485 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -42,7 +44,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.assertj.core.api.Assertions; import org.junit.Assert; public class TestFlinkIcebergSinkV2Base { @@ -231,7 +232,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> testChangeLogs( ImmutableList.of("id"), @@ -278,7 +279,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { expectedRecords, branch); } else { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> testChangeLogs( ImmutableList.of("id"), diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 0968f89f55e0..98f51c9001c7 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.Arrays; @@ -55,7 +57,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -216,7 +217,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - Assertions.assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); + assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); long expectedDataFiles = partitioned ? 2 : 1; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index 0ab7d7c3b2ba..8a25c7ad9898 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -20,13 +20,13 @@ import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { @@ -62,6 +62,6 @@ public void testAddsAndGet() { Map actual = (Map) dataStatistics.result(); Map expected = ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index dc3bad83947a..d47e41768072 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.Set; @@ -37,7 +39,6 @@ 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.Pair; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestMapRangePartitioner { @@ -120,7 +121,7 @@ public void testEvenlyDividableNoClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -143,7 +144,7 @@ public void testEvenlyDividableNoClosingFileCost() { 7, Pair.of(100L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -191,7 +192,7 @@ public void testEvenlyDividableWithClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -215,7 +216,7 @@ public void testEvenlyDividableWithClosingFileCost() { 7, Pair.of(75L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -258,7 +259,7 @@ public void testNonDividableNoClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -283,7 +284,7 @@ public void testNonDividableNoClosingFileCost() { 8, Pair.of(88L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -331,7 +332,7 @@ public void testNonDividableWithClosingFileCost() { new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); Map actualAssignment = partitioner.assignment(numPartitions); - Assertions.assertThat(actualAssignment).isEqualTo(expectedAssignment); + assertThat(actualAssignment).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -357,7 +358,7 @@ public void testNonDividableWithClosingFileCost() { 8, Pair.of(61L, 7)); Map> actualAssignmentInfo = partitioner.assignmentInfo(); - Assertions.assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); Map>> partitionResults = runPartitioner(partitioner, numPartitions); @@ -399,7 +400,7 @@ private void validatePartitionResults( Map>> partitionResults, double maxDriftPercentage) { - Assertions.assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); List expectedAssignedKeyCounts = Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); @@ -427,7 +428,7 @@ private void validatePartitionResults( }); // number of assigned keys should match exactly - Assertions.assertThat(actualAssignedKeyCounts) + assertThat(actualAssignedKeyCounts) .as("the number of assigned keys should match for every subtask") .isEqualTo(expectedAssignedKeyCounts); @@ -437,7 +438,7 @@ private void validatePartitionResults( double expectedWeight = expectedNormalizedWeights.get(subtaskId); double min = expectedWeight * (1 - maxDriftPercentage / 100); double max = expectedWeight * (1 + maxDriftPercentage / 100); - Assertions.assertThat(actualNormalizedWeights.get(subtaskId)) + assertThat(actualNormalizedWeights.get(subtaskId)) .as( "Subtask %d weight should within %.1f percent of the expected range %s", subtaskId, maxDriftPercentage, expectedWeight) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java index 658d31c01143..396bfae2f13c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java @@ -20,12 +20,12 @@ import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; import org.apache.iceberg.SortKey; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestSketchDataStatistics { @@ -51,7 +51,7 @@ public void testAddsAndGet() { dataStatistics.add(Fixtures.SORT_KEY); ReservoirItemsSketch actual = (ReservoirItemsSketch) dataStatistics.result(); - Assertions.assertThat(actual.getSamples()) + assertThat(actual.getSamples()) .isEqualTo( new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java index 146f1926257a..5dd7de545e11 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg.flink.source; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.junit.jupiter.api.Test; class TestScanContext { @@ -104,7 +105,7 @@ void testMaxAllowedPlanningFailures() { } private void assertException(ScanContext context, String message) { - Assertions.assertThatThrownBy(() -> context.validate()) + assertThatThrownBy(() -> context.validate()) .hasMessage(message) .isInstanceOf(IllegalArgumentException.class); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 420bcd52bfda..78090c55387a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Iterator; @@ -44,7 +45,6 @@ import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -420,7 +420,7 @@ public void testConsumeFromStartTag() throws Exception { } result.getJobClient().ifPresent(JobClient::cancel); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> exec( "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " 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 e78634e6b873..20555ad464a9 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 @@ -18,13 +18,14 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.ContentFile; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -38,7 +39,7 @@ protected SplitAssigner splitAssigner() { @Test public void testMultipleFilesInAnIcebergSplit() { SplitAssigner assigner = splitAssigner(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), "Multiple files in a split is not allowed") .isInstanceOf(IllegalArgumentException.class) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 349eb11cf549..95770bb7dc2f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -33,7 +36,6 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.ClassRule; import org.junit.Test; @@ -97,7 +99,7 @@ public void testDiscoverWhenReaderRegistered() throws Exception { enumeratorContext.triggerAllActions(); Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -144,7 +146,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio enumerator.handleSourceEvent(2, new SplitRequestEvent()); Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); - Assertions.assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -289,7 +291,7 @@ public void testOverMaxAllowedPlanningErrors() throws Exception { enumeratorContext.triggerAllActions(); Assert.assertTrue( enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) .hasCauseInstanceOf(RuntimeException.class) .hasMessageContaining("Failed to discover new split"); 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 1bb2b1c6bf85..f98e93835d15 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Set; @@ -36,7 +38,6 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.ClassRule; @@ -338,7 +339,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); } @@ -364,7 +365,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); } @@ -426,7 +427,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find a snapshot after: 1"); } @@ -447,7 +448,7 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio new ContinuousSplitPlannerImpl( tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - Assertions.assertThatThrownBy(() -> splitPlanner.planSplits(null)) + assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find a snapshot after:"); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index 2c94f215908a..ce041dbbcf59 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import org.apache.iceberg.FileFormat; @@ -29,7 +31,6 @@ import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -135,7 +136,7 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .build(); // empty table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( tableResource.table(), scanContextInvalidSnapshotId)) @@ -166,7 +167,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .build(); // empty table - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( tableResource.table(), scanContextInvalidSnapshotTimestamp)) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index afe8a5d0152c..604bc09619e0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.reader; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.time.LocalDateTime; @@ -39,7 +40,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.BeforeClass; @@ -149,7 +149,7 @@ public void testMultipleFiles() throws IOException { @Test public void testWrongColumn() { Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); - Assertions.assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); @@ -162,7 +162,7 @@ public void testEmptyStatistics() throws IOException { // Create an extractor for a column we do not have statistics ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(10, "missing_field"); - Assertions.assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + assertThatThrownBy(() -> extractor.extractWatermark(split(0))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Missing statistics for column"); } From cdfc09f46264bb3680b5ab594a2d8176aceda937 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 18 Jun 2024 11:52:40 +0200 Subject: [PATCH 0386/1019] Spark: Import Assertions statically (#10531) --- .../extensions/TestAddFilesProcedure.java | 14 +-- .../TestAlterTablePartitionFields.java | 27 ++--- .../extensions/TestAlterTableSchema.java | 17 ++- .../extensions/TestAncestorsOfProcedure.java | 10 +- .../spark/extensions/TestBranchDDL.java | 65 +++++----- .../extensions/TestCallStatementParser.java | 5 +- .../spark/extensions/TestChangelogTable.java | 5 +- .../TestCherrypickSnapshotProcedure.java | 17 ++- .../extensions/TestConflictValidation.java | 21 ++-- .../extensions/TestCopyOnWriteDelete.java | 4 +- .../extensions/TestCopyOnWriteMerge.java | 4 +- .../extensions/TestCopyOnWriteUpdate.java | 4 +- .../iceberg/spark/extensions/TestDelete.java | 25 ++-- .../TestExpireSnapshotsProcedure.java | 45 +++---- .../iceberg/spark/extensions/TestMerge.java | 87 ++++++------- .../extensions/TestMergeOnReadDelete.java | 5 +- ...TestMetaColumnProjectionWithStageScan.java | 5 +- .../extensions/TestMigrateTableProcedure.java | 12 +- .../TestPublishChangesProcedure.java | 13 +- .../TestRemoveOrphanFilesProcedure.java | 55 ++++----- .../spark/extensions/TestReplaceBranch.java | 7 +- .../TestRequiredDistributionAndOrdering.java | 5 +- .../TestRewriteDataFilesProcedure.java | 35 +++--- .../TestRewriteManifestsProcedure.java | 16 ++- ...stRewritePositionDeleteFilesProcedure.java | 6 +- .../TestRollbackToSnapshotProcedure.java | 24 ++-- .../TestRollbackToTimestampProcedure.java | 20 ++- .../TestSetCurrentSnapshotProcedure.java | 25 ++-- .../TestSetWriteDistributionAndOrdering.java | 7 +- .../TestSnapshotTableProcedure.java | 13 +- .../iceberg/spark/extensions/TestTagDDL.java | 28 ++--- .../iceberg/spark/extensions/TestUpdate.java | 37 +++--- .../spark/extensions/TestWriteAborts.java | 5 +- .../iceberg/TestDataFileSerialization.java | 6 +- .../TestManifestFileSerialization.java | 6 +- .../iceberg/TestScanTaskSerialization.java | 14 +-- .../org/apache/iceberg/ValidationHelpers.java | 5 +- .../iceberg/spark/TestFunctionCatalog.java | 18 +-- .../iceberg/spark/TestSparkTableUtil.java | 7 +- .../TestDeleteReachableFilesAction.java | 7 +- .../actions/TestExpireSnapshotsAction.java | 7 +- .../actions/TestRemoveOrphanFilesAction.java | 19 +-- .../actions/TestRewriteDataFilesAction.java | 26 ++-- .../actions/TestRewriteManifestsAction.java | 4 +- .../spark/actions/TestSparkFileRewriter.java | 47 ++++---- .../iceberg/spark/data/GenericsHelpers.java | 100 ++++++--------- .../iceberg/spark/data/TestHelpers.java | 110 +++++++---------- .../spark/data/TestSparkParquetReader.java | 4 +- .../TestParquetVectorizedReads.java | 7 +- .../spark/source/TestDataFrameWriterV2.java | 7 +- .../spark/source/TestDataFrameWrites.java | 5 +- .../spark/source/TestDataSourceOptions.java | 13 +- .../spark/source/TestFilteredScan.java | 4 +- .../source/TestForwardCompatibility.java | 6 +- .../source/TestIcebergSourceTablesBase.java | 9 +- .../spark/source/TestIcebergSpark.java | 9 +- ...tMetadataTablesWithPartitionEvolution.java | 4 +- .../spark/source/TestPathIdentifier.java | 7 +- .../spark/source/TestReadProjection.java | 5 +- .../TestRequiredDistributionAndOrdering.java | 7 +- .../spark/source/TestSnapshotSelection.java | 23 ++-- .../TestSparkCatalogCacheExpiration.java | 24 ++-- .../spark/source/TestSparkDataWrite.java | 7 +- .../source/TestSparkMetadataColumns.java | 6 +- .../spark/source/TestStructuredStreaming.java | 4 +- .../source/TestStructuredStreamingRead3.java | 38 +++--- .../source/TestTimestampWithoutZone.java | 6 +- .../spark/source/TestWriteMetricsConfig.java | 5 +- .../spark/sql/TestAggregatePushDown.java | 17 +-- .../iceberg/spark/sql/TestAlterTable.java | 12 +- .../iceberg/spark/sql/TestCreateTable.java | 21 ++-- .../iceberg/spark/sql/TestDeleteFrom.java | 6 +- .../iceberg/spark/sql/TestDropTable.java | 5 +- .../iceberg/spark/sql/TestFilterPushDown.java | 11 +- .../iceberg/spark/sql/TestNamespaceSQL.java | 5 +- .../sql/TestPartitionedWritesToWapBranch.java | 8 +- .../apache/iceberg/spark/sql/TestSelect.java | 21 ++-- .../spark/sql/TestSparkBucketFunction.java | 61 +++++----- .../spark/sql/TestSparkDaysFunction.java | 11 +- .../spark/sql/TestSparkHoursFunction.java | 11 +- .../spark/sql/TestSparkMonthsFunction.java | 16 +-- .../spark/sql/TestSparkTruncateFunction.java | 53 ++++---- .../spark/sql/TestSparkYearsFunction.java | 16 +-- .../spark/sql/TestTimestampWithoutZone.java | 6 +- .../sql/TestUnpartitionedWritesToBranch.java | 5 +- .../sql/UnpartitionedWritesTestBase.java | 7 +- .../extensions/TestAddFilesProcedure.java | 14 +-- .../TestAlterTablePartitionFields.java | 27 ++--- .../extensions/TestAlterTableSchema.java | 17 ++- .../extensions/TestAncestorsOfProcedure.java | 10 +- .../spark/extensions/TestBranchDDL.java | 65 +++++----- .../extensions/TestCallStatementParser.java | 5 +- .../spark/extensions/TestChangelogTable.java | 5 +- .../TestCherrypickSnapshotProcedure.java | 17 ++- .../extensions/TestConflictValidation.java | 21 ++-- .../extensions/TestCopyOnWriteDelete.java | 4 +- .../extensions/TestCopyOnWriteMerge.java | 4 +- .../extensions/TestCopyOnWriteUpdate.java | 4 +- .../iceberg/spark/extensions/TestDelete.java | 33 +++-- .../TestExpireSnapshotsProcedure.java | 45 +++---- .../iceberg/spark/extensions/TestMerge.java | 91 +++++++------- .../extensions/TestMergeOnReadDelete.java | 5 +- ...TestMetaColumnProjectionWithStageScan.java | 5 +- .../extensions/TestMigrateTableProcedure.java | 17 +-- .../TestPublishChangesProcedure.java | 13 +- .../TestRemoveOrphanFilesProcedure.java | 55 ++++----- .../spark/extensions/TestReplaceBranch.java | 7 +- .../TestRequiredDistributionAndOrdering.java | 5 +- .../TestRewriteDataFilesProcedure.java | 35 +++--- .../TestRewriteManifestsProcedure.java | 16 ++- ...stRewritePositionDeleteFilesProcedure.java | 6 +- .../TestRollbackToSnapshotProcedure.java | 24 ++-- .../TestRollbackToTimestampProcedure.java | 20 ++- .../TestSetCurrentSnapshotProcedure.java | 29 ++--- .../TestSetWriteDistributionAndOrdering.java | 4 +- .../TestSnapshotTableProcedure.java | 13 +- ...ePartitionedJoinsInRowLevelOperations.java | 10 +- .../TestSystemFunctionPushDownDQL.java | 28 ++--- .../iceberg/spark/extensions/TestTagDDL.java | 28 ++--- .../iceberg/spark/extensions/TestUpdate.java | 42 +++---- .../spark/extensions/TestWriteAborts.java | 5 +- .../org/apache/iceberg/spark/SmokeTest.java | 5 +- .../iceberg/TestDataFileSerialization.java | 6 +- .../TestManifestFileSerialization.java | 6 +- .../iceberg/TestScanTaskSerialization.java | 14 +-- .../org/apache/iceberg/ValidationHelpers.java | 5 +- .../iceberg/spark/TestFunctionCatalog.java | 18 +-- .../apache/iceberg/spark/TestSpark3Util.java | 19 ++- .../iceberg/spark/TestSparkTableUtil.java | 7 +- .../iceberg/spark/TestSparkV2Filters.java | 32 ++--- .../TestDeleteReachableFilesAction.java | 7 +- .../actions/TestExpireSnapshotsAction.java | 7 +- .../actions/TestRemoveOrphanFilesAction.java | 19 +-- .../actions/TestRewriteDataFilesAction.java | 28 ++--- .../actions/TestRewriteManifestsAction.java | 10 +- .../spark/actions/TestSparkFileRewriter.java | 47 ++++---- .../iceberg/spark/data/GenericsHelpers.java | 104 ++++++---------- .../iceberg/spark/data/TestHelpers.java | 114 +++++++----------- .../spark/data/TestSparkParquetReader.java | 4 +- .../TestParquetVectorizedReads.java | 7 +- .../spark/functions/TestSparkFunctions.java | 9 +- .../spark/source/TestCompressionSettings.java | 8 +- .../spark/source/TestDataFrameWriterV2.java | 8 +- .../spark/source/TestDataFrameWrites.java | 5 +- .../spark/source/TestDataSourceOptions.java | 13 +- .../spark/source/TestFilteredScan.java | 4 +- .../source/TestForwardCompatibility.java | 6 +- .../source/TestIcebergSourceTablesBase.java | 9 +- .../spark/source/TestIcebergSpark.java | 9 +- ...tMetadataTablesWithPartitionEvolution.java | 4 +- .../spark/source/TestPathIdentifier.java | 7 +- .../spark/source/TestReadProjection.java | 5 +- .../TestRequiredDistributionAndOrdering.java | 5 +- .../spark/source/TestSnapshotSelection.java | 23 ++-- .../TestSparkCatalogCacheExpiration.java | 24 ++-- .../spark/source/TestSparkDataWrite.java | 7 +- .../source/TestSparkMetadataColumns.java | 6 +- .../spark/source/TestSparkReadMetrics.java | 98 +++++++-------- .../iceberg/spark/source/TestSparkScan.java | 92 +++++++------- .../spark/source/TestStructuredStreaming.java | 4 +- .../source/TestStructuredStreamingRead3.java | 38 +++--- .../spark/source/TestWriteMetricsConfig.java | 5 +- .../spark/sql/TestAggregatePushDown.java | 17 +-- .../iceberg/spark/sql/TestCreateTable.java | 21 ++-- .../spark/sql/TestCreateTableAsSelect.java | 4 +- .../iceberg/spark/sql/TestDeleteFrom.java | 6 +- .../iceberg/spark/sql/TestDropTable.java | 5 +- .../iceberg/spark/sql/TestFilterPushDown.java | 10 +- .../iceberg/spark/sql/TestNamespaceSQL.java | 5 +- .../sql/TestPartitionedWritesToWapBranch.java | 8 +- .../apache/iceberg/spark/sql/TestSelect.java | 21 ++-- .../spark/sql/TestSparkBucketFunction.java | 61 +++++----- .../spark/sql/TestSparkDaysFunction.java | 11 +- .../spark/sql/TestSparkHoursFunction.java | 11 +- .../spark/sql/TestSparkMonthsFunction.java | 16 +-- .../spark/sql/TestSparkTruncateFunction.java | 53 ++++---- .../spark/sql/TestSparkYearsFunction.java | 16 +-- .../spark/sql/TestTimestampWithoutZone.java | 13 +- .../sql/TestUnpartitionedWritesToBranch.java | 5 +- .../sql/UnpartitionedWritesTestBase.java | 7 +- .../extensions/TestAddFilesProcedure.java | 14 +-- .../extensions/TestAlterTableSchema.java | 16 +-- .../extensions/TestAncestorsOfProcedure.java | 10 +- .../spark/extensions/TestBranchDDL.java | 25 ++-- .../extensions/TestCallStatementParser.java | 4 +- .../spark/extensions/TestChangelogTable.java | 5 +- .../TestCherrypickSnapshotProcedure.java | 17 ++- .../extensions/TestConflictValidation.java | 21 ++-- .../extensions/TestCopyOnWriteDelete.java | 4 +- .../extensions/TestCopyOnWriteMerge.java | 4 +- .../extensions/TestCopyOnWriteUpdate.java | 4 +- .../iceberg/spark/extensions/TestDelete.java | 13 +- .../TestExpireSnapshotsProcedure.java | 26 ++-- .../iceberg/spark/extensions/TestMerge.java | 94 +++++++-------- .../extensions/TestMergeOnReadDelete.java | 5 +- .../extensions/TestMigrateTableProcedure.java | 11 +- .../TestPublishChangesProcedure.java | 13 +- .../TestRemoveOrphanFilesProcedure.java | 30 +++-- .../spark/extensions/TestReplaceBranch.java | 6 +- .../TestRequiredDistributionAndOrdering.java | 5 +- .../TestRewriteDataFilesProcedure.java | 35 +++--- .../TestRewriteManifestsProcedure.java | 16 ++- .../TestRollbackToSnapshotProcedure.java | 23 ++-- .../TestRollbackToTimestampProcedure.java | 19 ++- .../TestSetCurrentSnapshotProcedure.java | 29 ++--- .../TestSetWriteDistributionAndOrdering.java | 4 +- .../TestSnapshotTableProcedure.java | 12 +- .../iceberg/spark/extensions/TestTagDDL.java | 28 ++--- .../iceberg/spark/extensions/TestUpdate.java | 45 ++++--- .../spark/extensions/TestWriteAborts.java | 5 +- .../org/apache/iceberg/spark/SmokeTest.java | 37 +++--- .../iceberg/spark/SparkTestHelperBase.java | 17 ++- .../actions/TestRewriteDataFilesAction.java | 3 +- .../spark/actions/TestSparkFileRewriter.java | 74 ++++++------ .../spark/functions/TestSparkFunctions.java | 9 +- .../spark/sql/TestAggregatePushDown.java | 15 +-- .../iceberg/spark/sql/TestCreateTable.java | 19 ++- .../iceberg/spark/sql/TestDeleteFrom.java | 5 +- .../iceberg/spark/sql/TestDropTable.java | 4 +- .../iceberg/spark/sql/TestNamespaceSQL.java | 4 +- .../sql/TestPartitionedWritesToWapBranch.java | 8 +- .../spark/sql/TestSparkBucketFunction.java | 59 ++++----- .../sql/UnpartitionedWritesTestBase.java | 6 +- 223 files changed, 1968 insertions(+), 2238 deletions(-) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index aab2ae65d010..b17ffddf8287 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.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.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -47,7 +48,6 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -688,7 +688,7 @@ public void invalidDataImport() { createIcebergTable("id Integer, name String, dept String, subdept String"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", @@ -697,7 +697,7 @@ public void invalidDataImport() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot use partition filter with an unpartitioned table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`')", @@ -714,7 +714,7 @@ public void invalidDataImportPartitioned() { createIcebergTable( "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", @@ -723,7 +723,7 @@ public void invalidDataImportPartitioned() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("is greater than the number of partitioned columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", @@ -783,7 +783,7 @@ public void duplicateDataPartitioned() { + "partition_filter => map('id', 1))", catalogName, tableName, sourceTableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files(" @@ -843,7 +843,7 @@ public void duplicateDataUnpartitioned() { sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '%s')", diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index 948fc462de99..2c109f1007d1 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -27,7 +29,6 @@ import org.apache.spark.sql.connector.catalog.CatalogManager; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -185,9 +186,7 @@ public void testAddYearPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD year(ts)", tableName); @@ -196,7 +195,7 @@ public void testAddYearPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).year("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test @@ -204,9 +203,7 @@ public void testAddMonthPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD month(ts)", tableName); @@ -215,7 +212,7 @@ public void testAddMonthPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).month("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test @@ -223,9 +220,7 @@ public void testAddDayPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD day(ts)", tableName); @@ -234,7 +229,7 @@ public void testAddDayPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test @@ -242,9 +237,7 @@ public void testAddHourPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD hour(ts)", tableName); @@ -253,7 +246,7 @@ public void testAddHourPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).hour("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java index acbf5c7a632d..da1b652e8a19 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -77,13 +78,12 @@ public void testSetInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) .as("should not allow setting unknown fields") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("not found in current schema or added columns"); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) .as("should not allow setting optional fields") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("not a required field"); @@ -139,21 +139,18 @@ public void testDropInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) .as("should not allow dropping unknown fields") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("field unknown not found"); sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) .as("should not allow dropping a field that is not an identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("data is not an identifier field"); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) .as("should not allow dropping a nested field that is not an identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("location.lon is not an identifier field"); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index 13fa96d1e08a..d111cdb01bb8 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -147,18 +148,17 @@ public void testAncestorOfUsingNamedArgs() { @Test public void testInvalidAncestorOfCases() { - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier for parameter 'table'"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); 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 8f3e88f3c811..762ade946583 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 @@ -19,6 +19,7 @@ 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; @@ -33,7 +34,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -85,8 +85,7 @@ public void testCreateBranch() throws NoSuchTableException { Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) .as("Cannot create an existing branch") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref b1 already exists"); @@ -99,20 +98,20 @@ public void testCreateBranchOnEmptyTable() { Table table = validationCatalog.loadTable(tableIdent); SnapshotRef mainRef = table.refs().get(SnapshotRef.MAIN_BRANCH); - Assertions.assertThat(mainRef).isNull(); + assertThat(mainRef).isNull(); SnapshotRef ref = table.refs().get(branchName); - Assertions.assertThat(ref).isNotNull(); - Assertions.assertThat(ref.minSnapshotsToKeep()).isNull(); - Assertions.assertThat(ref.maxSnapshotAgeMs()).isNull(); - Assertions.assertThat(ref.maxRefAgeMs()).isNull(); + assertThat(ref).isNotNull(); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); Snapshot snapshot = table.snapshot(ref.snapshotId()); - Assertions.assertThat(snapshot.parentId()).isNull(); - Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); } @Test @@ -195,7 +194,7 @@ public void testCreateBranchUseCustomMinSnapshotsToKeepAndMaxSnapshotAge() Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertNull(ref.maxRefAgeMs()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", @@ -218,20 +217,19 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { Assert.assertNull(ref.maxSnapshotAgeMs()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", @@ -260,8 +258,7 @@ public void testDropBranch() throws NoSuchTableException { @Test public void testDropBranchDoesNotExist() { - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) .as("Cannot perform drop branch on branch which does not exist") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Branch does not exist: nonExistingBranch"); @@ -273,7 +270,7 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) .as("Cannot perform drop branch on tag") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref b1 is a tag not a branch"); @@ -281,7 +278,7 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { @Test public void testDropBranchNonConformingName() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) .as("Non-conforming branch name") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); @@ -289,7 +286,7 @@ public void testDropBranchNonConformingName() { @Test public void testDropMainBranchFails() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) .as("Cannot drop the main branch") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot remove main branch"); @@ -331,20 +328,20 @@ public void testCreateOrReplaceBranchOnEmptyTable() { Table table = validationCatalog.loadTable(tableIdent); SnapshotRef mainRef = table.refs().get(SnapshotRef.MAIN_BRANCH); - Assertions.assertThat(mainRef).isNull(); + assertThat(mainRef).isNull(); SnapshotRef ref = table.refs().get(branchName); - Assertions.assertThat(ref).isNotNull(); - Assertions.assertThat(ref.minSnapshotsToKeep()).isNull(); - Assertions.assertThat(ref.maxSnapshotAgeMs()).isNull(); - Assertions.assertThat(ref.maxRefAgeMs()).isNull(); + assertThat(ref).isNotNull(); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); Snapshot snapshot = table.snapshot(ref.snapshotId()); - Assertions.assertThat(snapshot.parentId()).isNull(); - Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); } @Test @@ -387,7 +384,7 @@ public void replaceBranch() throws NoSuchTableException { public void replaceBranchDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", 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 c0f7ff1f8f9b..42f29c28c149 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; @@ -37,7 +39,6 @@ import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -144,7 +145,7 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - Assertions.assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) .as("Should fail with a sensible parse error") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("missing '(' at 'radish'"); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java index 3b6ac57cc03e..cf9ee47bbb54 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -34,7 +35,6 @@ import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -202,8 +202,7 @@ public void testTimeRangeValidation() { Snapshot snap3 = table.currentSnapshot(); long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); - Assertions.assertThatThrownBy( - () -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) + assertThatThrownBy(() -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) .as("Should fail if start time is after end time") .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); 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 cbecfcef3ed6..b489cf76f1ef 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -161,7 +161,7 @@ public void testCherrypickSnapshotRefreshesRelationCache() { public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)) .as("Should reject invalid snapshot id") .isInstanceOf(ValidationException.class) @@ -170,31 +170,28 @@ public void testCherrypickInvalidSnapshot() { @Test public void testInvalidCherrypickSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) .as("Should not allow mixed args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) + 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"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java index 7d183167572d..a5e3a4bf33ac 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.IsolationLevel; @@ -30,7 +32,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -69,7 +70,7 @@ public void testOverwriteFilterSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -114,7 +115,7 @@ public void testOverwriteFilterSerializableIsolation2() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -154,7 +155,7 @@ public void testOverwriteFilterSerializableIsolation3() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -192,7 +193,7 @@ public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { // Validating from no snapshot id defaults to beginning snapshot id and finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -236,7 +237,7 @@ public void testOverwriteFilterSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -291,7 +292,7 @@ public void testOverwritePartitionSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -334,7 +335,7 @@ public void testOverwritePartitionSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -375,7 +376,7 @@ public void testOverwritePartitionSnapshotIsolation2() throws Exception { spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf @@ -429,7 +430,7 @@ public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { // Validating from null snapshot is equivalent to validating from beginning Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { conflictingDf diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java index 53177340dadd..d0151ea402b0 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -47,7 +48,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -144,7 +144,7 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(deleteFuture::get) + assertThatThrownBy(deleteFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index ed1e05f822cf..3f56d82d9b70 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -45,7 +46,6 @@ import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -142,7 +142,7 @@ public synchronized void testMergeWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(mergeFuture::get) + assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java index f9f48e8f41c7..a36787477a62 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -44,7 +45,6 @@ import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -140,7 +140,7 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(updateFuture::get) + assertThatThrownBy(updateFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index af66df5a9fd4..beb3b0105987 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -28,6 +28,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; 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.Arrays; @@ -73,7 +74,6 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromIcebergTable; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -240,7 +240,7 @@ public void testDeleteFromNonExistingCustomBranch() { Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); createAndInitUnpartitionedTable(); - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): test"); } @@ -447,8 +447,7 @@ public void testDeleteWithNonDeterministicCondition() { sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); createBranchIfNeeded(); - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) .as("Should complain about non-deterministic expressions") .isInstanceOf(AnalysisException.class) .hasMessageContaining("nondeterministic expressions are only allowed"); @@ -736,7 +735,7 @@ public void testDeleteWithNotInSubquery() throws NoSuchTableException { public void testDeleteOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s WHERE c1 = -100", "testtable")) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE c1 = -100", "testtable")) .as("Delete is supported only for Iceberg tables") .isInstanceOf(AnalysisException.class) .hasMessageContaining("DELETE is only supported with v2 tables."); @@ -942,7 +941,7 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte }); try { - Assertions.assertThatThrownBy(deleteFuture::get) + assertThatThrownBy(deleteFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(SparkException.class) @@ -1184,7 +1183,7 @@ public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableE withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage( String.format( @@ -1210,9 +1209,9 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), () -> { sql("DELETE FROM %s t WHERE id=1", tableName); - Assertions.assertThat(spark.table(tableName).count()).isEqualTo(2L); - Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); - Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + assertThat(spark.table(tableName).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_main").count()) .as("Should not modify main branch") .isEqualTo(3L); }); @@ -1220,9 +1219,9 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), () -> { sql("DELETE FROM %s t", tableName); - Assertions.assertThat(spark.table(tableName).count()).isEqualTo(0L); - Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); - Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + assertThat(spark.table(tableName).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_main").count()) .as("Should not modify main branch") .isEqualTo(3L); }); 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 f677facb9902..e9f42f5e0195 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 @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,7 +53,6 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -162,8 +163,7 @@ public void testExpireSnapshotsGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) .as("Should reject call") .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot expire snapshots: GC is disabled"); @@ -171,30 +171,27 @@ public void testExpireSnapshotsGCDisabled() { @Test public void testInvalidExpireSnapshotsCases() { - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) .as("Should not allow mixed args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") .isInstanceOf(NoSuchProcedureException.class) .hasMessageContaining("not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); @@ -215,7 +212,7 @@ public void testResolvingTableInAnotherCatalog() throws IOException { "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", anotherCatalog, tableIdent); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots('%s')", @@ -252,7 +249,7 @@ public void testConcurrentExpireSnapshots() { public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", @@ -261,7 +258,7 @@ public void testConcurrentExpireSnapshotsWithInvalidInput() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("max_concurrent_deletes should have value > 0"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", @@ -401,7 +398,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(" @@ -481,21 +478,21 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.statisticsFiles()) + assertThat(table.statisticsFiles()) .as( "Statistics file entry in TableMetadata should be present only for the snapshot %s", statisticsFile2.snapshotId()) .extracting(StatisticsFile::snapshotId) .containsExactly(statisticsFile2.snapshotId()); - Assertions.assertThat(new File(statsFileLocation1)) + assertThat(new File(statsFileLocation1)) .as("Statistics file should not exist for snapshot %s", statisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(statsFileLocation2)) + assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); } @@ -526,25 +523,23 @@ public void testExpireSnapshotsWithPartitionStatisticFiles() { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]) - .as("should be 1 deleted partition statistics file") - .isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted partition statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.partitionStatisticsFiles()) + assertThat(table.partitionStatisticsFiles()) .as( "partition statistics file entry in TableMetadata should be present only for the snapshot %s", partitionStatisticsFile2.snapshotId()) .extracting(PartitionStatisticsFile::snapshotId) .containsExactly(partitionStatisticsFile2.snapshotId()); - Assertions.assertThat(new File(partitionStatsFileLocation1)) + assertThat(new File(partitionStatsFileLocation1)) .as( "partition statistics file should not exist for snapshot %s", partitionStatisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(partitionStatsFileLocation2)) + assertThat(new File(partitionStatsFileLocation2)) .as( "partition statistics file should exist for snapshot %s", partitionStatisticsFile2.snapshotId()) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index ead477d71b20..39f316655e4a 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -26,6 +26,8 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.Collections; @@ -63,7 +65,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -583,7 +584,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { ds.union(ds).createOrReplaceTempView("source"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -626,7 +627,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { () -> { String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -667,7 +668,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqua () -> { String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -706,7 +707,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM ds.union(ds).createOrReplaceTempView("source"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -741,7 +742,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM ds.union(ds).createOrReplaceTempView("source"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -778,7 +779,7 @@ public void testMergeWithMultipleUpdatesForTargetRow() { + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -850,7 +851,7 @@ public void testMergeWithSingleConditionalDelete() { + "{ \"id\": 6, \"dep\": \"emp-id-6\" }"); String errorMsg = "a single row from the target table with multiple rows of the source table"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s AS t USING source AS s " @@ -1255,7 +1256,7 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted }); try { - Assertions.assertThatThrownBy(mergeFuture::get) + assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(SparkException.class) @@ -1991,7 +1992,7 @@ public void testMergeWithNonExistingColumns() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2004,7 +2005,7 @@ public void testMergeWithNonExistingColumns() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("cannot resolve t.invalid_col"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2017,7 +2018,7 @@ public void testMergeWithNonExistingColumns() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("No such struct field invalid_col"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2040,7 +2041,7 @@ public void testMergeWithInvalidColumnsInInsert() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2055,7 +2056,7 @@ public void testMergeWithInvalidColumnsInInsert() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Nested fields are not supported inside INSERT clauses"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2070,7 +2071,7 @@ public void testMergeWithInvalidColumnsInInsert() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Duplicate column names inside INSERT clause"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2091,7 +2092,7 @@ public void testMergeWithInvalidUpdates() { "{ \"id\": 1, \"a\": [ { \"c1\": 2, \"c2\": 3 } ], \"m\": { \"k\": \"v\"} }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2104,7 +2105,7 @@ public void testMergeWithInvalidUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for structs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2125,7 +2126,7 @@ public void testMergeWithConflictingUpdates() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2138,7 +2139,7 @@ public void testMergeWithConflictingUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updates are in conflict"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2151,7 +2152,7 @@ public void testMergeWithConflictingUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updates are in conflict for these columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2179,7 +2180,7 @@ public void testMergeWithInvalidAssignments() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2192,7 +2193,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2205,7 +2206,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2218,7 +2219,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("missing fields"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2231,7 +2232,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2254,7 +2255,7 @@ public void testMergeWithNonDeterministicConditions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2267,7 +2268,7 @@ public void testMergeWithNonDeterministicConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Non-deterministic functions are not supported"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2280,7 +2281,7 @@ public void testMergeWithNonDeterministicConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Non-deterministic functions are not supported"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2293,7 +2294,7 @@ public void testMergeWithNonDeterministicConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Non-deterministic functions are not supported"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2314,7 +2315,7 @@ public void testMergeWithAggregateExpressions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2327,7 +2328,7 @@ public void testMergeWithAggregateExpressions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Agg functions are not supported"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2340,7 +2341,7 @@ public void testMergeWithAggregateExpressions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Agg functions are not supported"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2353,7 +2354,7 @@ public void testMergeWithAggregateExpressions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Agg functions are not supported"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2374,7 +2375,7 @@ public void testMergeWithSubqueriesInConditions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2387,7 +2388,7 @@ public void testMergeWithSubqueriesInConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Subqueries are not supported in conditions"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2400,7 +2401,7 @@ public void testMergeWithSubqueriesInConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Subqueries are not supported in conditions"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2413,7 +2414,7 @@ public void testMergeWithSubqueriesInConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Subqueries are not supported in conditions"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2432,7 +2433,7 @@ public void testMergeWithTargetColumnsInInsertConditions() { createAndInitTable("id INT, c2 INT", "{ \"id\": 1, \"c2\": 2 }"); createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO %s t USING source s " @@ -2451,7 +2452,7 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "MERGE INTO target t USING source s " @@ -2517,7 +2518,7 @@ public void testMergeNonExistingBranch() { // Coalesce forces our source into a SinglePartition distribution spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s ON t.id = s.id " @@ -2604,7 +2605,7 @@ public void testMergeToWapBranchWithTableBranchIdentifier() { withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s ON t.id = s.id " @@ -2626,9 +2627,9 @@ private void checkJoinAndFilterConditions(String query, String join, String iceb SparkPlan sparkPlan = executeAndKeepPlan(() -> sql(query)); String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); - Assertions.assertThat(planAsString).as("Join should match").contains(join + "\n"); + assertThat(planAsString).as("Join should match").contains(join + "\n"); - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Pushed filters must match") .contains("[filters=" + icebergFilters + ","); }); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index c156e4c77767..9010b7890fe8 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -37,7 +38,6 @@ import org.apache.iceberg.spark.source.TestSparkCatalog; import org.apache.spark.SparkException; import org.apache.spark.sql.connector.catalog.Identifier; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runners.Parameterized; @@ -115,8 +115,7 @@ public void testCommitUnknownException() { TestSparkCatalog.setTable(ident, sparkTable); // Although an exception is thrown here, write and commit have succeeded - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) .as("Should throw a Commit State Unknown Exception") .isInstanceOf(SparkException.class) .hasMessageContaining("Writing job aborted") diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java index e9013848cf11..17ed003f6a67 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.UUID; @@ -34,7 +36,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; import org.junit.runners.Parameterized; @@ -103,7 +104,7 @@ public void testReadStageTableMeta() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .load(tableLocation); - Assertions.assertThat(scanDF2.columns().length).isEqualTo(2); + assertThat(scanDF2.columns().length).isEqualTo(2); } try (CloseableIterable tasks = table.newBatchScan().planFiles()) { diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 3a51e87be804..347a81f0d9f3 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -131,7 +132,7 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { String props = "map('write.metadata.metrics.column.x', 'X')"; sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); @@ -166,18 +167,17 @@ public void testMigrateWithConflictingProps() throws IOException { @Test public void testInvalidMigrateCases() { - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); 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 19a065234ed8..e71ad687293b 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -157,7 +157,7 @@ public void testApplyWapChangesRefreshesRelationCache() { public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)) .as("Should reject invalid wap id") .isInstanceOf(ValidationException.class) @@ -166,26 +166,25 @@ public void testApplyInvalidWapId() { @Test public void testInvalidApplyWapChangesCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)) .as("Should not allow mixed args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( + 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"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); 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 415141968664..1701faf37892 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 @@ -20,6 +20,8 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -64,7 +66,6 @@ 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.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -226,7 +227,7 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)) .as("Should reject call") .isInstanceOf(ValidationException.class) @@ -257,30 +258,28 @@ public void testRemoveOrphanFilesWap() { @Test public void testInvalidRemoveOrphanFilesCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) .as("Should not allow mixed args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) + 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"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); @@ -348,7 +347,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", @@ -357,7 +356,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("max_concurrent_deletes should have value > 0"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", @@ -369,7 +368,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { String tempViewName = "file_list_test"; spark.emptyDataFrame().createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -383,7 +382,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -397,7 +396,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -515,10 +514,10 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) + assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); + assertThat(statsLocation.length()) .as("stats file length") .isEqualTo(statisticsFile.fileSizeInBytes()); @@ -532,11 +531,11 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly(statsLocation.toURI().toString()); - Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } @Test @@ -565,11 +564,9 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(new File(partitionStatsLocation)) - .as("partition stats file should exist") - .exists(); + assertThat(new File(partitionStatsLocation)).as("partition stats file should exist").exists(); removePartitionStatsTxn(table, partitionStatisticsFile); @@ -579,11 +576,11 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly("file:" + partitionStatsLocation); - Assertions.assertThat(new File(partitionStatsLocation)) + assertThat(new File(partitionStatsLocation)) .as("partition stats file should be deleted") .doesNotExist(); } @@ -676,7 +673,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() Assert.assertEquals(0, orphanFiles.size()); // Test with no equal schemes - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(" @@ -760,7 +757,7 @@ public void testRemoveOrphanFilesProcedureWithEqualAuthorities() Assert.assertEquals(0, orphanFiles.size()); // Test with no equal authorities - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(" diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java index ed428310b8f4..7cab771c0077 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -74,7 +75,7 @@ public void testReplaceBranchFailsForTag() throws NoSuchTableException { df.writeTo(tableName).append(); long second = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", @@ -129,7 +130,7 @@ public void testReplaceBranchDoesNotExist() throws NoSuchTableException { df.writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java index b459f3c24807..29fb193fe425 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.util.List; import java.util.Map; @@ -28,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -187,7 +188,7 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { inputDF 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 4a4e4d7dfd7b..1418d82cebf8 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 @@ -19,6 +19,7 @@ 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.Arrays; import java.util.List; @@ -40,7 +41,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -454,7 +454,7 @@ public void testRewriteDataFilesWithInvalidInputs() { insertData(2); // Test for invalid strategy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " @@ -465,7 +465,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageContaining("unsupported strategy: temp. Only binpack or sort is supported"); // Test for sort_order with binpack strategy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " @@ -476,7 +476,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageContaining("Must use only one rewriter type (bin-pack, sort, zorder)"); // Test for sort strategy without any (default/user defined) sort_order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", @@ -486,7 +486,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageContaining("Cannot sort data without a valid sort order"); // Test for sort_order with invalid null order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -497,7 +497,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageContaining("Unable to parse sortOrder:"); // Test for sort_order with invalid sort direction - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -508,7 +508,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageContaining("Unable to parse sortOrder:"); // Test for sort_order with invalid column name - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -521,7 +521,7 @@ public void testRewriteDataFilesWithInvalidInputs() { + " struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test with invalid filter column col1 - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", @@ -531,7 +531,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageContaining("Cannot parse predicates in where option: col1 = 3"); // Test for z_order with invalid column name - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -544,7 +544,7 @@ public void testRewriteDataFilesWithInvalidInputs() { + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test for z_order with sort_order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -558,30 +558,29 @@ public void testRewriteDataFilesWithInvalidInputs() { @Test public void testInvalidCasesForRewriteDataFiles() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) .as("Should not allow mixed args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) + 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"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)) .as("Should reject duplicate arg names name") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Duplicate procedure argument: table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); @@ -738,7 +737,7 @@ public void testDefaultSortOrder() { @Test public void testRewriteWithUntranslatedOrUnconvertedFilter() { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', where => 'lower(c2) = \"fo\"')", @@ -746,7 +745,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', where => 'c2 like \"%%fo\"')", 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 3afad8a7b839..591d88e83fca 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.sql.Date; import java.sql.Timestamp; @@ -31,7 +32,6 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -276,36 +276,34 @@ public void testRewriteManifestsCaseInsensitiveArgs() { @Test public void testInvalidRewriteManifestsCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) .as("Should not allow mixed args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) .as("Should not resolve procedures in arbitrary namespaces") .isInstanceOf(NoSuchProcedureException.class) .hasMessageContaining("not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) .as("Should reject duplicate arg names name") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Duplicate procedure argument: table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index 481e2f01f23b..6f39031d146f 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP; import static org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -29,7 +30,6 @@ import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Encoders; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -206,7 +206,7 @@ public void testInvalidOption() throws Exception { @Test public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'lower(data) = \"fo\"')", @@ -214,7 +214,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'data like \"%%fo\"')", 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 95d03a0104fd..b60a769f9f26 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Snapshot; @@ -28,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -240,7 +241,7 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)) .as("Should reject invalid snapshot id") .isInstanceOf(ValidationException.class) @@ -249,7 +250,7 @@ public void testRollbackToInvalidSnapshot() { @Test public void testInvalidRollbackToSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", @@ -258,37 +259,32 @@ public void testInvalidRollbackToSnapshotCases() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) + 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"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); 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 c11c7ab7e421..281bfbf7ad1b 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.time.LocalDateTime; import java.util.List; import java.util.Map; @@ -28,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -255,7 +256,7 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", @@ -264,19 +265,18 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( + 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"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp(timestamp => %s)", @@ -285,13 +285,12 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", @@ -300,8 +299,7 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Too many arguments"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type for timestamp: cannot cast"); 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 d996e4b5c805..55bda0df1b3c 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -30,7 +31,6 @@ 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.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -193,7 +193,7 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { Namespace namespace = tableIdent.namespace(); String tableName = tableIdent.name(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)) .as("Should reject invalid snapshot id") .isInstanceOf(ValidationException.class) @@ -202,7 +202,7 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { @Test public void testInvalidRollbackToSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", @@ -211,43 +211,38 @@ public void testInvalidRollbackToSnapshotCases() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) + 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"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type for snapshot_id: cannot cast"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java index d39e08beab0b..8c096ab32d6e 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Map; import org.apache.iceberg.NullOrder; @@ -27,7 +29,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -75,14 +76,14 @@ public void testSetWriteOrderWithCaseSensitiveColumnNames() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue("Table should start unsorted", table.sortOrder().isUnsorted()); sql("SET %s=true", SQLConf.CASE_SENSITIVE().key()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); }) .isInstanceOf(ValidationException.class); sql("SET %s=false", SQLConf.CASE_SENSITIVE().key()); - Assertions.assertThatNoException() + assertThatNoException() .isThrownBy( () -> { sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index d44f8d3721ca..cf297e7fc09c 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -195,18 +196,18 @@ public void testInvalidSnapshotsCases() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) .as("Should reject calls without all required args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Missing required parameters"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)) .as("Should reject calls with invalid arg types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Wrong arg type"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", @@ -215,12 +216,12 @@ public void testInvalidSnapshotsCases() throws IOException { .isInstanceOf(AnalysisException.class) .hasMessageContaining("cannot resolve 'map"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) .as("Should reject calls with empty table identifier") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot handle an empty identifier"); 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 3dadadd9c4de..6c1418b7a796 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 @@ -19,6 +19,7 @@ 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.Locale; @@ -34,7 +35,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -96,7 +96,7 @@ public void testCreateTagWithRetain() throws NoSuchTableException { } String tagName = "t1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", @@ -105,13 +105,13 @@ public void testCreateTagWithRetain() throws NoSuchTableException { .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) .as("Illegal statement") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d SECONDS", @@ -123,7 +123,7 @@ public void testCreateTagWithRetain() throws NoSuchTableException { @Test public void testCreateTagOnEmptyTable() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Cannot complete create or replace tag operation on %s, main has no snapshot", @@ -136,7 +136,7 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { long snapshotId = table.currentSnapshot().snapshotId(); String tagName = "t1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, -1)) .as("unknown snapshot") .isInstanceOf(ValidationException.class) @@ -150,12 +150,12 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { Assert.assertNull( "The tag needs to have the default max ref age, which is null.", ref.maxRefAgeMs()); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) .as("Cannot create an exist tag") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("already exists"); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) .as("Non-conforming tag name") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); @@ -204,8 +204,7 @@ public void testReplaceTagFailsForBranch() throws NoSuchTableException { insertRows(); long second = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) + assertThatThrownBy(() -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) .as("Cannot perform replace tag on branches") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref branch1 is a branch not a tag"); @@ -241,7 +240,7 @@ public void testReplaceTag() throws NoSuchTableException { public void testReplaceTagDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", @@ -313,7 +312,7 @@ public void testDropTag() throws NoSuchTableException { @Test public void testDropTagNonConformingName() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) .as("Non-conforming tag name") .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); @@ -321,8 +320,7 @@ public void testDropTagNonConformingName() { @Test public void testDropTagDoesNotExist() { - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) .as("Cannot perform drop tag on tag which does not exist") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Tag does not exist: nonExistingTag"); @@ -334,7 +332,7 @@ public void testDropTagFailesForBranch() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) .as("Cannot perform drop tag on branch") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref b1 is a branch not a tag"); 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 76c7a171f603..63af42ff4c4f 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 @@ -29,6 +29,7 @@ import static org.apache.iceberg.TableProperties.UPDATE_MODE; import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.List; @@ -65,7 +66,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -140,8 +140,7 @@ public void testUpdateNonExistingCustomBranch() { Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); createAndInitTable("id INT, dep STRING"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): test"); } @@ -527,7 +526,7 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte }); try { - Assertions.assertThatThrownBy(updateFuture::get) + assertThatThrownBy(updateFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(SparkException.class) @@ -1148,12 +1147,12 @@ public void testUpdateWithInvalidUpdates() { "id INT, a ARRAY>, m MAP", "{ \"id\": 0, \"a\": null, \"m\": null }"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) .as("Should complain about updating an array column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for structs"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) .as("Should complain about updating a map column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for structs"); @@ -1164,19 +1163,18 @@ public void testUpdateWithConflictingAssignments() { createAndInitTable( "id INT, c STRUCT>", "{ \"id\": 0, \"s\": null }"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) .as("Should complain about conflicting updates to a top-level column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updates are in conflict"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", commitTarget())) .as("Should complain about conflicting updates to a nested column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updates are in conflict for these columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "UPDATE %s SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", @@ -1197,30 +1195,28 @@ public void testUpdateWithInvalidAssignments() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), () -> { - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) .as("Should complain about writing nulls to a top-level column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) .as("Should complain about writing nulls to a nested column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) .as("Should complain about writing missing fields in structs") .isInstanceOf(AnalysisException.class) .hasMessageContaining("missing fields"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) .as("Should complain about writing invalid data types") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "UPDATE %s t SET t.s.n2 = named_struct('dn2', 1, 'dn1', 2)", @@ -1236,7 +1232,7 @@ public void testUpdateWithInvalidAssignments() { public void testUpdateWithNonDeterministicCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s SET id = -1 WHERE id = 1 AND rand() > 0.5", commitTarget())) .as("Should complain about non-deterministic expressions") .isInstanceOf(AnalysisException.class) @@ -1247,7 +1243,7 @@ public void testUpdateWithNonDeterministicCondition() { public void testUpdateOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) + assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) .as("UPDATE is not supported for non iceberg table") .isInstanceOf(UnsupportedOperationException.class) .hasMessageContaining("not supported temporarily"); @@ -1312,8 +1308,7 @@ public void testUpdateToWapBranchWithTableBranchIdentifier() { withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage( String.format( diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 2966e95fa3de..3db2265cf9f2 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -38,7 +40,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Rule; import org.junit.Test; @@ -105,7 +106,7 @@ public void testBatchAppend() throws Exception { new SimpleRecord(4, "b")); Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { // incoming records are not ordered by partitions so the job must fail diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java index 33b5316b72b7..2a7d3d4c331f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TaskCheckHelper.assertEquals; 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 com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; @@ -48,7 +49,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.catalyst.InternalRow; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -118,7 +118,7 @@ public void testDataFileKryoSerialization() throws Exception { try (Input in = new Input(new FileInputStream(data))) { for (int i = 0; i < 2; i += 1) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); assertEquals(DATA_FILE, (DataFile) obj); } } @@ -136,7 +136,7 @@ public void testDataFileJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); assertEquals(DATA_FILE, (DataFile) obj); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java index 92a646d3861b..a42781f95282 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java @@ -20,6 +20,7 @@ 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 com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; @@ -43,7 +44,6 @@ import org.apache.iceberg.types.Types; import org.apache.spark.SparkConf; import org.apache.spark.serializer.KryoSerializer; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -119,7 +119,7 @@ public void testManifestFileKryoSerialization() throws IOException { try (Input in = new Input(new FileInputStream(data))) { for (int i = 0; i < 3; i += 1) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); checkManifestFile(manifest, (ManifestFile) obj); } } @@ -141,7 +141,7 @@ public void testManifestFileJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 3; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); checkManifestFile(manifest, (ManifestFile) obj); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java index 14e9e7f605b5..f37d58c66412 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; @@ -46,7 +47,6 @@ import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -86,7 +86,7 @@ public void testBaseCombinedScanTaskKryoSerialization() throws Exception { try (Input in = new Input(new FileInputStream(data))) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj) + assertThat(obj) .as("Should be a BaseCombinedScanTask") .isInstanceOf(BaseCombinedScanTask.class); TaskCheckHelper.assertEquals(scanTask, (BaseCombinedScanTask) obj); @@ -105,7 +105,7 @@ public void testBaseCombinedScanTaskJavaSerialization() throws Exception { try (ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { Object obj = in.readObject(); - Assertions.assertThat(obj) + assertThat(obj) .as("Should be a BaseCombinedScanTask") .isInstanceOf(BaseCombinedScanTask.class); TaskCheckHelper.assertEquals(scanTask, (BaseCombinedScanTask) obj); @@ -129,9 +129,7 @@ public void testBaseScanTaskGroupKryoSerialization() throws Exception { try (Input in = new Input(Files.newInputStream(data.toPath()))) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj) - .as("should be a BaseScanTaskGroup") - .isInstanceOf(BaseScanTaskGroup.class); + assertThat(obj).as("should be a BaseScanTaskGroup").isInstanceOf(BaseScanTaskGroup.class); TaskCheckHelper.assertEquals(taskGroup, (BaseScanTaskGroup) obj); } } @@ -151,9 +149,7 @@ public void testBaseScanTaskGroupJavaSerialization() throws Exception { try (ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("should be a BaseScanTaskGroup") - .isInstanceOf(BaseScanTaskGroup.class); + assertThat(obj).as("should be a BaseScanTaskGroup").isInstanceOf(BaseScanTaskGroup.class); TaskCheckHelper.assertEquals(taskGroup, (BaseScanTaskGroup) obj); } } 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 70ab04f0a080..b0b3085dca70 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 @@ -18,11 +18,12 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; public class ValidationHelpers { @@ -72,6 +73,6 @@ public static void validateDataManifest( private static void assertSameElements(String context, List actual, List expected) { String errorMessage = String.format("%s must match", context); - Assertions.assertThat(actual).as(errorMessage).hasSameElementsAs(expected); + assertThat(actual).as(errorMessage).hasSameElementsAs(expected); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java index d224e175d3b0..cacb85147f4c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.IcebergBuild; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.functions.IcebergVersionFunction; @@ -27,7 +30,6 @@ import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -56,10 +58,10 @@ public void dropDefaultNamespace() { @Test public void testListFunctionsViaCatalog() throws NoSuchNamespaceException { - Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE)) + assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE)) .anyMatch(func -> "iceberg_version".equals(func.name())); - Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE)) + assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE)) .anyMatch(func -> "iceberg_version".equals(func.name())); Assert.assertArrayEquals( @@ -67,7 +69,7 @@ public void testListFunctionsViaCatalog() throws NoSuchNamespaceException { new Identifier[0], asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE)); - Assertions.assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE)) + assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE)) .as("Listing functions in a namespace that does not exist should throw") .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Namespace 'db' not found"); @@ -79,13 +81,13 @@ public void testLoadFunctions() throws NoSuchFunctionException { Identifier identifier = Identifier.of(namespace, "iceberg_version"); UnboundFunction func = asFunctionCatalog.loadFunction(identifier); - Assertions.assertThat(func) + assertThat(func) .isNotNull() .isInstanceOf(UnboundFunction.class) .isExactlyInstanceOf(IcebergVersionFunction.class); } - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))) .as( @@ -94,12 +96,12 @@ public void testLoadFunctions() throws NoSuchFunctionException { .hasMessageContaining("Undefined function: default.iceberg_version"); Identifier undefinedFunction = Identifier.of(SYSTEM_NAMESPACE, "undefined_function"); - Assertions.assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) + assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) .as("Cannot load a function that does not exist") .isInstanceOf(NoSuchFunctionException.class) .hasMessageContaining("Undefined function: system.undefined_function"); - Assertions.assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) + assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) .as("Using an undefined function from SQL should fail analysis") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Undefined function"); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java index 1e51caadd0e9..51c8d9643c11 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.KryoHelpers; @@ -27,7 +29,6 @@ import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -40,7 +41,7 @@ public void testSparkPartitionOKryoSerialization() throws IOException { SparkPartition sparkPartition = new SparkPartition(values, uri, format); SparkPartition deserialized = KryoHelpers.roundTripSerialize(sparkPartition); - Assertions.assertThat(sparkPartition).isEqualTo(deserialized); + assertThat(sparkPartition).isEqualTo(deserialized); } @Test @@ -51,7 +52,7 @@ public void testSparkPartitionJavaSerialization() throws IOException, ClassNotFo SparkPartition sparkPartition = new SparkPartition(values, uri, format); SparkPartition deserialized = TestHelpers.roundTripSerialize(sparkPartition); - Assertions.assertThat(sparkPartition).isEqualTo(deserialized); + assertThat(sparkPartition).isEqualTo(deserialized); } @Test 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 5b1cf8281784..6d1bbc2c18f1 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.util.Set; @@ -50,7 +51,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -362,7 +362,7 @@ public void testIgnoreMetadataFilesNotFound() { public void testEmptyIOThrowsException() { DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(null); - Assertions.assertThatThrownBy(baseRemoveFilesSparkAction::execute) + assertThatThrownBy(baseRemoveFilesSparkAction::execute) .as("FileIO can't be null in DeleteReachableFiles action") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("File IO cannot be null"); @@ -372,8 +372,7 @@ public void testEmptyIOThrowsException() { public void testRemoveFilesActionWhenGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - Assertions.assertThatThrownBy( - () -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) + assertThatThrownBy(() -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) .as("Should complain about removing files when GC is disabled") .isInstanceOf(ValidationException.class) .hasMessageContaining( 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 a3580bd2d980..71339b32421a 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -56,7 +57,6 @@ import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -447,7 +447,7 @@ public void testExpireSnapshotsWithDisabledGarbageCollection() { table.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) .as("Should complain about expiring snapshots") .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot expire snapshots: GC is disabled"); @@ -528,8 +528,7 @@ public void testRetainLastMultipleCalls() { @Test public void testRetainZeroSnapshots() { - Assertions.assertThatThrownBy( - () -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) .as( "Should fail retain 0 snapshots " + "because number of snapshots to retain cannot be zero") diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index 10af2454fa4a..07572864492c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.actions; 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 java.io.File; import java.io.IOException; @@ -78,7 +80,6 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -754,7 +755,7 @@ public void testGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - Assertions.assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) + assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) .as("Should complain about removing orphan files") .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot delete orphan files: GC is disabled"); @@ -942,8 +943,8 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) + assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); + assertThat(statsLocation.length()) .as("stats file length") .isEqualTo(statisticsFile.fileSizeInBytes()); @@ -957,11 +958,11 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .olderThan(System.currentTimeMillis() + 1000) .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); - Assertions.assertThat(orphanFileLocations).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(orphanFileLocations)) + assertThat(orphanFileLocations).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(orphanFileLocations)) .as("Deleted file") .isEqualTo(statsLocation.toURI().toString()); - Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } @Test @@ -989,7 +990,7 @@ public void testPathsWithActualFileHavingNoAuthority() { public void testPathsWithEqualSchemes() { List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> executeTest( validFiles, @@ -1018,7 +1019,7 @@ public void testPathsWithEqualSchemes() { public void testPathsWithEqualAuthorities() { List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> executeTest( validFiles, 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 5e8865dff2da..2934494bbab7 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 @@ -25,6 +25,7 @@ import static org.apache.spark.sql.functions.date_add; 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.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doAnswer; @@ -105,7 +106,6 @@ import org.apache.iceberg.util.StructLikeMap; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -642,7 +642,7 @@ public void testSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - Assertions.assertThatThrownBy(() -> spyRewrite.execute()) + assertThatThrownBy(() -> spyRewrite.execute()) .as("Should fail entire rewrite if part fails") .isInstanceOf(RuntimeException.class) .hasMessage("Rewrite Failed"); @@ -677,7 +677,7 @@ public void testSingleCommitWithCommitFailure() { doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); - Assertions.assertThatThrownBy(() -> spyRewrite.execute()) + assertThatThrownBy(() -> spyRewrite.execute()) .as("Should fail entire rewrite if commit fails") .isInstanceOf(RuntimeException.class) .hasMessage("Commit Failure"); @@ -713,7 +713,7 @@ public void testParallelSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - Assertions.assertThatThrownBy(() -> spyRewrite.execute()) + assertThatThrownBy(() -> spyRewrite.execute()) .as("Should fail entire rewrite if part fails") .isInstanceOf(RuntimeException.class) .hasMessage("Rewrite Failed"); @@ -858,7 +858,7 @@ public void testParallelPartialProgressWithCommitFailure() { public void testInvalidOptions() { Table table = createTable(20); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") @@ -870,7 +870,7 @@ public void testInvalidOptions() { "Cannot set partial-progress.max-commits to -5, " + "the value must be positive when partial-progress.enabled is true"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") @@ -880,13 +880,13 @@ public void testInvalidOptions() { .hasMessage( "Cannot set max-concurrent-file-group-rewrites to -5, the value must be positive."); - Assertions.assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) + assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) .as("No unknown options allowed") .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot use options [foobarity], they are not supported by the action or the rewriter BIN-PACK"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()) .as("Cannot set rewrite-job-order to foo") .isInstanceOf(IllegalArgumentException.class) @@ -1126,7 +1126,7 @@ public void testCommitStateUnknownException() { doReturn(util).when(spyAction).commitManager(table.currentSnapshot().snapshotId()); - Assertions.assertThatThrownBy(() -> spyAction.execute()) + assertThatThrownBy(() -> spyAction.execute()) .as("Should propagate CommitStateUnknown Exception") .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith( @@ -1246,17 +1246,17 @@ public void testInvalidAPIUsage() { SortOrder sortOrder = SortOrder.builderFor(table.schema()).asc("c2").build(); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) .as("Should be unable to set Strategy more than once") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Must use only one rewriter type"); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) .as("Should be unable to set Strategy more than once") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Must use only one rewriter type"); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) .as("Should be unable to set Strategy more than once") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Must use only one rewriter type"); @@ -1439,7 +1439,7 @@ public void testBinPackRewriterWithSpecificOutputSpec() { public void testBinpackRewriteWithInvalidOutputSpecId() { Table table = createTable(10); shouldHaveFiles(table, 10); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> actions() .rewriteDataFiles(table) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index 339bb1db328e..f8e6abffb43e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -25,6 +25,7 @@ import static org.apache.iceberg.ValidationHelpers.validateDataManifest; 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 static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; @@ -58,7 +59,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.TableIdentifier; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -231,7 +231,7 @@ public void testRewriteManifestsWithCommitStateUnknownException() { Table spyTable = spy(table); when(spyTable.rewriteManifests()).thenReturn(spyNewRewriteManifests); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> actions.rewriteManifests(spyTable).rewriteIf(manifest -> true).execute()) .as("Should throw a Commit State Unknown Exception") .cause() diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java index 6800ffd404ea..eef8fb43468f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.actions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.FileScanTask; @@ -37,7 +39,6 @@ import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StringType; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -199,16 +200,15 @@ private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { public void testInvalidConstructorUsagesSortData() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - Assertions.assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("is unsorted and no sort order is provided"); - Assertions.assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("the provided sort order is null or empty"); - Assertions.assertThatThrownBy( - () -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("the provided sort order is null or empty"); } @@ -217,20 +217,17 @@ public void testInvalidConstructorUsagesSortData() { public void testInvalidConstructorUsagesZOrderData() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA, SPEC); - Assertions.assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) .hasMessageContaining("Cannot ZOrder when no columns are specified"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) .hasMessageContaining("Cannot ZOrder when no columns are specified"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) .hasMessageContaining("Cannot ZOrder") .hasMessageContaining("all columns provided were identity partition columns"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) .hasMessageContaining("Cannot ZOrder") .hasMessageContaining("all columns provided were identity partition columns"); } @@ -303,7 +300,7 @@ public void testInvalidValuesForBinPackDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); } @@ -316,12 +313,12 @@ public void testInvalidValuesForSortDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); Map invalidCompressionFactorOptions = ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) + assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); } @@ -335,23 +332,23 @@ public void testInvalidValuesForZOrderDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); Map invalidCompressionFactorOptions = ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) + assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); Map invalidMaxOutputOptions = ImmutableMap.of(SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) .hasMessageContaining("Cannot have the interleaved ZOrder value use less than 1 byte") .hasMessageContaining("'max-output-size' was set to 0"); Map invalidVarLengthContributionOptions = ImmutableMap.of(SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) + assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) .hasMessageContaining("Cannot use less than 1 byte for variable length types with ZOrder") .hasMessageContaining("'var-length-contribution' was set to 0"); } @@ -359,19 +356,19 @@ public void testInvalidValuesForZOrderDataOptions() { private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewriter) { Map invalidTargetSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) .hasMessageContaining("'target-file-size-bytes' is set to 0 but must be > 0"); Map invalidMinSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) .hasMessageContaining("'min-file-size-bytes' is set to -1 but must be >= 0"); Map invalidTargetMinSizeOptions = ImmutableMap.of( SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "3", SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "5"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) .hasMessageContaining("'target-file-size-bytes' (3) must be > 'min-file-size-bytes' (5)") .hasMessageContaining("all new files will be smaller than the min threshold"); @@ -379,18 +376,18 @@ private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewrit ImmutableMap.of( SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "5", SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, "3"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) .hasMessageContaining("'target-file-size-bytes' (5) must be < 'max-file-size-bytes' (3)") .hasMessageContaining("all new files will be larger than the max threshold"); Map invalidMinInputFilesOptions = ImmutableMap.of(SizeBasedFileRewriter.MIN_INPUT_FILES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) .hasMessageContaining("'min-input-files' is set to 0 but must be > 0"); Map invalidMaxFileGroupSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.MAX_FILE_GROUP_SIZE_BYTES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) .hasMessageContaining("'max-file-group-size-bytes' is set to 0 but must be > 0"); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java index a96e3b1f57f5..256f2874745e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.data; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.assertj.core.api.Assertions.assertThat; import static scala.collection.JavaConverters.mapAsJavaMapConverter; import static scala.collection.JavaConverters.seqAsJavaListConverter; @@ -47,7 +48,6 @@ import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; -import org.assertj.core.api.Assertions; import org.junit.Assert; import scala.collection.Seq; @@ -119,79 +119,69 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case DATE: - Assertions.assertThat(expected) - .as("Should expect a LocalDate") - .isInstanceOf(LocalDate.class); - Assertions.assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); + assertThat(expected).as("Should expect a LocalDate").isInstanceOf(LocalDate.class); + assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); Assert.assertEquals( "ISO-8601 date should be equal", expected.toString(), actual.toString()); break; case TIMESTAMP: - Assertions.assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); + assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); Timestamp ts = (Timestamp) actual; // milliseconds from nanos has already been added by getTime OffsetDateTime actualTs = EPOCH.plusNanos((ts.getTime() * 1_000_000) + (ts.getNanos() % 1_000_000)); Types.TimestampType timestampType = (Types.TimestampType) type; if (timestampType.shouldAdjustToUTC()) { - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an OffsetDateTime") .isInstanceOf(OffsetDateTime.class); Assert.assertEquals("Timestamp should be equal", expected, actualTs); } else { - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an LocalDateTime") .isInstanceOf(LocalDateTime.class); Assert.assertEquals("Timestamp should be equal", expected, actualTs.toLocalDateTime()); } break; case STRING: - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("Strings should be equal", String.valueOf(expected), actual); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("UUID string representation should match", expected.toString(), actual); break; case FIXED: - Assertions.assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals("Bytes should match", (byte[]) expected, (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); Assert.assertEquals("BigDecimals should be equal", expected, actual); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); assertEqualsSafe(type.asNestedType().asStructType(), (Record) expected, (Row) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be a Map") - .isInstanceOf(scala.collection.Map.class); + assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); + assertThat(actual).as("Should be a Map").isInstanceOf(scala.collection.Map.class); Map asMap = mapAsJavaMapConverter((scala.collection.Map) actual).asJava(); assertEqualsSafe(type.asNestedType().asMapType(), (Map) expected, asMap); @@ -259,23 +249,21 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case DATE: - Assertions.assertThat(expected) - .as("Should expect a LocalDate") - .isInstanceOf(LocalDate.class); + assertThat(expected).as("Should expect a LocalDate").isInstanceOf(LocalDate.class); int expectedDays = (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) expected); Assert.assertEquals("Primitive value should be equal to expected", expectedDays, actual); break; case TIMESTAMP: Types.TimestampType timestampType = (Types.TimestampType) type; if (timestampType.shouldAdjustToUTC()) { - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an OffsetDateTime") .isInstanceOf(OffsetDateTime.class); long expectedMicros = ChronoUnit.MICROS.between(EPOCH, (OffsetDateTime) expected); Assert.assertEquals( "Primitive value should be equal to expected", expectedMicros, actual); } else { - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an LocalDateTime") .isInstanceOf(LocalDateTime.class); long expectedMicros = @@ -285,57 +273,47 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual } break; case STRING: - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals("Strings should be equal", expected, actual.toString()); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals( "UUID string representation should match", expected.toString(), actual.toString()); break; case FIXED: - Assertions.assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals("Bytes should match", (byte[]) expected, (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); Assert.assertEquals( "BigDecimals should be equal", expected, ((Decimal) actual).toJavaBigDecimal()); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual) - .as("Should be an InternalRow") - .isInstanceOf(InternalRow.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be an InternalRow").isInstanceOf(InternalRow.class); assertEqualsUnsafe( type.asNestedType().asStructType(), (Record) expected, (InternalRow) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be an ArrayBasedMapData") - .isInstanceOf(MapData.class); + 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); break; case TIME: diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index f2633fc67640..4bd3531ab954 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.data; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.assertj.core.api.Assertions.assertThat; import static scala.collection.JavaConverters.mapAsJavaMapConverter; import static scala.collection.JavaConverters.seqAsJavaListConverter; @@ -76,7 +77,6 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.apache.spark.unsafe.types.UTF8String; -import org.assertj.core.api.Assertions; import org.junit.Assert; import scala.collection.Seq; @@ -165,70 +165,60 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case DATE: - Assertions.assertThat(expected).as("Should be an int").isInstanceOf(Integer.class); - Assertions.assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); + assertThat(expected).as("Should be an int").isInstanceOf(Integer.class); + assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); int daysFromEpoch = (Integer) expected; LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); Assert.assertEquals("ISO-8601 date should be equal", date.toString(), actual.toString()); break; case TIMESTAMP: - Assertions.assertThat(expected).as("Should be a long").isInstanceOf(Long.class); - Assertions.assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); + assertThat(expected).as("Should be a long").isInstanceOf(Long.class); + assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); Timestamp ts = (Timestamp) actual; // milliseconds from nanos has already been added by getTime long tsMicros = (ts.getTime() * 1000) + ((ts.getNanos() / 1000) % 1000); Assert.assertEquals("Timestamp micros should be equal", expected, tsMicros); break; case STRING: - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("Strings should be equal", String.valueOf(expected), actual); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("UUID string representation should match", expected.toString(), actual); break; case FIXED: - Assertions.assertThat(expected) - .as("Should expect a Fixed") - .isInstanceOf(GenericData.Fixed.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a Fixed").isInstanceOf(GenericData.Fixed.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((GenericData.Fixed) expected).bytes(), (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); Assert.assertEquals("BigDecimals should be equal", expected, actual); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); assertEqualsSafe(type.asNestedType().asStructType(), (Record) expected, (Row) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be a Map") - .isInstanceOf(scala.collection.Map.class); + assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); + assertThat(actual).as("Should be a Map").isInstanceOf(scala.collection.Map.class); Map asMap = mapAsJavaMapConverter((scala.collection.Map) actual).asJava(); assertEqualsSafe(type.asNestedType().asMapType(), (Map) expected, asMap); @@ -288,7 +278,7 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual switch (type.typeId()) { case LONG: - Assertions.assertThat(actual).as("Should be a long").isInstanceOf(Long.class); + assertThat(actual).as("Should be a long").isInstanceOf(Long.class); if (expected instanceof Integer) { Assert.assertEquals("Values didn't match", ((Number) expected).longValue(), actual); } else { @@ -296,7 +286,7 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual } break; case DOUBLE: - Assertions.assertThat(actual).as("Should be a double").isInstanceOf(Double.class); + assertThat(actual).as("Should be a double").isInstanceOf(Double.class); if (expected instanceof Float) { Assert.assertEquals( "Values didn't match", @@ -314,60 +304,48 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case STRING: - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals("Strings should be equal", expected, actual.toString()); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals( "UUID string representation should match", expected.toString(), actual.toString()); break; case FIXED: - Assertions.assertThat(expected) - .as("Should expect a Fixed") - .isInstanceOf(GenericData.Fixed.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a Fixed").isInstanceOf(GenericData.Fixed.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((GenericData.Fixed) expected).bytes(), (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); Assert.assertEquals( "BigDecimals should be equal", expected, ((Decimal) actual).toJavaBigDecimal()); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual) - .as("Should be an InternalRow") - .isInstanceOf(InternalRow.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be an InternalRow").isInstanceOf(InternalRow.class); assertEqualsUnsafe( type.asNestedType().asStructType(), (Record) expected, (InternalRow) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be an ArrayBasedMapData") - .isInstanceOf(MapData.class); + 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); break; case TIME: @@ -688,30 +666,28 @@ private static void assertEquals(String context, DataType type, Object expected, } if (type instanceof StructType) { - Assertions.assertThat(expected) + assertThat(expected) .as("Expected should be an InternalRow: " + context) .isInstanceOf(InternalRow.class); - Assertions.assertThat(actual) + assertThat(actual) .as("Actual should be an InternalRow: " + context) .isInstanceOf(InternalRow.class); assertEquals(context, (StructType) type, (InternalRow) expected, (InternalRow) actual); } else if (type instanceof ArrayType) { - Assertions.assertThat(expected) + assertThat(expected) .as("Expected should be an ArrayData: " + context) .isInstanceOf(ArrayData.class); - Assertions.assertThat(actual) + assertThat(actual) .as("Actual should be an ArrayData: " + context) .isInstanceOf(ArrayData.class); assertEquals(context, (ArrayType) type, (ArrayData) expected, (ArrayData) actual); } else if (type instanceof MapType) { - Assertions.assertThat(expected) + assertThat(expected) .as("Expected should be a MapData: " + context) .isInstanceOf(MapData.class); - Assertions.assertThat(actual) - .as("Actual should be a MapData: " + context) - .isInstanceOf(MapData.class); + assertThat(actual).as("Actual should be a MapData: " + context).isInstanceOf(MapData.class); assertEquals(context, (MapType) type, (MapData) expected, (MapData) actual); } else if (type instanceof BinaryType) { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 024ce3a60c2b..77ad638542f5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -20,6 +20,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 java.io.File; import java.io.IOException; @@ -56,7 +57,6 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -158,7 +158,7 @@ public void testInt96TimestampProducedBySparkIsReadCorrectly() throws IOExceptio InputFile parquetInputFile = Files.localInput(outputFilePath); List readRows = rowsFromFile(parquetInputFile, schema); Assert.assertEquals(rows.size(), readRows.size()); - Assertions.assertThat(readRows).isEqualTo(rows); + assertThat(readRows).isEqualTo(rows); // Now we try to import that file as an Iceberg table to make sure Iceberg can read // Int96 end to end. diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index 866bbd240aa5..aca6202870ff 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -20,6 +20,7 @@ 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.File; import java.io.IOException; @@ -48,7 +49,6 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.apache.spark.sql.vectorized.ColumnarBatch; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Ignore; @@ -255,7 +255,7 @@ public void testMixedTypes() {} @Test @Override public void testNestedStruct() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> VectorizedSparkParquetReaders.buildReader( TypeUtil.assignIncreasingFreshIds( @@ -390,8 +390,7 @@ public void testUnsupportedReadsForParquetV2() throws Exception { try (FileAppender writer = parquetV2Writer(schema, dataFile)) { writer.addAll(data); } - Assertions.assertThatThrownBy( - () -> assertRecordsMatch(schema, 30000, data, dataFile, true, BATCH_SIZE)) + assertThatThrownBy(() -> assertRecordsMatch(schema, 30000, data, dataFile, true, BATCH_SIZE)) .as("Vectorized reads not supported") .isInstanceOf(UnsupportedOperationException.class) .hasMessageContaining("Cannot support vectorized reads for column"); 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 f57c87e8b809..59a32bd239df 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,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -32,7 +34,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -76,7 +77,7 @@ public void testMergeSchemaFailsWithoutWriterOption() throws Exception { // this has a different error message than the case without accept-any-schema because it uses // Iceberg checks - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { threeColDF.writeTo(tableName).append(); @@ -111,7 +112,7 @@ public void testMergeSchemaWithoutAcceptAnySchema() throws Exception { "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { threeColDF.writeTo(tableName).option("merge-schema", "true").append(); 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 310e69b827a9..a5a3da39ad1f 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 @@ -21,6 +21,7 @@ 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.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -63,7 +64,6 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.encoders.RowEncoder; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; @@ -408,8 +408,7 @@ public void testFaultToleranceOnWrite() throws IOException { Iterable records2 = RandomData.generate(schema, 100, 0L); - Assertions.assertThatThrownBy( - () -> writeDataWithFailOnPartition(records2, schema, location.toString())) + assertThatThrownBy(() -> writeDataWithFailOnPartition(records2, schema, location.toString())) .isInstanceOf(SparkException.class); table.refresh(); 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 2c3a7aba59d1..9cee25560366 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 @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.source; 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 java.io.IOException; import java.math.RoundingMode; @@ -54,7 +56,6 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -245,7 +246,7 @@ public void testIncrementalScanOptions() throws IOException { List snapshotIds = SnapshotUtil.currentAncestorIds(table); // start-snapshot-id and snapshot-id are both configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() @@ -261,7 +262,7 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set start-snapshot-id and end-snapshot-id for incremental scans"); // end-snapshot-id and as-of-timestamp are both configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() @@ -279,7 +280,7 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set start-snapshot-id and end-snapshot-id for incremental scans"); // only end-snapshot-id is configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() @@ -455,7 +456,7 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assertions.assertThat(snapshots.get(1).summary()) + assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") .containsEntry("extra-key", "someValue") .containsEntry("another-key", "anotherValue"); @@ -498,7 +499,7 @@ public void testExtraSnapshotMetadataWithDelete() List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assertions.assertThat(snapshots.get(1).summary()) + assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") .containsEntry("extra-key", "someValue") .containsEntry("another-key", "anotherValue"); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index a616b764a6b1..9f3432d25671 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -22,6 +22,7 @@ import static org.apache.spark.sql.catalyst.util.DateTimeUtils.fromJavaTimestamp; import static org.apache.spark.sql.functions.callUDF; import static org.apache.spark.sql.functions.column; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -71,7 +72,6 @@ import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -601,7 +601,7 @@ private List expected(int... ordinals) { } private void pushFilters(ScanBuilder scan, Filter... filters) { - Assertions.assertThat(scan).isInstanceOf(SupportsPushDownFilters.class); + assertThat(scan).isInstanceOf(SupportsPushDownFilters.class); SupportsPushDownFilters filterable = (SupportsPushDownFilters) scan; filterable.pushFilters(filters); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 8630c50151f3..e180d1a53bd8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.Files.localInput; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -54,7 +55,6 @@ import org.apache.spark.sql.execution.streaming.MemoryStream; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -120,7 +120,7 @@ public void testSparkWriteFailsUnknownTransform() throws IOException { Dataset df = spark.createDataFrame(expected, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> df.select("id", "data") .write() @@ -159,7 +159,7 @@ public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, T List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .as("Should reject streaming write with unsupported transform") .isInstanceOf(StreamingQueryException.class) .hasMessageContaining("Cannot write using unsupported transforms: zero"); 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 079ca5f77fa4..32b96a580da5 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 @@ -22,6 +22,8 @@ import static org.apache.iceberg.ManifestContent.DELETES; 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; @@ -91,7 +93,6 @@ import org.apache.spark.sql.functions; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -1118,7 +1119,7 @@ public void testPruneManifestsTable() { if (!spark.version().startsWith("2")) { // Spark 2 isn't able to actually push down nested struct projections so this will not break - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -2176,9 +2177,7 @@ private void testWithFilter(String filterExpr, TableIdentifier tableIdentifier) .select("tmp_col") .filter(filterExpr) .collectAsList(); - Assertions.assertThat(actual) - .as("Rows must match") - .containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); } private GenericData.Record manifestRecord( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java index 37e329a8b97b..0154506f86b8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.sql.Date; @@ -33,7 +35,6 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; import org.apache.spark.sql.types.VarcharType; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -181,7 +182,7 @@ public void testRegisterDecimalBucketUDF() { @Test public void testRegisterBooleanBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_boolean_16", DataTypes.BooleanType, 16)) @@ -191,7 +192,7 @@ public void testRegisterBooleanBucketUDF() { @Test public void testRegisterDoubleBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_double_16", DataTypes.DoubleType, 16)) @@ -201,7 +202,7 @@ public void testRegisterDoubleBucketUDF() { @Test public void testRegisterFloatBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_float_16", DataTypes.FloatType, 16)) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index 9c5d48e1debb..37a4cbf2e13f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -29,6 +29,7 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.List; @@ -53,7 +54,6 @@ import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -641,7 +641,7 @@ public void testMetadataTablesWithUnknownTransforms() { sql("REFRESH TABLE %s", tableName); for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES, ENTRIES, ALL_ENTRIES)) { - Assertions.assertThatThrownBy(() -> loadMetadataTable(tableType)) + assertThatThrownBy(() -> loadMetadataTable(tableType)) .as("Should complain about the partition type") .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot build table partition type, unknown transforms"); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java index 5baf6071233d..e18c4d32412f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; 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; @@ -35,7 +36,6 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -76,9 +76,8 @@ public void testPathIdentifier() throws TableAlreadyExistsException, NoSuchTable identifier, SparkSchemaUtil.convert(SCHEMA), new Transform[0], ImmutableMap.of()); Assert.assertEquals(table.table().location(), tableLocation.getAbsolutePath()); - Assertions.assertThat(table.table()).isInstanceOf(BaseTable.class); - Assertions.assertThat(((BaseTable) table.table()).operations()) - .isInstanceOf(HadoopTableOperations.class); + assertThat(table.table()).isInstanceOf(BaseTable.class); + assertThat(((BaseTable) table.table()).operations()).isInstanceOf(HadoopTableOperations.class); Assert.assertEquals(sparkCatalog.loadTable(identifier), table); Assert.assertTrue(sparkCatalog.dropTable(identifier)); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java index eecc405b1a09..38c8425fd923 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.avro.Schema.Type.UNION; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.List; @@ -33,7 +34,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -141,8 +141,7 @@ public void testEmptyProjection() throws Exception { Assert.assertNotNull("Should read a non-null record", projected); // this is expected because there are no values - Assertions.assertThatThrownBy(() -> projected.get(0)) - .isInstanceOf(ArrayIndexOutOfBoundsException.class); + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java index e8e402ac44cc..427546fc1d2b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Table; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -163,7 +164,7 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { inputDF @@ -233,7 +234,7 @@ public void testNoSortBucketTransformsWithoutExtensions() throws NoSuchTableExce Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail by default as extensions are disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try { inputDF.writeTo(tableName).append(); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index a161224275ca..64541e783171 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.source; 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 java.io.IOException; import java.util.List; @@ -38,7 +40,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -174,7 +175,7 @@ public void testSnapshotSelectionByInvalidSnapshotId() throws IOException { Dataset df = spark.read().format("iceberg").option("snapshot-id", -10).load(tableLocation); - Assertions.assertThatThrownBy(df::collectAsList) + assertThatThrownBy(df::collectAsList) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find snapshot with ID -10"); } @@ -188,7 +189,7 @@ public void testSnapshotSelectionByInvalidTimestamp() throws IOException { PartitionSpec spec = PartitionSpec.unpartitioned(); tables.create(SCHEMA, spec, tableLocation); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -216,7 +217,7 @@ public void testSnapshotSelectionBySnapshotIdAndTimestamp() throws IOException { long timestamp = System.currentTimeMillis(); long snapshotId = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -319,7 +320,7 @@ public void testSnapshotSelectionByBranchAndTagFails() throws IOException { table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -350,7 +351,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -362,7 +363,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Can specify only one of snapshot-id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -405,7 +406,7 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { table.updateSchema().deleteColumn("data").commit(); // The data should not have the deleted column - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -418,7 +419,7 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { // re-introducing the column should not let the data re-appear table.updateSchema().addColumn("data", Types.StringType.get()).commit(); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -460,7 +461,7 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { // Deleting and add a new column of the same type to indicate schema change table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -491,7 +492,7 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { .mode("append") .save(tableLocation); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index 3d668197fd51..abe70f94fe19 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.CatalogProperties; @@ -27,7 +29,6 @@ import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.BeforeClass; import org.junit.Test; @@ -93,18 +94,18 @@ public TestSparkCatalogCacheExpiration() { @Test public void testSparkSessionCatalogWithExpirationEnabled() { SparkSessionCatalog sparkCatalog = sparkSessionCatalog(); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .extracting("cacheEnabled") .isEqualTo(true); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, icebergCatalog -> { - Assertions.assertThat(icebergCatalog) + assertThat(icebergCatalog) .isExactlyInstanceOf(CachingCatalog.class) .extracting("expirationIntervalMillis") .isEqualTo(3000L); @@ -114,30 +115,27 @@ public void testSparkSessionCatalogWithExpirationEnabled() { @Test public void testCacheEnabledAndExpirationDisabled() { SparkCatalog sparkCatalog = getSparkCatalog("expiration_disabled"); - Assertions.assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .isInstanceOfSatisfying( CachingCatalog.class, icebergCatalog -> { - Assertions.assertThat(icebergCatalog) - .extracting("expirationIntervalMillis") - .isEqualTo(-1L); + assertThat(icebergCatalog).extracting("expirationIntervalMillis").isEqualTo(-1L); }); } @Test public void testCacheDisabledImplicitly() { SparkCatalog sparkCatalog = getSparkCatalog("cache_disabled_implicitly"); - Assertions.assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, - icebergCatalog -> - Assertions.assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); + icebergCatalog -> assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); } private SparkSessionCatalog sparkSessionCatalog() { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index 3d759759f608..a0de190e4be4 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -20,6 +20,8 @@ import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; 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.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -53,7 +55,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; @@ -690,7 +691,7 @@ public void testCommitUnknownException() throws IOException { ManualSource.setTable(manualTableName, sparkTable); // Although an exception is thrown here, write and commit have succeeded - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> df2.select("id", "data") .sort("data") @@ -712,7 +713,7 @@ public void testCommitUnknownException() throws IOException { result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); Assert.assertEquals( "Number of rows should match", records.size() + records2.size(), actual.size()); - Assertions.assertThat(actual) + assertThat(actual) .describedAs("Result rows should match") .containsExactlyInAnyOrder( ImmutableList.builder() diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 2b0033fdcbb5..e24e74383bc8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -25,6 +25,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.List; @@ -52,7 +53,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -224,7 +224,7 @@ public void testPartitionMetadataColumnWithUnknownTransforms() { TableMetadata base = ops.current(); ops.commit(base, base.updatePartitionSpec(UNKNOWN_SPEC)); - Assertions.assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) .as("Should fail to query the partition metadata column") .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot build table partition type, unknown transforms"); @@ -245,7 +245,7 @@ public void testConflictingColumns() { ImmutableList.of(row(1L, "a1")), sql("SELECT id, category FROM %s", TABLE_NAME)); - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) .as("Should fail to query conflicting columns") .isInstanceOf(ValidationException.class) .hasMessageContaining("column names conflict"); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 464f1f5922b3..bd20a628caa8 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.util.List; @@ -40,7 +41,6 @@ import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -283,7 +283,7 @@ public void testStreamingWriteUpdateMode() throws Exception { List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .isInstanceOf(StreamingQueryException.class) .hasMessageContaining("does not support Update mode"); } finally { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index c8e37d932290..a525b5cdb20a 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.expressions.Expressions.ref; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; @@ -57,7 +58,6 @@ import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.OutputMode; import org.apache.spark.sql.streaming.StreamingQuery; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -144,7 +144,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception StreamingQuery query = startStream(); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -183,7 +183,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_1 // check answer correctness only 1 record read the micro-batch will be stuck List actual = rowsAvailable(query); - Assertions.assertThat(actual) + assertThat(actual) .containsExactlyInAnyOrderElementsOf( Lists.newArrayList(TEST_DATA_MULTIPLE_SNAPSHOTS.get(0).get(0))); } @@ -207,7 +207,7 @@ public void testReadStreamOnIcebergThenAddData() throws Exception { appendDataAsMultipleSnapshots(expected); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -227,14 +227,14 @@ public void testReadingStreamFromTimestamp() throws Exception { startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(streamStartTimestamp)); List empty = rowsAvailable(query); - Assertions.assertThat(empty.isEmpty()).isTrue(); + assertThat(empty.isEmpty()).isTrue(); List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -245,7 +245,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(futureTimestamp)); List actual = rowsAvailable(query); - Assertions.assertThat(actual.isEmpty()).isTrue(); + assertThat(actual.isEmpty()).isTrue(); List data = Lists.newArrayList( @@ -258,7 +258,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { .forEach( x -> { appendData(data); - Assertions.assertThat(rowsAvailable(query).isEmpty()).isTrue(); + assertThat(rowsAvailable(query).isEmpty()).isTrue(); }); waitUntilAfter(futureTimestamp); @@ -266,7 +266,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { // Data appended after the timestamp should appear appendData(data); actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(data); + assertThat(actual).containsExactlyInAnyOrderElementsOf(data); } @Test @@ -288,7 +288,7 @@ public void testReadingStreamFromTimestampFutureWithExistingSnapshots() throws E waitUntilAfter(streamStartTimestamp); List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @@ -311,7 +311,7 @@ public void testReadingStreamFromTimestampOfExistingSnapshot() throws Exception } List actual = rowsAvailable(stream); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -340,7 +340,7 @@ public void testReadingStreamWithExpiredSnapshotFromTimestamp() throws TimeoutEx startStream( SparkReadOptions.STREAM_FROM_TIMESTAMP, String.valueOf(firstSnapshotCommitTime)); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); } @Test @@ -379,7 +379,7 @@ public void testResumingStreamReadFromCheckpoint() throws Exception { // Read data added by the stream List actual = spark.read().load(output.getPath()).as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } } @@ -440,7 +440,7 @@ public void testParquetOrcAvroDataInOneTable() throws Exception { appendData(avroFileRecords, "avro"); StreamingQuery query = startStream(); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf( Iterables.concat(parquetFileRecords, orcFileRecords, avroFileRecords)); } @@ -494,7 +494,7 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception StreamingQuery query = startStream(); - Assertions.assertThatThrownBy(() -> query.processAllAvailable()) + assertThatThrownBy(() -> query.processAllAvailable()) .as( "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND") .cause() @@ -516,7 +516,7 @@ public void testReadStreamWithSnapshotTypeReplaceIgnoresReplace() throws Excepti StreamingQuery query = startStream(); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -536,7 +536,7 @@ public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { StreamingQuery query = startStream(); - Assertions.assertThatThrownBy(() -> query.processAllAvailable()) + assertThatThrownBy(() -> query.processAllAvailable()) .as( "Streaming should fail with IllegalStateException, as the snapshot is not of type APPEND") .cause() @@ -560,7 +560,7 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipDeleteOption() throws Exc Assert.assertEquals(DataOperations.DELETE, table.currentSnapshot().operation()); StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS, "true"); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); } @@ -592,7 +592,7 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws Assert.assertEquals(DataOperations.OVERWRITE, table.currentSnapshot().operation()); StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS, "true"); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java index d85e1954950d..4f035aff93c1 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestTimestampWithoutZone.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -52,7 +53,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -159,7 +159,7 @@ public void testUnpartitionedTimestampWithoutZoneProjection() { @Test public void testUnpartitionedTimestampWithoutZoneError() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -218,7 +218,7 @@ public void testUnpartitionedTimestampWithoutZoneWriteError() { .mode(SaveMode.Append) .save(unpartitioned.toString()); - Assertions.assertThatThrownBy(writeOperation::run) + assertThatThrownBy(writeOperation::run) .as(errorMessage) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining(SparkUtil.TIMESTAMP_WITHOUT_TIMEZONE_ERROR); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index fd88eb2c907e..c2b396bdb9b6 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.spark.SparkSchemaUtil.convert; 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.nio.ByteBuffer; @@ -48,7 +49,6 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -231,8 +231,7 @@ public void testBadCustomMetricCollectionForParquet() throws IOException { properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); properties.put("write.metadata.metrics.column.ids", "full"); - Assertions.assertThatThrownBy( - () -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) + assertThatThrownBy(() -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) .as("Creating a table with invalid metrics should fail") .isInstanceOf(ValidationException.class) .hasMessageStartingWith( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 9ea1a563efa8..2579d614685c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; @@ -36,7 +38,6 @@ import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkTestBase; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.BeforeClass; @@ -571,15 +572,9 @@ private void assertAggregates( Object actualMax = actual.get(0)[1]; Object actualMin = actual.get(0)[2]; - Assertions.assertThat(actualCount) - .as("Expected and actual count should equal") - .isEqualTo(expectedCount); - Assertions.assertThat(actualMax) - .as("Expected and actual max should equal") - .isEqualTo(expectedMax); - Assertions.assertThat(actualMin) - .as("Expected and actual min should equal") - .isEqualTo(expectedMin); + assertThat(actualCount).as("Expected and actual count should equal").isEqualTo(expectedCount); + assertThat(actualMax).as("Expected and actual max should equal").isEqualTo(expectedMax); + assertThat(actualMin).as("Expected and actual min should equal").isEqualTo(expectedMin); } private void assertExplainContains(List explain, String... expectedFragments) { @@ -587,7 +582,7 @@ private void assertExplainContains(List explain, String... expectedFra Arrays.stream(expectedFragments) .forEach( fragment -> - Assertions.assertThat(explainString) + assertThat(explainString) .as("Expected to find plan fragment in explain plan") .contains(fragment)); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java index 901a7c15e78f..2570d4b81906 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestAlterTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; @@ -27,7 +29,6 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.spark.SparkException; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -55,7 +56,7 @@ public void removeTable() { @Test public void testAddColumnNotNull() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN c3 INT NOT NULL", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN c3 INT NOT NULL", tableName)) .as("Should reject adding NOT NULL column") .isInstanceOf(SparkException.class) .hasMessageContaining("Incompatible change: cannot add required column"); @@ -155,7 +156,7 @@ public void testAddColumnWithMap() { validationCatalog.loadTable(tableIdent).schema().asStruct()); // should not allow changing map key column - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN data2.key.y int", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s ADD COLUMN data2.key.y int", tableName)) .as("Should reject changing key of the map column") .isInstanceOf(SparkException.class) .hasMessageContaining("Unsupported table change: Cannot add fields to map keys:"); @@ -251,8 +252,7 @@ public void testAlterColumnSetNotNull() { expectedSchema, validationCatalog.loadTable(tableIdent).schema().asStruct()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s ALTER COLUMN data SET NOT NULL", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s ALTER COLUMN data SET NOT NULL", tableName)) .as("Should reject adding NOT NULL constraint to an optional column") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot change nullable column to non-nullable: data"); @@ -321,7 +321,7 @@ public void testSetTableProperties() { "Should not have the removed table property", validationCatalog.loadTable(tableIdent).properties().get("prop")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('sort-order'='value')", tableName)) .as("Cannot specify the 'sort-order' because it's a reserved table property") .isInstanceOf(UnsupportedOperationException.class) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index f4bef9cec155..bf2200004282 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.File; import java.util.Map; import java.util.UUID; @@ -33,7 +36,6 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -107,27 +109,24 @@ public void testCreateTable() { @Test public void testCreateTablePartitionedByUUID() { - Assertions.assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); validationCatalog.createTable(tableIdent, schema, spec); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table).isNotNull(); + assertThat(table).isNotNull(); StructType expectedSchema = StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); - Assertions.assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); - Assertions.assertThat(table.spec().fields()).hasSize(1); + assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + assertThat(table.spec().fields()).hasSize(1); String uuid = UUID.randomUUID().toString(); sql("INSERT INTO %s VALUES('%s')", tableName, uuid); - Assertions.assertThat(sql("SELECT uuid FROM %s", tableName)) - .hasSize(1) - .element(0) - .isEqualTo(row(uuid)); + assertThat(sql("SELECT uuid FROM %s", tableName)).hasSize(1).element(0).isEqualTo(row(uuid)); } @Test @@ -167,7 +166,7 @@ public void testCreateTableUsingParquet() { "parquet", table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE %s.default.fail (id BIGINT NOT NULL, data STRING) USING crocodile", @@ -381,7 +380,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { TableOperations ops = ((BaseTable) table).operations(); Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)) .as("should fail to downgrade to v1") .cause() diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index 55031188453a..e1c228be101e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Table; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -86,8 +87,7 @@ public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) + assertThatThrownBy(() -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) .as("Should not be able to delete from a table at a specific snapshot") .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot delete from table at a specific snapshot"); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java index c5c39e5d782f..e2fd0dd8125d 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -29,7 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.SparkCatalogTestBase; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -116,7 +117,7 @@ public void testPurgeTableGCDisabled() throws IOException { "There totally should have 2 files for manifests and files", 2, manifestAndFiles.size()); Assert.assertTrue("All files should be existed", checkFilesExist(manifestAndFiles, true)); - Assertions.assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) + assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) .as("Purge table is not allowed when GC is disabled") .isInstanceOf(ValidationException.class) .hasMessageContaining( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java index f1e2169af453..98112acb6e60 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; @@ -27,7 +29,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.execution.SparkPlan; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -555,16 +556,14 @@ private void checkFilters( String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); if (sparkFilter != null) { - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Post scan filter should match") .contains("Filter (" + sparkFilter + ")"); } else { - Assertions.assertThat(planAsString) - .as("Should be no post scan filter") - .doesNotContain("Filter ("); + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); } - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Pushed filters must match") .contains("[filters=" + icebergFilters + ","); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java index 77b564cd001b..cdfe3d856e86 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.File; import java.util.List; import java.util.Map; @@ -29,7 +31,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.spark.SparkCatalogTestBase; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -103,7 +104,7 @@ public void testDropNonEmptyNamespace() { Assert.assertTrue( "Table should exist", validationCatalog.tableExists(TableIdentifier.of(NS, "table"))); - Assertions.assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) + assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) .as("Should fail if trying to delete a non-empty namespace") .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageContaining("Namespace db is not empty."); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java index 5dde5f33d965..064ad0f1785c 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import java.util.UUID; @@ -25,7 +27,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.spark.SparkSQLProperties; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -72,8 +73,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { Table table = validationCatalog.loadTable(tableIdent); table.manageSnapshots().createBranch("test2", table.refs().get(BRANCH).snapshotId()).commit(); sql("REFRESH TABLE " + tableName); - Assertions.assertThatThrownBy( - () -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) + assertThatThrownBy(() -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot write to both branch and WAP branch, but got branch [test2] and WAP branch [%s]", @@ -84,7 +84,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { public void testWapIdAndWapBranchCannotBothBeSetForWrite() { String wapId = UUID.randomUUID().toString(); spark.conf().set(SparkSQLProperties.WAP_ID, wapId); - Assertions.assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) + assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", wapId, BRANCH); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index ea97949d1327..f5857973996f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.text.SimpleDateFormat; import java.util.Date; import java.util.List; @@ -34,7 +36,6 @@ import org.apache.iceberg.spark.SparkReadOptions; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -327,8 +328,7 @@ public void testBranchReference() { @Test public void testUnknownReferenceAsOf() { - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) + assertThatThrownBy(() -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) .hasMessageContaining("Cannot find matching snapshot ID or reference name for version") .isInstanceOf(ValidationException.class); } @@ -395,7 +395,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using snapshot in table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s VERSION AS OF %s", @@ -406,7 +406,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); // using snapshot in table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s VERSION AS OF %s", @@ -417,7 +417,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s TIMESTAMP AS OF %s", @@ -428,7 +428,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessageContaining("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s TIMESTAMP AS OF %s", @@ -448,14 +448,13 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using branch_b1 in the table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using branch_b1 in the table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + assertThatThrownBy(() -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @@ -471,7 +470,7 @@ public void testSpecifySnapshotAndTimestamp() { // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java index 2cf2865fe8bb..250e3004b83b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark.sql; +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.charset.StandardCharsets; @@ -28,7 +31,6 @@ import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.types.DataTypes; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -190,19 +192,19 @@ public void testNumBucketsAcceptsShortAndByte() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) .as("Function resolution should not work with zero arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (): Wrong number of inputs (expected numBuckets and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) .as("Function resolution should not work with only one argument") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (int): Wrong number of inputs (expected numBuckets and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) .as("Function resolution should not work with more than two arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining( @@ -211,33 +213,31 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) .as("Decimal type should not be coercible to the number of buckets") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (decimal(9,2), int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) .as("Long type should not be coercible to the number of buckets") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (bigint, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) .as("String type should not be coercible to the number of buckets") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (string, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) .as("Interval year to month type should not be coercible to the number of buckets") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (interval year to month, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.bucket(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) .as("Interval day-time type should not be coercible to the number of buckets") @@ -248,43 +248,40 @@ public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { @Test public void testInvalidTypesForBucketColumn() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) .as("Double type should not be bucketable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (int, float): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) .as("Double type should not be bucketable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (int, double): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) .as("Boolean type should not be bucketable") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'bucket' cannot process input: (int, boolean)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) .as("Map types should not be bucketable") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'bucket' cannot process input: (int, map)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) .as("Array types should not be bucketable") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'bucket' cannot process input: (int, array)"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) .as("Interval year-to-month type should not be bucketable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'bucket' cannot process input: (int, interval year to month)"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.bucket(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) .as("Interval day-time type should not be bucketable") @@ -296,63 +293,59 @@ public void testInvalidTypesForBucketColumn() { @Test public void testThatMagicFunctionsAreInvoked() { // TinyInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // SmallInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Int - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Date - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Long - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); // Timestamp - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); // String - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketString"); // Decimal - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketDecimal"); // Binary - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) .asString() .isNotNull() .contains( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java index e139d3d000ef..c60e5ea4d8a5 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.sql.Date; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -69,12 +70,12 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days()")) + assertThatThrownBy(() -> scalarSql("SELECT system.days()")) .as("Function resolution should not work with zero arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'days' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.days(date('1969-12-31'), date('1969-12-31'))")) .as("Function resolution should not work with more than one argument") .isInstanceOf(AnalysisException.class) @@ -84,13 +85,13 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) .as("Int type should not be coercible to date/timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'days' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) .as("Long type should not be coercible to date/timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java index 312299ffee09..02887257378b 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -51,12 +52,12 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) .as("Function resolution should not work with zero arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'hours' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.hours(date('1969-12-31'), date('1969-12-31'))")) .as("Function resolution should not work with more than one argument") .isInstanceOf(AnalysisException.class) @@ -66,13 +67,13 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) .as("Int type should not be coercible to timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'hours' cannot process input: (int): Expected value to be timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) .as("Long type should not be coercible to timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java index c95823b889c3..804baf5351e7 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.MonthsFunction; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -67,12 +69,12 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months()")) + assertThatThrownBy(() -> scalarSql("SELECT system.months()")) .as("Function resolution should not work with zero arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'months' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.months(date('1969-12-31'), date('1969-12-31'))")) .as("Function resolution should not work with more than one argument") .isInstanceOf(AnalysisException.class) @@ -82,13 +84,13 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) .as("Int type should not be coercible to date/timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'months' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) .as("Long type should not be coercible to date/timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( @@ -99,14 +101,14 @@ public void testInvalidInputTypes() { public void testThatMagicFunctionsAreInvoked() { String dateValue = "date('2017-12-01')"; String dateTransformClass = MonthsFunction.DateToMonthsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) .asString() .isNotNull() .contains("staticinvoke(class " + dateTransformClass); String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; String timestampTransformClass = MonthsFunction.TimestampToMonthsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) .asString() .isNotNull() .contains("staticinvoke(class " + timestampTransformClass); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java index aaf4f569b655..9000170a3eee 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java @@ -18,12 +18,14 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -297,19 +299,19 @@ public void testWidthAcceptsShortAndByte() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) .as("Function resolution should not work with zero arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (): Wrong number of inputs (expected width and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) .as("Function resolution should not work with only one argument") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int): Wrong number of inputs (expected width and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) .as("Function resolution should not work with more than two arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining( @@ -318,27 +320,27 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidTypesCannotBeUsedForWidth() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(CAST('12.34' as DECIMAL(9, 2)), 10)")) .as("Decimal type should not be coercible to the width field") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (decimal(9,2), int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) .as("String type should not be coercible to the width field") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (string, int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(INTERVAL '100-00' YEAR TO MONTH, 10)")) .as("Interval year to month type should not be coercible to the width field") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (interval year to month, int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "SELECT system.truncate(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) @@ -350,46 +352,44 @@ public void testInvalidTypesCannotBeUsedForWidth() { @Test public void testInvalidTypesForTruncationColumn() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) .as("FLoat type should not be truncatable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int, float): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) .as("Double type should not be truncatable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int, double): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) .as("Boolean type should not be truncatable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int, boolean): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) .as("Map types should not be truncatable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int, map): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) .as("Array types should not be truncatable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int, array): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(10, INTERVAL '100-00' YEAR TO MONTH)")) .as("Interval year-to-month type should not be truncatable") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'truncate' cannot process input: (int, interval year to month): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "SELECT system.truncate(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) @@ -405,13 +405,13 @@ public void testMagicFunctionsResolveForTinyIntAndSmallIntWidths() { // applyfunctionexpression instead. String tinyIntWidthExplain = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(1Y, 6)"); - Assertions.assertThat(tinyIntWidthExplain) + assertThat(tinyIntWidthExplain) .contains("cast(1 as int)") .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); String smallIntWidth = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5S, 6L)"); - Assertions.assertThat(smallIntWidth) + assertThat(smallIntWidth) .contains("cast(5 as int)") .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); @@ -423,50 +423,49 @@ public void testThatMagicFunctionsAreInvoked() { // Non-magic calls have `applyfunctionexpression` instead. // TinyInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateTinyInt"); // SmallInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateSmallInt"); // Int - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); // Long - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); // String - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateString"); // Decimal - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateDecimal"); // Binary - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) .asString() .isNotNull() .contains( diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java index 600b6ac0c1c5..3d2611c45f00 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -69,12 +71,12 @@ public void testTimestamps() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years()")) + assertThatThrownBy(() -> scalarSql("SELECT system.years()")) .as("Function resolution should not work with zero arguments") .isInstanceOf(AnalysisException.class) .hasMessageContaining("Function 'years' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.years(date('1969-12-31'), date('1969-12-31'))")) .as("Function resolution should not work with more than one argument") .isInstanceOf(AnalysisException.class) @@ -84,13 +86,13 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) .as("Int type should not be coercible to date/timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( "Function 'years' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) .as("Long type should not be coercible to date/timestamp") .isInstanceOf(AnalysisException.class) .hasMessageContaining( @@ -101,14 +103,14 @@ public void testInvalidInputTypes() { public void testThatMagicFunctionsAreInvoked() { String dateValue = "date('2017-12-01')"; String dateTransformClass = YearsFunction.DateToYearsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) .asString() .isNotNull() .contains("staticinvoke(class " + dateTransformClass); String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; String timestampTransformClass = YearsFunction.TimestampToYearsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) .asString() .isNotNull() .contains("staticinvoke(class " + timestampTransformClass); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index 5a8c5736a6b4..ba20a1e7bfed 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.sql.Timestamp; import java.util.Arrays; import java.util.List; @@ -36,7 +38,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -95,8 +96,7 @@ public void removeTables() { @Test public void testWriteTimestampWithoutZoneError() { - Assertions.assertThatThrownBy( - () -> sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values))) + assertThatThrownBy(() -> sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values))) .as( String.format( "Write operation performed on a timestamp without timezone field while " diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java index 1f5bee42af05..891813e3816e 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; -import org.assertj.core.api.Assertions; import org.junit.Test; public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase { @@ -53,7 +54,7 @@ protected String selectTarget() { @Test public void testInsertIntoNonExistingBranchFails() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("INSERT INTO %s.branch_not_exist VALUES (4, 'd'), (5, 'e')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): not_exist"); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java index 813e442c6604..06f5ba273b6f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -27,7 +29,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -97,7 +98,7 @@ public void testInsertAppendAtSnapshot() { Assume.assumeTrue(tableName.equals(commitTarget())); long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("INSERT INTO %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)) .as("Should not be able to insert into a table at a specific snapshot") @@ -110,7 +111,7 @@ public void testInsertOverwriteAtSnapshot() { Assume.assumeTrue(tableName.equals(commitTarget())); long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 3ed99da24947..676f7637cd09 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.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.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -47,7 +48,6 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -688,7 +688,7 @@ public void invalidDataImport() { createIcebergTable("id Integer, name String, dept String, subdept String"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", @@ -696,7 +696,7 @@ public void invalidDataImport() { .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot use partition filter with an unpartitioned table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`')", @@ -712,7 +712,7 @@ public void invalidDataImportPartitioned() { createIcebergTable( "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", @@ -721,7 +721,7 @@ public void invalidDataImportPartitioned() { .hasMessageStartingWith("Cannot add data files to target table") .hasMessageContaining("is greater than the number of partitioned columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", @@ -781,7 +781,7 @@ public void duplicateDataPartitioned() { + "partition_filter => map('id', 1))", catalogName, tableName, sourceTableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files(" @@ -840,7 +840,7 @@ public void duplicateDataUnpartitioned() { sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '%s')", diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java index 948fc462de99..2c109f1007d1 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTablePartitionFields.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -27,7 +29,6 @@ import org.apache.spark.sql.connector.catalog.CatalogManager; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -185,9 +186,7 @@ public void testAddYearPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD year(ts)", tableName); @@ -196,7 +195,7 @@ public void testAddYearPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).year("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test @@ -204,9 +203,7 @@ public void testAddMonthPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD month(ts)", tableName); @@ -215,7 +212,7 @@ public void testAddMonthPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).month("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test @@ -223,9 +220,7 @@ public void testAddDayPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD day(ts)", tableName); @@ -234,7 +229,7 @@ public void testAddDayPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).day("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test @@ -242,9 +237,7 @@ public void testAddHourPartition() { createTable("id bigint NOT NULL, category string, ts timestamp, data string"); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should start unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should start unpartitioned").isTrue(); sql("ALTER TABLE %s ADD PARTITION FIELD hour(ts)", tableName); @@ -253,7 +246,7 @@ public void testAddHourPartition() { PartitionSpec expected = PartitionSpec.builderFor(table.schema()).withSpecId(1).hour("ts").build(); - Assertions.assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); + assertThat(table.spec()).as("Should have new spec field").isEqualTo(expected); } @Test diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java index 7a6eb9aa387f..81bc1a59e144 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -77,12 +78,11 @@ public void testSetInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("not found in current schema or added columns"); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("not a required field"); } @@ -137,20 +137,17 @@ public void testDropInvalidIdentifierFields() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue( "Table should start without identifier", table.schema().identifierFieldIds().isEmpty()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot complete drop identifier fields operation: field unknown not found"); sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot complete drop identifier fields operation: data is not an identifier field"); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot complete drop identifier fields operation: location.lon is not an identifier field"); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index 3282cdf381ce..1aa3e17fce02 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -147,16 +148,15 @@ public void testAncestorOfUsingNamedArgs() { @Test public void testInvalidAncestorOfCases() { - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for parameter 'table'"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for snapshot_id: cannot cast"); } 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 a6bf194b3df5..6302c7d3f45b 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 @@ -19,6 +19,7 @@ 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; @@ -33,7 +34,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -85,8 +85,7 @@ public void testCreateBranch() throws NoSuchTableException { Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref b1 already exists"); } @@ -98,20 +97,20 @@ public void testCreateBranchOnEmptyTable() { Table table = validationCatalog.loadTable(tableIdent); SnapshotRef mainRef = table.refs().get(SnapshotRef.MAIN_BRANCH); - Assertions.assertThat(mainRef).isNull(); + assertThat(mainRef).isNull(); SnapshotRef ref = table.refs().get(branchName); - Assertions.assertThat(ref).isNotNull(); - Assertions.assertThat(ref.minSnapshotsToKeep()).isNull(); - Assertions.assertThat(ref.maxSnapshotAgeMs()).isNull(); - Assertions.assertThat(ref.maxRefAgeMs()).isNull(); + assertThat(ref).isNotNull(); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); Snapshot snapshot = table.snapshot(ref.snapshotId()); - Assertions.assertThat(snapshot.parentId()).isNull(); - Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); } @Test @@ -194,7 +193,7 @@ public void testCreateBranchUseCustomMinSnapshotsToKeepAndMaxSnapshotAge() Assert.assertEquals(TimeUnit.DAYS.toMillis(maxSnapshotAge), ref.maxSnapshotAgeMs().longValue()); Assert.assertNull(ref.maxRefAgeMs()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", @@ -216,18 +215,17 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { Assert.assertNull(ref.maxSnapshotAgeMs()); Assert.assertEquals(TimeUnit.DAYS.toMillis(maxRefAge), ref.maxRefAgeMs().longValue()); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", @@ -255,8 +253,7 @@ public void testDropBranch() throws NoSuchTableException { @Test public void testDropBranchDoesNotExist() { - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Branch does not exist: nonExistingBranch"); } @@ -267,21 +264,21 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref b1 is a tag not a branch"); } @Test public void testDropBranchNonConformingName() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); } @Test public void testDropMainBranchFails() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot remove main branch"); } @@ -330,20 +327,20 @@ public void testCreateOrReplaceBranchOnEmptyTable() { Table table = validationCatalog.loadTable(tableIdent); SnapshotRef mainRef = table.refs().get(SnapshotRef.MAIN_BRANCH); - Assertions.assertThat(mainRef).isNull(); + assertThat(mainRef).isNull(); SnapshotRef ref = table.refs().get(branchName); - Assertions.assertThat(ref).isNotNull(); - Assertions.assertThat(ref.minSnapshotsToKeep()).isNull(); - Assertions.assertThat(ref.maxSnapshotAgeMs()).isNull(); - Assertions.assertThat(ref.maxRefAgeMs()).isNull(); + assertThat(ref).isNotNull(); + assertThat(ref.minSnapshotsToKeep()).isNull(); + assertThat(ref.maxSnapshotAgeMs()).isNull(); + assertThat(ref.maxRefAgeMs()).isNull(); Snapshot snapshot = table.snapshot(ref.snapshotId()); - Assertions.assertThat(snapshot.parentId()).isNull(); - Assertions.assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); - Assertions.assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.parentId()).isNull(); + assertThat(snapshot.addedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDataFiles(table.io())).isEmpty(); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + assertThat(snapshot.removedDeleteFiles(table.io())).isEmpty(); } @Test @@ -386,7 +383,7 @@ public void replaceBranch() throws NoSuchTableException { public void replaceBranchDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", 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 5c41604afae1..a4630ab279e5 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.sql.Timestamp; import java.time.Instant; @@ -37,7 +39,6 @@ import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -144,7 +145,7 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - Assertions.assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("missing '(' at 'radish'"); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java index d82ed114556a..01523180b01f 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestChangelogTable.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -34,7 +35,6 @@ import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -202,8 +202,7 @@ public void testTimeRangeValidation() { Snapshot snap3 = table.currentSnapshot(); long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); - Assertions.assertThatThrownBy( - () -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) + assertThatThrownBy(() -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); } 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 c9af319f5c35..51762335a03b 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -161,7 +161,7 @@ public void testCherrypickSnapshotRefreshesRelationCache() { public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot cherry-pick unknown snapshot ID: -1"); @@ -169,27 +169,24 @@ public void testCherrypickInvalidSnapshot() { @Test public void testInvalidCherrypickSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.cherrypick_snapshot not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for snapshot_id: cannot cast"); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java index aaa275db9c16..b4366d93c73b 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.IsolationLevel; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.functions; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -68,7 +69,7 @@ public void testOverwriteFilterSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -106,7 +107,7 @@ public void testOverwriteFilterSerializableIsolation2() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -139,7 +140,7 @@ public void testOverwriteFilterSerializableIsolation3() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -171,7 +172,7 @@ public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { // Validating from no snapshot id defaults to beginning snapshot id and finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -208,7 +209,7 @@ public void testOverwriteFilterSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -256,7 +257,7 @@ public void testOverwritePartitionSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -292,7 +293,7 @@ public void testOverwritePartitionSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -326,7 +327,7 @@ public void testOverwritePartitionSnapshotIsolation2() throws Exception { spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -373,7 +374,7 @@ public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { // Validating from null snapshot is equivalent to validating from beginning Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java index 9ebe73da3323..f42769ce3c43 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -48,7 +49,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -156,7 +156,7 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(deleteFuture::get) + assertThatThrownBy(deleteFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index 6b6819a924e8..62aa3fefeadb 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Collections; import java.util.Map; @@ -46,7 +47,6 @@ import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -154,7 +154,7 @@ public synchronized void testMergeWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(mergeFuture::get) + assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java index 4354a1019c69..d90fde8a9aaa 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Map; import java.util.concurrent.ExecutionException; @@ -45,7 +46,6 @@ import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -152,7 +152,7 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(updateFuture::get) + assertThatThrownBy(updateFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java index e6114d4abcca..bd70243fa0d5 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestDelete.java @@ -30,6 +30,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; 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.Arrays; @@ -79,7 +80,6 @@ import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromTable; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -189,7 +189,7 @@ public void testCoalesceDelete() throws Exception { () -> { SparkPlan plan = executeAndKeepPlan("DELETE FROM %s WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -250,7 +250,7 @@ public void testSkewDelete() throws Exception { () -> { SparkPlan plan = executeAndKeepPlan("DELETE FROM %s WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -297,10 +297,10 @@ public void testDeleteWithoutScanningTable() throws Exception { LogicalPlan parsed = parsePlan("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); LogicalPlan analyzed = spark.sessionState().analyzer().execute(parsed); - Assertions.assertThat(analyzed).isInstanceOf(RowLevelWrite.class); + assertThat(analyzed).isInstanceOf(RowLevelWrite.class); LogicalPlan optimized = OptimizeMetadataOnlyDeleteFromTable.apply(analyzed); - Assertions.assertThat(optimized).isInstanceOf(DeleteFromTableWithFilters.class); + assertThat(optimized).isInstanceOf(DeleteFromTableWithFilters.class); }); sql("DELETE FROM %s WHERE dep = 'hr'", commitTarget()); @@ -409,7 +409,7 @@ public void testDeleteFromNonExistingCustomBranch() { Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); createAndInitUnpartitionedTable(); - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): test"); } @@ -616,8 +616,7 @@ public void testDeleteWithNonDeterministicCondition() { sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); createBranchIfNeeded(); - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("nondeterministic expressions are only allowed"); } @@ -906,7 +905,7 @@ public void testDeleteOnNonIcebergTableNotSupported() { sql("CREATE TABLE parquet_table (c1 INT, c2 INT) USING parquet"); - Assertions.assertThatThrownBy(() -> sql("DELETE FROM parquet_table WHERE c1 = -100")) + assertThatThrownBy(() -> sql("DELETE FROM parquet_table WHERE c1 = -100")) .isInstanceOf(AnalysisException.class) .hasMessageContaining("does not support DELETE"); } @@ -1114,7 +1113,7 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte }); try { - Assertions.assertThatThrownBy(deleteFuture::get) + assertThatThrownBy(deleteFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) @@ -1357,7 +1356,7 @@ public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableE withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage( String.format( @@ -1383,9 +1382,9 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), () -> { sql("DELETE FROM %s t WHERE id=1", tableName); - Assertions.assertThat(spark.table(tableName).count()).isEqualTo(2L); - Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); - Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + assertThat(spark.table(tableName).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(2L); + assertThat(spark.table(tableName + ".branch_main").count()) .as("Should not modify main branch") .isEqualTo(3L); }); @@ -1393,9 +1392,9 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, branch), () -> { sql("DELETE FROM %s t", tableName); - Assertions.assertThat(spark.table(tableName).count()).isEqualTo(0L); - Assertions.assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); - Assertions.assertThat(spark.table(tableName + ".branch_main").count()) + assertThat(spark.table(tableName).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_" + branch).count()).isEqualTo(0L); + assertThat(spark.table(tableName + ".branch_main").count()) .as("Should not modify main branch") .isEqualTo(3L); }); 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 7dacce5487d6..913d70f1ca91 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 @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.GC_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,7 +53,6 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -162,34 +163,30 @@ public void testExpireSnapshotsGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } @Test public void testInvalidExpireSnapshotsCases() { - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.expire_snapshots not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for older_than: cannot cast DecimalType(2,1) to TimestampType"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } @@ -209,7 +206,7 @@ public void testResolvingTableInAnotherCatalog() throws IOException { "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", anotherCatalog, tableIdent); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots('%s')", @@ -245,7 +242,7 @@ public void testConcurrentExpireSnapshots() { public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", @@ -253,7 +250,7 @@ public void testConcurrentExpireSnapshotsWithInvalidInput() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", @@ -392,7 +389,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertEquals("Should be 2 snapshots", 2, Iterables.size(table.snapshots())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(" @@ -471,21 +468,21 @@ public void testExpireSnapshotsWithStatisticFiles() throws Exception { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.statisticsFiles()) + assertThat(table.statisticsFiles()) .as( "Statistics file entry in TableMetadata should be present only for the snapshot %s", statisticsFile2.snapshotId()) .extracting(StatisticsFile::snapshotId) .containsExactly(statisticsFile2.snapshotId()); - Assertions.assertThat(new File(statsFileLocation1)) + assertThat(new File(statsFileLocation1)) .as("Statistics file should not exist for snapshot %s", statisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(statsFileLocation2)) + assertThat(new File(statsFileLocation2)) .as("Statistics file should exist for snapshot %s", statisticsFile2.snapshotId()) .exists(); } @@ -516,25 +513,23 @@ public void testExpireSnapshotsWithPartitionStatisticFiles() { sql( "CALL %s.system.expire_snapshots(older_than => TIMESTAMP '%s',table => '%s')", catalogName, currentTimestamp, tableIdent); - Assertions.assertThat(output.get(0)[5]) - .as("should be 1 deleted partition statistics file") - .isEqualTo(1L); + assertThat(output.get(0)[5]).as("should be 1 deleted partition statistics file").isEqualTo(1L); table.refresh(); - Assertions.assertThat(table.partitionStatisticsFiles()) + assertThat(table.partitionStatisticsFiles()) .as( "partition statistics file entry in TableMetadata should be present only for the snapshot %s", partitionStatisticsFile2.snapshotId()) .extracting(PartitionStatisticsFile::snapshotId) .containsExactly(partitionStatisticsFile2.snapshotId()); - Assertions.assertThat(new File(partitionStatsFileLocation1)) + assertThat(new File(partitionStatsFileLocation1)) .as( "partition statistics file should not exist for snapshot %s", partitionStatisticsFile1.snapshotId()) .doesNotExist(); - Assertions.assertThat(new File(partitionStatsFileLocation2)) + assertThat(new File(partitionStatsFileLocation2)) .as( "partition statistics file should exist for snapshot %s", partitionStatisticsFile2.snapshotId()) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java index bded60b956b5..e3e6e9700de8 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java @@ -28,6 +28,8 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; import static org.apache.spark.sql.functions.lit; +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.Arrays; @@ -67,7 +69,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -266,7 +267,7 @@ public void testSkewMerge() { + "WHEN MATCHED THEN " + " UPDATE SET salary = -1 ", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -786,7 +787,7 @@ public void testMergeWithOneMatchingBranchButMultipleSourceRowsForTargetRow() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -821,7 +822,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -862,7 +863,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { () -> { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -901,7 +902,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqua () -> { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -939,7 +940,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -973,7 +974,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1009,7 +1010,7 @@ public void testMergeWithMultipleUpdatesForTargetRow() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1080,7 +1081,7 @@ public void testMergeWithSingleConditionalDelete() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1486,7 +1487,7 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted }); try { - Assertions.assertThatThrownBy(mergeFuture::get) + assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) @@ -2223,7 +2224,7 @@ public void testMergeWithNonExistingColumns() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2234,7 +2235,7 @@ public void testMergeWithNonExistingColumns() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("cannot resolve t.invalid_col in MERGE command"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2245,7 +2246,7 @@ public void testMergeWithNonExistingColumns() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("No such struct field `invalid_col`"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2266,7 +2267,7 @@ public void testMergeWithInvalidColumnsInInsert() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2279,7 +2280,7 @@ public void testMergeWithInvalidColumnsInInsert() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Nested fields are not supported inside INSERT clauses"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2292,7 +2293,7 @@ public void testMergeWithInvalidColumnsInInsert() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Duplicate column names inside INSERT clause"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2311,7 +2312,7 @@ public void testMergeWithInvalidUpdates() { "{ \"id\": 1, \"a\": [ { \"c1\": 2, \"c2\": 3 } ], \"m\": { \"k\": \"v\"} }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2322,7 +2323,7 @@ public void testMergeWithInvalidUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for structs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2341,7 +2342,7 @@ public void testMergeWithConflictingUpdates() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2352,7 +2353,7 @@ public void testMergeWithConflictingUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updates are in conflict"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2363,7 +2364,7 @@ public void testMergeWithConflictingUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updates are in conflict for these columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2389,7 +2390,7 @@ public void testMergeWithInvalidAssignments() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2400,7 +2401,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2411,7 +2412,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2422,7 +2423,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("missing fields"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2433,7 +2434,7 @@ public void testMergeWithInvalidAssignments() { .isInstanceOf(AnalysisException.class) .hasMessageEndingWith("Cannot safely cast 'n1': string to int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2454,7 +2455,7 @@ public void testMergeWithNonDeterministicConditions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2466,7 +2467,7 @@ public void testMergeWithNonDeterministicConditions() { .hasMessageContaining( "Non-deterministic functions are not supported in SEARCH conditions of MERGE operations"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2478,7 +2479,7 @@ public void testMergeWithNonDeterministicConditions() { .hasMessageContaining( "Non-deterministic functions are not supported in UPDATE conditions of MERGE operations"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2490,7 +2491,7 @@ public void testMergeWithNonDeterministicConditions() { .hasMessageContaining( "Non-deterministic functions are not supported in DELETE conditions of MERGE operations"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2510,7 +2511,7 @@ public void testMergeWithAggregateExpressions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2522,7 +2523,7 @@ public void testMergeWithAggregateExpressions() { .hasMessageContaining( "Agg functions are not supported in SEARCH conditions of MERGE operations"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2534,7 +2535,7 @@ public void testMergeWithAggregateExpressions() { .hasMessageContaining( "Agg functions are not supported in UPDATE conditions of MERGE operations"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2546,7 +2547,7 @@ public void testMergeWithAggregateExpressions() { .hasMessageContaining( "Agg functions are not supported in DELETE conditions of MERGE operations"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2566,7 +2567,7 @@ public void testMergeWithSubqueriesInConditions() { "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2578,7 +2579,7 @@ public void testMergeWithSubqueriesInConditions() { .hasMessageContaining( "Subqueries are not supported in conditions of MERGE operations. Found a subquery in the SEARCH condition"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2590,7 +2591,7 @@ public void testMergeWithSubqueriesInConditions() { .hasMessageContaining( "Subqueries are not supported in conditions of MERGE operations. Found a subquery in the UPDATE condition"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2602,7 +2603,7 @@ public void testMergeWithSubqueriesInConditions() { .hasMessageContaining( "Subqueries are not supported in conditions of MERGE operations. Found a subquery in the DELETE condition"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2620,7 +2621,7 @@ public void testMergeWithTargetColumnsInInsertConditions() { createAndInitTable("id INT, c2 INT", "{ \"id\": 1, \"c2\": 2 }"); createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2637,7 +2638,7 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO target t USING source s " @@ -2701,7 +2702,7 @@ public void testMergeNonExistingBranch() { // Coalesce forces our source into a SinglePartition distribution spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s ON t.id = s.id " @@ -2788,7 +2789,7 @@ public void testMergeToWapBranchWithTableBranchIdentifier() { withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s ON t.id = s.id " @@ -2810,9 +2811,9 @@ private void checkJoinAndFilterConditions(String query, String join, String iceb SparkPlan sparkPlan = executeAndKeepPlan(() -> sql(query)); String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); - Assertions.assertThat(planAsString).as("Join should match").contains(join + "\n"); + assertThat(planAsString).as("Join should match").contains(join + "\n"); - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Pushed filters must match") .contains("[filters=" + icebergFilters + ","); }); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java index 91600d4df08d..5bec21392eba 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMergeOnReadDelete.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.Assertions.assertThatThrownBy; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -43,7 +44,6 @@ import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; import org.junit.runners.Parameterized; @@ -196,8 +196,7 @@ public void testCommitUnknownException() { TestSparkCatalog.setTable(ident, sparkTable); // Although an exception is thrown here, write and commit have succeeded - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java index e9013848cf11..17ed003f6a67 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMetaColumnProjectionWithStageScan.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import java.util.UUID; @@ -34,7 +36,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; import org.junit.runners.Parameterized; @@ -103,7 +104,7 @@ public void testReadStageTableMeta() throws Exception { .option(SparkReadOptions.SCAN_TASK_SET_ID, fileSetID) .load(tableLocation); - Assertions.assertThat(scanDF2.columns().length).isEqualTo(2); + assertThat(scanDF2.columns().length).isEqualTo(2); } try (CloseableIterable tasks = table.newBatchScan().planFiles()) { diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java index 0d23e4bbf068..7365f7a6df7b 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMigrateTableProcedure.java @@ -18,13 +18,15 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -137,9 +139,9 @@ public void testMigrateWithBackupTableName() throws IOException { "CALL %s.system.migrate(table => '%s', backup_table_name => '%s')", catalogName, tableName, backupTableName); - Assertions.assertThat(result).isEqualTo(1L); + assertThat(result).isEqualTo(1L); String dbName = tableName.split("\\.")[0]; - Assertions.assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); + assertThat(spark.catalog().tableExists(dbName + "." + backupTableName)).isTrue(); } @Test @@ -151,7 +153,7 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { String props = "map('write.metadata.metrics.column.x', 'X')"; sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); @@ -185,16 +187,15 @@ public void testMigrateWithConflictingProps() throws IOException { @Test public void testInvalidMigrateCases() { - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 94c4af3ad56d..fe0f12a9d0bb 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -31,7 +32,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -157,7 +157,7 @@ public void testApplyWapChangesRefreshesRelationCache() { public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot apply unknown WAP ID 'not_valid'"); @@ -165,23 +165,22 @@ public void testApplyInvalidWapId() { @Test public void testInvalidApplyWapChangesCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.publish_changes not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [wap_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 56854c561f5b..46d37ce20516 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 @@ -20,6 +20,8 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -64,7 +66,6 @@ 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.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -226,7 +227,7 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -257,26 +258,24 @@ public void testRemoveOrphanFilesWap() { @Test public void testInvalidRemoveOrphanFilesCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.remove_orphan_files not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for older_than"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } @@ -343,7 +342,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", @@ -351,7 +350,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", @@ -362,7 +361,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { String tempViewName = "file_list_test"; spark.emptyDataFrame().createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -375,7 +374,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -388,7 +387,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -505,10 +504,10 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) + assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); + assertThat(statsLocation.length()) .as("stats file length") .isEqualTo(statisticsFile.fileSizeInBytes()); @@ -522,11 +521,11 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly(statsLocation.toURI().toString()); - Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } @Test @@ -555,11 +554,9 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be no orphan files").isEmpty(); + assertThat(output).as("Should be no orphan files").isEmpty(); - Assertions.assertThat(new File(partitionStatsLocation)) - .as("partition stats file should exist") - .exists(); + assertThat(new File(partitionStatsLocation)).as("partition stats file should exist").exists(); removePartitionStatsTxn(table, partitionStatisticsFile); @@ -569,11 +566,11 @@ public void testRemoveOrphanFilesWithPartitionStatisticFiles() throws Exception + "table => '%s'," + "older_than => TIMESTAMP '%s')", catalogName, tableIdent, currentTimestamp); - Assertions.assertThat(output).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(output)) + assertThat(output).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(output)) .as("Deleted files") .containsExactly("file:" + partitionStatsLocation); - Assertions.assertThat(new File(partitionStatsLocation)) + assertThat(new File(partitionStatsLocation)) .as("partition stats file should be deleted") .doesNotExist(); } @@ -666,7 +663,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() Assert.assertEquals(0, orphanFiles.size()); // Test with no equal schemes - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(" @@ -749,7 +746,7 @@ public void testRemoveOrphanFilesProcedureWithEqualAuthorities() Assert.assertEquals(0, orphanFiles.size()); // Test with no equal authorities - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(" diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java index eb167ed25be4..443f299a49b3 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -74,7 +75,7 @@ public void testReplaceBranchFailsForTag() throws NoSuchTableException { df.writeTo(tableName).append(); long second = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", @@ -128,7 +129,7 @@ public void testReplaceBranchDoesNotExist() throws NoSuchTableException { df.writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java index 4c678ce9b767..6e5671bb1870 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.util.List; import java.util.Map; @@ -27,7 +29,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -186,7 +187,7 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> inputDF .writeTo(tableName) 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 009ab41228c8..24230d80b08d 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 @@ -19,6 +19,7 @@ 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.Arrays; import java.util.List; @@ -42,7 +43,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -598,7 +598,7 @@ public void testRewriteDataFilesWithInvalidInputs() { insertData(2); // Test for invalid strategy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " @@ -608,7 +608,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("unsupported strategy: temp. Only binpack or sort is supported"); // Test for sort_order with binpack strategy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " @@ -618,7 +618,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); // Test for sort strategy without any (default/user defined) sort_order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", @@ -627,7 +627,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageStartingWith("Cannot sort data without a valid sort order"); // Test for sort_order with invalid null order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -637,7 +637,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Unable to parse sortOrder: c1 ASC none"); // Test for sort_order with invalid sort direction - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -647,7 +647,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Unable to parse sortOrder: c1 none NULLS FIRST"); // Test for sort_order with invalid column name - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -657,7 +657,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageStartingWith("Cannot find field 'col1' in struct:"); // Test with invalid filter column col1 - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", @@ -666,7 +666,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Cannot parse predicates in where option: col1 = 3"); // Test for z_order with invalid column name - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -678,7 +678,7 @@ public void testRewriteDataFilesWithInvalidInputs() { + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test for z_order with sort_order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -691,26 +691,25 @@ public void testRewriteDataFilesWithInvalidInputs() { @Test public void testInvalidCasesForRewriteDataFiles() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rewrite_data_files not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageEndingWith("Duplicate procedure argument: table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for parameter 'table'"); } @@ -866,7 +865,7 @@ public void testDefaultSortOrder() { @Test public void testRewriteWithUntranslatedOrUnconvertedFilter() { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', where => 'substr(encode(c2, \"utf-8\"), 2) = \"fo\"')", @@ -874,7 +873,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', where => 'substr(c2, 2) = \"fo\"')", 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 8e64eaad44e7..3b8650de3454 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.sql.Date; import java.sql.Timestamp; @@ -31,7 +32,6 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -276,31 +276,29 @@ public void testRewriteManifestsCaseInsensitiveArgs() { @Test public void testInvalidRewriteManifestsCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rewrite_manifests not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for use_caching"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Could not build name to arg map: Duplicate procedure argument: table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java index 5dde5d698ee4..261dbcf7b7f3 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRewritePositionDeleteFilesProcedure.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.SnapshotSummary.ADDED_FILE_SIZE_PROP; import static org.apache.iceberg.SnapshotSummary.REMOVED_FILE_SIZE_PROP; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -29,7 +30,6 @@ import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.spark.source.SimpleRecord; import org.apache.spark.sql.Encoders; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -206,7 +206,7 @@ public void testInvalidOption() throws Exception { @Test public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(encode(data, \"utf-8\"), 2) = \"fo\"')", @@ -214,7 +214,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() throws Exception { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_position_delete_files(table => '%s', where => 'substr(data, 2) = \"fo\"')", 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 1b4cd2b4a3f6..3e9b324df75d 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Snapshot; @@ -28,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -240,7 +241,7 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot roll back to unknown snapshot id: -1"); @@ -248,7 +249,7 @@ public void testRollbackToInvalidSnapshot() { @Test public void testInvalidRollbackToSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", @@ -256,32 +257,27 @@ public void testInvalidRollbackToSnapshotCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rollback_to_snapshot not found"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for snapshot_id: cannot cast DecimalType(2,1) to LongType"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 80af6e7f5f4c..82df7567fce6 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.time.LocalDateTime; import java.util.List; import java.util.Map; @@ -28,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -255,7 +256,7 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", @@ -263,17 +264,16 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rollback_to_timestamp not found"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [timestamp]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp(timestamp => %s)", @@ -281,12 +281,11 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [timestamp]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", @@ -294,8 +293,7 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Too many arguments for procedure"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for timestamp: cannot cast DecimalType(2,1) to TimestampType"); } 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 e894ba4ff0ae..65db75f6086d 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.Map; @@ -29,7 +30,6 @@ 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.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assume; import org.junit.Test; @@ -189,7 +189,7 @@ public void tesSetCurrentSnapshotWithoutExplicitCatalog() { public void testSetCurrentSnapshotToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot roll back to unknown snapshot id: -1"); @@ -197,7 +197,7 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { @Test public void testInvalidRollbackToSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", @@ -205,41 +205,36 @@ public void testInvalidRollbackToSnapshotCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.set_current_snapshot not found"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Either snapshot_id or ref must be provided, not both"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse identifier for arg table: 1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Either snapshot_id or ref must be provided, not both"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for snapshot_id: cannot cast DecimalType(2,1) to LongType"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(table => 't', snapshot_id => 1L, ref => 's1')", @@ -284,7 +279,7 @@ public void testSetCurrentSnapshotToRef() { sql("SELECT * FROM %s ORDER BY id", tableName)); String notExistRef = "s2"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(table => '%s', ref => '%s')", diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java index caa9752004c2..4eaef55b693f 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSetWriteDistributionAndOrdering.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Map; import org.apache.iceberg.NullOrder; @@ -27,7 +28,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -75,7 +75,7 @@ public void testSetWriteOrderWithCaseSensitiveColumnNames() { Table table = validationCatalog.loadTable(tableIdent); Assert.assertTrue("Table should start unsorted", table.sortOrder().isUnsorted()); sql("SET %s=true", SQLConf.CASE_SENSITIVE().key()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); }) diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 367d95d8f619..bd4e1fb2f8f4 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -195,16 +196,16 @@ public void testInvalidSnapshotsCases() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for location"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", @@ -213,11 +214,11 @@ public void testInvalidSnapshotsCases() throws IOException { .hasMessageContaining( "The `map` requires 2n (n > 0) parameters but the actual number is 3"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument source_table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java index 9b9609ebb060..7b22fffb0bfa 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestStoragePartitionedJoinsInRowLevelOperations.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.RowLevelOperationMode.COPY_ON_WRITE; import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.commons.lang3.StringUtils; @@ -31,7 +32,6 @@ import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -138,7 +138,7 @@ private void checkDelete(RowLevelOperationMode mode) { + "dep = 'hr'", tableName, tableName(OTHER_TABLE_NAME)); String planAsString = plan.toString(); - Assertions.assertThat(planAsString).doesNotContain("Exchange"); + assertThat(planAsString).doesNotContain("Exchange"); }); ImmutableList expectedRows = @@ -201,7 +201,7 @@ private void checkUpdate(RowLevelOperationMode mode) { + "dep = 'hr'", tableName, tableName(OTHER_TABLE_NAME)); String planAsString = plan.toString(); - Assertions.assertThat(planAsString).doesNotContain("Exchange"); + assertThat(planAsString).doesNotContain("Exchange"); }); ImmutableList expectedRows = @@ -285,9 +285,9 @@ private void checkMerge(RowLevelOperationMode mode, boolean withPredicate) { if (mode == COPY_ON_WRITE) { int actualNumShuffles = StringUtils.countMatches(planAsString, "Exchange"); Assert.assertEquals("Should be 1 shuffle with SPJ", 1, actualNumShuffles); - Assertions.assertThat(planAsString).contains("Exchange hashpartitioning(_file"); + assertThat(planAsString).contains("Exchange hashpartitioning(_file"); } else { - Assertions.assertThat(planAsString).doesNotContain("Exchange"); + assertThat(planAsString).doesNotContain("Exchange"); } }); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java index 7f2857cce0b9..a63a3b65820a 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSystemFunctionPushDownDQL.java @@ -37,6 +37,7 @@ import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToHourOrdinal; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToMonthOrdinal; import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToYearOrdinal; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; import java.util.Map; @@ -49,7 +50,6 @@ import org.apache.spark.sql.catalyst.expressions.Expression; import org.apache.spark.sql.catalyst.expressions.objects.StaticInvoke; import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -107,7 +107,7 @@ private void testYearsFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, equal(year("ts"), targetYears)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual.size()).isEqualTo(5); } @Test @@ -135,7 +135,7 @@ private void testMonthsFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, greaterThan(month("ts"), targetMonths)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual.size()).isEqualTo(5); } @Test @@ -165,7 +165,7 @@ private void testDaysFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, lessThan(day("ts"), targetDays)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual.size()).isEqualTo(5); } @Test @@ -193,7 +193,7 @@ private void testHoursFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, greaterThanOrEqual(hour("ts"), targetHours)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(8); + assertThat(actual.size()).isEqualTo(8); } @Test @@ -221,7 +221,7 @@ private void testBucketLongFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, lessThanOrEqual(bucket("id", 5), target)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual.size()).isEqualTo(5); } @Test @@ -249,7 +249,7 @@ private void testBucketStringFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, notEqual(bucket("data", 5), target)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(8); + assertThat(actual.size()).isEqualTo(8); } @Test @@ -278,7 +278,7 @@ private void testTruncateFunction(boolean partitioned) { checkPushedFilters(optimizedPlan, equal(truncate("data", 4), target)); List actual = rowsToJava(df.collectAsList()); - Assertions.assertThat(actual.size()).isEqualTo(5); + assertThat(actual.size()).isEqualTo(5); } private void checkExpressions( @@ -286,18 +286,18 @@ private void checkExpressions( List staticInvokes = PlanUtils.collectSparkExpressions( optimizedPlan, expression -> expression instanceof StaticInvoke); - Assertions.assertThat(staticInvokes).isEmpty(); + assertThat(staticInvokes).isEmpty(); List applyExpressions = PlanUtils.collectSparkExpressions( optimizedPlan, expression -> expression instanceof ApplyFunctionExpression); if (partitioned) { - Assertions.assertThat(applyExpressions).isEmpty(); + assertThat(applyExpressions).isEmpty(); } else { - Assertions.assertThat(applyExpressions.size()).isEqualTo(1); + assertThat(applyExpressions.size()).isEqualTo(1); ApplyFunctionExpression expression = (ApplyFunctionExpression) applyExpressions.get(0); - Assertions.assertThat(expression.name()).isEqualTo(expectedFunctionName); + assertThat(expression.name()).isEqualTo(expectedFunctionName); } } @@ -305,9 +305,9 @@ private void checkPushedFilters( LogicalPlan optimizedPlan, org.apache.iceberg.expressions.Expression expected) { List pushedFilters = PlanUtils.collectPushDownFilters(optimizedPlan); - Assertions.assertThat(pushedFilters.size()).isEqualTo(1); + assertThat(pushedFilters.size()).isEqualTo(1); org.apache.iceberg.expressions.Expression actual = pushedFilters.get(0); - Assertions.assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)) + assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)) .as("Pushed filter should match") .isTrue(); } 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 52b9134089fb..b8673d0ae3bc 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 @@ -19,6 +19,7 @@ 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.Locale; @@ -34,7 +35,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -96,7 +96,7 @@ public void testCreateTagWithRetain() throws NoSuchTableException { } String tagName = "t1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", @@ -104,12 +104,12 @@ public void testCreateTagWithRetain() throws NoSuchTableException { .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d SECONDS", @@ -120,7 +120,7 @@ public void testCreateTagWithRetain() throws NoSuchTableException { @Test public void testCreateTagOnEmptyTable() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Cannot complete create or replace tag operation on %s, main has no snapshot", @@ -133,7 +133,7 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { long snapshotId = table.currentSnapshot().snapshotId(); String tagName = "t1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, -1)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot set " + tagName + " to unknown snapshot: -1"); @@ -146,11 +146,11 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { Assert.assertNull( "The tag needs to have the default max ref age, which is null.", ref.maxRefAgeMs()); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("already exists"); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); @@ -198,8 +198,7 @@ public void testReplaceTagFailsForBranch() throws NoSuchTableException { insertRows(); long second = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) + assertThatThrownBy(() -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref branch1 is a branch not a tag"); } @@ -234,7 +233,7 @@ public void testReplaceTag() throws NoSuchTableException { public void testReplaceTagDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", @@ -305,15 +304,14 @@ public void testDropTag() throws NoSuchTableException { @Test public void testDropTagNonConformingName() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); } @Test public void testDropTagDoesNotExist() { - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Tag does not exist: nonExistingTag"); } @@ -324,7 +322,7 @@ public void testDropTagFailesForBranch() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref b1 is a branch not a tag"); } 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 8b5e1cd1d41b..e84789de7abc 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 @@ -31,6 +31,8 @@ import static org.apache.iceberg.TableProperties.UPDATE_MODE; import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; import static org.apache.spark.sql.functions.lit; +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.Arrays; @@ -70,7 +72,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -169,7 +170,7 @@ public void testCoalesceUpdate() { () -> { SparkPlan plan = executeAndKeepPlan("UPDATE %s SET id = -1 WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -233,7 +234,7 @@ public void testSkewUpdate() { () -> { SparkPlan plan = executeAndKeepPlan("UPDATE %s SET id = -1 WHERE mod(id, 2) = 0", commitTarget()); - Assertions.assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); + assertThat(plan.toString()).contains("REBALANCE_PARTITIONS_BY_COL"); }); Table table = validationCatalog.loadTable(tableIdent); @@ -303,8 +304,7 @@ public void testUpdateNonExistingCustomBranch() { Assume.assumeTrue("Test only applicable to custom branch", "test".equals(branch)); createAndInitTable("id INT, dep STRING"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): test"); } @@ -693,7 +693,7 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte }); try { - Assertions.assertThatThrownBy(updateFuture::get) + assertThatThrownBy(updateFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) @@ -1315,11 +1315,11 @@ public void testUpdateWithInvalidUpdates() { "id INT, a ARRAY>, m MAP", "{ \"id\": 0, \"a\": null, \"m\": null }"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for structs"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for structs"); } @@ -1329,17 +1329,16 @@ public void testUpdateWithConflictingAssignments() { createAndInitTable( "id INT, c STRUCT>", "{ \"id\": 0, \"s\": null }"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Updates are in conflict for these columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Updates are in conflict for these columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "UPDATE %s SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", @@ -1358,26 +1357,24 @@ public void testUpdateWithInvalidAssignments() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", policy), () -> { - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Cannot write nullable values to non-null column"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Cannot write incompatible data:"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "UPDATE %s t SET t.s.n2 = named_struct('dn3', 1, 'dn1', 2)", @@ -1392,7 +1389,7 @@ public void testUpdateWithInvalidAssignments() { public void testUpdateWithNonDeterministicCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s SET id = -1 WHERE id = 1 AND rand() > 0.5", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( @@ -1403,7 +1400,7 @@ public void testUpdateWithNonDeterministicCondition() { public void testUpdateOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) + assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("UPDATE TABLE is not supported temporarily."); } @@ -1467,8 +1464,7 @@ public void testUpdateToWapBranchWithTableBranchIdentifier() { withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage( String.format( diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index 15484f45f895..a58dcb5d5eea 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.hadoop.conf.Configuration; @@ -37,7 +39,6 @@ import org.apache.spark.SparkException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Rule; import org.junit.Test; @@ -104,7 +105,7 @@ public void testBatchAppend() throws Exception { new SimpleRecord(4, "b")); Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> // incoming records are not ordered by partitions so the job must fail inputDF diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 4704bcbbdac5..59bc70854cbe 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.spark.extensions.SparkExtensionsTestBase; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -170,7 +171,7 @@ public void testCreateTable() { public void showView() { sql("DROP VIEW IF EXISTS %s", "test"); sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); - Assertions.assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); } private Table getTable(String name) { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java index 33b5316b72b7..2a7d3d4c331f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestDataFileSerialization.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TaskCheckHelper.assertEquals; 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 com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; @@ -48,7 +49,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.catalyst.InternalRow; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -118,7 +118,7 @@ public void testDataFileKryoSerialization() throws Exception { try (Input in = new Input(new FileInputStream(data))) { for (int i = 0; i < 2; i += 1) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); assertEquals(DATA_FILE, (DataFile) obj); } } @@ -136,7 +136,7 @@ public void testDataFileJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 2; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); assertEquals(DATA_FILE, (DataFile) obj); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java index 92a646d3861b..a42781f95282 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestManifestFileSerialization.java @@ -20,6 +20,7 @@ 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 com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; @@ -43,7 +44,6 @@ import org.apache.iceberg.types.Types; import org.apache.spark.SparkConf; import org.apache.spark.serializer.KryoSerializer; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -119,7 +119,7 @@ public void testManifestFileKryoSerialization() throws IOException { try (Input in = new Input(new FileInputStream(data))) { for (int i = 0; i < 3; i += 1) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); checkManifestFile(manifest, (ManifestFile) obj); } } @@ -141,7 +141,7 @@ public void testManifestFileJavaSerialization() throws Exception { new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { for (int i = 0; i < 3; i += 1) { Object obj = in.readObject(); - Assertions.assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); checkManifestFile(manifest, (ManifestFile) obj); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java index 281eb2543aee..b9e4b04953d5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/TestScanTaskSerialization.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import com.esotericsoftware.kryo.Kryo; import com.esotericsoftware.kryo.io.Input; @@ -46,7 +47,6 @@ import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -86,7 +86,7 @@ public void testBaseCombinedScanTaskKryoSerialization() throws Exception { try (Input in = new Input(new FileInputStream(data))) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj) + assertThat(obj) .as("Should be a BaseCombinedScanTask") .isInstanceOf(BaseCombinedScanTask.class); TaskCheckHelper.assertEquals(scanTask, (BaseCombinedScanTask) obj); @@ -105,7 +105,7 @@ public void testBaseCombinedScanTaskJavaSerialization() throws Exception { try (ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { Object obj = in.readObject(); - Assertions.assertThat(obj) + assertThat(obj) .as("Should be a BaseCombinedScanTask") .isInstanceOf(BaseCombinedScanTask.class); TaskCheckHelper.assertEquals(scanTask, (BaseCombinedScanTask) obj); @@ -129,9 +129,7 @@ public void testBaseScanTaskGroupKryoSerialization() throws Exception { try (Input in = new Input(Files.newInputStream(data.toPath()))) { Object obj = kryo.readClassAndObject(in); - Assertions.assertThat(obj) - .as("should be a BaseScanTaskGroup") - .isInstanceOf(BaseScanTaskGroup.class); + assertThat(obj).as("should be a BaseScanTaskGroup").isInstanceOf(BaseScanTaskGroup.class); TaskCheckHelper.assertEquals(taskGroup, (BaseScanTaskGroup) obj); } } @@ -151,9 +149,7 @@ public void testBaseScanTaskGroupJavaSerialization() throws Exception { try (ObjectInputStream in = new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { Object obj = in.readObject(); - Assertions.assertThat(obj) - .as("should be a BaseScanTaskGroup") - .isInstanceOf(BaseScanTaskGroup.class); + assertThat(obj).as("should be a BaseScanTaskGroup").isInstanceOf(BaseScanTaskGroup.class); TaskCheckHelper.assertEquals(taskGroup, (BaseScanTaskGroup) obj); } } 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 70ab04f0a080..b0b3085dca70 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 @@ -18,11 +18,12 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Arrays; import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; public class ValidationHelpers { @@ -72,6 +73,6 @@ public static void validateDataManifest( private static void assertSameElements(String context, List actual, List expected) { String errorMessage = String.format("%s must match", context); - Assertions.assertThat(actual).as(errorMessage).hasSameElementsAs(expected); + assertThat(actual).as(errorMessage).hasSameElementsAs(expected); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java index cad3478f3eba..b51861e4a2b3 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestFunctionCatalog.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.IcebergBuild; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.functions.IcebergVersionFunction; @@ -27,7 +30,6 @@ import org.apache.spark.sql.connector.catalog.FunctionCatalog; import org.apache.spark.sql.connector.catalog.Identifier; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -56,10 +58,10 @@ public void dropDefaultNamespace() { @Test public void testListFunctionsViaCatalog() throws NoSuchNamespaceException { - Assertions.assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE)) + assertThat(asFunctionCatalog.listFunctions(EMPTY_NAMESPACE)) .anyMatch(func -> "iceberg_version".equals(func.name())); - Assertions.assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE)) + assertThat(asFunctionCatalog.listFunctions(SYSTEM_NAMESPACE)) .anyMatch(func -> "iceberg_version".equals(func.name())); Assert.assertArrayEquals( @@ -67,7 +69,7 @@ public void testListFunctionsViaCatalog() throws NoSuchNamespaceException { new Identifier[0], asFunctionCatalog.listFunctions(DEFAULT_NAMESPACE)); - Assertions.assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE)) + assertThatThrownBy(() -> asFunctionCatalog.listFunctions(DB_NAMESPACE)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageStartingWith("[SCHEMA_NOT_FOUND] The schema `db` cannot be found."); } @@ -78,13 +80,13 @@ public void testLoadFunctions() throws NoSuchFunctionException { Identifier identifier = Identifier.of(namespace, "iceberg_version"); UnboundFunction func = asFunctionCatalog.loadFunction(identifier); - Assertions.assertThat(func) + assertThat(func) .isNotNull() .isInstanceOf(UnboundFunction.class) .isExactlyInstanceOf(IcebergVersionFunction.class); } - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> asFunctionCatalog.loadFunction(Identifier.of(DEFAULT_NAMESPACE, "iceberg_version"))) .isInstanceOf(NoSuchFunctionException.class) @@ -92,12 +94,12 @@ public void testLoadFunctions() throws NoSuchFunctionException { String.format("Cannot load function: %s.default.iceberg_version", catalogName)); Identifier undefinedFunction = Identifier.of(SYSTEM_NAMESPACE, "undefined_function"); - Assertions.assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) + assertThatThrownBy(() -> asFunctionCatalog.loadFunction(undefinedFunction)) .isInstanceOf(NoSuchFunctionException.class) .hasMessageStartingWith( String.format("Cannot load function: %s.system.undefined_function", catalogName)); - Assertions.assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) + assertThatThrownBy(() -> sql("SELECT undefined_function(1, 2)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "[UNRESOLVED_ROUTINE] Cannot resolve function `undefined_function` on search path"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java index ce11f0c05ffd..744660073954 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSpark3Util.java @@ -36,6 +36,7 @@ import static org.apache.iceberg.expressions.Expressions.year; 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 org.apache.iceberg.CachingCatalog; import org.apache.iceberg.Schema; @@ -45,7 +46,6 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -141,30 +141,29 @@ public void testLoadIcebergCatalog() throws Exception { @Test public void testDescribeExpression() { Expression refExpression = equal("id", 1); - Assertions.assertThat(Spark3Util.describe(refExpression)).isEqualTo("id = 1"); + assertThat(Spark3Util.describe(refExpression)).isEqualTo("id = 1"); Expression yearExpression = greaterThan(year("ts"), 10); - Assertions.assertThat(Spark3Util.describe(yearExpression)).isEqualTo("year(ts) > 10"); + assertThat(Spark3Util.describe(yearExpression)).isEqualTo("year(ts) > 10"); Expression monthExpression = greaterThanOrEqual(month("ts"), 10); - Assertions.assertThat(Spark3Util.describe(monthExpression)).isEqualTo("month(ts) >= 10"); + assertThat(Spark3Util.describe(monthExpression)).isEqualTo("month(ts) >= 10"); Expression dayExpression = lessThan(day("ts"), 10); - Assertions.assertThat(Spark3Util.describe(dayExpression)).isEqualTo("day(ts) < 10"); + assertThat(Spark3Util.describe(dayExpression)).isEqualTo("day(ts) < 10"); Expression hourExpression = lessThanOrEqual(hour("ts"), 10); - Assertions.assertThat(Spark3Util.describe(hourExpression)).isEqualTo("hour(ts) <= 10"); + assertThat(Spark3Util.describe(hourExpression)).isEqualTo("hour(ts) <= 10"); Expression bucketExpression = in(bucket("id", 5), 3); - Assertions.assertThat(Spark3Util.describe(bucketExpression)).isEqualTo("bucket[5](id) IN (3)"); + assertThat(Spark3Util.describe(bucketExpression)).isEqualTo("bucket[5](id) IN (3)"); Expression truncateExpression = notIn(truncate("name", 3), "abc"); - Assertions.assertThat(Spark3Util.describe(truncateExpression)) + assertThat(Spark3Util.describe(truncateExpression)) .isEqualTo("truncate[3](name) NOT IN ('abc')"); Expression andExpression = and(refExpression, yearExpression); - Assertions.assertThat(Spark3Util.describe(andExpression)) - .isEqualTo("(id = 1 AND year(ts) > 10)"); + assertThat(Spark3Util.describe(andExpression)).isEqualTo("(id = 1 AND year(ts) > 10)"); } private SortOrder buildSortOrder(String transform, Schema schema, int sourceId) { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java index 1e51caadd0e9..51c8d9643c11 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkTableUtil.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.KryoHelpers; @@ -27,7 +29,6 @@ import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.spark.SparkTableUtil.SparkPartition; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -40,7 +41,7 @@ public void testSparkPartitionOKryoSerialization() throws IOException { SparkPartition sparkPartition = new SparkPartition(values, uri, format); SparkPartition deserialized = KryoHelpers.roundTripSerialize(sparkPartition); - Assertions.assertThat(sparkPartition).isEqualTo(deserialized); + assertThat(sparkPartition).isEqualTo(deserialized); } @Test @@ -51,7 +52,7 @@ public void testSparkPartitionJavaSerialization() throws IOException, ClassNotFo SparkPartition sparkPartition = new SparkPartition(values, uri, format); SparkPartition deserialized = TestHelpers.roundTripSerialize(sparkPartition); - Assertions.assertThat(sparkPartition).isEqualTo(deserialized); + assertThat(sparkPartition).isEqualTo(deserialized); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java index 2a40e6e50fc5..eae8a6ec02da 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/TestSparkV2Filters.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.time.Instant; import java.time.LocalDate; import java.time.temporal.ChronoUnit; @@ -49,7 +52,6 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.apache.spark.unsafe.types.UTF8String; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -238,23 +240,23 @@ public void testEqualToNull() { new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; Predicate eq1 = new Predicate("=", attrAndValue); - Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(eq1)) + assertThatThrownBy(() -> SparkV2Filters.convert(eq1)) .isInstanceOf(NullPointerException.class) .hasMessageContaining("Expression is always false"); Predicate eq2 = new Predicate("=", valueAndAttr); - Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(eq2)) + assertThatThrownBy(() -> SparkV2Filters.convert(eq2)) .isInstanceOf(NullPointerException.class) .hasMessageContaining("Expression is always false"); Predicate eqNullSafe1 = new Predicate("<=>", attrAndValue); Expression expectedEqNullSafe = Expressions.isNull(col); Expression actualEqNullSafe1 = SparkV2Filters.convert(eqNullSafe1); - Assertions.assertThat(actualEqNullSafe1.toString()).isEqualTo(expectedEqNullSafe.toString()); + assertThat(actualEqNullSafe1.toString()).isEqualTo(expectedEqNullSafe.toString()); Predicate eqNullSafe2 = new Predicate("<=>", valueAndAttr); Expression actualEqNullSafe2 = SparkV2Filters.convert(eqNullSafe2); - Assertions.assertThat(actualEqNullSafe2.toString()).isEqualTo(expectedEqNullSafe.toString()); + assertThat(actualEqNullSafe2.toString()).isEqualTo(expectedEqNullSafe.toString()); } @Test @@ -271,11 +273,11 @@ public void testEqualToNaN() { Predicate eqNaN1 = new Predicate("=", attrAndValue); Expression expectedEqNaN = Expressions.isNaN(col); Expression actualEqNaN1 = SparkV2Filters.convert(eqNaN1); - Assertions.assertThat(actualEqNaN1.toString()).isEqualTo(expectedEqNaN.toString()); + assertThat(actualEqNaN1.toString()).isEqualTo(expectedEqNaN.toString()); Predicate eqNaN2 = new Predicate("=", valueAndAttr); Expression actualEqNaN2 = SparkV2Filters.convert(eqNaN2); - Assertions.assertThat(actualEqNaN2.toString()).isEqualTo(expectedEqNaN.toString()); + assertThat(actualEqNaN2.toString()).isEqualTo(expectedEqNaN.toString()); } @Test @@ -290,12 +292,12 @@ public void testNotEqualToNull() { new org.apache.spark.sql.connector.expressions.Expression[] {value, namedReference}; Predicate notEq1 = new Predicate("<>", attrAndValue); - Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(notEq1)) + assertThatThrownBy(() -> SparkV2Filters.convert(notEq1)) .isInstanceOf(NullPointerException.class) .hasMessageContaining("Expression is always false"); Predicate notEq2 = new Predicate("<>", valueAndAttr); - Assertions.assertThatThrownBy(() -> SparkV2Filters.convert(notEq2)) + assertThatThrownBy(() -> SparkV2Filters.convert(notEq2)) .isInstanceOf(NullPointerException.class) .hasMessageContaining("Expression is always false"); } @@ -314,11 +316,11 @@ public void testNotEqualToNaN() { Predicate notEqNaN1 = new Predicate("<>", attrAndValue); Expression expectedNotEqNaN = Expressions.notNaN(col); Expression actualNotEqNaN1 = SparkV2Filters.convert(notEqNaN1); - Assertions.assertThat(actualNotEqNaN1.toString()).isEqualTo(expectedNotEqNaN.toString()); + assertThat(actualNotEqNaN1.toString()).isEqualTo(expectedNotEqNaN.toString()); Predicate notEqNaN2 = new Predicate("<>", valueAndAttr); Expression actualNotEqNaN2 = SparkV2Filters.convert(notEqNaN2); - Assertions.assertThat(actualNotEqNaN2.toString()).isEqualTo(expectedNotEqNaN.toString()); + assertThat(actualNotEqNaN2.toString()).isEqualTo(expectedNotEqNaN.toString()); } @Test @@ -630,7 +632,7 @@ public void testUnsupportedUDFConvert() { Predicate predicate = new Predicate("=", expressions(udf, literalValue)); Expression icebergExpr = SparkV2Filters.convert(predicate); - Assertions.assertThat(icebergExpr).isNull(); + assertThat(icebergExpr).isNull(); } private void testUDF( @@ -746,7 +748,7 @@ private void testUDF( Predicate invalid = new Predicate("<", attrAndAttr); Predicate andWithInvalidLeft = new And(invalid, eq1); Expression convertedAnd = SparkV2Filters.convert(andWithInvalidLeft); - Assertions.assertThat(convertedAnd).isNull(); + assertThat(convertedAnd).isNull(); Predicate or = new Or(lt1, eq1); Expression expectedOr = Expressions.or(expectedLt1, expectedEq1); @@ -755,7 +757,7 @@ private void testUDF( Predicate orWithInvalidLeft = new Or(invalid, eq1); Expression convertedOr = SparkV2Filters.convert(orWithInvalidLeft); - Assertions.assertThat(convertedOr).isNull(); + assertThat(convertedOr).isNull(); Predicate not = new Not(lt1); Expression expectedNot = Expressions.not(expectedLt1); @@ -764,7 +766,7 @@ private void testUDF( } private static void assertEquals(Expression expected, Expression actual) { - Assertions.assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)).isTrue(); + assertThat(ExpressionUtil.equivalent(expected, actual, STRUCT, true)).isTrue(); } private org.apache.spark.sql.connector.expressions.Expression[] expressions( 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 384405265500..97843c152811 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.util.Set; @@ -50,7 +51,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.SparkTestBase; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -363,7 +363,7 @@ public void testEmptyIOThrowsException() { DeleteReachableFiles baseRemoveFilesSparkAction = sparkActions().deleteReachableFiles(metadataLocation(table)).io(null); - Assertions.assertThatThrownBy(baseRemoveFilesSparkAction::execute) + assertThatThrownBy(baseRemoveFilesSparkAction::execute) .isInstanceOf(IllegalArgumentException.class) .hasMessage("File IO cannot be null"); } @@ -372,8 +372,7 @@ public void testEmptyIOThrowsException() { public void testRemoveFilesActionWhenGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - Assertions.assertThatThrownBy( - () -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) + assertThatThrownBy(() -> sparkActions().deleteReachableFiles(metadataLocation(table)).execute()) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot delete files: GC is disabled (deleting files may corrupt other tables)"); 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 d5ac1b80649a..7a74329f3550 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -56,7 +57,6 @@ import org.apache.iceberg.spark.data.TestHelpers; import org.apache.iceberg.types.Types; import org.apache.spark.sql.Dataset; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -447,7 +447,7 @@ public void testExpireSnapshotsWithDisabledGarbageCollection() { table.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) + assertThatThrownBy(() -> SparkActions.get().expireSnapshots(table)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot expire snapshots: GC is disabled (deleting files may corrupt other tables)"); @@ -528,8 +528,7 @@ public void testRetainLastMultipleCalls() { @Test public void testRetainZeroSnapshots() { - Assertions.assertThatThrownBy( - () -> SparkActions.get().expireSnapshots(table).retainLast(0).execute()) + 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"); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java index a0caacd6d246..60bc9ca60f06 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRemoveOrphanFilesAction.java @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.actions; 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 java.io.File; import java.io.IOException; @@ -78,7 +80,6 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -750,7 +751,7 @@ public void testGarbageCollectionDisabled() { table.updateProperties().set(TableProperties.GC_ENABLED, "false").commit(); - Assertions.assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) + assertThatThrownBy(() -> SparkActions.get().deleteOrphanFiles(table).execute()) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot delete orphan files: GC is disabled (deleting files may corrupt other tables)"); @@ -938,8 +939,8 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .olderThan(System.currentTimeMillis() + 1000) .execute(); - Assertions.assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); - Assertions.assertThat(statsLocation.length()) + assertThat(statsLocation.exists()).as("stats file should exist").isTrue(); + assertThat(statsLocation.length()) .as("stats file length") .isEqualTo(statisticsFile.fileSizeInBytes()); @@ -953,11 +954,11 @@ public void testRemoveOrphanFilesWithStatisticFiles() throws Exception { .olderThan(System.currentTimeMillis() + 1000) .execute(); Iterable orphanFileLocations = result.orphanFileLocations(); - Assertions.assertThat(orphanFileLocations).as("Should be orphan files").hasSize(1); - Assertions.assertThat(Iterables.getOnlyElement(orphanFileLocations)) + assertThat(orphanFileLocations).as("Should be orphan files").hasSize(1); + assertThat(Iterables.getOnlyElement(orphanFileLocations)) .as("Deleted file") .isEqualTo(statsLocation.toURI().toString()); - Assertions.assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); + assertThat(statsLocation.exists()).as("stats file should be deleted").isFalse(); } @Test @@ -985,7 +986,7 @@ public void testPathsWithActualFileHavingNoAuthority() { public void testPathsWithEqualSchemes() { List validFiles = Lists.newArrayList("scheme1://bucket1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("scheme2://bucket1/dir1/dir2/file1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> executeTest( validFiles, @@ -1014,7 +1015,7 @@ public void testPathsWithEqualSchemes() { public void testPathsWithEqualAuthorities() { List validFiles = Lists.newArrayList("hdfs://servicename1/dir1/dir2/file1"); List actualFiles = Lists.newArrayList("hdfs://servicename2/dir1/dir2/file1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> executeTest( validFiles, 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 0eea49532346..66f16437f5dc 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 @@ -25,6 +25,7 @@ import static org.apache.spark.sql.functions.date_add; 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.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.argThat; import static org.mockito.Mockito.doAnswer; @@ -107,7 +108,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.BeforeClass; @@ -677,7 +677,7 @@ public void testSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - Assertions.assertThatThrownBy(spyRewrite::execute) + assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) .hasMessage("Rewrite Failed"); @@ -711,7 +711,7 @@ public void testSingleCommitWithCommitFailure() { doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); - Assertions.assertThatThrownBy(spyRewrite::execute) + assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) .hasMessage("Commit Failure"); @@ -746,7 +746,7 @@ public void testParallelSingleCommitWithRewriteFailure() { .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); - Assertions.assertThatThrownBy(spyRewrite::execute) + assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) .hasMessage("Rewrite Failed"); @@ -894,7 +894,7 @@ public void testParallelPartialProgressWithCommitFailure() { public void testInvalidOptions() { Table table = createTable(20); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .option(RewriteDataFiles.PARTIAL_PROGRESS_ENABLED, "true") @@ -905,7 +905,7 @@ public void testInvalidOptions() { "Cannot set partial-progress.max-commits to -5, " + "the value must be positive when partial-progress.enabled is true"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .option(RewriteDataFiles.MAX_CONCURRENT_FILE_GROUP_REWRITES, "-5") @@ -914,17 +914,17 @@ public void testInvalidOptions() { .hasMessage( "Cannot set max-concurrent-file-group-rewrites to -5, the value must be positive."); - Assertions.assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) + assertThatThrownBy(() -> basicRewrite(table).option("foobarity", "-5").execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot use options [foobarity], they are not supported by the action or the rewriter BIN-PACK"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table).option(RewriteDataFiles.REWRITE_JOB_ORDER, "foo").execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid rewrite job order name: foo"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> basicRewrite(table) .sort(SortOrder.builderFor(table.schema()).asc("c2").build()) @@ -1167,7 +1167,7 @@ public void testCommitStateUnknownException() { doReturn(util).when(spyAction).commitManager(table.currentSnapshot().snapshotId()); - Assertions.assertThatThrownBy(spyAction::execute) + assertThatThrownBy(spyAction::execute) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith( "Unknown State\n" + "Cannot determine whether the commit was successful or not"); @@ -1286,15 +1286,15 @@ public void testInvalidAPIUsage() { SortOrder sortOrder = SortOrder.builderFor(table.schema()).asc("c2").build(); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).binPack().sort()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); - Assertions.assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) + assertThatThrownBy(() -> actions().rewriteDataFiles(table).sort(sortOrder).binPack()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); } @@ -1476,7 +1476,7 @@ public void testBinPackRewriterWithSpecificOutputSpec() { public void testBinpackRewriteWithInvalidOutputSpecId() { Table table = createTable(10); shouldHaveFiles(table, 10); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> actions() .rewriteDataFiles(table) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java index e7326c73e838..bdc830e94610 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestRewriteManifestsAction.java @@ -25,6 +25,7 @@ import static org.apache.iceberg.ValidationHelpers.validateDataManifest; 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 static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; @@ -75,7 +76,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.TableIdentifier; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -248,7 +248,7 @@ public void testRewriteManifestsWithCommitStateUnknownException() { Table spyTable = spy(table); when(spyTable.rewriteManifests()).thenReturn(spyNewRewriteManifests); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> actions.rewriteManifests(spyTable).rewriteIf(manifest -> true).execute()) .cause() .isInstanceOf(RuntimeException.class) @@ -459,14 +459,14 @@ public void testRewriteLargeManifestsPartitionedTable() throws IOException { .stagingLocation(stagingLocation) .execute(); - Assertions.assertThat(result.rewrittenManifests()).hasSize(1); - Assertions.assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); + assertThat(result.rewrittenManifests()).hasSize(1); + assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); assertManifestsLocation(result.addedManifests(), stagingLocation); table.refresh(); List newManifests = table.currentSnapshot().allManifests(table.io()); - Assertions.assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); + assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java index 055e5be681c6..9722b40f2c45 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.actions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.FileScanTask; @@ -37,7 +39,6 @@ import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StringType; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -199,16 +200,15 @@ private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { public void testInvalidConstructorUsagesSortData() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - Assertions.assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("is unsorted and no sort order is provided"); - Assertions.assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("the provided sort order is null or empty"); - Assertions.assertThatThrownBy( - () -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("the provided sort order is null or empty"); } @@ -217,20 +217,17 @@ public void testInvalidConstructorUsagesSortData() { public void testInvalidConstructorUsagesZOrderData() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA, SPEC); - Assertions.assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) .hasMessageContaining("Cannot ZOrder when no columns are specified"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) .hasMessageContaining("Cannot ZOrder when no columns are specified"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) .hasMessageContaining("Cannot ZOrder") .hasMessageContaining("all columns provided were identity partition columns"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) .hasMessageContaining("Cannot ZOrder") .hasMessageContaining("all columns provided were identity partition columns"); } @@ -305,7 +302,7 @@ public void testInvalidValuesForBinPackDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); } @@ -318,12 +315,12 @@ public void testInvalidValuesForSortDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); Map invalidCompressionFactorOptions = ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) + assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); } @@ -337,23 +334,23 @@ public void testInvalidValuesForZOrderDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); Map invalidCompressionFactorOptions = ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) + assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); Map invalidMaxOutputOptions = ImmutableMap.of(SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) .hasMessageContaining("Cannot have the interleaved ZOrder value use less than 1 byte") .hasMessageContaining("'max-output-size' was set to 0"); Map invalidVarLengthContributionOptions = ImmutableMap.of(SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) + assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) .hasMessageContaining("Cannot use less than 1 byte for variable length types with ZOrder") .hasMessageContaining("'var-length-contribution' was set to 0"); } @@ -361,19 +358,19 @@ public void testInvalidValuesForZOrderDataOptions() { private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewriter) { Map invalidTargetSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) .hasMessageContaining("'target-file-size-bytes' is set to 0 but must be > 0"); Map invalidMinSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) .hasMessageContaining("'min-file-size-bytes' is set to -1 but must be >= 0"); Map invalidTargetMinSizeOptions = ImmutableMap.of( SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "3", SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "5"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) .hasMessageContaining("'target-file-size-bytes' (3) must be > 'min-file-size-bytes' (5)") .hasMessageContaining("all new files will be smaller than the min threshold"); @@ -381,18 +378,18 @@ private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewrit ImmutableMap.of( SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "5", SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, "3"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) .hasMessageContaining("'target-file-size-bytes' (5) must be < 'max-file-size-bytes' (3)") .hasMessageContaining("all new files will be larger than the max threshold"); Map invalidMinInputFilesOptions = ImmutableMap.of(SizeBasedFileRewriter.MIN_INPUT_FILES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) .hasMessageContaining("'min-input-files' is set to 0 but must be > 0"); Map invalidMaxFileGroupSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.MAX_FILE_GROUP_SIZE_BYTES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) .hasMessageContaining("'max-file-group-size-bytes' is set to 0 but must be > 0"); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java index a94d6525a7cf..1bad42f450fb 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/GenericsHelpers.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.data; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.assertj.core.api.Assertions.assertThat; import static scala.collection.JavaConverters.mapAsJavaMapConverter; import static scala.collection.JavaConverters.seqAsJavaListConverter; @@ -47,7 +48,6 @@ import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.UTF8String; -import org.assertj.core.api.Assertions; import org.junit.Assert; import scala.collection.Seq; @@ -119,10 +119,8 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case DATE: - Assertions.assertThat(expected) - .as("Should expect a LocalDate") - .isInstanceOf(LocalDate.class); - Assertions.assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); + assertThat(expected).as("Should expect a LocalDate").isInstanceOf(LocalDate.class); + assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); Assert.assertEquals( "ISO-8601 date should be equal", expected.toString(), actual.toString()); break; @@ -130,76 +128,66 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) Types.TimestampType timestampType = (Types.TimestampType) type; if (timestampType.shouldAdjustToUTC()) { // Timestamptz - Assertions.assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); + assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); Timestamp ts = (Timestamp) actual; // milliseconds from nanos has already been added by getTime OffsetDateTime actualTs = EPOCH.plusNanos((ts.getTime() * 1_000_000) + (ts.getNanos() % 1_000_000)); - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an OffsetDateTime") .isInstanceOf(OffsetDateTime.class); Assert.assertEquals("Timestamp should be equal", expected, actualTs); } else { // Timestamp - Assertions.assertThat(actual) - .as("Should be a LocalDateTime") - .isInstanceOf(LocalDateTime.class); + assertThat(actual).as("Should be a LocalDateTime").isInstanceOf(LocalDateTime.class); LocalDateTime ts = (LocalDateTime) actual; - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an LocalDateTime") .isInstanceOf(LocalDateTime.class); Assert.assertEquals("Timestamp should be equal", expected, ts); } break; case STRING: - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("Strings should be equal", String.valueOf(expected), actual); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("UUID string representation should match", expected.toString(), actual); break; case FIXED: - Assertions.assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals("Bytes should match", (byte[]) expected, (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); Assert.assertEquals("BigDecimals should be equal", expected, actual); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); assertEqualsSafe(type.asNestedType().asStructType(), (Record) expected, (Row) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be a Map") - .isInstanceOf(scala.collection.Map.class); + assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); + assertThat(actual).as("Should be a Map").isInstanceOf(scala.collection.Map.class); Map asMap = mapAsJavaMapConverter((scala.collection.Map) actual).asJava(); assertEqualsSafe(type.asNestedType().asMapType(), (Map) expected, asMap); @@ -267,23 +255,21 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case DATE: - Assertions.assertThat(expected) - .as("Should expect a LocalDate") - .isInstanceOf(LocalDate.class); + assertThat(expected).as("Should expect a LocalDate").isInstanceOf(LocalDate.class); int expectedDays = (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) expected); Assert.assertEquals("Primitive value should be equal to expected", expectedDays, actual); break; case TIMESTAMP: Types.TimestampType timestampType = (Types.TimestampType) type; if (timestampType.shouldAdjustToUTC()) { - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an OffsetDateTime") .isInstanceOf(OffsetDateTime.class); long expectedMicros = ChronoUnit.MICROS.between(EPOCH, (OffsetDateTime) expected); Assert.assertEquals( "Primitive value should be equal to expected", expectedMicros, actual); } else { - Assertions.assertThat(expected) + assertThat(expected) .as("Should expect an LocalDateTime") .isInstanceOf(LocalDateTime.class); long expectedMicros = @@ -293,57 +279,47 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual } break; case STRING: - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals("Strings should be equal", expected, actual.toString()); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals( "UUID string representation should match", expected.toString(), actual.toString()); break; case FIXED: - Assertions.assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a byte[]").isInstanceOf(byte[].class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals("Bytes should match", (byte[]) expected, (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); Assert.assertEquals( "BigDecimals should be equal", expected, ((Decimal) actual).toJavaBigDecimal()); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual) - .as("Should be an InternalRow") - .isInstanceOf(InternalRow.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be an InternalRow").isInstanceOf(InternalRow.class); assertEqualsUnsafe( type.asNestedType().asStructType(), (Record) expected, (InternalRow) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be an ArrayBasedMapData") - .isInstanceOf(MapData.class); + 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); break; case TIME: diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java index 8e6b576ddffb..d9870702dfe6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestHelpers.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.data; import static org.apache.iceberg.spark.SparkSchemaUtil.convert; +import static org.assertj.core.api.Assertions.assertThat; import static scala.collection.JavaConverters.mapAsJavaMapConverter; import static scala.collection.JavaConverters.seqAsJavaListConverter; @@ -77,7 +78,6 @@ import org.apache.spark.sql.types.StructType; import org.apache.spark.sql.vectorized.ColumnarBatch; import org.apache.spark.unsafe.types.UTF8String; -import org.assertj.core.api.Assertions; import org.junit.Assert; import scala.collection.Seq; @@ -166,8 +166,8 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case DATE: - Assertions.assertThat(expected).as("Should be an int").isInstanceOf(Integer.class); - Assertions.assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); + assertThat(expected).as("Should be an int").isInstanceOf(Integer.class); + assertThat(actual).as("Should be a Date").isInstanceOf(Date.class); int daysFromEpoch = (Integer) expected; LocalDate date = ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); Assert.assertEquals("ISO-8601 date should be equal", date.toString(), actual.toString()); @@ -175,18 +175,16 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) case TIMESTAMP: Types.TimestampType timestampType = (Types.TimestampType) type; - Assertions.assertThat(expected).as("Should be a long").isInstanceOf(Long.class); + assertThat(expected).as("Should be a long").isInstanceOf(Long.class); if (timestampType.shouldAdjustToUTC()) { - Assertions.assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); + assertThat(actual).as("Should be a Timestamp").isInstanceOf(Timestamp.class); Timestamp ts = (Timestamp) actual; // milliseconds from nanos has already been added by getTime long tsMicros = (ts.getTime() * 1000) + ((ts.getNanos() / 1000) % 1000); Assert.assertEquals("Timestamp micros should be equal", expected, tsMicros); } else { - Assertions.assertThat(actual) - .as("Should be a LocalDateTime") - .isInstanceOf(LocalDateTime.class); + assertThat(actual).as("Should be a LocalDateTime").isInstanceOf(LocalDateTime.class); LocalDateTime ts = (LocalDateTime) actual; Instant instant = ts.toInstant(ZoneOffset.UTC); @@ -196,55 +194,45 @@ private static void assertEqualsSafe(Type type, Object expected, Object actual) } break; case STRING: - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("Strings should be equal", String.valueOf(expected), actual); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a String").isInstanceOf(String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a String").isInstanceOf(String.class); Assert.assertEquals("UUID string representation should match", expected.toString(), actual); break; case FIXED: - Assertions.assertThat(expected) - .as("Should expect a Fixed") - .isInstanceOf(GenericData.Fixed.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a Fixed").isInstanceOf(GenericData.Fixed.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((GenericData.Fixed) expected).bytes(), (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a BigDecimal").isInstanceOf(BigDecimal.class); Assert.assertEquals("BigDecimals should be equal", expected, actual); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be a Row").isInstanceOf(Row.class); assertEqualsSafe(type.asNestedType().asStructType(), (Record) expected, (Row) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be a Seq").isInstanceOf(Seq.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be a Map") - .isInstanceOf(scala.collection.Map.class); + assertThat(expected).as("Should expect a Collection").isInstanceOf(Map.class); + assertThat(actual).as("Should be a Map").isInstanceOf(scala.collection.Map.class); Map asMap = mapAsJavaMapConverter((scala.collection.Map) actual).asJava(); assertEqualsSafe(type.asNestedType().asMapType(), (Map) expected, asMap); @@ -304,7 +292,7 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual switch (type.typeId()) { case LONG: - Assertions.assertThat(actual).as("Should be a long").isInstanceOf(Long.class); + assertThat(actual).as("Should be a long").isInstanceOf(Long.class); if (expected instanceof Integer) { Assert.assertEquals("Values didn't match", ((Number) expected).longValue(), actual); } else { @@ -312,7 +300,7 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual } break; case DOUBLE: - Assertions.assertThat(actual).as("Should be a double").isInstanceOf(Double.class); + assertThat(actual).as("Should be a double").isInstanceOf(Double.class); if (expected instanceof Float) { Assert.assertEquals( "Values didn't match", @@ -330,60 +318,48 @@ private static void assertEqualsUnsafe(Type type, Object expected, Object actual Assert.assertEquals("Primitive value should be equal to expected", expected, actual); break; case STRING: - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals("Strings should be equal", expected, actual.toString()); break; case UUID: - Assertions.assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - Assertions.assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + assertThat(actual).as("Should be a UTF8String").isInstanceOf(UTF8String.class); Assert.assertEquals( "UUID string representation should match", expected.toString(), actual.toString()); break; case FIXED: - Assertions.assertThat(expected) - .as("Should expect a Fixed") - .isInstanceOf(GenericData.Fixed.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a Fixed").isInstanceOf(GenericData.Fixed.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((GenericData.Fixed) expected).bytes(), (byte[]) actual); break; case BINARY: - Assertions.assertThat(expected) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class); - Assertions.assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Should expect a ByteBuffer").isInstanceOf(ByteBuffer.class); + assertThat(actual).as("Should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Bytes should match", ((ByteBuffer) expected).array(), (byte[]) actual); break; case DECIMAL: - Assertions.assertThat(expected) - .as("Should expect a BigDecimal") - .isInstanceOf(BigDecimal.class); - Assertions.assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + assertThat(actual).as("Should be a Decimal").isInstanceOf(Decimal.class); Assert.assertEquals( "BigDecimals should be equal", expected, ((Decimal) actual).toJavaBigDecimal()); break; case STRUCT: - Assertions.assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); - Assertions.assertThat(actual) - .as("Should be an InternalRow") - .isInstanceOf(InternalRow.class); + assertThat(expected).as("Should expect a Record").isInstanceOf(Record.class); + assertThat(actual).as("Should be an InternalRow").isInstanceOf(InternalRow.class); assertEqualsUnsafe( type.asNestedType().asStructType(), (Record) expected, (InternalRow) actual); break; case LIST: - Assertions.assertThat(expected) - .as("Should expect a Collection") - .isInstanceOf(Collection.class); - Assertions.assertThat(actual).as("Should be an ArrayData").isInstanceOf(ArrayData.class); + 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); break; case MAP: - Assertions.assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - Assertions.assertThat(actual) - .as("Should be an ArrayBasedMapData") - .isInstanceOf(MapData.class); + 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); break; case TIME: @@ -704,30 +680,28 @@ private static void assertEquals(String context, DataType type, Object expected, } if (type instanceof StructType) { - Assertions.assertThat(expected) + assertThat(expected) .as("Expected should be an InternalRow: " + context) .isInstanceOf(InternalRow.class); - Assertions.assertThat(actual) + assertThat(actual) .as("Actual should be an InternalRow: " + context) .isInstanceOf(InternalRow.class); assertEquals(context, (StructType) type, (InternalRow) expected, (InternalRow) actual); } else if (type instanceof ArrayType) { - Assertions.assertThat(expected) + assertThat(expected) .as("Expected should be an ArrayData: " + context) .isInstanceOf(ArrayData.class); - Assertions.assertThat(actual) + assertThat(actual) .as("Actual should be an ArrayData: " + context) .isInstanceOf(ArrayData.class); assertEquals(context, (ArrayType) type, (ArrayData) expected, (ArrayData) actual); } else if (type instanceof MapType) { - Assertions.assertThat(expected) + assertThat(expected) .as("Expected should be a MapData: " + context) .isInstanceOf(MapData.class); - Assertions.assertThat(actual) - .as("Actual should be a MapData: " + context) - .isInstanceOf(MapData.class); + assertThat(actual).as("Actual should be a MapData: " + context).isInstanceOf(MapData.class); assertEquals(context, (MapType) type, (MapData) expected, (MapData) actual); } else if (type instanceof BinaryType) { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java index 024ce3a60c2b..77ad638542f5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/TestSparkParquetReader.java @@ -20,6 +20,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 java.io.File; import java.io.IOException; @@ -56,7 +57,6 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Test; @@ -158,7 +158,7 @@ public void testInt96TimestampProducedBySparkIsReadCorrectly() throws IOExceptio InputFile parquetInputFile = Files.localInput(outputFilePath); List readRows = rowsFromFile(parquetInputFile, schema); Assert.assertEquals(rows.size(), readRows.size()); - Assertions.assertThat(readRows).isEqualTo(rows); + assertThat(readRows).isEqualTo(rows); // Now we try to import that file as an Iceberg table to make sure Iceberg can read // Int96 end to end. diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index c763b7b7cc12..e7bd149b9d25 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -20,6 +20,7 @@ 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.File; import java.io.IOException; @@ -46,7 +47,6 @@ import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.Type; import org.apache.spark.sql.vectorized.ColumnarBatch; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Assume; import org.junit.Ignore; @@ -199,7 +199,7 @@ public void testMixedTypes() {} @Test @Override public void testNestedStruct() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> VectorizedSparkParquetReaders.buildReader( TypeUtil.assignIncreasingFreshIds( @@ -325,8 +325,7 @@ public void testUnsupportedReadsForParquetV2() throws Exception { try (FileAppender writer = getParquetV2Writer(schema, dataFile)) { writer.addAll(data); } - Assertions.assertThatThrownBy( - () -> assertRecordsMatch(schema, 30000, data, dataFile, false, BATCH_SIZE)) + assertThatThrownBy(() -> assertRecordsMatch(schema, 30000, data, dataFile, false, BATCH_SIZE)) .isInstanceOf(UnsupportedOperationException.class) .hasMessageStartingWith("Cannot support vectorized reads for column") .hasMessageEndingWith("Disable vectorized reads to read this table/file"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java index 34308e77d27a..5d1f44897d0c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.spark.functions; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; -import org.assertj.core.api.Assertions; import org.junit.Test; public class TestSparkFunctions { @@ -150,8 +151,8 @@ public void testBuildTruncateFunctionFromClass() { private void checkBuildFunc(ScalarFunction function, UnboundFunction expected) { UnboundFunction actual = SparkFunctions.loadFunctionByClass(function.getClass()); - Assertions.assertThat(actual).isNotNull(); - Assertions.assertThat(actual.name()).isEqualTo(expected.name()); - Assertions.assertThat(actual.description()).isEqualTo(expected.description()); + assertThat(actual).isNotNull(); + assertThat(actual.name()).isEqualTo(expected.name()); + assertThat(actual.description()).isEqualTo(expected.description()); } } 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 1ec3c4726d0a..ea329f96d5b9 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 @@ -34,6 +34,7 @@ import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_LEVEL; import static org.apache.iceberg.spark.SparkSQLProperties.COMPRESSION_STRATEGY; import static org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER; +import static org.assertj.core.api.Assertions.assertThat; import java.util.List; import java.util.Map; @@ -69,7 +70,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.BeforeClass; import org.junit.Rule; @@ -170,7 +170,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); InputFile inputFile = table.io().newInputFile(file.path().toString()); - Assertions.assertThat(getCompressionType(inputFile)) + assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -184,7 +184,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); InputFile inputFile = table.io().newInputFile(file.path().toString()); - Assertions.assertThat(getCompressionType(inputFile)) + assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } @@ -198,7 +198,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { ManifestFiles.readDeleteManifest(deleteManifestFiles.get(0), table.io(), specMap)) { DeleteFile file = reader.iterator().next(); InputFile inputFile = table.io().newInputFile(file.path().toString()); - Assertions.assertThat(getCompressionType(inputFile)) + assertThat(getCompressionType(inputFile)) .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } } 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 76b138ced75c..824b0a17daef 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,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -32,7 +34,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -76,7 +77,7 @@ public void testMergeSchemaFailsWithoutWriterOption() throws Exception { // this has a different error message than the case without accept-any-schema because it uses // Iceberg checks - Assertions.assertThatThrownBy(() -> threeColDF.writeTo(tableName).append()) + assertThatThrownBy(() -> threeColDF.writeTo(tableName).append()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Field new_col not found in source schema"); } @@ -102,8 +103,7 @@ public void testMergeSchemaWithoutAcceptAnySchema() throws Exception { "{ \"id\": 3, \"data\": \"c\", \"new_col\": 12.06 }", "{ \"id\": 4, \"data\": \"d\", \"new_col\": 14.41 }"); - Assertions.assertThatThrownBy( - () -> threeColDF.writeTo(tableName).option("merge-schema", "true").append()) + assertThatThrownBy(() -> threeColDF.writeTo(tableName).option("merge-schema", "true").append()) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Cannot write to 'testhadoop.default.table', too many data columns"); 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 310e69b827a9..a5a3da39ad1f 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 @@ -21,6 +21,7 @@ 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.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -63,7 +64,6 @@ import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.encoders.RowEncoder; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; @@ -408,8 +408,7 @@ public void testFaultToleranceOnWrite() throws IOException { Iterable records2 = RandomData.generate(schema, 100, 0L); - Assertions.assertThatThrownBy( - () -> writeDataWithFailOnPartition(records2, schema, location.toString())) + assertThatThrownBy(() -> writeDataWithFailOnPartition(records2, schema, location.toString())) .isInstanceOf(SparkException.class); table.refresh(); 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 83d8953735c5..6afd4fdb514d 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 @@ -19,6 +19,8 @@ package org.apache.iceberg.spark.source; 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 java.io.IOException; import java.math.RoundingMode; @@ -54,7 +56,6 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -245,7 +246,7 @@ public void testIncrementalScanOptions() throws IOException { List snapshotIds = SnapshotUtil.currentAncestorIds(table); // start-snapshot-id and snapshot-id are both configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -259,7 +260,7 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // end-snapshot-id and as-of-timestamp are both configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -275,7 +276,7 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set start-snapshot-id and end-snapshot-id for incremental scans when either snapshot-id or as-of-timestamp is set"); // only end-snapshot-id is configured. - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -450,7 +451,7 @@ public void testExtraSnapshotMetadataWithSQL() throws InterruptedException, IOEx List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assertions.assertThat(snapshots.get(1).summary()) + assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-writer-thread") .containsEntry("extra-key", "someValue") .containsEntry("another-key", "anotherValue"); @@ -493,7 +494,7 @@ public void testExtraSnapshotMetadataWithDelete() List snapshots = Lists.newArrayList(table.snapshots()); Assert.assertEquals(2, snapshots.size()); Assert.assertNull(snapshots.get(0).summary().get("writer-thread")); - Assertions.assertThat(snapshots.get(1).summary()) + assertThat(snapshots.get(1).summary()) .containsEntry("writer-thread", "test-extra-commit-message-delete-thread") .containsEntry("extra-key", "someValue") .containsEntry("another-key", "anotherValue"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java index e8af5e51ec44..ba13d005bdc3 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestFilteredScan.java @@ -24,6 +24,7 @@ import static org.apache.spark.sql.catalyst.util.DateTimeUtils.fromJavaTimestamp; import static org.apache.spark.sql.functions.callUDF; import static org.apache.spark.sql.functions.column; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -77,7 +78,6 @@ import org.apache.spark.sql.types.LongType$; import org.apache.spark.sql.types.StringType$; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Before; @@ -618,7 +618,7 @@ private List expected(int... ordinals) { } private void pushFilters(ScanBuilder scan, Filter... filters) { - Assertions.assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); + assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); SupportsPushDownV2Filters filterable = (SupportsPushDownV2Filters) scan; filterable.pushPredicates(Arrays.stream(filters).map(Filter::toV2).toArray(Predicate[]::new)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java index 80a8196f8a09..9f97753094a5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestForwardCompatibility.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.Files.localInput; import static org.apache.iceberg.Files.localOutput; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -54,7 +55,6 @@ import org.apache.spark.sql.execution.streaming.MemoryStream; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -120,7 +120,7 @@ public void testSparkWriteFailsUnknownTransform() throws IOException { Dataset df = spark.createDataFrame(expected, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> df.select("id", "data") .write() @@ -158,7 +158,7 @@ public void testSparkStreamingWriteFailsUnknownTransform() throws IOException, T List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .isInstanceOf(StreamingQueryException.class) .hasMessageEndingWith("Cannot write using unsupported transforms: zero"); } 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 b5038f981bae..b990a37cdd76 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 @@ -22,6 +22,8 @@ import static org.apache.iceberg.ManifestContent.DELETES; 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; @@ -91,7 +93,6 @@ import org.apache.spark.sql.functions; import org.apache.spark.sql.internal.SQLConf; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Rule; @@ -1117,7 +1118,7 @@ public void testPruneManifestsTable() { if (!spark.version().startsWith("2")) { // Spark 2 isn't able to actually push down nested struct projections so this will not break - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -2175,9 +2176,7 @@ private void testWithFilter(String filterExpr, TableIdentifier tableIdentifier) .select("tmp_col") .filter(filterExpr) .collectAsList(); - Assertions.assertThat(actual) - .as("Rows must match") - .containsExactlyInAnyOrderElementsOf(expected); + assertThat(actual).as("Rows must match").containsExactlyInAnyOrderElementsOf(expected); } private GenericData.Record manifestRecord( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java index 37e329a8b97b..0154506f86b8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIcebergSpark.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.sql.Date; @@ -33,7 +35,6 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; import org.apache.spark.sql.types.VarcharType; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -181,7 +182,7 @@ public void testRegisterDecimalBucketUDF() { @Test public void testRegisterBooleanBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_boolean_16", DataTypes.BooleanType, 16)) @@ -191,7 +192,7 @@ public void testRegisterBooleanBucketUDF() { @Test public void testRegisterDoubleBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_double_16", DataTypes.DoubleType, 16)) @@ -201,7 +202,7 @@ public void testRegisterDoubleBucketUDF() { @Test public void testRegisterFloatBucketUDF() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> IcebergSpark.registerBucketUDF( spark, "iceberg_bucket_float_16", DataTypes.FloatType, 16)) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java index ea65fead10fc..13abdad9ef92 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestMetadataTablesWithPartitionEvolution.java @@ -29,6 +29,7 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Arrays; import java.util.List; @@ -53,7 +54,6 @@ import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -641,7 +641,7 @@ public void testMetadataTablesWithUnknownTransforms() { sql("REFRESH TABLE %s", tableName); for (MetadataTableType tableType : Arrays.asList(FILES, ALL_DATA_FILES, ENTRIES, ALL_ENTRIES)) { - Assertions.assertThatThrownBy(() -> loadMetadataTable(tableType)) + assertThatThrownBy(() -> loadMetadataTable(tableType)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot build table partition type, unknown transforms: [zero]"); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java index 5baf6071233d..e18c4d32412f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPathIdentifier.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; 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; @@ -35,7 +36,6 @@ import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; import org.apache.spark.sql.connector.expressions.Transform; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -76,9 +76,8 @@ public void testPathIdentifier() throws TableAlreadyExistsException, NoSuchTable identifier, SparkSchemaUtil.convert(SCHEMA), new Transform[0], ImmutableMap.of()); Assert.assertEquals(table.table().location(), tableLocation.getAbsolutePath()); - Assertions.assertThat(table.table()).isInstanceOf(BaseTable.class); - Assertions.assertThat(((BaseTable) table.table()).operations()) - .isInstanceOf(HadoopTableOperations.class); + assertThat(table.table()).isInstanceOf(BaseTable.class); + assertThat(((BaseTable) table.table()).operations()).isInstanceOf(HadoopTableOperations.class); Assert.assertEquals(sparkCatalog.loadTable(identifier), table); Assert.assertTrue(sparkCatalog.dropTable(identifier)); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java index eecc405b1a09..38c8425fd923 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestReadProjection.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.avro.Schema.Type.UNION; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.List; @@ -33,7 +34,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -141,8 +141,7 @@ public void testEmptyProjection() throws Exception { Assert.assertNotNull("Should read a non-null record", projected); // this is expected because there are no values - Assertions.assertThatThrownBy(() -> projected.get(0)) - .isInstanceOf(ArrayIndexOutOfBoundsException.class); + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java index 6c96a33a2579..b669c91313f3 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestRequiredDistributionAndOrdering.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Table; @@ -28,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; @@ -162,7 +163,7 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> inputDF .writeTo(tableName) diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java index 645afd4542e4..9fc576dde5e8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSnapshotSelection.java @@ -21,6 +21,8 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; 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 java.io.IOException; import java.util.List; @@ -44,7 +46,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -197,7 +198,7 @@ public void testSnapshotSelectionByInvalidSnapshotId() throws IOException { Dataset df = spark.read().format("iceberg").option("snapshot-id", -10).load(tableLocation); - Assertions.assertThatThrownBy(df::collectAsList) + assertThatThrownBy(df::collectAsList) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find snapshot with ID -10"); } @@ -211,7 +212,7 @@ public void testSnapshotSelectionByInvalidTimestamp() throws IOException { PartitionSpec spec = PartitionSpec.unpartitioned(); tables.create(SCHEMA, spec, properties, tableLocation); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -239,7 +240,7 @@ public void testSnapshotSelectionBySnapshotIdAndTimestamp() throws IOException { long timestamp = System.currentTimeMillis(); long snapshotId = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -342,7 +343,7 @@ public void testSnapshotSelectionByBranchAndTagFails() throws IOException { table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -373,7 +374,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep table.manageSnapshots().createBranch("branch", table.currentSnapshot().snapshotId()).commit(); table.manageSnapshots().createTag("tag", table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -385,7 +386,7 @@ public void testSnapshotSelectionByTimestampAndBranchOrTagFails() throws IOExcep .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Can specify only one of snapshot-id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> spark .read() @@ -428,7 +429,7 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { table.updateSchema().deleteColumn("data").commit(); // The data should not have the deleted column - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -441,7 +442,7 @@ public void testSnapshotSelectionByBranchWithSchemaChange() throws IOException { // re-introducing the column should not let the data re-appear table.updateSchema().addColumn("data", Types.StringType.get()).commit(); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -483,7 +484,7 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { // Deleting and add a new column of the same type to indicate schema change table.updateSchema().deleteColumn("data").addColumn("zip", Types.IntegerType.get()).commit(); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") @@ -514,7 +515,7 @@ public void testWritingToBranchAfterSchemaChange() throws IOException { .mode("append") .save(tableLocation); - Assertions.assertThat( + assertThat( spark .read() .format("iceberg") diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index 3d668197fd51..abe70f94fe19 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.iceberg.CachingCatalog; import org.apache.iceberg.CatalogProperties; @@ -27,7 +29,6 @@ import org.apache.iceberg.spark.SparkSessionCatalog; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.BeforeClass; import org.junit.Test; @@ -93,18 +94,18 @@ public TestSparkCatalogCacheExpiration() { @Test public void testSparkSessionCatalogWithExpirationEnabled() { SparkSessionCatalog sparkCatalog = sparkSessionCatalog(); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .extracting("cacheEnabled") .isEqualTo(true); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, icebergCatalog -> { - Assertions.assertThat(icebergCatalog) + assertThat(icebergCatalog) .isExactlyInstanceOf(CachingCatalog.class) .extracting("expirationIntervalMillis") .isEqualTo(3000L); @@ -114,30 +115,27 @@ public void testSparkSessionCatalogWithExpirationEnabled() { @Test public void testCacheEnabledAndExpirationDisabled() { SparkCatalog sparkCatalog = getSparkCatalog("expiration_disabled"); - Assertions.assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(true); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .isInstanceOfSatisfying( CachingCatalog.class, icebergCatalog -> { - Assertions.assertThat(icebergCatalog) - .extracting("expirationIntervalMillis") - .isEqualTo(-1L); + assertThat(icebergCatalog).extracting("expirationIntervalMillis").isEqualTo(-1L); }); } @Test public void testCacheDisabledImplicitly() { SparkCatalog sparkCatalog = getSparkCatalog("cache_disabled_implicitly"); - Assertions.assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); + assertThat(sparkCatalog).extracting("cacheEnabled").isEqualTo(false); - Assertions.assertThat(sparkCatalog) + assertThat(sparkCatalog) .extracting("icebergCatalog") .isInstanceOfSatisfying( Catalog.class, - icebergCatalog -> - Assertions.assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); + icebergCatalog -> assertThat(icebergCatalog).isNotInstanceOf(CachingCatalog.class)); } private SparkSessionCatalog sparkSessionCatalog() { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java index d6a235674d63..63c18277aa92 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkDataWrite.java @@ -20,6 +20,8 @@ import static org.apache.iceberg.TableProperties.SPARK_WRITE_PARTITIONED_FANOUT_ENABLED; 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.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -52,7 +54,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.Assume; @@ -689,7 +690,7 @@ public void testCommitUnknownException() throws IOException { ManualSource.setTable(manualTableName, sparkTable); // Although an exception is thrown here, write and commit have succeeded - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> df2.select("id", "data") .sort("data") @@ -707,7 +708,7 @@ public void testCommitUnknownException() throws IOException { result.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); Assert.assertEquals( "Number of rows should match", records.size() + records2.size(), actual.size()); - Assertions.assertThat(actual) + assertThat(actual) .describedAs("Result rows should match") .containsExactlyInAnyOrder( ImmutableList.builder() diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index b2361c188c17..778c46bba6b6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -25,6 +25,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; import static org.apache.spark.sql.functions.lit; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.List; @@ -52,7 +53,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -224,7 +224,7 @@ public void testPartitionMetadataColumnWithUnknownTransforms() { TableMetadata base = ops.current(); ops.commit(base, base.updatePartitionSpec(UNKNOWN_SPEC)); - Assertions.assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT _partition FROM %s", TABLE_NAME)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot build table partition type, unknown transforms: [zero]"); } @@ -244,7 +244,7 @@ public void testConflictingColumns() { ImmutableList.of(row(1L, "a1")), sql("SELECT id, category FROM %s", TABLE_NAME)); - Assertions.assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) + assertThatThrownBy(() -> sql("SELECT * FROM %s", TABLE_NAME)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( "Table column names conflict with names reserved for Iceberg metadata columns: [_spec_id, _file]."); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java index fea5a1d3e1c3..d5d51c27a0f6 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkReadMetrics.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.source; +import static org.assertj.core.api.Assertions.assertThat; import static scala.collection.JavaConverters.seqAsJavaListConverter; import java.util.List; @@ -28,7 +29,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.metric.SQLMetric; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; import scala.collection.JavaConverters; @@ -57,30 +57,30 @@ public void testReadMetricsForV1Table() throws NoSuchTableException { Map metricsMap = JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); // Common - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); // data manifests - Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); + assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); // data files - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); // delete manifests - Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); // delete files - Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); + assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } @Test @@ -101,30 +101,30 @@ public void testReadMetricsForV2Table() throws NoSuchTableException { JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); // Common - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); // data manifests - Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); - Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(2); + assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(2); + assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); // data files - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); // delete manifests - Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); // delete files - Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteFileSize").value()).isEqualTo(0); + assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } @Test @@ -152,29 +152,29 @@ public void testDeleteMetrics() throws NoSuchTableException { JavaConverters.mapAsJavaMapConverter(sparkPlans.get(0).metrics()).asJava(); // Common - Assertions.assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); + assertThat(metricsMap.get("totalPlanningDuration").value()).isNotEqualTo(0); // data manifests - Assertions.assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataManifest").value()).isEqualTo(1); + assertThat(metricsMap.get("scannedDataManifests").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataManifests").value()).isEqualTo(0); // data files - Assertions.assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDataFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDataFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDataFileSize").value()).isNotEqualTo(0); // delete manifests - Assertions.assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteManifests").value()).isEqualTo(1); + assertThat(metricsMap.get("scannedDeleteManifests").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDeleteManifests").value()).isEqualTo(0); // delete files - Assertions.assertThat(metricsMap.get("totalDeleteFileSize").value()).isNotEqualTo(0); - Assertions.assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); - Assertions.assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(1); - Assertions.assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("totalDeleteFileSize").value()).isNotEqualTo(0); + assertThat(metricsMap.get("resultDeleteFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("equalityDeleteFiles").value()).isEqualTo(0); + assertThat(metricsMap.get("indexedDeleteFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("positionalDeleteFiles").value()).isEqualTo(1); + assertThat(metricsMap.get("skippedDeleteFiles").value()).isEqualTo(0); } } 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 78d169bf7340..bc03fa042920 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 @@ -26,6 +26,7 @@ import static org.apache.iceberg.spark.SystemFunctionPushDownHelper.timestampStrToYearOrdinal; import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -57,7 +58,6 @@ import org.apache.spark.sql.connector.read.SupportsPushDownV2Filters; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -128,7 +128,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT Equal builder = scanBuilder(); @@ -137,7 +137,7 @@ public void testUnpartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -156,7 +156,7 @@ public void testPartitionedYears() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT Equal builder = scanBuilder(); @@ -165,7 +165,7 @@ public void testPartitionedYears() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } @Test @@ -185,7 +185,7 @@ public void testUnpartitionedMonths() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT GT builder = scanBuilder(); @@ -194,7 +194,7 @@ public void testUnpartitionedMonths() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -214,7 +214,7 @@ public void testPartitionedMonths() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT GT builder = scanBuilder(); @@ -223,7 +223,7 @@ public void testPartitionedMonths() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } @Test @@ -242,7 +242,7 @@ public void testUnpartitionedDays() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT LT builder = scanBuilder(); @@ -251,7 +251,7 @@ public void testUnpartitionedDays() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -270,7 +270,7 @@ public void testPartitionedDays() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT LT builder = scanBuilder(); @@ -279,7 +279,7 @@ public void testPartitionedDays() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } @Test @@ -298,7 +298,7 @@ public void testUnpartitionedHours() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT GTEQ builder = scanBuilder(); @@ -307,7 +307,7 @@ public void testUnpartitionedHours() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -326,7 +326,7 @@ public void testPartitionedHours() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(8); + assertThat(scan.planInputPartitions().length).isEqualTo(8); // NOT GTEQ builder = scanBuilder(); @@ -335,7 +335,7 @@ public void testPartitionedHours() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(2); + assertThat(scan.planInputPartitions().length).isEqualTo(2); } @Test @@ -350,7 +350,7 @@ public void testUnpartitionedBucketLong() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT GTEQ builder = scanBuilder(); @@ -359,7 +359,7 @@ public void testUnpartitionedBucketLong() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -374,7 +374,7 @@ public void testPartitionedBucketLong() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(6); + assertThat(scan.planInputPartitions().length).isEqualTo(6); // NOT GTEQ builder = scanBuilder(); @@ -383,7 +383,7 @@ public void testPartitionedBucketLong() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(4); + assertThat(scan.planInputPartitions().length).isEqualTo(4); } @Test @@ -398,7 +398,7 @@ public void testUnpartitionedBucketString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT LTEQ builder = scanBuilder(); @@ -407,7 +407,7 @@ public void testUnpartitionedBucketString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -422,7 +422,7 @@ public void testPartitionedBucketString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(6); + assertThat(scan.planInputPartitions().length).isEqualTo(6); // NOT LTEQ builder = scanBuilder(); @@ -431,7 +431,7 @@ public void testPartitionedBucketString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(4); + assertThat(scan.planInputPartitions().length).isEqualTo(4); } @Test @@ -446,7 +446,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT NotEqual builder = scanBuilder(); @@ -455,7 +455,7 @@ public void testUnpartitionedTruncateString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -470,7 +470,7 @@ public void testPartitionedTruncateString() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); // NOT NotEqual builder = scanBuilder(); @@ -479,7 +479,7 @@ public void testPartitionedTruncateString() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(5); + assertThat(scan.planInputPartitions().length).isEqualTo(5); } @Test @@ -494,7 +494,7 @@ public void testUnpartitionedIsNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT IsNull builder = scanBuilder(); @@ -503,7 +503,7 @@ public void testUnpartitionedIsNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -518,7 +518,7 @@ public void testPartitionedIsNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(0); + assertThat(scan.planInputPartitions().length).isEqualTo(0); // NOT IsNULL builder = scanBuilder(); @@ -527,7 +527,7 @@ public void testPartitionedIsNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -542,7 +542,7 @@ public void testUnpartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT IsNotNull builder = scanBuilder(); @@ -551,7 +551,7 @@ public void testUnpartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -566,7 +566,7 @@ public void testPartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT IsNotNULL builder = scanBuilder(); @@ -575,7 +575,7 @@ public void testPartitionedIsNotNull() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(0); + assertThat(scan.planInputPartitions().length).isEqualTo(0); } @Test @@ -596,7 +596,7 @@ public void testUnpartitionedAnd() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT (years(ts) = 47 AND bucket(id, 5) >= 2) builder = scanBuilder(); @@ -605,7 +605,7 @@ public void testUnpartitionedAnd() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -626,7 +626,7 @@ public void testPartitionedAnd() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(1); + assertThat(scan.planInputPartitions().length).isEqualTo(1); // NOT (years(ts) = 47 AND bucket(id, 5) >= 2) builder = scanBuilder(); @@ -635,7 +635,7 @@ public void testPartitionedAnd() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(9); + assertThat(scan.planInputPartitions().length).isEqualTo(9); } @Test @@ -656,7 +656,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); // NOT (years(ts) = 47 OR bucket(id, 5) >= 2) builder = scanBuilder(); @@ -665,7 +665,7 @@ public void testUnpartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(10); + assertThat(scan.planInputPartitions().length).isEqualTo(10); } @Test @@ -686,7 +686,7 @@ public void testPartitionedOr() throws Exception { pushFilters(builder, predicate); Batch scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(6); + assertThat(scan.planInputPartitions().length).isEqualTo(6); // NOT (years(ts) = 48 OR bucket(id, 5) >= 2) builder = scanBuilder(); @@ -695,7 +695,7 @@ public void testPartitionedOr() throws Exception { pushFilters(builder, predicate); scan = builder.build().toBatch(); - Assertions.assertThat(scan.planInputPartitions().length).isEqualTo(4); + assertThat(scan.planInputPartitions().length).isEqualTo(4); } private SparkScanBuilder scanBuilder() throws Exception { @@ -707,7 +707,7 @@ private SparkScanBuilder scanBuilder() throws Exception { } private void pushFilters(ScanBuilder scan, Predicate... predicates) { - Assertions.assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); + assertThat(scan).isInstanceOf(SupportsPushDownV2Filters.class); SupportsPushDownV2Filters filterable = (SupportsPushDownV2Filters) scan; filterable.pushPredicates(predicates); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 464f1f5922b3..bd20a628caa8 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.util.List; @@ -40,7 +41,6 @@ import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.StreamingQuery; import org.apache.spark.sql.streaming.StreamingQueryException; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -283,7 +283,7 @@ public void testStreamingWriteUpdateMode() throws Exception { List batch1 = Lists.newArrayList(1, 2); send(batch1, inputStream); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .isInstanceOf(StreamingQueryException.class) .hasMessageContaining("does not support Update mode"); } finally { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java index 190a189f8888..b7d415de3454 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreamingRead3.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.source; import static org.apache.iceberg.expressions.Expressions.ref; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; @@ -58,7 +59,6 @@ import org.apache.spark.sql.streaming.DataStreamWriter; import org.apache.spark.sql.streaming.OutputMode; import org.apache.spark.sql.streaming.StreamingQuery; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -152,7 +152,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots() throws Exception StreamingQuery query = startStream(); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -191,7 +191,7 @@ public void testReadStreamOnIcebergTableWithMultipleSnapshots_WithNumberOfRows_1 // check answer correctness only 1 record read the micro-batch will be stuck List actual = rowsAvailable(query); - Assertions.assertThat(actual) + assertThat(actual) .containsExactlyInAnyOrderElementsOf( Lists.newArrayList(TEST_DATA_MULTIPLE_SNAPSHOTS.get(0).get(0))); } @@ -215,7 +215,7 @@ public void testReadStreamOnIcebergThenAddData() throws Exception { appendDataAsMultipleSnapshots(expected); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -235,14 +235,14 @@ public void testReadingStreamFromTimestamp() throws Exception { startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(streamStartTimestamp)); List empty = rowsAvailable(query); - Assertions.assertThat(empty.isEmpty()).isTrue(); + assertThat(empty.isEmpty()).isTrue(); List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -253,7 +253,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { startStream(SparkReadOptions.STREAM_FROM_TIMESTAMP, Long.toString(futureTimestamp)); List actual = rowsAvailable(query); - Assertions.assertThat(actual.isEmpty()).isTrue(); + assertThat(actual.isEmpty()).isTrue(); List data = Lists.newArrayList( @@ -266,7 +266,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { .forEach( x -> { appendData(data); - Assertions.assertThat(rowsAvailable(query).isEmpty()).isTrue(); + assertThat(rowsAvailable(query).isEmpty()).isTrue(); }); waitUntilAfter(futureTimestamp); @@ -274,7 +274,7 @@ public void testReadingStreamFromFutureTimetsamp() throws Exception { // Data appended after the timestamp should appear appendData(data); actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(data); + assertThat(actual).containsExactlyInAnyOrderElementsOf(data); } @Test @@ -296,7 +296,7 @@ public void testReadingStreamFromTimestampFutureWithExistingSnapshots() throws E waitUntilAfter(streamStartTimestamp); List> expected = TEST_DATA_MULTIPLE_SNAPSHOTS; appendDataAsMultipleSnapshots(expected); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @@ -319,7 +319,7 @@ public void testReadingStreamFromTimestampOfExistingSnapshot() throws Exception } List actual = rowsAvailable(stream); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -348,7 +348,7 @@ public void testReadingStreamWithExpiredSnapshotFromTimestamp() throws TimeoutEx startStream( SparkReadOptions.STREAM_FROM_TIMESTAMP, String.valueOf(firstSnapshotCommitTime)); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); + assertThat(actual).containsExactlyInAnyOrderElementsOf(expectedRecordList); } @Test @@ -387,7 +387,7 @@ public void testResumingStreamReadFromCheckpoint() throws Exception { // Read data added by the stream List actual = spark.read().load(output.getPath()).as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } } @@ -448,7 +448,7 @@ public void testParquetOrcAvroDataInOneTable() throws Exception { appendData(avroFileRecords, "avro"); StreamingQuery query = startStream(); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf( Iterables.concat(parquetFileRecords, orcFileRecords, avroFileRecords)); } @@ -502,7 +502,7 @@ public void testReadStreamWithSnapshotTypeOverwriteErrorsOut() throws Exception StreamingQuery query = startStream(); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .cause() .isInstanceOf(IllegalStateException.class) .hasMessageStartingWith("Cannot process overwrite snapshot"); @@ -522,7 +522,7 @@ public void testReadStreamWithSnapshotTypeReplaceIgnoresReplace() throws Excepti StreamingQuery query = startStream(); List actual = rowsAvailable(query); - Assertions.assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); + assertThat(actual).containsExactlyInAnyOrderElementsOf(Iterables.concat(expected)); } @Test @@ -542,7 +542,7 @@ public void testReadStreamWithSnapshotTypeDeleteErrorsOut() throws Exception { StreamingQuery query = startStream(); - Assertions.assertThatThrownBy(query::processAllAvailable) + assertThatThrownBy(query::processAllAvailable) .cause() .isInstanceOf(IllegalStateException.class) .hasMessageStartingWith("Cannot process delete snapshot"); @@ -564,7 +564,7 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipDeleteOption() throws Exc Assert.assertEquals(DataOperations.DELETE, table.currentSnapshot().operation()); StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_DELETE_SNAPSHOTS, "true"); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); } @@ -596,7 +596,7 @@ public void testReadStreamWithSnapshotTypeDeleteAndSkipOverwriteOption() throws Assert.assertEquals(DataOperations.OVERWRITE, table.currentSnapshot().operation()); StreamingQuery query = startStream(SparkReadOptions.STREAMING_SKIP_OVERWRITE_SNAPSHOTS, "true"); - Assertions.assertThat(rowsAvailable(query)) + assertThat(rowsAvailable(query)) .containsExactlyInAnyOrderElementsOf(Iterables.concat(dataAcrossSnapshots)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java index 1e2a825d8e76..73827b309be2 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestWriteMetricsConfig.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.spark.SparkSchemaUtil.convert; 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.nio.ByteBuffer; @@ -48,7 +49,6 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.InternalRow; -import org.assertj.core.api.Assertions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -231,8 +231,7 @@ public void testBadCustomMetricCollectionForParquet() throws IOException { properties.put(TableProperties.DEFAULT_WRITE_METRICS_MODE, "counts"); properties.put("write.metadata.metrics.column.ids", "full"); - Assertions.assertThatThrownBy( - () -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) + assertThatThrownBy(() -> tables.create(SIMPLE_SCHEMA, spec, properties, tableLocation)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith( "Invalid metrics config, could not find column ids from table prop write.metadata.metrics.column.ids in schema table"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 9ea1a563efa8..2579d614685c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; @@ -36,7 +38,6 @@ import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.spark.SparkTestBase; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.BeforeClass; @@ -571,15 +572,9 @@ private void assertAggregates( Object actualMax = actual.get(0)[1]; Object actualMin = actual.get(0)[2]; - Assertions.assertThat(actualCount) - .as("Expected and actual count should equal") - .isEqualTo(expectedCount); - Assertions.assertThat(actualMax) - .as("Expected and actual max should equal") - .isEqualTo(expectedMax); - Assertions.assertThat(actualMin) - .as("Expected and actual min should equal") - .isEqualTo(expectedMin); + assertThat(actualCount).as("Expected and actual count should equal").isEqualTo(expectedCount); + assertThat(actualMax).as("Expected and actual max should equal").isEqualTo(expectedMax); + assertThat(actualMin).as("Expected and actual min should equal").isEqualTo(expectedMin); } private void assertExplainContains(List explain, String... expectedFragments) { @@ -587,7 +582,7 @@ private void assertExplainContains(List explain, String... expectedFra Arrays.stream(expectedFragments) .forEach( fragment -> - Assertions.assertThat(explainString) + assertThat(explainString) .as("Expected to find plan fragment in explain plan") .contains(fragment)); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index b4dde2f94500..4e34d662ffd1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.File; import java.util.Map; import java.util.UUID; @@ -33,7 +36,6 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -107,27 +109,24 @@ public void testCreateTable() { @Test public void testCreateTablePartitionedByUUID() { - Assertions.assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); validationCatalog.createTable(tableIdent, schema, spec); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table).isNotNull(); + assertThat(table).isNotNull(); StructType expectedSchema = StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); - Assertions.assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); - Assertions.assertThat(table.spec().fields()).hasSize(1); + assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + assertThat(table.spec().fields()).hasSize(1); String uuid = UUID.randomUUID().toString(); sql("INSERT INTO %s VALUES('%s')", tableName, uuid); - Assertions.assertThat(sql("SELECT uuid FROM %s", tableName)) - .hasSize(1) - .element(0) - .isEqualTo(row(uuid)); + assertThat(sql("SELECT uuid FROM %s", tableName)).hasSize(1).element(0).isEqualTo(row(uuid)); } @Test @@ -167,7 +166,7 @@ public void testCreateTableUsingParquet() { "parquet", table.properties().get(TableProperties.DEFAULT_FILE_FORMAT)); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE %s.default.fail (id BIGINT NOT NULL, data STRING) USING crocodile", @@ -380,7 +379,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { TableOperations ops = ((BaseTable) table).operations(); Assert.assertEquals("should create table using format v2", 2, ops.refresh().formatVersion()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)) .cause() .isInstanceOf(IllegalArgumentException.class) 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 74906241fa95..13dd8bc85300 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 @@ -21,6 +21,7 @@ import static org.apache.spark.sql.functions.col; import static org.apache.spark.sql.functions.lit; import static org.apache.spark.sql.functions.when; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Map; import org.apache.iceberg.PartitionSpec; @@ -30,7 +31,6 @@ import org.apache.iceberg.spark.SparkCatalogTestBase; import org.apache.iceberg.types.Types; import org.apache.spark.SparkException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -106,7 +106,7 @@ public void testPartitionedCTAS() { @Test public void testCTASWriteDistributionModeNotRespected() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE %s USING iceberg PARTITIONED BY (bucket(2, id)) AS SELECT * FROM %s", diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index a4b1ef3b9df6..d891507da658 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.Table; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Test; @@ -86,8 +87,7 @@ public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) + assertThatThrownBy(() -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot delete from table at a specific snapshot"); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java index 14b6861e4709..67dfeb3c8404 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.List; import java.util.Map; @@ -29,7 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.SparkCatalogTestBase; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -116,7 +117,7 @@ public void testPurgeTableGCDisabled() throws IOException { "There totally should have 2 files for manifests and files", 2, manifestAndFiles.size()); Assert.assertTrue("All files should be existed", checkFilesExist(manifestAndFiles, true)); - Assertions.assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) + assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Cannot purge table: GC is disabled (deleting files may corrupt other tables"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java index f2ef2d4705ea..f92055ab7a4d 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestFilterPushDown.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; +import static org.assertj.core.api.Assertions.assertThat; import java.math.BigDecimal; import java.sql.Timestamp; @@ -31,7 +32,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.execution.SparkPlan; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Test; import org.junit.runner.RunWith; @@ -585,16 +585,14 @@ private void checkFilters( String planAsString = sparkPlan.toString().replaceAll("#(\\d+L?)", ""); if (sparkFilter != null) { - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Post scan filter should match") .contains("Filter (" + sparkFilter + ")"); } else { - Assertions.assertThat(planAsString) - .as("Should be no post scan filter") - .doesNotContain("Filter ("); + assertThat(planAsString).as("Should be no post scan filter").doesNotContain("Filter ("); } - Assertions.assertThat(planAsString) + assertThat(planAsString) .as("Pushed filters must match") .contains("[filters=" + icebergFilters + ","); } 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 5ee2526a714d..a0573c5b09b7 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,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.File; import java.util.List; import java.util.Map; @@ -29,7 +31,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.spark.SparkCatalogTestBase; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -103,7 +104,7 @@ public void testDropNonEmptyNamespace() { Assert.assertTrue( "Table should exist", validationCatalog.tableExists(TableIdentifier.of(NS, "table"))); - Assertions.assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) + assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageStartingWith("Namespace db is not empty."); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java index 5dde5f33d965..064ad0f1785c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import java.util.UUID; @@ -25,7 +27,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.spark.SparkSQLProperties; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -72,8 +73,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { Table table = validationCatalog.loadTable(tableIdent); table.manageSnapshots().createBranch("test2", table.refs().get(BRANCH).snapshotId()).commit(); sql("REFRESH TABLE " + tableName); - Assertions.assertThatThrownBy( - () -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) + assertThatThrownBy(() -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot write to both branch and WAP branch, but got branch [test2] and WAP branch [%s]", @@ -84,7 +84,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { public void testWapIdAndWapBranchCannotBothBeSetForWrite() { String wapId = UUID.randomUUID().toString(); spark.conf().set(SparkSQLProperties.WAP_ID, wapId); - Assertions.assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) + assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", wapId, BRANCH); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java index 85d340c8b27b..a9b2f6395dff 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSelect.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.text.SimpleDateFormat; import java.util.Date; import java.util.List; @@ -34,7 +36,6 @@ import org.apache.iceberg.spark.SparkReadOptions; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -327,8 +328,7 @@ public void testBranchReference() { @Test public void testUnknownReferenceAsOf() { - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) + assertThatThrownBy(() -> sql("SELECT * FROM %s VERSION AS OF 'test_unknown'", tableName)) .hasMessageContaining("Cannot find matching snapshot ID or reference name for version") .isInstanceOf(ValidationException.class); } @@ -395,7 +395,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using snapshot in table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s VERSION AS OF %s", @@ -405,7 +405,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using snapshot in table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s VERSION AS OF %s", @@ -415,7 +415,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s TIMESTAMP AS OF %s", @@ -425,7 +425,7 @@ public void testInvalidTimeTravelBasedOnBothAsOfAndTableIdentifier() { .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using timestamp in table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "SELECT * FROM %s.%s TIMESTAMP AS OF %s", @@ -444,14 +444,13 @@ public void testInvalidTimeTravelAgainstBranchIdentifierWithAsOf() { sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); // using branch_b1 in the table identifier and VERSION AS OF - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("SELECT * FROM %s.branch_b1 VERSION AS OF %s", tableName, snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); // using branch_b1 in the table identifier and TIMESTAMP AS OF - Assertions.assertThatThrownBy( - () -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) + assertThatThrownBy(() -> sql("SELECT * FROM %s.branch_b1 TIMESTAMP AS OF now()", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot do time-travel based on both table identifier and AS OF"); } @@ -467,7 +466,7 @@ public void testSpecifySnapshotAndTimestamp() { // create a second snapshot sql("INSERT INTO %s VALUES (4, 'd', 4.0), (5, 'e', 5.0)", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java index c4113408aff9..e7c574366550 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark.sql; +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.charset.StandardCharsets; @@ -28,7 +31,6 @@ import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.types.DataTypes; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -197,17 +199,17 @@ public void testNumBucketsAcceptsShortAndByte() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (): Wrong number of inputs (expected numBuckets and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int): Wrong number of inputs (expected numBuckets and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, bigint, int): Wrong number of inputs (expected numBuckets and value)"); @@ -215,29 +217,27 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (decimal(9,2), int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (bigint, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (string, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (interval year to month, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.bucket(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) .isInstanceOf(AnalysisException.class) @@ -247,37 +247,34 @@ public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { @Test public void testInvalidTypesForBucketColumn() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, float): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, double): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'bucket' cannot process input: (int, boolean)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'bucket' cannot process input: (int, map)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'bucket' cannot process input: (int, array)"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, interval year to month)"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.bucket(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) .isInstanceOf(AnalysisException.class) @@ -288,63 +285,59 @@ public void testInvalidTypesForBucketColumn() { @Test public void testThatMagicFunctionsAreInvoked() { // TinyInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // SmallInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Int - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Date - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Long - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); // Timestamp - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); // String - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketString"); // Decimal - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketDecimal"); // Binary - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) .asString() .isNotNull() .contains( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java index 2c744e574fb9..f12bdd59ab3c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkDaysFunction.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.sql.Date; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -86,11 +87,11 @@ public void testTimestampNtz() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days()")) + assertThatThrownBy(() -> scalarSql("SELECT system.days()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'days' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.days(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( @@ -99,12 +100,12 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.days(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'days' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.days(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'days' cannot process input: (bigint): Expected value to be date or timestamp"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java index 5d839abc8871..c2f16e454a5c 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkHoursFunction.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -68,12 +69,12 @@ public void testTimestampsNtz() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.hours(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( @@ -82,12 +83,12 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (int): Expected value to be timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.hours(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'hours' cannot process input: (bigint): Expected value to be timestamp"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java index 9c90ec008d4c..60b93c466698 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkMonthsFunction.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.MonthsFunction; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -84,12 +86,12 @@ public void testTimestampNtz() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months()")) + assertThatThrownBy(() -> scalarSql("SELECT system.months()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.months(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( @@ -98,12 +100,12 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.months(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.months(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'months' cannot process input: (bigint): Expected value to be date or timestamp"); @@ -113,14 +115,14 @@ public void testInvalidInputTypes() { public void testThatMagicFunctionsAreInvoked() { String dateValue = "date('2017-12-01')"; String dateTransformClass = MonthsFunction.DateToMonthsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", dateValue)) .asString() .isNotNull() .contains("staticinvoke(class " + dateTransformClass); String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; String timestampTransformClass = MonthsFunction.TimestampToMonthsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.months(%s)", timestampValue)) .asString() .isNotNull() .contains("staticinvoke(class " + timestampTransformClass); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java index 7d3f360f8817..686b46888f18 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkTruncateFunction.java @@ -18,12 +18,14 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.nio.charset.StandardCharsets; import org.apache.iceberg.relocated.com.google.common.io.BaseEncoding; import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -297,17 +299,17 @@ public void testWidthAcceptsShortAndByte() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (): Wrong number of inputs (expected width and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int): Wrong number of inputs (expected width and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(1, 1L, 1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, bigint, int): Wrong number of inputs (expected width and value)"); @@ -315,24 +317,24 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidTypesCannotBeUsedForWidth() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(CAST('12.34' as DECIMAL(9, 2)), 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (decimal(9,2), int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate('5', 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (string, int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(INTERVAL '100-00' YEAR TO MONTH, 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (interval year to month, int): Expected truncation width to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "SELECT system.truncate(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) @@ -343,40 +345,38 @@ public void testInvalidTypesCannotBeUsedForWidth() { @Test public void testInvalidTypesForTruncationColumn() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as float))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, float): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, cast(12.3456 as double))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, double): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, true)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, boolean): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, map(1, 1))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, map): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) + assertThatThrownBy(() -> scalarSql("SELECT system.truncate(10, array(1L))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, array): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.truncate(10, INTERVAL '100-00' YEAR TO MONTH)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'truncate' cannot process input: (int, interval year to month): Expected truncation col to be tinyint, shortint, int, bigint, decimal, string, or binary"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "SELECT system.truncate(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) @@ -391,13 +391,13 @@ public void testMagicFunctionsResolveForTinyIntAndSmallIntWidths() { // applyfunctionexpression instead. String tinyIntWidthExplain = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(1Y, 6)"); - Assertions.assertThat(tinyIntWidthExplain) + assertThat(tinyIntWidthExplain) .contains("cast(1 as int)") .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); String smallIntWidth = (String) scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5S, 6L)"); - Assertions.assertThat(smallIntWidth) + assertThat(smallIntWidth) .contains("cast(5 as int)") .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); @@ -409,50 +409,49 @@ public void testThatMagicFunctionsAreInvoked() { // Non-magic calls have `applyfunctionexpression` instead. // TinyInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6Y)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateTinyInt"); // SmallInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6S)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateSmallInt"); // Int - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) + assertThat(scalarSql("EXPLAIN EXTENDED select system.truncate(5, 6)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateInt"); // Long - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 6L)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateBigInt"); // String - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 'abcdefg')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateString"); // Decimal - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(5, 12.34)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.TruncateFunction$TruncateDecimal"); // Binary - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.truncate(4, X'0102030405060708')")) .asString() .isNotNull() .contains( diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java index 84c91e1d29ef..ecd3f6cc7db7 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkYearsFunction.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.spark.SparkTestBaseWithCatalog; import org.apache.iceberg.spark.functions.YearsFunction; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -86,12 +88,12 @@ public void testTimestampNtz() { @Test public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years()")) + assertThatThrownBy(() -> scalarSql("SELECT system.years()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (): Wrong number of inputs"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.years(date('1969-12-31'), date('1969-12-31'))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( @@ -100,12 +102,12 @@ public void testWrongNumberOfArguments() { @Test public void testInvalidInputTypes() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.years(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (int): Expected value to be date or timestamp"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) + assertThatThrownBy(() -> scalarSql("SELECT system.years(1L)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'years' cannot process input: (bigint): Expected value to be date or timestamp"); @@ -115,14 +117,14 @@ public void testInvalidInputTypes() { public void testThatMagicFunctionsAreInvoked() { String dateValue = "date('2017-12-01')"; String dateTransformClass = YearsFunction.DateToYearsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", dateValue)) .asString() .isNotNull() .contains("staticinvoke(class " + dateTransformClass); String timestampValue = "TIMESTAMP '2017-12-01 10:12:55.038194 UTC+00:00'"; String timestampTransformClass = YearsFunction.TimestampToYearsFunction.class.getName(); - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.years(%s)", timestampValue)) .asString() .isNotNull() .contains("staticinvoke(class " + timestampTransformClass); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index f09d6ce72dff..196ab119866e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.sql.Timestamp; import java.time.LocalDateTime; import java.util.Arrays; @@ -38,7 +40,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.spark.sql.util.CaseInsensitiveStringMap; -import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.After; import org.junit.Assert; @@ -100,7 +101,7 @@ public void testDeprecatedTimezoneProperty() { withSQLConf( ImmutableMap.of(SparkSQLProperties.USE_TIMESTAMP_WITHOUT_TIME_ZONE_IN_NEW_TABLES, "true"), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .sessionState() @@ -119,7 +120,7 @@ public void testReadWithDeprecatedTimezoneProperty() { withSQLConf( ImmutableMap.of(SparkSQLProperties.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true"), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql("SELECT count(*) FROM %s", tableName); }) @@ -131,7 +132,7 @@ public void testReadWithDeprecatedTimezoneProperty() { @Test public void testReadWithDeprecatedTimezonePropertyReadOption() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { spark .read() @@ -153,7 +154,7 @@ public void testWriteWithDeprecatedTimezoneProperty() { "spark.sql.legacy.createHiveTableByDefault", "false"), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql( "CREATE OR REPLACE TABLE %s USING ICEBERG AS SELECT * FROM %s", @@ -167,7 +168,7 @@ public void testWriteWithDeprecatedTimezoneProperty() { @Test public void testWriteWithDeprecatedTimezonePropertyReadOption() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { withSQLConf( ImmutableMap.of("spark.sql.legacy.createHiveTableByDefault", "false"), diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java index 1f5bee42af05..891813e3816e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestUnpartitionedWritesToBranch.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.ValidationException; -import org.assertj.core.api.Assertions; import org.junit.Test; public class TestUnpartitionedWritesToBranch extends UnpartitionedWritesTestBase { @@ -53,7 +54,7 @@ protected String selectTarget() { @Test public void testInsertIntoNonExistingBranchFails() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("INSERT INTO %s.branch_not_exist VALUES (4, 'd'), (5, 'e')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): not_exist"); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java index 58918249d83f..c4534f8b67fd 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; @@ -27,7 +29,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; -import org.assertj.core.api.Assertions; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -98,7 +99,7 @@ public void testInsertAppendAtSnapshot() { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("INSERT INTO %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) @@ -111,7 +112,7 @@ public void testInsertOverwriteAtSnapshot() { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", 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 91c74a6ad435..ac2d39eaa485 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; 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; @@ -50,7 +51,6 @@ import org.apache.spark.sql.types.Metadata; import org.apache.spark.sql.types.StructField; import org.apache.spark.sql.types.StructType; -import org.assertj.core.api.Assertions; import org.joda.time.DateTime; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -679,7 +679,7 @@ public void invalidDataImport() { createIcebergTable("id Integer, name String, dept String, subdept String"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('id', 1))", @@ -687,7 +687,7 @@ public void invalidDataImport() { .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot use partition filter with an unpartitioned table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`')", @@ -703,7 +703,7 @@ public void invalidDataImportPartitioned() { createIcebergTable( "id Integer, name String, dept String, subdept String", "PARTITIONED BY (id)"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('x', '1', 'y', '2'))", @@ -712,7 +712,7 @@ public void invalidDataImportPartitioned() { .hasMessageStartingWith("Cannot add data files to target table") .hasMessageContaining("is greater than the number of partitioned columns"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '`parquet`.`%s`', map('dept', '2'))", @@ -772,7 +772,7 @@ public void duplicateDataPartitioned() { + "partition_filter => map('id', 1))", catalogName, tableName, sourceTableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files(" @@ -831,7 +831,7 @@ public void duplicateDataUnpartitioned() { sql("CALL %s.system.add_files('%s', '%s')", catalogName, tableName, sourceTableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql( "CALL %s.system.add_files('%s', '%s')", diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java index 3b2427eea7ca..71c85b135859 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAlterTableSchema.java @@ -19,11 +19,11 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -76,12 +76,11 @@ public void testSetInvalidIdentifierFields() { assertThat(table.schema().identifierFieldIds()) .as("Table should start without identifier") .isEmpty(); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS unknown", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("not found in current schema or added columns"); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s SET IDENTIFIER FIELDS id2", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageEndingWith("not a required field"); } @@ -138,20 +137,17 @@ public void testDropInvalidIdentifierFields() { assertThat(table.schema().identifierFieldIds()) .as("Table should start without identifier") .isEmpty(); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS unknown", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot complete drop identifier fields operation: field unknown not found"); sql("ALTER TABLE %s SET IDENTIFIER FIELDS id", tableName); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS data", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot complete drop identifier fields operation: data is not an identifier field"); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP IDENTIFIER FIELDS location.lon", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot complete drop identifier fields operation: location.lon is not an identifier field"); diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java index 437b4c368232..4a3a158dea52 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAncestorsOfProcedure.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -144,16 +145,15 @@ public void testAncestorOfUsingNamedArgs() { @TestTemplate public void testInvalidAncestorOfCases() { - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for parameter 'table'"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) + assertThatThrownBy(() -> sql("CALL %s.system.ancestors_of('%s', 1.1)", catalogName, tableIdent)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for snapshot_id: cannot cast"); } 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 022edecc31e0..fb7f73186ad3 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 @@ -19,6 +19,7 @@ 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.concurrent.TimeUnit; @@ -34,7 +35,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -83,8 +83,7 @@ public void testCreateBranch() throws NoSuchTableException { .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); assertThat(ref.maxRefAgeMs().longValue()).isEqualTo(TimeUnit.DAYS.toMillis(maxRefAge)); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s", tableName, branchName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref b1 already exists"); } @@ -195,7 +194,7 @@ public void testCreateBranchUseCustomMinSnapshotsToKeepAndMaxSnapshotAge() .isEqualTo(TimeUnit.DAYS.toMillis(maxSnapshotAge)); assertThat(ref.maxRefAgeMs()).isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE BRANCH %s WITH SNAPSHOT RETENTION", @@ -217,18 +216,17 @@ public void testCreateBranchUseCustomMaxRefAge() throws NoSuchTableException { assertThat(ref.maxSnapshotAgeMs()).isNull(); assertThat(ref.maxRefAgeMs().longValue()).isEqualTo(TimeUnit.DAYS.toMillis(maxRefAge)); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN", tableName, branchName)) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE BRANCH %s RETAIN %s DAYS", tableName, branchName, "abc")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE BRANCH %s RETAIN %d SECONDS", @@ -256,8 +254,7 @@ public void testDropBranch() throws NoSuchTableException { @TestTemplate public void testDropBranchDoesNotExist() { - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "nonExistingBranch")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Branch does not exist: nonExistingBranch"); } @@ -268,21 +265,21 @@ public void testDropBranchFailsForTag() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createTag(tagName, table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, tagName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Ref b1 is a tag not a branch"); } @TestTemplate public void testDropBranchNonConformingName() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); } @TestTemplate public void testDropMainBranchFails() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP BRANCH main", tableName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot remove main branch"); } @@ -387,7 +384,7 @@ public void replaceBranch() throws NoSuchTableException { public void replaceBranchDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", 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 490127ab52f5..65a36903735c 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static scala.collection.JavaConverters.seqAsJavaList; import java.math.BigDecimal; @@ -39,7 +40,6 @@ import org.apache.spark.sql.catalyst.plans.logical.PositionalArgument; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.DataTypes; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -135,7 +135,7 @@ public void testCallWithVarSubstitution() throws ParseException { @Test public void testCallParseError() { - Assertions.assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) + assertThatThrownBy(() -> parser.parsePlan("CALL cat.system radish kebab")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("missing '(' at 'radish'"); } 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 79c52f25e31f..a7ed065cae2c 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 @@ -22,6 +22,7 @@ import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.DataOperations; @@ -36,7 +37,6 @@ import org.apache.iceberg.spark.source.SparkChangelogTable; import org.apache.spark.sql.DataFrameReader; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -198,8 +198,7 @@ public void testTimeRangeValidation() { table.refresh(); Snapshot snap3 = table.currentSnapshot(); long rightAfterSnap3 = waitUntilAfter(snap3.timestampMillis()); - Assertions.assertThatThrownBy( - () -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) + assertThatThrownBy(() -> changelogRecords(snap3.timestampMillis(), snap2.timestampMillis())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot set start-timestamp to be greater than end-timestamp for changelogs"); } 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 794d77980c74..d61456fa738b 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.Snapshot; @@ -30,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -155,7 +155,7 @@ public void testCherrypickSnapshotRefreshesRelationCache() { public void testCherrypickInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('%s', -1L)", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot cherry-pick unknown snapshot ID: -1"); @@ -163,27 +163,24 @@ public void testCherrypickInvalidSnapshot() { @TestTemplate public void testInvalidCherrypickSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.cherrypick_snapshot('n', table => 't', 1L)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.cherrypick_snapshot('n', 't', 1L)", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.cherrypick_snapshot not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('', 1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.cherrypick_snapshot('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for snapshot_id: cannot cast"); } diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java index cc2bd91cf802..b5ba7eec1b01 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestConflictValidation.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import org.apache.iceberg.IsolationLevel; import org.apache.iceberg.ParameterizedTestExtension; @@ -29,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.functions; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -65,7 +66,7 @@ public void testOverwriteFilterSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -103,7 +104,7 @@ public void testOverwriteFilterSerializableIsolation2() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -136,7 +137,7 @@ public void testOverwriteFilterSerializableIsolation3() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -168,7 +169,7 @@ public void testOverwriteFilterNoSnapshotIdValidation() throws Exception { // Validating from no snapshot id defaults to beginning snapshot id and finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -205,7 +206,7 @@ public void testOverwriteFilterSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts List conflictingRecords = Lists.newArrayList(new SimpleRecord(1, "a")); Dataset conflictingDf = spark.createDataFrame(conflictingRecords, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -253,7 +254,7 @@ public void testOverwritePartitionSerializableIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -289,7 +290,7 @@ public void testOverwritePartitionSnapshotIsolation() throws Exception { // Validating from previous snapshot finds conflicts Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -323,7 +324,7 @@ public void testOverwritePartitionSnapshotIsolation2() throws Exception { spark.createDataFrame(records, SimpleRecord.class).coalesce(1).writeTo(tableName).append(); Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) @@ -370,7 +371,7 @@ public void testOverwritePartitionNoSnapshotIdValidation() throws Exception { // Validating from null snapshot is equivalent to validating from beginning Dataset conflictingDf = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> conflictingDf .writeTo(tableName) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java index edddb3bbb8fd..e5d44d97de1e 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteDelete.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; 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.Collections; @@ -49,7 +50,6 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -135,7 +135,7 @@ public synchronized void testDeleteWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(deleteFuture::get) + assertThatThrownBy(deleteFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index dfd22e96ddc1..1fb1238de635 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.MERGE_ISOLATION_LEVEL; 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.Collections; @@ -46,7 +47,6 @@ import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.TestTemplate; public class TestCopyOnWriteMerge extends TestMerge { @@ -130,7 +130,7 @@ public synchronized void testMergeWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(mergeFuture::get) + assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java index e9767c66bafc..5bc7b22f9a09 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteUpdate.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.UPDATE_ISOLATION_LEVEL; 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; @@ -45,7 +46,6 @@ import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.TestTemplate; public class TestCopyOnWriteUpdate extends TestUpdate { @@ -128,7 +128,7 @@ public synchronized void testUpdateWithConcurrentTableRefresh() throws Exception }); try { - Assertions.assertThatThrownBy(updateFuture::get) + assertThatThrownBy(updateFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(IllegalStateException.class) 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 b88df19ea9ba..d258079541cd 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 @@ -30,6 +30,7 @@ import static org.apache.iceberg.TableProperties.SPLIT_SIZE; import static org.apache.spark.sql.functions.lit; 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.Arrays; @@ -79,7 +80,6 @@ import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.execution.datasources.v2.OptimizeMetadataOnlyDeleteFromTable; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -407,7 +407,7 @@ public void testDeleteFromNonExistingCustomBranch() { assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); createAndInitUnpartitionedTable(); - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id IN (1)", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): test"); } @@ -614,8 +614,7 @@ public void testDeleteWithNonDeterministicCondition() { sql("INSERT INTO TABLE %s VALUES (1, 'hr'), (2, 'hardware')", tableName); createBranchIfNeeded(); - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 1 AND rand() > 0.5", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("The operator expects a deterministic expression"); } @@ -904,7 +903,7 @@ public void testDeleteOnNonIcebergTableNotSupported() { sql("CREATE TABLE parquet_table (c1 INT, c2 INT) USING parquet"); - Assertions.assertThatThrownBy(() -> sql("DELETE FROM parquet_table WHERE c1 = -100")) + assertThatThrownBy(() -> sql("DELETE FROM parquet_table WHERE c1 = -100")) .isInstanceOf(AnalysisException.class) .hasMessageContaining("does not support DELETE"); } @@ -1116,7 +1115,7 @@ public synchronized void testDeleteWithSerializableIsolation() throws Interrupte }); try { - Assertions.assertThatThrownBy(deleteFuture::get) + assertThatThrownBy(deleteFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) @@ -1358,7 +1357,7 @@ public void testDeleteToWapBranchWithTableBranchIdentifier() throws NoSuchTableE withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) + assertThatThrownBy(() -> sql("DELETE FROM %s t WHERE id=0", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage( String.format( 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 46847a03ca7b..970fcc47b80c 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -51,7 +52,6 @@ import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -156,34 +156,30 @@ public void testExpireSnapshotsGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('%s')", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot expire snapshots: GC is disabled"); } @TestTemplate public void testInvalidExpireSnapshotsCases() { - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.expire_snapshots('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.expire_snapshots not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('n', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for older_than: cannot cast DecimalType(2,1) to TimestampType"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.expire_snapshots('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } @@ -203,7 +199,7 @@ public void testResolvingTableInAnotherCatalog() throws IOException { "CREATE TABLE %s.%s (id bigint NOT NULL, data string) USING iceberg", anotherCatalog, tableIdent); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots('%s')", @@ -239,7 +235,7 @@ public void testConcurrentExpireSnapshots() { public void testConcurrentExpireSnapshotsWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", @@ -247,7 +243,7 @@ public void testConcurrentExpireSnapshotsWithInvalidInput() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(table => '%s', max_concurrent_deletes => %s)", @@ -386,7 +382,7 @@ public void testExpireSnapshotShouldFailForCurrentSnapshot() { Table table = validationCatalog.loadTable(tableIdent); assertThat(table.snapshots()).as("Should be 2 snapshots").hasSize(2); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.expire_snapshots(" 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 9f8ac9c993c2..a26707ef38aa 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 @@ -29,6 +29,7 @@ import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; import static org.apache.spark.sql.functions.lit; 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.Arrays; @@ -69,7 +70,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -880,7 +880,7 @@ public void testMergeWithOneMatchingBranchButMultipleSourceRowsForTargetRow() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -915,7 +915,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -956,7 +956,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSource() { () -> { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -995,7 +995,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoEqua () -> { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1033,7 +1033,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1066,7 +1066,7 @@ public void testMergeWithMultipleUpdatesForTargetRowSmallTargetLargeSourceNoNotM String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1102,7 +1102,7 @@ public void testMergeWithMultipleUpdatesForTargetRow() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1173,7 +1173,7 @@ public void testMergeWithSingleConditionalDelete() { String errorMsg = "MERGE statement matched a single row from the target table with multiple rows of the source table."; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s AS t USING source AS s " @@ -1582,7 +1582,7 @@ public synchronized void testMergeWithSerializableIsolation() throws Interrupted }); try { - Assertions.assertThatThrownBy(mergeFuture::get) + assertThatThrownBy(mergeFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) @@ -2321,7 +2321,7 @@ public void testMergeWithNonExistingColumns() { "id INT, c STRUCT>", "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2333,7 +2333,7 @@ public void testMergeWithNonExistingColumns() { .hasMessageContaining( "A column or function parameter with name `t`.`invalid_col` cannot be resolved"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2344,7 +2344,7 @@ public void testMergeWithNonExistingColumns() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("No such struct field `invalid_col`"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2365,7 +2365,7 @@ public void testMergeWithInvalidColumnsInInsert() { "id INT, c STRUCT> NOT NULL", "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2377,7 +2377,7 @@ public void testMergeWithInvalidColumnsInInsert() { commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("INSERT assignment keys cannot be nested fields"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2390,7 +2390,7 @@ public void testMergeWithInvalidColumnsInInsert() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Multiple assignments for 'id'"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2428,7 +2428,7 @@ public void testMergeWithInvalidUpdates() { "id INT, a ARRAY>, m MAP", "{ \"id\": 1, \"a\": [ { \"c1\": 2, \"c2\": 3 } ], \"m\": { \"k\": \"v\"} }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2439,7 +2439,7 @@ public void testMergeWithInvalidUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for StructType"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2457,7 +2457,7 @@ public void testMergeWithConflictingUpdates() { "id INT, c STRUCT>", "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2467,7 +2467,7 @@ public void testMergeWithConflictingUpdates() { commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Multiple assignments for 'id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2478,7 +2478,7 @@ public void testMergeWithConflictingUpdates() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Multiple assignments for 'c.n1'"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2503,7 +2503,7 @@ public void testMergeWithInvalidAssignmentsAnsi() { withSQLConf( ImmutableMap.of(SQLConf.STORE_ASSIGNMENT_POLICY().key(), "ansi"), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2513,7 +2513,7 @@ public void testMergeWithInvalidAssignmentsAnsi() { commitTarget())) .isInstanceOf(SparkException.class) .hasMessageContaining("Null value appeared in non-nullable field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2523,7 +2523,7 @@ public void testMergeWithInvalidAssignmentsAnsi() { commitTarget())) .isInstanceOf(SparkException.class) .hasMessageContaining("Null value appeared in non-nullable field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2533,7 +2533,7 @@ public void testMergeWithInvalidAssignmentsAnsi() { commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2544,7 +2544,7 @@ public void testMergeWithInvalidAssignmentsAnsi() { .isInstanceOf(AnalysisException.class) .hasMessageEndingWith("Cannot safely cast `s`.`n1` \"STRING\" to \"INT\"."); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2570,7 +2570,7 @@ public void testMergeWithInvalidAssignmentsStrict() { withSQLConf( ImmutableMap.of(SQLConf.STORE_ASSIGNMENT_POLICY().key(), "strict"), () -> { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2580,7 +2580,7 @@ public void testMergeWithInvalidAssignmentsStrict() { commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast `id` \"VOID\" to \"INT\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2591,7 +2591,7 @@ public void testMergeWithInvalidAssignmentsStrict() { .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast `s`.`n1` \"VOID\" to \"INT\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2601,7 +2601,7 @@ public void testMergeWithInvalidAssignmentsStrict() { commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2612,7 +2612,7 @@ public void testMergeWithInvalidAssignmentsStrict() { .isInstanceOf(AnalysisException.class) .hasMessageEndingWith("Cannot safely cast `s`.`n1` \"STRING\" to \"INT\"."); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2631,7 +2631,7 @@ public void testMergeWithNonDeterministicConditions() { "id INT, c STRUCT>", "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2642,7 +2642,7 @@ public void testMergeWithNonDeterministicConditions() { .isInstanceOf(AnalysisException.class) .hasMessageContaining( "MERGE operation contains unsupported SEARCH condition. Non-deterministic expressions are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2654,7 +2654,7 @@ public void testMergeWithNonDeterministicConditions() { .hasMessageContaining( "MERGE operation contains unsupported UPDATE condition. Non-deterministic expressions are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2666,7 +2666,7 @@ public void testMergeWithNonDeterministicConditions() { .hasMessageContaining( "MERGE operation contains unsupported DELETE condition. Non-deterministic expressions are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2685,7 +2685,7 @@ public void testMergeWithAggregateExpressions() { "id INT, c STRUCT>", "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2697,7 +2697,7 @@ public void testMergeWithAggregateExpressions() { .hasMessageContaining( "MERGE operation contains unsupported SEARCH condition. Aggregates are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2709,7 +2709,7 @@ public void testMergeWithAggregateExpressions() { .hasMessageContaining( "MERGE operation contains unsupported UPDATE condition. Aggregates are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2721,7 +2721,7 @@ public void testMergeWithAggregateExpressions() { .hasMessageContaining( "MERGE operation contains unsupported DELETE condition. Aggregates are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2740,7 +2740,7 @@ public void testMergeWithSubqueriesInConditions() { "id INT, c STRUCT>", "{ \"id\": 1, \"c\": { \"n1\": 2, \"n2\": { \"dn1\": 3, \"dn2\": 4 } } }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2752,7 +2752,7 @@ public void testMergeWithSubqueriesInConditions() { .hasMessageContaining( "MERGE operation contains unsupported SEARCH condition. Subqueries are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2764,7 +2764,7 @@ public void testMergeWithSubqueriesInConditions() { .hasMessageContaining( "MERGE operation contains unsupported UPDATE condition. Subqueries are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2776,7 +2776,7 @@ public void testMergeWithSubqueriesInConditions() { .hasMessageContaining( "MERGE operation contains unsupported DELETE condition. Subqueries are not allowed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2794,7 +2794,7 @@ public void testMergeWithTargetColumnsInInsertConditions() { createAndInitTable("id INT, c2 INT", "{ \"id\": 1, \"c2\": 2 }"); createOrReplaceView("source", "{ \"id\": 1, \"value\": 11 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s " @@ -2811,7 +2811,7 @@ public void testMergeWithNonIcebergTargetTableNotSupported() { createOrReplaceView("target", "{ \"c1\": -100, \"c2\": -200 }"); createOrReplaceView("source", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO target t USING source s " @@ -2875,7 +2875,7 @@ public void testMergeNonExistingBranch() { // Coalesce forces our source into a SinglePartition distribution spark.range(0, 5).coalesce(1).createOrReplaceTempView("source"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s ON t.id = s.id " @@ -2962,7 +2962,7 @@ public void testMergeToWapBranchWithTableBranchIdentifier() { withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "MERGE INTO %s t USING source s ON t.id = s.id " 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 0d6f3f395f01..589543f0375a 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.spy; import static org.mockito.Mockito.when; @@ -43,7 +44,6 @@ import org.apache.iceberg.util.SnapshotUtil; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.connector.catalog.Identifier; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -175,8 +175,7 @@ public void testCommitUnknownException() { TestSparkCatalog.setTable(ident, sparkTable); // Although an exception is thrown here, write and commit have succeeded - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) + assertThatThrownBy(() -> sql("DELETE FROM %s WHERE id = 2", "dummy_catalog.default.table")) .isInstanceOf(CommitStateUnknownException.class) .hasMessageStartingWith("Datacenter on Fire"); 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 26a888356ca6..735a3bdee863 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; 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; @@ -29,7 +30,6 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -146,7 +146,7 @@ public void testMigrateWithInvalidMetricsConfig() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", tableName, location); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { String props = "map('write.metadata.metrics.column.x', 'X')"; sql("CALL %s.system.migrate('%s', %s)", catalogName, tableName, props); @@ -180,16 +180,15 @@ public void testMigrateWithConflictingProps() throws IOException { @TestTemplate public void testInvalidMigrateCases() { - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate(map('foo','bar'))", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.migrate('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 7121c59df541..cfca0c068c19 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.Snapshot; @@ -30,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -151,7 +151,7 @@ public void testApplyWapChangesRefreshesRelationCache() { public void testApplyInvalidWapId() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.publish_changes('%s', 'not_valid')", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot apply unknown WAP ID 'not_valid'"); @@ -159,23 +159,22 @@ public void testApplyInvalidWapId() { @TestTemplate public void testInvalidApplyWapChangesCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.publish_changes('n', table => 't', 'not_valid')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.custom.publish_changes('n', 't', 'not_valid')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.publish_changes not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [wap_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.publish_changes('', 'not_valid')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 76eef6a73b98..303e6b13e0df 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 @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.GC_ENABLED; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -64,7 +65,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -215,7 +215,7 @@ public void testRemoveOrphanFilesGCDisabled() { sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' 'false')", tableName, GC_ENABLED); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('%s')", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage( @@ -246,26 +246,24 @@ public void testRemoveOrphanFilesWap() { @TestTemplate public void testInvalidRemoveOrphanFilesCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.remove_orphan_files('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.remove_orphan_files('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.remove_orphan_files not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('n', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for older_than"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.remove_orphan_files('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } @@ -332,7 +330,7 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { public void testConcurrentRemoveOrphanFilesWithInvalidInput() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", @@ -340,7 +338,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("max_concurrent_deletes should have value > 0, value: 0"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', max_concurrent_deletes => %s)", @@ -351,7 +349,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { String tempViewName = "file_list_test"; spark.emptyDataFrame().createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -364,7 +362,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -377,7 +375,7 @@ public void testConcurrentRemoveOrphanFilesWithInvalidInput() { .toDF("file_path", "last_modified") .createOrReplaceTempView(tempViewName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(table => '%s', file_list_view => '%s')", @@ -652,7 +650,7 @@ public void testRemoveOrphanFilesProcedureWithPrefixMode() assertThat(orphanFiles).isEmpty(); // Test with no equal schemes - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(" @@ -735,7 +733,7 @@ public void testRemoveOrphanFilesProcedureWithEqualAuthorities() assertThat(orphanFiles).isEmpty(); // Test with no equal authorities - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.remove_orphan_files(" diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java index 0899cd2870c3..414d5abc8792 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestReplaceBranch.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.Assertions.assertThatThrownBy; import java.util.List; import java.util.concurrent.TimeUnit; @@ -32,7 +33,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +73,7 @@ public void testReplaceBranchFailsForTag() throws NoSuchTableException { df.writeTo(tableName).append(); long second = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", @@ -127,7 +127,7 @@ public void testReplaceBranchDoesNotExist() throws NoSuchTableException { df.writeTo(tableName).append(); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE BRANCH %s AS OF VERSION %d", diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java index 43368ab20e0a..fe1c38482bed 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestRequiredDistributionAndOrdering.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.math.BigDecimal; import java.util.List; import org.apache.iceberg.ParameterizedTestExtension; @@ -27,7 +29,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -183,7 +184,7 @@ public void testDisabledDistributionAndOrdering() { Dataset inputDF = ds.coalesce(1).sortWithinPartitions("c1"); // should fail if ordering is disabled - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> inputDF .writeTo(tableName) 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 b7ae0ace5ceb..4a20521c48bb 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; 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.Arrays; @@ -46,7 +47,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; @@ -598,7 +598,7 @@ public void testRewriteDataFilesWithInvalidInputs() { insertData(2); // Test for invalid strategy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', options => map('min-input-files','2'), " @@ -608,7 +608,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("unsupported strategy: temp. Only binpack or sort is supported"); // Test for sort_order with binpack strategy - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'binpack', " @@ -618,7 +618,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Must use only one rewriter type (bin-pack, sort, zorder)"); // Test for sort strategy without any (default/user defined) sort_order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort')", @@ -627,7 +627,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageStartingWith("Cannot sort data without a valid sort order"); // Test for sort_order with invalid null order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -637,7 +637,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Unable to parse sortOrder: c1 ASC none"); // Test for sort_order with invalid sort direction - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -647,7 +647,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Unable to parse sortOrder: c1 none NULLS FIRST"); // Test for sort_order with invalid column name - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -657,7 +657,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessageStartingWith("Cannot find field 'col1' in struct:"); // Test with invalid filter column col1 - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', " + "where => 'col1 = 3')", @@ -666,7 +666,7 @@ public void testRewriteDataFilesWithInvalidInputs() { .hasMessage("Cannot parse predicates in where option: col1 = 3"); // Test for z_order with invalid column name - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -678,7 +678,7 @@ public void testRewriteDataFilesWithInvalidInputs() { + "struct<1: c1: optional int, 2: c2: optional string, 3: c3: optional string>"); // Test for z_order with sort_order - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', strategy => 'sort', " @@ -691,26 +691,25 @@ public void testRewriteDataFilesWithInvalidInputs() { @TestTemplate public void testInvalidCasesForRewriteDataFiles() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_data_files('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rewrite_data_files not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_data_files(table => 't', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageEndingWith("Duplicate procedure argument: table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_data_files('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for parameter 'table'"); } @@ -866,7 +865,7 @@ public void testDefaultSortOrder() { @TestTemplate public void testRewriteWithUntranslatedOrUnconvertedFilter() { createTable(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', where => 'substr(encode(c2, \"utf-8\"), 2) = \"fo\"')", @@ -874,7 +873,7 @@ public void testRewriteWithUntranslatedOrUnconvertedFilter() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot translate Spark expression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rewrite_data_files(table => '%s', where => 'substr(c2, 2) = \"fo\"')", 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 e527eadeb081..83ec1ef84dfc 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.sql.Date; import java.sql.Timestamp; @@ -32,7 +33,6 @@ import org.apache.spark.sql.RowFactory; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -287,31 +287,29 @@ public void testRewriteManifestsCaseInsensitiveArgs() { @TestTemplate public void testInvalidRewriteManifestsCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests('n', table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rewrite_manifests('n', 't')", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rewrite_manifests not found"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests()", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('n', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for use_caching"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rewrite_manifests(table => 't', tAbLe => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Could not build name to arg map: Duplicate procedure argument: table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rewrite_manifests('')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 ba833bcee35e..315f6c4c1b2b 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -30,7 +31,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -237,7 +237,7 @@ public void testRollbackToSnapshotWithoutExplicitCatalog() { public void testRollbackToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.rollback_to_snapshot('%s', -1L)", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot roll back to unknown snapshot id: -1"); @@ -245,7 +245,7 @@ public void testRollbackToInvalidSnapshot() { @TestTemplate public void testInvalidRollbackToSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_snapshot(namespace => 'n1', table => 't', 1L)", @@ -253,32 +253,27 @@ public void testInvalidRollbackToSnapshotCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.rollback_to_snapshot('n', 't', 1L)", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rollback_to_snapshot not found"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(1L)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot(table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [snapshot_id]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for snapshot_id: cannot cast DecimalType(2,1) to LongType"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_snapshot('', 1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 d9366c08dea3..08e6b1323c79 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.time.LocalDateTime; @@ -29,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchProcedureException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -250,7 +250,7 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp(namespace => 'n1', 't', %s)", @@ -258,17 +258,16 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.custom.rollback_to_timestamp('n', 't', %s)", catalogName, timestamp)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.rollback_to_timestamp not found"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [timestamp]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp(timestamp => %s)", @@ -276,12 +275,11 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp(table => 't')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [timestamp]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.rollback_to_timestamp('n', 't', %s, 1L)", @@ -289,8 +287,7 @@ public void testInvalidRollbackToTimestampCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Too many arguments for procedure"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.rollback_to_timestamp('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for timestamp: cannot cast DecimalType(2,1) to TimestampType"); } 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 52278026f829..1133d74e668a 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; import static org.apache.iceberg.TableProperties.WRITE_AUDIT_PUBLISH_ENABLED; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; import java.util.List; @@ -30,7 +31,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -186,7 +186,7 @@ public void tesSetCurrentSnapshotWithoutExplicitCatalog() { public void testSetCurrentSnapshotToInvalidSnapshot() { sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.set_current_snapshot('%s', -1L)", catalogName, tableIdent)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot roll back to unknown snapshot id: -1"); @@ -194,7 +194,7 @@ public void testSetCurrentSnapshotToInvalidSnapshot() { @TestTemplate public void testInvalidRollbackToSnapshotCases() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(namespace => 'n1', table => 't', 1L)", @@ -202,41 +202,36 @@ public void testInvalidRollbackToSnapshotCases() { .isInstanceOf(AnalysisException.class) .hasMessage("Named and positional arguments cannot be mixed"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.custom.set_current_snapshot('n', 't', 1L)", catalogName)) .isInstanceOf(NoSuchProcedureException.class) .hasMessage("Procedure custom.set_current_snapshot not found"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Either snapshot_id or ref must be provided, not both"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse identifier for arg table: 1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.set_current_snapshot(snapshot_id => 1L)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot(table => 't')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Either snapshot_id or ref must be provided, not both"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('t', 2.2)", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Wrong arg type for snapshot_id: cannot cast DecimalType(2,1) to LongType"); - Assertions.assertThatThrownBy( - () -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.set_current_snapshot('', 1L)", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(table => 't', snapshot_id => 1L, ref => 's1')", @@ -281,7 +276,7 @@ public void testSetCurrentSnapshotToRef() { sql("SELECT * FROM %s ORDER BY id", tableName)); String notExistRef = "s2"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.set_current_snapshot(table => '%s', ref => '%s')", 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 ce49a1630201..77b7797fe192 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.expressions.Expressions.bucket; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.NullOrder; import org.apache.iceberg.ParameterizedTestExtension; @@ -28,7 +29,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -73,7 +73,7 @@ public void testSetWriteOrderWithCaseSensitiveColumnNames() { Table table = validationCatalog.loadTable(tableIdent); assertThat(table.sortOrder().isUnsorted()).as("Table should start unsorted").isTrue(); sql("SET %s=true", SQLConf.CASE_SENSITIVE().key()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { sql("ALTER TABLE %s WRITE ORDERED BY category, 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 69bc35e9ea33..3b093947cacc 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.extensions; 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; @@ -29,7 +30,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.spark.sql.AnalysisException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -197,16 +197,16 @@ public void testInvalidSnapshotsCases() throws IOException { "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", sourceName, location); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessage("Missing required parameters: [table]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("CALL %s.system.snapshot('n', 't', map('foo', 'bar'))", catalogName)) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Wrong arg type for location"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", @@ -215,11 +215,11 @@ public void testInvalidSnapshotsCases() throws IOException { .hasMessageContaining( "The `map` requires 2n (n > 0) parameters but the actual number is 3"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('', 'dest')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument source_table"); - Assertions.assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) + assertThatThrownBy(() -> sql("CALL %s.system.snapshot('src', '')", catalogName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } 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 7f33d302fbf8..65c2c0f713cb 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 @@ -19,6 +19,7 @@ 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.Locale; @@ -35,7 +36,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.catalyst.parser.extensions.IcebergParseException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -93,7 +93,7 @@ public void testCreateTagWithRetain() throws NoSuchTableException { } String tagName = "t1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN", @@ -101,12 +101,12 @@ public void testCreateTagWithRetain() throws NoSuchTableException { .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s RETAIN %s DAYS", tableName, tagName, "abc")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s CREATE TAG %s AS OF VERSION %d RETAIN %d SECONDS", @@ -117,7 +117,7 @@ public void testCreateTagWithRetain() throws NoSuchTableException { @TestTemplate public void testCreateTagOnEmptyTable() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "abc")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Cannot complete create or replace tag operation on %s, main has no snapshot", @@ -130,7 +130,7 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { long snapshotId = table.currentSnapshot().snapshotId(); String tagName = "t1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s CREATE TAG %s AS OF VERSION %d", tableName, tagName, -1)) .isInstanceOf(ValidationException.class) .hasMessage("Cannot set " + tagName + " to unknown snapshot: -1"); @@ -145,11 +145,11 @@ public void testCreateTagUseDefaultConfig() throws NoSuchTableException { .as("The tag needs to have the default max ref age, which is null.") .isNull(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, tagName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("already exists"); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s CREATE TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); @@ -197,8 +197,7 @@ public void testReplaceTagFailsForBranch() throws NoSuchTableException { insertRows(); long second = table.currentSnapshot().snapshotId(); - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) + assertThatThrownBy(() -> sql("ALTER TABLE %s REPLACE Tag %s", tableName, branchName, second)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref branch1 is a branch not a tag"); } @@ -233,7 +232,7 @@ public void testReplaceTag() throws NoSuchTableException { public void testReplaceTagDoesNotExist() throws NoSuchTableException { Table table = insertRows(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "ALTER TABLE %s REPLACE Tag %s AS OF VERSION %d", @@ -302,15 +301,14 @@ public void testDropTag() throws NoSuchTableException { @TestTemplate public void testDropTagNonConformingName() { - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "123")) .isInstanceOf(IcebergParseException.class) .hasMessageContaining("mismatched input '123'"); } @TestTemplate public void testDropTagDoesNotExist() { - Assertions.assertThatThrownBy( - () -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, "nonExistingTag")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Tag does not exist: nonExistingTag"); } @@ -321,7 +319,7 @@ public void testDropTagFailesForBranch() throws NoSuchTableException { Table table = insertRows(); table.manageSnapshots().createBranch(branchName, table.currentSnapshot().snapshotId()).commit(); - Assertions.assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) + assertThatThrownBy(() -> sql("ALTER TABLE %s DROP TAG %s", tableName, branchName)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Ref b1 is a branch not a tag"); } 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 09ec61a6ad4e..09aa51f0460a 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 @@ -32,6 +32,7 @@ import static org.apache.iceberg.TableProperties.UPDATE_MODE_DEFAULT; import static org.apache.spark.sql.functions.lit; 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.Arrays; @@ -73,7 +74,6 @@ import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.execution.SparkPlan; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; @@ -297,8 +297,7 @@ public void testUpdateNonExistingCustomBranch() { assumeThat(branch).as("Test only applicable to custom branch").isEqualTo("test"); createAndInitTable("id INT, dep STRING"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET dep = 'invalid' WHERE id IN (1)", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage("Cannot use branch (does not exist): test"); } @@ -690,7 +689,7 @@ public synchronized void testUpdateWithSerializableIsolation() throws Interrupte }); try { - Assertions.assertThatThrownBy(updateFuture::get) + assertThatThrownBy(updateFuture::get) .isInstanceOf(ExecutionException.class) .cause() .isInstanceOf(ValidationException.class) @@ -1318,11 +1317,11 @@ public void testUpdateWithInvalidUpdates() { "id INT, a ARRAY>, m MAP", "{ \"id\": 0, \"a\": null, \"m\": null }"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET a.c1 = 1", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for StructType"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET m.key = 'new_key'", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Updating nested fields is only supported for StructType"); } @@ -1332,17 +1331,16 @@ public void testUpdateWithConflictingAssignments() { createAndInitTable( "id INT, c STRUCT>", "{ \"id\": 0, \"s\": null }"); - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = 1, t.c.n1 = 2, t.id = 2", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Multiple assignments for 'id'"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.c.n1 = 1, t.id = 2, t.c.n1 = 2", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Multiple assignments for 'c.n1"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "UPDATE %s SET c.n1 = 1, c = named_struct('n1', 1, 'n2', named_struct('dn1', 1, 'dn2', 2))", @@ -1360,24 +1358,24 @@ public void testUpdateWithInvalidAssignmentsAnsi() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", "ansi"), () -> { - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) .isInstanceOf(SparkException.class) .hasMessageContaining("Null value appeared in non-nullable field"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) .isInstanceOf(SparkException.class) .hasMessageContaining("Null value appeared in non-nullable field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot find data for the output column `s`.`n2`"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "UPDATE %s t SET t.s.n2 = named_struct('dn3', 1, 'dn1', 2)", @@ -1396,24 +1394,24 @@ public void testUpdateWithInvalidAssignmentsStrict() { withSQLConf( ImmutableMap.of("spark.sql.storeAssignmentPolicy", "strict"), () -> { - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.id = NULL", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast `id` \"VOID\" to \"INT\""); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = NULL", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast `s`.`n1` \"VOID\" to \"INT\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s t SET t.s = named_struct('n1', 1)", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot find data for the output column"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s t SET t.s.n1 = 'str'", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("Cannot safely cast"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "UPDATE %s t SET t.s.n2 = named_struct('dn3', 1, 'dn1', 2)", @@ -1427,7 +1425,7 @@ public void testUpdateWithInvalidAssignmentsStrict() { public void testUpdateWithNonDeterministicCondition() { createAndInitTable("id INT, dep STRING", "{ \"id\": 1, \"dep\": \"hr\" }"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("UPDATE %s SET id = -1 WHERE id = 1 AND rand() > 0.5", commitTarget())) .isInstanceOf(AnalysisException.class) .hasMessageContaining("The operator expects a deterministic expression"); @@ -1437,7 +1435,7 @@ public void testUpdateWithNonDeterministicCondition() { public void testUpdateOnNonIcebergTableNotSupported() { createOrReplaceView("testtable", "{ \"c1\": -100, \"c2\": -200 }"); - Assertions.assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) + assertThatThrownBy(() -> sql("UPDATE %s SET c1 = -1 WHERE c2 = 1", "testtable")) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("UPDATE TABLE is not supported temporarily."); } @@ -1495,8 +1493,7 @@ public void testUpdateToWapBranchWithTableBranchIdentifier() { withSQLConf( ImmutableMap.of(SparkSQLProperties.WAP_BRANCH, "wap"), () -> - Assertions.assertThatThrownBy( - () -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) + assertThatThrownBy(() -> sql("UPDATE %s SET dep='hr' WHERE dep='a'", commitTarget())) .isInstanceOf(ValidationException.class) .hasMessage( String.format( diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java index a40e66f5bf2b..68b17ef36ff3 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestWriteAborts.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.spark.extensions; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.nio.file.Files; import java.util.List; @@ -41,7 +43,6 @@ import org.apache.spark.SparkException; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -101,7 +102,7 @@ public void testBatchAppend() throws IOException { new SimpleRecord(4, "b")); Dataset inputDF = spark.createDataFrame(records, SimpleRecord.class); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> // incoming records are not ordered by partitions so the job must fail inputDF diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index ee735a4f84a8..89f7ad5dc62f 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.nio.file.Files; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.spark.extensions.ExtensionsTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -46,7 +47,7 @@ public void testGettingStarted() throws IOException { // Writing sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b'), (3, 'c')", tableName); - Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) .as("Should have inserted 3 rows") .isEqualTo(3L); @@ -57,7 +58,7 @@ public void testGettingStarted() throws IOException { sql("INSERT INTO source VALUES (10, 'd'), (11, 'ee')"); sql("INSERT INTO %s SELECT id, data FROM source WHERE length(data) = 1", tableName); - Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) .as("Table should now have 4 rows") .isEqualTo(4L); @@ -72,15 +73,15 @@ public void testGettingStarted() throws IOException { + "WHEN MATCHED THEN UPDATE SET t.data = u.data\n" + "WHEN NOT MATCHED THEN INSERT *", tableName); - Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) + assertThat(scalarSql("SELECT COUNT(*) FROM %s", tableName)) .as("Table should now have 5 rows") .isEqualTo(5L); - Assertions.assertThat(scalarSql("SELECT data FROM %s WHERE id = 1", tableName)) + assertThat(scalarSql("SELECT data FROM %s WHERE id = 1", tableName)) .as("Record 1 should now have data x") .isEqualTo("x"); // Reading - Assertions.assertThat( + assertThat( scalarSql( "SELECT count(1) as count FROM %s WHERE data = 'x' GROUP BY data ", tableName)) .as("There should be 2 records with data x") @@ -88,7 +89,7 @@ public void testGettingStarted() throws IOException { // Not supported because of Spark limitation if (!catalogName.equals("spark_catalog")) { - Assertions.assertThat(scalarSql("SELECT COUNT(*) FROM %s.snapshots", tableName)) + assertThat(scalarSql("SELECT COUNT(*) FROM %s.snapshots", tableName)) .as("There should be 3 snapshots") .isEqualTo(3L); } @@ -107,9 +108,7 @@ public void testAlterTable() { sql("ALTER TABLE %s ADD PARTITION FIELD years(ts)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, category) AS shard", tableName); table = getTable(); - Assertions.assertThat(table.spec().fields()) - .as("Table should have 4 partition fields") - .hasSize(4); + assertThat(table.spec().fields()).as("Table should have 4 partition fields").hasSize(4); // Drop Examples sql("ALTER TABLE %s DROP PARTITION FIELD bucket(16, id)", tableName); @@ -118,18 +117,14 @@ public void testAlterTable() { sql("ALTER TABLE %s DROP PARTITION FIELD shard", tableName); table = getTable(); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should be unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table should be unpartitioned").isTrue(); // Sort order examples sql("ALTER TABLE %s WRITE ORDERED BY category, id", tableName); sql("ALTER TABLE %s WRITE ORDERED BY category ASC, id DESC", tableName); sql("ALTER TABLE %s WRITE ORDERED BY category ASC NULLS LAST, id DESC NULLS FIRST", tableName); table = getTable(); - Assertions.assertThat(table.sortOrder().fields()) - .as("Table should be sorted on 2 fields") - .hasSize(2); + assertThat(table.sortOrder().fields()).as("Table should be sorted on 2 fields").hasSize(2); } @TestTemplate @@ -155,9 +150,7 @@ public void testCreateTable() { + "PARTITIONED BY (category)", tableName("second")); Table second = getTable("second"); - Assertions.assertThat(second.spec().fields()) - .as("Should be partitioned on 1 column") - .hasSize(1); + assertThat(second.spec().fields()).as("Should be partitioned on 1 column").hasSize(1); sql( "CREATE TABLE %s (\n" @@ -169,16 +162,14 @@ public void testCreateTable() { + "PARTITIONED BY (bucket(16, id), days(ts), category)", tableName("third")); Table third = getTable("third"); - Assertions.assertThat(third.spec().fields()) - .as("Should be partitioned on 3 columns") - .hasSize(3); + assertThat(third.spec().fields()).as("Should be partitioned on 3 columns").hasSize(3); } @TestTemplate public void showView() { sql("DROP VIEW IF EXISTS %s", "test"); sql("CREATE VIEW %s AS SELECT 1 AS id", "test"); - Assertions.assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); + assertThat(sql("SHOW VIEWS")).contains(row("default", "test", false)); } private Table getTable(String name) { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java index 6fa8da841171..0b3d0244a087 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/SparkTestHelperBase.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.spark; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import java.util.stream.IntStream; import org.apache.spark.sql.Row; -import org.assertj.core.api.Assertions; public class SparkTestHelperBase { protected static final Object ANY = new Object(); @@ -55,13 +56,13 @@ private Object[] toJava(Row row) { protected void assertEquals( String context, List expectedRows, List actualRows) { - Assertions.assertThat(actualRows) + assertThat(actualRows) .as(context + ": number of results should match") .hasSameSizeAs(expectedRows); for (int row = 0; row < expectedRows.size(); row += 1) { Object[] expected = expectedRows.get(row); Object[] actual = actualRows.get(row); - Assertions.assertThat(actual).as("Number of columns should match").hasSameSizeAs(expected); + assertThat(actual).as("Number of columns should match").hasSameSizeAs(expected); for (int col = 0; col < actualRows.get(row).length; col += 1) { String newContext = String.format("%s: row %d col %d", context, row + 1, col + 1); assertEquals(newContext, expected, actual); @@ -70,23 +71,19 @@ protected void assertEquals( } protected void assertEquals(String context, Object[] expectedRow, Object[] actualRow) { - Assertions.assertThat(actualRow) - .as("Number of columns should match") - .hasSameSizeAs(expectedRow); + assertThat(actualRow).as("Number of columns should match").hasSameSizeAs(expectedRow); for (int col = 0; col < actualRow.length; col += 1) { Object expectedValue = expectedRow[col]; Object actualValue = actualRow[col]; if (expectedValue != null && expectedValue.getClass().isArray()) { String newContext = String.format("%s (nested col %d)", context, col + 1); if (expectedValue instanceof byte[]) { - Assertions.assertThat(actualValue).as(newContext).isEqualTo(expectedValue); + assertThat(actualValue).as(newContext).isEqualTo(expectedValue); } else { assertEquals(newContext, (Object[]) expectedValue, (Object[]) actualValue); } } else if (expectedValue != ANY) { - Assertions.assertThat(actualValue) - .as(context + " contents should match") - .isEqualTo(expectedValue); + assertThat(actualValue).as(context + " contents should match").isEqualTo(expectedValue); } } } 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 776a8890b5eb..16082d78f0cd 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 @@ -109,7 +109,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.internal.SQLConf; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -1556,7 +1555,7 @@ public void testBinPackRewriterWithSpecificOutputSpec() { public void testBinpackRewriteWithInvalidOutputSpecId() { Table table = createTable(10); shouldHaveFiles(table, 10); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> actions() .rewriteDataFiles(table) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java index 0da6bdb3622b..e223d2e16411 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSparkFileRewriter.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.spark.actions; +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.FileScanTask; @@ -37,7 +40,6 @@ import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StringType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; @@ -109,9 +111,9 @@ private void checkDataFileSizeFiltering(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); + assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assertions.assertThat(group).as("Must rewrite 2 files").hasSize(2); + assertThat(group).as("Must rewrite 2 files").hasSize(2); } private void checkDataFilesDeleteThreshold(SizeBasedDataRewriter rewriter) { @@ -128,9 +130,9 @@ private void checkDataFilesDeleteThreshold(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); + assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assertions.assertThat(group).as("Must rewrite 1 file").hasSize(1); + assertThat(group).as("Must rewrite 1 file").hasSize(1); } private void checkDataFileGroupWithEnoughFiles(SizeBasedDataRewriter rewriter) { @@ -151,9 +153,9 @@ private void checkDataFileGroupWithEnoughFiles(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); + assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assertions.assertThat(group).as("Must rewrite 4 files").hasSize(4); + assertThat(group).as("Must rewrite 4 files").hasSize(4); } private void checkDataFileGroupWithEnoughData(SizeBasedDataRewriter rewriter) { @@ -171,9 +173,9 @@ private void checkDataFileGroupWithEnoughData(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); + assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assertions.assertThat(group).as("Must rewrite 3 files").hasSize(3); + assertThat(group).as("Must rewrite 3 files").hasSize(3); } private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { @@ -189,25 +191,24 @@ private void checkDataFileGroupWithTooMuchData(SizeBasedDataRewriter rewriter) { rewriter.init(options); Iterable> groups = rewriter.planFileGroups(tasks); - Assertions.assertThat(groups).as("Must have 1 group").hasSize(1); + assertThat(groups).as("Must have 1 group").hasSize(1); List group = Iterables.getOnlyElement(groups); - Assertions.assertThat(group).as("Must rewrite big file").hasSize(1); + assertThat(group).as("Must rewrite big file").hasSize(1); } @Test public void testInvalidConstructorUsagesSortData() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); - Assertions.assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table)) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("is unsorted and no sort order is provided"); - Assertions.assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, null)) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("the provided sort order is null or empty"); - Assertions.assertThatThrownBy( - () -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) + assertThatThrownBy(() -> new SparkSortDataRewriter(spark, table, SortOrder.unsorted())) .hasMessageContaining("Cannot sort data without a valid sort order") .hasMessageContaining("the provided sort order is null or empty"); } @@ -216,20 +217,17 @@ public void testInvalidConstructorUsagesSortData() { public void testInvalidConstructorUsagesZOrderData() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA, SPEC); - Assertions.assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, null)) .hasMessageContaining("Cannot ZOrder when no columns are specified"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of())) .hasMessageContaining("Cannot ZOrder when no columns are specified"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("dep"))) .hasMessageContaining("Cannot ZOrder") .hasMessageContaining("all columns provided were identity partition columns"); - Assertions.assertThatThrownBy( - () -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) + assertThatThrownBy(() -> new SparkZOrderDataRewriter(spark, table, ImmutableList.of("DeP"))) .hasMessageContaining("Cannot ZOrder") .hasMessageContaining("all columns provided were identity partition columns"); } @@ -239,7 +237,7 @@ public void testBinPackDataValidOptions() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); SparkBinPackDataRewriter rewriter = new SparkBinPackDataRewriter(spark, table); - Assertions.assertThat(rewriter.validOptions()) + assertThat(rewriter.validOptions()) .as("Rewriter must report all supported options") .isEqualTo( ImmutableSet.of( @@ -257,7 +255,7 @@ public void testSortDataValidOptions() { Table table = catalog.createTable(TABLE_IDENT, SCHEMA); SparkSortDataRewriter rewriter = new SparkSortDataRewriter(spark, table, SORT_ORDER); - Assertions.assertThat(rewriter.validOptions()) + assertThat(rewriter.validOptions()) .as("Rewriter must report all supported options") .isEqualTo( ImmutableSet.of( @@ -278,7 +276,7 @@ public void testZOrderDataValidOptions() { ImmutableList zOrderCols = ImmutableList.of("id"); SparkZOrderDataRewriter rewriter = new SparkZOrderDataRewriter(spark, table, zOrderCols); - Assertions.assertThat(rewriter.validOptions()) + assertThat(rewriter.validOptions()) .as("Rewriter must report all supported options") .isEqualTo( ImmutableSet.of( @@ -304,7 +302,7 @@ public void testInvalidValuesForBinPackDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); } @@ -317,12 +315,12 @@ public void testInvalidValuesForSortDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); Map invalidCompressionFactorOptions = ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) + assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); } @@ -336,23 +334,23 @@ public void testInvalidValuesForZOrderDataOptions() { Map invalidDeleteThresholdOptions = ImmutableMap.of(SizeBasedDataRewriter.DELETE_FILE_THRESHOLD, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) + assertThatThrownBy(() -> rewriter.init(invalidDeleteThresholdOptions)) .hasMessageContaining("'delete-file-threshold' is set to -1 but must be >= 0"); Map invalidCompressionFactorOptions = ImmutableMap.of(SparkShufflingDataRewriter.COMPRESSION_FACTOR, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) + assertThatThrownBy(() -> rewriter.init(invalidCompressionFactorOptions)) .hasMessageContaining("'compression-factor' is set to 0.0 but must be > 0"); Map invalidMaxOutputOptions = ImmutableMap.of(SparkZOrderDataRewriter.MAX_OUTPUT_SIZE, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMaxOutputOptions)) .hasMessageContaining("Cannot have the interleaved ZOrder value use less than 1 byte") .hasMessageContaining("'max-output-size' was set to 0"); Map invalidVarLengthContributionOptions = ImmutableMap.of(SparkZOrderDataRewriter.VAR_LENGTH_CONTRIBUTION, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) + assertThatThrownBy(() -> rewriter.init(invalidVarLengthContributionOptions)) .hasMessageContaining("Cannot use less than 1 byte for variable length types with ZOrder") .hasMessageContaining("'var-length-contribution' was set to 0"); } @@ -360,19 +358,19 @@ public void testInvalidValuesForZOrderDataOptions() { private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewriter) { Map invalidTargetSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetSizeOptions)) .hasMessageContaining("'target-file-size-bytes' is set to 0 but must be > 0"); Map invalidMinSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "-1"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMinSizeOptions)) .hasMessageContaining("'min-file-size-bytes' is set to -1 but must be >= 0"); Map invalidTargetMinSizeOptions = ImmutableMap.of( SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "3", SizeBasedFileRewriter.MIN_FILE_SIZE_BYTES, "5"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetMinSizeOptions)) .hasMessageContaining("'target-file-size-bytes' (3) must be > 'min-file-size-bytes' (5)") .hasMessageContaining("all new files will be smaller than the min threshold"); @@ -380,18 +378,18 @@ private void validateSizeBasedRewriterOptions(SizeBasedFileRewriter rewrit ImmutableMap.of( SizeBasedFileRewriter.TARGET_FILE_SIZE_BYTES, "5", SizeBasedFileRewriter.MAX_FILE_SIZE_BYTES, "3"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidTargetMaxSizeOptions)) .hasMessageContaining("'target-file-size-bytes' (5) must be < 'max-file-size-bytes' (3)") .hasMessageContaining("all new files will be larger than the max threshold"); Map invalidMinInputFilesOptions = ImmutableMap.of(SizeBasedFileRewriter.MIN_INPUT_FILES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMinInputFilesOptions)) .hasMessageContaining("'min-input-files' is set to 0 but must be > 0"); Map invalidMaxFileGroupSizeOptions = ImmutableMap.of(SizeBasedFileRewriter.MAX_FILE_GROUP_SIZE_BYTES, "0"); - Assertions.assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) + assertThatThrownBy(() -> rewriter.init(invalidMaxFileGroupSizeOptions)) .hasMessageContaining("'max-file-group-size-bytes' is set to 0 but must be > 0"); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java index ee264cb090fd..38ce0d4d95f1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/functions/TestSparkFunctions.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.spark.functions; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.spark.sql.connector.catalog.functions.ScalarFunction; import org.apache.spark.sql.connector.catalog.functions.UnboundFunction; import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.DecimalType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestSparkFunctions { @@ -150,8 +151,8 @@ public void testBuildTruncateFunctionFromClass() { private void checkBuildFunc(ScalarFunction function, UnboundFunction expected) { UnboundFunction actual = SparkFunctions.loadFunctionByClass(function.getClass()); - Assertions.assertThat(actual).isNotNull(); - Assertions.assertThat(actual.name()).isEqualTo(expected.name()); - Assertions.assertThat(actual.description()).isEqualTo(expected.description()); + assertThat(actual).isNotNull(); + assertThat(actual.name()).isEqualTo(expected.name()); + assertThat(actual.description()).isEqualTo(expected.description()); } } 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 7fdd5163f11b..05515946c145 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 @@ -37,7 +37,6 @@ import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.TestBase; import org.apache.spark.sql.SparkSession; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; @@ -579,15 +578,9 @@ private void assertAggregates( Object actualMax = actual.get(0)[1]; Object actualMin = actual.get(0)[2]; - Assertions.assertThat(actualCount) - .as("Expected and actual count should equal") - .isEqualTo(expectedCount); - Assertions.assertThat(actualMax) - .as("Expected and actual max should equal") - .isEqualTo(expectedMax); - Assertions.assertThat(actualMin) - .as("Expected and actual min should equal") - .isEqualTo(expectedMin); + assertThat(actualCount).as("Expected and actual count should equal").isEqualTo(expectedCount); + assertThat(actualMax).as("Expected and actual max should equal").isEqualTo(expectedMax); + assertThat(actualMin).as("Expected and actual min should equal").isEqualTo(expectedMin); } private void assertExplainContains(List explain, String... expectedFragments) { @@ -595,7 +588,7 @@ private void assertExplainContains(List explain, String... expectedFra Arrays.stream(expectedFragments) .forEach( fragment -> - Assertions.assertThat(explainString) + assertThat(explainString) .as("Expected to find plan fragment in explain plan") .contains(fragment)); } 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 73be40e050dd..a2a315f5625f 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; 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; @@ -36,7 +37,6 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; import org.apache.spark.sql.connector.catalog.TableCatalog; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -114,27 +114,24 @@ public void testCreateTable() { @TestTemplate public void testCreateTablePartitionedByUUID() { - Assertions.assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); + assertThat(validationCatalog.tableExists(tableIdent)).isFalse(); Schema schema = new Schema(1, Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("uuid", 16).build(); validationCatalog.createTable(tableIdent, schema, spec); Table table = validationCatalog.loadTable(tableIdent); - Assertions.assertThat(table).isNotNull(); + assertThat(table).isNotNull(); StructType expectedSchema = StructType.of(Types.NestedField.optional(1, "uuid", Types.UUIDType.get())); - Assertions.assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); - Assertions.assertThat(table.spec().fields()).hasSize(1); + assertThat(table.schema().asStruct()).isEqualTo(expectedSchema); + assertThat(table.spec().fields()).hasSize(1); String uuid = UUID.randomUUID().toString(); sql("INSERT INTO %s VALUES('%s')", tableName, uuid); - Assertions.assertThat(sql("SELECT uuid FROM %s", tableName)) - .hasSize(1) - .element(0) - .isEqualTo(row(uuid)); + assertThat(sql("SELECT uuid FROM %s", tableName)).hasSize(1).element(0).isEqualTo(row(uuid)); } @TestTemplate @@ -177,7 +174,7 @@ public void testCreateTableUsingParquet() { .as("Should not have default format parquet") .isEqualTo("parquet"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "CREATE TABLE %s.default.fail (id BIGINT NOT NULL, data STRING) USING crocodile", @@ -414,7 +411,7 @@ public void testDowngradeTableToFormatV1ThroughTablePropertyFails() { .as("should create table using format v2") .isEqualTo(2); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='1')", tableName)) .cause() .isInstanceOf(IllegalArgumentException.class) diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java index a68f6e239b16..7706c5aad4de 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDeleteFrom.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.Table; @@ -29,7 +30,6 @@ import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -81,8 +81,7 @@ public void testDeleteFromTableAtSnapshot() throws NoSuchTableException { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( - () -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) + assertThatThrownBy(() -> sql("DELETE FROM %s.%s WHERE id < 4", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot delete from table at a specific snapshot"); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java index f7a0fd9637a0..ec8308e1c772 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestDropTable.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.List; @@ -30,7 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.CatalogTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -113,7 +113,7 @@ public void testPurgeTableGCDisabled() throws IOException { assertThat(manifestAndFiles).as("There should be 2 files for manifests and files").hasSize(2); assertThat(checkFilesExist(manifestAndFiles, true)).as("All files should exist").isTrue(); - Assertions.assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) + assertThatThrownBy(() -> sql("DROP TABLE %s PURGE", tableName)) .isInstanceOf(ValidationException.class) .hasMessageContaining( "Cannot purge table: GC is disabled (deleting files may corrupt other tables"); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java index 8661715f3e33..ae9aee8ca416 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; 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; @@ -35,7 +36,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.TestTemplate; @@ -144,7 +144,7 @@ public void testDropNonEmptyNamespace() { .as("Table should exist") .isTrue(); - Assertions.assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) + assertThatThrownBy(() -> sql("DROP NAMESPACE %s", fullNamespace)) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageStartingWith("Namespace db is not empty."); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java index 0504bbc24d44..45c0eb763653 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestPartitionedWritesToWapBranch.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.spark.sql; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.List; import java.util.UUID; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.spark.SparkSQLProperties; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -66,8 +67,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { Table table = validationCatalog.loadTable(tableIdent); table.manageSnapshots().createBranch("test2", table.refs().get(BRANCH).snapshotId()).commit(); sql("REFRESH TABLE " + tableName); - Assertions.assertThatThrownBy( - () -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) + assertThatThrownBy(() -> sql("INSERT INTO %s.branch_test2 VALUES (4, 'd')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot write to both branch and WAP branch, but got branch [test2] and WAP branch [%s]", @@ -78,7 +78,7 @@ public void testBranchAndWapBranchCannotBothBeSetForWrite() { public void testWapIdAndWapBranchCannotBothBeSetForWrite() { String wapId = UUID.randomUUID().toString(); spark.conf().set(SparkSQLProperties.WAP_ID, wapId); - Assertions.assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) + assertThatThrownBy(() -> sql("INSERT INTO %s VALUES (4, 'd')", tableName)) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot set both WAP ID and branch, but got ID [%s] and branch [%s]", wapId, BRANCH); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java index 907b91177125..7c1897250b6f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestSparkBucketFunction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; import java.nio.ByteBuffer; @@ -30,7 +31,6 @@ import org.apache.iceberg.types.Types; import org.apache.spark.sql.AnalysisException; import org.apache.spark.sql.types.DataTypes; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -187,17 +187,17 @@ public void testNumBucketsAcceptsShortAndByte() { @TestTemplate public void testWrongNumberOfArguments() { - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket()")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (): Wrong number of inputs (expected numBuckets and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int): Wrong number of inputs (expected numBuckets and value)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(1, 1L, 1)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, bigint, int): Wrong number of inputs (expected numBuckets and value)"); @@ -205,29 +205,27 @@ public void testWrongNumberOfArguments() { @TestTemplate public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(CAST('12.34' as DECIMAL(9, 2)), 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (decimal(9,2), int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(12L, 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (bigint, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket('5', 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (string, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(INTERVAL '100-00' YEAR TO MONTH, 10)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (interval year to month, int): Expected number of buckets to be tinyint, shortint or int"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.bucket(CAST('11 23:4:0' AS INTERVAL DAY TO SECOND), 10)")) .isInstanceOf(AnalysisException.class) @@ -237,37 +235,34 @@ public void testInvalidTypesCannotBeUsedForNumberOfBuckets() { @TestTemplate public void testInvalidTypesForBucketColumn() { - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as float))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, float): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, cast(12.3456 as double))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, double): Expected column to be date, tinyint, smallint, int, bigint, decimal, timestamp, string, or binary"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, true)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'bucket' cannot process input: (int, boolean)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, map(1, 1))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'bucket' cannot process input: (int, map)"); - Assertions.assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, array(1L))")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith("Function 'bucket' cannot process input: (int, array)"); - Assertions.assertThatThrownBy( - () -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) + assertThatThrownBy(() -> scalarSql("SELECT system.bucket(10, INTERVAL '100-00' YEAR TO MONTH)")) .isInstanceOf(AnalysisException.class) .hasMessageStartingWith( "Function 'bucket' cannot process input: (int, interval year to month)"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> scalarSql("SELECT system.bucket(10, CAST('11 23:4:0' AS INTERVAL DAY TO SECOND))")) .isInstanceOf(AnalysisException.class) @@ -278,63 +273,59 @@ public void testInvalidTypesForBucketColumn() { @TestTemplate public void testThatMagicFunctionsAreInvoked() { // TinyInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6Y)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // SmallInt - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6S)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Int - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6)")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Date - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, DATE '2022-08-08')")) .asString() .isNotNull() .contains("staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketInt"); // Long - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 6L)")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); // Timestamp - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(100, TIMESTAMP '2022-08-08')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketLong"); // String - Assertions.assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, 'abcdefg')")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketString"); // Decimal - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(5, CAST('12.34' AS DECIMAL))")) .asString() .isNotNull() .contains( "staticinvoke(class org.apache.iceberg.spark.functions.BucketFunction$BucketDecimal"); // Binary - Assertions.assertThat( - scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) + assertThat(scalarSql("EXPLAIN EXTENDED SELECT system.bucket(4, X'0102030405060708')")) .asString() .isNotNull() .contains( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java index 2d97dfec186d..ab87b89a3529 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/UnpartitionedWritesTestBase.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.sql; 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.List; @@ -29,7 +30,6 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; import org.apache.spark.sql.functions; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -94,7 +94,7 @@ public void testInsertAppendAtSnapshot() { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql("INSERT INTO %s.%s VALUES (4, 'd'), (5, 'e')", tableName, prefix + snapshotId)) .isInstanceOf(IllegalArgumentException.class) @@ -107,7 +107,7 @@ public void testInsertOverwriteAtSnapshot() { long snapshotId = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); String prefix = "snapshot_id_"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> sql( "INSERT OVERWRITE %s.%s VALUES (4, 'd'), (5, 'e')", From c0199813d3d1c940fb267ac19fc16caabd078a5f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 18 Jun 2024 12:56:29 +0200 Subject: [PATCH 0387/1019] Statically import methods from AssertJ Assertions (#10517) --- .baseline/checkstyle/checkstyle.xml | 4 + .../aliyun/TestAliyunClientFactories.java | 19 +- .../iceberg/aliyun/oss/TestOSSFileIO.java | 47 +- .../iceberg/aliyun/oss/TestOSSInputFile.java | 21 +- .../aliyun/oss/TestOSSInputStream.java | 14 +- .../iceberg/aliyun/oss/TestOSSOutputFile.java | 35 +- .../aliyun/oss/TestOSSOutputStream.java | 15 +- .../apache/iceberg/aliyun/oss/TestOSSURI.java | 41 +- .../aliyun/oss/mock/TestLocalAliyunOSS.java | 24 +- .../java/org/apache/iceberg/TestHelpers.java | 9 +- .../iceberg/TestPartitionSpecValidation.java | 89 +-- .../org/apache/iceberg/TestSnapshotRef.java | 17 +- .../iceberg/TestSortOrderComparators.java | 48 +- .../apache/iceberg/catalog/TestNamespace.java | 22 +- .../iceberg/catalog/TestTableIdentifier.java | 12 +- .../expressions/TestAggregateBinding.java | 10 +- .../iceberg/expressions/TestEvaluator.java | 33 +- .../expressions/TestExpressionBinding.java | 10 +- .../expressions/TestExpressionHelpers.java | 8 +- .../expressions/TestExpressionUtil.java | 9 +- .../TestInclusiveManifestEvaluator.java | 6 +- .../TestInclusiveMetricsEvaluator.java | 6 +- .../expressions/TestPredicateBinding.java | 10 +- .../TestStrictMetricsEvaluator.java | 5 +- .../TestStringLiteralConversions.java | 6 +- .../apache/iceberg/io/TestCloseableGroup.java | 13 +- .../iceberg/io/TestCloseableIterable.java | 34 +- .../iceberg/io/TestClosingIterator.java | 5 +- .../iceberg/metrics/TestDefaultCounter.java | 18 +- .../metrics/TestDefaultMetricsContext.java | 47 +- .../iceberg/metrics/TestDefaultTimer.java | 101 +-- .../metrics/TestFixedReservoirHistogram.java | 67 +- .../iceberg/transforms/TestBucketing.java | 4 +- .../iceberg/transforms/TestProjection.java | 12 +- .../iceberg/transforms/TestResiduals.java | 4 +- .../iceberg/transforms/TestStartsWith.java | 3 +- .../iceberg/transforms/TestTruncate.java | 4 +- .../apache/iceberg/types/TestTypeUtil.java | 17 +- .../org/apache/iceberg/types/TestTypes.java | 22 +- .../iceberg/util/TestCharSequenceMap.java | 5 +- .../iceberg/util/TestCharSequenceSet.java | 41 +- .../apache/iceberg/util/TestDateTimeUtil.java | 12 +- .../iceberg/util/TestExceptionUtil.java | 11 +- .../arrow/vectorized/ArrowReaderTest.java | 5 +- .../parquet/DecimalVectorUtilTest.java | 4 +- .../TestLakeFormationAwsClientFactory.java | 4 +- .../iceberg/aws/AwsClientPropertiesTest.java | 21 +- .../iceberg/aws/HttpClientPropertiesTest.java | 11 +- .../iceberg/aws/TestAwsClientFactories.java | 43 +- .../apache/iceberg/aws/TestAwsProperties.java | 6 +- .../iceberg/aws/TestHttpClientProperties.java | 11 +- .../iceberg/aws/TestRESTSigV4Signer.java | 7 +- .../aws/TestS3FileIOAwsClientFactories.java | 7 +- .../iceberg/aws/TestS3FileIOProperties.java | 63 +- .../aws/dynamodb/TestDynamoDbCatalog.java | 11 +- .../iceberg/aws/glue/TestGlueCatalog.java | 47 +- .../aws/glue/TestGlueToIcebergConverter.java | 12 +- .../aws/glue/TestIcebergToGlueConverter.java | 59 +- .../apache/iceberg/aws/s3/TestS3FileIO.java | 56 +- .../aws/s3/TestS3FileIOProperties.java | 155 ++-- .../iceberg/aws/s3/TestS3InputStream.java | 17 +- .../iceberg/aws/s3/TestS3OutputStream.java | 31 +- .../iceberg/aws/s3/TestS3RequestUtil.java | 53 +- .../org/apache/iceberg/aws/s3/TestS3URI.java | 40 +- .../s3/signer/TestS3SignRequestParser.java | 34 +- .../s3/signer/TestS3SignResponseParser.java | 19 +- .../iceberg/aws/util/TestRetryDetector.java | 37 +- .../iceberg/azure/AzurePropertiesTest.java | 6 +- .../TestBaseIncrementalChangelogScan.java | 4 +- .../org/apache/iceberg/TestCatalogUtil.java | 60 +- .../apache/iceberg/TestContentFileParser.java | 59 +- .../iceberg/TestEnvironmentContext.java | 5 +- .../iceberg/TestFileScanTaskParser.java | 20 +- .../TestFixedSizeSplitScanTaskIterator.java | 7 +- .../apache/iceberg/TestManifestReader.java | 4 +- .../apache/iceberg/TestMetricsTruncation.java | 115 ++- ...TestOffsetsBasedSplitScanTaskIterator.java | 9 +- .../iceberg/TestSchemaUnionByFieldName.java | 58 +- .../iceberg/TestUpdateRequirementParser.java | 49 +- .../iceberg/avro/TestAvroNameMapping.java | 43 +- .../iceberg/avro/TestAvroReadProjection.java | 7 +- .../iceberg/avro/TestBuildAvroProjection.java | 47 +- .../iceberg/avro/TestReadProjection.java | 230 +++--- .../apache/iceberg/catalog/CatalogTests.java | 676 ++++++++---------- .../catalog/TestTableIdentifierParser.java | 28 +- .../iceberg/encryption/TestCiphers.java | 6 +- .../expressions/TestExpressionParser.java | 100 ++- .../iceberg/hadoop/HadoopFileIOTest.java | 33 +- .../iceberg/hadoop/TestCachingCatalog.java | 119 ++- .../hadoop/TestCatalogUtilDropTable.java | 35 +- .../iceberg/hadoop/TestHadoopCatalog.java | 190 +++-- .../iceberg/hadoop/TestHadoopCommits.java | 159 ++-- .../iceberg/hadoop/TestHadoopStreams.java | 5 +- .../iceberg/hadoop/TestHadoopTables.java | 51 +- .../iceberg/hadoop/TestStaticTable.java | 26 +- .../hadoop/TestTableSerialization.java | 15 +- .../iceberg/inmemory/TestInMemoryFileIO.java | 28 +- .../inmemory/TestInMemoryInputFile.java | 8 +- .../inmemory/TestInMemoryOutputFile.java | 9 +- .../io/TestByteBufferInputStreams.java | 214 +++--- .../org/apache/iceberg/io/TestIOUtil.java | 52 +- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 55 +- .../metrics/TestCommitReportParser.java | 35 +- .../metrics/TestCounterResultParser.java | 23 +- .../metrics/TestScanMetricsResultParser.java | 44 +- .../iceberg/metrics/TestScanReport.java | 70 +- .../iceberg/metrics/TestScanReportParser.java | 47 +- .../metrics/TestTimerResultParser.java | 62 +- .../iceberg/rest/RequestResponseTestBase.java | 11 +- .../apache/iceberg/rest/TestHTTPClient.java | 18 +- .../apache/iceberg/rest/TestRESTCatalog.java | 62 +- .../org/apache/iceberg/rest/TestRESTUtil.java | 22 +- .../iceberg/rest/TestResourcePaths.java | 97 ++- .../requests/TestCreateNamespaceRequest.java | 29 +- .../rest/requests/TestCreateTableRequest.java | 44 +- .../TestRegisterTableRequestParser.java | 13 +- .../rest/requests/TestRenameTableRequest.java | 24 +- .../TestReportMetricsRequestParser.java | 30 +- .../TestUpdateNamespacePropertiesRequest.java | 36 +- .../TestCatalogErrorResponseParser.java | 15 +- .../rest/responses/TestConfigResponse.java | 32 +- .../TestCreateNamespaceResponse.java | 28 +- .../responses/TestGetNamespaceResponse.java | 27 +- .../responses/TestListNamespacesResponse.java | 29 +- .../responses/TestListTablesResponse.java | 34 +- .../rest/responses/TestLoadTableResponse.java | 77 +- .../TestOAuthErrorResponseParser.java | 12 +- .../responses/TestOAuthTokenResponse.java | 26 +- ...TestUpdateNamespacePropertiesResponse.java | 36 +- .../iceberg/util/TestInMemoryLockManager.java | 4 +- .../org/apache/iceberg/util/TestJsonUtil.java | 247 +++---- .../apache/iceberg/util/TestLocationUtil.java | 4 +- .../apache/iceberg/util/TestLockManagers.java | 7 +- .../apache/iceberg/util/TestSnapshotUtil.java | 3 +- .../apache/iceberg/util/TestTruncateUtil.java | 8 +- .../view/TestSQLViewRepresentationParser.java | 20 +- .../view/TestViewHistoryEntryParser.java | 19 +- .../view/TestViewRepresentationParser.java | 13 +- .../iceberg/view/TestViewVersionParser.java | 16 +- .../apache/iceberg/data/DataTestHelpers.java | 23 +- .../iceberg/data/TestGenericRecord.java | 4 +- .../apache/iceberg/data/TestLocalScan.java | 8 +- .../data/TestMetricsRowGroupFilter.java | 7 +- .../iceberg/data/TestReadProjection.java | 6 +- .../data/avro/TestSingleMessageEncoding.java | 14 +- .../iceberg/dell/ecs/TestEcsCatalog.java | 22 +- .../iceberg/dell/ecs/TestEcsOutputFile.java | 4 +- .../dell/ecs/TestEcsTableOperations.java | 4 +- .../apache/iceberg/dell/ecs/TestEcsURI.java | 4 +- .../dell/mock/ecs/TestExceptionCode.java | 16 +- .../delta/TestSnapshotDeltaLakeTable.java | 42 +- .../TestBaseSnapshotDeltaLakeTableAction.java | 11 +- .../delta/TestDeltaLakeTypeToType.java | 64 +- .../apache/iceberg/gcp/GCPPropertiesTest.java | 4 +- .../iceberg/gcp/gcs/GCSInputStreamTest.java | 3 +- .../iceberg/gcp/gcs/GCSLocationTest.java | 24 +- .../mr/hive/TestHiveIcebergFilterFactory.java | 4 +- .../hive/TestHiveIcebergOutputCommitter.java | 5 +- .../iceberg/nessie/TestBranchVisibility.java | 86 ++- .../iceberg/nessie/TestMultipleClients.java | 59 +- .../apache/iceberg/nessie/TestNamespace.java | 79 +- .../iceberg/nessie/TestNessieCatalog.java | 5 +- .../nessie/TestNessieIcebergClient.java | 127 ++-- .../iceberg/nessie/TestNessieTable.java | 169 +++-- .../apache/iceberg/nessie/TestNessieUtil.java | 29 +- .../apache/iceberg/nessie/TestNessieView.java | 65 +- .../apache/iceberg/orc/TestBloomFilter.java | 18 +- .../iceberg/orc/TestBuildOrcProjection.java | 69 +- .../orc/TestEstimateOrcAvgWidthVisitor.java | 64 +- .../orc/TestExpressionToSearchArgument.java | 26 +- .../apache/iceberg/orc/TestIdToOrcName.java | 54 +- .../iceberg/orc/TestORCFileIOProxies.java | 22 +- .../apache/iceberg/orc/TestORCSchemaUtil.java | 44 +- .../apache/iceberg/orc/TestOrcDataWriter.java | 38 +- .../iceberg/orc/TestOrcDeleteWriters.java | 41 +- .../iceberg/orc/TestTableProperties.java | 37 +- .../snowflake/JdbcSnowflakeClientTest.java | 89 ++- .../snowflake/NamespaceHelpersTest.java | 31 +- .../snowflake/SnowflakeCatalogTest.java | 56 +- 179 files changed, 3295 insertions(+), 3731 deletions(-) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 8df46a5c688a..c36700c8c122 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -92,6 +92,10 @@ + + + + diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java index a329a3bda7c3..d4bd5fd24f51 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/TestAliyunClientFactories.java @@ -18,31 +18,32 @@ */ package org.apache.iceberg.aliyun; +import static org.assertj.core.api.Assertions.assertThat; + import com.aliyun.oss.OSS; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestAliyunClientFactories { @Test public void testLoadDefault() { - Assertions.assertThat(AliyunClientFactories.defaultFactory()) + assertThat(AliyunClientFactories.defaultFactory()) .as("Default client should be singleton") .isEqualTo(AliyunClientFactories.defaultFactory()); AliyunClientFactory defaultFactory = AliyunClientFactories.from(Maps.newHashMap()); - Assertions.assertThat(defaultFactory) + assertThat(defaultFactory) .as("Should load default when factory impl not configured") .isInstanceOf(AliyunClientFactories.DefaultAliyunClientFactory.class); - Assertions.assertThat(defaultFactory.aliyunProperties().accessKeyId()) + assertThat(defaultFactory.aliyunProperties().accessKeyId()) .as("Should have no Aliyun properties set") .isNull(); - Assertions.assertThat(defaultFactory.aliyunProperties().securityToken()) + assertThat(defaultFactory.aliyunProperties().securityToken()) .as("Should have no security token") .isNull(); @@ -53,15 +54,15 @@ public void testLoadDefault() { "key", AliyunProperties.CLIENT_SECURITY_TOKEN, "token")); - Assertions.assertThat(defaultFactoryWithConfig) + assertThat(defaultFactoryWithConfig) .as("Should load default when factory impl not configured") .isInstanceOf(AliyunClientFactories.DefaultAliyunClientFactory.class); - Assertions.assertThat(defaultFactoryWithConfig.aliyunProperties().accessKeyId()) + assertThat(defaultFactoryWithConfig.aliyunProperties().accessKeyId()) .as("Should have access key set") .isEqualTo("key"); - Assertions.assertThat(defaultFactoryWithConfig.aliyunProperties().securityToken()) + assertThat(defaultFactoryWithConfig.aliyunProperties().securityToken()) .as("Should have security token set") .isEqualTo("token"); } @@ -70,7 +71,7 @@ public void testLoadDefault() { public void testLoadCustom() { Map properties = Maps.newHashMap(); properties.put(AliyunProperties.CLIENT_FACTORY, CustomFactory.class.getName()); - Assertions.assertThat(AliyunClientFactories.from(properties)) + assertThat(AliyunClientFactories.from(properties)) .as("Should load custom class") .isInstanceOf(CustomFactory.class); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java index a4db1b9d04ed..dda4e75c2d00 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSFileIO.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aliyun.oss; +import static org.assertj.core.api.Assertions.assertThat; + import com.aliyun.oss.OSS; import com.aliyun.oss.OSSClient; import com.aliyun.oss.OSSClientBuilder; @@ -39,7 +41,6 @@ import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.apache.iceberg.util.SerializableSupplier; import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -73,33 +74,29 @@ public void testOutputFile() throws IOException { writeOSSData(out, data); OSSURI uri = new OSSURI(location); - Assertions.assertThat(ossClient().get().doesObjectExist(uri.bucket(), uri.key())) + assertThat(ossClient().get().doesObjectExist(uri.bucket(), uri.key())) .as("OSS file should exist") .isTrue(); - Assertions.assertThat(out.location()).as("Should have expected location").isEqualTo(location); - Assertions.assertThat(ossDataLength(uri)).as("Should have expected length").isEqualTo(dataSize); - Assertions.assertThat(ossDataContent(uri, dataSize)) - .as("Should have expected content") - .isEqualTo(data); + assertThat(out.location()).as("Should have expected location").isEqualTo(location); + assertThat(ossDataLength(uri)).as("Should have expected length").isEqualTo(dataSize); + assertThat(ossDataContent(uri, dataSize)).as("Should have expected content").isEqualTo(data); } @Test public void testInputFile() throws IOException { String location = randomLocation(); InputFile in = fileIO().newInputFile(location); - Assertions.assertThat(in.exists()).as("OSS file should not exist").isFalse(); + assertThat(in.exists()).as("OSS file should not exist").isFalse(); int dataSize = 1024 * 10; byte[] data = randomData(dataSize); OutputFile out = fileIO().newOutputFile(location); writeOSSData(out, data); - Assertions.assertThat(in.exists()).as("OSS file should exist").isTrue(); - Assertions.assertThat(in.location()).as("Should have expected location").isEqualTo(location); - Assertions.assertThat(in.getLength()).as("Should have expected length").isEqualTo(dataSize); - Assertions.assertThat(inFileContent(in, dataSize)) - .as("Should have expected content") - .isEqualTo(data); + assertThat(in.exists()).as("OSS file should exist").isTrue(); + assertThat(in.location()).as("Should have expected location").isEqualTo(location); + assertThat(in.getLength()).as("Should have expected length").isEqualTo(dataSize); + assertThat(inFileContent(in, dataSize)).as("Should have expected content").isEqualTo(data); } @Test @@ -111,22 +108,20 @@ public void testDeleteFile() throws IOException { writeOSSData(out, data); InputFile in = fileIO().newInputFile(location); - Assertions.assertThat(in.exists()).as("OSS file should exist").isTrue(); + assertThat(in.exists()).as("OSS file should exist").isTrue(); fileIO().deleteFile(in); - Assertions.assertThat(fileIO().newInputFile(location).exists()) - .as("OSS file should not exist") - .isFalse(); + assertThat(fileIO().newInputFile(location).exists()).as("OSS file should not exist").isFalse(); } @Test public void testLoadFileIO() { FileIO file = CatalogUtil.loadFileIO(OSS_IMPL_CLASS, ImmutableMap.of(), conf); - Assertions.assertThat(file).as("Should be OSSFileIO").isInstanceOf(OSSFileIO.class); + assertThat(file).as("Should be OSSFileIO").isInstanceOf(OSSFileIO.class); byte[] data = SerializationUtil.serializeToBytes(file); FileIO expectedFileIO = SerializationUtil.deserializeFromBytes(data); - Assertions.assertThat(expectedFileIO) + assertThat(expectedFileIO) .as("The deserialized FileIO should be OSSFileIO") .isInstanceOf(OSSFileIO.class); } @@ -143,22 +138,20 @@ public void serializeClient() throws URISyntaxException { SerializableSupplier post = SerializationUtil.deserializeFromBytes(data); OSS client = post.get(); - Assertions.assertThat(client) - .as("Should be instance of oss client") - .isInstanceOf(OSSClient.class); + assertThat(client).as("Should be instance of oss client").isInstanceOf(OSSClient.class); OSSClient oss = (OSSClient) client; - Assertions.assertThat(oss.getEndpoint()) + assertThat(oss.getEndpoint()) .as("Should have expected endpoint") .isEqualTo(new URI("http://" + endpoint)); - Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getAccessKeyId()) + assertThat(oss.getCredentialsProvider().getCredentials().getAccessKeyId()) .as("Should have expected access key") .isEqualTo(accessKeyId); - Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getSecretAccessKey()) + assertThat(oss.getCredentialsProvider().getCredentials().getSecretAccessKey()) .as("Should have expected secret key") .isEqualTo(accessSecret); - Assertions.assertThat(oss.getCredentialsProvider().getCredentials().getSecurityToken()) + assertThat(oss.getCredentialsProvider().getCredentials().getSecurityToken()) .as("Should have no security token") .isNull(); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java index 8d7cf51ccc84..f987113e2bfa 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputFile.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aliyun.oss; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.reset; @@ -36,7 +38,6 @@ import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOSSInputFile extends AliyunOSSTestBase { @@ -60,7 +61,7 @@ public void testReadFile() throws Exception { @Test public void testOSSInputFile() { OSSURI uri = randomURI(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new OSSInputFile( ossClient().get(), uri, aliyunProperties, -1, MetricsContext.nullMetrics())) @@ -74,7 +75,7 @@ public void testExists() { InputFile inputFile = new OSSInputFile(ossMock, uri, aliyunProperties, MetricsContext.nullMetrics()); - Assertions.assertThat(inputFile.exists()).as("OSS file should not exist").isFalse(); + assertThat(inputFile.exists()).as("OSS file should not exist").isFalse(); verify(ossMock, times(1)).getSimplifiedObjectMeta(uri.bucket(), uri.key()); reset(ossMock); @@ -82,7 +83,7 @@ public void testExists() { byte[] data = randomData(dataSize); writeOSSData(uri, data); - Assertions.assertThat(inputFile.exists()).as("OSS file should exist").isTrue(); + assertThat(inputFile.exists()).as("OSS file should exist").isTrue(); inputFile.exists(); verify(ossMock, times(1)).getSimplifiedObjectMeta(uri.bucket(), uri.key()); reset(ossMock); @@ -108,17 +109,15 @@ public void testGetLength() { private void readAndVerify(OSSURI uri, byte[] data) throws IOException { InputFile inputFile = new OSSInputFile(ossClient().get(), uri, aliyunProperties, MetricsContext.nullMetrics()); - Assertions.assertThat(inputFile.exists()).as("OSS file should exist").isTrue(); - Assertions.assertThat(inputFile.getLength()) - .as("Should have expected file length") - .isEqualTo(data.length); + assertThat(inputFile.exists()).as("OSS file should exist").isTrue(); + assertThat(inputFile.getLength()).as("Should have expected file length").isEqualTo(data.length); byte[] actual = new byte[data.length]; try (SeekableInputStream in = inputFile.newStream()) { ByteStreams.readFully(in, actual); } - Assertions.assertThat(actual).as("Should have same object content").isEqualTo(data); + assertThat(actual).as("Should have same object content").isEqualTo(data); } private void verifyLength(OSS ossClientMock, OSSURI uri, byte[] data, boolean isCache) { @@ -132,9 +131,7 @@ private void verifyLength(OSS ossClientMock, OSSURI uri, byte[] data, boolean is new OSSInputFile(ossClientMock, uri, aliyunProperties, MetricsContext.nullMetrics()); } inputFile.getLength(); - Assertions.assertThat(inputFile.getLength()) - .as("Should have expected file length") - .isEqualTo(data.length); + assertThat(inputFile.getLength()).as("Should have expected file length").isEqualTo(data.length); } private OSSURI randomURI() { diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java index ccbfa39c9959..053610983c10 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSInputStream.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aliyun.oss; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.Arrays; @@ -25,7 +28,6 @@ import java.util.concurrent.ThreadLocalRandom; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOSSInputStream extends AliyunOSSTestBase { @@ -69,7 +71,7 @@ private void readAndCheck( SeekableInputStream in, long rangeStart, int size, byte[] original, boolean buffered) throws IOException { in.seek(rangeStart); - Assertions.assertThat(in.getPos()).as("Should have the correct position").isEqualTo(rangeStart); + assertThat(in.getPos()).as("Should have the correct position").isEqualTo(rangeStart); long rangeEnd = rangeStart + size; byte[] actual = new byte[size]; @@ -83,9 +85,9 @@ private void readAndCheck( } } - Assertions.assertThat(in.getPos()).as("Should have the correct position").isEqualTo(rangeEnd); + assertThat(in.getPos()).as("Should have the correct position").isEqualTo(rangeEnd); - Assertions.assertThat(actual) + assertThat(actual) .as("Should have expected range data") .isEqualTo(Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd)); } @@ -95,7 +97,7 @@ public void testClose() throws Exception { OSSURI uri = new OSSURI(location("closed.dat")); SeekableInputStream closed = new OSSInputStream(ossClient().get(), uri); closed.close(); - Assertions.assertThatThrownBy(() -> closed.seek(0)) + assertThatThrownBy(() -> closed.seek(0)) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Cannot seek: already closed"); } @@ -111,7 +113,7 @@ public void testSeek() throws Exception { in.seek(expected.length / 2); byte[] actual = new byte[expected.length / 2]; ByteStreams.readFully(in, actual); - Assertions.assertThat(actual) + assertThat(actual) .as("Should have expected seeking stream") .isEqualTo(Arrays.copyOfRange(expected, expected.length / 2, expected.length)); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java index 75f095112b02..2a06d21c190c 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSOutputFile.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aliyun.oss; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.aliyun.oss.OSS; import java.io.ByteArrayInputStream; import java.io.IOException; @@ -32,7 +35,6 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOSSOutputFile extends AliyunOSSTestBase { @@ -53,20 +55,19 @@ public void testWriteFile() throws IOException { ByteStreams.copy(is, os); } - Assertions.assertThat(ossClient.doesObjectExist(uri.bucket(), uri.key())) + assertThat(ossClient.doesObjectExist(uri.bucket(), uri.key())) .as("OSS file should exist") .isTrue(); - Assertions.assertThat(ossDataLength(uri)).as("Object length should match").isEqualTo(dataSize); + assertThat(ossDataLength(uri)).as("Object length should match").isEqualTo(dataSize); byte[] actual = ossDataContent(uri, dataSize); - Assertions.assertThat(actual).as("Object content should match").isEqualTo(data); + assertThat(actual).as("Object content should match").isEqualTo(data); } @Test public void testFromLocation() { - Assertions.assertThatThrownBy( - () -> OSSOutputFile.fromLocation(ossClient, null, aliyunProperties)) + assertThatThrownBy(() -> OSSOutputFile.fromLocation(ossClient, null, aliyunProperties)) .isInstanceOf(NullPointerException.class) .hasMessageContaining("location cannot be null"); } @@ -81,7 +82,7 @@ public void testCreate() { OutputFile out = OSSOutputFile.fromLocation(ossClient, uri.location(), aliyunProperties); - Assertions.assertThatThrownBy(out::create) + assertThatThrownBy(out::create) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Location already exists"); } @@ -102,12 +103,12 @@ public void testCreateOrOverwrite() throws IOException { InputStream is = new ByteArrayInputStream(expect)) { ByteStreams.copy(is, os); } - Assertions.assertThat(ossDataLength(uri)) + assertThat(ossDataLength(uri)) .as(String.format("Should overwrite object length from %d to %d", dataSize, expectSize)) .isEqualTo(expectSize); byte[] actual = ossDataContent(uri, expectSize); - Assertions.assertThat(actual).as("Should overwrite object content").isEqualTo(expect); + assertThat(actual).as("Should overwrite object content").isEqualTo(expect); } @Test @@ -115,7 +116,7 @@ public void testLocation() { OSSURI uri = randomURI(); OutputFile out = new OSSOutputFile(ossClient, uri, aliyunProperties, MetricsContext.nullMetrics()); - Assertions.assertThat(out.location()).as("Location should match").isEqualTo(uri.location()); + assertThat(out.location()).as("Location should match").isEqualTo(uri.location()); } @Test @@ -131,20 +132,16 @@ public void testToInputFile() throws IOException { } InputFile in = out.toInputFile(); - Assertions.assertThat(in) - .as("Should be an instance of OSSInputFile") - .isInstanceOf(OSSInputFile.class); - Assertions.assertThat(in.exists()).as("OSS file should exist").isTrue(); - Assertions.assertThat(in.location()) - .as("Should have expected location") - .isEqualTo(out.location()); - Assertions.assertThat(in.getLength()).as("Should have expected length").isEqualTo(dataSize); + assertThat(in).as("Should be an instance of OSSInputFile").isInstanceOf(OSSInputFile.class); + assertThat(in.exists()).as("OSS file should exist").isTrue(); + assertThat(in.location()).as("Should have expected location").isEqualTo(out.location()); + assertThat(in.getLength()).as("Should have expected length").isEqualTo(dataSize); byte[] actual = new byte[dataSize]; try (InputStream as = in.newStream()) { ByteStreams.readFully(as, actual); } - Assertions.assertThat(actual).as("Should have expected content").isEqualTo(data); + assertThat(actual).as("Should have expected content").isEqualTo(data); } private OSSURI randomURI() { 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 fadad545d07f..069ff9111a5a 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.aliyun.oss; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -38,7 +39,6 @@ import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,28 +88,27 @@ private void writeAndVerify(OSS mock, OSSURI uri, byte[] data, boolean arrayWrit new OSSOutputStream(mock, uri, props, MetricsContext.nullMetrics())) { if (arrayWrite) { out.write(data); - Assertions.assertThat(out.getPos()).as("OSSOutputStream position").isEqualTo(data.length); + assertThat(out.getPos()).as("OSSOutputStream position").isEqualTo(data.length); } else { for (int i = 0; i < data.length; i++) { out.write(data[i]); - Assertions.assertThat(out.getPos()).as("OSSOutputStream position").isEqualTo(i + 1); + assertThat(out.getPos()).as("OSSOutputStream position").isEqualTo(i + 1); } } } - Assertions.assertThat(ossClient.doesObjectExist(uri.bucket(), uri.key())) + assertThat(ossClient.doesObjectExist(uri.bucket(), uri.key())) .as("OSS object should exist") .isTrue(); - Assertions.assertThat( - ossClient.getObject(uri.bucket(), uri.key()).getObjectMetadata().getContentLength()) + assertThat(ossClient.getObject(uri.bucket(), uri.key()).getObjectMetadata().getContentLength()) .as("Object length") .isEqualTo(data.length); byte[] actual = ossDataContent(uri, data.length); - Assertions.assertThat(actual).as("Object content").isEqualTo(data); + assertThat(actual).as("Object content").isEqualTo(data); // Verify all staging files are cleaned up. - Assertions.assertThat(Files.list(Paths.get(props.ossStagingDirectory())).count()) + assertThat(Files.list(Paths.get(props.ossStagingDirectory())).count()) .as("Staging files should clean up") .isEqualTo(0); } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java index 932dfe51c4f3..e388ee932589 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/TestOSSURI.java @@ -19,10 +19,11 @@ package org.apache.iceberg.aliyun.oss; import static com.aliyun.oss.internal.OSSUtils.OSS_RESOURCE_MANAGER; +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.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOSSURI { @@ -31,9 +32,9 @@ public void testUrlParsing() { String location = "oss://bucket/path/to/file"; OSSURI uri = new OSSURI(location); - Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); - Assertions.assertThat(uri.toString()).isEqualTo(location); + assertThat(uri.bucket()).isEqualTo("bucket"); + assertThat(uri.key()).isEqualTo("path/to/file"); + assertThat(uri.toString()).isEqualTo(location); } @Test @@ -41,15 +42,15 @@ public void testEncodedString() { String location = "oss://bucket/path%20to%20file"; OSSURI uri = new OSSURI(location); - Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri.key()).isEqualTo("path%20to%20file"); - Assertions.assertThat(uri.toString()).isEqualTo(location); + assertThat(uri.bucket()).isEqualTo("bucket"); + assertThat(uri.key()).isEqualTo("path%20to%20file"); + assertThat(uri.toString()).isEqualTo(location); } @Test public void invalidBucket() { - Assertions.assertThatThrownBy(() -> new OSSURI("https://test_bucket/path/to/file")) + assertThatThrownBy(() -> new OSSURI("https://test_bucket/path/to/file")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( OSS_RESOURCE_MANAGER.getFormattedString("BucketNameInvalid", "test_bucket")); @@ -58,14 +59,14 @@ public void invalidBucket() { @Test public void missingKey() { - Assertions.assertThatThrownBy(() -> new OSSURI("https://bucket/")) + assertThatThrownBy(() -> new OSSURI("https://bucket/")) .isInstanceOf(ValidationException.class) .hasMessageContaining("Missing key in OSS location"); } @Test public void invalidKey() { - Assertions.assertThatThrownBy(() -> new OSSURI("https://bucket/\\path/to/file")) + assertThatThrownBy(() -> new OSSURI("https://bucket/\\path/to/file")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( OSS_RESOURCE_MANAGER.getFormattedString("ObjectKeyInvalid", "\\path/to/file")); @@ -74,7 +75,7 @@ public void invalidKey() { @Test public void relativePathing() { - Assertions.assertThatThrownBy(() -> new OSSURI("/path/to/file")) + assertThatThrownBy(() -> new OSSURI("/path/to/file")) .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid OSS location"); } @@ -82,7 +83,7 @@ public void relativePathing() { @Test public void invalidScheme() { - Assertions.assertThatThrownBy(() -> new OSSURI("invalid://bucket/")) + assertThatThrownBy(() -> new OSSURI("invalid://bucket/")) .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid scheme"); } @@ -92,9 +93,9 @@ public void testFragment() { String location = "oss://bucket/path/to/file#print"; OSSURI uri = new OSSURI(location); - Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); - Assertions.assertThat(uri.toString()).isEqualTo(location); + assertThat(uri.bucket()).isEqualTo("bucket"); + assertThat(uri.key()).isEqualTo("path/to/file"); + assertThat(uri.toString()).isEqualTo(location); } @Test @@ -102,17 +103,17 @@ public void testQueryAndFragment() { String location = "oss://bucket/path/to/file?query=foo#bar"; OSSURI uri = new OSSURI(location); - Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); - Assertions.assertThat(uri.toString()).isEqualTo(location); + assertThat(uri.bucket()).isEqualTo("bucket"); + assertThat(uri.key()).isEqualTo("path/to/file"); + assertThat(uri.toString()).isEqualTo(location); } @Test public void testValidSchemes() { for (String scheme : Lists.newArrayList("https", "oss")) { OSSURI uri = new OSSURI(scheme + "://bucket/path/to/file"); - Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); + assertThat(uri.bucket()).isEqualTo("bucket"); + assertThat(uri.key()).isEqualTo("path/to/file"); } } } diff --git a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java index 5a47708fbd41..a661c172ed12 100644 --- a/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java +++ b/aliyun/src/test/java/org/apache/iceberg/aliyun/oss/mock/TestLocalAliyunOSS.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aliyun.oss.mock; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.aliyun.oss.OSS; import com.aliyun.oss.OSSErrorCode; import com.aliyun.oss.OSSException; @@ -33,7 +36,6 @@ import org.apache.iceberg.aliyun.TestUtility; import org.apache.iceberg.aliyun.oss.AliyunOSSExtension; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.assertj.core.api.Assertions; import org.assertj.core.api.Assumptions; import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.AfterEach; @@ -51,7 +53,7 @@ public class TestLocalAliyunOSS { private final Random random = new Random(1); private static void assertThrows(Runnable runnable, String expectedErrorCode) { - Assertions.assertThatThrownBy(runnable::run) + assertThatThrownBy(runnable::run) .isInstanceOf(OSSException.class) .asInstanceOf(InstanceOfAssertFactories.type(OSSException.class)) .extracting(ServiceException::getErrorCode) @@ -74,15 +76,15 @@ public void testBuckets() { .as("Aliyun integration test cannot delete existing bucket from test environment.") .isEqualTo(AliyunOSSMockExtension.class); - Assertions.assertThat(doesBucketExist(bucketName)).isTrue(); + assertThat(doesBucketExist(bucketName)).isTrue(); assertThrows(() -> oss.createBucket(bucketName), OSSErrorCode.BUCKET_ALREADY_EXISTS); oss.deleteBucket(bucketName); - Assertions.assertThat(doesBucketExist(bucketName)).isFalse(); + assertThat(doesBucketExist(bucketName)).isFalse(); oss.createBucket(bucketName); - Assertions.assertThat(doesBucketExist(bucketName)).isTrue(); + assertThat(doesBucketExist(bucketName)).isTrue(); } @Test @@ -108,7 +110,7 @@ public void testDeleteBucket() { oss.deleteObject(bucketName, "object2"); oss.deleteBucket(bucketName); - Assertions.assertThat(doesBucketExist(bucketName)).isFalse(); + assertThat(doesBucketExist(bucketName)).isFalse(); oss.createBucket(bucketName); } @@ -123,18 +125,18 @@ public void testPutObject() throws IOException { () -> oss.putObject(bucketNotExist, "object", wrap(bytes)), OSSErrorCode.NO_SUCH_BUCKET); PutObjectResult result = oss.putObject(bucketName, "object", wrap(bytes)); - Assertions.assertThat(result.getETag()).isEqualTo(AliyunOSSMockLocalStore.md5sum(wrap(bytes))); + assertThat(result.getETag()).isEqualTo(AliyunOSSMockLocalStore.md5sum(wrap(bytes))); } @Test public void testDoesObjectExist() { - Assertions.assertThat(oss.doesObjectExist(bucketName, "key")).isFalse(); + assertThat(oss.doesObjectExist(bucketName, "key")).isFalse(); byte[] bytes = new byte[4 * 1024]; random.nextBytes(bytes); oss.putObject(bucketName, "key", wrap(bytes)); - Assertions.assertThat(oss.doesObjectExist(bucketName, "key")).isTrue(); + assertThat(oss.doesObjectExist(bucketName, "key")).isTrue(); oss.deleteObject(bucketName, "key"); } @@ -154,7 +156,7 @@ public void testGetObject() throws IOException { try (InputStream is = oss.getObject(bucketName, "key").getObjectContent()) { ByteStreams.readFully(is, actual); } - Assertions.assertThat(actual).isEqualTo(bytes); + assertThat(actual).isEqualTo(bytes); oss.deleteObject(bucketName, "key"); } @@ -230,7 +232,7 @@ private void testRange(byte[] bytes, int start, int end) throws IOException { try (InputStream is = oss.getObject(getObjectRequest).getObjectContent()) { ByteStreams.readFully(is, actual); } - Assertions.assertThat(actual).isEqualTo(testBytes); + assertThat(actual).isEqualTo(testBytes); } private InputStream wrap(byte[] data) { diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index fcb528caba96..329a61fb44b3 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -47,7 +47,6 @@ import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.util.ByteBuffers; -import org.assertj.core.api.Assertions; import org.objenesis.strategy.StdInstantiatorStrategy; public class TestHelpers { @@ -110,9 +109,7 @@ public static T roundTripSerialize(T type) throws IOException, ClassNotFound } public static void assertSameSchemaList(List list1, List list2) { - Assertions.assertThat(list1) - .as("Should have same number of schemas in both lists") - .hasSameSizeAs(list2); + assertThat(list1).as("Should have same number of schemas in both lists").hasSameSizeAs(list2); IntStream.range(0, list1.size()) .forEach( @@ -151,9 +148,7 @@ public static void assertSerializedAndLoadedMetadata(Table expected, Table actua } public static void assertSameSchemaMap(Map map1, Map map2) { - Assertions.assertThat(map1) - .as("Should have same number of schemas in both maps") - .hasSameSizeAs(map2); + assertThat(map1).as("Should have same number of schemas in both maps").hasSameSizeAs(map2); map1.forEach( (schemaId, schema1) -> { diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java index eb0e74164688..263db427aa16 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionSpecValidation.java @@ -19,11 +19,11 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestPartitionSpecValidation { @@ -38,84 +38,70 @@ public class TestPartitionSpecValidation { @Test public void testMultipleTimestampPartitions() { - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).year("ts").year("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("ts").year("ts").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).year("ts").month("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("ts").month("ts").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).year("ts").day("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("ts").day("ts").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).year("ts").hour("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("ts").hour("ts").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).month("ts").month("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("ts").month("ts").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).month("ts").day("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("ts").day("ts").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).month("ts").hour("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("ts").hour("ts").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).day("ts").day("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("ts").day("ts").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).day("ts").hour("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("ts").hour("ts").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).hour("ts").hour("ts").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).hour("ts").hour("ts").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); } @Test public void testMultipleDatePartitions() { - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).year("d").year("d").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("d").year("d").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).year("d").month("d").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("d").month("d").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("d").day("d").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("d").day("d").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).month("d").month("d").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("d").month("d").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).month("d").day("d").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("d").day("d").build()) .hasMessageContaining("Cannot add redundant partition") .isInstanceOf(IllegalArgumentException.class); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("d").day("d").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("d").day("d").build()) .hasMessageContaining("Cannot use partition name more than once") .isInstanceOf(IllegalArgumentException.class); } @@ -151,17 +137,16 @@ public void testMultipleDatePartitionsWithDifferentSourceColumns() { @Test public void testMultipleIdentityPartitions() { PartitionSpec.builderFor(SCHEMA).year("d").identity("id").identity("d").identity("s").build(); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).identity("id").identity("id").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).identity("id").identity("id").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot use partition name more than once"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> PartitionSpec.builderFor(SCHEMA).identity("id").identity("id", "test-id").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot add redundant partition"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> PartitionSpec.builderFor(SCHEMA) .identity("id", "test-id") @@ -220,34 +205,31 @@ public void testSettingPartitionTransformsWithCustomTargetNames() { @Test public void testSettingPartitionTransformsWithCustomTargetNamesThatAlreadyExist() { - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("ts", "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("ts", "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create partition from name that exists in schema: another_ts"); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("ts", "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("ts", "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create partition from name that exists in schema: another_ts"); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("ts", "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("ts", "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create partition from name that exists in schema: another_ts"); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).hour("ts", "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).hour("ts", "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create partition from name that exists in schema: another_ts"); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).truncate("ts", 2, "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).truncate("ts", 2, "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create partition from name that exists in schema: another_ts"); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).bucket("ts", 4, "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).bucket("ts", 4, "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create partition from name that exists in schema: another_ts"); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).identity("ts", "another_ts")) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).identity("ts", "another_ts")) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot create identity partition sourced from different field in schema: another_ts"); @@ -255,34 +237,31 @@ public void testSettingPartitionTransformsWithCustomTargetNamesThatAlreadyExist( @Test public void testMissingSourceColumn() { - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("missing").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).year("missing").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("missing").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).month("missing").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("missing").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).day("missing").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); - Assertions.assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).hour("missing").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).hour("missing").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).bucket("missing", 4).build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).bucket("missing", 4).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).truncate("missing", 5).build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).truncate("missing", 5).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); - Assertions.assertThatThrownBy( - () -> PartitionSpec.builderFor(SCHEMA).identity("missing").build()) + assertThatThrownBy(() -> PartitionSpec.builderFor(SCHEMA).identity("missing").build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find source column: missing"); } diff --git a/api/src/test/java/org/apache/iceberg/TestSnapshotRef.java b/api/src/test/java/org/apache/iceberg/TestSnapshotRef.java index 21ad472f6524..1d36b66aaf5d 100644 --- a/api/src/test/java/org/apache/iceberg/TestSnapshotRef.java +++ b/api/src/test/java/org/apache/iceberg/TestSnapshotRef.java @@ -19,8 +19,8 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestSnapshotRef { @@ -69,38 +69,37 @@ public void testBranchWithOverride() { @Test public void testNoTypeFailure() { - Assertions.assertThatThrownBy(() -> SnapshotRef.builderFor(1L, null).build()) + assertThatThrownBy(() -> SnapshotRef.builderFor(1L, null).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Snapshot reference type must not be null"); } @Test public void testTagBuildFailures() { - Assertions.assertThatThrownBy(() -> SnapshotRef.tagBuilder(1L).maxRefAgeMs(-1L).build()) + assertThatThrownBy(() -> SnapshotRef.tagBuilder(1L).maxRefAgeMs(-1L).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Max reference age must be greater than 0"); - Assertions.assertThatThrownBy(() -> SnapshotRef.tagBuilder(1L).minSnapshotsToKeep(2).build()) + assertThatThrownBy(() -> SnapshotRef.tagBuilder(1L).minSnapshotsToKeep(2).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Tags do not support setting minSnapshotsToKeep"); - Assertions.assertThatThrownBy(() -> SnapshotRef.tagBuilder(1L).maxSnapshotAgeMs(2L).build()) + assertThatThrownBy(() -> SnapshotRef.tagBuilder(1L).maxSnapshotAgeMs(2L).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Tags do not support setting maxSnapshotAgeMs"); } @Test public void testBranchBuildFailures() { - Assertions.assertThatThrownBy(() -> SnapshotRef.branchBuilder(1L).maxSnapshotAgeMs(-1L).build()) + assertThatThrownBy(() -> SnapshotRef.branchBuilder(1L).maxSnapshotAgeMs(-1L).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Max snapshot age must be greater than 0 ms"); - Assertions.assertThatThrownBy( - () -> SnapshotRef.branchBuilder(1L).minSnapshotsToKeep(-1).build()) + assertThatThrownBy(() -> SnapshotRef.branchBuilder(1L).minSnapshotsToKeep(-1).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Min snapshots to keep must be greater than 0"); - Assertions.assertThatThrownBy(() -> SnapshotRef.branchBuilder(1L).maxRefAgeMs(-1L).build()) + assertThatThrownBy(() -> SnapshotRef.branchBuilder(1L).maxRefAgeMs(-1L).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Max reference age must be greater than 0"); } diff --git a/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java b/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java index 90ac3e09b6dd..1791d8abd405 100644 --- a/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java +++ b/api/src/test/java/org/apache/iceberg/TestSortOrderComparators.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + import java.math.BigDecimal; import java.nio.ByteBuffer; import java.time.LocalDateTime; @@ -31,7 +33,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializableFunction; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestSortOrderComparators { @@ -46,26 +47,25 @@ private static void assertComparesCorrectly( Comparator comparator = SortOrderComparators.forSchema(schema, sortOrder); // all fields should have the same sort direction in this test class - Assertions.assertThat(sortOrder.fields().stream().map(SortField::direction).distinct()) - .hasSize(1); + assertThat(sortOrder.fields().stream().map(SortField::direction).distinct()).hasSize(1); SortDirection direction = sortOrder.fields().get(0).direction(); - Assertions.assertThat(comparator.compare(less, less)).isEqualTo(0); - Assertions.assertThat(comparator.compare(greater, greater)).isEqualTo(0); - Assertions.assertThat(comparator.compare(less, lessCopy)).isEqualTo(0); + assertThat(comparator.compare(less, less)).isEqualTo(0); + assertThat(comparator.compare(greater, greater)).isEqualTo(0); + assertThat(comparator.compare(less, lessCopy)).isEqualTo(0); if (direction == SortDirection.ASC) { - Assertions.assertThat(comparator.compare(less, greater)).isEqualTo(-1); - Assertions.assertThat(comparator.compare(greater, less)).isEqualTo(1); + assertThat(comparator.compare(less, greater)).isEqualTo(-1); + assertThat(comparator.compare(greater, less)).isEqualTo(1); // null first - Assertions.assertThat(comparator.compare(nullValue, less)).isEqualTo(-1); - Assertions.assertThat(comparator.compare(less, nullValue)).isEqualTo(1); + assertThat(comparator.compare(nullValue, less)).isEqualTo(-1); + assertThat(comparator.compare(less, nullValue)).isEqualTo(1); } else { - Assertions.assertThat(comparator.compare(less, greater)).isEqualTo(1); - Assertions.assertThat(comparator.compare(greater, less)).isEqualTo(-1); + assertThat(comparator.compare(less, greater)).isEqualTo(1); + assertThat(comparator.compare(greater, less)).isEqualTo(-1); // null last - Assertions.assertThat(comparator.compare(nullValue, greater)).isEqualTo(1); - Assertions.assertThat(comparator.compare(less, nullValue)).isEqualTo(-1); + assertThat(comparator.compare(nullValue, greater)).isEqualTo(1); + assertThat(comparator.compare(less, nullValue)).isEqualTo(-1); } } @@ -196,8 +196,8 @@ public void testTimestampTransform() { TimeUnit.SECONDS.toMicros( LocalDateTime.of(2022, 1, 10, 1, 0, 0).toEpochSecond(ZoneOffset.UTC)); - Assertions.assertThat(transform.apply(lessMicro)).isLessThan(transform.apply(greaterMicro)); - Assertions.assertThat(transform.apply(lessMicro)).isEqualTo(transform.apply(lessCopyMicro)); + assertThat(transform.apply(lessMicro)).isLessThan(transform.apply(greaterMicro)); + assertThat(transform.apply(lessMicro)).isEqualTo(transform.apply(lessCopyMicro)); TestHelpers.Row less = TestHelpers.Row.of("id3", lessMicro); TestHelpers.Row greater = TestHelpers.Row.of("id2", greaterMicro); @@ -226,8 +226,8 @@ public void testStringBucket() { Transform bucket = Transforms.bucket(4); SerializableFunction transform = bucket.bind(Types.StringType.get()); - Assertions.assertThat(transform.apply("bbb")).isLessThan(transform.apply("aaa")); - Assertions.assertThat(transform.apply("bbb")).isEqualTo(transform.apply("cca")); + assertThat(transform.apply("bbb")).isLessThan(transform.apply("aaa")); + assertThat(transform.apply("bbb")).isEqualTo(transform.apply("cca")); TestHelpers.Row less = TestHelpers.Row.of("id3", "bbb"); TestHelpers.Row greater = TestHelpers.Row.of("id2", "aaa"); @@ -260,9 +260,9 @@ public void testUUIDBucket() { Transform bucket = Transforms.bucket(4); SerializableFunction transform = bucket.bind(Types.UUIDType.get()); - Assertions.assertThat(transform.apply(UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b"))) + assertThat(transform.apply(UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b"))) .isLessThan(transform.apply(UUID.fromString("86873e7d-1374-4493-8e1d-9095eff7046c"))); - Assertions.assertThat(transform.apply(UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b"))) + assertThat(transform.apply(UUID.fromString("fd02441d-1423-4a3f-8785-c7dd5647e26b"))) .isEqualTo(transform.apply(UUID.fromString("81873e7d-1374-4493-8e1d-9095eff7046c"))); TestHelpers.Row less = @@ -305,9 +305,9 @@ public void testBinaryTruncate() { Transform truncate = Transforms.truncate(2); SerializableFunction transform = truncate.bind(Types.BinaryType.get()); - Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 3}))) + assertThat(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 3}))) .isLessThan(transform.apply(ByteBuffer.wrap(new byte[] {1, 3, 1}))); - Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 3}))) + assertThat(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 3}))) .isEqualTo(transform.apply(ByteBuffer.wrap(new byte[] {1, 2, 5, 6}))); TestHelpers.Row less = TestHelpers.Row.of("id3", ByteBuffer.wrap(new byte[] {1, 2, 3})); @@ -414,9 +414,9 @@ public void testStructTransform() { Transform bucket = Transforms.truncate(2); SerializableFunction transform = bucket.bind(Types.BinaryType.get()); - Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 4}))) + assertThat(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 4}))) .isLessThan(transform.apply(ByteBuffer.wrap(new byte[] {9, 3, 4}))); - Assertions.assertThat(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 4}))) + assertThat(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 4}))) .isEqualTo(transform.apply(ByteBuffer.wrap(new byte[] {2, 3, 9}))); TestHelpers.Row less = diff --git a/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java b/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java index 62335b4cf771..77ddf8ccbbbf 100644 --- a/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java +++ b/api/src/test/java/org/apache/iceberg/catalog/TestNamespace.java @@ -18,46 +18,48 @@ */ package org.apache.iceberg.catalog; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.junit.jupiter.api.Test; public class TestNamespace { @Test public void testWithNullAndEmpty() { - Assertions.assertThatThrownBy(() -> Namespace.of((String[]) null)) + assertThatThrownBy(() -> Namespace.of((String[]) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create Namespace from null array"); - Assertions.assertThat(Namespace.of()).isEqualTo(Namespace.empty()); + assertThat(Namespace.of()).isEqualTo(Namespace.empty()); } @Test public void testNamespace() { String[] levels = {"a", "b", "c", "d"}; Namespace namespace = Namespace.of(levels); - Assertions.assertThat(namespace).isNotNull(); - Assertions.assertThat(namespace.levels()).hasSize(4); - Assertions.assertThat(namespace).hasToString("a.b.c.d"); + assertThat(namespace).isNotNull(); + assertThat(namespace.levels()).hasSize(4); + assertThat(namespace).hasToString("a.b.c.d"); for (int i = 0; i < levels.length; i++) { - Assertions.assertThat(namespace.level(i)).isEqualTo(levels[i]); + assertThat(namespace.level(i)).isEqualTo(levels[i]); } } @Test public void testWithNullInLevel() { - Assertions.assertThatThrownBy(() -> Namespace.of("a", null, "b")) + assertThatThrownBy(() -> Namespace.of("a", null, "b")) .isInstanceOf(NullPointerException.class) .hasMessage("Cannot create a namespace with a null level"); } @Test public void testDisallowsNamespaceWithNullByte() { - Assertions.assertThatThrownBy(() -> Namespace.of("ac", "\u0000c", "b")) + assertThatThrownBy(() -> Namespace.of("ac", "\u0000c", "b")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create a namespace with the null-byte character"); - Assertions.assertThatThrownBy(() -> Namespace.of("ac", "c\0", "b")) + assertThatThrownBy(() -> Namespace.of("ac", "c\0", "b")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create a namespace with the null-byte character"); } diff --git a/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java b/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java index 06e3295d5a76..ca9569436bab 100644 --- a/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java +++ b/api/src/test/java/org/apache/iceberg/catalog/TestTableIdentifier.java @@ -19,8 +19,8 @@ package org.apache.iceberg.catalog; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestTableIdentifier { @@ -54,26 +54,26 @@ public void testToLowerCase() { @Test public void testInvalidTableName() { - Assertions.assertThatThrownBy(() -> TableIdentifier.of(Namespace.empty(), "")) + assertThatThrownBy(() -> TableIdentifier.of(Namespace.empty(), "")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid table name: null or empty"); - Assertions.assertThatThrownBy(() -> TableIdentifier.of(Namespace.empty(), null)) + assertThatThrownBy(() -> TableIdentifier.of(Namespace.empty(), null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid table name: null or empty"); } @Test public void testNulls() { - Assertions.assertThatThrownBy(() -> TableIdentifier.of((String[]) null)) + assertThatThrownBy(() -> TableIdentifier.of((String[]) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create table identifier from null array"); - Assertions.assertThatThrownBy(() -> TableIdentifier.parse(null)) + assertThatThrownBy(() -> TableIdentifier.parse(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse table identifier: null"); - Assertions.assertThatThrownBy(() -> TableIdentifier.of(null, "name")) + assertThatThrownBy(() -> TableIdentifier.of(null, "name")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid Namespace: null"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java index 869be1cd6d0f..95a9ac2cc83a 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java @@ -19,13 +19,13 @@ package org.apache.iceberg.expressions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestAggregateBinding { @@ -60,7 +60,7 @@ public void testCountStarBinding() { @Test public void testBoundAggregateFails() { Expression unbound = Expressions.count("x"); - Assertions.assertThatThrownBy(() -> Binder.bind(struct, Binder.bind(struct, unbound))) + assertThatThrownBy(() -> Binder.bind(struct, Binder.bind(struct, unbound))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Found already bound aggregate"); } @@ -79,7 +79,7 @@ public void testCaseInsensitiveReference() { @Test public void testCaseSensitiveReference() { Expression expr = Expressions.max("X"); - Assertions.assertThatThrownBy(() -> Binder.bind(struct, expr, true)) + assertThatThrownBy(() -> Binder.bind(struct, expr, true)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'X' in struct"); } @@ -87,13 +87,13 @@ public void testCaseSensitiveReference() { @Test public void testMissingField() { UnboundAggregate unbound = Expressions.count("missing"); - Assertions.assertThatThrownBy(() -> unbound.bind(struct, false)) + assertThatThrownBy(() -> unbound.bind(struct, false)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing' in struct:"); } private static BoundAggregate assertAndUnwrapAggregate(Expression expr) { - Assertions.assertThat(expr).isInstanceOf(BoundAggregate.class); + assertThat(expr).isInstanceOf(BoundAggregate.class); return (BoundAggregate) expr; } } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java index 812b4cfa08c9..792e651a3d18 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestEvaluator.java @@ -41,6 +41,7 @@ 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.util.Arrays; import java.util.Collection; @@ -50,7 +51,6 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestEvaluator { @@ -590,7 +590,7 @@ public void testCaseInsensitiveNot() { @Test public void testCaseSensitiveNot() { - Assertions.assertThatThrownBy(() -> new Evaluator(STRUCT, not(equal("X", 7)), true)) + assertThatThrownBy(() -> new Evaluator(STRUCT, not(equal("X", 7)), true)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'X' in struct"); } @@ -678,32 +678,31 @@ public void testIn() { @Test public void testInExceptions() { - Assertions.assertThatThrownBy(() -> in("x", (Literal) null)) + assertThatThrownBy(() -> in("x", (Literal) null)) .isInstanceOf(NullPointerException.class) .hasMessage("Cannot create expression literal from null"); - Assertions.assertThatThrownBy(() -> in("x", (Collection) null)) + assertThatThrownBy(() -> in("x", (Collection) null)) .isInstanceOf(NullPointerException.class) .hasMessage("Values cannot be null for IN predicate."); - Assertions.assertThatThrownBy(() -> in("x", 5, 6).literal()) + assertThatThrownBy(() -> in("x", 5, 6).literal()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("IN predicate cannot return a literal"); - Assertions.assertThatThrownBy(() -> in("x", 1, 2, null)) + assertThatThrownBy(() -> in("x", 1, 2, null)) .isInstanceOf(NullPointerException.class) .hasMessage("Cannot create expression literal from null"); - Assertions.assertThatThrownBy(() -> new Evaluator(STRUCT, in("x", 7, 8, 9.1))) + assertThatThrownBy(() -> new Evaluator(STRUCT, in("x", 7, 8, 9.1))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid value for conversion to type int"); - Assertions.assertThatThrownBy(() -> predicate(Expression.Operation.IN, "x")) + assertThatThrownBy(() -> predicate(Expression.Operation.IN, "x")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create IN predicate without a value"); - Assertions.assertThatThrownBy( - () -> new Evaluator(STRUCT, predicate(Expression.Operation.IN, "x", 5.1))) + assertThatThrownBy(() -> new Evaluator(STRUCT, predicate(Expression.Operation.IN, "x", 5.1))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid value for conversion to type int"); } @@ -781,31 +780,31 @@ public void testNotIn() { @Test public void testNotInExceptions() { - Assertions.assertThatThrownBy(() -> notIn("x", (Literal) null)) + assertThatThrownBy(() -> notIn("x", (Literal) null)) .isInstanceOf(NullPointerException.class) .hasMessage("Cannot create expression literal from null"); - Assertions.assertThatThrownBy(() -> notIn("x", (Collection) null)) + assertThatThrownBy(() -> notIn("x", (Collection) null)) .isInstanceOf(NullPointerException.class) .hasMessage("Values cannot be null for NOT_IN predicate."); - Assertions.assertThatThrownBy(() -> notIn("x", 5, 6).literal()) + assertThatThrownBy(() -> notIn("x", 5, 6).literal()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("NOT_IN predicate cannot return a literal"); - Assertions.assertThatThrownBy(() -> notIn("x", 1, 2, null)) + assertThatThrownBy(() -> notIn("x", 1, 2, null)) .isInstanceOf(NullPointerException.class) .hasMessage("Cannot create expression literal from null"); - Assertions.assertThatThrownBy(() -> new Evaluator(STRUCT, notIn("x", 7, 8, 9.1))) + assertThatThrownBy(() -> new Evaluator(STRUCT, notIn("x", 7, 8, 9.1))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid value for conversion to type int"); - Assertions.assertThatThrownBy(() -> predicate(Expression.Operation.NOT_IN, "x")) + assertThatThrownBy(() -> predicate(Expression.Operation.NOT_IN, "x")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create NOT_IN predicate without a value"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new Evaluator(STRUCT, predicate(Expression.Operation.NOT_IN, "x", 5.1))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Invalid value for conversion to type int"); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java index 0a9cb4b8cd65..8dccc4e6a5d6 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionBinding.java @@ -31,12 +31,12 @@ import static org.apache.iceberg.expressions.Expressions.startsWith; 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 org.apache.iceberg.TestHelpers; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestExpressionBinding { @@ -50,7 +50,7 @@ public class TestExpressionBinding { @Test public void testMissingReference() { Expression expr = and(equal("t", 5), equal("x", 7)); - Assertions.assertThatThrownBy(() -> Binder.bind(STRUCT, expr)) + assertThatThrownBy(() -> Binder.bind(STRUCT, expr)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 't' in struct"); } @@ -58,7 +58,7 @@ public void testMissingReference() { @Test public void testBoundExpressionFails() { Expression expr = not(equal("x", 7)); - Assertions.assertThatThrownBy(() -> Binder.bind(STRUCT, Binder.bind(STRUCT, expr))) + assertThatThrownBy(() -> Binder.bind(STRUCT, Binder.bind(STRUCT, expr))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Found already bound predicate"); } @@ -78,7 +78,7 @@ public void testCaseInsensitiveReference() { @Test public void testCaseSensitiveReference() { Expression expr = not(equal("X", 7)); - Assertions.assertThatThrownBy(() -> Binder.bind(STRUCT, expr, true)) + assertThatThrownBy(() -> Binder.bind(STRUCT, expr, true)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'X' in struct"); } @@ -204,7 +204,7 @@ public void testTransformExpressionBinding() { Expression bound = Binder.bind(STRUCT, equal(bucket("x", 16), 10)); TestHelpers.assertAllReferencesBound("BoundTransform", bound); BoundPredicate pred = TestHelpers.assertAndUnwrap(bound); - Assertions.assertThat(pred.term()) + assertThat(pred.term()) .as("Should use a BoundTransform child") .isInstanceOf(BoundTransform.class); BoundTransform transformExpr = (BoundTransform) pred.term(); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java index debbc7493da7..8bb03c633ab5 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionHelpers.java @@ -45,13 +45,13 @@ import static org.apache.iceberg.expressions.Expressions.truncate; import static org.apache.iceberg.expressions.Expressions.year; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.concurrent.Callable; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.types.Types.StructType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestExpressionHelpers { @@ -171,14 +171,14 @@ public void testTransformExpressions() { @Test public void testNullName() { - Assertions.assertThatThrownBy(() -> equal((String) null, 5)) + assertThatThrownBy(() -> equal((String) null, 5)) .isInstanceOf(NullPointerException.class) .hasMessage("Name cannot be null"); } @Test public void testNullValueExpr() { - Assertions.assertThatThrownBy(() -> equal((UnboundTerm) null, 5)) + assertThatThrownBy(() -> equal((UnboundTerm) null, 5)) .isInstanceOf(NullPointerException.class) .hasMessage("Term cannot be null"); } @@ -222,7 +222,7 @@ public void testInvalidateNaNInput() { } private void assertInvalidateNaNThrows(Callable> callable) { - Assertions.assertThatThrownBy(callable::call) + assertThatThrownBy(callable::call) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot create expression literal from NaN"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index 9a27830543ad..894989f38e7d 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -33,7 +33,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestExpressionUtil { @@ -114,7 +113,7 @@ public void testSanitizeLongIn() { @Test public void zeroAndNegativeNumberHandling() { - Assertions.assertThat( + assertThat( ExpressionUtil.toSanitizedString( Expressions.in( "test", @@ -734,7 +733,7 @@ public void testSanitizeStringFallback() { "2022-04-29T23:70:51-07:00", "2022-04-29T23:49:51.123456+100:00")) { String sanitizedFilter = ExpressionUtil.toSanitizedString(Expressions.equal("test", filter)); - Assertions.assertThat(filterPattern.matcher(sanitizedFilter)).matches(); + assertThat(filterPattern.matcher(sanitizedFilter)).matches(); } } @@ -989,7 +988,7 @@ public void testSelectsPartitions() { } private void assertEquals(Expression expected, Expression actual) { - Assertions.assertThat(expected).isInstanceOf(UnboundPredicate.class); + assertThat(expected).isInstanceOf(UnboundPredicate.class); assertEquals((UnboundPredicate) expected, (UnboundPredicate) actual); } @@ -1000,7 +999,7 @@ private void assertEquals(UnboundPredicate expected, UnboundPredicate actu } private void assertEquals(UnboundTerm expected, UnboundTerm actual) { - Assertions.assertThat(expected) + assertThat(expected) .as("Unknown expected term: " + expected) .isOfAnyClassIn(NamedReference.class, UnboundTransform.class); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java index 1a0fac86d45e..068c862e2bda 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveManifestEvaluator.java @@ -39,6 +39,7 @@ 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.nio.ByteBuffer; import org.apache.iceberg.ManifestFile; @@ -48,7 +49,6 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestInclusiveManifestEvaluator { @@ -245,7 +245,7 @@ public void testNotNaN() { @Test public void testMissingColumn() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ManifestEvaluator.forRowFilter(lessThan("missing", 5), SPEC, true).eval(FILE)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing'"); @@ -546,7 +546,7 @@ public void testCaseInsensitiveIntegerNotEqRewritten() { @Test public void testCaseSensitiveIntegerNotEqRewritten() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ManifestEvaluator.forRowFilter(not(equal("ID", 5)), SPEC, true).eval(FILE)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'ID'"); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java index 6c3e97669972..251f1942e9dd 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestInclusiveMetricsEvaluator.java @@ -39,6 +39,7 @@ 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.util.List; import org.apache.iceberg.DataFile; @@ -52,7 +53,6 @@ import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.util.UnicodeUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestInclusiveMetricsEvaluator { @@ -295,7 +295,7 @@ public void testRequiredColumn() { @Test public void testMissingColumn() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new InclusiveMetricsEvaluator(SCHEMA, lessThan("missing", 5)).eval(FILE)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing'"); @@ -612,7 +612,7 @@ public void testCaseInsensitiveIntegerNotEqRewritten() { @Test public void testCaseSensitiveIntegerNotEqRewritten() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> new InclusiveMetricsEvaluator(SCHEMA, not(equal("ID", 5)), true).eval(FILE)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'ID'"); diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java index 3d7925920a0a..a07c8fd1569d 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestPredicateBinding.java @@ -38,6 +38,7 @@ 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.math.BigDecimal; import java.util.Arrays; @@ -46,7 +47,6 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestPredicateBinding { @@ -79,7 +79,7 @@ public void testMissingField() { StructType struct = StructType.of(required(13, "x", Types.IntegerType.get())); UnboundPredicate unbound = new UnboundPredicate<>(LT, ref("missing"), 6); - Assertions.assertThatThrownBy(() -> unbound.bind(struct)) + assertThatThrownBy(() -> unbound.bind(struct)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing' in struct:"); } @@ -147,7 +147,7 @@ public void testInvalidConversions() { for (Expression.Operation op : COMPARISONS) { UnboundPredicate unbound = new UnboundPredicate<>(op, ref("f"), "12.40"); - Assertions.assertThatThrownBy(() -> unbound.bind(struct)) + assertThatThrownBy(() -> unbound.bind(struct)) .isInstanceOf(ValidationException.class) .hasMessage("Invalid value for conversion to type float: 12.40 (java.lang.String)"); } @@ -377,7 +377,7 @@ public void testIsNaN() { // string (non-compatible) StructType strStruct = StructType.of(optional(21, "s", Types.StringType.get())); - Assertions.assertThatThrownBy(() -> new UnboundPredicate<>(IS_NAN, ref("s")).bind(strStruct)) + assertThatThrownBy(() -> new UnboundPredicate<>(IS_NAN, ref("s")).bind(strStruct)) .isInstanceOf(ValidationException.class) .hasMessage("IsNaN cannot be used with a non-floating-point column"); } @@ -406,7 +406,7 @@ public void testNotNaN() { // string (non-compatible) StructType strStruct = StructType.of(optional(21, "s", Types.StringType.get())); - Assertions.assertThatThrownBy(() -> new UnboundPredicate<>(NOT_NAN, ref("s")).bind(strStruct)) + assertThatThrownBy(() -> new UnboundPredicate<>(NOT_NAN, ref("s")).bind(strStruct)) .isInstanceOf(ValidationException.class) .hasMessage("NotNaN cannot be used with a non-floating-point column"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java index 82aaff02149a..a23afb15dde5 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -37,6 +37,7 @@ 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 org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; @@ -47,7 +48,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.IntegerType; import org.apache.iceberg.types.Types.StringType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestStrictMetricsEvaluator { @@ -280,8 +280,7 @@ public void testRequiredColumn() { @Test public void testMissingColumn() { - Assertions.assertThatThrownBy( - () -> new StrictMetricsEvaluator(SCHEMA, lessThan("missing", 5)).eval(FILE)) + assertThatThrownBy(() -> new StrictMetricsEvaluator(SCHEMA, lessThan("missing", 5)).eval(FILE)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing'"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java index f35b274eb3d5..1dc2225b3805 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java @@ -19,6 +19,7 @@ package org.apache.iceberg.expressions; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; import java.time.DateTimeException; @@ -32,7 +33,6 @@ import org.apache.avro.Schema; import org.apache.avro.data.TimeConversions; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestStringLiteralConversions { @@ -182,7 +182,7 @@ public void testNegativeStringToTimestampLiteral() { public void testTimestampWithZoneWithoutZoneInLiteral() { // Zone must be present in literals when converting to timestamp with zone Literal timestampStr = Literal.of("2017-08-18T14:21:01.919"); - Assertions.assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withZone())) + assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } @@ -191,7 +191,7 @@ public void testTimestampWithZoneWithoutZoneInLiteral() { public void testTimestampWithoutZoneWithZoneInLiteral() { // Zone must not be present in literals when converting to timestamp without zone Literal timestampStr = Literal.of("2017-08-18T14:21:01.919+07:00"); - Assertions.assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withoutZone())) + assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withoutZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } diff --git a/api/src/test/java/org/apache/iceberg/io/TestCloseableGroup.java b/api/src/test/java/org/apache/iceberg/io/TestCloseableGroup.java index 4e9604fc1b09..8003c34e6be3 100644 --- a/api/src/test/java/org/apache/iceberg/io/TestCloseableGroup.java +++ b/api/src/test/java/org/apache/iceberg/io/TestCloseableGroup.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.Closeable; import java.io.IOException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -91,7 +92,7 @@ public void notSuppressExceptionIfSetSuppressIsFalse() throws Exception { closeableGroup.addCloseable(closeable2); closeableGroup.addCloseable(closeable3); - Assertions.assertThatThrownBy(closeableGroup::close).isEqualTo(ioException); + assertThatThrownBy(closeableGroup::close).isEqualTo(ioException); Mockito.verify(closeable1).close(); Mockito.verify(closeable2).close(); Mockito.verifyNoInteractions(closeable3); @@ -111,7 +112,7 @@ public void notSuppressExceptionIfSetSuppressIsFalseForAutoCloseable() throws Ex closeableGroup.addCloseable(closeable2); closeableGroup.addCloseable(closeable3); - Assertions.assertThatThrownBy(closeableGroup::close).isEqualTo(ioException); + assertThatThrownBy(closeableGroup::close).isEqualTo(ioException); Mockito.verify(closeable1).close(); Mockito.verify(closeable2).close(); Mockito.verifyNoInteractions(closeable3); @@ -126,7 +127,7 @@ public void wrapAutoCloseableFailuresWithRuntimeException() throws Exception { CloseableGroup closeableGroup = new CloseableGroup(); closeableGroup.addCloseable(throwingAutoCloseable); - Assertions.assertThatThrownBy(closeableGroup::close) + assertThatThrownBy(closeableGroup::close) .isInstanceOf(RuntimeException.class) .hasRootCause(generalException); } @@ -141,7 +142,7 @@ public void notWrapRuntimeException() throws Exception { CloseableGroup closeableGroup = new CloseableGroup(); closeableGroup.addCloseable(throwingCloseable); - Assertions.assertThatThrownBy(closeableGroup::close).isEqualTo(runtimeException); + assertThatThrownBy(closeableGroup::close).isEqualTo(runtimeException); } @Test @@ -153,6 +154,6 @@ public void notWrapRuntimeExceptionFromAutoCloseable() throws Exception { CloseableGroup closeableGroup = new CloseableGroup(); closeableGroup.addCloseable(throwingAutoCloseable); - Assertions.assertThatThrownBy(closeableGroup::close).isEqualTo(runtimeException); + assertThatThrownBy(closeableGroup::close).isEqualTo(runtimeException); } } diff --git a/api/src/test/java/org/apache/iceberg/io/TestCloseableIterable.java b/api/src/test/java/org/apache/iceberg/io/TestCloseableIterable.java index 5a148746cf50..ac60c6ff82c4 100644 --- a/api/src/test/java/org/apache/iceberg/io/TestCloseableIterable.java +++ b/api/src/test/java/org/apache/iceberg/io/TestCloseableIterable.java @@ -19,6 +19,7 @@ package org.apache.iceberg.io; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Arrays; @@ -32,7 +33,6 @@ import org.apache.iceberg.metrics.DefaultMetricsContext; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCloseableIterable { @@ -92,15 +92,14 @@ public void testConcatWithEmptyIterables() { CloseableIterable concat5 = CloseableIterable.concat(Lists.newArrayList(empty, empty, empty)); - Assertions.assertThatThrownBy(() -> Iterables.getLast(concat5)) - .isInstanceOf(NoSuchElementException.class); + assertThatThrownBy(() -> Iterables.getLast(concat5)).isInstanceOf(NoSuchElementException.class); } @Test public void testWithCompletionRunnable() throws IOException { AtomicInteger completionCounter = new AtomicInteger(0); List items = Lists.newArrayList(1, 2, 3, 4, 5); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CloseableIterable.whenComplete(CloseableIterable.combine(items, () -> {}), null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid runnable: null"); @@ -120,9 +119,9 @@ public void testWithCompletionRunnableAndEmptyIterable() throws IOException { try (CloseableIterable iter = CloseableIterable.whenComplete( CloseableIterable.combine(empty, () -> {}), completionCounter::incrementAndGet)) { - iter.forEach(val -> Assertions.assertThat(completionCounter.get()).isZero()); + iter.forEach(val -> assertThat(completionCounter.get()).isZero()); } - Assertions.assertThat(completionCounter.get()).isOne(); + assertThat(completionCounter.get()).isOne(); } @Test @@ -142,7 +141,7 @@ public void testWithCompletionRunnableWhenIterableThrows() { AtomicInteger completionCounter = new AtomicInteger(0); List items = Lists.newArrayList(1, 2, 3, 4, 5); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try (CloseableIterable iter = CloseableIterable.whenComplete( @@ -158,7 +157,7 @@ public void testWithCompletionRunnableWhenIterableThrows() { .isInstanceOf(RuntimeException.class) .hasMessage("expected"); - Assertions.assertThat(completionCounter.get()).isOne(); + assertThat(completionCounter.get()).isOne(); } @Test @@ -217,7 +216,7 @@ public void count() { counter, CloseableIterable.withNoopClose(Arrays.asList(1, 2, 3, 4, 5))); assertThat(counter.value()).isZero(); items.forEach(item -> {}); - Assertions.assertThat(counter.value()).isEqualTo(5); + assertThat(counter.value()).isEqualTo(5); } @Test @@ -230,45 +229,42 @@ public void countSkipped() { x -> x % 2 == 0); assertThat(counter.value()).isZero(); items.forEach(item -> {}); - Assertions.assertThat(counter.value()).isEqualTo(3); + assertThat(counter.value()).isEqualTo(3); } @Test public void countNullCheck() { - Assertions.assertThatThrownBy(() -> CloseableIterable.count(null, CloseableIterable.empty())) + assertThatThrownBy(() -> CloseableIterable.count(null, CloseableIterable.empty())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid counter: null"); Counter counter = new DefaultMetricsContext().counter("x"); - Assertions.assertThatThrownBy(() -> CloseableIterable.count(counter, null)) + assertThatThrownBy(() -> CloseableIterable.count(counter, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid iterable: null"); } @Test public void countSkippedNullCheck() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CloseableIterable.filter(null, CloseableIterable.empty(), Predicate.isEqual(true))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid counter: null"); Counter counter = new DefaultMetricsContext().counter("x"); - Assertions.assertThatThrownBy( - () -> CloseableIterable.filter(counter, null, Predicate.isEqual(true))) + assertThatThrownBy(() -> CloseableIterable.filter(counter, null, Predicate.isEqual(true))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid iterable: null"); - Assertions.assertThatThrownBy( - () -> CloseableIterable.filter(counter, CloseableIterable.empty(), null)) + assertThatThrownBy(() -> CloseableIterable.filter(counter, CloseableIterable.empty(), null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid predicate: null"); } @Test public void transformNullCheck() { - Assertions.assertThatThrownBy( - () -> CloseableIterable.transform(CloseableIterable.empty(), null)) + assertThatThrownBy(() -> CloseableIterable.transform(CloseableIterable.empty(), null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid transform: null"); } diff --git a/api/src/test/java/org/apache/iceberg/io/TestClosingIterator.java b/api/src/test/java/org/apache/iceberg/io/TestClosingIterator.java index 19bca2edc25f..0d9690713096 100644 --- a/api/src/test/java/org/apache/iceberg/io/TestClosingIterator.java +++ b/api/src/test/java/org/apache/iceberg/io/TestClosingIterator.java @@ -19,12 +19,12 @@ package org.apache.iceberg.io; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestClosingIterator { @@ -67,8 +67,7 @@ public void testCloseCalledOnceForMultipleHasNextCalls() throws Exception { @Test public void transformNullCheck() { - Assertions.assertThatThrownBy( - () -> CloseableIterator.transform(CloseableIterator.empty(), null)) + assertThatThrownBy(() -> CloseableIterator.transform(CloseableIterator.empty(), null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid transform: null"); } diff --git a/api/src/test/java/org/apache/iceberg/metrics/TestDefaultCounter.java b/api/src/test/java/org/apache/iceberg/metrics/TestDefaultCounter.java index f77da34dcac3..3e606c22442e 100644 --- a/api/src/test/java/org/apache/iceberg/metrics/TestDefaultCounter.java +++ b/api/src/test/java/org/apache/iceberg/metrics/TestDefaultCounter.java @@ -18,24 +18,26 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.metrics.MetricsContext.Unit; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestDefaultCounter { @Test public void nullCheck() { - Assertions.assertThatThrownBy(() -> new DefaultMetricsContext().counter("test", null)) + assertThatThrownBy(() -> new DefaultMetricsContext().counter("test", null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid count unit: null"); } @Test public void noop() { - Assertions.assertThat(DefaultCounter.NOOP.unit()).isEqualTo(Unit.UNDEFINED); - Assertions.assertThat(DefaultCounter.NOOP.isNoop()).isTrue(); - Assertions.assertThatThrownBy(DefaultCounter.NOOP::value) + assertThat(DefaultCounter.NOOP.unit()).isEqualTo(Unit.UNDEFINED); + assertThat(DefaultCounter.NOOP.isNoop()).isTrue(); + assertThatThrownBy(DefaultCounter.NOOP::value) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("NOOP counter has no value"); } @@ -45,8 +47,8 @@ public void count() { Counter counter = new DefaultCounter(Unit.BYTES); counter.increment(); counter.increment(5L); - Assertions.assertThat(counter.value()).isEqualTo(6L); - Assertions.assertThat(counter.unit()).isEqualTo(MetricsContext.Unit.BYTES); - Assertions.assertThat(counter.isNoop()).isFalse(); + assertThat(counter.value()).isEqualTo(6L); + assertThat(counter.unit()).isEqualTo(MetricsContext.Unit.BYTES); + assertThat(counter.isNoop()).isFalse(); } } diff --git a/api/src/test/java/org/apache/iceberg/metrics/TestDefaultMetricsContext.java b/api/src/test/java/org/apache/iceberg/metrics/TestDefaultMetricsContext.java index 3819485040e3..68ca208ab07c 100644 --- a/api/src/test/java/org/apache/iceberg/metrics/TestDefaultMetricsContext.java +++ b/api/src/test/java/org/apache/iceberg/metrics/TestDefaultMetricsContext.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.withinPercentage; import java.time.Duration; import java.util.concurrent.TimeUnit; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestDefaultMetricsContext { @@ -30,7 +31,7 @@ public class TestDefaultMetricsContext { @Test public void unsupportedCounter() { MetricsContext metricsContext = new DefaultMetricsContext(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> metricsContext.counter("test", Double.class, MetricsContext.Unit.COUNT)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Counter for type java.lang.Double is not supported"); @@ -38,8 +39,7 @@ public void unsupportedCounter() { @Test public void intCounterNullCheck() { - Assertions.assertThatThrownBy( - () -> new DefaultMetricsContext().counter("name", Integer.class, null)) + assertThatThrownBy(() -> new DefaultMetricsContext().counter("name", Integer.class, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid count unit: null"); } @@ -50,8 +50,8 @@ public void intCounter() { MetricsContext.Counter counter = metricsContext.counter("intCounter", Integer.class, MetricsContext.Unit.BYTES); counter.increment(5); - Assertions.assertThat(counter.value()).isEqualTo(5); - Assertions.assertThat(counter.unit()).isEqualTo(MetricsContext.Unit.BYTES); + assertThat(counter.value()).isEqualTo(5); + assertThat(counter.unit()).isEqualTo(MetricsContext.Unit.BYTES); } @Test @@ -61,15 +61,14 @@ public void intCounterOverflow() { metricsContext.counter("test", Integer.class, MetricsContext.Unit.COUNT); counter.increment(Integer.MAX_VALUE); counter.increment(); - Assertions.assertThatThrownBy(counter::value) + assertThatThrownBy(counter::value) .isInstanceOf(ArithmeticException.class) .hasMessage("integer overflow"); } @Test public void longCounterNullCheck() { - Assertions.assertThatThrownBy( - () -> new DefaultMetricsContext().counter("name", Long.class, null)) + assertThatThrownBy(() -> new DefaultMetricsContext().counter("name", Long.class, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid count unit: null"); } @@ -80,8 +79,8 @@ public void longCounter() { MetricsContext.Counter counter = metricsContext.counter("longCounter", Long.class, MetricsContext.Unit.COUNT); counter.increment(5L); - Assertions.assertThat(counter.value()).isEqualTo(5L); - Assertions.assertThat(counter.unit()).isEqualTo(MetricsContext.Unit.COUNT); + assertThat(counter.value()).isEqualTo(5L); + assertThat(counter.unit()).isEqualTo(MetricsContext.Unit.COUNT); } @Test @@ -89,7 +88,7 @@ public void timer() { MetricsContext metricsContext = new DefaultMetricsContext(); Timer timer = metricsContext.timer("test", TimeUnit.MICROSECONDS); timer.record(10, TimeUnit.MINUTES); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ofMinutes(10L)); + assertThat(timer.totalDuration()).isEqualTo(Duration.ofMinutes(10L)); } @Test @@ -101,18 +100,18 @@ public void histogram() { histogram.update(i); } - Assertions.assertThat(histogram.count()).isEqualTo(reservoirSize); + assertThat(histogram.count()).isEqualTo(reservoirSize); Histogram.Statistics statistics = histogram.statistics(); - Assertions.assertThat(statistics.size()).isEqualTo(reservoirSize); - Assertions.assertThat(statistics.mean()).isEqualTo(500.5); - Assertions.assertThat(statistics.stdDev()).isCloseTo(288.67499, withinPercentage(0.001)); - Assertions.assertThat(statistics.max()).isEqualTo(1000L); - Assertions.assertThat(statistics.min()).isEqualTo(1L); - Assertions.assertThat(statistics.percentile(0.50)).isEqualTo(500); - Assertions.assertThat(statistics.percentile(0.75)).isEqualTo(750); - Assertions.assertThat(statistics.percentile(0.90)).isEqualTo(900); - Assertions.assertThat(statistics.percentile(0.95)).isEqualTo(950); - Assertions.assertThat(statistics.percentile(0.99)).isEqualTo(990); - Assertions.assertThat(statistics.percentile(0.999)).isEqualTo(999); + assertThat(statistics.size()).isEqualTo(reservoirSize); + assertThat(statistics.mean()).isEqualTo(500.5); + assertThat(statistics.stdDev()).isCloseTo(288.67499, withinPercentage(0.001)); + assertThat(statistics.max()).isEqualTo(1000L); + assertThat(statistics.min()).isEqualTo(1L); + assertThat(statistics.percentile(0.50)).isEqualTo(500); + assertThat(statistics.percentile(0.75)).isEqualTo(750); + assertThat(statistics.percentile(0.90)).isEqualTo(900); + assertThat(statistics.percentile(0.95)).isEqualTo(950); + assertThat(statistics.percentile(0.99)).isEqualTo(990); + assertThat(statistics.percentile(0.999)).isEqualTo(999); } } diff --git a/api/src/test/java/org/apache/iceberg/metrics/TestDefaultTimer.java b/api/src/test/java/org/apache/iceberg/metrics/TestDefaultTimer.java index 950bbc931da4..a07188804dce 100644 --- a/api/src/test/java/org/apache/iceberg/metrics/TestDefaultTimer.java +++ b/api/src/test/java/org/apache/iceberg/metrics/TestDefaultTimer.java @@ -20,6 +20,8 @@ import static java.util.concurrent.Executors.newFixedThreadPool; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.time.Duration; import java.util.List; @@ -31,14 +33,13 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestDefaultTimer { @Test public void nullCheck() { - Assertions.assertThatThrownBy(() -> new DefaultTimer(null)) + assertThatThrownBy(() -> new DefaultTimer(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid time unit: null"); } @@ -46,20 +47,20 @@ public void nullCheck() { @Test public void nameAndUnit() { DefaultTimer timer = new DefaultTimer(TimeUnit.MINUTES); - Assertions.assertThat(timer.unit()).isEqualTo(TimeUnit.MINUTES); - Assertions.assertThat(timer.isNoop()).isFalse(); + assertThat(timer.unit()).isEqualTo(TimeUnit.MINUTES); + assertThat(timer.isNoop()).isFalse(); } @Test public void noop() { - Assertions.assertThat(Timer.NOOP.isNoop()).isTrue(); - Assertions.assertThatThrownBy(Timer.NOOP::count) + assertThat(Timer.NOOP.isNoop()).isTrue(); + assertThatThrownBy(Timer.NOOP::count) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("NOOP timer has no count"); - Assertions.assertThatThrownBy(Timer.NOOP::totalDuration) + assertThatThrownBy(Timer.NOOP::totalDuration) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("NOOP timer has no duration"); - Assertions.assertThatThrownBy(Timer.NOOP::unit) + assertThatThrownBy(Timer.NOOP::unit) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("NOOP timer has no unit"); } @@ -67,12 +68,12 @@ public void noop() { @Test public void recordNegativeAmount() { Timer timer = new DefaultTimer(TimeUnit.NANOSECONDS); - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThatThrownBy(() -> timer.record(-1, TimeUnit.NANOSECONDS)) + assertThat(timer.count()).isEqualTo(0); + assertThatThrownBy(() -> timer.record(-1, TimeUnit.NANOSECONDS)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot record -1 NANOSECONDS: must be >= 0"); - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(timer.count()).isEqualTo(0); + assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); } @Test @@ -81,7 +82,7 @@ public void multipleStops() { Timer.Timed timed = timer.start(); timed.stop(); // we didn't start the timer again - Assertions.assertThatThrownBy(timed::stop) + assertThatThrownBy(timed::stop) .isInstanceOf(IllegalStateException.class) .hasMessage("stop() called multiple times"); } @@ -89,13 +90,13 @@ public void multipleStops() { @Test public void closeableTimer() throws InterruptedException { Timer timer = new DefaultTimer(TimeUnit.NANOSECONDS); - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(timer.count()).isEqualTo(0); + assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); try (Timer.Timed sample = timer.start()) { Thread.sleep(500L); } - Assertions.assertThat(timer.count()).isEqualTo(1); - Assertions.assertThat(timer.totalDuration()).isGreaterThan(Duration.ZERO); + assertThat(timer.count()).isEqualTo(1); + assertThat(timer.totalDuration()).isGreaterThan(Duration.ZERO); } @Test @@ -109,18 +110,18 @@ public void measureRunnable() { throw new RuntimeException(e); } }; - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(timer.count()).isEqualTo(0); + assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); timer.time(runnable); - Assertions.assertThat(timer.count()).isEqualTo(1); + assertThat(timer.count()).isEqualTo(1); Duration duration = timer.totalDuration(); - Assertions.assertThat(duration).isGreaterThan(Duration.ZERO); + assertThat(duration).isGreaterThan(Duration.ZERO); timer.time(runnable); - Assertions.assertThat(timer.count()).isEqualTo(2); + assertThat(timer.count()).isEqualTo(2); Duration secondDuration = timer.totalDuration(); - Assertions.assertThat(secondDuration).isGreaterThan(duration); + assertThat(secondDuration).isGreaterThan(duration); } @Test @@ -135,18 +136,18 @@ public void measureCallable() throws Exception { } return true; }; - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(timer.count()).isEqualTo(0); + assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); - Assertions.assertThat(timer.timeCallable(callable).booleanValue()).isTrue(); - Assertions.assertThat(timer.count()).isEqualTo(1); + assertThat(timer.timeCallable(callable).booleanValue()).isTrue(); + assertThat(timer.count()).isEqualTo(1); Duration duration = timer.totalDuration(); - Assertions.assertThat(duration).isGreaterThan(Duration.ZERO); + assertThat(duration).isGreaterThan(Duration.ZERO); - Assertions.assertThat(timer.timeCallable(callable).booleanValue()).isTrue(); - Assertions.assertThat(timer.count()).isEqualTo(2); + assertThat(timer.timeCallable(callable).booleanValue()).isTrue(); + assertThat(timer.count()).isEqualTo(2); Duration secondDuration = timer.totalDuration(); - Assertions.assertThat(secondDuration).isGreaterThan(duration); + assertThat(secondDuration).isGreaterThan(duration); } @Test @@ -161,18 +162,18 @@ public void measureSupplier() { } return true; }; - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(timer.count()).isEqualTo(0); + assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); - Assertions.assertThat(timer.time(supplier).booleanValue()).isTrue(); - Assertions.assertThat(timer.count()).isEqualTo(1); + assertThat(timer.time(supplier).booleanValue()).isTrue(); + assertThat(timer.count()).isEqualTo(1); Duration duration = timer.totalDuration(); - Assertions.assertThat(duration).isGreaterThan(Duration.ZERO); + assertThat(duration).isGreaterThan(Duration.ZERO); - Assertions.assertThat(timer.time(supplier).booleanValue()).isTrue(); - Assertions.assertThat(timer.count()).isEqualTo(2); + assertThat(timer.time(supplier).booleanValue()).isTrue(); + assertThat(timer.count()).isEqualTo(2); Duration secondDuration = timer.totalDuration(); - Assertions.assertThat(secondDuration).isGreaterThan(duration); + assertThat(secondDuration).isGreaterThan(duration); } @Test @@ -198,19 +199,19 @@ public void measureNestedRunnables() { } }; - Assertions.assertThat(timer.count()).isEqualTo(0); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); - Assertions.assertThat(innerTimer.count()).isEqualTo(0); - Assertions.assertThat(innerTimer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(timer.count()).isEqualTo(0); + assertThat(timer.totalDuration()).isEqualTo(Duration.ZERO); + assertThat(innerTimer.count()).isEqualTo(0); + assertThat(innerTimer.totalDuration()).isEqualTo(Duration.ZERO); timer.time(outer); - Assertions.assertThat(timer.count()).isEqualTo(1); + assertThat(timer.count()).isEqualTo(1); Duration outerDuration = timer.totalDuration(); - Assertions.assertThat(outerDuration).isGreaterThan(Duration.ZERO); - Assertions.assertThat(innerTimer.count()).isEqualTo(1); + assertThat(outerDuration).isGreaterThan(Duration.ZERO); + assertThat(innerTimer.count()).isEqualTo(1); Duration innerDuration = innerTimer.totalDuration(); - Assertions.assertThat(innerDuration).isGreaterThan(Duration.ZERO); - Assertions.assertThat(outerDuration).isGreaterThan(innerDuration); + assertThat(innerDuration).isGreaterThan(Duration.ZERO); + assertThat(outerDuration).isGreaterThan(innerDuration); } @Test @@ -249,7 +250,7 @@ public void multiThreadedStarts() throws InterruptedException { executor.shutdownNow(); executor.awaitTermination(5, SECONDS); - Assertions.assertThat(timer.totalDuration()).isEqualTo(Duration.ofNanos(5 * threads)); - Assertions.assertThat(timer.count()).isEqualTo(threads); + assertThat(timer.totalDuration()).isEqualTo(Duration.ofNanos(5 * threads)); + assertThat(timer.count()).isEqualTo(threads); } } diff --git a/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java b/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java index 25e323c02699..018c2c3b58b2 100644 --- a/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java +++ b/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java @@ -20,6 +20,8 @@ import static java.util.concurrent.Executors.newFixedThreadPool; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.withinPercentage; import java.util.List; @@ -28,37 +30,36 @@ import java.util.concurrent.Future; import java.util.stream.Collectors; import java.util.stream.IntStream; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestFixedReservoirHistogram { @Test public void emptyHistogram() { FixedReservoirHistogram histogram = new FixedReservoirHistogram(100); - Assertions.assertThat(histogram.count()).isEqualTo(0); + assertThat(histogram.count()).isEqualTo(0); Histogram.Statistics statistics = histogram.statistics(); - Assertions.assertThat(statistics.size()).isEqualTo(0); - Assertions.assertThat(statistics.mean()).isEqualTo(0.0); - Assertions.assertThat(statistics.stdDev()).isEqualTo(0.0); - Assertions.assertThat(statistics.max()).isEqualTo(0L); - Assertions.assertThat(statistics.min()).isEqualTo(0L); - Assertions.assertThat(statistics.percentile(0.50)).isEqualTo(0L); - Assertions.assertThat(statistics.percentile(0.99)).isEqualTo(0L); + assertThat(statistics.size()).isEqualTo(0); + assertThat(statistics.mean()).isEqualTo(0.0); + assertThat(statistics.stdDev()).isEqualTo(0.0); + assertThat(statistics.max()).isEqualTo(0L); + assertThat(statistics.min()).isEqualTo(0L); + assertThat(statistics.percentile(0.50)).isEqualTo(0L); + assertThat(statistics.percentile(0.99)).isEqualTo(0L); } @Test public void singleObservation() { FixedReservoirHistogram histogram = new FixedReservoirHistogram(100); histogram.update(123L); - Assertions.assertThat(histogram.count()).isEqualTo(1); + assertThat(histogram.count()).isEqualTo(1); Histogram.Statistics statistics = histogram.statistics(); - Assertions.assertThat(statistics.size()).isEqualTo(1); - Assertions.assertThat(statistics.mean()).isEqualTo(123.0); - Assertions.assertThat(statistics.stdDev()).isEqualTo(0.0); - Assertions.assertThat(statistics.max()).isEqualTo(123L); - Assertions.assertThat(statistics.min()).isEqualTo(123L); - Assertions.assertThat(statistics.percentile(0.50)).isEqualTo(123L); - Assertions.assertThat(statistics.percentile(0.99)).isEqualTo(123L); + assertThat(statistics.size()).isEqualTo(1); + assertThat(statistics.mean()).isEqualTo(123.0); + assertThat(statistics.stdDev()).isEqualTo(0.0); + assertThat(statistics.max()).isEqualTo(123L); + assertThat(statistics.min()).isEqualTo(123L); + assertThat(statistics.percentile(0.50)).isEqualTo(123L); + assertThat(statistics.percentile(0.99)).isEqualTo(123L); } @Test @@ -70,8 +71,8 @@ public void minMaxPercentilePoints() { } Histogram.Statistics statistics = histogram.statistics(); - Assertions.assertThat(statistics.percentile(0.0)).isEqualTo(0L); - Assertions.assertThat(statistics.percentile(1.0)).isEqualTo(99L); + assertThat(statistics.percentile(0.0)).isEqualTo(0L); + assertThat(statistics.percentile(1.0)).isEqualTo(99L); } @Test @@ -84,11 +85,11 @@ public void invalidPercentilePoints() { Histogram.Statistics statistics = histogram.statistics(); - Assertions.assertThatThrownBy(() -> statistics.percentile(-0.1)) + assertThatThrownBy(() -> statistics.percentile(-0.1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Percentile point cannot be outside the range of [0.0 - 1.0]: " + -0.1); - Assertions.assertThatThrownBy(() -> statistics.percentile(1.1)) + assertThatThrownBy(() -> statistics.percentile(1.1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Percentile point cannot be outside the range of [0.0 - 1.0]: " + 1.1); } @@ -136,17 +137,17 @@ public void testMultipleThreadWriters() throws InterruptedException { executor.awaitTermination(5, SECONDS); Histogram.Statistics statistics = histogram.statistics(); - Assertions.assertThat(histogram.count()).isEqualTo(totalSamples); - Assertions.assertThat(statistics.size()).isEqualTo(totalSamples); - Assertions.assertThat(statistics.mean()).isEqualTo(500.5); - Assertions.assertThat(statistics.stdDev()).isCloseTo(288.67499, withinPercentage(0.001)); - Assertions.assertThat(statistics.max()).isEqualTo(1000L); - Assertions.assertThat(statistics.min()).isEqualTo(1L); - Assertions.assertThat(statistics.percentile(0.50)).isEqualTo(500); - Assertions.assertThat(statistics.percentile(0.75)).isEqualTo(750); - Assertions.assertThat(statistics.percentile(0.90)).isEqualTo(900); - Assertions.assertThat(statistics.percentile(0.95)).isEqualTo(950); - Assertions.assertThat(statistics.percentile(0.99)).isEqualTo(990); - Assertions.assertThat(statistics.percentile(0.999)).isEqualTo(999); + assertThat(histogram.count()).isEqualTo(totalSamples); + assertThat(statistics.size()).isEqualTo(totalSamples); + assertThat(statistics.mean()).isEqualTo(500.5); + assertThat(statistics.stdDev()).isCloseTo(288.67499, withinPercentage(0.001)); + assertThat(statistics.max()).isEqualTo(1000L); + assertThat(statistics.min()).isEqualTo(1L); + assertThat(statistics.percentile(0.50)).isEqualTo(500); + assertThat(statistics.percentile(0.75)).isEqualTo(750); + assertThat(statistics.percentile(0.90)).isEqualTo(900); + assertThat(statistics.percentile(0.95)).isEqualTo(950); + assertThat(statistics.percentile(0.99)).isEqualTo(990); + assertThat(statistics.percentile(0.999)).isEqualTo(999); } } 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 b8a0e40c1110..f397f94f0ce5 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.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.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; @@ -34,7 +35,6 @@ import org.apache.iceberg.relocated.com.google.common.hash.Hashing; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.BucketUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -356,7 +356,7 @@ public void testUUIDHash() { @Test public void testVerifiedIllegalNumBuckets() { - Assertions.assertThatThrownBy(() -> Bucket.get(0)) + assertThatThrownBy(() -> Bucket.get(0)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid number of buckets: 0 (must be > 0)"); } diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestProjection.java b/api/src/test/java/org/apache/iceberg/transforms/TestProjection.java index ccfda895f9f1..6389e26c325e 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestProjection.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestProjection.java @@ -34,6 +34,7 @@ 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.util.List; import org.apache.iceberg.PartitionSpec; @@ -47,7 +48,6 @@ import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestProjection { @@ -130,8 +130,7 @@ public void testCaseInsensitiveIdentityProjection() { @Test public void testCaseSensitiveIdentityProjection() { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); - Assertions.assertThatThrownBy( - () -> Projections.inclusive(spec, true).project(Expressions.notNull("ID"))) + assertThatThrownBy(() -> Projections.inclusive(spec, true).project(Expressions.notNull("ID"))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'ID' in struct"); } @@ -213,8 +212,7 @@ public void testCaseInsensitiveStrictIdentityProjection() { @Test public void testCaseSensitiveStrictIdentityProjection() { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("id").build(); - Assertions.assertThatThrownBy( - () -> Projections.strict(spec, true).project(Expressions.notNull("ID"))) + assertThatThrownBy(() -> Projections.strict(spec, true).project(Expressions.notNull("ID"))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'ID' in struct"); } @@ -248,12 +246,12 @@ public void testBadSparkPartitionFilter() { Expression projection = Projections.inclusive(spec).project(filter); - Assertions.assertThat(projection).isInstanceOf(Or.class); + assertThat(projection).isInstanceOf(Or.class); Or or1 = (Or) projection; UnboundPredicate dateint1 = assertAndUnwrapUnbound(or1.left()); assertThat(dateint1.ref().name()).as("Should be a dateint predicate").isEqualTo("dateint"); assertThat(dateint1.literal().value()).as("Should be dateint=20180416").isEqualTo(20180416); - Assertions.assertThat(or1.right()).isInstanceOf(Or.class); + assertThat(or1.right()).isInstanceOf(Or.class); Or or2 = (Or) or1.right(); UnboundPredicate dateint2 = assertAndUnwrapUnbound(or2.left()); assertThat(dateint2.ref().name()).as("Should be a dateint predicate").isEqualTo("dateint"); diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestResiduals.java b/api/src/test/java/org/apache/iceberg/transforms/TestResiduals.java index fa3436e5701b..12214fce8bc3 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestResiduals.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestResiduals.java @@ -33,6 +33,7 @@ import static org.apache.iceberg.expressions.Expressions.notNaN; import static org.apache.iceberg.expressions.Expressions.or; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.function.Function; import org.apache.iceberg.PartitionSpec; @@ -45,7 +46,6 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestResiduals { @@ -144,7 +144,7 @@ public void testCaseSensitiveIdentityTransformResiduals() { ResidualEvaluator resEval = ResidualEvaluator.of(spec, lessThan("DATEINT", 20170815), true); - Assertions.assertThatThrownBy(() -> resEval.residualFor(Row.of(20170815))) + assertThatThrownBy(() -> resEval.residualFor(Row.of(20170815))) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'DATEINT' in struct"); } diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestStartsWith.java b/api/src/test/java/org/apache/iceberg/transforms/TestStartsWith.java index d8aa59e7fe73..a9a992ea2093 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestStartsWith.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestStartsWith.java @@ -35,7 +35,6 @@ import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestStartsWith { @@ -57,7 +56,7 @@ public void testTruncateProjections() { assertProjectionStrict(spec, startsWith(COLUMN, "abab"), "abab", Expression.Operation.EQ); Expression projection = Projections.strict(spec).project(startsWith(COLUMN, "ababab")); - Assertions.assertThat(projection).isInstanceOf(False.class); + assertThat(projection).isInstanceOf(False.class); } @Test diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java b/api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java index e9ee6cfde385..68527b0294d3 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTruncate.java @@ -19,13 +19,13 @@ 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; import java.nio.charset.StandardCharsets; import java.util.function.Function; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestTruncate { @@ -112,7 +112,7 @@ public void testTruncateByteBuffer() { @Test public void testVerifiedIllegalWidth() { - Assertions.assertThatThrownBy(() -> Truncate.get(0)) + assertThatThrownBy(() -> Truncate.get(0)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid truncate width: 0 (must be > 0)"); } 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 81f32f8ee94a..e8db0937eb73 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypeUtil.java @@ -28,7 +28,6 @@ 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.IntegerType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestTypeUtil { @@ -445,15 +444,15 @@ public void testProjectListNested() { required(17, "x", Types.IntegerType.get()), required(18, "y", Types.IntegerType.get())))))))); - Assertions.assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(12))) + assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(12))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot explicitly project List or Map types"); - Assertions.assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(13))) + assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(13))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot explicitly project List or Map types"); - Assertions.assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(14))) + assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(14))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot explicitly project List or Map types"); @@ -496,15 +495,15 @@ public void testProjectMapNested() { required(18, "x", Types.IntegerType.get()), required(19, "y", Types.IntegerType.get())))))))); - Assertions.assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(12))) + assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(12))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot explicitly project List or Map types"); - Assertions.assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(14))) + assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(14))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot explicitly project List or Map types"); - Assertions.assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(16))) + assertThatThrownBy(() -> TypeUtil.project(schema, Sets.newHashSet(16))) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot explicitly project List or Map types"); @@ -534,7 +533,7 @@ public void testReassignIdsIllegalArgumentException() { new Schema( required(1, "a", Types.IntegerType.get()), required(2, "b", Types.IntegerType.get())); Schema sourceSchema = new Schema(required(1, "a", Types.IntegerType.get())); - Assertions.assertThatThrownBy(() -> TypeUtil.reassignIds(schema, sourceSchema)) + assertThatThrownBy(() -> TypeUtil.reassignIds(schema, sourceSchema)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Field b not found in source schema"); } @@ -549,7 +548,7 @@ public void testValidateSchemaViaIndexByName() { required(2, "b", Types.StructType.of(required(3, "c", Types.BooleanType.get()))), required(4, "b.c", Types.BooleanType.get()))); - Assertions.assertThatThrownBy(() -> TypeUtil.indexByName(Types.StructType.of(nestedType))) + assertThatThrownBy(() -> TypeUtil.indexByName(Types.StructType.of(nestedType))) .isInstanceOf(RuntimeException.class) .hasMessageContaining("Invalid schema: multiple fields for name a.b.c"); } diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypes.java b/api/src/test/java/org/apache/iceberg/types/TestTypes.java index ca5c6edce16b..5db7ca7cd1bc 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypes.java @@ -18,29 +18,27 @@ */ package org.apache.iceberg.types; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import org.junit.jupiter.api.Test; public class TestTypes { @Test public void fromPrimitiveString() { - Assertions.assertThat(Types.fromPrimitiveString("boolean")).isSameAs(Types.BooleanType.get()); - Assertions.assertThat(Types.fromPrimitiveString("BooLean")).isSameAs(Types.BooleanType.get()); + assertThat(Types.fromPrimitiveString("boolean")).isSameAs(Types.BooleanType.get()); + assertThat(Types.fromPrimitiveString("BooLean")).isSameAs(Types.BooleanType.get()); - Assertions.assertThat(Types.fromPrimitiveString("timestamp")) - .isSameAs(Types.TimestampType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamp")).isSameAs(Types.TimestampType.withoutZone()); - Assertions.assertThat(Types.fromPrimitiveString("Fixed[ 3 ]")) - .isEqualTo(Types.FixedType.ofLength(3)); + assertThat(Types.fromPrimitiveString("Fixed[ 3 ]")).isEqualTo(Types.FixedType.ofLength(3)); - Assertions.assertThat(Types.fromPrimitiveString("Decimal( 2 , 3 )")) - .isEqualTo(Types.DecimalType.of(2, 3)); + assertThat(Types.fromPrimitiveString("Decimal( 2 , 3 )")).isEqualTo(Types.DecimalType.of(2, 3)); - Assertions.assertThat(Types.fromPrimitiveString("Decimal(2,3)")) - .isEqualTo(Types.DecimalType.of(2, 3)); + assertThat(Types.fromPrimitiveString("Decimal(2,3)")).isEqualTo(Types.DecimalType.of(2, 3)); - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> Types.fromPrimitiveString("Unknown")) .withMessageContaining("Unknown"); } diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java index 8ca7889b4717..2154cf31a9f6 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java @@ -25,15 +25,14 @@ import java.util.concurrent.TimeUnit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCharSequenceMap { @Test public void nullString() { - Assertions.assertThat(CharSequenceMap.create()).doesNotContainKey((String) null); - Assertions.assertThat(CharSequenceMap.create()).doesNotContainValue((String) null); + assertThat(CharSequenceMap.create()).doesNotContainKey((String) null); + assertThat(CharSequenceMap.create()).doesNotContainValue((String) null); } @Test diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java index b0f242c177f6..324742c07a2d 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceSet.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Arrays; import java.util.Collections; import java.util.Set; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCharSequenceSet { @@ -32,60 +33,60 @@ public class TestCharSequenceSet { @Test public void testSearchingInCharSequenceCollection() { Set set = CharSequenceSet.of(Arrays.asList("abc", new StringBuffer("def"))); - Assertions.assertThat(set).contains("abc"); - Assertions.assertThat(set.stream().anyMatch("def"::contains)).isTrue(); + assertThat(set).contains("abc"); + assertThat(set.stream().anyMatch("def"::contains)).isTrue(); // this would fail with a normal Set - Assertions.assertThat(set.contains("def")).isTrue(); + assertThat(set.contains("def")).isTrue(); } @Test public void nullString() { - Assertions.assertThat(CharSequenceSet.of(Arrays.asList((String) null))).contains((String) null); - Assertions.assertThat(CharSequenceSet.empty()).doesNotContain((String) null); + assertThat(CharSequenceSet.of(Arrays.asList((String) null))).contains((String) null); + assertThat(CharSequenceSet.empty()).doesNotContain((String) null); } @Test public void testRetainAll() { CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("123", "456")); - Assertions.assertThat(set.retainAll(ImmutableList.of("456", "789", 123))) + assertThat(set.retainAll(ImmutableList.of("456", "789", 123))) .overridingErrorMessage("Set should be changed") .isTrue(); - Assertions.assertThat(set).hasSize(1).contains("456"); + assertThat(set).hasSize(1).contains("456"); set = CharSequenceSet.of(ImmutableList.of("123", "456")); - Assertions.assertThat(set.retainAll(ImmutableList.of("123", "456"))) + assertThat(set.retainAll(ImmutableList.of("123", "456"))) .overridingErrorMessage("Set should not be changed") .isFalse(); - Assertions.assertThat(set.retainAll(ImmutableList.of(123, 456))) + assertThat(set.retainAll(ImmutableList.of(123, 456))) .overridingErrorMessage("Set should be changed") .isTrue(); - Assertions.assertThat(set).isEmpty(); + assertThat(set).isEmpty(); } @Test public void testRemoveAll() { CharSequenceSet set = CharSequenceSet.of(ImmutableList.of("123", "456")); - Assertions.assertThat(set.removeAll(ImmutableList.of("456", "789", 123))) + assertThat(set.removeAll(ImmutableList.of("456", "789", 123))) .overridingErrorMessage("Set should be changed") .isTrue(); - Assertions.assertThat(set).hasSize(1).contains("123"); + assertThat(set).hasSize(1).contains("123"); set = CharSequenceSet.of(ImmutableList.of("123", "456")); - Assertions.assertThat(set.removeAll(ImmutableList.of(123, 456))) + assertThat(set.removeAll(ImmutableList.of(123, 456))) .overridingErrorMessage("Set should not be changed") .isFalse(); - Assertions.assertThat(set.removeAll(ImmutableList.of("123", "456"))) + assertThat(set.removeAll(ImmutableList.of("123", "456"))) .overridingErrorMessage("Set should be changed") .isTrue(); - Assertions.assertThat(set).isEmpty(); + assertThat(set).isEmpty(); } @Test @@ -93,8 +94,8 @@ public void testEqualsAndHashCode() { CharSequenceSet set1 = CharSequenceSet.empty(); CharSequenceSet set2 = CharSequenceSet.empty(); - Assertions.assertThat(set1).isEqualTo(set2); - Assertions.assertThat(set1.hashCode()).isEqualTo(set2.hashCode()); + assertThat(set1).isEqualTo(set2); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()); set1.add("v1"); set1.add("v2"); @@ -112,8 +113,8 @@ public void testEqualsAndHashCode() { CharSequenceWrapper.wrap(new StringBuffer("v2")), CharSequenceWrapper.wrap(new StringBuffer("v3"))); - Assertions.assertThat(set1).isEqualTo(set2).isEqualTo(set3).isEqualTo(set4); - Assertions.assertThat(set1.hashCode()) + assertThat(set1).isEqualTo(set2).isEqualTo(set3).isEqualTo(set4); + assertThat(set1.hashCode()) .isEqualTo(set2.hashCode()) .isEqualTo(set3.hashCode()) .isEqualTo(set4.hashCode()); diff --git a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java index cf6b9ccb790d..42da3a3dd774 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.ZonedDateTime; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestDateTimeUtil { @@ -27,12 +28,11 @@ public class TestDateTimeUtil { @Test public void formatTimestampMillis() { String timestamp = "1970-01-01T00:00:00.001+00:00"; - Assertions.assertThat(DateTimeUtil.formatTimestampMillis(1L)).isEqualTo(timestamp); - Assertions.assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1L); + assertThat(DateTimeUtil.formatTimestampMillis(1L)).isEqualTo(timestamp); + assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1L); timestamp = "1970-01-01T00:16:40+00:00"; - Assertions.assertThat(DateTimeUtil.formatTimestampMillis(1000000L)).isEqualTo(timestamp); - Assertions.assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()) - .isEqualTo(1000000L); + assertThat(DateTimeUtil.formatTimestampMillis(1000000L)).isEqualTo(timestamp); + assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1000000L); } } 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 f2106cc62c5f..e6c3cf5c20ba 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestExceptionUtil.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Arrays; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestExceptionUtil { @@ -36,7 +37,7 @@ public void testRunSafely() { CustomCheckedException exc = new CustomCheckedException("test"); Exception suppressedOne = new Exception("test catch suppression"); RuntimeException suppressedTwo = new RuntimeException("test finally suppression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExceptionUtil.runSafely( () -> { @@ -62,7 +63,7 @@ public void testRunSafelyTwoExceptions() { CustomCheckedException exc = new CustomCheckedException("test"); Exception suppressedOne = new Exception("test catch suppression"); RuntimeException suppressedTwo = new RuntimeException("test finally suppression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExceptionUtil.runSafely( (ExceptionUtil.Block< @@ -91,7 +92,7 @@ public void testRunSafelyThreeExceptions() { CustomCheckedException exc = new CustomCheckedException("test"); Exception suppressedOne = new Exception("test catch suppression"); RuntimeException suppressedTwo = new RuntimeException("test finally suppression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExceptionUtil.runSafely( (ExceptionUtil.Block< @@ -121,7 +122,7 @@ public void testRunSafelyRuntimeExceptions() { RuntimeException exc = new RuntimeException("test"); Exception suppressedOne = new Exception("test catch suppression"); CustomCheckedException suppressedTwo = new CustomCheckedException("test finally suppression"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExceptionUtil.runSafely( () -> { diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java index 9cd9c8cc5abf..9364affe3915 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/ArrowReaderTest.java @@ -84,7 +84,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.UUIDUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -653,7 +652,7 @@ private static void checkColumnarArrayValues( // we need to use assertThat() here because it does a java.util.Objects.deepEquals() and // that // is relevant for byte[] - Assertions.assertThat(actualValue).as("Row#" + i + " mismatches").isEqualTo(expectedValue); + assertThat(actualValue).as("Row#" + i + " mismatches").isEqualTo(expectedValue); } } } @@ -1205,7 +1204,7 @@ private static void checkVectorValues( // we need to use assertThat() here because it does a java.util.Objects.deepEquals() and // that // is relevant for byte[] - Assertions.assertThat(actualValue).as("Row#" + i + " mismatches").isEqualTo(expectedValue); + assertThat(actualValue).as("Row#" + i + " mismatches").isEqualTo(expectedValue); } } } diff --git a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java index 88e16c18cbb9..63e5a5fc6755 100644 --- a/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java +++ b/arrow/src/test/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtilTest.java @@ -19,9 +19,9 @@ package org.apache.iceberg.arrow.vectorized.parquet; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigInteger; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class DecimalVectorUtilTest { @@ -68,7 +68,7 @@ public void testPadBigEndianBytesZero() { @Test public void testPadBigEndianBytesOverflow() { byte[] bytes = new byte[17]; - Assertions.assertThatThrownBy(() -> DecimalVectorUtil.padBigEndianBytes(bytes, 16)) + assertThatThrownBy(() -> DecimalVectorUtil.padBigEndianBytes(bytes, 16)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Buffer size of 17 is larger than requested size of 16"); } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java index 9af1e3dc0331..609dbf1a6acc 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.aws.lakeformation; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatNoException; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -30,7 +31,6 @@ import org.apache.iceberg.aws.glue.GlueCatalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -138,7 +138,7 @@ public void testLakeFormationEnabledGlueCatalog() throws Exception { .atMost(Duration.ofSeconds(10)) .untilAsserted( () -> - Assertions.assertThat( + assertThat( iam.getRolePolicy( GetRolePolicyRequest.builder() .roleName(roleName) 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 fd3bea68088b..c318538d9509 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/AwsClientPropertiesTest.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -45,7 +46,7 @@ public void testApplyClientRegion() { awsClientProperties.applyClientRegionConfiguration(mockS3ClientBuilder); Mockito.verify(mockS3ClientBuilder).region(regionArgumentCaptor.capture()); Region region = regionArgumentCaptor.getValue(); - Assertions.assertThat(region.id()) + assertThat(region.id()) .as("region parameter should match what is set in CLIENT_REGION") .isEqualTo("us-east-1"); } @@ -56,7 +57,7 @@ public void testDefaultCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider = awsClientProperties.credentialsProvider(null, null, null); - Assertions.assertThat(credentialsProvider) + assertThat(credentialsProvider) .as("Should use default credentials if nothing is set") .isInstanceOf(DefaultCredentialsProvider.class); } @@ -69,7 +70,7 @@ public void testCreatesNewInstanceOfDefaultCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider2 = awsClientProperties.credentialsProvider(null, null, null); - Assertions.assertThat(credentialsProvider) + assertThat(credentialsProvider) .as("Should create a new instance in each call") .isNotSameAs(credentialsProvider2); } @@ -81,14 +82,14 @@ public void testBasicCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider = awsClientProperties.credentialsProvider("key", "secret", null); - Assertions.assertThat(credentialsProvider.resolveCredentials()) + assertThat(credentialsProvider.resolveCredentials()) .as("Should use basic credentials if access key ID and secret access key are set") .isInstanceOf(AwsBasicCredentials.class); - Assertions.assertThat(credentialsProvider.resolveCredentials().accessKeyId()) + assertThat(credentialsProvider.resolveCredentials().accessKeyId()) .as("The access key id should be the same as the one set by tag ACCESS_KEY_ID") .isEqualTo("key"); - Assertions.assertThat(credentialsProvider.resolveCredentials().secretAccessKey()) + assertThat(credentialsProvider.resolveCredentials().secretAccessKey()) .as("The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") .isEqualTo("secret"); } @@ -100,13 +101,13 @@ public void testSessionCredentialsConfiguration() { AwsCredentialsProvider credentialsProvider = awsClientProperties.credentialsProvider("key", "secret", "token"); - Assertions.assertThat(credentialsProvider.resolveCredentials()) + assertThat(credentialsProvider.resolveCredentials()) .as("Should use session credentials if session token is set") .isInstanceOf(AwsSessionCredentials.class); - Assertions.assertThat(credentialsProvider.resolveCredentials().accessKeyId()) + assertThat(credentialsProvider.resolveCredentials().accessKeyId()) .as("The access key id should be the same as the one set by tag ACCESS_KEY_ID") .isEqualTo("key"); - Assertions.assertThat(credentialsProvider.resolveCredentials().secretAccessKey()) + assertThat(credentialsProvider.resolveCredentials().secretAccessKey()) .as("The secret access key should be the same as the one set by tag SECRET_ACCESS_KEY") .isEqualTo("secret"); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java b/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java index 7c6e5c19256d..5db09d3a6be0 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java +++ b/aws/src/test/java/org/apache/iceberg/aws/HttpClientPropertiesTest.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -45,7 +47,7 @@ public void testUrlHttpClientConfiguration() { Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assertions.assertThat(capturedHttpClientBuilder) + assertThat(capturedHttpClientBuilder) .as("Should use url connection http client") .isInstanceOf(UrlConnectionHttpClient.Builder.class); } @@ -62,7 +64,7 @@ public void testApacheHttpClientConfiguration() { httpClientProperties.applyHttpClientConfigurations(mockS3ClientBuilder); Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assertions.assertThat(capturedHttpClientBuilder) + assertThat(capturedHttpClientBuilder) .as("Should use apache http client") .isInstanceOf(ApacheHttpClient.Builder.class); } @@ -74,8 +76,7 @@ public void testInvalidHttpClientType() { HttpClientProperties httpClientProperties = new HttpClientProperties(properties); S3ClientBuilder s3ClientBuilder = S3Client.builder(); - Assertions.assertThatThrownBy( - () -> httpClientProperties.applyHttpClientConfigurations(s3ClientBuilder)) + assertThatThrownBy(() -> httpClientProperties.applyHttpClientConfigurations(s3ClientBuilder)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Unrecognized HTTP client type test"); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java index 9598860d6fdf..a22eb1549d34 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsClientFactories.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.TestHelpers; @@ -27,7 +30,6 @@ 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.SerializationUtil; -import org.assertj.core.api.Assertions; import org.assertj.core.api.ThrowableAssert; import org.junit.jupiter.api.Test; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; @@ -44,11 +46,11 @@ public class TestAwsClientFactories { @Test public void testLoadDefault() { - Assertions.assertThat(AwsClientFactories.defaultFactory()) + assertThat(AwsClientFactories.defaultFactory()) .as("default client should be singleton") .isSameAs(AwsClientFactories.defaultFactory()); - Assertions.assertThat(AwsClientFactories.from(Maps.newHashMap())) + assertThat(AwsClientFactories.from(Maps.newHashMap())) .as("should load default when not configured") .isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); } @@ -57,7 +59,7 @@ public void testLoadDefault() { public void testLoadCustom() { Map properties = Maps.newHashMap(); properties.put(AwsProperties.CLIENT_FACTORY, CustomFactory.class.getName()); - Assertions.assertThat(AwsClientFactories.from(properties)) + assertThat(AwsClientFactories.from(properties)) .as("should load custom class") .isInstanceOf(CustomFactory.class); } @@ -67,14 +69,14 @@ public void testS3FileIoCredentialsVerification() { Map properties = Maps.newHashMap(); properties.put(S3FileIOProperties.ACCESS_KEY_ID, "key"); - Assertions.assertThatThrownBy(() -> AwsClientFactories.from(properties)) + assertThatThrownBy(() -> AwsClientFactories.from(properties)) .isInstanceOf(ValidationException.class) .hasMessage("S3 client access key ID and secret access key must be set at the same time"); properties.remove(S3FileIOProperties.ACCESS_KEY_ID); properties.put(S3FileIOProperties.SECRET_ACCESS_KEY, "secret"); - Assertions.assertThatThrownBy(() -> AwsClientFactories.from(properties)) + assertThatThrownBy(() -> AwsClientFactories.from(properties)) .isInstanceOf(ValidationException.class) .hasMessage("S3 client access key ID and secret access key must be set at the same time"); } @@ -85,13 +87,12 @@ public void testDefaultAwsClientFactorySerializable() throws IOException { AwsClientFactory defaultAwsClientFactory = AwsClientFactories.from(properties); AwsClientFactory roundTripResult = TestHelpers.KryoHelpers.roundTripSerialize(defaultAwsClientFactory); - Assertions.assertThat(roundTripResult) - .isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); + assertThat(roundTripResult).isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); byte[] serializedFactoryBytes = SerializationUtil.serializeToBytes(defaultAwsClientFactory); AwsClientFactory deserializedClientFactory = SerializationUtil.deserializeFromBytes(serializedFactoryBytes); - Assertions.assertThat(deserializedClientFactory) + assertThat(deserializedClientFactory) .isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); } @@ -104,12 +105,12 @@ public void testAssumeRoleAwsClientFactorySerializable() throws IOException { AwsClientFactory assumeRoleAwsClientFactory = AwsClientFactories.from(properties); AwsClientFactory roundTripResult = TestHelpers.KryoHelpers.roundTripSerialize(assumeRoleAwsClientFactory); - Assertions.assertThat(roundTripResult).isInstanceOf(AssumeRoleAwsClientFactory.class); + assertThat(roundTripResult).isInstanceOf(AssumeRoleAwsClientFactory.class); byte[] serializedFactoryBytes = SerializationUtil.serializeToBytes(assumeRoleAwsClientFactory); AwsClientFactory deserializedClientFactory = SerializationUtil.deserializeFromBytes(serializedFactoryBytes); - Assertions.assertThat(deserializedClientFactory).isInstanceOf(AssumeRoleAwsClientFactory.class); + assertThat(deserializedClientFactory).isInstanceOf(AssumeRoleAwsClientFactory.class); } @Test @@ -125,14 +126,13 @@ public void testLakeFormationAwsClientFactorySerializable() throws IOException { AwsClientFactory lakeFormationAwsClientFactory = AwsClientFactories.from(properties); AwsClientFactory roundTripResult = TestHelpers.KryoHelpers.roundTripSerialize(lakeFormationAwsClientFactory); - Assertions.assertThat(roundTripResult).isInstanceOf(LakeFormationAwsClientFactory.class); + assertThat(roundTripResult).isInstanceOf(LakeFormationAwsClientFactory.class); byte[] serializedFactoryBytes = SerializationUtil.serializeToBytes(lakeFormationAwsClientFactory); AwsClientFactory deserializedClientFactory = SerializationUtil.deserializeFromBytes(serializedFactoryBytes); - Assertions.assertThat(deserializedClientFactory) - .isInstanceOf(LakeFormationAwsClientFactory.class); + assertThat(deserializedClientFactory).isInstanceOf(LakeFormationAwsClientFactory.class); } @Test @@ -143,7 +143,7 @@ public void testWithDummyValidCredentialsProvider() { assertClientObjectsNotNull(defaultAwsClientFactory); // Ensuring S3Exception thrown instead exception thrown by resolveCredentials() implemented by // test credentials provider - Assertions.assertThatThrownBy(() -> defaultAwsClientFactory.s3().listBuckets()) + assertThatThrownBy(() -> defaultAwsClientFactory.s3().listBuckets()) .isInstanceOf(software.amazon.awssdk.services.s3.model.S3Exception.class) .hasMessageContaining("The AWS Access Key Id you provided does not exist in our records"); } @@ -207,22 +207,21 @@ public void assertAllClientObjectsThrownBy( } private void assertClientObjectsNotNull(AwsClientFactory defaultAwsClientFactory) { - Assertions.assertThat(defaultAwsClientFactory.s3()).isNotNull(); - Assertions.assertThat(defaultAwsClientFactory.dynamo()).isNotNull(); - Assertions.assertThat(defaultAwsClientFactory.glue()).isNotNull(); - Assertions.assertThat(defaultAwsClientFactory.kms()).isNotNull(); + assertThat(defaultAwsClientFactory.s3()).isNotNull(); + assertThat(defaultAwsClientFactory.dynamo()).isNotNull(); + assertThat(defaultAwsClientFactory.glue()).isNotNull(); + assertThat(defaultAwsClientFactory.kms()).isNotNull(); } private void assertIllegalArgumentException( ThrowableAssert.ThrowingCallable shouldRaiseThrowable, String containsMessage) { - Assertions.assertThatThrownBy(shouldRaiseThrowable) + assertThatThrownBy(shouldRaiseThrowable) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining(containsMessage); } private void assertDefaultAwsClientFactory(AwsClientFactory awsClientFactory) { - Assertions.assertThat(awsClientFactory) - .isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); + assertThat(awsClientFactory).isInstanceOf(AwsClientFactories.DefaultAwsClientFactory.class); } private AwsClientFactory getAwsClientFactoryByCredentialsProvider(String providerClass) { diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java index 510c84215eb9..8a0dabcaffa8 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestAwsProperties.java @@ -20,11 +20,11 @@ import static org.apache.iceberg.aws.AwsProperties.DYNAMODB_TABLE_NAME; import static org.apache.iceberg.aws.AwsProperties.GLUE_CATALOG_ID; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestAwsProperties { @@ -35,9 +35,9 @@ public void testKryoSerialization() throws IOException { new AwsProperties(ImmutableMap.of(GLUE_CATALOG_ID, "foo", DYNAMODB_TABLE_NAME, "ice")); AwsProperties deSerializedAwsPropertiesWithProps = TestHelpers.KryoHelpers.roundTripSerialize(awsPropertiesWithProps); - Assertions.assertThat(deSerializedAwsPropertiesWithProps.glueCatalogId()) + assertThat(deSerializedAwsPropertiesWithProps.glueCatalogId()) .isEqualTo(awsPropertiesWithProps.glueCatalogId()); - Assertions.assertThat(deSerializedAwsPropertiesWithProps.dynamoDbTableName()) + assertThat(deSerializedAwsPropertiesWithProps.dynamoDbTableName()) .isEqualTo(awsPropertiesWithProps.dynamoDbTableName()); } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java index df338a5d2aea..b0602a074992 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestHttpClientProperties.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -45,7 +47,7 @@ public void testUrlHttpClientConfiguration() { Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assertions.assertThat(capturedHttpClientBuilder) + assertThat(capturedHttpClientBuilder) .as("Should use url connection http client") .isInstanceOf(UrlConnectionHttpClient.Builder.class); } @@ -62,7 +64,7 @@ public void testApacheHttpClientConfiguration() { httpProperties.applyHttpClientConfigurations(mockS3ClientBuilder); Mockito.verify(mockS3ClientBuilder).httpClientBuilder(httpClientBuilderCaptor.capture()); SdkHttpClient.Builder capturedHttpClientBuilder = httpClientBuilderCaptor.getValue(); - Assertions.assertThat(capturedHttpClientBuilder) + assertThat(capturedHttpClientBuilder) .as("Should use apache http client") .isInstanceOf(ApacheHttpClient.Builder.class); } @@ -74,8 +76,7 @@ public void testInvalidHttpClientType() { HttpClientProperties httpProperties = new HttpClientProperties(properties); S3ClientBuilder s3ClientBuilder = S3Client.builder(); - Assertions.assertThatThrownBy( - () -> httpProperties.applyHttpClientConfigurations(s3ClientBuilder)) + assertThatThrownBy(() -> httpProperties.applyHttpClientConfigurations(s3ClientBuilder)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Unrecognized HTTP client type test"); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java b/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java index b35363502939..88623edd9334 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestRESTSigV4Signer.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.Map; import org.apache.hc.core5.http.HttpHeaders; @@ -28,7 +30,6 @@ import org.apache.iceberg.rest.auth.OAuth2Util; import org.apache.iceberg.rest.responses.ConfigResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -101,7 +102,7 @@ public void signRequestWithoutBody() { client.get("v1/config", ConfigResponse.class, ImmutableMap.of(), e -> {}); mockServer.verify(request, VerificationTimes.exactly(1)); - Assertions.assertThat(response).isNotNull(); + assertThat(response).isNotNull(); } @Test @@ -141,6 +142,6 @@ public void signRequestWithBody() { "v1/oauth/token", formData, OAuthTokenResponse.class, ImmutableMap.of(), e -> {}); mockServer.verify(request, VerificationTimes.exactly(1)); - Assertions.assertThat(response).isNotNull(); + assertThat(response).isNotNull(); } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java index f09d4e543f17..cd1cd43dd0ba 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOAwsClientFactories.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.iceberg.aws.s3.S3FileIOAwsClientFactory; import org.apache.iceberg.aws.s3.S3FileIOProperties; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestS3FileIOAwsClientFactories { @@ -34,7 +35,7 @@ public void testS3FileIOImplCatalogPropertyDefined() { S3FileIOProperties.CLIENT_FACTORY, "org.apache.iceberg.aws.s3.DefaultS3FileIOAwsClientFactory"); Object factoryImpl = S3FileIOAwsClientFactories.initialize(properties); - Assertions.assertThat(factoryImpl) + assertThat(factoryImpl) .as( "should instantiate an object of type S3FileIOAwsClientFactory when s3.client-factory-impl is set") .isInstanceOf(S3FileIOAwsClientFactory.class); @@ -45,7 +46,7 @@ public void testS3FileIOImplCatalogPropertyNotDefined() { // don't set anything Map properties = Maps.newHashMap(); Object factoryImpl = S3FileIOAwsClientFactories.initialize(properties); - Assertions.assertThat(factoryImpl) + assertThat(factoryImpl) .as( "should instantiate an object of type AwsClientFactory when s3.client-factory-impl is not set") .isInstanceOf(AwsClientFactory.class); 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 b7a3f6048991..92ed3648bcfc 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import java.util.Optional; import org.apache.iceberg.CatalogProperties; @@ -25,7 +28,6 @@ import org.apache.iceberg.aws.s3.signer.S3V4RestSignerClient; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -46,7 +48,7 @@ public void testS3FileIoSseCustom_mustHaveCustomKey() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.SSE_TYPE, S3FileIOProperties.SSE_TYPE_CUSTOM); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot initialize SSE-C S3FileIO with null encryption key"); } @@ -57,7 +59,7 @@ public void testS3FileIoSseCustom_mustHaveCustomMd5() { map.put(S3FileIOProperties.SSE_TYPE, S3FileIOProperties.SSE_TYPE_CUSTOM); map.put(S3FileIOProperties.SSE_KEY, "something"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot initialize SSE-C S3FileIO with null encryption key MD5"); } @@ -67,7 +69,7 @@ public void testS3FileIoAcl() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.ACL, ObjectCannedACL.AUTHENTICATED_READ.toString()); S3FileIOProperties properties = new S3FileIOProperties(map); - Assertions.assertThat(properties.acl()).isEqualTo(ObjectCannedACL.AUTHENTICATED_READ); + assertThat(properties.acl()).isEqualTo(ObjectCannedACL.AUTHENTICATED_READ); } @Test @@ -75,7 +77,7 @@ public void testS3FileIoAcl_unknownType() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.ACL, "bad-input"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot support S3 CannedACL bad-input"); } @@ -85,7 +87,7 @@ public void testS3MultipartSizeTooSmall() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.MULTIPART_SIZE, "1"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Minimum multipart upload object size must be larger than 5 MB."); } @@ -95,7 +97,7 @@ public void testS3MultipartSizeTooLarge() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.MULTIPART_SIZE, "5368709120"); // 5GB - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Input malformed or exceeded maximum multipart upload size 5GB: 5368709120"); } @@ -105,7 +107,7 @@ public void testS3MultipartThresholdFactorLessThanOne() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.MULTIPART_THRESHOLD_FACTOR, "0.9"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Multipart threshold factor must be >= to 1.0"); } @@ -115,7 +117,7 @@ public void testS3FileIoDeleteBatchSizeTooLarge() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.DELETE_BATCH_SIZE, "2000"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Deletion batch size must be between 1 and 1000"); } @@ -125,7 +127,7 @@ public void testS3FileIoDeleteBatchSizeTooSmall() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.DELETE_BATCH_SIZE, "0"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Deletion batch size must be between 1 and 1000"); } @@ -144,7 +146,7 @@ public void testS3FileIoDefaultCredentialsConfiguration() { Mockito.verify(mockS3ClientBuilder).credentialsProvider(awsCredentialsProviderCaptor.capture()); AwsCredentialsProvider capturedAwsCredentialsProvider = awsCredentialsProviderCaptor.getValue(); - Assertions.assertThat(capturedAwsCredentialsProvider) + assertThat(capturedAwsCredentialsProvider) .as("Should use default credentials if nothing is set") .isInstanceOf(DefaultCredentialsProvider.class); } @@ -165,13 +167,13 @@ public void testS3FileIoBasicCredentialsConfiguration() { Mockito.verify(mockS3ClientBuilder).credentialsProvider(awsCredentialsProviderCaptor.capture()); AwsCredentialsProvider capturedAwsCredentialsProvider = awsCredentialsProviderCaptor.getValue(); - Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials()) + assertThat(capturedAwsCredentialsProvider.resolveCredentials()) .as("Should use basic credentials if access key ID and secret access key are set") .isInstanceOf(AwsBasicCredentials.class); - Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()) + assertThat(capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()) .as("The access key id should be the same as the one set by tag S3FILEIO_ACCESS_KEY_ID") .isEqualTo("key"); - Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()) + assertThat(capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()) .as( "The secret access key should be the same as the one set by tag S3FILEIO_SECRET_ACCESS_KEY") .isEqualTo("secret"); @@ -194,13 +196,13 @@ public void testS3FileIoSessionCredentialsConfiguration() { Mockito.verify(mockS3ClientBuilder).credentialsProvider(awsCredentialsProviderCaptor.capture()); AwsCredentialsProvider capturedAwsCredentialsProvider = awsCredentialsProviderCaptor.getValue(); - Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials()) + assertThat(capturedAwsCredentialsProvider.resolveCredentials()) .as("Should use session credentials if session token is set") .isInstanceOf(AwsSessionCredentials.class); - Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()) + assertThat(capturedAwsCredentialsProvider.resolveCredentials().accessKeyId()) .as("The access key id should be the same as the one set by tag S3FILEIO_ACCESS_KEY_ID") .isEqualTo("key"); - Assertions.assertThat(capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()) + assertThat(capturedAwsCredentialsProvider.resolveCredentials().secretAccessKey()) .as( "The secret access key should be the same as the one set by tag S3FILEIO_SECRET_ACCESS_KEY") .isEqualTo("secret"); @@ -212,7 +214,7 @@ public void testS3RemoteSignerWithoutUri() { ImmutableMap.of(S3FileIOProperties.REMOTE_SIGNING_ENABLED, "true"); S3FileIOProperties s3Properties = new S3FileIOProperties(properties); - Assertions.assertThatThrownBy(() -> s3Properties.applySignerConfiguration(S3Client.builder())) + assertThatThrownBy(() -> s3Properties.applySignerConfiguration(S3Client.builder())) .isInstanceOf(IllegalArgumentException.class) .hasMessage("S3 signer service URI is required"); } @@ -230,10 +232,10 @@ public void testS3RemoteSigningEnabled() { Optional signer = builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.SIGNER); - Assertions.assertThat(signer).isPresent().get().isInstanceOf(S3V4RestSignerClient.class); + assertThat(signer).isPresent().get().isInstanceOf(S3V4RestSignerClient.class); S3V4RestSignerClient signerClient = (S3V4RestSignerClient) signer.get(); - Assertions.assertThat(signerClient.baseSignerUri()).isEqualTo(uri); - Assertions.assertThat(signerClient.properties()).isEqualTo(properties); + assertThat(signerClient.baseSignerUri()).isEqualTo(uri); + assertThat(signerClient.properties()).isEqualTo(properties); } @Test @@ -250,17 +252,14 @@ public void s3RemoteSigningEnabledWithUserAgent() { Optional userAgent = builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX); - Assertions.assertThat(userAgent) - .isPresent() - .get() - .satisfies(x -> Assertions.assertThat(x).startsWith("s3fileio")); + assertThat(userAgent).isPresent().get().satisfies(x -> assertThat(x).startsWith("s3fileio")); Optional signer = builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.SIGNER); - Assertions.assertThat(signer).isPresent().get().isInstanceOf(S3V4RestSignerClient.class); + assertThat(signer).isPresent().get().isInstanceOf(S3V4RestSignerClient.class); S3V4RestSignerClient signerClient = (S3V4RestSignerClient) signer.get(); - Assertions.assertThat(signerClient.baseSignerUri()).isEqualTo(uri); - Assertions.assertThat(signerClient.properties()).isEqualTo(properties); + assertThat(signerClient.baseSignerUri()).isEqualTo(uri); + assertThat(signerClient.properties()).isEqualTo(properties); } @Test @@ -274,7 +273,7 @@ public void testS3RemoteSigningDisabled() { Optional signer = builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.SIGNER); - Assertions.assertThat(signer).isNotPresent(); + assertThat(signer).isNotPresent(); } @Test @@ -286,7 +285,7 @@ public void testS3AccessGrantsEnabled() { S3ClientBuilder builder = S3Client.builder(); s3Properties.applyS3AccessGrantsConfigurations(builder); - Assertions.assertThat(builder.plugins().size()).isEqualTo(1); + assertThat(builder.plugins().size()).isEqualTo(1); } @Test @@ -298,7 +297,7 @@ public void testS3AccessGrantsDisabled() { S3ClientBuilder builder = S3Client.builder(); s3Properties.applyS3AccessGrantsConfigurations(builder); - Assertions.assertThat(builder.plugins().size()).isEqualTo(0); + assertThat(builder.plugins().size()).isEqualTo(0); // Implicitly false properties = ImmutableMap.of(); @@ -306,6 +305,6 @@ public void testS3AccessGrantsDisabled() { builder = S3Client.builder(); s3Properties.applyS3AccessGrantsConfigurations(builder); - Assertions.assertThat(builder.plugins().size()).isEqualTo(0); + assertThat(builder.plugins().size()).isEqualTo(0); } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java b/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java index 5b5941c9a6e6..b602cea303d8 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java +++ b/aws/src/test/java/org/apache/iceberg/aws/dynamodb/TestDynamoDbCatalog.java @@ -19,6 +19,8 @@ package org.apache.iceberg.aws.dynamodb; import static org.apache.iceberg.aws.dynamodb.DynamoDbCatalog.toPropertyCol; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.ArgumentMatchers.any; import org.apache.iceberg.aws.AwsProperties; @@ -26,7 +28,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -60,7 +61,7 @@ public void testConstructorWarehousePathWithEndSlash() { .when(dynamo) .getItem(any(GetItemRequest.class)); String location = catalogWithSlash.defaultWarehouseLocation(TABLE_IDENTIFIER); - Assertions.assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); + assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); } @Test @@ -71,7 +72,7 @@ public void testDefaultWarehouseLocationNoDbUri() { String warehousePath = WAREHOUSE_PATH + "/db.db/table"; String defaultWarehouseLocation = dynamoCatalog.defaultWarehouseLocation(TABLE_IDENTIFIER); - Assertions.assertThat(defaultWarehouseLocation).isEqualTo(warehousePath); + assertThat(defaultWarehouseLocation).isEqualTo(warehousePath); } @Test @@ -88,7 +89,7 @@ public void testDefaultWarehouseLocationDbUri() { .getItem(any(GetItemRequest.class)); String defaultWarehouseLocation = dynamoCatalog.defaultWarehouseLocation(TABLE_IDENTIFIER); - Assertions.assertThat(defaultWarehouseLocation).isEqualTo("s3://bucket2/db/table"); + assertThat(defaultWarehouseLocation).isEqualTo("s3://bucket2/db/table"); } @Test @@ -97,7 +98,7 @@ public void testDefaultWarehouseLocationNoNamespace() { .when(dynamo) .getItem(any(GetItemRequest.class)); - Assertions.assertThatThrownBy(() -> dynamoCatalog.defaultWarehouseLocation(TABLE_IDENTIFIER)) + assertThatThrownBy(() -> dynamoCatalog.defaultWarehouseLocation(TABLE_IDENTIFIER)) .as("default warehouse can't be called on non existent namespace") .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Cannot find default warehouse location:"); diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java index 6d3b52154547..2042948eb3c9 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueCatalog.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.glue; +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 java.util.UUID; @@ -35,7 +38,6 @@ 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.LockManagers; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.Mockito; @@ -104,8 +106,7 @@ public void testConstructorEmptyWarehousePath() { Mockito.doThrow(EntityNotFoundException.builder().build()) .when(glue) .getTable(Mockito.any(GetTableRequest.class)); - Assertions.assertThatThrownBy( - () -> catalog.createTable(TableIdentifier.of("db", "table"), new Schema())) + assertThatThrownBy(() -> catalog.createTable(TableIdentifier.of("db", "table"), new Schema())) .hasMessageContaining( "Cannot derive default warehouse location, warehouse path must not be null or empty") .isInstanceOf(ValidationException.class); @@ -127,7 +128,7 @@ public void testConstructorWarehousePathWithEndSlash() { .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = catalogWithSlash.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assertions.assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); + assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); } @Test @@ -137,7 +138,7 @@ public void testDefaultWarehouseLocationNoDbUri() { .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = glueCatalog.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assertions.assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); + assertThat(location).isEqualTo(WAREHOUSE_PATH + "/db.db/table"); } @Test @@ -149,7 +150,7 @@ public void testDefaultWarehouseLocationDbUri() { .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = glueCatalog.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assertions.assertThat(location).isEqualTo("s3://bucket2/db/table"); + assertThat(location).isEqualTo("s3://bucket2/db/table"); } @Test @@ -162,7 +163,7 @@ public void testDefaultWarehouseLocationDbUriTrailingSlash() { .getDatabase(Mockito.any(GetDatabaseRequest.class)); String location = glueCatalog.defaultWarehouseLocation(TableIdentifier.of("db", "table")); - Assertions.assertThat(location).isEqualTo("s3://bucket2/db/table"); + assertThat(location).isEqualTo("s3://bucket2/db/table"); } @Test @@ -239,7 +240,7 @@ public void testListTables() { .build()) .when(glue) .getTables(Mockito.any(GetTablesRequest.class)); - Assertions.assertThat(glueCatalog.listTables(Namespace.of("db1"))) + assertThat(glueCatalog.listTables(Namespace.of("db1"))) .isEqualTo( Lists.newArrayList(TableIdentifier.of("db1", "t1"), TableIdentifier.of("db1", "t2"))); } @@ -285,7 +286,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { }) .when(glue) .getTables(Mockito.any(GetTablesRequest.class)); - Assertions.assertThat(glueCatalog.listTables(Namespace.of("db1"))).hasSize(10); + assertThat(glueCatalog.listTables(Namespace.of("db1"))).hasSize(10); } @Test @@ -343,7 +344,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { .when(glue) .deleteTable(Mockito.any(DeleteTableRequest.class)); glueCatalog.dropTable(TableIdentifier.of("db1", "t1")); - Assertions.assertThat(counter.get()).isEqualTo(0); + assertThat(counter.get()).isEqualTo(0); } @Test @@ -397,7 +398,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { .createTable(Mockito.any(CreateTableRequest.class)); glueCatalog.renameTable(TableIdentifier.of("db", "t"), TableIdentifier.of("db", "x_renamed")); - Assertions.assertThat(counter.get()).isEqualTo(0); + assertThat(counter.get()).isEqualTo(0); } @Test @@ -417,7 +418,7 @@ public void testCreateNamespaceBadName() { Lists.newArrayList(Namespace.of("db-1"), Namespace.of("db", "db2")); for (Namespace namespace : invalidNamespaces) { - Assertions.assertThatThrownBy(() -> glueCatalog.createNamespace(namespace)) + assertThatThrownBy(() -> glueCatalog.createNamespace(namespace)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot convert namespace") .hasMessageEndingWith( @@ -435,7 +436,7 @@ public void testListAllNamespaces() { .build()) .when(glue) .getDatabases(Mockito.any(GetDatabasesRequest.class)); - Assertions.assertThat(glueCatalog.listNamespaces()) + assertThat(glueCatalog.listNamespaces()) .isEqualTo(Lists.newArrayList(Namespace.of("db1"), Namespace.of("db2"))); } @@ -463,7 +464,7 @@ public Object answer(InvocationOnMock invocation) throws Throwable { }) .when(glue) .getDatabases(Mockito.any(GetDatabasesRequest.class)); - Assertions.assertThat(glueCatalog.listNamespaces()).hasSize(10); + assertThat(glueCatalog.listNamespaces()).hasSize(10); } @Test @@ -472,7 +473,7 @@ public void testListNamespacesWithNameShouldReturnItself() { GetDatabaseResponse.builder().database(Database.builder().name("db1").build()).build()) .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); - Assertions.assertThat(glueCatalog.listNamespaces(Namespace.of("db1"))) + assertThat(glueCatalog.listNamespaces(Namespace.of("db1"))) .as("list self should return empty list") .isEmpty(); } @@ -480,7 +481,7 @@ public void testListNamespacesWithNameShouldReturnItself() { @Test public void testListNamespacesBadName() { - Assertions.assertThatThrownBy(() -> glueCatalog.listNamespaces(Namespace.of("db-1"))) + assertThatThrownBy(() -> glueCatalog.listNamespaces(Namespace.of("db-1"))) .isInstanceOf(ValidationException.class) .hasMessage( "Cannot convert namespace db-1 to Glue database name, " @@ -503,8 +504,7 @@ public void testLoadNamespaceMetadata() { .build()) .when(glue) .getDatabase(Mockito.any(GetDatabaseRequest.class)); - Assertions.assertThat(glueCatalog.loadNamespaceMetadata(Namespace.of("db1"))) - .isEqualTo(parameters); + assertThat(glueCatalog.loadNamespaceMetadata(Namespace.of("db1"))).isEqualTo(parameters); } @Test @@ -546,7 +546,7 @@ public void testDropNamespaceThatContainsOnlyIcebergTable() { .when(glue) .deleteDatabase(Mockito.any(DeleteDatabaseRequest.class)); - Assertions.assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of("db1"))) + assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of("db1"))) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Cannot drop namespace db1 because it still contains Iceberg tables"); } @@ -567,7 +567,7 @@ public void testDropNamespaceThatContainsNonIcebergTable() { .when(glue) .deleteDatabase(Mockito.any(DeleteDatabaseRequest.class)); - Assertions.assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of("db1"))) + assertThatThrownBy(() -> glueCatalog.dropNamespace(Namespace.of("db1"))) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Cannot drop namespace db1 because it still contains non-Iceberg tables"); } @@ -622,7 +622,7 @@ public void testTablePropsDefinedAtCatalogLevel() { LockManagers.defaultLockManager(), catalogProps); Map properties = glueCatalog.properties(); - Assertions.assertThat(properties) + assertThat(properties) .isNotEmpty() .containsEntry("table-default.key1", "catalog-default-key1") .containsEntry("table-default.key2", "catalog-default-key2") @@ -644,8 +644,7 @@ public void testValidateIdentifierSkipNameValidation() { glue, LockManagers.defaultLockManager(), ImmutableMap.of()); - Assertions.assertThat(glueCatalog.isValidIdentifier(TableIdentifier.parse("db-1.a-1"))) - .isEqualTo(true); + assertThat(glueCatalog.isValidIdentifier(TableIdentifier.parse("db-1.a-1"))).isEqualTo(true); } @Test @@ -671,7 +670,7 @@ public void testTableLevelS3TagProperties() { glueCatalog.newTableOps(TableIdentifier.of(Namespace.of("db"), "table")); Map tableCatalogProperties = glueTableOperations.tableCatalogProperties(); - Assertions.assertThat(tableCatalogProperties) + assertThat(tableCatalogProperties) .containsEntry( S3FileIOProperties.WRITE_TAGS_PREFIX.concat(S3FileIOProperties.S3_TAG_ICEBERG_TABLE), "table") diff --git a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java index 0d2c3d825c42..71e2f721f6d2 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java +++ b/aws/src/test/java/org/apache/iceberg/aws/glue/TestGlueToIcebergConverter.java @@ -18,13 +18,15 @@ */ package org.apache.iceberg.aws.glue; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.util.Map; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.Database; import software.amazon.awssdk.services.glue.model.Table; @@ -35,21 +37,21 @@ public class TestGlueToIcebergConverter { public void testToNamespace() { Database database = Database.builder().name("db").build(); Namespace namespace = Namespace.of("db"); - Assertions.assertThat(GlueToIcebergConverter.toNamespace(database)).isEqualTo(namespace); + assertThat(GlueToIcebergConverter.toNamespace(database)).isEqualTo(namespace); } @Test public void testToTableId() { Table table = Table.builder().databaseName("db").name("name").build(); TableIdentifier icebergId = TableIdentifier.of("db", "name"); - Assertions.assertThat(GlueToIcebergConverter.toTableId(table)).isEqualTo(icebergId); + assertThat(GlueToIcebergConverter.toTableId(table)).isEqualTo(icebergId); } @Test public void testValidateTableIcebergPropertyNotFound() { Table table = Table.builder().parameters(ImmutableMap.of()).build(); - Assertions.assertThatThrownBy(() -> GlueTableOperations.checkIfTableIsIceberg(table, "name")) + assertThatThrownBy(() -> GlueTableOperations.checkIfTableIsIceberg(table, "name")) .isInstanceOf(NoSuchIcebergTableException.class) .hasMessage("Input Glue table is not an iceberg table: name (type=null)"); } @@ -60,7 +62,7 @@ public void testValidateTableIcebergPropertyValueWrong() { ImmutableMap.of(BaseMetastoreTableOperations.TABLE_TYPE_PROP, "other"); Table table = Table.builder().parameters(properties).build(); - Assertions.assertThatThrownBy(() -> GlueTableOperations.checkIfTableIsIceberg(table, "name")) + assertThatThrownBy(() -> GlueTableOperations.checkIfTableIsIceberg(table, "name")) .isInstanceOf(NoSuchIcebergTableException.class) .hasMessage("Input Glue table is not an iceberg table: name (type=other)"); } 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 599e324ebda7..7c646f7bf708 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 @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.glue; +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.PartitionSpec; @@ -32,7 +35,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.DatabaseInput; @@ -49,8 +51,7 @@ public class TestIcebergToGlueConverter { @Test public void testToDatabaseName() { - Assertions.assertThat(IcebergToGlueConverter.toDatabaseName(Namespace.of("db"), false)) - .isEqualTo("db"); + assertThat(IcebergToGlueConverter.toDatabaseName(Namespace.of("db"), false)).isEqualTo("db"); } @Test @@ -64,7 +65,7 @@ public void testToDatabaseNameFailure() { Namespace.of(new String(new char[600]).replace("\0", "a"))); for (Namespace name : badNames) { - Assertions.assertThatThrownBy(() -> IcebergToGlueConverter.toDatabaseName(name, false)) + assertThatThrownBy(() -> IcebergToGlueConverter.toDatabaseName(name, false)) .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot convert namespace") .hasMessageEndingWith( @@ -78,8 +79,7 @@ public void testSkipNamespaceValidation() { List acceptableNames = Lists.newArrayList(Namespace.of("db-1"), Namespace.of("db-1-1-1")); for (Namespace name : acceptableNames) { - Assertions.assertThat(IcebergToGlueConverter.toDatabaseName(name, true)) - .isEqualTo(name.toString()); + assertThat(IcebergToGlueConverter.toDatabaseName(name, true)).isEqualTo(name.toString()); } } @@ -91,7 +91,7 @@ public void testSkipTableNameValidation() { TableIdentifier.parse("db.a-1-1"), TableIdentifier.parse("db.a#1")); for (TableIdentifier identifier : acceptableIdentifiers) { - Assertions.assertThat(IcebergToGlueConverter.getTableName(identifier, true)) + assertThat(IcebergToGlueConverter.getTableName(identifier, true)) .isEqualTo(identifier.name()); } } @@ -108,24 +108,21 @@ public void testToDatabaseInput() { "val"); DatabaseInput databaseInput = IcebergToGlueConverter.toDatabaseInput(Namespace.of("ns"), properties, false); - Assertions.assertThat(databaseInput.locationUri()) - .as("Location should be set") - .isEqualTo("s3://location"); - Assertions.assertThat(databaseInput.description()) + assertThat(databaseInput.locationUri()).as("Location should be set").isEqualTo("s3://location"); + assertThat(databaseInput.description()) .as("Description should be set") .isEqualTo("description"); - Assertions.assertThat(databaseInput.parameters()) + assertThat(databaseInput.parameters()) .as("Parameters should be set") .isEqualTo(ImmutableMap.of("key", "val")); - Assertions.assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); + assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); } @Test public void testToDatabaseInputNoParameter() { DatabaseInput input = DatabaseInput.builder().name("db").parameters(ImmutableMap.of()).build(); Namespace namespace = Namespace.of("db"); - Assertions.assertThat( - IcebergToGlueConverter.toDatabaseInput(namespace, ImmutableMap.of(), false)) + assertThat(IcebergToGlueConverter.toDatabaseInput(namespace, ImmutableMap.of(), false)) .isEqualTo(input); } @@ -135,14 +132,14 @@ public void testToDatabaseInputEmptyLocation() { ImmutableMap.of(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, "description", "key", "val"); DatabaseInput databaseInput = IcebergToGlueConverter.toDatabaseInput(Namespace.of("ns"), properties, false); - Assertions.assertThat(databaseInput.locationUri()).as("Location should not be set").isNull(); - Assertions.assertThat(databaseInput.description()) + assertThat(databaseInput.locationUri()).as("Location should not be set").isNull(); + assertThat(databaseInput.description()) .as("Description should be set") .isEqualTo("description"); - Assertions.assertThat(databaseInput.parameters()) + assertThat(databaseInput.parameters()) .as("Parameters should be set") .isEqualTo(ImmutableMap.of("key", "val")); - Assertions.assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); + assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); } @Test @@ -151,14 +148,12 @@ public void testToDatabaseInputEmptyDescription() { ImmutableMap.of(IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, "s3://location", "key", "val"); DatabaseInput databaseInput = IcebergToGlueConverter.toDatabaseInput(Namespace.of("ns"), properties, false); - Assertions.assertThat(databaseInput.locationUri()) - .as("Location should be set") - .isEqualTo("s3://location"); - Assertions.assertThat(databaseInput.description()).as("Description should not be set").isNull(); - Assertions.assertThat(databaseInput.parameters()) + assertThat(databaseInput.locationUri()).as("Location should be set").isEqualTo("s3://location"); + assertThat(databaseInput.description()).as("Description should not be set").isNull(); + assertThat(databaseInput.parameters()) .as("Parameters should be set") .isEqualTo(ImmutableMap.of("key", "val")); - Assertions.assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); + assertThat(databaseInput.name()).as("Database name should be set").isEqualTo("ns"); } @Test @@ -212,13 +207,13 @@ public void testSetTableInputInformation() { .build()) .build(); - Assertions.assertThat(actualTableInput.storageDescriptor().additionalLocations()) + assertThat(actualTableInput.storageDescriptor().additionalLocations()) .as("additionalLocations should match") .isEqualTo(expectedTableInput.storageDescriptor().additionalLocations()); - Assertions.assertThat(actualTableInput.storageDescriptor().location()) + assertThat(actualTableInput.storageDescriptor().location()) .as("Location should match") .isEqualTo(expectedTableInput.storageDescriptor().location()); - Assertions.assertThat(actualTableInput.storageDescriptor().columns()) + assertThat(actualTableInput.storageDescriptor().columns()) .as("Columns should match") .isEqualTo(expectedTableInput.storageDescriptor().columns()); } @@ -278,13 +273,13 @@ public void testSetTableInputInformationWithRemovedColumns() { .build()) .build(); - Assertions.assertThat(actualTableInput.storageDescriptor().additionalLocations()) + assertThat(actualTableInput.storageDescriptor().additionalLocations()) .as("additionalLocations should match") .isEqualTo(expectedTableInput.storageDescriptor().additionalLocations()); - Assertions.assertThat(actualTableInput.storageDescriptor().location()) + assertThat(actualTableInput.storageDescriptor().location()) .as("Location should match") .isEqualTo(expectedTableInput.storageDescriptor().location()); - Assertions.assertThat(actualTableInput.storageDescriptor().columns()) + assertThat(actualTableInput.storageDescriptor().columns()) .as("Columns should match") .isEqualTo(expectedTableInput.storageDescriptor().columns()); } @@ -307,7 +302,7 @@ public void testSetTableDescription() { IcebergToGlueConverter.setTableInputInformation(actualTableInputBuilder, tableMetadata); TableInput actualTableInput = actualTableInputBuilder.build(); - Assertions.assertThat(actualTableInput.description()) + assertThat(actualTableInput.description()) .as("description should match") .isEqualTo(tableDescription); } 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 26c9bc133b13..135eb76772cc 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 @@ -18,6 +18,8 @@ */ 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.AdditionalAnswers.delegatesTo; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.doReturn; @@ -68,7 +70,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializableSupplier; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; @@ -131,25 +132,25 @@ public void testNewInputFile() throws IOException { random.nextBytes(expected); InputFile in = s3FileIO.newInputFile(location); - Assertions.assertThat(in.exists()).isFalse(); + assertThat(in.exists()).isFalse(); OutputFile out = s3FileIO.newOutputFile(location); try (OutputStream os = out.createOrOverwrite()) { IOUtil.writeFully(os, ByteBuffer.wrap(expected)); } - Assertions.assertThat(in.exists()).isTrue(); + assertThat(in.exists()).isTrue(); byte[] actual = new byte[1024 * 1024]; try (InputStream is = in.newStream()) { IOUtil.readFully(is, actual, 0, expected.length); } - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); s3FileIO.deleteFile(in); - Assertions.assertThat(s3FileIO.newInputFile(location).exists()).isFalse(); + assertThat(s3FileIO.newInputFile(location).exists()).isFalse(); } @Test @@ -171,7 +172,7 @@ public void testDeleteFilesSingleBatchWithRemainder() { public void testDeleteEmptyList() throws IOException { String location = "s3://bucket/path/to/file.txt"; InputFile in = s3FileIO.newInputFile(location); - Assertions.assertThat(in.exists()).isFalse(); + assertThat(in.exists()).isFalse(); OutputFile out = s3FileIO.newOutputFile(location); try (OutputStream os = out.createOrOverwrite()) { IOUtil.writeFully(os, ByteBuffer.wrap(new byte[1024 * 1024])); @@ -179,9 +180,9 @@ public void testDeleteEmptyList() throws IOException { s3FileIO.deleteFiles(Lists.newArrayList()); - Assertions.assertThat(s3FileIO.newInputFile(location).exists()).isTrue(); + assertThat(s3FileIO.newInputFile(location).exists()).isTrue(); s3FileIO.deleteFile(in); - Assertions.assertThat(s3FileIO.newInputFile(location).exists()).isFalse(); + assertThat(s3FileIO.newInputFile(location).exists()).isFalse(); } @Test @@ -193,7 +194,7 @@ public void testDeleteFilesS3ReturnsError() { .build(); doReturn(deleteObjectsResponse).when(s3mock).deleteObjects((DeleteObjectsRequest) any()); - Assertions.assertThatThrownBy(() -> s3FileIO.deleteFiles(Lists.newArrayList(location))) + assertThatThrownBy(() -> s3FileIO.deleteFiles(Lists.newArrayList(location))) .isInstanceOf(BulkDeletionFailureException.class) .hasMessage("Failed to delete 1 files"); } @@ -216,7 +217,7 @@ private void testBatchDelete(int numObjects) { int expectedDeleteRequests = expectedNumberOfBatchesPerBucket * numBucketsForBatchDeletion; verify(s3mock, times(expectedDeleteRequests)).deleteObjects((DeleteObjectsRequest) any()); for (String path : paths) { - Assertions.assertThat(s3FileIO.newInputFile(path).exists()).isFalse(); + assertThat(s3FileIO.newInputFile(path).exists()).isFalse(); } } @@ -232,7 +233,7 @@ public void testSerializeClient() throws IOException, ClassNotFoundException { byte[] data = TestHelpers.serialize(pre); SerializableSupplier post = TestHelpers.deserialize(data); - Assertions.assertThat(post.get().serviceName()).isEqualTo("s3"); + assertThat(post.get().serviceName()).isEqualTo("s3"); } @Test @@ -248,13 +249,12 @@ public void testPrefixList() { String scalePrefix = String.format("%s/%s/", prefix, scale); createRandomObjects(scalePrefix, scale); - Assertions.assertThat(Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()) + assertThat(Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()) .isEqualTo((long) scale); }); long totalFiles = scaleSizes.stream().mapToLong(Integer::longValue).sum(); - Assertions.assertThat(Streams.stream(s3FileIO.listPrefix(prefix)).count()) - .isEqualTo(totalFiles); + assertThat(Streams.stream(s3FileIO.listPrefix(prefix)).count()).isEqualTo(totalFiles); } /** @@ -273,8 +273,7 @@ public void testPrefixDelete() { createRandomObjects(scalePrefix, scale); s3FileIO.deletePrefix(scalePrefix); - Assertions.assertThat(Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()) - .isEqualTo(0); + assertThat(Streams.stream(s3FileIO.listPrefix(scalePrefix)).count()).isEqualTo(0); }); } @@ -283,7 +282,7 @@ public void testReadMissingLocation() { String location = "s3://bucket/path/to/data.parquet"; InputFile in = s3FileIO.newInputFile(location); - Assertions.assertThatThrownBy(() -> in.newStream().read()) + assertThatThrownBy(() -> in.newStream().read()) .isInstanceOf(NotFoundException.class) .hasMessage("Location does not exist: " + location); } @@ -313,12 +312,12 @@ public void testMissingTableMetadata() { long start = System.currentTimeMillis(); // to test NotFoundException, load the table again. refreshing the existing table doesn't // require reading metadata - Assertions.assertThatThrownBy(() -> catalog.loadTable(ident)) + assertThatThrownBy(() -> catalog.loadTable(ident)) .isInstanceOf(NotFoundException.class) .hasMessageStartingWith("Location does not exist"); long duration = System.currentTimeMillis() - start; - Assertions.assertThat(duration < 10_000).as("Should take less than 10 seconds").isTrue(); + assertThat(duration < 10_000).as("Should take less than 10 seconds").isTrue(); } } @@ -333,8 +332,8 @@ public void testFileIOJsonSerialization() { String json = FileIOParser.toJson(s3FileIO); try (FileIO deserialized = FileIOParser.fromJson(json, conf)) { - Assertions.assertThat(deserialized).isInstanceOf(S3FileIO.class); - Assertions.assertThat(deserialized.properties()).isEqualTo(s3FileIO.properties()); + assertThat(deserialized).isInstanceOf(S3FileIO.class); + assertThat(deserialized.properties()).isEqualTo(s3FileIO.properties()); } } @@ -346,8 +345,7 @@ public void testS3FileIOKryoSerialization() throws IOException { testS3FileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testS3FileIO); - Assertions.assertThat(roundTripSerializedFileIO.properties()) - .isEqualTo(testS3FileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testS3FileIO.properties()); } @Test @@ -358,8 +356,7 @@ public void testS3FileIOWithEmptyPropsKryoSerialization() throws IOException { testS3FileIO.initialize(ImmutableMap.of()); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testS3FileIO); - Assertions.assertThat(roundTripSerializedFileIO.properties()) - .isEqualTo(testS3FileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testS3FileIO.properties()); } @Test @@ -370,8 +367,7 @@ public void testS3FileIOJavaSerialization() throws IOException, ClassNotFoundExc testS3FileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.roundTripSerialize(testS3FileIO); - Assertions.assertThat(roundTripSerializedFileIO.properties()) - .isEqualTo(testS3FileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testS3FileIO.properties()); } @Test @@ -384,7 +380,7 @@ public void testResolvingFileIOLoad() { .hiddenImpl(ResolvingFileIO.class, String.class) .build(resolvingFileIO) .invoke("s3://foo/bar"); - Assertions.assertThat(result).isInstanceOf(S3FileIO.class); + assertThat(result).isInstanceOf(S3FileIO.class); } @Test @@ -405,7 +401,7 @@ public void testInputFileWithDataFile() throws IOException { InputFile inputFile = s3FileIO.newInputFile(dataFile); reset(s3mock); - Assertions.assertThat(inputFile.getLength()) + assertThat(inputFile.getLength()) .as("Data file length should be determined from the file size stats") .isEqualTo(123L); verify(s3mock, never()).headObject(any(HeadObjectRequest.class)); @@ -431,7 +427,7 @@ public void testInputFileWithManifest() throws IOException { InputFile inputFile = s3FileIO.newInputFile(manifest); reset(s3mock); - Assertions.assertThat(inputFile.getLength()).isEqualTo(manifest.length()); + assertThat(inputFile.getLength()).isEqualTo(manifest.length()); verify(s3mock, never()).headObject(any(HeadObjectRequest.class)); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index c6d3776b9b0e..f445a2d22422 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.s3; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.net.URI; import java.util.Collections; import java.util.List; @@ -29,7 +32,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.Mockito; @@ -51,71 +53,70 @@ public class TestS3FileIOProperties { public void testS3FileIOPropertiesDefaultValues() { S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); - Assertions.assertThat(S3FileIOProperties.SSE_TYPE_NONE).isEqualTo(s3FileIOProperties.sseType()); + assertThat(S3FileIOProperties.SSE_TYPE_NONE).isEqualTo(s3FileIOProperties.sseType()); - Assertions.assertThat(s3FileIOProperties.sseKey()).isNull(); - Assertions.assertThat(s3FileIOProperties.sseMd5()).isNull(); - Assertions.assertThat(s3FileIOProperties.accessKeyId()).isNull(); - Assertions.assertThat(s3FileIOProperties.secretAccessKey()).isNull(); - Assertions.assertThat(s3FileIOProperties.sessionToken()).isNull(); - Assertions.assertThat(s3FileIOProperties.acl()).isNull(); - Assertions.assertThat(s3FileIOProperties.endpoint()).isNull(); - Assertions.assertThat(s3FileIOProperties.writeStorageClass()).isNull(); + assertThat(s3FileIOProperties.sseKey()).isNull(); + assertThat(s3FileIOProperties.sseMd5()).isNull(); + assertThat(s3FileIOProperties.accessKeyId()).isNull(); + assertThat(s3FileIOProperties.secretAccessKey()).isNull(); + assertThat(s3FileIOProperties.sessionToken()).isNull(); + assertThat(s3FileIOProperties.acl()).isNull(); + assertThat(s3FileIOProperties.endpoint()).isNull(); + assertThat(s3FileIOProperties.writeStorageClass()).isNull(); - Assertions.assertThat(S3FileIOProperties.PRELOAD_CLIENT_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.PRELOAD_CLIENT_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isPreloadClientEnabled()); - Assertions.assertThat(S3FileIOProperties.DUALSTACK_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.DUALSTACK_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isDualStackEnabled()); - Assertions.assertThat(S3FileIOProperties.PATH_STYLE_ACCESS_DEFAULT) + assertThat(S3FileIOProperties.PATH_STYLE_ACCESS_DEFAULT) .isEqualTo(s3FileIOProperties.isPathStyleAccess()); - Assertions.assertThat(S3FileIOProperties.USE_ARN_REGION_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.USE_ARN_REGION_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isUseArnRegionEnabled()); - Assertions.assertThat(S3FileIOProperties.ACCELERATION_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.ACCELERATION_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isAccelerationEnabled()); - Assertions.assertThat(S3FileIOProperties.REMOTE_SIGNING_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.REMOTE_SIGNING_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isRemoteSigningEnabled()); - Assertions.assertThat(Runtime.getRuntime().availableProcessors()) + assertThat(Runtime.getRuntime().availableProcessors()) .isEqualTo(s3FileIOProperties.multipartUploadThreads()); - Assertions.assertThat(S3FileIOProperties.MULTIPART_SIZE_DEFAULT) + assertThat(S3FileIOProperties.MULTIPART_SIZE_DEFAULT) .isEqualTo(s3FileIOProperties.multiPartSize()); - Assertions.assertThat(S3FileIOProperties.MULTIPART_THRESHOLD_FACTOR_DEFAULT) + assertThat(S3FileIOProperties.MULTIPART_THRESHOLD_FACTOR_DEFAULT) .isEqualTo(s3FileIOProperties.multipartThresholdFactor()); - Assertions.assertThat(S3FileIOProperties.DELETE_BATCH_SIZE_DEFAULT) + assertThat(S3FileIOProperties.DELETE_BATCH_SIZE_DEFAULT) .isEqualTo(s3FileIOProperties.deleteBatchSize()); - Assertions.assertThat(System.getProperty("java.io.tmpdir")) + assertThat(System.getProperty("java.io.tmpdir")) .isEqualTo(s3FileIOProperties.stagingDirectory()); - Assertions.assertThat(S3FileIOProperties.CHECKSUM_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.CHECKSUM_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isChecksumEnabled()); - Assertions.assertThat(Sets.newHashSet()).isEqualTo(s3FileIOProperties.writeTags()); + assertThat(Sets.newHashSet()).isEqualTo(s3FileIOProperties.writeTags()); - Assertions.assertThat(S3FileIOProperties.WRITE_TABLE_TAG_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.WRITE_TABLE_TAG_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.writeTableTagEnabled()); - Assertions.assertThat(S3FileIOProperties.WRITE_NAMESPACE_TAG_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.WRITE_NAMESPACE_TAG_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isWriteNamespaceTagEnabled()); - Assertions.assertThat(Sets.newHashSet()).isEqualTo(s3FileIOProperties.deleteTags()); + assertThat(Sets.newHashSet()).isEqualTo(s3FileIOProperties.deleteTags()); - Assertions.assertThat(Runtime.getRuntime().availableProcessors()) + assertThat(Runtime.getRuntime().availableProcessors()) .isEqualTo(s3FileIOProperties.deleteThreads()); - Assertions.assertThat(S3FileIOProperties.DELETE_ENABLED_DEFAULT) + assertThat(S3FileIOProperties.DELETE_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isDeleteEnabled()); - Assertions.assertThat(Collections.emptyMap()) - .isEqualTo(s3FileIOProperties.bucketToAccessPointMapping()); + assertThat(Collections.emptyMap()).isEqualTo(s3FileIOProperties.bucketToAccessPointMapping()); } @Test @@ -123,85 +124,80 @@ public void testS3FileIOProperties() { Map map = getTestProperties(); S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(map); - Assertions.assertThat(map) - .containsEntry(S3FileIOProperties.SSE_TYPE, s3FileIOProperties.sseType()); + assertThat(map).containsEntry(S3FileIOProperties.SSE_TYPE, s3FileIOProperties.sseType()); - Assertions.assertThat(map) - .containsEntry(S3FileIOProperties.SSE_KEY, s3FileIOProperties.sseKey()); + assertThat(map).containsEntry(S3FileIOProperties.SSE_KEY, s3FileIOProperties.sseKey()); - Assertions.assertThat(map) - .containsEntry(S3FileIOProperties.SSE_MD5, s3FileIOProperties.sseMd5()); + assertThat(map).containsEntry(S3FileIOProperties.SSE_MD5, s3FileIOProperties.sseMd5()); - Assertions.assertThat(map) + assertThat(map) .containsEntry(S3FileIOProperties.ACCESS_KEY_ID, s3FileIOProperties.accessKeyId()); - Assertions.assertThat(map) + assertThat(map) .containsEntry(S3FileIOProperties.SECRET_ACCESS_KEY, s3FileIOProperties.secretAccessKey()); - Assertions.assertThat(map) + assertThat(map) .containsEntry(S3FileIOProperties.SESSION_TOKEN, s3FileIOProperties.sessionToken()); - Assertions.assertThat(map) - .containsEntry(S3FileIOProperties.ACL, s3FileIOProperties.acl().toString()); + assertThat(map).containsEntry(S3FileIOProperties.ACL, s3FileIOProperties.acl().toString()); - Assertions.assertThat(map) - .containsEntry(S3FileIOProperties.ENDPOINT, s3FileIOProperties.endpoint()); + assertThat(map).containsEntry(S3FileIOProperties.ENDPOINT, s3FileIOProperties.endpoint()); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.PRELOAD_CLIENT_ENABLED, String.valueOf(s3FileIOProperties.isPreloadClientEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.DUALSTACK_ENABLED, String.valueOf(s3FileIOProperties.isDualStackEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.PATH_STYLE_ACCESS, String.valueOf(s3FileIOProperties.isPathStyleAccess())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.USE_ARN_REGION_ENABLED, String.valueOf(s3FileIOProperties.isUseArnRegionEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.ACCELERATION_ENABLED, String.valueOf(s3FileIOProperties.isAccelerationEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.REMOTE_SIGNING_ENABLED, String.valueOf(s3FileIOProperties.isRemoteSigningEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.MULTIPART_UPLOAD_THREADS, String.valueOf(s3FileIOProperties.multipartUploadThreads())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.MULTIPART_SIZE, String.valueOf(s3FileIOProperties.multiPartSize())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.MULTIPART_THRESHOLD_FACTOR, String.valueOf(s3FileIOProperties.multipartThresholdFactor())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.DELETE_BATCH_SIZE, String.valueOf(s3FileIOProperties.deleteBatchSize())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.STAGING_DIRECTORY, String.valueOf(s3FileIOProperties.stagingDirectory())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.CHECKSUM_ENABLED, String.valueOf(s3FileIOProperties.isChecksumEnabled())); @@ -210,15 +206,15 @@ public void testS3FileIOProperties() { s3FileIOProperties.writeTags().stream().map(Tag::value).collect(Collectors.toList()); writeTagValues.forEach( value -> - Assertions.assertThat(map) + assertThat(map) .containsEntry(S3FileIOProperties.WRITE_TAGS_PREFIX + S3_WRITE_TAG_KEY, value)); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.WRITE_TABLE_TAG_ENABLED, String.valueOf(s3FileIOProperties.writeTableTagEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.WRITE_NAMESPACE_TAG_ENABLED, String.valueOf(s3FileIOProperties.isWriteNamespaceTagEnabled())); @@ -227,14 +223,14 @@ public void testS3FileIOProperties() { s3FileIOProperties.deleteTags().stream().map(Tag::value).collect(Collectors.toList()); deleteTagValues.forEach( value -> - Assertions.assertThat(map) + assertThat(map) .containsEntry(S3FileIOProperties.DELETE_TAGS_PREFIX + S3_DELETE_TAG_KEY, value)); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.DELETE_THREADS, String.valueOf(s3FileIOProperties.deleteThreads())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.DELETE_ENABLED, String.valueOf(s3FileIOProperties.isDeleteEnabled())); @@ -244,22 +240,21 @@ public void testS3FileIOProperties() { .values() .forEach( value -> - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.ACCESS_POINTS_PREFIX + S3_TEST_BUCKET_NAME, value)); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.PRELOAD_CLIENT_ENABLED, String.valueOf(s3FileIOProperties.isPreloadClientEnabled())); - Assertions.assertThat(map) + assertThat(map) .containsEntry( S3FileIOProperties.REMOTE_SIGNING_ENABLED, String.valueOf(s3FileIOProperties.isRemoteSigningEnabled())); - Assertions.assertThat(map) - .containsEntry(S3FileIOProperties.WRITE_STORAGE_CLASS, "INTELLIGENT_TIERING"); + assertThat(map).containsEntry(S3FileIOProperties.WRITE_STORAGE_CLASS, "INTELLIGENT_TIERING"); } @Test @@ -267,7 +262,7 @@ public void testS3AccessKeySet_secretKeyNotSet() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.ACCESS_KEY_ID, "s3-access-key"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(ValidationException.class) .hasMessage("S3 client access key ID and secret access key must be set at the same time"); } @@ -277,7 +272,7 @@ public void testS3SecretKeySet_accessKeyNotSet() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.SECRET_ACCESS_KEY, "s3-secret-key"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(ValidationException.class) .hasMessage("S3 client access key ID and secret access key must be set at the same time"); } @@ -287,7 +282,7 @@ public void testS3FileIoSseCustom_mustHaveCustomKey() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.SSE_TYPE, S3FileIOProperties.SSE_TYPE_CUSTOM); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot initialize SSE-C S3FileIO with null encryption key"); } @@ -298,7 +293,7 @@ public void testS3FileIoSseCustom_mustHaveCustomMd5() { map.put(S3FileIOProperties.SSE_TYPE, S3FileIOProperties.SSE_TYPE_CUSTOM); map.put(S3FileIOProperties.SSE_KEY, "something"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot initialize SSE-C S3FileIO with null encryption key MD5"); } @@ -308,7 +303,7 @@ public void testS3FileIoAcl() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.ACL, ObjectCannedACL.AUTHENTICATED_READ.toString()); S3FileIOProperties properties = new S3FileIOProperties(map); - Assertions.assertThat(ObjectCannedACL.AUTHENTICATED_READ).isEqualTo(properties.acl()); + assertThat(ObjectCannedACL.AUTHENTICATED_READ).isEqualTo(properties.acl()); } @Test @@ -316,7 +311,7 @@ public void testS3FileIoAcl_unknownType() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.ACL, "bad-input"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot support S3 CannedACL bad-input"); } @@ -326,7 +321,7 @@ public void testS3MultipartSizeTooSmall() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.MULTIPART_SIZE, "1"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Minimum multipart upload object size must be larger than 5 MB."); } @@ -336,7 +331,7 @@ public void testS3MultipartSizeTooLarge() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.MULTIPART_SIZE, "5368709120"); // 5GB - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Input malformed or exceeded maximum multipart upload size 5GB: 5368709120"); } @@ -346,7 +341,7 @@ public void testS3MultipartThresholdFactorLessThanOne() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.MULTIPART_THRESHOLD_FACTOR, "0.9"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Multipart threshold factor must be >= to 1.0"); } @@ -356,7 +351,7 @@ public void testS3FileIoDeleteBatchSizeTooLarge() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.DELETE_BATCH_SIZE, "2000"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Deletion batch size must be between 1 and 1000"); } @@ -366,7 +361,7 @@ public void testS3FileIoDeleteBatchSizeTooSmall() { Map map = Maps.newHashMap(); map.put(S3FileIOProperties.DELETE_BATCH_SIZE, "0"); - Assertions.assertThatThrownBy(() -> new S3FileIOProperties(map)) + assertThatThrownBy(() -> new S3FileIOProperties(map)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Deletion batch size must be between 1 and 1000"); } @@ -448,13 +443,13 @@ public void testApplyS3ServiceConfigurations() { Mockito.verify(mockA).serviceConfiguration(s3ConfigurationCaptor.capture()); S3Configuration s3Configuration = s3ConfigurationCaptor.getValue(); - Assertions.assertThat(s3Configuration.pathStyleAccessEnabled()) + assertThat(s3Configuration.pathStyleAccessEnabled()) .as("s3 path style access enabled parameter should be set to true") .isTrue(); - Assertions.assertThat(s3Configuration.useArnRegionEnabled()) + assertThat(s3Configuration.useArnRegionEnabled()) .as("s3 use arn region enabled parameter should be set to true") .isTrue(); - Assertions.assertThat(s3Configuration.accelerateModeEnabled()) + assertThat(s3Configuration.accelerateModeEnabled()) .as("s3 acceleration mode enabled parameter should be set to true") .isFalse(); } 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 feaac4eadad5..95f4d09a7e0b 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 @@ -18,6 +18,9 @@ */ package org.apache.iceberg.aws.s3; +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; @@ -25,7 +28,6 @@ import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.RangeReadable; import org.apache.iceberg.io.SeekableInputStream; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -88,7 +90,7 @@ private void readAndCheck( SeekableInputStream in, long rangeStart, int size, byte[] original, boolean buffered) throws IOException { in.seek(rangeStart); - Assertions.assertThat(in.getPos()).isEqualTo(rangeStart); + assertThat(in.getPos()).isEqualTo(rangeStart); long rangeEnd = rangeStart + size; byte[] actual = new byte[size]; @@ -102,9 +104,8 @@ private void readAndCheck( } } - Assertions.assertThat(in.getPos()).isEqualTo(rangeEnd); - Assertions.assertThat(actual) - .isEqualTo(Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd)); + assertThat(in.getPos()).isEqualTo(rangeEnd); + assertThat(actual).isEqualTo(Arrays.copyOfRange(original, (int) rangeStart, (int) rangeEnd)); } @Test @@ -145,7 +146,7 @@ private void readAndCheckRanges( throws IOException { in.readFully(position, buffer, offset, length); - Assertions.assertThat(Arrays.copyOfRange(buffer, offset, offset + length)) + assertThat(Arrays.copyOfRange(buffer, offset, offset + length)) .isEqualTo(Arrays.copyOfRange(original, offset, offset + length)); } @@ -154,7 +155,7 @@ public void testClose() throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/closed.dat"); SeekableInputStream closed = new S3InputStream(s3, uri); closed.close(); - Assertions.assertThatThrownBy(() -> closed.seek(0)) + assertThatThrownBy(() -> closed.seek(0)) .isInstanceOf(IllegalStateException.class) .hasMessage("already closed"); } @@ -170,7 +171,7 @@ public void testSeek() throws Exception { in.seek(expected.length / 2); byte[] actual = new byte[expected.length / 2]; IOUtil.readFully(in, actual, 0, expected.length / 2); - Assertions.assertThat(actual) + assertThat(actual) .isEqualTo(Arrays.copyOfRange(expected, expected.length / 2, expected.length)); } } 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 6f8d1d6cb988..6fbe59e47f7f 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 @@ -19,6 +19,10 @@ package org.apache.iceberg.aws.s3; import static org.apache.iceberg.metrics.MetricsContext.nullMetrics; +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.Assertions.fail; import static org.mockito.AdditionalAnswers.delegatesTo; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doThrow; @@ -43,7 +47,6 @@ import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -123,7 +126,7 @@ public void testAbortAfterFailedPartUpload() { RuntimeException mockException = new RuntimeException("mock uploadPart failure"); doThrow(mockException).when(s3mock).uploadPart((UploadPartRequest) any(), (RequestBody) any()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try (S3OutputStream stream = new S3OutputStream(s3mock, randomURI(), properties, nullMetrics())) { @@ -143,7 +146,7 @@ public void testAbortMultipart() { .when(s3mock) .completeMultipartUpload((CompleteMultipartUploadRequest) any()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> { try (S3OutputStream stream = new S3OutputStream(s3mock, randomURI(), properties, nullMetrics())) { @@ -188,11 +191,11 @@ public void testDoubleClose() throws IOException { .putObject(any(PutObjectRequest.class), any(RequestBody.class)); S3OutputStream stream = new S3OutputStream(s3mock, randomURI(), properties, nullMetrics()); - Assertions.assertThatThrownBy(stream::close) + assertThatThrownBy(stream::close) .isInstanceOf(mockException.getClass()) .hasMessageContaining(mockException.getMessage()); - Assertions.assertThatNoException().isThrownBy(stream::close); + assertThatNoException().isThrownBy(stream::close); } private void writeTest() { @@ -252,8 +255,7 @@ private void checkUploadPartRequestContent( for (int i = 0; i < uploadPartRequests.size(); ++i) { int offset = i * FIVE_MBS; int len = (i + 1) * FIVE_MBS - 1 > data.length ? data.length - offset : FIVE_MBS; - Assertions.assertThat(uploadPartRequests.get(i).contentMD5()) - .isEqualTo(getDigest(data, offset, len)); + assertThat(uploadPartRequests.get(i).contentMD5()).isEqualTo(getDigest(data, offset, len)); } } } @@ -262,8 +264,7 @@ private void checkPutObjectRequestContent( byte[] data, ArgumentCaptor putObjectRequestArgumentCaptor) { if (properties.isChecksumEnabled()) { List putObjectRequests = putObjectRequestArgumentCaptor.getAllValues(); - Assertions.assertThat(putObjectRequests.get(0).contentMD5()) - .isEqualTo(getDigest(data, 0, data.length)); + assertThat(putObjectRequests.get(0).contentMD5()).isEqualTo(getDigest(data, 0, data.length)); } } @@ -271,7 +272,7 @@ private void checkTags(ArgumentCaptor putObjectRequestArgument if (properties.isChecksumEnabled()) { List putObjectRequests = putObjectRequestArgumentCaptor.getAllValues(); String tagging = putObjectRequests.get(0).tagging(); - Assertions.assertThat(getTags(properties.writeTags())).isEqualTo(tagging); + assertThat(getTags(properties.writeTags())).isEqualTo(tagging); } } @@ -285,7 +286,7 @@ private String getDigest(byte[] data, int offset, int length) { md5.update(data, offset, length); return BinaryUtils.toBase64(md5.digest()); } catch (NoSuchAlgorithmException e) { - Assertions.fail("Failed to get MD5 MessageDigest. %s", e); + fail("Failed to get MD5 MessageDigest. %s", e); } return null; } @@ -294,11 +295,11 @@ private void writeAndVerify(S3Client client, S3URI uri, byte[] data, boolean arr try (S3OutputStream stream = new S3OutputStream(client, uri, properties, nullMetrics())) { if (arrayWrite) { stream.write(data); - Assertions.assertThat(stream.getPos()).isEqualTo(data.length); + assertThat(stream.getPos()).isEqualTo(data.length); } else { for (int i = 0; i < data.length; i++) { stream.write(data[i]); - Assertions.assertThat(stream.getPos()).isEqualTo(i + 1); + assertThat(stream.getPos()).isEqualTo(i + 1); } } } catch (IOException e) { @@ -306,11 +307,11 @@ private void writeAndVerify(S3Client client, S3URI uri, byte[] data, boolean arr } byte[] actual = readS3Data(uri); - Assertions.assertThat(actual).isEqualTo(data); + assertThat(actual).isEqualTo(data); // Verify all staging files are cleaned up try { - Assertions.assertThat(Files.list(tmpDir)).isEmpty(); + assertThat(Files.list(tmpDir)).isEmpty(); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java index 379fde9244b4..67c67c1d5604 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3RequestUtil.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg.aws.s3; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; + import org.junit.jupiter.api.Test; import software.amazon.awssdk.services.s3.model.S3Request; import software.amazon.awssdk.services.s3.model.ServerSideEncryption; @@ -44,11 +45,11 @@ public void testConfigureServerSideCustomEncryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assertions.assertThat(serverSideEncryption).isNull(); - Assertions.assertThat(kmsKeyId).isNull(); - Assertions.assertThat(customAlgorithm).isEqualTo(ServerSideEncryption.AES256.name()); - Assertions.assertThat(customKey).isEqualTo("key"); - Assertions.assertThat(customMd5).isEqualTo("md5"); + assertThat(serverSideEncryption).isNull(); + assertThat(kmsKeyId).isNull(); + assertThat(customAlgorithm).isEqualTo(ServerSideEncryption.AES256.name()); + assertThat(customKey).isEqualTo("key"); + assertThat(customMd5).isEqualTo("md5"); } @Test @@ -62,11 +63,11 @@ public void testConfigureServerSideS3Encryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assertions.assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AES256); - Assertions.assertThat(kmsKeyId).isNull(); - Assertions.assertThat(customAlgorithm).isNull(); - Assertions.assertThat(customKey).isNull(); - Assertions.assertThat(customMd5).isNull(); + assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AES256); + assertThat(kmsKeyId).isNull(); + assertThat(customAlgorithm).isNull(); + assertThat(customKey).isNull(); + assertThat(customMd5).isNull(); } @Test @@ -81,11 +82,11 @@ public void testConfigureServerSideKmsEncryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assertions.assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AWS_KMS); - Assertions.assertThat(kmsKeyId).isEqualTo("key"); - Assertions.assertThat(customAlgorithm).isNull(); - Assertions.assertThat(customKey).isNull(); - Assertions.assertThat(customMd5).isNull(); + assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AWS_KMS); + assertThat(kmsKeyId).isEqualTo("key"); + assertThat(customAlgorithm).isNull(); + assertThat(customKey).isNull(); + assertThat(customMd5).isNull(); } @Test @@ -100,11 +101,11 @@ public void testConfigureDualLayerServerSideKmsEncryption() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assertions.assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AWS_KMS_DSSE); - Assertions.assertThat(kmsKeyId).isEqualTo("key"); - Assertions.assertThat(customAlgorithm).isNull(); - Assertions.assertThat(customKey).isNull(); - Assertions.assertThat(customMd5).isNull(); + assertThat(serverSideEncryption).isEqualTo(ServerSideEncryption.AWS_KMS_DSSE); + assertThat(kmsKeyId).isEqualTo("key"); + assertThat(customAlgorithm).isNull(); + assertThat(customKey).isNull(); + assertThat(customMd5).isNull(); } @Test @@ -119,11 +120,11 @@ public void testConfigureEncryptionSkipNullSetters() { this::setCustomAlgorithm, this::setCustomKey, this::setCustomMd5); - Assertions.assertThat(serverSideEncryption).isNull(); - Assertions.assertThat(kmsKeyId).isNull(); - Assertions.assertThat(customAlgorithm).isNull(); - Assertions.assertThat(customKey).isNull(); - Assertions.assertThat(customMd5).isNull(); + assertThat(serverSideEncryption).isNull(); + assertThat(kmsKeyId).isNull(); + assertThat(customAlgorithm).isNull(); + assertThat(customKey).isNull(); + assertThat(customMd5).isNull(); } public S3Request.Builder setCustomAlgorithm(String algorithm) { 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 803cbe098416..383ff67d161d 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 @@ -18,11 +18,13 @@ */ 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.Map; 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.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestS3URI { @@ -32,9 +34,9 @@ public void testLocationParsing() { String p1 = "s3://bucket/path/to/file"; S3URI uri1 = new S3URI(p1); - Assertions.assertThat(uri1.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri1.key()).isEqualTo("path/to/file"); - Assertions.assertThat(uri1.toString()).isEqualTo(p1); + assertThat(uri1.bucket()).isEqualTo("bucket"); + assertThat(uri1.key()).isEqualTo("path/to/file"); + assertThat(uri1.toString()).isEqualTo(p1); } @Test @@ -42,15 +44,15 @@ public void testEncodedString() { String p1 = "s3://bucket/path%20to%20file"; S3URI uri1 = new S3URI(p1); - Assertions.assertThat(uri1.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri1.key()).isEqualTo("path%20to%20file"); - Assertions.assertThat(uri1.toString()).isEqualTo(p1); + assertThat(uri1.bucket()).isEqualTo("bucket"); + assertThat(uri1.key()).isEqualTo("path%20to%20file"); + assertThat(uri1.toString()).isEqualTo(p1); } @Test public void testMissingScheme() { - Assertions.assertThatThrownBy(() -> new S3URI("/path/to/file")) + assertThatThrownBy(() -> new S3URI("/path/to/file")) .isInstanceOf(ValidationException.class) .hasMessage("Invalid S3 URI, cannot determine scheme: /path/to/file"); } @@ -60,9 +62,9 @@ public void testOnlyBucketNameLocation() { String p1 = "s3://bucket"; S3URI url1 = new S3URI(p1); - Assertions.assertThat(url1.bucket()).isEqualTo("bucket"); - Assertions.assertThat(url1.key()).isEqualTo(""); - Assertions.assertThat(url1.toString()).isEqualTo(p1); + assertThat(url1.bucket()).isEqualTo("bucket"); + assertThat(url1.key()).isEqualTo(""); + assertThat(url1.toString()).isEqualTo(p1); } @Test @@ -70,17 +72,17 @@ public void testQueryAndFragment() { String p1 = "s3://bucket/path/to/file?query=foo#bar"; S3URI uri1 = new S3URI(p1); - Assertions.assertThat(uri1.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri1.key()).isEqualTo("path/to/file"); - Assertions.assertThat(uri1.toString()).isEqualTo(p1); + assertThat(uri1.bucket()).isEqualTo("bucket"); + assertThat(uri1.key()).isEqualTo("path/to/file"); + assertThat(uri1.toString()).isEqualTo(p1); } @Test public void testValidSchemes() { for (String scheme : Lists.newArrayList("https", "s3", "s3a", "s3n", "gs")) { S3URI uri = new S3URI(scheme + "://bucket/path/to/file"); - Assertions.assertThat(uri.bucket()).isEqualTo("bucket"); - Assertions.assertThat(uri.key()).isEqualTo("path/to/file"); + assertThat(uri.bucket()).isEqualTo("bucket"); + assertThat(uri.key()).isEqualTo("path/to/file"); } } @@ -90,8 +92,8 @@ public void testS3URIWithBucketToAccessPointMapping() { Map bucketToAccessPointMapping = ImmutableMap.of("bucket", "access-point"); S3URI uri1 = new S3URI(p1, bucketToAccessPointMapping); - Assertions.assertThat(uri1.bucket()).isEqualTo("access-point"); - Assertions.assertThat(uri1.key()).isEqualTo("path/to/file"); - Assertions.assertThat(uri1.toString()).isEqualTo(p1); + assertThat(uri1.bucket()).isEqualTo("access-point"); + assertThat(uri1.key()).isEqualTo("path/to/file"); + assertThat(uri1.toString()).isEqualTo(p1); } } diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java index 2c5f74ad8064..75ae2d88cccf 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignRequestParser.java @@ -18,43 +18,45 @@ */ package org.apache.iceberg.aws.s3.signer; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestS3SignRequestParser { @Test public void nullRequest() { - Assertions.assertThatThrownBy(() -> S3SignRequestParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> S3SignRequestParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse s3 sign request from null object"); - Assertions.assertThatThrownBy(() -> S3SignRequestParser.toJson(null)) + assertThatThrownBy(() -> S3SignRequestParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid s3 sign request: null"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> S3SignRequestParser.fromJson("{}")) + assertThatThrownBy(() -> S3SignRequestParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: region"); - Assertions.assertThatThrownBy(() -> S3SignRequestParser.fromJson("{\"region\":\"us-west-2\"}")) + assertThatThrownBy(() -> S3SignRequestParser.fromJson("{\"region\":\"us-west-2\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: method"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> S3SignRequestParser.fromJson("{\"region\":\"us-west-2\", \"method\" : \"PUT\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: uri"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> S3SignRequestParser.fromJson( "{\n" @@ -68,7 +70,7 @@ public void missingFields() { @Test public void invalidMethod() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> S3SignRequestParser.fromJson( "{\n" @@ -83,7 +85,7 @@ public void invalidMethod() { @Test public void invalidUri() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> S3SignRequestParser.fromJson( "{\n" @@ -98,7 +100,7 @@ public void invalidUri() { @Test public void invalidRegion() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> S3SignRequestParser.fromJson( "{\n" @@ -131,8 +133,8 @@ public void roundTripSerde() { .build(); String json = S3SignRequestParser.toJson(s3SignRequest, true); - Assertions.assertThat(S3SignRequestParser.fromJson(json)).isEqualTo(s3SignRequest); - Assertions.assertThat(json) + assertThat(S3SignRequestParser.fromJson(json)).isEqualTo(s3SignRequest); + assertThat(json) .isEqualTo( "{\n" + " \"region\" : \"us-west-2\",\n" @@ -168,8 +170,8 @@ public void roundTripSerdeWithProperties() { .build(); String json = S3SignRequestParser.toJson(s3SignRequest, true); - Assertions.assertThat(S3SignRequestParser.fromJson(json)).isEqualTo(s3SignRequest); - Assertions.assertThat(json) + assertThat(S3SignRequestParser.fromJson(json)).isEqualTo(s3SignRequest); + assertThat(json) .isEqualTo( "{\n" + " \"region\" : \"us-west-2\",\n" @@ -209,8 +211,8 @@ public void roundTripWithBody() { .build(); String json = S3SignRequestParser.toJson(s3SignRequest, true); - Assertions.assertThat(S3SignRequestParser.fromJson(json)).isEqualTo(s3SignRequest); - Assertions.assertThat(json) + assertThat(S3SignRequestParser.fromJson(json)).isEqualTo(s3SignRequest); + assertThat(json) .isEqualTo( "{\n" + " \"region\" : \"us-west-2\",\n" diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java index d2cf132ba598..19f2f540d765 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/signer/TestS3SignResponseParser.java @@ -18,34 +18,36 @@ */ package org.apache.iceberg.aws.s3.signer; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.net.URI; import java.util.Arrays; import java.util.Collections; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestS3SignResponseParser { @Test public void nullResponse() { - Assertions.assertThatThrownBy(() -> S3SignResponseParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> S3SignResponseParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse s3 sign response from null object"); - Assertions.assertThatThrownBy(() -> S3SignResponseParser.toJson(null)) + assertThatThrownBy(() -> S3SignResponseParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid s3 sign response: null"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> S3SignResponseParser.fromJson("{}")) + assertThatThrownBy(() -> S3SignResponseParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: uri"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> S3SignResponseParser.fromJson( "{\"uri\" : \"http://localhost:49208/iceberg-signer-test\"}")) @@ -55,8 +57,7 @@ public void missingFields() { @Test public void invalidUri() { - Assertions.assertThatThrownBy( - () -> S3SignResponseParser.fromJson("{\"uri\" : 45, \"headers\" : {}}}")) + assertThatThrownBy(() -> S3SignResponseParser.fromJson("{\"uri\" : 45, \"headers\" : {}}}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: uri: 45"); } @@ -79,8 +80,8 @@ public void roundTripSerde() { .build(); String json = S3SignResponseParser.toJson(s3SignResponse, true); - Assertions.assertThat(S3SignResponseParser.fromJson(json)).isEqualTo(s3SignResponse); - Assertions.assertThat(json) + assertThat(S3SignResponseParser.fromJson(json)).isEqualTo(s3SignResponse); + assertThat(json) .isEqualTo( "{\n" + " \"uri\" : \"http://localhost:49208/iceberg-signer-test\",\n" diff --git a/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java b/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java index 72cc2f91ecdc..ddf476f1ece4 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java +++ b/aws/src/test/java/org/apache/iceberg/aws/util/TestRetryDetector.java @@ -18,7 +18,8 @@ */ package org.apache.iceberg.aws.util; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; + import org.junit.jupiter.api.Test; import org.mockito.Mockito; import software.amazon.awssdk.core.metrics.CoreMetric; @@ -31,7 +32,7 @@ public class TestRetryDetector { @Test public void testNoMetrics() { RetryDetector detector = new RetryDetector(); - Assertions.assertThat(detector.retried()).as("Should default to false").isFalse(); + assertThat(detector.retried()).as("Should default to false").isFalse(); } @Test @@ -40,7 +41,7 @@ public void testRetryCountMissing() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) + assertThat(detector.retried()) .as("Should not detect retries if RETRY_COUNT metric is not reported") .isFalse(); } @@ -52,9 +53,7 @@ public void testRetryCountZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) - .as("Should not detect retries if RETRY_COUNT is zero") - .isFalse(); + assertThat(detector.retried()).as("Should not detect retries if RETRY_COUNT is zero").isFalse(); } @Test @@ -64,9 +63,7 @@ public void testRetryCountNonZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) - .as("Should detect retries if RETRY_COUNT is non-zero") - .isTrue(); + assertThat(detector.retried()).as("Should detect retries if RETRY_COUNT is non-zero").isTrue(); } @Test @@ -77,7 +74,7 @@ public void testMultipleRetryCounts() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) + assertThat(detector.retried()) .as("Should detect retries if even one RETRY_COUNT is non-zero") .isTrue(); } @@ -91,7 +88,7 @@ public void testNestedRetryCountZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) + assertThat(detector.retried()) .as("Should not detect retries if nested RETRY_COUNT is zero") .isFalse(); } @@ -105,7 +102,7 @@ public void testNestedRetryCountNonZero() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) + assertThat(detector.retried()) .as("Should detect retries if nested RETRY_COUNT is non-zero") .isTrue(); } @@ -124,7 +121,7 @@ public void testNestedRetryCountMultipleChildren() { RetryDetector detector = new RetryDetector(); detector.publish(metrics.collect()); - Assertions.assertThat(detector.retried()) + assertThat(detector.retried()) .as("Should detect retries if even one nested RETRY_COUNT is non-zero") .isTrue(); } @@ -138,11 +135,9 @@ public void testMultipleCollectionsReported() { RetryDetector detector = new RetryDetector(); detector.publish(metrics1.collect()); - Assertions.assertThat(detector.retried()) - .as("Should not detect retries if RETRY_COUNT is zero") - .isFalse(); + assertThat(detector.retried()).as("Should not detect retries if RETRY_COUNT is zero").isFalse(); detector.publish(metrics2.collect()); - Assertions.assertThat(detector.retried()) + assertThat(detector.retried()) .as("Should continue detecting retries in additional metrics") .isTrue(); } @@ -158,13 +153,9 @@ public void testNoOpAfterDetection() { RetryDetector detector = new RetryDetector(); detector.publish(metrics1Spy); - Assertions.assertThat(detector.retried()) - .as("Should detect retries if RETRY_COUNT is zero") - .isTrue(); + assertThat(detector.retried()).as("Should detect retries if RETRY_COUNT is zero").isTrue(); detector.publish(metrics2Spy); - Assertions.assertThat(detector.retried()) - .as("Should remain true once a retry is detected") - .isTrue(); + assertThat(detector.retried()).as("Should remain true once a retry is detected").isTrue(); Mockito.verify(metrics1Spy).metricValues(Mockito.eq(CoreMetric.RETRY_COUNT)); Mockito.verifyNoMoreInteractions(metrics1Spy, metrics2Spy); 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 2ebccb625898..32f491119ebc 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_KEY; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_NAME; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.never; @@ -30,7 +31,6 @@ import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class AzurePropertiesTest { @@ -101,7 +101,7 @@ public void testNoConnectionString() { @Test public void testSharedKey() { - Assertions.assertThatIllegalArgumentException() + assertThatIllegalArgumentException() .isThrownBy( () -> new AzureProperties( @@ -111,7 +111,7 @@ public void testSharedKey() { String.format( "Azure authentication: shared-key requires both %s and %s", ADLS_SHARED_KEY_ACCOUNT_NAME, ADLS_SHARED_KEY_ACCOUNT_KEY)); - Assertions.assertThatIllegalArgumentException() + assertThatIllegalArgumentException() .isThrownBy( () -> new AzureProperties(ImmutableMap.of(ADLS_SHARED_KEY_ACCOUNT_NAME, "account"))) .withMessage( diff --git a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java index bb647627a04b..3b9488613367 100644 --- a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java +++ b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalChangelogScan.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.MANIFEST_MERGE_ENABLED; import static org.apache.iceberg.TableProperties.MANIFEST_MIN_MERGE_COUNT; 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; @@ -33,7 +34,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.assertj.core.api.Assertions; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -255,7 +255,7 @@ public void testDeleteFilesAreNotSupported() { table.newRowDelta().addDeletes(FILE_A2_DELETES).commit(); - Assertions.assertThatThrownBy(() -> plan(newScan())) + assertThatThrownBy(() -> plan(newScan())) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Delete files are currently not supported in changelog scans"); } diff --git a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java index 878ca36a10ef..25530d0e5975 100644 --- a/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java +++ b/core/src/test/java/org/apache/iceberg/TestCatalogUtil.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg; +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.hadoop.conf.Configurable; @@ -33,7 +36,6 @@ 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.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCatalogUtil { @@ -46,9 +48,9 @@ public void loadCustomCatalog() { String name = "custom"; Catalog catalog = CatalogUtil.loadCatalog(TestCatalog.class.getName(), name, options, hadoopConf); - Assertions.assertThat(catalog).isInstanceOf(TestCatalog.class); - Assertions.assertThat(((TestCatalog) catalog).catalogName).isEqualTo(name); - Assertions.assertThat(((TestCatalog) catalog).catalogProperties).isEqualTo(options); + assertThat(catalog).isInstanceOf(TestCatalog.class); + assertThat(((TestCatalog) catalog).catalogName).isEqualTo(name); + assertThat(((TestCatalog) catalog).catalogProperties).isEqualTo(options); } @Test @@ -60,10 +62,10 @@ public void loadCustomCatalog_withHadoopConfig() { String name = "custom"; Catalog catalog = CatalogUtil.loadCatalog(TestCatalogConfigurable.class.getName(), name, options, hadoopConf); - Assertions.assertThat(catalog).isInstanceOf(TestCatalogConfigurable.class); - Assertions.assertThat(((TestCatalogConfigurable) catalog).catalogName).isEqualTo(name); - Assertions.assertThat(((TestCatalogConfigurable) catalog).catalogProperties).isEqualTo(options); - Assertions.assertThat(((TestCatalogConfigurable) catalog).configuration).isEqualTo(hadoopConf); + assertThat(catalog).isInstanceOf(TestCatalogConfigurable.class); + assertThat(((TestCatalogConfigurable) catalog).catalogName).isEqualTo(name); + assertThat(((TestCatalogConfigurable) catalog).catalogProperties).isEqualTo(options); + assertThat(((TestCatalogConfigurable) catalog).configuration).isEqualTo(hadoopConf); } @Test @@ -72,7 +74,7 @@ public void loadCustomCatalog_NoArgConstructorNotFound() { options.put("key", "val"); Configuration hadoopConf = new Configuration(); String name = "custom"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CatalogUtil.loadCatalog( TestCatalogBadConstructor.class.getName(), name, options, hadoopConf)) @@ -89,7 +91,7 @@ public void loadCustomCatalog_NotImplementCatalog() { Configuration hadoopConf = new Configuration(); String name = "custom"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CatalogUtil.loadCatalog( TestCatalogNoInterface.class.getName(), name, options, hadoopConf)) @@ -106,7 +108,7 @@ public void loadCustomCatalog_ConstructorErrorCatalog() { String name = "custom"; String impl = TestCatalogErrorConstructor.class.getName(); - Assertions.assertThatThrownBy(() -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)) + assertThatThrownBy(() -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot initialize Catalog implementation") .hasMessageContaining("NoClassDefFoundError: Error while initializing class"); @@ -119,7 +121,7 @@ public void loadCustomCatalog_BadCatalogNameCatalog() { Configuration hadoopConf = new Configuration(); String name = "custom"; String impl = "CatalogDoesNotExist"; - Assertions.assertThatThrownBy(() -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)) + assertThatThrownBy(() -> CatalogUtil.loadCatalog(impl, name, options, hadoopConf)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot initialize Catalog implementation") .hasMessageContaining("java.lang.ClassNotFoundException: CatalogDoesNotExist"); @@ -130,8 +132,8 @@ public void loadCustomFileIO_noArg() { Map properties = Maps.newHashMap(); properties.put("key", "val"); FileIO fileIO = CatalogUtil.loadFileIO(TestFileIONoArg.class.getName(), properties, null); - Assertions.assertThat(fileIO).isInstanceOf(TestFileIONoArg.class); - Assertions.assertThat(((TestFileIONoArg) fileIO).map).isEqualTo(properties); + assertThat(fileIO).isInstanceOf(TestFileIONoArg.class); + assertThat(((TestFileIONoArg) fileIO).map).isEqualTo(properties); } @Test @@ -140,8 +142,8 @@ public void loadCustomFileIO_hadoopConfigConstructor() { configuration.set("key", "val"); FileIO fileIO = CatalogUtil.loadFileIO(HadoopFileIO.class.getName(), Maps.newHashMap(), configuration); - Assertions.assertThat(fileIO).isInstanceOf(HadoopFileIO.class); - Assertions.assertThat(((HadoopFileIO) fileIO).conf().get("key")).isEqualTo("val"); + assertThat(fileIO).isInstanceOf(HadoopFileIO.class); + assertThat(((HadoopFileIO) fileIO).conf().get("key")).isEqualTo("val"); } @Test @@ -151,13 +153,13 @@ public void loadCustomFileIO_configurable() { FileIO fileIO = CatalogUtil.loadFileIO( TestFileIOConfigurable.class.getName(), Maps.newHashMap(), configuration); - Assertions.assertThat(fileIO).isInstanceOf(TestFileIOConfigurable.class); - Assertions.assertThat(((TestFileIOConfigurable) fileIO).configuration).isEqualTo(configuration); + assertThat(fileIO).isInstanceOf(TestFileIOConfigurable.class); + assertThat(((TestFileIOConfigurable) fileIO).configuration).isEqualTo(configuration); } @Test public void loadCustomFileIO_badArg() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CatalogUtil.loadFileIO(TestFileIOBadArg.class.getName(), Maps.newHashMap(), null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith( @@ -167,7 +169,7 @@ public void loadCustomFileIO_badArg() { @Test public void loadCustomFileIO_badClass() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CatalogUtil.loadFileIO(TestFileIONotImpl.class.getName(), Maps.newHashMap(), null)) .isInstanceOf(IllegalArgumentException.class) @@ -182,7 +184,7 @@ public void buildCustomCatalog_withTypeSet() { options.put(CatalogUtil.ICEBERG_CATALOG_TYPE, "hive"); Configuration hadoopConf = new Configuration(); String name = "custom"; - Assertions.assertThatThrownBy(() -> CatalogUtil.buildIcebergCatalog(name, options, hadoopConf)) + assertThatThrownBy(() -> CatalogUtil.buildIcebergCatalog(name, options, hadoopConf)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot create catalog custom, both type and catalog-impl are set: type=hive, catalog-impl=CustomCatalog"); @@ -196,12 +198,12 @@ public void loadCustomMetricsReporter_noArg() { CatalogProperties.METRICS_REPORTER_IMPL, TestMetricsReporterDefault.class.getName()); MetricsReporter metricsReporter = CatalogUtil.loadMetricsReporter(properties); - Assertions.assertThat(metricsReporter).isInstanceOf(TestMetricsReporterDefault.class); + assertThat(metricsReporter).isInstanceOf(TestMetricsReporterDefault.class); } @Test public void loadCustomMetricsReporter_badArg() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CatalogUtil.loadMetricsReporter( ImmutableMap.of( @@ -213,7 +215,7 @@ public void loadCustomMetricsReporter_badArg() { @Test public void loadCustomMetricsReporter_badClass() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CatalogUtil.loadMetricsReporter( ImmutableMap.of( @@ -230,22 +232,22 @@ public void fullTableNameWithDifferentValues() { String nameSpaceWithTwoLevels = "ns.l2"; String tableName = "tbl"; TableIdentifier tableIdentifier = TableIdentifier.of(namespace, tableName); - Assertions.assertThat(CatalogUtil.fullTableName(uriTypeCatalogName, tableIdentifier)) + assertThat(CatalogUtil.fullTableName(uriTypeCatalogName, tableIdentifier)) .isEqualTo(uriTypeCatalogName + "/" + namespace + "." + tableName); tableIdentifier = TableIdentifier.of(nameSpaceWithTwoLevels, tableName); - Assertions.assertThat(CatalogUtil.fullTableName(uriTypeCatalogName, tableIdentifier)) + assertThat(CatalogUtil.fullTableName(uriTypeCatalogName, tableIdentifier)) .isEqualTo(uriTypeCatalogName + "/" + nameSpaceWithTwoLevels + "." + tableName); - Assertions.assertThat(CatalogUtil.fullTableName(uriTypeCatalogName + "/", tableIdentifier)) + assertThat(CatalogUtil.fullTableName(uriTypeCatalogName + "/", tableIdentifier)) .isEqualTo(uriTypeCatalogName + "/" + nameSpaceWithTwoLevels + "." + tableName); String nonUriCatalogName = "test.db.catalog"; - Assertions.assertThat(CatalogUtil.fullTableName(nonUriCatalogName, tableIdentifier)) + assertThat(CatalogUtil.fullTableName(nonUriCatalogName, tableIdentifier)) .isEqualTo(nonUriCatalogName + "." + nameSpaceWithTwoLevels + "." + tableName); String pathStyleCatalogName = "/test/db"; - Assertions.assertThat(CatalogUtil.fullTableName(pathStyleCatalogName, tableIdentifier)) + assertThat(CatalogUtil.fullTableName(pathStyleCatalogName, tableIdentifier)) .isEqualTo(pathStyleCatalogName + "/" + nameSpaceWithTwoLevels + "." + tableName); } diff --git a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java index 1e185b6544e1..83f7fc1f6220 100644 --- a/core/src/test/java/org/apache/iceberg/TestContentFileParser.java +++ b/core/src/test/java/org/apache/iceberg/TestContentFileParser.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.nio.ByteBuffer; import java.util.Arrays; @@ -28,7 +31,6 @@ import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; @@ -37,26 +39,25 @@ public class TestContentFileParser { @Test public void testNullArguments() throws Exception { - Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(null, TestBase.SPEC)) + assertThatThrownBy(() -> ContentFileParser.toJson(null, TestBase.SPEC)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid content file: null"); - Assertions.assertThatThrownBy(() -> ContentFileParser.toJson(TestBase.FILE_A, null)) + assertThatThrownBy(() -> ContentFileParser.toJson(TestBase.FILE_A, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid partition spec: null"); - Assertions.assertThatThrownBy( - () -> ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC, null)) + assertThatThrownBy(() -> ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON generator: null"); - Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(null, TestBase.SPEC)) + assertThatThrownBy(() -> ContentFileParser.fromJson(null, TestBase.SPEC)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON node for content file: null"); String jsonStr = ContentFileParser.toJson(TestBase.FILE_A, TestBase.SPEC); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); - Assertions.assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, null)) + assertThatThrownBy(() -> ContentFileParser.fromJson(jsonNode, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid partition spec: null"); } @@ -66,10 +67,10 @@ public void testNullArguments() throws Exception { public void testDataFile(PartitionSpec spec, DataFile dataFile, String expectedJson) throws Exception { String jsonStr = ContentFileParser.toJson(dataFile, spec); - Assertions.assertThat(jsonStr).isEqualTo(expectedJson); + assertThat(jsonStr).isEqualTo(expectedJson); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); ContentFile deserializedContentFile = ContentFileParser.fromJson(jsonNode, spec); - Assertions.assertThat(deserializedContentFile).isInstanceOf(DataFile.class); + assertThat(deserializedContentFile).isInstanceOf(DataFile.class); assertContentFileEquals(dataFile, deserializedContentFile, spec); } @@ -78,10 +79,10 @@ public void testDataFile(PartitionSpec spec, DataFile dataFile, String expectedJ public void testDeleteFile(PartitionSpec spec, DeleteFile deleteFile, String expectedJson) throws Exception { String jsonStr = ContentFileParser.toJson(deleteFile, spec); - Assertions.assertThat(jsonStr).isEqualTo(expectedJson); + assertThat(jsonStr).isEqualTo(expectedJson); JsonNode jsonNode = JsonUtil.mapper().readTree(jsonStr); ContentFile deserializedContentFile = ContentFileParser.fromJson(jsonNode, spec); - Assertions.assertThat(deserializedContentFile).isInstanceOf(DeleteFile.class); + assertThat(deserializedContentFile).isInstanceOf(DeleteFile.class); assertContentFileEquals(deleteFile, deserializedContentFile, spec); } @@ -313,25 +314,25 @@ private static String deleteFileJsonWithAllOptional(PartitionSpec spec) { static void assertContentFileEquals( ContentFile expected, ContentFile actual, PartitionSpec spec) { - Assertions.assertThat(actual.getClass()).isEqualTo(expected.getClass()); - Assertions.assertThat(actual.specId()).isEqualTo(expected.specId()); - Assertions.assertThat(actual.content()).isEqualTo(expected.content()); - Assertions.assertThat(actual.path()).isEqualTo(expected.path()); - Assertions.assertThat(actual.format()).isEqualTo(expected.format()); - Assertions.assertThat(actual.partition()) + 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.format()).isEqualTo(expected.format()); + assertThat(actual.partition()) .usingComparator(Comparators.forType(spec.partitionType())) .isEqualTo(expected.partition()); - Assertions.assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); - Assertions.assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); - Assertions.assertThat(actual.columnSizes()).isEqualTo(expected.columnSizes()); - Assertions.assertThat(actual.valueCounts()).isEqualTo(expected.valueCounts()); - Assertions.assertThat(actual.nullValueCounts()).isEqualTo(expected.nullValueCounts()); - Assertions.assertThat(actual.nanValueCounts()).isEqualTo(expected.nanValueCounts()); - Assertions.assertThat(actual.lowerBounds()).isEqualTo(expected.lowerBounds()); - Assertions.assertThat(actual.upperBounds()).isEqualTo(expected.upperBounds()); - Assertions.assertThat(actual.keyMetadata()).isEqualTo(expected.keyMetadata()); - Assertions.assertThat(actual.splitOffsets()).isEqualTo(expected.splitOffsets()); - Assertions.assertThat(actual.equalityFieldIds()).isEqualTo(expected.equalityFieldIds()); - Assertions.assertThat(actual.sortOrderId()).isEqualTo(expected.sortOrderId()); + assertThat(actual.recordCount()).isEqualTo(expected.recordCount()); + assertThat(actual.fileSizeInBytes()).isEqualTo(expected.fileSizeInBytes()); + assertThat(actual.columnSizes()).isEqualTo(expected.columnSizes()); + assertThat(actual.valueCounts()).isEqualTo(expected.valueCounts()); + assertThat(actual.nullValueCounts()).isEqualTo(expected.nullValueCounts()); + assertThat(actual.nanValueCounts()).isEqualTo(expected.nanValueCounts()); + assertThat(actual.lowerBounds()).isEqualTo(expected.lowerBounds()); + assertThat(actual.upperBounds()).isEqualTo(expected.upperBounds()); + assertThat(actual.keyMetadata()).isEqualTo(expected.keyMetadata()); + assertThat(actual.splitOffsets()).isEqualTo(expected.splitOffsets()); + assertThat(actual.equalityFieldIds()).isEqualTo(expected.equalityFieldIds()); + assertThat(actual.sortOrderId()).isEqualTo(expected.sortOrderId()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java b/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java index 52d38fa97061..405604ad814a 100644 --- a/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java +++ b/core/src/test/java/org/apache/iceberg/TestEnvironmentContext.java @@ -18,14 +18,15 @@ */ package org.apache.iceberg; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; + import org.junit.jupiter.api.Test; public class TestEnvironmentContext { @Test public void testDefaultValue() { - Assertions.assertThat(EnvironmentContext.get().get("iceberg-version")) + assertThat(EnvironmentContext.get().get("iceberg-version")) .isEqualTo(IcebergBuild.fullVersion()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 73bfba996ebf..6e274c4811ba 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.expressions.ExpressionUtil; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.expressions.ResidualEvaluator; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -29,11 +31,11 @@ public class TestFileScanTaskParser { @Test public void testNullArguments() { - Assertions.assertThatThrownBy(() -> FileScanTaskParser.toJson(null)) + assertThatThrownBy(() -> FileScanTaskParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file scan task: null"); - Assertions.assertThatThrownBy(() -> FileScanTaskParser.fromJson(null, true)) + assertThatThrownBy(() -> FileScanTaskParser.fromJson(null, true)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON string for file scan task: null"); } @@ -44,7 +46,7 @@ public void testParser(boolean caseSensitive) { PartitionSpec spec = TestBase.SPEC; FileScanTask fileScanTask = createScanTask(spec, caseSensitive); String jsonStr = FileScanTaskParser.toJson(fileScanTask); - Assertions.assertThat(jsonStr).isEqualTo(expectedFileScanTaskJson()); + assertThat(jsonStr).isEqualTo(expectedFileScanTaskJson()); FileScanTask deserializedTask = FileScanTaskParser.fromJson(jsonStr, caseSensitive); assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); } @@ -87,17 +89,15 @@ private String expectedFileScanTaskJson() { private static void assertFileScanTaskEquals( FileScanTask expected, FileScanTask actual, PartitionSpec spec, boolean caseSensitive) { TestContentFileParser.assertContentFileEquals(expected.file(), actual.file(), spec); - Assertions.assertThat(actual.deletes()).hasSameSizeAs(expected.deletes()); + assertThat(actual.deletes()).hasSameSizeAs(expected.deletes()); for (int pos = 0; pos < expected.deletes().size(); ++pos) { TestContentFileParser.assertContentFileEquals( expected.deletes().get(pos), actual.deletes().get(pos), spec); } - Assertions.assertThat(expected.schema().sameSchema(actual.schema())) - .as("Schema should match") - .isTrue(); - Assertions.assertThat(actual.spec()).isEqualTo(expected.spec()); - Assertions.assertThat( + assertThat(expected.schema().sameSchema(actual.schema())).as("Schema should match").isTrue(); + assertThat(actual.spec()).isEqualTo(expected.spec()); + assertThat( ExpressionUtil.equivalent( expected.residual(), actual.residual(), TestBase.SCHEMA.asStruct(), caseSensitive)) .as("Residual expression should match") diff --git a/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java b/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java index 314e805b827e..68e9b6b2d10b 100644 --- a/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java +++ b/core/src/test/java/org/apache/iceberg/TestFixedSizeSplitScanTaskIterator.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import org.apache.iceberg.BaseFileScanTask.SplitScanTask; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestFixedSizeSplitScanTaskIterator { @@ -57,8 +58,8 @@ private static void verify(long splitSize, long fileLen, List> offset List split = offsetLenPairs.get(i); long offset = split.get(0); long length = split.get(1); - Assertions.assertThat(task.start()).isEqualTo(offset); - Assertions.assertThat(task.length()).isEqualTo(length); + assertThat(task.start()).isEqualTo(offset); + assertThat(task.length()).isEqualTo(length); } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index 44b09081d7a3..999fd0d92eb2 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -19,6 +19,7 @@ package org.apache.iceberg; 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; @@ -31,7 +32,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.assertj.core.api.recursive.comparison.RecursiveComparisonConfiguration; import org.junit.jupiter.api.TestTemplate; @@ -73,7 +73,7 @@ public void testReaderWithFilterWithoutSelect() throws IOException { @TestTemplate public void testInvalidUsage() throws IOException { ManifestFile manifest = writeManifest(FILE_A, FILE_B); - Assertions.assertThatThrownBy(() -> ManifestFiles.read(manifest, FILE_IO)) + assertThatThrownBy(() -> ManifestFiles.read(manifest, FILE_IO)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot read from ManifestFile with null (unassigned) snapshot ID"); } diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java index 3fe974f086a1..5bd1fcd69120 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java @@ -23,11 +23,12 @@ import static org.apache.iceberg.util.BinaryUtil.truncateBinaryMin; import static org.apache.iceberg.util.UnicodeUtil.truncateStringMax; import static org.apache.iceberg.util.UnicodeUtil.truncateStringMin; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.nio.ByteBuffer; import java.util.Comparator; import org.apache.iceberg.expressions.Literal; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @SuppressWarnings("checkstyle:LocalVariableName") @@ -39,32 +40,32 @@ public void testTruncateBinary() { ByteBuffer emptyByteBuffer = ByteBuffer.allocate(0); Comparator cmp = Literal.of(original).comparator(); - Assertions.assertThat(cmp.compare(truncateBinary(original, 0), emptyByteBuffer)) + assertThat(cmp.compare(truncateBinary(original, 0), emptyByteBuffer)) .as("Truncating to a length of zero should return an empty ByteBuffer") .isEqualTo(0); - Assertions.assertThat(truncateBinary(original, original.remaining())) + assertThat(truncateBinary(original, original.remaining())) .as("Truncating to the original buffer's remaining size should return the original buffer") .isEqualTo(original); - Assertions.assertThat(truncateBinary(original, 16)) + assertThat(truncateBinary(original, 16)) .as( "Truncating with a length greater than the input's remaining size should return the input") .isEqualTo(original); ByteBuffer truncated = truncateBinary(original, 2); - Assertions.assertThat(truncated.remaining()) + assertThat(truncated.remaining()) .as( "Truncating with a length less than the input's remaining size should truncate properly") .isEqualTo(2); - Assertions.assertThat(truncated.position()) + assertThat(truncated.position()) .as( "Truncating with a length less than the input's remaining size should truncate properly") .isEqualTo(0); - Assertions.assertThat(original.remaining()) + assertThat(original.remaining()) .as("Truncating should not modify the input buffer") .isEqualTo(4); - Assertions.assertThat(original.position()) + assertThat(original.position()) .as("Truncating should not modify the input buffer") .isEqualTo(0); - Assertions.assertThatThrownBy(() -> truncateBinary(original, -1)) + assertThatThrownBy(() -> truncateBinary(original, -1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Truncate length should be non-negative"); } @@ -78,20 +79,19 @@ public void testTruncateBinaryMin() { ByteBuffer test2_2 = ByteBuffer.wrap(new byte[] {(byte) 0xFF, (byte) 0xFF}); Comparator cmp = Literal.of(test1).comparator(); - Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1)) + assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1)) .as("Truncated lower bound should be lower than or equal to the actual lower bound") .isLessThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1_2_expected)) + assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 2).value(), test1_2_expected)) .as("Output must have the first two bytes of the input") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 5).value(), test1)) + assertThat(cmp.compare(truncateBinaryMin(Literal.of(test1), 5).value(), test1)) .as("No truncation required as truncate length is greater than the input size") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2)) + assertThat(cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2)) .as("Truncated lower bound should be lower than or equal to the actual lower bound") .isLessThanOrEqualTo(0); - Assertions.assertThat(cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2_2)) + assertThat(cmp.compare(truncateBinaryMin(Literal.of(test2), 2).value(), test2_2)) .as( "Output must have the first two bytes of the input. A lower bound exists " + "even though the first two bytes are the max value") @@ -107,33 +107,30 @@ public void testTruncateBinaryMax() { ByteBuffer expectedOutput = ByteBuffer.wrap(new byte[] {1, 2}); Comparator cmp = Literal.of(test1).comparator(); - Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), test1)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), test1)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), expectedOutput)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test1), 2).value(), expectedOutput)) .as("Output must have two bytes and the second byte of the input must be incremented") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test2), 2).value(), test2)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test2), 2).value(), test2)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateBinaryMax(Literal.of(test2), 3).value(), expectedOutput)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test2), 3).value(), expectedOutput)) .as( "Since the third byte is already the max value, output must have two bytes " + "with the second byte incremented ") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test3), 5).value(), test3)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test3), 5).value(), test3)) .as("No truncation required as truncate length is greater than the input size") .isEqualTo(0); - Assertions.assertThat(truncateBinaryMax(Literal.of(test3), 2)) + assertThat(truncateBinaryMax(Literal.of(test3), 2)) .as("An upper bound doesn't exist since the first two bytes are the max value") .isNull(); - Assertions.assertThat(cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), test4)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), test4)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), expectedOutput)) + assertThat(cmp.compare(truncateBinaryMax(Literal.of(test4), 2).value(), expectedOutput)) .as( "Since a shorter sequence is considered smaller, output must have two bytes " + "and the second byte of the input must be incremented") @@ -155,38 +152,34 @@ public void testTruncateStringMin() { String test4 = "\uD800\uDC00\uD800\uDC00"; String test4_1_expected = "\uD800\uDC00"; Comparator cmp = Literal.of(test1).comparator(); - Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1)) .as("Truncated lower bound should be lower than or equal to the actual lower bound") .isLessThanOrEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 8).value(), test1)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 8).value(), test1)) .as("No truncation required as truncate length is greater than the input size") .isEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMin(Literal.of(test1), 2).value(), test1_2_expected)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 2).value(), test1_2_expected)) .as("Output must have the first two characters of the input") .isEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1_3_expected)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test1), 3).value(), test1_3_expected)) .as("Output must have the first three characters of the input") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test2), 16).value(), test2)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test2), 16).value(), test2)) .as("Truncated lower bound should be lower than or equal to the actual lower bound") .isLessThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMin(Literal.of(test2), 7).value(), test2_7_expected)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test2), 7).value(), test2_7_expected)) .as("Output must have the first seven characters of the input") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3)) .as("Truncated lower bound should be lower than or equal to the actual lower bound") .isLessThanOrEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test3), 2).value(), test3)) .as("No truncation required as truncate length is equal to the input size") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4)) .as("Truncated lower bound should be lower than or equal to the actual lower bound") .isLessThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4_1_expected)) + assertThat(cmp.compare(truncateStringMin(Literal.of(test4), 1).value(), test4_1_expected)) .as("Output must have the first 4 byte UTF-8 character of the input") .isEqualTo(0); } @@ -215,76 +208,68 @@ public void testTruncateStringMax() { String test7_1_expected = "\uD83D\uDE03"; Comparator cmp = Literal.of(test1).comparator(); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 4).value(), test1)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 4).value(), test1)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 7).value(), test1)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 7).value(), test1)) .as("No truncation required as truncate length is equal to the input size") .isEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test1), 2).value(), test1_2_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 2).value(), test1_2_expected)) .as( "Output must have two characters and the second character of the input must " + "be incremented") .isEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test1), 3).value(), test1_3_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 3).value(), test1_3_expected)) .as( "Output must have three characters and the third character of the input must " + "be incremented") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 8).value(), test1)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 8).value(), test1)) .as("No truncation required as truncate length is greater than the input size") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test2), 8).value(), test2)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test2), 8).value(), test2)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test2), 7).value(), test2_7_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test2), 7).value(), test2_7_expected)) .as( "Output must have seven characters and the seventh character of the input must be incremented") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3_3_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test3), 3).value(), test3_3_expected)) .as( "Output must have three characters and the third character of the input must " + "be incremented. The second perceivable character in this string is actually a glyph. It consists of " + "two unicode characters") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4_1_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test4), 1).value(), test4_1_expected)) .as( "Output must have one character. Since the first character is the max 3 byte " + "UTF-8 character, it should be incremented to the lowest 4 byte UTF-8 character") .isEqualTo(0); - Assertions.assertThat(truncateStringMax(Literal.of(test5), 1)) + assertThat(truncateStringMax(Literal.of(test5), 1)) .as("An upper bound doesn't exist since the first two characters are max UTF-8 characters") .isNull(); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 2).value(), test6)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 2).value(), test6)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_2_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_2_expected)) .as( "Test 4 byte UTF-8 character increment. Output must have one character with " + "the first character incremented") .isEqualTo(0); - Assertions.assertThat(cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7_2_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test7), 2).value(), test7_2_expected)) .as( "Test input with multiple 4 byte UTF-8 character where the second unicode character should be incremented") .isEqualTo(0); - Assertions.assertThat( - cmp.compare(truncateStringMax(Literal.of(test7), 1).value(), test7_1_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test7), 1).value(), test7_1_expected)) .as( "Test input with multiple 4 byte UTF-8 character where the first unicode character should be incremented") .isEqualTo(0); diff --git a/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java b/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java index d9ad4fb3cbbd..89eaaafe77ad 100644 --- a/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java +++ b/core/src/test/java/org/apache/iceberg/TestOffsetsBasedSplitScanTaskIterator.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import org.apache.iceberg.BaseFileScanTask.SplitScanTask; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOffsetsBasedSplitScanTaskIterator { @@ -62,15 +63,15 @@ private static void verify( offsetRanges, TestOffsetsBasedSplitScanTaskIterator::createSplitTask); List tasks = Lists.newArrayList(splitTaskIterator); - Assertions.assertThat(tasks).as("Number of tasks don't match").hasSameSizeAs(offsetLenPairs); + assertThat(tasks).as("Number of tasks don't match").hasSameSizeAs(offsetLenPairs); for (int i = 0; i < tasks.size(); i++) { FileScanTask task = tasks.get(i); List split = offsetLenPairs.get(i); long offset = split.get(0); long length = split.get(1); - Assertions.assertThat(task.start()).isEqualTo(offset); - Assertions.assertThat(task.length()).isEqualTo(length); + assertThat(task.start()).isEqualTo(offset); + assertThat(task.length()).isEqualTo(length); } } diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java index 5aedde6ce5b0..bda76469e1fa 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaUnionByFieldName.java @@ -20,6 +20,8 @@ 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.util.List; import java.util.concurrent.atomic.AtomicInteger; @@ -42,7 +44,6 @@ import org.apache.iceberg.types.Types.TimeType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestSchemaUnionByFieldName { @@ -82,7 +83,7 @@ private static NestedField[] primitiveFields( public void testAddTopLevelPrimitives() { Schema newSchema = new Schema(primitiveFields(0, primitiveTypes())); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -91,7 +92,7 @@ public void testAddTopLevelListOfPrimitives() { Schema newSchema = new Schema(optional(1, "aList", Types.ListType.ofOptional(2, primitiveType))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } } @@ -102,7 +103,7 @@ public void testAddTopLevelMapOfPrimitives() { new Schema( optional(1, "aMap", Types.MapType.ofOptional(2, 3, primitiveType, primitiveType))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } } @@ -113,7 +114,7 @@ public void testAddTopLevelStructOfPrimitives() { new Schema( optional(1, "aStruct", Types.StructType.of(optional(2, "primitive", primitiveType)))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(currentSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(currentSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(currentSchema.asStruct()); } } @@ -125,7 +126,7 @@ public void testAddNestedPrimitive() { new Schema( optional(1, "aStruct", Types.StructType.of(optional(2, "primitive", primitiveType)))); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } } @@ -136,7 +137,7 @@ public void testAddNestedPrimitives() { new Schema( optional(1, "aStruct", Types.StructType.of(primitiveFields(1, primitiveTypes())))); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -165,7 +166,7 @@ public void testAddNestedLists() { Types.ListType.ofOptional( 10, DecimalType.of(11, 20)))))))))))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -201,7 +202,7 @@ public void testAddNestedStruct() { "aString", StringType.get())))))))))))))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -234,7 +235,7 @@ public void testAddNestedMaps() { Types.MapType.ofOptional( 12, 13, StringType.get(), StringType.get())))))))); Schema applied = new SchemaUpdate(new Schema(), 0).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -243,8 +244,7 @@ public void testDetectInvalidTopLevelList() { new Schema(optional(1, "aList", Types.ListType.ofOptional(2, StringType.get()))); Schema newSchema = new Schema(optional(1, "aList", Types.ListType.ofOptional(2, LongType.get()))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(currentSchema, 2).unionByNameWith(newSchema).apply()) + assertThatThrownBy(() -> new SchemaUpdate(currentSchema, 2).unionByNameWith(newSchema).apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column type: aList.element: string -> long"); } @@ -260,8 +260,7 @@ public void testDetectInvalidTopLevelMapValue() { new Schema( optional(1, "aMap", Types.MapType.ofOptional(2, 3, StringType.get(), LongType.get()))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(currentSchema, 3).unionByNameWith(newSchema).apply()) + assertThatThrownBy(() -> new SchemaUpdate(currentSchema, 3).unionByNameWith(newSchema).apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column type: aMap.value: string -> long"); } @@ -275,8 +274,7 @@ public void testDetectInvalidTopLevelMapKey() { Schema newSchema = new Schema( optional(1, "aMap", Types.MapType.ofOptional(2, 3, UUIDType.get(), StringType.get()))); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(currentSchema, 3).unionByNameWith(newSchema).apply()) + assertThatThrownBy(() -> new SchemaUpdate(currentSchema, 3).unionByNameWith(newSchema).apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column type: aMap.key: string -> uuid"); } @@ -288,8 +286,8 @@ public void testTypePromoteIntegerToLong() { Schema newSchema = new Schema(required(1, "aCol", LongType.get())); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct().fields()).hasSize(1); - Assertions.assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); + assertThat(applied.asStruct().fields()).hasSize(1); + assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(LongType.get()); } @Test @@ -299,9 +297,9 @@ public void testTypePromoteFloatToDouble() { Schema newSchema = new Schema(required(1, "aCol", DoubleType.get())); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); - Assertions.assertThat(applied.asStruct().fields()).hasSize(1); - Assertions.assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct().fields()).hasSize(1); + assertThat(applied.asStruct().fields().get(0).type()).isEqualTo(DoubleType.get()); } @Test @@ -309,8 +307,7 @@ public void testInvalidTypePromoteDoubleToFloat() { Schema currentSchema = new Schema(required(1, "aCol", DoubleType.get())); Schema newSchema = new Schema(required(1, "aCol", FloatType.get())); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply()) + assertThatThrownBy(() -> new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column type: aCol: double -> float"); } @@ -323,7 +320,7 @@ public void testTypePromoteDecimalToFixedScaleWithWiderPrecision() { Schema newSchema = new Schema(required(1, "aCol", DecimalType.of(22, 1))); Schema applied = new SchemaUpdate(currentSchema, 1).unionByNameWith(newSchema).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); + assertThat(applied.asStruct()).isEqualTo(newSchema.asStruct()); } @Test @@ -384,7 +381,7 @@ public void testAddPrimitiveToNestedStruct() { optional(5, "value", StringType.get()), optional(6, "time", TimeType.get()))))))))); - Assertions.assertThat(applied.asStruct()).isEqualTo(expected.asStruct()); + assertThat(applied.asStruct()).isEqualTo(expected.asStruct()); } @Test @@ -392,8 +389,7 @@ public void testReplaceListWithPrimitive() { Schema currentSchema = new Schema(optional(1, "aColumn", Types.ListType.ofOptional(2, StringType.get()))); Schema newSchema = new Schema(optional(1, "aColumn", StringType.get())); - Assertions.assertThatThrownBy( - () -> new SchemaUpdate(currentSchema, 2).unionByNameWith(newSchema).apply()) + assertThatThrownBy(() -> new SchemaUpdate(currentSchema, 2).unionByNameWith(newSchema).apply()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot change column type: aColumn: list -> string"); } @@ -423,7 +419,7 @@ public void testMirroredSchemas() { Schema union = new SchemaUpdate(aSchema, 0).unionByNameWith(mirrored).apply(); // We don't expect the original schema to have been altered. - Assertions.assertThat(union.asStruct()).isEqualTo(aSchema.asStruct()); + assertThat(union.asStruct()).isEqualTo(aSchema.asStruct()); } @Test @@ -459,7 +455,7 @@ public void addNewTopLevelStruct() { 7, "d1", Types.StructType.of(optional(8, "d2", Types.StringType.get())))))); Schema union = new SchemaUpdate(schema, 5).unionByNameWith(observed).apply(); - Assertions.assertThat(union.asStruct()).isEqualTo(observed.asStruct()); + assertThat(union.asStruct()).isEqualTo(observed.asStruct()); } @Test @@ -510,7 +506,7 @@ public void testAppendNestedStruct() { StringType.get())))))))))))))); Schema applied = new SchemaUpdate(schema, 4).unionByNameWith(observed).apply(); - Assertions.assertThat(applied.asStruct()).isEqualTo(observed.asStruct()); + assertThat(applied.asStruct()).isEqualTo(observed.asStruct()); } @Test @@ -575,6 +571,6 @@ public void testAppendNestedLists() { "list2", ListType.ofOptional(7, StringType.get()))))))))); - Assertions.assertThat(union.asStruct()).isEqualTo(expected.asStruct()); + assertThat(union.asStruct()).isEqualTo(expected.asStruct()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java index cd32b9606d55..277fdf763617 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirementParser.java @@ -18,9 +18,12 @@ */ package org.apache.iceberg; +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.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestUpdateRequirementParser { @@ -33,7 +36,7 @@ public void testUpdateRequirementWithoutRequirementTypeCannotParse() { "{\"uuid\":\"2cc52516-5e73-41f2-b139-545d41a4e151\"}"); for (String json : invalidJson) { - Assertions.assertThatThrownBy(() -> UpdateRequirementParser.fromJson(json)) + assertThatThrownBy(() -> UpdateRequirementParser.fromJson(json)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse update requirement. Missing field: type"); } @@ -53,7 +56,7 @@ public void testAssertUUIDToJson() { String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; String expected = String.format("{\"type\":\"assert-table-uuid\",\"uuid\":\"%s\"}", uuid); UpdateRequirement actual = new UpdateRequirement.AssertTableUUID(uuid); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertTableUUID should convert to the correct JSON value") .isEqualTo(expected); } @@ -72,7 +75,7 @@ public void testAssertViewUUIDToJson() { String uuid = "2cc52516-5e73-41f2-b139-545d41a4e151"; String expected = String.format("{\"type\":\"assert-view-uuid\",\"uuid\":\"%s\"}", uuid); UpdateRequirement actual = new UpdateRequirement.AssertViewUUID(uuid); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertViewUUID should convert to the correct JSON value") .isEqualTo(expected); } @@ -89,7 +92,7 @@ public void testAssertTableDoesNotExistFromJson() { public void testAssertTableDoesNotExistToJson() { String expected = "{\"type\":\"assert-create\"}"; UpdateRequirement actual = new UpdateRequirement.AssertTableDoesNotExist(); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertTableDoesNotExist should convert to the correct JSON value") .isEqualTo(expected); } @@ -130,7 +133,7 @@ public void testAssertRefSnapshotIdFromJson() { "{\"type\":\"%s\",\"ref\":\"%s\",\"snapshot-id\":%d}", requirementType, refName, snapshotId); UpdateRequirement actual = new UpdateRequirement.AssertRefSnapshotID(refName, snapshotId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertRefSnapshotId should convert to the correct JSON value") .isEqualTo(expected); } @@ -145,7 +148,7 @@ public void testAssertRefSnapshotIdFromJsonWithNullSnapshotId() { "{\"type\":\"%s\",\"ref\":\"%s\",\"snapshot-id\":%d}", requirementType, refName, snapshotId); UpdateRequirement actual = new UpdateRequirement.AssertRefSnapshotID(refName, snapshotId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertRefSnapshotId should convert to the correct JSON value") .isEqualTo(expected); } @@ -172,7 +175,7 @@ public void testAssertLastAssignedFieldIdToJson() { "{\"type\":\"%s\",\"last-assigned-field-id\":%d}", requirementType, lastAssignedFieldId); UpdateRequirement actual = new UpdateRequirement.AssertLastAssignedFieldId(lastAssignedFieldId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertLastAssignedFieldId should convert to the correct JSON value") .isEqualTo(expected); } @@ -194,7 +197,7 @@ public void testAssertCurrentSchemaIdToJson() { String expected = String.format("{\"type\":\"%s\",\"current-schema-id\":%d}", requirementType, schemaId); UpdateRequirement actual = new UpdateRequirement.AssertCurrentSchemaID(schemaId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertCurrentSchemaId should convert to the correct JSON value") .isEqualTo(expected); } @@ -222,7 +225,7 @@ public void testAssertLastAssignedPartitionIdToJson() { requirementType, lastAssignedPartitionId); UpdateRequirement actual = new UpdateRequirement.AssertLastAssignedPartitionId(lastAssignedPartitionId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertLastAssignedPartitionId should convert to the correct JSON value") .isEqualTo(expected); } @@ -244,7 +247,7 @@ public void testAssertDefaultSpecIdToJson() { String expected = String.format("{\"type\":\"%s\",\"default-spec-id\":%d}", requirementType, specId); UpdateRequirement actual = new UpdateRequirement.AssertDefaultSpecID(specId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertDefaultSpecId should convert to the correct JSON value") .isEqualTo(expected); } @@ -268,7 +271,7 @@ public void testAssertDefaultSortOrderIdToJson() { String.format( "{\"type\":\"%s\",\"default-sort-order-id\":%d}", requirementType, sortOrderId); UpdateRequirement actual = new UpdateRequirement.AssertDefaultSortOrderID(sortOrderId); - Assertions.assertThat(UpdateRequirementParser.toJson(actual)) + assertThat(UpdateRequirementParser.toJson(actual)) .as("AssertDefaultSortOrderId should convert to the correct JSON value") .isEqualTo(expected); } @@ -322,13 +325,13 @@ public void assertEquals( (UpdateRequirement.AssertDefaultSortOrderID) actual); break; default: - Assertions.fail("Unrecognized update requirement type: " + requirementType); + fail("Unrecognized update requirement type: " + requirementType); } } private static void compareAssertTableUUID( UpdateRequirement.AssertTableUUID expected, UpdateRequirement.AssertTableUUID actual) { - Assertions.assertThat(actual.uuid()) + assertThat(actual.uuid()) .as("UUID from JSON should not be null") .isNotNull() .as("UUID should parse correctly from JSON") @@ -337,7 +340,7 @@ private static void compareAssertTableUUID( private static void compareAssertViewUUID( UpdateRequirement.AssertViewUUID expected, UpdateRequirement.AssertViewUUID actual) { - Assertions.assertThat(actual.uuid()) + assertThat(actual.uuid()) .as("UUID from JSON should not be null") .isNotNull() .as("UUID should parse correctly from JSON") @@ -349,7 +352,7 @@ private static void compareAssertViewUUID( // are the same and as expected. private static void compareAssertTableDoesNotExist( UpdateRequirement expected, UpdateRequirement actual) { - Assertions.assertThat(actual) + assertThat(actual) .isOfAnyClassIn(UpdateRequirement.AssertTableDoesNotExist.class) .hasSameClassAs(expected); } @@ -357,10 +360,10 @@ private static void compareAssertTableDoesNotExist( private static void compareAssertRefSnapshotId( UpdateRequirement.AssertRefSnapshotID expected, UpdateRequirement.AssertRefSnapshotID actual) { - Assertions.assertThat(actual.refName()) + assertThat(actual.refName()) .as("Ref name should parse correctly from JSON") .isEqualTo(expected.refName()); - Assertions.assertThat(actual.snapshotId()) + assertThat(actual.snapshotId()) .as("Snapshot ID should parse correctly from JSON") .isEqualTo(expected.snapshotId()); } @@ -368,7 +371,7 @@ private static void compareAssertRefSnapshotId( private static void compareAssertLastAssignedFieldId( UpdateRequirement.AssertLastAssignedFieldId expected, UpdateRequirement.AssertLastAssignedFieldId actual) { - Assertions.assertThat(actual.lastAssignedFieldId()) + assertThat(actual.lastAssignedFieldId()) .as("Last assigned field id should parse correctly from JSON") .isEqualTo(expected.lastAssignedFieldId()); } @@ -376,7 +379,7 @@ private static void compareAssertLastAssignedFieldId( private static void compareAssertCurrentSchemaId( UpdateRequirement.AssertCurrentSchemaID expected, UpdateRequirement.AssertCurrentSchemaID actual) { - Assertions.assertThat(actual.schemaId()) + assertThat(actual.schemaId()) .as("Current schema id should parse correctly from JSON") .isEqualTo(expected.schemaId()); } @@ -384,7 +387,7 @@ private static void compareAssertCurrentSchemaId( private static void compareAssertLastAssignedPartitionId( UpdateRequirement.AssertLastAssignedPartitionId expected, UpdateRequirement.AssertLastAssignedPartitionId actual) { - Assertions.assertThat(actual.lastAssignedPartitionId()) + assertThat(actual.lastAssignedPartitionId()) .as("Last assigned partition id should parse correctly from JSON") .isEqualTo(expected.lastAssignedPartitionId()); } @@ -392,7 +395,7 @@ private static void compareAssertLastAssignedPartitionId( private static void compareAssertDefaultSpecId( UpdateRequirement.AssertDefaultSpecID expected, UpdateRequirement.AssertDefaultSpecID actual) { - Assertions.assertThat(actual.specId()) + assertThat(actual.specId()) .as("Default spec id should parse correctly from JSON") .isEqualTo(expected.specId()); } @@ -400,7 +403,7 @@ private static void compareAssertDefaultSpecId( private static void compareAssertDefaultSortOrderId( UpdateRequirement.AssertDefaultSortOrderID expected, UpdateRequirement.AssertDefaultSortOrderID actual) { - Assertions.assertThat(actual.sortOrderId()) + assertThat(actual.sortOrderId()) .as("Default sort order id should parse correctly from JSON") .isEqualTo(expected.sortOrderId()); } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java index 2af098445c6f..cabc9f250c13 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroNameMapping.java @@ -19,6 +19,8 @@ package org.apache.iceberg.avro; import static org.apache.avro.generic.GenericData.Record; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -40,7 +42,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @SuppressWarnings("unchecked") @@ -80,10 +81,8 @@ public void testMapProjections() throws IOException { Record projected = writeAndRead(writeSchema, readSchema, record, nameMapping); // field id 5 comes from read schema - Assertions.assertThat(projected.get("location")) - .as("location field should not be read") - .isNull(); - Assertions.assertThat(projected.get("id")).isEqualTo(34L); + assertThat(projected.get("location")).as("location field should not be read").isNull(); + assertThat(projected.get("id")).isEqualTo(34L); // Table mapping partially project `location` map value nameMapping = @@ -102,9 +101,7 @@ public void testMapProjections() throws IOException { projected = writeAndRead(writeSchema, readSchema, record, nameMapping); Record projectedL1 = ((Map) projected.get("location")).get("l1"); - Assertions.assertThat(projectedL1.get("long")) - .as("location.value.long, should not be read") - .isNull(); + assertThat(projectedL1.get("long")).as("location.value.long, should not be read").isNull(); } @Test @@ -174,14 +171,12 @@ public void testComplexMapKeys() throws IOException { Map projectedLocation = (Map) projected.get("location"); Record projectedKey = projectedLocation.keySet().iterator().next(); Record projectedValue = projectedLocation.values().iterator().next(); - Assertions.assertThat( - Comparators.charSequences().compare("k1", (CharSequence) projectedKey.get("k1"))) + assertThat(Comparators.charSequences().compare("k1", (CharSequence) projectedKey.get("k1"))) .isEqualTo(0); - Assertions.assertThat( - Comparators.charSequences().compare("k2", (CharSequence) projectedKey.get("k2"))) + assertThat(Comparators.charSequences().compare("k2", (CharSequence) projectedKey.get("k2"))) .isEqualTo(0); - Assertions.assertThat(projectedValue.get("lat")).isEqualTo(52.995143f); - Assertions.assertThat(projectedValue.get("long")).isNull(); + assertThat(projectedValue.get("lat")).isEqualTo(52.995143f); + assertThat(projectedValue.get("long")).isNull(); } @Test @@ -201,7 +196,7 @@ public void testMissingRequiredFields() { new Schema(Types.NestedField.optional(18, "y", Types.IntegerType.get()))); Schema readSchema = writeSchema; - Assertions.assertThatThrownBy( + assertThatThrownBy( // In this case, pruneColumns result is an empty record () -> writeAndRead(writeSchema, readSchema, record, nameMapping)) .isInstanceOf(IllegalArgumentException.class) @@ -242,8 +237,8 @@ public void testArrayProjections() throws Exception { Schema readSchema = writeSchema; Record projected = writeAndRead(writeSchema, readSchema, record, nameMapping); - Assertions.assertThat(projected.get("point")).as("point is not projected").isNull(); - Assertions.assertThat(projected.get("id")).isEqualTo(34L); + assertThat(projected.get("point")).as("point is not projected").isNull(); + assertThat(projected.get("id")).isEqualTo(34L); // point array is partially projected nameMapping = MappingUtil.create( @@ -259,9 +254,9 @@ public void testArrayProjections() throws Exception { projected = writeAndRead(writeSchema, readSchema, record, nameMapping); Record point = ((List) projected.get("point")).get(0); - Assertions.assertThat(point.get("x")).as("point.x is projected").isEqualTo(1); - Assertions.assertThat(point.get("y")).as("point.y is not projected").isNull(); - Assertions.assertThat(projected.get("id")).isEqualTo(34L); + assertThat(point.get("x")).as("point.x is projected").isEqualTo(1); + assertThat(point.get("y")).as("point.y is not projected").isNull(); + assertThat(projected.get("id")).isEqualTo(34L); } @Test @@ -309,7 +304,7 @@ public void testAliases() throws IOException { Types.NestedField.required(19, "y", Types.IntegerType.get()))))); Record projected = writeAndRead(writeSchema, readSchema, record, nameMapping); - Assertions.assertThat(((List) projected.get("points")).get(0).get("y")) + assertThat(((List) projected.get("points")).get(0).get("y")) .as("x is read as y") .isEqualTo(1); @@ -325,7 +320,7 @@ public void testAliases() throws IOException { Types.NestedField.required(19, "z", Types.IntegerType.get()))))); projected = writeAndRead(writeSchema, readSchema, record, nameMapping); - Assertions.assertThat(((List) projected.get("points")).get(0).get("z")) + assertThat(((List) projected.get("points")).get(0).get("z")) .as("x is read as z") .isEqualTo(1); } @@ -344,7 +339,7 @@ public void testInferredMapping() throws IOException { Schema readSchema = writeSchema; // Pass null for nameMapping so that it is automatically inferred from read schema Record projected = writeAndRead(writeSchema, readSchema, record, null); - Assertions.assertThat(projected).isEqualTo(record); + assertThat(projected).isEqualTo(record); } @Test @@ -364,7 +359,7 @@ protected Record writeAndRead( Record record = super.writeAndRead(desc, writeSchema, readSchema, inputRecord); Record projectedWithNameMapping = writeAndRead(writeSchema, readSchema, inputRecord, MappingUtil.create(writeSchema)); - Assertions.assertThat(projectedWithNameMapping).isEqualTo(record); + assertThat(projectedWithNameMapping).isEqualTo(record); return record; } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java index e3aca9baef8d..9364bd013f90 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestAvroReadProjection.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.avro; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -30,7 +32,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestAvroReadProjection extends TestReadProjection { @@ -72,10 +73,10 @@ public void testAvroArrayAsLogicalMap() throws IOException { GenericData.Record projected = writeAndRead("full_projection", writeSchema, writeSchema, record); - Assertions.assertThat(((Map>) projected.get("map")).get(100L)) + assertThat(((Map>) projected.get("map")).get(100L)) .as("Should contain correct value list") .isEqualTo(values1); - Assertions.assertThat(((Map>) projected.get("map")).get(200L)) + assertThat(((Map>) projected.get("map")).get(200L)) .as("Should contain correct value list") .isEqualTo(values2); } diff --git a/core/src/test/java/org/apache/iceberg/avro/TestBuildAvroProjection.java b/core/src/test/java/org/apache/iceberg/avro/TestBuildAvroProjection.java index 40c04de050db..eaea4394dbfd 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestBuildAvroProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestBuildAvroProjection.java @@ -19,13 +19,13 @@ package org.apache.iceberg.avro; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Collections; import java.util.function.Supplier; import org.apache.avro.SchemaBuilder; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestBuildAvroProjection { @@ -68,11 +68,8 @@ public void projectArrayWithElementSchemaUnchanged() { final org.apache.avro.Schema actual = testSubject.array(expected, supplier); - Assertions.assertThat(actual) - .as("Array projection produced undesired array schema") - .isEqualTo(expected); - Assertions.assertThat( - Integer.valueOf(actual.getProp(AvroSchemaUtil.ELEMENT_ID_PROP)).intValue()) + assertThat(actual).as("Array projection produced undesired array schema").isEqualTo(expected); + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.ELEMENT_ID_PROP)).intValue()) .as("Unexpected element ID discovered on the projected array schema") .isEqualTo(0); } @@ -143,11 +140,8 @@ public void projectArrayWithExtraFieldInElementSchema() { final org.apache.avro.Schema actual = testSubject.array(extraField, supplier); - Assertions.assertThat(actual) - .as("Array projection produced undesired array schema") - .isEqualTo(expected); - Assertions.assertThat( - Integer.valueOf(actual.getProp(AvroSchemaUtil.ELEMENT_ID_PROP)).intValue()) + assertThat(actual).as("Array projection produced undesired array schema").isEqualTo(expected); + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.ELEMENT_ID_PROP)).intValue()) .as("Unexpected element ID discovered on the projected array schema") .isEqualTo(0); } @@ -206,11 +200,8 @@ public void projectArrayWithLessFieldInElementSchema() { final org.apache.avro.Schema actual = testSubject.array(lessField, supplier); - Assertions.assertThat(actual) - .as("Array projection produced undesired array schema") - .isEqualTo(expected); - Assertions.assertThat( - Integer.valueOf(actual.getProp(AvroSchemaUtil.ELEMENT_ID_PROP)).intValue()) + assertThat(actual).as("Array projection produced undesired array schema").isEqualTo(expected); + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.ELEMENT_ID_PROP)).intValue()) .as("Unexpected element ID discovered on the projected array schema") .isEqualTo(0); } @@ -256,13 +247,11 @@ public void projectMapWithValueSchemaUnchanged() { final org.apache.avro.Schema actual = testSubject.map(expected, supplier); - Assertions.assertThat(actual) - .as("Map projection produced undesired map schema") - .isEqualTo(expected); - Assertions.assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.KEY_ID_PROP)).intValue()) + assertThat(actual).as("Map projection produced undesired map schema").isEqualTo(expected); + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.KEY_ID_PROP)).intValue()) .as("Unexpected key ID discovered on the projected map schema") .isEqualTo(0); - Assertions.assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.VALUE_ID_PROP)).intValue()) + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.VALUE_ID_PROP)).intValue()) .as("Unexpected value ID discovered on the projected map schema") .isEqualTo(1); } @@ -337,13 +326,11 @@ public void projectMapWithExtraFieldInValueSchema() { final org.apache.avro.Schema actual = testSubject.map(extraField, supplier); - Assertions.assertThat(actual) - .as("Map projection produced undesired map schema") - .isEqualTo(expected); - Assertions.assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.KEY_ID_PROP)).intValue()) + assertThat(actual).as("Map projection produced undesired map schema").isEqualTo(expected); + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.KEY_ID_PROP)).intValue()) .as("Unexpected key ID discovered on the projected map schema") .isEqualTo(0); - Assertions.assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.VALUE_ID_PROP)).intValue()) + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.VALUE_ID_PROP)).intValue()) .as("Unexpected value ID discovered on the projected map schema") .isEqualTo(1); } @@ -406,13 +393,11 @@ public void projectMapWithLessFieldInValueSchema() { final org.apache.avro.Schema actual = testSubject.map(lessField, supplier); - Assertions.assertThat(actual) - .as("Map projection produced undesired map schema") - .isEqualTo(expected); - Assertions.assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.KEY_ID_PROP)).intValue()) + assertThat(actual).as("Map projection produced undesired map schema").isEqualTo(expected); + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.KEY_ID_PROP)).intValue()) .as("Unexpected key ID discovered on the projected map schema") .isEqualTo(0); - Assertions.assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.VALUE_ID_PROP)).intValue()) + assertThat(Integer.valueOf(actual.getProp(AvroSchemaUtil.VALUE_ID_PROP)).intValue()) .as("Unexpected value ID discovered on the projected map schema") .isEqualTo(1); } 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 331e427861c6..ead17e9f9c42 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadProjection.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.avro; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.within; + import java.io.IOException; import java.nio.file.Path; import java.util.List; @@ -31,7 +35,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -54,11 +57,9 @@ public void testFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, schema, record); - Assertions.assertThat((Long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((Long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("data")); - Assertions.assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } @Test @@ -78,10 +79,10 @@ public void testReorderedFullProjection() throws Exception { Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("full_projection", schema, reordered, record); - Assertions.assertThat(projected.get(0).toString()) + assertThat(projected.get(0).toString()) .as("Should contain the correct 0 value") .isEqualTo("test"); - Assertions.assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); + assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); } @Test @@ -102,11 +103,11 @@ public void testReorderedProjection() throws Exception { Types.NestedField.optional(3, "missing_2", Types.LongType.get())); Record projected = writeAndRead("full_projection", schema, reordered, record); - Assertions.assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); - Assertions.assertThat(projected.get(1).toString()) + assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); + assertThat(projected.get(1).toString()) .as("Should contain the correct 1 value") .isEqualTo("test"); - Assertions.assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); + assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } @Test @@ -122,10 +123,9 @@ public void testEmptyProjection() throws Exception { Record projected = writeAndRead("empty_projection", schema, schema.select(), record); - Assertions.assertThat(projected).as("Should read a non-null record").isNotNull(); + assertThat(projected).as("Should read a non-null record").isNotNull(); // this is expected because there are no values - Assertions.assertThatThrownBy(() -> projected.get(0)) - .isInstanceOf(ArrayIndexOutOfBoundsException.class); + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @Test @@ -143,9 +143,7 @@ public void testBasicProjection() throws Exception { Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); assertEmptyAvroField(projected, "data"); - Assertions.assertThat((Long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((Long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); @@ -153,7 +151,7 @@ public void testBasicProjection() throws Exception { assertEmptyAvroField(projected, "id"); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("data")); - Assertions.assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); + assertThat(cmp).as("Should contain the correct data value").isEqualTo(0); } @Test @@ -174,11 +172,9 @@ public void testRename() throws Exception { Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); - Assertions.assertThat((Long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((Long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); int cmp = Comparators.charSequences().compare("test", (CharSequence) projected.get("renamed")); - Assertions.assertThat(cmp).as("Should contain the correct data/renamed value").isEqualTo(0); + assertThat(cmp).as("Should contain the correct data/renamed value").isEqualTo(0); } @Test @@ -205,9 +201,7 @@ public void testNestedStructProjection() throws Exception { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); assertEmptyAvroField(projected, "location"); - Assertions.assertThat((long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); Schema latOnly = new Schema( @@ -219,11 +213,11 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); Record projectedLocation = (Record) projected.get("location"); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("location")).as("Should project location").isNotNull(); + assertThat(projected.get("location")).as("Should project location").isNotNull(); assertEmptyAvroField(projectedLocation, "long"); - Assertions.assertThat((Float) projectedLocation.get("lat")) + assertThat((Float) projectedLocation.get("lat")) .as("Should project latitude") - .isCloseTo(52.995143f, Assertions.within(0.000001f)); + .isCloseTo(52.995143f, within(0.000001f)); Schema longOnly = new Schema( @@ -235,23 +229,23 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.get("location"); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("location")).as("Should project location").isNotNull(); + assertThat(projected.get("location")).as("Should project location").isNotNull(); assertEmptyAvroField(projectedLocation, "lat"); - Assertions.assertThat((Float) projectedLocation.get("long")) + assertThat((Float) projectedLocation.get("long")) .as("Should project longitude") - .isCloseTo(-1.539054f, Assertions.within(0.000001f)); + .isCloseTo(-1.539054f, within(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.get("location"); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("location")).as("Should project location").isNotNull(); - Assertions.assertThat((Float) projectedLocation.get("lat")) + assertThat(projected.get("location")).as("Should project location").isNotNull(); + assertThat((Float) projectedLocation.get("lat")) .as("Should project latitude") - .isCloseTo(52.995143f, Assertions.within(0.000001f)); - Assertions.assertThat((Float) projectedLocation.get("long")) + .isCloseTo(52.995143f, within(0.000001f)); + assertThat((Float) projectedLocation.get("long")) .as("Should project longitude") - .isCloseTo(-1.539054f, Assertions.within(0.000001f)); + .isCloseTo(-1.539054f, within(0.000001f)); } @Test @@ -273,29 +267,27 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assertions.assertThat((long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); assertEmptyAvroField(projected, "properties"); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(toStringMap((Map) projected.get("properties"))) + assertThat(toStringMap((Map) projected.get("properties"))) .as("Should project entire map") .isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(toStringMap((Map) projected.get("properties"))) + assertThat(toStringMap((Map) projected.get("properties"))) .as("Should project entire map") .isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(toStringMap((Map) projected.get("properties"))) + assertThat(toStringMap((Map) projected.get("properties"))) .as("Should project entire map") .isEqualTo(properties); } @@ -345,57 +337,51 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assertions.assertThat((long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); assertEmptyAvroField(projected, "locations"); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(toStringMap((Map) projected.get("locations"))) + assertThat(toStringMap((Map) projected.get("locations"))) .as("Should project locations map") .isEqualTo(record.get("locations")); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); assertEmptyAvroField(projected, "id"); Map locations = toStringMap((Map) projected.get("locations")); - Assertions.assertThat(locations).as("Should project locations map").isNotNull(); - Assertions.assertThat(locations.keySet()) - .as("Should contain L1 and L2") - .containsExactly("L1", "L2"); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); Record projectedL1 = (Record) locations.get("L1"); - Assertions.assertThat(projectedL1).as("L1 should not be null").isNotNull(); - Assertions.assertThat((float) projectedL1.get("lat")) + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.get("lat")) .as("L1 should contain lat") - .isCloseTo(53.992811f, Assertions.within(0.000001f)); + .isCloseTo(53.992811f, within(0.000001f)); assertEmptyAvroField(projectedL1, "long"); Record projectedL2 = (Record) locations.get("L2"); - Assertions.assertThat(projectedL2).as("L2 should not be null").isNotNull(); - Assertions.assertThat((float) projectedL2.get("lat")) + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.get("lat")) .as("L2 should contain lat") - .isCloseTo(52.995143f, Assertions.within(0.000001f)); + .isCloseTo(52.995143f, within(0.000001f)); assertEmptyAvroField(projectedL2, "y"); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); - Assertions.assertThat(locations).as("Should project locations map").isNotNull(); - Assertions.assertThat(locations.keySet()) - .as("Should contain L1 and L2") - .containsExactly("L1", "L2"); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); projectedL1 = (Record) locations.get("L1"); - Assertions.assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); assertEmptyAvroField(projectedL1, "lat"); - Assertions.assertThat((float) projectedL1.get("long")) + assertThat((float) projectedL1.get("long")) .as("L1 should contain long") - .isCloseTo(-1.542616f, Assertions.within(0.000001f)); + .isCloseTo(-1.542616f, within(0.000001f)); projectedL2 = (Record) locations.get("L2"); - Assertions.assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); assertEmptyAvroField(projectedL2, "lat"); - Assertions.assertThat((float) projectedL2.get("long")) + assertThat((float) projectedL2.get("long")) .as("L2 should contain long") - .isCloseTo(-1.539054f, Assertions.within(0.000001f)); + .isCloseTo(-1.539054f, within(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -412,22 +398,20 @@ public void testMapOfStructsProjection() throws IOException { projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); assertEmptyAvroField(projected, "id"); locations = toStringMap((Map) projected.get("locations")); - Assertions.assertThat(locations).as("Should project locations map").isNotNull(); - Assertions.assertThat(locations.keySet()) - .as("Should contain L1 and L2") - .containsExactly("L1", "L2"); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()).as("Should contain L1 and L2").containsExactly("L1", "L2"); projectedL1 = (Record) locations.get("L1"); - Assertions.assertThat(projectedL1).as("L1 should not be null").isNotNull(); - Assertions.assertThat((float) projectedL1.get("latitude")) + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.get("latitude")) .as("L1 should contain latitude") - .isCloseTo(53.992811f, Assertions.within(0.000001f)); + .isCloseTo(53.992811f, within(0.000001f)); assertEmptyAvroField(projectedL1, "lat"); assertEmptyAvroField(projectedL1, "long"); projectedL2 = (Record) locations.get("L2"); - Assertions.assertThat(projectedL2).as("L2 should not be null").isNotNull(); - Assertions.assertThat((float) projectedL2.get("latitude")) + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.get("latitude")) .as("L2 should contain latitude") - .isCloseTo(52.995143f, Assertions.within(0.000001f)); + .isCloseTo(52.995143f, within(0.000001f)); assertEmptyAvroField(projectedL2, "lat"); assertEmptyAvroField(projectedL2, "long"); } @@ -449,24 +433,18 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assertions.assertThat((long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); assertEmptyAvroField(projected, "values"); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("values")) - .as("Should project entire list") - .isEqualTo(values); + assertThat(projected.get("values")).as("Should project entire list").isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("values")) - .as("Should project entire list") - .isEqualTo(values); + assertThat(projected.get("values")).as("Should project entire list").isEqualTo(values); } @Test @@ -501,40 +479,38 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assertions.assertThat((long) projected.get("id")) - .as("Should contain the correct id value") - .isEqualTo(34L); + assertThat((long) projected.get("id")).as("Should contain the correct id value").isEqualTo(34L); assertEmptyAvroField(projected, "points"); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("points")) + assertThat(projected.get("points")) .as("Should project points list") .isEqualTo(record.get("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("points")).as("Should project points list").isNotNull(); + assertThat(projected.get("points")).as("Should project points list").isNotNull(); List points = (List) projected.get("points"); - Assertions.assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points).as("Should read 2 points").hasSize(2); Record projectedP1 = points.get(0); - Assertions.assertThat((int) projectedP1.get("x")).as("Should project x").isEqualTo(1); + assertThat((int) projectedP1.get("x")).as("Should project x").isEqualTo(1); assertEmptyAvroField(projectedP1, "y"); Record projectedP2 = points.get(1); - Assertions.assertThat((int) projectedP2.get("x")).as("Should project x").isEqualTo(3); + assertThat((int) projectedP2.get("x")).as("Should project x").isEqualTo(3); assertEmptyAvroField(projectedP2, "y"); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("points")).as("Should project points list").isNotNull(); + assertThat(projected.get("points")).as("Should project points list").isNotNull(); points = (List) projected.get("points"); - Assertions.assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points).as("Should read 2 points").hasSize(2); projectedP1 = points.get(0); assertEmptyAvroField(projectedP1, "x"); - Assertions.assertThat((int) projectedP1.get("y")).as("Should project y").isEqualTo(2); + assertThat((int) projectedP1.get("y")).as("Should project y").isEqualTo(2); projectedP2 = points.get(1); assertEmptyAvroField(projectedP2, "x"); - Assertions.assertThat(projectedP2.get("y")).as("Should project null y").isNull(); + assertThat(projectedP2.get("y")).as("Should project null y").isNull(); Schema yRenamed = new Schema( @@ -548,17 +524,17 @@ public void testListOfStructsProjection() throws IOException { projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); assertEmptyAvroField(projected, "id"); - Assertions.assertThat(projected.get("points")).as("Should project points list").isNotNull(); + assertThat(projected.get("points")).as("Should project points list").isNotNull(); points = (List) projected.get("points"); - Assertions.assertThat(points).as("Should read 2 points").hasSize(2); + assertThat(points).as("Should read 2 points").hasSize(2); projectedP1 = points.get(0); assertEmptyAvroField(projectedP1, "x"); assertEmptyAvroField(projectedP1, "y"); - Assertions.assertThat((int) projectedP1.get("z")).as("Should project z").isEqualTo(2); + assertThat((int) projectedP1.get("z")).as("Should project z").isEqualTo(2); projectedP2 = points.get(1); assertEmptyAvroField(projectedP2, "x"); assertEmptyAvroField(projectedP2, "y"); - Assertions.assertThat(projectedP2.get("z")).as("Should project null z").isNull(); + assertThat(projectedP2.get("z")).as("Should project null z").isNull(); } @Test @@ -588,10 +564,8 @@ public void testEmptyStructProjection() throws Exception { assertEmptyAvroField(projected, "id"); Record result = (Record) projected.get("location"); - Assertions.assertThat(projected.get(0)) - .as("location should be in the 0th position") - .isEqualTo(result); - Assertions.assertThat(result).as("Should contain an empty record").isNotNull(); + assertThat(projected.get(0)).as("location should be in the 0th position").isEqualTo(result); + assertThat(result).as("Should contain an empty record").isNotNull(); assertEmptyAvroField(result, "lat"); assertEmptyAvroField(result, "long"); } @@ -621,10 +595,8 @@ public void testEmptyStructRequiredProjection() throws Exception { Record projected = writeAndRead("empty_req_proj", writeSchema, emptyStruct, record); assertEmptyAvroField(projected, "id"); Record result = (Record) projected.get("location"); - Assertions.assertThat(projected.get(0)) - .as("location should be in the 0th position") - .isEqualTo(result); - Assertions.assertThat(result).as("Should contain an empty record").isNotNull(); + assertThat(projected.get(0)).as("location should be in the 0th position").isEqualTo(result); + assertThat(result).as("Should contain an empty record").isNotNull(); assertEmptyAvroField(result, "lat"); assertEmptyAvroField(result, "long"); } @@ -659,19 +631,15 @@ public void testRequiredEmptyStructInRequiredStruct() throws Exception { Types.NestedField.required(4, "empty", Types.StructType.of())))); Record projected = writeAndRead("req_empty_req_proj", writeSchema, emptyStruct, record); - Assertions.assertThat(projected.get("id")).as("Should project id").isEqualTo(34L); + assertThat(projected.get("id")).as("Should project id").isEqualTo(34L); Record result = (Record) projected.get("location"); - Assertions.assertThat(projected.get(1)) - .as("location should be in the 1st position") - .isEqualTo(result); - Assertions.assertThat(result).as("Should contain an empty record").isNotNull(); + assertThat(projected.get(1)).as("location should be in the 1st position").isEqualTo(result); + assertThat(result).as("Should contain an empty record").isNotNull(); assertEmptyAvroField(result, "lat"); assertEmptyAvroField(result, "long"); - Assertions.assertThat(result.getSchema().getField("empty")) - .as("Should project empty") - .isNotNull(); - Assertions.assertThat(result.get("empty")).as("Empty should not be null").isNotNull(); - Assertions.assertThat(((Record) result.get("empty")).getSchema().getFields()) + assertThat(result.getSchema().getField("empty")).as("Should project empty").isNotNull(); + assertThat(result.get("empty")).as("Empty should not be null").isNotNull(); + assertThat(((Record) result.get("empty")).getSchema().getFields()) .as("Empty should be empty") .isEmpty(); } @@ -714,16 +682,12 @@ public void testEmptyNestedStructProjection() throws Exception { Record projected = writeAndRead("nested_empty_proj", writeSchema, emptyStruct, record); assertEmptyAvroField(projected, "id"); Record outerResult = (Record) projected.get("outer"); - Assertions.assertThat(projected.get(0)) - .as("Outer should be in the 0th position") - .isEqualTo(outerResult); - Assertions.assertThat(outerResult).as("Should contain the outer record").isNotNull(); + assertThat(projected.get(0)).as("Outer should be in the 0th position").isEqualTo(outerResult); + assertThat(outerResult).as("Should contain the outer record").isNotNull(); assertEmptyAvroField(outerResult, "lat"); Record innerResult = (Record) outerResult.get("inner"); - Assertions.assertThat(outerResult.get(0)) - .as("Inner should be in the 0th position") - .isEqualTo(innerResult); - Assertions.assertThat(innerResult).as("Should contain the inner record").isNotNull(); + assertThat(outerResult.get(0)).as("Inner should be in the 0th position").isEqualTo(innerResult); + assertThat(innerResult).as("Should contain the inner record").isNotNull(); assertEmptyAvroField(innerResult, "lon"); } @@ -763,21 +727,17 @@ public void testEmptyNestedStructRequiredProjection() throws Exception { Record projected = writeAndRead("nested_empty_req_proj", writeSchema, emptyStruct, record); assertEmptyAvroField(projected, "id"); Record outerResult = (Record) projected.get("outer"); - Assertions.assertThat(projected.get(0)) - .as("Outer should be in the 0th position") - .isEqualTo(outerResult); - Assertions.assertThat(outerResult).as("Should contain the outer record").isNotNull(); + assertThat(projected.get(0)).as("Outer should be in the 0th position").isEqualTo(outerResult); + assertThat(outerResult).as("Should contain the outer record").isNotNull(); assertEmptyAvroField(outerResult, "lat"); Record innerResult = (Record) outerResult.get("inner"); - Assertions.assertThat(outerResult.get(0)) - .as("Inner should be in the 0th position") - .isEqualTo(innerResult); - Assertions.assertThat(innerResult).as("Should contain the inner record").isNotNull(); + assertThat(outerResult.get(0)).as("Inner should be in the 0th position").isEqualTo(innerResult); + assertThat(innerResult).as("Should contain the inner record").isNotNull(); assertEmptyAvroField(innerResult, "lon"); } private void assertEmptyAvroField(GenericRecord record, String field) { - Assertions.assertThatThrownBy(() -> record.get(field)) + assertThatThrownBy(() -> record.get(field)) .isInstanceOf(AvroRuntimeException.class) .hasMessage("Not a valid schema field: " + field); } 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 ff01d80679f1..8c95f2785bcf 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -19,6 +19,9 @@ package org.apache.iceberg.catalog; 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.Assertions.setMaxStackTraceElementsDisplayed; import java.io.IOException; import java.io.UncheckedIOException; @@ -63,7 +66,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; @@ -172,29 +174,29 @@ protected boolean supportsNamesWithDot() { public void testCreateNamespace() { C catalog = catalog(); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); catalog.createNamespace(NS); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .as("Catalog should have the created namespace") .contains(NS); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); } @Test public void testCreateExistingNamespace() { C catalog = catalog(); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); catalog.createNamespace(NS); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); - Assertions.assertThatThrownBy(() -> catalog.createNamespace(NS)) + assertThatThrownBy(() -> catalog.createNamespace(NS)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Namespace already exists"); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should still exist").isTrue(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should still exist").isTrue(); } @Test @@ -203,15 +205,15 @@ public void testCreateNamespaceWithProperties() { C catalog = catalog(); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); Map createProps = ImmutableMap.of("prop", "val"); catalog.createNamespace(NS, createProps); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); Map props = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(Sets.intersection(createProps.entrySet(), props.entrySet())) + assertThat(Sets.intersection(createProps.entrySet(), props.entrySet())) .as("Create properties should be a subset of returned properties") .containsExactlyInAnyOrderElementsOf(createProps.entrySet()); } @@ -220,16 +222,16 @@ public void testCreateNamespaceWithProperties() { public void testLoadNamespaceMetadata() { C catalog = catalog(); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); - Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(NS)) + assertThatThrownBy(() -> catalog.loadNamespaceMetadata(NS)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageStartingWith("Namespace does not exist: newdb"); catalog.createNamespace(NS); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); Map props = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(props).as("Should return non-null property map").isNotNull(); + assertThat(props).as("Should return non-null property map").isNotNull(); // note that there are no requirements for the properties returned by the catalog } @@ -245,7 +247,7 @@ public void testSetNamespaceProperties() { catalog.setProperties(NS, properties); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(actualProperties.entrySet()) + assertThat(actualProperties.entrySet()) .as("Set properties should be a subset of returned properties") .containsAll(properties.entrySet()); } @@ -262,7 +264,7 @@ public void testUpdateNamespaceProperties() { catalog.setProperties(NS, initialProperties); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(actualProperties.entrySet()) + assertThat(actualProperties.entrySet()) .as("Set properties should be a subset of returned properties") .containsAll(initialProperties.entrySet()); @@ -271,7 +273,7 @@ public void testUpdateNamespaceProperties() { catalog.setProperties(NS, updatedProperties); Map finalProperties = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(finalProperties.entrySet()) + assertThat(finalProperties.entrySet()) .as("Updated properties should be a subset of returned properties") .containsAll(updatedProperties.entrySet()); } @@ -288,7 +290,7 @@ public void testUpdateAndSetNamespaceProperties() { catalog.setProperties(NS, initialProperties); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(actualProperties.entrySet()) + assertThat(actualProperties.entrySet()) .as("Set properties should be a subset of returned properties") .containsAll(initialProperties.entrySet()); @@ -298,7 +300,7 @@ public void testUpdateAndSetNamespaceProperties() { catalog.setProperties(NS, updatedProperties); Map finalProperties = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(finalProperties.entrySet()) + assertThat(finalProperties.entrySet()) .as("Updated properties should be a subset of returned properties") .containsAll(updatedProperties.entrySet()); } @@ -309,7 +311,7 @@ public void testSetNamespacePropertiesNamespaceDoesNotExist() { C catalog = catalog(); - Assertions.assertThatThrownBy(() -> catalog.setProperties(NS, ImmutableMap.of("test", "value"))) + assertThatThrownBy(() -> catalog.setProperties(NS, ImmutableMap.of("test", "value"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageStartingWith("Namespace does not exist: newdb"); } @@ -327,10 +329,10 @@ public void testRemoveNamespaceProperties() { catalog.removeProperties(NS, ImmutableSet.of("created-at")); Map actualProperties = catalog.loadNamespaceMetadata(NS); - Assertions.assertThat(actualProperties.containsKey("created-at")) + assertThat(actualProperties.containsKey("created-at")) .as("Should not contain deleted property key") .isFalse(); - Assertions.assertThat(Sets.intersection(properties.entrySet(), actualProperties.entrySet())) + assertThat(Sets.intersection(properties.entrySet(), actualProperties.entrySet())) .as("Expected properties should be a subset of returned properties") .containsExactlyInAnyOrderElementsOf(ImmutableMap.of("owner", "user").entrySet()); } @@ -341,7 +343,7 @@ public void testRemoveNamespacePropertiesNamespaceDoesNotExist() { C catalog = catalog(); - Assertions.assertThatThrownBy(() -> catalog.removeProperties(NS, ImmutableSet.of("a", "b"))) + assertThatThrownBy(() -> catalog.removeProperties(NS, ImmutableSet.of("a", "b"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageStartingWith("Namespace does not exist: newdb"); } @@ -350,21 +352,21 @@ public void testRemoveNamespacePropertiesNamespaceDoesNotExist() { public void testDropNamespace() { C catalog = catalog(); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); catalog.createNamespace(NS); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); - Assertions.assertThat(catalog.dropNamespace(NS)) + assertThat(catalog.namespaceExists(NS)).as("Namespace should exist").isTrue(); + assertThat(catalog.dropNamespace(NS)) .as("Dropping an existing namespace should return true") .isTrue(); - Assertions.assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); + assertThat(catalog.namespaceExists(NS)).as("Namespace should not exist").isFalse(); } @Test public void testDropNonexistentNamespace() { C catalog = catalog(); - Assertions.assertThat(catalog.dropNamespace(NS)) + assertThat(catalog.dropNamespace(NS)) .as("Dropping a nonexistent namespace should return false") .isFalse(); } @@ -379,22 +381,22 @@ public void testListNamespaces() { Namespace ns2 = Namespace.of("newdb_2"); catalog.createNamespace(ns1); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .as("Should include newdb_1") .hasSameElementsAs(concat(starting, ns1)); catalog.createNamespace(ns2); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .as("Should include newdb_1 and newdb_2") .hasSameElementsAs(concat(starting, ns1, ns2)); catalog.dropNamespace(ns1); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .as("Should include newdb_2, not newdb_1") .hasSameElementsAs(concat(starting, ns2)); catalog.dropNamespace(ns2); - Assertions.assertThat(catalog.listNamespaces().containsAll(starting)) + assertThat(catalog.listNamespaces().containsAll(starting)) .as("Should include only starting namespaces") .isTrue(); } @@ -414,35 +416,35 @@ public void testListNestedNamespaces() { Namespace child2 = Namespace.of("parent", "child2"); catalog.createNamespace(parent); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .as("Should include parent") .hasSameElementsAs(concat(starting, parent)); - Assertions.assertThat(catalog.listNamespaces(parent)) + assertThat(catalog.listNamespaces(parent)) .as("Should have no children in newly created parent namespace") .isEmpty(); catalog.createNamespace(child1); - Assertions.assertThat(catalog.listNamespaces(parent)) + assertThat(catalog.listNamespaces(parent)) .as("Should include child1") .hasSameElementsAs(ImmutableList.of(child1)); catalog.createNamespace(child2); - Assertions.assertThat(catalog.listNamespaces(parent)) + assertThat(catalog.listNamespaces(parent)) .as("Should include child1 and child2") .hasSameElementsAs(ImmutableList.of(child1, child2)); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .as("Should not change listing the root") .hasSameElementsAs(concat(starting, parent)); catalog.dropNamespace(child1); - Assertions.assertThat(catalog.listNamespaces(parent)) + assertThat(catalog.listNamespaces(parent)) .as("Should include only child2") .hasSameElementsAs(ImmutableList.of(child2)); catalog.dropNamespace(child2); - Assertions.assertThat(catalog.listNamespaces(parent)).as("Should be empty").isEmpty(); + assertThat(catalog.listNamespaces(parent)).as("Should be empty").isEmpty(); } @Test @@ -453,21 +455,17 @@ public void testNamespaceWithSlash() { Namespace withSlash = Namespace.of("new/db"); - Assertions.assertThat(catalog.namespaceExists(withSlash)) - .as("Namespace should not exist") - .isFalse(); + assertThat(catalog.namespaceExists(withSlash)).as("Namespace should not exist").isFalse(); catalog.createNamespace(withSlash); - Assertions.assertThat(catalog.namespaceExists(withSlash)).as("Namespace should exist").isTrue(); + assertThat(catalog.namespaceExists(withSlash)).as("Namespace should exist").isTrue(); Map properties = catalog.loadNamespaceMetadata(withSlash); - Assertions.assertThat(properties).as("Properties should be accessible").isNotNull(); - Assertions.assertThat(catalog.dropNamespace(withSlash)) + assertThat(properties).as("Properties should be accessible").isNotNull(); + assertThat(catalog.dropNamespace(withSlash)) .as("Dropping the namespace should succeed") .isTrue(); - Assertions.assertThat(catalog.namespaceExists(withSlash)) - .as("Namespace should not exist") - .isFalse(); + assertThat(catalog.namespaceExists(withSlash)).as("Namespace should not exist").isFalse(); } @Test @@ -478,21 +476,15 @@ public void testNamespaceWithDot() { Namespace withDot = Namespace.of("new.db"); - Assertions.assertThat(catalog.namespaceExists(withDot)) - .as("Namespace should not exist") - .isFalse(); + assertThat(catalog.namespaceExists(withDot)).as("Namespace should not exist").isFalse(); catalog.createNamespace(withDot); - Assertions.assertThat(catalog.namespaceExists(withDot)).as("Namespace should exist").isTrue(); + assertThat(catalog.namespaceExists(withDot)).as("Namespace should exist").isTrue(); Map properties = catalog.loadNamespaceMetadata(withDot); - Assertions.assertThat(properties).as("Properties should be accessible").isNotNull(); - Assertions.assertThat(catalog.dropNamespace(withDot)) - .as("Dropping the namespace should succeed") - .isTrue(); - Assertions.assertThat(catalog.namespaceExists(withDot)) - .as("Namespace should not exist") - .isFalse(); + assertThat(properties).as("Properties should be accessible").isNotNull(); + assertThat(catalog.dropNamespace(withDot)).as("Dropping the namespace should succeed").isTrue(); + assertThat(catalog.namespaceExists(withDot)).as("Namespace should not exist").isFalse(); } @Test @@ -501,26 +493,26 @@ public void testBasicCreateTable() { TableIdentifier ident = TableIdentifier.of("ns", "table"); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); if (requiresNamespaceCreate()) { catalog.createNamespace(ident.namespace()); } Table table = catalog.buildTable(ident, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); // validate table settings - Assertions.assertThat(table.name()) + assertThat(table.name()) .as("Table name should report its full name") .isEqualTo(catalog.name() + "." + ident); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Schema should match expected ID assignment") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.location()).as("Should have a location").isNotNull(); - Assertions.assertThat(table.spec().isUnpartitioned()).as("Should be unpartitioned").isTrue(); - Assertions.assertThat(table.sortOrder().isUnsorted()).as("Should be unsorted").isTrue(); - Assertions.assertThat(table.properties()).as("Should have table properties").isNotNull(); + assertThat(table.location()).as("Should have a location").isNotNull(); + assertThat(table.spec().isUnpartitioned()).as("Should be unpartitioned").isTrue(); + assertThat(table.sortOrder().isUnsorted()).as("Should be unsorted").isTrue(); + assertThat(table.properties()).as("Should have table properties").isNotNull(); } @Test @@ -534,19 +526,19 @@ public void testTableNameWithSlash() { catalog.createNamespace(Namespace.of("ns")); } - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); catalog.buildTable(ident, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Table loaded = catalog.loadTable(ident); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Schema should match expected ID assignment") .isEqualTo(TABLE_SCHEMA.asStruct()); catalog.dropTable(ident); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); } @Test @@ -560,19 +552,19 @@ public void testTableNameWithDot() { catalog.createNamespace(Namespace.of("ns")); } - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); catalog.buildTable(ident, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Table loaded = catalog.loadTable(ident); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Schema should match expected ID assignment") .isEqualTo(TABLE_SCHEMA.asStruct()); catalog.dropTable(ident); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); } @Test @@ -585,17 +577,17 @@ public void testBasicCreateTableThatAlreadyExists() { catalog.createNamespace(ident.namespace()); } - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); catalog.buildTable(ident, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); - Assertions.assertThatThrownBy(() -> catalog.buildTable(ident, OTHER_SCHEMA).create()) + assertThatThrownBy(() -> catalog.buildTable(ident, OTHER_SCHEMA).create()) .isInstanceOf(AlreadyExistsException.class) .hasMessageStartingWith("Table already exists: ns.table"); Table table = catalog.loadTable(ident); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Schema should match original table schema") .isEqualTo(TABLE_SCHEMA.asStruct()); } @@ -610,7 +602,7 @@ public void testCompleteCreateTable() { catalog.createNamespace(ident.namespace()); } - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); Map properties = ImmutableMap.of("user", "someone", "created-at", "2022-02-25T00:38:19"); @@ -624,24 +616,22 @@ public void testCompleteCreateTable() { .create(); // validate table settings - Assertions.assertThat(table.name()) + assertThat(table.name()) .as("Table name should report its full name") .isEqualTo(catalog.name() + "." + ident); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); - Assertions.assertThat(table.schema().asStruct()) + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(table.schema().asStruct()) .as("Schema should match expected ID assignment") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.location()).as("Should have a location").isNotNull(); - Assertions.assertThat(table.spec()) - .as("Should use requested partition spec") - .isEqualTo(TABLE_SPEC); - Assertions.assertThat(table.sortOrder()) + assertThat(table.location()).as("Should have a location").isNotNull(); + assertThat(table.spec()).as("Should use requested partition spec").isEqualTo(TABLE_SPEC); + assertThat(table.sortOrder()) .as("Should use requested write order") .isEqualTo(TABLE_WRITE_ORDER); - Assertions.assertThat(table.properties().entrySet()) + assertThat(table.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); - Assertions.assertThat(table.uuid()) + assertThat(table.uuid()) .isEqualTo(UUID.fromString(((BaseTable) table).operations().current().uuid())); } @@ -655,7 +645,7 @@ public void testLoadTable() { catalog.createNamespace(ident.namespace()); } - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); Map properties = ImmutableMap.of("user", "someone", "created-at", "2022-02-25T00:38:19"); @@ -666,25 +656,23 @@ public void testLoadTable() { .withSortOrder(WRITE_ORDER) .withProperties(properties) .create(); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); Table table = catalog.loadTable(ident); // validate table settings - Assertions.assertThat(table.name()) + assertThat(table.name()) .as("Table name should report its full name") .isEqualTo(catalog.name() + "." + ident); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); - Assertions.assertThat(table.schema().asStruct()) + assertThat(catalog.tableExists(ident)).as("Table should exist").isTrue(); + assertThat(table.schema().asStruct()) .as("Schema should match expected ID assignment") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.location()).as("Should have a location").isNotNull(); - Assertions.assertThat(table.spec()) - .as("Should use requested partition spec") - .isEqualTo(TABLE_SPEC); - Assertions.assertThat(table.sortOrder()) + assertThat(table.location()).as("Should have a location").isNotNull(); + assertThat(table.spec()).as("Should use requested partition spec").isEqualTo(TABLE_SPEC); + assertThat(table.sortOrder()) .as("Should use requested write order") .isEqualTo(TABLE_WRITE_ORDER); - Assertions.assertThat(table.properties().entrySet()) + assertThat(table.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); } @@ -703,13 +691,13 @@ public void testLoadMetadataTable() { catalog.buildTable(tableIdent, SCHEMA).create(); Table table = catalog.loadTable(metaIdent); - Assertions.assertThat(table).isNotNull(); - Assertions.assertThat(table).isInstanceOf(FilesTable.class); + assertThat(table).isNotNull(); + assertThat(table).isInstanceOf(FilesTable.class); // check that the table metadata can be refreshed table.refresh(); - Assertions.assertThat(table.name()).isEqualTo(catalog.name() + "." + metaIdent); + assertThat(table.name()).isEqualTo(catalog.name() + "." + metaIdent); } @Test @@ -718,8 +706,8 @@ public void testLoadMissingTable() { TableIdentifier ident = TableIdentifier.of("ns", "table"); - Assertions.assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); - Assertions.assertThatThrownBy(() -> catalog.loadTable(ident)) + assertThat(catalog.tableExists(ident)).as("Table should not exist").isFalse(); + assertThatThrownBy(() -> catalog.loadTable(ident)) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist: ns.table"); } @@ -732,26 +720,20 @@ public void testRenameTable() { catalog.createNamespace(NS); } - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Source table should not exist before create") .isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after create") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after create").isTrue(); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + assertThat(catalog.tableExists(RENAMED_TABLE)) .as("Destination table should not exist before rename") .isFalse(); catalog.renameTable(TABLE, RENAMED_TABLE); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) - .as("Table should exist with new name") - .isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Original table should no longer exist") - .isFalse(); + assertThat(catalog.tableExists(RENAMED_TABLE)).as("Table should exist with new name").isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Original table should no longer exist").isFalse(); catalog.dropTable(RENAMED_TABLE); assertEmpty("Should not contain table after drop", catalog, NS); @@ -765,18 +747,18 @@ public void testRenameTableMissingSourceTable() { catalog.createNamespace(NS); } - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Source table should not exist before rename") .isFalse(); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + assertThat(catalog.tableExists(RENAMED_TABLE)) .as("Destination table should not exist before rename") .isFalse(); - Assertions.assertThatThrownBy(() -> catalog.renameTable(TABLE, RENAMED_TABLE)) + assertThatThrownBy(() -> catalog.renameTable(TABLE, RENAMED_TABLE)) .isInstanceOf(NoSuchTableException.class) .hasMessageContaining("Table does not exist"); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + assertThat(catalog.tableExists(RENAMED_TABLE)) .as("Destination table should not exist after failed rename") .isFalse(); } @@ -790,13 +772,13 @@ public void renameTableNamespaceMissing() { catalog().createNamespace(from.namespace()); } - Assertions.assertThat(catalog().tableExists(from)).as("Table should not exist").isFalse(); + assertThat(catalog().tableExists(from)).as("Table should not exist").isFalse(); catalog().buildTable(from, SCHEMA).create(); - Assertions.assertThat(catalog().tableExists(from)).as("Table should exist").isTrue(); + assertThat(catalog().tableExists(from)).as("Table should exist").isTrue(); - Assertions.assertThatThrownBy(() -> catalog().renameTable(from, to)) + assertThatThrownBy(() -> catalog().renameTable(from, to)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Namespace does not exist: non_existing"); } @@ -809,30 +791,28 @@ public void testRenameTableDestinationTableAlreadyExists() { catalog.createNamespace(NS); } - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Source table should not exist before create") .isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Source table should exist after create") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Source table should exist after create").isTrue(); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + assertThat(catalog.tableExists(RENAMED_TABLE)) .as("Destination table should not exist before create") .isFalse(); catalog.buildTable(RENAMED_TABLE, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + assertThat(catalog.tableExists(RENAMED_TABLE)) .as("Destination table should exist after create") .isTrue(); - Assertions.assertThatThrownBy(() -> catalog.renameTable(TABLE, RENAMED_TABLE)) + assertThatThrownBy(() -> catalog.renameTable(TABLE, RENAMED_TABLE)) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Table already exists"); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Source table should still exist after failed rename") .isTrue(); - Assertions.assertThat(catalog.tableExists(RENAMED_TABLE)) + assertThat(catalog.tableExists(RENAMED_TABLE)) .as("Destination table should still exist after failed rename") .isTrue(); @@ -840,7 +820,7 @@ public void testRenameTableDestinationTableAlreadyExists() { ((HasTableOperations) catalog.loadTable(TABLE)).operations().current().uuid(); String destinationTableUUID = ((HasTableOperations) catalog.loadTable(RENAMED_TABLE)).operations().current().uuid(); - Assertions.assertThat(sourceTableUUID) + assertThat(sourceTableUUID) .as("Source and destination table should remain distinct after failed rename") .isNotEqualTo(destinationTableUUID); } @@ -853,20 +833,14 @@ public void testDropTable() { catalog.createNamespace(NS); } - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should not exist before create") - .isFalse(); + assertThat(catalog.tableExists(TABLE)).as("Table should not exist before create").isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after create") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after create").isTrue(); boolean dropped = catalog.dropTable(TABLE); - Assertions.assertThat(dropped).as("Should drop a table that does exist").isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should not exist after drop") - .isFalse(); + assertThat(dropped).as("Should drop a table that does exist").isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should not exist after drop").isFalse(); } @Test @@ -877,20 +851,14 @@ public void testDropTableWithPurge() { catalog.createNamespace(NS); } - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should not exist before create") - .isFalse(); + assertThat(catalog.tableExists(TABLE)).as("Table should not exist before create").isFalse(); catalog.buildTable(TABLE, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after create") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after create").isTrue(); boolean dropped = catalog.dropTable(TABLE, true); - Assertions.assertThat(dropped).as("Should drop a table that does exist").isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should not exist after drop") - .isFalse(); + assertThat(dropped).as("Should drop a table that does exist").isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should not exist after drop").isFalse(); } @Test @@ -901,24 +869,18 @@ public void testDropTableWithoutPurge() { catalog.createNamespace(NS); } - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should not exist before create") - .isFalse(); + assertThat(catalog.tableExists(TABLE)).as("Table should not exist before create").isFalse(); Table table = catalog.buildTable(TABLE, SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after create") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after create").isTrue(); Set actualMetadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); boolean dropped = catalog.dropTable(TABLE, false); - Assertions.assertThat(dropped).as("Should drop a table that does exist").isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should not exist after drop") - .isFalse(); + assertThat(dropped).as("Should drop a table that does exist").isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should not exist after drop").isFalse(); Set expectedMetadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); - Assertions.assertThat(actualMetadataFileLocations) + assertThat(actualMetadataFileLocations) .hasSameElementsAs(expectedMetadataFileLocations) .hasSize(1) .as("Should have one metadata file"); @@ -933,10 +895,8 @@ public void testDropMissingTable() { } TableIdentifier noSuchTableIdent = TableIdentifier.of(NS, "notable"); - Assertions.assertThat(catalog.tableExists(noSuchTableIdent)) - .as("Table should not exist") - .isFalse(); - Assertions.assertThat(catalog.dropTable(noSuchTableIdent)) + assertThat(catalog.tableExists(noSuchTableIdent)).as("Table should not exist").isFalse(); + assertThat(catalog.dropTable(noSuchTableIdent)) .as("Should not drop a table that does not exist") .isFalse(); } @@ -962,40 +922,40 @@ public void testListTables() { catalog.buildTable(ns1Table1, SCHEMA).create(); - Assertions.assertThat(catalog.listTables(ns1)) + assertThat(catalog.listTables(ns1)) .as("Should contain ns_1.table_1 after create") .containsExactlyInAnyOrder(ns1Table1); catalog.buildTable(ns2Table1, SCHEMA).create(); - Assertions.assertThat(catalog.listTables(ns2)) + assertThat(catalog.listTables(ns2)) .as("Should contain ns_2.table_1 after create") .containsExactlyInAnyOrder(ns2Table1); - Assertions.assertThat(catalog.listTables(ns1)) + assertThat(catalog.listTables(ns1)) .as("Should not show changes to ns_2 in ns_1") .containsExactlyInAnyOrder(ns1Table1); catalog.buildTable(ns1Table2, SCHEMA).create(); - Assertions.assertThat(catalog.listTables(ns2)) + assertThat(catalog.listTables(ns2)) .as("Should not show changes to ns_1 in ns_2") .containsExactlyInAnyOrder(ns2Table1); - Assertions.assertThat(catalog.listTables(ns1)) + assertThat(catalog.listTables(ns1)) .as("Should contain ns_1.table_2 after create") .containsExactlyInAnyOrder(ns1Table1, ns1Table2); catalog.dropTable(ns1Table1); - Assertions.assertThat(catalog.listTables(ns2)) + assertThat(catalog.listTables(ns2)) .as("Should not show changes to ns_1 in ns_2") .containsExactlyInAnyOrder(ns2Table1); - Assertions.assertThat(catalog.listTables(ns1)) + assertThat(catalog.listTables(ns1)) .as("Should not contain ns_1.table_1 after drop") .containsExactlyInAnyOrder(ns1Table2); catalog.dropTable(ns1Table2); - Assertions.assertThat(catalog.listTables(ns2)) + assertThat(catalog.listTables(ns2)) .as("Should not show changes to ns_1 in ns_2") .containsExactlyInAnyOrder(ns2Table1); @@ -1022,7 +982,7 @@ public void testUpdateTableSchema() { Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(expected.asStruct()); } @@ -1038,17 +998,17 @@ public void testUUIDValidation() { Table table = catalog.buildTable(TABLE, SCHEMA).create(); UpdateSchema update = table.updateSchema().addColumn("new_col", Types.LongType.get()); - Assertions.assertThat(catalog.dropTable(TABLE)).as("Should successfully drop table").isTrue(); + assertThat(catalog.dropTable(TABLE)).as("Should successfully drop table").isTrue(); catalog.buildTable(TABLE, OTHER_SCHEMA).create(); String expectedMessage = supportsServerSideRetry() ? "Requirement failed: UUID does not match" : "Cannot commit"; - Assertions.assertThatThrownBy(update::commit) + assertThatThrownBy(update::commit) .isInstanceOf(CommitFailedException.class) .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(OTHER_SCHEMA.asStruct()); } @@ -1076,7 +1036,7 @@ public void testUpdateTableSchemaServerSideRetry() { update.commit(); Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(expected.asStruct()); } @@ -1101,12 +1061,12 @@ public void testUpdateTableSchemaConflict() { // attempt to commit the original update String expectedMessage = supportsServerSideRetry() ? "Requirement failed: current schema changed" : "Cannot commit"; - Assertions.assertThatThrownBy(update::commit) + assertThatThrownBy(update::commit) .isInstanceOf(CommitFailedException.class) .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(expected.asStruct()); } @@ -1134,12 +1094,12 @@ public void testUpdateTableSchemaAssignmentConflict() { supportsServerSideRetry() ? "Requirement failed: last assigned field id changed" : "Cannot commit"; - Assertions.assertThatThrownBy(update::commit) + assertThatThrownBy(update::commit) .isInstanceOf(CommitFailedException.class) .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(expected.asStruct()); } @@ -1163,7 +1123,7 @@ public void testUpdateTableSchemaThenRevert() { table.updateSchema().deleteColumn("col1").deleteColumn("col2").deleteColumn("col3").commit(); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(TABLE_SCHEMA.asStruct()); } @@ -1186,7 +1146,7 @@ public void testUpdateTableSpec() { Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assertions.assertThat(loaded.spec().fields()) + assertThat(loaded.spec().fields()) .as("Loaded table should have expected spec") .isEqualTo(expected.fields()); } @@ -1219,7 +1179,7 @@ public void testUpdateTableSpecServerSideRetry() { Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assertions.assertThat(loaded.spec().fields()) + assertThat(loaded.spec().fields()) .as("Loaded table should have expected spec") .isEqualTo(expected.fields()); } @@ -1248,14 +1208,14 @@ public void testUpdateTableSpecConflict() { supportsServerSideRetry() ? "Requirement failed: default partition spec changed" : "Cannot commit"; - Assertions.assertThatThrownBy(update::commit) + assertThatThrownBy(update::commit) .isInstanceOf(CommitFailedException.class) .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assertions.assertThat(loaded.spec().fields()) + assertThat(loaded.spec().fields()) .as("Loaded table should have expected spec") .isEqualTo(expected.fields()); } @@ -1283,14 +1243,14 @@ public void testUpdateTableAssignmentSpecConflict() { supportsServerSideRetry() ? "Requirement failed: last assigned partition id changed" : "Cannot commit"; - Assertions.assertThatThrownBy(update::commit) + assertThatThrownBy(update::commit) .isInstanceOf(CommitFailedException.class) .hasMessageContaining(expectedMessage); Table loaded = catalog.loadTable(TABLE); // the spec ID may not match, so check equality of the fields - Assertions.assertThat(loaded.spec().fields()) + assertThat(loaded.spec().fields()) .as("Loaded table should have expected spec") .isEqualTo(expected.fields()); } @@ -1311,7 +1271,7 @@ public void testUpdateTableSpecThenRevert() { .withPartitionSpec(SPEC) .withProperty("format-version", "2") .create(); - Assertions.assertThat(((BaseTable) table).operations().current().formatVersion()) + assertThat(((BaseTable) table).operations().current().formatVersion()) .as("Should be a v2 table") .isEqualTo(2); @@ -1319,9 +1279,7 @@ public void testUpdateTableSpecThenRevert() { table.updateSpec().removeField("id").commit(); - Assertions.assertThat(table.spec()) - .as("Loaded table should have expected spec") - .isEqualTo(TABLE_SPEC); + assertThat(table.spec()).as("Loaded table should have expected spec").isEqualTo(TABLE_SPEC); } @Test @@ -1342,7 +1300,7 @@ public void testUpdateTableSortOrder() { Table loaded = catalog.loadTable(TABLE); // the sort order ID may not match, so check equality of the fields - Assertions.assertThat(loaded.sortOrder().fields()) + assertThat(loaded.sortOrder().fields()) .as("Loaded table should have expected order") .isEqualTo(expected.fields()); } @@ -1376,7 +1334,7 @@ public void testUpdateTableSortOrderServerSideRetry() { Table loaded = catalog.loadTable(TABLE); // the sort order ID may not match, so check equality of the fields - Assertions.assertThat(loaded.sortOrder().fields()) + assertThat(loaded.sortOrder().fields()) .as("Loaded table should have expected order") .isEqualTo(expected.fields()); } @@ -1395,7 +1353,7 @@ public void testUpdateTableOrderThenRevert() { table.replaceSortOrder().asc(Expressions.bucket("id", 16)).asc("id").commit(); - Assertions.assertThat(table.sortOrder()) + assertThat(table.sortOrder()) .as("Loaded table should have expected order") .isEqualTo(TABLE_WRITE_ORDER); } @@ -1411,7 +1369,7 @@ public void testAppend() throws IOException { Table table = catalog.buildTable(TABLE, SCHEMA).withPartitionSpec(SPEC).create(); try (CloseableIterable tasks = table.newScan().planFiles()) { - Assertions.assertThat(tasks.iterator().hasNext()).as("Should contain no files").isFalse(); + assertThat(tasks.iterator().hasNext()).as("Should contain no files").isFalse(); } table.newFastAppend().appendFile(FILE_A).commit(); @@ -1498,10 +1456,10 @@ public void testUpdateTransaction() { Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Loaded table should have expected schema") .isEqualTo(expectedSchema.asStruct()); - Assertions.assertThat(loaded.spec().fields()) + assertThat(loaded.spec().fields()) .as("Loaded table should have expected spec") .isEqualTo(expectedSpec.fields()); @@ -1518,21 +1476,19 @@ public void testCreateTransaction() { Transaction create = catalog.buildTable(TABLE, SCHEMA).createTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); create.commitTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); Table table = catalog.loadTable(TABLE); assertFiles(table, FILE_A); assertPreviousMetadataFileCount(table, 0); @@ -1557,37 +1513,35 @@ public void testCompleteCreateTransaction() { .withProperties(properties) .createTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); create.commitTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.spec().fields()) + assertThat(table.spec().fields()) .as("Table should have create partition spec") .isEqualTo(TABLE_SPEC.fields()); - Assertions.assertThat(table.sortOrder()) + assertThat(table.sortOrder()) .as("Table should have create sort order") .isEqualTo(TABLE_WRITE_ORDER); - Assertions.assertThat(table.properties().entrySet()) + assertThat(table.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assertions.assertThat(table.location()) + assertThat(table.location()) .as("Table location should match requested") .isEqualTo("file:/tmp/ns/table"); } @@ -1615,7 +1569,7 @@ public void testCompleteCreateTransactionMultipleSchemas() { .withProperties(properties) .createTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); @@ -1643,15 +1597,13 @@ public void testCompleteCreateTransactionMultipleSchemas() { create.newFastAppend().appendFile(anotherFile).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); create.commitTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); Table table = catalog.loadTable(TABLE); @@ -1663,29 +1615,29 @@ public void testCompleteCreateTransactionMultipleSchemas() { final int updateSpecId = initialSpecId + 1; final int updateOrderId = initialOrderId + 1; - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(newSchema.asStruct()); - Assertions.assertThat(table.schema().schemaId()) + assertThat(table.schema().schemaId()) .as("Table schema should match the new schema ID") .isEqualTo(updateSchemaId); - Assertions.assertThat(table.spec().fields()) + assertThat(table.spec().fields()) .as("Table should have updated partition spec") .isEqualTo(newSpec.fields()); - Assertions.assertThat(table.spec().specId()) + assertThat(table.spec().specId()) .as("Table should have updated partition spec ID") .isEqualTo(updateSpecId); - Assertions.assertThat(table.sortOrder().fields()) + assertThat(table.sortOrder().fields()) .as("Table should have updated sort order") .isEqualTo(newSortOrder.fields()); - Assertions.assertThat(table.sortOrder().orderId()) + assertThat(table.sortOrder().orderId()) .as("Table should have updated sort order ID") .isEqualTo(updateOrderId); - Assertions.assertThat(table.properties().entrySet()) + assertThat(table.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assertions.assertThat(table.location()) + assertThat(table.location()) .as("Table location should match requested") .isEqualTo("file:/tmp/ns/table"); } @@ -1716,44 +1668,42 @@ public void testCompleteCreateTransactionV2() { .withProperties(properties) .createTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); create.commitTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); Table table = catalog.loadTable(TABLE); Map expectedProps = Maps.newHashMap(properties); expectedProps.remove("format-version"); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.spec().fields()) + assertThat(table.spec().fields()) .as("Table should have create partition spec") .isEqualTo(TABLE_SPEC.fields()); - Assertions.assertThat(table.sortOrder()) + assertThat(table.sortOrder()) .as("Table should have create sort order") .isEqualTo(TABLE_WRITE_ORDER); - Assertions.assertThat(Sets.intersection(properties.entrySet(), table.properties().entrySet())) + assertThat(Sets.intersection(properties.entrySet(), table.properties().entrySet())) .as("Table properties should be a superset of the requested properties") .containsExactlyInAnyOrderElementsOf(expectedProps.entrySet()); - Assertions.assertThat(table.currentSnapshot().sequenceNumber()) + assertThat(table.currentSnapshot().sequenceNumber()) .as("Sequence number should start at 1 for v2 format") .isEqualTo(1); if (!overridesRequestedLocation()) { - Assertions.assertThat(table.location()) + assertThat(table.location()) .as("Table location should match requested") .isEqualTo("file:/tmp/ns/table"); } @@ -1773,30 +1723,30 @@ public void testConcurrentCreateTransaction() { Transaction create = catalog.buildTable(TABLE, SCHEMA).createTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assertions.setMaxStackTraceElementsDisplayed(Integer.MAX_VALUE); + setMaxStackTraceElementsDisplayed(Integer.MAX_VALUE); String expectedMessage = supportsServerSideRetry() ? "Requirement failed: table already exists" : "Table already exists"; - Assertions.assertThatThrownBy(create::commitTransaction) + assertThatThrownBy(create::commitTransaction) .isInstanceOf(AlreadyExistsException.class) .hasMessageStartingWith(expectedMessage); // validate the concurrently created table is unmodified Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match concurrent create") .isEqualTo(OTHER_SCHEMA.asStruct()); assertNoFiles(table); @@ -1812,21 +1762,19 @@ public void testCreateOrReplaceTransactionCreate() { Transaction create = catalog.buildTable(TABLE, SCHEMA).createOrReplaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); create.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); create.commitTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); Table table = catalog.loadTable(TABLE); assertFiles(table, FILE_A); @@ -1852,38 +1800,36 @@ public void testCompleteCreateOrReplaceTransactionCreate() { .withProperties(properties) .createOrReplaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); createOrReplace.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); createOrReplace.commitTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.spec().fields()) + assertThat(table.spec().fields()) .as("Table should have create partition spec") .isEqualTo(TABLE_SPEC.fields()); - Assertions.assertThat(table.sortOrder()) + assertThat(table.sortOrder()) .as("Table should have create sort order") .isEqualTo(TABLE_WRITE_ORDER); - Assertions.assertThat(table.properties().entrySet()) + assertThat(table.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assertions.assertThat(table.location()) + assertThat(table.location()) .as("Table location should match requested") .isEqualTo("file:/tmp/ns/table"); } @@ -1903,13 +1849,13 @@ public void testCreateOrReplaceReplaceTransactionReplace() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should exist before replaceTransaction") .isTrue(); Transaction createOrReplace = catalog.buildTable(TABLE, SCHEMA).createOrReplaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should still exist after replaceTransaction") .isTrue(); @@ -1918,7 +1864,7 @@ public void testCreateOrReplaceReplaceTransactionReplace() { // validate table has not changed Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match concurrent create") .isEqualTo(OTHER_SCHEMA.asStruct()); @@ -1928,14 +1874,12 @@ public void testCreateOrReplaceReplaceTransactionReplace() { createOrReplace.commitTransaction(); // validate the table after replace - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); assertUUIDsMatch(original, loaded); @@ -1953,7 +1897,7 @@ public void testCompleteCreateOrReplaceTransactionReplace() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should exist before replaceTransaction") .isTrue(); @@ -1968,7 +1912,7 @@ public void testCompleteCreateOrReplaceTransactionReplace() { .withProperties(properties) .createOrReplaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should still exist after replaceTransaction") .isTrue(); @@ -1976,14 +1920,12 @@ public void testCompleteCreateOrReplaceTransactionReplace() { // validate table has not changed Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match concurrent create") .isEqualTo(OTHER_SCHEMA.asStruct()); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should be unpartitioned") - .isTrue(); - Assertions.assertThat(table.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); - Assertions.assertThat(table.properties().get("created-at")) + assertThat(table.spec().isUnpartitioned()).as("Table should be unpartitioned").isTrue(); + assertThat(table.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); + assertThat(table.properties().get("created-at")) .as("Created at should not match") .isNotEqualTo("2022-02-25T00:38:19"); assertUUIDsMatch(original, table); @@ -1992,27 +1934,25 @@ public void testCompleteCreateOrReplaceTransactionReplace() { createOrReplace.commitTransaction(); // validate the table after replace - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); - Assertions.assertThat(loaded.spec()) + assertThat(loaded.spec()) .as("Table should have replace partition spec") .isEqualTo(REPLACE_SPEC); - Assertions.assertThat(loaded.sortOrder()) + assertThat(loaded.sortOrder()) .as("Table should have replace sort order") .isEqualTo(REPLACE_WRITE_ORDER); - Assertions.assertThat(loaded.properties().entrySet()) + assertThat(loaded.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assertions.assertThat(table.location()) + assertThat(table.location()) .as("Table location should be replaced") .isEqualTo("file:/tmp/ns/table"); } @@ -2036,13 +1976,13 @@ public void testCreateOrReplaceTransactionConcurrentCreate() { Transaction createOrReplace = catalog.buildTable(TABLE, SCHEMA).createOrReplaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after createTransaction") .isFalse(); createOrReplace.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should not exist after append commit") .isFalse(); @@ -2052,13 +1992,13 @@ public void testCreateOrReplaceTransactionConcurrentCreate() { supportsServerSideRetry() ? "Requirement failed: table already exists" : "Table already exists"; - Assertions.assertThatThrownBy(createOrReplace::commitTransaction) + assertThatThrownBy(createOrReplace::commitTransaction) .isInstanceOf(AlreadyExistsException.class) .hasMessageStartingWith(expectedMessage); // validate the concurrently created table is unmodified Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match concurrent create") .isEqualTo(OTHER_SCHEMA.asStruct()); assertNoFiles(table); @@ -2074,13 +2014,13 @@ public void testReplaceTransaction() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should exist before replaceTransaction") .isTrue(); Transaction replace = catalog.buildTable(TABLE, SCHEMA).replaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should still exist after replaceTransaction") .isTrue(); @@ -2088,7 +2028,7 @@ public void testReplaceTransaction() { // validate table has not changed Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match concurrent create") .isEqualTo(OTHER_SCHEMA.asStruct()); assertUUIDsMatch(original, table); @@ -2097,14 +2037,12 @@ public void testReplaceTransaction() { replace.commitTransaction(); // validate the table after replace - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); @@ -2123,7 +2061,7 @@ public void testCompleteReplaceTransaction() { Table original = catalog.buildTable(TABLE, OTHER_SCHEMA).create(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should exist before replaceTransaction") .isTrue(); @@ -2138,7 +2076,7 @@ public void testCompleteReplaceTransaction() { .withProperties(properties) .replaceTransaction(); - Assertions.assertThat(catalog.tableExists(TABLE)) + assertThat(catalog.tableExists(TABLE)) .as("Table should still exist after replaceTransaction") .isTrue(); @@ -2147,14 +2085,12 @@ public void testCompleteReplaceTransaction() { // validate table has not changed Table table = catalog.loadTable(TABLE); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match concurrent create") .isEqualTo(OTHER_SCHEMA.asStruct()); - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table should be unpartitioned") - .isTrue(); - Assertions.assertThat(table.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); - Assertions.assertThat(table.properties().get("created-at")) + assertThat(table.spec().isUnpartitioned()).as("Table should be unpartitioned").isTrue(); + assertThat(table.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); + assertThat(table.properties().get("created-at")) .as("Created at should not match") .isNotEqualTo("2022-02-25T00:38:19"); @@ -2164,27 +2100,25 @@ public void testCompleteReplaceTransaction() { replace.commitTransaction(); // validate the table after replace - Assertions.assertThat(catalog.tableExists(TABLE)) - .as("Table should exist after append commit") - .isTrue(); + assertThat(catalog.tableExists(TABLE)).as("Table should exist after append commit").isTrue(); table.refresh(); // refresh should work with UUID validation Table loaded = catalog.loadTable(TABLE); - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); - Assertions.assertThat(loaded.spec()) + assertThat(loaded.spec()) .as("Table should have replace partition spec") .isEqualTo(REPLACE_SPEC); - Assertions.assertThat(loaded.sortOrder()) + assertThat(loaded.sortOrder()) .as("Table should have replace sort order") .isEqualTo(REPLACE_WRITE_ORDER); - Assertions.assertThat(loaded.properties().entrySet()) + assertThat(loaded.properties().entrySet()) .as("Table properties should be a superset of the requested properties") .containsAll(properties.entrySet()); if (!overridesRequestedLocation()) { - Assertions.assertThat(table.location()) + assertThat(table.location()) .as("Table location should be replaced") .isEqualTo("file:/tmp/ns/table"); } @@ -2202,7 +2136,7 @@ public void testReplaceTransactionRequiresTableExists() { catalog.createNamespace(NS); } - Assertions.assertThatThrownBy(() -> catalog.buildTable(TABLE, SCHEMA).replaceTransaction()) + assertThatThrownBy(() -> catalog.buildTable(TABLE, SCHEMA).replaceTransaction()) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist: newdb.table"); } @@ -2230,30 +2164,26 @@ public void testConcurrentReplaceTransactions() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.schema().asStruct()) + assertThat(afterFirstReplace.schema().asStruct()) .as("Table schema should match the original schema") .isEqualTo(original.schema().asStruct()); - Assertions.assertThat(afterFirstReplace.spec().isUnpartitioned()) + assertThat(afterFirstReplace.spec().isUnpartitioned()) .as("Table should be unpartitioned") .isTrue(); - Assertions.assertThat(afterFirstReplace.sortOrder().isUnsorted()) - .as("Table should be unsorted") - .isTrue(); + assertThat(afterFirstReplace.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.schema().asStruct()) + assertThat(afterSecondReplace.schema().asStruct()) .as("Table schema should match the original schema") .isEqualTo(original.schema().asStruct()); - Assertions.assertThat(afterSecondReplace.spec().isUnpartitioned()) + assertThat(afterSecondReplace.spec().isUnpartitioned()) .as("Table should be unpartitioned") .isTrue(); - Assertions.assertThat(afterSecondReplace.sortOrder().isUnsorted()) - .as("Table should be unsorted") - .isTrue(); + assertThat(afterSecondReplace.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2281,7 +2211,7 @@ public void testConcurrentReplaceTransactionSchema() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.schema().asStruct()) + assertThat(afterFirstReplace.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); assertUUIDsMatch(original, afterFirstReplace); @@ -2290,7 +2220,7 @@ public void testConcurrentReplaceTransactionSchema() { secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.schema().asStruct()) + assertThat(afterSecondReplace.schema().asStruct()) .as("Table schema should match the original schema") .isEqualTo(original.schema().asStruct()); assertUUIDsMatch(original, afterSecondReplace); @@ -2320,7 +2250,7 @@ public void testConcurrentReplaceTransactionSchema2() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.schema().asStruct()) + assertThat(afterFirstReplace.schema().asStruct()) .as("Table schema should match the original schema") .isEqualTo(original.schema().asStruct()); assertUUIDsMatch(original, afterFirstReplace); @@ -2329,7 +2259,7 @@ public void testConcurrentReplaceTransactionSchema2() { secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.schema().asStruct()) + assertThat(afterSecondReplace.schema().asStruct()) .as("Table schema should match the new schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); assertUUIDsMatch(original, afterSecondReplace); @@ -2361,7 +2291,7 @@ public void testConcurrentReplaceTransactionSchemaConflict() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.schema().asStruct()) + assertThat(afterFirstReplace.schema().asStruct()) .as("Table schema should match the original schema") .isEqualTo(REPLACE_SCHEMA.asStruct()); @@ -2370,7 +2300,7 @@ public void testConcurrentReplaceTransactionSchemaConflict() { // even though the new schema is identical, the assertion that the last assigned id has not // changed will fail - Assertions.assertThatThrownBy(secondReplace::commitTransaction) + assertThatThrownBy(secondReplace::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith( "Commit failed: Requirement failed: last assigned field id changed"); @@ -2400,7 +2330,7 @@ public void testConcurrentReplaceTransactionPartitionSpec() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.spec().fields()) + assertThat(afterFirstReplace.spec().fields()) .as("Table spec should match the new spec") .isEqualTo(TABLE_SPEC.fields()); assertUUIDsMatch(original, afterFirstReplace); @@ -2409,7 +2339,7 @@ public void testConcurrentReplaceTransactionPartitionSpec() { secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.spec().isUnpartitioned()) + assertThat(afterSecondReplace.spec().isUnpartitioned()) .as("Table should be unpartitioned") .isTrue(); assertUUIDsMatch(original, afterSecondReplace); @@ -2440,7 +2370,7 @@ public void testConcurrentReplaceTransactionPartitionSpec2() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.spec().isUnpartitioned()) + assertThat(afterFirstReplace.spec().isUnpartitioned()) .as("Table should be unpartitioned") .isTrue(); assertUUIDsMatch(original, afterFirstReplace); @@ -2449,7 +2379,7 @@ public void testConcurrentReplaceTransactionPartitionSpec2() { secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.spec().fields()) + assertThat(afterSecondReplace.spec().fields()) .as("Table spec should match the new spec") .isEqualTo(TABLE_SPEC.fields()); assertUUIDsMatch(original, afterSecondReplace); @@ -2482,7 +2412,7 @@ public void testConcurrentReplaceTransactionPartitionSpecConflict() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.spec().fields()) + assertThat(afterFirstReplace.spec().fields()) .as("Table spec should match the new spec") .isEqualTo(TABLE_SPEC.fields()); assertUUIDsMatch(original, afterFirstReplace); @@ -2490,7 +2420,7 @@ public void testConcurrentReplaceTransactionPartitionSpecConflict() { // even though the new spec is identical, the assertion that the last assigned id has not // changed will fail - Assertions.assertThatThrownBy(secondReplace::commitTransaction) + assertThatThrownBy(secondReplace::commitTransaction) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith( "Commit failed: Requirement failed: last assigned partition id changed"); @@ -2520,7 +2450,7 @@ public void testConcurrentReplaceTransactionSortOrder() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.sortOrder()) + assertThat(afterFirstReplace.sortOrder()) .as("Table order should match the new order") .isEqualTo(TABLE_WRITE_ORDER); assertUUIDsMatch(original, afterFirstReplace); @@ -2529,9 +2459,7 @@ public void testConcurrentReplaceTransactionSortOrder() { secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.sortOrder().isUnsorted()) - .as("Table should be unsorted") - .isTrue(); + assertThat(afterSecondReplace.sortOrder().isUnsorted()).as("Table should be unsorted").isTrue(); assertUUIDsMatch(original, afterSecondReplace); assertFiles(afterSecondReplace, FILE_C); } @@ -2565,16 +2493,14 @@ public void testConcurrentReplaceTransactionSortOrderConflict() { firstReplace.commitTransaction(); Table afterFirstReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterFirstReplace.sortOrder().isSorted()) - .as("Table order should be set") - .isTrue(); + assertThat(afterFirstReplace.sortOrder().isSorted()).as("Table order should be set").isTrue(); assertUUIDsMatch(original, afterFirstReplace); assertFiles(afterFirstReplace, FILE_B); secondReplace.commitTransaction(); Table afterSecondReplace = catalog.loadTable(TABLE); - Assertions.assertThat(afterSecondReplace.sortOrder().fields()) + assertThat(afterSecondReplace.sortOrder().fields()) .as("Table order should match the new order") .isEqualTo(TABLE_WRITE_ORDER.fields()); assertUUIDsMatch(original, afterSecondReplace); @@ -2595,7 +2521,7 @@ public void testMetadataFileLocationsRemovalAfterCommit() { table.updateSchema().addColumn("c", Types.LongType.get()).commit(); Set metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); - Assertions.assertThat(metadataFileLocations).hasSize(4); + assertThat(metadataFileLocations).hasSize(4); int maxPreviousVersionsToKeep = 2; table @@ -2607,14 +2533,14 @@ public void testMetadataFileLocationsRemovalAfterCommit() { .commit(); metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); - Assertions.assertThat(metadataFileLocations).hasSize(maxPreviousVersionsToKeep + 1); + assertThat(metadataFileLocations).hasSize(maxPreviousVersionsToKeep + 1); // for each new commit, the amount of metadata files should stay the same and old files should // be deleted for (int i = 1; i <= 5; i++) { table.updateSchema().addColumn("d" + i, Types.LongType.get()).commit(); metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); - Assertions.assertThat(metadataFileLocations).hasSize(maxPreviousVersionsToKeep + 1); + assertThat(metadataFileLocations).hasSize(maxPreviousVersionsToKeep + 1); } maxPreviousVersionsToKeep = 4; @@ -2630,7 +2556,7 @@ public void testMetadataFileLocationsRemovalAfterCommit() { for (int i = 1; i <= 10; i++) { table.updateSchema().addColumn("e" + i, Types.LongType.get()).commit(); metadataFileLocations = ReachableFileUtil.metadataFileLocations(table, false); - Assertions.assertThat(metadataFileLocations).hasSize(maxPreviousVersionsToKeep + 1); + assertThat(metadataFileLocations).hasSize(maxPreviousVersionsToKeep + 1); } } @@ -2638,7 +2564,7 @@ public void testMetadataFileLocationsRemovalAfterCommit() { public void tableCreationWithoutNamespace() { Assumptions.assumeTrue(requiresNamespaceCreate()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog().buildTable(TableIdentifier.of("non-existing", "table"), SCHEMA).create()) .isInstanceOf(NoSuchNamespaceException.class) @@ -2675,27 +2601,25 @@ public void testRegisterTable() { Table registeredTable = catalog.registerTable(TABLE, metadataLocation); - Assertions.assertThat(registeredTable).isNotNull(); - Assertions.assertThat(catalog.tableExists(TABLE)).as("Table must exist").isTrue(); - Assertions.assertThat(registeredTable.properties()) + assertThat(registeredTable).isNotNull(); + assertThat(catalog.tableExists(TABLE)).as("Table must exist").isTrue(); + assertThat(registeredTable.properties()) .as("Props must match") .containsAllEntriesOf(properties); - Assertions.assertThat(registeredTable.schema().asStruct()) + assertThat(registeredTable.schema().asStruct()) .as("Schema must match") .isEqualTo(originalTable.schema().asStruct()); - Assertions.assertThat(registeredTable.specs()) - .as("Specs must match") - .isEqualTo(originalTable.specs()); - Assertions.assertThat(registeredTable.sortOrders()) + assertThat(registeredTable.specs()).as("Specs must match").isEqualTo(originalTable.specs()); + assertThat(registeredTable.sortOrders()) .as("Sort orders must match") .isEqualTo(originalTable.sortOrders()); - Assertions.assertThat(registeredTable.currentSnapshot()) + assertThat(registeredTable.currentSnapshot()) .as("Current snapshot must match") .isEqualTo(originalTable.currentSnapshot()); - Assertions.assertThat(registeredTable.snapshots()) + assertThat(registeredTable.snapshots()) .as("Snapshots must match") .isEqualTo(originalTable.snapshots()); - Assertions.assertThat(registeredTable.history()) + assertThat(registeredTable.history()) .as("History must match") .isEqualTo(originalTable.history()); @@ -2705,9 +2629,9 @@ public void testRegisterTable() { registeredTable.newFastAppend().appendFile(FILE_A).commit(); assertFiles(registeredTable, FILE_B, FILE_C, FILE_A); - Assertions.assertThat(catalog.loadTable(TABLE)).isNotNull(); - Assertions.assertThat(catalog.dropTable(TABLE)).isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE)).isFalse(); + assertThat(catalog.loadTable(TABLE)).isNotNull(); + assertThat(catalog.dropTable(TABLE)).isTrue(); + assertThat(catalog.tableExists(TABLE)).isFalse(); } @Test @@ -2724,36 +2648,36 @@ public void testRegisterExistingTable() { Table table = catalog.loadTable(identifier); TableOperations ops = ((BaseTable) table).operations(); String metadataLocation = ops.current().metadataFileLocation(); - Assertions.assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) + assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) .isInstanceOf(AlreadyExistsException.class) .hasMessageStartingWith("Table already exists: a.t1"); - Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); + assertThat(catalog.dropTable(identifier)).isTrue(); } private static void assertEmpty(String context, Catalog catalog, Namespace ns) { try { - Assertions.assertThat(catalog.listTables(ns)).as(context).isEmpty(); + assertThat(catalog.listTables(ns)).as(context).isEmpty(); } catch (NoSuchNamespaceException e) { // it is okay if the catalog throws NoSuchNamespaceException when it is empty } } public void assertUUIDsMatch(Table expected, Table actual) { - Assertions.assertThat(((BaseTable) actual).operations().current().uuid()) + assertThat(((BaseTable) actual).operations().current().uuid()) .as("Table UUID should not change") .isEqualTo(((BaseTable) expected).operations().current().uuid()); } public void assertPreviousMetadataFileCount(Table table, int metadataFileCount) { TableOperations ops = ((BaseTable) table).operations(); - Assertions.assertThat(ops.current().previousFiles()) + assertThat(ops.current().previousFiles()) .as("Table should have correct number of previous metadata locations") .hasSize(metadataFileCount); } public void assertNoFiles(Table table) { try (CloseableIterable tasks = table.newScan().planFiles()) { - Assertions.assertThat(tasks).as("Should contain no files").isEmpty(); + assertThat(tasks).as("Should contain no files").isEmpty(); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -2766,10 +2690,8 @@ public void assertFiles(Table table, DataFile... files) { .map(FileScanTask::file) .map(DataFile::path) .collect(Collectors.toList()); - Assertions.assertThat(paths) - .as("Should contain expected number of data files") - .hasSize(files.length); - Assertions.assertThat(CharSequenceSet.of(paths)) + 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))); } catch (IOException e) { @@ -2782,11 +2704,7 @@ public void assertFilePartitionSpec(Table table, DataFile dataFile, int specId) Streams.stream(tasks) .map(FileScanTask::file) .filter(file -> file.path().equals(dataFile.path())) - .forEach( - file -> - Assertions.assertThat(file.specId()) - .as("Spec ID should match") - .isEqualTo(specId)); + .forEach(file -> assertThat(file.specId()).as("Spec ID should match").isEqualTo(specId)); } catch (IOException e) { throw new UncheckedIOException(e); } @@ -2798,7 +2716,7 @@ public void assertFilesPartitionSpec(Table table) { .map(FileScanTask::file) .forEach( file -> - Assertions.assertThat(file.specId()) + assertThat(file.specId()) .as("Spec ID should match") .isEqualTo(table.spec().specId())); } catch (IOException e) { diff --git a/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java b/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java index 587b9395f514..739a87c241b8 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java +++ b/core/src/test/java/org/apache/iceberg/catalog/TestTableIdentifierParser.java @@ -18,7 +18,9 @@ */ package org.apache.iceberg.catalog; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.junit.jupiter.api.Test; public class TestTableIdentifierParser { @@ -27,13 +29,13 @@ public class TestTableIdentifierParser { public void testTableIdentifierToJson() { String json = "{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}"; TableIdentifier identifier = TableIdentifier.of(Namespace.of("accounting", "tax"), "paid"); - Assertions.assertThat(TableIdentifierParser.toJson(identifier)) + assertThat(TableIdentifierParser.toJson(identifier)) .as("Should be able to serialize a table identifier with both namespace and name") .isEqualTo(json); TableIdentifier identifierWithEmptyNamespace = TableIdentifier.of(Namespace.empty(), "paid"); String jsonWithEmptyNamespace = "{\"namespace\":[],\"name\":\"paid\"}"; - Assertions.assertThat(TableIdentifierParser.toJson(identifierWithEmptyNamespace)) + assertThat(TableIdentifierParser.toJson(identifierWithEmptyNamespace)) .as("Should be able to serialize a table identifier that uses the empty namespace") .isEqualTo(jsonWithEmptyNamespace); } @@ -42,18 +44,18 @@ public void testTableIdentifierToJson() { public void testTableIdentifierFromJson() { String json = "{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}"; TableIdentifier identifier = TableIdentifier.of(Namespace.of("accounting", "tax"), "paid"); - Assertions.assertThat(TableIdentifierParser.fromJson(json)) + assertThat(TableIdentifierParser.fromJson(json)) .as("Should be able to deserialize a valid table identifier") .isEqualTo(identifier); TableIdentifier identifierWithEmptyNamespace = TableIdentifier.of(Namespace.empty(), "paid"); String jsonWithEmptyNamespace = "{\"namespace\":[],\"name\":\"paid\"}"; - Assertions.assertThat(TableIdentifierParser.fromJson(jsonWithEmptyNamespace)) + assertThat(TableIdentifierParser.fromJson(jsonWithEmptyNamespace)) .as("Should be able to deserialize a valid multi-level table identifier") .isEqualTo(identifierWithEmptyNamespace); String identifierMissingNamespace = "{\"name\":\"paid\"}"; - Assertions.assertThat(TableIdentifierParser.fromJson(identifierMissingNamespace)) + assertThat(TableIdentifierParser.fromJson(identifierMissingNamespace)) .as( "Should implicitly convert a missing namespace into the the empty namespace when parsing") .isEqualTo(identifierWithEmptyNamespace); @@ -62,22 +64,22 @@ public void testTableIdentifierFromJson() { @Test public void testFailParsingWhenNullOrEmptyJson() { String nullJson = null; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(nullJson)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(nullJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse table identifier from invalid JSON: null"); String emptyString = ""; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(emptyString)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(emptyString)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse table identifier from invalid JSON: ''"); String emptyJson = "{}"; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(emptyJson)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: name"); String emptyJsonArray = "[]"; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(emptyJsonArray)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(emptyJsonArray)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing or non-object table identifier: []"); } @@ -85,7 +87,7 @@ public void testFailParsingWhenNullOrEmptyJson() { @Test public void testFailParsingWhenMissingRequiredFields() { String identifierMissingName = "{\"namespace\":[\"accounting\",\"tax\"]}"; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(identifierMissingName)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(identifierMissingName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: name"); } @@ -93,12 +95,12 @@ public void testFailParsingWhenMissingRequiredFields() { @Test public void testFailWhenFieldsHaveInvalidValues() { String invalidNamespace = "{\"namespace\":\"accounting.tax\",\"name\":\"paid\"}"; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(invalidNamespace)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(invalidNamespace)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: namespace: \"accounting.tax\""); String invalidName = "{\"namespace\":[\"accounting\",\"tax\"],\"name\":1234}"; - Assertions.assertThatThrownBy(() -> TableIdentifierParser.fromJson(invalidName)) + assertThatThrownBy(() -> TableIdentifierParser.fromJson(invalidName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: name: 1234"); } diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java index eedcea4ea2c1..72706de0331e 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java @@ -19,10 +19,10 @@ package org.apache.iceberg.encryption; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.nio.charset.StandardCharsets; import java.security.SecureRandom; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCiphers { @@ -74,7 +74,7 @@ private void testEncryptDecrypt( final byte[] badAad = (aad == null) ? new byte[1] : aad; badAad[0]++; - Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, badAad)) + assertThatThrownBy(() -> decryptor.decrypt(ciphertext, badAad)) .isInstanceOf(RuntimeException.class) .hasMessageContaining("GCM tag check failed"); } @@ -82,7 +82,7 @@ private void testEncryptDecrypt( if (testCorruption) { ciphertext[ciphertext.length / 2]++; - Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, aad)) + assertThatThrownBy(() -> decryptor.decrypt(ciphertext, aad)) .isInstanceOf(RuntimeException.class) .hasMessageContaining("GCM tag check failed"); } diff --git a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java index a0ffe0af05b5..43e2f13b55c9 100644 --- a/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java +++ b/core/src/test/java/org/apache/iceberg/expressions/TestExpressionParser.java @@ -20,6 +20,8 @@ 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 com.fasterxml.jackson.databind.JsonNode; import java.math.BigDecimal; @@ -27,7 +29,6 @@ import java.util.UUID; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestExpressionParser { @@ -101,12 +102,12 @@ public void testSimpleExpressions() { String boundJson = ExpressionParser.toJson(bound, true); String unboundJson = ExpressionParser.toJson(expr, true); - Assertions.assertThat(boundJson) + assertThat(boundJson) .as("Bound and unbound should produce identical json") .isEqualTo(unboundJson); Expression parsed = ExpressionParser.fromJson(boundJson, SCHEMA); - Assertions.assertThat(ExpressionUtil.equivalent(expr, parsed, SUPPORTED_PRIMITIVES, true)) + assertThat(ExpressionUtil.equivalent(expr, parsed, SUPPORTED_PRIMITIVES, true)) .as("Round-trip value should be equivalent") .isTrue(); } @@ -114,45 +115,42 @@ public void testSimpleExpressions() { @Test public void nullExpression() { - Assertions.assertThatThrownBy(() -> ExpressionParser.toJson(null)) + assertThatThrownBy(() -> ExpressionParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid expression: null"); - Assertions.assertThatThrownBy(() -> ExpressionParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> ExpressionParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse expression from null object"); } @Test public void trueExpression() { - Assertions.assertThat(ExpressionParser.toJson(Expressions.alwaysTrue(), true)) - .isEqualTo("true"); - Assertions.assertThat(ExpressionParser.fromJson("true")).isEqualTo(Expressions.alwaysTrue()); + assertThat(ExpressionParser.toJson(Expressions.alwaysTrue(), true)).isEqualTo("true"); + assertThat(ExpressionParser.fromJson("true")).isEqualTo(Expressions.alwaysTrue()); // type=literal is also supported String longJson = "{\n \"type\" : \"literal\",\n \"value\" : true\n}"; - Assertions.assertThat(ExpressionParser.fromJson(longJson)).isEqualTo(Expressions.alwaysTrue()); + assertThat(ExpressionParser.fromJson(longJson)).isEqualTo(Expressions.alwaysTrue()); } @Test public void falseExpression() { - Assertions.assertThat(ExpressionParser.toJson(Expressions.alwaysFalse(), true)) - .isEqualTo("false"); - Assertions.assertThat(ExpressionParser.fromJson("false")).isEqualTo(Expressions.alwaysFalse()); + assertThat(ExpressionParser.toJson(Expressions.alwaysFalse(), true)).isEqualTo("false"); + assertThat(ExpressionParser.fromJson("false")).isEqualTo(Expressions.alwaysFalse()); // type=literal is also supported String longJson = "{\n \"type\" : \"literal\",\n \"value\" : false\n}"; - Assertions.assertThat(ExpressionParser.fromJson(longJson)).isEqualTo(Expressions.alwaysFalse()); + assertThat(ExpressionParser.fromJson(longJson)).isEqualTo(Expressions.alwaysFalse()); } @Test public void eqExpression() { String expected = "{\n" + " \"type\" : \"eq\",\n" + " \"term\" : \"name\",\n" + " \"value\" : 25\n" + "}"; - Assertions.assertThat(ExpressionParser.toJson(Expressions.equal("name", 25), true)) - .isEqualTo(expected); + assertThat(ExpressionParser.toJson(Expressions.equal("name", 25), true)).isEqualTo(expected); Expression expression = ExpressionParser.fromJson(expected); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); } @Test @@ -168,19 +166,18 @@ public void testTransform() { + " \"value\" : 50\n" + "}"; - Assertions.assertThat( + assertThat( ExpressionParser.toJson( Expressions.lessThanOrEqual(Expressions.bucket("id", 100), 50), true)) .isEqualTo(expected); // schema is required to parse transform expressions - Assertions.assertThat( - ExpressionParser.toJson(ExpressionParser.fromJson(expected, SCHEMA), true)) + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected, SCHEMA), true)) .isEqualTo(expected); } @Test public void extraFields() { - Assertions.assertThat( + assertThat( ExpressionParser.toJson( ExpressionParser.fromJson( "{\n" @@ -201,7 +198,7 @@ public void extraFields() { @Test public void invalidTerm() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -218,7 +215,7 @@ public void invalidTerm() { @Test public void invalidValues() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -229,7 +226,7 @@ public void invalidValues() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse NOT_NAN predicate: has invalid value field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -240,7 +237,7 @@ public void invalidValues() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse IS_NAN predicate: has invalid values field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -251,7 +248,7 @@ public void invalidValues() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse LT predicate: missing value"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -263,7 +260,7 @@ public void invalidValues() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse LT predicate: has invalid values field"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -274,7 +271,7 @@ public void invalidValues() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse NOT_IN predicate: missing values"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -289,7 +286,7 @@ public void invalidValues() { @Test public void invalidOperationType() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -303,7 +300,7 @@ public void invalidOperationType() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid operation type: illegal"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson( "{\n" @@ -320,16 +317,16 @@ public void invalidOperationType() { @Test public void invalidAnd() { - Assertions.assertThatThrownBy(() -> ExpressionParser.fromJson("{\n \"type\" : \"and\"\n}")) + assertThatThrownBy(() -> ExpressionParser.fromJson("{\n \"type\" : \"and\"\n}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: left"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson("{\n \"type\" : \"and\",\n \"left\": true}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: right"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ExpressionParser.fromJson("{\n \"type\" : \"and\",\n \"right\": true}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: left"); @@ -344,10 +341,9 @@ public void testPredicate() { + " \"value\" : 50\n" + "}"; - Assertions.assertThat( - ExpressionParser.toJson(Expressions.lessThanOrEqual("column-name", 50), true)) + assertThat(ExpressionParser.toJson(Expressions.lessThanOrEqual("column-name", 50), true)) .isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -370,8 +366,7 @@ public void testPredicateWithObjectLiteral() { + " }\n" + "}"; - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(json), true)) - .isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(json), true)).isEqualTo(expected); } @Test @@ -393,8 +388,7 @@ public void testPredicateWithObjectReference() { + " \"value\" : 50\n" + "}"; - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(json), true)) - .isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(json), true)).isEqualTo(expected); } @Test @@ -419,8 +413,8 @@ public void testAnd() { Expressions.greaterThanOrEqual("column-name-1", 50), Expressions.in("column-name-2", "one", "two")); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -443,8 +437,8 @@ public void testOr() { Expression expression = Expressions.or( Expressions.lessThan("column-name-1", 50), Expressions.notNull("column-name-2")); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -462,8 +456,8 @@ public void testNot() { Expression expression = Expressions.not(Expressions.greaterThanOrEqual("column-name-1", 50)); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -497,8 +491,8 @@ public void testNestedExpression() { Expressions.equal("column-name-2", "test")); Expression expression = Expressions.or(and, Expressions.isNaN("column-name-3")); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -514,8 +508,8 @@ public void testFixedLiteral() { ByteBuffer byteBuffer = ByteBuffer.wrap(new byte[] {1, 2, 3}); Expression expression = Expressions.equal("column-name", byteBuffer); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -530,8 +524,8 @@ public void testDecimalLiteral() { Expression expression = Expressions.in("column-name", new BigDecimal("3.14")); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } @@ -546,8 +540,8 @@ public void testNegativeScaleDecimalLiteral() { Expression expression = Expressions.in("column-name", new BigDecimal("3.14E+4")); - Assertions.assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); - Assertions.assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) + assertThat(ExpressionParser.toJson(expression, true)).isEqualTo(expected); + assertThat(ExpressionParser.toJson(ExpressionParser.fromJson(expected), true)) .isEqualTo(expected); } } 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 0a195819e17c..95d0b6adbd02 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.hadoop; +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.io.UncheckedIOException; @@ -37,7 +40,6 @@ 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.Streams; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -71,14 +73,13 @@ public void testListPrefix() { Path scalePath = new Path(parent, Integer.toString(scale)); createRandomFiles(scalePath, scale); - Assertions.assertThat( + assertThat( Streams.stream(hadoopFileIO.listPrefix(scalePath.toUri().toString())).count()) .isEqualTo((long) scale); }); long totalFiles = scaleSizes.stream().mapToLong(Integer::longValue).sum(); - Assertions.assertThat( - Streams.stream(hadoopFileIO.listPrefix(parent.toUri().toString())).count()) + assertThat(Streams.stream(hadoopFileIO.listPrefix(parent.toUri().toString())).count()) .isEqualTo(totalFiles); } @@ -89,11 +90,9 @@ public void testFileExists() throws IOException { fs.createNewFile(randomFilePath); // check existence of the created file - Assertions.assertThat(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()) - .isTrue(); + assertThat(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()).isTrue(); fs.delete(randomFilePath, false); - Assertions.assertThat(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()) - .isFalse(); + assertThat(hadoopFileIO.newInputFile(randomFilePath.toUri().toString()).exists()).isFalse(); } @Test @@ -112,7 +111,7 @@ public void testDeletePrefix() { hadoopFileIO.deletePrefix(scalePath.toUri().toString()); // Hadoop filesystem will throw if the path does not exist - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> hadoopFileIO.listPrefix(scalePath.toUri().toString()).iterator()) .isInstanceOf(UncheckedIOException.class) .hasMessageContaining("java.io.FileNotFoundException"); @@ -120,8 +119,7 @@ public void testDeletePrefix() { hadoopFileIO.deletePrefix(parent.toUri().toString()); // Hadoop filesystem will throw if the path does not exist - Assertions.assertThatThrownBy( - () -> hadoopFileIO.listPrefix(parent.toUri().toString()).iterator()) + assertThatThrownBy(() -> hadoopFileIO.listPrefix(parent.toUri().toString()).iterator()) .isInstanceOf(UncheckedIOException.class) .hasMessageContaining("java.io.FileNotFoundException"); } @@ -133,15 +131,14 @@ public void testDeleteFiles() { hadoopFileIO.deleteFiles( filesCreated.stream().map(Path::toString).collect(Collectors.toList())); filesCreated.forEach( - file -> - Assertions.assertThat(hadoopFileIO.newInputFile(file.toString()).exists()).isFalse()); + file -> assertThat(hadoopFileIO.newInputFile(file.toString()).exists()).isFalse()); } @Test public void testDeleteFilesErrorHandling() { List filesCreated = random.ints(2).mapToObj(x -> "fakefsnotreal://file-" + x).collect(Collectors.toList()); - Assertions.assertThatThrownBy(() -> hadoopFileIO.deleteFiles(filesCreated)) + assertThatThrownBy(() -> hadoopFileIO.deleteFiles(filesCreated)) .isInstanceOf(BulkDeletionFailureException.class) .hasMessage("Failed to delete 2 files"); } @@ -154,8 +151,7 @@ public void testHadoopFileIOKryoSerialization() throws IOException { testHadoopFileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.KryoHelpers.roundTripSerialize(testHadoopFileIO); - Assertions.assertThat(roundTripSerializedFileIO.properties()) - .isEqualTo(testHadoopFileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testHadoopFileIO.properties()); } @Test @@ -166,8 +162,7 @@ public void testHadoopFileIOJavaSerialization() throws IOException, ClassNotFoun testHadoopFileIO.initialize(ImmutableMap.of("k1", "v1")); FileIO roundTripSerializedFileIO = TestHelpers.roundTripSerialize(testHadoopFileIO); - Assertions.assertThat(roundTripSerializedFileIO.properties()) - .isEqualTo(testHadoopFileIO.properties()); + assertThat(roundTripSerializedFileIO.properties()).isEqualTo(testHadoopFileIO.properties()); } @Test @@ -180,7 +175,7 @@ public void testResolvingFileIOLoad() { .hiddenImpl(ResolvingFileIO.class, String.class) .build(resolvingFileIO) .invoke("hdfs://foo/bar"); - Assertions.assertThat(result).isInstanceOf(HadoopFileIO.class); + assertThat(result).isInstanceOf(HadoopFileIO.class); } private List createRandomFiles(Path parent, int count) { diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java b/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java index 31b41434ec69..cb7ca641ea05 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCachingCatalog.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.hadoop; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.github.benmanes.caffeine.cache.Cache; import java.io.IOException; import java.time.Duration; @@ -41,7 +44,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.FakeTicker; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -86,15 +88,15 @@ public void testInvalidateMetadataTablesIfBaseTableIsModified() throws Exception Table manifestsMetaTable2 = catalog.loadTable(manifestsMetaTableIdent); // metadata tables are cached - Assertions.assertThat(filesMetaTable2).isEqualTo(filesMetaTable); - Assertions.assertThat(manifestsMetaTable2).isEqualTo(manifestsMetaTable); + assertThat(filesMetaTable2).isEqualTo(filesMetaTable); + assertThat(manifestsMetaTable2).isEqualTo(manifestsMetaTable); // the current snapshot of origin table is updated after committing - Assertions.assertThat(table.currentSnapshot()).isNotEqualTo(oldSnapshot); + assertThat(table.currentSnapshot()).isNotEqualTo(oldSnapshot); // underlying table operation in metadata tables are shared with the origin table - Assertions.assertThat(filesMetaTable2.currentSnapshot()).isEqualTo(table.currentSnapshot()); - Assertions.assertThat(manifestsMetaTable2.currentSnapshot()).isEqualTo(table.currentSnapshot()); + assertThat(filesMetaTable2.currentSnapshot()).isEqualTo(table.currentSnapshot()); + assertThat(manifestsMetaTable2.currentSnapshot()).isEqualTo(table.currentSnapshot()); } @Test @@ -128,20 +130,20 @@ public void testInvalidateMetadataTablesIfBaseTableIsDropped() throws IOExceptio // remember the new snapshot Snapshot newSnapshot = table.currentSnapshot(); - Assertions.assertThat(newSnapshot).as("Snapshots must be different").isNotEqualTo(oldSnapshot); + assertThat(newSnapshot).as("Snapshots must be different").isNotEqualTo(oldSnapshot); // validate metadata tables were correctly invalidated for (MetadataTableType type : MetadataTableType.values()) { TableIdentifier metadataIdent1 = TableIdentifier.parse(tableIdent + "." + type.name()); Table metadataTable1 = catalog.loadTable(metadataIdent1); - Assertions.assertThat(metadataTable1.currentSnapshot()) + assertThat(metadataTable1.currentSnapshot()) .as("Snapshot must be new") .isEqualTo(newSnapshot); TableIdentifier metadataIdent2 = TableIdentifier.parse(tableIdent + "." + type.name().toLowerCase(Locale.ROOT)); Table metadataTable2 = catalog.loadTable(metadataIdent2); - Assertions.assertThat(metadataTable2.currentSnapshot()) + assertThat(metadataTable2.currentSnapshot()) .as("Snapshot must be new") .isEqualTo(newSnapshot); } @@ -154,12 +156,12 @@ public void testTableName() throws Exception { catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key2", "value2")); Table table = catalog.loadTable(tableIdent); - Assertions.assertThat(table.name()).as("Name must match").isEqualTo("hadoop.db.ns1.ns2.tbl"); + assertThat(table.name()).as("Name must match").isEqualTo("hadoop.db.ns1.ns2.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assertions.assertThat(snapshotsTable.name()) + assertThat(snapshotsTable.name()) .as("Name must match") .isEqualTo("hadoop.db.ns1.ns2.tbl.snapshots"); } @@ -174,22 +176,16 @@ public void testTableExpiresAfterInterval() throws IOException { catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value")); // Ensure table is cached with full ttl remaining upon creation - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) - .isPresent() - .get() - .isEqualTo(EXPIRATION_TTL); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.remainingAgeFor(tableIdent)).isPresent().get().isEqualTo(EXPIRATION_TTL); ticker.advance(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)) - .isPresent() - .get() - .isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.ageOf(tableIdent)).isPresent().get().isEqualTo(HALF_OF_EXPIRATION); ticker.advance(HALF_OF_EXPIRATION.plus(Duration.ofSeconds(10))); - Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); - Assertions.assertThat(catalog.loadTable(tableIdent)) + assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); + assertThat(catalog.loadTable(tableIdent)) .as("CachingCatalog should return a new instance after expiration") .isNotSameAs(table); } @@ -202,48 +198,42 @@ public void testCatalogExpirationTtlRefreshesAfterAccessViaCatalog() throws IOEx TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl"); catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key", "value")); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)).isPresent().get().isEqualTo(Duration.ZERO); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.ageOf(tableIdent)).isPresent().get().isEqualTo(Duration.ZERO); ticker.advance(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)) - .isPresent() - .get() - .isEqualTo(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) - .isPresent() - .get() - .isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.ageOf(tableIdent)).isPresent().get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.remainingAgeFor(tableIdent)).isPresent().get().isEqualTo(HALF_OF_EXPIRATION); Duration oneMinute = Duration.ofMinutes(1L); ticker.advance(oneMinute); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)) + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.ageOf(tableIdent)) .isPresent() .get() .isEqualTo(HALF_OF_EXPIRATION.plus(oneMinute)); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) + assertThat(catalog.remainingAgeFor(tableIdent)) .get() .isEqualTo(HALF_OF_EXPIRATION.minus(oneMinute)); // Access the table via the catalog, which should refresh the TTL Table table = catalog.loadTable(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(Duration.ZERO); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(EXPIRATION_TTL); + assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(Duration.ZERO); + assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(EXPIRATION_TTL); ticker.advance(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); // Check that accessing the table object directly does not affect the cache TTL table.refresh(); - Assertions.assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); table.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.remainingAgeFor(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); } @Test @@ -253,46 +243,46 @@ public void testCacheExpirationEagerlyRemovesMetadataTables() throws IOException Namespace namespace = Namespace.of("db", "ns1", "ns2"); TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl"); Table table = catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key2", "value2")); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); table.newAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(Duration.ZERO); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(Duration.ZERO); ticker.advance(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); - Assertions.assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.ageOf(tableIdent)).get().isEqualTo(HALF_OF_EXPIRATION); // Load the metadata tables for the first time. Their age should be zero as they're new entries. Arrays.stream(metadataTables(tableIdent)).forEach(catalog::loadTable); - Assertions.assertThat(catalog.cache().asMap()).containsKeys(metadataTables(tableIdent)); - Assertions.assertThat(Arrays.stream(metadataTables(tableIdent)).map(catalog::ageOf)) + assertThat(catalog.cache().asMap()).containsKeys(metadataTables(tableIdent)); + assertThat(Arrays.stream(metadataTables(tableIdent)).map(catalog::ageOf)) .isNotEmpty() .allMatch(age -> age.isPresent() && age.get().equals(Duration.ZERO)); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) + assertThat(catalog.remainingAgeFor(tableIdent)) .as("Loading a non-cached metadata table should refresh the main table's age") .isEqualTo(Optional.of(EXPIRATION_TTL)); // Move time forward and access already cached metadata tables. ticker.advance(HALF_OF_EXPIRATION); Arrays.stream(metadataTables(tableIdent)).forEach(catalog::loadTable); - Assertions.assertThat(Arrays.stream(metadataTables(tableIdent)).map(catalog::ageOf)) + assertThat(Arrays.stream(metadataTables(tableIdent)).map(catalog::ageOf)) .isNotEmpty() .allMatch(age -> age.isPresent() && age.get().equals(Duration.ZERO)); - Assertions.assertThat(catalog.remainingAgeFor(tableIdent)) + assertThat(catalog.remainingAgeFor(tableIdent)) .as("Accessing a cached metadata table should not affect the main table's age") .isEqualTo(Optional.of(HALF_OF_EXPIRATION)); // Move time forward so the data table drops. ticker.advance(HALF_OF_EXPIRATION); - Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); + assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); Arrays.stream(metadataTables(tableIdent)) .forEach( metadataTable -> - Assertions.assertThat(catalog.cache().asMap()) + assertThat(catalog.cache().asMap()) .as( "When a data table expires, its metadata tables should expire regardless of age") .doesNotContainKeys(metadataTable)); @@ -335,8 +325,8 @@ public void testDeadlock() throws IOException, InterruptedException { } } executor.awaitTermination(2, TimeUnit.SECONDS); - Assertions.assertThat(cacheGetCount).hasValue(numThreads / 2); - Assertions.assertThat(cacheCleanupCount).hasValue(numThreads / 2); + assertThat(cacheGetCount).hasValue(numThreads / 2); + assertThat(cacheCleanupCount).hasValue(numThreads / 2); executor.shutdown(); createdTables.forEach(table -> catalog.dropTable(table, true)); @@ -344,8 +334,7 @@ public void testDeadlock() throws IOException, InterruptedException { @Test public void testCachingCatalogRejectsExpirationIntervalOfZero() { - Assertions.assertThatThrownBy( - () -> TestableCachingCatalog.wrap(hadoopCatalog(), Duration.ZERO, ticker)) + assertThatThrownBy(() -> TestableCachingCatalog.wrap(hadoopCatalog(), Duration.ZERO, ticker)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "When cache.expiration-interval-ms is set to 0, the catalog cache should be disabled. This indicates a bug."); @@ -359,7 +348,7 @@ public void testCacheExpirationIsDisabledByANegativeValue() throws IOException { Duration.ofMillis(CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF), ticker); - Assertions.assertThat(catalog.isCacheExpirationEnabled()) + assertThat(catalog.isCacheExpirationEnabled()) .as( "When a negative value is used as the expiration interval, the cache should not expire entries based on a TTL") .isFalse(); @@ -374,10 +363,10 @@ public void testInvalidateTableForChainedCachingCatalogs() throws Exception { Namespace namespace = Namespace.of("db", "ns1", "ns2"); TableIdentifier tableIdent = TableIdentifier.of(namespace, "tbl"); catalog.createTable(tableIdent, SCHEMA, SPEC, ImmutableMap.of("key2", "value2")); - Assertions.assertThat(catalog.cache().asMap()).containsKey(tableIdent); + assertThat(catalog.cache().asMap()).containsKey(tableIdent); catalog.invalidateTable(tableIdent); - Assertions.assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); - Assertions.assertThat(wrappedCatalog.cache().asMap()).doesNotContainKey(tableIdent); + assertThat(catalog.cache().asMap()).doesNotContainKey(tableIdent); + assertThat(wrappedCatalog.cache().asMap()).doesNotContainKey(tableIdent); } public static TableIdentifier[] metadataTables(TableIdentifier tableIdent) { 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 e3d2b4d8db92..79f30e109f7d 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestCatalogUtilDropTable.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.hadoop; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.io.UncheckedIOException; import java.nio.ByteBuffer; @@ -46,7 +48,6 @@ import org.apache.iceberg.puffin.PuffinWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.mockito.ArgumentCaptor; import org.mockito.ArgumentMatchers; @@ -83,12 +84,12 @@ public void dropTableDataDeletesExpectedFiles() throws IOException { Set statsLocations = statsLocations(tableMetadata); Set partitionStatsLocations = partitionStatsLocations(tableMetadata); - Assertions.assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); - Assertions.assertThat(metadataLocations).as("should have 5 metadata locations").hasSize(5); - Assertions.assertThat(statsLocations) + assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); + assertThat(metadataLocations).as("should have 5 metadata locations").hasSize(5); + assertThat(statsLocations) .as("should have 1 stats file") .containsExactly(statisticsFile.path()); - Assertions.assertThat(partitionStatsLocations) + assertThat(partitionStatsLocations) .as("should have 1 partition stats file") .containsExactly(partitionStatisticsFile.path()); @@ -109,22 +110,20 @@ public void dropTableDataDeletesExpectedFiles() throws IOException { .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created manifest lists") .containsAll(manifestListLocations); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created manifests") .containsAll(manifestLocations); - Assertions.assertThat(deletedPaths) - .as("should contain all created data") - .containsAll(dataLocations); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths).as("should contain all created data").containsAll(dataLocations); + assertThat(deletedPaths) .as("should contain all created metadata locations") .containsAll(metadataLocations); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created statistics") .containsAll(statsLocations); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created partition stats files") .containsAll(partitionStatsLocations); } @@ -162,8 +161,8 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { Set manifestListLocations = manifestListLocations(snapshotSet); Set manifestLocations = manifestLocations(snapshotSet, table.io()); Set metadataLocations = metadataLocations(tableMetadata); - Assertions.assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); - Assertions.assertThat(metadataLocations).as("should have 4 metadata locations").hasSize(4); + assertThat(manifestListLocations).as("should have 2 manifest lists").hasSize(2); + assertThat(metadataLocations).as("should have 4 metadata locations").hasSize(4); FileIO fileIO = createMockFileIO(table.io()); @@ -177,13 +176,13 @@ public void shouldNotDropDataFilesIfGcNotEnabled() { .deleteFile(argumentCaptor.capture()); List deletedPaths = argumentCaptor.getAllValues(); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created manifest lists") .containsAll(manifestListLocations); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created manifests") .containsAll(manifestLocations); - Assertions.assertThat(deletedPaths) + assertThat(deletedPaths) .as("should contain all created metadata locations") .containsAll(metadataLocations); } 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 8365129a5615..fbd6f833499e 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java @@ -20,6 +20,8 @@ import static org.apache.iceberg.NullOrder.NULLS_FIRST; import static org.apache.iceberg.SortDirection.ASC; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.nio.charset.StandardCharsets; @@ -51,7 +53,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.transforms.Transforms; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -73,11 +74,9 @@ public void testCreateTableBuilder(int formatVersion) throws Exception { .withProperties(ImmutableMap.of("key2", "value2")) .create(); - Assertions.assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); - Assertions.assertThat(table.spec().fields()).hasSize(1); - Assertions.assertThat(table.properties()) - .containsEntry("key1", "value1") - .containsEntry("key2", "value2"); + assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); + assertThat(table.spec().fields()).hasSize(1); + assertThat(table.properties()).containsEntry("key1", "value1").containsEntry("key2", "value2"); } @ParameterizedTest @@ -94,8 +93,8 @@ public void testCreateTableTxnBuilder(int formatVersion) throws Exception { txn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assertions.assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); - Assertions.assertThat(table.spec().isUnpartitioned()).isTrue(); + assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); + assertThat(table.spec().isUnpartitioned()).isTrue(); } @ParameterizedTest @@ -117,14 +116,14 @@ public void testReplaceTxnBuilder(int formatVersion) throws Exception { createTxn.commitTransaction(); Table table = catalog.loadTable(tableIdent); - Assertions.assertThat(table.currentSnapshot()).isNotNull(); + assertThat(table.currentSnapshot()).isNotNull(); Transaction replaceTxn = catalog.buildTable(tableIdent, SCHEMA).withProperty("key2", "value2").replaceTransaction(); replaceTxn.commitTransaction(); table = catalog.loadTable(tableIdent); - Assertions.assertThat(table.currentSnapshot()).isNull(); + assertThat(table.currentSnapshot()).isNull(); if (formatVersion == 1) { PartitionSpec v1Expected = @@ -132,18 +131,14 @@ public void testReplaceTxnBuilder(int formatVersion) throws Exception { .alwaysNull("data", "data_bucket") .withSpecId(1) .build(); - Assertions.assertThat(table.spec()) + assertThat(table.spec()) .as("Table should have a spec with one void field") .isEqualTo(v1Expected); } else { - Assertions.assertThat(table.spec().isUnpartitioned()) - .as("Table spec should be unpartitioned") - .isTrue(); + assertThat(table.spec().isUnpartitioned()).as("Table spec should be unpartitioned").isTrue(); } - Assertions.assertThat(table.properties()) - .containsEntry("key1", "value1") - .containsEntry("key2", "value2"); + assertThat(table.properties()).containsEntry("key1", "value1").containsEntry("key2", "value2"); } @Test @@ -151,17 +146,16 @@ public void testTableBuilderWithLocation() throws Exception { HadoopCatalog catalog = hadoopCatalog(); TableIdentifier tableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl"); - Assertions.assertThatThrownBy( - () -> catalog.buildTable(tableIdent, SCHEMA).withLocation("custom").create()) + assertThatThrownBy(() -> catalog.buildTable(tableIdent, SCHEMA).withLocation("custom").create()) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot set a custom location for a path-based table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.buildTable(tableIdent, SCHEMA).withLocation("custom").createTransaction()) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot set a custom location for a path-based table"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog .buildTable(tableIdent, SCHEMA) @@ -177,8 +171,8 @@ public void testCreateTableDefaultSortOrder() throws Exception { Table table = hadoopCatalog().createTable(tableIdent, SCHEMA, SPEC); SortOrder sortOrder = table.sortOrder(); - Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(0); - Assertions.assertThat(sortOrder.isUnsorted()).as("Order must be unsorted").isTrue(); + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(0); + assertThat(sortOrder.isUnsorted()).as("Order must be unsorted").isTrue(); } @Test @@ -193,16 +187,14 @@ public void testCreateTableCustomSortOrder() throws Exception { .create(); SortOrder sortOrder = table.sortOrder(); - Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); - Assertions.assertThat(sortOrder.fields().size()).as("Order must have 1 field").isEqualTo(1); - Assertions.assertThat(sortOrder.fields().get(0).direction()) - .as("Direction must match") - .isEqualTo(ASC); - Assertions.assertThat(sortOrder.fields().get(0).nullOrder()) + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + assertThat(sortOrder.fields().size()).as("Order must have 1 field").isEqualTo(1); + assertThat(sortOrder.fields().get(0).direction()).as("Direction must match").isEqualTo(ASC); + assertThat(sortOrder.fields().get(0).nullOrder()) .as("Null order must match") .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(); - Assertions.assertThat(sortOrder.fields().get(0).transform()) + assertThat(sortOrder.fields().get(0).transform()) .as("Transform must match") .isEqualTo(transform); } @@ -215,10 +207,10 @@ public void testBasicCatalog() throws Exception { String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); + assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -234,8 +226,8 @@ public void testHadoopFileIOProperties() { catalog.initialize("hadoop", catalogProps); FileIO fileIO = catalog.newTableOps(tableIdent).io(); - Assertions.assertThat(fileIO.properties()).containsEntry("warehouse", "/hive/testwarehouse"); - Assertions.assertThat(fileIO.properties()).containsEntry("io.manifest.cache-enabled", "true"); + assertThat(fileIO.properties()).containsEntry("warehouse", "/hive/testwarehouse"); + assertThat(fileIO.properties()).containsEntry("io.manifest.cache-enabled", "true"); } @Test @@ -245,15 +237,15 @@ public void testCreateAndDropTableWithoutNamespace() throws Exception { TableIdentifier testTable = TableIdentifier.of("tbl"); Table table = catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); - Assertions.assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); - Assertions.assertThat(table.name()).isEqualTo("hadoop.tbl"); + assertThat(table.schema().toString()).isEqualTo(TABLE_SCHEMA.toString()); + assertThat(table.name()).isEqualTo("hadoop.tbl"); String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); + assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -264,10 +256,10 @@ public void testDropTable() throws Exception { String metaLocation = catalog.defaultWarehouseLocation(testTable); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); catalog.dropTable(testTable); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); + assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -276,14 +268,14 @@ public void testDropNonIcebergTable() throws Exception { TableIdentifier testTable = TableIdentifier.of("db", "ns1", "ns2", "tbl"); String metaLocation = catalog.defaultWarehouseLocation(testTable); // testing with non existent directory - Assertions.assertThat(catalog.dropTable(testTable)).isFalse(); + assertThat(catalog.dropTable(testTable)).isFalse(); FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); fs.mkdirs(new Path(metaLocation)); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); - Assertions.assertThat(catalog.dropTable(testTable)).isFalse(); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); + assertThat(catalog.dropTable(testTable)).isFalse(); + assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); } @Test @@ -291,8 +283,7 @@ public void testRenameTable() throws Exception { HadoopCatalog catalog = hadoopCatalog(); TableIdentifier testTable = TableIdentifier.of("db", "tbl1"); catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned()); - Assertions.assertThatThrownBy( - () -> catalog.renameTable(testTable, TableIdentifier.of("db", "tbl2"))) + assertThatThrownBy(() -> catalog.renameTable(testTable, TableIdentifier.of("db", "tbl2"))) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot rename Hadoop tables"); } @@ -311,13 +302,13 @@ public void testListTables() throws Exception { List tbls1 = catalog.listTables(Namespace.of("db")); Set tblSet = Sets.newHashSet(tbls1.stream().map(t -> t.name()).iterator()); - Assertions.assertThat(tblSet).hasSize(2).contains("tbl1").contains("tbl2"); + assertThat(tblSet).hasSize(2).contains("tbl1").contains("tbl2"); List tbls2 = catalog.listTables(Namespace.of("db", "ns1")); - Assertions.assertThat(tbls2).hasSize(1); - Assertions.assertThat(tbls2.get(0).name()).isEqualTo("tbl3"); + assertThat(tbls2).hasSize(1); + assertThat(tbls2.get(0).name()).isEqualTo("tbl3"); - Assertions.assertThatThrownBy(() -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))) + assertThatThrownBy(() -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.ns1.ns2"); } @@ -331,9 +322,7 @@ public void testCallingLocationProviderWhenNoCurrentMetadata() throws IOExceptio create.table().locationProvider(); // NPE triggered if not handled appropriately create.commitTransaction(); - Assertions.assertThat(catalog.listTables(Namespace.of("ns1", "ns2"))) - .as("1 table expected") - .hasSize(1); + assertThat(catalog.listTables(Namespace.of("ns1", "ns2"))).as("1 table expected").hasSize(1); catalog.dropTable(tableIdent, true); } @@ -352,13 +341,13 @@ public void testCreateNamespace() throws Exception { String metaLocation1 = warehouseLocation + "/" + "db/ns1/ns2"; FileSystem fs1 = Util.getFs(new Path(metaLocation1), catalog.getConf()); - Assertions.assertThat(fs1.isDirectory(new Path(metaLocation1))).isTrue(); + assertThat(fs1.isDirectory(new Path(metaLocation1))).isTrue(); String metaLocation2 = warehouseLocation + "/" + "db/ns2/ns3"; FileSystem fs2 = Util.getFs(new Path(metaLocation2), catalog.getConf()); - Assertions.assertThat(fs2.isDirectory(new Path(metaLocation2))).isTrue(); + assertThat(fs2.isDirectory(new Path(metaLocation2))).isTrue(); - Assertions.assertThatThrownBy(() -> catalog.createNamespace(tbl1.namespace())) + assertThatThrownBy(() -> catalog.createNamespace(tbl1.namespace())) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Namespace already exists: " + tbl1.namespace()); } @@ -378,25 +367,21 @@ public void testListNamespace() throws Exception { List nsp1 = catalog.listNamespaces(Namespace.of("db")); Set tblSet = Sets.newHashSet(nsp1.stream().map(t -> t.toString()).iterator()); - Assertions.assertThat(tblSet) - .hasSize(3) - .contains("db.ns1") - .contains("db.ns2") - .contains("db.ns3"); + assertThat(tblSet).hasSize(3).contains("db.ns1").contains("db.ns2").contains("db.ns3"); List nsp2 = catalog.listNamespaces(Namespace.of("db", "ns1")); - Assertions.assertThat(nsp2).hasSize(1); - Assertions.assertThat(nsp2.get(0).toString()).isEqualTo("db.ns1.ns2"); + assertThat(nsp2).hasSize(1); + assertThat(nsp2.get(0).toString()).isEqualTo("db.ns1.ns2"); List nsp3 = catalog.listNamespaces(); Set tblSet2 = Sets.newHashSet(nsp3.stream().map(t -> t.toString()).iterator()); - Assertions.assertThat(tblSet2).hasSize(2).contains("db").contains("db2"); + assertThat(tblSet2).hasSize(2).contains("db").contains("db2"); List nsp4 = catalog.listNamespaces(); Set tblSet3 = Sets.newHashSet(nsp4.stream().map(t -> t.toString()).iterator()); - Assertions.assertThat(tblSet3).hasSize(2).contains("db").contains("db2"); + assertThat(tblSet3).hasSize(2).contains("db").contains("db2"); - Assertions.assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) + assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.db2.ns2"); } @@ -414,8 +399,7 @@ public void testLoadNamespaceMeta() throws IOException { .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); catalog.loadNamespaceMetadata(Namespace.of("db")); - Assertions.assertThatThrownBy( - () -> catalog.loadNamespaceMetadata(Namespace.of("db", "db2", "ns2"))) + assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.db2.ns2"); } @@ -431,10 +415,10 @@ public void testNamespaceExists() throws IOException { Lists.newArrayList(tbl1, tbl2, tbl3, tbl4) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); - Assertions.assertThat(catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))) + assertThat(catalog.namespaceExists(Namespace.of("db", "ns1", "ns2"))) .as("Should be true as namespace exists") .isTrue(); - Assertions.assertThat(catalog.namespaceExists(Namespace.of("db", "db2", "ns2"))) + assertThat(catalog.namespaceExists(Namespace.of("db", "db2", "ns2"))) .as("Should be false as namespace doesn't exist") .isFalse(); } @@ -442,7 +426,7 @@ public void testNamespaceExists() throws IOException { @Test public void testAlterNamespaceMeta() throws IOException { HadoopCatalog catalog = hadoopCatalog(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.setProperties( Namespace.of("db", "db2", "ns2"), ImmutableMap.of("property", "test"))) @@ -466,19 +450,19 @@ public void testDropNamespace() throws IOException { Lists.newArrayList(tbl1, tbl2) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); - Assertions.assertThatThrownBy(() -> catalog.dropNamespace(Namespace.of("db"))) + assertThatThrownBy(() -> catalog.dropNamespace(Namespace.of("db"))) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace " + namespace1 + " is not empty."); - Assertions.assertThat(catalog.dropNamespace(Namespace.of("db2"))) + assertThat(catalog.dropNamespace(Namespace.of("db2"))) .as("Should fail to drop namespace that doesn't exist") .isFalse(); - Assertions.assertThat(catalog.dropTable(tbl1)).isTrue(); - Assertions.assertThat(catalog.dropTable(tbl2)).isTrue(); - Assertions.assertThat(catalog.dropNamespace(namespace2)).isTrue(); - Assertions.assertThat(catalog.dropNamespace(namespace1)).isTrue(); + assertThat(catalog.dropTable(tbl1)).isTrue(); + assertThat(catalog.dropTable(tbl2)).isTrue(); + assertThat(catalog.dropNamespace(namespace2)).isTrue(); + assertThat(catalog.dropNamespace(namespace1)).isTrue(); String metaLocation = warehouseLocation + "/" + "db"; FileSystem fs = Util.getFs(new Path(metaLocation), catalog.getConf()); - Assertions.assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); + assertThat(fs.isDirectory(new Path(metaLocation))).isFalse(); } @Test @@ -498,8 +482,8 @@ public void testVersionHintFileErrorWithFile() throws Exception { } // Check the result of the findVersion(), and load the table and check the current snapshotId - Assertions.assertThat(tableOperations.findVersion()).isEqualTo(1); - Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + assertThat(tableOperations.findVersion()).isEqualTo(1); + assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) .isEqualTo(secondSnapshotId); // Write newer data to confirm that we are writing the correct file @@ -509,8 +493,8 @@ public void testVersionHintFileErrorWithFile() throws Exception { } // Check the result of the findVersion(), and load the table and check the current snapshotId - Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); - Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + assertThat(tableOperations.findVersion()).isEqualTo(3); + assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) .isEqualTo(secondSnapshotId); // Write an empty version hint file @@ -518,16 +502,16 @@ public void testVersionHintFileErrorWithFile() throws Exception { io.newOutputFile(versionHintFile.getPath()).create().close(); // Check the result of the findVersion(), and load the table and check the current snapshotId - Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); - Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + assertThat(tableOperations.findVersion()).isEqualTo(3); + assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) .isEqualTo(secondSnapshotId); // Just delete the file io.deleteFile(versionHintFile.getPath()); // Check the result of the versionHint(), and load the table and check the current snapshotId - Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); - Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + assertThat(tableOperations.findVersion()).isEqualTo(3); + assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) .isEqualTo(secondSnapshotId); } @@ -548,8 +532,8 @@ public void testVersionHintFileMissingMetadata() throws Exception { io.deleteFile(tableOperations.getMetadataFile(1).toString()); // Check the result of the findVersion(), and load the table and check the current snapshotId - Assertions.assertThat(tableOperations.findVersion()).isEqualTo(3); - Assertions.assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) + assertThat(tableOperations.findVersion()).isEqualTo(3); + assertThat(TABLES.load(tableLocation).currentSnapshot().snapshotId()) .isEqualTo(secondSnapshotId); // Remove all the version files, and see if we can recover. Hint... not :) @@ -558,8 +542,8 @@ public void testVersionHintFileMissingMetadata() throws Exception { // Check that we got 0 findVersion, and a NoSuchTableException is thrown when trying to load the // table - Assertions.assertThat(tableOperations.findVersion()).isEqualTo(0); - Assertions.assertThatThrownBy(() -> TABLES.load(tableLocation)) + assertThat(tableOperations.findVersion()).isEqualTo(0); + assertThatThrownBy(() -> TABLES.load(tableLocation)) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); } @@ -571,12 +555,12 @@ public void testTableName() throws Exception { catalog.buildTable(tableIdent, SCHEMA).withPartitionSpec(SPEC).create(); Table table = catalog.loadTable(tableIdent); - Assertions.assertThat(table.name()).isEqualTo("hadoop.db.ns1.ns2.tbl"); + assertThat(table.name()).isEqualTo("hadoop.db.ns1.ns2.tbl"); TableIdentifier snapshotsTableIdent = TableIdentifier.of("db", "ns1", "ns2", "tbl", "snapshots"); Table snapshotsTable = catalog.loadTable(snapshotsTableIdent); - Assertions.assertThat(snapshotsTable.name()).isEqualTo("hadoop.db.ns1.ns2.tbl.snapshots"); + assertThat(snapshotsTable.name()).isEqualTo("hadoop.db.ns1.ns2.tbl.snapshots"); } private static void addVersionsToTable(Table table) { @@ -619,21 +603,21 @@ public void testTablePropsDefinedAtCatalogLevel() throws IOException { .withProperty("key5", "table-key5") .create(); - Assertions.assertThat(table.properties().get("key1")) + assertThat(table.properties().get("key1")) .as("Table defaults set for the catalog must be added to the table properties.") .isEqualTo("catalog-default-key1"); - Assertions.assertThat(table.properties().get("key2")) + assertThat(table.properties().get("key2")) .as("Table property must override table default properties set at catalog level.") .isEqualTo("table-key2"); - Assertions.assertThat(table.properties().get("key3")) + assertThat(table.properties().get("key3")) .as( "Table property override set at catalog level must override table default" + " properties set at catalog level and table property specified.") .isEqualTo("catalog-override-key3"); - Assertions.assertThat(table.properties().get("key4")) + assertThat(table.properties().get("key4")) .as("Table override not in table props or defaults should be added to table properties") .isEqualTo("catalog-override-key4"); - Assertions.assertThat(table.properties().get("key5")) + assertThat(table.properties().get("key5")) .as( "Table properties without any catalog level default or override should be added to table" + " properties.") @@ -649,10 +633,10 @@ public void testRegisterTable() throws IOException { Table registeringTable = catalog.loadTable(identifier); TableOperations ops = ((HasTableOperations) registeringTable).operations(); String metadataLocation = ((HadoopTableOperations) ops).current().metadataFileLocation(); - Assertions.assertThat(catalog.registerTable(identifier2, metadataLocation)).isNotNull(); - Assertions.assertThat(catalog.loadTable(identifier2)).isNotNull(); - Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); - Assertions.assertThat(catalog.dropTable(identifier2)).isTrue(); + assertThat(catalog.registerTable(identifier2, metadataLocation)).isNotNull(); + assertThat(catalog.loadTable(identifier2)).isNotNull(); + assertThat(catalog.dropTable(identifier)).isTrue(); + assertThat(catalog.dropTable(identifier2)).isTrue(); } @Test @@ -663,9 +647,9 @@ public void testRegisterExistingTable() throws IOException { Table registeringTable = catalog.loadTable(identifier); TableOperations ops = ((HasTableOperations) registeringTable).operations(); String metadataLocation = ((HadoopTableOperations) ops).current().metadataFileLocation(); - Assertions.assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) + assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: a.t1"); - Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); + assertThat(catalog.dropTable(identifier)).isTrue(); } } 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 b3ddc09c0ffd..60bef7fe0b14 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCommits.java @@ -21,6 +21,8 @@ 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; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.mockito.Mockito.any; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.mock; @@ -58,7 +60,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Tasks; -import org.assertj.core.api.Assertions; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -70,53 +71,53 @@ public class TestHadoopCommits extends HadoopTableTestBase { public void testCreateTable() throws Exception { PartitionSpec expectedSpec = PartitionSpec.builderFor(TABLE_SCHEMA).bucket("data", 16).build(); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match schema with reassigned ids") .isEqualTo(TABLE_SCHEMA.asStruct()); - Assertions.assertThat(table.spec()) + assertThat(table.spec()) .as("Table partition spec should match with reassigned ids") .isEqualTo(expectedSpec); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should not create any scan tasks").isEmpty(); - Assertions.assertThat(tableDir).as("Table location should exist").exists(); - Assertions.assertThat(metadataDir).as("Should create metadata folder").exists().isDirectory(); - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); - Assertions.assertThat(versionHintFile).as("Should create version hint file").exists(); - Assertions.assertThat(readVersionHint()) + assertThat(tasks).as("Should not create any scan tasks").isEmpty(); + assertThat(tableDir).as("Table location should exist").exists(); + assertThat(metadataDir).as("Should create metadata folder").exists().isDirectory(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(versionHintFile).as("Should create version hint file").exists(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(1); List manifests = listManifestFiles(); - Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); + assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testSchemaUpdate() throws Exception { - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(2); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match schema with reassigned ids") .isEqualTo(UPDATED_SCHEMA.asStruct()); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should not create any scan tasks").isEmpty(); + assertThat(tasks).as("Should not create any scan tasks").isEmpty(); List manifests = listManifestFiles(); - Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); + assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testSchemaUpdateComplexType() throws Exception { - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); Types.StructType complexColumn = Types.StructType.of( @@ -146,25 +147,25 @@ public void testSchemaUpdateComplexType() throws Exception { table.updateSchema().addColumn("complex", complexColumn).commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(2); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match schema with reassigned ids") .isEqualTo(updatedSchema.asStruct()); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should not create any scan tasks").isEmpty(); + assertThat(tasks).as("Should not create any scan tasks").isEmpty(); List manifests = listManifestFiles(); - Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); + assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testSchemaUpdateIdentifierFields() throws Exception { - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); Schema updatedSchema = new Schema( @@ -175,14 +176,14 @@ public void testSchemaUpdateIdentifierFields() throws Exception { table.updateSchema().setIdentifierFields("id").commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(2); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Table schema should match schema with reassigned ids") .isEqualTo(updatedSchema.asStruct()); - Assertions.assertThat(table.schema().identifierFieldIds()) + assertThat(table.schema().identifierFieldIds()) .as("Identifier fields should match schema with reassigned ids") .isEqualTo(updatedSchema.identifierFieldIds()); } @@ -193,25 +194,25 @@ public void testFailedCommit() throws Exception { UpdateSchema update = table.updateSchema().addColumn("n", Types.IntegerType.get()); update.apply(); - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); version(2).createNewFile(); - Assertions.assertThatThrownBy(update::commit) + assertThatThrownBy(update::commit) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith("Version 2 already exists"); List manifests = listManifestFiles(); - Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); + assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testStaleMetadata() throws Exception { Table tableCopy = TABLES.load(tableLocation); - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); // prepare changes on the copy without committing UpdateSchema updateCopy = tableCopy.updateSchema().addColumn("m", Types.IntegerType.get()); @@ -219,40 +220,40 @@ public void testStaleMetadata() throws Exception { table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(table.schema().asStruct()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(table.schema().asStruct()) .as("Unmodified copy should be out of date after update") .isNotEqualTo(tableCopy.schema().asStruct()); // update the table tableCopy.refresh(); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Copy should be back in sync") .isEqualTo(tableCopy.schema().asStruct()); - Assertions.assertThatThrownBy(updateCopy::commit) + assertThatThrownBy(updateCopy::commit) .isInstanceOf(CommitFailedException.class) .hasMessage("Cannot commit changes based on stale table metadata"); List manifests = listManifestFiles(); - Assertions.assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); + assertThat(manifests).as("Should contain 0 Avro manifest files").isEmpty(); } @Test public void testStaleVersionHint() throws Exception { Table stale = TABLES.load(tableLocation); - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(2); - Assertions.assertThat(stale.schema().asStruct()) + assertThat(stale.schema().asStruct()) .as("Stable table schema should not match") .isNotEqualTo(UPDATED_SCHEMA.asStruct()); @@ -260,12 +261,12 @@ public void testStaleVersionHint() throws Exception { replaceVersionHint(1); Table reloaded = TABLES.load(tableLocation); - Assertions.assertThat(reloaded.schema().asStruct()) + assertThat(reloaded.schema().asStruct()) .as("Updated schema for newly loaded table should match") .isEqualTo(UPDATED_SCHEMA.asStruct()); stale.refresh(); - Assertions.assertThat(reloaded.schema().asStruct()) + assertThat(reloaded.schema().asStruct()) .as("Refreshed schema for stale table should match") .isEqualTo(UPDATED_SCHEMA.asStruct()); } @@ -275,33 +276,31 @@ public void testFastAppend() throws Exception { // first append table.newFastAppend().appendFile(FILE_A).commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(2); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should scan 1 file").hasSize(1); + assertThat(tasks).as("Should scan 1 file").hasSize(1); List manifests = listManifestFiles(); - Assertions.assertThat(manifests).as("Should contain only one Avro manifest file").hasSize(1); + assertThat(manifests).as("Should contain only one Avro manifest file").hasSize(1); // second append table.newFastAppend().appendFile(FILE_B).commit(); - Assertions.assertThat(version(3)).as("Should create v3 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(3)).as("Should create v3 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(3); tasks = Lists.newArrayList(table.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should scan 2 files").hasSize(2); - Assertions.assertThat(listManifestFiles()) - .as("Should contain 2 Avro manifest files") - .hasSize(2); + assertThat(tasks).as("Should scan 2 files").hasSize(2); + assertThat(listManifestFiles()).as("Should contain 2 Avro manifest files").hasSize(2); TableMetadata metadata = readMetadataVersion(3); - Assertions.assertThat(metadata.currentSnapshot().allManifests(table.io())) + assertThat(metadata.currentSnapshot().allManifests(table.io())) .as("Current snapshot should contain 2 manifests") .hasSize(2); } @@ -317,14 +316,12 @@ public void testMergeAppend() throws Exception { table.newAppend().appendFile(FILE_C).commit(); List tasks = Lists.newArrayList(table.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should scan 3 files").hasSize(3); + assertThat(tasks).as("Should scan 3 files").hasSize(3); - Assertions.assertThat(listManifestFiles()) - .as("Should contain 3 Avro manifest files") - .hasSize(3); + assertThat(listManifestFiles()).as("Should contain 3 Avro manifest files").hasSize(3); TableMetadata metadata = readMetadataVersion(5); - Assertions.assertThat(metadata.currentSnapshot().allManifests(table.io())) + assertThat(metadata.currentSnapshot().allManifests(table.io())) .as("Current snapshot should contain 1 merged manifest") .hasSize(1); } @@ -350,9 +347,9 @@ public void testRenameThrow() throws Exception { * provided {@link FileSystem} object. The provided FileSystem will be injected for commit call. */ private void testRenameWithFileSystem(FileSystem mockFs) throws Exception { - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); - Assertions.assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); - Assertions.assertThat(table).isInstanceOf(BaseTable.class); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(2)).as("Should not create v2 or newer versions").doesNotExist(); + assertThat(table).isInstanceOf(BaseTable.class); BaseTable baseTable = (BaseTable) table; // use v1 metafile as the test rename destination. TableMetadata meta1 = baseTable.operations().current(); @@ -361,33 +358,31 @@ private void testRenameWithFileSystem(FileSystem mockFs) throws Exception { // (so that we have 2 valid and different metadata files, which will reach the rename part // during commit) table.updateSchema().addColumn("n", Types.IntegerType.get()).commit(); - Assertions.assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); - Assertions.assertThat(readVersionHint()) + assertThat(version(2)).as("Should create v2 for the update").exists().isFile(); + assertThat(readVersionHint()) .as("Should write the current version to the hint file") .isEqualTo(2); // mock / spy the classes for testing TableOperations tops = baseTable.operations(); - Assertions.assertThat(tops).isInstanceOf(HadoopTableOperations.class); + assertThat(tops).isInstanceOf(HadoopTableOperations.class); HadoopTableOperations spyOps = Mockito.spy((HadoopTableOperations) tops); // inject the mockFS into the TableOperations doReturn(mockFs).when(spyOps).getFileSystem(any(), any()); - Assertions.assertThatThrownBy(() -> spyOps.commit(tops.current(), meta1)) + assertThatThrownBy(() -> spyOps.commit(tops.current(), meta1)) .isInstanceOf(CommitFailedException.class); // Verifies that there is no temporary metadata.json files left on rename failures. Set actual = listMetadataJsonFiles().stream().map(File::getName).collect(Collectors.toSet()); Set expected = Sets.newHashSet("v1.metadata.json", "v2.metadata.json"); - Assertions.assertThat(actual) - .as("only v1 and v2 metadata.json should exist.") - .isEqualTo(expected); + assertThat(actual).as("only v1 and v2 metadata.json should exist.").isEqualTo(expected); } @Test public void testCanReadOldCompressedManifestFiles() throws Exception { - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); // do a file append table.newAppend().appendFile(FILE_A).commit(); @@ -399,20 +394,20 @@ public void testCanReadOldCompressedManifestFiles() throws Exception { List metadataFiles = listMetadataJsonFiles(); - Assertions.assertThat(metadataFiles).as("Should have two versions").hasSize(2); - Assertions.assertThat(metadataFiles.stream().map(File::getName)) + assertThat(metadataFiles).as("Should have two versions").hasSize(2); + assertThat(metadataFiles.stream().map(File::getName)) .as("Metadata should be compressed with old format.") .allMatch(f -> f.endsWith(".metadata.json.gz")); Table reloaded = TABLES.load(tableLocation); List tasks = Lists.newArrayList(reloaded.newScan().planFiles()); - Assertions.assertThat(tasks).as("Should scan 1 files").hasSize(1); + assertThat(tasks).as("Should scan 1 files").hasSize(1); } @Test public void testConcurrentFastAppends(@TempDir File dir) throws Exception { - Assertions.assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); + assertThat(version(1)).as("Should create v1 metadata").exists().isFile(); int threadsCount = 5; int numberOfCommitedFilesPerThread = 10; Table tableWithHighRetries = @@ -452,7 +447,7 @@ public void testConcurrentFastAppends(@TempDir File dir) throws Exception { }); tableWithHighRetries.refresh(); - Assertions.assertThat(Lists.newArrayList(tableWithHighRetries.snapshots())) + assertThat(Lists.newArrayList(tableWithHighRetries.snapshots())) .hasSize(threadsCount * numberOfCommitedFilesPerThread); } @@ -467,7 +462,7 @@ public void testCommitFailedToAcquireLock() { tableOperations.refresh(); BaseTable baseTable = (BaseTable) table; TableMetadata meta2 = baseTable.operations().current(); - Assertions.assertThatThrownBy(() -> tableOperations.commit(tableOperations.current(), meta2)) + assertThatThrownBy(() -> tableOperations.commit(tableOperations.current(), meta2)) .isInstanceOf(CommitFailedException.class) .hasMessageStartingWith("Failed to acquire lock on file"); } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java index 09b478e4a6c1..af3fb569d114 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopStreams.java @@ -18,12 +18,13 @@ */ package org.apache.iceberg.hadoop; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.util.concurrent.Executors; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.s3a.S3ABlockOutputStream; import org.apache.iceberg.io.PositionOutputStream; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; class TestHadoopStreams { @@ -46,7 +47,7 @@ void closeShouldThrowIOExceptionWhenInterrupted() throws Exception { s3ABlockOutputStream.interruptClose(); }); - Assertions.assertThatThrownBy(wrap::close) + assertThatThrownBy(wrap::close) .isInstanceOf(IOException.class) .hasMessage("S3ABlockOutputStream failed to upload object after stream was closed"); } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java index e3d32442e2df..70a04b990544 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopTables.java @@ -21,6 +21,8 @@ import static org.apache.iceberg.NullOrder.NULLS_FIRST; import static org.apache.iceberg.SortDirection.ASC; 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; @@ -40,7 +42,6 @@ import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -57,10 +58,10 @@ public class TestHadoopTables { @Test public void testTableExists() { - Assertions.assertThat(TABLES.exists(tableDir.toURI().toString())).isFalse(); + assertThat(TABLES.exists(tableDir.toURI().toString())).isFalse(); PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).bucket("data", 16).build(); TABLES.create(SCHEMA, spec, tableDir.toURI().toString()); - Assertions.assertThat(TABLES.exists(tableDir.toURI().toString())).isTrue(); + assertThat(TABLES.exists(tableDir.toURI().toString())).isTrue(); } @Test @@ -68,7 +69,7 @@ public void testDropTable() { TABLES.create(SCHEMA, tableDir.toURI().toString()); TABLES.dropTable(tableDir.toURI().toString()); - Assertions.assertThatThrownBy(() -> TABLES.load(tableDir.toURI().toString())) + assertThatThrownBy(() -> TABLES.load(tableDir.toURI().toString())) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); } @@ -79,13 +80,13 @@ public void testDropTableWithPurge() throws IOException { createDummyTable(tableDir, dataDir); TABLES.dropTable(tableDir.toURI().toString(), true); - Assertions.assertThatThrownBy(() -> TABLES.load(tableDir.toURI().toString())) + assertThatThrownBy(() -> TABLES.load(tableDir.toURI().toString())) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); - Assertions.assertThat(dataDir.listFiles()).hasSize(0); - Assertions.assertThat(tableDir).doesNotExist(); - Assertions.assertThat(TABLES.dropTable(tableDir.toURI().toString())).isFalse(); + assertThat(dataDir.listFiles()).hasSize(0); + assertThat(tableDir).doesNotExist(); + assertThat(TABLES.dropTable(tableDir.toURI().toString())).isFalse(); } @Test @@ -93,13 +94,13 @@ public void testDropTableWithoutPurge() throws IOException { createDummyTable(tableDir, dataDir); TABLES.dropTable(tableDir.toURI().toString(), false); - Assertions.assertThatThrownBy(() -> TABLES.load(tableDir.toURI().toString())) + assertThatThrownBy(() -> TABLES.load(tableDir.toURI().toString())) .isInstanceOf(NoSuchTableException.class) .hasMessageStartingWith("Table does not exist"); - Assertions.assertThat(dataDir.listFiles()).hasSize(1); - Assertions.assertThat(tableDir).doesNotExist(); - Assertions.assertThat(TABLES.dropTable(tableDir.toURI().toString())).isFalse(); + assertThat(dataDir.listFiles()).hasSize(1); + assertThat(tableDir).doesNotExist(); + assertThat(TABLES.dropTable(tableDir.toURI().toString())).isFalse(); } @Test @@ -108,8 +109,8 @@ public void testDefaultSortOrder() { Table table = TABLES.create(SCHEMA, spec, tableDir.toURI().toString()); SortOrder sortOrder = table.sortOrder(); - Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(0); - Assertions.assertThat(sortOrder.isUnsorted()).as("Order must be unsorted").isTrue(); + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(0); + assertThat(sortOrder.isUnsorted()).as("Order must be unsorted").isTrue(); } @Test @@ -120,16 +121,14 @@ public void testCustomSortOrder() { TABLES.create(SCHEMA, spec, order, Maps.newHashMap(), tableDir.toURI().toString()); SortOrder sortOrder = table.sortOrder(); - Assertions.assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); - Assertions.assertThat(sortOrder.fields()).as("Order must have 1 field").hasSize(1); - Assertions.assertThat(sortOrder.fields().get(0).direction()) - .as("Direction must match") - .isEqualTo(ASC); - Assertions.assertThat(sortOrder.fields().get(0).nullOrder()) + assertThat(sortOrder.orderId()).as("Order ID must match").isEqualTo(1); + assertThat(sortOrder.fields()).as("Order must have 1 field").hasSize(1); + assertThat(sortOrder.fields().get(0).direction()).as("Direction must match").isEqualTo(ASC); + assertThat(sortOrder.fields().get(0).nullOrder()) .as("Null order must match") .isEqualTo(NULLS_FIRST); Transform transform = Transforms.identity(); - Assertions.assertThat(sortOrder.fields().get(0).transform()) + assertThat(sortOrder.fields().get(0).transform()) .as("Transform must match") .isEqualTo(transform); } @@ -141,12 +140,10 @@ public void testTableName() { TABLES.create(SCHEMA, spec, location); Table table = TABLES.load(location); - Assertions.assertThat(table.name()).as("Name must match").isEqualTo(location); + assertThat(table.name()).as("Name must match").isEqualTo(location); Table snapshotsTable = TABLES.load(location + "#snapshots"); - Assertions.assertThat(snapshotsTable.name()) - .as("Name must match") - .isEqualTo(location + "#snapshots"); + assertThat(snapshotsTable.name()).as("Name must match").isEqualTo(location + "#snapshots"); } private static void createDummyTable(File tableDir, File dataDir) throws IOException { @@ -164,7 +161,7 @@ private static void createDummyTable(File tableDir, File dataDir) throws IOExcep append.commit(); // Make sure that the data file and the manifest dir is created - Assertions.assertThat(dataDir.listFiles()).hasSize(1); - Assertions.assertThat(tableDir.listFiles()).hasSize(1); + assertThat(dataDir.listFiles()).hasSize(1); + assertThat(tableDir.listFiles()).hasSize(1); } } diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java index 7cb57d72d359..377ef42ca989 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java @@ -18,11 +18,13 @@ */ package org.apache.iceberg.hadoop; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.StaticTableOperations; import org.apache.iceberg.Table; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestStaticTable extends HadoopTableTestBase { @@ -39,7 +41,7 @@ private Table getStaticTable(MetadataTableType type) { @Test public void testLoadFromMetadata() { Table staticTable = getStaticTable(); - Assertions.assertThat(((HasTableOperations) staticTable).operations()) + assertThat(((HasTableOperations) staticTable).operations()) .as("Loading a metadata file based table should return StaticTableOperations") .isInstanceOf(StaticTableOperations.class); } @@ -47,7 +49,7 @@ public void testLoadFromMetadata() { @Test public void testCannotBeAddedTo() { Table staticTable = getStaticTable(); - Assertions.assertThatThrownBy(() -> staticTable.newOverwrite().addFile(FILE_A).commit()) + assertThatThrownBy(() -> staticTable.newOverwrite().addFile(FILE_A).commit()) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot modify a static table"); } @@ -56,7 +58,7 @@ public void testCannotBeAddedTo() { public void testCannotBeDeletedFrom() { table.newAppend().appendFile(FILE_A).commit(); Table staticTable = getStaticTable(); - Assertions.assertThatThrownBy(() -> staticTable.newDelete().deleteFile(FILE_A).commit()) + assertThatThrownBy(() -> staticTable.newDelete().deleteFile(FILE_A).commit()) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot modify a static table"); } @@ -69,11 +71,11 @@ public void testCannotDoIncrementalScanOnMetadataTable() { Table staticTable = getStaticTable(type); if (type.equals(MetadataTableType.POSITION_DELETES)) { - Assertions.assertThatThrownBy(staticTable::newScan) + assertThatThrownBy(staticTable::newScan) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot create TableScan from table of type POSITION_DELETES"); } else { - Assertions.assertThatThrownBy(() -> staticTable.newScan().appendsAfter(1)) + assertThatThrownBy(() -> staticTable.newScan().appendsAfter(1)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage(String.format("Cannot incrementally scan table of type %s", type)); } @@ -86,13 +88,11 @@ public void testHasSameProperties() { table.newAppend().appendFile(FILE_B).commit(); table.newOverwrite().deleteFile(FILE_B).addFile(FILE_C).commit(); Table staticTable = getStaticTable(); - Assertions.assertThat(table.history()).as("Same history?").containsAll(staticTable.history()); - Assertions.assertThat(table.currentSnapshot().snapshotId()) + assertThat(table.history()).as("Same history?").containsAll(staticTable.history()); + assertThat(table.currentSnapshot().snapshotId()) .as("Same snapshot?") .isEqualTo(staticTable.currentSnapshot().snapshotId()); - Assertions.assertThat(table.properties()) - .as("Same properties?") - .isEqualTo(staticTable.properties()); + assertThat(table.properties()).as("Same properties?").isEqualTo(staticTable.properties()); } @Test @@ -105,7 +105,7 @@ public void testImmutable() { table.newOverwrite().deleteFile(FILE_B).addFile(FILE_C).commit(); staticTable.refresh(); - Assertions.assertThat(staticTable.currentSnapshot().snapshotId()) + assertThat(staticTable.currentSnapshot().snapshotId()) .as("Snapshot unchanged after table modified") .isEqualTo(originalSnapshot); } @@ -114,7 +114,7 @@ public void testImmutable() { public void testMetadataTables() { for (MetadataTableType type : MetadataTableType.values()) { String enumName = type.name().replace("_", "").toLowerCase(); - Assertions.assertThat(getStaticTable(type).getClass().getName().toLowerCase()) + assertThat(getStaticTable(type).getClass().getName().toLowerCase()) .as("Should be able to get MetadataTable of type : " + type) .contains(enumName); } 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 78a724225bac..fa4b227a1a9d 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestTableSerialization.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.hadoop; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -43,7 +45,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; @@ -62,8 +63,8 @@ public void testSerializableTable() throws IOException, ClassNotFoundException { Table serializableTable = SerializableTable.copyOf(table); TestHelpers.assertSerializedAndLoadedMetadata( serializableTable, TestHelpers.KryoHelpers.roundTripSerialize(serializableTable)); - Assertions.assertThat(serializableTable).isInstanceOf(HasTableOperations.class); - Assertions.assertThat(((HasTableOperations) serializableTable).operations()) + assertThat(serializableTable).isInstanceOf(HasTableOperations.class); + assertThat(((HasTableOperations) serializableTable).operations()) .isInstanceOf(StaticTableOperations.class); } @@ -119,10 +120,10 @@ public void testSerializableTablePlanning() throws IOException { Set deserializedFiles = getFiles(deserialized); // Checks that the deserialized data stays the same - Assertions.assertThat(deserializedFiles).isEqualTo(expected); + assertThat(deserializedFiles).isEqualTo(expected); // We expect that the files changed in the meantime - Assertions.assertThat(deserializedFiles).isNotEqualTo(getFiles(table)); + assertThat(deserializedFiles).isNotEqualTo(getFiles(table)); } @ParameterizedTest @@ -153,13 +154,13 @@ public void testSerializableMetadataTablesPlanning(boolean fromSerialized) throw Set deserializedFiles = getFiles(deserializeFromBytes(serialized.get(type))); // Checks that the deserialized data stays the same - Assertions.assertThat(deserializedFiles).isEqualTo(expected.get(type)); + assertThat(deserializedFiles).isEqualTo(expected.get(type)); // Collect the current data Set newFiles = getFiles(getMetaDataTable(table, type)); // Expect that the new data is changed in the meantime - Assertions.assertThat(deserializedFiles).isNotEqualTo(newFiles); + assertThat(deserializedFiles).isNotEqualTo(newFiles); } } diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java index 12f5bf84ccba..174d054e9c6e 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryFileIO.java @@ -18,13 +18,15 @@ */ package org.apache.iceberg.inmemory; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; import java.util.UUID; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NotFoundException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestInMemoryFileIO { @@ -33,35 +35,35 @@ public class TestInMemoryFileIO { public void testBasicEndToEnd() throws IOException { InMemoryFileIO fileIO = new InMemoryFileIO(); String location = randomLocation(); - Assertions.assertThat(fileIO.fileExists(location)).isFalse(); + assertThat(fileIO.fileExists(location)).isFalse(); OutputStream outputStream = fileIO.newOutputFile(location).create(); byte[] data = "hello world".getBytes(); outputStream.write(data); outputStream.close(); - Assertions.assertThat(fileIO.fileExists(location)).isTrue(); + assertThat(fileIO.fileExists(location)).isTrue(); InputStream inputStream = fileIO.newInputFile(location).newStream(); byte[] buf = new byte[data.length]; inputStream.read(buf); inputStream.close(); - Assertions.assertThat(new String(buf)).isEqualTo("hello world"); + assertThat(new String(buf)).isEqualTo("hello world"); fileIO.deleteFile(location); - Assertions.assertThat(fileIO.fileExists(location)).isFalse(); + assertThat(fileIO.fileExists(location)).isFalse(); } @Test public void testNewInputFileNotFound() { InMemoryFileIO fileIO = new InMemoryFileIO(); - Assertions.assertThatExceptionOfType(NotFoundException.class) + assertThatExceptionOfType(NotFoundException.class) .isThrownBy(() -> fileIO.newInputFile("s3://nonexistent/file")); } @Test public void testDeleteFileNotFound() { InMemoryFileIO fileIO = new InMemoryFileIO(); - Assertions.assertThatExceptionOfType(NotFoundException.class) + assertThatExceptionOfType(NotFoundException.class) .isThrownBy(() -> fileIO.deleteFile("s3://nonexistent/file")); } @@ -70,7 +72,7 @@ public void testCreateNoOverwrite() { String location = randomLocation(); InMemoryFileIO fileIO = new InMemoryFileIO(); fileIO.addFile(location, "hello world".getBytes()); - Assertions.assertThatExceptionOfType(AlreadyExistsException.class) + assertThatExceptionOfType(AlreadyExistsException.class) .isThrownBy(() -> fileIO.newOutputFile(location).create()); } @@ -86,11 +88,11 @@ public void testOverwriteBeforeAndAfterClose() throws IOException { // Even though we've called create() and started writing data, this file won't yet exist // in the parentFileIO before we've closed it. - Assertions.assertThat(fileIO.fileExists(location)).isFalse(); + assertThat(fileIO.fileExists(location)).isFalse(); // File appears after closing it. outputStream.close(); - Assertions.assertThat(fileIO.fileExists(location)).isTrue(); + assertThat(fileIO.fileExists(location)).isTrue(); // Start a new OutputFile and write new data but don't close() it yet. outputStream = fileIO.newOutputFile(location).createOrOverwrite(); @@ -101,7 +103,7 @@ public void testOverwriteBeforeAndAfterClose() throws IOException { byte[] buf = new byte[oldData.length]; inputStream.read(buf); inputStream.close(); - Assertions.assertThat(new String(buf)).isEqualTo("old data"); + assertThat(new String(buf)).isEqualTo("old data"); // Finally, close the new output stream; data should be overwritten with new data now. outputStream.close(); @@ -109,7 +111,7 @@ public void testOverwriteBeforeAndAfterClose() throws IOException { buf = new byte[newData.length]; inputStream.read(buf); inputStream.close(); - Assertions.assertThat(new String(buf)).isEqualTo("new data"); + assertThat(new String(buf)).isEqualTo("new data"); } @Test @@ -119,7 +121,7 @@ public void testFilesAreSharedAcrossMultipleInstances() { fileIO.addFile(location, "hello world".getBytes()); InMemoryFileIO fileIO2 = new InMemoryFileIO(); - Assertions.assertThat(fileIO2.fileExists(location)) + assertThat(fileIO2.fileExists(location)) .isTrue() .as("Files should be shared across all InMemoryFileIO instances"); } diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java index 5aa5e427c164..63168a592885 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryInputFile.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.inmemory; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.io.InputStream; import java.nio.charset.StandardCharsets; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestInMemoryInputFile { @@ -30,8 +32,8 @@ public void testReadAfterClose() throws IOException { InMemoryInputFile inputFile = new InMemoryInputFile("abc".getBytes(StandardCharsets.ISO_8859_1)); InputStream inputStream = inputFile.newStream(); - Assertions.assertThat(inputStream.read()).isEqualTo('a'); + assertThat(inputStream.read()).isEqualTo('a'); inputStream.close(); - Assertions.assertThatThrownBy(inputStream::read).hasMessage("Stream is closed"); + assertThatThrownBy(inputStream::read).hasMessage("Stream is closed"); } } diff --git a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java index 8015c5d1ca17..2ae59364a386 100644 --- a/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java +++ b/core/src/test/java/org/apache/iceberg/inmemory/TestInMemoryOutputFile.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.inmemory; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.io.OutputStream; import java.nio.charset.StandardCharsets; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestInMemoryOutputFile { @@ -32,8 +34,7 @@ public void testWriteAfterClose() throws IOException { outputStream.write('a'); outputStream.write('b'); outputStream.close(); - Assertions.assertThatThrownBy(() -> outputStream.write('c')).hasMessage("Stream is closed"); - Assertions.assertThat(outputFile.toByteArray()) - .isEqualTo("ab".getBytes(StandardCharsets.ISO_8859_1)); + assertThatThrownBy(() -> outputStream.write('c')).hasMessage("Stream is closed"); + assertThat(outputFile.toByteArray()).isEqualTo("ab".getBytes(StandardCharsets.ISO_8859_1)); } } diff --git a/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java b/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java index 408de7ce6e71..c0850d82fa98 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java +++ b/core/src/test/java/org/apache/iceberg/io/TestByteBufferInputStreams.java @@ -18,13 +18,15 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Collections; import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public abstract class TestByteBufferInputStreams { @@ -39,14 +41,12 @@ public void testRead0() throws Exception { ByteBufferInputStream stream = newStream(); - Assertions.assertThat(stream.read(bytes)).as("Should read 0 bytes").isEqualTo(0); + assertThat(stream.read(bytes)).as("Should read 0 bytes").isEqualTo(0); int bytesRead = stream.read(new byte[100]); - Assertions.assertThat(bytesRead).as("Should read to end of stream").isLessThan(100); + assertThat(bytesRead).as("Should read to end of stream").isLessThan(100); - Assertions.assertThat(stream.read(bytes)) - .as("Should read 0 bytes at end of stream") - .isEqualTo(0); + assertThat(stream.read(bytes)).as("Should read 0 bytes at end of stream").isEqualTo(0); } @Test @@ -56,22 +56,18 @@ public void testReadAll() throws Exception { ByteBufferInputStream stream = newStream(); int bytesRead = stream.read(bytes); - Assertions.assertThat(bytesRead).as("Should read the entire buffer").isEqualTo(bytes.length); + assertThat(bytesRead).as("Should read the entire buffer").isEqualTo(bytes.length); for (int i = 0; i < bytes.length; i += 1) { - Assertions.assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) i); - Assertions.assertThat(stream.getPos()).as("Should advance position").isEqualTo(35); + assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) i); + assertThat(stream.getPos()).as("Should advance position").isEqualTo(35); } - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(0); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(0); - Assertions.assertThat(stream.read(bytes)).as("Should return -1 at end of stream").isEqualTo(-1); + assertThat(stream.read(bytes)).as("Should return -1 at end of stream").isEqualTo(-1); - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(0); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(0); checkOriginalData(); } @@ -86,37 +82,27 @@ public void testSmallReads() throws Exception { int lastBytesRead = bytes.length; for (int offset = 0; offset < length; offset += bytes.length) { - Assertions.assertThat(lastBytesRead) - .as("Should read requested len") - .isEqualTo(bytes.length); + assertThat(lastBytesRead).as("Should read requested len").isEqualTo(bytes.length); lastBytesRead = stream.read(bytes, 0, bytes.length); - Assertions.assertThat(stream.getPos()) - .as("Should advance position") - .isEqualTo(offset + lastBytesRead); + assertThat(stream.getPos()).as("Should advance position").isEqualTo(offset + lastBytesRead); // validate the bytes that were read for (int i = 0; i < lastBytesRead; i += 1) { - Assertions.assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) (offset + i)); + assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) (offset + i)); } } - Assertions.assertThat(lastBytesRead % bytes.length) + assertThat(lastBytesRead % bytes.length) .as("Should read fewer bytes at end of buffer") .isEqualTo(length % bytes.length); - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(0); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(0); - Assertions.assertThat(stream.read(bytes)) - .as("Should return -1 at end of stream") - .isEqualTo(-1); + assertThat(stream.read(bytes)).as("Should return -1 at end of stream").isEqualTo(-1); - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(0); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(0); } checkOriginalData(); @@ -131,40 +117,32 @@ public void testPartialBufferReads() throws Exception { int lastBytesRead = size; for (int offset = 0; offset < bytes.length; offset += size) { - Assertions.assertThat(lastBytesRead).as("Should read requested len").isEqualTo(size); + assertThat(lastBytesRead).as("Should read requested len").isEqualTo(size); lastBytesRead = stream.read(bytes, offset, Math.min(size, bytes.length - offset)); - Assertions.assertThat(stream.getPos()) + assertThat(stream.getPos()) .as("Should advance position") .isEqualTo(lastBytesRead > 0 ? offset + lastBytesRead : offset); } - Assertions.assertThat(lastBytesRead % size) + assertThat(lastBytesRead % size) .as("Should read fewer bytes at end of buffer") .isEqualTo(bytes.length % size); for (int i = 0; i < bytes.length; i += 1) { - Assertions.assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) i); + assertThat(bytes[i]).as("Byte i should be i").isEqualTo((byte) i); } - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(2); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(2); - Assertions.assertThat(stream.read(bytes)).as("Should return 2 more bytes").isEqualTo(2); + assertThat(stream.read(bytes)).as("Should return 2 more bytes").isEqualTo(2); - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(0); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(0); - Assertions.assertThat(stream.read(bytes)) - .as("Should return -1 at end of stream") - .isEqualTo(-1); + assertThat(stream.read(bytes)).as("Should return -1 at end of stream").isEqualTo(-1); - Assertions.assertThat(stream.available()) - .as("Should have no more remaining content") - .isEqualTo(0); + assertThat(stream.available()).as("Should have no more remaining content").isEqualTo(0); } checkOriginalData(); @@ -176,11 +154,11 @@ public void testReadByte() throws Exception { int length = stream.available(); for (int i = 0; i < length; i += 1) { - Assertions.assertThat(stream.getPos()).as("Position should increment").isEqualTo(i); - Assertions.assertThat(stream.read()).isEqualTo(i); + assertThat(stream.getPos()).as("Position should increment").isEqualTo(i); + assertThat(stream.read()).isEqualTo(i); } - Assertions.assertThatThrownBy(stream::read).isInstanceOf(EOFException.class).hasMessage(null); + assertThatThrownBy(stream::read).isInstanceOf(EOFException.class).hasMessage(null); checkOriginalData(); } @@ -193,11 +171,9 @@ public void testSlice() throws Exception { ByteBuffer empty = stream.slice(0); - Assertions.assertThat(empty).as("slice(0) should produce a non-null buffer").isNotNull(); - Assertions.assertThat(empty.remaining()) - .as("slice(0) should produce an empty buffer") - .isEqualTo(0); - Assertions.assertThat(stream.getPos()).as("Position should be at start").isEqualTo(0); + assertThat(empty).as("slice(0) should produce a non-null buffer").isNotNull(); + assertThat(empty.remaining()).as("slice(0) should produce an empty buffer").isEqualTo(0); + assertThat(stream.getPos()).as("Position should be at start").isEqualTo(0); int i = 0; while (stream.available() > 0) { @@ -205,13 +181,13 @@ public void testSlice() throws Exception { ByteBuffer buffer = stream.slice(bytesToSlice); for (int j = 0; j < bytesToSlice; j += 1) { - Assertions.assertThat(buffer.get()).as("Data should be correct").isEqualTo((byte) (i + j)); + assertThat(buffer.get()).as("Data should be correct").isEqualTo((byte) (i + j)); } i += bytesToSlice; } - Assertions.assertThat(stream.getPos()).as("Position should be at end").isEqualTo(length); + assertThat(stream.getPos()).as("Position should be at end").isEqualTo(length); checkOriginalData(); } @@ -220,7 +196,7 @@ public void testSlice() throws Exception { public void testSliceBuffers0() throws Exception { ByteBufferInputStream stream = newStream(); - Assertions.assertThat(stream.sliceBuffers(0)) + assertThat(stream.sliceBuffers(0)) .as("Should return an empty list") .isEqualTo(Collections.emptyList()); } @@ -232,15 +208,15 @@ public void testWholeSliceBuffers() throws Exception { List buffers = stream.sliceBuffers(stream.available()); - Assertions.assertThat(stream.getPos()).as("Should consume all buffers").isEqualTo(length); + assertThat(stream.getPos()).as("Should consume all buffers").isEqualTo(length); - Assertions.assertThatThrownBy(() -> stream.sliceBuffers(length)) + assertThatThrownBy(() -> stream.sliceBuffers(length)) .isInstanceOf(EOFException.class) .hasMessage(null); ByteBufferInputStream copy = ByteBufferInputStream.wrap(buffers); for (int i = 0; i < length; i += 1) { - Assertions.assertThat(copy.read()).as("Slice should have identical data").isEqualTo(i); + assertThat(copy.read()).as("Slice should have identical data").isEqualTo(i); } checkOriginalData(); @@ -257,12 +233,12 @@ public void testSliceBuffersCoverage() throws Exception { buffers.addAll(stream.sliceBuffers(Math.min(size, stream.available()))); } - Assertions.assertThat(stream.getPos()).as("Should consume all content").isEqualTo(length); + assertThat(stream.getPos()).as("Should consume all content").isEqualTo(length); ByteBufferInputStream newStream = new MultiBufferInputStream(buffers); for (int i = 0; i < length; i += 1) { - Assertions.assertThat(newStream.read()).as("Data should be correct").isEqualTo(i); + assertThat(newStream.read()).as("Data should be correct").isEqualTo(i); } } @@ -277,36 +253,32 @@ public void testSliceBuffersModification() throws Exception { int sliceLength = 5; List buffers = stream.sliceBuffers(sliceLength); - Assertions.assertThat(stream.available()) + assertThat(stream.available()) .as("Should advance the original stream") .isEqualTo(length - sliceLength); - Assertions.assertThat(stream.getPos()) + assertThat(stream.getPos()) .as("Should advance the original stream position") .isEqualTo(sliceLength); - Assertions.assertThat(buffers.size()) - .as("Should return a slice of the first buffer") - .isEqualTo(1); + assertThat(buffers.size()).as("Should return a slice of the first buffer").isEqualTo(1); ByteBuffer buffer = buffers.get(0); - Assertions.assertThat(buffer.remaining()) - .as("Should have requested bytes") - .isEqualTo(sliceLength); + assertThat(buffer.remaining()).as("Should have requested bytes").isEqualTo(sliceLength); // read the buffer one past the returned limit. this should not change the // next value in the original stream buffer.limit(sliceLength + 1); for (int i = 0; i < sliceLength + 1; i += 1) { - Assertions.assertThat(buffer.get()).as("Should have correct data").isEqualTo((byte) i); + assertThat(buffer.get()).as("Should have correct data").isEqualTo((byte) i); } - Assertions.assertThat(stream.getPos()) + assertThat(stream.getPos()) .as("Reading a slice shouldn't advance the original stream") .isEqualTo(sliceLength); - Assertions.assertThat(stream.read()) + assertThat(stream.read()) .as("Reading a slice shouldn't change the underlying data") .isEqualTo(sliceLength); @@ -318,11 +290,11 @@ public void testSliceBuffersModification() throws Exception { try { buffer.put((byte) 255); - Assertions.assertThat(stream.getPos()) + assertThat(stream.getPos()) .as("Writing to a slice shouldn't advance the original stream") .isEqualTo(sliceLength + 1); - Assertions.assertThat(stream.read()) + assertThat(stream.read()) .as("Writing to a slice should change the underlying data") .isEqualTo(255); @@ -337,20 +309,20 @@ public void testSkip() throws Exception { while (stream.available() > 0) { int bytesToSkip = Math.min(stream.available(), 10); - Assertions.assertThat(stream.skip(bytesToSkip)) + assertThat(stream.skip(bytesToSkip)) .as("Should skip all, regardless of backing buffers") .isEqualTo(bytesToSkip); } stream = newStream(); - Assertions.assertThat(stream.skip(0)).isEqualTo(0); + assertThat(stream.skip(0)).isEqualTo(0); int length = stream.available(); - Assertions.assertThat(stream.skip(length + 10)) + assertThat(stream.skip(length + 10)) .as("Should stop at end when out of bytes") .isEqualTo(length); - Assertions.assertThat(stream.skip(10)).as("Should return -1 when at end").isEqualTo(-1); + assertThat(stream.skip(10)).as("Should return -1 when at end").isEqualTo(-1); } @Test @@ -363,7 +335,7 @@ public void testSkipFully() throws Exception { stream.skipFully(bytesToSkip); - Assertions.assertThat(stream.getPos() - lastPosition) + assertThat(stream.getPos() - lastPosition) .as("Should skip all, regardless of backing buffers") .isEqualTo(bytesToSkip); @@ -372,11 +344,11 @@ public void testSkipFully() throws Exception { ByteBufferInputStream stream2 = newStream(); stream2.skipFully(0); - Assertions.assertThat(stream2.getPos()).as("Check initial position").isEqualTo(0); + assertThat(stream2.getPos()).as("Check initial position").isEqualTo(0); int length = stream2.available(); - Assertions.assertThatThrownBy(() -> stream2.skipFully(length + 10)) + assertThatThrownBy(() -> stream2.skipFully(length + 10)) .isInstanceOf(EOFException.class) .hasMessageStartingWith("Not enough bytes to skip"); } @@ -397,20 +369,18 @@ public void testMark() throws Exception { stream.reset(); - Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); + assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[100]; int bytesReadAfterReset = stream.read(afterReset); - Assertions.assertThat(bytesReadAfterReset) + assertThat(bytesReadAfterReset) .as("Should read the same number of bytes") .isEqualTo(expectedBytesRead); - Assertions.assertThat(stream.getPos()) - .as("Read should end at the same position") - .isEqualTo(end); + assertThat(stream.getPos()).as("Read should end at the same position").isEqualTo(end); - Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); + assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -430,19 +400,17 @@ public void testMarkTwice() throws Exception { stream.reset(); - Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); + assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[100]; int bytesReadAfterReset = stream.read(afterReset); - Assertions.assertThat(bytesReadAfterReset) + assertThat(bytesReadAfterReset) .as("Should read the same number of bytes") .isEqualTo(expectedBytesRead); - Assertions.assertThat(stream.getPos()) - .as("Read should end at the same position") - .isEqualTo(end); + assertThat(stream.getPos()).as("Read should end at the same position").isEqualTo(end); - Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); + assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -454,22 +422,20 @@ public void testMarkAtStart() throws Exception { long mark = stream.getPos(); byte[] expected = new byte[10]; - Assertions.assertThat(stream.read(expected)).as("Should read 10 bytes").isEqualTo(10); + assertThat(stream.read(expected)).as("Should read 10 bytes").isEqualTo(10); long end = stream.getPos(); stream.reset(); - Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); + assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[10]; - Assertions.assertThat(stream.read(afterReset)).as("Should read 10 bytes").isEqualTo(10); + assertThat(stream.read(afterReset)).as("Should read 10 bytes").isEqualTo(10); - Assertions.assertThat(stream.getPos()) - .as("Read should end at the same position") - .isEqualTo(end); + assertThat(stream.getPos()).as("Read should end at the same position").isEqualTo(end); - Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); + assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test @@ -477,36 +443,34 @@ public void testMarkAtEnd() throws Exception { ByteBufferInputStream stream = newStream(); int bytesRead = stream.read(new byte[100]); - Assertions.assertThat(bytesRead < 100).as("Should read to end of stream").isTrue(); + assertThat(bytesRead < 100).as("Should read to end of stream").isTrue(); stream.mark(100); long mark = stream.getPos(); byte[] expected = new byte[10]; - Assertions.assertThat(stream.read(expected)).as("Should read 0 bytes").isEqualTo(-1); + assertThat(stream.read(expected)).as("Should read 0 bytes").isEqualTo(-1); long end = stream.getPos(); stream.reset(); - Assertions.assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); + assertThat(stream.getPos()).as("Position should return to the mark").isEqualTo(mark); byte[] afterReset = new byte[10]; - Assertions.assertThat(stream.read(afterReset)).as("Should read 0 bytes").isEqualTo(-1); + assertThat(stream.read(afterReset)).as("Should read 0 bytes").isEqualTo(-1); - Assertions.assertThat(stream.getPos()) - .as("Read should end at the same position") - .isEqualTo(end); + assertThat(stream.getPos()).as("Read should end at the same position").isEqualTo(end); - Assertions.assertThat(afterReset).as("Content should be equal").isEqualTo(expected); + assertThat(afterReset).as("Content should be equal").isEqualTo(expected); } @Test public void testMarkUnset() { ByteBufferInputStream stream = newStream(); - Assertions.assertThatThrownBy(stream::reset) + assertThatThrownBy(stream::reset) .isInstanceOf(IOException.class) .hasMessageStartingWith("No mark defined"); } @@ -517,28 +481,26 @@ public void testMarkAndResetTwiceOverSameRange() throws Exception { byte[] expected = new byte[6]; stream.mark(10); - Assertions.assertThat(stream.read(expected)) - .as("Should read expected bytes") - .isEqualTo(expected.length); + assertThat(stream.read(expected)).as("Should read expected bytes").isEqualTo(expected.length); stream.reset(); stream.mark(10); byte[] firstRead = new byte[6]; - Assertions.assertThat(stream.read(firstRead)) + assertThat(stream.read(firstRead)) .as("Should read firstRead bytes") .isEqualTo(firstRead.length); stream.reset(); byte[] secondRead = new byte[6]; - Assertions.assertThat(stream.read(secondRead)) + assertThat(stream.read(secondRead)) .as("Should read secondRead bytes") .isEqualTo(secondRead.length); - Assertions.assertThat(firstRead).as("First read should be correct").isEqualTo(expected); + assertThat(firstRead).as("First read should be correct").isEqualTo(expected); - Assertions.assertThat(secondRead).as("Second read should be correct").isEqualTo(expected); + assertThat(secondRead).as("Second read should be correct").isEqualTo(expected); } @Test @@ -546,13 +508,13 @@ public void testMarkLimit() throws Exception { ByteBufferInputStream stream = newStream(); stream.mark(5); - Assertions.assertThat(stream.read(new byte[5])).as("Should read 5 bytes").isEqualTo(5); + assertThat(stream.read(new byte[5])).as("Should read 5 bytes").isEqualTo(5); stream.reset(); - Assertions.assertThat(stream.read(new byte[6])).as("Should read 6 bytes").isEqualTo(6); + assertThat(stream.read(new byte[6])).as("Should read 6 bytes").isEqualTo(6); - Assertions.assertThatThrownBy(stream::reset) + assertThatThrownBy(stream::reset) .isInstanceOf(IOException.class) .hasMessageStartingWith("No mark defined"); } @@ -562,11 +524,11 @@ public void testMarkDoubleReset() throws Exception { ByteBufferInputStream stream = newStream(); stream.mark(5); - Assertions.assertThat(stream.read(new byte[5])).as("Should read 5 bytes").isEqualTo(5); + assertThat(stream.read(new byte[5])).as("Should read 5 bytes").isEqualTo(5); stream.reset(); - Assertions.assertThatThrownBy(stream::reset) + assertThatThrownBy(stream::reset) .isInstanceOf(IOException.class) .hasMessageStartingWith("No mark defined"); } diff --git a/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java b/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java index e6b4cb967df4..48c75b582fa1 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java +++ b/core/src/test/java/org/apache/iceberg/io/TestIOUtil.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.io; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.EOFException; import java.io.IOException; import java.nio.ByteBuffer; @@ -25,7 +28,6 @@ import java.util.Arrays; import org.apache.iceberg.inmemory.InMemoryOutputFile; import org.apache.iceberg.relocated.com.google.common.base.Strings; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestIOUtil { @@ -36,13 +38,11 @@ public void testReadFully() throws Exception { MockInputStream stream = new MockInputStream(); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assertions.assertThat(buffer) + assertThat(buffer) .as("Byte array contents should match") .isEqualTo(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(5); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(5); } @Test @@ -52,13 +52,11 @@ public void testReadFullySmallReads() throws Exception { MockInputStream stream = new MockInputStream(2, 3, 3); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assertions.assertThat(buffer) + assertThat(buffer) .as("Byte array contents should match") .containsExactly(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(5); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(5); } @Test @@ -68,15 +66,11 @@ public void testReadFullyJustRight() throws Exception { final MockInputStream stream = new MockInputStream(2, 3, 3); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assertions.assertThat(buffer) - .as("Byte array contents should match") - .isEqualTo(MockInputStream.TEST_ARRAY); + assertThat(buffer).as("Byte array contents should match").isEqualTo(MockInputStream.TEST_ARRAY); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(10); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(10); - Assertions.assertThatThrownBy(() -> IOUtil.readFully(stream, buffer, 0, 1)) + assertThatThrownBy(() -> IOUtil.readFully(stream, buffer, 0, 1)) .isInstanceOf(EOFException.class) .hasMessage("Reached the end of stream with 1 bytes left to read"); } @@ -87,17 +81,15 @@ public void testReadFullyUnderflow() { final MockInputStream stream = new MockInputStream(2, 3, 3); - Assertions.assertThatThrownBy(() -> IOUtil.readFully(stream, buffer, 0, buffer.length)) + assertThatThrownBy(() -> IOUtil.readFully(stream, buffer, 0, buffer.length)) .isInstanceOf(EOFException.class) .hasMessage("Reached the end of stream with 1 bytes left to read"); - Assertions.assertThat(Arrays.copyOfRange(buffer, 0, 10)) + assertThat(Arrays.copyOfRange(buffer, 0, 10)) .as("Should have consumed bytes") .isEqualTo(MockInputStream.TEST_ARRAY); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(10); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(10); } @Test @@ -107,13 +99,11 @@ public void testReadFullyStartAndLength() throws IOException { MockInputStream stream = new MockInputStream(); IOUtil.readFully(stream, buffer, 2, 5); - Assertions.assertThat(Arrays.copyOfRange(buffer, 2, 7)) + assertThat(Arrays.copyOfRange(buffer, 2, 7)) .as("Byte array contents should match") .isEqualTo(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(5); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(5); } @Test @@ -123,9 +113,7 @@ public void testReadFullyZeroByteRead() throws IOException { MockInputStream stream = new MockInputStream(); IOUtil.readFully(stream, buffer, 0, buffer.length); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(0); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(0); } @Test @@ -135,13 +123,11 @@ public void testReadFullySmallReadsWithStartAndLength() throws IOException { MockInputStream stream = new MockInputStream(2, 2, 3); IOUtil.readFully(stream, buffer, 2, 5); - Assertions.assertThat(Arrays.copyOfRange(buffer, 2, 7)) + assertThat(Arrays.copyOfRange(buffer, 2, 7)) .as("Byte array contents should match") .isEqualTo(Arrays.copyOfRange(MockInputStream.TEST_ARRAY, 0, 5)); - Assertions.assertThat(stream.getPos()) - .as("Stream position should reflect bytes read") - .isEqualTo(5); + assertThat(stream.getPos()).as("Stream position should reflect bytes read").isEqualTo(5); } @Test @@ -151,6 +137,6 @@ public void testWriteFully() throws Exception { try (PositionOutputStream outputStream = outputFile.create()) { IOUtil.writeFully(outputStream, ByteBuffer.wrap(input.clone())); } - Assertions.assertThat(outputFile.toByteArray()).isEqualTo(input); + assertThat(outputFile.toByteArray()).isEqualTo(input); } } 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 90492b51097a..c0c9afafd9a7 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -79,7 +79,6 @@ import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -256,7 +255,7 @@ public void testSqlNonTransientExceptionNotRetryable() { jdbcCatalog.setConf(conf); jdbcCatalog.initialize("test_catalog_with_retryable_status_codes", properties); JdbcClientPool jdbcClientPool = jdbcCatalog.connectionPool(); - Assertions.assertThat( + assertThat( jdbcClientPool.isConnectionException( new SQLNonTransientConnectionException("Failed to authenticate"))) .as("SQL Non Transient exception is not retryable") @@ -522,8 +521,7 @@ public void testBasicCatalog() throws Exception { FileSystem fs = Util.getFs(new Path(metaLocation), conf); assertThat(fs.isDirectory(new Path(metaLocation))).isTrue(); - Assertions.assertThatThrownBy( - () -> catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> catalog.createTable(testTable, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: db.ns1.ns2.tbl"); @@ -580,7 +578,7 @@ public void testConcurrentCommit() throws IOException { .withRecordCount(1) .build(); - Assertions.assertThatThrownBy(() -> table.newAppend().appendFile(dataFile2).commit()) + assertThatThrownBy(() -> table.newAppend().appendFile(dataFile2).commit()) .isInstanceOf(NoSuchTableException.class) .hasMessage( "Failed to load table db.table from catalog test_jdbc_catalog: dropped by another process"); @@ -636,7 +634,7 @@ public void testDropTable() { assertThat(catalog.listTables(testTable.namespace())).doesNotContain(testTable); catalog.dropTable(testTable2); - Assertions.assertThatThrownBy(() -> catalog.listTables(testTable2.namespace())) + assertThatThrownBy(() -> catalog.listTables(testTable2.namespace())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.ns1.ns2"); @@ -665,15 +663,14 @@ public void testRenameTable() { assertThat(catalog.listTables(to.namespace())).contains(to).doesNotContain(from); assertThat(catalog.loadTable(to).name()).endsWith(to.name()); - Assertions.assertThatThrownBy( - () -> catalog.renameTable(TableIdentifier.of("db", "tbl-not-exists"), to)) + assertThatThrownBy(() -> catalog.renameTable(TableIdentifier.of("db", "tbl-not-exists"), to)) .isInstanceOf(NoSuchTableException.class) .hasMessage("Table does not exist: db.tbl-not-exists"); // rename table to existing table name! TableIdentifier from2 = TableIdentifier.of("db", "tbl2"); catalog.createTable(from2, SCHEMA, PartitionSpec.unpartitioned()); - Assertions.assertThatThrownBy(() -> catalog.renameTable(from2, to)) + assertThatThrownBy(() -> catalog.renameTable(from2, to)) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: db.tbl2-newtable"); } @@ -697,7 +694,7 @@ public void testListTables() { assertThat(tbls2).hasSize(1); assertThat(tbls2.get(0).name()).isEqualTo("tbl3"); - Assertions.assertThatThrownBy(() -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))) + assertThatThrownBy(() -> catalog.listTables(Namespace.of("db", "ns1", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.ns1.ns2"); } @@ -774,7 +771,7 @@ public void testListNamespace() { Set tblSet3 = Sets.newHashSet(nsp4.stream().map(Namespace::toString).iterator()); assertThat(tblSet3).hasSize(3).contains("db", "db2", ""); - Assertions.assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) + assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.db2.ns2"); } @@ -791,8 +788,7 @@ public void testLoadNamespaceMeta() { assertThat(catalog.loadNamespaceMetadata(Namespace.of("db"))).containsKey("location"); - Assertions.assertThatThrownBy( - () -> catalog.loadNamespaceMetadata(Namespace.of("db", "db2", "ns2"))) + assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: db.db2.ns2"); } @@ -829,15 +825,15 @@ public void testDropNamespace() { Lists.newArrayList(tbl0, tbl1, tbl2, tbl3, tbl4) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); - Assertions.assertThatThrownBy(() -> catalog.dropNamespace(tbl1.namespace())) + assertThatThrownBy(() -> catalog.dropNamespace(tbl1.namespace())) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace db.ns1.ns2 is not empty. 2 tables exist."); - Assertions.assertThatThrownBy(() -> catalog.dropNamespace(tbl2.namespace())) + assertThatThrownBy(() -> catalog.dropNamespace(tbl2.namespace())) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace db.ns1 is not empty. 1 tables exist."); - Assertions.assertThatThrownBy(() -> catalog.dropNamespace(tbl4.namespace())) + assertThatThrownBy(() -> catalog.dropNamespace(tbl4.namespace())) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace db is not empty. 1 tables exist."); } @@ -915,12 +911,12 @@ public void testCreateTableInNonExistingNamespace() { try (JdbcCatalog jdbcCatalog = initCatalog("non_strict_jdbc_catalog", ImmutableMap.of())) { Namespace namespace = Namespace.of("test\\D_b%", "ns1", "ns2"); TableIdentifier identifier = TableIdentifier.of(namespace, "someTable"); - Assertions.assertThat(jdbcCatalog.namespaceExists(namespace)).isFalse(); - Assertions.assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); + assertThat(jdbcCatalog.namespaceExists(namespace)).isFalse(); + assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); // default=non-strict mode allows creating a table in a non-existing namespace jdbcCatalog.createTable(identifier, SCHEMA, PARTITION_SPEC); - Assertions.assertThat(jdbcCatalog.loadTable(identifier)).isNotNull(); + assertThat(jdbcCatalog.loadTable(identifier)).isNotNull(); } } @@ -931,20 +927,19 @@ public void testCreateTableInNonExistingNamespaceStrictMode() { "strict_jdbc_catalog", ImmutableMap.of(JdbcUtil.STRICT_MODE_PROPERTY, "true"))) { Namespace namespace = Namespace.of("testDb", "ns1", "ns2"); TableIdentifier identifier = TableIdentifier.of(namespace, "someTable"); - Assertions.assertThat(jdbcCatalog.namespaceExists(namespace)).isFalse(); - Assertions.assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); - Assertions.assertThatThrownBy( - () -> jdbcCatalog.createTable(identifier, SCHEMA, PARTITION_SPEC)) + assertThat(jdbcCatalog.namespaceExists(namespace)).isFalse(); + assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); + assertThatThrownBy(() -> jdbcCatalog.createTable(identifier, SCHEMA, PARTITION_SPEC)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage( "Cannot create table testDb.ns1.ns2.someTable in catalog strict_jdbc_catalog. Namespace testDb.ns1.ns2 does not exist"); - Assertions.assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); + assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); jdbcCatalog.createNamespace(namespace); - Assertions.assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); + assertThat(jdbcCatalog.tableExists(identifier)).isFalse(); jdbcCatalog.createTable(identifier, SCHEMA, PARTITION_SPEC); - Assertions.assertThat(jdbcCatalog.loadTable(identifier)).isNotNull(); + assertThat(jdbcCatalog.loadTable(identifier)).isNotNull(); } } @@ -967,7 +962,7 @@ public void testNamespaceLocation() { Map testMetadata = ImmutableMap.of(); catalog.createNamespace(testNamespace, testMetadata); - Assertions.assertThat(catalog.loadNamespaceMetadata(testNamespace)).containsKey("location"); + assertThat(catalog.loadNamespaceMetadata(testNamespace)).containsKey("location"); } @Test @@ -979,7 +974,7 @@ public void testNamespaceCustomLocation() { Map testMetadata = ImmutableMap.of("location", namespaceLocation); catalog.createNamespace(testNamespace, testMetadata); - Assertions.assertThat(catalog.loadNamespaceMetadata(testNamespace)) + assertThat(catalog.loadNamespaceMetadata(testNamespace)) .containsEntry("location", namespaceLocation); } @@ -1073,14 +1068,14 @@ public void testCatalogWithCustomMetricsReporter() throws IOException { .build()) .commit(); try (CloseableIterable tasks = table.newScan().planFiles()) { - Assertions.assertThat(tasks.iterator()).hasNext(); + 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 - Assertions.assertThat(CustomMetricsReporter.COUNTER.get()).isEqualTo(2); + assertThat(CustomMetricsReporter.COUNTER.get()).isEqualTo(2); } @Test diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java index 8c018e1b52bc..dc7450311216 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCommitReportParser.java @@ -18,53 +18,54 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCommitReportParser { @Test public void nullCommitReport() { - Assertions.assertThatThrownBy(() -> CommitReportParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> CommitReportParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse commit report from null object"); - Assertions.assertThatThrownBy(() -> CommitReportParser.toJson(null)) + assertThatThrownBy(() -> CommitReportParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid commit report: null"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> CommitReportParser.fromJson("{}")) + assertThatThrownBy(() -> CommitReportParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: table-name"); - Assertions.assertThatThrownBy( - () -> CommitReportParser.fromJson("{\"table-name\":\"roundTripTableName\"}")) + assertThatThrownBy(() -> CommitReportParser.fromJson("{\"table-name\":\"roundTripTableName\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: snapshot-id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CommitReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: sequence-number"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CommitReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"sequence-number\":24}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: operation"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CommitReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"sequence-number\":24, \"operation\": \"DELETE\"}")) @@ -74,14 +75,14 @@ public void missingFields() { @Test public void invalidTableName() { - Assertions.assertThatThrownBy(() -> CommitReportParser.fromJson("{\"table-name\":23}")) + assertThatThrownBy(() -> CommitReportParser.fromJson("{\"table-name\":23}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: table-name: 23"); } @Test public void invalidSnapshotId() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CommitReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":\"invalid\"}")) @@ -239,8 +240,8 @@ public void roundTripSerde() { + "}"; String json = CommitReportParser.toJson(commitReport, true); - Assertions.assertThat(CommitReportParser.fromJson(json)).isEqualTo(commitReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(CommitReportParser.fromJson(json)).isEqualTo(commitReport); + assertThat(json).isEqualTo(expectedJson); } @Test @@ -265,8 +266,8 @@ public void roundTripSerdeWithNoopMetrics() { + "}"; String json = CommitReportParser.toJson(commitReport, true); - Assertions.assertThat(CommitReportParser.fromJson(json)).isEqualTo(commitReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(CommitReportParser.fromJson(json)).isEqualTo(commitReport); + assertThat(json).isEqualTo(expectedJson); } @Test @@ -296,7 +297,7 @@ public void roundTripSerdeWithMetadata() { + "}"; String json = CommitReportParser.toJson(commitReport, true); - Assertions.assertThat(CommitReportParser.fromJson(json)).isEqualTo(commitReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(CommitReportParser.fromJson(json)).isEqualTo(commitReport); + assertThat(json).isEqualTo(expectedJson); } } diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java index ea936a0265d8..89808c54c788 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestCounterResultParser.java @@ -18,53 +18,54 @@ */ package org.apache.iceberg.metrics; +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.metrics.MetricsContext.Unit; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCounterResultParser { @Test public void nullCounter() { - Assertions.assertThatThrownBy(() -> CounterResultParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> CounterResultParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse counter from null object"); - Assertions.assertThatThrownBy(() -> CounterResultParser.toJson(null)) + assertThatThrownBy(() -> CounterResultParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid counter: null"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> CounterResultParser.fromJson("{}")) + assertThatThrownBy(() -> CounterResultParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: unit"); - Assertions.assertThatThrownBy(() -> CounterResultParser.fromJson("{\"unit\":\"bytes\"}")) + assertThatThrownBy(() -> CounterResultParser.fromJson("{\"unit\":\"bytes\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: value"); } @Test public void extraFields() { - Assertions.assertThat( + assertThat( CounterResultParser.fromJson("{\"unit\":\"bytes\",\"value\":23,\"extra\": \"value\"}")) .isEqualTo(CounterResult.of(Unit.BYTES, 23L)); } @Test public void unsupportedUnit() { - Assertions.assertThatThrownBy( - () -> CounterResultParser.fromJson("{\"unit\":\"unknown\",\"value\":23}")) + assertThatThrownBy(() -> CounterResultParser.fromJson("{\"unit\":\"unknown\",\"value\":23}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid unit: unknown"); } @Test public void invalidValue() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CounterResultParser.fromJson("{\"unit\":\"count\",\"value\":\"illegal\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: value: \"illegal\""); @@ -75,7 +76,7 @@ public void roundTripSerde() { CounterResult counter = CounterResult.of(Unit.BYTES, Long.MAX_VALUE); String json = CounterResultParser.toJson(counter); - Assertions.assertThat(CounterResultParser.fromJson(json)).isEqualTo(counter); - Assertions.assertThat(json).isEqualTo("{\"unit\":\"bytes\",\"value\":9223372036854775807}"); + assertThat(CounterResultParser.fromJson(json)).isEqualTo(counter); + assertThat(json).isEqualTo("{\"unit\":\"bytes\",\"value\":9223372036854775807}"); } } 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 2daa2b459220..44d5803c4a3a 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanMetricsResultParser.java @@ -18,22 +18,24 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.time.Duration; import java.util.concurrent.TimeUnit; import org.apache.iceberg.metrics.MetricsContext.Unit; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestScanMetricsResultParser { @Test public void nullMetrics() { - Assertions.assertThatThrownBy(() -> ScanMetricsResultParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> ScanMetricsResultParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse scan metrics from null object"); - Assertions.assertThatThrownBy(() -> ScanMetricsResultParser.toJson(null)) + assertThatThrownBy(() -> ScanMetricsResultParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid scan metrics: null"); } @@ -41,27 +43,27 @@ public void nullMetrics() { @SuppressWarnings("MethodLength") @Test public void missingFields() { - Assertions.assertThat(ScanMetricsResultParser.fromJson("{}")) + assertThat(ScanMetricsResultParser.fromJson("{}")) .isEqualTo(ImmutableScanMetricsResult.builder().build()); ImmutableScanMetricsResult scanMetricsResult = ImmutableScanMetricsResult.builder() .totalPlanningDuration(TimerResult.of(TimeUnit.HOURS, Duration.ofHours(10), 3L)) .build(); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}}")) .isEqualTo(scanMetricsResult); scanMetricsResult = scanMetricsResult.withResultDataFiles(CounterResult.of(Unit.COUNT, 5L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}}")) .isEqualTo(scanMetricsResult); scanMetricsResult = scanMetricsResult.withResultDeleteFiles(CounterResult.of(Unit.COUNT, 5L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -69,7 +71,7 @@ public void missingFields() { .isEqualTo(scanMetricsResult); scanMetricsResult = scanMetricsResult.withTotalDataManifests(CounterResult.of(Unit.COUNT, 5L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -79,7 +81,7 @@ public void missingFields() { scanMetricsResult = scanMetricsResult.withTotalDeleteManifests(CounterResult.of(Unit.COUNT, 0L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -90,7 +92,7 @@ public void missingFields() { scanMetricsResult = scanMetricsResult.withScannedDataManifests(CounterResult.of(Unit.COUNT, 5L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -102,7 +104,7 @@ public void missingFields() { scanMetricsResult = scanMetricsResult.withSkippedDataManifests(CounterResult.of(Unit.COUNT, 5L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -115,7 +117,7 @@ public void missingFields() { scanMetricsResult = scanMetricsResult.withTotalFileSizeInBytes(CounterResult.of(Unit.BYTES, 1069L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -129,7 +131,7 @@ public void missingFields() { scanMetricsResult = scanMetricsResult.withTotalDeleteFileSizeInBytes(CounterResult.of(Unit.BYTES, 1023L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -143,7 +145,7 @@ public void missingFields() { .isEqualTo(scanMetricsResult); scanMetricsResult = scanMetricsResult.withSkippedDataFiles(CounterResult.of(Unit.COUNT, 23L)); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":3,\"time-unit\":\"hours\",\"total-duration\":10}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -179,7 +181,7 @@ public void extraFields() { scanMetrics.equalityDeleteFiles().increment(4L); ScanMetricsResult scanMetricsResult = ScanMetricsResult.fromScanMetrics(scanMetrics); - Assertions.assertThat( + assertThat( ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":1,\"time-unit\":\"nanoseconds\",\"total-duration\":600000000000}," + "\"result-data-files\":{\"unit\":\"count\",\"value\":5}," @@ -203,7 +205,7 @@ public void extraFields() { @Test public void invalidTimer() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"unit\":\"count\",\"value\":5}}")) @@ -213,7 +215,7 @@ public void invalidTimer() { @Test public void invalidCounter() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanMetricsResultParser.fromJson( "{\"total-planning-duration\":{\"count\":1,\"time-unit\":\"nanoseconds\",\"total-duration\":600000000000}," @@ -314,8 +316,8 @@ public void roundTripSerde() { + "}"; String json = ScanMetricsResultParser.toJson(scanMetricsResult, true); - Assertions.assertThat(ScanMetricsResultParser.fromJson(json)).isEqualTo(scanMetricsResult); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(ScanMetricsResultParser.fromJson(json)).isEqualTo(scanMetricsResult); + assertThat(json).isEqualTo(expectedJson); } @Test @@ -324,8 +326,8 @@ public void roundTripSerdeNoopScanMetrics() { String expectedJson = "{ }"; String json = ScanMetricsResultParser.toJson(scanMetricsResult, true); - Assertions.assertThat(json).isEqualTo(expectedJson); - Assertions.assertThat(ScanMetricsResultParser.fromJson(json)) + assertThat(json).isEqualTo(expectedJson); + assertThat(ScanMetricsResultParser.fromJson(json)) .isEqualTo(ImmutableScanMetricsResult.builder().build()); } } diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java b/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java index caaf1bcaed21..89256cb6f34f 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanReport.java @@ -18,29 +18,31 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.time.Duration; import java.util.Arrays; import java.util.List; import java.util.concurrent.TimeUnit; import org.apache.iceberg.expressions.Expressions; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestScanReport { @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> ImmutableScanReport.builder().build()) + assertThatThrownBy(() -> ImmutableScanReport.builder().build()) .isInstanceOf(IllegalStateException.class) .hasMessage( "Cannot build ScanReport, some of required attributes are not set [tableName, snapshotId, filter, schemaId, scanMetrics]"); - Assertions.assertThatThrownBy(() -> ImmutableScanReport.builder().tableName("x").build()) + assertThatThrownBy(() -> ImmutableScanReport.builder().tableName("x").build()) .isInstanceOf(IllegalStateException.class) .hasMessage( "Cannot build ScanReport, some of required attributes are not set [snapshotId, filter, schemaId, scanMetrics]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ImmutableScanReport.builder() .tableName("x") @@ -50,7 +52,7 @@ public void missingFields() { .hasMessage( "Cannot build ScanReport, some of required attributes are not set [snapshotId, schemaId, scanMetrics]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ImmutableScanReport.builder() .tableName("x") @@ -61,7 +63,7 @@ public void missingFields() { .hasMessage( "Cannot build ScanReport, some of required attributes are not set [schemaId, scanMetrics]"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ImmutableScanReport.builder() .tableName("x") @@ -93,21 +95,21 @@ public void fromEmptyScanMetrics() { .scanMetrics(ScanMetricsResult.fromScanMetrics(ScanMetrics.noop())) .build(); - Assertions.assertThat(scanReport.tableName()).isEqualTo(tableName); - Assertions.assertThat(scanReport.schemaId()).isEqualTo(schemaId); - Assertions.assertThat(scanReport.projectedFieldIds()).isEqualTo(fieldIds); - Assertions.assertThat(scanReport.projectedFieldNames()).isEqualTo(fieldNames); - Assertions.assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue()); - Assertions.assertThat(scanReport.snapshotId()).isEqualTo(23L); - Assertions.assertThat(scanReport.scanMetrics().totalPlanningDuration()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().resultDataFiles()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().resultDeleteFiles()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().totalDataManifests()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().totalDeleteManifests()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().scannedDataManifests()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().skippedDataManifests()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().totalFileSizeInBytes()).isNull(); - Assertions.assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes()).isNull(); + assertThat(scanReport.tableName()).isEqualTo(tableName); + assertThat(scanReport.schemaId()).isEqualTo(schemaId); + assertThat(scanReport.projectedFieldIds()).isEqualTo(fieldIds); + assertThat(scanReport.projectedFieldNames()).isEqualTo(fieldNames); + assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue()); + assertThat(scanReport.snapshotId()).isEqualTo(23L); + assertThat(scanReport.scanMetrics().totalPlanningDuration()).isNull(); + assertThat(scanReport.scanMetrics().resultDataFiles()).isNull(); + assertThat(scanReport.scanMetrics().resultDeleteFiles()).isNull(); + assertThat(scanReport.scanMetrics().totalDataManifests()).isNull(); + assertThat(scanReport.scanMetrics().totalDeleteManifests()).isNull(); + assertThat(scanReport.scanMetrics().scannedDataManifests()).isNull(); + assertThat(scanReport.scanMetrics().skippedDataManifests()).isNull(); + assertThat(scanReport.scanMetrics().totalFileSizeInBytes()).isNull(); + assertThat(scanReport.scanMetrics().totalDeleteFileSizeInBytes()).isNull(); } @Test @@ -136,24 +138,24 @@ public void fromScanMetrics() { .scanMetrics(ScanMetricsResult.fromScanMetrics(scanMetrics)) .build(); - Assertions.assertThat(scanReport.tableName()).isEqualTo(tableName); - Assertions.assertThat(scanReport.schemaId()).isEqualTo(schemaId); - Assertions.assertThat(scanReport.projectedFieldIds()).isEqualTo(fieldIds); - Assertions.assertThat(scanReport.projectedFieldNames()).isEqualTo(fieldNames); - Assertions.assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue()); - Assertions.assertThat(scanReport.snapshotId()).isEqualTo(23L); - Assertions.assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration()) + assertThat(scanReport.tableName()).isEqualTo(tableName); + assertThat(scanReport.schemaId()).isEqualTo(schemaId); + assertThat(scanReport.projectedFieldIds()).isEqualTo(fieldIds); + assertThat(scanReport.projectedFieldNames()).isEqualTo(fieldNames); + assertThat(scanReport.filter()).isEqualTo(Expressions.alwaysTrue()); + assertThat(scanReport.snapshotId()).isEqualTo(23L); + assertThat(scanReport.scanMetrics().totalPlanningDuration().totalDuration()) .isEqualTo(Duration.ofMinutes(10L)); - Assertions.assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(5); - Assertions.assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(5); - Assertions.assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(5); - Assertions.assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(5); - Assertions.assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isEqualTo(1024L); + assertThat(scanReport.scanMetrics().resultDataFiles().value()).isEqualTo(5); + assertThat(scanReport.scanMetrics().resultDeleteFiles().value()).isEqualTo(5); + assertThat(scanReport.scanMetrics().scannedDataManifests().value()).isEqualTo(5); + assertThat(scanReport.scanMetrics().totalDataManifests().value()).isEqualTo(5); + assertThat(scanReport.scanMetrics().totalFileSizeInBytes().value()).isEqualTo(1024L); } @Test public void nullScanMetrics() { - Assertions.assertThatThrownBy(() -> ScanMetrics.of(null)) + assertThatThrownBy(() -> ScanMetrics.of(null)) .isInstanceOf(NullPointerException.class) .hasMessage("metricsContext"); } 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 df126601c8f1..51e21ad9bf01 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestScanReportParser.java @@ -18,45 +18,46 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.util.concurrent.TimeUnit; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestScanReportParser { @Test public void nullScanReport() { - Assertions.assertThatThrownBy(() -> ScanReportParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> ScanReportParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse scan report from null object"); - Assertions.assertThatThrownBy(() -> ScanReportParser.toJson(null)) + assertThatThrownBy(() -> ScanReportParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid scan report: null"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> ScanReportParser.fromJson("{}")) + assertThatThrownBy(() -> ScanReportParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: table-name"); - Assertions.assertThatThrownBy( - () -> ScanReportParser.fromJson("{\"table-name\":\"roundTripTableName\"}")) + assertThatThrownBy(() -> ScanReportParser.fromJson("{\"table-name\":\"roundTripTableName\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: snapshot-id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"filter\":true}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing int: schema-id"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"filter\":true," @@ -97,7 +98,7 @@ public void extraFields() { .scanMetrics(ScanMetricsResult.fromScanMetrics(scanMetrics)) .build(); - Assertions.assertThat( + assertThat( ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23," + "\"filter\":true,\"schema-id\": 4,\"projected-field-ids\": [ 1, 2, 3 ],\"projected-field-names\": [ \"c1\", \"c2\", \"c3\" ]," @@ -124,14 +125,14 @@ public void extraFields() { @Test public void invalidTableName() { - Assertions.assertThatThrownBy(() -> ScanReportParser.fromJson("{\"table-name\":23}")) + assertThatThrownBy(() -> ScanReportParser.fromJson("{\"table-name\":23}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: table-name: 23"); } @Test public void invalidSnapshotId() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":\"invalid\"}")) @@ -141,7 +142,7 @@ public void invalidSnapshotId() { @Test public void invalidExpressionFilter() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"filter\":23,\"projection\":23}")) @@ -151,21 +152,21 @@ public void invalidExpressionFilter() { @Test public void invalidSchema() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"filter\":true,\"schema-id\":\"23\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: schema-id: \"23\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"filter\":true,\"schema-id\":23,\"projected-field-ids\": [\"1\"],\"metrics\":{}}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse integer from non-int value in projected-field-ids: \"1\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ScanReportParser.fromJson( "{\"table-name\":\"roundTripTableName\",\"snapshot-id\":23,\"filter\":true,\"schema-id\":23,\"projected-field-ids\": [1],\"projected-field-names\": [1],\"metrics\":{}}")) @@ -283,8 +284,8 @@ public void roundTripSerde() { + "}"; String json = ScanReportParser.toJson(scanReport, true); - Assertions.assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); + assertThat(json).isEqualTo(expectedJson); } @Test @@ -313,8 +314,8 @@ public void roundTripSerdeWithNoopMetrics() { + "}"; String json = ScanReportParser.toJson(scanReport, true); - Assertions.assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); + assertThat(json).isEqualTo(expectedJson); } @Test @@ -341,8 +342,8 @@ public void roundTripSerdeWithEmptyFieldIdsAndNames() { + "}"; String json = ScanReportParser.toJson(scanReport, true); - Assertions.assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); + assertThat(json).isEqualTo(expectedJson); } @Test @@ -374,7 +375,7 @@ public void roundTripSerdeWithMetadata() { + "}"; String json = ScanReportParser.toJson(scanReport, true); - Assertions.assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(ScanReportParser.fromJson(json)).isEqualTo(scanReport); + assertThat(json).isEqualTo(expectedJson); } } diff --git a/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java b/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java index 3bbf2b88829b..1bcb24997ef4 100644 --- a/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java +++ b/core/src/test/java/org/apache/iceberg/metrics/TestTimerResultParser.java @@ -18,45 +18,46 @@ */ package org.apache.iceberg.metrics; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; import java.time.Duration; import java.time.temporal.ChronoUnit; import java.util.concurrent.TimeUnit; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestTimerResultParser { @Test public void nullTimer() { - Assertions.assertThatThrownBy(() -> TimerResultParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> TimerResultParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse timer from null object"); - Assertions.assertThatThrownBy(() -> TimerResultParser.toJson(null)) + assertThatThrownBy(() -> TimerResultParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid timer: null"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> TimerResultParser.fromJson("{}")) + assertThatThrownBy(() -> TimerResultParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: count"); - Assertions.assertThatThrownBy(() -> TimerResultParser.fromJson("{\"count\":44}")) + assertThatThrownBy(() -> TimerResultParser.fromJson("{\"count\":44}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: time-unit"); - Assertions.assertThatThrownBy( - () -> TimerResultParser.fromJson("{\"count\":44,\"time-unit\":\"hours\"}")) + assertThatThrownBy(() -> TimerResultParser.fromJson("{\"count\":44,\"time-unit\":\"hours\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: total-duration"); } @Test public void extraFields() { - Assertions.assertThat( + assertThat( TimerResultParser.fromJson( "{\"count\":44,\"time-unit\":\"hours\",\"total-duration\":24,\"extra\": \"value\"}")) .isEqualTo(TimerResult.of(TimeUnit.HOURS, Duration.ofHours(24), 44)); @@ -64,7 +65,7 @@ public void extraFields() { @Test public void unsupportedDuration() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TimerResultParser.fromJson( "{\"count\":44,\"time-unit\":\"hours\",\"total-duration\":\"xx\"}")) @@ -74,7 +75,7 @@ public void unsupportedDuration() { @Test public void unsupportedTimeUnit() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TimerResultParser.fromJson( "{\"count\":44,\"time-unit\":\"unknown\",\"total-duration\":24}")) @@ -84,7 +85,7 @@ public void unsupportedTimeUnit() { @Test public void invalidCount() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TimerResultParser.fromJson( "{\"count\":\"illegal\",\"time-unit\":\"hours\",\"total-duration\":24}")) @@ -97,48 +98,41 @@ public void roundTripSerde() { TimerResult timer = TimerResult.of(TimeUnit.HOURS, Duration.ofHours(23), 44); String json = TimerResultParser.toJson(timer); - Assertions.assertThat(TimerResultParser.fromJson(json)).isEqualTo(timer); - Assertions.assertThat(json) - .isEqualTo("{\"count\":44,\"time-unit\":\"hours\",\"total-duration\":23}"); + assertThat(TimerResultParser.fromJson(json)).isEqualTo(timer); + assertThat(json).isEqualTo("{\"count\":44,\"time-unit\":\"hours\",\"total-duration\":23}"); } @Test public void toDuration() { - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.NANOSECONDS)) + assertThat(TimerResultParser.toDuration(5L, TimeUnit.NANOSECONDS)) .isEqualTo(Duration.ofNanos(5L)); - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.MICROSECONDS)) + assertThat(TimerResultParser.toDuration(5L, TimeUnit.MICROSECONDS)) .isEqualTo(Duration.of(5L, ChronoUnit.MICROS)); - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.MILLISECONDS)) + assertThat(TimerResultParser.toDuration(5L, TimeUnit.MILLISECONDS)) .isEqualTo(Duration.ofMillis(5L)); - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.SECONDS)) + assertThat(TimerResultParser.toDuration(5L, TimeUnit.SECONDS)) .isEqualTo(Duration.ofSeconds(5L)); - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.MINUTES)) + assertThat(TimerResultParser.toDuration(5L, TimeUnit.MINUTES)) .isEqualTo(Duration.ofMinutes(5L)); - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.HOURS)) - .isEqualTo(Duration.ofHours(5L)); - Assertions.assertThat(TimerResultParser.toDuration(5L, TimeUnit.DAYS)) - .isEqualTo(Duration.ofDays(5L)); + assertThat(TimerResultParser.toDuration(5L, TimeUnit.HOURS)).isEqualTo(Duration.ofHours(5L)); + assertThat(TimerResultParser.toDuration(5L, TimeUnit.DAYS)).isEqualTo(Duration.ofDays(5L)); } @Test public void fromDuration() { - Assertions.assertThat( - TimerResultParser.fromDuration(Duration.ofNanos(5L), TimeUnit.NANOSECONDS)) + assertThat(TimerResultParser.fromDuration(Duration.ofNanos(5L), TimeUnit.NANOSECONDS)) .isEqualTo(5L); - Assertions.assertThat( + assertThat( TimerResultParser.fromDuration( Duration.of(5L, ChronoUnit.MICROS), TimeUnit.MICROSECONDS)) .isEqualTo(5L); - Assertions.assertThat( - TimerResultParser.fromDuration(Duration.ofMillis(5L), TimeUnit.MILLISECONDS)) - .isEqualTo(5L); - Assertions.assertThat(TimerResultParser.fromDuration(Duration.ofSeconds(5L), TimeUnit.SECONDS)) - .isEqualTo(5L); - Assertions.assertThat(TimerResultParser.fromDuration(Duration.ofMinutes(5L), TimeUnit.MINUTES)) + assertThat(TimerResultParser.fromDuration(Duration.ofMillis(5L), TimeUnit.MILLISECONDS)) .isEqualTo(5L); - Assertions.assertThat(TimerResultParser.fromDuration(Duration.ofHours(5L), TimeUnit.HOURS)) + assertThat(TimerResultParser.fromDuration(Duration.ofSeconds(5L), TimeUnit.SECONDS)) .isEqualTo(5L); - Assertions.assertThat(TimerResultParser.fromDuration(Duration.ofDays(5L), TimeUnit.DAYS)) + assertThat(TimerResultParser.fromDuration(Duration.ofMinutes(5L), TimeUnit.MINUTES)) .isEqualTo(5L); + assertThat(TimerResultParser.fromDuration(Duration.ofHours(5L), TimeUnit.HOURS)).isEqualTo(5L); + assertThat(TimerResultParser.fromDuration(Duration.ofDays(5L), TimeUnit.DAYS)).isEqualTo(5L); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java b/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java index 4855e9a6c779..9c49db60850c 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java +++ b/core/src/test/java/org/apache/iceberg/rest/RequestResponseTestBase.java @@ -18,13 +18,14 @@ */ package org.apache.iceberg.rest; +import static org.assertj.core.api.Assertions.assertThat; + import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.ObjectMapper; import java.util.Collections; import java.util.Set; import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public abstract class RequestResponseTestBase { @@ -75,13 +76,11 @@ public void testHasOnlyKnownFields() { try { JsonNode node = mapper().readValue(serialize(createExampleInstance()), JsonNode.class); for (String field : fieldsFromSpec) { - Assertions.assertThat(node.has(field)).as("Should have field: %s", field).isTrue(); + assertThat(node.has(field)).as("Should have field: %s", field).isTrue(); } for (String field : ((Iterable) node::fieldNames)) { - Assertions.assertThat(fieldsFromSpec) - .as("Should not have field: %s", field) - .contains(field); + assertThat(fieldsFromSpec).as("Should not have field: %s", field).contains(field); } } catch (JsonProcessingException e) { throw new RuntimeException(e); @@ -99,6 +98,6 @@ protected void assertRoundTripSerializesEquallyFrom(String json, T expected) assertEquals(actual, expected); // Check that the deserialized value serializes back into the original JSON - Assertions.assertThat(serialize(expected)).isEqualTo(json); + assertThat(serialize(expected)).isEqualTo(json); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java index ffd10dff932c..1229639aba03 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestHTTPClient.java @@ -19,6 +19,7 @@ package org.apache.iceberg.rest; 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.doThrow; import static org.mockito.Mockito.mock; @@ -51,7 +52,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -154,7 +154,7 @@ public void testProxyServer() throws IOException { @Test public void testProxyCredentialProviderWithoutProxyServer() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> HTTPClient.builder(ImmutableMap.of()) .uri(URI) @@ -166,7 +166,7 @@ public void testProxyCredentialProviderWithoutProxyServer() { @Test public void testProxyServerWithNullHostname() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> HTTPClient.builder(ImmutableMap.of()).uri(URI).withProxy(null, 1070).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid hostname for http client proxy: null"); @@ -212,7 +212,7 @@ public void accept(ErrorResponse errorResponse) { } }; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> clientWithProxy.get("v1/config", Item.class, ImmutableMap.of(), onError)) .isInstanceOf(RuntimeException.class) .hasMessage( @@ -270,7 +270,7 @@ public void testSocketTimeout() throws IOException { .withDelay(TimeUnit.MILLISECONDS, 5000); mockServer.when(mockRequest).respond(mockResponse); - Assertions.assertThatThrownBy(() -> client.head(path, ImmutableMap.of(), (unused) -> {})) + assertThatThrownBy(() -> client.head(path, ImmutableMap.of(), (unused) -> {})) .cause() .isInstanceOf(SocketTimeoutException.class) .hasMessage("Read timed out"); @@ -281,7 +281,7 @@ public void testSocketTimeout() throws IOException { @ValueSource(strings = {HTTPClient.REST_CONNECTION_TIMEOUT_MS, HTTPClient.REST_SOCKET_TIMEOUT_MS}) public void testInvalidTimeout(String timeoutMsType) { String invalidTimeoutMs = "invalidMs"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> HTTPClient.builder(ImmutableMap.of(timeoutMsType, invalidTimeoutMs)) .uri(URI) @@ -290,7 +290,7 @@ public void testInvalidTimeout(String timeoutMsType) { .hasMessage(String.format("For input string: \"%s\"", invalidTimeoutMs)); String invalidNegativeTimeoutMs = "-1"; - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> HTTPClient.builder(ImmutableMap.of(timeoutMsType, invalidNegativeTimeoutMs)) .uri(URI) @@ -312,7 +312,7 @@ public static void testHttpMethodOnSuccess(HttpMethod method) throws JsonProcess doExecuteRequest(method, path, body, onError, h -> assertThat(h).isNotEmpty()); if (method.usesRequestBody()) { - Assertions.assertThat(body) + assertThat(body) .as("On a successful " + method + ", the correct response body should be returned") .isEqualTo(successResponse); } @@ -335,7 +335,7 @@ public static void testHttpMethodOnFailure(HttpMethod method) throws JsonProcess String path = addRequestTestCaseAndGetPath(method, body, statusCode); - Assertions.assertThatThrownBy(() -> doExecuteRequest(method, path, body, onError, h -> {})) + assertThatThrownBy(() -> doExecuteRequest(method, path, body, onError, h -> {})) .isInstanceOf(RuntimeException.class) .hasMessage( String.format( 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 34d088cb60d5..6f79733179ed 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -84,7 +84,6 @@ import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.rest.responses.OAuthTokenResponse; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.assertj.core.api.InstanceOfAssertFactories; import org.awaitility.Awaitility; import org.eclipse.jetty.server.Server; @@ -292,15 +291,15 @@ public T get( restCat.setConf(new Configuration()); restCat.initialize("prod", initialConfig); - Assertions.assertThat(restCat.properties().get(CatalogProperties.CACHE_ENABLED)) + assertThat(restCat.properties().get(CatalogProperties.CACHE_ENABLED)) .as("Catalog properties after initialize should use the server's override properties") .isEqualTo("false"); - Assertions.assertThat(restCat.properties().get(CatalogProperties.CLIENT_POOL_SIZE)) + assertThat(restCat.properties().get(CatalogProperties.CLIENT_POOL_SIZE)) .as("Catalog after initialize should use the server's default properties if not specified") .isEqualTo("1"); - Assertions.assertThat(restCat.properties().get(CatalogProperties.WAREHOUSE_LOCATION)) + assertThat(restCat.properties().get(CatalogProperties.WAREHOUSE_LOCATION)) .as("Catalog should return final warehouse location") .isEqualTo("s3://bucket/warehouse"); @@ -310,12 +309,11 @@ public T get( @Test public void testInitializeWithBadArguments() throws IOException { RESTCatalog restCat = new RESTCatalog(); - org.assertj.core.api.Assertions.assertThatThrownBy(() -> restCat.initialize("prod", null)) + assertThatThrownBy(() -> restCat.initialize("prod", null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid configuration: null"); - org.assertj.core.api.Assertions.assertThatThrownBy( - () -> restCat.initialize("prod", ImmutableMap.of())) + assertThatThrownBy(() -> restCat.initialize("prod", ImmutableMap.of())) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid uri for http client: null"); @@ -333,7 +331,7 @@ public void testCatalogBasicBearerToken() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", "bearer-token")); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // the bearer token should be used for all interactions Mockito.verify(adapter) @@ -369,7 +367,7 @@ public void testCatalogCredentialNoOauth2ServerUri() { catalog.initialize( "prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "credential", "catalog:secret")); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // no token or credential for catalog token exchange Mockito.verify(adapter) @@ -424,7 +422,7 @@ public void testCatalogCredential(String oauth2ServerUri) { OAuth2Properties.OAUTH2_SERVER_URI, oauth2ServerUri)); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // no token or credential for catalog token exchange Mockito.verify(adapter) @@ -485,7 +483,7 @@ public void testCatalogBearerTokenWithClientCredential(String oauth2ServerUri) { OAuth2Properties.OAUTH2_SERVER_URI, oauth2ServerUri)); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // use the bearer token for config Mockito.verify(adapter) @@ -548,7 +546,7 @@ public void testCatalogCredentialWithClientCredential(String oauth2ServerUri) { OAuth2Properties.OAUTH2_SERVER_URI, oauth2ServerUri)); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -623,7 +621,7 @@ public void testCatalogBearerTokenAndCredentialWithClientCredential(String oauth OAuth2Properties.OAUTH2_SERVER_URI, oauth2ServerUri)); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // use the bearer token for client credentials Mockito.verify(adapter) @@ -819,7 +817,7 @@ private void testClientAuth( .build(); catalog.initialize("prod", initializationProperties); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); Mockito.verify(adapter) .execute( @@ -971,7 +969,7 @@ public void testTableSnapshotLoading() { // don't call snapshots() directly as that would cause to load all snapshots. Instead, // make sure the snapshots field holds exactly 1 snapshot - Assertions.assertThat(refsMetadata) + assertThat(refsMetadata) .extracting("snapshots") .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class)) .hasSize(1); @@ -1097,7 +1095,7 @@ public void testTableSnapshotLoadingWithDivergedBranches(String formatVersion) { // don't call snapshots() directly as that would cause to load all snapshots. Instead, // make sure the snapshots field holds exactly 2 snapshots (the latest snapshot for main // and the branch) - Assertions.assertThat(refsMetadata) + assertThat(refsMetadata) .extracting("snapshots") .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class)) .hasSize(2); @@ -1213,7 +1211,7 @@ public void lazySnapshotLoadingWithDivergedHistory() { // don't call snapshots() directly as that would cause to load all snapshots. Instead, // make sure the snapshots field holds exactly 1 snapshot - Assertions.assertThat(refsMetadata) + assertThat(refsMetadata) .extracting("snapshots") .asInstanceOf(InstanceOfAssertFactories.list(Snapshot.class)) .hasSize(1); @@ -1312,12 +1310,12 @@ public void testTableAuth( } Table table = catalog.createTable(ident, expectedSchema); - Assertions.assertThat(table.schema().asStruct()) + assertThat(table.schema().asStruct()) .as("Schema should match") .isEqualTo(expectedSchema.asStruct()); Table loaded = catalog.loadTable(ident); // the first load will send the token - Assertions.assertThat(loaded.schema().asStruct()) + assertThat(loaded.schema().asStruct()) .as("Schema should match") .isEqualTo(expectedSchema.asStruct()); @@ -1570,8 +1568,7 @@ public void testCatalogRefreshedTokenIsUsed(String oauth2ServerUri) { .untilAsserted( () -> { // use the exchanged catalog token - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))) - .isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -1732,7 +1729,7 @@ public void testCatalogExpiredBearerTokenIsRefreshedWithCredential(String oauth2 OAuth2Properties.OAUTH2_SERVER_URI, oauth2ServerUri)); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -1818,7 +1815,7 @@ public void testCatalogValidBearerTokenIsNotRefreshed() { RESTCatalog catalog = new RESTCatalog(context, (config) -> adapter); catalog.initialize("prod", ImmutableMap.of(CatalogProperties.URI, "ignored", "token", token)); - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); Mockito.verify(adapter) .execute( @@ -1919,8 +1916,7 @@ public void testCatalogTokenRefreshFailsAndUsesCredentialForRefresh(String oauth .untilAsserted( () -> { // use the exchanged catalog token - Assertions.assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))) - .isFalse(); + assertThat(catalog.tableExists(TableIdentifier.of("ns", "table"))).isFalse(); // call client credentials with no initial auth Mockito.verify(adapter) @@ -2337,7 +2333,7 @@ public void testInvalidPageSize() { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.initialize( "test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "-1"))) @@ -2505,8 +2501,7 @@ public void testCleanupUncommitedFilesForCleanableFailures() { // cleaned up UpdateTableRequest request = captor.getValue(); MetadataUpdate.AddSnapshot addSnapshot = (MetadataUpdate.AddSnapshot) request.updates().get(0); - Assertions.assertThatThrownBy( - () -> table.io().newInputFile(addSnapshot.snapshot().manifestListLocation())) + assertThatThrownBy(() -> table.io().newInputFile(addSnapshot.snapshot().manifestListLocation())) .isInstanceOf(NotFoundException.class); } @@ -2535,8 +2530,7 @@ public void testNoCleanupForNonCleanableExceptions() { // exist even though the commit failed UpdateTableRequest request = captor.getValue(); MetadataUpdate.AddSnapshot addSnapshot = (MetadataUpdate.AddSnapshot) request.updates().get(0); - Assertions.assertThat( - table.io().newInputFile(addSnapshot.snapshot().manifestListLocation()).exists()) + assertThat(table.io().newInputFile(addSnapshot.snapshot().manifestListLocation()).exists()) .isTrue(); } @@ -2570,7 +2564,7 @@ public void testCleanupCleanableExceptionsCreate() { assertThat(appendSnapshot).isPresent(); MetadataUpdate.AddSnapshot addSnapshot = (MetadataUpdate.AddSnapshot) appendSnapshot.get(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog .loadTable(TABLE) @@ -2608,7 +2602,7 @@ public void testNoCleanupForNonCleanableCreateTransaction() { assertThat(appendSnapshot).isPresent(); MetadataUpdate.AddSnapshot addSnapshot = (MetadataUpdate.AddSnapshot) appendSnapshot.get(); - Assertions.assertThat( + assertThat( catalog .loadTable(TABLE) .io() @@ -2646,7 +2640,7 @@ public void testCleanupCleanableExceptionsReplace() { assertThat(appendSnapshot).isPresent(); MetadataUpdate.AddSnapshot addSnapshot = (MetadataUpdate.AddSnapshot) appendSnapshot.get(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog .loadTable(TABLE) @@ -2683,7 +2677,7 @@ public void testNoCleanupForNonCleanableReplaceTransaction() { assertThat(appendSnapshot).isPresent(); MetadataUpdate.AddSnapshot addSnapshot = (MetadataUpdate.AddSnapshot) appendSnapshot.get(); - Assertions.assertThat( + assertThat( catalog .loadTable(TABLE) .io() diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java index 680a8bcaa377..c7667d90ac6f 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTUtil.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.rest; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import java.util.Map; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestRESTUtil { @@ -45,7 +47,7 @@ public void testExtractPrefixMap() { Map actual = RESTUtil.extractPrefixMap(input, "rest."); - Assertions.assertThat(actual).isEqualTo(expected); + assertThat(actual).isEqualTo(expected); } @Test @@ -61,7 +63,7 @@ public void testStripTrailingSlash() { for (String[] testCase : testCases) { String input = testCase[0]; String expected = testCase[1]; - Assertions.assertThat(RESTUtil.stripTrailingSlash(input)).isEqualTo(expected); + assertThat(RESTUtil.stripTrailingSlash(input)).isEqualTo(expected); } } @@ -87,21 +89,21 @@ public void testRoundTripUrlEncodeDecodeNamespace() { Namespace namespace = Namespace.of(levels); // To be placed into a URL path as query parameter or path parameter - Assertions.assertThat(RESTUtil.encodeNamespace(namespace)).isEqualTo(encodedNs); + assertThat(RESTUtil.encodeNamespace(namespace)).isEqualTo(encodedNs); // Decoded (after pulling as String) from URL Namespace asNamespace = RESTUtil.decodeNamespace(encodedNs); - Assertions.assertThat(asNamespace).isEqualTo(namespace); + assertThat(asNamespace).isEqualTo(namespace); } } @Test public void testNamespaceUrlEncodeDecodeDoesNotAllowNull() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> RESTUtil.encodeNamespace(null)) .withMessage("Invalid namespace: null"); - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> RESTUtil.decodeNamespace(null)) .withMessage("Invalid namespace: null"); } @@ -113,7 +115,7 @@ public void testOAuth2URLEncoding() { String utf8 = "\u0020\u0025\u0026\u002B\u00A3\u20AC"; String expected = "+%25%26%2B%C2%A3%E2%82%AC"; - Assertions.assertThat(RESTUtil.encodeString(utf8)).isEqualTo(expected); + assertThat(RESTUtil.encodeString(utf8)).isEqualTo(expected); } @Test @@ -124,7 +126,7 @@ public void testOAuth2FormDataEncoding() { Map formData = ImmutableMap.of("client_id", "12345", "client_secret", utf8); String expected = "client_id=12345&client_secret=" + asString; - Assertions.assertThat(RESTUtil.encodeFormData(formData)).isEqualTo(expected); + assertThat(RESTUtil.encodeFormData(formData)).isEqualTo(expected); } @Test @@ -135,6 +137,6 @@ public void testOAuth2FormDataDecoding() { Map expected = ImmutableMap.of("client_id", "12345", "client_secret", utf8); String formString = "client_id=12345&client_secret=" + asString; - Assertions.assertThat(RESTUtil.decodeFormData(formString)).isEqualTo(expected); + assertThat(RESTUtil.decodeFormData(formString)).isEqualTo(expected); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java index 4b91fbbad380..c2d3fe560345 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestResourcePaths.java @@ -18,10 +18,11 @@ */ package org.apache.iceberg.rest; +import static org.assertj.core.api.Assertions.assertThat; + import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestResourcePaths { @@ -33,161 +34,149 @@ public class TestResourcePaths { @Test public void testConfigPath() { // prefix does not affect the config route because config is merged into catalog properties - Assertions.assertThat(ResourcePaths.config()).isEqualTo("v1/config"); + assertThat(ResourcePaths.config()).isEqualTo("v1/config"); } @Test public void testNamespaces() { - Assertions.assertThat(withPrefix.namespaces()).isEqualTo("v1/ws/catalog/namespaces"); - Assertions.assertThat(withoutPrefix.namespaces()).isEqualTo("v1/namespaces"); + assertThat(withPrefix.namespaces()).isEqualTo("v1/ws/catalog/namespaces"); + assertThat(withoutPrefix.namespaces()).isEqualTo("v1/namespaces"); } @Test public void testNamespace() { Namespace ns = Namespace.of("ns"); - Assertions.assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/ns"); - Assertions.assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/ns"); + assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/ns"); + assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/ns"); } @Test public void testNamespaceWithSlash() { Namespace ns = Namespace.of("n/s"); - Assertions.assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs"); - Assertions.assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%2Fs"); + assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs"); + assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%2Fs"); } @Test public void testNamespaceWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assertions.assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs"); - Assertions.assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%1Fs"); + assertThat(withPrefix.namespace(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs"); + assertThat(withoutPrefix.namespace(ns)).isEqualTo("v1/namespaces/n%1Fs"); } @Test public void testNamespaceProperties() { Namespace ns = Namespace.of("ns"); - Assertions.assertThat(withPrefix.namespaceProperties(ns)) + assertThat(withPrefix.namespaceProperties(ns)) .isEqualTo("v1/ws/catalog/namespaces/ns/properties"); - Assertions.assertThat(withoutPrefix.namespaceProperties(ns)) - .isEqualTo("v1/namespaces/ns/properties"); + assertThat(withoutPrefix.namespaceProperties(ns)).isEqualTo("v1/namespaces/ns/properties"); } @Test public void testNamespacePropertiesWithSlash() { Namespace ns = Namespace.of("n/s"); - Assertions.assertThat(withPrefix.namespaceProperties(ns)) + assertThat(withPrefix.namespaceProperties(ns)) .isEqualTo("v1/ws/catalog/namespaces/n%2Fs/properties"); - Assertions.assertThat(withoutPrefix.namespaceProperties(ns)) - .isEqualTo("v1/namespaces/n%2Fs/properties"); + assertThat(withoutPrefix.namespaceProperties(ns)).isEqualTo("v1/namespaces/n%2Fs/properties"); } @Test public void testNamespacePropertiesWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assertions.assertThat(withPrefix.namespaceProperties(ns)) + assertThat(withPrefix.namespaceProperties(ns)) .isEqualTo("v1/ws/catalog/namespaces/n%1Fs/properties"); - Assertions.assertThat(withoutPrefix.namespaceProperties(ns)) - .isEqualTo("v1/namespaces/n%1Fs/properties"); + assertThat(withoutPrefix.namespaceProperties(ns)).isEqualTo("v1/namespaces/n%1Fs/properties"); } @Test public void testTables() { Namespace ns = Namespace.of("ns"); - Assertions.assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/tables"); - Assertions.assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/ns/tables"); + assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/tables"); + assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/ns/tables"); } @Test public void testTablesWithSlash() { Namespace ns = Namespace.of("n/s"); - Assertions.assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/tables"); - Assertions.assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/n%2Fs/tables"); + assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/tables"); + assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/n%2Fs/tables"); } @Test public void testTablesWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assertions.assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/tables"); - Assertions.assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/n%1Fs/tables"); + assertThat(withPrefix.tables(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/tables"); + assertThat(withoutPrefix.tables(ns)).isEqualTo("v1/namespaces/n%1Fs/tables"); } @Test public void testTable() { TableIdentifier ident = TableIdentifier.of("ns", "table"); - Assertions.assertThat(withPrefix.table(ident)) - .isEqualTo("v1/ws/catalog/namespaces/ns/tables/table"); - Assertions.assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/ns/tables/table"); + assertThat(withPrefix.table(ident)).isEqualTo("v1/ws/catalog/namespaces/ns/tables/table"); + assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/ns/tables/table"); } @Test public void testTableWithSlash() { TableIdentifier ident = TableIdentifier.of("n/s", "tab/le"); - Assertions.assertThat(withPrefix.table(ident)) - .isEqualTo("v1/ws/catalog/namespaces/n%2Fs/tables/tab%2Fle"); - Assertions.assertThat(withoutPrefix.table(ident)) - .isEqualTo("v1/namespaces/n%2Fs/tables/tab%2Fle"); + assertThat(withPrefix.table(ident)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/tables/tab%2Fle"); + assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/n%2Fs/tables/tab%2Fle"); } @Test public void testTableWithMultipartNamespace() { TableIdentifier ident = TableIdentifier.of("n", "s", "table"); - Assertions.assertThat(withPrefix.table(ident)) - .isEqualTo("v1/ws/catalog/namespaces/n%1Fs/tables/table"); - Assertions.assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/n%1Fs/tables/table"); + assertThat(withPrefix.table(ident)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/tables/table"); + assertThat(withoutPrefix.table(ident)).isEqualTo("v1/namespaces/n%1Fs/tables/table"); } @Test public void testRegister() { Namespace ns = Namespace.of("ns"); - Assertions.assertThat(withPrefix.register(ns)) - .isEqualTo("v1/ws/catalog/namespaces/ns/register"); - Assertions.assertThat(withoutPrefix.register(ns)).isEqualTo("v1/namespaces/ns/register"); + assertThat(withPrefix.register(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/register"); + assertThat(withoutPrefix.register(ns)).isEqualTo("v1/namespaces/ns/register"); } @Test public void views() { Namespace ns = Namespace.of("ns"); - Assertions.assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/views"); - Assertions.assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/ns/views"); + assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/ns/views"); + assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/ns/views"); } @Test public void viewsWithSlash() { Namespace ns = Namespace.of("n/s"); - Assertions.assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/views"); - Assertions.assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/n%2Fs/views"); + assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/n%2Fs/views"); + assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/n%2Fs/views"); } @Test public void viewsWithMultipartNamespace() { Namespace ns = Namespace.of("n", "s"); - Assertions.assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views"); - Assertions.assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/n%1Fs/views"); + assertThat(withPrefix.views(ns)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views"); + assertThat(withoutPrefix.views(ns)).isEqualTo("v1/namespaces/n%1Fs/views"); } @Test public void view() { TableIdentifier ident = TableIdentifier.of("ns", "view-name"); - Assertions.assertThat(withPrefix.view(ident)) - .isEqualTo("v1/ws/catalog/namespaces/ns/views/view-name"); - Assertions.assertThat(withoutPrefix.view(ident)).isEqualTo("v1/namespaces/ns/views/view-name"); + assertThat(withPrefix.view(ident)).isEqualTo("v1/ws/catalog/namespaces/ns/views/view-name"); + assertThat(withoutPrefix.view(ident)).isEqualTo("v1/namespaces/ns/views/view-name"); } @Test public void viewWithSlash() { TableIdentifier ident = TableIdentifier.of("n/s", "vi/ew-name"); - Assertions.assertThat(withPrefix.view(ident)) + assertThat(withPrefix.view(ident)) .isEqualTo("v1/ws/catalog/namespaces/n%2Fs/views/vi%2Few-name"); - Assertions.assertThat(withoutPrefix.view(ident)) - .isEqualTo("v1/namespaces/n%2Fs/views/vi%2Few-name"); + assertThat(withoutPrefix.view(ident)).isEqualTo("v1/namespaces/n%2Fs/views/vi%2Few-name"); } @Test public void viewWithMultipartNamespace() { TableIdentifier ident = TableIdentifier.of("n", "s", "view-name"); - Assertions.assertThat(withPrefix.view(ident)) - .isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views/view-name"); - Assertions.assertThat(withoutPrefix.view(ident)) - .isEqualTo("v1/namespaces/n%1Fs/views/view-name"); + assertThat(withPrefix.view(ident)).isEqualTo("v1/ws/catalog/namespaces/n%1Fs/views/view-name"); + assertThat(withoutPrefix.view(ident)).isEqualTo("v1/namespaces/n%1Fs/views/view-name"); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java index 8b849b2cfb44..313f5e19d3ab 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateNamespaceRequest.java @@ -18,13 +18,15 @@ */ package org.apache.iceberg.rest.requests; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.core.JsonProcessingException; import java.util.Map; import org.apache.iceberg.catalog.Namespace; 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.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCreateNamespaceRequest extends RequestResponseTestBase { @@ -76,49 +78,46 @@ public void testCanDeserializeWithoutDefaultValues() throws JsonProcessingExcept public void testDeserializeInvalidRequest() { String jsonIncorrectTypeForNamespace = "{\"namespace\":\"accounting%1Ftax\",\"properties\":null}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonIncorrectTypeForNamespace)) + assertThatThrownBy(() -> deserialize(jsonIncorrectTypeForNamespace)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot parse string array from non-array"); String jsonIncorrectTypeForProperties = "{\"namespace\":[\"accounting\",\"tax\"],\"properties\":[]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonIncorrectTypeForProperties)) + assertThatThrownBy(() -> deserialize(jsonIncorrectTypeForProperties)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot deserialize value of type"); String jsonMisspelledKeys = "{\"namepsace\":[\"accounting\",\"tax\"],\"propertiezzzz\":{\"owner\":\"Hank\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonMisspelledKeys)) + assertThatThrownBy(() -> deserialize(jsonMisspelledKeys)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); String emptyJson = "{}"; - Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + assertThatThrownBy(() -> deserialize(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); - Assertions.assertThatThrownBy(() -> deserialize(null)) + assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - Assertions.assertThatThrownBy( - () -> CreateNamespaceRequest.builder().withNamespace(null).build()) + assertThatThrownBy(() -> CreateNamespaceRequest.builder().withNamespace(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid namespace: null"); - Assertions.assertThatThrownBy( - () -> CreateNamespaceRequest.builder().setProperties(null).build()) + assertThatThrownBy(() -> CreateNamespaceRequest.builder().setProperties(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid collection of properties: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - Assertions.assertThatThrownBy( - () -> CreateNamespaceRequest.builder().setProperties(mapWithNullKey).build()) + assertThatThrownBy(() -> CreateNamespaceRequest.builder().setProperties(mapWithNullKey).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property: null"); @@ -126,7 +125,7 @@ public void testBuilderDoesNotBuildInvalidRequests() { mapWithMultipleNullValues.put("a", null); mapWithMultipleNullValues.put("b", "b"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CreateNamespaceRequest.builder().setProperties(mapWithMultipleNullValues).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid value for properties [a]: null"); @@ -147,8 +146,8 @@ public CreateNamespaceRequest createExampleInstance() { @Override public void assertEquals(CreateNamespaceRequest actual, CreateNamespaceRequest expected) { - Assertions.assertThat(actual.namespace()).isEqualTo(expected.namespace()); - Assertions.assertThat(actual.properties()).isEqualTo(expected.properties()); + assertThat(actual.namespace()).isEqualTo(expected.namespace()); + assertThat(actual.properties()).isEqualTo(expected.properties()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java index a5895d380708..0d4280c25a79 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestCreateTableRequest.java @@ -20,6 +20,8 @@ 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 com.fasterxml.jackson.core.JsonProcessingException; import java.util.Map; @@ -34,7 +36,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.rest.RequestResponseTestBase; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCreateTableRequest extends RequestResponseTestBase { @@ -167,7 +168,7 @@ public void testDeserializeInvalidRequest() { String jsonMissingSchema = "{\"name\":\"foo\",\"location\":null,\"partition-spec\":null,\"write-order\":null,\"properties\":{}," + "\"stage-create\":false}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonMissingSchema)) + assertThatThrownBy(() -> deserialize(jsonMissingSchema)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid schema: null"); @@ -176,7 +177,7 @@ public void testDeserializeInvalidRequest() { "{\"location\":null,\"schema\":%s,\"spec\":null,\"write-order\":null,\"properties\":{}," + "\"stage-create\":false}", SAMPLE_SCHEMA_JSON); - Assertions.assertThatThrownBy(() -> deserialize(jsonMissingName)) + assertThatThrownBy(() -> deserialize(jsonMissingName)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid table name: null"); @@ -185,53 +186,52 @@ public void testDeserializeInvalidRequest() { "{\"name\":\"foo\",\"location\":null,\"schema\":%s,\"partition-spec\":null,\"write-order\":null," + "\"properties\":[],\"stage-create\":false}", SAMPLE_SCHEMA_JSON); - Assertions.assertThatThrownBy(() -> deserialize(jsonIncorrectTypeForProperties)) + assertThatThrownBy(() -> deserialize(jsonIncorrectTypeForProperties)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot deserialize value of type"); - Assertions.assertThatThrownBy(() -> deserialize("{}")) + assertThatThrownBy(() -> deserialize("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid table name: null"); - Assertions.assertThatThrownBy(() -> deserialize(null)) + assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - Assertions.assertThatThrownBy(() -> CreateTableRequest.builder().withName(null)) + assertThatThrownBy(() -> CreateTableRequest.builder().withName(null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid name: null"); - Assertions.assertThatThrownBy(() -> CreateTableRequest.builder().withSchema(null)) + assertThatThrownBy(() -> CreateTableRequest.builder().withSchema(null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid schema: null"); - Assertions.assertThatThrownBy(() -> CreateTableRequest.builder().setProperties(null)) + assertThatThrownBy(() -> CreateTableRequest.builder().setProperties(null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid collection of properties: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - Assertions.assertThatThrownBy(() -> CreateTableRequest.builder().setProperties(mapWithNullKey)) + assertThatThrownBy(() -> CreateTableRequest.builder().setProperties(mapWithNullKey)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property: null"); Map mapWithNullValue = Maps.newHashMap(); mapWithNullValue.put("a", null); mapWithNullValue.put("b", "b"); - Assertions.assertThatThrownBy( - () -> CreateTableRequest.builder().setProperties(mapWithNullValue).build()) + assertThatThrownBy(() -> CreateTableRequest.builder().setProperties(mapWithNullValue).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid value for properties [a]: null"); - Assertions.assertThatThrownBy(() -> CreateTableRequest.builder().setProperty("foo", null)) + assertThatThrownBy(() -> CreateTableRequest.builder().setProperty("foo", null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid value for property foo: null"); - Assertions.assertThatThrownBy(() -> CreateTableRequest.builder().setProperty(null, "foo")) + assertThatThrownBy(() -> CreateTableRequest.builder().setProperty(null, "foo")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property: null"); } @@ -258,25 +258,23 @@ public CreateTableRequest createExampleInstance() { @Override public void assertEquals(CreateTableRequest actual, CreateTableRequest expected) { - Assertions.assertThat(actual.name()).as("Name should be the same").isEqualTo(expected.name()); - Assertions.assertThat(actual.location()) + assertThat(actual.name()).as("Name should be the same").isEqualTo(expected.name()); + assertThat(actual.location()) .as("Location should be the same if provided") .isEqualTo(expected.location()); - Assertions.assertThat( + assertThat( expected.schema().sameSchema(actual.schema()) && expected.schema().schemaId() == actual.schema().schemaId()) .as("Schemas should be equivalent and have same schema id") .isTrue(); - Assertions.assertThat(actual.spec()) - .as("Partition spec should be equal") - .isEqualTo(expected.spec()); - Assertions.assertThat(actual.writeOrder()) + assertThat(actual.spec()).as("Partition spec should be equal").isEqualTo(expected.spec()); + assertThat(actual.writeOrder()) .as("Write [sort] order should be the same") .isEqualTo(expected.writeOrder()); - Assertions.assertThat(actual.properties()) + assertThat(actual.properties()) .as("Properties should be the same") .isEqualTo(expected.properties()); - Assertions.assertThat(actual.stageCreate()) + assertThat(actual.stageCreate()) .as("Stage create should be equal") .isEqualTo(expected.stageCreate()); } diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java index 9b479d89d7d7..50a47df974a3 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestRegisterTableRequestParser.java @@ -19,36 +19,35 @@ package org.apache.iceberg.rest.requests; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import com.fasterxml.jackson.databind.JsonNode; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestRegisterTableRequestParser { @Test public void nullCheck() { - Assertions.assertThatThrownBy(() -> RegisterTableRequestParser.toJson(null)) + assertThatThrownBy(() -> RegisterTableRequestParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid register table request: null"); - Assertions.assertThatThrownBy(() -> RegisterTableRequestParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> RegisterTableRequestParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse register table request from null object"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> RegisterTableRequestParser.fromJson("{}")) + assertThatThrownBy(() -> RegisterTableRequestParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: name"); - Assertions.assertThatThrownBy( - () -> RegisterTableRequestParser.fromJson("{\"name\" : \"test_tbl\"}")) + assertThatThrownBy(() -> RegisterTableRequestParser.fromJson("{\"name\" : \"test_tbl\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: metadata-location"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> RegisterTableRequestParser.fromJson( "{\"metadata-location\" : \"file://tmp/NS/test_tbl/metadata/00000-d4f60d2f-2ad2-408b-8832-0ed7fbd851ee.metadata.json\"}")) diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java index aeff8755aab3..de236a913619 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestRenameTableRequest.java @@ -18,12 +18,14 @@ */ package org.apache.iceberg.rest.requests; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.core.JsonProcessingException; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.catalog.TableIdentifierParser; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestRenameTableRequest extends RequestResponseTestBase { @@ -51,44 +53,44 @@ public void testDeserializeInvalidRequestThrows() { String jsonSourceNullName = "{\"source\":{\"namespace\":[\"accounting\",\"tax\"],\"name\":null}," + "\"destination\":{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid_2022\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonSourceNullName)) + assertThatThrownBy(() -> deserialize(jsonSourceNullName)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot parse to a string value: name: null"); String jsonDestinationNullName = "{\"source\":{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}," + "\"destination\":{\"namespace\":[\"accounting\",\"tax\"],\"name\":null}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonDestinationNullName)) + assertThatThrownBy(() -> deserialize(jsonDestinationNullName)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot parse to a string value: name: null"); String jsonSourceMissingName = "{\"source\":{\"namespace\":[\"accounting\",\"tax\"]}," + "\"destination\":{\"name\":\"paid_2022\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonSourceMissingName)) + assertThatThrownBy(() -> deserialize(jsonSourceMissingName)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot parse missing string: name"); String jsonDestinationMissingName = "{\"source\":{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}," + "\"destination\":{\"namespace\":[\"accounting\",\"tax\"]}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonDestinationMissingName)) + assertThatThrownBy(() -> deserialize(jsonDestinationMissingName)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot parse missing string: name"); String emptyJson = "{}"; - Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + assertThatThrownBy(() -> deserialize(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid source table: null"); - Assertions.assertThatThrownBy(() -> deserialize(null)) + assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> RenameTableRequest.builder() .withSource(null) @@ -97,7 +99,7 @@ public void testBuilderDoesNotBuildInvalidRequests() { .isInstanceOf(NullPointerException.class) .hasMessage("Invalid source table identifier: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> RenameTableRequest.builder().withSource(TAX_PAID).withDestination(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid destination table identifier: null"); @@ -118,10 +120,10 @@ public RenameTableRequest createExampleInstance() { @Override public void assertEquals(RenameTableRequest actual, RenameTableRequest expected) { - Assertions.assertThat(actual.source()) + assertThat(actual.source()) .as("Source table identifier should be equal") .isEqualTo(expected.source()); - Assertions.assertThat(actual.destination()) + assertThat(actual.destination()) .as("Destination table identifier should be equal") .isEqualTo(expected.destination()); } diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java index f04938908b64..b0f4306655ff 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestReportMetricsRequestParser.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.rest.requests; +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.expressions.Expressions; import org.apache.iceberg.metrics.CommitMetrics; @@ -30,34 +33,33 @@ import org.apache.iceberg.metrics.ScanMetricsResult; import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestReportMetricsRequestParser { @Test public void nullCheck() { - Assertions.assertThatThrownBy(() -> ReportMetricsRequestParser.toJson(null)) + assertThatThrownBy(() -> ReportMetricsRequestParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid metrics request: null"); - Assertions.assertThatThrownBy(() -> ReportMetricsRequestParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> ReportMetricsRequestParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse metrics request from null object"); } @Test public void missingFields() { - Assertions.assertThatThrownBy(() -> ReportMetricsRequestParser.fromJson("{}")) + assertThatThrownBy(() -> ReportMetricsRequestParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: report-type"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ReportMetricsRequestParser.fromJson("{\"report-type\":\"scan-report\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: table-name"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ReportMetricsRequestParser.fromJson( "{\"report-type\":\"scan-report\", \"table-name\" : \"x\"}")) @@ -67,11 +69,10 @@ public void missingFields() { @Test public void invalidReportType() { - Assertions.assertThat( - ReportMetricsRequestParser.fromJson("{\"report-type\":\"invalid\"}").reportType()) + assertThat(ReportMetricsRequestParser.fromJson("{\"report-type\":\"invalid\"}").reportType()) .isEqualTo(ReportMetricsRequest.unknown().reportType()); - Assertions.assertThat( + assertThat( ReportMetricsRequestParser.fromJson( ReportMetricsRequestParser.toJson( ReportMetricsRequest.of(new MetricsReport() {}))) @@ -93,8 +94,7 @@ public void invalidReportType() { + "}"; ReportMetricsRequest request = ReportMetricsRequestParser.fromJson(json); - Assertions.assertThat(request.reportType()) - .isEqualTo(ReportMetricsRequest.unknown().reportType()); + assertThat(request.reportType()).isEqualTo(ReportMetricsRequest.unknown().reportType()); } @Test @@ -126,9 +126,9 @@ public void roundTripSerdeWithScanReport() { ReportMetricsRequest metricsRequest = ReportMetricsRequest.of(scanReport); String json = ReportMetricsRequestParser.toJson(metricsRequest, true); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(json).isEqualTo(expectedJson); - Assertions.assertThat(ReportMetricsRequestParser.fromJson(json).report()) + assertThat(ReportMetricsRequestParser.fromJson(json).report()) .isEqualTo(metricsRequest.report()); } @@ -157,9 +157,9 @@ public void roundTripSerdeWithCommitReport() { ReportMetricsRequest metricsRequest = ReportMetricsRequest.of(commitReport); String json = ReportMetricsRequestParser.toJson(metricsRequest, true); - Assertions.assertThat(json).isEqualTo(expectedJson); + assertThat(json).isEqualTo(expectedJson); - Assertions.assertThat(ReportMetricsRequestParser.fromJson(json).report()) + assertThat(ReportMetricsRequestParser.fromJson(json).report()) .isEqualTo(metricsRequest.report()); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java b/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java index daef718e1510..38e04ab2348a 100644 --- a/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java +++ b/core/src/test/java/org/apache/iceberg/rest/requests/TestUpdateNamespacePropertiesRequest.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.rest.requests; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.core.JsonProcessingException; import java.util.List; import java.util.Map; @@ -27,7 +30,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestUpdateNamespacePropertiesRequest @@ -117,13 +119,13 @@ public void testParseInvalidJson() { // Invalid top-level types String jsonInvalidTypeOnRemovalField = "{\"removals\":{\"foo\":\"bar\"},\"updates\":{\"owner\":\"Hank\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnRemovalField)) + assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnRemovalField)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot deserialize value of type"); String jsonInvalidTypeOnUpdatesField = "{\"removals\":[\"foo\":\"bar\"],\"updates\":[\"owner\"]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnUpdatesField)) + assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnUpdatesField)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Unexpected character") .hasMessageContaining("expecting comma to separate Array entries"); @@ -133,52 +135,48 @@ public void testParseInvalidJson() { // e.g. { removals: [ "foo", "bar", 1234 ] } will parse correctly. String invalidJsonWrongTypeInRemovalsList = "{\"removals\":[\"foo\",\"bar\", {\"owner\": \"Hank\"}],\"updates\":{\"owner\":\"Hank\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(invalidJsonWrongTypeInRemovalsList)) + assertThatThrownBy(() -> deserialize(invalidJsonWrongTypeInRemovalsList)) .isInstanceOf(JsonProcessingException.class) .hasMessageStartingWith("Cannot deserialize value of type"); String nullJson = null; - Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + assertThatThrownBy(() -> deserialize(nullJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - Assertions.assertThatThrownBy( - () -> UpdateNamespacePropertiesRequest.builder().remove(null).build()) + assertThatThrownBy(() -> UpdateNamespacePropertiesRequest.builder().remove(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid property to remove: null"); - Assertions.assertThatThrownBy( - () -> UpdateNamespacePropertiesRequest.builder().removeAll(null).build()) + assertThatThrownBy(() -> UpdateNamespacePropertiesRequest.builder().removeAll(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid list of properties to remove: null"); List listWithNull = Lists.newArrayList("a", null, null); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesRequest.builder().removeAll(listWithNull).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property to remove: null"); - Assertions.assertThatThrownBy( - () -> UpdateNamespacePropertiesRequest.builder().update(null, "100").build()) + assertThatThrownBy(() -> UpdateNamespacePropertiesRequest.builder().update(null, "100").build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid property to update: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesRequest.builder().update("owner", null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid value to update for key [owner]: null. Use remove instead"); - Assertions.assertThatThrownBy( - () -> UpdateNamespacePropertiesRequest.builder().updateAll(null).build()) + assertThatThrownBy(() -> UpdateNamespacePropertiesRequest.builder().updateAll(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid collection of properties to update: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesRequest.builder().updateAll(mapWithNullKey).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property to update: null"); @@ -186,7 +184,7 @@ public void testBuilderDoesNotCreateInvalidObjects() { Map mapWithMultipleNullValues = Maps.newHashMap(); mapWithMultipleNullValues.put("a", null); mapWithMultipleNullValues.put("b", "b"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesRequest.builder() .updateAll(mapWithMultipleNullValues) @@ -211,10 +209,10 @@ public UpdateNamespacePropertiesRequest createExampleInstance() { @Override public void assertEquals( UpdateNamespacePropertiesRequest actual, UpdateNamespacePropertiesRequest expected) { - Assertions.assertThat(actual.updates()) + assertThat(actual.updates()) .as("Properties to update should be equal") .isEqualTo(expected.updates()); - Assertions.assertThat(Sets.newHashSet(actual.removals())) + assertThat(Sets.newHashSet(actual.removals())) .as("Properties to remove should be equal") .containsExactlyInAnyOrderElementsOf(Sets.newHashSet(expected.removals())); } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java index e4fb413bed98..6144bcae7b40 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestCatalogErrorResponseParser.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.rest.responses; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Arrays; import java.util.List; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCatalogErrorResponseParser { @@ -35,7 +36,7 @@ public void testErrorResponseToJson() { String json = "{\"error\":" + errorModelJson + "}"; ErrorResponse response = ErrorResponse.builder().withMessage(message).withType(type).responseCode(code).build(); - Assertions.assertThat(ErrorResponseParser.toJson(response)) + assertThat(ErrorResponseParser.toJson(response)) .as("Should be able to serialize an error response as json") .isEqualTo(json); } @@ -58,7 +59,7 @@ public void testErrorResponseToJsonWithStack() { .responseCode(code) .withStackTrace(stack) .build(); - Assertions.assertThat(ErrorResponseParser.toJson(response)) + assertThat(ErrorResponseParser.toJson(response)) .as("Should be able to serialize an error response as json") .isEqualTo(json); } @@ -121,9 +122,9 @@ public void testErrorResponseFromJsonWithExplicitNullStack() { } public void assertEquals(ErrorResponse expected, ErrorResponse actual) { - Assertions.assertThat(actual.message()).isEqualTo(expected.message()); - Assertions.assertThat(actual.type()).isEqualTo(expected.type()); - Assertions.assertThat(actual.code()).isEqualTo(expected.code()); - Assertions.assertThat(actual.stack()).isEqualTo(expected.stack()); + assertThat(actual.message()).isEqualTo(expected.message()); + assertThat(actual.type()).isEqualTo(expected.type()); + assertThat(actual.code()).isEqualTo(expected.code()); + assertThat(actual.stack()).isEqualTo(expected.stack()); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java index 273fe48e2dcb..0cf30e6e02b8 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponse.java @@ -18,12 +18,14 @@ */ 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.core.JsonProcessingException; import java.util.Map; 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.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -144,51 +146,49 @@ public void testCanUseNullAsPropertyValue() throws JsonProcessingException { public void testDeserializeInvalidResponse() { String jsonDefaultsHasWrongType = "{\"defaults\":[\"warehouse\",\"s3://bucket/warehouse\"],\"overrides\":{\"clients\":\"5\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonDefaultsHasWrongType)) + assertThatThrownBy(() -> deserialize(jsonDefaultsHasWrongType)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Cannot parse string map from non-object value: defaults: [\"warehouse\",\"s3://bucket/warehouse\"]"); String jsonOverridesHasWrongType = "{\"defaults\":{\"warehouse\":\"s3://bucket/warehouse\"},\"overrides\":\"clients\"}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonOverridesHasWrongType)) + assertThatThrownBy(() -> deserialize(jsonOverridesHasWrongType)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Cannot parse string map from non-object value: overrides: \"clients\""); - Assertions.assertThatThrownBy(() -> deserialize(null)) + assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withOverride(null, "100").build()) + assertThatThrownBy(() -> ConfigResponse.builder().withOverride(null, "100").build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid override property: null"); - Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withDefault(null, "100").build()) + assertThatThrownBy(() -> ConfigResponse.builder().withDefault(null, "100").build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid default property: null"); - Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withOverrides(null).build()) + assertThatThrownBy(() -> ConfigResponse.builder().withOverrides(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid override properties map: null"); - Assertions.assertThatThrownBy(() -> ConfigResponse.builder().withDefaults(null).build()) + assertThatThrownBy(() -> ConfigResponse.builder().withDefaults(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid default properties map: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "a"); mapWithNullKey.put("b", "b"); - Assertions.assertThatThrownBy( - () -> ConfigResponse.builder().withDefaults(mapWithNullKey).build()) + assertThatThrownBy(() -> ConfigResponse.builder().withDefaults(mapWithNullKey).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid default property: null"); - Assertions.assertThatThrownBy( - () -> ConfigResponse.builder().withOverrides(mapWithNullKey).build()) + assertThatThrownBy(() -> ConfigResponse.builder().withOverrides(mapWithNullKey).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid override property: null"); } @@ -214,11 +214,11 @@ public void testMergeStripsNullValuedEntries() { "b", "from_overrides", "c", "from_client"); - Assertions.assertThat(merged) + assertThat(merged) .as( "The merged properties map should use values from defaults, then client config, and finally overrides") .isEqualTo(expected); - Assertions.assertThat(merged) + assertThat(merged) .as("The merged properties map should omit keys with null values") .doesNotContainValue(null); } @@ -235,10 +235,10 @@ public ConfigResponse createExampleInstance() { @Override public void assertEquals(ConfigResponse actual, ConfigResponse expected) { - Assertions.assertThat(actual.defaults()) + assertThat(actual.defaults()) .as("Config properties to use as defaults should be equal") .isEqualTo(expected.defaults()); - Assertions.assertThat(actual.overrides()) + assertThat(actual.overrides()) .as("Config properties to use as overrides should be equal") .isEqualTo(expected.overrides()); } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java index ece9f1e688d1..dfde5bfb4658 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestCreateNamespaceResponse.java @@ -18,13 +18,15 @@ */ 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.core.JsonProcessingException; import java.util.Map; import org.apache.iceberg.catalog.Namespace; 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.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCreateNamespaceResponse extends RequestResponseTestBase { @@ -84,47 +86,45 @@ public void testCanDeserializeWithoutDefaultValues() throws JsonProcessingExcept public void testDeserializeInvalidResponse() { String jsonResponseMalformedNamespaceValue = "{\"namespace\":\"accounting%1Ftax\",\"properties\":null}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonResponseMalformedNamespaceValue)) + assertThatThrownBy(() -> deserialize(jsonResponseMalformedNamespaceValue)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining("Cannot parse string array from non-array"); String jsonResponsePropertiesHasWrongType = "{\"namespace\":[\"accounting\",\"tax\"],\"properties\":[]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonResponsePropertiesHasWrongType)) + assertThatThrownBy(() -> deserialize(jsonResponsePropertiesHasWrongType)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining( "Cannot deserialize value of type `java.util.LinkedHashMap`"); - Assertions.assertThatThrownBy(() -> deserialize("{}")) + assertThatThrownBy(() -> deserialize("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); String jsonMisspelledKeys = "{\"namepsace\":[\"accounting\",\"tax\"],\"propertiezzzz\":{\"owner\":\"Hank\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonMisspelledKeys)) + assertThatThrownBy(() -> deserialize(jsonMisspelledKeys)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); - Assertions.assertThatThrownBy(() -> deserialize(null)) + assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - Assertions.assertThatThrownBy( - () -> CreateNamespaceResponse.builder().withNamespace(null).build()) + assertThatThrownBy(() -> CreateNamespaceResponse.builder().withNamespace(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid namespace: null"); - Assertions.assertThatThrownBy( - () -> CreateNamespaceResponse.builder().setProperties(null).build()) + assertThatThrownBy(() -> CreateNamespaceResponse.builder().setProperties(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid collection of properties: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CreateNamespaceResponse.builder().setProperties(mapWithNullKey).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property to set: null"); @@ -132,7 +132,7 @@ public void testBuilderDoesNotBuildInvalidRequests() { Map mapWithMultipleNullValues = Maps.newHashMap(); mapWithMultipleNullValues.put("a", null); mapWithMultipleNullValues.put("b", "b"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> CreateNamespaceResponse.builder().setProperties(mapWithMultipleNullValues).build()) .isInstanceOf(IllegalArgumentException.class) @@ -154,8 +154,8 @@ public CreateNamespaceResponse createExampleInstance() { @Override public void assertEquals(CreateNamespaceResponse actual, CreateNamespaceResponse expected) { - Assertions.assertThat(actual.namespace()).isEqualTo(expected.namespace()); - Assertions.assertThat(actual.properties()).isEqualTo(expected.properties()); + assertThat(actual.namespace()).isEqualTo(expected.namespace()); + assertThat(actual.properties()).isEqualTo(expected.properties()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java index d228e9c45b2a..680b67dbfb9b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestGetNamespaceResponse.java @@ -18,13 +18,15 @@ */ 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.core.JsonProcessingException; import java.util.Map; import org.apache.iceberg.catalog.Namespace; 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.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestGetNamespaceResponse extends RequestResponseTestBase { @@ -66,56 +68,55 @@ public void testCanDeserializeWithoutDefaultValues() throws JsonProcessingExcept @Test public void testDeserializeInvalidResponse() { String jsonNamespaceHasWrongType = "{\"namespace\":\"accounting%1Ftax\",\"properties\":null}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonNamespaceHasWrongType)) + assertThatThrownBy(() -> deserialize(jsonNamespaceHasWrongType)) .as("A JSON response with the wrong type for a field should fail to deserialize") .isInstanceOf(JsonProcessingException.class) .hasMessageContaining("Cannot parse string array from non-array"); String jsonPropertiesHasWrongType = "{\"namespace\":[\"accounting\",\"tax\"],\"properties\":[]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonPropertiesHasWrongType)) + assertThatThrownBy(() -> deserialize(jsonPropertiesHasWrongType)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining( "Cannot deserialize value of type `java.util.LinkedHashMap`"); String emptyJson = "{}"; - Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + assertThatThrownBy(() -> deserialize(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); String jsonWithKeysSpelledIncorrectly = "{\"namepsace\":[\"accounting\",\"tax\"],\"propertiezzzz\":{\"owner\":\"Hank\"}}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) + assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); String nullJson = null; - Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + assertThatThrownBy(() -> deserialize(nullJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotBuildInvalidRequests() { - Assertions.assertThatThrownBy(() -> GetNamespaceResponse.builder().withNamespace(null).build()) + assertThatThrownBy(() -> GetNamespaceResponse.builder().withNamespace(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid namespace: null"); - Assertions.assertThatThrownBy(() -> GetNamespaceResponse.builder().setProperties(null).build()) + assertThatThrownBy(() -> GetNamespaceResponse.builder().setProperties(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid properties map: null"); Map mapWithNullKey = Maps.newHashMap(); mapWithNullKey.put(null, "hello"); - Assertions.assertThatThrownBy( - () -> GetNamespaceResponse.builder().setProperties(mapWithNullKey).build()) + assertThatThrownBy(() -> GetNamespaceResponse.builder().setProperties(mapWithNullKey).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid property: null"); Map mapWithMultipleNullValues = Maps.newHashMap(); mapWithMultipleNullValues.put("a", null); mapWithMultipleNullValues.put("b", "b"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> GetNamespaceResponse.builder().setProperties(mapWithMultipleNullValues).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid value for properties [a]: null"); @@ -136,8 +137,8 @@ public GetNamespaceResponse createExampleInstance() { @Override public void assertEquals(GetNamespaceResponse actual, GetNamespaceResponse expected) { - Assertions.assertThat(actual.namespace()).isEqualTo(expected.namespace()); - Assertions.assertThat(actual.properties()).isEqualTo(expected.properties()); + assertThat(actual.namespace()).isEqualTo(expected.namespace()); + assertThat(actual.properties()).isEqualTo(expected.properties()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java index d9ed801de0cb..0db75a1148e8 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestListNamespacesResponse.java @@ -18,13 +18,15 @@ */ 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.core.JsonProcessingException; import java.util.List; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestListNamespacesResponse extends RequestResponseTestBase { @@ -45,40 +47,39 @@ public void testRoundTripSerDe() throws JsonProcessingException { @Test public void testDeserializeInvalidResponseThrows() { String jsonNamespacesHasWrongType = "{\"namespaces\":\"accounting\"}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonNamespacesHasWrongType)) + assertThatThrownBy(() -> deserialize(jsonNamespacesHasWrongType)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining( "Cannot deserialize value of type `java.util.ArrayList`"); String emptyJson = "{}"; - Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + assertThatThrownBy(() -> deserialize(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); String jsonWithKeysSpelledIncorrectly = "{\"namepsacezz\":[\"accounting\",\"tax\"]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) + assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); String nullJson = null; - Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + assertThatThrownBy(() -> deserialize(nullJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - Assertions.assertThatThrownBy(() -> ListNamespacesResponse.builder().add(null).build()) + assertThatThrownBy(() -> ListNamespacesResponse.builder().add(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid namespace: null"); - Assertions.assertThatThrownBy(() -> ListNamespacesResponse.builder().addAll(null).build()) + assertThatThrownBy(() -> ListNamespacesResponse.builder().addAll(null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid namespace list: null"); List listWithNullElement = Lists.newArrayList(Namespace.of("a"), null); - Assertions.assertThatThrownBy( - () -> ListNamespacesResponse.builder().addAll(listWithNullElement).build()) + assertThatThrownBy(() -> ListNamespacesResponse.builder().addAll(listWithNullElement).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid namespace: null"); } @@ -90,8 +91,8 @@ public void testWithNullPaginationToken() throws JsonProcessingException { ListNamespacesResponse response = ListNamespacesResponse.builder().addAll(NAMESPACES).nextPageToken(null).build(); assertRoundTripSerializesEquallyFrom(jsonWithNullPageToken, response); - Assertions.assertThat(response.nextPageToken()).isNull(); - Assertions.assertThat(response.namespaces()).isEqualTo(NAMESPACES); + assertThat(response.nextPageToken()).isNull(); + assertThat(response.namespaces()).isEqualTo(NAMESPACES); } @Test @@ -102,8 +103,8 @@ public void testWithPaginationToken() throws JsonProcessingException { ListNamespacesResponse response = ListNamespacesResponse.builder().addAll(NAMESPACES).nextPageToken(pageToken).build(); assertRoundTripSerializesEquallyFrom(jsonWithPageToken, response); - Assertions.assertThat(response.nextPageToken()).isEqualTo("token"); - Assertions.assertThat(response.namespaces()).isEqualTo(NAMESPACES); + assertThat(response.nextPageToken()).isEqualTo("token"); + assertThat(response.namespaces()).isEqualTo(NAMESPACES); } @Override @@ -118,7 +119,7 @@ public ListNamespacesResponse createExampleInstance() { @Override public void assertEquals(ListNamespacesResponse actual, ListNamespacesResponse expected) { - Assertions.assertThat(actual.namespaces()) + assertThat(actual.namespaces()) .as("Namespaces list should be equal") .hasSize(expected.namespaces().size()) .containsExactlyInAnyOrderElementsOf(expected.namespaces()); diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java index d46228f1886a..bc9526114aa1 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestListTablesResponse.java @@ -18,6 +18,9 @@ */ 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.core.JsonProcessingException; import java.util.List; import org.apache.iceberg.catalog.Namespace; @@ -25,7 +28,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestListTablesResponse extends RequestResponseTestBase { @@ -47,60 +49,60 @@ public void testRoundTripSerDe() throws JsonProcessingException { @Test public void testDeserializeInvalidResponsesThrows() { String identifiersHasWrongType = "{\"identifiers\":\"accounting%1Ftax\"}"; - Assertions.assertThatThrownBy(() -> deserialize(identifiersHasWrongType)) + assertThatThrownBy(() -> deserialize(identifiersHasWrongType)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining( "Cannot deserialize value of type `java.util.ArrayList`"); String emptyJson = "{}"; - Assertions.assertThatThrownBy(() -> deserialize(emptyJson)) + assertThatThrownBy(() -> deserialize(emptyJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid identifier list: null"); String jsonWithKeysSpelledIncorrectly = "{\"identifyrezzzz\":[{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"}]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) + assertThatThrownBy(() -> deserialize(jsonWithKeysSpelledIncorrectly)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid identifier list: null"); String jsonWithInvalidIdentifiersInList = "{\"identifiers\":[{\"namespace\":\"accounting.tax\",\"name\":\"paid\"}]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonWithInvalidIdentifiersInList)) + assertThatThrownBy(() -> deserialize(jsonWithInvalidIdentifiersInList)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining( "Cannot parse JSON array from non-array value: namespace: \"accounting.tax\""); String jsonWithInvalidIdentifiersInList2 = "{\"identifiers\":[{\"namespace\":[\"accounting\",\"tax\"],\"name\":\"paid\"},\"accounting.tax.paid\"]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonWithInvalidIdentifiersInList2)) + assertThatThrownBy(() -> deserialize(jsonWithInvalidIdentifiersInList2)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining("Cannot parse missing or non-object table identifier"); String jsonWithInvalidTypeForNamePartOfIdentifier = "{\"identifiers\":[{\"namespace\":[\"accounting\",\"tax\"],\"name\":true}]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonWithInvalidTypeForNamePartOfIdentifier)) + assertThatThrownBy(() -> deserialize(jsonWithInvalidTypeForNamePartOfIdentifier)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining("Cannot parse to a string value"); String nullJson = null; - Assertions.assertThatThrownBy(() -> deserialize(nullJson)) + assertThatThrownBy(() -> deserialize(nullJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @Test public void testBuilderDoesNotCreateInvalidObjects() { - Assertions.assertThatThrownBy(() -> ListTablesResponse.builder().add(null)) + assertThatThrownBy(() -> ListTablesResponse.builder().add(null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid table identifier: null"); - Assertions.assertThatThrownBy(() -> ListTablesResponse.builder().addAll(null)) + assertThatThrownBy(() -> ListTablesResponse.builder().addAll(null)) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid table identifier list: null"); List listWithNullElement = Lists.newArrayList(TableIdentifier.of(Namespace.of("foo"), "bar"), null); - Assertions.assertThatThrownBy(() -> ListTablesResponse.builder().addAll(listWithNullElement)) + assertThatThrownBy(() -> ListTablesResponse.builder().addAll(listWithNullElement)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid table identifier: null"); } @@ -112,8 +114,8 @@ public void testWithNullPaginationToken() throws JsonProcessingException { ListTablesResponse response = ListTablesResponse.builder().addAll(IDENTIFIERS).nextPageToken(null).build(); assertRoundTripSerializesEquallyFrom(jsonWithNullPageToken, response); - Assertions.assertThat(response.nextPageToken()).isNull(); - Assertions.assertThat(response.identifiers()).isEqualTo(IDENTIFIERS); + assertThat(response.nextPageToken()).isNull(); + assertThat(response.identifiers()).isEqualTo(IDENTIFIERS); } @Test @@ -124,8 +126,8 @@ public void testWithPaginationToken() throws JsonProcessingException { ListTablesResponse response = ListTablesResponse.builder().addAll(IDENTIFIERS).nextPageToken(pageToken).build(); assertRoundTripSerializesEquallyFrom(jsonWithPageToken, response); - Assertions.assertThat(response.nextPageToken()).isEqualTo("token"); - Assertions.assertThat(response.identifiers()).isEqualTo(IDENTIFIERS); + assertThat(response.nextPageToken()).isEqualTo("token"); + assertThat(response.identifiers()).isEqualTo(IDENTIFIERS); } @Override @@ -140,7 +142,7 @@ public ListTablesResponse createExampleInstance() { @Override public void assertEquals(ListTablesResponse actual, ListTablesResponse expected) { - Assertions.assertThat(actual.identifiers()) + assertThat(actual.identifiers()) .as("Identifiers should be equal") .hasSameSizeAs(expected.identifiers()) .containsExactlyInAnyOrderElementsOf(expected.identifiers()); diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java index 0b65541fa415..23252604e17b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponse.java @@ -19,6 +19,8 @@ package org.apache.iceberg.rest.responses; import static org.apache.iceberg.TestHelpers.assertSameSchemaList; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import com.fasterxml.jackson.core.JsonProcessingException; import java.nio.file.Path; @@ -36,7 +38,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.RequestResponseTestBase; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestLoadTableResponse extends RequestResponseTestBase { @@ -97,7 +98,7 @@ public LoadTableResponse deserialize(String json) throws JsonProcessingException @Test public void testFailures() { - Assertions.assertThatThrownBy(() -> LoadTableResponse.builder().build()) + assertThatThrownBy(() -> LoadTableResponse.builder().build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid metadata: null"); } @@ -123,7 +124,7 @@ public void testRoundTripSerdeWithV1TableMetadata() throws Exception { public void testMissingSchemaType() throws Exception { // When the schema type (struct) is missing String tableMetadataJson = readTableMetadataInputFile("TableMetadataV1MissingSchemaType.json"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> TableMetadataParser.fromJson(TEST_METADATA_LOCATION, tableMetadataJson)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot parse type from json:"); @@ -157,105 +158,93 @@ public void testCanDeserializeWithoutDefaultValues() throws Exception { LoadTableResponse actual = deserialize(json); LoadTableResponse expected = LoadTableResponse.builder().withTableMetadata(metadata).build(); assertEquals(actual, expected); - Assertions.assertThat(actual.config()) + assertThat(actual.config()) .as("Deserialized JSON with missing fields should have the default values") .isEqualTo(ImmutableMap.of()); } @Override public void assertEquals(LoadTableResponse actual, LoadTableResponse expected) { - Assertions.assertThat(actual.config()) + assertThat(actual.config()) .as("Should have the same configuration") .isEqualTo(expected.config()); assertEqualTableMetadata(actual.tableMetadata(), expected.tableMetadata()); - Assertions.assertThat(actual.metadataLocation()) + assertThat(actual.metadataLocation()) .as("Should have the same metadata location") .isEqualTo(expected.metadataLocation()); } private void assertEqualTableMetadata(TableMetadata actual, TableMetadata expected) { - Assertions.assertThat(actual.formatVersion()) + assertThat(actual.formatVersion()) .as("Format version should match") .isEqualTo(expected.formatVersion()); - Assertions.assertThat(actual.uuid()).as("Table UUID should match").isEqualTo(expected.uuid()); - Assertions.assertThat(actual.location()) - .as("Table location should match") - .isEqualTo(expected.location()); - Assertions.assertThat(actual.lastColumnId()) - .as("Last column id") - .isEqualTo(expected.lastColumnId()); - Assertions.assertThat(actual.schema().asStruct()) + assertThat(actual.uuid()).as("Table UUID should match").isEqualTo(expected.uuid()); + assertThat(actual.location()).as("Table location should match").isEqualTo(expected.location()); + assertThat(actual.lastColumnId()).as("Last column id").isEqualTo(expected.lastColumnId()); + assertThat(actual.schema().asStruct()) .as("Schema should match") .isEqualTo(expected.schema().asStruct()); assertSameSchemaList(expected.schemas(), actual.schemas()); - Assertions.assertThat(actual.currentSchemaId()) + assertThat(actual.currentSchemaId()) .as("Current schema id should match") .isEqualTo(expected.currentSchemaId()); - Assertions.assertThat(actual.schema().asStruct()) + assertThat(actual.schema().asStruct()) .as("Schema should match") .isEqualTo(expected.schema().asStruct()); - Assertions.assertThat(actual.lastSequenceNumber()) + assertThat(actual.lastSequenceNumber()) .as("Last sequence number should match") .isEqualTo(expected.lastSequenceNumber()); - Assertions.assertThat(actual.spec().toString()) + assertThat(actual.spec().toString()) .as("Partition spec should match") .isEqualTo(expected.spec().toString()); - Assertions.assertThat(actual.defaultSpecId()) + assertThat(actual.defaultSpecId()) .as("Default spec ID should match") .isEqualTo(expected.defaultSpecId()); - Assertions.assertThat(actual.specs()) - .as("PartitionSpec map should match") - .isEqualTo(expected.specs()); - Assertions.assertThat(actual.defaultSortOrderId()) + assertThat(actual.specs()).as("PartitionSpec map should match").isEqualTo(expected.specs()); + assertThat(actual.defaultSortOrderId()) .as("Default Sort ID should match") .isEqualTo(expected.defaultSortOrderId()); - Assertions.assertThat(actual.sortOrder()) - .as("Sort order should match") - .isEqualTo(expected.sortOrder()); - Assertions.assertThat(actual.sortOrders()) + assertThat(actual.sortOrder()).as("Sort order should match").isEqualTo(expected.sortOrder()); + assertThat(actual.sortOrders()) .as("Sort order map should match") .isEqualTo(expected.sortOrders()); - Assertions.assertThat(actual.properties()) - .as("Properties should match") - .isEqualTo(expected.properties()); - Assertions.assertThat(Lists.transform(actual.snapshots(), Snapshot::snapshotId)) + assertThat(actual.properties()).as("Properties should match").isEqualTo(expected.properties()); + assertThat(Lists.transform(actual.snapshots(), Snapshot::snapshotId)) .as("Snapshots should match") .isEqualTo(Lists.transform(expected.snapshots(), Snapshot::snapshotId)); - Assertions.assertThat(actual.snapshotLog()) - .as("History should match") - .isEqualTo(expected.snapshotLog()); + assertThat(actual.snapshotLog()).as("History should match").isEqualTo(expected.snapshotLog()); Snapshot expectedCurrentSnapshot = expected.currentSnapshot(); Snapshot actualCurrentSnapshot = actual.currentSnapshot(); - Assertions.assertThat( + assertThat( expectedCurrentSnapshot != null && actualCurrentSnapshot != null || expectedCurrentSnapshot == null && actualCurrentSnapshot == null) .as("Both expected and actual current snapshot should either be null or non-null") .isTrue(); if (expectedCurrentSnapshot != null) { - Assertions.assertThat(actual.currentSnapshot().snapshotId()) + assertThat(actual.currentSnapshot().snapshotId()) .as("Current snapshot ID should match") .isEqualTo(expected.currentSnapshot().snapshotId()); - Assertions.assertThat(actual.currentSnapshot().parentId()) + assertThat(actual.currentSnapshot().parentId()) .as("Parent snapshot ID should match") .isEqualTo(expected.currentSnapshot().parentId()); - Assertions.assertThat(actual.currentSnapshot().schemaId()) + assertThat(actual.currentSnapshot().schemaId()) .as("Schema ID for current snapshot should match") .isEqualTo(expected.currentSnapshot().schemaId()); } - Assertions.assertThat(actual.metadataFileLocation()) + assertThat(actual.metadataFileLocation()) .as("Metadata file location should match") .isEqualTo(expected.metadataFileLocation()); - Assertions.assertThat(actual.lastColumnId()) + assertThat(actual.lastColumnId()) .as("Last column id should match") .isEqualTo(expected.lastColumnId()); - Assertions.assertThat(actual.schema().asStruct()) + assertThat(actual.schema().asStruct()) .as("Schema should match") .isEqualTo(expected.schema().asStruct()); assertSameSchemaList(expected.schemas(), actual.schemas()); - Assertions.assertThat(actual.currentSchemaId()) + assertThat(actual.currentSchemaId()) .as("Current schema id should match") .isEqualTo(expected.currentSchemaId()); - Assertions.assertThat(actual.refs()).as("Refs map should match").isEqualTo(expected.refs()); + assertThat(actual.refs()).as("Refs map should match").isEqualTo(expected.refs()); } private String readTableMetadataInputFile(String fileName) throws Exception { diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java index 96953d3e6069..b041aac46572 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthErrorResponseParser.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.rest.responses; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.rest.auth.OAuth2Properties; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOAuthErrorResponseParser { @@ -56,14 +58,14 @@ public void testOAuthErrorResponseFromJsonMissingError() { String uri = "http://iceberg.apache.org"; String json = String.format("{\"error_description\":\"%s\",\"error_uri\":\"%s\"}", description, uri); - Assertions.assertThatThrownBy(() -> OAuthErrorResponseParser.fromJson(400, json)) + assertThatThrownBy(() -> OAuthErrorResponseParser.fromJson(400, json)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: error"); } public void assertEquals(ErrorResponse expected, ErrorResponse actual) { - Assertions.assertThat(actual.code()).isEqualTo(expected.code()); - Assertions.assertThat(actual.type()).isEqualTo(expected.type()); - Assertions.assertThat(actual.message()).isEqualTo(expected.message()); + assertThat(actual.code()).isEqualTo(expected.code()); + assertThat(actual.type()).isEqualTo(expected.type()); + assertThat(actual.message()).isEqualTo(expected.message()); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java index cf568ac0c6a1..aec052de9aa6 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestOAuthTokenResponse.java @@ -18,10 +18,12 @@ */ 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.core.JsonProcessingException; import org.apache.iceberg.rest.RequestResponseTestBase; import org.apache.iceberg.rest.auth.OAuth2Util; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestOAuthTokenResponse extends RequestResponseTestBase { @@ -43,17 +45,15 @@ public OAuthTokenResponse createExampleInstance() { @Override public void assertEquals(OAuthTokenResponse actual, OAuthTokenResponse expected) { - Assertions.assertThat(actual.token()).as("Token should match").isEqualTo(expected.token()); - Assertions.assertThat(actual.tokenType()) - .as("Token type should match") - .isEqualTo(expected.tokenType()); - Assertions.assertThat(actual.issuedTokenType()) + assertThat(actual.token()).as("Token should match").isEqualTo(expected.token()); + assertThat(actual.tokenType()).as("Token type should match").isEqualTo(expected.tokenType()); + assertThat(actual.issuedTokenType()) .as("Issued token type should match") .isEqualTo(expected.issuedTokenType()); - Assertions.assertThat(actual.expiresInSeconds()) + assertThat(actual.expiresInSeconds()) .as("Expiration should match") .isEqualTo(expected.expiresInSeconds()); - Assertions.assertThat(actual.scopes()).as("Scope should match").isEqualTo(expected.scopes()); + assertThat(actual.scopes()).as("Scope should match").isEqualTo(expected.scopes()); } @Override @@ -114,21 +114,19 @@ public void testRoundTrip() throws Exception { @Test public void testFailures() { - Assertions.assertThatThrownBy(() -> deserialize("{\"token_type\":\"bearer\"}")) + assertThatThrownBy(() -> deserialize("{\"token_type\":\"bearer\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("missing string: access_token"); - Assertions.assertThatThrownBy( - () -> deserialize("{\"access_token\":34,\"token_type\":\"bearer\"}")) + assertThatThrownBy(() -> deserialize("{\"access_token\":34,\"token_type\":\"bearer\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot parse to a string value: access_token: 34"); - Assertions.assertThatThrownBy(() -> deserialize("{\"access_token\":\"bearer-token\"}")) + assertThatThrownBy(() -> deserialize("{\"access_token\":\"bearer-token\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("missing string: token_type"); - Assertions.assertThatThrownBy( - () -> deserialize("{\"access_token\":\"bearer-token\",\"token_type\":34}")) + assertThatThrownBy(() -> deserialize("{\"access_token\":\"bearer-token\",\"token_type\":34}")) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Cannot parse to a string value: token_type: 34"); } diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java index 17176eff717d..33c6c5366e9b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestUpdateNamespacePropertiesResponse.java @@ -18,12 +18,14 @@ */ 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.core.JsonProcessingException; import java.util.List; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.RequestResponseTestBase; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestUpdateNamespacePropertiesResponse @@ -143,25 +145,25 @@ public void testDeserializeInvalidResponse() { // Invalid top-level types String jsonInvalidTypeOnRemovedField = "{\"removed\":{\"foo\":true},\"updated\":[\"owner\"],\"missing\":[\"bar\"]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnRemovedField)) + assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnRemovedField)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining( "Cannot deserialize value of type `java.util.ArrayList`"); String jsonInvalidTypeOnUpdatedField = "{\"updated\":\"owner\",\"missing\":[\"bar\"]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnUpdatedField)) + assertThatThrownBy(() -> deserialize(jsonInvalidTypeOnUpdatedField)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining("Cannot construct instance of `java.util.ArrayList`"); // Valid top-level (array) types, but at least one entry in the list is not the expected type String jsonInvalidValueOfTypeIntNestedInRemovedList = "{\"removed\":[\"foo\", \"bar\", 123456], ,\"updated\":[\"owner\"],\"missing\":[\"bar\"]}"; - Assertions.assertThatThrownBy(() -> deserialize(jsonInvalidValueOfTypeIntNestedInRemovedList)) + assertThatThrownBy(() -> deserialize(jsonInvalidValueOfTypeIntNestedInRemovedList)) .isInstanceOf(JsonProcessingException.class) .hasMessageContaining("Unexpected character (',' (code 44))"); // Exception comes from Jackson - Assertions.assertThatThrownBy(() -> deserialize(null)) + assertThatThrownBy(() -> deserialize(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("argument \"content\" is null"); } @@ -171,54 +173,54 @@ public void testBuilderDoesNotCreateInvalidObjects() { List listContainingNull = Lists.newArrayList("a", null, null); // updated - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addUpdated((String) null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid updated property: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addUpdated((List) null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid updated property list: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addUpdated(listContainingNull).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid updated property: null"); // removed - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addRemoved((String) null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid removed property: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addRemoved((List) null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid removed property list: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addRemoved(listContainingNull).build()) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid removed property: null"); // missing - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addMissing((String) null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid missing property: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addMissing((List) null).build()) .isInstanceOf(NullPointerException.class) .hasMessage("Invalid missing property list: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> UpdateNamespacePropertiesResponse.builder().addMissing(listContainingNull).build()) .isInstanceOf(IllegalArgumentException.class) @@ -242,13 +244,13 @@ public UpdateNamespacePropertiesResponse createExampleInstance() { @Override public void assertEquals( UpdateNamespacePropertiesResponse actual, UpdateNamespacePropertiesResponse expected) { - Assertions.assertThat(actual.updated()) + assertThat(actual.updated()) .as("Properties updated should be equal") .containsExactlyInAnyOrderElementsOf(expected.updated()); - Assertions.assertThat(actual.removed()) + assertThat(actual.removed()) .as("Properties removed should be equal") .containsExactlyInAnyOrderElementsOf(expected.removed()); - Assertions.assertThat(actual.missing()) + assertThat(actual.missing()) .as("Properties missing should be equal") .containsExactlyInAnyOrderElementsOf(expected.missing()); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java b/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java index b4d9296387ee..da7de5e2fbf8 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java +++ b/core/src/test/java/org/apache/iceberg/util/TestInMemoryLockManager.java @@ -19,6 +19,7 @@ package org.apache.iceberg.util; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; import java.util.UUID; @@ -28,7 +29,6 @@ import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -56,7 +56,7 @@ public void after() throws Exception { @Test public void testAcquireOnceSingleProcess() { lockManager.acquireOnce(lockEntityId, ownerId); - Assertions.assertThatThrownBy(() -> lockManager.acquireOnce(lockEntityId, ownerId)) + assertThatThrownBy(() -> lockManager.acquireOnce(lockEntityId, ownerId)) .isInstanceOf(IllegalStateException.class) .hasMessageStartingWith("Lock for") .hasMessageContaining("currently held by") diff --git a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java index f5d92129fb3d..7702d691afd0 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestJsonUtil.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.core.JsonProcessingException; import java.nio.ByteBuffer; import java.util.Arrays; @@ -26,65 +29,57 @@ 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.io.BaseEncoding; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestJsonUtil { @Test public void get() throws JsonProcessingException { - Assertions.assertThatThrownBy(() -> JsonUtil.get("x", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.get("x", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: x"); - Assertions.assertThatThrownBy( - () -> JsonUtil.get("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + assertThatThrownBy(() -> JsonUtil.get("x", JsonUtil.mapper().readTree("{\"x\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: x"); - Assertions.assertThat(JsonUtil.get("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}")).asText()) + assertThat(JsonUtil.get("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}")).asText()) .isEqualTo("23"); } @Test public void getInt() throws JsonProcessingException { - Assertions.assertThatThrownBy(() -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing int: x"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + assertThatThrownBy(() -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: x: null"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) + assertThatThrownBy(() -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: x: \"23\""); - Assertions.assertThatThrownBy( - () -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": 23.0}"))) + assertThatThrownBy(() -> JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": 23.0}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: x: 23.0"); - Assertions.assertThat(JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) - .isEqualTo(23); + assertThat(JsonUtil.getInt("x", JsonUtil.mapper().readTree("{\"x\": 23}"))).isEqualTo(23); } @Test public void getIntOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat(JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) - .isEqualTo(23); - Assertions.assertThat(JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) - .isNull(); + assertThat(JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + assertThat(JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))).isEqualTo(23); + assertThat(JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))).isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: x: \"23\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23.0}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to an integer value: x: 23.0"); @@ -92,43 +87,38 @@ public void getIntOrNull() throws JsonProcessingException { @Test public void getLong() throws JsonProcessingException { - Assertions.assertThatThrownBy(() -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: x"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + assertThatThrownBy(() -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: x: null"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) + assertThatThrownBy(() -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: x: \"23\""); - Assertions.assertThatThrownBy( - () -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": 23.0}"))) + assertThatThrownBy(() -> JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": 23.0}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: x: 23.0"); - Assertions.assertThat(JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) - .isEqualTo(23); + assertThat(JsonUtil.getLong("x", JsonUtil.mapper().readTree("{\"x\": 23}"))).isEqualTo(23); } @Test public void getLongOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat(JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) + assertThat(JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + assertThat(JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) .isEqualTo(23); - Assertions.assertThat(JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) - .isNull(); + assertThat(JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))).isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: x: \"23\""); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23.0}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a long value: x: 23.0"); @@ -136,35 +126,30 @@ public void getLongOrNull() throws JsonProcessingException { @Test public void getString() throws JsonProcessingException { - Assertions.assertThatThrownBy(() -> JsonUtil.getString("x", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getString("x", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: x"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getString("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + assertThatThrownBy(() -> JsonUtil.getString("x", JsonUtil.mapper().readTree("{\"x\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: x: null"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getString("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) + assertThatThrownBy(() -> JsonUtil.getString("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: x: 23"); - Assertions.assertThat(JsonUtil.getString("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) + assertThat(JsonUtil.getString("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isEqualTo("23"); } @Test public void getStringOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat( - JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) + assertThat(JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + assertThat(JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isEqualTo("23"); - Assertions.assertThat( - JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) - .isNull(); + assertThat(JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))).isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a string value: x: 23"); @@ -172,19 +157,17 @@ public void getStringOrNull() throws JsonProcessingException { @Test public void getByteBufferOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{}"))) - .isNull(); - Assertions.assertThat( - JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + assertThat(JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{}"))).isNull(); + assertThat(JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{\"x\": null}"))) .isNull(); byte[] bytes = new byte[] {1, 2, 3, 4}; String base16Str = BaseEncoding.base16().encode(bytes); String json = String.format("{\"x\": \"%s\"}", base16Str); ByteBuffer byteBuffer = JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree(json)); - Assertions.assertThat(byteBuffer.array()).isEqualTo(bytes); + assertThat(byteBuffer.array()).isEqualTo(bytes); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getByteBufferOrNull("x", JsonUtil.mapper().readTree("{\"x\": 23}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse byte buffer from non-text value: x: 23"); @@ -192,48 +175,41 @@ public void getByteBufferOrNull() throws JsonProcessingException { @Test public void getBool() throws JsonProcessingException { - Assertions.assertThatThrownBy(() -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing boolean: x"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": null}"))) + assertThatThrownBy(() -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a boolean value: x: null"); - Assertions.assertThatThrownBy( - () -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) + assertThatThrownBy(() -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": \"23\"}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a boolean value: x: \"23\""); - Assertions.assertThatThrownBy( - () -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": \"true\"}"))) + assertThatThrownBy(() -> JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": \"true\"}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse to a boolean value: x: \"true\""); - Assertions.assertThat(JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": true}"))) - .isTrue(); - Assertions.assertThat(JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": false}"))) - .isFalse(); + assertThat(JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": true}"))).isTrue(); + assertThat(JsonUtil.getBool("x", JsonUtil.mapper().readTree("{\"x\": false}"))).isFalse(); } @Test public void getIntArrayOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{}"))) - .isNull(); + assertThat(JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat( - JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) + assertThat(JsonUtil.getIntArrayOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntArrayOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse integer from non-int value in items: \"23\""); - Assertions.assertThat( + assertThat( JsonUtil.getIntArrayOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .isEqualTo(new int[] {23, 45}); @@ -241,17 +217,16 @@ public void getIntArrayOrNull() throws JsonProcessingException { @Test public void getIntegerList() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing list: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntegerList( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) @@ -259,7 +234,7 @@ public void getIntegerList() throws JsonProcessingException { .hasMessage("Cannot parse integer from non-int value in items: \"23\""); List items = Arrays.asList(23, 45); - Assertions.assertThat( + assertThat( JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .isEqualTo(items); @@ -271,51 +246,47 @@ public void getIntegerList() throws JsonProcessingException { gen.writeEndObject(); }, false); - Assertions.assertThat(JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree(json))) - .isEqualTo(items); + assertThat(JsonUtil.getIntegerList("items", JsonUtil.mapper().readTree(json))).isEqualTo(items); } @Test public void getIntegerSet() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getIntegerSet("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getIntegerSet("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing set: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntegerSet("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntegerSet( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse integer from non-int value in items: \"23\""); - Assertions.assertThat( - JsonUtil.getIntegerSet("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) + assertThat(JsonUtil.getIntegerSet("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .containsExactlyElementsOf(Arrays.asList(23, 45)); } @Test public void getIntegerSetOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getIntegerSetOrNull("items", JsonUtil.mapper().readTree("{}"))) - .isNull(); + assertThat(JsonUtil.getIntegerSetOrNull("items", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat( + assertThat( JsonUtil.getIntegerSetOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getIntegerSetOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse integer from non-int value in items: \"23\""); - Assertions.assertThat( + assertThat( JsonUtil.getIntegerSetOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .containsExactlyElementsOf(Arrays.asList(23, 45)); @@ -323,17 +294,16 @@ public void getIntegerSetOrNull() throws JsonProcessingException { @Test public void getLongList() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getLongList("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getLongList("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing list: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongList("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongList( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) @@ -341,8 +311,7 @@ public void getLongList() throws JsonProcessingException { .hasMessage("Cannot parse long from non-long value in items: \"23\""); List items = Arrays.asList(23L, 45L); - Assertions.assertThat( - JsonUtil.getLongList("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) + assertThat(JsonUtil.getLongList("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .isEqualTo(items); String json = @@ -353,27 +322,24 @@ public void getLongList() throws JsonProcessingException { gen.writeEndObject(); }, false); - Assertions.assertThat(JsonUtil.getLongList("items", JsonUtil.mapper().readTree(json))) - .isEqualTo(items); + assertThat(JsonUtil.getLongList("items", JsonUtil.mapper().readTree(json))).isEqualTo(items); } @Test public void getLongListOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getLongListOrNull("items", JsonUtil.mapper().readTree("{}"))) - .isNull(); + assertThat(JsonUtil.getLongListOrNull("items", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat( - JsonUtil.getLongListOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) + assertThat(JsonUtil.getLongListOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongListOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse long from non-long value in items: \"23\""); - Assertions.assertThat( + assertThat( JsonUtil.getLongListOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .containsExactlyElementsOf(Arrays.asList(23L, 45L)); @@ -381,62 +347,57 @@ public void getLongListOrNull() throws JsonProcessingException { @Test public void getLongSet() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getLongSet("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getLongSet("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing set: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongSet("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongSet( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse long from non-long value in items: \"23\""); - Assertions.assertThat( - JsonUtil.getLongSet("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) + assertThat(JsonUtil.getLongSet("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .containsExactlyElementsOf(Arrays.asList(23L, 45L)); } @Test public void getLongSetOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getLongSetOrNull("items", JsonUtil.mapper().readTree("{}"))) - .isNull(); + assertThat(JsonUtil.getLongSetOrNull("items", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat( - JsonUtil.getLongSetOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) + assertThat(JsonUtil.getLongSetOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getLongSetOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [13, \"23\"]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse long from non-long value in items: \"23\""); - Assertions.assertThat( + assertThat( JsonUtil.getLongSetOrNull("items", JsonUtil.mapper().readTree("{\"items\": [23, 45]}"))) .containsExactlyElementsOf(Arrays.asList(23L, 45L)); } @Test public void getStringList() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getStringList("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getStringList("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing list: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringList("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringList( "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", 45]}"))) @@ -444,7 +405,7 @@ public void getStringList() throws JsonProcessingException { .hasMessage("Cannot parse string from non-text value in items: 45"); List items = Arrays.asList("23", "45"); - Assertions.assertThat( + assertThat( JsonUtil.getStringList( "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", \"45\"]}"))) .containsExactlyElementsOf(items); @@ -457,27 +418,25 @@ public void getStringList() throws JsonProcessingException { gen.writeEndObject(); }, false); - Assertions.assertThat(JsonUtil.getStringList("items", JsonUtil.mapper().readTree(json))) - .isEqualTo(items); + assertThat(JsonUtil.getStringList("items", JsonUtil.mapper().readTree(json))).isEqualTo(items); } @Test public void getStringListOrNull() throws JsonProcessingException { - Assertions.assertThat(JsonUtil.getStringListOrNull("items", JsonUtil.mapper().readTree("{}"))) - .isNull(); + assertThat(JsonUtil.getStringListOrNull("items", JsonUtil.mapper().readTree("{}"))).isNull(); - Assertions.assertThat( + assertThat( JsonUtil.getStringListOrNull("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isNull(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringListOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", 45]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse string from non-text value in items: 45"); - Assertions.assertThat( + assertThat( JsonUtil.getStringListOrNull( "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", \"45\"]}"))) .containsExactlyElementsOf(Arrays.asList("23", "45")); @@ -485,24 +444,23 @@ public void getStringListOrNull() throws JsonProcessingException { @Test public void getStringSet() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getStringSet("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getStringSet("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing set: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringSet("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse JSON array from non-array value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringSet( "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", 45]}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse string from non-text value in items: 45"); - Assertions.assertThat( + assertThat( JsonUtil.getStringSet( "items", JsonUtil.mapper().readTree("{\"items\": [\"23\", \"45\"]}"))) .containsExactlyElementsOf(Arrays.asList("23", "45")); @@ -510,17 +468,16 @@ public void getStringSet() throws JsonProcessingException { @Test public void getStringMap() throws JsonProcessingException { - Assertions.assertThatThrownBy( - () -> JsonUtil.getStringMap("items", JsonUtil.mapper().readTree("{}"))) + assertThatThrownBy(() -> JsonUtil.getStringMap("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing map: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringMap("items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse string map from non-object value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringMap( "items", JsonUtil.mapper().readTree("{\"items\": {\"a\":\"23\", \"b\":45}}"))) @@ -528,7 +485,7 @@ public void getStringMap() throws JsonProcessingException { .hasMessage("Cannot parse to a string value: b: 45"); Map items = ImmutableMap.of("a", "23", "b", "45"); - Assertions.assertThat( + assertThat( JsonUtil.getStringMap( "items", JsonUtil.mapper().readTree("{\"items\": {\"a\":\"23\", \"b\":\"45\"}}"))) .isEqualTo(items); @@ -541,25 +498,24 @@ public void getStringMap() throws JsonProcessingException { gen.writeEndObject(); }, false); - Assertions.assertThat(JsonUtil.getStringMap("items", JsonUtil.mapper().readTree(json))) - .isEqualTo(items); + assertThat(JsonUtil.getStringMap("items", JsonUtil.mapper().readTree(json))).isEqualTo(items); } @Test public void getStringMapNullableValues() throws JsonProcessingException { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringMapNullableValues("items", JsonUtil.mapper().readTree("{}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing map: items"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringMapNullableValues( "items", JsonUtil.mapper().readTree("{\"items\": null}"))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse string map from non-object value: items: null"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> JsonUtil.getStringMapNullableValues( "items", JsonUtil.mapper().readTree("{\"items\": {\"a\":\"23\", \"b\":45}}"))) @@ -570,7 +526,7 @@ public void getStringMapNullableValues() throws JsonProcessingException { itemsWithNullableValues.put("a", null); itemsWithNullableValues.put("b", null); itemsWithNullableValues.put("c", "23"); - Assertions.assertThat( + assertThat( JsonUtil.getStringMapNullableValues( "items", JsonUtil.mapper() @@ -586,8 +542,7 @@ public void getStringMapNullableValues() throws JsonProcessingException { }, false); - Assertions.assertThat( - JsonUtil.getStringMapNullableValues("items", JsonUtil.mapper().readTree(json))) + assertThat(JsonUtil.getStringMapNullableValues("items", JsonUtil.mapper().readTree(json))) .isEqualTo(itemsWithNullableValues); } } diff --git a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java index 277500461659..9a7b2768d995 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestLocationUtil.java @@ -19,8 +19,8 @@ package org.apache.iceberg.util; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestLocationUtil { @@ -53,7 +53,7 @@ public void testStripTrailingSlashWithInvalidPath() { String[] invalidPaths = new String[] {null, ""}; for (String invalidPath : invalidPaths) { - Assertions.assertThatThrownBy(() -> LocationUtil.stripTrailingSlash(invalidPath)) + assertThatThrownBy(() -> LocationUtil.stripTrailingSlash(invalidPath)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("path must not be null or empty"); } diff --git a/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java b/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java index 5dd23c3a459d..c3207ae13426 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java +++ b/core/src/test/java/org/apache/iceberg/util/TestLockManagers.java @@ -18,18 +18,19 @@ */ package org.apache.iceberg.util; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.Map; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.LockManager; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestLockManagers { @Test public void testLoadDefaultLockManager() { - Assertions.assertThat(LockManagers.defaultLockManager()) + assertThat(LockManagers.defaultLockManager()) .isInstanceOf(LockManagers.InMemoryLockManager.class); } @@ -37,7 +38,7 @@ public void testLoadDefaultLockManager() { public void testLoadCustomLockManager() { Map properties = Maps.newHashMap(); properties.put(CatalogProperties.LOCK_IMPL, CustomLockManager.class.getName()); - Assertions.assertThat(LockManagers.from(properties)).isInstanceOf(CustomLockManager.class); + assertThat(LockManagers.from(properties)).isInstanceOf(CustomLockManager.class); } static class CustomLockManager implements LockManager { 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 db6be5bcfe48..af39a59971f9 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestSnapshotUtil.java @@ -35,7 +35,6 @@ import org.apache.iceberg.TestHelpers; import org.apache.iceberg.TestTables; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -178,7 +177,7 @@ public void ancestorsOf() { } // Once snapshot iterator has been exhausted, call hasNext again to make sure it is stable. - Assertions.assertThat(snapshotIter).isExhausted(); + assertThat(snapshotIter).isExhausted(); } private void expectedSnapshots(long[] snapshotIdExpected, Iterable snapshotsActual) { diff --git a/core/src/test/java/org/apache/iceberg/util/TestTruncateUtil.java b/core/src/test/java/org/apache/iceberg/util/TestTruncateUtil.java index 6bf6ec17cbe6..025f0c96968d 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestTruncateUtil.java +++ b/core/src/test/java/org/apache/iceberg/util/TestTruncateUtil.java @@ -18,17 +18,19 @@ */ package org.apache.iceberg.util; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThatException; +import static org.assertj.core.api.Assertions.assertThatNoException; + import org.junit.jupiter.api.Test; class TestTruncateUtil { @Test public void testInvalidInputWidthBehavior() { - Assertions.assertThatNoException() + assertThatNoException() .as("Invalid width input shouldn't necessarily throw an exception as it's not validated") .isThrownBy(() -> TruncateUtil.truncateInt(-1, 100)); - Assertions.assertThatException() + assertThatException() .as("Invalid width input can possibly throw an exception") .isThrownBy(() -> TruncateUtil.truncateInt(0, 100)); } diff --git a/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java b/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java index 8eae11a4e67d..8c58f131676a 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestSQLViewRepresentationParser.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.view; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestSQLViewRepresentationParser { @@ -33,7 +35,7 @@ public void testParseSqlViewRepresentation() { .dialect("spark-sql") .build(); - Assertions.assertThat(SQLViewRepresentationParser.fromJson(requiredFields)) + assertThat(SQLViewRepresentationParser.fromJson(requiredFields)) .as("Should be able to parse valid SQL view representation") .isEqualTo(viewRepresentation); @@ -45,7 +47,7 @@ public void testParseSqlViewRepresentation() { .sql("select * from foo") .dialect("spark-sql") .build(); - Assertions.assertThat(SQLViewRepresentationParser.fromJson(requiredAndOptionalFields)) + assertThat(SQLViewRepresentationParser.fromJson(requiredAndOptionalFields)) .as("Should be able to parse valid SQL view representation") .isEqualTo(viewWithOptionalFields); } @@ -53,12 +55,12 @@ public void testParseSqlViewRepresentation() { @Test public void testParseSqlViewRepresentationMissingRequiredFields() { String missingDialect = "{\"type\":\"sql\", \"sql\": \"select * from foo\"}"; - Assertions.assertThatThrownBy(() -> ViewRepresentationParser.fromJson(missingDialect)) + assertThatThrownBy(() -> ViewRepresentationParser.fromJson(missingDialect)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: dialect"); String missingType = "{\"sql\":\"select * from foo\",\"dialect\":\"spark-sql\"}"; - Assertions.assertThatThrownBy(() -> ViewRepresentationParser.fromJson(missingType)) + assertThatThrownBy(() -> ViewRepresentationParser.fromJson(missingType)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: type"); } @@ -71,21 +73,21 @@ public void testViewRepresentationSerialization() { .sql("select * from foo") .dialect("spark-sql") .build(); - Assertions.assertThat(ViewRepresentationParser.toJson(viewRepresentation)) + assertThat(ViewRepresentationParser.toJson(viewRepresentation)) .as("Should be able to serialize valid SQL view representation") .isEqualTo(json); - Assertions.assertThat( + assertThat( ViewRepresentationParser.fromJson(ViewRepresentationParser.toJson(viewRepresentation))) .isEqualTo(viewRepresentation); } @Test public void testNullSqlViewRepresentation() { - Assertions.assertThatThrownBy(() -> SQLViewRepresentationParser.toJson(null)) + assertThatThrownBy(() -> SQLViewRepresentationParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid SQL view representation: null"); - Assertions.assertThatThrownBy(() -> SQLViewRepresentationParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> SQLViewRepresentationParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse SQL view representation from null object"); } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java index 99c24fbff456..387b90cca3a7 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewHistoryEntryParser.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.view; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.fasterxml.jackson.databind.JsonNode; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestViewHistoryEntryParser { @@ -29,7 +31,7 @@ public void testViewHistoryEntryFromJson() { String json = "{\"timestamp-ms\":123,\"version-id\":1}"; ViewHistoryEntry viewHistoryEntry = ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(123).build(); - Assertions.assertThat(ViewHistoryEntryParser.fromJson(json)) + assertThat(ViewHistoryEntryParser.fromJson(json)) .as("Should be able to deserialize valid view history entry") .isEqualTo(viewHistoryEntry); } @@ -39,34 +41,33 @@ public void testViewHistoryEntryToJson() { String json = "{\"timestamp-ms\":123,\"version-id\":1}"; ViewHistoryEntry viewHistoryEntry = ImmutableViewHistoryEntry.builder().versionId(1).timestampMillis(123).build(); - Assertions.assertThat(ViewHistoryEntryParser.toJson(viewHistoryEntry)) + assertThat(ViewHistoryEntryParser.toJson(viewHistoryEntry)) .as("Should be able to serialize view history entry") .isEqualTo(json); } @Test public void testNullViewHistoryEntry() { - Assertions.assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse view history entry from null object"); - Assertions.assertThatThrownBy(() -> ViewHistoryEntryParser.toJson(null)) + assertThatThrownBy(() -> ViewHistoryEntryParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid view history entry: null"); } @Test public void testViewHistoryEntryMissingFields() { - Assertions.assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson("{}")) + assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson("{}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing int: version-id"); - Assertions.assertThatThrownBy( - () -> ViewHistoryEntryParser.fromJson("{\"timestamp-ms\":\"123\"}")) + assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson("{\"timestamp-ms\":\"123\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing int: version-id"); - Assertions.assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson("{\"version-id\":1}")) + assertThatThrownBy(() -> ViewHistoryEntryParser.fromJson("{\"version-id\":1}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing long: timestamp-ms"); } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java index 37f9ae2eaa12..d81dfb756017 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewRepresentationParser.java @@ -18,7 +18,9 @@ */ package org.apache.iceberg.view; -import org.assertj.core.api.Assertions; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.junit.jupiter.api.Test; public class TestViewRepresentationParser { @@ -27,26 +29,25 @@ public class TestViewRepresentationParser { public void testParseUnknownViewRepresentation() { String json = "{\"type\":\"unknown-sql-representation\"}"; ViewRepresentation unknownRepresentation = ViewRepresentationParser.fromJson(json); - Assertions.assertThat( + assertThat( ImmutableUnknownViewRepresentation.builder().type("unknown-sql-representation").build()) .isEqualTo(unknownRepresentation); - Assertions.assertThatThrownBy(() -> ViewRepresentationParser.toJson(unknownRepresentation)) + assertThatThrownBy(() -> ViewRepresentationParser.toJson(unknownRepresentation)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot serialize unsupported view representation: unknown-sql-representation"); } @Test public void testNullViewRepresentation() { - Assertions.assertThatThrownBy(() -> ViewRepresentationParser.toJson(null)) + assertThatThrownBy(() -> ViewRepresentationParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid view representation: null"); } @Test public void testViewRepresentationMissingType() { - Assertions.assertThatThrownBy( - () -> ViewRepresentationParser.fromJson("{\"sql\":\"select * from foo\"}")) + assertThatThrownBy(() -> ViewRepresentationParser.fromJson("{\"sql\":\"select * from foo\"}")) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing string: type"); } diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java index 03db3897d1fd..1d115c04c393 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.view; +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.catalog.Namespace; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestViewVersionParser { @@ -58,7 +60,7 @@ public void testParseViewVersion() { "{\"version-id\":1, \"timestamp-ms\":12345, \"schema-id\":1, \"summary\":{\"user\":\"some-user\"}, \"representations\":%s, \"default-namespace\":[\"one\",\"two\"]}", serializedRepresentations); - Assertions.assertThat(ViewVersionParser.fromJson(serializedViewVersion)) + assertThat(ViewVersionParser.fromJson(serializedViewVersion)) .as("Should be able to parse valid view version") .isEqualTo(expectedViewVersion); } @@ -97,29 +99,29 @@ public void testSerializeViewVersion() { + "\"default-catalog\":\"catalog\",\"default-namespace\":[\"one\",\"two\"],\"representations\":%s}", expectedRepresentations); - Assertions.assertThat(ViewVersionParser.toJson(viewVersion)) + assertThat(ViewVersionParser.toJson(viewVersion)) .as("Should be able to serialize valid view version") .isEqualTo(expectedViewVersion); } @Test public void testNullViewVersion() { - Assertions.assertThatThrownBy(() -> ViewVersionParser.toJson(null)) + assertThatThrownBy(() -> ViewVersionParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot serialize null view version"); - Assertions.assertThatThrownBy(() -> ViewVersionParser.fromJson((JsonNode) null)) + assertThatThrownBy(() -> ViewVersionParser.fromJson((JsonNode) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse view version from null object"); - Assertions.assertThatThrownBy(() -> ViewVersionParser.fromJson((String) null)) + assertThatThrownBy(() -> ViewVersionParser.fromJson((String) null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse view version from null string"); } @Test public void missingDefaultCatalog() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ViewVersionParser.fromJson( "{\"version-id\":1,\"timestamp-ms\":12345,\"schema-id\":1," 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 d4813ca7bc77..e02ec8ec2cdc 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.data; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.Map; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; public class DataTestHelpers { @@ -88,28 +89,24 @@ private static void assertEquals(Type type, Object expected, Object actual) { "Primitive value should be equal to expected for type " + type, expected, actual); break; case FIXED: - Assertions.assertThat(expected) - .as("Expected should be a byte[]") - .isInstanceOf(byte[].class); - Assertions.assertThat(expected).as("Actual should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Expected should be a byte[]").isInstanceOf(byte[].class); + assertThat(expected).as("Actual should be a byte[]").isInstanceOf(byte[].class); Assert.assertArrayEquals( "Array contents should be equal", (byte[]) expected, (byte[]) actual); break; case STRUCT: - Assertions.assertThat(expected) - .as("Expected should be a Record") - .isInstanceOf(Record.class); - Assertions.assertThat(actual).as("Actual should be a Record").isInstanceOf(Record.class); + assertThat(expected).as("Expected should be a Record").isInstanceOf(Record.class); + assertThat(actual).as("Actual should be a Record").isInstanceOf(Record.class); assertEquals(type.asStructType(), (Record) expected, (Record) actual); break; case LIST: - Assertions.assertThat(expected).as("Expected should be a List").isInstanceOf(List.class); - Assertions.assertThat(actual).as("Actual should be a List").isInstanceOf(List.class); + assertThat(expected).as("Expected should be a List").isInstanceOf(List.class); + assertThat(actual).as("Actual should be a List").isInstanceOf(List.class); assertEquals(type.asListType(), (List) expected, (List) actual); break; case MAP: - Assertions.assertThat(expected).as("Expected should be a Map").isInstanceOf(Map.class); - Assertions.assertThat(actual).as("Actual should be a Map").isInstanceOf(Map.class); + assertThat(expected).as("Expected should be a Map").isInstanceOf(Map.class); + assertThat(actual).as("Actual should be a Map").isInstanceOf(Map.class); assertEquals(type.asMapType(), (Map) expected, (Map) actual); break; default: diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java b/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java index 74f04e1078cf..dee38b992c08 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java @@ -19,10 +19,10 @@ package org.apache.iceberg.data; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -54,7 +54,7 @@ public void testGetIncorrectClassInstance() { GenericRecord record = GenericRecord.create(schema); record.set(0, 10L); - Assertions.assertThatThrownBy(() -> record.get(0, CharSequence.class)) + assertThatThrownBy(() -> record.get(0, CharSequence.class)) .isInstanceOf(IllegalStateException.class) .hasMessage("Not an instance of java.lang.CharSequence: 10"); } diff --git a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java index 47209aaf8cf4..5855d249ff82 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java +++ b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java @@ -27,6 +27,7 @@ import static org.apache.iceberg.relocated.com.google.common.collect.Iterables.transform; 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.File; import java.io.IOException; @@ -59,7 +60,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Before; import org.junit.Rule; @@ -505,8 +505,7 @@ public void testUnknownSnapshotId() { IcebergGenerics.ScanBuilder scanBuilder = IcebergGenerics.read(sharedTable); - Assertions.assertThatThrownBy( - () -> scanBuilder.useSnapshot(/* unknown snapshot id */ minSnapshotId - 1)) + assertThatThrownBy(() -> scanBuilder.useSnapshot(/* unknown snapshot id */ minSnapshotId - 1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot find snapshot with ID " + (minSnapshotId - 1)); } @@ -516,8 +515,7 @@ public void testAsOfTimeOlderThanFirstSnapshot() { IcebergGenerics.ScanBuilder scanBuilder = IcebergGenerics.read(sharedTable); long timestamp = sharedTable.history().get(0).timestampMillis() - 1; - Assertions.assertThatThrownBy( - () -> scanBuilder.asOfTime(/* older than first snapshot */ timestamp)) + assertThatThrownBy(() -> scanBuilder.asOfTime(/* older than first snapshot */ timestamp)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot find a snapshot older than " + DateTimeUtil.formatTimestampMillis(timestamp)); diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java index cb6b8d7af9c0..9525001b6265 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java @@ -39,6 +39,8 @@ import static org.apache.iceberg.expressions.Expressions.truncate; 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; @@ -77,7 +79,6 @@ import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.io.DelegatingSeekableInputStream; import org.apache.parquet.schema.MessageType; -import org.assertj.core.api.Assertions; import org.assertj.core.api.Assumptions; import org.junit.Assert; import org.junit.Assume; @@ -397,7 +398,7 @@ public void testRequiredColumn() { @Test public void testMissingColumn() { - Assertions.assertThatThrownBy(() -> shouldRead(lessThan("missing", 5))) + assertThatThrownBy(() -> shouldRead(lessThan("missing", 5))) .as("Should complain about missing column in expression") .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Cannot find field 'missing'"); @@ -946,7 +947,7 @@ public void testTransformFilter() { boolean shouldRead = new ParquetMetricsRowGroupFilter(SCHEMA, equal(truncate("required", 2), "some_value"), true) .shouldRead(parquetSchema, rowGroupMetadata); - Assertions.assertThat(shouldRead) + assertThat(shouldRead) .as("Should read: filter contains non-reference evaluate as True") .isTrue(); } diff --git a/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java index 0b5f9ed6ddd8..374a97be0384 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.data; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.IOException; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -30,7 +32,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -188,8 +189,7 @@ public void testEmptyProjection() throws Exception { Assert.assertNotNull("Should read a non-null record", projected); // this is expected because there are no values - Assertions.assertThatThrownBy(() -> projected.get(0)) - .isInstanceOf(ArrayIndexOutOfBoundsException.class); + assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @Test diff --git a/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java index 1b8da1eafc67..b68c20f4f4f6 100644 --- a/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java @@ -20,6 +20,7 @@ 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.nio.ByteBuffer; import java.util.Arrays; @@ -40,7 +41,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Ordering; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Test; @@ -132,7 +132,7 @@ public void testCompatibleReadFailsWithoutSchema() throws Exception { ByteBuffer v1Buffer = v1Encoder.encode(V1_RECORDS.get(3)); - Assertions.assertThatThrownBy(() -> v2Decoder.decode(v1Buffer)) + assertThatThrownBy(() -> v2Decoder.decode(v1Buffer)) .isInstanceOf(MissingSchemaException.class) .hasMessageContaining("Cannot resolve schema for fingerprint"); } @@ -206,7 +206,7 @@ public void testByteBufferMissingPayload() throws Exception { buffer.limit(12); - Assertions.assertThatThrownBy(() -> decoder.decode(buffer)) + assertThatThrownBy(() -> decoder.decode(buffer)) .isInstanceOf(AvroRuntimeException.class) .hasMessageContaining("Decoding datum failed"); } @@ -220,7 +220,7 @@ public void testByteBufferMissingFullHeader() throws Exception { buffer.limit(8); - Assertions.assertThatThrownBy(() -> decoder.decode(buffer)) + assertThatThrownBy(() -> decoder.decode(buffer)) .isInstanceOf(BadHeaderException.class) .hasMessage("Not enough header bytes"); } @@ -233,7 +233,7 @@ public void testByteBufferBadMarkerByte() throws Exception { ByteBuffer buffer = encoder.encode(V2_RECORDS.get(0)); buffer.array()[0] = 0x00; - Assertions.assertThatThrownBy(() -> decoder.decode(buffer)) + assertThatThrownBy(() -> decoder.decode(buffer)) .isInstanceOf(BadHeaderException.class) .hasMessageContaining("Unrecognized header bytes"); } @@ -246,7 +246,7 @@ public void testByteBufferBadVersionByte() throws Exception { ByteBuffer buffer = encoder.encode(V2_RECORDS.get(0)); buffer.array()[1] = 0x00; - Assertions.assertThatThrownBy(() -> decoder.decode(buffer)) + assertThatThrownBy(() -> decoder.decode(buffer)) .isInstanceOf(BadHeaderException.class) .hasMessageContaining("Unrecognized header bytes"); } @@ -259,7 +259,7 @@ public void testByteBufferUnknownSchema() throws Exception { ByteBuffer buffer = encoder.encode(V2_RECORDS.get(0)); buffer.array()[4] = 0x00; - Assertions.assertThatThrownBy(() -> decoder.decode(buffer)) + assertThatThrownBy(() -> decoder.decode(buffer)) .isInstanceOf(MissingSchemaException.class) .hasMessageContaining("Cannot resolve schema for fingerprint"); } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java index a35ba3102bb7..b0e2cfa51f90 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java @@ -20,6 +20,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 java.io.IOException; import java.util.Map; @@ -40,7 +41,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -128,11 +128,11 @@ public void testDropNamespace() { ecsCatalog.createNamespace(Namespace.of("a", "b1")); ecsCatalog.createTable(TableIdentifier.of("a", "t1"), SCHEMA); - Assertions.assertThatThrownBy(() -> ecsCatalog.dropNamespace(Namespace.of("unknown"))) + assertThatThrownBy(() -> ecsCatalog.dropNamespace(Namespace.of("unknown"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace unknown does not exist"); - Assertions.assertThatThrownBy(() -> ecsCatalog.dropNamespace(Namespace.of("a"))) + assertThatThrownBy(() -> ecsCatalog.dropNamespace(Namespace.of("a"))) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessage("Namespace a is not empty"); @@ -165,14 +165,14 @@ public void testRenameTable() { ecsCatalog.createTable(TableIdentifier.of("a", "t1"), SCHEMA); ecsCatalog.createNamespace(Namespace.of("b")); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ecsCatalog.renameTable( TableIdentifier.of("unknown"), TableIdentifier.of("b", "t2"))) .isInstanceOf(NoSuchTableException.class) .hasMessage("Cannot rename table because table unknown does not exist"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ecsCatalog.renameTable( TableIdentifier.of("a", "t1"), TableIdentifier.of("unknown", "t2"))) @@ -198,12 +198,12 @@ public void testRegisterTable() { TableOperations ops = ((HasTableOperations) registeringTable).operations(); String metadataLocation = ((EcsTableOperations) ops).currentMetadataLocation(); Table registeredTable = ecsCatalog.registerTable(identifier, metadataLocation); - Assertions.assertThat(registeredTable).isNotNull(); + assertThat(registeredTable).isNotNull(); String expectedMetadataLocation = ((HasTableOperations) registeredTable).operations().current().metadataFileLocation(); - Assertions.assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); - Assertions.assertThat(ecsCatalog.loadTable(identifier)).isNotNull(); - Assertions.assertThat(ecsCatalog.dropTable(identifier, true)).isTrue(); + assertThat(metadataLocation).isEqualTo(expectedMetadataLocation); + assertThat(ecsCatalog.loadTable(identifier)).isNotNull(); + assertThat(ecsCatalog.dropTable(identifier, true)).isTrue(); } @Test @@ -213,9 +213,9 @@ public void testRegisterExistingTable() { Table registeringTable = ecsCatalog.loadTable(identifier); TableOperations ops = ((HasTableOperations) registeringTable).operations(); String metadataLocation = ((EcsTableOperations) ops).currentMetadataLocation(); - Assertions.assertThatThrownBy(() -> ecsCatalog.registerTable(identifier, metadataLocation)) + assertThatThrownBy(() -> ecsCatalog.registerTable(identifier, metadataLocation)) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: a.t1"); - Assertions.assertThat(ecsCatalog.dropTable(identifier, true)).isTrue(); + assertThat(ecsCatalog.dropTable(identifier, true)).isTrue(); } } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java index 9c8ef253db30..5b346e29b8e8 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsOutputFile.java @@ -19,6 +19,7 @@ package org.apache.iceberg.dell.ecs; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import com.emc.object.Range; import java.io.IOException; @@ -28,7 +29,6 @@ import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -87,7 +87,7 @@ public void testFileAlreadyExists() throws IOException { output.write("1234567890".getBytes()); } - Assertions.assertThatThrownBy(outputFile::create) + assertThatThrownBy(outputFile::create) .isInstanceOf(AlreadyExistsException.class) .hasMessage("ECS object already exists: " + outputFile.location()); } diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsTableOperations.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsTableOperations.java index 809af32ca023..b8167ba004d2 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsTableOperations.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsTableOperations.java @@ -19,6 +19,7 @@ package org.apache.iceberg.dell.ecs; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.Map; import org.apache.iceberg.CatalogProperties; @@ -33,7 +34,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -57,7 +57,7 @@ public void testConcurrentCommit() { // Use the TableOperations to test the CommitFailedException // High level actions, such as Table#updateProperties(), may refresh metadata. TableOperations operations = ((HasTableOperations) catalog2Table).operations(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> operations.commit( operations.current(), diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java index 0e80ac03d505..ec56e76bb717 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsURI.java @@ -19,9 +19,9 @@ package org.apache.iceberg.dell.ecs; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import org.apache.iceberg.exceptions.ValidationException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestEcsURI { @@ -55,7 +55,7 @@ private void assertURI(String bucket, String name, EcsURI ecsURI) { @Test public void testInvalidLocation() { - Assertions.assertThatThrownBy(() -> new EcsURI("http://bucket/a")) + assertThatThrownBy(() -> new EcsURI("http://bucket/a")) .isInstanceOf(ValidationException.class) .hasMessage("Invalid ecs location: http://bucket/a"); } diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/TestExceptionCode.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/TestExceptionCode.java index 719c346267c1..4e7455619237 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/TestExceptionCode.java +++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/TestExceptionCode.java @@ -18,10 +18,12 @@ */ package org.apache.iceberg.dell.mock.ecs; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import com.emc.object.Range; import com.emc.object.s3.S3Exception; import com.emc.object.s3.request.PutObjectRequest; -import org.assertj.core.api.Assertions; import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; @@ -67,17 +69,11 @@ public void testExceptionCode() { } public void assertS3Exception(String message, int httpCode, String errorCode, Runnable task) { - Assertions.assertThatThrownBy(task::run) + assertThatThrownBy(task::run) .isInstanceOf(S3Exception.class) .asInstanceOf(InstanceOfAssertFactories.type(S3Exception.class)) .satisfies( - e -> - Assertions.assertThat(e.getErrorCode()) - .as(message + ", http code") - .isEqualTo(errorCode), - e -> - Assertions.assertThat(e.getHttpCode()) - .as(message + ", error code") - .isEqualTo(httpCode)); + e -> assertThat(e.getErrorCode()).as(message + ", http code").isEqualTo(errorCode), + e -> assertThat(e.getHttpCode()).as(message + ", error code").isEqualTo(httpCode)); } } 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 cebbea65f57f..945e89670de8 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 @@ -23,6 +23,7 @@ import static org.apache.spark.sql.functions.date_add; import static org.apache.spark.sql.functions.date_format; import static org.apache.spark.sql.functions.expr; +import static org.assertj.core.api.Assertions.assertThat; import io.delta.standalone.DeltaLog; import io.delta.standalone.Operation; @@ -59,7 +60,6 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.connector.catalog.CatalogPlugin; import org.apache.spark.sql.delta.catalog.DeltaCatalog; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; @@ -327,7 +327,7 @@ public void testSnapshotVacuumTable() throws IOException { Paths.get( URI.create( vacuumTestTableLocation.concat("/_delta_log/00000000000000000000.json")))); - Assertions.assertThat(deleteResult).isTrue(); + assertThat(deleteResult).isTrue(); spark.sql("VACUUM " + vacuumTestIdentifier + " RETAIN 0 HOURS"); String newTableIdentifier = destName(icebergCatalogName, "iceberg_vacuum_table"); @@ -362,7 +362,7 @@ public void testSnapshotLogCleanTable() throws IOException { Paths.get( URI.create( logCleanTestTableLocation.concat("/_delta_log/00000000000000000000.json")))); - Assertions.assertThat(deleteResult).isTrue(); + assertThat(deleteResult).isTrue(); String newTableIdentifier = destName(icebergCatalogName, "iceberg_log_clean_table"); SnapshotDeltaLakeTable.Result result = @@ -388,11 +388,10 @@ private void checkSnapshotIntegrity( List icebergTableContents = spark.sql("SELECT * FROM " + icebergTableIdentifier).collectAsList(); - Assertions.assertThat(deltaTableContents).hasSize(icebergTableContents.size()); - Assertions.assertThat(snapshotReport.snapshotDataFilesCount()) + assertThat(deltaTableContents).hasSize(icebergTableContents.size()); + assertThat(snapshotReport.snapshotDataFilesCount()) .isEqualTo(countDataFilesInDeltaLakeTable(deltaLog, firstConstructableVersion)); - Assertions.assertThat(icebergTableContents) - .containsExactlyInAnyOrderElementsOf(deltaTableContents); + assertThat(icebergTableContents).containsExactlyInAnyOrderElementsOf(deltaTableContents); } private void checkTagContentAndOrder( @@ -403,8 +402,7 @@ private void checkTagContentAndOrder( Map icebergSnapshotRefs = icebergTable.refs(); List icebergSnapshots = Lists.newArrayList(icebergTable.snapshots()); - Assertions.assertThat(icebergSnapshots.size()) - .isEqualTo(currentVersion - firstConstructableVersion + 1); + assertThat(icebergSnapshots.size()).isEqualTo(currentVersion - firstConstructableVersion + 1); for (int i = 0; i < icebergSnapshots.size(); i++) { long deltaVersion = firstConstructableVersion + i; @@ -412,25 +410,25 @@ private void checkTagContentAndOrder( String expectedVersionTag = "delta-version-" + deltaVersion; icebergSnapshotRefs.get(expectedVersionTag); - Assertions.assertThat(icebergSnapshotRefs.get(expectedVersionTag)).isNotNull(); - Assertions.assertThat(icebergSnapshotRefs.get(expectedVersionTag).isTag()).isTrue(); - Assertions.assertThat(icebergSnapshotRefs.get(expectedVersionTag).snapshotId()) + assertThat(icebergSnapshotRefs.get(expectedVersionTag)).isNotNull(); + assertThat(icebergSnapshotRefs.get(expectedVersionTag).isTag()).isTrue(); + assertThat(icebergSnapshotRefs.get(expectedVersionTag).snapshotId()) .isEqualTo(currentIcebergSnapshot.snapshotId()); Timestamp deltaVersionTimestamp = deltaLog.getCommitInfoAt(deltaVersion).getTimestamp(); - Assertions.assertThat(deltaVersionTimestamp).isNotNull(); + assertThat(deltaVersionTimestamp).isNotNull(); String expectedTimestampTag = "delta-ts-" + deltaVersionTimestamp.getTime(); - Assertions.assertThat(icebergSnapshotRefs.get(expectedTimestampTag)).isNotNull(); - Assertions.assertThat(icebergSnapshotRefs.get(expectedTimestampTag).isTag()).isTrue(); - Assertions.assertThat(icebergSnapshotRefs.get(expectedTimestampTag).snapshotId()) + assertThat(icebergSnapshotRefs.get(expectedTimestampTag)).isNotNull(); + assertThat(icebergSnapshotRefs.get(expectedTimestampTag).isTag()).isTrue(); + assertThat(icebergSnapshotRefs.get(expectedTimestampTag).snapshotId()) .isEqualTo(currentIcebergSnapshot.snapshotId()); } } private void checkIcebergTableLocation(String icebergTableIdentifier, String expectedLocation) { Table icebergTable = getIcebergTable(icebergTableIdentifier); - Assertions.assertThat(icebergTable.location()) + assertThat(icebergTable.location()) .isEqualTo(LocationUtil.stripTrailingSlash(expectedLocation)); } @@ -445,10 +443,8 @@ private void checkIcebergTableProperties( expectedPropertiesBuilder.putAll(expectedAdditionalProperties); ImmutableMap expectedProperties = expectedPropertiesBuilder.build(); - Assertions.assertThat(icebergTable.properties().entrySet()) - .containsAll(expectedProperties.entrySet()); - Assertions.assertThat(icebergTable.properties()) - .containsEntry(ORIGINAL_LOCATION_PROP, deltaTableLocation); + assertThat(icebergTable.properties().entrySet()).containsAll(expectedProperties.entrySet()); + assertThat(icebergTable.properties()).containsEntry(ORIGINAL_LOCATION_PROP, deltaTableLocation); } private void checkDataFilePathsIntegrity( @@ -466,8 +462,8 @@ private void checkDataFilePathsIntegrity( .addedDataFiles(icebergTable.io()) .forEach( dataFile -> { - Assertions.assertThat(URI.create(dataFile.path().toString()).isAbsolute()).isTrue(); - Assertions.assertThat(deltaTableDataFilePaths).contains(dataFile.path().toString()); + assertThat(URI.create(dataFile.path().toString()).isAbsolute()).isTrue(); + assertThat(deltaTableDataFilePaths).contains(dataFile.path().toString()); }); } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java index 22e27c1e51e1..f77ce441067e 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestBaseSnapshotDeltaLakeTableAction.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.delta; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import java.io.File; import java.io.IOException; import java.util.List; @@ -27,7 +29,6 @@ import org.apache.iceberg.catalog.Catalog; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -53,7 +54,7 @@ public void testRequiredTableIdentifier() { .icebergCatalog(testCatalog) .deltaLakeConfiguration(testHadoopConf) .tableLocation(newTableLocation); - Assertions.assertThatThrownBy(testAction::execute) + assertThatThrownBy(testAction::execute) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Iceberg catalog and identifier cannot be null. Make sure to configure the action with a valid Iceberg catalog and identifier."); @@ -66,7 +67,7 @@ public void testRequiredIcebergCatalog() { .as(TableIdentifier.of("test", "test")) .deltaLakeConfiguration(testHadoopConf) .tableLocation(newTableLocation); - Assertions.assertThatThrownBy(testAction::execute) + assertThatThrownBy(testAction::execute) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Iceberg catalog and identifier cannot be null. Make sure to configure the action with a valid Iceberg catalog and identifier."); @@ -79,7 +80,7 @@ public void testRequiredDeltaLakeConfiguration() { .as(TableIdentifier.of("test", "test")) .icebergCatalog(testCatalog) .tableLocation(newTableLocation); - Assertions.assertThatThrownBy(testAction::execute) + assertThatThrownBy(testAction::execute) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Make sure to configure the action with a valid deltaLakeConfiguration"); } @@ -92,7 +93,7 @@ public void testDeltaTableNotExist() { .deltaLakeConfiguration(testHadoopConf) .icebergCatalog(testCatalog) .tableLocation(newTableLocation); - Assertions.assertThatThrownBy(testAction::execute) + assertThatThrownBy(testAction::execute) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Delta Lake table does not exist at the given location: %s", sourceTableLocation); diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index 20989ee8c013..6d99d64470f8 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.delta; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import io.delta.standalone.types.ArrayType; import io.delta.standalone.types.BinaryType; import io.delta.standalone.types.BooleanType; @@ -31,7 +34,6 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -75,12 +77,10 @@ public void testAtomicTypeConversion() { deltaAtomicSchema, new DeltaLakeTypeToType(deltaAtomicSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - Assertions.assertThat(convertedSchema.findType(optionalBooleanType)) - .isInstanceOf(Types.BooleanType.class); - Assertions.assertThat(convertedSchema.findField(optionalBooleanType).isOptional()).isTrue(); - Assertions.assertThat(convertedSchema.findType(requiredBinaryType)) - .isInstanceOf(Types.BinaryType.class); - Assertions.assertThat(convertedSchema.findField(requiredBinaryType).isRequired()).isTrue(); + assertThat(convertedSchema.findType(optionalBooleanType)).isInstanceOf(Types.BooleanType.class); + assertThat(convertedSchema.findField(optionalBooleanType).isOptional()).isTrue(); + assertThat(convertedSchema.findType(requiredBinaryType)).isInstanceOf(Types.BinaryType.class); + assertThat(convertedSchema.findField(requiredBinaryType).isRequired()).isTrue(); } @Test @@ -90,49 +90,41 @@ public void testNestedTypeConversion() { deltaNestedSchema, new DeltaLakeTypeToType(deltaNestedSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - Assertions.assertThat(convertedSchema.findType(innerAtomicSchema)) - .isInstanceOf(Types.StructType.class); - Assertions.assertThat(convertedSchema.findField(innerAtomicSchema).isOptional()).isTrue(); - Assertions.assertThat( + assertThat(convertedSchema.findType(innerAtomicSchema)).isInstanceOf(Types.StructType.class); + assertThat(convertedSchema.findField(innerAtomicSchema).isOptional()).isTrue(); + assertThat( convertedSchema .findType(innerAtomicSchema) .asStructType() .fieldType(optionalBooleanType)) .isInstanceOf(Types.BooleanType.class); - Assertions.assertThat( + assertThat( convertedSchema .findType(innerAtomicSchema) .asStructType() .fieldType(requiredBinaryType)) .isInstanceOf(Types.BinaryType.class); - Assertions.assertThat( + assertThat( convertedSchema .findType(innerAtomicSchema) .asStructType() .field(requiredBinaryType) .isRequired()) .isTrue(); - Assertions.assertThat(convertedSchema.findType(stringLongMapType)) - .isInstanceOf(Types.MapType.class); - Assertions.assertThat(convertedSchema.findType(stringLongMapType).asMapType().keyType()) + assertThat(convertedSchema.findType(stringLongMapType)).isInstanceOf(Types.MapType.class); + assertThat(convertedSchema.findType(stringLongMapType).asMapType().keyType()) .isInstanceOf(Types.StringType.class); - Assertions.assertThat(convertedSchema.findType(stringLongMapType).asMapType().valueType()) + assertThat(convertedSchema.findType(stringLongMapType).asMapType().valueType()) .isInstanceOf(Types.LongType.class); - Assertions.assertThat(convertedSchema.findType(doubleArrayType)) - .isInstanceOf(Types.ListType.class); - Assertions.assertThat(convertedSchema.findField(doubleArrayType).isRequired()).isTrue(); - Assertions.assertThat( - convertedSchema.findType(doubleArrayType).asListType().isElementOptional()) - .isTrue(); - Assertions.assertThat(convertedSchema.findType(structArrayType)) - .isInstanceOf(Types.ListType.class); - Assertions.assertThat(convertedSchema.findField(structArrayType).isRequired()).isTrue(); - Assertions.assertThat( - convertedSchema.findType(structArrayType).asListType().isElementOptional()) - .isTrue(); - Assertions.assertThat(convertedSchema.findType(structArrayType).asListType().elementType()) + assertThat(convertedSchema.findType(doubleArrayType)).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField(doubleArrayType).isRequired()).isTrue(); + assertThat(convertedSchema.findType(doubleArrayType).asListType().isElementOptional()).isTrue(); + assertThat(convertedSchema.findType(structArrayType)).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField(structArrayType).isRequired()).isTrue(); + assertThat(convertedSchema.findType(structArrayType).asListType().isElementOptional()).isTrue(); + assertThat(convertedSchema.findType(structArrayType).asListType().elementType()) .isInstanceOf(Types.StructType.class); - Assertions.assertThat( + assertThat( convertedSchema .findType(structArrayType) .asListType() @@ -140,7 +132,7 @@ public void testNestedTypeConversion() { .asStructType() .fieldType(optionalBooleanType)) .isInstanceOf(Types.BooleanType.class); - Assertions.assertThat( + assertThat( convertedSchema .findType(structArrayType) .asListType() @@ -149,7 +141,7 @@ public void testNestedTypeConversion() { .field(optionalBooleanType) .isOptional()) .isTrue(); - Assertions.assertThat( + assertThat( convertedSchema .findType(structArrayType) .asListType() @@ -157,7 +149,7 @@ public void testNestedTypeConversion() { .asStructType() .fieldType(requiredBinaryType)) .isInstanceOf(Types.BinaryType.class); - Assertions.assertThat( + assertThat( convertedSchema .findType(structArrayType) .asListType() @@ -170,13 +162,13 @@ public void testNestedTypeConversion() { @Test public void testNullTypeConversion() { - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> DeltaLakeDataTypeVisitor.visit( deltaNullTypeSchema, new DeltaLakeTypeToType(deltaNullTypeSchema))) .isInstanceOf(ValidationException.class) .hasMessage(String.format("Not a supported type: %s", new NullType().getCatalogString())); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> DeltaLakeDataTypeVisitor.visit( deltaShallowNullTypeSchema, 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 f7c770dbb583..c71b55828758 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/GCPPropertiesTest.java @@ -21,16 +21,16 @@ import static org.apache.iceberg.gcp.GCPProperties.GCS_NO_AUTH; 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; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class GCPPropertiesTest { @Test public void testOAuthWithNoAuth() { - Assertions.assertThatIllegalStateException() + assertThatIllegalStateException() .isThrownBy( () -> new GCPProperties(ImmutableMap.of(GCS_OAUTH2_TOKEN, "oauth", GCS_NO_AUTH, "true"))) diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java index 76a0fa6f52ba..db6b5d93893b 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSInputStreamTest.java @@ -34,7 +34,6 @@ import org.apache.iceberg.io.RangeReadable; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.metrics.MetricsContext; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class GCSInputStreamTest { @@ -157,7 +156,7 @@ private void readAndCheckRanges( throws IOException { in.readFully(position, buffer, offset, length); - Assertions.assertThat(Arrays.copyOfRange(buffer, offset, offset + length)) + assertThat(Arrays.copyOfRange(buffer, offset, offset + length)) .isEqualTo(Arrays.copyOfRange(original, offset, offset + length)); } diff --git a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSLocationTest.java b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSLocationTest.java index 551eb1374d88..e2ad925cf307 100644 --- a/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSLocationTest.java +++ b/gcp/src/test/java/org/apache/iceberg/gcp/gcs/GCSLocationTest.java @@ -18,8 +18,10 @@ */ package org.apache.iceberg.gcp.gcs; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.exceptions.ValidationException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class GCSLocationTest { @@ -28,8 +30,8 @@ public void testLocationParsing() { String p1 = "gs://bucket/path/to/prefix"; GCSLocation location = new GCSLocation(p1); - Assertions.assertThat(location.bucket()).isEqualTo("bucket"); - Assertions.assertThat(location.prefix()).isEqualTo("path/to/prefix"); + assertThat(location.bucket()).isEqualTo("bucket"); + assertThat(location.prefix()).isEqualTo("path/to/prefix"); } @Test @@ -37,20 +39,20 @@ public void testEncodedString() { String p1 = "gs://bucket/path%20to%20prefix"; GCSLocation location = new GCSLocation(p1); - Assertions.assertThat(location.bucket()).isEqualTo("bucket"); - Assertions.assertThat(location.prefix()).isEqualTo("path%20to%20prefix"); + assertThat(location.bucket()).isEqualTo("bucket"); + assertThat(location.prefix()).isEqualTo("path%20to%20prefix"); } @Test public void testMissingScheme() { - Assertions.assertThatThrownBy(() -> new GCSLocation("/path/to/prefix")) + assertThatThrownBy(() -> new GCSLocation("/path/to/prefix")) .isInstanceOf(ValidationException.class) .hasMessage("Invalid GCS URI, cannot determine scheme: /path/to/prefix"); } @Test public void testInvalidScheme() { - Assertions.assertThatThrownBy(() -> new GCSLocation("s3://bucket/path/to/prefix")) + assertThatThrownBy(() -> new GCSLocation("s3://bucket/path/to/prefix")) .isInstanceOf(ValidationException.class) .hasMessage("Invalid GCS URI, invalid scheme: s3"); } @@ -60,8 +62,8 @@ public void testOnlyBucketNameLocation() { String p1 = "gs://bucket"; GCSLocation location = new GCSLocation(p1); - Assertions.assertThat(location.bucket()).isEqualTo("bucket"); - Assertions.assertThat(location.prefix()).isEqualTo(""); + assertThat(location.bucket()).isEqualTo("bucket"); + assertThat(location.prefix()).isEqualTo(""); } @Test @@ -69,7 +71,7 @@ public void testQueryAndFragment() { String p1 = "gs://bucket/path/to/prefix?query=foo#bar"; GCSLocation location = new GCSLocation(p1); - Assertions.assertThat(location.bucket()).isEqualTo("bucket"); - Assertions.assertThat(location.prefix()).isEqualTo("path/to/prefix"); + assertThat(location.bucket()).isEqualTo("bucket"); + assertThat(location.prefix()).isEqualTo("path/to/prefix"); } } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java index 121e2c8b6d8a..579a4b810c54 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergFilterFactory.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.math.BigDecimal; import java.sql.Date; @@ -39,7 +40,6 @@ import org.apache.iceberg.expressions.UnboundPredicate; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestHiveIcebergFilterFactory { @@ -159,7 +159,7 @@ public void testUnsupportedBetweenOperandEmptyLeaves() { .between("salary", PredicateLeaf.Type.LONG, 9000L, 15000L) .end() .build()); - Assertions.assertThatThrownBy(() -> HiveIcebergFilterFactory.generateFilterExpression(arg)) + assertThatThrownBy(() -> HiveIcebergFilterFactory.generateFilterExpression(arg)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Missing leaf literals: Leaf[empty]"); } diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java index 8b8e209144fa..147e0ba4685a 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergOutputCommitter.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.mr.hive.HiveIcebergRecordWriter.getWriters; 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 java.nio.file.Path; @@ -56,7 +57,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializationUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.mockito.ArgumentCaptor; @@ -205,8 +205,7 @@ public void writerIsClosedAfterTaskCommitFailure() throws IOException { Table table = table(temp.toFile().getPath(), false); JobConf conf = jobConf(table, 1); - Assertions.assertThatThrownBy( - () -> writeRecords(table.name(), 1, 0, true, false, conf, failingCommitter)) + assertThatThrownBy(() -> writeRecords(table.name(), 1, 0, true, false, conf, failingCommitter)) .isInstanceOf(RuntimeException.class) .hasMessage(exceptionMessage); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java index f38be059360b..00aa8458bf5a 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestBranchVisibility.java @@ -19,6 +19,8 @@ package org.apache.iceberg.nessie; 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.util.Arrays; import java.util.Collections; @@ -41,7 +43,6 @@ import org.apache.iceberg.view.BaseView; import org.apache.iceberg.view.View; import org.assertj.core.api.AbstractStringAssert; -import org.assertj.core.api.Assertions; import org.assertj.core.api.InstanceOfAssertFactories; import org.assertj.core.api.ThrowableAssert.ThrowingCallable; import org.junit.jupiter.api.AfterEach; @@ -121,15 +122,14 @@ public void testCatalogWithTableNames() { String mainName = "main"; // asking for table@branch gives expected regardless of catalog - Assertions.assertThat(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@test"))) + assertThat(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@test"))) .isEqualTo(metadataLocation(testCatalog, tableIdentifier1)); // Asking for table@branch gives expected regardless of catalog. // Earlier versions used "table1@" + tree.getReferenceByName("main").getHash() before, but since // Nessie 0.8.2 the branch name became mandatory and specifying a hash within a branch is not // possible. - Assertions.assertThat( - metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@" + mainName))) + assertThat(metadataLocation(catalog, TableIdentifier.of("test-ns", "table1@" + mainName))) .isEqualTo(metadataLocation(testCatalog, tableIdentifier1)); } @@ -160,7 +160,7 @@ public void testSchemaSnapshot() throws Exception { String metadataOnTest2 = addRow(catalog, tableIdentifier1, "added-data-on-test", ImmutableMap.of("id0", 5L)); - Assertions.assertThat(metadataOnTest2).isNotEqualTo(metadataOnTest); + assertThat(metadataOnTest2).isNotEqualTo(metadataOnTest); long snapshotIdOnTest2 = snapshotIdFromMetadata(catalog, metadataOnTest2); verifyRefState(catalog, tableIdentifier1, snapshotIdOnTest2, 0); @@ -173,7 +173,7 @@ public void testSchemaSnapshot() throws Exception { tableIdentifier1, "testSchemaSnapshot-in-1", ImmutableMap.of("id0", 42L, "id1", "world")); - Assertions.assertThat(metadataOn1).isNotEqualTo(metadataOnTest).isNotEqualTo(metadataOnTest2); + assertThat(metadataOn1).isNotEqualTo(metadataOnTest).isNotEqualTo(metadataOnTest2); NessieCatalog catalogBranch2 = initCatalog(branch2); updateSchema(catalogBranch2, tableIdentifier1, Types.IntegerType.get()); @@ -184,7 +184,7 @@ public void testSchemaSnapshot() throws Exception { tableIdentifier1, "testSchemaSnapshot-in-2", ImmutableMap.of("id0", 43L, "id2", 666)); - Assertions.assertThat(metadataOn2).isNotEqualTo(metadataOnTest).isNotEqualTo(metadataOnTest2); + assertThat(metadataOn2).isNotEqualTo(metadataOnTest).isNotEqualTo(metadataOnTest2); } @Test @@ -202,15 +202,13 @@ public void testMetadataLocation() throws Exception { catalog = initCatalog(branch2); String metadataLocationOfCommit2 = addRow(catalog, tableIdentifier1, "some-more-data", ImmutableMap.of("id0", 42L)); - Assertions.assertThat(metadataLocationOfCommit2) - .isNotNull() - .isNotEqualTo(metadataLocationOfCommit1); + assertThat(metadataLocationOfCommit2).isNotNull().isNotEqualTo(metadataLocationOfCommit1); catalog = initCatalog(branch1); // load tableIdentifier1 on branch1 BaseTable table = (BaseTable) catalog.loadTable(tableIdentifier1); // branch1's tableIdentifier1's metadata location must not have changed - Assertions.assertThat(table.operations().current().metadataFileLocation()) + assertThat(table.operations().current().metadataFileLocation()) .isNotNull() .isNotEqualTo(metadataLocationOfCommit2); } @@ -236,7 +234,7 @@ public void testStateTrackingOnMultipleBranches() throws Exception { // Add a row and verify that the String metadataOnTest = addRow(catalog, tableIdentifier1, "initial-data", Collections.singletonMap("id0", 1L)); - Assertions.assertThat(metadataOnTest).isNotEqualTo(initialLocation); + assertThat(metadataOnTest).isNotEqualTo(initialLocation); long snapshotIdOnTest = snapshotIdFromMetadata(catalog, metadataOnTest); verifyRefState(catalog, tableIdentifier1, snapshotIdOnTest, 0); @@ -260,15 +258,15 @@ public void testStateTrackingOnMultipleBranches() throws Exception { "branch-a-1", ImmutableMap.of("id0", 2L, "id1", "hello")); // addRow() must produce a new metadata - Assertions.assertThat(metadataOnA1).isNotEqualTo(metadataOnTest); + assertThat(metadataOnA1).isNotEqualTo(metadataOnTest); long snapshotIdOnA1 = snapshotIdFromMetadata(catalogBranchA, metadataOnA1); - Assertions.assertThat(snapshotIdOnA1).isNotEqualTo(snapshotIdOnTest); + assertThat(snapshotIdOnA1).isNotEqualTo(snapshotIdOnTest); verifyRefState(catalogBranchA, tableIdentifier1, snapshotIdOnA1, 1); verifyRefState(catalog, tableIdentifier1, snapshotIdOnTest, 0); NessieCatalog catalogBranchB = initCatalog(branchB); long snapshotIdOnB = snapshotIdFromNessie(catalogBranchB, tableIdentifier1); - Assertions.assertThat(snapshotIdOnB).isEqualTo(snapshotIdOnTest); + assertThat(snapshotIdOnB).isEqualTo(snapshotIdOnTest); // branchB hasn't been modified yet, so it must be "equal" to branch "test" verifyRefState(catalogBranchB, tableIdentifier1, snapshotIdOnB, 0); // updateSchema should use schema-id 1, because it's not tracked globally @@ -282,7 +280,7 @@ public void testStateTrackingOnMultipleBranches() throws Exception { catalogBranchB, tableIdentifier1, "branch-b-1", ImmutableMap.of("id0", 3L, "id2", 42L)); long snapshotIdOnB1 = snapshotIdFromMetadata(catalogBranchB, metadataOnB1); // addRow() must produce a new metadata - Assertions.assertThat(metadataOnB1).isNotEqualTo(metadataOnA1).isNotEqualTo(metadataOnTest); + assertThat(metadataOnB1).isNotEqualTo(metadataOnA1).isNotEqualTo(metadataOnTest); verifyRefState(catalogBranchB, tableIdentifier1, snapshotIdOnB1, 1); verifyRefState(catalog, tableIdentifier1, snapshotIdOnTest, 0); @@ -296,7 +294,7 @@ public void testStateTrackingOnMultipleBranches() throws Exception { "branch-a-2", ImmutableMap.of("id0", 4L, "id1", "hello")); long snapshotIdOnA2 = snapshotIdFromMetadata(catalogBranchA, metadataOnA2); - Assertions.assertThat(metadataOnA2) + assertThat(metadataOnA2) .isNotEqualTo(metadataOnA1) .isNotEqualTo(metadataOnB1) .isNotEqualTo(metadataOnTest); @@ -312,7 +310,7 @@ public void testStateTrackingOnMultipleBranches() throws Exception { "branch-b-2", ImmutableMap.of("id0", 5L, "id2", 666L)); long snapshotIdOnB2 = snapshotIdFromMetadata(catalogBranchA, metadataOnB2); - Assertions.assertThat(metadataOnB2) + assertThat(metadataOnB2) .isNotEqualTo(metadataOnA1) .isNotEqualTo(metadataOnA2) .isNotEqualTo(metadataOnB1) @@ -327,7 +325,7 @@ private void verifyRefState( NessieCatalog catalog, TableIdentifier identifier, long snapshotId, int schemaId) throws Exception { IcebergTable icebergTable = loadIcebergTable(catalog, identifier); - Assertions.assertThat(icebergTable) + assertThat(icebergTable) .extracting(IcebergTable::getSnapshotId, IcebergTable::getSchemaId) .containsExactly(snapshotId, schemaId); } @@ -378,7 +376,7 @@ private String addRow( } private void verifySchema(NessieCatalog catalog, TableIdentifier identifier, Type... types) { - Assertions.assertThat(catalog.loadTable(identifier)) + assertThat(catalog.loadTable(identifier)) .extracting(t -> t.schema().columns().stream().map(NestedField::type)) .asInstanceOf(InstanceOfAssertFactories.stream(Type.class)) .containsExactly(types); @@ -416,7 +414,7 @@ private void testCatalogEquality( String table2 = metadataLocation(catalog, tableIdentifier2); AbstractStringAssert assertion = - Assertions.assertThat(table1) + assertThat(table1) .describedAs( "Table %s on ref %s should%s be equal to table %s on ref %s", tableIdentifier1.name(), @@ -431,7 +429,7 @@ private void testCatalogEquality( } assertion = - Assertions.assertThat(table2) + assertThat(table2) .describedAs( "Table %s on ref %s should%s be equal to table %s on ref %s", tableIdentifier2.name(), @@ -457,41 +455,41 @@ public void testWithRefAndHash() throws NessieConflictException, NessieNotFoundE String hashBeforeNamespaceCreation = api.getReference().refName(testBranch).get().getHash(); Namespace namespaceA = Namespace.of("a"); Namespace namespaceAB = Namespace.of("a", "b"); - Assertions.assertThat(nessieCatalog.listNamespaces(namespaceAB)).isEmpty(); + assertThat(nessieCatalog.listNamespaces(namespaceAB)).isEmpty(); createMissingNamespaces( nessieCatalog, Namespace.of(Arrays.copyOf(namespaceAB.levels(), namespaceAB.length() - 1))); nessieCatalog.createNamespace(namespaceAB); - Assertions.assertThat(nessieCatalog.listNamespaces(namespaceAB)).isEmpty(); - Assertions.assertThat(nessieCatalog.listNamespaces(namespaceA)).containsExactly(namespaceAB); - Assertions.assertThat(nessieCatalog.listTables(namespaceAB)).isEmpty(); + assertThat(nessieCatalog.listNamespaces(namespaceAB)).isEmpty(); + assertThat(nessieCatalog.listNamespaces(namespaceA)).containsExactly(namespaceAB); + assertThat(nessieCatalog.listTables(namespaceAB)).isEmpty(); NessieCatalog catalogAtHash1 = initCatalog(testBranch, hashBeforeNamespaceCreation); - Assertions.assertThat(catalogAtHash1.listNamespaces(namespaceAB)).isEmpty(); - Assertions.assertThat(catalogAtHash1.listTables(namespaceAB)).isEmpty(); + assertThat(catalogAtHash1.listNamespaces(namespaceAB)).isEmpty(); + assertThat(catalogAtHash1.listTables(namespaceAB)).isEmpty(); TableIdentifier identifier = TableIdentifier.of(namespaceAB, "table"); String hashBeforeTableCreation = nessieCatalog.currentHash(); nessieCatalog.createTable(identifier, schema); - Assertions.assertThat(nessieCatalog.listTables(namespaceAB)).hasSize(1); + assertThat(nessieCatalog.listTables(namespaceAB)).hasSize(1); NessieCatalog catalogAtHash2 = initCatalog(testBranch, hashBeforeTableCreation); - Assertions.assertThat(catalogAtHash2.listNamespaces(namespaceAB)).isEmpty(); - Assertions.assertThat(catalogAtHash2.listNamespaces(namespaceA)).containsExactly(namespaceAB); - Assertions.assertThat(catalogAtHash2.listTables(namespaceAB)).isEmpty(); + assertThat(catalogAtHash2.listNamespaces(namespaceAB)).isEmpty(); + assertThat(catalogAtHash2.listNamespaces(namespaceA)).containsExactly(namespaceAB); + assertThat(catalogAtHash2.listTables(namespaceAB)).isEmpty(); // updates should not be possible - Assertions.assertThatThrownBy(() -> catalogAtHash2.createTable(identifier, schema)) + assertThatThrownBy(() -> catalogAtHash2.createTable(identifier, schema)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "You can only mutate tables/views when using a branch without a hash or timestamp."); - Assertions.assertThat(catalogAtHash2.listTables(namespaceAB)).isEmpty(); + assertThat(catalogAtHash2.listTables(namespaceAB)).isEmpty(); // updates should be still possible here nessieCatalog = initCatalog(testBranch); TableIdentifier identifier2 = TableIdentifier.of(namespaceAB, "table2"); nessieCatalog.createTable(identifier2, schema); - Assertions.assertThat(nessieCatalog.listTables(namespaceAB)).hasSize(2); + assertThat(nessieCatalog.listTables(namespaceAB)).hasSize(2); } @Test @@ -515,14 +513,14 @@ public void testDifferentTableSameName() throws NessieConflictException, NessieN createMissingNamespaces(nessieCatalog, identifier); Table table1 = nessieCatalog.createTable(identifier, schema1); - Assertions.assertThat(table1.schema().asStruct()).isEqualTo(schema1.asStruct()); + assertThat(table1.schema().asStruct()).isEqualTo(schema1.asStruct()); nessieCatalog = initCatalog(branch2); createMissingNamespaces(nessieCatalog, identifier); Table table2 = nessieCatalog.createTable(identifier, schema2); - Assertions.assertThat(table2.schema().asStruct()).isEqualTo(schema2.asStruct()); + assertThat(table2.schema().asStruct()).isEqualTo(schema2.asStruct()); - Assertions.assertThat(table1.location()).isNotEqualTo(table2.location()); + assertThat(table1.location()).isNotEqualTo(table2.location()); } @Test @@ -551,15 +549,13 @@ public void testViewMetadataLocation() throws Exception { .current() .metadataFileLocation(); - Assertions.assertThat(metadataLocationOfCommit2) - .isNotNull() - .isNotEqualTo(metadataLocationOfCommit1); + assertThat(metadataLocationOfCommit2).isNotNull().isNotEqualTo(metadataLocationOfCommit1); catalog = initCatalog(branch1); // load viewIdentifier on branch1 BaseView view = (BaseView) catalog.loadView(viewIdentifier); // branch1's viewIdentifier's metadata location must not have changed - Assertions.assertThat(view.operations().current().metadataFileLocation()) + assertThat(view.operations().current().metadataFileLocation()) .isNotNull() .isNotEqualTo(metadataLocationOfCommit2); @@ -587,13 +583,13 @@ public void testDifferentViewSameName() throws NessieConflictException, NessieNo createMissingNamespaces(nessieCatalog, identifier); View view1 = createView(nessieCatalog, identifier, schema1); - Assertions.assertThat(view1.schema().asStruct()).isEqualTo(schema1.asStruct()); + assertThat(view1.schema().asStruct()).isEqualTo(schema1.asStruct()); nessieCatalog = initCatalog(branch2); createMissingNamespaces(nessieCatalog, identifier); View view2 = createView(nessieCatalog, identifier, schema2); - Assertions.assertThat(view2.schema().asStruct()).isEqualTo(schema2.asStruct()); + assertThat(view2.schema().asStruct()).isEqualTo(schema2.asStruct()); - Assertions.assertThat(view1.location()).isNotEqualTo(view2.location()); + assertThat(view1.location()).isNotEqualTo(view2.location()); } } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java index b6ae90650e89..d6f4f68f511e 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java @@ -19,6 +19,8 @@ package org.apache.iceberg.nessie; 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 java.net.URI; @@ -34,7 +36,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -72,34 +73,33 @@ public void afterEach() throws Exception { @Test public void testListNamespaces() throws NessieConflictException, NessieNotFoundException { - Assertions.assertThat(catalog.listNamespaces()).isEmpty(); - Assertions.assertThat(anotherCatalog.listNamespaces()).isEmpty(); + assertThat(catalog.listNamespaces()).isEmpty(); + assertThat(anotherCatalog.listNamespaces()).isEmpty(); // listing a non-existent namespace should return empty - Assertions.assertThat(catalog.listNamespaces(Namespace.of("db1"))).isEmpty(); - Assertions.assertThat(anotherCatalog.listNamespaces(Namespace.of("db1"))).isEmpty(); + assertThat(catalog.listNamespaces(Namespace.of("db1"))).isEmpty(); + assertThat(anotherCatalog.listNamespaces(Namespace.of("db1"))).isEmpty(); catalog.createNamespace(Namespace.of("db1"), Collections.emptyMap()); - Assertions.assertThat(catalog.listNamespaces()).containsExactlyInAnyOrder(Namespace.of("db1")); - Assertions.assertThat(anotherCatalog.listNamespaces()) - .containsExactlyInAnyOrder(Namespace.of("db1")); + assertThat(catalog.listNamespaces()).containsExactlyInAnyOrder(Namespace.of("db1")); + assertThat(anotherCatalog.listNamespaces()).containsExactlyInAnyOrder(Namespace.of("db1")); // another client creates a namespace with the same nessie server anotherCatalog.createNamespace(Namespace.of("db2"), Collections.emptyMap()); - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); - Assertions.assertThat(anotherCatalog.listNamespaces()) + assertThat(anotherCatalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("db1"), Namespace.of("db2")); api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); - Assertions.assertThatThrownBy(() -> catalog.listNamespaces()) + assertThatThrownBy(() -> catalog.listNamespaces()) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining( "Cannot list top-level namespaces: ref '%s' is no longer valid", branch); - Assertions.assertThatThrownBy(() -> anotherCatalog.listNamespaces(Namespace.of("db1"))) + assertThatThrownBy(() -> anotherCatalog.listNamespaces(Namespace.of("db1"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining( "Cannot list child namespaces from 'db1': ref '%s' is no longer valid", branch); @@ -107,25 +107,23 @@ public void testListNamespaces() throws NessieConflictException, NessieNotFoundE @Test public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNotFoundException { - Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Namespace does not exist: namespace1"); - Assertions.assertThatThrownBy( - () -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + assertThatThrownBy(() -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Namespace does not exist: namespace1"); catalog.createNamespace(Namespace.of("namespace1"), Collections.emptyMap()); // both clients should see the namespace because we read the HEAD of the ref - Assertions.assertThat(catalog.listNamespaces()) - .containsExactlyInAnyOrder(Namespace.of("namespace1")); - Assertions.assertThat(anotherCatalog.listNamespaces()) + assertThat(catalog.listNamespaces()).containsExactlyInAnyOrder(Namespace.of("namespace1")); + assertThat(anotherCatalog.listNamespaces()) .containsExactlyInAnyOrder(Namespace.of("namespace1")); // the other client should not be able to update the namespace // because it is still on the old ref hash - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> anotherCatalog.setProperties( Namespace.of("namespace1"), Collections.singletonMap("k1", "v1"))) @@ -136,19 +134,18 @@ public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNo // load metadata from the same client and another client both should work fine // because we read the HEAD of the ref - Assertions.assertThat(anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + assertThat(anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) .containsExactly(Map.entry("k1", "v1")); - Assertions.assertThat(catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + assertThat(catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) .containsExactly(Map.entry("k1", "v1")); api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); - Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.of("namespace1"))) .isInstanceOf(RuntimeException.class) .hasMessageContaining( "Cannot load namespace 'namespace1': ref '%s' is no longer valid", branch); - Assertions.assertThatThrownBy( - () -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) + assertThatThrownBy(() -> anotherCatalog.loadNamespaceMetadata(Namespace.of("namespace1"))) .isInstanceOf(RuntimeException.class) .hasMessageContaining( "Cannot load namespace 'namespace1': ref '%s' is no longer valid", branch); @@ -157,16 +154,16 @@ public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNo @Test public void testListTables() { createTable(TableIdentifier.parse("foo.tbl1"), schema); - Assertions.assertThat(catalog.listTables(Namespace.of("foo"))) + assertThat(catalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder(TableIdentifier.parse("foo.tbl1")); // another client creates a table with the same nessie server anotherCatalog.createTable(TableIdentifier.parse("foo.tbl2"), schema); - Assertions.assertThat(anotherCatalog.listTables(Namespace.of("foo"))) + assertThat(anotherCatalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder( TableIdentifier.parse("foo.tbl1"), TableIdentifier.parse("foo.tbl2")); - Assertions.assertThat(catalog.listTables(Namespace.of("foo"))) + assertThat(catalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder( TableIdentifier.parse("foo.tbl1"), TableIdentifier.parse("foo.tbl2")); } @@ -184,8 +181,8 @@ public void testCommits() { tableFromCatalog.updateSchema().addColumn("x3", Types.LongType.get()).commit(); tableFromAnotherCatalog.updateSchema().addColumn("x4", Types.LongType.get()).commit(); - Assertions.assertThat(catalog.loadTable(identifier).schema().columns()).hasSize(5); - Assertions.assertThat(anotherCatalog.loadTable(identifier).schema().columns()).hasSize(5); + assertThat(catalog.loadTable(identifier).schema().columns()).hasSize(5); + assertThat(anotherCatalog.loadTable(identifier).schema().columns()).hasSize(5); } @Test @@ -207,10 +204,10 @@ public void testConcurrentCommitsWithRefresh() { // refresh the catalog's client. String hashAfter = catalog.currentHash(); - Assertions.assertThat(hashBefore).isNotEqualTo(hashAfter); + assertThat(hashBefore).isNotEqualTo(hashAfter); // client refresh should not affect the ongoing commits (commit should still fail due staleness) - Assertions.assertThatThrownBy(() -> ops1.commit(ops1.current(), metadata1)) + assertThatThrownBy(() -> ops1.commit(ops1.current(), metadata1)) .isInstanceOf(CommitFailedException.class) .hasMessageContaining( "Cannot commit: Reference hash is out of date. Update the reference 'multiple-clients-test' and try again"); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java index 7238df9c1281..29b9f73cb00d 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNamespace.java @@ -19,6 +19,8 @@ package org.apache.iceberg.nessie; 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.util.Arrays; import java.util.Collections; @@ -34,7 +36,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.projectnessie.error.NessieNotFoundException; import org.projectnessie.model.ContentKey; @@ -63,56 +64,55 @@ public void testListNamespaces() { createTable(TableIdentifier.of("t6")); List tables = catalog.listTables(nsABC); - Assertions.assertThat(tables).isNotNull().hasSize(1); + assertThat(tables).isNotNull().hasSize(1); tables = catalog.listTables(nsAB); - Assertions.assertThat(tables).isNotNull().hasSize(2); + assertThat(tables).isNotNull().hasSize(2); tables = catalog.listTables(nsA); - Assertions.assertThat(tables).isNotNull().hasSize(3); + assertThat(tables).isNotNull().hasSize(3); tables = catalog.listTables(null); - Assertions.assertThat(tables).isNotNull().hasSize(6); + assertThat(tables).isNotNull().hasSize(6); List namespaces = catalog.listNamespaces(); - Assertions.assertThat(namespaces).containsExactly(nsA, nsB); + assertThat(namespaces).containsExactly(nsA, nsB); namespaces = catalog.listNamespaces(nsA); - Assertions.assertThat(namespaces).containsExactly(nsAB); + assertThat(namespaces).containsExactly(nsAB); namespaces = catalog.listNamespaces(nsAB); - Assertions.assertThat(namespaces).containsExactly(nsABC); + assertThat(namespaces).containsExactly(nsABC); namespaces = catalog.listNamespaces(nsB); - Assertions.assertThat(namespaces).containsExactly(nsBC); + assertThat(namespaces).containsExactly(nsBC); } @Test public void testCreatingAndDroppingNamespace() { Namespace namespace = Namespace.of("test"); catalog.createNamespace(namespace, ImmutableMap.of()); - Assertions.assertThat(catalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.namespaceExists(namespace)).isTrue(); catalog.dropNamespace(namespace); - Assertions.assertThat(catalog.namespaceExists(namespace)).isFalse(); + assertThat(catalog.namespaceExists(namespace)).isFalse(); } @Test public void testCreatingAndDroppingNamespaceWithContent() throws NessieNotFoundException { Namespace namespace = Namespace.of("test"); catalog.createNamespace(namespace, ImmutableMap.of()); - Assertions.assertThat(catalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.namespaceExists(namespace)).isTrue(); TableIdentifier identifier = TableIdentifier.of(namespace, "tbl"); Schema schema = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - Assertions.assertThat(catalog.createTable(identifier, schema)).isNotNull(); + assertThat(catalog.createTable(identifier, schema)).isNotNull(); ContentKey key = NessieUtil.toKey(identifier); - Assertions.assertThat( - api.getContent().key(key).refName(BRANCH).get().get(key).unwrap(IcebergTable.class)) + assertThat(api.getContent().key(key).refName(BRANCH).get().get(key).unwrap(IcebergTable.class)) .isPresent(); - Assertions.assertThatThrownBy(() -> catalog.dropNamespace(namespace)) + assertThatThrownBy(() -> catalog.dropNamespace(namespace)) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageContaining("Namespace 'test' is not empty"); catalog.dropTable(identifier, true); catalog.dropNamespace(namespace); - Assertions.assertThat(catalog.namespaceExists(namespace)).isFalse(); + assertThat(catalog.namespaceExists(namespace)).isFalse(); } @Test @@ -120,46 +120,43 @@ public void testSettingProperties() { Map properties = ImmutableMap.of("prop", "val"); Namespace namespace = Namespace.of("withProperties"); catalog.createNamespace(namespace, properties); - Assertions.assertThat(catalog.namespaceExists(namespace)).isTrue(); - Assertions.assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); + assertThat(catalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); ImmutableMap updatedProperties = ImmutableMap.of("prop2", "val2", "prop", "new_val"); catalog.setProperties(namespace, updatedProperties); - Assertions.assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(updatedProperties); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(updatedProperties); - Assertions.assertThatThrownBy( - () -> catalog.setProperties(Namespace.of("unknown"), updatedProperties)) + assertThatThrownBy(() -> catalog.setProperties(Namespace.of("unknown"), updatedProperties)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: unknown"); } @Test public void testEmptyNamespace() { - Assertions.assertThatThrownBy( - () -> catalog.createNamespace(Namespace.empty(), Collections.emptyMap())) + assertThatThrownBy(() -> catalog.createNamespace(Namespace.empty(), Collections.emptyMap())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Invalid namespace: "); - Assertions.assertThat(catalog.namespaceExists(Namespace.empty())).isFalse(); + assertThat(catalog.namespaceExists(Namespace.empty())).isFalse(); - Assertions.assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.empty())) + assertThatThrownBy(() -> catalog.loadNamespaceMetadata(Namespace.empty())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Invalid namespace: "); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.setProperties( Namespace.empty(), ImmutableMap.of("prop2", "val2", "prop", "val"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Invalid namespace: "); - Assertions.assertThatThrownBy( - () -> catalog.removeProperties(Namespace.empty(), ImmutableSet.of("prop2"))) + assertThatThrownBy(() -> catalog.removeProperties(Namespace.empty(), ImmutableSet.of("prop2"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Invalid namespace: "); - Assertions.assertThatThrownBy(() -> catalog.dropNamespace(Namespace.empty())) + assertThatThrownBy(() -> catalog.dropNamespace(Namespace.empty())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Invalid namespace: "); } @@ -169,15 +166,14 @@ public void testRemovingProperties() { Map properties = ImmutableMap.of("prop2", "val2", "prop", "val"); Namespace namespace = Namespace.of("withPropertyDeletes"); catalog.createNamespace(namespace, properties); - Assertions.assertThat(catalog.namespaceExists(namespace)).isTrue(); - Assertions.assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); + assertThat(catalog.namespaceExists(namespace)).isTrue(); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(properties); Set toRemove = Sets.newHashSet(Arrays.asList("prop1", "prop2", "prop3")); catalog.removeProperties(namespace, toRemove); - Assertions.assertThat(catalog.loadNamespaceMetadata(namespace)) - .isEqualTo(ImmutableMap.of("prop", "val")); + assertThat(catalog.loadNamespaceMetadata(namespace)).isEqualTo(ImmutableMap.of("prop", "val")); - Assertions.assertThatThrownBy(() -> catalog.removeProperties(Namespace.of("unknown"), toRemove)) + assertThatThrownBy(() -> catalog.removeProperties(Namespace.of("unknown"), toRemove)) .isInstanceOf(NoSuchNamespaceException.class) .hasMessage("Namespace does not exist: unknown"); } @@ -187,18 +183,15 @@ public void testCustomLocation() { Map properties = ImmutableMap.of("location", "/custom/location"); Namespace namespaceWithLocation = Namespace.of("withLocation"); catalog.createNamespace(namespaceWithLocation, properties); - Assertions.assertThat(catalog.namespaceExists(namespaceWithLocation)).isTrue(); - Assertions.assertThat( - catalog.defaultWarehouseLocation(TableIdentifier.of("withLocation", "testTable"))) + assertThat(catalog.namespaceExists(namespaceWithLocation)).isTrue(); + assertThat(catalog.defaultWarehouseLocation(TableIdentifier.of("withLocation", "testTable"))) .startsWith("/custom/location/testTable"); Namespace namespaceWithoutLocation = Namespace.of("withoutLocation"); catalog.createNamespace(namespaceWithoutLocation, ImmutableMap.of()); - Assertions.assertThat(catalog.namespaceExists(namespaceWithoutLocation)).isTrue(); - Assertions.assertThat( - catalog.defaultWarehouseLocation(TableIdentifier.of("withoutLocation", "testTable"))) + assertThat(catalog.namespaceExists(namespaceWithoutLocation)).isTrue(); + assertThat(catalog.defaultWarehouseLocation(TableIdentifier.of("withoutLocation", "testTable"))) .contains("/withoutLocation/testTable"); - Assertions.assertThat( - catalog.defaultWarehouseLocation(TableIdentifier.of("badNamespace", "testTable"))) + assertThat(catalog.defaultWarehouseLocation(TableIdentifier.of("badNamespace", "testTable"))) .contains("/badNamespace/testTable"); } } 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 dbe8f92cdf81..55be034221ae 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieCatalog.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.nessie; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.net.URI; import java.nio.file.Path; @@ -28,7 +30,6 @@ import org.apache.iceberg.catalog.CatalogTests; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.util.LocationUtil; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; @@ -161,7 +162,7 @@ public void testConcurrentCreateTransaction() { @Test public void testWarehouseLocationWithTrailingSlash() { - Assertions.assertThat(catalog.defaultWarehouseLocation(TABLE)) + assertThat(catalog.defaultWarehouseLocation(TABLE)) .startsWith( LocationUtil.stripTrailingSlash(temp.toUri().toString()) + "/" diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index e49990f3601c..05ee755d7c68 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -19,6 +19,10 @@ package org.apache.iceberg.nessie; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; +import static org.assertj.core.api.Assertions.assertThatRuntimeException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; import java.util.Arrays; @@ -37,7 +41,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.assertj.core.data.Index; import org.junit.jupiter.api.Test; import org.projectnessie.error.NessieConflictException; @@ -62,30 +65,26 @@ public TestNessieIcebergClient() { @Test public void testWithNullRefLoadsMain() throws NessieNotFoundException { NessieIcebergClient client = new NessieIcebergClient(api, null, null, ImmutableMap.of()); - Assertions.assertThat(client.getRef().getReference()) - .isEqualTo(api.getReference().refName("main").get()); + assertThat(client.getRef().getReference()).isEqualTo(api.getReference().refName("main").get()); } @Test public void testWithNullHash() throws NessieNotFoundException { NessieIcebergClient client = new NessieIcebergClient(api, BRANCH, null, ImmutableMap.of()); - Assertions.assertThat(client.getRef().getReference()) - .isEqualTo(api.getReference().refName(BRANCH).get()); + assertThat(client.getRef().getReference()).isEqualTo(api.getReference().refName(BRANCH).get()); } @Test public void testWithReference() throws NessieNotFoundException { NessieIcebergClient client = new NessieIcebergClient(api, "main", null, ImmutableMap.of()); - Assertions.assertThat(client.withReference(null, null)).isEqualTo(client); - Assertions.assertThat(client.withReference("main", null)).isNotEqualTo(client); - Assertions.assertThat( - client.withReference("main", api.getReference().refName("main").get().getHash())) + assertThat(client.withReference(null, null)).isEqualTo(client); + assertThat(client.withReference("main", null)).isNotEqualTo(client); + assertThat(client.withReference("main", api.getReference().refName("main").get().getHash())) .isEqualTo(client); - Assertions.assertThat(client.withReference(BRANCH, null)).isNotEqualTo(client); - Assertions.assertThat( - client.withReference(BRANCH, api.getReference().refName(BRANCH).get().getHash())) + assertThat(client.withReference(BRANCH, null)).isNotEqualTo(client); + assertThat(client.withReference(BRANCH, api.getReference().refName(BRANCH).get().getHash())) .isNotEqualTo(client); } @@ -99,7 +98,7 @@ public void testWithReferenceAfterRecreatingBranch() // just create a new commit on the branch and then delete & re-create it Namespace namespace = Namespace.of("a"); client.createNamespace(namespace, ImmutableMap.of()); - Assertions.assertThat(client.listNamespaces(namespace)).isNotNull(); + assertThat(client.listNamespaces(namespace)).isNotNull(); client .getApi() .deleteBranch() @@ -109,9 +108,8 @@ public void testWithReferenceAfterRecreatingBranch() // make sure the client uses the re-created branch Reference ref = client.getApi().getReference().refName(branch).get(); - Assertions.assertThat(client.withReference(branch, null).getRef().getReference()) - .isEqualTo(ref); - Assertions.assertThat(client.withReference(branch, null)).isNotEqualTo(client); + assertThat(client.withReference(branch, null).getRef().getReference()).isEqualTo(ref); + assertThat(client.withReference(branch, null)).isNotEqualTo(client); } @Test @@ -125,10 +123,10 @@ public void testCreateNamespace() throws NessieConflictException, NessieNotFound NessieIcebergClient client = new NessieIcebergClient(api, branch, null, catalogOptions); client.createNamespace(Namespace.of("a"), Map.of()); - Assertions.assertThat(client.listNamespaces(Namespace.of("a"))).isNotNull(); + assertThat(client.listNamespaces(Namespace.of("a"))).isNotNull(); List entries = api.getCommitLog().refName(branch).get().getLogEntries(); - Assertions.assertThat(entries) + assertThat(entries) .isNotEmpty() .first() .extracting(LogResponse.LogEntry::getCommitMeta) @@ -147,11 +145,11 @@ public void testCreateNamespaceInvalid() throws NessieConflictException, NessieN createBranch(branch); NessieIcebergClient client = new NessieIcebergClient(api, branch, null, Map.of()); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.empty(), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.empty(), Map.of())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Invalid namespace: "); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "b"), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "b"), Map.of())) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Cannot create namespace 'a.b': parent namespace 'a' does not exist"); } @@ -165,14 +163,14 @@ public void testCreateNamespaceConflict() client.createNamespace(Namespace.of("a"), Map.of()); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Namespace already exists: a"); client.commitTable( null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Another content object with name 'a.tbl' already exists"); } @@ -188,14 +186,14 @@ public void testCreateNamespaceExternalConflict() org.projectnessie.model.Namespace.of(ContentKey.of("a")); commit(branch, "create namespace a", Operation.Put.of(ContentKey.of("a"), nessieNs)); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.of("a"), Map.of())) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Namespace already exists: a"); IcebergTable table = IcebergTable.of("file:///tmp/iceberg", 1, 1, 1, 1); commit(branch, "create table a.tbl2", Operation.Put.of(ContentKey.of("a", "tbl"), table)); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) .isInstanceOf(AlreadyExistsException.class) .hasMessageContaining("Another content object with name 'a.tbl' already exists"); } @@ -211,7 +209,7 @@ public void testCreateNamespaceNonExistingRef() api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); - Assertions.assertThatThrownBy(() -> client.createNamespace(Namespace.of("b"), Map.of())) + assertThatThrownBy(() -> client.createNamespace(Namespace.of("b"), Map.of())) .isInstanceOf(RuntimeException.class) .hasMessageContaining( "Cannot create namespace 'b': ref 'createNamespaceNonExistingRefBranch' is no longer valid"); @@ -230,33 +228,33 @@ public void testDropNamespace() throws NessieConflictException, NessieNotFoundEx Namespace parent = Namespace.of("a"); Namespace child = Namespace.of("a", "b"); - Assertions.assertThat(client.dropNamespace(parent)).isFalse(); - Assertions.assertThat(client.dropNamespace(child)).isFalse(); + assertThat(client.dropNamespace(parent)).isFalse(); + assertThat(client.dropNamespace(child)).isFalse(); client.createNamespace(parent, Map.of()); client.createNamespace(child, Map.of()); - Assertions.assertThat(client.dropNamespace(child)).isTrue(); - Assertions.assertThat(client.dropNamespace(parent)).isTrue(); + assertThat(client.dropNamespace(child)).isTrue(); + assertThat(client.dropNamespace(parent)).isTrue(); List entries = api.getCommitLog().refName(branch).get().getLogEntries(); - Assertions.assertThat(entries) + assertThat(entries) .isNotEmpty() .extracting(LogResponse.LogEntry::getCommitMeta) .satisfies( meta -> { - Assertions.assertThat(meta.getMessage()).contains("drop namespace a"); - Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); - Assertions.assertThat(meta.getProperties()) + assertThat(meta.getMessage()).contains("drop namespace a"); + assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + assertThat(meta.getProperties()) .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); }, Index.atIndex(0)) .satisfies( meta -> { - Assertions.assertThat(meta.getMessage()).contains("drop namespace a.b"); - Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); - Assertions.assertThat(meta.getProperties()) + assertThat(meta.getMessage()).contains("drop namespace a.b"); + assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + assertThat(meta.getProperties()) .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); }, @@ -272,7 +270,7 @@ public void testDropNamespaceNotEmpty() throws NessieConflictException, NessieNo client.createNamespace(Namespace.of("a"), Map.of()); client.createNamespace(Namespace.of("a", "b"), Map.of()); - Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) + assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) .isInstanceOf(NamespaceNotEmptyException.class) .hasMessageContaining("Namespace 'a' is not empty."); } @@ -288,7 +286,7 @@ public void testDropNamespaceConflict() throws NessieConflictException, NessieNo client.commitTable( null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); - Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a", "tbl"))) + assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a", "tbl"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Content object with name 'a.tbl' is not a namespace."); } @@ -310,7 +308,7 @@ public void testDropNamespaceExternalConflict() .build(); commit(branch, "update namespace a", Operation.Put.of(ContentKey.of("a"), updated)); - Assertions.assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) + assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a"))) .isInstanceOf(RuntimeException.class) .hasMessageContaining( "Cannot drop namespace 'a': Values of existing and expected content for key 'a' are different."); @@ -327,7 +325,7 @@ public void testDropNamespaceNonExistingRef() api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); - Assertions.assertThat(client.dropNamespace(Namespace.of("a"))).isFalse(); + assertThat(client.dropNamespace(Namespace.of("a"))).isFalse(); } @Test @@ -343,23 +341,23 @@ public void testSetProperties() throws NessieConflictException, NessieNotFoundEx Namespace ns = Namespace.of("a"); client.createNamespace(ns, Map.of("k1", "v1a")); - Assertions.assertThat(client.setProperties(ns, Map.of("k1", "v1b", "k2", "v2"))).isTrue(); + assertThat(client.setProperties(ns, Map.of("k1", "v1b", "k2", "v2"))).isTrue(); - Assertions.assertThat(client.loadNamespaceMetadata(ns)) + assertThat(client.loadNamespaceMetadata(ns)) .hasSize(2) .containsEntry("k1", "v1b") .containsEntry("k2", "v2"); List entries = api.getCommitLog().refName(branch).get().getLogEntries(); - Assertions.assertThat(entries) + assertThat(entries) .isNotEmpty() .first() .extracting(LogResponse.LogEntry::getCommitMeta) .satisfies( meta -> { - Assertions.assertThat(meta.getMessage()).contains("update namespace a"); - Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); - Assertions.assertThat(meta.getProperties()) + assertThat(meta.getMessage()).contains("update namespace a"); + assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + assertThat(meta.getProperties()) .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); }); @@ -385,9 +383,9 @@ public void testSetPropertiesExternalConflict() commit(branch, "update namespace a", Operation.Put.of(key, updated)); // will generate a conflict and a retry - Assertions.assertThat(client.setProperties(ns, Map.of("k1", "v1c", "k3", "v3"))).isTrue(); + assertThat(client.setProperties(ns, Map.of("k1", "v1c", "k3", "v3"))).isTrue(); - Assertions.assertThat(client.loadNamespaceMetadata(ns)) + assertThat(client.loadNamespaceMetadata(ns)) .hasSize(3) .containsEntry("k1", "v1c") .containsEntry("k2", "v2") @@ -405,8 +403,7 @@ public void testSetPropertiesNonExistingNs() commit(branch, "delete namespace a", Operation.Delete.of(ContentKey.of("a"))); - Assertions.assertThatThrownBy( - () -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1a"))) + assertThatThrownBy(() -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1a"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Namespace does not exist: a"); } @@ -422,7 +419,7 @@ public void testSetPropertiesNonExistingRef() api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); - Assertions.assertThatThrownBy(() -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1"))) + assertThatThrownBy(() -> client.setProperties(Namespace.of("a"), Map.of("k1", "v1"))) .isInstanceOf(RuntimeException.class) .hasMessageContaining( "Cannot update properties on namespace 'a': ref 'setPropertiesNonExistingRefBranch' is no longer valid"); @@ -442,20 +439,20 @@ public void testRemoveProperties() throws NessieConflictException, NessieNotFoun client.createNamespace(ns, Map.of("k1", "v1", "k2", "v2")); - Assertions.assertThat(client.removeProperties(ns, Set.of("k1"))).isTrue(); + assertThat(client.removeProperties(ns, Set.of("k1"))).isTrue(); - Assertions.assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k2"); + assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k2"); List entries = api.getCommitLog().refName(branch).get().getLogEntries(); - Assertions.assertThat(entries) + assertThat(entries) .isNotEmpty() .first() .extracting(LogResponse.LogEntry::getCommitMeta) .satisfies( meta -> { - Assertions.assertThat(meta.getMessage()).contains("update namespace a"); - Assertions.assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); - Assertions.assertThat(meta.getProperties()) + assertThat(meta.getMessage()).contains("update namespace a"); + assertThat(meta.getAuthor()).isEqualTo("iceberg-user"); + assertThat(meta.getProperties()) .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg") .containsEntry(CatalogProperties.APP_ID, "iceberg-nessie"); }); @@ -481,9 +478,9 @@ public void testRemovePropertiesExternalConflict() commit(branch, "update namespace a", Operation.Put.of(key, updated)); // will generate a conflict and a retry - Assertions.assertThat(client.removeProperties(ns, Set.of("k2"))).isTrue(); + assertThat(client.removeProperties(ns, Set.of("k2"))).isTrue(); - Assertions.assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k3"); + assertThat(client.loadNamespaceMetadata(ns)).hasSize(1).containsOnlyKeys("k3"); } @Test @@ -497,7 +494,7 @@ public void testRemovePropertiesNonExistingNs() commit(branch, "delete namespace a", Operation.Delete.of(ContentKey.of("a"))); - Assertions.assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) + assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) .isInstanceOf(NoSuchNamespaceException.class) .hasMessageContaining("Namespace does not exist: a"); } @@ -513,7 +510,7 @@ public void testRemovePropertiesNonExistingRef() api.deleteBranch().branch((Branch) api.getReference().refName(branch).get()).delete(); - Assertions.assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) + assertThatThrownBy(() -> client.removeProperties(Namespace.of("a"), Set.of("k1"))) .isInstanceOf(RuntimeException.class) .hasMessageContaining( "Cannot update properties on namespace 'a': ref 'removePropertiesNonExistingRefBranch' is no longer valid"); @@ -525,7 +522,7 @@ public void testInvalidClientApiVersion() throws IOException { newCatalog.setConf(hadoopConfig); ImmutableMap.Builder options = ImmutableMap.builder().put("client-api-version", "3"); - Assertions.assertThatIllegalArgumentException() + assertThatIllegalArgumentException() .isThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) .withMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); } @@ -537,14 +534,14 @@ public void testInvalidClientApiVersionViaURI() throws IOException { newCatalog.setConf(hadoopConfig); ImmutableMap.Builder options = ImmutableMap.builder().put("uri", "some/uri/"); - Assertions.assertThatIllegalArgumentException() + assertThatIllegalArgumentException() .isThrownBy(() -> newCatalog.initialize("nessie", options.buildOrThrow())) .withMessage( "URI doesn't end with the version: some/uri/. Please configure `client-api-version` in the catalog properties explicitly."); ImmutableMap.Builder newOptions = ImmutableMap.builder().put("uri", "some/uri/v3"); - Assertions.assertThatIllegalArgumentException() + assertThatIllegalArgumentException() .isThrownBy(() -> newCatalog.initialize("nessie", newOptions.buildOrThrow())) .withMessage("Unsupported client-api-version: 3. Can only be 1 or 2"); } @@ -564,7 +561,7 @@ public void testClientApiVersionOverride() { .put("client-api-version", version); newCatalog.initialize("nessie", options.buildOrThrow()); // Since client-api-version is configured, API version should not be based on URI. - Assertions.assertThatRuntimeException() + assertThatRuntimeException() .isThrownBy(() -> newCatalog.loadTable(TableIdentifier.of("foo", "t1"))) .withMessageStartingWith("API version mismatch, check URI prefix"); } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index 25016100e00b..94eb3144a0a4 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -20,6 +20,9 @@ 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.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.File; import java.io.IOException; @@ -53,7 +56,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -145,10 +147,10 @@ public void verifyStateMovesForDML() throws Exception { Table tableInitialMain = catalog.loadTable(TABLE_IDENTIFIER); // verify table-metadata-location + snapshot-id - Assertions.assertThat(contentInitialMain) + assertThat(contentInitialMain) .as("global-contents + snapshot-id equal on both branches in Nessie") .isEqualTo(contentInitialBranch); - Assertions.assertThat(tableInitialMain.currentSnapshot()).isNull(); + assertThat(tableInitialMain.currentSnapshot()).isNull(); // 3. modify table in "main" branch (add some data) @@ -161,18 +163,18 @@ public void verifyStateMovesForDML() throws Exception { // --> assert getValue() against both branches returns the updated metadata-location // verify table-metadata-location - Assertions.assertThat(contentInitialMain.getMetadataLocation()) + assertThat(contentInitialMain.getMetadataLocation()) .describedAs("metadata-location must change on %s", BRANCH) .isNotEqualTo(contentsAfter1Main.getMetadataLocation()); - Assertions.assertThat(contentInitialBranch.getMetadataLocation()) + assertThat(contentInitialBranch.getMetadataLocation()) .describedAs("metadata-location must not change on %s", testCaseBranch) .isEqualTo(contentsAfter1Branch.getMetadataLocation()); - Assertions.assertThat(contentsAfter1Main) + assertThat(contentsAfter1Main) .extracting(IcebergTable::getSchemaId) .describedAs("on-reference-state must not be equal on both branches") .isEqualTo(contentsAfter1Branch.getSchemaId()); // verify manifests - Assertions.assertThat(tableAfter1Main.currentSnapshot().allManifests(tableAfter1Main.io())) + assertThat(tableAfter1Main.currentSnapshot().allManifests(tableAfter1Main.io())) .describedAs("verify number of manifests on 'main'") .hasSize(1); @@ -187,14 +189,14 @@ public void verifyStateMovesForDML() throws Exception { // --> assert getValue() against both branches returns the updated metadata-location // verify table-metadata-location - Assertions.assertThat(contentsAfter2Main.getMetadataLocation()) + assertThat(contentsAfter2Main.getMetadataLocation()) .describedAs("metadata-location must change on %s", BRANCH) .isNotEqualTo(contentsAfter1Main.getMetadataLocation()); - Assertions.assertThat(contentsAfter2Branch.getMetadataLocation()) + assertThat(contentsAfter2Branch.getMetadataLocation()) .describedAs("on-reference-state must not change on %s", testCaseBranch) .isEqualTo(contentsAfter1Branch.getMetadataLocation()); // verify manifests - Assertions.assertThat(tableAfter2Main.currentSnapshot().allManifests(tableAfter2Main.io())) + assertThat(tableAfter2Main.currentSnapshot().allManifests(tableAfter2Main.io())) .describedAs("verify number of manifests on 'main'") .hasSize(2); } @@ -211,11 +213,11 @@ public void testCreate() throws IOException { getTable(KEY); // sanity, check table exists // check parameters are in expected state String expected = temp.toUri() + DB_NAME + "/" + tableName; - Assertions.assertThat(getTableBasePath(tableName)).isEqualTo(expected); + assertThat(getTableBasePath(tableName)).isEqualTo(expected); // Only 1 snapshotFile Should exist and no manifests should exist - Assertions.assertThat(metadataVersionFiles(tableLocation)).isNotNull().hasSize(2); - Assertions.assertThat(manifestFiles(tableLocation)).isNotNull().isEmpty(); + assertThat(metadataVersionFiles(tableLocation)).isNotNull().hasSize(2); + assertThat(manifestFiles(tableLocation)).isNotNull().isEmpty(); verifyCommitMetadata(); } @@ -229,17 +231,17 @@ public void testRename() throws NessieNotFoundException { Table original = catalog.loadTable(TABLE_IDENTIFIER); catalog.renameTable(TABLE_IDENTIFIER, renameTableIdentifier); - Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); - Assertions.assertThat(catalog.tableExists(renameTableIdentifier)).isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + assertThat(catalog.tableExists(renameTableIdentifier)).isTrue(); Table renamed = catalog.loadTable(renameTableIdentifier); - Assertions.assertThat(original.schema().asStruct()).isEqualTo(renamed.schema().asStruct()); - Assertions.assertThat(original.spec()).isEqualTo(renamed.spec()); - Assertions.assertThat(original.location()).isEqualTo(renamed.location()); - Assertions.assertThat(original.currentSnapshot()).isEqualTo(renamed.currentSnapshot()); + assertThat(original.schema().asStruct()).isEqualTo(renamed.schema().asStruct()); + assertThat(original.spec()).isEqualTo(renamed.spec()); + assertThat(original.location()).isEqualTo(renamed.location()); + assertThat(original.currentSnapshot()).isEqualTo(renamed.currentSnapshot()); - Assertions.assertThat(catalog.dropTable(renameTableIdentifier)).isTrue(); + assertThat(catalog.dropTable(renameTableIdentifier)).isTrue(); verifyCommitMetadata(); } @@ -268,17 +270,17 @@ public void testRenameWithTableReference() throws NessieNotFoundException { Table original = catalog.loadTable(fromIdentifier); catalog.renameTable(fromIdentifier, toIdentifier); - Assertions.assertThat(catalog.tableExists(fromIdentifier)).isFalse(); - Assertions.assertThat(catalog.tableExists(toIdentifier)).isTrue(); + assertThat(catalog.tableExists(fromIdentifier)).isFalse(); + assertThat(catalog.tableExists(toIdentifier)).isTrue(); Table renamed = catalog.loadTable(toIdentifier); - Assertions.assertThat(original.schema().asStruct()).isEqualTo(renamed.schema().asStruct()); - Assertions.assertThat(original.spec()).isEqualTo(renamed.spec()); - Assertions.assertThat(original.location()).isEqualTo(renamed.location()); - Assertions.assertThat(original.currentSnapshot()).isEqualTo(renamed.currentSnapshot()); + assertThat(original.schema().asStruct()).isEqualTo(renamed.schema().asStruct()); + assertThat(original.spec()).isEqualTo(renamed.spec()); + assertThat(original.location()).isEqualTo(renamed.location()); + assertThat(original.currentSnapshot()).isEqualTo(renamed.currentSnapshot()); - Assertions.assertThat(catalog.dropTable(toIdentifier)).isTrue(); + assertThat(catalog.dropTable(toIdentifier)).isTrue(); verifyCommitMetadata(); } @@ -304,7 +306,7 @@ public void testRenameWithTableReferenceInvalidCase() throws NessieNotFoundExcep TableIdentifier toIdentifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(), toTableReference.toString()); - Assertions.assertThatThrownBy(() -> catalog.renameTable(fromIdentifier, toIdentifier)) + assertThatThrownBy(() -> catalog.renameTable(fromIdentifier, toIdentifier)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot rename table 'tbl' on reference 'Something' to 'rename_table_name' on reference 'iceberg-table-test': source and target references must be the same."); @@ -324,7 +326,7 @@ public void testRenameWithTableReferenceInvalidCase() throws NessieNotFoundExcep TableIdentifier toIdentifierNew = TableIdentifier.of(TABLE_IDENTIFIER.namespace(), toTableReference.toString()); - Assertions.assertThatThrownBy(() -> catalog.renameTable(fromIdentifierNew, toIdentifierNew)) + assertThatThrownBy(() -> catalog.renameTable(fromIdentifierNew, toIdentifierNew)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot rename table 'tbl' on reference 'iceberg-table-test' to 'rename_table_name' on reference 'Something': source and target references must be the same."); @@ -333,26 +335,26 @@ public void testRenameWithTableReferenceInvalidCase() throws NessieNotFoundExcep private void verifyCommitMetadata() throws NessieNotFoundException { // check that the author is properly set List log = api.getCommitLog().refName(BRANCH).get().getLogEntries(); - Assertions.assertThat(log) + assertThat(log) .isNotNull() .isNotEmpty() .filteredOn(e -> !e.getCommitMeta().getMessage().startsWith("create namespace ")) .allSatisfy( logEntry -> { CommitMeta commit = logEntry.getCommitMeta(); - Assertions.assertThat(commit.getAuthor()).isNotNull().isNotEmpty(); - Assertions.assertThat(commit.getAuthor()).isEqualTo(System.getProperty("user.name")); - Assertions.assertThat(commit.getProperties().get(NessieUtil.APPLICATION_TYPE)) + assertThat(commit.getAuthor()).isNotNull().isNotEmpty(); + assertThat(commit.getAuthor()).isEqualTo(System.getProperty("user.name")); + assertThat(commit.getProperties().get(NessieUtil.APPLICATION_TYPE)) .isEqualTo("iceberg"); - Assertions.assertThat(commit.getMessage()).startsWith("Iceberg"); + assertThat(commit.getMessage()).startsWith("Iceberg"); }); } @Test public void testDrop() throws NessieNotFoundException { - Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); - Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); + assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); verifyCommitMetadata(); } @@ -365,9 +367,9 @@ public void testDropWithTableReference() throws NessieNotFoundException { .build(); TableIdentifier identifier = TableIdentifier.of(TABLE_IDENTIFIER.namespace(), tableReference.toString()); - Assertions.assertThat(catalog.tableExists(identifier)).isTrue(); - Assertions.assertThat(catalog.dropTable(identifier)).isTrue(); - Assertions.assertThat(catalog.tableExists(identifier)).isFalse(); + assertThat(catalog.tableExists(identifier)).isTrue(); + assertThat(catalog.dropTable(identifier)).isTrue(); + assertThat(catalog.tableExists(identifier)).isFalse(); verifyCommitMetadata(); } @@ -384,11 +386,11 @@ public void testDropWithoutPurgeLeavesTableData() throws IOException { String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); - Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); - Assertions.assertThat(new File(fileLocation)).exists(); - Assertions.assertThat(new File(manifestListLocation)).exists(); + assertThat(new File(fileLocation)).exists(); + assertThat(new File(manifestListLocation)).exists(); } @Test @@ -412,37 +414,36 @@ public void testDropTable() throws IOException { List manifests = table.currentSnapshot().allManifests(table.io()); - Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); - Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); + assertThat(catalog.dropTable(TABLE_IDENTIFIER)).isTrue(); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isFalse(); - Assertions.assertThat(new File(location1)).exists(); - Assertions.assertThat(new File(location2)).exists(); - Assertions.assertThat(new File(manifestListLocation)).exists(); + assertThat(new File(location1)).exists(); + assertThat(new File(location2)).exists(); + assertThat(new File(manifestListLocation)).exists(); for (ManifestFile manifest : manifests) { - Assertions.assertThat(new File(manifest.path().replace("file:", ""))).exists(); + assertThat(new File(manifest.path().replace("file:", ""))).exists(); } TableOperations ops = ((HasTableOperations) table).operations(); String metadataLocation = ((NessieTableOperations) ops).currentMetadataLocation(); - Assertions.assertThat(new File(metadataLocation.replace("file:", ""))).exists(); + assertThat(new File(metadataLocation.replace("file:", ""))).exists(); verifyCommitMetadata(); } private void validateRegister(TableIdentifier identifier, String metadataVersionFiles) { - Assertions.assertThat(catalog.registerTable(identifier, "file:" + metadataVersionFiles)) - .isNotNull(); + assertThat(catalog.registerTable(identifier, "file:" + metadataVersionFiles)).isNotNull(); Table newTable = catalog.loadTable(identifier); - Assertions.assertThat(newTable).isNotNull(); + assertThat(newTable).isNotNull(); TableOperations ops = ((HasTableOperations) newTable).operations(); String metadataLocation = ((NessieTableOperations) ops).currentMetadataLocation(); - Assertions.assertThat("file:" + metadataVersionFiles).isEqualTo(metadataLocation); - Assertions.assertThat(catalog.dropTable(identifier, false)).isTrue(); + assertThat("file:" + metadataVersionFiles).isEqualTo(metadataLocation); + assertThat(catalog.dropTable(identifier, false)).isTrue(); } @Test public void testRegisterTableWithGivenBranch() throws Exception { List metadataVersionFiles = metadataVersionFiles(tableLocation); - Assertions.assertThat(1).isEqualTo(metadataVersionFiles.size()); + assertThat(1).isEqualTo(metadataVersionFiles.size()); ImmutableTableReference tableReference = ImmutableTableReference.builder().reference("main").name(TABLE_NAME).build(); TableIdentifier identifier = TableIdentifier.of(DB_NAME, tableReference.toString()); @@ -458,18 +459,18 @@ public void testRegisterTableWithGivenBranch() throws Exception { public void testRegisterTableFailureScenarios() throws NessieConflictException, NessieNotFoundException { List metadataVersionFiles = metadataVersionFiles(tableLocation); - Assertions.assertThat(1).isEqualTo(metadataVersionFiles.size()); + assertThat(1).isEqualTo(metadataVersionFiles.size()); // Case 1: Branch does not exist ImmutableTableReference defaultTableReference = ImmutableTableReference.builder().reference("default").name(TABLE_NAME).build(); TableIdentifier defaultIdentifier = TableIdentifier.of(DB_NAME, defaultTableReference.toString()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable(defaultIdentifier, "file:" + metadataVersionFiles.get(0))) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Nessie ref 'default' does not exist"); // Case 2: Table Already Exists - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable(TABLE_IDENTIFIER, "file:" + metadataVersionFiles.get(0))) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: db.tbl"); @@ -477,25 +478,25 @@ public void testRegisterTableFailureScenarios() ImmutableTableReference branchTableReference = ImmutableTableReference.builder().reference(BRANCH).name(TABLE_NAME).build(); TableIdentifier branchIdentifier = TableIdentifier.of(DB_NAME, branchTableReference.toString()); - Assertions.assertThat(catalog.dropTable(branchIdentifier, false)).isTrue(); + assertThat(catalog.dropTable(branchIdentifier, false)).isTrue(); String hash = api.getReference().refName(BRANCH).get().getHash(); api.createReference().sourceRefName(BRANCH).reference(Tag.of("tag_1", hash)).create(); ImmutableTableReference tagTableReference = ImmutableTableReference.builder().reference("tag_1").name(TABLE_NAME).build(); TableIdentifier tagIdentifier = TableIdentifier.of(DB_NAME, tagTableReference.toString()); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable(tagIdentifier, "file:" + metadataVersionFiles.get(0))) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "You can only mutate tables/views when using a branch without a hash or timestamp."); // Case 4: non-null metadata path with null metadata location - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable( TABLE_IDENTIFIER, "file:" + metadataVersionFiles.get(0) + "invalidName")) .isInstanceOf(NotFoundException.class); // Case 5: null identifier - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> catalog.registerTable(null, "file:" + metadataVersionFiles.get(0) + "invalidName")) .isInstanceOf(IllegalArgumentException.class) @@ -505,15 +506,15 @@ public void testRegisterTableFailureScenarios() @Test public void testRegisterTableWithDefaultBranch() { List metadataVersionFiles = metadataVersionFiles(tableLocation); - Assertions.assertThat(1).isEqualTo(metadataVersionFiles.size()); - Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); + assertThat(1).isEqualTo(metadataVersionFiles.size()); + assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); validateRegister(TABLE_IDENTIFIER, metadataVersionFiles.get(0)); } @Test public void testRegisterTableMoreThanOneBranch() throws Exception { List metadataVersionFiles = metadataVersionFiles(tableLocation); - Assertions.assertThat(1).isEqualTo(metadataVersionFiles.size()); + assertThat(1).isEqualTo(metadataVersionFiles.size()); ImmutableTableReference tableReference = ImmutableTableReference.builder().reference("main").name(TABLE_NAME).build(); TableIdentifier identifier = TableIdentifier.of(DB_NAME, tableReference.toString()); @@ -523,7 +524,7 @@ public void testRegisterTableMoreThanOneBranch() throws Exception { // ignore } validateRegister(identifier, metadataVersionFiles.get(0)); - Assertions.assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); + assertThat(catalog.dropTable(TABLE_IDENTIFIER, false)).isTrue(); validateRegister(TABLE_IDENTIFIER, metadataVersionFiles.get(0)); } @@ -536,9 +537,9 @@ public void testExistingTableUpdate() { icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Only 2 snapshotFile Should exist and no manifests should exist - Assertions.assertThat(metadataVersionFiles(tableLocation)).isNotNull().hasSize(2); - Assertions.assertThat(manifestFiles(tableLocation)).isNotNull().isEmpty(); - Assertions.assertThat(altered.asStruct()).isEqualTo(icebergTable.schema().asStruct()); + assertThat(metadataVersionFiles(tableLocation)).isNotNull().hasSize(2); + assertThat(manifestFiles(tableLocation)).isNotNull().isEmpty(); + assertThat(altered.asStruct()).isEqualTo(icebergTable.schema().asStruct()); } @Test @@ -555,7 +556,7 @@ public void testFailure() throws NessieNotFoundException, NessieConflictExceptio .commitMeta(CommitMeta.fromMessage("")) .commit(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> icebergTable.updateSchema().addColumn("data", Types.LongType.get()).commit()) .isInstanceOf(CommitFailedException.class) .hasMessage( @@ -570,18 +571,17 @@ public void testListTables() { .filter(t -> t.namespace().level(0).equals(DB_NAME) && t.name().equals(TABLE_NAME)) .collect(Collectors.toList()); - Assertions.assertThat(expectedIdents).hasSize(1); - Assertions.assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); + assertThat(expectedIdents).hasSize(1); + assertThat(catalog.tableExists(TABLE_IDENTIFIER)).isTrue(); } @Test public void testGCDisabled() { Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); - Assertions.assertThat(icebergTable.properties()) - .containsEntry(TableProperties.GC_ENABLED, "false"); + assertThat(icebergTable.properties()).containsEntry(TableProperties.GC_ENABLED, "false"); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> icebergTable.expireSnapshots().expireOlderThan(System.currentTimeMillis()).commit()) .isInstanceOf(ValidationException.class) @@ -593,10 +593,9 @@ public void testGCDisabled() { public void testGCEnabled() { Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); icebergTable.updateProperties().set(TableProperties.GC_ENABLED, "true").commit(); - Assertions.assertThat(icebergTable.properties()) - .containsEntry(TableProperties.GC_ENABLED, "true"); + assertThat(icebergTable.properties()).containsEntry(TableProperties.GC_ENABLED, "true"); - Assertions.assertThatCode( + assertThatCode( () -> icebergTable.expireSnapshots().expireOlderThan(System.currentTimeMillis()).commit()) .doesNotThrowAnyException(); @@ -618,7 +617,7 @@ public void testGCEnabledViaTableDefaultCatalogProperty() { tableLocation = createTable(TABLE_IDENTIFIER, schema).location().replaceFirst("file:", ""); Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); - Assertions.assertThatCode( + assertThatCode( () -> icebergTable.expireSnapshots().expireOlderThan(System.currentTimeMillis()).commit()) .doesNotThrowAnyException(); @@ -638,19 +637,19 @@ public void testTableMetadataFilesCleanupDisable() throws NessieNotFoundExceptio ((BaseTable) icebergTable).operations().current().metadataFileLocation(); Path metadataFileLocationPath = Paths.get(metadataFileLocation.replaceFirst("file:", "")); - Assertions.assertThat(metadataFileLocationPath).exists(); + assertThat(metadataFileLocationPath).exists(); icebergTable.updateSchema().addColumn("x1", Types.LongType.get()).commit(); icebergTable.updateSchema().addColumn("x2", Types.LongType.get()).commit(); // old table metadata file should still exist after commits. - Assertions.assertThat(metadataFileLocationPath).exists(); + assertThat(metadataFileLocationPath).exists(); // load the table from the specific hash which reads the mapping metadataFileLocation ImmutableTableReference tableReference = ImmutableTableReference.builder().reference(BRANCH).hash(hash).name(TABLE_NAME).build(); TableIdentifier identifier = TableIdentifier.of(DB_NAME, tableReference.toString()); - Assertions.assertThat( + assertThat( ((BaseTable) catalog.loadTable(identifier)) .operations() .current() @@ -663,7 +662,7 @@ public void testTableMetadataFilesCleanupDisable() throws NessieNotFoundExceptio .operations().current().previousFiles().stream() .map(TableMetadata.MetadataLogEntry::file) .collect(Collectors.toSet()); - Assertions.assertThat(tableMetadataFiles).hasSize(1).doesNotContain(metadataFileLocation); + assertThat(tableMetadataFiles).hasSize(1).doesNotContain(metadataFileLocation); } private String getTableBasePath(String tableName) { diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieUtil.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieUtil.java index 624ce99adf4b..1e1326d66f92 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieUtil.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieUtil.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.nessie; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.projectnessie.model.CommitMeta; @@ -28,7 +30,7 @@ public class TestNessieUtil { @Test public void testBuildingCommitMetadataWithNullCatalogOptions() { - Assertions.assertThatThrownBy(() -> NessieUtil.buildCommitMetadata("msg", null)) + assertThatThrownBy(() -> NessieUtil.buildCommitMetadata("msg", null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("catalogOptions must not be null"); } @@ -42,24 +44,21 @@ public void testSparkAppIdAndUserIsSetOnCommitMetadata() { NessieUtil.buildCommitMetadata( commitMsg, ImmutableMap.of(CatalogProperties.APP_ID, appId, CatalogProperties.USER, user)); - Assertions.assertThat(commitMeta.getMessage()).isEqualTo(commitMsg); - Assertions.assertThat(commitMeta.getAuthor()).isEqualTo(user); - Assertions.assertThat(commitMeta.getProperties()).hasSize(2); - Assertions.assertThat(commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)) - .isEqualTo("iceberg"); - Assertions.assertThat(commitMeta.getProperties().get(CatalogProperties.APP_ID)) - .isEqualTo(appId); + assertThat(commitMeta.getMessage()).isEqualTo(commitMsg); + assertThat(commitMeta.getAuthor()).isEqualTo(user); + assertThat(commitMeta.getProperties()).hasSize(2); + assertThat(commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)).isEqualTo("iceberg"); + assertThat(commitMeta.getProperties().get(CatalogProperties.APP_ID)).isEqualTo(appId); } @Test public void testAuthorIsSetOnCommitMetadata() { String commitMsg = "commit msg"; CommitMeta commitMeta = NessieUtil.buildCommitMetadata(commitMsg, ImmutableMap.of()); - Assertions.assertThat(commitMeta.getMessage()).isEqualTo(commitMsg); - Assertions.assertThat(commitMeta.getAuthor()).isEqualTo(System.getProperty("user.name")); - Assertions.assertThat(commitMeta.getProperties()).hasSize(1); - Assertions.assertThat(commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)) - .isEqualTo("iceberg"); + assertThat(commitMeta.getMessage()).isEqualTo(commitMsg); + assertThat(commitMeta.getAuthor()).isEqualTo(System.getProperty("user.name")); + assertThat(commitMeta.getProperties()).hasSize(1); + assertThat(commitMeta.getProperties().get(NessieUtil.APPLICATION_TYPE)).isEqualTo("iceberg"); } @Test @@ -68,7 +67,7 @@ public void testAuthorIsNullWithoutJvmUser() { try { System.clearProperty("user.name"); CommitMeta commitMeta = NessieUtil.buildCommitMetadata("commit msg", ImmutableMap.of()); - Assertions.assertThat(commitMeta.getAuthor()).isNull(); + assertThat(commitMeta.getAuthor()).isNull(); } finally { System.setProperty("user.name", jvmUserName); } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieView.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieView.java index 656363ff072b..9c97a5c302ef 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieView.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieView.java @@ -20,6 +20,8 @@ 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; @@ -35,7 +37,6 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.view.SQLViewRepresentation; import org.apache.iceberg.view.View; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -127,10 +128,10 @@ public void verifyStateMovesForDML() throws Exception { View viewInitialMain = catalog.loadView(VIEW_IDENTIFIER); // verify view-metadata-location + version-id - Assertions.assertThat(contentInitialMain) + assertThat(contentInitialMain) .as("global-contents + snapshot-id equal on both branches in Nessie") .isEqualTo(contentInitialBranch); - Assertions.assertThat(viewInitialMain.currentVersion().versionId()).isEqualTo(2); + assertThat(viewInitialMain.currentVersion().versionId()).isEqualTo(2); // 3. modify view in "main" branch icebergView @@ -146,19 +147,19 @@ public void verifyStateMovesForDML() throws Exception { // --> assert getValue() against both branches returns the updated metadata-location // verify view-metadata-location - Assertions.assertThat(contentInitialMain.getMetadataLocation()) + assertThat(contentInitialMain.getMetadataLocation()) .describedAs("metadata-location must change on %s", BRANCH) .isNotEqualTo(contentsAfter1Main.getMetadataLocation()); - Assertions.assertThat(contentInitialBranch.getMetadataLocation()) + assertThat(contentInitialBranch.getMetadataLocation()) .describedAs("metadata-location must not change on %s", testCaseBranch) .isEqualTo(contentsAfter1Branch.getMetadataLocation()); - Assertions.assertThat(contentsAfter1Main) + assertThat(contentsAfter1Main) .extracting(IcebergView::getSchemaId) .describedAs("schema ID must be same across branches") .isEqualTo(contentsAfter1Branch.getSchemaId()); // verify updates - Assertions.assertThat(viewAfter1Main.currentVersion().versionId()).isEqualTo(3); - Assertions.assertThat( + assertThat(viewAfter1Main.currentVersion().versionId()).isEqualTo(3); + assertThat( ((SQLViewRepresentation) viewAfter1Main.currentVersion().representations().get(0)) .dialect()) .isEqualTo("trino"); @@ -178,16 +179,16 @@ public void verifyStateMovesForDML() throws Exception { // --> assert getValue() against both branches returns the updated metadata-location // verify view-metadata-location - Assertions.assertThat(contentsAfter2Main.getVersionId()).isEqualTo(4); - Assertions.assertThat(contentsAfter2Main.getMetadataLocation()) + assertThat(contentsAfter2Main.getVersionId()).isEqualTo(4); + assertThat(contentsAfter2Main.getMetadataLocation()) .describedAs("metadata-location must change on %s", BRANCH) .isNotEqualTo(contentsAfter1Main.getMetadataLocation()); - Assertions.assertThat(contentsAfter1Main.getVersionId()).isEqualTo(3); - Assertions.assertThat(contentsAfter2Branch.getMetadataLocation()) + assertThat(contentsAfter1Main.getVersionId()).isEqualTo(3); + assertThat(contentsAfter2Branch.getMetadataLocation()) .describedAs("on-reference-state must not change on %s", testCaseBranch) .isEqualTo(contentsAfter1Branch.getMetadataLocation()); - Assertions.assertThat(viewAfter2Main.currentVersion().versionId()).isEqualTo(4); - Assertions.assertThat( + assertThat(viewAfter2Main.currentVersion().versionId()).isEqualTo(4); + assertThat( ((SQLViewRepresentation) viewAfter2Main.currentVersion().representations().get(0)) .dialect()) .isEqualTo("flink"); @@ -208,9 +209,9 @@ public void testUpdate() throws IOException { getView(KEY); // sanity, check view exists // check parameters are in expected state String expected = temp.toUri() + DB_NAME + "/" + viewName; - Assertions.assertThat(getViewBasePath(viewName)).isEqualTo(expected); + assertThat(getViewBasePath(viewName)).isEqualTo(expected); - Assertions.assertThat(metadataVersionFiles(viewLocation)).isNotNull().hasSize(2); + assertThat(metadataVersionFiles(viewLocation)).isNotNull().hasSize(2); verifyCommitMetadata(); } @@ -237,10 +238,10 @@ public void testRenameWithTableReference() throws NessieNotFoundException { TableIdentifier.of(VIEW_IDENTIFIER.namespace(), toTableReference.toString()); catalog.renameView(fromIdentifier, toIdentifier); - Assertions.assertThat(catalog.viewExists(fromIdentifier)).isFalse(); - Assertions.assertThat(catalog.viewExists(toIdentifier)).isTrue(); + assertThat(catalog.viewExists(fromIdentifier)).isFalse(); + assertThat(catalog.viewExists(toIdentifier)).isTrue(); - Assertions.assertThat(catalog.dropView(toIdentifier)).isTrue(); + assertThat(catalog.dropView(toIdentifier)).isTrue(); verifyCommitMetadata(); } @@ -266,7 +267,7 @@ public void testRenameWithTableReferenceInvalidCase() { TableIdentifier toIdentifier = TableIdentifier.of(VIEW_IDENTIFIER.namespace(), toTableReference.toString()); - Assertions.assertThatThrownBy(() -> catalog.renameView(fromIdentifier, toIdentifier)) + assertThatThrownBy(() -> catalog.renameView(fromIdentifier, toIdentifier)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot rename view 'view' on reference 'Something' to 'rename_view_name' on reference 'iceberg-view-test': source and target references must be the same."); @@ -286,7 +287,7 @@ public void testRenameWithTableReferenceInvalidCase() { TableIdentifier toIdentifierNew = TableIdentifier.of(VIEW_IDENTIFIER.namespace(), toTableReference.toString()); - Assertions.assertThatThrownBy(() -> catalog.renameView(fromIdentifierNew, toIdentifierNew)) + assertThatThrownBy(() -> catalog.renameView(fromIdentifierNew, toIdentifierNew)) .isInstanceOf(IllegalArgumentException.class) .hasMessage( "Cannot rename view 'view' on reference 'iceberg-view-test' to 'rename_view_name' on reference 'Something': source and target references must be the same."); @@ -295,29 +296,29 @@ public void testRenameWithTableReferenceInvalidCase() { private void verifyCommitMetadata() throws NessieNotFoundException { // check that the author is properly set List log = api.getCommitLog().refName(BRANCH).get().getLogEntries(); - Assertions.assertThat(log) + assertThat(log) .isNotNull() .isNotEmpty() .filteredOn(e -> !e.getCommitMeta().getMessage().startsWith("create namespace ")) .allSatisfy( logEntry -> { CommitMeta commit = logEntry.getCommitMeta(); - Assertions.assertThat(commit.getAuthor()) + assertThat(commit.getAuthor()) .isNotNull() .isNotEmpty() .isEqualTo(System.getProperty("user.name")); - Assertions.assertThat(commit.getProperties()) + assertThat(commit.getProperties()) .containsEntry(NessieUtil.APPLICATION_TYPE, "iceberg"); - Assertions.assertThat(commit.getMessage()).startsWith("Iceberg"); + assertThat(commit.getMessage()).startsWith("Iceberg"); }); } @Test public void testDrop() throws NessieNotFoundException { - Assertions.assertThat(catalog.viewExists(VIEW_IDENTIFIER)).isTrue(); - Assertions.assertThat(catalog.dropView(VIEW_IDENTIFIER)).isTrue(); - Assertions.assertThat(catalog.viewExists(VIEW_IDENTIFIER)).isFalse(); - Assertions.assertThat(catalog.dropView(VIEW_IDENTIFIER)).isFalse(); + assertThat(catalog.viewExists(VIEW_IDENTIFIER)).isTrue(); + assertThat(catalog.dropView(VIEW_IDENTIFIER)).isTrue(); + assertThat(catalog.viewExists(VIEW_IDENTIFIER)).isFalse(); + assertThat(catalog.dropView(VIEW_IDENTIFIER)).isFalse(); verifyCommitMetadata(); } @@ -327,9 +328,9 @@ public void testListViews() { createView(catalog, newIdentifier, SCHEMA); List viewIdents = catalog.listViews(VIEW_IDENTIFIER.namespace()); - Assertions.assertThat(viewIdents).contains(VIEW_IDENTIFIER, newIdentifier); - Assertions.assertThat(catalog.viewExists(VIEW_IDENTIFIER)).isTrue(); - Assertions.assertThat(catalog.viewExists(newIdentifier)).isTrue(); + assertThat(viewIdents).contains(VIEW_IDENTIFIER, newIdentifier); + assertThat(catalog.viewExists(VIEW_IDENTIFIER)).isTrue(); + assertThat(catalog.viewExists(newIdentifier)).isTrue(); } private String getViewBasePath(String viewName) { diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java index ca932f205bff..cfc29623dad6 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java @@ -19,6 +19,9 @@ package org.apache.iceberg.orc; 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.Assertions.offset; import java.io.File; import java.lang.reflect.Field; @@ -41,7 +44,6 @@ import org.apache.orc.impl.OrcIndex; import org.apache.orc.impl.RecordReaderImpl; import org.apache.orc.impl.WriterImpl; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -56,7 +58,7 @@ public class TestBloomFilter { @Test public void testWriteOption() throws Exception { - Assertions.assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); OutputFile outFile = Files.localOutput(testFile); try (FileAppender writer = @@ -81,9 +83,9 @@ public void testWriteOption() throws Exception { double bloomFilterFpp = (double) bloomFilterFppField.get(orcWriter); // Validate whether the bloom filters are set in ORC SDK or not - Assertions.assertThat(bloomFilterColumns[1]).isTrue(); - Assertions.assertThat(bloomFilterColumns[2]).isTrue(); - Assertions.assertThat(bloomFilterFpp).isCloseTo(0.04, Assertions.offset(1e-15)); + assertThat(bloomFilterColumns[1]).isTrue(); + assertThat(bloomFilterColumns[2]).isTrue(); + assertThat(bloomFilterFpp).isCloseTo(0.04, offset(1e-15)); Record recordTemplate = GenericRecord.create(DATA_SCHEMA); Record record1 = recordTemplate.copy("id", 1L, "name", "foo", "price", 1.0); @@ -123,15 +125,15 @@ public void testWriteOption() throws Exception { footer.getColumns(1)); // Validate whether the bloom filters are written ORC files or not - Assertions.assertThat(bloomFilterString).contains("Bloom filters for column"); + assertThat(bloomFilterString).contains("Bloom filters for column"); } } @Test public void testInvalidFppOption() throws Exception { - Assertions.assertThat(testFile.delete()).as("Delete should succeed").isTrue(); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); - Assertions.assertThatThrownBy( + assertThatThrownBy( () -> ORC.write(Files.localOutput(testFile)) .createWriterFunc(GenericOrcWriter::buildWriter) diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java b/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java index b8f17f3364bc..a179cb2f6454 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestBuildOrcProjection.java @@ -20,11 +20,12 @@ 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 org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; /** Test projections on ORC types. */ @@ -38,13 +39,11 @@ public void testProjectionPrimitiveNoOp() { // Original mapping (stored in ORC) TypeDescription orcSchema = ORCSchemaUtil.convert(originalSchema); - Assertions.assertThat(orcSchema.getChildren()).hasSize(2); - Assertions.assertThat(orcSchema.findSubtype("a").getId()).isEqualTo(1); - Assertions.assertThat(orcSchema.findSubtype("a").getCategory()) - .isEqualTo(TypeDescription.Category.INT); - Assertions.assertThat(orcSchema.findSubtype("b").getId()).isEqualTo(2); - Assertions.assertThat(orcSchema.findSubtype("b").getCategory()) - .isEqualTo(TypeDescription.Category.STRING); + assertThat(orcSchema.getChildren()).hasSize(2); + assertThat(orcSchema.findSubtype("a").getId()).isEqualTo(1); + assertThat(orcSchema.findSubtype("a").getCategory()).isEqualTo(TypeDescription.Category.INT); + assertThat(orcSchema.findSubtype("b").getId()).isEqualTo(2); + assertThat(orcSchema.findSubtype("b").getCategory()).isEqualTo(TypeDescription.Category.STRING); } @Test @@ -64,12 +63,12 @@ public void testProjectionPrimitive() { ); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema); - Assertions.assertThat(newOrcSchema.getChildren()).hasSize(2); - Assertions.assertThat(newOrcSchema.findSubtype("b").getId()).isEqualTo(1); - Assertions.assertThat(newOrcSchema.findSubtype("b").getCategory()) + assertThat(newOrcSchema.getChildren()).hasSize(2); + assertThat(newOrcSchema.findSubtype("b").getId()).isEqualTo(1); + assertThat(newOrcSchema.findSubtype("b").getCategory()) .isEqualTo(TypeDescription.Category.STRING); - Assertions.assertThat(newOrcSchema.findSubtype("c_r3").getId()).isEqualTo(2); - Assertions.assertThat(newOrcSchema.findSubtype("c_r3").getCategory()) + assertThat(newOrcSchema.findSubtype("c_r3").getId()).isEqualTo(2); + assertThat(newOrcSchema.findSubtype("c_r3").getCategory()) .isEqualTo(TypeDescription.Category.DATE); } @@ -84,16 +83,14 @@ public void testProjectionNestedNoOp() { TypeDescription orcSchema = ORCSchemaUtil.convert(originalSchema); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(originalSchema, orcSchema); - Assertions.assertThat(newOrcSchema.getChildren()).hasSize(1); - Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + assertThat(newOrcSchema.getChildren()).hasSize(1); + assertThat(newOrcSchema.findSubtype("a").getCategory()) .isEqualTo(TypeDescription.Category.STRUCT); TypeDescription nestedCol = newOrcSchema.findSubtype("a"); - Assertions.assertThat(nestedCol.findSubtype("b").getId()).isEqualTo(2); - Assertions.assertThat(nestedCol.findSubtype("b").getCategory()) - .isEqualTo(TypeDescription.Category.STRING); - Assertions.assertThat(nestedCol.findSubtype("c").getId()).isEqualTo(3); - Assertions.assertThat(nestedCol.findSubtype("c").getCategory()) - .isEqualTo(TypeDescription.Category.DATE); + assertThat(nestedCol.findSubtype("b").getId()).isEqualTo(2); + assertThat(nestedCol.findSubtype("b").getCategory()).isEqualTo(TypeDescription.Category.STRING); + assertThat(nestedCol.findSubtype("c").getId()).isEqualTo(3); + assertThat(nestedCol.findSubtype("c").getCategory()).isEqualTo(TypeDescription.Category.DATE); } @Test @@ -113,16 +110,14 @@ public void testProjectionNested() { Schema evolveSchema = new Schema(optional(1, "aa", newNestedStructType)); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema); - Assertions.assertThat(newOrcSchema.getChildren()).hasSize(1); - Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + assertThat(newOrcSchema.getChildren()).hasSize(1); + assertThat(newOrcSchema.findSubtype("a").getCategory()) .isEqualTo(TypeDescription.Category.STRUCT); TypeDescription nestedCol = newOrcSchema.findSubtype("a"); - Assertions.assertThat(nestedCol.findSubtype("c").getId()).isEqualTo(2); - Assertions.assertThat(nestedCol.findSubtype("c").getCategory()) - .isEqualTo(TypeDescription.Category.DATE); - Assertions.assertThat(nestedCol.findSubtype("b").getId()).isEqualTo(3); - Assertions.assertThat(nestedCol.findSubtype("b").getCategory()) - .isEqualTo(TypeDescription.Category.STRING); + assertThat(nestedCol.findSubtype("c").getId()).isEqualTo(2); + assertThat(nestedCol.findSubtype("c").getCategory()).isEqualTo(TypeDescription.Category.DATE); + assertThat(nestedCol.findSubtype("b").getId()).isEqualTo(3); + assertThat(nestedCol.findSubtype("b").getCategory()).isEqualTo(TypeDescription.Category.STRING); } @Test @@ -136,15 +131,14 @@ public void testEvolutionAddContainerField() { optional(2, "b", Types.StructType.of(required(3, "c", Types.LongType.get())))); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolvedSchema, baseOrcSchema); - Assertions.assertThat(newOrcSchema.getChildren()).hasSize(2); - Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) - .isEqualTo(TypeDescription.Category.INT); - Assertions.assertThat(newOrcSchema.findSubtype("b_r2").getId()).isEqualTo(2); - Assertions.assertThat(newOrcSchema.findSubtype("b_r2").getCategory()) + assertThat(newOrcSchema.getChildren()).hasSize(2); + assertThat(newOrcSchema.findSubtype("a").getCategory()).isEqualTo(TypeDescription.Category.INT); + assertThat(newOrcSchema.findSubtype("b_r2").getId()).isEqualTo(2); + assertThat(newOrcSchema.findSubtype("b_r2").getCategory()) .isEqualTo(TypeDescription.Category.STRUCT); TypeDescription nestedCol = newOrcSchema.findSubtype("b_r2"); - Assertions.assertThat(nestedCol.findSubtype("c_r3").getId()).isEqualTo(3); - Assertions.assertThat(nestedCol.findSubtype("c_r3").getCategory()) + assertThat(nestedCol.findSubtype("c_r3").getId()).isEqualTo(3); + assertThat(nestedCol.findSubtype("c_r3").getCategory()) .isEqualTo(TypeDescription.Category.LONG); } @@ -166,8 +160,7 @@ public void testRequiredNestedFieldMissingInFile() { required(3, "c", Types.LongType.get()), required(4, "d", Types.LongType.get())))); - Assertions.assertThatThrownBy( - () -> ORCSchemaUtil.buildOrcProjection(evolvedSchema, baseOrcSchema)) + assertThatThrownBy(() -> ORCSchemaUtil.buildOrcProjection(evolvedSchema, baseOrcSchema)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Field 4 of type long is required and was not found."); } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java b/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java index 1aa7dda04761..0be1254a093f 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestEstimateOrcAvgWidthVisitor.java @@ -20,11 +20,11 @@ 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 org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestEstimateOrcAvgWidthVisitor { @@ -78,9 +78,7 @@ public void testEstimateIntegerWidth() { Schema integerSchema = new Schema(ID_FIELD); TypeDescription integerOrcSchema = ORCSchemaUtil.convert(integerSchema); long estimateLength = getEstimateLength(integerOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of integer must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of integer must be 8.").isEqualTo(8); } @Test @@ -88,9 +86,7 @@ public void testEstimateStringWidth() { Schema stringSchema = new Schema(DATA_FIELD); TypeDescription stringOrcSchema = ORCSchemaUtil.convert(stringSchema); long estimateLength = getEstimateLength(stringOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of string must be 128.") - .isEqualTo(128); + assertThat(estimateLength).as("Estimated average length of string must be 128.").isEqualTo(128); } @Test @@ -98,9 +94,7 @@ public void testEstimateFloatWidth() { Schema floatSchema = new Schema(FLOAT_FIELD); TypeDescription floatOrcSchema = ORCSchemaUtil.convert(floatSchema); long estimateLength = getEstimateLength(floatOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of float must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of float must be 8.").isEqualTo(8); } @Test @@ -108,9 +102,7 @@ public void testEstimateDoubleWidth() { Schema doubleSchema = new Schema(DOUBLE_FIELD); TypeDescription doubleOrcSchema = ORCSchemaUtil.convert(doubleSchema); long estimateLength = getEstimateLength(doubleOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of double must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of double must be 8.").isEqualTo(8); } @Test @@ -118,9 +110,7 @@ public void testEstimateDecimalWidth() { Schema decimalSchema = new Schema(DECIMAL_FIELD); TypeDescription decimalOrcSchema = ORCSchemaUtil.convert(decimalSchema); long estimateLength = getEstimateLength(decimalOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of decimal must be 7.") - .isEqualTo(7); + assertThat(estimateLength).as("Estimated average length of decimal must be 7.").isEqualTo(7); } @Test @@ -128,9 +118,7 @@ public void testEstimateFixedWidth() { Schema fixedSchema = new Schema(FIXED_FIELD); TypeDescription fixedOrcSchema = ORCSchemaUtil.convert(fixedSchema); long estimateLength = getEstimateLength(fixedOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of fixed must be 128.") - .isEqualTo(128); + assertThat(estimateLength).as("Estimated average length of fixed must be 128.").isEqualTo(128); } @Test @@ -138,9 +126,7 @@ public void testEstimateBinaryWidth() { Schema binarySchema = new Schema(BINARY_FIELD); TypeDescription binaryOrcSchema = ORCSchemaUtil.convert(binarySchema); long estimateLength = getEstimateLength(binaryOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of binary must be 128.") - .isEqualTo(128); + assertThat(estimateLength).as("Estimated average length of binary must be 128.").isEqualTo(128); } @Test @@ -148,9 +134,7 @@ public void testEstimateListWidth() { Schema listSchema = new Schema(FLOAT_LIST_FIELD); TypeDescription listOrcSchema = ORCSchemaUtil.convert(listSchema); long estimateLength = getEstimateLength(listOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of list must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of list must be 8.").isEqualTo(8); } @Test @@ -158,9 +142,7 @@ public void testEstimateLongWidth() { Schema longSchema = new Schema(LONG_FIELD); TypeDescription longOrcSchema = ORCSchemaUtil.convert(longSchema); long estimateLength = getEstimateLength(longOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of long must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of long must be 8.").isEqualTo(8); } @Test @@ -168,9 +150,7 @@ public void testEstimateBooleanWidth() { Schema booleanSchema = new Schema(BOOLEAN_FIELD); TypeDescription booleanOrcSchema = ORCSchemaUtil.convert(booleanSchema); long estimateLength = getEstimateLength(booleanOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of boolean must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of boolean must be 8.").isEqualTo(8); } @Test @@ -178,14 +158,14 @@ public void testEstimateTimestampWidth() { Schema timestampZoneSchema = new Schema(TIMESTAMP_ZONE_FIELD); TypeDescription timestampZoneOrcSchema = ORCSchemaUtil.convert(timestampZoneSchema); long estimateLength = getEstimateLength(timestampZoneOrcSchema); - Assertions.assertThat(estimateLength) + assertThat(estimateLength) .as("Estimated average length of timestamps with zone must be 12.") .isEqualTo(12); Schema timestampSchema = new Schema(TIMESTAMP_FIELD); TypeDescription timestampOrcSchema = ORCSchemaUtil.convert(timestampSchema); estimateLength = getEstimateLength(timestampOrcSchema); - Assertions.assertThat(estimateLength) + assertThat(estimateLength) .as("Estimated average length of timestamp must be 12.") .isEqualTo(12); } @@ -195,9 +175,7 @@ public void testEstimateDateWidth() { Schema dateSchema = new Schema(DATE_FIELD); TypeDescription dateOrcSchema = ORCSchemaUtil.convert(dateSchema); long estimateLength = getEstimateLength(dateOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of date must be 8.") - .isEqualTo(8); + assertThat(estimateLength).as("Estimated average length of date must be 8.").isEqualTo(8); } @Test @@ -205,9 +183,7 @@ public void testEstimateUUIDWidth() { Schema uuidSchema = new Schema(UUID_FIELD); TypeDescription uuidOrcSchema = ORCSchemaUtil.convert(uuidSchema); long estimateLength = getEstimateLength(uuidOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of uuid must be 128.") - .isEqualTo(128); + assertThat(estimateLength).as("Estimated average length of uuid must be 128.").isEqualTo(128); } @Test @@ -215,9 +191,7 @@ public void testEstimateMapWidth() { Schema mapSchema = new Schema(MAP_FIELD_1); TypeDescription mapOrcSchema = ORCSchemaUtil.convert(mapSchema); long estimateLength = getEstimateLength(mapOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of map must be 136.") - .isEqualTo(136); + assertThat(estimateLength).as("Estimated average length of map must be 136.").isEqualTo(136); } @Test @@ -225,9 +199,7 @@ public void testEstimateStructWidth() { Schema structSchema = new Schema(STRUCT_FIELD); TypeDescription structOrcSchema = ORCSchemaUtil.convert(structSchema); long estimateLength = getEstimateLength(structOrcSchema); - Assertions.assertThat(estimateLength) - .as("Estimated average length of struct must be 28.") - .isEqualTo(28); + assertThat(estimateLength).as("Estimated average length of struct must be 28.").isEqualTo(28); } @Test @@ -248,7 +220,7 @@ public void testEstimateFullWidth() { STRUCT_FIELD); TypeDescription fullOrcSchema = ORCSchemaUtil.convert(fullSchema); long estimateLength = getEstimateLength(fullOrcSchema); - Assertions.assertThat(estimateLength) + assertThat(estimateLength) .as("Estimated average length of the row must be 611.") .isEqualTo(611); } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java b/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java index c7c7a8aeb07e..32b815f56d52 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestExpressionToSearchArgument.java @@ -34,6 +34,7 @@ import static org.apache.iceberg.expressions.Expressions.year; 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.math.BigDecimal; import java.nio.ByteBuffer; @@ -59,7 +60,6 @@ import org.apache.orc.storage.ql.io.sarg.SearchArgument.TruthValue; import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory; import org.apache.orc.storage.serde2.io.HiveDecimalWritable; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestExpressionToSearchArgument { @@ -135,7 +135,7 @@ public void testPrimitiveTypes() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -175,7 +175,7 @@ public void testTimezoneSensitiveTypes() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } } finally { TimeZone.setDefault(currentTz); @@ -213,7 +213,7 @@ public void testUnsupportedTypes() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -262,7 +262,7 @@ public void testNestedPrimitives() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -292,7 +292,7 @@ public void testSpecialCharacters() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -316,7 +316,7 @@ public void testEvolvedSchema() { SearchArgumentFactory.newBuilder().equals("`int`", Type.LONG, 1L).build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); // for columns not in the file, buildOrcProjection will append field names with _r // this will be passed down to ORC, but ORC will handle such cases and return a TruthValue @@ -327,7 +327,7 @@ public void testEvolvedSchema() { SearchArgumentFactory.newBuilder().equals("`float_added_r3`", Type.FLOAT, 1.0).build(); actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -353,7 +353,7 @@ public void testOriginalSchemaNameMapping() { .build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -380,7 +380,7 @@ public void testModifiedSimpleSchemaNameMapping() { SearchArgumentFactory.newBuilder().equals("`int`", Type.LONG, 1L).build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); // for columns not in the file, buildOrcProjection will append field names with _r // this will be passed down to ORC, but ORC will handle such cases and return a TruthValue @@ -391,7 +391,7 @@ public void testModifiedSimpleSchemaNameMapping() { SearchArgumentFactory.newBuilder().equals("`new_float_field_r3`", Type.FLOAT, 1.0).build(); actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -475,7 +475,7 @@ public void testModifiedComplexSchemaNameMapping() { .build(); SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, readSchema); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } @Test @@ -490,6 +490,6 @@ public void testExpressionContainsNonReferenceTerm() { SearchArgument actual = ExpressionToSearchArgument.convert(boundFilter, ORCSchemaUtil.convert(schema)); - Assertions.assertThat(actual.toString()).isEqualTo(expected.toString()); + assertThat(actual.toString()).isEqualTo(expected.toString()); } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java b/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java index a323499b42d2..34bc21542388 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestIdToOrcName.java @@ -19,11 +19,11 @@ package org.apache.iceberg.orc; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestIdToOrcName { @@ -70,31 +70,31 @@ public void testIdToQuotedColumnName() { required(26, "colWith`Quotes`", Types.LongType.get())); Map actual = ORCSchemaUtil.idToOrcName(schema); - Assertions.assertThat(actual.get(1)).isEqualTo("`long`"); - Assertions.assertThat(actual.get(2)).isEqualTo("`struct`"); - Assertions.assertThat(actual.get(3)).isEqualTo("`struct`.`long`"); - Assertions.assertThat(actual.get(4)).isEqualTo("`listOfLongs`"); - Assertions.assertThat(actual.get(5)).isEqualTo("`listOfLongs`.`_elem`"); - Assertions.assertThat(actual.get(6)).isEqualTo("`listOfStructs`"); - Assertions.assertThat(actual.get(7)).isEqualTo("`listOfStructs`.`_elem`"); - Assertions.assertThat(actual.get(8)).isEqualTo("`listOfStructs`.`_elem`.`long`"); - Assertions.assertThat(actual.get(9)).isEqualTo("`map`"); - Assertions.assertThat(actual.get(10)).isEqualTo("`map`.`_key`"); - Assertions.assertThat(actual.get(11)).isEqualTo("`map`.`_value`"); - Assertions.assertThat(actual.get(12)).isEqualTo("`mapOfStructs`"); - Assertions.assertThat(actual.get(13)).isEqualTo("`mapOfStructs`.`_key`"); - Assertions.assertThat(actual.get(14)).isEqualTo("`mapOfStructs`.`_value`"); - Assertions.assertThat(actual.get(15)).isEqualTo("`mapOfStructs`.`_key`.`long`"); - Assertions.assertThat(actual.get(16)).isEqualTo("`mapOfStructs`.`_value`.`long`"); - Assertions.assertThat(actual.get(17)).isEqualTo("`listOfMapsOfStruct`"); - Assertions.assertThat(actual.get(18)).isEqualTo("`listOfMapsOfStruct`.`_elem`"); - Assertions.assertThat(actual.get(19)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_key`"); - Assertions.assertThat(actual.get(20)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_value`"); - Assertions.assertThat(actual.get(21)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_key`.`long`"); - Assertions.assertThat(actual.get(22)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_value`.`long`"); - Assertions.assertThat(actual.get(23)).isEqualTo("`col.with.dots`"); - Assertions.assertThat(actual.get(24)).isEqualTo("`col.with.dots`.`inner.col.with.dots`"); - Assertions.assertThat(actual.get(25)).isEqualTo("`colW!th$peci@lCh@rs`"); - Assertions.assertThat(actual.get(26)).isEqualTo("`colWith``Quotes```"); + assertThat(actual.get(1)).isEqualTo("`long`"); + assertThat(actual.get(2)).isEqualTo("`struct`"); + assertThat(actual.get(3)).isEqualTo("`struct`.`long`"); + assertThat(actual.get(4)).isEqualTo("`listOfLongs`"); + assertThat(actual.get(5)).isEqualTo("`listOfLongs`.`_elem`"); + assertThat(actual.get(6)).isEqualTo("`listOfStructs`"); + assertThat(actual.get(7)).isEqualTo("`listOfStructs`.`_elem`"); + assertThat(actual.get(8)).isEqualTo("`listOfStructs`.`_elem`.`long`"); + assertThat(actual.get(9)).isEqualTo("`map`"); + assertThat(actual.get(10)).isEqualTo("`map`.`_key`"); + assertThat(actual.get(11)).isEqualTo("`map`.`_value`"); + assertThat(actual.get(12)).isEqualTo("`mapOfStructs`"); + assertThat(actual.get(13)).isEqualTo("`mapOfStructs`.`_key`"); + assertThat(actual.get(14)).isEqualTo("`mapOfStructs`.`_value`"); + assertThat(actual.get(15)).isEqualTo("`mapOfStructs`.`_key`.`long`"); + assertThat(actual.get(16)).isEqualTo("`mapOfStructs`.`_value`.`long`"); + assertThat(actual.get(17)).isEqualTo("`listOfMapsOfStruct`"); + assertThat(actual.get(18)).isEqualTo("`listOfMapsOfStruct`.`_elem`"); + assertThat(actual.get(19)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_key`"); + assertThat(actual.get(20)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_value`"); + assertThat(actual.get(21)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_key`.`long`"); + assertThat(actual.get(22)).isEqualTo("`listOfMapsOfStruct`.`_elem`.`_value`.`long`"); + assertThat(actual.get(23)).isEqualTo("`col.with.dots`"); + assertThat(actual.get(24)).isEqualTo("`col.with.dots`.`inner.col.with.dots`"); + assertThat(actual.get(25)).isEqualTo("`colW!th$peci@lCh@rs`"); + assertThat(actual.get(26)).isEqualTo("`colWith``Quotes```"); } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java b/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java index 9b9eb688aa2a..9338c27b9733 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestORCFileIOProxies.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.orc; +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.io.InputStream; @@ -27,7 +30,6 @@ import org.apache.iceberg.Files; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestORCFileIOProxies { @@ -39,14 +41,14 @@ public void testInputFileSystem() throws IOException { InputFile localFile = Files.localInput(inputFile); FileIOFSUtil.InputFileSystem ifs = new FileIOFSUtil.InputFileSystem(localFile); InputStream is = ifs.open(new Path(localFile.location())); - Assertions.assertThat(is).isNotNull(); + assertThat(is).isNotNull(); // Cannot use the filesystem for any other operation - Assertions.assertThatThrownBy(() -> ifs.getFileStatus(new Path(localFile.location()))) + assertThatThrownBy(() -> ifs.getFileStatus(new Path(localFile.location()))) .isInstanceOf(UnsupportedOperationException.class); // Cannot use the filesystem for any other path - Assertions.assertThatThrownBy(() -> ifs.open(new Path("/tmp/dummy"))) + assertThatThrownBy(() -> ifs.open(new Path("/tmp/dummy"))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Input /tmp/dummy does not equal expected"); } @@ -64,19 +66,19 @@ public void testOutputFileSystem() throws IOException { os.write('C'); } // No other operation is supported - Assertions.assertThatThrownBy(() -> ofs.open(new Path(outputFile.location()))) + assertThatThrownBy(() -> ofs.open(new Path(outputFile.location()))) .isInstanceOf(UnsupportedOperationException.class); // No other path is supported - Assertions.assertThatThrownBy(() -> ofs.create(new Path("/tmp/dummy"))) + assertThatThrownBy(() -> ofs.create(new Path("/tmp/dummy"))) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Input /tmp/dummy does not equal expected"); FileSystem ifs = new FileIOFSUtil.InputFileSystem(outputFile.toInputFile()); try (InputStream is = ifs.open(new Path(outputFile.location()))) { - Assertions.assertThat(is.read()).isEqualTo('O'); - Assertions.assertThat(is.read()).isEqualTo('R'); - Assertions.assertThat(is.read()).isEqualTo('C'); - Assertions.assertThat(is.read()).isEqualTo(-1); + assertThat(is.read()).isEqualTo('O'); + assertThat(is.read()).isEqualTo('R'); + assertThat(is.read()).isEqualTo('C'); + assertThat(is.read()).isEqualTo(-1); } } } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java b/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java index 24a376cd71d3..f0f02da9ef54 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestORCSchemaUtil.java @@ -22,6 +22,8 @@ import static org.apache.iceberg.orc.ORCSchemaUtil.ICEBERG_REQUIRED_ATTRIBUTE; 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.util.Collections; import java.util.List; @@ -33,7 +35,6 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.orc.TypeDescription; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestORCSchemaUtil { @@ -62,8 +63,7 @@ public class TestORCSchemaUtil { @Test public void testRoundtripConversionPrimitive() { TypeDescription orcSchema = ORCSchemaUtil.convert(new Schema(SUPPORTED_PRIMITIVES.fields())); - Assertions.assertThat(ORCSchemaUtil.convert(orcSchema).asStruct()) - .isEqualTo(SUPPORTED_PRIMITIVES); + assertThat(ORCSchemaUtil.convert(orcSchema).asStruct()).isEqualTo(SUPPORTED_PRIMITIVES); } @Test @@ -189,8 +189,7 @@ public void testRoundtripConversionNested() { Types.ListType.ofRequired( 1250, nestedStructTypeForStruct)))))))); TypeDescription orcSchema = ORCSchemaUtil.convert(expectedSchema); - Assertions.assertThat(ORCSchemaUtil.convert(orcSchema).asStruct()) - .isEqualTo(expectedSchema.asStruct()); + assertThat(ORCSchemaUtil.convert(orcSchema).asStruct()).isEqualTo(expectedSchema.asStruct()); } @Test @@ -212,18 +211,18 @@ public void testTypePromotions() { optional(3, "c", Types.DecimalType.of(15, 2))); TypeDescription newOrcSchema = ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema); - Assertions.assertThat(newOrcSchema.getChildren()).hasSize(3); - Assertions.assertThat(newOrcSchema.findSubtype("a").getId()).isEqualTo(1); - Assertions.assertThat(newOrcSchema.findSubtype("a").getCategory()) + assertThat(newOrcSchema.getChildren()).hasSize(3); + assertThat(newOrcSchema.findSubtype("a").getId()).isEqualTo(1); + assertThat(newOrcSchema.findSubtype("a").getCategory()) .isEqualTo(TypeDescription.Category.LONG); - Assertions.assertThat(newOrcSchema.findSubtype("b").getId()).isEqualTo(2); - Assertions.assertThat(newOrcSchema.findSubtype("b").getCategory()) + assertThat(newOrcSchema.findSubtype("b").getId()).isEqualTo(2); + assertThat(newOrcSchema.findSubtype("b").getCategory()) .isEqualTo(TypeDescription.Category.DOUBLE); TypeDescription decimalC = newOrcSchema.findSubtype("c"); - Assertions.assertThat(decimalC.getId()).isEqualTo(3); - Assertions.assertThat(decimalC.getCategory()).isEqualTo(TypeDescription.Category.DECIMAL); - Assertions.assertThat(decimalC.getPrecision()).isEqualTo(15); - Assertions.assertThat(decimalC.getScale()).isEqualTo(2); + assertThat(decimalC.getId()).isEqualTo(3); + assertThat(decimalC.getCategory()).isEqualTo(TypeDescription.Category.DECIMAL); + assertThat(decimalC.getPrecision()).isEqualTo(15); + assertThat(decimalC.getScale()).isEqualTo(2); } @Test @@ -233,7 +232,7 @@ public void testInvalidTypePromotions() { TypeDescription orcSchema = ORCSchemaUtil.convert(originalSchema); Schema evolveSchema = new Schema(optional(1, "a", Types.IntegerType.get())); - Assertions.assertThatThrownBy(() -> ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema)) + assertThatThrownBy(() -> ORCSchemaUtil.buildOrcProjection(evolveSchema, orcSchema)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Can not promote LONG type to INTEGER"); } @@ -268,7 +267,7 @@ public void testSkipNonIcebergColumns() { 5, "mapCol", Types.MapType.ofOptional(3, 4, Types.StringType.get(), Types.BooleanType.get()))); - Assertions.assertThat(icebergSchema.asStruct()) + assertThat(icebergSchema.asStruct()) .as("Schemas must match.") .isEqualTo(expectedSchema.asStruct()); @@ -304,7 +303,7 @@ public void testSkipNonIcebergColumns() { required(6, "doubleCol", Types.DoubleType.get()) // Skipped mapCol2 since value has no Iceberg ID ))); - Assertions.assertThat(icebergSchema2.asStruct()) + assertThat(icebergSchema2.asStruct()) .as("Schemas must match.") .isEqualTo(expectedSchema2.asStruct()); } @@ -327,13 +326,13 @@ public void testHasIds() { optional(4, "listCol", Types.ListType.ofOptional(40, Types.DoubleType.get()))); TypeDescription orcSchema = ORCSchemaUtil.removeIds(ORCSchemaUtil.convert(schema)); - Assertions.assertThat(ORCSchemaUtil.hasIds(orcSchema)).as("Should not have Ids").isFalse(); + assertThat(ORCSchemaUtil.hasIds(orcSchema)).as("Should not have Ids").isFalse(); TypeDescription map2Col = TypeDescription.createMap(TypeDescription.createString(), TypeDescription.createBinary()); map2Col.setAttribute(ICEBERG_ID_ATTRIBUTE, "4"); orcSchema.addField("map2Col", map2Col); - Assertions.assertThat(ORCSchemaUtil.hasIds(orcSchema)) + assertThat(ORCSchemaUtil.hasIds(orcSchema)) .as("Should have Ids after adding one type with Id") .isTrue(); } @@ -403,8 +402,7 @@ public void testAssignIdsByNameMapping() { ORCSchemaUtil.applyNameMapping( ORCSchemaUtil.removeIds(typeDescriptionWithIds), nameMapping); - Assertions.assertThat( - equalsWithIds(typeDescriptionWithIds, typeDescriptionWithIdsFromNameMapping)) + assertThat(equalsWithIds(typeDescriptionWithIds, typeDescriptionWithIdsFromNameMapping)) .as("TypeDescription schemas should be equal, including IDs") .isTrue(); } @@ -512,14 +510,14 @@ public void testAssignIdsByNameMappingAndProject() { longField.setAttribute(ICEBERG_ID_ATTRIBUTE, "40"); expected.addField("long_r40", longField); - Assertions.assertThat(typeDescriptionWithIdsFromNameMapping.equals(fileSchema, false)) + assertThat(typeDescriptionWithIdsFromNameMapping.equals(fileSchema, false)) .as("ORC Schema must have the same structure, but one has Iceberg IDs") .isTrue(); TypeDescription projectedOrcSchema = ORCSchemaUtil.buildOrcProjection(mappingSchema, typeDescriptionWithIdsFromNameMapping); - Assertions.assertThat(equalsWithIds(expected, projectedOrcSchema)) + assertThat(equalsWithIds(expected, projectedOrcSchema)) .as("Schema should be the prunned by projection") .isTrue(); } 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 f95884dfa708..e026c0e4c115 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDataWriter.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.orc; +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.util.List; @@ -47,7 +50,6 @@ import org.apache.iceberg.types.Types; import org.apache.orc.OrcFile; import org.apache.orc.StripeInformation; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -112,13 +114,13 @@ public void testDataWriter() throws IOException { } DataFile dataFile = dataWriter.toDataFile(); - Assertions.assertThat(dataFile.splitOffsets()).isEqualTo(stripeOffsetsFromReader(dataFile)); - Assertions.assertThat(dataFile.format()).isEqualTo(FileFormat.ORC); - Assertions.assertThat(dataFile.content()).isEqualTo(FileContent.DATA); - Assertions.assertThat(dataFile.recordCount()).isEqualTo(records.size()); - Assertions.assertThat(dataFile.partition().size()).isEqualTo(0); - Assertions.assertThat(dataFile.sortOrderId()).isEqualTo(sortOrder.orderId()); - Assertions.assertThat(dataFile.keyMetadata()).isNull(); + assertThat(dataFile.splitOffsets()).isEqualTo(stripeOffsetsFromReader(dataFile)); + assertThat(dataFile.format()).isEqualTo(FileFormat.ORC); + assertThat(dataFile.content()).isEqualTo(FileContent.DATA); + assertThat(dataFile.recordCount()).isEqualTo(records.size()); + assertThat(dataFile.partition().size()).isEqualTo(0); + assertThat(dataFile.sortOrderId()).isEqualTo(sortOrder.orderId()); + assertThat(dataFile.keyMetadata()).isNull(); List writtenRecords; try (CloseableIterable reader = ORC.read(file.toInputFile()) @@ -127,7 +129,7 @@ public void testDataWriter() throws IOException { .build()) { writtenRecords = Lists.newArrayList(reader); } - Assertions.assertThat(writtenRecords).as("Written records should match").isEqualTo(records); + assertThat(writtenRecords).as("Written records should match").isEqualTo(records); } @Test @@ -138,8 +140,7 @@ public void testUsingFileIO() throws IOException { // use a scheme `dummy` that is not handled. Note that Hadoop 2.7.3 throws IOException // while latest Hadoop versions throw UnsupportedFileSystemException (extends IOException) ProxyOutputFile outFile = new ProxyOutputFile(Files.localOutput(temp)); - Assertions.assertThatThrownBy( - () -> new Path(outFile.location()).getFileSystem(new Configuration())) + assertThatThrownBy(() -> new Path(outFile.location()).getFileSystem(new Configuration())) .isInstanceOf(IOException.class) .hasMessageStartingWith("No FileSystem for scheme"); @@ -167,13 +168,12 @@ public void testUsingFileIO() throws IOException { OrcFile.readerOptions(new Configuration()) .filesystem(new FileIOFSUtil.InputFileSystem(outFile.toInputFile())) .maxLength(outFile.toInputFile().getLength()); - Assertions.assertThat(dataFile.splitOffsets()) - .isEqualTo(stripeOffsetsFromReader(dataFile, options)); - Assertions.assertThat(dataFile.format()).isEqualTo(FileFormat.ORC); - Assertions.assertThat(dataFile.content()).isEqualTo(FileContent.DATA); - Assertions.assertThat(dataFile.recordCount()).isEqualTo(records.size()); - Assertions.assertThat(dataFile.partition().size()).isEqualTo(0); - Assertions.assertThat(dataFile.keyMetadata()).isNull(); + assertThat(dataFile.splitOffsets()).isEqualTo(stripeOffsetsFromReader(dataFile, options)); + assertThat(dataFile.format()).isEqualTo(FileFormat.ORC); + assertThat(dataFile.content()).isEqualTo(FileContent.DATA); + assertThat(dataFile.recordCount()).isEqualTo(records.size()); + assertThat(dataFile.partition().size()).isEqualTo(0); + assertThat(dataFile.keyMetadata()).isNull(); List writtenRecords; try (CloseableIterable reader = ORC.read(outFile.toInputFile()) @@ -182,7 +182,7 @@ public void testUsingFileIO() throws IOException { .build()) { writtenRecords = Lists.newArrayList(reader); } - Assertions.assertThat(writtenRecords).as("Written records should match").isEqualTo(records); + assertThat(writtenRecords).as("Written records should match").isEqualTo(records); } private static class ProxyInputFile implements InputFile { diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java index 3e0c13313624..112885bb00a4 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestOrcDeleteWriters.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.orc; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import java.util.List; @@ -41,7 +43,6 @@ 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.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -87,11 +88,11 @@ public void testEqualityDeleteWriter() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assertions.assertThat(metadata.format()).isEqualTo(FileFormat.ORC); - Assertions.assertThat(metadata.content()).isEqualTo(FileContent.EQUALITY_DELETES); - Assertions.assertThat(metadata.recordCount()).isEqualTo(records.size()); - Assertions.assertThat(metadata.partition().size()).isEqualTo(0); - Assertions.assertThat(metadata.keyMetadata()).isNull(); + assertThat(metadata.format()).isEqualTo(FileFormat.ORC); + assertThat(metadata.content()).isEqualTo(FileContent.EQUALITY_DELETES); + assertThat(metadata.recordCount()).isEqualTo(records.size()); + assertThat(metadata.partition().size()).isEqualTo(0); + assertThat(metadata.keyMetadata()).isNull(); List deletedRecords; try (CloseableIterable reader = @@ -102,9 +103,7 @@ public void testEqualityDeleteWriter() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assertions.assertThat(deletedRecords) - .as("Deleted records should match expected") - .isEqualTo(records); + assertThat(deletedRecords).as("Deleted records should match expected").isEqualTo(records); } @Test @@ -143,11 +142,11 @@ public void testPositionDeleteWriter() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assertions.assertThat(metadata.format()).isEqualTo(FileFormat.ORC); - Assertions.assertThat(metadata.content()).isEqualTo(FileContent.POSITION_DELETES); - Assertions.assertThat(metadata.recordCount()).isEqualTo(records.size()); - Assertions.assertThat(metadata.partition().size()).isEqualTo(0); - Assertions.assertThat(metadata.keyMetadata()).isNull(); + assertThat(metadata.format()).isEqualTo(FileFormat.ORC); + assertThat(metadata.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(metadata.recordCount()).isEqualTo(records.size()); + assertThat(metadata.partition().size()).isEqualTo(0); + assertThat(metadata.keyMetadata()).isNull(); List deletedRecords; try (CloseableIterable reader = @@ -158,7 +157,7 @@ public void testPositionDeleteWriter() throws IOException { deletedRecords = Lists.newArrayList(reader); } - Assertions.assertThat(deletedRecords) + assertThat(deletedRecords) .as("Deleted records should match expected") .isEqualTo(expectedDeleteRecords); } @@ -196,11 +195,11 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { } DeleteFile metadata = deleteWriter.toDeleteFile(); - Assertions.assertThat(metadata.format()).isEqualTo(FileFormat.ORC); - Assertions.assertThat(metadata.content()).isEqualTo(FileContent.POSITION_DELETES); - Assertions.assertThat(metadata.recordCount()).isEqualTo(records.size()); - Assertions.assertThat(metadata.partition().size()).isEqualTo(0); - Assertions.assertThat(metadata.keyMetadata()).isNull(); + assertThat(metadata.format()).isEqualTo(FileFormat.ORC); + assertThat(metadata.content()).isEqualTo(FileContent.POSITION_DELETES); + assertThat(metadata.recordCount()).isEqualTo(records.size()); + assertThat(metadata.partition().size()).isEqualTo(0); + assertThat(metadata.keyMetadata()).isNull(); List deletedRecords; try (CloseableIterable reader = @@ -210,7 +209,7 @@ public void testPositionDeleteWriterWithEmptyRow() throws IOException { .build()) { deletedRecords = Lists.newArrayList(reader); } - Assertions.assertThat(deletedRecords) + assertThat(deletedRecords) .as("Deleted records should match expected") .isEqualTo(expectedDeleteRecords); } diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java b/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java index 339e885e581e..ce3985597ed0 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestTableProperties.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.orc; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.util.Random; import org.apache.hadoop.conf.Configuration; @@ -38,7 +40,6 @@ import org.apache.orc.CompressionKind; import org.apache.orc.OrcConf; import org.apache.orc.OrcFile.CompressionStrategy; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -73,14 +74,12 @@ public void testOrcTableProperties() throws Exception { String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assertions.assertThat(new File(tablePath).mkdir()) - .as("Should create the table path correctly.") - .isTrue(); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); PartitionSpec spec = PartitionSpec.unpartitioned(); Table table = new HadoopTables().create(SCHEMA, spec, properties, tablePath); - Assertions.assertThat(testFile.delete()).isTrue(); + assertThat(testFile.delete()).isTrue(); FileAppender writer = ORC.write(Files.localOutput(testFile)) @@ -92,12 +91,11 @@ public void testOrcTableProperties() throws Exception { DynFields.builder().hiddenImpl(writer.getClass(), "conf").build(writer); Configuration configuration = confField.get(); - Assertions.assertThat(OrcConf.BLOCK_SIZE.getLong(configuration)).isEqualTo(blockSizeBytes); - Assertions.assertThat(OrcConf.STRIPE_SIZE.getLong(configuration)).isEqualTo(stripeSizeBytes); - Assertions.assertThat(OrcConf.COMPRESS.getString(configuration)).isEqualTo(codecAsString); - Assertions.assertThat(OrcConf.COMPRESSION_STRATEGY.getString(configuration)) - .isEqualTo(strategyAsString); - Assertions.assertThat(configuration.get(TableProperties.DEFAULT_FILE_FORMAT)) + assertThat(OrcConf.BLOCK_SIZE.getLong(configuration)).isEqualTo(blockSizeBytes); + assertThat(OrcConf.STRIPE_SIZE.getLong(configuration)).isEqualTo(stripeSizeBytes); + assertThat(OrcConf.COMPRESS.getString(configuration)).isEqualTo(codecAsString); + assertThat(OrcConf.COMPRESSION_STRATEGY.getString(configuration)).isEqualTo(strategyAsString); + assertThat(configuration.get(TableProperties.DEFAULT_FILE_FORMAT)) .isEqualTo(FileFormat.ORC.name()); } @@ -122,14 +120,12 @@ public void testOrcTableDeleteProperties() throws Exception { String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assertions.assertThat(new File(tablePath).mkdir()) - .as("Should create the table path correctly.") - .isTrue(); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); PartitionSpec spec = PartitionSpec.unpartitioned(); Table table = new HadoopTables().create(SCHEMA, spec, properties, tablePath); - Assertions.assertThat(testFile.delete()).isTrue(); + assertThat(testFile.delete()).isTrue(); EqualityDeleteWriter deleteWriter = ORC.writeDeletes(Files.localOutput(testFile)) @@ -146,12 +142,11 @@ public void testOrcTableDeleteProperties() throws Exception { DynFields.builder().hiddenImpl(orcFileAppender.getClass(), "conf").build(orcFileAppender); Configuration configuration = confField.get(); - Assertions.assertThat(OrcConf.BLOCK_SIZE.getLong(configuration)).isEqualTo(blockSizeBytes); - Assertions.assertThat(OrcConf.STRIPE_SIZE.getLong(configuration)).isEqualTo(stripeSizeBytes); - Assertions.assertThat(OrcConf.COMPRESS.getString(configuration)).isEqualTo(codecAsString); - Assertions.assertThat(OrcConf.COMPRESSION_STRATEGY.getString(configuration)) - .isEqualTo(strategyAsString); - Assertions.assertThat(configuration.get(TableProperties.DEFAULT_FILE_FORMAT)) + assertThat(OrcConf.BLOCK_SIZE.getLong(configuration)).isEqualTo(blockSizeBytes); + assertThat(OrcConf.STRIPE_SIZE.getLong(configuration)).isEqualTo(stripeSizeBytes); + assertThat(OrcConf.COMPRESS.getString(configuration)).isEqualTo(codecAsString); + assertThat(OrcConf.COMPRESSION_STRATEGY.getString(configuration)).isEqualTo(strategyAsString); + assertThat(configuration.get(TableProperties.DEFAULT_FILE_FORMAT)) .isEqualTo(FileFormat.ORC.name()); } } diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java index c2ea335fabe5..f0c5f2a0f762 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/JdbcSnowflakeClientTest.java @@ -21,6 +21,8 @@ import static org.apache.iceberg.snowflake.JdbcSnowflakeClient.DATABASE_NOT_FOUND_ERROR_CODES; import static org.apache.iceberg.snowflake.JdbcSnowflakeClient.SCHEMA_NOT_FOUND_ERROR_CODES; import static org.apache.iceberg.snowflake.JdbcSnowflakeClient.TABLE_NOT_FOUND_ERROR_CODES; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; import static org.mockito.ArgumentMatchers.any; import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; @@ -37,7 +39,6 @@ import org.apache.iceberg.jdbc.JdbcClientPool; import org.apache.iceberg.jdbc.UncheckedInterruptedException; import org.apache.iceberg.jdbc.UncheckedSQLException; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.ExtendWith; @@ -79,7 +80,7 @@ public void before() throws SQLException, InterruptedException { @Test public void testNullClientPoolInConstructor() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> new JdbcSnowflakeClient(null)) .withMessageContaining("JdbcClientPool must be non-null"); } @@ -91,8 +92,7 @@ public void testDatabaseExists() throws SQLException { when(mockResultSet.getString("database_name")).thenReturn("DB_1"); when(mockResultSet.getString("name")).thenReturn("SCHEMA_1"); - Assertions.assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))) - .isTrue(); + assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))).isTrue(); verify(mockQueryHarness) .query( @@ -113,16 +113,13 @@ public void testDatabaseDoesntExist() throws SQLException { new SQLException("Database does not exist or not authorized", "2000", 2001, null)); // Error code 2003 - Assertions.assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))) - .isFalse(); + assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))).isFalse(); // Error code 2043 - Assertions.assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))) - .isFalse(); + assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))).isFalse(); // Error code 2001 - Assertions.assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))) - .isFalse(); + assertThat(snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))).isFalse(); } @Test @@ -130,7 +127,7 @@ public void testDatabaseFailureWithOtherException() throws SQLException { Exception injectedException = new SQLException("Some other exception", "2000", 2, null); when(mockResultSet.next()).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining("Failed to check if database 'DATABASE: 'DB_1'' exists") .withCause(injectedException); @@ -143,7 +140,7 @@ public void testDatabaseFailureWithInterruptedException() Exception injectedException = new InterruptedException("Fake interrupted exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedInterruptedException.class) + assertThatExceptionOfType(UncheckedInterruptedException.class) .isThrownBy(() -> snowflakeClient.databaseExists(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining("Interrupted while checking if database 'DATABASE: 'DB_1'' exists") .withCause(injectedException); @@ -161,8 +158,7 @@ public void testSchemaExists() throws SQLException { when(mockResultSet.getString("database_name")).thenReturn("DB1"); when(mockResultSet.getString("schema_name")).thenReturn("SCHEMA1"); - Assertions.assertThat( - snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB1", "SCHEMA1"))) + assertThat(snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB1", "SCHEMA1"))) .isTrue(); verify(mockQueryHarness) @@ -201,18 +197,15 @@ public void testSchemaDoesntExistNoSchemaFoundException() throws SQLException { when(mockResultSet.getString("database_name")).thenReturn("DB1"); // Error code 2003 - Assertions.assertThat( - snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) + assertThat(snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) .isFalse(); // Error code 2043 - Assertions.assertThat( - snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) + assertThat(snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) .isFalse(); // Error code 2001 - Assertions.assertThat( - snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) + assertThat(snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) .isFalse(); } @@ -228,7 +221,7 @@ public void testSchemaFailureWithOtherException() throws SQLException { when(mockResultSet.getString("name")).thenReturn("DB1").thenReturn("SCHEMA1"); when(mockResultSet.getString("database_name")).thenReturn("DB1"); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy( () -> snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) .withMessageContaining("Failed to check if schema 'SCHEMA: 'DB_1.SCHEMA_2'' exists") @@ -242,7 +235,7 @@ public void testSchemaFailureWithInterruptedException() Exception injectedException = new InterruptedException("Fake Interrupted exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedInterruptedException.class) + assertThatExceptionOfType(UncheckedInterruptedException.class) .isThrownBy( () -> snowflakeClient.schemaExists(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) .withMessageContaining("Interrupted while checking if database 'DATABASE: 'DB_1'' exists") @@ -263,7 +256,7 @@ public void testListDatabasesInAccount() throws SQLException { eq("SHOW DATABASES IN ACCOUNT"), any(JdbcSnowflakeClient.ResultSetParser.class)); - Assertions.assertThat(actualList) + assertThat(actualList) .containsExactly( SnowflakeIdentifier.ofDatabase("DB_1"), SnowflakeIdentifier.ofDatabase("DB_2"), @@ -281,7 +274,7 @@ public void testListDatabasesSQLExceptionAtRootLevel() throws SQLException, Inte new SQLException(String.format("SQL exception with Error Code %d", 0), "2000", 0, null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.listDatabases()) .withMessageContaining("Failed to list databases") .withCause(injectedException); @@ -298,7 +291,7 @@ public void testListDatabasesSQLExceptionWithoutErrorCode() Exception injectedException = new SQLException("Fake SQL exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.listDatabases()) .withMessageContaining("Failed to list databases") .withCause(injectedException); @@ -314,7 +307,7 @@ public void testListDatabasesInterruptedException() throws SQLException, Interru Exception injectedException = new InterruptedException("Fake interrupted exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedInterruptedException.class) + assertThatExceptionOfType(UncheckedInterruptedException.class) .isThrownBy(() -> snowflakeClient.listDatabases()) .withMessageContaining("Interrupted while listing databases") .withCause(injectedException); @@ -347,7 +340,7 @@ public void testListSchemasInAccount() throws SQLException { any(JdbcSnowflakeClient.ResultSetParser.class), eq(null)); - Assertions.assertThat(actualList) + assertThat(actualList) .containsExactly( SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_1"), SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"), @@ -375,7 +368,7 @@ public void testListSchemasInDatabase() throws SQLException { any(JdbcSnowflakeClient.ResultSetParser.class), eq("DB_1")); - Assertions.assertThat(actualList) + assertThat(actualList) .containsExactly( SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_1"), SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2")); @@ -392,7 +385,7 @@ public void testListSchemasSQLExceptionAtRootLevel() throws SQLException, Interr new SQLException(String.format("SQL exception with Error Code %d", 0), "2000", 0, null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.listSchemas(SnowflakeIdentifier.ofRoot())) .withMessageContaining("Failed to list schemas for scope 'ROOT: '''") .withCause(injectedException); @@ -415,7 +408,7 @@ public void testListSchemasSQLExceptionAtDatabaseLevel() null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(NoSuchNamespaceException.class) + assertThatExceptionOfType(NoSuchNamespaceException.class) .isThrownBy(() -> snowflakeClient.listSchemas(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining( String.format( @@ -428,7 +421,7 @@ public void testListSchemasSQLExceptionAtDatabaseLevel() /** List schemas is not supported at Schema level */ @Test public void testListSchemasAtSchemaLevel() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> snowflakeClient.listSchemas(SnowflakeIdentifier.ofSchema("DB_1", "SCHEMA_2"))) .withMessageContaining("Unsupported scope type for listSchemas: SCHEMA: 'DB_1.SCHEMA_2'"); @@ -445,7 +438,7 @@ public void testListSchemasSQLExceptionWithoutErrorCode() Exception injectedException = new SQLException("Fake SQL exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.listSchemas(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining("Failed to list schemas for scope 'DATABASE: 'DB_1''") .withCause(injectedException); @@ -461,7 +454,7 @@ public void testListSchemasInterruptedException() throws SQLException, Interrupt Exception injectedException = new InterruptedException("Fake interrupted exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedInterruptedException.class) + assertThatExceptionOfType(UncheckedInterruptedException.class) .isThrownBy(() -> snowflakeClient.listSchemas(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining("Interrupted while listing schemas for scope 'DATABASE: 'DB_1''") .withCause(injectedException); @@ -506,7 +499,7 @@ public void testListIcebergTablesInAccount() throws SQLException { any(JdbcSnowflakeClient.ResultSetParser.class), eq(null)); - Assertions.assertThat(actualList) + assertThat(actualList) .containsExactly( SnowflakeIdentifier.ofTable("DB_1", "SCHEMA_1", "TABLE_1"), SnowflakeIdentifier.ofTable("DB_1", "SCHEMA_1", "TABLE_2"), @@ -545,7 +538,7 @@ public void testListIcebergTablesInDatabase() throws SQLException { any(JdbcSnowflakeClient.ResultSetParser.class), eq("DB_1")); - Assertions.assertThat(actualList) + assertThat(actualList) .containsExactly( SnowflakeIdentifier.ofTable("DB_1", "SCHEMA_1", "TABLE_1"), SnowflakeIdentifier.ofTable("DB_1", "SCHEMA_1", "TABLE_2"), @@ -574,7 +567,7 @@ public void testListIcebergTablesInSchema() throws SQLException { any(JdbcSnowflakeClient.ResultSetParser.class), eq("DB_1.SCHEMA_1")); - Assertions.assertThat(actualList) + assertThat(actualList) .containsExactly( SnowflakeIdentifier.ofTable("DB_1", "SCHEMA_1", "TABLE_1"), SnowflakeIdentifier.ofTable("DB_1", "SCHEMA_1", "TABLE_2")); @@ -592,7 +585,7 @@ public void testListIcebergTablesSQLExceptionAtRootLevel() new SQLException(String.format("SQL exception with Error Code %d", 0), "2000", 0, null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofRoot())) .withMessageContaining("Failed to list tables for scope 'ROOT: '''") .withCause(injectedException); @@ -615,7 +608,7 @@ public void testListIcebergTablesSQLExceptionAtDatabaseLevel() null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(NoSuchNamespaceException.class) + assertThatExceptionOfType(NoSuchNamespaceException.class) .isThrownBy( () -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining( @@ -643,7 +636,7 @@ public void testListIcebergTablesSQLExceptionAtSchemaLevel() null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(NoSuchNamespaceException.class) + assertThatExceptionOfType(NoSuchNamespaceException.class) .isThrownBy( () -> snowflakeClient.listIcebergTables( @@ -667,7 +660,7 @@ public void testListIcebergTablesSQLExceptionWithoutErrorCode() Exception injectedException = new SQLException("Fake SQL exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy(() -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining("Failed to list tables for scope 'DATABASE: 'DB_1''") .withCause(injectedException); @@ -684,7 +677,7 @@ public void testListIcebergTablesInterruptedException() Exception injectedException = new InterruptedException("Fake interrupted exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedInterruptedException.class) + assertThatExceptionOfType(UncheckedInterruptedException.class) .isThrownBy(() -> snowflakeClient.listIcebergTables(SnowflakeIdentifier.ofDatabase("DB_1"))) .withMessageContaining("Interrupted while listing tables for scope 'DATABASE: 'DB_1''") .withCause(injectedException); @@ -719,7 +712,7 @@ public void testGetS3TableMetadata() throws SQLException { "s3://tab1/metadata/v3.metadata.json", "success", null); - Assertions.assertThat(actualMetadata).isEqualTo(expectedMetadata); + assertThat(actualMetadata).isEqualTo(expectedMetadata); } /** @@ -751,7 +744,7 @@ public void testGetAzureTableMetadata() throws SQLException { "wasbs://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", "success", null); - Assertions.assertThat(actualMetadata).isEqualTo(expectedMetadata); + assertThat(actualMetadata).isEqualTo(expectedMetadata); } /** @@ -783,7 +776,7 @@ public void testGetGcsTableMetadata() throws SQLException { "gs://tab5/metadata/v793.metadata.json", "success", null); - Assertions.assertThat(actualMetadata).isEqualTo(expectedMetadata); + assertThat(actualMetadata).isEqualTo(expectedMetadata); } /** Malformed JSON from a ResultSet should propagate as an IllegalArgumentException. */ @@ -791,7 +784,7 @@ public void testGetGcsTableMetadata() throws SQLException { public void testGetTableMetadataMalformedJson() throws SQLException { when(mockResultSet.next()).thenReturn(true); when(mockResultSet.getString("METADATA")).thenReturn("{\"malformed_no_closing_bracket"); - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> snowflakeClient.loadTableMetadata( @@ -815,7 +808,7 @@ public void testGetTableMetadataSQLException() throws SQLException, InterruptedE null); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(NoSuchTableException.class) + assertThatExceptionOfType(NoSuchTableException.class) .isThrownBy( () -> snowflakeClient.loadTableMetadata( @@ -839,7 +832,7 @@ public void testGetTableMetadataSQLExceptionWithoutErrorCode() Exception injectedException = new SQLException("Fake SQL exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedSQLException.class) + assertThatExceptionOfType(UncheckedSQLException.class) .isThrownBy( () -> snowflakeClient.loadTableMetadata( @@ -858,7 +851,7 @@ public void testGetTableMetadataInterruptedException() throws SQLException, Inte Exception injectedException = new InterruptedException("Fake interrupted exception"); when(mockClientPool.run(any(ClientPool.Action.class))).thenThrow(injectedException); - Assertions.assertThatExceptionOfType(UncheckedInterruptedException.class) + assertThatExceptionOfType(UncheckedInterruptedException.class) .isThrownBy( () -> snowflakeClient.loadTableMetadata( diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java index ef47f5b784fa..4de368a07a85 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/NamespaceHelpersTest.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.snowflake; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class NamespaceHelpersTest { @@ -29,8 +31,8 @@ public void testRoundTripRoot() { Namespace icebergNamespace = Namespace.empty(); SnowflakeIdentifier snowflakeIdentifier = NamespaceHelpers.toSnowflakeIdentifier(icebergNamespace); - Assertions.assertThat(snowflakeIdentifier).isEqualTo(SnowflakeIdentifier.ofRoot()); - Assertions.assertThat(NamespaceHelpers.toIcebergNamespace(snowflakeIdentifier)) + assertThat(snowflakeIdentifier).isEqualTo(SnowflakeIdentifier.ofRoot()); + assertThat(NamespaceHelpers.toIcebergNamespace(snowflakeIdentifier)) .isEqualTo(icebergNamespace); } @@ -39,8 +41,8 @@ public void testRoundTripDatabase() { Namespace icebergNamespace = Namespace.of("DB1"); SnowflakeIdentifier snowflakeIdentifier = NamespaceHelpers.toSnowflakeIdentifier(icebergNamespace); - Assertions.assertThat(snowflakeIdentifier).isEqualTo(SnowflakeIdentifier.ofDatabase("DB1")); - Assertions.assertThat(NamespaceHelpers.toIcebergNamespace(snowflakeIdentifier)) + assertThat(snowflakeIdentifier).isEqualTo(SnowflakeIdentifier.ofDatabase("DB1")); + assertThat(NamespaceHelpers.toIcebergNamespace(snowflakeIdentifier)) .isEqualTo(icebergNamespace); } @@ -49,9 +51,8 @@ public void testRoundTripSchema() { Namespace icebergNamespace = Namespace.of("DB1", "SCHEMA1"); SnowflakeIdentifier snowflakeIdentifier = NamespaceHelpers.toSnowflakeIdentifier(icebergNamespace); - Assertions.assertThat(snowflakeIdentifier) - .isEqualTo(SnowflakeIdentifier.ofSchema("DB1", "SCHEMA1")); - Assertions.assertThat(NamespaceHelpers.toIcebergNamespace(snowflakeIdentifier)) + assertThat(snowflakeIdentifier).isEqualTo(SnowflakeIdentifier.ofSchema("DB1", "SCHEMA1")); + assertThat(NamespaceHelpers.toIcebergNamespace(snowflakeIdentifier)) .isEqualTo(icebergNamespace); } @@ -59,15 +60,15 @@ public void testRoundTripSchema() { public void testRoundTripTable() { TableIdentifier icebergTable = TableIdentifier.of("DB1", "SCHEMA1", "TABLE1"); SnowflakeIdentifier snowflakeIdentifier = NamespaceHelpers.toSnowflakeIdentifier(icebergTable); - Assertions.assertThat(snowflakeIdentifier) + assertThat(snowflakeIdentifier) .isEqualTo(SnowflakeIdentifier.ofTable("DB1", "SCHEMA1", "TABLE1")); - Assertions.assertThat(NamespaceHelpers.toIcebergTableIdentifier(snowflakeIdentifier)) + assertThat(NamespaceHelpers.toIcebergTableIdentifier(snowflakeIdentifier)) .isEqualTo(icebergTable); } @Test public void testToSnowflakeIdentifierMaxNamespaceLevel() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> NamespaceHelpers.toSnowflakeIdentifier( @@ -77,7 +78,7 @@ public void testToSnowflakeIdentifierMaxNamespaceLevel() { @Test public void testToSnowflakeIdentifierTableBadNamespace() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> NamespaceHelpers.toSnowflakeIdentifier( @@ -87,7 +88,7 @@ public void testToSnowflakeIdentifierTableBadNamespace() { @Test public void testToIcebergNamespaceTableFails() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> NamespaceHelpers.toIcebergNamespace( @@ -97,7 +98,7 @@ public void testToIcebergNamespaceTableFails() { @Test public void testToIcebergTableIdentifier() { - Assertions.assertThat( + assertThat( NamespaceHelpers.toIcebergTableIdentifier( SnowflakeIdentifier.ofTable("DB1", "SCHEMA1", "TABLE1"))) .isEqualTo(TableIdentifier.of("DB1", "SCHEMA1", "TABLE1")); @@ -105,7 +106,7 @@ public void testToIcebergTableIdentifier() { @Test public void testToIcebergTableIdentifierWrongType() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> NamespaceHelpers.toIcebergTableIdentifier( diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java index adda8bc65f20..b3d5cef431e5 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.snowflake; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + import java.io.IOException; import java.util.Map; import org.apache.iceberg.PartitionSpec; @@ -32,7 +35,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -119,7 +121,7 @@ public FileIO newFileIO(String impl, Map prop, Object hadoopConf @Test public void testInitializeNullClient() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> catalog.initialize(TEST_CATALOG_NAME, null, fakeFileIOFactory, properties)) .withMessageContaining("snowflakeClient must be non-null"); @@ -127,21 +129,21 @@ public void testInitializeNullClient() { @Test public void testInitializeNullFileIO() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> catalog.initialize(TEST_CATALOG_NAME, fakeClient, null, properties)) .withMessageContaining("fileIOFactory must be non-null"); } @Test public void testListNamespaceInRoot() { - Assertions.assertThat(catalog.listNamespaces()) + assertThat(catalog.listNamespaces()) .containsExactly(Namespace.of("DB_1"), Namespace.of("DB_2"), Namespace.of("DB_3")); } @Test public void testListNamespaceWithinDB() { String dbName = "DB_1"; - Assertions.assertThat(catalog.listNamespaces(Namespace.of(dbName))) + assertThat(catalog.listNamespaces(Namespace.of(dbName))) .containsExactly(Namespace.of(dbName, "SCHEMA_1")); } @@ -150,7 +152,7 @@ public void testListNamespaceWithinNonExistentDB() { // Existence check for nonexistent parent namespaces is optional in the SupportsNamespaces // interface. String dbName = "NONEXISTENT_DB"; - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName))) .withMessageContaining("does not exist") .withMessageContaining(dbName); @@ -162,7 +164,7 @@ public void testListNamespaceWithinSchema() { // a database.schema. String dbName = "DB_3"; String schemaName = "SCHEMA_4"; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> catalog.listNamespaces(Namespace.of(dbName, schemaName))) .withMessageContaining("level") .withMessageContaining("DB_3.SCHEMA_4"); @@ -170,7 +172,7 @@ public void testListNamespaceWithinSchema() { @Test public void testListTables() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> catalog.listTables(Namespace.empty())) .withMessageContaining("listTables must be at SCHEMA level"); } @@ -178,7 +180,7 @@ public void testListTables() { @Test public void testListTablesWithinDB() { String dbName = "DB_1"; - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy(() -> catalog.listTables(Namespace.of(dbName))) .withMessageContaining("listTables must be at SCHEMA level"); } @@ -187,7 +189,7 @@ public void testListTablesWithinDB() { public void testListTablesWithinNonexistentDB() { String dbName = "NONEXISTENT_DB"; String schemaName = "NONEXISTENT_SCHEMA"; - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> catalog.listTables(Namespace.of(dbName, schemaName))) .withMessageContaining("does not exist") .withMessageContaining(dbName); @@ -197,7 +199,7 @@ public void testListTablesWithinNonexistentDB() { public void testListTablesWithinSchema() { String dbName = "DB_2"; String schemaName = "SCHEMA_2"; - Assertions.assertThat(catalog.listTables(Namespace.of(dbName, schemaName))) + assertThat(catalog.listTables(Namespace.of(dbName, schemaName))) .containsExactly( TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_3"), TableIdentifier.of("DB_2", "SCHEMA_2", "TAB_4")); @@ -207,7 +209,7 @@ public void testListTablesWithinSchema() { public void testListTablesWithinNonexistentSchema() { String dbName = "DB_2"; String schemaName = "NONEXISTENT_SCHEMA"; - Assertions.assertThatExceptionOfType(RuntimeException.class) + assertThatExceptionOfType(RuntimeException.class) .isThrownBy(() -> catalog.listTables(Namespace.of(dbName, schemaName))) .withMessageContaining("does not exist") .withMessageContaining("DB_2.NONEXISTENT_SCHEMA"); @@ -216,32 +218,32 @@ public void testListTablesWithinNonexistentSchema() { @Test public void testLoadS3Table() { Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1"), "TAB_1")); - Assertions.assertThat(table.location()).isEqualTo("s3://tab1"); + assertThat(table.location()).isEqualTo("s3://tab1"); } @Test public void testLoadAzureTable() { Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_2", "SCHEMA_2"), "TAB_3")); - Assertions.assertThat(table.location()) + assertThat(table.location()) .isEqualTo("wasbs://mycontainer@myaccount.blob.core.windows.net/tab1"); } @Test public void testLoadGcsTable() { Table table = catalog.loadTable(TableIdentifier.of(Namespace.of("DB_3", "SCHEMA_3"), "TAB_5")); - Assertions.assertThat(table.location()).isEqualTo("gs://tab5"); + assertThat(table.location()).isEqualTo("gs://tab5"); } @Test public void testLoadTableWithMalformedTableIdentifier() { - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> catalog.loadTable( TableIdentifier.of(Namespace.of("DB_1", "SCHEMA_1", "BAD_NS_LEVEL"), "TAB_1"))) .withMessageContaining("level") .withMessageContaining("DB_1.SCHEMA_1.BAD_NS_LEVEL"); - Assertions.assertThatExceptionOfType(IllegalArgumentException.class) + assertThatExceptionOfType(IllegalArgumentException.class) .isThrownBy( () -> catalog.loadTable(TableIdentifier.of(Namespace.of("DB_WITHOUT_SCHEMA"), "TAB_1"))) .withMessageContaining("level") @@ -256,7 +258,7 @@ public void testCloseBeforeInitializeDoesntThrow() throws IOException { // add a catalog to auto-close() helpers but end up never using/initializing a catalog. catalog.close(); - Assertions.assertThat(fakeClient.isClosed()) + assertThat(fakeClient.isClosed()) .overridingErrorMessage("expected not to have called close() on snowflakeClient") .isFalse(); } @@ -265,10 +267,10 @@ public void testCloseBeforeInitializeDoesntThrow() throws IOException { public void testClose() throws IOException { catalog.newTableOps(TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1")); catalog.close(); - Assertions.assertThat(fakeClient.isClosed()) + assertThat(fakeClient.isClosed()) .overridingErrorMessage("expected close() to propagate to snowflakeClient") .isTrue(); - Assertions.assertThat(fakeFileIO.isClosed()) + assertThat(fakeFileIO.isClosed()) .overridingErrorMessage("expected close() to propagate to fileIO") .isTrue(); } @@ -278,21 +280,19 @@ public void testTableNameFromTableOperations() { SnowflakeTableOperations castedTableOps = (SnowflakeTableOperations) catalog.newTableOps(TableIdentifier.of("DB_1", "SCHEMA_1", "TAB_1")); - Assertions.assertThat(castedTableOps.fullTableName()).isEqualTo("slushLog.DB_1.SCHEMA_1.TAB_1"); + assertThat(castedTableOps.fullTableName()).isEqualTo("slushLog.DB_1.SCHEMA_1.TAB_1"); } @Test public void testDatabaseExists() { - Assertions.assertThat(catalog.namespaceExists(Namespace.of("DB_1"))).isTrue(); - Assertions.assertThat(catalog.namespaceExists(Namespace.of("NONEXISTENT_DB"))).isFalse(); + assertThat(catalog.namespaceExists(Namespace.of("DB_1"))).isTrue(); + assertThat(catalog.namespaceExists(Namespace.of("NONEXISTENT_DB"))).isFalse(); } @Test public void testSchemaExists() { - Assertions.assertThat(catalog.namespaceExists(Namespace.of("DB_1", "SCHEMA_1"))).isTrue(); - Assertions.assertThat(catalog.namespaceExists(Namespace.of("DB_1", "NONEXISTENT_SCHEMA"))) - .isFalse(); - Assertions.assertThat(catalog.namespaceExists(Namespace.of("NONEXISTENT_DB", "SCHEMA_1"))) - .isFalse(); + assertThat(catalog.namespaceExists(Namespace.of("DB_1", "SCHEMA_1"))).isTrue(); + assertThat(catalog.namespaceExists(Namespace.of("DB_1", "NONEXISTENT_SCHEMA"))).isFalse(); + assertThat(catalog.namespaceExists(Namespace.of("NONEXISTENT_DB", "SCHEMA_1"))).isFalse(); } } From 0e91d70e0b356b1bc3293c110ab2665735516db5 Mon Sep 17 00:00:00 2001 From: DaqianLiao <360989637@qq.com> Date: Tue, 18 Jun 2024 19:07:33 +0800 Subject: [PATCH 0388/1019] Core: Simplify newTableMetadata method in TableMetadata (#10528) --- core/src/main/java/org/apache/iceberg/TableMetadata.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 9587c57a0fd2..a7edddecad52 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -73,12 +73,7 @@ public static TableMetadata newTableMetadata( public static TableMetadata newTableMetadata( Schema schema, PartitionSpec spec, String location, Map properties) { - SortOrder sortOrder = SortOrder.unsorted(); - int formatVersion = - PropertyUtil.propertyAsInt( - properties, TableProperties.FORMAT_VERSION, DEFAULT_TABLE_FORMAT_VERSION); - return newTableMetadata( - schema, spec, sortOrder, location, persistedProperties(properties), formatVersion); + return newTableMetadata(schema, spec, SortOrder.unsorted(), location, properties); } private static Map unreservedProperties(Map rawProperties) { From c9f1568e9a649c8e07d99074b866cd97babea906 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 18 Jun 2024 13:54:48 +0200 Subject: [PATCH 0389/1019] Build: Run revapi workflow on workflow/build system changes (#10485) --- .github/workflows/api-binary-compatibility.yml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index 49107000d8de..80dc0da8856c 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -27,6 +27,9 @@ on: - 'apache-iceberg-**' pull_request: paths: + - '.github/workflows/api-binary-compatibility.yml' + - '*.gradle' + - 'gradle*' - 'api/**' - '.palantir/revapi.yml' From fb48027ee55eadbfc31bae0e8a36bd3eeec6a508 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 18 Jun 2024 17:22:47 +0200 Subject: [PATCH 0390/1019] API, Core, Flink, Avro, Parquet: Remove dead code and update javadocs (#10530) --- .../iceberg/expressions/ExpressionUtil.java | 2 +- .../iceberg/expressions/Projections.java | 4 -- .../GenericArrowVectorAccessorFactory.java | 6 --- .../org/apache/iceberg/DeleteFileIndex.java | 16 ------- .../apache/iceberg/hadoop/HadoopStreams.java | 5 --- .../sink/shuffle/MapRangePartitioner.java | 8 +++- .../source/enumerator/EnumerationHistory.java | 6 ++- .../sink/shuffle/MapRangePartitioner.java | 8 +++- .../source/enumerator/EnumerationHistory.java | 6 ++- .../sink/shuffle/MapRangePartitioner.java | 8 +++- .../source/enumerator/EnumerationHistory.java | 6 ++- .../apache/iceberg/parquet/ParquetAvro.java | 4 -- .../iceberg/parquet/ParquetValueReaders.java | 42 ------------------- .../iceberg/parquet/ParquetValueWriters.java | 8 ---- 14 files changed, 37 insertions(+), 92 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 3708dafc4126..bf72e03bc406 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -484,7 +484,7 @@ public String predicate(UnboundPredicate pred) { } } - private static List abbreviateValues(List sanitizedValues) { + private static List abbreviateValues(List sanitizedValues) { if (sanitizedValues.size() >= LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD) { Set distinctValues = ImmutableSet.copyOf(sanitizedValues); if (distinctValues.size() diff --git a/api/src/main/java/org/apache/iceberg/expressions/Projections.java b/api/src/main/java/org/apache/iceberg/expressions/Projections.java index f873edfff5b1..216f8a739cc3 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Projections.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Projections.java @@ -190,10 +190,6 @@ public Expression predicate(UnboundPredicate pred) { PartitionSpec spec() { return spec; } - - boolean isCaseSensitive() { - return caseSensitive; - } } private static class InclusiveProjection extends BaseProjectionEvaluator { diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java index a988516bc6df..6350660d57bf 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/GenericArrowVectorAccessorFactory.java @@ -561,12 +561,6 @@ private static class FixedSizeBinaryAccessor< private final FixedSizeBinaryVector vector; private final StringFactory stringFactory; - FixedSizeBinaryAccessor(FixedSizeBinaryVector vector) { - super(vector); - this.vector = vector; - this.stringFactory = null; - } - FixedSizeBinaryAccessor( FixedSizeBinaryVector vector, StringFactory stringFactory) { super(vector); diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index e401a8179ea0..c26716481836 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -737,22 +737,10 @@ public DeleteFile wrapped() { return wrapped; } - public PartitionSpec spec() { - return spec; - } - - public StructLike partition() { - return wrapped.partition(); - } - public long applySequenceNumber() { return applySequenceNumber; } - public FileContent content() { - return wrapped.content(); - } - public List equalityFields() { if (equalityFields == null) { synchronized (this) { @@ -778,10 +766,6 @@ public Map nullValueCounts() { return wrapped.nullValueCounts(); } - public Map nanValueCounts() { - return wrapped.nanValueCounts(); - } - public boolean hasLowerAndUpperBounds() { return wrapped.lowerBounds() != null && wrapped.upperBounds() != null; } diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java index f9b43b684666..c817b1d90afb 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java @@ -21,7 +21,6 @@ import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; -import java.nio.ByteBuffer; import java.util.Arrays; import org.apache.hadoop.fs.FSDataInputStream; import org.apache.hadoop.fs.FSDataOutputStream; @@ -123,10 +122,6 @@ public int read(byte[] b, int off, int len) throws IOException { return stream.read(b, off, len); } - public int read(ByteBuffer buf) throws IOException { - return stream.read(buf); - } - @SuppressWarnings("checkstyle:NoFinalizer") @Override protected void finalize() throws Throwable { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index fb1a8f03a65c..dde86b5b6047 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -163,6 +163,8 @@ Map mapStatistics() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ @@ -328,7 +330,11 @@ static class KeyAssignment { } } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.length == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java index ef21dad0199d..ec56a9ecdac1 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java @@ -78,7 +78,11 @@ synchronized boolean hasFullHistory() { return count >= history.length; } - /** @return true if split discovery should pause because assigner has too many splits already. */ + /** + * Checks whether split discovery should be paused. + * + * @return true if split discovery should pause because assigner has too many splits already. + */ synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { if (count < history.length) { // only check throttling when full history is obtained. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index fb1a8f03a65c..dde86b5b6047 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -163,6 +163,8 @@ Map mapStatistics() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ @@ -328,7 +330,11 @@ static class KeyAssignment { } } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.length == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java index ef21dad0199d..ec56a9ecdac1 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java @@ -78,7 +78,11 @@ synchronized boolean hasFullHistory() { return count >= history.length; } - /** @return true if split discovery should pause because assigner has too many splits already. */ + /** + * Checks whether split discovery should be paused. + * + * @return true if split discovery should pause because assigner has too many splits already. + */ synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { if (count < history.length) { // only check throttling when full history is obtained. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index 4f52915a925e..298426cee872 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -161,6 +161,8 @@ Map mapStatistics() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ @@ -326,7 +328,11 @@ static class KeyAssignment { } } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.length == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java index ef21dad0199d..ec56a9ecdac1 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java @@ -78,7 +78,11 @@ synchronized boolean hasFullHistory() { return count >= history.length; } - /** @return true if split discovery should pause because assigner has too many splits already. */ + /** + * Checks whether split discovery should be paused. + * + * @return true if split discovery should pause because assigner has too many splits already. + */ synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { if (count < history.length) { // only check throttling when full history is obtained. diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java index 974c00076b7d..680eb1435967 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java @@ -377,10 +377,6 @@ private static class Pair { this.second = second; } - public static Pair of(K first, V second) { - return new Pair<>(first, second); - } - public K getFirst() { return first; } 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 c1f76e7bdb9a..a335fe11cf49 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java @@ -696,28 +696,21 @@ public V setValue(V newValue) { } public abstract static class StructReader implements ParquetValueReader { - private interface Setter { - void set(R record, int pos, Object reuse); - } - private final ParquetValueReader[] readers; private final TripleIterator column; private final List> children; - @SuppressWarnings("unchecked") protected StructReader(List types, List> readers) { this.readers = (ParquetValueReader[]) Array.newInstance(ParquetValueReader.class, readers.size()); TripleIterator[] columns = (TripleIterator[]) Array.newInstance(TripleIterator.class, readers.size()); - Setter[] setters = (Setter[]) Array.newInstance(Setter.class, readers.size()); ImmutableList.Builder> columnsBuilder = ImmutableList.builder(); for (int i = 0; i < readers.size(); i += 1) { ParquetValueReader reader = readers.get(i); this.readers[i] = readers.get(i); columns[i] = reader.column(); - setters[i] = newSetter(reader, types.get(i)); columnsBuilder.addAll(reader.columns()); } @@ -754,41 +747,6 @@ public List> columns() { return children; } - @SuppressWarnings("unchecked") - private Setter newSetter(ParquetValueReader reader, Type type) { - if (reader instanceof UnboxedReader && type.isPrimitive()) { - UnboxedReader unboxed = (UnboxedReader) reader; - switch (type.asPrimitiveType().getPrimitiveTypeName()) { - case BOOLEAN: - return (record, pos, ignored) -> setBoolean(record, pos, unboxed.readBoolean()); - case INT32: - return (record, pos, ignored) -> setInteger(record, pos, unboxed.readInteger()); - case INT64: - return (record, pos, ignored) -> setLong(record, pos, unboxed.readLong()); - case FLOAT: - return (record, pos, ignored) -> setFloat(record, pos, unboxed.readFloat()); - case DOUBLE: - return (record, pos, ignored) -> setDouble(record, pos, unboxed.readDouble()); - case INT96: - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return (record, pos, ignored) -> set(record, pos, unboxed.readBinary()); - default: - throw new UnsupportedOperationException("Unsupported type: " + type); - } - } - - // TODO: Add support for options to avoid the null check - return (record, pos, reuse) -> { - Object obj = reader.read((E) reuse); - if (obj != null) { - set(record, pos, obj); - } else { - setNull(record, pos); - } - }; - } - @SuppressWarnings("unchecked") private E get(I intermediate, int pos) { return (E) getField(intermediate, pos); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java index b2d91c99ef44..4eddf91a182f 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java @@ -147,18 +147,10 @@ private UnboxedWriter(ColumnDescriptor desc) { super(desc); } - public void writeBoolean(int repetitionLevel, boolean value) { - column.writeBoolean(repetitionLevel, value); - } - public void writeInteger(int repetitionLevel, int value) { column.writeInteger(repetitionLevel, value); } - public void writeLong(int repetitionLevel, long value) { - column.writeLong(repetitionLevel, value); - } - public void writeFloat(int repetitionLevel, float value) { column.writeFloat(repetitionLevel, value); } From 4f8dd628a6542ceceb0fceb068b67f75990598af Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 18 Jun 2024 18:01:44 +0200 Subject: [PATCH 0391/1019] Fix JVM locale dependent casing (#10521) --- .../java/org/apache/iceberg/aliyun/oss/OSSURI.java | 3 ++- .../org/apache/iceberg/catalog/TableIdentifier.java | 3 ++- .../iceberg/aws/s3/signer/S3SignerServlet.java | 5 +++-- .../iceberg/aws/s3/signer/TestS3RestSigner.java | 6 +++++- .../org/apache/iceberg/avro/AvroSchemaUtil.java | 3 ++- .../iceberg/encryption/KeyStoreKmsClient.java | 5 +++-- .../org/apache/iceberg/hadoop/TestStaticTable.java | 5 +++-- .../org/apache/iceberg/io/TestBaseTaskWriter.java | 3 ++- .../java/org/apache/iceberg/dell/ecs/EcsURI.java | 5 ++++- .../iceberg/flink/sink/TestIcebergStreamWriter.java | 3 ++- .../iceberg/flink/sink/TestIcebergStreamWriter.java | 3 ++- .../iceberg/flink/sink/TestIcebergStreamWriter.java | 3 ++- .../org/apache/iceberg/hive/CachedClientPool.java | 2 +- .../hive/ql/exec/vector/VectorizedSupport.java | 3 ++- .../apache/iceberg/mr/hive/HiveIcebergMetaHook.java | 3 ++- .../IcebergRecordObjectInspector.java | 3 ++- .../hive/TestHiveIcebergStorageHandlerNoScan.java | 3 ++- .../TestHiveIcebergStorageHandlerWithEngine.java | 13 +++++++------ 18 files changed, 48 insertions(+), 26 deletions(-) diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSURI.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSURI.java index 74b937ac9bf5..2fb8144c45bd 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSURI.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSURI.java @@ -19,6 +19,7 @@ package org.apache.iceberg.aliyun.oss; import com.aliyun.oss.internal.OSSUtils; +import java.util.Locale; import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -64,7 +65,7 @@ public OSSURI(String location) { String scheme = schemeSplit[0]; ValidationException.check( - VALID_SCHEMES.contains(scheme.toLowerCase()), + VALID_SCHEMES.contains(scheme.toLowerCase(Locale.ROOT)), "Invalid scheme: %s in OSS location %s", scheme, location); diff --git a/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java b/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java index 8531fc63615a..cbb5dc8d8fd2 100644 --- a/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java +++ b/api/src/main/java/org/apache/iceberg/catalog/TableIdentifier.java @@ -19,6 +19,7 @@ package org.apache.iceberg.catalog; import java.util.Arrays; +import java.util.Locale; import java.util.Objects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Splitter; @@ -80,7 +81,7 @@ public String name() { public TableIdentifier toLowerCase() { String[] newLevels = Arrays.stream(namespace().levels()).map(String::toLowerCase).toArray(String[]::new); - String newName = name().toLowerCase(); + String newName = name().toLowerCase(Locale.ROOT); return TableIdentifier.of(Namespace.of(newLevels), newName); } 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 bc9fb44988e3..06c099e3be5e 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 @@ -31,6 +31,7 @@ import java.time.ZoneId; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.function.Predicate; @@ -185,12 +186,12 @@ private S3SignResponse signRequest(S3SignRequest request) { Map> unsignedHeaders = request.headers().entrySet().stream() - .filter(e -> UNSIGNED_HEADERS.contains(e.getKey().toLowerCase())) + .filter(e -> UNSIGNED_HEADERS.contains(e.getKey().toLowerCase(Locale.ROOT))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); Map> signedHeaders = request.headers().entrySet().stream() - .filter(e -> !UNSIGNED_HEADERS.contains(e.getKey().toLowerCase())) + .filter(e -> !UNSIGNED_HEADERS.contains(e.getKey().toLowerCase(Locale.ROOT))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); SdkHttpFullRequest sign = 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 67a5d423b80b..5e20b71e438c 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 @@ -23,6 +23,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.ScheduledThreadPoolExecutor; import java.util.stream.Collectors; @@ -327,7 +328,10 @@ private SdkHttpFullRequest signWithAwsSigner( // back after signing Map> unsignedHeaders = request.headers().entrySet().stream() - .filter(e -> S3SignerServlet.UNSIGNED_HEADERS.contains(e.getKey().toLowerCase())) + .filter( + e -> + S3SignerServlet.UNSIGNED_HEADERS.contains( + e.getKey().toLowerCase(Locale.ROOT))) .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); SdkHttpFullRequest.Builder builder = request.toBuilder(); diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java index 89cebf7598b5..57c99faa2073 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.avro; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.function.BiFunction; @@ -528,6 +529,6 @@ private static String sanitize(char character) { if (Character.isDigit(character)) { return "_" + character; } - return "_x" + Integer.toHexString(character).toUpperCase(); + return "_x" + Integer.toHexString(character).toUpperCase(Locale.ROOT); } } diff --git a/core/src/test/java/org/apache/iceberg/encryption/KeyStoreKmsClient.java b/core/src/test/java/org/apache/iceberg/encryption/KeyStoreKmsClient.java index 558118683e55..5d142708c0e8 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/KeyStoreKmsClient.java +++ b/core/src/test/java/org/apache/iceberg/encryption/KeyStoreKmsClient.java @@ -28,6 +28,7 @@ import java.security.UnrecoverableKeyException; import java.security.cert.CertificateException; import java.util.Enumeration; +import java.util.Locale; import java.util.Map; import javax.crypto.SecretKey; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -50,13 +51,13 @@ public class KeyStoreKmsClient extends MemoryMockKMS { @Override public ByteBuffer wrapKey(ByteBuffer key, String wrappingKeyId) { // keytool keeps key names in lower case - return super.wrapKey(key, wrappingKeyId.toLowerCase()); + return super.wrapKey(key, wrappingKeyId.toLowerCase(Locale.ROOT)); } @Override public ByteBuffer unwrapKey(ByteBuffer wrappedKey, String wrappingKeyId) { // keytool keeps key names in lower case - return super.unwrapKey(wrappedKey, wrappingKeyId.toLowerCase()); + return super.unwrapKey(wrappedKey, wrappingKeyId.toLowerCase(Locale.ROOT)); } @Override diff --git a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java index 377ef42ca989..a3626085c336 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestStaticTable.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.util.Locale; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataTableType; import org.apache.iceberg.StaticTableOperations; @@ -113,8 +114,8 @@ public void testImmutable() { @Test public void testMetadataTables() { for (MetadataTableType type : MetadataTableType.values()) { - String enumName = type.name().replace("_", "").toLowerCase(); - assertThat(getStaticTable(type).getClass().getName().toLowerCase()) + String enumName = type.name().replace("_", "").toLowerCase(Locale.ROOT); + assertThat(getStaticTable(type).getClass().getName().toLowerCase(Locale.ROOT)) .as("Should be able to get MetadataTable of type : " + type) .contains(enumName); } 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 2a235f5d7428..85ddeb78f473 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java @@ -27,6 +27,7 @@ import java.nio.file.Paths; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.stream.Collectors; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; @@ -175,7 +176,7 @@ public void testRollIfExceedTargetFileSize() throws IOException { int id = record.get(0, Integer.class); String data = record.get(1, String.class); - Record newRecord = createRecord(id, data.toUpperCase()); + Record newRecord = createRecord(id, data.toUpperCase(Locale.ROOT)); expected.add(newRecord); taskWriter.write(newRecord); } diff --git a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsURI.java b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsURI.java index b782c6c76969..f5d2022e0bda 100644 --- a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsURI.java +++ b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsURI.java @@ -19,6 +19,7 @@ package org.apache.iceberg.dell.ecs; import java.net.URI; +import java.util.Locale; import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -40,7 +41,9 @@ class EcsURI { URI uri = URI.create(location); ValidationException.check( - VALID_SCHEME.contains(uri.getScheme().toLowerCase()), "Invalid ecs location: %s", location); + VALID_SCHEME.contains(uri.getScheme().toLowerCase(Locale.ROOT)), + "Invalid ecs location: %s", + location); this.bucket = uri.getHost(); this.name = uri.getPath().replaceAll("^/*", ""); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 98f51c9001c7..c05f08b82f51 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import org.apache.flink.streaming.api.operators.BoundedOneInput; @@ -201,7 +202,7 @@ private Set scanDataFiles() throws IOException { LocatedFileStatus status = iterators.next(); if (status.isFile()) { Path path = status.getPath(); - if (path.getName().endsWith("." + format.toString().toLowerCase())) { + if (path.getName().endsWith("." + format.toString().toLowerCase(Locale.ROOT))) { paths.add(path.toString()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 98f51c9001c7..c05f08b82f51 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import org.apache.flink.streaming.api.operators.BoundedOneInput; @@ -201,7 +202,7 @@ private Set scanDataFiles() throws IOException { LocatedFileStatus status = iterators.next(); if (status.isFile()) { Path path = status.getPath(); - if (path.getName().endsWith("." + format.toString().toLowerCase())) { + if (path.getName().endsWith("." + format.toString().toLowerCase(Locale.ROOT))) { paths.add(path.toString()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 98f51c9001c7..c05f08b82f51 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -24,6 +24,7 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import org.apache.flink.streaming.api.operators.BoundedOneInput; @@ -201,7 +202,7 @@ private Set scanDataFiles() throws IOException { LocatedFileStatus status = iterators.next(); if (status.isFile()) { Path path = status.getPath(); - if (path.getName().endsWith("." + format.toString().toLowerCase())) { + if (path.getName().endsWith("." + format.toString().toLowerCase(Locale.ROOT))) { paths.add(path.toString()); } } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java index b3501db0fbb4..9ce123943fca 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/CachedClientPool.java @@ -148,7 +148,7 @@ static Key extractKey(String cacheKeys, Configuration conf) { !confElements.containsKey(key), "Conf key element %s already specified", key); confElements.put(key, conf.get(key)); } else { - KeyElementType type = KeyElementType.valueOf(trimmed.toUpperCase()); + KeyElementType type = KeyElementType.valueOf(trimmed.toUpperCase(Locale.ROOT)); switch (type) { case UGI: case USER_NAME: diff --git a/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java b/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java index aa3ae709f69c..6558f79b93df 100644 --- a/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java +++ b/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java @@ -18,6 +18,7 @@ */ package org.apache.hadoop.hive.ql.exec.vector; +import java.util.Locale; import java.util.Map; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -30,7 +31,7 @@ public enum Support { final String lowerCaseName; Support() { - this.lowerCaseName = name().toLowerCase(); + this.lowerCaseName = name().toLowerCase(Locale.ROOT); } public static final Map nameToSupportMap = Maps.newHashMap(); diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 97c93955bc7d..49f5b86aa031 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -19,6 +19,7 @@ package org.apache.iceberg.mr.hive; import java.util.List; +import java.util.Locale; import java.util.Properties; import java.util.Set; import org.apache.hadoop.conf.Configuration; @@ -84,7 +85,7 @@ public void preCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) .getParameters() .put( BaseMetastoreTableOperations.TABLE_TYPE_PROP, - BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase()); + BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ROOT)); if (!Catalogs.hiveCatalog(conf, catalogProperties)) { // For non-HiveCatalog tables too, we should set the input and output format diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java b/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java index b5204068c726..aaa09e51cf23 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergRecordObjectInspector.java @@ -20,6 +20,7 @@ import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.stream.Collectors; import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; @@ -52,7 +53,7 @@ public IcebergRecordObjectInspector( Types.NestedField.of( field.fieldId(), field.isOptional(), - field.name().toLowerCase(), + field.name().toLowerCase(Locale.ROOT), field.type(), field.doc()); IcebergRecordStructField structField = diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java index 328b9f3b5b95..6a297e4913e4 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java @@ -28,6 +28,7 @@ import java.util.Collection; import java.util.Collections; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -785,7 +786,7 @@ public void testIcebergAndHmsTableProperties() throws Exception { hive_metastoreConstants.META_TABLE_STORAGE, HiveIcebergStorageHandler.class.getName()) .containsEntry( BaseMetastoreTableOperations.TABLE_TYPE_PROP, - BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase()) + BaseMetastoreTableOperations.ICEBERG_TABLE_TYPE_VALUE.toUpperCase(Locale.ROOT)) .containsEntry( BaseMetastoreTableOperations.METADATA_LOCATION_PROP, getCurrentSnapshotForHiveCatalogTable(icebergTable)) diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java index b080f4bd49f4..18fc4f305070 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java @@ -29,6 +29,7 @@ import java.time.LocalDateTime; import java.util.Collection; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; @@ -311,8 +312,8 @@ public void testJoinTablesSupportedTypes() throws IOException { if (type == Types.UUIDType.get() && fileFormat == FileFormat.PARQUET) { continue; } - String tableName = type.typeId().toString().toLowerCase() + "_table_" + i; - String columnName = type.typeId().toString().toLowerCase() + "_column"; + String tableName = type.typeId().toString().toLowerCase(Locale.ROOT) + "_table_" + i; + String columnName = type.typeId().toString().toLowerCase(Locale.ROOT) + "_column"; Schema schema = new Schema(required(1, columnName, type)); List records = TestHelper.generateRandomRecords(schema, 1, 0L); @@ -349,8 +350,8 @@ public void testSelectDistinctFromTable() throws IOException { if (type == Types.UUIDType.get() && fileFormat == FileFormat.PARQUET) { continue; } - String tableName = type.typeId().toString().toLowerCase() + "_table_" + i; - String columnName = type.typeId().toString().toLowerCase() + "_column"; + String tableName = type.typeId().toString().toLowerCase(Locale.ROOT) + "_table_" + i; + String columnName = type.typeId().toString().toLowerCase(Locale.ROOT) + "_column"; Schema schema = new Schema(required(1, columnName, type)); List records = TestHelper.generateRandomRecords(schema, 4, 0L); @@ -411,7 +412,7 @@ public void testInsertSupportedTypes() throws IOException { if (type.equals(Types.BinaryType.get()) || type.equals(Types.FixedType.ofLength(5))) { continue; } - String columnName = type.typeId().toString().toLowerCase() + "_column"; + String columnName = type.typeId().toString().toLowerCase(Locale.ROOT) + "_column"; Schema schema = new Schema(required(1, "id", Types.LongType.get()), required(2, columnName, type)); @@ -420,7 +421,7 @@ public void testInsertSupportedTypes() throws IOException { Table table = testTables.createTable( shell, - type.typeId().toString().toLowerCase() + "_table_" + i, + type.typeId().toString().toLowerCase(Locale.ROOT) + "_table_" + i, schema, PartitionSpec.unpartitioned(), fileFormat, From cdcc68c472bccff76b3e375f8fe1b83e0c9cc209 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 19 Jun 2024 03:40:51 +0200 Subject: [PATCH 0392/1019] Fix code depending on JVM default charset (#10529) --- .../flink/sink/shuffle/MapRangePartitionerBenchmark.java | 4 +++- .../flink/sink/shuffle/MapRangePartitionerBenchmark.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index c3917165753d..3b2c74fd6ece 100644 --- a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -133,7 +134,8 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + // CHARS is all ASCII + return prefix + new String(buffer, StandardCharsets.US_ASCII); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index c3917165753d..3b2c74fd6ece 100644 --- a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -133,7 +134,8 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + // CHARS is all ASCII + return prefix + new String(buffer, StandardCharsets.US_ASCII); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ From 08b28da9f2372c444b6e02fc62b03074c334fc5b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 19 Jun 2024 11:05:29 +0200 Subject: [PATCH 0393/1019] Run Flink tests on Java 17 (#10477) Project supports building with 8, 11 and 17. In most CI scripts we run the jobs on all supported Java versions, let's do same here to ensure the build would work locally too. --- .github/workflows/flink-ci.yml | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 0791f5b73337..e360e0f2934b 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -71,8 +71,12 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11] + jvm: [8, 11, 17] flink: ['1.17', '1.18', '1.19'] + exclude: + # Flink 1.17 does not support Java 17. + - jvm: 17 + flink: '1.17' env: SPARK_LOCAL_IP: localhost steps: From 94a317a63c2f81753e9fb4be4829dae55a046bbe Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 19 Jun 2024 18:53:27 +0200 Subject: [PATCH 0394/1019] Run Hive3 tests on Java 11 and 17 too (#10482) Project supports building with 8, 11 and 17. In most CI scripts we run the jobs on all supported Java versions, let's do same here to ensure the build would work locally too. --- .github/workflows/hive-ci.yml | 5 ++++- settings.gradle | 13 +++++-------- 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index ffa49f6da3db..cd201a09d0fa 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -94,6 +94,9 @@ jobs: hive3-tests: runs-on: ubuntu-22.04 + strategy: + matrix: + jvm: [8, 11, 17] env: SPARK_LOCAL_IP: localhost steps: @@ -101,7 +104,7 @@ jobs: - uses: actions/setup-java@v4 with: distribution: zulu - java-version: 8 + java-version: ${{ matrix.jvm }} - uses: actions/cache@v4 with: path: | diff --git a/settings.gradle b/settings.gradle index 2fa05fb46d0d..46c85fb65546 100644 --- a/settings.gradle +++ b/settings.gradle @@ -184,14 +184,11 @@ if (hiveVersions.contains("2") || hiveVersions.contains("3")) { project(':hive-runtime').name = 'iceberg-hive-runtime' } -if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - if (hiveVersions.contains("3")) { - include 'hive3' - include 'hive3-orc-bundle' - - project(':hive3').name = 'iceberg-hive3' - project(':hive3-orc-bundle').name = 'iceberg-hive3-orc-bundle' - } +if (hiveVersions.contains("3")) { + include 'hive3' + include 'hive3-orc-bundle' + project(':hive3').name = 'iceberg-hive3' + project(':hive3-orc-bundle').name = 'iceberg-hive3-orc-bundle' } include ":iceberg-kafka-connect:kafka-connect-events" From 106dda92842578658b8713e6bc5ce26fd808ade2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 20 Jun 2024 09:44:13 +0200 Subject: [PATCH 0395/1019] Core: Prevent duplicate data/delete files (#10007) --- .../java/org/apache/iceberg/FastAppend.java | 12 +- .../iceberg/MergingSnapshotProducer.java | 21 +- .../TestBaseIncrementalAppendScan.java | 26 +- .../org/apache/iceberg/TestFastAppend.java | 7 + .../org/apache/iceberg/TestMergeAppend.java | 7 + .../org/apache/iceberg/TestRewriteFiles.java | 6 +- .../apache/iceberg/TestSnapshotSummary.java | 275 +++++++++++++++++- 7 files changed, 321 insertions(+), 33 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 14e776a92d6a..1439289130d7 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -30,6 +30,7 @@ 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.Lists; +import org.apache.iceberg.util.CharSequenceSet; /** * {@link AppendFiles Append} implementation that adds a new manifest file for the write. @@ -43,6 +44,7 @@ class FastAppend extends SnapshotProducer implements AppendFiles { private final PartitionSpec spec; private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder(); private final List newFiles = Lists.newArrayList(); + private final CharSequenceSet newFilePaths = CharSequenceSet.empty(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private List newManifests = null; @@ -83,9 +85,13 @@ protected Map summary() { @Override public FastAppend appendFile(DataFile file) { - this.hasNewFiles = true; - newFiles.add(file); - summaryBuilder.addedFile(spec, file); + Preconditions.checkNotNull(file, "Invalid data file: null"); + if (newFilePaths.add(file.path())) { + this.hasNewFiles = true; + newFiles.add(file); + summaryBuilder.addedFile(spec, file); + } + return this; } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index c1dc6b58b784..1a4560416dc2 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -80,6 +80,8 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { // update data private final List newDataFiles = Lists.newArrayList(); + private final CharSequenceSet newDataFilePaths = CharSequenceSet.empty(); + private final CharSequenceSet newDeleteFilePaths = CharSequenceSet.empty(); private Long newDataFilesDataSequenceNumber; private final Map> newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); @@ -220,10 +222,12 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - setDataSpec(file); - addedFilesSummary.addedFile(dataSpec(), file); - hasNewDataFiles = true; - newDataFiles.add(file); + if (newDataFilePaths.add(file.path())) { + setDataSpec(file); + addedFilesSummary.addedFile(dataSpec(), file); + hasNewDataFiles = true; + newDataFiles.add(file); + } } /** Add a delete file to the new snapshot. */ @@ -243,9 +247,12 @@ private void add(DeleteFileHolder fileHolder) { PartitionSpec fileSpec = ops.current().spec(specId); List deleteFiles = newDeleteFilesBySpec.computeIfAbsent(specId, s -> Lists.newArrayList()); - deleteFiles.add(fileHolder); - addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); - hasNewDeleteFiles = true; + + if (newDeleteFilePaths.add(fileHolder.deleteFile().path())) { + deleteFiles.add(fileHolder); + addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); + hasNewDeleteFiles = true; + } } private void setDataSpec(DataFile file) { diff --git a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java index 7b011b9134ab..119131a95a98 100644 --- a/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java +++ b/core/src/test/java/org/apache/iceberg/TestBaseIncrementalAppendScan.java @@ -67,13 +67,13 @@ public void fromSnapshotInclusiveWithTag() { table.manageSnapshots().createTag(tagSnapshotAName, snapshotAId).commit(); String tagSnapshotBName = "t2"; - table.newFastAppend().appendFile(FILE_B).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); long snapshotBId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag(tagSnapshotBName, snapshotBId).commit(); - table.newFastAppend().appendFile(FILE_C).appendFile(FILE_C).commit(); + table.newFastAppend().appendFile(FILE_D).appendFile(FILE_A2).commit(); /* - files:FILE_A files:FILE_B FILE_B files:FILE_C FILE_C + files:FILE_A files:FILE_B FILE_C files:FILE_D FILE_A2 ---- snapshotAId(tag:t1) ---- snapshotMainB(tag:t2) ---- currentSnapshot */ IncrementalAppendScan scan = newScan().fromSnapshotInclusive(tagSnapshotAName); @@ -111,11 +111,11 @@ public void testUseBranch() { table.manageSnapshots().createTag(tagSnapshotAName, snapshotAId).commit(); String tagName2 = "t2"; - table.newFastAppend().appendFile(FILE_B).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); long snapshotMainBId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag(tagName2, snapshotMainBId).commit(); - table.newFastAppend().appendFile(FILE_B).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_D).appendFile(FILE_A2).commit(); table.newFastAppend().appendFile(FILE_C).toBranch(branchName).commit(); long snapshotBranchBId = table.snapshot(branchName).snapshotId(); @@ -125,7 +125,7 @@ public void testUseBranch() { /* - files:FILE_A files:FILE_B FILE_B files:FILE_B FILE_B + files:FILE_A files:FILE_B FILE_C files:FILE_D FILE_A2 ---- snapshotA(tag:t1) ---- snapshotMainB(tag:t2) ---- currentSnapshot \ \ @@ -175,21 +175,21 @@ public void testUseBranchWithInvalidSnapshotShouldFail() { table.newFastAppend().appendFile(FILE_A).commit(); long snapshotAId = table.currentSnapshot().snapshotId(); - table.newFastAppend().appendFile(FILE_B).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); long snapshotMainBId = table.currentSnapshot().snapshotId(); String branchName = "b1"; table.manageSnapshots().createBranch(branchName, snapshotAId).commit(); - table.newFastAppend().appendFile(FILE_C).toBranch(branchName).commit(); + table.newFastAppend().appendFile(FILE_D).toBranch(branchName).commit(); long snapshotBranchBId = table.snapshot(branchName).snapshotId(); /* - files:FILE_A files:FILE_B FILE_B + files:FILE_A files:FILE_B FILE_C ---- snapshotA ------ snapshotMainB \ \ - \files:FILE_C + \files:FILE_D snapshotBranchB(branch:b1) */ assertThatThrownBy( @@ -267,13 +267,13 @@ public void testFromSnapshotExclusiveWithTag() { table.manageSnapshots().createTag(tagSnapshotAName, snapshotAId).commit(); String tagSnapshotBName = "t2"; - table.newFastAppend().appendFile(FILE_B).appendFile(FILE_B).commit(); + table.newFastAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); long snapshotBId = table.currentSnapshot().snapshotId(); table.manageSnapshots().createTag(tagSnapshotBName, snapshotBId).commit(); - table.newFastAppend().appendFile(FILE_C).appendFile(FILE_C).commit(); + table.newFastAppend().appendFile(FILE_D).appendFile(FILE_A2).commit(); /* - files:FILE_A files:FILE_B FILE_B files:FILE_C FILE_C + files:FILE_A files:FILE_B FILE_C files:FILE_D FILE_A2 ---- snapshotAId(tag:t1) ---- snapshotMainB(tag:t2) ---- currentSnapshot */ IncrementalAppendScan scan = newScan().fromSnapshotExclusive(tagSnapshotAName); diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index c3fc710ebffb..9dd479ecf0fc 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -42,6 +42,13 @@ protected static List parameters() { return Arrays.asList(1, 2); } + @TestTemplate + public void appendNullFile() { + assertThatThrownBy(() -> table.newFastAppend().appendFile(null).commit()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid data file: null"); + } + @TestTemplate public void testEmptyTableAppend() { assertThat(listManifestFiles()).isEmpty(); diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 17d6bd5a19ba..4719923e72ac 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -53,6 +53,13 @@ protected static List parameters() { new Object[] {2, "testBranch"}); } + @TestTemplate + public void appendNullFile() { + assertThatThrownBy(() -> table.newAppend().appendFile(null).commit()) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid data file: null"); + } + @TestTemplate public void testEmptyTableAppend() { assertThat(listManifestFiles()).isEmpty(); diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java index 948eda552817..124cc2f28dd5 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteFiles.java @@ -194,9 +194,9 @@ public void testDeleteWithDuplicateEntriesInManifest() { validateManifestEntries( pending.allManifests(table.io()).get(1), - ids(pendingId, pendingId, baseSnapshotId), - files(FILE_A, FILE_A, FILE_B), - statuses(DELETED, DELETED, EXISTING)); + ids(pendingId, baseSnapshotId), + files(FILE_A, FILE_B), + statuses(DELETED, EXISTING)); // We should only get the 3 manifests that this test is expected to add. assertThat(listManifestFiles()).hasSize(3); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 23982c510dfa..053a9c374178 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -19,6 +19,7 @@ package org.apache.iceberg; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.util.Arrays; import java.util.List; @@ -60,27 +61,24 @@ public void testFileSizeSummary() { .deleteFile(FILE_B) .addFile(FILE_C) .addFile(FILE_D) - .addFile(FILE_D) .commit(); summary = table.currentSnapshot().summary(); assertThat(summary) - .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "30") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "20") .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "20") - .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "30"); + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "20"); table.newDelete().deleteFile(FILE_C).deleteFile(FILE_D).commit(); summary = table.currentSnapshot().summary(); assertThat(summary) .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "20") - .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "0") .doesNotContainKey(SnapshotSummary.ADDED_FILE_SIZE_PROP); } @TestTemplate public void testFileSizeSummaryWithDeletes() { - if (formatVersion == 1) { - return; - } + assumeThat(formatVersion).isGreaterThan(1); table.newRowDelta().addDeletes(FILE_A_DELETES).addDeletes(FILE_A2_DELETES).commit(); @@ -97,4 +95,267 @@ public void testIcebergVersionInSummary() { Map summary = table.currentSnapshot().summary(); assertThat(summary).containsKey("iceberg-version"); } + + @TestTemplate + public void fastAppendWithDuplicates() { + assertThat(listManifestFiles()).isEmpty(); + + table + .newFastAppend() + .appendFile(FILE_A) + .appendFile(DataFiles.builder(SPEC).copy(FILE_A).build()) + .appendFile(FILE_A) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(11) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void mergeAppendWithDuplicates() { + assertThat(listManifestFiles()).isEmpty(); + + table + .newAppend() + .appendFile(FILE_A) + .appendFile(DataFiles.builder(SPEC).copy(FILE_A).build()) + .appendFile(FILE_A) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(11) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void overwriteWithDuplicates() { + assertThat(listManifestFiles()).isEmpty(); + table.newFastAppend().appendFile(FILE_A).commit(); + + table + .newOverwrite() + .deleteFile(FILE_A) + .deleteFile(DataFiles.builder(SPEC).copy(FILE_A).build()) + .deleteFile(FILE_A) + .addFile(FILE_C) + .addFile(DataFiles.builder(SPEC).copy(FILE_C).build()) + .addFile(FILE_C) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(14) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "2") + .containsEntry(SnapshotSummary.DELETED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.DELETED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void deleteWithDuplicates() { + assertThat(listManifestFiles()).isEmpty(); + table.newFastAppend().appendFile(FILE_C).appendFile(FILE_D).commit(); + + table + .newDelete() + .deleteFile(FILE_C) + .deleteFile(DataFiles.builder(SPEC).copy(FILE_C).build()) + .deleteFile(FILE_C) + .deleteFile(FILE_D) + .deleteFile(DataFiles.builder(SPEC).copy(FILE_D).build()) + .deleteFile(FILE_D) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(11) + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "2") + .containsEntry(SnapshotSummary.DELETED_FILES_PROP, "2") + .containsEntry(SnapshotSummary.DELETED_RECORDS_PROP, "2") + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "20") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "0"); + } + + @TestTemplate + public void replacePartitionsWithDuplicates() { + assertThat(listManifestFiles()).isEmpty(); + + table + .newReplacePartitions() + .addFile(FILE_A) + .addFile(DataFiles.builder(SPEC).copy(FILE_A).build()) + .addFile(FILE_A) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(12) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry(SnapshotSummary.REPLACE_PARTITIONS_PROP, "true") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void rowDeltaWithDuplicates() { + assertThat(listManifestFiles()).isEmpty(); + + table + .newRowDelta() + .addRows(FILE_A) + .addRows(DataFiles.builder(SPEC).copy(FILE_A).build()) + .addRows(FILE_A) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(11) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void rowDeltaWithDeletesAndDuplicates() { + assumeThat(formatVersion).isGreaterThan(1); + assertThat(listManifestFiles()).isEmpty(); + + table + .newRowDelta() + .addRows(FILE_A) + .addRows(DataFiles.builder(SPEC).copy(FILE_A).build()) + .addRows(FILE_A) + .addDeletes(FILE_A_DELETES) + .addDeletes(FileMetadata.deleteFileBuilder(SPEC).copy(FILE_A_DELETES).build()) + .addDeletes(FILE_A_DELETES) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(14) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "20") // size of data + delete file + .containsEntry(SnapshotSummary.ADD_POS_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_POS_DELETES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "20") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void rewriteWithDuplicateFiles() { + assertThat(listManifestFiles()).isEmpty(); + + table.newAppend().appendFile(FILE_A2).appendFile(FILE_A2).appendFile(FILE_A2).commit(); + + table + .newRewrite() + .deleteFile(FILE_A2) + .deleteFile(DataFiles.builder(SPEC).copy(FILE_A2).build()) + .deleteFile(FILE_A2) + .addFile(FILE_A) + .addFile(DataFiles.builder(SPEC).copy(FILE_A).build()) + .addFile(FILE_A) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(14) + .containsEntry(SnapshotSummary.ADDED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADDED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "1") + .containsEntry(SnapshotSummary.DELETED_FILES_PROP, "1") + .containsEntry(SnapshotSummary.DELETED_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } + + @TestTemplate + public void rewriteWithDeletesAndDuplicates() { + assumeThat(formatVersion).isGreaterThan(1); + assertThat(listManifestFiles()).isEmpty(); + + table.newRowDelta().addRows(FILE_A2).addDeletes(FILE_A_DELETES).commit(); + + table + .newRewrite() + .deleteFile(FILE_A_DELETES) + .deleteFile(FileMetadata.deleteFileBuilder(SPEC).copy(FILE_A_DELETES).build()) + .deleteFile(FILE_A_DELETES) + .addFile(FILE_B_DELETES) + .addFile(FileMetadata.deleteFileBuilder(SPEC).copy(FILE_B_DELETES).build()) + .addFile(FILE_B_DELETES) + .commit(); + + assertThat(table.currentSnapshot().summary()) + .hasSize(16) + .containsEntry(SnapshotSummary.ADDED_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.ADD_POS_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.ADDED_POS_DELETES_PROP, "1") + .containsEntry(SnapshotSummary.CHANGED_PARTITION_COUNT_PROP, "2") + .containsEntry(SnapshotSummary.REMOVED_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_FILE_SIZE_PROP, "10") + .containsEntry(SnapshotSummary.REMOVED_POS_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.REMOVED_POS_DELETES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DELETE_FILES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_EQ_DELETES_PROP, "0") + .containsEntry(SnapshotSummary.TOTAL_POS_DELETES_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_FILE_SIZE_PROP, "20") + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1"); + } } From 774080538495193d08b99ae4a4e66bd2d0649c6e Mon Sep 17 00:00:00 2001 From: Qishang Zhong Date: Thu, 20 Jun 2024 18:26:45 +0800 Subject: [PATCH 0396/1019] Flink: Fix the condition of `formatVersion` for skipping test cases (#10541) --- .../apache/iceberg/flink/sink/TestIcebergFilesCommitter.java | 4 ++-- .../apache/iceberg/flink/sink/TestIcebergFilesCommitter.java | 4 ++-- .../apache/iceberg/flink/sink/TestIcebergFilesCommitter.java | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java index 06b6bc9a977b..948c7b31430c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -766,7 +766,7 @@ public void testFlinkManifests() throws Exception { public void testDeleteFiles() throws Exception { assumeThat(formatVersion) .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); + .isGreaterThan(1); long timestamp = 0; long checkpoint = 10; @@ -837,7 +837,7 @@ public void testDeleteFiles() throws Exception { public void testCommitTwoCheckpointsInSingleTxn() throws Exception { assumeThat(formatVersion) .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); + .isGreaterThan(1); long timestamp = 0; long checkpoint = 10; 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 06b6bc9a977b..948c7b31430c 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 @@ -766,7 +766,7 @@ public void testFlinkManifests() throws Exception { public void testDeleteFiles() throws Exception { assumeThat(formatVersion) .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); + .isGreaterThan(1); long timestamp = 0; long checkpoint = 10; @@ -837,7 +837,7 @@ public void testDeleteFiles() throws Exception { public void testCommitTwoCheckpointsInSingleTxn() throws Exception { assumeThat(formatVersion) .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); + .isGreaterThan(1); long timestamp = 0; long checkpoint = 10; 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 06b6bc9a977b..948c7b31430c 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 @@ -766,7 +766,7 @@ public void testFlinkManifests() throws Exception { public void testDeleteFiles() throws Exception { assumeThat(formatVersion) .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); + .isGreaterThan(1); long timestamp = 0; long checkpoint = 10; @@ -837,7 +837,7 @@ public void testDeleteFiles() throws Exception { public void testCommitTwoCheckpointsInSingleTxn() throws Exception { assumeThat(formatVersion) .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(2); + .isGreaterThan(1); long timestamp = 0; long checkpoint = 10; From 1326cdcd2cda6fb3aaddc13376b74b4d053a678f Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 20 Jun 2024 15:36:38 +0200 Subject: [PATCH 0397/1019] Build: Sort error-prone configuration options (#10540) This makes it clear where new config option should be added (at the end, grouped semantically, etc.) --- baseline.gradle | 49 +++++++++++++++++++++++++++---------------------- 1 file changed, 27 insertions(+), 22 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index dab3cab0d9dd..c890ada8de57 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -73,39 +73,44 @@ subprojects { options.errorprone.errorproneArgs.addAll ( // error-prone is slow, don't run on tests/generated-src/generated '-XepExcludedPaths:.*/(test|generated-src|generated)/.*', - // specific to Palantir - '-Xep:ConsistentLoggerName:OFF', // Uses name `log` but we use name `LOG` - '-Xep:FinalClass:OFF', - '-Xep:PreferSafeLoggingPreconditions:OFF', - '-Xep:PreferSafeLoggableExceptions:OFF', - '-Xep:Slf4jLogsafeArgs:OFF', - '-Xep:RawTypes:OFF', - // enforce logging conventions - '-Xep:LoggerEnclosingClass:ERROR', - '-Xep:PreferStaticLoggers:ERROR', - '-Xep:Slf4jThrowable:ERROR', + '-Xep:AnnotateFormatMethod:ERROR', + '-Xep:CollectionUndefinedEquality:ERROR', + // specific to Palantir - Uses name `log` but we use name `LOG` + '-Xep:ConsistentLoggerName:OFF', + '-Xep:DangerousThreadPoolExecutorUsage:OFF', // subclasses are not equal '-Xep:EqualsGetClass:OFF', + // specific to Palantir + '-Xep:FinalClass:OFF', + '-Xep:IntLongMath:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', + // enforce logging conventions + '-Xep:LoggerEnclosingClass:ERROR', // patterns that are allowed '-Xep:MissingCasesInEnumSwitch:OFF', - //Added because it errors out compile, but we need to figure out if we want it - '-Xep:StrictUnusedVariable:OFF', - '-Xep:TypeParameterShadowing:OFF', - '-Xep:TypeParameterUnusedInFormals:OFF', - '-Xep:DangerousThreadPoolExecutorUsage:OFF', + // Enforce missing override + '-Xep:MissingOverride:ERROR', + '-Xep:MissingSummary:ERROR', // Enforce hashCode over hash '-Xep:ObjectsHashCodeUnnecessaryVarargs:ERROR', + // specific to Palantir + '-Xep:PreferSafeLoggableExceptions:OFF', '-Xep:PreferSafeLogger:OFF', + // specific to Palantir + '-Xep:PreferSafeLoggingPreconditions:OFF', + '-Xep:PreferStaticLoggers:ERROR', + // specific to Palantir + '-Xep:RawTypes:OFF', + // specific to Palantir + '-Xep:Slf4jLogsafeArgs:OFF', + '-Xep:Slf4jThrowable:ERROR', + // Added because it errors out compile, but we need to figure out if we want it + '-Xep:StrictUnusedVariable:OFF', // Enforce safe string splitting '-Xep:StringSplitter:ERROR', - // Enforce missing override - '-Xep:MissingOverride:ERROR', - '-Xep:IntLongMath:ERROR', - '-Xep:MissingSummary:ERROR', - '-Xep:AnnotateFormatMethod:ERROR', - '-Xep:CollectionUndefinedEquality:ERROR', + '-Xep:TypeParameterShadowing:OFF', + '-Xep:TypeParameterUnusedInFormals:OFF', ) } } From 281de28d11a6efc2d419d7f9c83d5eec1644f9c1 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 20 Jun 2024 08:27:59 -0700 Subject: [PATCH 0398/1019] Core, Spark: Spark writes/actions should only perform cleanup if failure is cleanable (#10373) --- .../actions/BaseRewriteDataFilesAction.java | 16 ++++-- .../RewriteDataFilesCommitManager.java | 9 ++- .../RewritePositionDeletesCommitManager.java | 9 ++- .../actions/TestRewriteDataFilesAction.java | 56 +++++++++++++++++-- .../actions/TestRewriteDataFilesAction.java | 56 +++++++++++++++++-- .../actions/RewriteManifestsSparkAction.java | 8 ++- .../spark/source/SparkPositionDeltaWrite.java | 10 ++-- .../iceberg/spark/source/SparkWrite.java | 10 ++-- .../actions/TestRewriteDataFilesAction.java | 56 +++++++++++++++++-- 9 files changed, 193 insertions(+), 37 deletions(-) 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 5f229be579b7..c0f2fc6174b3 100644 --- a/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java +++ b/core/src/main/java/org/apache/iceberg/actions/BaseRewriteDataFilesAction.java @@ -31,6 +31,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -308,12 +309,15 @@ private void replaceDataFiles( LOG.warn("Commit state unknown, cannot clean up files that may have been committed", e); throw e; } catch (Exception e) { - LOG.warn("Failed to commit rewrite, cleaning up rewritten files", e); - Tasks.foreach(Iterables.transform(addedDataFiles, f -> f.path().toString())) - .noRetry() - .suppressFailureWhenFinished() - .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) - .run(fileIO::deleteFile); + if (e instanceof CleanableFailure) { + LOG.warn("Failed to commit rewrite, cleaning up rewritten files", e); + Tasks.foreach(Iterables.transform(addedDataFiles, f -> f.path().toString())) + .noRetry() + .suppressFailureWhenFinished() + .onFailure((location, exc) -> LOG.warn("Failed to delete: {}", location, exc)) + .run(fileIO::deleteFile); + } + throw e; } } 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 7f89db467d73..45b4bcf0a4d9 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java @@ -23,6 +23,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -119,8 +120,12 @@ public void commitOrClean(Set rewriteGroups) { e); throw e; } catch (Exception e) { - LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e); - rewriteGroups.forEach(this::abortFileGroup); + if (e instanceof CleanableFailure) { + LOG.error( + "Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e); + rewriteGroups.forEach(this::abortFileGroup); + } + throw e; } } 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 01b2f7528ee3..b1322d5e58b4 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesCommitManager.java @@ -24,6 +24,7 @@ import org.apache.iceberg.DeleteFile; import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -102,8 +103,12 @@ public void commitOrClean(Set rewriteGroups) { e); throw e; } catch (Exception e) { - LOG.error("Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e); - rewriteGroups.forEach(this::abort); + if (e instanceof CleanableFailure) { + LOG.error( + "Cannot commit groups {}, attempting to clean up written files", rewriteGroups, e); + rewriteGroups.forEach(this::abort); + } + throw e; } } 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 2934494bbab7..c978be9e3d91 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 @@ -77,6 +77,7 @@ import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -673,14 +674,14 @@ public void testSingleCommitWithCommitFailure() { RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); // Fail to commit - doThrow(new RuntimeException("Commit Failure")).when(util).commitFileGroups(any()); + doThrow(new CommitFailedException("Commit Failure")).when(util).commitFileGroups(any()); doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); assertThatThrownBy(() -> spyRewrite.execute()) .as("Should fail entire rewrite if commit fails") .isInstanceOf(RuntimeException.class) - .hasMessage("Commit Failure"); + .hasMessageContaining("Cannot commit rewrite"); table.refresh(); @@ -692,6 +693,40 @@ public void testSingleCommitWithCommitFailure() { shouldHaveACleanCache(table); } + @Test + public void testCommitFailsWithUncleanableFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)); + + RewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // Fail to commit with an arbitrary failure and validate that orphans are not cleaned up + doThrow(new RuntimeException("Arbitrary Failure")).when(util).commitFileGroups(any()); + + doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Arbitrary Failure"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveOrphans(table); + shouldHaveACleanCache(table); + } + @Test public void testParallelSingleCommitWithRewriteFailure() { Table table = createTable(20); @@ -709,13 +744,13 @@ public void testParallelSingleCommitWithRewriteFailure() { // Fail groups 1, 3, and 7 during rewrite GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); - doThrow(new RuntimeException("Rewrite Failed")) + doThrow(new CommitFailedException("Rewrite Failed")) .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); assertThatThrownBy(() -> spyRewrite.execute()) .as("Should fail entire rewrite if part fails") - .isInstanceOf(RuntimeException.class) + .isInstanceOf(CommitFailedException.class) .hasMessage("Rewrite Failed"); table.refresh(); @@ -830,7 +865,7 @@ public void testParallelPartialProgressWithCommitFailure() { // First and Third commits work, second does not doCallRealMethod() - .doThrow(new RuntimeException("Commit Failed")) + .doThrow(new CommitFailedException("Commit Failed")) .doCallRealMethod() .when(util) .commitFileGroups(any()); @@ -1562,6 +1597,17 @@ protected void shouldHaveNoOrphans(Table table) { .orphanFileLocations()); } + protected void shouldHaveOrphans(Table table) { + assertThat( + actions() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()) + .as("Should have found orphan files") + .isNotEmpty(); + } + protected void shouldHaveACleanCache(Table table) { Assert.assertEquals( "Should not have any entries in cache", ImmutableSet.of(), cacheContents(table)); 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 66f16437f5dc..ba173d02498e 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 @@ -77,6 +77,7 @@ import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -707,13 +708,13 @@ public void testSingleCommitWithCommitFailure() { RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); // Fail to commit - doThrow(new RuntimeException("Commit Failure")).when(util).commitFileGroups(any()); + doThrow(new CommitFailedException("Commit Failure")).when(util).commitFileGroups(any()); doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) - .hasMessage("Commit Failure"); + .hasMessageContaining("Cannot commit rewrite"); table.refresh(); @@ -725,6 +726,40 @@ public void testSingleCommitWithCommitFailure() { shouldHaveACleanCache(table); } + @Test + public void testCommitFailsWithUncleanableFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)); + + RewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // Fail to commit with an arbitrary failure and validate that orphans are not cleaned up + doThrow(new RuntimeException("Arbitrary Failure")).when(util).commitFileGroups(any()); + + doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Arbitrary Failure"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveOrphans(table); + shouldHaveACleanCache(table); + } + @Test public void testParallelSingleCommitWithRewriteFailure() { Table table = createTable(20); @@ -742,12 +777,12 @@ public void testParallelSingleCommitWithRewriteFailure() { // Fail groups 1, 3, and 7 during rewrite GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); - doThrow(new RuntimeException("Rewrite Failed")) + doThrow(new CommitFailedException("Rewrite Failed")) .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); assertThatThrownBy(spyRewrite::execute) - .isInstanceOf(RuntimeException.class) + .isInstanceOf(CommitFailedException.class) .hasMessage("Rewrite Failed"); table.refresh(); @@ -866,7 +901,7 @@ public void testParallelPartialProgressWithCommitFailure() { // First and Third commits work, second does not doCallRealMethod() - .doThrow(new RuntimeException("Commit Failed")) + .doThrow(new CommitFailedException("Commit Failed")) .doCallRealMethod() .when(util) .commitFileGroups(any()); @@ -1599,6 +1634,17 @@ protected void shouldHaveNoOrphans(Table table) { .orphanFileLocations()); } + protected void shouldHaveOrphans(Table table) { + assertThat( + actions() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()) + .as("Should have found orphan files") + .isNotEmpty(); + } + protected void shouldHaveACleanCache(Table table) { Assert.assertEquals( "Should not have any entries in cache", ImmutableSet.of(), cacheContents(table)); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 8c35c9c7fe96..de66b06924f3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -49,6 +49,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.ImmutableRewriteManifests; import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.OutputFile; @@ -440,8 +441,11 @@ private void replaceManifests( // don't clean up added manifest files, because they may have been successfully committed. throw commitStateUnknownException; } catch (Exception e) { - // delete all new manifests because the rewrite failed - deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + if (e instanceof CleanableFailure) { + // delete all new manifests because the rewrite failed + deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + } + throw e; } } 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 a964f7686394..02c87b53e762 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 @@ -44,7 +44,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.BasePositionDeltaWriter; @@ -105,7 +105,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde private final Context context; private final Map writeProperties; - private boolean cleanupOnAbort = true; + private boolean cleanupOnAbort = false; SparkPositionDeltaWrite( SparkSession spark, @@ -304,9 +304,9 @@ private void commitOperation(SnapshotUpdate operation, String description) { operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); - } catch (CommitStateUnknownException commitStateUnknownException) { - cleanupOnAbort = false; - throw commitStateUnknownException; + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; } } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index d23c473bb46b..e4a0eb700be6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -40,7 +40,7 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.ClusteredDataWriter; @@ -102,7 +102,7 @@ abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { private final SparkWriteRequirements writeRequirements; private final Map writeProperties; - private boolean cleanupOnAbort = true; + private boolean cleanupOnAbort = false; SparkWrite( SparkSession spark, @@ -233,9 +233,9 @@ private void commitOperation(SnapshotUpdate operation, String description) { operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); - } catch (CommitStateUnknownException commitStateUnknownException) { - cleanupOnAbort = false; - throw commitStateUnknownException; + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; } } 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 16082d78f0cd..b67ee87c7d3e 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 @@ -79,6 +79,7 @@ import org.apache.iceberg.encryption.EncryptedFiles; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -725,13 +726,13 @@ public void testSingleCommitWithCommitFailure() { RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); // Fail to commit - doThrow(new RuntimeException("Commit Failure")).when(util).commitFileGroups(any()); + doThrow(new CommitFailedException("Commit Failure")).when(util).commitFileGroups(any()); doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); assertThatThrownBy(spyRewrite::execute) .isInstanceOf(RuntimeException.class) - .hasMessage("Commit Failure"); + .hasMessageContaining("Cannot commit rewrite"); table.refresh(); @@ -743,6 +744,40 @@ public void testSingleCommitWithCommitFailure() { shouldHaveACleanCache(table); } + @Test + public void testCommitFailsWithUncleanableFailure() { + Table table = createTable(20); + int fileSize = averageFileSize(table); + + List originalData = currentData(); + + RewriteDataFilesSparkAction realRewrite = + basicRewrite(table) + .option( + RewriteDataFiles.MAX_FILE_GROUP_SIZE_BYTES, Integer.toString(fileSize * 2 + 1000)); + + RewriteDataFilesSparkAction spyRewrite = spy(realRewrite); + RewriteDataFilesCommitManager util = spy(new RewriteDataFilesCommitManager(table)); + + // Fail to commit with an arbitrary failure and validate that orphans are not cleaned up + doThrow(new RuntimeException("Arbitrary Failure")).when(util).commitFileGroups(any()); + + doReturn(util).when(spyRewrite).commitManager(table.currentSnapshot().snapshotId()); + + assertThatThrownBy(spyRewrite::execute) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("Arbitrary Failure"); + + table.refresh(); + + List postRewriteData = currentData(); + assertEquals("We shouldn't have changed the data", originalData, postRewriteData); + + shouldHaveSnapshots(table, 1); + shouldHaveOrphans(table); + shouldHaveACleanCache(table); + } + @Test public void testParallelSingleCommitWithRewriteFailure() { Table table = createTable(20); @@ -760,12 +795,12 @@ public void testParallelSingleCommitWithRewriteFailure() { // Fail groups 1, 3, and 7 during rewrite GroupInfoMatcher failGroup = new GroupInfoMatcher(1, 3, 7); - doThrow(new RuntimeException("Rewrite Failed")) + doThrow(new CommitFailedException("Rewrite Failed")) .when(spyRewrite) .rewriteFiles(any(), argThat(failGroup)); assertThatThrownBy(spyRewrite::execute) - .isInstanceOf(RuntimeException.class) + .isInstanceOf(CommitFailedException.class) .hasMessage("Rewrite Failed"); table.refresh(); @@ -884,7 +919,7 @@ public void testParallelPartialProgressWithCommitFailure() { // First and Third commits work, second does not doCallRealMethod() - .doThrow(new RuntimeException("Commit Failed")) + .doThrow(new CommitFailedException("Commit Failed")) .doCallRealMethod() .when(util) .commitFileGroups(any()); @@ -1681,6 +1716,17 @@ protected void shouldHaveNoOrphans(Table table) { .isEmpty(); } + protected void shouldHaveOrphans(Table table) { + assertThat( + actions() + .deleteOrphanFiles(table) + .olderThan(System.currentTimeMillis()) + .execute() + .orphanFileLocations()) + .as("Should have found orphan files") + .isNotEmpty(); + } + protected void shouldHaveACleanCache(Table table) { assertThat(cacheContents(table)).as("Should not have any entries in cache").isEmpty(); } From 243f24948b34df8f2f5eebb150a4c4fde7ca71d0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 20 Jun 2024 18:10:14 +0200 Subject: [PATCH 0399/1019] Docs: Allow Java 17 in contribute.md (#10545) Java 17 is equally supported build-time JVM. This syncs the wording between contribute.md and README.md. --- site/docs/contribute.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 2c9add8df28c..d1e16e1c9939 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -84,7 +84,7 @@ settle disagreements or to force a decision. ## Building the Project Locally -Iceberg is built using Gradle with Java 8 or Java 11. +Iceberg is built using Gradle with Java 8, 11, or 17. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` From 9a3ea9d58aa117769feb770eb8e1eeba86f9829b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 21 Jun 2024 07:33:26 -0700 Subject: [PATCH 0400/1019] Spark: Backport #10373 to Spark 3.3 and 3.4 (#10546) --- .../spark/actions/RewriteManifestsSparkAction.java | 8 ++++++-- .../iceberg/spark/source/SparkPositionDeltaWrite.java | 10 +++++----- .../org/apache/iceberg/spark/source/SparkWrite.java | 10 +++++----- .../spark/actions/RewriteManifestsSparkAction.java | 8 ++++++-- .../iceberg/spark/source/SparkPositionDeltaWrite.java | 10 +++++----- .../org/apache/iceberg/spark/source/SparkWrite.java | 10 +++++----- 6 files changed, 32 insertions(+), 24 deletions(-) diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 288b2d417f1a..78eb010b9b9c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -42,6 +42,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.ImmutableRewriteManifests; import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.OutputFile; @@ -345,8 +346,11 @@ private void replaceManifests( // don't clean up added manifest files, because they may have been successfully committed. throw commitStateUnknownException; } catch (Exception e) { - // delete all new manifests because the rewrite failed - deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + if (e instanceof CleanableFailure) { + // delete all new manifests because the rewrite failed + deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + } + throw e; } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index 2d3829243990..8e5aeef6e13c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -43,7 +43,7 @@ import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.BasePositionDeltaWriter; @@ -102,7 +102,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde private final Distribution requiredDistribution; private final SortOrder[] requiredOrdering; - private boolean cleanupOnAbort = true; + private boolean cleanupOnAbort = false; SparkPositionDeltaWrite( SparkSession spark, @@ -284,9 +284,9 @@ private void commitOperation(SnapshotUpdate operation, String description) { operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); - } catch (CommitStateUnknownException commitStateUnknownException) { - cleanupOnAbort = false; - throw commitStateUnknownException; + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; } } } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index c26b7f5f389b..18db2ee2c797 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -40,7 +40,7 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.ClusteredDataWriter; @@ -101,7 +101,7 @@ abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { private final Distribution requiredDistribution; private final SortOrder[] requiredOrdering; - private boolean cleanupOnAbort = true; + private boolean cleanupOnAbort = false; SparkWrite( SparkSession spark, @@ -216,9 +216,9 @@ private void commitOperation(SnapshotUpdate operation, String description) { operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); - } catch (CommitStateUnknownException commitStateUnknownException) { - cleanupOnAbort = false; - throw commitStateUnknownException; + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 8ec3b44f9284..e9edfeb985c7 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -46,6 +46,7 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.actions.ImmutableRewriteManifests; import org.apache.iceberg.actions.RewriteManifests; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.io.OutputFile; @@ -355,8 +356,11 @@ private void replaceManifests( // don't clean up added manifest files, because they may have been successfully committed. throw commitStateUnknownException; } catch (Exception e) { - // delete all new manifests because the rewrite failed - deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + if (e instanceof CleanableFailure) { + // delete all new manifests because the rewrite failed + deleteFiles(Iterables.transform(addedManifests, ManifestFile::path)); + } + throw e; } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java index c5fc8e0b0fe1..808764b31f44 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeltaWrite.java @@ -44,7 +44,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.PositionDelete; -import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.BasePositionDeltaWriter; @@ -105,7 +105,7 @@ class SparkPositionDeltaWrite implements DeltaWrite, RequiresDistributionAndOrde private final Context context; private final Map writeProperties; - private boolean cleanupOnAbort = true; + private boolean cleanupOnAbort = false; SparkPositionDeltaWrite( SparkSession spark, @@ -293,9 +293,9 @@ private void commitOperation(SnapshotUpdate operation, String description) { operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); - } catch (CommitStateUnknownException commitStateUnknownException) { - cleanupOnAbort = false; - throw commitStateUnknownException; + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; } } } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index d879a1f961e5..4db434f65a89 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -40,7 +40,7 @@ import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.ClusteredDataWriter; @@ -102,7 +102,7 @@ abstract class SparkWrite implements Write, RequiresDistributionAndOrdering { private final SparkWriteRequirements writeRequirements; private final Map writeProperties; - private boolean cleanupOnAbort = true; + private boolean cleanupOnAbort = false; SparkWrite( SparkSession spark, @@ -222,9 +222,9 @@ private void commitOperation(SnapshotUpdate operation, String description) { operation.commit(); // abort is automatically called if this fails long duration = System.currentTimeMillis() - start; LOG.info("Committed in {} ms", duration); - } catch (CommitStateUnknownException commitStateUnknownException) { - cleanupOnAbort = false; - throw commitStateUnknownException; + } catch (Exception e) { + cleanupOnAbort = e instanceof CleanableFailure; + throw e; } } From 038a91b744166b9cb7eb99a1a050009a686636b9 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 21 Jun 2024 10:34:13 -0700 Subject: [PATCH 0401/1019] Spark 3.5: Support Aggregate push down for incremental scan (#10538) --- .../spark/source/SparkScanBuilder.java | 89 ++++++++----------- .../spark/source/TestDataSourceOptions.java | 32 +++++-- .../spark/sql/TestAggregatePushDown.java | 50 +++++++++++ 3 files changed, 109 insertions(+), 62 deletions(-) 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 d6f34231ae75..b430e6fca233 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 @@ -37,7 +37,6 @@ import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; import org.apache.iceberg.expressions.AggregateEvaluator; import org.apache.iceberg.expressions.Binder; import org.apache.iceberg.expressions.BoundAggregate; @@ -232,15 +231,8 @@ public boolean pushAggregation(Aggregation aggregation) { return false; } - TableScan scan = table.newScan().includeColumnStats(); - Snapshot snapshot = readSnapshot(); - if (snapshot == null) { - LOG.info("Skipping aggregate pushdown: table snapshot is null"); - return false; - } - scan = scan.useSnapshot(snapshot.snapshotId()); - scan = configureSplitPlanning(scan); - scan = scan.filter(filterExpression()); + org.apache.iceberg.Scan scan = + buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); try (CloseableIterable fileScanTasks = scan.planFiles()) { List tasks = ImmutableList.copyOf(fileScanTasks); @@ -282,11 +274,6 @@ private boolean canPushDownAggregation(Aggregation aggregation) { return false; } - if (readConf.startSnapshotId() != null) { - LOG.info("Skipping aggregate pushdown: incremental scan is not supported"); - return false; - } - // If group by expression is the same as the partition, the statistics information can still // be used to calculate min/max/count, will enable aggregate push down in next phase. // TODO: enable aggregate push down for partition col group by expression @@ -298,17 +285,6 @@ private boolean canPushDownAggregation(Aggregation aggregation) { return true; } - private Snapshot readSnapshot() { - Snapshot snapshot; - if (readConf.snapshotId() != null) { - snapshot = table.snapshot(readConf.snapshotId()); - } else { - snapshot = SnapshotUtil.latestSnapshot(table, readConf.branch()); - } - - return snapshot; - } - private boolean metricsModeSupportsAggregatePushDown(List> aggregates) { MetricsConfig config = MetricsConfig.forTable(table); for (BoundAggregate aggregate : aggregates) { @@ -387,6 +363,18 @@ public Scan build() { } private Scan buildBatchScan() { + Schema expectedSchema = schemaWithMetadataColumns(); + return new SparkBatchQueryScan( + spark, + table, + buildIcebergBatchScan(false /* not include Column Stats */, expectedSchema), + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); + } + + private org.apache.iceberg.Scan buildIcebergBatchScan(boolean withStats, Schema expectedSchema) { Long snapshotId = readConf.snapshotId(); Long asOfTimestamp = readConf.asOfTimestamp(); String branch = readConf.branch(); @@ -427,15 +415,19 @@ private Scan buildBatchScan() { SparkReadOptions.END_TIMESTAMP); if (startSnapshotId != null) { - return buildIncrementalAppendScan(startSnapshotId, endSnapshotId); + return buildIncrementalAppendScan(startSnapshotId, endSnapshotId, withStats, expectedSchema); } else { - return buildBatchScan(snapshotId, asOfTimestamp, branch, tag); + return buildBatchScan(snapshotId, asOfTimestamp, branch, tag, withStats, expectedSchema); } } - private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, String tag) { - Schema expectedSchema = schemaWithMetadataColumns(); - + private org.apache.iceberg.Scan buildBatchScan( + Long snapshotId, + Long asOfTimestamp, + String branch, + String tag, + boolean withStats, + Schema expectedSchema) { BatchScan scan = newBatchScan() .caseSensitive(caseSensitive) @@ -443,6 +435,10 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, .project(expectedSchema) .metricsReporter(metricsReporter); + if (withStats) { + scan = scan.includeColumnStats(); + } + if (snapshotId != null) { scan = scan.useSnapshot(snapshotId); } @@ -459,21 +455,11 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, scan = scan.useRef(tag); } - scan = configureSplitPlanning(scan); - - return new SparkBatchQueryScan( - spark, - table, - scan, - readConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); + return configureSplitPlanning(scan); } - private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) { - Schema expectedSchema = schemaWithMetadataColumns(); - + private org.apache.iceberg.Scan buildIncrementalAppendScan( + long startSnapshotId, Long endSnapshotId, boolean withStats, Schema expectedSchema) { IncrementalAppendScan scan = table .newIncrementalAppendScan() @@ -483,20 +469,15 @@ private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId .project(expectedSchema) .metricsReporter(metricsReporter); + if (withStats) { + scan = scan.includeColumnStats(); + } + if (endSnapshotId != null) { scan = scan.toSnapshot(endSnapshotId); } - scan = configureSplitPlanning(scan); - - return new SparkBatchQueryScan( - spark, - table, - scan, - readConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); + return configureSplitPlanning(scan); } @SuppressWarnings("CyclomaticComplexity") 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 ff6ddea32360..627fe15f2819 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 @@ -57,6 +57,7 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.functions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; @@ -290,29 +291,44 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); // test (1st snapshot, current snapshot] incremental scan. - List result = + Dataset unboundedIncrementalResult = spark .read() .format("iceberg") .option("start-snapshot-id", snapshotIds.get(3).toString()) - .load(tableLocation) + .load(tableLocation); + List result1 = + unboundedIncrementalResult .orderBy("id") .as(Encoders.bean(SimpleRecord.class)) .collectAsList(); - assertThat(result).as("Records should match").isEqualTo(expectedRecords.subList(1, 4)); + assertThat(result1).as("Records should match").isEqualTo(expectedRecords.subList(1, 4)); + assertThat(unboundedIncrementalResult.count()) + .as("Unprocessed count should match record count") + .isEqualTo(3); + + Row row1 = unboundedIncrementalResult.agg(functions.min("id"), functions.max("id")).head(); + assertThat(row1.getInt(0)).as("min value should match").isEqualTo(2); + assertThat(row1.getInt(1)).as("max value should match").isEqualTo(4); // test (2nd snapshot, 3rd snapshot] incremental scan. - Dataset resultDf = + Dataset incrementalResult = spark .read() .format("iceberg") .option("start-snapshot-id", snapshotIds.get(2).toString()) .option("end-snapshot-id", snapshotIds.get(1).toString()) .load(tableLocation); - List result1 = - resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - assertThat(result1).as("Records should match").isEqualTo(expectedRecords.subList(2, 3)); - assertThat(resultDf.count()).as("Unprocessed count should match record count").isEqualTo(1); + List result2 = + incrementalResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(result2).as("Records should match").isEqualTo(expectedRecords.subList(2, 3)); + assertThat(incrementalResult.count()) + .as("Unprocessed count should match record count") + .isEqualTo(1); + + Row row2 = incrementalResult.agg(functions.min("id"), functions.max("id")).head(); + assertThat(row2.getInt(0)).as("min value should match").isEqualTo(3); + assertThat(row2.getInt(1)).as("max value should match").isEqualTo(3); } @TestTemplate 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 05515946c145..7e9bdeec8af0 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 @@ -35,8 +35,13 @@ 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.CatalogTestBase; +import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.TestBase; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.ExplainMode; +import org.apache.spark.sql.functions; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.TestTemplate; @@ -808,4 +813,49 @@ public void testInfinity() { }); assertEquals("min/max/count push down", expected, actual); } + + @TestTemplate + public void testAggregatePushDownForIncrementalScan() { + sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + long snapshotId1 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (4, 7777), (5, 8888)", tableName); + long snapshotId2 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (6, -7777), (7, 8888)", tableName); + long snapshotId3 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (8, 7777), (9, 9999)", tableName); + + Dataset pushdownDs = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.START_SNAPSHOT_ID, snapshotId2) + .option(SparkReadOptions.END_SNAPSHOT_ID, snapshotId3) + .load(tableName) + .agg(functions.min("data"), functions.max("data"), functions.count("data")); + String explain1 = pushdownDs.queryExecution().explainString(ExplainMode.fromString("simple")); + assertThat(explain1).contains("LocalTableScan", "min(data)", "max(data)", "count(data)"); + + List expected1 = Lists.newArrayList(); + expected1.add(new Object[] {-7777, 8888, 2L}); + assertEquals("min/max/count push down", expected1, rowsToJava(pushdownDs.collectAsList())); + + Dataset unboundedPushdownDs = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.START_SNAPSHOT_ID, snapshotId1) + .load(tableName) + .agg(functions.min("data"), functions.max("data"), functions.count("data")); + String explain2 = + unboundedPushdownDs.queryExecution().explainString(ExplainMode.fromString("simple")); + assertThat(explain2).contains("LocalTableScan", "min(data)", "max(data)", "count(data)"); + + List expected2 = Lists.newArrayList(); + expected2.add(new Object[] {-7777, 9999, 6L}); + assertEquals( + "min/max/count push down", expected2, rowsToJava(unboundedPushdownDs.collectAsList())); + } } From e7af21b6fa27fa6e0aeef1052fea7f37c695502c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Jun 2024 10:20:54 +0200 Subject: [PATCH 0402/1019] Build: Bump mkdocs-material from 9.5.26 to 9.5.27 (#10555) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.26 to 9.5.27. - [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.26...9.5.27) --- 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 963e97c866c9..b1300034ee4e 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.26 +mkdocs-material==9.5.27 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 bc9ca90c24b54b7ece16fe3d3e73d2ca502a6620 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Jun 2024 10:24:27 +0200 Subject: [PATCH 0403/1019] Build: Bump software.amazon.awssdk:bom from 2.26.3 to 2.26.7 (#10554) Bumps software.amazon.awssdk:bom from 2.26.3 to 2.26.7. --- 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 b913be8f3b9c..c6dadd9b758d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.26.3" +awssdk-bom = "2.26.7" azuresdk-bom = "1.2.24" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From b308f473a2bc3406f591da9ab4106409548a401c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Jun 2024 10:28:52 +0200 Subject: [PATCH 0404/1019] Build: Bump nessie from 0.90.4 to 0.91.1 (#10551) Bumps `nessie` from 0.90.4 to 0.91.1. Updates `org.projectnessie.nessie:nessie-client` from 0.90.4 to 0.91.1 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.90.4 to 0.91.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.90.4 to 0.91.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.90.4 to 0.91.1 --- 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 c6dadd9b758d..51892e187fdf 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.90.4" +nessie = "0.91.1" netty-buffer = "4.1.111.Final" netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" From e441ef2eae78d8f996328dfe3448ccba8d0ab785 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 24 Jun 2024 10:34:55 +0200 Subject: [PATCH 0405/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.0.6 to 1.1.0 (#10552) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.0.6 to 1.1.0. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.0.6...1.1.0) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 51892e187fdf..8f606f5596e8 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -73,7 +73,7 @@ object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.0.6" +roaringbitmap = "1.1.0" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" From 24404c993c6af3025a9642c3f1d48cbb510a4ab5 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Mon, 24 Jun 2024 10:39:26 -0700 Subject: [PATCH 0406/1019] Core: Pushdown data_file.content filter in entries metadata table (#10203) --- .../org/apache/iceberg/BaseEntriesTable.java | 185 +++++++++++++++++- .../iceberg/MetadataTableScanTestBase.java | 5 +- .../iceberg/TestMetadataTableScans.java | 148 ++++++++++++-- 3 files changed, 322 insertions(+), 16 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index 43d8a71f8706..4e485d516f12 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -23,8 +23,12 @@ import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import org.apache.iceberg.expressions.Binder; +import org.apache.iceberg.expressions.BoundReference; import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionVisitors; import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.expressions.ManifestEvaluator; import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; @@ -68,6 +72,7 @@ static CloseableIterable planFiles( Expression rowFilter = context.rowFilter(); boolean caseSensitive = context.caseSensitive(); boolean ignoreResiduals = context.ignoreResiduals(); + Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : rowFilter; LoadingCache evalCache = Caffeine.newBuilder() @@ -77,14 +82,18 @@ static CloseableIterable planFiles( PartitionSpec transformedSpec = BaseFilesTable.transformSpec(tableSchema, spec); return ManifestEvaluator.forRowFilter(rowFilter, transformedSpec, caseSensitive); }); + ManifestContentEvaluator manifestContentEvaluator = + new ManifestContentEvaluator(filter, tableSchema.asStruct(), caseSensitive); CloseableIterable filteredManifests = CloseableIterable.filter( - manifests, manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); + manifests, + manifest -> + evalCache.get(manifest.partitionSpecId()).eval(manifest) + && manifestContentEvaluator.eval(manifest)); String schemaString = SchemaParser.toJson(projectedSchema); String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); - Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : rowFilter; ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter); return CloseableIterable.transform( @@ -94,6 +103,178 @@ static CloseableIterable planFiles( table, manifest, projectedSchema, schemaString, specString, residuals)); } + /** + * Evaluates an {@link Expression} on a {@link ManifestFile} to test whether a given data or + * delete manifests shall be included in the scan + */ + private static class ManifestContentEvaluator { + + private final Expression boundExpr; + + private ManifestContentEvaluator( + Expression expr, Types.StructType structType, boolean caseSensitive) { + Expression rewritten = Expressions.rewriteNot(expr); + this.boundExpr = Binder.bind(structType, rewritten, caseSensitive); + } + + private boolean eval(ManifestFile manifest) { + return new ManifestEvalVisitor().eval(manifest); + } + + private class ManifestEvalVisitor extends ExpressionVisitors.BoundExpressionVisitor { + + private int manifestContentId; + + private static final boolean ROWS_MIGHT_MATCH = true; + private static final boolean ROWS_CANNOT_MATCH = false; + + private boolean eval(ManifestFile manifestFile) { + this.manifestContentId = manifestFile.content().id(); + return ExpressionVisitors.visitEvaluator(boundExpr, this); + } + + @Override + public Boolean alwaysTrue() { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean alwaysFalse() { + return ROWS_CANNOT_MATCH; + } + + @Override + public Boolean not(Boolean result) { + return !result; + } + + @Override + public Boolean and(Boolean leftResult, Boolean rightResult) { + return leftResult && rightResult; + } + + @Override + public Boolean or(Boolean leftResult, Boolean rightResult) { + return leftResult || rightResult; + } + + @Override + public Boolean isNull(BoundReference ref) { + if (fileContent(ref)) { + return ROWS_CANNOT_MATCH; // date_file.content should not be null + } else { + return ROWS_MIGHT_MATCH; + } + } + + @Override + public Boolean notNull(BoundReference ref) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean isNaN(BoundReference ref) { + if (fileContent(ref)) { + return ROWS_CANNOT_MATCH; // date_file.content should not be nan + } else { + return ROWS_MIGHT_MATCH; + } + } + + @Override + public Boolean notNaN(BoundReference ref) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean lt(BoundReference ref, Literal lit) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean ltEq(BoundReference ref, Literal lit) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean gt(BoundReference ref, Literal lit) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean gtEq(BoundReference ref, Literal lit) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean eq(BoundReference ref, Literal lit) { + if (fileContent(ref)) { + Literal intLit = lit.to(Types.IntegerType.get()); + if (!contentMatch(intLit.value())) { + return ROWS_CANNOT_MATCH; + } + } + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean notEq(BoundReference ref, Literal lit) { + if (fileContent(ref)) { + Literal intLit = lit.to(Types.IntegerType.get()); + if (contentMatch(intLit.value())) { + return ROWS_CANNOT_MATCH; + } + } + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean in(BoundReference ref, Set literalSet) { + if (fileContent(ref)) { + if (literalSet.stream().noneMatch(lit -> contentMatch((Integer) lit))) { + return ROWS_CANNOT_MATCH; + } + } + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean notIn(BoundReference ref, Set literalSet) { + if (fileContent(ref)) { + if (literalSet.stream().anyMatch(lit -> contentMatch((Integer) lit))) { + return ROWS_CANNOT_MATCH; + } + } + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean startsWith(BoundReference ref, Literal lit) { + return ROWS_MIGHT_MATCH; + } + + @Override + public Boolean notStartsWith(BoundReference ref, Literal lit) { + return ROWS_MIGHT_MATCH; + } + + private boolean fileContent(BoundReference ref) { + return ref.fieldId() == DataFile.CONTENT.fieldId(); + } + + private boolean contentMatch(Integer fileContentId) { + if (FileContent.DATA.id() == fileContentId) { + return ManifestContent.DATA.id() == manifestContentId; + } else if (FileContent.EQUALITY_DELETES.id() == fileContentId + || FileContent.POSITION_DELETES.id() == fileContentId) { + return ManifestContent.DELETES.id() == manifestContentId; + } else { + return false; + } + } + } + } + static class ManifestReadTask extends BaseFileScanTask implements DataTask { private final Schema projection; private final Schema fileProjection; diff --git a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java index 9c732e843c8b..a4e964b017ba 100644 --- a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java @@ -43,9 +43,8 @@ protected static List parameters() { return Arrays.asList(1, 2); } - protected Set actualManifestListPaths(TableScan allManifestsTableScan) { - return StreamSupport.stream(allManifestsTableScan.planFiles().spliterator(), false) - .map(t -> (AllManifestsTable.ManifestListReadTask) t) + protected Set scannedPaths(TableScan scan) { + return StreamSupport.stream(scan.planFiles().spliterator(), false) .map(t -> t.file().path().toString()) .collect(Collectors.toSet()); } diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index df314f6a802f..0a3040939c04 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -41,6 +41,7 @@ import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.relocated.com.google.common.collect.Iterators; 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.collect.Streams; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; @@ -227,6 +228,131 @@ public void testManifestEntriesTableWithDroppedPartition() throws IOException { } } + @TestTemplate + public void testEntriesTableDataFileContentEq() { + preparePartitionedTable(); + + Table entriesTable = new ManifestEntriesTable(table); + + Expression dataOnly = Expressions.equal("data_file.content", 0); + TableScan entriesTableScan = entriesTable.newScan().filter(dataOnly); + Set expected = + table.currentSnapshot().dataManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + + assertThat(scannedPaths(entriesTableScan)) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(expected); + + assertThat( + scannedPaths(entriesTable.newScan().filter(Expressions.equal("data_file.content", 3)))) + .as("Expected manifest filter by data file content does not match") + .isEmpty(); + } + + @TestTemplate + public void testEntriesTableDateFileContentNotEq() { + preparePartitionedTable(); + + Table entriesTable = new ManifestEntriesTable(table); + + Expression notData = Expressions.notEqual("data_file.content", 0); + TableScan entriesTableScan = entriesTable.newScan().filter(notData); + Set expected = + table.currentSnapshot().deleteManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + + assertThat(scannedPaths(entriesTableScan)) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(expected); + + Set allManifests = + table.currentSnapshot().allManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + assertThat( + scannedPaths( + entriesTable.newScan().filter(Expressions.notEqual("data_file.content", 3)))) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(allManifests); + } + + @TestTemplate + public void testEntriesTableDataFileContentIn() { + preparePartitionedTable(); + Table entriesTable = new ManifestEntriesTable(table); + + Expression in0 = Expressions.in("data_file.content", 0); + TableScan scan1 = entriesTable.newScan().filter(in0); + Set expectedDataManifestPath = + table.currentSnapshot().dataManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + assertThat(scannedPaths(scan1)) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(expectedDataManifestPath); + + Expression in12 = Expressions.in("data_file.content", 1, 2); + TableScan scan2 = entriesTable.newScan().filter(in12); + Set expectedDeleteManifestPath = + table.currentSnapshot().deleteManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + assertThat(scannedPaths(scan2)) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(expectedDeleteManifestPath); + + Expression inAll = Expressions.in("data_file.content", 0, 1, 2); + Set allManifests = Sets.union(expectedDataManifestPath, expectedDeleteManifestPath); + assertThat(scannedPaths(entriesTable.newScan().filter(inAll))) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(allManifests); + + Expression inNeither = Expressions.in("data_file.content", 3, 4); + assertThat(scannedPaths(entriesTable.newScan().filter(inNeither))) + .as("Expected manifest filter by data file content does not match") + .isEmpty(); + } + + @TestTemplate + public void testEntriesTableDataFileContentNotIn() { + preparePartitionedTable(); + Table entriesTable = new ManifestEntriesTable(table); + + Expression notIn0 = Expressions.notIn("data_file.content", 0); + TableScan scan1 = entriesTable.newScan().filter(notIn0); + Set expectedDeleteManifestPath = + table.currentSnapshot().deleteManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + assertThat(scannedPaths(scan1)) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(expectedDeleteManifestPath); + + Expression notIn12 = Expressions.notIn("data_file.content", 1, 2); + TableScan scan2 = entriesTable.newScan().filter(notIn12); + Set expectedDataManifestPath = + table.currentSnapshot().dataManifests(table.io()).stream() + .map(ManifestFile::path) + .collect(Collectors.toSet()); + assertThat(scannedPaths(scan2)) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(expectedDataManifestPath); + + Expression notInNeither = Expressions.notIn("data_file.content", 3); + Set allManifests = Sets.union(expectedDataManifestPath, expectedDeleteManifestPath); + assertThat(scannedPaths(entriesTable.newScan().filter(notInNeither))) + .as("Expected manifest filter by data file content does not match") + .isEqualTo(allManifests); + + Expression notInAll = Expressions.notIn("data_file.content", 0, 1, 2); + assertThat(scannedPaths(entriesTable.newScan().filter(notInAll))) + .as("Expected manifest filter by data file content does not match") + .isEmpty(); + } + @TestTemplate public void testAllDataFilesTableHonorsIgnoreResiduals() throws IOException { table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B).commit(); @@ -1081,7 +1207,7 @@ public void testAllManifestsTableSnapshotGt() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.greaterThan("reference_snapshot_id", 2)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 3L, 4L)); } @@ -1095,7 +1221,7 @@ public void testAllManifestsTableSnapshotGte() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.greaterThanOrEqual("reference_snapshot_id", 3)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 3L, 4L)); } @@ -1109,7 +1235,7 @@ public void testAllManifestsTableSnapshotLt() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.lessThan("reference_snapshot_id", 3)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 1L, 2L)); } @@ -1123,7 +1249,7 @@ public void testAllManifestsTableSnapshotLte() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.lessThanOrEqual("reference_snapshot_id", 2)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 1L, 2L)); } @@ -1137,7 +1263,7 @@ public void testAllManifestsTableSnapshotEq() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.equal("reference_snapshot_id", 2)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 2L)); } @@ -1151,7 +1277,7 @@ public void testAllManifestsTableSnapshotNotEq() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.notEqual("reference_snapshot_id", 2)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 1L, 3L, 4L)); } @@ -1165,7 +1291,7 @@ public void testAllManifestsTableSnapshotIn() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.in("reference_snapshot_id", 1, 3)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 1L, 3L)); } @@ -1179,7 +1305,7 @@ public void testAllManifestsTableSnapshotNotIn() { TableScan manifestsTableScan = manifestsTable.newScan().filter(Expressions.notIn("reference_snapshot_id", 1, 3)); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 2L, 4L)); } @@ -1198,7 +1324,7 @@ public void testAllManifestsTableSnapshotAnd() { Expressions.and( Expressions.equal("reference_snapshot_id", 2), Expressions.greaterThan("length", 0))); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 2L)); } @@ -1217,7 +1343,7 @@ public void testAllManifestsTableSnapshotOr() { Expressions.or( Expressions.equal("reference_snapshot_id", 2), Expressions.equal("reference_snapshot_id", 4))); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 2L, 4L)); } @@ -1233,7 +1359,7 @@ public void testAllManifestsTableSnapshotNot() { .newScan() .filter(Expressions.not(Expressions.equal("reference_snapshot_id", 2))); - assertThat(actualManifestListPaths(manifestsTableScan)) + assertThat(scannedPaths(manifestsTableScan)) .as("Expected snapshots do not match") .isEqualTo(expectedManifestListPaths(table.snapshots(), 1L, 3L, 4L)); } From 068d62a0d3773cc0462ce7b8baae986a755fbb78 Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Tue, 25 Jun 2024 12:17:04 +0200 Subject: [PATCH 0407/1019] Build: Bump Nessie to 0.91.2 (#10563) --- 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 8f606f5596e8..0c22fae3fd81 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.91.1" +nessie = "0.91.2" netty-buffer = "4.1.111.Final" netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" From bdeb0cb0085783a42db3a58c237b27ef90bed163 Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 25 Jun 2024 18:18:05 +0800 Subject: [PATCH 0408/1019] Spark: Remove useless code in `TestRemoveOrphanFilesProcedure` (#10562) --- .../iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java | 1 - .../iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java | 1 - .../iceberg/spark/extensions/TestRemoveOrphanFilesProcedure.java | 1 - 3 files changed, 3 deletions(-) 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 1701faf37892..6e193b9a8642 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 @@ -302,7 +302,6 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { Table table = validationCatalog.loadTable(tableIdent); - String metadataLocation = table.location() + "/metadata"; String dataLocation = table.location() + "/data"; // produce orphan files in the data location using parquet 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 46d37ce20516..b7f82725aab8 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 @@ -297,7 +297,6 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { Table table = validationCatalog.loadTable(tableIdent); - String metadataLocation = table.location() + "/metadata"; String dataLocation = table.location() + "/data"; // produce orphan files in the data location using parquet 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 303e6b13e0df..042b846cb15a 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 @@ -285,7 +285,6 @@ public void testConcurrentRemoveOrphanFiles() throws IOException { Table table = validationCatalog.loadTable(tableIdent); - String metadataLocation = table.location() + "/metadata"; String dataLocation = table.location() + "/data"; // produce orphan files in the data location using parquet From 07f8e8d62b362b377ca19de41c18a375f7ad1906 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 25 Jun 2024 17:29:23 +0200 Subject: [PATCH 0409/1019] Build: Upgrade to gradle 8.8 (#8486) --- build.gradle | 8 ++++++-- gradle/wrapper/gradle-wrapper.properties | 5 ++--- gradlew | 24 ++++++++++++++---------- 3 files changed, 22 insertions(+), 15 deletions(-) diff --git a/build.gradle b/build.gradle index 9b3c35270519..64396585deff 100644 --- a/build.gradle +++ b/build.gradle @@ -38,7 +38,7 @@ buildscript { classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.5.0" - classpath 'com.palantir.gradle.revapi:gradle-revapi:1.7.0' + classpath 'io.github.nastra.gradle.revapi:gradle-revapi:1.8.1' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.2' classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.1.0' classpath 'org.openapitools:openapi-generator-gradle-plugin:6.6.0' @@ -139,7 +139,7 @@ subprojects { apply plugin: 'java-library' if (project.name in REVAPI_PROJECTS) { - apply plugin: 'com.palantir.revapi' + apply plugin: 'io.github.nastra.revapi' revapi { oldGroup = project.group oldName = project.name @@ -165,6 +165,10 @@ subprojects { rootTask.finalizedBy showDeprecationRulesOnRevApiFailure } } + + tasks.named("revapiAnalyze").configure { + dependsOn(":iceberg-common:jar") + } } configurations { diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index 4a0a58d550be..d296387b57c6 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,8 +1,7 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -# checksum was taken from https://gradle.org/release-checksums -distributionSha256Sum=e111cb9948407e26351227dabce49822fb88c37ee72f1d1582a69c68af2e702f -distributionUrl=https\://services.gradle.org/distributions/gradle-8.1.1-bin.zip +distributionSha256Sum=a4b4158601f8636cdeeab09bd76afb640030bb5b144aafe261a5e8af027dc612 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-bin.zip networkTimeout=10000 zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew index 0c913deb5545..68bbcf0a5c21 100755 --- a/gradlew +++ b/gradlew @@ -55,7 +55,7 @@ # Darwin, MinGW, and NonStop. # # (3) This script is generated from the Groovy template -# https://github.com/gradle/gradle/blob/HEAD/subprojects/plugins/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt +# https://github.com/gradle/gradle/blob/HEAD/platforms/jvm/plugins-application/src/main/resources/org/gradle/api/internal/plugins/unixStartScript.txt # within the Gradle project. # # You can find Gradle at https://github.com/gradle/gradle/. @@ -83,10 +83,11 @@ done # This is normally unused # 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:-./}" && 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.1.1/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.8.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. @@ -134,10 +135,13 @@ location of your Java installation." fi else JAVACMD=java - which java >/dev/null 2>&1 || die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. + if ! command -v java >/dev/null 2>&1 + then + die "ERROR: JAVA_HOME is not set and no 'java' command could be found in your PATH. Please set the JAVA_HOME variable in your environment to match the location of your Java installation." + fi fi # Increase the maximum file descriptors if we can. @@ -145,7 +149,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then case $MAX_FD in #( max*) # In POSIX sh, ulimit -H is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC2039,SC3045 MAX_FD=$( ulimit -H -n ) || warn "Could not query maximum file descriptor limit" esac @@ -153,7 +157,7 @@ if ! "$cygwin" && ! "$darwin" && ! "$nonstop" ; then '' | soft) :;; #( *) # In POSIX sh, ulimit -n is undefined. That's why the result is checked to see if it worked. - # shellcheck disable=SC3045 + # shellcheck disable=SC2039,SC3045 ulimit -n "$MAX_FD" || warn "Could not set maximum file descriptor limit to $MAX_FD" esac @@ -202,11 +206,11 @@ fi # Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script. DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' -# Collect all arguments for the java command; -# * $DEFAULT_JVM_OPTS, $JAVA_OPTS, and $GRADLE_OPTS can contain fragments of -# shell script including quotes and variable substitutions, so put them in -# double quotes to make sure that they get re-expanded; and -# * put everything else in single quotes, so that it's not re-expanded. +# Collect all arguments for the java command: +# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, +# and any embedded shellness will be escaped. +# * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be +# treated as '${Hostname}' itself on the command line. set -- \ "-Dorg.gradle.appname=$APP_BASE_NAME" \ From 4d1d1e6957c0af47c442be9644a3e41573f7a117 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 25 Jun 2024 09:21:23 -0700 Subject: [PATCH 0410/1019] Spark: Backport support for Aggregate push down for incremental scan to Spark 3.4 (#10561) --- .../spark/source/SparkScanBuilder.java | 89 ++++++++----------- .../spark/source/TestDataSourceOptions.java | 32 +++++-- .../spark/sql/TestAggregatePushDown.java | 50 +++++++++++ 3 files changed, 109 insertions(+), 62 deletions(-) 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 d6f34231ae75..b430e6fca233 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 @@ -37,7 +37,6 @@ import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; import org.apache.iceberg.expressions.AggregateEvaluator; import org.apache.iceberg.expressions.Binder; import org.apache.iceberg.expressions.BoundAggregate; @@ -232,15 +231,8 @@ public boolean pushAggregation(Aggregation aggregation) { return false; } - TableScan scan = table.newScan().includeColumnStats(); - Snapshot snapshot = readSnapshot(); - if (snapshot == null) { - LOG.info("Skipping aggregate pushdown: table snapshot is null"); - return false; - } - scan = scan.useSnapshot(snapshot.snapshotId()); - scan = configureSplitPlanning(scan); - scan = scan.filter(filterExpression()); + org.apache.iceberg.Scan scan = + buildIcebergBatchScan(true /* include Column Stats */, schemaWithMetadataColumns()); try (CloseableIterable fileScanTasks = scan.planFiles()) { List tasks = ImmutableList.copyOf(fileScanTasks); @@ -282,11 +274,6 @@ private boolean canPushDownAggregation(Aggregation aggregation) { return false; } - if (readConf.startSnapshotId() != null) { - LOG.info("Skipping aggregate pushdown: incremental scan is not supported"); - return false; - } - // If group by expression is the same as the partition, the statistics information can still // be used to calculate min/max/count, will enable aggregate push down in next phase. // TODO: enable aggregate push down for partition col group by expression @@ -298,17 +285,6 @@ private boolean canPushDownAggregation(Aggregation aggregation) { return true; } - private Snapshot readSnapshot() { - Snapshot snapshot; - if (readConf.snapshotId() != null) { - snapshot = table.snapshot(readConf.snapshotId()); - } else { - snapshot = SnapshotUtil.latestSnapshot(table, readConf.branch()); - } - - return snapshot; - } - private boolean metricsModeSupportsAggregatePushDown(List> aggregates) { MetricsConfig config = MetricsConfig.forTable(table); for (BoundAggregate aggregate : aggregates) { @@ -387,6 +363,18 @@ public Scan build() { } private Scan buildBatchScan() { + Schema expectedSchema = schemaWithMetadataColumns(); + return new SparkBatchQueryScan( + spark, + table, + buildIcebergBatchScan(false /* not include Column Stats */, expectedSchema), + readConf, + expectedSchema, + filterExpressions, + metricsReporter::scanReport); + } + + private org.apache.iceberg.Scan buildIcebergBatchScan(boolean withStats, Schema expectedSchema) { Long snapshotId = readConf.snapshotId(); Long asOfTimestamp = readConf.asOfTimestamp(); String branch = readConf.branch(); @@ -427,15 +415,19 @@ private Scan buildBatchScan() { SparkReadOptions.END_TIMESTAMP); if (startSnapshotId != null) { - return buildIncrementalAppendScan(startSnapshotId, endSnapshotId); + return buildIncrementalAppendScan(startSnapshotId, endSnapshotId, withStats, expectedSchema); } else { - return buildBatchScan(snapshotId, asOfTimestamp, branch, tag); + return buildBatchScan(snapshotId, asOfTimestamp, branch, tag, withStats, expectedSchema); } } - private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, String tag) { - Schema expectedSchema = schemaWithMetadataColumns(); - + private org.apache.iceberg.Scan buildBatchScan( + Long snapshotId, + Long asOfTimestamp, + String branch, + String tag, + boolean withStats, + Schema expectedSchema) { BatchScan scan = newBatchScan() .caseSensitive(caseSensitive) @@ -443,6 +435,10 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, .project(expectedSchema) .metricsReporter(metricsReporter); + if (withStats) { + scan = scan.includeColumnStats(); + } + if (snapshotId != null) { scan = scan.useSnapshot(snapshotId); } @@ -459,21 +455,11 @@ private Scan buildBatchScan(Long snapshotId, Long asOfTimestamp, String branch, scan = scan.useRef(tag); } - scan = configureSplitPlanning(scan); - - return new SparkBatchQueryScan( - spark, - table, - scan, - readConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); + return configureSplitPlanning(scan); } - private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId) { - Schema expectedSchema = schemaWithMetadataColumns(); - + private org.apache.iceberg.Scan buildIncrementalAppendScan( + long startSnapshotId, Long endSnapshotId, boolean withStats, Schema expectedSchema) { IncrementalAppendScan scan = table .newIncrementalAppendScan() @@ -483,20 +469,15 @@ private Scan buildIncrementalAppendScan(long startSnapshotId, Long endSnapshotId .project(expectedSchema) .metricsReporter(metricsReporter); + if (withStats) { + scan = scan.includeColumnStats(); + } + if (endSnapshotId != null) { scan = scan.toSnapshot(endSnapshotId); } - scan = configureSplitPlanning(scan); - - return new SparkBatchQueryScan( - spark, - table, - scan, - readConf, - expectedSchema, - filterExpressions, - metricsReporter::scanReport); + return configureSplitPlanning(scan); } @SuppressWarnings("CyclomaticComplexity") 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 6afd4fdb514d..77e1c373e8d9 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 @@ -56,6 +56,7 @@ import org.apache.spark.sql.SaveMode; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.functions; import org.junit.AfterClass; import org.junit.Assert; import org.junit.BeforeClass; @@ -289,29 +290,44 @@ public void testIncrementalScanOptions() throws IOException { "Cannot set only end-snapshot-id for incremental scans. Please, set start-snapshot-id too."); // test (1st snapshot, current snapshot] incremental scan. - List result = + Dataset unboundedIncrementalResult = spark .read() .format("iceberg") .option("start-snapshot-id", snapshotIds.get(3).toString()) - .load(tableLocation) + .load(tableLocation); + List result1 = + unboundedIncrementalResult .orderBy("id") .as(Encoders.bean(SimpleRecord.class)) .collectAsList(); - Assert.assertEquals("Records should match", expectedRecords.subList(1, 4), result); + assertThat(result1).as("Records should match").isEqualTo(expectedRecords.subList(1, 4)); + assertThat(unboundedIncrementalResult.count()) + .as("Unprocessed count should match record count") + .isEqualTo(3); + + Row row1 = unboundedIncrementalResult.agg(functions.min("id"), functions.max("id")).head(); + assertThat(row1.getInt(0)).as("min value should match").isEqualTo(2); + assertThat(row1.getInt(1)).as("max value should match").isEqualTo(4); // test (2nd snapshot, 3rd snapshot] incremental scan. - Dataset resultDf = + Dataset incrementalResult = spark .read() .format("iceberg") .option("start-snapshot-id", snapshotIds.get(2).toString()) .option("end-snapshot-id", snapshotIds.get(1).toString()) .load(tableLocation); - List result1 = - resultDf.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); - Assert.assertEquals("Records should match", expectedRecords.subList(2, 3), result1); - Assert.assertEquals("Unprocessed count should match record count", 1, resultDf.count()); + List result2 = + incrementalResult.orderBy("id").as(Encoders.bean(SimpleRecord.class)).collectAsList(); + assertThat(result2).as("Records should match").isEqualTo(expectedRecords.subList(2, 3)); + assertThat(incrementalResult.count()) + .as("Unprocessed count should match record count") + .isEqualTo(1); + + Row row2 = incrementalResult.agg(functions.min("id"), functions.max("id")).head(); + assertThat(row2.getInt(0)).as("min value should match").isEqualTo(3); + assertThat(row2.getInt(1)).as("max value should match").isEqualTo(3); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java index 2579d614685c..1a4e2f3e1c79 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestAggregatePushDown.java @@ -36,8 +36,13 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkReadOptions; import org.apache.iceberg.spark.SparkTestBase; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.execution.ExplainMode; +import org.apache.spark.sql.functions; import org.junit.After; import org.junit.Assert; import org.junit.BeforeClass; @@ -796,4 +801,49 @@ public void testInfinity() { }); assertEquals("min/max/count push down", expected, actual); } + + @Test + public void testAggregatePushDownForIncrementalScan() { + sql("CREATE TABLE %s (id LONG, data INT) USING iceberg", tableName); + sql( + "INSERT INTO TABLE %s VALUES (1, 1111), (1, 2222), (2, 3333), (2, 4444), (3, 5555), (3, 6666) ", + tableName); + long snapshotId1 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (4, 7777), (5, 8888)", tableName); + long snapshotId2 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (6, -7777), (7, 8888)", tableName); + long snapshotId3 = validationCatalog.loadTable(tableIdent).currentSnapshot().snapshotId(); + sql("INSERT INTO %s VALUES (8, 7777), (9, 9999)", tableName); + + Dataset pushdownDs = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.START_SNAPSHOT_ID, snapshotId2) + .option(SparkReadOptions.END_SNAPSHOT_ID, snapshotId3) + .load(tableName) + .agg(functions.min("data"), functions.max("data"), functions.count("data")); + String explain1 = pushdownDs.queryExecution().explainString(ExplainMode.fromString("simple")); + assertThat(explain1).contains("LocalTableScan", "min(data)", "max(data)", "count(data)"); + + List expected1 = Lists.newArrayList(); + expected1.add(new Object[] {-7777, 8888, 2L}); + assertEquals("min/max/count push down", expected1, rowsToJava(pushdownDs.collectAsList())); + + Dataset unboundedPushdownDs = + spark + .read() + .format("iceberg") + .option(SparkReadOptions.START_SNAPSHOT_ID, snapshotId1) + .load(tableName) + .agg(functions.min("data"), functions.max("data"), functions.count("data")); + String explain2 = + unboundedPushdownDs.queryExecution().explainString(ExplainMode.fromString("simple")); + assertThat(explain2).contains("LocalTableScan", "min(data)", "max(data)", "count(data)"); + + List expected2 = Lists.newArrayList(); + expected2.add(new Object[] {-7777, 9999, 6L}); + assertEquals( + "min/max/count push down", expected2, rowsToJava(unboundedPushdownDs.collectAsList())); + } } From da250ae726986609bb15e252f73ab256beff4e0b Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 25 Jun 2024 23:42:43 +0200 Subject: [PATCH 0411/1019] Build: Move to `goooler` shadow plugin (#10568) Co-authored-by: Eduard Tudenhoefner --- aws-bundle/build.gradle | 2 +- azure-bundle/build.gradle | 2 +- build.gradle | 4 ++-- flink/v1.17/build.gradle | 2 +- flink/v1.18/build.gradle | 2 +- flink/v1.19/build.gradle | 2 +- gcp-bundle/build.gradle | 2 +- hive-runtime/build.gradle | 2 +- hive3-orc-bundle/build.gradle | 2 +- spark/v3.3/build.gradle | 2 +- spark/v3.4/build.gradle | 2 +- spark/v3.5/build.gradle | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/aws-bundle/build.gradle b/aws-bundle/build.gradle index 00b2cc4b3ae7..6b1eae46d89e 100644 --- a/aws-bundle/build.gradle +++ b/aws-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-aws-bundle") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/azure-bundle/build.gradle b/azure-bundle/build.gradle index 30bc1dde3631..3bc0a31cb46d 100644 --- a/azure-bundle/build.gradle +++ b/azure-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-azure-bundle") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/build.gradle b/build.gradle index 64396585deff..8118b7a4af5b 100644 --- a/build.gradle +++ b/build.gradle @@ -26,7 +26,7 @@ buildscript { gradlePluginPortal() } dependencies { - classpath 'com.github.johnrengelman:shadow:8.1.1' + classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.7' classpath 'com.palantir.baseline:gradle-baseline-java:4.42.0' // com.palantir.baseline:gradle-baseline-java:4.42.0 (the last version supporting Java 8) pulls // in an old version of the errorprone, which doesn't work w/ Gradle 8, so bump errorpone as @@ -257,7 +257,7 @@ subprojects { } project(':iceberg-bundled-guava') { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle index f2bd37a6d17c..2ced7a5a5cb7 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.17/build.gradle @@ -125,7 +125,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index c08ae5d8cc1f..f06318af83a3 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -125,7 +125,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index a200a33890cc..392a1cb124f0 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -127,7 +127,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/gcp-bundle/build.gradle b/gcp-bundle/build.gradle index 5e28433cdfa4..98e67c214ce0 100644 --- a/gcp-bundle/build.gradle +++ b/gcp-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-gcp-bundle") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/hive-runtime/build.gradle b/hive-runtime/build.gradle index da21f66f6afd..b2051d529512 100644 --- a/hive-runtime/build.gradle +++ b/hive-runtime/build.gradle @@ -20,7 +20,7 @@ def hiveVersions = (System.getProperty("hiveVersions") != null ? System.getProperty("hiveVersions") : System.getProperty("defaultHiveVersions")).split(",") project(':iceberg-hive-runtime') { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/hive3-orc-bundle/build.gradle b/hive3-orc-bundle/build.gradle index 2b83e7796800..11bb2abf85c4 100644 --- a/hive3-orc-bundle/build.gradle +++ b/hive3-orc-bundle/build.gradle @@ -21,7 +21,7 @@ // name. This is to be used by Hive3 for features including e.g. vectorization. project(':iceberg-hive3-orc-bundle') { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 9551d683f237..2f1ddef0a325 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -177,7 +177,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 7d81137a6b80..1da53ecfc61d 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -180,7 +180,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index b29ba6761ebc..f1af2120ef8c 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -184,7 +184,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'com.github.johnrengelman.shadow' + apply plugin: 'io.github.goooler.shadow' tasks.jar.dependsOn tasks.shadowJar From 2407725d1868bf57b2f173d062f56281ccf3cf66 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 26 Jun 2024 00:40:42 +0200 Subject: [PATCH 0412/1019] Spec: Fix Typo (#10564) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index e590a8b95116..9a3c16e3ac91 100644 --- a/format/spec.md +++ b/format/spec.md @@ -168,7 +168,7 @@ A **`map`** is a collection of key-value pairs with a key type and a value type. 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. -| Added by verison | Primitive type | Description | Requirements | +| Added by version | Primitive type | Description | Requirements | |------------------|--------------------|--------------------------------------------------------------------------|--------------------------------------------------| | | **`boolean`** | True or false | | | | **`int`** | 32-bit signed integers | Can promote to `long` | From ac875ce3dbab8d98d253a928e259403166c52ce3 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 26 Jun 2024 16:37:37 +0800 Subject: [PATCH 0413/1019] Core, Flink: Add task-type field to JSON serde of scan task / Add JSON serde for StaticDataTask. (#9728) --- .../org/apache/iceberg/DataTaskParser.java | 81 ++++++ .../apache/iceberg/FileScanTaskParser.java | 39 ++- .../org/apache/iceberg/ScanTaskParser.java | 105 +++++++ .../org/apache/iceberg/StaticDataTask.java | 26 ++ .../apache/iceberg/TestDataTaskParser.java | 274 ++++++++++++++++++ .../iceberg/TestFileScanTaskParser.java | 75 ++++- .../apache/iceberg/TestScanTaskParser.java | 54 ++++ .../source/split/IcebergSourceSplit.java | 6 +- .../source/split/IcebergSourceSplit.java | 6 +- .../source/split/IcebergSourceSplit.java | 6 +- 10 files changed, 644 insertions(+), 28 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/DataTaskParser.java create mode 100644 core/src/main/java/org/apache/iceberg/ScanTaskParser.java create mode 100644 core/src/test/java/org/apache/iceberg/TestDataTaskParser.java create mode 100644 core/src/test/java/org/apache/iceberg/TestScanTaskParser.java diff --git a/core/src/main/java/org/apache/iceberg/DataTaskParser.java b/core/src/main/java/org/apache/iceberg/DataTaskParser.java new file mode 100644 index 000000000000..428bcf15e7e2 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/DataTaskParser.java @@ -0,0 +1,81 @@ +/* + * 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 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.util.JsonUtil; + +class DataTaskParser { + private static final String SCHEMA = "schema"; + private static final String PROJECTED_SCHEMA = "projection"; + private static final String METADATA_FILE = "metadata-file"; + private static final String ROWS = "rows"; + + private DataTaskParser() {} + + static void toJson(StaticDataTask dataTask, JsonGenerator generator) throws IOException { + Preconditions.checkArgument(dataTask != null, "Invalid data task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(dataTask.schema(), generator); + + generator.writeFieldName(PROJECTED_SCHEMA); + SchemaParser.toJson(dataTask.projectedSchema(), generator); + + generator.writeFieldName(METADATA_FILE); + ContentFileParser.toJson(dataTask.metadataFile(), PartitionSpec.unpartitioned(), generator); + + Preconditions.checkArgument(dataTask.tableRows() != null, "Invalid data task: null table rows"); + generator.writeArrayFieldStart(ROWS); + for (StructLike row : dataTask.tableRows()) { + SingleValueParser.toJson(dataTask.schema().asStruct(), row, generator); + } + + generator.writeEndArray(); + } + + static StaticDataTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for data task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for data task: non-object (%s)", jsonNode); + + Schema schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + Schema projectedSchema = SchemaParser.fromJson(JsonUtil.get(PROJECTED_SCHEMA, jsonNode)); + DataFile metadataFile = + (DataFile) + ContentFileParser.fromJson( + JsonUtil.get(METADATA_FILE, jsonNode), PartitionSpec.unpartitioned()); + + JsonNode rowsArray = JsonUtil.get(ROWS, jsonNode); + Preconditions.checkArgument( + rowsArray.isArray(), "Invalid JSON node for rows: non-array (%s)", rowsArray); + + StructLike[] rows = new StructLike[rowsArray.size()]; + for (int i = 0; i < rowsArray.size(); ++i) { + JsonNode rowNode = rowsArray.get(i); + rows[i] = (StructLike) SingleValueParser.fromJson(schema.asStruct(), rowNode); + } + + return new StaticDataTask(metadataFile, schema, projectedSchema, rows); + } +} diff --git a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java index 0a708f2668cc..a6ea41319f4e 100644 --- a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java +++ b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java @@ -40,16 +40,38 @@ public class FileScanTaskParser { private FileScanTaskParser() {} + /** + * Serialize file scan task to JSON string + * + * @deprecated will be removed in 1.7.0; use {@link ScanTaskParser#toJson(FileScanTask)} instead + */ + @Deprecated public static String toJson(FileScanTask fileScanTask) { + Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); return JsonUtil.generate( - generator -> FileScanTaskParser.toJson(fileScanTask, generator), false); + generator -> { + generator.writeStartObject(); + toJson(fileScanTask, generator); + generator.writeEndObject(); + }, + false); + } + + /** + * Deserialize file scan task from JSON string + * + * @deprecated will be removed in 1.7.0; use {@link ScanTaskParser#fromJson(String, boolean)} + * instead + */ + @Deprecated + public static FileScanTask fromJson(String json, boolean caseSensitive) { + Preconditions.checkArgument(json != null, "Invalid JSON string for file scan task: null"); + return JsonUtil.parse(json, node -> fromJson(node, caseSensitive)); } - private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) - throws IOException { + static void toJson(FileScanTask fileScanTask, JsonGenerator generator) throws IOException { Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); - generator.writeStartObject(); generator.writeFieldName(SCHEMA); SchemaParser.toJson(fileScanTask.schema(), generator); @@ -78,16 +100,9 @@ private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) generator.writeFieldName(RESIDUAL); ExpressionParser.toJson(fileScanTask.residual(), generator); } - - generator.writeEndObject(); - } - - public static FileScanTask fromJson(String json, boolean caseSensitive) { - Preconditions.checkArgument(json != null, "Invalid JSON string for file scan task: null"); - return JsonUtil.parse(json, node -> FileScanTaskParser.fromJson(node, caseSensitive)); } - private static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { + static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for file scan task: null"); Preconditions.checkArgument( jsonNode.isObject(), "Invalid JSON node for file scan task: non-object (%s)", jsonNode); diff --git a/core/src/main/java/org/apache/iceberg/ScanTaskParser.java b/core/src/main/java/org/apache/iceberg/ScanTaskParser.java new file mode 100644 index 000000000000..9447d0668a1f --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ScanTaskParser.java @@ -0,0 +1,105 @@ +/* + * 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 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.relocated.com.google.common.base.Strings; +import org.apache.iceberg.util.JsonUtil; + +public class ScanTaskParser { + private static final String TASK_TYPE = "task-type"; + + private enum TaskType { + FILE_SCAN_TASK("file-scan-task"), + DATA_TASK("data-task"); + + private final String value; + + TaskType(String value) { + this.value = value; + } + + public static TaskType fromTypeName(String value) { + Preconditions.checkArgument( + !Strings.isNullOrEmpty(value), "Invalid task type name: null or empty"); + if (FILE_SCAN_TASK.typeName().equalsIgnoreCase(value)) { + return FILE_SCAN_TASK; + } else if (DATA_TASK.typeName().equalsIgnoreCase(value)) { + return DATA_TASK; + } else { + throw new IllegalArgumentException("Unknown task type: " + value); + } + } + + public String typeName() { + return value; + } + } + + private ScanTaskParser() {} + + public static String toJson(FileScanTask fileScanTask) { + Preconditions.checkArgument(fileScanTask != null, "Invalid scan task: null"); + return JsonUtil.generate(generator -> toJson(fileScanTask, generator), false); + } + + public static FileScanTask fromJson(String json, boolean caseSensitive) { + Preconditions.checkArgument(json != null, "Invalid JSON string for scan task: null"); + return JsonUtil.parse(json, node -> fromJson(node, caseSensitive)); + } + + private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) + throws IOException { + generator.writeStartObject(); + + if (fileScanTask instanceof StaticDataTask) { + generator.writeStringField(TASK_TYPE, TaskType.DATA_TASK.typeName()); + DataTaskParser.toJson((StaticDataTask) fileScanTask, generator); + } else if (fileScanTask instanceof BaseFileScanTask + || fileScanTask instanceof BaseFileScanTask.SplitScanTask) { + generator.writeStringField(TASK_TYPE, TaskType.FILE_SCAN_TASK.typeName()); + FileScanTaskParser.toJson(fileScanTask, generator); + } else { + throw new UnsupportedOperationException( + "Unsupported task type: " + fileScanTask.getClass().getCanonicalName()); + } + + generator.writeEndObject(); + } + + private static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { + TaskType taskType = TaskType.FILE_SCAN_TASK; + String taskTypeStr = JsonUtil.getStringOrNull(TASK_TYPE, jsonNode); + if (null != taskTypeStr) { + taskType = TaskType.fromTypeName(taskTypeStr); + } + + switch (taskType) { + case FILE_SCAN_TASK: + return FileScanTaskParser.fromJson(jsonNode, caseSensitive); + case DATA_TASK: + return DataTaskParser.fromJson(jsonNode); + default: + throw new UnsupportedOperationException("Unsupported task type: " + taskType.typeName()); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/StaticDataTask.java b/core/src/main/java/org/apache/iceberg/StaticDataTask.java index cffb42427960..f25ebd49c9d8 100644 --- a/core/src/main/java/org/apache/iceberg/StaticDataTask.java +++ b/core/src/main/java/org/apache/iceberg/StaticDataTask.java @@ -64,6 +64,19 @@ private StaticDataTask( this.rows = rows; } + StaticDataTask( + DataFile metadataFile, Schema tableSchema, Schema projectedSchema, StructLike[] rows) { + this.tableSchema = tableSchema; + this.projectedSchema = projectedSchema; + this.metadataFile = metadataFile; + this.rows = rows; + } + + @Override + public Schema schema() { + return tableSchema; + } + @Override public List deletes() { return ImmutableList.of(); @@ -106,6 +119,19 @@ public Iterable split(long splitSize) { return ImmutableList.of(this); } + Schema projectedSchema() { + return projectedSchema; + } + + DataFile metadataFile() { + return metadataFile; + } + + /** @return the table rows before projection */ + StructLike[] tableRows() { + return rows; + } + /** Implements {@link StructLike#get} for passing static rows. */ static class Row implements StructLike, Serializable { public static Row of(Object... values) { diff --git a/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java new file mode 100644 index 000000000000..5a3d119046f5 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestDataTaskParser.java @@ -0,0 +1,274 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +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.apache.iceberg.util.JsonUtil; +import org.junit.jupiter.api.Test; + +public class TestDataTaskParser { + // copied from SnapshotsTable to avoid making it package public + private static final Schema SNAPSHOT_SCHEMA = + new Schema( + Types.NestedField.required(1, "committed_at", Types.TimestampType.withZone()), + Types.NestedField.required(2, "snapshot_id", Types.LongType.get()), + Types.NestedField.optional(3, "parent_id", Types.LongType.get()), + Types.NestedField.optional(4, "operation", Types.StringType.get()), + Types.NestedField.optional(5, "manifest_list", Types.StringType.get()), + Types.NestedField.optional( + 6, + "summary", + Types.MapType.ofRequired(7, 8, Types.StringType.get(), Types.StringType.get()))); + + // copied from SnapshotsTable to avoid making it package public + private static StaticDataTask.Row snapshotToRow(Snapshot snap) { + return StaticDataTask.Row.of( + snap.timestampMillis() * 1000, + snap.snapshotId(), + snap.parentId(), + snap.operation(), + snap.manifestListLocation(), + snap.summary()); + } + + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> DataTaskParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid data task: null"); + + assertThatThrownBy(() -> DataTaskParser.toJson((StaticDataTask) createDataTask(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> DataTaskParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for data task: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> DataTaskParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for data task: non-object "); + + assertThatThrownBy(() -> DataTaskParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for data task: non-object "); + } + + @Test + public void missingFields() throws Exception { + ObjectMapper mapper = new ObjectMapper(); + + String missingSchemaStr = "{}"; + JsonNode missingSchemaNode = mapper.reader().readTree(missingSchemaStr); + assertThatThrownBy(() -> DataTaskParser.fromJson(missingSchemaNode)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse missing field: schema"); + + String missingProjectionStr = + "{" + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}" + + "}"; + JsonNode missingProjectionNode = mapper.reader().readTree(missingProjectionStr); + assertThatThrownBy(() -> DataTaskParser.fromJson(missingProjectionNode)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse missing field: projection"); + + String missingMetadataFileStr = + "{" + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}]}" + + "}"; + JsonNode missingMetadataFileNode = mapper.reader().readTree(missingMetadataFileStr); + assertThatThrownBy(() -> DataTaskParser.fromJson(missingMetadataFileNode)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse missing field: metadata-file"); + + String missingTableRowsStr = + "{\"task-type\":\"data-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}," + + "{\"id\":2,\"name\":\"snapshot_id\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":3,\"name\":\"parent_id\",\"required\":false,\"type\":\"long\"}," + + "{\"id\":4,\"name\":\"operation\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":5,\"name\":\"manifest_list\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":6,\"name\":\"summary\",\"required\":false,\"type\":{\"type\":\"map\"," + + "\"key-id\":7,\"key\":\"string\",\"value-id\":8," + + "\"value\":\"string\",\"value-required\":true}}]}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}," + + "{\"id\":2,\"name\":\"snapshot_id\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":3,\"name\":\"parent_id\",\"required\":false,\"type\":\"long\"}," + + "{\"id\":4,\"name\":\"operation\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":5,\"name\":\"manifest_list\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":6,\"name\":\"summary\",\"required\":false,\"type\":{\"type\":\"map\"," + + "\"key-id\":7,\"key\":\"string\",\"value-id\":8," + + "\"value\":\"string\",\"value-required\":true}}]}," + + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\"," + + "\"file-path\":\"/tmp/metadata2.json\"," + + "\"file-format\":\"METADATA\",\"partition\":{}," + + "\"file-size-in-bytes\":0,\"record-count\":2,\"sort-order-id\":0}" + + "}"; + JsonNode missingTableRowsNode = mapper.reader().readTree(missingTableRowsStr); + assertThatThrownBy(() -> DataTaskParser.fromJson(missingTableRowsNode)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot parse missing field: rows"); + } + + @Test + public void roundTripSerde() { + StaticDataTask dataTask = (StaticDataTask) createDataTask(); + String jsonStr = ScanTaskParser.toJson(dataTask); + assertThat(jsonStr).isEqualTo(snapshotsDataTaskJson()); + StaticDataTask deserializedTask = (StaticDataTask) ScanTaskParser.fromJson(jsonStr, true); + assertDataTaskEquals(dataTask, deserializedTask); + } + + private DataTask createDataTask() { + Map summary1 = + ImmutableMap.of( + "added-data-files", "1", + "added-records", "1", + "added-files-size", "10", + "changed-partition-count", "1", + "total-records", "1", + "total-files-size", "10", + "total-data-files", "1", + "total-delete-files", "0", + "total-position-deletes", "0", + "total-equality-deletes", "0"); + + Map summary2 = + ImmutableMap.of( + "added-data-files", "1", + "added-records", "1", + "added-files-size", "10", + "changed-partition-count", "1", + "total-records", "2", + "total-files-size", "20", + "total-data-files", "2", + "total-delete-files", "0", + "total-position-deletes", "0", + "total-equality-deletes", "0"); + + List snapshots = + Arrays.asList( + new BaseSnapshot( + 1L, 1L, null, 1234567890000L, "append", summary1, 1, "file:/tmp/manifest1.avro"), + new BaseSnapshot( + 2L, 2L, 1L, 9876543210000L, "append", summary2, 1, "file:/tmp/manifest2.avro")); + + return StaticDataTask.of( + Files.localInput("file:/tmp/metadata2.json"), + SNAPSHOT_SCHEMA, + SNAPSHOT_SCHEMA, + snapshots, + TestDataTaskParser::snapshotToRow); + } + + private String snapshotsDataTaskJson() { + return "{\"task-type\":\"data-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}," + + "{\"id\":2,\"name\":\"snapshot_id\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":3,\"name\":\"parent_id\",\"required\":false,\"type\":\"long\"}," + + "{\"id\":4,\"name\":\"operation\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":5,\"name\":\"manifest_list\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":6,\"name\":\"summary\",\"required\":false,\"type\":{\"type\":\"map\"," + + "\"key-id\":7,\"key\":\"string\",\"value-id\":8," + + "\"value\":\"string\",\"value-required\":true}}]}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":1,\"name\":\"committed_at\",\"required\":true,\"type\":\"timestamptz\"}," + + "{\"id\":2,\"name\":\"snapshot_id\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":3,\"name\":\"parent_id\",\"required\":false,\"type\":\"long\"}," + + "{\"id\":4,\"name\":\"operation\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":5,\"name\":\"manifest_list\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":6,\"name\":\"summary\",\"required\":false,\"type\":{\"type\":\"map\"," + + "\"key-id\":7,\"key\":\"string\",\"value-id\":8," + + "\"value\":\"string\",\"value-required\":true}}]}," + + "\"metadata-file\":{\"spec-id\":0,\"content\":\"DATA\"," + + "\"file-path\":\"/tmp/metadata2.json\"," + + "\"file-format\":\"METADATA\",\"partition\":{}," + + "\"file-size-in-bytes\":0,\"record-count\":2,\"sort-order-id\":0}," + + "\"rows\":[{\"1\":\"2009-02-13T23:31:30+00:00\",\"2\":1,\"4\":\"append\"," + + "\"5\":\"file:/tmp/manifest1.avro\"," + + "\"6\":{\"keys\":[\"added-data-files\",\"added-records\",\"added-files-size\",\"changed-partition-count\"," + + "\"total-records\",\"total-files-size\",\"total-data-files\",\"total-delete-files\"," + + "\"total-position-deletes\",\"total-equality-deletes\"]," + + "\"values\":[\"1\",\"1\",\"10\",\"1\",\"1\",\"10\",\"1\",\"0\",\"0\",\"0\"]}}," + + "{\"1\":\"2282-12-22T20:13:30+00:00\",\"2\":2,\"3\":1,\"4\":\"append\"," + + "\"5\":\"file:/tmp/manifest2.avro\"," + + "\"6\":{\"keys\":[\"added-data-files\",\"added-records\",\"added-files-size\",\"changed-partition-count\"," + + "\"total-records\",\"total-files-size\",\"total-data-files\",\"total-delete-files\"," + + "\"total-position-deletes\",\"total-equality-deletes\"]," + + "\"values\":[\"1\",\"1\",\"10\",\"1\",\"2\",\"20\",\"2\",\"0\",\"0\",\"0\"]}}]}"; + } + + private void assertDataTaskEquals(StaticDataTask expected, StaticDataTask actual) { + assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); + + assertThat(actual.projectedSchema().asStruct()) + .as("Projected schema should match") + .isEqualTo(expected.projectedSchema().asStruct()); + + TestContentFileParser.assertContentFileEquals( + expected.metadataFile(), actual.metadataFile(), PartitionSpec.unpartitioned()); + + List expectedRows = Lists.newArrayList(expected.rows()); + List actualRows = Lists.newArrayList(actual.rows()); + assertThat(actualRows).hasSameSizeAs(expectedRows); + + // all fields are primitive types or map + Schema schema = expected.schema(); + for (int i = 0; i < expectedRows.size(); ++i) { + StructLike expectedRow = expectedRows.get(i); + StructLike actualRow = actualRows.get(i); + for (int pos = 0; pos < expectedRow.size(); ++pos) { + Class javaClass = schema.columns().get(pos).type().typeId().javaClass(); + assertThat(actualRow.get(pos, javaClass)).isEqualTo(expectedRow.get(pos, javaClass)); + } + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 6e274c4811ba..137e7897385b 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -35,23 +35,64 @@ public void testNullArguments() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid file scan task: null"); - assertThatThrownBy(() -> FileScanTaskParser.fromJson(null, true)) + assertThatThrownBy(() -> FileScanTaskParser.fromJson((String) null, true)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid JSON string for file scan task: null"); + + assertThatThrownBy(() -> ScanTaskParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid scan task: null"); + + assertThatThrownBy(() -> ScanTaskParser.fromJson(null, true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON string for scan task: null"); } @ParameterizedTest @ValueSource(booleans = {true, false}) - public void testParser(boolean caseSensitive) { + public void testFileScanTaskParser(boolean caseSensitive) { PartitionSpec spec = TestBase.SPEC; - FileScanTask fileScanTask = createScanTask(spec, caseSensitive); + FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); String jsonStr = FileScanTaskParser.toJson(fileScanTask); - assertThat(jsonStr).isEqualTo(expectedFileScanTaskJson()); + assertThat(jsonStr).isEqualTo(fileScanTaskJsonWithoutTaskType()); FileScanTask deserializedTask = FileScanTaskParser.fromJson(jsonStr, caseSensitive); assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); } - private FileScanTask createScanTask(PartitionSpec spec, boolean caseSensitive) { + /** Test backward compatibility where task-type field is absent from the JSON string */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testFileScanTaskParserWithoutTaskTypeField(boolean caseSensitive) { + PartitionSpec spec = TestBase.SPEC; + FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); + FileScanTask deserializedTask = + FileScanTaskParser.fromJson(fileScanTaskJsonWithoutTaskType(), caseSensitive); + assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testScanTaskParser(boolean caseSensitive) { + PartitionSpec spec = TestBase.SPEC; + FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); + String jsonStr = ScanTaskParser.toJson(fileScanTask); + assertThat(jsonStr).isEqualTo(fileScanTaskJson()); + FileScanTask deserializedTask = ScanTaskParser.fromJson(jsonStr, caseSensitive); + assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); + } + + /** Test backward compatibility where task-type field is absent from the JSON string */ + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testScanTaskParserWithoutTaskTypeField(boolean caseSensitive) { + PartitionSpec spec = TestBase.SPEC; + FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); + FileScanTask deserializedTask = + ScanTaskParser.fromJson(fileScanTaskJsonWithoutTaskType(), caseSensitive); + assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); + } + + private FileScanTask createFileScanTask(PartitionSpec spec, boolean caseSensitive) { ResidualEvaluator residualEvaluator; if (spec.isUnpartitioned()) { residualEvaluator = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); @@ -67,7 +108,7 @@ private FileScanTask createScanTask(PartitionSpec spec, boolean caseSensitive) { residualEvaluator); } - private String expectedFileScanTaskJson() { + private String fileScanTaskJsonWithoutTaskType() { return "{\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" + "{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," @@ -86,6 +127,26 @@ private String expectedFileScanTaskJson() { + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; } + private String fileScanTaskJson() { + return "{\"task-type\":\"file-scan-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[" + + "{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"spec\":{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}," + + "\"data-file\":{\"spec-id\":0,\"content\":\"DATA\",\"file-path\":\"/path/to/data-a.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0}," + + "\"file-size-in-bytes\":10,\"record-count\":1,\"sort-order-id\":0}," + + "\"start\":0,\"length\":10," + + "\"delete-files\":[{\"spec-id\":0,\"content\":\"POSITION_DELETES\"," + + "\"file-path\":\"/path/to/data-a-deletes.parquet\",\"file-format\":\"PARQUET\"," + + "\"partition\":{\"1000\":0},\"file-size-in-bytes\":10,\"record-count\":1}," + + "{\"spec-id\":0,\"content\":\"EQUALITY_DELETES\",\"file-path\":\"/path/to/data-a2-deletes.parquet\"," + + "\"file-format\":\"PARQUET\",\"partition\":{\"1000\":0},\"file-size-in-bytes\":10," + + "\"record-count\":1,\"equality-ids\":[1],\"sort-order-id\":0}]," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}}"; + } + private static void assertFileScanTaskEquals( FileScanTask expected, FileScanTask actual, PartitionSpec spec, boolean caseSensitive) { TestContentFileParser.assertContentFileEquals(expected.file(), actual.file(), spec); @@ -95,7 +156,7 @@ private static void assertFileScanTaskEquals( expected.deletes().get(pos), actual.deletes().get(pos), spec); } - assertThat(expected.schema().sameSchema(actual.schema())).as("Schema should match").isTrue(); + assertThat(actual.schema().asStruct()).isEqualTo(expected.schema().asStruct()); assertThat(actual.spec()).isEqualTo(expected.spec()); assertThat( ExpressionUtil.equivalent( diff --git a/core/src/test/java/org/apache/iceberg/TestScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestScanTaskParser.java new file mode 100644 index 000000000000..aad87514983a --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestScanTaskParser.java @@ -0,0 +1,54 @@ +/* + * 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.assertThatThrownBy; + +import org.junit.jupiter.api.Test; +import org.mockito.Mockito; + +public class TestScanTaskParser { + @Test + public void nullCheck() { + assertThatThrownBy(() -> ScanTaskParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid scan task: null"); + + assertThatThrownBy(() -> ScanTaskParser.fromJson(null, true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON string for scan task: null"); + } + + @Test + public void invalidTaskType() { + String jsonStr = "{\"task-type\":\"junk\"}"; + assertThatThrownBy(() -> ScanTaskParser.fromJson(jsonStr, true)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Unknown task type: junk"); + } + + @Test + public void unsupportedTask() { + FileScanTask mockTask = Mockito.mock(FileScanTask.class); + assertThatThrownBy(() -> ScanTaskParser.toJson(mockTask)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageContaining( + "Unsupported task type: org.apache.iceberg.FileScanTask$MockitoMock$"); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java index 44e37afcfc60..344f64833b62 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -32,7 +32,7 @@ import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.FileScanTaskParser; +import org.apache.iceberg.ScanTaskParser; 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.Iterables; @@ -154,7 +154,7 @@ private byte[] serialize(int version) throws IOException { out.writeInt(fileScanTasks.size()); for (FileScanTask fileScanTask : fileScanTasks) { - String taskJson = FileScanTaskParser.toJson(fileScanTask); + String taskJson = ScanTaskParser.toJson(fileScanTask); writeTaskJson(out, taskJson, version); } @@ -199,7 +199,7 @@ private static IcebergSourceSplit deserialize( List tasks = Lists.newArrayListWithCapacity(taskCount); for (int i = 0; i < taskCount; ++i) { String taskJson = readTaskJson(in, version); - FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); + FileScanTask task = ScanTaskParser.fromJson(taskJson, caseSensitive); tasks.add(task); } 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 44e37afcfc60..344f64833b62 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 @@ -32,7 +32,7 @@ import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.FileScanTaskParser; +import org.apache.iceberg.ScanTaskParser; 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.Iterables; @@ -154,7 +154,7 @@ private byte[] serialize(int version) throws IOException { out.writeInt(fileScanTasks.size()); for (FileScanTask fileScanTask : fileScanTasks) { - String taskJson = FileScanTaskParser.toJson(fileScanTask); + String taskJson = ScanTaskParser.toJson(fileScanTask); writeTaskJson(out, taskJson, version); } @@ -199,7 +199,7 @@ private static IcebergSourceSplit deserialize( List tasks = Lists.newArrayListWithCapacity(taskCount); for (int i = 0; i < taskCount; ++i) { String taskJson = readTaskJson(in, version); - FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); + FileScanTask task = ScanTaskParser.fromJson(taskJson, caseSensitive); tasks.add(task); } 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 44e37afcfc60..344f64833b62 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 @@ -32,7 +32,7 @@ import org.apache.iceberg.BaseCombinedScanTask; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.FileScanTaskParser; +import org.apache.iceberg.ScanTaskParser; 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.Iterables; @@ -154,7 +154,7 @@ private byte[] serialize(int version) throws IOException { out.writeInt(fileScanTasks.size()); for (FileScanTask fileScanTask : fileScanTasks) { - String taskJson = FileScanTaskParser.toJson(fileScanTask); + String taskJson = ScanTaskParser.toJson(fileScanTask); writeTaskJson(out, taskJson, version); } @@ -199,7 +199,7 @@ private static IcebergSourceSplit deserialize( List tasks = Lists.newArrayListWithCapacity(taskCount); for (int i = 0; i < taskCount; ++i) { String taskJson = readTaskJson(in, version); - FileScanTask task = FileScanTaskParser.fromJson(taskJson, caseSensitive); + FileScanTask task = ScanTaskParser.fromJson(taskJson, caseSensitive); tasks.add(task); } From 10ca26f7b1d3e07ccc236fa6f80399568b15a163 Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Wed, 26 Jun 2024 15:41:14 +0200 Subject: [PATCH 0414/1019] Azure: Make AzureProperties w/ shared-key creds serializable (#10045) As `StorageSharedKeyCredential` is not serializable, shared key auth doesn't work with Spark. --- .../apache/iceberg/azure/AzureProperties.java | 9 +++---- .../iceberg/azure/AzurePropertiesTest.java | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+), 4 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 8fcb0a098fbf..2d363cbc5231 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java +++ b/azure/src/main/java/org/apache/iceberg/azure/AzureProperties.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Optional; 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.PropertyUtil; public class AzureProperties implements Serializable { @@ -38,7 +39,7 @@ public class AzureProperties implements Serializable { private Map adlsSasTokens = Collections.emptyMap(); private Map adlsConnectionStrings = Collections.emptyMap(); - private StorageSharedKeyCredential namedKeyCreds; + private Map.Entry namedKeyCreds; private Integer adlsReadBlockSize; private Long adlsWriteBlockSize; @@ -57,8 +58,7 @@ public AzureProperties(Map properties) { "Azure authentication: shared-key requires both %s and %s", ADLS_SHARED_KEY_ACCOUNT_NAME, ADLS_SHARED_KEY_ACCOUNT_KEY); - this.namedKeyCreds = - new StorageSharedKeyCredential(sharedKeyAccountName, sharedKeyAccountKey); + this.namedKeyCreds = Maps.immutableEntry(sharedKeyAccountName, sharedKeyAccountKey); } if (properties.containsKey(ADLS_READ_BLOCK_SIZE)) { @@ -82,7 +82,8 @@ public void applyClientConfiguration(String account, DataLakeFileSystemClientBui if (sasToken != null && !sasToken.isEmpty()) { builder.sasToken(sasToken); } else if (namedKeyCreds != null) { - builder.credential(namedKeyCreds); + builder.credential( + new StorageSharedKeyCredential(namedKeyCreds.getKey(), namedKeyCreds.getValue())); } else { builder.credential(new DefaultAzureCredentialBuilder().build()); } 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 32f491119ebc..6b8287c44e58 100644 --- a/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java +++ b/azure/src/test/java/org/apache/iceberg/azure/AzurePropertiesTest.java @@ -18,8 +18,13 @@ */ package org.apache.iceberg.azure; +import static org.apache.iceberg.azure.AzureProperties.ADLS_CONNECTION_STRING_PREFIX; +import static org.apache.iceberg.azure.AzureProperties.ADLS_READ_BLOCK_SIZE; +import static org.apache.iceberg.azure.AzureProperties.ADLS_SAS_TOKEN_PREFIX; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_KEY; import static org.apache.iceberg.azure.AzureProperties.ADLS_SHARED_KEY_ACCOUNT_NAME; +import static org.apache.iceberg.azure.AzureProperties.ADLS_WRITE_BLOCK_SIZE; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.mock; @@ -30,11 +35,30 @@ import com.azure.core.credential.TokenCredential; import com.azure.storage.common.StorageSharedKeyCredential; import com.azure.storage.file.datalake.DataLakeFileSystemClientBuilder; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; public class AzurePropertiesTest { + @Test + public void testSerializable() throws Exception { + AzureProperties props = + new AzureProperties( + ImmutableMap.builder() + .put(ADLS_SAS_TOKEN_PREFIX + "foo", "bar") + .put(ADLS_CONNECTION_STRING_PREFIX + "foo", "bar") + .put(ADLS_READ_BLOCK_SIZE, "42") + .put(ADLS_WRITE_BLOCK_SIZE, "42") + .put(ADLS_SHARED_KEY_ACCOUNT_NAME, "me") + .put(ADLS_SHARED_KEY_ACCOUNT_KEY, "secret") + .build()); + + AzureProperties serdedProps = TestHelpers.roundTripSerialize(props); + assertThat(serdedProps.adlsReadBlockSize()).isEqualTo(props.adlsReadBlockSize()); + assertThat(serdedProps.adlsWriteBlockSize()).isEqualTo(props.adlsWriteBlockSize()); + } + @Test public void testWithSasToken() { AzureProperties props = From b53c5050d2733bbab57135c41895e2fbca811d5e Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 27 Jun 2024 00:28:00 +0800 Subject: [PATCH 0415/1019] Spark 3.5: Parallelize reading files in snapshot and migrate procedures (#10037) --- .../apache/iceberg/actions/MigrateTable.java | 12 ++ .../apache/iceberg/actions/SnapshotTable.java | 12 ++ .../iceberg/data/TableMigrationUtil.java | 57 +++++-- docs/docs/spark-procedures.md | 4 +- .../extensions/TestMigrateTableProcedure.java | 42 ++++++ .../TestSnapshotTableProcedure.java | 39 +++++ .../apache/iceberg/spark/SparkTableUtil.java | 139 +++++++++++++++++- .../actions/MigrateTableSparkAction.java | 11 +- .../actions/SnapshotTableSparkAction.java | 11 +- .../procedures/MigrateTableProcedure.java | 10 +- .../procedures/SnapshotTableProcedure.java | 9 +- .../spark/actions/TestMigrateTableAction.java | 68 +++++++++ .../actions/TestSnapshotTableAction.java | 68 +++++++++ 13 files changed, 459 insertions(+), 23 deletions(-) create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java diff --git a/api/src/main/java/org/apache/iceberg/actions/MigrateTable.java b/api/src/main/java/org/apache/iceberg/actions/MigrateTable.java index 5438c4b65a46..8f9e8d69c90a 100644 --- a/api/src/main/java/org/apache/iceberg/actions/MigrateTable.java +++ b/api/src/main/java/org/apache/iceberg/actions/MigrateTable.java @@ -19,6 +19,7 @@ package org.apache.iceberg.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; /** An action that migrates an existing table to Iceberg. */ public interface MigrateTable extends Action { @@ -60,6 +61,17 @@ default MigrateTable backupTableName(String tableName) { throw new UnsupportedOperationException("Backup table name cannot be specified"); } + /** + * Sets the executor service to use for parallel file reading. The default is not using executor + * service. + * + * @param service executor service + * @return this for method chaining + */ + default MigrateTable executeWith(ExecutorService service) { + throw new UnsupportedOperationException("Setting executor service is not supported"); + } + /** The action result that contains a summary of the execution. */ interface Result { /** Returns the number of migrated data files. */ diff --git a/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java b/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java index 37c600ab0392..a28e94bcdbe5 100644 --- a/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java +++ b/api/src/main/java/org/apache/iceberg/actions/SnapshotTable.java @@ -19,6 +19,7 @@ package org.apache.iceberg.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; /** An action that creates an independent snapshot of an existing table. */ public interface SnapshotTable extends Action { @@ -57,6 +58,17 @@ public interface SnapshotTable extends Action listPartition( MetricsConfig metricsSpec, NameMapping mapping, int parallelism) { - ExecutorService service = null; + return listPartition( + partition, + partitionUri, + format, + spec, + conf, + metricsSpec, + mapping, + migrationService(parallelism)); + } + + /** + * Returns the data files in a partition by listing the partition location. Metrics are read from + * the files and the file reading is done in parallel by a specified number of threads. + * + *

    For Parquet and ORC partitions, this will read metrics from the file footer. For Avro + * partitions, metrics other than row count are set to null. + * + *

    Note: certain metrics, like NaN counts, that are only supported by Iceberg file writers but + * not file footers, will not be populated. + * + * @param partition map of column names to column values for the partition + * @param partitionUri partition location URI + * @param format partition format, avro, parquet or orc + * @param spec a partition spec + * @param conf a Hadoop conf + * @param metricsSpec a metrics conf + * @param mapping a name mapping + * @param service executor service to use for file reading + * @return a List of DataFile + */ + public static List listPartition( + Map partition, + String partitionUri, + String format, + PartitionSpec spec, + Configuration conf, + MetricsConfig metricsSpec, + NameMapping mapping, + ExecutorService service) { try { List partitionValues = spec.fields().stream() @@ -130,8 +166,7 @@ public static List listPartition( Tasks.Builder task = Tasks.range(fileStatus.size()).stopOnFailure().throwFailureWhenFinished(); - if (parallelism > 1) { - service = migrationService(parallelism); + if (service != null) { task.executeWith(service); } @@ -215,11 +250,7 @@ private static DataFile buildDataFile( .build(); } - private static ExecutorService migrationService(int parallelism) { - return MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) - Executors.newFixedThreadPool( - parallelism, - new ThreadFactoryBuilder().setNameFormat("table-migration-%d").build())); + public static ExecutorService migrationService(int parallelism) { + return parallelism == 1 ? null : ThreadPools.newWorkerPool("table-migration", parallelism); } } diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index dc439c04c855..31172fb531bd 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -546,6 +546,7 @@ See [`migrate`](#migrate) to replace an existing table with an Iceberg table. | `table` | ✔️ | string | Name of the new Iceberg table to create | | `location` | | string | Table location for the new table (delegated to the catalog by default) | | `properties` | ️ | map | Properties to add to the newly created table | +| `parallelism` | | int | Number of threads to use for file reading (defaults to 1) | #### Output @@ -588,6 +589,7 @@ By default, the original table is retained with the name `table_BACKUP_`. | `properties` | ️ | map | Properties for the new Iceberg table | | `drop_backup` | | boolean | When true, the original table will not be retained as backup (defaults to false) | | `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) | #### Output @@ -629,7 +631,7 @@ will then treat these files as if they are part of the set of files owned by Ic | `source_table` | ✔️ | string | Table where files should come from, paths are also possible in the form of \`file_format\`.\`path\` | | `partition_filter` | ️ | map | A map of partitions in the source table to import from | | `check_duplicate_files` | ️ | boolean | Whether to prevent files existing in the table from being added (defaults to true) | -| `parallelism` | | int | number of threads to use for file reading (defaults to 1) | +| `parallelism` | | int | Number of threads to use for file reading (defaults to 1) | Warning : Schema is not validated, adding files with different schema to the Iceberg table will cause issues. 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 735a3bdee863..23c08b2572f4 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 @@ -24,6 +24,7 @@ import java.io.IOException; import java.nio.file.Files; +import java.util.List; import java.util.Map; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; @@ -231,4 +232,45 @@ public void testMigrateEmptyTable() throws Exception { Object result = scalarSql("CALL %s.system.migrate('%s')", catalogName, tableName); assertThat(result).isEqualTo(0L); } + + @TestTemplate + public void testMigrateWithParallelism() 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 LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (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(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testMigrateWithInvalidParallelism() 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 LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.migrate(table => '%s', parallelism => %d)", + catalogName, tableName, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be larger than 0"); + } } 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 3b093947cacc..cb184043490c 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 @@ -24,6 +24,7 @@ import java.io.IOException; import java.nio.file.Files; +import java.util.List; import java.util.Map; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Table; @@ -223,4 +224,42 @@ public void testInvalidSnapshotsCases() throws IOException { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot handle an empty identifier for argument table"); } + + @TestTemplate + public void testSnapshotWithParallelism() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + sourceName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", sourceName); + + List result = + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", + catalogName, sourceName, tableName, 2); + assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("SELECT * FROM %s ORDER BY id", tableName)); + } + + @TestTemplate + public void testSnapshotWithInvalidParallelism() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + sourceName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", sourceName); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", + catalogName, sourceName, tableName, -1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be larger than 0"); + } } 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 eb27e1483d13..8447dbdcead1 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -290,6 +291,24 @@ private static List listPartition( parallelism); } + private static List listPartition( + SparkPartition partition, + PartitionSpec spec, + SerializableConfiguration conf, + MetricsConfig metricsConfig, + NameMapping mapping, + ExecutorService service) { + return TableMigrationUtil.listPartition( + partition.values, + partition.uri, + partition.format, + spec, + conf.get(), + metricsConfig, + mapping, + service); + } + private static SparkPartition toSparkPartition( CatalogTablePartition partition, CatalogTable table) { Option locationUri = partition.storage().locationUri(); @@ -388,6 +407,54 @@ public static void importSparkTable( spark, sourceTableIdent, targetTable, stagingDir, partitionFilter, checkDuplicateFiles, 1); } + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param parallelism number of threads to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + ExecutorService service) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, service); + } + /** * Import files from an existing Spark table to an Iceberg table. * @@ -411,6 +478,39 @@ public static void importSparkTable( Map partitionFilter, boolean checkDuplicateFiles, int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + partitionFilter, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + ExecutorService service) { SessionCatalog catalog = spark.sessionState().catalog(); String db = @@ -431,7 +531,7 @@ public static void importSparkTable( if (Objects.equal(spec, PartitionSpec.unpartitioned())) { importUnpartitionedSparkTable( - spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, parallelism); + spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, service); } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); @@ -445,7 +545,7 @@ public static void importSparkTable( spec, stagingDir, checkDuplicateFiles, - parallelism); + service); } } } catch (AnalysisException e) { @@ -504,7 +604,7 @@ private static void importUnpartitionedSparkTable( TableIdentifier sourceTableIdent, Table targetTable, boolean checkDuplicateFiles, - int parallelism) { + ExecutorService service) { try { CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); Option format = @@ -530,7 +630,7 @@ private static void importUnpartitionedSparkTable( conf, metricsConfig, nameMapping, - parallelism); + service); if (checkDuplicateFiles) { Dataset importedFiles = @@ -600,6 +700,35 @@ public static void importSparkPartitions( String stagingDir, boolean checkDuplicateFiles, int parallelism) { + importSparkPartitions( + spark, + partitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + ExecutorService service) { Configuration conf = spark.sessionState().newHadoopConf(); SerializableConfiguration serializableConf = new SerializableConfiguration(conf); int listingParallelism = @@ -627,7 +756,7 @@ public static void importSparkPartitions( serializableConf, metricsConfig, nameMapping, - parallelism) + service) .iterator(), Encoders.javaSerialization(DataFile.class)); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java index 5f3cdd3f035c..bdffeb465405 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; @@ -59,6 +60,7 @@ public class MigrateTableSparkAction extends BaseTableCreationSparkAction 0, "Parallelism should be larger than 0"); + migrateTableSparkAction = + migrateTableSparkAction.executeWith(executorService(parallelism, "table-migration")); + } + MigrateTable.Result result = migrateTableSparkAction.execute(); return new InternalRow[] {newInternalRow(result.migratedDataFilesCount())}; } 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 7a015a51e8ed..f709f64ebf62 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 @@ -38,7 +38,8 @@ class SnapshotTableProcedure extends BaseProcedure { ProcedureParameter.required("source_table", DataTypes.StringType), ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("location", DataTypes.StringType), - ProcedureParameter.optional("properties", STRING_MAP) + ProcedureParameter.optional("properties", STRING_MAP), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -102,6 +103,12 @@ public InternalRow[] call(InternalRow args) { action.tableLocation(snapshotLocation); } + 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")); + } + SnapshotTable.Result result = action.tableProperties(properties).execute(); return new InternalRow[] {newInternalRow(result.importedDataFilesCount())}; } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java new file mode 100644 index 000000000000..94afa50cf4b8 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java @@ -0,0 +1,68 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.CatalogTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestMigrateTableAction extends CatalogTestBase { + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @TestTemplate + public void testMigrateWithParallelTasks() 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 LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + AtomicInteger migrationThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .migrateTable(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-migration-" + migrationThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + assertThat(migrationThreadsIndex.get()).isEqualTo(2); + } +} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java new file mode 100644 index 000000000000..3b6869c397a5 --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -0,0 +1,68 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.spark.CatalogTestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestSnapshotTableAction extends CatalogTestBase { + private static final String sourceName = "spark_catalog.default.source"; + + @AfterEach + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", sourceName); + } + + @TestTemplate + public void testSnapshotWithParallelTasks() throws IOException { + String location = Files.createTempDirectory(temp, "junit").toFile().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + sourceName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", sourceName); + + AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .snapshotTable(sourceName) + .as(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-snapshot-" + snapshotThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + assertThat(snapshotThreadsIndex.get()).isEqualTo(2); + } +} From 8a8381ca1097fe3044a8dec0dc5dd564d6741df3 Mon Sep 17 00:00:00 2001 From: edson duarte Date: Wed, 26 Jun 2024 23:37:06 -0300 Subject: [PATCH 0416/1019] Docs: Add BigQuery docs url to sidebar (#10574) --- docs/mkdocs.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 10479067770f..5e43aa1d26d9 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -59,6 +59,7 @@ nav: - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog - Amazon Athena: https://docs.aws.amazon.com/athena/latest/ug/querying-iceberg.html - Amazon EMR: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html + - Google BigQuery: https://cloud.google.com/bigquery/docs/iceberg-tables - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - Doris: https://doris.apache.org/docs/dev/lakehouse/datalake-analytics/iceberg From 88c47515e266e3a08ee89a9b2c17fa0530f8c9dc Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 28 Jun 2024 09:04:43 +0200 Subject: [PATCH 0417/1019] Build: Run CI checks on all supported JDKs (#10473) This guarantees the build will succeed locally. --- .github/workflows/java-ci.yml | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index bc4b01ed2203..22cc60d2262e 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -90,20 +90,26 @@ jobs: build-checks: runs-on: ubuntu-22.04 + strategy: + matrix: + jvm: [8, 11, 17] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: zulu - java-version: 8 + java-version: ${{ matrix.jvm }} - run: ./gradlew -DallModules build -x test -x javadoc -x integrationTest build-javadoc: runs-on: ubuntu-22.04 + strategy: + matrix: + jvm: [8, 11, 17] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 with: distribution: zulu - java-version: 8 + java-version: ${{ matrix.jvm }} - run: ./gradlew -Pquick=true javadoc From b120904d464194d051c7e5a0064b9e37a4bf869b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 28 Jun 2024 11:32:05 +0200 Subject: [PATCH 0418/1019] Common: DynConstructors cleanup (#10542) The `DynConstructors.Builder.hiddenImpl(Class...)` is a varargs method, but cannot be used as such, because `hiddenImpl(Class, Class...)` exists and would be used instead. Since there are no usages, deprecate it instead of fixing. --- .../iceberg/common/DynConstructors.java | 34 +++++---- .../iceberg/common/TestDynConstructors.java | 75 +++++++++++++++++++ 2 files changed, 95 insertions(+), 14 deletions(-) create mode 100644 common/src/test/java/org/apache/iceberg/common/TestDynConstructors.java diff --git a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java index 4761be4f3e32..7c777112871a 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java +++ b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java @@ -43,6 +43,8 @@ private Ctor(Constructor constructor, Class constructed) { this.constructed = constructed; } + /** @deprecated since 1.6.0, will be removed in 1.7.0 */ + @Deprecated public Class getConstructedClass() { return constructed; } @@ -57,9 +59,9 @@ public C newInstanceChecked(Object... args) throws Exception { } catch (InstantiationException | IllegalAccessException e) { throw e; } catch (InvocationTargetException e) { - Throwables.propagateIfInstanceOf(e.getCause(), Exception.class); - Throwables.propagateIfInstanceOf(e.getCause(), RuntimeException.class); - throw Throwables.propagate(e.getCause()); + Throwables.throwIfInstanceOf(e.getCause(), Exception.class); + Throwables.throwIfInstanceOf(e.getCause(), RuntimeException.class); + throw new RuntimeException(e.getCause()); } } @@ -67,8 +69,8 @@ public C newInstance(Object... args) { try { return newInstanceChecked(args); } catch (Exception e) { - Throwables.propagateIfInstanceOf(e, RuntimeException.class); - throw Throwables.propagate(e); + Throwables.throwIfInstanceOf(e, RuntimeException.class); + throw new RuntimeException(e.getCause()); } } @@ -115,8 +117,8 @@ public static Builder builder(Class baseClass) { public static class Builder { private final Class baseClass; private ClassLoader loader = Thread.currentThread().getContextClassLoader(); - private Ctor ctor = null; - private Map problems = Maps.newHashMap(); + private Ctor ctor = null; + private final Map problems = Maps.newHashMap(); public Builder(Class baseClass) { this.baseClass = baseClass; @@ -162,7 +164,7 @@ public Builder impl(Class targetClass, Class... types) { } try { - ctor = new Ctor(targetClass.getConstructor(types), targetClass); + ctor = new Ctor<>(targetClass.getConstructor(types), targetClass); } catch (NoSuchMethodException e) { // not the right implementation problems.put(methodName(targetClass, types), e); @@ -170,12 +172,16 @@ public Builder impl(Class targetClass, Class... types) { return this; } + /** + * @deprecated since 1.6.0, will be removed in 1.7.0; This varargs method conflicts with {@link + * #hiddenImpl(Class, Class...)}. Use {@link #builder(Class)} instead. + */ + @Deprecated public Builder hiddenImpl(Class... types) { hiddenImpl(baseClass, types); return this; } - @SuppressWarnings("unchecked") public Builder hiddenImpl(String className, Class... types) { // don't do any work if an implementation has been found if (ctor != null) { @@ -183,7 +189,7 @@ public Builder hiddenImpl(String className, Class... types) { } try { - Class targetClass = Class.forName(className, true, loader); + Class targetClass = Class.forName(className, true, loader); hiddenImpl(targetClass, types); } catch (NoClassDefFoundError | ClassNotFoundException e) { // cannot load this implementation @@ -201,7 +207,7 @@ public Builder hiddenImpl(Class targetClass, Class... types) { try { Constructor hidden = targetClass.getDeclaredConstructor(types); AccessController.doPrivileged(new MakeAccessible(hidden)); - ctor = new Ctor(hidden, targetClass); + ctor = new Ctor<>(hidden, targetClass); } catch (SecurityException e) { // unusable problems.put(methodName(targetClass, types), e); @@ -215,7 +221,7 @@ public Builder hiddenImpl(Class targetClass, Class... types) { @SuppressWarnings("unchecked") public Ctor buildChecked() throws NoSuchMethodException { if (ctor != null) { - return ctor; + return (Ctor) ctor; } throw buildCheckedException(baseClass, problems); } @@ -223,14 +229,14 @@ public Ctor buildChecked() throws NoSuchMethodException { @SuppressWarnings("unchecked") public Ctor build() { if (ctor != null) { - return ctor; + return (Ctor) ctor; } throw buildRuntimeException(baseClass, problems); } } private static class MakeAccessible implements PrivilegedAction { - private Constructor hidden; + private final Constructor hidden; MakeAccessible(Constructor hidden) { this.hidden = hidden; diff --git a/common/src/test/java/org/apache/iceberg/common/TestDynConstructors.java b/common/src/test/java/org/apache/iceberg/common/TestDynConstructors.java new file mode 100644 index 000000000000..baddcc8e2fd7 --- /dev/null +++ b/common/src/test/java/org/apache/iceberg/common/TestDynConstructors.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.common; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.junit.jupiter.api.Test; + +public class TestDynConstructors { + @Test + public void testImplNewInstance() throws Exception { + DynConstructors.Ctor ctor = + DynConstructors.builder().impl(MyClass.class).buildChecked(); + assertThat(ctor.newInstance()).isInstanceOf(MyClass.class); + } + + @Test + public void testInterfaceImplNewInstance() throws Exception { + DynConstructors.Ctor ctor = + DynConstructors.builder(MyInterface.class) + .impl("org.apache.iceberg.common.TestDynConstructors$MyClass") + .buildChecked(); + assertThat(ctor.newInstance()).isInstanceOf(MyClass.class); + } + + @Test + public void testInterfaceWrongImplString() throws Exception { + DynConstructors.Ctor ctor = + DynConstructors.builder(MyInterface.class) + // TODO this should throw, since the MyUnrelatedClass does not implement MyInterface + .impl("org.apache.iceberg.common.TestDynConstructors$MyUnrelatedClass") + .buildChecked(); + assertThatThrownBy(ctor::newInstance) + .isInstanceOf(ClassCastException.class) + .hasMessage( + "org.apache.iceberg.common.TestDynConstructors$MyUnrelatedClass cannot be cast to org.apache.iceberg.common.TestDynConstructors$MyInterface"); + } + + @Test + public void testInterfaceWrongImplClass() throws Exception { + DynConstructors.Ctor ctor = + DynConstructors.builder(MyInterface.class) + // TODO this should throw or not compile at all, since the MyUnrelatedClass does not + // implement MyInterface + .impl(MyUnrelatedClass.class) + .buildChecked(); + assertThatThrownBy(ctor::newInstance) + .isInstanceOf(ClassCastException.class) + .hasMessage( + "org.apache.iceberg.common.TestDynConstructors$MyUnrelatedClass cannot be cast to org.apache.iceberg.common.TestDynConstructors$MyInterface"); + } + + public interface MyInterface {} + + public static class MyClass implements MyInterface {} + + public static class MyUnrelatedClass {} +} From 94a423d2527b4d7f02180e68f1e7d622f847e0b0 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 28 Jun 2024 12:30:13 +0200 Subject: [PATCH 0419/1019] Fix CI script inclusion of release branches (#10514) The CI scripts are meant to run on release maintenance branches such as 1.5.x. They were run for releases up to 1.0 because of the `0.*` pattern This commit: - includes `1.*` branches (but that's likely needs to be propagated to existing branches) - includes `2.*` branches foreseeing imminent 2.0 release - changes the `0.**` pattern to `0.*`. Multiple asterisks are needed to match branch names including `/` but release branche naming convention do not expect such branch names. --- .github/workflows/api-binary-compatibility.yml | 4 +++- .github/workflows/delta-conversion-ci.yml | 4 +++- .github/workflows/flink-ci.yml | 4 +++- .github/workflows/hive-ci.yml | 4 +++- .github/workflows/java-ci.yml | 4 +++- .github/workflows/open-api.yml | 4 +++- .github/workflows/spark-ci.yml | 4 +++- 7 files changed, 21 insertions(+), 7 deletions(-) diff --git a/.github/workflows/api-binary-compatibility.yml b/.github/workflows/api-binary-compatibility.yml index 80dc0da8856c..fbdb740e6e72 100644 --- a/.github/workflows/api-binary-compatibility.yml +++ b/.github/workflows/api-binary-compatibility.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index c89fbb6060af..90c1ad4109fd 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index e360e0f2934b..b74cbcc84291 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index cd201a09d0fa..6e97e2164758 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 22cc60d2262e..083ae2f5fe36 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: diff --git a/.github/workflows/open-api.yml b/.github/workflows/open-api.yml index 397d1bf30e4a..40aefc989244 100644 --- a/.github/workflows/open-api.yml +++ b/.github/workflows/open-api.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 5ddff7a506dd..d923cf860f1d 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -22,7 +22,9 @@ on: push: branches: - 'main' - - '0.**' + - '0.*' + - '1.*' + - '2.*' tags: - 'apache-iceberg-**' pull_request: From bb312af83d81b6064c77f7463ef6888d6a9435ba Mon Sep 17 00:00:00 2001 From: Tai Le Manh <49281946+tlm365@users.noreply.github.com> Date: Sat, 29 Jun 2024 21:22:01 +0700 Subject: [PATCH 0420/1019] Fix incorrect double-checked-locking around TestStreamScanSql#tEnv (#10605) Signed-off-by: Tai Le Manh --- .../flink/source/TestStreamScanSql.java | 40 +++++++++--------- .../flink/source/TestStreamScanSql.java | 40 +++++++++--------- .../flink/source/TestStreamScanSql.java | 42 ++++++++++--------- 3 files changed, 64 insertions(+), 58 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 581e063bb279..d6cf6791270e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -59,25 +59,27 @@ public class TestStreamScanSql extends CatalogTestBase { @Override protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = - EnvironmentSettings.newInstance().inStreamingMode(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - - StreamTableEnvironment streamTableEnv = - StreamTableEnvironment.create(env, settingsBuilder.build()); - streamTableEnv - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - tEnv = streamTableEnv; - } + TableEnvironment tableEnv = tEnv; + if (tableEnv != null) { + return tableEnv; + } + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = + EnvironmentSettings.newInstance().inStreamingMode(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + + StreamTableEnvironment streamTableEnv = + StreamTableEnvironment.create(env, settingsBuilder.build()); + streamTableEnv + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + tEnv = streamTableEnv; } } return tEnv; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 581e063bb279..d6cf6791270e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -59,25 +59,27 @@ public class TestStreamScanSql extends CatalogTestBase { @Override protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = - EnvironmentSettings.newInstance().inStreamingMode(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - - StreamTableEnvironment streamTableEnv = - StreamTableEnvironment.create(env, settingsBuilder.build()); - streamTableEnv - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - tEnv = streamTableEnv; - } + TableEnvironment tableEnv = tEnv; + if (tableEnv != null) { + return tableEnv; + } + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = + EnvironmentSettings.newInstance().inStreamingMode(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + + StreamTableEnvironment streamTableEnv = + StreamTableEnvironment.create(env, settingsBuilder.build()); + streamTableEnv + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + tEnv = streamTableEnv; } } return tEnv; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index 78090c55387a..fcf5c1479df5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -55,29 +55,31 @@ public class TestStreamScanSql extends CatalogTestBase { private static final String TABLE = "test_table"; private static final FileFormat FORMAT = FileFormat.PARQUET; - private TableEnvironment tEnv; + private volatile TableEnvironment tEnv; @Override protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = - EnvironmentSettings.newInstance().inStreamingMode(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - - StreamTableEnvironment streamTableEnv = - StreamTableEnvironment.create(env, settingsBuilder.build()); - streamTableEnv - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - tEnv = streamTableEnv; - } + TableEnvironment tableEnv = tEnv; + if (tableEnv != null) { + return tableEnv; + } + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = + EnvironmentSettings.newInstance().inStreamingMode(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + + StreamTableEnvironment streamTableEnv = + StreamTableEnvironment.create(env, settingsBuilder.build()); + streamTableEnv + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + tEnv = streamTableEnv; } } return tEnv; From aeeb333de160e007de672e0265efbc27918dd3ba Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 30 Jun 2024 15:15:21 +0200 Subject: [PATCH 0421/1019] Build: Bump nessie from 0.91.2 to 0.91.3 (#10608) Bumps `nessie` from 0.91.2 to 0.91.3. Updates `org.projectnessie.nessie:nessie-client` from 0.91.2 to 0.91.3 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.91.2 to 0.91.3 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.91.2 to 0.91.3 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.91.2 to 0.91.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 0c22fae3fd81..9fd25accfd7a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.91.2" +nessie = "0.91.3" netty-buffer = "4.1.111.Final" netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" From 3916e35128eb59cd15655d09858feeb21e87383f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 30 Jun 2024 19:12:33 +0200 Subject: [PATCH 0422/1019] Build: Bump software.amazon.awssdk:bom from 2.26.7 to 2.26.12 (#10611) Bumps software.amazon.awssdk:bom from 2.26.7 to 2.26.12. --- 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 9fd25accfd7a..3d11f3a3eafc 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.26.7" +awssdk-bom = "2.26.12" azuresdk-bom = "1.2.24" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From df328c5437be5a80a00c75d8150cdf559c94e5a3 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Mon, 1 Jul 2024 22:24:02 +0900 Subject: [PATCH 0423/1019] Flink: Migrate HadoopCatalog related tests (#10358) --- .../iceberg/flink/HadoopTableExtension.java | 59 +++++++ .../flink/sink/TestFlinkIcebergSink.java | 148 ++++++++---------- .../sink/TestFlinkIcebergSinkBranch.java | 66 ++++---- .../flink/sink/TestFlinkIcebergSinkV2.java | 111 +++++-------- .../sink/TestFlinkIcebergSinkV2Base.java | 42 ++++- .../sink/TestFlinkIcebergSinkV2Branch.java | 69 ++++---- .../flink/source/TestFlinkSourceSql.java | 4 +- .../flink/source/TestIcebergSourceSql.java | 4 +- .../iceberg/flink/source/TestSqlBase.java | 46 +++--- .../flink/source/reader/ReaderUtil.java | 22 +++ .../TestColumnStatsWatermarkExtractor.java | 90 ++++++----- 11 files changed, 356 insertions(+), 305 deletions(-) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java new file mode 100644 index 000000000000..dc6ef400a4a9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.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; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopTableExtension extends HadoopCatalogExtension { + private final Schema schema; + private final PartitionSpec partitionSpec; + + private Table table; + + public HadoopTableExtension(String database, String tableName, Schema schema) { + this(database, tableName, schema, null); + } + + public HadoopTableExtension( + String database, String tableName, Schema schema, PartitionSpec partitionSpec) { + super(database, tableName); + this.schema = schema; + this.partitionSpec = partitionSpec; + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + super.beforeEach(context); + if (partitionSpec == null) { + this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); + } else { + this.table = + catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); + } + tableLoader.open(); + } + + public Table table() { + return table; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 9ff79419b020..527525e9f167 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; @@ -29,17 +31,21 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; 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.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -47,59 +53,52 @@ 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.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private TableLoader tableLoader; - private final FileFormat format; - private final int parallelism; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; - @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") public static Object[][] parameters() { return new Object[][] { - {"avro", 1, true}, - {"avro", 1, false}, - {"avro", 2, true}, - {"avro", 2, false}, - {"orc", 1, true}, - {"orc", 1, false}, - {"orc", 2, true}, - {"orc", 2, false}, - {"parquet", 1, true}, - {"parquet", 1, false}, - {"parquet", 2, true}, - {"parquet", 2, false} + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} }; } - public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -122,7 +121,7 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testWriteRowData() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = @@ -165,17 +164,17 @@ private int partitionFiles(String partition) throws IOException { return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); } - @Test + @TestTemplate public void testWriteRow() throws Exception { testWriteRow(null, DistributionMode.NONE); } - @Test + @TestTemplate public void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } - @Test + @TestTemplate public void testJobNoneDistributeMode() throws Exception { table .updateProperties() @@ -187,12 +186,12 @@ public void testJobNoneDistributeMode() throws Exception { if (parallelism > 1) { if (partitioned) { int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); + assertThat(files).isGreaterThan(3); } } } - @Test + @TestTemplate public void testJobHashDistributionMode() { table .updateProperties() @@ -204,7 +203,7 @@ public void testJobHashDistributionMode() { .hasMessage("Flink does not support 'range' write distribution mode now."); } - @Test + @TestTemplate public void testJobNullDistributionMode() throws Exception { table .updateProperties() @@ -214,42 +213,33 @@ public void testJobNullDistributionMode() throws Exception { testWriteRow(null, null); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testPartitionWriteMode() throws Exception { testWriteRow(null, DistributionMode.HASH); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testShuffleByPartitionWithSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); @@ -323,16 +313,14 @@ public void testTwoSinksInDisjointedDAG() throws Exception { SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); leftTable.refresh(); - Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); - Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); rightTable.refresh(); - Assert.assertEquals( - TestFlinkIcebergSink.class.getName(), - rightTable.currentSnapshot().summary().get("flink.test")); - Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); } - @Test + @TestTemplate public void testOverrideWriteConfigWithUnknownDistributionMode() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); @@ -352,7 +340,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } - @Test + @TestTemplate public void testOverrideWriteConfigWithUnknownFileFormat() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); @@ -372,7 +360,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .hasMessage("Invalid file format: UNRECOGNIZED"); } - @Test + @TestTemplate public void testWriteRowWithTableRefreshInterval() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 16b4542b00d3..547b4937c5bd 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -18,60 +18,60 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; 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; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { + @RegisterExtension + public static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @Parameter(index = 0) + private String formatVersion; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @Parameter(index = 1) + private String branch; - private final String branch; private TableLoader tableLoader; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkBranch(String branch) { - this.branch = branch; + @Parameters(name = "formatVersion = {0}, branch = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"1", "main"}, + {"1", "testBranch"}, + {"2", "main"}, + {"2", "testBranch"} + }; } - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -84,7 +84,7 @@ public void before() throws IOException { TableProperties.DEFAULT_FILE_FORMAT, FileFormat.AVRO.name(), TableProperties.FORMAT_VERSION, - "1")); + formatVersion)); env = StreamExecutionEnvironment.getExecutionEnvironment( @@ -94,7 +94,7 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); verifyOtherBranchUnmodified(); @@ -129,9 +129,9 @@ private void verifyOtherBranchUnmodified() { String otherBranch = branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); } - Assert.assertTrue(table.snapshot(otherBranch) == null); + assertThat(table.snapshot(otherBranch)).isNull(); } } 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 7712481d33d9..9cbb9f091e15 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 @@ -18,85 +18,53 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; 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.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Rule public final Timeout globalTimeout = Timeout.seconds(60); - - @Parameterized.Parameters( - name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} - }; - } - - public TestFlinkIcebergSinkV2( - String format, int parallelism, boolean partitioned, String writeDistributionMode) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - this.writeDistributionMode = writeDistributionMode; - } + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Before + @BeforeEach public void setupTable() { table = catalogResource @@ -129,7 +97,7 @@ public void setupTable() { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testCheckAndGetEqualityFieldIds() { table .updateSchema() @@ -144,28 +112,25 @@ public void testCheckAndGetEqualityFieldIds() { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); // Use schema identifier field IDs as equality field id list by default - Assert.assertEquals( - table.schema().identifierFieldIds(), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrderElementsOf(table.schema().identifierFieldIds()); // Use user-provided equality field column as equality field id list builder.equalityFieldColumns(Lists.newArrayList("id")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("id").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); builder.equalityFieldColumns(Lists.newArrayList("type")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("type").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("type").fieldId()); } - @Test + @TestTemplate public void testChangeLogOnIdKey() throws Exception { testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnlyDeletesOnDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -184,22 +149,22 @@ public void testUpsertOnlyDeletesOnDataKey() throws Exception { SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnSameKey() throws Exception { testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertModeCheck() throws Exception { DataStream dataStream = env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); @@ -227,22 +192,22 @@ public void testUpsertModeCheck() throws Exception { "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); } - @Test + @TestTemplate public void testUpsertOnIdKey() throws Exception { testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnDataKey() throws Exception { testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testDeleteStats() throws Exception { assumeThat(format).isNotEqualTo(FileFormat.AVRO); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 9cdf7743c485..fc33c2fea5e6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; @@ -31,6 +32,8 @@ import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -44,7 +47,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; public class TestFlinkIcebergSinkV2Base { @@ -55,14 +57,40 @@ public class TestFlinkIcebergSinkV2Base { protected static final int ROW_ID_POS = 0; protected static final int ROW_DATA_POS = 1; - protected int parallelism = 1; protected TableLoader tableLoader; protected Table table; protected StreamExecutionEnvironment env; + + @Parameter(index = 0) protected FileFormat format; + + @Parameter(index = 1) + protected int parallelism = 1; + + @Parameter(index = 2) protected boolean partitioned; + + @Parameter(index = 3) protected String writeDistributionMode; + @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.ORC, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.PARQUET, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} + }; + } + protected static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, @@ -319,16 +347,14 @@ protected void testChangeLogs( table.refresh(); List snapshots = findValidSnapshots(); int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + assertThat(snapshots).hasSize(expectedSnapshotNum); for (int i = 0; i < expectedSnapshotNum; i++) { long snapshotId = snapshots.get(i).snapshotId(); List expectedRecords = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(expectedRecords.toArray(new Record[0])), - actualRowSet(snapshotId, "*")); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); } } 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 fed333848279..1c5c97b58d2d 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 @@ -18,52 +18,43 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; 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; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "branch = {0}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } + @Parameter(index = 0) + private String branch; - public TestFlinkIcebergSinkV2Branch(String branch) { - this.branch = branch; + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; } - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -86,37 +77,37 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testChangeLogOnIdKey() throws Exception { testChangeLogOnIdKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { testChangeLogOnDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { testChangeLogOnIdDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnIdKey() throws Exception { testUpsertOnIdKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnDataKey() throws Exception { testUpsertOnDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(branch); verifyOtherBranchUnmodified(); @@ -126,9 +117,9 @@ private void verifyOtherBranchUnmodified() { String otherBranch = branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); } - Assert.assertTrue(table.snapshot(otherBranch) == null); + assertThat(table.snapshot(otherBranch)).isNull(); } } 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 affd90c347dd..6857e0a7a366 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,7 +33,7 @@ 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.Test; +import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { @@ -61,7 +61,7 @@ public void testInferParallelismWithGlobalSetting() throws IOException { .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); long maxFileLen = 0; for (int i = 0; i < 5; i++) { 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 4250460d278d..645af7cfa339 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,7 +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.Test; +import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ public class TestIcebergSourceSql extends TestSqlBase { @@ -78,7 +78,7 @@ private List generateExpectedRecords(boolean ascending) throws Exception long baseTime = 1702382109000L; GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); Record file1Record1 = generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index dda46033143e..94962e02bb05 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; @@ -34,30 +38,28 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TableLoader; 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.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @TempDir protected Path temporaryFolder; private volatile TableEnvironment tEnv; @@ -73,7 +75,7 @@ protected TableEnvironment getTableEnv() { return tEnv; } - @Before + @BeforeEach public abstract void before() throws IOException; @Test @@ -90,7 +92,7 @@ public void testResiduals() throws Exception { writeRecords.get(1).set(2, "2020-03-20"); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); expectedRecords.add(writeRecords.get(0)); @@ -120,7 +122,7 @@ public void testExposeLocality() throws Exception { expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); helper.appendToTable(dataFile); @@ -140,9 +142,9 @@ public void testExposeLocality() throws Exception { // When running with CI or local, `localityEnabled` will be false even if this configuration is // enabled - Assert.assertFalse( - "Expose split locality info should be false.", - SourceUtil.isLocalityEnabled(table, tableConf, true)); + assertThat(SourceUtil.isLocalityEnabled(table, tableConf, true)) + .as("Expose split locality info should be false.") + .isFalse(); results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); org.apache.iceberg.flink.TestHelpers.assertRecords( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f06d9b83bcd4..e3e341ca2c76 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import org.apache.flink.table.data.RowData; @@ -106,6 +107,7 @@ public static List> createRecordBatchList( return Lists.partition(records, batchCount); } + // Only for JUnit4 tests. Keep this method for test migration from JUnit4 to JUnit5 public static CombinedScanTask createCombinedScanTask( List> recordBatchList, TemporaryFolder temporaryFolder, @@ -122,4 +124,24 @@ public static CombinedScanTask createCombinedScanTask( return new BaseCombinedScanTask(fileTasks); } + + public static CombinedScanTask createCombinedScanTask( + List> recordBatchList, + Path temporaryFolder, + FileFormat fileFormat, + GenericAppenderFactory appenderFactory) + throws IOException { + List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); + for (List recordBatch : recordBatchList) { + FileScanTask fileTask = + ReaderUtil.createFileTask( + recordBatch, + File.createTempFile("junit", null, temporaryFolder.toFile()), + fileFormat, + appenderFactory); + fileTasks.add(fileTask); + } + + return new BaseCombinedScanTask(fileTasks); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 604bc09619e0..7033fd30e84f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -18,10 +18,14 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; 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.IOException; +import java.nio.file.Path; import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -30,27 +34,26 @@ import java.util.Map; import java.util.concurrent.TimeUnit; 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.data.GenericAppenderFactory; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeAll; +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 class TestColumnStatsWatermarkExtractor { public static final Schema SCHEMA = new Schema( @@ -68,15 +71,16 @@ public class TestColumnStatsWatermarkExtractor { private static final List> MIN_VALUES = ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + @RegisterExtension + private static final HadoopTableExtension sourceTableResource = + new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); - private final String columnName; + @Parameter(index = 0) + private String columnName; - @BeforeClass + @BeforeAll public static void updateMinValue() { for (int i = 0; i < TEST_RECORDS.size(); ++i) { for (Record r : TEST_RECORDS.get(i)) { @@ -94,7 +98,7 @@ public static void updateMinValue() { } } - @Parameterized.Parameters(name = "{0}") + @Parameters(name = "columnName = {0}") public static Collection data() { return ImmutableList.of( new Object[] {"timestamp_column"}, @@ -102,62 +106,56 @@ public static Collection data() { new Object[] {"long_column"}); } - public TestColumnStatsWatermarkExtractor(String columnName) { - this.columnName = columnName; - } - - @Test + @TestTemplate public void testSingle() throws IOException { ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); } - @Test + @TestTemplate public void testTimeUnit() throws IOException { - Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + assumeThat(columnName).isEqualTo("long_column"); ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue() / 1000L, - extractor.extractWatermark(split(0))); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L); } - @Test + @TestTemplate public void testMultipleFiles() throws IOException { - Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + assumeThat(columnName).isEqualTo("timestamp_column"); IcebergSourceSplit combinedSplit = IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( - TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + TEST_RECORDS, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - Assert.assertEquals( - MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); - Assert.assertEquals( - Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), - extractor.extractWatermark(combinedSplit)); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); + assertThat(extractor.extractWatermark(split(1))) + .isEqualTo(MIN_VALUES.get(1).get(columnName).longValue()); + assertThat(extractor.extractWatermark(combinedSplit)) + .isEqualTo(Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName))); } - @Test + @TestTemplate public void testWrongColumn() { - Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + assumeThat(columnName).isEqualTo("string_column"); assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); } - @Test + @TestTemplate public void testEmptyStatistics() throws IOException { - Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + assumeThat(columnName).isEqualTo("timestamp_column"); // Create an extractor for a column we do not have statistics ColumnStatsWatermarkExtractor extractor = @@ -171,7 +169,7 @@ private IcebergSourceSplit split(int id) throws IOException { return IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( ImmutableList.of(TEST_RECORDS.get(id)), - TEMPORARY_FOLDER, + temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); } From b4dc055b984667d9a0e4b0a8206190cdc50b603d Mon Sep 17 00:00:00 2001 From: Helt Date: Tue, 2 Jul 2024 05:33:17 +0800 Subject: [PATCH 0424/1019] Core: Fix ParallelIterable memory leak where queue continues to be populated even after iterator close (#9402) --- .../apache/iceberg/util/ParallelIterable.java | 6 ++ .../iceberg/util/TestParallelIterable.java | 61 +++++++++++++++++++ 2 files changed, 67 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 108757b4155c..d7221e7d4545 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -67,6 +67,12 @@ private ParallelIterator( try (Closeable ignored = (iterable instanceof Closeable) ? (Closeable) iterable : () -> {}) { for (T item : iterable) { + // exit manually because `ConcurrentLinkedQueue` can't be + // interrupted + if (closed) { + return; + } + queue.add(item); } } catch (IOException e) { 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 68685614d3da..af9c6ec5212c 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -23,6 +23,7 @@ import java.io.IOException; import java.lang.reflect.Field; import java.util.Collections; +import java.util.Iterator; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; @@ -72,6 +73,66 @@ public CloseableIterator iterator() { .untilAsserted(() -> assertThat(queue).isEmpty()); } + @Test + public void closeMoreDataParallelIteratorWithoutCompleteIteration() + throws IOException, IllegalAccessException, NoSuchFieldException { + ExecutorService executor = Executors.newFixedThreadPool(1); + Iterator integerIterator = + new Iterator() { + private int number = 1; + + @Override + public boolean hasNext() { + if (number > 1000) { + return false; + } + + 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! + } + 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); + CloseableIterator iterator = parallelIterable.iterator(); + Field queueField = iterator.getClass().getDeclaredField("queue"); + queueField.setAccessible(true); + ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + + assertThat(iterator.hasNext()).isTrue(); + assertThat(iterator.next()).isNotNull(); + Awaitility.await("Queue is populated") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> queueHasElements(iterator, queue)); + iterator.close(); + Awaitility.await("Queue is cleared") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted(() -> assertThat(queue).as("Queue is not empty after cleaning").isEmpty()); + } + private void queueHasElements(CloseableIterator iterator, Queue queue) { assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); From 8a22c143c9b20e08e6e420fb2ee914b6adce1765 Mon Sep 17 00:00:00 2001 From: Tai Le Manh <49281946+tlm365@users.noreply.github.com> Date: Tue, 2 Jul 2024 07:23:30 +0700 Subject: [PATCH 0425/1019] Core: Handle potential NPE in RESTSessionCatalog#newSessionCache (#10607) Signed-off-by: Tai Le Manh --- .../java/org/apache/iceberg/rest/RESTSessionCatalog.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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 da40d4c3ae31..11a6aa6f27da 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -1049,7 +1049,12 @@ private static Cache newSessionCache(Map pr return Caffeine.newBuilder() .expireAfterAccess(Duration.ofMillis(expirationIntervalMs)) .removalListener( - (RemovalListener) (id, auth, cause) -> auth.stopRefreshing()) + (RemovalListener) + (id, auth, cause) -> { + if (auth != null) { + auth.stopRefreshing(); + } + }) .build(); } From 988c9ae2e2d90e42780ae514d5711193068fc820 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 2 Jul 2024 09:47:23 +0200 Subject: [PATCH 0426/1019] Build: Bump io.github.goooler.shadow:shadow-gradle-plugin (#10612) Bumps [io.github.goooler.shadow:shadow-gradle-plugin](https://github.com/Goooler/shadow) from 8.1.7 to 8.1.8. - [Release notes](https://github.com/Goooler/shadow/releases) - [Commits](https://github.com/Goooler/shadow/compare/v8.1.7...v8.1.8) --- updated-dependencies: - dependency-name: io.github.goooler.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 8118b7a4af5b..cde7342b8945 100644 --- a/build.gradle +++ b/build.gradle @@ -26,7 +26,7 @@ buildscript { gradlePluginPortal() } dependencies { - classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.7' + classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' classpath 'com.palantir.baseline:gradle-baseline-java:4.42.0' // com.palantir.baseline:gradle-baseline-java:4.42.0 (the last version supporting Java 8) pulls // in an old version of the errorprone, which doesn't work w/ Gradle 8, so bump errorpone as From 2d83c8b77eebbedc83974102d73af46c2d0ccc03 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 3 Jul 2024 01:26:47 +0900 Subject: [PATCH 0427/1019] Flink 1.17, 1.18: Migrate HadoopCatalog related tests (#10620) --- .../iceberg/flink/HadoopTableExtension.java | 59 +++++++ .../flink/sink/TestFlinkIcebergSink.java | 148 ++++++++---------- .../sink/TestFlinkIcebergSinkBranch.java | 66 ++++---- .../flink/sink/TestFlinkIcebergSinkV2.java | 111 +++++-------- .../sink/TestFlinkIcebergSinkV2Base.java | 42 ++++- .../sink/TestFlinkIcebergSinkV2Branch.java | 69 ++++---- .../flink/source/TestFlinkSourceSql.java | 4 +- .../flink/source/TestIcebergSourceSql.java | 4 +- .../iceberg/flink/source/TestSqlBase.java | 46 +++--- .../flink/source/reader/ReaderUtil.java | 22 +++ .../TestColumnStatsWatermarkExtractor.java | 90 ++++++----- .../iceberg/flink/HadoopTableExtension.java | 59 +++++++ .../flink/sink/TestFlinkIcebergSink.java | 148 ++++++++---------- .../sink/TestFlinkIcebergSinkBranch.java | 66 ++++---- .../flink/sink/TestFlinkIcebergSinkV2.java | 111 +++++-------- .../sink/TestFlinkIcebergSinkV2Base.java | 42 ++++- .../sink/TestFlinkIcebergSinkV2Branch.java | 69 ++++---- .../flink/source/TestFlinkSourceSql.java | 4 +- .../flink/source/TestIcebergSourceSql.java | 4 +- .../iceberg/flink/source/TestSqlBase.java | 46 +++--- .../flink/source/reader/ReaderUtil.java | 22 +++ .../TestColumnStatsWatermarkExtractor.java | 90 ++++++----- 22 files changed, 712 insertions(+), 610 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java new file mode 100644 index 000000000000..dc6ef400a4a9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.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; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopTableExtension extends HadoopCatalogExtension { + private final Schema schema; + private final PartitionSpec partitionSpec; + + private Table table; + + public HadoopTableExtension(String database, String tableName, Schema schema) { + this(database, tableName, schema, null); + } + + public HadoopTableExtension( + String database, String tableName, Schema schema, PartitionSpec partitionSpec) { + super(database, tableName); + this.schema = schema; + this.partitionSpec = partitionSpec; + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + super.beforeEach(context); + if (partitionSpec == null) { + this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); + } else { + this.table = + catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); + } + tableLoader.open(); + } + + public Table table() { + return table; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 9ff79419b020..527525e9f167 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; @@ -29,17 +31,21 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; 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.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -47,59 +53,52 @@ 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.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private TableLoader tableLoader; - private final FileFormat format; - private final int parallelism; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; - @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") public static Object[][] parameters() { return new Object[][] { - {"avro", 1, true}, - {"avro", 1, false}, - {"avro", 2, true}, - {"avro", 2, false}, - {"orc", 1, true}, - {"orc", 1, false}, - {"orc", 2, true}, - {"orc", 2, false}, - {"parquet", 1, true}, - {"parquet", 1, false}, - {"parquet", 2, true}, - {"parquet", 2, false} + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} }; } - public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -122,7 +121,7 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testWriteRowData() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = @@ -165,17 +164,17 @@ private int partitionFiles(String partition) throws IOException { return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); } - @Test + @TestTemplate public void testWriteRow() throws Exception { testWriteRow(null, DistributionMode.NONE); } - @Test + @TestTemplate public void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } - @Test + @TestTemplate public void testJobNoneDistributeMode() throws Exception { table .updateProperties() @@ -187,12 +186,12 @@ public void testJobNoneDistributeMode() throws Exception { if (parallelism > 1) { if (partitioned) { int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); + assertThat(files).isGreaterThan(3); } } } - @Test + @TestTemplate public void testJobHashDistributionMode() { table .updateProperties() @@ -204,7 +203,7 @@ public void testJobHashDistributionMode() { .hasMessage("Flink does not support 'range' write distribution mode now."); } - @Test + @TestTemplate public void testJobNullDistributionMode() throws Exception { table .updateProperties() @@ -214,42 +213,33 @@ public void testJobNullDistributionMode() throws Exception { testWriteRow(null, null); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testPartitionWriteMode() throws Exception { testWriteRow(null, DistributionMode.HASH); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testShuffleByPartitionWithSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); @@ -323,16 +313,14 @@ public void testTwoSinksInDisjointedDAG() throws Exception { SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); leftTable.refresh(); - Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); - Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); rightTable.refresh(); - Assert.assertEquals( - TestFlinkIcebergSink.class.getName(), - rightTable.currentSnapshot().summary().get("flink.test")); - Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); } - @Test + @TestTemplate public void testOverrideWriteConfigWithUnknownDistributionMode() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); @@ -352,7 +340,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } - @Test + @TestTemplate public void testOverrideWriteConfigWithUnknownFileFormat() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); @@ -372,7 +360,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .hasMessage("Invalid file format: UNRECOGNIZED"); } - @Test + @TestTemplate public void testWriteRowWithTableRefreshInterval() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 16b4542b00d3..547b4937c5bd 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -18,60 +18,60 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; 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; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { + @RegisterExtension + public static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @Parameter(index = 0) + private String formatVersion; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @Parameter(index = 1) + private String branch; - private final String branch; private TableLoader tableLoader; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkBranch(String branch) { - this.branch = branch; + @Parameters(name = "formatVersion = {0}, branch = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"1", "main"}, + {"1", "testBranch"}, + {"2", "main"}, + {"2", "testBranch"} + }; } - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -84,7 +84,7 @@ public void before() throws IOException { TableProperties.DEFAULT_FILE_FORMAT, FileFormat.AVRO.name(), TableProperties.FORMAT_VERSION, - "1")); + formatVersion)); env = StreamExecutionEnvironment.getExecutionEnvironment( @@ -94,7 +94,7 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); verifyOtherBranchUnmodified(); @@ -129,9 +129,9 @@ private void verifyOtherBranchUnmodified() { String otherBranch = branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); } - Assert.assertTrue(table.snapshot(otherBranch) == null); + assertThat(table.snapshot(otherBranch)).isNull(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 7712481d33d9..9cbb9f091e15 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -18,85 +18,53 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; 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.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Rule public final Timeout globalTimeout = Timeout.seconds(60); - - @Parameterized.Parameters( - name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} - }; - } - - public TestFlinkIcebergSinkV2( - String format, int parallelism, boolean partitioned, String writeDistributionMode) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - this.writeDistributionMode = writeDistributionMode; - } + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Before + @BeforeEach public void setupTable() { table = catalogResource @@ -129,7 +97,7 @@ public void setupTable() { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testCheckAndGetEqualityFieldIds() { table .updateSchema() @@ -144,28 +112,25 @@ public void testCheckAndGetEqualityFieldIds() { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); // Use schema identifier field IDs as equality field id list by default - Assert.assertEquals( - table.schema().identifierFieldIds(), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrderElementsOf(table.schema().identifierFieldIds()); // Use user-provided equality field column as equality field id list builder.equalityFieldColumns(Lists.newArrayList("id")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("id").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); builder.equalityFieldColumns(Lists.newArrayList("type")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("type").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("type").fieldId()); } - @Test + @TestTemplate public void testChangeLogOnIdKey() throws Exception { testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnlyDeletesOnDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -184,22 +149,22 @@ public void testUpsertOnlyDeletesOnDataKey() throws Exception { SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnSameKey() throws Exception { testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertModeCheck() throws Exception { DataStream dataStream = env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); @@ -227,22 +192,22 @@ public void testUpsertModeCheck() throws Exception { "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); } - @Test + @TestTemplate public void testUpsertOnIdKey() throws Exception { testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnDataKey() throws Exception { testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testDeleteStats() throws Exception { assumeThat(format).isNotEqualTo(FileFormat.AVRO); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 9cdf7743c485..fc33c2fea5e6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; @@ -31,6 +32,8 @@ import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -44,7 +47,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; public class TestFlinkIcebergSinkV2Base { @@ -55,14 +57,40 @@ public class TestFlinkIcebergSinkV2Base { protected static final int ROW_ID_POS = 0; protected static final int ROW_DATA_POS = 1; - protected int parallelism = 1; protected TableLoader tableLoader; protected Table table; protected StreamExecutionEnvironment env; + + @Parameter(index = 0) protected FileFormat format; + + @Parameter(index = 1) + protected int parallelism = 1; + + @Parameter(index = 2) protected boolean partitioned; + + @Parameter(index = 3) protected String writeDistributionMode; + @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.ORC, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.PARQUET, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} + }; + } + protected static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, @@ -319,16 +347,14 @@ protected void testChangeLogs( table.refresh(); List snapshots = findValidSnapshots(); int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + assertThat(snapshots).hasSize(expectedSnapshotNum); for (int i = 0; i < expectedSnapshotNum; i++) { long snapshotId = snapshots.get(i).snapshotId(); List expectedRecords = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(expectedRecords.toArray(new Record[0])), - actualRowSet(snapshotId, "*")); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index fed333848279..1c5c97b58d2d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -18,52 +18,43 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; 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; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "branch = {0}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } + @Parameter(index = 0) + private String branch; - public TestFlinkIcebergSinkV2Branch(String branch) { - this.branch = branch; + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; } - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -86,37 +77,37 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testChangeLogOnIdKey() throws Exception { testChangeLogOnIdKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { testChangeLogOnDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { testChangeLogOnIdDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnIdKey() throws Exception { testUpsertOnIdKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnDataKey() throws Exception { testUpsertOnDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(branch); verifyOtherBranchUnmodified(); @@ -126,9 +117,9 @@ private void verifyOtherBranchUnmodified() { String otherBranch = branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); } - Assert.assertTrue(table.snapshot(otherBranch) == null); + assertThat(table.snapshot(otherBranch)).isNull(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index affd90c347dd..6857e0a7a366 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -33,7 +33,7 @@ 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.Test; +import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { @@ -61,7 +61,7 @@ public void testInferParallelismWithGlobalSetting() throws IOException { .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); long maxFileLen = 0; for (int i = 0; i < 5; i++) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 4250460d278d..645af7cfa339 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -40,7 +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.Test; +import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ public class TestIcebergSourceSql extends TestSqlBase { @@ -78,7 +78,7 @@ private List generateExpectedRecords(boolean ascending) throws Exception long baseTime = 1702382109000L; GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); Record file1Record1 = generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index dda46033143e..94962e02bb05 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; @@ -34,30 +38,28 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TableLoader; 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.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @TempDir protected Path temporaryFolder; private volatile TableEnvironment tEnv; @@ -73,7 +75,7 @@ protected TableEnvironment getTableEnv() { return tEnv; } - @Before + @BeforeEach public abstract void before() throws IOException; @Test @@ -90,7 +92,7 @@ public void testResiduals() throws Exception { writeRecords.get(1).set(2, "2020-03-20"); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); expectedRecords.add(writeRecords.get(0)); @@ -120,7 +122,7 @@ public void testExposeLocality() throws Exception { expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); helper.appendToTable(dataFile); @@ -140,9 +142,9 @@ public void testExposeLocality() throws Exception { // When running with CI or local, `localityEnabled` will be false even if this configuration is // enabled - Assert.assertFalse( - "Expose split locality info should be false.", - SourceUtil.isLocalityEnabled(table, tableConf, true)); + assertThat(SourceUtil.isLocalityEnabled(table, tableConf, true)) + .as("Expose split locality info should be false.") + .isFalse(); results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); org.apache.iceberg.flink.TestHelpers.assertRecords( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f06d9b83bcd4..e3e341ca2c76 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import org.apache.flink.table.data.RowData; @@ -106,6 +107,7 @@ public static List> createRecordBatchList( return Lists.partition(records, batchCount); } + // Only for JUnit4 tests. Keep this method for test migration from JUnit4 to JUnit5 public static CombinedScanTask createCombinedScanTask( List> recordBatchList, TemporaryFolder temporaryFolder, @@ -122,4 +124,24 @@ public static CombinedScanTask createCombinedScanTask( return new BaseCombinedScanTask(fileTasks); } + + public static CombinedScanTask createCombinedScanTask( + List> recordBatchList, + Path temporaryFolder, + FileFormat fileFormat, + GenericAppenderFactory appenderFactory) + throws IOException { + List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); + for (List recordBatch : recordBatchList) { + FileScanTask fileTask = + ReaderUtil.createFileTask( + recordBatch, + File.createTempFile("junit", null, temporaryFolder.toFile()), + fileFormat, + appenderFactory); + fileTasks.add(fileTask); + } + + return new BaseCombinedScanTask(fileTasks); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 604bc09619e0..7033fd30e84f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -18,10 +18,14 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; 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.IOException; +import java.nio.file.Path; import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -30,27 +34,26 @@ import java.util.Map; import java.util.concurrent.TimeUnit; 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.data.GenericAppenderFactory; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeAll; +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 class TestColumnStatsWatermarkExtractor { public static final Schema SCHEMA = new Schema( @@ -68,15 +71,16 @@ public class TestColumnStatsWatermarkExtractor { private static final List> MIN_VALUES = ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + @RegisterExtension + private static final HadoopTableExtension sourceTableResource = + new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); - private final String columnName; + @Parameter(index = 0) + private String columnName; - @BeforeClass + @BeforeAll public static void updateMinValue() { for (int i = 0; i < TEST_RECORDS.size(); ++i) { for (Record r : TEST_RECORDS.get(i)) { @@ -94,7 +98,7 @@ public static void updateMinValue() { } } - @Parameterized.Parameters(name = "{0}") + @Parameters(name = "columnName = {0}") public static Collection data() { return ImmutableList.of( new Object[] {"timestamp_column"}, @@ -102,62 +106,56 @@ public static Collection data() { new Object[] {"long_column"}); } - public TestColumnStatsWatermarkExtractor(String columnName) { - this.columnName = columnName; - } - - @Test + @TestTemplate public void testSingle() throws IOException { ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); } - @Test + @TestTemplate public void testTimeUnit() throws IOException { - Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + assumeThat(columnName).isEqualTo("long_column"); ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue() / 1000L, - extractor.extractWatermark(split(0))); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L); } - @Test + @TestTemplate public void testMultipleFiles() throws IOException { - Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + assumeThat(columnName).isEqualTo("timestamp_column"); IcebergSourceSplit combinedSplit = IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( - TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + TEST_RECORDS, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - Assert.assertEquals( - MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); - Assert.assertEquals( - Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), - extractor.extractWatermark(combinedSplit)); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); + assertThat(extractor.extractWatermark(split(1))) + .isEqualTo(MIN_VALUES.get(1).get(columnName).longValue()); + assertThat(extractor.extractWatermark(combinedSplit)) + .isEqualTo(Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName))); } - @Test + @TestTemplate public void testWrongColumn() { - Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + assumeThat(columnName).isEqualTo("string_column"); assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); } - @Test + @TestTemplate public void testEmptyStatistics() throws IOException { - Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + assumeThat(columnName).isEqualTo("timestamp_column"); // Create an extractor for a column we do not have statistics ColumnStatsWatermarkExtractor extractor = @@ -171,7 +169,7 @@ private IcebergSourceSplit split(int id) throws IOException { return IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( ImmutableList.of(TEST_RECORDS.get(id)), - TEMPORARY_FOLDER, + temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java new file mode 100644 index 000000000000..dc6ef400a4a9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.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; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopTableExtension extends HadoopCatalogExtension { + private final Schema schema; + private final PartitionSpec partitionSpec; + + private Table table; + + public HadoopTableExtension(String database, String tableName, Schema schema) { + this(database, tableName, schema, null); + } + + public HadoopTableExtension( + String database, String tableName, Schema schema, PartitionSpec partitionSpec) { + super(database, tableName); + this.schema = schema; + this.partitionSpec = partitionSpec; + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + super.beforeEach(context); + if (partitionSpec == null) { + this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); + } else { + this.table = + catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); + } + tableLoader.open(); + } + + public Table table() { + return table; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 9ff79419b020..527525e9f167 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; @@ -29,17 +31,21 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; 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.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -47,59 +53,52 @@ 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.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private TableLoader tableLoader; - private final FileFormat format; - private final int parallelism; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; - @Parameterized.Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") public static Object[][] parameters() { return new Object[][] { - {"avro", 1, true}, - {"avro", 1, false}, - {"avro", 2, true}, - {"avro", 2, false}, - {"orc", 1, true}, - {"orc", 1, false}, - {"orc", 2, true}, - {"orc", 2, false}, - {"parquet", 1, true}, - {"parquet", 1, false}, - {"parquet", 2, true}, - {"parquet", 2, false} + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} }; } - public TestFlinkIcebergSink(String format, int parallelism, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -122,7 +121,7 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testWriteRowData() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = @@ -165,17 +164,17 @@ private int partitionFiles(String partition) throws IOException { return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); } - @Test + @TestTemplate public void testWriteRow() throws Exception { testWriteRow(null, DistributionMode.NONE); } - @Test + @TestTemplate public void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } - @Test + @TestTemplate public void testJobNoneDistributeMode() throws Exception { table .updateProperties() @@ -187,12 +186,12 @@ public void testJobNoneDistributeMode() throws Exception { if (parallelism > 1) { if (partitioned) { int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - Assert.assertTrue("Should have more than 3 files in iceberg table.", files > 3); + assertThat(files).isGreaterThan(3); } } } - @Test + @TestTemplate public void testJobHashDistributionMode() { table .updateProperties() @@ -204,7 +203,7 @@ public void testJobHashDistributionMode() { .hasMessage("Flink does not support 'range' write distribution mode now."); } - @Test + @TestTemplate public void testJobNullDistributionMode() throws Exception { table .updateProperties() @@ -214,42 +213,33 @@ public void testJobNullDistributionMode() throws Exception { testWriteRow(null, null); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testPartitionWriteMode() throws Exception { testWriteRow(null, DistributionMode.HASH); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testShuffleByPartitionWithSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); if (partitioned) { - Assert.assertEquals( - "There should be only 1 data file in partition 'aaa'", 1, partitionFiles("aaa")); - Assert.assertEquals( - "There should be only 1 data file in partition 'bbb'", 1, partitionFiles("bbb")); - Assert.assertEquals( - "There should be only 1 data file in partition 'ccc'", 1, partitionFiles("ccc")); + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); } } - @Test + @TestTemplate public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); @@ -323,16 +313,14 @@ public void testTwoSinksInDisjointedDAG() throws Exception { SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); leftTable.refresh(); - Assert.assertNull(leftTable.currentSnapshot().summary().get("flink.test")); - Assert.assertNull(leftTable.currentSnapshot().summary().get("direction")); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); rightTable.refresh(); - Assert.assertEquals( - TestFlinkIcebergSink.class.getName(), - rightTable.currentSnapshot().summary().get("flink.test")); - Assert.assertEquals("rightTable", rightTable.currentSnapshot().summary().get("direction")); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); } - @Test + @TestTemplate public void testOverrideWriteConfigWithUnknownDistributionMode() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); @@ -352,7 +340,7 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } - @Test + @TestTemplate public void testOverrideWriteConfigWithUnknownFileFormat() { Map newProps = Maps.newHashMap(); newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); @@ -372,7 +360,7 @@ public void testOverrideWriteConfigWithUnknownFileFormat() { .hasMessage("Invalid file format: UNRECOGNIZED"); } - @Test + @TestTemplate public void testWriteRowWithTableRefreshInterval() throws Exception { List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); DataStream dataStream = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 16b4542b00d3..547b4937c5bd 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -18,60 +18,60 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import java.util.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; -import org.apache.flink.test.util.MiniClusterWithClientResource; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; 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; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { + @RegisterExtension + public static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @Parameter(index = 0) + private String formatVersion; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @Parameter(index = 1) + private String branch; - private final String branch; private TableLoader tableLoader; - @Parameterized.Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } - - public TestFlinkIcebergSinkBranch(String branch) { - this.branch = branch; + @Parameters(name = "formatVersion = {0}, branch = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"1", "main"}, + {"1", "testBranch"}, + {"2", "main"}, + {"2", "testBranch"} + }; } - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -84,7 +84,7 @@ public void before() throws IOException { TableProperties.DEFAULT_FILE_FORMAT, FileFormat.AVRO.name(), TableProperties.FORMAT_VERSION, - "1")); + formatVersion)); env = StreamExecutionEnvironment.getExecutionEnvironment( @@ -94,7 +94,7 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testWriteRowWithTableSchema() throws Exception { testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); verifyOtherBranchUnmodified(); @@ -129,9 +129,9 @@ private void verifyOtherBranchUnmodified() { String otherBranch = branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); } - Assert.assertTrue(table.snapshot(otherBranch) == null); + assertThat(table.snapshot(otherBranch)).isNull(); } } 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 7712481d33d9..9cbb9f091e15 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 @@ -18,85 +18,53 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.List; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; 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.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - @Rule public final Timeout globalTimeout = Timeout.seconds(60); - - @Parameterized.Parameters( - name = "FileFormat = {0}, Parallelism = {1}, Partitioned={2}, WriteDistributionMode ={3}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {"avro", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"avro", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {"orc", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"orc", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {"parquet", 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {"parquet", 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} - }; - } - - public TestFlinkIcebergSinkV2( - String format, int parallelism, boolean partitioned, String writeDistributionMode) { - this.format = FileFormat.fromString(format); - this.parallelism = parallelism; - this.partitioned = partitioned; - this.writeDistributionMode = writeDistributionMode; - } + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Before + @BeforeEach public void setupTable() { table = catalogResource @@ -129,7 +97,7 @@ public void setupTable() { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testCheckAndGetEqualityFieldIds() { table .updateSchema() @@ -144,28 +112,25 @@ public void testCheckAndGetEqualityFieldIds() { FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); // Use schema identifier field IDs as equality field id list by default - Assert.assertEquals( - table.schema().identifierFieldIds(), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrderElementsOf(table.schema().identifierFieldIds()); // Use user-provided equality field column as equality field id list builder.equalityFieldColumns(Lists.newArrayList("id")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("id").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); builder.equalityFieldColumns(Lists.newArrayList("type")); - Assert.assertEquals( - Sets.newHashSet(table.schema().findField("type").fieldId()), - Sets.newHashSet(builder.checkAndGetEqualityFieldIds())); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("type").fieldId()); } - @Test + @TestTemplate public void testChangeLogOnIdKey() throws Exception { testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnlyDeletesOnDataKey() throws Exception { List> elementsPerCheckpoint = ImmutableList.of( @@ -184,22 +149,22 @@ public void testUpsertOnlyDeletesOnDataKey() throws Exception { SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testChangeLogOnSameKey() throws Exception { testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertModeCheck() throws Exception { DataStream dataStream = env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); @@ -227,22 +192,22 @@ public void testUpsertModeCheck() throws Exception { "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); } - @Test + @TestTemplate public void testUpsertOnIdKey() throws Exception { testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnDataKey() throws Exception { testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); } - @Test + @TestTemplate public void testDeleteStats() throws Exception { assumeThat(format).isNotEqualTo(FileFormat.AVRO); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index 9cdf7743c485..fc33c2fea5e6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; @@ -31,6 +32,8 @@ import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -44,7 +47,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -import org.junit.Assert; public class TestFlinkIcebergSinkV2Base { @@ -55,14 +57,40 @@ public class TestFlinkIcebergSinkV2Base { protected static final int ROW_ID_POS = 0; protected static final int ROW_DATA_POS = 1; - protected int parallelism = 1; protected TableLoader tableLoader; protected Table table; protected StreamExecutionEnvironment env; + + @Parameter(index = 0) protected FileFormat format; + + @Parameter(index = 1) + protected int parallelism = 1; + + @Parameter(index = 2) protected boolean partitioned; + + @Parameter(index = 3) protected String writeDistributionMode; + @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.ORC, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.PARQUET, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} + }; + } + protected static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, @@ -319,16 +347,14 @@ protected void testChangeLogs( table.refresh(); List snapshots = findValidSnapshots(); int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - Assert.assertEquals( - "Should have the expected snapshot number", expectedSnapshotNum, snapshots.size()); + assertThat(snapshots).hasSize(expectedSnapshotNum); for (int i = 0; i < expectedSnapshotNum; i++) { long snapshotId = snapshots.get(i).snapshotId(); List expectedRecords = expectedRecordsPerCheckpoint.get(i); - Assert.assertEquals( - "Should have the expected records for the checkpoint#" + i, - expectedRowSet(expectedRecords.toArray(new Record[0])), - actualRowSet(snapshotId, "*")); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); } } 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 fed333848279..1c5c97b58d2d 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 @@ -18,52 +18,43 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; 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; -import org.apache.iceberg.flink.HadoopCatalogResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { - - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); - - private final String branch; +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + private static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "branch = {0}") - public static Object[] parameters() { - return new Object[] {"main", "testBranch"}; - } + @Parameter(index = 0) + private String branch; - public TestFlinkIcebergSinkV2Branch(String branch) { - this.branch = branch; + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; } - @Before + @BeforeEach public void before() throws IOException { table = catalogResource @@ -86,37 +77,37 @@ public void before() throws IOException { tableLoader = catalogResource.tableLoader(); } - @Test + @TestTemplate public void testChangeLogOnIdKey() throws Exception { testChangeLogOnIdKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testChangeLogOnDataKey() throws Exception { testChangeLogOnDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testChangeLogOnIdDataKey() throws Exception { testChangeLogOnIdDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnIdKey() throws Exception { testUpsertOnIdKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnDataKey() throws Exception { testUpsertOnDataKey(branch); verifyOtherBranchUnmodified(); } - @Test + @TestTemplate public void testUpsertOnIdDataKey() throws Exception { testUpsertOnIdDataKey(branch); verifyOtherBranchUnmodified(); @@ -126,9 +117,9 @@ private void verifyOtherBranchUnmodified() { String otherBranch = branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - Assert.assertNull(table.currentSnapshot()); + assertThat(table.currentSnapshot()).isNull(); } - Assert.assertTrue(table.snapshot(otherBranch) == null); + assertThat(table.snapshot(otherBranch)).isNull(); } } 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 affd90c347dd..6857e0a7a366 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,7 +33,7 @@ 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.Test; +import org.junit.jupiter.api.Test; /** Use the FlinkSource */ public class TestFlinkSourceSql extends TestSqlBase { @@ -61,7 +61,7 @@ public void testInferParallelismWithGlobalSetting() throws IOException { .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); long maxFileLen = 0; for (int i = 0; i < 5; i++) { 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 4250460d278d..645af7cfa339 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,7 +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.Test; +import org.junit.jupiter.api.Test; /** Use the IcebergSource (FLIP-27) */ public class TestIcebergSourceSql extends TestSqlBase { @@ -78,7 +78,7 @@ private List generateExpectedRecords(boolean ascending) throws Exception long baseTime = 1702382109000L; GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); Record file1Record1 = generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index dda46033143e..94962e02bb05 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -18,13 +18,17 @@ */ package org.apache.iceberg.flink.source; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Map; import org.apache.flink.configuration.Configuration; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; @@ -34,30 +38,28 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TableLoader; 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.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static final HadoopCatalogExtension catalogResource = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @TempDir protected Path temporaryFolder; private volatile TableEnvironment tEnv; @@ -73,7 +75,7 @@ protected TableEnvironment getTableEnv() { return tEnv; } - @Before + @BeforeEach public abstract void before() throws IOException; @Test @@ -90,7 +92,7 @@ public void testResiduals() throws Exception { writeRecords.get(1).set(2, "2020-03-20"); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); expectedRecords.add(writeRecords.get(0)); @@ -120,7 +122,7 @@ public void testExposeLocality() throws Exception { expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); helper.appendToTable(dataFile); @@ -140,9 +142,9 @@ public void testExposeLocality() throws Exception { // When running with CI or local, `localityEnabled` will be false even if this configuration is // enabled - Assert.assertFalse( - "Expose split locality info should be false.", - SourceUtil.isLocalityEnabled(table, tableConf, true)); + assertThat(SourceUtil.isLocalityEnabled(table, tableConf, true)) + .as("Expose split locality info should be false.") + .isFalse(); results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); org.apache.iceberg.flink.TestHelpers.assertRecords( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index f06d9b83bcd4..e3e341ca2c76 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -20,6 +20,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Collections; import java.util.List; import org.apache.flink.table.data.RowData; @@ -106,6 +107,7 @@ public static List> createRecordBatchList( return Lists.partition(records, batchCount); } + // Only for JUnit4 tests. Keep this method for test migration from JUnit4 to JUnit5 public static CombinedScanTask createCombinedScanTask( List> recordBatchList, TemporaryFolder temporaryFolder, @@ -122,4 +124,24 @@ public static CombinedScanTask createCombinedScanTask( return new BaseCombinedScanTask(fileTasks); } + + public static CombinedScanTask createCombinedScanTask( + List> recordBatchList, + Path temporaryFolder, + FileFormat fileFormat, + GenericAppenderFactory appenderFactory) + throws IOException { + List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); + for (List recordBatch : recordBatchList) { + FileScanTask fileTask = + ReaderUtil.createFileTask( + recordBatch, + File.createTempFile("junit", null, temporaryFolder.toFile()), + fileFormat, + appenderFactory); + fileTasks.add(fileTask); + } + + return new BaseCombinedScanTask(fileTasks); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 604bc09619e0..7033fd30e84f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -18,10 +18,14 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; 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.IOException; +import java.nio.file.Path; import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -30,27 +34,26 @@ import java.util.Map; import java.util.concurrent.TimeUnit; 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.data.GenericAppenderFactory; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +import org.junit.jupiter.api.BeforeAll; +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 class TestColumnStatsWatermarkExtractor { public static final Schema SCHEMA = new Schema( @@ -68,15 +71,16 @@ public class TestColumnStatsWatermarkExtractor { private static final List> MIN_VALUES = ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, SCHEMA); + @RegisterExtension + private static final HadoopTableExtension sourceTableResource = + new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); - private final String columnName; + @Parameter(index = 0) + private String columnName; - @BeforeClass + @BeforeAll public static void updateMinValue() { for (int i = 0; i < TEST_RECORDS.size(); ++i) { for (Record r : TEST_RECORDS.get(i)) { @@ -94,7 +98,7 @@ public static void updateMinValue() { } } - @Parameterized.Parameters(name = "{0}") + @Parameters(name = "columnName = {0}") public static Collection data() { return ImmutableList.of( new Object[] {"timestamp_column"}, @@ -102,62 +106,56 @@ public static Collection data() { new Object[] {"long_column"}); } - public TestColumnStatsWatermarkExtractor(String columnName) { - this.columnName = columnName; - } - - @Test + @TestTemplate public void testSingle() throws IOException { ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); } - @Test + @TestTemplate public void testTimeUnit() throws IOException { - Assume.assumeTrue("Run only for long column", columnName.equals("long_column")); + assumeThat(columnName).isEqualTo("long_column"); ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue() / 1000L, - extractor.extractWatermark(split(0))); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L); } - @Test + @TestTemplate public void testMultipleFiles() throws IOException { - Assume.assumeTrue("Run only for the timestamp column", columnName.equals("timestamp_column")); + assumeThat(columnName).isEqualTo("timestamp_column"); IcebergSourceSplit combinedSplit = IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( - TEST_RECORDS, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + TEST_RECORDS, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); ColumnStatsWatermarkExtractor extractor = new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); - Assert.assertEquals( - MIN_VALUES.get(0).get(columnName).longValue(), extractor.extractWatermark(split(0))); - Assert.assertEquals( - MIN_VALUES.get(1).get(columnName).longValue(), extractor.extractWatermark(split(1))); - Assert.assertEquals( - Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName)), - extractor.extractWatermark(combinedSplit)); + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); + assertThat(extractor.extractWatermark(split(1))) + .isEqualTo(MIN_VALUES.get(1).get(columnName).longValue()); + assertThat(extractor.extractWatermark(combinedSplit)) + .isEqualTo(Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName))); } - @Test + @TestTemplate public void testWrongColumn() { - Assume.assumeTrue("Run only for string column", columnName.equals("string_column")); + assumeThat(columnName).isEqualTo("string_column"); assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining( "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); } - @Test + @TestTemplate public void testEmptyStatistics() throws IOException { - Assume.assumeTrue("Run only for timestamp column", columnName.equals("timestamp_column")); + assumeThat(columnName).isEqualTo("timestamp_column"); // Create an extractor for a column we do not have statistics ColumnStatsWatermarkExtractor extractor = @@ -171,7 +169,7 @@ private IcebergSourceSplit split(int id) throws IOException { return IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( ImmutableList.of(TEST_RECORDS.get(id)), - TEMPORARY_FOLDER, + temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); } From 0fdd223d06d76d78cdf024f5986c81929f49616c Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Wed, 3 Jul 2024 14:36:53 +0200 Subject: [PATCH 0428/1019] Address IntelliJ inspection findings (#10583) --- .../iceberg/aliyun/oss/BaseOSSFile.java | 2 +- .../oss/mock/AliyunOSSMockExtension.java | 2 +- .../mock/AliyunOSSMockLocalController.java | 4 +- .../org/apache/iceberg/PartitionSpec.java | 5 +- .../main/java/org/apache/iceberg/Schema.java | 4 +- .../org/apache/iceberg/SortOrderBuilder.java | 2 +- .../java/org/apache/iceberg/io/FileIO.java | 2 +- .../apache/iceberg/metrics/DefaultTimer.java | 15 +- .../org/apache/iceberg/util/BucketUtil.java | 4 +- .../iceberg/io/TestableCloseableIterable.java | 2 +- .../iceberg/transforms/TestBucketing.java | 2 +- .../transforms/TestBucketingProjection.java | 9 +- .../transforms/TestTruncatesProjection.java | 15 +- .../iceberg/util/TestCharSequenceMap.java | 4 +- .../BaseVectorizedParquetValuesReader.java | 4 +- .../lakeformation/LakeFormationTestBase.java | 15 +- .../aws/s3/TestS3FileIOIntegration.java | 2 +- .../iceberg/aws/s3/TestS3MultipartUpload.java | 2 +- .../iceberg/aws/AwsClientProperties.java | 2 +- .../org/apache/iceberg/aws/AwsProperties.java | 18 +-- .../apache/iceberg/aws/glue/GlueCatalog.java | 2 +- .../LakeFormationAwsClientFactory.java | 4 +- .../iceberg/aws/s3/S3FileIOProperties.java | 16 +- .../org/apache/iceberg/common/DynClasses.java | 2 +- .../org/apache/iceberg/common/DynFields.java | 2 +- .../org/apache/iceberg/common/DynMethods.java | 2 +- .../org/apache/iceberg/BaseEntriesTable.java | 2 +- .../org/apache/iceberg/BaseFilesTable.java | 2 +- .../iceberg/BaseMetastoreTableOperations.java | 2 +- .../apache/iceberg/BaseReplacePartitions.java | 2 +- .../apache/iceberg/BaseRewriteManifests.java | 2 +- .../org/apache/iceberg/BaseTransaction.java | 2 +- .../org/apache/iceberg/ClientPoolImpl.java | 4 +- .../org/apache/iceberg/MetadataUpdate.java | 6 +- .../java/org/apache/iceberg/MetricsModes.java | 2 +- .../org/apache/iceberg/PropertiesUpdate.java | 2 +- .../org/apache/iceberg/SnapshotProducer.java | 142 +++++++++--------- .../org/apache/iceberg/TableMetadata.java | 2 +- .../UpdateSnapshotReferencesOperation.java | 2 +- .../actions/RewriteDataFilesActionResult.java | 4 +- .../avro/AvroCustomOrderSchemaVisitor.java | 2 +- .../apache/iceberg/avro/AvroFileAppender.java | 8 +- .../apache/iceberg/avro/AvroSchemaUtil.java | 4 +- .../iceberg/avro/AvroSchemaVisitor.java | 4 +- .../avro/AvroSchemaWithTypeVisitor.java | 2 +- .../AvroWithPartnerByStructureVisitor.java | 2 +- .../iceberg/avro/AvroWithPartnerVisitor.java | 2 +- .../iceberg/avro/GenericAvroReader.java | 2 +- .../org/apache/iceberg/avro/ValueWriters.java | 2 +- .../NativeFileCryptoParameters.java | 6 +- .../iceberg/hadoop/HadoopTableOperations.java | 4 +- .../apache/iceberg/jdbc/JdbcClientPool.java | 2 +- .../iceberg/rest/responses/ErrorResponse.java | 8 +- .../rest/responses/LoadTableResponse.java | 2 +- .../iceberg/schema/UnionByNameVisitor.java | 2 +- .../org/apache/iceberg/util/ArrayUtil.java | 14 +- .../java/org/apache/iceberg/util/Pair.java | 6 +- .../java/org/apache/iceberg/util/Tasks.java | 4 +- .../apache/iceberg/util/ZOrderByteUtils.java | 14 +- .../iceberg/view/BaseViewOperations.java | 2 +- .../iceberg/TableMetadataParserTest.java | 5 +- .../java/org/apache/iceberg/TestBase.java | 2 +- .../org/apache/iceberg/TestTableMetadata.java | 47 ++---- .../iceberg/TestV1ToV2RowDeltaDelete.java | 6 +- .../avro/TestNameMappingWithAvroSchema.java | 68 ++++----- .../data/avro/TestDecoderResolver.java | 2 + .../iceberg/hadoop/HadoopFileIOTest.java | 2 +- .../iceberg/hadoop/TestHadoopCatalog.java | 6 +- .../apache/iceberg/io/MockInputStream.java | 2 +- .../apache/iceberg/jdbc/TestJdbcCatalog.java | 2 +- .../data/TestMetricsRowGroupFilter.java | 2 +- .../apache/iceberg/io/TestBaseTaskWriter.java | 4 +- .../iceberg/dell/mock/ecs/EcsS3MockRule.java | 2 +- .../iceberg/dell/mock/ecs/ObjectData.java | 2 +- .../apache/iceberg/flink/actions/Actions.java | 4 +- .../flink/actions/RewriteDataFilesAction.java | 2 +- .../AvroGenericRecordToRowDataMapper.java | 2 +- .../flink/sink/shuffle/MapDataStatistics.java | 2 +- .../RowDataToAvroGenericRecordConverter.java | 2 +- .../ContinuousSplitPlannerImpl.java | 2 +- .../enumerator/IcebergEnumeratorState.java | 2 +- .../IcebergEnumeratorStateSerializer.java | 6 +- .../flink/source/split/SerializerHelper.java | 6 +- .../iceberg/flink/util/FlinkPackage.java | 2 +- .../TestFlinkCatalogTablePartitions.java | 2 +- .../apache/iceberg/flink/TestTableLoader.java | 2 +- .../operator/FlinkSqlExtension.java | 5 +- .../TestColumnStatsWatermarkExtractor.java | 2 +- .../apache/iceberg/hive/HiveClientPool.java | 3 +- .../iceberg/hive/HiveSchemaConverter.java | 2 +- .../org/apache/iceberg/hive/ScriptRunner.java | 14 +- .../apache/iceberg/hive/TestHiveCatalog.java | 2 +- .../iceberg/hive/TestHiveCommitLocks.java | 4 +- .../apache/iceberg/mr/hive/Deserializer.java | 6 +- .../iceberg/mr/hive/HiveIcebergSerDe.java | 5 +- .../org/apache/iceberg/mr/TestHelper.java | 4 +- .../iceberg/mr/hive/HiveIcebergTestUtils.java | 14 +- ...estHiveIcebergStorageHandlerLocalScan.java | 2 +- ...stHiveIcebergStorageHandlerWithEngine.java | 17 ++- .../apache/iceberg/mr/hive/TestTables.java | 5 +- .../iceberg/nessie/NessieIcebergClient.java | 2 +- .../iceberg/nessie/BaseTestIceberg.java | 2 +- .../nessie/TestNessieIcebergClient.java | 4 +- .../iceberg/data/orc/GenericOrcReaders.java | 2 +- .../orc/ExpressionToSearchArgument.java | 4 +- .../main/java/org/apache/iceberg/orc/ORC.java | 2 +- .../org/apache/iceberg/orc/ORCSchemaUtil.java | 6 +- .../apache/iceberg/orc/TestBloomFilter.java | 2 +- .../data/parquet/BaseParquetReaders.java | 8 +- .../org/apache/iceberg/parquet/Parquet.java | 4 +- .../apache/iceberg/parquet/ParquetAvro.java | 2 +- .../parquet/ParquetBloomRowGroupFilter.java | 2 +- .../iceberg/parquet/ParquetTypeVisitor.java | 2 +- .../iceberg/parquet/ParquetValueReaders.java | 2 +- .../iceberg/parquet/ParquetValueWriters.java | 2 +- .../iceberg/parquet/ParquetWriteAdapter.java | 2 +- .../parquet/VectorizedParquetReader.java | 2 +- .../parquet/ParquetWritingTestUtils.java | 2 +- .../parquet/TestBloomRowGroupFilter.java | 2 +- .../parquet/TestDictionaryRowGroupFilter.java | 2 +- .../apache/iceberg/pig/IcebergStorage.java | 4 +- .../snowflake/JdbcSnowflakeClient.java | 10 +- .../iceberg/snowflake/SnowflakeCatalog.java | 2 +- .../snowflake/SnowflakeTableMetadata.java | 2 +- .../snowflake/SnowflakeCatalogTest.java | 6 +- .../SparkRowLevelOperationsTestBase.java | 2 +- .../spark/action/RandomGeneratingUDF.java | 5 +- .../spark/PruneColumnsWithReordering.java | 2 +- .../org/apache/iceberg/spark/Spark3Util.java | 2 +- .../apache/iceberg/spark/SparkTableUtil.java | 2 +- .../actions/DeleteOrphanFilesSparkAction.java | 2 +- .../actions/RewriteDataFilesSparkAction.java | 2 +- .../actions/RewriteManifestsSparkAction.java | 4 +- ...RewritePositionDeleteFilesSparkAction.java | 4 +- .../spark/data/SparkParquetWriters.java | 2 +- .../spark/source/SparkFileWriterFactory.java | 2 +- .../spark/source/SparkMicroBatchStream.java | 1 - .../spark/source/SparkScanBuilder.java | 2 +- .../spark/source/SparkStagedScanBuilder.java | 2 +- .../analysis/NoSuchProcedureException.java | 2 +- .../org/apache/iceberg/spark/TestBase.java | 2 +- .../spark/TestSparkCatalogOperations.java | 2 +- .../spark/actions/TestCreateActions.java | 2 +- .../apache/iceberg/spark/data/RandomData.java | 4 +- .../iceberg/spark/source/LogMessage.java | 2 +- .../iceberg/spark/source/TestBaseReader.java | 2 +- .../spark/source/TestCompressionSettings.java | 2 +- .../spark/source/TestDataFrameWrites.java | 33 ++-- .../source/TestIdentityPartitionData.java | 2 +- .../spark/source/TestPartitionPruning.java | 2 +- .../spark/source/TestStructuredStreaming.java | 6 +- .../iceberg/spark/sql/TestCreateTable.java | 2 +- .../iceberg/spark/sql/TestNamespaceSQL.java | 2 +- 153 files changed, 412 insertions(+), 449 deletions(-) diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java index d957e82f92ed..417785c400a1 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/BaseOSSFile.java @@ -29,7 +29,7 @@ abstract class BaseOSSFile { private final OSS client; private final OSSURI uri; - private AliyunProperties aliyunProperties; + private final AliyunProperties aliyunProperties; private SimplifiedObjectMeta metadata; private final MetricsContext metrics; 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 5a9b06090392..9aae5b777a8d 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 @@ -101,7 +101,7 @@ public void tearDownBucket(String bucket) { } public static class Builder { - private Map props = Maps.newHashMap(); + private final Map props = Maps.newHashMap(); public Builder silent() { props.put(AliyunOSSMockApp.PROP_SILENT, true); 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 index e98f228b6f37..7f7546ec233b 100644 --- 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 @@ -420,11 +420,11 @@ public boolean markSupported() { * A caller has caused a request that would cross the {@code maxLength} boundary. * * @param maxLength The max count of bytes to read. - * @param count The count of bytes 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 pCount) throws IOException { + protected void onMaxLength(final long maxLength, final long bytesRead) throws IOException { // for subclasses } diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 8f1df794030a..08a1c4f9ecfd 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -365,7 +365,7 @@ public static class Builder { private final Schema schema; private final List fields = Lists.newArrayList(); private final Set partitionNames = Sets.newHashSet(); - private Map, PartitionField> dedupFields = Maps.newHashMap(); + private final Map, PartitionField> dedupFields = Maps.newHashMap(); private int specId = 0; private final AtomicInteger lastAssignedFieldId = new AtomicInteger(unpartitionedLastAssignedId()); @@ -409,8 +409,7 @@ private void checkAndAddPartitionName(String name, Integer sourceColumnId) { name); } } - Preconditions.checkArgument( - name != null && !name.isEmpty(), "Cannot use empty or null partition name: %s", name); + Preconditions.checkArgument(!name.isEmpty(), "Cannot use empty partition name: %s", name); Preconditions.checkArgument( !partitionNames.contains(name), "Cannot use partition name more than once: %s", name); partitionNames.add(name); diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index d5ec3f250982..7ff712b62790 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -67,8 +67,8 @@ public class Schema implements Serializable { private transient Map> idToAccessor = null; private transient Map idToName = null; private transient Set identifierFieldIdSet = null; - private transient Map idsToReassigned; - private transient Map idsToOriginal; + private final transient Map idsToReassigned; + private final transient Map idsToOriginal; public Schema(List columns, Map aliases) { this(columns, aliases, ImmutableSet.of()); diff --git a/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java b/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java index 4a112e21c26c..48d3f7b57734 100644 --- a/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java +++ b/api/src/main/java/org/apache/iceberg/SortOrderBuilder.java @@ -113,5 +113,5 @@ default R desc(Term term) { default R caseSensitive(boolean caseSensitive) { throw new UnsupportedOperationException( this.getClass().getName() + " doesn't implement caseSensitive"); - }; + } } 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 a521cbf79d7f..de4bc2e12a81 100644 --- a/api/src/main/java/org/apache/iceberg/io/FileIO.java +++ b/api/src/main/java/org/apache/iceberg/io/FileIO.java @@ -94,7 +94,7 @@ default void deleteFile(OutputFile file) { */ default Map properties() { throw new UnsupportedOperationException( - String.format("%s does not expose configuration properties", this.getClass().toString())); + String.format("%s does not expose configuration properties", this.getClass())); } /** diff --git a/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java b/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java index 6cf0b380b4cb..4ad0c8f2b161 100644 --- a/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java +++ b/api/src/main/java/org/apache/iceberg/metrics/DefaultTimer.java @@ -66,31 +66,22 @@ public void record(long amount, TimeUnit unit) { @Override public T time(Supplier supplier) { - Timed timed = start(); - try { + try (Timed ignore = start()) { return supplier.get(); - } finally { - timed.stop(); } } @Override public T timeCallable(Callable callable) throws Exception { - Timed timed = start(); - try { + try (Timed ignore = start()) { return callable.call(); - } finally { - timed.stop(); } } @Override public void time(Runnable runnable) { - Timed timed = start(); - try { + try (Timed ignore = start()) { runnable.run(); - } finally { - timed.stop(); } } diff --git a/api/src/main/java/org/apache/iceberg/util/BucketUtil.java b/api/src/main/java/org/apache/iceberg/util/BucketUtil.java index 1f3a68aef52b..113850c7b603 100644 --- a/api/src/main/java/org/apache/iceberg/util/BucketUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/BucketUtil.java @@ -36,7 +36,7 @@ public class BucketUtil { private BucketUtil() {} public static int hash(int value) { - return MURMUR3.hashLong((long) value).asInt(); + return MURMUR3.hashLong(value).asInt(); } public static int hash(long value) { @@ -56,7 +56,7 @@ private static long doubleToLongBits(double value) { } public static int hash(float value) { - return MURMUR3.hashLong(doubleToLongBits((double) value)).asInt(); + return MURMUR3.hashLong(doubleToLongBits(value)).asInt(); } public static int hash(double value) { diff --git a/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java b/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java index abd088cf6b25..adb6a60e2491 100644 --- a/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java +++ b/api/src/test/java/org/apache/iceberg/io/TestableCloseableIterable.java @@ -22,7 +22,7 @@ public class TestableCloseableIterable implements CloseableIterable { private Boolean closed = false; - private TestableCloseableIterator iterator = new TestableCloseableIterator(); + private final TestableCloseableIterator iterator = new TestableCloseableIterator(); @Override public CloseableIterator iterator() { 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 f397f94f0ce5..28d01efa6d3b 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -383,7 +383,7 @@ private int hashBytes(byte[] bytes, int offset, int length) { */ private static UUID newUUID(byte[] bytes) { try { - return uuidBytesConstructor.newInstance((Object) bytes); + return uuidBytesConstructor.newInstance(bytes); } catch (InstantiationException | IllegalAccessException | InvocationTargetException e) { throw new RuntimeException(e); } diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java b/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java index 52a6d2cd49db..9ff11ef74b26 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketingProjection.java @@ -32,6 +32,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.UUID; import java.util.stream.Collectors; import org.apache.iceberg.PartitionSpec; @@ -309,7 +310,7 @@ public void testBucketStringInclusive() { @Test public void testBucketByteBufferStrict() throws Exception { - ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8")); + ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8)); Schema schema = new Schema(optional(1, "value", Types.BinaryType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 10).build(); @@ -322,7 +323,7 @@ public void testBucketByteBufferStrict() throws Exception { assertProjectionStrictValue( spec, greaterThanOrEqual("value", value), Expression.Operation.FALSE); - ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8")); + ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8)); assertProjectionStrict( spec, notIn("value", value, anotherValue), Expression.Operation.NOT_IN, "[4, 6]"); assertProjectionStrictValue(spec, in("value", value, anotherValue), Expression.Operation.FALSE); @@ -330,7 +331,7 @@ public void testBucketByteBufferStrict() throws Exception { @Test public void testBucketByteBufferInclusive() throws Exception { - ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8")); + ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8)); Schema schema = new Schema(optional(1, "value", Types.BinaryType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).bucket("value", 10).build(); @@ -344,7 +345,7 @@ public void testBucketByteBufferInclusive() throws Exception { assertProjectionInclusiveValue( spec, greaterThanOrEqual("value", value), Expression.Operation.TRUE); - ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8")); + ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8)); assertProjectionInclusive( spec, in("value", value, anotherValue), Expression.Operation.IN, "[4, 6]"); assertProjectionInclusiveValue( diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java b/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java index 588f6fc3bd03..67fc27a8d1d5 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTruncatesProjection.java @@ -32,6 +32,7 @@ import java.math.BigDecimal; import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; import java.util.stream.Collectors; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; @@ -447,10 +448,11 @@ public void testStringInclusive() { @Test public void testBinaryStrict() throws Exception { - ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8")); + ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8)); Schema schema = new Schema(optional(1, "value", Types.BinaryType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("value", 5).build(); - String expectedValue = TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes("UTF-8"))); + String expectedValue = + TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes(StandardCharsets.UTF_8))); assertProjectionStrict(spec, lessThan("value", value), Expression.Operation.LT, expectedValue); assertProjectionStrict( @@ -463,7 +465,7 @@ public void testBinaryStrict() throws Exception { spec, notEqual("value", value), Expression.Operation.NOT_EQ, expectedValue); assertProjectionStrictValue(spec, equal("value", value), Expression.Operation.FALSE); - ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8")); + ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8)); assertProjectionStrict( spec, notIn("value", value, anotherValue), @@ -474,10 +476,11 @@ public void testBinaryStrict() throws Exception { @Test public void testBinaryInclusive() throws Exception { - ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes("UTF-8")); + ByteBuffer value = ByteBuffer.wrap("abcdefg".getBytes(StandardCharsets.UTF_8)); Schema schema = new Schema(optional(1, "value", Types.BinaryType.get())); PartitionSpec spec = PartitionSpec.builderFor(schema).truncate("value", 5).build(); - String expectedValue = TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes("UTF-8"))); + String expectedValue = + TransformUtil.base64encode(ByteBuffer.wrap("abcde".getBytes(StandardCharsets.UTF_8))); assertProjectionInclusive( spec, lessThan("value", value), Expression.Operation.LT_EQ, expectedValue); @@ -490,7 +493,7 @@ public void testBinaryInclusive() throws Exception { assertProjectionInclusive(spec, equal("value", value), Expression.Operation.EQ, expectedValue); assertProjectionInclusiveValue(spec, notEqual("value", value), Expression.Operation.TRUE); - ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes("UTF-8")); + ByteBuffer anotherValue = ByteBuffer.wrap("abcdehij".getBytes(StandardCharsets.UTF_8)); assertProjectionInclusive( spec, in("value", value, anotherValue), diff --git a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java index 2154cf31a9f6..0d74aa5bf2f2 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java +++ b/api/src/test/java/org/apache/iceberg/util/TestCharSequenceMap.java @@ -31,8 +31,8 @@ public class TestCharSequenceMap { @Test public void nullString() { - assertThat(CharSequenceMap.create()).doesNotContainKey((String) null); - assertThat(CharSequenceMap.create()).doesNotContainValue((String) null); + assertThat(CharSequenceMap.create()).doesNotContainKey(null); + assertThat(CharSequenceMap.create()).doesNotContainValue(null); } @Test diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java index 247f4d4f849e..b0a75cceb5b3 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/BaseVectorizedParquetValuesReader.java @@ -145,7 +145,7 @@ private int readIntLittleEndian() throws IOException { int ch3 = inputStream.read(); int ch2 = inputStream.read(); int ch1 = inputStream.read(); - return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + (ch4 << 0); + return (ch1 << 24) + (ch2 << 16) + (ch3 << 8) + ch4; } /** Reads the next byteWidth little endian int. */ @@ -166,7 +166,7 @@ private int readIntLittleEndianPaddedOnBitWidth() throws IOException { int ch3 = inputStream.read(); int ch2 = inputStream.read(); int ch1 = inputStream.read(); - return (ch1 << 16) + (ch2 << 8) + (ch3 << 0); + return (ch1 << 16) + (ch2 << 8) + ch3; } case 4: { diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java index 5e0f66610c42..630db7a5a1b4 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java @@ -128,15 +128,12 @@ public class LakeFormationTestBase { @BeforeAll public static void beforeClass() throws Exception { - lfRegisterPathRoleName = LF_REGISTER_PATH_ROLE_PREFIX + UUID.randomUUID().toString(); - lfPrivilegedRoleName = LF_PRIVILEGED_ROLE_PREFIX + UUID.randomUUID().toString(); - lfRegisterPathRoleS3PolicyName = - LF_REGISTER_PATH_ROLE_S3_POLICY_PREFIX + UUID.randomUUID().toString(); - lfRegisterPathRoleLfPolicyName = - LF_REGISTER_PATH_ROLE_LF_POLICY_PREFIX + UUID.randomUUID().toString(); - lfRegisterPathRoleIamPolicyName = - LF_REGISTER_PATH_ROLE_IAM_POLICY_PREFIX + UUID.randomUUID().toString(); - lfPrivilegedRolePolicyName = LF_PRIVILEGED_ROLE_POLICY_PREFIX + UUID.randomUUID().toString(); + lfRegisterPathRoleName = LF_REGISTER_PATH_ROLE_PREFIX + UUID.randomUUID(); + lfPrivilegedRoleName = LF_PRIVILEGED_ROLE_PREFIX + UUID.randomUUID(); + lfRegisterPathRoleS3PolicyName = LF_REGISTER_PATH_ROLE_S3_POLICY_PREFIX + UUID.randomUUID(); + lfRegisterPathRoleLfPolicyName = LF_REGISTER_PATH_ROLE_LF_POLICY_PREFIX + UUID.randomUUID(); + lfRegisterPathRoleIamPolicyName = LF_REGISTER_PATH_ROLE_IAM_POLICY_PREFIX + UUID.randomUUID(); + lfPrivilegedRolePolicyName = LF_PRIVILEGED_ROLE_POLICY_PREFIX + UUID.randomUUID(); iam = IamClient.builder() 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 9a52ae5ebfd8..18abb82ce74a 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 @@ -119,7 +119,7 @@ public static void afterClass() { @BeforeEach public void beforeEach() { - objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString()); + objectKey = String.format("%s/%s", prefix, UUID.randomUUID()); objectUri = String.format("s3://%s/%s", bucketName, objectKey); clientFactory.initialize(Maps.newHashMap()); } 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 ac34807db676..29d4c48927fc 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 @@ -64,7 +64,7 @@ public static void afterClass() { @BeforeEach public void before() { - String objectKey = String.format("%s/%s", prefix, UUID.randomUUID().toString()); + String objectKey = String.format("%s/%s", prefix, UUID.randomUUID()); objectUri = String.format("s3://%s/%s", bucketName, objectKey); } 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 1d03a0acbd2c..0c91f8685ae9 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientProperties.java @@ -67,7 +67,7 @@ public class AwsClientProperties implements Serializable { public static final String CLIENT_REGION = "client.region"; private String clientRegion; - private String clientCredentialsProvider; + private final String clientCredentialsProvider; private final Map clientCredentialsProviderProperties; public AwsClientProperties() { diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index 0d75e2451629..5c3afc28a98b 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -213,25 +213,25 @@ public class AwsProperties implements Serializable { private static final String HTTP_CLIENT_PREFIX = "http-client."; private final Set stsClientAssumeRoleTags; - private String clientAssumeRoleArn; - private String clientAssumeRoleExternalId; - private int clientAssumeRoleTimeoutSec; - private String clientAssumeRoleRegion; - private String clientAssumeRoleSessionName; - private String clientCredentialsProvider; + private final String clientAssumeRoleArn; + private final String clientAssumeRoleExternalId; + private final int clientAssumeRoleTimeoutSec; + private final String clientAssumeRoleRegion; + private final String clientAssumeRoleSessionName; + private final String clientCredentialsProvider; private final Map clientCredentialsProviderProperties; - private String glueEndpoint; + private final String glueEndpoint; private String glueCatalogId; private boolean glueCatalogSkipArchive; private boolean glueCatalogSkipNameValidation; private boolean glueLakeFormationEnabled; private String dynamoDbTableName; - private String dynamoDbEndpoint; + private final String dynamoDbEndpoint; private String restSigningRegion; - private String restSigningName; + private final String restSigningName; private String restAccessKeyId; private String restSecretAccessKey; private String restSessionToken; diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index cdbcc79d43fa..c6b157bb5c79 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -387,7 +387,7 @@ public void renameTable(TableIdentifier from, TableIdentifier to) { "Cannot rename %s to %s because namespace %s does not exist", from, to, to.namespace()); } // keep metadata - Table fromTable = null; + Table fromTable; String fromTableDbName = IcebergToGlueConverter.getDatabaseName(from, awsProperties.glueCatalogSkipNameValidation()); String fromTableName = 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 fb0b1c23efc4..552da4bc949c 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 @@ -139,8 +139,8 @@ private LakeFormationClient lakeFormation() { } static class LakeFormationCredentialsProvider implements AwsCredentialsProvider { - private LakeFormationClient client; - private String tableArn; + private final LakeFormationClient client; + private final String tableArn; LakeFormationCredentialsProvider(LakeFormationClient lakeFormationClient, String tableArn) { this.client = lakeFormationClient; 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 1bcbdf7ed36b..3414c9df7046 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 @@ -396,9 +396,9 @@ public class S3FileIOProperties implements Serializable { private String sseType; private String sseKey; private String sseMd5; - private String accessKeyId; - private String secretAccessKey; - private String sessionToken; + private final String accessKeyId; + private final String secretAccessKey; + private final String sessionToken; private boolean isS3AccessGrantsEnabled; private boolean isS3AccessGrantsFallbackToIamEnabled; private int multipartUploadThreads; @@ -416,11 +416,11 @@ public class S3FileIOProperties implements Serializable { private boolean isDeleteEnabled; private final Map bucketToAccessPointMapping; private boolean isPreloadClientEnabled; - private boolean isDualStackEnabled; - private boolean isPathStyleAccess; - private boolean isUseArnRegionEnabled; - private boolean isAccelerationEnabled; - private String endpoint; + private final boolean isDualStackEnabled; + private final boolean isPathStyleAccess; + private final boolean isUseArnRegionEnabled; + private final boolean isAccelerationEnabled; + private final String endpoint; private final boolean isRemoteSigningEnabled; private String writeStorageClass; private final Map allProperties; diff --git a/common/src/main/java/org/apache/iceberg/common/DynClasses.java b/common/src/main/java/org/apache/iceberg/common/DynClasses.java index af33aded80b1..3d42171847d3 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynClasses.java +++ b/common/src/main/java/org/apache/iceberg/common/DynClasses.java @@ -34,7 +34,7 @@ public static class Builder { private ClassLoader loader = Thread.currentThread().getContextClassLoader(); private Class foundClass = null; private boolean nullOk = false; - private Set classNames = Sets.newLinkedHashSet(); + private final Set classNames = Sets.newLinkedHashSet(); private Builder() {} diff --git a/common/src/main/java/org/apache/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java index 01afbad22ecb..a96da851315b 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynFields.java +++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java @@ -414,7 +414,7 @@ public StaticField buildStatic() { } private static class MakeFieldAccessible implements PrivilegedAction { - private Field hidden; + private final Field hidden; MakeFieldAccessible(Field hidden) { this.hidden = hidden; diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index ae331da84ed4..98de1e7f06ca 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -508,7 +508,7 @@ public StaticMethod buildStatic() { } private static class MakeAccessible implements PrivilegedAction { - private Method hidden; + private final Method hidden; MakeAccessible(Method hidden) { this.hidden = hidden; diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index 4e485d516f12..5f35704bc35b 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -54,7 +54,7 @@ abstract class BaseEntriesTable extends BaseMetadataTable { public Schema schema() { StructType partitionType = Partitioning.partitionType(table()); Schema schema = ManifestEntry.getSchema(partitionType); - if (partitionType.fields().size() < 1) { + if (partitionType.fields().isEmpty()) { // avoid returning an empty struct, which is not always supported. // instead, drop the partition field (id 102) schema = TypeUtil.selectNot(schema, Sets.newHashSet(DataFile.PARTITION_ID)); diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java index 62e6f8acf7a5..011abffd8a2a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java @@ -48,7 +48,7 @@ abstract class BaseFilesTable extends BaseMetadataTable { public Schema schema() { StructType partitionType = Partitioning.partitionType(table()); Schema schema = new Schema(DataFile.getType(partitionType).fields()); - if (partitionType.fields().size() < 1) { + if (partitionType.fields().isEmpty()) { // avoid returning an empty struct, which is not always supported. // instead, drop the partition field schema = TypeUtil.selectNot(schema, Sets.newHashSet(DataFile.PARTITION_ID)); diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 6443cf6e60ea..50266b2edcbf 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -355,7 +355,7 @@ private static int parseVersion(String metadataLocation) { } try { - return Integer.valueOf(metadataLocation.substring(versionStart, versionEnd)); + return Integer.parseInt(metadataLocation.substring(versionStart, versionEnd)); } catch (NumberFormatException e) { LOG.warn("Unable to parse version from metadata location: {}", metadataLocation, e); return -1; diff --git a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java index d3a8edbc7cdd..04611f50f15a 100644 --- a/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java +++ b/core/src/main/java/org/apache/iceberg/BaseReplacePartitions.java @@ -111,7 +111,7 @@ public void validate(TableMetadata currentMetadata, Snapshot parent) { @Override public List apply(TableMetadata base, Snapshot snapshot) { - if (dataSpec().fields().size() <= 0) { + if (dataSpec().fields().isEmpty()) { // replace all data in an unpartitioned table deleteByRowFilter(Expressions.alwaysTrue()); } diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java index f3f8e5fcd7cb..dce6d4a995bd 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteManifests.java @@ -200,7 +200,7 @@ private boolean requiresRewrite(Set currentManifests) { return false; } - if (rewrittenManifests.size() == 0) { + if (rewrittenManifests.isEmpty()) { // nothing yet processed so perform a full rewrite return true; } diff --git a/core/src/main/java/org/apache/iceberg/BaseTransaction.java b/core/src/main/java/org/apache/iceberg/BaseTransaction.java index 2f051466bb8b..eb8dbd2538e6 100644 --- a/core/src/main/java/org/apache/iceberg/BaseTransaction.java +++ b/core/src/main/java/org/apache/iceberg/BaseTransaction.java @@ -71,7 +71,7 @@ enum TransactionType { private final Set deletedFiles = Sets.newHashSet(); // keep track of files deleted in the most recent commit private final Consumer enqueueDelete = deletedFiles::add; - private TransactionType type; + private final TransactionType type; private TableMetadata base; private TableMetadata current; private boolean hasLastOpCommitted; diff --git a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java index c3534fa22a4a..f454c4aeea53 100644 --- a/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java +++ b/core/src/main/java/org/apache/iceberg/ClientPoolImpl.java @@ -40,7 +40,7 @@ public abstract class ClientPoolImpl private volatile int currentSize; private boolean closed; - private int connectionRetryWaitPeriodMs = 1000; + private static final int CONNECTION_RETRY_WAIT_PERIOD_MS = 1000; public ClientPoolImpl(int poolSize, Class reconnectExc, boolean retryByDefault) { this(poolSize, reconnectExc, retryByDefault, 1); @@ -80,7 +80,7 @@ public R run(Action action, boolean retry) throws E, InterruptedExc } catch (Exception e) { if (isConnectionException(e)) { retryAttempts++; - Thread.sleep(connectionRetryWaitPeriodMs); + Thread.sleep(CONNECTION_RETRY_WAIT_PERIOD_MS); } else { throw reconnectExc.cast(exc); } diff --git a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java index d133b76901da..49fb1fe01c44 100644 --- a/core/src/main/java/org/apache/iceberg/MetadataUpdate.java +++ b/core/src/main/java/org/apache/iceberg/MetadataUpdate.java @@ -336,9 +336,9 @@ class SetSnapshotRef implements MetadataUpdate { private final String refName; private final Long snapshotId; private final SnapshotRefType type; - private Integer minSnapshotsToKeep; - private Long maxSnapshotAgeMs; - private Long maxRefAgeMs; + private final Integer minSnapshotsToKeep; + private final Long maxSnapshotAgeMs; + private final Long maxRefAgeMs; public SetSnapshotRef( String refName, diff --git a/core/src/main/java/org/apache/iceberg/MetricsModes.java b/core/src/main/java/org/apache/iceberg/MetricsModes.java index fd6b9c580ee2..3dc2434310a5 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsModes.java +++ b/core/src/main/java/org/apache/iceberg/MetricsModes.java @@ -160,7 +160,7 @@ Object writeReplace() throws ObjectStreamException { } private static class MetricsModeProxy implements Serializable { - private String modeAsString; + private final String modeAsString; MetricsModeProxy(String modeAsString) { this.modeAsString = modeAsString; diff --git a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java index 9168b84b4042..35338a689205 100644 --- a/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java +++ b/core/src/main/java/org/apache/iceberg/PropertiesUpdate.java @@ -62,7 +62,7 @@ public UpdateProperties set(String key, String value) { public UpdateProperties remove(String key) { Preconditions.checkNotNull(key, "Key cannot be null"); Preconditions.checkArgument( - !updates.keySet().contains(key), "Cannot remove and update the same key: %s", key); + !updates.containsKey(key), "Cannot remove and update the same key: %s", key); removals.add(key); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 218daca64dbd..9f4bcbc6bba9 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -370,82 +370,84 @@ protected TableMetadata refresh() { public void commit() { // this is always set to the latest commit attempt's snapshot id. AtomicLong newSnapshotId = new AtomicLong(-1L); - Timed totalDuration = commitMetrics().totalDuration().start(); - try { - Tasks.foreach(ops) - .retry(base.propertyAsInt(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), - 2.0 /* exponential */) - .onlyRetryOn(CommitFailedException.class) - .countAttempts(commitMetrics().attempts()) - .run( - taskOps -> { - Snapshot newSnapshot = apply(); - newSnapshotId.set(newSnapshot.snapshotId()); - TableMetadata.Builder update = TableMetadata.buildFrom(base); - if (base.snapshot(newSnapshot.snapshotId()) != null) { - // this is a rollback operation - update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch); - } else if (stageOnly) { - update.addSnapshot(newSnapshot); - } else { - update.setBranchSnapshot(newSnapshot, targetBranch); - } - - TableMetadata updated = update.build(); - if (updated.changes().isEmpty()) { - // do not commit if the metadata has not changed. for example, this may happen - // when setting the current - // snapshot to an ID that is already current. note that this check uses identity. - return; - } - - // if the table UUID is missing, add it here. the UUID will be re-created each time - // this operation retries - // to ensure that if a concurrent operation assigns the UUID, this operation will - // not fail. - taskOps.commit(base, updated.withUUID()); - }); - - } catch (CommitStateUnknownException commitStateUnknownException) { - throw commitStateUnknownException; - } catch (RuntimeException e) { - if (!strictCleanup || e instanceof CleanableFailure) { - Exceptions.suppressAndThrow(e, this::cleanAll); - } + try (Timed ignore = commitMetrics().totalDuration().start()) { + try { + Tasks.foreach(ops) + .retry(base.propertyAsInt(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), + 2.0 /* exponential */) + .onlyRetryOn(CommitFailedException.class) + .countAttempts(commitMetrics().attempts()) + .run( + taskOps -> { + Snapshot newSnapshot = apply(); + newSnapshotId.set(newSnapshot.snapshotId()); + TableMetadata.Builder update = TableMetadata.buildFrom(base); + if (base.snapshot(newSnapshot.snapshotId()) != null) { + // this is a rollback operation + update.setBranchSnapshot(newSnapshot.snapshotId(), targetBranch); + } else if (stageOnly) { + update.addSnapshot(newSnapshot); + } else { + update.setBranchSnapshot(newSnapshot, targetBranch); + } - throw e; - } + TableMetadata updated = update.build(); + if (updated.changes().isEmpty()) { + // do not commit if the metadata has not changed. for example, this may happen + // when setting the current + // snapshot to an ID that is already current. note that this check uses + // identity. + return; + } - try { - LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName()); - - // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by - // id in case another commit was added between this commit and the refresh. - Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); - if (saved != null) { - cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io()))); - // also clean up unused manifest lists created by multiple attempts - for (String manifestList : manifestLists) { - if (!saved.manifestListLocation().equals(manifestList)) { - deleteFile(manifestList); - } + // if the table UUID is missing, add it here. the UUID will be re-created each + // time + // this operation retries + // to ensure that if a concurrent operation assigns the UUID, this operation will + // not fail. + taskOps.commit(base, updated.withUUID()); + }); + + } catch (CommitStateUnknownException commitStateUnknownException) { + throw commitStateUnknownException; + } catch (RuntimeException e) { + if (!strictCleanup || e instanceof CleanableFailure) { + Exceptions.suppressAndThrow(e, this::cleanAll); } - } else { - // saved may not be present if the latest metadata couldn't be loaded due to eventual - // consistency problems in refresh. in that case, don't clean up. - LOG.warn("Failed to load committed snapshot, skipping manifest clean-up"); + + throw e; } - } catch (Throwable e) { - LOG.warn( - "Failed to load committed table metadata or during cleanup, skipping further cleanup", e); - } + try { + LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName()); + + // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by + // id in case another commit was added between this commit and the refresh. + Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); + if (saved != null) { + cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io()))); + // also clean up unused manifest lists created by multiple attempts + for (String manifestList : manifestLists) { + if (!saved.manifestListLocation().equals(manifestList)) { + deleteFile(manifestList); + } + } + } else { + // saved may not be present if the latest metadata couldn't be loaded due to eventual + // consistency problems in refresh. in that case, don't clean up. + LOG.warn("Failed to load committed snapshot, skipping manifest clean-up"); + } - totalDuration.stop(); + } catch (Throwable e) { + LOG.warn( + "Failed to load committed table metadata or during cleanup, skipping further cleanup", + e); + } + } try { notifyListeners(); diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index a7edddecad52..6b23c337ae98 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1434,7 +1434,7 @@ public TableMetadata build() { // what is in the metadata file, which does not store changes. metadata location with changes // is inconsistent. Preconditions.checkArgument( - changes.size() == 0 || discardChanges || metadataLocation == null, + changes.isEmpty() || discardChanges || metadataLocation == null, "Cannot set metadata location with changes to table metadata: %s changes", changes.size()); diff --git a/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java b/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java index 9d15bf0ee207..f7ccea747a60 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java +++ b/core/src/main/java/org/apache/iceberg/UpdateSnapshotReferencesOperation.java @@ -31,7 +31,7 @@ class UpdateSnapshotReferencesOperation implements PendingUpdate updatedRefs; - private TableMetadata base; + private final TableMetadata base; UpdateSnapshotReferencesOperation(TableOperations ops) { this.ops = ops; diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java index a3b0292d0463..df2fdfdcdc3b 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesActionResult.java @@ -27,8 +27,8 @@ public class RewriteDataFilesActionResult { private static final RewriteDataFilesActionResult EMPTY = new RewriteDataFilesActionResult(ImmutableList.of(), ImmutableList.of()); - private List deletedDataFiles; - private List addedDataFiles; + private final List deletedDataFiles; + private final List addedDataFiles; public RewriteDataFilesActionResult( List deletedDataFiles, List addedDataFiles) { diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java index 575e44a83e42..69159b65be3e 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroCustomOrderSchemaVisitor.java @@ -68,7 +68,7 @@ public static T visit(Schema schema, AvroCustomOrderSchemaVisitor v } } - private Deque recordLevels = Lists.newLinkedList(); + private final Deque recordLevels = Lists.newLinkedList(); public T record(Schema record, List names, Iterable fields) { return null; diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java index c69a8626f6c9..a583f9a84e1c 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroFileAppender.java @@ -34,11 +34,11 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; class AvroFileAppender implements FileAppender { - private PositionOutputStream stream; + private final PositionOutputStream stream; private DataFileWriter writer; - private DatumWriter datumWriter; - private org.apache.iceberg.Schema icebergSchema; - private MetricsConfig metricsConfig; + private final DatumWriter datumWriter; + private final org.apache.iceberg.Schema icebergSchema; + private final MetricsConfig metricsConfig; private long numRecords = 0L; private boolean isClosed = false; diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java index 57c99faa2073..032d63105dfe 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaUtil.java @@ -222,7 +222,7 @@ public static boolean isKeyValueSchema(Schema schema) { static Schema createMap(int keyId, Schema keySchema, int valueId, Schema valueSchema) { String keyValueName = "k" + keyId + "_v" + valueId; - Schema.Field keyField = new Schema.Field("key", keySchema, null, (Object) null); + Schema.Field keyField = new Schema.Field("key", keySchema, null, null); keyField.addProp(FIELD_ID_PROP, keyId); Schema.Field valueField = @@ -250,7 +250,7 @@ static Schema createProjectionMap( Schema valueSchema) { String keyValueName = "k" + keyId + "_v" + valueId; - Schema.Field keyField = new Schema.Field("key", keySchema, null, (Object) null); + Schema.Field keyField = new Schema.Field("key", keySchema, null, null); if (!"key".equals(keyName)) { keyField.addAlias(keyName); } diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java index f22a3592ad3d..9819924ffa99 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaVisitor.java @@ -71,8 +71,8 @@ public static T visit(Schema schema, AvroSchemaVisitor visitor) { } } - private Deque recordLevels = Lists.newLinkedList(); - private Deque fieldNames = Lists.newLinkedList(); + private final Deque recordLevels = Lists.newLinkedList(); + private final Deque fieldNames = Lists.newLinkedList(); protected Deque fieldNames() { return fieldNames; diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java index 85a8718abfce..45892d3de151 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroSchemaWithTypeVisitor.java @@ -115,7 +115,7 @@ private static T visitArray(Type type, Schema array, AvroSchemaWithTypeVisit } } - private Deque recordLevels = Lists.newLinkedList(); + private final Deque recordLevels = Lists.newLinkedList(); public T record(Types.StructType iStruct, Schema record, List names, List fields) { return null; diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java index 2c080005d862..126a6373d210 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerByStructureVisitor.java @@ -146,7 +146,7 @@ private static T visitArray( } /** Just for checking state. */ - private Deque recordLevels = Lists.newLinkedList(); + private final Deque recordLevels = Lists.newLinkedList(); // ---------------------------------- Partner type methods // --------------------------------------------- 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 0147dbf37d31..b23b195d959a 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroWithPartnerVisitor.java @@ -67,7 +67,7 @@ public Type listElementPartner(Type partner) { } /** Used to fail on recursive types. */ - private Deque recordLevels = Lists.newLinkedList(); + private final Deque recordLevels = Lists.newLinkedList(); public R record(P partner, Schema record, List fieldResults) { return null; diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java index d89489d92a28..93bfa2398466 100644 --- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java @@ -43,7 +43,7 @@ public class GenericAvroReader private final Types.StructType expectedType; private ClassLoader loader = Thread.currentThread().getContextClassLoader(); private Map renames = ImmutableMap.of(); - private Map idToConstant = ImmutableMap.of(); + private final Map idToConstant = ImmutableMap.of(); private Schema fileSchema = null; private ValueReader reader = null; diff --git a/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java index 1658db96c21a..3844ede1c16a 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueWriters.java @@ -232,7 +232,7 @@ public void write(Object s, Encoder encoder) throws IOException { throw new IllegalArgumentException("Cannot write null to required string column"); } else { throw new IllegalArgumentException( - "Cannot write unknown string type: " + s.getClass().getName() + ": " + s.toString()); + "Cannot write unknown string type: " + s.getClass().getName() + ": " + s); } } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java b/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java index f11506a1bbc9..2cade89dc75d 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java +++ b/core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java @@ -27,8 +27,8 @@ * encryption support (Parquet and ORC). */ public class NativeFileCryptoParameters { - private ByteBuffer fileKey; - private EncryptionAlgorithm fileEncryptionAlgorithm; + private final ByteBuffer fileKey; + private final EncryptionAlgorithm fileEncryptionAlgorithm; private NativeFileCryptoParameters( ByteBuffer fileKey, EncryptionAlgorithm fileEncryptionAlgorithm) { @@ -48,7 +48,7 @@ public static Builder create(ByteBuffer fileKey) { } public static class Builder { - private ByteBuffer fileKey; + private final ByteBuffer fileKey; private EncryptionAlgorithm fileEncryptionAlgorithm; private Builder(ByteBuffer fileKey) { 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 4e815ceff59a..1e0cf4422120 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopTableOperations.java @@ -153,7 +153,7 @@ public void commit(TableMetadata base, TableMetadata metadata) { TableProperties.METADATA_COMPRESSION, TableProperties.METADATA_COMPRESSION_DEFAULT); TableMetadataParser.Codec codec = TableMetadataParser.Codec.fromName(codecName); String fileExtension = TableMetadataParser.getFileExtension(codec); - Path tempMetadataFile = metadataPath(UUID.randomUUID().toString() + fileExtension); + Path tempMetadataFile = metadataPath(UUID.randomUUID() + fileExtension); TableMetadataParser.write(metadata, io().newOutputFile(tempMetadataFile.toString())); int nextVersion = (current.first() != null ? current.first() : 0) + 1; @@ -296,7 +296,7 @@ private void writeVersionHint(int versionToWrite) { FileSystem fs = getFileSystem(versionHintFile, conf); try { - Path tempVersionHintFile = metadataPath(UUID.randomUUID().toString() + "-version-hint.temp"); + Path tempVersionHintFile = metadataPath(UUID.randomUUID() + "-version-hint.temp"); writeVersionToPath(fs, tempVersionHintFile, versionToWrite); fs.delete(versionHintFile, false /* recursive delete */); fs.rename(tempVersionHintFile, versionHintFile); diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java index 487b8409b1cc..9b6fca95bfe4 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcClientPool.java @@ -53,7 +53,7 @@ public class JdbcClientPool extends ClientPoolImpl { private final String dbUrl; private final Map properties; - private Set retryableStatusCodes; + private final Set retryableStatusCodes; public JdbcClientPool(String dbUrl, Map props) { this( diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java index 5543259af1e0..da93b821cedd 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ErrorResponse.java @@ -28,10 +28,10 @@ /** Standard response body for all API errors */ public class ErrorResponse implements RESTResponse { - private String message; - private String type; - private int code; - private List stack; + private final String message; + private final String type; + private final int code; + private final List stack; private ErrorResponse(String message, String type, int code, List stack) { this.message = message; 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 a389479fff10..5e5353c3909a 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 @@ -84,7 +84,7 @@ public static Builder builder() { public static class Builder { private String metadataLocation; private TableMetadata metadata; - private Map config = Maps.newHashMap(); + private final Map config = Maps.newHashMap(); private Builder() {} 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 291677741e93..1497ba59c582 100644 --- a/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java +++ b/core/src/main/java/org/apache/iceberg/schema/UnionByNameVisitor.java @@ -34,7 +34,7 @@ public class UnionByNameVisitor extends SchemaWithPartnerVisitor, Class> key) { null, false, Lists.newArrayList( - new Schema.Field("x", xSchema, null, (Object) null), - new Schema.Field("y", ySchema, null, (Object) null))); + new Schema.Field("x", xSchema, null, null), + new Schema.Field("y", ySchema, null, null))); } }); @@ -109,7 +109,7 @@ public Y second() { @Override public String toString() { - return "(" + String.valueOf(first) + ", " + String.valueOf(second) + ")"; + return "(" + first + ", " + second + ")"; } @Override 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 e420145c8dce..02d2b834311f 100644 --- a/core/src/main/java/org/apache/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -78,7 +78,7 @@ public static class Builder { private boolean stopAbortsOnFailure = false; // retry settings - private List> stopRetryExceptions = + private final List> stopRetryExceptions = Lists.newArrayList(UnrecoverableException.class); private List> onlyRetryExceptions = null; private Predicate shouldRetryPredicate = null; @@ -496,7 +496,7 @@ private static Collection waitFor(Collection> futures) { } catch (ExecutionException e) { Throwable cause = e.getCause(); - if (Error.class.isInstance(cause)) { + if (cause instanceof Error) { for (Throwable t : uncaught) { cause.addSuppressed(t); } diff --git a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java index c687fc4e03dc..4a5805742443 100644 --- a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java +++ b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java @@ -76,9 +76,7 @@ public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { * ByteBuffer)} */ public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { - ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); - bytes.putLong(((long) val) ^ 0x8000000000000000L); - return bytes; + return intToOrderedBytes(val, reuse); } /** @@ -86,9 +84,7 @@ public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { * ByteBuffer)} */ public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { - ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); - bytes.putLong(((long) val) ^ 0x8000000000000000L); - return bytes; + return intToOrderedBytes(val, reuse); } /** @@ -100,11 +96,7 @@ public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { * into lexicographically comparable bytes */ public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { - ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); - long lval = Double.doubleToLongBits(val); - lval ^= ((lval >> (Integer.SIZE - 1)) | Long.MIN_VALUE); - bytes.putLong(lval); - return bytes; + return doubleToOrderedBytes(val, reuse); } /** Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)} */ diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java index 7a4f546b8860..18b452f98367 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -219,7 +219,7 @@ private static int parseVersion(String metadataLocation) { } try { - return Integer.valueOf(metadataLocation.substring(versionStart, versionEnd)); + return Integer.parseInt(metadataLocation.substring(versionStart, versionEnd)); } catch (NumberFormatException e) { LOG.warn("Unable to parse version from metadata location: {}", metadataLocation, e); return -1; diff --git a/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java index 5581818aa1f0..56e3f7b67037 100644 --- a/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java +++ b/core/src/test/java/org/apache/iceberg/TableMetadataParserTest.java @@ -36,7 +36,6 @@ import java.util.zip.GZIPInputStream; import java.util.zip.ZipException; import org.apache.iceberg.TableMetadataParser.Codec; -import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types.BooleanType; @@ -69,7 +68,7 @@ public void testGzipCompressionProperty() throws IOException { TableMetadataParser.write(metadata, outputFile); assertThat(isCompressed(fileName)).isEqualTo(codec == Codec.GZIP); TableMetadata actualMetadata = - TableMetadataParser.read((FileIO) null, Files.localInput(new File(fileName))); + TableMetadataParser.read(null, Files.localInput(new File(fileName))); verifyMetadata(metadata, actualMetadata); } @@ -88,7 +87,7 @@ private void verifyMetadata(TableMetadata expected, TableMetadata actual) { } private boolean isCompressed(String path) throws IOException { - try (InputStream ignored = new GZIPInputStream(new FileInputStream(new File(path)))) { + try (InputStream ignored = new GZIPInputStream(new FileInputStream(path))) { return true; } catch (ZipException e) { if (e.getMessage().equals("Not in GZIP format")) { diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index dc7bf0e8d8cb..2322062dad85 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -390,7 +390,7 @@ Snapshot commit(Table table, SnapshotUpdate snapshotUpdate, String branch) { Snapshot apply(SnapshotUpdate snapshotUpdate, String branch) { if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return ((SnapshotProducer) snapshotUpdate).apply(); + return snapshotUpdate.apply(); } else { return ((SnapshotProducer) snapshotUpdate.toBranch(branch)).apply(); } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 8075372d09c1..e11cc500df55 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -537,7 +537,7 @@ public void testJsonWithPreviousMetadataLog() throws Exception { List previousMetadataLog = Lists.newArrayList(); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp, "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json")); TableMetadata base = new TableMetadata( @@ -606,17 +606,14 @@ public void testAddPreviousMetadataRemoveNone() throws IOException { List previousMetadataLog = Lists.newArrayList(); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 100, - "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 100, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 90, - "/tmp/000002-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 90, "/tmp/000002-" + UUID.randomUUID() + ".metadata.json")); MetadataLogEntry latestPreviousMetadata = new MetadataLogEntry( - currentTimestamp - 80, - "/tmp/000003-" + UUID.randomUUID().toString() + ".metadata.json"); + currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json"); TableMetadata base = new TableMetadata( @@ -691,29 +688,23 @@ public void testAddPreviousMetadataRemoveOne() throws IOException { List previousMetadataLog = Lists.newArrayList(); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 100, - "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 100, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 90, - "/tmp/000002-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 90, "/tmp/000002-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 80, - "/tmp/000003-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 70, - "/tmp/000004-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 70, "/tmp/000004-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 60, - "/tmp/000005-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 60, "/tmp/000005-" + UUID.randomUUID() + ".metadata.json")); MetadataLogEntry latestPreviousMetadata = new MetadataLogEntry( - currentTimestamp - 50, - "/tmp/000006-" + UUID.randomUUID().toString() + ".metadata.json"); + currentTimestamp - 50, "/tmp/000006-" + UUID.randomUUID() + ".metadata.json"); TableMetadata base = new TableMetadata( @@ -792,29 +783,23 @@ public void testAddPreviousMetadataRemoveMultiple() throws IOException { List previousMetadataLog = Lists.newArrayList(); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 100, - "/tmp/000001-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 100, "/tmp/000001-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 90, - "/tmp/000002-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 90, "/tmp/000002-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 80, - "/tmp/000003-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 80, "/tmp/000003-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 70, - "/tmp/000004-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 70, "/tmp/000004-" + UUID.randomUUID() + ".metadata.json")); previousMetadataLog.add( new MetadataLogEntry( - currentTimestamp - 60, - "/tmp/000005-" + UUID.randomUUID().toString() + ".metadata.json")); + currentTimestamp - 60, "/tmp/000005-" + UUID.randomUUID() + ".metadata.json")); MetadataLogEntry latestPreviousMetadata = new MetadataLogEntry( - currentTimestamp - 50, - "/tmp/000006-" + UUID.randomUUID().toString() + ".metadata.json"); + currentTimestamp - 50, "/tmp/000006-" + UUID.randomUUID() + ".metadata.json"); TableMetadata base = new TableMetadata( diff --git a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java index 28695e2fffe0..80f4d059f5f0 100644 --- a/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java +++ b/core/src/test/java/org/apache/iceberg/TestV1ToV2RowDeltaDelete.java @@ -70,7 +70,7 @@ public void testPartitionedTableWithPartitionEqDeletes() { verifyManifestSequenceNumber(dataManifest, 0, 0); // update table version to 2 - TableOperations ops = ((BaseTable) table).operations(); + TableOperations ops = table.operations(); TableMetadata base = ops.current(); ops.commit(base, base.upgradeToFormatVersion(2)); @@ -127,7 +127,7 @@ public void testPartitionedTableWithUnrelatedPartitionDeletes() { table.newAppend().appendFile(FILE_B).appendFile(FILE_C).appendFile(FILE_D).commit(); // update table version to 2 - TableOperations ops = ((BaseTable) table).operations(); + TableOperations ops = table.operations(); TableMetadata base = ops.current(); ops.commit(base, base.upgradeToFormatVersion(2)); @@ -162,7 +162,7 @@ public void testPartitionedTableWithExistingDeleteFile() { table.newAppend().appendFile(FILE_A).commit(); // update table version to 2 - TableOperations ops = ((BaseTable) table).operations(); + TableOperations ops = table.operations(); TableMetadata base = ops.current(); ops.commit(base, base.upgradeToFormatVersion(2)); diff --git a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java index 686456b3b16f..e3b25969fdb0 100644 --- a/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java +++ b/core/src/test/java/org/apache/iceberg/avro/TestNameMappingWithAvroSchema.java @@ -59,41 +59,39 @@ public void testNameMappingWithAvroSchema() { new Schema.Field( "complexUnion", Schema.createUnion( - new Schema[] { - Schema.create(Schema.Type.NULL), - Schema.create(Schema.Type.STRING), - Schema.createRecord( - "innerRecord1", - null, - "namespace1", - false, - Lists.newArrayList( - new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)), - new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))), - Schema.createRecord( - "innerRecord2", - null, - "namespace2", - false, - Lists.newArrayList( - new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)), - new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))), - Schema.createRecord( - "innerRecord3", - null, - "namespace3", - false, - Lists.newArrayList( - new Schema.Field( - "innerUnion", - Schema.createUnion( - Lists.newArrayList( - Schema.create(Schema.Type.STRING), - Schema.create(Schema.Type.INT)))))), - Schema.createEnum( - "timezone", null, null, Lists.newArrayList("UTC", "PST", "EST")), - Schema.createFixed("bitmap", null, null, 1) - })))); + Schema.create(Schema.Type.NULL), + Schema.create(Schema.Type.STRING), + Schema.createRecord( + "innerRecord1", + null, + "namespace1", + false, + Lists.newArrayList( + new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)), + new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))), + Schema.createRecord( + "innerRecord2", + null, + "namespace2", + false, + Lists.newArrayList( + new Schema.Field("lat", Schema.create(Schema.Type.DOUBLE)), + new Schema.Field("long", Schema.create(Schema.Type.DOUBLE)))), + Schema.createRecord( + "innerRecord3", + null, + "namespace3", + false, + Lists.newArrayList( + new Schema.Field( + "innerUnion", + Schema.createUnion( + Lists.newArrayList( + Schema.create(Schema.Type.STRING), + Schema.create(Schema.Type.INT)))))), + Schema.createEnum( + "timezone", null, null, Lists.newArrayList("UTC", "PST", "EST")), + Schema.createFixed("bitmap", null, null, 1))))); NameMappingWithAvroSchema nameMappingWithAvroSchema = new NameMappingWithAvroSchema(); diff --git a/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java b/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java index 9b355464a9d8..0024363701a5 100644 --- a/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java +++ b/core/src/test/java/org/apache/iceberg/data/avro/TestDecoderResolver.java @@ -40,6 +40,7 @@ public void before() { DecoderResolver.DECODER_CACHES.get().clear(); } + @SuppressWarnings("UnusedAssignment") // the unused assignments are necessary for this test @Test public void testDecoderCachingReadSchemaSameAsFileSchema() throws Exception { Decoder dummyDecoder = DecoderFactory.get().binaryDecoder(new byte[] {}, null); @@ -84,6 +85,7 @@ public void testDecoderCachingReadSchemaSameAsFileSchema() throws Exception { checkCachedSize(0); } + @SuppressWarnings("UnusedAssignment") // the unused assignments are necessary for this test @Test public void testDecoderCachingReadSchemaNotSameAsFileSchema() throws Exception { Decoder dummyDecoder = DecoderFactory.get().binaryDecoder(new byte[] {}, null); 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 95d0b6adbd02..109c88daeaa5 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -86,7 +86,7 @@ public void testListPrefix() { @Test public void testFileExists() throws IOException { Path parent = new Path(tempDir.toURI()); - Path randomFilePath = new Path(parent, "random-file-" + UUID.randomUUID().toString()); + Path randomFilePath = new Path(parent, "random-file-" + UUID.randomUUID()); fs.createNewFile(randomFilePath); // check existence of the created file 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 fbd6f833499e..6512b2499071 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java @@ -58,7 +58,7 @@ import org.junit.jupiter.params.provider.ValueSource; public class TestHadoopCatalog extends HadoopTableTestBase { - private static ImmutableMap meta = ImmutableMap.of(); + private static final ImmutableMap meta = ImmutableMap.of(); @ParameterizedTest @ValueSource(ints = {1, 2}) @@ -632,7 +632,7 @@ public void testRegisterTable() throws IOException { catalog.createTable(identifier, SCHEMA); Table registeringTable = catalog.loadTable(identifier); TableOperations ops = ((HasTableOperations) registeringTable).operations(); - String metadataLocation = ((HadoopTableOperations) ops).current().metadataFileLocation(); + String metadataLocation = ops.current().metadataFileLocation(); assertThat(catalog.registerTable(identifier2, metadataLocation)).isNotNull(); assertThat(catalog.loadTable(identifier2)).isNotNull(); assertThat(catalog.dropTable(identifier)).isTrue(); @@ -646,7 +646,7 @@ public void testRegisterExistingTable() throws IOException { catalog.createTable(identifier, SCHEMA); Table registeringTable = catalog.loadTable(identifier); TableOperations ops = ((HasTableOperations) registeringTable).operations(); - String metadataLocation = ((HadoopTableOperations) ops).current().metadataFileLocation(); + String metadataLocation = ops.current().metadataFileLocation(); assertThatThrownBy(() -> catalog.registerTable(identifier, metadataLocation)) .isInstanceOf(AlreadyExistsException.class) .hasMessage("Table already exists: a.t1"); diff --git a/core/src/test/java/org/apache/iceberg/io/MockInputStream.java b/core/src/test/java/org/apache/iceberg/io/MockInputStream.java index 17a6d537b267..00ce7fb78ebb 100644 --- a/core/src/test/java/org/apache/iceberg/io/MockInputStream.java +++ b/core/src/test/java/org/apache/iceberg/io/MockInputStream.java @@ -24,7 +24,7 @@ class MockInputStream extends ByteArrayInputStream { static final byte[] TEST_ARRAY = new byte[] {1, 2, 3, 4, 5, 6, 7, 8, 9, 10}; - private int[] lengths; + private final int[] lengths; private int current = 0; MockInputStream(int... actualReadLengths) { 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 c0c9afafd9a7..148a483b3477 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -1130,7 +1130,7 @@ public void report(MetricsReport report) { private String createMetadataLocationViaJdbcCatalog(TableIdentifier identifier) throws SQLException { // temporary connection just to actually create a concrete metadata location - String jdbcUrl = null; + String jdbcUrl; try { java.nio.file.Path dbFile = Files.createTempFile("temp", "metadata"); jdbcUrl = "jdbc:sqlite:" + dbFile.toAbsolutePath(); diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java index 9525001b6265..4e542b8455c4 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java @@ -149,7 +149,7 @@ public TestMetricsRowGroupFilter(String format) { static { StringBuilder sb = new StringBuilder(); for (int i = 0; i < 200; i += 1) { - sb.append(UUID.randomUUID().toString()); + sb.append(UUID.randomUUID()); } TOO_LONG_FOR_STATS_PARQUET = sb.toString(); } 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 85ddeb78f473..fb93a79d32a3 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java +++ b/data/src/test/java/org/apache/iceberg/io/TestBaseTaskWriter.java @@ -217,8 +217,8 @@ private TestTaskWriter createTaskWriter(long targetFileSize) { private static class TestTaskWriter extends BaseTaskWriter { - private RollingFileWriter dataWriter; - private RollingEqDeleteWriter deleteWriter; + private final RollingFileWriter dataWriter; + private final RollingEqDeleteWriter deleteWriter; private TestTaskWriter( PartitionSpec spec, diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java index f9db7f055751..c51241b76c4b 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java +++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/EcsS3MockRule.java @@ -110,7 +110,7 @@ private void initialize() { private void cleanUp() { if (mock) { // clean up - TEST_RULE_FOR_MOCK_CLIENT.set(null); + TEST_RULE_FOR_MOCK_CLIENT.remove(); } else { if (bucketCreated) { deleteBucket(); diff --git a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java index 08ffacb84e36..5f50dcc7d341 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java +++ b/dell/src/test/java/org/apache/iceberg/dell/mock/ecs/ObjectData.java @@ -72,7 +72,7 @@ public InputStream createInputStream(Range range) { public S3ObjectMetadata createFullMetadata() { S3ObjectMetadata metadata = new S3ObjectMetadata(); - MessageDigest md = null; + MessageDigest md; try { md = MessageDigest.getInstance("MD5"); } catch (NoSuchAlgorithmException e) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java index 06ac54617ae6..b96b47c5a785 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java @@ -30,8 +30,8 @@ public class Actions { // disable classloader check as Avro may cache class/object in the serializers. .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - private StreamExecutionEnvironment env; - private Table table; + private final StreamExecutionEnvironment env; + private final Table table; private Actions(StreamExecutionEnvironment env, Table table) { this.env = env; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java index 9876bb3861c4..670abebcb58a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java @@ -31,7 +31,7 @@ public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { - private StreamExecutionEnvironment env; + private final StreamExecutionEnvironment env; private int maxParallelism; public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java index 04e168385a36..f7e8e0c884cf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java @@ -55,7 +55,7 @@ public RowData map(GenericRecord genericRecord) throws Exception { public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); return new AvroGenericRecordToRowDataMapper(rowType); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 22dd2388cc3e..05b943f6046f 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -43,7 +43,7 @@ public StatisticsType type() { @Override public boolean isEmpty() { - return keyFrequency.size() == 0; + return keyFrequency.isEmpty(); } @Override diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java index 8f95e3e554a0..8ef1f1fbb833 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java @@ -64,7 +64,7 @@ public static RowDataToAvroGenericRecordConverter fromIcebergSchema( public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java index e9e3c159b07b..fef4ec45ed8a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java @@ -121,7 +121,7 @@ private ContinuousEnumerationResult discoverIncrementalSplits( LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); } else { - Long lastConsumedSnapshotId = lastPosition != null ? lastPosition.snapshotId() : null; + Long lastConsumedSnapshotId = lastPosition.snapshotId(); Snapshot toSnapshotInclusive = toSnapshotInclusive( lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java index 024d0b101165..26fbad46c128 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java @@ -29,7 +29,7 @@ public class IcebergEnumeratorState implements Serializable { @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; private final Collection pendingSplits; - private int[] enumerationSplitCountHistory; + private final int[] enumerationSplitCountHistory; public IcebergEnumeratorState(Collection pendingSplits) { this(null, pendingSplits); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java index 95d6db2cfbc4..f76f8a69ff0e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java @@ -174,10 +174,8 @@ private static Collection deserializePendingSplits( private static void serializeEnumerationSplitCountHistory( DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { out.writeInt(enumerationSplitCountHistory.length); - if (enumerationSplitCountHistory.length > 0) { - for (int enumerationSplitCount : enumerationSplitCountHistory) { - out.writeInt(enumerationSplitCount); - } + for (int enumerationSplitCount : enumerationSplitCountHistory) { + out.writeInt(enumerationSplitCount); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java index a0395f29ac5b..841969666ee5 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java @@ -129,7 +129,7 @@ public static String readLongUTF(DataInputDeserializer in) throws IOException { if (count > utflen) { throw new UTFDataFormatException("malformed input: partial character at end"); } - char2 = (int) bytearr[count - 1]; + char2 = bytearr[count - 1]; if ((char2 & 0xC0) != 0x80) { throw new UTFDataFormatException("malformed input around byte " + count); } @@ -141,8 +141,8 @@ public static String readLongUTF(DataInputDeserializer in) throws IOException { if (count > utflen) { throw new UTFDataFormatException("malformed input: partial character at end"); } - char2 = (int) bytearr[count - 2]; - char3 = (int) bytearr[count - 1]; + char2 = bytearr[count - 2]; + char3 = bytearr[count - 1]; if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { throw new UTFDataFormatException("malformed input around byte " + (count - 1)); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java index 353cee56bebb..20b33e615e5f 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java @@ -32,7 +32,7 @@ private FlinkPackage() {} /** Returns Flink version string like x.y.z */ public static String version() { if (null == VERSION.get()) { - String detectedVersion = null; + String detectedVersion; try { detectedVersion = versionFromJar(); // use unknown version in case exact implementation version can't be found from the jar diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index 97ab78716b8b..e69e1ac4d713 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -39,7 +39,7 @@ public class TestFlinkCatalogTablePartitions extends CatalogTestBase { - private String tableName = "test_table"; + private final String tableName = "test_table"; @Parameter(index = 2) private FileFormat format; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java index 4ad302dde436..a7c58e551112 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java @@ -23,7 +23,7 @@ import org.apache.iceberg.TestTables; public class TestTableLoader implements TableLoader { - private File dir; + private final File dir; public static TableLoader of(String dir) { return new TestTableLoader(dir); 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 index 90790b373d5f..91d36aa3e85d 100644 --- 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 @@ -26,6 +26,7 @@ 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; @@ -89,7 +90,9 @@ public void afterEach(ExtensionContext context) throws IOException { tables.forEach(t -> exec("DROP TABLE IF EXISTS %s", t.getField(0))); exec("USE CATALOG default_catalog"); exec("DROP CATALOG IF EXISTS %s", catalogName); - Files.walk(warehouse).sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + try (Stream files = Files.walk(warehouse)) { + files.sorted(Comparator.reverseOrder()).map(Path::toFile).forEach(File::delete); + } } /** diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 7033fd30e84f..187f2b0b9e31 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -122,7 +122,7 @@ public void testTimeUnit() throws IOException { new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); assertThat(extractor.extractWatermark(split(0))) - .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L); + .isEqualTo(MIN_VALUES.get(0).get(columnName) / 1000L); } @TestTemplate diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java index b0ecb0ceffaa..c5328d8f97e9 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveClientPool.java @@ -100,8 +100,7 @@ protected IMetaStoreClient reconnect(IMetaStoreClient client) { @Override protected boolean isConnectionException(Exception e) { return super.isConnectionException(e) - || (e != null - && e instanceof MetaException + || (e instanceof MetaException && e.getMessage() .contains("Got exception: org.apache.thrift.transport.TTransportException")); } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java index dfad78d344d9..da9ca21feea9 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveSchemaConverter.java @@ -42,7 +42,7 @@ class HiveSchemaConverter { private static final Logger LOG = LoggerFactory.getLogger(HiveSchemaConverter.class); private int id; - private boolean autoConvert; + private final boolean autoConvert; private HiveSchemaConverter(boolean autoConvert) { this.autoConvert = autoConvert; diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java index a0b9e8fa45d7..5b318fc753cb 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/ScriptRunner.java @@ -33,10 +33,10 @@ public class ScriptRunner { private static final String DEFAULT_DELIMITER = ";"; - private Connection connection; + private final Connection connection; - private boolean stopOnError; - private boolean autoCommit; + private final boolean stopOnError; + private final boolean autoCommit; private PrintWriter logWriter = new PrintWriter(System.out); private PrintWriter errorLogWriter = new PrintWriter(System.err); @@ -110,7 +110,7 @@ private void runScript(Connection conn, Reader reader) throws IOException, SQLEx StringBuilder command = null; try { LineNumberReader lineReader = new LineNumberReader(reader); - String line = null; + String line; while ((line = lineReader.readLine()) != null) { if (command == null) { command = new StringBuilder(); @@ -118,13 +118,11 @@ private void runScript(Connection conn, Reader reader) throws IOException, SQLEx String trimmedLine = line.trim(); if (trimmedLine.startsWith("--")) { println(trimmedLine); - } else if (trimmedLine.length() < 1 || trimmedLine.startsWith("//")) { - // Do nothing - } else if (trimmedLine.length() < 1 || trimmedLine.startsWith("--")) { + } else if (trimmedLine.isEmpty() || trimmedLine.startsWith("//")) { // Do nothing } else if (!fullLineDelimiter && trimmedLine.endsWith(getDelimiter()) || fullLineDelimiter && trimmedLine.equals(getDelimiter())) { - command.append(line.substring(0, line.lastIndexOf(getDelimiter()))); + command.append(line, 0, line.lastIndexOf(getDelimiter())); command.append(" "); Statement statement = conn.createStatement(); 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 95bf6c697c32..ccb8fc3a88f7 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 @@ -93,7 +93,7 @@ * Run all the tests from abstract of {@link CatalogTests} with few specific tests related to HIVE. */ public class TestHiveCatalog extends CatalogTests { - private static ImmutableMap meta = + private static final ImmutableMap meta = ImmutableMap.of( "owner", "apache", "group", "iceberg", diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java index 44707f216e18..61af2c7e79f3 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java @@ -86,7 +86,7 @@ public class TestHiveCommitLocks { private static HiveClientPool spyClientPool = null; private static CachedClientPool spyCachedClientPool = null; private static Configuration overriddenHiveConf; - private static AtomicReference spyClientRef = new AtomicReference<>(); + private static final AtomicReference spyClientRef = new AtomicReference<>(); private static IMetaStoreClient spyClient = null; HiveTableOperations ops = null; TableMetadata metadataV1 = null; @@ -387,7 +387,7 @@ public void testUnLockAfterInterruptedGetTable() throws TException { /** Wraps an ArgumentCaptor to provide data based on the request */ private class ShowLocksResponseElementWrapper extends ShowLocksResponseElement { - private ArgumentCaptor wrapped; + private final ArgumentCaptor wrapped; private ShowLocksResponseElementWrapper(ArgumentCaptor wrapped) { this.wrapped = wrapped; diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java b/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java index 62ce4930cc8a..585242cd1119 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/Deserializer.java @@ -41,7 +41,7 @@ import org.apache.iceberg.types.Types.StructType; class Deserializer { - private FieldDeserializer fieldDeserializer; + private final FieldDeserializer fieldDeserializer; /** * Builder to create a Deserializer instance. Requires an Iceberg Schema and the Hive @@ -279,8 +279,8 @@ String sourceName(String originalName) { * the provided writerInspector. */ private static class ObjectInspectorPair { - private ObjectInspector writerInspector; - private ObjectInspector sourceInspector; + private final ObjectInspector writerInspector; + private final ObjectInspector sourceInspector; ObjectInspectorPair(ObjectInspector writerInspector, ObjectInspector sourceInspector) { this.writerInspector = writerInspector; diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java index f206bba47605..59bb38e5b2d6 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergSerDe.java @@ -54,8 +54,9 @@ public class HiveIcebergSerDe extends AbstractSerDe { private ObjectInspector inspector; private Schema tableSchema; - private Map deserializers = Maps.newHashMapWithExpectedSize(1); - private Container row = new Container<>(); + private final Map deserializers = + Maps.newHashMapWithExpectedSize(1); + private final Container row = new Container<>(); @Override public void initialize(@Nullable Configuration configuration, Properties serDeProperties) diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java b/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java index 72475f70d718..f9f2334cfcf4 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java @@ -20,7 +20,6 @@ import java.io.IOException; import java.nio.file.Path; -import java.util.ArrayList; import java.util.Collections; import java.util.List; import java.util.Map; @@ -40,6 +39,7 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.rules.TemporaryFolder; public class TestHelper { @@ -152,7 +152,7 @@ private GenericAppenderHelper appender() { public static class RecordsBuilder { - private final List records = new ArrayList(); + private final List records = Lists.newArrayList(); private final Schema schema; private RecordsBuilder(Schema schema) { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java index 43a61d83e434..4e1779411add 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/HiveIcebergTestUtils.java @@ -39,6 +39,7 @@ import java.util.List; import java.util.UUID; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.common.type.HiveDecimal; import org.apache.hadoop.hive.serde2.io.DateWritable; @@ -279,11 +280,14 @@ public static void validateData(List expected, List actual, int */ public static void validateFiles(Table table, Configuration conf, JobID jobId, int dataFileNum) throws IOException { - List dataFiles = - Files.walk(Paths.get(table.location() + "/data")) - .filter(Files::isRegularFile) - .filter(path -> !path.getFileName().toString().startsWith(".")) - .collect(Collectors.toList()); + List dataFiles; + try (Stream files = Files.walk(Paths.get(table.location() + "/data"))) { + dataFiles = + files + .filter(Files::isRegularFile) + .filter(path -> !path.getFileName().toString().startsWith(".")) + .collect(Collectors.toList()); + } assertThat(dataFiles).hasSize(dataFileNum); assertThat( diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java index fd15a9a62621..9018d4518cbf 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerLocalScan.java @@ -781,7 +781,7 @@ private void runCreateAndReadTest( expected.addAll(data.get(partition)); } - List descRows = shell.executeStatement("SELECT * FROM " + identifier.toString()); + List descRows = shell.executeStatement("SELECT * FROM " + identifier); List records = HiveIcebergTestUtils.valueForRow(icebergTable.schema(), descRows); HiveIcebergTestUtils.validateData(expected, records, 0); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java index 18fc4f305070..ce3a6fd92441 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerWithEngine.java @@ -1341,14 +1341,15 @@ private StringBuilder buildComplexTypeInnerQuery(Object field, Type type) { } else if (type instanceof Types.StructType) { query.append("named_struct("); ((GenericRecord) field) - .struct().fields().stream() - .forEach( - f -> - query - .append(buildComplexTypeInnerQuery(f.name(), Types.StringType.get())) - .append( - buildComplexTypeInnerQuery( - ((GenericRecord) field).getField(f.name()), f.type()))); + .struct() + .fields() + .forEach( + f -> + query + .append(buildComplexTypeInnerQuery(f.name(), Types.StringType.get())) + .append( + buildComplexTypeInnerQuery( + ((GenericRecord) field).getField(f.name()), f.type()))); query.setLength(query.length() - 1); query.append("),"); } else if (type instanceof Types.StringType) { diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java index 8c8cf894c9f1..f2710290d5c2 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestTables.java @@ -552,10 +552,9 @@ private static String tablePath(TableIdentifier identifier) { private String getStringValueForInsert(Object value, Type type) { String template = "\'%s\'"; if (type.equals(Types.TimestampType.withoutZone())) { - return String.format(template, Timestamp.valueOf((LocalDateTime) value).toString()); + return String.format(template, Timestamp.valueOf((LocalDateTime) value)); } else if (type.equals(Types.TimestampType.withZone())) { - return String.format( - template, Timestamp.from(((OffsetDateTime) value).toInstant()).toString()); + return String.format(template, Timestamp.from(((OffsetDateTime) value).toInstant())); } else if (type.equals(Types.BooleanType.get())) { // in hive2 boolean type values must not be surrounded in apostrophes. Otherwise the value is // translated to true. diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 3fc38e436fb9..8bf6bc115d13 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -192,7 +192,7 @@ private Predicate namespacePredicate(Namespace ns) { private TableIdentifier toIdentifier(EntriesResponse.Entry entry) { List elements = entry.getName().getElements(); - return TableIdentifier.of(elements.toArray(new String[elements.size()])); + return TableIdentifier.of(elements.toArray(new String[0])); } public IcebergTable table(TableIdentifier tableIdentifier) { diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java index d56d07f7d27a..40e33759791f 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/BaseTestIceberg.java @@ -172,7 +172,7 @@ protected Table createTable(TableIdentifier tableIdentifier, int count) { createMissingNamespaces(tableIdentifier); return catalog.createTable(tableIdentifier, schema(count)); } catch (Throwable t) { - LOG.error("unable to do create " + tableIdentifier.toString(), t); + LOG.error("unable to do create {}", tableIdentifier, t); throw t; } } diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java index 05ee755d7c68..b1f844c6d991 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieIcebergClient.java @@ -168,7 +168,7 @@ public void testCreateNamespaceConflict() .hasMessageContaining("Namespace already exists: a"); client.commitTable( - null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); + null, newTableMetadata(), "file:///tmp/iceberg", null, ContentKey.of("a", "tbl")); assertThatThrownBy(() -> client.createNamespace(Namespace.of("a", "tbl"), Map.of())) .isInstanceOf(AlreadyExistsException.class) @@ -284,7 +284,7 @@ public void testDropNamespaceConflict() throws NessieConflictException, NessieNo client.createNamespace(Namespace.of("a"), Map.of()); client.commitTable( - null, newTableMetadata(), "file:///tmp/iceberg", (String) null, ContentKey.of("a", "tbl")); + null, newTableMetadata(), "file:///tmp/iceberg", null, ContentKey.of("a", "tbl")); assertThatThrownBy(() -> client.dropNamespace(Namespace.of("a", "tbl"))) .isInstanceOf(NoSuchNamespaceException.class) diff --git a/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java b/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java index 18ce07ac8b74..2dde66cdc488 100644 --- a/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java +++ b/orc/src/main/java/org/apache/iceberg/data/orc/GenericOrcReaders.java @@ -209,7 +209,7 @@ protected StructReader( Types.StructType structType, Map idToConstant) { super(readers, structType, idToConstant); - this.template = structType != null ? GenericRecord.create(structType) : null; + this.template = GenericRecord.create(structType); } @Override diff --git a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java index 609c8aedb563..2be64044e222 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java @@ -60,8 +60,8 @@ static SearchArgument convert(Expression expr, TypeDescription readSchema) { ImmutableSet.of( TypeID.BINARY, TypeID.FIXED, TypeID.UUID, TypeID.STRUCT, TypeID.MAP, TypeID.LIST); - private SearchArgument.Builder builder; - private Map idToColumnName; + private final SearchArgument.Builder builder; + private final Map idToColumnName; private ExpressionToSearchArgument( SearchArgument.Builder builder, Map idToColumnName) { diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 7d1405bbd45b..18186fe3f20e 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -115,7 +115,7 @@ public static class WriteBuilder { private final Configuration conf; private Schema schema = null; private BiFunction> createWriterFunc; - private Map metadata = Maps.newHashMap(); + private final Map metadata = Maps.newHashMap(); private MetricsConfig metricsConfig; private Function, Context> createContextFunc = Context::dataContext; private final Map config = Maps.newLinkedHashMap(); diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java index fae1a76c3706..ba6d6eedd1d4 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java @@ -232,7 +232,7 @@ public static Schema convert(TypeDescription orcSchema) { .map(Optional::get) .collect(Collectors.toList()); - if (fields.size() == 0) { + if (fields.isEmpty()) { throw new IllegalArgumentException("ORC schema does not contain Iceberg IDs"); } @@ -326,9 +326,7 @@ private static TypeDescription buildOrcProjection( } else { if (isRequired) { throw new IllegalArgumentException( - String.format( - "Field %d of type %s is required and was not found.", - fieldId, type.toString())); + String.format("Field %d of type %s is required and was not found.", fieldId, type)); } orcType = convert(fieldId, type, false); diff --git a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java index cfc29623dad6..2941fb89db62 100644 --- a/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java +++ b/orc/src/test/java/org/apache/iceberg/orc/TestBloomFilter.java @@ -107,7 +107,7 @@ public void testWriteOption() throws Exception { try (Reader reader = OrcFile.createReader( - new Path(outFile.location()), new OrcFile.ReaderOptions(new Configuration())); ) { + new Path(outFile.location()), new OrcFile.ReaderOptions(new Configuration()))) { boolean[] readCols = new boolean[] {false, true, true, false}; RecordReaderImpl rows = (RecordReaderImpl) reader.rows(); OrcIndex indices = rows.readRowIndex(0, null, readCols); 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 79585b097d27..38fd69393023 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 @@ -362,21 +362,19 @@ public ParquetValueReader primitive( case FIXED_LEN_BYTE_ARRAY: return new FixedReader(desc); case BINARY: - if (expected != null - && expected.typeId() == org.apache.iceberg.types.Type.TypeID.STRING) { + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.STRING) { return new ParquetValueReaders.StringReader(desc); } else { return new ParquetValueReaders.BytesReader(desc); } case INT32: - if (expected != null && expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { return new ParquetValueReaders.IntAsLongReader(desc); } else { return new ParquetValueReaders.UnboxedReader<>(desc); } case FLOAT: - if (expected != null - && expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { return new ParquetValueReaders.FloatAsDoubleReader(desc); } else { return new ParquetValueReaders.UnboxedReader<>(desc); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index c97512a17d87..3421c0b86d3a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -346,7 +346,7 @@ public FileAppender build() throws IOException { for (Map.Entry entry : columnBloomFilterEnabled.entrySet()) { String colPath = entry.getKey(); String bloomEnabled = entry.getValue(); - propsBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled)); + propsBuilder.withBloomFilterEnabled(colPath, Boolean.parseBoolean(bloomEnabled)); } for (Map.Entry entry : columnBloomFilterFpp.entrySet()) { @@ -389,7 +389,7 @@ public FileAppender build() throws IOException { for (Map.Entry entry : columnBloomFilterEnabled.entrySet()) { String colPath = entry.getKey(); String bloomEnabled = entry.getValue(); - parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled)); + parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.parseBoolean(bloomEnabled)); } for (Map.Entry entry : columnBloomFilterFpp.entrySet()) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java index 680eb1435967..65c472178d8a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetAvro.java @@ -301,7 +301,7 @@ public Schema map(Schema map, Schema value) { @Override public Schema primitive(Schema primitive) { LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null && logicalType instanceof LogicalTypes.Decimal) { + if (logicalType instanceof LogicalTypes.Decimal) { LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; if (decimal.getPrecision() <= 9) { return new ParquetDecimal(decimal.getPrecision(), decimal.getScale()) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java index 11201bd88e5a..553b8a0fa3f8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetBloomRowGroupFilter.java @@ -277,7 +277,7 @@ private BloomFilter loadBloomFilter(int id) { private boolean shouldRead( PrimitiveType primitiveType, T value, BloomFilter bloom, Type type) { - long hashValue = 0; + long hashValue; switch (primitiveType.getPrimitiveTypeName()) { case INT32: switch (type.typeId()) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java index 1f6e8554c463..17518a175f61 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetTypeVisitor.java @@ -90,7 +90,7 @@ private static T visitThreeLevelList( } private static T visitListElement(Type listElement, ParquetTypeVisitor visitor) { - T elementResult = null; + T elementResult; visitor.beforeElementField(listElement); try { 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 a335fe11cf49..62a49da25e56 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueReaders.java @@ -343,7 +343,7 @@ public BigDecimal read(BigDecimal ignored) { } public static class BinaryAsDecimalReader extends PrimitiveReader { - private int scale; + private final int scale; public BinaryAsDecimalReader(ColumnDescriptor desc, int scale) { super(desc); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java index 4eddf91a182f..90766983d8c8 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetValueWriters.java @@ -371,7 +371,7 @@ public Stream> metrics() { List> fieldMetricsFromWriter = writer.metrics().collect(Collectors.toList()); - if (fieldMetricsFromWriter.size() == 0) { + if (fieldMetricsFromWriter.isEmpty()) { // we are not tracking field metrics for this type ourselves return Stream.empty(); } else if (fieldMetricsFromWriter.size() == 1) { diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java index 049fed0decde..a4a325527710 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetWriteAdapter.java @@ -38,7 +38,7 @@ @Deprecated public class ParquetWriteAdapter implements FileAppender { private ParquetWriter writer; - private MetricsConfig metricsConfig; + private final MetricsConfig metricsConfig; private ParquetMetadata footer; public ParquetWriteAdapter(ParquetWriter writer, MetricsConfig metricsConfig) { 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 773e0f7a85d0..35d94f328d60 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/VectorizedParquetReader.java @@ -45,7 +45,7 @@ public class VectorizedParquetReader extends CloseableGroup implements Closea private final ParquetReadOptions options; private final Function> batchReaderFunc; private final Expression filter; - private boolean reuseContainers; + private final boolean reuseContainers; private final boolean caseSensitive; private final int batchSize; private final NameMapping nameMapping; diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java index 09673e603338..bced36aa4570 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/ParquetWritingTestUtils.java @@ -71,7 +71,7 @@ static long write( GenericData.Record... records) throws IOException { - long len = 0; + long len; FileAppender writer = Parquet.write(localOutput(file)) diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java index 5f64c7230601..2da9b2b9deba 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java @@ -149,7 +149,7 @@ public class TestBloomRowGroupFilter { static { StringBuilder sb = new StringBuilder(); for (int i = 0; i < 200; i += 1) { - sb.append(UUID.randomUUID().toString()); + sb.append(UUID.randomUUID()); } TOO_LONG_FOR_STATS = sb.toString(); } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java index cf2c1188bf44..d690d3cf5175 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java @@ -140,7 +140,7 @@ public class TestDictionaryRowGroupFilter { static { StringBuilder sb = new StringBuilder(); for (int i = 0; i < 200; i += 1) { - sb.append(UUID.randomUUID().toString()); + sb.append(UUID.randomUUID()); } TOO_LONG_FOR_STATS = sb.toString(); } 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 51221666a02b..5f3d3ac665fd 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java @@ -70,8 +70,8 @@ public class IcebergStorage extends LoadFunc public static final String PIG_ICEBERG_TABLES_IMPL = "pig.iceberg.tables.impl"; private static Tables iceberg; - private static Map tables = Maps.newConcurrentMap(); - private static Map locations = Maps.newConcurrentMap(); + private static final Map tables = Maps.newConcurrentMap(); + private static final Map locations = Maps.newConcurrentMap(); private String signature; diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java index d283fa906447..e947db362a3a 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/JdbcSnowflakeClient.java @@ -191,13 +191,11 @@ public boolean schemaExists(SnowflakeIdentifier schema) { final String finalQuery = "SHOW TABLES IN SCHEMA IDENTIFIER(?) LIMIT 1"; - List tables; try { - tables = - connectionPool.run( - conn -> - queryHarness.query( - conn, finalQuery, TABLE_RESULT_SET_HANDLER, schema.toIdentifierString())); + connectionPool.run( + conn -> + queryHarness.query( + conn, finalQuery, TABLE_RESULT_SET_HANDLER, schema.toIdentifierString())); } catch (SQLException e) { if (SCHEMA_NOT_FOUND_ERROR_CODES.contains(e.getErrorCode())) { return false; diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java index 06dacad185c6..7f64b4ca49fd 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java @@ -176,7 +176,7 @@ public void createNamespace(Namespace namespace, Map metadata) { @Override public List listNamespaces(Namespace namespace) { SnowflakeIdentifier scope = NamespaceHelpers.toSnowflakeIdentifier(namespace); - List results = null; + List results; switch (scope.type()) { case ROOT: results = snowflakeClient.listDatabases(); diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java index c550b3e13a3a..512c818eeaeb 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeTableMetadata.java @@ -93,7 +93,7 @@ public String toString() { } public String toDebugString() { - return String.format("%s, rawJsonVal: %s", toString(), rawJsonVal); + return String.format("%s, rawJsonVal: %s", this, rawJsonVal); } /** diff --git a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java index b3d5cef431e5..ecad072de724 100644 --- a/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java +++ b/snowflake/src/test/java/org/apache/iceberg/snowflake/SnowflakeCatalogTest.java @@ -89,7 +89,7 @@ public void before() { "s3://tab1/metadata/v3.metadata.json", TableMetadataParser.toJson( TableMetadata.newTableMetadata( - schema, partitionSpec, "s3://tab1", ImmutableMap.of())) + schema, partitionSpec, "s3://tab1", ImmutableMap.of())) .getBytes()); fakeFileIO.addFile( "wasbs://mycontainer@myaccount.blob.core.windows.net/tab3/metadata/v334.metadata.json", @@ -98,13 +98,13 @@ public void before() { schema, partitionSpec, "wasbs://mycontainer@myaccount.blob.core.windows.net/tab1/", - ImmutableMap.of())) + ImmutableMap.of())) .getBytes()); fakeFileIO.addFile( "gs://tab5/metadata/v793.metadata.json", TableMetadataParser.toJson( TableMetadata.newTableMetadata( - schema, partitionSpec, "gs://tab5/", ImmutableMap.of())) + schema, partitionSpec, "gs://tab5/", ImmutableMap.of())) .getBytes()); fakeFileIOFactory = 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 f00b94219078..7af9dfc58737 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 @@ -255,7 +255,7 @@ protected void createOrReplaceView(String name, List data, Encoder enc private Dataset toDS(String schema, String jsonData) { List jsonRows = Arrays.stream(jsonData.split("\n")) - .filter(str -> str.trim().length() > 0) + .filter(str -> !str.trim().isEmpty()) .collect(Collectors.toList()); Dataset jsonDS = spark.createDataset(jsonRows, Encoders.STRING()); diff --git a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java index 63d24f7da553..d8f9301a7d82 100644 --- a/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java +++ b/spark/v3.5/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java @@ -29,7 +29,7 @@ class RandomGeneratingUDF implements Serializable { private final long uniqueValues; - private Random rand = new Random(); + private final Random rand = new Random(); RandomGeneratingUDF(long uniqueValues) { this.uniqueValues = uniqueValues; @@ -43,8 +43,7 @@ UserDefinedFunction randomLongUDF() { UserDefinedFunction randomString() { return udf( - () -> (String) RandomUtil.generatePrimitive(Types.StringType.get(), rand), - DataTypes.StringType) + () -> RandomUtil.generatePrimitive(Types.StringType.get(), rand), DataTypes.StringType) .asNondeterministic() .asNonNullable(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java index bbbb46e803e2..f76f12355f1f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java @@ -203,7 +203,7 @@ public Type map(Types.MapType map, Supplier keyResult, Supplier valu "Cannot project a map of optional values as required values: %s", map); Preconditions.checkArgument( - StringType.class.isInstance(requestedMap.keyType()), + requestedMap.keyType() instanceof StringType, "Invalid map key type (not string): %s", requestedMap.keyType()); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java index 39518ffbb945..af0fa84f67a1 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -888,7 +888,7 @@ public static List getPartitions( JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(rootPath)) .asScala() .toSeq(), - scala.collection.immutable.Map$.MODULE$.empty(), + scala.collection.immutable.Map$.MODULE$.empty(), userSpecifiedSchema, fileStatusCache, Option.empty(), 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 8447dbdcead1..7a96e97fb98a 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 @@ -183,7 +183,7 @@ public static List getPartitions( Option> scalaPartitionFilter; if (partitionFilter != null && !partitionFilter.isEmpty()) { Builder, scala.collection.immutable.Map> builder = - Map$.MODULE$.newBuilder(); + Map$.MODULE$.newBuilder(); partitionFilter.forEach((key, value) -> builder.$plus$eq(Tuple2.apply(key, value))); scalaPartitionFilter = Option.apply(builder.result()); } else { diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index b00ed42008f1..5fbb4117feb8 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -116,7 +116,7 @@ public class DeleteOrphanFilesSparkAction extends BaseSparkAction equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT); private Map equalAuthorities = Collections.emptyMap(); private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR; - private String location = null; + private String location; private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); private Dataset compareToFileList; private Consumer deleteFunc = null; 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 e301ad512c70..d33e5e540893 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 @@ -393,7 +393,7 @@ private Result doExecuteWithPartialProgress( Stream toGroupStream( RewriteExecutionContext ctx, Map>> groupsByPartition) { return groupsByPartition.entrySet().stream() - .filter(e -> e.getValue().size() != 0) + .filter(e -> !e.getValue().isEmpty()) .flatMap( e -> { StructLike partition = e.getKey(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index de66b06924f3..8410fa7bf93e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -112,9 +112,9 @@ public class RewriteManifestsSparkAction private final long targetManifestSizeBytes; private final boolean shouldStageManifests; - private PartitionSpec spec = null; + private PartitionSpec spec; private Predicate predicate = manifest -> true; - private String outputLocation = null; + private String outputLocation; private List partitionFieldClustering = null; private Function partitionClusteringFunction = null; 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 1166740f441a..282222ae716f 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 @@ -309,7 +309,7 @@ private Result doExecuteWithPartialProgress( // stop commit service commitService.close(); List commitResults = commitService.results(); - if (commitResults.size() == 0) { + if (commitResults.isEmpty()) { LOG.error( "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " @@ -331,7 +331,7 @@ private Stream toGroupStream( RewriteExecutionContext ctx, Map>> groupsByPartition) { return groupsByPartition.entrySet().stream() - .filter(e -> e.getValue().size() != 0) + .filter(e -> !e.getValue().isEmpty()) .flatMap( e -> { StructLike partition = e.getKey(); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index 209c06bacb3e..678ebd218d71 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -560,7 +560,7 @@ private static class InternalRowWriter extends ParquetValueWriters.StructWriter< private InternalRowWriter(List> writers, List types) { super(writers); - this.types = types.toArray(new DataType[types.size()]); + this.types = types.toArray(new DataType[0]); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java index 9b075b675565..50a1259c8626 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java @@ -48,7 +48,7 @@ class SparkFileWriterFactory extends BaseFileWriterFactory { private StructType dataSparkType; private StructType equalityDeleteSparkType; private StructType positionDeleteSparkType; - private Map writeProperties; + private final Map writeProperties; SparkFileWriterFactory( Table table, diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java index 320d2e14adc9..49180e07c465 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkMicroBatchStream.java @@ -395,7 +395,6 @@ public Offset latestOffset(Offset startOffset, ReadLimit limit) { Snapshot nextValid = nextValidSnapshot(curSnapshot); if (nextValid == null) { // nextValide implies all the remaining snapshots should be skipped. - shouldContinueReading = false; break; } // we found the next available snapshot, continue from there. 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 b430e6fca233..6b97e48133fd 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 @@ -95,7 +95,7 @@ public class SparkScanBuilder private final List metaColumns = Lists.newArrayList(); private final InMemoryMetricsReporter metricsReporter; - private Schema schema = null; + private Schema schema; private boolean caseSensitive; private List filterExpressions = null; private Predicate[] pushedPredicates = NO_PREDICATES; diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 25393888f95c..c5c86c3ebf28 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -44,7 +44,7 @@ class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredCol private final SparkReadConf readConf; private final List metaColumns = Lists.newArrayList(); - private Schema schema = null; + private Schema schema; SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { this.spark = spark; diff --git a/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java b/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java index 075733aeab71..9ed7167c94c6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java +++ b/spark/v3.5/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java @@ -32,7 +32,7 @@ public NoSuchProcedureException(Identifier ident) { Option.empty(), Option.empty(), Option.empty(), - Map$.MODULE$.empty(), + Map$.MODULE$.empty(), new QueryContext[0]); } } 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 e7d5a0f0398a..40bc4bbce54e 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 @@ -118,7 +118,7 @@ protected long waitUntilAfter(long timestampMillis) { protected List sql(String query, Object... args) { List rows = spark.sql(String.format(query, args)).collectAsList(); - if (rows.size() < 1) { + if (rows.isEmpty()) { return ImmutableList.of(); } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java index d0860ff01451..8f4c09356d13 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java @@ -38,7 +38,7 @@ import org.junit.jupiter.api.TestTemplate; public class TestSparkCatalogOperations extends CatalogTestBase { - private static boolean useNullableQuerySchema = ThreadLocalRandom.current().nextBoolean(); + private static final boolean useNullableQuerySchema = ThreadLocalRandom.current().nextBoolean(); @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") protected static Object[][] parameters() { 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 db77da449e5d..a0f1fba0434c 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 @@ -141,7 +141,7 @@ public static Object[][] parameters() { }; } - private String baseTableName = "baseTable"; + private final String baseTableName = "baseTable"; private File tableDir; private String tableLocation; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java index 478afcf09ae3..360b9ff20ec0 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java @@ -67,7 +67,7 @@ public static List generateList(Schema schema, int numRecords, long seed public static Iterable generateSpark(Schema schema, int numRecords, long seed) { return () -> new Iterator() { - private SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); + private final SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); private int count = 0; @Override @@ -114,7 +114,7 @@ private static Iterable newIterable( return () -> new Iterator() { private int count = 0; - private RandomDataGenerator generator = newGenerator.get(); + private final RandomDataGenerator generator = newGenerator.get(); @Override public boolean hasNext() { diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java index 53a35eec61ce..2627ec0d2b55 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger; public class LogMessage { - private static AtomicInteger idCounter = new AtomicInteger(0); + private static final AtomicInteger idCounter = new AtomicInteger(0); static LogMessage debug(String date, String message) { return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message); 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 27e7d7c496ef..7258344823f7 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 @@ -87,7 +87,7 @@ public Integer next() { // Main reader class to test base class iteration logic. // Keeps track of iterator closure. private static class ClosureTrackingReader extends BaseReader { - private Map tracker = Maps.newHashMap(); + private final Map tracker = Maps.newHashMap(); ClosureTrackingReader(Table table, List tasks) { super(table, new BaseCombinedScanTask(tasks), null, null, false); 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 34315bc8a991..44885eb84551 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 @@ -245,7 +245,7 @@ private String getCompressionType(InputFile inputFile) throws Exception { return footer.getBlocks().get(0).getColumns().get(0).getCodec().name(); default: FileContext fc = FileContext.getFileContext(CONF); - GenericDatumReader reader = new GenericDatumReader(); + GenericDatumReader reader = new GenericDatumReader<>(); DataFileReader fileReader = (DataFileReader) DataFileReader.openReader( 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 e702f1f01eed..336ee5a8d2ea 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 @@ -91,7 +91,7 @@ public static Collection parameters() { private Map tableProperties; - private org.apache.spark.sql.types.StructType sparkSchema = + 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( @@ -106,16 +106,16 @@ public static Collection parameters() { org.apache.spark.sql.types.Metadata.empty()) }); - private Schema icebergSchema = + private final Schema icebergSchema = new Schema( Types.NestedField.optional(1, "optionalField", Types.StringType.get()), Types.NestedField.required(2, "requiredField", Types.StringType.get())); - private List data0 = + private final List data0 = Arrays.asList( "{\"optionalField\": \"a1\", \"requiredField\": \"bid_001\"}", "{\"optionalField\": \"a2\", \"requiredField\": \"bid_002\"}"); - private List data1 = + private final List data1 = Arrays.asList( "{\"optionalField\": \"d1\", \"requiredField\": \"bid_101\"}", "{\"optionalField\": \"d2\", \"requiredField\": \"bid_102\"}", @@ -220,16 +220,15 @@ private void writeDataWithFailOnPartition( final int numPartitions = 10; final int partitionToFail = new Random().nextInt(numPartitions); MapPartitionsFunction failOnFirstPartitionFunc = - (MapPartitionsFunction) - input -> { - int partitionId = TaskContext.getPartitionId(); + input -> { + int partitionId = TaskContext.getPartitionId(); - if (partitionId == partitionToFail) { - throw new SparkException( - String.format("Intended exception in partition %d !", partitionId)); - } - return input; - }; + if (partitionId == partitionToFail) { + throw new SparkException( + String.format("Intended exception in partition %d !", partitionId)); + } + return input; + }; Dataset df = createDataset(records, schema) @@ -287,8 +286,8 @@ public void testNullableWithWriteOption() throws IOException { .startsWith("2"); File location = temp.resolve("parquet").resolve("test").toFile(); - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString()); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString()); + 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); @@ -341,8 +340,8 @@ public void testNullableWithSparkSqlOption() throws IOException { .startsWith("2"); File location = temp.resolve("parquet").resolve("test").toFile(); - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString()); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString()); + 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); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java index 0febfd687a00..35a675029c1c 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java @@ -134,7 +134,7 @@ public static Object[][] parameters() { @TempDir private Path temp; - private PartitionSpec spec = + private final PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build(); private Table table = null; private Dataset logs = null; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java index 2bd6dadff016..9464f687b0eb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java @@ -170,7 +170,7 @@ private static Instant getInstant(String timestampWithoutZone) { @TempDir private java.nio.file.Path temp; - private PartitionSpec spec = + private final PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA) .identity("date") .identity("level") diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index 3d9d6eb8ebaf..17db46b85c35 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -115,7 +115,7 @@ public void testStreamingWriteAppendMode() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); // restart the query from the checkpoint @@ -176,7 +176,7 @@ public void testStreamingWriteCompleteMode() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); // restart the query from the checkpoint @@ -237,7 +237,7 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); assertThat(lastCommitFile.delete()).as("The commit file must be deleted").isTrue(); // restart the query from the checkpoint 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 a2a315f5625f..ae0aa2cda49b 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 @@ -295,7 +295,7 @@ public void testCreateTableLocation() throws Exception { File tableLocation = Files.createTempDirectory(temp, "junit").toFile(); assertThat(tableLocation.delete()).isTrue(); - String location = "file:" + tableLocation.toString(); + String location = "file:" + tableLocation; sql( "CREATE TABLE %s " diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java index ae9aee8ca416..0ba480692523 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestNamespaceSQL.java @@ -64,7 +64,7 @@ protected static Object[][] parameters() { SparkCatalogConfig.HADOOP.catalogName(), SparkCatalogConfig.HADOOP.implementation(), SparkCatalogConfig.HADOOP.properties(), - SparkCatalogConfig.HADOOP.catalogName() + "." + NS.toString(), + SparkCatalogConfig.HADOOP.catalogName() + "." + NS, true }, { From 854ed03d87689cede90327e856bb8257aeccb4bf Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Wed, 3 Jul 2024 14:39:08 +0200 Subject: [PATCH 0429/1019] Build: Enable the Gradle build cache (#10602) --- gradle.properties | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/gradle.properties b/gradle.properties index f593e700fcd1..c6b8dec17bc5 100644 --- a/gradle.properties +++ b/gradle.properties @@ -24,5 +24,12 @@ systemProp.defaultSparkVersions=3.5 systemProp.knownSparkVersions=3.3,3.4,3.5 systemProp.defaultScalaVersion=2.12 systemProp.knownScalaVersions=2.12,2.13 +# enable the Gradle build cache - speeds up builds! +org.gradle.caching=true +# enable Gradle parallel builds org.gradle.parallel=true +# configure only necessary Gradle tasks +org.gradle.configureondemand=true +# explicitly disable the configuration cache +org.gradle.configuration-cache=false org.gradle.jvmargs=-Xmx1024m From c0b67c028ad5c30c46b842b06bc1b35a1b3e2f5d Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Thu, 4 Jul 2024 01:36:49 +0200 Subject: [PATCH 0430/1019] API, Flink, ORC: Fix implicit `long` casting issues (#10580) --- .../apache/iceberg/metrics/TestFixedReservoirHistogram.java | 2 +- .../org/apache/iceberg/flink/data/FlinkParquetReaders.java | 4 ++-- .../source/assigner/TestWatermarkBasedSplitAssigner.java | 2 +- .../org/apache/iceberg/orc/ExpressionToSearchArgument.java | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java b/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java index 018c2c3b58b2..f5f47648ecb0 100644 --- a/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java +++ b/api/src/test/java/org/apache/iceberg/metrics/TestFixedReservoirHistogram.java @@ -113,7 +113,7 @@ public void testMultipleThreadWriters() throws InterruptedException { try { barrier.await(30, SECONDS); for (int i = 1; i <= 100; ++i) { - histogram.update(threadIndex * samplesPerThread + i); + histogram.update((long) threadIndex * samplesPerThread + i); } return threadIndex; } catch (Exception e) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index ad4310a6d195..d0a1974636be 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -422,7 +422,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromLocalDateTime( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) + Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000L) .atOffset(ZoneOffset.UTC) .toLocalDateTime()); } @@ -444,7 +444,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromInstant( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); + Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000L)); } @Override diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java index e1fc63fda918..6c0cbb733320 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -123,7 +123,7 @@ protected List createSplits( .mapToObj( fileNum -> RandomGenericData.generate( - SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + SCHEMA, 2, (long) splitNum * filesPerSplit + fileNum)) .collect(Collectors.toList()))) .collect(Collectors.toList()); } diff --git a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java index 2be64044e222..650292302ea9 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ExpressionToSearchArgument.java @@ -332,7 +332,7 @@ private Object literal(Type icebergType, T icebergLiteral) { return Timestamp.from( Instant.ofEpochSecond( Math.floorDiv(microsFromEpoch, 1_000_000), - Math.floorMod(microsFromEpoch, 1_000_000) * 1_000)); + Math.floorMod(microsFromEpoch, 1_000_000) * 1_000L)); case DECIMAL: return new HiveDecimalWritable(HiveDecimal.create((BigDecimal) icebergLiteral, false)); default: From 9b363db5b6592fff45a0b0c1bbdbcb329a984662 Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Thu, 4 Jul 2024 08:14:58 +0200 Subject: [PATCH 0431/1019] REST: disallow overriding "credential" in table sessions (#10345) See #10256 for context. This change disallows overriding the "credential" property in table sessions, by introducing an allow-list of auth-related properties that can be overridden in such situations. Only the "token" property and properties used to exchange one token for another ("urn:ietf:params:oauth:token-type:*") are now allowed. --- .../iceberg/rest/RESTSessionCatalog.java | 17 ++++- .../apache/iceberg/rest/TestRESTCatalog.java | 74 ++++++++++--------- 2 files changed, 54 insertions(+), 37 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 11a6aa6f27da..56e07202f873 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -68,6 +68,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.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.rest.auth.AuthConfig; @@ -124,6 +125,13 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog OAuth2Properties.SAML2_TOKEN_TYPE, OAuth2Properties.SAML1_TOKEN_TYPE); + // Auth-related properties that are allowed to be passed to the table session + private static final Set TABLE_SESSION_ALLOW_LIST = + ImmutableSet.builder() + .add(OAuth2Properties.TOKEN) + .addAll(TOKEN_PREFERENCE_ORDER) + .build(); + private final Function, RESTClient> clientBuilder; private final BiFunction, FileIO> ioBuilder; private Cache sessions = null; @@ -922,7 +930,14 @@ private FileIO tableFileIO(SessionContext context, Map config) { } private AuthSession tableSession(Map tableConf, AuthSession parent) { - Pair> newSession = newSession(tableConf, tableConf, parent); + Map credentials = Maps.newHashMapWithExpectedSize(tableConf.size()); + for (String prop : tableConf.keySet()) { + if (TABLE_SESSION_ALLOW_LIST.contains(prop)) { + credentials.put(prop, tableConf.get(prop)); + } + } + + Pair> newSession = newSession(credentials, tableConf, parent); if (null == newSession) { return parent; } 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 6f79733179ed..ced15b865115 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -900,9 +900,9 @@ public void testTableCredential(String oauth2ServerUri) { testTableAuth( "catalog", ImmutableMap.of("urn:ietf:params:oauth:token-type:id_token", "id-token"), - ImmutableMap.of("credential", "table-user:secret"), + ImmutableMap.of("credential", "table-user:secret"), // will be ignored + ImmutableMap.of("Authorization", "Bearer token-exchange-token:sub=id-token,act=catalog"), ImmutableMap.of("Authorization", "Bearer token-exchange-token:sub=id-token,act=catalog"), - ImmutableMap.of("Authorization", "Bearer client-credentials-token:sub=table-user"), oauth2ServerUri); } @@ -1352,9 +1352,9 @@ public void testTableAuth( eq(expectedContextHeaders), any()); - // if the table returned a bearer token, there will be no token request - if (!tableConfig.containsKey("token")) { - // client credentials or token exchange to get a table token + // if the table returned a bearer token or a credential, there will be no token request + if (!tableConfig.containsKey("token") && !tableConfig.containsKey("credential")) { + // token exchange to get a table token Mockito.verify(adapter, times(1)) .execute( eq(HTTPMethod.POST), @@ -1366,38 +1366,40 @@ public void testTableAuth( any()); } - // automatic refresh when metadata is accessed after commit - Mockito.verify(adapter) - .execute( - eq(HTTPMethod.GET), - eq("v1/namespaces/ns/tables/table"), - any(), - any(), - eq(LoadTableResponse.class), - eq(expectedTableHeaders), - any()); - - // load table from catalog - Mockito.verify(adapter) - .execute( - eq(HTTPMethod.GET), - eq("v1/namespaces/ns/tables/table"), - any(), - any(), - eq(LoadTableResponse.class), - eq(expectedContextHeaders), - any()); + if (expectedContextHeaders.equals(expectedTableHeaders)) { + // load table from catalog + refresh loaded table + Mockito.verify(adapter, times(2)) + .execute( + eq(HTTPMethod.GET), + eq("v1/namespaces/ns/tables/table"), + any(), + any(), + eq(LoadTableResponse.class), + eq(expectedTableHeaders), + any()); + } else { + // load table from catalog + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/namespaces/ns/tables/table"), + any(), + any(), + eq(LoadTableResponse.class), + eq(expectedContextHeaders), + any()); - // refresh loaded table - Mockito.verify(adapter) - .execute( - eq(HTTPMethod.GET), - eq("v1/namespaces/ns/tables/table"), - any(), - any(), - eq(LoadTableResponse.class), - eq(expectedTableHeaders), - any()); + // refresh loaded table + Mockito.verify(adapter) + .execute( + eq(HTTPMethod.GET), + eq("v1/namespaces/ns/tables/table"), + any(), + any(), + eq(LoadTableResponse.class), + eq(expectedTableHeaders), + any()); + } } @ParameterizedTest From 95f5ecc881d78036ab74c668e50045aa76b4f0b9 Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Thu, 4 Jul 2024 08:23:54 +0200 Subject: [PATCH 0432/1019] Apply IntelliJ inspection findings to older Spark + Flink versions (#10625) --- .../apache/iceberg/flink/actions/Actions.java | 4 +-- .../flink/actions/RewriteDataFilesAction.java | 2 +- .../AvroGenericRecordToRowDataMapper.java | 2 +- .../flink/sink/shuffle/MapDataStatistics.java | 2 +- .../RowDataToAvroGenericRecordConverter.java | 2 +- .../ContinuousSplitPlannerImpl.java | 2 +- .../enumerator/IcebergEnumeratorState.java | 2 +- .../IcebergEnumeratorStateSerializer.java | 6 ++-- .../flink/source/split/SerializerHelper.java | 6 ++-- .../iceberg/flink/util/FlinkPackage.java | 2 +- .../TestFlinkCatalogTablePartitions.java | 2 +- .../apache/iceberg/flink/TestTableLoader.java | 2 +- .../TestColumnStatsWatermarkExtractor.java | 2 +- .../apache/iceberg/flink/actions/Actions.java | 4 +-- .../flink/actions/RewriteDataFilesAction.java | 2 +- .../AvroGenericRecordToRowDataMapper.java | 2 +- .../flink/sink/shuffle/MapDataStatistics.java | 2 +- .../RowDataToAvroGenericRecordConverter.java | 2 +- .../ContinuousSplitPlannerImpl.java | 2 +- .../enumerator/IcebergEnumeratorState.java | 2 +- .../IcebergEnumeratorStateSerializer.java | 6 ++-- .../flink/source/split/SerializerHelper.java | 6 ++-- .../iceberg/flink/util/FlinkPackage.java | 2 +- .../TestFlinkCatalogTablePartitions.java | 2 +- .../apache/iceberg/flink/TestTableLoader.java | 2 +- .../TestColumnStatsWatermarkExtractor.java | 2 +- .../spark/action/RandomGeneratingUDF.java | 5 ++- .../spark/PruneColumnsWithReordering.java | 2 +- .../org/apache/iceberg/spark/Spark3Util.java | 2 +- .../apache/iceberg/spark/SparkTableUtil.java | 2 +- .../actions/DeleteOrphanFilesSparkAction.java | 2 +- .../actions/RewriteDataFilesSparkAction.java | 2 +- .../actions/RewriteManifestsSparkAction.java | 4 +-- ...RewritePositionDeleteFilesSparkAction.java | 4 +-- .../spark/data/SparkParquetWriters.java | 2 +- .../spark/source/SparkScanBuilder.java | 2 +- .../spark/source/SparkStagedScanBuilder.java | 2 +- .../spark/actions/TestCreateActions.java | 2 +- .../apache/iceberg/spark/data/RandomData.java | 4 +-- .../iceberg/spark/source/LogMessage.java | 2 +- .../iceberg/spark/source/TestBaseReader.java | 2 +- .../spark/source/TestDataFrameWrites.java | 33 +++++++++---------- .../source/TestIdentityPartitionData.java | 2 +- .../spark/source/TestPartitionPruning.java | 2 +- .../spark/source/TestStructuredStreaming.java | 6 ++-- .../iceberg/spark/sql/TestCreateTable.java | 2 +- .../SparkRowLevelOperationsTestBase.java | 2 +- .../spark/action/RandomGeneratingUDF.java | 5 ++- .../spark/PruneColumnsWithReordering.java | 2 +- .../org/apache/iceberg/spark/Spark3Util.java | 2 +- .../apache/iceberg/spark/SparkTableUtil.java | 2 +- .../actions/DeleteOrphanFilesSparkAction.java | 2 +- .../actions/RewriteDataFilesSparkAction.java | 2 +- .../actions/RewriteManifestsSparkAction.java | 4 +-- ...RewritePositionDeleteFilesSparkAction.java | 4 +-- .../spark/data/SparkParquetWriters.java | 2 +- .../spark/source/SparkFileWriterFactory.java | 2 +- .../spark/source/SparkScanBuilder.java | 2 +- .../spark/source/SparkStagedScanBuilder.java | 2 +- .../analysis/NoSuchProcedureException.java | 2 +- .../spark/actions/TestCreateActions.java | 2 +- .../apache/iceberg/spark/data/RandomData.java | 4 +-- .../iceberg/spark/source/LogMessage.java | 2 +- .../iceberg/spark/source/TestBaseReader.java | 2 +- .../spark/source/TestCompressionSettings.java | 2 +- .../spark/source/TestDataFrameWrites.java | 33 +++++++++---------- .../source/TestIdentityPartitionData.java | 2 +- .../spark/source/TestPartitionPruning.java | 2 +- .../spark/source/TestStructuredStreaming.java | 6 ++-- .../iceberg/spark/sql/TestCreateTable.java | 2 +- 70 files changed, 120 insertions(+), 128 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java index 06ac54617ae6..b96b47c5a785 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java @@ -30,8 +30,8 @@ public class Actions { // disable classloader check as Avro may cache class/object in the serializers. .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - private StreamExecutionEnvironment env; - private Table table; + private final StreamExecutionEnvironment env; + private final Table table; private Actions(StreamExecutionEnvironment env, Table table) { this.env = env; diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java index 9876bb3861c4..670abebcb58a 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java @@ -31,7 +31,7 @@ public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { - private StreamExecutionEnvironment env; + private final StreamExecutionEnvironment env; private int maxParallelism; public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java index 04e168385a36..f7e8e0c884cf 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java @@ -55,7 +55,7 @@ public RowData map(GenericRecord genericRecord) throws Exception { public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); return new AvroGenericRecordToRowDataMapper(rowType); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 0ffffd9cf49f..0b63e2721178 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -39,7 +39,7 @@ class MapDataStatistics implements DataStatistics pendingSplits; - private int[] enumerationSplitCountHistory; + private final int[] enumerationSplitCountHistory; public IcebergEnumeratorState(Collection pendingSplits) { this(null, pendingSplits); diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java index 95d6db2cfbc4..f76f8a69ff0e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java @@ -174,10 +174,8 @@ private static Collection deserializePendingSplits( private static void serializeEnumerationSplitCountHistory( DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { out.writeInt(enumerationSplitCountHistory.length); - if (enumerationSplitCountHistory.length > 0) { - for (int enumerationSplitCount : enumerationSplitCountHistory) { - out.writeInt(enumerationSplitCount); - } + for (int enumerationSplitCount : enumerationSplitCountHistory) { + out.writeInt(enumerationSplitCount); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java index a0395f29ac5b..841969666ee5 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java @@ -129,7 +129,7 @@ public static String readLongUTF(DataInputDeserializer in) throws IOException { if (count > utflen) { throw new UTFDataFormatException("malformed input: partial character at end"); } - char2 = (int) bytearr[count - 1]; + char2 = bytearr[count - 1]; if ((char2 & 0xC0) != 0x80) { throw new UTFDataFormatException("malformed input around byte " + count); } @@ -141,8 +141,8 @@ public static String readLongUTF(DataInputDeserializer in) throws IOException { if (count > utflen) { throw new UTFDataFormatException("malformed input: partial character at end"); } - char2 = (int) bytearr[count - 2]; - char3 = (int) bytearr[count - 1]; + char2 = bytearr[count - 2]; + char3 = bytearr[count - 1]; if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { throw new UTFDataFormatException("malformed input around byte " + (count - 1)); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java index 353cee56bebb..20b33e615e5f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java @@ -32,7 +32,7 @@ private FlinkPackage() {} /** Returns Flink version string like x.y.z */ public static String version() { if (null == VERSION.get()) { - String detectedVersion = null; + String detectedVersion; try { detectedVersion = versionFromJar(); // use unknown version in case exact implementation version can't be found from the jar diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index 0bb08e4f265a..e395414e925d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -39,7 +39,7 @@ public class TestFlinkCatalogTablePartitions extends CatalogTestBase { - private String tableName = "test_table"; + private final String tableName = "test_table"; @Parameter(index = 2) private FileFormat format; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java index 4ad302dde436..a7c58e551112 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java @@ -23,7 +23,7 @@ import org.apache.iceberg.TestTables; public class TestTableLoader implements TableLoader { - private File dir; + private final File dir; public static TableLoader of(String dir) { return new TestTableLoader(dir); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 7033fd30e84f..187f2b0b9e31 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -122,7 +122,7 @@ public void testTimeUnit() throws IOException { new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); assertThat(extractor.extractWatermark(split(0))) - .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L); + .isEqualTo(MIN_VALUES.get(0).get(columnName) / 1000L); } @TestTemplate diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java index 06ac54617ae6..b96b47c5a785 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java @@ -30,8 +30,8 @@ public class Actions { // disable classloader check as Avro may cache class/object in the serializers. .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - private StreamExecutionEnvironment env; - private Table table; + private final StreamExecutionEnvironment env; + private final Table table; private Actions(StreamExecutionEnvironment env, Table table) { this.env = env; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java index 9876bb3861c4..670abebcb58a 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java @@ -31,7 +31,7 @@ public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { - private StreamExecutionEnvironment env; + private final StreamExecutionEnvironment env; private int maxParallelism; public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java index 04e168385a36..f7e8e0c884cf 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java @@ -55,7 +55,7 @@ public RowData map(GenericRecord genericRecord) throws Exception { public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().stream().toArray(LogicalType[]::new)); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); return new AvroGenericRecordToRowDataMapper(rowType); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 0ffffd9cf49f..0b63e2721178 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -39,7 +39,7 @@ class MapDataStatistics implements DataStatistics pendingSplits; - private int[] enumerationSplitCountHistory; + private final int[] enumerationSplitCountHistory; public IcebergEnumeratorState(Collection pendingSplits) { this(null, pendingSplits); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java index 95d6db2cfbc4..f76f8a69ff0e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java @@ -174,10 +174,8 @@ private static Collection deserializePendingSplits( private static void serializeEnumerationSplitCountHistory( DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { out.writeInt(enumerationSplitCountHistory.length); - if (enumerationSplitCountHistory.length > 0) { - for (int enumerationSplitCount : enumerationSplitCountHistory) { - out.writeInt(enumerationSplitCount); - } + for (int enumerationSplitCount : enumerationSplitCountHistory) { + out.writeInt(enumerationSplitCount); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java index a0395f29ac5b..841969666ee5 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java @@ -129,7 +129,7 @@ public static String readLongUTF(DataInputDeserializer in) throws IOException { if (count > utflen) { throw new UTFDataFormatException("malformed input: partial character at end"); } - char2 = (int) bytearr[count - 1]; + char2 = bytearr[count - 1]; if ((char2 & 0xC0) != 0x80) { throw new UTFDataFormatException("malformed input around byte " + count); } @@ -141,8 +141,8 @@ public static String readLongUTF(DataInputDeserializer in) throws IOException { if (count > utflen) { throw new UTFDataFormatException("malformed input: partial character at end"); } - char2 = (int) bytearr[count - 2]; - char3 = (int) bytearr[count - 1]; + char2 = bytearr[count - 2]; + char3 = bytearr[count - 1]; if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { throw new UTFDataFormatException("malformed input around byte " + (count - 1)); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java index 353cee56bebb..20b33e615e5f 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java @@ -32,7 +32,7 @@ private FlinkPackage() {} /** Returns Flink version string like x.y.z */ public static String version() { if (null == VERSION.get()) { - String detectedVersion = null; + String detectedVersion; try { detectedVersion = versionFromJar(); // use unknown version in case exact implementation version can't be found from the jar diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java index 0bb08e4f265a..e395414e925d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -39,7 +39,7 @@ public class TestFlinkCatalogTablePartitions extends CatalogTestBase { - private String tableName = "test_table"; + private final String tableName = "test_table"; @Parameter(index = 2) private FileFormat format; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java index 4ad302dde436..a7c58e551112 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java @@ -23,7 +23,7 @@ import org.apache.iceberg.TestTables; public class TestTableLoader implements TableLoader { - private File dir; + private final File dir; public static TableLoader of(String dir) { return new TestTableLoader(dir); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 7033fd30e84f..187f2b0b9e31 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -122,7 +122,7 @@ public void testTimeUnit() throws IOException { new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); assertThat(extractor.extractWatermark(split(0))) - .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue() / 1000L); + .isEqualTo(MIN_VALUES.get(0).get(columnName) / 1000L); } @TestTemplate diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java index 63d24f7da553..d8f9301a7d82 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java @@ -29,7 +29,7 @@ class RandomGeneratingUDF implements Serializable { private final long uniqueValues; - private Random rand = new Random(); + private final Random rand = new Random(); RandomGeneratingUDF(long uniqueValues) { this.uniqueValues = uniqueValues; @@ -43,8 +43,7 @@ UserDefinedFunction randomLongUDF() { UserDefinedFunction randomString() { return udf( - () -> (String) RandomUtil.generatePrimitive(Types.StringType.get(), rand), - DataTypes.StringType) + () -> RandomUtil.generatePrimitive(Types.StringType.get(), rand), DataTypes.StringType) .asNondeterministic() .asNonNullable(); } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java index cdc0bf5f3cad..a4bcd0d8a5b5 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java @@ -199,7 +199,7 @@ public Type map(Types.MapType map, Supplier keyResult, Supplier valu "Cannot project a map of optional values as required values: %s", map); Preconditions.checkArgument( - StringType.class.isInstance(requestedMap.keyType()), + requestedMap.keyType() instanceof StringType, "Invalid map key type (not string): %s", requestedMap.keyType()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java index 10b856a7acf0..f5d9a7af1c8f 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -885,7 +885,7 @@ public static List getPartitions( JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(rootPath)) .asScala() .toSeq(), - scala.collection.immutable.Map$.MODULE$.empty(), + scala.collection.immutable.Map$.MODULE$.empty(), userSpecifiedSchema, fileStatusCache, Option.empty(), 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 47690656b41d..af1e99df71d3 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 @@ -182,7 +182,7 @@ public static List getPartitions( Option> scalaPartitionFilter; if (partitionFilter != null && !partitionFilter.isEmpty()) { Builder, scala.collection.immutable.Map> builder = - Map$.MODULE$.newBuilder(); + Map$.MODULE$.newBuilder(); partitionFilter.forEach((key, value) -> builder.$plus$eq(Tuple2.apply(key, value))); scalaPartitionFilter = Option.apply(builder.result()); } else { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index b00ed42008f1..5fbb4117feb8 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -116,7 +116,7 @@ public class DeleteOrphanFilesSparkAction extends BaseSparkAction equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT); private Map equalAuthorities = Collections.emptyMap(); private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR; - private String location = null; + private String location; private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); private Dataset compareToFileList; private Consumer deleteFunc = null; 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 ae547e206324..eed0b2b67b0a 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 @@ -367,7 +367,7 @@ private Result doExecuteWithPartialProgress( Stream toGroupStream( RewriteExecutionContext ctx, Map>> groupsByPartition) { return groupsByPartition.entrySet().stream() - .filter(e -> e.getValue().size() != 0) + .filter(e -> !e.getValue().isEmpty()) .flatMap( e -> { StructLike partition = e.getKey(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 78eb010b9b9c..62b7978f3800 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -93,9 +93,9 @@ public class RewriteManifestsSparkAction private final long targetManifestSizeBytes; private final boolean shouldStageManifests; - private PartitionSpec spec = null; + private PartitionSpec spec; private Predicate predicate = manifest -> true; - private String outputLocation = null; + private String outputLocation; RewriteManifestsSparkAction(SparkSession spark, Table table) { super(spark); 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 ea1c52940175..8bc67c35c3d6 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 @@ -309,7 +309,7 @@ private Result doExecuteWithPartialProgress( // stop commit service commitService.close(); List commitResults = commitService.results(); - if (commitResults.size() == 0) { + if (commitResults.isEmpty()) { LOG.error( "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " @@ -331,7 +331,7 @@ private Stream toGroupStream( RewriteExecutionContext ctx, Map>> groupsByPartition) { return groupsByPartition.entrySet().stream() - .filter(e -> e.getValue().size() != 0) + .filter(e -> !e.getValue().isEmpty()) .flatMap( e -> { StructLike partition = e.getKey(); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index 1a4f7052de39..e306bab32bcd 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -561,7 +561,7 @@ private static class InternalRowWriter extends ParquetValueWriters.StructWriter< private InternalRowWriter(List> writers, List types) { super(writers); - this.types = types.toArray(new DataType[types.size()]); + this.types = types.toArray(new DataType[0]); } @Override 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 c24bcaad58b1..ef3138d677c6 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 @@ -93,7 +93,7 @@ public class SparkScanBuilder private final SparkReadConf readConf; private final List metaColumns = Lists.newArrayList(); - private Schema schema = null; + private Schema schema; private boolean caseSensitive; private List filterExpressions = null; private Filter[] pushedFilters = NO_FILTERS; diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 25393888f95c..c5c86c3ebf28 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -44,7 +44,7 @@ class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredCol private final SparkReadConf readConf; private final List metaColumns = Lists.newArrayList(); - private Schema schema = null; + private Schema schema; SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { this.spark = spark; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index c96ed2909fe7..028d495b894d 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -138,7 +138,7 @@ public static Object[][] parameters() { @Rule public TemporaryFolder temp = new TemporaryFolder(); - private String baseTableName = "baseTable"; + private final String baseTableName = "baseTable"; private File tableDir; private String tableLocation; private final String type; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java index 478afcf09ae3..360b9ff20ec0 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java @@ -67,7 +67,7 @@ public static List generateList(Schema schema, int numRecords, long seed public static Iterable generateSpark(Schema schema, int numRecords, long seed) { return () -> new Iterator() { - private SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); + private final SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); private int count = 0; @Override @@ -114,7 +114,7 @@ private static Iterable newIterable( return () -> new Iterator() { private int count = 0; - private RandomDataGenerator generator = newGenerator.get(); + private final RandomDataGenerator generator = newGenerator.get(); @Override public boolean hasNext() { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java index 53a35eec61ce..2627ec0d2b55 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger; public class LogMessage { - private static AtomicInteger idCounter = new AtomicInteger(0); + private static final AtomicInteger idCounter = new AtomicInteger(0); static LogMessage debug(String date, String message) { return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message); 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 3d94966eb76c..e5e0e350e50b 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 @@ -87,7 +87,7 @@ public Integer next() { // Main reader class to test base class iteration logic. // Keeps track of iterator closure. private static class ClosureTrackingReader extends BaseReader { - private Map tracker = Maps.newHashMap(); + private final Map tracker = Maps.newHashMap(); ClosureTrackingReader(Table table, List tasks) { super(table, new BaseCombinedScanTask(tasks), null, null, false); 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 a5a3da39ad1f..693bcec9eb05 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 @@ -92,7 +92,7 @@ public TestDataFrameWrites(String format) { private Map tableProperties; - private org.apache.spark.sql.types.StructType sparkSchema = + 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( @@ -107,16 +107,16 @@ public TestDataFrameWrites(String format) { org.apache.spark.sql.types.Metadata.empty()) }); - private Schema icebergSchema = + private final Schema icebergSchema = new Schema( Types.NestedField.optional(1, "optionalField", Types.StringType.get()), Types.NestedField.required(2, "requiredField", Types.StringType.get())); - private List data0 = + private final List data0 = Arrays.asList( "{\"optionalField\": \"a1\", \"requiredField\": \"bid_001\"}", "{\"optionalField\": \"a2\", \"requiredField\": \"bid_002\"}"); - private List data1 = + private final List data1 = Arrays.asList( "{\"optionalField\": \"d1\", \"requiredField\": \"bid_101\"}", "{\"optionalField\": \"d2\", \"requiredField\": \"bid_102\"}", @@ -221,16 +221,15 @@ private void writeDataWithFailOnPartition( final int numPartitions = 10; final int partitionToFail = new Random().nextInt(numPartitions); MapPartitionsFunction failOnFirstPartitionFunc = - (MapPartitionsFunction) - input -> { - int partitionId = TaskContext.getPartitionId(); + input -> { + int partitionId = TaskContext.getPartitionId(); - if (partitionId == partitionToFail) { - throw new SparkException( - String.format("Intended exception in partition %d !", partitionId)); - } - return input; - }; + if (partitionId == partitionToFail) { + throw new SparkException( + String.format("Intended exception in partition %d !", partitionId)); + } + return input; + }; Dataset df = createDataset(records, schema) @@ -286,8 +285,8 @@ public void testNullableWithWriteOption() throws IOException { "Spark 3 rejects writing nulls to a required column", spark.version().startsWith("2")); File location = new File(temp.newFolder("parquet"), "test"); - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString()); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString()); + 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); @@ -339,8 +338,8 @@ public void testNullableWithSparkSqlOption() throws IOException { "Spark 3 rejects writing nulls to a required column", spark.version().startsWith("2")); File location = new File(temp.newFolder("parquet"), "test"); - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString()); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString()); + 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); diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java index 7313c18cc09d..6508b80e9c92 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java @@ -92,7 +92,7 @@ public TestIdentityPartitionData(String format, boolean vectorized) { @Rule public TemporaryFolder temp = new TemporaryFolder(); - private PartitionSpec spec = + private final PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build(); private Table table = null; private Dataset logs = null; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java index 4ef022c50c59..86d709cf17ba 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java @@ -163,7 +163,7 @@ private static Instant getInstant(String timestampWithoutZone) { @Rule public TemporaryFolder temp = new TemporaryFolder(); - private PartitionSpec spec = + private final PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA) .identity("date") .identity("level") diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index bd20a628caa8..f420f1b955c0 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -116,7 +116,7 @@ public void testStreamingWriteAppendMode() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); // restart the query from the checkpoint @@ -176,7 +176,7 @@ public void testStreamingWriteCompleteMode() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); // restart the query from the checkpoint @@ -236,7 +236,7 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); // restart the query from the checkpoint diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index bf2200004282..36e887f49b55 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -278,7 +278,7 @@ public void testCreateTableLocation() throws Exception { File tableLocation = temp.newFolder(); Assert.assertTrue(tableLocation.delete()); - String location = "file:" + tableLocation.toString(); + String location = "file:" + tableLocation; sql( "CREATE TABLE %s " diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java index 5a1cc6343424..5b5dc1501335 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/SparkRowLevelOperationsTestBase.java @@ -262,7 +262,7 @@ protected void createOrReplaceView(String name, List data, Encoder enc private Dataset toDS(String schema, String jsonData) { List jsonRows = Arrays.stream(jsonData.split("\n")) - .filter(str -> str.trim().length() > 0) + .filter(str -> !str.trim().isEmpty()) .collect(Collectors.toList()); Dataset jsonDS = spark.createDataset(jsonRows, Encoders.STRING()); diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java index 63d24f7da553..d8f9301a7d82 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/RandomGeneratingUDF.java @@ -29,7 +29,7 @@ class RandomGeneratingUDF implements Serializable { private final long uniqueValues; - private Random rand = new Random(); + private final Random rand = new Random(); RandomGeneratingUDF(long uniqueValues) { this.uniqueValues = uniqueValues; @@ -43,8 +43,7 @@ UserDefinedFunction randomLongUDF() { UserDefinedFunction randomString() { return udf( - () -> (String) RandomUtil.generatePrimitive(Types.StringType.get(), rand), - DataTypes.StringType) + () -> RandomUtil.generatePrimitive(Types.StringType.get(), rand), DataTypes.StringType) .asNondeterministic() .asNonNullable(); } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java index bbbb46e803e2..f76f12355f1f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/PruneColumnsWithReordering.java @@ -203,7 +203,7 @@ public Type map(Types.MapType map, Supplier keyResult, Supplier valu "Cannot project a map of optional values as required values: %s", map); Preconditions.checkArgument( - StringType.class.isInstance(requestedMap.keyType()), + requestedMap.keyType() instanceof StringType, "Invalid map key type (not string): %s", requestedMap.keyType()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java index a81877c98b3d..cdf250b61b8f 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/Spark3Util.java @@ -888,7 +888,7 @@ public static List getPartitions( JavaConverters.collectionAsScalaIterableConverter(ImmutableList.of(rootPath)) .asScala() .toSeq(), - scala.collection.immutable.Map$.MODULE$.empty(), + scala.collection.immutable.Map$.MODULE$.empty(), userSpecifiedSchema, fileStatusCache, Option.empty(), 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 5af6b46b3178..6f57c7ae376c 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 @@ -182,7 +182,7 @@ public static List getPartitions( Option> scalaPartitionFilter; if (partitionFilter != null && !partitionFilter.isEmpty()) { Builder, scala.collection.immutable.Map> builder = - Map$.MODULE$.newBuilder(); + Map$.MODULE$.newBuilder(); partitionFilter.forEach((key, value) -> builder.$plus$eq(Tuple2.apply(key, value))); scalaPartitionFilter = Option.apply(builder.result()); } else { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java index b00ed42008f1..5fbb4117feb8 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/DeleteOrphanFilesSparkAction.java @@ -116,7 +116,7 @@ public class DeleteOrphanFilesSparkAction extends BaseSparkAction equalSchemes = flattenMap(EQUAL_SCHEMES_DEFAULT); private Map equalAuthorities = Collections.emptyMap(); private PrefixMismatchMode prefixMismatchMode = PrefixMismatchMode.ERROR; - private String location = null; + private String location; private long olderThanTimestamp = System.currentTimeMillis() - TimeUnit.DAYS.toMillis(3); private Dataset compareToFileList; private Consumer deleteFunc = null; 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 7c516b96754a..a4c6642a3edf 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 @@ -379,7 +379,7 @@ private Result doExecuteWithPartialProgress( Stream toGroupStream( RewriteExecutionContext ctx, Map>> groupsByPartition) { return groupsByPartition.entrySet().stream() - .filter(e -> e.getValue().size() != 0) + .filter(e -> !e.getValue().isEmpty()) .flatMap( e -> { StructLike partition = e.getKey(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index e9edfeb985c7..60e2b11881cb 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -103,9 +103,9 @@ public class RewriteManifestsSparkAction private final long targetManifestSizeBytes; private final boolean shouldStageManifests; - private PartitionSpec spec = null; + private PartitionSpec spec; private Predicate predicate = manifest -> true; - private String outputLocation = null; + private String outputLocation; RewriteManifestsSparkAction(SparkSession spark, Table table) { super(spark); 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 bdb0ee35273f..ccf874716db0 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 @@ -310,7 +310,7 @@ private Result doExecuteWithPartialProgress( // stop commit service commitService.close(); List commitResults = commitService.results(); - if (commitResults.size() == 0) { + if (commitResults.isEmpty()) { LOG.error( "{} is true but no rewrite commits succeeded. Check the logs to determine why the individual " + "commits failed. If this is persistent it may help to increase {} which will break the rewrite operation " @@ -332,7 +332,7 @@ private Stream toGroupStream( RewriteExecutionContext ctx, Map>> groupsByPartition) { return groupsByPartition.entrySet().stream() - .filter(e -> e.getValue().size() != 0) + .filter(e -> !e.getValue().isEmpty()) .flatMap( e -> { StructLike partition = e.getKey(); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java index 1a4f7052de39..e306bab32bcd 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/SparkParquetWriters.java @@ -561,7 +561,7 @@ private static class InternalRowWriter extends ParquetValueWriters.StructWriter< private InternalRowWriter(List> writers, List types) { super(writers); - this.types = types.toArray(new DataType[types.size()]); + this.types = types.toArray(new DataType[0]); } @Override diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java index 9b075b675565..50a1259c8626 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkFileWriterFactory.java @@ -48,7 +48,7 @@ class SparkFileWriterFactory extends BaseFileWriterFactory { private StructType dataSparkType; private StructType equalityDeleteSparkType; private StructType positionDeleteSparkType; - private Map writeProperties; + private final Map writeProperties; SparkFileWriterFactory( Table table, 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 b430e6fca233..6b97e48133fd 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 @@ -95,7 +95,7 @@ public class SparkScanBuilder private final List metaColumns = Lists.newArrayList(); private final InMemoryMetricsReporter metricsReporter; - private Schema schema = null; + private Schema schema; private boolean caseSensitive; private List filterExpressions = null; private Predicate[] pushedPredicates = NO_PREDICATES; diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java index 25393888f95c..c5c86c3ebf28 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/source/SparkStagedScanBuilder.java @@ -44,7 +44,7 @@ class SparkStagedScanBuilder implements ScanBuilder, SupportsPushDownRequiredCol private final SparkReadConf readConf; private final List metaColumns = Lists.newArrayList(); - private Schema schema = null; + private Schema schema; SparkStagedScanBuilder(SparkSession spark, Table table, CaseInsensitiveStringMap options) { this.spark = spark; diff --git a/spark/v3.4/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java b/spark/v3.4/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java index 2a89ac73e2c6..0cae0451905b 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java +++ b/spark/v3.4/spark/src/main/java/org/apache/spark/sql/catalyst/analysis/NoSuchProcedureException.java @@ -33,7 +33,7 @@ public NoSuchProcedureException(Identifier ident) { Option.empty(), Option.empty(), Option.empty(), - Map$.MODULE$.empty(), + Map$.MODULE$.empty(), new QueryContext[0]); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index c96ed2909fe7..028d495b894d 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -138,7 +138,7 @@ public static Object[][] parameters() { @Rule public TemporaryFolder temp = new TemporaryFolder(); - private String baseTableName = "baseTable"; + private final String baseTableName = "baseTable"; private File tableDir; private String tableLocation; private final String type; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java index 478afcf09ae3..360b9ff20ec0 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/data/RandomData.java @@ -67,7 +67,7 @@ public static List generateList(Schema schema, int numRecords, long seed public static Iterable generateSpark(Schema schema, int numRecords, long seed) { return () -> new Iterator() { - private SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); + private final SparkRandomDataGenerator generator = new SparkRandomDataGenerator(seed); private int count = 0; @Override @@ -114,7 +114,7 @@ private static Iterable newIterable( return () -> new Iterator() { private int count = 0; - private RandomDataGenerator generator = newGenerator.get(); + private final RandomDataGenerator generator = newGenerator.get(); @Override public boolean hasNext() { diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java index 53a35eec61ce..2627ec0d2b55 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -22,7 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger; public class LogMessage { - private static AtomicInteger idCounter = new AtomicInteger(0); + private static final AtomicInteger idCounter = new AtomicInteger(0); static LogMessage debug(String date, String message) { return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message); 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 3d94966eb76c..e5e0e350e50b 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 @@ -87,7 +87,7 @@ public Integer next() { // Main reader class to test base class iteration logic. // Keeps track of iterator closure. private static class ClosureTrackingReader extends BaseReader { - private Map tracker = Maps.newHashMap(); + private final Map tracker = Maps.newHashMap(); ClosureTrackingReader(Table table, List tasks) { super(table, new BaseCombinedScanTask(tasks), null, null, false); 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 ea329f96d5b9..6b188cd1b120 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 @@ -215,7 +215,7 @@ private String getCompressionType(InputFile inputFile) throws Exception { return footer.getBlocks().get(0).getColumns().get(0).getCodec().name(); default: FileContext fc = FileContext.getFileContext(CONF); - GenericDatumReader reader = new GenericDatumReader(); + GenericDatumReader reader = new GenericDatumReader<>(); DataFileReader fileReader = (DataFileReader) DataFileReader.openReader( 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 a5a3da39ad1f..693bcec9eb05 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 @@ -92,7 +92,7 @@ public TestDataFrameWrites(String format) { private Map tableProperties; - private org.apache.spark.sql.types.StructType sparkSchema = + 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( @@ -107,16 +107,16 @@ public TestDataFrameWrites(String format) { org.apache.spark.sql.types.Metadata.empty()) }); - private Schema icebergSchema = + private final Schema icebergSchema = new Schema( Types.NestedField.optional(1, "optionalField", Types.StringType.get()), Types.NestedField.required(2, "requiredField", Types.StringType.get())); - private List data0 = + private final List data0 = Arrays.asList( "{\"optionalField\": \"a1\", \"requiredField\": \"bid_001\"}", "{\"optionalField\": \"a2\", \"requiredField\": \"bid_002\"}"); - private List data1 = + private final List data1 = Arrays.asList( "{\"optionalField\": \"d1\", \"requiredField\": \"bid_101\"}", "{\"optionalField\": \"d2\", \"requiredField\": \"bid_102\"}", @@ -221,16 +221,15 @@ private void writeDataWithFailOnPartition( final int numPartitions = 10; final int partitionToFail = new Random().nextInt(numPartitions); MapPartitionsFunction failOnFirstPartitionFunc = - (MapPartitionsFunction) - input -> { - int partitionId = TaskContext.getPartitionId(); + input -> { + int partitionId = TaskContext.getPartitionId(); - if (partitionId == partitionToFail) { - throw new SparkException( - String.format("Intended exception in partition %d !", partitionId)); - } - return input; - }; + if (partitionId == partitionToFail) { + throw new SparkException( + String.format("Intended exception in partition %d !", partitionId)); + } + return input; + }; Dataset df = createDataset(records, schema) @@ -286,8 +285,8 @@ public void testNullableWithWriteOption() throws IOException { "Spark 3 rejects writing nulls to a required column", spark.version().startsWith("2")); File location = new File(temp.newFolder("parquet"), "test"); - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString()); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString()); + 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); @@ -339,8 +338,8 @@ public void testNullableWithSparkSqlOption() throws IOException { "Spark 3 rejects writing nulls to a required column", spark.version().startsWith("2")); File location = new File(temp.newFolder("parquet"), "test"); - String sourcePath = String.format("%s/nullable_poc/sourceFolder/", location.toString()); - String targetPath = String.format("%s/nullable_poc/targetFolder/", location.toString()); + 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); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java index 45a523917f05..4ee77345dbe5 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestIdentityPartitionData.java @@ -102,7 +102,7 @@ public TestIdentityPartitionData(String format, boolean vectorized, PlanningMode @Rule public TemporaryFolder temp = new TemporaryFolder(); - private PartitionSpec spec = + private final PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA).identity("date").identity("level").build(); private Table table = null; private Dataset logs = null; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java index c00549c68f3b..639d37c79336 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestPartitionPruning.java @@ -169,7 +169,7 @@ private static Instant getInstant(String timestampWithoutZone) { @Rule public TemporaryFolder temp = new TemporaryFolder(); - private PartitionSpec spec = + private final PartitionSpec spec = PartitionSpec.builderFor(LOG_SCHEMA) .identity("date") .identity("level") diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java index bd20a628caa8..f420f1b955c0 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestStructuredStreaming.java @@ -116,7 +116,7 @@ public void testStreamingWriteAppendMode() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); // restart the query from the checkpoint @@ -176,7 +176,7 @@ public void testStreamingWriteCompleteMode() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); // restart the query from the checkpoint @@ -236,7 +236,7 @@ public void testStreamingWriteCompleteModeWithProjection() throws Exception { query.stop(); // remove the last commit to force Spark to reprocess batch #1 - File lastCommitFile = new File(checkpoint.toString() + "/commits/1"); + File lastCommitFile = new File(checkpoint + "/commits/1"); Assert.assertTrue("The commit file must be deleted", lastCommitFile.delete()); // restart the query from the checkpoint diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java index 4e34d662ffd1..927f6e21439f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestCreateTable.java @@ -277,7 +277,7 @@ public void testCreateTableLocation() throws Exception { File tableLocation = temp.newFolder(); Assert.assertTrue(tableLocation.delete()); - String location = "file:" + tableLocation.toString(); + String location = "file:" + tableLocation; sql( "CREATE TABLE %s " From d1fbd3e9d5c1dc045b9efc1f7795c8ba51872cee Mon Sep 17 00:00:00 2001 From: Alexandre Dutra Date: Thu, 4 Jul 2024 14:06:57 +0200 Subject: [PATCH 0433/1019] Core: Assume issued_token_type is access_token to fully comply with RFC 6749 (#10314) The REST client assumes that the `issued_token_type` field is present in all OAuth responses due to adhering to RFC 8693 but currently not every authentication server fully supports RFC 8693. This changesset ensures that the OAuth response adheres to RFC 6749 --- .../java/org/apache/iceberg/rest/auth/OAuth2Util.java | 10 +++++++++- .../org/apache/iceberg/rest/RESTCatalogAdapter.java | 1 - 2 files changed, 9 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 76386027c629..d2062ea7df97 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -738,13 +738,21 @@ private static AuthSession fromTokenResponse( long startTimeMillis, AuthSession parent, String credential) { + // issued_token_type is required in RFC 8693 but not in RFC 6749, + // thus assume type is access_token for compatibility with RFC 6749. + // See https://datatracker.ietf.org/doc/html/rfc6749#section-4.4.3 + // for an example of a response that does not include the issued token type. + String issuedTokenType = response.issuedTokenType(); + if (issuedTokenType == null) { + issuedTokenType = OAuth2Properties.ACCESS_TOKEN_TYPE; + } AuthSession session = new AuthSession( parent.headers(), AuthConfig.builder() .from(parent.config()) .token(response.token()) - .tokenType(response.issuedTokenType()) + .tokenType(issuedTokenType) .credential(credential) .build()); 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 357b05e85c20..53d57bee510f 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -254,7 +254,6 @@ private static OAuthTokenResponse handleOAuthRequest(Object body) { case "client_credentials": return OAuthTokenResponse.builder() .withToken("client-credentials-token:sub=" + request.get("client_id")) - .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") .withTokenType("Bearer") .build(); From 11df2a36652e790efb394715becb61d068579c13 Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Thu, 4 Jul 2024 18:02:47 +0200 Subject: [PATCH 0434/1019] Flink: Fix `long` casting issues (#10629) --- .../org/apache/iceberg/flink/data/FlinkParquetReaders.java | 6 +++--- .../source/assigner/TestWatermarkBasedSplitAssigner.java | 2 +- .../org/apache/iceberg/flink/data/FlinkParquetReaders.java | 6 +++--- .../source/assigner/TestWatermarkBasedSplitAssigner.java | 2 +- .../org/apache/iceberg/flink/data/FlinkParquetReaders.java | 6 +++--- 5 files changed, 11 insertions(+), 11 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index ad4310a6d195..a5f2bb738960 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -422,7 +422,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromLocalDateTime( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L) .atOffset(ZoneOffset.UTC) .toLocalDateTime()); } @@ -444,7 +444,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromInstant( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L)); } @Override @@ -517,7 +517,7 @@ private static class LossyMicrosToMillisTimeReader @Override public Integer read(Integer reuse) { // Discard microseconds since Flink uses millisecond unit for TIME type. - return (int) Math.floorDiv(column.nextLong(), 1000); + return (int) Math.floorDiv(column.nextLong(), 1000L); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java index e1fc63fda918..6c0cbb733320 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -123,7 +123,7 @@ protected List createSplits( .mapToObj( fileNum -> RandomGenericData.generate( - SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + SCHEMA, 2, (long) splitNum * filesPerSplit + fileNum)) .collect(Collectors.toList()))) .collect(Collectors.toList()); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index ad4310a6d195..a5f2bb738960 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -422,7 +422,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromLocalDateTime( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000) + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L) .atOffset(ZoneOffset.UTC) .toLocalDateTime()); } @@ -444,7 +444,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromInstant( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000)); + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L)); } @Override @@ -517,7 +517,7 @@ private static class LossyMicrosToMillisTimeReader @Override public Integer read(Integer reuse) { // Discard microseconds since Flink uses millisecond unit for TIME type. - return (int) Math.floorDiv(column.nextLong(), 1000); + return (int) Math.floorDiv(column.nextLong(), 1000L); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java index e1fc63fda918..6c0cbb733320 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -123,7 +123,7 @@ protected List createSplits( .mapToObj( fileNum -> RandomGenericData.generate( - SCHEMA, 2, splitNum * filesPerSplit + fileNum)) + SCHEMA, 2, (long) splitNum * filesPerSplit + fileNum)) .collect(Collectors.toList()))) .collect(Collectors.toList()); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java index d0a1974636be..a5f2bb738960 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -422,7 +422,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromLocalDateTime( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000L) + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L) .atOffset(ZoneOffset.UTC) .toLocalDateTime()); } @@ -444,7 +444,7 @@ public TimestampData read(TimestampData ignored) { long value = readLong(); return TimestampData.fromInstant( Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000), Math.floorMod(value, 1000_000) * 1000L)); + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L)); } @Override @@ -517,7 +517,7 @@ private static class LossyMicrosToMillisTimeReader @Override public Integer read(Integer reuse) { // Discard microseconds since Flink uses millisecond unit for TIME type. - return (int) Math.floorDiv(column.nextLong(), 1000); + return (int) Math.floorDiv(column.nextLong(), 1000L); } } From 36743023ae66e1975ce7581e4c8d5208cb0c9506 Mon Sep 17 00:00:00 2001 From: Tai Le Manh <49281946+tlm365@users.noreply.github.com> Date: Fri, 5 Jul 2024 00:34:22 +0700 Subject: [PATCH 0435/1019] Core: Handle possible heap data corruption in OAuth2Util.AuthSession#headers (#10615) Signed-off-by: Tai Le Manh --- .../src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index d2062ea7df97..189e5fde2cad 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -578,7 +578,8 @@ public Pair refresh(RESTClient client) { .token(response.token()) .tokenType(response.issuedTokenType()) .build(); - this.headers = RESTUtil.merge(headers, authHeaders(config.token())); + Map currentHeaders = this.headers; + this.headers = RESTUtil.merge(currentHeaders, authHeaders(config.token())); if (response.expiresInSeconds() != null) { return Pair.of(response.expiresInSeconds(), TimeUnit.SECONDS); From f4c64ad49169cb52b58f9022273ca086dbc95aa3 Mon Sep 17 00:00:00 2001 From: Sotaro Hikita <70102274+lawofcycles@users.noreply.github.com> Date: Fri, 5 Jul 2024 15:46:24 +0900 Subject: [PATCH 0436/1019] AWS: Retain Glue Catalog column comment (#10276) * Retain Glue Catalog column comment * merge handling existing column comment to creating column set * apply spotless * Add newline after else block for improved readability --- .../apache/iceberg/aws/glue/GlueTestBase.java | 36 +++++ .../aws/glue/TestGlueCatalogTable.java | 129 ++++++++++++++++++ .../iceberg/aws/glue/GlueTableOperations.java | 6 +- .../aws/glue/IcebergToGlueConverter.java | 76 +++++++++-- .../aws/glue/TestIcebergToGlueConverter.java | 78 +++++++++++ 5 files changed, 310 insertions(+), 15 deletions(-) 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 ed3a235eb0c8..aa0c7f18319c 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 @@ -21,6 +21,8 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.function.Function; +import java.util.stream.Collectors; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.TableProperties; @@ -39,6 +41,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.GetTableRequest; import software.amazon.awssdk.services.glue.model.GetTableResponse; import software.amazon.awssdk.services.glue.model.Table; @@ -158,4 +161,37 @@ public static void updateTableDescription( .build(); glue.updateTable(request); } + + public static void updateTableColumns( + String namespace, String tableName, Function columnUpdater) { + GetTableResponse response = + glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + Table existingTable = response.table(); + List updatedColumns = + existingTable.storageDescriptor().columns().stream() + .map(columnUpdater) + .collect(Collectors.toList()); + + UpdateTableRequest request = + UpdateTableRequest.builder() + .catalogId(existingTable.catalogId()) + .databaseName(existingTable.databaseName()) + .tableInput( + TableInput.builder() + .description(existingTable.description()) + .name(existingTable.name()) + .partitionKeys(existingTable.partitionKeys()) + .tableType(existingTable.tableType()) + .owner(existingTable.owner()) + .parameters(existingTable.parameters()) + .storageDescriptor( + existingTable + .storageDescriptor() + .toBuilder() + .columns(updatedColumns) + .build()) + .build()) + .build(); + glue.updateTable(request); + } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 6dffdb5b9253..9c4d1839a4eb 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -221,6 +221,69 @@ public void testUpdateTable() { assertThat(response.table().description()).isEqualTo(updatedComment); } + @Test + public void testDropColumn() { + String namespace = createNamespace(); + String tableName = createTable(namespace); + Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); + table + .updateSchema() + .addColumn("c2", Types.StringType.get(), "updated from Iceberg API") + .addColumn("c3", Types.StringType.get()) + .commit(); + + updateTableColumns( + namespace, + tableName, + column -> { + if (column.name().equals("c3")) { + return column.toBuilder().comment("updated from Glue API").build(); + } else { + return column; + } + }); + + table.updateSchema().deleteColumn("c2").deleteColumn("c3").commit(); + + GetTableResponse response = + glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + List actualColumns = response.table().storageDescriptor().columns(); + + List expectedColumns = + ImmutableList.of( + Column.builder() + .name("c1") + .type("string") + .comment("c1") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "1", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "false", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build(), + Column.builder() + .name("c2") + .type("string") + .comment("updated from Iceberg API") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "2", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "true", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "false")) + .build(), + Column.builder() + .name("c3") + .type("string") + .comment("updated from Glue API") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "3", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "true", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "false")) + .build()); + assertThat(actualColumns).isEqualTo(expectedColumns); + } + @Test public void testRenameTable() { String namespace = createNamespace(); @@ -514,6 +577,72 @@ public void testColumnCommentsAndParameters() { assertThat(actualColumns).isEqualTo(expectedColumns); } + @Test + public void testGlueTableColumnCommentsPreserved() { + String namespace = createNamespace(); + String tableName = createTable(namespace); + Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); + table + .updateSchema() + .addColumn("c2", Types.StringType.get()) + .addColumn("c3", Types.StringType.get()) + .commit(); + + updateTableColumns( + namespace, + tableName, + column -> { + if (column.name().equals("c2") || column.name().equals("c3")) { + return column.toBuilder().comment("updated from Glue API").build(); + } else { + return column; + } + }); + + table + .updateSchema() + .updateColumn("c2", Types.StringType.get(), "updated from Iceberg API") + .commit(); + + GetTableResponse response = + glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + List actualColumns = response.table().storageDescriptor().columns(); + + List expectedColumns = + ImmutableList.of( + Column.builder() + .name("c1") + .type("string") + .comment("c1") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "1", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "false", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build(), + Column.builder() + .name("c2") + .type("string") + .comment("updated from Iceberg API") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "2", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "true", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build(), + Column.builder() + .name("c3") + .type("string") + .comment("updated from Glue API") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "3", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "true", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build()); + assertThat(actualColumns).isEqualTo(expectedColumns); + } + @Test public void testTablePropsDefinedAtCatalogLevel() { String namespace = createNamespace(); diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java index aedf78523485..4c63dfdb2a70 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueTableOperations.java @@ -316,12 +316,10 @@ void persistGlueTable( .skipArchive(awsProperties.glueCatalogSkipArchive()) .tableInput( TableInput.builder() - // Call description before applyMutation so that applyMutation overwrites the - // description with the comment specified in the query - .description(glueTable.description()) .applyMutation( builder -> - IcebergToGlueConverter.setTableInputInformation(builder, metadata)) + IcebergToGlueConverter.setTableInputInformation( + builder, metadata, glueTable)) .name(tableName) .tableType(GLUE_EXTERNAL_TABLE_TYPE) .parameters(parameters) diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/IcebergToGlueConverter.java b/aws/src/main/java/org/apache/iceberg/aws/glue/IcebergToGlueConverter.java index 2c7ed1fe6457..56b38a47e968 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/IcebergToGlueConverter.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/IcebergToGlueConverter.java @@ -19,6 +19,7 @@ package org.apache.iceberg.aws.glue; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.Locale; import java.util.Map; @@ -49,6 +50,7 @@ import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.DatabaseInput; import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.Table; import software.amazon.awssdk.services.glue.model.TableInput; class IcebergToGlueConverter { @@ -219,6 +221,29 @@ static String getTableName(TableIdentifier tableIdentifier, boolean skipNameVali */ static void setTableInputInformation( TableInput.Builder tableInputBuilder, TableMetadata metadata) { + setTableInputInformation(tableInputBuilder, metadata, null); + } + + /** + * Set Glue table input information based on Iceberg table metadata, optionally preserving + * comments from an existing Glue table's columns. + * + *

    A best-effort conversion of Iceberg metadata to Glue table is performed to display Iceberg + * information in Glue, but such information is only intended for informational human read access + * through tools like UI or CLI, and should never be used by any query processing engine to infer + * information like schema, partition spec, etc. The source of truth is stored in the actual + * Iceberg metadata file defined by the metadata_location table property. + * + *

    If an existing Glue table is provided, the comments from its columns will be preserved in + * the resulting Glue TableInput. This is useful when updating an existing Glue table to retain + * any user-defined comments on the columns. + * + * @param tableInputBuilder Glue TableInput builder + * @param metadata Iceberg table metadata + * @param existingTable optional existing Glue table, used to preserve column comments + */ + static void setTableInputInformation( + TableInput.Builder tableInputBuilder, TableMetadata metadata, Table existingTable) { try { Map properties = metadata.properties(); StorageDescriptor.Builder storageDescriptor = StorageDescriptor.builder(); @@ -231,11 +256,28 @@ static void setTableInputInformation( .collect(Collectors.toSet())); } - Optional.ofNullable(properties.get(GLUE_DESCRIPTION_KEY)) - .ifPresent(tableInputBuilder::description); + String description = properties.get(GLUE_DESCRIPTION_KEY); + if (description != null) { + tableInputBuilder.description(description); + } else if (existingTable != null) { + Optional.ofNullable(existingTable.description()).ifPresent(tableInputBuilder::description); + } + + Map existingColumnMap = null; + if (existingTable != null) { + List existingColumns = existingTable.storageDescriptor().columns(); + existingColumnMap = + existingColumns.stream() + .filter(column -> column.comment() != null) + .collect(Collectors.toMap(Column::name, Column::comment)); + } else { + existingColumnMap = Collections.emptyMap(); + } + + List columns = toColumns(metadata, existingColumnMap); tableInputBuilder.storageDescriptor( - storageDescriptor.location(metadata.location()).columns(toColumns(metadata)).build()); + storageDescriptor.location(metadata.location()).columns(columns).build()); } catch (RuntimeException e) { LOG.warn( "Encountered unexpected exception while converting Iceberg metadata to Glue table information", @@ -297,18 +339,20 @@ private static String toTypeString(Type type) { } } - private static List toColumns(TableMetadata metadata) { + private static List toColumns( + TableMetadata metadata, Map existingColumnMap) { List columns = Lists.newArrayList(); Set addedNames = Sets.newHashSet(); for (NestedField field : metadata.schema().columns()) { - addColumnWithDedupe(columns, addedNames, field, true /* is current */); + addColumnWithDedupe(columns, addedNames, field, true /* is current */, existingColumnMap); } for (Schema schema : metadata.schemas()) { if (schema.schemaId() != metadata.currentSchemaId()) { for (NestedField field : schema.columns()) { - addColumnWithDedupe(columns, addedNames, field, false /* is not current */); + addColumnWithDedupe( + columns, addedNames, field, false /* is not current */, existingColumnMap); } } } @@ -317,19 +361,29 @@ private static List toColumns(TableMetadata metadata) { } private static void addColumnWithDedupe( - List columns, Set dedupe, NestedField field, boolean isCurrent) { + List columns, + Set dedupe, + NestedField field, + boolean isCurrent, + Map existingColumnMap) { if (!dedupe.contains(field.name())) { - columns.add( + Column.Builder builder = Column.builder() .name(field.name()) .type(toTypeString(field.type())) - .comment(field.doc()) .parameters( ImmutableMap.of( ICEBERG_FIELD_ID, Integer.toString(field.fieldId()), ICEBERG_FIELD_OPTIONAL, Boolean.toString(field.isOptional()), - ICEBERG_FIELD_CURRENT, Boolean.toString(isCurrent))) - .build()); + ICEBERG_FIELD_CURRENT, Boolean.toString(isCurrent))); + + if (field.doc() != null && !field.doc().isEmpty()) { + builder.comment(field.doc()); + } else if (existingColumnMap != null && existingColumnMap.containsKey(field.name())) { + builder.comment(existingColumnMap.get(field.name())); + } + + columns.add(builder.build()); dedupe.add(field.name()); } } 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 7c646f7bf708..1136ad63b410 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 @@ -39,6 +39,7 @@ import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.DatabaseInput; import software.amazon.awssdk.services.glue.model.StorageDescriptor; +import software.amazon.awssdk.services.glue.model.Table; import software.amazon.awssdk.services.glue.model.TableInput; public class TestIcebergToGlueConverter { @@ -306,4 +307,81 @@ public void testSetTableDescription() { .as("description should match") .isEqualTo(tableDescription); } + + @Test + public void testSetTableInputInformationWithExistingTable() { + // Actual TableInput + TableInput.Builder actualTableInputBuilder = TableInput.builder(); + Schema schema = + new Schema( + Types.NestedField.required(1, "x", Types.StringType.get()), + Types.NestedField.required(2, "y", Types.StringType.get(), "new comment"), + Types.NestedField.required(3, "z", Types.StringType.get(), "new comment")); + PartitionSpec partitionSpec = + PartitionSpec.builderFor(schema).identity("x").withSpecId(1000).build(); + TableMetadata tableMetadata = + TableMetadata.newTableMetadata(schema, partitionSpec, "s3://test", tableLocationProperties); + + // Existing Table + Table existingGlueTable = + Table.builder() + .storageDescriptor( + StorageDescriptor.builder() + .columns( + ImmutableList.of( + Column.builder().name("x").comment("existing comment").build(), + Column.builder().name("y").comment("existing comment").build())) + .build()) + .build(); + + IcebergToGlueConverter.setTableInputInformation( + actualTableInputBuilder, tableMetadata, existingGlueTable); + TableInput actualTableInput = actualTableInputBuilder.build(); + + // Expected TableInput + TableInput expectedTableInput = + TableInput.builder() + .storageDescriptor( + StorageDescriptor.builder() + .location("s3://test") + .additionalLocations(Sets.newHashSet(tableLocationProperties.values())) + .columns( + ImmutableList.of( + Column.builder() + .name("x") + .type("string") + .comment("existing comment") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "1", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "false", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build(), + Column.builder() + .name("y") + .type("string") + .comment("new comment") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "2", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "false", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build(), + Column.builder() + .name("z") + .type("string") + .comment("new comment") + .parameters( + ImmutableMap.of( + IcebergToGlueConverter.ICEBERG_FIELD_ID, "3", + IcebergToGlueConverter.ICEBERG_FIELD_OPTIONAL, "false", + IcebergToGlueConverter.ICEBERG_FIELD_CURRENT, "true")) + .build())) + .build()) + .build(); + + assertThat(actualTableInput.storageDescriptor().columns()) + .as("Columns should match") + .isEqualTo(expectedTableInput.storageDescriptor().columns()); + } } From c30aedb5054c763f8b622964b4dbeeebf91fe55e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Fri, 5 Jul 2024 12:23:38 +0530 Subject: [PATCH 0437/1019] Build: Use official revapi Gradle plugin (#10631) --- build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/build.gradle b/build.gradle index cde7342b8945..6df806db59e4 100644 --- a/build.gradle +++ b/build.gradle @@ -38,7 +38,7 @@ buildscript { classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' classpath "com.github.alisiikh:gradle-scalastyle-plugin:3.5.0" - classpath 'io.github.nastra.gradle.revapi:gradle-revapi:1.8.1' + classpath 'org.revapi:gradle-revapi:1.8.0' classpath 'com.gorylenko.gradle-git-properties:gradle-git-properties:2.4.2' classpath 'com.palantir.gradle.gitversion:gradle-git-version:3.1.0' classpath 'org.openapitools:openapi-generator-gradle-plugin:6.6.0' @@ -139,7 +139,7 @@ subprojects { apply plugin: 'java-library' if (project.name in REVAPI_PROJECTS) { - apply plugin: 'io.github.nastra.revapi' + apply plugin: 'org.revapi.revapi-gradle-plugin' revapi { oldGroup = project.group oldName = project.name From 44ae9f6c4530c704867c6b2d0ecbb56a44494d01 Mon Sep 17 00:00:00 2001 From: dongwang Date: Fri, 5 Jul 2024 15:08:44 +0800 Subject: [PATCH 0438/1019] Core: Fix create v1 table on REST Catalog (#10369) Co-authored-by: Amogh Jahagirdar --- .../org/apache/iceberg/TableMetadata.java | 12 +++++- .../apache/iceberg/rest/CatalogHandlers.java | 13 ++++-- .../apache/iceberg/catalog/CatalogTests.java | 41 +++++++++++++++++++ 3 files changed, 61 insertions(+), 5 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 6b23c337ae98..e8dcfc85fbf0 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -853,7 +853,11 @@ public static Builder buildFrom(TableMetadata base) { } public static Builder buildFromEmpty() { - return new Builder(); + return new Builder(DEFAULT_TABLE_FORMAT_VERSION); + } + + public static Builder buildFromEmpty(int formatVersion) { + return new Builder(formatVersion); } public static class Builder { @@ -903,8 +907,12 @@ public static class Builder { private final Map sortOrdersById; private Builder() { + this(DEFAULT_TABLE_FORMAT_VERSION); + } + + public Builder(int formatVersion) { this.base = null; - this.formatVersion = DEFAULT_TABLE_FORMAT_VERSION; + this.formatVersion = formatVersion; this.lastSequenceNumber = INITIAL_SEQUENCE_NUMBER; this.uuid = UUID.randomUUID().toString(); this.schemas = Lists.newArrayList(); 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 746da5ffcab0..f1b7aa32d679 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -27,12 +27,14 @@ import java.time.ZoneOffset; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import org.apache.iceberg.BaseMetadataTable; import org.apache.iceberg.BaseTable; import org.apache.iceberg.BaseTransaction; +import org.apache.iceberg.MetadataUpdate.UpgradeFormatVersion; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; @@ -373,10 +375,15 @@ private static boolean isCreate(UpdateTableRequest request) { private static TableMetadata create(TableOperations ops, UpdateTableRequest request) { // the only valid requirement is that the table will be created request.requirements().forEach(requirement -> requirement.validate(ops.current())); - - TableMetadata.Builder builder = TableMetadata.buildFromEmpty(); + Optional formatVersion = + request.updates().stream() + .filter(update -> update instanceof UpgradeFormatVersion) + .map(update -> ((UpgradeFormatVersion) update).formatVersion()) + .findFirst(); + + TableMetadata.Builder builder = + formatVersion.map(TableMetadata::buildFromEmpty).orElseGet(TableMetadata::buildFromEmpty); request.updates().forEach(update -> update.applyTo(builder)); - // create transactions do not retry. if the table exists, retrying is not a solution ops.commit(null, builder.build()); 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 8c95f2785bcf..5402a13d7d4b 100644 --- a/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java +++ b/core/src/test/java/org/apache/iceberg/catalog/CatalogTests.java @@ -68,6 +68,8 @@ import org.apache.iceberg.util.CharSequenceSet; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public abstract class CatalogTests { protected static final Namespace NS = Namespace.of("newdb"); @@ -2507,6 +2509,45 @@ public void testConcurrentReplaceTransactionSortOrderConflict() { assertFiles(afterSecondReplace, FILE_C); } + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void createTableTransaction(int formatVersion) { + if (requiresNamespaceCreate()) { + catalog().createNamespace(NS); + } + + catalog() + .newCreateTableTransaction( + TABLE, + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of("format-version", String.valueOf(formatVersion))) + .commitTransaction(); + + BaseTable table = (BaseTable) catalog().loadTable(TABLE); + assertThat(table.operations().current().formatVersion()).isEqualTo(formatVersion); + } + + @ParameterizedTest + @ValueSource(ints = {1, 2}) + public void replaceTableTransaction(int formatVersion) { + if (requiresNamespaceCreate()) { + catalog().createNamespace(NS); + } + + catalog() + .newReplaceTableTransaction( + TABLE, + SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of("format-version", String.valueOf(formatVersion)), + true) + .commitTransaction(); + + BaseTable table = (BaseTable) catalog().loadTable(TABLE); + assertThat(table.operations().current().formatVersion()).isEqualTo(formatVersion); + } + @Test public void testMetadataFileLocationsRemovalAfterCommit() { C catalog = catalog(); From 513428b38a7d2eeac1f4ab6f258f3325dc13d5a3 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Fri, 5 Jul 2024 19:10:02 +0900 Subject: [PATCH 0439/1019] Flink 1.19: Migrate source package to JUnit5 (#10632) --- .../iceberg/flink/source/SplitHelpers.java | 21 +- .../assigner/SplitAssignerTestBase.java | 28 +- .../assigner/TestDefaultSplitAssigner.java | 4 +- ...tFileSequenceNumberBasedSplitAssigner.java | 8 +- .../TestWatermarkBasedSplitAssigner.java | 10 +- .../TestContinuousIcebergEnumerator.java | 97 +++-- .../TestContinuousSplitPlannerImpl.java | 351 +++++++++--------- ...ntinuousSplitPlannerImplStartStrategy.java | 75 ++-- .../enumerator/TestEnumerationHistory.java | 23 +- .../TestIcebergEnumeratorStateSerializer.java | 67 ++-- .../source/reader/ReaderFunctionTestBase.java | 66 ++-- .../flink/source/reader/ReaderUtil.java | 19 - .../source/reader/TestArrayBatchRecords.java | 19 +- ...stArrayPoolDataIteratorBatcherRowData.java | 242 ++++++------ .../reader/TestIcebergSourceReader.java | 33 +- .../reader/TestRowDataReaderFunction.java | 13 +- .../TestIcebergSourceSplitSerializer.java | 47 +-- 17 files changed, 554 insertions(+), 569 deletions(-) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index ebd220b00dba..540902f3cea5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -18,11 +18,13 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; 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.hadoop.conf.Configuration; @@ -48,8 +50,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; public class SplitHelpers { @@ -67,14 +67,14 @@ private SplitHelpers() {} * attempt to read the data files. * *

    By default, v1 Iceberg table is created. For v2 table use {@link - * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * SplitHelpers#createSplitsFromTransientHadoopTable(Path, int, int, String)} * * @param temporaryFolder Folder to place the data to * @param fileCount The number of files to create and add to the table * @param filesPerSplit The number of files used for a split */ public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + Path temporaryFolder, int fileCount, int filesPerSplit) throws Exception { return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); } @@ -95,10 +95,9 @@ public static List createSplitsFromTransientHadoopTable( * @param version The table version to create */ public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) - throws Exception { - final File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); + Path temporaryFolder, int fileCount, int filesPerSplit, String version) throws Exception { + final File warehouseFile = File.createTempFile("junit", null, temporaryFolder.toFile()); + assertThat(warehouseFile.delete()).isTrue(); final String warehouse = "file:" + warehouseFile; Configuration hadoopConf = new Configuration(); final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); @@ -155,9 +154,7 @@ public static List createSplitsFromTransientHadoopTable( * @throws IOException If there is any error creating the mock delete files */ public static List equipSplitsWithMockDeleteFiles( - List icebergSourceSplits, - TemporaryFolder temporaryFolder, - int deleteFilesPerSplit) + List icebergSourceSplits, Path temporaryFolder, int deleteFilesPerSplit) throws IOException { List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); for (IcebergSourceSplit split : icebergSourceSplits) { @@ -171,7 +168,7 @@ public static List equipSplitsWithMockDeleteFiles( final DeleteFile deleteFile = FileMetadata.deleteFileBuilder(spec) .withFormat(FileFormat.PARQUET) - .withPath(temporaryFolder.newFile().getPath()) + .withPath(File.createTempFile("junit", null, temporaryFolder.toFile()).getPath()) .ofPositionDeletes() .withFileSizeInBytes(1000) .withRecordCount(1000) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index 090b304942c6..1e612b0a2b2a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +import java.nio.file.Path; import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -25,13 +29,11 @@ import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class SplitAssignerTestBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; @Test public void testEmptyInitialization() { @@ -86,11 +88,11 @@ private void assertAvailableFuture( // calling isAvailable again should return the same object reference // note that thenAccept will return a new future. // we want to assert the same instance on the assigner returned future - Assert.assertSame(future, assigner.isAvailable()); + assertThat(assigner.isAvailable()).isSameAs(future); // now add some splits addSplitsRunnable.run(); - Assert.assertEquals(true, futureCompleted.get()); + assertThat(futureCompleted.get()).isTrue(); for (int i = 0; i < splitCount; ++i) { assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -101,29 +103,29 @@ private void assertAvailableFuture( protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(expectedStatus, result.status()); + assertThat(result.status()).isEqualTo(expectedStatus); switch (expectedStatus) { case AVAILABLE: - Assert.assertNotNull(result.split()); + assertThat(result.split()).isNotNull(); break; case CONSTRAINED: case UNAVAILABLE: - Assert.assertNull(result.split()); + assertThat(result.split()).isNull(); break; default: - Assert.fail("Unknown status: " + expectedStatus); + fail("Unknown status: %s", expectedStatus); } } protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Collection stateBeforeGet = assigner.state(); - Assert.assertEquals(splitCount, stateBeforeGet.size()); + assertThat(stateBeforeGet).hasSize(splitCount); } protected List createSplits(int fileCount, int filesPerSplit, String version) throws Exception { return SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + temporaryFolder, fileCount, filesPerSplit, version); } protected abstract SplitAssigner splitAssigner(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java index 8994f3054abe..17e64bbf0594 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.source.assigner; import org.apache.iceberg.flink.source.SplitHelpers; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDefaultSplitAssigner extends SplitAssignerTestBase { @Override @@ -32,7 +32,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInASplit() throws Exception { SplitAssigner assigner = splitAssigner(); assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 4, 2)); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); 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 20555ad464a9..ff63ba8e58a0 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; @@ -26,8 +27,7 @@ import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { @Override @@ -70,12 +70,12 @@ public void testSerializable() { byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); SerializableComparator comparator = SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); + assertThat(comparator).isNotNull(); } private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); - Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + assertThat(file.fileSequenceNumber()).isEqualTo(expectedSequenceNumber); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java index 6c0cbb733320..84f04d5a530a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.assigner; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Instant; @@ -44,8 +45,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { public static final Schema SCHEMA = @@ -104,12 +104,12 @@ public void testSerializable() { TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); SerializableComparator comparator = SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); + assertThat(comparator).isNotNull(); } private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(result.split(), split); + assertThat(split).isEqualTo(result.split()); } @Override @@ -138,7 +138,7 @@ private IcebergSourceSplit splitFromRecords(List> records) { try { return IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( - records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + records, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); } catch (IOException e) { throw new RuntimeException("Split creation exception", e); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 95770bb7dc2f..272734185c66 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -36,13 +37,11 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousIcebergEnumerator { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; @Test public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { @@ -59,19 +58,19 @@ public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { Collection pendingSplitsEmpty = enumerator.snapshotState(1).pendingSplits(); - Assert.assertEquals(0, pendingSplitsEmpty.size()); + assertThat(pendingSplitsEmpty).isEmpty(); // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } @Test @@ -94,11 +93,11 @@ public void testDiscoverWhenReaderRegistered() throws Exception { // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); - Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -126,26 +125,26 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - Assert.assertEquals(1, splits.size()); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + assertThat(splits).hasSize(1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); - Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); + assertThat(enumeratorContext.getSplitAssignments()).doesNotContainKey(2); List pendingSplitIds = enumerator.snapshotState(1).pendingSplits().stream() .map(IcebergSourceSplitState::split) .map(IcebergSourceSplit::splitId) .collect(Collectors.toList()); - Assert.assertEquals(splits.size(), pendingSplitIds.size()); - Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); + assertThat(pendingSplitIds).hasSameSizeAs(splits); + assertThat(pendingSplitIds).first().isEqualTo(splits.get(0).splitId()); // register the reader again, and let it request a split enumeratorContext.registerReader(2, "localhost"); enumerator.addReader(2); enumerator.handleSourceEvent(2, new SplitRequestEvent()); - Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -154,7 +153,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio public void testThrottlingDiscovery() throws Exception { // create 10 splits List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 1); TestingSplitEnumeratorContext enumeratorContext = new TestingSplitEnumeratorContext<>(4); @@ -179,10 +178,10 @@ public void testThrottlingDiscovery() throws Exception { enumeratorContext.triggerAllActions(); // because discovered split was assigned to reader, pending splits should be empty - Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); // split assignment to reader-2 should contain splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); // add the remaining 9 splits (one for every snapshot) // run discovery cycles while reader-2 still processing the splits[0] @@ -192,20 +191,20 @@ public void testThrottlingDiscovery() throws Exception { } // can only discover up to 3 snapshots/splits - Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); + assertThat(enumerator.snapshotState(2).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); // now reader-2 finished splits[0] enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); + assertThat(enumerator.snapshotState(3).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); // run 3 more split discovery cycles for (int i = 0; i < 3; ++i) { @@ -213,20 +212,20 @@ public void testThrottlingDiscovery() throws Exception { } // no more splits are discovered due to throttling - Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); + assertThat(enumerator.snapshotState(4).pendingSplits()).hasSize(3); // split assignment to reader-2 should still be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); // now reader-2 finished splits[1] enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); + assertThat(enumerator.snapshotState(5).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 3) - Assert.assertEquals( - splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 3)); } @Test @@ -246,20 +245,20 @@ public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); // Trigger a planning and check that no splits returned due to the planning error enumeratorContext.triggerAllActions(); - Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); // Second scan planning should succeed and discover the expected splits enumeratorContext.triggerAllActions(); Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } @Test @@ -278,19 +277,19 @@ public void testOverMaxAllowedPlanningErrors() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); // Check that the scheduler response ignores the current error and continues to run until the // failure limit is reached enumeratorContext.triggerAllActions(); - Assert.assertFalse( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isFalse(); // Check that the task has failed with the expected exception after the failure limit is reached enumeratorContext.triggerAllActions(); - Assert.assertTrue( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isTrue(); assertThatThrownBy( () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) .hasCauseInstanceOf(RuntimeException.class) @@ -316,7 +315,7 @@ public void testPlanningIgnoringErrors() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); Collection pendingSplits; @@ -324,16 +323,16 @@ public void testPlanningIgnoringErrors() throws Exception { for (int i = 0; i < expectedFailures; ++i) { enumeratorContext.triggerAllActions(); pendingSplits = enumerator.snapshotState(i).pendingSplits(); - Assert.assertEquals(0, pendingSplits.size()); + assertThat(pendingSplits).isEmpty(); } // Discovered the new split after a successful scan planning enumeratorContext.triggerAllActions(); pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } private static ContinuousIcebergEnumerator createEnumerator( 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 f98e93835d15..381c340edd6a 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 @@ -18,9 +18,11 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -31,33 +33,27 @@ import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousSplitPlannerImpl { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private static final FileFormat fileFormat = FileFormat.PARQUET; private static final AtomicLong randomSeed = new AtomicLong(); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - @Rule public TestName testName = new TestName(); + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private GenericAppenderHelper dataAppender; private DataFile dataFile1; @@ -65,9 +61,9 @@ public class TestContinuousSplitPlannerImpl { private DataFile dataFile2; private Snapshot snapshot2; - @Before + @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), fileFormat, temporaryFolder); } private void appendTwoSnapshots() throws IOException { @@ -75,13 +71,13 @@ private void appendTwoSnapshots() throws IOException { List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); dataFile1 = dataAppender.writeFile(null, batch1); dataAppender.appendToTable(dataFile1); - snapshot1 = tableResource.table().currentSnapshot(); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); // snapshot2 List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); dataFile2 = dataAppender.writeFile(null, batch2); dataAppender.appendToTable(dataFile2); - snapshot2 = tableResource.table().currentSnapshot(); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } /** @return the last enumerated snapshot id */ @@ -92,21 +88,22 @@ private CycleResult verifyOneCycle( RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); - Snapshot snapshot = tableResource.table().currentSnapshot(); + Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); - Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); - Assert.assertEquals( - lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, result.splits().size()); + assertThat(result.fromPosition().snapshotId()).isEqualTo(lastPosition.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs()) + .isEqualTo(lastPosition.snapshotTimestampMs()); + assertThat(result.toPosition().snapshotId().longValue()).isEqualTo(snapshot.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot.timestampMillis()); + assertThat(result.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Assert.assertEquals( - dataFile.path().toString(), - Iterables.getOnlyElement(split.task().files()).file().path().toString()); + assertThat(split.task().files()) + .hasSize(1) + .first() + .satisfies( + fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); return new CycleResult(result.toPosition(), split); } @@ -117,21 +114,21 @@ public void testTableScanThenIncrementalWithEmptyTable() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); @@ -149,24 +146,24 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - Assert.assertEquals( - snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.fromPosition()).isNull(); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); Set expectedFiles = ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -182,27 +179,27 @@ public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { .splitSize(1L) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // latest mode should discover both snapshots, as latest position is marked by when job starts appendTwoSnapshots(); ContinuousEnumerationResult afterTwoSnapshotsAppended = splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); - Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); + assertThat(afterTwoSnapshotsAppended.splits()).hasSize(2); // next 3 snapshots IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); @@ -220,35 +217,36 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1 // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -263,21 +261,21 @@ public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); @@ -295,24 +293,25 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1's parent, // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertNull(secondResult.fromPosition().snapshotId()); - Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId()).isNull(); + assertThat(secondResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) @@ -320,7 +319,7 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except // should discover files appended in both snapshot1 and snapshot2 Set expectedFiles = ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -337,7 +336,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { .build(); ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -363,7 +362,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -380,35 +379,36 @@ public void testIncrementalFromSnapshotId() throws Exception { .startSnapshotId(snapshot2.snapshotId()) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as // snapshot2's parent) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -425,7 +425,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { .build(); ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -446,7 +446,7 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -463,34 +463,35 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { .startSnapshotTimestamp(snapshot2.timestampMillis()) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -513,15 +514,15 @@ public void testMaxPlanningSnapshotCount() throws Exception { .maxPlanningSnapshotCount(1) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1's parent, // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 @@ -544,12 +545,12 @@ public void testTableScanNoStats() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 0); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -570,12 +571,12 @@ public void testTableScanAllStats() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 3); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -596,12 +597,12 @@ public void testTableScanSingleStat() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 1); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -619,12 +620,12 @@ private void verifyStatCount(IcebergSourceSplit split, int expected) { .files() .forEach( f -> { - Assert.assertNull(f.file().valueCounts()); - Assert.assertNull(f.file().columnSizes()); - Assert.assertNull(f.file().lowerBounds()); - Assert.assertNull(f.file().upperBounds()); - Assert.assertNull(f.file().nanValueCounts()); - Assert.assertNull(f.file().nullValueCounts()); + assertThat(f.file().valueCounts()).isNull(); + assertThat(f.file().columnSizes()).isNull(); + assertThat(f.file().lowerBounds()).isNull(); + assertThat(f.file().upperBounds()).isNull(); + assertThat(f.file().nanValueCounts()).isNull(); + assertThat(f.file().nullValueCounts()).isNull(); }); } else { split @@ -632,13 +633,13 @@ private void verifyStatCount(IcebergSourceSplit split, int expected) { .files() .forEach( f -> { - Assert.assertEquals(expected, f.file().valueCounts().size()); - Assert.assertEquals(expected, f.file().columnSizes().size()); - Assert.assertEquals(expected, f.file().lowerBounds().size()); - Assert.assertEquals(expected, f.file().upperBounds().size()); - Assert.assertEquals(expected, f.file().nullValueCounts().size()); + assertThat(f.file().valueCounts()).hasSize(expected); + assertThat(f.file().columnSizes()).hasSize(expected); + assertThat(f.file().lowerBounds()).hasSize(expected); + assertThat(f.file().upperBounds()).hasSize(expected); + assertThat(f.file().nullValueCounts()).hasSize(expected); // The nanValue is not counted for long and string fields - Assert.assertEquals(0, f.file().nanValueCounts().size()); + assertThat(f.file().nanValueCounts()).isEmpty(); }); } } @@ -649,36 +650,34 @@ private void verifyMaxPlanningSnapshotCountResult( Snapshot toSnapshotInclusive, Set expectedFiles) { if (fromSnapshotExclusive == null) { - Assert.assertNull(result.fromPosition().snapshotId()); - Assert.assertNull(result.fromPosition().snapshotTimestampMs()); + assertThat(result.fromPosition().snapshotId()).isNull(); + assertThat(result.fromPosition().snapshotTimestampMs()).isNull(); } else { - Assert.assertEquals( - fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - fromSnapshotExclusive.timestampMillis(), - result.fromPosition().snapshotTimestampMs().longValue()); + assertThat(result.fromPosition().snapshotId().longValue()) + .isEqualTo(fromSnapshotExclusive.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(fromSnapshotExclusive.timestampMillis()); } - Assert.assertEquals( - toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - toSnapshotInclusive.timestampMillis(), - result.toPosition().snapshotTimestampMs().longValue()); + assertThat(result.toPosition().snapshotId().longValue()) + .isEqualTo(toSnapshotInclusive.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(toSnapshotInclusive.timestampMillis()); // should only have one split with one data file, because split discover is limited to // one snapshot and each snapshot has only one data file appended. IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); - return tableResource.table().currentSnapshot(); + return TABLE_RESOURCE.table().currentSnapshot(); } private static class CycleResult { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index ce041dbbcf59..1cc2e175e698 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -18,58 +18,57 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Snapshot; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.RuleChain; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestRule; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousSplitPlannerImplStartStrategy { private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - public final HadoopTableResource tableResource = - new HadoopTableResource( - temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); + @TempDir protected Path temporaryFolder; + + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private GenericAppenderHelper dataAppender; private Snapshot snapshot1; private Snapshot snapshot2; private Snapshot snapshot3; - @Before + @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); } private void appendThreeSnapshots() throws IOException { List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); dataAppender.appendToTable(batch1); - snapshot1 = tableResource.table().currentSnapshot(); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); dataAppender.appendToTable(batch2); - snapshot2 = tableResource.table().currentSnapshot(); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); dataAppender.appendToTable(batch3); - snapshot3 = tableResource.table().currentSnapshot(); + snapshot3 = TABLE_RESOURCE.table().currentSnapshot(); } @Test @@ -81,13 +80,13 @@ public void testTableScanThenIncrementalStrategy() throws IOException { .build(); // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); } @Test @@ -99,13 +98,13 @@ public void testForLatestSnapshotStrategy() throws IOException { .build(); // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); } @Test @@ -117,13 +116,13 @@ public void testForEarliestSnapshotStrategy() throws IOException { .build(); // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot1.snapshotId()); } @Test @@ -139,7 +138,7 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotId)) + TABLE_RESOURCE.table(), scanContextInvalidSnapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); @@ -153,8 +152,8 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } @Test @@ -170,7 +169,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotTimestamp)) + TABLE_RESOURCE.table(), scanContextInvalidSnapshotTimestamp)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find a snapshot after: "); @@ -184,8 +183,8 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } @Test @@ -200,7 +199,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOE .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), config).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java index e2be0b4b0398..feefcb98646b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.flink.source.enumerator; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; public class TestEnumerationHistory { private static final int MAX_HISTORY_SIZE = 3; @@ -89,28 +90,28 @@ public void testThreeMoreThanFullHistory() { } private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); if (history.hasFullHistory()) { // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); } else { // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isFalse(); } int[] historySnapshot = history.snapshot(); - Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); + assertThat(historySnapshot).containsExactly(expectedHistorySnapshot); EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); restoredHistory.restore(historySnapshot); - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); if (history.hasFullHistory()) { // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); } else { // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); + assertThat(history.shouldPauseSplitDiscovery(30)).isFalse(); } } @@ -125,10 +126,10 @@ public void testRestoreDifferentSize() { EnumerationHistory smallerHistory = new EnumerationHistory(2); smallerHistory.restore(historySnapshot); int[] expectedRestoredHistorySnapshot = {2, 3}; - Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); + assertThat(smallerHistory.snapshot()).containsExactly(expectedRestoredHistorySnapshot); EnumerationHistory largerHisotry = new EnumerationHistory(4); largerHisotry.restore(historySnapshot); - Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); + assertThat(largerHisotry.snapshot()).containsExactly(historySnapshot); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java index 1d12d9f66a8a..2520a6b763e4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java @@ -18,48 +18,48 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergEnumeratorStateSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private final IcebergEnumeratorStateSerializer serializer = new IcebergEnumeratorStateSerializer(true); - protected final int version; - - @Parameterized.Parameters(name = "version={0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } + @Parameter(index = 0) + protected int version; - public TestIcebergEnumeratorStateSerializer(int version) { - this.version = version; + @Parameters(name = "version={0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {1}, new Object[] {2}}; } - @Test + @TestTemplate public void testEmptySnapshotIdAndPendingSplits() throws Exception { IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); testSerializer(enumeratorState); } - @Test + @TestTemplate public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); @@ -69,12 +69,12 @@ public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { testSerializer(enumeratorState); } - @Test + @TestTemplate public void testSomeSnapshotIdAndPendingSplits() throws Exception { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); Collection pendingSplits = Lists.newArrayList(); pendingSplits.add( new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); @@ -87,13 +87,13 @@ public void testSomeSnapshotIdAndPendingSplits() throws Exception { testSerializer(enumeratorState); } - @Test + @TestTemplate public void testEnumerationSplitCountHistory() throws Exception { if (version == 2) { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); Collection pendingSplits = Lists.newArrayList(); pendingSplits.add( new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); @@ -123,23 +123,24 @@ private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOExc private void assertEnumeratorStateEquals( IcebergEnumeratorState expected, IcebergEnumeratorState actual) { - Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); + assertThat(actual.lastEnumeratedPosition()).isEqualTo(expected.lastEnumeratedPosition()); - Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); + assertThat(actual.pendingSplits()).hasSameSizeAs(expected.pendingSplits()); Iterator expectedIterator = expected.pendingSplits().iterator(); Iterator actualIterator = actual.pendingSplits().iterator(); for (int i = 0; i < expected.pendingSplits().size(); ++i) { IcebergSourceSplitState expectedSplitState = expectedIterator.next(); IcebergSourceSplitState actualSplitState = actualIterator.next(); - Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); - Assert.assertEquals( - expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); - Assert.assertEquals( - expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); - Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); + assertThat(actualSplitState.split().splitId()) + .isEqualTo(expectedSplitState.split().splitId()); + assertThat(actualSplitState.split().fileOffset()) + .isEqualTo(expectedSplitState.split().fileOffset()); + assertThat(actualSplitState.split().recordOffset()) + .isEqualTo(expectedSplitState.split().recordOffset()); + assertThat(actualSplitState.status()).isEqualTo(expectedSplitState.status()); } - Assert.assertArrayEquals( - expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); + assertThat(actual.enumerationSplitCountHistory()) + .containsExactly(expected.enumerationSplitCountHistory()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java index 0d33e4ed08ad..e6dde153ea6f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java @@ -18,11 +18,17 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.iceberg.CombinedScanTask; 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.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; @@ -30,17 +36,14 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 ReaderFunctionTestBase { - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.AVRO}, @@ -49,19 +52,18 @@ public static Object[][] parameters() { }; } - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; protected abstract ReaderFunction readerFunction(); protected abstract void assertRecords(List expected, List actual, Schema schema); - private final FileFormat fileFormat; - private final GenericAppenderFactory appenderFactory; + @Parameter(index = 0) + private FileFormat fileFormat; - public ReaderFunctionTestBase(FileFormat fileFormat) { - this.fileFormat = fileFormat; - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + @Parameter(index = 1) + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); private void assertRecordsAndPosition( List expectedRecords, @@ -74,24 +76,22 @@ private void assertRecordsAndPosition( RecordAndPosition recordAndPosition; while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { actualRecords.add(recordAndPosition.record()); - Assert.assertEquals( - "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); - Assert.assertEquals( - "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); + assertThat(recordAndPosition.fileOffset()).isEqualTo(expectedFileOffset); + assertThat(recordAndPosition.recordOffset() - 1).isEqualTo(recordOffset); recordOffset++; } - Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); + assertThat(actualRecords).hasSameSizeAs(expectedRecords); assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testNoCheckpointedPosition() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); CloseableIterator>> reader = readerFunction().apply(split); @@ -109,13 +109,13 @@ public void testNoCheckpointedPosition() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionBeforeFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); CloseableIterator>> reader = readerFunction().apply(split); @@ -133,13 +133,13 @@ public void testCheckpointedPositionBeforeFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionMiddleFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); CloseableIterator>> reader = readerFunction().apply(split); @@ -157,13 +157,13 @@ public void testCheckpointedPositionMiddleFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionAfterFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); CloseableIterator>> reader = readerFunction().apply(split); @@ -177,13 +177,13 @@ public void testCheckpointedPositionAfterFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionBeforeSecondFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); CloseableIterator>> reader = readerFunction().apply(split); @@ -197,13 +197,13 @@ public void testCheckpointedPositionBeforeSecondFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionMidSecondFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); CloseableIterator>> reader = readerFunction().apply(split); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index e3e341ca2c76..0edf8ae009fe 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.rules.TemporaryFolder; public class ReaderUtil { @@ -107,24 +106,6 @@ public static List> createRecordBatchList( return Lists.partition(records, batchCount); } - // Only for JUnit4 tests. Keep this method for test migration from JUnit4 to JUnit5 - public static CombinedScanTask createCombinedScanTask( - List> recordBatchList, - TemporaryFolder temporaryFolder, - FileFormat fileFormat, - GenericAppenderFactory appenderFactory) - throws IOException { - List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); - for (List recordBatch : recordBatchList) { - FileScanTask fileTask = - ReaderUtil.createFileTask( - recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); - fileTasks.add(fileTask); - } - - return new BaseCombinedScanTask(fileTasks); - } - public static CombinedScanTask createCombinedScanTask( List> recordBatchList, Path temporaryFolder, diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java index 644ac2bad6b8..6f09bd9a56d6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestArrayBatchRecords { @@ -50,19 +51,19 @@ private void testArray( fileOffset, startingRecordOffset); - Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); + assertThat(recordsWithSplitIds.nextSplit()).isEqualTo(splitId); for (int i = 0; i < numberOfRecords; i++) { RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); - Assert.assertEquals(elements[i], recAndPos.record()); - Assert.assertEquals(fileOffset, recAndPos.fileOffset()); + assertThat(recAndPos.record()).isEqualTo(elements[i]); + assertThat(recAndPos.fileOffset()).isEqualTo(fileOffset); // recordOffset points to the position after this one - Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); + assertThat(recAndPos.recordOffset()).isEqualTo(startingRecordOffset + i + 1); } - Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); - Assert.assertNull(recordsWithSplitIds.nextSplit()); + assertThat(recordsWithSplitIds.nextRecordFromSplit()).isNull(); + assertThat(recordsWithSplitIds.nextSplit()).isNull(); recordsWithSplitIds.recycle(); - Assert.assertTrue(recycled.get()); + assertThat(recycled.get()).isTrue(); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java index f964a7707689..bcdc827bc695 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Path; import java.util.Arrays; import java.util.List; import org.apache.flink.configuration.Configuration; @@ -36,28 +40,27 @@ import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.DataIterator; import org.apache.iceberg.io.CloseableIterator; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestArrayPoolDataIteratorBatcherRowData { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private static final FileFormat fileFormat = FileFormat.PARQUET; + @TempDir protected Path temporaryFolder; + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private static final Configuration config = new Configuration(); - private final GenericAppenderFactory appenderFactory; - private final DataIteratorBatcher batcher; + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA);; + private final DataIteratorBatcher batcher = + new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - public TestArrayPoolDataIteratorBatcherRowData() { - Configuration config = new Configuration(); + @BeforeAll + public static void setConfig() { // set array pool size to 1 config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); // set batch array size to 2 config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - this.batcher = - new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); } /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ @@ -65,7 +68,11 @@ public TestArrayPoolDataIteratorBatcherRowData() { public void testSingleFileLessThanOneFullBatch() throws Exception { List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); String splitId = "someSplitId"; @@ -73,29 +80,29 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { batcher.batch(splitId, dataIterator); ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch.nextSplit()); + assertThat(batch.finishedSplits()).isEmpty(); + assertThat(batch.nextSplit()).isEqualTo(splitId); // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch.records().length); + assertThat(batch.records()).hasSize(2); // assert actual number of records in the array - Assert.assertEquals(1, batch.numberOfRecords()); + assertThat(batch.numberOfRecords()).isEqualTo(1); RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); /////////////////////////////// // assert first record - Assert.assertEquals(0, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - Assert.assertNull(batch.nextRecordFromSplit()); - Assert.assertNull(batch.nextSplit()); + assertThat(batch.nextRecordFromSplit()).isNull(); + assertThat(batch.nextSplit()).isNull(); batch.recycle(); // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } /** @@ -107,7 +114,11 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { public void testSingleFileWithMultipleBatches() throws Exception { List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); String splitId = "someSplitId"; @@ -118,31 +129,31 @@ public void testSingleFileWithMultipleBatches() throws Exception { // assert first batch with full batch of 2 records ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch0.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch0.nextSplit()); + assertThat(batch0.finishedSplits()).isEmpty(); + assertThat(batch0.nextSplit()).isEqualTo(splitId); // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch0.records().length); + assertThat(batch0.records()).hasSize(2); // assert actual number of records in the array - Assert.assertEquals(2, batch0.numberOfRecords()); + assertThat(batch0.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; // assert first record recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); // assert second record recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); - Assert.assertNull(batch0.nextRecordFromSplit()); - Assert.assertNull(batch0.nextSplit()); + assertThat(batch0.nextRecordFromSplit()).isNull(); + assertThat(batch0.nextSplit()).isNull(); batch0.recycle(); /////////////////////////////// @@ -150,30 +161,30 @@ public void testSingleFileWithMultipleBatches() throws Exception { ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); // assert array is reused - Assert.assertSame(batch0.records(), batch1.records()); - Assert.assertTrue(batch1.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch1.nextSplit()); + assertThat(batch1.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch1.finishedSplits()).isEmpty(); + assertThat(batch1.nextSplit()).isEqualTo(splitId); // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch1.records().length); + assertThat(batch1.records()).hasSize(2); // assert actual number of records in the array - Assert.assertEquals(2, batch1.numberOfRecords()); + assertThat(batch1.numberOfRecords()).isEqualTo(2); // assert third record recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); // assert fourth record recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); - Assert.assertNull(batch1.nextRecordFromSplit()); - Assert.assertNull(batch1.nextSplit()); + assertThat(batch1.nextRecordFromSplit()).isNull(); + assertThat(batch1.nextSplit()).isNull(); batch1.recycle(); /////////////////////////////// @@ -181,27 +192,27 @@ public void testSingleFileWithMultipleBatches() throws Exception { ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); // assert array is reused - Assert.assertSame(batch0.records(), batch2.records()); - Assert.assertTrue(batch2.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch2.nextSplit()); + assertThat(batch2.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch2.finishedSplits()).isEmpty(); + assertThat(batch2.nextSplit()).isEqualTo(splitId); // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch2.records().length); + assertThat(batch2.records()).hasSize(2); // assert actual number of records in the array - Assert.assertEquals(1, batch2.numberOfRecords()); + assertThat(batch2.numberOfRecords()).isEqualTo(1); // assert fifth record recordAndPosition = batch2.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(5, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(5); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); - Assert.assertNull(batch2.nextRecordFromSplit()); - Assert.assertNull(batch2.nextSplit()); + assertThat(batch2.nextRecordFromSplit()).isNull(); + assertThat(batch2.nextSplit()).isNull(); batch2.recycle(); // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } /** @@ -214,20 +225,28 @@ public void testMultipleFilesWithSeekPosition() throws Exception { List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); FileScanTask fileTask0 = ReaderUtil.createFileTask( - records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records0, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); FileScanTask fileTask1 = ReaderUtil.createFileTask( - records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records1, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); FileScanTask fileTask2 = ReaderUtil.createFileTask( - records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records2, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - // seek to file1 and after record 1 dataIterator.seek(1, 1); String splitId = "someSplitId"; @@ -246,52 +265,49 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch10.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch10.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch10.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch10.numberOfRecords()); + assertThat(batch10.finishedSplits()).isEmpty(); + assertThat(batch10.nextSplit()).isEqualTo(splitId); + assertThat(batch10.records()).hasSize(2); + assertThat(batch10.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); // seek should skip the first record in file1. starting from the second record - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); - Assert.assertNull(batch10.nextRecordFromSplit()); - Assert.assertNull(batch10.nextSplit()); + assertThat(batch10.nextRecordFromSplit()).isNull(); + assertThat(batch10.nextSplit()).isNull(); batch10.recycle(); // assert second batch from file1 with partial batch of 1 record // variable naming convention: batch__ ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch11.records()); - Assert.assertTrue(batch11.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch11.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch11.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch11.numberOfRecords()); + assertThat(batch11.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch11.finishedSplits()).isEmpty(); + assertThat(batch11.nextSplit()).isEqualTo(splitId); + assertThat(batch11.records()).hasSize(2); + assertThat(batch11.numberOfRecords()).isEqualTo(1); recordAndPosition = batch11.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); - Assert.assertNull(batch11.nextRecordFromSplit()); - Assert.assertNull(batch11.nextSplit()); + assertThat(batch11.nextRecordFromSplit()).isNull(); + assertThat(batch11.nextSplit()).isNull(); batch11.recycle(); /////////////////////////////// @@ -303,29 +319,27 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch__ ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch20.records()); - Assert.assertTrue(batch20.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch20.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch20.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch20.numberOfRecords()); + assertThat(batch20.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch20.finishedSplits()).isEmpty(); + assertThat(batch20.nextSplit()).isEqualTo(splitId); + assertThat(batch20.records()).hasSize(2); + assertThat(batch20.numberOfRecords()).isEqualTo(2); recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.recordOffset()).isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); - Assert.assertNull(batch20.nextRecordFromSplit()); - Assert.assertNull(batch20.nextSplit()); + assertThat(batch20.nextRecordFromSplit()).isNull(); + assertThat(batch20.nextSplit()).isNull(); batch20.recycle(); /////////////////////////////// @@ -333,26 +347,24 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch__ ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch21.records()); - Assert.assertTrue(batch21.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch21.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch21.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch21.numberOfRecords()); + assertThat(batch21.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch21.finishedSplits()).isEmpty(); + assertThat(batch21.nextSplit()).isEqualTo(splitId); + assertThat(batch21.records()).hasSize(2); + assertThat(batch21.numberOfRecords()).isEqualTo(1); recordAndPosition = batch21.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); - Assert.assertNull(batch21.nextRecordFromSplit()); - Assert.assertNull(batch21.nextSplit()); + assertThat(batch21.nextRecordFromSplit()).isNull(); + assertThat(batch21.nextSplit()).isNull(); batch21.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index b9af6aca2895..8d6782586676 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -37,19 +40,14 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.hadoop.HadoopFileIO; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceReader { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final GenericAppenderFactory appenderFactory; + @TempDir protected Path temporaryFolder; - public TestIcebergSourceReader() { - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); @Test public void testReaderMetrics() throws Exception { @@ -70,13 +68,13 @@ public void testReaderOrder() throws Exception { ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task1 = ReaderUtil.createCombinedScanTask( - recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList1, temporaryFolder, FileFormat.PARQUET, appenderFactory); List> recordBatchList2 = ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task2 = ReaderUtil.createCombinedScanTask( - recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList2, temporaryFolder, FileFormat.PARQUET, appenderFactory); // Sort the splits in one way List rowDataList1 = @@ -95,8 +93,7 @@ public void testReaderOrder() throws Exception { 2); // Check that the order of the elements is not changed - Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); - Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + assertThat(rowDataList1).containsExactlyElementsOf(rowDataList2); } private List read(List splits, long expected) throws Exception { @@ -114,7 +111,7 @@ private List read(List splits, long expected) throw reader.pollNext(readerOutput); - Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + assertThat(readerOutput.getEmittedRecords()).hasSize((int) expected); return readerOutput.getEmittedRecords(); } @@ -130,7 +127,7 @@ private void testOneSplitFetcher( ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); reader.addSplits(Collections.singletonList(split)); @@ -138,12 +135,12 @@ private void testOneSplitFetcher( reader.pollNext(readerOutput); } - Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); + assertThat(readerOutput.getEmittedRecords()).hasSize(expectedCount); TestHelpers.assertRowData( TestFixtures.SCHEMA, recordBatchList.get(0).get(0), readerOutput.getEmittedRecords().get(expectedCount - 1)); - Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); + assertThat(metricGroup.counters().get("assignedSplits").getCount()).isEqualTo(expectedCount); // One more poll will get null record batch. // That will finish the split and cause split fetcher to be closed due to idleness. diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java index 764215e904d3..55f9c0af3a29 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -28,7 +28,6 @@ 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.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.data.Record; import org.apache.iceberg.encryption.PlaintextEncryptionManager; @@ -39,13 +38,9 @@ public class TestRowDataReaderFunction extends ReaderFunctionTestBase { - protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final DataStructureConverter rowDataConverter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - - public TestRowDataReaderFunction(FileFormat fileFormat) { - super(fileFormat); - } + protected static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final DataStructureConverter ROW_DATA_CONVERTER = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(ROW_TYPE)); @Override protected ReaderFunction readerFunction() { @@ -68,7 +63,7 @@ protected void assertRecords(List expected, List actual, Schema private List toRows(List actual) { return actual.stream() - .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) + .map(rowData -> (Row) ROW_DATA_CONVERTER.toExternal(rowData)) .collect(Collectors.toList()); } } 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 c72d622f86ba..12bacdcd074d 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 @@ -18,20 +18,21 @@ */ package org.apache.iceberg.flink.source.split; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceSplitSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); @@ -44,14 +45,14 @@ public void testLatestVersion() throws Exception { private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = serializer.serialize(split); IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); assertSplitEquals(split, deserialized); byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); + assertThat(cachedResult).isSameAs(result); IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult); assertSplitEquals(split, deserialized2); @@ -59,7 +60,7 @@ private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws E split.updatePosition(0, 100); byte[] resultAfterUpdatePosition = serializer.serialize(split); // after position change, serialized bytes should have changed - Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); + assertThat(resultAfterUpdatePosition).isNotSameAs(cachedResult); IcebergSourceSplit deserialized3 = serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); assertSplitEquals(split, deserialized3); @@ -75,7 +76,7 @@ public void testV1() throws Exception { private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV1(); IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); @@ -92,7 +93,7 @@ public void testV2() throws Exception { private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV2(); IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); @@ -109,9 +110,9 @@ private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mo throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); final List splitsWithMockDeleteFiles = - SplitHelpers.equipSplitsWithMockDeleteFiles(splits, TEMPORARY_FOLDER, mockDeletesPerSplit); + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, temporaryFolder, mockDeletesPerSplit); for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { byte[] result = split.serializeV3(); @@ -123,7 +124,7 @@ private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mo @Test public void testDeserializeV1() throws Exception { final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV1(); IcebergSourceSplit deserialized = serializer.deserialize(1, result); @@ -135,7 +136,7 @@ public void testDeserializeV1() throws Exception { public void testCheckpointedPosition() throws Exception { final AtomicInteger index = new AtomicInteger(); final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 2).stream() .map( split -> { IcebergSourceSplit result; @@ -155,7 +156,7 @@ public void testCheckpointedPosition() throws Exception { assertSplitEquals(split, deserialized); byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); + assertThat(cachedResult).isSameAs(result); IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult); assertSplitEquals(split, deserialized2); @@ -165,18 +166,18 @@ public void testCheckpointedPosition() throws Exception { private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); - Assert.assertEquals(expectedTasks.size(), actualTasks.size()); + assertThat(actualTasks).hasSameSizeAs(expectedTasks); for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); - Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); - Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); - Assert.assertEquals(expectedTask.start(), actualTask.start()); - Assert.assertEquals(expectedTask.length(), actualTask.length()); + assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); + assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); + assertThat(actualTask.start()).isEqualTo(expectedTask.start()); + assertThat(actualTask.length()).isEqualTo(expectedTask.length()); } - Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); - Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); + assertThat(actual.fileOffset()).isEqualTo(expected.fileOffset()); + assertThat(actual.recordOffset()).isEqualTo(expected.recordOffset()); } } From 7b023c9bdf91c1cf9f3c90530c4b8b8258e1842e Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Fri, 5 Jul 2024 09:49:18 -0700 Subject: [PATCH 0440/1019] Spark 3.5: Support read of partition metadata column when table has over 1k columns (#10547) --- .../spark/source/SparkScanBuilder.java | 49 +++++++++++++++++-- .../source/TestSparkMetadataColumns.java | 47 ++++++++++++++++++ 2 files changed, 93 insertions(+), 3 deletions(-) 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 6b97e48133fd..9dc214a755d3 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 @@ -20,6 +20,9 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.BaseTable; @@ -48,6 +51,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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkAggregates; import org.apache.iceberg.spark.SparkReadConf; @@ -342,15 +346,54 @@ public void pruneColumns(StructType requestedSchema) { private Schema schemaWithMetadataColumns() { // metadata columns - List fields = + List metadataFields = metaColumns.stream() .distinct() .map(name -> MetadataColumns.metadataColumn(table, name)) .collect(Collectors.toList()); - Schema meta = new Schema(fields); + Schema metadataSchema = calculateMetadataSchema(metadataFields); // schema or rows returned by readers - return TypeUtil.join(schema, meta); + return TypeUtil.join(schema, metadataSchema); + } + + private Schema calculateMetadataSchema(List metaColumnFields) { + Optional partitionField = + metaColumnFields.stream() + .filter(f -> MetadataColumns.PARTITION_COLUMN_ID == f.fieldId()) + .findFirst(); + + // only calculate potential column id collision if partition metadata column was requested + if (!partitionField.isPresent()) { + return new Schema(metaColumnFields); + } + + Set idsToReassign = + TypeUtil.indexById(partitionField.get().type().asStructType()).keySet(); + + // Calculate used ids by union metadata columns with all base table schemas + Set currentlyUsedIds = + metaColumnFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + Set allUsedIds = + table.schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Reassign selected ids to deduplicate with used ids. + AtomicInteger nextId = new AtomicInteger(); + return new Schema( + metaColumnFields, + table.schema().identifierFieldIds(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); } @Override 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 0fb0ac2f05df..230a660c0117 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 @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_BATCH_SIZE; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.spark.sql.functions.expr; import static org.apache.spark.sql.functions.lit; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -35,6 +36,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; @@ -60,6 +62,7 @@ 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.types.StructType; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -172,6 +175,50 @@ public void testSpecAndPartitionMetadataColumns() { sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); } + @TestTemplate + public void testPartitionMetadataColumnWithManyColumns() { + List fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema manyColumnsSchema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(manyColumnsSchema).identity("id").build(); + + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit( + base, + base.updateSchema(manyColumnsSchema, manyColumnsSchema.highestFieldId()) + .updatePartitionSpec(spec)); + + Dataset df = + spark + .range(2) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(TABLE_NAME); + + assertThat(spark.table(TABLE_NAME).select("*", "_partition").count()).isEqualTo(2); + List expected = + ImmutableList.of(row(row(0L), 0L, "0", "0", "0"), row(row(1L), 1L, "1", "1", "1")); + assertEquals( + "Rows must match", + expected, + sql("SELECT _partition, id, c999, c1000, c1001 FROM %s ORDER BY id", TABLE_NAME)); + } + @TestTemplate public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { assumeThat(fileFormat).isEqualTo(FileFormat.PARQUET); From 01984d4c45953de4c8d20686bf8437dcfe383510 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 7 Jul 2024 08:10:05 +0200 Subject: [PATCH 0441/1019] Build: Bump com.azure:azure-sdk-bom from 1.2.24 to 1.2.25 (#10652) --- 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 3d11f3a3eafc..1fc889fe160a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -29,7 +29,7 @@ avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" awssdk-bom = "2.26.12" -azuresdk-bom = "1.2.24" +azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" From 2d08d43ae860072ce23f864e909e30210030a3a5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 7 Jul 2024 13:51:56 +0200 Subject: [PATCH 0442/1019] Build: Bump software.amazon.awssdk:bom from 2.26.12 to 2.26.16 (#10650) --- 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 1fc889fe160a..9d7fefbf1d6a 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.0" awaitility = "4.2.1" -awssdk-bom = "2.26.12" +awssdk-bom = "2.26.16" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 23926b99672701e878bc0dcf1d15545370d5f944 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 7 Jul 2024 19:09:17 +0200 Subject: [PATCH 0443/1019] Build: Bump jetty from 9.4.54.v20240208 to 9.4.55.v20240627 (#10654) Bumps `jetty` from 9.4.54.v20240208 to 9.4.55.v20240627. Updates `org.eclipse.jetty:jetty-server` from 9.4.54.v20240208 to 9.4.55.v20240627 Updates `org.eclipse.jetty:jetty-servlet` from 9.4.54.v20240208 to 9.4.55.v20240627 --- updated-dependencies: - dependency-name: org.eclipse.jetty:jetty-server dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.eclipse.jetty:jetty-servlet 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 9d7fefbf1d6a..1738667697cd 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -59,7 +59,7 @@ jackson215 = { strictly = "2.15.2"} jakarta-el-api = "3.0.3" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" -jetty = "9.4.54.v20240208" +jetty = "9.4.55.v20240627" junit = "5.10.1" kafka = "3.7.0" kryo-shaded = "4.0.3" From beb8797b699ce248b44dd67871a8c4d0f361fae2 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 7 Jul 2024 19:35:22 +0200 Subject: [PATCH 0444/1019] Build: Bump kafka from 3.7.0 to 3.7.1 (#10653) Bumps `kafka` from 3.7.0 to 3.7.1. Updates `org.apache.kafka:kafka-clients` from 3.7.0 to 3.7.1 Updates `org.apache.kafka:connect-api` from 3.7.0 to 3.7.1 Updates `org.apache.kafka:connect-json` from 3.7.0 to 3.7.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 1738667697cd..ad609267badc 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -61,7 +61,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "9.4.55.v20240627" junit = "5.10.1" -kafka = "3.7.0" +kafka = "3.7.1" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" From bfc2705b1bdd18ec6289352c3a06f38337c3d402 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Jul 2024 09:19:25 +0200 Subject: [PATCH 0445/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.1.0 to 1.2.0 (#10655) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.1.0 to 1.2.0. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.1.0...1.2.0) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 ad609267badc..5f07fb949278 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -73,7 +73,7 @@ object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.1.0" +roaringbitmap = "1.2.0" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" From 72678beaf7c7111d2c5a337f95ea504b47f44e92 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Jul 2024 13:27:18 +0200 Subject: [PATCH 0446/1019] Build: Bump datamodel-code-generator from 0.25.7 to 0.25.8 (#10649) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.7 to 0.25.8. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.7...0.25.8) --- 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 75c821241f08..f6233ff2ace4 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.25.7 +datamodel-code-generator==0.25.8 From f752e84ad69ee7fab94a18e452016cd0767146ba Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 8 Jul 2024 14:28:01 +0200 Subject: [PATCH 0447/1019] Build: Bump mkdocs-material from 9.5.27 to 9.5.28 (#10648) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.27 to 9.5.28. - [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.27...9.5.28) --- 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 b1300034ee4e..75beac22648f 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.27 +mkdocs-material==9.5.28 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 c83bbac93382cb3d11f876aa52028750622fce7a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 8 Jul 2024 15:12:47 +0200 Subject: [PATCH 0448/1019] Build: Downgrade Gradle from 8.8 to 8.7 due to bug with older OSX versions (#10637) --- gradle/wrapper/gradle-wrapper.properties | 5 +++-- gradlew | 4 ++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/gradle/wrapper/gradle-wrapper.properties b/gradle/wrapper/gradle-wrapper.properties index d296387b57c6..381baa9cef1e 100644 --- a/gradle/wrapper/gradle-wrapper.properties +++ b/gradle/wrapper/gradle-wrapper.properties @@ -1,7 +1,8 @@ distributionBase=GRADLE_USER_HOME distributionPath=wrapper/dists -distributionSha256Sum=a4b4158601f8636cdeeab09bd76afb640030bb5b144aafe261a5e8af027dc612 -distributionUrl=https\://services.gradle.org/distributions/gradle-8.8-bin.zip +distributionSha256Sum=544c35d6bd849ae8a5ed0bcea39ba677dc40f49df7d1835561582da2009b961d +distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip networkTimeout=10000 +validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME zipStorePath=wrapper/dists diff --git a/gradlew b/gradlew index 68bbcf0a5c21..1757b45dfead 100755 --- a/gradlew +++ b/gradlew @@ -84,10 +84,10 @@ 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:-./}" && pwd -P ) || exit +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.8.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.7.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From 21c96c9ef5b6990c6c297fc0db6c3bd9a2067639 Mon Sep 17 00:00:00 2001 From: Attila Kreiner Date: Tue, 9 Jul 2024 16:55:37 +0200 Subject: [PATCH 0449/1019] Data: Switch tests to JUnit5 + AssertJ-style assertions (#10657) --- .../org/apache/iceberg/RecordWrapperTest.java | 5 +- .../apache/iceberg/TestMergingMetrics.java | 56 +- .../org/apache/iceberg/TestSplitScan.java | 57 +- .../apache/iceberg/data/DataTestHelpers.java | 13 +- .../iceberg/data/GenericAppenderHelper.java | 10 +- .../data/TestDataFileIndexStatsFilters.java | 102 ++-- .../iceberg/data/TestGenericRecord.java | 8 +- .../apache/iceberg/data/TestLocalScan.java | 178 +++---- .../data/TestMetricsRowGroupFilter.java | 504 ++++++++++-------- .../data/TestMetricsRowGroupFilterTypes.java | 134 +++-- .../iceberg/data/TestReadProjection.java | 421 ++++++++------- .../data/avro/TestGenericReadProjection.java | 6 +- .../data/avro/TestSingleMessageEncoding.java | 28 +- .../data/orc/TestGenericReadProjection.java | 6 +- .../iceberg/data/orc/TestOrcDataWriter.java | 32 +- .../iceberg/data/orc/TestOrcRowIterator.java | 17 +- .../parquet/TestGenericReadProjection.java | 6 +- .../apache/iceberg/io/TestWriterMetrics.java | 193 +++---- .../parquet/TestGenericMergingMetrics.java | 10 +- .../iceberg/flink/TestRowDataWrapper.java | 4 +- .../flink/source/TestFlinkMergingMetrics.java | 30 +- .../iceberg/flink/TestRowDataWrapper.java | 4 +- .../flink/source/TestFlinkMergingMetrics.java | 30 +- .../iceberg/flink/TestRowDataWrapper.java | 4 +- .../flink/source/TestFlinkMergingMetrics.java | 30 +- .../org/apache/iceberg/mr/TestHelper.java | 25 - .../spark/source/TestSparkMergingMetrics.java | 9 +- .../spark/source/TestSparkMergingMetrics.java | 9 +- .../spark/source/TestSparkMergingMetrics.java | 10 +- 29 files changed, 985 insertions(+), 956 deletions(-) diff --git a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java index 94e9825fa5f3..7e426e11382e 100644 --- a/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java +++ b/data/src/test/java/org/apache/iceberg/RecordWrapperTest.java @@ -20,10 +20,10 @@ 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 org.apache.iceberg.types.Types; import org.apache.iceberg.util.StructLikeWrapper; -import org.junit.Assert; import org.junit.jupiter.api.Test; public abstract class RecordWrapperTest { @@ -103,7 +103,8 @@ public void testNestedSchema() { } private void generateAndValidate(Schema schema) { - generateAndValidate(schema, Assert::assertEquals); + generateAndValidate( + schema, (message, expected, actual) -> assertThat(actual).as(message).isEqualTo(expected)); } public interface AssertMethod { diff --git a/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java b/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java index 0e8464c51275..cf4d67285b06 100644 --- a/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java +++ b/data/src/test/java/org/apache/iceberg/TestMergingMetrics.java @@ -20,8 +20,11 @@ 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.nio.ByteBuffer; +import java.util.Arrays; import java.util.Collection; import java.util.List; import java.util.Map; @@ -41,14 +44,11 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.NaNUtil; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestMergingMetrics { // all supported fields, except for UUID which is on deprecation path: see @@ -110,22 +110,17 @@ public abstract class TestMergingMetrics { MAP_FIELD_2, STRUCT_FIELD); - protected final FileFormat fileFormat; - - @Parameterized.Parameters(name = "fileFormat = {0}") - public static Object[] parameters() { - return new Object[] {FileFormat.PARQUET, FileFormat.ORC}; - } + protected abstract FileAppender writeAndGetAppender(List records) throws Exception; - public TestMergingMetrics(FileFormat fileFormat) { - this.fileFormat = fileFormat; + @Parameters(name = "fileFormat = {0}") + public static List parameters() { + return Arrays.asList(FileFormat.PARQUET, FileFormat.ORC); } - protected abstract FileAppender writeAndGetAppender(List records) throws Exception; - - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @Parameter protected FileFormat fileFormat; + @TempDir protected File tempDir; - @Test + @TestTemplate public void verifySingleRecordMetric() throws Exception { Record record = GenericRecord.create(SCHEMA); record.setField(ID_FIELD.name(), 3); @@ -166,7 +161,7 @@ public void verifySingleRecordMetric() throws Exception { assertBoundValueMatch(0D, lowerBounds, MAP_FIELD_2); } - @Test + @TestTemplate public void verifyRandomlyGeneratedRecordsMetric() throws Exception { // too big of the record count will more likely to make all upper/lower bounds +/-infinity, // which makes the tests easier to pass @@ -192,16 +187,16 @@ public void verifyRandomlyGeneratedRecordsMetric() throws Exception { .map(Types.NestedField::fieldId) .forEach( id -> - Assert.assertNull( - "NaN count for field %s should be null", metrics.nanValueCounts().get(id))); + assertThat(metrics.nanValueCounts().get(id)) + .as("NaN count for field %s should be null") + .isNull()); } private void assertNaNCountMatch( Long expected, Map nanValueCount, Types.NestedField field) { - Assert.assertEquals( - String.format("NaN count for field %s does not match expected", field.name()), - expected, - nanValueCount.get(FIELDS_WITH_NAN_COUNT_TO_ID.get(field))); + assertThat(nanValueCount) + .as(String.format("NaN count for field %s does not match expected", field.name())) + .containsEntry(FIELDS_WITH_NAN_COUNT_TO_ID.get(field), expected); } private void assertBoundValueMatch( @@ -214,10 +209,9 @@ private void assertBoundValueMatch( int actualFieldId = FIELDS_WITH_NAN_COUNT_TO_ID.get(field); ByteBuffer byteBuffer = boundMap.get(actualFieldId); Type type = SCHEMA.findType(actualFieldId); - Assert.assertEquals( - String.format("Bound value for field %s must match", field.name()), - expected, - byteBuffer == null ? null : Conversions.fromByteBuffer(type, byteBuffer)); + assertThat(byteBuffer == null ? null : Conversions.fromByteBuffer(type, byteBuffer)) + .as(String.format("Bound value for field %s must match", field.name())) + .isEqualTo(expected); } private void populateExpectedValues( diff --git a/data/src/test/java/org/apache/iceberg/TestSplitScan.java b/data/src/test/java/org/apache/iceberg/TestSplitScan.java index d75269f23bad..85d0571e3104 100644 --- a/data/src/test/java/org/apache/iceberg/TestSplitScan.java +++ b/data/src/test/java/org/apache/iceberg/TestSplitScan.java @@ -19,9 +19,11 @@ package org.apache.iceberg; 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.util.Arrays; import java.util.List; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.data.GenericAppenderFactory; @@ -32,19 +34,15 @@ 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.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestSplitScan { private static final Configuration CONF = new Configuration(); private static final HadoopTables TABLES = new HadoopTables(CONF); - private static final long SPLIT_SIZE = 16 * 1024 * 1024; private static final Schema SCHEMA = @@ -53,38 +51,31 @@ public class TestSplitScan { private Table table; private File tableLocation; - - @Rule public TemporaryFolder temp = new TemporaryFolder(); private List expectedRecords; - @Parameterized.Parameters(name = "format = {0}") - public static Object[] parameters() { - return new Object[] {"parquet", "avro"}; + @Parameters(name = "fileFormat = {0}") + public static List parameters() { + return Arrays.asList(FileFormat.PARQUET, FileFormat.AVRO); } - private final FileFormat format; + @Parameter private FileFormat format; + @TempDir private File tempDir; - public TestSplitScan(String format) { - this.format = FileFormat.fromString(format); - } - - @Before + @BeforeEach public void before() throws IOException { - tableLocation = new File(temp.newFolder(), "table"); + tableLocation = java.nio.file.Files.createTempDirectory(tempDir.toPath(), "table").toFile(); setupTable(); } - @Test + @TestTemplate public void test() { - Assert.assertEquals( - "There should be 4 tasks created since file size is approximately close to 64MB and split size 16MB", - 4, - Lists.newArrayList(table.newScan().planTasks()).size()); + assertThat(Lists.newArrayList(table.newScan().planTasks())) + .as( + "There should be 4 tasks created since file size is approximately close to 64MB and split size 16MB") + .hasSize(4); + List records = Lists.newArrayList(IcebergGenerics.read(table).build()); - Assert.assertEquals(expectedRecords.size(), records.size()); - for (int i = 0; i < expectedRecords.size(); i++) { - Assert.assertEquals(expectedRecords.get(i), records.get(i)); - } + assertThat(records).isEqualTo(expectedRecords); } private void setupTable() throws IOException { @@ -109,8 +100,8 @@ private void setupTable() throws IOException { } private File writeToFile(List records, FileFormat fileFormat) throws IOException { - File file = temp.newFile(); - Assert.assertTrue(file.delete()); + File file = File.createTempFile("junit", null, tempDir); + assertThat(file.delete()).isTrue(); GenericAppenderFactory factory = new GenericAppenderFactory(SCHEMA) 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 e02ec8ec2cdc..0573897dab72 100644 --- a/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java +++ b/data/src/test/java/org/apache/iceberg/data/DataTestHelpers.java @@ -24,7 +24,6 @@ import java.util.Map; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; -import org.junit.Assert; public class DataTestHelpers { private DataTestHelpers() {} @@ -44,7 +43,7 @@ public static void assertEquals(Types.StructType struct, Record expected, Record public static void assertEquals(Types.ListType list, List expected, List actual) { Type elementType = list.elementType(); - Assert.assertEquals("List size should match", expected.size(), actual.size()); + assertThat(actual).as("List size should match").hasSameSizeAs(expected); for (int i = 0; i < expected.size(); i += 1) { Object expectedValue = expected.get(i); @@ -57,7 +56,7 @@ public static void assertEquals(Types.ListType list, List expected, List a public static void assertEquals(Types.MapType map, Map expected, Map actual) { Type valueType = map.valueType(); - Assert.assertEquals("Map size should match", expected.size(), actual.size()); + assertThat(actual).as("Map size should match").hasSameSizeAs(expected); for (Object expectedKey : expected.keySet()) { Object expectedValue = expected.get(expectedKey); @@ -85,14 +84,14 @@ private static void assertEquals(Type type, Object expected, Object actual) { case UUID: case BINARY: case DECIMAL: - Assert.assertEquals( - "Primitive value should be equal to expected for type " + type, expected, actual); + assertThat(actual) + .as("Primitive value should be equal to expected for type " + type) + .isEqualTo(expected); break; case FIXED: assertThat(expected).as("Expected should be a byte[]").isInstanceOf(byte[].class); assertThat(expected).as("Actual should be a byte[]").isInstanceOf(byte[].class); - Assert.assertArrayEquals( - "Array contents should be equal", (byte[]) expected, (byte[]) actual); + assertThat(actual).as("Array contents should be equal").isEqualTo(expected); break; case STRUCT: assertThat(expected).as("Expected should be a Record").isInstanceOf(Record.class); diff --git a/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java b/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java index 1fe8765a8a68..05f8e0e8e83d 100644 --- a/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java +++ b/data/src/test/java/org/apache/iceberg/data/GenericAppenderHelper.java @@ -45,7 +45,6 @@ public class GenericAppenderHelper { private final Table table; private final FileFormat fileFormat; - private final TemporaryFolder tmp; private final Path temp; private final Configuration conf; @@ -54,19 +53,18 @@ public GenericAppenderHelper( Table table, FileFormat fileFormat, TemporaryFolder tmp, Configuration conf) { this.table = table; this.fileFormat = fileFormat; - this.tmp = tmp; - this.temp = null; + this.temp = tmp.getRoot().toPath(); this.conf = conf; } public GenericAppenderHelper(Table table, FileFormat fileFormat, Path temp, Configuration conf) { this.table = table; this.fileFormat = fileFormat; - this.tmp = null; this.temp = temp; this.conf = conf; } + @Deprecated public GenericAppenderHelper(Table table, FileFormat fileFormat, TemporaryFolder tmp) { this(table, fileFormat, tmp, null); } @@ -111,14 +109,14 @@ public void appendToTable(StructLike partition, List records) throws IOE public DataFile writeFile(List records) throws IOException { Preconditions.checkNotNull(table, "table not set"); - File file = null != tmp ? tmp.newFile() : File.createTempFile("junit", null, temp.toFile()); + File file = File.createTempFile("junit", null, temp.toFile()); assertThat(file.delete()).isTrue(); return appendToLocalFile(table, file, fileFormat, null, records, conf); } public DataFile writeFile(StructLike partition, List records) throws IOException { Preconditions.checkNotNull(table, "table not set"); - File file = null != tmp ? tmp.newFile() : File.createTempFile("junit", null, temp.toFile()); + File file = File.createTempFile("junit", null, temp.toFile()); assertThat(file.delete()).isTrue(); return appendToLocalFile(table, file, fileFormat, partition, records, conf); } 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 7961755cdbfb..0c97d9ecd4da 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java +++ b/data/src/test/java/org/apache/iceberg/data/TestDataFileIndexStatsFilters.java @@ -44,12 +44,10 @@ import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; import org.apache.iceberg.util.Pair; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestDataFileIndexStatsFilters { private static final Schema SCHEMA = @@ -58,7 +56,7 @@ public class TestDataFileIndexStatsFilters { Types.NestedField.optional(2, "data", Types.StringType.get()), Types.NestedField.required(3, "category", Types.StringType.get())); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File tempDir; private Table table; private List records = null; @@ -68,9 +66,10 @@ public class TestDataFileIndexStatsFilters { private DataFile dataFileWithoutNulls = null; private DataFile dataFileOnlyNulls = null; - @Before + @BeforeEach public void createTableAndData() throws IOException { - File location = temp.newFolder(); + File location = java.nio.file.Files.createTempDirectory(tempDir.toPath(), "table").toFile(); + this.table = TestTables.create(location, "test", SCHEMA, PartitionSpec.unpartitioned(), 2); this.records = Lists.newArrayList(); @@ -94,24 +93,24 @@ public void createTableAndData() throws IOException { .filter(rec -> rec.getField("category").equals("even")) .collect(Collectors.toList()); - this.dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + this.dataFile = FileHelpers.writeDataFile(table, Files.localOutput(createTempFile()), records); this.dataFileWithoutNulls = FileHelpers.writeDataFile( table, - Files.localOutput(temp.newFile()), + Files.localOutput(createTempFile()), records.stream() .filter(rec -> rec.getField("data") != null) .collect(Collectors.toList())); this.dataFileOnlyNulls = FileHelpers.writeDataFile( table, - Files.localOutput(temp.newFile()), + Files.localOutput(createTempFile()), records.stream() .filter(rec -> rec.getField("data") == null) .collect(Collectors.toList())); } - @After + @AfterEach public void dropTable() { TestTables.clearTables(); } @@ -125,7 +124,7 @@ public void testPositionDeletePlanningPath() throws IOException { deletes.add(Pair.of(dataFile.path(), 1L)); Pair posDeletes = - FileHelpers.writeDeleteFile(table, Files.localOutput(temp.newFile()), deletes); + FileHelpers.writeDeleteFile(table, Files.localOutput(createTempFile()), deletes); table .newRowDelta() .addDeletes(posDeletes.first()) @@ -137,9 +136,9 @@ public void testPositionDeletePlanningPath() throws IOException { tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals("Should have one delete file, file_path matches", 1, task.deletes().size()); + assertThat(task.deletes()).as("Should have one delete file, file_path matches").hasSize(1); } @Test @@ -151,7 +150,7 @@ public void testPositionDeletePlanningPathFilter() throws IOException { deletes.add(Pair.of("some-other-file.parquet", 1L)); Pair posDeletes = - FileHelpers.writeDeleteFile(table, Files.localOutput(temp.newFile()), deletes); + FileHelpers.writeDeleteFile(table, Files.localOutput(createTempFile()), deletes); table .newRowDelta() .addDeletes(posDeletes.first()) @@ -163,10 +162,11 @@ public void testPositionDeletePlanningPathFilter() throws IOException { tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should not have delete file, filtered by file_path stats", 0, task.deletes().size()); + assertThat(task.deletes()) + .as("Should not have delete file, filtered by file_path stats") + .isEmpty(); } @Test @@ -180,7 +180,7 @@ public void testEqualityDeletePlanningStats() throws IOException { DeleteFile posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(temp.newFile()), deletes, deleteRowSchema); + table, Files.localOutput(createTempFile()), deletes, deleteRowSchema); table.newRowDelta().addDeletes(posDeletes).commit(); @@ -189,10 +189,11 @@ public void testEqualityDeletePlanningStats() throws IOException { tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should have one delete file, data contains a matching value", 1, task.deletes().size()); + assertThat(task.deletes()) + .as("Should have one delete file, data contains a matching value") + .hasSize(1); } @Test @@ -208,7 +209,7 @@ public void testEqualityDeletePlanningStatsFilter() throws IOException { DeleteFile posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(temp.newFile()), deletes, deleteRowSchema); + table, Files.localOutput(createTempFile()), deletes, deleteRowSchema); table.newRowDelta().addDeletes(posDeletes).commit(); @@ -217,10 +218,11 @@ public void testEqualityDeletePlanningStatsFilter() throws IOException { tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should not have delete file, filtered by data column stats", 0, task.deletes().size()); + assertThat(task.deletes()) + .as("Should not have delete file, filtered by data column stats") + .isEmpty(); } @Test @@ -234,7 +236,7 @@ public void testEqualityDeletePlanningStatsNullValueWithAllNullDeletes() throws DeleteFile posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(temp.newFile()), deletes, deleteRowSchema); + table, Files.localOutput(createTempFile()), deletes, deleteRowSchema); table.newRowDelta().addDeletes(posDeletes).commit(); @@ -243,10 +245,9 @@ public void testEqualityDeletePlanningStatsNullValueWithAllNullDeletes() throws tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should have delete file, data contains a null value", 1, task.deletes().size()); + assertThat(task.deletes()).as("Should have delete file, data contains a null value").hasSize(1); } @Test @@ -263,7 +264,7 @@ public void testEqualityDeletePlanningStatsNoNullValuesWithAllNullDeletes() thro DeleteFile posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(temp.newFile()), deletes, deleteRowSchema); + table, Files.localOutput(createTempFile()), deletes, deleteRowSchema); table.newRowDelta().addDeletes(posDeletes).commit(); @@ -272,10 +273,11 @@ public void testEqualityDeletePlanningStatsNoNullValuesWithAllNullDeletes() thro tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should have no delete files, data contains no null values", 0, task.deletes().size()); + assertThat(task.deletes()) + .as("Should have no delete files, data contains no null values") + .isEmpty(); } @Test @@ -292,7 +294,7 @@ public void testEqualityDeletePlanningStatsAllNullValuesWithNoNullDeletes() thro DeleteFile posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(temp.newFile()), deletes, deleteRowSchema); + table, Files.localOutput(createTempFile()), deletes, deleteRowSchema); table.newRowDelta().addDeletes(posDeletes).commit(); @@ -301,10 +303,11 @@ public void testEqualityDeletePlanningStatsAllNullValuesWithNoNullDeletes() thro tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should have no delete files, data contains no null values", 0, task.deletes().size()); + assertThat(task.deletes()) + .as("Should have no delete files, data contains no null values") + .isEmpty(); } @Test @@ -324,7 +327,7 @@ public void testEqualityDeletePlanningStatsSomeNullValuesWithSomeNullDeletes() DeleteFile posDeletes = FileHelpers.writeDeleteFile( - table, Files.localOutput(temp.newFile()), deletes, deleteRowSchema); + table, Files.localOutput(createTempFile()), deletes, deleteRowSchema); table.newRowDelta().addDeletes(posDeletes).commit(); @@ -333,10 +336,11 @@ public void testEqualityDeletePlanningStatsSomeNullValuesWithSomeNullDeletes() tasks = Lists.newArrayList(tasksIterable); } - Assert.assertEquals("Should produce one task", 1, tasks.size()); + assertThat(tasks).as("Should produce one task").hasSize(1); FileScanTask task = tasks.get(0); - Assert.assertEquals( - "Should have one delete file, data and deletes have null values", 1, task.deletes().size()); + assertThat(task.deletes()) + .as("Should have one delete file, data and deletes have null values") + .hasSize(1); } @Test @@ -457,7 +461,7 @@ private List planTasks() throws IOException { } private DataFile writeData(StructLike partition, List data) throws IOException { - return FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), partition, data); + return FileHelpers.writeDataFile(table, Files.localOutput(createTempFile()), partition, data); } private DeleteFile writeEqDeletes(String col, Object... values) throws IOException { @@ -474,13 +478,17 @@ private DeleteFile writeEqDeletes(StructLike partition, String col, Object... va deletes.add(delete.copy(col, value)); } - OutputFile out = Files.localOutput(temp.newFile()); + OutputFile out = Files.localOutput(createTempFile()); return FileHelpers.writeDeleteFile(table, out, partition, deletes, deleteSchema); } private Pair writePosDeletes( StructLike partition, List> deletes) throws IOException { - OutputFile out = Files.localOutput(temp.newFile()); + OutputFile out = Files.localOutput(createTempFile()); return FileHelpers.writeDeleteFile(table, out, partition, deletes); } + + private File createTempFile() throws IOException { + return File.createTempFile("junit", null, tempDir); + } } diff --git a/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java b/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java index dee38b992c08..cf28c20efdc9 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java +++ b/data/src/test/java/org/apache/iceberg/data/TestGenericRecord.java @@ -19,12 +19,12 @@ package org.apache.iceberg.data; 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 org.apache.iceberg.Schema; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestGenericRecord { @@ -35,7 +35,7 @@ public void testGetNullValue() { GenericRecord record = GenericRecord.create(schema); record.set(0, null); - Assert.assertNull(record.get(0, type.typeId().javaClass())); + assertThat(record.get(0, type.typeId().javaClass())).isNull(); } @Test @@ -45,7 +45,7 @@ public void testGetNotNullValue() { GenericRecord record = GenericRecord.create(schema); record.set(0, 10L); - Assert.assertEquals(10L, record.get(0, type.typeId().javaClass())); + assertThat(record.get(0, type.typeId().javaClass())).isEqualTo(10L); } @Test diff --git a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java index 5855d249ff82..e97c12574913 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java +++ b/data/src/test/java/org/apache/iceberg/data/TestLocalScan.java @@ -27,12 +27,15 @@ import static org.apache.iceberg.relocated.com.google.common.collect.Iterables.transform; 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; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.file.Files; +import java.util.Arrays; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -43,6 +46,9 @@ import org.apache.iceberg.DataFiles; import org.apache.iceberg.FileFormat; 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.Schema; import org.apache.iceberg.Table; @@ -60,15 +66,12 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.DateTimeUtil; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestLocalScan { private static final Schema SCHEMA = new Schema( @@ -77,18 +80,13 @@ public class TestLocalScan { private static final Configuration CONF = new Configuration(); private static final Tables TABLES = new HadoopTables(CONF); - @Rule public final TemporaryFolder temp = new TemporaryFolder(); - - @Parameterized.Parameters(name = "format = {0}") - public static Object[] parameters() { - return new Object[] {"parquet", "orc", "avro"}; + @Parameters(name = "fileFormat = {0}") + public static List parameters() { + return Arrays.asList(FileFormat.PARQUET, FileFormat.ORC, FileFormat.AVRO); } - private final FileFormat format; - - public TestLocalScan(String format) { - this.format = FileFormat.fromString(format); - } + @Parameter private FileFormat format; + @TempDir private File tempDir; private String sharedTableLocation = null; private Table sharedTable = null; @@ -195,10 +193,9 @@ private void appendData() throws IOException { sharedTable.newFastAppend().appendFile(file13).appendFile(file23).appendFile(file33).commit(); } - @Before + @BeforeEach public void createTables() throws IOException { - File location = temp.newFolder("shared"); - Assert.assertTrue(location.delete()); + File location = Files.createTempDirectory(tempDir.toPath(), "shared").toFile(); this.sharedTableLocation = location.toString(); this.sharedTable = TABLES.create( @@ -225,12 +222,13 @@ public void createTables() throws IOException { sharedTable.newAppend().appendFile(file1).appendFile(file2).appendFile(file3).commit(); } - @Test + @TestTemplate public void testRandomData() throws IOException { List expected = RandomGenericData.generate(SCHEMA, 1000, 435691832918L); - File location = temp.newFolder(format.name()); - Assert.assertTrue(location.delete()); + File location = Files.createTempDirectory(tempDir.toPath(), "junit").toFile(); + assertThat(location.delete()).isTrue(); + Table table = TABLES.create( SCHEMA, @@ -266,12 +264,11 @@ public void testRandomData() throws IOException { append.commit(); Set records = Sets.newHashSet(IcebergGenerics.read(table).build()); - Assert.assertEquals( - "Should produce correct number of records", expected.size(), records.size()); - Assert.assertEquals("Random record set should match", Sets.newHashSet(expected), records); + assertThat(records).as("Should produce correct number of records").hasSameSizeAs(expected); + assertThat(records).as("Random record set should match").isEqualTo(Sets.newHashSet(expected)); } - @Test + @TestTemplate public void testFullScan() { Iterable results = IcebergGenerics.read(sharedTable).build(); @@ -281,36 +278,33 @@ public void testFullScan() { expected.addAll(file3FirstSnapshotRecords); Set records = Sets.newHashSet(results); - Assert.assertEquals( - "Should produce correct number of records", expected.size(), records.size()); - Assert.assertEquals("Random record set should match", Sets.newHashSet(expected), records); + assertThat(records).as("Should produce correct number of records").hasSameSizeAs(expected); + assertThat(records).as("Random record set should match").isEqualTo(expected); } - @Test + @TestTemplate public void testFilter() { Iterable result = IcebergGenerics.read(sharedTable).where(lessThan("id", 3)).build(); - Assert.assertEquals( - "Records should match file 1", - Sets.newHashSet(file1FirstSnapshotRecords), - Sets.newHashSet(result)); + assertThat(Sets.newHashSet(result)) + .as("Records should match file 1") + .isEqualTo(Sets.newHashSet(file1FirstSnapshotRecords)); result = IcebergGenerics.read(sharedTable).where(lessThan("iD", 3)).caseInsensitive().build(); - Assert.assertEquals( - "Records should match file 1", - Sets.newHashSet(file1FirstSnapshotRecords), - Sets.newHashSet(result)); + assertThat(Sets.newHashSet(result)) + .as("Records should match file 1") + .isEqualTo(Sets.newHashSet(file1FirstSnapshotRecords)); result = IcebergGenerics.read(sharedTable).where(lessThanOrEqual("id", 1)).build(); - Assert.assertEquals( - "Records should match file 1 without id 2", - Sets.newHashSet(filter(file1FirstSnapshotRecords, r -> (Long) r.getField("id") <= 1)), - Sets.newHashSet(result)); + assertThat(Sets.newHashSet(result)) + .as("Records should match file 1 without id 2") + .isEqualTo( + Sets.newHashSet(filter(file1FirstSnapshotRecords, r -> (Long) r.getField("id") <= 1))); } - @Test + @TestTemplate public void testProject() { verifyProjectIdColumn(IcebergGenerics.read(sharedTable).select("id").build()); verifyProjectIdColumn(IcebergGenerics.read(sharedTable).select("iD").caseInsensitive().build()); @@ -326,15 +320,15 @@ private void verifyProjectIdColumn(Iterable results) { Lists.transform(file3FirstSnapshotRecords, record -> (Long) record.getField("id"))); results.forEach( - record -> Assert.assertEquals("Record should have one projected field", 1, record.size())); + record -> + assertThat(record.size()).as("Record should have one projected field").isEqualTo(1)); - Assert.assertEquals( - "Should project only id columns", - expected, - Sets.newHashSet(transform(results, record -> (Long) record.getField("id")))); + assertThat(Sets.newHashSet(transform(results, record -> (Long) record.getField("id")))) + .as("Should project only id columns") + .isEqualTo(expected); } - @Test + @TestTemplate public void testProjectWithSchema() { // Test with table schema Iterable results = IcebergGenerics.read(sharedTable).project(SCHEMA).build(); @@ -345,7 +339,7 @@ public void testProjectWithSchema() { expected.addAll(file3FirstSnapshotRecords); results.forEach(record -> expected.remove(record)); - Assert.assertTrue(expected.isEmpty()); + assertThat(expected).isEmpty(); // Test with projected schema Schema schema = new Schema(required(1, "id", Types.LongType.get())); @@ -356,7 +350,7 @@ public void testProjectWithSchema() { IcebergGenerics.read(sharedTable) .project(schema) .build() - .forEach(r -> Assert.assertNull(r.get(0))); + .forEach(r -> assertThat(r.get(0)).isNull()); // Test with reading some metadata columns schema = @@ -378,11 +372,11 @@ public void testProjectWithSchema() { GenericRecord.create(schema) .copy(ImmutableMap.of("id", 2L, "data", "falafel", "_spec_id", 0, "_pos", 2L)); expectedRecord.setField("_partition", null); - Assert.assertEquals(expectedRecord, iterator.next()); - Assert.assertFalse(iterator.hasNext()); + assertThat(iterator.next()).isEqualTo(expectedRecord); + assertThat(iterator).isExhausted(); } - @Test + @TestTemplate public void testProjectWithMissingFilterColumn() { Iterable results = IcebergGenerics.read(sharedTable) @@ -401,15 +395,15 @@ public void testProjectWithMissingFilterColumn() { } results.forEach( - record -> Assert.assertEquals("Record should have two projected fields", 2, record.size())); + record -> + assertThat(record.size()).as("Record should have two projected fields").isEqualTo(2)); - Assert.assertEquals( - "Should project correct rows", - expected, - Sets.newHashSet(transform(results, record -> record.getField("data").toString()))); + assertThat(Sets.newHashSet(transform(results, record -> record.getField("data").toString()))) + .as("Should project correct rows") + .isEqualTo(expected); } - @Test + @TestTemplate public void testUseSnapshot() throws IOException { overwriteExistingData(); Iterable results = @@ -423,14 +417,13 @@ public void testUseSnapshot() throws IOException { expected.addAll(file3SecondSnapshotRecords); Set records = Sets.newHashSet(results); - Assert.assertEquals( - "Should produce correct number of records", expected.size(), records.size()); - Assert.assertEquals("Record set should match", Sets.newHashSet(expected), records); - Assert.assertNotNull(Iterables.get(records, 0).getField("id")); - Assert.assertNotNull(Iterables.get(records, 0).getField("data")); + assertThat(records).as("Should produce correct number of records").hasSameSizeAs(expected); + assertThat(records).as("Record set should match").isEqualTo(Sets.newHashSet(expected)); + assertThat(Iterables.get(records, 0).getField("id")).isNotNull(); + assertThat(Iterables.get(records, 0).getField("data")).isNotNull(); } - @Test + @TestTemplate public void testAsOfTime() throws IOException { overwriteExistingData(); Iterable results = @@ -444,14 +437,13 @@ public void testAsOfTime() throws IOException { expected.addAll(file3ThirdSnapshotRecords); Set records = Sets.newHashSet(results); - Assert.assertEquals( - "Should produce correct number of records", expected.size(), records.size()); - Assert.assertEquals("Record set should match", Sets.newHashSet(expected), records); - Assert.assertNotNull(Iterables.get(records, 0).getField("id")); - Assert.assertNotNull(Iterables.get(records, 0).getField("data")); + assertThat(records).as("Should produce correct number of records").hasSameSizeAs(expected); + assertThat(records).as("Record set should match").isEqualTo(Sets.newHashSet(expected)); + assertThat(Iterables.get(records, 0).getField("id")).isNotNull(); + assertThat(Iterables.get(records, 0).getField("data")).isNotNull(); } - @Test + @TestTemplate public void testAppendsBetween() throws IOException { appendData(); Iterable results = @@ -467,14 +459,13 @@ public void testAppendsBetween() throws IOException { expected.addAll(file3ThirdSnapshotRecords); Set records = Sets.newHashSet(results); - Assert.assertEquals( - "Should produce correct number of records", expected.size(), records.size()); - Assert.assertEquals("Record set should match", Sets.newHashSet(expected), records); - Assert.assertNotNull(Iterables.get(records, 0).getField("id")); - Assert.assertNotNull(Iterables.get(records, 0).getField("data")); + assertThat(records).as("Should produce correct number of records").hasSameSizeAs(expected); + assertThat(records).as("Record set should match").isEqualTo(Sets.newHashSet(expected)); + assertThat(Iterables.get(records, 0).getField("id")).isNotNull(); + assertThat(Iterables.get(records, 0).getField("data")).isNotNull(); } - @Test + @TestTemplate public void testAppendsAfter() throws IOException { appendData(); Iterable results = @@ -491,14 +482,13 @@ public void testAppendsAfter() throws IOException { expected.addAll(file3ThirdSnapshotRecords); Set records = Sets.newHashSet(results); - Assert.assertEquals( - "Should produce correct number of records", expected.size(), records.size()); - Assert.assertEquals("Record set should match", Sets.newHashSet(expected), records); - Assert.assertNotNull(Iterables.get(records, 0).getField("id")); - Assert.assertNotNull(Iterables.get(records, 0).getField("data")); + assertThat(records).as("Should produce correct number of records").hasSameSizeAs(expected); + assertThat(records).as("Record set should match").isEqualTo(Sets.newHashSet(expected)); + assertThat(Iterables.get(records, 0).getField("id")).isNotNull(); + assertThat(Iterables.get(records, 0).getField("data")).isNotNull(); } - @Test + @TestTemplate public void testUnknownSnapshotId() { Long minSnapshotId = sharedTable.history().stream().map(h -> h.snapshotId()).min(Long::compareTo).get(); @@ -510,7 +500,7 @@ public void testUnknownSnapshotId() { .hasMessage("Cannot find snapshot with ID " + (minSnapshotId - 1)); } - @Test + @TestTemplate public void testAsOfTimeOlderThanFirstSnapshot() { IcebergGenerics.ScanBuilder scanBuilder = IcebergGenerics.read(sharedTable); long timestamp = sharedTable.history().get(0).timestampMillis() - 1; @@ -544,7 +534,7 @@ private DataFile writeFile(String location, String filename, Schema schema, List .build(); } - @Test + @TestTemplate public void testFilterWithDateAndTimestamp() throws IOException { // TODO: Add multiple timestamp tests - there's an issue with ORC caching TZ in ThreadLocal, so // it's not possible @@ -556,8 +546,8 @@ public void testFilterWithDateAndTimestamp() throws IOException { required(3, "date", Types.DateType.get()), required(4, "time", Types.TimeType.get())); - File tableLocation = temp.newFolder("complex_filter_table"); - Assert.assertTrue(tableLocation.delete()); + File tableLocation = Files.createTempDirectory(tempDir.toPath(), "junit").toFile(); + assertThat(tableLocation.delete()).isTrue(); Table table = TABLES.create( @@ -581,10 +571,10 @@ public void testFilterWithDateAndTimestamp() throws IOException { .where(equal("time", r.getField("time").toString())) .build(); - Assert.assertTrue(filterResult.iterator().hasNext()); + assertThat(filterResult.iterator()).hasNext(); Record readRecord = filterResult.iterator().next(); - Assert.assertEquals( - r.getField("timestamp_with_zone"), readRecord.getField("timestamp_with_zone")); + assertThat(readRecord.getField("timestamp_with_zone")) + .isEqualTo(r.getField("timestamp_with_zone")); } } diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java index 4e542b8455c4..574acf15cb9b 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java @@ -41,10 +41,12 @@ 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; import java.io.IOException; import java.io.UncheckedIOException; +import java.util.Arrays; import java.util.List; import java.util.UUID; import org.apache.avro.generic.GenericData.Record; @@ -53,6 +55,9 @@ import org.apache.hadoop.fs.Path; import org.apache.iceberg.FileFormat; 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.AvroSchemaUtil; import org.apache.iceberg.data.orc.GenericOrcReader; @@ -79,28 +84,17 @@ import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.io.DelegatingSeekableInputStream; import org.apache.parquet.schema.MessageType; -import org.assertj.core.api.Assumptions; -import org.junit.Assert; -import org.junit.Assume; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestMetricsRowGroupFilter { - - @Parameterized.Parameters(name = "format = {0}") - public static Object[] parameters() { - return new Object[] {"parquet", "orc"}; - } +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; - private final FileFormat format; +@ExtendWith(ParameterizedTestExtension.class) +public class TestMetricsRowGroupFilter { - public TestMetricsRowGroupFilter(String format) { - this.format = FileFormat.fromString(format); + @Parameters(name = "fileFormat = {0}") + public static List parameters() { + return Arrays.asList(FileFormat.PARQUET, FileFormat.ORC); } private static final Types.StructType structFieldType = @@ -161,9 +155,10 @@ public TestMetricsRowGroupFilter(String format) { private MessageType parquetSchema = null; private BlockMetaData rowGroupMetadata = null; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @Parameter private FileFormat format; + @TempDir private File tempDir; - @Before + @BeforeEach public void createInputFile() throws IOException { switch (format) { case ORC: @@ -179,8 +174,8 @@ public void createInputFile() throws IOException { } public void createOrcInputFile() throws IOException { - this.orcFile = temp.newFile(); - Assert.assertTrue(orcFile.delete()); + this.orcFile = File.createTempFile("junit", null, tempDir); + assertThat(orcFile.delete()).isTrue(); OutputFile outFile = Files.localOutput(orcFile); try (FileAppender appender = @@ -219,15 +214,15 @@ public void createOrcInputFile() throws IOException { try (Reader reader = OrcFile.createReader( new Path(inFile.location()), OrcFile.readerOptions(new Configuration()))) { - Assert.assertEquals("Should create only one stripe", 1, reader.getStripes().size()); + assertThat(reader.getStripes()).as("Should create only one stripe").hasSize(1); } orcFile.deleteOnExit(); } private void createParquetInputFile() throws IOException { - File parquetFile = temp.newFile(); - Assert.assertTrue(parquetFile.delete()); + File parquetFile = File.createTempFile("junit", null, tempDir); + assertThat(parquetFile.delete()).isTrue(); // build struct field schema org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); @@ -263,7 +258,7 @@ private void createParquetInputFile() throws IOException { InputFile inFile = Files.localInput(parquetFile); try (ParquetFileReader reader = ParquetFileReader.open(parquetInputFile(inFile))) { - Assert.assertEquals("Should create only one row group", 1, reader.getRowGroups().size()); + assertThat(reader.getRowGroups()).as("Should create only one row group").hasSize(1); rowGroupMetadata = reader.getRowGroups().get(0); parquetSchema = reader.getFileMetaData().getSchema(); } @@ -271,97 +266,108 @@ private void createParquetInputFile() throws IOException { parquetFile.deleteOnExit(); } - @Test + @TestTemplate public void testAllNulls() { boolean shouldRead; shouldRead = shouldRead(notNull("all_nulls")); - Assert.assertFalse("Should skip: no non-null value in all null column", shouldRead); + assertThat(shouldRead).as("Should skip: no non-null value in all null column").isFalse(); shouldRead = shouldRead(notNull("some_nulls")); - Assert.assertTrue("Should read: column with some nulls contains a non-null value", shouldRead); + assertThat(shouldRead) + .as("Should read: column with some nulls contains a non-null value") + .isTrue(); shouldRead = shouldRead(notNull("no_nulls")); - Assert.assertTrue("Should read: non-null column contains a non-null value", shouldRead); + assertThat(shouldRead).as("Should read: non-null column contains a non-null value").isTrue(); shouldRead = shouldRead(notNull("map_not_null")); - Assert.assertTrue("Should read: map type is not skipped", shouldRead); + assertThat(shouldRead).as("Should read: map type is not skipped").isTrue(); shouldRead = shouldRead(notNull("struct_not_null")); - Assert.assertTrue("Should read: struct type is not skipped", shouldRead); + assertThat(shouldRead).as("Should read: struct type is not skipped").isTrue(); } - @Test + @TestTemplate public void testNoNulls() { boolean shouldRead = shouldRead(isNull("all_nulls")); - Assert.assertTrue("Should read: at least one null value in all null column", shouldRead); + assertThat(shouldRead).as("Should read: at least one null value in all null column").isTrue(); shouldRead = shouldRead(isNull("some_nulls")); - Assert.assertTrue("Should read: column with some nulls contains a null value", shouldRead); + assertThat(shouldRead).as("Should read: column with some nulls contains a null value").isTrue(); shouldRead = shouldRead(isNull("no_nulls")); - Assert.assertFalse("Should skip: non-null column contains no null values", shouldRead); + assertThat(shouldRead).as("Should skip: non-null column contains no null values").isFalse(); shouldRead = shouldRead(isNull("map_not_null")); - Assert.assertTrue("Should read: map type is not skipped", shouldRead); + assertThat(shouldRead).as("Should read: map type is not skipped").isTrue(); shouldRead = shouldRead(isNull("struct_not_null")); - Assert.assertTrue("Should read: struct type is not skipped", shouldRead); + assertThat(shouldRead).as("Should read: struct type is not skipped").isTrue(); } - @Test + @TestTemplate public void testFloatWithNan() { // NaN's should break Parquet's Min/Max stats we should be reading in all cases boolean shouldRead = shouldRead(greaterThan("some_nans", 1.0)); - Assert.assertTrue(shouldRead); + assertThat(shouldRead).isTrue(); shouldRead = shouldRead(greaterThanOrEqual("some_nans", 1.0)); - Assert.assertTrue(shouldRead); + assertThat(shouldRead).isTrue(); shouldRead = shouldRead(lessThan("some_nans", 3.0)); - Assert.assertTrue(shouldRead); + assertThat(shouldRead).isTrue(); shouldRead = shouldRead(lessThanOrEqual("some_nans", 1.0)); - Assert.assertTrue(shouldRead); + assertThat(shouldRead).isTrue(); shouldRead = shouldRead(equal("some_nans", 2.0)); - Assert.assertTrue(shouldRead); + assertThat(shouldRead).isTrue(); } - @Test + @TestTemplate public void testDoubleWithNan() { boolean shouldRead = shouldRead(greaterThan("some_double_nans", 1.0)); - Assert.assertTrue("Should read: column with some nans contains target value", shouldRead); + assertThat(shouldRead).as("Should read: column with some nans contains target value").isTrue(); shouldRead = shouldRead(greaterThanOrEqual("some_double_nans", 1.0)); - Assert.assertTrue("Should read: column with some nans contains the target value", shouldRead); + assertThat(shouldRead) + .as("Should read: column with some nans contains the target value") + .isTrue(); shouldRead = shouldRead(lessThan("some_double_nans", 3.0)); - Assert.assertTrue("Should read: column with some nans contains target value", shouldRead); + assertThat(shouldRead).as("Should read: column with some nans contains target value").isTrue(); shouldRead = shouldRead(lessThanOrEqual("some_double_nans", 1.0)); - Assert.assertTrue("Should read: column with some nans contains target value", shouldRead); + assertThat(shouldRead).as("Should read: column with some nans contains target value").isTrue(); shouldRead = shouldRead(equal("some_double_nans", 2.0)); - Assert.assertTrue("Should read: column with some nans contains target value", shouldRead); + assertThat(shouldRead).as("Should read: column with some nans contains target value").isTrue(); } - @Test + @TestTemplate public void testIsNaN() { boolean shouldRead = shouldRead(isNaN("all_nans")); - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); shouldRead = shouldRead(isNaN("some_nans")); - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); shouldRead = shouldRead(isNaN("no_nans")); switch (format) { case ORC: - Assert.assertFalse( - "Should read 0 rows due to the ORC filter push-down feature", shouldRead); + assertThat(shouldRead) + .as("Should read 0 rows due to the ORC filter push-down feature") + .isFalse(); break; case PARQUET: - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); break; default: throw new UnsupportedOperationException( @@ -369,34 +375,42 @@ public void testIsNaN() { } shouldRead = shouldRead(isNaN("all_nulls")); - Assert.assertFalse("Should skip: all null column will not contain nan value", shouldRead); + assertThat(shouldRead).as("Should skip: all null column will not contain nan value").isFalse(); } - @Test + @TestTemplate public void testNotNaN() { boolean shouldRead = shouldRead(notNaN("all_nans")); - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); shouldRead = shouldRead(notNaN("some_nans")); - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); shouldRead = shouldRead(notNaN("no_nans")); - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); shouldRead = shouldRead(notNaN("all_nulls")); - Assert.assertTrue("Should read: NaN counts are not tracked in Parquet metrics", shouldRead); + assertThat(shouldRead) + .as("Should read: NaN counts are not tracked in Parquet metrics") + .isTrue(); } - @Test + @TestTemplate public void testRequiredColumn() { boolean shouldRead = shouldRead(notNull("required")); - Assert.assertTrue("Should read: required columns are always non-null", shouldRead); + assertThat(shouldRead).as("Should read: required columns are always non-null").isTrue(); shouldRead = shouldRead(isNull("required")); - Assert.assertFalse("Should skip: required columns are always non-null", shouldRead); + assertThat(shouldRead).as("Should skip: required columns are always non-null").isFalse(); } - @Test + @TestTemplate public void testMissingColumn() { assertThatThrownBy(() -> shouldRead(lessThan("missing", 5))) .as("Should complain about missing column in expression") @@ -404,11 +418,13 @@ public void testMissingColumn() { .hasMessageStartingWith("Cannot find field 'missing'"); } - @Test + @TestTemplate public void testColumnNotInFile() { - Assume.assumeFalse( - "If a column is not in file, ORC does NOT try to apply predicates assuming null values for the column", - format == FileFormat.ORC); + assumeThat(format) + .as( + "If a column is not in file, ORC does NOT try to apply predicates assuming null values for the column") + .isNotEqualTo(FileFormat.ORC); + Expression[] cannotMatch = new Expression[] { lessThan("not_in_file", 1.0f), lessThanOrEqual("not_in_file", 1.0f), @@ -418,20 +434,25 @@ public void testColumnNotInFile() { for (Expression expr : cannotMatch) { boolean shouldRead = shouldRead(expr); - Assert.assertFalse("Should skip when column is not in file (all nulls): " + expr, shouldRead); + assertThat(shouldRead) + .as("Should skip when column is not in file (all nulls): " + expr) + .isFalse(); } Expression[] canMatch = new Expression[] {isNull("not_in_file"), notEqual("not_in_file", 1.0f)}; for (Expression expr : canMatch) { boolean shouldRead = shouldRead(expr); - Assert.assertTrue("Should read when column is not in file (all nulls): " + expr, shouldRead); + assertThat(shouldRead) + .as("Should read when column is not in file (all nulls): " + expr) + .isTrue(); } } - @Test + @TestTemplate public void testMissingStatsParquet() { - Assume.assumeTrue(format == FileFormat.PARQUET); + assumeThat(format).isEqualTo(FileFormat.PARQUET); + Expression[] exprs = new Expression[] { lessThan("no_stats_parquet", "a"), @@ -448,13 +469,14 @@ public void testMissingStatsParquet() { for (Expression expr : exprs) { boolean shouldRead = shouldRead(expr); - Assert.assertTrue("Should read when missing stats for expr: " + expr, shouldRead); + assertThat(shouldRead).as("Should read when missing stats for expr: " + expr).isTrue(); } } - @Test + @TestTemplate public void testZeroRecordFileParquet() { - Assume.assumeTrue(format == FileFormat.PARQUET); + assumeThat(format).isEqualTo(FileFormat.PARQUET); + BlockMetaData emptyBlock = new BlockMetaData(); emptyBlock.setRowCount(0); @@ -472,455 +494,469 @@ public void testZeroRecordFileParquet() { for (Expression expr : exprs) { boolean shouldRead = shouldReadParquet(expr, true, parquetSchema, emptyBlock); - Assert.assertFalse("Should never read 0-record file: " + expr, shouldRead); + assertThat(shouldRead).as("Should never read 0-record file: " + expr).isFalse(); } } - @Test + @TestTemplate public void testNot() { // this test case must use a real predicate, not alwaysTrue(), or binding will simplify it out boolean shouldRead = shouldRead(not(lessThan("id", INT_MIN_VALUE - 25))); - Assert.assertTrue("Should read: not(false)", shouldRead); + assertThat(shouldRead).as("Should read: not(false)").isTrue(); shouldRead = shouldRead(not(greaterThan("id", INT_MIN_VALUE - 25))); - Assert.assertFalse("Should skip: not(true)", shouldRead); + assertThat(shouldRead).as("Should skip: not(true)").isFalse(); } - @Test + @TestTemplate public void testAnd() { // this test case must use a real predicate, not alwaysTrue(), or binding will simplify it out boolean shouldRead = shouldRead( and(lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MIN_VALUE - 30))); - Assert.assertFalse("Should skip: and(false, true)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, true)").isFalse(); shouldRead = shouldRead( and(lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MAX_VALUE + 1))); - Assert.assertFalse("Should skip: and(false, false)", shouldRead); + assertThat(shouldRead).as("Should skip: and(false, false)").isFalse(); shouldRead = shouldRead( and(greaterThan("id", INT_MIN_VALUE - 25), lessThanOrEqual("id", INT_MIN_VALUE))); - Assert.assertTrue("Should read: and(true, true)", shouldRead); + assertThat(shouldRead).as("Should read: and(true, true)").isTrue(); } - @Test + @TestTemplate public void testOr() { // this test case must use a real predicate, not alwaysTrue(), or binding will simplify it out boolean shouldRead = shouldRead( or(lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MAX_VALUE + 1))); - Assert.assertFalse("Should skip: or(false, false)", shouldRead); + assertThat(shouldRead).as("Should skip: or(false, false)").isFalse(); shouldRead = shouldRead( or(lessThan("id", INT_MIN_VALUE - 25), greaterThanOrEqual("id", INT_MAX_VALUE - 19))); - Assert.assertTrue("Should read: or(false, true)", shouldRead); + assertThat(shouldRead).as("Should read: or(false, true)").isTrue(); } - @Test + @TestTemplate public void testIntegerLt() { boolean shouldRead = shouldRead(lessThan("id", INT_MIN_VALUE - 25)); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = shouldRead(lessThan("id", INT_MIN_VALUE)); - Assert.assertFalse("Should not read: id range below lower bound (30 is not < 30)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range below lower bound (30 is not < 30)") + .isFalse(); shouldRead = shouldRead(lessThan("id", INT_MIN_VALUE + 1)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(lessThan("id", INT_MAX_VALUE)); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } - @Test + @TestTemplate public void testIntegerLtEq() { boolean shouldRead = shouldRead(lessThanOrEqual("id", INT_MIN_VALUE - 25)); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = shouldRead(lessThanOrEqual("id", INT_MIN_VALUE - 1)); - Assert.assertFalse("Should not read: id range below lower bound (29 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (29 < 30)").isFalse(); shouldRead = shouldRead(lessThanOrEqual("id", INT_MIN_VALUE)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(lessThanOrEqual("id", INT_MAX_VALUE)); - Assert.assertTrue("Should read: many possible ids", shouldRead); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } - @Test + @TestTemplate public void testIntegerGt() { boolean shouldRead = shouldRead(greaterThan("id", INT_MAX_VALUE + 6)); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = shouldRead(greaterThan("id", INT_MAX_VALUE)); - Assert.assertFalse("Should not read: id range above upper bound (79 is not > 79)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range above upper bound (79 is not > 79)") + .isFalse(); shouldRead = shouldRead(greaterThan("id", INT_MAX_VALUE - 1)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThan("id", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } - @Test + @TestTemplate public void testIntegerGtEq() { boolean shouldRead = shouldRead(greaterThanOrEqual("id", INT_MAX_VALUE + 6)); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = shouldRead(greaterThanOrEqual("id", INT_MAX_VALUE + 1)); - Assert.assertFalse("Should not read: id range above upper bound (80 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (80 > 79)").isFalse(); shouldRead = shouldRead(greaterThanOrEqual("id", INT_MAX_VALUE)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThanOrEqual("id", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } - @Test + @TestTemplate public void testIntegerEq() { boolean shouldRead = shouldRead(equal("id", INT_MIN_VALUE - 25)); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = shouldRead(equal("id", INT_MIN_VALUE - 1)); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = shouldRead(equal("id", INT_MIN_VALUE)); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = shouldRead(equal("id", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = shouldRead(equal("id", INT_MAX_VALUE)); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = shouldRead(equal("id", INT_MAX_VALUE + 1)); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); shouldRead = shouldRead(equal("id", INT_MAX_VALUE + 6)); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); } - @Test + @TestTemplate public void testIntegerNotEq() { boolean shouldRead = shouldRead(notEqual("id", INT_MIN_VALUE - 25)); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MIN_VALUE - 1)); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MIN_VALUE)); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MAX_VALUE)); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MAX_VALUE + 1)); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MAX_VALUE + 6)); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); } - @Test + @TestTemplate public void testIntegerNotEqRewritten() { boolean shouldRead = shouldRead(not(equal("id", INT_MIN_VALUE - 25))); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = shouldRead(not(equal("id", INT_MIN_VALUE - 1))); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = shouldRead(not(equal("id", INT_MIN_VALUE))); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = shouldRead(not(equal("id", INT_MAX_VALUE - 4))); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = shouldRead(not(equal("id", INT_MAX_VALUE))); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = shouldRead(not(equal("id", INT_MAX_VALUE + 1))); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); shouldRead = shouldRead(not(equal("id", INT_MAX_VALUE + 6))); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); } - @Test + @TestTemplate public void testStructFieldLt() { boolean shouldRead = shouldRead(lessThan("struct_not_null.int_field", INT_MIN_VALUE - 25)); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = shouldRead(lessThan("struct_not_null.int_field", INT_MIN_VALUE)); - Assert.assertFalse("Should not read: id range below lower bound (30 is not < 30)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range below lower bound (30 is not < 30)") + .isFalse(); shouldRead = shouldRead(lessThan("struct_not_null.int_field", INT_MIN_VALUE + 1)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(lessThan("struct_not_null.int_field", INT_MAX_VALUE)); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } - @Test + @TestTemplate public void testStructFieldLtEq() { boolean shouldRead = shouldRead(lessThanOrEqual("struct_not_null.int_field", INT_MIN_VALUE - 25)); - Assert.assertFalse("Should not read: id range below lower bound (5 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (5 < 30)").isFalse(); shouldRead = shouldRead(lessThanOrEqual("struct_not_null.int_field", INT_MIN_VALUE - 1)); - Assert.assertFalse("Should not read: id range below lower bound (29 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id range below lower bound (29 < 30)").isFalse(); shouldRead = shouldRead(lessThanOrEqual("struct_not_null.int_field", INT_MIN_VALUE)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(lessThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE)); - Assert.assertTrue("Should read: many possible ids", shouldRead); + assertThat(shouldRead).as("Should read: many possible ids").isTrue(); } - @Test + @TestTemplate public void testStructFieldGt() { boolean shouldRead = shouldRead(greaterThan("struct_not_null.int_field", INT_MAX_VALUE + 6)); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = shouldRead(greaterThan("struct_not_null.int_field", INT_MAX_VALUE)); - Assert.assertFalse("Should not read: id range above upper bound (79 is not > 79)", shouldRead); + assertThat(shouldRead) + .as("Should not read: id range above upper bound (79 is not > 79)") + .isFalse(); shouldRead = shouldRead(greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 1)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThan("struct_not_null.int_field", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } - @Test + @TestTemplate public void testStructFieldGtEq() { boolean shouldRead = shouldRead(greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE + 6)); - Assert.assertFalse("Should not read: id range above upper bound (85 < 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (85 < 79)").isFalse(); shouldRead = shouldRead(greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE + 1)); - Assert.assertFalse("Should not read: id range above upper bound (80 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id range above upper bound (80 > 79)").isFalse(); shouldRead = shouldRead(greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE)); - Assert.assertTrue("Should read: one possible id", shouldRead); + assertThat(shouldRead).as("Should read: one possible id").isTrue(); shouldRead = shouldRead(greaterThanOrEqual("struct_not_null.int_field", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: may possible ids", shouldRead); + assertThat(shouldRead).as("Should read: may possible ids").isTrue(); } - @Test + @TestTemplate public void testStructFieldEq() { boolean shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MIN_VALUE - 25)); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MIN_VALUE - 1)); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MIN_VALUE)); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MAX_VALUE)); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MAX_VALUE + 1)); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); shouldRead = shouldRead(equal("struct_not_null.int_field", INT_MAX_VALUE + 6)); - Assert.assertFalse("Should not read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound").isFalse(); } - @Test + @TestTemplate public void testStructFieldNotEq() { boolean shouldRead = shouldRead(notEqual("struct_not_null.int_field", INT_MIN_VALUE - 25)); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = shouldRead(notEqual("struct_not_null.int_field", INT_MIN_VALUE - 1)); - Assert.assertTrue("Should read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound").isTrue(); shouldRead = shouldRead(notEqual("struct_not_null.int_field", INT_MIN_VALUE)); - Assert.assertTrue("Should read: id equal to lower bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound").isTrue(); shouldRead = shouldRead(notEqual("struct_not_null.int_field", INT_MAX_VALUE - 4)); - Assert.assertTrue("Should read: id between lower and upper bounds", shouldRead); + assertThat(shouldRead).as("Should read: id between lower and upper bounds").isTrue(); shouldRead = shouldRead(notEqual("struct_not_null.int_field", INT_MAX_VALUE)); - Assert.assertTrue("Should read: id equal to upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound").isTrue(); shouldRead = shouldRead(notEqual("id", INT_MAX_VALUE + 1)); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); shouldRead = shouldRead(notEqual("struct_not_null.int_field", INT_MAX_VALUE + 6)); - Assert.assertTrue("Should read: id above upper bound", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound").isTrue(); } - @Test + @TestTemplate public void testCaseInsensitive() { boolean shouldRead = shouldRead(equal("ID", INT_MIN_VALUE - 25), false); - Assert.assertFalse("Should not read: id below lower bound", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound").isFalse(); } - @Test + @TestTemplate public void testStringStartsWith() { - Assume.assumeFalse( - "ORC row group filter does not support StringStartsWith", format == FileFormat.ORC); + assumeThat(format) + .as("ORC row group filter does not support StringStartsWith") + .isNotEqualTo(FileFormat.ORC); + boolean shouldRead = shouldRead(startsWith("str", "1")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(startsWith("str", "0st")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(startsWith("str", "1str1")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(startsWith("str", "1str1_xgd")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(startsWith("str", "2str")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(startsWith("str", "9xstr")); - Assert.assertFalse("Should not read: range doesn't match", shouldRead); + assertThat(shouldRead).as("Should not read: range doesn't match").isFalse(); shouldRead = shouldRead(startsWith("str", "0S")); - Assert.assertFalse("Should not read: range doesn't match", shouldRead); + assertThat(shouldRead).as("Should not read: range doesn't match").isFalse(); shouldRead = shouldRead(startsWith("str", "x")); - Assert.assertFalse("Should not read: range doesn't match", shouldRead); + assertThat(shouldRead).as("Should not read: range doesn't match").isFalse(); shouldRead = shouldRead(startsWith("str", "9str9aaa")); - Assert.assertFalse("Should not read: range doesn't match", shouldRead); + assertThat(shouldRead).as("Should not read: range doesn't match").isFalse(); } - @Test + @TestTemplate public void testStringNotStartsWith() { - Assume.assumeFalse( - "ORC row group filter does not support StringStartsWith", format == FileFormat.ORC); + assumeThat(format) + .as("ORC row group filter does not support StringStartsWith") + .isNotEqualTo(FileFormat.ORC); + boolean shouldRead = shouldRead(notStartsWith("str", "1")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("str", "0st")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("str", "1str1")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("str", "1str1_xgd")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("str", "2str")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("str", "9xstr")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("required", "r")); - Assert.assertFalse("Should not read: range doesn't match", shouldRead); + assertThat(shouldRead).as("Should not read: range doesn't match").isFalse(); shouldRead = shouldRead(notStartsWith("required", "requ")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("some_nulls", "ssome")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); shouldRead = shouldRead(notStartsWith("some_nulls", "som")); - Assert.assertTrue("Should read: range matches", shouldRead); + assertThat(shouldRead).as("Should read: range matches").isTrue(); } - @Test + @TestTemplate public void testIntegerIn() { boolean shouldRead = shouldRead(in("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)); - Assert.assertFalse("Should not read: id below lower bound (5 < 30, 6 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound (5 < 30, 6 < 30)").isFalse(); shouldRead = shouldRead(in("id", INT_MIN_VALUE - 2, INT_MIN_VALUE - 1)); - Assert.assertFalse("Should not read: id below lower bound (28 < 30, 29 < 30)", shouldRead); + assertThat(shouldRead).as("Should not read: id below lower bound (28 < 30, 29 < 30)").isFalse(); shouldRead = shouldRead(in("id", INT_MIN_VALUE - 1, INT_MIN_VALUE)); - Assert.assertTrue("Should read: id equal to lower bound (30 == 30)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound (30 == 30)").isTrue(); shouldRead = shouldRead(in("id", INT_MAX_VALUE - 4, INT_MAX_VALUE - 3)); - Assert.assertTrue( - "Should read: id between lower and upper bounds (30 < 75 < 79, 30 < 76 < 79)", shouldRead); + assertThat(shouldRead) + .as("Should read: id between lower and upper bounds (30 < 75 < 79, 30 < 76 < 79)") + .isTrue(); shouldRead = shouldRead(in("id", INT_MAX_VALUE, INT_MAX_VALUE + 1)); - Assert.assertTrue("Should read: id equal to upper bound (79 == 79)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound (79 == 79)").isTrue(); shouldRead = shouldRead(in("id", INT_MAX_VALUE + 1, INT_MAX_VALUE + 2)); - Assert.assertFalse("Should not read: id above upper bound (80 > 79, 81 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound (80 > 79, 81 > 79)").isFalse(); shouldRead = shouldRead(in("id", INT_MAX_VALUE + 6, INT_MAX_VALUE + 7)); - Assert.assertFalse("Should not read: id above upper bound (85 > 79, 86 > 79)", shouldRead); + assertThat(shouldRead).as("Should not read: id above upper bound (85 > 79, 86 > 79)").isFalse(); shouldRead = shouldRead(in("all_nulls", 1, 2)); - Assert.assertFalse("Should skip: in on all nulls column", shouldRead); + assertThat(shouldRead).as("Should skip: in on all nulls column").isFalse(); shouldRead = shouldRead(in("some_nulls", "aaa", "some")); - Assert.assertTrue("Should read: in on some nulls column", shouldRead); + assertThat(shouldRead).as("Should read: in on some nulls column").isTrue(); shouldRead = shouldRead(in("no_nulls", "aaa", "")); - Assert.assertTrue("Should read: in on no nulls column", shouldRead); + assertThat(shouldRead).as("Should read: in on no nulls column").isTrue(); } - @Test + @TestTemplate public void testIntegerNotIn() { boolean shouldRead = shouldRead(notIn("id", INT_MIN_VALUE - 25, INT_MIN_VALUE - 24)); - Assert.assertTrue("Should read: id below lower bound (5 < 30, 6 < 30)", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound (5 < 30, 6 < 30)").isTrue(); shouldRead = shouldRead(notIn("id", INT_MIN_VALUE - 2, INT_MIN_VALUE - 1)); - Assert.assertTrue("Should read: id below lower bound (28 < 30, 29 < 30)", shouldRead); + assertThat(shouldRead).as("Should read: id below lower bound (28 < 30, 29 < 30)").isTrue(); shouldRead = shouldRead(notIn("id", INT_MIN_VALUE - 1, INT_MIN_VALUE)); - Assert.assertTrue("Should read: id equal to lower bound (30 == 30)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to lower bound (30 == 30)").isTrue(); shouldRead = shouldRead(notIn("id", INT_MAX_VALUE - 4, INT_MAX_VALUE - 3)); - Assert.assertTrue( - "Should read: id between lower and upper bounds (30 < 75 < 79, 30 < 76 < 79)", shouldRead); + assertThat(shouldRead) + .as("Should read: id between lower and upper bounds (30 < 75 < 79, 30 < 76 < 79)") + .isTrue(); shouldRead = shouldRead(notIn("id", INT_MAX_VALUE, INT_MAX_VALUE + 1)); - Assert.assertTrue("Should read: id equal to upper bound (79 == 79)", shouldRead); + assertThat(shouldRead).as("Should read: id equal to upper bound (79 == 79)").isTrue(); shouldRead = shouldRead(notIn("id", INT_MAX_VALUE + 1, INT_MAX_VALUE + 2)); - Assert.assertTrue("Should read: id above upper bound (80 > 79, 81 > 79)", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound (80 > 79, 81 > 79)").isTrue(); shouldRead = shouldRead(notIn("id", INT_MAX_VALUE + 6, INT_MAX_VALUE + 7)); - Assert.assertTrue("Should read: id above upper bound (85 > 79, 86 > 79)", shouldRead); + assertThat(shouldRead).as("Should read: id above upper bound (85 > 79, 86 > 79)").isTrue(); shouldRead = shouldRead(notIn("all_nulls", 1, 2)); - Assert.assertTrue("Should read: notIn on all nulls column", shouldRead); + assertThat(shouldRead).as("Should read: notIn on all nulls column").isTrue(); shouldRead = shouldRead(notIn("some_nulls", "aaa", "some")); - Assert.assertTrue("Should read: notIn on some nulls column", shouldRead); + assertThat(shouldRead).as("Should read: notIn on some nulls column").isTrue(); shouldRead = shouldRead(notIn("no_nulls", "aaa", "")); if (format == FileFormat.PARQUET) { // no_nulls column has all values == "", so notIn("no_nulls", "") should always be false and // so should be skipped // However, the metrics evaluator in Parquets always reads row group for a notIn filter - Assert.assertTrue("Should read: notIn on no nulls column", shouldRead); + assertThat(shouldRead).as("Should read: notIn on no nulls column").isTrue(); } else { - Assert.assertFalse("Should skip: notIn on no nulls column", shouldRead); + assertThat(shouldRead).as("Should skip: notIn on no nulls column").isFalse(); } } - @Test + @TestTemplate public void testSomeNullsNotEq() { boolean shouldRead = shouldRead(notEqual("some_nulls", "some")); - Assert.assertTrue("Should read: notEqual on some nulls column", shouldRead); + assertThat(shouldRead).as("Should read: notEqual on some nulls column").isTrue(); } - @Test + @TestTemplate public void testInLimitParquet() { - Assume.assumeTrue(format == FileFormat.PARQUET); + assumeThat(format).isEqualTo(FileFormat.PARQUET); boolean shouldRead = shouldRead(in("id", 1, 2)); - Assert.assertFalse("Should not read if IN is evaluated", shouldRead); + assertThat(shouldRead).as("Should not read if IN is evaluated").isFalse(); List ids = Lists.newArrayListWithExpectedSize(400); for (int id = -400; id <= 0; id++) { @@ -928,22 +964,24 @@ public void testInLimitParquet() { } shouldRead = shouldRead(in("id", ids)); - Assert.assertTrue("Should read if IN is not evaluated", shouldRead); + assertThat(shouldRead).as("Should read if IN is not evaluated").isTrue(); } - @Test + @TestTemplate public void testParquetTypePromotion() { - Assume.assumeTrue("Only valid for Parquet", format == FileFormat.PARQUET); + assumeThat(format).as("Only valid for Parquet").isEqualTo(FileFormat.PARQUET); + Schema promotedSchema = new Schema(required(1, "id", Types.LongType.get())); boolean shouldRead = new ParquetMetricsRowGroupFilter(promotedSchema, equal("id", INT_MIN_VALUE + 1), true) .shouldRead(parquetSchema, rowGroupMetadata); - Assert.assertTrue("Should succeed with promoted schema", shouldRead); + assertThat(shouldRead).as("Should succeed with promoted schema").isTrue(); } - @Test + @TestTemplate public void testTransformFilter() { - Assumptions.assumeThat(format).isEqualTo(FileFormat.PARQUET); + assumeThat(format).isEqualTo(FileFormat.PARQUET); + boolean shouldRead = new ParquetMetricsRowGroupFilter(SCHEMA, equal(truncate("required", 2), "some_value"), true) .shouldRead(parquetSchema, rowGroupMetadata); diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java index bb275f3adc2e..75b19554ef2a 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java @@ -20,6 +20,7 @@ import static org.apache.iceberg.expressions.Expressions.equal; import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; @@ -38,6 +39,9 @@ import org.apache.hadoop.fs.Path; import org.apache.iceberg.FileFormat; 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.data.orc.GenericOrcReader; import org.apache.iceberg.data.orc.GenericOrcWriter; @@ -71,13 +75,11 @@ import org.apache.parquet.hadoop.metadata.BlockMetaData; import org.apache.parquet.io.DelegatingSeekableInputStream; import org.apache.parquet.schema.MessageType; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; -@RunWith(Parameterized.class) +@ExtendWith(ParameterizedTestExtension.class) public class TestMetricsRowGroupFilterTypes { private static final Schema SCHEMA = new Schema( @@ -135,7 +137,7 @@ public class TestMetricsRowGroupFilterTypes { LocalDateTime.parse("2018-06-29T10:02:34.000000", DateTimeFormatter.ISO_LOCAL_DATE_TIME); private static final byte[] fixed = "abcd".getBytes(StandardCharsets.UTF_8); - @Before + @BeforeEach public void createInputFile() throws IOException { List records = Lists.newArrayList(); // create 50 records @@ -176,7 +178,7 @@ public void createInputFile() throws IOException { public void createOrcInputFile(List records) throws IOException { if (ORC_FILE.exists()) { - Assert.assertTrue(ORC_FILE.delete()); + assertThat(ORC_FILE.delete()).isTrue(); } OutputFile outFile = Files.localOutput(ORC_FILE); @@ -192,7 +194,7 @@ public void createOrcInputFile(List records) throws IOException { try (Reader reader = OrcFile.createReader( new Path(inFile.location()), OrcFile.readerOptions(new Configuration()))) { - Assert.assertEquals("Should create only one stripe", 1, reader.getStripes().size()); + assertThat(reader.getStripes()).as("Should create only one stripe").hasSize(1); } ORC_FILE.deleteOnExit(); @@ -200,7 +202,7 @@ public void createOrcInputFile(List records) throws IOException { public void createParquetInputFile(List records) throws IOException { if (PARQUET_FILE.exists()) { - Assert.assertTrue(PARQUET_FILE.delete()); + assertThat(PARQUET_FILE.delete()).isTrue(); } OutputFile outFile = Files.localOutput(PARQUET_FILE); @@ -214,7 +216,7 @@ public void createParquetInputFile(List records) throws IOException { InputFile inFile = Files.localInput(PARQUET_FILE); try (ParquetFileReader reader = ParquetFileReader.open(parquetInputFile(inFile))) { - Assert.assertEquals("Should create only one row group", 1, reader.getRowGroups().size()); + assertThat(reader.getRowGroups()).as("Should create only one row group").hasSize(1); rowGroupMetadata = reader.getRowGroups().get(0); parquetSchema = reader.getFileMetaData().getSchema(); } @@ -222,73 +224,87 @@ public void createParquetInputFile(List records) throws IOException { PARQUET_FILE.deleteOnExit(); } - private final FileFormat format; - private final String column; - private final Object readValue; - private final Object skipValue; + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "format = {0} column = {1} readValue = {2} skipValue = {3}") + @Parameter(index = 1) + private String column; + + @Parameter(index = 2) + private Object readValue; + + @Parameter(index = 3) + private Object skipValue; + + @Parameters(name = "format = {0}, column = {1}, readValue = {2}, skipValue = {3}") public static Object[][] parameters() { return new Object[][] { - {"parquet", "boolean", false, true}, - {"parquet", "int", 5, 55}, - {"parquet", "long", 5_000_000_049L, 5_000L}, - {"parquet", "float", 1.97f, 2.11f}, - {"parquet", "double", 2.11d, 1.97d}, - {"parquet", "date", "2018-06-29", "2018-05-03"}, - {"parquet", "time", "10:02:34.000000", "10:02:34.000001"}, - {"parquet", "timestamp", "2018-06-29T10:02:34.000000", "2018-06-29T15:02:34.000000"}, + {FileFormat.PARQUET, "boolean", false, true}, + {FileFormat.PARQUET, "int", 5, 55}, + {FileFormat.PARQUET, "long", 5_000_000_049L, 5_000L}, + {FileFormat.PARQUET, "float", 1.97f, 2.11f}, + {FileFormat.PARQUET, "double", 2.11d, 1.97d}, + {FileFormat.PARQUET, "date", "2018-06-29", "2018-05-03"}, + {FileFormat.PARQUET, "time", "10:02:34.000000", "10:02:34.000001"}, + {FileFormat.PARQUET, "timestamp", "2018-06-29T10:02:34.000000", "2018-06-29T15:02:34.000000"}, { - "parquet", + FileFormat.PARQUET, "timestamptz", "2018-06-29T10:02:34.000000+00:00", "2018-06-29T10:02:34.000000-07:00" }, - {"parquet", "string", "tapir", "monthly"}, - // { "parquet", "uuid", uuid, UUID.randomUUID() }, // not supported yet - {"parquet", "fixed", "abcd".getBytes(StandardCharsets.UTF_8), new byte[] {0, 1, 2, 3}}, - {"parquet", "binary", "xyz".getBytes(StandardCharsets.UTF_8), new byte[] {0, 1, 2, 3, 4, 5}}, - {"parquet", "int_decimal", "77.77", "12.34"}, - {"parquet", "long_decimal", "88.88", "12.34"}, - {"parquet", "fixed_decimal", "99.99", "12.34"}, - {"orc", "boolean", false, true}, - {"orc", "int", 5, 55}, - {"orc", "long", 5_000_000_049L, 5_000L}, - {"orc", "float", 1.97f, 2.11f}, - {"orc", "double", 2.11d, 1.97d}, - {"orc", "date", "2018-06-29", "2018-05-03"}, - {"orc", "time", "10:02:34.000000", "10:02:34.000001"}, - {"orc", "timestamp", "2018-06-29T10:02:34.000000", "2018-06-29T15:02:34.000000"}, + {FileFormat.PARQUET, "string", "tapir", "monthly"}, + // { FileFormat.PARQUET, "uuid", uuid, UUID.randomUUID() }, // not supported yet + { + FileFormat.PARQUET, + "fixed", + "abcd".getBytes(StandardCharsets.UTF_8), + new byte[] {0, 1, 2, 3} + }, + { + FileFormat.PARQUET, + "binary", + "xyz".getBytes(StandardCharsets.UTF_8), + new byte[] {0, 1, 2, 3, 4, 5} + }, + {FileFormat.PARQUET, "int_decimal", "77.77", "12.34"}, + {FileFormat.PARQUET, "long_decimal", "88.88", "12.34"}, + {FileFormat.PARQUET, "fixed_decimal", "99.99", "12.34"}, + {FileFormat.ORC, "boolean", false, true}, + {FileFormat.ORC, "int", 5, 55}, + {FileFormat.ORC, "long", 5_000_000_049L, 5_000L}, + {FileFormat.ORC, "float", 1.97f, 2.11f}, + {FileFormat.ORC, "double", 2.11d, 1.97d}, + {FileFormat.ORC, "date", "2018-06-29", "2018-05-03"}, + {FileFormat.ORC, "time", "10:02:34.000000", "10:02:34.000001"}, + {FileFormat.ORC, "timestamp", "2018-06-29T10:02:34.000000", "2018-06-29T15:02:34.000000"}, { - "orc", "timestamptz", "2018-06-29T10:02:34.000000+00:00", "2018-06-29T10:02:34.000000-07:00" + FileFormat.ORC, + "timestamptz", + "2018-06-29T10:02:34.000000+00:00", + "2018-06-29T10:02:34.000000-07:00" }, - {"orc", "string", "tapir", "monthly"}, + {FileFormat.ORC, "string", "tapir", "monthly"}, // uuid, fixed and binary types not supported yet - // { "orc", "uuid", uuid, UUID.randomUUID() }, - // { "orc", "fixed", "abcd".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, 2, 3 } }, - // { "orc", "binary", "xyz".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, 2, 3, 4, 5 } + // { FileFormat.ORC, "uuid", uuid, UUID.randomUUID() }, + // { FileFormat.ORC, "fixed", "abcd".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, + // 2, 3 } }, + // { FileFormat.ORC, "binary", "xyz".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, + // 2, 3, 4, 5 } // }, - {"orc", "int_decimal", "77.77", "12.34"}, - {"orc", "long_decimal", "88.88", "12.34"}, - {"orc", "fixed_decimal", "99.99", "12.34"}, + {FileFormat.ORC, "int_decimal", "77.77", "12.34"}, + {FileFormat.ORC, "long_decimal", "88.88", "12.34"}, + {FileFormat.ORC, "fixed_decimal", "99.99", "12.34"}, }; } - public TestMetricsRowGroupFilterTypes( - String format, String column, Object readValue, Object skipValue) { - this.format = FileFormat.fromString(format); - this.column = column; - this.readValue = readValue; - this.skipValue = skipValue; - } - - @Test + @TestTemplate public void testEq() { boolean shouldRead = shouldRead(readValue); - Assert.assertTrue("Should read: value is in the row group: " + readValue, shouldRead); + assertThat(shouldRead).as("Should read: value is in the row group: " + readValue).isTrue(); shouldRead = shouldRead(skipValue); - Assert.assertFalse("Should skip: value is not in the row group: " + skipValue, shouldRead); + assertThat(shouldRead).as("Should skip: value is not in the row group: " + skipValue).isFalse(); } private boolean shouldRead(Object value) { diff --git a/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java index 374a97be0384..2f0e17c7907d 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/TestReadProjection.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.data; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assertions.within; +import java.io.File; import java.io.IOException; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -29,19 +32,15 @@ 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.types.Comparators; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class TestReadProjection { protected abstract Record writeAndRead( String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir protected File tempDir; @Test public void testFullProjection() throws Exception { @@ -56,12 +55,13 @@ public void testFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, schema, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); - int cmp = - Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - Assert.assertTrue("Should contain the correct data value", cmp == 0); + assertThat((CharSequence) projected.getField("data")) + .as("Should contain the correct data value") + .isEqualTo("test"); } @Test @@ -77,20 +77,19 @@ public void testSpecialCharacterProjection() throws Exception { Record full = writeAndRead("special_chars", schema, schema, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) full.getField("user id")); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", (CharSequence) full.getField("data%0"))); + assertThat((long) full.getField("user id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat((CharSequence) full.getField("data%0")) + .as("Should contain the correct data value") + .isEqualTo("test"); Record projected = writeAndRead("special_characters", schema, schema.select("data%0"), record); - Assert.assertNull("Should not contain id value", projected.getField("user id")); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", (CharSequence) projected.getField("data%0"))); + assertThat(projected.getField("user id")).as("Should not contain id value").isNull(); + assertThat(((CharSequence) projected.getField("data%0"))) + .as("Should contain the correct data value") + .isEqualTo("test"); } @Test @@ -111,8 +110,10 @@ public void testReorderedFullProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, reordered, record); - Assert.assertEquals("Should contain the correct 0 value", "test", projected.get(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.get(1)); + assertThat(projected.get(0).toString()) + .as("Should contain the correct 0 value") + .isEqualTo("test"); + assertThat(projected.get(1)).as("Should contain the correct 1 value").isEqualTo(34L); } @Test @@ -134,9 +135,11 @@ public void testReorderedProjection() throws Exception { Record projected = writeAndRead("full_projection", schema, reordered, record); - Assert.assertNull("Should contain the correct 0 value", projected.get(0)); - Assert.assertEquals("Should contain the correct 1 value", "test", projected.get(1).toString()); - Assert.assertNull("Should contain the correct 2 value", projected.get(2)); + assertThat(projected.get(0)).as("Should contain the correct 0 value").isNull(); + assertThat(projected.get(1).toString()) + .as("Should contain the correct 1 value") + .isEqualTo("test"); + assertThat(projected.get(2)).as("Should contain the correct 2 value").isNull(); } @Test @@ -161,17 +164,20 @@ public void testRenamedAddedField() throws Exception { Record projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, record); - Assert.assertEquals("Should contain the correct value in column 1", projected.get(0), 100L); - Assert.assertEquals( - "Should contain the correct value in column a", projected.getField("a"), 100L); - Assert.assertEquals("Should contain the correct value in column 2", projected.get(1), 200L); - Assert.assertEquals( - "Should contain the correct value in column b", projected.getField("b"), 200L); - Assert.assertEquals("Should contain the correct value in column 3", projected.get(2), 300L); - Assert.assertEquals( - "Should contain the correct value in column c", projected.getField("c"), 300L); - Assert.assertNull("Should contain empty value on new column 4", projected.get(3)); - Assert.assertNull("Should contain the correct value in column d", projected.getField("d")); + assertThat(projected.get(0)).as("Should contain the correct value in column 1").isEqualTo(100L); + assertThat(projected.getField("a")) + .as("Should contain the correct value in column a") + .isEqualTo(100L); + assertThat(projected.get(1)).as("Should contain the correct value in column 2").isEqualTo(200L); + assertThat(projected.getField("b")) + .as("Should contain the correct value in column b") + .isEqualTo(200L); + assertThat(projected.get(2)).as("Should contain the correct value in column 3").isEqualTo(300L); + assertThat(projected.getField("c")) + .as("Should contain the correct value in column c") + .isEqualTo(300L); + assertThat(projected.get(3)).as("Should contain the correct value in column 4").isNull(); + assertThat(projected.getField("d")).as("Should contain the correct value in column d").isNull(); } @Test @@ -187,7 +193,7 @@ public void testEmptyProjection() throws Exception { Record projected = writeAndRead("empty_projection", schema, schema.select(), record); - Assert.assertNotNull("Should read a non-null record", projected); + assertThat(projected).as("Should read a non-null record").isNotNull(); // this is expected because there are no values assertThatThrownBy(() -> projected.get(0)).isInstanceOf(ArrayIndexOutOfBoundsException.class); } @@ -208,18 +214,19 @@ public void testBasicProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("basic_projection_id", writeSchema, idOnly, record); - Assert.assertNull("Should not project data", projected.getField("data")); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); + assertThat(projected.getField("data")).as("Should not project data").isNull(); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - int cmp = - Comparators.charSequences().compare("test", (CharSequence) projected.getField("data")); - Assert.assertTrue("Should contain the correct data value", cmp == 0); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat((CharSequence) projected.getField("data")) + .as("Should contain the correct data value") + .isEqualTo("test"); } @Test @@ -240,11 +247,12 @@ public void testRename() throws Exception { Record projected = writeAndRead("project_and_rename", writeSchema, readSchema, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - int cmp = - Comparators.charSequences().compare("test", (CharSequence) projected.getField("renamed")); - Assert.assertTrue("Should contain the correct data/renamed value", cmp == 0); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat((CharSequence) projected.getField("renamed")) + .as("Should contain the correct data/renamed value") + .isEqualTo("test"); } @Test @@ -270,9 +278,10 @@ public void testNestedStructProjection() throws Exception { Record projected = writeAndRead("id_only", writeSchema, idOnly, record); Record projectedLocation = (Record) projected.getField("location"); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project location", projectedLocation); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projectedLocation).as("Should not project location").isNull(); Schema latOnly = new Schema( @@ -283,14 +292,12 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); projectedLocation = (Record) projected.getField("location"); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project location", projected.getField("location")); - Assert.assertNull("Should not project longitude", projectedLocation.getField("long")); - Assert.assertEquals( - "Should project latitude", - 52.995143f, - (float) projectedLocation.getField("lat"), - 0.000001f); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(projected.getField("long")).as("Should not project longitude").isNull(); + assertThat((float) projectedLocation.getField("lat")) + .as("Should project latitude") + .isCloseTo(52.995143f, within(0.000001f)); Schema longOnly = new Schema( @@ -301,30 +308,25 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = (Record) projected.getField("location"); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project location", projected.getField("location")); - Assert.assertNull("Should not project latitutde", projectedLocation.getField("lat")); - Assert.assertEquals( - "Should project longitude", - -1.539054f, - (float) projectedLocation.getField("long"), - 0.000001f); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + assertThat(projectedLocation.getField("lat")).as("Should not project latitude").isNull(); + assertThat((float) projectedLocation.getField("long")) + .as("Should project longitude") + .isCloseTo(-1.539054f, within(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = (Record) projected.getField("location"); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project location", projected.getField("location")); - Assert.assertEquals( - "Should project latitude", - 52.995143f, - (float) projectedLocation.getField("lat"), - 0.000001f); - Assert.assertEquals( - "Should project longitude", - -1.539054f, - (float) projectedLocation.getField("long"), - 0.000001f); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("location")).as("Should project location").isNotNull(); + + assertThat((float) projectedLocation.getField("lat")) + .as("Should project latitude") + .isCloseTo(52.995143f, within(0.000001f)); + assertThat((float) projectedLocation.getField("long")) + .as("Should project longitude") + .isCloseTo(-1.539054f, within(0.000001f)); } @Test @@ -346,33 +348,31 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project properties map", projected.getField("properties")); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("properties")).as("Should not project properties map").isNull(); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project entire map", - properties, - toStringMap((Map) projected.getField("properties"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("properties"))) + .as("Should project entire map") + .isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project entire map", - properties, - toStringMap((Map) projected.getField("properties"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("properties"))) + .as("Should project entire map") + .isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project entire map", - properties, - toStringMap((Map) projected.getField("properties"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("properties"))) + .as("Should project entire map") + .isEqualTo(properties); } private Map toStringMap(Map map) { @@ -420,51 +420,61 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project locations map", projected.getField("locations")); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("locations")).as("Should not project locations map").isNull(); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project locations map", - record.getField("locations"), - toStringMap((Map) projected.getField("locations"))); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(toStringMap((Map) projected.getField("locations"))) + .as("Should project locations map") + .isEqualTo(record.getField("locations")); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), record); - Assert.assertNull("Should not project id", projected.getField("id")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); Map locations = toStringMap((Map) projected.getField("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()) + .as("Should contain L1 and L2") + .containsExactlyInAnyOrder("L1", "L2"); + Record projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain lat", 53.992811f, (float) projectedL1.getField("lat"), 0.000001); - Assert.assertNull("L1 should not contain long", projectedL1.getField("long")); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.getField("lat")) + .as("L1 should contain lat") + .isCloseTo(53.992811f, within(0.000001f)); + assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); + Record projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain lat", 52.995143f, (float) projectedL2.getField("lat"), 0.000001); - Assert.assertNull("L2 should not contain long", projectedL2.getField("long")); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.getField("lat")) + .as("L2 should contain lat") + .isCloseTo(52.995143f, within(0.000001f)); + assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), record); - Assert.assertNull("Should not project id", projected.getField("id")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); locations = toStringMap((Map) projected.getField("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()) + .as("Should contain L1 and L2") + .containsExactlyInAnyOrder("L1", "L2"); + projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertNull("L1 should not contain lat", projectedL1.getField("lat")); - Assert.assertEquals( - "L1 should contain long", -1.542616f, (float) projectedL1.getField("long"), 0.000001); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); + assertThat((float) projectedL1.getField("long")) + .as("L1 should contain long") + .isCloseTo(-1.542616f, within(0.000001f)); + projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertNull("L2 should not contain lat", projectedL2.getField("lat")); - Assert.assertEquals( - "L2 should contain long", -1.539054f, (float) projectedL2.getField("long"), 0.000001); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); + assertThat((float) projectedL2.getField("long")) + .as("L2 should contain long") + .isCloseTo(-1.539054f, within(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -479,29 +489,29 @@ public void testMapOfStructsProjection() throws IOException { Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, record); - Assert.assertNull("Should not project id", projected.getField("id")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + locations = toStringMap((Map) projected.getField("locations")); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals( - "Should contain L1 and L2", Sets.newHashSet("L1", "L2"), locations.keySet()); + assertThat(locations).as("Should project locations map").isNotNull(); + assertThat(locations.keySet()) + .as("Should contain L1 and L2") + .containsExactlyInAnyOrder("L1", "L2"); + projectedL1 = (Record) locations.get("L1"); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", - 53.992811f, - (float) projectedL1.getField("latitude"), - 0.000001); - Assert.assertNull("L1 should not contain lat", projectedL1.getField("lat")); - Assert.assertNull("L1 should not contain long", projectedL1.getField("long")); + assertThat(projectedL1).as("L1 should not be null").isNotNull(); + assertThat((float) projectedL1.getField("latitude")) + .as("L1 should contain latitude") + .isCloseTo(53.992811f, within(0.000001f)); + assertThat(projectedL1.getField("lat")).as("L1 should not contain lat").isNull(); + assertThat(projectedL1.getField("long")).as("L1 should not contain long").isNull(); + projectedL2 = (Record) locations.get("L2"); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", - 52.995143f, - (float) projectedL2.getField("latitude"), - 0.000001); - Assert.assertNull("L2 should not contain lat", projectedL2.getField("lat")); - Assert.assertNull("L2 should not contain long", projectedL2.getField("long")); + assertThat(projectedL2).as("L2 should not be null").isNotNull(); + assertThat((float) projectedL2.getField("latitude")) + .as("L2 should contain latitude") + .isCloseTo(52.995143f, within(0.000001f)); + assertThat(projectedL2.getField("lat")).as("L2 should not contain lat").isNull(); + assertThat(projectedL2.getField("long")).as("L2 should not contain long").isNull(); } @Test @@ -521,19 +531,20 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project values list", projected.getField("values")); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("values")).as("Should not project values list").isNull(); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals("Should project entire list", values, projected.getField("values")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals("Should project entire list", values, projected.getField("values")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("values")).as("Should project entire list").isEqualTo(values); } @Test @@ -568,38 +579,46 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); Record projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals( - "Should contain the correct id value", 34L, (long) projected.getField("id")); - Assert.assertNull("Should not project points list", projected.getField("points")); + assertThat((long) projected.getField("id")) + .as("Should contain the correct id value") + .isEqualTo(34L); + assertThat(projected.getField("points")).as("Should not project points list").isNull(); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertEquals( - "Should project points list", record.getField("points"), projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")) + .as("Should project points list") + .isEqualTo(record.getField("points")); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + List points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + Record projectedP1 = points.get(0); - Assert.assertEquals("Should project x", 1, (int) projectedP1.getField("x")); - Assert.assertNull("Should not project y", projectedP1.getField("y")); + assertThat((int) projectedP1.getField("x")).as("Should project x").isEqualTo(1); + assertThat(projectedP1.getField("y")).as("Should not project y").isNull(); + Record projectedP2 = points.get(1); - Assert.assertEquals("Should project x", 3, (int) projectedP2.getField("x")); - Assert.assertNull("Should not project y", projectedP2.getField("y")); + assertThat((int) projectedP2.getField("x")).as("Should project x").isEqualTo(3); + assertThat(projectedP2.getField("y")).as("Should not project y").isNull(); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.getField("x")); - Assert.assertEquals("Should project y", 2, (int) projectedP1.getField("y")); + assertThat(projectedP1.getField("x")).as("Should not project x").isNull(); + assertThat((int) projectedP1.getField("y")).as("Should project y").isEqualTo(2); + projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.getField("x")); - Assert.assertEquals("Should project null y", null, projectedP2.getField("y")); + assertThat(projectedP2.getField("x")).as("Should not project x").isNull(); + assertThat(projectedP2.getField("y")).as("Should project null y").isNull(); Schema yRenamed = new Schema( @@ -612,18 +631,21 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); projected = writeAndRead("y_renamed", writeSchema, yRenamed, record); - Assert.assertNull("Should not project id", projected.getField("id")); - Assert.assertNotNull("Should project points list", projected.getField("points")); + assertThat(projected.getField("id")).as("Should not project id").isNull(); + assertThat(projected.getField("points")).as("Should project points list").isNotNull(); + points = (List) projected.getField("points"); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points).as("Should read 2 points").hasSize(2); + projectedP1 = points.get(0); - Assert.assertNull("Should not project x", projectedP1.getField("x")); - Assert.assertNull("Should not project y", projectedP1.getField("y")); - Assert.assertEquals("Should project z", 2, (int) projectedP1.getField("z")); + assertThat(projectedP1.getField("x")).as("Should not project x").isNull(); + assertThat(projectedP1.getField("y")).as("Should not project y").isNull(); + assertThat((int) projectedP1.getField("z")).as("Should project z").isEqualTo(2); + projectedP2 = points.get(1); - Assert.assertNull("Should not project x", projectedP2.getField("x")); - Assert.assertNull("Should not project y", projectedP2.getField("y")); - Assert.assertEquals("Should project null z", null, projectedP2.getField("z")); + assertThat(projectedP2.getField("x")).as("Should not project x").isNull(); + assertThat(projectedP2.getField("y")).as("Should not project y").isNull(); + assertThat(projectedP2.getField("z")).as("Should project null z").isNull(); } @Test @@ -648,14 +670,17 @@ public void testAddedFieldsWithRequiredChildren() throws Exception { Record projected = writeAndRead("add_fields_with_required_children_projection", schema, addedFields, record); - Assert.assertEquals("Should contain the correct value in column 1", projected.get(0), 100L); - Assert.assertEquals( - "Should contain the correct value in column a", projected.getField("a"), 100L); - Assert.assertNull("Should contain empty value in new column 2", projected.get(1)); - Assert.assertNull("Should contain empty value in column b", projected.getField("b")); - Assert.assertNull("Should contain empty value in new column 4", projected.get(2)); - Assert.assertNull("Should contain empty value in column d", projected.getField("d")); - Assert.assertNull("Should contain empty value in new column 6", projected.get(3)); - Assert.assertNull("Should contain empty value in column e", projected.getField("e")); + assertThat((long) projected.get(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat((long) projected.getField("a")) + .as("Should contain the correct value in column a") + .isEqualTo(100L); + assertThat(projected.get(1)).as("Should contain empty value in new column 2").isNull(); + assertThat(projected.getField("b")).as("Should contain empty value in column b").isNull(); + assertThat(projected.get(2)).as("Should contain empty value in new column 4").isNull(); + assertThat(projected.getField("d")).as("Should contain empty value in column d").isNull(); + assertThat(projected.get(3)).as("Should contain empty value in new column 6").isNull(); + assertThat(projected.getField("e")).as("Should contain empty value in column e").isNull(); } } 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 7b2f2933716a..b6083906c74b 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.data.avro; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import org.apache.iceberg.Files; @@ -32,8 +34,8 @@ public class TestGenericReadProjection extends TestReadProjection { @Override protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException { - File file = temp.newFile(desc + ".avro"); - file.delete(); + File file = File.createTempFile("junit", ".avro", tempDir); + assertThat(file.delete()).isTrue(); try (FileAppender appender = Avro.write(Files.localOutput(file)) diff --git a/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java index b68c20f4f4f6..179c9e37eb96 100644 --- a/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java +++ b/data/src/test/java/org/apache/iceberg/data/avro/TestSingleMessageEncoding.java @@ -20,6 +20,7 @@ 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.nio.ByteBuffer; @@ -41,8 +42,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Ordering; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSingleMessageEncoding { private static final Schema SCHEMA_V1 = @@ -87,8 +87,8 @@ public void testByteBufferRoundTrip() throws Exception { Record copy = decoder.decode(encoder.encode(V2_RECORDS.get(0))); - Assert.assertTrue("Copy should not be the same object", copy != V2_RECORDS.get(0)); - Assert.assertEquals("Record should be identical after round-trip", V2_RECORDS.get(0), copy); + assertThat(copy).as("Copy should not be the same object").isNotSameAs(V2_RECORDS.get(0)); + assertThat(copy).as("Record should be identical after round-trip").isEqualTo(V2_RECORDS.get(0)); } @Test @@ -122,7 +122,7 @@ public void testSchemaEvolution() throws Exception { decodedUsingV2.add(v2Decoder.decode(buffer)); } - Assert.assertEquals(allAsV2, decodedUsingV2); + assertThat(decodedUsingV2).isEqualTo(allAsV2); } @Test @@ -147,7 +147,7 @@ public void testCompatibleReadWithSchema() throws Exception { Record record = v2Decoder.decode(v1Buffer); - Assert.assertEquals(v2Record(6L, "m-6", null), record); + assertThat(record).isEqualTo(v2Record(6L, "m-6", null)); } @Test @@ -162,7 +162,7 @@ public void testCompatibleReadWithSchemaFromLookup() throws Exception { Record record = v2Decoder.decode(v1Buffer); - Assert.assertEquals(v2Record(4L, "m-4", null), record); + assertThat(record).isEqualTo((v2Record(4L, "m-4", null))); } @Test @@ -175,11 +175,12 @@ public void testBufferReuse() throws Exception { ByteBuffer b0 = encoder.encode(V1_RECORDS.get(0)); ByteBuffer b1 = encoder.encode(V1_RECORDS.get(1)); - Assert.assertEquals(b0.array(), b1.array()); + assertThat(b1.array()).isEqualTo(b0.array()); MessageDecoder decoder = new IcebergDecoder<>(SCHEMA_V1); - Assert.assertEquals( - "Buffer was reused, decode(b0) should be record 1", V1_RECORDS.get(1), decoder.decode(b0)); + assertThat(decoder.decode(b0)) + .as("Buffer was reused, decode(b0) should be record 1") + .isEqualTo(V1_RECORDS.get(1)); } @Test @@ -189,12 +190,13 @@ public void testBufferCopy() throws Exception { ByteBuffer b0 = encoder.encode(V1_RECORDS.get(0)); ByteBuffer b1 = encoder.encode(V1_RECORDS.get(1)); - Assert.assertNotEquals(b0.array(), b1.array()); + assertThat(b1.array()).isNotEqualTo(b0.array()); MessageDecoder decoder = new IcebergDecoder<>(SCHEMA_V1); // bytes are not changed by reusing the encoder - Assert.assertEquals( - "Buffer was copied, decode(b0) should be record 0", V1_RECORDS.get(0), decoder.decode(b0)); + assertThat(decoder.decode(b0)) + .as("Buffer was copied, decode(b0) should be record 0") + .isEqualTo(V1_RECORDS.get(0)); } @Test diff --git a/data/src/test/java/org/apache/iceberg/data/orc/TestGenericReadProjection.java b/data/src/test/java/org/apache/iceberg/data/orc/TestGenericReadProjection.java index 0d8a58e4771b..e0c7c342e53c 100644 --- a/data/src/test/java/org/apache/iceberg/data/orc/TestGenericReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/orc/TestGenericReadProjection.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.data.orc; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import org.apache.iceberg.Files; @@ -33,8 +35,8 @@ public class TestGenericReadProjection extends TestReadProjection { @Override protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException { - File file = temp.newFile(desc + ".orc"); - file.delete(); + File file = File.createTempFile("junit", ".orc", tempDir); + assertThat(file.delete()).isTrue(); try (FileAppender appender = ORC.write(Files.localOutput(file)) diff --git a/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java index f2e2f1d4f354..fee7475e3274 100644 --- a/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java +++ b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcDataWriter.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.data.orc; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; import java.io.IOException; import java.util.List; import org.apache.iceberg.DataFile; @@ -37,11 +40,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.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestOrcDataWriter { private static final Schema SCHEMA = @@ -51,9 +52,9 @@ public class TestOrcDataWriter { private List records; - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File tempDir; - @Before + @BeforeEach public void createRecords() { GenericRecord record = GenericRecord.create(SCHEMA); @@ -69,7 +70,7 @@ public void createRecords() { @Test public void testDataWriter() throws IOException { - OutputFile file = Files.localOutput(temp.newFile()); + OutputFile file = Files.localOutput(File.createTempFile("junit", null, tempDir)); SortOrder sortOrder = SortOrder.builderFor(SCHEMA).withOrderId(10).asc("id").build(); @@ -92,13 +93,12 @@ public void testDataWriter() throws IOException { DataFile dataFile = dataWriter.toDataFile(); - Assert.assertEquals("Format should be ORC", FileFormat.ORC, dataFile.format()); - Assert.assertEquals("Should be data file", FileContent.DATA, dataFile.content()); - Assert.assertEquals("Record count should match", records.size(), dataFile.recordCount()); - Assert.assertEquals("Partition should be empty", 0, dataFile.partition().size()); - Assert.assertEquals( - "Sort order should match", sortOrder.orderId(), (int) dataFile.sortOrderId()); - Assert.assertNull("Key metadata should be null", dataFile.keyMetadata()); + assertThat(dataFile.format()).as("Format should be ORC").isEqualTo(FileFormat.ORC); + assertThat(dataFile.content()).as("Should be data file").isEqualTo(FileContent.DATA); + assertThat(dataFile.recordCount()).as("Record count should match").isEqualTo(records.size()); + assertThat(dataFile.partition().size()).as("Partition should be empty").isEqualTo(0); + assertThat(dataFile.sortOrderId()).as("Sort order should match").isEqualTo(sortOrder.orderId()); + assertThat(dataFile.keyMetadata()).as("Key metadata should be null").isNull(); List writtenRecords; try (CloseableIterable reader = @@ -109,6 +109,6 @@ public void testDataWriter() throws IOException { writtenRecords = Lists.newArrayList(reader); } - Assert.assertEquals("Written records should match", records, writtenRecords); + assertThat(writtenRecords).as("Written records should match").isEqualTo(records); } } diff --git a/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowIterator.java b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowIterator.java index 8dcf73776007..186f8fbe91e7 100644 --- a/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowIterator.java +++ b/data/src/test/java/org/apache/iceberg/data/orc/TestOrcRowIterator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.data.orc; 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; @@ -37,11 +38,9 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; import org.apache.orc.OrcConf; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestOrcRowIterator { @@ -59,14 +58,14 @@ public class TestOrcRowIterator { } } - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File tempDir; private File testFile; - @Before + @BeforeEach public void writeFile() throws IOException { - testFile = temp.newFile(); - Assert.assertTrue("Delete should succeed", testFile.delete()); + testFile = File.createTempFile("junit", null, tempDir); + assertThat(testFile.delete()).as("Delete should succeed").isTrue(); try (FileAppender writer = ORC.write(Files.localOutput(testFile)) diff --git a/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java index 87f770bfccfd..8c5ce8e5db6d 100644 --- a/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestGenericReadProjection.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.data.parquet; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; import org.apache.iceberg.Files; @@ -32,8 +34,8 @@ public class TestGenericReadProjection extends TestReadProjection { @Override protected Record writeAndRead(String desc, Schema writeSchema, Schema readSchema, Record record) throws IOException { - File file = temp.newFile(desc + ".parquet"); - file.delete(); + File file = File.createTempFile("junit", ".parquet", tempDir); + assertThat(file.delete()).isTrue(); try (FileAppender appender = Parquet.write(Files.localOutput(file)) 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 d1a782057006..74105458dbcc 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java +++ b/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java @@ -20,17 +20,22 @@ 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.ByteBuffer; import java.nio.CharBuffer; +import java.nio.file.Files; +import java.util.Arrays; import java.util.List; import java.util.Map; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; @@ -44,16 +49,13 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Types; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 abstract class TestWriterMetrics { private static final int FORMAT_V2 = 2; @@ -77,15 +79,15 @@ public abstract class TestWriterMetrics { protected static final Map properties = ImmutableMap.of(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private File tempDir; protected FileFormat fileFormat; protected TestTables.TestTable table = null; private OutputFileFactory fileFactory = null; - @Parameterized.Parameters(name = "FileFormat = {0}") - public static Object[][] parameters() { - return new Object[][] {{FileFormat.ORC}, {FileFormat.PARQUET}}; + @Parameters(name = "fileFormat = {0}") + public static List parameters() { + return Arrays.asList(FileFormat.ORC, FileFormat.PARQUET); } public TestWriterMetrics(FileFormat fileFormat) { @@ -98,9 +100,9 @@ public TestWriterMetrics(FileFormat fileFormat) { protected abstract T toGenericRow(int value, int repeated); - @Before + @BeforeEach public void setupTable() throws Exception { - File tableDir = temp.newFolder(); + File tableDir = Files.createTempDirectory(tempDir.toPath(), "junit").toFile(); tableDir.delete(); // created by table create this.table = @@ -111,12 +113,12 @@ public void setupTable() throws Exception { this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); } - @After + @AfterEach public void after() { TestTables.clearTables(); } - @Test + @TestTemplate public void verifySortedColMetric() throws Exception { T row = toRow(3, "3", true, 3L); DataWriter dataWriter = @@ -128,25 +130,25 @@ public void verifySortedColMetric() throws Exception { // Only two sorted fields (id, structField.longValue) will have metrics Map lowerBounds = dataFile.lowerBounds(); - Assert.assertEquals( - 3, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))); - Assert.assertFalse(lowerBounds.containsKey(2)); - Assert.assertFalse(lowerBounds.containsKey(3)); - Assert.assertFalse(lowerBounds.containsKey(4)); - Assert.assertEquals( - 3L, (long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(5))); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))) + .isEqualTo(3); + assertThat(lowerBounds).doesNotContainKey(2); + assertThat(lowerBounds).doesNotContainKey(3); + assertThat(lowerBounds).doesNotContainKey(4); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(5))) + .isEqualTo(3L); Map upperBounds = dataFile.upperBounds(); - Assert.assertEquals( - 3, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))); - Assert.assertFalse(upperBounds.containsKey(2)); - Assert.assertFalse(upperBounds.containsKey(3)); - Assert.assertFalse(upperBounds.containsKey(4)); - Assert.assertEquals( - 3L, (long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(5))); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))) + .isEqualTo(3); + assertThat(upperBounds).doesNotContainKey(2); + assertThat(upperBounds).doesNotContainKey(3); + assertThat(upperBounds).doesNotContainKey(4); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(5))) + .isEqualTo(3L); } - @Test + @TestTemplate public void testPositionDeleteMetrics() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table); EncryptedOutputFile outputFile = fileFactory.newOutputFile(); @@ -172,38 +174,36 @@ public void testPositionDeleteMetrics() throws IOException { Map lowerBounds = deleteFile.lowerBounds(); - Assert.assertEquals( - CharBuffer.wrap("File A"), - Conversions.fromByteBuffer(Types.StringType.get(), lowerBounds.get(pathFieldId))); - Assert.assertEquals( - 1L, (long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(posFieldId))); + assertThat(Conversions.fromByteBuffer(Types.StringType.get(), lowerBounds.get(pathFieldId))) + .isEqualTo(CharBuffer.wrap("File A")); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(posFieldId))) + .isEqualTo(1L); - Assert.assertEquals( - 3, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))); - Assert.assertFalse(lowerBounds.containsKey(2)); - Assert.assertFalse(lowerBounds.containsKey(3)); - Assert.assertFalse(lowerBounds.containsKey(4)); - Assert.assertEquals( - 3L, (long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(5))); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))) + .isEqualTo(3); + assertThat(lowerBounds).doesNotContainKey(2); + assertThat(lowerBounds).doesNotContainKey(3); + assertThat(lowerBounds).doesNotContainKey(4); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(5))) + .isEqualTo(3L); Map upperBounds = deleteFile.upperBounds(); - Assert.assertEquals( - CharBuffer.wrap("File A"), - Conversions.fromByteBuffer(Types.StringType.get(), upperBounds.get(pathFieldId))); - Assert.assertEquals( - 1L, (long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(posFieldId))); - - Assert.assertEquals( - 3, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))); - Assert.assertFalse(upperBounds.containsKey(2)); - Assert.assertFalse(upperBounds.containsKey(3)); - Assert.assertFalse(upperBounds.containsKey(4)); - Assert.assertEquals( - 3L, (long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(5))); + assertThat(Conversions.fromByteBuffer(Types.StringType.get(), upperBounds.get(pathFieldId))) + .isEqualTo(CharBuffer.wrap("File A")); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(posFieldId))) + .isEqualTo(1L); + + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))) + .isEqualTo(3); + assertThat(upperBounds).doesNotContainKey(2); + assertThat(upperBounds).doesNotContainKey(3); + assertThat(upperBounds).doesNotContainKey(4); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(5))) + .isEqualTo(3L); } - @Test + @TestTemplate public void testPositionDeleteMetricsCoveringMultipleDataFiles() throws IOException { FileWriterFactory writerFactory = newWriterFactory(table); EncryptedOutputFile outputFile = fileFactory.newOutputFile(); @@ -227,24 +227,24 @@ public void testPositionDeleteMetricsCoveringMultipleDataFiles() throws IOExcept // should have NO bounds for path and position as the file covers multiple data paths Map lowerBounds = deleteFile.lowerBounds(); - Assert.assertEquals(2, lowerBounds.size()); - Assert.assertEquals( - 3, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))); - Assert.assertEquals( - 3L, (long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(5))); + assertThat(lowerBounds).hasSize(2); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))) + .isEqualTo(3); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), lowerBounds.get(5))) + .isEqualTo(3L); Map upperBounds = deleteFile.upperBounds(); - Assert.assertEquals(2, upperBounds.size()); - Assert.assertEquals( - 3, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))); - Assert.assertEquals( - 3L, (long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(5))); + assertThat(upperBounds).hasSize(2); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))) + .isEqualTo(3); + assertThat((long) Conversions.fromByteBuffer(Types.LongType.get(), upperBounds.get(5))) + .isEqualTo(3L); } - @Test + @TestTemplate public void testMaxColumns() throws IOException { - File tableDir = temp.newFolder(); - tableDir.delete(); // created by table create + 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); @@ -275,30 +275,39 @@ public void testMaxColumns() throws IOException { // start at 1 because IDs were reassigned in the table int id = 1; for (; id <= TableProperties.METRICS_MAX_INFERRED_COLUMN_DEFAULTS_DEFAULT; id += 1) { - Assert.assertNotNull("Should have lower bound metrics", dataFile.lowerBounds().get(id)); - Assert.assertNotNull("Should have upper bound metrics", dataFile.upperBounds().get(id)); - Assert.assertNull( - "Should not have nan value metrics (not floating point)", - dataFile.nanValueCounts().get(id)); - Assert.assertNotNull("Should have null value metrics", dataFile.nullValueCounts().get(id)); - Assert.assertNotNull("Should have value metrics", dataFile.valueCounts().get(id)); + assertThat(dataFile.lowerBounds().get(id)).as("Should have lower bound metrics").isNotNull(); + assertThat(dataFile.upperBounds().get(id)).as("Should have upper bound metrics").isNotNull(); + assertThat(dataFile.nanValueCounts().get(id)) + .as("Should not have nan value metrics (not floating point)") + .isNull(); + assertThat(dataFile.nullValueCounts().get(id)) + .as("Should have null value metrics") + .isNotNull(); + assertThat(dataFile.valueCounts().get(id)).as("Should have value metrics").isNotNull(); } // Remaining fields should not have metrics for (; id <= numColumns; id += 1) { - Assert.assertNull("Should not have any lower bound metrics", dataFile.lowerBounds().get(id)); - Assert.assertNull("Should not have any upper bound metrics", dataFile.upperBounds().get(id)); - Assert.assertNull("Should not have any nan value metrics", dataFile.nanValueCounts().get(id)); - Assert.assertNull( - "Should not have any null value metrics", dataFile.nullValueCounts().get(id)); - Assert.assertNull("Should not have any value metrics", dataFile.valueCounts().get(id)); + assertThat(dataFile.lowerBounds().get(id)) + .as("Should not have any lower bound metrics") + .isNull(); + assertThat(dataFile.upperBounds().get(id)) + .as("Should not have any upper bound metrics") + .isNull(); + assertThat(dataFile.nanValueCounts().get(id)) + .as("Should not have any nan value metrics") + .isNull(); + assertThat(dataFile.nullValueCounts().get(id)) + .as("Should not have any null value metrics") + .isNull(); + assertThat(dataFile.valueCounts().get(id)).as("Should not have any value metrics").isNull(); } } - @Test + @TestTemplate public void testMaxColumnsWithDefaultOverride() throws IOException { - File tableDir = temp.newFolder(); - tableDir.delete(); // created by table create + 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); @@ -336,10 +345,10 @@ public void testMaxColumnsWithDefaultOverride() throws IOException { Map upperBounds = dataFile.upperBounds(); Map lowerBounds = dataFile.upperBounds(); for (int i = 0; i < numColumns; i++) { - Assert.assertEquals( - 1, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))); - Assert.assertEquals( - 1, (int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), upperBounds.get(1))) + .isEqualTo(1); + assertThat((int) Conversions.fromByteBuffer(Types.IntegerType.get(), lowerBounds.get(1))) + .isEqualTo(1); } } } diff --git a/data/src/test/java/org/apache/iceberg/parquet/TestGenericMergingMetrics.java b/data/src/test/java/org/apache/iceberg/parquet/TestGenericMergingMetrics.java index b1c03872981a..aac00aebf09e 100644 --- a/data/src/test/java/org/apache/iceberg/parquet/TestGenericMergingMetrics.java +++ b/data/src/test/java/org/apache/iceberg/parquet/TestGenericMergingMetrics.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.parquet; +import java.io.File; import java.io.IOException; import java.util.List; -import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; import org.apache.iceberg.TestMergingMetrics; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; @@ -28,15 +29,12 @@ public class TestGenericMergingMetrics extends TestMergingMetrics { - public TestGenericMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileAppender writeAndGetAppender(List records) throws IOException { FileAppender appender = new GenericAppenderFactory(SCHEMA) - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender( + Files.localOutput(File.createTempFile("junit", null, tempDir)), fileFormat); try (FileAppender fileAppender = appender) { records.forEach(fileAppender::add); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index caefbb5a5429..0af49e9e2365 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -83,8 +83,8 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod.assertEquals( "Should have expected StructLike values", - actualWrapper.set(recordStructLike), - expectedWrapper.set(rowDataStructLike)); + expectedWrapper.set(rowDataStructLike), + actualWrapper.set(recordStructLike)); } assertThat(actual).isExhausted(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java index 1d52acb2fe7b..5be4a31b4ac8 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java @@ -18,43 +18,37 @@ */ package org.apache.iceberg.flink.source; +import java.io.File; import java.io.IOException; import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; import org.apache.iceberg.TestMergingMetrics; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkAppenderFactory; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkMergingMetrics extends TestMergingMetrics { - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); - - public TestFlinkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension("test_db", "test_table"); @Override protected FileAppender writeAndGetAppender(List records) throws IOException { + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - FileAppender appender = new FlinkAppenderFactory( - tableResource.table(), + table, SCHEMA, flinkSchema, ImmutableMap.of(), @@ -62,7 +56,9 @@ protected FileAppender writeAndGetAppender(List records) throws null, null, null) - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender( + org.apache.iceberg.Files.localOutput(File.createTempFile("junit", null, tempDir)), + fileFormat); try (FileAppender fileAppender = appender) { records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index caefbb5a5429..0af49e9e2365 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -83,8 +83,8 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod.assertEquals( "Should have expected StructLike values", - actualWrapper.set(recordStructLike), - expectedWrapper.set(rowDataStructLike)); + expectedWrapper.set(rowDataStructLike), + actualWrapper.set(recordStructLike)); } assertThat(actual).isExhausted(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java index 1d52acb2fe7b..5be4a31b4ac8 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java @@ -18,43 +18,37 @@ */ package org.apache.iceberg.flink.source; +import java.io.File; import java.io.IOException; import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; import org.apache.iceberg.TestMergingMetrics; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkAppenderFactory; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkMergingMetrics extends TestMergingMetrics { - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); - - public TestFlinkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension("test_db", "test_table"); @Override protected FileAppender writeAndGetAppender(List records) throws IOException { + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - FileAppender appender = new FlinkAppenderFactory( - tableResource.table(), + table, SCHEMA, flinkSchema, ImmutableMap.of(), @@ -62,7 +56,9 @@ protected FileAppender writeAndGetAppender(List records) throws null, null, null) - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender( + org.apache.iceberg.Files.localOutput(File.createTempFile("junit", null, tempDir)), + fileFormat); try (FileAppender fileAppender = appender) { records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java index caefbb5a5429..0af49e9e2365 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -83,8 +83,8 @@ protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod.assertEquals( "Should have expected StructLike values", - actualWrapper.set(recordStructLike), - expectedWrapper.set(rowDataStructLike)); + expectedWrapper.set(rowDataStructLike), + actualWrapper.set(recordStructLike)); } assertThat(actual).isExhausted(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java index 1d52acb2fe7b..59a4c3118cdf 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java @@ -18,43 +18,38 @@ */ package org.apache.iceberg.flink.source; +import java.io.File; import java.io.IOException; import java.util.List; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; import org.apache.iceberg.TestMergingMetrics; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkAppenderFactory; import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkMergingMetrics extends TestMergingMetrics { - @ClassRule public static final TemporaryFolder TEMP_FOLDER = new TemporaryFolder(); - - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource(TEMP_FOLDER, "test_db", "test_table", SCHEMA); - - public TestFlinkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension("test_db", "test_table"); @Override protected FileAppender writeAndGetAppender(List records) throws IOException { + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - FileAppender appender = new FlinkAppenderFactory( - tableResource.table(), + table, SCHEMA, flinkSchema, ImmutableMap.of(), @@ -62,7 +57,8 @@ protected FileAppender writeAndGetAppender(List records) throws null, null, null) - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender( + Files.localOutput(File.createTempFile("junit", null, tempDir)), fileFormat); try (FileAppender fileAppender = appender) { records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); } diff --git a/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java b/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java index f9f2334cfcf4..634d43490b09 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestHelper.java @@ -40,7 +40,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.rules.TemporaryFolder; public class TestHelper { private final Configuration conf; @@ -49,30 +48,10 @@ public class TestHelper { private final Schema schema; private final PartitionSpec spec; private final FileFormat fileFormat; - private final TemporaryFolder tmp; private final Path temp; private Table table; - @Deprecated - public TestHelper( - Configuration conf, - Tables tables, - String tableIdentifier, - Schema schema, - PartitionSpec spec, - FileFormat fileFormat, - TemporaryFolder tmp) { - this.conf = conf; - this.tables = tables; - this.tableIdentifier = tableIdentifier; - this.schema = schema; - this.spec = spec; - this.fileFormat = fileFormat; - this.temp = null; - this.tmp = tmp; - } - public TestHelper( Configuration conf, Tables tables, @@ -88,7 +67,6 @@ public TestHelper( this.spec = spec; this.fileFormat = fileFormat; this.temp = temp; - this.tmp = null; } public void setTable(Table table) { @@ -144,9 +122,6 @@ public DataFile writeFile(StructLike partition, List records) throws IOE } private GenericAppenderHelper appender() { - if (null != tmp) { - return new GenericAppenderHelper(table, fileFormat, tmp, conf); - } return new GenericAppenderHelper(table, fileFormat, temp, conf); } 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 index c3bb35ca7df8..fa1ff9fbfde0 100644 --- 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 @@ -18,12 +18,12 @@ */ 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.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; @@ -35,10 +35,6 @@ public class TestSparkMergingMetrics extends TestMergingMetrics { - public TestSparkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileAppender writeAndGetAppender(List records) throws IOException { Table testTable = @@ -59,10 +55,11 @@ public PartitionSpec spec() { } }; + File tempFile = File.createTempFile("junit", null, tempDir); FileAppender appender = SparkAppenderFactory.builderFor(testTable, SCHEMA, SparkSchemaUtil.convert(SCHEMA)) .build() - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender(org.apache.iceberg.Files.localOutput(tempFile), fileFormat); try (FileAppender fileAppender = appender) { records.stream() .map(r -> new StructInternalRow(SCHEMA.asStruct()).setStruct(r)) 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 index c3bb35ca7df8..fa1ff9fbfde0 100644 --- 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 @@ -18,12 +18,12 @@ */ 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.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; @@ -35,10 +35,6 @@ public class TestSparkMergingMetrics extends TestMergingMetrics { - public TestSparkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileAppender writeAndGetAppender(List records) throws IOException { Table testTable = @@ -59,10 +55,11 @@ public PartitionSpec spec() { } }; + File tempFile = File.createTempFile("junit", null, tempDir); FileAppender appender = SparkAppenderFactory.builderFor(testTable, SCHEMA, SparkSchemaUtil.convert(SCHEMA)) .build() - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender(org.apache.iceberg.Files.localOutput(tempFile), fileFormat); try (FileAppender fileAppender = appender) { records.stream() .map(r -> new StructInternalRow(SCHEMA.asStruct()).setStruct(r)) 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 index c3bb35ca7df8..29425398f395 100644 --- 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 @@ -18,12 +18,13 @@ */ 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.FileFormat; +import org.apache.iceberg.Files; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; @@ -35,10 +36,6 @@ public class TestSparkMergingMetrics extends TestMergingMetrics { - public TestSparkMergingMetrics(FileFormat fileFormat) { - super(fileFormat); - } - @Override protected FileAppender writeAndGetAppender(List records) throws IOException { Table testTable = @@ -59,10 +56,11 @@ public PartitionSpec spec() { } }; + File tempFile = File.createTempFile("junit", null, tempDir); FileAppender appender = SparkAppenderFactory.builderFor(testTable, SCHEMA, SparkSchemaUtil.convert(SCHEMA)) .build() - .newAppender(org.apache.iceberg.Files.localOutput(temp.newFile()), fileFormat); + .newAppender(Files.localOutput(tempFile), fileFormat); try (FileAppender fileAppender = appender) { records.stream() .map(r -> new StructInternalRow(SCHEMA.asStruct()).setStruct(r)) From 5ae3e033c97ed91115763d0958884e571ddfe4fa Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 10 Jul 2024 15:13:43 +0900 Subject: [PATCH 0450/1019] Flink 1.17, 1.18: Migrate tests to JUnit5 (#10663) --- .../iceberg/flink/source/SplitHelpers.java | 21 +- .../assigner/SplitAssignerTestBase.java | 28 +- .../assigner/TestDefaultSplitAssigner.java | 4 +- ...tFileSequenceNumberBasedSplitAssigner.java | 8 +- .../TestWatermarkBasedSplitAssigner.java | 10 +- .../TestContinuousIcebergEnumerator.java | 96 +++-- .../TestContinuousSplitPlannerImpl.java | 351 +++++++++--------- ...ntinuousSplitPlannerImplStartStrategy.java | 80 ++-- .../enumerator/TestEnumerationHistory.java | 23 +- .../TestIcebergEnumeratorStateSerializer.java | 67 ++-- .../source/reader/ReaderFunctionTestBase.java | 65 ++-- .../flink/source/reader/ReaderUtil.java | 19 - .../source/reader/TestArrayBatchRecords.java | 19 +- ...stArrayPoolDataIteratorBatcherRowData.java | 286 +++++++------- .../reader/TestIcebergSourceReader.java | 35 +- .../reader/TestRowDataReaderFunction.java | 13 +- .../TestIcebergSourceSplitSerializer.java | 47 +-- .../iceberg/flink/source/SplitHelpers.java | 21 +- .../assigner/SplitAssignerTestBase.java | 28 +- .../assigner/TestDefaultSplitAssigner.java | 4 +- ...tFileSequenceNumberBasedSplitAssigner.java | 8 +- .../TestWatermarkBasedSplitAssigner.java | 10 +- .../TestContinuousIcebergEnumerator.java | 96 +++-- .../TestContinuousSplitPlannerImpl.java | 351 +++++++++--------- ...ntinuousSplitPlannerImplStartStrategy.java | 80 ++-- .../enumerator/TestEnumerationHistory.java | 23 +- .../TestIcebergEnumeratorStateSerializer.java | 67 ++-- .../source/reader/ReaderFunctionTestBase.java | 65 ++-- .../flink/source/reader/ReaderUtil.java | 19 - .../source/reader/TestArrayBatchRecords.java | 19 +- ...stArrayPoolDataIteratorBatcherRowData.java | 286 +++++++------- .../reader/TestIcebergSourceReader.java | 33 +- .../reader/TestRowDataReaderFunction.java | 13 +- .../TestIcebergSourceSplitSerializer.java | 47 +-- .../TestContinuousIcebergEnumerator.java | 3 +- ...ntinuousSplitPlannerImplStartStrategy.java | 5 - .../source/reader/ReaderFunctionTestBase.java | 1 - ...stArrayPoolDataIteratorBatcherRowData.java | 68 ++-- 38 files changed, 1169 insertions(+), 1250 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index ebd220b00dba..540902f3cea5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -18,11 +18,13 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; 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.hadoop.conf.Configuration; @@ -48,8 +50,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; public class SplitHelpers { @@ -67,14 +67,14 @@ private SplitHelpers() {} * attempt to read the data files. * *

    By default, v1 Iceberg table is created. For v2 table use {@link - * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * SplitHelpers#createSplitsFromTransientHadoopTable(Path, int, int, String)} * * @param temporaryFolder Folder to place the data to * @param fileCount The number of files to create and add to the table * @param filesPerSplit The number of files used for a split */ public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + Path temporaryFolder, int fileCount, int filesPerSplit) throws Exception { return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); } @@ -95,10 +95,9 @@ public static List createSplitsFromTransientHadoopTable( * @param version The table version to create */ public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) - throws Exception { - final File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); + Path temporaryFolder, int fileCount, int filesPerSplit, String version) throws Exception { + final File warehouseFile = File.createTempFile("junit", null, temporaryFolder.toFile()); + assertThat(warehouseFile.delete()).isTrue(); final String warehouse = "file:" + warehouseFile; Configuration hadoopConf = new Configuration(); final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); @@ -155,9 +154,7 @@ public static List createSplitsFromTransientHadoopTable( * @throws IOException If there is any error creating the mock delete files */ public static List equipSplitsWithMockDeleteFiles( - List icebergSourceSplits, - TemporaryFolder temporaryFolder, - int deleteFilesPerSplit) + List icebergSourceSplits, Path temporaryFolder, int deleteFilesPerSplit) throws IOException { List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); for (IcebergSourceSplit split : icebergSourceSplits) { @@ -171,7 +168,7 @@ public static List equipSplitsWithMockDeleteFiles( final DeleteFile deleteFile = FileMetadata.deleteFileBuilder(spec) .withFormat(FileFormat.PARQUET) - .withPath(temporaryFolder.newFile().getPath()) + .withPath(File.createTempFile("junit", null, temporaryFolder.toFile()).getPath()) .ofPositionDeletes() .withFileSizeInBytes(1000) .withRecordCount(1000) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index 090b304942c6..1e612b0a2b2a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +import java.nio.file.Path; import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -25,13 +29,11 @@ import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class SplitAssignerTestBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; @Test public void testEmptyInitialization() { @@ -86,11 +88,11 @@ private void assertAvailableFuture( // calling isAvailable again should return the same object reference // note that thenAccept will return a new future. // we want to assert the same instance on the assigner returned future - Assert.assertSame(future, assigner.isAvailable()); + assertThat(assigner.isAvailable()).isSameAs(future); // now add some splits addSplitsRunnable.run(); - Assert.assertEquals(true, futureCompleted.get()); + assertThat(futureCompleted.get()).isTrue(); for (int i = 0; i < splitCount; ++i) { assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -101,29 +103,29 @@ private void assertAvailableFuture( protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(expectedStatus, result.status()); + assertThat(result.status()).isEqualTo(expectedStatus); switch (expectedStatus) { case AVAILABLE: - Assert.assertNotNull(result.split()); + assertThat(result.split()).isNotNull(); break; case CONSTRAINED: case UNAVAILABLE: - Assert.assertNull(result.split()); + assertThat(result.split()).isNull(); break; default: - Assert.fail("Unknown status: " + expectedStatus); + fail("Unknown status: %s", expectedStatus); } } protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Collection stateBeforeGet = assigner.state(); - Assert.assertEquals(splitCount, stateBeforeGet.size()); + assertThat(stateBeforeGet).hasSize(splitCount); } protected List createSplits(int fileCount, int filesPerSplit, String version) throws Exception { return SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + temporaryFolder, fileCount, filesPerSplit, version); } protected abstract SplitAssigner splitAssigner(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java index 8994f3054abe..17e64bbf0594 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.source.assigner; import org.apache.iceberg.flink.source.SplitHelpers; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDefaultSplitAssigner extends SplitAssignerTestBase { @Override @@ -32,7 +32,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInASplit() throws Exception { SplitAssigner assigner = splitAssigner(); assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 4, 2)); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java index 20555ad464a9..ff63ba8e58a0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; @@ -26,8 +27,7 @@ import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { @Override @@ -70,12 +70,12 @@ public void testSerializable() { byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); SerializableComparator comparator = SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); + assertThat(comparator).isNotNull(); } private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); - Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + assertThat(file.fileSequenceNumber()).isEqualTo(expectedSequenceNumber); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java index 6c0cbb733320..84f04d5a530a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.assigner; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Instant; @@ -44,8 +45,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { public static final Schema SCHEMA = @@ -104,12 +104,12 @@ public void testSerializable() { TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); SerializableComparator comparator = SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); + assertThat(comparator).isNotNull(); } private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(result.split(), split); + assertThat(split).isEqualTo(result.split()); } @Override @@ -138,7 +138,7 @@ private IcebergSourceSplit splitFromRecords(List> records) { try { return IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( - records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + records, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); } catch (IOException e) { throw new RuntimeException("Split creation exception", e); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 95770bb7dc2f..41a787762fda 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -36,13 +37,11 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousIcebergEnumerator { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; @Test public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { @@ -59,19 +58,19 @@ public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { Collection pendingSplitsEmpty = enumerator.snapshotState(1).pendingSplits(); - Assert.assertEquals(0, pendingSplitsEmpty.size()); + assertThat(pendingSplitsEmpty).isEmpty(); // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } @Test @@ -94,11 +93,11 @@ public void testDiscoverWhenReaderRegistered() throws Exception { // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); - Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -126,26 +125,25 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - Assert.assertEquals(1, splits.size()); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + assertThat(splits).hasSize(1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); - Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); + assertThat(enumeratorContext.getSplitAssignments()).doesNotContainKey(2); List pendingSplitIds = enumerator.snapshotState(1).pendingSplits().stream() .map(IcebergSourceSplitState::split) .map(IcebergSourceSplit::splitId) .collect(Collectors.toList()); - Assert.assertEquals(splits.size(), pendingSplitIds.size()); - Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); + assertThat(pendingSplitIds).hasSameSizeAs(splits).first().isEqualTo(splits.get(0).splitId()); // register the reader again, and let it request a split enumeratorContext.registerReader(2, "localhost"); enumerator.addReader(2); enumerator.handleSourceEvent(2, new SplitRequestEvent()); - Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -154,7 +152,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio public void testThrottlingDiscovery() throws Exception { // create 10 splits List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 1); TestingSplitEnumeratorContext enumeratorContext = new TestingSplitEnumeratorContext<>(4); @@ -179,10 +177,10 @@ public void testThrottlingDiscovery() throws Exception { enumeratorContext.triggerAllActions(); // because discovered split was assigned to reader, pending splits should be empty - Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); // split assignment to reader-2 should contain splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); // add the remaining 9 splits (one for every snapshot) // run discovery cycles while reader-2 still processing the splits[0] @@ -192,20 +190,20 @@ public void testThrottlingDiscovery() throws Exception { } // can only discover up to 3 snapshots/splits - Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); + assertThat(enumerator.snapshotState(2).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); // now reader-2 finished splits[0] enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); + assertThat(enumerator.snapshotState(3).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); // run 3 more split discovery cycles for (int i = 0; i < 3; ++i) { @@ -213,20 +211,20 @@ public void testThrottlingDiscovery() throws Exception { } // no more splits are discovered due to throttling - Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); + assertThat(enumerator.snapshotState(4).pendingSplits()).hasSize(3); // split assignment to reader-2 should still be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); // now reader-2 finished splits[1] enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); + assertThat(enumerator.snapshotState(5).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 3) - Assert.assertEquals( - splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 3)); } @Test @@ -246,20 +244,20 @@ public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); // Trigger a planning and check that no splits returned due to the planning error enumeratorContext.triggerAllActions(); - Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); // Second scan planning should succeed and discover the expected splits enumeratorContext.triggerAllActions(); Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } @Test @@ -278,19 +276,19 @@ public void testOverMaxAllowedPlanningErrors() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); // Check that the scheduler response ignores the current error and continues to run until the // failure limit is reached enumeratorContext.triggerAllActions(); - Assert.assertFalse( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isFalse(); // Check that the task has failed with the expected exception after the failure limit is reached enumeratorContext.triggerAllActions(); - Assert.assertTrue( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isTrue(); assertThatThrownBy( () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) .hasCauseInstanceOf(RuntimeException.class) @@ -316,7 +314,7 @@ public void testPlanningIgnoringErrors() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); Collection pendingSplits; @@ -324,16 +322,16 @@ public void testPlanningIgnoringErrors() throws Exception { for (int i = 0; i < expectedFailures; ++i) { enumeratorContext.triggerAllActions(); pendingSplits = enumerator.snapshotState(i).pendingSplits(); - Assert.assertEquals(0, pendingSplits.size()); + assertThat(pendingSplits).isEmpty(); } // Discovered the new split after a successful scan planning enumeratorContext.triggerAllActions(); pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } private static ContinuousIcebergEnumerator createEnumerator( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index f98e93835d15..381c340edd6a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -18,9 +18,11 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -31,33 +33,27 @@ import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousSplitPlannerImpl { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private static final FileFormat fileFormat = FileFormat.PARQUET; private static final AtomicLong randomSeed = new AtomicLong(); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - @Rule public TestName testName = new TestName(); + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private GenericAppenderHelper dataAppender; private DataFile dataFile1; @@ -65,9 +61,9 @@ public class TestContinuousSplitPlannerImpl { private DataFile dataFile2; private Snapshot snapshot2; - @Before + @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), fileFormat, temporaryFolder); } private void appendTwoSnapshots() throws IOException { @@ -75,13 +71,13 @@ private void appendTwoSnapshots() throws IOException { List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); dataFile1 = dataAppender.writeFile(null, batch1); dataAppender.appendToTable(dataFile1); - snapshot1 = tableResource.table().currentSnapshot(); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); // snapshot2 List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); dataFile2 = dataAppender.writeFile(null, batch2); dataAppender.appendToTable(dataFile2); - snapshot2 = tableResource.table().currentSnapshot(); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } /** @return the last enumerated snapshot id */ @@ -92,21 +88,22 @@ private CycleResult verifyOneCycle( RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); - Snapshot snapshot = tableResource.table().currentSnapshot(); + Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); - Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); - Assert.assertEquals( - lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, result.splits().size()); + assertThat(result.fromPosition().snapshotId()).isEqualTo(lastPosition.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs()) + .isEqualTo(lastPosition.snapshotTimestampMs()); + assertThat(result.toPosition().snapshotId().longValue()).isEqualTo(snapshot.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot.timestampMillis()); + assertThat(result.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Assert.assertEquals( - dataFile.path().toString(), - Iterables.getOnlyElement(split.task().files()).file().path().toString()); + assertThat(split.task().files()) + .hasSize(1) + .first() + .satisfies( + fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); return new CycleResult(result.toPosition(), split); } @@ -117,21 +114,21 @@ public void testTableScanThenIncrementalWithEmptyTable() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); @@ -149,24 +146,24 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - Assert.assertEquals( - snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.fromPosition()).isNull(); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); Set expectedFiles = ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -182,27 +179,27 @@ public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { .splitSize(1L) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // latest mode should discover both snapshots, as latest position is marked by when job starts appendTwoSnapshots(); ContinuousEnumerationResult afterTwoSnapshotsAppended = splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); - Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); + assertThat(afterTwoSnapshotsAppended.splits()).hasSize(2); // next 3 snapshots IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); @@ -220,35 +217,36 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1 // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -263,21 +261,21 @@ public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); @@ -295,24 +293,25 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1's parent, // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertNull(secondResult.fromPosition().snapshotId()); - Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId()).isNull(); + assertThat(secondResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) @@ -320,7 +319,7 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except // should discover files appended in both snapshot1 and snapshot2 Set expectedFiles = ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -337,7 +336,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { .build(); ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -363,7 +362,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -380,35 +379,36 @@ public void testIncrementalFromSnapshotId() throws Exception { .startSnapshotId(snapshot2.snapshotId()) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as // snapshot2's parent) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -425,7 +425,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { .build(); ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -446,7 +446,7 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -463,34 +463,35 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { .startSnapshotTimestamp(snapshot2.timestampMillis()) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -513,15 +514,15 @@ public void testMaxPlanningSnapshotCount() throws Exception { .maxPlanningSnapshotCount(1) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1's parent, // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 @@ -544,12 +545,12 @@ public void testTableScanNoStats() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 0); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -570,12 +571,12 @@ public void testTableScanAllStats() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 3); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -596,12 +597,12 @@ public void testTableScanSingleStat() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 1); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -619,12 +620,12 @@ private void verifyStatCount(IcebergSourceSplit split, int expected) { .files() .forEach( f -> { - Assert.assertNull(f.file().valueCounts()); - Assert.assertNull(f.file().columnSizes()); - Assert.assertNull(f.file().lowerBounds()); - Assert.assertNull(f.file().upperBounds()); - Assert.assertNull(f.file().nanValueCounts()); - Assert.assertNull(f.file().nullValueCounts()); + assertThat(f.file().valueCounts()).isNull(); + assertThat(f.file().columnSizes()).isNull(); + assertThat(f.file().lowerBounds()).isNull(); + assertThat(f.file().upperBounds()).isNull(); + assertThat(f.file().nanValueCounts()).isNull(); + assertThat(f.file().nullValueCounts()).isNull(); }); } else { split @@ -632,13 +633,13 @@ private void verifyStatCount(IcebergSourceSplit split, int expected) { .files() .forEach( f -> { - Assert.assertEquals(expected, f.file().valueCounts().size()); - Assert.assertEquals(expected, f.file().columnSizes().size()); - Assert.assertEquals(expected, f.file().lowerBounds().size()); - Assert.assertEquals(expected, f.file().upperBounds().size()); - Assert.assertEquals(expected, f.file().nullValueCounts().size()); + assertThat(f.file().valueCounts()).hasSize(expected); + assertThat(f.file().columnSizes()).hasSize(expected); + assertThat(f.file().lowerBounds()).hasSize(expected); + assertThat(f.file().upperBounds()).hasSize(expected); + assertThat(f.file().nullValueCounts()).hasSize(expected); // The nanValue is not counted for long and string fields - Assert.assertEquals(0, f.file().nanValueCounts().size()); + assertThat(f.file().nanValueCounts()).isEmpty(); }); } } @@ -649,36 +650,34 @@ private void verifyMaxPlanningSnapshotCountResult( Snapshot toSnapshotInclusive, Set expectedFiles) { if (fromSnapshotExclusive == null) { - Assert.assertNull(result.fromPosition().snapshotId()); - Assert.assertNull(result.fromPosition().snapshotTimestampMs()); + assertThat(result.fromPosition().snapshotId()).isNull(); + assertThat(result.fromPosition().snapshotTimestampMs()).isNull(); } else { - Assert.assertEquals( - fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - fromSnapshotExclusive.timestampMillis(), - result.fromPosition().snapshotTimestampMs().longValue()); + assertThat(result.fromPosition().snapshotId().longValue()) + .isEqualTo(fromSnapshotExclusive.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(fromSnapshotExclusive.timestampMillis()); } - Assert.assertEquals( - toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - toSnapshotInclusive.timestampMillis(), - result.toPosition().snapshotTimestampMs().longValue()); + assertThat(result.toPosition().snapshotId().longValue()) + .isEqualTo(toSnapshotInclusive.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(toSnapshotInclusive.timestampMillis()); // should only have one split with one data file, because split discover is limited to // one snapshot and each snapshot has only one data file appended. IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); - return tableResource.table().currentSnapshot(); + return TABLE_RESOURCE.table().currentSnapshot(); } private static class CycleResult { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index ce041dbbcf59..b2185675340f 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -18,58 +18,57 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Snapshot; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.RuleChain; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestRule; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousSplitPlannerImplStartStrategy { private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - public final HadoopTableResource tableResource = - new HadoopTableResource( - temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); + @TempDir protected Path temporaryFolder; + + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private GenericAppenderHelper dataAppender; private Snapshot snapshot1; private Snapshot snapshot2; private Snapshot snapshot3; - @Before + @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); } private void appendThreeSnapshots() throws IOException { List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); dataAppender.appendToTable(batch1); - snapshot1 = tableResource.table().currentSnapshot(); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); dataAppender.appendToTable(batch2); - snapshot2 = tableResource.table().currentSnapshot(); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); dataAppender.appendToTable(batch3); - snapshot3 = tableResource.table().currentSnapshot(); + snapshot3 = TABLE_RESOURCE.table().currentSnapshot(); } @Test @@ -80,14 +79,13 @@ public void testTableScanThenIncrementalStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); } @Test @@ -98,14 +96,13 @@ public void testForLatestSnapshotStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .build(); - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); } @Test @@ -116,14 +113,13 @@ public void testForEarliestSnapshotStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot1.snapshotId()); } @Test @@ -135,11 +131,10 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .startSnapshotId(1L) .build(); - // empty table assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotId)) + TABLE_RESOURCE.table(), scanContextInvalidSnapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); @@ -153,8 +148,8 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } @Test @@ -166,11 +161,10 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .startSnapshotTimestamp(1L) .build(); - // empty table assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotTimestamp)) + TABLE_RESOURCE.table(), scanContextInvalidSnapshotTimestamp)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find a snapshot after: "); @@ -184,8 +178,8 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } @Test @@ -200,7 +194,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOE .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), config).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java index e2be0b4b0398..feefcb98646b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.flink.source.enumerator; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; public class TestEnumerationHistory { private static final int MAX_HISTORY_SIZE = 3; @@ -89,28 +90,28 @@ public void testThreeMoreThanFullHistory() { } private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); if (history.hasFullHistory()) { // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); } else { // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isFalse(); } int[] historySnapshot = history.snapshot(); - Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); + assertThat(historySnapshot).containsExactly(expectedHistorySnapshot); EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); restoredHistory.restore(historySnapshot); - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); if (history.hasFullHistory()) { // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); } else { // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); + assertThat(history.shouldPauseSplitDiscovery(30)).isFalse(); } } @@ -125,10 +126,10 @@ public void testRestoreDifferentSize() { EnumerationHistory smallerHistory = new EnumerationHistory(2); smallerHistory.restore(historySnapshot); int[] expectedRestoredHistorySnapshot = {2, 3}; - Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); + assertThat(smallerHistory.snapshot()).containsExactly(expectedRestoredHistorySnapshot); EnumerationHistory largerHisotry = new EnumerationHistory(4); largerHisotry.restore(historySnapshot); - Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); + assertThat(largerHisotry.snapshot()).containsExactly(historySnapshot); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java index 1d12d9f66a8a..2520a6b763e4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java @@ -18,48 +18,48 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergEnumeratorStateSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private final IcebergEnumeratorStateSerializer serializer = new IcebergEnumeratorStateSerializer(true); - protected final int version; - - @Parameterized.Parameters(name = "version={0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } + @Parameter(index = 0) + protected int version; - public TestIcebergEnumeratorStateSerializer(int version) { - this.version = version; + @Parameters(name = "version={0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {1}, new Object[] {2}}; } - @Test + @TestTemplate public void testEmptySnapshotIdAndPendingSplits() throws Exception { IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); testSerializer(enumeratorState); } - @Test + @TestTemplate public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); @@ -69,12 +69,12 @@ public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { testSerializer(enumeratorState); } - @Test + @TestTemplate public void testSomeSnapshotIdAndPendingSplits() throws Exception { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); Collection pendingSplits = Lists.newArrayList(); pendingSplits.add( new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); @@ -87,13 +87,13 @@ public void testSomeSnapshotIdAndPendingSplits() throws Exception { testSerializer(enumeratorState); } - @Test + @TestTemplate public void testEnumerationSplitCountHistory() throws Exception { if (version == 2) { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); Collection pendingSplits = Lists.newArrayList(); pendingSplits.add( new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); @@ -123,23 +123,24 @@ private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOExc private void assertEnumeratorStateEquals( IcebergEnumeratorState expected, IcebergEnumeratorState actual) { - Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); + assertThat(actual.lastEnumeratedPosition()).isEqualTo(expected.lastEnumeratedPosition()); - Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); + assertThat(actual.pendingSplits()).hasSameSizeAs(expected.pendingSplits()); Iterator expectedIterator = expected.pendingSplits().iterator(); Iterator actualIterator = actual.pendingSplits().iterator(); for (int i = 0; i < expected.pendingSplits().size(); ++i) { IcebergSourceSplitState expectedSplitState = expectedIterator.next(); IcebergSourceSplitState actualSplitState = actualIterator.next(); - Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); - Assert.assertEquals( - expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); - Assert.assertEquals( - expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); - Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); + assertThat(actualSplitState.split().splitId()) + .isEqualTo(expectedSplitState.split().splitId()); + assertThat(actualSplitState.split().fileOffset()) + .isEqualTo(expectedSplitState.split().fileOffset()); + assertThat(actualSplitState.split().recordOffset()) + .isEqualTo(expectedSplitState.split().recordOffset()); + assertThat(actualSplitState.status()).isEqualTo(expectedSplitState.status()); } - Assert.assertArrayEquals( - expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); + assertThat(actual.enumerationSplitCountHistory()) + .containsExactly(expected.enumerationSplitCountHistory()); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java index 0d33e4ed08ad..0d1d0ce3217c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java @@ -18,11 +18,17 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.iceberg.CombinedScanTask; 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.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; @@ -30,17 +36,14 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 ReaderFunctionTestBase { - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.AVRO}, @@ -49,19 +52,17 @@ public static Object[][] parameters() { }; } - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; protected abstract ReaderFunction readerFunction(); protected abstract void assertRecords(List expected, List actual, Schema schema); - private final FileFormat fileFormat; - private final GenericAppenderFactory appenderFactory; + @Parameter(index = 0) + private FileFormat fileFormat; - public ReaderFunctionTestBase(FileFormat fileFormat) { - this.fileFormat = fileFormat; - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); private void assertRecordsAndPosition( List expectedRecords, @@ -74,24 +75,22 @@ private void assertRecordsAndPosition( RecordAndPosition recordAndPosition; while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { actualRecords.add(recordAndPosition.record()); - Assert.assertEquals( - "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); - Assert.assertEquals( - "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); + assertThat(recordAndPosition.fileOffset()).isEqualTo(expectedFileOffset); + assertThat(recordAndPosition.recordOffset() - 1).isEqualTo(recordOffset); recordOffset++; } - Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); + assertThat(actualRecords).hasSameSizeAs(expectedRecords); assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testNoCheckpointedPosition() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); CloseableIterator>> reader = readerFunction().apply(split); @@ -109,13 +108,13 @@ public void testNoCheckpointedPosition() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionBeforeFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); CloseableIterator>> reader = readerFunction().apply(split); @@ -133,13 +132,13 @@ public void testCheckpointedPositionBeforeFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionMiddleFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); CloseableIterator>> reader = readerFunction().apply(split); @@ -157,13 +156,13 @@ public void testCheckpointedPositionMiddleFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionAfterFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); CloseableIterator>> reader = readerFunction().apply(split); @@ -177,13 +176,13 @@ public void testCheckpointedPositionAfterFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionBeforeSecondFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); CloseableIterator>> reader = readerFunction().apply(split); @@ -197,13 +196,13 @@ public void testCheckpointedPositionBeforeSecondFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionMidSecondFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); CloseableIterator>> reader = readerFunction().apply(split); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index e3e341ca2c76..0edf8ae009fe 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.rules.TemporaryFolder; public class ReaderUtil { @@ -107,24 +106,6 @@ public static List> createRecordBatchList( return Lists.partition(records, batchCount); } - // Only for JUnit4 tests. Keep this method for test migration from JUnit4 to JUnit5 - public static CombinedScanTask createCombinedScanTask( - List> recordBatchList, - TemporaryFolder temporaryFolder, - FileFormat fileFormat, - GenericAppenderFactory appenderFactory) - throws IOException { - List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); - for (List recordBatch : recordBatchList) { - FileScanTask fileTask = - ReaderUtil.createFileTask( - recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); - fileTasks.add(fileTask); - } - - return new BaseCombinedScanTask(fileTasks); - } - public static CombinedScanTask createCombinedScanTask( List> recordBatchList, Path temporaryFolder, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java index 644ac2bad6b8..6f09bd9a56d6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestArrayBatchRecords { @@ -50,19 +51,19 @@ private void testArray( fileOffset, startingRecordOffset); - Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); + assertThat(recordsWithSplitIds.nextSplit()).isEqualTo(splitId); for (int i = 0; i < numberOfRecords; i++) { RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); - Assert.assertEquals(elements[i], recAndPos.record()); - Assert.assertEquals(fileOffset, recAndPos.fileOffset()); + assertThat(recAndPos.record()).isEqualTo(elements[i]); + assertThat(recAndPos.fileOffset()).isEqualTo(fileOffset); // recordOffset points to the position after this one - Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); + assertThat(recAndPos.recordOffset()).isEqualTo(startingRecordOffset + i + 1); } - Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); - Assert.assertNull(recordsWithSplitIds.nextSplit()); + assertThat(recordsWithSplitIds.nextRecordFromSplit()).isNull(); + assertThat(recordsWithSplitIds.nextSplit()).isNull(); recordsWithSplitIds.recycle(); - Assert.assertTrue(recycled.get()); + assertThat(recycled.get()).isTrue(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java index f964a7707689..1a78bb1b0010 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Path; import java.util.Arrays; import java.util.List; import org.apache.flink.configuration.Configuration; @@ -36,36 +40,33 @@ import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.DataIterator; import org.apache.iceberg.io.CloseableIterator; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestArrayPoolDataIteratorBatcherRowData { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private static final FileFormat fileFormat = FileFormat.PARQUET; - - private final GenericAppenderFactory appenderFactory; - private final DataIteratorBatcher batcher; - - public TestArrayPoolDataIteratorBatcherRowData() { - Configuration config = new Configuration(); - // set array pool size to 1 - config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); - // set batch array size to 2 - config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - this.batcher = - new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + @TempDir protected Path temporaryFolder; + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private final Configuration config = + new Configuration() + .set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1) + .set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); + + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); + private final DataIteratorBatcher batcher = + new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ @Test public void testSingleFileLessThanOneFullBatch() throws Exception { List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); String splitId = "someSplitId"; @@ -73,29 +74,27 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { batcher.batch(splitId, dataIterator); ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch.numberOfRecords()); + assertThat(batch.finishedSplits()).isEmpty(); + assertThat(batch.nextSplit()).isEqualTo(splitId); + assertThat(batch.records()).hasSize(2); + assertThat(batch.numberOfRecords()).isEqualTo(1); RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); /////////////////////////////// // assert first record - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - Assert.assertNull(batch.nextRecordFromSplit()); - Assert.assertNull(batch.nextSplit()); + assertThat(batch.nextRecordFromSplit()).isNull(); + assertThat(batch.nextSplit()).isNull(); batch.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } /** @@ -107,7 +106,11 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { public void testSingleFileWithMultipleBatches() throws Exception { List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); String splitId = "someSplitId"; @@ -118,90 +121,86 @@ public void testSingleFileWithMultipleBatches() throws Exception { // assert first batch with full batch of 2 records ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch0.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch0.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch0.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch0.numberOfRecords()); + assertThat(batch0.finishedSplits()).isEmpty(); + assertThat(batch0.nextSplit()).isEqualTo(splitId); + assertThat(batch0.records()).hasSize(2); + assertThat(batch0.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; // assert first record recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); // assert second record recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); - Assert.assertNull(batch0.nextRecordFromSplit()); - Assert.assertNull(batch0.nextSplit()); + assertThat(batch0.nextRecordFromSplit()).isNull(); + assertThat(batch0.nextSplit()).isNull(); batch0.recycle(); /////////////////////////////// // assert second batch with full batch of 2 records ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch1.records()); - Assert.assertTrue(batch1.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch1.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch1.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch1.numberOfRecords()); + assertThat(batch1.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch1.finishedSplits()).isEmpty(); + assertThat(batch1.nextSplit()).isEqualTo(splitId); + assertThat(batch1.records()).hasSize(2); + assertThat(batch1.numberOfRecords()).isEqualTo(2); // assert third record recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); // assert fourth record recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); - Assert.assertNull(batch1.nextRecordFromSplit()); - Assert.assertNull(batch1.nextSplit()); + assertThat(batch1.nextRecordFromSplit()).isNull(); + assertThat(batch1.nextSplit()).isNull(); batch1.recycle(); /////////////////////////////// // assert third batch with partial batch of 1 record ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch2.records()); - Assert.assertTrue(batch2.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch2.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch2.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch2.numberOfRecords()); + assertThat(batch2.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch2.finishedSplits()).isEmpty(); + assertThat(batch2.nextSplit()).isEqualTo(splitId); + assertThat(batch2.records()).hasSize(2); + assertThat(batch2.numberOfRecords()).isEqualTo(1); // assert fifth record recordAndPosition = batch2.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(5, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(5); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); - Assert.assertNull(batch2.nextRecordFromSplit()); - Assert.assertNull(batch2.nextSplit()); + assertThat(batch2.nextRecordFromSplit()).isNull(); + assertThat(batch2.nextSplit()).isNull(); batch2.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } /** @@ -214,20 +213,28 @@ public void testMultipleFilesWithSeekPosition() throws Exception { List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); FileScanTask fileTask0 = ReaderUtil.createFileTask( - records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records0, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); FileScanTask fileTask1 = ReaderUtil.createFileTask( - records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records1, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); FileScanTask fileTask2 = ReaderUtil.createFileTask( - records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records2, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - // seek to file1 and after record 1 dataIterator.seek(1, 1); String splitId = "someSplitId"; @@ -246,52 +253,50 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch10.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch10.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch10.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch10.numberOfRecords()); + assertThat(batch10.finishedSplits()).isEmpty(); + assertThat(batch10.nextSplit()).isEqualTo(splitId); + assertThat(batch10.records()).hasSize(2); + assertThat(batch10.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // seek should skip the first record in file1. starting from the second record - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("seek should skip the first record in file1. starting from the second record") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); - Assert.assertNull(batch10.nextRecordFromSplit()); - Assert.assertNull(batch10.nextSplit()); + assertThat(batch10.nextRecordFromSplit()).isNull(); + assertThat(batch10.nextSplit()).isNull(); batch10.recycle(); // assert second batch from file1 with partial batch of 1 record // variable naming convention: batch__ ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch11.records()); - Assert.assertTrue(batch11.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch11.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch11.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch11.numberOfRecords()); + assertThat(batch11.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch11.finishedSplits()).isEmpty(); + assertThat(batch11.nextSplit()).isEqualTo(splitId); + assertThat(batch11.records()).hasSize(2); + assertThat(batch11.numberOfRecords()).isEqualTo(1); recordAndPosition = batch11.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); - Assert.assertNull(batch11.nextRecordFromSplit()); - Assert.assertNull(batch11.nextSplit()); + assertThat(batch11.nextRecordFromSplit()).isNull(); + assertThat(batch11.nextSplit()).isNull(); batch11.recycle(); /////////////////////////////// @@ -303,29 +308,28 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch__ ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch20.records()); - Assert.assertTrue(batch20.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch20.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch20.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch20.numberOfRecords()); + assertThat(batch20.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch20.finishedSplits()).isEmpty(); + assertThat(batch20.nextSplit()).isEqualTo(splitId); + assertThat(batch20.records()).hasSize(2); + assertThat(batch20.numberOfRecords()).isEqualTo(2); recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); - Assert.assertNull(batch20.nextRecordFromSplit()); - Assert.assertNull(batch20.nextSplit()); + assertThat(batch20.nextRecordFromSplit()).isNull(); + assertThat(batch20.nextSplit()).isNull(); batch20.recycle(); /////////////////////////////// @@ -333,26 +337,24 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch__ ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch21.records()); - Assert.assertTrue(batch21.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch21.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch21.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch21.numberOfRecords()); + assertThat(batch21.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch21.finishedSplits()).isEmpty(); + assertThat(batch21.nextSplit()).isEqualTo(splitId); + assertThat(batch21.records()).hasSize(2); + assertThat(batch21.numberOfRecords()).isEqualTo(1); recordAndPosition = batch21.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); - Assert.assertNull(batch21.nextRecordFromSplit()); - Assert.assertNull(batch21.nextSplit()); + assertThat(batch21.nextRecordFromSplit()).isNull(); + assertThat(batch21.nextSplit()).isNull(); batch21.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index 790c3bba8a42..8d6782586676 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -37,19 +40,14 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.hadoop.HadoopFileIO; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceReader { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final GenericAppenderFactory appenderFactory; + @TempDir protected Path temporaryFolder; - public TestIcebergSourceReader() { - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); @Test public void testReaderMetrics() throws Exception { @@ -70,13 +68,13 @@ public void testReaderOrder() throws Exception { ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task1 = ReaderUtil.createCombinedScanTask( - recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList1, temporaryFolder, FileFormat.PARQUET, appenderFactory); List> recordBatchList2 = ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task2 = ReaderUtil.createCombinedScanTask( - recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList2, temporaryFolder, FileFormat.PARQUET, appenderFactory); // Sort the splits in one way List rowDataList1 = @@ -95,8 +93,7 @@ public void testReaderOrder() throws Exception { 2); // Check that the order of the elements is not changed - Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); - Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + assertThat(rowDataList1).containsExactlyElementsOf(rowDataList2); } private List read(List splits, long expected) throws Exception { @@ -114,7 +111,7 @@ private List read(List splits, long expected) throw reader.pollNext(readerOutput); - Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + assertThat(readerOutput.getEmittedRecords()).hasSize((int) expected); return readerOutput.getEmittedRecords(); } @@ -130,20 +127,20 @@ private void testOneSplitFetcher( ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Arrays.asList(split)); + reader.addSplits(Collections.singletonList(split)); while (readerOutput.getEmittedRecords().size() < expectedCount) { reader.pollNext(readerOutput); } - Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); + assertThat(readerOutput.getEmittedRecords()).hasSize(expectedCount); TestHelpers.assertRowData( TestFixtures.SCHEMA, recordBatchList.get(0).get(0), readerOutput.getEmittedRecords().get(expectedCount - 1)); - Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); + assertThat(metricGroup.counters().get("assignedSplits").getCount()).isEqualTo(expectedCount); // One more poll will get null record batch. // That will finish the split and cause split fetcher to be closed due to idleness. diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java index 764215e904d3..55f9c0af3a29 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -28,7 +28,6 @@ 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.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.data.Record; import org.apache.iceberg.encryption.PlaintextEncryptionManager; @@ -39,13 +38,9 @@ public class TestRowDataReaderFunction extends ReaderFunctionTestBase { - protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final DataStructureConverter rowDataConverter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - - public TestRowDataReaderFunction(FileFormat fileFormat) { - super(fileFormat); - } + protected static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final DataStructureConverter ROW_DATA_CONVERTER = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(ROW_TYPE)); @Override protected ReaderFunction readerFunction() { @@ -68,7 +63,7 @@ protected void assertRecords(List expected, List actual, Schema private List toRows(List actual) { return actual.stream() - .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) + .map(rowData -> (Row) ROW_DATA_CONVERTER.toExternal(rowData)) .collect(Collectors.toList()); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index c72d622f86ba..12bacdcd074d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -18,20 +18,21 @@ */ package org.apache.iceberg.flink.source.split; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceSplitSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); @@ -44,14 +45,14 @@ public void testLatestVersion() throws Exception { private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = serializer.serialize(split); IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); assertSplitEquals(split, deserialized); byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); + assertThat(cachedResult).isSameAs(result); IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult); assertSplitEquals(split, deserialized2); @@ -59,7 +60,7 @@ private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws E split.updatePosition(0, 100); byte[] resultAfterUpdatePosition = serializer.serialize(split); // after position change, serialized bytes should have changed - Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); + assertThat(resultAfterUpdatePosition).isNotSameAs(cachedResult); IcebergSourceSplit deserialized3 = serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); assertSplitEquals(split, deserialized3); @@ -75,7 +76,7 @@ public void testV1() throws Exception { private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV1(); IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); @@ -92,7 +93,7 @@ public void testV2() throws Exception { private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV2(); IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); @@ -109,9 +110,9 @@ private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mo throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); final List splitsWithMockDeleteFiles = - SplitHelpers.equipSplitsWithMockDeleteFiles(splits, TEMPORARY_FOLDER, mockDeletesPerSplit); + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, temporaryFolder, mockDeletesPerSplit); for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { byte[] result = split.serializeV3(); @@ -123,7 +124,7 @@ private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mo @Test public void testDeserializeV1() throws Exception { final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV1(); IcebergSourceSplit deserialized = serializer.deserialize(1, result); @@ -135,7 +136,7 @@ public void testDeserializeV1() throws Exception { public void testCheckpointedPosition() throws Exception { final AtomicInteger index = new AtomicInteger(); final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 2).stream() .map( split -> { IcebergSourceSplit result; @@ -155,7 +156,7 @@ public void testCheckpointedPosition() throws Exception { assertSplitEquals(split, deserialized); byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); + assertThat(cachedResult).isSameAs(result); IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult); assertSplitEquals(split, deserialized2); @@ -165,18 +166,18 @@ public void testCheckpointedPosition() throws Exception { private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); - Assert.assertEquals(expectedTasks.size(), actualTasks.size()); + assertThat(actualTasks).hasSameSizeAs(expectedTasks); for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); - Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); - Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); - Assert.assertEquals(expectedTask.start(), actualTask.start()); - Assert.assertEquals(expectedTask.length(), actualTask.length()); + assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); + assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); + assertThat(actualTask.start()).isEqualTo(expectedTask.start()); + assertThat(actualTask.length()).isEqualTo(expectedTask.length()); } - Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); - Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); + assertThat(actual.fileOffset()).isEqualTo(expected.fileOffset()); + assertThat(actual.recordOffset()).isEqualTo(expected.recordOffset()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java index ebd220b00dba..540902f3cea5 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -18,11 +18,13 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; import static org.mockito.Mockito.doReturn; import static org.mockito.Mockito.spy; 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.hadoop.conf.Configuration; @@ -48,8 +50,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.ThreadPools; -import org.junit.Assert; -import org.junit.rules.TemporaryFolder; public class SplitHelpers { @@ -67,14 +67,14 @@ private SplitHelpers() {} * attempt to read the data files. * *

    By default, v1 Iceberg table is created. For v2 table use {@link - * SplitHelpers#createSplitsFromTransientHadoopTable(TemporaryFolder, int, int, String)} + * SplitHelpers#createSplitsFromTransientHadoopTable(Path, int, int, String)} * * @param temporaryFolder Folder to place the data to * @param fileCount The number of files to create and add to the table * @param filesPerSplit The number of files used for a split */ public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + Path temporaryFolder, int fileCount, int filesPerSplit) throws Exception { return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); } @@ -95,10 +95,9 @@ public static List createSplitsFromTransientHadoopTable( * @param version The table version to create */ public static List createSplitsFromTransientHadoopTable( - TemporaryFolder temporaryFolder, int fileCount, int filesPerSplit, String version) - throws Exception { - final File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); + Path temporaryFolder, int fileCount, int filesPerSplit, String version) throws Exception { + final File warehouseFile = File.createTempFile("junit", null, temporaryFolder.toFile()); + assertThat(warehouseFile.delete()).isTrue(); final String warehouse = "file:" + warehouseFile; Configuration hadoopConf = new Configuration(); final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); @@ -155,9 +154,7 @@ public static List createSplitsFromTransientHadoopTable( * @throws IOException If there is any error creating the mock delete files */ public static List equipSplitsWithMockDeleteFiles( - List icebergSourceSplits, - TemporaryFolder temporaryFolder, - int deleteFilesPerSplit) + List icebergSourceSplits, Path temporaryFolder, int deleteFilesPerSplit) throws IOException { List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); for (IcebergSourceSplit split : icebergSourceSplits) { @@ -171,7 +168,7 @@ public static List equipSplitsWithMockDeleteFiles( final DeleteFile deleteFile = FileMetadata.deleteFileBuilder(spec) .withFormat(FileFormat.PARQUET) - .withPath(temporaryFolder.newFile().getPath()) + .withPath(File.createTempFile("junit", null, temporaryFolder.toFile()).getPath()) .ofPositionDeletes() .withFileSizeInBytes(1000) .withRecordCount(1000) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java index 090b304942c6..1e612b0a2b2a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +import java.nio.file.Path; import java.util.Collection; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -25,13 +29,11 @@ import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public abstract class SplitAssignerTestBase { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; @Test public void testEmptyInitialization() { @@ -86,11 +88,11 @@ private void assertAvailableFuture( // calling isAvailable again should return the same object reference // note that thenAccept will return a new future. // we want to assert the same instance on the assigner returned future - Assert.assertSame(future, assigner.isAvailable()); + assertThat(assigner.isAvailable()).isSameAs(future); // now add some splits addSplitsRunnable.run(); - Assert.assertEquals(true, futureCompleted.get()); + assertThat(futureCompleted.get()).isTrue(); for (int i = 0; i < splitCount; ++i) { assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); @@ -101,29 +103,29 @@ private void assertAvailableFuture( protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(expectedStatus, result.status()); + assertThat(result.status()).isEqualTo(expectedStatus); switch (expectedStatus) { case AVAILABLE: - Assert.assertNotNull(result.split()); + assertThat(result.split()).isNotNull(); break; case CONSTRAINED: case UNAVAILABLE: - Assert.assertNull(result.split()); + assertThat(result.split()).isNull(); break; default: - Assert.fail("Unknown status: " + expectedStatus); + fail("Unknown status: %s", expectedStatus); } } protected void assertSnapshot(SplitAssigner assigner, int splitCount) { Collection stateBeforeGet = assigner.state(); - Assert.assertEquals(splitCount, stateBeforeGet.size()); + assertThat(stateBeforeGet).hasSize(splitCount); } protected List createSplits(int fileCount, int filesPerSplit, String version) throws Exception { return SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, fileCount, filesPerSplit, version); + temporaryFolder, fileCount, filesPerSplit, version); } protected abstract SplitAssigner splitAssigner(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java index 8994f3054abe..17e64bbf0594 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -19,7 +19,7 @@ package org.apache.iceberg.flink.source.assigner; import org.apache.iceberg.flink.source.SplitHelpers; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestDefaultSplitAssigner extends SplitAssignerTestBase { @Override @@ -32,7 +32,7 @@ protected SplitAssigner splitAssigner() { public void testMultipleFilesInASplit() throws Exception { SplitAssigner assigner = splitAssigner(); assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 4, 2)); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 4, 2)); assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); assertSnapshot(assigner, 1); 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 20555ad464a9..ff63ba8e58a0 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.assigner; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.util.List; @@ -26,8 +27,7 @@ import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { @Override @@ -70,12 +70,12 @@ public void testSerializable() { byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); SerializableComparator comparator = SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); + assertThat(comparator).isNotNull(); } private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { GetSplitResult result = assigner.getNext(null); ContentFile file = result.split().task().files().iterator().next().file(); - Assert.assertEquals(expectedSequenceNumber, file.fileSequenceNumber()); + assertThat(file.fileSequenceNumber()).isEqualTo(expectedSequenceNumber); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java index 6c0cbb733320..84f04d5a530a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.assigner; import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Instant; @@ -44,8 +45,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.SerializationUtil; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { public static final Schema SCHEMA = @@ -104,12 +104,12 @@ public void testSerializable() { TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); SerializableComparator comparator = SerializationUtil.deserializeFromBytes(bytes); - Assert.assertNotNull(comparator); + assertThat(comparator).isNotNull(); } private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { GetSplitResult result = assigner.getNext(null); - Assert.assertEquals(result.split(), split); + assertThat(split).isEqualTo(result.split()); } @Override @@ -138,7 +138,7 @@ private IcebergSourceSplit splitFromRecords(List> records) { try { return IcebergSourceSplit.fromCombinedScanTask( ReaderUtil.createCombinedScanTask( - records, TEMPORARY_FOLDER, FileFormat.PARQUET, APPENDER_FACTORY)); + records, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); } catch (IOException e) { throw new RuntimeException("Split creation exception", e); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 95770bb7dc2f..41a787762fda 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.util.Arrays; import java.util.Collection; import java.util.Collections; @@ -36,13 +37,11 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousIcebergEnumerator { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; @Test public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { @@ -59,19 +58,19 @@ public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { Collection pendingSplitsEmpty = enumerator.snapshotState(1).pendingSplits(); - Assert.assertEquals(0, pendingSplitsEmpty.size()); + assertThat(pendingSplitsEmpty).isEmpty(); // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } @Test @@ -94,11 +93,11 @@ public void testDiscoverWhenReaderRegistered() throws Exception { // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); - Assert.assertTrue(enumerator.snapshotState(1).pendingSplits().isEmpty()); + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -126,26 +125,25 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio // make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); - Assert.assertEquals(1, splits.size()); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + assertThat(splits).hasSize(1); splitPlanner.addSplits(splits); enumeratorContext.triggerAllActions(); - Assert.assertFalse(enumeratorContext.getSplitAssignments().containsKey(2)); + assertThat(enumeratorContext.getSplitAssignments()).doesNotContainKey(2); List pendingSplitIds = enumerator.snapshotState(1).pendingSplits().stream() .map(IcebergSourceSplitState::split) .map(IcebergSourceSplit::splitId) .collect(Collectors.toList()); - Assert.assertEquals(splits.size(), pendingSplitIds.size()); - Assert.assertEquals(splits.get(0).splitId(), pendingSplitIds.get(0)); + assertThat(pendingSplitIds).hasSameSizeAs(splits).first().isEqualTo(splits.get(0).splitId()); // register the reader again, and let it request a split enumeratorContext.registerReader(2, "localhost"); enumerator.addReader(2); enumerator.handleSourceEvent(2, new SplitRequestEvent()); - Assert.assertTrue(enumerator.snapshotState(2).pendingSplits().isEmpty()); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) .contains(splits.get(0)); } @@ -154,7 +152,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio public void testThrottlingDiscovery() throws Exception { // create 10 splits List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 1); TestingSplitEnumeratorContext enumeratorContext = new TestingSplitEnumeratorContext<>(4); @@ -179,10 +177,10 @@ public void testThrottlingDiscovery() throws Exception { enumeratorContext.triggerAllActions(); // because discovered split was assigned to reader, pending splits should be empty - Assert.assertEquals(0, enumerator.snapshotState(1).pendingSplits().size()); + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); // split assignment to reader-2 should contain splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); // add the remaining 9 splits (one for every snapshot) // run discovery cycles while reader-2 still processing the splits[0] @@ -192,20 +190,20 @@ public void testThrottlingDiscovery() throws Exception { } // can only discover up to 3 snapshots/splits - Assert.assertEquals(3, enumerator.snapshotState(2).pendingSplits().size()); + assertThat(enumerator.snapshotState(2).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 1) - Assert.assertEquals( - splits.subList(0, 1), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); // now reader-2 finished splits[0] enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(3).pendingSplits().size()); + assertThat(enumerator.snapshotState(3).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); // run 3 more split discovery cycles for (int i = 0; i < 3; ++i) { @@ -213,20 +211,20 @@ public void testThrottlingDiscovery() throws Exception { } // no more splits are discovered due to throttling - Assert.assertEquals(3, enumerator.snapshotState(4).pendingSplits().size()); + assertThat(enumerator.snapshotState(4).pendingSplits()).hasSize(3); // split assignment to reader-2 should still be splits[0, 2) - Assert.assertEquals( - splits.subList(0, 2), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); // now reader-2 finished splits[1] enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); enumeratorContext.triggerAllActions(); // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was // discovered and added. - Assert.assertEquals(3, enumerator.snapshotState(5).pendingSplits().size()); + assertThat(enumerator.snapshotState(5).pendingSplits()).hasSize(3); // split assignment to reader-2 should be splits[0, 3) - Assert.assertEquals( - splits.subList(0, 3), enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 3)); } @Test @@ -246,20 +244,20 @@ public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); // Trigger a planning and check that no splits returned due to the planning error enumeratorContext.triggerAllActions(); - Assert.assertEquals(0, enumerator.snapshotState(2).pendingSplits().size()); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); // Second scan planning should succeed and discover the expected splits enumeratorContext.triggerAllActions(); Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } @Test @@ -278,19 +276,19 @@ public void testOverMaxAllowedPlanningErrors() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); // Check that the scheduler response ignores the current error and continues to run until the // failure limit is reached enumeratorContext.triggerAllActions(); - Assert.assertFalse( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isFalse(); // Check that the task has failed with the expected exception after the failure limit is reached enumeratorContext.triggerAllActions(); - Assert.assertTrue( - enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isTrue(); assertThatThrownBy( () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) .hasCauseInstanceOf(RuntimeException.class) @@ -316,7 +314,7 @@ public void testPlanningIgnoringErrors() throws Exception { // Make one split available and trigger the periodic discovery List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); splitPlanner.addSplits(splits); Collection pendingSplits; @@ -324,16 +322,16 @@ public void testPlanningIgnoringErrors() throws Exception { for (int i = 0; i < expectedFailures; ++i) { enumeratorContext.triggerAllActions(); pendingSplits = enumerator.snapshotState(i).pendingSplits(); - Assert.assertEquals(0, pendingSplits.size()); + assertThat(pendingSplits).isEmpty(); } // Discovered the new split after a successful scan planning enumeratorContext.triggerAllActions(); pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); - Assert.assertEquals(1, pendingSplits.size()); + assertThat(pendingSplits).hasSize(1); IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - Assert.assertEquals(splits.get(0).splitId(), pendingSplit.split().splitId()); - Assert.assertEquals(IcebergSourceSplitStatus.UNASSIGNED, pendingSplit.status()); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); } private static ContinuousIcebergEnumerator createEnumerator( 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 f98e93835d15..381c340edd6a 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 @@ -18,9 +18,11 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import java.util.Set; import java.util.concurrent.atomic.AtomicLong; @@ -31,33 +33,27 @@ import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.junit.Assert; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestName; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousSplitPlannerImpl { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private static final FileFormat fileFormat = FileFormat.PARQUET; private static final AtomicLong randomSeed = new AtomicLong(); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - @Rule public TestName testName = new TestName(); + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private GenericAppenderHelper dataAppender; private DataFile dataFile1; @@ -65,9 +61,9 @@ public class TestContinuousSplitPlannerImpl { private DataFile dataFile2; private Snapshot snapshot2; - @Before + @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), fileFormat, TEMPORARY_FOLDER); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), fileFormat, temporaryFolder); } private void appendTwoSnapshots() throws IOException { @@ -75,13 +71,13 @@ private void appendTwoSnapshots() throws IOException { List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); dataFile1 = dataAppender.writeFile(null, batch1); dataAppender.appendToTable(dataFile1); - snapshot1 = tableResource.table().currentSnapshot(); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); // snapshot2 List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); dataFile2 = dataAppender.writeFile(null, batch2); dataAppender.appendToTable(dataFile2); - snapshot2 = tableResource.table().currentSnapshot(); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } /** @return the last enumerated snapshot id */ @@ -92,21 +88,22 @@ private CycleResult verifyOneCycle( RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); - Snapshot snapshot = tableResource.table().currentSnapshot(); + Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); - Assert.assertEquals(lastPosition.snapshotId(), result.fromPosition().snapshotId()); - Assert.assertEquals( - lastPosition.snapshotTimestampMs(), result.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot.timestampMillis(), result.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, result.splits().size()); + assertThat(result.fromPosition().snapshotId()).isEqualTo(lastPosition.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs()) + .isEqualTo(lastPosition.snapshotTimestampMs()); + assertThat(result.toPosition().snapshotId().longValue()).isEqualTo(snapshot.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot.timestampMillis()); + assertThat(result.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); - Assert.assertEquals( - dataFile.path().toString(), - Iterables.getOnlyElement(split.task().files()).file().path().toString()); + assertThat(split.task().files()) + .hasSize(1) + .first() + .satisfies( + fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); return new CycleResult(result.toPosition(), split); } @@ -117,21 +114,21 @@ public void testTableScanThenIncrementalWithEmptyTable() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); @@ -149,24 +146,24 @@ public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); - Assert.assertEquals( - snapshot2.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.fromPosition()).isNull(); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); Set expectedFiles = ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -182,27 +179,27 @@ public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { .splitSize(1L) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertTrue(emptyTableInitialDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.toPosition().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // latest mode should discover both snapshots, as latest position is marked by when job starts appendTwoSnapshots(); ContinuousEnumerationResult afterTwoSnapshotsAppended = splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); - Assert.assertEquals(2, afterTwoSnapshotsAppended.splits().size()); + assertThat(afterTwoSnapshotsAppended.splits()).hasSize(2); // next 3 snapshots IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); @@ -220,35 +217,36 @@ public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exceptio .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1 // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -263,21 +261,21 @@ public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - Assert.assertTrue(emptyTableInitialDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableInitialDiscoveryResult.fromPosition()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); ContinuousEnumerationResult emptyTableSecondDiscoveryResult = splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - Assert.assertTrue(emptyTableSecondDiscoveryResult.splits().isEmpty()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotId()); - Assert.assertNull(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); // next 3 snapshots IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); @@ -295,24 +293,25 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1's parent, // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertNull(secondResult.fromPosition().snapshotId()); - Assert.assertNull(secondResult.fromPosition().snapshotTimestampMs()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId()).isNull(); + assertThat(secondResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) @@ -320,7 +319,7 @@ public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Except // should discover files appended in both snapshot1 and snapshot2 Set expectedFiles = ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -337,7 +336,7 @@ public void testIncrementalFromSnapshotIdWithEmptyTable() { .build(); ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -363,7 +362,7 @@ public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -380,35 +379,36 @@ public void testIncrementalFromSnapshotId() throws Exception { .startSnapshotId(snapshot2.snapshotId()) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as // snapshot2's parent) - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -425,7 +425,7 @@ public void testIncrementalFromSnapshotTimestampWithEmptyTable() { .build(); ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -446,7 +446,7 @@ public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exceptio ContinuousSplitPlannerImpl splitPlanner = new ContinuousSplitPlannerImpl( - tableResource.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); assertThatThrownBy(() -> splitPlanner.planSplits(null)) .isInstanceOf(IllegalArgumentException.class) @@ -463,34 +463,35 @@ public void testIncrementalFromSnapshotTimestamp() throws Exception { .startSnapshotTimestamp(snapshot2.timestampMillis()) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). - Assert.assertEquals( - snapshot1.snapshotId(), initialResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), initialResult.toPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - Assert.assertEquals( - snapshot1.snapshotId(), secondResult.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot1.timestampMillis(), secondResult.fromPosition().snapshotTimestampMs().longValue()); - Assert.assertEquals(snapshot2.snapshotId(), secondResult.toPosition().snapshotId().longValue()); - Assert.assertEquals( - snapshot2.timestampMillis(), secondResult.toPosition().snapshotTimestampMs().longValue()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); // should discover dataFile2 appended in snapshot2 Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); for (int i = 0; i < 3; ++i) { @@ -513,15 +514,15 @@ public void testMaxPlanningSnapshotCount() throws Exception { .maxPlanningSnapshotCount(1) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertNull(initialResult.fromPosition()); + assertThat(initialResult.fromPosition()).isNull(); // For inclusive behavior, the initial result should point to snapshot1's parent, // which leads to null snapshotId and snapshotTimestampMs. - Assert.assertNull(initialResult.toPosition().snapshotId()); - Assert.assertNull(initialResult.toPosition().snapshotTimestampMs()); - Assert.assertEquals(0, initialResult.splits().size()); + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); // should discover dataFile1 appended in snapshot1 @@ -544,12 +545,12 @@ public void testTableScanNoStats() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 0); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -570,12 +571,12 @@ public void testTableScanAllStats() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 3); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -596,12 +597,12 @@ public void testTableScanSingleStat() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(tableResource.tableLoader().clone(), scanContext, null); + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - Assert.assertEquals(1, initialResult.splits().size()); + assertThat(initialResult.splits()).hasSize(1); IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - Assert.assertEquals(2, split.task().files().size()); + assertThat(split.task().files()).hasSize(2); verifyStatCount(split, 1); IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); @@ -619,12 +620,12 @@ private void verifyStatCount(IcebergSourceSplit split, int expected) { .files() .forEach( f -> { - Assert.assertNull(f.file().valueCounts()); - Assert.assertNull(f.file().columnSizes()); - Assert.assertNull(f.file().lowerBounds()); - Assert.assertNull(f.file().upperBounds()); - Assert.assertNull(f.file().nanValueCounts()); - Assert.assertNull(f.file().nullValueCounts()); + assertThat(f.file().valueCounts()).isNull(); + assertThat(f.file().columnSizes()).isNull(); + assertThat(f.file().lowerBounds()).isNull(); + assertThat(f.file().upperBounds()).isNull(); + assertThat(f.file().nanValueCounts()).isNull(); + assertThat(f.file().nullValueCounts()).isNull(); }); } else { split @@ -632,13 +633,13 @@ private void verifyStatCount(IcebergSourceSplit split, int expected) { .files() .forEach( f -> { - Assert.assertEquals(expected, f.file().valueCounts().size()); - Assert.assertEquals(expected, f.file().columnSizes().size()); - Assert.assertEquals(expected, f.file().lowerBounds().size()); - Assert.assertEquals(expected, f.file().upperBounds().size()); - Assert.assertEquals(expected, f.file().nullValueCounts().size()); + assertThat(f.file().valueCounts()).hasSize(expected); + assertThat(f.file().columnSizes()).hasSize(expected); + assertThat(f.file().lowerBounds()).hasSize(expected); + assertThat(f.file().upperBounds()).hasSize(expected); + assertThat(f.file().nullValueCounts()).hasSize(expected); // The nanValue is not counted for long and string fields - Assert.assertEquals(0, f.file().nanValueCounts().size()); + assertThat(f.file().nanValueCounts()).isEmpty(); }); } } @@ -649,36 +650,34 @@ private void verifyMaxPlanningSnapshotCountResult( Snapshot toSnapshotInclusive, Set expectedFiles) { if (fromSnapshotExclusive == null) { - Assert.assertNull(result.fromPosition().snapshotId()); - Assert.assertNull(result.fromPosition().snapshotTimestampMs()); + assertThat(result.fromPosition().snapshotId()).isNull(); + assertThat(result.fromPosition().snapshotTimestampMs()).isNull(); } else { - Assert.assertEquals( - fromSnapshotExclusive.snapshotId(), result.fromPosition().snapshotId().longValue()); - Assert.assertEquals( - fromSnapshotExclusive.timestampMillis(), - result.fromPosition().snapshotTimestampMs().longValue()); + assertThat(result.fromPosition().snapshotId().longValue()) + .isEqualTo(fromSnapshotExclusive.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(fromSnapshotExclusive.timestampMillis()); } - Assert.assertEquals( - toSnapshotInclusive.snapshotId(), result.toPosition().snapshotId().longValue()); - Assert.assertEquals( - toSnapshotInclusive.timestampMillis(), - result.toPosition().snapshotTimestampMs().longValue()); + assertThat(result.toPosition().snapshotId().longValue()) + .isEqualTo(toSnapshotInclusive.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(toSnapshotInclusive.timestampMillis()); // should only have one split with one data file, because split discover is limited to // one snapshot and each snapshot has only one data file appended. IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - Assert.assertEquals(1, split.task().files().size()); + assertThat(split.task().files()).hasSize(1); Set discoveredFiles = split.task().files().stream() .map(fileScanTask -> fileScanTask.file().path().toString()) .collect(Collectors.toSet()); - Assert.assertEquals(expectedFiles, discoveredFiles); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); } private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); - return tableResource.table().currentSnapshot(); + return TABLE_RESOURCE.table().currentSnapshot(); } private static class CycleResult { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index ce041dbbcf59..b2185675340f 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -18,58 +18,57 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import java.io.IOException; +import java.nio.file.Path; import java.util.List; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Snapshot; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.ScanContext; import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.RuleChain; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.TestRule; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestContinuousSplitPlannerImplStartStrategy { private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - public final TemporaryFolder temporaryFolder = new TemporaryFolder(); - public final HadoopTableResource tableResource = - new HadoopTableResource( - temporaryFolder, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - @Rule public final TestRule chain = RuleChain.outerRule(temporaryFolder).around(tableResource); + @TempDir protected Path temporaryFolder; + + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private GenericAppenderHelper dataAppender; private Snapshot snapshot1; private Snapshot snapshot2; private Snapshot snapshot3; - @Before + @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(tableResource.table(), FILE_FORMAT, temporaryFolder); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); } private void appendThreeSnapshots() throws IOException { List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); dataAppender.appendToTable(batch1); - snapshot1 = tableResource.table().currentSnapshot(); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); dataAppender.appendToTable(batch2); - snapshot2 = tableResource.table().currentSnapshot(); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); dataAppender.appendToTable(batch3); - snapshot3 = tableResource.table().currentSnapshot(); + snapshot3 = TABLE_RESOURCE.table().currentSnapshot(); } @Test @@ -80,14 +79,13 @@ public void testTableScanThenIncrementalStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); } @Test @@ -98,14 +96,13 @@ public void testForLatestSnapshotStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .build(); - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot3.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); } @Test @@ -116,14 +113,13 @@ public void testForEarliestSnapshotStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); - // empty table - Assert.assertFalse( - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).isPresent()); + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); appendThreeSnapshots(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot1.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot1.snapshotId()); } @Test @@ -135,11 +131,10 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .startSnapshotId(1L) .build(); - // empty table assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotId)) + TABLE_RESOURCE.table(), scanContextInvalidSnapshotId)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Start snapshot id not found in history: 1"); @@ -153,8 +148,8 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } @Test @@ -166,11 +161,10 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .startSnapshotTimestamp(1L) .build(); - // empty table assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( - tableResource.table(), scanContextInvalidSnapshotTimestamp)) + TABLE_RESOURCE.table(), scanContextInvalidSnapshotTimestamp)) .isInstanceOf(IllegalArgumentException.class) .hasMessageStartingWith("Cannot find a snapshot after: "); @@ -184,8 +178,8 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), scanContext).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } @Test @@ -200,7 +194,7 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOE .build(); Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(tableResource.table(), config).get(); - Assert.assertEquals(snapshot2.snapshotId(), startSnapshot.snapshotId()); + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), config).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java index e2be0b4b0398..feefcb98646b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java @@ -18,8 +18,9 @@ */ package org.apache.iceberg.flink.source.enumerator; -import org.junit.Assert; -import org.junit.Test; +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; public class TestEnumerationHistory { private static final int MAX_HISTORY_SIZE = 3; @@ -89,28 +90,28 @@ public void testThreeMoreThanFullHistory() { } private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); if (history.hasFullHistory()) { // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); } else { // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isFalse(); } int[] historySnapshot = history.snapshot(); - Assert.assertArrayEquals(expectedHistorySnapshot, historySnapshot); + assertThat(historySnapshot).containsExactly(expectedHistorySnapshot); EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); restoredHistory.restore(historySnapshot); - Assert.assertFalse(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); if (history.hasFullHistory()) { // throttle because pending split count is more than the sum of enumeration history - Assert.assertTrue(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)); + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); } else { // skipped throttling check because there is not enough history - Assert.assertFalse(history.shouldPauseSplitDiscovery(30)); + assertThat(history.shouldPauseSplitDiscovery(30)).isFalse(); } } @@ -125,10 +126,10 @@ public void testRestoreDifferentSize() { EnumerationHistory smallerHistory = new EnumerationHistory(2); smallerHistory.restore(historySnapshot); int[] expectedRestoredHistorySnapshot = {2, 3}; - Assert.assertArrayEquals(expectedRestoredHistorySnapshot, smallerHistory.snapshot()); + assertThat(smallerHistory.snapshot()).containsExactly(expectedRestoredHistorySnapshot); EnumerationHistory largerHisotry = new EnumerationHistory(4); largerHisotry.restore(historySnapshot); - Assert.assertArrayEquals(historySnapshot, largerHisotry.snapshot()); + assertThat(largerHisotry.snapshot()).containsExactly(historySnapshot); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java index 1d12d9f66a8a..2520a6b763e4 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java @@ -18,48 +18,48 @@ */ package org.apache.iceberg.flink.source.enumerator; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergEnumeratorStateSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private final IcebergEnumeratorStateSerializer serializer = new IcebergEnumeratorStateSerializer(true); - protected final int version; - - @Parameterized.Parameters(name = "version={0}") - public static Object[] parameters() { - return new Object[] {1, 2}; - } + @Parameter(index = 0) + protected int version; - public TestIcebergEnumeratorStateSerializer(int version) { - this.version = version; + @Parameters(name = "version={0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {1}, new Object[] {2}}; } - @Test + @TestTemplate public void testEmptySnapshotIdAndPendingSplits() throws Exception { IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); testSerializer(enumeratorState); } - @Test + @TestTemplate public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); @@ -69,12 +69,12 @@ public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { testSerializer(enumeratorState); } - @Test + @TestTemplate public void testSomeSnapshotIdAndPendingSplits() throws Exception { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); Collection pendingSplits = Lists.newArrayList(); pendingSplits.add( new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); @@ -87,13 +87,13 @@ public void testSomeSnapshotIdAndPendingSplits() throws Exception { testSerializer(enumeratorState); } - @Test + @TestTemplate public void testEnumerationSplitCountHistory() throws Exception { if (version == 2) { IcebergEnumeratorPosition position = IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 3, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); Collection pendingSplits = Lists.newArrayList(); pendingSplits.add( new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); @@ -123,23 +123,24 @@ private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOExc private void assertEnumeratorStateEquals( IcebergEnumeratorState expected, IcebergEnumeratorState actual) { - Assert.assertEquals(expected.lastEnumeratedPosition(), actual.lastEnumeratedPosition()); + assertThat(actual.lastEnumeratedPosition()).isEqualTo(expected.lastEnumeratedPosition()); - Assert.assertEquals(expected.pendingSplits().size(), actual.pendingSplits().size()); + assertThat(actual.pendingSplits()).hasSameSizeAs(expected.pendingSplits()); Iterator expectedIterator = expected.pendingSplits().iterator(); Iterator actualIterator = actual.pendingSplits().iterator(); for (int i = 0; i < expected.pendingSplits().size(); ++i) { IcebergSourceSplitState expectedSplitState = expectedIterator.next(); IcebergSourceSplitState actualSplitState = actualIterator.next(); - Assert.assertEquals(expectedSplitState.split().splitId(), actualSplitState.split().splitId()); - Assert.assertEquals( - expectedSplitState.split().fileOffset(), actualSplitState.split().fileOffset()); - Assert.assertEquals( - expectedSplitState.split().recordOffset(), actualSplitState.split().recordOffset()); - Assert.assertEquals(expectedSplitState.status(), actualSplitState.status()); + assertThat(actualSplitState.split().splitId()) + .isEqualTo(expectedSplitState.split().splitId()); + assertThat(actualSplitState.split().fileOffset()) + .isEqualTo(expectedSplitState.split().fileOffset()); + assertThat(actualSplitState.split().recordOffset()) + .isEqualTo(expectedSplitState.split().recordOffset()); + assertThat(actualSplitState.status()).isEqualTo(expectedSplitState.status()); } - Assert.assertArrayEquals( - expected.enumerationSplitCountHistory(), actual.enumerationSplitCountHistory()); + assertThat(actual.enumerationSplitCountHistory()) + .containsExactly(expected.enumerationSplitCountHistory()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java index 0d33e4ed08ad..0d1d0ce3217c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java @@ -18,11 +18,17 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Path; import java.util.List; import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; import org.apache.iceberg.CombinedScanTask; 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.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; @@ -30,17 +36,14 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.CloseableIterator; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 ReaderFunctionTestBase { - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Object[][] parameters() { return new Object[][] { new Object[] {FileFormat.AVRO}, @@ -49,19 +52,17 @@ public static Object[][] parameters() { }; } - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; protected abstract ReaderFunction readerFunction(); protected abstract void assertRecords(List expected, List actual, Schema schema); - private final FileFormat fileFormat; - private final GenericAppenderFactory appenderFactory; + @Parameter(index = 0) + private FileFormat fileFormat; - public ReaderFunctionTestBase(FileFormat fileFormat) { - this.fileFormat = fileFormat; - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); private void assertRecordsAndPosition( List expectedRecords, @@ -74,24 +75,22 @@ private void assertRecordsAndPosition( RecordAndPosition recordAndPosition; while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { actualRecords.add(recordAndPosition.record()); - Assert.assertEquals( - "expected file offset", expectedFileOffset, recordAndPosition.fileOffset()); - Assert.assertEquals( - "expected record offset", recordOffset, recordAndPosition.recordOffset() - 1); + assertThat(recordAndPosition.fileOffset()).isEqualTo(expectedFileOffset); + assertThat(recordAndPosition.recordOffset() - 1).isEqualTo(recordOffset); recordOffset++; } - Assert.assertEquals("expected record count", expectedRecords.size(), actualRecords.size()); + assertThat(actualRecords).hasSameSizeAs(expectedRecords); assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testNoCheckpointedPosition() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); CloseableIterator>> reader = readerFunction().apply(split); @@ -109,13 +108,13 @@ public void testNoCheckpointedPosition() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionBeforeFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); CloseableIterator>> reader = readerFunction().apply(split); @@ -133,13 +132,13 @@ public void testCheckpointedPositionBeforeFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionMiddleFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); CloseableIterator>> reader = readerFunction().apply(split); @@ -157,13 +156,13 @@ public void testCheckpointedPositionMiddleFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionAfterFirstFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); CloseableIterator>> reader = readerFunction().apply(split); @@ -177,13 +176,13 @@ public void testCheckpointedPositionAfterFirstFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionBeforeSecondFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); CloseableIterator>> reader = readerFunction().apply(split); @@ -197,13 +196,13 @@ public void testCheckpointedPositionBeforeSecondFile() throws IOException { batch2.recycle(); } - @Test + @TestTemplate public void testCheckpointedPositionMidSecondFile() throws IOException { List> recordBatchList = ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); CombinedScanTask combinedScanTask = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, fileFormat, appenderFactory); + recordBatchList, temporaryFolder, fileFormat, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); CloseableIterator>> reader = readerFunction().apply(split); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java index e3e341ca2c76..0edf8ae009fe 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -49,7 +49,6 @@ import org.apache.iceberg.io.FileAppender; import org.apache.iceberg.io.FileAppenderFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.rules.TemporaryFolder; public class ReaderUtil { @@ -107,24 +106,6 @@ public static List> createRecordBatchList( return Lists.partition(records, batchCount); } - // Only for JUnit4 tests. Keep this method for test migration from JUnit4 to JUnit5 - public static CombinedScanTask createCombinedScanTask( - List> recordBatchList, - TemporaryFolder temporaryFolder, - FileFormat fileFormat, - GenericAppenderFactory appenderFactory) - throws IOException { - List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); - for (List recordBatch : recordBatchList) { - FileScanTask fileTask = - ReaderUtil.createFileTask( - recordBatch, temporaryFolder.newFile(), fileFormat, appenderFactory); - fileTasks.add(fileTask); - } - - return new BaseCombinedScanTask(fileTasks); - } - public static CombinedScanTask createCombinedScanTask( List> recordBatchList, Path temporaryFolder, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java index 644ac2bad6b8..6f09bd9a56d6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java @@ -18,9 +18,10 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestArrayBatchRecords { @@ -50,19 +51,19 @@ private void testArray( fileOffset, startingRecordOffset); - Assert.assertEquals(splitId, recordsWithSplitIds.nextSplit()); + assertThat(recordsWithSplitIds.nextSplit()).isEqualTo(splitId); for (int i = 0; i < numberOfRecords; i++) { RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); - Assert.assertEquals(elements[i], recAndPos.record()); - Assert.assertEquals(fileOffset, recAndPos.fileOffset()); + assertThat(recAndPos.record()).isEqualTo(elements[i]); + assertThat(recAndPos.fileOffset()).isEqualTo(fileOffset); // recordOffset points to the position after this one - Assert.assertEquals(startingRecordOffset + i + 1, recAndPos.recordOffset()); + assertThat(recAndPos.recordOffset()).isEqualTo(startingRecordOffset + i + 1); } - Assert.assertNull(recordsWithSplitIds.nextRecordFromSplit()); - Assert.assertNull(recordsWithSplitIds.nextSplit()); + assertThat(recordsWithSplitIds.nextRecordFromSplit()).isNull(); + assertThat(recordsWithSplitIds.nextSplit()).isNull(); recordsWithSplitIds.recycle(); - Assert.assertTrue(recycled.get()); + assertThat(recycled.get()).isTrue(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java index f964a7707689..1a78bb1b0010 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Path; import java.util.Arrays; import java.util.List; import org.apache.flink.configuration.Configuration; @@ -36,36 +40,33 @@ import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.DataIterator; import org.apache.iceberg.io.CloseableIterator; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestArrayPoolDataIteratorBatcherRowData { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - private static final FileFormat fileFormat = FileFormat.PARQUET; - - private final GenericAppenderFactory appenderFactory; - private final DataIteratorBatcher batcher; - - public TestArrayPoolDataIteratorBatcherRowData() { - Configuration config = new Configuration(); - // set array pool size to 1 - config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); - // set batch array size to 2 - config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - this.batcher = - new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + @TempDir protected Path temporaryFolder; + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private final Configuration config = + new Configuration() + .set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1) + .set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); + + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); + private final DataIteratorBatcher batcher = + new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ @Test public void testSingleFileLessThanOneFullBatch() throws Exception { List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); String splitId = "someSplitId"; @@ -73,29 +74,27 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { batcher.batch(splitId, dataIterator); ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch.numberOfRecords()); + assertThat(batch.finishedSplits()).isEmpty(); + assertThat(batch.nextSplit()).isEqualTo(splitId); + assertThat(batch.records()).hasSize(2); + assertThat(batch.numberOfRecords()).isEqualTo(1); RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); /////////////////////////////// // assert first record - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - Assert.assertNull(batch.nextRecordFromSplit()); - Assert.assertNull(batch.nextSplit()); + assertThat(batch.nextRecordFromSplit()).isNull(); + assertThat(batch.nextSplit()).isNull(); batch.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } /** @@ -107,7 +106,11 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { public void testSingleFileWithMultipleBatches() throws Exception { List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); FileScanTask fileTask = - ReaderUtil.createFileTask(records, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); String splitId = "someSplitId"; @@ -118,90 +121,86 @@ public void testSingleFileWithMultipleBatches() throws Exception { // assert first batch with full batch of 2 records ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch0.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch0.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch0.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch0.numberOfRecords()); + assertThat(batch0.finishedSplits()).isEmpty(); + assertThat(batch0.nextSplit()).isEqualTo(splitId); + assertThat(batch0.records()).hasSize(2); + assertThat(batch0.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; // assert first record recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); // assert second record recordAndPosition = batch0.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); - Assert.assertNull(batch0.nextRecordFromSplit()); - Assert.assertNull(batch0.nextSplit()); + assertThat(batch0.nextRecordFromSplit()).isNull(); + assertThat(batch0.nextSplit()).isNull(); batch0.recycle(); /////////////////////////////// // assert second batch with full batch of 2 records ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch1.records()); - Assert.assertTrue(batch1.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch1.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch1.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch1.numberOfRecords()); + assertThat(batch1.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch1.finishedSplits()).isEmpty(); + assertThat(batch1.nextSplit()).isEqualTo(splitId); + assertThat(batch1.records()).hasSize(2); + assertThat(batch1.numberOfRecords()).isEqualTo(2); // assert third record recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); // assert fourth record recordAndPosition = batch1.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); - Assert.assertNull(batch1.nextRecordFromSplit()); - Assert.assertNull(batch1.nextSplit()); + assertThat(batch1.nextRecordFromSplit()).isNull(); + assertThat(batch1.nextSplit()).isNull(); batch1.recycle(); /////////////////////////////// // assert third batch with partial batch of 1 record ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch0.records(), batch2.records()); - Assert.assertTrue(batch2.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch2.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch2.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch2.numberOfRecords()); + assertThat(batch2.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch2.finishedSplits()).isEmpty(); + assertThat(batch2.nextSplit()).isEqualTo(splitId); + assertThat(batch2.records()).hasSize(2); + assertThat(batch2.numberOfRecords()).isEqualTo(1); // assert fifth record recordAndPosition = batch2.nextRecordFromSplit(); - Assert.assertEquals(0, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(5, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(5); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); - Assert.assertNull(batch2.nextRecordFromSplit()); - Assert.assertNull(batch2.nextSplit()); + assertThat(batch2.nextRecordFromSplit()).isNull(); + assertThat(batch2.nextSplit()).isNull(); batch2.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } /** @@ -214,20 +213,28 @@ public void testMultipleFilesWithSeekPosition() throws Exception { List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); FileScanTask fileTask0 = ReaderUtil.createFileTask( - records0, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records0, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); FileScanTask fileTask1 = ReaderUtil.createFileTask( - records1, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records1, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); FileScanTask fileTask2 = ReaderUtil.createFileTask( - records2, TEMPORARY_FOLDER.newFile(), fileFormat, appenderFactory); + records2, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); CombinedScanTask combinedTask = new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - // seek to file1 and after record 1 dataIterator.seek(1, 1); String splitId = "someSplitId"; @@ -246,52 +253,50 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); - Assert.assertTrue(batch10.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch10.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch10.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch10.numberOfRecords()); + assertThat(batch10.finishedSplits()).isEmpty(); + assertThat(batch10.nextSplit()).isEqualTo(splitId); + assertThat(batch10.records()).hasSize(2); + assertThat(batch10.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // seek should skip the first record in file1. starting from the second record - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("seek should skip the first record in file1. starting from the second record") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); recordAndPosition = batch10.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); - Assert.assertNull(batch10.nextRecordFromSplit()); - Assert.assertNull(batch10.nextSplit()); + assertThat(batch10.nextRecordFromSplit()).isNull(); + assertThat(batch10.nextSplit()).isNull(); batch10.recycle(); // assert second batch from file1 with partial batch of 1 record // variable naming convention: batch__ ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch11.records()); - Assert.assertTrue(batch11.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch11.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch11.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch11.numberOfRecords()); + assertThat(batch11.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch11.finishedSplits()).isEmpty(); + assertThat(batch11.nextSplit()).isEqualTo(splitId); + assertThat(batch11.records()).hasSize(2); + assertThat(batch11.numberOfRecords()).isEqualTo(1); recordAndPosition = batch11.nextRecordFromSplit(); - Assert.assertEquals(1, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(4, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); - Assert.assertNull(batch11.nextRecordFromSplit()); - Assert.assertNull(batch11.nextSplit()); + assertThat(batch11.nextRecordFromSplit()).isNull(); + assertThat(batch11.nextSplit()).isNull(); batch11.recycle(); /////////////////////////////// @@ -303,29 +308,28 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch__ ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch20.records()); - Assert.assertTrue(batch20.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch20.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch20.records().length); - // assert actual number of records in the array - Assert.assertEquals(2, batch20.numberOfRecords()); + assertThat(batch20.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch20.finishedSplits()).isEmpty(); + assertThat(batch20.nextSplit()).isEqualTo(splitId); + assertThat(batch20.records()).hasSize(2); + assertThat(batch20.numberOfRecords()).isEqualTo(2); recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(1, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); recordAndPosition = batch20.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(2, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); - Assert.assertNull(batch20.nextRecordFromSplit()); - Assert.assertNull(batch20.nextSplit()); + assertThat(batch20.nextRecordFromSplit()).isNull(); + assertThat(batch20.nextSplit()).isNull(); batch20.recycle(); /////////////////////////////// @@ -333,26 +337,24 @@ public void testMultipleFilesWithSeekPosition() throws Exception { // variable naming convention: batch__ ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused - Assert.assertSame(batch10.records(), batch21.records()); - Assert.assertTrue(batch21.finishedSplits().isEmpty()); - Assert.assertEquals(splitId, batch21.nextSplit()); - // reusable array size should be the configured value of 2 - Assert.assertEquals(2, batch21.records().length); - // assert actual number of records in the array - Assert.assertEquals(1, batch21.numberOfRecords()); + assertThat(batch21.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch21.finishedSplits()).isEmpty(); + assertThat(batch21.nextSplit()).isEqualTo(splitId); + assertThat(batch21.records()).hasSize(2); + assertThat(batch21.numberOfRecords()).isEqualTo(1); recordAndPosition = batch21.nextRecordFromSplit(); - Assert.assertEquals(2, recordAndPosition.fileOffset()); - // The position points to where the reader should resume after this record is processed. - Assert.assertEquals(3, recordAndPosition.recordOffset()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); - Assert.assertNull(batch21.nextRecordFromSplit()); - Assert.assertNull(batch21.nextSplit()); + assertThat(batch21.nextRecordFromSplit()).isNull(); + assertThat(batch21.nextSplit()).isNull(); batch21.recycle(); - // assert end of input - Assert.assertFalse(recordBatchIterator.hasNext()); + assertThat(recordBatchIterator).isExhausted(); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java index b9af6aca2895..8d6782586676 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -18,6 +18,9 @@ */ package org.apache.iceberg.flink.source.reader; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; import java.util.Arrays; import java.util.Collections; import java.util.List; @@ -37,19 +40,14 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.hadoop.HadoopFileIO; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceReader { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - private final GenericAppenderFactory appenderFactory; + @TempDir protected Path temporaryFolder; - public TestIcebergSourceReader() { - this.appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - } + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); @Test public void testReaderMetrics() throws Exception { @@ -70,13 +68,13 @@ public void testReaderOrder() throws Exception { ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task1 = ReaderUtil.createCombinedScanTask( - recordBatchList1, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList1, temporaryFolder, FileFormat.PARQUET, appenderFactory); List> recordBatchList2 = ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task2 = ReaderUtil.createCombinedScanTask( - recordBatchList2, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList2, temporaryFolder, FileFormat.PARQUET, appenderFactory); // Sort the splits in one way List rowDataList1 = @@ -95,8 +93,7 @@ public void testReaderOrder() throws Exception { 2); // Check that the order of the elements is not changed - Assert.assertEquals(rowDataList1.get(0), rowDataList2.get(0)); - Assert.assertEquals(rowDataList1.get(1), rowDataList2.get(1)); + assertThat(rowDataList1).containsExactlyElementsOf(rowDataList2); } private List read(List splits, long expected) throws Exception { @@ -114,7 +111,7 @@ private List read(List splits, long expected) throw reader.pollNext(readerOutput); - Assert.assertEquals(expected, readerOutput.getEmittedRecords().size()); + assertThat(readerOutput.getEmittedRecords()).hasSize((int) expected); return readerOutput.getEmittedRecords(); } @@ -130,7 +127,7 @@ private void testOneSplitFetcher( ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); CombinedScanTask task = ReaderUtil.createCombinedScanTask( - recordBatchList, TEMPORARY_FOLDER, FileFormat.PARQUET, appenderFactory); + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); reader.addSplits(Collections.singletonList(split)); @@ -138,12 +135,12 @@ private void testOneSplitFetcher( reader.pollNext(readerOutput); } - Assert.assertEquals(expectedCount, readerOutput.getEmittedRecords().size()); + assertThat(readerOutput.getEmittedRecords()).hasSize(expectedCount); TestHelpers.assertRowData( TestFixtures.SCHEMA, recordBatchList.get(0).get(0), readerOutput.getEmittedRecords().get(expectedCount - 1)); - Assert.assertEquals(expectedCount, metricGroup.counters().get("assignedSplits").getCount()); + assertThat(metricGroup.counters().get("assignedSplits").getCount()).isEqualTo(expectedCount); // One more poll will get null record batch. // That will finish the split and cause split fetcher to be closed due to idleness. diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java index 764215e904d3..55f9c0af3a29 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java @@ -28,7 +28,6 @@ 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.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.data.Record; import org.apache.iceberg.encryption.PlaintextEncryptionManager; @@ -39,13 +38,9 @@ public class TestRowDataReaderFunction extends ReaderFunctionTestBase { - protected static final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final DataStructureConverter rowDataConverter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - - public TestRowDataReaderFunction(FileFormat fileFormat) { - super(fileFormat); - } + protected static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final DataStructureConverter ROW_DATA_CONVERTER = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(ROW_TYPE)); @Override protected ReaderFunction readerFunction() { @@ -68,7 +63,7 @@ protected void assertRecords(List expected, List actual, Schema private List toRows(List actual) { return actual.stream() - .map(rowData -> (Row) rowDataConverter.toExternal(rowData)) + .map(rowData -> (Row) ROW_DATA_CONVERTER.toExternal(rowData)) .collect(Collectors.toList()); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java index c72d622f86ba..12bacdcd074d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -18,20 +18,21 @@ */ package org.apache.iceberg.flink.source.split; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; import java.util.List; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.FileScanTask; import org.apache.iceberg.flink.source.SplitHelpers; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceSplitSerializer { - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); @@ -44,14 +45,14 @@ public void testLatestVersion() throws Exception { private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = serializer.serialize(split); IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); assertSplitEquals(split, deserialized); byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); + assertThat(cachedResult).isSameAs(result); IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult); assertSplitEquals(split, deserialized2); @@ -59,7 +60,7 @@ private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws E split.updatePosition(0, 100); byte[] resultAfterUpdatePosition = serializer.serialize(split); // after position change, serialized bytes should have changed - Assert.assertNotSame(cachedResult, resultAfterUpdatePosition); + assertThat(resultAfterUpdatePosition).isNotSameAs(cachedResult); IcebergSourceSplit deserialized3 = serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); assertSplitEquals(split, deserialized3); @@ -75,7 +76,7 @@ public void testV1() throws Exception { private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV1(); IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); @@ -92,7 +93,7 @@ public void testV2() throws Exception { private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV2(); IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); @@ -109,9 +110,9 @@ private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mo throws Exception { final List splits = SplitHelpers.createSplitsFromTransientHadoopTable( - TEMPORARY_FOLDER, splitCount, filesPerSplit); + temporaryFolder, splitCount, filesPerSplit); final List splitsWithMockDeleteFiles = - SplitHelpers.equipSplitsWithMockDeleteFiles(splits, TEMPORARY_FOLDER, mockDeletesPerSplit); + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, temporaryFolder, mockDeletesPerSplit); for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { byte[] result = split.serializeV3(); @@ -123,7 +124,7 @@ private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mo @Test public void testDeserializeV1() throws Exception { final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 1, 1); + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); for (IcebergSourceSplit split : splits) { byte[] result = split.serializeV1(); IcebergSourceSplit deserialized = serializer.deserialize(1, result); @@ -135,7 +136,7 @@ public void testDeserializeV1() throws Exception { public void testCheckpointedPosition() throws Exception { final AtomicInteger index = new AtomicInteger(); final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(TEMPORARY_FOLDER, 10, 2).stream() + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 2).stream() .map( split -> { IcebergSourceSplit result; @@ -155,7 +156,7 @@ public void testCheckpointedPosition() throws Exception { assertSplitEquals(split, deserialized); byte[] cachedResult = serializer.serialize(split); - Assert.assertSame(result, cachedResult); + assertThat(cachedResult).isSameAs(result); IcebergSourceSplit deserialized2 = serializer.deserialize(serializer.getVersion(), cachedResult); assertSplitEquals(split, deserialized2); @@ -165,18 +166,18 @@ public void testCheckpointedPosition() throws Exception { private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); - Assert.assertEquals(expectedTasks.size(), actualTasks.size()); + assertThat(actualTasks).hasSameSizeAs(expectedTasks); for (int i = 0; i < expectedTasks.size(); ++i) { FileScanTask expectedTask = expectedTasks.get(i); FileScanTask actualTask = actualTasks.get(i); - Assert.assertEquals(expectedTask.file().path(), actualTask.file().path()); - Assert.assertEquals(expectedTask.sizeBytes(), actualTask.sizeBytes()); - Assert.assertEquals(expectedTask.filesCount(), actualTask.filesCount()); - Assert.assertEquals(expectedTask.start(), actualTask.start()); - Assert.assertEquals(expectedTask.length(), actualTask.length()); + assertThat(actualTask.file().path()).isEqualTo(expectedTask.file().path()); + assertThat(actualTask.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); + assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); + assertThat(actualTask.start()).isEqualTo(expectedTask.start()); + assertThat(actualTask.length()).isEqualTo(expectedTask.length()); } - Assert.assertEquals(expected.fileOffset(), actual.fileOffset()); - Assert.assertEquals(expected.recordOffset(), actual.recordOffset()); + assertThat(actual.fileOffset()).isEqualTo(expected.fileOffset()); + assertThat(actual.recordOffset()).isEqualTo(expected.recordOffset()); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java index 272734185c66..41a787762fda 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -136,8 +136,7 @@ public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exceptio .map(IcebergSourceSplitState::split) .map(IcebergSourceSplit::splitId) .collect(Collectors.toList()); - assertThat(pendingSplitIds).hasSameSizeAs(splits); - assertThat(pendingSplitIds).first().isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplitIds).hasSameSizeAs(splits).first().isEqualTo(splits.get(0).splitId()); // register the reader again, and let it request a split enumeratorContext.registerReader(2, "localhost"); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java index 1cc2e175e698..b2185675340f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -79,7 +79,6 @@ public void testTableScanThenIncrementalStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - // empty table assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) .isNotPresent(); @@ -97,7 +96,6 @@ public void testForLatestSnapshotStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) .build(); - // empty table assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) .isNotPresent(); @@ -115,7 +113,6 @@ public void testForEarliestSnapshotStrategy() throws IOException { .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) .build(); - // empty table assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) .isNotPresent(); @@ -134,7 +131,6 @@ public void testForSpecificSnapshotIdStrategy() throws IOException { .startSnapshotId(1L) .build(); - // empty table assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( @@ -165,7 +161,6 @@ public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOExcepti .startSnapshotTimestamp(1L) .build(); - // empty table assertThatThrownBy( () -> ContinuousSplitPlannerImpl.startSnapshot( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java index e6dde153ea6f..0d1d0ce3217c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java @@ -61,7 +61,6 @@ public static Object[][] parameters() { @Parameter(index = 0) private FileFormat fileFormat; - @Parameter(index = 1) private final GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java index bcdc827bc695..1a78bb1b0010 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.source.DataIterator; import org.apache.iceberg.io.CloseableIterator; -import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -48,21 +47,16 @@ public class TestArrayPoolDataIteratorBatcherRowData { @TempDir protected Path temporaryFolder; private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - private static final Configuration config = new Configuration(); + private final Configuration config = + new Configuration() + .set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1) + .set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); private final GenericAppenderFactory appenderFactory = - new GenericAppenderFactory(TestFixtures.SCHEMA);; + new GenericAppenderFactory(TestFixtures.SCHEMA); private final DataIteratorBatcher batcher = new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - @BeforeAll - public static void setConfig() { - // set array pool size to 1 - config.set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1); - // set batch array size to 2 - config.set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - } - /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ @Test public void testSingleFileLessThanOneFullBatch() throws Exception { @@ -82,9 +76,7 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); assertThat(batch.finishedSplits()).isEmpty(); assertThat(batch.nextSplit()).isEqualTo(splitId); - // reusable array size should be the configured value of 2 assertThat(batch.records()).hasSize(2); - // assert actual number of records in the array assertThat(batch.numberOfRecords()).isEqualTo(1); RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); @@ -93,15 +85,15 @@ public void testSingleFileLessThanOneFullBatch() throws Exception { // assert first record assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); assertThat(batch.nextRecordFromSplit()).isNull(); assertThat(batch.nextSplit()).isNull(); batch.recycle(); - // assert end of input assertThat(recordBatchIterator).isExhausted(); } @@ -131,9 +123,7 @@ public void testSingleFileWithMultipleBatches() throws Exception { ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); assertThat(batch0.finishedSplits()).isEmpty(); assertThat(batch0.nextSplit()).isEqualTo(splitId); - // reusable array size should be the configured value of 2 assertThat(batch0.records()).hasSize(2); - // assert actual number of records in the array assertThat(batch0.numberOfRecords()).isEqualTo(2); RecordAndPosition recordAndPosition; @@ -141,15 +131,17 @@ public void testSingleFileWithMultipleBatches() throws Exception { // assert first record recordAndPosition = batch0.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); // assert second record recordAndPosition = batch0.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); assertThat(batch0.nextRecordFromSplit()).isNull(); @@ -160,27 +152,26 @@ public void testSingleFileWithMultipleBatches() throws Exception { // assert second batch with full batch of 2 records ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused assertThat(batch1.records()).containsExactlyInAnyOrder(batch0.records()); assertThat(batch1.finishedSplits()).isEmpty(); assertThat(batch1.nextSplit()).isEqualTo(splitId); - // reusable array size should be the configured value of 2 assertThat(batch1.records()).hasSize(2); - // assert actual number of records in the array assertThat(batch1.numberOfRecords()).isEqualTo(2); // assert third record recordAndPosition = batch1.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(3); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); // assert fourth record recordAndPosition = batch1.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(4); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); assertThat(batch1.nextRecordFromSplit()).isNull(); @@ -191,27 +182,24 @@ public void testSingleFileWithMultipleBatches() throws Exception { // assert third batch with partial batch of 1 record ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); - // assert array is reused assertThat(batch2.records()).containsExactlyInAnyOrder(batch0.records()); assertThat(batch2.finishedSplits()).isEmpty(); assertThat(batch2.nextSplit()).isEqualTo(splitId); - // reusable array size should be the configured value of 2 assertThat(batch2.records()).hasSize(2); - // assert actual number of records in the array assertThat(batch2.numberOfRecords()).isEqualTo(1); // assert fifth record recordAndPosition = batch2.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(5); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(5); TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); assertThat(batch2.nextRecordFromSplit()).isNull(); assertThat(batch2.nextSplit()).isNull(); batch2.recycle(); - // assert end of input assertThat(recordBatchIterator).isExhausted(); } @@ -274,8 +262,9 @@ public void testMultipleFilesWithSeekPosition() throws Exception { recordAndPosition = batch10.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(1); - // seek should skip the first record in file1. starting from the second record - assertThat(recordAndPosition.recordOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("seek should skip the first record in file1. starting from the second record") + .isEqualTo(2); TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); recordAndPosition = batch10.nextRecordFromSplit(); @@ -327,8 +316,9 @@ public void testMultipleFilesWithSeekPosition() throws Exception { recordAndPosition = batch20.nextRecordFromSplit(); assertThat(recordAndPosition.fileOffset()).isEqualTo(2); - // The position points to where the reader should resume after this record is processed. - assertThat(recordAndPosition.recordOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); recordAndPosition = batch20.nextRecordFromSplit(); From c03d47be1e10163769086797298b419ccbf11396 Mon Sep 17 00:00:00 2001 From: fengjiajie Date: Wed, 10 Jul 2024 14:38:23 +0800 Subject: [PATCH 0451/1019] Flink: Pre-create fieldGetters to avoid constructing them for each row (#10565) * Flink: Pre-create fieldGetters to avoid constructing them for each row * modified as suggested * modified as suggested * modified as suggested --- .../iceberg/flink/data/RowDataUtil.java | 29 +++++++++++++++++-- .../source/reader/RowDataRecordFactory.java | 14 ++++++++- .../org/apache/iceberg/flink/TestHelpers.java | 7 ++++- 3 files changed, 45 insertions(+), 5 deletions(-) diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index c5cb51b7eae4..3a8f5ccc6c03 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -79,22 +79,45 @@ public static Object convertConstant(Type type, Object value) { * the arity check. */ public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData from, + RowData reuse, + RowType rowType, + TypeSerializer[] fieldSerializers, + RowData.FieldGetter[] fieldGetters) { GenericRowData ret; if (reuse instanceof GenericRowData) { ret = (GenericRowData) reuse; } else { ret = new GenericRowData(from.getArity()); } + ret.setRowKind(from.getRowKind()); for (int i = 0; i < rowType.getFieldCount(); i++) { if (!from.isNullAt(i)) { - RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); - ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); + ret.setField(i, fieldSerializers[i].copy(fieldGetters[i].getFieldOrNull(from))); } else { ret.setField(i, null); } } + return ret; } + + /** + * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to + * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], + * RowData.FieldGetter[])} instead. + */ + @Deprecated + public static RowData clone( + RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + if (!from.isNullAt(i)) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + } + + return clone(from, reuse, rowType, fieldSerializers, fieldGetters); + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java index 1e265b2663ce..40d5c28d7bc7 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java @@ -28,10 +28,12 @@ class RowDataRecordFactory implements RecordFactory { private final RowType rowType; private final TypeSerializer[] fieldSerializers; + private final RowData.FieldGetter[] fieldGetters; RowDataRecordFactory(RowType rowType) { this.rowType = rowType; this.fieldSerializers = createFieldSerializers(rowType); + this.fieldGetters = createFieldGetters(rowType); } static TypeSerializer[] createFieldSerializers(RowType rowType) { @@ -40,6 +42,15 @@ static TypeSerializer[] createFieldSerializers(RowType rowType) { .toArray(TypeSerializer[]::new); } + static RowData.FieldGetter[] createFieldGetters(RowType rowType) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return fieldGetters; + } + @Override public RowData[] createBatch(int batchSize) { RowData[] arr = new RowData[batchSize]; @@ -56,6 +67,7 @@ public void clone(RowData from, RowData[] batch, int position) { // Clone method will allocate a new GenericRowData object // if the target object is NOT a GenericRowData. // So we should always set the clone return value back to the array. - batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); + batch[position] = + RowDataUtil.clone(from, batch[position], rowType, fieldSerializers, fieldGetters); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index a243be5834a8..8cebf950c5f0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -87,7 +87,12 @@ public static RowData copyRowData(RowData from, RowType rowType) { rowType.getChildren().stream() .map((LogicalType type) -> InternalSerializers.create(type)) .toArray(TypeSerializer[]::new); - return RowDataUtil.clone(from, null, rowType, fieldSerializers); + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return RowDataUtil.clone(from, null, rowType, fieldSerializers, fieldGetters); } public static void readRowData(FlinkInputFormat input, Consumer visitor) From 9b152332f8f64975dc4edd39fdcb47c96f452cd0 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 10 Jul 2024 09:25:12 +0200 Subject: [PATCH 0452/1019] Dell, Hive3: Convert remaining tests to JUnit5 (#10670) --- .../iceberg/dell/ecs/TestEcsCatalog.java | 43 +++++------------ .../TestIcebergDateObjectInspectorHive3.java | 34 ++++++------- ...tIcebergTimestampObjectInspectorHive3.java | 40 ++++++++-------- ...TimestampWithZoneObjectInspectorHive3.java | 48 +++++++++---------- 4 files changed, 72 insertions(+), 93 deletions(-) diff --git a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java index b0e2cfa51f90..4714d37d72b9 100644 --- a/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java +++ b/dell/src/test/java/org/apache/iceberg/dell/ecs/TestEcsCatalog.java @@ -36,12 +36,10 @@ import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; 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.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Types; -import org.junit.Assert; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -80,46 +78,31 @@ public void testListTablesAndNamespaces() { ecsCatalog.createTable(TableIdentifier.of("a", "t4"), SCHEMA); ecsCatalog.createTable(TableIdentifier.of("a", "b1", "t5"), SCHEMA); - Assert.assertEquals( - "List namespaces with empty namespace", - ImmutableList.of(Namespace.of("a")), - ecsCatalog.listNamespaces()); - Assert.assertEquals( - "List tables with empty namespace", - ImmutableList.of(TableIdentifier.of("t1"), TableIdentifier.of("t2")), - ecsCatalog.listTables(Namespace.empty())); - Assert.assertEquals( - "List namespaces in [a]", - ImmutableList.of(Namespace.of("a", "b1"), Namespace.of("a", "b2")), - ecsCatalog.listNamespaces(Namespace.of("a"))); - Assert.assertEquals( - "List tables in [a]", - ImmutableList.of(TableIdentifier.of("a", "t3"), TableIdentifier.of("a", "t4")), - ecsCatalog.listTables(Namespace.of("a"))); + assertThat(ecsCatalog.listNamespaces()).containsExactly(Namespace.of("a")); + assertThat(ecsCatalog.listTables(Namespace.empty())) + .containsExactly(TableIdentifier.of("t1"), TableIdentifier.of("t2")); + assertThat(ecsCatalog.listNamespaces(Namespace.of("a"))) + .containsExactly(Namespace.of("a", "b1"), Namespace.of("a", "b2")); + assertThat(ecsCatalog.listTables(Namespace.of("a"))) + .containsExactly(TableIdentifier.of("a", "t3"), TableIdentifier.of("a", "t4")); } @Test public void testNamespaceProperties() { ecsCatalog.createNamespace(Namespace.of("a"), ImmutableMap.of("a", "a")); - Assert.assertEquals( - "The initial properties", - ImmutableMap.of("a", "a"), - ecsCatalog.loadNamespaceMetadata(Namespace.of("a"))); + assertThat(ecsCatalog.loadNamespaceMetadata(Namespace.of("a"))) + .isEqualTo(ImmutableMap.of("a", "a")); ecsCatalog.setProperties(Namespace.of("a"), ImmutableMap.of("b", "b")); - Assert.assertEquals( - "Update properties", - ImmutableMap.of("a", "a", "b", "b"), - ecsCatalog.loadNamespaceMetadata(Namespace.of("a"))); + assertThat(ecsCatalog.loadNamespaceMetadata(Namespace.of("a"))) + .isEqualTo(ImmutableMap.of("a", "a", "b", "b")); ecsCatalog.removeProperties(Namespace.of("a"), ImmutableSet.of("a")); - Assert.assertEquals( - "Remove properties", - ImmutableMap.of("b", "b"), - ecsCatalog.loadNamespaceMetadata(Namespace.of("a"))); + assertThat(ecsCatalog.loadNamespaceMetadata(Namespace.of("a"))) + .isEqualTo(ImmutableMap.of("b", "b")); } @Test diff --git a/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspectorHive3.java b/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspectorHive3.java index 102fbd0e1e5c..e3e06c8560e4 100644 --- a/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspectorHive3.java +++ b/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergDateObjectInspectorHive3.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.LocalDate; import org.apache.hadoop.hive.common.type.Date; import org.apache.hadoop.hive.serde2.io.DateWritableV2; @@ -25,8 +27,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.primitive.DateObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergDateObjectInspectorHive3 { @@ -34,31 +35,30 @@ public class TestIcebergDateObjectInspectorHive3 { public void testIcebergDateObjectInspector() { DateObjectInspector oi = IcebergDateObjectInspectorHive3.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals(PrimitiveObjectInspector.PrimitiveCategory.DATE, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.DATE); - Assert.assertEquals(TypeInfoFactory.dateTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.dateTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.dateTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.dateTypeInfo.getTypeName()); - Assert.assertEquals(Date.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(DateWritableV2.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Date.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(DateWritableV2.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); int epochDays = 5005; LocalDate local = LocalDate.ofEpochDay(epochDays); Date date = Date.ofEpochDay(epochDays); - Assert.assertEquals(date, oi.getPrimitiveJavaObject(local)); - Assert.assertEquals(new DateWritableV2(date), oi.getPrimitiveWritableObject(local)); + assertThat(oi.getPrimitiveJavaObject(local)).isEqualTo(date); + assertThat(oi.getPrimitiveWritableObject(local)).isEqualTo(new DateWritableV2(date)); Date copy = (Date) oi.copyObject(date); - Assert.assertEquals(date, copy); - Assert.assertNotSame(date, copy); - - Assert.assertFalse(oi.preferWritable()); + assertThat(copy).isEqualTo(date).isNotSameAs(date); + assertThat(oi.preferWritable()).isFalse(); } } diff --git a/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspectorHive3.java b/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspectorHive3.java index 661e1b148ee4..523d530bfd2a 100644 --- a/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspectorHive3.java +++ b/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampObjectInspectorHive3.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.Instant; import java.time.LocalDateTime; import java.time.ZoneId; @@ -26,8 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimestampObjectInspectorHive3 { @@ -35,20 +36,20 @@ public class TestIcebergTimestampObjectInspectorHive3 { public void testIcebergTimestampObjectInspector() { IcebergTimestampObjectInspectorHive3 oi = IcebergTimestampObjectInspectorHive3.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMP); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.timestampTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.timestampTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.timestampTypeInfo.getTypeName()); - Assert.assertEquals(Timestamp.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(TimestampWritableV2.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(Timestamp.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(TimestampWritableV2.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); long epochMilli = 1601471970000L; LocalDateTime local = @@ -57,16 +58,13 @@ public void testIcebergTimestampObjectInspector() { Timestamp ts = Timestamp.ofEpochMilli(epochMilli); ts.setNanos(34000); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(local)); - Assert.assertEquals(new TimestampWritableV2(ts), oi.getPrimitiveWritableObject(local)); + assertThat(oi.getPrimitiveJavaObject(local)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(local)).isEqualTo(new TimestampWritableV2(ts)); Timestamp copy = (Timestamp) oi.copyObject(ts); - Assert.assertEquals(ts, copy); - Assert.assertNotSame(ts, copy); - - Assert.assertFalse(oi.preferWritable()); - - Assert.assertEquals(local, oi.convert(ts)); + assertThat(copy).isEqualTo(ts).isNotSameAs(ts); + assertThat(oi.preferWritable()).isFalse(); + assertThat(oi.convert(ts)).isEqualTo(local); } } diff --git a/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspectorHive3.java b/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspectorHive3.java index 17e6047cfaae..d14cb893d82e 100644 --- a/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspectorHive3.java +++ b/hive3/src/test/java/org/apache/iceberg/mr/hive/serde/objectinspector/TestIcebergTimestampWithZoneObjectInspectorHive3.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.mr.hive.serde.objectinspector; +import static org.assertj.core.api.Assertions.assertThat; + import java.time.LocalDateTime; import java.time.OffsetDateTime; import java.time.ZoneId; @@ -27,8 +29,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; import org.apache.hadoop.hive.serde2.objectinspector.PrimitiveObjectInspector; import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestIcebergTimestampWithZoneObjectInspectorHive3 { @@ -37,43 +38,40 @@ public void testIcebergTimestampLocalTZObjectInspector() { IcebergTimestampWithZoneObjectInspectorHive3 oi = IcebergTimestampWithZoneObjectInspectorHive3.get(); - Assert.assertEquals(ObjectInspector.Category.PRIMITIVE, oi.getCategory()); - Assert.assertEquals( - PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMPLOCALTZ, oi.getPrimitiveCategory()); + assertThat(oi.getCategory()).isEqualTo(ObjectInspector.Category.PRIMITIVE); + assertThat(oi.getPrimitiveCategory()) + .isEqualTo(PrimitiveObjectInspector.PrimitiveCategory.TIMESTAMPLOCALTZ); - Assert.assertEquals(TypeInfoFactory.timestampLocalTZTypeInfo, oi.getTypeInfo()); - Assert.assertEquals(TypeInfoFactory.timestampLocalTZTypeInfo.getTypeName(), oi.getTypeName()); + assertThat(oi.getTypeInfo()).isEqualTo(TypeInfoFactory.timestampLocalTZTypeInfo); + assertThat(oi.getTypeName()).isEqualTo(TypeInfoFactory.timestampLocalTZTypeInfo.getTypeName()); - Assert.assertEquals(TimestampTZ.class, oi.getJavaPrimitiveClass()); - Assert.assertEquals(TimestampLocalTZWritable.class, oi.getPrimitiveWritableClass()); + assertThat(oi.getJavaPrimitiveClass()).isEqualTo(TimestampTZ.class); + assertThat(oi.getPrimitiveWritableClass()).isEqualTo(TimestampLocalTZWritable.class); - Assert.assertNull(oi.copyObject(null)); - Assert.assertNull(oi.getPrimitiveJavaObject(null)); - Assert.assertNull(oi.getPrimitiveWritableObject(null)); - Assert.assertNull(oi.convert(null)); + assertThat(oi.copyObject(null)).isNull(); + assertThat(oi.getPrimitiveJavaObject(null)).isNull(); + assertThat(oi.getPrimitiveWritableObject(null)).isNull(); + assertThat(oi.convert(null)).isNull(); LocalDateTime dateTimeAtUTC = LocalDateTime.of(2020, 12, 10, 15, 55, 20, 30000); OffsetDateTime offsetDateTime = OffsetDateTime.of(dateTimeAtUTC.plusHours(4), ZoneOffset.ofHours(4)); TimestampTZ ts = new TimestampTZ(dateTimeAtUTC.atZone(ZoneId.of("UTC"))); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(offsetDateTime)); - Assert.assertEquals( - new TimestampLocalTZWritable(ts), oi.getPrimitiveWritableObject(offsetDateTime)); + assertThat(oi.getPrimitiveJavaObject(offsetDateTime)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(offsetDateTime)) + .isEqualTo(new TimestampLocalTZWritable(ts)); // try with another offset as well offsetDateTime = OffsetDateTime.of(dateTimeAtUTC.plusHours(11), ZoneOffset.ofHours(11)); - Assert.assertEquals(ts, oi.getPrimitiveJavaObject(offsetDateTime)); - Assert.assertEquals( - new TimestampLocalTZWritable(ts), oi.getPrimitiveWritableObject(offsetDateTime)); + assertThat(oi.getPrimitiveJavaObject(offsetDateTime)).isEqualTo(ts); + assertThat(oi.getPrimitiveWritableObject(offsetDateTime)) + .isEqualTo(new TimestampLocalTZWritable(ts)); TimestampTZ copy = (TimestampTZ) oi.copyObject(ts); - Assert.assertEquals(ts, copy); - Assert.assertNotSame(ts, copy); - - Assert.assertFalse(oi.preferWritable()); - - Assert.assertEquals(OffsetDateTime.of(dateTimeAtUTC, ZoneOffset.UTC), oi.convert(ts)); + assertThat(copy).isEqualTo(ts).isNotSameAs(ts); + assertThat(oi.preferWritable()).isFalse(); + assertThat(oi.convert(ts)).isEqualTo(OffsetDateTime.of(dateTimeAtUTC, ZoneOffset.UTC)); } } From 7337aabbe49f00c5433b9d2abf4b04e7e4139a25 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 10 Jul 2024 14:40:22 +0200 Subject: [PATCH 0453/1019] Build: Define JUnit4 dependency only where necessary (#10672) Most modules have already been converted to JUnit5, except for Flink and Spark. Defining the JUnit4 dependency only where necessary reduces the risk that new JUnit4-style tests are being added in the already converted modules. --- api/src/test/java/org/apache/iceberg/Parameter.java | 5 ++--- build.gradle | 2 +- flink/v1.17/build.gradle | 1 + flink/v1.18/build.gradle | 1 + flink/v1.19/build.gradle | 1 + spark/v3.3/build.gradle | 2 ++ spark/v3.4/build.gradle | 2 ++ 7 files changed, 10 insertions(+), 4 deletions(-) diff --git a/api/src/test/java/org/apache/iceberg/Parameter.java b/api/src/test/java/org/apache/iceberg/Parameter.java index 6a47cc9a377f..a3c2c0a0e537 100644 --- a/api/src/test/java/org/apache/iceberg/Parameter.java +++ b/api/src/test/java/org/apache/iceberg/Parameter.java @@ -22,11 +22,10 @@ import java.lang.annotation.Retention; import java.lang.annotation.RetentionPolicy; import java.lang.annotation.Target; -import org.junit.runners.Parameterized; /** - * The annotation is used to replace {@link Parameterized.Parameter} for Junit 5 parameterized - * tests. + * The annotation is used to replace {@link org.junit.runners.Parameterized.Parameter} for Junit 5 + * parameterized tests. * *

    This implementation has been taken from Flink repository. The only difference is the "index" * field, renamed to be more intuitive diff --git a/build.gradle b/build.gradle index 6df806db59e4..1cb28078d233 100644 --- a/build.gradle +++ b/build.gradle @@ -212,7 +212,6 @@ subprojects { dependencies { implementation libs.slf4j.api - testImplementation libs.junit.vintage.engine testImplementation libs.junit.jupiter testImplementation libs.junit.jupiter.engine testImplementation libs.slf4j.simple @@ -413,6 +412,7 @@ project(':iceberg-data') { testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation libs.junit.vintage.engine } test { diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle index 2ced7a5a5cb7..2509371bdb8b 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.17/build.gradle @@ -117,6 +117,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.junit.vintage.engine } test { diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index f06318af83a3..afd4c3c6e114 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -117,6 +117,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.junit.vintage.engine } test { diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index 392a1cb124f0..a0954942f0e9 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -119,6 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.junit.vintage.engine } test { diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 2f1ddef0a325..63c1a8e90a2e 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -103,6 +103,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') testImplementation libs.sqlite.jdbc testImplementation libs.awaitility + testImplementation libs.junit.vintage.engine } test { @@ -164,6 +165,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop + testImplementation libs.junit.vintage.engine // Required because we remove antlr plugin dependencies from the compile configuration, see note above runtimeOnly libs.antlr.runtime diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 1da53ecfc61d..9c584ad3a21b 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -104,6 +104,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') testImplementation libs.sqlite.jdbc testImplementation libs.awaitility + testImplementation libs.junit.vintage.engine } test { @@ -167,6 +168,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer testImplementation libs.avro.avro testImplementation libs.parquet.hadoop + testImplementation libs.junit.vintage.engine // Required because we remove antlr plugin dependencies from the compile configuration, see note above runtimeOnly libs.antlr.runtime From f14c6d74e1fc6ad5d5a6d831839a0828debcdcd2 Mon Sep 17 00:00:00 2001 From: Attila Kreiner Date: Wed, 10 Jul 2024 16:44:14 +0200 Subject: [PATCH 0454/1019] Flink, Spark: Rename constants to be all uppercase (#10675) --- ...TestBucketPartitionerFlinkIcebergSink.java | 6 +- .../flink/sink/TestFlinkIcebergSink.java | 14 ++--- .../sink/TestFlinkIcebergSinkBranch.java | 6 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 6 +- .../flink/source/TestFlinkInputFormat.java | 8 +-- .../iceberg/flink/source/TestFlinkScan.java | 30 +++++----- .../flink/source/TestFlinkScanSql.java | 2 +- .../iceberg/flink/source/TestFlinkSource.java | 2 +- .../flink/source/TestFlinkSourceSql.java | 4 +- .../source/TestIcebergSourceBounded.java | 4 +- .../source/TestIcebergSourceBoundedSql.java | 2 +- .../flink/source/TestIcebergSourceSql.java | 4 +- ...stIcebergSourceWithWatermarkExtractor.java | 10 ++-- .../iceberg/flink/source/TestSqlBase.java | 6 +- .../TestContinuousSplitPlannerImpl.java | 8 +-- .../TestColumnStatsWatermarkExtractor.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 6 +- .../flink/sink/TestFlinkIcebergSink.java | 14 ++--- .../sink/TestFlinkIcebergSinkBranch.java | 6 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 6 +- .../flink/source/TestFlinkInputFormat.java | 8 +-- .../iceberg/flink/source/TestFlinkScan.java | 30 +++++----- .../flink/source/TestFlinkScanSql.java | 2 +- .../iceberg/flink/source/TestFlinkSource.java | 2 +- .../flink/source/TestFlinkSourceSql.java | 4 +- .../source/TestIcebergSourceBounded.java | 4 +- .../source/TestIcebergSourceBoundedSql.java | 2 +- .../flink/source/TestIcebergSourceSql.java | 4 +- ...stIcebergSourceWithWatermarkExtractor.java | 10 ++-- .../iceberg/flink/source/TestSqlBase.java | 6 +- .../TestContinuousSplitPlannerImpl.java | 8 +-- .../TestColumnStatsWatermarkExtractor.java | 2 +- .../maintenance/operator/CollectingSink.java | 16 +++--- .../maintenance/operator/ManualSource.java | 20 +++---- ...TestBucketPartitionerFlinkIcebergSink.java | 6 +- .../flink/sink/TestFlinkIcebergSink.java | 14 ++--- .../sink/TestFlinkIcebergSinkBranch.java | 6 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 6 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 6 +- .../flink/source/TestFlinkInputFormat.java | 8 +-- .../iceberg/flink/source/TestFlinkScan.java | 30 +++++----- .../flink/source/TestFlinkScanSql.java | 2 +- .../iceberg/flink/source/TestFlinkSource.java | 2 +- .../flink/source/TestFlinkSourceSql.java | 4 +- .../source/TestIcebergSourceBounded.java | 4 +- .../source/TestIcebergSourceBoundedSql.java | 2 +- .../flink/source/TestIcebergSourceSql.java | 4 +- ...stIcebergSourceWithWatermarkExtractor.java | 10 ++-- .../iceberg/flink/source/TestSqlBase.java | 6 +- .../TestContinuousSplitPlannerImpl.java | 8 +-- .../TestColumnStatsWatermarkExtractor.java | 2 +- .../extensions/TestAddFilesProcedure.java | 18 +++--- .../TestSnapshotTableProcedure.java | 40 ++++++------- .../data/vectorized/ArrowVectorAccessors.java | 4 +- .../TestParquetVectorizedReads.java | 24 ++++---- .../iceberg/spark/source/LogMessage.java | 18 +++--- .../iceberg/spark/source/ManualSource.java | 12 ++-- .../spark/source/TestSparkCatalog.java | 12 ++-- .../TestSparkCatalogCacheExpiration.java | 8 +-- .../TestSparkCatalogHadoopOverrides.java | 32 +++++------ .../spark/sql/TestTimestampWithoutZone.java | 30 +++++----- .../extensions/TestAddFilesProcedure.java | 18 +++--- .../TestSnapshotTableProcedure.java | 40 ++++++------- .../data/vectorized/ArrowVectorAccessors.java | 4 +- .../iceberg/spark/source/LogMessage.java | 18 +++--- .../iceberg/spark/source/ManualSource.java | 12 ++-- .../spark/source/TestCompressionSettings.java | 19 +++---- .../spark/source/TestSparkCatalog.java | 12 ++-- .../TestSparkCatalogCacheExpiration.java | 8 +-- .../TestSparkCatalogHadoopOverrides.java | 32 +++++------ .../spark/sql/TestTimestampWithoutZone.java | 34 +++++------ .../extensions/TestAddFilesProcedure.java | 18 +++--- .../TestSnapshotTableProcedure.java | 56 +++++++++---------- .../data/vectorized/ArrowVectorAccessors.java | 4 +- .../spark/TestSparkCatalogOperations.java | 13 +++-- .../actions/TestSnapshotTableAction.java | 12 ++-- .../iceberg/spark/source/LogMessage.java | 18 +++--- .../iceberg/spark/source/ManualSource.java | 12 ++-- .../spark/source/TestCompressionSettings.java | 18 +++--- .../spark/source/TestSparkCatalog.java | 12 ++-- .../TestSparkCatalogCacheExpiration.java | 4 +- .../TestSparkCatalogHadoopOverrides.java | 32 +++++------ .../spark/sql/TestTimestampWithoutZone.java | 30 +++++----- 85 files changed, 510 insertions(+), 510 deletions(-) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index 1ec0cc53b45a..dc3eb93280df 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -72,7 +72,7 @@ public class TestBucketPartitionerFlinkIcebergSink { .build()); @RegisterExtension - private static final HadoopCatalogExtension catalogExtension = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private static final TypeInformation ROW_TYPE_INFO = @@ -90,7 +90,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private void setupEnvironment(TableSchemaType tableSchemaType) { PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable( TABLE_IDENTIFIER, @@ -102,7 +102,7 @@ private void setupEnvironment(TableSchemaType tableSchemaType) { .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism * 2); - tableLoader = catalogExtension.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } private void appendRowsToTable(List allRows) throws Exception { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 527525e9f167..8cad35c859c4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -66,7 +66,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private TableLoader tableLoader; @@ -101,7 +101,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -118,7 +118,7 @@ public void before() throws IOException { .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -244,7 +244,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table leftTable = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TableIdentifier.of("left"), @@ -254,10 +254,10 @@ public void testTwoSinksInDisjointedDAG() throws Exception { : PartitionSpec.unpartitioned(), props); TableLoader leftTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); Table rightTable = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TableIdentifier.of("right"), @@ -267,7 +267,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { : PartitionSpec.unpartitioned(), props); TableLoader rightTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); env = StreamExecutionEnvironment.getExecutionEnvironment( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 547b4937c5bd..3edaafca0e42 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -50,7 +50,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { @RegisterExtension - public static final HadoopCatalogExtension catalogResource = + public static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -74,7 +74,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -91,7 +91,7 @@ public void before() throws IOException { MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 9cbb9f091e15..55909874ccce 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -61,13 +61,13 @@ public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @BeforeEach public void setupTable() { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -94,7 +94,7 @@ public void setupTable() { .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index 1c5c97b58d2d..ffeab673386d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -57,7 +57,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -74,7 +74,7 @@ public void before() throws IOException { MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index 366a3c2b5087..c8b65e131c33 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -75,7 +75,7 @@ public void testNestedProjection() throws Exception { required(6, "id", Types.LongType.get())); Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), schema); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), schema); List writeRecords = RandomGenericData.generate(schema, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); @@ -115,7 +115,7 @@ public void testBasicProjection() throws IOException { Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); @@ -157,7 +157,7 @@ public void testReadPartitionColumn() throws Exception { PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); List records = RandomGenericData.generate(nestedSchema, 10, 0L); GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -189,7 +189,7 @@ public void testReadPartitionColumn() throws Exception { @TestTemplate public void testValidation() { - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); assertThatThrownBy( () -> diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 428da49f1de6..049ddf9e3f1e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -70,7 +70,7 @@ public abstract class TestFlinkScan { @TempDir protected Path temporaryDirectory; @RegisterExtension - protected static final HadoopCatalogExtension catalogExtension = + protected static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); @Parameter protected FileFormat fileFormat; @@ -81,7 +81,7 @@ public static Collection fileFormat() { } protected TableLoader tableLoader() { - return catalogExtension.tableLoader(); + return CATALOG_EXTENSION.tableLoader(); } protected abstract List runWithProjection(String... projected) throws Exception; @@ -100,7 +100,7 @@ protected List runWithFilter(Expression filter, String sqlFilter) throws Ex @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); @@ -109,7 +109,7 @@ public void testUnpartitionedTable() throws Exception { @TestTemplate public void testPartitionedTable() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); @@ -122,7 +122,7 @@ public void testPartitionedTable() throws Exception { @TestTemplate public void testProjection() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); @@ -143,7 +143,7 @@ public void testIdentityPartitionProjections() throws Exception { PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); int idx = 0; @@ -208,7 +208,7 @@ private void validateIdentityPartitionProjections( @TestTemplate public void testSnapshotReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -235,7 +235,7 @@ public void testSnapshotReads() throws Exception { @TestTemplate public void testTagReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -264,7 +264,7 @@ public void testTagReads() throws Exception { @TestTemplate public void testBranchReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -300,7 +300,7 @@ public void testBranchReads() throws Exception { @TestTemplate public void testIncrementalReadViaTag() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -378,7 +378,7 @@ public void testIncrementalReadViaTag() throws Exception { @TestTemplate public void testIncrementalRead() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -413,7 +413,7 @@ public void testIncrementalRead() throws Exception { @TestTemplate public void testFilterExpPartition() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); @@ -438,7 +438,7 @@ public void testFilterExpPartition() throws Exception { private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); expectedRecords.get(0).set(0, "a"); @@ -489,7 +489,7 @@ public void testPartitionTypes() throws Exception { .build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); List records = RandomGenericData.generate(typesSchema, 10, 0L); GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -520,7 +520,7 @@ public void testCustomizedFlinkDataTypes() throws Exception { Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), Types.NestedField.required( 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); - Table table = catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); List records = RandomGenericData.generate(schema, 10, 0L); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); helper.appendToTable(records); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java index b5bddc7767fb..1493c0932044 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -36,7 +36,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index ff7892071231..dd50170f0fd7 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -38,7 +38,7 @@ protected List runWithProjection(String... projected) throws Exception { TableSchema schema = FlinkSchemaUtil.toSchema( FlinkSchemaUtil.convert( - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); for (String field : projected) { TableColumn column = schema.getTableColumn(field).get(); builder.field(column.getName(), column.getType()); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java index 6857e0a7a366..e1162c3225b1 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -42,7 +42,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() @@ -56,7 +56,7 @@ public void testInferParallelismWithGlobalSetting() throws IOException { cfg.set(PipelineOptions.MAX_PARALLELISM, 1); Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index 0bf09b1643be..b7447d15c05a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -50,7 +50,7 @@ public class TestIcebergSourceBounded extends TestFlinkScan { @TestTemplate public void testValidation() { - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); assertThatThrownBy( () -> @@ -68,7 +68,7 @@ public void testValidation() { @Override protected List runWithProjection(String... projected) throws Exception { Schema icebergTableSchema = - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); TableSchema.Builder builder = TableSchema.builder(); TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); for (String field : projected) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index ff3348bbc3a3..0f41c5af4c95 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -41,7 +41,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java index 645af7cfa339..75f0a785a8c5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -59,7 +59,7 @@ public void before() throws IOException { SqlHelpers.sql( tableEnvironment, "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); @@ -74,7 +74,7 @@ private Record generateRecord(Instant t1, long t2) { /** Generates the records in the expected order, with respect to their datafile */ private List generateExpectedRecords(boolean ascending) throws Exception { - Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); long baseTime = 1702382109000L; GenericAppenderHelper helper = diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index bc1808ee77da..8341c8074f21 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -72,7 +72,7 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; private static final String SOURCE_NAME = "IcebergSource"; private static final int RECORD_NUM_FOR_2_SPLITS = 200; - private static final ConcurrentMap windows = Maps.newConcurrentMap(); + private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -187,12 +187,12 @@ public void apply( AtomicInteger count = new AtomicInteger(0); values.forEach(a -> count.incrementAndGet()); out.collect(row(window.getStart(), count.get())); - windows.put(window.getStart(), count.get()); + WINDOWS.put(window.getStart(), count.get()); } }); // Use static variable to collect the windows, since other solutions were flaky - windows.clear(); + WINDOWS.clear(); env.executeAsync("Iceberg Source Windowing Test"); // Wait for the 2 first windows from File 2 and File 3 @@ -201,7 +201,7 @@ public void apply( .atMost(30, TimeUnit.SECONDS) .until( () -> - windows.equals( + WINDOWS.equals( ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); // Write data so the windows containing test data are closed @@ -214,7 +214,7 @@ public void apply( .atMost(30, TimeUnit.SECONDS) .until( () -> - windows.equals( + WINDOWS.equals( ImmutableMap.of( 0L, RECORD_NUM_FOR_2_SPLITS, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 94962e02bb05..8013bce3f415 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -56,7 +56,7 @@ public abstract class TestSqlBase { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - public static final HadoopCatalogExtension catalogResource = + public static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @TempDir protected Path temporaryFolder; @@ -81,7 +81,7 @@ protected TableEnvironment getTableEnv() { @Test public void testResiduals() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); @@ -113,7 +113,7 @@ public void testResiduals() throws Exception { @Test public void testExposeLocality() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java index 381c340edd6a..0690b456e033 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -48,8 +48,8 @@ public class TestContinuousSplitPlannerImpl { @TempDir protected Path temporaryFolder; - private static final FileFormat fileFormat = FileFormat.PARQUET; - private static final AtomicLong randomSeed = new AtomicLong(); + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private static final AtomicLong RANDOM_SEED = new AtomicLong(); @RegisterExtension private static final HadoopTableExtension TABLE_RESOURCE = @@ -63,7 +63,7 @@ public class TestContinuousSplitPlannerImpl { @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), fileFormat, temporaryFolder); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); } private void appendTwoSnapshots() throws IOException { @@ -85,7 +85,7 @@ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { List batch = - RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); + RandomGenericData.generate(TestFixtures.SCHEMA, 2, RANDOM_SEED.incrementAndGet()); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 187f2b0b9e31..8bd1214bd960 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -74,7 +74,7 @@ public class TestColumnStatsWatermarkExtractor { @TempDir protected Path temporaryFolder; @RegisterExtension - private static final HadoopTableExtension sourceTableResource = + private static final HadoopTableExtension SOURCE_TABLE_RESOURCE = new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); @Parameter(index = 0) 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 1ec0cc53b45a..dc3eb93280df 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 @@ -72,7 +72,7 @@ public class TestBucketPartitionerFlinkIcebergSink { .build()); @RegisterExtension - private static final HadoopCatalogExtension catalogExtension = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private static final TypeInformation ROW_TYPE_INFO = @@ -90,7 +90,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private void setupEnvironment(TableSchemaType tableSchemaType) { PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable( TABLE_IDENTIFIER, @@ -102,7 +102,7 @@ private void setupEnvironment(TableSchemaType tableSchemaType) { .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism * 2); - tableLoader = catalogExtension.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } private void appendRowsToTable(List allRows) throws Exception { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 527525e9f167..8cad35c859c4 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -66,7 +66,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private TableLoader tableLoader; @@ -101,7 +101,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -118,7 +118,7 @@ public void before() throws IOException { .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -244,7 +244,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table leftTable = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TableIdentifier.of("left"), @@ -254,10 +254,10 @@ public void testTwoSinksInDisjointedDAG() throws Exception { : PartitionSpec.unpartitioned(), props); TableLoader leftTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); Table rightTable = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TableIdentifier.of("right"), @@ -267,7 +267,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { : PartitionSpec.unpartitioned(), props); TableLoader rightTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); env = StreamExecutionEnvironment.getExecutionEnvironment( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 547b4937c5bd..3edaafca0e42 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -50,7 +50,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { @RegisterExtension - public static final HadoopCatalogExtension catalogResource = + public static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -74,7 +74,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -91,7 +91,7 @@ public void before() throws IOException { MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate 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 9cbb9f091e15..55909874ccce 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 @@ -61,13 +61,13 @@ public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @BeforeEach public void setupTable() { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -94,7 +94,7 @@ public void setupTable() { .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate 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 1c5c97b58d2d..ffeab673386d 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 @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -57,7 +57,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -74,7 +74,7 @@ public void before() throws IOException { MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index 366a3c2b5087..c8b65e131c33 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -75,7 +75,7 @@ public void testNestedProjection() throws Exception { required(6, "id", Types.LongType.get())); Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), schema); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), schema); List writeRecords = RandomGenericData.generate(schema, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); @@ -115,7 +115,7 @@ public void testBasicProjection() throws IOException { Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); @@ -157,7 +157,7 @@ public void testReadPartitionColumn() throws Exception { PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); List records = RandomGenericData.generate(nestedSchema, 10, 0L); GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -189,7 +189,7 @@ public void testReadPartitionColumn() throws Exception { @TestTemplate public void testValidation() { - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); assertThatThrownBy( () -> diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 428da49f1de6..049ddf9e3f1e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -70,7 +70,7 @@ public abstract class TestFlinkScan { @TempDir protected Path temporaryDirectory; @RegisterExtension - protected static final HadoopCatalogExtension catalogExtension = + protected static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); @Parameter protected FileFormat fileFormat; @@ -81,7 +81,7 @@ public static Collection fileFormat() { } protected TableLoader tableLoader() { - return catalogExtension.tableLoader(); + return CATALOG_EXTENSION.tableLoader(); } protected abstract List runWithProjection(String... projected) throws Exception; @@ -100,7 +100,7 @@ protected List runWithFilter(Expression filter, String sqlFilter) throws Ex @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); @@ -109,7 +109,7 @@ public void testUnpartitionedTable() throws Exception { @TestTemplate public void testPartitionedTable() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); @@ -122,7 +122,7 @@ public void testPartitionedTable() throws Exception { @TestTemplate public void testProjection() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); @@ -143,7 +143,7 @@ public void testIdentityPartitionProjections() throws Exception { PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); int idx = 0; @@ -208,7 +208,7 @@ private void validateIdentityPartitionProjections( @TestTemplate public void testSnapshotReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -235,7 +235,7 @@ public void testSnapshotReads() throws Exception { @TestTemplate public void testTagReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -264,7 +264,7 @@ public void testTagReads() throws Exception { @TestTemplate public void testBranchReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -300,7 +300,7 @@ public void testBranchReads() throws Exception { @TestTemplate public void testIncrementalReadViaTag() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -378,7 +378,7 @@ public void testIncrementalReadViaTag() throws Exception { @TestTemplate public void testIncrementalRead() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -413,7 +413,7 @@ public void testIncrementalRead() throws Exception { @TestTemplate public void testFilterExpPartition() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); @@ -438,7 +438,7 @@ public void testFilterExpPartition() throws Exception { private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); expectedRecords.get(0).set(0, "a"); @@ -489,7 +489,7 @@ public void testPartitionTypes() throws Exception { .build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); List records = RandomGenericData.generate(typesSchema, 10, 0L); GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -520,7 +520,7 @@ public void testCustomizedFlinkDataTypes() throws Exception { Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), Types.NestedField.required( 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); - Table table = catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); List records = RandomGenericData.generate(schema, 10, 0L); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); helper.appendToTable(records); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java index b5bddc7767fb..1493c0932044 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -36,7 +36,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index ff7892071231..dd50170f0fd7 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -38,7 +38,7 @@ protected List runWithProjection(String... projected) throws Exception { TableSchema schema = FlinkSchemaUtil.toSchema( FlinkSchemaUtil.convert( - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); for (String field : projected) { TableColumn column = schema.getTableColumn(field).get(); builder.field(column.getName(), column.getType()); 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 6857e0a7a366..e1162c3225b1 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 @@ -42,7 +42,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() @@ -56,7 +56,7 @@ public void testInferParallelismWithGlobalSetting() throws IOException { cfg.set(PipelineOptions.MAX_PARALLELISM, 1); Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index 0bf09b1643be..b7447d15c05a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -50,7 +50,7 @@ public class TestIcebergSourceBounded extends TestFlinkScan { @TestTemplate public void testValidation() { - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); assertThatThrownBy( () -> @@ -68,7 +68,7 @@ public void testValidation() { @Override protected List runWithProjection(String... projected) throws Exception { Schema icebergTableSchema = - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); TableSchema.Builder builder = TableSchema.builder(); TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); for (String field : projected) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index ff3348bbc3a3..0f41c5af4c95 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -41,7 +41,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() 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 645af7cfa339..75f0a785a8c5 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 @@ -59,7 +59,7 @@ public void before() throws IOException { SqlHelpers.sql( tableEnvironment, "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); @@ -74,7 +74,7 @@ private Record generateRecord(Instant t1, long t2) { /** Generates the records in the expected order, with respect to their datafile */ private List generateExpectedRecords(boolean ascending) throws Exception { - Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); long baseTime = 1702382109000L; GenericAppenderHelper helper = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index bc1808ee77da..8341c8074f21 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -72,7 +72,7 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; private static final String SOURCE_NAME = "IcebergSource"; private static final int RECORD_NUM_FOR_2_SPLITS = 200; - private static final ConcurrentMap windows = Maps.newConcurrentMap(); + private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -187,12 +187,12 @@ public void apply( AtomicInteger count = new AtomicInteger(0); values.forEach(a -> count.incrementAndGet()); out.collect(row(window.getStart(), count.get())); - windows.put(window.getStart(), count.get()); + WINDOWS.put(window.getStart(), count.get()); } }); // Use static variable to collect the windows, since other solutions were flaky - windows.clear(); + WINDOWS.clear(); env.executeAsync("Iceberg Source Windowing Test"); // Wait for the 2 first windows from File 2 and File 3 @@ -201,7 +201,7 @@ public void apply( .atMost(30, TimeUnit.SECONDS) .until( () -> - windows.equals( + WINDOWS.equals( ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); // Write data so the windows containing test data are closed @@ -214,7 +214,7 @@ public void apply( .atMost(30, TimeUnit.SECONDS) .until( () -> - windows.equals( + WINDOWS.equals( ImmutableMap.of( 0L, RECORD_NUM_FOR_2_SPLITS, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 94962e02bb05..8013bce3f415 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -56,7 +56,7 @@ public abstract class TestSqlBase { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - public static final HadoopCatalogExtension catalogResource = + public static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @TempDir protected Path temporaryFolder; @@ -81,7 +81,7 @@ protected TableEnvironment getTableEnv() { @Test public void testResiduals() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); @@ -113,7 +113,7 @@ public void testResiduals() throws Exception { @Test public void testExposeLocality() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); 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 381c340edd6a..0690b456e033 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 @@ -48,8 +48,8 @@ public class TestContinuousSplitPlannerImpl { @TempDir protected Path temporaryFolder; - private static final FileFormat fileFormat = FileFormat.PARQUET; - private static final AtomicLong randomSeed = new AtomicLong(); + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private static final AtomicLong RANDOM_SEED = new AtomicLong(); @RegisterExtension private static final HadoopTableExtension TABLE_RESOURCE = @@ -63,7 +63,7 @@ public class TestContinuousSplitPlannerImpl { @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), fileFormat, temporaryFolder); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); } private void appendTwoSnapshots() throws IOException { @@ -85,7 +85,7 @@ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { List batch = - RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); + RandomGenericData.generate(TestFixtures.SCHEMA, 2, RANDOM_SEED.incrementAndGet()); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 187f2b0b9e31..af806d4c655d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -74,7 +74,7 @@ public class TestColumnStatsWatermarkExtractor { @TempDir protected Path temporaryFolder; @RegisterExtension - private static final HadoopTableExtension sourceTableResource = + private static final HadoopTableExtension SOURCE_TABLE_EXTENSION = new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); @Parameter(index = 0) 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 a49459d61a54..9b6580fad0bf 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 @@ -33,15 +33,15 @@ /** Sink for collecting output during testing. */ class CollectingSink implements Sink { private static final long serialVersionUID = 1L; - private static final List> queues = + private static final List> QUEUES = Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); - private static final AtomicInteger numSinks = new AtomicInteger(-1); + private static final AtomicInteger NUM_SINKS = new AtomicInteger(-1); private final int index; /** Creates a new sink which collects the elements received. */ CollectingSink() { - this.index = numSinks.incrementAndGet(); - queues.add(new LinkedBlockingQueue<>()); + this.index = NUM_SINKS.incrementAndGet(); + QUEUES.add(new LinkedBlockingQueue<>()); } /** @@ -50,7 +50,7 @@ class CollectingSink implements Sink { * @return all the remaining output */ List remainingOutput() { - return Lists.newArrayList((BlockingQueue) queues.get(this.index)); + return Lists.newArrayList((BlockingQueue) QUEUES.get(this.index)); } /** @@ -59,7 +59,7 @@ List remainingOutput() { * @return true if there is no remaining output */ boolean isEmpty() { - return queues.get(this.index).isEmpty(); + return QUEUES.get(this.index).isEmpty(); } /** @@ -73,7 +73,7 @@ T poll(Duration timeout) throws TimeoutException { Object element; try { - element = queues.get(this.index).poll(timeout.toMillis(), TimeUnit.MILLISECONDS); + element = QUEUES.get(this.index).poll(timeout.toMillis(), TimeUnit.MILLISECONDS); } catch (InterruptedException var4) { throw new RuntimeException(var4); } @@ -99,7 +99,7 @@ private static class CollectingWriter implements SinkWriter { @Override public void write(T element, Context context) { - queues.get(index).add(element); + QUEUES.get(index).add(element); } @Override 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 e08742a89ddb..38bb9c393fa9 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 @@ -48,9 +48,9 @@ class ManualSource ResultTypeQueryable { private static final long serialVersionUID = 1L; - private static final List>> queues = + private static final List>> QUEUES = Collections.synchronizedList(Lists.newArrayList()); - private static final List> availabilities = + private static final List> AVAILABILITIES = Collections.synchronizedList(Lists.newArrayList()); private static int numSources = 0; private final TypeInformation type; @@ -68,8 +68,8 @@ class ManualSource this.type = type; this.env = env; this.index = numSources++; - queues.add(Queues.newArrayDeque()); - availabilities.add(new CompletableFuture<>()); + QUEUES.add(Queues.newArrayDeque()); + AVAILABILITIES.add(new CompletableFuture<>()); } /** @@ -123,8 +123,8 @@ DataStream dataStream() { } private void sendInternal(Tuple2 tuple) { - queues.get(index).offer(tuple); - availabilities.get(index).complete(null); + QUEUES.get(index).offer(tuple); + AVAILABILITIES.get(index).complete(null); } @Override @@ -164,7 +164,7 @@ public void start() { @Override public InputStatus pollNext(ReaderOutput output) { - Tuple2 next = (Tuple2) queues.get(index).poll(); + Tuple2 next = (Tuple2) QUEUES.get(index).poll(); if (next != null) { if (next.f0 == null) { @@ -181,8 +181,8 @@ public InputStatus pollNext(ReaderOutput output) { } } - availabilities.set(index, new CompletableFuture<>()); - return queues.get(index).isEmpty() + AVAILABILITIES.set(index, new CompletableFuture<>()); + return QUEUES.get(index).isEmpty() ? InputStatus.NOTHING_AVAILABLE : InputStatus.MORE_AVAILABLE; } @@ -194,7 +194,7 @@ public List snapshotState(long checkpointId) { @Override public CompletableFuture isAvailable() { - return availabilities.get(index); + return AVAILABILITIES.get(index); } @Override 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 1ec0cc53b45a..dc3eb93280df 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 @@ -72,7 +72,7 @@ public class TestBucketPartitionerFlinkIcebergSink { .build()); @RegisterExtension - private static final HadoopCatalogExtension catalogExtension = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private static final TypeInformation ROW_TYPE_INFO = @@ -90,7 +90,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private void setupEnvironment(TableSchemaType tableSchemaType) { PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable( TABLE_IDENTIFIER, @@ -102,7 +102,7 @@ private void setupEnvironment(TableSchemaType tableSchemaType) { .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism * 2); - tableLoader = catalogExtension.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } private void appendRowsToTable(List allRows) throws Exception { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 527525e9f167..8cad35c859c4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -66,7 +66,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); private TableLoader tableLoader; @@ -101,7 +101,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -118,7 +118,7 @@ public void before() throws IOException { .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -244,7 +244,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table leftTable = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TableIdentifier.of("left"), @@ -254,10 +254,10 @@ public void testTwoSinksInDisjointedDAG() throws Exception { : PartitionSpec.unpartitioned(), props); TableLoader leftTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("left")); + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); Table rightTable = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TableIdentifier.of("right"), @@ -267,7 +267,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { : PartitionSpec.unpartitioned(), props); TableLoader rightTableLoader = - TableLoader.fromCatalog(catalogResource.catalogLoader(), TableIdentifier.of("right")); + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); env = StreamExecutionEnvironment.getExecutionEnvironment( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 547b4937c5bd..3edaafca0e42 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -50,7 +50,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { @RegisterExtension - public static final HadoopCatalogExtension catalogResource = + public static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -74,7 +74,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -91,7 +91,7 @@ public void before() throws IOException { MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate 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 9cbb9f091e15..55909874ccce 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 @@ -61,13 +61,13 @@ public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @BeforeEach public void setupTable() { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -94,7 +94,7 @@ public void setupTable() { .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate 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 1c5c97b58d2d..ffeab673386d 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 @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension catalogResource = + private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) @@ -57,7 +57,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable( TestFixtures.TABLE_IDENTIFIER, @@ -74,7 +74,7 @@ public void before() throws IOException { MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); - tableLoader = catalogResource.tableLoader(); + tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java index 366a3c2b5087..c8b65e131c33 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java @@ -75,7 +75,7 @@ public void testNestedProjection() throws Exception { required(6, "id", Types.LongType.get())); Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), schema); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), schema); List writeRecords = RandomGenericData.generate(schema, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); @@ -115,7 +115,7 @@ public void testBasicProjection() throws IOException { Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); Table table = - catalogExtension.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); @@ -157,7 +157,7 @@ public void testReadPartitionColumn() throws Exception { PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); List records = RandomGenericData.generate(nestedSchema, 10, 0L); GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -189,7 +189,7 @@ public void testReadPartitionColumn() throws Exception { @TestTemplate public void testValidation() { - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); assertThatThrownBy( () -> diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 428da49f1de6..049ddf9e3f1e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -70,7 +70,7 @@ public abstract class TestFlinkScan { @TempDir protected Path temporaryDirectory; @RegisterExtension - protected static final HadoopCatalogExtension catalogExtension = + protected static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); @Parameter protected FileFormat fileFormat; @@ -81,7 +81,7 @@ public static Collection fileFormat() { } protected TableLoader tableLoader() { - return catalogExtension.tableLoader(); + return CATALOG_EXTENSION.tableLoader(); } protected abstract List runWithProjection(String... projected) throws Exception; @@ -100,7 +100,7 @@ protected List runWithFilter(Expression filter, String sqlFilter) throws Ex @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); @@ -109,7 +109,7 @@ public void testUnpartitionedTable() throws Exception { @TestTemplate public void testPartitionedTable() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); @@ -122,7 +122,7 @@ public void testPartitionedTable() throws Exception { @TestTemplate public void testProjection() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); @@ -143,7 +143,7 @@ public void testIdentityPartitionProjections() throws Exception { PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); int idx = 0; @@ -208,7 +208,7 @@ private void validateIdentityPartitionProjections( @TestTemplate public void testSnapshotReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -235,7 +235,7 @@ public void testSnapshotReads() throws Exception { @TestTemplate public void testTagReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -264,7 +264,7 @@ public void testTagReads() throws Exception { @TestTemplate public void testBranchReads() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -300,7 +300,7 @@ public void testBranchReads() throws Exception { @TestTemplate public void testIncrementalReadViaTag() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -378,7 +378,7 @@ public void testIncrementalReadViaTag() throws Exception { @TestTemplate public void testIncrementalRead() throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -413,7 +413,7 @@ public void testIncrementalRead() throws Exception { @TestTemplate public void testFilterExpPartition() throws Exception { Table table = - catalogExtension + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); @@ -438,7 +438,7 @@ public void testFilterExpPartition() throws Exception { private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) throws Exception { Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); expectedRecords.get(0).set(0, "a"); @@ -489,7 +489,7 @@ public void testPartitionTypes() throws Exception { .build(); Table table = - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); List records = RandomGenericData.generate(typesSchema, 10, 0L); GenericAppenderHelper appender = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); @@ -520,7 +520,7 @@ public void testCustomizedFlinkDataTypes() throws Exception { Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), Types.NestedField.required( 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); - Table table = catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); List records = RandomGenericData.generate(schema, 10, 0L); GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); helper.appendToTable(records); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java index b5bddc7767fb..1493c0932044 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java @@ -36,7 +36,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java index ff7892071231..dd50170f0fd7 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -38,7 +38,7 @@ protected List runWithProjection(String... projected) throws Exception { TableSchema schema = FlinkSchemaUtil.toSchema( FlinkSchemaUtil.convert( - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); for (String field : projected) { TableColumn column = schema.getTableColumn(field).get(); builder.field(column.getName(), column.getType()); 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 6857e0a7a366..e1162c3225b1 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 @@ -42,7 +42,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() @@ -56,7 +56,7 @@ public void testInferParallelismWithGlobalSetting() throws IOException { cfg.set(PipelineOptions.MAX_PARALLELISM, 1); Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index 0bf09b1643be..b7447d15c05a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -50,7 +50,7 @@ public class TestIcebergSourceBounded extends TestFlinkScan { @TestTemplate public void testValidation() { - catalogExtension.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); assertThatThrownBy( () -> @@ -68,7 +68,7 @@ public void testValidation() { @Override protected List runWithProjection(String... projected) throws Exception { Schema icebergTableSchema = - catalogExtension.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); TableSchema.Builder builder = TableSchema.builder(); TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); for (String field : projected) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index ff3348bbc3a3..0f41c5af4c95 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -41,7 +41,7 @@ public void before() throws IOException { SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogExtension.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); getTableEnv() .getConfig() 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 645af7cfa339..75f0a785a8c5 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 @@ -59,7 +59,7 @@ public void before() throws IOException { SqlHelpers.sql( tableEnvironment, "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - catalogResource.warehouse()); + CATALOG_EXTENSION.warehouse()); SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); @@ -74,7 +74,7 @@ private Record generateRecord(Instant t1, long t2) { /** Generates the records in the expected order, with respect to their datafile */ private List generateExpectedRecords(boolean ascending) throws Exception { - Table table = catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); long baseTime = 1702382109000L; GenericAppenderHelper helper = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index bc1808ee77da..8341c8074f21 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -72,7 +72,7 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; private static final String SOURCE_NAME = "IcebergSource"; private static final int RECORD_NUM_FOR_2_SPLITS = 200; - private static final ConcurrentMap windows = Maps.newConcurrentMap(); + private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); @@ -187,12 +187,12 @@ public void apply( AtomicInteger count = new AtomicInteger(0); values.forEach(a -> count.incrementAndGet()); out.collect(row(window.getStart(), count.get())); - windows.put(window.getStart(), count.get()); + WINDOWS.put(window.getStart(), count.get()); } }); // Use static variable to collect the windows, since other solutions were flaky - windows.clear(); + WINDOWS.clear(); env.executeAsync("Iceberg Source Windowing Test"); // Wait for the 2 first windows from File 2 and File 3 @@ -201,7 +201,7 @@ public void apply( .atMost(30, TimeUnit.SECONDS) .until( () -> - windows.equals( + WINDOWS.equals( ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); // Write data so the windows containing test data are closed @@ -214,7 +214,7 @@ public void apply( .atMost(30, TimeUnit.SECONDS) .until( () -> - windows.equals( + WINDOWS.equals( ImmutableMap.of( 0L, RECORD_NUM_FOR_2_SPLITS, diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 94962e02bb05..8013bce3f415 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -56,7 +56,7 @@ public abstract class TestSqlBase { MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension - public static final HadoopCatalogExtension catalogResource = + public static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @TempDir protected Path temporaryFolder; @@ -81,7 +81,7 @@ protected TableEnvironment getTableEnv() { @Test public void testResiduals() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); @@ -113,7 +113,7 @@ public void testResiduals() throws Exception { @Test public void testExposeLocality() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); 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 381c340edd6a..0690b456e033 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 @@ -48,8 +48,8 @@ public class TestContinuousSplitPlannerImpl { @TempDir protected Path temporaryFolder; - private static final FileFormat fileFormat = FileFormat.PARQUET; - private static final AtomicLong randomSeed = new AtomicLong(); + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private static final AtomicLong RANDOM_SEED = new AtomicLong(); @RegisterExtension private static final HadoopTableExtension TABLE_RESOURCE = @@ -63,7 +63,7 @@ public class TestContinuousSplitPlannerImpl { @BeforeEach public void before() throws IOException { - dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), fileFormat, temporaryFolder); + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); } private void appendTwoSnapshots() throws IOException { @@ -85,7 +85,7 @@ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { List batch = - RandomGenericData.generate(TestFixtures.SCHEMA, 2, randomSeed.incrementAndGet()); + RandomGenericData.generate(TestFixtures.SCHEMA, 2, RANDOM_SEED.incrementAndGet()); DataFile dataFile = dataAppender.writeFile(null, batch); dataAppender.appendToTable(dataFile); Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java index 187f2b0b9e31..af806d4c655d 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -74,7 +74,7 @@ public class TestColumnStatsWatermarkExtractor { @TempDir protected Path temporaryFolder; @RegisterExtension - private static final HadoopTableExtension sourceTableResource = + private static final HadoopTableExtension SOURCE_TABLE_EXTENSION = new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); @Parameter(index = 0) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index b17ffddf8287..44ee2ebdb646 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -894,7 +894,7 @@ public void testEmptyImportDoesNotThrow() { assertOutput(pathResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty path", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); // Empty table based import @@ -907,7 +907,7 @@ public void testEmptyImportDoesNotThrow() { assertOutput(tableResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty table", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); } @@ -935,13 +935,13 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { assertOutput(tableResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty table", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); } - private static final List emptyQueryResult = Lists.newArrayList(); + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); - private static final StructField[] struct = { + private static final StructField[] STRUCT = { new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), new StructField("name", DataTypes.StringType, true, Metadata.empty()), new StructField("dept", DataTypes.StringType, true, Metadata.empty()), @@ -956,14 +956,14 @@ private Dataset unpartitionedDF() { RowFactory.create(2, "Jane Doe", "hr", "salary"), RowFactory.create(3, "Matt Doe", "hr", "communications"), RowFactory.create(4, "Will Doe", "facilities", "all")), - new StructType(struct)) + new StructType(STRUCT)) .repartition(1); } private Dataset singleNullRecordDF() { return spark .createDataFrame( - ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(struct)) + ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(STRUCT)) .repartition(1); } @@ -988,7 +988,7 @@ private Dataset weirdColumnNamesDF() { unpartitionedDF.col("name").as("naMe")); } - private static final StructField[] dateStruct = { + private static final StructField[] DATE_STRUCT = { new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), new StructField("name", DataTypes.StringType, true, Metadata.empty()), new StructField("ts", DataTypes.DateType, true, Metadata.empty()), @@ -1007,7 +1007,7 @@ private Dataset dateDF() { RowFactory.create(2, "Jane Doe", toDate("2021-01-01"), "01"), RowFactory.create(3, "Matt Doe", toDate("2021-01-02"), "02"), RowFactory.create(4, "Will Doe", toDate("2021-01-02"), "02")), - new StructType(dateStruct)) + new StructType(DATE_STRUCT)) .repartition(2); } diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index cf297e7fc09c..5b7b9097755b 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -34,7 +34,7 @@ import org.junit.rules.TemporaryFolder; public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { - private static final String sourceName = "spark_catalog.default.source"; + private static final String SOURCE_NAME = "spark_catalog.default.source"; // Currently we can only Snapshot only out of the Spark Session Catalog public TestSnapshotTableProcedure( @@ -47,7 +47,7 @@ public TestSnapshotTableProcedure( @After public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %S", sourceName); + sql("DROP TABLE IF EXISTS %S", SOURCE_NAME); } @Test @@ -55,10 +55,10 @@ public void testSnapshot() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = - scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); Assert.assertEquals("Should have added one file", 1L, result); @@ -79,12 +79,12 @@ public void testSnapshotWithProperties() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = scalarSql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('foo','bar'))", - catalogName, sourceName, tableName); + catalogName, SOURCE_NAME, tableName); Assert.assertEquals("Should have added one file", 1L, result); @@ -113,12 +113,12 @@ public void testSnapshotWithAlternateLocation() throws IOException { String snapshotLocation = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object[] result = sql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', location => '%s')", - catalogName, sourceName, tableName, snapshotLocation) + catalogName, SOURCE_NAME, tableName, snapshotLocation) .get(0); Assert.assertEquals("Should have added one file", 1L, result[0]); @@ -140,11 +140,11 @@ public void testDropTable() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = - scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); Assert.assertEquals("Should have added one file", 1L, result); assertEquals( @@ -157,7 +157,7 @@ public void testDropTable() throws IOException { assertEquals( "Source table should be intact", ImmutableList.of(row(1L, "a")), - sql("SELECT * FROM %s", sourceName)); + sql("SELECT * FROM %s", SOURCE_NAME)); } @Test @@ -165,8 +165,8 @@ public void testSnapshotWithConflictingProps() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = scalarSql( @@ -174,7 +174,7 @@ public void testSnapshotWithConflictingProps() throws IOException { + "source_table => '%s'," + "table => '%s'," + "properties => map('%s', 'true', 'snapshot', 'false'))", - catalogName, sourceName, tableName, TableProperties.GC_ENABLED); + catalogName, SOURCE_NAME, tableName, TableProperties.GC_ENABLED); Assert.assertEquals("Should have added one file", 1L, result); assertEquals( @@ -194,7 +194,7 @@ public void testInvalidSnapshotsCases() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); + SOURCE_NAME, location); assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) .as("Should reject calls without all required args") @@ -211,7 +211,7 @@ public void testInvalidSnapshotsCases() throws IOException { () -> sql( "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", - catalogName, sourceName)) + catalogName, SOURCE_NAME)) .as("Should reject calls with invalid map args") .isInstanceOf(AnalysisException.class) .hasMessageContaining("cannot resolve 'map"); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java index 810fef81b5bb..4e02dafb3c13 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java @@ -27,11 +27,11 @@ public class ArrowVectorAccessors { - private static final ArrowVectorAccessorFactory factory = new ArrowVectorAccessorFactory(); + private static final ArrowVectorAccessorFactory FACTORY = new ArrowVectorAccessorFactory(); static ArrowVectorAccessor getVectorAccessor(VectorHolder holder) { - return factory.getVectorAccessor(holder); + return FACTORY.getVectorAccessor(holder); } private ArrowVectorAccessors() {} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java index aca6202870ff..e3854bfeb529 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/data/parquet/vectorized/TestParquetVectorizedReads.java @@ -56,8 +56,8 @@ public class TestParquetVectorizedReads extends AvroDataTest { private static final int NUM_ROWS = 200_000; - private static final ByteBuffer fileDek = ByteBuffer.allocate(16); - private static final ByteBuffer aadPrefix = ByteBuffer.allocate(16); + private static final ByteBuffer FILE_DEK = ByteBuffer.allocate(16); + private static final ByteBuffer AAD_PREFIX = ByteBuffer.allocate(16); static final int BATCH_SIZE = 10_000; static final Function IDENTITY = record -> record; @@ -134,12 +134,12 @@ FileAppender parquetWriter(Schema schema, File testFile) thr FileAppender encryptedParquetWriter(Schema schema, File testFile) throws IOException { SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); return Parquet.write(Files.localOutput(testFile)) .schema(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .named("test") .build(); } @@ -156,12 +156,12 @@ FileAppender parquetV2Writer(Schema schema, File testFile) FileAppender encryptedParquetV2Writer(Schema schema, File testFile) throws IOException { SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); return Parquet.write(Files.localOutput(testFile)) .schema(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .named("test") .writerVersion(ParquetProperties.WriterVersion.PARQUET_2_0) .build(); @@ -200,8 +200,8 @@ void assertRecordsMatch( } if (encrypted) { - readBuilder.withFileEncryptionKey(fileDek); - readBuilder.withAADPrefix(aadPrefix); + readBuilder.withFileEncryptionKey(FILE_DEK); + readBuilder.withAADPrefix(AAD_PREFIX); } try (CloseableIterable batchReader = readBuilder.build()) { diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java index 2627ec0d2b55..875b1009c37f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -22,38 +22,38 @@ import java.util.concurrent.atomic.AtomicInteger; public class LogMessage { - private static final AtomicInteger idCounter = new AtomicInteger(0); + private static final AtomicInteger ID_COUNTER = new AtomicInteger(0); static LogMessage debug(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message); } static LogMessage debug(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message, timestamp); } static LogMessage info(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "INFO", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message); } static LogMessage info(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "INFO", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message, timestamp); } static LogMessage error(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "ERROR", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message); } static LogMessage error(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "ERROR", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message, timestamp); } static LogMessage warn(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "WARN", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message); } static LogMessage warn(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "WARN", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message, timestamp); } private int id; diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java index c9c1c29ea8fc..b6f172248ea9 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java @@ -31,16 +31,16 @@ public class ManualSource implements TableProvider, DataSourceRegister { public static final String SHORT_NAME = "manual_source"; public static final String TABLE_NAME = "TABLE_NAME"; - private static final Map tableMap = Maps.newHashMap(); + private static final Map TABLE_MAP = Maps.newHashMap(); public static void setTable(String name, Table table) { Preconditions.checkArgument( - !tableMap.containsKey(name), "Cannot set " + name + ". It is already set"); - tableMap.put(name, table); + !TABLE_MAP.containsKey(name), "Cannot set " + name + ". It is already set"); + TABLE_MAP.put(name, table); } public static void clearTables() { - tableMap.clear(); + TABLE_MAP.clear(); } @Override @@ -64,8 +64,8 @@ public org.apache.spark.sql.connector.catalog.Table getTable( Preconditions.checkArgument( properties.containsKey(TABLE_NAME), "Missing property " + TABLE_NAME); String tableName = properties.get(TABLE_NAME); - Preconditions.checkArgument(tableMap.containsKey(tableName), "Table missing " + tableName); - return tableMap.get(tableName); + Preconditions.checkArgument(TABLE_MAP.containsKey(tableName), "Table missing " + tableName); + return TABLE_MAP.get(tableName); } @Override diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java index 0c6cad7f369c..8ac666b462e0 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java @@ -35,18 +35,18 @@ public class TestSparkCatalog extends SparkSessionCatalog { - private static final Map tableMap = Maps.newHashMap(); + private static final Map TABLE_MAP = Maps.newHashMap(); public static void setTable(Identifier ident, Table table) { Preconditions.checkArgument( - !tableMap.containsKey(ident), "Cannot set " + ident + ". It is already set"); - tableMap.put(ident, table); + !TABLE_MAP.containsKey(ident), "Cannot set " + ident + ". It is already set"); + TABLE_MAP.put(ident, table); } @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - if (tableMap.containsKey(ident)) { - return tableMap.get(ident); + if (TABLE_MAP.containsKey(ident)) { + return TABLE_MAP.get(ident); } TableIdentifier tableIdentifier = Spark3Util.identifierToTableIdentifier(ident); @@ -61,6 +61,6 @@ public Table loadTable(Identifier ident) throws NoSuchTableException { } public static void clearTables() { - tableMap.clear(); + TABLE_MAP.clear(); } } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index abe70f94fe19..f94d9fde1989 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -34,9 +34,9 @@ public class TestSparkCatalogCacheExpiration extends SparkTestBaseWithCatalog { - private static final String sessionCatalogName = "spark_catalog"; - private static final String sessionCatalogImpl = SparkSessionCatalog.class.getName(); - private static final Map sessionCatalogConfig = + private static final String SESSION_CATALOG_NAME = "spark_catalog"; + private static final String SESSION_CATALOG_IMPL = SparkSessionCatalog.class.getName(); + private static final Map SESSION_CATALOG_CONFIG = ImmutableMap.of( "type", "hadoop", @@ -88,7 +88,7 @@ public static void beforeClass() { } public TestSparkCatalogCacheExpiration() { - super(sessionCatalogName, sessionCatalogImpl, sessionCatalogConfig); + super(SESSION_CATALOG_NAME, SESSION_CATALOG_IMPL, SESSION_CATALOG_CONFIG); } @Test diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java index 607f1d45ba3a..c27671311374 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java @@ -38,11 +38,11 @@ public class TestSparkCatalogHadoopOverrides extends SparkCatalogTestBase { - private static final String configToOverride = "fs.s3a.buffer.dir"; + private static final String CONFIG_TO_OVERRIDE = "fs.s3a.buffer.dir"; // prepend "hadoop." so that the test base formats SQLConf correctly // as `spark.sql.catalogs..hadoop. - private static final String hadoopPrefixedConfigToOverride = "hadoop." + configToOverride; - private static final String configOverrideValue = "/tmp-overridden"; + private static final String HADOOP_PREFIXED_CONFIG_TO_OVERRIDE = "hadoop." + CONFIG_TO_OVERRIDE; + private static final String CONFIG_OVERRIDE_VALUE = "/tmp-overridden"; @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { @@ -55,13 +55,13 @@ public static Object[][] parameters() { "hive", "default-namespace", "default", - hadoopPrefixedConfigToOverride, - configOverrideValue) + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) }, { "testhadoop", SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop", hadoopPrefixedConfigToOverride, configOverrideValue) + ImmutableMap.of("type", "hadoop", HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, CONFIG_OVERRIDE_VALUE) }, { "spark_catalog", @@ -71,8 +71,8 @@ public static Object[][] parameters() { "hive", "default-namespace", "default", - hadoopPrefixedConfigToOverride, - configOverrideValue) + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) } }; } @@ -96,10 +96,10 @@ public void dropTable() { public void testTableFromCatalogHasOverrides() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration conf = ((Configurable) table.io()).getConf(); - String actualCatalogOverride = conf.get(configToOverride, "/whammies"); + String actualCatalogOverride = conf.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, actualCatalogOverride); } @@ -107,10 +107,10 @@ public void testTableFromCatalogHasOverrides() throws Exception { public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration originalConf = ((Configurable) table.io()).getConf(); - String actualCatalogOverride = originalConf.get(configToOverride, "/whammies"); + String actualCatalogOverride = originalConf.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, actualCatalogOverride); // Now convert to SerializableTable and ensure overridden property is still present. @@ -118,19 +118,19 @@ public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception Table kryoSerializedTable = KryoHelpers.roundTripSerialize(SerializableTableWithSize.copyOf(table)); Configuration configFromKryoSerde = ((Configurable) kryoSerializedTable.io()).getConf(); - String kryoSerializedCatalogOverride = configFromKryoSerde.get(configToOverride, "/whammies"); + String kryoSerializedCatalogOverride = configFromKryoSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, kryoSerializedCatalogOverride); // Do the same for Java based serde Table javaSerializedTable = TestHelpers.roundTripSerialize(serializableTable); Configuration configFromJavaSerde = ((Configurable) javaSerializedTable.io()).getConf(); - String javaSerializedCatalogOverride = configFromJavaSerde.get(configToOverride, "/whammies"); + String javaSerializedCatalogOverride = configFromJavaSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Tables serialized with Java serialization should retain overridden hadoop configuration properties", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, javaSerializedCatalogOverride); } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index ba20a1e7bfed..942e2bbbca76 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -47,10 +47,10 @@ public class TestTimestampWithoutZone extends SparkCatalogTestBase { - private static final String newTableName = "created_table"; + private static final String NEW_TABLE_NAME = "created_table"; private final Map config; - private static final Schema schema = + private static final Schema SCHEMA = new Schema( Types.NestedField.required(1, "id", Types.LongType.get()), Types.NestedField.required(2, "ts", Types.TimestampType.withoutZone()), @@ -85,13 +85,13 @@ public TestTimestampWithoutZone( @Before public void createTables() { - validationCatalog.createTable(tableIdent, schema); + validationCatalog.createTable(tableIdent, SCHEMA); } @After public void removeTables() { validationCatalog.dropTable(tableIdent, true); - sql("DROP TABLE IF EXISTS %s", newTableName); + sql("DROP TABLE IF EXISTS %s", NEW_TABLE_NAME); } @Test @@ -132,17 +132,17 @@ public void testCreateAsSelectWithTimestampWithoutZone() { () -> { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); Assert.assertEquals( "Should have " + values.size() + " row", (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)); assertEquals( "Row data should match expected", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); }); } @@ -153,20 +153,20 @@ public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { () -> { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); Assert.assertEquals( "Should have " + values.size() + " row", (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)); assertEquals( "Data from created table should match data from base table", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); Table createdTable = - validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "ts", "tsz"); }); } @@ -185,19 +185,19 @@ public void testCreateNewTableShouldHaveTimestampWithoutZoneIcebergType() { .initialize(catalog.name(), new CaseInsensitiveStringMap(config)); sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); Assert.assertEquals( "Should have " + values.size() + " row", (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)); assertEquals( "Row data should match expected", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); Table createdTable = - validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts", "tsz"); }); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 676f7637cd09..5f995a7776c3 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -890,7 +890,7 @@ public void testEmptyImportDoesNotThrow() { assertOutput(pathResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty path", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); // Empty table based import @@ -903,7 +903,7 @@ public void testEmptyImportDoesNotThrow() { assertOutput(tableResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty table", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); } @@ -931,13 +931,13 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { assertOutput(tableResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty table", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); } - private static final List emptyQueryResult = Lists.newArrayList(); + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); - private static final StructField[] struct = { + private static final StructField[] STRUCT = { new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), new StructField("name", DataTypes.StringType, true, Metadata.empty()), new StructField("dept", DataTypes.StringType, true, Metadata.empty()), @@ -952,14 +952,14 @@ private Dataset unpartitionedDF() { RowFactory.create(2, "Jane Doe", "hr", "salary"), RowFactory.create(3, "Matt Doe", "hr", "communications"), RowFactory.create(4, "Will Doe", "facilities", "all")), - new StructType(struct)) + new StructType(STRUCT)) .repartition(1); } private Dataset singleNullRecordDF() { return spark .createDataFrame( - ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(struct)) + ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(STRUCT)) .repartition(1); } @@ -984,7 +984,7 @@ private Dataset weirdColumnNamesDF() { unpartitionedDF.col("name").as("naMe")); } - private static final StructField[] dateStruct = { + private static final StructField[] DATE_STRUCT = { new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), new StructField("name", DataTypes.StringType, true, Metadata.empty()), new StructField("ts", DataTypes.DateType, true, Metadata.empty()), @@ -1003,7 +1003,7 @@ private Dataset dateDF() { RowFactory.create(2, "Jane Doe", toDate("2021-01-01"), "01"), RowFactory.create(3, "Matt Doe", toDate("2021-01-02"), "02"), RowFactory.create(4, "Will Doe", toDate("2021-01-02"), "02")), - new StructType(dateStruct)) + new StructType(DATE_STRUCT)) .repartition(2); } diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index bd4e1fb2f8f4..905cb8fe07fb 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -34,7 +34,7 @@ import org.junit.rules.TemporaryFolder; public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { - private static final String sourceName = "spark_catalog.default.source"; + private static final String SOURCE_NAME = "spark_catalog.default.source"; // Currently we can only Snapshot only out of the Spark Session Catalog public TestSnapshotTableProcedure( @@ -47,7 +47,7 @@ public TestSnapshotTableProcedure( @After public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %s PURGE", sourceName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); } @Test @@ -55,10 +55,10 @@ public void testSnapshot() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = - scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); Assert.assertEquals("Should have added one file", 1L, result); @@ -79,12 +79,12 @@ public void testSnapshotWithProperties() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = scalarSql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('foo','bar'))", - catalogName, sourceName, tableName); + catalogName, SOURCE_NAME, tableName); Assert.assertEquals("Should have added one file", 1L, result); @@ -113,12 +113,12 @@ public void testSnapshotWithAlternateLocation() throws IOException { String snapshotLocation = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object[] result = sql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', location => '%s')", - catalogName, sourceName, tableName, snapshotLocation) + catalogName, SOURCE_NAME, tableName, snapshotLocation) .get(0); Assert.assertEquals("Should have added one file", 1L, result[0]); @@ -140,11 +140,11 @@ public void testDropTable() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = - scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); Assert.assertEquals("Should have added one file", 1L, result); assertEquals( @@ -157,7 +157,7 @@ public void testDropTable() throws IOException { assertEquals( "Source table should be intact", ImmutableList.of(row(1L, "a")), - sql("SELECT * FROM %s", sourceName)); + sql("SELECT * FROM %s", SOURCE_NAME)); } @Test @@ -165,8 +165,8 @@ public void testSnapshotWithConflictingProps() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = scalarSql( @@ -174,7 +174,7 @@ public void testSnapshotWithConflictingProps() throws IOException { + "source_table => '%s'," + "table => '%s'," + "properties => map('%s', 'true', 'snapshot', 'false'))", - catalogName, sourceName, tableName, TableProperties.GC_ENABLED); + catalogName, SOURCE_NAME, tableName, TableProperties.GC_ENABLED); Assert.assertEquals("Should have added one file", 1L, result); assertEquals( @@ -194,7 +194,7 @@ public void testInvalidSnapshotsCases() throws IOException { String location = temp.newFolder().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); + SOURCE_NAME, location); assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) .isInstanceOf(AnalysisException.class) @@ -209,7 +209,7 @@ public void testInvalidSnapshotsCases() throws IOException { () -> sql( "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", - catalogName, sourceName)) + catalogName, SOURCE_NAME)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( "The `map` requires 2n (n > 0) parameters but the actual number is 3"); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java index 810fef81b5bb..4e02dafb3c13 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java @@ -27,11 +27,11 @@ public class ArrowVectorAccessors { - private static final ArrowVectorAccessorFactory factory = new ArrowVectorAccessorFactory(); + private static final ArrowVectorAccessorFactory FACTORY = new ArrowVectorAccessorFactory(); static ArrowVectorAccessor getVectorAccessor(VectorHolder holder) { - return factory.getVectorAccessor(holder); + return FACTORY.getVectorAccessor(holder); } private ArrowVectorAccessors() {} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java index 2627ec0d2b55..875b1009c37f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -22,38 +22,38 @@ import java.util.concurrent.atomic.AtomicInteger; public class LogMessage { - private static final AtomicInteger idCounter = new AtomicInteger(0); + private static final AtomicInteger ID_COUNTER = new AtomicInteger(0); static LogMessage debug(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message); } static LogMessage debug(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message, timestamp); } static LogMessage info(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "INFO", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message); } static LogMessage info(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "INFO", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message, timestamp); } static LogMessage error(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "ERROR", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message); } static LogMessage error(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "ERROR", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message, timestamp); } static LogMessage warn(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "WARN", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message); } static LogMessage warn(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "WARN", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message, timestamp); } private int id; diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java index c9c1c29ea8fc..b6f172248ea9 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java @@ -31,16 +31,16 @@ public class ManualSource implements TableProvider, DataSourceRegister { public static final String SHORT_NAME = "manual_source"; public static final String TABLE_NAME = "TABLE_NAME"; - private static final Map tableMap = Maps.newHashMap(); + private static final Map TABLE_MAP = Maps.newHashMap(); public static void setTable(String name, Table table) { Preconditions.checkArgument( - !tableMap.containsKey(name), "Cannot set " + name + ". It is already set"); - tableMap.put(name, table); + !TABLE_MAP.containsKey(name), "Cannot set " + name + ". It is already set"); + TABLE_MAP.put(name, table); } public static void clearTables() { - tableMap.clear(); + TABLE_MAP.clear(); } @Override @@ -64,8 +64,8 @@ public org.apache.spark.sql.connector.catalog.Table getTable( Preconditions.checkArgument( properties.containsKey(TABLE_NAME), "Missing property " + TABLE_NAME); String tableName = properties.get(TABLE_NAME); - Preconditions.checkArgument(tableMap.containsKey(tableName), "Table missing " + tableName); - return tableMap.get(tableName); + Preconditions.checkArgument(TABLE_MAP.containsKey(tableName), "Table missing " + tableName); + return TABLE_MAP.get(tableName); } @Override 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 6b188cd1b120..15325fe03c4e 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 @@ -18,7 +18,6 @@ */ package org.apache.iceberg.spark.source; -import static org.apache.iceberg.FileFormat.PARQUET; import static org.apache.iceberg.RowLevelOperationMode.MERGE_ON_READ; import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; @@ -82,7 +81,7 @@ public class TestCompressionSettings extends SparkCatalogTestBase { private static final Configuration CONF = new Configuration(); - private static final String tableName = "testWriteData"; + private static final String TABLE_NAME = "testWriteData"; private static SparkSession spark = null; @@ -109,7 +108,7 @@ public static void startSpark() { @Parameterized.AfterParam public static void clearSourceCache() { - spark.sql(String.format("DROP TABLE IF EXISTS %s", tableName)); + spark.sql(String.format("DROP TABLE IF EXISTS %s", TABLE_NAME)); } @AfterClass @@ -130,7 +129,7 @@ public TestCompressionSettings(String format, ImmutableMap properties) { @Test public void testWriteDataWithDifferentSetting() throws Exception { - sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("CREATE TABLE %s (id int, data string) USING iceberg", TABLE_NAME); Map tableProperties = Maps.newHashMap(); tableProperties.put(PARQUET_COMPRESSION, "gzip"); tableProperties.put(AVRO_COMPRESSION, "gzip"); @@ -140,14 +139,14 @@ public void testWriteDataWithDifferentSetting() throws Exception { tableProperties.put(DELETE_ORC_COMPRESSION, "zlib"); tableProperties.put(DELETE_MODE, MERGE_ON_READ.modeName()); tableProperties.put(FORMAT_VERSION, "2"); - sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", tableName, DEFAULT_FILE_FORMAT, format); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", TABLE_NAME, DEFAULT_FILE_FORMAT, format); sql( "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, DELETE_DEFAULT_FILE_FORMAT, format); + TABLE_NAME, DELETE_DEFAULT_FILE_FORMAT, format); for (Map.Entry entry : tableProperties.entrySet()) { sql( "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, entry.getKey(), entry.getValue()); + TABLE_NAME, entry.getKey(), entry.getValue()); } List expectedOrigin = Lists.newArrayList(); @@ -162,10 +161,10 @@ public void testWriteDataWithDifferentSetting() throws Exception { } df.select("id", "data") - .writeTo(tableName) + .writeTo(TABLE_NAME) .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .append(); - Table table = catalog.loadTable(TableIdentifier.of("default", tableName)); + Table table = catalog.loadTable(TableIdentifier.of("default", TABLE_NAME)); List manifestFiles = table.currentSnapshot().dataManifests(table.io()); try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); @@ -174,7 +173,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } - sql("DELETE from %s where id < 100", tableName); + sql("DELETE from %s where id < 100", TABLE_NAME); table.refresh(); List deleteManifestFiles = table.currentSnapshot().deleteManifests(table.io()); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java index 02a74e5803be..1f266380cdc1 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java @@ -37,18 +37,18 @@ public class TestSparkCatalog< T extends TableCatalog & FunctionCatalog & SupportsNamespaces & ViewCatalog> extends SparkSessionCatalog { - private static final Map tableMap = Maps.newHashMap(); + private static final Map TABLE_MAP = Maps.newHashMap(); public static void setTable(Identifier ident, Table table) { Preconditions.checkArgument( - !tableMap.containsKey(ident), "Cannot set " + ident + ". It is already set"); - tableMap.put(ident, table); + !TABLE_MAP.containsKey(ident), "Cannot set " + ident + ". It is already set"); + TABLE_MAP.put(ident, table); } @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - if (tableMap.containsKey(ident)) { - return tableMap.get(ident); + if (TABLE_MAP.containsKey(ident)) { + return TABLE_MAP.get(ident); } TableIdentifier tableIdentifier = Spark3Util.identifierToTableIdentifier(ident); @@ -63,6 +63,6 @@ public Table loadTable(Identifier ident) throws NoSuchTableException { } public static void clearTables() { - tableMap.clear(); + TABLE_MAP.clear(); } } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index abe70f94fe19..f94d9fde1989 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -34,9 +34,9 @@ public class TestSparkCatalogCacheExpiration extends SparkTestBaseWithCatalog { - private static final String sessionCatalogName = "spark_catalog"; - private static final String sessionCatalogImpl = SparkSessionCatalog.class.getName(); - private static final Map sessionCatalogConfig = + private static final String SESSION_CATALOG_NAME = "spark_catalog"; + private static final String SESSION_CATALOG_IMPL = SparkSessionCatalog.class.getName(); + private static final Map SESSION_CATALOG_CONFIG = ImmutableMap.of( "type", "hadoop", @@ -88,7 +88,7 @@ public static void beforeClass() { } public TestSparkCatalogCacheExpiration() { - super(sessionCatalogName, sessionCatalogImpl, sessionCatalogConfig); + super(SESSION_CATALOG_NAME, SESSION_CATALOG_IMPL, SESSION_CATALOG_CONFIG); } @Test diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java index 607f1d45ba3a..c27671311374 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java @@ -38,11 +38,11 @@ public class TestSparkCatalogHadoopOverrides extends SparkCatalogTestBase { - private static final String configToOverride = "fs.s3a.buffer.dir"; + private static final String CONFIG_TO_OVERRIDE = "fs.s3a.buffer.dir"; // prepend "hadoop." so that the test base formats SQLConf correctly // as `spark.sql.catalogs..hadoop. - private static final String hadoopPrefixedConfigToOverride = "hadoop." + configToOverride; - private static final String configOverrideValue = "/tmp-overridden"; + private static final String HADOOP_PREFIXED_CONFIG_TO_OVERRIDE = "hadoop." + CONFIG_TO_OVERRIDE; + private static final String CONFIG_OVERRIDE_VALUE = "/tmp-overridden"; @Parameterized.Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { @@ -55,13 +55,13 @@ public static Object[][] parameters() { "hive", "default-namespace", "default", - hadoopPrefixedConfigToOverride, - configOverrideValue) + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) }, { "testhadoop", SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop", hadoopPrefixedConfigToOverride, configOverrideValue) + ImmutableMap.of("type", "hadoop", HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, CONFIG_OVERRIDE_VALUE) }, { "spark_catalog", @@ -71,8 +71,8 @@ public static Object[][] parameters() { "hive", "default-namespace", "default", - hadoopPrefixedConfigToOverride, - configOverrideValue) + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) } }; } @@ -96,10 +96,10 @@ public void dropTable() { public void testTableFromCatalogHasOverrides() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration conf = ((Configurable) table.io()).getConf(); - String actualCatalogOverride = conf.get(configToOverride, "/whammies"); + String actualCatalogOverride = conf.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, actualCatalogOverride); } @@ -107,10 +107,10 @@ public void testTableFromCatalogHasOverrides() throws Exception { public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration originalConf = ((Configurable) table.io()).getConf(); - String actualCatalogOverride = originalConf.get(configToOverride, "/whammies"); + String actualCatalogOverride = originalConf.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Iceberg tables from spark should have the overridden hadoop configurations from the spark config", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, actualCatalogOverride); // Now convert to SerializableTable and ensure overridden property is still present. @@ -118,19 +118,19 @@ public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception Table kryoSerializedTable = KryoHelpers.roundTripSerialize(SerializableTableWithSize.copyOf(table)); Configuration configFromKryoSerde = ((Configurable) kryoSerializedTable.io()).getConf(); - String kryoSerializedCatalogOverride = configFromKryoSerde.get(configToOverride, "/whammies"); + String kryoSerializedCatalogOverride = configFromKryoSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, kryoSerializedCatalogOverride); // Do the same for Java based serde Table javaSerializedTable = TestHelpers.roundTripSerialize(serializableTable); Configuration configFromJavaSerde = ((Configurable) javaSerializedTable.io()).getConf(); - String javaSerializedCatalogOverride = configFromJavaSerde.get(configToOverride, "/whammies"); + String javaSerializedCatalogOverride = configFromJavaSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); Assert.assertEquals( "Tables serialized with Java serialization should retain overridden hadoop configuration properties", - configOverrideValue, + CONFIG_OVERRIDE_VALUE, javaSerializedCatalogOverride); } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index 196ab119866e..e34d253dd39e 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -49,10 +49,10 @@ public class TestTimestampWithoutZone extends SparkCatalogTestBase { - private static final String newTableName = "created_table"; + private static final String NEW_TABLE_NAME = "created_table"; private final Map config; - private static final Schema schema = + private static final Schema SCHEMA = new Schema( Types.NestedField.required(1, "id", Types.LongType.get()), Types.NestedField.required(2, "ts", Types.TimestampType.withoutZone()), @@ -87,13 +87,13 @@ public TestTimestampWithoutZone( @Before public void createTables() { - validationCatalog.createTable(tableIdent, schema); + validationCatalog.createTable(tableIdent, SCHEMA); } @After public void removeTables() { validationCatalog.dropTable(tableIdent, true); - sql("DROP TABLE IF EXISTS %s", newTableName); + sql("DROP TABLE IF EXISTS %s", NEW_TABLE_NAME); } @Test @@ -158,7 +158,7 @@ public void testWriteWithDeprecatedTimezoneProperty() { () -> { sql( "CREATE OR REPLACE TABLE %s USING ICEBERG AS SELECT * FROM %s", - newTableName, tableName); + NEW_TABLE_NAME, tableName); }) .isInstanceOf(UnsupportedOperationException.class) .hasMessage( @@ -176,7 +176,7 @@ public void testWriteWithDeprecatedTimezonePropertyReadOption() { spark .read() .table(tableName) - .writeTo(newTableName) + .writeTo(NEW_TABLE_NAME) .option(SparkWriteOptions.HANDLE_TIMESTAMP_WITHOUT_TIMEZONE, "true") .using("iceberg") .createOrReplace(); @@ -245,36 +245,36 @@ public void testAppendTimestampWithZone() { public void testCreateAsSelectWithTimestampWithoutZone() { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); Assert.assertEquals( "Should have " + values.size() + " row", (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)); assertEquals( "Row data should match expected", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); } @Test public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); Assert.assertEquals( "Should have " + values.size() + " row", (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)); assertEquals( "Data from created table should match data from base table", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); - Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts"); assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); } @@ -288,18 +288,18 @@ public void testCreateNewTableShouldHaveTimestampWithoutZoneIcebergType() { .initialize(catalog.name(), new CaseInsensitiveStringMap(config)); sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); Assert.assertEquals( "Should have " + values.size() + " row", (long) values.size(), - scalarSql("SELECT count(*) FROM %s", newTableName)); + scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)); assertEquals( "Row data should match expected", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); - Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts"); assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); } 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 ac2d39eaa485..920c2f55eaaf 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 @@ -881,7 +881,7 @@ public void testEmptyImportDoesNotThrow() { assertOutput(pathResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty path", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); // Empty table based import @@ -894,7 +894,7 @@ public void testEmptyImportDoesNotThrow() { assertOutput(tableResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty table", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); } @@ -922,7 +922,7 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { assertOutput(tableResult, 0L, 0L); assertEquals( "Iceberg table contains no added data when importing from an empty table", - emptyQueryResult, + EMPTY_QUERY_RESULT, sql("SELECT * FROM %s ORDER BY id", tableName)); } @@ -948,9 +948,9 @@ public void testAddFilesWithParallelism() { sql("SELECT * FROM %s ORDER BY id", tableName)); } - private static final List emptyQueryResult = Lists.newArrayList(); + private static final List EMPTY_QUERY_RESULT = Lists.newArrayList(); - private static final StructField[] struct = { + private static final StructField[] STRUCT = { new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), new StructField("name", DataTypes.StringType, true, Metadata.empty()), new StructField("dept", DataTypes.StringType, true, Metadata.empty()), @@ -965,14 +965,14 @@ private Dataset unpartitionedDF() { RowFactory.create(2, "Jane Doe", "hr", "salary"), RowFactory.create(3, "Matt Doe", "hr", "communications"), RowFactory.create(4, "Will Doe", "facilities", "all")), - new StructType(struct)) + new StructType(STRUCT)) .repartition(1); } private Dataset singleNullRecordDF() { return spark .createDataFrame( - ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(struct)) + ImmutableList.of(RowFactory.create(null, null, null, null)), new StructType(STRUCT)) .repartition(1); } @@ -997,7 +997,7 @@ private Dataset weirdColumnNamesDF() { unpartitionedDF.col("name").as("naMe")); } - private static final StructField[] dateStruct = { + private static final StructField[] DATE_STRUCT = { new StructField("id", DataTypes.IntegerType, true, Metadata.empty()), new StructField("name", DataTypes.StringType, true, Metadata.empty()), new StructField("ts", DataTypes.DateType, true, Metadata.empty()), @@ -1016,7 +1016,7 @@ private Dataset dateDF() { RowFactory.create(2, "Jane Doe", toDate("2021-01-01"), "01"), RowFactory.create(3, "Matt Doe", toDate("2021-01-02"), "02"), RowFactory.create(4, "Will Doe", toDate("2021-01-02"), "02")), - new StructType(dateStruct)) + new StructType(DATE_STRUCT)) .repartition(2); } 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 cb184043490c..a4d0a2dfd3d0 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 @@ -37,13 +37,13 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableProcedure extends ExtensionsTestBase { - private static final String sourceName = "spark_catalog.default.source"; + private static final String SOURCE_NAME = "spark_catalog.default.source"; // Currently we can only Snapshot only out of the Spark Session Catalog @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %s PURGE", sourceName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); } @TestTemplate @@ -51,10 +51,10 @@ public void testSnapshot() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = - scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); assertThat(result).as("Should have added one file").isEqualTo(1L); @@ -77,12 +77,12 @@ public void testSnapshotWithProperties() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = scalarSql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', properties => map('foo','bar'))", - catalogName, sourceName, tableName); + catalogName, SOURCE_NAME, tableName); assertThat(result).as("Should have added one file").isEqualTo(1L); @@ -113,12 +113,12 @@ public void testSnapshotWithAlternateLocation() throws IOException { String snapshotLocation = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object[] result = sql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', location => '%s')", - catalogName, sourceName, tableName, snapshotLocation) + catalogName, SOURCE_NAME, tableName, snapshotLocation) .get(0); assertThat(result[0]).as("Should have added one file").isEqualTo(1L); @@ -141,11 +141,11 @@ public void testDropTable() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = - scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, sourceName, tableName); + scalarSql("CALL %s.system.snapshot('%s', '%s')", catalogName, SOURCE_NAME, tableName); assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( @@ -158,7 +158,7 @@ public void testDropTable() throws IOException { assertEquals( "Source table should be intact", ImmutableList.of(row(1L, "a")), - sql("SELECT * FROM %s", sourceName)); + sql("SELECT * FROM %s", SOURCE_NAME)); } @TestTemplate @@ -166,8 +166,8 @@ public void testSnapshotWithConflictingProps() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); Object result = scalarSql( @@ -175,7 +175,7 @@ public void testSnapshotWithConflictingProps() throws IOException { + "source_table => '%s'," + "table => '%s'," + "properties => map('%s', 'true', 'snapshot', 'false'))", - catalogName, sourceName, tableName, TableProperties.GC_ENABLED); + catalogName, SOURCE_NAME, tableName, TableProperties.GC_ENABLED); assertThat(result).as("Should have added one file").isEqualTo(1L); assertEquals( @@ -196,7 +196,7 @@ public void testInvalidSnapshotsCases() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); + SOURCE_NAME, location); assertThatThrownBy(() -> sql("CALL %s.system.snapshot('foo')", catalogName)) .isInstanceOf(AnalysisException.class) @@ -211,7 +211,7 @@ public void testInvalidSnapshotsCases() throws IOException { () -> sql( "CALL %s.system.snapshot('%s', 'fable', 'loc', map(2, 1, 1))", - catalogName, sourceName)) + catalogName, SOURCE_NAME)) .isInstanceOf(AnalysisException.class) .hasMessageContaining( "The `map` requires 2n (n > 0) parameters but the actual number is 3"); @@ -230,14 +230,14 @@ public void testSnapshotWithParallelism() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); - sql("INSERT INTO TABLE %s VALUES (2, 'b')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); List result = sql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", - catalogName, sourceName, tableName, 2); + catalogName, SOURCE_NAME, tableName, 2); assertEquals("Procedure output must match", ImmutableList.of(row(2L)), result); assertEquals( "Should have expected rows", @@ -250,15 +250,15 @@ public void testSnapshotWithInvalidParallelism() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); - sql("INSERT INTO TABLE %s VALUES (2, 'b')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); assertThatThrownBy( () -> sql( "CALL %s.system.snapshot(source_table => '%s', table => '%s', parallelism => %d)", - catalogName, sourceName, tableName, -1)) + catalogName, SOURCE_NAME, tableName, -1)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Parallelism should be larger than 0"); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java index 810fef81b5bb..4e02dafb3c13 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/data/vectorized/ArrowVectorAccessors.java @@ -27,11 +27,11 @@ public class ArrowVectorAccessors { - private static final ArrowVectorAccessorFactory factory = new ArrowVectorAccessorFactory(); + private static final ArrowVectorAccessorFactory FACTORY = new ArrowVectorAccessorFactory(); static ArrowVectorAccessor getVectorAccessor(VectorHolder holder) { - return factory.getVectorAccessor(holder); + return FACTORY.getVectorAccessor(holder); } private ArrowVectorAccessors() {} diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java index 8f4c09356d13..186042283bdb 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestSparkCatalogOperations.java @@ -38,7 +38,8 @@ import org.junit.jupiter.api.TestTemplate; public class TestSparkCatalogOperations extends CatalogTestBase { - private static final boolean useNullableQuerySchema = ThreadLocalRandom.current().nextBoolean(); + private static final boolean USE_NULLABLE_QUERY_SCHEMA = + ThreadLocalRandom.current().nextBoolean(); @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") protected static Object[][] parameters() { @@ -49,7 +50,7 @@ protected static Object[][] parameters() { ImmutableMap.of( "type", "hive", "default-namespace", "default", - "use-nullable-query-schema", Boolean.toString(useNullableQuerySchema)) + "use-nullable-query-schema", Boolean.toString(USE_NULLABLE_QUERY_SCHEMA)) }, { SparkCatalogConfig.HADOOP.catalogName(), @@ -60,7 +61,7 @@ protected static Object[][] parameters() { "cache-enabled", "false", "use-nullable-query-schema", - Boolean.toString(useNullableQuerySchema)) + Boolean.toString(USE_NULLABLE_QUERY_SCHEMA)) }, { SparkCatalogConfig.SPARK.catalogName(), @@ -75,7 +76,7 @@ protected static Object[][] parameters() { "cache-enabled", "false", // Spark will delete tables using v1, leaving the cache out of sync "use-nullable-query-schema", - Boolean.toString(useNullableQuerySchema)), + Boolean.toString(USE_NULLABLE_QUERY_SCHEMA)), } }; } @@ -146,7 +147,7 @@ public void testCTASUseNullableQuerySchema() { Schema expectedSchema = new Schema( - useNullableQuerySchema + USE_NULLABLE_QUERY_SCHEMA ? Types.NestedField.optional(1, "id", Types.LongType.get()) : Types.NestedField.required(1, "id", Types.LongType.get()), Types.NestedField.optional(2, "data", Types.StringType.get())); @@ -172,7 +173,7 @@ public void testRTASUseNullableQuerySchema() { Schema expectedSchema = new Schema( - useNullableQuerySchema + USE_NULLABLE_QUERY_SCHEMA ? Types.NestedField.optional(1, "id", Types.LongType.get()) : Types.NestedField.required(1, "id", Types.LongType.get()), Types.NestedField.optional(2, "data", Types.StringType.get())); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java index 3b6869c397a5..d9c42a07b853 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java @@ -32,12 +32,12 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableAction extends CatalogTestBase { - private static final String sourceName = "spark_catalog.default.source"; + private static final String SOURCE_NAME = "spark_catalog.default.source"; @AfterEach public void removeTables() { sql("DROP TABLE IF EXISTS %s", tableName); - sql("DROP TABLE IF EXISTS %s PURGE", sourceName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); } @TestTemplate @@ -45,13 +45,13 @@ public void testSnapshotWithParallelTasks() throws IOException { String location = Files.createTempDirectory(temp, "junit").toFile().toString(); sql( "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", - sourceName, location); - sql("INSERT INTO TABLE %s VALUES (1, 'a')", sourceName); - sql("INSERT INTO TABLE %s VALUES (2, 'b')", sourceName); + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); SparkActions.get() - .snapshotTable(sourceName) + .snapshotTable(SOURCE_NAME) .as(tableName) .executeWith( Executors.newFixedThreadPool( diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java index 2627ec0d2b55..875b1009c37f 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/LogMessage.java @@ -22,38 +22,38 @@ import java.util.concurrent.atomic.AtomicInteger; public class LogMessage { - private static final AtomicInteger idCounter = new AtomicInteger(0); + private static final AtomicInteger ID_COUNTER = new AtomicInteger(0); static LogMessage debug(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message); } static LogMessage debug(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "DEBUG", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "DEBUG", message, timestamp); } static LogMessage info(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "INFO", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message); } static LogMessage info(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "INFO", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "INFO", message, timestamp); } static LogMessage error(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "ERROR", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message); } static LogMessage error(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "ERROR", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "ERROR", message, timestamp); } static LogMessage warn(String date, String message) { - return new LogMessage(idCounter.getAndIncrement(), date, "WARN", message); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message); } static LogMessage warn(String date, String message, Instant timestamp) { - return new LogMessage(idCounter.getAndIncrement(), date, "WARN", message, timestamp); + return new LogMessage(ID_COUNTER.getAndIncrement(), date, "WARN", message, timestamp); } private int id; diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java index c9c1c29ea8fc..b6f172248ea9 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/ManualSource.java @@ -31,16 +31,16 @@ public class ManualSource implements TableProvider, DataSourceRegister { public static final String SHORT_NAME = "manual_source"; public static final String TABLE_NAME = "TABLE_NAME"; - private static final Map tableMap = Maps.newHashMap(); + private static final Map TABLE_MAP = Maps.newHashMap(); public static void setTable(String name, Table table) { Preconditions.checkArgument( - !tableMap.containsKey(name), "Cannot set " + name + ". It is already set"); - tableMap.put(name, table); + !TABLE_MAP.containsKey(name), "Cannot set " + name + ". It is already set"); + TABLE_MAP.put(name, table); } public static void clearTables() { - tableMap.clear(); + TABLE_MAP.clear(); } @Override @@ -64,8 +64,8 @@ public org.apache.spark.sql.connector.catalog.Table getTable( Preconditions.checkArgument( properties.containsKey(TABLE_NAME), "Missing property " + TABLE_NAME); String tableName = properties.get(TABLE_NAME); - Preconditions.checkArgument(tableMap.containsKey(tableName), "Table missing " + tableName); - return tableMap.get(tableName); + Preconditions.checkArgument(TABLE_MAP.containsKey(tableName), "Table missing " + tableName); + return TABLE_MAP.get(tableName); } @Override 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 44885eb84551..f569446f772b 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 @@ -86,7 +86,7 @@ public class TestCompressionSettings extends CatalogTestBase { private static final Configuration CONF = new Configuration(); - private static final String tableName = "testWriteData"; + private static final String TABLE_NAME = "testWriteData"; private static SparkSession spark = null; @@ -148,7 +148,7 @@ public static void startSpark() { @AfterEach public void afterEach() { - spark.sql(String.format("DROP TABLE IF EXISTS %s", tableName)); + spark.sql(String.format("DROP TABLE IF EXISTS %s", TABLE_NAME)); } @AfterAll @@ -160,7 +160,7 @@ public static void stopSpark() { @TestTemplate public void testWriteDataWithDifferentSetting() throws Exception { - sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("CREATE TABLE %s (id int, data string) USING iceberg", TABLE_NAME); Map tableProperties = Maps.newHashMap(); tableProperties.put(PARQUET_COMPRESSION, "gzip"); tableProperties.put(AVRO_COMPRESSION, "gzip"); @@ -170,14 +170,14 @@ public void testWriteDataWithDifferentSetting() throws Exception { tableProperties.put(DELETE_ORC_COMPRESSION, "zlib"); tableProperties.put(DELETE_MODE, MERGE_ON_READ.modeName()); tableProperties.put(FORMAT_VERSION, "2"); - sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", tableName, DEFAULT_FILE_FORMAT, format); + sql("ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", TABLE_NAME, DEFAULT_FILE_FORMAT, format); sql( "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, DELETE_DEFAULT_FILE_FORMAT, format); + TABLE_NAME, DELETE_DEFAULT_FILE_FORMAT, format); for (Map.Entry entry : tableProperties.entrySet()) { sql( "ALTER TABLE %s SET TBLPROPERTIES ('%s' '%s')", - tableName, entry.getKey(), entry.getValue()); + TABLE_NAME, entry.getKey(), entry.getValue()); } List expectedOrigin = Lists.newArrayList(); @@ -192,10 +192,10 @@ public void testWriteDataWithDifferentSetting() throws Exception { } df.select("id", "data") - .writeTo(tableName) + .writeTo(TABLE_NAME) .option(SparkWriteOptions.WRITE_FORMAT, format.toString()) .append(); - Table table = catalog.loadTable(TableIdentifier.of("default", tableName)); + Table table = catalog.loadTable(TableIdentifier.of("default", TABLE_NAME)); List manifestFiles = table.currentSnapshot().dataManifests(table.io()); try (ManifestReader reader = ManifestFiles.read(manifestFiles.get(0), table.io())) { DataFile file = reader.iterator().next(); @@ -204,7 +204,7 @@ public void testWriteDataWithDifferentSetting() throws Exception { .isEqualToIgnoringCase(properties.get(COMPRESSION_CODEC)); } - sql("DELETE from %s where id < 100", tableName); + sql("DELETE from %s where id < 100", TABLE_NAME); table.refresh(); List deleteManifestFiles = table.currentSnapshot().deleteManifests(table.io()); diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java index 02a74e5803be..1f266380cdc1 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalog.java @@ -37,18 +37,18 @@ public class TestSparkCatalog< T extends TableCatalog & FunctionCatalog & SupportsNamespaces & ViewCatalog> extends SparkSessionCatalog { - private static final Map tableMap = Maps.newHashMap(); + private static final Map TABLE_MAP = Maps.newHashMap(); public static void setTable(Identifier ident, Table table) { Preconditions.checkArgument( - !tableMap.containsKey(ident), "Cannot set " + ident + ". It is already set"); - tableMap.put(ident, table); + !TABLE_MAP.containsKey(ident), "Cannot set " + ident + ". It is already set"); + TABLE_MAP.put(ident, table); } @Override public Table loadTable(Identifier ident) throws NoSuchTableException { - if (tableMap.containsKey(ident)) { - return tableMap.get(ident); + if (TABLE_MAP.containsKey(ident)) { + return TABLE_MAP.get(ident); } TableIdentifier tableIdentifier = Spark3Util.identifierToTableIdentifier(ident); @@ -63,6 +63,6 @@ public Table loadTable(Identifier ident) throws NoSuchTableException { } public static void clearTables() { - tableMap.clear(); + TABLE_MAP.clear(); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java index 99eeb930af28..2a9bbca40f94 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogCacheExpiration.java @@ -35,7 +35,7 @@ public class TestSparkCatalogCacheExpiration extends TestBaseWithCatalog { - private static final Map sessionCatalogConfig = + private static final Map SESSION_CATALOG_CONFIG = ImmutableMap.of( "type", "hadoop", @@ -49,7 +49,7 @@ public class TestSparkCatalogCacheExpiration extends TestBaseWithCatalog { @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { return new Object[][] { - {"spark_catalog", SparkSessionCatalog.class.getName(), sessionCatalogConfig}, + {"spark_catalog", SparkSessionCatalog.class.getName(), SESSION_CATALOG_CONFIG}, }; } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java index 7ff507ed0694..c031f2991fed 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkCatalogHadoopOverrides.java @@ -38,11 +38,11 @@ public class TestSparkCatalogHadoopOverrides extends CatalogTestBase { - private static final String configToOverride = "fs.s3a.buffer.dir"; + private static final String CONFIG_TO_OVERRIDE = "fs.s3a.buffer.dir"; // prepend "hadoop." so that the test base formats SQLConf correctly // as `spark.sql.catalogs..hadoop. - private static final String hadoopPrefixedConfigToOverride = "hadoop." + configToOverride; - private static final String configOverrideValue = "/tmp-overridden"; + private static final String HADOOP_PREFIXED_CONFIG_TO_OVERRIDE = "hadoop." + CONFIG_TO_OVERRIDE; + private static final String CONFIG_OVERRIDE_VALUE = "/tmp-overridden"; @Parameters(name = "catalogName = {0}, implementation = {1}, config = {2}") public static Object[][] parameters() { @@ -55,13 +55,13 @@ public static Object[][] parameters() { "hive", "default-namespace", "default", - hadoopPrefixedConfigToOverride, - configOverrideValue) + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) }, { "testhadoop", SparkCatalog.class.getName(), - ImmutableMap.of("type", "hadoop", hadoopPrefixedConfigToOverride, configOverrideValue) + ImmutableMap.of("type", "hadoop", HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, CONFIG_OVERRIDE_VALUE) }, { "spark_catalog", @@ -71,8 +71,8 @@ public static Object[][] parameters() { "hive", "default-namespace", "default", - hadoopPrefixedConfigToOverride, - configOverrideValue) + HADOOP_PREFIXED_CONFIG_TO_OVERRIDE, + CONFIG_OVERRIDE_VALUE) } }; } @@ -91,42 +91,42 @@ public void dropTable() { public void testTableFromCatalogHasOverrides() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration conf = ((Configurable) table.io()).getConf(); - String actualCatalogOverride = conf.get(configToOverride, "/whammies"); + String actualCatalogOverride = conf.get(CONFIG_TO_OVERRIDE, "/whammies"); assertThat(actualCatalogOverride) .as( "Iceberg tables from spark should have the overridden hadoop configurations from the spark config") - .isEqualTo(configOverrideValue); + .isEqualTo(CONFIG_OVERRIDE_VALUE); } @TestTemplate public void ensureRoundTripSerializedTableRetainsHadoopConfig() throws Exception { Table table = getIcebergTableFromSparkCatalog(); Configuration originalConf = ((Configurable) table.io()).getConf(); - String actualCatalogOverride = originalConf.get(configToOverride, "/whammies"); + String actualCatalogOverride = originalConf.get(CONFIG_TO_OVERRIDE, "/whammies"); assertThat(actualCatalogOverride) .as( "Iceberg tables from spark should have the overridden hadoop configurations from the spark config") - .isEqualTo(configOverrideValue); + .isEqualTo(CONFIG_OVERRIDE_VALUE); // Now convert to SerializableTable and ensure overridden property is still present. Table serializableTable = SerializableTableWithSize.copyOf(table); Table kryoSerializedTable = KryoHelpers.roundTripSerialize(SerializableTableWithSize.copyOf(table)); Configuration configFromKryoSerde = ((Configurable) kryoSerializedTable.io()).getConf(); - String kryoSerializedCatalogOverride = configFromKryoSerde.get(configToOverride, "/whammies"); + String kryoSerializedCatalogOverride = configFromKryoSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); assertThat(kryoSerializedCatalogOverride) .as( "Tables serialized with Kryo serialization should retain overridden hadoop configuration properties") - .isEqualTo(configOverrideValue); + .isEqualTo(CONFIG_OVERRIDE_VALUE); // Do the same for Java based serde Table javaSerializedTable = TestHelpers.roundTripSerialize(serializableTable); Configuration configFromJavaSerde = ((Configurable) javaSerializedTable.io()).getConf(); - String javaSerializedCatalogOverride = configFromJavaSerde.get(configToOverride, "/whammies"); + String javaSerializedCatalogOverride = configFromJavaSerde.get(CONFIG_TO_OVERRIDE, "/whammies"); assertThat(javaSerializedCatalogOverride) .as( "Tables serialized with Java serialization should retain overridden hadoop configuration properties") - .isEqualTo(configOverrideValue); + .isEqualTo(CONFIG_OVERRIDE_VALUE); } @SuppressWarnings("ThrowSpecificity") diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java index 636e789e16fa..44d895dd44c5 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/sql/TestTimestampWithoutZone.java @@ -44,8 +44,8 @@ public class TestTimestampWithoutZone extends CatalogTestBase { - private static final String newTableName = "created_table"; - private static final Schema schema = + private static final String NEW_TABLE_NAME = "created_table"; + private static final Schema SCHEMA = new Schema( Types.NestedField.required(1, "id", Types.LongType.get()), Types.NestedField.required(2, "ts", Types.TimestampType.withoutZone()), @@ -74,13 +74,13 @@ public static Object[][] parameters() { @BeforeEach public void createTables() { - validationCatalog.createTable(tableIdent, schema); + validationCatalog.createTable(tableIdent, SCHEMA); } @AfterEach public void removeTables() { validationCatalog.dropTable(tableIdent, true); - sql("DROP TABLE IF EXISTS %s", newTableName); + sql("DROP TABLE IF EXISTS %s", NEW_TABLE_NAME); } /* @@ -141,34 +141,34 @@ public void testAppendTimestampWithZone() { public void testCreateAsSelectWithTimestampWithoutZone() { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); - assertThat(scalarSql("SELECT count(*) FROM %s", newTableName)) + assertThat(scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)) .as("Should have " + values.size() + " row") .isEqualTo((long) values.size()); assertEquals( "Row data should match expected", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); } @TestTemplate public void testCreateNewTableShouldHaveTimestampWithZoneIcebergType() { sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); - assertThat(scalarSql("SELECT count(*) FROM %s", newTableName)) + assertThat(scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)) .as("Should have " + values.size() + " row") .isEqualTo((long) values.size()); assertEquals( "Data from created table should match data from base table", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); - Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts"); assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); } @@ -182,17 +182,17 @@ public void testCreateNewTableShouldHaveTimestampWithoutZoneIcebergType() { .initialize(catalog.name(), new CaseInsensitiveStringMap(catalogConfig)); sql("INSERT INTO %s VALUES %s", tableName, rowToSqlValues(values)); - sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", newTableName, tableName); + sql("CREATE TABLE %s USING iceberg AS SELECT * FROM %s", NEW_TABLE_NAME, tableName); - assertThat(scalarSql("SELECT count(*) FROM %s", newTableName)) + assertThat(scalarSql("SELECT count(*) FROM %s", NEW_TABLE_NAME)) .as("Should have " + values.size() + " row") .isEqualTo((long) values.size()); assertEquals( "Row data should match expected", sql("SELECT * FROM %s ORDER BY id", tableName), - sql("SELECT * FROM %s ORDER BY id", newTableName)); - Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", newTableName)); + sql("SELECT * FROM %s ORDER BY id", NEW_TABLE_NAME)); + Table createdTable = validationCatalog.loadTable(TableIdentifier.of("default", NEW_TABLE_NAME)); assertFieldsType(createdTable.schema(), Types.TimestampType.withoutZone(), "ts"); assertFieldsType(createdTable.schema(), Types.TimestampType.withZone(), "tsz"); } From 726ce2aef5c13522d12cc5619e93ece1d698dd9d Mon Sep 17 00:00:00 2001 From: fengjiajie Date: Thu, 11 Jul 2024 03:46:38 +0800 Subject: [PATCH 0455/1019] Flink: Backport #10565 to v1.18 and v1.19 (#10676) --- .../iceberg/flink/data/RowDataUtil.java | 29 +++++++++++++++++-- .../source/reader/RowDataRecordFactory.java | 14 ++++++++- .../org/apache/iceberg/flink/TestHelpers.java | 7 ++++- .../iceberg/flink/data/RowDataUtil.java | 29 +++++++++++++++++-- .../source/reader/RowDataRecordFactory.java | 14 ++++++++- .../org/apache/iceberg/flink/TestHelpers.java | 7 ++++- 6 files changed, 90 insertions(+), 10 deletions(-) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index c5cb51b7eae4..3a8f5ccc6c03 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -79,22 +79,45 @@ public static Object convertConstant(Type type, Object value) { * the arity check. */ public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData from, + RowData reuse, + RowType rowType, + TypeSerializer[] fieldSerializers, + RowData.FieldGetter[] fieldGetters) { GenericRowData ret; if (reuse instanceof GenericRowData) { ret = (GenericRowData) reuse; } else { ret = new GenericRowData(from.getArity()); } + ret.setRowKind(from.getRowKind()); for (int i = 0; i < rowType.getFieldCount(); i++) { if (!from.isNullAt(i)) { - RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); - ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); + ret.setField(i, fieldSerializers[i].copy(fieldGetters[i].getFieldOrNull(from))); } else { ret.setField(i, null); } } + return ret; } + + /** + * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to + * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], + * RowData.FieldGetter[])} instead. + */ + @Deprecated + public static RowData clone( + RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + if (!from.isNullAt(i)) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + } + + return clone(from, reuse, rowType, fieldSerializers, fieldGetters); + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java index 1e265b2663ce..40d5c28d7bc7 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java @@ -28,10 +28,12 @@ class RowDataRecordFactory implements RecordFactory { private final RowType rowType; private final TypeSerializer[] fieldSerializers; + private final RowData.FieldGetter[] fieldGetters; RowDataRecordFactory(RowType rowType) { this.rowType = rowType; this.fieldSerializers = createFieldSerializers(rowType); + this.fieldGetters = createFieldGetters(rowType); } static TypeSerializer[] createFieldSerializers(RowType rowType) { @@ -40,6 +42,15 @@ static TypeSerializer[] createFieldSerializers(RowType rowType) { .toArray(TypeSerializer[]::new); } + static RowData.FieldGetter[] createFieldGetters(RowType rowType) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return fieldGetters; + } + @Override public RowData[] createBatch(int batchSize) { RowData[] arr = new RowData[batchSize]; @@ -56,6 +67,7 @@ public void clone(RowData from, RowData[] batch, int position) { // Clone method will allocate a new GenericRowData object // if the target object is NOT a GenericRowData. // So we should always set the clone return value back to the array. - batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); + batch[position] = + RowDataUtil.clone(from, batch[position], rowType, fieldSerializers, fieldGetters); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java index a243be5834a8..8cebf950c5f0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -87,7 +87,12 @@ public static RowData copyRowData(RowData from, RowType rowType) { rowType.getChildren().stream() .map((LogicalType type) -> InternalSerializers.create(type)) .toArray(TypeSerializer[]::new); - return RowDataUtil.clone(from, null, rowType, fieldSerializers); + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return RowDataUtil.clone(from, null, rowType, fieldSerializers, fieldGetters); } public static void readRowData(FlinkInputFormat input, Consumer visitor) diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index c5cb51b7eae4..3a8f5ccc6c03 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -79,22 +79,45 @@ public static Object convertConstant(Type type, Object value) { * the arity check. */ public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData from, + RowData reuse, + RowType rowType, + TypeSerializer[] fieldSerializers, + RowData.FieldGetter[] fieldGetters) { GenericRowData ret; if (reuse instanceof GenericRowData) { ret = (GenericRowData) reuse; } else { ret = new GenericRowData(from.getArity()); } + ret.setRowKind(from.getRowKind()); for (int i = 0; i < rowType.getFieldCount(); i++) { if (!from.isNullAt(i)) { - RowData.FieldGetter getter = RowData.createFieldGetter(rowType.getTypeAt(i), i); - ret.setField(i, fieldSerializers[i].copy(getter.getFieldOrNull(from))); + ret.setField(i, fieldSerializers[i].copy(fieldGetters[i].getFieldOrNull(from))); } else { ret.setField(i, null); } } + return ret; } + + /** + * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to + * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], + * RowData.FieldGetter[])} instead. + */ + @Deprecated + public static RowData clone( + RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + if (!from.isNullAt(i)) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + } + + return clone(from, reuse, rowType, fieldSerializers, fieldGetters); + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java index 1e265b2663ce..40d5c28d7bc7 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java @@ -28,10 +28,12 @@ class RowDataRecordFactory implements RecordFactory { private final RowType rowType; private final TypeSerializer[] fieldSerializers; + private final RowData.FieldGetter[] fieldGetters; RowDataRecordFactory(RowType rowType) { this.rowType = rowType; this.fieldSerializers = createFieldSerializers(rowType); + this.fieldGetters = createFieldGetters(rowType); } static TypeSerializer[] createFieldSerializers(RowType rowType) { @@ -40,6 +42,15 @@ static TypeSerializer[] createFieldSerializers(RowType rowType) { .toArray(TypeSerializer[]::new); } + static RowData.FieldGetter[] createFieldGetters(RowType rowType) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return fieldGetters; + } + @Override public RowData[] createBatch(int batchSize) { RowData[] arr = new RowData[batchSize]; @@ -56,6 +67,7 @@ public void clone(RowData from, RowData[] batch, int position) { // Clone method will allocate a new GenericRowData object // if the target object is NOT a GenericRowData. // So we should always set the clone return value back to the array. - batch[position] = RowDataUtil.clone(from, batch[position], rowType, fieldSerializers); + batch[position] = + RowDataUtil.clone(from, batch[position], rowType, fieldSerializers, fieldGetters); } } 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 a243be5834a8..8cebf950c5f0 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 @@ -87,7 +87,12 @@ public static RowData copyRowData(RowData from, RowType rowType) { rowType.getChildren().stream() .map((LogicalType type) -> InternalSerializers.create(type)) .toArray(TypeSerializer[]::new); - return RowDataUtil.clone(from, null, rowType, fieldSerializers); + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return RowDataUtil.clone(from, null, rowType, fieldSerializers, fieldGetters); } public static void readRowData(FlinkInputFormat input, Consumer visitor) From d12e6c66eae2b6722570bc94b914dd3041cb8c0e Mon Sep 17 00:00:00 2001 From: Attila Kreiner Date: Thu, 11 Jul 2024 15:42:38 +0200 Subject: [PATCH 0456/1019] Rename & enforce constants to be all uppercase (#10673) --- .baseline/checkstyle/checkstyle.xml | 4 + .../aliyun/oss/TestOSSOutputStream.java | 4 +- .../org/apache/iceberg/events/Listeners.java | 6 +- .../apache/iceberg/util/CharSequenceSet.java | 6 +- .../expressions/TestAggregateBinding.java | 16 +-- .../expressions/TestAggregateEvaluator.java | 10 +- .../vectorized/ArrowVectorAccessors.java | 6 +- .../apache/iceberg/aws/glue/GlueTestBase.java | 50 +++++----- .../glue/TestGlueCatalogCommitFailure.java | 4 +- .../iceberg/aws/glue/TestGlueCatalogLock.java | 8 +- .../aws/glue/TestGlueCatalogNamespace.java | 16 +-- .../aws/glue/TestGlueCatalogTable.java | 80 +++++++-------- .../apache/iceberg/aws/s3/S3OutputStream.java | 6 +- .../java/org/apache/iceberg/avro/AvroIO.java | 18 ++-- .../encryption/StandardKeyMetadata.java | 8 +- .../iceberg/TestMetadataTableFilters.java | 4 +- .../iceberg/hadoop/TestHadoopCatalog.java | 4 +- .../data/TestMetricsRowGroupFilter.java | 12 +-- .../data/TestMetricsRowGroupFilterTypes.java | 29 +++--- ...TestParquetEncryptionWithWriteSupport.java | 32 +++--- .../apache/iceberg/io/TestWriterMetrics.java | 6 +- .../delta/TestSnapshotDeltaLakeTable.java | 48 ++++----- ...LakeToIcebergMigrationActionsProvider.java | 4 +- .../delta/TestDeltaLakeTypeToType.java | 97 ++++++++++--------- .../org/apache/iceberg/hive/HiveVersion.java | 6 +- .../iceberg/hive/HiveTableBaseTest.java | 8 +- .../apache/iceberg/hive/HiveTableTest.java | 32 +++--- .../apache/iceberg/hive/TestHiveCatalog.java | 14 +-- .../iceberg/hive/TestHiveCommitLocks.java | 16 +-- .../apache/iceberg/hive/TestHiveCommits.java | 4 +- .../ql/exec/vector/VectorizedSupport.java | 1 + .../mr/hive/HiveIcebergRecordWriter.java | 10 +- .../iceberg/mr/hive/TestHiveIcebergSerDe.java | 8 +- ...TestHiveIcebergStorageHandlerTimezone.java | 8 +- .../iceberg/nessie/TestMultipleClients.java | 10 +- .../iceberg/nessie/TestNessieTable.java | 14 +-- .../parquet/TestBloomRowGroupFilter.java | 28 +++--- .../parquet/TestDictionaryRowGroupFilter.java | 10 +- .../parquet/TestParquetEncryption.java | 44 ++++----- .../apache/iceberg/pig/IcebergStorage.java | 14 +-- 40 files changed, 359 insertions(+), 346 deletions(-) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index c36700c8c122..48ef3d690ef7 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -284,6 +284,10 @@ + + + + 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 069ff9111a5a..8fc661e5be10 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 @@ -50,7 +50,7 @@ public class TestOSSOutputStream extends AliyunOSSTestBase { private final OSS ossMock = mock(OSS.class, delegatesTo(ossClient)); private final Path tmpDir = Files.createTempDirectory("oss-file-io-test-"); - private static final Random random = ThreadLocalRandom.current(); + private static final Random RANDOM = ThreadLocalRandom.current(); private final AliyunProperties props = new AliyunProperties( @@ -127,7 +127,7 @@ private byte[] data256() { private byte[] randomData(int size) { byte[] data = new byte[size]; - random.nextBytes(data); + RANDOM.nextBytes(data); return data; } diff --git a/api/src/main/java/org/apache/iceberg/events/Listeners.java b/api/src/main/java/org/apache/iceberg/events/Listeners.java index 27c9c0590658..429cf7c7e272 100644 --- a/api/src/main/java/org/apache/iceberg/events/Listeners.java +++ b/api/src/main/java/org/apache/iceberg/events/Listeners.java @@ -28,11 +28,11 @@ public class Listeners { private Listeners() {} - private static final Map, Queue>> listeners = Maps.newConcurrentMap(); + private static final Map, Queue>> LISTENERS = Maps.newConcurrentMap(); public static void register(Listener listener, Class eventType) { Queue> list = - listeners.computeIfAbsent(eventType, k -> new ConcurrentLinkedQueue<>()); + LISTENERS.computeIfAbsent(eventType, k -> new ConcurrentLinkedQueue<>()); list.add(listener); } @@ -40,7 +40,7 @@ public static void register(Listener listener, Class eventType) { public static void notifyAll(E event) { Preconditions.checkNotNull(event, "Cannot notify listeners for a null event."); - Queue> list = listeners.get(event.getClass()); + Queue> list = LISTENERS.get(event.getClass()); if (list != null) { for (Listener value : list) { Listener listener = (Listener) value; diff --git a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java index 1bb5a1dc4e87..cfdac0104c47 100644 --- a/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java +++ b/api/src/main/java/org/apache/iceberg/util/CharSequenceSet.java @@ -30,7 +30,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Streams; public class CharSequenceSet implements Set, Serializable { - private static final ThreadLocal wrappers = + private static final ThreadLocal WRAPPERS = ThreadLocal.withInitial(() -> CharSequenceWrapper.wrap(null)); public static CharSequenceSet of(Iterable charSequences) { @@ -61,7 +61,7 @@ public boolean isEmpty() { @Override public boolean contains(Object obj) { if (obj instanceof CharSequence) { - CharSequenceWrapper wrapper = wrappers.get(); + CharSequenceWrapper wrapper = WRAPPERS.get(); boolean result = wrapperSet.contains(wrapper.set((CharSequence) obj)); wrapper.set(null); // don't hold a reference to the value return result; @@ -109,7 +109,7 @@ public boolean add(CharSequence charSequence) { @Override public boolean remove(Object obj) { if (obj instanceof CharSequence) { - CharSequenceWrapper wrapper = wrappers.get(); + CharSequenceWrapper wrapper = WRAPPERS.get(); boolean result = wrapperSet.remove(wrapper.set((CharSequence) obj)); wrapper.set(null); // don't hold a reference to the value return result; diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java index 95a9ac2cc83a..23c15b5461e7 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateBinding.java @@ -29,15 +29,15 @@ import org.junit.jupiter.api.Test; public class TestAggregateBinding { - private static final List> list = + private static final List> LIST = ImmutableList.of(Expressions.count("x"), Expressions.max("x"), Expressions.min("x")); - private static final StructType struct = + private static final StructType STRUCT = StructType.of(Types.NestedField.required(10, "x", Types.IntegerType.get())); @Test public void testAggregateBinding() { - for (UnboundAggregate unbound : list) { - Expression expr = unbound.bind(struct, true); + for (UnboundAggregate unbound : LIST) { + Expression expr = unbound.bind(STRUCT, true); BoundAggregate bound = assertAndUnwrapAggregate(expr); assertThat(bound.ref().fieldId()).as("Should reference correct field ID").isEqualTo(10); assertThat(bound.op()) @@ -60,7 +60,7 @@ public void testCountStarBinding() { @Test public void testBoundAggregateFails() { Expression unbound = Expressions.count("x"); - assertThatThrownBy(() -> Binder.bind(struct, Binder.bind(struct, unbound))) + assertThatThrownBy(() -> Binder.bind(STRUCT, Binder.bind(STRUCT, unbound))) .isInstanceOf(IllegalStateException.class) .hasMessageContaining("Found already bound aggregate"); } @@ -68,7 +68,7 @@ public void testBoundAggregateFails() { @Test public void testCaseInsensitiveReference() { Expression expr = Expressions.max("X"); - Expression boundExpr = Binder.bind(struct, expr, false); + Expression boundExpr = Binder.bind(STRUCT, expr, false); BoundAggregate bound = assertAndUnwrapAggregate(boundExpr); assertThat(bound.ref().fieldId()).as("Should reference correct field ID").isEqualTo(10); assertThat(bound.op()) @@ -79,7 +79,7 @@ public void testCaseInsensitiveReference() { @Test public void testCaseSensitiveReference() { Expression expr = Expressions.max("X"); - assertThatThrownBy(() -> Binder.bind(struct, expr, true)) + assertThatThrownBy(() -> Binder.bind(STRUCT, expr, true)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'X' in struct"); } @@ -87,7 +87,7 @@ public void testCaseSensitiveReference() { @Test public void testMissingField() { UnboundAggregate unbound = Expressions.count("missing"); - assertThatThrownBy(() -> unbound.bind(struct, false)) + assertThatThrownBy(() -> unbound.bind(STRUCT, false)) .isInstanceOf(ValidationException.class) .hasMessageContaining("Cannot find field 'missing' in struct:"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java index b418dede8610..aa15d36de3f7 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestAggregateEvaluator.java @@ -91,7 +91,7 @@ public class TestAggregateEvaluator { // upper bounds ImmutableMap.of(1, toByteBuffer(IntegerType.get(), 3333))); - private static final DataFile[] dataFiles = { + private static final DataFile[] DATA_FILES = { FILE, MISSING_SOME_NULLS_STATS_1, MISSING_SOME_NULLS_STATS_2 }; @@ -121,7 +121,7 @@ public void testIntAggregate() { Expressions.min("id")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } @@ -141,7 +141,7 @@ public void testAllNulls() { Expressions.min("all_nulls")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } @@ -160,7 +160,7 @@ public void testSomeNulls() { Expressions.max("some_nulls"), Expressions.min("some_nulls")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } @@ -179,7 +179,7 @@ public void testNoStats() { Expressions.max("no_stats"), Expressions.min("no_stats")); AggregateEvaluator aggregateEvaluator = AggregateEvaluator.create(SCHEMA, list); - for (DataFile dataFile : dataFiles) { + for (DataFile dataFile : DATA_FILES) { aggregateEvaluator.update(dataFile); } diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java index 34800cff490e..24af804b18aa 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/ArrowVectorAccessors.java @@ -28,10 +28,10 @@ final class ArrowVectorAccessors { - private static final GenericArrowVectorAccessorFactory factory; + private static final GenericArrowVectorAccessorFactory FACTORY; static { - factory = + FACTORY = new GenericArrowVectorAccessorFactory<>( JavaDecimalFactory::new, JavaStringFactory::new, @@ -51,7 +51,7 @@ private ArrowVectorAccessors() { } static ArrowVectorAccessor getVectorAccessor(VectorHolder holder) { - return factory.getVectorAccessor(holder); + return FACTORY.getVectorAccessor(holder); } private static final class JavaStringFactory implements StringFactory { 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 aa0c7f18319c..ecf589d7c07a 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 @@ -55,16 +55,16 @@ public class GlueTestBase { private static final Logger LOG = LoggerFactory.getLogger(GlueTestBase.class); // the integration test requires the following env variables - static final String testBucketName = AwsIntegTestUtil.testBucketName(); + static final String TEST_BUCKET_NAME = AwsIntegTestUtil.testBucketName(); - static final String catalogName = "glue"; - static final String testPathPrefix = getRandomName(); - static final List namespaces = Lists.newArrayList(); + static final String CATALOG_NAME = "glue"; + static final String TEST_PATH_PREFIX = getRandomName(); + static final List NAMESPACES = Lists.newArrayList(); // aws clients - static final AwsClientFactory clientFactory = AwsClientFactories.defaultFactory(); - static final GlueClient glue = clientFactory.glue(); - static final S3Client s3 = clientFactory.s3(); + static final AwsClientFactory CLIENT_FACTORY = AwsClientFactories.defaultFactory(); + static final GlueClient GLUE = CLIENT_FACTORY.glue(); + static final S3Client S3 = CLIENT_FACTORY.s3(); // iceberg static GlueCatalog glueCatalog; @@ -74,14 +74,14 @@ public class GlueTestBase { new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1")); static PartitionSpec partitionSpec = PartitionSpec.builderFor(schema).build(); // table location properties - static final Map tableLocationProperties = + static final Map TABLE_LOCATION_PROPERTIES = ImmutableMap.of( - TableProperties.WRITE_DATA_LOCATION, "s3://" + testBucketName + "/writeDataLoc", - TableProperties.WRITE_METADATA_LOCATION, "s3://" + testBucketName + "/writeMetaDataLoc", + TableProperties.WRITE_DATA_LOCATION, "s3://" + TEST_BUCKET_NAME + "/writeDataLoc", + TableProperties.WRITE_METADATA_LOCATION, "s3://" + TEST_BUCKET_NAME + "/writeMetaDataLoc", TableProperties.WRITE_FOLDER_STORAGE_LOCATION, - "s3://" + testBucketName + "/writeFolderStorageLoc"); + "s3://" + TEST_BUCKET_NAME + "/writeFolderStorageLoc"); - static final String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; + static final String TEST_BUCKET_PATH = "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX; @BeforeAll public static void beforeClass() { @@ -90,11 +90,11 @@ public static void beforeClass() { S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); s3FileIOProperties.setDeleteBatchSize(10); glueCatalog.initialize( - catalogName, - testBucketPath, + CATALOG_NAME, + TEST_BUCKET_PATH, awsProperties, s3FileIOProperties, - glue, + GLUE, null, ImmutableMap.of()); @@ -102,19 +102,19 @@ public static void beforeClass() { AwsProperties propertiesSkipNameValidation = new AwsProperties(); propertiesSkipNameValidation.setGlueCatalogSkipNameValidation(true); glueCatalogWithSkipNameValidation.initialize( - catalogName, - testBucketPath, + CATALOG_NAME, + TEST_BUCKET_PATH, propertiesSkipNameValidation, new S3FileIOProperties(), - glue, + GLUE, null, ImmutableMap.of()); } @AfterAll public static void afterClass() { - AwsIntegTestUtil.cleanGlueCatalog(glue, namespaces); - AwsIntegTestUtil.cleanS3Bucket(s3, testBucketName, testPathPrefix); + AwsIntegTestUtil.cleanGlueCatalog(GLUE, NAMESPACES); + AwsIntegTestUtil.cleanS3Bucket(S3, TEST_BUCKET_NAME, TEST_PATH_PREFIX); } public static String getRandomName() { @@ -123,7 +123,7 @@ public static String getRandomName() { public static String createNamespace() { String namespace = getRandomName(); - namespaces.add(namespace); + NAMESPACES.add(namespace); glueCatalog.createNamespace(Namespace.of(namespace)); return namespace; } @@ -142,7 +142,7 @@ public static String createTable(String namespace, String tableName) { public static void updateTableDescription( String namespace, String tableName, String description) { GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); Table table = response.table(); UpdateTableRequest request = UpdateTableRequest.builder() @@ -159,13 +159,13 @@ public static void updateTableDescription( .storageDescriptor(table.storageDescriptor()) .build()) .build(); - glue.updateTable(request); + GLUE.updateTable(request); } public static void updateTableColumns( String namespace, String tableName, Function columnUpdater) { GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); Table existingTable = response.table(); List updatedColumns = existingTable.storageDescriptor().columns().stream() @@ -192,6 +192,6 @@ public static void updateTableColumns( .build()) .build()) .build(); - glue.updateTable(request); + GLUE.updateTable(request); } } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java index f1748737874e..42b527a03742 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java @@ -510,7 +510,7 @@ private void breakFallbackCatalogCommitCheck(GlueTableOperations spyOperations) private boolean metadataFileExists(TableMetadata metadata) { try { - s3.headObject( + S3.headObject( HeadObjectRequest.builder() .bucket(S3TestUtil.getBucketFromUri(metadata.metadataFileLocation())) .key(S3TestUtil.getKeyFromUri(metadata.metadataFileLocation())) @@ -523,7 +523,7 @@ private boolean metadataFileExists(TableMetadata metadata) { private int metadataFileCount(TableMetadata metadata) { return (int) - s3 + S3 .listObjectsV2( ListObjectsV2Request.builder() .bucket(S3TestUtil.getBucketFromUri(metadata.metadataFileLocation())) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java index 53ec2a252fea..3edd9e4acdb4 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogLock.java @@ -56,18 +56,18 @@ public class TestGlueCatalogLock extends GlueTestBase { @BeforeAll public static void beforeClass() { GlueTestBase.beforeClass(); - String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; + String testBucketPath = "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX; lockTableName = getRandomName(); glueCatalog = new GlueCatalog(); AwsProperties awsProperties = new AwsProperties(); S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(); - dynamo = clientFactory.dynamo(); + dynamo = CLIENT_FACTORY.dynamo(); glueCatalog.initialize( - catalogName, + CATALOG_NAME, testBucketPath, awsProperties, s3FileIOProperties, - glue, + GLUE, new DynamoDbLockManager(dynamo, lockTableName), ImmutableMap.of()); } diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java index f36207005116..7a249c5509f2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogNamespace.java @@ -44,8 +44,8 @@ public class TestGlueCatalogNamespace extends GlueTestBase { @Test public void testCreateNamespace() { String namespace = getRandomName(); - namespaces.add(namespace); - assertThatThrownBy(() -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) + NAMESPACES.add(namespace); + assertThatThrownBy(() -> GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) .as("namespace does not exist before create") .isInstanceOf(EntityNotFoundException.class) .hasMessageContaining("not found"); @@ -60,7 +60,7 @@ public void testCreateNamespace() { Namespace ns = Namespace.of(namespace); glueCatalog.createNamespace(ns, properties); Database database = - glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.name()).isEqualTo(namespace); assertThat(database.description()).isEqualTo("description"); assertThat(database.locationUri()).isEqualTo("s3://location"); @@ -117,7 +117,7 @@ public void testNamespaceProperties() { properties.put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, "description"); glueCatalog.setProperties(Namespace.of(namespace), properties); Database database = - glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.parameters()).containsEntry("key", "val").containsEntry("key2", "val2"); assertThat(database.locationUri()).isEqualTo("s3://test"); assertThat(database.description()).isEqualTo("description"); @@ -128,7 +128,7 @@ public void testNamespaceProperties() { "key", IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, IcebergToGlueConverter.GLUE_DESCRIPTION_KEY)); - database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + database = GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.parameters()).doesNotContainKey("key").containsEntry("key2", "val2"); assertThat(database.locationUri()).isNull(); assertThat(database.description()).isNull(); @@ -138,7 +138,7 @@ public void testNamespaceProperties() { properties.put(IcebergToGlueConverter.GLUE_DB_LOCATION_KEY, "s3://test2"); properties.put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, "description2"); glueCatalog.setProperties(Namespace.of(namespace), properties); - database = glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); + database = GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build()).database(); assertThat(database.parameters()).containsEntry("key", "val").containsEntry("key2", "val2"); assertThat(database.locationUri()).isEqualTo("s3://test2"); assertThat(database.description()).isEqualTo("description2"); @@ -148,7 +148,7 @@ public void testNamespaceProperties() { public void testDropNamespace() { String namespace = createNamespace(); glueCatalog.dropNamespace(Namespace.of(namespace)); - assertThatThrownBy(() -> glue.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) + assertThatThrownBy(() -> GLUE.getDatabase(GetDatabaseRequest.builder().name(namespace).build())) .as("namespace should not exist after deletion") .isInstanceOf(EntityNotFoundException.class) .hasMessageContaining("not found"); @@ -167,7 +167,7 @@ public void testDropNamespaceThatContainsOnlyIcebergTable() { @Test public void testDropNamespaceThatContainsNonIcebergTable() { String namespace = createNamespace(); - glue.createTable( + GLUE.createTable( CreateTableRequest.builder() .databaseName(namespace) .tableInput(TableInput.builder().name(UUID.randomUUID().toString()).build()) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 9c4d1839a4eb..6bd6a4ad383d 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -76,14 +76,14 @@ public void testCreateTable() { String tableDescription = "Test table"; Map tableProperties = ImmutableMap.builder() - .putAll(tableLocationProperties) + .putAll(TABLE_LOCATION_PROPERTIES) .put(IcebergToGlueConverter.GLUE_DESCRIPTION_KEY, tableDescription) .build(); glueCatalog.createTable( TableIdentifier.of(namespace, tableName), schema, partitionSpec, tableProperties); // verify table exists in Glue GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().databaseName()).isEqualTo(namespace); assertThat(response.table().name()).isEqualTo(tableName); assertThat(response.table().parameters()) @@ -94,12 +94,12 @@ public void testCreateTable() { assertThat(response.table().storageDescriptor().columns()).hasSameSizeAs(schema.columns()); assertThat(response.table().partitionKeys()).hasSameSizeAs(partitionSpec.fields()); assertThat(response.table().storageDescriptor().additionalLocations()) - .containsExactlyInAnyOrderElementsOf(tableLocationProperties.values()); + .containsExactlyInAnyOrderElementsOf(TABLE_LOCATION_PROPERTIES.values()); // verify metadata file exists in S3 String metaLocation = response.table().parameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); - String key = metaLocation.split(testBucketName, -1)[1].substring(1); - s3.headObject(HeadObjectRequest.builder().bucket(testBucketName).key(key).build()); + String key = metaLocation.split(TEST_BUCKET_NAME, -1)[1].substring(1); + S3.headObject(HeadObjectRequest.builder().bucket(TEST_BUCKET_NAME).key(key).build()); Table table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); assertThat(table.spec()).isEqualTo(partitionSpec); assertThat(table.schema()).asString().isEqualTo(schema.toString()); @@ -137,18 +137,18 @@ public void testCreateTableBadName() { public void testCreateAndLoadTableWithoutWarehouseLocation() { GlueCatalog glueCatalogWithoutWarehouse = new GlueCatalog(); glueCatalogWithoutWarehouse.initialize( - catalogName, + CATALOG_NAME, null, new AwsProperties(), new S3FileIOProperties(), - glue, + GLUE, LockManagers.defaultLockManager(), ImmutableMap.of()); String namespace = createNamespace(); String tableName = getRandomName(); TableIdentifier identifier = TableIdentifier.of(namespace, tableName); try { - glueCatalog.createTable(identifier, schema, partitionSpec, tableLocationProperties); + glueCatalog.createTable(identifier, schema, partitionSpec, TABLE_LOCATION_PROPERTIES); glueCatalog.loadTable(identifier); } catch (RuntimeException e) { throw new RuntimeException( @@ -202,7 +202,7 @@ public void testUpdateTable() { assertThat(table.history()).hasSize(1); // check table in Glue GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().tableType()) .as("external table type is set after update") .isEqualTo("EXTERNAL_TABLE"); @@ -217,7 +217,7 @@ public void testUpdateTable() { .commit(); // check table in Glue response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().description()).isEqualTo(updatedComment); } @@ -246,7 +246,7 @@ public void testDropColumn() { table.updateSchema().deleteColumn("c2").deleteColumn("c3").commit(); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); List actualColumns = response.table().storageDescriptor().columns(); List expectedColumns = @@ -308,7 +308,7 @@ public void testRenameTableFailsToCreateNewTable() { Table table = glueCatalog.loadTable(id); // create a new table in Glue, so that rename to that table will fail String newTableName = tableName + "_2"; - glue.createTable( + GLUE.createTable( CreateTableRequest.builder() .databaseName(namespace) .tableInput(TableInput.builder().name(newTableName).build()) @@ -337,7 +337,7 @@ public void testRenameTableFailsToDeleteOldTable() { Table table = glueCatalog.loadTable(id); // delete the old table metadata, so that drop old table will fail String newTableName = tableName + "_2"; - glue.updateTable( + GLUE.updateTable( UpdateTableRequest.builder() .databaseName(namespace) .tableInput(TableInput.builder().name(tableName).parameters(Maps.newHashMap()).build()) @@ -352,7 +352,7 @@ public void testRenameTableFailsToDeleteOldTable() { .hasMessageContaining("Input Glue table is not an iceberg table"); assertThatThrownBy( () -> - glue.getTable( + GLUE.getTable( GetTableRequest.builder().databaseName(namespace).name(newTableName).build())) .isInstanceOf(EntityNotFoundException.class) .as("renamed table should be deleted") @@ -370,11 +370,11 @@ public void testDeleteTableWithoutPurge() { .hasMessageContaining("Table does not exist"); String warehouseLocation = glueCatalog.defaultWarehouseLocation(TableIdentifier.of(namespace, tableName)); - String prefix = warehouseLocation.split(testBucketName + "/", -1)[1]; + String prefix = warehouseLocation.split(TEST_BUCKET_NAME + "/", -1)[1]; ListObjectsV2Response response = - s3.listObjectsV2( + S3.listObjectsV2( ListObjectsV2Request.builder() - .bucket(testBucketName) + .bucket(TEST_BUCKET_NAME) .prefix(prefix + "/metadata/") .build()); assertThat(response.hasContents()).isTrue(); @@ -423,10 +423,10 @@ public void testDeleteTableWithPurge() { .hasMessageContaining("Table does not exist"); String warehouseLocation = glueCatalog.defaultWarehouseLocation(TableIdentifier.of(namespace, tableName)); - String prefix = warehouseLocation.split(testBucketName + "/", -1)[1]; + String prefix = warehouseLocation.split(TEST_BUCKET_NAME + "/", -1)[1]; ListObjectsV2Response response = - s3.listObjectsV2( - ListObjectsV2Request.builder().bucket(testBucketName).prefix(prefix).build()); + S3.listObjectsV2( + ListObjectsV2Request.builder().bucket(TEST_BUCKET_NAME).prefix(prefix).build()); if (response.hasContents()) { // might have directory markers left for (S3Object s3Object : response.contents()) { @@ -441,7 +441,7 @@ public void testDeleteTableWithPurge() { public void testCommitTableSkipArchive() { // create ns String namespace = getRandomName(); - namespaces.add(namespace); + NAMESPACES.add(namespace); glueCatalog.createNamespace(Namespace.of(namespace)); // create table and commit without skip Schema schema = new Schema(Types.NestedField.required(1, "c1", Types.StringType.get(), "c1")); @@ -450,11 +450,11 @@ public void testCommitTableSkipArchive() { AwsProperties properties = new AwsProperties(); properties.setGlueCatalogSkipArchive(false); glueCatalog.initialize( - catalogName, - testBucketPath, + CATALOG_NAME, + TEST_BUCKET_PATH, properties, new S3FileIOProperties(), - glue, + GLUE, LockManagers.defaultLockManager(), ImmutableMap.of()); glueCatalog.createTable(TableIdentifier.of(namespace, tableName), schema, partitionSpec); @@ -467,7 +467,7 @@ public void testCommitTableSkipArchive() { .build(); table.newAppend().appendFile(dataFile).commit(); assertThat( - glue.getTableVersions( + GLUE.getTableVersions( GetTableVersionsRequest.builder() .databaseName(namespace) .tableName(tableName) @@ -476,12 +476,12 @@ public void testCommitTableSkipArchive() { .hasSize(2); // create table and commit with skip tableName = getRandomName(); - glueCatalog.initialize(catalogName, ImmutableMap.of()); + glueCatalog.initialize(CATALOG_NAME, ImmutableMap.of()); glueCatalog.createTable(TableIdentifier.of(namespace, tableName), schema, partitionSpec); table = glueCatalog.loadTable(TableIdentifier.of(namespace, tableName)); table.newAppend().appendFile(dataFile).commit(); assertThat( - glue.getTableVersions( + GLUE.getTableVersions( GetTableVersionsRequest.builder() .databaseName(namespace) .tableName(tableName) @@ -494,13 +494,13 @@ public void testCommitTableSkipArchive() { @Test public void testCommitTableSkipNameValidation() { String namespace = "dd-dd"; - namespaces.add(namespace); + NAMESPACES.add(namespace); glueCatalogWithSkipNameValidation.createNamespace(Namespace.of(namespace)); String tableName = "cc-cc"; glueCatalogWithSkipNameValidation.createTable( - TableIdentifier.of(namespace, tableName), schema, partitionSpec, tableLocationProperties); + TableIdentifier.of(namespace, tableName), schema, partitionSpec, TABLE_LOCATION_PROPERTIES); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); assertThat(response.table().databaseName()).isEqualTo(namespace); assertThat(response.table().name()).isEqualTo(tableName); } @@ -522,7 +522,7 @@ public void testColumnCommentsAndParameters() { table.updateSpec().addField(truncate("c1", 8)).commit(); table.updateSchema().deleteColumn("c3").renameColumn("c4", "c5").commit(); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); List actualColumns = response.table().storageDescriptor().columns(); List expectedColumns = @@ -605,7 +605,7 @@ public void testGlueTableColumnCommentsPreserved() { .commit(); GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); List actualColumns = response.table().storageDescriptor().columns(); List expectedColumns = @@ -655,7 +655,7 @@ public void testTablePropsDefinedAtCatalogLevel() { "table-default.key3", "catalog-default-key3", "table-override.key3", "catalog-override-key3", "table-override.key4", "catalog-override-key4", - "warehouse", "s3://" + testBucketName + "/" + testPathPrefix); + "warehouse", "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX); glueCatalog.initialize("glue", catalogProps); @@ -722,7 +722,7 @@ public void testRegisterTableAlreadyExists() { @Test public void testTableLevelS3Tags() { - String testBucketPath = "s3://" + testBucketName + "/" + testPathPrefix; + String testBucketPath = "s3://" + TEST_BUCKET_NAME + "/" + TEST_PATH_PREFIX; Map properties = ImmutableMap.of( S3FileIOProperties.WRITE_TABLE_TAG_ENABLED, @@ -730,11 +730,11 @@ public void testTableLevelS3Tags() { S3FileIOProperties.WRITE_NAMESPACE_TAG_ENABLED, "true"); glueCatalog.initialize( - catalogName, + CATALOG_NAME, testBucketPath, new AwsProperties(properties), new S3FileIOProperties(properties), - glue, + GLUE, null); String namespace = createNamespace(); String tableName = getRandomName(); @@ -742,13 +742,13 @@ public void testTableLevelS3Tags() { // Get metadata object tag from S3 GetTableResponse response = - glue.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); + GLUE.getTable(GetTableRequest.builder().databaseName(namespace).name(tableName).build()); String metaLocation = response.table().parameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); - String key = metaLocation.split(testBucketName, -1)[1].substring(1); + String key = metaLocation.split(TEST_BUCKET_NAME, -1)[1].substring(1); List tags = - s3.getObjectTagging( - GetObjectTaggingRequest.builder().bucket(testBucketName).key(key).build()) + S3.getObjectTagging( + GetObjectTaggingRequest.builder().bucket(TEST_BUCKET_NAME).key(key).build()) .tagSet(); Map tagMap = tags.stream().collect(Collectors.toMap(Tag::key, Tag::value)); diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java index 046abdb61e13..2a9275045d5a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java @@ -76,7 +76,7 @@ class S3OutputStream extends PositionOutputStream { private static final Logger LOG = LoggerFactory.getLogger(S3OutputStream.class); - private static final String digestAlgorithm = "MD5"; + private static final String DIGEST_ALGORITHM = "MD5"; private static volatile ExecutorService executorService; @@ -138,7 +138,7 @@ class S3OutputStream extends PositionOutputStream { this.isChecksumEnabled = s3FileIOProperties.isChecksumEnabled(); try { this.completeMessageDigest = - isChecksumEnabled ? MessageDigest.getInstance(digestAlgorithm) : null; + isChecksumEnabled ? MessageDigest.getInstance(DIGEST_ALGORITHM) : null; } catch (NoSuchAlgorithmException e) { throw new RuntimeException( "Failed to create message digest needed for s3 checksum checks", e); @@ -220,7 +220,7 @@ private void newStream() throws IOException { currentStagingFile.deleteOnExit(); try { currentPartMessageDigest = - isChecksumEnabled ? MessageDigest.getInstance(digestAlgorithm) : null; + isChecksumEnabled ? MessageDigest.getInstance(DIGEST_ALGORITHM) : null; } catch (NoSuchAlgorithmException e) { throw new RuntimeException( "Failed to create message digest needed for s3 checksum checks.", e); diff --git a/core/src/main/java/org/apache/iceberg/avro/AvroIO.java b/core/src/main/java/org/apache/iceberg/avro/AvroIO.java index cf575fb0e8bf..ef263853190a 100644 --- a/core/src/main/java/org/apache/iceberg/avro/AvroIO.java +++ b/core/src/main/java/org/apache/iceberg/avro/AvroIO.java @@ -43,26 +43,26 @@ class AvroIO { private AvroIO() {} - private static final Class fsDataInputStreamClass = + private static final Class FS_DATA_INPUT_STREAM_CLASS = DynClasses.builder().impl("org.apache.hadoop.fs.FSDataInputStream").orNull().build(); - private static final boolean relocated = + private static final boolean RELOCATED = "org.apache.avro.file.SeekableInput".equals(SeekableInput.class.getName()); - private static final DynConstructors.Ctor avroFsInputCtor = - !relocated && fsDataInputStreamClass != null + private static final DynConstructors.Ctor AVRO_FS_INPUT_CTOR = + !RELOCATED && FS_DATA_INPUT_STREAM_CLASS != null ? DynConstructors.builder(SeekableInput.class) - .impl("org.apache.hadoop.fs.AvroFSInput", fsDataInputStreamClass, Long.TYPE) + .impl("org.apache.hadoop.fs.AvroFSInput", FS_DATA_INPUT_STREAM_CLASS, Long.TYPE) .build() : null; static SeekableInput stream(SeekableInputStream stream, long length) { if (stream instanceof DelegatingInputStream) { InputStream wrapped = ((DelegatingInputStream) stream).getDelegate(); - if (avroFsInputCtor != null - && fsDataInputStreamClass != null - && fsDataInputStreamClass.isInstance(wrapped)) { - return avroFsInputCtor.newInstance(wrapped, length); + if (AVRO_FS_INPUT_CTOR != null + && FS_DATA_INPUT_STREAM_CLASS != null + && FS_DATA_INPUT_STREAM_CLASS.isInstance(wrapped)) { + return AVRO_FS_INPUT_CTOR.newInstance(wrapped, length); } } return new AvroInputStreamAdapter(stream, length); diff --git a/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java b/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java index 08466f75fe21..98f87c65d95f 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java @@ -40,8 +40,8 @@ class StandardKeyMetadata implements NativeEncryptionKeyMetadata, IndexedRecord private static final org.apache.avro.Schema AVRO_SCHEMA_V1 = AvroSchemaUtil.convert(SCHEMA_V1, StandardKeyMetadata.class.getCanonicalName()); - private static final Map schemaVersions = ImmutableMap.of(V1, SCHEMA_V1); - private static final Map avroSchemaVersions = + private static final Map SCHEMA_VERSIONS = ImmutableMap.of(V1, SCHEMA_V1); + private static final Map AVRO_SCHEMA_VERSIONS = ImmutableMap.of(V1, AVRO_SCHEMA_V1); private static final KeyMetadataEncoder KEY_METADATA_ENCODER = new KeyMetadataEncoder(V1); @@ -66,11 +66,11 @@ private StandardKeyMetadata(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { } static Map supportedSchemaVersions() { - return schemaVersions; + return SCHEMA_VERSIONS; } static Map supportedAvroSchemaVersions() { - return avroSchemaVersions; + return AVRO_SCHEMA_VERSIONS; } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index fadaeb079330..8125e064f0d0 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -37,7 +37,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestMetadataTableFilters extends TestBase { - private static final Set aggFileTables = + private static final Set AGG_FILE_TABLES = Sets.newHashSet( MetadataTableType.ALL_DATA_FILES, MetadataTableType.ALL_DATA_FILES, @@ -149,7 +149,7 @@ private int expectedScanTaskCount(int partitions) { } private boolean isAggFileTable(MetadataTableType tableType) { - return aggFileTables.contains(tableType); + return AGG_FILE_TABLES.contains(tableType); } private String partitionColumn(String colName) { 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 6512b2499071..2b342936fdd9 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/TestHadoopCatalog.java @@ -58,7 +58,7 @@ import org.junit.jupiter.params.provider.ValueSource; public class TestHadoopCatalog extends HadoopTableTestBase { - private static final ImmutableMap meta = ImmutableMap.of(); + private static final ImmutableMap META = ImmutableMap.of(); @ParameterizedTest @ValueSource(ints = {1, 2}) @@ -337,7 +337,7 @@ public void testCreateNamespace() throws Exception { TableIdentifier tbl1 = TableIdentifier.of("db", "ns1", "ns2", "metadata"); TableIdentifier tbl2 = TableIdentifier.of("db", "ns2", "ns3", "tbl2"); - Lists.newArrayList(tbl1, tbl2).forEach(t -> catalog.createNamespace(t.namespace(), meta)); + Lists.newArrayList(tbl1, tbl2).forEach(t -> catalog.createNamespace(t.namespace(), META)); String metaLocation1 = warehouseLocation + "/" + "db/ns1/ns2"; FileSystem fs1 = Util.getFs(new Path(metaLocation1), catalog.getConf()); diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java index 574acf15cb9b..2f4a55b6fdc3 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilter.java @@ -97,7 +97,7 @@ public static List parameters() { return Arrays.asList(FileFormat.PARQUET, FileFormat.ORC); } - private static final Types.StructType structFieldType = + private static final Types.StructType STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(8, "int_field", IntegerType.get())); private static final Schema SCHEMA = @@ -108,7 +108,7 @@ public static List parameters() { optional(4, "all_nulls", DoubleType.get()), optional(5, "some_nulls", StringType.get()), optional(6, "no_nulls", StringType.get()), - optional(7, "struct_not_null", structFieldType), + optional(7, "struct_not_null", STRUCT_FIELD_TYPE), optional(9, "not_in_file", FloatType.get()), optional(10, "str", StringType.get()), optional( @@ -120,7 +120,7 @@ public static List parameters() { optional(16, "no_nans", DoubleType.get()), optional(17, "some_double_nans", DoubleType.get())); - private static final Types.StructType _structFieldType = + private static final Types.StructType UNDERSCORE_STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(8, "_int_field", IntegerType.get())); private static final Schema FILE_SCHEMA = @@ -131,7 +131,7 @@ public static List parameters() { optional(4, "_all_nulls", DoubleType.get()), optional(5, "_some_nulls", StringType.get()), optional(6, "_no_nulls", StringType.get()), - optional(7, "_struct_not_null", _structFieldType), + optional(7, "_struct_not_null", UNDERSCORE_STRUCT_FIELD_TYPE), optional(10, "_str", StringType.get()), optional(14, "_all_nans", Types.DoubleType.get()), optional(15, "_some_nans", FloatType.get()), @@ -202,7 +202,7 @@ public void createOrcInputFile() throws IOException { "_some_double_nans", (i % 10 == 0) ? Double.NaN : 2D); // includes some nan values record.setField("_no_nans", 3D); // optional, but always non-nan - GenericRecord structNotNull = GenericRecord.create(_structFieldType); + GenericRecord structNotNull = GenericRecord.create(UNDERSCORE_STRUCT_FIELD_TYPE); structNotNull.setField("_int_field", INT_MIN_VALUE + i); record.setField("_struct_not_null", structNotNull); // struct with int @@ -225,7 +225,7 @@ private void createParquetInputFile() throws IOException { assertThat(parquetFile.delete()).isTrue(); // build struct field schema - org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); + org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); OutputFile outFile = Files.localOutput(parquetFile); try (FileAppender appender = Parquet.write(outFile).schema(FILE_SCHEMA).build()) { diff --git a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java index 75b19554ef2a..19ae28a44045 100644 --- a/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java +++ b/data/src/test/java/org/apache/iceberg/data/TestMetricsRowGroupFilterTypes.java @@ -126,16 +126,16 @@ public class TestMetricsRowGroupFilterTypes { private static MessageType parquetSchema = null; private static BlockMetaData rowGroupMetadata = null; - private static final UUID uuid = UUID.randomUUID(); - private static final LocalDate date = + private static final UUID UUID_VALUE = UUID.randomUUID(); + private static final LocalDate DATE = LocalDate.parse("2018-06-29", DateTimeFormatter.ISO_LOCAL_DATE); - private static final LocalTime time = + private static final LocalTime TIME = LocalTime.parse("10:02:34.000000", DateTimeFormatter.ISO_LOCAL_TIME); - private static final OffsetDateTime timestamptz = + private static final OffsetDateTime TIMESTAMPTZ = OffsetDateTime.parse("2018-06-29T10:02:34.000000+00:00", DateTimeFormatter.ISO_DATE_TIME); - private static final LocalDateTime timestamp = + private static final LocalDateTime TIMESTAMP = LocalDateTime.parse("2018-06-29T10:02:34.000000", DateTimeFormatter.ISO_LOCAL_DATE_TIME); - private static final byte[] fixed = "abcd".getBytes(StandardCharsets.UTF_8); + private static final byte[] FIXED = "abcd".getBytes(StandardCharsets.UTF_8); @BeforeEach public void createInputFile() throws IOException { @@ -148,15 +148,16 @@ public void createInputFile() throws IOException { record.setField("_long", 5_000_000_000L + i); record.setField("_float", ((float) (100 - i)) / 100F + 1.0F); // 2.0f, 1.99f, 1.98f, ... record.setField("_double", ((double) i) / 100.0D + 2.0D); // 2.0d, 2.01d, 2.02d, ... - record.setField("_date", date); - record.setField("_time", time); - record.setField("_timestamp", timestamp); - record.setField("_timestamptz", timestamptz); + record.setField("_date", DATE); + record.setField("_time", TIME); + record.setField("_timestamp", TIMESTAMP); + record.setField("_timestamptz", TIMESTAMPTZ); record.setField("_string", "tapir"); - // record.setField("_uuid", uuid); // Disable writing UUID value as GenericParquetWriter does + // record.setField("_uuid", UUID_VALUE); // Disable writing UUID value as GenericParquetWriter + // does // not handle UUID type // correctly; Also UUID tests are disabled for both ORC and Parquet anyway - record.setField("_fixed", fixed); + record.setField("_fixed", FIXED); record.setField("_binary", ByteBuffer.wrap("xyz".getBytes(StandardCharsets.UTF_8))); record.setField("_int_decimal", new BigDecimal("77.77")); record.setField("_long_decimal", new BigDecimal("88.88")); @@ -254,7 +255,7 @@ public static Object[][] parameters() { "2018-06-29T10:02:34.000000-07:00" }, {FileFormat.PARQUET, "string", "tapir", "monthly"}, - // { FileFormat.PARQUET, "uuid", uuid, UUID.randomUUID() }, // not supported yet + // { FileFormat.PARQUET, "uuid", UUID_VALUE, UUID.randomUUID() }, // not supported yet { FileFormat.PARQUET, "fixed", @@ -286,7 +287,7 @@ public static Object[][] parameters() { }, {FileFormat.ORC, "string", "tapir", "monthly"}, // uuid, fixed and binary types not supported yet - // { FileFormat.ORC, "uuid", uuid, UUID.randomUUID() }, + // { FileFormat.ORC, "uuid", UUID_VALUE, UUID.randomUUID() }, // { FileFormat.ORC, "fixed", "abcd".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, // 2, 3 } }, // { FileFormat.ORC, "binary", "xyz".getBytes(StandardCharsets.UTF_8), new byte[] { 0, 1, diff --git a/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java b/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java index c6a5ed9f6d86..4b0a10830221 100644 --- a/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java +++ b/data/src/test/java/org/apache/iceberg/data/parquet/TestParquetEncryptionWithWriteSupport.java @@ -53,8 +53,8 @@ import org.junit.jupiter.api.Test; public class TestParquetEncryptionWithWriteSupport extends DataTest { - private static final ByteBuffer fileDek = ByteBuffer.allocate(16); - private static final ByteBuffer aadPrefix = ByteBuffer.allocate(16); + private static final ByteBuffer FILE_DEK = ByteBuffer.allocate(16); + private static final ByteBuffer AAD_PREFIX = ByteBuffer.allocate(16); @Override protected void writeAndValidate(Schema schema) throws IOException { @@ -64,14 +64,14 @@ protected void writeAndValidate(Schema schema) throws IOException { assertThat(testFile.delete()).isTrue(); SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); try (FileAppender appender = Parquet.write(Files.localOutput(testFile)) .schema(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .createWriterFunc(GenericParquetWriter::buildWriter) .build()) { appender.addAll(expected); @@ -92,8 +92,8 @@ protected void writeAndValidate(Schema schema) throws IOException { try (CloseableIterable reader = Parquet.read(Files.localInput(testFile)) .project(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { rows = Lists.newArrayList(reader); @@ -107,8 +107,8 @@ protected void writeAndValidate(Schema schema) throws IOException { try (CloseableIterable reader = Parquet.read(Files.localInput(testFile)) .project(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .reuseContainers() .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { @@ -134,10 +134,12 @@ public void testTwoLevelList() throws IOException { assertThat(testFile.delete()).isTrue(); SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); FileEncryptionProperties fileEncryptionProperties = - FileEncryptionProperties.builder(fileDek.array()).withAADPrefix(aadPrefix.array()).build(); + FileEncryptionProperties.builder(FILE_DEK.array()) + .withAADPrefix(AAD_PREFIX.array()) + .build(); ParquetWriter writer = AvroParquetWriter.builder(new Path(testFile.toURI())) @@ -164,8 +166,8 @@ public void testTwoLevelList() throws IOException { try (CloseableIterable reader = Parquet.read(Files.localInput(testFile)) .project(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .reuseContainers() .createReaderFunc(fileSchema -> GenericParquetReaders.buildReader(schema, fileSchema)) .build()) { 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 74105458dbcc..e4f6c028bc8e 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java +++ b/data/src/test/java/org/apache/iceberg/io/TestWriterMetrics.java @@ -73,10 +73,10 @@ public abstract class TestWriterMetrics { // create a schema with all supported fields protected static final Schema SCHEMA = new Schema(ID_FIELD, DATA_FIELD, STRUCT_FIELD); - protected static final SortOrder sortOrder = + protected static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").asc("structField.longValue").build(); - protected static final Map properties = + protected static final Map PROPERTIES = ImmutableMap.of(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none"); @TempDir private File tempDir; @@ -107,7 +107,7 @@ public void setupTable() throws Exception { this.table = TestTables.create( - tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), sortOrder, FORMAT_V2); + tableDir, "test", SCHEMA, PartitionSpec.unpartitioned(), SORT_ORDER, FORMAT_V2); table.updateProperties().set(TableProperties.DEFAULT_WRITE_METRICS_MODE, "none").commit(); this.fileFactory = OutputFileFactory.builderFor(table, 1, 1).format(fileFormat).build(); 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 945e89670de8..01a998c65efe 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 @@ -70,9 +70,9 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { private static final String DELTA_SOURCE_VALUE = "delta"; private static final String ORIGINAL_LOCATION_PROP = "original_location"; private static final String NAMESPACE = "delta_conversion_test"; - private static final String defaultSparkCatalog = "spark_catalog"; - private static final String icebergCatalogName = "iceberg_hive"; - private static final Map config = + private static final String DEFAULT_SPARK_CATALOG = "spark_catalog"; + private static final String ICEBERG_CATALOG_NAME = "iceberg_hive"; + private static final Map CONFIG = ImmutableMap.of( "type", "hive", "default-namespace", "default", @@ -87,8 +87,8 @@ public class TestSnapshotDeltaLakeTable extends SparkDeltaLakeSnapshotTestBase { @TempDir private File tempB; public TestSnapshotDeltaLakeTable() { - super(icebergCatalogName, SparkCatalog.class.getName(), config); - spark.conf().set("spark.sql.catalog." + defaultSparkCatalog, DeltaCatalog.class.getName()); + super(ICEBERG_CATALOG_NAME, SparkCatalog.class.getName(), CONFIG); + spark.conf().set("spark.sql.catalog." + DEFAULT_SPARK_CATALOG, DeltaCatalog.class.getName()); } @BeforeAll @@ -152,14 +152,14 @@ public static void afterClass() { @Test public void testBasicSnapshotPartitioned() { - String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); + String partitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "partitioned_table"); String partitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, partitionedIdentifier, partitionedLocation, "id"); spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_partitioned_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_partitioned_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) @@ -173,14 +173,14 @@ public void testBasicSnapshotPartitioned() { @Test public void testBasicSnapshotUnpartitioned() { - String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); + String unpartitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "unpartitioned_table"); String unpartitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); spark.sql("DELETE FROM " + unpartitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + unpartitionedIdentifier + " SET id=3 WHERE id=1"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_unpartitioned_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_unpartitioned_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) @@ -194,7 +194,7 @@ public void testBasicSnapshotUnpartitioned() { @Test public void testSnapshotWithNewLocation() { - String partitionedIdentifier = destName(defaultSparkCatalog, "partitioned_table"); + String partitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "partitioned_table"); String partitionedLocation = tempA.toURI().toString(); String newIcebergTableLocation = tempB.toURI().toString(); @@ -202,7 +202,7 @@ public void testSnapshotWithNewLocation() { spark.sql("DELETE FROM " + partitionedIdentifier + " WHERE id=3"); spark.sql("UPDATE " + partitionedIdentifier + " SET id=3 WHERE id=1"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_new_table_location_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_new_table_location_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, partitionedLocation) @@ -217,7 +217,7 @@ public void testSnapshotWithNewLocation() { @Test public void testSnapshotWithAdditionalProperties() { - String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); + String unpartitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "unpartitioned_table"); String unpartitionedLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, unpartitionedIdentifier, unpartitionedLocation); @@ -230,7 +230,8 @@ public void testSnapshotWithAdditionalProperties() { + unpartitionedIdentifier + " SET TBLPROPERTIES ('foo'='bar', 'test0'='test0')"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_additional_properties_table"); + String newTableIdentifier = + destName(ICEBERG_CATALOG_NAME, "iceberg_additional_properties_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, unpartitionedLocation) @@ -255,8 +256,9 @@ public void testSnapshotWithAdditionalProperties() { @Test public void testSnapshotTableWithExternalDataFiles() { - String unpartitionedIdentifier = destName(defaultSparkCatalog, "unpartitioned_table"); - String externalDataFilesIdentifier = destName(defaultSparkCatalog, "external_data_files_table"); + String unpartitionedIdentifier = destName(DEFAULT_SPARK_CATALOG, "unpartitioned_table"); + String externalDataFilesIdentifier = + destName(DEFAULT_SPARK_CATALOG, "external_data_files_table"); String unpartitionedLocation = tempA.toURI().toString(); String externalDataFilesTableLocation = tempB.toURI().toString(); @@ -269,7 +271,7 @@ public void testSnapshotTableWithExternalDataFiles() { // are not at the same location as the table. addExternalDatafiles(externalDataFilesTableLocation, unpartitionedLocation); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_external_data_files_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_external_data_files_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, externalDataFilesTableLocation) @@ -283,7 +285,7 @@ public void testSnapshotTableWithExternalDataFiles() { @Test public void testSnapshotSupportedTypes() { - String typeTestIdentifier = destName(defaultSparkCatalog, "type_test_table"); + String typeTestIdentifier = destName(DEFAULT_SPARK_CATALOG, "type_test_table"); String typeTestTableLocation = tempA.toURI().toString(); writeDeltaTable( @@ -294,7 +296,7 @@ public void testSnapshotSupportedTypes() { "timestampStrCol", "booleanCol", "longCol"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_type_test_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_type_test_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, typeTestTableLocation) @@ -308,7 +310,7 @@ public void testSnapshotSupportedTypes() { @Test public void testSnapshotVacuumTable() throws IOException { - String vacuumTestIdentifier = destName(defaultSparkCatalog, "vacuum_test_table"); + String vacuumTestIdentifier = destName(DEFAULT_SPARK_CATALOG, "vacuum_test_table"); String vacuumTestTableLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, vacuumTestIdentifier, vacuumTestTableLocation); @@ -330,7 +332,7 @@ public void testSnapshotVacuumTable() throws IOException { assertThat(deleteResult).isTrue(); spark.sql("VACUUM " + vacuumTestIdentifier + " RETAIN 0 HOURS"); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_vacuum_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_vacuum_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, vacuumTestTableLocation) @@ -343,7 +345,7 @@ public void testSnapshotVacuumTable() throws IOException { @Test public void testSnapshotLogCleanTable() throws IOException { - String logCleanTestIdentifier = destName(defaultSparkCatalog, "log_clean_test_table"); + String logCleanTestIdentifier = destName(DEFAULT_SPARK_CATALOG, "log_clean_test_table"); String logCleanTestTableLocation = tempA.toURI().toString(); writeDeltaTable(nestedDataFrame, logCleanTestIdentifier, logCleanTestTableLocation, "id"); @@ -364,7 +366,7 @@ public void testSnapshotLogCleanTable() throws IOException { logCleanTestTableLocation.concat("/_delta_log/00000000000000000000.json")))); assertThat(deleteResult).isTrue(); - String newTableIdentifier = destName(icebergCatalogName, "iceberg_log_clean_table"); + String newTableIdentifier = destName(ICEBERG_CATALOG_NAME, "iceberg_log_clean_table"); SnapshotDeltaLakeTable.Result result = DeltaLakeToIcebergMigrationSparkIntegration.snapshotDeltaLakeTable( spark, newTableIdentifier, logCleanTestTableLocation) @@ -477,7 +479,7 @@ private Table getIcebergTable(String icebergTableIdentifier) { } private String destName(String catalogName, String dest) { - if (catalogName.equals(defaultSparkCatalog)) { + if (catalogName.equals(DEFAULT_SPARK_CATALOG)) { return NAMESPACE + "." + catalogName + "_" + dest; } return catalogName + "." + NAMESPACE + "." + catalogName + "_" + dest; diff --git a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java index 8699eb3b5d26..f43ce39d1c5f 100644 --- a/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java +++ b/delta-lake/src/main/java/org/apache/iceberg/delta/DeltaLakeToIcebergMigrationActionsProvider.java @@ -46,13 +46,13 @@ static DeltaLakeToIcebergMigrationActionsProvider defaultActions() { class DefaultDeltaLakeToIcebergMigrationActions implements DeltaLakeToIcebergMigrationActionsProvider { - private static final DefaultDeltaLakeToIcebergMigrationActions defaultMigrationActions = + private static final DefaultDeltaLakeToIcebergMigrationActions DEFAULT_MIGRATION_ACTIONS = new DefaultDeltaLakeToIcebergMigrationActions(); private DefaultDeltaLakeToIcebergMigrationActions() {} static DefaultDeltaLakeToIcebergMigrationActions defaultMigrationActions() { - return defaultMigrationActions; + return DEFAULT_MIGRATION_ACTIONS; } } } diff --git a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java index 6d99d64470f8..817310203109 100644 --- a/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java +++ b/delta-lake/src/test/java/org/apache/iceberg/delta/TestDeltaLakeTypeToType.java @@ -38,13 +38,13 @@ import org.junit.jupiter.api.Test; public class TestDeltaLakeTypeToType { - private static final String optionalBooleanType = "testNullableBoolType"; - private static final String requiredBinaryType = "testRequiredBinaryType"; - private static final String doubleArrayType = "testNullableArrayType"; - private static final String structArrayType = "testStructArrayType"; - private static final String innerAtomicSchema = "testInnerAtomicSchema"; - private static final String stringLongMapType = "testStringLongMap"; - private static final String nullType = "testNullType"; + private static final String OPTIONAL_BOOLEAN_TYPE = "testNullableBoolType"; + private static final String REQUIRED_BINARY_TYPE = "testRequiredBinaryType"; + private static final String DOUBLE_ARRAY_TYPE = "testNullableArrayType"; + private static final String STRUCT_ARRAY_TYPE = "testStructArrayType"; + private static final String INNER_ATOMIC_SCHEMA = "testInnerAtomicSchema"; + private static final String STRING_LONG_MAP_TYPE = "testStringLongMap"; + private static final String NULL_TYPE = "testNullType"; private StructType deltaAtomicSchema; private StructType deltaNestedSchema; private StructType deltaShallowNullTypeSchema; @@ -54,20 +54,20 @@ public class TestDeltaLakeTypeToType { public void constructDeltaLakeSchema() { deltaAtomicSchema = new StructType() - .add(optionalBooleanType, new BooleanType()) - .add(requiredBinaryType, new BinaryType(), false); + .add(OPTIONAL_BOOLEAN_TYPE, new BooleanType()) + .add(REQUIRED_BINARY_TYPE, new BinaryType(), false); deltaNestedSchema = new StructType() - .add(innerAtomicSchema, deltaAtomicSchema) - .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) - .add(structArrayType, new ArrayType(deltaAtomicSchema, true), false) - .add(stringLongMapType, new MapType(new StringType(), new LongType(), false), false); + .add(INNER_ATOMIC_SCHEMA, deltaAtomicSchema) + .add(DOUBLE_ARRAY_TYPE, new ArrayType(new DoubleType(), true), false) + .add(STRUCT_ARRAY_TYPE, new ArrayType(deltaAtomicSchema, true), false) + .add(STRING_LONG_MAP_TYPE, new MapType(new StringType(), new LongType(), false), false); deltaNullTypeSchema = new StructType() - .add(innerAtomicSchema, deltaAtomicSchema) - .add(doubleArrayType, new ArrayType(new DoubleType(), true), false) - .add(stringLongMapType, new MapType(new NullType(), new LongType(), false), false); - deltaShallowNullTypeSchema = new StructType().add(nullType, new NullType(), false); + .add(INNER_ATOMIC_SCHEMA, deltaAtomicSchema) + .add(DOUBLE_ARRAY_TYPE, new ArrayType(new DoubleType(), true), false) + .add(STRING_LONG_MAP_TYPE, new MapType(new NullType(), new LongType(), false), false); + deltaShallowNullTypeSchema = new StructType().add(NULL_TYPE, new NullType(), false); } @Test @@ -77,10 +77,11 @@ public void testAtomicTypeConversion() { deltaAtomicSchema, new DeltaLakeTypeToType(deltaAtomicSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - assertThat(convertedSchema.findType(optionalBooleanType)).isInstanceOf(Types.BooleanType.class); - assertThat(convertedSchema.findField(optionalBooleanType).isOptional()).isTrue(); - assertThat(convertedSchema.findType(requiredBinaryType)).isInstanceOf(Types.BinaryType.class); - assertThat(convertedSchema.findField(requiredBinaryType).isRequired()).isTrue(); + assertThat(convertedSchema.findType(OPTIONAL_BOOLEAN_TYPE)) + .isInstanceOf(Types.BooleanType.class); + assertThat(convertedSchema.findField(OPTIONAL_BOOLEAN_TYPE).isOptional()).isTrue(); + assertThat(convertedSchema.findType(REQUIRED_BINARY_TYPE)).isInstanceOf(Types.BinaryType.class); + assertThat(convertedSchema.findField(REQUIRED_BINARY_TYPE).isRequired()).isTrue(); } @Test @@ -90,72 +91,74 @@ public void testNestedTypeConversion() { deltaNestedSchema, new DeltaLakeTypeToType(deltaNestedSchema)); Schema convertedSchema = new Schema(converted.asNestedType().asStructType().fields()); - assertThat(convertedSchema.findType(innerAtomicSchema)).isInstanceOf(Types.StructType.class); - assertThat(convertedSchema.findField(innerAtomicSchema).isOptional()).isTrue(); + assertThat(convertedSchema.findType(INNER_ATOMIC_SCHEMA)).isInstanceOf(Types.StructType.class); + assertThat(convertedSchema.findField(INNER_ATOMIC_SCHEMA).isOptional()).isTrue(); assertThat( convertedSchema - .findType(innerAtomicSchema) + .findType(INNER_ATOMIC_SCHEMA) .asStructType() - .fieldType(optionalBooleanType)) + .fieldType(OPTIONAL_BOOLEAN_TYPE)) .isInstanceOf(Types.BooleanType.class); assertThat( convertedSchema - .findType(innerAtomicSchema) + .findType(INNER_ATOMIC_SCHEMA) .asStructType() - .fieldType(requiredBinaryType)) + .fieldType(REQUIRED_BINARY_TYPE)) .isInstanceOf(Types.BinaryType.class); assertThat( convertedSchema - .findType(innerAtomicSchema) + .findType(INNER_ATOMIC_SCHEMA) .asStructType() - .field(requiredBinaryType) + .field(REQUIRED_BINARY_TYPE) .isRequired()) .isTrue(); - assertThat(convertedSchema.findType(stringLongMapType)).isInstanceOf(Types.MapType.class); - assertThat(convertedSchema.findType(stringLongMapType).asMapType().keyType()) + assertThat(convertedSchema.findType(STRING_LONG_MAP_TYPE)).isInstanceOf(Types.MapType.class); + assertThat(convertedSchema.findType(STRING_LONG_MAP_TYPE).asMapType().keyType()) .isInstanceOf(Types.StringType.class); - assertThat(convertedSchema.findType(stringLongMapType).asMapType().valueType()) + assertThat(convertedSchema.findType(STRING_LONG_MAP_TYPE).asMapType().valueType()) .isInstanceOf(Types.LongType.class); - assertThat(convertedSchema.findType(doubleArrayType)).isInstanceOf(Types.ListType.class); - assertThat(convertedSchema.findField(doubleArrayType).isRequired()).isTrue(); - assertThat(convertedSchema.findType(doubleArrayType).asListType().isElementOptional()).isTrue(); - assertThat(convertedSchema.findType(structArrayType)).isInstanceOf(Types.ListType.class); - assertThat(convertedSchema.findField(structArrayType).isRequired()).isTrue(); - assertThat(convertedSchema.findType(structArrayType).asListType().isElementOptional()).isTrue(); - assertThat(convertedSchema.findType(structArrayType).asListType().elementType()) + assertThat(convertedSchema.findType(DOUBLE_ARRAY_TYPE)).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField(DOUBLE_ARRAY_TYPE).isRequired()).isTrue(); + assertThat(convertedSchema.findType(DOUBLE_ARRAY_TYPE).asListType().isElementOptional()) + .isTrue(); + assertThat(convertedSchema.findType(STRUCT_ARRAY_TYPE)).isInstanceOf(Types.ListType.class); + assertThat(convertedSchema.findField(STRUCT_ARRAY_TYPE).isRequired()).isTrue(); + assertThat(convertedSchema.findType(STRUCT_ARRAY_TYPE).asListType().isElementOptional()) + .isTrue(); + assertThat(convertedSchema.findType(STRUCT_ARRAY_TYPE).asListType().elementType()) .isInstanceOf(Types.StructType.class); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .fieldType(optionalBooleanType)) + .fieldType(OPTIONAL_BOOLEAN_TYPE)) .isInstanceOf(Types.BooleanType.class); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .field(optionalBooleanType) + .field(OPTIONAL_BOOLEAN_TYPE) .isOptional()) .isTrue(); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .fieldType(requiredBinaryType)) + .fieldType(REQUIRED_BINARY_TYPE)) .isInstanceOf(Types.BinaryType.class); assertThat( convertedSchema - .findType(structArrayType) + .findType(STRUCT_ARRAY_TYPE) .asListType() .elementType() .asStructType() - .field(requiredBinaryType) + .field(REQUIRED_BINARY_TYPE) .isRequired()) .isTrue(); } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java index a1d4fb16b7e7..de6c8a0f6e5a 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveVersion.java @@ -30,18 +30,18 @@ public enum HiveVersion { NOT_SUPPORTED(0); private final int order; - private static final HiveVersion current = calculate(); + private static final HiveVersion CURRENT = calculate(); HiveVersion(int order) { this.order = order; } public static HiveVersion current() { - return current; + return CURRENT; } public static boolean min(HiveVersion other) { - return current.order >= other.order; + return CURRENT.order >= other.order; } private static HiveVersion calculate() { diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java index 39cb1835c353..da28919ed647 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/HiveTableBaseTest.java @@ -55,17 +55,17 @@ public class HiveTableBaseTest { protected static HiveCatalog catalog; - static final Schema schema = + static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - static final Schema altered = + static final Schema ALTERED = new Schema( Types.StructType.of( required(1, "id", Types.LongType.get()), optional(2, "data", Types.LongType.get())) .fields()); - private static final PartitionSpec partitionSpec = builderFor(schema).identity("id").build(); + private static final PartitionSpec PARTITION_SPEC = builderFor(SCHEMA).identity("id").build(); private Path tableLocation; @@ -85,7 +85,7 @@ public static void initCatalog() { @BeforeEach public void createTestTable() { this.tableLocation = - new Path(catalog.createTable(TABLE_IDENTIFIER, schema, partitionSpec).location()); + new Path(catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PARTITION_SPEC).location()); } @AfterEach 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 6d8e9b4391c3..9ae3c97db47c 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 @@ -116,7 +116,7 @@ public void testCreate() throws TException { final Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); // Iceberg schema should match the loaded table - assertThat(icebergTable.schema().asStruct()).isEqualTo(schema.asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(SCHEMA.asStruct()); } @Test @@ -172,7 +172,7 @@ public void testDropTable() throws IOException { Table table = catalog.loadTable(TABLE_IDENTIFIER); GenericRecordBuilder recordBuilder = - new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test")); + new GenericRecordBuilder(AvroSchemaUtil.convert(SCHEMA, "test")); List records = Lists.newArrayList( recordBuilder.set("id", 1L).build(), @@ -181,7 +181,7 @@ public void testDropTable() throws IOException { String location1 = table.location().replace("file:", "") + "/data/file1.avro"; try (FileAppender writer = - Avro.write(Files.localOutput(location1)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(location1)).schema(SCHEMA).named("test").build()) { for (GenericData.Record rec : records) { writer.add(rec); } @@ -189,7 +189,7 @@ public void testDropTable() throws IOException { String location2 = table.location().replace("file:", "") + "/data/file2.avro"; try (FileAppender writer = - Avro.write(Files.localOutput(location2)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(location2)).schema(SCHEMA).named("test").build()) { for (GenericData.Record rec : records) { writer.add(rec); } @@ -257,14 +257,14 @@ public void testExistingTableUpdate() throws TException { // Only 2 snapshotFile Should exist and no manifests should exist assertThat(metadataVersionFiles(TABLE_NAME)).hasSize(2); assertThat(manifestFiles(TABLE_NAME)).hasSize(0); - assertThat(icebergTable.schema().asStruct()).isEqualTo(altered.asStruct()); + assertThat(icebergTable.schema().asStruct()).isEqualTo(ALTERED.asStruct()); final org.apache.hadoop.hive.metastore.api.Table table = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); final List hiveColumns = table.getSd().getCols().stream().map(FieldSchema::getName).collect(Collectors.toList()); final List icebergColumns = - altered.columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); + ALTERED.columns().stream().map(Types.NestedField::name).collect(Collectors.toList()); assertThat(hiveColumns).isEqualTo(icebergColumns); } @@ -378,7 +378,7 @@ public void testHiveTableAndIcebergTableWithSameName(TableType tableType) catalog.setListAllTables(false); // reset to default. // create an iceberg table with the same name - assertThatThrownBy(() -> catalog.createTable(identifier, schema, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> catalog.createTable(identifier, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(NoSuchIcebergTableException.class) .hasMessageStartingWith(String.format("Not an iceberg table: hive.%s", identifier)); @@ -444,7 +444,7 @@ public void testNonDefaultDatabaseLocation() throws IOException, TException { assertThat("file:" + nonDefaultLocation.getPath()).isEqualTo(namespaceMeta.get("location")); TableIdentifier tableIdentifier = TableIdentifier.of(namespace, TABLE_NAME); - catalog.createTable(tableIdentifier, schema); + catalog.createTable(tableIdentifier, SCHEMA); // Let's check the location loaded through the catalog Table table = catalog.loadTable(tableIdentifier); @@ -492,7 +492,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio TableIdentifier identifier = TableIdentifier.of(DB_NAME, "table1"); Table table = hadoopCatalog.createTable( - identifier, schema, PartitionSpec.unpartitioned(), Maps.newHashMap()); + identifier, SCHEMA, PartitionSpec.unpartitioned(), Maps.newHashMap()); // insert some data String file1Location = appendData(table, "file1"); List tasks = Lists.newArrayList(table.newScan().planFiles()); @@ -534,7 +534,7 @@ public void testRegisterHadoopTableToHiveCatalog() throws IOException, TExceptio private String appendData(Table table, String fileName) throws IOException { GenericRecordBuilder recordBuilder = - new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test")); + new GenericRecordBuilder(AvroSchemaUtil.convert(SCHEMA, "test")); List records = Lists.newArrayList( recordBuilder.set("id", 1L).build(), @@ -543,7 +543,7 @@ private String appendData(Table table, String fileName) throws IOException { String fileLocation = table.location().replace("file:", "") + "/data/" + fileName + ".avro"; try (FileAppender writer = - Avro.write(Files.localOutput(fileLocation)).schema(schema).named("test").build()) { + Avro.write(Files.localOutput(fileLocation)).schema(SCHEMA).named("test").build()) { for (GenericData.Record rec : records) { writer.add(rec); } @@ -589,7 +589,7 @@ public void testEngineHiveEnabledDefault() throws TException { // Unset in hive-conf catalog.getConf().unset(ConfigProperties.ENGINE_HIVE_ENABLED); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); @@ -604,7 +604,7 @@ public void testEngineHiveEnabledConfig() throws TException { // Enable by hive-conf catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "true"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned()); org.apache.hadoop.hive.metastore.api.Table hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); @@ -615,7 +615,7 @@ public void testEngineHiveEnabledConfig() throws TException { // Disable by hive-conf catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "false"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned()); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned()); hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); assertHiveEnabled(hmsTable, false); @@ -631,7 +631,7 @@ public void testEngineHiveEnabledTableProperty() throws TException { tableProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "true"); catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "false"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned(), tableProperties); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned(), tableProperties); org.apache.hadoop.hive.metastore.api.Table hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); @@ -643,7 +643,7 @@ public void testEngineHiveEnabledTableProperty() throws TException { tableProperties.put(TableProperties.ENGINE_HIVE_ENABLED, "false"); catalog.getConf().set(ConfigProperties.ENGINE_HIVE_ENABLED, "true"); - catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned(), tableProperties); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned(), tableProperties); hmsTable = HIVE_METASTORE_EXTENSION.metastoreClient().getTable(DB_NAME, TABLE_NAME); assertHiveEnabled(hmsTable, false); 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 ccb8fc3a88f7..9249deb7598e 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 @@ -93,7 +93,7 @@ * Run all the tests from abstract of {@link CatalogTests} with few specific tests related to HIVE. */ public class TestHiveCatalog extends CatalogTests { - private static final ImmutableMap meta = + private static final ImmutableMap META = ImmutableMap.of( "owner", "apache", "group", "iceberg", @@ -411,7 +411,7 @@ public void testCreateTableCustomSortOrder() throws Exception { @Test public void testDatabaseAndNamespaceWithLocation() throws Exception { Namespace namespace1 = Namespace.of("noLocation"); - catalog.createNamespace(namespace1, meta); + catalog.createNamespace(namespace1, META); Database database1 = HIVE_METASTORE_EXTENSION.metastoreClient().getDatabase(namespace1.toString()); @@ -430,7 +430,7 @@ public void testDatabaseAndNamespaceWithLocation() throws Exception { hiveLocalDir = hiveLocalDir.substring(0, hiveLocalDir.length() - 1); ImmutableMap newMeta = ImmutableMap.builder() - .putAll(meta) + .putAll(META) .put("location", hiveLocalDir) .buildOrThrow(); Namespace namespace2 = Namespace.of("haveLocation"); @@ -527,12 +527,12 @@ private void createNamespaceAndVerifyOwnership( public void testLoadNamespaceMeta() throws TException { Namespace namespace = Namespace.of("dbname_load"); - catalog.createNamespace(namespace, meta); + catalog.createNamespace(namespace, META); Map nameMata = catalog.loadNamespaceMetadata(namespace); assertThat(nameMata).containsEntry("owner", "apache"); assertThat(nameMata).containsEntry("group", "iceberg"); - assertThat(catalog.convertToDatabase(namespace, meta).getLocationUri()) + assertThat(catalog.convertToDatabase(namespace, META).getLocationUri()) .as("There no same location for db and namespace") .isEqualTo(nameMata.get("location")); } @@ -541,7 +541,7 @@ public void testLoadNamespaceMeta() throws TException { public void testNamespaceExists() throws TException { Namespace namespace = Namespace.of("dbname_exists"); - catalog.createNamespace(namespace, meta); + catalog.createNamespace(namespace, META); assertThat(catalog.namespaceExists(namespace)).as("Should true to namespace exist").isTrue(); assertThat(catalog.namespaceExists(Namespace.of("db2", "db2", "ns2"))) @@ -861,7 +861,7 @@ public void dropNamespace() { TableIdentifier identifier = TableIdentifier.of(namespace, "table"); Schema schema = getTestSchema(); - catalog.createNamespace(namespace, meta); + catalog.createNamespace(namespace, META); catalog.createTable(identifier, schema); Map nameMata = catalog.loadNamespaceMetadata(namespace); assertThat(nameMata).containsEntry("owner", "apache"); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java index 61af2c7e79f3..d12a8503313b 100644 --- a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveCommitLocks.java @@ -86,7 +86,7 @@ public class TestHiveCommitLocks { private static HiveClientPool spyClientPool = null; private static CachedClientPool spyCachedClientPool = null; private static Configuration overriddenHiveConf; - private static final AtomicReference spyClientRef = new AtomicReference<>(); + private static final AtomicReference SPY_CLIENT_REF = new AtomicReference<>(); private static IMetaStoreClient spyClient = null; HiveTableOperations ops = null; TableMetadata metadataV1 = null; @@ -100,9 +100,9 @@ public class TestHiveCommitLocks { private static final String DB_NAME = "hivedb"; private static final String TABLE_NAME = "tbl"; - private static final Schema schema = + private static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - private static final PartitionSpec partitionSpec = builderFor(schema).identity("id").build(); + private static final PartitionSpec PARTITION_SPEC = builderFor(SCHEMA).identity("id").build(); static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME); @RegisterExtension @@ -143,8 +143,8 @@ public static void initCatalog() throws Exception { // cannot spy on RetryingHiveMetastoreClient as it is a proxy IMetaStoreClient client = spy(new HiveMetaStoreClient(HIVE_METASTORE_EXTENSION.hiveConf())); - spyClientRef.set(client); - return spyClientRef.get(); + SPY_CLIENT_REF.set(client); + return SPY_CLIENT_REF.get(); }); spyClientPool.run(IMetaStoreClient::isLocalMetaStore); // To ensure new client is created. @@ -153,15 +153,15 @@ public static void initCatalog() throws Exception { spy(new CachedClientPool(HIVE_METASTORE_EXTENSION.hiveConf(), Collections.emptyMap())); when(spyCachedClientPool.clientPool()).thenAnswer(invocation -> spyClientPool); - assertThat(spyClientRef.get()).isNotNull(); + assertThat(SPY_CLIENT_REF.get()).isNotNull(); - spyClient = spyClientRef.get(); + spyClient = SPY_CLIENT_REF.get(); } @BeforeEach public void before() throws Exception { this.tableLocation = - new Path(catalog.createTable(TABLE_IDENTIFIER, schema, partitionSpec).location()); + new Path(catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PARTITION_SPEC).location()); Table table = catalog.loadTable(TABLE_IDENTIFIER); ops = (HiveTableOperations) ((HasTableOperations) table).operations(); String dbName = TABLE_IDENTIFIER.namespace().level(0); 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 b3bbde460667..136c96934189 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 @@ -302,7 +302,7 @@ public void testThriftExceptionConcurrentCommit() throws TException, Interrupted @Test public void testInvalidObjectException() { TableIdentifier badTi = TableIdentifier.of(DB_NAME, "`tbl`"); - assertThatThrownBy(() -> catalog.createTable(badTi, schema, PartitionSpec.unpartitioned())) + assertThatThrownBy(() -> catalog.createTable(badTi, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(ValidationException.class) .hasMessage(String.format("Invalid Hive object for %s.%s", DB_NAME, "`tbl`")); } @@ -310,7 +310,7 @@ public void testInvalidObjectException() { @Test public void testAlreadyExistsException() { assertThatThrownBy( - () -> catalog.createTable(TABLE_IDENTIFIER, schema, PartitionSpec.unpartitioned())) + () -> catalog.createTable(TABLE_IDENTIFIER, SCHEMA, PartitionSpec.unpartitioned())) .isInstanceOf(AlreadyExistsException.class) .hasMessage(String.format("Table already exists: %s.%s", DB_NAME, TABLE_NAME)); } diff --git a/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java b/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java index 6558f79b93df..b6dd984a5843 100644 --- a/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java +++ b/mr/src/main/java/org/apache/hadoop/hive/ql/exec/vector/VectorizedSupport.java @@ -34,6 +34,7 @@ public enum Support { this.lowerCaseName = name().toLowerCase(Locale.ROOT); } + @SuppressWarnings("checkstyle:ConstantName") public static final Map nameToSupportMap = Maps.newHashMap(); static { 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 f87d79b5537e..0c698aa4b2fd 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 @@ -55,15 +55,15 @@ class HiveIcebergRecordWriter extends PartitionedFanoutWriter // > map to store the active writers // Stored in concurrent map, since some executor engines can share containers - private static final Map> writers = + private static final Map> WRITERS = Maps.newConcurrentMap(); static Map removeWriters(TaskAttemptID taskAttemptID) { - return writers.remove(taskAttemptID); + return WRITERS.remove(taskAttemptID); } static Map getWriters(TaskAttemptID taskAttemptID) { - return writers.get(taskAttemptID); + return WRITERS.get(taskAttemptID); } HiveIcebergRecordWriter( @@ -80,8 +80,8 @@ static Map getWriters(TaskAttemptID taskAttempt this.io = io; this.currentKey = new PartitionKey(spec, schema); this.wrapper = new InternalRecordWrapper(schema.asStruct()); - writers.putIfAbsent(taskAttemptID, Maps.newConcurrentMap()); - writers.get(taskAttemptID).put(tableName, this); + WRITERS.putIfAbsent(taskAttemptID, Maps.newConcurrentMap()); + WRITERS.get(taskAttemptID).put(tableName, this); } @Override diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java index 919230a9fbd9..3ca39c9fec36 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergSerDe.java @@ -41,7 +41,7 @@ public class TestHiveIcebergSerDe { - private static final Schema schema = + private static final Schema SCHEMA = new Schema(required(1, "string_field", Types.StringType.get())); @TempDir private Path tmp; @@ -58,19 +58,19 @@ public void testInitialize() throws IOException, SerDeException { properties.setProperty(InputFormatConfig.CATALOG_NAME, Catalogs.ICEBERG_HADOOP_TABLE_NAME); HadoopTables tables = new HadoopTables(conf); - tables.create(schema, location.toString()); + tables.create(SCHEMA, location.toString()); HiveIcebergSerDe serDe = new HiveIcebergSerDe(); serDe.initialize(conf, properties); - assertThat(serDe.getObjectInspector()).isEqualTo(IcebergObjectInspector.create(schema)); + assertThat(serDe.getObjectInspector()).isEqualTo(IcebergObjectInspector.create(SCHEMA)); } @Test public void testDeserialize() { HiveIcebergSerDe serDe = new HiveIcebergSerDe(); - Record record = RandomGenericData.generate(schema, 1, 0).get(0); + Record record = RandomGenericData.generate(SCHEMA, 1, 0).get(0); Container container = new Container<>(); container.set(record); diff --git a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java index a7aa5126e2e2..b8a454d01f02 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java +++ b/mr/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerTimezone.java @@ -52,7 +52,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestHiveIcebergStorageHandlerTimezone { - private static final Optional> dateFormat = + private static final Optional> DATE_FORMAT = Optional.ofNullable( (ThreadLocal) DynFields.builder() @@ -61,7 +61,7 @@ public class TestHiveIcebergStorageHandlerTimezone { .buildStatic() .get()); - private static final Optional> localTimeZone = + private static final Optional> LOCAL_TIME_ZONE = Optional.ofNullable( (ThreadLocal) DynFields.builder() @@ -103,8 +103,8 @@ public void before() throws IOException { // Magic to clean cached date format and local timezone for Hive where the default timezone is // used/stored in the // cached object - dateFormat.ifPresent(ThreadLocal::remove); - localTimeZone.ifPresent(ThreadLocal::remove); + DATE_FORMAT.ifPresent(ThreadLocal::remove); + LOCAL_TIME_ZONE.ifPresent(ThreadLocal::remove); this.testTables = HiveIcebergStorageHandlerTestUtils.testTables( diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java index d6f4f68f511e..49b721d0e5f4 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestMultipleClients.java @@ -48,7 +48,7 @@ public class TestMultipleClients extends BaseTestIceberg { private static final String BRANCH = "multiple-clients-test"; - private static final Schema schema = + private static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); public TestMultipleClients() { @@ -153,12 +153,12 @@ public void testLoadNamespaceMetadata() throws NessieConflictException, NessieNo @Test public void testListTables() { - createTable(TableIdentifier.parse("foo.tbl1"), schema); + createTable(TableIdentifier.parse("foo.tbl1"), SCHEMA); assertThat(catalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder(TableIdentifier.parse("foo.tbl1")); // another client creates a table with the same nessie server - anotherCatalog.createTable(TableIdentifier.parse("foo.tbl2"), schema); + anotherCatalog.createTable(TableIdentifier.parse("foo.tbl2"), SCHEMA); assertThat(anotherCatalog.listTables(Namespace.of("foo"))) .containsExactlyInAnyOrder( TableIdentifier.parse("foo.tbl1"), TableIdentifier.parse("foo.tbl2")); @@ -171,7 +171,7 @@ public void testListTables() { @Test public void testCommits() { TableIdentifier identifier = TableIdentifier.parse("foo.tbl1"); - createTable(identifier, schema); + createTable(identifier, SCHEMA); Table tableFromCatalog = catalog.loadTable(identifier); tableFromCatalog.updateSchema().addColumn("x1", Types.LongType.get()).commit(); @@ -188,7 +188,7 @@ public void testCommits() { @Test public void testConcurrentCommitsWithRefresh() { TableIdentifier identifier = TableIdentifier.parse("foo.tbl1"); - createTable(identifier, schema); + createTable(identifier, SCHEMA); String hashBefore = catalog.currentHash(); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index 94eb3144a0a4..f0f75c842429 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -82,9 +82,9 @@ public class TestNessieTable extends BaseTestIceberg { private static final String TABLE_NAME = "tbl"; private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DB_NAME, TABLE_NAME); private static final ContentKey KEY = ContentKey.of(DB_NAME, TABLE_NAME); - private static final Schema schema = + private static final Schema SCHEMA = new Schema(Types.StructType.of(required(1, "id", Types.LongType.get())).fields()); - private static final Schema altered = + private static final Schema ALTERED = new Schema( Types.StructType.of( required(1, "id", Types.LongType.get()), @@ -102,7 +102,7 @@ public TestNessieTable() { public void beforeEach(NessieClientFactory clientFactory, @NessieClientUri URI nessieUri) throws IOException { super.beforeEach(clientFactory, nessieUri); - this.tableLocation = createTable(TABLE_IDENTIFIER, schema).location().replaceFirst("file:", ""); + this.tableLocation = createTable(TABLE_IDENTIFIER, SCHEMA).location().replaceFirst("file:", ""); } @Override @@ -539,7 +539,7 @@ public void testExistingTableUpdate() { // Only 2 snapshotFile Should exist and no manifests should exist assertThat(metadataVersionFiles(tableLocation)).isNotNull().hasSize(2); assertThat(manifestFiles(tableLocation)).isNotNull().isEmpty(); - assertThat(altered.asStruct()).isEqualTo(icebergTable.schema().asStruct()); + assertThat(ALTERED.asStruct()).isEqualTo(icebergTable.schema().asStruct()); } @Test @@ -614,7 +614,7 @@ public void testGCEnabledViaTableDefaultCatalogProperty() { .build()); // Create the table again using updated config defaults. - tableLocation = createTable(TABLE_IDENTIFIER, schema).location().replaceFirst("file:", ""); + tableLocation = createTable(TABLE_IDENTIFIER, SCHEMA).location().replaceFirst("file:", ""); Table icebergTable = catalog.loadTable(TABLE_IDENTIFIER); assertThatCode( @@ -675,12 +675,12 @@ protected List manifestFiles(String tablePath) { private static String addRecordsToFile(Table table, String filename) throws IOException { GenericRecordBuilder recordBuilder = - new GenericRecordBuilder(AvroSchemaUtil.convert(schema, "test")); + new GenericRecordBuilder(AvroSchemaUtil.convert(SCHEMA, "test")); List records = Lists.newArrayListWithCapacity(3); records.add(recordBuilder.set("id", 1L).build()); records.add(recordBuilder.set("id", 2L).build()); records.add(recordBuilder.set("id", 3L).build()); - return writeRecordsToFile(table, schema, filename, records); + return writeRecordsToFile(table, SCHEMA, filename, records); } } diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java index 2da9b2b9deba..62f330f9f572 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestBloomRowGroupFilter.java @@ -82,7 +82,7 @@ public class TestBloomRowGroupFilter { - private static final Types.StructType structFieldType = + private static final Types.StructType STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(16, "int_field", IntegerType.get())); private static final Schema SCHEMA = new Schema( @@ -100,7 +100,7 @@ public class TestBloomRowGroupFilter { optional(12, "all_nans", DoubleType.get()), optional(13, "some_nans", FloatType.get()), optional(14, "no_nans", DoubleType.get()), - optional(15, "struct_not_null", structFieldType), + optional(15, "struct_not_null", STRUCT_FIELD_TYPE), optional(17, "not_in_file", FloatType.get()), optional(18, "no_stats", StringType.get()), optional(19, "boolean", Types.BooleanType.get()), @@ -113,7 +113,7 @@ public class TestBloomRowGroupFilter { optional(26, "long_decimal", Types.DecimalType.of(14, 2)), optional(27, "fixed_decimal", Types.DecimalType.of(31, 2))); - private static final Types.StructType _structFieldType = + private static final Types.StructType UNDERSCORE_STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(16, "_int_field", IntegerType.get())); private static final Schema FILE_SCHEMA = @@ -132,7 +132,7 @@ public class TestBloomRowGroupFilter { optional(12, "_all_nans", DoubleType.get()), optional(13, "_some_nans", FloatType.get()), optional(14, "_no_nans", DoubleType.get()), - optional(15, "_struct_not_null", _structFieldType), + optional(15, "_struct_not_null", UNDERSCORE_STRUCT_FIELD_TYPE), optional(18, "_no_stats", StringType.get()), optional(19, "_boolean", Types.BooleanType.get()), optional(20, "_time", Types.TimeType.get()), @@ -161,7 +161,7 @@ public class TestBloomRowGroupFilter { private static final double DOUBLE_BASE = 1000D; private static final float FLOAT_BASE = 10000F; private static final String BINARY_PREFIX = "BINARY测试_"; - private static final Instant instant = Instant.parse("2018-10-10T00:00:00.000Z"); + private static final Instant INSTANT = Instant.parse("2018-10-10T00:00:00.000Z"); private static final List RANDOM_UUIDS; private static final List RANDOM_BYTES; @@ -192,7 +192,7 @@ public void createInputFile() throws IOException { assertThat(temp.delete()).isTrue(); // build struct field schema - org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); + org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); OutputFile outFile = Files.localOutput(temp); try (FileAppender appender = @@ -251,10 +251,10 @@ public void createInputFile() throws IOException { builder.set("_struct_not_null", structNotNull); // struct with int builder.set("_no_stats", TOO_LONG_FOR_STATS); // value longer than 4k will produce no stats builder.set("_boolean", i % 2 == 0); - builder.set("_time", instant.plusSeconds(i * 86400).toEpochMilli()); - builder.set("_date", instant.plusSeconds(i * 86400).getEpochSecond()); - builder.set("_timestamp", instant.plusSeconds(i * 86400).toEpochMilli()); - builder.set("_timestamptz", instant.plusSeconds(i * 86400).toEpochMilli()); + builder.set("_time", INSTANT.plusSeconds(i * 86400).toEpochMilli()); + builder.set("_date", INSTANT.plusSeconds(i * 86400).getEpochSecond()); + builder.set("_timestamp", INSTANT.plusSeconds(i * 86400).toEpochMilli()); + builder.set("_timestamptz", INSTANT.plusSeconds(i * 86400).toEpochMilli()); builder.set("_binary", RANDOM_BYTES.get(i)); builder.set("_int_decimal", new BigDecimal(String.valueOf(77.77 + i))); builder.set("_long_decimal", new BigDecimal(String.valueOf(88.88 + i))); @@ -807,7 +807,7 @@ public void testBooleanEq() { @Test public void testTimeEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("time", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); @@ -822,7 +822,7 @@ public void testTimeEq() { @Test public void testDateEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("date", ins.getEpochSecond())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); @@ -837,7 +837,7 @@ public void testDateEq() { @Test public void testTimestampEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("timestamp", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); @@ -852,7 +852,7 @@ public void testTimestampEq() { @Test public void testTimestamptzEq() { for (int i = -20; i < INT_VALUE_COUNT + 20; i++) { - Instant ins = instant.plusSeconds(i * 86400); + Instant ins = INSTANT.plusSeconds(i * 86400); boolean shouldRead = new ParquetBloomRowGroupFilter(SCHEMA, equal("timestamptz", ins.toEpochMilli())) .shouldRead(parquetSchema, rowGroupMetadata, bloomStore); diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java index d690d3cf5175..fe3e8a402bb5 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestDictionaryRowGroupFilter.java @@ -90,7 +90,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestDictionaryRowGroupFilter { - private static final Types.StructType structFieldType = + private static final Types.StructType STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(9, "int_field", IntegerType.get())); private static final Schema SCHEMA = @@ -102,7 +102,7 @@ public class TestDictionaryRowGroupFilter { optional(5, "some_nulls", StringType.get()), optional(6, "no_nulls", StringType.get()), optional(7, "non_dict", StringType.get()), - optional(8, "struct_not_null", structFieldType), + optional(8, "struct_not_null", STRUCT_FIELD_TYPE), optional(10, "not_in_file", FloatType.get()), optional(11, "all_nans", DoubleType.get()), optional(12, "some_nans", FloatType.get()), @@ -113,7 +113,7 @@ public class TestDictionaryRowGroupFilter { DecimalType.of(20, 10)), // >18 precision to enforce FIXED_LEN_BYTE_ARRAY optional(15, "_nans_and_nulls", DoubleType.get())); - private static final Types.StructType _structFieldType = + private static final Types.StructType UNDERSCORE_STRUCT_FIELD_TYPE = Types.StructType.of(Types.NestedField.required(9, "_int_field", IntegerType.get())); private static final Schema FILE_SCHEMA = @@ -125,7 +125,7 @@ public class TestDictionaryRowGroupFilter { optional(5, "_some_nulls", StringType.get()), optional(6, "_no_nulls", StringType.get()), optional(7, "_non_dict", StringType.get()), - optional(8, "_struct_not_null", _structFieldType), + optional(8, "_struct_not_null", UNDERSCORE_STRUCT_FIELD_TYPE), optional(11, "_all_nans", DoubleType.get()), optional(12, "_some_nans", FloatType.get()), optional(13, "_no_nans", DoubleType.get()), @@ -171,7 +171,7 @@ public void createInputFile() throws IOException { assertThat(parquetFile.delete()).isTrue(); // build struct field schema - org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(_structFieldType); + org.apache.avro.Schema structSchema = AvroSchemaUtil.convert(UNDERSCORE_STRUCT_FIELD_TYPE); OutputFile outFile = Files.localOutput(parquetFile); try (FileAppender appender = diff --git a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java index ea47ecb1c268..63f512ee632b 100644 --- a/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java +++ b/parquet/src/test/java/org/apache/iceberg/parquet/TestParquetEncryption.java @@ -46,36 +46,36 @@ public class TestParquetEncryption { - private static final String columnName = "intCol"; - private static final int recordCount = 100; - private static final ByteBuffer fileDek = ByteBuffer.allocate(16); - private static final ByteBuffer aadPrefix = ByteBuffer.allocate(16); + private static final String COLUMN_NAME = "intCol"; + private static final int RECORD_COUNT = 100; + private static final ByteBuffer FILE_DEK = ByteBuffer.allocate(16); + private static final ByteBuffer AAD_PREFIX = ByteBuffer.allocate(16); + private static final Schema SCHEMA = new Schema(optional(1, COLUMN_NAME, IntegerType.get())); private static File file; - private static final Schema schema = new Schema(optional(1, columnName, IntegerType.get())); @TempDir private Path temp; @BeforeEach public void writeEncryptedFile() throws IOException { - List records = Lists.newArrayListWithCapacity(recordCount); - org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - for (int i = 1; i <= recordCount; i++) { + List records = Lists.newArrayListWithCapacity(RECORD_COUNT); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(SCHEMA.asStruct()); + for (int i = 1; i <= RECORD_COUNT; i++) { GenericData.Record record = new GenericData.Record(avroSchema); - record.put(columnName, i); + record.put(COLUMN_NAME, i); records.add(record); } SecureRandom rand = new SecureRandom(); - rand.nextBytes(fileDek.array()); - rand.nextBytes(aadPrefix.array()); + rand.nextBytes(FILE_DEK.array()); + rand.nextBytes(AAD_PREFIX.array()); file = createTempFile(temp); FileAppender writer = Parquet.write(localOutput(file)) - .schema(schema) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) + .schema(SCHEMA) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) .build(); try (Closeable toClose = writer) { @@ -86,7 +86,7 @@ public void writeEncryptedFile() throws IOException { @Test public void testReadEncryptedFileWithoutKeys() throws IOException { assertThatThrownBy( - () -> Parquet.read(localInput(file)).project(schema).callInit().build().iterator()) + () -> Parquet.read(localInput(file)).project(SCHEMA).callInit().build().iterator()) .as("Decrypted without keys") .isInstanceOf(ParquetCryptoRuntimeException.class) .hasMessage("Trying to read file with encrypted footer. No keys available"); @@ -97,8 +97,8 @@ public void testReadEncryptedFileWithoutAADPrefix() throws IOException { assertThatThrownBy( () -> Parquet.read(localInput(file)) - .project(schema) - .withFileEncryptionKey(fileDek) + .project(SCHEMA) + .withFileEncryptionKey(FILE_DEK) .callInit() .build() .iterator()) @@ -113,15 +113,15 @@ public void testReadEncryptedFileWithoutAADPrefix() throws IOException { public void testReadEncryptedFile() throws IOException { try (CloseableIterator readRecords = Parquet.read(localInput(file)) - .withFileEncryptionKey(fileDek) - .withAADPrefix(aadPrefix) - .project(schema) + .withFileEncryptionKey(FILE_DEK) + .withAADPrefix(AAD_PREFIX) + .project(SCHEMA) .callInit() .build() .iterator()) { - for (int i = 1; i <= recordCount; i++) { + for (int i = 1; i <= RECORD_COUNT; i++) { GenericData.Record readRecord = (GenericData.Record) readRecords.next(); - assertThat(readRecord.get(columnName)).isEqualTo(i); + assertThat(readRecord.get(COLUMN_NAME)).isEqualTo(i); } } } 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 5f3d3ac665fd..88233c58a372 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java +++ b/pig/src/main/java/org/apache/iceberg/pig/IcebergStorage.java @@ -70,8 +70,8 @@ public class IcebergStorage extends LoadFunc 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 static final Map TABLES = Maps.newConcurrentMap(); + private static final Map LOCATIONS = Maps.newConcurrentMap(); private String signature; @@ -81,7 +81,7 @@ public class IcebergStorage extends LoadFunc public void setLocation(String location, Job job) { LOG.info("[{}]: setLocation() -> {}", signature, location); - locations.put(signature, location); + LOCATIONS.put(signature, location); Configuration conf = job.getConfiguration(); @@ -93,9 +93,9 @@ public void setLocation(String location, Job job) { @Override public InputFormat getInputFormat() { LOG.info("[{}]: getInputFormat()", signature); - String location = locations.get(signature); + String location = LOCATIONS.get(signature); - return new IcebergPigInputFormat(tables.get(location), signature); + return new IcebergPigInputFormat(TABLES.get(location), signature); } @Override @@ -323,13 +323,13 @@ private Table load(String location, Job job) throws IOException { iceberg = (Tables) ReflectionUtils.newInstance(tablesImpl, job.getConfiguration()); } - Table result = tables.get(location); + 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); + TABLES.put(location, result); } catch (Exception e) { throw new FrontendException("Failed to instantiate tables implementation", e); } From a0cf24058e306ad9096fbeec4a0f5759e64ccf37 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Thu, 11 Jul 2024 06:47:12 -0700 Subject: [PATCH 0457/1019] Build: don't include slf4j-api in bundled JARs (#10665) This excludes `slf4j-api` from being shadowed into `iceberg-aws-bundle`, `iceberg-azure-bundle`, `iceberg-gcp-bundle`, and `iceberg-hive3-orc-bundle`. This uses the same exclude pattern established by `iceberg-bundled-guava`. Fixes #10534 --- aws-bundle/build.gradle | 4 ++++ azure-bundle/build.gradle | 4 ++++ gcp-bundle/build.gradle | 4 ++++ hive3-orc-bundle/build.gradle | 4 ++++ 4 files changed, 16 insertions(+) diff --git a/aws-bundle/build.gradle b/aws-bundle/build.gradle index 6b1eae46d89e..eea591a7b76d 100644 --- a/aws-bundle/build.gradle +++ b/aws-bundle/build.gradle @@ -47,6 +47,10 @@ project(":iceberg-aws-bundle") { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // relocate AWS-specific versions relocate 'org.apache.http', 'org.apache.iceberg.aws.shaded.org.apache.http' relocate 'io.netty', 'org.apache.iceberg.aws.shaded.io.netty' diff --git a/azure-bundle/build.gradle b/azure-bundle/build.gradle index 3bc0a31cb46d..9b1cd5b42fb8 100644 --- a/azure-bundle/build.gradle +++ b/azure-bundle/build.gradle @@ -39,6 +39,10 @@ project(":iceberg-azure-bundle") { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // relocate Azure-specific versions relocate 'io.netty', 'org.apache.iceberg.azure.shaded.io.netty' relocate 'com.fasterxml.jackson', 'org.apache.iceberg.azure.shaded.com.fasterxml.jackson' diff --git a/gcp-bundle/build.gradle b/gcp-bundle/build.gradle index 98e67c214ce0..1b339a66c195 100644 --- a/gcp-bundle/build.gradle +++ b/gcp-bundle/build.gradle @@ -38,6 +38,10 @@ project(":iceberg-gcp-bundle") { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // relocate GCP-specific versions relocate 'com.fasterxml.jackson', 'org.apache.iceberg.gcp.shaded.com.fasterxml.jackson' relocate 'com.google.common', 'org.apache.iceberg.gcp.shaded.com.google.common' diff --git a/hive3-orc-bundle/build.gradle b/hive3-orc-bundle/build.gradle index 11bb2abf85c4..3ca89bd47435 100644 --- a/hive3-orc-bundle/build.gradle +++ b/hive3-orc-bundle/build.gradle @@ -50,6 +50,10 @@ project(':iceberg-hive3-orc-bundle') { include 'NOTICE' } + dependencies { + exclude(dependency('org.slf4j:slf4j-api')) + } + // Relocate dependencies to avoid conflicts relocate 'org.apache.orc.storage', 'org.apache.hadoop.hive' From 8993f3505df101ae94a40866a4989dd4bebb6d49 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Thu, 11 Jul 2024 08:22:20 -0700 Subject: [PATCH 0458/1019] Kafka Connect: Commit coordination (#10351) * Kafka Connect: Commit coordination * PR feedback * Get partition assignment from consumer * test fix --- .../connect/events/TableReference.java | 20 ++ .../iceberg/connect/events/EventTestUtil.java | 3 +- .../apache/iceberg/connect/CatalogUtils.java | 98 ++++++ .../org/apache/iceberg/connect/Committer.java | 32 ++ .../iceberg/connect/CommitterFactory.java | 29 ++ .../iceberg/connect/IcebergSinkConfig.java | 23 +- .../iceberg/connect/IcebergSinkConnector.java | 4 +- .../iceberg/connect/IcebergSinkTask.java | 107 ++++++ .../iceberg/connect/channel/Channel.java | 167 ++++++++++ .../iceberg/connect/channel/CommitState.java | 167 ++++++++++ .../connect/channel/CommitterImpl.java | 128 +++++++ .../iceberg/connect/channel/Coordinator.java | 314 ++++++++++++++++++ .../connect/channel/CoordinatorThread.java | 69 ++++ .../iceberg/connect/channel/Envelope.java | 45 +++ .../connect/channel/KafkaClientFactory.java | 68 ++++ .../iceberg/connect/channel/KafkaUtils.java | 66 ++++ .../connect/channel/NotRunningException.java | 25 ++ .../iceberg/connect/channel/Worker.java | 126 +++++++ .../iceberg/connect/data/IcebergWriter.java | 14 +- .../connect/data/IcebergWriterFactory.java | 7 +- ...erResult.java => IcebergWriterResult.java} | 4 +- .../iceberg/connect/data/NoOpWriter.java | 5 +- .../apache/iceberg/connect/data/Offset.java | 54 +++ .../data/{Utilities.java => RecordUtils.java} | 78 +---- .../iceberg/connect/data/RecordWriter.java | 4 +- .../iceberg/connect/data/SinkWriter.java | 140 ++++++++ .../connect/data/SinkWriterResult.java | 42 +++ ...ilitiesTest.java => CatalogUtilsTest.java} | 74 +---- .../connect/channel/ChannelTestBase.java | 131 ++++++++ .../connect/channel/CommitStateTest.java | 107 ++++++ .../connect/channel/CommitterImplTest.java | 60 ++++ .../connect/channel/CoordinatorTest.java | 213 ++++++++++++ .../channel/CoordinatorThreadTest.java | 48 +++ .../connect/channel/EventTestUtil.java | 98 ++++++ .../iceberg/connect/channel/WorkerTest.java | 116 +++++++ .../iceberg/connect/data/BaseWriterTest.java | 2 +- .../iceberg/connect/data/RecordUtilsTest.java | 93 ++++++ .../connect/data/SchemaUpdateTest.java | 68 ++++ .../iceberg/connect/data/SinkWriterTest.java | 210 ++++++++++++ 39 files changed, 2869 insertions(+), 190 deletions(-) create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.java rename kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/{WriterResult.java => IcebergWriterResult.java} (96%) create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java rename kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/{Utilities.java => RecordUtils.java} (68%) create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java create mode 100644 kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java rename kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/{data/UtilitiesTest.java => CatalogUtilsTest.java} (60%) create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java create mode 100644 kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java diff --git a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java index 50eaa1050485..f30eac892400 100644 --- a/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java +++ b/kafka-connect/kafka-connect-events/src/main/java/org/apache/iceberg/connect/events/TableReference.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.stream.Collectors; import org.apache.avro.Schema; import org.apache.avro.generic.IndexedRecord; @@ -120,4 +121,23 @@ public Object get(int i) { throw new UnsupportedOperationException("Unknown field ordinal: " + i); } } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + TableReference that = (TableReference) o; + return Objects.equals(catalog, that.catalog) + && Objects.equals(namespace, that.namespace) + && Objects.equals(name, that.name); + } + + @Override + public int hashCode() { + return Objects.hash(catalog, namespace, name); + } } diff --git a/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java b/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java index 8f1f7a601f86..48e268bf0561 100644 --- a/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java +++ b/kafka-connect/kafka-connect-events/src/test/java/org/apache/iceberg/connect/events/EventTestUtil.java @@ -44,8 +44,7 @@ private EventTestUtil() {} static final Schema SCHEMA = new Schema(ImmutableList.of(Types.NestedField.required(1, "id", Types.LongType.get()))); - static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("id").withSpecId(1).build(); + static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); static final SortOrder ORDER = SortOrder.builderFor(SCHEMA).sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST).build(); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java new file mode 100644 index 000000000000..a3c6358e1bdf --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CatalogUtils.java @@ -0,0 +1,98 @@ +/* + * 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.connect; + +import java.io.IOException; +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.common.DynClasses; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.common.DynMethods; +import org.apache.iceberg.common.DynMethods.BoundMethod; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class CatalogUtils { + + private static final Logger LOG = LoggerFactory.getLogger(CatalogUtils.class.getName()); + private static final List HADOOP_CONF_FILES = + ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); + + static Catalog loadCatalog(IcebergSinkConfig config) { + return CatalogUtil.buildIcebergCatalog( + config.catalogName(), config.catalogProps(), loadHadoopConfig(config)); + } + + // use reflection here to avoid requiring Hadoop as a dependency + private static Object loadHadoopConfig(IcebergSinkConfig config) { + Class configClass = + DynClasses.builder() + .impl("org.apache.hadoop.hdfs.HdfsConfiguration") + .impl("org.apache.hadoop.conf.Configuration") + .orNull() + .build(); + + if (configClass == null) { + LOG.info("Hadoop not found on classpath, not creating Hadoop config"); + return null; + } + + try { + Object result = DynConstructors.builder().hiddenImpl(configClass).build().newInstance(); + BoundMethod addResourceMethod = + DynMethods.builder("addResource").impl(configClass, URL.class).build(result); + BoundMethod setMethod = + DynMethods.builder("set").impl(configClass, String.class, String.class).build(result); + + // load any config files in the specified config directory + String hadoopConfDir = config.hadoopConfDir(); + if (hadoopConfDir != null) { + HADOOP_CONF_FILES.forEach( + confFile -> { + Path path = Paths.get(hadoopConfDir, confFile); + if (Files.exists(path)) { + try { + addResourceMethod.invoke(path.toUri().toURL()); + } catch (IOException e) { + LOG.warn("Error adding Hadoop resource {}, resource was not added", path, e); + } + } + }); + } + + // set any Hadoop properties specified in the sink config + config.hadoopProps().forEach(setMethod::invoke); + + LOG.info("Hadoop config initialized: {}", configClass.getName()); + return result; + } catch (Exception e) { + LOG.warn( + "Hadoop found on classpath but could not create config, proceeding without config", e); + } + return null; + } + + private CatalogUtils() {} +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java new file mode 100644 index 000000000000..edc217d1b0e4 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/Committer.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect; + +import java.util.Collection; +import org.apache.iceberg.catalog.Catalog; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +public interface Committer { + void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context); + + void stop(); + + void save(Collection sinkRecords); +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.java new file mode 100644 index 000000000000..18ff118c7773 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/CommitterFactory.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.connect; + +import org.apache.iceberg.connect.channel.CommitterImpl; + +class CommitterFactory { + static Committer createCommitter(IcebergSinkConfig config) { + return new CommitterImpl(); + } + + private CommitterFactory() {} +} 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 e64e183089cf..aed11ab0b169 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,7 +80,6 @@ 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_PROP = "iceberg.control.group-id"; 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"; @@ -104,11 +103,7 @@ public class IcebergSinkConfig extends AbstractConfig { public static final ConfigDef CONFIG_DEF = newConfigDef(); public static String version() { - String kcVersion = IcebergSinkConfig.class.getPackage().getImplementationVersion(); - if (kcVersion == null) { - kcVersion = "unknown"; - } - return IcebergBuild.version() + "-kc-" + kcVersion; + return IcebergBuild.version(); } private static ConfigDef newConfigDef() { @@ -185,12 +180,6 @@ private static ConfigDef newConfigDef() { DEFAULT_CONTROL_TOPIC, Importance.MEDIUM, "Name of the control topic"); - configDef.define( - CONTROL_GROUP_ID_PROP, - ConfigDef.Type.STRING, - null, - Importance.MEDIUM, - "Name of the consumer group to store offsets"); configDef.define( CONNECT_GROUP_ID_PROP, ConfigDef.Type.STRING, @@ -370,16 +359,6 @@ public String controlTopic() { return getString(CONTROL_TOPIC_PROP); } - public String controlGroupId() { - String result = getString(CONTROL_GROUP_ID_PROP); - if (result != null) { - return result; - } - String connectorName = connectorName(); - Preconditions.checkNotNull(connectorName, "Connector name cannot be null"); - return DEFAULT_CONTROL_GROUP_PREFIX + connectorName; - } - 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/IcebergSinkConnector.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java index 8be8518f4407..be1f9a50b8f6 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkConnector.java @@ -44,9 +44,7 @@ public void start(Map connectorProps) { @Override public Class taskClass() { - // FIXME: update this when the connector channel is added - // return IcebergSinkTask.class; - return null; + return IcebergSinkTask.class; } @Override diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java new file mode 100644 index 000000000000..460b18fd7fc2 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.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.connect; + +import java.util.Collection; +import java.util.Map; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTask; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class IcebergSinkTask extends SinkTask { + + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkTask.class); + + private IcebergSinkConfig config; + private Catalog catalog; + private Committer committer; + + @Override + public String version() { + return IcebergSinkConfig.version(); + } + + @Override + public void start(Map props) { + this.config = new IcebergSinkConfig(props); + } + + @Override + public void open(Collection partitions) { + Preconditions.checkArgument(catalog == null, "Catalog already open"); + Preconditions.checkArgument(committer == null, "Committer already open"); + + catalog = CatalogUtils.loadCatalog(config); + committer = CommitterFactory.createCommitter(config); + committer.start(catalog, config, context); + } + + @Override + public void close(Collection partitions) { + close(); + } + + private void close() { + if (committer != null) { + committer.stop(); + committer = null; + } + + if (catalog != null) { + if (catalog instanceof AutoCloseable) { + try { + ((AutoCloseable) catalog).close(); + } catch (Exception e) { + LOG.warn("An error occurred closing catalog instance, ignoring...", e); + } + } + catalog = null; + } + } + + @Override + public void put(Collection sinkRecords) { + Preconditions.checkNotNull(committer, "Committer wasn't initialized"); + committer.save(sinkRecords); + } + + @Override + public void flush(Map currentOffsets) { + Preconditions.checkNotNull(committer, "Committer wasn't initialized"); + committer.save(null); + } + + @Override + public Map preCommit( + Map currentOffsets) { + // offset commit is handled by the worker + return ImmutableMap.of(); + } + + @Override + public void stop() { + close(); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java new file mode 100644 index 000000000000..993fcf67c989 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Channel.java @@ -0,0 +1,167 @@ +/* + * 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.connect.channel; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.Offset; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.Event; +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.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class Channel { + + private static final Logger LOG = LoggerFactory.getLogger(Channel.class); + + private final String controlTopic; + private final String connectGroupId; + private final Producer producer; + private final Consumer consumer; + private final SinkTaskContext context; + private final Admin admin; + private final Map controlTopicOffsets = Maps.newHashMap(); + private final String producerId; + + Channel( + String name, + String consumerGroupId, + IcebergSinkConfig config, + KafkaClientFactory clientFactory, + SinkTaskContext context) { + this.controlTopic = config.controlTopic(); + this.connectGroupId = config.connectGroupId(); + this.context = context; + + String transactionalId = name + config.transactionalSuffix(); + this.producer = clientFactory.createProducer(transactionalId); + this.consumer = clientFactory.createConsumer(consumerGroupId); + this.admin = clientFactory.createAdmin(); + + this.producerId = UUID.randomUUID().toString(); + } + + protected void send(Event event) { + send(ImmutableList.of(event), ImmutableMap.of()); + } + + @SuppressWarnings("FutureReturnValueIgnored") + protected void send(List events, Map sourceOffsets) { + Map offsetsToCommit = Maps.newHashMap(); + sourceOffsets.forEach((k, v) -> offsetsToCommit.put(k, new OffsetAndMetadata(v.offset()))); + + List> recordList = + events.stream() + .map( + event -> { + LOG.info("Sending event of type: {}", event.type().name()); + byte[] data = AvroUtil.encode(event); + // key by producer ID to keep event order + return new ProducerRecord<>(controlTopic, producerId, data); + }) + .collect(Collectors.toList()); + + synchronized (producer) { + producer.beginTransaction(); + try { + // NOTE: we shouldn't call get() on the future in a transactional context, + // see docs for org.apache.kafka.clients.producer.KafkaProducer + recordList.forEach(producer::send); + if (!sourceOffsets.isEmpty()) { + producer.sendOffsetsToTransaction( + offsetsToCommit, KafkaUtils.consumerGroupMetadata(context)); + } + producer.commitTransaction(); + } catch (Exception e) { + try { + producer.abortTransaction(); + } catch (Exception ex) { + LOG.warn("Error aborting producer transaction", ex); + } + throw e; + } + } + } + + protected abstract boolean receive(Envelope envelope); + + protected void consumeAvailable(Duration pollDuration) { + ConsumerRecords records = consumer.poll(pollDuration); + while (!records.isEmpty()) { + records.forEach( + record -> { + // the consumer stores the offsets that corresponds to the next record to consume, + // so increment the record offset by one + controlTopicOffsets.put(record.partition(), record.offset() + 1); + + Event event = AvroUtil.decode(record.value()); + + if (event.groupId().equals(connectGroupId)) { + LOG.debug("Received event of type: {}", event.type().name()); + if (receive(new Envelope(event, record.partition(), record.offset()))) { + LOG.info("Handled event of type: {}", event.type().name()); + } + } + }); + records = consumer.poll(pollDuration); + } + } + + protected Map controlTopicOffsets() { + return controlTopicOffsets; + } + + protected void commitConsumerOffsets() { + Map offsetsToCommit = Maps.newHashMap(); + controlTopicOffsets() + .forEach( + (k, v) -> + offsetsToCommit.put(new TopicPartition(controlTopic, k), new OffsetAndMetadata(v))); + consumer.commitSync(offsetsToCommit); + } + + void start() { + consumer.subscribe(ImmutableList.of(controlTopic)); + + // initial poll with longer duration so the consumer will initialize... + consumeAvailable(Duration.ofSeconds(1)); + } + + void stop() { + LOG.info("Channel stopping"); + producer.close(); + consumer.close(); + admin.close(); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java new file mode 100644 index 000000000000..6cad33c3e387 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitState.java @@ -0,0 +1,167 @@ +/* + * 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.connect.channel; + +import java.time.OffsetDateTime; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class CommitState { + private static final Logger LOG = LoggerFactory.getLogger(CommitState.class); + + private final List commitBuffer = Lists.newArrayList(); + private final List readyBuffer = Lists.newArrayList(); + private long startTime; + private UUID currentCommitId; + private final IcebergSinkConfig config; + + CommitState(IcebergSinkConfig config) { + this.config = config; + } + + void addResponse(Envelope envelope) { + commitBuffer.add(envelope); + if (!isCommitInProgress()) { + DataWritten dataWritten = (DataWritten) envelope.event().payload(); + LOG.warn( + "Received commit response when no commit in progress, this can happen during recovery. Commit ID: {}", + dataWritten.commitId()); + } + } + + void addReady(Envelope envelope) { + DataComplete dataComplete = (DataComplete) envelope.event().payload(); + readyBuffer.add(dataComplete); + if (!isCommitInProgress()) { + LOG.warn( + "Received commit ready when no commit in progress, this can happen during recovery. Commit ID: {}", + dataComplete.commitId()); + } + } + + UUID currentCommitId() { + return currentCommitId; + } + + boolean isCommitInProgress() { + return currentCommitId != null; + } + + boolean isCommitIntervalReached() { + if (startTime == 0) { + startTime = System.currentTimeMillis(); + } + + return (!isCommitInProgress() + && System.currentTimeMillis() - startTime >= config.commitIntervalMs()); + } + + void startNewCommit() { + currentCommitId = UUID.randomUUID(); + startTime = System.currentTimeMillis(); + } + + void endCurrentCommit() { + readyBuffer.clear(); + currentCommitId = null; + } + + void clearResponses() { + commitBuffer.clear(); + } + + boolean isCommitTimedOut() { + if (!isCommitInProgress()) { + return false; + } + + if (System.currentTimeMillis() - startTime > config.commitTimeoutMs()) { + LOG.info("Commit timeout reached. Commit ID: {}", currentCommitId); + return true; + } + return false; + } + + boolean isCommitReady(int expectedPartitionCount) { + if (!isCommitInProgress()) { + return false; + } + + int receivedPartitionCount = + readyBuffer.stream() + .filter(payload -> payload.commitId().equals(currentCommitId)) + .mapToInt(payload -> payload.assignments().size()) + .sum(); + + if (receivedPartitionCount >= expectedPartitionCount) { + LOG.info( + "Commit {} ready, received responses for all {} partitions", + currentCommitId, + receivedPartitionCount); + return true; + } + + LOG.info( + "Commit {} not ready, received responses for {} of {} partitions, waiting for more", + currentCommitId, + receivedPartitionCount, + expectedPartitionCount); + + return false; + } + + Map> tableCommitMap() { + return commitBuffer.stream() + .collect( + Collectors.groupingBy( + envelope -> ((DataWritten) envelope.event().payload()).tableReference())); + } + + OffsetDateTime validThroughTs(boolean partialCommit) { + boolean hasValidThroughTs = + !partialCommit + && readyBuffer.stream() + .flatMap(event -> event.assignments().stream()) + .allMatch(offset -> offset.timestamp() != null); + + OffsetDateTime result; + if (hasValidThroughTs) { + result = + readyBuffer.stream() + .flatMap(event -> event.assignments().stream()) + .map(TopicPartitionOffset::timestamp) + .min(Comparator.naturalOrder()) + .orElse(null); + } else { + result = null; + } + return result; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java new file mode 100644 index 000000000000..53b7b76e8ea0 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CommitterImpl.java @@ -0,0 +1,128 @@ +/* + * 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.connect.channel; + +import java.util.Collection; +import java.util.Comparator; +import java.util.Set; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.connect.Committer; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.common.ConsumerGroupState; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CommitterImpl implements Committer { + + private static final Logger LOG = LoggerFactory.getLogger(CommitterImpl.class); + + private CoordinatorThread coordinatorThread; + private Worker worker; + + static class TopicPartitionComparator implements Comparator { + + @Override + public int compare(TopicPartition o1, TopicPartition o2) { + int result = o1.topic().compareTo(o2.topic()); + if (result == 0) { + result = Integer.compare(o1.partition(), o2.partition()); + } + return result; + } + } + + @Override + public void start(Catalog catalog, IcebergSinkConfig config, SinkTaskContext context) { + KafkaClientFactory clientFactory = new KafkaClientFactory(config.kafkaProps()); + + ConsumerGroupDescription groupDesc; + try (Admin admin = clientFactory.createAdmin()) { + groupDesc = KafkaUtils.consumerGroupDescription(config.connectGroupId(), admin); + } + + if (groupDesc.state() == ConsumerGroupState.STABLE) { + Collection members = groupDesc.members(); + Set partitions = context.assignment(); + if (isLeader(members, partitions)) { + LOG.info("Task elected leader, starting commit coordinator"); + Coordinator coordinator = new Coordinator(catalog, config, members, clientFactory, context); + coordinatorThread = new CoordinatorThread(coordinator); + coordinatorThread.start(); + } + } + + LOG.info("Starting commit worker"); + SinkWriter sinkWriter = new SinkWriter(catalog, config); + worker = new Worker(config, clientFactory, sinkWriter, context); + worker.start(); + } + + @Override + public void save(Collection sinkRecords) { + if (sinkRecords != null && !sinkRecords.isEmpty()) { + worker.save(sinkRecords); + } + processControlEvents(); + } + + @Override + public void stop() { + if (worker != null) { + worker.stop(); + worker = null; + } + + if (coordinatorThread != null) { + coordinatorThread.terminate(); + coordinatorThread = null; + } + } + + @VisibleForTesting + boolean isLeader(Collection members, Collection partitions) { + // there should only be one task assigned partition 0 of the first topic, + // so elect that one the leader + TopicPartition firstTopicPartition = + members.stream() + .flatMap(member -> member.assignment().topicPartitions().stream()) + .min(new TopicPartitionComparator()) + .orElseThrow( + () -> new ConnectException("No partitions assigned, cannot determine leader")); + + return partitions.contains(firstTopicPartition); + } + + private void processControlEvents() { + if (coordinatorThread != null && coordinatorThread.isTerminated()) { + throw new NotRunningException("Coordinator unexpectedly terminated"); + } + if (worker != null) { + worker.process(); + } + } +} 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 new file mode 100644 index 000000000000..7274f77e0c85 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Coordinator.java @@ -0,0 +1,314 @@ +/* + * 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.connect.channel; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.time.OffsetDateTime; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.function.Function; +import java.util.function.Predicate; +import java.util.stream.Collectors; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.exceptions.NoSuchTableException; +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.Tasks; +import org.apache.iceberg.util.ThreadPools; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class Coordinator extends Channel { + + private static final Logger LOG = LoggerFactory.getLogger(Coordinator.class); + private static final ObjectMapper MAPPER = new ObjectMapper(); + private static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; + private static final String VALID_THROUGH_TS_SNAPSHOT_PROP = "kafka.connect.valid-through-ts"; + private static final Duration POLL_DURATION = Duration.ofSeconds(1); + + private final Catalog catalog; + private final IcebergSinkConfig config; + private final int totalPartitionCount; + private final String snapshotOffsetsProp; + private final ExecutorService exec; + private final CommitState commitState; + + Coordinator( + Catalog catalog, + IcebergSinkConfig config, + Collection members, + KafkaClientFactory clientFactory, + SinkTaskContext context) { + // pass consumer group ID to which we commit low watermark offsets + super("coordinator", config.connectGroupId() + "-coord", config, clientFactory, context); + + this.catalog = catalog; + this.config = config; + this.totalPartitionCount = + members.stream().mapToInt(desc -> desc.assignment().topicPartitions().size()).sum(); + this.snapshotOffsetsProp = + String.format( + "kafka.connect.offsets.%s.%s", config.controlTopic(), config.connectGroupId()); + this.exec = ThreadPools.newWorkerPool("iceberg-committer", config.commitThreads()); + this.commitState = new CommitState(config); + } + + void process() { + if (commitState.isCommitIntervalReached()) { + // send out begin commit + commitState.startNewCommit(); + Event event = + new Event(config.connectGroupId(), new StartCommit(commitState.currentCommitId())); + send(event); + LOG.info("Commit {} initiated", commitState.currentCommitId()); + } + + consumeAvailable(POLL_DURATION); + + if (commitState.isCommitTimedOut()) { + commit(true); + } + } + + @Override + protected boolean receive(Envelope envelope) { + switch (envelope.event().payload().type()) { + case DATA_WRITTEN: + commitState.addResponse(envelope); + return true; + case DATA_COMPLETE: + commitState.addReady(envelope); + if (commitState.isCommitReady(totalPartitionCount)) { + commit(false); + } + return true; + } + return false; + } + + private void commit(boolean partialCommit) { + try { + doCommit(partialCommit); + } catch (Exception e) { + LOG.warn("Commit failed, will try again next cycle", e); + } finally { + commitState.endCurrentCommit(); + } + } + + private void doCommit(boolean partialCommit) { + Map> commitMap = commitState.tableCommitMap(); + + String offsetsJson = offsetsJson(); + OffsetDateTime validThroughTs = commitState.validThroughTs(partialCommit); + + Tasks.foreach(commitMap.entrySet()) + .executeWith(exec) + .stopOnFailure() + .run( + entry -> { + commitToTable(entry.getKey(), entry.getValue(), offsetsJson, validThroughTs); + }); + + // we should only get here if all tables committed successfully... + commitConsumerOffsets(); + commitState.clearResponses(); + + Event event = + new Event( + config.connectGroupId(), + new CommitComplete(commitState.currentCommitId(), validThroughTs)); + send(event); + + LOG.info( + "Commit {} complete, committed to {} table(s), valid-through {}", + commitState.currentCommitId(), + commitMap.size(), + validThroughTs); + } + + private String offsetsJson() { + try { + return MAPPER.writeValueAsString(controlTopicOffsets()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private void commitToTable( + TableReference tableReference, + List envelopeList, + String offsetsJson, + OffsetDateTime validThroughTs) { + TableIdentifier tableIdentifier = tableReference.identifier(); + Table table; + try { + table = catalog.loadTable(tableIdentifier); + } catch (NoSuchTableException e) { + LOG.warn("Table not found, skipping commit: {}", tableIdentifier, e); + return; + } + + String branch = config.tableConfig(tableIdentifier.toString()).commitBranch(); + + Map committedOffsets = lastCommittedOffsetsForTable(table, branch); + + List payloads = + envelopeList.stream() + .filter( + envelope -> { + Long minOffset = committedOffsets.get(envelope.partition()); + return minOffset == null || envelope.offset() >= minOffset; + }) + .map(envelope -> (DataWritten) envelope.event().payload()) + .collect(Collectors.toList()); + + List dataFiles = + payloads.stream() + .filter(payload -> payload.dataFiles() != null) + .flatMap(payload -> payload.dataFiles().stream()) + .filter(dataFile -> dataFile.recordCount() > 0) + .filter(distinctByKey(dataFile -> dataFile.path().toString())) + .collect(Collectors.toList()); + + List deleteFiles = + payloads.stream() + .filter(payload -> payload.deleteFiles() != null) + .flatMap(payload -> payload.deleteFiles().stream()) + .filter(deleteFile -> deleteFile.recordCount() > 0) + .filter(distinctByKey(deleteFile -> deleteFile.path().toString())) + .collect(Collectors.toList()); + + if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { + LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); + } else { + if (deleteFiles.isEmpty()) { + AppendFiles appendOp = table.newAppend(); + if (branch != null) { + appendOp.toBranch(branch); + } + appendOp.set(snapshotOffsetsProp, offsetsJson); + appendOp.set(COMMIT_ID_SNAPSHOT_PROP, commitState.currentCommitId().toString()); + if (validThroughTs != null) { + appendOp.set(VALID_THROUGH_TS_SNAPSHOT_PROP, validThroughTs.toString()); + } + dataFiles.forEach(appendOp::appendFile); + appendOp.commit(); + } else { + RowDelta deltaOp = table.newRowDelta(); + if (branch != null) { + deltaOp.toBranch(branch); + } + deltaOp.set(snapshotOffsetsProp, offsetsJson); + deltaOp.set(COMMIT_ID_SNAPSHOT_PROP, commitState.currentCommitId().toString()); + if (validThroughTs != null) { + deltaOp.set(VALID_THROUGH_TS_SNAPSHOT_PROP, validThroughTs.toString()); + } + dataFiles.forEach(deltaOp::addRows); + deleteFiles.forEach(deltaOp::addDeletes); + deltaOp.commit(); + } + + Long snapshotId = latestSnapshot(table, branch).snapshotId(); + Event event = + new Event( + config.connectGroupId(), + new CommitToTable( + commitState.currentCommitId(), tableReference, snapshotId, validThroughTs)); + send(event); + + LOG.info( + "Commit complete to table {}, snapshot {}, commit ID {}, valid-through {}", + tableIdentifier, + snapshotId, + commitState.currentCommitId(), + validThroughTs); + } + } + + private Predicate distinctByKey(Function keyExtractor) { + Map seen = Maps.newConcurrentMap(); + return t -> seen.putIfAbsent(keyExtractor.apply(t), Boolean.TRUE) == null; + } + + private Snapshot latestSnapshot(Table table, String branch) { + if (branch == null) { + return table.currentSnapshot(); + } + return table.snapshot(branch); + } + + private Map lastCommittedOffsetsForTable(Table table, String branch) { + Snapshot snapshot = latestSnapshot(table, branch); + while (snapshot != null) { + Map summary = snapshot.summary(); + String value = summary.get(snapshotOffsetsProp); + if (value != null) { + TypeReference> typeRef = new TypeReference>() {}; + try { + return MAPPER.readValue(value, typeRef); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + return ImmutableMap.of(); + } + + @Override + void stop() { + exec.shutdownNow(); + + // ensure coordinator tasks are shut down, else cause the sink worker to fail + try { + if (!exec.awaitTermination(1, TimeUnit.MINUTES)) { + throw new RuntimeException("Timed out waiting for coordinator shutdown"); + } + } catch (InterruptedException e) { + throw new RuntimeException("Interrupted while waiting for coordinator shutdown", e); + } + + super.stop(); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java new file mode 100644 index 000000000000..6a31b17fc606 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.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.connect.channel; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class CoordinatorThread extends Thread { + private static final Logger LOG = LoggerFactory.getLogger(CoordinatorThread.class); + private static final String THREAD_NAME = "iceberg-coord"; + + private Coordinator coordinator; + private volatile boolean terminated; + + CoordinatorThread(Coordinator coordinator) { + super(THREAD_NAME); + this.coordinator = coordinator; + } + + @Override + public void run() { + try { + coordinator.start(); + } catch (Exception e) { + LOG.error("Coordinator error during start, exiting thread", e); + terminated = true; + } + + while (!terminated) { + try { + coordinator.process(); + } catch (Exception e) { + LOG.error("Coordinator error during process, exiting thread", e); + terminated = true; + } + } + + try { + coordinator.stop(); + } catch (Exception e) { + LOG.error("Coordinator error during stop, ignoring", e); + } + coordinator = null; + } + + boolean isTerminated() { + return terminated; + } + + void terminate() { + terminated = true; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java new file mode 100644 index 000000000000..87a93d058509 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Envelope.java @@ -0,0 +1,45 @@ +/* + * 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.connect.channel; + +import org.apache.iceberg.connect.events.Event; + +class Envelope { + private final Event event; + private final int partition; + private final long offset; + + Envelope(Event event, int partition, long offset) { + this.event = event; + this.partition = partition; + this.offset = offset; + } + + Event event() { + return event; + } + + int partition() { + return partition; + } + + long offset() { + return offset; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java new file mode 100644 index 000000000000..fd5d27ae34e2 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaClientFactory.java @@ -0,0 +1,68 @@ +/* + * 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.connect.channel; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; + +class KafkaClientFactory { + private final Map kafkaProps; + + KafkaClientFactory(Map kafkaProps) { + this.kafkaProps = kafkaProps; + } + + Producer createProducer(String transactionalId) { + Map producerProps = Maps.newHashMap(kafkaProps); + producerProps.putIfAbsent(ProducerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString()); + producerProps.put(ProducerConfig.TRANSACTIONAL_ID_CONFIG, transactionalId); + KafkaProducer result = + new KafkaProducer<>(producerProps, new StringSerializer(), new ByteArraySerializer()); + result.initTransactions(); + return result; + } + + Consumer createConsumer(String consumerGroupId) { + Map consumerProps = Maps.newHashMap(kafkaProps); + consumerProps.putIfAbsent(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "latest"); + consumerProps.putIfAbsent(ConsumerConfig.CLIENT_ID_CONFIG, UUID.randomUUID().toString()); + consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false); + consumerProps.put(ConsumerConfig.ISOLATION_LEVEL_CONFIG, "read_committed"); + consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, consumerGroupId); + return new KafkaConsumer<>( + consumerProps, new StringDeserializer(), new ByteArrayDeserializer()); + } + + Admin createAdmin() { + Map adminProps = Maps.newHashMap(kafkaProps); + return Admin.create(adminProps); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java new file mode 100644 index 000000000000..be51fff8bfbc --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/KafkaUtils.java @@ -0,0 +1,66 @@ +/* + * 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.connect.channel; + +import java.util.concurrent.ExecutionException; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.ConsumerGroupDescription; +import org.apache.kafka.clients.admin.DescribeConsumerGroupsResult; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.sink.SinkTaskContext; + +class KafkaUtils { + + private static final String CONTEXT_CLASS_NAME = + "org.apache.kafka.connect.runtime.WorkerSinkTaskContext"; + + static ConsumerGroupDescription consumerGroupDescription(String consumerGroupId, Admin admin) { + try { + DescribeConsumerGroupsResult result = + admin.describeConsumerGroups(ImmutableList.of(consumerGroupId)); + return result.describedGroups().get(consumerGroupId).get(); + + } catch (InterruptedException | ExecutionException e) { + throw new ConnectException( + "Cannot retrieve members for consumer group: " + consumerGroupId, e); + } + } + + static ConsumerGroupMetadata consumerGroupMetadata(SinkTaskContext context) { + return kafkaConsumer(context).groupMetadata(); + } + + @SuppressWarnings("unchecked") + private static Consumer kafkaConsumer(SinkTaskContext context) { + String contextClassName = context.getClass().getName(); + try { + return ((Consumer) + DynFields.builder().hiddenImpl(CONTEXT_CLASS_NAME, "consumer").build(context).get()); + } catch (Exception e) { + throw new ConnectException( + "Unable to retrieve consumer from context: " + contextClassName, e); + } + } + + private KafkaUtils() {} +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java new file mode 100644 index 000000000000..72a362ceacb0 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/NotRunningException.java @@ -0,0 +1,25 @@ +/* + * 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.connect.channel; + +public class NotRunningException extends RuntimeException { + public NotRunningException(String msg) { + super(msg); + } +} 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 new file mode 100644 index 000000000000..7555b216cd45 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/Worker.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.connect.channel; + +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.data.Offset; +import org.apache.iceberg.connect.data.SinkWriter; +import org.apache.iceberg.connect.data.SinkWriterResult; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; + +class Worker extends Channel { + + private final IcebergSinkConfig config; + private final SinkTaskContext context; + private final SinkWriter sinkWriter; + + Worker( + IcebergSinkConfig config, + KafkaClientFactory clientFactory, + SinkWriter sinkWriter, + SinkTaskContext context) { + // pass transient consumer group ID to which we never commit offsets + super( + "worker", + IcebergSinkConfig.DEFAULT_CONTROL_GROUP_PREFIX + UUID.randomUUID(), + config, + clientFactory, + context); + + this.config = config; + this.context = context; + this.sinkWriter = sinkWriter; + } + + void process() { + consumeAvailable(Duration.ZERO); + } + + @Override + protected boolean receive(Envelope envelope) { + Event event = envelope.event(); + if (event.payload().type() != PayloadType.START_COMMIT) { + return false; + } + + SinkWriterResult results = sinkWriter.completeWrite(); + + // include all assigned topic partitions even if no messages were read + // from a partition, as the coordinator will use that to determine + // when all data for a commit has been received + List assignments = + context.assignment().stream() + .map( + tp -> { + Offset offset = results.sourceOffsets().get(tp); + if (offset == null) { + offset = Offset.NULL_OFFSET; + } + return new TopicPartitionOffset( + tp.topic(), tp.partition(), offset.offset(), offset.timestamp()); + }) + .collect(Collectors.toList()); + + UUID commitId = ((StartCommit) event.payload()).commitId(); + + List events = + results.writerResults().stream() + .map( + writeResult -> + new Event( + config.connectGroupId(), + new DataWritten( + writeResult.partitionStruct(), + commitId, + TableReference.of(config.catalogName(), writeResult.tableIdentifier()), + writeResult.dataFiles(), + writeResult.deleteFiles()))) + .collect(Collectors.toList()); + + Event readyEvent = new Event(config.connectGroupId(), new DataComplete(commitId, assignments)); + events.add(readyEvent); + + send(events, results.sourceOffsets()); + + return true; + } + + @Override + void stop() { + super.stop(); + sinkWriter.close(); + } + + void save(Collection sinkRecords) { + sinkWriter.save(sinkRecords); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java index 27ffc4de9973..6df6b091510b 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java @@ -32,16 +32,16 @@ import org.apache.kafka.connect.errors.DataException; import org.apache.kafka.connect.sink.SinkRecord; -public class IcebergWriter implements RecordWriter { +class IcebergWriter implements RecordWriter { private final Table table; private final String tableName; private final IcebergSinkConfig config; - private final List writerResults; + private final List writerResults; private RecordConverter recordConverter; private TaskWriter writer; - public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { + IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { this.table = table; this.tableName = tableName; this.config = config; @@ -50,7 +50,7 @@ public IcebergWriter(Table table, String tableName, IcebergSinkConfig config) { } private void initNewWriter() { - this.writer = Utilities.createTableWriter(table, tableName, config); + this.writer = RecordUtils.createTableWriter(table, tableName, config); this.recordConverter = new RecordConverter(table, config); } @@ -102,7 +102,7 @@ private void flush() { } writerResults.add( - new WriterResult( + new IcebergWriterResult( TableIdentifier.parse(tableName), Arrays.asList(writeResult.dataFiles()), Arrays.asList(writeResult.deleteFiles()), @@ -110,10 +110,10 @@ private void flush() { } @Override - public List complete() { + public List complete() { flush(); - List result = Lists.newArrayList(writerResults); + List result = Lists.newArrayList(writerResults); writerResults.clear(); return result; diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java index 47dcddcb9925..92f5af2d7a87 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterFactory.java @@ -40,20 +40,19 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class IcebergWriterFactory { +class IcebergWriterFactory { private static final Logger LOG = LoggerFactory.getLogger(IcebergWriterFactory.class); private final Catalog catalog; private final IcebergSinkConfig config; - public IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { + IcebergWriterFactory(Catalog catalog, IcebergSinkConfig config) { this.catalog = catalog; this.config = config; } - public RecordWriter createWriter( - String tableName, SinkRecord sample, boolean ignoreMissingTable) { + RecordWriter createWriter(String tableName, SinkRecord sample, boolean ignoreMissingTable) { TableIdentifier identifier = TableIdentifier.parse(tableName); Table table; try { diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/WriterResult.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java similarity index 96% rename from kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/WriterResult.java rename to kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java index cb3a700da247..58695a5572b5 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/WriterResult.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriterResult.java @@ -24,14 +24,14 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.types.Types.StructType; -public class WriterResult { +public class IcebergWriterResult { private final TableIdentifier tableIdentifier; private final List dataFiles; private final List deleteFiles; private final StructType partitionStruct; - public WriterResult( + public IcebergWriterResult( TableIdentifier tableIdentifier, List dataFiles, List deleteFiles, diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java index 64ca44f03209..a7d2c90972d7 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/NoOpWriter.java @@ -19,6 +19,7 @@ package org.apache.iceberg.connect.data; import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.kafka.connect.sink.SinkRecord; class NoOpWriter implements RecordWriter { @@ -28,9 +29,9 @@ public void write(SinkRecord record) { } @Override - public List complete() { + public List complete() { // NO-OP - return null; + return ImmutableList.of(); } @Override diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java new file mode 100644 index 000000000000..c4522a40711b --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Offset.java @@ -0,0 +1,54 @@ +/* + * 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.connect.data; + +import java.time.OffsetDateTime; +import java.util.Objects; + +public class Offset implements Comparable { + + public static final Offset NULL_OFFSET = new Offset(null, null); + + private final Long offset; + private final OffsetDateTime timestamp; + + public Offset(Long offset, OffsetDateTime timestamp) { + this.offset = offset; + this.timestamp = timestamp; + } + + public Long offset() { + return offset; + } + + public OffsetDateTime timestamp() { + return timestamp; + } + + @Override + public int compareTo(Offset other) { + if (Objects.equals(this.offset, other.offset)) { + return 0; + } + if (this.offset == null || (other.offset != null && other.offset > this.offset)) { + return -1; + } + return 1; + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java similarity index 68% rename from kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java rename to kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java index 4ff83f777527..5ac930739738 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/Utilities.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordUtils.java @@ -18,25 +18,14 @@ */ package org.apache.iceberg.connect.data; -import java.io.IOException; -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; import java.util.List; import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.stream.Collectors; -import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.common.DynClasses; -import org.apache.iceberg.common.DynConstructors; -import org.apache.iceberg.common.DynMethods; -import org.apache.iceberg.common.DynMethods.BoundMethod; import org.apache.iceberg.connect.IcebergSinkConfig; import org.apache.iceberg.data.GenericAppenderFactory; import org.apache.iceberg.data.Record; @@ -46,7 +35,6 @@ import org.apache.iceberg.io.UnpartitionedWriter; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Splitter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.relocated.com.google.common.primitives.Ints; @@ -55,71 +43,11 @@ import org.apache.iceberg.util.PropertyUtil; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Struct; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; -public class Utilities { - - private static final Logger LOG = LoggerFactory.getLogger(Utilities.class.getName()); - private static final List HADOOP_CONF_FILES = - ImmutableList.of("core-site.xml", "hdfs-site.xml", "hive-site.xml"); - - public static Catalog loadCatalog(IcebergSinkConfig config) { - return CatalogUtil.buildIcebergCatalog( - config.catalogName(), config.catalogProps(), loadHadoopConfig(config)); - } - - // use reflection here to avoid requiring Hadoop as a dependency - private static Object loadHadoopConfig(IcebergSinkConfig config) { - Class configClass = - DynClasses.builder() - .impl("org.apache.hadoop.hdfs.HdfsConfiguration") - .impl("org.apache.hadoop.conf.Configuration") - .orNull() - .build(); - - if (configClass == null) { - LOG.info("Hadoop not found on classpath, not creating Hadoop config"); - return null; - } - - try { - Object result = DynConstructors.builder().hiddenImpl(configClass).build().newInstance(); - BoundMethod addResourceMethod = - DynMethods.builder("addResource").impl(configClass, URL.class).build(result); - BoundMethod setMethod = - DynMethods.builder("set").impl(configClass, String.class, String.class).build(result); - - // load any config files in the specified config directory - String hadoopConfDir = config.hadoopConfDir(); - if (hadoopConfDir != null) { - HADOOP_CONF_FILES.forEach( - confFile -> { - Path path = Paths.get(hadoopConfDir, confFile); - if (Files.exists(path)) { - try { - addResourceMethod.invoke(path.toUri().toURL()); - } catch (IOException e) { - LOG.warn("Error adding Hadoop resource {}, resource was not added", path, e); - } - } - }); - } - - // set any Hadoop properties specified in the sink config - config.hadoopProps().forEach(setMethod::invoke); - - LOG.info("Hadoop config initialized: {}", configClass.getName()); - return result; - } catch (Exception e) { - LOG.warn( - "Hadoop found on classpath but could not create config, proceeding without config", e); - } - return null; - } +class RecordUtils { @SuppressWarnings("unchecked") - public static Object extractFromRecordValue(Object recordValue, String fieldName) { + static Object extractFromRecordValue(Object recordValue, String fieldName) { List fields = Splitter.on('.').splitToList(fieldName); if (recordValue instanceof Struct) { return valueFromStruct((Struct) recordValue, fields); @@ -243,5 +171,5 @@ public static TaskWriter createTableWriter( return writer; } - private Utilities() {} + private RecordUtils() {} } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java index 0b4d7566eab7..56438dde2e40 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/RecordWriter.java @@ -21,11 +21,11 @@ import java.util.List; import org.apache.kafka.connect.sink.SinkRecord; -public interface RecordWriter extends Cloneable { +interface RecordWriter extends Cloneable { void write(SinkRecord record); - List complete(); + List complete(); void close(); } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java new file mode 100644 index 000000000000..35a2957f0122 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.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.connect.data; + +import java.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkRecord; + +public class SinkWriter { + private final IcebergSinkConfig config; + private final IcebergWriterFactory writerFactory; + private final Map writers; + private final Map sourceOffsets; + + public SinkWriter(Catalog catalog, IcebergSinkConfig config) { + this.config = config; + this.writerFactory = new IcebergWriterFactory(catalog, config); + this.writers = Maps.newHashMap(); + this.sourceOffsets = Maps.newHashMap(); + } + + public void close() { + writers.values().forEach(RecordWriter::close); + } + + public SinkWriterResult completeWrite() { + List writerResults = + writers.values().stream() + .flatMap(writer -> writer.complete().stream()) + .collect(Collectors.toList()); + Map offsets = Maps.newHashMap(sourceOffsets); + + writers.clear(); + sourceOffsets.clear(); + + return new SinkWriterResult(writerResults, offsets); + } + + public void save(Collection sinkRecords) { + sinkRecords.forEach(this::save); + } + + private void save(SinkRecord record) { + // the consumer stores the offsets that corresponds to the next record to consume, + // so increment the record offset by one + OffsetDateTime timestamp = + record.timestamp() == null + ? null + : OffsetDateTime.ofInstant(Instant.ofEpochMilli(record.timestamp()), ZoneOffset.UTC); + sourceOffsets.put( + new TopicPartition(record.topic(), record.kafkaPartition()), + new Offset(record.kafkaOffset() + 1, timestamp)); + + if (config.dynamicTablesEnabled()) { + routeRecordDynamically(record); + } else { + routeRecordStatically(record); + } + } + + private void routeRecordStatically(SinkRecord record) { + String routeField = config.tablesRouteField(); + + if (routeField == null) { + // route to all tables + config + .tables() + .forEach( + tableName -> { + writerForTable(tableName, record, false).write(record); + }); + + } else { + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + config + .tables() + .forEach( + tableName -> { + Pattern regex = config.tableConfig(tableName).routeRegex(); + if (regex != null && regex.matcher(routeValue).matches()) { + writerForTable(tableName, record, false).write(record); + } + }); + } + } + } + + private void routeRecordDynamically(SinkRecord record) { + String routeField = config.tablesRouteField(); + Preconditions.checkNotNull(routeField, "Route field cannot be null with dynamic routing"); + + String routeValue = extractRouteValue(record.value(), routeField); + if (routeValue != null) { + String tableName = routeValue.toLowerCase(); + writerForTable(tableName, record, true).write(record); + } + } + + private String extractRouteValue(Object recordValue, String routeField) { + if (recordValue == null) { + return null; + } + Object routeValue = RecordUtils.extractFromRecordValue(recordValue, routeField); + return routeValue == null ? null : routeValue.toString(); + } + + private RecordWriter writerForTable( + String tableName, SinkRecord sample, boolean ignoreMissingTable) { + return writers.computeIfAbsent( + tableName, notUsed -> writerFactory.createWriter(tableName, sample, ignoreMissingTable)); + } +} diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java new file mode 100644 index 000000000000..ef899102bb64 --- /dev/null +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriterResult.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.connect.data; + +import java.util.List; +import java.util.Map; +import org.apache.kafka.common.TopicPartition; + +public class SinkWriterResult { + private final List writerResults; + private final Map sourceOffsets; + + public SinkWriterResult( + List writerResults, Map sourceOffsets) { + this.writerResults = writerResults; + this.sourceOffsets = sourceOffsets; + } + + public List writerResults() { + return writerResults; + } + + public Map sourceOffsets() { + return sourceOffsets; + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/UtilitiesTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/CatalogUtilsTest.java similarity index 60% rename from kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/UtilitiesTest.java rename to kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/CatalogUtilsTest.java index cfa1709da744..ce92b3efc3ed 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/UtilitiesTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/CatalogUtilsTest.java @@ -16,7 +16,7 @@ * specific language governing permissions and limitations * under the License. */ -package org.apache.iceberg.connect.data; +package org.apache.iceberg.connect; import static org.assertj.core.api.Assertions.assertThat; @@ -27,19 +27,15 @@ import java.util.Map; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.connect.IcebergSinkConfig; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.inmemory.InMemoryCatalog; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.kafka.connect.data.Schema; -import org.apache.kafka.connect.data.SchemaBuilder; -import org.apache.kafka.connect.data.Struct; 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 UtilitiesTest { +public class CatalogUtilsTest { private static final String HADOOP_CONF_TEMPLATE = "%s%s"; @@ -68,7 +64,7 @@ public void testLoadCatalogNoHadoopDir() { "iceberg.catalog.catalog-impl", TestCatalog.class.getName()); IcebergSinkConfig config = new IcebergSinkConfig(props); - Catalog result = Utilities.loadCatalog(config); + Catalog result = CatalogUtils.loadCatalog(config); assertThat(result).isInstanceOf(TestCatalog.class); @@ -102,7 +98,7 @@ public void testLoadCatalogWithHadoopDir(String confFile) throws IOException { "iceberg.catalog.catalog-impl", TestCatalog.class.getName()); IcebergSinkConfig config = new IcebergSinkConfig(props); - Catalog result = Utilities.loadCatalog(config); + Catalog result = CatalogUtils.loadCatalog(config); assertThat(result).isInstanceOf(TestCatalog.class); @@ -118,66 +114,4 @@ public void testLoadCatalogWithHadoopDir(String confFile) throws IOException { // check that core-site.xml was loaded assertThat(conf.get("foo")).isEqualTo("bar"); } - - @Test - public void testExtractFromRecordValueStruct() { - Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); - Struct val = new Struct(valSchema).put("key", 123L); - Object result = Utilities.extractFromRecordValue(val, "key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueStructNested() { - Schema idSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); - Schema dataSchema = SchemaBuilder.struct().field("id", idSchema).build(); - Schema valSchema = SchemaBuilder.struct().field("data", dataSchema).build(); - - Struct id = new Struct(idSchema).put("key", 123L); - Struct data = new Struct(dataSchema).put("id", id); - Struct val = new Struct(valSchema).put("data", data); - - Object result = Utilities.extractFromRecordValue(val, "data.id.key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueStructNull() { - Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); - Struct val = new Struct(valSchema).put("key", 123L); - - Object result = Utilities.extractFromRecordValue(val, ""); - assertThat(result).isNull(); - - result = Utilities.extractFromRecordValue(val, "xkey"); - assertThat(result).isNull(); - } - - @Test - public void testExtractFromRecordValueMap() { - Map val = ImmutableMap.of("key", 123L); - Object result = Utilities.extractFromRecordValue(val, "key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueMapNested() { - Map id = ImmutableMap.of("key", 123L); - Map data = ImmutableMap.of("id", id); - Map val = ImmutableMap.of("data", data); - - Object result = Utilities.extractFromRecordValue(val, "data.id.key"); - assertThat(result).isEqualTo(123L); - } - - @Test - public void testExtractFromRecordValueMapNull() { - Map val = ImmutableMap.of("key", 123L); - - Object result = Utilities.extractFromRecordValue(val, ""); - assertThat(result).isNull(); - - result = Utilities.extractFromRecordValue(val, "xkey"); - assertThat(result).isNull(); - } } diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java new file mode 100644 index 000000000000..e6ffefbd9799 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/ChannelTestBase.java @@ -0,0 +1,131 @@ +/* + * 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.connect.channel; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyCollection; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.TableSinkConfig; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.DescribeTopicsResult; +import org.apache.kafka.clients.admin.TopicDescription; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.common.KafkaFuture; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.TopicPartitionInfo; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +public class ChannelTestBase { + protected static final String SRC_TOPIC_NAME = "src-topic"; + protected static final String CTL_TOPIC_NAME = "ctl-topic"; + protected static final String CONNECT_CONSUMER_GROUP_ID = "cg-connect"; + protected InMemoryCatalog catalog; + protected Table table; + protected IcebergSinkConfig config; + protected KafkaClientFactory clientFactory; + protected MockProducer producer; + protected MockConsumer consumer; + protected Admin admin; + + private InMemoryCatalog initInMemoryCatalog() { + InMemoryCatalog inMemoryCatalog = new InMemoryCatalog(); + inMemoryCatalog.initialize(null, ImmutableMap.of()); + return inMemoryCatalog; + } + + protected static final Namespace NAMESPACE = Namespace.of("db"); + protected static final String TABLE_NAME = "tbl"; + protected static final TableIdentifier TABLE_IDENTIFIER = + TableIdentifier.of(NAMESPACE, TABLE_NAME); + protected static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get())); + + protected static final String COMMIT_ID_SNAPSHOT_PROP = "kafka.connect.commit-id"; + protected static final String OFFSETS_SNAPSHOT_PROP = + String.format("kafka.connect.offsets.%s.%s", CTL_TOPIC_NAME, CONNECT_CONSUMER_GROUP_ID); + protected static final String VALID_THROUGH_TS_SNAPSHOT_PROP = "kafka.connect.valid-through-ts"; + + @BeforeEach + @SuppressWarnings("deprecation") + public void before() { + catalog = initInMemoryCatalog(); + catalog.createNamespace(NAMESPACE); + table = catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + + config = mock(IcebergSinkConfig.class); + when(config.controlTopic()).thenReturn(CTL_TOPIC_NAME); + when(config.commitThreads()).thenReturn(1); + when(config.connectGroupId()).thenReturn(CONNECT_CONSUMER_GROUP_ID); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + + TopicPartitionInfo partitionInfo = mock(TopicPartitionInfo.class); + when(partitionInfo.partition()).thenReturn(0); + TopicDescription topicDesc = + new TopicDescription(SRC_TOPIC_NAME, false, ImmutableList.of(partitionInfo)); + DescribeTopicsResult describeResult = mock(DescribeTopicsResult.class); + when(describeResult.values()) + .thenReturn(ImmutableMap.of(SRC_TOPIC_NAME, KafkaFuture.completedFuture(topicDesc))); + + admin = mock(Admin.class); + when(admin.describeTopics(anyCollection())).thenReturn(describeResult); + + producer = new MockProducer<>(false, new StringSerializer(), new ByteArraySerializer()); + producer.initTransactions(); + + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + + clientFactory = mock(KafkaClientFactory.class); + when(clientFactory.createProducer(any())).thenReturn(producer); + when(clientFactory.createConsumer(any())).thenReturn(consumer); + when(clientFactory.createAdmin()).thenReturn(admin); + } + + @AfterEach + public void after() throws IOException { + catalog.close(); + } + + protected void initConsumer() { + TopicPartition tp = new TopicPartition(CTL_TOPIC_NAME, 0); + consumer.rebalance(ImmutableList.of(tp)); + consumer.updateBeginningOffsets(ImmutableMap.of(tp, 0L)); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.java new file mode 100644 index 000000000000..a9fe1ad099cb --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitStateTest.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.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.OffsetDateTime; +import java.util.UUID; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.Payload; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +public class CommitStateTest { + @Test + public void testIsCommitReady() { + TopicPartitionOffset tp = mock(TopicPartitionOffset.class); + + CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); + commitState.startNewCommit(); + + DataComplete payload1 = mock(DataComplete.class); + when(payload1.commitId()).thenReturn(commitState.currentCommitId()); + when(payload1.assignments()).thenReturn(ImmutableList.of(tp, tp)); + + DataComplete payload2 = mock(DataComplete.class); + when(payload2.commitId()).thenReturn(commitState.currentCommitId()); + when(payload2.assignments()).thenReturn(ImmutableList.of(tp)); + + DataComplete payload3 = mock(DataComplete.class); + when(payload3.commitId()).thenReturn(UUID.randomUUID()); + when(payload3.assignments()).thenReturn(ImmutableList.of(tp)); + + commitState.addReady(wrapInEnvelope(payload1)); + commitState.addReady(wrapInEnvelope(payload2)); + commitState.addReady(wrapInEnvelope(payload3)); + + assertThat(commitState.isCommitReady(3)).isTrue(); + assertThat(commitState.isCommitReady(4)).isFalse(); + } + + @Test + public void testGetValidThroughTs() { + DataComplete payload1 = mock(DataComplete.class); + TopicPartitionOffset tp1 = mock(TopicPartitionOffset.class); + OffsetDateTime ts1 = EventTestUtil.now(); + when(tp1.timestamp()).thenReturn(ts1); + + TopicPartitionOffset tp2 = mock(TopicPartitionOffset.class); + OffsetDateTime ts2 = ts1.plusSeconds(1); + when(tp2.timestamp()).thenReturn(ts2); + when(payload1.assignments()).thenReturn(ImmutableList.of(tp1, tp2)); + + DataComplete payload2 = mock(DataComplete.class); + TopicPartitionOffset tp3 = mock(TopicPartitionOffset.class); + OffsetDateTime ts3 = ts1.plusSeconds(2); + when(tp3.timestamp()).thenReturn(ts3); + when(payload2.assignments()).thenReturn(ImmutableList.of(tp3)); + + CommitState commitState = new CommitState(mock(IcebergSinkConfig.class)); + commitState.startNewCommit(); + + commitState.addReady(wrapInEnvelope(payload1)); + commitState.addReady(wrapInEnvelope(payload2)); + + assertThat(commitState.validThroughTs(false)).isEqualTo(ts1); + assertThat(commitState.validThroughTs(true)).isNull(); + + // null timestamp for one, so should not set a valid-through timestamp + DataComplete payload3 = mock(DataComplete.class); + TopicPartitionOffset tp4 = mock(TopicPartitionOffset.class); + when(tp4.timestamp()).thenReturn(null); + when(payload3.assignments()).thenReturn(ImmutableList.of(tp4)); + + commitState.addReady(wrapInEnvelope(payload3)); + + assertThat(commitState.validThroughTs(false)).isNull(); + assertThat(commitState.validThroughTs(true)).isNull(); + } + + private Envelope wrapInEnvelope(Payload payload) { + Event event = mock(Event.class); + when(event.payload()).thenReturn(payload); + return new Envelope(event, 0, 0); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.java new file mode 100644 index 000000000000..7c8ccf8ef669 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CommitterImplTest.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.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Optional; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.kafka.clients.admin.MemberAssignment; +import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.common.TopicPartition; +import org.junit.jupiter.api.Test; + +public class CommitterImplTest { + + @Test + public void testIsLeader() { + CommitterImpl committer = new CommitterImpl(); + + MemberAssignment assignment1 = + new MemberAssignment( + ImmutableSet.of(new TopicPartition("topic1", 0), new TopicPartition("topic2", 1))); + MemberDescription member1 = + new MemberDescription(null, Optional.empty(), null, null, assignment1); + + MemberAssignment assignment2 = + new MemberAssignment( + ImmutableSet.of(new TopicPartition("topic2", 0), new TopicPartition("topic1", 1))); + MemberDescription member2 = + new MemberDescription(null, Optional.empty(), null, null, assignment2); + + List members = ImmutableList.of(member1, member2); + + List assignments = + ImmutableList.of(new TopicPartition("topic2", 1), new TopicPartition("topic1", 0)); + assertThat(committer.isLeader(members, assignments)).isTrue(); + + assignments = + ImmutableList.of(new TopicPartition("topic2", 0), new TopicPartition("topic1", 1)); + assertThat(committer.isLeader(members, assignments)).isFalse(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java new file mode 100644 index 000000000000..9c0b8122ae42 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java @@ -0,0 +1,213 @@ +/* + * 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.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.OffsetDateTime; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.CommitComplete; +import org.apache.iceberg.connect.events.CommitToTable; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +import org.apache.iceberg.connect.events.TableReference; +import org.apache.iceberg.connect.events.TopicPartitionOffset; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types.StructType; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; + +public class CoordinatorTest extends ChannelTestBase { + + @Test + public void testCommitAppend() { + Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); + + OffsetDateTime ts = EventTestUtil.now(); + UUID commitId = + coordinatorTest(ImmutableList.of(EventTestUtil.createDataFile()), ImmutableList.of(), ts); + table.refresh(); + + assertThat(producer.history()).hasSize(3); + assertCommitTable(1, commitId, ts); + assertCommitComplete(2, commitId, ts); + + List snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(1, snapshots.size()); + + Snapshot snapshot = snapshots.get(0); + Assertions.assertEquals(DataOperations.APPEND, snapshot.operation()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); + Assertions.assertEquals(0, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); + + Map summary = snapshot.summary(); + Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); + Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); + Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + } + + @Test + public void testCommitDelta() { + OffsetDateTime ts = EventTestUtil.now(); + UUID commitId = + coordinatorTest( + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(EventTestUtil.createDeleteFile()), + ts); + + assertThat(producer.history()).hasSize(3); + assertCommitTable(1, commitId, ts); + assertCommitComplete(2, commitId, ts); + + List snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(1, snapshots.size()); + + Snapshot snapshot = snapshots.get(0); + Assertions.assertEquals(DataOperations.OVERWRITE, snapshot.operation()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); + Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); + + Map summary = snapshot.summary(); + Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); + Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); + Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + } + + @Test + public void testCommitNoFiles() { + OffsetDateTime ts = EventTestUtil.now(); + UUID commitId = coordinatorTest(ImmutableList.of(), ImmutableList.of(), ts); + + assertThat(producer.history()).hasSize(2); + assertCommitComplete(1, commitId, ts); + + List snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(0, snapshots.size()); + } + + @Test + public void testCommitError() { + // this spec isn't registered with the table + PartitionSpec badPartitionSpec = + PartitionSpec.builderFor(SCHEMA).withSpecId(1).identity("id").build(); + DataFile badDataFile = + DataFiles.builder(badPartitionSpec) + .withPath(UUID.randomUUID() + ".parquet") + .withFormat(FileFormat.PARQUET) + .withFileSizeInBytes(100L) + .withRecordCount(5) + .build(); + + coordinatorTest(ImmutableList.of(badDataFile), ImmutableList.of(), null); + + // no commit messages sent + assertThat(producer.history()).hasSize(1); + + List snapshots = ImmutableList.copyOf(table.snapshots()); + Assertions.assertEquals(0, snapshots.size()); + } + + private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { + byte[] bytes = producer.history().get(idx).value(); + Event commitTable = AvroUtil.decode(bytes); + assertThat(commitTable.type()).isEqualTo(PayloadType.COMMIT_TO_TABLE); + CommitToTable commitToTablePayload = (CommitToTable) commitTable.payload(); + assertThat(commitToTablePayload.commitId()).isEqualTo(commitId); + assertThat(commitToTablePayload.tableReference().identifier().toString()) + .isEqualTo(TABLE_IDENTIFIER.toString()); + assertThat(commitToTablePayload.validThroughTs()).isEqualTo(ts); + } + + private void assertCommitComplete(int idx, UUID commitId, OffsetDateTime ts) { + byte[] bytes = producer.history().get(idx).value(); + Event commitComplete = AvroUtil.decode(bytes); + assertThat(commitComplete.type()).isEqualTo(PayloadType.COMMIT_COMPLETE); + CommitComplete commitCompletePayload = (CommitComplete) commitComplete.payload(); + assertThat(commitCompletePayload.commitId()).isEqualTo(commitId); + assertThat(commitCompletePayload.validThroughTs()).isEqualTo(ts); + } + + private UUID coordinatorTest( + List dataFiles, List deleteFiles, OffsetDateTime ts) { + when(config.commitIntervalMs()).thenReturn(0); + when(config.commitTimeoutMs()).thenReturn(Integer.MAX_VALUE); + + SinkTaskContext context = mock(SinkTaskContext.class); + Coordinator coordinator = + new Coordinator(catalog, config, ImmutableList.of(), clientFactory, context); + coordinator.start(); + + // init consumer after subscribe() + initConsumer(); + + coordinator.process(); + + assertThat(producer.transactionCommitted()).isTrue(); + assertThat(producer.history()).hasSize(1); + + byte[] bytes = producer.history().get(0).value(); + Event commitRequest = AvroUtil.decode(bytes); + assertThat(commitRequest.type()).isEqualTo(PayloadType.START_COMMIT); + + UUID commitId = ((StartCommit) commitRequest.payload()).commitId(); + + Event commitResponse = + new Event( + config.connectGroupId(), + new DataWritten( + StructType.of(), + commitId, + new TableReference("catalog", ImmutableList.of("db"), "tbl"), + dataFiles, + deleteFiles)); + bytes = AvroUtil.encode(commitResponse); + consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 1, "key", bytes)); + + Event commitReady = + new Event( + config.connectGroupId(), + new DataComplete( + commitId, ImmutableList.of(new TopicPartitionOffset("topic", 1, 1L, ts)))); + bytes = AvroUtil.encode(commitReady); + consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 2, "key", bytes)); + + when(config.commitIntervalMs()).thenReturn(0); + + coordinator.process(); + + return commitId; + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java new file mode 100644 index 000000000000..da0d881f8927 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorThreadTest.java @@ -0,0 +1,48 @@ +/* + * 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.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.timeout; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import org.junit.jupiter.api.Test; + +public class CoordinatorThreadTest { + + @Test + public void testRun() { + Coordinator coordinator = mock(Coordinator.class); + CoordinatorThread coordinatorThread = new CoordinatorThread(coordinator); + + coordinatorThread.start(); + + verify(coordinator, timeout(1000)).start(); + verify(coordinator, timeout(1000).atLeast(1)).process(); + verify(coordinator, times(0)).stop(); + assertThat(coordinatorThread.isTerminated()).isFalse(); + + coordinatorThread.terminate(); + + verify(coordinator, timeout(1000)).stop(); + assertThat(coordinatorThread.isTerminated()).isTrue(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java new file mode 100644 index 000000000000..8c3625b74a5d --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/EventTestUtil.java @@ -0,0 +1,98 @@ +/* + * 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.connect.channel; + +import java.nio.ByteBuffer; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; + +class EventTestUtil { + private EventTestUtil() {} + + static final Schema SCHEMA = + new Schema(ImmutableList.of(Types.NestedField.required(1, "id", Types.LongType.get()))); + + static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("id").build(); + + static final SortOrder ORDER = + SortOrder.builderFor(SCHEMA).sortBy("id", SortDirection.ASC, NullOrder.NULLS_FIRST).build(); + + static final Metrics METRICS = + new Metrics( + 1L, + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, 1L), + ImmutableMap.of(1, ByteBuffer.wrap(new byte[10])), + ImmutableMap.of(1, ByteBuffer.wrap(new byte[10]))); + + static OffsetDateTime now() { + return OffsetDateTime.now(ZoneOffset.UTC).truncatedTo(ChronoUnit.MICROS); + } + + static DataFile createDataFile() { + PartitionData data = new PartitionData(SPEC.partitionType()); + data.set(0, 1L); + + return DataFiles.builder(SPEC) + .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[] {0})) + .withFileSizeInBytes(100L) + .withFormat(FileFormat.PARQUET) + .withMetrics(METRICS) + .withPartition(data) + .withPath("path/to/file.parquet") + .withSortOrder(ORDER) + .withSplitOffsets(ImmutableList.of(4L)) + .build(); + } + + static DeleteFile createDeleteFile() { + PartitionData data = new PartitionData(SPEC.partitionType()); + data.set(0, 1L); + + return FileMetadata.deleteFileBuilder(SPEC) + .ofEqualityDeletes(1) + .withEncryptionKeyMetadata(ByteBuffer.wrap(new byte[] {0})) + .withFileSizeInBytes(100L) + .withFormat(FileFormat.PARQUET) + .withMetrics(METRICS) + .withPartition(data) + .withPath("path/to/file.parquet") + .withSortOrder(ORDER) + .withSplitOffsets(ImmutableList.of(4L)) + .build(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.java new file mode 100644 index 000000000000..577c28fe6375 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/WorkerTest.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.connect.channel; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.mockStatic; +import static org.mockito.Mockito.when; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.data.IcebergWriterResult; +import org.apache.iceberg.connect.data.Offset; +import org.apache.iceberg.connect.data.SinkWriter; +import org.apache.iceberg.connect.data.SinkWriterResult; +import org.apache.iceberg.connect.events.AvroUtil; +import org.apache.iceberg.connect.events.DataComplete; +import org.apache.iceberg.connect.events.DataWritten; +import org.apache.iceberg.connect.events.Event; +import org.apache.iceberg.connect.events.PayloadType; +import org.apache.iceberg.connect.events.StartCommit; +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.types.Types.StructType; +import org.apache.kafka.clients.consumer.ConsumerGroupMetadata; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.connect.sink.SinkRecord; +import org.apache.kafka.connect.sink.SinkTaskContext; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; + +public class WorkerTest extends ChannelTestBase { + + @Test + public void testSave() { + when(config.catalogName()).thenReturn("catalog"); + + try (MockedStatic mockKafkaUtils = mockStatic(KafkaUtils.class)) { + ConsumerGroupMetadata consumerGroupMetadata = mock(ConsumerGroupMetadata.class); + mockKafkaUtils + .when(() -> KafkaUtils.consumerGroupMetadata(any())) + .thenReturn(consumerGroupMetadata); + + SinkTaskContext context = mock(SinkTaskContext.class); + TopicPartition topicPartition = new TopicPartition(SRC_TOPIC_NAME, 0); + when(context.assignment()).thenReturn(ImmutableSet.of(topicPartition)); + + IcebergWriterResult writeResult = + new IcebergWriterResult( + TableIdentifier.parse(TABLE_NAME), + ImmutableList.of(EventTestUtil.createDataFile()), + ImmutableList.of(), + StructType.of()); + + Map offsets = + ImmutableMap.of(topicPartition, new Offset(1L, EventTestUtil.now())); + + SinkWriterResult sinkWriterResult = + new SinkWriterResult(ImmutableList.of(writeResult), offsets); + SinkWriter sinkWriter = mock(SinkWriter.class); + when(sinkWriter.completeWrite()).thenReturn(sinkWriterResult); + + Worker worker = new Worker(config, clientFactory, sinkWriter, context); + worker.start(); + + // init consumer after subscribe() + initConsumer(); + + // save a record + Map value = ImmutableMap.of(); + SinkRecord rec = new SinkRecord(SRC_TOPIC_NAME, 0, null, "key", null, value, 0L); + worker.save(ImmutableList.of(rec)); + + UUID commitId = UUID.randomUUID(); + Event commitRequest = new Event(config.connectGroupId(), new StartCommit(commitId)); + byte[] bytes = AvroUtil.encode(commitRequest); + consumer.addRecord(new ConsumerRecord<>(CTL_TOPIC_NAME, 0, 1, "key", bytes)); + + worker.process(); + + assertThat(producer.history()).hasSize(2); + + Event event = AvroUtil.decode(producer.history().get(0).value()); + assertThat(event.payload().type()).isEqualTo(PayloadType.DATA_WRITTEN); + DataWritten dataWritten = (DataWritten) event.payload(); + assertThat(dataWritten.commitId()).isEqualTo(commitId); + + event = AvroUtil.decode(producer.history().get(1).value()); + assertThat(event.type()).isEqualTo(PayloadType.DATA_COMPLETE); + DataComplete dataComplete = (DataComplete) event.payload(); + assertThat(dataComplete.commitId()).isEqualTo(commitId); + assertThat(dataComplete.assignments()).hasSize(1); + assertThat(dataComplete.assignments().get(0).offset()).isEqualTo(1L); + } + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java index 80adc7fc3e03..ac44952a5c15 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/BaseWriterTest.java @@ -73,7 +73,7 @@ public void before() { protected WriteResult writeTest( List rows, IcebergSinkConfig config, Class expectedWriterClass) { - try (TaskWriter writer = Utilities.createTableWriter(table, "name", config)) { + try (TaskWriter writer = RecordUtils.createTableWriter(table, "name", config)) { assertThat(writer.getClass()).isEqualTo(expectedWriterClass); rows.forEach( diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java new file mode 100644 index 000000000000..08e832256a28 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/RecordUtilsTest.java @@ -0,0 +1,93 @@ +/* + * 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.connect.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.kafka.connect.data.Schema; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.junit.jupiter.api.Test; + +public class RecordUtilsTest { + + @Test + public void testExtractFromRecordValueStruct() { + Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); + Struct val = new Struct(valSchema).put("key", 123L); + Object result = RecordUtils.extractFromRecordValue(val, "key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueStructNested() { + Schema idSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); + Schema dataSchema = SchemaBuilder.struct().field("id", idSchema).build(); + Schema valSchema = SchemaBuilder.struct().field("data", dataSchema).build(); + + Struct id = new Struct(idSchema).put("key", 123L); + Struct data = new Struct(dataSchema).put("id", id); + Struct val = new Struct(valSchema).put("data", data); + + Object result = RecordUtils.extractFromRecordValue(val, "data.id.key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueStructNull() { + Schema valSchema = SchemaBuilder.struct().field("key", Schema.INT64_SCHEMA).build(); + Struct val = new Struct(valSchema).put("key", 123L); + + Object result = RecordUtils.extractFromRecordValue(val, ""); + assertThat(result).isNull(); + + result = RecordUtils.extractFromRecordValue(val, "xkey"); + assertThat(result).isNull(); + } + + @Test + public void testExtractFromRecordValueMap() { + Map val = ImmutableMap.of("key", 123L); + Object result = RecordUtils.extractFromRecordValue(val, "key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueMapNested() { + Map id = ImmutableMap.of("key", 123L); + Map data = ImmutableMap.of("id", id); + Map val = ImmutableMap.of("data", data); + + Object result = RecordUtils.extractFromRecordValue(val, "data.id.key"); + assertThat(result).isEqualTo(123L); + } + + @Test + public void testExtractFromRecordValueMapNull() { + Map val = ImmutableMap.of("key", 123L); + + Object result = RecordUtils.extractFromRecordValue(val, ""); + assertThat(result).isNull(); + + result = RecordUtils.extractFromRecordValue(val, "xkey"); + assertThat(result).isNull(); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java new file mode 100644 index 000000000000..be29ef1022a4 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SchemaUpdateTest.java @@ -0,0 +1,68 @@ +/* + * 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.connect.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class SchemaUpdateTest { + + @Test + public void testAddColumn() { + SchemaUpdate.Consumer updateConsumer = new SchemaUpdate.Consumer(); + updateConsumer.addColumn("parent", "name", Types.StringType.get()); + assertThat(updateConsumer.addColumns()).hasSize(1); + assertThat(updateConsumer.updateTypes()).isEmpty(); + assertThat(updateConsumer.makeOptionals()).isEmpty(); + + SchemaUpdate.AddColumn addColumn = updateConsumer.addColumns().iterator().next(); + assertThat(addColumn.parentName()).isEqualTo("parent"); + assertThat(addColumn.name()).isEqualTo("name"); + assertThat(addColumn.type()).isEqualTo(Types.StringType.get()); + } + + @Test + public void testUpdateType() { + SchemaUpdate.Consumer updateConsumer = new SchemaUpdate.Consumer(); + updateConsumer.updateType("name", Types.LongType.get()); + assertThat(updateConsumer.addColumns()).isEmpty(); + + assertThat(updateConsumer.updateTypes()).hasSize(1); + assertThat(updateConsumer.makeOptionals()).isEmpty(); + + SchemaUpdate.UpdateType updateType = updateConsumer.updateTypes().iterator().next(); + assertThat(updateType.name()).isEqualTo("name"); + assertThat(updateType.type()).isEqualTo(Types.LongType.get()); + } + + @Test + public void testMakeOptional() { + SchemaUpdate.Consumer updateConsumer = new SchemaUpdate.Consumer(); + updateConsumer.makeOptional("name"); + assertThat(updateConsumer.addColumns()).isEmpty(); + + assertThat(updateConsumer.updateTypes()).isEmpty(); + assertThat(updateConsumer.makeOptionals()).hasSize(1); + + SchemaUpdate.MakeOptional makeOptional = updateConsumer.makeOptionals().iterator().next(); + assertThat(makeOptional.name()).isEqualTo("name"); + } +} diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java new file mode 100644 index 000000000000..4a17b926fc56 --- /dev/null +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/data/SinkWriterTest.java @@ -0,0 +1,210 @@ +/* + * 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.connect.data; + +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyBoolean; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.time.Instant; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.Map; +import java.util.regex.Pattern; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.connect.IcebergSinkConfig; +import org.apache.iceberg.connect.TableSinkConfig; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.TimestampType; +import org.apache.kafka.connect.sink.SinkRecord; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +public class SinkWriterTest { + + private InMemoryCatalog catalog; + + private static final Namespace NAMESPACE = Namespace.of("db"); + private static final String TABLE_NAME = "tbl"; + private static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(NAMESPACE, TABLE_NAME); + private static final Schema SCHEMA = + new Schema( + optional(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + optional(3, "date", Types.StringType.get())); + private static final String ROUTE_FIELD = "fld"; + + @BeforeEach + public void before() { + catalog = initInMemoryCatalog(); + catalog.createNamespace(NAMESPACE); + catalog.createTable(TABLE_IDENTIFIER, SCHEMA); + } + + @AfterEach + public void after() throws IOException { + catalog.close(); + } + + private InMemoryCatalog initInMemoryCatalog() { + InMemoryCatalog inMemoryCatalog = new InMemoryCatalog(); + inMemoryCatalog.initialize(null, ImmutableMap.of()); + return inMemoryCatalog; + } + + @Test + public void testDefaultRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + Map value = ImmutableMap.of(); + + List writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(1); + IcebergWriterResult writerResult = writerResults.get(0); + assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + } + + @Test + public void testDefaultNoRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.tables()).thenReturn(ImmutableList.of()); + Map value = ImmutableMap.of(); + + List writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(0); + } + + @Test + public void testStaticRoute() { + TableSinkConfig tableConfig = mock(TableSinkConfig.class); + when(tableConfig.routeRegex()).thenReturn(Pattern.compile("val")); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(tableConfig); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map value = ImmutableMap.of(ROUTE_FIELD, "val"); + List writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(1); + IcebergWriterResult writerResult = writerResults.get(0); + assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + } + + @Test + public void testStaticNoRoute() { + TableSinkConfig tableConfig = mock(TableSinkConfig.class); + when(tableConfig.routeRegex()).thenReturn(Pattern.compile("val")); + + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(tableConfig); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map value = ImmutableMap.of(ROUTE_FIELD, "foobar"); + List writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(0); + } + + @Test + public void testDynamicRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map value = ImmutableMap.of(ROUTE_FIELD, TABLE_IDENTIFIER.toString()); + + List writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(1); + IcebergWriterResult writerResult = writerResults.get(0); + assertThat(writerResult.tableIdentifier()).isEqualTo(TABLE_IDENTIFIER); + } + + @Test + public void testDynamicNoRoute() { + IcebergSinkConfig config = mock(IcebergSinkConfig.class); + when(config.tables()).thenReturn(ImmutableList.of(TABLE_IDENTIFIER.toString())); + when(config.tableConfig(any())).thenReturn(mock(TableSinkConfig.class)); + when(config.dynamicTablesEnabled()).thenReturn(true); + when(config.tablesRouteField()).thenReturn(ROUTE_FIELD); + + Map value = ImmutableMap.of(ROUTE_FIELD, "db.foobar"); + + List writerResults = sinkWriterTest(value, config); + assertThat(writerResults.size()).isEqualTo(0); + } + + private List sinkWriterTest( + Map value, IcebergSinkConfig config) { + IcebergWriterResult writeResult = + new IcebergWriterResult( + TableIdentifier.parse(TABLE_NAME), + ImmutableList.of(mock(DataFile.class)), + ImmutableList.of(), + Types.StructType.of()); + IcebergWriter writer = mock(IcebergWriter.class); + when(writer.complete()).thenReturn(ImmutableList.of(writeResult)); + + IcebergWriterFactory writerFactory = mock(IcebergWriterFactory.class); + when(writerFactory.createWriter(any(), any(), anyBoolean())).thenReturn(writer); + + SinkWriter sinkWriter = new SinkWriter(catalog, config); + + // save a record + Instant now = Instant.now().truncatedTo(ChronoUnit.MILLIS); + SinkRecord rec = + new SinkRecord( + "topic", + 1, + null, + "key", + null, + value, + 100L, + now.toEpochMilli(), + TimestampType.LOG_APPEND_TIME); + sinkWriter.save(ImmutableList.of(rec)); + + SinkWriterResult result = sinkWriter.completeWrite(); + + Offset offset = result.sourceOffsets().get(new TopicPartition("topic", 1)); + assertThat(offset).isNotNull(); + assertThat(offset.offset()).isEqualTo(101L); // should be 1 more than current offset + assertThat(offset.timestamp()).isEqualTo(now.atOffset(ZoneOffset.UTC)); + + return result.writerResults(); + } +} From 1ccf56faf8dccbbe6d969f2a2c6d902eb70732d7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 11 Jul 2024 21:12:03 +0200 Subject: [PATCH 0459/1019] Upgrade to Gradle 8.9 (#10686) --- 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 381baa9cef1e..68e8816d71c9 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=544c35d6bd849ae8a5ed0bcea39ba677dc40f49df7d1835561582da2009b961d -distributionUrl=https\://services.gradle.org/distributions/gradle-8.7-bin.zip +distributionSha256Sum=d725d707bfabd4dfdc958c624003b3c80accc03f7037b5122c4b1d0ef15cecab +distributionUrl=https\://services.gradle.org/distributions/gradle-8.9-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 1757b45dfead..e353c27d23f8 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.7.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.9.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From c41fd3b3bf26d73d689cb0196350ee64faa15188 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 11 Jul 2024 20:30:35 -0700 Subject: [PATCH 0460/1019] Core: Use bulk deletes when removing old metadata files (#10679) --- .../iceberg/BaseMetastoreTableOperations.java | 25 +++++++++++++------ 1 file changed, 17 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 50266b2edcbf..90d435811020 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -31,8 +31,10 @@ 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; @@ -387,14 +389,21 @@ private void deleteRemovedMetadataFiles(TableMetadata base, TableMetadata metada // the log, thus we don't include metadata.previousFiles() for deletion - everything else can // be removed removedPreviousMetadataFiles.removeAll(metadata.previousFiles()); - Tasks.foreach(removedPreviousMetadataFiles) - .noRetry() - .suppressFailureWhenFinished() - .onFailure( - (previousMetadataFile, exc) -> - LOG.warn( - "Delete failed for previous metadata file: {}", previousMetadataFile, exc)) - .run(previousMetadataFile -> io().deleteFile(previousMetadataFile.file())); + 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())); + } } } } From b168d1db51912cbce4e0ed94fade77a6d3316226 Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Fri, 12 Jul 2024 10:06:00 +0200 Subject: [PATCH 0461/1019] Core: Expose incremental/changelog scan in SerializableTable (#10682) --- .../java/org/apache/iceberg/SerializableTable.java | 10 ++++++++++ 1 file 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 9e0055a10376..082e50b840dc 100644 --- a/core/src/main/java/org/apache/iceberg/SerializableTable.java +++ b/core/src/main/java/org/apache/iceberg/SerializableTable.java @@ -278,6 +278,16 @@ public TableScan newScan() { return lazyTable().newScan(); } + @Override + public IncrementalAppendScan newIncrementalAppendScan() { + return lazyTable().newIncrementalAppendScan(); + } + + @Override + public IncrementalChangelogScan newIncrementalChangelogScan() { + return lazyTable().newIncrementalChangelogScan(); + } + @Override public BatchScan newBatchScan() { return lazyTable().newBatchScan(); From 2ab48330bec33acf978c3a9df92e33a1e0f5e88a Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Fri, 12 Jul 2024 11:05:10 +0200 Subject: [PATCH 0462/1019] OpenAPI: Deprecate `oauth/tokens` endpoint (#10603) * Deprecate `oauth/tokens` endpoint This PR implements "M1" of [this document](https://docs.google.com/document/d/1Xi5MRk8WdBWFC3N_eSmVcrLhk3yu5nJ9x_wC0ec6kVQ/), see #10537. * update wording in spec * 2 * left-over --- .../iceberg/rest/RESTSessionCatalog.java | 17 +++++++- open-api/rest-catalog-open-api.py | 17 +++++++- open-api/rest-catalog-open-api.yaml | 39 ++++++++++++++++++- 3 files changed, 69 insertions(+), 4 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 56e07202f873..a72d3958c140 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -184,19 +184,34 @@ public void initialize(String name, Map unresolved) { long startTimeMillis = System.currentTimeMillis(); // keep track of the init start time for token refresh String initToken = props.get(OAuth2Properties.TOKEN); + boolean hasInitToken = initToken != null; // fetch auth and config to complete initialization ConfigResponse config; OAuthTokenResponse authResponse; String credential = props.get(OAuth2Properties.CREDENTIAL); + boolean hasCredential = credential != null && !credential.isEmpty(); String scope = props.getOrDefault(OAuth2Properties.SCOPE, OAuth2Properties.CATALOG_SCOPE); Map optionalOAuthParams = OAuth2Util.buildOptionalParam(props); + if (!props.containsKey(OAuth2Properties.OAUTH2_SERVER_URI) + && (hasInitToken || hasCredential) + && !PropertyUtil.propertyAsBoolean(props, "rest.sigv4-enabled", false)) { + LOG.warn( + "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), + ResourcePaths.tokens(), + OAuth2Properties.OAUTH2_SERVER_URI); + } String oauth2ServerUri = props.getOrDefault(OAuth2Properties.OAUTH2_SERVER_URI, ResourcePaths.tokens()); try (RESTClient initClient = clientBuilder.apply(props)) { Map initHeaders = RESTUtil.merge(configHeaders(props), OAuth2Util.authHeaders(initToken)); - if (credential != null && !credential.isEmpty()) { + if (hasCredential) { authResponse = OAuth2Util.fetchToken( initClient, initHeaders, credential, scope, oauth2ServerUri, optionalOAuthParams); diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index c5af1940c324..67498055502d 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -462,6 +462,8 @@ class TokenType(BaseModel): class OAuthClientCredentialsRequest(BaseModel): """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + OAuth2 client credentials request See https://datatracker.ietf.org/doc/html/rfc6749#section-4.4 @@ -481,6 +483,8 @@ class OAuthClientCredentialsRequest(BaseModel): class OAuthTokenExchangeRequest(BaseModel): """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + OAuth2 token exchange request See https://datatracker.ietf.org/doc/html/rfc8693 @@ -500,7 +504,10 @@ class OAuthTokenExchangeRequest(BaseModel): class OAuthTokenRequest(BaseModel): - __root__: Union[OAuthClientCredentialsRequest, OAuthTokenExchangeRequest] + __root__: Union[OAuthClientCredentialsRequest, OAuthTokenExchangeRequest] = Field( + ..., + description='The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint.', + ) class CounterResult(BaseModel): @@ -532,6 +539,10 @@ class CommitReport(BaseModel): class OAuthError(BaseModel): + """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + """ + error: Literal[ 'invalid_request', 'invalid_client', @@ -545,6 +556,10 @@ class OAuthError(BaseModel): class OAuthTokenResponse(BaseModel): + """ + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this spec, see description of the endpoint. + """ + access_token: str = Field( ..., description='The access token, for client credentials or token exchange' ) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 4bb73cd44120..feb2fdbc63cd 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -134,9 +134,22 @@ paths: post: tags: - OAuth2 API - summary: Get a token using an OAuth2 flow + summary: Get a token using an OAuth2 flow (DEPRECATED for REMOVAL) + deprecated: true operationId: getToken description: + The `oauth/tokens` endpoint is **DEPRECATED for REMOVAL**. It is _not_ recommended to + implement this endpoint, unless you are fully aware of the potential security implications. + + All clients are encouraged to explicitly set the configuration property `oauth2-server-uri` + to the correct OAuth endpoint. + + Deprecated since Iceberg (Java) 1.6.0. The endpoint and related types will be removed from + this spec in Iceberg (Java) 2.0. + + See [Security improvements in the Iceberg REST specification](https://github.com/apache/iceberg/issues/10537) + + Exchange credentials for a token using the OAuth2 client credentials flow or token exchange. @@ -2922,7 +2935,12 @@ components: See https://datatracker.ietf.org/doc/html/rfc8693#section-3 OAuthClientCredentialsRequest: - description: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. + + OAuth2 client credentials request @@ -2957,7 +2975,12 @@ components: a Basic Authorization header. OAuthTokenExchangeRequest: + deprecated: true description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. + + OAuth2 token exchange request @@ -2988,6 +3011,10 @@ components: $ref: '#/components/schemas/TokenType' OAuthTokenRequest: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. anyOf: - $ref: '#/components/schemas/OAuthClientCredentialsRequest' - $ref: '#/components/schemas/OAuthTokenExchangeRequest' @@ -3142,6 +3169,10 @@ components: type: string OAuthError: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. type: object required: - error @@ -3161,6 +3192,10 @@ components: type: string OAuthTokenResponse: + deprecated: true + description: + The `oauth/tokens` endpoint and related schemas are **DEPRECATED for REMOVAL** from this + spec, see description of the endpoint. type: object required: - access_token From b528bde627d824e0aace9747d54dba305da3cf75 Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Fri, 12 Jul 2024 11:05:59 +0200 Subject: [PATCH 0463/1019] Bump Nessie from 0.91.3 to 0.92.0 (#10689) --- 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 5f07fb949278..214cdff51216 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.91.3" +nessie = "0.92.0" netty-buffer = "4.1.111.Final" netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" From 7772f1325762438b923efa1ccf12e5eb4eb0e222 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Fri, 12 Jul 2024 19:15:24 +0900 Subject: [PATCH 0464/1019] Core: Exclude unexpected namespaces JdbcCatalog.listNamespaces (#10498) * Core: Exclude unexpected namespaces JdbcCatalog.listNamespaces * Swap order * Swap it back --------- Co-authored-by: Fokko Driesprong --- .../org/apache/iceberg/jdbc/JdbcCatalog.java | 10 ++++++++++ .../apache/iceberg/jdbc/TestJdbcCatalog.java | 17 ++++++++++++++--- 2 files changed, 24 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 4e10ee96d125..5e985faafbed 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -468,6 +468,16 @@ public List listNamespaces(Namespace namespace) throws NoSuchNamespac .toArray(String[]::new))) // remove duplicates .distinct() + // exclude fuzzy matches when `namespace` contains `%` or `_` + .filter( + n -> { + for (int i = 0; i < namespace.levels().length; i++) { + if (!n.levels()[i].equals(namespace.levels()[i])) { + return false; + } + } + return true; + }) .collect(Collectors.toList()); return namespaces; 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 148a483b3477..d21605cace21 100644 --- a/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java +++ b/core/src/test/java/org/apache/iceberg/jdbc/TestJdbcCatalog.java @@ -750,8 +750,11 @@ public void testListNamespace() { TableIdentifier tbl4 = TableIdentifier.of("db", "metadata"); TableIdentifier tbl5 = TableIdentifier.of("db2", "metadata"); TableIdentifier tbl6 = TableIdentifier.of("tbl6"); + TableIdentifier tbl7 = TableIdentifier.of("db2", "ns4", "tbl5"); + TableIdentifier tbl8 = TableIdentifier.of("d_", "ns5", "tbl6"); + TableIdentifier tbl9 = TableIdentifier.of("d%", "ns6", "tbl7"); - Lists.newArrayList(tbl1, tbl2, tbl3, tbl4, tbl5, tbl6) + Lists.newArrayList(tbl1, tbl2, tbl3, tbl4, tbl5, tbl6, tbl7, tbl8, tbl9) .forEach(t -> catalog.createTable(t, SCHEMA, PartitionSpec.unpartitioned())); List nsp1 = catalog.listNamespaces(Namespace.of("db")); @@ -765,11 +768,19 @@ public void testListNamespace() { List nsp3 = catalog.listNamespaces(); Set tblSet2 = Sets.newHashSet(nsp3.stream().map(Namespace::toString).iterator()); - assertThat(tblSet2).hasSize(3).contains("db", "db2", ""); + assertThat(tblSet2).hasSize(5).contains("db", "db2", "d_", "d%", ""); List nsp4 = catalog.listNamespaces(); Set tblSet3 = Sets.newHashSet(nsp4.stream().map(Namespace::toString).iterator()); - assertThat(tblSet3).hasSize(3).contains("db", "db2", ""); + assertThat(tblSet3).hasSize(5).contains("db", "db2", "d_", "d%", ""); + + List nsp5 = catalog.listNamespaces(Namespace.of("d_")); + assertThat(nsp5).hasSize(1); + assertThat(nsp5.get(0)).hasToString("d_.ns5"); + + List nsp6 = catalog.listNamespaces(Namespace.of("d%")); + assertThat(nsp6).hasSize(1); + assertThat(nsp6.get(0)).hasToString("d%.ns6"); assertThatThrownBy(() -> catalog.listNamespaces(Namespace.of("db", "db2", "ns2"))) .isInstanceOf(NoSuchNamespaceException.class) From 50458d526288dd37aaf57cd77486459d8a0ee448 Mon Sep 17 00:00:00 2001 From: boroknagyz Date: Fri, 12 Jul 2024 14:25:39 +0200 Subject: [PATCH 0465/1019] Core: Fix NPE during conflict handling of NULL partitions (#10680) * Core: Fix NPE during conflict handling of NULL partitions Partition values can be NULLs, or we can have NULLs because of the VOID transforms. If a conflict is found in such partitions we get a NullPointerException instead of a proper error message. * Fix style issues * Use String.valueOf() * Reduce visibility of constant Co-authored-by: Eduard Tudenhoefner * Indentation * Update core/src/main/java/org/apache/iceberg/util/PartitionSet.java --------- Co-authored-by: Fokko Driesprong Co-authored-by: Eduard Tudenhoefner --- .../org/apache/iceberg/util/PartitionSet.java | 2 +- .../apache/iceberg/TestReplacePartitions.java | 77 +++++++++++++++++++ 2 files changed, 78 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java index eff37fa5a9e8..184f38b32461 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionSet.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionSet.java @@ -200,7 +200,7 @@ public String toString() { StringBuilder partitionStringBuilder = new StringBuilder(); partitionStringBuilder.append(structType.fields().get(i).name()); partitionStringBuilder.append("="); - partitionStringBuilder.append(s.get(i, Object.class).toString()); + partitionStringBuilder.append(s.get(i, Object.class)); partitionDataJoiner.add(partitionStringBuilder.toString()); } } diff --git a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java index a25920a1d733..6fa77ae05c9a 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestReplacePartitions.java @@ -67,6 +67,34 @@ public class TestReplacePartitions extends TestBase { .withRecordCount(1) .build(); + static final DataFile FILE_NULL_PARTITION = + DataFiles.builder(SPEC) + .withPath("/path/to/data-null-partition.parquet") + .withFileSizeInBytes(0) + .withPartitionPath("data_bucket=__HIVE_DEFAULT_PARTITION__") + .withRecordCount(0) + .build(); + + // Partition spec with VOID partition transform ("alwaysNull" in Java code.) + static final PartitionSpec SPEC_VOID = + PartitionSpec.builderFor(SCHEMA).alwaysNull("id").bucket("data", BUCKETS_NUMBER).build(); + + static final DataFile FILE_A_VOID_PARTITION = + DataFiles.builder(SPEC_VOID) + .withPath("/path/to/data-a-void-partition.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("id_null=__HIVE_DEFAULT_PARTITION__/data_bucket=0") + .withRecordCount(1) + .build(); + + static final DataFile FILE_B_VOID_PARTITION = + DataFiles.builder(SPEC_VOID) + .withPath("/path/to/data-b-void-partition.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("id_null=__HIVE_DEFAULT_PARTITION__/data_bucket=1") + .withRecordCount(10) + .build(); + static final DeleteFile FILE_UNPARTITIONED_A_DELETES = FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) .ofPositionDeletes() @@ -317,6 +345,55 @@ public void testValidateWithDefaultSnapshotId() { + "[data_bucket=0, data_bucket=1]: [/path/to/data-a.parquet]"); } + @TestTemplate + public void testValidateWithNullPartition() { + commit(table, table.newReplacePartitions().addFile(FILE_NULL_PARTITION), branch); + + // Concurrent Replace Partitions should fail with ValidationException + ReplacePartitions replace = table.newReplacePartitions(); + assertThatThrownBy( + () -> + commit( + table, + replace + .addFile(FILE_NULL_PARTITION) + .addFile(FILE_B) + .validateNoConflictingData() + .validateNoConflictingDeletes(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[data_bucket=null, data_bucket=1]: [/path/to/data-null-partition.parquet]"); + } + + @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); + + // Concurrent Replace Partitions should fail with ValidationException + ReplacePartitions replace = tableVoid.newReplacePartitions(); + assertThatThrownBy( + () -> + commit( + tableVoid, + replace + .addFile(FILE_A_VOID_PARTITION) + .addFile(FILE_B_VOID_PARTITION) + .validateNoConflictingData() + .validateNoConflictingDeletes(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage( + "Found conflicting files that can contain records matching partitions " + + "[id_null=null, data_bucket=1, id_null=null, data_bucket=0]: " + + "[/path/to/data-a-void-partition.parquet]"); + } + @TestTemplate public void testConcurrentReplaceConflict() { commit(table, table.newFastAppend().appendFile(FILE_A).appendFile(FILE_B), branch); From a80b2f30f068fa2fcab5c4ab9b642a7f2a319cbf Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Fri, 12 Jul 2024 16:32:12 -0700 Subject: [PATCH 0466/1019] Spark 3.3, 3.4: Support read of partition metadata column when table is over 1k (#10641) --- .../spark/source/SparkScanBuilder.java | 49 +++++++++++++++++-- .../source/TestSparkMetadataColumns.java | 49 +++++++++++++++++++ .../spark/source/SparkScanBuilder.java | 49 +++++++++++++++++-- .../source/TestSparkMetadataColumns.java | 47 ++++++++++++++++++ 4 files changed, 188 insertions(+), 6 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 ef3138d677c6..afb0f434aa41 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 @@ -20,6 +20,9 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.BaseTable; @@ -47,6 +50,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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkAggregates; import org.apache.iceberg.spark.SparkFilters; @@ -362,15 +366,54 @@ public void pruneColumns(StructType requestedSchema) { private Schema schemaWithMetadataColumns() { // metadata columns - List fields = + List metadataFields = metaColumns.stream() .distinct() .map(name -> MetadataColumns.metadataColumn(table, name)) .collect(Collectors.toList()); - Schema meta = new Schema(fields); + Schema metadataSchema = calculateMetadataSchema(metadataFields); // schema or rows returned by readers - return TypeUtil.join(schema, meta); + return TypeUtil.join(schema, metadataSchema); + } + + private Schema calculateMetadataSchema(List metaColumnFields) { + Optional partitionField = + metaColumnFields.stream() + .filter(f -> MetadataColumns.PARTITION_COLUMN_ID == f.fieldId()) + .findFirst(); + + // only calculate potential column id collision if partition metadata column was requested + if (!partitionField.isPresent()) { + return new Schema(metaColumnFields); + } + + Set idsToReassign = + TypeUtil.indexById(partitionField.get().type().asStructType()).keySet(); + + // Calculate used ids by union metadata columns with all base table schemas + Set currentlyUsedIds = + metaColumnFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + Set allUsedIds = + table.schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Reassign selected ids to deduplicate with used ids. + AtomicInteger nextId = new AtomicInteger(); + return new Schema( + metaColumnFields, + table.schema().identifierFieldIds(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); } @Override diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index e24e74383bc8..127b0eb66fe9 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_BATCH_SIZE; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.spark.sql.functions.expr; import static org.apache.spark.sql.functions.lit; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -31,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; @@ -53,6 +55,7 @@ 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.types.StructType; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -169,6 +172,52 @@ public void testSpecAndPartitionMetadataColumns() { sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); } + @Test + public void testPartitionMetadataColumnWithManyColumns() { + // TODO: support metadata structs in vectorized ORC reads + Assume.assumeFalse(fileFormat == FileFormat.ORC && vectorized); + List fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema manyColumnsSchema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(manyColumnsSchema).identity("id").build(); + + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit( + base, + base.updateSchema(manyColumnsSchema, manyColumnsSchema.highestFieldId()) + .updatePartitionSpec(spec)); + + Dataset df = + spark + .range(2) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(TABLE_NAME); + + Assert.assertEquals(2, spark.table(TABLE_NAME).select("*", "_partition").count()); + List expected = + ImmutableList.of(row(row(0L), 0L, "0", "0", "0"), row(row(1L), 1L, "1", "1", "1")); + assertEquals( + "Rows must match", + expected, + sql("SELECT _partition, id, c999, c1000, c1001 FROM %s ORDER BY id", TABLE_NAME)); + } + @Test public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { Assume.assumeTrue(fileFormat == FileFormat.PARQUET); 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 6b97e48133fd..9dc214a755d3 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 @@ -20,6 +20,9 @@ import java.io.IOException; import java.util.List; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; import org.apache.iceberg.BaseTable; @@ -48,6 +51,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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.SparkAggregates; import org.apache.iceberg.spark.SparkReadConf; @@ -342,15 +346,54 @@ public void pruneColumns(StructType requestedSchema) { private Schema schemaWithMetadataColumns() { // metadata columns - List fields = + List metadataFields = metaColumns.stream() .distinct() .map(name -> MetadataColumns.metadataColumn(table, name)) .collect(Collectors.toList()); - Schema meta = new Schema(fields); + Schema metadataSchema = calculateMetadataSchema(metadataFields); // schema or rows returned by readers - return TypeUtil.join(schema, meta); + return TypeUtil.join(schema, metadataSchema); + } + + private Schema calculateMetadataSchema(List metaColumnFields) { + Optional partitionField = + metaColumnFields.stream() + .filter(f -> MetadataColumns.PARTITION_COLUMN_ID == f.fieldId()) + .findFirst(); + + // only calculate potential column id collision if partition metadata column was requested + if (!partitionField.isPresent()) { + return new Schema(metaColumnFields); + } + + Set idsToReassign = + TypeUtil.indexById(partitionField.get().type().asStructType()).keySet(); + + // Calculate used ids by union metadata columns with all base table schemas + Set currentlyUsedIds = + metaColumnFields.stream().map(Types.NestedField::fieldId).collect(Collectors.toSet()); + Set allUsedIds = + table.schemas().values().stream() + .map(currSchema -> TypeUtil.indexById(currSchema.asStruct()).keySet()) + .reduce(currentlyUsedIds, Sets::union); + + // Reassign selected ids to deduplicate with used ids. + AtomicInteger nextId = new AtomicInteger(); + return new Schema( + metaColumnFields, + table.schema().identifierFieldIds(), + oldId -> { + if (!idsToReassign.contains(oldId)) { + return oldId; + } + int candidate = nextId.incrementAndGet(); + while (allUsedIds.contains(candidate)) { + candidate = nextId.incrementAndGet(); + } + return candidate; + }); } @Override diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java index 778c46bba6b6..0ba34a638a63 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/source/TestSparkMetadataColumns.java @@ -24,6 +24,7 @@ import static org.apache.iceberg.TableProperties.PARQUET_BATCH_SIZE; import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES; import static org.apache.iceberg.TableProperties.PARQUET_VECTORIZATION_ENABLED; +import static org.apache.spark.sql.functions.expr; import static org.apache.spark.sql.functions.lit; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -31,6 +32,7 @@ import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.iceberg.FileFormat; import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.MetadataColumns; @@ -53,6 +55,7 @@ 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.types.StructType; import org.junit.After; import org.junit.Assert; import org.junit.Assume; @@ -169,6 +172,50 @@ public void testSpecAndPartitionMetadataColumns() { sql("SELECT _spec_id, _partition FROM %s ORDER BY _spec_id", TABLE_NAME)); } + @Test + public void testPartitionMetadataColumnWithManyColumns() { + List fields = + Lists.newArrayList(Types.NestedField.required(0, "id", Types.LongType.get())); + List additionalCols = + IntStream.range(1, 1010) + .mapToObj(i -> Types.NestedField.optional(i, "c" + i, Types.StringType.get())) + .collect(Collectors.toList()); + fields.addAll(additionalCols); + Schema manyColumnsSchema = new Schema(fields); + PartitionSpec spec = PartitionSpec.builderFor(manyColumnsSchema).identity("id").build(); + + TableOperations ops = ((HasTableOperations) table).operations(); + TableMetadata base = ops.current(); + ops.commit( + base, + base.updateSchema(manyColumnsSchema, manyColumnsSchema.highestFieldId()) + .updatePartitionSpec(spec)); + + Dataset df = + spark + .range(2) + .withColumns( + IntStream.range(1, 1010) + .boxed() + .collect(Collectors.toMap(i -> "c" + i, i -> expr("CAST(id as STRING)")))); + StructType sparkSchema = spark.table(TABLE_NAME).schema(); + spark + .createDataFrame(df.rdd(), sparkSchema) + .coalesce(1) + .write() + .format("iceberg") + .mode("append") + .save(TABLE_NAME); + + Assert.assertEquals(2, spark.table(TABLE_NAME).select("*", "_partition").count()); + List expected = + ImmutableList.of(row(row(0L), 0L, "0", "0", "0"), row(row(1L), 1L, "1", "1", "1")); + assertEquals( + "Rows must match", + expected, + sql("SELECT _partition, id, c999, c1000, c1001 FROM %s ORDER BY id", TABLE_NAME)); + } + @Test public void testPositionMetadataColumnWithMultipleRowGroups() throws NoSuchTableException { Assume.assumeTrue(fileFormat == FileFormat.PARQUET); From dc6622683bcde26f92df7d4225d2c51d5dc8f896 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 14 Jul 2024 09:53:12 +0200 Subject: [PATCH 0467/1019] Build: Bump org.assertj:assertj-core from 3.26.0 to 3.26.3 (#10698) Bumps [org.assertj:assertj-core](https://github.com/assertj/assertj) from 3.26.0 to 3.26.3. - [Release notes](https://github.com/assertj/assertj/releases) - [Commits](https://github.com/assertj/assertj/compare/assertj-build-3.26.0...assertj-build-3.26.3) --- updated-dependencies: - dependency-name: org.assertj:assertj-core 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 214cdff51216..4e42cb8917a4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -26,7 +26,7 @@ antlr = "4.9.3" aircompressor = "0.27" arrow = "15.0.2" avro = "1.11.3" -assertj-core = "3.26.0" +assertj-core = "3.26.3" awaitility = "4.2.1" awssdk-bom = "2.26.16" azuresdk-bom = "1.2.25" From a33e20d370be69c560864572e5286ff6ad98dd37 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 15 Jul 2024 08:42:20 +0200 Subject: [PATCH 0468/1019] Build: Bump com.google.cloud:libraries-bom from 26.28.0 to 26.43.0 (#10699) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.28.0 to 26.43.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.28.0...v26.43.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 4e42cb8917a4..d255380f1ac1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -42,7 +42,7 @@ findbugs-jsr305 = "3.0.2" flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} -google-libraries-bom = "26.28.0" +google-libraries-bom = "26.43.0" guava = "33.2.1-jre" hadoop2 = "2.7.3" hadoop3-client = "3.3.6" From 4e03de0b09fcf9f9c7f19def1f99ce192bc64f40 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 15 Jul 2024 08:42:45 +0200 Subject: [PATCH 0469/1019] Build: Bump nessie from 0.92.0 to 0.92.1 (#10697) Bumps `nessie` from 0.92.0 to 0.92.1. Updates `org.projectnessie.nessie:nessie-client` from 0.92.0 to 0.92.1 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.92.0 to 0.92.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.92.0 to 0.92.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.92.0 to 0.92.1 --- 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 d255380f1ac1..37e28cb14331 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.92.0" +nessie = "0.92.1" netty-buffer = "4.1.111.Final" netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" From 782c10e060ffcdaf4998dd17115023f53f0c2ee0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 15 Jul 2024 08:43:35 +0200 Subject: [PATCH 0470/1019] Build: Bump software.amazon.awssdk:bom from 2.26.16 to 2.26.20 (#10700) Bumps software.amazon.awssdk:bom from 2.26.16 to 2.26.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 37e28cb14331..7fc4ee888f0f 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.3" awaitility = "4.2.1" -awssdk-bom = "2.26.16" +awssdk-bom = "2.26.20" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 49f19531a277009fc5d4325ace2c3d95a106f778 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 15 Jul 2024 11:13:47 +0200 Subject: [PATCH 0471/1019] OpenAPI: Fix property names for stats/partition stats (#10662) --- open-api/rest-catalog-open-api.py | 10 ++++------ open-api/rest-catalog-open-api.yaml | 6 ++++-- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 67498055502d..56212f1ac0b5 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -637,7 +637,7 @@ class BlobMetadata(BaseModel): snapshot_id: int = Field(..., alias='snapshot-id') sequence_number: int = Field(..., alias='sequence-number') fields: List[int] - properties: Optional[Dict[str, Any]] = None + properties: Optional[Dict[str, str]] = None class PartitionStatisticsFile(BaseModel): @@ -1024,11 +1024,9 @@ class TableMetadata(BaseModel): last_sequence_number: Optional[int] = Field(None, alias='last-sequence-number') snapshot_log: Optional[SnapshotLog] = Field(None, alias='snapshot-log') metadata_log: Optional[MetadataLog] = Field(None, alias='metadata-log') - statistics_files: Optional[List[StatisticsFile]] = Field( - None, alias='statistics-files' - ) - partition_statistics_files: Optional[List[PartitionStatisticsFile]] = Field( - None, alias='partition-statistics-files' + statistics: Optional[List[StatisticsFile]] = None + partition_statistics: Optional[List[PartitionStatisticsFile]] = Field( + None, alias='partition-statistics' ) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index feb2fdbc63cd..661af11efc4b 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2132,11 +2132,11 @@ components: metadata-log: $ref: '#/components/schemas/MetadataLog' # statistics - statistics-files: + statistics: type: array items: $ref: '#/components/schemas/StatisticsFile' - partition-statistics-files: + partition-statistics: type: array items: $ref: '#/components/schemas/PartitionStatisticsFile' @@ -3389,6 +3389,8 @@ components: type: integer properties: type: object + additionalProperties: + type: string PartitionStatisticsFile: type: object From 36e538db91b3840d2edf6ffa20bb327eef80d8aa Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Mon, 15 Jul 2024 21:38:03 +0200 Subject: [PATCH 0472/1019] Nit: fix/suppress false-positivie errorprone warning (#10690) --- .../objectinspector/IcebergTimestampObjectInspectorHive3.java | 1 + 1 file changed, 1 insertion(+) diff --git a/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java b/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java index 9ba99c8dec6c..3db2940f3cac 100644 --- a/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java +++ b/hive3/src/main/java/org/apache/iceberg/mr/hive/serde/objectinspector/IcebergTimestampObjectInspectorHive3.java @@ -51,6 +51,7 @@ public LocalDateTime convert(Object o) { } @Override + @SuppressWarnings("JavaLocalDateTimeGetNano") public Timestamp getPrimitiveJavaObject(Object o) { if (o == null) { return null; From 55ffa7d757395e42224f8b0aebfecb0ede3dd17b Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 15 Jul 2024 21:02:35 -0700 Subject: [PATCH 0473/1019] Spec: remove the JSON spec for content file and file scan task sections. (#9771) They shouldn't be part of the core table spec although the JSON serializer is valuable for FileScanTask serialization. See discussion thread for more context: https://lists.apache.org/thread/2ty27yx4q0zlqd5h71cyyhb5k47yf9bv --- format/spec.md | 36 ------------------------------------ 1 file changed, 36 deletions(-) diff --git a/format/spec.md b/format/spec.md index 9a3c16e3ac91..dd4e901f3706 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1230,42 +1230,6 @@ Example ] } ] ``` -### Content File (Data and Delete) Serialization - -Content file (data or delete) is serialized as a JSON object according to the following table. - -| Metadata field |JSON representation|Example| -|--------------------------|--- |--- | -| **`spec-id`** |`JSON int`|`1`| -| **`content`** |`JSON string`|`DATA`, `POSITION_DELETES`, `EQUALITY_DELETES`| -| **`file-path`** |`JSON string`|`"s3://b/wh/data.db/table"`| -| **`file-format`** |`JSON string`|`AVRO`, `ORC`, `PARQUET`| -| **`partition`** |`JSON object: Partition data tuple using partition field ids for the struct field ids`|`{"1000":1}`| -| **`record-count`** |`JSON long`|`1`| -| **`file-size-in-bytes`** |`JSON long`|`1024`| -| **`column-sizes`** |`JSON object: Map from column id to the total size on disk of all regions that store the column.`|`{"keys":[3,4],"values":[100,200]}`| -| **`value-counts`** |`JSON object: Map from column id to number of values in the column (including null and NaN values)`|`{"keys":[3,4],"values":[90,180]}`| -| **`null-value-counts`** |`JSON object: Map from column id to number of null values in the column`|`{"keys":[3,4],"values":[10,20]}`| -| **`nan-value-counts`** |`JSON object: Map from column id to number of NaN values in the column`|`{"keys":[3,4],"values":[0,0]}`| -| **`lower-bounds`** |`JSON object: Map from column id to lower bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["01000000","02000000"]}`| -| **`upper-bounds`** |`JSON object: Map from column id to upper bound binary in the column serialized as hexadecimal string`|`{"keys":[3,4],"values":["05000000","0A000000"]}`| -| **`key-metadata`** |`JSON string: Encryption key metadata binary serialized as hexadecimal string`|`00000000000000000000000000000000`| -| **`split-offsets`** |`JSON list of long: Split offsets for the data file`|`[128,256]`| -| **`equality-ids`** |`JSON list of int: Field ids used to determine row equality in equality delete files`|`[1]`| -| **`sort-order-id`** |`JSON int`|`1`| - -### File Scan Task Serialization - -File scan task is serialized as a JSON object according to the following table. - -| Metadata field |JSON representation|Example| -|--------------------------|--- |--- | -| **`schema`** |`JSON object`|`See above, read schemas instead`| -| **`spec`** |`JSON object`|`See above, read partition specs instead`| -| **`data-file`** |`JSON object`|`See above, read content file instead`| -| **`delete-files`** |`JSON list of objects`|`See above, read content file instead`| -| **`residual-filter`** |`JSON object: residual filter expression`|`{"type":"eq","term":"id","value":1}`| - ## Appendix D: Single-value serialization ### Binary single-value serialization From 8a830f40d3c04e04f6304df0e33c3c040bb5008f Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 16 Jul 2024 14:51:42 +0530 Subject: [PATCH 0474/1019] Infra: Fix stale PR workflow (#10706) --- .github/workflows/stale.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 0f599e5582cc..33887304441b 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -25,6 +25,7 @@ on: permissions: # All other permissions are set to none issues: write + pull-requests: write jobs: stale: From 33d709f08c0289d2124622c45d273b099e3a3ada Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 16 Jul 2024 12:33:45 +0200 Subject: [PATCH 0475/1019] Update references to `main` branch (#10705) Some docs were referring to `master`, update them to refer to `main`. --- site/docs/benchmarks.md | 2 +- site/docs/concepts/catalog.md | 2 +- site/docs/how-to-release.md | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/site/docs/benchmarks.md b/site/docs/benchmarks.md index 636ad36b1646..a74b65d48f4e 100644 --- a/site/docs/benchmarks.md +++ b/site/docs/benchmarks.md @@ -27,7 +27,7 @@ Also note that JMH benchmarks run within the same JVM as the system-under-test, It is possible to run one or more Benchmarks via the **JMH Benchmarks** GH action on your own fork of the Iceberg repo. This GH action takes the following inputs: * The repository name where those benchmarks should be run against, such as `apache/iceberg` or `/iceberg` -* The branch name to run benchmarks against, such as `master` or `my-cool-feature-branch` +* The branch name to run benchmarks against, such as `main` or `my-cool-feature-branch` * A list of comma-separated double-quoted Benchmark names, such as `"IcebergSourceFlatParquetDataReadBenchmark", "IcebergSourceFlatParquetDataFilterBenchmark", "IcebergSourceNestedListParquetDataWriteBenchmark"` Benchmark results will be uploaded once **all** benchmarks are done. diff --git a/site/docs/concepts/catalog.md b/site/docs/concepts/catalog.md index fdd664f0cf05..e83947e07f78 100644 --- a/site/docs/concepts/catalog.md +++ b/site/docs/concepts/catalog.md @@ -43,7 +43,7 @@ There are more catalog types in addition to the ones listed here as well as cust ## Decoupling Using the REST Catalog -The REST catalog was introduced in the Iceberg 0.14.0 release and provides greater control over how Iceberg catalogs are implemented. Instead of using technology-specific logic contained in the catalog clients, the implementation details of a REST catalog lives on the catalog server. If you’re familiar with Hive, this is somewhat similar to the Hive thrift service that allows access to a hive server over a single port. The server-side logic can be written in any language and use any custom technology, as long as the API follows the [Iceberg REST Open API specification](https://github.com/apache/iceberg/blob/master/open-api/rest-catalog-open-api.yaml). +The REST catalog was introduced in the Iceberg 0.14.0 release and provides greater control over how Iceberg catalogs are implemented. Instead of using technology-specific logic contained in the catalog clients, the implementation details of a REST catalog lives on the catalog server. If you’re familiar with Hive, this is somewhat similar to the Hive thrift service that allows access to a hive server over a single port. The server-side logic can be written in any language and use any custom technology, as long as the API follows the [Iceberg REST Open API specification](https://github.com/apache/iceberg/blob/main/open-api/rest-catalog-open-api.yaml). A great benefit of the REST catalog is that it allows you to use a single client to talk to any catalog backend. This increased flexibility makes it easier to make custom catalogs compatible with engines like Athena or Starburst without requiring the inclusion of a Jar into the classpath. diff --git a/site/docs/how-to-release.md b/site/docs/how-to-release.md index 0d2bc48464a6..6befe74b78de 100644 --- a/site/docs/how-to-release.md +++ b/site/docs/how-to-release.md @@ -84,7 +84,7 @@ This step can be useful to gather ongoing patches that the community thinks shou The communication can be started via a [DISCUSS] mail on the dev@ channel and the desired tickets can be added to the github milestone of the next release. -Note, creating a milestone in github requires a committer. However, a non-committer can assign tasks to a milestone if added to the list of collaborators in [.asf.yaml](https://github.com/apache/iceberg/blob/master/.asf.yaml) +Note, creating a milestone in github requires a committer. However, a non-committer can assign tasks to a milestone if added to the list of collaborators in [.asf.yaml](https://github.com/apache/iceberg/blob/main/.asf.yaml) The release status is discussed during each community sync meeting. Release manager should join the meeting to report status and discuss any release blocker. @@ -101,7 +101,7 @@ Example console output: ```text Preparing source for apache-iceberg-0.13.0-rc1 Adding version.txt and tagging release... -[master ca8bb7d0] Add version.txt for release 0.13.0 +[main ca8bb7d0] Add version.txt for release 0.13.0 1 file changed, 1 insertion(+) create mode 100644 version.txt Pushing apache-iceberg-0.13.0-rc1 to origin... From 5bfec438cd72c83b0494ca5765cf384ac4563dfd Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Tue, 16 Jul 2024 13:58:13 -0700 Subject: [PATCH 0476/1019] Docs: Clarify defaults for distribution mode (#10575) --- docs/docs/configuration.md | 2 +- docs/docs/spark-configuration.md | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/docs/configuration.md b/docs/docs/configuration.md index 117adca09f68..264b9edfa7cc 100644 --- a/docs/docs/configuration.md +++ b/docs/docs/configuration.md @@ -67,7 +67,7 @@ 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 | 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.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 | diff --git a/docs/docs/spark-configuration.md b/docs/docs/spark-configuration.md index 9ff739649839..5b281b19891a 100644 --- a/docs/docs/spark-configuration.md +++ b/docs/docs/spark-configuration.md @@ -190,6 +190,7 @@ df.write | compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | | compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | | compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | +| distribution-mode | See [Spark Writes](spark-writes.md#writing-distribution-modes) for defaults | Override this table's distribution mode for this write | CommitMetadata provides an interface to add custom metadata to a snapshot summary during a SQL execution, which can be beneficial for purposes such as auditing or change tracking. If properties start with `snapshot-property.`, then that prefix will be removed from each property. Here is an example: From 4a7ac9ef823c130dc5ae3c02db362947f3b7f9de Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 17 Jul 2024 08:29:38 +0200 Subject: [PATCH 0477/1019] Infra: Improve Bug report template (#10708) --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index a6d4fc7e5c2e..3bad6325552d 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml @@ -76,3 +76,11 @@ body: You can include files by dragging and dropping them here. validations: required: true + - type: checkboxes + attributes: + label: Willingness to contribute + description: The Apache Iceberg community encourages bug fix contributions. Would you or another member of your organization be willing to contribute a fix for this bug to the Apache Iceberg codebase? + options: + - label: I can contribute a fix for this bug independently + - label: I would be willing to contribute a fix for this bug with guidance from the Iceberg community + - label: I cannot contribute a fix for this bug at this time From c7e972b31002e518bdb691306753c4575dbf7f8e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 17 Jul 2024 09:08:01 +0200 Subject: [PATCH 0478/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.16.1 to 3.17.0 (#10696) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.16.1 to 3.17.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.16.1...v3.17.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 7fc4ee888f0f..267d4c2f0da5 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -77,7 +77,7 @@ roaringbitmap = "1.2.0" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" -snowflake-jdbc = "3.16.1" +snowflake-jdbc = "3.17.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" From 05716df272ddd19231830b1c9aa8677f20e8d03a Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Wed, 17 Jul 2024 15:13:13 +0800 Subject: [PATCH 0479/1019] Spark 3.3: Ignore flaky test taking up all device space (#10704) --- .../apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java index 3f56d82d9b70..7741ae39d620 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCopyOnWriteMerge.java @@ -48,6 +48,7 @@ import org.apache.spark.sql.internal.SQLConf; import org.junit.Assert; import org.junit.Assume; +import org.junit.Ignore; import org.junit.Test; public class TestCopyOnWriteMerge extends TestMerge { @@ -70,6 +71,7 @@ protected Map extraTableProperties() { } @Test + @Ignore // Ignored due to https://github.com/apache/iceberg/issues/10040 public synchronized void testMergeWithConcurrentTableRefresh() throws Exception { // this test can only be run with Hive tables as it requires a reliable lock // also, the table cache must be enabled so that the same table instance can be reused From df334c22b93969b69ac65d567106c7520b64ac4e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 17 Jul 2024 15:00:45 +0530 Subject: [PATCH 0480/1019] Infra: Increase operation per limit for stale bot workflow (#10712) --- .github/workflows/stale.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/stale.yml b/.github/workflows/stale.yml index 33887304441b..03276d60b524 100644 --- a/.github/workflows/stale.yml +++ b/.github/workflows/stale.yml @@ -55,4 +55,4 @@ jobs: days-before-pr-stale: 30 days-before-pr-close: 7 ascending: true - operations-per-run: 100 + operations-per-run: 200 From 2c7e9520be01176f54a1ddd8e968cc72503baaab Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 17 Jul 2024 13:49:02 +0200 Subject: [PATCH 0481/1019] DynFields, DynMethods code cleanup (#10543) * Remove unused throws declaration Avoid a warning in an IDE. * Mark final fields as such * Prepare for removal unused DynFields, DynMethods methods --- .../main/java/org/apache/iceberg/common/DynFields.java | 2 ++ .../main/java/org/apache/iceberg/common/DynMethods.java | 8 +++++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/common/src/main/java/org/apache/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java index a96da851315b..1b982d206fdb 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynFields.java +++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java @@ -394,7 +394,9 @@ public BoundField build(Object target) { * @return a {@link StaticField} with a valid implementation * @throws IllegalStateException if the method is not static * @throws NoSuchFieldException if no implementation was found + * @deprecated since 1.7.0, will be removed in 2.0.0 */ + @Deprecated public StaticField buildStaticChecked() throws NoSuchFieldException { return this.buildChecked().asStatic(); } diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index 98de1e7f06ca..eaaad876f133 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -51,6 +51,8 @@ public static class UnboundMethod { (method == null || method.isVarArgs()) ? -1 : method.getParameterTypes().length; } + /** @deprecated since 1.7.0, will be removed in 2.0.0 */ + @Deprecated // will become private @SuppressWarnings("unchecked") public R invokeChecked(Object target, Object... args) throws Exception { try { @@ -126,7 +128,7 @@ public String toString() { private static final UnboundMethod NOOP = new UnboundMethod(null, "NOOP") { @Override - public R invokeChecked(Object target, Object... args) throws Exception { + public R invokeChecked(Object target, Object... args) { return null; } @@ -313,6 +315,8 @@ public Builder impl(Class targetClass, Class... argClasses) { return this; } + /** @deprecated since 1.7.0, will be removed in 2.0.0 */ + @Deprecated public Builder ctorImpl(Class targetClass, Class... argClasses) { // don't do any work if an implementation has been found if (method != null) { @@ -327,6 +331,8 @@ public Builder ctorImpl(Class targetClass, Class... argClasses) { return this; } + /** @deprecated since 1.7.0, will be removed in 2.0.0 */ + @Deprecated public Builder ctorImpl(String className, Class... argClasses) { // don't do any work if an implementation has been found if (method != null) { From dca55e083c050e8e2c6b12c299bf953348b8b351 Mon Sep 17 00:00:00 2001 From: Anurag Mantripragada Date: Wed, 17 Jul 2024 23:37:10 +0530 Subject: [PATCH 0482/1019] Docs: Add examples for DataFrame branch writes (#10644) --- docs/docs/spark-writes.md | 31 +++++++++++++++++++++++++------ 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/docs/docs/spark-writes.md b/docs/docs/spark-writes.md index 96fcc5f7ce32..cc8ca76fe5f7 100644 --- a/docs/docs/spark-writes.md +++ b/docs/docs/spark-writes.md @@ -195,16 +195,19 @@ WHERE EXISTS (SELECT oid FROM prod.db.returned_orders WHERE t1.oid = oid) For more complex row-level updates based on incoming data, see the section on `MERGE INTO`. ## Writing to Branches -Branch writes can be performed via SQL by providing a branch identifier, `branch_yourBranch` in the operation. -Branch writes can also be performed as part of a write-audit-publish (WAP) workflow by specifying the `spark.wap.branch` config. -Note WAP branch and branch identifier cannot both be specified. -Also, the branch must exist before performing the write. -The operation does **not** create the branch if it does not exist. -For more information on branches please refer to [branches](branching.md). + +The branch must exist before performing write. Operations do **not** create the branch if it does not exist. +A branch can be created using [Spark DDL](spark-ddl.md#branching-and-tagging-ddl). !!! info Note: When writing to a branch, the current schema of the table will be used for validation. +### Via SQL + +Branch writes can be performed by providing a branch identifier, `branch_yourBranch` in the operation. + +Branch writes can also be performed as part of a write-audit-publish (WAP) workflow by specifying the `spark.wap.branch` config. +Note WAP branch and branch identifier cannot both be specified. ```sql -- INSERT (1,' a') (2, 'b') into the audit branch. @@ -228,6 +231,22 @@ SET spark.wap.branch = audit-branch INSERT INTO prod.db.table VALUES (3, 'c'); ``` +### Via DataFrames + +Branch writes via DataFrames can be performed by providing a branch identifier, `branch_yourBranch` in the operation. + +```scala +// To insert into `audit` branch +val data: DataFrame = ... +data.writeTo("prod.db.table.branch_audit").append() +``` + +```scala +// To overwrite `audit` branch +val data: DataFrame = ... +data.writeTo("prod.db.table.branch_audit").overwritePartitions() +``` + ## Writing with DataFrames Spark 3 introduced the new `DataFrameWriterV2` API for writing to tables using data frames. The v2 API is recommended for several reasons: From 77404520996f0d03dac21ae03a6c0fb0fffbbc1b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 17 Jul 2024 14:14:43 -0600 Subject: [PATCH 0483/1019] Core: Make new TableMetadata.Builder constructor private (#10714) --- core/src/main/java/org/apache/iceberg/TableMetadata.java | 2 +- 1 file changed, 1 insertion(+), 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 e8dcfc85fbf0..74b8ad0bbddc 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -910,7 +910,7 @@ private Builder() { this(DEFAULT_TABLE_FORMAT_VERSION); } - public Builder(int formatVersion) { + private Builder(int formatVersion) { this.base = null; this.formatVersion = formatVersion; this.lastSequenceNumber = INITIAL_SEQUENCE_NUMBER; From 111c1e295a54c8f52178ad31d5405223d71b2a06 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 18 Jul 2024 02:35:21 +0200 Subject: [PATCH 0484/1019] Common: Update the version in deprecation messages (#10715) --- .../src/main/java/org/apache/iceberg/common/DynFields.java | 2 +- .../src/main/java/org/apache/iceberg/common/DynMethods.java | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/common/src/main/java/org/apache/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java index 1b982d206fdb..e88affa0cbdd 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynFields.java +++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java @@ -394,7 +394,7 @@ public BoundField build(Object target) { * @return a {@link StaticField} with a valid implementation * @throws IllegalStateException if the method is not static * @throws NoSuchFieldException if no implementation was found - * @deprecated since 1.7.0, will be removed in 2.0.0 + * @deprecated since 1.6.0, will be removed in 1.7.0 */ @Deprecated public StaticField buildStaticChecked() throws NoSuchFieldException { diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index eaaad876f133..fc0e578c7dd1 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -51,7 +51,7 @@ public static class UnboundMethod { (method == null || method.isVarArgs()) ? -1 : method.getParameterTypes().length; } - /** @deprecated since 1.7.0, will be removed in 2.0.0 */ + /** @deprecated since 1.6.0, will be removed in 1.7.0 */ @Deprecated // will become private @SuppressWarnings("unchecked") public R invokeChecked(Object target, Object... args) throws Exception { @@ -315,7 +315,7 @@ public Builder impl(Class targetClass, Class... argClasses) { return this; } - /** @deprecated since 1.7.0, will be removed in 2.0.0 */ + /** @deprecated since 1.6.0, will be removed in 1.7.0 */ @Deprecated public Builder ctorImpl(Class targetClass, Class... argClasses) { // don't do any work if an implementation has been found @@ -331,7 +331,7 @@ public Builder ctorImpl(Class targetClass, Class... argClasses) { return this; } - /** @deprecated since 1.7.0, will be removed in 2.0.0 */ + /** @deprecated since 1.6.0, will be removed in 1.7.0 */ @Deprecated public Builder ctorImpl(String className, Class... argClasses) { // don't do any work if an implementation has been found From fcbf1463871a88de53fd55422bb2c623a3064dcf Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Thu, 18 Jul 2024 16:58:05 +0200 Subject: [PATCH 0485/1019] Docs: Fix link on Concepts page (#10718) --- site/docs/concepts/catalog.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/site/docs/concepts/catalog.md b/site/docs/concepts/catalog.md index e83947e07f78..ee0de3b93836 100644 --- a/site/docs/concepts/catalog.md +++ b/site/docs/concepts/catalog.md @@ -26,7 +26,7 @@ You may think of Iceberg as a format for managing data in a single table, but th The first step when using an Iceberg client is almost always initializing and configuring a catalog. The configured catalog is then used by compute engines to execute catalog operations. Multiple types of compute engines using a shared Iceberg catalog allows them to share a common data layer. -A catalog is almost always configured through the processing engine which passes along a set of properties during initialization. Different processing engines have different ways to configure a catalog. When configuring a catalog, it’s always best to refer to the [Iceberg documentation](docs/latest/configuration.md#catalog-properties) as well as the docs for the specific processing engine being used. Ultimately, these configurations boil down to a common set of catalog properties that will be passed to configure the Iceberg catalog. +A catalog is almost always configured through the processing engine which passes along a set of properties during initialization. Different processing engines have different ways to configure a catalog. When configuring a catalog, it’s always best to refer to the [Iceberg documentation](../docs/latest/configuration.md#catalog-properties) as well as the docs for the specific processing engine being used. Ultimately, these configurations boil down to a common set of catalog properties that will be passed to configure the Iceberg catalog. ## Catalog Implementations From dd1653dc5c4230c60a7aafcfba682e5728b1f7b8 Mon Sep 17 00:00:00 2001 From: Farooq Qaiser Date: Thu, 18 Jul 2024 19:27:06 -0400 Subject: [PATCH 0486/1019] Core: Support appending files with different specs (#9860) --- .../iceberg/MergingSnapshotProducer.java | 102 ++++++++++-------- .../org/apache/iceberg/TestMergeAppend.java | 88 +++++++++++++++ 2 files changed, 145 insertions(+), 45 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 1a4560416dc2..b4c0567ab73a 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -30,6 +30,7 @@ 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.RuntimeIOException; @@ -42,6 +43,7 @@ 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; @@ -79,7 +81,7 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final ManifestFilterManager deleteFilterManager; // update data - private final List newDataFiles = Lists.newArrayList(); + private final Map> newDataFilesBySpec = Maps.newHashMap(); private final CharSequenceSet newDataFilePaths = CharSequenceSet.empty(); private final CharSequenceSet newDeleteFilePaths = CharSequenceSet.empty(); private Long newDataFilesDataSequenceNumber; @@ -89,10 +91,9 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { private final SnapshotSummary.Builder addedFilesSummary = SnapshotSummary.builder(); private final SnapshotSummary.Builder appendedManifestsSummary = SnapshotSummary.builder(); private Expression deleteExpression = Expressions.alwaysFalse(); - private PartitionSpec dataSpec; // cache new data manifests after writing - private List cachedNewDataManifests = null; + private final List cachedNewDataManifests = Lists.newLinkedList(); private boolean hasNewDataFiles = false; // cache new manifests for delete files @@ -105,7 +106,6 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { super(ops); this.tableName = tableName; this.ops = ops; - this.dataSpec = null; long targetSizeBytes = ops.current() .propertyAsLong(MANIFEST_TARGET_SIZE_BYTES, MANIFEST_TARGET_SIZE_BYTES_DEFAULT); @@ -141,10 +141,18 @@ protected boolean isCaseSensitive() { } protected PartitionSpec dataSpec() { + Set specs = dataSpecs(); Preconditions.checkState( - dataSpec != null, "Cannot determine partition spec: no data files have been added"); - // the spec is set when the write is started - return dataSpec; + 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(); + Preconditions.checkState( + !specs.isEmpty(), "Cannot determine partition specs: no data files have been added"); + return ImmutableSet.copyOf(specs); } protected Expression rowFilter() { @@ -152,7 +160,12 @@ protected Expression rowFilter() { } protected List addedDataFiles() { - return ImmutableList.copyOf(newDataFiles); + return ImmutableList.copyOf( + newDataFilesBySpec.values().stream().flatMap(List::stream).collect(Collectors.toList())); + } + + protected Map> addedDataFilesBySpec() { + return ImmutableMap.copyOf(newDataFilesBySpec); } protected void failAnyDelete() { @@ -212,7 +225,7 @@ protected boolean deletesDeleteFiles() { } protected boolean addsDataFiles() { - return !newDataFiles.isEmpty(); + return !newDataFilesBySpec.isEmpty(); } protected boolean addsDeleteFiles() { @@ -223,9 +236,17 @@ protected boolean addsDeleteFiles() { protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); if (newDataFilePaths.add(file.path())) { - setDataSpec(file); - addedFilesSummary.addedFile(dataSpec(), 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); hasNewDataFiles = true; + List newDataFiles = + newDataFilesBySpec.computeIfAbsent(fileSpec, ignored -> Lists.newArrayList()); newDataFiles.add(file); } } @@ -255,17 +276,6 @@ private void add(DeleteFileHolder fileHolder) { } } - private void setDataSpec(DataFile file) { - PartitionSpec fileSpec = ops.current().spec(file.specId()); - Preconditions.checkNotNull( - fileSpec, "Cannot find partition spec for data file: %s", file.path()); - if (dataSpec == null) { - dataSpec = fileSpec; - } else if (dataSpec.specId() != file.specId()) { - throw new ValidationException("Invalid data file, expected spec id: %d", dataSpec.specId()); - } - } - /** Add all files in a manifest to the new snapshot. */ protected void add(ManifestFile manifest) { Preconditions.checkArgument( @@ -885,7 +895,7 @@ public Object updateEvent() { @SuppressWarnings("checkstyle:CyclomaticComplexity") private void cleanUncommittedAppends(Set committed) { - if (cachedNewDataManifests != null) { + if (!cachedNewDataManifests.isEmpty()) { boolean hasDeletes = false; for (ManifestFile manifest : cachedNewDataManifests) { if (!committed.contains(manifest)) { @@ -895,7 +905,7 @@ private void cleanUncommittedAppends(Set committed) { } if (hasDeletes) { - this.cachedNewDataManifests = null; + this.cachedNewDataManifests.clear(); } } @@ -941,7 +951,7 @@ protected void cleanUncommitted(Set committed) { private Iterable prepareNewDataManifests() { Iterable newManifests; - if (!newDataFiles.isEmpty()) { + if (!newDataFilesBySpec.isEmpty()) { List dataFileManifests = newDataFilesAsManifests(); newManifests = Iterables.concat(dataFileManifests, appendManifests, rewrittenAppendManifests); } else { @@ -954,29 +964,31 @@ private Iterable prepareNewDataManifests() { } private List newDataFilesAsManifests() { - if (hasNewDataFiles && cachedNewDataManifests != null) { + if (hasNewDataFiles && !cachedNewDataManifests.isEmpty()) { cachedNewDataManifests.forEach(file -> deleteFile(file.path())); - cachedNewDataManifests = null; + cachedNewDataManifests.clear(); } - if (cachedNewDataManifests == null) { - try { - RollingManifestWriter writer = newRollingManifestWriter(dataSpec()); - try { - if (newDataFilesDataSequenceNumber == null) { - newDataFiles.forEach(writer::add); - } else { - newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); - } - } finally { - writer.close(); - } - - this.cachedNewDataManifests = writer.toManifestFiles(); - this.hasNewDataFiles = false; - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close manifest writer"); - } + if (cachedNewDataManifests.isEmpty()) { + newDataFilesBySpec.forEach( + (dataSpec, newDataFiles) -> { + try { + RollingManifestWriter writer = newRollingManifestWriter(dataSpec); + try { + if (newDataFilesDataSequenceNumber == null) { + newDataFiles.forEach(writer::add); + } else { + newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); + } + } finally { + writer.close(); + } + this.cachedNewDataManifests.addAll(writer.toManifestFiles()); + this.hasNewDataFiles = false; + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to close manifest writer"); + } + }); } return cachedNewDataManifests; diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index 4719923e72ac..abfcb318334b 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -27,12 +27,14 @@ import java.io.IOException; import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; import org.apache.iceberg.exceptions.CommitFailedException; +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.Sets; import org.apache.iceberg.types.Types; @@ -90,6 +92,92 @@ public void testEmptyTableAppend() { statuses(Status.ADDED, Status.ADDED)); } + @TestTemplate + public void testEmptyTableAppendFilesWithDifferentSpecs() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + TableMetadata base = readMetadata(); + assertThat(base.currentSnapshot()).as("Should not have a current snapshot").isNull(); + assertThat(base.lastSequenceNumber()).as("Last sequence number should be 0").isEqualTo(0); + + table.updateSpec().addField("id").commit(); + PartitionSpec newSpec = table.spec(); + + assertThat(table.specs()).as("Table should have 2 specs").hasSize(2); + + DataFile fileNewSpec = + DataFiles.builder(newSpec) + .withPath("/path/to/data-b.parquet") + .withPartitionPath("data_bucket=0/id=0") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + Snapshot committedSnapshot = + commit(table, table.newAppend().appendFile(FILE_A).appendFile(fileNewSpec), branch); + + assertThat(committedSnapshot).as("Should create a snapshot").isNotNull(); + V1Assert.assertEquals( + "Last sequence number should be 0", 0, table.ops().current().lastSequenceNumber()); + V2Assert.assertEquals( + "Last sequence number should be 1", 1, table.ops().current().lastSequenceNumber()); + + assertThat(committedSnapshot.allManifests(table.io())) + .as("Should create 2 manifests for initial write, 1 manifest per spec") + .hasSize(2); + + long snapshotId = committedSnapshot.snapshotId(); + + ImmutableMap expectedFileBySpec = + ImmutableMap.of(SPEC.specId(), FILE_A, newSpec.specId(), fileNewSpec); + + expectedFileBySpec.forEach( + (specId, expectedDataFile) -> { + ManifestFile manifestFileForSpecId = + committedSnapshot.allManifests(table.io()).stream() + .filter(m -> Objects.equals(m.partitionSpecId(), specId)) + .findAny() + .get(); + + validateManifest( + manifestFileForSpecId, + dataSeqs(1L), + fileSeqs(1L), + ids(snapshotId), + files(expectedDataFile), + statuses(Status.ADDED)); + }); + } + + @TestTemplate + public void testDataSpecThrowsExceptionIfDataFilesWithDifferentSpecsAreAdded() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + TableMetadata base = readMetadata(); + assertThat(base.currentSnapshot()).as("Should not have a current snapshot").isNull(); + assertThat(base.lastSequenceNumber()).as("Last sequence number should be 0").isEqualTo(0); + + table.updateSpec().addField("id").commit(); + PartitionSpec newSpec = table.spec(); + + assertThat(table.specs()).as("Table should have 2 specs").hasSize(2); + + DataFile fileNewSpec = + DataFiles.builder(newSpec) + .withPath("/path/to/data-b.parquet") + .withPartitionPath("data_bucket=0/id=0") + .withFileSizeInBytes(10) + .withRecordCount(1) + .build(); + + MergeAppend mergeAppend = + (MergeAppend) table.newAppend().appendFile(FILE_A).appendFile(fileNewSpec); + assertThatThrownBy(mergeAppend::dataSpec) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Cannot return a single partition spec: data files with different partition specs have been added"); + } + @TestTemplate public void testEmptyTableAppendManifest() throws IOException { assertThat(listManifestFiles()).isEmpty(); From d9a1b6a445bcfbaaf2211345c87250e0870a39ec Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 19 Jul 2024 03:30:01 +0200 Subject: [PATCH 0487/1019] Core: Remove unnecessary class-level synchronized in ManifestFiles (#10544) --- core/src/main/java/org/apache/iceberg/ManifestFiles.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 9009f19ec947..840c90bebdde 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -81,7 +81,7 @@ static ContentCache contentCache(FileIO io) { } /** Drop manifest file cache object for a FileIO if exists. */ - public static synchronized void dropCache(FileIO fileIO) { + public static void dropCache(FileIO fileIO) { CONTENT_CACHES.invalidate(fileIO); CONTENT_CACHES.cleanUp(); } From 10e757a93474feb68839152adaa7a4389f344bee Mon Sep 17 00:00:00 2001 From: emkornfield Date: Fri, 19 Jul 2024 10:23:15 -0700 Subject: [PATCH 0488/1019] Spec: Clarify which columns can be used for equality delete files. (#8981) Co-authored-by: Fokko Driesprong --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index dd4e901f3706..1c029bdcb563 100644 --- a/format/spec.md +++ b/format/spec.md @@ -851,7 +851,7 @@ The rows in the delete file must be sorted by `file_path` then `pos` to optimize 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. -Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). Float and double columns cannot be used as delete columns in equality delete files. +Equality delete files store any subset of a table's columns and use the table's field ids. The _delete columns_ are the columns of the delete file used to match data rows. Delete columns are identified by id in the delete file [metadata column `equality_ids`](#manifests). The column restrictions for columns used in equality delete files are the same as those for [identifier fields](#identifier-field-ids) with the exception that optional columns and columns nested under optional structs are allowed (if a parent struct column is null it implies the leaf column is null). A data row is deleted if its values are equal to all delete columns for any row in an equality delete file that applies to the row's data file (see [`Scan Planning`](#scan-planning)). From 68a39cda206889f988a12cf72c85f0b62b1f7510 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 21 Jul 2024 19:14:14 +0200 Subject: [PATCH 0489/1019] Build: Bump nessie from 0.92.1 to 0.93.1 (#10727) --- 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 267d4c2f0da5..460a0ee20733 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.92.1" +nessie = "0.93.1" netty-buffer = "4.1.111.Final" netty-buffer-compat = "4.1.111.Final" object-client-bundle = "3.3.2" From ec42cece79b24b3efb85d5e8520552000567d083 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 08:41:28 +0200 Subject: [PATCH 0490/1019] Build: Bump org.testcontainers:testcontainers from 1.19.8 to 1.20.0 (#10730) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.19.8 to 1.20.0. - [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.19.8...1.20.0) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 460a0ee20733..90a1944ae8f7 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,7 +84,7 @@ spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.37" sqlite-jdbc = "3.46.0.0" -testcontainers = "1.19.8" +testcontainers = "1.20.0" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 972ccb044023e5eb9e8962fd0c03fbdb0d9fb703 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 09:16:53 +0200 Subject: [PATCH 0491/1019] Build: Bump mkdocs-material from 9.5.28 to 9.5.29 (#10734) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.28 to 9.5.29. - [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.28...9.5.29) --- 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 75beac22648f..8383b631b6b5 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.28 +mkdocs-material==9.5.29 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 1f0f9c09342e753290b84d1a27632a26aeb9cc14 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 09:18:17 +0200 Subject: [PATCH 0492/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.2.0 to 1.2.1 (#10733) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.2.0 to 1.2.1. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.2.0...1.2.1) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 90a1944ae8f7..afa0be4ad27c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -73,7 +73,7 @@ object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.2.0" +roaringbitmap = "1.2.1" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" From c9a0434160de2eea3d17b7733096dba1d848a9c1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 09:18:30 +0200 Subject: [PATCH 0493/1019] Build: Bump software.amazon.awssdk:bom from 2.26.20 to 2.26.21 (#10729) Bumps software.amazon.awssdk:bom from 2.26.20 to 2.26.21. --- 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 afa0be4ad27c..498e2467acfa 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.3" awaitility = "4.2.1" -awssdk-bom = "2.26.20" +awssdk-bom = "2.26.21" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From c0e41da580083a373ef1af1b47af9d55a065a4be Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 09:18:40 +0200 Subject: [PATCH 0494/1019] Build: Bump io.netty:netty-buffer from 4.1.111.Final to 4.1.112.Final (#10726) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.111.Final to 4.1.112.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.111.Final...netty-4.1.112.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 498e2467acfa..a21e2d980b11 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -67,8 +67,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.93.1" -netty-buffer = "4.1.111.Final" -netty-buffer-compat = "4.1.111.Final" +netty-buffer = "4.1.112.Final" +netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" orc = "1.9.3" parquet = "1.13.1" From 62cee230c232ca92fc2311c5895faf5e25a3c6c6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 09:18:52 +0200 Subject: [PATCH 0495/1019] Build: Bump orc from 1.9.3 to 1.9.4 (#10728) Bumps `orc` from 1.9.3 to 1.9.4. Updates `org.apache.orc:orc-core` from 1.9.3 to 1.9.4 Updates `org.apache.orc:orc-tools` from 1.9.3 to 1.9.4 --- 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 a21e2d980b11..d64bb8a0d70b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,7 +70,7 @@ nessie = "0.93.1" netty-buffer = "4.1.112.Final" netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" -orc = "1.9.3" +orc = "1.9.4" parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.2.1" From 01df735516e42f5d8c2714d43a893f3d658e680a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 22 Jul 2024 09:47:13 +0200 Subject: [PATCH 0496/1019] Build: Bump com.google.errorprone:error_prone_annotations (#10731) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.28.0 to 2.29.2. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.28.0...v2.29.2) --- 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 d64bb8a0d70b..e2e2bf34a882 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -37,7 +37,7 @@ datasketches = "6.0.0" delta-standalone = "3.2.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.28.0" +errorprone-annotations = "2.29.2" findbugs-jsr305 = "3.0.2" flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} From e739e2f0d75965fe27306c69ee5787b085a8e556 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Mon, 22 Jul 2024 17:25:00 +0900 Subject: [PATCH 0497/1019] Flink: Migrate remaining classes to JUnit5 (#10684) --- .../iceberg/flink/HadoopCatalogResource.java | 90 -------- .../iceberg/flink/HadoopTableResource.java | 64 ------ .../flink/MiniFlinkClusterExtension.java | 14 ++ .../apache/iceberg/flink/SimpleDataUtil.java | 10 +- .../apache/iceberg/flink/TestFixtures.java | 2 + .../flink/sink/TestCompressionSettings.java | 163 +++++++------- .../iceberg/flink/sink/TestFlinkManifest.java | 51 +++-- .../flink/sink/TestIcebergStreamWriter.java | 126 +++++------ .../flink/sink/TestRowDataPartitionKey.java | 67 +++--- .../iceberg/flink/sink/TestTaskWriters.java | 87 ++++---- .../TestAggregatedStatisticsTracker.java | 2 +- ...TestIcebergSourceBoundedGenericRecord.java | 75 +++---- .../source/TestIcebergSourceContinuous.java | 199 ++++++++++-------- .../source/TestIcebergSourceFailover.java | 154 ++++++++------ ...gSourceFailoverWithWatermarkExtractor.java | 16 +- ...stIcebergSourceWithWatermarkExtractor.java | 44 ++-- .../flink/source/TestProjectMetaColumn.java | 46 ++-- .../iceberg/flink/source/TestSourceUtil.java | 17 +- 18 files changed, 569 insertions(+), 658 deletions(-) delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +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; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.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.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java index 9a73b80e077d..d2e086aa448e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; @@ -50,4 +51,17 @@ public static MiniClusterExtension createWithClassloaderCheckDisabled() { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ce6caca12158..1767f774922a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Duration; @@ -71,7 +72,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; -import org.junit.Assert; public class SimpleDataUtil { @@ -268,13 +268,13 @@ public static boolean equalsRecords(List expected, List actual, } public static void assertRecordsEqual(List expected, List actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); + assertThat(actual).hasSameSizeAs(expected); Types.StructType type = schema.asStruct(); StructLikeSet expectedSet = StructLikeSet.create(type); expectedSet.addAll(expected); StructLikeSet actualSet = StructLikeSet.create(type); actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } /** @@ -297,7 +297,7 @@ public static void assertTableRecords(Table table, List expected, String Snapshot snapshot = latestSnapshot(table, branch); if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); + assertThat(expected).isEmpty(); return; } @@ -313,7 +313,7 @@ public static void assertTableRecords(Table table, List expected, String actualSet.add(record); } - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java index 884ea2d1d3b1..b9a7d5b1d589 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -46,6 +46,8 @@ private TestFixtures() {} public static final String SINK_TABLE = "t_sink"; public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); public static final Schema TS_SCHEMA = new Schema( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 14f12422da96..8faae1b05a4e 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; @@ -35,53 +42,49 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; - private final Map initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } + @Parameter(index = 0) + private Map initProperties; - public TestCompressionSettings(Map initProperties) { - this.initProperties = initProperties; + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; } - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); } - @Test + @TestTemplate public void testCompressionAvro() throws Exception { // No override provided Map resultProperties = @@ -91,19 +94,17 @@ public void testCompressionAvro() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -119,11 +120,12 @@ public void testCompressionAvro() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionParquet() throws Exception { // No override provided Map resultProperties = @@ -133,19 +135,19 @@ public void testCompressionParquet() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -161,11 +163,12 @@ public void testCompressionParquet() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionOrc() throws Exception { // No override provided Map resultProperties = @@ -175,19 +178,18 @@ public void testCompressionOrc() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); } // Override compression to snappy and a different strategy @@ -203,8 +205,9 @@ public void testCompressionOrc() throws Exception { FlinkWriteOptions.COMPRESSION_STRATEGY.key(), "speed")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index ce1f208a4b07..53b7c4c0cc91 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -19,9 +19,12 @@ package org.apache.iceberg.flink.sink; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -45,28 +48,26 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkManifest { private static final Configuration CONF = new Configuration(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; private FileAppenderFactory appenderFactory; private final AtomicInteger fileCount = new AtomicInteger(0); - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + assertThat(new File(tablePath).mkdir()).isTrue(); // Construct the iceberg table. table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); @@ -112,11 +113,11 @@ public void testIO() throws IOException { WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < 5; i++) { TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); } @@ -131,7 +132,7 @@ public void testUserProvidedManifestLocation() throws IOException { long checkpointId = 1; String flinkJobId = newFlinkJobId(); String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = @@ -144,21 +145,18 @@ public void testUserProvidedManifestLocation() throws IOException { () -> factory.create(checkpointId), table.spec()); - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } @@ -198,7 +196,7 @@ public void testVersionedSerializer() throws IOException { byte[] versionedSerializeData2 = SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); } @Test @@ -220,14 +218,13 @@ public void testCompatibility() throws IOException { DeltaManifests delta = SimpleVersionedSerialization.readVersionAndDeSerialize( DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); TestHelpers.assertEquals(manifest, delta.dataManifest()); List actualFiles = FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); + assertThat(actualFiles).hasSize(10); for (int i = 0; i < 10; i++) { TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index c05f08b82f51..50283f7ad215 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Locale; @@ -42,6 +43,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; 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; @@ -58,49 +62,44 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; private Table table; - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; try (OneInputStreamOperatorTestHarness testHarness = @@ -111,10 +110,10 @@ public void testWritingTable() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); checkpointId = checkpointId + 1; @@ -125,8 +124,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); @@ -145,7 +144,7 @@ public void testWritingTable() throws Exception { } } - @Test + @TestTemplate public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; @@ -155,39 +154,39 @@ public void testSnapshotTwice() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } } - @Test + @TestTemplate public void testTableWithoutSnapshot() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); + assertThat(scanDataFiles()).isEmpty(); try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); + assertThat(scanDataFiles()).hasSize(1); } private Set scanDataFiles() throws IOException { @@ -211,7 +210,7 @@ private Set scanDataFiles() throws IOException { } } - @Test + @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { @@ -221,21 +220,21 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { @@ -244,22 +243,22 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testTableWithTargetFileSize() throws Exception { // Adjust the target-file-size in table properties. table @@ -285,12 +284,12 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); // Assert that the data file have the expected records. for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); + assertThat(dataFile.recordCount()).isEqualTo(1000); } // Commit the iceberg transaction. @@ -303,7 +302,7 @@ public void testTableWithTargetFileSize() throws Exception { SimpleDataUtil.assertTableRecords(table, records); } - @Test + @TestTemplate public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( @@ -329,7 +328,8 @@ public void testPromotedFlinkDataType() throws Exception { spec = PartitionSpec.unpartitioned(); } - String location = tempFolder.newFolder().getAbsolutePath(); + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); @@ -353,8 +353,8 @@ public void testPromotedFlinkDataType() throws Exception { } testHarness.prepareSnapshotPreBarrier(1); WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java index 79be96d20902..919fef579ab0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +37,7 @@ import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRowDataPartitionKey { private static final Schema SCHEMA = @@ -91,10 +92,10 @@ public void testNullPartitionValue() { for (RowData row : rows) { PartitionKey partitionKey = new PartitionKey(spec, schema); partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); + assertThat(partitionKey.size()).isEqualTo(1); String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); } } @@ -116,15 +117,15 @@ public void testPartitionWithOneNestedField() { PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); + assertThat(partitionKey1.size()).isEqualTo(1); - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); + assertThat(partitionKey2.size()).isEqualTo(1); - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); } } @@ -154,16 +155,16 @@ public void testPartitionMultipleNestedField() { Record record = (Record) records.get(i).get(0); pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); + assertThat(pk1.size()).isEqualTo(2); - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); + assertThat(pk2.size()).isEqualTo(2); - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); } } @@ -190,19 +191,18 @@ public void testPartitionValueTypes() { pk.partition(rowWrapper.wrap(row)); expectedPK.partition(recordWrapper.wrap(record)); - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } @@ -232,19 +232,18 @@ public void testNestedPartitionValues() { pk.partition(rowWrapper.wrap(rows.get(j))); expectedPK.partition(recordWrapper.wrap(records.get(j))); - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } 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 112dbb511310..8bfd6cb3d043 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 @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; @@ -30,6 +33,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -39,69 +45,62 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestTaskWriters { private static final Configuration CONF = new Configuration(); private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; - private Table table; + @Parameter(index = 1) + private boolean partitioned; - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } + private Table table; - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table with the specified file format. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); // Close again. taskWriter.close(); dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); } } - @Test + @TestTemplate public void testCloseTwice() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -111,16 +110,16 @@ public void testCloseTwice() throws IOException { int expectedFiles = partitioned ? 2 : 1; DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } } } - @Test + @TestTemplate public void testAbort() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -130,16 +129,16 @@ public void testAbort() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); } } } - @Test + @TestTemplate public void testCompleteFiles() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "a")); @@ -149,14 +148,14 @@ public void testCompleteFiles() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); @@ -176,7 +175,7 @@ public void testCompleteFiles() throws IOException { } } - @Test + @TestTemplate public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter taskWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); @@ -193,7 +192,7 @@ public void testRollingWithTargetFileSize() throws IOException { } DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); + assertThat(dataFiles).hasSize(8); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { @@ -206,7 +205,7 @@ public void testRollingWithTargetFileSize() throws IOException { } } - @Test + @TestTemplate public void testRandomData() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 52485a3c67d9..13e2e0ce2115 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -24,7 +24,7 @@ import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Test; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 0337f3597053..7bfed00a9eb4 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -18,10 +18,10 @@ */ 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.Locale; import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -30,10 +30,12 @@ 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.test.util.MiniClusterWithClientResource; 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; @@ -43,8 +45,7 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; @@ -55,56 +56,48 @@ 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.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +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; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedGenericRecord { + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} + {FileFormat.AVRO, 2}, + {FileFormat.PARQUET, 2}, + {FileFormat.ORC, 2} }; } - private final FileFormat fileFormat; - private final int parallelism; + @Parameter(index = 0) + private FileFormat fileFormat; - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } + @Parameter(index = 1) + private int parallelism; - @Test + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { String dateStr = "2020-03-20"; Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); @@ -112,19 +105,19 @@ public void testPartitionedTable() throws Exception { expectedRecords.get(i).setField("dt", dateStr); } - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testProjection() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + 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)); @@ -149,7 +142,7 @@ private List run( Configuration config = new Configuration(); config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { + try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { tableLoader.open(); table = tableLoader.loadTable(); } @@ -168,7 +161,7 @@ private List run( IcebergSource.Builder sourceBuilder = IcebergSource.builder() - .tableLoader(catalogResource.tableLoader()) + .tableLoader(CATALOG_EXTENSION.tableLoader()) .readerFunction(readerFunction) .assignerFactory(new SimpleSplitAssignerFactory()) .flinkConfig(config); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 5765b73a1f63..749cbf89338a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -38,7 +39,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; @@ -46,45 +48,43 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceContinuous { public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(METRIC_REPORTER); + @TempDir protected Path temporaryFolder; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private final AtomicLong randomSeed = new AtomicLong(0L); @Test public void testTableScanThenIncremental() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -97,27 +97,27 @@ public void testTableScanThenIncremental() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -126,22 +126,24 @@ public void testTableScanThenIncremental() throws Exception { @Test public void testTableScanThenIncrementalAfterExpiration() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot2 List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - Assert.assertEquals(1, tableResource.table().history().size()); + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); ScanContext scanContext = ScanContext.builder() @@ -150,8 +152,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { @@ -159,17 +161,17 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List initialRecords = Lists.newArrayList(); initialRecords.addAll(batch1); initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -178,16 +180,18 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { @Test public void testEarliestSnapshot() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -202,43 +206,46 @@ public void testEarliestSnapshot() throws Exception { List result1 = waitForResult(iter, 4); List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } } @Test - public void testLatestSnapshot() throws Exception { + public void testLatestSnapshot(@InjectClusterClient ClusterClient clusterClient) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -252,29 +259,29 @@ public void testLatestSnapshot() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // we want to make sure job is running first so that enumerator can // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + waitUntilJobIsRunning(clusterClient); // inclusive behavior for starting snapshot List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -283,19 +290,21 @@ public void testLatestSnapshot() throws Exception { @Test public void testSpecificSnapshotId() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); ScanContext scanContext = ScanContext.builder() @@ -308,25 +317,25 @@ public void testSpecificSnapshotId() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -335,22 +344,24 @@ public void testSpecificSnapshotId() throws Exception { @Test public void testSpecificSnapshotTimestamp() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); // sleep for 2 ms to make sure snapshot1 has a higher timestamp value Thread.sleep(2); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); ScanContext scanContext = ScanContext.builder() @@ -364,25 +375,25 @@ public void testSpecificSnapshotTimestamp() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // consume data from snapshot1 List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -392,27 +403,30 @@ public void testSpecificSnapshotTimestamp() throws Exception { public void testReadingFromBranch() throws Exception { String branch = "b1"; GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); List batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchBase); // create branch - tableResource + TABLE_EXTENSION .table() .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) .commit(); // snapshot1 to branch List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch1); // snapshot2 to branch List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch2); List branchExpectedRecords = Lists.newArrayList(); @@ -432,25 +446,26 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); // snapshot3 to branch List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); // snapshot4 to branch List batch4 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch4); List result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); } // read only from main branch. Should contain only the first snapshot @@ -463,14 +478,14 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); List batchMain2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchMain2); resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); } } @@ -479,7 +494,7 @@ public void testValidation() { assertThatThrownBy( () -> IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(true) .endTag("tag") @@ -495,7 +510,7 @@ private DataStream createStream(ScanContext scanContext) throws Exception { DataStream stream = env.fromSource( IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(scanContext.isStreaming()) .streamingStartingStrategy(scanContext.streamingStartingStrategy()) @@ -507,7 +522,7 @@ private DataStream createStream(ScanContext scanContext) throws Exception { WatermarkStrategy.noWatermarks(), "icebergSource", TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); return stream; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index a378a0b93be3..62558fc9c8e5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Path; import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,6 +31,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; @@ -41,7 +43,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -51,53 +56,66 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +@Timeout(value = 120) public class TestIcebergSourceFailover { // Parallelism higher than 1, but lower than the number of splits used by some of our tests // The goal is to allow some splits to remain in the enumerator when restoring the state private static final int PARALLELISM = 2; private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) // Prevent combining splits .set( @@ -120,12 +138,11 @@ protected void assertRecords(Table table, List expectedRecords, Duration } @Test - public void testBoundedWithSavepoint() throws Exception { + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) + throws Exception { List expectedRecords = Lists.newArrayList(); - Table sinkTable = sinkTableResource.table(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List records = generateRecords(2, i); expectedRecords.addAll(records); @@ -141,13 +158,8 @@ public void testBoundedWithSavepoint() throws Exception { // Write something, but do not finish before checkpoint is created RecordCounterToWait.waitForCondition(); CompletableFuture savepoint = - miniClusterResource - .getClusterClient() - .stopWithSavepoint( - jobId, - false, - TEMPORARY_FOLDER.newFolder().toPath().toString(), - SavepointFormatType.CANONICAL); + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint @@ -170,19 +182,21 @@ public void testBoundedWithSavepoint() throws Exception { @Test public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); } - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { List expectedRecords = Lists.newArrayList(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List records = generateRecords(2, i); expectedRecords.addAll(records); @@ -197,29 +211,27 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio JobID jobId = jobClient.getJobID(); RecordCounterToWait.waitForCondition(); - triggerFailover( - failoverType, - jobId, - RecordCounterToWait::continueProcessing, - miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } @Test public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); } - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); List batch = generateRecords(2, 0); @@ -247,8 +259,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); @@ -260,13 +272,13 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep expectedRecords.addAll(records); dataAppender.appendToTable(records); if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, () -> {}, miniCluster); } } // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { @@ -286,8 +298,8 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); } @@ -295,6 +307,20 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ + private static void runTestWithNewMiniCluster(ThrowingConsumer testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + private enum FailoverType { NONE, TM, diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index cde39ec98d61..4f61d2f7308a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -49,11 +50,24 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + @Override protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) // Prevent combining splits diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 8341c8074f21..5e6a2b3caec6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; +import java.nio.file.Path; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -37,6 +38,7 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.InMemoryReporter; @@ -49,24 +51,24 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; 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.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; @@ -74,25 +76,24 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int RECORD_NUM_FOR_2_SPLITS = 200; private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) .withHaLeadershipControl() .build()); - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); /** * This is an integration test for watermark handling and windowing. Integration testing the @@ -259,7 +260,7 @@ public void apply( * the readers continue reading. */ @Test - public void testThrottling() throws Exception { + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { GenericAppenderHelper dataAppender = appender(); // Generate records in advance @@ -310,8 +311,7 @@ public void testThrottling() throws Exception { try (CloseableIterator resultIterator = stream.collectAsync()) { JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); // Insert the first data into the table dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); @@ -353,7 +353,7 @@ public void testThrottling() throws Exception { protected IcebergSource source() { return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) .splitSize(100L) @@ -377,7 +377,7 @@ protected Record generateRecord(int minutes, String str) { private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() + return REPORTER.findMetrics(jobID, metricsName).values().stream() .map(m -> (Gauge) m) .filter(m -> m.getValue() == withValue) .findFirst(); @@ -389,7 +389,7 @@ private GenericAppenderHelper appender() { hadoopConf.set("write.parquet.page-size-bytes", "64"); hadoopConf.set("write.parquet.row-group-size-bytes", "64"); return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); } private static RowData row(long time, long count) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java index 25ecec23d216..ce9054ad49b6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowDelta; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; @@ -40,20 +47,19 @@ 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.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestProjectMetaColumn { - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Iterable parameters() { return Lists.newArrayList( new Object[] {FileFormat.PARQUET}, @@ -61,13 +67,9 @@ public static Iterable parameters() { new Object[] {FileFormat.AVRO}); } - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, @@ -89,7 +91,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); + assertThat(rowData).isInstanceOf(GenericRowData.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); @@ -97,20 +99,20 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); } - @Test + @TestTemplate public void testV1SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(1); } - @Test + @TestTemplate public void testV2SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(2); } - @Test + @TestTemplate public void testV2RemoveMetaColumn() throws Exception { // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); @@ -132,7 +134,7 @@ public void testV2RemoveMetaColumn() throws Exception { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); + assertThat(rowData).isInstanceOf(RowDataProjection.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java index 317301260f66..b701419a7499 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSourceUtil { @Test @@ -30,31 +31,31 @@ public void testInferedParallelism() throws IOException { Configuration configuration = new Configuration(); // Empty table, infer parallelism should be at least 1 int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits // num : 2 parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + assertThat(parallelism).isEqualTo(2); // 2 splits and limit is 1 , max infer parallelism is default 100, // which is greater than splits num and limit, the parallelism is the limit value : 1 parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : // 1 parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); } } From 592f0128aed29610cbe0e76bc8473f5b0bc5b7b2 Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 23 Jul 2024 00:36:05 +0800 Subject: [PATCH 0498/1019] API, Build: Fix typo in comments in `Table` and `gradlew` (#10744) --- api/src/main/java/org/apache/iceberg/Table.java | 4 ++-- gradlew | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java index 6da9bbf2f4fe..97ea9ba76526 100644 --- a/api/src/main/java/org/apache/iceberg/Table.java +++ b/api/src/main/java/org/apache/iceberg/Table.java @@ -270,7 +270,7 @@ default AppendFiles newFastAppend() { ReplacePartitions newReplacePartitions(); /** - * Create a new {@link DeleteFiles delete API} to replace files in this table and commit. + * Create a new {@link DeleteFiles delete API} to delete files in this table and commit. * * @return a new {@link DeleteFiles} */ @@ -299,7 +299,7 @@ default UpdatePartitionStatistics updatePartitionStatistics() { } /** - * Create a new {@link ExpireSnapshots expire API} to manage snapshots in this table and commit. + * Create a new {@link ExpireSnapshots expire API} to expire snapshots in this table and commit. * * @return a new {@link ExpireSnapshots} */ diff --git a/gradlew b/gradlew index e353c27d23f8..f7166e1f63af 100755 --- a/gradlew +++ b/gradlew @@ -207,7 +207,7 @@ fi DEFAULT_JVM_OPTS='"-Xmx64m" "-Xms64m"' # Collect all arguments for the java command: -# * DEFAULT_JVM_OPTS, JAVA_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, +# * DEFAULT_JVM_OPTS, JAVA_OPTS, and optsEnvironmentVar are not allowed to contain shell fragments, # and any embedded shellness will be escaped. # * For example: A user cannot expect ${Hostname} to be expanded, as it is an environment variable and will be # treated as '${Hostname}' itself on the command line. From 3746091c5c1198b45f95fecae124fa9bd2f660dd Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 22 Jul 2024 10:10:37 -0700 Subject: [PATCH 0499/1019] Flink: parameterize Flink table source tests to test both old and FLIP-27 source implementations (#10741) --- .../flink/source/TableSourceTestBase.java | 104 +++++++++++++ .../flink/source/TestFlinkSourceConfig.java | 14 +- .../flink/source/TestFlinkTableSource.java | 138 +++++------------- .../flink/source/TableSourceTestBase.java | 104 +++++++++++++ .../flink/source/TestFlinkSourceConfig.java | 14 +- .../flink/source/TestFlinkTableSource.java | 138 +++++------------- .../flink/source/TableSourceTestBase.java | 105 +++++++++++++ .../flink/source/TestFlinkSourceConfig.java | 14 +- .../flink/source/TestFlinkTableSource.java | 138 +++++------------- 9 files changed, 454 insertions(+), 315 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java new file mode 100644 index 000000000000..32c81d9465a4 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,104 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index bc7194e38088..8131bd7ab0d3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,16 +20,17 @@ 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.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestFlinkSourceConfig extends TestFlinkTableSource { +public class TestFlinkSourceConfig extends TableSourceTestBase { private static final String TABLE = "test_table"; - @Test + @TestTemplate public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) @@ -37,7 +38,7 @@ public void testFlinkSessionConfig() { .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } - @Test + @TestTemplate public void testFlinkHintConfig() { List result = sql( @@ -46,8 +47,11 @@ public void testFlinkHintConfig() { assertThat(result).hasSize(3); } - @Test + @TestTemplate public void testReadOptionHierarchy() { + // TODO: FLIP-27 source doesn't implement limit pushdown yet + assumeThat(useFlip27Source).isFalse(); + getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); assertThat(result).hasSize(1); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index 01bab6d063fd..18528c789114 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -21,78 +21,16 @@ 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.util.List; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestFlinkTableSource extends TestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - } - - @Test +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate public void testLimitPushDown() { assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) @@ -121,7 +59,7 @@ public void testLimitPushDown() { assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } - @Test + @TestTemplate public void testNoFilterPushDown() { String sql = String.format("SELECT * FROM %s ", TABLE_NAME); List result = sql(sql); @@ -133,7 +71,7 @@ public void testNoFilterPushDown() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDownEqual() { String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -147,7 +85,7 @@ public void testFilterPushDownEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); @@ -156,7 +94,7 @@ public void testFilterPushDownEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownEqualLiteralOnLeft() { String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -170,7 +108,7 @@ public void testFilterPushDownEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqual() { String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") != 1"; @@ -187,7 +125,7 @@ public void testFilterPushDownNoEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); @@ -196,7 +134,7 @@ public void testFilterPushDownNoEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownAnd() { String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); @@ -211,7 +149,7 @@ public void testFilterPushDownAnd() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownOr() { String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; @@ -229,7 +167,7 @@ public void testFilterPushDownOr() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThan() { String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 1"; @@ -247,7 +185,7 @@ public void testFilterPushDownGreaterThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); @@ -256,7 +194,7 @@ public void testFilterPushDownGreaterThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanLiteralOnLeft() { String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 3"; @@ -274,7 +212,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqual() { String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 2"; @@ -292,7 +230,7 @@ public void testFilterPushDownGreaterThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); @@ -301,7 +239,7 @@ public void testFilterPushDownGreaterThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 2"; @@ -319,7 +257,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThan() { String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 2"; @@ -334,7 +272,7 @@ public void testFilterPushDownLessThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); @@ -343,7 +281,7 @@ public void testFilterPushDownLessThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanLiteralOnLeft() { String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 2"; @@ -358,7 +296,7 @@ public void testFilterPushDownLessThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqual() { String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 1"; @@ -373,7 +311,7 @@ public void testFilterPushDownLessThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); @@ -382,7 +320,7 @@ public void testFilterPushDownLessThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualLiteralOnLeft() { String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 3"; @@ -397,7 +335,7 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; @@ -413,7 +351,7 @@ public void testFilterPushDownIn() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownInNull() { String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); @@ -430,7 +368,7 @@ public void testFilterPushDownInNull() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); @@ -444,7 +382,7 @@ public void testFilterPushDownNotIn() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); @@ -455,7 +393,7 @@ public void testFilterPushDownNotInNull() { .isNull(); } - @Test + @TestTemplate public void testFilterPushDownIsNotNull() { String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); String expectedFilter = "not_null(ref(name=\"data\"))"; @@ -473,7 +411,7 @@ public void testFilterPushDownIsNotNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIsNull() { String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); String expectedFilter = "is_null(ref(name=\"data\"))"; @@ -488,7 +426,7 @@ public void testFilterPushDownIsNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); @@ -503,7 +441,7 @@ public void testFilterPushDownNot() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); @@ -522,7 +460,7 @@ public void testFilterPushDownBetween() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownNotBetween() { String sqlNotBetween = String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); @@ -538,7 +476,7 @@ public void testFilterPushDownNotBetween() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLike() { String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; @@ -565,7 +503,7 @@ public void testFilterPushDownLike() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; @@ -604,7 +542,7 @@ public void testFilterNotPushDownLike() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDown2Literal() { String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); List result = sql(sql2Literal); @@ -616,7 +554,7 @@ public void testFilterPushDown2Literal() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testSqlParseNaN() { // todo add some test case to test NaN } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java new file mode 100644 index 000000000000..32c81d9465a4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,104 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index bc7194e38088..8131bd7ab0d3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,16 +20,17 @@ 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.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestFlinkSourceConfig extends TestFlinkTableSource { +public class TestFlinkSourceConfig extends TableSourceTestBase { private static final String TABLE = "test_table"; - @Test + @TestTemplate public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) @@ -37,7 +38,7 @@ public void testFlinkSessionConfig() { .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } - @Test + @TestTemplate public void testFlinkHintConfig() { List result = sql( @@ -46,8 +47,11 @@ public void testFlinkHintConfig() { assertThat(result).hasSize(3); } - @Test + @TestTemplate public void testReadOptionHierarchy() { + // TODO: FLIP-27 source doesn't implement limit pushdown yet + assumeThat(useFlip27Source).isFalse(); + getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); assertThat(result).hasSize(1); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index 01bab6d063fd..18528c789114 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -21,78 +21,16 @@ 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.util.List; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestFlinkTableSource extends TestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - } - - @Test +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate public void testLimitPushDown() { assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) @@ -121,7 +59,7 @@ public void testLimitPushDown() { assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } - @Test + @TestTemplate public void testNoFilterPushDown() { String sql = String.format("SELECT * FROM %s ", TABLE_NAME); List result = sql(sql); @@ -133,7 +71,7 @@ public void testNoFilterPushDown() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDownEqual() { String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -147,7 +85,7 @@ public void testFilterPushDownEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); @@ -156,7 +94,7 @@ public void testFilterPushDownEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownEqualLiteralOnLeft() { String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -170,7 +108,7 @@ public void testFilterPushDownEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqual() { String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") != 1"; @@ -187,7 +125,7 @@ public void testFilterPushDownNoEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); @@ -196,7 +134,7 @@ public void testFilterPushDownNoEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownAnd() { String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); @@ -211,7 +149,7 @@ public void testFilterPushDownAnd() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownOr() { String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; @@ -229,7 +167,7 @@ public void testFilterPushDownOr() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThan() { String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 1"; @@ -247,7 +185,7 @@ public void testFilterPushDownGreaterThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); @@ -256,7 +194,7 @@ public void testFilterPushDownGreaterThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanLiteralOnLeft() { String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 3"; @@ -274,7 +212,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqual() { String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 2"; @@ -292,7 +230,7 @@ public void testFilterPushDownGreaterThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); @@ -301,7 +239,7 @@ public void testFilterPushDownGreaterThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 2"; @@ -319,7 +257,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThan() { String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 2"; @@ -334,7 +272,7 @@ public void testFilterPushDownLessThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); @@ -343,7 +281,7 @@ public void testFilterPushDownLessThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanLiteralOnLeft() { String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 2"; @@ -358,7 +296,7 @@ public void testFilterPushDownLessThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqual() { String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 1"; @@ -373,7 +311,7 @@ public void testFilterPushDownLessThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); @@ -382,7 +320,7 @@ public void testFilterPushDownLessThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualLiteralOnLeft() { String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 3"; @@ -397,7 +335,7 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; @@ -413,7 +351,7 @@ public void testFilterPushDownIn() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownInNull() { String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); @@ -430,7 +368,7 @@ public void testFilterPushDownInNull() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); @@ -444,7 +382,7 @@ public void testFilterPushDownNotIn() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); @@ -455,7 +393,7 @@ public void testFilterPushDownNotInNull() { .isNull(); } - @Test + @TestTemplate public void testFilterPushDownIsNotNull() { String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); String expectedFilter = "not_null(ref(name=\"data\"))"; @@ -473,7 +411,7 @@ public void testFilterPushDownIsNotNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIsNull() { String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); String expectedFilter = "is_null(ref(name=\"data\"))"; @@ -488,7 +426,7 @@ public void testFilterPushDownIsNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); @@ -503,7 +441,7 @@ public void testFilterPushDownNot() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); @@ -522,7 +460,7 @@ public void testFilterPushDownBetween() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownNotBetween() { String sqlNotBetween = String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); @@ -538,7 +476,7 @@ public void testFilterPushDownNotBetween() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLike() { String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; @@ -565,7 +503,7 @@ public void testFilterPushDownLike() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; @@ -604,7 +542,7 @@ public void testFilterNotPushDownLike() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDown2Literal() { String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); List result = sql(sql2Literal); @@ -616,7 +554,7 @@ public void testFilterPushDown2Literal() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testSqlParseNaN() { // todo add some test case to test NaN } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java new file mode 100644 index 000000000000..f89d63ac73e3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,105 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index bc7194e38088..8131bd7ab0d3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,16 +20,17 @@ 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.List; import org.apache.flink.types.Row; import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestTemplate; -public class TestFlinkSourceConfig extends TestFlinkTableSource { +public class TestFlinkSourceConfig extends TableSourceTestBase { private static final String TABLE = "test_table"; - @Test + @TestTemplate public void testFlinkSessionConfig() { getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) @@ -37,7 +38,7 @@ public void testFlinkSessionConfig() { .hasMessage("Cannot set as-of-timestamp option for streaming reader"); } - @Test + @TestTemplate public void testFlinkHintConfig() { List result = sql( @@ -46,8 +47,11 @@ public void testFlinkHintConfig() { assertThat(result).hasSize(3); } - @Test + @TestTemplate public void testReadOptionHierarchy() { + // TODO: FLIP-27 source doesn't implement limit pushdown yet + assumeThat(useFlip27Source).isFalse(); + getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); assertThat(result).hasSize(1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java index facbdcaaa533..18528c789114 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -21,78 +21,16 @@ 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.util.List; -import org.apache.flink.configuration.CoreOptions; import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.TestBase; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -public class TestFlinkTableSource extends TestBase { - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - - private int scanEventCount = 0; - private ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - dropDatabase(DATABASE_NAME, true); - dropCatalog(CATALOG_NAME, true); - } - - @Test +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate public void testLimitPushDown() { assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) @@ -121,7 +59,7 @@ public void testLimitPushDown() { assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); } - @Test + @TestTemplate public void testNoFilterPushDown() { String sql = String.format("SELECT * FROM %s ", TABLE_NAME); List result = sql(sql); @@ -133,7 +71,7 @@ public void testNoFilterPushDown() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDownEqual() { String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -147,7 +85,7 @@ public void testFilterPushDownEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownEqualNull() { String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); @@ -156,7 +94,7 @@ public void testFilterPushDownEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownEqualLiteralOnLeft() { String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") == 1"; @@ -170,7 +108,7 @@ public void testFilterPushDownEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqual() { String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") != 1"; @@ -187,7 +125,7 @@ public void testFilterPushDownNoEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNoEqualNull() { String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); @@ -196,7 +134,7 @@ public void testFilterPushDownNoEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownAnd() { String sqlAnd = String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); @@ -211,7 +149,7 @@ public void testFilterPushDownAnd() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownOr() { String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; @@ -229,7 +167,7 @@ public void testFilterPushDownOr() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThan() { String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 1"; @@ -247,7 +185,7 @@ public void testFilterPushDownGreaterThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanNull() { String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); @@ -256,7 +194,7 @@ public void testFilterPushDownGreaterThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanLiteralOnLeft() { String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 3"; @@ -274,7 +212,7 @@ public void testFilterPushDownGreaterThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqual() { String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 2"; @@ -292,7 +230,7 @@ public void testFilterPushDownGreaterThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualNull() { String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); @@ -301,7 +239,7 @@ public void testFilterPushDownGreaterThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 2"; @@ -319,7 +257,7 @@ public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThan() { String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") < 2"; @@ -334,7 +272,7 @@ public void testFilterPushDownLessThan() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanNull() { String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); @@ -343,7 +281,7 @@ public void testFilterPushDownLessThanNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanLiteralOnLeft() { String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") > 2"; @@ -358,7 +296,7 @@ public void testFilterPushDownLessThanLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqual() { String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") <= 1"; @@ -373,7 +311,7 @@ public void testFilterPushDownLessThanEqual() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualNull() { String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); @@ -382,7 +320,7 @@ public void testFilterPushDownLessThanEqualNull() { assertThat(lastScanEvent).as("Should not push down a filter").isNull(); } - @Test + @TestTemplate public void testFilterPushDownLessThanEqualLiteralOnLeft() { String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); String expectedFilter = "ref(name=\"id\") >= 3"; @@ -397,7 +335,7 @@ public void testFilterPushDownLessThanEqualLiteralOnLeft() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIn() { String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; @@ -413,7 +351,7 @@ public void testFilterPushDownIn() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownInNull() { String sqlInNull = String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); @@ -430,7 +368,7 @@ public void testFilterPushDownInNull() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotIn() { String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); @@ -444,7 +382,7 @@ public void testFilterPushDownNotIn() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterPushDownNotInNull() { String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); List resultGT = sql(sqlNotInNull); @@ -455,7 +393,7 @@ public void testFilterPushDownNotInNull() { .isNull(); } - @Test + @TestTemplate public void testFilterPushDownIsNotNull() { String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); String expectedFilter = "not_null(ref(name=\"data\"))"; @@ -473,7 +411,7 @@ public void testFilterPushDownIsNotNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownIsNull() { String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); String expectedFilter = "is_null(ref(name=\"data\"))"; @@ -488,7 +426,7 @@ public void testFilterPushDownIsNull() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownNot() { String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); @@ -503,7 +441,7 @@ public void testFilterPushDownNot() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownBetween() { String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); @@ -522,7 +460,7 @@ public void testFilterPushDownBetween() { .isEqualTo(expected); } - @Test + @TestTemplate public void testFilterPushDownNotBetween() { String sqlNotBetween = String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); @@ -538,7 +476,7 @@ public void testFilterPushDownNotBetween() { .isEqualTo(expectedFilter); } - @Test + @TestTemplate public void testFilterPushDownLike() { String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; @@ -565,7 +503,7 @@ public void testFilterPushDownLike() { .isEqualTo(expectedScan); } - @Test + @TestTemplate public void testFilterNotPushDownLike() { Row expectRecord = Row.of(1, "iceberg", 10.0); String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; @@ -604,7 +542,7 @@ public void testFilterNotPushDownLike() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testFilterPushDown2Literal() { String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); List result = sql(sql2Literal); @@ -616,7 +554,7 @@ public void testFilterPushDown2Literal() { .isEqualTo(Expressions.alwaysTrue()); } - @Test + @TestTemplate public void testSqlParseNaN() { // todo add some test case to test NaN } From c1b0d5268fd52f02ab0c2e240e085060c7393d18 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 22 Jul 2024 22:08:09 +0200 Subject: [PATCH 0500/1019] Core: Limit ParallelIterable memory consumption by yielding in tasks (#10691) ParallelIterable schedules 2 * WORKER_THREAD_POOL_SIZE tasks for processing input iterables. This defaults to 2 * # CPU cores. When one or some of the input iterables are considerable in size and the ParallelIterable consumer is not quick enough, this could result in unbounded allocation inside `ParallelIterator.queue`. This commit bounds the queue. When queue is full, the tasks yield and get removed from the executor. They are resumed when consumer catches up. --- .../apache/iceberg/util/ParallelIterable.java | 222 +++++++++++++----- .../iceberg/util/TestParallelIterable.java | 48 ++++ 2 files changed, 215 insertions(+), 55 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index d7221e7d4545..6486bd7fd483 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -20,84 +20,117 @@ import java.io.Closeable; import java.io.IOException; +import java.io.UncheckedIOException; +import java.util.ArrayDeque; +import java.util.Deque; import java.util.Iterator; import java.util.NoSuchElementException; +import java.util.Optional; +import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; -import java.util.concurrent.Future; -import org.apache.iceberg.exceptions.RuntimeIOException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.function.Supplier; 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.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.io.Closer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; public class ParallelIterable extends CloseableGroup implements CloseableIterable { + + private static final Logger LOG = LoggerFactory.getLogger(ParallelIterable.class); + + // Logic behind default value: ParallelIterable is often used for file planning. + // Assuming that a DataFile or DeleteFile is about 500 bytes, a 30k limit uses 14.3 MB of memory. + private static final int DEFAULT_MAX_QUEUE_SIZE = 30_000; + private final Iterable> iterables; private final ExecutorService workerPool; + // Bound for number of items in the queue to limit memory consumption + // even in the case when input iterables are large. + private final int approximateMaxQueueSize; + public ParallelIterable(Iterable> iterables, ExecutorService workerPool) { - this.iterables = iterables; - this.workerPool = workerPool; + this(iterables, workerPool, DEFAULT_MAX_QUEUE_SIZE); + } + + public ParallelIterable( + Iterable> iterables, + ExecutorService workerPool, + int approximateMaxQueueSize) { + this.iterables = Preconditions.checkNotNull(iterables, "Input iterables cannot be null"); + this.workerPool = Preconditions.checkNotNull(workerPool, "Worker pool cannot be null"); + this.approximateMaxQueueSize = approximateMaxQueueSize; } @Override public CloseableIterator iterator() { - ParallelIterator iter = new ParallelIterator<>(iterables, workerPool); + ParallelIterator iter = + new ParallelIterator<>(iterables, workerPool, approximateMaxQueueSize); addCloseable(iter); return iter; } private static class ParallelIterator implements CloseableIterator { - private final Iterator tasks; + private final Iterator> tasks; + private final Deque> yieldedTasks = new ArrayDeque<>(); private final ExecutorService workerPool; - private final Future[] taskFutures; + private final CompletableFuture>>[] taskFutures; private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - private volatile boolean closed = false; + private final int maxQueueSize; + private final AtomicBoolean closed = new AtomicBoolean(false); private ParallelIterator( - Iterable> iterables, ExecutorService workerPool) { + Iterable> iterables, ExecutorService workerPool, int maxQueueSize) { this.tasks = Iterables.transform( - iterables, - iterable -> - (Runnable) - () -> { - try (Closeable ignored = - (iterable instanceof Closeable) ? (Closeable) iterable : () -> {}) { - for (T item : iterable) { - // exit manually because `ConcurrentLinkedQueue` can't be - // interrupted - if (closed) { - return; - } - - queue.add(item); - } - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close iterable"); - } - }) + iterables, iterable -> new Task<>(iterable, queue, closed, maxQueueSize)) .iterator(); this.workerPool = workerPool; + this.maxQueueSize = maxQueueSize; // submit 2 tasks per worker at a time - this.taskFutures = new Future[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; + this.taskFutures = new CompletableFuture[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; } @Override public void close() { // close first, avoid new task submit - this.closed = true; + this.closed.set(true); - // cancel background tasks - for (Future taskFuture : taskFutures) { - if (taskFuture != null && !taskFuture.isDone()) { - taskFuture.cancel(true); + try (Closer closer = Closer.create()) { + synchronized (this) { + yieldedTasks.forEach(closer::register); + yieldedTasks.clear(); } + + // cancel background tasks and close continuations if any + for (CompletableFuture>> taskFuture : taskFutures) { + if (taskFuture != null) { + taskFuture.cancel(true); + taskFuture.thenAccept( + continuation -> { + if (continuation.isPresent()) { + try { + continuation.get().close(); + } catch (IOException e) { + LOG.error("Task close failed", e); + } + } + }); + } + } + + // clean queue + this.queue.clear(); + } catch (IOException e) { + throw new UncheckedIOException("Close failed", e); } - // clean queue - this.queue.clear(); } /** @@ -107,15 +140,17 @@ public void close() { * * @return true if there are pending tasks, false otherwise */ - private boolean checkTasks() { + private synchronized boolean checkTasks() { + Preconditions.checkState(!closed.get(), "Already closed"); boolean hasRunningTask = false; for (int i = 0; i < taskFutures.length; i += 1) { if (taskFutures[i] == null || taskFutures[i].isDone()) { if (taskFutures[i] != null) { - // check for task failure and re-throw any exception + // check for task failure and re-throw any exception. Enqueue continuation if any. try { - taskFutures[i].get(); + Optional> continuation = taskFutures[i].get(); + continuation.ifPresent(yieldedTasks::addLast); } catch (ExecutionException e) { if (e.getCause() instanceof RuntimeException) { // rethrow a runtime exception @@ -136,30 +171,33 @@ private boolean checkTasks() { } } - return !closed && (tasks.hasNext() || hasRunningTask); + return !closed.get() && (tasks.hasNext() || hasRunningTask); } - private Future submitNextTask() { - if (!closed && tasks.hasNext()) { - return workerPool.submit(tasks.next()); + private CompletableFuture>> submitNextTask() { + if (!closed.get()) { + if (!yieldedTasks.isEmpty()) { + return CompletableFuture.supplyAsync(yieldedTasks.removeFirst(), workerPool); + } else if (tasks.hasNext()) { + return CompletableFuture.supplyAsync(tasks.next(), workerPool); + } } return null; } @Override public synchronized boolean hasNext() { - Preconditions.checkState(!closed, "Already closed"); - - // if the consumer is processing records more slowly than the producers, then this check will - // prevent tasks from being submitted. while the producers are running, this will always - // return here before running checkTasks. when enough of the tasks are finished that the - // consumer catches up, then lots of new tasks will be submitted at once. this behavior is - // okay because it ensures that records are not stacking up waiting to be consumed and taking - // up memory. - // - // consumers that process results quickly will periodically exhaust the queue and submit new - // tasks when checkTasks runs. fast consumers should not be delayed. - if (!queue.isEmpty()) { + Preconditions.checkState(!closed.get(), "Already closed"); + + // If the consumer is processing records more slowly than the producers, the producers will + // eventually fill the queue and yield, returning continuations. Continuations and new tasks + // are started by checkTasks(). The check here prevents us from restarting continuations or + // starting new tasks too early (when queue is almost full) or too late (when queue is already + // emptied). Restarting too early would lead to tasks yielding very quickly (CPU waste on + // scheduling). Restarting too late would mean the consumer may need to wait for the tasks + // to produce new items. A consumer slower than producers shouldn't need to wait. + int queueLowWaterMark = maxQueueSize / 2; + if (queue.size() > queueLowWaterMark) { return true; } @@ -192,4 +230,78 @@ public synchronized T next() { return queue.poll(); } } + + private static class Task implements Supplier>>, Closeable { + private final Iterable input; + private final ConcurrentLinkedQueue queue; + private final AtomicBoolean closed; + private final int approximateMaxQueueSize; + + private Iterator iterator = null; + + Task( + Iterable input, + ConcurrentLinkedQueue queue, + AtomicBoolean closed, + int approximateMaxQueueSize) { + this.input = Preconditions.checkNotNull(input, "input cannot be null"); + this.queue = Preconditions.checkNotNull(queue, "queue cannot be null"); + this.closed = Preconditions.checkNotNull(closed, "closed cannot be null"); + this.approximateMaxQueueSize = approximateMaxQueueSize; + } + + @Override + public Optional> get() { + try { + if (iterator == null) { + iterator = input.iterator(); + } + + while (iterator.hasNext()) { + if (queue.size() >= approximateMaxQueueSize) { + // Yield when queue is over the size limit. Task will be resubmitted later and continue + // the work. + return Optional.of(this); + } + + T next = iterator.next(); + if (closed.get()) { + break; + } + + queue.add(next); + } + } catch (Throwable e) { + try { + close(); + } catch (IOException closeException) { + // self-suppression is not permitted + // (e and closeException to be the same is unlikely, but possible) + if (closeException != e) { + e.addSuppressed(closeException); + } + } + + throw e; + } + + try { + close(); + } catch (IOException e) { + throw new UncheckedIOException("Close failed", e); + } + + // The task is complete. Returning empty means there is no continuation that should be + // executed. + return Optional.empty(); + } + + @Override + public void close() throws IOException { + iterator = null; + if (input instanceof Closeable) { + ((Closeable) input).close(); + } + } + } } 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 af9c6ec5212c..4910732f6e35 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -24,15 +24,22 @@ import java.lang.reflect.Field; import java.util.Collections; import java.util.Iterator; +import java.util.List; import java.util.Queue; import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.HashMultiset; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMultiset; 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.Multiset; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; @@ -133,6 +140,47 @@ public CloseableIterator iterator() { .untilAsserted(() -> assertThat(queue).as("Queue is not empty after cleaning").isEmpty()); } + @Test + public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchFieldException { + + 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); + CloseableIterator iterator = parallelIterable.iterator(); + Field queueField = iterator.getClass().getDeclaredField("queue"); + queueField.setAccessible(true); + ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + + Multiset actualValues = HashMultiset.create(); + + while (iterator.hasNext()) { + assertThat(queue) + .as("iterator internal queue") + .hasSizeLessThanOrEqualTo(maxQueueSize + iterables.size()); + actualValues.add(iterator.next()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + executor.shutdownNow(); + } + private void queueHasElements(CloseableIterator iterator, Queue queue) { assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); From 01338b8114d0a24ef683c635e401229fd74d4159 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 22 Jul 2024 13:59:18 -0700 Subject: [PATCH 0501/1019] Flink: handle rescale properly and refactor statistics (#10457) --- .../shuffle/AggregatedStatisticsTracker.java | 27 +- ...atistics.java => CompletedStatistics.java} | 63 ++-- ...ava => CompletedStatisticsSerializer.java} | 73 ++-- .../shuffle/DataStatisticsCoordinator.java | 143 ++++++-- .../DataStatisticsCoordinatorProvider.java | 13 +- .../sink/shuffle/DataStatisticsOperator.java | 57 +++- .../flink/sink/shuffle/GlobalStatistics.java | 114 +++++++ .../shuffle/GlobalStatisticsSerializer.java | 199 +++++++++++ .../flink/sink/shuffle/KeyAssignment.java | 151 +++++++++ .../flink/sink/shuffle/MapAssignment.java | 240 ++++++++++++++ .../sink/shuffle/MapRangePartitioner.java | 312 +----------------- .../flink/sink/shuffle/RangePartitioner.java | 110 ++++++ .../shuffle/RequestGlobalStatisticsEvent.java | 38 +++ .../sink/shuffle/SketchRangePartitioner.java | 64 ++++ .../flink/sink/shuffle/SketchUtil.java | 40 +-- .../flink/sink/shuffle/StatisticsEvent.java | 27 +- .../sink/shuffle/StatisticsOrRecord.java | 12 +- .../shuffle/StatisticsOrRecordSerializer.java | 15 +- .../flink/sink/shuffle/StatisticsUtil.java | 47 ++- .../iceberg/flink/sink/shuffle/Fixtures.java | 6 +- .../TestAggregatedStatisticsTracker.java | 54 ++- ...=> TestCompletedStatisticsSerializer.java} | 20 +- .../TestDataStatisticsCoordinator.java | 113 ++++++- ...TestDataStatisticsCoordinatorProvider.java | 122 ++++--- .../shuffle/TestDataStatisticsOperator.java | 87 +++-- .../TestGlobalStatisticsSerializer.java | 59 ++++ .../sink/shuffle/TestMapRangePartitioner.java | 196 +++++------ .../flink/sink/shuffle/TestSketchUtil.java | 8 +- 28 files changed, 1695 insertions(+), 715 deletions(-) rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{AggregatedStatistics.java => CompletedStatistics.java} (56%) rename flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{AggregatedStatisticsSerializer.java => CompletedStatisticsSerializer.java} (62%) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java rename flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/{TestAggregatedStatisticsSerializer.java => TestCompletedStatisticsSerializer.java} (70%) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 52d8a2f16f99..338523b7b074 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Comparator; import java.util.Map; import java.util.NavigableMap; import java.util.Set; @@ -29,8 +28,6 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -38,9 +35,8 @@ import org.slf4j.LoggerFactory; /** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. */ class AggregatedStatisticsTracker { private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); @@ -51,10 +47,9 @@ class AggregatedStatisticsTracker { private final int downstreamParallelism; private final StatisticsType statisticsType; private final int switchToSketchThreshold; - private final Comparator comparator; private final NavigableMap aggregationsPerCheckpoint; - private AggregatedStatistics completedStatistics; + private CompletedStatistics completedStatistics; AggregatedStatisticsTracker( String operatorName, @@ -64,7 +59,7 @@ class AggregatedStatisticsTracker { int downstreamParallelism, StatisticsType statisticsType, int switchToSketchThreshold, - @Nullable AggregatedStatistics restoredStatistics) { + @Nullable CompletedStatistics restoredStatistics) { this.operatorName = operatorName; this.parallelism = parallelism; this.statisticsSerializer = @@ -74,11 +69,10 @@ class AggregatedStatisticsTracker { this.switchToSketchThreshold = switchToSketchThreshold; this.completedStatistics = restoredStatistics; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); this.aggregationsPerCheckpoint = Maps.newTreeMap(); } - AggregatedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { long checkpointId = event.checkpointId(); LOG.debug( "Handling statistics event from subtask {} of operator {} for checkpoint {}", @@ -105,7 +99,6 @@ AggregatedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event parallelism, downstreamParallelism, switchToSketchThreshold, - comparator, statisticsType, StatisticsUtil.collectType(statisticsType, completedStatistics))); DataStatistics dataStatistics = @@ -140,7 +133,6 @@ static class Aggregation { private final int parallelism; private final int downstreamParallelism; private final int switchToSketchThreshold; - private final Comparator comparator; private final StatisticsType configuredType; private StatisticsType currentType; private Map mapStatistics; @@ -150,14 +142,12 @@ static class Aggregation { int parallelism, int downstreamParallelism, int switchToSketchThreshold, - Comparator comparator, StatisticsType configuredType, StatisticsType currentType) { this.subtaskSet = Sets.newHashSet(); this.parallelism = parallelism; this.downstreamParallelism = downstreamParallelism; this.switchToSketchThreshold = switchToSketchThreshold; - this.comparator = comparator; this.configuredType = configuredType; this.currentType = currentType; @@ -246,10 +236,10 @@ private void convertCoordinatorToSketch() { this.mapStatistics = null; } - private AggregatedStatistics completedStatistics(long checkpointId) { + private CompletedStatistics completedStatistics(long checkpointId) { if (currentType == StatisticsType.Map) { LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); - return AggregatedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); LOG.info( @@ -258,8 +248,7 @@ private AggregatedStatistics completedStatistics(long checkpointId) { sketch.getK(), sketch.getN(), sketch.getNumSamples()); - return AggregatedStatistics.fromRangeBounds( - checkpointId, SketchUtil.rangeBounds(downstreamParallelism, comparator, sketch)); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java similarity index 56% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index 8a91411c1d21..c0e228965ddd 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -18,46 +18,39 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.io.Serializable; import java.util.Arrays; import java.util.Map; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Objects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; /** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). */ -class AggregatedStatistics implements Serializable { +class CompletedStatistics { private final long checkpointId; private final StatisticsType type; private final Map keyFrequency; - private final SortKey[] rangeBounds; + private final SortKey[] keySamples; - AggregatedStatistics( + static CompletedStatistics fromKeyFrequency(long checkpointId, Map stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( long checkpointId, StatisticsType type, Map keyFrequency, - SortKey[] rangeBounds) { - Preconditions.checkArgument( - (keyFrequency != null && rangeBounds == null) - || (keyFrequency == null && rangeBounds != null), - "Invalid key frequency or range bounds: both are non-null or null"); + SortKey[] keySamples) { this.checkpointId = checkpointId; this.type = type; this.keyFrequency = keyFrequency; - this.rangeBounds = rangeBounds; - } - - static AggregatedStatistics fromKeyFrequency(long checkpointId, Map stats) { - return new AggregatedStatistics(checkpointId, StatisticsType.Map, stats, null); - } - - static AggregatedStatistics fromRangeBounds(long checkpointId, SortKey[] stats) { - return new AggregatedStatistics(checkpointId, StatisticsType.Sketch, null, stats); + this.keySamples = keySamples; } @Override @@ -66,7 +59,7 @@ public String toString() { .add("checkpointId", checkpointId) .add("type", type) .add("keyFrequency", keyFrequency) - .add("rangeBounds", rangeBounds) + .add("keySamples", keySamples) .toString(); } @@ -76,20 +69,24 @@ public boolean equals(Object o) { return true; } - if (!(o instanceof AggregatedStatistics)) { + if (!(o instanceof CompletedStatistics)) { return false; } - AggregatedStatistics other = (AggregatedStatistics) o; - return Objects.equal(checkpointId, other.checkpointId()) - && Objects.equal(type, other.type()) + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) && Objects.equal(keyFrequency, other.keyFrequency()) - && Arrays.equals(rangeBounds, other.rangeBounds()); + && Arrays.equals(keySamples, other.keySamples()); } @Override public int hashCode() { - return Objects.hashCode(checkpointId, type, keyFrequency, rangeBounds); + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; } StatisticsType type() { @@ -100,11 +97,7 @@ Map keyFrequency() { return keyFrequency; } - SortKey[] rangeBounds() { - return rangeBounds; - } - - long checkpointId() { - return checkpointId; + SortKey[] keySamples() { + return keySamples; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java similarity index 62% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java rename to flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java index 56ba5e04f05a..1ac0e386a011 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -35,17 +35,17 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.iceberg.SortKey; -public class AggregatedStatisticsSerializer extends TypeSerializer { +class CompletedStatisticsSerializer extends TypeSerializer { private final TypeSerializer sortKeySerializer; private final EnumSerializer statisticsTypeSerializer; private final MapSerializer keyFrequencySerializer; - private final ListSerializer rangeBoundsSerializer; + private final ListSerializer keySamplesSerializer; - AggregatedStatisticsSerializer(TypeSerializer sortKeySerializer) { + CompletedStatisticsSerializer(TypeSerializer sortKeySerializer) { this.sortKeySerializer = sortKeySerializer; this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); - this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); } @Override @@ -54,23 +54,23 @@ public boolean isImmutableType() { } @Override - public TypeSerializer duplicate() { - return new AggregatedStatisticsSerializer(sortKeySerializer); + public TypeSerializer duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); } @Override - public AggregatedStatistics createInstance() { - return new AggregatedStatistics(0, StatisticsType.Map, Collections.emptyMap(), null); + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); } @Override - public AggregatedStatistics copy(AggregatedStatistics from) { - return new AggregatedStatistics( - from.checkpointId(), from.type(), from.keyFrequency(), from.rangeBounds()); + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); } @Override - public AggregatedStatistics copy(AggregatedStatistics from, AggregatedStatistics reuse) { + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { // no benefit of reuse return copy(from); } @@ -81,35 +81,33 @@ public int getLength() { } @Override - public void serialize(AggregatedStatistics record, DataOutputView target) throws IOException { + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { target.writeLong(record.checkpointId()); statisticsTypeSerializer.serialize(record.type(), target); if (record.type() == StatisticsType.Map) { keyFrequencySerializer.serialize(record.keyFrequency(), target); } else { - rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); } } @Override - public AggregatedStatistics deserialize(DataInputView source) throws IOException { + public CompletedStatistics deserialize(DataInputView source) throws IOException { long checkpointId = source.readLong(); StatisticsType type = statisticsTypeSerializer.deserialize(source); - Map keyFrequency = null; - SortKey[] rangeBounds = null; if (type == StatisticsType.Map) { - keyFrequency = keyFrequencySerializer.deserialize(source); + Map keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); } else { - List sortKeys = rangeBoundsSerializer.deserialize(source); - rangeBounds = new SortKey[sortKeys.size()]; - rangeBounds = sortKeys.toArray(rangeBounds); + List sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); } - - return new AggregatedStatistics(checkpointId, type, keyFrequency, rangeBounds); } @Override - public AggregatedStatistics deserialize(AggregatedStatistics reuse, DataInputView source) + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) throws IOException { // not much benefit to reuse return deserialize(source); @@ -122,11 +120,15 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof AggregatedStatisticsSerializer)) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { return false; } - AggregatedStatisticsSerializer other = (AggregatedStatisticsSerializer) obj; + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; return Objects.equals(sortKeySerializer, other.sortKeySerializer); } @@ -136,21 +138,20 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new AggregatedStatisticsSerializerSnapshot(this); + public TypeSerializerSnapshot snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); } - public static class AggregatedStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - AggregatedStatistics, AggregatedStatisticsSerializer> { + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { private static final int CURRENT_VERSION = 1; /** Constructor for read instantiation. */ @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public AggregatedStatisticsSerializerSnapshot() {} + public CompletedStatisticsSerializerSnapshot() {} @SuppressWarnings("checkstyle:RedundantModifier") - public AggregatedStatisticsSerializerSnapshot(AggregatedStatisticsSerializer serializer) { + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { super(serializer); } @@ -161,15 +162,15 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer[] getNestedSerializers( - AggregatedStatisticsSerializer outerSerializer) { + CompletedStatisticsSerializer outerSerializer) { return new TypeSerializer[] {outerSerializer.sortKeySerializer}; } @Override - protected AggregatedStatisticsSerializer createOuterSerializerWithNestedSerializers( + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; - return new AggregatedStatisticsSerializer(sortKeySerializer); + return new CompletedStatisticsSerializer(sortKeySerializer); } } } 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 6e9e4f6fa6c2..3b21fbae315a 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -37,6 +38,8 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -60,17 +63,21 @@ class DataStatisticsCoordinator implements OperatorCoordinator { private final OperatorCoordinator.Context context; private final Schema schema; private final SortOrder sortOrder; + private final Comparator comparator; private final int downstreamParallelism; private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; private final ExecutorService coordinatorExecutor; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer aggregatedStatisticsSerializer; + private final TypeSerializer completedStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; private transient boolean started; private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private transient AggregatedStatistics completedStatistics; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; DataStatisticsCoordinator( String operatorName, @@ -78,13 +85,16 @@ class DataStatisticsCoordinator implements OperatorCoordinator { Schema schema, SortOrder sortOrder, int downstreamParallelism, - StatisticsType statisticsType) { + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { this.operatorName = operatorName; this.context = context; this.schema = schema; this.sortOrder = sortOrder; + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( @@ -92,13 +102,16 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); - this.aggregatedStatisticsSerializer = new AggregatedStatisticsSerializer(sortKeySerializer); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called this.aggregatedStatisticsTracker = new AggregatedStatisticsTracker( operatorName, @@ -185,32 +198,98 @@ private void ensureStarted() { } private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = + CompletedStatistics maybeCompletedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendAggregatedStatisticsToSubtasks(completedStatistics.checkpointId(), completedStatistics); + if (maybeCompletedStatistics != null) { + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendAggregatedStatisticsToSubtasks( - long checkpointId, AggregatedStatistics globalStatistics) { - callInCoordinatorThread( + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( () -> { + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary StatisticsEvent statisticsEvent = - StatisticsEvent.createAggregatedStatisticsEvent( - checkpointId, globalStatistics, aggregatedStatisticsSerializer); + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } - - return null; }, String.format( "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } } @Override @@ -223,8 +302,14 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof StatisticsEvent); - handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -240,8 +325,8 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r operatorName, checkpointId); resultFuture.complete( - StatisticsUtil.serializeAggregatedStatistics( - completedStatistics, aggregatedStatisticsSerializer)); + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); }, String.format("taking checkpoint %d", checkpointId)); } @@ -253,7 +338,6 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", @@ -265,8 +349,12 @@ public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); this.completedStatistics = - StatisticsUtil.deserializeAggregatedStatistics( - checkpointData, aggregatedStatisticsSerializer); + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); } @Override @@ -290,7 +378,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", subtask, attemptNumber, operatorName); @@ -316,10 +404,15 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics completedStatistics() { + CompletedStatistics completedStatistics() { return completedStatistics; } + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + private static class SubtaskGateways { private final String operatorName; private final Map[] gateways; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java index ffb428283785..9d7d989c298e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -37,6 +37,7 @@ public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCo private final SortOrder sortOrder; private final int downstreamParallelism; private final StatisticsType type; + private final double closeFileCostWeightPercentage; public DataStatisticsCoordinatorProvider( String operatorName, @@ -44,18 +45,26 @@ public DataStatisticsCoordinatorProvider( Schema schema, SortOrder sortOrder, int downstreamParallelism, - StatisticsType type) { + StatisticsType type, + double closeFileCostWeightPercentage) { super(operatorID); this.operatorName = operatorName; this.schema = schema; this.sortOrder = sortOrder; this.downstreamParallelism = downstreamParallelism; this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { return new DataStatisticsCoordinator( - operatorName, context, schema, sortOrder, downstreamParallelism, type); + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 2910471762cd..7995a8a5b181 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -60,16 +60,16 @@ public class DataStatisticsOperator extends AbstractStreamOperator taskStatisticsSerializer; - private final TypeSerializer aggregatedStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; private transient int parallelism; private transient int subtaskIndex; - private transient ListState globalStatisticsState; + private transient ListState globalStatisticsState; // current statistics type may be different from the config due to possible // migration from Map statistics to Sketch statistics when high cardinality detected private transient volatile StatisticsType taskStatisticsType; private transient volatile DataStatistics localStatistics; - private transient volatile AggregatedStatistics globalStatistics; + private transient volatile GlobalStatistics globalStatistics; DataStatisticsOperator( String operatorName, @@ -87,31 +87,57 @@ public class DataStatisticsOperator extends AbstractStreamOperator("globalStatisticsState", aggregatedStatisticsSerializer)); + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); if (context.isRestored()) { if (globalStatisticsState.get() == null || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( + LOG.info( "Operator {} subtask {} doesn't have global statistics state to restore", operatorName, subtaskIndex); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); LOG.info( - "Operator {} subtask {} restoring global statistics state", operatorName, subtaskIndex); - this.globalStatistics = globalStatisticsState.get().iterator().next(); + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; } + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); } this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); @@ -139,14 +165,16 @@ public void handleOperatorEvent(OperatorEvent event) { operatorName, subtaskIndex, statisticsEvent.checkpointId()); - globalStatistics = - StatisticsUtil.deserializeAggregatedStatistics( - statisticsEvent.statisticsBytes(), aggregatedStatisticsSerializer); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); checkStatisticsTypeMigration(); - output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } } - @SuppressWarnings("unchecked") @Override public void processElement(StreamRecord streamRecord) { // collect data statistics @@ -204,6 +232,7 @@ && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); } + @SuppressWarnings("unchecked") private void checkStatisticsTypeMigration() { // only check if the statisticsType config is Auto and localStatistics is currently Map type if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { @@ -231,7 +260,7 @@ DataStatistics localStatistics() { } @VisibleForTesting - AggregatedStatistics globalStatistics() { + GlobalStatistics globalStatistics() { return globalStatistics; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * 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.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..a7fe2b30b865 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,199 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final ListSerializer rangeBoundsSerializer; + private final ListSerializer intsSerializer; + private final ListSerializer longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List assignedSubtasks = intsSerializer.deserialize(source); + List subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..a164d83ac3b0 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,151 @@ +/* + * 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.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List assignedSubtasks; + private final List subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List assignedSubtasks() { + return assignedSubtasks; + } + + List subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..0abb030c2279 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.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.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map keyAssignments; + + MapAssignment(int numPartitions, Map keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map keyAssignments() { + return keyAssignments; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map assignment( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index 298426cee872..f36a078c94e0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -18,29 +18,14 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; -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; -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.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,47 +46,28 @@ class MapRangePartitioner implements Partitioner { private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; - private final Comparator comparator; - private final Map mapStatistics; - private final double closeFileCostInWeightPercentage; + private final MapAssignment mapAssignment; // Counter that tracks how many times a new key encountered // where there is no traffic statistics learned about it. private long newSortKeyCounter; private long lastNewSortKeyLogTimeMilli; - // lazily computed due to the need of numPartitions - private Map assignment; - private NavigableMap sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - Map mapStatistics, - double closeFileCostInWeightPercentage) { - mapStatistics.forEach( - (key, value) -> - Preconditions.checkArgument( - value > 0, "Invalid statistics: weight is 0 for key %s", key)); - + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = mapStatistics; - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.mapAssignment = mapAssignment; this.newSortKeyCounter = 0; this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); } @Override public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map assignmentMap = assignment(numPartitions); // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; if (keyAssignment == null) { LOG.trace( "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", @@ -118,268 +84,12 @@ public int partition(RowData row, int numPartitions) { lastNewSortKeyLogTimeMilli = now; newSortKeyCounter = 0; } - return (int) (newSortKeyCounter % numPartitions); + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); } - return keyAssignment.select(); - } - - @VisibleForTesting - Map assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map mapStatistics() { - return mapStatistics; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map> assignmentInfo() { - Map> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map buildAssignment( - int numPartitions, - NavigableMap sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List assignedSubtasks = Lists.newArrayList(); - List subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List assignedSubtasks, - List subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java new file mode 100644 index 000000000000..83a9461233d2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.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.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** The wrapper class */ +@Internal +public class RangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. + // when rescale is detected, operator requests new statistics from coordinator upon + // initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.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.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java new file mode 100644 index 000000000000..af78271ea5dc --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.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.sink.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +class SketchRangePartitioner implements Partitioner { + private final SortKey sortKey; + private final Comparator comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index bfd3082a5aa7..a58310611e8d 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -22,7 +22,6 @@ import java.util.Comparator; import java.util.Map; import java.util.function.Consumer; -import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.iceberg.SortKey; import org.apache.iceberg.StructLike; @@ -90,37 +89,32 @@ static int determineOperatorReservoirSize(int operatorParallelism, int numPartit /** * To understand how range bounds are used in range partitioning, here is an example for human * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be - *
  • age <= 15 - *
  • age > 15 && age <= 32 - *
  • age >32 && age <= 60 - *
  • age > 60 + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + * + *

    Assumption is that a single key is not dominant enough to span multiple subtasks. * * @param numPartitions number of partitions which maps to downstream operator parallelism - * @param sketch aggregated reservoir sampling sketch - * @return list of range partition bounds. It should be a sorted list (ascending). Number of items - * should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list */ static SortKey[] rangeBounds( - int numPartitions, Comparator comparator, ReservoirItemsSketch sketch) { - SortKey[] sortKeys = sketch.getSamples(); - return determineBounds(Math.min(numPartitions, sortKeys.length), comparator, sortKeys); - } - - /** - * This assumes the sort keys have equal weight, which is usually the case for high-cardinality - * scenarios (like device_id, user_id, uuid etc.). - */ - static SortKey[] determineBounds( - int numPartitions, Comparator comparator, SortKey[] sortKeys) { + int numPartitions, Comparator comparator, SortKey[] samples) { // sort the keys first - Arrays.sort(sortKeys, comparator); + Arrays.sort(samples, comparator); int numCandidates = numPartitions - 1; SortKey[] candidates = new SortKey[numCandidates]; - int step = (int) Math.ceil((double) sortKeys.length / numPartitions); + int step = (int) Math.ceil((double) samples.length / numPartitions); int position = step - 1; int numChosen = 0; - while (position < sortKeys.length && numChosen < numCandidates) { - SortKey candidate = sortKeys[position]; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; // skip duplicate values if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { // linear probe for the next distinct value diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index f3391d72297f..f6fcdb8b16ef 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -32,27 +32,34 @@ class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; + private final boolean applyImmediately; - private StatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; } static StatisticsEvent createTaskStatisticsEvent( long checkpointId, DataStatistics statistics, TypeSerializer statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. return new StatisticsEvent( - checkpointId, StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer)); + checkpointId, + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); } - static StatisticsEvent createAggregatedStatisticsEvent( - long checkpointId, - AggregatedStatistics statistics, - TypeSerializer statisticsSerializer) { + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer statisticsSerializer, + boolean applyImmediately) { return new StatisticsEvent( - checkpointId, - StatisticsUtil.serializeAggregatedStatistics(statistics, statisticsSerializer)); + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); } long checkpointId() { @@ -62,4 +69,8 @@ long checkpointId() { byte[] statisticsBytes() { return statisticsBytes; } + + boolean applyImmediately() { + return applyImmediately; + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index c251ba1360fc..bc28df2b0e22 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -40,10 +40,10 @@ public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private AggregatedStatistics statistics; + private GlobalStatistics statistics; private RowData record; - private StatisticsOrRecord(AggregatedStatistics statistics, RowData record) { + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; @@ -54,7 +54,7 @@ static StatisticsOrRecord fromRecord(RowData record) { return new StatisticsOrRecord(null, record); } - static StatisticsOrRecord fromStatistics(AggregatedStatistics statistics) { + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { return new StatisticsOrRecord(statistics, null); } @@ -69,7 +69,7 @@ static StatisticsOrRecord reuseRecord( } static StatisticsOrRecord reuseStatistics( - StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { + StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { if (reuse.hasStatistics()) { return reuse; } else { @@ -86,11 +86,11 @@ public boolean hasRecord() { return record != null; } - AggregatedStatistics statistics() { + GlobalStatistics statistics() { return statistics; } - void statistics(AggregatedStatistics newStatistics) { + void statistics(GlobalStatistics newStatistics) { this.statistics = newStatistics; } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index 7e690f61a58b..d4ae2b359679 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -30,11 +30,11 @@ @Internal class StatisticsOrRecordSerializer extends TypeSerializer { - private final TypeSerializer statisticsSerializer; + private final TypeSerializer statisticsSerializer; private final TypeSerializer recordSerializer; StatisticsOrRecordSerializer( - TypeSerializer statisticsSerializer, + TypeSerializer statisticsSerializer, TypeSerializer recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,7 +48,7 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override public TypeSerializer duplicate() { - TypeSerializer duplicateStatisticsSerializer = + TypeSerializer duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) @@ -84,8 +84,7 @@ public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse to.record(record); } else { to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - AggregatedStatistics statistics = - statisticsSerializer.copy(from.statistics(), to.statistics()); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); to.statistics(statistics); } @@ -130,7 +129,7 @@ public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView so to.record(record); } else { to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - AggregatedStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); to.statistics(statistics); } @@ -200,8 +199,8 @@ protected TypeSerializer[] getNestedSerializers( @Override protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { - TypeSerializer statisticsSerializer = - (TypeSerializer) nestedSerializers[0]; + TypeSerializer statisticsSerializer = + (TypeSerializer) nestedSerializers[0]; TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } 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 9d3d128535fe..5d48ec57ca49 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 @@ -60,20 +60,41 @@ static DataStatistics deserializeDataStatistics( } } - static byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer statisticsSerializer) { + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer statisticsSerializer) { try { DataOutputSerializer out = new DataOutputSerializer(1024); - statisticsSerializer.serialize(aggregatedStatistics, out); + statisticsSerializer.serialize(completedStatistics, out); return out.getCopyOfBuffer(); } catch (IOException e) { throw new UncheckedIOException("Fail to serialize aggregated statistics", e); } } - static AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { try { DataInputDeserializer input = new DataInputDeserializer(bytes); return statisticsSerializer.deserialize(input); @@ -86,10 +107,18 @@ static StatisticsType collectType(StatisticsType config) { return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; } + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + static StatisticsType collectType( - StatisticsType config, @Nullable AggregatedStatistics restoredStatistics) { - if (restoredStatistics != null) { - return restoredStatistics.type(); + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); } return collectType(config); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java index c7e9f19abb02..5910bd685510 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -53,8 +53,10 @@ private Fixtures() {} new SortKeySerializer(SCHEMA, SORT_ORDER); public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = new DataStatisticsSerializer(SORT_KEY_SERIALIZER); - public static final AggregatedStatisticsSerializer AGGREGATED_STATISTICS_SERIALIZER = - new AggregatedStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); public static final Map CHAR_KEYS = createCharKeys(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 13e2e0ce2115..8322ce683768 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -36,7 +36,7 @@ public void receiveNewerStatisticsEvent(StatisticsType type) { StatisticsEvent checkpoint1Subtask0StatisticsEvent = createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -86,7 +86,8 @@ public void receiveNewerStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); } // checkpoint 2 remains @@ -115,7 +116,7 @@ public void receiveOlderStatisticsEventTest(StatisticsType type) { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); @@ -189,7 +190,14 @@ public void receiveOlderStatisticsEventTest(StatisticsType type) { CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 4L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); } } @@ -207,7 +215,7 @@ public void receiveCompletedStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -246,7 +254,14 @@ public void receiveCompletedStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("a"), 3L, CHAR_KEYS.get("b"), 3L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); } StatisticsEvent checkpoint2Subtask0DataStatisticEvent = @@ -280,7 +295,8 @@ public void receiveCompletedStatisticsEvent(StatisticsType type) { CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); } else { - assertThat(completedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); } } @@ -307,7 +323,7 @@ public void coordinatorSwitchToSketchOverThreshold() { 1L, CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -349,8 +365,14 @@ public void coordinatorSwitchToSketchOverThreshold() { assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); - assertThat(completedStatistics.rangeBounds()) - .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); } @Test @@ -376,7 +398,7 @@ public void coordinatorMapOperatorSketch() { 1L, CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - AggregatedStatistics completedStatistics = + CompletedStatistics completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); assertThat(completedStatistics).isNull(); assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); @@ -419,8 +441,14 @@ public void coordinatorMapOperatorSketch() { assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); - assertThat(completedStatistics.rangeBounds()) - .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); } private AggregatedStatisticsTracker createTracker(StatisticsType type) { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java similarity index 70% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java rename to flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java index 0ce73fa4aaad..4ee9888934a8 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsSerializer.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -25,11 +25,11 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -public class TestAggregatedStatisticsSerializer extends SerializerTestBase { +public class TestCompletedStatisticsSerializer extends SerializerTestBase { @Override - protected TypeSerializer createSerializer() { - return Fixtures.AGGREGATED_STATISTICS_SERIALIZER; + protected TypeSerializer createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; } @Override @@ -38,17 +38,17 @@ protected int getLength() { } @Override - protected Class getTypeClass() { - return AggregatedStatistics.class; + protected Class getTypeClass() { + return CompletedStatistics.class; } @Override - protected AggregatedStatistics[] getTestData() { - return new AggregatedStatistics[] { - AggregatedStatistics.fromKeyFrequency( + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), - AggregatedStatistics.fromRangeBounds( - 2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) }; } } 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 fe1d07c3286a..a08a8a73e80c 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 @@ -19,17 +19,24 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.time.Duration; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; @@ -45,7 +52,7 @@ public void before() throws Exception { } private void tasksReady(DataStatisticsCoordinator coordinator) { - setAllTasksReady(Fixtures.NUM_SUBTASKS, coordinator, receivingTasks); + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); } @ParameterizedTest @@ -106,20 +113,95 @@ public void testDataStatisticsEventHandling(StatisticsType type) throws Exceptio waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - // Verify global data statistics is the aggregation of all subtasks data statistics - AggregatedStatistics aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.checkpointId()).isEqualTo(1L); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + Map keyFrequency = + ImmutableMap.of( + CHAR_KEYS.get("a"), 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(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), 2L, - CHAR_KEYS.get("b"), 3L, - CHAR_KEYS.get("c"), 5L)); + assertThat(completedStatistics.keyFrequency()).isEqualTo(keyFrequency); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("b")); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + 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 = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); } } @@ -154,10 +236,11 @@ static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordin private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { return new DataStatisticsCoordinator( OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, Fixtures.NUM_SUBTASKS), + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), Fixtures.SCHEMA, Fixtures.SORT_ORDER, - Fixtures.NUM_SUBTASKS, - type); + NUM_SUBTASKS, + type, + 0.0d); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index 966c8474b42b..6317f2bfde18 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -19,16 +19,19 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; +import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.params.ParameterizedTest; @@ -47,7 +50,7 @@ public void before() { @ParameterizedTest @EnumSource(StatisticsType.class) public void testCheckpointAndReset(StatisticsType type) throws Exception { - DataStatisticsCoordinatorProvider provider = createProvider(type); + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) provider.create( @@ -62,42 +65,39 @@ public void testCheckpointAndReset(StatisticsType type) throws Exception { // Handle events from operators for checkpoint 1 StatisticsEvent checkpoint1Subtask0StatisticsEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c")); + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent( - type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); // Verify checkpoint 1 global data statistics - assertThat(dataStatisticsCoordinator.completedStatistics()).isNotNull(); - AggregatedStatistics aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + Map checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), - 1L, - CHAR_KEYS.get("b"), - 1L, - CHAR_KEYS.get("c"), - 1L, - CHAR_KEYS.get("d"), - 1L, - CHAR_KEYS.get("e"), - 1L)); + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); } byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); @@ -114,42 +114,54 @@ public void testCheckpointAndReset(StatisticsType type) throws Exception { TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); // Verify checkpoint 2 global data statistics - aggregatedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + Map checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L)); + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); } + waitForCheckpoint(2L, dataStatisticsCoordinator); // Reset coordinator to checkpoint 1 coordinator.resetToCheckpoint(1L, checkpoint1Bytes); DataStatisticsCoordinator restoredDataStatisticsCoordinator = (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); - aggregatedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); - assertThat(aggregatedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); // Verify restored data statistics if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregatedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), - 1L, - CHAR_KEYS.get("b"), - 1L, - CHAR_KEYS.get("c"), - 1L, - CHAR_KEYS.get("d"), - 1L, - CHAR_KEYS.get("e"), - 1L)); + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); } else { - assertThat(aggregatedStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("c")); + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); } } } @@ -161,13 +173,15 @@ private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator co return future.get(); } - private static DataStatisticsCoordinatorProvider createProvider(StatisticsType type) { + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { return new DataStatisticsCoordinatorProvider( "DataStatisticsCoordinatorProvider", OPERATOR_ID, Fixtures.SCHEMA, Fixtures.SORT_ORDER, - Fixtures.NUM_SUBTASKS, - type); + downstreamParallelism, + type, + 0.0); } } 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 6c864791a1bc..bc248b778184 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 @@ -19,12 +19,16 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; @@ -56,7 +60,10 @@ import org.junit.jupiter.api.BeforeEach; 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.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; public class TestDataStatisticsOperator { @@ -75,15 +82,22 @@ public void before() throws Exception { new TestTaskStateManager()); } - private DataStatisticsOperator createOperator(StatisticsType type) throws Exception { + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + return createOperator(type, downstreamParallelism, mockGateway); + } + + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { DataStatisticsOperator operator = new DataStatisticsOperator( "testOperator", Fixtures.SCHEMA, Fixtures.SORT_ORDER, mockGateway, - Fixtures.NUM_SUBTASKS, + downstreamParallelism, type); operator.setup( new OneInputStreamTask(env), @@ -96,7 +110,7 @@ private DataStatisticsOperator createOperator(StatisticsType type) throws Except @ParameterizedTest @EnumSource(StatisticsType.class) public void testProcessElement(StatisticsType type) throws Exception { - DataStatisticsOperator operator = createOperator(type); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness testHarness = createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); @@ -125,7 +139,7 @@ public void testProcessElement(StatisticsType type) throws Exception { @ParameterizedTest @EnumSource(StatisticsType.class) public void testOperatorOutput(StatisticsType type) throws Exception { - DataStatisticsOperator operator = createOperator(type); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness testHarness = createHarness(operator)) { testHarness.processElement( @@ -149,35 +163,47 @@ public void testOperatorOutput(StatisticsType type) throws Exception { } } + private static Stream provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testRestoreState(StatisticsType type) throws Exception { + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { Map keyFrequency = ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; - DataStatisticsOperator operator = createOperator(type); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); OperatorSubtaskState snapshot; try (OneInputStreamOperatorTestHarness testHarness1 = createHarness(operator)) { - AggregatedStatistics statistics; + GlobalStatistics statistics; if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - statistics = AggregatedStatistics.fromKeyFrequency(1L, keyFrequency); + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); } else { - statistics = AggregatedStatistics.fromRangeBounds(1L, rangeBounds); + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); } StatisticsEvent event = - StatisticsEvent.createAggregatedStatisticsEvent( - 1L, statistics, Fixtures.AGGREGATED_STATISTICS_SERIALIZER); + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); operator.handleOperatorEvent(event); - AggregatedStatistics globalStatistics = operator.globalStatistics(); + GlobalStatistics globalStatistics = operator.globalStatistics(); assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.keyFrequency()).isEqualTo(keyFrequency); + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); assertThat(globalStatistics.rangeBounds()).isNull(); } else { - assertThat(globalStatistics.keyFrequency()).isNull(); + assertThat(globalStatistics.mapAssignment()).isNull(); assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); } @@ -186,19 +212,28 @@ public void testRestoreState(StatisticsType type) throws Exception { // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator restoredOperator = createOperator(type); + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); try (OneInputStreamOperatorTestHarness testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); - AggregatedStatistics globalStatistics = restoredOperator.globalStatistics(); + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.keyFrequency()).isEqualTo(keyFrequency); + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); assertThat(globalStatistics.rangeBounds()).isNull(); } else { - assertThat(globalStatistics.keyFrequency()).isNull(); + assertThat(globalStatistics.mapAssignment()).isNull(); assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); } } @@ -207,7 +242,7 @@ public void testRestoreState(StatisticsType type) throws Exception { @SuppressWarnings("unchecked") @Test public void testMigrationWithLocalStatsOverThreshold() throws Exception { - DataStatisticsOperator operator = createOperator(StatisticsType.Auto); + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness testHarness = createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); @@ -259,7 +294,7 @@ public void testMigrationWithLocalStatsOverThreshold() throws Exception { @SuppressWarnings("unchecked") @Test public void testMigrationWithGlobalSketchStatistics() throws Exception { - DataStatisticsOperator operator = createOperator(StatisticsType.Auto); + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); try (OneInputStreamOperatorTestHarness testHarness = createHarness(operator)) { StateInitializationContext stateContext = getStateContext(); @@ -272,12 +307,12 @@ public void testMigrationWithGlobalSketchStatistics() throws Exception { .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); // received global statistics with sketch type - AggregatedStatistics globalStatistics = - AggregatedStatistics.fromRangeBounds( + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); operator.handleOperatorEvent( - StatisticsEvent.createAggregatedStatisticsEvent( - 1L, globalStatistics, Fixtures.AGGREGATED_STATISTICS_SERIALIZER)); + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); int reservoirSize = SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); @@ -310,7 +345,7 @@ private OneInputStreamOperatorTestHarness createHar dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); harness.setup( new StatisticsOrRecordSerializer( - Fixtures.AGGREGATED_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); harness.open(); return harness; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.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.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d47e41768072..d5a0bebc74e7 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -89,39 +90,35 @@ private static SortKey[] initSortKeys() { @Test public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 0.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // each task should get targeted weight of 100 (=800/8) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -143,19 +140,20 @@ public void testEvenlyDividableNoClosingFileCost() { Pair.of(100L, 1), 7, Pair.of(100L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 5.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 100 before close file cost factored in. // close file cost is 5 = 5% * 100. @@ -164,35 +162,30 @@ public void testEvenlyDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight with close cost per subtask is 110 (880/8) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -215,51 +208,48 @@ public void testEvenlyDividableWithClosingFileCost() { Pair.of(100L, 2), 7, Pair.of(75L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 0.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 // each task should get targeted weight of 89 = ceiling(800/9) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -283,19 +273,20 @@ public void testNonDividableNoClosingFileCost() { Pair.of(89L, 2), 8, Pair.of(88L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapStatistics, 5.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 89 before close file cost factored in. // close file cost is 5 (= 5% * 89) per file. @@ -304,35 +295,31 @@ public void testNonDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight per subtask is 98 ceiling(880/9) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -357,40 +344,39 @@ public void testNonDividableWithClosingFileCost() { Pair.of(88L, 2), 8, Pair.of(61L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); // drift threshold is high for non-dividable scenario with close cost validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); } private static Map>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { + MapRangePartitioner partitioner, int numPartitions, Map mapStatistics) { // The Map key is the subtaskId. // For the map value pair, the first element is the count of assigned and // the second element of Set is for the set of assigned keys. Map>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); return partitionResults; } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 435748281f83..31dae5c76aeb 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -53,7 +53,7 @@ public void testOperatorReservoirSize() { @Test public void testRangeBoundsOneChannel() { assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 1, Fixtures.SORT_ORDER_COMPARTOR, new SortKey[] { @@ -70,7 +70,7 @@ public void testRangeBoundsOneChannel() { @Test public void testRangeBoundsDivisible() { assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 3, Fixtures.SORT_ORDER_COMPARTOR, new SortKey[] { @@ -88,7 +88,7 @@ public void testRangeBoundsDivisible() { public void testRangeBoundsNonDivisible() { // step is 3 = ceiling(11/4) assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 4, Fixtures.SORT_ORDER_COMPARTOR, new SortKey[] { @@ -111,7 +111,7 @@ public void testRangeBoundsNonDivisible() { public void testRangeBoundsSkipDuplicates() { // step is 3 = ceiling(11/4) assertThat( - SketchUtil.determineBounds( + SketchUtil.rangeBounds( 4, Fixtures.SORT_ORDER_COMPARTOR, new SortKey[] { From 0671fc2cf27b8c76cb01c757cd4f7eec483a0304 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Tue, 23 Jul 2024 21:35:22 +0900 Subject: [PATCH 0502/1019] Flink 1.17, 1.18: Migrate remaining tests to JUnit5 (#10749) --- .../iceberg/flink/HadoopCatalogResource.java | 90 ------ .../iceberg/flink/HadoopTableResource.java | 64 ---- .../flink/MiniFlinkClusterExtension.java | 14 + .../apache/iceberg/flink/SimpleDataUtil.java | 10 +- .../apache/iceberg/flink/TestFixtures.java | 2 + .../iceberg/flink/data/TestRowProjection.java | 282 +++++++++--------- .../iceberg/flink/data/TestStructRowData.java | 2 +- .../flink/sink/TestCompressionSettings.java | 163 +++++----- .../iceberg/flink/sink/TestFlinkManifest.java | 51 ++-- .../flink/sink/TestIcebergStreamWriter.java | 126 ++++---- .../flink/sink/TestRowDataPartitionKey.java | 67 ++--- .../iceberg/flink/sink/TestTaskWriters.java | 87 +++--- .../shuffle/TestAggregatedStatistics.java | 2 +- .../TestAggregatedStatisticsTracker.java | 6 +- .../TestDataStatisticsCoordinator.java | 6 +- ...TestDataStatisticsCoordinatorProvider.java | 6 +- .../shuffle/TestDataStatisticsOperator.java | 10 +- ...TestIcebergSourceBoundedGenericRecord.java | 75 +++-- .../source/TestIcebergSourceContinuous.java | 199 ++++++------ .../source/TestIcebergSourceFailover.java | 154 ++++++---- ...gSourceFailoverWithWatermarkExtractor.java | 16 +- ...stIcebergSourceWithWatermarkExtractor.java | 44 +-- .../flink/source/TestProjectMetaColumn.java | 46 +-- .../iceberg/flink/source/TestSourceUtil.java | 17 +- .../iceberg/flink/HadoopCatalogResource.java | 90 ------ .../iceberg/flink/HadoopTableResource.java | 64 ---- .../flink/MiniFlinkClusterExtension.java | 14 + .../apache/iceberg/flink/SimpleDataUtil.java | 10 +- .../apache/iceberg/flink/TestFixtures.java | 2 + .../iceberg/flink/data/TestRowProjection.java | 2 - .../flink/sink/TestCompressionSettings.java | 163 +++++----- .../iceberg/flink/sink/TestFlinkManifest.java | 51 ++-- .../flink/sink/TestIcebergStreamWriter.java | 126 ++++---- .../flink/sink/TestRowDataPartitionKey.java | 67 ++--- .../iceberg/flink/sink/TestTaskWriters.java | 87 +++--- .../shuffle/TestAggregatedStatistics.java | 2 +- .../TestAggregatedStatisticsTracker.java | 6 +- .../TestDataStatisticsCoordinator.java | 6 +- ...TestDataStatisticsCoordinatorProvider.java | 6 +- .../shuffle/TestDataStatisticsOperator.java | 10 +- ...TestIcebergSourceBoundedGenericRecord.java | 75 +++-- .../source/TestIcebergSourceContinuous.java | 199 ++++++------ .../source/TestIcebergSourceFailover.java | 154 ++++++---- ...gSourceFailoverWithWatermarkExtractor.java | 16 +- ...stIcebergSourceWithWatermarkExtractor.java | 44 +-- .../flink/source/TestProjectMetaColumn.java | 46 +-- .../iceberg/flink/source/TestSourceUtil.java | 17 +- 47 files changed, 1315 insertions(+), 1481 deletions(-) delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +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; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.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.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java index 9a73b80e077d..d2e086aa448e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; @@ -50,4 +51,17 @@ public static MiniClusterExtension createWithClassloaderCheckDisabled() { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ce6caca12158..1767f774922a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Duration; @@ -71,7 +72,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; -import org.junit.Assert; public class SimpleDataUtil { @@ -268,13 +268,13 @@ public static boolean equalsRecords(List expected, List actual, } public static void assertRecordsEqual(List expected, List actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); + assertThat(actual).hasSameSizeAs(expected); Types.StructType type = schema.asStruct(); StructLikeSet expectedSet = StructLikeSet.create(type); expectedSet.addAll(expected); StructLikeSet actualSet = StructLikeSet.create(type); actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } /** @@ -297,7 +297,7 @@ public static void assertTableRecords(Table table, List expected, String Snapshot snapshot = latestSnapshot(table, branch); if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); + assertThat(expected).isEmpty(); return; } @@ -313,7 +313,7 @@ public static void assertTableRecords(Table table, List expected, String actualSet.add(record); } - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java index 884ea2d1d3b1..b9a7d5b1d589 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -46,6 +46,8 @@ private TestFixtures() {} public static final String SINK_TABLE = "t_sink"; public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); public static final Schema TS_SCHEMA = new Schema( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java index df2e6ae21c7e..e76452b7cea0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -18,8 +18,12 @@ */ package org.apache.iceberg.flink.data; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.withPrecision; + import java.io.File; import java.io.IOException; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.table.data.ArrayData; import org.apache.flink.table.data.GenericArrayData; @@ -35,21 +39,18 @@ 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.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestRowProjection { - @Rule public TemporaryFolder temp = new TemporaryFolder(); + @TempDir private Path temp; private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) throws IOException { - File file = temp.newFile(desc + ".avro"); - Assert.assertTrue(file.delete()); + File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); + assertThat(file.delete()).isTrue(); try (FileAppender appender = Avro.write(Files.localOutput(file)) @@ -79,10 +80,8 @@ public void testFullProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, schema, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data value", cmp, 0); + assertThat(projected.getLong(0)).isEqualTo(34); + assertThat(projected.getString(1)).asString().isEqualTo("test"); } @Test @@ -96,19 +95,13 @@ public void testSpecialCharacterProjection() throws Exception { RowData full = writeAndRead("special_chars", schema, schema, row); - Assert.assertEquals("Should contain the correct id value", 34L, full.getLong(0)); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", full.getString(1).toString())); + assertThat(full.getLong(0)).isEqualTo(34L); + assertThat(full.getString(1)).asString().isEqualTo("test"); RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); - Assert.assertEquals("Should not contain id value", 1, projected.getArity()); - Assert.assertEquals( - "Should contain the correct data value", - 0, - Comparators.charSequences().compare("test", projected.getString(0).toString())); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getString(0)).asString().isEqualTo("test"); } @Test @@ -127,9 +120,8 @@ public void testReorderedFullProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, reordered, row); - Assert.assertEquals( - "Should contain the correct 0 value", "test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct 1 value", 34L, projected.getLong(1)); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + assertThat(projected.getLong(1)).isEqualTo(34); } @Test @@ -149,10 +141,9 @@ public void testReorderedProjection() throws Exception { RowData projected = writeAndRead("full_projection", schema, reordered, row); - Assert.assertTrue("Should contain the correct 0 value", projected.isNullAt(0)); - Assert.assertEquals( - "Should contain the correct 1 value", "test", projected.getString(1).toString()); - Assert.assertTrue("Should contain the correct 2 value", projected.isNullAt(2)); + assertThat(projected.isNullAt(0)).isTrue(); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + assertThat(projected.isNullAt(2)).isTrue(); } @Test @@ -173,10 +164,16 @@ public void testRenamedAddedField() throws Exception { Types.NestedField.optional(4, "d", Types.LongType.get())); RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertEquals("Should contain the correct value in column 2", projected.getLong(1), 200L); - Assert.assertEquals("Should contain the correct value in column 3", projected.getLong(2), 300L); - Assert.assertTrue("Should contain empty value on new column 4", projected.isNullAt(3)); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.getLong(1)) + .as("Should contain the correct value in column 2") + .isEqualTo(200L); + assertThat(projected.getLong(2)) + .as("Should contain the correct value in column 1") + .isEqualTo(300L); + assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); } @Test @@ -190,8 +187,8 @@ public void testEmptyProjection() throws Exception { RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); - Assert.assertNotNull("Should read a non-null record", projected); - Assert.assertEquals(0, projected.getArity()); + assertThat(projected).isNotNull(); + assertThat(projected.getArity()).isEqualTo(0); } @Test @@ -206,16 +203,15 @@ public void testBasicProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); - Assert.assertEquals("Should not project data", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + assertThat(projected.getArity()).as("Should not project data").isEqualTo(1); + assertThat(projected.getLong(0)).isEqualTo(34L); Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); - Assert.assertEquals("Should contain the correct data value", 0, cmp); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getString(0)).asString().isEqualTo("test"); } @Test @@ -234,9 +230,11 @@ public void testRename() throws Exception { RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - int cmp = Comparators.charSequences().compare("test", projected.getString(1).toString()); - Assert.assertEquals("Should contain the correct data/renamed value", 0, cmp); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getString(1)) + .as("Should contain the correct data/renamed value") + .asString() + .isEqualTo("test"); } @Test @@ -257,8 +255,8 @@ public void testNestedStructProjection() throws Exception { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); - Assert.assertEquals("Should not project location", 1, projected.getArity()); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getLong(0)).as("Should contain the correct id value").isEqualTo(34L); Schema latOnly = new Schema( @@ -269,11 +267,12 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("latitude_only", writeSchema, latOnly, record); RowData projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project longitude", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project longitude").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); Schema longOnly = new Schema( @@ -284,21 +283,24 @@ public void testNestedStructProjection() throws Exception { projected = writeAndRead("longitude_only", writeSchema, longOnly, record); projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals("Should not project latitutde", 1, projectedLocation.getArity()); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(0), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project latitutde").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); Schema locationOnly = writeSchema.select("location"); projected = writeAndRead("location_only", writeSchema, locationOnly, record); projectedLocation = projected.getRow(0, 1); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project location", projected.isNullAt(0)); - Assert.assertEquals( - "Should project latitude", 52.995143f, projectedLocation.getFloat(0), 0.000001f); - Assert.assertEquals( - "Should project longitude", -1.539054f, projectedLocation.getFloat(1), 0.000001f); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedLocation.getFloat(1)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); } @Test @@ -324,23 +326,23 @@ public void testMapProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project properties map", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project properties map").isEqualTo(1); Schema keyOnly = writeSchema.select("properties.key"); projected = writeAndRead("key_only", writeSchema, keyOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); Schema valueOnly = writeSchema.select("properties.value"); projected = writeAndRead("value_only", writeSchema, valueOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); Schema mapOnly = writeSchema.select("properties"); projected = writeAndRead("map_only", writeSchema, mapOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire map", properties, projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); } private Map toStringMap(Map map) { @@ -381,42 +383,50 @@ public void testMapOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project locations map", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project locations map").isEqualTo(1); projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project locations map", row.getMap(1), projected.getMap(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(row.getMap(1)); projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); GenericMapData locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); + assertThat(locations).isNotNull(); GenericArrayData l1l2Array = new GenericArrayData( new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should contain lat", 53.992811f, projectedL1.getFloat(0), 0.000001); - Assert.assertEquals("L1 should not contain long", 1, projectedL1.getArity()); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain lat") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + assertThat(projectedL1.getArity()).as("L1 should not contain long").isEqualTo(1); RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should contain lat", 52.995143f, projectedL2.getFloat(0), 0.000001); - Assert.assertEquals("L2 should not contain long", 1, projectedL2.getArity()); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain lat") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedL2.getArity()).as("L2 should not contain long").isEqualTo(1); projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals("L1 should not contain lat", 1, projectedL1.getArity()); - Assert.assertEquals("L1 should contain long", -1.542616f, projectedL1.getFloat(0), 0.000001); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getArity()).as("L1 should not contain lat").isEqualTo(1); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain long") + .isEqualTo(-1.542616f, withPrecision(0.000001f)); projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals("L2 should not contain lat", 1, projectedL2.getArity()); - Assert.assertEquals("L2 should contain long", -1.539054f, projectedL2.getFloat(0), 0.000001); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getArity()).as("L2 should not contain lat").isEqualTo(1); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain long") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); Schema latitiudeRenamed = new Schema( @@ -431,18 +441,20 @@ public void testMapOfStructsProjection() throws IOException { Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); locations = (GenericMapData) projected.getMap(0); - Assert.assertNotNull("Should project locations map", locations); - Assert.assertEquals("Should contain L1 and L2", l1l2Array, locations.keyArray()); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - Assert.assertNotNull("L1 should not be null", projectedL1); - Assert.assertEquals( - "L1 should contain latitude", 53.992811f, projectedL1.getFloat(0), 0.000001); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain latitude") + .isEqualTo(53.992811f, withPrecision(0.000001f)); projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - Assert.assertNotNull("L2 should not be null", projectedL2); - Assert.assertEquals( - "L2 should contain latitude", 52.995143f, projectedL2.getFloat(0), 0.000001); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); } @Test @@ -460,18 +472,18 @@ public void testListProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project values list", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project values list").isEqualTo(1); Schema elementOnly = writeSchema.select("values.element"); projected = writeAndRead("element_only", writeSchema, elementOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); Schema listOnly = writeSchema.select("values"); projected = writeAndRead("list_only", writeSchema, listOnly, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project entire list", values, projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); } @Test @@ -497,36 +509,36 @@ public void testListOfStructsProjection() throws IOException { Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - Assert.assertEquals("Should contain the correct id value", 34L, projected.getLong(0)); - Assert.assertEquals("Should not project points list", 1, projected.getArity()); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).isEqualTo(1); projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertEquals("Should project points list", row.getArray(1), projected.getArray(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(row.getArray(1)); projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); ArrayData points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); RowData projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should project x", 1, projectedP1.getInt(0)); - Assert.assertEquals("Should not project y", 1, projectedP1.getArity()); + assertThat(projectedP1.getInt(0)).as("Should project x").isEqualTo(1); + assertThat(projectedP1.getArity()).as("Should not project y").isEqualTo(1); RowData projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project y", 1, projectedP2.getArity()); - Assert.assertEquals("Should project x", 3, projectedP2.getInt(0)); + assertThat(projectedP2.getArity()).as("Should not project y").isEqualTo(1); + assertThat(projectedP2.getInt(0)).as("Should project x").isEqualTo(3); projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x", 1, projectedP1.getArity()); - Assert.assertEquals("Should project y", 2, projectedP1.getInt(0)); + assertThat(projectedP1.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project y").isEqualTo(2); projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null y", projectedP2.isNullAt(0)); + assertThat(projectedP2.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null y").isTrue(); Schema yRenamed = new Schema( @@ -539,16 +551,16 @@ public void testListOfStructsProjection() throws IOException { Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); - Assert.assertEquals("Should not project id", 1, projected.getArity()); - Assert.assertFalse("Should project points list", projected.isNullAt(0)); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); points = projected.getArray(0); - Assert.assertEquals("Should read 2 points", 2, points.size()); + assertThat(points.size()).isEqualTo(2); projectedP1 = points.getRow(0, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP1.getArity()); - Assert.assertEquals("Should project z", 2, projectedP1.getInt(0)); + assertThat(projectedP1.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project z").isEqualTo(2); projectedP2 = points.getRow(1, 2); - Assert.assertEquals("Should not project x and y", 1, projectedP2.getArity()); - Assert.assertTrue("Should project null z", projectedP2.isNullAt(0)); + assertThat(projectedP2.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); } @Test @@ -572,9 +584,11 @@ public void testAddedFieldsWithRequiredChildren() throws Exception { RowData projected = writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); - Assert.assertEquals("Should contain the correct value in column 1", projected.getLong(0), 100L); - Assert.assertTrue("Should contain empty value in new column 2", projected.isNullAt(1)); - Assert.assertTrue("Should contain empty value in new column 4", projected.isNullAt(2)); - Assert.assertTrue("Should contain empty value in new column 6", projected.isNullAt(3)); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.isNullAt(1)).as("Should contain empty value in new column 2").isTrue(); + assertThat(projected.isNullAt(2)).as("Should contain empty value in new column 4").isTrue(); + assertThat(projected.isNullAt(3)).as("Should contain empty value in new column 6").isTrue(); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java index e0340e0743b0..eccab20e04fc 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java @@ -22,7 +22,7 @@ import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; import org.apache.iceberg.flink.TestHelpers; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestStructRowData { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 14f12422da96..8faae1b05a4e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; @@ -35,53 +42,49 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; - private final Map initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } + @Parameter(index = 0) + private Map initProperties; - public TestCompressionSettings(Map initProperties) { - this.initProperties = initProperties; + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; } - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); } - @Test + @TestTemplate public void testCompressionAvro() throws Exception { // No override provided Map resultProperties = @@ -91,19 +94,17 @@ public void testCompressionAvro() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -119,11 +120,12 @@ public void testCompressionAvro() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionParquet() throws Exception { // No override provided Map resultProperties = @@ -133,19 +135,19 @@ public void testCompressionParquet() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -161,11 +163,12 @@ public void testCompressionParquet() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionOrc() throws Exception { // No override provided Map resultProperties = @@ -175,19 +178,18 @@ public void testCompressionOrc() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); } // Override compression to snappy and a different strategy @@ -203,8 +205,9 @@ public void testCompressionOrc() throws Exception { FlinkWriteOptions.COMPRESSION_STRATEGY.key(), "speed")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index ce1f208a4b07..53b7c4c0cc91 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -19,9 +19,12 @@ package org.apache.iceberg.flink.sink; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -45,28 +48,26 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkManifest { private static final Configuration CONF = new Configuration(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; private FileAppenderFactory appenderFactory; private final AtomicInteger fileCount = new AtomicInteger(0); - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + assertThat(new File(tablePath).mkdir()).isTrue(); // Construct the iceberg table. table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); @@ -112,11 +113,11 @@ public void testIO() throws IOException { WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < 5; i++) { TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); } @@ -131,7 +132,7 @@ public void testUserProvidedManifestLocation() throws IOException { long checkpointId = 1; String flinkJobId = newFlinkJobId(); String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = @@ -144,21 +145,18 @@ public void testUserProvidedManifestLocation() throws IOException { () -> factory.create(checkpointId), table.spec()); - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } @@ -198,7 +196,7 @@ public void testVersionedSerializer() throws IOException { byte[] versionedSerializeData2 = SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); } @Test @@ -220,14 +218,13 @@ public void testCompatibility() throws IOException { DeltaManifests delta = SimpleVersionedSerialization.readVersionAndDeSerialize( DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); TestHelpers.assertEquals(manifest, delta.dataManifest()); List actualFiles = FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); + assertThat(actualFiles).hasSize(10); for (int i = 0; i < 10; i++) { TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index c05f08b82f51..50283f7ad215 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Locale; @@ -42,6 +43,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; 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; @@ -58,49 +62,44 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; private Table table; - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; try (OneInputStreamOperatorTestHarness testHarness = @@ -111,10 +110,10 @@ public void testWritingTable() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); checkpointId = checkpointId + 1; @@ -125,8 +124,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); @@ -145,7 +144,7 @@ public void testWritingTable() throws Exception { } } - @Test + @TestTemplate public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; @@ -155,39 +154,39 @@ public void testSnapshotTwice() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } } - @Test + @TestTemplate public void testTableWithoutSnapshot() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); + assertThat(scanDataFiles()).isEmpty(); try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); + assertThat(scanDataFiles()).hasSize(1); } private Set scanDataFiles() throws IOException { @@ -211,7 +210,7 @@ private Set scanDataFiles() throws IOException { } } - @Test + @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { @@ -221,21 +220,21 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { @@ -244,22 +243,22 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testTableWithTargetFileSize() throws Exception { // Adjust the target-file-size in table properties. table @@ -285,12 +284,12 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); // Assert that the data file have the expected records. for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); + assertThat(dataFile.recordCount()).isEqualTo(1000); } // Commit the iceberg transaction. @@ -303,7 +302,7 @@ public void testTableWithTargetFileSize() throws Exception { SimpleDataUtil.assertTableRecords(table, records); } - @Test + @TestTemplate public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( @@ -329,7 +328,8 @@ public void testPromotedFlinkDataType() throws Exception { spec = PartitionSpec.unpartitioned(); } - String location = tempFolder.newFolder().getAbsolutePath(); + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); @@ -353,8 +353,8 @@ public void testPromotedFlinkDataType() throws Exception { } testHarness.prepareSnapshotPreBarrier(1); WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java index 79be96d20902..919fef579ab0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +37,7 @@ import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRowDataPartitionKey { private static final Schema SCHEMA = @@ -91,10 +92,10 @@ public void testNullPartitionValue() { for (RowData row : rows) { PartitionKey partitionKey = new PartitionKey(spec, schema); partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); + assertThat(partitionKey.size()).isEqualTo(1); String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); } } @@ -116,15 +117,15 @@ public void testPartitionWithOneNestedField() { PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); + assertThat(partitionKey1.size()).isEqualTo(1); - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); + assertThat(partitionKey2.size()).isEqualTo(1); - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); } } @@ -154,16 +155,16 @@ public void testPartitionMultipleNestedField() { Record record = (Record) records.get(i).get(0); pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); + assertThat(pk1.size()).isEqualTo(2); - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); + assertThat(pk2.size()).isEqualTo(2); - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); } } @@ -190,19 +191,18 @@ public void testPartitionValueTypes() { pk.partition(rowWrapper.wrap(row)); expectedPK.partition(recordWrapper.wrap(record)); - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } @@ -232,19 +232,18 @@ public void testNestedPartitionValues() { pk.partition(rowWrapper.wrap(rows.get(j))); expectedPK.partition(recordWrapper.wrap(records.get(j))); - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java index 112dbb511310..8bfd6cb3d043 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; @@ -30,6 +33,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -39,69 +45,62 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestTaskWriters { private static final Configuration CONF = new Configuration(); private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; - private Table table; + @Parameter(index = 1) + private boolean partitioned; - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } + private Table table; - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table with the specified file format. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); // Close again. taskWriter.close(); dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); } } - @Test + @TestTemplate public void testCloseTwice() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -111,16 +110,16 @@ public void testCloseTwice() throws IOException { int expectedFiles = partitioned ? 2 : 1; DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } } } - @Test + @TestTemplate public void testAbort() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -130,16 +129,16 @@ public void testAbort() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); } } } - @Test + @TestTemplate public void testCompleteFiles() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "a")); @@ -149,14 +148,14 @@ public void testCompleteFiles() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); @@ -176,7 +175,7 @@ public void testCompleteFiles() throws IOException { } } - @Test + @TestTemplate public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter taskWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); @@ -193,7 +192,7 @@ public void testRollingWithTargetFileSize() throws IOException { } DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); + assertThat(dataFiles).hasSize(8); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { @@ -206,7 +205,7 @@ public void testRollingWithTargetFileSize() throws IOException { } } - @Test + @TestTemplate public void testRandomData() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index 890cc361b246..739cf764e2a6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -25,7 +25,7 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; import org.apache.iceberg.types.Types; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestAggregatedStatistics { private final Schema schema = diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 4c64ce522201..0064c91340bf 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -25,8 +25,8 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; @@ -48,7 +48,7 @@ public TestAggregatedStatisticsTracker() { keyB.set(0, "b"); } - @Before + @BeforeEach public void before() throws Exception { aggregatedStatisticsTracker = new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 3df714059c37..849253564209 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -33,8 +33,8 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; @@ -52,7 +52,7 @@ public class TestDataStatisticsCoordinator { private DataStatisticsCoordinator> dataStatisticsCoordinator; - @Before + @BeforeEach public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); dataStatisticsCoordinator = diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index 5e0a752be506..c5216eeb712a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -31,8 +31,8 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); @@ -48,7 +48,7 @@ public class TestDataStatisticsCoordinatorProvider { private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - @Before + @BeforeEach public void before() { provider = new DataStatisticsCoordinatorProvider<>( diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 0e99a2d74ccb..5e6f971807ba 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -61,9 +61,9 @@ 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.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDataStatisticsOperator { private final Schema schema = @@ -92,7 +92,7 @@ private Environment getTestingEnvironment() { new TestTaskStateManager()); } - @Before + @BeforeEach public void before() throws Exception { this.operator = createOperator(); Environment env = getTestingEnvironment(); @@ -108,7 +108,7 @@ private DataStatisticsOperator> createOper "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } - @After + @AfterEach public void clean() throws Exception { operator.close(); } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 0337f3597053..7bfed00a9eb4 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -18,10 +18,10 @@ */ 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.Locale; import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -30,10 +30,12 @@ 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.test.util.MiniClusterWithClientResource; 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; @@ -43,8 +45,7 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; @@ -55,56 +56,48 @@ 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.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +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; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedGenericRecord { + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} + {FileFormat.AVRO, 2}, + {FileFormat.PARQUET, 2}, + {FileFormat.ORC, 2} }; } - private final FileFormat fileFormat; - private final int parallelism; + @Parameter(index = 0) + private FileFormat fileFormat; - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } + @Parameter(index = 1) + private int parallelism; - @Test + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { String dateStr = "2020-03-20"; Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); @@ -112,19 +105,19 @@ public void testPartitionedTable() throws Exception { expectedRecords.get(i).setField("dt", dateStr); } - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testProjection() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + 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)); @@ -149,7 +142,7 @@ private List run( Configuration config = new Configuration(); config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { + try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { tableLoader.open(); table = tableLoader.loadTable(); } @@ -168,7 +161,7 @@ private List run( IcebergSource.Builder sourceBuilder = IcebergSource.builder() - .tableLoader(catalogResource.tableLoader()) + .tableLoader(CATALOG_EXTENSION.tableLoader()) .readerFunction(readerFunction) .assignerFactory(new SimpleSplitAssignerFactory()) .flinkConfig(config); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 3f5af78704ca..9c7006e16b8e 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -35,7 +36,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; @@ -43,43 +45,41 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceContinuous { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); + @TempDir protected Path temporaryFolder; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private final AtomicLong randomSeed = new AtomicLong(0L); @Test public void testTableScanThenIncremental() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -92,49 +92,51 @@ public void testTableScanThenIncremental() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); } } @Test public void testTableScanThenIncrementalAfterExpiration() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot2 List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - Assert.assertEquals(1, tableResource.table().history().size()); + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); ScanContext scanContext = ScanContext.builder() @@ -143,8 +145,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { @@ -152,33 +154,35 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List initialRecords = Lists.newArrayList(); initialRecords.addAll(batch1); initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); } } @Test public void testEarliestSnapshot() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -193,41 +197,44 @@ public void testEarliestSnapshot() throws Exception { List result1 = waitForResult(iter, 4); List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); } } @Test - public void testLatestSnapshot() throws Exception { + public void testLatestSnapshot(@InjectClusterClient ClusterClient clusterClient) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -241,48 +248,50 @@ public void testLatestSnapshot() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // we want to make sure job is running first so that enumerator can // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + waitUntilJobIsRunning(clusterClient); // inclusive behavior for starting snapshot List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); } } @Test public void testSpecificSnapshotId() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); ScanContext scanContext = ScanContext.builder() @@ -295,47 +304,49 @@ public void testSpecificSnapshotId() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); } } @Test public void testSpecificSnapshotTimestamp() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); // sleep for 2 ms to make sure snapshot1 has a higher timestamp value Thread.sleep(2); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); ScanContext scanContext = ScanContext.builder() @@ -349,25 +360,25 @@ public void testSpecificSnapshotTimestamp() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // consume data from snapshot1 List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); } } @@ -375,27 +386,30 @@ public void testSpecificSnapshotTimestamp() throws Exception { public void testReadingFromBranch() throws Exception { String branch = "b1"; GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); List batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchBase); // create branch - tableResource + TABLE_EXTENSION .table() .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) .commit(); // snapshot1 to branch List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch1); // snapshot2 to branch List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch2); List branchExpectedRecords = Lists.newArrayList(); @@ -415,25 +429,26 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); // snapshot3 to branch List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); // snapshot4 to branch List batch4 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch4); List result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); } // read only from main branch. Should contain only the first snapshot @@ -446,14 +461,14 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); List batchMain2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchMain2); resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); } } @@ -462,7 +477,7 @@ public void testValidation() { assertThatThrownBy( () -> IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(true) .endTag("tag") @@ -478,7 +493,7 @@ private DataStream createStream(ScanContext scanContext) throws Exception { DataStream stream = env.fromSource( IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(scanContext.isStreaming()) .streamingStartingStrategy(scanContext.streamingStartingStrategy()) @@ -490,7 +505,7 @@ private DataStream createStream(ScanContext scanContext) throws Exception { WatermarkStrategy.noWatermarks(), "icebergSource", TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); return stream; } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index a378a0b93be3..62558fc9c8e5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Path; import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,6 +31,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; @@ -41,7 +43,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -51,53 +56,66 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +@Timeout(value = 120) public class TestIcebergSourceFailover { // Parallelism higher than 1, but lower than the number of splits used by some of our tests // The goal is to allow some splits to remain in the enumerator when restoring the state private static final int PARALLELISM = 2; private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) // Prevent combining splits .set( @@ -120,12 +138,11 @@ protected void assertRecords(Table table, List expectedRecords, Duration } @Test - public void testBoundedWithSavepoint() throws Exception { + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) + throws Exception { List expectedRecords = Lists.newArrayList(); - Table sinkTable = sinkTableResource.table(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List records = generateRecords(2, i); expectedRecords.addAll(records); @@ -141,13 +158,8 @@ public void testBoundedWithSavepoint() throws Exception { // Write something, but do not finish before checkpoint is created RecordCounterToWait.waitForCondition(); CompletableFuture savepoint = - miniClusterResource - .getClusterClient() - .stopWithSavepoint( - jobId, - false, - TEMPORARY_FOLDER.newFolder().toPath().toString(), - SavepointFormatType.CANONICAL); + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint @@ -170,19 +182,21 @@ public void testBoundedWithSavepoint() throws Exception { @Test public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); } - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { List expectedRecords = Lists.newArrayList(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List records = generateRecords(2, i); expectedRecords.addAll(records); @@ -197,29 +211,27 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio JobID jobId = jobClient.getJobID(); RecordCounterToWait.waitForCondition(); - triggerFailover( - failoverType, - jobId, - RecordCounterToWait::continueProcessing, - miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } @Test public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); } - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); List batch = generateRecords(2, 0); @@ -247,8 +259,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); @@ -260,13 +272,13 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep expectedRecords.addAll(records); dataAppender.appendToTable(records); if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, () -> {}, miniCluster); } } // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { @@ -286,8 +298,8 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); } @@ -295,6 +307,20 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ + private static void runTestWithNewMiniCluster(ThrowingConsumer testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + private enum FailoverType { NONE, TM, diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index cde39ec98d61..4f61d2f7308a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -49,11 +50,24 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + @Override protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) // Prevent combining splits diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 8341c8074f21..5e6a2b3caec6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; +import java.nio.file.Path; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -37,6 +38,7 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.InMemoryReporter; @@ -49,24 +51,24 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; 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.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; @@ -74,25 +76,24 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int RECORD_NUM_FOR_2_SPLITS = 200; private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) .withHaLeadershipControl() .build()); - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); /** * This is an integration test for watermark handling and windowing. Integration testing the @@ -259,7 +260,7 @@ public void apply( * the readers continue reading. */ @Test - public void testThrottling() throws Exception { + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { GenericAppenderHelper dataAppender = appender(); // Generate records in advance @@ -310,8 +311,7 @@ public void testThrottling() throws Exception { try (CloseableIterator resultIterator = stream.collectAsync()) { JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); // Insert the first data into the table dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); @@ -353,7 +353,7 @@ public void testThrottling() throws Exception { protected IcebergSource source() { return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) .splitSize(100L) @@ -377,7 +377,7 @@ protected Record generateRecord(int minutes, String str) { private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() + return REPORTER.findMetrics(jobID, metricsName).values().stream() .map(m -> (Gauge) m) .filter(m -> m.getValue() == withValue) .findFirst(); @@ -389,7 +389,7 @@ private GenericAppenderHelper appender() { hadoopConf.set("write.parquet.page-size-bytes", "64"); hadoopConf.set("write.parquet.row-group-size-bytes", "64"); return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); } private static RowData row(long time, long count) { diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java index 25ecec23d216..ce9054ad49b6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowDelta; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; @@ -40,20 +47,19 @@ 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.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestProjectMetaColumn { - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Iterable parameters() { return Lists.newArrayList( new Object[] {FileFormat.PARQUET}, @@ -61,13 +67,9 @@ public static Iterable parameters() { new Object[] {FileFormat.AVRO}); } - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, @@ -89,7 +91,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); + assertThat(rowData).isInstanceOf(GenericRowData.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); @@ -97,20 +99,20 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); } - @Test + @TestTemplate public void testV1SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(1); } - @Test + @TestTemplate public void testV2SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(2); } - @Test + @TestTemplate public void testV2RemoveMetaColumn() throws Exception { // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); @@ -132,7 +134,7 @@ public void testV2RemoveMetaColumn() throws Exception { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); + assertThat(rowData).isInstanceOf(RowDataProjection.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java index 317301260f66..b701419a7499 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSourceUtil { @Test @@ -30,31 +31,31 @@ public void testInferedParallelism() throws IOException { Configuration configuration = new Configuration(); // Empty table, infer parallelism should be at least 1 int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits // num : 2 parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + assertThat(parallelism).isEqualTo(2); // 2 splits and limit is 1 , max infer parallelism is default 100, // which is greater than splits num and limit, the parallelism is the limit value : 1 parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : // 1 parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java deleted file mode 100644 index 2b4694d3c268..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogResource.java +++ /dev/null @@ -1,90 +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; - -import java.io.File; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.rules.ExternalResource; -import org.junit.rules.TemporaryFolder; - -public class HadoopCatalogResource extends ExternalResource { - protected final TemporaryFolder temporaryFolder; - protected final String database; - protected final String tableName; - - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogResource(TemporaryFolder temporaryFolder, String database, String tableName) { - this.temporaryFolder = temporaryFolder; - this.database = database; - this.tableName = tableName; - } - - @Override - protected void before() throws Throwable { - File warehouseFile = temporaryFolder.newFolder(); - Assert.assertTrue(warehouseFile.delete()); - // before variables - this.warehouse = "file:" + warehouseFile; - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - protected void after() { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.java deleted file mode 100644 index 2da6f74b91d8..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/HadoopTableResource.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.flink; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.rules.TemporaryFolder; - -public class HadoopTableResource extends HadoopCatalogResource { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableResource( - TemporaryFolder temporaryFolder, String database, String tableName, Schema schema) { - this(temporaryFolder, database, tableName, schema, null); - } - - public HadoopTableResource( - TemporaryFolder temporaryFolder, - String database, - String tableName, - Schema schema, - PartitionSpec partitionSpec) { - super(temporaryFolder, database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - protected void before() throws Throwable { - super.before(); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java index 9a73b80e077d..d2e086aa448e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java @@ -20,6 +20,7 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; import org.apache.flink.test.junit5.MiniClusterExtension; @@ -50,4 +51,17 @@ public static MiniClusterExtension createWithClassloaderCheckDisabled() { .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) .build()); } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java index ce6caca12158..1767f774922a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink; import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; import java.time.Duration; @@ -71,7 +72,6 @@ import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; -import org.junit.Assert; public class SimpleDataUtil { @@ -268,13 +268,13 @@ public static boolean equalsRecords(List expected, List actual, } public static void assertRecordsEqual(List expected, List actual, Schema schema) { - Assert.assertEquals(expected.size(), actual.size()); + assertThat(actual).hasSameSizeAs(expected); Types.StructType type = schema.asStruct(); StructLikeSet expectedSet = StructLikeSet.create(type); expectedSet.addAll(expected); StructLikeSet actualSet = StructLikeSet.create(type); actualSet.addAll(actual); - Assert.assertEquals(expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } /** @@ -297,7 +297,7 @@ public static void assertTableRecords(Table table, List expected, String Snapshot snapshot = latestSnapshot(table, branch); if (snapshot == null) { - Assert.assertEquals(expected, ImmutableList.of()); + assertThat(expected).isEmpty(); return; } @@ -313,7 +313,7 @@ public static void assertTableRecords(Table table, List expected, String actualSet.add(record); } - Assert.assertEquals("Should produce the expected record", expectedSet, actualSet); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java index 884ea2d1d3b1..b9a7d5b1d589 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -46,6 +46,8 @@ private TestFixtures() {} public static final String SINK_TABLE = "t_sink"; public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); public static final Schema TS_SCHEMA = new Schema( 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 7dd4e8759c0e..e76452b7cea0 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 @@ -39,7 +39,6 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; @@ -212,7 +211,6 @@ public void testBasicProjection() throws Exception { projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); assertThat(projected.getString(0)).asString().isEqualTo("test"); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 14f12422da96..8faae1b05a4e 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.Map; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.common.DynFields; @@ -35,53 +42,49 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestCompressionSettings { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; - private final Map initProperties; - - @Parameterized.Parameters(name = "tableProperties = {0}") - public static Object[] parameters() { - return new Object[] { - ImmutableMap.of(), - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - }; - } + @Parameter(index = 0) + private Map initProperties; - public TestCompressionSettings(Map initProperties) { - this.initProperties = initProperties; + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; } - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); } - @Test + @TestTemplate public void testCompressionAvro() throws Exception { // No override provided Map resultProperties = @@ -91,19 +94,17 @@ public void testCompressionAvro() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION), - resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -119,11 +120,12 @@ public void testCompressionAvro() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.AVRO_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionParquet() throws Exception { // No override provided Map resultProperties = @@ -133,19 +135,19 @@ public void testCompressionParquet() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0, - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT, - resultProperties.get(TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); } else { - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION), - resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL), - resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); } // Override compression to snappy and some random level @@ -161,11 +163,12 @@ public void testCompressionParquet() throws Exception { FlinkWriteOptions.COMPRESSION_LEVEL.key(), "6")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.PARQUET_COMPRESSION)); - Assert.assertEquals("6", resultProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); } - @Test + @TestTemplate public void testCompressionOrc() throws Exception { // No override provided Map resultProperties = @@ -175,19 +178,18 @@ public void testCompressionOrc() throws Exception { ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT, - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); } else { - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION), - resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals( - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY), - resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); } // Override compression to snappy and a different strategy @@ -203,8 +205,9 @@ public void testCompressionOrc() throws Exception { FlinkWriteOptions.COMPRESSION_STRATEGY.key(), "speed")); - Assert.assertEquals("snappy", resultProperties.get(TableProperties.ORC_COMPRESSION)); - Assert.assertEquals("speed", resultProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index ce1f208a4b07..53b7c4c0cc91 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -19,9 +19,12 @@ package org.apache.iceberg.flink.sink; import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.nio.file.Paths; import java.util.List; import java.util.Map; @@ -45,28 +48,26 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.Pair; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; public class TestFlinkManifest { private static final Configuration CONF = new Configuration(); - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; private Table table; private FileAppenderFactory appenderFactory; private final AtomicInteger fileCount = new AtomicInteger(0); - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); String warehouse = folder.getAbsolutePath(); String tablePath = warehouse.concat("/test"); - Assert.assertTrue("Should create the table directory correctly.", new File(tablePath).mkdir()); + assertThat(new File(tablePath).mkdir()).isTrue(); // Construct the iceberg table. table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); @@ -112,11 +113,11 @@ public void testIO() throws IOException { WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals("Size of data file list are not equal.", 10, result.deleteFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } - Assert.assertEquals("Size of delete file list are not equal.", 10, result.dataFiles().length); + assertThat(result.deleteFiles()).hasSize(10); for (int i = 0; i < 5; i++) { TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); } @@ -131,7 +132,7 @@ public void testUserProvidedManifestLocation() throws IOException { long checkpointId = 1; String flinkJobId = newFlinkJobId(); String operatorId = newOperatorUniqueId(); - File userProvidedFolder = tempFolder.newFolder(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = @@ -144,21 +145,18 @@ public void testUserProvidedManifestLocation() throws IOException { () -> factory.create(checkpointId), table.spec()); - Assert.assertNotNull("Data manifest shouldn't be null", deltaManifests.dataManifest()); - Assert.assertNull("Delete manifest should be null", deltaManifests.deleteManifest()); - Assert.assertEquals( - "The newly created manifest file should be located under the user provided directory", - userProvidedFolder.toPath(), - Paths.get(deltaManifests.dataManifest().path()).getParent()); + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); WriteResult result = FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(5, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); - Assert.assertEquals( - "Size of data file list are not equal.", dataFiles.size(), result.dataFiles().length); + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); for (int i = 0; i < dataFiles.size(); i++) { TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); } @@ -198,7 +196,7 @@ public void testVersionedSerializer() throws IOException { byte[] versionedSerializeData2 = SimpleVersionedSerialization.writeVersionAndSerialize( DeltaManifestsSerializer.INSTANCE, actual); - Assert.assertArrayEquals(versionedSerializeData, versionedSerializeData2); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); } @Test @@ -220,14 +218,13 @@ public void testCompatibility() throws IOException { DeltaManifests delta = SimpleVersionedSerialization.readVersionAndDeSerialize( DeltaManifestsSerializer.INSTANCE, dataV1); - Assert.assertNull("Serialization v1 don't include delete files.", delta.deleteManifest()); - Assert.assertNotNull( - "Serialization v1 should not have null data manifest.", delta.dataManifest()); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); TestHelpers.assertEquals(manifest, delta.dataManifest()); List actualFiles = FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - Assert.assertEquals(10, actualFiles.size()); + assertThat(actualFiles).hasSize(10); for (int i = 0; i < 10; i++) { TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index c05f08b82f51..50283f7ad215 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -22,6 +22,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.Arrays; import java.util.List; import java.util.Locale; @@ -42,6 +43,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; 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; @@ -58,49 +62,44 @@ import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestIcebergStreamWriter { - @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; private Table table; - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - public TestIcebergStreamWriter(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } - - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; try (OneInputStreamOperatorTestHarness testHarness = @@ -111,10 +110,10 @@ public void testWritingTable() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); testHarness.prepareSnapshotPreBarrier(checkpointId); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); checkpointId = checkpointId + 1; @@ -125,8 +124,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // Commit the iceberg transaction. AppendFiles appendFiles = table.newAppend(); @@ -145,7 +144,7 @@ public void testWritingTable() throws Exception { } } - @Test + @TestTemplate public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; @@ -155,39 +154,39 @@ public void testSnapshotTwice() throws Exception { testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); // snapshot again immediately. for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } } - @Test + @TestTemplate public void testTableWithoutSnapshot() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. - Assert.assertEquals(0, scanDataFiles().size()); + assertThat(scanDataFiles()).isEmpty(); try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. - Assert.assertEquals(0, testHarness.extractOutputValues().size()); + assertThat(testHarness.extractOutputValues()).isEmpty(); } // Once we closed the iceberg stream writer, there will left an orphan data file. - Assert.assertEquals(1, scanDataFiles().size()); + assertThat(scanDataFiles()).hasSize(1); } private Set scanDataFiles() throws IOException { @@ -211,7 +210,7 @@ private Set scanDataFiles() throws IOException { } } - @Test + @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { @@ -221,21 +220,21 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { @@ -244,22 +243,22 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); - long expectedDataFiles = partitioned ? 2 : 1; + int expectedDataFiles = partitioned ? 2 : 1; WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); + assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once - Assert.assertEquals(expectedDataFiles, result.dataFiles().length); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); } } - @Test + @TestTemplate public void testTableWithTargetFileSize() throws Exception { // Adjust the target-file-size in table properties. table @@ -285,12 +284,12 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(8, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); // Assert that the data file have the expected records. for (DataFile dataFile : result.dataFiles()) { - Assert.assertEquals(1000, dataFile.recordCount()); + assertThat(dataFile.recordCount()).isEqualTo(1000); } // Commit the iceberg transaction. @@ -303,7 +302,7 @@ public void testTableWithTargetFileSize() throws Exception { SimpleDataUtil.assertTableRecords(table, records); } - @Test + @TestTemplate public void testPromotedFlinkDataType() throws Exception { Schema iSchema = new Schema( @@ -329,7 +328,8 @@ public void testPromotedFlinkDataType() throws Exception { spec = PartitionSpec.unpartitioned(); } - String location = tempFolder.newFolder().getAbsolutePath(); + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); @@ -353,8 +353,8 @@ public void testPromotedFlinkDataType() throws Exception { } testHarness.prepareSnapshotPreBarrier(1); WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - Assert.assertEquals(0, result.deleteFiles().length); - Assert.assertEquals(partitioned ? 3 : 1, result.dataFiles().length); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); // Commit the iceberg transaction. AppendFiles appendFiles = icebergTable.newAppend(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java index 79be96d20902..919fef579ab0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.util.List; import java.util.stream.Collectors; import org.apache.flink.table.data.GenericRowData; @@ -35,8 +37,7 @@ import org.apache.iceberg.flink.data.RandomRowData; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.types.Types; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestRowDataPartitionKey { private static final Schema SCHEMA = @@ -91,10 +92,10 @@ public void testNullPartitionValue() { for (RowData row : rows) { PartitionKey partitionKey = new PartitionKey(spec, schema); partitionKey.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey.size(), 1); + assertThat(partitionKey.size()).isEqualTo(1); String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - Assert.assertEquals(expectedStr, partitionKey.get(0, String.class)); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); } } @@ -116,15 +117,15 @@ public void testPartitionWithOneNestedField() { PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); partitionKey1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey1.size(), 1); + assertThat(partitionKey1.size()).isEqualTo(1); - Assert.assertEquals(record.get(0), partitionKey1.get(0, String.class)); + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); partitionKey2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(partitionKey2.size(), 1); + assertThat(partitionKey2.size()).isEqualTo(1); - Assert.assertEquals(record.get(1), partitionKey2.get(0, Integer.class)); + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); } } @@ -154,16 +155,16 @@ public void testPartitionMultipleNestedField() { Record record = (Record) records.get(i).get(0); pk1.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk1.size()); + assertThat(pk1.size()).isEqualTo(2); - Assert.assertEquals(record.get(1), pk1.get(0, Integer.class)); - Assert.assertEquals(record.get(0), pk1.get(1, String.class)); + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); pk2.partition(rowWrapper.wrap(row)); - Assert.assertEquals(2, pk2.size()); + assertThat(pk2.size()).isEqualTo(2); - Assert.assertEquals(record.get(0), pk2.get(0, String.class)); - Assert.assertEquals(record.get(1), pk2.get(1, Integer.class)); + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); } } @@ -190,19 +191,18 @@ public void testPartitionValueTypes() { pk.partition(rowWrapper.wrap(row)); expectedPK.partition(recordWrapper.wrap(record)); - Assert.assertEquals( - "Partition with column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("timeType")) { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with column " + column + " should have the expected values", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } @@ -232,19 +232,18 @@ public void testNestedPartitionValues() { pk.partition(rowWrapper.wrap(rows.get(j))); expectedPK.partition(recordWrapper.wrap(records.get(j))); - Assert.assertEquals( - "Partition with nested column " + column + " should have one field.", 1, pk.size()); + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); if (column.equals("nested.timeType")) { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, Long.class) / 1000, - pk.get(0, Long.class) / 1000); + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); } else { - Assert.assertEquals( - "Partition with nested column " + column + " should have the expected values.", - expectedPK.get(0, javaClasses[0]), - pk.get(0, javaClasses[0])); + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); } } } 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 112dbb511310..8bfd6cb3d043 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 @@ -18,8 +18,11 @@ */ package org.apache.iceberg.flink.sink; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.File; import java.io.IOException; +import java.nio.file.Files; import java.util.List; import java.util.Map; import org.apache.flink.table.data.RowData; @@ -30,6 +33,9 @@ import org.apache.iceberg.AppendFiles; import org.apache.iceberg.DataFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; @@ -39,69 +45,62 @@ import org.apache.iceberg.io.TaskWriter; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestTaskWriters { private static final Configuration CONF = new Configuration(); private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - @Rule public final TemporaryFolder tempFolder = new TemporaryFolder(); + @TempDir protected java.nio.file.Path temporaryFolder; - @Parameterized.Parameters(name = "format = {0}, partitioned = {1}") + @Parameters(name = "format = {0}, partitioned = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", true}, - {"avro", false}, - {"orc", true}, - {"orc", false}, - {"parquet", true}, - {"parquet", false} + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} }; } - private final FileFormat format; - private final boolean partitioned; + @Parameter(index = 0) + private FileFormat format; - private Table table; + @Parameter(index = 1) + private boolean partitioned; - public TestTaskWriters(String format, boolean partitioned) { - this.format = FileFormat.fromString(format); - this.partitioned = partitioned; - } + private Table table; - @Before + @BeforeEach public void before() throws IOException { - File folder = tempFolder.newFolder(); + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); // Construct the iceberg table with the specified file format. Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); } - @Test + @TestTemplate public void testWriteZeroRecord() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.close(); DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); // Close again. taskWriter.close(); dataFiles = taskWriter.dataFiles(); - Assert.assertNotNull(dataFiles); - Assert.assertEquals(0, dataFiles.length); + assertThat(dataFiles).isNotNull().isEmpty(); } } - @Test + @TestTemplate public void testCloseTwice() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -111,16 +110,16 @@ public void testCloseTwice() throws IOException { int expectedFiles = partitioned ? 2 : 1; DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } } } - @Test + @TestTemplate public void testAbort() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); @@ -130,16 +129,16 @@ public void testAbort() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 2 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertFalse(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); } } } - @Test + @TestTemplate public void testCompleteFiles() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { taskWriter.write(SimpleDataUtil.createRowData(1, "a")); @@ -149,14 +148,14 @@ public void testCompleteFiles() throws IOException { DataFile[] dataFiles = taskWriter.dataFiles(); int expectedFiles = partitioned ? 4 : 1; - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(expectedFiles, dataFiles.length); + assertThat(dataFiles).hasSize(expectedFiles); FileSystem fs = FileSystem.get(CONF); for (DataFile dataFile : dataFiles) { - Assert.assertTrue(fs.exists(new Path(dataFile.path().toString()))); + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); } AppendFiles appendFiles = table.newAppend(); @@ -176,7 +175,7 @@ public void testCompleteFiles() throws IOException { } } - @Test + @TestTemplate public void testRollingWithTargetFileSize() throws IOException { try (TaskWriter taskWriter = createTaskWriter(4)) { List rows = Lists.newArrayListWithCapacity(8000); @@ -193,7 +192,7 @@ public void testRollingWithTargetFileSize() throws IOException { } DataFile[] dataFiles = taskWriter.dataFiles(); - Assert.assertEquals(8, dataFiles.length); + assertThat(dataFiles).hasSize(8); AppendFiles appendFiles = table.newAppend(); for (DataFile dataFile : dataFiles) { @@ -206,7 +205,7 @@ public void testRollingWithTargetFileSize() throws IOException { } } - @Test + @TestTemplate public void testRandomData() throws IOException { try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java index 890cc361b246..739cf764e2a6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java @@ -25,7 +25,7 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; import org.apache.iceberg.types.Types; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestAggregatedStatistics { private final Schema schema = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 4c64ce522201..0064c91340bf 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -25,8 +25,8 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestAggregatedStatisticsTracker { private static final int NUM_SUBTASKS = 2; @@ -48,7 +48,7 @@ public TestAggregatedStatisticsTracker() { keyB.set(0, "b"); } - @Before + @BeforeEach public void before() throws Exception { aggregatedStatisticsTracker = new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); 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 3df714059c37..849253564209 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 @@ -33,8 +33,8 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; @@ -52,7 +52,7 @@ public class TestDataStatisticsCoordinator { private DataStatisticsCoordinator> dataStatisticsCoordinator; - @Before + @BeforeEach public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); dataStatisticsCoordinator = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index 5e0a752be506..c5216eeb712a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -31,8 +31,8 @@ import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; import org.apache.iceberg.types.Types; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); @@ -48,7 +48,7 @@ public class TestDataStatisticsCoordinatorProvider { private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; - @Before + @BeforeEach public void before() { provider = new DataStatisticsCoordinatorProvider<>( 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 0e99a2d74ccb..5e6f971807ba 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 @@ -61,9 +61,9 @@ 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.junit.After; -import org.junit.Before; -import org.junit.Test; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; public class TestDataStatisticsOperator { private final Schema schema = @@ -92,7 +92,7 @@ private Environment getTestingEnvironment() { new TestTaskStateManager()); } - @Before + @BeforeEach public void before() throws Exception { this.operator = createOperator(); Environment env = getTestingEnvironment(); @@ -108,7 +108,7 @@ private DataStatisticsOperator> createOper "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); } - @After + @AfterEach public void clean() throws Exception { operator.close(); } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 0337f3597053..7bfed00a9eb4 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -18,10 +18,10 @@ */ 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.Locale; import java.util.Map; import org.apache.avro.generic.GenericRecord; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -30,10 +30,12 @@ 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.test.util.MiniClusterWithClientResource; 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; @@ -43,8 +45,7 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; @@ -55,56 +56,48 @@ 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.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) -public class TestIcebergSourceBoundedGenericRecord { - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(); +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; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSourceBoundedGenericRecord { + @TempDir protected Path temporaryFolder; - @Rule - public final HadoopCatalogResource catalogResource = - new HadoopCatalogResource(TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE); + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameterized.Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}") public static Object[][] parameters() { return new Object[][] { - {"avro", 2}, - {"parquet", 2}, - {"orc", 2} + {FileFormat.AVRO, 2}, + {FileFormat.PARQUET, 2}, + {FileFormat.ORC, 2} }; } - private final FileFormat fileFormat; - private final int parallelism; + @Parameter(index = 0) + private FileFormat fileFormat; - public TestIcebergSourceBoundedGenericRecord(String format, int parallelism) { - this.fileFormat = FileFormat.valueOf(format.toUpperCase(Locale.ENGLISH)); - this.parallelism = parallelism; - } + @Parameter(index = 1) + private int parallelism; - @Test + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = - catalogResource.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER).appendToTable(expectedRecords); + new GenericAppenderHelper(table, fileFormat, temporaryFolder).appendToTable(expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testPartitionedTable() throws Exception { String dateStr = "2020-03-20"; Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); @@ -112,19 +105,19 @@ public void testPartitionedTable() throws Exception { expectedRecords.get(i).setField("dt", dateStr); } - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + new GenericAppenderHelper(table, fileFormat, temporaryFolder) .appendToTable(org.apache.iceberg.TestHelpers.Row.of(dateStr, 0), expectedRecords); TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); } - @Test + @TestTemplate public void testProjection() throws Exception { Table table = - catalogResource + CATALOG_EXTENSION .catalog() .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - new GenericAppenderHelper(table, fileFormat, TEMPORARY_FOLDER) + 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)); @@ -149,7 +142,7 @@ private List run( Configuration config = new Configuration(); config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); Table table; - try (TableLoader tableLoader = catalogResource.tableLoader()) { + try (TableLoader tableLoader = CATALOG_EXTENSION.tableLoader()) { tableLoader.open(); table = tableLoader.loadTable(); } @@ -168,7 +161,7 @@ private List run( IcebergSource.Builder sourceBuilder = IcebergSource.builder() - .tableLoader(catalogResource.tableLoader()) + .tableLoader(CATALOG_EXTENSION.tableLoader()) .readerFunction(readerFunction) .assignerFactory(new SimpleSplitAssignerFactory()) .flinkConfig(config); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 5765b73a1f63..749cbf89338a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -21,6 +21,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.nio.file.Path; import java.time.Duration; import java.util.Collection; import java.util.List; @@ -38,7 +39,8 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.iceberg.FileFormat; @@ -46,45 +48,43 @@ import org.apache.iceberg.data.RandomGenericData; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableResource; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.TestHelpers; import org.apache.iceberg.flink.data.RowDataToRowMapper; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.awaitility.Awaitility; -import org.junit.Assert; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceContinuous { public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); - @ClassRule - public static final MiniClusterWithClientResource MINI_CLUSTER_RESOURCE = - MiniClusterResource.createWithClassloaderCheckDisabled(METRIC_REPORTER); + @TempDir protected Path temporaryFolder; - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); - @Rule - public final HadoopTableResource tableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); private final AtomicLong randomSeed = new AtomicLong(0L); @Test public void testTableScanThenIncremental() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -97,27 +97,27 @@ public void testTableScanThenIncremental() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -126,22 +126,24 @@ public void testTableScanThenIncremental() throws Exception { @Test public void testTableScanThenIncrementalAfterExpiration() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshotId = tableResource.table().currentSnapshot().snapshotId(); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot2 List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); - tableResource.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - Assert.assertEquals(1, tableResource.table().history().size()); + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); ScanContext scanContext = ScanContext.builder() @@ -150,8 +152,8 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) .build(); - Assert.assertEquals( - FlinkSplitPlanner.ScanMode.BATCH, FlinkSplitPlanner.checkScanMode(scanContext)); + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { @@ -159,17 +161,17 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { List initialRecords = Lists.newArrayList(); initialRecords.addAll(batch1); initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, tableResource.table().schema()); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); - tableResource.table().currentSnapshot().snapshotId(); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -178,16 +180,18 @@ public void testTableScanThenIncrementalAfterExpiration() throws Exception { @Test public void testEarliestSnapshot() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -202,43 +206,46 @@ public void testEarliestSnapshot() throws Exception { List result1 = waitForResult(iter, 4); List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } } @Test - public void testLatestSnapshot() throws Exception { + public void testLatestSnapshot(@InjectClusterClient ClusterClient clusterClient) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); ScanContext scanContext = @@ -252,29 +259,29 @@ public void testLatestSnapshot() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // we want to make sure job is running first so that enumerator can // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(MINI_CLUSTER_RESOURCE.getClusterClient()); + waitUntilJobIsRunning(clusterClient); // inclusive behavior for starting snapshot List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -283,19 +290,21 @@ public void testLatestSnapshot() throws Exception { @Test public void testSpecificSnapshotId() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1 = tableResource.table().currentSnapshot().snapshotId(); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); ScanContext scanContext = ScanContext.builder() @@ -308,25 +317,25 @@ public void testSpecificSnapshotId() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -335,22 +344,24 @@ public void testSpecificSnapshotId() throws Exception { @Test public void testSpecificSnapshotTimestamp() throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); // snapshot0 List batch0 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch0); - long snapshot0Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); // sleep for 2 ms to make sure snapshot1 has a higher timestamp value Thread.sleep(2); // snapshot1 List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch1); - long snapshot1Timestamp = tableResource.table().currentSnapshot().timestampMillis(); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); ScanContext scanContext = ScanContext.builder() @@ -364,25 +375,25 @@ public void testSpecificSnapshotTimestamp() throws Exception { createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { // consume data from snapshot1 List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, tableResource.table().schema()); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); // snapshot2 List batch2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch2); List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, tableResource.table().schema()); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); // snapshot3 List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); assertThatIcebergEnumeratorMetricsExist(); } @@ -392,27 +403,30 @@ public void testSpecificSnapshotTimestamp() throws Exception { public void testReadingFromBranch() throws Exception { String branch = "b1"; GenericAppenderHelper dataAppender = - new GenericAppenderHelper(tableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); List batchBase = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchBase); // create branch - tableResource + TABLE_EXTENSION .table() .manageSnapshots() - .createBranch(branch, tableResource.table().currentSnapshot().snapshotId()) + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) .commit(); // snapshot1 to branch List batch1 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch1); // snapshot2 to branch List batch2 = - RandomGenericData.generate(tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch2); List branchExpectedRecords = Lists.newArrayList(); @@ -432,25 +446,26 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords(resultMain, branchExpectedRecords, tableResource.table().schema()); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); // snapshot3 to branch List batch3 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch3); List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, tableResource.table().schema()); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); // snapshot4 to branch List batch4 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(branch, batch4); List result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, tableResource.table().schema()); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); } // read only from main branch. Should contain only the first snapshot @@ -463,14 +478,14 @@ public void testReadingFromBranch() throws Exception { try (CloseableIterator iter = createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { List resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); List batchMain2 = RandomGenericData.generate( - tableResource.table().schema(), 2, randomSeed.incrementAndGet()); + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); dataAppender.appendToTable(batchMain2); resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, tableResource.table().schema()); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); } } @@ -479,7 +494,7 @@ public void testValidation() { assertThatThrownBy( () -> IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(true) .endTag("tag") @@ -495,7 +510,7 @@ private DataStream createStream(ScanContext scanContext) throws Exception { DataStream stream = env.fromSource( IcebergSource.forRowData() - .tableLoader(tableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) .streaming(scanContext.isStreaming()) .streamingStartingStrategy(scanContext.streamingStartingStrategy()) @@ -507,7 +522,7 @@ private DataStream createStream(ScanContext scanContext) throws Exception { WatermarkStrategy.noWatermarks(), "icebergSource", TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(tableResource.table().schema()))); + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); return stream; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index a378a0b93be3..62558fc9c8e5 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; import static org.assertj.core.api.Assertions.assertThat; +import java.nio.file.Path; import java.time.Duration; import java.util.List; import java.util.concurrent.CompletableFuture; @@ -30,6 +31,7 @@ import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.restartstrategy.RestartStrategies; import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.execution.JobClient; import org.apache.flink.core.execution.SavepointFormatType; @@ -41,7 +43,10 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Schema; import org.apache.iceberg.Table; @@ -51,53 +56,66 @@ import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.FlinkConfigOptions; import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.sink.FlinkSink; import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.rules.Timeout; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; +@Timeout(value = 120) public class TestIcebergSourceFailover { // Parallelism higher than 1, but lower than the number of splits used by some of our tests // The goal is to allow some splits to remain in the enumerator when restoring the state private static final int PARALLELISM = 2; private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); - - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build()); - - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, schema()); - - @Rule - public final HadoopTableResource sinkTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.SINK_TABLE, schema()); - - @Rule public Timeout globalTimeout = Timeout.seconds(120); + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .assignerFactory(new SimpleSplitAssignerFactory()) // Prevent combining splits .set( @@ -120,12 +138,11 @@ protected void assertRecords(Table table, List expectedRecords, Duration } @Test - public void testBoundedWithSavepoint() throws Exception { + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) + throws Exception { List expectedRecords = Lists.newArrayList(); - Table sinkTable = sinkTableResource.table(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List records = generateRecords(2, i); expectedRecords.addAll(records); @@ -141,13 +158,8 @@ public void testBoundedWithSavepoint() throws Exception { // Write something, but do not finish before checkpoint is created RecordCounterToWait.waitForCondition(); CompletableFuture savepoint = - miniClusterResource - .getClusterClient() - .stopWithSavepoint( - jobId, - false, - TEMPORARY_FOLDER.newFolder().toPath().toString(), - SavepointFormatType.CANONICAL); + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); RecordCounterToWait.continueProcessing(); // Wait for the job to stop with the savepoint @@ -170,19 +182,21 @@ public void testBoundedWithSavepoint() throws Exception { @Test public void testBoundedWithTaskManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testBoundedWithJobManagerFailover() throws Exception { - testBoundedIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); } - private void testBoundedIcebergSource(FailoverType failoverType) throws Exception { + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { List expectedRecords = Lists.newArrayList(); GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); for (int i = 0; i < 4; ++i) { List records = generateRecords(2, i); expectedRecords.addAll(records); @@ -197,29 +211,27 @@ private void testBoundedIcebergSource(FailoverType failoverType) throws Exceptio JobID jobId = jobClient.getJobID(); RecordCounterToWait.waitForCondition(); - triggerFailover( - failoverType, - jobId, - RecordCounterToWait::continueProcessing, - miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } @Test public void testContinuousWithTaskManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.TM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); } @Test public void testContinuousWithJobManagerFailover() throws Exception { - testContinuousIcebergSource(FailoverType.JM); + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); } - private void testContinuousIcebergSource(FailoverType failoverType) throws Exception { + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { GenericAppenderHelper dataAppender = - new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER); + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); List expectedRecords = Lists.newArrayList(); List batch = generateRecords(2, 0); @@ -247,8 +259,8 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(stream) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); @@ -260,13 +272,13 @@ private void testContinuousIcebergSource(FailoverType failoverType) throws Excep expectedRecords.addAll(records); dataAppender.appendToTable(records); if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniClusterResource.getMiniCluster()); + triggerFailover(failoverType, jobId, () -> {}, miniCluster); } } // wait longer for continuous source to reduce flakiness // because CI servers tend to be overloaded. - assertRecords(sinkTableResource.table(), expectedRecords, Duration.ofSeconds(120)); + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); } private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { @@ -286,8 +298,8 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // exactly-once behavior. When Iceberg sink, we can verify end-to-end // exactly-once. Here we mainly about source exactly-once behavior. FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTableResource.table()) - .tableLoader(sinkTableResource.tableLoader()) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) .append(); } @@ -295,6 +307,20 @@ private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) // test utilities copied from Flink's FileSourceTextLinesITCase // ------------------------------------------------------------------------ + private static void runTestWithNewMiniCluster(ThrowingConsumer testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + private enum FailoverType { NONE, TM, diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java index cde39ec98d61..4f61d2f7308a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java @@ -40,6 +40,7 @@ import org.apache.iceberg.types.Comparators; import org.apache.iceberg.util.StructLikeWrapper; import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { // Increment ts by 15 minutes for each generateRecords batch @@ -49,11 +50,24 @@ public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIceberg private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + @Override protected IcebergSource.Builder sourceBuilder() { Configuration config = new Configuration(); return IcebergSource.forRowData() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) // Prevent combining splits diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 8341c8074f21..5e6a2b3caec6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; +import java.nio.file.Path; import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; @@ -37,6 +38,7 @@ import org.apache.flink.core.execution.JobClient; import org.apache.flink.metrics.Gauge; import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; import org.apache.flink.runtime.minicluster.RpcServiceSharing; import org.apache.flink.runtime.testutils.CommonTestUtils; import org.apache.flink.runtime.testutils.InMemoryReporter; @@ -49,24 +51,24 @@ import org.apache.flink.streaming.api.windowing.windows.TimeWindow; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.util.CloseableIterator; import org.apache.flink.util.Collector; import org.apache.iceberg.FileFormat; import org.apache.iceberg.data.GenericAppenderHelper; import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableResource; +import org.apache.iceberg.flink.HadoopTableExtension; import org.apache.iceberg.flink.TestFixtures; 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.awaitility.Awaitility; -import org.junit.ClassRule; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int PARALLELISM = 4; @@ -74,25 +76,24 @@ public class TestIcebergSourceWithWatermarkExtractor implements Serializable { private static final int RECORD_NUM_FOR_2_SPLITS = 200; private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); - @ClassRule public static final TemporaryFolder TEMPORARY_FOLDER = new TemporaryFolder(); + @TempDir protected Path temporaryFolder; - private final InMemoryReporter reporter = InMemoryReporter.createWithRetainedMetrics(); + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - @Rule - public final MiniClusterWithClientResource miniClusterResource = - new MiniClusterWithClientResource( + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(1) .setNumberSlotsPerTaskManager(PARALLELISM) .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(reporter.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) .withHaLeadershipControl() .build()); - @Rule - public final HadoopTableResource sourceTableResource = - new HadoopTableResource( - TEMPORARY_FOLDER, TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); /** * This is an integration test for watermark handling and windowing. Integration testing the @@ -259,7 +260,7 @@ public void apply( * the readers continue reading. */ @Test - public void testThrottling() throws Exception { + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { GenericAppenderHelper dataAppender = appender(); // Generate records in advance @@ -310,8 +311,7 @@ public void testThrottling() throws Exception { try (CloseableIterator resultIterator = stream.collectAsync()) { JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning( - miniClusterResource.getMiniCluster(), jobClient.getJobID(), false); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); // Insert the first data into the table dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); @@ -353,7 +353,7 @@ public void testThrottling() throws Exception { protected IcebergSource source() { return IcebergSource.builder() - .tableLoader(sourceTableResource.tableLoader()) + .tableLoader(TABLE_EXTENSION.tableLoader()) .watermarkColumn("ts") .project(TestFixtures.TS_SCHEMA) .splitSize(100L) @@ -377,7 +377,7 @@ protected Record generateRecord(int minutes, String str) { private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return reporter.findMetrics(jobID, metricsName).values().stream() + return REPORTER.findMetrics(jobID, metricsName).values().stream() .map(m -> (Gauge) m) .filter(m -> m.getValue() == withValue) .findFirst(); @@ -389,7 +389,7 @@ private GenericAppenderHelper appender() { hadoopConf.set("write.parquet.page-size-bytes", "64"); hadoopConf.set("write.parquet.row-group-size-bytes", "64"); return new GenericAppenderHelper( - sourceTableResource.table(), FileFormat.PARQUET, TEMPORARY_FOLDER, hadoopConf); + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); } private static RowData row(long time, long count) { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java index 25ecec23d216..ce9054ad49b6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -18,13 +18,20 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.List; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; import org.apache.iceberg.RowDelta; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.Table; @@ -40,20 +47,19 @@ 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.junit.Assert; -import org.junit.Rule; -import org.junit.Test; -import org.junit.rules.TemporaryFolder; -import org.junit.runner.RunWith; -import org.junit.runners.Parameterized; - -@RunWith(Parameterized.class) +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 TestProjectMetaColumn { - @Rule public final TemporaryFolder folder = new TemporaryFolder(); - private final FileFormat format; + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; - @Parameterized.Parameters(name = "fileFormat={0}") + @Parameters(name = "fileFormat={0}") public static Iterable parameters() { return Lists.newArrayList( new Object[] {FileFormat.PARQUET}, @@ -61,13 +67,9 @@ public static Iterable parameters() { new Object[] {FileFormat.AVRO}); } - public TestProjectMetaColumn(FileFormat format) { - this.format = format; - } - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { // Create the table with given format version. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, @@ -89,7 +91,7 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof GenericRowData); + assertThat(rowData).isInstanceOf(GenericRowData.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); @@ -97,20 +99,20 @@ private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); } - @Test + @TestTemplate public void testV1SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(1); } - @Test + @TestTemplate public void testV2SkipToRemoveMetaColumn() throws IOException { testSkipToRemoveMetaColumn(2); } - @Test + @TestTemplate public void testV2RemoveMetaColumn() throws Exception { // Create the v2 table. - String location = folder.getRoot().getAbsolutePath(); + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); Table table = SimpleDataUtil.createTable( location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); @@ -132,7 +134,7 @@ public void testV2RemoveMetaColumn() throws Exception { input, rowData -> { // If project to remove the meta columns, it will get a RowDataProjection. - Assert.assertTrue(rowData instanceof RowDataProjection); + assertThat(rowData).isInstanceOf(RowDataProjection.class); results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); }); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java index 317301260f66..b701419a7499 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -18,11 +18,12 @@ */ package org.apache.iceberg.flink.source; +import static org.assertj.core.api.Assertions.assertThat; + import java.io.IOException; import org.apache.flink.configuration.Configuration; import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.Assert; -import org.junit.Test; +import org.junit.jupiter.api.Test; public class TestSourceUtil { @Test @@ -30,31 +31,31 @@ public void testInferedParallelism() throws IOException { Configuration configuration = new Configuration(); // Empty table, infer parallelism should be at least 1 int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits // num : 2 parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 2, parallelism); + assertThat(parallelism).isEqualTo(2); // 2 splits and limit is 1 , max infer parallelism is default 100, // which is greater than splits num and limit, the parallelism is the limit value : 1 parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : // 1 parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - Assert.assertEquals("Should produce the expected parallelism.", 1, parallelism); + assertThat(parallelism).isEqualTo(1); } } From f1adeb7ad6f0bdfbe7f33b3adb6c3059a9781528 Mon Sep 17 00:00:00 2001 From: emkornfield Date: Tue, 23 Jul 2024 14:01:47 -0700 Subject: [PATCH 0503/1019] Spec: Clarify time travel implementation in Iceberg (#8982) Co-authored-by: Eduard Tudenhoefner Co-authored-by: Fokko Driesprong --- format/spec.md | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/format/spec.md b/format/spec.md index 1c029bdcb563..5a90f6fd978d 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1383,3 +1383,14 @@ Writing v2 metadata: * `sort_columns` was removed Note that these requirements apply when writing data to a v2 table. Tables that are upgraded from v1 may contain metadata that does not follow these requirements. Implementations should remain backward-compatible with v1 metadata requirements. + +## Appendix F: Implementation Notes + +This section covers topics not required by the specification but recommendations for systems implementing the Iceberg specification to help maintain a uniform experience. + +### Point in Time Reads (Time Travel) + +Iceberg supports two types of histories for tables. A history of previous "current snapshots" stored in ["snapshot-log" table metadata](#table-metadata-fields) and [parent-child lineage stored in "snapshots"](#table-metadata-fields). These two histories +might indicate different snapshot IDs for a specific timestamp. The discrepancies can be caused by a variety of table operations (e.g. updating the `current-snapshot-id` can be used to set the snapshot of a table to any arbitrary snapshot, which might have a lineage derived from a table branch or no lineage at all). + +When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. \ No newline at end of file From d8e49e384b80ef1efc5a21a34c92b69eb69f3837 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 24 Jul 2024 03:21:55 +0530 Subject: [PATCH 0504/1019] Build: Update revapi to compare against 1.6.0 (#10754) --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 1cb28078d233..425b1415abff 100644 --- a/build.gradle +++ b/build.gradle @@ -143,7 +143,7 @@ subprojects { revapi { oldGroup = project.group oldName = project.name - oldVersion = "1.5.0" + oldVersion = "1.6.0" } tasks.register('showDeprecationRulesOnRevApiFailure') { From 2b5edce89ba2b16c7e32176ea7f0e731900491ac Mon Sep 17 00:00:00 2001 From: Venkata krishnan Sowrirajan Date: Tue, 23 Jul 2024 22:18:50 -0700 Subject: [PATCH 0505/1019] Support for Flink's SpeculativeExecution in batch execution mode (#10548) --- .../enumerator/AbstractIcebergEnumerator.java | 11 +- ...estIcebergSpeculativeExecutionSupport.java | 184 ++++++++++++++++++ 2 files changed, 194 insertions(+), 1 deletion(-) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 6c9a855bc149..280a126a46ce 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -37,7 +38,8 @@ import org.slf4j.LoggerFactory; abstract class AbstractIcebergEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -95,6 +97,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..3285a16a1277 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the even subtask indices with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), + getRuntimeContext().getTaskInfo().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} From 1bf585e45b165bb7c02d3cab7f8fa5f6070fa7b8 Mon Sep 17 00:00:00 2001 From: rice <18717838093@126.com> Date: Wed, 24 Jul 2024 22:40:30 +0800 Subject: [PATCH 0506/1019] API: Update StatisticsFile javadoc (#10769) --- api/src/main/java/org/apache/iceberg/StatisticsFile.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/StatisticsFile.java b/api/src/main/java/org/apache/iceberg/StatisticsFile.java index f3d74b72852a..8f5166cf9fad 100644 --- a/api/src/main/java/org/apache/iceberg/StatisticsFile.java +++ b/api/src/main/java/org/apache/iceberg/StatisticsFile.java @@ -28,7 +28,7 @@ * support is not required to read the table correctly. */ public interface StatisticsFile { - /** ID of the Iceberg table's snapshot the statistics were computed from. */ + /** ID of the Iceberg table's snapshot the statistics file is associated with. */ long snapshotId(); /** From 0fc9bb0a914aad7be4253820fb2d25f7e41dd251 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 24 Jul 2024 17:07:42 +0200 Subject: [PATCH 0507/1019] Flink: Remove JUnit4 dependency (#10770) --- flink/v1.17/build.gradle | 1 - flink/v1.18/build.gradle | 1 - flink/v1.19/build.gradle | 1 - 3 files changed, 3 deletions(-) diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle index 2509371bdb8b..2ced7a5a5cb7 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.17/build.gradle @@ -117,7 +117,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core - testImplementation libs.junit.vintage.engine } test { diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index afd4c3c6e114..f06318af83a3 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -117,7 +117,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core - testImplementation libs.junit.vintage.engine } test { diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index a0954942f0e9..392a1cb124f0 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -119,7 +119,6 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core - testImplementation libs.junit.vintage.engine } test { From b196ec3985e138a47dad031888f1992050ce9465 Mon Sep 17 00:00:00 2001 From: ritwika314 Date: Wed, 24 Jul 2024 13:26:29 -0400 Subject: [PATCH 0508/1019] Docs: Add bodo to iceberg vendors (#10756) Co-authored-by: ritwika314 --- site/docs/vendors.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/site/docs/vendors.md b/site/docs/vendors.md index dc20783da5ba..dc8cf135046c 100644 --- a/site/docs/vendors.md +++ b/site/docs/vendors.md @@ -22,6 +22,13 @@ title: "Vendors" This page contains some of the vendors who are shipping and supporting Apache Iceberg in their products +### [Bodo](https://bodo.ai) + +Bodo is a high performance SQL & Python compute engine that brings HPC and supercomputing techniques to data analytics. +Bodo supports Apache Iceberg tables as a first-class table format and storage, enabling users to read and write Iceberg +tables with Bodo's high-performance data processing engine. Bodo is available as a cloud service on +AWS and Azure, and as well as an on-premises solution. + ### [CelerData](https://celerdata.com) CelerData provides commercial offerings for [StarRocks](https://www.starrocks.io/), a distributed MPP SQL engine for enterprise analytics on Iceberg. With its fully vectorized technology, local caching, and intelligent materialized view, StarRocks delivers sub-second query latency for both batch and real-time analytics. CelerData offers both an [enterprise deployment](https://celerdata.com/celerdata-enterprise) and a [cloud service](https://celerdata.com/celerdata-cloud) to help customers use StarRocks more smoothly. Learn more about how to query Iceberg with StarRocks [here](https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog). From 969fea21feffc7d70e1ed41954884c8aefa5a8e9 Mon Sep 17 00:00:00 2001 From: Szehon Ho Date: Wed, 24 Jul 2024 13:00:26 -0700 Subject: [PATCH 0509/1019] Core: Implement estimateRowCount for Files and Entries Metadata Tables (#10759) --- .../org/apache/iceberg/BaseEntriesTable.java | 7 ++++ .../org/apache/iceberg/BaseFilesTable.java | 7 ++++ .../iceberg/TestMetadataTableScans.java | 32 +++++++++++++++++++ 3 files changed, 46 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index 5f35704bc35b..f4019d688cb8 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -304,6 +304,13 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask { : new Schema(); } + @Override + public long estimatedRowsCount() { + return (long) manifest.addedFilesCount() + + (long) manifest.deletedFilesCount() + + (long) manifest.existingFilesCount(); + } + @VisibleForTesting ManifestFile manifest() { return manifest; diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java index 011abffd8a2a..149edf950032 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java @@ -169,6 +169,13 @@ public CloseableIterable rows() { } } + @Override + public long estimatedRowsCount() { + return (long) manifest.addedFilesCount() + + (long) manifest.deletedFilesCount() + + (long) manifest.existingFilesCount(); + } + private CloseableIterable> files(Schema fileProjection) { switch (manifest.content()) { case DATA: diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 0a3040939c04..0a14a89c01b5 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -1753,4 +1753,36 @@ public void testPositionDeletesManyColumns() { assertThat(scanTasks.get(0).file().path().toString()).isEqualTo("/path/to/delete1.parquet"); assertThat(scanTasks.get(1).file().path().toString()).isEqualTo("/path/to/delete2.parquet"); } + + @TestTemplate + public void testFilesTableEstimateSize() throws Exception { + preparePartitionedTable(true); + + assertEstimatedRowCount(new DataFilesTable(table), 4); + assertEstimatedRowCount(new AllDataFilesTable(table), 4); + assertEstimatedRowCount(new AllFilesTable(table), 4); + + if (formatVersion == 2) { + assertEstimatedRowCount(new DeleteFilesTable(table), 4); + assertEstimatedRowCount(new AllDeleteFilesTable(table), 4); + } + } + + @TestTemplate + public void testEntriesTableEstimateSize() throws Exception { + preparePartitionedTable(true); + + assertEstimatedRowCount(new ManifestEntriesTable(table), 4); + assertEstimatedRowCount(new AllEntriesTable(table), 4); + } + + private void assertEstimatedRowCount(Table metadataTable, int size) throws Exception { + TableScan scan = metadataTable.newScan(); + + try (CloseableIterable tasks = scan.planFiles()) { + List taskList = Lists.newArrayList(tasks); + assertThat(taskList.size()).isGreaterThan(0); + taskList.forEach(task -> assertThat(task.estimatedRowsCount()).isEqualTo(size)); + } + } } From 2618b47e78d763f41b17db9d3e5c2b8b5ef80587 Mon Sep 17 00:00:00 2001 From: Attila Kreiner Date: Wed, 24 Jul 2024 23:52:47 +0300 Subject: [PATCH 0510/1019] Build: Updates Checkstyle definition (#10681) --- .baseline/checkstyle/checkstyle.xml | 21 ++++++--------------- 1 file changed, 6 insertions(+), 15 deletions(-) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 48ef3d690ef7..1491a35db194 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -159,7 +159,6 @@ - @@ -281,16 +280,13 @@ - - + - - + - - + @@ -308,8 +304,7 @@ - - + @@ -448,7 +443,6 @@ - @@ -484,14 +478,12 @@ - + - - @@ -499,8 +491,7 @@ - - + From c9d141dd3fbf74ea96d7fd95e65f4a12d63d5bf3 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 25 Jul 2024 01:02:56 -0600 Subject: [PATCH 0511/1019] API: Fix typo in RewriteManifestFiles java doc (#10778) --- api/src/main/java/org/apache/iceberg/RewriteManifests.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/RewriteManifests.java index ca823e94d265..32a9011ad6f6 100644 --- a/api/src/main/java/org/apache/iceberg/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/RewriteManifests.java @@ -54,7 +54,7 @@ public interface RewriteManifests extends SnapshotUpdate { * then all manifests will be rewritten. * * @param predicate Predicate used to determine which manifests to rewrite. If true then the - * manifest file will be included for rewrite. If false then then manifest is kept as-is. + * manifest file will be included for rewrite. If false then the manifest is kept as-is. * @return this for method chaining */ RewriteManifests rewriteIf(Predicate predicate); From 834f3274c409407d00231a4a54344f4aad1d9575 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 25 Jul 2024 17:50:18 +0530 Subject: [PATCH 0512/1019] Update .asf.yaml (#10767) Remove a contributor who is now a committer. --- .asf.yaml | 1 - 1 file changed, 1 deletion(-) diff --git a/.asf.yaml b/.asf.yaml index 260641b35819..83672d9a83ad 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -49,7 +49,6 @@ github: - jun-he - marton-bod - samarthjain - - findepi - SreeramGarlapati - samredai - gaborkaszab From 9f15840e3da85b8ac43089c59af6a890435e16aa Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 25 Jul 2024 14:45:48 +0200 Subject: [PATCH 0513/1019] Build: Support building with Java 21 (#10474) --- .github/workflows/delta-conversion-ci.yml | 4 ++-- .github/workflows/flink-ci.yml | 5 ++++- .github/workflows/hive-ci.yml | 4 ++-- .github/workflows/java-ci.yml | 6 +++--- .github/workflows/spark-ci.yml | 9 ++++++++- README.md | 2 +- baseline.gradle | 11 ++++++++++- build.gradle | 6 +++--- jmh.gradle | 4 ++-- site/docs/contribute.md | 2 +- spark/v3.3/build.gradle | 8 ++++++++ spark/v3.4/build.gradle | 8 ++++++++ spark/v3.5/build.gradle | 8 ++++++++ 13 files changed, 60 insertions(+), 17 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index 90c1ad4109fd..ac5314e8afb8 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -71,7 +71,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -100,7 +100,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index b74cbcc84291..d2e249c279f7 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -73,12 +73,15 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] flink: ['1.17', '1.18', '1.19'] exclude: # Flink 1.17 does not support Java 17. - jvm: 17 flink: '1.17' + # Flink 1.17 does not support Java 21. + - jvm: 21 + flink: '1.17' env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 6e97e2164758..ee487807cff4 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -69,7 +69,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -98,7 +98,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 083ae2f5fe36..e1fd90116ef1 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -65,7 +65,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -94,7 +94,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 @@ -107,7 +107,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index d923cf860f1d..7a47beeed7a4 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -71,9 +71,16 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17] + jvm: [8, 11, 17, 21] spark: ['3.3', '3.4', '3.5'] scala: ['2.12', '2.13'] + exclude: + # Spark 3.5 is the first version not failing on Java 21 (https://issues.apache.org/jira/browse/SPARK-42369) + # Full Java 21 support is coming in Spark 4 (https://issues.apache.org/jira/browse/SPARK-43831) + - jvm: 21 + spark: '3.3' + - jvm: 21 + spark: '3.4' env: SPARK_LOCAL_IP: localhost steps: diff --git a/README.md b/README.md index 8200a23ed63b..fe0d2b94c308 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ Community discussions happen primarily on the [dev mailing list][dev-list] or on ### Building -Iceberg is built using Gradle with Java 8, 11, or 17. +Iceberg is built using Gradle with Java 8, 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/baseline.gradle b/baseline.gradle index c890ada8de57..62ade9a632f4 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -45,7 +45,16 @@ subprojects { apply plugin: 'com.palantir.baseline-reproducibility' apply plugin: 'com.palantir.baseline-exact-dependencies' apply plugin: 'com.palantir.baseline-release-compatibility' - apply plugin: 'com.diffplug.spotless' + // We need to update Google Java Format to 1.17.0+ to run spotless on JDK 8, but that requires dropping support for JDK 8. + if (JavaVersion.current() == JavaVersion.VERSION_21) { + task spotlessApply { + doLast { + throw new GradleException("Spotless plugin is currently disabled when running on JDK 21 (until we drop JDK 8). To run spotlessApply please use a different JDK version.") + } + } + } else { + apply plugin: 'com.diffplug.spotless' + } pluginManager.withPlugin('com.palantir.baseline-checkstyle') { checkstyle { diff --git a/build.gradle b/build.gradle index 425b1415abff..09a04e3ae977 100644 --- a/build.gradle +++ b/build.gradle @@ -62,8 +62,8 @@ if (JavaVersion.current() == JavaVersion.VERSION_1_8) { } else if (JavaVersion.current() == JavaVersion.VERSION_11) { project.ext.jdkVersion = '11' project.ext.extraJvmArgs = [] -} else if (JavaVersion.current() == JavaVersion.VERSION_17) { - project.ext.jdkVersion = '17' +} else if (JavaVersion.current() == JavaVersion.VERSION_17 || JavaVersion.current() == JavaVersion.VERSION_21) { + project.ext.jdkVersion = JavaVersion.current().getMajorVersion().toString() project.ext.extraJvmArgs = ["--add-opens", "java.base/java.io=ALL-UNNAMED", "--add-opens", "java.base/java.lang.invoke=ALL-UNNAMED", "--add-opens", "java.base/java.lang.reflect=ALL-UNNAMED", @@ -86,7 +86,7 @@ if (JavaVersion.current() == JavaVersion.VERSION_1_8) { "--add-opens", "java.base/sun.security.action=ALL-UNNAMED", "--add-opens", "java.base/sun.util.calendar=ALL-UNNAMED"] } else { - throw new GradleException("This build must be run with JDK 8 or 11 or 17 but was executed with JDK " + JavaVersion.current()) + throw new GradleException("This build must be run with JDK 8 or 11 or 17 or 21 but was executed with JDK " + JavaVersion.current()) } tasks.withType(AbstractArchiveTask).configureEach { diff --git a/jmh.gradle b/jmh.gradle index de50162cb099..80f5f8d0ea63 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (jdkVersion != '8' && jdkVersion != '11' && jdkVersion != '17') { - throw new GradleException("The JMH benchmarks must be run with JDK 8 or JDK 11 or JDK 17") +if (jdkVersion != '8' && jdkVersion != '11' && jdkVersion != '17' && jdkVersion != '21') { + throw new GradleException("The JMH benchmarks must be run with JDK 8 or JDK 11 or JDK 17 or JDK 21") } def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") diff --git a/site/docs/contribute.md b/site/docs/contribute.md index d1e16e1c9939..17099c56bd79 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -84,7 +84,7 @@ settle disagreements or to force a decision. ## Building the Project Locally -Iceberg is built using Gradle with Java 8, 11, or 17. +Iceberg is built using Gradle with Java 8, 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index 63c1a8e90a2e..c081cffeb644 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -59,6 +59,10 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } compileOnly libs.errorprone.annotations compileOnly libs.avro.avro @@ -137,6 +141,10 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer dependencies { implementation "org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}" + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } implementation libs.roaringbitmap compileOnly "org.scala-lang:scala-library" diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 9c584ad3a21b..3b1761d39f63 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -59,6 +59,10 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } compileOnly libs.errorprone.annotations compileOnly libs.avro.avro @@ -138,6 +142,10 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer dependencies { implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } implementation libs.roaringbitmap compileOnly "org.scala-lang:scala-library" diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index f1af2120ef8c..2ba5d493c6cd 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -59,6 +59,10 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } compileOnly libs.errorprone.annotations compileOnly libs.avro.avro @@ -137,6 +141,10 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer dependencies { implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + if (scalaVersion == '2.12') { + // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support + implementation 'org.scala-lang:scala-library:2.12.18' + } implementation libs.roaringbitmap compileOnly "org.scala-lang:scala-library" From 24fa1f533951f671ead75f700082a7192f67df5c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 25 Jul 2024 16:00:17 +0200 Subject: [PATCH 0514/1019] Infra, Docs: Publish Apache Iceberg 1.6.0 release (#10752) Co-authored-by: Eduard Tudenhoefner --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 3 +- doap.rdf | 6 +- site/docs/releases.md | 96 ++++++++++++++++++- 3 files changed, 99 insertions(+), 6 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 3bad6325552d..9e7d6034fe2e 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.5.2 (latest release)" + - "1.6.0 (latest release)" + - "1.5.2" - "1.5.1" - "1.5.0" - "1.4.3" diff --git a/doap.rdf b/doap.rdf index 7f49fc42f793..4f2bd4222c1e 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.5.2 - 2024-05-09 - 1.5.2 + 1.6.0 + 2024-07-23 + 1.6.0 diff --git a/site/docs/releases.md b/site/docs/releases.md index 6ba0b481fce9..73d8bc32a071 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -71,6 +71,100 @@ To add a dependency on Iceberg in Maven, add the following to your `pom.xml`: ... ``` + +### 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). + +* Build + - Upgrade to Gradle 8.9 ([\#10686](https://github.com/apache/iceberg/pull/10686)) +* Core + - Add EnvironmentContext to commit summary ([\#9273](https://github.com/apache/iceberg/pull/9273)) + - Add explicit JSON parser for ConfigResponse ([\#9952](https://github.com/apache/iceberg/pull/9952)) + - Calling rewrite_position_delete_files fails on tables with more than 1k columns ([\#10020](https://github.com/apache/iceberg/pull/10020)) + - Expose table incremental scan for appends API in SerializableTable ([\#10682](Expose table incremental scan for appends API in SerializableTable)) + - Fix NPE during conflict handling of NULL partitions ([\#10680](https://github.com/apache/iceberg/pull/10680)) + - Fix ParallelIterable memory leak where queue continues to be populated even after iterator close ([\#9402](https://github.com/apache/iceberg/pull/9402)) + - Fix logging table name in scanning metadata table ([\#10141](https://github.com/apache/iceberg/pull/10141)) + - Lazily compute & cache hashCode in CharSequenceWrapper ([\#10023](https://github.com/apache/iceberg/pull/10023)) + - Pushdown data_file.content when filter manifests in entries table ([\#10203](https://github.com/apache/iceberg/pull/10203)) + - Use bulk delete when removing old metadata.json files ([\#10679](https://github.com/apache/iceberg/pull/10679)) + - JDBC Catalog: Add property to disable table initialization for JdbcCatalog ([\#10124](https://github.com/apache/iceberg/pull/10124)) + - JDBC Catalog: Exclude unexpected namespaces JdbcCatalog.listNamespaces ([\#10498](https://github.com/apache/iceberg/pull/10498)) + - JDBC Catalog: Fix JDBC Catalog table commit when migrating from schema V0 to V1 ([\#10111](https://github.com/apache/iceberg/pull/10111)) + - JDBC Catalog: Retry connections in JDBC catalog with user configured error code list ([\#10140](https://github.com/apache/iceberg/pull/10140)) + - JDBC Catalog: fix namespace SQL statement using ESCAPE character working with MySQL, PostgreSQL, ... ([\#10167](https://github.com/apache/iceberg/pull/10167)) + - REST Catalog: Assume issued_token_type is access_token to fully comply with RFC 6749 ([\#10314](https://github.com/apache/iceberg/pull/10314)) + - REST Catalog: Fix create v1 table on REST Catalog ([\#10369](https://github.com/apache/iceberg/pull/10369)) + - REST Catalog: Handles possible heap data corruption of OAuth2Util.AuthSession#headers ([\#10615](https://github.com/apache/iceberg/pull/10615)) + - REST Catalog: Handles potential NPE in RESTSessionCatalog#newSessionCache ([\#10607](https://github.com/apache/iceberg/pull/10607)) + - REST Catalog: Introduce AuthConfig ([\#10161](https://github.com/apache/iceberg/pull/10161)) + - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](Mark 502 and 504 statuses as retryable to the REST exponential retry strategy)) + - REST Catalog: disallow overriding "credential" in table sessions ([\#10345](https://github.com/apache/iceberg/pull/10345)) + - REST Catalog: fix incorrect token refresh thread name ([\#10223](https://github.com/apache/iceberg/pull/10223)) + - REST Catalog: fix spurious warning when shutting down refresh executor ([\#10087](https://github.com/apache/iceberg/pull/10087)) +* Kafka Connect + - Commit coordinator ([\#10351](https://github.com/apache/iceberg/pull/10351)) + - Record converters ([\#9641](https://github.com/apache/iceberg/pull/9641)) + - Handle namespace creation for auto table creation ([\#10186](https://github.com/apache/iceberg/pull/10186)) + - Update iceberg.hadoop-conf-dir config description ([\#10184](https://github.com/apache/iceberg/pull/10184)) +* Parquet + - Don't write column sizes when metrics mode is None ([\#10440](https://github.com/apache/iceberg/pull/10440)) +* Spark + - Fix handling of null binary values when sorting with zorder ([\#10026](https://github.com/apache/iceberg/pull/10026)) + - Spark writes/actions should only perform cleanup if failure is cleanable ([\#10373](https://github.com/apache/iceberg/pull/10373)) + - Use 'delete' if RowDelta only has delete files ([\#10123](https://github.com/apache/iceberg/pull/10123)) + - Support read of partition metadata column when table is over 1k ([\#10641](https://github.com/apache/iceberg/pull/10641)) + - Fix the setting of equalAuthorities in RemoveOrphanFilesProcedure ([\#10342](https://github.com/apache/iceberg/pull/10342)) + - Fix system function pushdown in CoW row-level commands ([\#10119](https://github.com/apache/iceberg/pull/10119)) + - Only traverse ancestors of current snapshot when building changelog scan ([\#10405](https://github.com/apache/iceberg/pull/10405)) + - Add max allowed failed commits to RewriteDataFiles when partial progress is enabled ([\#9611](https://github.com/apache/iceberg/pull/9611)) + - Fix issue when partitioning by UUID ([\#8250](https://github.com/apache/iceberg/pull/8250)) + - Use bulk deletes in rewrite manifests action ([\#10343](https://github.com/apache/iceberg/pull/10343)) +* Flink + - Remove Flink 1.16 support ([\#10154](https://github.com/apache/iceberg/pull/10154)) + - Add support for Flink 1.19 ([\#10112](https://github.com/apache/iceberg/pull/10112)) + - Apply DeleteGranularity for writes ([\#10200](https://github.com/apache/iceberg/pull/10200)) + - Move ParquetReader to LogicalTypeAnnotationVisitor ([\#9719](https://github.com/apache/iceberg/pull/9719)) + - Pre-create fieldGetters to avoid constructing them for each row ([\#10565](https://github.com/apache/iceberg/pull/10565)) + - Prevent setting endTag/endSnapshotId for streaming source ([\#10207](https://github.com/apache/iceberg/pull/10207)) + - Implement range partitioner for map data statistics ([\#9321](https://github.com/apache/iceberg/pull/9321)) + - refactor sink shuffling statistics collection ([\#10331](https://github.com/apache/iceberg/pull/10331)) +* Hive + - Fix metadata file not found ([\#10069](https://github.com/apache/iceberg/pull/10069)) + - Use base table metadata to create HiveLock ([\#10016](https://github.com/apache/iceberg/pull/10016)) + - Turn off the stats gathering when iceberg.hive.keep.stats is false ([\#10148](https://github.com/apache/iceberg/pull/10148)) +* Specs + - OpenAPI: TableRequirements should use union of subclasses ([\#10434](https://github.com/apache/iceberg/pull/10434)) + - OpenAPI: Deprecate oauth/tokens endpoint ([\#10603](https://github.com/apache/iceberg/pull/10603)) + - OpenAPI: Fix additionalProperties for SnapshotSummary ([\#9838](https://github.com/apache/iceberg/pull/9838)) + - OpenAPI: Fix property names for stats/partition stats ([\#10662](https://github.com/apache/iceberg/pull/10662)) +* Vendor Integrations + - AWS: Make sure Signer + User Agent config are both applied ([\#10198](https://github.com/apache/iceberg/pull/10198)) + - AWS: Retain Glue Catalog column comment after updating Iceberg table ([\#10276](https://github.com/apache/iceberg/pull/10276)) + - AWS: Retain Glue Catalog table description after updating Iceberg table ([\#10199](https://github.com/apache/iceberg/pull/10199)) + - AWS: Support S3 DSSE-KMS encryption ([\#8370](https://github.com/apache/iceberg/pull/8370)) + - AWS: Close underlying executor for DynamoDb LockManager ([\#10132](https://github.com/apache/iceberg/pull/10132)) + - AWS: Add Iceberg version to UserAgent in S3 requests ([\#9963](https://github.com/apache/iceberg/pull/9963)) + - Azure: Make AzureProperties w/ shared-key creds serializable ([\#10045](https://github.com/apache/iceberg/pull/10045)) +* Dependencies + - Bump Nessie to 0.92.1 + - Bump Spark 3.5 to 3.5.1 + - Bump Apache Arrow to 15.0.2 + - Bump Azure SDK to 1.2.25 + - Bump Kryo to 4.0.3 + - Bump Netty to 4.1.111.Final + - Bump Jetty to 9.4.55.v20240627 + - Bump Kafka to 3.7.1 + - Bump Apache ORC to 1.9.3 + - Bump AWS SDK to 2.26.12 + - Bump Google Cloud Libraries to 26.43.0 + +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. @@ -161,8 +255,6 @@ The 1.5.0 release adds a variety of new features and bug fixes. For more details, please visit [Github](https://github.com/apache/iceberg/releases/tag/apache-iceberg-1.5.0). -## Past releases - ### 1.4.3 Release Apache Iceberg 1.4.3 was released on December 27, 2023. The main issue it solves is missing files from a transaction retry with conflicting manifests. It is recommended to upgrade if you use transactions. From 774eb00dfd97b4af65173baf90950084e14f4bcf Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 25 Jul 2024 16:54:41 +0200 Subject: [PATCH 0515/1019] Update iceberg version on site to 1.6.0 (#10783) --- site/mkdocs.yml | 4 ++-- site/nav.yml | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 0642281a7c24..3c6782bc850a 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,8 +78,8 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.5.2' - nessieVersion: '0.77.1' + icebergVersion: '1.6.0' + nessieVersion: '0.92.1' flinkVersion: '1.19.0' flinkVersionMajor: '1.19' social: diff --git a/site/nav.yml b/site/nav.yml index 3fe17a94953f..91f1494a1065 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.6.0: '!include docs/docs/1.6.0/mkdocs.yml' - 1.5.2: '!include docs/docs/1.5.2/mkdocs.yml' - 1.5.1: '!include docs/docs/1.5.1/mkdocs.yml' - 1.5.0: '!include docs/docs/1.5.0/mkdocs.yml' From b853f7ad3ea16c54d794b0d30395b88f61bbe2fa Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Thu, 25 Jul 2024 19:50:19 +0200 Subject: [PATCH 0516/1019] Hive: close the fileIO client when closing the hive catalog (#10771) Co-authored-by: Amogh Jahagirdar Co-authored-by: Eduard Tudenhoefner --- .../org/apache/iceberg/hive/HiveCatalog.java | 33 ++++++++++++++++++- 1 file changed, 32 insertions(+), 1 deletion(-) 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 b4f49e29fc49..8944cf93947b 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,6 +18,10 @@ */ package org.apache.iceberg.hive; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalListener; +import java.io.IOException; import java.util.List; import java.util.Map; import java.util.Set; @@ -79,6 +83,7 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa private ClientPool clients; private boolean listAllTables = false; private Map catalogProperties; + private Cache fileIOCloser; public HiveCatalog() {} @@ -111,6 +116,20 @@ public void initialize(String inputName, Map properties) { : CatalogUtil.loadFileIO(fileIOImpl, properties, conf); this.clients = new CachedClientPool(conf, properties); + this.fileIOCloser = newFileIOCloser(); + } + + private Cache newFileIOCloser() { + return Caffeine.newBuilder() + .weakKeys() + .removalListener( + (RemovalListener) + (ops, fileIOInstance, cause) -> { + if (null != fileIOInstance) { + fileIOInstance.close(); + } + }) + .build(); } @Override @@ -512,7 +531,10 @@ private boolean isValidateNamespace(Namespace namespace) { public TableOperations newTableOps(TableIdentifier tableIdentifier) { String dbName = tableIdentifier.namespace().level(0); String tableName = tableIdentifier.name(); - return new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); + HiveTableOperations ops = + new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); + fileIOCloser.put(ops, ops.io()); + return ops; } @Override @@ -636,6 +658,15 @@ protected Map properties() { return catalogProperties == null ? ImmutableMap.of() : catalogProperties; } + @Override + public void close() throws IOException { + super.close(); + if (fileIOCloser != null) { + fileIOCloser.invalidateAll(); + fileIOCloser.cleanUp(); + } + } + @VisibleForTesting void setListAllTables(boolean listAllTables) { this.listAllTables = listAllTables; From c32d9e108cfcd19d14a2a37a57d924df4462efcc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 25 Jul 2024 21:12:55 +0200 Subject: [PATCH 0517/1019] Infra: Add jbonofre as collaborator on the project (#10782) --- .asf.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/.asf.yaml b/.asf.yaml index 83672d9a83ad..bf45b54a1787 100644 --- a/.asf.yaml +++ b/.asf.yaml @@ -54,6 +54,7 @@ github: - gaborkaszab - bitsondatadev - ajantha-bhat + - jbonofre ghp_branch: gh-pages ghp_path: / From 42d16bbf0e71c50751a9b201248a2dc21c5f6a0c Mon Sep 17 00:00:00 2001 From: emkornfield Date: Thu, 25 Jul 2024 18:49:06 -0700 Subject: [PATCH 0518/1019] Docs: Make compatibility example consistent (#10781) Co-authored-by: Amogh Jahagirdar --- site/docs/contribute.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 17099c56bd79..88a14e7153b4 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -196,6 +196,8 @@ public interface ManageSnapshots extends PendingUpdate { // existing code... // adding this method introduces an API-breaking change + // since existing classes implementing ManageSnapshots + // will no longer compile. ManageSnapshots createBranch(String name); } ``` @@ -207,8 +209,8 @@ public class SnapshotManager implements ManageSnapshots { // existing code... @Override - public ManageSnapshots createBranch(String name, long snapshotId) { - updateSnapshotReferencesOperation().createBranch(name, snapshotId); + public ManageSnapshots createBranch(String name) { + updateSnapshotReferencesOperation().createBranch(name); return this; } } From c2628302caef90db4f716a34de46166b7b6f4db1 Mon Sep 17 00:00:00 2001 From: liu yang Date: Fri, 26 Jul 2024 21:30:00 +0800 Subject: [PATCH 0519/1019] =?UTF-8?q?mr=EF=BC=9AFix=20ugi=20not=20correct?= =?UTF-8?q?=20in=20WORKER=5FPOOL=20(#10661)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../mr/mapreduce/IcebergInputFormat.java | 18 ++++++++ .../iceberg/mr/TestIcebergInputFormats.java | 44 +++++++++++++++++++ 2 files changed, 62 insertions(+) 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 a95454b8b0ee..a222080d71db 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 @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.function.BiFunction; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputFormat; @@ -44,6 +45,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.SerializableTable; import org.apache.iceberg.StructLike; +import org.apache.iceberg.SystemConfigs; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.TableScan; @@ -78,6 +80,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.SerializationUtil; +import org.apache.iceberg.util.ThreadPools; /** * Generic Mrv2 InputFormat API for Iceberg. @@ -104,7 +107,21 @@ public List getSplits(JobContext context) { Optional.ofNullable( HiveIcebergStorageHandler.table(conf, conf.get(InputFormatConfig.TABLE_IDENTIFIER))) .orElseGet(() -> Catalogs.loadTable(conf)); + final ExecutorService workerPool = + ThreadPools.newWorkerPool( + "iceberg-plan-worker-pool", + conf.getInt( + SystemConfigs.WORKER_THREAD_POOL_SIZE.propertyKey(), + ThreadPools.WORKER_THREAD_POOL_SIZE)); + try { + return planInputSplits(table, conf, workerPool); + } finally { + workerPool.shutdown(); + } + } + private List planInputSplits( + Table table, Configuration conf, ExecutorService workerPool) { TableScan scan = table .newScan() @@ -144,6 +161,7 @@ public List getSplits(JobContext context) { InputFormatConfig.InMemoryDataModel model = conf.getEnum( InputFormatConfig.IN_MEMORY_DATA_MODEL, InputFormatConfig.InMemoryDataModel.GENERIC); + scan = scan.planWith(workerPool); try (CloseableIterable tasksIterable = scan.planTasks()) { Table serializableTable = SerializableTable.copyOf(table); tasksIterable.forEach( 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 c9d5d487de38..86d390ca9ffe 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java @@ -25,11 +25,14 @@ import java.io.File; import java.io.IOException; import java.io.UncheckedIOException; +import java.lang.reflect.Method; import java.nio.file.Path; import java.nio.file.Paths; +import java.security.PrivilegedAction; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.concurrent.ExecutorService; import java.util.function.Function; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapred.JobConf; @@ -39,6 +42,7 @@ import org.apache.hadoop.mapreduce.RecordReader; import org.apache.hadoop.mapreduce.TaskAttemptContext; import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl; +import org.apache.hadoop.security.UserGroupInformation; import org.apache.iceberg.AppendFiles; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; @@ -67,6 +71,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ThreadPools; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -381,6 +386,45 @@ public void testCustomCatalog() throws IOException { testInputFormat.create(builder.conf()).validate(expectedRecords); } + @TestTemplate + public void testWorkerPool() throws Exception { + Table table = helper.createUnpartitionedTable(); + UserGroupInformation user1 = + UserGroupInformation.createUserForTesting("user1", new String[] {}); + UserGroupInformation user2 = + UserGroupInformation.createUserForTesting("user2", new String[] {}); + final ExecutorService workerPool1 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); + final ExecutorService workerPool2 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); + try { + assertThat(getUserFromWorkerPool(user1, table, workerPool1)).isEqualTo("user1"); + assertThat(getUserFromWorkerPool(user2, table, workerPool1)).isEqualTo("user1"); + assertThat(getUserFromWorkerPool(user2, table, workerPool2)).isEqualTo("user2"); + } finally { + workerPool1.shutdown(); + workerPool2.shutdown(); + } + } + + private String getUserFromWorkerPool( + UserGroupInformation user, Table table, ExecutorService workerpool) throws Exception { + Method method = + IcebergInputFormat.class.getDeclaredMethod( + "planInputSplits", Table.class, Configuration.class, ExecutorService.class); + method.setAccessible(true); + return user.doAs( + (PrivilegedAction) + () -> { + try { + method.invoke(new IcebergInputFormat<>(), table, conf, workerpool); + return workerpool + .submit(() -> UserGroupInformation.getCurrentUser().getUserName()) + .get(); + } catch (Exception e) { + throw new RuntimeException("Failed to get user from worker pool", e); + } + }); + } + // TODO - Capture template type T in toString method: // https://github.com/apache/iceberg/issues/1542 public abstract static class TestInputFormat { From 9d23bb0d4690bfe24e01c014c67aaa0a38bbaf36 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 26 Jul 2024 09:13:18 -0700 Subject: [PATCH 0520/1019] Flink: backport PR #10331 and PR #10457 (#10757) --- flink/v1.17/build.gradle | 2 + .../shuffle/MapRangePartitionerBenchmark.java | 10 +- .../sink/shuffle/AggregatedStatistics.java | 71 --- .../shuffle/AggregatedStatisticsTracker.java | 266 ++++++--- .../sink/shuffle/CompletedStatistics.java | 103 ++++ .../CompletedStatisticsSerializer.java | 178 ++++++ .../flink/sink/shuffle/DataStatistics.java | 25 +- .../shuffle/DataStatisticsCoordinator.java | 219 +++++-- .../DataStatisticsCoordinatorProvider.java | 33 +- .../sink/shuffle/DataStatisticsOperator.java | 182 ++++-- .../shuffle/DataStatisticsSerializer.java | 206 +++++++ .../sink/shuffle/DataStatisticsUtil.java | 96 --- .../flink/sink/shuffle/GlobalStatistics.java | 114 ++++ .../shuffle/GlobalStatisticsSerializer.java | 201 +++++++ .../flink/sink/shuffle/KeyAssignment.java | 151 +++++ .../flink/sink/shuffle/MapAssignment.java | 240 ++++++++ .../flink/sink/shuffle/MapDataStatistics.java | 52 +- .../shuffle/MapDataStatisticsSerializer.java | 187 ------ .../sink/shuffle/MapRangePartitioner.java | 322 +--------- .../flink/sink/shuffle/RangePartitioner.java | 110 ++++ .../shuffle/RequestGlobalStatisticsEvent.java | 38 ++ .../sink/shuffle/SketchDataStatistics.java | 87 +++ .../sink/shuffle/SketchRangePartitioner.java | 64 ++ .../flink/sink/shuffle/SketchUtil.java | 142 +++++ .../flink/sink/shuffle/SortKeySerializer.java | 13 +- .../sink/shuffle/SortKeySketchSerializer.java | 143 +++++ ...tisticsEvent.java => StatisticsEvent.java} | 33 +- .../sink/shuffle/StatisticsOrRecord.java} | 43 +- .../StatisticsOrRecordSerializer.java} | 101 ++-- .../flink/sink/shuffle/StatisticsType.java | 55 ++ .../flink/sink/shuffle/StatisticsUtil.java | 126 ++++ .../iceberg/flink/sink/shuffle/Fixtures.java | 100 ++++ .../shuffle/TestAggregatedStatistics.java | 63 -- .../TestAggregatedStatisticsTracker.java | 563 +++++++++++++----- .../TestCompletedStatisticsSerializer.java | 54 ++ .../TestDataStatisticsCoordinator.java | 266 ++++++--- ...TestDataStatisticsCoordinatorProvider.java | 190 +++--- .../shuffle/TestDataStatisticsOperator.java | 357 +++++++---- .../shuffle/TestDataStatisticsSerializer.java | 53 ++ .../TestGlobalStatisticsSerializer.java | 59 ++ .../sink/shuffle/TestMapDataStatistics.java | 62 +- .../sink/shuffle/TestMapRangePartitioner.java | 241 ++++---- .../shuffle/TestSketchDataStatistics.java | 60 ++ .../flink/sink/shuffle/TestSketchUtil.java | 133 +++++ .../TestSortKeySerializerPrimitives.java | 33 + flink/v1.18/build.gradle | 2 + .../shuffle/MapRangePartitionerBenchmark.java | 10 +- .../sink/shuffle/AggregatedStatistics.java | 71 --- .../shuffle/AggregatedStatisticsTracker.java | 266 ++++++--- .../sink/shuffle/CompletedStatistics.java | 103 ++++ .../CompletedStatisticsSerializer.java | 178 ++++++ .../flink/sink/shuffle/DataStatistics.java | 25 +- .../shuffle/DataStatisticsCoordinator.java | 219 +++++-- .../DataStatisticsCoordinatorProvider.java | 33 +- .../sink/shuffle/DataStatisticsOperator.java | 182 ++++-- .../shuffle/DataStatisticsSerializer.java | 206 +++++++ .../sink/shuffle/DataStatisticsUtil.java | 96 --- .../flink/sink/shuffle/GlobalStatistics.java | 114 ++++ .../shuffle/GlobalStatisticsSerializer.java | 201 +++++++ .../flink/sink/shuffle/KeyAssignment.java | 151 +++++ .../flink/sink/shuffle/MapAssignment.java | 240 ++++++++ .../flink/sink/shuffle/MapDataStatistics.java | 52 +- .../shuffle/MapDataStatisticsSerializer.java | 187 ------ .../sink/shuffle/MapRangePartitioner.java | 322 +--------- .../flink/sink/shuffle/RangePartitioner.java | 110 ++++ .../shuffle/RequestGlobalStatisticsEvent.java | 38 ++ .../sink/shuffle/SketchDataStatistics.java | 87 +++ .../sink/shuffle/SketchRangePartitioner.java | 64 ++ .../flink/sink/shuffle/SketchUtil.java | 142 +++++ .../flink/sink/shuffle/SortKeySerializer.java | 13 +- .../sink/shuffle/SortKeySketchSerializer.java | 143 +++++ ...tisticsEvent.java => StatisticsEvent.java} | 33 +- .../sink/shuffle/StatisticsOrRecord.java} | 43 +- .../StatisticsOrRecordSerializer.java} | 101 ++-- .../flink/sink/shuffle/StatisticsType.java | 55 ++ .../flink/sink/shuffle/StatisticsUtil.java | 126 ++++ .../iceberg/flink/sink/shuffle/Fixtures.java | 100 ++++ .../shuffle/TestAggregatedStatistics.java | 63 -- .../TestAggregatedStatisticsTracker.java | 563 +++++++++++++----- .../TestCompletedStatisticsSerializer.java | 54 ++ .../TestDataStatisticsCoordinator.java | 266 ++++++--- ...TestDataStatisticsCoordinatorProvider.java | 190 +++--- .../shuffle/TestDataStatisticsOperator.java | 357 +++++++---- .../shuffle/TestDataStatisticsSerializer.java | 53 ++ .../TestGlobalStatisticsSerializer.java | 59 ++ .../sink/shuffle/TestMapDataStatistics.java | 62 +- .../sink/shuffle/TestMapRangePartitioner.java | 241 ++++---- .../shuffle/TestSketchDataStatistics.java | 60 ++ .../flink/sink/shuffle/TestSketchUtil.java | 133 +++++ .../TestSortKeySerializerPrimitives.java | 33 + 90 files changed, 8396 insertions(+), 3298 deletions(-) delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java rename flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{DataStatisticsEvent.java => StatisticsEvent.java} (58%) rename flink/{v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java => v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java} (66%) rename flink/{v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java => v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java} (53%) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java delete mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java delete mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java delete mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java rename flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/{DataStatisticsEvent.java => StatisticsEvent.java} (58%) rename flink/{v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java => v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java} (66%) rename flink/{v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java => v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java} (53%) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle index 2ced7a5a5cb7..0278e4dc3b73 100644 --- a/flink/v1.17/build.gradle +++ b/flink/v1.17/build.gradle @@ -66,6 +66,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } + implementation libs.datasketches + testImplementation libs.flink117.connector.test.utils testImplementation libs.flink117.core testImplementation libs.flink117.runtime diff --git a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 3b2c74fd6ece..a9ad386a5a4a 100644 --- a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.nio.charset.StandardCharsets; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -28,6 +29,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; 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; @@ -67,6 +70,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -83,10 +88,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java deleted file mode 100644 index 157f04b8b0ed..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.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.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. - */ -class AggregatedStatistics, S> implements Serializable { - - private final long checkpointId; - private final DataStatistics dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } - - AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; - } - - DataStatistics dataStatistics() { - return dataStatistics; - } - - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) - .toString(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index e8ff61dbeb27..338523b7b074 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -18,116 +18,238 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import java.util.NavigableMap; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.datasketches.sampling.ReservoirItemsUnion; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. */ -class AggregatedStatisticsTracker, S> { +class AggregatedStatisticsTracker { private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; + private final String operatorName; - private final TypeSerializer> statisticsSerializer; private final int parallelism; - private final Set inProgressSubtaskSet; - private volatile AggregatedStatistics inProgressStatistics; + private final TypeSerializer statisticsSerializer; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final int switchToSketchThreshold; + private final NavigableMap aggregationsPerCheckpoint; + + private CompletedStatistics completedStatistics; AggregatedStatisticsTracker( String operatorName, - TypeSerializer> statisticsSerializer, - int parallelism) { + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable CompletedStatistics restoredStatistics) { this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); + this.statisticsSerializer = + new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.switchToSketchThreshold = switchToSketchThreshold; + this.completedStatistics = restoredStatistics; + + this.aggregationsPerCheckpoint = Maps.newTreeMap(); } - AggregatedStatistics updateAndCheckCompletion( - int subtask, DataStatisticsEvent event) { + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { long checkpointId = event.checkpointId(); + LOG.debug( + "Handling statistics event from subtask {} of operator {} for checkpoint {}", + subtask, + operatorName, + checkpointId); - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { + if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " + + "Was expecting data statistics from checkpoint higher than {}", operatorName, - inProgressStatistics.checkpointId(), - checkpointId); + subtask, + checkpointId, + completedStatistics.checkpointId()); return null; } - AggregatedStatistics completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, + Aggregation aggregation = + aggregationsPerCheckpoint.computeIfAbsent( checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); + ignored -> + new Aggregation( + parallelism, + downstreamParallelism, + switchToSketchThreshold, + statisticsType, + StatisticsUtil.collectType(statisticsType, completedStatistics))); + DataStatistics dataStatistics = + StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); + if (!aggregation.merge(subtask, dataStatistics)) { + LOG.debug( + "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } + + if (aggregation.isComplete()) { + this.completedStatistics = aggregation.completedStatistics(checkpointId); + // clean up aggregations up to the completed checkpoint id + aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); + return completedStatistics; + } + + return null; + } + + @VisibleForTesting + NavigableMap aggregationsPerCheckpoint() { + return aggregationsPerCheckpoint; + } + + static class Aggregation { + private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); + + private final Set subtaskSet; + private final int parallelism; + private final int downstreamParallelism; + private final int switchToSketchThreshold; + private final StatisticsType configuredType; + private StatisticsType currentType; + private Map mapStatistics; + private ReservoirItemsUnion sketchStatistics; + + Aggregation( + int parallelism, + int downstreamParallelism, + int switchToSketchThreshold, + StatisticsType configuredType, + StatisticsType currentType) { + this.subtaskSet = Sets.newHashSet(); + this.parallelism = parallelism; + this.downstreamParallelism = downstreamParallelism; + this.switchToSketchThreshold = switchToSketchThreshold; + this.configuredType = configuredType; + this.currentType = currentType; + + if (currentType == StatisticsType.Map) { + this.mapStatistics = Maps.newHashMap(); + this.sketchStatistics = null; } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); + this.mapStatistics = null; + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); } + } - inProgressStatistics = null; - inProgressSubtaskSet.clear(); + @VisibleForTesting + Set subtaskSet() { + return subtaskSet; } - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + StatisticsType currentType() { + return currentType; } - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; } - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + ReservoirItemsUnion sketchStatistics() { + return sketchStatistics; } - return completedStatistics; - } + private boolean isComplete() { + return subtaskSet.size() == parallelism; + } - @VisibleForTesting - AggregatedStatistics inProgressStatistics() { - return inProgressStatistics; + /** @return false if duplicate */ + private boolean merge(int subtask, DataStatistics taskStatistics) { + if (subtaskSet.contains(subtask)) { + return false; + } + + subtaskSet.add(subtask); + merge(taskStatistics); + return true; + } + + @SuppressWarnings("unchecked") + private void merge(DataStatistics taskStatistics) { + if (taskStatistics.type() == StatisticsType.Map) { + Map taskMapStats = (Map) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); + if (configuredType == StatisticsType.Auto + && mapStatistics.size() > switchToSketchThreshold) { + convertCoordinatorToSketch(); + } + } else { + // convert task stats to sketch first + ReservoirItemsSketch taskSketch = + ReservoirItemsSketch.newInstance( + SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); + SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); + sketchStatistics.update(taskSketch); + } + } else { + ReservoirItemsSketch taskSketch = + (ReservoirItemsSketch) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + // convert global stats to sketch first + convertCoordinatorToSketch(); + } + + sketchStatistics.update(taskSketch); + } + } + + private void convertCoordinatorToSketch() { + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); + this.currentType = StatisticsType.Sketch; + this.mapStatistics = null; + } + + private CompletedStatistics completedStatistics(long checkpointId) { + if (currentType == StatisticsType.Map) { + LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + } else { + ReservoirItemsSketch sketch = sketchStatistics.getResult(); + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } + } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java new file mode 100644 index 000000000000..c0e228965ddd --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.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.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). + */ +class CompletedStatistics { + private final long checkpointId; + private final StatisticsType type; + private final Map keyFrequency; + private final SortKey[] keySamples; + + static CompletedStatistics fromKeyFrequency(long checkpointId, Map stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( + long checkpointId, + StatisticsType type, + Map keyFrequency, + SortKey[] keySamples) { + this.checkpointId = checkpointId; + this.type = type; + this.keyFrequency = keyFrequency; + this.keySamples = keySamples; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("keyFrequency", keyFrequency) + .add("keySamples", keySamples) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof CompletedStatistics)) { + return false; + } + + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(keyFrequency, other.keyFrequency()) + && Arrays.equals(keySamples, other.keySamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + Map keyFrequency() { + return keyFrequency; + } + + SortKey[] keySamples() { + return keySamples; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java new file mode 100644 index 000000000000..7f55188e7f8c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -0,0 +1,178 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; + +class CompletedStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer keyFrequencySerializer; + private final ListSerializer keySamplesSerializer; + + CompletedStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); + } + + @Override + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + keyFrequencySerializer.serialize(record.keyFrequency(), target); + } else { + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); + } + } + + @Override + public CompletedStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + Map keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); + } else { + List sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); + } + } + + @Override + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); + } + + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public CompletedStatisticsSerializerSnapshot() { + super(CompletedStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + CompletedStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new CompletedStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 9d7cf179ab1c..76c59cd5f4b8 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; @@ -29,29 +31,18 @@ * (sketching) can be used. */ @Internal -interface DataStatistics, S> { +interface DataStatistics { + + StatisticsType type(); - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ boolean isEmpty(); /** Add row sortKey to data statistics. */ void add(SortKey sortKey); /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics + * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link + * ReservoirItemsSketch} (high cardinality) */ - S statistics(); + Object result(); } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index c8ac79c61bf6..3b21fbae315a 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -35,6 +36,10 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.ThrowableCatchingRunnable; import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -44,51 +49,86 @@ import org.slf4j.LoggerFactory; /** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. + * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} + * every subtask and then merge them together. Once aggregation for all subtasks data statistics + * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link + * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the + * aggregated data statistics to improve data clustering. */ @Internal -class DataStatisticsCoordinator, S> implements OperatorCoordinator { +class DataStatisticsCoordinator implements OperatorCoordinator { private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); private final String operatorName; + private final OperatorCoordinator.Context context; + private final Schema schema; + private final SortOrder sortOrder; + private final Comparator comparator; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; + private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer> statisticsSerializer; - private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private volatile AggregatedStatistics completedStatistics; - private volatile boolean started; + private final TypeSerializer completedStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; + + private transient boolean started; + private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; DataStatisticsCoordinator( String operatorName, OperatorCoordinator.Context context, - TypeSerializer> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { this.operatorName = operatorName; + this.context = context; + this.schema = schema; + this.sortOrder = sortOrder; + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; + this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker( + operatorName, + context.currentParallelism(), + schema, + sortOrder, + downstreamParallelism, + statisticsType, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + completedStatistics); } @Override public void close() throws Exception { coordinatorExecutor.shutdown(); + this.aggregatedStatisticsTracker = null; + this.started = false; LOG.info("Closed data statistics coordinator: {}.", operatorName); } @@ -148,7 +188,7 @@ private void runInCoordinatorThread(ThrowingRunnable action, String a operatorName, actionString, t); - operatorCoordinatorContext.failJob(t); + context.failJob(t); } }); } @@ -157,42 +197,102 @@ private void ensureStarted() { Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); } - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = + private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { + CompletedStatistics maybeCompletedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + if (maybeCompletedStatistics != null) { + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics globalDataStatistics) { - callInCoordinatorThread( + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( () -> { - DataStatisticsEvent dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); + for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference + subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } - - return null; }, String.format( "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } } @Override - @SuppressWarnings("unchecked") public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { runInCoordinatorThread( () -> { @@ -202,8 +302,14 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -219,8 +325,8 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r operatorName, checkpointId); resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); }, String.format("taking checkpoint %d", checkpointId)); } @@ -229,11 +335,9 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r public void notifyCheckpointComplete(long checkpointId) {} @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", @@ -244,8 +348,13 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + this.completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); } @Override @@ -269,7 +378,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", subtask, attemptNumber, operatorName); @@ -295,14 +404,20 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics completedStatistics() { + CompletedStatistics completedStatistics() { return completedStatistics; } + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + private static class SubtaskGateways { private final String operatorName; private final Map[] gateways; + @SuppressWarnings("unchecked") private SubtaskGateways(String operatorName, int parallelism) { this.operatorName = operatorName; gateways = new Map[parallelism]; diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java index 47dbfc3cfbe1..9d7d989c298e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -19,33 +19,52 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; /** * DataStatisticsCoordinatorProvider provides the method to create new {@link * DataStatisticsCoordinator} */ @Internal -public class DataStatisticsCoordinatorProvider, S> - extends RecreateOnResetOperatorCoordinator.Provider { +public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { private final String operatorName; - private final TypeSerializer> statisticsSerializer; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; public DataStatisticsCoordinatorProvider( String operatorName, OperatorID operatorID, - TypeSerializer> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { super(operatorID); this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + return new DataStatisticsCoordinator( + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 5157a37cf2cd..59c38b239725 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -47,9 +48,8 @@ * distribution to downstream subtasks. */ @Internal -class DataStatisticsOperator, S> - extends AbstractStreamOperator> - implements OneInputStreamOperator>, OperatorEventHandler { +public class DataStatisticsOperator extends AbstractStreamOperator + implements OneInputStreamOperator, OperatorEventHandler { private static final long serialVersionUID = 1L; @@ -57,141 +57,209 @@ class DataStatisticsOperator, S> private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer> statisticsSerializer; - private transient volatile DataStatistics localStatistics; - private transient volatile DataStatistics globalStatistics; - private transient ListState> globalStatisticsState; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final TypeSerializer taskStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; + + private transient int parallelism; + private transient int subtaskIndex; + private transient ListState globalStatisticsState; + // current statistics type may be different from the config due to possible + // migration from Map statistics to Sketch statistics when high cardinality detected + private transient volatile StatisticsType taskStatisticsType; + private transient volatile DataStatistics localStatistics; + private transient volatile GlobalStatistics globalStatistics; DataStatisticsOperator( String operatorName, Schema schema, SortOrder sortOrder, OperatorEventGateway operatorEventGateway, - TypeSerializer> statisticsSerializer) { + int downstreamParallelism, + StatisticsType statisticsType) { this.operatorName = operatorName; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = + this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); + this.subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + + // Use union state so that new subtasks can also restore global statistics during scale-up. + this.globalStatisticsState = context .getOperatorStateStore() .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); if (globalStatisticsState.get() == null || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( + LOG.info( "Operator {} subtask {} doesn't have global statistics state to restore", operatorName, subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; } - } else { - globalStatistics = statisticsSerializer.createInstance(); + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); } + + this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); + this.localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); } @Override public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } } @Override - @SuppressWarnings("unchecked") public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); Preconditions.checkArgument( - event instanceof DataStatisticsEvent, + event instanceof StatisticsEvent, String.format( "Operator %s subtask %s received unexpected operator event %s", operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; + StatisticsEvent statisticsEvent = (StatisticsEvent) event; LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", + "Operator {} subtask {} received global data event from coordinator checkpoint {}", operatorName, + subtaskIndex, statisticsEvent.checkpointId()); - globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); + checkStatisticsTypeMigration(); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } } @Override public void processElement(StreamRecord streamRecord) { + // collect data statistics RowData record = streamRecord.getValue(); StructLike struct = rowDataWrapper.wrap(record); sortKey.wrap(struct); localStatistics.add(sortKey); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); + + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", + "Operator {} subtask {} snapshotting data statistics for checkpoint {}", operatorName, - checkpointId, - subTaskId); + subtaskIndex, + checkpointId); - // Pass global statistics to partitioners so that all the operators refresh statistics + // Pass global statistics to partitioner so that all the operators refresh statistics // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { + if (globalStatistics != null && getRuntimeContext().getIndexOfThisSubtask() == 0) { globalStatisticsState.clear(); LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); + "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); globalStatisticsState.add(globalStatistics); + LOG.debug( + "Operator {} subtask {} saved global statistics to state: {}", + operatorName, + subtaskIndex, + globalStatistics); } // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, + LOG.info( + "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", operatorName, - checkpointId, - localStatistics); + subtaskIndex, + checkpointId); + operatorEventGateway.sendEventToCoordinator( + StatisticsEvent.createTaskStatisticsEvent( + checkpointId, localStatistics, taskStatisticsSerializer)); // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); + localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @SuppressWarnings("unchecked") + private void checkStatisticsTypeMigration() { + // only check if the statisticsType config is Auto and localStatistics is currently Map type + if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { + Map mapStatistics = (Map) localStatistics.result(); + // convert if local statistics has cardinality over the threshold or + // if received global statistics is already sketch type + if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { + LOG.info( + "Operator {} subtask {} switched local statistics from Map to Sketch.", + operatorName, + subtaskIndex); + this.taskStatisticsType = StatisticsType.Sketch; + this.localStatistics = + StatisticsUtil.createTaskStatistics( + taskStatisticsType, parallelism, downstreamParallelism); + SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); + } + } } @VisibleForTesting - DataStatistics localDataStatistics() { + DataStatistics localStatistics() { return localStatistics; } @VisibleForTesting - DataStatistics globalDataStatistics() { + GlobalStatistics globalStatistics() { return globalStatistics; } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java new file mode 100644 index 000000000000..c25481b3c1f2 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class DataStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer mapSerializer; + private final SortKeySketchSerializer sketchSerializer; + + DataStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer duplicate() { + TypeSerializer duplicateSortKeySerializer = sortKeySerializer.duplicate(); + return (duplicateSortKeySerializer == sortKeySerializer) + ? this + : new DataStatisticsSerializer(duplicateSortKeySerializer); + } + + @Override + public DataStatistics createInstance() { + return new MapDataStatistics(); + } + + @SuppressWarnings("unchecked") + @Override + public DataStatistics copy(DataStatistics obj) { + StatisticsType statisticsType = obj.type(); + if (statisticsType == StatisticsType.Map) { + MapDataStatistics from = (MapDataStatistics) obj; + Map fromStats = (Map) from.result(); + Map toStats = Maps.newHashMap(fromStats); + return new MapDataStatistics(toStats); + } else if (statisticsType == StatisticsType.Sketch) { + // because ReservoirItemsSketch doesn't expose enough public methods for cloning, + // this implementation adopted the less efficient serialization and deserialization. + SketchDataStatistics from = (SketchDataStatistics) obj; + ReservoirItemsSketch fromStats = (ReservoirItemsSketch) from.result(); + byte[] bytes = fromStats.toByteArray(sketchSerializer); + Memory memory = Memory.wrap(bytes); + ReservoirItemsSketch toStats = + ReservoirItemsSketch.heapify(memory, sketchSerializer); + return new SketchDataStatistics(toStats); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics copy(DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @SuppressWarnings("unchecked") + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + StatisticsType statisticsType = obj.type(); + statisticsTypeSerializer.serialize(obj.type(), target); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = (Map) obj.result(); + mapSerializer.serialize(mapStatistics, target); + } else if (statisticsType == StatisticsType.Sketch) { + ReservoirItemsSketch sketch = (ReservoirItemsSketch) obj.result(); + byte[] sketchBytes = sketch.toByteArray(sketchSerializer); + target.writeInt(sketchBytes.length); + target.write(sketchBytes); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataInputView source) throws IOException { + StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = mapSerializer.deserialize(source); + return new MapDataStatistics(mapStatistics); + } else if (statisticsType == StatisticsType.Sketch) { + int numBytes = source.readInt(); + byte[] sketchBytes = new byte[numBytes]; + source.read(sketchBytes); + Memory sketchMemory = Memory.wrap(sketchBytes); + ReservoirItemsSketch sketch = + ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); + return new SketchDataStatistics(sketch); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsSerializer)) { + return false; + } + + DataStatisticsSerializer other = (DataStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DataStatisticsSerializerSnapshot(this); + } + + public static class DataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DataStatisticsSerializerSnapshot() { + super(DataStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new DataStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 8716cb872d0e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,96 +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.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static , S> byte[] serializeDataStatistics( - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static , S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static , S> byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - static , S> - AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * 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.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..dfb947a84a0c --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,201 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final ListSerializer rangeBoundsSerializer; + private final ListSerializer intsSerializer; + private final ListSerializer longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List assignedSubtasks = intsSerializer.deserialize(source); + List subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() { + super(GlobalStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..a164d83ac3b0 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,151 @@ +/* + * 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.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List assignedSubtasks; + private final List subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List assignedSubtasks() { + return assignedSubtasks; + } + + List subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..0abb030c2279 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.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.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map keyAssignments; + + MapAssignment(int numPartitions, Map keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map keyAssignments() { + return keyAssignments; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map assignment( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 0b63e2721178..05b943f6046f 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -19,52 +19,70 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics { + private final Map keyFrequency; MapDataStatistics() { - this.statistics = Maps.newHashMap(); + this.keyFrequency = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { - this.statistics = statistics; + MapDataStatistics(Map keyFrequency) { + this.keyFrequency = keyFrequency; + } + + @Override + public StatisticsType type() { + return StatisticsType.Map; } @Override public boolean isEmpty() { - return statistics.isEmpty(); + return keyFrequency.isEmpty(); } @Override public void add(SortKey sortKey) { - if (statistics.containsKey(sortKey)) { - statistics.merge(sortKey, 1L, Long::sum); + if (keyFrequency.containsKey(sortKey)) { + keyFrequency.merge(sortKey, 1L, Long::sum); } else { // clone the sort key before adding to map because input sortKey object can be reused SortKey copiedKey = sortKey.copy(); - statistics.put(copiedKey, 1L); + keyFrequency.put(copiedKey, 1L); } } @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + public Object result() { + return keyFrequency; } @Override - public Map statistics() { - return statistics; + public String toString() { + return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); } @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof MapDataStatistics)) { + return false; + } + + MapDataStatistics other = (MapDataStatistics) o; + return Objects.equal(keyFrequency, other.keyFrequency); + } + + @Override + public int hashCode() { + return Objects.hashCode(keyFrequency); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index b6cccd0566fc..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,187 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; - - static MapDataStatisticsSerializer fromSortKeySerializer( - TypeSerializer sortKeySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public MapDataStatistics createInstance() { - return new MapDataStatistics(); - } - - @Override - public MapDataStatistics copy(DataStatistics> obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - SortKey newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics> copy( - DataStatistics> from, - DataStatistics> reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize( - DataStatistics> obj, DataOutputView target) - throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics> deserialize( - DataStatistics> reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index dde86b5b6047..f36a078c94e0 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -18,29 +18,14 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; -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; -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.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,52 +46,28 @@ class MapRangePartitioner implements Partitioner { private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; - private final Comparator comparator; - private final Map mapStatistics; - private final double closeFileCostInWeightPercentage; + private final MapAssignment mapAssignment; // Counter that tracks how many times a new key encountered // where there is no traffic statistics learned about it. private long newSortKeyCounter; private long lastNewSortKeyLogTimeMilli; - // lazily computed due to the need of numPartitions - private Map assignment; - private NavigableMap sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - MapDataStatistics dataStatistics, - double closeFileCostInWeightPercentage) { - dataStatistics - .statistics() - .entrySet() - .forEach( - entry -> - Preconditions.checkArgument( - entry.getValue() > 0, - "Invalid statistics: weight is 0 for key %s", - entry.getKey())); - + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = dataStatistics.statistics(); - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.mapAssignment = mapAssignment; this.newSortKeyCounter = 0; this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); } @Override public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map assignmentMap = assignment(numPartitions); // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; if (keyAssignment == null) { LOG.trace( "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", @@ -117,271 +78,18 @@ public int partition(RowData row, int numPartitions) { newSortKeyCounter += 1; long now = System.currentTimeMillis(); if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + LOG.info( + "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", + newSortKeyCounter); lastNewSortKeyLogTimeMilli = now; + newSortKeyCounter = 0; } - return (int) (newSortKeyCounter % numPartitions); + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); } - return keyAssignment.select(); - } - - @VisibleForTesting - Map assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map mapStatistics() { - return mapStatistics; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map> assignmentInfo() { - Map> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map buildAssignment( - int numPartitions, - NavigableMap sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List assignedSubtasks = Lists.newArrayList(); - List subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List assignedSubtasks, - List subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java new file mode 100644 index 000000000000..83a9461233d2 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.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.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** The wrapper class */ +@Internal +public class RangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. + // when rescale is detected, operator requests new statistics from coordinator upon + // initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.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.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java new file mode 100644 index 000000000000..35bbb27baf16 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java @@ -0,0 +1,87 @@ +/* + * 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.Arrays; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** MapDataStatistics uses map to count key frequency */ +class SketchDataStatistics implements DataStatistics { + + private final ReservoirItemsSketch sketch; + + SketchDataStatistics(int reservoirSize) { + this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); + } + + SketchDataStatistics(ReservoirItemsSketch sketchStats) { + this.sketch = sketchStats; + } + + @Override + public StatisticsType type() { + return StatisticsType.Sketch; + } + + @Override + public boolean isEmpty() { + return sketch.getNumSamples() == 0; + } + + @Override + public void add(SortKey sortKey) { + // clone the sort key first because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + sketch.update(copiedKey); + } + + @Override + public Object result() { + return sketch; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SketchDataStatistics)) { + return false; + } + + ReservoirItemsSketch otherSketch = ((SketchDataStatistics) o).sketch; + return Objects.equal(sketch.getK(), otherSketch.getK()) + && Objects.equal(sketch.getN(), otherSketch.getN()) + && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java new file mode 100644 index 000000000000..af78271ea5dc --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.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.sink.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +class SketchRangePartitioner implements Partitioner { + private final SortKey sortKey; + private final Comparator comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java new file mode 100644 index 000000000000..a58310611e8d --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -0,0 +1,142 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; + +class SketchUtil { + static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; + static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; + static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; + static final int OPERATOR_OVER_SAMPLE_RATIO = 10; + + // switch the statistics tracking from map to sketch if the cardinality of the sort key is over + // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. + static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; + static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; + + private SketchUtil() {} + + /** + * The larger the reservoir size, the more accurate for range bounds calculation and the more + * balanced range distribution. + * + *

    Here are the heuristic rules + *

  • Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the + * range bounds + *
  • Min is 10K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 1M to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @return reservoir size + */ + static int determineCoordinatorReservoirSize(int numPartitions) { + int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; + + if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { + // adjust it up and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); + } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { + // adjust it down and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; + } + + return reservoirSize; + } + + /** + * Determine the sampling reservoir size where operator subtasks collect data statistics. + * + *

    Here are the heuristic rules + *

  • Target size is "coordinator reservoir size * over sampling ration (10) / operator + * parallelism" + *
  • Min is 1K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 100K to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @param operatorParallelism data statistics operator parallelism + * @return reservoir size + */ + static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { + int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); + int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; + return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); + } + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + * + *

    Assumption is that a single key is not dominant enough to span multiple subtasks. + * + * @param numPartitions number of partitions which maps to downstream operator parallelism + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + */ + static SortKey[] rangeBounds( + int numPartitions, Comparator comparator, SortKey[] samples) { + // sort the keys first + Arrays.sort(samples, comparator); + int numCandidates = numPartitions - 1; + SortKey[] candidates = new SortKey[numCandidates]; + int step = (int) Math.ceil((double) samples.length / numPartitions); + int position = step - 1; + int numChosen = 0; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; + // skip duplicate values + if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { + // linear probe for the next distinct value + position += 1; + } else { + candidates[numChosen] = candidate; + position += step; + numChosen += 1; + } + } + + return candidates; + } + + /** This can be a bit expensive since it is quadratic. */ + static void convertMapToSketch( + Map taskMapStats, Consumer sketchConsumer) { + taskMapStats.forEach( + (sortKey, count) -> { + for (int i = 0; i < count; ++i) { + sketchConsumer.accept(sortKey); + } + }); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index d03409f2a430..4ddc5a32d6bf 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -276,13 +276,12 @@ public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot< private Schema schema; private SortOrder sortOrder; - @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { // this constructor is used when restoring from a checkpoint. } - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". @SuppressWarnings("checkstyle:RedundantModifier") public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { this.schema = schema; @@ -320,8 +319,12 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; - return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + SortKeySerializer sortKeySerializer = (SortKeySerializer) newSerializer; + if (!sortOrder.sameOrder(sortKeySerializer.sortOrder)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + return resolveSchemaCompatibility(sortKeySerializer.schema, schema); } @Override diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java new file mode 100644 index 000000000000..d6c23f035015 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +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.base.Preconditions; + +/** + * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link + * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The + * implementation is modeled after {@link ArrayOfStringsSerDe} + */ +class SortKeySketchSerializer extends ArrayOfItemsSerDe implements Serializable { + private static final int DEFAULT_SORT_KEY_SIZE = 128; + + private final TypeSerializer itemSerializer; + private final ListSerializer listSerializer; + private final DataInputDeserializer input; + + SortKeySketchSerializer(TypeSerializer itemSerializer) { + this.itemSerializer = itemSerializer; + this.listSerializer = new ListSerializer<>(itemSerializer); + this.input = new DataInputDeserializer(); + } + + @Override + public byte[] serializeToByteArray(SortKey item) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); + itemSerializer.serialize(item, output); + byte[] itemBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[numBytes + Integer.BYTES]; + ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); + ByteArrayUtil.putIntLE(out, 0, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public byte[] serializeToByteArray(SortKey[] items) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); + listSerializer.serialize(Arrays.asList(items), output); + byte[] itemsBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[Integer.BYTES + numBytes]; + ByteArrayUtil.putIntLE(out, 0, numBytes); + System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return new SortKey[0]; + } + + long offset = startingOffset; + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + offset += Integer.BYTES; + + Util.checkBounds(offset, numBytes, mem.getCapacity()); + byte[] sortKeyBytes = new byte[numBytes]; + mem.getByteArray(offset, sortKeyBytes, 0, numBytes); + input.setBuffer(sortKeyBytes); + + try { + List sortKeys = listSerializer.deserialize(input); + SortKey[] array = new SortKey[numItems]; + sortKeys.toArray(array); + input.releaseArrays(); + return array; + } catch (IOException e) { + throw new UncheckedIOException("Failed to deserialize sort key sketch", e); + } + } + + @Override + public int sizeOf(SortKey item) { + return serializeToByteArray(item).length; + } + + @Override + public int sizeOf(Memory mem, long offset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return 0; + } + + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + return Integer.BYTES + numBytes; + } + + @Override + public String toString(SortKey item) { + return item.toString(); + } + + @Override + public Class getClassOfT() { + return SortKey.class; + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 58% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index 852d2157b8cb..f6fcdb8b16ef 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -27,24 +27,39 @@ * statistics in bytes */ @Internal -class DataStatisticsEvent, S> implements OperatorEvent { +class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; + private final boolean applyImmediately; - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; } - static , S> DataStatisticsEvent create( + static StatisticsEvent createTaskStatisticsEvent( long checkpointId, - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - return new DataStatisticsEvent<>( + DataStatistics statistics, + TypeSerializer statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. + return new StatisticsEvent( checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); + } + + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer statisticsSerializer, + boolean applyImmediately) { + return new StatisticsEvent( + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); } long checkpointId() { @@ -54,4 +69,8 @@ long checkpointId() { byte[] statisticsBytes() { return statisticsBytes; } + + boolean applyImmediately() { + return applyImmediately; + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 66% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index 889e85112e16..bc28df2b0e22 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.io.Serializable; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.data.RowData; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -34,68 +35,66 @@ * After shuffling, a filter and mapper are required to filter out the data distribution weight, * unwrap the object and extract the original record type T. */ -class DataStatisticsOrRecord, S> implements Serializable { +@Internal +public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private DataStatistics statistics; + private GlobalStatistics statistics; private RowData record; - private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; this.record = record; } - static , S> DataStatisticsOrRecord fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); + static StatisticsOrRecord fromRecord(RowData record) { + return new StatisticsOrRecord(null, record); } - static , S> DataStatisticsOrRecord fromDataStatistics( - DataStatistics statistics) { - return new DataStatisticsOrRecord<>(statistics, null); + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { + return new StatisticsOrRecord(statistics, null); } - static , S> DataStatisticsOrRecord reuseRecord( - DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { + static StatisticsOrRecord reuseRecord( + StatisticsOrRecord reuse, TypeSerializer recordSerializer) { if (reuse.hasRecord()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } } - static , S> DataStatisticsOrRecord reuseStatistics( - DataStatisticsOrRecord reuse, - TypeSerializer> statisticsSerializer) { - if (reuse.hasDataStatistics()) { + static StatisticsOrRecord reuseStatistics( + StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { + if (reuse.hasStatistics()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); } } - boolean hasDataStatistics() { + boolean hasStatistics() { return statistics != null; } - boolean hasRecord() { + public boolean hasRecord() { return record != null; } - DataStatistics dataStatistics() { + GlobalStatistics statistics() { return statistics; } - void dataStatistics(DataStatistics newStatistics) { + void statistics(GlobalStatistics newStatistics) { this.statistics = newStatistics; } - RowData record() { + public RowData record() { return record; } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 53% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index e9a6fa0cbfc5..6e403425938d 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -29,13 +29,12 @@ import org.apache.flink.table.data.RowData; @Internal -class DataStatisticsOrRecordSerializer, S> - extends TypeSerializer> { - private final TypeSerializer> statisticsSerializer; +class StatisticsOrRecordSerializer extends TypeSerializer { + private final TypeSerializer statisticsSerializer; private final TypeSerializer recordSerializer; - DataStatisticsOrRecordSerializer( - TypeSerializer> statisticsSerializer, + StatisticsOrRecordSerializer( + TypeSerializer statisticsSerializer, TypeSerializer recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,13 +47,13 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer> duplicate() { - TypeSerializer> duplicateStatisticsSerializer = + public TypeSerializer duplicate() { + TypeSerializer duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( + return new StatisticsOrRecordSerializer( duplicateStatisticsSerializer, duplicateRowDataSerializer); } else { return this; @@ -62,34 +61,31 @@ public TypeSerializer> duplicate() { } @Override - public DataStatisticsOrRecord createInstance() { + public StatisticsOrRecord createInstance() { // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } @Override - public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { + public StatisticsOrRecord copy(StatisticsOrRecord from) { if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); } } @Override - public DataStatisticsOrRecord copy( - DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { - DataStatisticsOrRecord to; + public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { + StatisticsOrRecord to; if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.copy(from.record(), to.record()); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); + to.statistics(statistics); } return to; @@ -101,41 +97,40 @@ public int getLength() { } @Override - public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) + public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) throws IOException { if (statisticsOrRecord.hasRecord()) { target.writeBoolean(true); recordSerializer.serialize(statisticsOrRecord.record(), target); } else { target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); } } @Override - public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { + public StatisticsOrRecord deserialize(DataInputView source) throws IOException { boolean isRecord = source.readBoolean(); if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); } } @Override - public DataStatisticsOrRecord deserialize( - DataStatisticsOrRecord reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord to; + public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) + throws IOException { + StatisticsOrRecord to; boolean isRecord = source.readBoolean(); if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.deserialize(to.record(), source); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + to.statistics(statistics); } return to; @@ -154,12 +149,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + if (!(obj instanceof StatisticsOrRecordSerializer)) { return false; } - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; + StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; return Objects.equals(statisticsSerializer, other.statisticsSerializer) && Objects.equals(recordSerializer, other.recordSerializer); } @@ -170,25 +164,22 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); + public TypeSerializerSnapshot snapshotConfiguration() { + return new StatisticsOrRecordSerializerSnapshot(this); } - public static class DataStatisticsOrRecordSerializerSnapshot, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { + public static class StatisticsOrRecordSerializerSnapshot + extends CompositeTypeSerializerSnapshot { private static final int CURRENT_VERSION = 1; - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public StatisticsOrRecordSerializerSnapshot() { + super(StatisticsOrRecordSerializer.class); } @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer serializer) { + public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { super(serializer); } @@ -200,7 +191,7 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer[] getNestedSerializers( - DataStatisticsOrRecordSerializer outerSerializer) { + StatisticsOrRecordSerializer outerSerializer) { return new TypeSerializer[] { outerSerializer.statisticsSerializer, outerSerializer.recordSerializer }; @@ -208,12 +199,12 @@ protected TypeSerializer[] getNestedSerializers( @SuppressWarnings("unchecked") @Override - protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( + protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { - TypeSerializer> statisticsSerializer = - (TypeSerializer>) nestedSerializers[0]; + TypeSerializer statisticsSerializer = + (TypeSerializer) nestedSerializers[0]; TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java new file mode 100644 index 000000000000..43f72e336e06 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +/** + * Range distribution requires gathering statistics on the sort keys to determine proper range + * boundaries to distribute/cluster rows before writer operators. + */ +public enum StatisticsType { + /** + * Tracks the data statistics as {@code Map} frequency. It works better for + * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in + * hundreds or thousands. + * + *

      + *
    • Pro: accurate measurement on the statistics/weight of every key. + *
    • Con: memory footprint can be large if the key cardinality is high. + *
    + */ + Map, + + /** + * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds + * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, + * uuid etc.) where the cardinalities can be in millions or billions. + * + *
      + *
    • Pro: relatively low memory footprint for high-cardinality sort keys. + *
    • Con: non-precise approximation with potentially lower accuracy. + *
    + */ + Sketch, + + /** + * Initially use Map for statistics tracking. If key cardinality turns out to be high, + * automatically switch to sketch sampling. + */ + Auto +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java new file mode 100644 index 000000000000..5d48ec57ca49 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.UncheckedIOException; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class StatisticsUtil { + + private StatisticsUtil() {} + + static DataStatistics createTaskStatistics( + StatisticsType type, int operatorParallelism, int numPartitions) { + if (type == StatisticsType.Map) { + return new MapDataStatistics(); + } else { + return new SketchDataStatistics( + SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); + } + } + + static byte[] serializeDataStatistics( + DataStatistics dataStatistics, TypeSerializer statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize data statistics", e); + } + } + + static DataStatistics deserializeDataStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize data statistics", e); + } + } + + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(completedStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static StatisticsType collectType(StatisticsType config) { + return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; + } + + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + + static StatisticsType collectType( + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java new file mode 100644 index 000000000000..5910bd685510 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -0,0 +1,100 @@ +/* + * 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.Comparator; +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +class Fixtures { + private Fixtures() {} + + public static final int NUM_SUBTASKS = 2; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); + public static final TypeSerializer ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); + public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + public static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); + public static final SortKeySerializer SORT_KEY_SERIALIZER = + new SortKeySerializer(SCHEMA, SORT_ORDER); + public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = + new DataStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); + + public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + public static final Map CHAR_KEYS = createCharKeys(); + + public static StatisticsEvent createStatisticsEvent( + StatisticsType type, + TypeSerializer statisticsSerializer, + long checkpointId, + SortKey... keys) { + DataStatistics statistics = createTaskStatistics(type, keys); + return StatisticsEvent.createTaskStatisticsEvent( + checkpointId, statistics, statisticsSerializer); + } + + public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { + DataStatistics statistics; + if (type == StatisticsType.Sketch) { + statistics = new SketchDataStatistics(128); + } else { + statistics = new MapDataStatistics(); + } + + for (SortKey key : keys) { + statistics.add(key); + } + + return statistics; + } + + private static Map createCharKeys() { + Map keys = Maps.newHashMap(); + for (char c = 'a'; c <= 'z'; ++c) { + String key = Character.toString(c); + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, key); + keys.put(key, sortKey); + } + + return keys; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index 739cf764e2a6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,63 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestAggregatedStatistics { - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - @Test - public void mergeDataStatisticTest() { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>(1, statisticsSerializer); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(keyA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyB) - + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 0064c91340bf..8322ce683768 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -18,161 +18,448 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.BeforeEach; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - private final SortKey keyA = sortKey.copy(); - private final SortKey keyB = sortKey.copy(); - - private AggregatedStatisticsTracker> - aggregatedStatisticsTracker; - - public TestAggregatedStatisticsTracker() { - keyA.set(0, "a"); - keyB.set(0, "b"); - } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveNewerStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); - @BeforeEach - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + // checkpoint 1 is completed + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + // checkpoint 2 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } } - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - checkpoint2Subtask0DataStatistic.add(keyB); - checkpoint2Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveOlderStatisticsEventTest(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint3Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); + // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 4L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + } } - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveCompletedStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(keyA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyA) - + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); - assertThat((long) globalDataStatistics.statistics().get(keyB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyB) - + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 3L, + CHAR_KEYS.get("b"), 3L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint2Subtask0DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); + completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask1DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); // Receive data statistics from all subtasks at checkpoint 2 completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); + tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + } + + @Test + public void coordinatorSwitchToSketchOverThreshold() { + int parallelism = 3; + int downstreamParallelism = 3; + int switchToSketchThreshold = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + switchToSketchThreshold, + null); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + // converted to sketch statistics as map size is 4 (over the switch threshold of 3) + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + @Test + public void coordinatorMapOperatorSketch() { + int parallelism = 3; + int downstreamParallelism = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + + // first operator event has map statistics + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + // second operator event contains sketch statistics + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Sketch, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + // third operator event has Map statistics + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + private AggregatedStatisticsTracker createTracker(StatisticsType type) { + return new AggregatedStatisticsTracker( + "testOperator", + Fixtures.NUM_SUBTASKS, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java new file mode 100644 index 000000000000..4ee9888934a8 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -0,0 +1,54 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestCompletedStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return CompletedStatistics.class; + } + + @Override + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( + 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java index 849253564209..a08a8a73e80c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -18,9 +18,13 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.time.Duration; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -28,128 +32,182 @@ import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> - dataStatisticsCoordinator; @BeforeEach public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); } - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + private void tasksReady(DataStatisticsCoordinator coordinator) { + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); } - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + StatisticsEvent.createTaskStatisticsEvent( + 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + 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, + CHAR_KEYS.get("a"), + 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( + CHAR_KEYS.get("a"), 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( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + 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 testDataStatisticsEventHandling() throws Exception { - tasksReady(); - SortKey key = sortKey.copy(); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify global data statistics is the aggregation of all subtasks data statistics - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - keyA, 2L, - keyB, 3L, - keyC, 5L)); + public void testRequestGlobalStatisticsEventHandling() throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); + } } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -157,8 +215,7 @@ static void setAllTasksReady( } } - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { @@ -175,4 +232,15 @@ static void waitForCoordinatorToProcessActions( ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); } } + + private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { + return new DataStatisticsCoordinator( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + NUM_SUBTASKS, + type, + 0.0d); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index c5216eeb712a..6317f2bfde18 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; @@ -27,117 +31,157 @@ import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; @BeforeEach public void before() { - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); receivingTasks = EventReceivingTasks.createForRunningTasks(); } - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - SortKey keyD = sortKey.copy(); - keyD.set(0, "c"); - SortKey keyE = sortKey.copy(); - keyE.set(0, "c"); - + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testCheckpointAndReset(StatisticsType type) throws Exception { + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create( + new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { + DataStatisticsCoordinator dataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); // Start the coordinator coordinator.start(); TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + Map checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyD); - checkpoint2Subtask0DataStatistic.add(keyE); - checkpoint2Subtask0DataStatistic.add(keyE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); + coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + Map checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + } + waitForCheckpoint(2L, dataStatisticsCoordinator); // Reset coordinator to checkpoint 1 coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + DataStatisticsCoordinator restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } } } - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator> coordinator) + private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); return future.get(); } + + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { + return new DataStatisticsCoordinatorProvider( + "DataStatisticsCoordinatorProvider", + OPERATOR_ID, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + type, + 0.0); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java index 5e6f971807ba..c760f1ba96d3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -18,22 +18,25 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.StateInitializationContext; @@ -49,102 +52,95 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; 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.types.Types; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; 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.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; public class TestDataStatisticsOperator { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final RowType rowType = RowType.of(new VarCharType(), new IntType()); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer( - new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsOperator> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } + + private Environment env; @BeforeEach public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); + this.env = + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return new DataStatisticsOperator<>( - "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); + return createOperator(type, downstreamParallelism, mockGateway); } - @AfterEach - public void clean() throws Exception { - operator.close(); + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { + DataStatisticsOperator operator = + new DataStatisticsOperator( + "testOperator", + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + mockGateway, + downstreamParallelism, + type); + operator.setup( + new OneInputStreamTask(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + return operator; } - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { + @SuppressWarnings("unchecked") + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElement(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(StringData.fromString("a"), 5))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); - - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + DataStatistics localStatistics = operator.localStatistics(); + assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + Map keyFrequency = (Map) localStatistics.result(); + assertThat(keyFrequency) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); + } else { + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) localStatistics.result(); + assertThat(sketch.getSamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } testHarness.endInput(); } } - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testOperatorOutput(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { testHarness.processElement( new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); testHarness.processElement( @@ -154,8 +150,8 @@ public void testOperatorOutput() throws Exception { List recordsOutput = testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record) .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( @@ -166,70 +162,172 @@ public void testOperatorOutput() throws Exception { } } - @Test - public void testRestoreState() throws Exception { + private static Stream provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + + @ParameterizedTest + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { + Map keyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); + SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness1 = createHarness(this.operator)) { - MapDataStatistics mapDataStatistics = new MapDataStatistics(); - - SortKey key = sortKey.copy(); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "b"); - mapDataStatistics.add(key); - key.set(0, "c"); - mapDataStatistics.add(key); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - DataStatisticsEvent> event = - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + try (OneInputStreamOperatorTestHarness testHarness1 = + createHarness(operator)) { + GlobalStatistics statistics; + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); + } else { + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); + } + + StatisticsEvent event = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); operator.handleOperatorEvent(event); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf(expectedMap); + + GlobalStatistics globalStatistics = operator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); + try (OneInputStreamOperatorTestHarness testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - // restored RowData is BinaryRowData. convert to GenericRowData for comparison - Map restoredStatistics = Maps.newHashMap(); - restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + } + } - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithLocalStatsOverThreshold() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // add rows with unique keys + for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()).hasSize(i + 1); + } + + // one more item should trigger the migration to sketch stats + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); + // reservoir not full yet + assertThat(sketch.getN()).isLessThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); + + // add more items to saturate the reservoir + for (int i = 0; i < reservoirSize; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + } + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + sketch = (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()) + .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); + // reservoir is full now + assertThat(sketch.getN()).isGreaterThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize(reservoirSize); + + testHarness.endInput(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithGlobalSketchStatistics() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); - assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); + // started with Map stype + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + + // received global statistics with sketch type + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( + 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); + operator.handleOperatorEvent( + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(1); + assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); + + testHarness.endInput(); } } private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); OperatorStateStore operatorStateStore = @@ -238,17 +336,14 @@ private StateInitializationContext getStateContext() throws Exception { return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); } - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - createHarness( - final DataStatisticsOperator> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); + private OneInputStreamOperatorTestHarness createHarness( + DataStatisticsOperator dataStatisticsOperator) throws Exception { + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>( + dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); + harness.setup( + new StatisticsOrRecordSerializer( + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); harness.open(); return harness; } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java new file mode 100644 index 000000000000..59ce6df05d9d --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class TestDataStatisticsSerializer extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return Fixtures.TASK_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return DataStatistics.class; + } + + @Override + protected DataStatistics[] getTestData() { + return new DataStatistics[] { + new MapDataStatistics(), + Fixtures.createTaskStatistics( + StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), + new SketchDataStatistics(128), + Fixtures.createTaskStatistics( + StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) + }; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.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.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index be2beeebc93c..8a25c7ad9898 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -18,74 +18,50 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { - private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); - private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private final RowDataWrapper rowWrapper = - new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); - + @SuppressWarnings("unchecked") @Test public void testAddsAndGet() { MapDataStatistics dataStatistics = new MapDataStatistics(); - GenericRowData reusedRow = - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("c")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("a")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - Map actual = dataStatistics.statistics(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyA = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyB = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyC = sortKey.copy(); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); - Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Map actual = (Map) dataStatistics.result(); + Map expected = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index e6726e7db785..d5a0bebc74e7 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -64,65 +65,60 @@ private static SortKey[] initSortKeys() { } // Total weight is 800 - private final MapDataStatistics mapDataStatistics = - new MapDataStatistics( - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L)); + private final Map mapStatistics = + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L); @Test public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // each task should get targeted weight of 100 (=800/8) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -144,19 +140,20 @@ public void testEvenlyDividableNoClosingFileCost() { Pair.of(100L, 1), 7, Pair.of(100L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 100 before close file cost factored in. // close file cost is 5 = 5% * 100. @@ -165,35 +162,30 @@ public void testEvenlyDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight with close cost per subtask is 110 (880/8) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -216,51 +208,48 @@ public void testEvenlyDividableWithClosingFileCost() { Pair.of(100L, 2), 7, Pair.of(75L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 // each task should get targeted weight of 89 = ceiling(800/9) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -284,19 +273,20 @@ public void testNonDividableNoClosingFileCost() { Pair.of(89L, 2), 8, Pair.of(88L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 89 before close file cost factored in. // close file cost is 5 (= 5% * 89) per file. @@ -305,35 +295,31 @@ public void testNonDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight per subtask is 98 ceiling(880/9) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -358,40 +344,39 @@ public void testNonDividableWithClosingFileCost() { Pair.of(88L, 2), 8, Pair.of(61L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); // drift threshold is high for non-dividable scenario with close cost validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); } private static Map>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { + MapRangePartitioner partitioner, int numPartitions, Map mapStatistics) { // The Map key is the subtaskId. // For the map value pair, the first element is the count of assigned and // the second element of Set is for the set of assigned keys. Map>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); return partitionResults; } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java new file mode 100644 index 000000000000..396bfae2f13c --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + SketchDataStatistics dataStatistics = new SketchDataStatistics(128); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + ReservoirItemsSketch actual = (ReservoirItemsSketch) dataStatistics.result(); + assertThat(actual.getSamples()) + .isEqualTo( + new SortKey[] { + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") + }); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java new file mode 100644 index 000000000000..31dae5c76aeb --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -0,0 +1,133 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchUtil { + @Test + public void testCoordinatorReservoirSize() { + // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); + // adjust to multiplier of 100 + assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); + // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) + .isEqualTo(1_000_000 - (1_000_000 % 10_123)); + } + + @Test + public void testOperatorReservoirSize() { + assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) + .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); + assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) + .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); + assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) + .isEqualTo( + (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); + assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) + .isEqualTo( + (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); + } + + @Test + public void testRangeBoundsOneChannel() { + assertThat( + SketchUtil.rangeBounds( + 1, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .isEmpty(); + } + + @Test + public void testRangeBoundsDivisible() { + assertThat( + SketchUtil.rangeBounds( + 3, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void testRangeBoundsNonDivisible() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); + } + + @Test + public void testRangeBoundsSkipDuplicates() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + // skipped duplicate c's + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java index 291302aef486..54cceae6e55b 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -18,14 +18,24 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.iceberg.NullOrder; import org.apache.iceberg.Schema; import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.RowDataWrapper; +import org.junit.jupiter.api.Test; public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { private final DataGenerator generator = new DataGenerators.Primitives(); @@ -54,4 +64,27 @@ protected SortOrder sortOrder() { protected GenericRowData rowData() { return generator.generateFlinkRowData(); } + + @Test + public void testSerializationSize() throws Exception { + RowData rowData = + GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = Fixtures.SORT_KEY.copy(); + sortKey.wrap(struct); + SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataOutputSerializer output = new DataOutputSerializer(1024); + serializer.serialize(sortKey, output); + 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); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = serializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } } diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index f06318af83a3..aac01c9c6931 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -66,6 +66,8 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { exclude group: 'org.slf4j' } + implementation libs.datasketches + testImplementation libs.flink118.connector.test.utils testImplementation libs.flink118.core testImplementation libs.flink118.runtime diff --git a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 3b2c74fd6ece..a9ad386a5a4a 100644 --- a/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.18/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.nio.charset.StandardCharsets; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -28,6 +29,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; 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; @@ -67,6 +70,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -83,10 +88,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.java deleted file mode 100644 index 157f04b8b0ed..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatistics.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.flink.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * AggregatedStatistics is used by {@link DataStatisticsCoordinator} to collect {@link - * DataStatistics} from {@link DataStatisticsOperator} subtasks for specific checkpoint. It stores - * the merged {@link DataStatistics} result from all reported subtasks. - */ -class AggregatedStatistics, S> implements Serializable { - - private final long checkpointId; - private final DataStatistics dataStatistics; - - AggregatedStatistics(long checkpoint, TypeSerializer> statisticsSerializer) { - this.checkpointId = checkpoint; - this.dataStatistics = statisticsSerializer.createInstance(); - } - - AggregatedStatistics(long checkpoint, DataStatistics dataStatistics) { - this.checkpointId = checkpoint; - this.dataStatistics = dataStatistics; - } - - long checkpointId() { - return checkpointId; - } - - DataStatistics dataStatistics() { - return dataStatistics; - } - - void mergeDataStatistic(String operatorName, long eventCheckpointId, D eventDataStatistics) { - Preconditions.checkArgument( - checkpointId == eventCheckpointId, - "Received unexpected event from operator %s checkpoint %s. Expected checkpoint %s", - operatorName, - eventCheckpointId, - checkpointId); - dataStatistics.merge(eventDataStatistics); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("dataStatistics", dataStatistics) - .toString(); - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index e8ff61dbeb27..338523b7b074 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -18,116 +18,238 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import java.util.NavigableMap; import java.util.Set; +import javax.annotation.Nullable; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.datasketches.sampling.ReservoirItemsUnion; import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * AggregatedStatisticsTracker is used by {@link DataStatisticsCoordinator} to track the in progress - * {@link AggregatedStatistics} received from {@link DataStatisticsOperator} subtasks for specific - * checkpoint. + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. */ -class AggregatedStatisticsTracker, S> { +class AggregatedStatisticsTracker { private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - private static final double ACCEPT_PARTIAL_AGGR_THRESHOLD = 90; + private final String operatorName; - private final TypeSerializer> statisticsSerializer; private final int parallelism; - private final Set inProgressSubtaskSet; - private volatile AggregatedStatistics inProgressStatistics; + private final TypeSerializer statisticsSerializer; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final int switchToSketchThreshold; + private final NavigableMap aggregationsPerCheckpoint; + + private CompletedStatistics completedStatistics; AggregatedStatisticsTracker( String operatorName, - TypeSerializer> statisticsSerializer, - int parallelism) { + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable CompletedStatistics restoredStatistics) { this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; this.parallelism = parallelism; - this.inProgressSubtaskSet = Sets.newHashSet(); + this.statisticsSerializer = + new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.switchToSketchThreshold = switchToSketchThreshold; + this.completedStatistics = restoredStatistics; + + this.aggregationsPerCheckpoint = Maps.newTreeMap(); } - AggregatedStatistics updateAndCheckCompletion( - int subtask, DataStatisticsEvent event) { + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { long checkpointId = event.checkpointId(); + LOG.debug( + "Handling statistics event from subtask {} of operator {} for checkpoint {}", + subtask, + operatorName, + checkpointId); - if (inProgressStatistics != null && inProgressStatistics.checkpointId() > checkpointId) { + if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { LOG.info( - "Expect data statistics for operator {} checkpoint {}, but receive event from older checkpoint {}. Ignore it.", + "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " + + "Was expecting data statistics from checkpoint higher than {}", operatorName, - inProgressStatistics.checkpointId(), - checkpointId); + subtask, + checkpointId, + completedStatistics.checkpointId()); return null; } - AggregatedStatistics completedStatistics = null; - if (inProgressStatistics != null && inProgressStatistics.checkpointId() < checkpointId) { - if ((double) inProgressSubtaskSet.size() / parallelism * 100 - >= ACCEPT_PARTIAL_AGGR_THRESHOLD) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Complete data statistics aggregation at checkpoint {} as it is more than the threshold of {} percentage", - inProgressSubtaskSet.size(), - parallelism, - operatorName, + Aggregation aggregation = + aggregationsPerCheckpoint.computeIfAbsent( checkpointId, - inProgressStatistics.checkpointId(), - ACCEPT_PARTIAL_AGGR_THRESHOLD); + ignored -> + new Aggregation( + parallelism, + downstreamParallelism, + switchToSketchThreshold, + statisticsType, + StatisticsUtil.collectType(statisticsType, completedStatistics))); + DataStatistics dataStatistics = + StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); + if (!aggregation.merge(subtask, dataStatistics)) { + LOG.debug( + "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } + + if (aggregation.isComplete()) { + this.completedStatistics = aggregation.completedStatistics(checkpointId); + // clean up aggregations up to the completed checkpoint id + aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); + return completedStatistics; + } + + return null; + } + + @VisibleForTesting + NavigableMap aggregationsPerCheckpoint() { + return aggregationsPerCheckpoint; + } + + static class Aggregation { + private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); + + private final Set subtaskSet; + private final int parallelism; + private final int downstreamParallelism; + private final int switchToSketchThreshold; + private final StatisticsType configuredType; + private StatisticsType currentType; + private Map mapStatistics; + private ReservoirItemsUnion sketchStatistics; + + Aggregation( + int parallelism, + int downstreamParallelism, + int switchToSketchThreshold, + StatisticsType configuredType, + StatisticsType currentType) { + this.subtaskSet = Sets.newHashSet(); + this.parallelism = parallelism; + this.downstreamParallelism = downstreamParallelism; + this.switchToSketchThreshold = switchToSketchThreshold; + this.configuredType = configuredType; + this.currentType = currentType; + + if (currentType == StatisticsType.Map) { + this.mapStatistics = Maps.newHashMap(); + this.sketchStatistics = null; } else { - LOG.info( - "Received data statistics from {} subtasks out of total {} for operator {} at checkpoint {}. " - + "Aborting the incomplete aggregation for checkpoint {}", - inProgressSubtaskSet.size(), - parallelism, - operatorName, - checkpointId, - inProgressStatistics.checkpointId()); + this.mapStatistics = null; + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); } + } - inProgressStatistics = null; - inProgressSubtaskSet.clear(); + @VisibleForTesting + Set subtaskSet() { + return subtaskSet; } - if (inProgressStatistics == null) { - LOG.info("Starting a new data statistics for checkpoint {}", checkpointId); - inProgressStatistics = new AggregatedStatistics<>(checkpointId, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + StatisticsType currentType() { + return currentType; } - if (!inProgressSubtaskSet.add(subtask)) { - LOG.debug( - "Ignore duplicated data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } else { - inProgressStatistics.mergeDataStatistic( - operatorName, - event.checkpointId(), - DataStatisticsUtil.deserializeDataStatistics( - event.statisticsBytes(), statisticsSerializer)); + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; } - if (inProgressSubtaskSet.size() == parallelism) { - completedStatistics = inProgressStatistics; - LOG.info( - "Received data statistics from all {} operators {} for checkpoint {}. Return last completed aggregator {}.", - parallelism, - operatorName, - inProgressStatistics.checkpointId(), - completedStatistics.dataStatistics()); - inProgressStatistics = new AggregatedStatistics<>(checkpointId + 1, statisticsSerializer); - inProgressSubtaskSet.clear(); + @VisibleForTesting + ReservoirItemsUnion sketchStatistics() { + return sketchStatistics; } - return completedStatistics; - } + private boolean isComplete() { + return subtaskSet.size() == parallelism; + } - @VisibleForTesting - AggregatedStatistics inProgressStatistics() { - return inProgressStatistics; + /** @return false if duplicate */ + private boolean merge(int subtask, DataStatistics taskStatistics) { + if (subtaskSet.contains(subtask)) { + return false; + } + + subtaskSet.add(subtask); + merge(taskStatistics); + return true; + } + + @SuppressWarnings("unchecked") + private void merge(DataStatistics taskStatistics) { + if (taskStatistics.type() == StatisticsType.Map) { + Map taskMapStats = (Map) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); + if (configuredType == StatisticsType.Auto + && mapStatistics.size() > switchToSketchThreshold) { + convertCoordinatorToSketch(); + } + } else { + // convert task stats to sketch first + ReservoirItemsSketch taskSketch = + ReservoirItemsSketch.newInstance( + SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); + SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); + sketchStatistics.update(taskSketch); + } + } else { + ReservoirItemsSketch taskSketch = + (ReservoirItemsSketch) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + // convert global stats to sketch first + convertCoordinatorToSketch(); + } + + sketchStatistics.update(taskSketch); + } + } + + private void convertCoordinatorToSketch() { + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); + this.currentType = StatisticsType.Sketch; + this.mapStatistics = null; + } + + private CompletedStatistics completedStatistics(long checkpointId) { + if (currentType == StatisticsType.Map) { + LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + } else { + ReservoirItemsSketch sketch = sketchStatistics.getResult(); + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } + } } } 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 new file mode 100644 index 000000000000..c0e228965ddd --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.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.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). + */ +class CompletedStatistics { + private final long checkpointId; + private final StatisticsType type; + private final Map keyFrequency; + private final SortKey[] keySamples; + + static CompletedStatistics fromKeyFrequency(long checkpointId, Map stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( + long checkpointId, + StatisticsType type, + Map keyFrequency, + SortKey[] keySamples) { + this.checkpointId = checkpointId; + this.type = type; + this.keyFrequency = keyFrequency; + this.keySamples = keySamples; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("keyFrequency", keyFrequency) + .add("keySamples", keySamples) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof CompletedStatistics)) { + return false; + } + + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(keyFrequency, other.keyFrequency()) + && Arrays.equals(keySamples, other.keySamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + Map keyFrequency() { + return keyFrequency; + } + + SortKey[] keySamples() { + return keySamples; + } +} 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 new file mode 100644 index 000000000000..7f55188e7f8c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -0,0 +1,178 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; + +class CompletedStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer keyFrequencySerializer; + private final ListSerializer keySamplesSerializer; + + CompletedStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); + } + + @Override + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + keyFrequencySerializer.serialize(record.keyFrequency(), target); + } else { + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); + } + } + + @Override + public CompletedStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + Map keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); + } else { + List sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); + } + } + + @Override + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); + } + + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public CompletedStatisticsSerializerSnapshot() { + super(CompletedStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + CompletedStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new CompletedStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java index 9d7cf179ab1c..76c59cd5f4b8 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; @@ -29,29 +31,18 @@ * (sketching) can be used. */ @Internal -interface DataStatistics, S> { +interface DataStatistics { + + StatisticsType type(); - /** - * Check if data statistics contains any statistics information. - * - * @return true if data statistics doesn't contain any statistics information - */ boolean isEmpty(); /** Add row sortKey to data statistics. */ void add(SortKey sortKey); /** - * Merge current statistics with other statistics. - * - * @param otherStatistics the statistics to be merged - */ - void merge(D otherStatistics); - - /** - * Get the underline statistics. - * - * @return the underline statistics + * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link + * ReservoirItemsSketch} (high cardinality) */ - S statistics(); + Object result(); } 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 c8ac79c61bf6..3b21fbae315a 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 @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -35,6 +36,10 @@ import org.apache.flink.util.Preconditions; import org.apache.flink.util.ThrowableCatchingRunnable; import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Maps; @@ -44,51 +49,86 @@ import org.slf4j.LoggerFactory; /** - * DataStatisticsCoordinator receives {@link DataStatisticsEvent} from {@link - * DataStatisticsOperator} every subtask and then merge them together. Once aggregation for all - * subtasks data statistics completes, DataStatisticsCoordinator will send the aggregated data - * statistics back to {@link DataStatisticsOperator}. In the end a custom partitioner will - * distribute traffic based on the aggregated data statistics to improve data clustering. + * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} + * every subtask and then merge them together. Once aggregation for all subtasks data statistics + * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link + * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the + * aggregated data statistics to improve data clustering. */ @Internal -class DataStatisticsCoordinator, S> implements OperatorCoordinator { +class DataStatisticsCoordinator implements OperatorCoordinator { private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); private final String operatorName; + private final OperatorCoordinator.Context context; + private final Schema schema; + private final SortOrder sortOrder; + private final Comparator comparator; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; + private final ExecutorService coordinatorExecutor; - private final OperatorCoordinator.Context operatorCoordinatorContext; private final SubtaskGateways subtaskGateways; private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer> statisticsSerializer; - private final transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private volatile AggregatedStatistics completedStatistics; - private volatile boolean started; + private final TypeSerializer completedStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; + + private transient boolean started; + private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; DataStatisticsCoordinator( String operatorName, OperatorCoordinator.Context context, - TypeSerializer> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { this.operatorName = operatorName; + this.context = context; + this.schema = schema; + this.sortOrder = sortOrder; + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + this.coordinatorThreadFactory = new CoordinatorExecutorThreadFactory( "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.operatorCoordinatorContext = context; - this.subtaskGateways = new SubtaskGateways(operatorName, parallelism()); - this.statisticsSerializer = statisticsSerializer; - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>(operatorName, statisticsSerializer, parallelism()); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void start() throws Exception { LOG.info("Starting data statistics coordinator: {}.", operatorName); - started = true; + this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker( + operatorName, + context.currentParallelism(), + schema, + sortOrder, + downstreamParallelism, + statisticsType, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + completedStatistics); } @Override public void close() throws Exception { coordinatorExecutor.shutdown(); + this.aggregatedStatisticsTracker = null; + this.started = false; LOG.info("Closed data statistics coordinator: {}.", operatorName); } @@ -148,7 +188,7 @@ private void runInCoordinatorThread(ThrowingRunnable action, String a operatorName, actionString, t); - operatorCoordinatorContext.failJob(t); + context.failJob(t); } }); } @@ -157,42 +197,102 @@ private void ensureStarted() { Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); } - private int parallelism() { - return operatorCoordinatorContext.currentParallelism(); - } - - private void handleDataStatisticRequest(int subtask, DataStatisticsEvent event) { - AggregatedStatistics aggregatedStatistics = + private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { + CompletedStatistics maybeCompletedStatistics = aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - if (aggregatedStatistics != null) { - completedStatistics = aggregatedStatistics; - sendDataStatisticsToSubtasks( - completedStatistics.checkpointId(), completedStatistics.dataStatistics()); + if (maybeCompletedStatistics != null) { + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); } } @SuppressWarnings("FutureReturnValueIgnored") - private void sendDataStatisticsToSubtasks( - long checkpointId, DataStatistics globalDataStatistics) { - callInCoordinatorThread( + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( () -> { - DataStatisticsEvent dataStatisticsEvent = - DataStatisticsEvent.create(checkpointId, globalDataStatistics, statisticsSerializer); - int parallelism = parallelism(); - for (int i = 0; i < parallelism; ++i) { - subtaskGateways.getSubtaskGateway(i).sendEvent(dataStatisticsEvent); + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); + for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference + subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); } - - return null; }, String.format( "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, checkpointId)); + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } } @Override - @SuppressWarnings("unchecked") public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { runInCoordinatorThread( () -> { @@ -202,8 +302,14 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven attemptNumber, operatorName, event); - Preconditions.checkArgument(event instanceof DataStatisticsEvent); - handleDataStatisticRequest(subtask, ((DataStatisticsEvent) event)); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } }, String.format( "handling operator event %s from subtask %d (#%d)", @@ -219,8 +325,8 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r operatorName, checkpointId); resultFuture.complete( - DataStatisticsUtil.serializeAggregatedStatistics( - completedStatistics, statisticsSerializer)); + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); }, String.format("taking checkpoint %d", checkpointId)); } @@ -229,11 +335,9 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r public void notifyCheckpointComplete(long checkpointId) {} @Override - public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData) - throws Exception { + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", @@ -244,8 +348,13 @@ public void resetToCheckpoint(long checkpointId, @Nullable byte[] checkpointData LOG.info( "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - completedStatistics = - DataStatisticsUtil.deserializeAggregatedStatistics(checkpointData, statisticsSerializer); + this.completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); } @Override @@ -269,7 +378,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr runInCoordinatorThread( () -> { LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistic {}", + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", subtask, attemptNumber, operatorName); @@ -295,14 +404,20 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway } @VisibleForTesting - AggregatedStatistics completedStatistics() { + CompletedStatistics completedStatistics() { return completedStatistics; } + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + private static class SubtaskGateways { private final String operatorName; private final Map[] gateways; + @SuppressWarnings("unchecked") private SubtaskGateways(String operatorName, int parallelism) { this.operatorName = operatorName; gateways = new Map[parallelism]; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java index 47dbfc3cfbe1..9d7d989c298e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -19,33 +19,52 @@ package org.apache.iceberg.flink.sink.shuffle; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.runtime.jobgraph.OperatorID; import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; /** * DataStatisticsCoordinatorProvider provides the method to create new {@link * DataStatisticsCoordinator} */ @Internal -public class DataStatisticsCoordinatorProvider, S> - extends RecreateOnResetOperatorCoordinator.Provider { +public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { private final String operatorName; - private final TypeSerializer> statisticsSerializer; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; public DataStatisticsCoordinatorProvider( String operatorName, OperatorID operatorID, - TypeSerializer> statisticsSerializer) { + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { super(operatorID); this.operatorName = operatorName; - this.statisticsSerializer = statisticsSerializer; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; } @Override public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator<>(operatorName, context, statisticsSerializer); + return new DataStatisticsCoordinator( + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java index 5157a37cf2cd..59c38b239725 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Map; import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -47,9 +48,8 @@ * distribution to downstream subtasks. */ @Internal -class DataStatisticsOperator, S> - extends AbstractStreamOperator> - implements OneInputStreamOperator>, OperatorEventHandler { +public class DataStatisticsOperator extends AbstractStreamOperator + implements OneInputStreamOperator, OperatorEventHandler { private static final long serialVersionUID = 1L; @@ -57,141 +57,209 @@ class DataStatisticsOperator, S> private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; private final OperatorEventGateway operatorEventGateway; - private final TypeSerializer> statisticsSerializer; - private transient volatile DataStatistics localStatistics; - private transient volatile DataStatistics globalStatistics; - private transient ListState> globalStatisticsState; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final TypeSerializer taskStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; + + private transient int parallelism; + private transient int subtaskIndex; + private transient ListState globalStatisticsState; + // current statistics type may be different from the config due to possible + // migration from Map statistics to Sketch statistics when high cardinality detected + private transient volatile StatisticsType taskStatisticsType; + private transient volatile DataStatistics localStatistics; + private transient volatile GlobalStatistics globalStatistics; DataStatisticsOperator( String operatorName, Schema schema, SortOrder sortOrder, OperatorEventGateway operatorEventGateway, - TypeSerializer> statisticsSerializer) { + int downstreamParallelism, + StatisticsType statisticsType) { this.operatorName = operatorName; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); this.operatorEventGateway = operatorEventGateway; - this.statisticsSerializer = statisticsSerializer; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); } @Override public void initializeState(StateInitializationContext context) throws Exception { - localStatistics = statisticsSerializer.createInstance(); - globalStatisticsState = + this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); + this.subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); + + // Use union state so that new subtasks can also restore global statistics during scale-up. + this.globalStatisticsState = context .getOperatorStateStore() .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", statisticsSerializer)); + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); if (context.isRestored()) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); if (globalStatisticsState.get() == null || !globalStatisticsState.get().iterator().hasNext()) { - LOG.warn( + LOG.info( "Operator {} subtask {} doesn't have global statistics state to restore", operatorName, subtaskIndex); - globalStatistics = statisticsSerializer.createInstance(); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); LOG.info( - "Restoring operator {} global statistics state for subtask {}", - operatorName, - subtaskIndex); - globalStatistics = globalStatisticsState.get().iterator().next(); + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; } - } else { - globalStatistics = statisticsSerializer.createInstance(); + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); } + + this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); + this.localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); } @Override public void open() throws Exception { - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } } @Override - @SuppressWarnings("unchecked") public void handleOperatorEvent(OperatorEvent event) { - int subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); Preconditions.checkArgument( - event instanceof DataStatisticsEvent, + event instanceof StatisticsEvent, String.format( "Operator %s subtask %s received unexpected operator event %s", operatorName, subtaskIndex, event.getClass())); - DataStatisticsEvent statisticsEvent = (DataStatisticsEvent) event; + StatisticsEvent statisticsEvent = (StatisticsEvent) event; LOG.info( - "Operator {} received global data event from coordinator checkpoint {}", + "Operator {} subtask {} received global data event from coordinator checkpoint {}", operatorName, + subtaskIndex, statisticsEvent.checkpointId()); - globalStatistics = - DataStatisticsUtil.deserializeDataStatistics( - statisticsEvent.statisticsBytes(), statisticsSerializer); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); + checkStatisticsTypeMigration(); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } } @Override public void processElement(StreamRecord streamRecord) { + // collect data statistics RowData record = streamRecord.getValue(); StructLike struct = rowDataWrapper.wrap(record); sortKey.wrap(struct); localStatistics.add(sortKey); - output.collect(new StreamRecord<>(DataStatisticsOrRecord.fromRecord(record))); + + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); } @Override public void snapshotState(StateSnapshotContext context) throws Exception { long checkpointId = context.getCheckpointId(); - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); LOG.info( - "Snapshotting data statistics operator {} for checkpoint {} in subtask {}", + "Operator {} subtask {} snapshotting data statistics for checkpoint {}", operatorName, - checkpointId, - subTaskId); + subtaskIndex, + checkpointId); - // Pass global statistics to partitioners so that all the operators refresh statistics + // Pass global statistics to partitioner so that all the operators refresh statistics // at same checkpoint barrier - if (!globalStatistics.isEmpty()) { - output.collect( - new StreamRecord<>(DataStatisticsOrRecord.fromDataStatistics(globalStatistics))); + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); } // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores // an exact copy of globalStatistics - if (!globalStatistics.isEmpty() && getRuntimeContext().getIndexOfThisSubtask() == 0) { + if (globalStatistics != null && getRuntimeContext().getIndexOfThisSubtask() == 0) { globalStatisticsState.clear(); LOG.info( - "Saving operator {} global statistics {} to state in subtask {}", - operatorName, - globalStatistics, - subTaskId); + "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); globalStatisticsState.add(globalStatistics); + LOG.debug( + "Operator {} subtask {} saved global statistics to state: {}", + operatorName, + subtaskIndex, + globalStatistics); } // For now, local statistics are sent to coordinator at checkpoint - operatorEventGateway.sendEventToCoordinator( - DataStatisticsEvent.create(checkpointId, localStatistics, statisticsSerializer)); - LOG.debug( - "Subtask {} of operator {} sent local statistics to coordinator at checkpoint{}: {}", - subTaskId, + LOG.info( + "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", operatorName, - checkpointId, - localStatistics); + subtaskIndex, + checkpointId); + operatorEventGateway.sendEventToCoordinator( + StatisticsEvent.createTaskStatisticsEvent( + checkpointId, localStatistics, taskStatisticsSerializer)); // Recreate the local statistics - localStatistics = statisticsSerializer.createInstance(); + localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @SuppressWarnings("unchecked") + private void checkStatisticsTypeMigration() { + // only check if the statisticsType config is Auto and localStatistics is currently Map type + if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { + Map mapStatistics = (Map) localStatistics.result(); + // convert if local statistics has cardinality over the threshold or + // if received global statistics is already sketch type + if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { + LOG.info( + "Operator {} subtask {} switched local statistics from Map to Sketch.", + operatorName, + subtaskIndex); + this.taskStatisticsType = StatisticsType.Sketch; + this.localStatistics = + StatisticsUtil.createTaskStatistics( + taskStatisticsType, parallelism, downstreamParallelism); + SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); + } + } } @VisibleForTesting - DataStatistics localDataStatistics() { + DataStatistics localStatistics() { return localStatistics; } @VisibleForTesting - DataStatistics globalDataStatistics() { + GlobalStatistics globalStatistics() { return globalStatistics; } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java new file mode 100644 index 000000000000..c25481b3c1f2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class DataStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer mapSerializer; + private final SortKeySketchSerializer sketchSerializer; + + DataStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer duplicate() { + TypeSerializer duplicateSortKeySerializer = sortKeySerializer.duplicate(); + return (duplicateSortKeySerializer == sortKeySerializer) + ? this + : new DataStatisticsSerializer(duplicateSortKeySerializer); + } + + @Override + public DataStatistics createInstance() { + return new MapDataStatistics(); + } + + @SuppressWarnings("unchecked") + @Override + public DataStatistics copy(DataStatistics obj) { + StatisticsType statisticsType = obj.type(); + if (statisticsType == StatisticsType.Map) { + MapDataStatistics from = (MapDataStatistics) obj; + Map fromStats = (Map) from.result(); + Map toStats = Maps.newHashMap(fromStats); + return new MapDataStatistics(toStats); + } else if (statisticsType == StatisticsType.Sketch) { + // because ReservoirItemsSketch doesn't expose enough public methods for cloning, + // this implementation adopted the less efficient serialization and deserialization. + SketchDataStatistics from = (SketchDataStatistics) obj; + ReservoirItemsSketch fromStats = (ReservoirItemsSketch) from.result(); + byte[] bytes = fromStats.toByteArray(sketchSerializer); + Memory memory = Memory.wrap(bytes); + ReservoirItemsSketch toStats = + ReservoirItemsSketch.heapify(memory, sketchSerializer); + return new SketchDataStatistics(toStats); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics copy(DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @SuppressWarnings("unchecked") + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + StatisticsType statisticsType = obj.type(); + statisticsTypeSerializer.serialize(obj.type(), target); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = (Map) obj.result(); + mapSerializer.serialize(mapStatistics, target); + } else if (statisticsType == StatisticsType.Sketch) { + ReservoirItemsSketch sketch = (ReservoirItemsSketch) obj.result(); + byte[] sketchBytes = sketch.toByteArray(sketchSerializer); + target.writeInt(sketchBytes.length); + target.write(sketchBytes); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataInputView source) throws IOException { + StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = mapSerializer.deserialize(source); + return new MapDataStatistics(mapStatistics); + } else if (statisticsType == StatisticsType.Sketch) { + int numBytes = source.readInt(); + byte[] sketchBytes = new byte[numBytes]; + source.read(sketchBytes); + Memory sketchMemory = Memory.wrap(sketchBytes); + ReservoirItemsSketch sketch = + ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); + return new SketchDataStatistics(sketch); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsSerializer)) { + return false; + } + + DataStatisticsSerializer other = (DataStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DataStatisticsSerializerSnapshot(this); + } + + public static class DataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DataStatisticsSerializerSnapshot() { + super(DataStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new DataStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java deleted file mode 100644 index 8716cb872d0e..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsUtil.java +++ /dev/null @@ -1,96 +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.sink.shuffle; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * DataStatisticsUtil is the utility to serialize and deserialize {@link DataStatistics} and {@link - * AggregatedStatistics} - */ -class DataStatisticsUtil { - - private DataStatisticsUtil() {} - - static , S> byte[] serializeDataStatistics( - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new IllegalStateException("Fail to serialize data statistics", e); - } - } - - @SuppressWarnings("unchecked") - static , S> D deserializeDataStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return (D) statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new IllegalStateException("Fail to deserialize data statistics", e); - } - } - - static , S> byte[] serializeAggregatedStatistics( - AggregatedStatistics aggregatedStatistics, - TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - ObjectOutputStream out = new ObjectOutputStream(bytes); - - DataOutputSerializer outSerializer = new DataOutputSerializer(64); - out.writeLong(aggregatedStatistics.checkpointId()); - statisticsSerializer.serialize(aggregatedStatistics.dataStatistics(), outSerializer); - byte[] statisticsBytes = outSerializer.getCopyOfBuffer(); - out.writeInt(statisticsBytes.length); - out.write(statisticsBytes); - out.flush(); - - return bytes.toByteArray(); - } - - static , S> - AggregatedStatistics deserializeAggregatedStatistics( - byte[] bytes, TypeSerializer> statisticsSerializer) - throws IOException { - ByteArrayInputStream bytesIn = new ByteArrayInputStream(bytes); - ObjectInputStream in = new ObjectInputStream(bytesIn); - - long completedCheckpointId = in.readLong(); - int statisticsBytesLength = in.readInt(); - byte[] statisticsBytes = new byte[statisticsBytesLength]; - in.readFully(statisticsBytes); - DataInputDeserializer input = - new DataInputDeserializer(statisticsBytes, 0, statisticsBytesLength); - DataStatistics dataStatistics = statisticsSerializer.deserialize(input); - - return new AggregatedStatistics<>(completedCheckpointId, dataStatistics); - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * 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.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..dfb947a84a0c --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,201 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final ListSerializer rangeBoundsSerializer; + private final ListSerializer intsSerializer; + private final ListSerializer longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List assignedSubtasks = intsSerializer.deserialize(source); + List subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() { + super(GlobalStatisticsSerializer.class); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..a164d83ac3b0 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,151 @@ +/* + * 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.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List assignedSubtasks; + private final List subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List assignedSubtasks() { + return assignedSubtasks; + } + + List subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** @return subtask id */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..0abb030c2279 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.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.flink.sink.shuffle; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map keyAssignments; + + MapAssignment(int numPartitions, Map keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map keyAssignments() { + return keyAssignments; + } + + /** + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map assignment( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java index 0b63e2721178..05b943f6046f 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java @@ -19,52 +19,70 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Map; -import org.apache.flink.annotation.Internal; import org.apache.iceberg.SortKey; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Maps; /** MapDataStatistics uses map to count key frequency */ -@Internal -class MapDataStatistics implements DataStatistics> { - private final Map statistics; +class MapDataStatistics implements DataStatistics { + private final Map keyFrequency; MapDataStatistics() { - this.statistics = Maps.newHashMap(); + this.keyFrequency = Maps.newHashMap(); } - MapDataStatistics(Map statistics) { - this.statistics = statistics; + MapDataStatistics(Map keyFrequency) { + this.keyFrequency = keyFrequency; + } + + @Override + public StatisticsType type() { + return StatisticsType.Map; } @Override public boolean isEmpty() { - return statistics.isEmpty(); + return keyFrequency.isEmpty(); } @Override public void add(SortKey sortKey) { - if (statistics.containsKey(sortKey)) { - statistics.merge(sortKey, 1L, Long::sum); + if (keyFrequency.containsKey(sortKey)) { + keyFrequency.merge(sortKey, 1L, Long::sum); } else { // clone the sort key before adding to map because input sortKey object can be reused SortKey copiedKey = sortKey.copy(); - statistics.put(copiedKey, 1L); + keyFrequency.put(copiedKey, 1L); } } @Override - public void merge(MapDataStatistics otherStatistics) { - otherStatistics.statistics().forEach((key, count) -> statistics.merge(key, count, Long::sum)); + public Object result() { + return keyFrequency; } @Override - public Map statistics() { - return statistics; + public String toString() { + return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); } @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("statistics", statistics).toString(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof MapDataStatistics)) { + return false; + } + + MapDataStatistics other = (MapDataStatistics) o; + return Objects.equal(keyFrequency, other.keyFrequency); + } + + @Override + public int hashCode() { + return Objects.hashCode(keyFrequency); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java deleted file mode 100644 index b6cccd0566fc..000000000000 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatisticsSerializer.java +++ /dev/null @@ -1,187 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class MapDataStatisticsSerializer - extends TypeSerializer>> { - private final MapSerializer mapSerializer; - - static MapDataStatisticsSerializer fromSortKeySerializer( - TypeSerializer sortKeySerializer) { - return new MapDataStatisticsSerializer( - new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE)); - } - - MapDataStatisticsSerializer(MapSerializer mapSerializer) { - this.mapSerializer = mapSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer>> duplicate() { - MapSerializer duplicateMapSerializer = - (MapSerializer) mapSerializer.duplicate(); - return (duplicateMapSerializer == mapSerializer) - ? this - : new MapDataStatisticsSerializer(duplicateMapSerializer); - } - - @Override - public MapDataStatistics createInstance() { - return new MapDataStatistics(); - } - - @Override - public MapDataStatistics copy(DataStatistics> obj) { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics from = (MapDataStatistics) obj; - TypeSerializer keySerializer = mapSerializer.getKeySerializer(); - Map newMap = Maps.newHashMapWithExpectedSize(from.statistics().size()); - for (Map.Entry entry : from.statistics().entrySet()) { - SortKey newKey = keySerializer.copy(entry.getKey()); - // no need to copy value since it is just a Long - newMap.put(newKey, entry.getValue()); - } - - return new MapDataStatistics(newMap); - } - - @Override - public DataStatistics> copy( - DataStatistics> from, - DataStatistics> reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize( - DataStatistics> obj, DataOutputView target) - throws IOException { - Preconditions.checkArgument( - obj instanceof MapDataStatistics, "Invalid data statistics type: " + obj.getClass()); - MapDataStatistics mapStatistics = (MapDataStatistics) obj; - mapSerializer.serialize(mapStatistics.statistics(), target); - } - - @Override - public DataStatistics> deserialize(DataInputView source) - throws IOException { - return new MapDataStatistics(mapSerializer.deserialize(source)); - } - - @Override - public DataStatistics> deserialize( - DataStatistics> reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - mapSerializer.copy(source, target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof MapDataStatisticsSerializer)) { - return false; - } - - MapDataStatisticsSerializer other = (MapDataStatisticsSerializer) obj; - return Objects.equals(mapSerializer, other.mapSerializer); - } - - @Override - public int hashCode() { - return mapSerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot>> - snapshotConfiguration() { - return new MapDataStatisticsSerializerSnapshot(this); - } - - public static class MapDataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot< - DataStatistics>, MapDataStatisticsSerializer> { - private static final int CURRENT_VERSION = 1; - - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot() { - super(MapDataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public MapDataStatisticsSerializerSnapshot(MapDataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - MapDataStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.mapSerializer}; - } - - @Override - protected MapDataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - @SuppressWarnings("unchecked") - MapSerializer mapSerializer = - (MapSerializer) nestedSerializers[0]; - return new MapDataStatisticsSerializer(mapSerializer); - } - } -} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java index dde86b5b6047..f36a078c94e0 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -18,29 +18,14 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.TimeUnit; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; -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; -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.Pair; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -61,52 +46,28 @@ class MapRangePartitioner implements Partitioner { private final RowDataWrapper rowDataWrapper; private final SortKey sortKey; - private final Comparator comparator; - private final Map mapStatistics; - private final double closeFileCostInWeightPercentage; + private final MapAssignment mapAssignment; // Counter that tracks how many times a new key encountered // where there is no traffic statistics learned about it. private long newSortKeyCounter; private long lastNewSortKeyLogTimeMilli; - // lazily computed due to the need of numPartitions - private Map assignment; - private NavigableMap sortedStatsWithCloseFileCost; - - MapRangePartitioner( - Schema schema, - SortOrder sortOrder, - MapDataStatistics dataStatistics, - double closeFileCostInWeightPercentage) { - dataStatistics - .statistics() - .entrySet() - .forEach( - entry -> - Preconditions.checkArgument( - entry.getValue() > 0, - "Invalid statistics: weight is 0 for key %s", - entry.getKey())); - + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); - this.mapStatistics = dataStatistics.statistics(); - this.closeFileCostInWeightPercentage = closeFileCostInWeightPercentage; + this.mapAssignment = mapAssignment; this.newSortKeyCounter = 0; this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); } @Override public int partition(RowData row, int numPartitions) { - // assignment table can only be built lazily when first referenced here, - // because number of partitions (downstream subtasks) is needed. - // the numPartitions is not available in the constructor. - Map assignmentMap = assignment(numPartitions); // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = assignmentMap.get(sortKey); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; if (keyAssignment == null) { LOG.trace( "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", @@ -117,271 +78,18 @@ public int partition(RowData row, int numPartitions) { newSortKeyCounter += 1; long now = System.currentTimeMillis(); if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info("Encounter new sort keys in total {} times", newSortKeyCounter); + LOG.info( + "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", + newSortKeyCounter); lastNewSortKeyLogTimeMilli = now; + newSortKeyCounter = 0; } - return (int) (newSortKeyCounter % numPartitions); + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); } - return keyAssignment.select(); - } - - @VisibleForTesting - Map assignment(int numPartitions) { - if (assignment == null) { - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostInWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostInWeightPercentage / 100); - - this.sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostInWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - this.assignment = - buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostInWeight); - } - - return assignment; - } - - @VisibleForTesting - Map mapStatistics() { - return mapStatistics; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map> assignmentInfo() { - Map> assignmentInfo = Maps.newTreeMap(); - assignment.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks.length; ++i) { - int subtaskId = keyAssignment.assignedSubtasks[i]; - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost[i]; - Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - private Map buildAssignment( - int numPartitions, - NavigableMap sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostInWeight) { - Map assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List assignedSubtasks = Lists.newArrayList(); - List subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostInWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostInWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostInWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostInWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostInWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks.clear(); - subtaskWeights.clear(); - currentKey = null; - } - } - - return assignmentMap; - } - - /** Subtask assignment for a key */ - @VisibleForTesting - static class KeyAssignment { - private final int[] assignedSubtasks; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It - * could also be multiple subtasks if the key has heavy weight that should be handled by - * multiple subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the - * keyWeight is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain - * values as [10, 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List assignedSubtasks, - List subtaskWeightsWithCloseFileCost, - long closeFileCostInWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostInWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostInWeight)); - - this.assignedSubtasks = assignedSubtasks.stream().mapToInt(i -> i).toArray(); - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostInWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.length == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks[0]; - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.length, - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks[position]; - } - } - - @Override - public int hashCode() { - return 31 * Arrays.hashCode(assignedSubtasks) - + Arrays.hashCode(subtaskWeightsExcludingCloseCost); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Arrays.equals(assignedSubtasks, that.assignedSubtasks) - && Arrays.equals(subtaskWeightsExcludingCloseCost, that.subtaskWeightsExcludingCloseCost); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsExcludingCloseCost", subtaskWeightsExcludingCloseCost) - .toString(); - } + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java new file mode 100644 index 000000000000..83a9461233d2 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.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.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** The wrapper class */ +@Internal +public class RangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. + // when rescale is detected, operator requests new statistics from coordinator upon + // initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.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.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java new file mode 100644 index 000000000000..35bbb27baf16 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java @@ -0,0 +1,87 @@ +/* + * 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.Arrays; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** MapDataStatistics uses map to count key frequency */ +class SketchDataStatistics implements DataStatistics { + + private final ReservoirItemsSketch sketch; + + SketchDataStatistics(int reservoirSize) { + this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); + } + + SketchDataStatistics(ReservoirItemsSketch sketchStats) { + this.sketch = sketchStats; + } + + @Override + public StatisticsType type() { + return StatisticsType.Sketch; + } + + @Override + public boolean isEmpty() { + return sketch.getNumSamples() == 0; + } + + @Override + public void add(SortKey sortKey) { + // clone the sort key first because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + sketch.update(copiedKey); + } + + @Override + public Object result() { + return sketch; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SketchDataStatistics)) { + return false; + } + + ReservoirItemsSketch otherSketch = ((SketchDataStatistics) o).sketch; + return Objects.equal(sketch.getK(), otherSketch.getK()) + && Objects.equal(sketch.getN(), otherSketch.getN()) + && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java new file mode 100644 index 000000000000..af78271ea5dc --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.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.sink.shuffle; + +import java.util.Arrays; +import java.util.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +class SketchRangePartitioner implements Partitioner { + private final SortKey sortKey; + private final Comparator comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java new file mode 100644 index 000000000000..a58310611e8d --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -0,0 +1,142 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; + +class SketchUtil { + static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; + static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; + static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; + static final int OPERATOR_OVER_SAMPLE_RATIO = 10; + + // switch the statistics tracking from map to sketch if the cardinality of the sort key is over + // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. + static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; + static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; + + private SketchUtil() {} + + /** + * The larger the reservoir size, the more accurate for range bounds calculation and the more + * balanced range distribution. + * + *

    Here are the heuristic rules + *

  • Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the + * range bounds + *
  • Min is 10K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 1M to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @return reservoir size + */ + static int determineCoordinatorReservoirSize(int numPartitions) { + int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; + + if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { + // adjust it up and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); + } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { + // adjust it down and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; + } + + return reservoirSize; + } + + /** + * Determine the sampling reservoir size where operator subtasks collect data statistics. + * + *

    Here are the heuristic rules + *

  • Target size is "coordinator reservoir size * over sampling ration (10) / operator + * parallelism" + *
  • Min is 1K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 100K to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @param operatorParallelism data statistics operator parallelism + * @return reservoir size + */ + static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { + int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); + int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; + return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); + } + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + * + *

    Assumption is that a single key is not dominant enough to span multiple subtasks. + * + * @param numPartitions number of partitions which maps to downstream operator parallelism + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + */ + static SortKey[] rangeBounds( + int numPartitions, Comparator comparator, SortKey[] samples) { + // sort the keys first + Arrays.sort(samples, comparator); + int numCandidates = numPartitions - 1; + SortKey[] candidates = new SortKey[numCandidates]; + int step = (int) Math.ceil((double) samples.length / numPartitions); + int position = step - 1; + int numChosen = 0; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; + // skip duplicate values + if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { + // linear probe for the next distinct value + position += 1; + } else { + candidates[numChosen] = candidate; + position += step; + numChosen += 1; + } + } + + return candidates; + } + + /** This can be a bit expensive since it is quadratic. */ + static void convertMapToSketch( + Map taskMapStats, Consumer sketchConsumer) { + taskMapStats.forEach( + (sortKey, count) -> { + for (int i = 0; i < count; ++i) { + sketchConsumer.accept(sortKey); + } + }); + } +} 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 d03409f2a430..4ddc5a32d6bf 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 @@ -276,13 +276,12 @@ public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot< private Schema schema; private SortOrder sortOrder; - @SuppressWarnings({"checkstyle:RedundantModifier", "WeakerAccess"}) + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) public SortKeySerializerSnapshot() { // this constructor is used when restoring from a checkpoint. } - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". @SuppressWarnings("checkstyle:RedundantModifier") public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { this.schema = schema; @@ -320,8 +319,12 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer newAvroSerializer = (SortKeySerializer) newSerializer; - return resolveSchemaCompatibility(newAvroSerializer.schema, schema); + SortKeySerializer sortKeySerializer = (SortKeySerializer) newSerializer; + if (!sortOrder.sameOrder(sortKeySerializer.sortOrder)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + return resolveSchemaCompatibility(sortKeySerializer.schema, schema); } @Override diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java new file mode 100644 index 000000000000..d6c23f035015 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +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.base.Preconditions; + +/** + * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link + * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The + * implementation is modeled after {@link ArrayOfStringsSerDe} + */ +class SortKeySketchSerializer extends ArrayOfItemsSerDe implements Serializable { + private static final int DEFAULT_SORT_KEY_SIZE = 128; + + private final TypeSerializer itemSerializer; + private final ListSerializer listSerializer; + private final DataInputDeserializer input; + + SortKeySketchSerializer(TypeSerializer itemSerializer) { + this.itemSerializer = itemSerializer; + this.listSerializer = new ListSerializer<>(itemSerializer); + this.input = new DataInputDeserializer(); + } + + @Override + public byte[] serializeToByteArray(SortKey item) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); + itemSerializer.serialize(item, output); + byte[] itemBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[numBytes + Integer.BYTES]; + ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); + ByteArrayUtil.putIntLE(out, 0, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public byte[] serializeToByteArray(SortKey[] items) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); + listSerializer.serialize(Arrays.asList(items), output); + byte[] itemsBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[Integer.BYTES + numBytes]; + ByteArrayUtil.putIntLE(out, 0, numBytes); + System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return new SortKey[0]; + } + + long offset = startingOffset; + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + offset += Integer.BYTES; + + Util.checkBounds(offset, numBytes, mem.getCapacity()); + byte[] sortKeyBytes = new byte[numBytes]; + mem.getByteArray(offset, sortKeyBytes, 0, numBytes); + input.setBuffer(sortKeyBytes); + + try { + List sortKeys = listSerializer.deserialize(input); + SortKey[] array = new SortKey[numItems]; + sortKeys.toArray(array); + input.releaseArrays(); + return array; + } catch (IOException e) { + throw new UncheckedIOException("Failed to deserialize sort key sketch", e); + } + } + + @Override + public int sizeOf(SortKey item) { + return serializeToByteArray(item).length; + } + + @Override + public int sizeOf(Memory mem, long offset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return 0; + } + + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + return Integer.BYTES + numBytes; + } + + @Override + public String toString(SortKey item) { + return item.toString(); + } + + @Override + public Class getClassOfT() { + return SortKey.class; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 58% rename from flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java index 852d2157b8cb..f6fcdb8b16ef 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsEvent.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -27,24 +27,39 @@ * statistics in bytes */ @Internal -class DataStatisticsEvent, S> implements OperatorEvent { +class StatisticsEvent implements OperatorEvent { private static final long serialVersionUID = 1L; private final long checkpointId; private final byte[] statisticsBytes; + private final boolean applyImmediately; - private DataStatisticsEvent(long checkpointId, byte[] statisticsBytes) { + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { this.checkpointId = checkpointId; this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; } - static , S> DataStatisticsEvent create( + static StatisticsEvent createTaskStatisticsEvent( long checkpointId, - DataStatistics dataStatistics, - TypeSerializer> statisticsSerializer) { - return new DataStatisticsEvent<>( + DataStatistics statistics, + TypeSerializer statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. + return new StatisticsEvent( checkpointId, - DataStatisticsUtil.serializeDataStatistics(dataStatistics, statisticsSerializer)); + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); + } + + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer statisticsSerializer, + boolean applyImmediately) { + return new StatisticsEvent( + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); } long checkpointId() { @@ -54,4 +69,8 @@ long checkpointId() { byte[] statisticsBytes() { return statisticsBytes; } + + boolean applyImmediately() { + return applyImmediately; + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 66% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java index 889e85112e16..bc28df2b0e22 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecord.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.io.Serializable; +import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.table.data.RowData; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -34,68 +35,66 @@ * After shuffling, a filter and mapper are required to filter out the data distribution weight, * unwrap the object and extract the original record type T. */ -class DataStatisticsOrRecord, S> implements Serializable { +@Internal +public class StatisticsOrRecord implements Serializable { private static final long serialVersionUID = 1L; - private DataStatistics statistics; + private GlobalStatistics statistics; private RowData record; - private DataStatisticsOrRecord(DataStatistics statistics, RowData record) { + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { Preconditions.checkArgument( record != null ^ statistics != null, "DataStatistics or record, not neither or both"); this.statistics = statistics; this.record = record; } - static , S> DataStatisticsOrRecord fromRecord( - RowData record) { - return new DataStatisticsOrRecord<>(null, record); + static StatisticsOrRecord fromRecord(RowData record) { + return new StatisticsOrRecord(null, record); } - static , S> DataStatisticsOrRecord fromDataStatistics( - DataStatistics statistics) { - return new DataStatisticsOrRecord<>(statistics, null); + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { + return new StatisticsOrRecord(statistics, null); } - static , S> DataStatisticsOrRecord reuseRecord( - DataStatisticsOrRecord reuse, TypeSerializer recordSerializer) { + static StatisticsOrRecord reuseRecord( + StatisticsOrRecord reuse, TypeSerializer recordSerializer) { if (reuse.hasRecord()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } } - static , S> DataStatisticsOrRecord reuseStatistics( - DataStatisticsOrRecord reuse, - TypeSerializer> statisticsSerializer) { - if (reuse.hasDataStatistics()) { + static StatisticsOrRecord reuseStatistics( + StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { + if (reuse.hasStatistics()) { return reuse; } else { // not reusable - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.createInstance()); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); } } - boolean hasDataStatistics() { + boolean hasStatistics() { return statistics != null; } - boolean hasRecord() { + public boolean hasRecord() { return record != null; } - DataStatistics dataStatistics() { + GlobalStatistics statistics() { return statistics; } - void dataStatistics(DataStatistics newStatistics) { + void statistics(GlobalStatistics newStatistics) { this.statistics = newStatistics; } - RowData record() { + public RowData record() { return record; } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 53% rename from flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java rename to flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java index e9a6fa0cbfc5..6e403425938d 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOrRecordSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -29,13 +29,12 @@ import org.apache.flink.table.data.RowData; @Internal -class DataStatisticsOrRecordSerializer, S> - extends TypeSerializer> { - private final TypeSerializer> statisticsSerializer; +class StatisticsOrRecordSerializer extends TypeSerializer { + private final TypeSerializer statisticsSerializer; private final TypeSerializer recordSerializer; - DataStatisticsOrRecordSerializer( - TypeSerializer> statisticsSerializer, + StatisticsOrRecordSerializer( + TypeSerializer statisticsSerializer, TypeSerializer recordSerializer) { this.statisticsSerializer = statisticsSerializer; this.recordSerializer = recordSerializer; @@ -48,13 +47,13 @@ public boolean isImmutableType() { @SuppressWarnings("ReferenceEquality") @Override - public TypeSerializer> duplicate() { - TypeSerializer> duplicateStatisticsSerializer = + public TypeSerializer duplicate() { + TypeSerializer duplicateStatisticsSerializer = statisticsSerializer.duplicate(); TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); if ((statisticsSerializer != duplicateStatisticsSerializer) || (recordSerializer != duplicateRowDataSerializer)) { - return new DataStatisticsOrRecordSerializer<>( + return new StatisticsOrRecordSerializer( duplicateStatisticsSerializer, duplicateRowDataSerializer); } else { return this; @@ -62,34 +61,31 @@ public TypeSerializer> duplicate() { } @Override - public DataStatisticsOrRecord createInstance() { + public StatisticsOrRecord createInstance() { // arbitrarily always create RowData value instance - return DataStatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); } @Override - public DataStatisticsOrRecord copy(DataStatisticsOrRecord from) { + public StatisticsOrRecord copy(StatisticsOrRecord from) { if (from.hasRecord()) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); } else { - return DataStatisticsOrRecord.fromDataStatistics( - statisticsSerializer.copy(from.dataStatistics())); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); } } @Override - public DataStatisticsOrRecord copy( - DataStatisticsOrRecord from, DataStatisticsOrRecord reuse) { - DataStatisticsOrRecord to; + public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { + StatisticsOrRecord to; if (from.hasRecord()) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.copy(from.record(), to.record()); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.copy(from.dataStatistics(), to.dataStatistics()); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); + to.statistics(statistics); } return to; @@ -101,41 +97,40 @@ public int getLength() { } @Override - public void serialize(DataStatisticsOrRecord statisticsOrRecord, DataOutputView target) + public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) throws IOException { if (statisticsOrRecord.hasRecord()) { target.writeBoolean(true); recordSerializer.serialize(statisticsOrRecord.record(), target); } else { target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.dataStatistics(), target); + statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); } } @Override - public DataStatisticsOrRecord deserialize(DataInputView source) throws IOException { + public StatisticsOrRecord deserialize(DataInputView source) throws IOException { boolean isRecord = source.readBoolean(); if (isRecord) { - return DataStatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); } else { - return DataStatisticsOrRecord.fromDataStatistics(statisticsSerializer.deserialize(source)); + return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); } } @Override - public DataStatisticsOrRecord deserialize( - DataStatisticsOrRecord reuse, DataInputView source) throws IOException { - DataStatisticsOrRecord to; + public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) + throws IOException { + StatisticsOrRecord to; boolean isRecord = source.readBoolean(); if (isRecord) { - to = DataStatisticsOrRecord.reuseRecord(reuse, recordSerializer); + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); RowData record = recordSerializer.deserialize(to.record(), source); to.record(record); } else { - to = DataStatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - DataStatistics statistics = - statisticsSerializer.deserialize(to.dataStatistics(), source); - to.dataStatistics(statistics); + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + to.statistics(statistics); } return to; @@ -154,12 +149,11 @@ public void copy(DataInputView source, DataOutputView target) throws IOException @Override public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsOrRecordSerializer)) { + if (!(obj instanceof StatisticsOrRecordSerializer)) { return false; } - @SuppressWarnings("unchecked") - DataStatisticsOrRecordSerializer other = (DataStatisticsOrRecordSerializer) obj; + StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; return Objects.equals(statisticsSerializer, other.statisticsSerializer) && Objects.equals(recordSerializer, other.recordSerializer); } @@ -170,25 +164,22 @@ public int hashCode() { } @Override - public TypeSerializerSnapshot> snapshotConfiguration() { - return new DataStatisticsOrRecordSerializerSnapshot<>(this); + public TypeSerializerSnapshot snapshotConfiguration() { + return new StatisticsOrRecordSerializerSnapshot(this); } - public static class DataStatisticsOrRecordSerializerSnapshot, S> - extends CompositeTypeSerializerSnapshot< - DataStatisticsOrRecord, DataStatisticsOrRecordSerializer> { + public static class StatisticsOrRecordSerializerSnapshot + extends CompositeTypeSerializerSnapshot { private static final int CURRENT_VERSION = 1; - // constructors need to public. Otherwise, Flink state restore would complain - // "The class has no (implicit) public nullary constructor". - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot() { - super(DataStatisticsOrRecordSerializer.class); + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public StatisticsOrRecordSerializerSnapshot() { + super(StatisticsOrRecordSerializer.class); } @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsOrRecordSerializerSnapshot( - DataStatisticsOrRecordSerializer serializer) { + public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { super(serializer); } @@ -200,7 +191,7 @@ protected int getCurrentOuterSnapshotVersion() { @Override protected TypeSerializer[] getNestedSerializers( - DataStatisticsOrRecordSerializer outerSerializer) { + StatisticsOrRecordSerializer outerSerializer) { return new TypeSerializer[] { outerSerializer.statisticsSerializer, outerSerializer.recordSerializer }; @@ -208,12 +199,12 @@ protected TypeSerializer[] getNestedSerializers( @SuppressWarnings("unchecked") @Override - protected DataStatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( + protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( TypeSerializer[] nestedSerializers) { - TypeSerializer> statisticsSerializer = - (TypeSerializer>) nestedSerializers[0]; + TypeSerializer statisticsSerializer = + (TypeSerializer) nestedSerializers[0]; TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; - return new DataStatisticsOrRecordSerializer<>(statisticsSerializer, recordSerializer); + return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); } } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java new file mode 100644 index 000000000000..43f72e336e06 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +/** + * Range distribution requires gathering statistics on the sort keys to determine proper range + * boundaries to distribute/cluster rows before writer operators. + */ +public enum StatisticsType { + /** + * Tracks the data statistics as {@code Map} frequency. It works better for + * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in + * hundreds or thousands. + * + *

      + *
    • Pro: accurate measurement on the statistics/weight of every key. + *
    • Con: memory footprint can be large if the key cardinality is high. + *
    + */ + Map, + + /** + * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds + * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, + * uuid etc.) where the cardinalities can be in millions or billions. + * + *
      + *
    • Pro: relatively low memory footprint for high-cardinality sort keys. + *
    • Con: non-precise approximation with potentially lower accuracy. + *
    + */ + Sketch, + + /** + * Initially use Map for statistics tracking. If key cardinality turns out to be high, + * automatically switch to sketch sampling. + */ + Auto +} 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 new file mode 100644 index 000000000000..5d48ec57ca49 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.UncheckedIOException; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class StatisticsUtil { + + private StatisticsUtil() {} + + static DataStatistics createTaskStatistics( + StatisticsType type, int operatorParallelism, int numPartitions) { + if (type == StatisticsType.Map) { + return new MapDataStatistics(); + } else { + return new SketchDataStatistics( + SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); + } + } + + static byte[] serializeDataStatistics( + DataStatistics dataStatistics, TypeSerializer statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize data statistics", e); + } + } + + static DataStatistics deserializeDataStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize data statistics", e); + } + } + + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(completedStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static StatisticsType collectType(StatisticsType config) { + return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; + } + + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + + static StatisticsType collectType( + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java new file mode 100644 index 000000000000..5910bd685510 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -0,0 +1,100 @@ +/* + * 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.Comparator; +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +class Fixtures { + private Fixtures() {} + + public static final int NUM_SUBTASKS = 2; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); + public static final TypeSerializer ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); + public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + public static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); + public static final SortKeySerializer SORT_KEY_SERIALIZER = + new SortKeySerializer(SCHEMA, SORT_ORDER); + public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = + new DataStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); + + public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + public static final Map CHAR_KEYS = createCharKeys(); + + public static StatisticsEvent createStatisticsEvent( + StatisticsType type, + TypeSerializer statisticsSerializer, + long checkpointId, + SortKey... keys) { + DataStatistics statistics = createTaskStatistics(type, keys); + return StatisticsEvent.createTaskStatisticsEvent( + checkpointId, statistics, statisticsSerializer); + } + + public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { + DataStatistics statistics; + if (type == StatisticsType.Sketch) { + statistics = new SketchDataStatistics(128); + } else { + statistics = new MapDataStatistics(); + } + + for (SortKey key : keys) { + statistics.add(key); + } + + return statistics; + } + + private static Map createCharKeys() { + Map keys = Maps.newHashMap(); + for (char c = 'a'; c <= 'z'; ++c) { + String key = Character.toString(c); + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, key); + keys.put(key, sortKey); + } + + return keys; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java deleted file mode 100644 index 739cf764e2a6..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatistics.java +++ /dev/null @@ -1,63 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestAggregatedStatistics { - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - @Test - public void mergeDataStatisticTest() { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - - AggregatedStatistics> aggregatedStatistics = - new AggregatedStatistics<>(1, statisticsSerializer); - MapDataStatistics mapDataStatistics1 = new MapDataStatistics(); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyA); - mapDataStatistics1.add(keyB); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics1); - MapDataStatistics mapDataStatistics2 = new MapDataStatistics(); - mapDataStatistics2.add(keyA); - aggregatedStatistics.mergeDataStatistic("testOperator", 1, mapDataStatistics2); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyA)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyA) + mapDataStatistics2.statistics().get(keyA)); - assertThat(aggregatedStatistics.dataStatistics().statistics().get(keyB)) - .isEqualTo( - mapDataStatistics1.statistics().get(keyB) - + mapDataStatistics2.statistics().getOrDefault(keyB, 0L)); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java index 0064c91340bf..8322ce683768 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java @@ -18,161 +18,448 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; -import java.util.Map; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.BeforeEach; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestAggregatedStatisticsTracker { - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - private final SortKey keyA = sortKey.copy(); - private final SortKey keyB = sortKey.copy(); - - private AggregatedStatisticsTracker> - aggregatedStatisticsTracker; - - public TestAggregatedStatisticsTracker() { - keyA.set(0, "a"); - keyB.set(0, "b"); - } + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveNewerStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); - @BeforeEach - public void before() throws Exception { - aggregatedStatisticsTracker = - new AggregatedStatisticsTracker<>("testOperator", statisticsSerializer, NUM_SUBTASKS); - } + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } - @Test - public void receiveNewerDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - // Checkpoint 2 is newer than checkpoint1, thus dropping in progress statistics for checkpoint1 - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + // checkpoint 1 is completed + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + // checkpoint 2 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } } - @Test - public void receiveOlderDataStatisticEventTest() { - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - checkpoint2Subtask0DataStatistic.add(keyB); - checkpoint2Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint3Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint3Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - // Receive event from old checkpoint, aggregatedStatisticsAggregatorTracker won't return - // completed statistics and in progress statistics won't be updated - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent)) - .isNull(); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()).isEqualTo(2); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveOlderStatisticsEventTest(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint3Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); + // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 4L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + } } - @Test - public void receiveCompletedDataStatisticEvent() { - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint1Subtask0DataStatisticEvent)) - .isNull(); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyA); - checkpoint1Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveCompletedStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + // Receive data statistics from all subtasks at checkpoint 1 - AggregatedStatistics> completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint1Subtask1DataStatisticEvent); + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) completedStatistics.dataStatistics(); - assertThat((long) globalDataStatistics.statistics().get(keyA)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyA) - + checkpoint1Subtask1DataStatistic.statistics().get(keyA)); - assertThat((long) globalDataStatistics.statistics().get(keyB)) - .isEqualTo( - checkpoint1Subtask0DataStatistic.statistics().get(keyB) - + checkpoint1Subtask1DataStatistic.statistics().get(keyB)); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); - - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyA); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - assertThat( - aggregatedStatisticsTracker.updateAndCheckCompletion( - 0, checkpoint2Subtask0DataStatisticEvent)) - .isNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(1); - - MapDataStatistics checkpoint2Subtask1DataStatistic = new MapDataStatistics(); - checkpoint2Subtask1DataStatistic.add(keyB); - DataStatisticsEvent> - checkpoint2Subtask1DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask1DataStatistic, statisticsSerializer); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 3L, + CHAR_KEYS.get("b"), 3L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint2Subtask0DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); + completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask1DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); // Receive data statistics from all subtasks at checkpoint 2 completedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion( - 1, checkpoint2Subtask1DataStatisticEvent); + tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + } + + @Test + public void coordinatorSwitchToSketchOverThreshold() { + int parallelism = 3; + int downstreamParallelism = 3; + int switchToSketchThreshold = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + switchToSketchThreshold, + null); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + // converted to sketch statistics as map size is 4 (over the switch threshold of 3) + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2); - assertThat(aggregatedStatisticsTracker.inProgressStatistics().checkpointId()) - .isEqualTo(completedStatistics.checkpointId() + 1); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + @Test + public void coordinatorMapOperatorSketch() { + int parallelism = 3; + int downstreamParallelism = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + + // first operator event has map statistics + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + // second operator event contains sketch statistics + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Sketch, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + // third operator event has Map statistics + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + private AggregatedStatisticsTracker createTracker(StatisticsType type) { + return new AggregatedStatisticsTracker( + "testOperator", + Fixtures.NUM_SUBTASKS, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); } } 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 new file mode 100644 index 000000000000..4ee9888934a8 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -0,0 +1,54 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestCompletedStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return CompletedStatistics.class; + } + + @Override + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( + 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} 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 849253564209..a08a8a73e80c 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 @@ -18,9 +18,13 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.time.Duration; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; @@ -28,128 +32,182 @@ import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; +import org.awaitility.Awaitility; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinator { private static final String OPERATOR_NAME = "TestCoordinator"; private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - private static final int NUM_SUBTASKS = 2; - - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); private EventReceivingTasks receivingTasks; - private DataStatisticsCoordinator> - dataStatisticsCoordinator; @BeforeEach public void before() throws Exception { receivingTasks = EventReceivingTasks.createForRunningTasks(); - dataStatisticsCoordinator = - new DataStatisticsCoordinator<>( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - statisticsSerializer); } - private void tasksReady() throws Exception { - dataStatisticsCoordinator.start(); - setAllTasksReady(NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + private void tasksReady(DataStatisticsCoordinator coordinator) { + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); } - @Test - public void testThrowExceptionWhenNotStarted() { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - DataStatisticsEvent.create(0, new MapDataStatistics(), statisticsSerializer))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + StatisticsEvent.createTaskStatisticsEvent( + 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + 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, + CHAR_KEYS.get("a"), + 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( + CHAR_KEYS.get("a"), 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( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + 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 testDataStatisticsEventHandling() throws Exception { - tasksReady(); - SortKey key = sortKey.copy(); - - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask0DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); - - MapDataStatistics checkpoint1Subtask1DataStatistic = new MapDataStatistics(); - key.set(0, "a"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "b"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - key.set(0, "c"); - checkpoint1Subtask1DataStatistic.add(key); - - DataStatisticsEvent> - checkpoint1Subtask1DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask1DataStatistic, statisticsSerializer); - - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify global data statistics is the aggregation of all subtasks data statistics - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - MapDataStatistics globalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(globalDataStatistics.statistics()) - .containsExactlyInAnyOrderEntriesOf( - ImmutableMap.of( - keyA, 2L, - keyB, 3L, - keyC, 5L)); + public void testRequestGlobalStatisticsEventHandling() throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); + } } static void setAllTasksReady( int subtasks, - DataStatisticsCoordinator> dataStatisticsCoordinator, + DataStatisticsCoordinator dataStatisticsCoordinator, EventReceivingTasks receivingTasks) { for (int i = 0; i < subtasks; i++) { dataStatisticsCoordinator.executionAttemptReady( @@ -157,8 +215,7 @@ static void setAllTasksReady( } } - static void waitForCoordinatorToProcessActions( - DataStatisticsCoordinator> coordinator) { + static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { CompletableFuture future = new CompletableFuture<>(); coordinator.callInCoordinatorThread( () -> { @@ -175,4 +232,15 @@ static void waitForCoordinatorToProcessActions( ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); } } + + private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { + return new DataStatisticsCoordinator( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + NUM_SUBTASKS, + type, + 0.0d); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java index c5216eeb712a..6317f2bfde18 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java @@ -18,6 +18,10 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; @@ -27,117 +31,157 @@ import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.types.Types; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; public class TestDataStatisticsCoordinatorProvider { private static final OperatorID OPERATOR_ID = new OperatorID(); - private static final int NUM_SUBTASKS = 1; - private final Schema schema = - new Schema(Types.NestedField.optional(1, "str", Types.StringType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final MapDataStatisticsSerializer statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer(new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsCoordinatorProvider> provider; private EventReceivingTasks receivingTasks; @BeforeEach public void before() { - provider = - new DataStatisticsCoordinatorProvider<>( - "DataStatisticsCoordinatorProvider", OPERATOR_ID, statisticsSerializer); receivingTasks = EventReceivingTasks.createForRunningTasks(); } - @Test - @SuppressWarnings("unchecked") - public void testCheckpointAndReset() throws Exception { - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - SortKey keyD = sortKey.copy(); - keyD.set(0, "c"); - SortKey keyE = sortKey.copy(); - keyE.set(0, "c"); - + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testCheckpointAndReset(StatisticsType type) throws Exception { + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); try (RecreateOnResetOperatorCoordinator coordinator = (RecreateOnResetOperatorCoordinator) - provider.create(new MockOperatorCoordinatorContext(OPERATOR_ID, NUM_SUBTASKS))) { - DataStatisticsCoordinator> dataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); + provider.create( + new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { + DataStatisticsCoordinator dataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); // Start the coordinator coordinator.start(); TestDataStatisticsCoordinator.setAllTasksReady( - NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - MapDataStatistics checkpoint1Subtask0DataStatistic = new MapDataStatistics(); - checkpoint1Subtask0DataStatistic.add(keyA); - checkpoint1Subtask0DataStatistic.add(keyB); - checkpoint1Subtask0DataStatistic.add(keyC); - DataStatisticsEvent> - checkpoint1Subtask0DataStatisticEvent = - DataStatisticsEvent.create(1, checkpoint1Subtask0DataStatistic, statisticsSerializer); + Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); // Handle events from operators for checkpoint 1 - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0DataStatisticEvent); + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 1 global data statistics - MapDataStatistics checkpoint1GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint1GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint1Subtask0DataStatistic.statistics()); + Map checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - MapDataStatistics checkpoint2Subtask0DataStatistic = new MapDataStatistics(); - checkpoint2Subtask0DataStatistic.add(keyD); - checkpoint2Subtask0DataStatistic.add(keyE); - checkpoint2Subtask0DataStatistic.add(keyE); - DataStatisticsEvent> - checkpoint2Subtask0DataStatisticEvent = - DataStatisticsEvent.create(2, checkpoint2Subtask0DataStatistic, statisticsSerializer); - // Handle events from operators for checkpoint 2 - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0DataStatisticEvent); + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); + coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + // Verify checkpoint 2 global data statistics - MapDataStatistics checkpoint2GlobalDataStatistics = - (MapDataStatistics) dataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(checkpoint2GlobalDataStatistics.statistics()) - .isEqualTo(checkpoint2Subtask0DataStatistic.statistics()); + Map checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + } + waitForCheckpoint(2L, dataStatisticsCoordinator); // Reset coordinator to checkpoint 1 coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator> - restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator>) - coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotEqualTo(restoredDataStatisticsCoordinator); + DataStatisticsCoordinator restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); // Verify restored data statistics - MapDataStatistics restoredAggregateDataStatistics = - (MapDataStatistics) - restoredDataStatisticsCoordinator.completedStatistics().dataStatistics(); - assertThat(restoredAggregateDataStatistics.statistics()) - .isEqualTo(checkpoint1GlobalDataStatistics.statistics()); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } } } - private byte[] waitForCheckpoint( - long checkpointId, - DataStatisticsCoordinator> coordinator) + private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) throws InterruptedException, ExecutionException { CompletableFuture future = new CompletableFuture<>(); coordinator.checkpointCoordinator(checkpointId, future); return future.get(); } + + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { + return new DataStatisticsCoordinatorProvider( + "DataStatisticsCoordinatorProvider", + OPERATOR_ID, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + type, + 0.0); + } } 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 5e6f971807ba..c760f1ba96d3 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 @@ -18,22 +18,25 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; import java.util.Collections; import java.util.List; import java.util.Map; import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.datasketches.sampling.ReservoirItemsSketch; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.configuration.Configuration; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.runtime.execution.Environment; import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; import org.apache.flink.runtime.state.AbstractStateBackend; import org.apache.flink.runtime.state.StateInitializationContext; @@ -49,102 +52,95 @@ import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; 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.types.Types; -import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; 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.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; public class TestDataStatisticsOperator { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("id").build(); - private final SortKey sortKey = new SortKey(schema, sortOrder); - private final RowType rowType = RowType.of(new VarCharType(), new IntType()); - private final TypeSerializer rowSerializer = new RowDataSerializer(rowType); - private final TypeSerializer>> - statisticsSerializer = - MapDataStatisticsSerializer.fromSortKeySerializer( - new SortKeySerializer(schema, sortOrder)); - - private DataStatisticsOperator> operator; - - private Environment getTestingEnvironment() { - return new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } + + private Environment env; @BeforeEach public void before() throws Exception { - this.operator = createOperator(); - Environment env = getTestingEnvironment(); - this.operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); + this.env = + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); } - private DataStatisticsOperator> createOperator() { + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return new DataStatisticsOperator<>( - "testOperator", schema, sortOrder, mockGateway, statisticsSerializer); + return createOperator(type, downstreamParallelism, mockGateway); } - @AfterEach - public void clean() throws Exception { - operator.close(); + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { + DataStatisticsOperator operator = + new DataStatisticsOperator( + "testOperator", + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + mockGateway, + downstreamParallelism, + type); + operator.setup( + new OneInputStreamTask(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + return operator; } - @Test - public void testProcessElement() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { + @SuppressWarnings("unchecked") + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElement(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(StringData.fromString("a"), 5))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - assertThat(operator.localDataStatistics()).isInstanceOf(MapDataStatistics.class); - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L); - - MapDataStatistics mapDataStatistics = (MapDataStatistics) operator.localDataStatistics(); - Map statsMap = mapDataStatistics.statistics(); - assertThat(statsMap).hasSize(2); - assertThat(statsMap).containsExactlyInAnyOrderEntriesOf(expectedMap); + DataStatistics localStatistics = operator.localStatistics(); + assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + Map keyFrequency = (Map) localStatistics.result(); + assertThat(keyFrequency) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); + } else { + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) localStatistics.result(); + assertThat(sketch.getSamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } testHarness.endInput(); } } - @Test - public void testOperatorOutput() throws Exception { - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness = createHarness(this.operator)) { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testOperatorOutput(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { testHarness.processElement( new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); testHarness.processElement( @@ -154,8 +150,8 @@ public void testOperatorOutput() throws Exception { List recordsOutput = testHarness.extractOutputValues().stream() - .filter(DataStatisticsOrRecord::hasRecord) - .map(DataStatisticsOrRecord::record) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record) .collect(Collectors.toList()); assertThat(recordsOutput) .containsExactlyInAnyOrderElementsOf( @@ -166,70 +162,172 @@ public void testOperatorOutput() throws Exception { } } - @Test - public void testRestoreState() throws Exception { + private static Stream provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + + @ParameterizedTest + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { + Map keyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); + SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness1 = createHarness(this.operator)) { - MapDataStatistics mapDataStatistics = new MapDataStatistics(); - - SortKey key = sortKey.copy(); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "a"); - mapDataStatistics.add(key); - key.set(0, "b"); - mapDataStatistics.add(key); - key.set(0, "c"); - mapDataStatistics.add(key); - - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); - - DataStatisticsEvent> event = - DataStatisticsEvent.create(0, mapDataStatistics, statisticsSerializer); + try (OneInputStreamOperatorTestHarness testHarness1 = + createHarness(operator)) { + GlobalStatistics statistics; + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); + } else { + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); + } + + StatisticsEvent event = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); operator.handleOperatorEvent(event); - assertThat(operator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - assertThat(operator.globalDataStatistics().statistics()) - .containsExactlyInAnyOrderEntriesOf(expectedMap); + + GlobalStatistics globalStatistics = operator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + snapshot = testHarness1.snapshot(1L, 0); } // Use the snapshot to initialize state for another new operator and then verify that the global // statistics for the new operator is same as before - DataStatisticsOperator> restoredOperator = - createOperator(); - try (OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - testHarness2 = new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); + try (OneInputStreamOperatorTestHarness testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { testHarness2.setup(); testHarness2.initializeState(snapshot); - assertThat(restoredOperator.globalDataStatistics()).isInstanceOf(MapDataStatistics.class); - // restored RowData is BinaryRowData. convert to GenericRowData for comparison - Map restoredStatistics = Maps.newHashMap(); - restoredStatistics.putAll(restoredOperator.globalDataStatistics().statistics()); + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + } + } - SortKey keyA = sortKey.copy(); - keyA.set(0, "a"); - SortKey keyB = sortKey.copy(); - keyB.set(0, "b"); - SortKey keyC = sortKey.copy(); - keyC.set(0, "c"); - Map expectedMap = ImmutableMap.of(keyA, 2L, keyB, 1L, keyC, 1L); + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithLocalStatsOverThreshold() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // add rows with unique keys + for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()).hasSize(i + 1); + } + + // one more item should trigger the migration to sketch stats + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); + // reservoir not full yet + assertThat(sketch.getN()).isLessThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); + + // add more items to saturate the reservoir + for (int i = 0; i < reservoirSize; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + } + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + sketch = (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()) + .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); + // reservoir is full now + assertThat(sketch.getN()).isGreaterThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize(reservoirSize); + + testHarness.endInput(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithGlobalSketchStatistics() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); - assertThat(restoredStatistics).containsExactlyInAnyOrderEntriesOf(expectedMap); + // started with Map stype + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + + // received global statistics with sketch type + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( + 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); + operator.handleOperatorEvent( + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(1); + assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); + + testHarness.endInput(); } } private StateInitializationContext getStateContext() throws Exception { - MockEnvironment env = new MockEnvironmentBuilder().build(); AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); OperatorStateStore operatorStateStore = @@ -238,17 +336,14 @@ private StateInitializationContext getStateContext() throws Exception { return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); } - private OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - createHarness( - final DataStatisticsOperator> - dataStatisticsOperator) - throws Exception { - - OneInputStreamOperatorTestHarness< - RowData, DataStatisticsOrRecord>> - harness = new OneInputStreamOperatorTestHarness<>(dataStatisticsOperator, 1, 1, 0); - harness.setup(new DataStatisticsOrRecordSerializer<>(statisticsSerializer, rowSerializer)); + private OneInputStreamOperatorTestHarness createHarness( + DataStatisticsOperator dataStatisticsOperator) throws Exception { + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>( + dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); + harness.setup( + new StatisticsOrRecordSerializer( + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); harness.open(); return harness; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java new file mode 100644 index 000000000000..59ce6df05d9d --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class TestDataStatisticsSerializer extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return Fixtures.TASK_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return DataStatistics.class; + } + + @Override + protected DataStatistics[] getTestData() { + return new DataStatistics[] { + new MapDataStatistics(), + Fixtures.createTaskStatistics( + StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), + new SketchDataStatistics(128), + Fixtures.createTaskStatistics( + StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) + }; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.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.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java index be2beeebc93c..8a25c7ad9898 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java @@ -18,74 +18,50 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; import static org.assertj.core.api.Assertions.assertThat; import java.util.Map; import org.apache.flink.table.data.GenericRowData; import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.Test; public class TestMapDataStatistics { - private final SortOrder sortOrder = SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - private final SortKey sortKey = new SortKey(TestFixtures.SCHEMA, sortOrder); - private final RowType rowType = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private final RowDataWrapper rowWrapper = - new RowDataWrapper(rowType, TestFixtures.SCHEMA.asStruct()); - + @SuppressWarnings("unchecked") @Test public void testAddsAndGet() { MapDataStatistics dataStatistics = new MapDataStatistics(); - GenericRowData reusedRow = - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("c")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("a")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); reusedRow.setField(0, StringData.fromString("b")); - sortKey.wrap(rowWrapper.wrap(reusedRow)); - dataStatistics.add(sortKey); - - Map actual = dataStatistics.statistics(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("a"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyA = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("b"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyB = sortKey.copy(); - - rowWrapper.wrap( - GenericRowData.of(StringData.fromString("c"), 1, StringData.fromString("2023-06-20"))); - sortKey.wrap(rowWrapper); - SortKey keyC = sortKey.copy(); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); - Map expected = ImmutableMap.of(keyA, 2L, keyB, 3L, keyC, 1L); + Map actual = (Map) dataStatistics.result(); + Map expected = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); assertThat(actual).isEqualTo(expected); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index e6726e7db785..d5a0bebc74e7 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import java.util.List; @@ -64,65 +65,60 @@ private static SortKey[] initSortKeys() { } // Total weight is 800 - private final MapDataStatistics mapDataStatistics = - new MapDataStatistics( - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L)); + private final Map mapStatistics = + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L); @Test public void testEvenlyDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // each task should get targeted weight of 100 (=800/8) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -144,19 +140,20 @@ public void testEvenlyDividableNoClosingFileCost() { Pair.of(100L, 1), 7, Pair.of(100L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testEvenlyDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 100 before close file cost factored in. // close file cost is 5 = 5% * 100. @@ -165,35 +162,30 @@ public void testEvenlyDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight with close cost per subtask is 110 (880/8) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight (excluding close file cost) for the subtask, @@ -216,51 +208,48 @@ public void testEvenlyDividableWithClosingFileCost() { Pair.of(100L, 2), 7, Pair.of(75L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableNoClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 0.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 // each task should get targeted weight of 89 = ceiling(800/9) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight, second is the number of assigned keys @@ -284,19 +273,20 @@ public void testNonDividableNoClosingFileCost() { Pair.of(89L, 2), 8, Pair.of(88L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); } @Test public void testNonDividableWithClosingFileCost() { - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapDataStatistics, 5.0); int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); // target subtask weight is 89 before close file cost factored in. // close file cost is 5 (= 5% * 89) per file. @@ -305,35 +295,31 @@ public void testNonDividableWithClosingFileCost() { // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 // target subtask weight per subtask is 98 ceiling(880/9) - Map expectedAssignment = + Map expectedAssignment = ImmutableMap.of( SORT_KEYS[0], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), SORT_KEYS[1], - new MapRangePartitioner.KeyAssignment( + new KeyAssignment( ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), SORT_KEYS[2], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), SORT_KEYS[3], - new MapRangePartitioner.KeyAssignment( - ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), SORT_KEYS[4], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[5], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[6], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[7], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[8], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), SORT_KEYS[9], - new MapRangePartitioner.KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - Map actualAssignment = - partitioner.assignment(numPartitions); - assertThat(actualAssignment).isEqualTo(expectedAssignment); + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); // key: subtask id // value pair: first is the assigned weight for the subtask, second is the number of keys @@ -358,40 +344,39 @@ public void testNonDividableWithClosingFileCost() { Pair.of(88L, 2), 8, Pair.of(61L, 7)); - Map> actualAssignmentInfo = partitioner.assignmentInfo(); - assertThat(actualAssignmentInfo).isEqualTo(expectedAssignmentInfo); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); Map>> partitionResults = - runPartitioner(partitioner, numPartitions); + runPartitioner(partitioner, numPartitions, mapStatistics); // drift threshold is high for non-dividable scenario with close cost validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); } private static Map>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions) { + MapRangePartitioner partitioner, int numPartitions, Map mapStatistics) { // The Map key is the subtaskId. // For the map value pair, the first element is the count of assigned and // the second element of Set is for the set of assigned keys. Map>> partitionResults = Maps.newHashMap(); - partitioner - .mapStatistics() - .forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); return partitionResults; } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java new file mode 100644 index 000000000000..396bfae2f13c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + SketchDataStatistics dataStatistics = new SketchDataStatistics(128); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + ReservoirItemsSketch actual = (ReservoirItemsSketch) dataStatistics.result(); + assertThat(actual.getSamples()) + .isEqualTo( + new SortKey[] { + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") + }); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java new file mode 100644 index 000000000000..31dae5c76aeb --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -0,0 +1,133 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchUtil { + @Test + public void testCoordinatorReservoirSize() { + // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); + // adjust to multiplier of 100 + assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); + // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) + .isEqualTo(1_000_000 - (1_000_000 % 10_123)); + } + + @Test + public void testOperatorReservoirSize() { + assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) + .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); + assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) + .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); + assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) + .isEqualTo( + (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); + assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) + .isEqualTo( + (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); + } + + @Test + public void testRangeBoundsOneChannel() { + assertThat( + SketchUtil.rangeBounds( + 1, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .isEmpty(); + } + + @Test + public void testRangeBoundsDivisible() { + assertThat( + SketchUtil.rangeBounds( + 3, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void testRangeBoundsNonDivisible() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); + } + + @Test + public void testRangeBoundsSkipDuplicates() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + Fixtures.SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + // skipped duplicate c's + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); + } +} 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 291302aef486..54cceae6e55b 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 @@ -18,14 +18,24 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; import org.apache.iceberg.NullOrder; import org.apache.iceberg.Schema; import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.flink.DataGenerator; import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.RowDataWrapper; +import org.junit.jupiter.api.Test; public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { private final DataGenerator generator = new DataGenerators.Primitives(); @@ -54,4 +64,27 @@ protected SortOrder sortOrder() { protected GenericRowData rowData() { return generator.generateFlinkRowData(); } + + @Test + public void testSerializationSize() throws Exception { + RowData rowData = + GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = Fixtures.SORT_KEY.copy(); + sortKey.wrap(struct); + SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataOutputSerializer output = new DataOutputSerializer(1024); + serializer.serialize(sortKey, output); + 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); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = serializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } } From f16370fb63874f184bd1c1ca6a7a72bdea9ad279 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:10:42 +0200 Subject: [PATCH 0521/1019] Build: Bump software.amazon.awssdk:bom from 2.26.21 to 2.26.25 (#10800) Bumps software.amazon.awssdk:bom from 2.26.21 to 2.26.25. --- 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 e2e2bf34a882..5783edd3d737 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.3" awaitility = "4.2.1" -awssdk-bom = "2.26.21" +awssdk-bom = "2.26.25" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From b3fecc79bd355b5aeefc45e55daa531d482c91c9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:11:02 +0200 Subject: [PATCH 0522/1019] Build: Bump nessie from 0.93.1 to 0.94.2 (#10798) Bumps `nessie` from 0.93.1 to 0.94.2. Updates `org.projectnessie.nessie:nessie-client` from 0.93.1 to 0.94.2 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.93.1 to 0.94.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.93.1 to 0.94.2 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.93.1 to 0.94.2 --- 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 5783edd3d737..8136959afa18 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -66,7 +66,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.93.1" +nessie = "0.94.2" netty-buffer = "4.1.112.Final" netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" From 468370b2a92d37a6121f9f024c87f1079fc8fb30 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:30:36 +0200 Subject: [PATCH 0523/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.17.0 to 3.18.0 (#10801) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.17.0 to 3.18.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.17.0...v3.18.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 8136959afa18..c98cc323c9fe 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -77,7 +77,7 @@ roaringbitmap = "1.2.1" s3mock-junit5 = "2.11.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" -snowflake-jdbc = "3.17.0" +snowflake-jdbc = "3.18.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" From 7fb2a72f6042609c34d63003f35d28994ce4cc53 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Jul 2024 09:33:22 +0200 Subject: [PATCH 0524/1019] Build: Bump mkdocs-material from 9.5.29 to 9.5.30 (#10796) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.29 to 9.5.30. - [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.29...9.5.30) --- 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 8383b631b6b5..94dbe8a1e379 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.2 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.29 +mkdocs-material==9.5.30 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 3013c64318a54f273dd684974410a112a04a4819 Mon Sep 17 00:00:00 2001 From: pvary Date: Mon, 29 Jul 2024 10:06:22 +0200 Subject: [PATCH 0525/1019] Flink: Disabling flaky test TestIcebergSourceFailover.testBoundedWithSavepoint (#10802) --- .../apache/iceberg/flink/source/TestIcebergSourceFailover.java | 2 ++ .../apache/iceberg/flink/source/TestIcebergSourceFailover.java | 2 ++ .../apache/iceberg/flink/source/TestIcebergSourceFailover.java | 2 ++ 3 files changed, 6 insertions(+) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 62558fc9c8e5..938ae4d9bb0a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -63,6 +63,7 @@ import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; 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; @@ -137,6 +138,7 @@ protected void assertRecords(Table table, List expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Disabled("Disabled for now as it is flaky on CI") @Test public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) throws Exception { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 62558fc9c8e5..938ae4d9bb0a 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -63,6 +63,7 @@ import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; 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; @@ -137,6 +138,7 @@ protected void assertRecords(Table table, List expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Disabled("Disabled for now as it is flaky on CI") @Test public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) throws Exception { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java index 62558fc9c8e5..938ae4d9bb0a 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -63,6 +63,7 @@ import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; import org.apache.iceberg.relocated.com.google.common.collect.Lists; 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; @@ -137,6 +138,7 @@ protected void assertRecords(Table table, List expectedRecords, Duration SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); } + @Disabled("Disabled for now as it is flaky on CI") @Test public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) throws Exception { From f2a99d0f8141a00c22e60931b83f398019093271 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 29 Jul 2024 11:05:33 +0200 Subject: [PATCH 0526/1019] Build: Bump mkdocs-awesome-pages-plugin from 2.9.2 to 2.9.3 (#10795) Bumps [mkdocs-awesome-pages-plugin](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin) from 2.9.2 to 2.9.3. - [Release notes](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin/releases) - [Commits](https://github.com/lukasgeiter/mkdocs-awesome-pages-plugin/compare/v2.9.2...v2.9.3) --- 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 94dbe8a1e379..3fd9cf45e443 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.2 +mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.0.5 mkdocs-material==9.5.30 mkdocs-material-extensions==1.3.1 From b5b7e471ec3e0baac7b01eda020b7d7c4008411f Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Mon, 29 Jul 2024 07:55:45 -0700 Subject: [PATCH 0527/1019] Kafka Connect: Runtime distribution with integration tests (#10739) --- gradle/libs.versions.toml | 5 +- kafka-connect/build.gradle | 184 +++++++++++++- .../docker/docker-compose.yml | 107 +++++++++ .../connect/IntegrationDynamicTableTest.java | 135 +++++++++++ .../connect/IntegrationMultiTableTest.java | 140 +++++++++++ .../iceberg/connect/IntegrationTest.java | 225 ++++++++++++++++++ .../iceberg/connect/IntegrationTestBase.java | 155 ++++++++++++ .../iceberg/connect/KafkaConnectUtils.java | 117 +++++++++ .../apache/iceberg/connect/TestContext.java | 116 +++++++++ .../org/apache/iceberg/connect/TestEvent.java | 113 +++++++++ .../src/main/resources/iceberg.png | Bin 0 -> 17608 bytes .../src/main/resources/manifest.json | 47 ++++ .../iceberg/connect/IcebergSinkConfig.java | 2 +- .../iceberg/connect/IcebergSinkTask.java | 10 +- settings.gradle | 4 + 15 files changed, 1350 insertions(+), 10 deletions(-) create mode 100644 kafka-connect/kafka-connect-runtime/docker/docker-compose.yml create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.java create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java create mode 100644 kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.java create mode 100644 kafka-connect/kafka-connect-runtime/src/main/resources/iceberg.png create mode 100644 kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index c98cc323c9fe..055475809296 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -45,7 +45,7 @@ flink119 = { strictly = "1.19.0"} google-libraries-bom = "26.43.0" guava = "33.2.1-jre" hadoop2 = "2.7.3" -hadoop3-client = "3.3.6" +hadoop3 = "3.3.6" httpcomponents-httpclient5 = "5.3.1" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" @@ -132,7 +132,8 @@ hadoop2-common = { module = "org.apache.hadoop:hadoop-common", version.ref = "ha hadoop2-hdfs = { module = "org.apache.hadoop:hadoop-hdfs", version.ref = "hadoop2" } hadoop2-mapreduce-client-core = { module = "org.apache.hadoop:hadoop-mapreduce-client-core", version.ref = "hadoop2" } hadoop2-minicluster = { module = "org.apache.hadoop:hadoop-minicluster", version.ref = "hadoop2" } -hadoop3-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop3-client" } +hadoop3-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop3" } +hadoop3-common = { module = "org.apache.hadoop:hadoop-common", version.ref = "hadoop3" } hive2-exec = { module = "org.apache.hive:hive-exec", version.ref = "hive2" } hive2-metastore = { module = "org.apache.hive:hive-metastore", version.ref = "hive2" } hive2-serde = { module = "org.apache.hive:hive-serde", version.ref = "hive2" } diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 1fdd6bc6ea4c..60fa879d291f 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -project(":iceberg-kafka-connect:iceberg-kafka-connect-events") { +project(':iceberg-kafka-connect:iceberg-kafka-connect-events') { dependencies { api project(':iceberg-api') implementation project(':iceberg-core') @@ -28,10 +28,10 @@ project(":iceberg-kafka-connect:iceberg-kafka-connect-events") { test { useJUnitPlatform() - } + } } -project(":iceberg-kafka-connect:iceberg-kafka-connect") { +project(':iceberg-kafka-connect:iceberg-kafka-connect') { dependencies { api project(':iceberg-api') implementation project(':iceberg-core') @@ -57,3 +57,181 @@ project(":iceberg-kafka-connect:iceberg-kafka-connect") { useJUnitPlatform() } } + +project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { + apply plugin: 'distribution' + + configurations { + hive { + extendsFrom runtimeClasspath + } + all { + exclude group: 'javax.activation', module: 'activation' + // force upgrades for dependencies with known vulnerabilities... + resolutionStrategy { + force 'org.codehaus.jettison:jettison:1.5.4' + force 'org.xerial.snappy:snappy-java:1.1.10.5' + force 'org.apache.commons:commons-compress:1.26.0' + force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' + } + } + } + + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + } + } + + configurations { + integrationImplementation.extendsFrom testImplementation + integrationRuntime.extendsFrom testRuntimeOnly + } + + dependencies { + implementation project(':iceberg-kafka-connect:iceberg-kafka-connect') + implementation(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' + } + implementation project(':iceberg-orc') + implementation project(':iceberg-parquet') + + implementation project(':iceberg-aws') + implementation platform(libs.awssdk.bom) + implementation 'software.amazon.awssdk:apache-client' + implementation 'software.amazon.awssdk:auth' + implementation 'software.amazon.awssdk:iam' + implementation 'software.amazon.awssdk:sso' + implementation 'software.amazon.awssdk:s3' + implementation 'software.amazon.awssdk:kms' + implementation 'software.amazon.awssdk:glue' + implementation 'software.amazon.awssdk:sts' + implementation 'software.amazon.awssdk:dynamodb' + implementation 'software.amazon.awssdk:lakeformation' + + implementation project(':iceberg-gcp') + implementation platform(libs.google.libraries.bom) + implementation 'com.google.cloud:google-cloud-storage' + + implementation project(':iceberg-azure') + implementation platform(libs.azuresdk.bom) + implementation 'com.azure:azure-storage-file-datalake' + implementation 'com.azure:azure-identity' + + hive project(':iceberg-hive-metastore') + hive(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-common' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-applicationhistoryservice' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-resourcemanager' + exclude group: 'org.apache.hadoop', module: 'hadoop-yarn-server-web-proxy' + exclude group: 'org.apache.hive', module: 'hive-service-rpc' + exclude group: 'com.github.joshelser', module: 'dropwizard-metrics-hadoop-metrics2-reporter' + } + hive(libs.hadoop3.client) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + } + + integrationImplementation project(':iceberg-api') + integrationImplementation project(':iceberg-common') + integrationImplementation project(':iceberg-core') + integrationImplementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + integrationImplementation platform(libs.jackson.bom) + integrationImplementation libs.jackson.core + integrationImplementation libs.jackson.databind + integrationImplementation libs.jackson.databind + integrationImplementation libs.kafka.clients + integrationImplementation libs.kafka.connect.api + integrationImplementation libs.kafka.connect.json + integrationImplementation libs.testcontainers + integrationImplementation libs.httpcomponents.httpclient5 + integrationImplementation libs.awaitility + } + + task integrationTest(type: Test) { + useJUnitPlatform() + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + jvmArgs += project.property('extraJvmArgs') + } + + processResources { + filter { + it.replace('__VERSION__', project.version.toString()) + } + } + + distributions { + main { + contents { + from "${processResources.destinationDir}/manifest.json" + into('lib/') { + from configurations.runtimeClasspath + } + into('doc/') { + from "$rootDir/LICENSE" + } + into('assets/') { + from "${processResources.destinationDir}/iceberg.png" + } + } + } + hive { + contents { + from "${processResources.destinationDir}/manifest.json" + into('lib/') { + from configurations.hive + } + into('doc/') { + from "$rootDir/LICENSE" + } + into('assets/') { + from "${processResources.destinationDir}/iceberg.png" + } + } + } + } + + tasks.jar.enabled = false + + tasks.distTar.enabled = false + distZip.dependsOn processResources + installDist.dependsOn processResources + + tasks.hiveDistTar.enabled = false + hiveDistZip.dependsOn processResources + installHiveDist.dependsOn processResources + + integrationTest.dependsOn installDist + check.dependsOn integrationTest + + assemble.dependsOn distZip, hiveDistZip +} diff --git a/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml b/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml new file mode 100644 index 000000000000..202180289d96 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/docker/docker-compose.yml @@ -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. + +volumes: + data: {} + +services: + minio: + image: minio/minio + hostname: minio + environment: + - MINIO_ROOT_USER=minioadmin + - MINIO_ROOT_PASSWORD=minioadmin + ports: + - 9000:9000 + - 9001:9001 + volumes: + - data:/data + command: server /data --console-address ":9001" + + create-bucket: + image: minio/mc + depends_on: + - minio + volumes: + - data:/data + entrypoint: mc mb /data/bucket + + iceberg: + image: tabulario/iceberg-rest + depends_on: + - create-bucket + hostname: iceberg + ports: + - 8181:8181 + environment: + - AWS_REGION=us-east-1 + - CATALOG_WAREHOUSE=s3://bucket/warehouse/ + - CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO + - CATALOG_S3_ENDPOINT=http://minio:9000 + - CATALOG_S3_PATH__STYLE__ACCESS=true + - CATALOG_S3_ACCESS__KEY__ID=minioadmin + - CATALOG_S3_SECRET__ACCESS__KEY=minioadmin + + kafka: + image: confluentinc/cp-kafka + hostname: kafka + ports: + - 29092:29092 + environment: + KAFKA_NODE_ID: 1 + KAFKA_LISTENERS: BROKER://0.0.0.0:9092,CONTROLLER://0.0.0.0:9093,EXTERNAL://0.0.0.0:29092 + KAFKA_ADVERTISED_LISTENERS: BROKER://kafka:9092,EXTERNAL://localhost:29092 + KAFKA_LISTENER_SECURITY_PROTOCOL_MAP: BROKER:PLAINTEXT,CONTROLLER:PLAINTEXT,EXTERNAL:PLAINTEXT + KAFKA_INTER_BROKER_LISTENER_NAME: BROKER + KAFKA_CONTROLLER_LISTENER_NAMES: CONTROLLER + KAFKA_CONTROLLER_QUORUM_VOTERS: 1@kafka:9093 + KAFKA_PROCESS_ROLES: broker,controller + KAFKA_JMX_PORT: 9101 + KAFKA_JMX_HOSTNAME: localhost + KAFKA_OFFSETS_TOPIC_REPLICATION_FACTOR: 1 + KAFKA_OFFSETS_TOPIC_NUM_PARTITIONS: 1 + KAFKA_TRANSACTION_STATE_LOG_REPLICATION_FACTOR: 1 + KAFKA_TRANSACTION_STATE_LOG_MIN_ISR: 1 + KAFKA_GROUP_INITIAL_REBALANCE_DELAY_MS: 0 + CLUSTER_ID: E-JXLvCaTiaUYDb1LwZ1JQ + + connect: + image: confluentinc/cp-kafka-connect + depends_on: + - kafka + hostname: connect + ports: + - 8083:8083 + volumes: + - ../build/install:/test/kafka-connect + environment: + CONNECT_REST_ADVERTISED_HOST_NAME: localhost + CONNECT_REST_PORT: 8083 + CONNECT_GROUP_ID: kc + CONNECT_CONFIG_STORAGE_TOPIC: kc-config + CONNECT_CONFIG_STORAGE_REPLICATION_FACTOR: 1 + CONNECT_OFFSET_STORAGE_TOPIC: kc-offsets + CONNECT_OFFSET_STORAGE_REPLICATION_FACTOR: 1 + CONNECT_STATUS_STORAGE_TOPIC: kc-storage + CONNECT_STATUS_STORAGE_REPLICATION_FACTOR: 1 + CONNECT_KEY_CONVERTER: org.apache.kafka.connect.json.JsonConverter + CONNECT_KEY_CONVERTER_SCHEMAS_ENABLE: false + CONNECT_VALUE_CONVERTER: org.apache.kafka.connect.json.JsonConverter + CONNECT_VALUE_CONVERTER_SCHEMAS_ENABLE: false + CONNECT_PLUGIN_PATH: /test/kafka-connect + CONNECT_BOOTSTRAP_SERVERS: kafka:9092 + CONNECT_OFFSET_FLUSH_INTERVAL_MS: 500 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 new file mode 100644 index 000000000000..5c458ad3fa78 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationDynamicTableTest.java @@ -0,0 +1,135 @@ +/* + * 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.connect; + +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.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") + public void testIcebergSink(String branch) { + // partitioned table + catalog().createTable(TABLE_IDENTIFIER1, TestEvent.TEST_SCHEMA, TestEvent.TEST_SPEC); + // unpartitioned table + catalog().createTable(TABLE_IDENTIFIER2, TestEvent.TEST_SCHEMA); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema); + + List files = dataFiles(TABLE_IDENTIFIER1, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER1, branch); + + files = dataFiles(TABLE_IDENTIFIER2, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + 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); + + 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"); + + 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); + } +} 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 new file mode 100644 index 000000000000..7cffbd8838b2 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationMultiTableTest.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.connect; + +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.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") + public void testIcebergSink(String branch) { + // partitioned table + catalog().createTable(TABLE_IDENTIFIER1, TestEvent.TEST_SCHEMA, TestEvent.TEST_SPEC); + // unpartitioned table + catalog().createTable(TABLE_IDENTIFIER2, TestEvent.TEST_SCHEMA); + + boolean useSchema = branch == null; // use a schema for one of the tests + runTest(branch, useSchema); + + List files = dataFiles(TABLE_IDENTIFIER1, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER1, branch); + + files = dataFiles(TABLE_IDENTIFIER2, branch); + assertThat(files).hasSize(1); + assertThat(files.get(0).recordCount()).isEqualTo(1); + 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); + + 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"); + + 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); + } +} 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 new file mode 100644 index 000000000000..80a74539311c --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTest.java @@ -0,0 +1,225 @@ +/* + * 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.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; +import java.util.List; +import java.util.Map; +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; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +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") + 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()); + + List files = dataFiles(TABLE_IDENTIFIER, branch); + // partition may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.get(0).recordCount()).isEqualTo(1); + assertThat(files.get(1).recordCount()).isEqualTo(1); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + 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()); + + List files = dataFiles(TABLE_IDENTIFIER, branch); + // may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.stream().mapToLong(DataFile::recordCount).sum()).isEqualTo(2); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSinkSchemaEvolution(String branch) { + Schema initialSchema = + new Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "type", Types.StringType.get()))); + 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")); + + List files = dataFiles(TABLE_IDENTIFIER, branch); + // may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.stream().mapToLong(DataFile::recordCount).sum()).isEqualTo(2); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + + // when not using a value schema, the ID data type will not be updated + Class expectedIdType = + useSchema ? Types.LongType.class : Types.IntegerType.class; + + assertGeneratedSchema(useSchema, expectedIdType); + } + + @ParameterizedTest + @NullSource + @ValueSource(strings = "test_branch") + public void testIcebergSinkAutoCreate(String branch) { + boolean useSchema = branch == null; // use a schema for one of the tests + + Map extraConfig = Maps.newHashMap(); + extraConfig.put("iceberg.tables.auto-create-enabled", "true"); + if (useSchema) { + // partition the table for one of the tests + extraConfig.put("iceberg.tables.default-partition-by", "hour(ts)"); + } + + runTest(branch, useSchema, extraConfig); + + List files = dataFiles(TABLE_IDENTIFIER, branch); + // may involve 1 or 2 workers + assertThat(files).hasSizeBetween(1, 2); + assertThat(files.stream().mapToLong(DataFile::recordCount).sum()).isEqualTo(2); + assertSnapshotProps(TABLE_IDENTIFIER, branch); + + assertGeneratedSchema(useSchema, LongType.class); + + PartitionSpec spec = catalog().loadTable(TABLE_IDENTIFIER).spec(); + assertThat(spec.isPartitioned()).isEqualTo(useSchema); + } + + private void assertGeneratedSchema(boolean useSchema, Class expectedIdType) { + Schema tableSchema = catalog().loadTable(TABLE_IDENTIFIER).schema(); + assertThat(tableSchema.findField("id").type()).isInstanceOf(expectedIdType); + assertThat(tableSchema.findField("type").type()).isInstanceOf(StringType.class); + assertThat(tableSchema.findField("payload").type()).isInstanceOf(StringType.class); + + if (!useSchema) { + // without a schema we can only map the primitive type + assertThat(tableSchema.findField("ts").type()).isInstanceOf(LongType.class); + // null values should be ignored when not using a value schema + assertThat(tableSchema.findField("op")).isNull(); + } else { + assertThat(tableSchema.findField("ts").type()).isInstanceOf(TimestampType.class); + assertThat(tableSchema.findField("op").type()).isInstanceOf(StringType.class); + } + } + + 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); + + TestEvent event1 = new TestEvent(1, "type1", Instant.now(), "hello world!"); + + Instant threeDaysAgo = Instant.now().minus(Duration.ofDays(3)); + TestEvent event2 = new TestEvent(2, "type2", threeDaysAgo, "having fun?"); + + 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"); + } + } +} 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 new file mode 100644 index 000000000000..f90d4da0379e --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/IntegrationTestBase.java @@ -0,0 +1,155 @@ +/* + * 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.connect; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +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; +import org.apache.kafka.clients.admin.NewTopic; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.assertj.core.api.Condition; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; + +public class IntegrationTestBase { + + private final TestContext context = TestContext.INSTANCE; + private Catalog catalog; + private Admin admin; + private String connectorName; + private String testTopic; + + private KafkaProducer producer; + + protected static final int TEST_TOPIC_PARTITIONS = 2; + + protected TestContext context() { + return context; + } + + protected Catalog catalog() { + return catalog; + } + + protected String connectorName() { + return connectorName; + } + + protected String testTopic() { + return testTopic; + } + + @BeforeEach + public void baseBefore() { + catalog = context.initLocalCatalog(); + producer = context.initLocalProducer(); + admin = context.initLocalAdmin(); + + this.connectorName = "test_connector-" + UUID.randomUUID(); + this.testTopic = "test-topic-" + UUID.randomUUID(); + } + + @AfterEach + public void baseAfter() { + try { + if (catalog instanceof AutoCloseable) { + ((AutoCloseable) catalog).close(); + } + } catch (Exception e) { + throw new RuntimeException(e); + } + producer.close(); + admin.close(); + } + + protected void assertSnapshotProps(TableIdentifier tableIdentifier, String branch) { + Table table = catalog.loadTable(tableIdentifier); + Map props = latestSnapshot(table, branch).summary(); + assertThat(props) + .hasKeySatisfying( + new Condition() { + @Override + public boolean matches(String str) { + return str.startsWith("kafka.connect.offsets."); + } + }); + assertThat(props).containsKey("kafka.connect.commit-id"); + } + + protected List dataFiles(TableIdentifier tableIdentifier, String branch) { + Table table = catalog.loadTable(tableIdentifier); + return Lists.newArrayList(latestSnapshot(table, branch).addedDataFiles(table.io())); + } + + protected List deleteFiles(TableIdentifier tableIdentifier, String branch) { + Table table = catalog.loadTable(tableIdentifier); + return Lists.newArrayList(latestSnapshot(table, branch).addedDeleteFiles(table.io())); + } + + private Snapshot latestSnapshot(Table table, String branch) { + return branch == null ? table.currentSnapshot() : table.snapshot(branch); + } + + protected void createTopic(String topicName, int partitions) { + try { + admin + .createTopics(ImmutableList.of(new NewTopic(topicName, partitions, (short) 1))) + .all() + .get(10, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + protected void deleteTopic(String topicName) { + try { + admin.deleteTopics(ImmutableList.of(topicName)).all().get(10, TimeUnit.SECONDS); + } catch (InterruptedException | ExecutionException | TimeoutException e) { + throw new RuntimeException(e); + } + } + + protected void send(String topicName, TestEvent event, boolean useSchema) { + String eventStr = event.serialize(useSchema); + try { + producer.send(new ProducerRecord<>(topicName, Long.toString(event.id()), eventStr)).get(); + } catch (InterruptedException | ExecutionException e) { + throw new RuntimeException(e); + } + } + + protected void flush() { + producer.flush(); + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java new file mode 100644 index 000000000000..098ab2395b34 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java @@ -0,0 +1,117 @@ +/* + * 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.connect; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ArrayNode; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.hc.client5.http.classic.HttpClient; +import org.apache.hc.client5.http.classic.methods.HttpDelete; +import org.apache.hc.client5.http.classic.methods.HttpGet; +import org.apache.hc.client5.http.classic.methods.HttpPost; +import org.apache.hc.client5.http.impl.classic.HttpClients; +import org.apache.hc.core5.http.HttpStatus; +import org.apache.hc.core5.http.io.entity.StringEntity; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.awaitility.Awaitility; + +public class KafkaConnectUtils { + + private static final HttpClient HTTP = HttpClients.createDefault(); + + // JavaBean-style for serialization + public static class Config { + + private final String name; + private final Map config = Maps.newHashMap(); + + public Config(String name) { + this.name = name; + } + + public String getName() { + return name; + } + + public Map getConfig() { + return config; + } + + public Config config(String key, Object value) { + config.put(key, value); + return this; + } + } + + public static void startConnector(Config config) { + try { + HttpPost request = + new HttpPost(String.format("http://localhost:%d/connectors", TestContext.CONNECT_PORT)); + String body = TestContext.MAPPER.writeValueAsString(config); + request.setHeader("Content-Type", "application/json"); + request.setEntity(new StringEntity(body)); + HTTP.execute(request, response -> null); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + public static void ensureConnectorRunning(String name) { + HttpGet request = + new HttpGet( + String.format( + "http://localhost:%d/connectors/%s/status", TestContext.CONNECT_PORT, name)); + Awaitility.await() + .atMost(60, TimeUnit.SECONDS) + .until( + () -> + HTTP.execute( + request, + response -> { + if (response.getCode() == HttpStatus.SC_OK) { + JsonNode root = + TestContext.MAPPER.readTree(response.getEntity().getContent()); + String connectorState = root.get("connector").get("state").asText(); + ArrayNode taskNodes = (ArrayNode) root.get("tasks"); + List taskStates = Lists.newArrayList(); + taskNodes.forEach(node -> taskStates.add(node.get("state").asText())); + return "RUNNING".equals(connectorState) + && taskStates.stream().allMatch("RUNNING"::equals); + } + return false; + })); + } + + public static void stopConnector(String name) { + try { + HttpDelete request = + new HttpDelete( + String.format("http://localhost:%d/connectors/%s", TestContext.CONNECT_PORT, name)); + HTTP.execute(request, response -> null); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private KafkaConnectUtils() {} +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java new file mode 100644 index 000000000000..729d4bb264e5 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.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.connect; + +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.File; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.RESTCatalog; +import org.apache.kafka.clients.admin.Admin; +import org.apache.kafka.clients.admin.AdminClientConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.StringSerializer; +import org.testcontainers.containers.ComposeContainer; +import org.testcontainers.containers.wait.strategy.Wait; + +public class TestContext { + + public static final TestContext INSTANCE = new TestContext(); + public static final ObjectMapper MAPPER = new ObjectMapper(); + public static final int CONNECT_PORT = 8083; + + private static final int MINIO_PORT = 9000; + private static final int CATALOG_PORT = 8181; + private static final String BOOTSTRAP_SERVERS = "localhost:29092"; + private static final String AWS_ACCESS_KEY = "minioadmin"; + private static final String AWS_SECRET_KEY = "minioadmin"; + private static final String AWS_REGION = "us-east-1"; + + private TestContext() { + ComposeContainer container = + new ComposeContainer(new File("./docker/docker-compose.yml")) + .waitingFor("connect", Wait.forHttp("/connectors")); + container.start(); + } + + public void startConnector(KafkaConnectUtils.Config config) { + KafkaConnectUtils.startConnector(config); + KafkaConnectUtils.ensureConnectorRunning(config.getName()); + } + + public void stopConnector(String name) { + KafkaConnectUtils.stopConnector(name); + } + + public Catalog initLocalCatalog() { + String localCatalogUri = "http://localhost:" + CATALOG_PORT; + RESTCatalog result = new RESTCatalog(); + result.initialize( + "local", + ImmutableMap.builder() + .put(CatalogProperties.URI, localCatalogUri) + .put(CatalogProperties.FILE_IO_IMPL, "org.apache.iceberg.aws.s3.S3FileIO") + .put("s3.endpoint", "http://localhost:" + MINIO_PORT) + .put("s3.access-key-id", AWS_ACCESS_KEY) + .put("s3.secret-access-key", AWS_SECRET_KEY) + .put("s3.path-style-access", "true") + .put("client.region", AWS_REGION) + .build()); + return result; + } + + public Map connectorCatalogProperties() { + return ImmutableMap.builder() + .put( + "iceberg.catalog." + CatalogUtil.ICEBERG_CATALOG_TYPE, + CatalogUtil.ICEBERG_CATALOG_TYPE_REST) + .put("iceberg.catalog." + CatalogProperties.URI, "http://iceberg:" + CATALOG_PORT) + .put( + "iceberg.catalog." + CatalogProperties.FILE_IO_IMPL, + "org.apache.iceberg.aws.s3.S3FileIO") + .put("iceberg.catalog.s3.endpoint", "http://minio:" + MINIO_PORT) + .put("iceberg.catalog.s3.access-key-id", AWS_ACCESS_KEY) + .put("iceberg.catalog.s3.secret-access-key", AWS_SECRET_KEY) + .put("iceberg.catalog.s3.path-style-access", true) + .put("iceberg.catalog.client.region", AWS_REGION) + .build(); + } + + public KafkaProducer initLocalProducer() { + return new KafkaProducer<>( + ImmutableMap.of( + ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, + BOOTSTRAP_SERVERS, + ProducerConfig.CLIENT_ID_CONFIG, + UUID.randomUUID().toString()), + new StringSerializer(), + new StringSerializer()); + } + + public Admin initLocalAdmin() { + return Admin.create( + ImmutableMap.of(AdminClientConfig.BOOTSTRAP_SERVERS_CONFIG, BOOTSTRAP_SERVERS)); + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.java new file mode 100644 index 000000000000..27de3885a4f9 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestEvent.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.connect; + +import com.fasterxml.jackson.core.JsonProcessingException; +import com.fasterxml.jackson.databind.JsonNode; +import java.time.Instant; +import java.util.Date; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.common.DynMethods; +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.types.Types; +import org.apache.kafka.connect.data.SchemaBuilder; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.data.Timestamp; +import org.apache.kafka.connect.json.JsonConverter; +import org.apache.kafka.connect.storage.ConverterConfig; +import org.apache.kafka.connect.storage.ConverterType; + +public class TestEvent { + + public static final Schema TEST_SCHEMA = + new Schema( + ImmutableList.of( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "type", Types.StringType.get()), + Types.NestedField.required(3, "ts", Types.TimestampType.withZone()), + Types.NestedField.required(4, "payload", Types.StringType.get())), + ImmutableSet.of(1)); + + public static final org.apache.kafka.connect.data.Schema TEST_CONNECT_SCHEMA = + SchemaBuilder.struct() + .field("id", org.apache.kafka.connect.data.Schema.INT64_SCHEMA) + .field("type", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) + .field("ts", Timestamp.SCHEMA) + .field("payload", org.apache.kafka.connect.data.Schema.STRING_SCHEMA) + .field("op", org.apache.kafka.connect.data.Schema.OPTIONAL_STRING_SCHEMA); + + public static final PartitionSpec TEST_SPEC = + PartitionSpec.builderFor(TEST_SCHEMA).day("ts").build(); + + private static final JsonConverter JSON_CONVERTER = new JsonConverter(); + + static { + JSON_CONVERTER.configure( + ImmutableMap.of(ConverterConfig.TYPE_CONFIG, ConverterType.VALUE.getName())); + } + + private final long id; + private final String type; + private final Instant ts; + private final String payload; + private final String op; + + public TestEvent(long id, String type, Instant ts, String payload) { + this(id, type, ts, payload, null); + } + + public TestEvent(long id, String type, Instant ts, String payload, String op) { + this.id = id; + this.type = type; + this.ts = ts; + this.payload = payload; + this.op = op; + } + + public long id() { + return id; + } + + protected String serialize(boolean useSchema) { + try { + Struct value = + new Struct(TEST_CONNECT_SCHEMA) + .put("id", id) + .put("type", type) + .put("ts", Date.from(ts)) + .put("payload", payload) + .put("op", op); + + String convertMethod = + useSchema ? "convertToJsonWithEnvelope" : "convertToJsonWithoutEnvelope"; + JsonNode json = + DynMethods.builder(convertMethod) + .hiddenImpl( + JsonConverter.class, org.apache.kafka.connect.data.Schema.class, Object.class) + .build(JSON_CONVERTER) + .invoke(TestEvent.TEST_CONNECT_SCHEMA, value); + return TestContext.MAPPER.writeValueAsString(json); + } catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } +} diff --git a/kafka-connect/kafka-connect-runtime/src/main/resources/iceberg.png b/kafka-connect/kafka-connect-runtime/src/main/resources/iceberg.png new file mode 100644 index 0000000000000000000000000000000000000000..e4a99c3951e6aa99d6e44575f236f32aeb5710eb GIT binary patch literal 17608 zcmbXKcR1Dm`v8ugWAA+sSqbGFl*q`IGD68X_90}?j0h*QC`5?t%yW+8WN!!ADI+Un zWJN-e6~2%2dVj9#`}^nj&)0Qzb)Cn!_kBOseLtVyHMq@4$3q8!Kp1tjZyG@$WJuCK z8Yhxv?{CDn=7V^=3FQ-R=cD{}fbq6ndM?oDAyN8ZOj&=@@efk`gAds7`Iycpg zgTDTpolbl`k+HR7vRa(aTpw}_`CIQT$}8D@{hotMK}N4*wUiS+;tc)jZJXdq8t+?P zPY<^>&4N8@SJ$EN%J&n2$Y^i?q-omv5ZU$ex_V$syJ2Y zqPw+$iqQ!fz@S8bpK*`C=8yE4C8zw8sdI27O>47@hEgCP2~n3HH&3z_qv}lAY85rj z;vJzRl4$S8rd-(*x9i^+8hb%zmLfuG-Hk2Vxp+tCS+KRTQcJA2_PlFA6rNL$jD-N% z<($`*TIz0Z#N?N|suHb3UjPBlc7`0yZ5^bJ<-V9!zS?yK4M0 zFbYEjk090*^iao*J_Fd+t|4W=bV{h*5{DSNVggLuGK+vphynQ+{?IqN5hmF%Xk+qU zw@@PI1vMg>Vx&nUWzg#}2-O8Bd{!gtfdXjkpf{=9+^;RLpnRidLQjDZB%sE$HO4rw zP=|uDwJ|a%{q}XFPTo<2eKT27dKyLf+K$R}Ea1vBR=gGpy`g_QG37-tiE}to=S}eK ztM*WW^)*@x{WLNP1o0bc?C!1^6i+2_R1J_J7)*`cyApWNqZSzJ;S0`B(UdGW2un5K zkH!=-BUr-d=zUp?S)E7*e$rR~EEI5vr}O1nrvIO&S_*hZ?J0jdof?<=Mi57}K@|rW zXi1H2bTz{jBN-ve&f-*Q5ok&!R@?~Ok?hHLNrM4D@f8WN_SpB z&chB9O&>)T^bL6rfD^}LYE+Y~g1u}?j*k3<`CGeSo<;fiHZ10`Gu9V*B_? zZEtQp+^Xc@=71p>cyNWTBGXTAn=?ZAT`p(Ns}nU07UJNZkb?wxpc+xbsXYz@Lnl>p zj3IWP@gsJ5@FO+=4*KqwcRgaePWfEMgdqONIhU&(mole6+2nRRMz(CR~#P z>@l$69q?n*I0}eTe%aSC9H$fTj-i1>IV*qE{`STp?<6g%Ki*+oK3dD18)sitR~aDg@$DoRZg- zCIqF2Qo(B}pv3dF475m^D^@&DD-?KkkFQQn2n}9 z1{xmz$OOS3Uuh8FOn4AKl;|y@y7Q!07lM{6KHKbi^#DN*r9X_u2ExQBp~Tx+S1zK2 zO1wo(fEkcMnnm=tyjUt}kiDYFt1(bW2c3D3yqz{N;?TQ`n+ifh|KL0gQqT$tp)t36 za3}J{0>+@844v1d>Tr9GIiZr{@4JZQTlI=rQCJ^9eBM`O^HCByAP9(1Jem?%6gr8L zjB|kwqSg!i`u|URyth_88H7Z}|4<2{Sn&`@6oQ37148-#+fMSO|Fv5&ntj))SN!M- zfk^M(;;767SmV7ZvC7x1OpgQ2zU1h>>VnABHxddYKUb8F?J;t8z989=Bamwp9O+nU zeD`?Y_{+7XevH-90QHc5$5?cOi2rdrPrQ2X1ynv3y*IGddC#;%pIuXA@29uODGjl? zX~1p0<7aX(;g;;c-CjkOXq*D@`Q_mHqbo;Ox^(Nc#Ep&0lH*d1@el_+jJX+?>Dcn2 zBd4Zn8LEW@wc$MeTf9p1pl-cpFs86h7h~d1PIlN-SBXDWtm6o$AW4kY?EY{pvpKDt z_UKX7@B>``bwQMH=KRYy31s6})1IQ2jp~!bu(p+K0V(Be4`bl2XP%kkP%dr5@vbJB zc6K1b{1`aZa|_RA)1xq67Ier(l+I8Atqm{LD>ed-*^CX-dckTcw%DI2RcUg`GN#g> zM&jEGjd2$M+tIDwemvFD*@F7tE%xT?9|6hcyo5LYj;xvEPYFX566<;(Su4c;P^{rUwY8sGf-;V`oa8_h##(4Rw zZvcKM!-aCEVTw6+15P%AG_#r2eAd%y7wZn#G#WxBqHHcbV(0l>^nib8*MQiMu+kwE zl+6mdSfuCYvEd+UaxM{VER+%$bL$&?lsGwaDzdabZ>{@HLNev#y=?m1%I|(;qa6`e zFU1xxpIp~8M-|aZBxFyz=@rq2MJLt3rutsFEqp^=uDwl(ZSodrUuQS9 z_a3`sd@}ms3OzLF*T~Q0s_^~gOI8wipMGy@F_PN`n#YvG%p>?ZO)|&x+x?ob2r>xP zB#sipd?Q5cvI-hKCP!;_$8|ytW5%xeQ{m`n51-kn;>knWlYo7=mp4Zvj{DFv&wSa=C7i(A+QoP}e6p z_O2%cxS|`@TSSUQ(r4VcP(1SZ%@-!3{utSuXvmdR4iLzLBckBCSv<;hO&u2?-urJI z@|C&hXTt>?u}SE?XKR!@we}YyO^quc2WW9qL-~uhh`|@Ft|qrb00~b7VYmyzyuriT zahwzgf@~=-U`D0vz5XONFuB!<%(OHa>hf`Gh{Q?m6?W`HqiUL?A>>d6cP-UmUN9w5 zZpLsjJ+F>VGrZ$0vi)*KU%M*g zbHuDT!|xk*$;DGVGUyxB&;$2p9vxx5rVAWUI71eX3{b)to9Zld+bsH+_IS_6nOr-P z9;p-Oxot6#2q>9@EhcHfu*~fqFseOqosnzl_juIy=+rK~c|KP6Zg_uz3OZo8b-9-6 zB_L6dqew$2EKjO^#stZxN@y_?~vx&pYHq%EZ+X8U>9M{t=z(i^=LJ;vuS_J^j>{3 zYR}JRV1j}cSiu1vlTD_Yu zt;-(9er?x1!pX}rEk%lttE~@rP7maTR-mr_>1X-Tq`0jV-Hsar^cZpAy89{Mf?grw zMfJ(itt~0lzgY-Ytjcy)EAB&X6+a)aN?#Nsvz;_~@peK42_^I(Uy6^Tn-MWV=LpHl z){=hLZ(_`3>K^WPXUpJ^gD$XacU>x>rS_L``5}Z-Q0sslo5h?zYQGqd`W;`H_vKpx z`%@V_c|eMU#+3L6wPvjSV=h8+%S_$l^2~eKv@dXK7f(4DZiM}*Av+FIGW1i0&^f%n zb(xIwvOE&7C(2|#bU5nO<#t~Wf)Tg}Q2Z@sJab8U(YFmK z$h>(?a8GsWe>+k4*U@pl+>GgjO)N5a8a9MpxXUv^Yl>E_SG*mcdu-J8TNEWCywBTw zeb7#(6YbTi^Zn>l2tZH~=cJ3mwF4LhV?8BwoGoTleKqpN6-SRdJ>>9y7g%V>r=u+H z9A0_yB_5rcrBwPSQ)aa$J@)}?u;9y$AJ5fXti+H|yVBIS9ddHN*5#63_NUq2Wqw}m z;vQD3Hw9TlF-Qw-*ualzWEL~BFIC>(CZ1~IO-kgAWcH`W3aoDiEvi}xGulNQ`w#3; z26dOeAiT(48BB+Y=8S@E}F75pz4?=W!B%h{z-Jg1`JVV<0F!wo^^gk z5n1$dBa4P&yZ*WLdAgAgQy1c@pzt!AldOpx1?l(M>n5v)MVsAw0=^emh)fQcraOT> z#H@1r2CMdhV8=IB@sR=F8)p~~$ayB9yQu2!I|%(OoV~(1La9wgPtm8u9GV$RAtZF| zB?Dj4lr_B=7Kt|{^)bDd(x(V1tIDEz-R%z~ImB9-7wu<)SS8~e!@4)~%>T`k!B50a zi;=llQRk6gdtdE?k6X5<)cbY5Fxc^~cDmv~t?lY_A`kf+sPo0aUm@I6dGYs6a2M!b z^tAZ)o_)ja-UJYU3eY+aWHHE08LP&3LuKf$?8p_MqLh$W1MTsNiD6^-sRe|E^HtJ4U;I{wHbTtks+K4pOf@AFr*j< zMkaVWB;$Ee3VRo4x;QfHyv)avZ`f*|Z1$%R+<}wjr7M-{x$pXJLdnMSlHX-8+T33` z@Gh%zKi=GwXVw0$2zh&D%9@wf%!{3{8x%j0VlHXBKJ)m(wo3#IqSrSYCDY{g-~^wx z2Zg&hyKdRU67$X^ng8QF6h1#=A zy*2VNe7^HWtGhSn|1)!M0bs@EmR*&~sH?~?S1jL7kN&5$ZT)iN7Xm|zrR9;g%R^&Q zDs8hT5)`i6tRwzkn;y8;UGO`UEHJ@ALB9jYn&42{M5mX@X4(!3unkwEJhaI3v?YsvCr=6Xd;#N z5{AH-Q8_*?tCGPF%%WBEeE1;OMM9A?hNXiXP&ay+y0?LA)F~Mhz>wy@(Y4b-Y8lOC zgWt=%$4t%E(s3^I3ua7u{6>|1%q^hZogVqIRl4^}(WjAm8v)N; znnD0OSlBWuq(1j9ms{p+IW2NH;=HL$9dA+9zHaS_BCw?TJt*YpOg+^@BO6F75jdB; z)sK2kG3EaQ4^b|R(5h;>ZtBkdm4mR>Cha^l%;54}QH<;aQ-RXQ#So7ApN%{BxX*sFxy||Rd!O_>^DeHGLfnDxpsFN2H5egOj>nBU7;0pwj zC-koH`sSu=Z)bnl4V#Ya-VHAE`}&`&^zx_7)|@l+jG>hS#|{SSNl62y z{@S_!i`!}{OP(DT&mT->E;ZgTWe9~nsU5A62Zop{iiDzJ&H)on;!`5@QYatU1ar&m z^LGp>`j-bl^nrxn$}6x%sW48B+!^Ux3SSIo0;S)d?5wBC;1h4Da)a$IB*5I<+C;AH ztx4e4oLQH(;pHxQHfx7;cg4gr?{Ah-UOO{yg(emY|Gow>qsWJJ@C%^Cx&?Y-+QN@z z7y4hkd+fo!V%n}-OU?XeW+NRKsTi3febP5^2w`AxgxvqnFZbP@uw%J`yC&y(gssle znTyAYo90+XG3NMP!=xtj@QIfxi;&J<@gL`G?*^{4g>d6WyhT#67le5j`W)X9vi^uM zu-g*p2iE^h`0Y^a>-q@e?O(ojcA`c%&se`n48b|*ZTm!jrCI35Ssk_b{b;wZFMYiF z>GKBxv+>l%QJ4{OVQm zTY_cKn#EYRToo{@9JxU=@ebt=r$hcu`i3}%uy~Nh2jQkQt=j+9y`US`lB(gzwIF_@ z(k(!7;0$%O$B-i?H0;;c; zfWb<1b~Bl2qU9Qiv;D3bM*qF`m`0B1@}_%37GiEtwYhe2{u;uSHV=Yp z_^Jd}!#^r|AZfq+QV7oXI zKvAjV4pMY+RAa!CDo+{iup8;mrYm#waD^edX{`%4}DrL}Xj zWaQ`jVkEt?VC2ik3`n*T5+l3u5c%WyS3eFO?^yZP^zSBn$9Q3!NC};r$+eK+krENJ zof-3~yTZ#FPx>B3kM=?k;Qq*?Wpxy79Rqi=HZOnG6<)YDEf!U-Q{On$(;H)>L%HE8 zZKx;Ob#9K4O=w#h6zYSu<3udLy7f~X)y>>aH|3ara=kM{Yjgq1C z2)ef=l#u4v{|MNPr|P|^lW;U?cip!7s?Ia}`QhEJV5_esdv%n+6dZ*4YPKh^)$2pmdt8{X_0+T5O4_;O%C*d|O7DJUxQ+xA`;rXwcl7XMB)CC4 zsz8V==oU9MsR5G~o!HUN<$S>3aou#8*YXlD2^WI2vTHt;1U^?Livf90+I@RwYVgZR zRB>Ub1y@Q4vl)#AdZK;Zo-S$rRrx&kC|*o|cE-Ni8b(P*amNd2(nOpw@y(j)T`{Yx0u8$Er;JYqRgD10+%6HQrwhlVR zm>4bqr~0NCwD{tcM@K<>o44}Ol~4Cqsmkmh|4G?yG`gm@sUmR*7)IqD9@F~}8kPJc z=GdR!2upVuH-XI1A>&3P4PIp+YZKkqQbC+Z*$Bc*`S3BLA|HL}kgM~YWF0&U97DNxGmSJ#$nw`^Q*^s$ld)$@jIN6qlz#)9@)9qFLmAwBl?C#@qTsh$GA zF4wKPFs29bK~AW!df^R)_!_*zvBH%NaiqT(Sz1BxG6INlv%C50{+*x_rzO1R<6h!O z%NSXxg<86k*C$NH+25MR$D75Yxzttoh~iLBF(|#^!S8=}lA!RK*H%5kp%xExOyX4} zP=8P&F!i)Yr+l@W==J~;VX|80`_}mVlCfR6ZckuI&__*t$ff3p!Hk|2k=q zlgToT1M|a4EB1^9e-!zFaz`WdmM-S)A`kg-izIJyNV2^{fx!3f^iq%Rq7pQBPr^1jPod-U15(75UmecT7P*Mw0$W!{jW=(zfxU#LjrZcE7viBus_s$ zJ%(r8rr@R!R-6ClV*YbFPGKG;;pr$LW)Qf)hgyuV*uIOUQJ{iXI3fM54N&8-#tTra zqct%uSzw`m6utKqejFj7gwcGMJN_*$ZcCNoj^1qv?Rq*%qzK7>){>Xr6#GN<-Yc(_ zOMRZZdv^4l3rVBDs8TMgz*wiA%lU#D?ggT1q`6t|<(@SrBx4S*s!r9ho7C2W?NX=g zKh=vN%;)|<7&Xq-44IJV41Twn$6ulNNQ)}5|52a;N2UE4cqF1SsZW*o4Xn!GU}yf~#c&Dix&q~;7L$<& zT7<7;RrtJEf6{*cX_EN(tK2W<&(4!fmtZVzvFmckhl>!X8FC~sFSC*{dYuwMp+S{+ zI*@3Ka)_i%OTNln7vJ~SF8Z-WtM+Y)k5<`2$jvBovmv)TJpu*SP_jV=9X`K0UIcG0 zuOeymNE-s@S<(u7ZzyxS*%144N6yFzxyMR!ik9l1y*qI?!=W4h7|GJ23k~73Dl;j@ z)lYto@?ej+2;4P84_g5p?kDPZzvu3W51eBH5!y2PV^`%w%h<~U_1v^sPn)K5`(~`T zBN||a5$DED2=ZFo+*FB*Y%xDZE7{(0QZbM^6fAz<8Ln+(nGuS-T`~LSq0sqPe_c*I zz)s5c>s>g5(&=0DjsYY~4N&y>^@8yD5H?z_BvJ*jdBM36G% z;AfxAaU_Yn-#Y~o(0GXsqRzV;0N&rA+;Qiwt&N}nI@6DYenAqp!?I~M0>u9*8Wwf; zxSG2rNs~13{4HX{bu+JzJ(0IF;$aCaN7YL=^RlQeA|1hcB8Upz(n@2~sHLSvF8ug8 znjDw*an!fPF#aBRU?Pc{jJ!_e2OZKshrSTHQDgS^YLLv`krLUe*X{!SEwLv102(PI zvEqD*IQV2I2J4HSOkedX4L{kTN8BV98oC8I3WtI1gY(qsMi8JWQMUmnUZnePgSyDB z`zn#QIXORDrm5}kZxyP<3**2wc|rxoeE2>Sx@{_|Ewl(D4@!+py6lhL!b>eoitLXm zZkAtYll1?0)R)4_<-~@f5^UTY1&goKjO~~n;VUM*tZL!y zsR)DuG09hH6P6H7R0A^8z1Ibk7*=pqkGN~0v;Na|-Tk0AoEALEe>?a$Q&S(|4{nG# z$;neNtd>#x)&jY;?8(abrlXFo<){2SBE==ba>5e$-)0P(Ky@Pw=F$SZSBGfY^#`k|%1aczZ#N6}qz z%m@wp7m&Y!zO8LLE)}-Nl{rU#F=Z;Z4EALRy>)A3TT-1K@`M)YhY)`B#ul0IcCPIx zCf%5(&`2cIbSL^#=IO`ZHnnunAU1+zmZEQ(W0Yj2{=tT!)X2YI4WFLoE5b0#^*{I^ z;5rpeE7NwYLlvV-SU=B=v2<~NHid*da%LmUi)7B7Br`FTJNK%<-8L*wy*6V4MVrZc zxuEo>?+#}}Z!j`!_WbNvuRUwB$`>~K0Rg(4BpcjHXl;0;aOKPE-`o#)NdoXF8r;e~ z`(er12|ZzA7}MeT*1*6+N(<>&UB!0|ZtXgnAH_(|6YtK}|F1n+rfnciNN7N^kt$~0 z;K}kmz)3wX!8t*_xbVk`s_H5G;1=hVt5=LS-p+WZ4&*`Qc?sOt<38$8f}N_-u!hPV zMustdGJr)b5wJf}ErfzLL?eFm6&xyCnlmtrJqL3fUZN?bhW_R7Ws_?j%l?@)`h1o< z@=H8jogAcTv=ki9BG}0_t3vDG-sfZ>t7N9}MNy2-VRFfOzI@ymb}(#hgc!U3h!QuT z#6_SS+&Y@Pa>ZCqGLmX2`YH3)do3C-&9q$ylJ1c&!oG1Mc>g!6cPaF>2>LTPL_DvU z%$LH3NC&q!KF!bc0a(n)IQ-{2!ve-)L=hoq5$9FtklZc8(pczH4f;8G;gaiWsl*>e&!(gp{&@XQP~Fd#+sCO1o>{T=dV2e&^r$TbIn z=me1gCWJ@lT`1fg?~~f9Smm0T&CF>8mBzC#d`UZk@J`*4kL$#Kik{T!YHNsq8}`gc`C;!xH#PS7VbT+7~Ld3~s4?sO+^ zcqNc3#?#a8^07)in+ZaY<7>d;qc$(O$DS-~K$0O0{&H;tYtawMxw+SMiOH`RNsOpZ z8=O~0648ZH8z5N3r>*P*2PV*ym?2a}_%D26HDJA<{fBBqqU6->Cq}OHvNkOG$f0(ba5-gN%6ITYeM5tnd z^JO|5htRRrD_mrpsJjpUG%+xQWI#Fq!~ zZZn)|eTNcpOBTYH!}GZEvPDN~6rv-o6NHfMm{06cqU7W0N+v{2c45n0ML0|C%qJwE z{2;IFq@{hx_B9>n(1JiN?~nDl5GD7%fD1S7{sopzeEijq9!lR!yf54$r>TcejybUF zNZSK2y}vomhUdLPr~Z5(JCYj)<3fv=W>AK;z+l5ZRzT*VwHeKC}(ND?~bwYI7h)@w#QBX9w3?ggcN!vp+U@h z&RUn5Ek|!D^1B_fMTo8!phwrdSmt>5U-a;cLPQ}F*`pde$0Y}fdp=9xY1ZW{ctx**e}2j&)QKaUwf*;c zlN(bRylt&*_t*Vi59Fl_;4am)vM%`{7S=ZosfSD2+Ufbt`wQn&%qihNIxjc~(Ezdq zY72$J>>P#p{pa=1jEL##?D(YgONkD?+maiBWr>enS;-7oGM`L{V>HSa^@r$oq@6op zzF$86EbpH%WayOx=Jv*M$Tt*kto`5Is*iNa8B{nB789=Y8ENWM$|Ls@ z8}U{h4k!~bwr;syuYs3-JCJcZw59J4eeS)M{>z20^)U4Vvw6$M?g&~FqXr9eF)Jat zazyvFrE`R}&WH(lb?u?`t=Y-BfxJ;~k?Baa1nCwN(Ld-^lYNyWE(4aY+|}p%ko-1O z`t=c7(%C&3X3CjKlwLj&Lmhdt#`mM+c-03=2J&dl2)$P0lsV00Af#=}0}h;R`(5Rd zn^>x!G%_5mw}^DW++RJmi>2etV-=&JwNN}5+LjEYm&D_DG9x9V+wfB974r_5WAhxZ zgP8TyNOi1L^9ET)jo7wTGGOMFjKYC;zL8_K^0cOC$R=wM*+2nU_ z_v51U0$pu?QC> z=#j@%5^tp<)4WA2Bhf{^y!5GVMEkcaD*3_7@9nQnF#Im-wQwzw7y`-qVvslk9E&jp=K6CIupXM07~XfW;n&f=kzIm%1D$87XSju5QZVj-y5De0ASx zM90gkKP$g)8j_IuD-ULvnSTETl}>46aH2Qm>JyMV1eRL)=j>6v!p>a}co_z}lkbA( zs#O0DSk}m$m^bVM{%eD7gE^Mc{Z5M zvRk~@d(FwlKNUYNC&ID>qcQ?lJNCJB17X!v^$cLOvzlj+2N!TJQlarqf%P!LeyV8BW$Bn>)JUaY_2%jw+Dmi9_rZi|JpxP>kn>@Q{$Z9Uwl~`G)r`9!Z>n5 z8z>|my9eYys!Lb@jF1Gw#L~zwp`t< z>OVs6!24|1|_LQ!moA2-hDH+m`!Mt3DzY_lbI(8gc77+M9BhwR%b8 z$3iCN&d+;u<#<;7Jnu$eeOv+pwCQG(_6o)xVHzXuO~QpOQJ+V za1S3Vw~GP+yi5~7lx>uIeaQ_!i*DR4p?3`ka2fSsLVy2OcOtbm!Xi*+AKL3iS36H3 zJNGF1WK^@G>etTJcIB zYxsPc+xTyeOMupFGbL(91|p^4b5Qyw^c+WimF0kX`wq?%l~_mi0t7^ePdN+eyio|UA+xmt+p2(rs}D*R=2F$>ey7R z0?)n`GzRRr?d=`vxdyBR?4OuSFJssFd8QXY295$3p`S@goLsgL z>pAVy^uM>kKIpb;$KB7t8C zH7UAm)fc_T;x@OVzdhO6wEyS*-2DV_<4&1rbUx>qCg;3RnN7>WqV3FHg*FD+YyD8f z?S`dE!&eyo(j%=l{N3f>5$=aYEhdV!XSFAn%?%jY@beWZ_|vZJ({q_Nh{?tApP{mC z1RPWIkmY!KPC50x?Mzz}*Xuu)Ev)XldT3ri3gLqngr-9X4$U1k@n&CA+rnUGcn8a& zQhOa(#w1g?!c<7=?}Z+I43eNxmx}WsPnYSh@G1?#{x&pjuo8rc|G*~4jeT8r1&P}B z9=YSu@odUdV8 z$YruC)|w*bimjQ=b|%UUKlcIxTc4?%U#X$Byz`6i6T)HhL#()28c4%38k}u@@CYc- z>+5o+IK!|nT?OB^5b7|Xc0XwMAZkL?-N!B9UMRTNv0$w%_1#1a?9iPQVKth9DGxRZ znHR)QefKsq6u=`I1&*rX-sFd3-SGhT3f~M2%B6{UXqPv@BWfUPp-N#4Z!>tf-FG0L z{R_E!5I1qz-TwanR2)-~*rXd~(okrpoK47a>kO`2z!-~IwFioY(gON%iJyl6{e@GZ zaTDV1-*v&HAG2{aM#m4*Oz`S(0;AvAB*gXs(Ko?Uq@R4enjs z!aKqXcq(oaQ9kfMv#hXRBps%wV(x!r~WJJv1lTrtd z9^+JOI$fGf*?^AP+iv)75z^DF0urvEYAjQV28W%P#E2wE&09mp%j=>jozi62RCmhY z_W^A9`j6+rZp;t#wHvgCkLmWkde2BmW}c3oP3G$g!Sb%#{4?CV!Rc?>FmenLF;=iH z5B?`Mzs}zQd$Qk}2t;|bcarhIo!p(v2rxNo+@J_Yx^yyH+w=)DdL|hz1q5+$XW4l(%t2;#IL^OIL_;J;dA#-!;`y=n)d0o zr+v2CUTX$%rm!ttwfQXQ&>j%IuC0XR*nT5Zn=67kWUtH|jvZ~rtM~d^malJdy5?KO z3!r-}hQB+|f0 zK3DnV#utT<8*|OsL-mc%vs|l~Z%Ju2a2G@QYa2)rOlUMXe}6WUPR_q;FEn4}R2jnY z`Ys&FR{8CvO4hPJvqqZbUsbt<;Tzr=PYbrK2@jIvo~rJA?2bqg+5j;U5yR|7)Xd_E z+K-s#xd<}I=@vYD75^NY5`^3yeMofL4ss(PS11{CDH~+9iNV~J-%Q+L9qRcpI935N z(Nq7Kuw) z9U#`q)yc*Idh0Gyr-63>r)jfneJmH^3Fp6YL8zObm-sILY2XxX;$NlEe0N##e9%_V znq?#fPT$*%1kZb?V^xE+*d_K^N0JIc+^|U^$K567v@Df*Qr0lz{O*tNpz@+gz>>eZ z%|&+@u>P`PWzUzc&pKgXRr`0m0?xGutvbZ7Hoq@CPDzCb4HyUu|FV+0f5~KuCu+;# zqGpPtAj+S&a;Zy0l{1Uk{J%@d3w4i7?nVna*|fVoOu2smvfw!?m$W+9sN^^Q_)RS4 zEvK9u=lBZS_G!mxJm(U6(&fnvbrXEyiT>k&9jZ%m46RQtZ|?^=`MG=#E|NpBUVZVg zDXfAH=EbHpA(PJe*E`~wR6*-2bW^S9;!+OlGd(ppHq~s5DkMNT`Gf)^#QcJ4%dt4YI2)(H>x`p{~ zvE!SEAAtw>Tg~}vJ!C=ybSdlsp6xKe@7!9`1=Nq8CkU3xpLXA)Kl|JdDeI4?Z&lQp zgFy!zdR=b`Bvi2pIxGwajv9dk{P~j6k6?x)YvodFhl?p$6z<>8aL*3B-00U!6AMJM z-&E}`!DOaR&cW0|^zRSE&(~5dl+??{giz5UGk&b~ENW3#XpE2i&SVdt8)ns$e3G`| zy(ImRXxQ^^E0BYo2AL6yODp)nmlKUk4F6*$rZN6Z1PW&|b7$-@9QmR$D9u9nO=qs( z^-V^}7o<#NB&CMbCvSK=gs^Ng>u*vZ;2LQcpBSmon4f!_#XWcxScIMPYcBdHkJ37z z(1GG45vgubI8?(l;CEIuEYabT_cV92=KUMc0bMU~1I~z~RBIw8?GLS4rR*h5E$CUt z81EfdF_gKO&c7ZGF?#jh{trtIDwdBZ@tsd|<*#ZJD-|93;!|#bp5MmJvPi_A0(9bn zYbLOu2A0Zt1L`P5=dFDQC5#ZrnqBv_Vb~sg2N)0ls~WALWz3ar2802;K&P3jVgLKL?|CKWw@MW5^FmbBrmR!;_v zPoj)EUs=5hKi7Bn(vCxfH>lncqNlm$uo>>D;Z-kgG7SXY0RvT32*IVKB3TsU%il}w z)T4j_PNDdLeD&V->9uFA^tr%lZ-YufZyKbwM%ur^b}kAUb7h`k7~#Kt-3l%F0v!cH zgLB@r_h_9rB(Ni1^MTSmK`T(<^tjia`*JgbdE4o3fnxZN+P|tHzV?LHvihZ1@sO4d)glr07=je0kQ>}GVF-qsW6|D6Cuj7FbcApeL#3vTOIfcYf5~ap?@vZ$` z6-@qrTl^?Es7W*T0mWa+pDK9-9h6-i3hS^Qbjue1Am@cu9nPM4YY`((lSFK-Kv|ty zFAv(CaeR^|DrI5)7Kz=Pi&{&d#%bVA_En46_ptAkq-*gLV{TYb*M5X|1; zLQh02qm2f6{$=Cu)JNzUuz~PH5OeSe7m8pqPhq#rWk4{!e4peU{jokc9k@$7KB)_b zpx~{nZQNE+0iIO#j9{%8W0zS3`aUJ4Ciut?x(Ze&?JeD~dVW9ByM5}Sm8>~k_!Wk3 zO|WfPQ{O#MO;HVXQ#$GichZ`Vx2)P9``1yaxZQCedgi8{eYPQ&xPs01!I%-UqcGu# z9VYQb+w?F*qLwY8=NqjIU|; zM1Pk4qZ?cS3E}%*C{)#+b38D9~y9TAtiAFTotlo~>@SpeT zjr-r}jJ`|tr|+gdx3e$vM-bbOg1|P%Iw5T88NViu#%f~fAneEQ7r})0Z`Wb&7f(#j zBx941aaXn_IZ7kNKAlfc`6nIx#hmc~<-0{N+JLW*&%!v#ufLLL|L#u9Mqn}RO|JUo z8@q7NlpIo>QcisEU#sU(@QtHwemE5J^$s>*IgWh>yzBB;PD)cilrJJ%K7(M2mT`FR zNCP=}pvOXiaObTCedup@Kq)Jz$K?{zt7Rz4^nU{?QYCcqk}+d0X=K^lGI9DMBAyNQ zu_|PcNvr!cC6(;>Z3=PZ4!@-&GXM9>L?}Kb24dAss}3B&f5`QsIu=yydZUdo zDkwaNDc(X3I9YB6U7Wy@L?I)uecUKYaqqoGVA znRaqQXCJ5z*Z&F3FFgt571lD$Imtqn^2)yI|CAurgCB5Lug6W9Ln?)z#%-#B8tXel z$v#Y#9>|BX5N_fRQp=;E#eP0WqdqoZEKUV5!SblLK$RZNq_33=h$6D9V{<6VL2<5H zV*C2VKx$A|?JiKwqca4m$|1GGAO}~5RJ5!%M5aanHLipU&VA22J8@Jj)eR)|kyu!x zx3~|g&oNazYToVZaD7jzQ}>K5FT3b`Rgbilox9FTh#DQe6BnXAlSu}ztb67+!ea==0GMd3SJ8ZAd;xc8w)U$ zyK$~CP-oxBWN9>blu=+tKdF1a=IwZyDKM(M)jW~!d?OMc{`-bN@?F`%XOtvy&{^(X zbnmN=RfI>VY5Tic)EJUgQqlmH-r4D`xD>tIiIOC**wCNbCmlZ}UQRF5aW@ACAbxt!aQ*qp21}x^sl*gQnXh7d(K7itgMI5qE9i`?%d1x?|T!p z)-E;J93>k=4K7$kd(D?l%$Mb*2k{sVAX=7y156-=)Z^yNsEXZOO}X?a(H)l_)z4R9 zd?aLTP+Zl>n6^rS|Gyk;D#7%(c6g}5;oj86oA;tYQU@7IU(ikIuchT4x&^{y+WYg`e4?p^(2HZucrT(7;GAbhHd^;x$ka{|~C-bCdu8 literal 0 HcmV?d00001 diff --git a/kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json b/kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json new file mode 100644 index 000000000000..5b51e5dea875 --- /dev/null +++ b/kafka-connect/kafka-connect-runtime/src/main/resources/manifest.json @@ -0,0 +1,47 @@ +{ + "title": "Apache Iceberg Sink Connector", + "name": "iceberg-kafka-connect", + "logo": "assets/iceberg.png", + "version": "__VERSION__", + "description": "The Apache Iceberg Sink Connector for Kafka Connect is a sink connector for writing data from Kafka into Iceberg tables.", + + "component_types": [ + "sink" + ], + + "requirements": [], + + "features": { + "confluent_control_center_integration": true, + "delivery_guarantee": [ + "exactly_once" + ], + "kafka_connect_api": true, + "single_message_transforms": true, + "supported_encodings": [ + "any" + ] + }, + + "license": [ + { + "name": "Apache License, Version 2.0", + "url": "https://www.apache.org/licenses/LICENSE-2.0" + } + ], + + "owner": { + "name": "Apache Software Foundation", + "logo": "assets/iceberg.png", + "type": "organization", + "url": "https://iceberg.apache.org", + "username": "iceberg" + }, + + "support": { + "provider_name": "Iceberg OSS Community", + "logo": "assets/iceberg.png", + "summary": "Support for this connector is provided by the Iceberg open source community. You can open an issue in the Iceberg GitHub repo or post a question in the Iceberg Slack workspace in the #kafka-connect channel.", + "url": "https://github.com/apache/iceberg/issues" + } +} 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 aed11ab0b169..cf34b0bcd4c8 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 @@ -62,7 +62,7 @@ public class IcebergSinkConfig extends AbstractConfig { private static final String KAFKA_PROP_PREFIX = "iceberg.kafka."; private static final String TABLE_PROP_PREFIX = "iceberg.table."; private static final String AUTO_CREATE_PROP_PREFIX = "iceberg.tables.auto-create-props."; - private static final String WRITE_PROP_PREFIX = "iceberg.table.write-props."; + private static final String WRITE_PROP_PREFIX = "iceberg.tables.write-props."; private static final String CATALOG_NAME_PROP = "iceberg.catalog"; private static final String TABLES_PROP = "iceberg.tables"; diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java index 460b18fd7fc2..bb9370d3d563 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/IcebergSinkTask.java @@ -83,14 +83,16 @@ private void close() { @Override public void put(Collection sinkRecords) { - Preconditions.checkNotNull(committer, "Committer wasn't initialized"); - committer.save(sinkRecords); + if (committer != null) { + committer.save(sinkRecords); + } } @Override public void flush(Map currentOffsets) { - Preconditions.checkNotNull(committer, "Committer wasn't initialized"); - committer.save(null); + if (committer != null) { + committer.save(null); + } } @Override diff --git a/settings.gradle b/settings.gradle index 46c85fb65546..cdc69b0e2071 100644 --- a/settings.gradle +++ b/settings.gradle @@ -198,3 +198,7 @@ project(":iceberg-kafka-connect:kafka-connect-events").name = "iceberg-kafka-con include ":iceberg-kafka-connect:kafka-connect" project(":iceberg-kafka-connect:kafka-connect").projectDir = file('kafka-connect/kafka-connect') project(":iceberg-kafka-connect:kafka-connect").name = "iceberg-kafka-connect" + +include ":iceberg-kafka-connect:kafka-connect-runtime" +project(":iceberg-kafka-connect:kafka-connect-runtime").projectDir = file('kafka-connect/kafka-connect-runtime') +project(":iceberg-kafka-connect:kafka-connect-runtime").name = "iceberg-kafka-connect-runtime" From 3d5528aeaaf0552f8c5f57dabc0dac10b91ae5d1 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 29 Jul 2024 15:34:32 -0700 Subject: [PATCH 0528/1019] Flink: improve snapshot compatibility check by comparing projected sort schema in SortKeySerializer. also add unit tests for serializer snapshot. (#10794) --- .../flink/sink/shuffle/SortKeySerializer.java | 23 +- .../TestSortKeySerializerSnapshot.java | 213 ++++++++++++++++++ .../flink/sink/shuffle/SortKeySerializer.java | 23 +- .../TestSortKeySerializerSnapshot.java | 213 ++++++++++++++++++ .../flink/sink/shuffle/SortKeySerializer.java | 31 +-- .../TestSortKeySerializerSnapshot.java | 213 ++++++++++++++++++ 6 files changed, 696 insertions(+), 20 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java index 4ddc5a32d6bf..d1d75019fa2e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -24,7 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; @@ -41,6 +43,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer { @@ -319,12 +322,26 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer sortKeySerializer = (SortKeySerializer) newSerializer; - if (!sortOrder.sameOrder(sortKeySerializer.sortOrder)) { + // Sort order should be identical + SortKeySerializerSnapshot newSnapshot = + (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); + if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(sortKeySerializer.schema, schema); + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java new file mode 100644 index 000000000000..012654603b04 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() 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); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + 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); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} 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 4ddc5a32d6bf..d1d75019fa2e 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 @@ -24,7 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; +import java.util.stream.Collectors; import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; @@ -41,6 +43,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer { @@ -319,12 +322,26 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } - SortKeySerializer sortKeySerializer = (SortKeySerializer) newSerializer; - if (!sortOrder.sameOrder(sortKeySerializer.sortOrder)) { + // Sort order should be identical + SortKeySerializerSnapshot newSnapshot = + (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); + if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(sortKeySerializer.schema, schema); + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override 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 new file mode 100644 index 000000000000..012654603b04 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() 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); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + 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); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + roundTrip(oldSerializer.snapshotConfiguration()); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); + + TypeSerializerSchemaCompatibility resultCompatibility = + oldSnapshot.resolveSchemaCompatibility(newSerializer); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} 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 9c0e65a1fc49..b82fc8250763 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 @@ -24,8 +24,9 @@ import java.nio.ByteBuffer; import java.util.List; import java.util.Objects; +import java.util.Set; import java.util.UUID; -import org.apache.flink.annotation.VisibleForTesting; +import java.util.stream.Collectors; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; @@ -41,6 +42,7 @@ import org.apache.iceberg.SortOrderParser; import org.apache.iceberg.types.CheckCompatibility; import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; class SortKeySerializer extends TypeSerializer { @@ -319,12 +321,25 @@ public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( return TypeSerializerSchemaCompatibility.incompatible(); } + // Sort order should be identical SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { return TypeSerializerSchemaCompatibility.incompatible(); } - return resolveSchemaCompatibility(oldSnapshot.schema, schema); + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema oldSortSchema = TypeUtil.project(oldSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, oldSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); } @Override @@ -340,17 +355,5 @@ private void readV1(DataInputView in) throws IOException { this.schema = SchemaParser.fromJson(schemaJson); this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); } - - @VisibleForTesting - static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } } } 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 new file mode 100644 index 000000000000..c0f688f2589e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() 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); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + 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 = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} From 70b0aa68c8c119d09c67ce7ece5cd5fe2e2257e7 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 29 Jul 2024 15:35:18 -0700 Subject: [PATCH 0529/1019] Flink: support limit pushdown in FLIP-27 source (#10748) --- .../iceberg/flink/source/IcebergSource.java | 3 +- .../source/reader/LimitableDataIterator.java | 56 +++++++++++++ .../flink/source/reader/RecordLimiter.java | 45 ++++++++++ .../source/reader/RowDataReaderFunction.java | 40 ++++++++- .../flink/source/TestFlinkSourceConfig.java | 7 +- .../reader/TestLimitableDataIterator.java | 84 +++++++++++++++++++ 6 files changed, 228 insertions(+), 7 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java 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 48201ea09359..ccbd0d9997ed 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 @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.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.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * 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 java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ 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.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} From 7fae4bf0dc094c912dd59c2896f47150d34cd112 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 31 Jul 2024 00:18:39 +0900 Subject: [PATCH 0530/1019] Flink: Remove MiniClusterResource (#10817) --- .../iceberg/flink/MiniClusterResource.java | 53 --------------- .../org/apache/iceberg/flink/TestBase.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 4 +- .../flink/sink/TestFlinkIcebergSink.java | 7 +- .../sink/TestFlinkIcebergSinkBranch.java | 4 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 5 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 4 +- .../flink/source/ChangeLogTableTestBase.java | 4 +- .../iceberg/flink/source/TestFlinkScan.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 2 +- .../iceberg/flink/source/TestSqlBase.java | 2 +- .../flink/source/TestStreamScanSql.java | 4 +- .../iceberg/flink/MiniClusterResource.java | 68 ------------------- .../org/apache/iceberg/flink/TestBase.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- ...TestBucketPartitionerFlinkIcebergSink.java | 4 +- .../flink/sink/TestFlinkIcebergSink.java | 7 +- .../sink/TestFlinkIcebergSinkBranch.java | 4 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 5 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 4 +- .../flink/source/ChangeLogTableTestBase.java | 4 +- .../iceberg/flink/source/TestFlinkScan.java | 2 +- .../source/TestIcebergSourceContinuous.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 2 +- .../iceberg/flink/source/TestSqlBase.java | 2 +- .../flink/source/TestStreamScanSql.java | 4 +- .../iceberg/flink/MiniClusterResource.java | 68 ------------------- .../org/apache/iceberg/flink/TestBase.java | 2 +- .../iceberg/flink/TestFlinkTableSink.java | 2 +- .../apache/iceberg/flink/TestFlinkUpsert.java | 2 +- .../iceberg/flink/TestIcebergConnector.java | 2 +- .../operator/OperatorTestBase.java | 4 +- ...TestBucketPartitionerFlinkIcebergSink.java | 4 +- .../flink/sink/TestFlinkIcebergSink.java | 7 +- .../sink/TestFlinkIcebergSinkBranch.java | 4 +- .../flink/sink/TestFlinkIcebergSinkV2.java | 5 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 4 +- .../flink/source/ChangeLogTableTestBase.java | 4 +- .../iceberg/flink/source/TestFlinkScan.java | 2 +- ...stIcebergSourceWithWatermarkExtractor.java | 2 +- ...estIcebergSpeculativeExecutionSupport.java | 2 +- .../iceberg/flink/source/TestSqlBase.java | 2 +- .../flink/source/TestStreamScanSql.java | 4 +- 48 files changed, 70 insertions(+), 265 deletions(-) delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java delete mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java delete mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 45af9241b743..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java +++ /dev/null @@ -1,53 +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; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..a74226092f38 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -43,7 +43,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..b73300e3f170 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..d52d54e159e6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 013b98e3b82b..b709c0058f7d 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java index dc3eb93280df..ba0ea867ffb7 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java index 55909874ccce..577c54976b9a 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java index ffeab673386d..0b0c55f51c32 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d6cf6791270e..57ee7baf202c 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 399d7aaff64c..000000000000 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.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.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = - new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 6367a064f283..a74226092f38 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -43,7 +43,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b7fce104f490..b73300e3f170 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index 5674c83e40b8..d52d54e159e6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index 013b98e3b82b..b709c0058f7d 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); 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 dc3eb93280df..ba0ea867ffb7 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 @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); 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 55909874ccce..577c54976b9a 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 @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); 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 ffeab673386d..0b0c55f51c32 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 @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java index 749cbf89338a..e0e2bf5e61e2 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -67,7 +67,7 @@ public class TestIcebergSourceContinuous { @TempDir protected Path temporaryFolder; @RegisterExtension - public static MiniClusterExtension miniClusterExtension = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); @RegisterExtension diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index d6cf6791270e..57ee7baf202c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.java deleted file mode 100644 index 399d7aaff64c..000000000000 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniClusterResource.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.flink; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.util.MiniClusterWithClientResource; - -public class MiniClusterResource { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniClusterResource() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled() { - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterWithClientResource createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = - new Configuration(MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterWithClientResource( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(MiniClusterResource.DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(MiniClusterResource.DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 773d22e19e64..633690044692 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -44,7 +44,7 @@ public abstract class TestBase extends TestBaseUtils { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; 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 3f66174049a4..a0341e6834d4 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 @@ -91,7 +91,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java index baf13017ff99..c5becb6caca1 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -75,7 +75,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index c978ec6f8bd4..fdb0e0cf19df 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -176,7 +176,7 @@ protected TableEnvironment getTableEnv() { settingsBuilder.inStreamingMode(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); env.setMaxParallelism(2); env.setParallelism(2); 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 269ae681b02e..272e0b693fd3 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 @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import org.apache.flink.configuration.Configuration; import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; @@ -34,7 +34,7 @@ class OperatorTestBase { static final String TABLE_NAME = "test_table"; @RegisterExtension - protected static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) 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 dc3eb93280df..ba0ea867ffb7 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 @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import static org.apache.iceberg.flink.TestFixtures.DATABASE; import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; import static org.assertj.core.api.Assertions.assertThat; @@ -63,7 +63,7 @@ public class TestBucketPartitionerFlinkIcebergSink { private static final int SLOTS_PER_TASK_MANAGER = 8; @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_RESOURCE = + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 8cad35c859c4..61ab087f2ca3 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -44,7 +44,6 @@ import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; @@ -62,7 +61,7 @@ public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -113,7 +112,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); @@ -271,7 +270,7 @@ public void testTwoSinksInDisjointedDAG() throws Exception { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java index 3edaafca0e42..441b5ed2a4ae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java @@ -36,7 +36,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; @@ -88,7 +88,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); 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 55909874ccce..577c54976b9a 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 @@ -38,7 +38,6 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; @@ -57,7 +56,7 @@ @Timeout(value = 60) public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension @@ -89,7 +88,7 @@ public void setupTable() { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100L) .setParallelism(parallelism) .setMaxParallelism(parallelism); 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 ffeab673386d..0b0c55f51c32 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 @@ -31,7 +31,7 @@ import org.apache.iceberg.SnapshotRef; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -71,7 +71,7 @@ public void before() throws IOException { env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100); tableLoader = CATALOG_EXTENSION.tableLoader(); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java index d3748e008b8e..5dfbbe3abe73 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -28,7 +28,7 @@ import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; import org.apache.flink.types.Row; import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.TestBase; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -61,7 +61,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG) + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(400) .setMaxParallelism(1) .setParallelism(1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java index 049ddf9e3f1e..cf6b233dcec6 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -64,7 +64,7 @@ @ExtendWith(ParameterizedTestExtension.class) public abstract class TestFlinkScan { @RegisterExtension - protected static MiniClusterExtension miniClusterResource = + protected static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @TempDir protected Path temporaryDirectory; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java index 5e6a2b3caec6..70889f4f76aa 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -18,7 +18,7 @@ */ package org.apache.iceberg.flink.source; -import static org.apache.iceberg.flink.MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; import java.io.Serializable; import java.nio.file.Path; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 3285a16a1277..b21010a91bed 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -55,7 +55,7 @@ public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_SLOTS = 3; @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUM_TASK_MANAGERS) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java index 8013bce3f415..f9b776397cfc 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -52,7 +52,7 @@ /** Test other more advanced usage of SQL. They don't need to run for every file format. */ public abstract class TestSqlBase { @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static MiniClusterExtension miniClusterExtension = MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); @RegisterExtension diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java index fcf5c1479df5..97ed4ca1e93f 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -42,7 +42,7 @@ import org.apache.iceberg.data.GenericRecord; import org.apache.iceberg.data.Record; import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniClusterResource; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.junit.jupiter.api.AfterEach; @@ -70,7 +70,7 @@ protected TableEnvironment getTableEnv() { StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment( - MiniClusterResource.DISABLE_CLASSLOADER_CHECK_CONFIG); + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); env.enableCheckpointing(400); StreamTableEnvironment streamTableEnv = From 6cae31b0545f084656683233628f2c6aed5e6420 Mon Sep 17 00:00:00 2001 From: liu yang Date: Wed, 31 Jul 2024 05:25:56 +0800 Subject: [PATCH 0531/1019] Docs: Use link addresses instead of descriptions in releases.md (#10815) --- site/docs/releases.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/site/docs/releases.md b/site/docs/releases.md index 73d8bc32a071..cc29857ed802 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -83,7 +83,7 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf - Add EnvironmentContext to commit summary ([\#9273](https://github.com/apache/iceberg/pull/9273)) - Add explicit JSON parser for ConfigResponse ([\#9952](https://github.com/apache/iceberg/pull/9952)) - Calling rewrite_position_delete_files fails on tables with more than 1k columns ([\#10020](https://github.com/apache/iceberg/pull/10020)) - - Expose table incremental scan for appends API in SerializableTable ([\#10682](Expose table incremental scan for appends API in SerializableTable)) + - Expose table incremental scan for appends API in SerializableTable ([\#10682](https://github.com/apache/iceberg/pull/10682)) - Fix NPE during conflict handling of NULL partitions ([\#10680](https://github.com/apache/iceberg/pull/10680)) - Fix ParallelIterable memory leak where queue continues to be populated even after iterator close ([\#9402](https://github.com/apache/iceberg/pull/9402)) - Fix logging table name in scanning metadata table ([\#10141](https://github.com/apache/iceberg/pull/10141)) @@ -100,7 +100,7 @@ The 1.6.0 release contains fixes, dependency updates, and new features (like Kaf - REST Catalog: Handles possible heap data corruption of OAuth2Util.AuthSession#headers ([\#10615](https://github.com/apache/iceberg/pull/10615)) - REST Catalog: Handles potential NPE in RESTSessionCatalog#newSessionCache ([\#10607](https://github.com/apache/iceberg/pull/10607)) - REST Catalog: Introduce AuthConfig ([\#10161](https://github.com/apache/iceberg/pull/10161)) - - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](Mark 502 and 504 statuses as retryable to the REST exponential retry strategy)) + - REST Catalog: Mark 502 and 504 statuses as retryable to the REST exponential retry strategy ([\#9885](https://github.com/apache/iceberg/pull/9885)) - REST Catalog: disallow overriding "credential" in table sessions ([\#10345](https://github.com/apache/iceberg/pull/10345)) - REST Catalog: fix incorrect token refresh thread name ([\#10223](https://github.com/apache/iceberg/pull/10223)) - REST Catalog: fix spurious warning when shutting down refresh executor ([\#10087](https://github.com/apache/iceberg/pull/10087)) From 1228e1f9c583eb030cc7aa102e2f89a7b37394a8 Mon Sep 17 00:00:00 2001 From: Devin Smith Date: Tue, 30 Jul 2024 14:26:55 -0700 Subject: [PATCH 0532/1019] Build: Declare avro as an api dependency of iceberg-core (#10573) iceberg-core should declare an api dependency on avro. For example, the public class `org.apache.iceberg.PartitionData` extends avro-specific types. In addition, there are public methods that deal with avro types, see `org.apache.iceberg.avro.AvroSchemaUtil` --- build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle b/build.gradle index 09a04e3ae977..a0b1a2e018bf 100644 --- a/build.gradle +++ b/build.gradle @@ -347,7 +347,7 @@ project(':iceberg-core') { annotationProcessor libs.immutables.value compileOnly libs.immutables.value - implementation(libs.avro.avro) { + api(libs.avro.avro) { exclude group: 'org.tukaani' // xz compression is not supported } From 146ea290d2b81d4f7eb7a6bb1196ccd173639c33 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 30 Jul 2024 14:49:44 -0700 Subject: [PATCH 0533/1019] Flink: backport PR #10748 for limit pushdown (#10813) --- .../iceberg/flink/source/IcebergSource.java | 3 +- .../source/reader/LimitableDataIterator.java | 56 +++++++++++++ .../flink/source/reader/RecordLimiter.java | 45 ++++++++++ .../source/reader/RowDataReaderFunction.java | 40 ++++++++- .../flink/source/TestFlinkSourceConfig.java | 7 +- .../reader/TestLimitableDataIterator.java | 84 +++++++++++++++++++ .../iceberg/flink/source/IcebergSource.java | 3 +- .../source/reader/LimitableDataIterator.java | 56 +++++++++++++ .../flink/source/reader/RecordLimiter.java | 45 ++++++++++ .../source/reader/RowDataReaderFunction.java | 40 ++++++++- .../flink/source/TestFlinkSourceConfig.java | 7 +- .../reader/TestLimitableDataIterator.java | 84 +++++++++++++++++++ 12 files changed, 456 insertions(+), 14 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java index 48201ea09359..ccbd0d9997ed 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.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.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * 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 java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ 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.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} 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 48201ea09359..ccbd0d9997ed 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 @@ -521,7 +521,8 @@ public IcebergSource build() { context.caseSensitive(), table.io(), table.encryption(), - context.filters()); + context.filters(), + context.limit()); this.readerFunction = (ReaderFunction) rowDataReaderFunction; } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.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.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * 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 java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java index 5d0a00954e7a..c9208a0e1834 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java @@ -39,6 +39,9 @@ public class RowDataReaderFunction extends DataIteratorReaderFunction { private final FileIO io; private final EncryptionManager encryption; private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; public RowDataReaderFunction( ReadableConfig config, @@ -49,6 +52,28 @@ public RowDataReaderFunction( FileIO io, EncryptionManager encryption, List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { super( new ArrayPoolDataIteratorBatcher<>( config, @@ -61,19 +86,30 @@ public RowDataReaderFunction( this.io = io; this.encryption = encryption; this.filters = filters; + this.limit = limit; } @Override public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( + return new LimitableDataIterator<>( new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), split.task(), io, - encryption); + encryption, + lazyLimiter()); } private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); return projectedSchema == null ? tableSchema : projectedSchema; } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java index 8131bd7ab0d3..14131d9e96d5 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -20,7 +20,6 @@ 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.List; import org.apache.flink.types.Row; @@ -49,11 +48,11 @@ public void testFlinkHintConfig() { @TestTemplate public void testReadOptionHierarchy() { - // TODO: FLIP-27 source doesn't implement limit pushdown yet - assumeThat(useFlip27Source).isFalse(); - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. assertThat(result).hasSize(1); result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} From 3ea3ff878d7b1864529ac7e1d2e393c25d5ac350 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Wed, 31 Jul 2024 12:59:13 +0200 Subject: [PATCH 0534/1019] Docs: Fix header for entries metadata table (#10826) --- docs/docs/spark-queries.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/docs/spark-queries.md b/docs/docs/spark-queries.md index b606d849a692..494ca698533a 100644 --- a/docs/docs/spark-queries.md +++ b/docs/docs/spark-queries.md @@ -288,6 +288,7 @@ order by made_current_at; | 2019-02-09 16:24:30.13 | delete | 29641004024753 | false | application_1520379288616_151109 | | 2019-02-09 16:32:47.336 | append | 57897183625154 | true | application_1520379288616_155055 | | 2019-02-08 03:47:55.948 | overwrite | 51792995261850 | true | application_1520379288616_152431 | + ### Entries To show all the table's current manifest entries for both data and delete files. From 9ad682c64b7ad92d3a7846855b3adcee49e6513a Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 31 Jul 2024 08:41:20 -0700 Subject: [PATCH 0535/1019] Spark 3.5: Support Reporting Column Stats (#10659) Co-authored-by: Karuppayya Rajendran --- .../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 | 54 +++++- .../apache/iceberg/spark/source/Stats.java | 12 +- .../iceberg/spark/source/TestSparkScan.java | 183 ++++++++++++++++++ 7 files changed, 346 insertions(+), 5 deletions(-) create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java index bd29fb0d6d42..67e9d78ada4d 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkReadConf.java @@ -347,4 +347,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.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index ea8f6fe0718b..77ae796ffb76 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -90,4 +90,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.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java index 7cde3e1fbe11..71b53d70262f 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkChangelogScan.java +++ b/spark/v3.5/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.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkColumnStatistics.java new file mode 100644 index 000000000000..faaff3631d7c --- /dev/null +++ b/spark/v3.5/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.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 6efe8a080bde..8b88cf49c692 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 @@ -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,43 @@ 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(); + + for (BlobMetadata blobMetadata : metadataList) { + int id = blobMetadata.fields().get(0); + String colName = table.schema().findColumnName(id); + 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); + } else { + LOG.debug("ndv is not set in BlobMetadata for column {}", colName); + } + } else { + LOG.debug("DataSketch blob is not available for column {}", colName); + } + + 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 +233,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.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java index ddf6ca834d9b..ccf523cb4b05 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/Stats.java +++ b/spark/v3.5/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.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 d539b0123951..7d5475ff919e 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 @@ -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,14 +29,22 @@ 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.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; 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.SparkCatalogConfig; +import org.apache.iceberg.spark.SparkSQLProperties; import org.apache.iceberg.spark.TestBaseWithCatalog; import org.apache.iceberg.spark.functions.BucketFunction; import org.apache.iceberg.spark.functions.DaysFunction; @@ -44,6 +53,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; @@ -60,6 +70,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.jupiter.api.AfterEach; @@ -130,6 +142,157 @@ public void testEstimatedRowCount() throws NoSuchTableException { assertThat(stats.numRows().getAsLong()).isEqualTo(10000L); } + @TestTemplate + 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())); + } + + @TestTemplate + 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)); + } + + @TestTemplate + 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)); + } + @TestTemplate public void testUnpartitionedYears() throws Exception { createUnpartitionedTable(spark, tableName); @@ -734,6 +897,26 @@ 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(); + 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); } From 9aa5f678d8082f78aa115a29b263fe16d3bb50dc Mon Sep 17 00:00:00 2001 From: Venkata krishnan Sowrirajan Date: Thu, 1 Aug 2024 00:22:51 -0700 Subject: [PATCH 0536/1019] Flink: Backport #10548 to v1.18 and v1.17 (#10776) --- .../enumerator/AbstractIcebergEnumerator.java | 11 +- ...estIcebergSpeculativeExecutionSupport.java | 184 ++++++++++++++++++ .../enumerator/AbstractIcebergEnumerator.java | 11 +- ...estIcebergSpeculativeExecutionSupport.java | 184 ++++++++++++++++++ ...estIcebergSpeculativeExecutionSupport.java | 2 +- 5 files changed, 389 insertions(+), 3 deletions(-) create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 3aca390755ed..801baf77a612 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -41,7 +42,8 @@ * resolved */ abstract class AbstractIcebergEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -93,6 +95,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..95d0b90b6ca9 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + sql("DROP DATABASE %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 6c9a855bc149..280a126a46ce 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -29,6 +29,7 @@ import org.apache.flink.api.connector.source.SourceEvent; import org.apache.flink.api.connector.source.SplitEnumerator; import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; import org.apache.iceberg.flink.source.assigner.GetSplitResult; import org.apache.iceberg.flink.source.assigner.SplitAssigner; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; @@ -37,7 +38,8 @@ import org.slf4j.LoggerFactory; abstract class AbstractIcebergEnumerator - implements SplitEnumerator { + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); private final SplitEnumeratorContext enumeratorContext; @@ -95,6 +97,13 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } } + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + @Override public void addSplitsBack(List splits, int subtaskId) { LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..95d0b90b6ca9 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + sql("DROP DATABASE %s", DATABASE_NAME); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getIndexOfThisSubtask(), + getRuntimeContext().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index b21010a91bed..41b023b93617 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -144,7 +144,7 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the even subtask indices with the first attempt to sleep to trigger speculative + // Put the subtasks with the first attempt to sleep to trigger speculative // execution if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); From 79653c46a9f15e595aa0820a5041398866e9fe10 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 1 Aug 2024 09:30:04 +0200 Subject: [PATCH 0537/1019] Infra: Improve feature request template (#10825) --- .github/ISSUE_TEMPLATE/iceberg_improvement.yml | 10 +++++++++- 1 file changed, 9 insertions(+), 1 deletion(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml index f3d6d6de923a..80048f99465a 100644 --- a/.github/ISSUE_TEMPLATE/iceberg_improvement.yml +++ b/.github/ISSUE_TEMPLATE/iceberg_improvement.yml @@ -50,4 +50,12 @@ body: - Hive - Other validations: - required: false \ No newline at end of file + required: false + - type: checkboxes + attributes: + label: Willingness to contribute + description: The Apache Iceberg community encourages contributions. Would you or another member of your organization be willing to contribute this improvement/feature to the Apache Iceberg codebase? + options: + - label: I can contribute this improvement/feature independently + - label: I would be willing to contribute this improvement/feature with guidance from the Iceberg community + - label: I cannot contribute this improvement/feature at this time From 2e72734a44d7150845f8d39bede5d84da4b5e10e Mon Sep 17 00:00:00 2001 From: hsiang-c <137842490+hsiang-c@users.noreply.github.com> Date: Thu, 1 Aug 2024 23:23:05 +0800 Subject: [PATCH 0538/1019] Core: Replace the duplicated ALL_DATA_FILES with ALL_DELETE_FILES (#10836) --- .../java/org/apache/iceberg/TestMetadataTableFilters.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index 8125e064f0d0..9e535dd77747 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -40,7 +40,7 @@ public class TestMetadataTableFilters extends TestBase { private static final Set AGG_FILE_TABLES = Sets.newHashSet( MetadataTableType.ALL_DATA_FILES, - MetadataTableType.ALL_DATA_FILES, + MetadataTableType.ALL_DELETE_FILES, MetadataTableType.ALL_FILES, MetadataTableType.ALL_ENTRIES); @@ -132,9 +132,9 @@ private int expectedScanTaskCount(int partitions) { } case DATA_FILES: case DELETE_FILES: - case ALL_DELETE_FILES: return partitions; case ALL_DATA_FILES: + case ALL_DELETE_FILES: return partitions * 2; // ScanTask for Data Manifest in DELETED and ADDED states case ALL_FILES: case ALL_ENTRIES: From 57f8cb133bc7c42a36aed71dbd3162bfbc02325f Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 1 Aug 2024 10:23:49 -0500 Subject: [PATCH 0539/1019] Core: Adds Basic Classes for Iceberg Table Version 3 (#10760) --- .../iceberg/BaseUpdatePartitionSpec.java | 2 +- .../org/apache/iceberg/ManifestFiles.java | 4 + .../apache/iceberg/ManifestListWriter.java | 35 ++ .../org/apache/iceberg/ManifestLists.java | 3 + .../org/apache/iceberg/ManifestWriter.java | 73 +++ .../org/apache/iceberg/TableMetadata.java | 2 +- .../java/org/apache/iceberg/V3Metadata.java | 575 ++++++++++++++++++ .../iceberg/DeleteFileIndexTestBase.java | 2 +- .../iceberg/MetadataTableScanTestBase.java | 2 +- .../ScanPlanningAndReportingTestBase.java | 2 +- .../java/org/apache/iceberg/TestBase.java | 2 +- .../org/apache/iceberg/TestBatchScans.java | 2 +- .../apache/iceberg/TestCommitReporting.java | 2 +- .../apache/iceberg/TestCreateTransaction.java | 2 +- .../iceberg/TestEntriesMetadataTable.java | 2 +- .../org/apache/iceberg/TestFastAppend.java | 2 +- .../org/apache/iceberg/TestFindFiles.java | 2 +- .../apache/iceberg/TestFormatVersions.java | 2 +- .../iceberg/TestIncrementalDataTableScan.java | 2 +- .../apache/iceberg/TestLocalFilterFiles.java | 2 +- .../apache/iceberg/TestLocationProvider.java | 2 +- .../apache/iceberg/TestManifestCleanup.java | 2 +- .../iceberg/TestManifestReaderStats.java | 2 +- .../apache/iceberg/TestManifestWriter.java | 2 +- .../iceberg/TestMetadataTableScans.java | 46 +- ...adataTableScansWithPartitionEvolution.java | 3 +- .../java/org/apache/iceberg/TestMetrics.java | 2 +- .../org/apache/iceberg/TestMetricsModes.java | 2 +- .../apache/iceberg/TestMicroBatchBuilder.java | 2 +- .../apache/iceberg/TestPartitionSpecInfo.java | 2 +- .../iceberg/TestReplaceTransaction.java | 2 +- .../apache/iceberg/TestRewriteManifests.java | 2 +- .../org/apache/iceberg/TestScanSummary.java | 2 +- .../iceberg/TestScansAndSchemaEvolution.java | 2 +- .../iceberg/TestSchemaAndMappingUpdate.java | 2 +- .../java/org/apache/iceberg/TestSchemaID.java | 2 +- .../iceberg/TestSetPartitionStatistics.java | 2 +- .../org/apache/iceberg/TestSetStatistics.java | 2 +- .../java/org/apache/iceberg/TestSnapshot.java | 2 +- .../apache/iceberg/TestSnapshotLoading.java | 2 +- .../apache/iceberg/TestSnapshotManager.java | 2 +- .../apache/iceberg/TestSnapshotSelection.java | 2 +- .../apache/iceberg/TestSnapshotSummary.java | 2 +- .../org/apache/iceberg/TestSortOrder.java | 2 +- .../org/apache/iceberg/TestSplitPlanning.java | 2 +- .../TestTableMetadataSerialization.java | 2 +- .../iceberg/TestTableUpdatePartitionSpec.java | 2 +- .../iceberg/TestTimestampPartitions.java | 2 +- .../org/apache/iceberg/TestTransaction.java | 2 +- .../iceberg/TestUpdatePartitionSpec.java | 2 +- .../org/apache/iceberg/TestWapWorkflow.java | 2 +- .../actions/TestSizeBasedRewriter.java | 2 +- .../iceberg/io/TestOutputFileFactory.java | 2 +- .../iceberg/mapping/TestMappingUpdates.java | 2 +- .../TableMetadataUnsupportedVersion.json | 4 +- 55 files changed, 762 insertions(+), 75 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/V3Metadata.java diff --git a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java index b59292c397a1..2e1c9199174c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java +++ b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java @@ -118,7 +118,7 @@ private int assignFieldId() { */ private PartitionField recycleOrCreatePartitionField( Pair> sourceTransform, String name) { - if (formatVersion == 2 && base != null) { + if (formatVersion >= 2 && base != null) { int sourceId = sourceTransform.first(); Transform transform = sourceTransform.second(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestFiles.java b/core/src/main/java/org/apache/iceberg/ManifestFiles.java index 840c90bebdde..f630bb3eb743 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFiles.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFiles.java @@ -182,6 +182,8 @@ public static ManifestWriter write( return new ManifestWriter.V1Writer(spec, encryptedOutputFile, snapshotId); case 2: return new ManifestWriter.V2Writer(spec, encryptedOutputFile, snapshotId); + case 3: + return new ManifestWriter.V3Writer(spec, encryptedOutputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); @@ -238,6 +240,8 @@ public static ManifestWriter writeDeleteManifest( throw new IllegalArgumentException("Cannot write delete files in a v1 table"); case 2: return new ManifestWriter.V2DeleteWriter(spec, outputFile, snapshotId); + case 3: + return new ManifestWriter.V3DeleteWriter(spec, outputFile, snapshotId); } throw new UnsupportedOperationException( "Cannot write manifest for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java index 3f7f20d4df6c..b17eedad18af 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestListWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestListWriter.java @@ -70,6 +70,41 @@ public long length() { return writer.length(); } + static class V3Writer extends ManifestListWriter { + private final V3Metadata.IndexedManifestFile wrapper; + + V3Writer(OutputFile snapshotFile, long snapshotId, Long parentSnapshotId, long sequenceNumber) { + super( + snapshotFile, + ImmutableMap.of( + "snapshot-id", String.valueOf(snapshotId), + "parent-snapshot-id", String.valueOf(parentSnapshotId), + "sequence-number", String.valueOf(sequenceNumber), + "format-version", "3")); + this.wrapper = new V3Metadata.IndexedManifestFile(snapshotId, sequenceNumber); + } + + @Override + protected ManifestFile prepare(ManifestFile manifest) { + return wrapper.wrap(manifest); + } + + @Override + protected FileAppender newAppender(OutputFile file, Map meta) { + try { + return Avro.write(file) + .schema(V3Metadata.MANIFEST_LIST_SCHEMA) + .named("manifest_file") + .meta(meta) + .overwrite() + .build(); + + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create snapshot list writer for path: %s", file); + } + } + } + static class V2Writer extends ManifestListWriter { private final V2Metadata.IndexedManifestFile wrapper; diff --git a/core/src/main/java/org/apache/iceberg/ManifestLists.java b/core/src/main/java/org/apache/iceberg/ManifestLists.java index c7b3e5fee5a9..f20a481cf25a 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestLists.java +++ b/core/src/main/java/org/apache/iceberg/ManifestLists.java @@ -66,6 +66,9 @@ static ManifestListWriter write( case 2: return new ManifestListWriter.V2Writer( manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); + case 3: + return new ManifestListWriter.V3Writer( + manifestListFile, snapshotId, parentSnapshotId, sequenceNumber); } throw new UnsupportedOperationException( "Cannot write manifest list for table version: " + formatVersion); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index cea907ddaca1..88587a1ebc89 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -217,6 +217,79 @@ public void close() throws IOException { writer.close(); } + static class V3Writer extends ManifestWriter { + private final V3Metadata.IndexedManifestEntry entryWrapper; + + V3Writer(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry prepare(ManifestEntry entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender> newAppender( + PartitionSpec spec, OutputFile file) { + Schema manifestSchema = V3Metadata.entrySchema(spec.partitionType()); + try { + return Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "3") + .meta("content", "data") + .overwrite() + .build(); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); + } + } + } + + static class V3DeleteWriter extends ManifestWriter { + private final V3Metadata.IndexedManifestEntry entryWrapper; + + V3DeleteWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapshotId) { + super(spec, file, snapshotId); + this.entryWrapper = new V3Metadata.IndexedManifestEntry<>(snapshotId, spec.partitionType()); + } + + @Override + protected ManifestEntry prepare(ManifestEntry entry) { + return entryWrapper.wrap(entry); + } + + @Override + protected FileAppender> newAppender( + PartitionSpec spec, OutputFile file) { + Schema manifestSchema = V3Metadata.entrySchema(spec.partitionType()); + try { + return Avro.write(file) + .schema(manifestSchema) + .named("manifest_entry") + .meta("schema", SchemaParser.toJson(spec.schema())) + .meta("partition-spec", PartitionSpecParser.toJsonFields(spec)) + .meta("partition-spec-id", String.valueOf(spec.specId())) + .meta("format-version", "3") + .meta("content", "deletes") + .overwrite() + .build(); + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to create manifest writer for path: %s", file); + } + } + + @Override + protected ManifestContent content() { + return ManifestContent.DELETES; + } + } + static class V2Writer extends ManifestWriter { private final V2Metadata.IndexedManifestEntry entryWrapper; diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 74b8ad0bbddc..bd1c8a1a0371 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -51,7 +51,7 @@ public class TableMetadata implements Serializable { static final long INITIAL_SEQUENCE_NUMBER = 0; static final long INVALID_SEQUENCE_NUMBER = -1; static final int DEFAULT_TABLE_FORMAT_VERSION = 2; - static final int SUPPORTED_TABLE_FORMAT_VERSION = 2; + static final int SUPPORTED_TABLE_FORMAT_VERSION = 3; static final int INITIAL_SPEC_ID = 0; static final int INITIAL_SORT_ORDER_ID = 1; static final int INITIAL_SCHEMA_ID = 0; diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java new file mode 100644 index 000000000000..94e20ea99858 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -0,0 +1,575 @@ +/* + * 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.required; + +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Map; +import java.util.Set; +import org.apache.avro.generic.IndexedRecord; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Types; + +class V3Metadata { + private V3Metadata() {} + + static final Schema MANIFEST_LIST_SCHEMA = + new Schema( + ManifestFile.PATH, + ManifestFile.LENGTH, + ManifestFile.SPEC_ID, + ManifestFile.MANIFEST_CONTENT.asRequired(), + ManifestFile.SEQUENCE_NUMBER.asRequired(), + ManifestFile.MIN_SEQUENCE_NUMBER.asRequired(), + ManifestFile.SNAPSHOT_ID.asRequired(), + ManifestFile.ADDED_FILES_COUNT.asRequired(), + ManifestFile.EXISTING_FILES_COUNT.asRequired(), + ManifestFile.DELETED_FILES_COUNT.asRequired(), + ManifestFile.ADDED_ROWS_COUNT.asRequired(), + ManifestFile.EXISTING_ROWS_COUNT.asRequired(), + ManifestFile.DELETED_ROWS_COUNT.asRequired(), + ManifestFile.PARTITION_SUMMARIES, + ManifestFile.KEY_METADATA); + + /** + * A wrapper class to write any ManifestFile implementation to Avro using the v3 write schema. + * + *

    This is used to maintain compatibility with v3 by writing manifest list files with the old + * schema, instead of writing a sequence number into metadata files in v3 tables. + */ + static class IndexedManifestFile implements ManifestFile, IndexedRecord { + private static final org.apache.avro.Schema AVRO_SCHEMA = + AvroSchemaUtil.convert(MANIFEST_LIST_SCHEMA, "manifest_file"); + + private final long commitSnapshotId; + private final long sequenceNumber; + private ManifestFile wrapped = null; + + IndexedManifestFile(long commitSnapshotId, long sequenceNumber) { + this.commitSnapshotId = commitSnapshotId; + this.sequenceNumber = sequenceNumber; + } + + public ManifestFile wrap(ManifestFile file) { + this.wrapped = file; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return AVRO_SCHEMA; + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedManifestFile wrapper via put"); + } + + @Override + public Object get(int pos) { + switch (pos) { + case 0: + return wrapped.path(); + case 1: + return wrapped.length(); + case 2: + return wrapped.partitionSpecId(); + case 3: + return wrapped.content().id(); + case 4: + if (wrapped.sequenceNumber() == ManifestWriter.UNASSIGNED_SEQ) { + // if the sequence number is being assigned here, then the manifest must be created by + // the current + // operation. to validate this, check that the snapshot id matches the current commit + Preconditions.checkState( + commitSnapshotId == wrapped.snapshotId(), + "Found unassigned sequence number for a manifest from snapshot: %s", + wrapped.snapshotId()); + return sequenceNumber; + } else { + return wrapped.sequenceNumber(); + } + case 5: + if (wrapped.minSequenceNumber() == ManifestWriter.UNASSIGNED_SEQ) { + // same sanity check as above + Preconditions.checkState( + commitSnapshotId == wrapped.snapshotId(), + "Found unassigned sequence number for a manifest from snapshot: %s", + wrapped.snapshotId()); + // if the min sequence number is not determined, then there was no assigned sequence + // number for any file + // written to the wrapped manifest. replace the unassigned sequence number with the one + // for this commit + return sequenceNumber; + } else { + return wrapped.minSequenceNumber(); + } + case 6: + return wrapped.snapshotId(); + case 7: + return wrapped.addedFilesCount(); + case 8: + return wrapped.existingFilesCount(); + case 9: + return wrapped.deletedFilesCount(); + case 10: + return wrapped.addedRowsCount(); + case 11: + return wrapped.existingRowsCount(); + case 12: + return wrapped.deletedRowsCount(); + case 13: + return wrapped.partitions(); + case 14: + return wrapped.keyMetadata(); + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + } + } + + @Override + public String path() { + return wrapped.path(); + } + + @Override + public long length() { + return wrapped.length(); + } + + @Override + public int partitionSpecId() { + return wrapped.partitionSpecId(); + } + + @Override + public ManifestContent content() { + return wrapped.content(); + } + + @Override + public long sequenceNumber() { + return wrapped.sequenceNumber(); + } + + @Override + public long minSequenceNumber() { + return wrapped.minSequenceNumber(); + } + + @Override + public Long snapshotId() { + return wrapped.snapshotId(); + } + + @Override + public boolean hasAddedFiles() { + return wrapped.hasAddedFiles(); + } + + @Override + public Integer addedFilesCount() { + return wrapped.addedFilesCount(); + } + + @Override + public Long addedRowsCount() { + return wrapped.addedRowsCount(); + } + + @Override + public boolean hasExistingFiles() { + return wrapped.hasExistingFiles(); + } + + @Override + public Integer existingFilesCount() { + return wrapped.existingFilesCount(); + } + + @Override + public Long existingRowsCount() { + return wrapped.existingRowsCount(); + } + + @Override + public boolean hasDeletedFiles() { + return wrapped.hasDeletedFiles(); + } + + @Override + public Integer deletedFilesCount() { + return wrapped.deletedFilesCount(); + } + + @Override + public Long deletedRowsCount() { + return wrapped.deletedRowsCount(); + } + + @Override + public List partitions() { + return wrapped.partitions(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public ManifestFile copy() { + return wrapped.copy(); + } + } + + static Schema entrySchema(Types.StructType partitionType) { + return wrapFileSchema(fileType(partitionType)); + } + + static Schema wrapFileSchema(Types.StructType fileSchema) { + // this is used to build projection schemas + return new Schema( + ManifestEntry.STATUS, + ManifestEntry.SNAPSHOT_ID, + ManifestEntry.SEQUENCE_NUMBER, + ManifestEntry.FILE_SEQUENCE_NUMBER, + required(ManifestEntry.DATA_FILE_ID, "data_file", fileSchema)); + } + + static Types.StructType fileType(Types.StructType partitionType) { + return Types.StructType.of( + DataFile.CONTENT.asRequired(), + DataFile.FILE_PATH, + DataFile.FILE_FORMAT, + required( + DataFile.PARTITION_ID, DataFile.PARTITION_NAME, partitionType, DataFile.PARTITION_DOC), + DataFile.RECORD_COUNT, + DataFile.FILE_SIZE, + DataFile.COLUMN_SIZES, + DataFile.VALUE_COUNTS, + DataFile.NULL_VALUE_COUNTS, + DataFile.NAN_VALUE_COUNTS, + DataFile.LOWER_BOUNDS, + DataFile.UPPER_BOUNDS, + DataFile.KEY_METADATA, + DataFile.SPLIT_OFFSETS, + DataFile.EQUALITY_IDS, + DataFile.SORT_ORDER_ID); + } + + static class IndexedManifestEntry> + implements ManifestEntry, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final Long commitSnapshotId; + private final IndexedDataFile fileWrapper; + private ManifestEntry wrapped = null; + + IndexedManifestEntry(Long commitSnapshotId, Types.StructType partitionType) { + this.avroSchema = AvroSchemaUtil.convert(entrySchema(partitionType), "manifest_entry"); + this.commitSnapshotId = commitSnapshotId; + this.fileWrapper = new IndexedDataFile<>(partitionType); + } + + public IndexedManifestEntry wrap(ManifestEntry entry) { + this.wrapped = entry; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedManifestEntry wrapper via put"); + } + + @Override + public Object get(int i) { + switch (i) { + case 0: + return wrapped.status().id(); + case 1: + return wrapped.snapshotId(); + case 2: + if (wrapped.dataSequenceNumber() == null) { + // if the entry's data sequence number is null, + // then it will inherit the sequence number of the current commit. + // to validate that this is correct, check that the snapshot id is either null (will + // also be inherited) or that it matches the id of the current commit. + Preconditions.checkState( + wrapped.snapshotId() == null || wrapped.snapshotId().equals(commitSnapshotId), + "Found unassigned sequence number for an entry from snapshot: %s", + wrapped.snapshotId()); + + // inheritance should work only for ADDED entries + Preconditions.checkState( + wrapped.status() == Status.ADDED, + "Only entries with status ADDED can have null sequence number"); + + return null; + } + return wrapped.dataSequenceNumber(); + case 3: + return wrapped.fileSequenceNumber(); + case 4: + return fileWrapper.wrap(wrapped.file()); + default: + throw new UnsupportedOperationException("Unknown field ordinal: " + i); + } + } + + @Override + public Status status() { + return wrapped.status(); + } + + @Override + public Long snapshotId() { + return wrapped.snapshotId(); + } + + @Override + public void setSnapshotId(long snapshotId) { + wrapped.setSnapshotId(snapshotId); + } + + @Override + public Long dataSequenceNumber() { + return wrapped.dataSequenceNumber(); + } + + @Override + public void setDataSequenceNumber(long dataSequenceNumber) { + wrapped.setDataSequenceNumber(dataSequenceNumber); + } + + @Override + public Long fileSequenceNumber() { + return wrapped.fileSequenceNumber(); + } + + @Override + public void setFileSequenceNumber(long fileSequenceNumber) { + wrapped.setFileSequenceNumber(fileSequenceNumber); + } + + @Override + public F file() { + return wrapped.file(); + } + + @Override + public ManifestEntry copy() { + return wrapped.copy(); + } + + @Override + public ManifestEntry copyWithoutStats() { + return wrapped.copyWithoutStats(); + } + } + + /** Wrapper used to write DataFile or DeleteFile to v3 metadata. */ + static class IndexedDataFile implements ContentFile, IndexedRecord { + private final org.apache.avro.Schema avroSchema; + private final IndexedStructLike partitionWrapper; + private ContentFile wrapped = null; + + IndexedDataFile(Types.StructType partitionType) { + this.avroSchema = AvroSchemaUtil.convert(fileType(partitionType), "data_file"); + this.partitionWrapper = new IndexedStructLike(avroSchema.getField("partition").schema()); + } + + @SuppressWarnings("unchecked") + IndexedDataFile wrap(ContentFile file) { + this.wrapped = (ContentFile) file; + return this; + } + + @Override + public org.apache.avro.Schema getSchema() { + return avroSchema; + } + + @Override + public Object get(int pos) { + switch (pos) { + case 0: + return wrapped.content().id(); + case 1: + return wrapped.path().toString(); + case 2: + return wrapped.format() != null ? wrapped.format().toString() : null; + case 3: + return partitionWrapper.wrap(wrapped.partition()); + case 4: + return wrapped.recordCount(); + case 5: + return wrapped.fileSizeInBytes(); + case 6: + return wrapped.columnSizes(); + case 7: + return wrapped.valueCounts(); + case 8: + return wrapped.nullValueCounts(); + case 9: + return wrapped.nanValueCounts(); + case 10: + return wrapped.lowerBounds(); + case 11: + return wrapped.upperBounds(); + case 12: + return wrapped.keyMetadata(); + case 13: + return wrapped.splitOffsets(); + case 14: + return wrapped.equalityFieldIds(); + case 15: + return wrapped.sortOrderId(); + } + throw new IllegalArgumentException("Unknown field ordinal: " + pos); + } + + @Override + public void put(int i, Object v) { + throw new UnsupportedOperationException("Cannot modify IndexedDataFile wrapper via put"); + } + + @Override + public Long pos() { + return null; + } + + @Override + public int specId() { + return wrapped.specId(); + } + + @Override + public FileContent content() { + return wrapped.content(); + } + + @Override + public CharSequence path() { + return wrapped.path(); + } + + @Override + public FileFormat format() { + return wrapped.format(); + } + + @Override + public StructLike partition() { + return wrapped.partition(); + } + + @Override + public long recordCount() { + return wrapped.recordCount(); + } + + @Override + public long fileSizeInBytes() { + return wrapped.fileSizeInBytes(); + } + + @Override + public Map columnSizes() { + return wrapped.columnSizes(); + } + + @Override + public Map valueCounts() { + return wrapped.valueCounts(); + } + + @Override + public Map nullValueCounts() { + return wrapped.nullValueCounts(); + } + + @Override + public Map nanValueCounts() { + return wrapped.nanValueCounts(); + } + + @Override + public Map lowerBounds() { + return wrapped.lowerBounds(); + } + + @Override + public Map upperBounds() { + return wrapped.upperBounds(); + } + + @Override + public ByteBuffer keyMetadata() { + return wrapped.keyMetadata(); + } + + @Override + public List splitOffsets() { + return wrapped.splitOffsets(); + } + + @Override + public List equalityFieldIds() { + return wrapped.equalityFieldIds(); + } + + @Override + public Integer sortOrderId() { + return wrapped.sortOrderId(); + } + + @Override + public Long dataSequenceNumber() { + return wrapped.dataSequenceNumber(); + } + + @Override + public Long fileSequenceNumber() { + return wrapped.fileSequenceNumber(); + } + + @Override + public F copy() { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + + @Override + public F copyWithStats(Set requestedColumnIds) { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + + @Override + public F copyWithoutStats() { + throw new UnsupportedOperationException("Cannot copy IndexedDataFile wrapper"); + } + } +} diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 229650566ca8..836a1ddd80f5 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -46,7 +46,7 @@ public abstract class DeleteFileIndexTestBase< @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } static final DeleteFile FILE_A_POS_1 = diff --git a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java index a4e964b017ba..98d2d8f38af6 100644 --- a/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/MetadataTableScanTestBase.java @@ -40,7 +40,7 @@ public abstract class MetadataTableScanTestBase extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } protected Set scannedPaths(TableScan scan) { diff --git a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java index 3ba74c3b4b71..13e96869b454 100644 --- a/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java +++ b/core/src/test/java/org/apache/iceberg/ScanPlanningAndReportingTestBase.java @@ -47,7 +47,7 @@ public abstract class ScanPlanningAndReportingTestBase< @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } protected abstract ScanT newScan(Table table); diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 2322062dad85..e03a1efd5156 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -174,7 +174,7 @@ public class TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter protected int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestBatchScans.java b/core/src/test/java/org/apache/iceberg/TestBatchScans.java index c7210486e05e..1597f44f6338 100644 --- a/core/src/test/java/org/apache/iceberg/TestBatchScans.java +++ b/core/src/test/java/org/apache/iceberg/TestBatchScans.java @@ -34,7 +34,7 @@ public class TestBatchScans extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java index bbba192fab7f..41b301668722 100644 --- a/core/src/test/java/org/apache/iceberg/TestCommitReporting.java +++ b/core/src/test/java/org/apache/iceberg/TestCommitReporting.java @@ -37,7 +37,7 @@ public class TestCommitReporting extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(2); + return Arrays.asList(2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java index 0c6b50b37792..766dd85a0655 100644 --- a/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestCreateTransaction.java @@ -40,7 +40,7 @@ public class TestCreateTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java index 60e79aad91ce..9bce4e60a4f3 100644 --- a/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java +++ b/core/src/test/java/org/apache/iceberg/TestEntriesMetadataTable.java @@ -34,7 +34,7 @@ public class TestEntriesMetadataTable extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 9dd479ecf0fc..b281536ab0fa 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -39,7 +39,7 @@ public class TestFastAppend extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFindFiles.java b/core/src/test/java/org/apache/iceberg/TestFindFiles.java index 68d757de5cdb..191dbd384197 100644 --- a/core/src/test/java/org/apache/iceberg/TestFindFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestFindFiles.java @@ -36,7 +36,7 @@ public class TestFindFiles extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java index 2f6a01c6e603..4a9f18581d09 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -71,7 +71,7 @@ public void testFormatVersionUpgradeNotSupported() { base, base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot upgrade table to unsupported format version: v3 (supported: v2)"); + .hasMessage("Cannot upgrade table to unsupported format version: v4 (supported: v3)"); assertThat(ops.current().formatVersion()).isEqualTo(1); } diff --git a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java index fe6b9b0c763c..ecd6a14ffefb 100644 --- a/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java +++ b/core/src/test/java/org/apache/iceberg/TestIncrementalDataTableScan.java @@ -43,7 +43,7 @@ public class TestIncrementalDataTableScan extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java index 275b89df3695..8a4f5db256f0 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestLocalFilterFiles.java @@ -28,7 +28,7 @@ public class TestLocalFilterFiles @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java index 93dd1a8555ce..7afb69483490 100644 --- a/core/src/test/java/org/apache/iceberg/TestLocationProvider.java +++ b/core/src/test/java/org/apache/iceberg/TestLocationProvider.java @@ -34,7 +34,7 @@ public class TestLocationProvider extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } // publicly visible for testing to be dynamically loaded diff --git a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java index b5f6d05cc6a0..37ccbb8bb845 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestCleanup.java @@ -30,7 +30,7 @@ public class TestManifestCleanup extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java index 4c60a943f76c..175178e48167 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReaderStats.java @@ -39,7 +39,7 @@ public class TestManifestReaderStats extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final Map VALUE_COUNT = ImmutableMap.of(3, 3L); diff --git a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java index eb7910a79fc5..7dd3ea2d1ce7 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestWriter.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestWriter.java @@ -38,7 +38,7 @@ public class TestManifestWriter extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final int FILE_SIZE_CHECK_ROWS_DIVISOR = 250; diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 0a14a89c01b5..755eb57de8e8 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -55,7 +55,7 @@ public class TestMetadataTableScans extends MetadataTableScanTestBase { private void preparePartitionedTable(boolean transactional) { preparePartitionedTableData(transactional); - if (formatVersion == 2) { + if (formatVersion >= 2) { if (transactional) { table .newRowDelta() @@ -485,7 +485,7 @@ public void testPartitionsTableScanNoFilter() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanNoFilter); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -510,7 +510,7 @@ public void testPartitionsTableScanWithProjection() { assertThat(scanWithProjection.schema().asStruct()).isEqualTo(expected); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanWithProjection); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -552,7 +552,7 @@ public void testPartitionsTableScanAndFilter() { TableScan scanAndEq = partitionsTable.newScan().filter(andEquals); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanAndEq); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(2); } else { assertThat(entries).hasSize(1); @@ -574,7 +574,7 @@ public void testPartitionsTableScanLtFilter() { TableScan scanLtAnd = partitionsTable.newScan().filter(ltAnd); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanLtAnd); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -598,7 +598,7 @@ public void testPartitionsTableScanOrFilter() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanOr); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -619,7 +619,7 @@ public void testPartitionsScanNotFilter() { TableScan scanNot = partitionsTable.newScan().filter(not); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanNot); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -639,7 +639,7 @@ public void testPartitionsTableScanInFilter() { TableScan scanSet = partitionsTable.newScan().filter(set); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanSet); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(4); } else { assertThat(entries).hasSize(2); @@ -659,7 +659,7 @@ public void testPartitionsTableScanNotNullFilter() { TableScan scanUnary = partitionsTable.newScan().filter(unary); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scanUnary); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -719,8 +719,7 @@ public void testFilesTableScanWithDroppedPartition() throws IOException { @TestTemplate public void testDeleteFilesTableSelection() throws IOException { - assumeThat(formatVersion).as("Only V2 Tables Support Deletes").isGreaterThanOrEqualTo(2); - + 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(); @@ -960,7 +959,7 @@ public void testPartitionSpecEvolutionAdditive() { TableScan scan = metadataTable.newScan().filter(filter); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // Four data files and delete files of old spec, one new data file of new spec assertThat(entries).hasSize(9); } else { @@ -975,7 +974,7 @@ public void testPartitionSpecEvolutionAdditive() { scan = metadataTable.newScan().filter(filter); entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // 1 original data file and delete file written by old spec, plus 1 new data file written by // new spec assertThat(entries).hasSize(3); @@ -1026,7 +1025,7 @@ public void testPartitionSpecEvolutionRemoval() { CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { // Four data and delete files of original spec, one data file written by new spec assertThat(entries).hasSize(9); } else { @@ -1187,7 +1186,7 @@ public void testPartitionsTableScanWithPlanExecutor() { })); CloseableIterable> entries = PartitionsTable.planEntries((StaticTableScan) scan); - if (formatVersion == 2) { + if (formatVersion >= 2) { assertThat(entries).hasSize(8); } else { assertThat(entries).hasSize(4); @@ -1366,7 +1365,7 @@ public void testAllManifestsTableSnapshotNot() { @TestTemplate public void testPositionDeletesWithFilter() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1429,7 +1428,7 @@ public void testPositionDeletesBaseTableFilterEntriesLevel() { } private void testPositionDeletesBaseTableFilter(boolean transactional) { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(transactional); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1490,9 +1489,9 @@ private void testPositionDeletesBaseTableFilter(boolean transactional) { @TestTemplate public void testPositionDeletesWithBaseTableFilterNot() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - - // use identity rather than bucket partition spec, + assumeThat(formatVersion) + .as("Position deletes are not supported by V1 Tables") + .isNotEqualTo(1); // 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(); @@ -1574,7 +1573,7 @@ public void testPositionDeletesWithBaseTableFilterNot() { @TestTemplate public void testPositionDeletesResiduals() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); preparePartitionedTable(); PositionDeletesTable positionDeletesTable = new PositionDeletesTable(table); @@ -1603,7 +1602,7 @@ public void testPositionDeletesResiduals() { @TestTemplate public void testPositionDeletesUnpartitioned() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); table.updateSpec().removeField(Expressions.bucket("data", BUCKETS_NUMBER)).commit(); assertThat(table.spec().fields()).as("Table should now be unpartitioned").hasSize(0); @@ -1694,8 +1693,7 @@ public void testPositionDeletesUnpartitioned() { @TestTemplate public void testPositionDeletesManyColumns() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); UpdateSchema updateSchema = table.updateSchema(); for (int i = 0; i <= 2000; i++) { updateSchema.addColumn(String.valueOf(i), Types.IntegerType.get()); diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java index a2e5386d29df..ac96642319a3 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScansWithPartitionEvolution.java @@ -163,8 +163,7 @@ public void testPartitionsTableScanWithAddPartitionOnNestedField() { @TestTemplate public void testPositionDeletesPartitionSpecRemoval() { - assumeThat(formatVersion).as("Position deletes supported only for v2 tables").isEqualTo(2); - + assumeThat(formatVersion).as("Position deletes are not supported by V1 Tables").isNotEqualTo(1); 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/TestMetrics.java b/core/src/test/java/org/apache/iceberg/TestMetrics.java index b95b92979f91..2c4849135f64 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetrics.java +++ b/core/src/test/java/org/apache/iceberg/TestMetrics.java @@ -69,7 +69,7 @@ public abstract class TestMetrics { @Parameters(name = "formatVersion = {0}") public static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TempDir public Path temp; diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java index 31ae459df506..00e961097c34 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsModes.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsModes.java @@ -47,7 +47,7 @@ public class TestMetricsModes { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TempDir private Path temp; diff --git a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java index 733bb0bb38fd..8c6f18619ac0 100644 --- a/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java +++ b/core/src/test/java/org/apache/iceberg/TestMicroBatchBuilder.java @@ -34,7 +34,7 @@ public class TestMicroBatchBuilder extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index ff54929504ee..a324b8af2e70 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -47,7 +47,7 @@ public class TestPartitionSpecInfo { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java index 3a6d2017eb82..b1b481dd5305 100644 --- a/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestReplaceTransaction.java @@ -50,7 +50,7 @@ public class TestReplaceTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java index 176f61079fc3..f1d23de32a42 100644 --- a/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java +++ b/core/src/test/java/org/apache/iceberg/TestRewriteManifests.java @@ -48,7 +48,7 @@ public class TestRewriteManifests extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestScanSummary.java b/core/src/test/java/org/apache/iceberg/TestScanSummary.java index b326274842a3..10c22b759988 100644 --- a/core/src/test/java/org/apache/iceberg/TestScanSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestScanSummary.java @@ -39,7 +39,7 @@ public class TestScanSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java index 0d5b1bc7066b..3fc691ce942a 100644 --- a/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java +++ b/core/src/test/java/org/apache/iceberg/TestScansAndSchemaEvolution.java @@ -55,7 +55,7 @@ public class TestScansAndSchemaEvolution { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java index 1bd1761ffc60..111693792f78 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaAndMappingUpdate.java @@ -43,7 +43,7 @@ public class TestSchemaAndMappingUpdate extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSchemaID.java b/core/src/test/java/org/apache/iceberg/TestSchemaID.java index 33ac84d20801..e3c58af1cbe3 100644 --- a/core/src/test/java/org/apache/iceberg/TestSchemaID.java +++ b/core/src/test/java/org/apache/iceberg/TestSchemaID.java @@ -36,7 +36,7 @@ public class TestSchemaID extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java index 3ebe047e90b7..9504f78f5345 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetPartitionStatistics.java @@ -29,7 +29,7 @@ public class TestSetPartitionStatistics extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java index 41941e3c6630..03c644117a95 100644 --- a/core/src/test/java/org/apache/iceberg/TestSetStatistics.java +++ b/core/src/test/java/org/apache/iceberg/TestSetStatistics.java @@ -31,7 +31,7 @@ public class TestSetStatistics extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshot.java b/core/src/test/java/org/apache/iceberg/TestSnapshot.java index 2ec6abd4e428..8a30036f3242 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshot.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshot.java @@ -33,7 +33,7 @@ public class TestSnapshot extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java index 89312201265d..96dde0f48eb7 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotLoading.java @@ -43,7 +43,7 @@ public class TestSnapshotLoading extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private Snapshot currentSnapshot; diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java index 88233dd99097..61dd082701c2 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotManager.java @@ -51,7 +51,7 @@ public class TestSnapshotManager extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java index 7ce59e9df1c9..6d339c0cbeaf 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSelection.java @@ -33,7 +33,7 @@ public class TestSnapshotSelection extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java index 053a9c374178..529e0cc614f6 100644 --- a/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotSummary.java @@ -32,7 +32,7 @@ public class TestSnapshotSummary extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestSortOrder.java b/core/src/test/java/org/apache/iceberg/TestSortOrder.java index a3ba69a808b3..ad773192b417 100644 --- a/core/src/test/java/org/apache/iceberg/TestSortOrder.java +++ b/core/src/test/java/org/apache/iceberg/TestSortOrder.java @@ -76,7 +76,7 @@ public class TestSortOrder { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Parameter private int formatVersion; diff --git a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java index 04bb2ae215d8..ea2142982382 100644 --- a/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java +++ b/core/src/test/java/org/apache/iceberg/TestSplitPlanning.java @@ -58,7 +58,7 @@ public class TestSplitPlanning extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @Override diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java index f47968434bfe..94a3d35b35a6 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadataSerialization.java @@ -35,7 +35,7 @@ public class TestTableMetadataSerialization extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java index a4e587068e74..482514c40093 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java @@ -36,7 +36,7 @@ public class TestTableUpdatePartitionSpec extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java index 08714dec01f1..656dd782cfe4 100644 --- a/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java +++ b/core/src/test/java/org/apache/iceberg/TestTimestampPartitions.java @@ -35,7 +35,7 @@ public class TestTimestampPartitions extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestTransaction.java b/core/src/test/java/org/apache/iceberg/TestTransaction.java index 393494da1283..8fed7134fae1 100644 --- a/core/src/test/java/org/apache/iceberg/TestTransaction.java +++ b/core/src/test/java/org/apache/iceberg/TestTransaction.java @@ -42,7 +42,7 @@ public class TestTransaction extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java index 33b003cfd8c5..97f695315e16 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdatePartitionSpec.java @@ -54,7 +54,7 @@ public class TestUpdatePartitionSpec extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java index 58b83f43f399..b1e8e57850e8 100644 --- a/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java +++ b/core/src/test/java/org/apache/iceberg/TestWapWorkflow.java @@ -35,7 +35,7 @@ public class TestWapWorkflow extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @BeforeEach diff --git a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java index a9a23d587ac9..77d16d3bc821 100644 --- a/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java +++ b/core/src/test/java/org/apache/iceberg/actions/TestSizeBasedRewriter.java @@ -41,7 +41,7 @@ public class TestSizeBasedRewriter extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java index ceffeb3749da..bb98925f504c 100644 --- a/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java +++ b/core/src/test/java/org/apache/iceberg/io/TestOutputFileFactory.java @@ -40,7 +40,7 @@ public class TestOutputFileFactory extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } private static final int PARTITION_ID = 1; diff --git a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java index b9ae9dc273f2..2161b40a60b5 100644 --- a/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java +++ b/core/src/test/java/org/apache/iceberg/mapping/TestMappingUpdates.java @@ -36,7 +36,7 @@ public class TestMappingUpdates extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1, 2); + return Arrays.asList(1, 2, 3); } @TestTemplate diff --git a/core/src/test/resources/TableMetadataUnsupportedVersion.json b/core/src/test/resources/TableMetadataUnsupportedVersion.json index 0633a71d24d5..c40a0c9cd5ae 100644 --- a/core/src/test/resources/TableMetadataUnsupportedVersion.json +++ b/core/src/test/resources/TableMetadataUnsupportedVersion.json @@ -1,5 +1,5 @@ { - "format-version": 3, + "format-version": 4, "table-uuid": "d20125c8-7284-442c-9aea-15fee620737c", "location": "s3://bucket/test/location", "last-updated-ms": 1602638573874, @@ -33,4 +33,4 @@ "properties": {}, "current-snapshot-id": -1, "snapshots": [] -} \ No newline at end of file +} From ba558783b0d347181e62a23af7770ad0e97a9423 Mon Sep 17 00:00:00 2001 From: Grant Nicholas <43971820+grantatspothero@users.noreply.github.com> Date: Thu, 1 Aug 2024 14:31:18 -0500 Subject: [PATCH 0540/1019] Core: Allow SnapshotProducer to skip uncommitted manifest cleanup after commit (#10523) --- .../java/org/apache/iceberg/FastAppend.java | 10 ++++ .../org/apache/iceberg/SnapshotProducer.java | 43 ++++++++-------- .../org/apache/iceberg/TestFastAppend.java | 50 +++++++++++++++++++ 3 files changed, 82 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 1439289130d7..4976a8081c44 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -198,6 +198,16 @@ protected void cleanUncommitted(Set committed) { } } + /** + * Cleanup after committing is disabled for FastAppend unless there are rewrittenAppendManifests + * because: 1.) Appended manifests are never rewritten 2.) Manifests which are written out as part + * of appendFile are already cleaned up between commit attempts in writeNewManifests + */ + @Override + protected boolean cleanupAfterCommit() { + return !rewrittenAppendManifests.isEmpty(); + } + private List writeNewManifests() throws IOException { if (hasNewFiles && newManifests != null) { newManifests.forEach(file -> deleteFile(file.path())); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 9f4bcbc6bba9..0a040fe34471 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -41,7 +41,7 @@ import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptingFileIO; @@ -368,8 +368,8 @@ protected TableMetadata refresh() { @Override @SuppressWarnings("checkstyle:CyclomaticComplexity") public void commit() { - // this is always set to the latest commit attempt's snapshot id. - AtomicLong newSnapshotId = new AtomicLong(-1L); + // this is always set to the latest commit attempt's snapshot + AtomicReference stagedSnapshot = new AtomicReference<>(); try (Timed ignore = commitMetrics().totalDuration().start()) { try { Tasks.foreach(ops) @@ -384,7 +384,7 @@ public void commit() { .run( taskOps -> { Snapshot newSnapshot = apply(); - newSnapshotId.set(newSnapshot.snapshotId()); + stagedSnapshot.set(newSnapshot); TableMetadata.Builder update = TableMetadata.buildFrom(base); if (base.snapshot(newSnapshot.snapshotId()) != null) { // this is a rollback operation @@ -422,26 +422,23 @@ public void commit() { throw e; } + // at this point, the commit must have succeeded so the stagedSnapshot is committed + Snapshot committedSnapshot = stagedSnapshot.get(); try { - LOG.info("Committed snapshot {} ({})", newSnapshotId.get(), getClass().getSimpleName()); - - // at this point, the commit must have succeeded. after a refresh, the snapshot is loaded by - // id in case another commit was added between this commit and the refresh. - Snapshot saved = ops.refresh().snapshot(newSnapshotId.get()); - if (saved != null) { - cleanUncommitted(Sets.newHashSet(saved.allManifests(ops.io()))); - // also clean up unused manifest lists created by multiple attempts - for (String manifestList : manifestLists) { - if (!saved.manifestListLocation().equals(manifestList)) { - deleteFile(manifestList); - } + LOG.info( + "Committed snapshot {} ({})", + committedSnapshot.snapshotId(), + getClass().getSimpleName()); + + if (cleanupAfterCommit()) { + cleanUncommitted(Sets.newHashSet(committedSnapshot.allManifests(ops.io()))); + } + // also clean up unused manifest lists created by multiple attempts + for (String manifestList : manifestLists) { + if (!committedSnapshot.manifestListLocation().equals(manifestList)) { + deleteFile(manifestList); } - } else { - // saved may not be present if the latest metadata couldn't be loaded due to eventual - // consistency problems in refresh. in that case, don't clean up. - LOG.warn("Failed to load committed snapshot, skipping manifest clean-up"); } - } catch (Throwable e) { LOG.warn( "Failed to load committed table metadata or during cleanup, skipping further cleanup", @@ -565,6 +562,10 @@ protected boolean canInheritSnapshotId() { return canInheritSnapshotId; } + protected boolean cleanupAfterCommit() { + return true; + } + private static ManifestFile addMetadata(TableOperations ops, ManifestFile manifest) { try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById())) { diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index b281536ab0fa..8125c528d9c3 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -324,6 +324,56 @@ public void testRecoveryWithoutManifestList() { assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); } + @TestTemplate + public void testWriteNewManifestsIdempotency() { + // inject 3 failures, the last try will succeed + TestTables.TestTableOperations ops = table.ops(); + ops.failCommits(3); + + AppendFiles append = table.newFastAppend().appendFile(FILE_B); + Snapshot pending = append.apply(); + ManifestFile newManifest = pending.allManifests(FILE_IO).get(0); + assertThat(new File(newManifest.path())).exists(); + + append.commit(); + + TableMetadata metadata = readMetadata(); + + // contains only a single manifest, does not duplicate manifests on retries + validateSnapshot(null, metadata.currentSnapshot(), FILE_B); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).contains(newManifest); + assertThat(listManifestFiles(tableDir)).containsExactly(new File(newManifest.path())); + } + + @TestTemplate + public void testWriteNewManifestsCleanup() { + // append file, stage changes with apply() but do not commit + AppendFiles append = table.newFastAppend().appendFile(FILE_A); + Snapshot pending = append.apply(); + ManifestFile oldManifest = pending.allManifests(FILE_IO).get(0); + assertThat(new File(oldManifest.path())).exists(); + + // append file, stage changes with apply() but do not commit + // validate writeNewManifests deleted the old staged manifest + append.appendFile(FILE_B); + Snapshot newPending = append.apply(); + List manifestFiles = newPending.allManifests(FILE_IO); + assertThat(manifestFiles).hasSize(1); + ManifestFile newManifest = manifestFiles.get(0); + assertThat(newManifest.path()).isNotEqualTo(oldManifest.path()); + + append.commit(); + TableMetadata metadata = readMetadata(); + + // contains only a single manifest, old staged manifest is deleted + validateSnapshot(null, metadata.currentSnapshot(), FILE_A, FILE_B); + assertThat(new File(oldManifest.path())).doesNotExist(); + assertThat(new File(newManifest.path())).exists(); + assertThat(metadata.currentSnapshot().allManifests(FILE_IO)).containsExactly(newManifest); + assertThat(listManifestFiles(tableDir)).containsExactly(new File(newManifest.path())); + } + @TestTemplate public void testAppendManifestWithSnapshotIdInheritance() throws IOException { table.updateProperties().set(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, "true").commit(); From a95514380e310ff99510fc5a5c387d5d623be978 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 1 Aug 2024 14:10:37 -0700 Subject: [PATCH 0541/1019] Flink: a few small fixes or tuning for range partitioner (#10823) --- .../shuffle/AggregatedStatisticsTracker.java | 23 +++-- .../sink/shuffle/CompletedStatistics.java | 8 ++ .../shuffle/DataStatisticsCoordinator.java | 45 ++++++---- .../flink/sink/shuffle/RangePartitioner.java | 8 +- .../sink/shuffle/SketchRangePartitioner.java | 19 +--- .../flink/sink/shuffle/SketchUtil.java | 17 ++++ .../flink/sink/shuffle/SortKeyUtil.java | 59 +++++++++++++ .../sink/shuffle/TestRangePartitioner.java | 65 ++++++++++++++ .../shuffle/TestSketchRangePartitioner.java | 88 +++++++++++++++++++ .../flink/sink/shuffle/TestSketchUtil.java | 64 +++++++++++++- .../flink/sink/shuffle/TestSortKeyUtil.java | 73 +++++++++++++++ 11 files changed, 420 insertions(+), 49 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } 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 c0e228965ddd..e4cba174f0f2 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 @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } 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 3b21fbae315a..4bfde7204acf 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 @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.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.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.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.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} From f14e58128cb38e46e3f756612dd7fd40459338cd Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 2 Aug 2024 09:23:23 +0200 Subject: [PATCH 0542/1019] Drop support for Java 8 (#10518) --- .github/workflows/delta-conversion-ci.yml | 4 +- .github/workflows/flink-ci.yml | 2 +- .github/workflows/hive-ci.yml | 4 +- .github/workflows/java-ci.yml | 6 +-- .github/workflows/publish-snapshot.yml | 2 +- .github/workflows/spark-ci.yml | 2 +- README.md | 2 +- build.gradle | 52 ++++++++--------------- deploy.gradle | 4 +- hive-runtime/build.gradle | 2 +- jmh.gradle | 4 +- site/docs/contribute.md | 2 +- tasks.gradle | 8 ++-- 13 files changed, 38 insertions(+), 56 deletions(-) diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index ac5314e8afb8..cd16847cf95a 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -71,7 +71,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -100,7 +100,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index d2e249c279f7..3df36e2be86a 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -73,7 +73,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] flink: ['1.17', '1.18', '1.19'] exclude: # Flink 1.17 does not support Java 17. diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index ee487807cff4..6ad9f58410d6 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -69,7 +69,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -98,7 +98,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index e1fd90116ef1..0d39ee8646ad 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -65,7 +65,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] env: SPARK_LOCAL_IP: localhost steps: @@ -94,7 +94,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 @@ -107,7 +107,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] steps: - uses: actions/checkout@v4 - uses: actions/setup-java@v4 diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 411c3a5636c9..6acee54bde3e 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -37,7 +37,7 @@ jobs: - uses: actions/setup-java@v4 with: distribution: zulu - java-version: 8 + java-version: 11 - run: | ./gradlew printVersion ./gradlew -DallModules publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 7a47beeed7a4..1cc0425b73a3 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -71,7 +71,7 @@ jobs: runs-on: ubuntu-22.04 strategy: matrix: - jvm: [8, 11, 17, 21] + jvm: [11, 17, 21] spark: ['3.3', '3.4', '3.5'] scala: ['2.12', '2.13'] exclude: diff --git a/README.md b/README.md index fe0d2b94c308..7d2056077804 100644 --- a/README.md +++ b/README.md @@ -51,7 +51,7 @@ Community discussions happen primarily on the [dev mailing list][dev-list] or on ### Building -Iceberg is built using Gradle with Java 8, 11, 17, or 21. +Iceberg is built using Gradle with Java 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/build.gradle b/build.gradle index a0b1a2e018bf..450a92f24b7c 100644 --- a/build.gradle +++ b/build.gradle @@ -56,10 +56,7 @@ try { project.logger.error(e.getMessage()) } -if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - project.ext.jdkVersion = '8' - project.ext.extraJvmArgs = [] -} else if (JavaVersion.current() == JavaVersion.VERSION_11) { +if (JavaVersion.current() == JavaVersion.VERSION_11) { project.ext.jdkVersion = '11' project.ext.extraJvmArgs = [] } else if (JavaVersion.current() == JavaVersion.VERSION_17 || JavaVersion.current() == JavaVersion.VERSION_21) { @@ -86,7 +83,7 @@ if (JavaVersion.current() == JavaVersion.VERSION_1_8) { "--add-opens", "java.base/sun.security.action=ALL-UNNAMED", "--add-opens", "java.base/sun.util.calendar=ALL-UNNAMED"] } else { - throw new GradleException("This build must be run with JDK 8 or 11 or 17 or 21 but was executed with JDK " + JavaVersion.current()) + throw new GradleException("This build must be run with JDK 11 or 17 or 21 but was executed with JDK " + JavaVersion.current()) } tasks.withType(AbstractArchiveTask).configureEach { @@ -887,23 +884,12 @@ project(':iceberg-pig') { } project(':iceberg-nessie') { - if (JavaVersion.current().isJava11Compatible()) { - test { - useJUnitPlatform() - } - compileTestJava { - sourceCompatibility = "11" - targetCompatibility = "11" - } - } else { - // Do not test Nessie against Java 8, because in-JVM testing requires Nessie server components, - // which require Java 11+. - test { - enabled = false - } - compileTestJava { - enabled = false - } + test { + useJUnitPlatform() + } + compileTestJava { + sourceCompatibility = "11" + targetCompatibility = "11" } dependencies { @@ -922,21 +908,19 @@ project(':iceberg-nessie') { // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages compileOnly libs.microprofile.openapi.api - if (JavaVersion.current().isJava11Compatible()) { - testImplementation libs.nessie.jaxrs.testextension - testImplementation libs.nessie.versioned.storage.inmemory.tests - testImplementation libs.nessie.versioned.storage.testextension - // Need to "pull in" el-api explicitly :( - testImplementation libs.jakarta.el.api + testImplementation libs.nessie.jaxrs.testextension + testImplementation libs.nessie.versioned.storage.inmemory.tests + testImplementation libs.nessie.versioned.storage.testextension + // Need to "pull in" el-api explicitly :( + testImplementation libs.jakarta.el.api - testImplementation libs.avro.avro + testImplementation libs.avro.avro - testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages - testCompileOnly libs.microprofile.openapi.api - } + // Only there to prevent "warning: unknown enum constant SchemaType.OBJECT" compile messages + testCompileOnly libs.microprofile.openapi.api } } diff --git a/deploy.gradle b/deploy.gradle index 3e2eda2a5a60..462448303253 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (project.hasProperty('release') && jdkVersion != '8') { - throw new GradleException("Releases must be built with Java 8") +if (project.hasProperty('release') && jdkVersion != '11') { + throw new GradleException("Releases must be built with Java 11") } subprojects { diff --git a/hive-runtime/build.gradle b/hive-runtime/build.gradle index b2051d529512..6f891be8087e 100644 --- a/hive-runtime/build.gradle +++ b/hive-runtime/build.gradle @@ -38,7 +38,7 @@ project(':iceberg-hive-runtime') { dependencies { implementation project(':iceberg-mr') - if (jdkVersion == '8' && hiveVersions.contains("3")) { + if (hiveVersions.contains("3")) { implementation project(':iceberg-hive3') } implementation(project(':iceberg-nessie')) { diff --git a/jmh.gradle b/jmh.gradle index 80f5f8d0ea63..5e5e0151219f 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -17,8 +17,8 @@ * under the License. */ -if (jdkVersion != '8' && jdkVersion != '11' && jdkVersion != '17' && jdkVersion != '21') { - throw new GradleException("The JMH benchmarks must be run with JDK 8 or JDK 11 or JDK 17 or JDK 21") +if (jdkVersion != '11' && jdkVersion != '17' && jdkVersion != '21') { + throw new GradleException("The JMH benchmarks must be run with JDK 11 or JDK 17 or JDK 21") } def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 88a14e7153b4..60bc89f9537f 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -84,7 +84,7 @@ settle disagreements or to force a decision. ## Building the Project Locally -Iceberg is built using Gradle with Java 8, 11, 17, or 21. +Iceberg is built using Gradle with Java 11, 17, or 21. * To invoke a build and run tests: `./gradlew build` * To skip tests: `./gradlew build -x test -x integrationTest` diff --git a/tasks.gradle b/tasks.gradle index 5fc24d41ae62..5515d7b75052 100644 --- a/tasks.gradle +++ b/tasks.gradle @@ -32,11 +32,9 @@ task aggregateJavadoc(type: Javadoc) { doLast { // Fix bug with search - if (JavaVersion.current() >= JavaVersion.VERSION_11) { - // Append the fix to the file - def searchScript = new File("site/docs/javadoc/${getJavadocVersion()}" + '/search.js') - searchScript.append JAVADOC_FIX_SEARCH_STR - } + // Append the fix to the file + def searchScript = new File("site/docs/javadoc/${getJavadocVersion()}" + '/search.js') + searchScript.append JAVADOC_FIX_SEARCH_STR } } From a0229c0cdc79f79cd084bc94d383596b5bb6e0c6 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 2 Aug 2024 14:02:56 +0200 Subject: [PATCH 0543/1019] Build: Bump com.adobe.testing:s3mock-junit5 from 2.11.0 to 2.17.0 (#10851) --- aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIO.java | 3 ++- .../test/java/org/apache/iceberg/aws/s3/TestS3InputStream.java | 3 ++- .../java/org/apache/iceberg/aws/s3/TestS3OutputStream.java | 3 ++- gradle/libs.versions.toml | 2 +- 4 files changed, 7 insertions(+), 4 deletions(-) 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 135eb76772cc..38489e367434 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 @@ -81,6 +81,7 @@ import software.amazon.awssdk.regions.Region; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; +import software.amazon.awssdk.services.s3.model.BucketAlreadyOwnedByYouException; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; import software.amazon.awssdk.services.s3.model.DeleteObjectsResponse; @@ -447,7 +448,7 @@ private void createRandomObjects(String prefix, int count) { private void createBucket(String bucketName) { try { s3.get().createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); - } catch (BucketAlreadyExistsException e) { + } catch (BucketAlreadyExistsException | BucketAlreadyOwnedByYouException e) { // do nothing } } 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 95f4d09a7e0b..ed71e259a26c 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 @@ -35,6 +35,7 @@ import software.amazon.awssdk.core.sync.RequestBody; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; +import software.amazon.awssdk.services.s3.model.BucketAlreadyOwnedByYouException; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; @@ -195,7 +196,7 @@ private void writeS3Data(S3URI uri, byte[] data) throws IOException { private void createBucket(String bucketName) { try { s3.createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); - } catch (BucketAlreadyExistsException e) { + } catch (BucketAlreadyExistsException | BucketAlreadyOwnedByYouException e) { // don't do anything } } 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 6fbe59e47f7f..88488bf4c313 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 @@ -62,6 +62,7 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.AbortMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.BucketAlreadyExistsException; +import software.amazon.awssdk.services.s3.model.BucketAlreadyOwnedByYouException; import software.amazon.awssdk.services.s3.model.CompleteMultipartUploadRequest; import software.amazon.awssdk.services.s3.model.CreateBucketRequest; import software.amazon.awssdk.services.s3.model.GetObjectRequest; @@ -339,7 +340,7 @@ private S3URI randomURI() { private void createBucket(String bucketName) { try { s3.createBucket(CreateBucketRequest.builder().bucket(bucketName).build()); - } catch (BucketAlreadyExistsException e) { + } catch (BucketAlreadyExistsException | BucketAlreadyOwnedByYouException e) { // do nothing } } diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 055475809296..5ede5abf23cc 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -74,7 +74,7 @@ orc = "1.9.4" parquet = "1.13.1" pig = "0.17.0" roaringbitmap = "1.2.1" -s3mock-junit5 = "2.11.0" +s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" slf4j = "1.7.36" snowflake-jdbc = "3.18.0" From a5a0873dafe310f17402bafcf09b630f7d4e9026 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 2 Aug 2024 14:08:43 +0200 Subject: [PATCH 0544/1019] Core: Upgrade Jetty and Servlet API (#10850) This is the latest Jetty version that runs with JDK11 --- build.gradle | 1 + .../java/org/apache/iceberg/rest/RESTCatalogServlet.java | 6 +++--- gradle/libs.versions.toml | 4 +++- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/build.gradle b/build.gradle index 450a92f24b7c..36928c869502 100644 --- a/build.gradle +++ b/build.gradle @@ -361,6 +361,7 @@ project(':iceberg-core') { } testImplementation libs.jetty.servlet + testImplementation libs.jakarta.servlet testImplementation libs.jetty.server testImplementation libs.mockserver.netty testImplementation libs.mockserver.client.java diff --git a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java index 954f5130eca2..f456bb4d354d 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogServlet.java @@ -20,6 +20,9 @@ import static java.lang.String.format; +import jakarta.servlet.http.HttpServlet; +import jakarta.servlet.http.HttpServletRequest; +import jakarta.servlet.http.HttpServletResponse; import java.io.IOException; import java.io.InputStreamReader; import java.io.Reader; @@ -30,9 +33,6 @@ import java.util.function.Consumer; import java.util.function.Function; import java.util.stream.Collectors; -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/gradle/libs.versions.toml b/gradle/libs.versions.toml index 5ede5abf23cc..de109572dec4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -57,9 +57,10 @@ jackson213 = { strictly = "2.13.4"} jackson214 = { strictly = "2.14.2"} jackson215 = { strictly = "2.15.2"} jakarta-el-api = "3.0.3" +jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" -jetty = "9.4.55.v20240627" +jetty = "11.0.22" junit = "5.10.1" kafka = "3.7.1" kryo-shaded = "4.0.3" @@ -196,6 +197,7 @@ flink119-test-utils = { module = "org.apache.flink:flink-test-utils", version.re flink119-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink119" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jakarta-el-api = { module = "jakarta.el:jakarta.el-api", version.ref = "jakarta-el-api" } +jakarta-servlet = {module = "jakarta.servlet:jakarta.servlet-api", version.ref = "jakarta-servlet-api"} jetty-server = { module = "org.eclipse.jetty:jetty-server", version.ref = "jetty" } jetty-servlet = { module = "org.eclipse.jetty:jetty-servlet", version.ref = "jetty" } junit-jupiter = { module = "org.junit.jupiter:junit-jupiter", version.ref = "junit" } From 2e6d2f1b0b0d35fd6e26b6f0745794144062c1fd Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Fri, 2 Aug 2024 15:14:40 +0200 Subject: [PATCH 0545/1019] Build: Configure options.release = 11 / remove com.palantir.baseline-release-compatibility plugin (#10849) --- baseline.gradle | 9 +++++++-- build.gradle | 14 ++------------ 2 files changed, 9 insertions(+), 14 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index 62ade9a632f4..f97b872e93e7 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -42,9 +42,14 @@ subprojects { apply plugin: 'com.palantir.baseline-error-prone' } apply plugin: 'com.palantir.baseline-class-uniqueness' - apply plugin: 'com.palantir.baseline-reproducibility' + // What 'com.palantir.baseline-reproducibility' used to do, except the check for the + // `sourceCompatibility` Java compile option, which conflicts with the `release` compile option. + tasks.withType(AbstractArchiveTask.class).configureEach(t -> { + t.setPreserveFileTimestamps(false); + t.setReproducibleFileOrder(true); + t.setDuplicatesStrategy(DuplicatesStrategy.WARN); + }); apply plugin: 'com.palantir.baseline-exact-dependencies' - apply plugin: 'com.palantir.baseline-release-compatibility' // We need to update Google Java Format to 1.17.0+ to run spotless on JDK 8, but that requires dropping support for JDK 8. if (JavaVersion.current() == JavaVersion.VERSION_21) { task spotlessApply { diff --git a/build.gradle b/build.gradle index 36928c869502..a1aef9b37fd1 100644 --- a/build.gradle +++ b/build.gradle @@ -191,21 +191,15 @@ subprojects { testArtifacts } - compileJava { - options.encoding = "UTF-8" - } - - compileTestJava { + tasks.withType(JavaCompile.class).configureEach { options.encoding = "UTF-8" + options.release = 11 } javadoc { options.encoding = 'UTF-8' } - sourceCompatibility = '1.8' - targetCompatibility = '1.8' - dependencies { implementation libs.slf4j.api @@ -888,10 +882,6 @@ project(':iceberg-nessie') { test { useJUnitPlatform() } - compileTestJava { - sourceCompatibility = "11" - targetCompatibility = "11" - } dependencies { api project(':iceberg-api') From 5413c1b70c4f2f7e4ce182c3920b8aae08c83525 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 2 Aug 2024 15:15:06 +0200 Subject: [PATCH 0546/1019] Build: Bump kafka from 3.7.1 to 3.8.0 (#10797) Bumps `kafka` from 3.7.1 to 3.8.0. Updates `org.apache.kafka:kafka-clients` from 3.7.1 to 3.8.0 Updates `org.apache.kafka:connect-api` from 3.7.1 to 3.8.0 Updates `org.apache.kafka:connect-json` from 3.7.1 to 3.8.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 de109572dec4..02702955a5fe 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -62,7 +62,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.22" junit = "5.10.1" -kafka = "3.7.1" +kafka = "3.8.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" From 0bb850d677a22d59e3ef9ebf935499e8ecfb1c5b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 2 Aug 2024 15:44:46 +0200 Subject: [PATCH 0547/1019] Build: Update baseline gradle plugin to 5.58.0 (#10788) --- baseline.gradle | 10 ++++++++++ build.gradle | 7 +------ .../main/java/org/apache/iceberg/StaticDataTask.java | 6 +++++- .../iceberg/flink/sink/shuffle/KeyAssignment.java | 6 +++++- .../iceberg/flink/sink/shuffle/MapAssignment.java | 2 ++ .../iceberg/flink/sink/shuffle/KeyAssignment.java | 6 +++++- .../iceberg/flink/sink/shuffle/MapAssignment.java | 2 ++ .../iceberg/flink/sink/shuffle/KeyAssignment.java | 6 +++++- .../iceberg/flink/sink/shuffle/MapAssignment.java | 2 ++ 9 files changed, 37 insertions(+), 10 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index f97b872e93e7..e3fe602e91c9 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -91,11 +91,17 @@ subprojects { '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', + // TODO (https://github.com/apache/iceberg/issues/10853) this is a recently added check. Figure out whether we adjust the code or suppress for good + '-Xep:DangerousJavaDeserialization:WARN', '-Xep:DangerousThreadPoolExecutorUsage:OFF', // subclasses are not equal '-Xep:EqualsGetClass:OFF', // specific to Palantir '-Xep:FinalClass:OFF', + // TODO (https://github.com/apache/iceberg/issues/10854) this is a recently added check. Figure out whether we adjust the code or suppress for good + '-Xep:FormatStringAnnotation:WARN', + // TODO (https://github.com/apache/iceberg/issues/10855) this is a recently added check. Figure out whether we adjust the code or suppress for good + '-Xep:ImmutablesReferenceEquality:WARN', '-Xep:IntLongMath:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', @@ -108,6 +114,8 @@ subprojects { '-Xep:MissingSummary:ERROR', // Enforce hashCode over hash '-Xep:ObjectsHashCodeUnnecessaryVarargs:ERROR', + // Triggers false-positives whenever relocated @VisibleForTesting is used + '-Xep:PreferCommonAnnotations:OFF', // specific to Palantir '-Xep:PreferSafeLoggableExceptions:OFF', '-Xep:PreferSafeLogger:OFF', @@ -125,6 +133,8 @@ subprojects { '-Xep:StringSplitter:ERROR', '-Xep:TypeParameterShadowing:OFF', '-Xep:TypeParameterUnusedInFormals:OFF', + // Palantir's UnnecessarilyQualified may throw during analysis + '-Xep:UnnecessarilyQualified:OFF', ) } } diff --git a/build.gradle b/build.gradle index a1aef9b37fd1..60fb5e7830d3 100644 --- a/build.gradle +++ b/build.gradle @@ -27,12 +27,7 @@ buildscript { } dependencies { classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' - classpath 'com.palantir.baseline:gradle-baseline-java:4.42.0' - // com.palantir.baseline:gradle-baseline-java:4.42.0 (the last version supporting Java 8) pulls - // in an old version of the errorprone, which doesn't work w/ Gradle 8, so bump errorpone as - // well. - classpath "net.ltgt.gradle:gradle-errorprone-plugin:3.1.0" - + classpath 'com.palantir.baseline:gradle-baseline-java:5.58.0' classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' diff --git a/core/src/main/java/org/apache/iceberg/StaticDataTask.java b/core/src/main/java/org/apache/iceberg/StaticDataTask.java index f25ebd49c9d8..1a396f0bfc7e 100644 --- a/core/src/main/java/org/apache/iceberg/StaticDataTask.java +++ b/core/src/main/java/org/apache/iceberg/StaticDataTask.java @@ -127,7 +127,11 @@ DataFile metadataFile() { return metadataFile; } - /** @return the table rows before projection */ + /** + * Returns the table rows before projection. + * + * @return the table rows before projection + */ StructLike[] tableRows() { return rows; } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java index a164d83ac3b0..781bcc646023 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -98,7 +98,11 @@ long[] subtaskWeightsExcludingCloseCost() { return subtaskWeightsExcludingCloseCost; } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.size() == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java index 0abb030c2279..9d8167460a1b 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -93,6 +93,8 @@ Map keyAssignments() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java index a164d83ac3b0..781bcc646023 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -98,7 +98,11 @@ long[] subtaskWeightsExcludingCloseCost() { return subtaskWeightsExcludingCloseCost; } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.size() == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java index 0abb030c2279..9d8167460a1b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -93,6 +93,8 @@ Map keyAssignments() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java index a164d83ac3b0..781bcc646023 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -98,7 +98,11 @@ long[] subtaskWeightsExcludingCloseCost() { return subtaskWeightsExcludingCloseCost; } - /** @return subtask id */ + /** + * Select a subtask for the key. + * + * @return subtask id + */ int select() { if (assignedSubtasks.size() == 1) { // only choice. no need to run random number generator. diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java index 0abb030c2279..9d8167460a1b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -93,6 +93,8 @@ Map keyAssignments() { } /** + * Returns assignment summary for every subtask. + * * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned * to the subtask, number of keys assigned to the subtask) */ From 05b1ba5a33f63214302febf74593a45f9658d6ab Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 2 Aug 2024 08:39:10 -0700 Subject: [PATCH 0548/1019] Flink: refactor sink tests to reduce the number of combinations with parameterized tests (#10777) --- .../apache/iceberg/flink/CatalogTestBase.java | 22 -- .../org/apache/iceberg/flink/SqlBase.java | 110 +++++++ .../org/apache/iceberg/flink/TestBase.java | 4 +- .../iceberg/flink/TestFlinkTableSink.java | 114 -------- .../flink/TestFlinkTableSinkExtended.java | 244 ++++++++++++++++ .../iceberg/flink/TestIcebergConnector.java | 4 - .../flink/sink/TestFlinkIcebergSink.java | 270 +----------------- .../flink/sink/TestFlinkIcebergSinkBase.java | 51 +++- .../TestFlinkIcebergSinkDistributionMode.java | 180 ++++++++++++ .../sink/TestFlinkIcebergSinkExtended.java | 208 ++++++++++++++ 10 files changed, 798 insertions(+), 409 deletions(-) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java index 91ed3c4adea3..062ff68d5d85 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -118,26 +118,4 @@ protected String getFullQualifiedTableName(String tableName) { static String getURI(HiveConf conf) { return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.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.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +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.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } + + protected static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index 633690044692..401960c3591b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -27,7 +27,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -41,7 +40,7 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -public abstract class TestBase extends TestBaseUtils { +public abstract class TestBase extends SqlBase { @RegisterExtension public static MiniClusterExtension miniClusterExtension = @@ -72,6 +71,7 @@ public static void stopMetastore() throws Exception { TestBase.catalog = null; } + @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { synchronized (this) { 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 a0341e6834d4..2978a92945a2 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 @@ -18,36 +18,21 @@ */ package org.apache.iceberg.flink; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.source.BoundedTableFactory; -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.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -169,39 +154,6 @@ public void testOverwriteTable() throws Exception { icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); } - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - 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 /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, 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)); - } - @TestTemplate public void testReplacePartitions() throws Exception { assumeThat(isStreamingJob) @@ -289,70 +241,4 @@ public void testInsertIntoPartition() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); } } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } } 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 new file mode 100644 index 000000000000..482cfd110bde --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,244 @@ +/* + * 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; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +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.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.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List parameters() { + return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + } + + protected synchronized TableEnvironment getTableEnv() { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + return tEnv; + } + + @BeforeEach + public void before() { + String warehouseLocation = "file:" + warehouseRoot.getPath(); + this.catalog = new HadoopCatalog(new Configuration(), warehouseLocation); + Map config = Maps.newHashMap(); + config.put("type", "iceberg"); + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HADOOP); + config.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + sql("CREATE CATALOG %s WITH %s", CATALOG, toWithClause(config)); + + sql("CREATE DATABASE %s", FLINK_DATABASE); + sql("USE CATALOG %s", CATALOG); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE, FileFormat.PARQUET.name()); + } + + @AfterEach + public void clean() throws Exception { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, TABLE); + dropDatabase(FLINK_DATABASE, true); + BoundedTableFactory.clearDataSets(); + + dropCatalog(CATALOG, true); + catalog.close(); + } + + @TestTemplate + public void testWriteParallelism() { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + 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 /*+ 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)); + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index fdb0e0cf19df..47f5485df879 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -319,10 +319,6 @@ private String databaseName() { return properties.getOrDefault("catalog-database", "default_database"); } - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - private String createWarehouse() { try { return String.format( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 61ab087f2ca3..b778037c559c 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,20 +18,11 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - import java.io.IOException; -import java.util.Collections; import java.util.List; -import java.util.Map; -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.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -39,37 +30,19 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; 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.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - @Parameter(index = 0) private FileFormat format; @@ -99,7 +72,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { - table = + this.table = CATALOG_EXTENSION .catalog() .createTable( @@ -110,14 +83,14 @@ public void before() throws IOException { : PartitionSpec.unpartitioned(), ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = + this.env = StreamExecutionEnvironment.getExecutionEnvironment( MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = CATALOG_EXTENSION.tableLoader(); + this.tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -140,246 +113,13 @@ public void testWriteRowData() throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); } - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - @TestTemplate public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); + testWriteRow(parallelism, null, DistributionMode.NONE); } @TestTemplate public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @TestTemplate - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); - } - } - } - - @TestTemplate - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @TestTemplate - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); - rightTable.refresh(); - assertThat(rightTable.currentSnapshot().summary()) - .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) - .containsEntry("direction", "rightTable"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @TestTemplate - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index b38aa6b50ce6..9ce36cc1e8d0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -18,29 +18,52 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; 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.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkIcebergSinkBase { - protected Table table; - protected StreamExecutionEnvironment env; + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + protected static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); protected static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List createRows(String prefix) { protected List convertToRowData(List rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } } 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 new file mode 100644 index 000000000000..75e397d3f203 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,180 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * This tests the distribution mode of Flink sink. Extract them separately since it is unnecessary + * to test different file formats (Avro, Orc, Parquet) like in {@link TestFlinkIcebergSink}. + * Removing the file format dimension reduces the number of combinations from 12 to 4, which helps + * reduce test run time. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private final FileFormat format = FileFormat.PARQUET; + + @Parameter(index = 0) + private int parallelism; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "parallelism = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {1, true}, + {1, false}, + {2, true}, + {2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).isGreaterThan(3); + } + } + } + + @TestTemplate + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testPartitionWriteMode() throws Exception { + testWriteRow(parallelism, null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +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.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); + rightTable.refresh(); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} From fa792b2611e93edcc7c35c0e98236db202dbe9c1 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 2 Aug 2024 08:39:47 -0700 Subject: [PATCH 0549/1019] Flink: backport PR #10823 for range partitioner fixup (#10847) --- .../shuffle/AggregatedStatisticsTracker.java | 23 +++-- .../sink/shuffle/CompletedStatistics.java | 8 ++ .../shuffle/DataStatisticsCoordinator.java | 45 ++++++---- .../flink/sink/shuffle/RangePartitioner.java | 8 +- .../sink/shuffle/SketchRangePartitioner.java | 19 +--- .../flink/sink/shuffle/SketchUtil.java | 17 ++++ .../flink/sink/shuffle/SortKeyUtil.java | 59 +++++++++++++ .../sink/shuffle/TestRangePartitioner.java | 65 ++++++++++++++ .../shuffle/TestSketchRangePartitioner.java | 88 +++++++++++++++++++ .../flink/sink/shuffle/TestSketchUtil.java | 64 +++++++++++++- .../flink/sink/shuffle/TestSortKeyUtil.java | 73 +++++++++++++++ .../shuffle/AggregatedStatisticsTracker.java | 23 +++-- .../sink/shuffle/CompletedStatistics.java | 8 ++ .../shuffle/DataStatisticsCoordinator.java | 45 ++++++---- .../flink/sink/shuffle/RangePartitioner.java | 8 +- .../sink/shuffle/SketchRangePartitioner.java | 19 +--- .../flink/sink/shuffle/SketchUtil.java | 17 ++++ .../flink/sink/shuffle/SortKeyUtil.java | 59 +++++++++++++ .../sink/shuffle/TestRangePartitioner.java | 65 ++++++++++++++ .../shuffle/TestSketchRangePartitioner.java | 88 +++++++++++++++++++ .../flink/sink/shuffle/TestSketchUtil.java | 64 +++++++++++++- .../flink/sink/shuffle/TestSortKeyUtil.java | 73 +++++++++++++++ 22 files changed, 840 insertions(+), 98 deletions(-) create mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java create mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java index c0e228965ddd..e4cba174f0f2 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java index 3b21fbae315a..4bfde7204acf 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.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.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.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.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 338523b7b074..5525f02c873e 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -223,7 +223,9 @@ private void merge(DataStatistics taskStatistics) { convertCoordinatorToSketch(); } - sketchStatistics.update(taskSketch); + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } } } @@ -242,13 +244,18 @@ private CompletedStatistics completedStatistics(long checkpointId) { return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); } else { ReservoirItemsSketch sketch = sketchStatistics.getResult(); - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } } } } 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 c0e228965ddd..e4cba174f0f2 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 @@ -100,4 +100,12 @@ Map keyFrequency() { SortKey[] keySamples() { return keySamples; } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } } 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 3b21fbae315a..4bfde7204acf 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 @@ -38,11 +38,11 @@ import org.apache.flink.util.function.ThrowingRunnable; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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.jetbrains.annotations.NotNull; import org.jetbrains.annotations.Nullable; import org.slf4j.Logger; @@ -91,7 +91,7 @@ class DataStatisticsCoordinator implements OperatorCoordinator { this.context = context; this.schema = schema; this.sortOrder = sortOrder; - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.downstreamParallelism = downstreamParallelism; this.statisticsType = statisticsType; this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; @@ -202,17 +202,23 @@ private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); if (maybeCompletedStatistics != null) { - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } } } @@ -324,9 +330,14 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r "Snapshotting data statistics coordinator {} for checkpoint {}", operatorName, checkpointId); - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } }, String.format("taking checkpoint %d", checkpointId)); } @@ -338,7 +349,7 @@ public void notifyCheckpointComplete(long checkpointId) {} public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { Preconditions.checkState( !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null) { + if (checkpointData == null || checkpointData.length == 0) { LOG.info( "Data statistic coordinator {} has nothing to restore from checkpoint {}", operatorName, diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java index 83a9461233d2..6608b938f5a8 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java @@ -23,12 +23,13 @@ import org.apache.flink.annotation.Internal; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -/** The wrapper class */ +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ @Internal public class RangePartitioner implements Partitioner { private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); @@ -94,9 +95,8 @@ static int adjustPartitionWithRescale( if (numPartitionsStatsCalculation <= numPartitions) { // no rescale or scale-up case. // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. - // when rescale is detected, operator requests new statistics from coordinator upon - // initialization. + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. return partition; } else { // scale-down case. diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java index af78271ea5dc..dddb0d8722c0 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -18,17 +18,16 @@ */ package org.apache.iceberg.flink.sink.shuffle; -import java.util.Arrays; import java.util.Comparator; import org.apache.flink.api.common.functions.Partitioner; import org.apache.flink.table.data.RowData; import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; import org.apache.iceberg.StructLike; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; class SketchRangePartitioner implements Partitioner { private final SortKey sortKey; @@ -38,7 +37,7 @@ class SketchRangePartitioner implements Partitioner { SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { this.sortKey = new SortKey(schema, sortOrder); - this.comparator = SortOrderComparators.forSchema(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); this.rangeBounds = rangeBounds; this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); } @@ -47,18 +46,6 @@ class SketchRangePartitioner implements Partitioner { public int partition(RowData row, int numPartitions) { // reuse the sortKey and rowDataWrapper sortKey.wrap(rowDataWrapper.wrap(row)); - int partition = Arrays.binarySearch(rangeBounds, sortKey, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java index a58310611e8d..871ef9ef1149 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -139,4 +139,21 @@ static void convertMapToSketch( } }); } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.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.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.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.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java index 31dae5c76aeb..16202c075ea0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -19,10 +19,13 @@ package org.apache.iceberg.flink.sink.shuffle; import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; import static org.assertj.core.api.Assertions.assertThat; import org.apache.iceberg.SortKey; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestSketchUtil { @Test @@ -55,7 +58,7 @@ public void testRangeBoundsOneChannel() { assertThat( SketchUtil.rangeBounds( 1, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -72,7 +75,7 @@ public void testRangeBoundsDivisible() { assertThat( SketchUtil.rangeBounds( 3, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -90,7 +93,7 @@ public void testRangeBoundsNonDivisible() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -113,7 +116,7 @@ public void testRangeBoundsSkipDuplicates() { assertThat( SketchUtil.rangeBounds( 4, - Fixtures.SORT_ORDER_COMPARTOR, + SORT_ORDER_COMPARTOR, new SortKey[] { CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), @@ -130,4 +133,57 @@ public void testRangeBoundsSkipDuplicates() { // skipped duplicate c's .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} From fd5c8989908cdbb77b284ea31c5cbc554b653cfe Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 2 Aug 2024 20:44:48 +0200 Subject: [PATCH 0550/1019] Core: Remove reflection from TestParallelIterable (#10857) This is a unit test, so can leverage package-private access. --- .../apache/iceberg/util/ParallelIterable.java | 9 +++- .../iceberg/util/TestParallelIterable.java | 41 ++++++++----------- 2 files changed, 25 insertions(+), 25 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 6486bd7fd483..16fa6f3d8537 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -35,6 +35,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.annotations.VisibleForTesting; 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.io.Closer; @@ -77,7 +78,8 @@ public CloseableIterator iterator() { return iter; } - private static class ParallelIterator implements CloseableIterator { + @VisibleForTesting + static class ParallelIterator implements CloseableIterator { private final Iterator> tasks; private final Deque> yieldedTasks = new ArrayDeque<>(); private final ExecutorService workerPool; @@ -229,6 +231,11 @@ public synchronized T next() { } return queue.poll(); } + + @VisibleForTesting + int queueSize() { + return queue.size(); + } } private static class Task implements Supplier>>, Closeable { 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 4910732f6e35..c259bbd0a7e2 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -21,12 +21,9 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.IOException; -import java.lang.reflect.Field; import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Queue; -import java.util.concurrent.ConcurrentLinkedQueue; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -40,6 +37,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.Multiset; +import org.apache.iceberg.util.ParallelIterable.ParallelIterator; import org.awaitility.Awaitility; import org.junit.jupiter.api.Test; @@ -64,20 +62,17 @@ public CloseableIterator iterator() { }); ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); - CloseableIterator iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + 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, queue)); + .untilAsserted(() -> queueHasElements(iterator)); iterator.close(); Awaitility.await("Queue is cleared") .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertThat(queue).isEmpty()); + .untilAsserted(() -> assertThat(iterator.queueSize()).isEqualTo(0)); } @Test @@ -124,20 +119,21 @@ public CloseableIterator iterator() { }); ParallelIterable parallelIterable = new ParallelIterable<>(transform, executor); - CloseableIterator iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + 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, queue)); + .untilAsserted(() -> queueHasElements(iterator)); iterator.close(); Awaitility.await("Queue is cleared") .atMost(5, TimeUnit.SECONDS) - .untilAsserted(() -> assertThat(queue).as("Queue is not empty after cleaning").isEmpty()); + .untilAsserted( + () -> + assertThat(iterator.queueSize()) + .as("Queue is not empty after cleaning") + .isEqualTo(0)); } @Test @@ -159,17 +155,14 @@ public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchF ExecutorService executor = Executors.newCachedThreadPool(); ParallelIterable parallelIterable = new ParallelIterable<>(iterables, executor, maxQueueSize); - CloseableIterator iterator = parallelIterable.iterator(); - Field queueField = iterator.getClass().getDeclaredField("queue"); - queueField.setAccessible(true); - ConcurrentLinkedQueue queue = (ConcurrentLinkedQueue) queueField.get(iterator); + ParallelIterator iterator = (ParallelIterator) parallelIterable.iterator(); Multiset actualValues = HashMultiset.create(); while (iterator.hasNext()) { - assertThat(queue) - .as("iterator internal queue") - .hasSizeLessThanOrEqualTo(maxQueueSize + iterables.size()); + assertThat(iterator.queueSize()) + .as("iterator internal queue size") + .isLessThanOrEqualTo(maxQueueSize + iterables.size()); actualValues.add(iterator.next()); } @@ -181,9 +174,9 @@ public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchF executor.shutdownNow(); } - private void queueHasElements(CloseableIterator iterator, Queue queue) { + private void queueHasElements(ParallelIterator iterator) { assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); - assertThat(queue).isNotEmpty(); + assertThat(iterator.queueSize()).as("queue size").isGreaterThan(0); } } From a3eb1b2f38fbd2dd62edad02eec509af03158e5c Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sun, 4 Aug 2024 14:32:52 -0700 Subject: [PATCH 0551/1019] Spec: Deprecate the file system table scheme (#10833) --- format/spec.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index 5a90f6fd978d..daef7538e730 100644 --- a/format/spec.md +++ b/format/spec.md @@ -779,7 +779,9 @@ When two commits happen at the same time and are based on the same version, only #### File System Tables -An atomic swap can be implemented using atomic rename in file systems that support it, like HDFS or most local file systems [1]. +_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._ + +An atomic swap can be implemented using atomic rename in file systems that support it, like HDFS [1]. Each version of table metadata is stored in a metadata folder under the table’s base location using a file naming scheme that includes a version number, `V`: `v.metadata.json`. To commit a new metadata version, `V+1`, the writer performs the following steps: @@ -1393,4 +1395,4 @@ This section covers topics not required by the specification but recommendations Iceberg supports two types of histories for tables. A history of previous "current snapshots" stored in ["snapshot-log" table metadata](#table-metadata-fields) and [parent-child lineage stored in "snapshots"](#table-metadata-fields). These two histories might indicate different snapshot IDs for a specific timestamp. The discrepancies can be caused by a variety of table operations (e.g. updating the `current-snapshot-id` can be used to set the snapshot of a table to any arbitrary snapshot, which might have a lineage derived from a table branch or no lineage at all). -When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. \ No newline at end of file +When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. From 2a6f61b7923782f59082b10ba6281a78cd328bdd Mon Sep 17 00:00:00 2001 From: Shani Elharrar Date: Mon, 5 Aug 2024 05:45:46 +0300 Subject: [PATCH 0552/1019] Core, API: UpdatePartitionSpec: Added ability to create a new Partition Spec but not set it as the Default --- .../apache/iceberg/UpdatePartitionSpec.java | 11 +++++++++++ .../iceberg/BaseUpdatePartitionSpec.java | 16 +++++++++++++++- .../org/apache/iceberg/TableMetadata.java | 4 ++++ .../iceberg/TestTableUpdatePartitionSpec.java | 19 +++++++++++++++++++ 4 files changed, 49 insertions(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java index f48d590af1ce..eeb596d42d5c 100644 --- a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java @@ -122,4 +122,15 @@ public interface UpdatePartitionSpec extends PendingUpdate { * change conflicts with other additions, removals, or renames. */ UpdatePartitionSpec renameField(String name, String newName); + + /** + * Sets that the new partition spec will be NOT set as the default partition spec for the table, + * the default behavior is to do so. + * + * @return this for method chaining + */ + default UpdatePartitionSpec addNonDefaultSpec() { + throw new UnsupportedOperationException( + this.getClass().getName() + " doesn't implement addNonDefaultSpec()"); + }; } diff --git a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java index 2e1c9199174c..c69f6f3844f9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java +++ b/core/src/main/java/org/apache/iceberg/BaseUpdatePartitionSpec.java @@ -59,11 +59,13 @@ class BaseUpdatePartitionSpec implements UpdatePartitionSpec { private final Map renames = Maps.newHashMap(); private boolean caseSensitive; + private boolean setAsDefault; private int lastAssignedPartitionId; BaseUpdatePartitionSpec(TableOperations ops) { this.ops = ops; this.caseSensitive = true; + this.setAsDefault = true; this.base = ops.current(); this.formatVersion = base.formatVersion(); this.spec = base.spec(); @@ -95,6 +97,7 @@ class BaseUpdatePartitionSpec implements UpdatePartitionSpec { this.base = null; this.formatVersion = formatVersion; this.caseSensitive = true; + this.setAsDefault = true; this.spec = spec; this.schema = spec.schema(); this.nameToField = indexSpecByName(spec); @@ -146,6 +149,12 @@ public UpdatePartitionSpec caseSensitive(boolean isCaseSensitive) { return this; } + @Override + public UpdatePartitionSpec addNonDefaultSpec() { + this.setAsDefault = false; + return this; + } + @Override public BaseUpdatePartitionSpec addField(String sourceName) { return addField(Expressions.ref(sourceName)); @@ -327,7 +336,12 @@ public PartitionSpec apply() { @Override public void commit() { - TableMetadata update = base.updatePartitionSpec(apply()); + TableMetadata update; + if (setAsDefault) { + update = base.updatePartitionSpec(apply()); + } else { + update = base.addPartitionSpec(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 bd1c8a1a0371..923db6bbd68f 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -564,6 +564,10 @@ public TableMetadata updatePartitionSpec(PartitionSpec newPartitionSpec) { return new Builder(this).setDefaultPartitionSpec(newPartitionSpec).build(); } + public TableMetadata addPartitionSpec(PartitionSpec newPartitionSpec) { + return new Builder(this).addPartitionSpec(newPartitionSpec).build(); + } + public TableMetadata replaceSortOrder(SortOrder newOrder) { return new Builder(this).setDefaultSortOrder(newOrder).build(); } diff --git a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java index 482514c40093..f327ef752947 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java +++ b/core/src/test/java/org/apache/iceberg/TestTableUpdatePartitionSpec.java @@ -287,4 +287,23 @@ public void testAddAfterLastFieldRemoved() { assertThat(table.spec().lastAssignedFieldId()).isEqualTo(1001); assertThat(table.ops().current().lastAssignedPartitionId()).isEqualTo(1001); } + + @TestTemplate + public void testCommitUpdatedSpecWithoutSettingNewDefault() { + PartitionSpec originalSpec = table.spec(); + table.updateSpec().addField("id").addNonDefaultSpec().commit(); + + assertThat(table.spec()) + .as("Should not set the default spec for the table") + .isSameAs(originalSpec); + + assertThat(table.specs().get(1)) + .as("The new spec created for the table") + .isEqualTo( + PartitionSpec.builderFor(table.schema()) + .withSpecId(1) + .bucket("data", 16) + .identity("id") + .build()); + } } From 25e52b6d600e74ae55134d31b23c7dfd2300c9ef Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 01:14:32 -0500 Subject: [PATCH 0553/1019] Build: Bump com.palantir.baseline:gradle-baseline-java (#10864) Bumps [com.palantir.baseline:gradle-baseline-java](https://github.com/palantir/gradle-baseline) from 5.58.0 to 5.61.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.58.0...5.61.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 60fb5e7830d3..74029f3db2eb 100644 --- a/build.gradle +++ b/build.gradle @@ -27,7 +27,7 @@ buildscript { } dependencies { classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' - classpath 'com.palantir.baseline:gradle-baseline-java:5.58.0' + classpath 'com.palantir.baseline:gradle-baseline-java:5.61.0' classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' classpath 'me.champeau.jmh:jmh-gradle-plugin:0.7.2' From b6ef4cc26f1d9d86b9f685cab28dd85af96d8111 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 08:57:45 +0200 Subject: [PATCH 0554/1019] Build: Bump nessie from 0.94.2 to 0.94.4 (#10869) Bumps `nessie` from 0.94.2 to 0.94.4. Updates `org.projectnessie.nessie:nessie-client` from 0.94.2 to 0.94.4 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.94.2 to 0.94.4 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.94.2 to 0.94.4 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.94.2 to 0.94.4 --- 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 02702955a5fe..2889ce9cb033 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -67,7 +67,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.94.2" +nessie = "0.94.4" netty-buffer = "4.1.112.Final" netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" From 153129278377f4872cea65ee8870bf16a847f0a4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 09:02:39 +0200 Subject: [PATCH 0555/1019] Build: Bump org.xerial:sqlite-jdbc from 3.46.0.0 to 3.46.0.1 (#10871) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.46.0.0 to 3.46.0.1. - [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.0.0...3.46.0.1) --- 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 2889ce9cb033..16ea78901858 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,7 +84,7 @@ spark-hive34 = "3.4.3" spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.37" -sqlite-jdbc = "3.46.0.0" +sqlite-jdbc = "3.46.0.1" testcontainers = "1.20.0" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 2b0ccee4d0505e11b635446f812adb936ba3c461 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 09:08:07 +0200 Subject: [PATCH 0556/1019] Build: Bump org.apache.commons:commons-compress from 1.26.0 to 1.26.2 (#10868) Bumps org.apache.commons:commons-compress from 1.26.0 to 1.26.2. --- updated-dependencies: - dependency-name: org.apache.commons:commons-compress 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> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 60fa879d291f..a0e55f23405d 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -71,7 +71,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' force 'org.xerial.snappy:snappy-java:1.1.10.5' - force 'org.apache.commons:commons-compress:1.26.0' + force 'org.apache.commons:commons-compress:1.26.2' force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' } } From a71ce2c200c7c149b4c89669f02e4cb46d058454 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 09:08:21 +0200 Subject: [PATCH 0557/1019] Build: Bump software.amazon.awssdk:bom from 2.26.25 to 2.26.29 (#10866) Bumps software.amazon.awssdk:bom from 2.26.25 to 2.26.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 16ea78901858..b8f20f3a2799 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.11.3" assertj-core = "3.26.3" awaitility = "4.2.1" -awssdk-bom = "2.26.25" +awssdk-bom = "2.26.29" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 6e4d16f87df3ee5ef485644a8b0bda3af5b3abc8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Aug 2024 09:08:35 +0200 Subject: [PATCH 0558/1019] Build: Bump mkdocs-material from 9.5.30 to 9.5.31 (#10863) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.30 to 9.5.31. - [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.30...9.5.31) --- 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 3fd9cf45e443..cb85511b7d96 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.30 +mkdocs-material==9.5.31 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 83c5b399dcb2cb38260f50e78fb7a664a2827a4d Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Mon, 5 Aug 2024 09:08:58 +0200 Subject: [PATCH 0559/1019] Build: Fix Scala compilation (#10860) `ScalaCompile` does not respect `options.release` and `-release:11` in the aruments is not enough. Re-adding `sourceCompatibility` + `targetCompatibility` for 11. --- build.gradle | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/build.gradle b/build.gradle index 74029f3db2eb..7a11943cf8be 100644 --- a/build.gradle +++ b/build.gradle @@ -237,6 +237,10 @@ subprojects { plugins.withType(ScalaPlugin.class) { tasks.withType(ScalaCompile.class) { scalaCompileOptions.keepAliveMode.set(KeepAliveMode.DAEMON) + // `options.release` doesn't seem to work for ScalaCompile :( + sourceCompatibility = "11" + targetCompatibility = "11" + scalaCompileOptions.additionalParameters.add("-release:11") } } } From 419e32ae5d94fa7895a58280ec09e50235a851a4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 5 Aug 2024 09:17:42 +0200 Subject: [PATCH 0560/1019] Build: Enable FormatStringAnnotation error-prone check (#10856) --- baseline.gradle | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index e3fe602e91c9..42be6d8a24c6 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -98,8 +98,7 @@ subprojects { '-Xep:EqualsGetClass:OFF', // specific to Palantir '-Xep:FinalClass:OFF', - // TODO (https://github.com/apache/iceberg/issues/10854) this is a recently added check. Figure out whether we adjust the code or suppress for good - '-Xep:FormatStringAnnotation:WARN', + '-Xep:FormatStringAnnotation:ERROR', // TODO (https://github.com/apache/iceberg/issues/10855) this is a recently added check. Figure out whether we adjust the code or suppress for good '-Xep:ImmutablesReferenceEquality:WARN', '-Xep:IntLongMath:ERROR', From 0bb19dab50116b5033cd578d50b44837082e884f Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 5 Aug 2024 14:35:07 +0200 Subject: [PATCH 0561/1019] Core: Use encoding/decoding methods for namespaces and deprecate Splitter/Joiner (#10858) --- .../apache/iceberg/rest/RESTSessionCatalog.java | 2 +- .../java/org/apache/iceberg/rest/RESTUtil.java | 16 +++++++++++++--- .../apache/iceberg/rest/RESTCatalogAdapter.java | 6 +----- 3 files changed, 15 insertions(+), 9 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 a72d3958c140..1c607e3b0220 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -547,7 +547,7 @@ public void createNamespace( public List listNamespaces(SessionContext context, Namespace namespace) { Map queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { - queryParams.put("parent", RESTUtil.NAMESPACE_JOINER.join(namespace.levels())); + queryParams.put("parent", RESTUtil.encodeNamespace(namespace)); } 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 fab01162cad7..45422b8ae8b5 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTUtil.java @@ -33,14 +33,24 @@ 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 53d57bee510f..2c928c06e52b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -288,11 +288,7 @@ public T handleRequest( if (asNamespaceCatalog != null) { Namespace ns; if (vars.containsKey("parent")) { - ns = - Namespace.of( - RESTUtil.NAMESPACE_SPLITTER - .splitToStream(vars.get("parent")) - .toArray(String[]::new)); + ns = RESTUtil.decodeNamespace(vars.get("parent")); } else { ns = Namespace.empty(); } From c2849f7b3a6360e2874958425b5319b7b7eb21e7 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 5 Aug 2024 20:40:26 +0200 Subject: [PATCH 0562/1019] Aliyun: Replace assert usage with assertThat (#10880) --- .../oss/mock/AliyunOSSMockLocalStore.java | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) 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 499e61495fc0..f7a4b72e4b97 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 @@ -18,6 +18,8 @@ */ package org.apache.iceberg.aliyun.oss.mock; +import static org.assertj.core.api.Assertions.assertThat; + import com.aliyun.oss.OSSErrorCode; import com.aliyun.oss.model.Bucket; import com.fasterxml.jackson.databind.ObjectMapper; @@ -137,7 +139,9 @@ ObjectMetadata putObject( Map userMetaData) throws IOException { File bucketDir = new File(root, bucketName); - assert bucketDir.exists() || bucketDir.mkdirs(); + assertThat(bucketDir) + .satisfiesAnyOf( + bucket -> assertThat(bucket).exists(), bucket -> assertThat(bucket.mkdirs()).isTrue()); File dataFile = new File(bucketDir, fileName + DATA_FILE); File metaFile = new File(bucketDir, fileName + META_FILE); @@ -170,17 +174,21 @@ ObjectMetadata putObject( void deleteObject(String bucketName, String filename) { File bucketDir = new File(root, bucketName); - assert bucketDir.exists(); + assertThat(bucketDir).exists(); File dataFile = new File(bucketDir, filename + DATA_FILE); File metaFile = new File(bucketDir, filename + META_FILE); - assert !dataFile.exists() || dataFile.delete(); - assert !metaFile.exists() || metaFile.delete(); + assertThat(dataFile) + .satisfiesAnyOf( + file -> assertThat(file).doesNotExist(), file -> assertThat(file.delete()).isTrue()); + assertThat(metaFile) + .satisfiesAnyOf( + file -> assertThat(file).doesNotExist(), file -> assertThat(file.delete()).isTrue()); } ObjectMetadata getObjectMetadata(String bucketName, String filename) throws IOException { File bucketDir = new File(root, bucketName); - assert bucketDir.exists(); + assertThat(bucketDir).exists(); File dataFile = new File(bucketDir, filename + DATA_FILE); if (!dataFile.exists()) { From 11858d1fd20be1621adbbc70f8d93e579464433b Mon Sep 17 00:00:00 2001 From: Denys Kuzmenko Date: Mon, 5 Aug 2024 22:43:34 +0200 Subject: [PATCH 0563/1019] Core: Extract filePath comparator into it's own class (#10664) --- .../org/apache/iceberg/types/Comparators.java | 41 +++++++++++++++++++ .../org/apache/iceberg/deletes/Deletes.java | 31 ++------------ 2 files changed, 45 insertions(+), 27 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java index d09d9f5395ce..a803afac104f 100644 --- a/api/src/main/java/org/apache/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -173,6 +173,10 @@ public static Comparator charSequences() { return CharSeqComparator.INSTANCE; } + public static Comparator filePath() { + return FilePathComparator.INSTANCE; + } + private static class NullsFirst implements Comparator { private static final NullsFirst INSTANCE = new NullsFirst<>(); @@ -351,4 +355,41 @@ public int compare(CharSequence s1, CharSequence s2) { return Integer.compare(s1.length(), s2.length()); } } + + private static class FilePathComparator implements Comparator { + private static final FilePathComparator INSTANCE = new FilePathComparator(); + + private FilePathComparator() {} + + @Override + public int compare(CharSequence s1, CharSequence s2) { + if (s1 == s2) { + return 0; + } + int count = s1.length(); + + int cmp = Integer.compare(count, s2.length()); + if (cmp != 0) { + return cmp; + } + + if (s1 instanceof String && s2 instanceof String) { + cmp = Integer.compare(s1.hashCode(), s2.hashCode()); + if (cmp != 0) { + return cmp; + } + } + // File paths inside a delete file normally have more identical chars at the beginning. For + // example, a typical + // path is like "s3:/bucket/db/table/data/partition/00000-0-[uuid]-00001.parquet". + // The uuid is where the difference starts. So it's faster to find the first diff backward. + for (int i = count - 1; i >= 0; i--) { + cmp = Character.compare(s1.charAt(i), s2.charAt(i)); + if (cmp != 0) { + return cmp; + } + } + return 0; + } + } } 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 ff20ba53ff70..cef57cd16726 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -36,6 +36,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.types.Comparators; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.Filter; @@ -398,33 +399,9 @@ private static class DataFileFilter extends Filter { @Override protected boolean shouldKeep(T posDelete) { - return charSeqEquals(dataLocation, (CharSequence) FILENAME_ACCESSOR.get(posDelete)); - } - - private boolean charSeqEquals(CharSequence s1, CharSequence s2) { - if (s1 == s2) { - return true; - } - - int count = s1.length(); - if (count != s2.length()) { - return false; - } - - if (s1 instanceof String && s2 instanceof String && s1.hashCode() != s2.hashCode()) { - return false; - } - - // File paths inside a delete file normally have more identical chars at the beginning. For - // example, a typical - // path is like "s3:/bucket/db/table/data/partition/00000-0-[uuid]-00001.parquet". - // The uuid is where the difference starts. So it's faster to find the first diff backward. - for (int i = count - 1; i >= 0; i--) { - if (s1.charAt(i) != s2.charAt(i)) { - return false; - } - } - return true; + return Comparators.filePath() + .compare(dataLocation, (CharSequence) FILENAME_ACCESSOR.get(posDelete)) + == 0; } } } From 16563b236066f9b02fdcaba523ef9d9ac7dbb69d Mon Sep 17 00:00:00 2001 From: "k.nakagaki" <141020064+nakaken-churadata@users.noreply.github.com> Date: Tue, 6 Aug 2024 05:50:21 +0900 Subject: [PATCH 0564/1019] Docs: Fix SQL in branching docs (#10876) --- docs/docs/branching.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/branching.md b/docs/docs/branching.md index 3379264d8a5f..f00defda665b 100644 --- a/docs/docs/branching.md +++ b/docs/docs/branching.md @@ -148,7 +148,7 @@ SELECT * FROM db.table.branch_test_branch; Modify the table's schema by dropping the `col` column and adding a new column named `new_col`: ```sql -ALTER TABLE db.table drop column float; +ALTER TABLE db.table drop column col; ALTER TABLE db.table add column new_col date; From abaf2157084ebfcb54453c4cc09d41cf77c2baff Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 5 Aug 2024 14:36:20 -0700 Subject: [PATCH 0565/1019] API: Add SupportsRecoveryOperations mixin for FileIO (#10711) --- .../io/SupportsRecoveryOperations.java | 36 +++++++++++++++++++ 1 file changed, 36 insertions(+) create mode 100644 api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.java diff --git a/api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.java b/api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.java new file mode 100644 index 000000000000..c402d2e68e7d --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/io/SupportsRecoveryOperations.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.io; + +/** + * This interface is intended as an extension for FileIO implementations to provide additional + * best-effort recovery operations that can be useful for repairing corrupted tables where there are + * reachable files missing from disk. (e.g. a live manifest points to data file entry which no + * longer exists on disk) + */ +public interface SupportsRecoveryOperations { + + /** + * Perform a best-effort recovery of a file at a given path + * + * @param path Absolute path of file to attempt recovery for + * @return true if recovery was successful, false otherwise + */ + boolean recoverFile(String path); +} From f3266abd1f235796359223b224310903ccad0df9 Mon Sep 17 00:00:00 2001 From: emkornfield Date: Mon, 5 Aug 2024 18:06:36 -0700 Subject: [PATCH 0566/1019] Spec: Clarify identity partition edge cases (#10835) --- format/spec.md | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index daef7538e730..c3321fa6991c 100644 --- a/format/spec.md +++ b/format/spec.md @@ -150,6 +150,10 @@ 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 v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. +##### 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. + ### 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. @@ -241,7 +245,14 @@ Struct evolution requires the following rules for default values: #### 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. If a field id is missing from a data file, its value for each row should be `null`. +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. + +Values for field ids which are not present in a data file must be resolved according the following rules: + +* Return the value from partition metadata if an [Identity Transform](#partition-transforms) exists for the field and the partition value is present in the `partition` struct on `data_file` object in the manifest. This allows for metadata only migrations of Hive tables. +* Use `schema.name-mapping.default` metadata to map field id to columns without field id as described below and use the column if it is present. +* Return the default value if it has a defined `initial-default` (See [Default values](#default-values) section for more details). +* Return `null` in all other cases. For example, a file may be written with schema `1: a int, 2: b string, 3: c double` and read using projection schema `3: measurement, 2: name, 4: a`. This must select file columns `c` (renamed to `measurement`), `b` (now called `name`), and a column of `null` values called `a`; in that order. From f3cb30b5f6fc8c523da7f60b5163071ca2ff4e4e Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Aug 2024 11:16:19 +0200 Subject: [PATCH 0567/1019] Build: Bump org.testcontainers:testcontainers from 1.20.0 to 1.20.1 (#10865) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.20.0 to 1.20.1. - [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.0...1.20.1) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 b8f20f3a2799..12caeda95407 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -85,7 +85,7 @@ spark-hive35 = "3.5.1" spring-boot = "2.7.18" spring-web = "5.3.37" sqlite-jdbc = "3.46.0.1" -testcontainers = "1.20.0" +testcontainers = "1.20.1" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 215a1cd0d4f792cef968dee61b5c0718bd92df93 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Mon, 5 Aug 2024 08:57:16 -0700 Subject: [PATCH 0568/1019] Flink: move v1.19 to v.120 --- flink/{v1.19 => v1.20}/build.gradle | 0 flink/{v1.19 => v1.20}/flink-runtime/LICENSE | 0 flink/{v1.19 => v1.20}/flink-runtime/NOTICE | 0 .../java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java | 0 .../iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java | 0 .../src/main/java/org/apache/iceberg/flink/CatalogLoader.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkCatalog.java | 0 .../main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkConfParser.java | 0 .../main/java/org/apache/iceberg/flink/FlinkConfigOptions.java | 0 .../java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java | 0 .../java/org/apache/iceberg/flink/FlinkEnvironmentContext.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkFilters.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkReadConf.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java | 0 .../src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java | 0 .../src/main/java/org/apache/iceberg/flink/IcebergTableSink.java | 0 .../src/main/java/org/apache/iceberg/flink/RowDataWrapper.java | 0 .../flink/src/main/java/org/apache/iceberg/flink/TableLoader.java | 0 .../src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java | 0 .../src/main/java/org/apache/iceberg/flink/actions/Actions.java | 0 .../org/apache/iceberg/flink/actions/RewriteDataFilesAction.java | 0 .../org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java | 0 .../main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java | 0 .../java/org/apache/iceberg/flink/data/FlinkParquetReaders.java | 0 .../java/org/apache/iceberg/flink/data/FlinkParquetWriters.java | 0 .../java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java | 0 .../java/org/apache/iceberg/flink/data/FlinkValueReaders.java | 0 .../java/org/apache/iceberg/flink/data/FlinkValueWriters.java | 0 .../apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java | 0 .../java/org/apache/iceberg/flink/data/RowDataProjection.java | 0 .../src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java | 0 .../main/java/org/apache/iceberg/flink/data/StructRowData.java | 0 .../apache/iceberg/flink/maintenance/operator/MonitorSource.java | 0 .../flink/maintenance/operator/SingleThreadedIteratorSource.java | 0 .../apache/iceberg/flink/maintenance/operator/TableChange.java | 0 .../iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java | 0 .../java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java | 0 .../org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/BucketPartitioner.java | 0 .../java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java | 0 .../java/org/apache/iceberg/flink/sink/CachingTableSupplier.java | 0 .../main/java/org/apache/iceberg/flink/sink/CommitSummary.java | 0 .../main/java/org/apache/iceberg/flink/sink/DeltaManifests.java | 0 .../org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java | 0 .../org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java | 0 .../org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java | 0 .../java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java | 0 .../apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java | 0 .../java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java | 0 .../org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java | 0 .../org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java | 0 .../java/org/apache/iceberg/flink/sink/PartitionKeySelector.java | 0 .../org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java | 0 .../org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/TaskWriterFactory.java | 0 .../org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java | 0 .../iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java | 0 .../apache/iceberg/flink/sink/shuffle/CompletedStatistics.java | 0 .../iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java | 0 .../org/apache/iceberg/flink/sink/shuffle/DataStatistics.java | 0 .../iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java | 0 .../flink/sink/shuffle/DataStatisticsCoordinatorProvider.java | 0 .../apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java | 0 .../iceberg/flink/sink/shuffle/DataStatisticsSerializer.java | 0 .../org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java | 0 .../iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java | 0 .../java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java | 0 .../java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java | 0 .../org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java | 0 .../apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java | 0 .../org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java | 0 .../iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java | 0 .../apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java | 0 .../apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java | 0 .../java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java | 0 .../org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java | 0 .../iceberg/flink/sink/shuffle/SortKeySketchSerializer.java | 0 .../java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java | 0 .../org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java | 0 .../org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java | 0 .../iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java | 0 .../org/apache/iceberg/flink/sink/shuffle/StatisticsType.java | 0 .../org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java | 0 .../iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java | 0 .../main/java/org/apache/iceberg/flink/source/DataIterator.java | 0 .../main/java/org/apache/iceberg/flink/source/DataTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/FileScanTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/FlinkInputFormat.java | 0 .../java/org/apache/iceberg/flink/source/FlinkInputSplit.java | 0 .../main/java/org/apache/iceberg/flink/source/FlinkSource.java | 0 .../java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java | 0 .../main/java/org/apache/iceberg/flink/source/IcebergSource.java | 0 .../java/org/apache/iceberg/flink/source/IcebergTableSource.java | 0 .../apache/iceberg/flink/source/RowDataFileScanTaskReader.java | 0 .../java/org/apache/iceberg/flink/source/RowDataRewriter.java | 0 .../iceberg/flink/source/RowDataToAvroGenericRecordConverter.java | 0 .../main/java/org/apache/iceberg/flink/source/ScanContext.java | 0 .../src/main/java/org/apache/iceberg/flink/source/SourceUtil.java | 0 .../org/apache/iceberg/flink/source/StreamingMonitorFunction.java | 0 .../org/apache/iceberg/flink/source/StreamingReaderOperator.java | 0 .../apache/iceberg/flink/source/StreamingStartingStrategy.java | 0 .../iceberg/flink/source/assigner/DefaultSplitAssigner.java | 0 .../org/apache/iceberg/flink/source/assigner/GetSplitResult.java | 0 .../flink/source/assigner/OrderedSplitAssignerFactory.java | 0 .../iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java | 0 .../org/apache/iceberg/flink/source/assigner/SplitAssigner.java | 0 .../iceberg/flink/source/assigner/SplitAssignerFactory.java | 0 .../apache/iceberg/flink/source/assigner/SplitAssignerType.java | 0 .../flink/source/enumerator/AbstractIcebergEnumerator.java | 0 .../flink/source/enumerator/ContinuousEnumerationResult.java | 0 .../flink/source/enumerator/ContinuousIcebergEnumerator.java | 0 .../iceberg/flink/source/enumerator/ContinuousSplitPlanner.java | 0 .../flink/source/enumerator/ContinuousSplitPlannerImpl.java | 0 .../iceberg/flink/source/enumerator/EnumerationHistory.java | 0 .../flink/source/enumerator/IcebergEnumeratorPosition.java | 0 .../source/enumerator/IcebergEnumeratorPositionSerializer.java | 0 .../iceberg/flink/source/enumerator/IcebergEnumeratorState.java | 0 .../flink/source/enumerator/IcebergEnumeratorStateSerializer.java | 0 .../iceberg/flink/source/enumerator/StaticIcebergEnumerator.java | 0 .../org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java | 0 .../iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java | 0 .../flink/source/reader/AvroGenericRecordReaderFunction.java | 0 .../flink/source/reader/ColumnStatsWatermarkExtractor.java | 0 .../apache/iceberg/flink/source/reader/DataIteratorBatcher.java | 0 .../iceberg/flink/source/reader/DataIteratorReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/IcebergSourceReader.java | 0 .../iceberg/flink/source/reader/IcebergSourceReaderMetrics.java | 0 .../iceberg/flink/source/reader/IcebergSourceSplitReader.java | 0 .../apache/iceberg/flink/source/reader/LimitableDataIterator.java | 0 .../org/apache/iceberg/flink/source/reader/ListBatchRecords.java | 0 .../iceberg/flink/source/reader/ListDataIteratorBatcher.java | 0 .../iceberg/flink/source/reader/MetaDataReaderFunction.java | 0 .../org/apache/iceberg/flink/source/reader/ReaderFunction.java | 0 .../org/apache/iceberg/flink/source/reader/RecordAndPosition.java | 0 .../org/apache/iceberg/flink/source/reader/RecordFactory.java | 0 .../org/apache/iceberg/flink/source/reader/RecordLimiter.java | 0 .../apache/iceberg/flink/source/reader/RowDataReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/RowDataRecordFactory.java | 0 .../iceberg/flink/source/reader/SerializableRecordEmitter.java | 0 .../iceberg/flink/source/reader/SplitWatermarkExtractor.java | 0 .../flink/source/reader/WatermarkExtractorRecordEmitter.java | 0 .../org/apache/iceberg/flink/source/split/IcebergSourceSplit.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitSerializer.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitState.java | 0 .../iceberg/flink/source/split/IcebergSourceSplitStatus.java | 0 .../apache/iceberg/flink/source/split/SerializableComparator.java | 0 .../org/apache/iceberg/flink/source/split/SerializerHelper.java | 0 .../org/apache/iceberg/flink/source/split/SplitComparators.java | 0 .../org/apache/iceberg/flink/source/split/SplitRequestEvent.java | 0 .../main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java | 0 .../java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java | 0 .../org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java | 0 .../src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 .../services/org.apache.flink.table.factories.TableFactory | 0 .../org/apache/iceberg/flink/AvroGenericRecordConverterBase.java | 0 .../src/test/java/org/apache/iceberg/flink/CatalogTestBase.java | 0 .../src/test/java/org/apache/iceberg/flink/DataGenerator.java | 0 .../src/test/java/org/apache/iceberg/flink/DataGenerators.java | 0 .../java/org/apache/iceberg/flink/HadoopCatalogExtension.java | 0 .../test/java/org/apache/iceberg/flink/HadoopTableExtension.java | 0 .../java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java | 0 .../src/test/java/org/apache/iceberg/flink/RowDataConverter.java | 0 .../src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/SqlBase.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/TestBase.java | 0 .../src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java | 0 .../java/org/apache/iceberg/flink/TestCatalogTableLoader.java | 0 .../test/java/org/apache/iceberg/flink/TestChangeLogTable.java | 0 .../java/org/apache/iceberg/flink/TestDataFileSerialization.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFixtures.java | 0 .../java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java | 0 .../java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java | 0 .../java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java | 0 .../org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkConfParser.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java | 0 .../test/java/org/apache/iceberg/flink/TestFlinkTableSink.java | 0 .../java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java | 0 .../src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java | 0 .../flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java | 0 .../test/java/org/apache/iceberg/flink/TestIcebergConnector.java | 0 .../org/apache/iceberg/flink/TestManifestFileSerialization.java | 0 .../test/java/org/apache/iceberg/flink/TestRowDataWrapper.java | 0 .../src/test/java/org/apache/iceberg/flink/TestTableLoader.java | 0 .../java/org/apache/iceberg/flink/TestTableSerialization.java | 0 .../apache/iceberg/flink/actions/TestRewriteDataFilesAction.java | 0 .../test/java/org/apache/iceberg/flink/data/RandomRowData.java | 0 .../java/org/apache/iceberg/flink/data/RowDataToRowMapper.java | 0 .../org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java | 0 .../org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java | 0 .../org/apache/iceberg/flink/data/TestFlinkParquetReader.java | 0 .../org/apache/iceberg/flink/data/TestFlinkParquetWriter.java | 0 .../java/org/apache/iceberg/flink/data/TestRowDataProjection.java | 0 .../java/org/apache/iceberg/flink/data/TestRowProjection.java | 0 .../java/org/apache/iceberg/flink/data/TestStructRowData.java | 0 .../apache/iceberg/flink/maintenance/operator/CollectingSink.java | 0 .../iceberg/flink/maintenance/operator/FlinkSqlExtension.java | 0 .../flink/maintenance/operator/FlinkStreamingTestUtils.java | 0 .../apache/iceberg/flink/maintenance/operator/ManualSource.java | 0 .../iceberg/flink/maintenance/operator/OperatorTestBase.java | 0 .../iceberg/flink/maintenance/operator/TestMonitorSource.java | 0 .../iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java | 0 .../apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java | 0 .../java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java | 0 .../iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java | 0 .../org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java | 0 .../org/apache/iceberg/flink/sink/TestCachingTableSupplier.java | 0 .../org/apache/iceberg/flink/sink/TestCompressionSettings.java | 0 .../java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java | 0 .../java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java | 0 .../iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java | 0 .../apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java | 0 .../apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java | 0 .../java/org/apache/iceberg/flink/sink/TestFlinkManifest.java | 0 .../apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java | 0 .../apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java | 0 .../apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java | 0 .../org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java | 0 .../org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java | 0 .../org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java | 0 .../org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java | 0 .../test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java | 0 .../test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java | 0 .../flink/sink/shuffle/TestAggregatedStatisticsTracker.java | 0 .../flink/sink/shuffle/TestCompletedStatisticsSerializer.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java | 0 .../flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java | 0 .../iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java | 0 .../flink/sink/shuffle/TestGlobalStatisticsSerializer.java | 0 .../apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java | 0 .../iceberg/flink/sink/shuffle/TestMapRangePartitioner.java | 0 .../apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java | 0 .../iceberg/flink/sink/shuffle/TestSketchDataStatistics.java | 0 .../iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java | 0 .../org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java | 0 .../iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java | 0 .../flink/sink/shuffle/TestSortKeySerializerNestedStruct.java | 0 .../flink/sink/shuffle/TestSortKeySerializerPrimitives.java | 0 .../iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java | 0 .../org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java | 0 .../java/org/apache/iceberg/flink/source/BoundedTableFactory.java | 0 .../java/org/apache/iceberg/flink/source/BoundedTestSource.java | 0 .../org/apache/iceberg/flink/source/ChangeLogTableTestBase.java | 0 .../test/java/org/apache/iceberg/flink/source/SplitHelpers.java | 0 .../src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java | 0 .../java/org/apache/iceberg/flink/source/TableSourceTestBase.java | 0 .../org/apache/iceberg/flink/source/TestBoundedTableFactory.java | 0 .../org/apache/iceberg/flink/source/TestFlinkInputFormat.java | 0 .../iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java | 0 .../org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java | 0 .../org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java | 0 .../apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java | 0 .../test/java/org/apache/iceberg/flink/source/TestFlinkScan.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkScanSql.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkSource.java | 0 .../org/apache/iceberg/flink/source/TestFlinkSourceConfig.java | 0 .../java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java | 0 .../org/apache/iceberg/flink/source/TestFlinkTableSource.java | 0 .../org/apache/iceberg/flink/source/TestIcebergSourceBounded.java | 0 .../flink/source/TestIcebergSourceBoundedGenericRecord.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceContinuous.java | 0 .../apache/iceberg/flink/source/TestIcebergSourceFailover.java | 0 .../source/TestIcebergSourceFailoverWithWatermarkExtractor.java | 0 .../iceberg/flink/source/TestIcebergSourceReaderDeletes.java | 0 .../org/apache/iceberg/flink/source/TestIcebergSourceSql.java | 0 .../flink/source/TestIcebergSourceWithWatermarkExtractor.java | 0 .../flink/source/TestIcebergSpeculativeExecutionSupport.java | 0 .../iceberg/flink/source/TestMetadataTableReadableMetrics.java | 0 .../org/apache/iceberg/flink/source/TestProjectMetaColumn.java | 0 .../flink/source/TestRowDataToAvroGenericRecordConverter.java | 0 .../java/org/apache/iceberg/flink/source/TestScanContext.java | 0 .../test/java/org/apache/iceberg/flink/source/TestSourceUtil.java | 0 .../test/java/org/apache/iceberg/flink/source/TestSqlBase.java | 0 .../java/org/apache/iceberg/flink/source/TestStreamScanSql.java | 0 .../apache/iceberg/flink/source/TestStreamingMonitorFunction.java | 0 .../apache/iceberg/flink/source/TestStreamingReaderOperator.java | 0 .../iceberg/flink/source/assigner/SplitAssignerTestBase.java | 0 .../iceberg/flink/source/assigner/TestDefaultSplitAssigner.java | 0 .../source/assigner/TestFileSequenceNumberBasedSplitAssigner.java | 0 .../flink/source/assigner/TestWatermarkBasedSplitAssigner.java | 0 .../flink/source/enumerator/ManualContinuousSplitPlanner.java | 0 .../flink/source/enumerator/TestContinuousIcebergEnumerator.java | 0 .../flink/source/enumerator/TestContinuousSplitPlannerImpl.java | 0 .../enumerator/TestContinuousSplitPlannerImplStartStrategy.java | 0 .../iceberg/flink/source/enumerator/TestEnumerationHistory.java | 0 .../source/enumerator/TestIcebergEnumeratorStateSerializer.java | 0 .../iceberg/flink/source/reader/ReaderFunctionTestBase.java | 0 .../java/org/apache/iceberg/flink/source/reader/ReaderUtil.java | 0 .../apache/iceberg/flink/source/reader/TestArrayBatchRecords.java | 0 .../source/reader/TestArrayPoolDataIteratorBatcherRowData.java | 0 .../flink/source/reader/TestColumnStatsWatermarkExtractor.java | 0 .../iceberg/flink/source/reader/TestIcebergSourceReader.java | 0 .../iceberg/flink/source/reader/TestLimitableDataIterator.java | 0 .../iceberg/flink/source/reader/TestRowDataReaderFunction.java | 0 .../apache/iceberg/flink/source/reader/TestingMetricGroup.java | 0 .../flink/source/split/TestIcebergSourceSplitSerializer.java | 0 .../test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java | 0 .../META-INF/services/org.apache.flink.table.factories.Factory | 0 325 files changed, 0 insertions(+), 0 deletions(-) rename flink/{v1.19 => v1.20}/build.gradle (100%) rename flink/{v1.19 => v1.20}/flink-runtime/LICENSE (100%) rename flink/{v1.19 => v1.20}/flink-runtime/NOTICE (100%) rename flink/{v1.19 => v1.20}/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java (100%) rename flink/{v1.19 => v1.20}/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java (100%) rename flink/{v1.19 => v1.20}/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) rename flink/{v1.19 => v1.20}/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java (100%) rename flink/{v1.19 => v1.20}/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory (100%) diff --git a/flink/v1.19/build.gradle b/flink/v1.20/build.gradle similarity index 100% rename from flink/v1.19/build.gradle rename to flink/v1.20/build.gradle diff --git a/flink/v1.19/flink-runtime/LICENSE b/flink/v1.20/flink-runtime/LICENSE similarity index 100% rename from flink/v1.19/flink-runtime/LICENSE rename to flink/v1.20/flink-runtime/LICENSE diff --git a/flink/v1.19/flink-runtime/NOTICE b/flink/v1.20/flink-runtime/NOTICE similarity index 100% rename from flink/v1.19/flink-runtime/NOTICE rename to flink/v1.20/flink-runtime/NOTICE diff --git a/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.20/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java similarity index 100% rename from flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java rename to flink/v1.20/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java similarity index 100% rename from flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java rename to flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java similarity index 100% rename from flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java rename to flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java diff --git a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory similarity index 100% rename from flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory rename to flink/v1.20/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/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/TestFlinkAvroReaderWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java diff --git a/flink/v1.19/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 similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java similarity index 100% rename from flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java rename to flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory similarity index 100% rename from flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory rename to flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory From 375babd39194d21889eb364f6097d927c6f99954 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Mon, 5 Aug 2024 09:00:42 -0700 Subject: [PATCH 0569/1019] Flink: add v1.19 back after coping from 1.20 --- flink/v1.19/build.gradle | 268 ++++ flink/v1.19/flink-runtime/LICENSE | 502 +++++++ flink/v1.19/flink-runtime/NOTICE | 91 ++ .../flink/IcebergConnectorSmokeTest.java | 21 + .../shuffle/MapRangePartitionerBenchmark.java | 199 +++ .../apache/iceberg/flink/CatalogLoader.java | 215 +++ .../apache/iceberg/flink/FlinkCatalog.java | 833 ++++++++++++ .../iceberg/flink/FlinkCatalogFactory.java | 213 +++ .../apache/iceberg/flink/FlinkConfParser.java | 261 ++++ .../iceberg/flink/FlinkConfigOptions.java | 107 ++ .../flink/FlinkDynamicTableFactory.java | 208 +++ .../flink/FlinkEnvironmentContext.java | 31 + .../apache/iceberg/flink/FlinkFilters.java | 266 ++++ .../apache/iceberg/flink/FlinkFixupTypes.java | 50 + .../apache/iceberg/flink/FlinkReadConf.java | 213 +++ .../iceberg/flink/FlinkReadOptions.java | 123 ++ .../apache/iceberg/flink/FlinkSchemaUtil.java | 232 ++++ .../iceberg/flink/FlinkSourceFilter.java | 49 + .../apache/iceberg/flink/FlinkTypeToType.java | 203 +++ .../iceberg/flink/FlinkTypeVisitor.java | 80 ++ .../apache/iceberg/flink/FlinkWriteConf.java | 205 +++ .../iceberg/flink/FlinkWriteOptions.java | 73 + .../iceberg/flink/IcebergTableSink.java | 121 ++ .../apache/iceberg/flink/RowDataWrapper.java | 142 ++ .../org/apache/iceberg/flink/TableLoader.java | 159 +++ .../apache/iceberg/flink/TypeToFlinkType.java | 134 ++ .../apache/iceberg/flink/actions/Actions.java | 52 + .../flink/actions/RewriteDataFilesAction.java | 72 + .../data/AvroWithFlinkSchemaVisitor.java | 75 ++ .../iceberg/flink/data/FlinkAvroReader.java | 169 +++ .../iceberg/flink/data/FlinkAvroWriter.java | 165 +++ .../iceberg/flink/data/FlinkOrcReader.java | 131 ++ .../iceberg/flink/data/FlinkOrcReaders.java | 283 ++++ .../iceberg/flink/data/FlinkOrcWriter.java | 163 +++ .../iceberg/flink/data/FlinkOrcWriters.java | 317 +++++ .../flink/data/FlinkParquetReaders.java | 905 +++++++++++++ .../flink/data/FlinkParquetWriters.java | 504 +++++++ .../flink/data/FlinkSchemaVisitor.java | 161 +++ .../iceberg/flink/data/FlinkValueReaders.java | 312 +++++ .../iceberg/flink/data/FlinkValueWriters.java | 253 ++++ .../data/ParquetWithFlinkSchemaVisitor.java | 222 ++++ .../iceberg/flink/data/RowDataProjection.java | 341 +++++ .../iceberg/flink/data/RowDataUtil.java | 123 ++ .../iceberg/flink/data/StructRowData.java | 300 +++++ .../maintenance/operator/MonitorSource.java | 206 +++ .../SingleThreadedIteratorSource.java | 197 +++ .../maintenance/operator/TableChange.java | 133 ++ .../AvroGenericRecordToRowDataMapper.java | 61 + .../flink/sink/BaseDeltaTaskWriter.java | 126 ++ .../sink/BucketPartitionKeySelector.java | 70 + .../iceberg/flink/sink/BucketPartitioner.java | 103 ++ .../flink/sink/BucketPartitionerUtil.java | 125 ++ .../flink/sink/CachingTableSupplier.java | 91 ++ .../iceberg/flink/sink/CommitSummary.java | 93 ++ .../iceberg/flink/sink/DeltaManifests.java | 71 + .../flink/sink/DeltaManifestsSerializer.java | 122 ++ .../flink/sink/EqualityFieldKeySelector.java | 86 ++ .../flink/sink/FlinkAppenderFactory.java | 274 ++++ .../flink/sink/FlinkFileWriterFactory.java | 293 +++++ .../iceberg/flink/sink/FlinkManifestUtil.java | 132 ++ .../apache/iceberg/flink/sink/FlinkSink.java | 654 +++++++++ .../flink/sink/IcebergFilesCommitter.java | 516 ++++++++ .../sink/IcebergFilesCommitterMetrics.java | 73 + .../flink/sink/IcebergStreamWriter.java | 120 ++ .../sink/IcebergStreamWriterMetrics.java | 89 ++ .../flink/sink/ManifestOutputFileFactory.java | 94 ++ .../flink/sink/PartitionKeySelector.java | 64 + .../flink/sink/PartitionedDeltaWriter.java | 97 ++ .../flink/sink/RowDataTaskWriterFactory.java | 244 ++++ .../iceberg/flink/sink/TaskWriterFactory.java | 45 + .../flink/sink/UnpartitionedDeltaWriter.java | 69 + .../shuffle/AggregatedStatisticsTracker.java | 262 ++++ .../sink/shuffle/CompletedStatistics.java | 111 ++ .../CompletedStatisticsSerializer.java | 176 +++ .../flink/sink/shuffle/DataStatistics.java | 48 + .../shuffle/DataStatisticsCoordinator.java | 522 ++++++++ .../DataStatisticsCoordinatorProvider.java | 70 + .../sink/shuffle/DataStatisticsOperator.java | 266 ++++ .../shuffle/DataStatisticsSerializer.java | 204 +++ .../flink/sink/shuffle/GlobalStatistics.java | 114 ++ .../shuffle/GlobalStatisticsSerializer.java | 199 +++ .../flink/sink/shuffle/KeyAssignment.java | 155 +++ .../flink/sink/shuffle/MapAssignment.java | 242 ++++ .../flink/sink/shuffle/MapDataStatistics.java | 88 ++ .../sink/shuffle/MapRangePartitioner.java | 95 ++ .../flink/sink/shuffle/RangePartitioner.java | 110 ++ .../shuffle/RequestGlobalStatisticsEvent.java | 38 + .../sink/shuffle/SketchDataStatistics.java | 87 ++ .../sink/shuffle/SketchRangePartitioner.java | 51 + .../flink/sink/shuffle/SketchUtil.java | 159 +++ .../flink/sink/shuffle/SortKeySerializer.java | 359 +++++ .../sink/shuffle/SortKeySketchSerializer.java | 143 ++ .../flink/sink/shuffle/SortKeyUtil.java | 59 + .../flink/sink/shuffle/StatisticsEvent.java | 76 ++ .../sink/shuffle/StatisticsOrRecord.java | 112 ++ .../shuffle/StatisticsOrRecordSerializer.java | 208 +++ .../flink/sink/shuffle/StatisticsType.java | 55 + .../flink/sink/shuffle/StatisticsUtil.java | 126 ++ .../AvroGenericRecordFileScanTaskReader.java | 42 + .../iceberg/flink/source/DataIterator.java | 156 +++ .../iceberg/flink/source/DataTaskReader.java | 47 + .../flink/source/FileScanTaskReader.java | 35 + .../flink/source/FlinkInputFormat.java | 141 ++ .../iceberg/flink/source/FlinkInputSplit.java | 48 + .../iceberg/flink/source/FlinkSource.java | 307 +++++ .../flink/source/FlinkSplitPlanner.java | 189 +++ .../iceberg/flink/source/IcebergSource.java | 549 ++++++++ .../flink/source/IcebergTableSource.java | 229 ++++ .../source/RowDataFileScanTaskReader.java | 243 ++++ .../iceberg/flink/source/RowDataRewriter.java | 172 +++ .../RowDataToAvroGenericRecordConverter.java | 70 + .../iceberg/flink/source/ScanContext.java | 597 +++++++++ .../iceberg/flink/source/SourceUtil.java | 77 ++ .../source/StreamingMonitorFunction.java | 269 ++++ .../flink/source/StreamingReaderOperator.java | 246 ++++ .../source/StreamingStartingStrategy.java | 54 + .../source/assigner/DefaultSplitAssigner.java | 119 ++ .../flink/source/assigner/GetSplitResult.java | 77 ++ .../assigner/OrderedSplitAssignerFactory.java | 46 + .../assigner/SimpleSplitAssignerFactory.java | 37 + .../flink/source/assigner/SplitAssigner.java | 124 ++ .../source/assigner/SplitAssignerFactory.java | 30 + .../source/assigner/SplitAssignerType.java | 33 + .../enumerator/AbstractIcebergEnumerator.java | 184 +++ .../ContinuousEnumerationResult.java | 57 + .../ContinuousIcebergEnumerator.java | 187 +++ .../enumerator/ContinuousSplitPlanner.java | 30 + .../ContinuousSplitPlannerImpl.java | 240 ++++ .../source/enumerator/EnumerationHistory.java | 100 ++ .../enumerator/IcebergEnumeratorPosition.java | 79 ++ .../IcebergEnumeratorPositionSerializer.java | 90 ++ .../enumerator/IcebergEnumeratorState.java | 65 + .../IcebergEnumeratorStateSerializer.java | 194 +++ .../enumerator/StaticIcebergEnumerator.java | 51 + .../source/reader/ArrayBatchRecords.java | 171 +++ .../reader/ArrayPoolDataIteratorBatcher.java | 130 ++ .../AvroGenericRecordReaderFunction.java | 102 ++ .../reader/ColumnStatsWatermarkExtractor.java | 98 ++ .../source/reader/DataIteratorBatcher.java | 36 + .../reader/DataIteratorReaderFunction.java | 43 + .../source/reader/IcebergSourceReader.java | 77 ++ .../reader/IcebergSourceReaderMetrics.java | 61 + .../reader/IcebergSourceSplitReader.java | 167 +++ .../source/reader/LimitableDataIterator.java | 56 + .../flink/source/reader/ListBatchRecords.java | 85 ++ .../reader/ListDataIteratorBatcher.java | 94 ++ .../source/reader/MetaDataReaderFunction.java | 65 + .../flink/source/reader/ReaderFunction.java | 31 + .../source/reader/RecordAndPosition.java | 78 ++ .../flink/source/reader/RecordFactory.java | 34 + .../flink/source/reader/RecordLimiter.java | 45 + .../source/reader/RowDataReaderFunction.java | 115 ++ .../source/reader/RowDataRecordFactory.java | 73 + .../reader/SerializableRecordEmitter.java | 40 + .../reader/SplitWatermarkExtractor.java | 28 + .../WatermarkExtractorRecordEmitter.java | 67 + .../source/split/IcebergSourceSplit.java | 220 ++++ .../split/IcebergSourceSplitSerializer.java | 62 + .../source/split/IcebergSourceSplitState.java | 37 + .../split/IcebergSourceSplitStatus.java | 25 + .../source/split/SerializableComparator.java | 24 + .../flink/source/split/SerializerHelper.java | 206 +++ .../flink/source/split/SplitComparators.java | 76 ++ .../flink/source/split/SplitRequestEvent.java | 54 + .../iceberg/flink/util/ElapsedTimeGauge.java | 47 + .../flink/util/FlinkAlterTableUtil.java | 248 ++++ .../flink/util/FlinkCompatibilityUtil.java | 47 + .../iceberg/flink/util/FlinkPackage.java | 61 + .../org.apache.flink.table.factories.Factory | 16 + ....apache.flink.table.factories.TableFactory | 16 + .../flink/AvroGenericRecordConverterBase.java | 90 ++ .../apache/iceberg/flink/CatalogTestBase.java | 121 ++ .../apache/iceberg/flink/DataGenerator.java | 42 + .../apache/iceberg/flink/DataGenerators.java | 1172 +++++++++++++++++ .../iceberg/flink/HadoopCatalogExtension.java | 105 ++ .../iceberg/flink/HadoopTableExtension.java | 59 + .../flink/MiniFlinkClusterExtension.java | 67 + .../iceberg/flink/RowDataConverter.java | 135 ++ .../apache/iceberg/flink/SimpleDataUtil.java | 439 ++++++ .../org/apache/iceberg/flink/SqlBase.java | 110 ++ .../org/apache/iceberg/flink/TestBase.java | 145 ++ .../iceberg/flink/TestCatalogLoader.java | 116 ++ .../iceberg/flink/TestCatalogTableLoader.java | 113 ++ .../iceberg/flink/TestChangeLogTable.java | 296 +++++ .../flink/TestDataFileSerialization.java | 203 +++ .../apache/iceberg/flink/TestFixtures.java | 61 + .../flink/TestFlinkAnonymousTable.java | 65 + .../flink/TestFlinkCatalogDatabase.java | 253 ++++ .../flink/TestFlinkCatalogFactory.java | 119 ++ .../iceberg/flink/TestFlinkCatalogTable.java | 669 ++++++++++ .../TestFlinkCatalogTablePartitions.java | 119 ++ .../iceberg/flink/TestFlinkConfParser.java | 61 + .../iceberg/flink/TestFlinkFilters.java | 462 +++++++ .../iceberg/flink/TestFlinkHiveCatalog.java | 101 ++ .../iceberg/flink/TestFlinkSchemaUtil.java | 416 ++++++ .../iceberg/flink/TestFlinkTableSink.java | 244 ++++ .../flink/TestFlinkTableSinkExtended.java | 244 ++++ .../apache/iceberg/flink/TestFlinkUpsert.java | 334 +++++ .../org/apache/iceberg/flink/TestHelpers.java | 632 +++++++++ .../iceberg/flink/TestIcebergConnector.java | 331 +++++ .../flink/TestManifestFileSerialization.java | 173 +++ .../iceberg/flink/TestRowDataWrapper.java | 93 ++ .../apache/iceberg/flink/TestTableLoader.java | 57 + .../iceberg/flink/TestTableSerialization.java | 110 ++ .../actions/TestRewriteDataFilesAction.java | 481 +++++++ .../iceberg/flink/data/RandomRowData.java | 38 + .../flink/data/RowDataToRowMapper.java | 50 + .../flink/data/TestFlinkAvroReaderWriter.java | 185 +++ .../flink/data/TestFlinkOrcReaderWriter.java | 107 ++ .../flink/data/TestFlinkParquetReader.java | 239 ++++ .../flink/data/TestFlinkParquetWriter.java | 94 ++ .../flink/data/TestRowDataProjection.java | 593 +++++++++ .../iceberg/flink/data/TestRowProjection.java | 596 +++++++++ .../iceberg/flink/data/TestStructRowData.java | 100 ++ .../maintenance/operator/CollectingSink.java | 115 ++ .../operator/FlinkSqlExtension.java | 135 ++ .../operator/FlinkStreamingTestUtils.java | 73 + .../maintenance/operator/ManualSource.java | 316 +++++ .../operator/OperatorTestBase.java | 51 + .../operator/TestMonitorSource.java | 362 +++++ .../TestAvroGenericRecordToRowDataMapper.java | 38 + .../sink/TestBucketPartitionKeySelector.java | 67 + .../flink/sink/TestBucketPartitioner.java | 108 ++ ...TestBucketPartitionerFlinkIcebergSink.java | 227 ++++ .../flink/sink/TestBucketPartitionerUtil.java | 126 ++ .../flink/sink/TestCachingTableSupplier.java | 81 ++ .../flink/sink/TestCompressionSettings.java | 257 ++++ .../flink/sink/TestDeltaTaskWriter.java | 429 ++++++ .../flink/sink/TestFlinkAppenderFactory.java | 65 + .../sink/TestFlinkFileWriterFactory.java | 66 + .../flink/sink/TestFlinkIcebergSink.java | 125 ++ .../flink/sink/TestFlinkIcebergSinkBase.java | 111 ++ .../sink/TestFlinkIcebergSinkBranch.java | 137 ++ .../TestFlinkIcebergSinkDistributionMode.java | 180 +++ .../sink/TestFlinkIcebergSinkExtended.java | 208 +++ .../flink/sink/TestFlinkIcebergSinkV2.java | 235 ++++ .../sink/TestFlinkIcebergSinkV2Base.java | 389 ++++++ .../sink/TestFlinkIcebergSinkV2Branch.java | 125 ++ .../iceberg/flink/sink/TestFlinkManifest.java | 312 +++++ .../sink/TestFlinkPartitioningWriters.java | 77 ++ .../sink/TestFlinkPositionDeltaWriters.java | 66 + .../sink/TestFlinkRollingFileWriters.java | 51 + .../flink/sink/TestFlinkWriterMetrics.java | 60 + .../flink/sink/TestIcebergFilesCommitter.java | 1148 ++++++++++++++++ .../flink/sink/TestIcebergStreamWriter.java | 390 ++++++ .../flink/sink/TestRowDataPartitionKey.java | 251 ++++ .../iceberg/flink/sink/TestTaskWriters.java | 242 ++++ .../iceberg/flink/sink/shuffle/Fixtures.java | 100 ++ .../TestAggregatedStatisticsTracker.java | 465 +++++++ .../TestCompletedStatisticsSerializer.java | 54 + .../TestDataStatisticsCoordinator.java | 246 ++++ ...TestDataStatisticsCoordinatorProvider.java | 187 +++ .../shuffle/TestDataStatisticsOperator.java | 352 +++++ .../shuffle/TestDataStatisticsSerializer.java | 53 + .../TestGlobalStatisticsSerializer.java | 59 + .../sink/shuffle/TestMapDataStatistics.java | 67 + .../sink/shuffle/TestMapRangePartitioner.java | 434 ++++++ .../sink/shuffle/TestRangePartitioner.java | 65 + .../shuffle/TestSketchDataStatistics.java | 60 + .../shuffle/TestSketchRangePartitioner.java | 88 ++ .../flink/sink/shuffle/TestSketchUtil.java | 189 +++ .../shuffle/TestSortKeySerializerBase.java | 65 + .../TestSortKeySerializerNestedStruct.java | 55 + .../TestSortKeySerializerPrimitives.java | 90 ++ .../TestSortKeySerializerSnapshot.java | 213 +++ .../flink/sink/shuffle/TestSortKeyUtil.java | 73 + .../flink/source/BoundedTableFactory.java | 170 +++ .../flink/source/BoundedTestSource.java | 108 ++ .../flink/source/ChangeLogTableTestBase.java | 95 ++ .../iceberg/flink/source/SplitHelpers.java | 200 +++ .../iceberg/flink/source/SqlHelpers.java | 60 + .../flink/source/TableSourceTestBase.java | 105 ++ .../flink/source/TestBoundedTableFactory.java | 81 ++ .../flink/source/TestFlinkInputFormat.java | 211 +++ .../TestFlinkInputFormatReaderDeletes.java | 71 + .../flink/source/TestFlinkMergingMetrics.java | 67 + .../flink/source/TestFlinkMetaDataTable.java | 813 ++++++++++++ .../source/TestFlinkReaderDeletesBase.java | 90 ++ .../iceberg/flink/source/TestFlinkScan.java | 540 ++++++++ .../flink/source/TestFlinkScanSql.java | 69 + .../iceberg/flink/source/TestFlinkSource.java | 90 ++ .../flink/source/TestFlinkSourceConfig.java | 61 + .../flink/source/TestFlinkSourceSql.java | 85 ++ .../flink/source/TestFlinkTableSource.java | 561 ++++++++ .../source/TestIcebergSourceBounded.java | 147 +++ ...TestIcebergSourceBoundedGenericRecord.java | 196 +++ .../source/TestIcebergSourceBoundedSql.java | 76 ++ .../source/TestIcebergSourceContinuous.java | 573 ++++++++ .../source/TestIcebergSourceFailover.java | 394 ++++++ ...gSourceFailoverWithWatermarkExtractor.java | 130 ++ .../TestIcebergSourceReaderDeletes.java | 102 ++ .../flink/source/TestIcebergSourceSql.java | 158 +++ ...stIcebergSourceWithWatermarkExtractor.java | 408 ++++++ ...estIcebergSpeculativeExecutionSupport.java | 184 +++ .../TestMetadataTableReadableMetrics.java | 299 +++++ .../flink/source/TestProjectMetaColumn.java | 188 +++ ...stRowDataToAvroGenericRecordConverter.java | 36 + .../iceberg/flink/source/TestScanContext.java | 112 ++ .../iceberg/flink/source/TestSourceUtil.java | 61 + .../iceberg/flink/source/TestSqlBase.java | 160 +++ .../flink/source/TestStreamScanSql.java | 434 ++++++ .../source/TestStreamingMonitorFunction.java | 402 ++++++ .../source/TestStreamingReaderOperator.java | 293 +++++ .../assigner/SplitAssignerTestBase.java | 132 ++ .../assigner/TestDefaultSplitAssigner.java | 43 + ...tFileSequenceNumberBasedSplitAssigner.java | 81 ++ .../TestWatermarkBasedSplitAssigner.java | 146 ++ .../ManualContinuousSplitPlanner.java | 97 ++ .../TestContinuousIcebergEnumerator.java | 352 +++++ .../TestContinuousSplitPlannerImpl.java | 692 ++++++++++ ...ntinuousSplitPlannerImplStartStrategy.java | 200 +++ .../enumerator/TestEnumerationHistory.java | 135 ++ .../TestIcebergEnumeratorStateSerializer.java | 146 ++ .../source/reader/ReaderFunctionTestBase.java | 218 +++ .../flink/source/reader/ReaderUtil.java | 128 ++ .../source/reader/TestArrayBatchRecords.java | 69 + ...stArrayPoolDataIteratorBatcherRowData.java | 360 +++++ .../TestColumnStatsWatermarkExtractor.java | 176 +++ .../reader/TestIcebergSourceReader.java | 181 +++ .../reader/TestLimitableDataIterator.java | 84 ++ .../reader/TestRowDataReaderFunction.java | 69 + .../source/reader/TestingMetricGroup.java | 102 ++ .../TestIcebergSourceSplitSerializer.java | 183 +++ .../iceberg/flink/util/TestFlinkPackage.java | 55 + .../org.apache.flink.table.factories.Factory | 16 + 325 files changed, 57252 insertions(+) create mode 100644 flink/v1.19/build.gradle create mode 100644 flink/v1.19/flink-runtime/LICENSE create mode 100644 flink/v1.19/flink-runtime/NOTICE create mode 100644 flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java create mode 100644 flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java create mode 100644 flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory create mode 100644 flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java create 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/ManualSource.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java create mode 100644 flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle new file mode 100644 index 000000000000..392a1cb124f0 --- /dev/null +++ b/flink/v1.19/build.gradle @@ -0,0 +1,268 @@ +/* + * 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. + */ + +String flinkMajorVersion = '1.19' +String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") + +project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { + + dependencies { + implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') + api project(':iceberg-api') + implementation project(':iceberg-common') + implementation project(':iceberg-core') + api project(':iceberg-data') + implementation project(':iceberg-orc') + implementation project(':iceberg-parquet') + implementation project(':iceberg-hive-metastore') + + compileOnly libs.flink119.avro + // for dropwizard histogram metrics implementation + compileOnly libs.flink119.metrics.dropwizard + compileOnly libs.flink119.streaming.java + compileOnly "${libs.flink119.streaming.java.get().module}:${libs.flink119.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink119.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" + compileOnly libs.flink119.connector.base + compileOnly libs.flink119.connector.files + + compileOnly libs.hadoop2.hdfs + compileOnly libs.hadoop2.common + compileOnly(libs.hadoop2.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + } + + 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.avro.avro + + implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { + exclude group: 'org.apache.hadoop' + exclude group: 'commons-lang' + // These artifacts are shaded and included in the orc-core fat jar + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.hive', module: 'hive-storage-api' + exclude group: 'org.slf4j' + } + + implementation libs.datasketches + + testImplementation libs.flink119.connector.test.utils + testImplementation libs.flink119.core + testImplementation libs.flink119.runtime + testImplementation(libs.flink119.test.utilsjunit) { + exclude group: 'junit' + } + testImplementation(libs.flink119.test.utils) { + exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' + } + + testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') + testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') + + // By default, hive-exec is a fat/uber jar and it exports a guava library + // that's really old. We use the core classifier to be able to override our guava + // version. Luckily, hive-exec seems to work okay so far with this version of guava + // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. + testImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + testImplementation(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + exclude group: 'org.slf4j' + } + + testImplementation libs.awaitility + testImplementation libs.assertj.core + } + + test { + useJUnitPlatform() + } +} + +project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { + apply plugin: 'io.github.goooler.shadow' + + tasks.jar.dependsOn tasks.shadowJar + + sourceSets { + integration { + java.srcDir "$projectDir/src/integration/java" + resources.srcDir "$projectDir/src/integration/resources" + } + } + + configurations { + implementation { + // included in Flink + exclude group: 'org.slf4j' + exclude group: 'org.apache.commons' + exclude group: 'commons-pool' + exclude group: 'commons-codec' + exclude group: 'org.xerial.snappy' + exclude group: 'javax.xml.bind' + exclude group: 'javax.annotation' + } + } + + dependencies { + implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) { + exclude group: 'org.apache.flink' + } + implementation project(':iceberg-aws') + implementation project(':iceberg-azure') + implementation(project(':iceberg-aliyun')) { + exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' + exclude group: 'org.apache.httpcomponents', module: 'httpclient' + exclude group: 'commons-logging', module: 'commons-logging' + } + implementation project(':iceberg-gcp') + implementation(project(':iceberg-nessie')) { + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + + // for dropwizard histogram metrics implementation + implementation libs.flink119.metrics.dropwizard + + // for integration testing with the flink-runtime-jar + // all of those dependencies are required because the integration test extends FlinkTestBase + integrationCompileOnly project(':iceberg-api') + integrationImplementation libs.junit.vintage.engine + integrationImplementation libs.assertj.core + integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") + integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') + integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') + integrationImplementation(libs.flink119.test.utils) { + exclude group: "org.apache.curator", module: 'curator-test' + exclude group: 'junit' + } + + integrationImplementation libs.flink119.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" + + integrationImplementation libs.hadoop2.common + integrationImplementation libs.hadoop2.hdfs + integrationImplementation(libs.hadoop2.minicluster) { + exclude group: 'org.apache.avro', module: 'avro' + } + + integrationImplementation(libs.hive2.metastore) { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hbase' + exclude group: 'org.apache.logging.log4j' + exclude group: 'co.cask.tephra' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' + exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' + exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' + exclude group: 'com.tdunning', module: 'json' + exclude group: 'javax.transaction', module: 'transaction-api' + exclude group: 'com.zaxxer', module: 'HikariCP' + exclude group: 'org.slf4j' + } + + integrationImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { + exclude group: 'org.apache.avro', module: 'avro' + exclude group: 'org.slf4j', module: 'slf4j-log4j12' + exclude group: 'org.pentaho' // missing dependency + exclude group: 'org.apache.hive', module: 'hive-llap-tez' + exclude group: 'org.apache.logging.log4j' + exclude group: 'com.google.protobuf', module: 'protobuf-java' + exclude group: 'org.apache.calcite' + exclude group: 'org.apache.calcite.avatica' + exclude group: 'com.google.code.findbugs', module: 'jsr305' + } + } + + shadowJar { + configurations = [project.configurations.runtimeClasspath] + + zip64 true + + // include the LICENSE and NOTICE files for the shaded Jar + from(projectDir) { + include 'LICENSE' + include 'NOTICE' + } + + // Relocate dependencies to avoid conflicts + relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' + relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' + relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' + relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' + relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' + relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' + relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' + relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' + relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' + relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' + relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' + relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' + relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' + + archiveClassifier.set(null) + } + + task integrationTest(type: Test) { + description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" + group = "verification" + jvmArgs += project.property('extraJvmArgs') + testClassesDirs = sourceSets.integration.output.classesDirs + classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) + inputs.file(shadowJar.archiveFile.get().asFile.path) + } + integrationTest.dependsOn shadowJar + check.dependsOn integrationTest + + jar { + enabled = false + } +} diff --git a/flink/v1.19/flink-runtime/LICENSE b/flink/v1.19/flink-runtime/LICENSE new file mode 100644 index 000000000000..8ab53469eb87 --- /dev/null +++ b/flink/v1.19/flink-runtime/LICENSE @@ -0,0 +1,502 @@ + + 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 Apache Avro. + +Copyright: 2014-2020 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains the Jackson JSON processor. + +Copyright: 2007-2020 Tatu Saloranta and other contributors +Home page: http://jackson.codehaus.org/ +License: http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Parquet. + +Copyright: 2014-2020 The Apache Software Foundation. +Home page: https://parquet.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Thrift. + +Copyright: 2006-2010 The Apache Software Foundation. +Home page: https://thrift.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains fastutil. + +Copyright: 2002-2014 Sebastiano Vigna +Home page: http://fastutil.di.unimi.it/ +License: http://www.apache.org/licenses/LICENSE-2.0.html + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache ORC. + +Copyright: 2013-2020 The Apache Software Foundation. +Home page: https://orc.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Hive's storage API via ORC. + +Copyright: 2013-2020 The Apache Software Foundation. +Home page: https://hive.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Aircompressor. + +Copyright: 2011-2020 Aircompressor authors. +Home page: https://github.com/airlift/aircompressor +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Airlift Slice. + +Copyright: 2013-2020 Slice authors. +Home page: https://github.com/airlift/slice +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains JetBrains annotations. + +Copyright: 2000-2020 JetBrains s.r.o. +Home page: https://github.com/JetBrains/java-annotations +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Guava. + +Copyright: 2006-2020 The Guava Authors +Home page: https://github.com/google/guava +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google Error Prone Annotations. + +Copyright: Copyright 2011-2019 The Error Prone Authors +Home page: https://github.com/google/error-prone +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains checkerframework checker-qual Annotations. + +Copyright: 2004-2020 the Checker Framework developers +Home page: https://github.com/typetools/checker-framework +License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) + +License text: +| The annotations are licensed under the MIT License. (The text of this +| license appears below.) More specifically, all the parts of the Checker +| Framework that you might want to include with your own program use the +| MIT License. This is the checker-qual.jar file and all the files that +| appear in it: every file in a qual/ directory, plus utility files such +| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. +| In addition, the cleanroom implementations of third-party annotations, +| which the Checker Framework recognizes as aliases for its own +| annotations, are licensed under the MIT License. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Animal Sniffer Annotations. + +Copyright: 2009-2018 codehaus.org +Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/ +License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license) + +License text: +| The MIT License +| +| Copyright (c) 2009 codehaus.org. +| +| Permission is hereby granted, free of charge, to any person obtaining a copy +| of this software and associated documentation files (the "Software"), to deal +| in the Software without restriction, including without limitation the rights +| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell +| copies of the Software, and to permit persons to whom the Software is +| furnished to do so, subject to the following conditions: +| +| The above copyright notice and this permission notice shall be included in +| all copies or substantial portions of the Software. +| +| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR +| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, +| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE +| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER +| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, +| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN +| THE SOFTWARE. + +-------------------------------------------------------------------------------- + +This binary artifact contains Caffeine by Ben Manes. + +Copyright: 2014-2020 Ben Manes and contributors +Home page: https://github.com/ben-manes/caffeine +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Apache Yetus audience annotations. + +Copyright: 2008-2020 The Apache Software Foundation. +Home page: https://yetus.apache.org/ +License: http://www.apache.org/licenses/LICENSE-2.0 + +-------------------------------------------------------------------------------- + +This binary artifact contains Google protobuf. + +Copyright: 2008 Google Inc. +Home page: https://developers.google.com/protocol-buffers +License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) + +License text: + +| Copyright 2008 Google Inc. All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +| +| Code generated by the Protocol Buffer compiler is owned by the owner +| of the input file used when generating it. This code is not +| standalone and requires a support library to be linked with it. This +| support library is itself covered by the above license. + +-------------------------------------------------------------------------------- + +This binary artifact contains ThreeTen. + +Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. +Home page: https://www.threeten.org/threeten-extra/ +License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) + +License text: + +| All rights reserved. +| +| * Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are met: +| +| * Redistributions of source code must retain the above copyright notice, +| this list of conditions and the following disclaimer. +| +| * Redistributions in binary form must reproduce the above copyright notice, +| this list of conditions and the following disclaimer in the documentation +| and/or other materials provided with the distribution. +| +| * Neither the name of JSR-310 nor the names of its contributors +| may be used to endorse or promote products derived from this software +| without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR +| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, +| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, +| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR +| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF +| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS +| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Dremio +| Copyright 2015-2017 Dremio Corporation +| +| This product includes software developed at +| The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary 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 binary artifact contains Apache HttpComponents Client. + +Copyright: 1999-2022 The Apache Software Foundation. +Home page: https://hc.apache.org/ +License: http://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 diff --git a/flink/v1.19/flink-runtime/NOTICE b/flink/v1.19/flink-runtime/NOTICE new file mode 100644 index 000000000000..dc36f84c4ac5 --- /dev/null +++ b/flink/v1.19/flink-runtime/NOTICE @@ -0,0 +1,91 @@ + +Apache Iceberg +Copyright 2017-2024 The Apache Software Foundation + +This product includes software developed at +The Apache Software Foundation (http://www.apache.org/). + +-------------------------------------------------------------------------------- + +This binary artifact includes Apache ORC with the following in its NOTICE file: + +| Apache ORC +| Copyright 2013-2019 The Apache Software Foundation +| +| This product includes software developed by The Apache Software +| Foundation (http://www.apache.org/). +| +| This product includes software developed by Hewlett-Packard: +| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P + +-------------------------------------------------------------------------------- + +This binary artifact includes Airlift Aircompressor with the following in its +NOTICE file: + +| Snappy Copyright Notices +| ========================= +| +| * Copyright 2011 Dain Sundstrom +| * Copyright 2011, Google Inc. +| +| +| Snappy License +| =============== +| Copyright 2011, Google Inc. +| All rights reserved. +| +| Redistribution and use in source and binary forms, with or without +| modification, are permitted provided that the following conditions are +| met: +| +| * Redistributions of source code must retain the above copyright +| notice, this list of conditions and the following disclaimer. +| * Redistributions in binary form must reproduce the above +| copyright notice, this list of conditions and the following disclaimer +| in the documentation and/or other materials provided with the +| distribution. +| * Neither the name of Google Inc. nor the names of its +| contributors may be used to endorse or promote products derived from +| this software without specific prior written permission. +| +| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS +| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT +| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR +| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT +| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, +| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT +| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, +| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT +| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE +| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + +-------------------------------------------------------------------------------- + +This binary artifact includes Apache Yetus with the following in its NOTICE +file: + +| Apache Yetus +| Copyright 2008-2020 The Apache Software Foundation +| +| This product includes software developed at +| The Apache Software Foundation (https://www.apache.org/). +| +| --- +| Additional licenses for the Apache Yetus Source/Website: +| --- +| +| +| See LICENSE for terms. + +-------------------------------------------------------------------------------- + +This binary artifact includes Project Nessie with the following in its NOTICE +file: + +| Dremio +| Copyright 2015-2017 Dremio Corporation +| +| This product includes software developed at +| The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java new file mode 100644 index 000000000000..0d9bbf9d3601 --- /dev/null +++ b/flink/v1.19/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java @@ -0,0 +1,21 @@ +/* + * 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; + +public class IcebergConnectorSmokeTest extends TestIcebergConnector {} diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java new file mode 100644 index 000000000000..c3917165753d --- /dev/null +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -0,0 +1,199 @@ +/* + * 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.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +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.apache.iceberg.types.Types; +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.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Warmup; +import org.openjdk.jmh.infra.Blackhole; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +public class MapRangePartitionerBenchmark { + private static final String CHARS = + "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; + private static final int SAMPLE_SIZE = 100_000; + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name2", Types.StringType.get()), + Types.NestedField.required(3, "name3", Types.StringType.get()), + Types.NestedField.required(4, "name4", Types.StringType.get()), + Types.NestedField.required(5, "name5", Types.StringType.get()), + Types.NestedField.required(6, "name6", Types.StringType.get()), + Types.NestedField.required(7, "name7", Types.StringType.get()), + Types.NestedField.required(8, "name8", Types.StringType.get()), + Types.NestedField.required(9, "name9", Types.StringType.get())); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + + private MapRangePartitioner partitioner; + private RowData[] rows; + + @Setup + public void setupBenchmark() { + NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); + Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); + weights.forEach( + (id, weight) -> { + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, id); + mapStatistics.put(sortKey, weight); + }); + + MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + this.partitioner = + new MapRangePartitioner( + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + + List keys = Lists.newArrayList(weights.keySet().iterator()); + long[] weightsCDF = new long[keys.size()]; + long totalWeight = 0; + for (int i = 0; i < keys.size(); ++i) { + totalWeight += weights.get(keys.get(i)); + weightsCDF[i] = totalWeight; + } + + // pre-calculate the samples for benchmark run + this.rows = new GenericRowData[SAMPLE_SIZE]; + for (int i = 0; i < SAMPLE_SIZE; ++i) { + long weight = ThreadLocalRandom.current().nextLong(totalWeight); + int index = binarySearchIndex(weightsCDF, weight); + rows[i] = + GenericRowData.of( + keys.get(index), + randomString("name2-"), + randomString("name3-"), + randomString("name4-"), + randomString("name5-"), + randomString("name6-"), + randomString("name7-"), + randomString("name8-"), + randomString("name9-")); + } + } + + @TearDown + public void tearDownBenchmark() {} + + @Benchmark + @Threads(1) + public void testPartitionerLongTailDistribution(Blackhole blackhole) { + for (int i = 0; i < SAMPLE_SIZE; ++i) { + blackhole.consume(partitioner.partition(rows[i], 128)); + } + } + + private static String randomString(String prefix) { + int length = ThreadLocalRandom.current().nextInt(200); + byte[] buffer = new byte[length]; + + for (int i = 0; i < length; i += 1) { + buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); + } + + return prefix + new String(buffer); + } + + /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ + private static int binarySearchIndex(long[] weightsUDF, long target) { + Preconditions.checkArgument( + target < weightsUDF[weightsUDF.length - 1], + "weight is out of range: total weight = %s, search target = %s", + weightsUDF[weightsUDF.length - 1], + target); + int start = 0; + int end = weightsUDF.length - 1; + while (start < end) { + int mid = (start + end) / 2; + if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { + return mid; + } + + if (weightsUDF[mid] >= target) { + end = mid - 1; + } else if (weightsUDF[mid + 1] < target) { + start = mid + 1; + } + } + return start; + } + + /** Key is the id string and value is the weight in long value. */ + private static NavigableMap longTailDistribution( + long startingWeight, + int longTailStartingIndex, + int longTailLength, + long longTailBaseWeight, + double weightRandomJitterPercentage) { + + NavigableMap weights = Maps.newTreeMap(); + + // first part just decays the weight by half + long currentWeight = startingWeight; + for (int index = 0; index < longTailStartingIndex; ++index) { + double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); + long weight = (long) (currentWeight * (1.0 + jitter)); + weight = weight > 0 ? weight : 1; + weights.put(index, weight); + if (currentWeight > longTailBaseWeight) { + currentWeight = currentWeight / 2; + } + } + + // long tail part + for (int index = longTailStartingIndex; + index < longTailStartingIndex + longTailLength; + ++index) { + long longTailWeight = + (long) + (longTailBaseWeight + * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); + longTailWeight = longTailWeight > 0 ? longTailWeight : 1; + weights.put(index, longTailWeight); + } + + return weights; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java new file mode 100644 index 000000000000..18473bf4f190 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java @@ -0,0 +1,215 @@ +/* + * 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; + +import java.io.Serializable; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.hive.HiveCatalog; +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.Maps; +import org.apache.iceberg.rest.RESTCatalog; + +/** Serializable loader to load an Iceberg {@link Catalog}. */ +public interface CatalogLoader extends Serializable, Cloneable { + + /** + * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the + * {@link CatalogLoader} at flink sql client side or job manager side, and then serialize this + * catalog loader to task manager, finally deserialize it and create a new catalog at task manager + * side. + * + * @return a newly created {@link Catalog} + */ + Catalog loadCatalog(); + + /** Clone a CatalogLoader. */ + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + CatalogLoader clone(); + + static CatalogLoader hadoop( + String name, Configuration hadoopConf, Map properties) { + return new HadoopCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader hive(String name, Configuration hadoopConf, Map properties) { + return new HiveCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader rest(String name, Configuration hadoopConf, Map properties) { + return new RESTCatalogLoader(name, hadoopConf, properties); + } + + static CatalogLoader custom( + String name, Map properties, Configuration hadoopConf, String impl) { + return new CustomCatalogLoader(name, properties, hadoopConf, impl); + } + + class HadoopCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final String warehouseLocation; + private final Map properties; + + private HadoopCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new HadoopCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("warehouseLocation", warehouseLocation) + .toString(); + } + } + + class HiveCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final String uri; + private final String warehouse; + private final int clientPoolSize; + private final Map properties; + + private HiveCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.uri = properties.get(CatalogProperties.URI); + this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); + this.clientPoolSize = + properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) + ? Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) + : CatalogProperties.CLIENT_POOL_SIZE_DEFAULT; + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new HiveCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("uri", uri) + .add("warehouse", warehouse) + .add("clientPoolSize", clientPoolSize) + .toString(); + } + } + + class RESTCatalogLoader implements CatalogLoader { + private final String catalogName; + private final SerializableConfiguration hadoopConf; + private final Map properties; + + private RESTCatalogLoader( + String catalogName, Configuration conf, Map properties) { + this.catalogName = catalogName; + this.hadoopConf = new SerializableConfiguration(conf); + this.properties = Maps.newHashMap(properties); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog( + RESTCatalog.class.getName(), catalogName, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new RESTCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("catalogName", catalogName) + .add("properties", properties) + .toString(); + } + } + + class CustomCatalogLoader implements CatalogLoader { + + private final SerializableConfiguration hadoopConf; + private final Map properties; + private final String name; + private final String impl; + + private CustomCatalogLoader( + String name, Map properties, Configuration conf, String impl) { + this.hadoopConf = new SerializableConfiguration(conf); + this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization + this.name = name; + this.impl = + Preconditions.checkNotNull( + impl, "Cannot initialize custom Catalog, impl class name is null"); + } + + @Override + public Catalog loadCatalog() { + return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get()); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public CatalogLoader clone() { + return new CustomCatalogLoader(name, properties, new Configuration(hadoopConf.get()), impl); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("name", name).add("impl", impl).toString(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java new file mode 100644 index 000000000000..86295d78cc13 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java @@ -0,0 +1,833 @@ +/* + * 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; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.AbstractCatalog; +import org.apache.flink.table.catalog.CatalogBaseTable; +import org.apache.flink.table.catalog.CatalogDatabase; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.CatalogFunction; +import org.apache.flink.table.catalog.CatalogPartition; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.CatalogTableImpl; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.exceptions.CatalogException; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; +import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; +import org.apache.flink.table.catalog.stats.CatalogTableStatistics; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.factories.Factory; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.CachingCatalog; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +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.AlreadyExistsException; +import org.apache.iceberg.exceptions.NamespaceNotEmptyException; +import org.apache.iceberg.exceptions.NoSuchNamespaceException; +import org.apache.iceberg.flink.util.FlinkAlterTableUtil; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Splitter; +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.relocated.com.google.common.collect.Sets; + +/** + * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}. + * + *

    The mapping between Flink database and Iceberg namespace: Supplying a base namespace for a + * given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the + * first level in the catalog configuration and the second level would be exposed as Flink + * databases. + * + *

    The Iceberg table manages its partitions by itself. The partition of the Iceberg table is + * independent of the partition of Flink. + */ +public class FlinkCatalog extends AbstractCatalog { + private final CatalogLoader catalogLoader; + private final Catalog icebergCatalog; + private final Namespace baseNamespace; + private final SupportsNamespaces asNamespaceCatalog; + private final Closeable closeable; + private final boolean cacheEnabled; + + public FlinkCatalog( + String catalogName, + String defaultDatabase, + Namespace baseNamespace, + CatalogLoader catalogLoader, + boolean cacheEnabled, + long cacheExpirationIntervalMs) { + super(catalogName, defaultDatabase); + this.catalogLoader = catalogLoader; + this.baseNamespace = baseNamespace; + this.cacheEnabled = cacheEnabled; + + Catalog originalCatalog = catalogLoader.loadCatalog(); + icebergCatalog = + cacheEnabled + ? CachingCatalog.wrap(originalCatalog, cacheExpirationIntervalMs) + : originalCatalog; + asNamespaceCatalog = + originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null; + closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null; + + FlinkEnvironmentContext.init(); + } + + @Override + public void open() throws CatalogException {} + + @Override + public void close() throws CatalogException { + if (closeable != null) { + try { + closeable.close(); + } catch (IOException e) { + throw new CatalogException(e); + } + } + } + + public Catalog catalog() { + return icebergCatalog; + } + + /** Append a new level to the base namespace */ + private static Namespace appendLevel(Namespace baseNamespace, String newLevel) { + String[] namespace = new String[baseNamespace.levels().length + 1]; + System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length); + namespace[baseNamespace.levels().length] = newLevel; + return Namespace.of(namespace); + } + + TableIdentifier toIdentifier(ObjectPath path) { + String objectName = path.getObjectName(); + List tableName = Splitter.on('$').splitToList(objectName); + + if (tableName.size() == 1) { + return TableIdentifier.of( + appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName()); + } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) { + return TableIdentifier.of( + appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)), + tableName.get(1)); + } else { + throw new IllegalArgumentException("Illegal table name:" + objectName); + } + } + + @Override + public List listDatabases() throws CatalogException { + if (asNamespaceCatalog == null) { + return Collections.singletonList(getDefaultDatabase()); + } + + return asNamespaceCatalog.listNamespaces(baseNamespace).stream() + .map(n -> n.level(n.levels().length - 1)) + .collect(Collectors.toList()); + } + + @Override + public CatalogDatabase getDatabase(String databaseName) + throws DatabaseNotExistException, CatalogException { + if (asNamespaceCatalog == null) { + if (!getDefaultDatabase().equals(databaseName)) { + throw new DatabaseNotExistException(getName(), databaseName); + } else { + return new CatalogDatabaseImpl(Maps.newHashMap(), ""); + } + } else { + try { + Map metadata = + Maps.newHashMap( + asNamespaceCatalog.loadNamespaceMetadata(appendLevel(baseNamespace, databaseName))); + String comment = metadata.remove("comment"); + return new CatalogDatabaseImpl(metadata, comment); + } catch (NoSuchNamespaceException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } + } + } + + @Override + public boolean databaseExists(String databaseName) throws CatalogException { + try { + getDatabase(databaseName); + return true; + } catch (DatabaseNotExistException ignore) { + return false; + } + } + + @Override + public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + createDatabase( + name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists); + } + + private void createDatabase( + String databaseName, Map metadata, boolean ignoreIfExists) + throws DatabaseAlreadyExistException, CatalogException { + if (asNamespaceCatalog != null) { + try { + asNamespaceCatalog.createNamespace(appendLevel(baseNamespace, databaseName), metadata); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new DatabaseAlreadyExistException(getName(), databaseName, e); + } + } + } else { + throw new UnsupportedOperationException( + "Namespaces are not supported by catalog: " + getName()); + } + } + + private Map mergeComment(Map metadata, String comment) { + Map ret = Maps.newHashMap(metadata); + if (metadata.containsKey("comment")) { + throw new CatalogException("Database properties should not contain key: 'comment'."); + } + + if (!StringUtils.isNullOrWhitespaceOnly(comment)) { + ret.put("comment", comment); + } + return ret; + } + + @Override + public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) + throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { + if (asNamespaceCatalog != null) { + try { + boolean success = asNamespaceCatalog.dropNamespace(appendLevel(baseNamespace, name)); + if (!success && !ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } catch (NoSuchNamespaceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name, e); + } + } catch (NamespaceNotEmptyException e) { + throw new DatabaseNotEmptyException(getName(), name, e); + } + } else { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + @Override + public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) + throws DatabaseNotExistException, CatalogException { + if (asNamespaceCatalog != null) { + Namespace namespace = appendLevel(baseNamespace, name); + Map updates = Maps.newHashMap(); + Set removals = Sets.newHashSet(); + + try { + Map oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace); + Map newProperties = + mergeComment(newDatabase.getProperties(), newDatabase.getComment()); + + for (String key : oldProperties.keySet()) { + if (!newProperties.containsKey(key)) { + removals.add(key); + } + } + + for (Map.Entry entry : newProperties.entrySet()) { + if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) { + updates.put(entry.getKey(), entry.getValue()); + } + } + + if (!updates.isEmpty()) { + asNamespaceCatalog.setProperties(namespace, updates); + } + + if (!removals.isEmpty()) { + asNamespaceCatalog.removeProperties(namespace, removals); + } + + } catch (NoSuchNamespaceException e) { + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name, e); + } + } + } else { + if (getDefaultDatabase().equals(name)) { + throw new CatalogException( + "Can not alter the default database when the iceberg catalog doesn't support namespaces."); + } + if (!ignoreIfNotExists) { + throw new DatabaseNotExistException(getName(), name); + } + } + } + + @Override + public List listTables(String databaseName) + throws DatabaseNotExistException, CatalogException { + try { + return icebergCatalog.listTables(appendLevel(baseNamespace, databaseName)).stream() + .map(TableIdentifier::name) + .collect(Collectors.toList()); + } catch (NoSuchNamespaceException e) { + throw new DatabaseNotExistException(getName(), databaseName, e); + } + } + + @Override + public CatalogTable getTable(ObjectPath tablePath) + throws TableNotExistException, CatalogException { + Table table = loadIcebergTable(tablePath); + return toCatalogTable(table); + } + + private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { + try { + Table table = icebergCatalog.loadTable(toIdentifier(tablePath)); + if (cacheEnabled) { + table.refresh(); + } + + return table; + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + throw new TableNotExistException(getName(), tablePath, e); + } + } + + @Override + public boolean tableExists(ObjectPath tablePath) throws CatalogException { + return icebergCatalog.tableExists(toIdentifier(tablePath)); + } + + @Override + public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + try { + icebergCatalog.dropTable(toIdentifier(tablePath)); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } + } + } + + @Override + public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) + throws TableNotExistException, TableAlreadyExistException, CatalogException { + try { + icebergCatalog.renameTable( + toIdentifier(tablePath), + toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName))); + } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { + if (!ignoreIfNotExists) { + throw new TableNotExistException(getName(), tablePath, e); + } + } catch (AlreadyExistsException e) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } + + @Override + public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + if (Objects.equals( + table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { + throw new IllegalArgumentException( + "Cannot create the table with 'connector'='iceberg' table property in " + + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + + "create table without 'connector'='iceberg' related properties in an iceberg table."); + } + Preconditions.checkArgument(table instanceof ResolvedCatalogTable, "table should be resolved"); + createIcebergTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists); + } + + void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolean ignoreIfExists) + throws CatalogException, TableAlreadyExistException { + validateFlinkTable(table); + + Schema icebergSchema = FlinkSchemaUtil.convert(table.getResolvedSchema()); + PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); + ImmutableMap.Builder properties = ImmutableMap.builder(); + String location = null; + for (Map.Entry entry : table.getOptions().entrySet()) { + if ("location".equalsIgnoreCase(entry.getKey())) { + location = entry.getValue(); + } else { + properties.put(entry.getKey(), entry.getValue()); + } + } + + try { + icebergCatalog.createTable( + toIdentifier(tablePath), icebergSchema, spec, location, properties.build()); + } catch (AlreadyExistsException e) { + if (!ignoreIfExists) { + throw new TableAlreadyExistException(getName(), tablePath, e); + } + } + } + + private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) { + TableSchema ts1 = ct1.getSchema(); + TableSchema ts2 = ct2.getSchema(); + boolean equalsPrimary = false; + + if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { + equalsPrimary = + Objects.equals(ts1.getPrimaryKey().get().getType(), ts2.getPrimaryKey().get().getType()) + && Objects.equals( + ts1.getPrimaryKey().get().getColumns(), ts2.getPrimaryKey().get().getColumns()); + } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { + equalsPrimary = true; + } + + if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) + && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) + && equalsPrimary)) { + throw new UnsupportedOperationException( + "Altering schema is not supported in the old alterTable API. " + + "To alter schema, use the other alterTable API and provide a list of TableChange's."); + } + + validateTablePartition(ct1, ct2); + } + + private static void validateTablePartition(CatalogTable ct1, CatalogTable ct2) { + if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { + throw new UnsupportedOperationException("Altering partition keys is not supported yet."); + } + } + + /** + * This alterTable API only supports altering table properties. + * + *

    Support for adding/removing/renaming columns cannot be done by comparing CatalogTable + * instances, unless the Flink schema contains Iceberg column IDs. + * + *

    To alter columns, use the other alterTable API and provide a list of TableChange's. + * + * @param tablePath path of the table or view to be modified + * @param newTable the new table definition + * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if set + * to false, throw an exception, if set to true, do nothing. + * @throws CatalogException in case of any runtime exception + * @throws TableNotExistException if the table does not exist + */ + @Override + public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) + throws CatalogException, TableNotExistException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + CatalogTable table = toCatalogTable(icebergTable); + validateTableSchemaAndPartition(table, (CatalogTable) newTable); + + Map oldProperties = table.getOptions(); + Map setProperties = Maps.newHashMap(); + + String setLocation = null; + String setSnapshotId = null; + String pickSnapshotId = null; + + for (Map.Entry entry : newTable.getOptions().entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + + if (Objects.equals(value, oldProperties.get(key))) { + continue; + } + + if ("location".equalsIgnoreCase(key)) { + setLocation = value; + } else if ("current-snapshot-id".equalsIgnoreCase(key)) { + setSnapshotId = value; + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) { + pickSnapshotId = value; + } else { + setProperties.put(key, value); + } + } + + oldProperties + .keySet() + .forEach( + k -> { + if (!newTable.getOptions().containsKey(k)) { + setProperties.put(k, null); + } + }); + + FlinkAlterTableUtil.commitChanges( + icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); + } + + @Override + public void alterTable( + ObjectPath tablePath, + CatalogBaseTable newTable, + List tableChanges, + boolean ignoreIfNotExists) + throws TableNotExistException, CatalogException { + validateFlinkTable(newTable); + + Table icebergTable; + try { + icebergTable = loadIcebergTable(tablePath); + } catch (TableNotExistException e) { + if (!ignoreIfNotExists) { + throw e; + } else { + return; + } + } + + // Does not support altering partition yet. + validateTablePartition(toCatalogTable(icebergTable), (CatalogTable) newTable); + + String setLocation = null; + String setSnapshotId = null; + String cherrypickSnapshotId = null; + + List propertyChanges = Lists.newArrayList(); + List schemaChanges = Lists.newArrayList(); + for (TableChange change : tableChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption set = (TableChange.SetOption) change; + + if ("location".equalsIgnoreCase(set.getKey())) { + setLocation = set.getValue(); + } else if ("current-snapshot-id".equalsIgnoreCase(set.getKey())) { + setSnapshotId = set.getValue(); + } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.getKey())) { + cherrypickSnapshotId = set.getValue(); + } else { + propertyChanges.add(change); + } + } else if (change instanceof TableChange.ResetOption) { + propertyChanges.add(change); + } else { + schemaChanges.add(change); + } + } + + FlinkAlterTableUtil.commitChanges( + icebergTable, + setLocation, + setSnapshotId, + cherrypickSnapshotId, + schemaChanges, + propertyChanges); + } + + private static void validateFlinkTable(CatalogBaseTable table) { + Preconditions.checkArgument( + table instanceof CatalogTable, "The Table should be a CatalogTable."); + + TableSchema schema = table.getSchema(); + schema + .getTableColumns() + .forEach( + column -> { + if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) { + throw new UnsupportedOperationException( + "Creating table with computed columns is not supported yet."); + } + }); + + if (!schema.getWatermarkSpecs().isEmpty()) { + throw new UnsupportedOperationException( + "Creating table with watermark specs is not supported yet."); + } + } + + private static PartitionSpec toPartitionSpec(List partitionKeys, Schema icebergSchema) { + PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema); + partitionKeys.forEach(builder::identity); + return builder.build(); + } + + private static List toPartitionKeys(PartitionSpec spec, Schema icebergSchema) { + ImmutableList.Builder partitionKeysBuilder = ImmutableList.builder(); + for (PartitionField field : spec.fields()) { + if (field.transform().isIdentity()) { + partitionKeysBuilder.add(icebergSchema.findColumnName(field.sourceId())); + } else { + // Not created by Flink SQL. + // For compatibility with iceberg tables, return empty. + // TODO modify this after Flink support partition transform. + return Collections.emptyList(); + } + } + return partitionKeysBuilder.build(); + } + + static CatalogTable toCatalogTable(Table table) { + TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); + List partitionKeys = toPartitionKeys(table.spec(), table.schema()); + + // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer + // may use + // CatalogTableImpl to copy a new catalog table. + // Let's re-loading table from Iceberg catalog when creating source/sink operators. + // Iceberg does not have Table comment, so pass a null (Default comment value in Flink). + return new CatalogTableImpl(schema, partitionKeys, table.properties(), null); + } + + @Override + public Optional getFactory() { + return Optional.of(new FlinkDynamicTableFactory(this)); + } + + CatalogLoader getCatalogLoader() { + return catalogLoader; + } + + // ------------------------------ Unsupported methods + // --------------------------------------------- + + @Override + public List listViews(String databaseName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void createPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition partition, + boolean ignoreIfExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropPartition( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartition( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogPartition newPartition, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listFunctions(String dbName) throws CatalogException { + return Collections.emptyList(); + } + + @Override + public CatalogFunction getFunction(ObjectPath functionPath) + throws FunctionNotExistException, CatalogException { + throw new FunctionNotExistException(getName(), functionPath); + } + + @Override + public boolean functionExists(ObjectPath functionPath) throws CatalogException { + return false; + } + + @Override + public void createFunction( + ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterFunction( + ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableStatistics( + ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterTableColumnStatistics( + ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogTableStatistics partitionStatistics, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public void alterPartitionColumnStatistics( + ObjectPath tablePath, + CatalogPartitionSpec partitionSpec, + CatalogColumnStatistics columnStatistics, + boolean ignoreIfNotExists) + throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitions(ObjectPath tablePath) + throws TableNotExistException, TableNotPartitionedException, CatalogException { + Table table = loadIcebergTable(tablePath); + + if (table.spec().isUnpartitioned()) { + throw new TableNotPartitionedException(icebergCatalog.name(), tablePath); + } + + Set set = Sets.newHashSet(); + try (CloseableIterable tasks = table.newScan().planFiles()) { + for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) { + Map map = Maps.newHashMap(); + StructLike structLike = dataFile.partition(); + PartitionSpec spec = table.specs().get(dataFile.specId()); + for (int i = 0; i < structLike.size(); i++) { + map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class))); + } + set.add(new CatalogPartitionSpec(map)); + } + } catch (IOException e) { + throw new CatalogException( + String.format("Failed to list partitions of table %s", tablePath), e); + } + + return Lists.newArrayList(set); + } + + @Override + public List listPartitions( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + throw new UnsupportedOperationException(); + } + + @Override + public List listPartitionsByFilter( + ObjectPath tablePath, List filters) throws CatalogException { + throw new UnsupportedOperationException(); + } + + // After partition pruning and filter push down, the statistics have become very inaccurate, so + // the statistics from + // here are of little significance. + // Flink will support something like SupportsReportStatistics in future. + + @Override + public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) + throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } + + @Override + public CatalogTableStatistics getPartitionStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogTableStatistics.UNKNOWN; + } + + @Override + public CatalogColumnStatistics getPartitionColumnStatistics( + ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { + return CatalogColumnStatistics.UNKNOWN; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java new file mode 100644 index 000000000000..fe4008a13ce5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -0,0 +1,213 @@ +/* + * 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; + +import java.net.URL; +import java.nio.file.Files; +import java.nio.file.Paths; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import org.apache.flink.configuration.GlobalConfiguration; +import org.apache.flink.runtime.util.HadoopUtils; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.factories.CatalogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; + +/** + * A Flink Catalog factory implementation that creates {@link FlinkCatalog}. + * + *

    This supports the following catalog configuration options: + * + *

      + *
    • type - Flink catalog factory key, should be "iceberg" + *
    • catalog-type - iceberg catalog type, "hive", "hadoop" or "rest" + *
    • uri - the Hive Metastore URI (Hive catalog only) + *
    • clients - the Hive Client Pool Size (Hive catalog only) + *
    • warehouse - the warehouse path (Hadoop catalog only) + *
    • default-database - a database name to use as the default + *
    • base-namespace - a base namespace as the prefix for all databases (Hadoop + * catalog only) + *
    • cache-enabled - whether to enable catalog cache + *
    + * + *

    To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override + * {@link #createCatalogLoader(String, Map, Configuration)}. + */ +public class FlinkCatalogFactory implements CatalogFactory { + + // Can not just use "type", it conflicts with CATALOG_TYPE. + public static final String ICEBERG_CATALOG_TYPE = "catalog-type"; + public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop"; + public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive"; + public static final String ICEBERG_CATALOG_TYPE_REST = "rest"; + + public static final String HIVE_CONF_DIR = "hive-conf-dir"; + public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; + public static final String DEFAULT_DATABASE = "default-database"; + public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String DEFAULT_CATALOG_NAME = "default_catalog"; + public static final String BASE_NAMESPACE = "base-namespace"; + public static final String TYPE = "type"; + public static final String PROPERTY_VERSION = "property-version"; + + /** + * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink + * catalog adapter. + * + * @param name Flink's catalog name + * @param properties Flink's catalog properties + * @param hadoopConf Hadoop configuration for catalog + * @return an Iceberg catalog loader + */ + static CatalogLoader createCatalogLoader( + String name, Map properties, Configuration hadoopConf) { + String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); + if (catalogImpl != null) { + String catalogType = properties.get(ICEBERG_CATALOG_TYPE); + Preconditions.checkArgument( + catalogType == null, + "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s", + name, + catalogType, + catalogImpl); + return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl); + } + + String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE); + switch (catalogType.toLowerCase(Locale.ENGLISH)) { + case ICEBERG_CATALOG_TYPE_HIVE: + // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in + // that case it will + // fallback to parse those values from hadoop configuration which is loaded from classpath. + String hiveConfDir = properties.get(HIVE_CONF_DIR); + String hadoopConfDir = properties.get(HADOOP_CONF_DIR); + Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); + return CatalogLoader.hive(name, newHadoopConf, properties); + + case ICEBERG_CATALOG_TYPE_HADOOP: + return CatalogLoader.hadoop(name, hadoopConf, properties); + + case ICEBERG_CATALOG_TYPE_REST: + return CatalogLoader.rest(name, hadoopConf, properties); + + default: + throw new UnsupportedOperationException( + "Unknown catalog-type: " + catalogType + " (Must be 'hive', 'hadoop' or 'rest')"); + } + } + + @Override + public Map requiredContext() { + Map context = Maps.newHashMap(); + context.put(TYPE, "iceberg"); + context.put(PROPERTY_VERSION, "1"); + return context; + } + + @Override + public List supportedProperties() { + return ImmutableList.of("*"); + } + + @Override + public Catalog createCatalog(String name, Map properties) { + return createCatalog(name, properties, clusterHadoopConf()); + } + + protected Catalog createCatalog( + String name, Map properties, Configuration hadoopConf) { + CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); + String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); + + Namespace baseNamespace = Namespace.empty(); + if (properties.containsKey(BASE_NAMESPACE)) { + baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\.")); + } + + boolean cacheEnabled = + PropertyUtil.propertyAsBoolean( + properties, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); + + long cacheExpirationIntervalMs = + PropertyUtil.propertyAsLong( + properties, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); + Preconditions.checkArgument( + cacheExpirationIntervalMs != 0, + "%s is not allowed to be 0.", + CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS); + + return new FlinkCatalog( + name, + defaultDatabase, + baseNamespace, + catalogLoader, + cacheEnabled, + cacheExpirationIntervalMs); + } + + private static Configuration mergeHiveConf( + Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { + Configuration newConf = new Configuration(hadoopConf); + if (!Strings.isNullOrEmpty(hiveConfDir)) { + Preconditions.checkState( + Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), + "There should be a hive-site.xml file under the directory %s", + hiveConfDir); + newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); + } else { + // If don't provide the hive-site.xml path explicitly, it will try to load resource from + // classpath. If still + // couldn't load the configuration file, then it will throw exception in HiveCatalog. + URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); + if (configFile != null) { + newConf.addResource(configFile); + } + } + + if (!Strings.isNullOrEmpty(hadoopConfDir)) { + Preconditions.checkState( + Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), + "Failed to load Hadoop configuration: missing %s", + Paths.get(hadoopConfDir, "hdfs-site.xml")); + newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); + Preconditions.checkState( + Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), + "Failed to load Hadoop configuration: missing %s", + Paths.get(hadoopConfDir, "core-site.xml")); + newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); + } + + return newConf; + } + + public static Configuration clusterHadoopConf() { + return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java new file mode 100644 index 000000000000..7167859e600c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -0,0 +1,261 @@ +/* + * 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; + +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.function.Function; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class FlinkConfParser { + + private final Map tableProperties; + private final Map options; + private final ReadableConfig readableConfig; + + FlinkConfParser(Table table, Map options, ReadableConfig readableConfig) { + this.tableProperties = table.properties(); + this.options = options; + this.readableConfig = readableConfig; + } + + public BooleanConfParser booleanConf() { + return new BooleanConfParser(); + } + + public IntConfParser intConf() { + return new IntConfParser(); + } + + public LongConfParser longConf() { + return new LongConfParser(); + } + + public > EnumConfParser enumConfParser(Class enumClass) { + return new EnumConfParser<>(enumClass); + } + + public StringConfParser stringConf() { + return new StringConfParser(); + } + + public DurationConfParser durationConf() { + return new DurationConfParser(); + } + + class BooleanConfParser extends ConfParser { + private Boolean defaultValue; + + @Override + protected BooleanConfParser self() { + return this; + } + + public BooleanConfParser defaultValue(boolean value) { + this.defaultValue = value; + return self(); + } + + public BooleanConfParser defaultValue(String value) { + this.defaultValue = Boolean.parseBoolean(value); + return self(); + } + + public boolean parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Boolean::parseBoolean, defaultValue); + } + } + + class IntConfParser extends ConfParser { + private Integer defaultValue; + + @Override + protected IntConfParser self() { + return this; + } + + public IntConfParser defaultValue(int value) { + this.defaultValue = value; + return self(); + } + + public int parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Integer::parseInt, defaultValue); + } + + public Integer parseOptional() { + return parse(Integer::parseInt, null); + } + } + + class LongConfParser extends ConfParser { + private Long defaultValue; + + @Override + protected LongConfParser self() { + return this; + } + + public LongConfParser defaultValue(long value) { + this.defaultValue = value; + return self(); + } + + public long parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Long::parseLong, defaultValue); + } + + public Long parseOptional() { + return parse(Long::parseLong, null); + } + } + + class StringConfParser extends ConfParser { + private String defaultValue; + + @Override + protected StringConfParser self() { + return this; + } + + public StringConfParser defaultValue(String value) { + this.defaultValue = value; + return self(); + } + + public String parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Function.identity(), defaultValue); + } + + public String parseOptional() { + return parse(Function.identity(), null); + } + } + + class EnumConfParser> extends ConfParser, E> { + private E defaultValue; + private final Class enumClass; + + EnumConfParser(Class enumClass) { + this.enumClass = enumClass; + } + + @Override + protected EnumConfParser self() { + return this; + } + + public EnumConfParser defaultValue(E value) { + this.defaultValue = value; + return self(); + } + + public E parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(s -> Enum.valueOf(enumClass, s), defaultValue); + } + + public E parseOptional() { + return parse(s -> Enum.valueOf(enumClass, s), null); + } + } + + class DurationConfParser extends ConfParser { + private Duration defaultValue; + + @Override + protected DurationConfParser self() { + return this; + } + + public DurationConfParser defaultValue(Duration value) { + this.defaultValue = value; + return self(); + } + + public Duration parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(TimeUtils::parseDuration, defaultValue); + } + + public Duration parseOptional() { + return parse(TimeUtils::parseDuration, null); + } + } + + abstract class ConfParser { + private final List optionNames = Lists.newArrayList(); + private String tablePropertyName; + private ConfigOption configOption; + + protected abstract ThisT self(); + + public ThisT option(String name) { + this.optionNames.add(name); + return self(); + } + + public ThisT flinkConfig(ConfigOption newConfigOption) { + this.configOption = newConfigOption; + return self(); + } + + public ThisT tableProperty(String name) { + this.tablePropertyName = name; + return self(); + } + + protected T parse(Function conversion, T defaultValue) { + if (!optionNames.isEmpty()) { + for (String optionName : optionNames) { + String optionValue = options.get(optionName); + if (optionValue != null) { + return conversion.apply(optionValue); + } + } + } + + if (configOption != null) { + T propertyValue = readableConfig.get(configOption); + if (propertyValue != null) { + return propertyValue; + } + } + + if (tablePropertyName != null) { + String propertyValue = tableProperties.get(tablePropertyName); + if (propertyValue != null) { + return conversion.apply(propertyValue); + } + } + + return defaultValue; + } + } +} 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 new file mode 100644 index 000000000000..7c7afd24ed8e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.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.flink; + +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.description.Description; +import org.apache.flink.configuration.description.TextElement; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.flink.source.assigner.SplitAssignerType; +import org.apache.iceberg.util.ThreadPools; + +/** + * When constructing Flink Iceberg source via Java API, configs can be set in {@link Configuration} + * passed to source builder. E.g. + * + *

    + *   configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    + *   FlinkSource.forRowData()
    + *       .flinkConf(configuration)
    + *       ...
    + * 
    + * + *

    When using Flink SQL/table API, connector options can be set in Flink's {@link + * TableEnvironment}. + * + *

    + *   TableEnvironment tEnv = createTableEnv();
    + *   tEnv.getConfig()
    + *        .getConfiguration()
    + *        .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    + * 
    + */ +public class FlinkConfigOptions { + + private FlinkConfigOptions() {} + + public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = + ConfigOptions.key("table.exec.iceberg.infer-source-parallelism") + .booleanType() + .defaultValue(true) + .withDescription( + "If is false, parallelism of source are set by config.\n" + + "If is true, source parallelism is inferred according to splits number.\n"); + + public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX = + ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max") + .intType() + .defaultValue(100) + .withDescription("Sets max infer parallelism for source operator."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = + ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") + .booleanType() + .noDefaultValue() + .withDescription( + "Expose split host information to use Flink's locality aware split assigner."); + + public static final ConfigOption SOURCE_READER_FETCH_BATCH_RECORD_COUNT = + ConfigOptions.key("table.exec.iceberg.fetch-batch-record-count") + .intType() + .defaultValue(2048) + .withDescription("The target number of records for Iceberg reader fetch batch."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE = + ConfigOptions.key("table.exec.iceberg.worker-pool-size") + .intType() + .defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE) + .withDescription("The size of workers pool used to plan or scan manifests."); + + public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = + ConfigOptions.key("table.exec.iceberg.use-flip27-source") + .booleanType() + .defaultValue(false) + .withDescription("Use the FLIP-27 based Iceberg source implementation."); + + public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = + ConfigOptions.key("table.exec.iceberg.split-assigner-type") + .enumType(SplitAssignerType.class) + .defaultValue(SplitAssignerType.SIMPLE) + .withDescription( + Description.builder() + .text("Split assigner type that determine how splits are assigned to readers.") + .linebreak() + .list( + TextElement.text( + SplitAssignerType.SIMPLE + + ": simple assigner that doesn't provide any guarantee on order or locality.")) + .build()); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java new file mode 100644 index 000000000000..b7f1be4b93fb --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import java.util.Map; +import java.util.Set; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.CatalogDatabaseImpl; +import org.apache.flink.table.catalog.ObjectIdentifier; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.ResolvedCatalogTable; +import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; +import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.factories.DynamicTableSinkFactory; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.flink.source.IcebergTableSource; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; + +public class FlinkDynamicTableFactory + implements DynamicTableSinkFactory, DynamicTableSourceFactory { + static final String FACTORY_IDENTIFIER = "iceberg"; + + private static final ConfigOption CATALOG_NAME = + ConfigOptions.key("catalog-name") + .stringType() + .noDefaultValue() + .withDescription("Catalog name"); + + private static final ConfigOption CATALOG_TYPE = + ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) + .stringType() + .noDefaultValue() + .withDescription("Catalog type, the optional types are: custom, hadoop, hive."); + + private static final ConfigOption CATALOG_DATABASE = + ConfigOptions.key("catalog-database") + .stringType() + .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) + .withDescription("Database name managed in the iceberg catalog."); + + private static final ConfigOption CATALOG_TABLE = + ConfigOptions.key("catalog-table") + .stringType() + .noDefaultValue() + .withDescription("Table name managed in the underlying iceberg catalog and database."); + + private final FlinkCatalog catalog; + + public FlinkDynamicTableFactory() { + this.catalog = null; + } + + public FlinkDynamicTableFactory(FlinkCatalog catalog) { + this.catalog = catalog; + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map tableProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = + createTableLoader( + resolvedCatalogTable, + tableProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } + + return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); + } + + @Override + public DynamicTableSink createDynamicTableSink(Context context) { + ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); + ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); + Map writeProps = resolvedCatalogTable.getOptions(); + TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); + + TableLoader tableLoader; + if (catalog != null) { + tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); + } else { + tableLoader = + createTableLoader( + resolvedCatalogTable, + writeProps, + objectIdentifier.getDatabaseName(), + objectIdentifier.getObjectName()); + } + + return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); + } + + @Override + public Set> requiredOptions() { + Set> options = Sets.newHashSet(); + options.add(CATALOG_TYPE); + options.add(CATALOG_NAME); + return options; + } + + @Override + public Set> optionalOptions() { + Set> options = Sets.newHashSet(); + options.add(CATALOG_DATABASE); + options.add(CATALOG_TABLE); + return options; + } + + @Override + public String factoryIdentifier() { + return FACTORY_IDENTIFIER; + } + + private static TableLoader createTableLoader( + ResolvedCatalogTable resolvedCatalogTable, + Map tableProps, + String databaseName, + String tableName) { + Configuration flinkConf = new Configuration(); + tableProps.forEach(flinkConf::setString); + + String catalogName = flinkConf.getString(CATALOG_NAME); + Preconditions.checkNotNull( + catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); + + String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); + Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); + + String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); + Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); + + org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); + FlinkCatalogFactory factory = new FlinkCatalogFactory(); + FlinkCatalog flinkCatalog = + (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); + ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); + + // Create database if not exists in the external catalog. + if (!flinkCatalog.databaseExists(catalogDatabase)) { + try { + flinkCatalog.createDatabase( + catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true); + } catch (DatabaseAlreadyExistException e) { + throw new AlreadyExistsException( + e, + "Database %s already exists in the iceberg catalog %s.", + catalogName, + catalogDatabase); + } + } + + // Create table if not exists in the external catalog. + if (!flinkCatalog.tableExists(objectPath)) { + try { + flinkCatalog.createIcebergTable(objectPath, resolvedCatalogTable, true); + } catch (TableAlreadyExistException e) { + throw new AlreadyExistsException( + e, + "Table %s already exists in the database %s and catalog %s", + catalogTable, + catalogDatabase, + catalogName); + } + } + + return TableLoader.fromCatalog( + flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); + } + + private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { + Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); + return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java new file mode 100644 index 000000000000..f35bb577fbba --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java @@ -0,0 +1,31 @@ +/* + * 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; + +import org.apache.iceberg.EnvironmentContext; +import org.apache.iceberg.flink.util.FlinkPackage; + +class FlinkEnvironmentContext { + private FlinkEnvironmentContext() {} + + public static void init() { + EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "flink"); + EnvironmentContext.put(EnvironmentContext.ENGINE_VERSION, FlinkPackage.version()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java new file mode 100644 index 000000000000..f2244d5137a1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java @@ -0,0 +1,266 @@ +/* + * 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; + +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.function.BiFunction; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.flink.table.functions.FunctionDefinition; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expression.Operation; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.NaNUtil; + +public class FlinkFilters { + private FlinkFilters() {} + + private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); + + private static final Map FILTERS = + ImmutableMap.builder() + .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) + .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) + .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) + .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) + .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) + .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) + .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) + .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) + .put(BuiltInFunctionDefinitions.AND, Operation.AND) + .put(BuiltInFunctionDefinitions.OR, Operation.OR) + .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) + .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) + .buildOrThrow(); + + /** + * Convert flink expression to iceberg expression. + * + *

    the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the + * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR + * GT_EQ), the IN will be converted to OR, so we do not add the conversion here + * + * @param flinkExpression the flink expression + * @return the iceberg expression + */ + public static Optional convert( + org.apache.flink.table.expressions.Expression flinkExpression) { + if (!(flinkExpression instanceof CallExpression)) { + return Optional.empty(); + } + + CallExpression call = (CallExpression) flinkExpression; + Operation op = FILTERS.get(call.getFunctionDefinition()); + if (op != null) { + switch (op) { + case IS_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(Expressions::isNull); + + case NOT_NULL: + return onlyChildAs(call, FieldReferenceExpression.class) + .map(FieldReferenceExpression::getName) + .map(Expressions::notNull); + + case LT: + return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call); + + case LT_EQ: + return convertFieldAndLiteral( + Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call); + + case GT: + return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call); + + case GT_EQ: + return convertFieldAndLiteral( + Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call); + + case EQ: + return convertFieldAndLiteral( + (ref, lit) -> { + if (NaNUtil.isNaN(lit)) { + return Expressions.isNaN(ref); + } else { + return Expressions.equal(ref, lit); + } + }, + call); + + case NOT_EQ: + return convertFieldAndLiteral( + (ref, lit) -> { + if (NaNUtil.isNaN(lit)) { + return Expressions.notNaN(ref); + } else { + return Expressions.notEqual(ref, lit); + } + }, + call); + + case NOT: + return onlyChildAs(call, CallExpression.class) + .flatMap(FlinkFilters::convert) + .map(Expressions::not); + + case AND: + return convertLogicExpression(Expressions::and, call); + + case OR: + return convertLogicExpression(Expressions::or, call); + + case STARTS_WITH: + return convertLike(call); + } + } + + return Optional.empty(); + } + + private static Optional onlyChildAs( + CallExpression call, Class expectedChildClass) { + List children = call.getResolvedChildren(); + if (children.size() != 1) { + return Optional.empty(); + } + + ResolvedExpression child = children.get(0); + if (!expectedChildClass.isInstance(child)) { + return Optional.empty(); + } + + return Optional.of(expectedChildClass.cast(child)); + } + + private static Optional convertLike(CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + org.apache.flink.table.expressions.Expression left = args.get(0); + org.apache.flink.table.expressions.Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + return convertLiteral((ValueLiteralExpression) right) + .flatMap( + lit -> { + if (lit instanceof String) { + String pattern = (String) lit; + Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); + // exclude special char of LIKE + // '_' is the wildcard of the SQL LIKE + if (!pattern.contains("_") && matcher.matches()) { + return Optional.of(Expressions.startsWith(name, matcher.group(1))); + } + } + + return Optional.empty(); + }); + } + + return Optional.empty(); + } + + private static Optional convertLogicExpression( + BiFunction function, CallExpression call) { + List args = call.getResolvedChildren(); + if (args == null || args.size() != 2) { + return Optional.empty(); + } + + Optional left = convert(args.get(0)); + Optional right = convert(args.get(1)); + if (left.isPresent() && right.isPresent()) { + return Optional.of(function.apply(left.get(), right.get())); + } + + return Optional.empty(); + } + + private static Optional convertLiteral(ValueLiteralExpression expression) { + Optional value = + expression.getValueAs( + expression.getOutputDataType().getLogicalType().getDefaultConversion()); + return value.map( + o -> { + if (o instanceof LocalDateTime) { + return DateTimeUtil.microsFromTimestamp((LocalDateTime) o); + } else if (o instanceof Instant) { + return DateTimeUtil.microsFromInstant((Instant) o); + } else if (o instanceof LocalTime) { + return DateTimeUtil.microsFromTime((LocalTime) o); + } else if (o instanceof LocalDate) { + return DateTimeUtil.daysFromDate((LocalDate) o); + } + + return o; + }); + } + + private static Optional convertFieldAndLiteral( + BiFunction expr, CallExpression call) { + return convertFieldAndLiteral(expr, expr, call); + } + + private static Optional convertFieldAndLiteral( + BiFunction convertLR, + BiFunction convertRL, + CallExpression call) { + List args = call.getResolvedChildren(); + if (args.size() != 2) { + return Optional.empty(); + } + + org.apache.flink.table.expressions.Expression left = args.get(0); + org.apache.flink.table.expressions.Expression right = args.get(1); + + if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { + String name = ((FieldReferenceExpression) left).getName(); + Optional lit = convertLiteral((ValueLiteralExpression) right); + if (lit.isPresent()) { + return Optional.of(convertLR.apply(name, lit.get())); + } + } else if (left instanceof ValueLiteralExpression + && right instanceof FieldReferenceExpression) { + Optional lit = convertLiteral((ValueLiteralExpression) left); + String name = ((FieldReferenceExpression) right).getName(); + if (lit.isPresent()) { + return Optional.of(convertRL.apply(name, lit.get())); + } + } + + return Optional.empty(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java new file mode 100644 index 000000000000..767d4497ac91 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.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.flink; + +import org.apache.iceberg.Schema; +import org.apache.iceberg.types.FixupTypes; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, + * which may not be correct. + */ +class FlinkFixupTypes extends FixupTypes { + + private FlinkFixupTypes(Schema referenceSchema) { + super(referenceSchema); + } + + static Schema fixup(Schema schema, Schema referenceSchema) { + return new Schema( + TypeUtil.visit(schema, new FlinkFixupTypes(referenceSchema)).asStructType().fields()); + } + + @Override + protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { + if (type instanceof Types.FixedType) { + int length = ((Types.FixedType) type).length(); + return source.typeId() == Type.TypeID.UUID && length == 16; + } + return false; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java new file mode 100644 index 000000000000..804a956ec9b9 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java @@ -0,0 +1,213 @@ +/* + * 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; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; + +public class FlinkReadConf { + + private final FlinkConfParser confParser; + + public FlinkReadConf( + Table table, Map readOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(table, readOptions, readableConfig); + } + + public Long snapshotId() { + return confParser.longConf().option(FlinkReadOptions.SNAPSHOT_ID.key()).parseOptional(); + } + + public String tag() { + return confParser.stringConf().option(FlinkReadOptions.TAG.key()).parseOptional(); + } + + public String startTag() { + return confParser.stringConf().option(FlinkReadOptions.START_TAG.key()).parseOptional(); + } + + public String endTag() { + return confParser.stringConf().option(FlinkReadOptions.END_TAG.key()).parseOptional(); + } + + public String branch() { + return confParser.stringConf().option(FlinkReadOptions.BRANCH.key()).parseOptional(); + } + + public boolean caseSensitive() { + return confParser + .booleanConf() + .option(FlinkReadOptions.CASE_SENSITIVE) + .flinkConfig(FlinkReadOptions.CASE_SENSITIVE_OPTION) + .defaultValue(FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue()) + .parse(); + } + + public Long asOfTimestamp() { + return confParser.longConf().option(FlinkReadOptions.AS_OF_TIMESTAMP.key()).parseOptional(); + } + + public StreamingStartingStrategy startingStrategy() { + return confParser + .enumConfParser(StreamingStartingStrategy.class) + .option(FlinkReadOptions.STARTING_STRATEGY) + .flinkConfig(FlinkReadOptions.STARTING_STRATEGY_OPTION) + .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .parse(); + } + + public Long startSnapshotTimestamp() { + return confParser + .longConf() + .option(FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key()) + .parseOptional(); + } + + public Long startSnapshotId() { + return confParser.longConf().option(FlinkReadOptions.START_SNAPSHOT_ID.key()).parseOptional(); + } + + public Long endSnapshotId() { + return confParser.longConf().option(FlinkReadOptions.END_SNAPSHOT_ID.key()).parseOptional(); + } + + public long splitSize() { + return confParser + .longConf() + .option(FlinkReadOptions.SPLIT_SIZE) + .flinkConfig(FlinkReadOptions.SPLIT_SIZE_OPTION) + .tableProperty(TableProperties.SPLIT_SIZE) + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) + .parse(); + } + + public int splitLookback() { + return confParser + .intConf() + .option(FlinkReadOptions.SPLIT_LOOKBACK) + .flinkConfig(FlinkReadOptions.SPLIT_LOOKBACK_OPTION) + .tableProperty(TableProperties.SPLIT_LOOKBACK) + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) + .parse(); + } + + public long splitFileOpenCost() { + return confParser + .longConf() + .option(FlinkReadOptions.SPLIT_FILE_OPEN_COST) + .flinkConfig(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION) + .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) + .parse(); + } + + public boolean streaming() { + return confParser + .booleanConf() + .option(FlinkReadOptions.STREAMING) + .flinkConfig(FlinkReadOptions.STREAMING_OPTION) + .defaultValue(FlinkReadOptions.STREAMING_OPTION.defaultValue()) + .parse(); + } + + public Duration monitorInterval() { + String duration = + confParser + .stringConf() + .option(FlinkReadOptions.MONITOR_INTERVAL) + .flinkConfig(FlinkReadOptions.MONITOR_INTERVAL_OPTION) + .defaultValue(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()) + .parse(); + + return TimeUtils.parseDuration(duration); + } + + public boolean includeColumnStats() { + return confParser + .booleanConf() + .option(FlinkReadOptions.INCLUDE_COLUMN_STATS) + .flinkConfig(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION) + .defaultValue(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue()) + .parse(); + } + + public int maxPlanningSnapshotCount() { + return confParser + .intConf() + .option(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT) + .flinkConfig(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION) + .defaultValue(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue()) + .parse(); + } + + public String nameMapping() { + return confParser.stringConf().option(TableProperties.DEFAULT_NAME_MAPPING).parseOptional(); + } + + public long limit() { + return confParser + .longConf() + .option(FlinkReadOptions.LIMIT) + .flinkConfig(FlinkReadOptions.LIMIT_OPTION) + .defaultValue(FlinkReadOptions.LIMIT_OPTION.defaultValue()) + .parse(); + } + + public int workerPoolSize() { + return confParser + .intConf() + .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) + .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) + .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) + .parse(); + } + + public int maxAllowedPlanningFailures() { + return confParser + .intConf() + .option(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES) + .flinkConfig(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION) + .defaultValue(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue()) + .parse(); + } + + public String watermarkColumn() { + return confParser + .stringConf() + .option(FlinkReadOptions.WATERMARK_COLUMN) + .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_OPTION) + .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue()) + .parseOptional(); + } + + public TimeUnit watermarkColumnTimeUnit() { + return confParser + .enumConfParser(TimeUnit.class) + .option(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT) + .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION) + .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue()) + .parse(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java new file mode 100644 index 000000000000..1bbd88146c8f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.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.flink; + +import java.util.concurrent.TimeUnit; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; + +/** Flink source read options */ +public class FlinkReadOptions { + private static final String PREFIX = "connector.iceberg."; + + private FlinkReadOptions() {} + + public static final ConfigOption SNAPSHOT_ID = + ConfigOptions.key("snapshot-id").longType().defaultValue(null); + + public static final ConfigOption TAG = + ConfigOptions.key("tag").stringType().defaultValue(null); + + public static final ConfigOption BRANCH = + ConfigOptions.key("branch").stringType().defaultValue(null); + + public static final ConfigOption START_TAG = + ConfigOptions.key("start-tag").stringType().defaultValue(null); + + public static final ConfigOption END_TAG = + ConfigOptions.key("end-tag").stringType().defaultValue(null); + + public static final String CASE_SENSITIVE = "case-sensitive"; + public static final ConfigOption CASE_SENSITIVE_OPTION = + ConfigOptions.key(PREFIX + CASE_SENSITIVE).booleanType().defaultValue(false); + + public static final ConfigOption AS_OF_TIMESTAMP = + ConfigOptions.key("as-of-timestamp").longType().defaultValue(null); + + public static final String STARTING_STRATEGY = "starting-strategy"; + public static final ConfigOption STARTING_STRATEGY_OPTION = + ConfigOptions.key(PREFIX + STARTING_STRATEGY) + .enumType(StreamingStartingStrategy.class) + .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT); + + public static final ConfigOption START_SNAPSHOT_TIMESTAMP = + ConfigOptions.key("start-snapshot-timestamp").longType().defaultValue(null); + + public static final ConfigOption START_SNAPSHOT_ID = + ConfigOptions.key("start-snapshot-id").longType().defaultValue(null); + + public static final ConfigOption END_SNAPSHOT_ID = + ConfigOptions.key("end-snapshot-id").longType().defaultValue(null); + + public static final String SPLIT_SIZE = "split-size"; + public static final ConfigOption SPLIT_SIZE_OPTION = + ConfigOptions.key(PREFIX + SPLIT_SIZE) + .longType() + .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT); + + public static final String SPLIT_LOOKBACK = "split-lookback"; + public static final ConfigOption SPLIT_LOOKBACK_OPTION = + ConfigOptions.key(PREFIX + SPLIT_LOOKBACK) + .intType() + .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT); + + public static final String SPLIT_FILE_OPEN_COST = "split-file-open-cost"; + public static final ConfigOption SPLIT_FILE_OPEN_COST_OPTION = + ConfigOptions.key(PREFIX + SPLIT_FILE_OPEN_COST) + .longType() + .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); + + public static final String STREAMING = "streaming"; + public static final ConfigOption STREAMING_OPTION = + ConfigOptions.key(PREFIX + STREAMING).booleanType().defaultValue(false); + + public static final String MONITOR_INTERVAL = "monitor-interval"; + public static final ConfigOption MONITOR_INTERVAL_OPTION = + ConfigOptions.key(PREFIX + MONITOR_INTERVAL).stringType().defaultValue("60s"); + + public static final String INCLUDE_COLUMN_STATS = "include-column-stats"; + public static final ConfigOption INCLUDE_COLUMN_STATS_OPTION = + ConfigOptions.key(PREFIX + INCLUDE_COLUMN_STATS).booleanType().defaultValue(false); + + public static final String MAX_PLANNING_SNAPSHOT_COUNT = "max-planning-snapshot-count"; + public static final ConfigOption MAX_PLANNING_SNAPSHOT_COUNT_OPTION = + ConfigOptions.key(PREFIX + MAX_PLANNING_SNAPSHOT_COUNT) + .intType() + .defaultValue(Integer.MAX_VALUE); + + public static final String LIMIT = "limit"; + public static final ConfigOption LIMIT_OPTION = + ConfigOptions.key(PREFIX + LIMIT).longType().defaultValue(-1L); + + public static final String MAX_ALLOWED_PLANNING_FAILURES = "max-allowed-planning-failures"; + public static final ConfigOption MAX_ALLOWED_PLANNING_FAILURES_OPTION = + ConfigOptions.key(PREFIX + MAX_ALLOWED_PLANNING_FAILURES).intType().defaultValue(3); + + public static final String WATERMARK_COLUMN = "watermark-column"; + public static final ConfigOption WATERMARK_COLUMN_OPTION = + ConfigOptions.key(PREFIX + WATERMARK_COLUMN).stringType().noDefaultValue(); + + public static final String WATERMARK_COLUMN_TIME_UNIT = "watermark-column-time-unit"; + public static final ConfigOption WATERMARK_COLUMN_TIME_UNIT_OPTION = + ConfigOptions.key(PREFIX + WATERMARK_COLUMN_TIME_UNIT) + .enumType(TimeUnit.class) + .defaultValue(TimeUnit.MICROSECONDS); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java new file mode 100644 index 000000000000..4790dc85bf28 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -0,0 +1,232 @@ +/* + * 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; + +import java.util.List; +import java.util.Set; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.ResolvedSchema; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.Schema; +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.Sets; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +/** + * Converter between Flink types and Iceberg type. The conversion is not a 1:1 mapping that not + * allows back-and-forth conversion. So some information might get lost during the back-and-forth + * conversion. + * + *

    This inconsistent types: + * + *

      + *
    • map Iceberg UUID type to Flink BinaryType(16) + *
    • map Flink VarCharType(_) and CharType(_) to Iceberg String type + *
    • map Flink VarBinaryType(_) to Iceberg Binary type + *
    • map Flink TimeType(_) to Iceberg Time type (microseconds) + *
    • map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds) + *
    • map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds) + *
    • map Flink MultiSetType to Iceberg Map type(element, int) + *
    + * + *

    + */ +public class FlinkSchemaUtil { + + private FlinkSchemaUtil() {} + + /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + @Deprecated + public static Schema convert(TableSchema schema) { + LogicalType schemaType = schema.toRowDataType().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be row type."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (schema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, schema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } + } + + /** Convert the flink table schema to apache iceberg schema with column comment. */ + public static Schema convert(ResolvedSchema flinkSchema) { + List tableColumns = flinkSchema.getColumns(); + // copy from org.apache.flink.table.api.Schema#toRowDataType + DataTypes.Field[] fields = + tableColumns.stream() + .map( + column -> { + if (column.getComment().isPresent()) { + return DataTypes.FIELD( + column.getName(), column.getDataType(), column.getComment().get()); + } else { + return DataTypes.FIELD(column.getName(), column.getDataType()); + } + }) + .toArray(DataTypes.Field[]::new); + + LogicalType schemaType = DataTypes.ROW(fields).notNull().getLogicalType(); + Preconditions.checkArgument( + schemaType instanceof RowType, "Schema logical type should be row type."); + + RowType root = (RowType) schemaType; + Type converted = root.accept(new FlinkTypeToType(root)); + Schema icebergSchema = new Schema(converted.asStructType().fields()); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(icebergSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return icebergSchema; + } + } + + private static Schema freshIdentifierFieldIds(Schema icebergSchema, List primaryKeys) { + // Locate the identifier field id list. + Set identifierFieldIds = Sets.newHashSet(); + for (String primaryKey : primaryKeys) { + Types.NestedField field = icebergSchema.findField(primaryKey); + Preconditions.checkNotNull( + field, + "Cannot find field ID for the primary key column %s in schema %s", + primaryKey, + icebergSchema); + identifierFieldIds.add(field.fieldId()); + } + return new Schema( + icebergSchema.schemaId(), icebergSchema.asStruct().fields(), identifierFieldIds); + } + + /** + * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema. + * + *

    This conversion does not assign new ids; it uses ids from the base schema. + * + *

    Data types, field order, and nullability will match the Flink type. This conversion may + * return a schema that is not compatible with base schema. + * + * @param baseSchema a Schema on which conversion is based + * @param flinkSchema a Flink TableSchema + * @return the equivalent Schema + * @throws IllegalArgumentException if the type cannot be converted or there are missing ids + */ + public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { + // convert to a type with fresh ids + Types.StructType struct = convert(flinkSchema).asStruct(); + // reassign ids to match the base schema + Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema); + // reassign doc to match the base schema + schema = TypeUtil.reassignDoc(schema, baseSchema); + + // fix types that can't be represented in Flink (UUID) + Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); + if (flinkSchema.getPrimaryKey().isPresent()) { + return freshIdentifierFieldIds(fixedSchema, flinkSchema.getPrimaryKey().get().getColumns()); + } else { + return fixedSchema; + } + } + + /** + * Convert a {@link Schema} to a {@link RowType Flink type}. + * + * @param schema a Schema + * @return the equivalent Flink type + * @throws IllegalArgumentException if the type cannot be converted to Flink + */ + public static RowType convert(Schema schema) { + return (RowType) TypeUtil.visit(schema, new TypeToFlinkType()); + } + + /** + * Convert a {@link Type} to a {@link LogicalType Flink type}. + * + * @param type a Type + * @return the equivalent Flink type + * @throws IllegalArgumentException if the type cannot be converted to Flink + */ + public static LogicalType convert(Type type) { + return TypeUtil.visit(type, new TypeToFlinkType()); + } + + /** + * Convert a {@link LogicalType Flink type} to a {@link Type}. + * + * @param flinkType a FlinkType + * @return the equivalent Iceberg type + */ + public static Type convert(LogicalType flinkType) { + return flinkType.accept(new FlinkTypeToType()); + } + + /** + * Convert a {@link RowType} to a {@link TableSchema}. + * + * @param rowType a RowType + * @return Flink TableSchema + */ + public static TableSchema toSchema(RowType rowType) { + TableSchema.Builder builder = TableSchema.builder(); + for (RowType.RowField field : rowType.getFields()) { + builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); + } + return builder.build(); + } + + /** + * Convert a {@link Schema} to a {@link TableSchema}. + * + * @param schema iceberg schema to convert. + * @return Flink TableSchema. + */ + public static TableSchema toSchema(Schema schema) { + TableSchema.Builder builder = TableSchema.builder(); + + // Add columns. + for (RowType.RowField field : convert(schema).getFields()) { + builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); + } + + // Add primary key. + Set identifierFieldIds = schema.identifierFieldIds(); + if (!identifierFieldIds.isEmpty()) { + List columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size()); + for (Integer identifierFieldId : identifierFieldIds) { + String columnName = schema.findColumnName(identifierFieldId); + Preconditions.checkNotNull( + columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema); + + columns.add(columnName); + } + builder.primaryKey(columns.toArray(new String[0])); + } + + return builder.build(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java new file mode 100644 index 000000000000..5fbd84909d69 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java @@ -0,0 +1,49 @@ +/* + * 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; + +import org.apache.flink.api.common.functions.FilterFunction; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Evaluator; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.types.Types; + +public class FlinkSourceFilter implements FilterFunction { + + private final RowType rowType; + private final Evaluator evaluator; + private final Types.StructType struct; + private volatile RowDataWrapper wrapper; + + public FlinkSourceFilter(Schema schema, Expression expr, boolean caseSensitive) { + this.rowType = FlinkSchemaUtil.convert(schema); + this.struct = schema.asStruct(); + this.evaluator = new Evaluator(struct, expr, caseSensitive); + } + + @Override + public boolean filter(RowData value) { + if (wrapper == null) { + this.wrapper = new RowDataWrapper(rowType, struct); + } + return evaluator.eval(wrapper.wrap(value)); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java new file mode 100644 index 000000000000..408065f06057 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java @@ -0,0 +1,203 @@ +/* + * 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; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.MultisetType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class FlinkTypeToType extends FlinkTypeVisitor { + + private final RowType root; + private int nextId; + + FlinkTypeToType() { + this.root = null; + } + + FlinkTypeToType(RowType root) { + this.root = root; + // the root struct's fields use the first ids + this.nextId = root.getFieldCount(); + } + + private int getNextId() { + int next = nextId; + nextId += 1; + return next; + } + + @Override + public Type visit(CharType charType) { + return Types.StringType.get(); + } + + @Override + public Type visit(VarCharType varCharType) { + return Types.StringType.get(); + } + + @Override + public Type visit(BooleanType booleanType) { + return Types.BooleanType.get(); + } + + @Override + public Type visit(BinaryType binaryType) { + return Types.FixedType.ofLength(binaryType.getLength()); + } + + @Override + public Type visit(VarBinaryType varBinaryType) { + return Types.BinaryType.get(); + } + + @Override + public Type visit(DecimalType decimalType) { + return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); + } + + @Override + public Type visit(TinyIntType tinyIntType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(SmallIntType smallIntType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(IntType intType) { + return Types.IntegerType.get(); + } + + @Override + public Type visit(BigIntType bigIntType) { + return Types.LongType.get(); + } + + @Override + public Type visit(FloatType floatType) { + return Types.FloatType.get(); + } + + @Override + public Type visit(DoubleType doubleType) { + return Types.DoubleType.get(); + } + + @Override + public Type visit(DateType dateType) { + return Types.DateType.get(); + } + + @Override + public Type visit(TimeType timeType) { + return Types.TimeType.get(); + } + + @Override + public Type visit(TimestampType timestampType) { + return Types.TimestampType.withoutZone(); + } + + @Override + public Type visit(LocalZonedTimestampType localZonedTimestampType) { + return Types.TimestampType.withZone(); + } + + @Override + public Type visit(ArrayType arrayType) { + Type elementType = arrayType.getElementType().accept(this); + if (arrayType.getElementType().isNullable()) { + return Types.ListType.ofOptional(getNextId(), elementType); + } else { + return Types.ListType.ofRequired(getNextId(), elementType); + } + } + + @Override + public Type visit(MultisetType multisetType) { + Type elementType = multisetType.getElementType().accept(this); + return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get()); + } + + @Override + public Type visit(MapType mapType) { + // keys in map are not allowed to be null. + Type keyType = mapType.getKeyType().accept(this); + Type valueType = mapType.getValueType().accept(this); + if (mapType.getValueType().isNullable()) { + return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); + } else { + return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); + } + } + + @Override + @SuppressWarnings("ReferenceEquality") + public Type visit(RowType rowType) { + List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); + boolean isRoot = root == rowType; + + List types = + rowType.getFields().stream() + .map(f -> f.getType().accept(this)) + .collect(Collectors.toList()); + + for (int i = 0; i < rowType.getFieldCount(); i++) { + int id = isRoot ? i : getNextId(); + + RowType.RowField field = rowType.getFields().get(i); + String name = field.getName(); + String comment = field.getDescription().orElse(null); + + if (field.getType().isNullable()) { + newFields.add(Types.NestedField.optional(id, name, types.get(i), comment)); + } else { + newFields.add(Types.NestedField.required(id, name, types.get(i), comment)); + } + } + + return Types.StructType.of(newFields); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java new file mode 100644 index 000000000000..f3de2416088c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.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; + +import org.apache.flink.table.types.logical.DayTimeIntervalType; +import org.apache.flink.table.types.logical.DistinctType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeVisitor; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RawType; +import org.apache.flink.table.types.logical.StructuredType; +import org.apache.flink.table.types.logical.SymbolType; +import org.apache.flink.table.types.logical.YearMonthIntervalType; +import org.apache.flink.table.types.logical.ZonedTimestampType; + +public abstract class FlinkTypeVisitor implements LogicalTypeVisitor { + + // ------------------------- Unsupported types ------------------------------ + + @Override + public T visit(ZonedTimestampType zonedTimestampType) { + throw new UnsupportedOperationException("Unsupported ZonedTimestampType."); + } + + @Override + public T visit(YearMonthIntervalType yearMonthIntervalType) { + throw new UnsupportedOperationException("Unsupported YearMonthIntervalType."); + } + + @Override + public T visit(DayTimeIntervalType dayTimeIntervalType) { + throw new UnsupportedOperationException("Unsupported DayTimeIntervalType."); + } + + @Override + public T visit(DistinctType distinctType) { + throw new UnsupportedOperationException("Unsupported DistinctType."); + } + + @Override + public T visit(StructuredType structuredType) { + throw new UnsupportedOperationException("Unsupported StructuredType."); + } + + @Override + public T visit(NullType nullType) { + throw new UnsupportedOperationException("Unsupported NullType."); + } + + @Override + public T visit(RawType rawType) { + throw new UnsupportedOperationException("Unsupported RawType."); + } + + @Override + public T visit(SymbolType symbolType) { + throw new UnsupportedOperationException("Unsupported SymbolType."); + } + + @Override + public T visit(LogicalType other) { + throw new UnsupportedOperationException("Unsupported type: " + other); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java new file mode 100644 index 000000000000..ca7b1120bc81 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -0,0 +1,205 @@ +/* + * 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; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; + +/** + * A class for common Iceberg configs for Flink writes. + * + *

    If a config is set at multiple levels, the following order of precedence is used (top to + * bottom): + * + *

      + *
    1. Write options + *
    2. flink ReadableConfig + *
    3. Table metadata + *
    + * + * The most specific value is set in write options and takes precedence over all other configs. If + * no write option is provided, this class checks the flink configuration for any overrides. If no + * applicable value is found in the write options, this class uses the table metadata. + * + *

    Note this class is NOT meant to be serialized. + */ +public class FlinkWriteConf { + + private final FlinkConfParser confParser; + + public FlinkWriteConf( + Table table, Map writeOptions, ReadableConfig readableConfig) { + this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); + } + + public boolean overwriteMode() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.OVERWRITE_MODE.key()) + .flinkConfig(FlinkWriteOptions.OVERWRITE_MODE) + .defaultValue(FlinkWriteOptions.OVERWRITE_MODE.defaultValue()) + .parse(); + } + + public boolean upsertMode() { + return confParser + .booleanConf() + .option(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key()) + .flinkConfig(FlinkWriteOptions.WRITE_UPSERT_ENABLED) + .tableProperty(TableProperties.UPSERT_ENABLED) + .defaultValue(TableProperties.UPSERT_ENABLED_DEFAULT) + .parse(); + } + + public FileFormat dataFileFormat() { + String valueAsString = + confParser + .stringConf() + .option(FlinkWriteOptions.WRITE_FORMAT.key()) + .flinkConfig(FlinkWriteOptions.WRITE_FORMAT) + .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) + .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) + .parse(); + return FileFormat.fromString(valueAsString); + } + + public long targetDataFileSize() { + return confParser + .longConf() + .option(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES.key()) + .flinkConfig(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES) + .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) + .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) + .parse(); + } + + public String parquetCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.PARQUET_COMPRESSION) + .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) + .parse(); + } + + public String parquetCompressionLevel() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) + .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) + .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + public String avroCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.AVRO_COMPRESSION) + .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) + .parse(); + } + + public String avroCompressionLevel() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) + .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) + .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) + .parseOptional(); + } + + public String orcCompressionCodec() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) + .tableProperty(TableProperties.ORC_COMPRESSION) + .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) + .parse(); + } + + public String orcCompressionStrategy() { + return confParser + .stringConf() + .option(FlinkWriteOptions.COMPRESSION_STRATEGY.key()) + .flinkConfig(FlinkWriteOptions.COMPRESSION_STRATEGY) + .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) + .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) + .parse(); + } + + public DistributionMode distributionMode() { + String modeName = + confParser + .stringConf() + .option(FlinkWriteOptions.DISTRIBUTION_MODE.key()) + .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE) + .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) + .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE) + .parse(); + return DistributionMode.fromName(modeName); + } + + public int workerPoolSize() { + return confParser + .intConf() + .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) + .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) + .parse(); + } + + public String branch() { + return confParser + .stringConf() + .option(FlinkWriteOptions.BRANCH.key()) + .defaultValue(FlinkWriteOptions.BRANCH.defaultValue()) + .parse(); + } + + public Integer writeParallelism() { + return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional(); + } + + /** + * NOTE: This may be removed or changed in a future release. This value specifies the interval for + * refreshing the table instances in sink writer subtasks. If not specified then the default + * behavior is to not refresh the table. + * + * @return the interval for refreshing the table in sink writer subtasks + */ + @Experimental + public Duration tableRefreshInterval() { + return confParser + .durationConf() + .option(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key()) + .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) + .parseOptional(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java new file mode 100644 index 000000000000..df73f2e09cac --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -0,0 +1,73 @@ +/* + * 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; + +import java.time.Duration; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.iceberg.SnapshotRef; + +/** Flink sink write options */ +public class FlinkWriteOptions { + + private FlinkWriteOptions() {} + + // File format for write operations(default: Table write.format.default ) + public static final ConfigOption WRITE_FORMAT = + ConfigOptions.key("write-format").stringType().noDefaultValue(); + + // Overrides this table's write.target-file-size-bytes + public static final ConfigOption TARGET_FILE_SIZE_BYTES = + ConfigOptions.key("target-file-size-bytes").longType().noDefaultValue(); + + // Overrides this table's write..compression-codec + public static final ConfigOption COMPRESSION_CODEC = + ConfigOptions.key("compression-codec").stringType().noDefaultValue(); + + // Overrides this table's write..compression-level + public static final ConfigOption COMPRESSION_LEVEL = + ConfigOptions.key("compression-level").stringType().noDefaultValue(); + + // Overrides this table's write..compression-strategy + public static final ConfigOption COMPRESSION_STRATEGY = + ConfigOptions.key("compression-strategy").stringType().noDefaultValue(); + + // Overrides this table's write.upsert.enabled + public static final ConfigOption WRITE_UPSERT_ENABLED = + ConfigOptions.key("upsert-enabled").booleanType().noDefaultValue(); + + public static final ConfigOption OVERWRITE_MODE = + ConfigOptions.key("overwrite-enabled").booleanType().defaultValue(false); + + // Overrides the table's write.distribution-mode + public static final ConfigOption DISTRIBUTION_MODE = + ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + + // Branch to write to + public static final ConfigOption BRANCH = + ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); + + public static final ConfigOption WRITE_PARALLELISM = + ConfigOptions.key("write-parallelism").intType().noDefaultValue(); + + @Experimental + public static final ConfigOption TABLE_REFRESH_INTERVAL = + ConfigOptions.key("table-refresh-interval").durationType().noDefaultValue(); +} 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 new file mode 100644 index 000000000000..1b9268569d9a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java @@ -0,0 +1,121 @@ +/* + * 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; + +import java.util.List; +import java.util.Map; +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.api.constraints.UniqueConstraint; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.sink.DataStreamSinkProvider; +import org.apache.flink.table.connector.sink.DynamicTableSink; +import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; +import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; + +public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { + private final TableLoader tableLoader; + private final TableSchema tableSchema; + private final ReadableConfig readableConfig; + private final Map writeProps; + + private boolean overwrite = false; + + private IcebergTableSink(IcebergTableSink toCopy) { + this.tableLoader = toCopy.tableLoader; + this.tableSchema = toCopy.tableSchema; + this.overwrite = toCopy.overwrite; + this.readableConfig = toCopy.readableConfig; + this.writeProps = toCopy.writeProps; + } + + public IcebergTableSink( + TableLoader tableLoader, + TableSchema tableSchema, + ReadableConfig readableConfig, + Map writeProps) { + this.tableLoader = tableLoader; + this.tableSchema = tableSchema; + this.readableConfig = readableConfig; + this.writeProps = writeProps; + } + + @Override + public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { + Preconditions.checkState( + !overwrite || context.isBounded(), + "Unbounded data stream doesn't support overwrite operation."); + + List equalityColumns = + tableSchema.getPrimaryKey().map(UniqueConstraint::getColumns).orElseGet(ImmutableList::of); + + return new DataStreamSinkProvider() { + @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(); + } + }; + } + + @Override + public void applyStaticPartition(Map partition) { + // The flink's PartitionFanoutWriter will handle the static partition write policy + // automatically. + } + + @Override + public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { + ChangelogMode.Builder builder = ChangelogMode.newBuilder(); + for (RowKind kind : requestedMode.getContainedKinds()) { + builder.addContainedKind(kind); + } + return builder.build(); + } + + @Override + public DynamicTableSink copy() { + return new IcebergTableSink(this); + } + + @Override + public String asSummaryString() { + return "Iceberg table sink"; + } + + @Override + public void applyOverwrite(boolean newOverwrite) { + this.overwrite = newOverwrite; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java new file mode 100644 index 000000000000..d4cec7a3e80b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java @@ -0,0 +1,142 @@ +/* + * 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; + +import java.lang.reflect.Array; +import java.nio.ByteBuffer; +import java.time.LocalDateTime; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.UUIDUtil; + +public class RowDataWrapper implements StructLike { + + private final LogicalType[] types; + private final PositionalGetter[] getters; + private RowData rowData = null; + + public RowDataWrapper(RowType rowType, Types.StructType struct) { + int size = rowType.getFieldCount(); + + types = (LogicalType[]) Array.newInstance(LogicalType.class, size); + getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); + + for (int i = 0; i < size; i++) { + types[i] = rowType.getTypeAt(i); + getters[i] = buildGetter(types[i], struct.fields().get(i).type()); + } + } + + public RowDataWrapper wrap(RowData data) { + this.rowData = data; + return this; + } + + @Override + public int size() { + return types.length; + } + + @Override + public T get(int pos, Class javaClass) { + if (rowData.isNullAt(pos)) { + return null; + } else if (getters[pos] != null) { + return javaClass.cast(getters[pos].get(rowData, pos)); + } + + Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); + return javaClass.cast(value); + } + + @Override + public void set(int pos, T value) { + throw new UnsupportedOperationException( + "Could not set a field in the RowDataWrapper because rowData is read-only"); + } + + private interface PositionalGetter { + T get(RowData data, int pos); + } + + private static PositionalGetter buildGetter(LogicalType logicalType, Type type) { + switch (logicalType.getTypeRoot()) { + case TINYINT: + return (row, pos) -> (int) row.getByte(pos); + case SMALLINT: + return (row, pos) -> (int) row.getShort(pos); + case CHAR: + case VARCHAR: + return (row, pos) -> row.getString(pos).toString(); + + case BINARY: + case VARBINARY: + if (Type.TypeID.UUID == type.typeId()) { + return (row, pos) -> UUIDUtil.convert(row.getBinary(pos)); + } else { + return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); + } + + case DECIMAL: + DecimalType decimalType = (DecimalType) logicalType; + return (row, pos) -> + row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); + + case TIME_WITHOUT_TIME_ZONE: + // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds + // (Long). + return (row, pos) -> ((long) row.getInt(pos)) * 1_000; + + case TIMESTAMP_WITHOUT_TIME_ZONE: + TimestampType timestampType = (TimestampType) logicalType; + return (row, pos) -> { + LocalDateTime localDateTime = + row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); + return DateTimeUtil.microsFromTimestamp(localDateTime); + }; + + case TIMESTAMP_WITH_LOCAL_TIME_ZONE: + LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; + return (row, pos) -> { + TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); + return timestampData.getMillisecond() * 1000 + + timestampData.getNanoOfMillisecond() / 1000; + }; + + case ROW: + RowType rowType = (RowType) logicalType; + Types.StructType structType = (Types.StructType) type; + + RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); + return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); + + default: + return null; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java new file mode 100644 index 000000000000..da509451fee7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java @@ -0,0 +1,159 @@ +/* + * 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; + +import java.io.Closeable; +import java.io.IOException; +import java.io.Serializable; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.hadoop.SerializableConfiguration; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * Serializable loader to load an Iceberg {@link Table}. Flink needs to get {@link Table} objects in + * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg + * table loader to get the {@link Table} object. + */ +public interface TableLoader extends Closeable, Serializable, Cloneable { + + void open(); + + boolean isOpen(); + + Table loadTable(); + + /** Clone a TableLoader */ + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + TableLoader clone(); + + static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { + return new CatalogTableLoader(catalogLoader, identifier); + } + + static TableLoader fromHadoopTable(String location) { + return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf()); + } + + static TableLoader fromHadoopTable(String location, Configuration hadoopConf) { + return new HadoopTableLoader(location, hadoopConf); + } + + class HadoopTableLoader implements TableLoader { + + private static final long serialVersionUID = 1L; + + private final String location; + private final SerializableConfiguration hadoopConf; + + private transient HadoopTables tables; + + private HadoopTableLoader(String location, Configuration conf) { + this.location = location; + this.hadoopConf = new SerializableConfiguration(conf); + } + + @Override + public void open() { + tables = new HadoopTables(hadoopConf.get()); + } + + @Override + public boolean isOpen() { + return tables != null; + } + + @Override + public Table loadTable() { + FlinkEnvironmentContext.init(); + return tables.load(location); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new HadoopTableLoader(location, new Configuration(hadoopConf.get())); + } + + @Override + public void close() {} + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("location", location).toString(); + } + } + + class CatalogTableLoader implements TableLoader { + + private static final long serialVersionUID = 1L; + + private final CatalogLoader catalogLoader; + private final String identifier; + + private transient Catalog catalog; + + private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { + this.catalogLoader = catalogLoader; + this.identifier = tableIdentifier.toString(); + } + + @Override + public void open() { + catalog = catalogLoader.loadCatalog(); + } + + @Override + public boolean isOpen() { + return catalog != null; + } + + @Override + public Table loadTable() { + FlinkEnvironmentContext.init(); + return catalog.loadTable(TableIdentifier.parse(identifier)); + } + + @Override + public void close() throws IOException { + if (catalog instanceof Closeable) { + ((Closeable) catalog).close(); + } + + catalog = null; + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier)); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableIdentifier", identifier) + .add("catalogLoader", catalogLoader) + .toString(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java new file mode 100644 index 000000000000..f8f1b74b1ceb --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java @@ -0,0 +1,134 @@ +/* + * 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; + +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.BigIntType; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.BooleanType; +import org.apache.flink.table.types.logical.DateType; +import org.apache.flink.table.types.logical.DecimalType; +import org.apache.flink.table.types.logical.DoubleType; +import org.apache.flink.table.types.logical.FloatType; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +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.TypeUtil; +import org.apache.iceberg.types.Types; + +class TypeToFlinkType extends TypeUtil.SchemaVisitor { + TypeToFlinkType() {} + + @Override + public LogicalType schema(Schema schema, LogicalType structType) { + return structType; + } + + @Override + public LogicalType struct(Types.StructType struct, List fieldResults) { + List fields = struct.fields(); + + List flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + LogicalType type = fieldResults.get(i); + RowType.RowField flinkField = + new RowType.RowField(field.name(), type.copy(field.isOptional()), field.doc()); + flinkFields.add(flinkField); + } + + return new RowType(flinkFields); + } + + @Override + public LogicalType field(Types.NestedField field, LogicalType fieldResult) { + return fieldResult; + } + + @Override + public LogicalType list(Types.ListType list, LogicalType elementResult) { + return new ArrayType(elementResult.copy(list.isElementOptional())); + } + + @Override + public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) { + // keys in map are not allowed to be null. + return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional())); + } + + @Override + public LogicalType primitive(Type.PrimitiveType primitive) { + switch (primitive.typeId()) { + case BOOLEAN: + return new BooleanType(); + case INTEGER: + return new IntType(); + case LONG: + return new BigIntType(); + case FLOAT: + return new FloatType(); + case DOUBLE: + return new DoubleType(); + case DATE: + return new DateType(); + case TIME: + // For the type: Flink only support TimeType with default precision (second) now. The + // precision of time is + // not supported in Flink, so we can think of it as a simple time type directly. + // For the data: Flink uses int that support mills to represent time data, so it supports + // mills precision. + return new TimeType(); + case TIMESTAMP: + Types.TimestampType timestamp = (Types.TimestampType) primitive; + if (timestamp.shouldAdjustToUTC()) { + // MICROS + return new LocalZonedTimestampType(6); + } else { + // MICROS + return new TimestampType(6); + } + case STRING: + return new VarCharType(VarCharType.MAX_LENGTH); + case UUID: + // UUID length is 16 + return new BinaryType(16); + case FIXED: + Types.FixedType fixedType = (Types.FixedType) primitive; + return new BinaryType(fixedType.length()); + case BINARY: + return new VarBinaryType(VarBinaryType.MAX_LENGTH); + case DECIMAL: + Types.DecimalType decimal = (Types.DecimalType) primitive; + return new DecimalType(decimal.precision(), decimal.scale()); + default: + throw new UnsupportedOperationException( + "Cannot convert unknown type to Flink: " + primitive); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java new file mode 100644 index 000000000000..b96b47c5a785 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.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.actions; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.Table; + +public class Actions { + + public static final Configuration CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private final StreamExecutionEnvironment env; + private final Table table; + + private Actions(StreamExecutionEnvironment env, Table table) { + this.env = env; + this.table = table; + } + + public static Actions forTable(StreamExecutionEnvironment env, Table table) { + return new Actions(env, table); + } + + public static Actions forTable(Table table) { + return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table); + } + + public RewriteDataFilesAction rewriteDataFiles() { + return new RewriteDataFilesAction(env, table); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java new file mode 100644 index 000000000000..670abebcb58a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.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.flink.actions; + +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.BaseRewriteDataFilesAction; +import org.apache.iceberg.flink.source.RowDataRewriter; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { + + private final StreamExecutionEnvironment env; + private int maxParallelism; + + public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { + super(table); + this.env = env; + this.maxParallelism = env.getParallelism(); + } + + @Override + protected FileIO fileIO() { + return table().io(); + } + + @Override + protected List rewriteDataForTasks(List combinedScanTasks) { + int size = combinedScanTasks.size(); + int parallelism = Math.min(size, maxParallelism); + DataStream dataStream = env.fromCollection(combinedScanTasks); + RowDataRewriter rowDataRewriter = + new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); + try { + return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); + } catch (Exception e) { + throw new RuntimeException("Rewrite data file error.", e); + } + } + + @Override + protected RewriteDataFilesAction self() { + return this; + } + + public RewriteDataFilesAction maxParallelism(int parallelism) { + Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %s", parallelism); + this.maxParallelism = parallelism; + return this; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java new file mode 100644 index 000000000000..8103224a0b6c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.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.flink.data; + +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.LogicalTypeFamily; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.NullType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.Pair; + +public abstract class AvroWithFlinkSchemaVisitor + extends AvroWithPartnerByStructureVisitor { + + @Override + protected boolean isStringType(LogicalType logicalType) { + return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING); + } + + @Override + protected boolean isMapType(LogicalType logicalType) { + return logicalType instanceof MapType; + } + + @Override + protected LogicalType arrayElementType(LogicalType arrayType) { + Preconditions.checkArgument( + arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); + return ((ArrayType) arrayType).getElementType(); + } + + @Override + protected LogicalType mapKeyType(LogicalType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).getKeyType(); + } + + @Override + protected LogicalType mapValueType(LogicalType mapType) { + Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); + return ((MapType) mapType).getValueType(); + } + + @Override + protected Pair fieldNameAndType(LogicalType structType, int pos) { + Preconditions.checkArgument( + structType instanceof RowType, "Invalid struct: %s is not a struct", structType); + RowType.RowField field = ((RowType) structType).getFields().get(pos); + return Pair.of(field.getName(), field.getType()); + } + + @Override + protected LogicalType nullType() { + return new NullType(); + } +} 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 new file mode 100644 index 000000000000..86404959735a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java @@ -0,0 +1,169 @@ +/* + * 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; + +public class FlinkAvroReader implements DatumReader, SupportsRowPosition { + + private final Schema readSchema; + private final ValueReader reader; + private Schema fileSchema = null; + + public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { + this(expectedSchema, readSchema, ImmutableMap.of()); + } + + @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/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java new file mode 100644 index 000000000000..873e65783119 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java @@ -0,0 +1,165 @@ +/* + * 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.stream.Collectors; +import java.util.stream.IntStream; +import java.util.stream.Stream; +import org.apache.avro.LogicalTypes; +import org.apache.avro.Schema; +import org.apache.avro.io.Encoder; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.avro.MetricsAwareDatumWriter; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.avro.ValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class FlinkAvroWriter implements MetricsAwareDatumWriter { + private final RowType rowType; + private ValueWriter writer = null; + + public FlinkAvroWriter(RowType rowType) { + this.rowType = rowType; + } + + @Override + @SuppressWarnings("unchecked") + public void setSchema(Schema schema) { + this.writer = + (ValueWriter) + AvroWithFlinkSchemaVisitor.visit(rowType, schema, new WriteBuilder()); + } + + @Override + public void write(RowData datum, Encoder out) throws IOException { + writer.write(datum, out); + } + + @Override + public Stream metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends AvroWithFlinkSchemaVisitor> { + @Override + public ValueWriter record( + LogicalType struct, Schema record, List names, List> fields) { + return FlinkValueWriters.row( + fields, + IntStream.range(0, names.size()) + .mapToObj(i -> fieldNameAndType(struct, i).second()) + .collect(Collectors.toList())); + } + + @Override + public ValueWriter union(LogicalType type, Schema union, List> options) { + Preconditions.checkArgument( + options.contains(ValueWriters.nulls()), + "Cannot create writer for non-option union: %s", + union); + Preconditions.checkArgument( + options.size() == 2, "Cannot create writer for non-option union: %s", union); + if (union.getTypes().get(0).getType() == Schema.Type.NULL) { + return ValueWriters.option(0, options.get(1)); + } else { + return ValueWriters.option(1, options.get(0)); + } + } + + @Override + public ValueWriter array(LogicalType sArray, Schema array, ValueWriter elementWriter) { + return FlinkValueWriters.array(elementWriter, arrayElementType(sArray)); + } + + @Override + public ValueWriter map(LogicalType sMap, Schema map, ValueWriter valueReader) { + return FlinkValueWriters.map( + FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); + } + + @Override + public ValueWriter map( + LogicalType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { + return FlinkValueWriters.arrayMap( + keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); + } + + @Override + public ValueWriter primitive(LogicalType type, Schema primitive) { + org.apache.avro.LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return ValueWriters.ints(); + + case "time-micros": + return FlinkValueWriters.timeMicros(); + + case "timestamp-micros": + return FlinkValueWriters.timestampMicros(); + + case "decimal": + LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; + return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); + + case "uuid": + return ValueWriters.uuids(); + + default: + throw new IllegalArgumentException("Unsupported logical type: " + logicalType); + } + } + + switch (primitive.getType()) { + case NULL: + return ValueWriters.nulls(); + case BOOLEAN: + return ValueWriters.booleans(); + case INT: + switch (type.getTypeRoot()) { + case TINYINT: + return ValueWriters.tinyints(); + case SMALLINT: + return ValueWriters.shorts(); + default: + return ValueWriters.ints(); + } + case LONG: + return ValueWriters.longs(); + case FLOAT: + return ValueWriters.floats(); + case DOUBLE: + return ValueWriters.doubles(); + case STRING: + return FlinkValueWriters.strings(); + case FIXED: + return ValueWriters.fixed(primitive.getFixedSize()); + case BYTES: + return ValueWriters.bytes(); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java new file mode 100644 index 000000000000..65b9d44ad4b8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java @@ -0,0 +1,131 @@ +/* + * 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.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.orc.OrcRowReader; +import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +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.orc.TypeDescription; +import org.apache.orc.storage.ql.exec.vector.StructColumnVector; +import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; + +public class FlinkOrcReader implements OrcRowReader { + private final OrcValueReader reader; + + public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) { + this(iSchema, readSchema, ImmutableMap.of()); + } + + public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map idToConstant) { + this.reader = + OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant)); + } + + @Override + public RowData read(VectorizedRowBatch batch, int row) { + return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + reader.setBatchContext(batchOffsetInFile); + } + + private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { + private final Map idToConstant; + + private ReadBuilder(Map idToConstant) { + this.idToConstant = idToConstant; + } + + @Override + public OrcValueReader record( + Types.StructType iStruct, + TypeDescription record, + List names, + List> fields) { + return FlinkOrcReaders.struct(fields, iStruct, idToConstant); + } + + @Override + public OrcValueReader list( + Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { + return FlinkOrcReaders.array(elementReader); + } + + @Override + public OrcValueReader map( + Types.MapType iMap, + TypeDescription map, + OrcValueReader keyReader, + OrcValueReader valueReader) { + return FlinkOrcReaders.map(keyReader, valueReader); + } + + @Override + public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { + switch (iPrimitive.typeId()) { + case BOOLEAN: + return OrcValueReaders.booleans(); + case INTEGER: + return OrcValueReaders.ints(); + case LONG: + return OrcValueReaders.longs(); + case FLOAT: + return OrcValueReaders.floats(); + case DOUBLE: + return OrcValueReaders.doubles(); + case DATE: + return FlinkOrcReaders.dates(); + case TIME: + return FlinkOrcReaders.times(); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; + if (timestampType.shouldAdjustToUTC()) { + return FlinkOrcReaders.timestampTzs(); + } else { + return FlinkOrcReaders.timestamps(); + } + case STRING: + return FlinkOrcReaders.strings(); + case UUID: + case FIXED: + case BINARY: + return OrcValueReaders.bytes(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; + return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale()); + default: + throw new IllegalArgumentException( + String.format( + "Invalid iceberg type %s corresponding to ORC type %s", iPrimitive, primitive)); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java new file mode 100644 index 000000000000..7a4a15c7e600 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java @@ -0,0 +1,283 @@ +/* + * 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.math.BigDecimal; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.orc.OrcValueReader; +import org.apache.iceberg.orc.OrcValueReaders; +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.apache.iceberg.types.Types; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; +import org.apache.orc.storage.serde2.io.HiveDecimalWritable; + +class FlinkOrcReaders { + private FlinkOrcReaders() {} + + static OrcValueReader strings() { + return StringReader.INSTANCE; + } + + static OrcValueReader dates() { + return DateReader.INSTANCE; + } + + static OrcValueReader decimals(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Reader(precision, scale); + } else if (precision <= 38) { + return new Decimal38Reader(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueReader times() { + return TimeReader.INSTANCE; + } + + static OrcValueReader timestamps() { + return TimestampReader.INSTANCE; + } + + static OrcValueReader timestampTzs() { + return TimestampTzReader.INSTANCE; + } + + static OrcValueReader array(OrcValueReader elementReader) { + return new ArrayReader<>(elementReader); + } + + public static OrcValueReader map( + OrcValueReader keyReader, OrcValueReader valueReader) { + return new MapReader<>(keyReader, valueReader); + } + + public static OrcValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements OrcValueReader { + private static final StringReader INSTANCE = new StringReader(); + + @Override + public StringData nonNullRead(ColumnVector vector, int row) { + BytesColumnVector bytesVector = (BytesColumnVector) vector; + return StringData.fromBytes( + bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); + } + } + + private static class DateReader implements OrcValueReader { + private static final DateReader INSTANCE = new DateReader(); + + @Override + public Integer nonNullRead(ColumnVector vector, int row) { + return (int) ((LongColumnVector) vector).vector[row]; + } + } + + private static class Decimal18Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal18Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData nonNullRead(ColumnVector vector, int row) { + HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; + + // The hive ORC writer may will adjust the scale of decimal data. + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale); + } + } + + private static class Decimal38Reader implements OrcValueReader { + private final int precision; + private final int scale; + + Decimal38Reader(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData nonNullRead(ColumnVector vector, int row) { + BigDecimal value = + ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); + + Preconditions.checkArgument( + value.precision() <= precision, + "Cannot read value as decimal(%s,%s), too large: %s", + precision, + scale, + value); + + return DecimalData.fromBigDecimal(value, precision, scale); + } + } + + private static class TimeReader implements OrcValueReader { + private static final TimeReader INSTANCE = new TimeReader(); + + @Override + public Integer nonNullRead(ColumnVector vector, int row) { + long micros = ((LongColumnVector) vector).vector[row]; + // Flink only support time mills, just erase micros. + return (int) (micros / 1000); + } + } + + private static class TimestampReader implements OrcValueReader { + private static final TimestampReader INSTANCE = new TimestampReader(); + + @Override + public TimestampData nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + LocalDateTime localDate = + Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) + .atOffset(ZoneOffset.UTC) + .toLocalDateTime(); + return TimestampData.fromLocalDateTime(localDate); + } + } + + private static class TimestampTzReader implements OrcValueReader { + private static final TimestampTzReader INSTANCE = new TimestampTzReader(); + + @Override + public TimestampData nonNullRead(ColumnVector vector, int row) { + TimestampColumnVector tcv = (TimestampColumnVector) vector; + Instant instant = + Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) + .atOffset(ZoneOffset.UTC) + .toInstant(); + return TimestampData.fromInstant(instant); + } + } + + private static class ArrayReader implements OrcValueReader { + private final OrcValueReader elementReader; + + private ArrayReader(OrcValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public ArrayData nonNullRead(ColumnVector vector, int row) { + ListColumnVector listVector = (ListColumnVector) vector; + int offset = (int) listVector.offsets[row]; + int length = (int) listVector.lengths[row]; + List elements = Lists.newArrayListWithExpectedSize(length); + for (int c = 0; c < length; ++c) { + elements.add(elementReader.read(listVector.child, offset + c)); + } + return new GenericArrayData(elements.toArray()); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + elementReader.setBatchContext(batchOffsetInFile); + } + } + + private static class MapReader implements OrcValueReader { + private final OrcValueReader keyReader; + private final OrcValueReader valueReader; + + private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData nonNullRead(ColumnVector vector, int row) { + MapColumnVector mapVector = (MapColumnVector) vector; + int offset = (int) mapVector.offsets[row]; + long length = mapVector.lengths[row]; + + Map map = Maps.newHashMap(); + for (int c = 0; c < length; c++) { + K key = keyReader.read(mapVector.keys, offset + c); + V value = valueReader.read(mapVector.values, offset + c); + map.put(key, value); + } + + return new GenericMapData(map); + } + + @Override + public void setBatchContext(long batchOffsetInFile) { + keyReader.setBatchContext(batchOffsetInFile); + valueReader.setBatchContext(batchOffsetInFile); + } + } + + private static class StructReader extends OrcValueReaders.StructReader { + private final int numFields; + + StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = struct.fields().size(); + } + + @Override + protected RowData create() { + return new GenericRowData(numFields); + } + + @Override + protected void set(RowData struct, int pos, Object value) { + ((GenericRowData) struct).setField(pos, value); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java new file mode 100644 index 000000000000..6a31accffd22 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java @@ -0,0 +1,163 @@ +/* + * 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.util.Deque; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.OrcRowWriter; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.orc.storage.ql.exec.vector.VectorizedRowBatch; + +public class FlinkOrcWriter implements OrcRowWriter { + private final FlinkOrcWriters.RowDataWriter writer; + + private FlinkOrcWriter(RowType rowType, Schema iSchema) { + this.writer = + (FlinkOrcWriters.RowDataWriter) + FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder()); + } + + public static OrcRowWriter buildWriter(RowType rowType, Schema iSchema) { + return new FlinkOrcWriter(rowType, iSchema); + } + + @Override + public void write(RowData row, VectorizedRowBatch output) { + Preconditions.checkArgument(row != null, "value must not be null"); + writer.writeRow(row, output); + } + + @Override + public List> writers() { + return writer.writers(); + } + + @Override + public Stream> metrics() { + return writer.metrics(); + } + + private static class WriteBuilder extends FlinkSchemaVisitor> { + private final Deque fieldIds = Lists.newLinkedList(); + + private WriteBuilder() {} + + @Override + public void beforeField(Types.NestedField field) { + fieldIds.push(field.fieldId()); + } + + @Override + public void afterField(Types.NestedField field) { + fieldIds.pop(); + } + + @Override + public OrcValueWriter record( + Types.StructType iStruct, List> results, List fieldType) { + return FlinkOrcWriters.struct(results, fieldType); + } + + @Override + public OrcValueWriter map( + Types.MapType iMap, + OrcValueWriter key, + OrcValueWriter value, + LogicalType keyType, + LogicalType valueType) { + return FlinkOrcWriters.map(key, value, keyType, valueType); + } + + @Override + public OrcValueWriter list( + Types.ListType iList, OrcValueWriter element, LogicalType elementType) { + return FlinkOrcWriters.list(element, elementType); + } + + @Override + public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { + switch (iPrimitive.typeId()) { + case BOOLEAN: + return GenericOrcWriters.booleans(); + case INTEGER: + switch (flinkPrimitive.getTypeRoot()) { + case TINYINT: + return GenericOrcWriters.bytes(); + case SMALLINT: + return GenericOrcWriters.shorts(); + } + return GenericOrcWriters.ints(); + case LONG: + return GenericOrcWriters.longs(); + case FLOAT: + Preconditions.checkArgument( + fieldIds.peek() != null, + String.format( + "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " + + "information is not properly pushed during schema visiting.", + iPrimitive)); + return GenericOrcWriters.floats(fieldIds.peek()); + case DOUBLE: + Preconditions.checkArgument( + fieldIds.peek() != null, + String.format( + "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " + + "information is not properly pushed during schema visiting.", + iPrimitive)); + return GenericOrcWriters.doubles(fieldIds.peek()); + case DATE: + return FlinkOrcWriters.dates(); + case TIME: + return FlinkOrcWriters.times(); + case TIMESTAMP: + Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; + if (timestampType.shouldAdjustToUTC()) { + return FlinkOrcWriters.timestampTzs(); + } else { + return FlinkOrcWriters.timestamps(); + } + case STRING: + return FlinkOrcWriters.strings(); + case UUID: + case FIXED: + case BINARY: + return GenericOrcWriters.byteArrays(); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; + return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale()); + default: + throw new IllegalArgumentException( + String.format( + "Invalid iceberg type %s corresponding to Flink logical type %s", + iPrimitive, flinkPrimitive)); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java new file mode 100644 index 000000000000..da2f95cf822f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java @@ -0,0 +1,317 @@ +/* + * 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.time.Instant; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.stream.Stream; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.FieldMetrics; +import org.apache.iceberg.data.orc.GenericOrcWriters; +import org.apache.iceberg.orc.OrcValueWriter; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.orc.storage.common.type.HiveDecimal; +import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; +import org.apache.orc.storage.ql.exec.vector.ColumnVector; +import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; +import org.apache.orc.storage.ql.exec.vector.ListColumnVector; +import org.apache.orc.storage.ql.exec.vector.LongColumnVector; +import org.apache.orc.storage.ql.exec.vector.MapColumnVector; +import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; + +class FlinkOrcWriters { + + private FlinkOrcWriters() {} + + static OrcValueWriter strings() { + return StringWriter.INSTANCE; + } + + static OrcValueWriter dates() { + return DateWriter.INSTANCE; + } + + static OrcValueWriter times() { + return TimeWriter.INSTANCE; + } + + static OrcValueWriter timestamps() { + return TimestampWriter.INSTANCE; + } + + static OrcValueWriter timestampTzs() { + return TimestampTzWriter.INSTANCE; + } + + static OrcValueWriter decimals(int precision, int scale) { + if (precision <= 18) { + return new Decimal18Writer(precision, scale); + } else if (precision <= 38) { + return new Decimal38Writer(precision, scale); + } else { + throw new IllegalArgumentException("Invalid precision: " + precision); + } + } + + static OrcValueWriter list( + OrcValueWriter elementWriter, LogicalType elementType) { + return new ListWriter<>(elementWriter, elementType); + } + + static OrcValueWriter map( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + return new MapWriter<>(keyWriter, valueWriter, keyType, valueType); + } + + static OrcValueWriter struct(List> writers, List types) { + return new RowDataWriter(writers, types); + } + + private static class StringWriter implements OrcValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + @Override + public void nonNullWrite(int rowId, StringData data, ColumnVector output) { + byte[] value = data.toBytes(); + ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); + } + } + + private static class DateWriter implements OrcValueWriter { + private static final DateWriter INSTANCE = new DateWriter(); + + @Override + public void nonNullWrite(int rowId, Integer data, ColumnVector output) { + ((LongColumnVector) output).vector[rowId] = data; + } + } + + private static class TimeWriter implements OrcValueWriter { + private static final TimeWriter INSTANCE = new TimeWriter(); + + @Override + public void nonNullWrite(int rowId, Integer millis, ColumnVector output) { + // The time in flink is in millisecond, while the standard time in iceberg is microsecond. + // So we need to transform it to microsecond. + ((LongColumnVector) output).vector[rowId] = millis * 1000L; + } + } + + private static class TimestampWriter implements OrcValueWriter { + private static final TimestampWriter INSTANCE = new TimestampWriter(); + + @Override + public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + cv.setIsUTC(true); + // millis + OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC); + cv.time[rowId] = + offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000; + // truncate nanos to only keep microsecond precision. + cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000; + } + } + + private static class TimestampTzWriter implements OrcValueWriter { + private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); + + @SuppressWarnings("JavaInstantGetSecondsGetNano") + @Override + public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { + TimestampColumnVector cv = (TimestampColumnVector) output; + // millis + Instant instant = data.toInstant(); + cv.time[rowId] = instant.toEpochMilli(); + // truncate nanos to only keep microsecond precision. + cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000; + } + } + + private static class Decimal18Writer implements OrcValueWriter { + private final int precision; + private final int scale; + + Decimal18Writer(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { + Preconditions.checkArgument( + scale == data.scale(), + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + data); + Preconditions.checkArgument( + data.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + data); + + ((DecimalColumnVector) output) + .vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale()); + } + } + + private static class Decimal38Writer implements OrcValueWriter { + private final int precision; + private final int scale; + + Decimal38Writer(int precision, int scale) { + this.precision = precision; + this.scale = scale; + } + + @Override + public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { + Preconditions.checkArgument( + scale == data.scale(), + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + data); + Preconditions.checkArgument( + data.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + data); + + ((DecimalColumnVector) output) + .vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false)); + } + } + + static class ListWriter implements OrcValueWriter { + private final OrcValueWriter elementWriter; + private final ArrayData.ElementGetter elementGetter; + + ListWriter(OrcValueWriter elementWriter, LogicalType elementType) { + this.elementWriter = elementWriter; + this.elementGetter = ArrayData.createElementGetter(elementType); + } + + @Override + @SuppressWarnings("unchecked") + public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { + ListColumnVector cv = (ListColumnVector) output; + cv.lengths[rowId] = data.size(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough. + growColumnVector(cv.child, cv.childCount); + + for (int e = 0; e < cv.lengths[rowId]; ++e) { + Object value = elementGetter.getElementOrNull(data, e); + elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child); + } + } + + @Override + public Stream> metrics() { + return elementWriter.metrics(); + } + } + + static class MapWriter implements OrcValueWriter { + private final OrcValueWriter keyWriter; + private final OrcValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + MapWriter( + OrcValueWriter keyWriter, + OrcValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.valueWriter = valueWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void nonNullWrite(int rowId, MapData data, ColumnVector output) { + MapColumnVector cv = (MapColumnVector) output; + ArrayData keyArray = data.keyArray(); + ArrayData valArray = data.valueArray(); + + // record the length and start of the list elements + cv.lengths[rowId] = data.size(); + cv.offsets[rowId] = cv.childCount; + cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); + // make sure the child is big enough + growColumnVector(cv.keys, cv.childCount); + growColumnVector(cv.values, cv.childCount); + // Add each element + for (int e = 0; e < cv.lengths[rowId]; ++e) { + int pos = (int) (e + cv.offsets[rowId]); + keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys); + valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values); + } + } + + @Override + public Stream> metrics() { + return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); + } + } + + static class RowDataWriter extends GenericOrcWriters.StructWriter { + private final List fieldGetters; + + RowDataWriter(List> writers, List types) { + super(writers); + + this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); + for (int i = 0; i < types.size(); i++) { + fieldGetters.add(RowData.createFieldGetter(types.get(i), i)); + } + } + + @Override + protected Object get(RowData struct, int index) { + return fieldGetters.get(index).getFieldOrNull(struct); + } + } + + private static void growColumnVector(ColumnVector cv, int requestedSize) { + if (cv.isNull.length < requestedSize) { + // Use growth factor of 3 to avoid frequent array allocations + cv.ensureSize(requestedSize * 3, true); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java new file mode 100644 index 000000000000..a5f2bb738960 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java @@ -0,0 +1,905 @@ +/* + * 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.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.MetadataColumns; +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.TypeWithSchemaVisitor; +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.types.Types; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.LogicalTypeAnnotation; +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; + +public class FlinkParquetReaders { + private FlinkParquetReaders() {} + + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema) { + return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); + } + + @SuppressWarnings("unchecked") + public static ParquetValueReader buildReader( + Schema expectedSchema, MessageType fileSchema, Map idToConstant) { + return (ParquetValueReader) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); + } + + private static class ReadBuilder extends TypeWithSchemaVisitor> { + private final MessageType type; + private final Map idToConstant; + + ReadBuilder(MessageType type, Map idToConstant) { + this.type = type; + this.idToConstant = idToConstant; + } + + @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); + if (fieldReaders.get(i) != null) { + int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; + if (fieldType.getId() != null) { + int id = fieldType.getId().intValue(); + readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); + typesById.put(id, fieldType); + if (idToConstant.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 (idToConstant.containsKey(id)) { + // containsKey is used because the constant may be null + int fieldMaxDefinitionLevel = + maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); + reorderedFields.add( + ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel)); + types.add(null); + } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { + reorderedFields.add(ParquetValueReaders.position()); + types.add(null); + } else if (id == MetadataColumns.IS_DELETED.fieldId()) { + reorderedFields.add(ParquetValueReaders.constant(false)); + 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 RowDataReader(types, reorderedFields); + } + + @Override + public ParquetValueReader list( + Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { + if (expectedList == null) { + return null; + } + + 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) { + if (expectedMap == null) { + return null; + } + + 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)); + } + + private static class LogicalTypeAnnotationParquetValueReaderVisitor + implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { + + private final PrimitiveType primitive; + private final ColumnDescriptor desc; + private final org.apache.iceberg.types.Type.PrimitiveType expected; + + LogicalTypeAnnotationParquetValueReaderVisitor( + PrimitiveType primitive, + ColumnDescriptor desc, + org.apache.iceberg.types.Type.PrimitiveType expected) { + this.primitive = primitive; + this.desc = desc; + this.expected = expected; + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { + return Optional.of(new StringReader(desc)); + } + + @Override + public Optional> visit( + DecimalLogicalTypeAnnotation decimalLogicalType) { + switch (primitive.getPrimitiveTypeName()) { + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return Optional.of( + new BinaryDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT64: + return Optional.of( + new LongDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + case INT32: + return Optional.of( + new IntegerDecimalReader( + desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(decimalLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { + if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + return Optional.of(new MillisTimeReader(desc)); + } else if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + return Optional.of(new LossyMicrosToMillisTimeReader(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timeLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { + if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MillisToTimestampTzReader(desc)); + } else { + return Optional.of(new MillisToTimestampReader(desc)); + } + } else if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { + if (timestampLogicalType.isAdjustedToUTC()) { + return Optional.of(new MicrosToTimestampTzReader(desc)); + } else { + return Optional.of(new MicrosToTimestampReader(desc)); + } + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timestampLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { + int width = intLogicalType.getBitWidth(); + if (width <= 32) { + if (expected.typeId() == Types.LongType.get().typeId()) { + return Optional.of(new ParquetValueReaders.IntAsLongReader(desc)); + } else { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + } else if (width <= 64) { + return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); + } + + return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(intLogicalType); + } + + @Override + public Optional> visit( + LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { + return Optional.of(new ParquetValueReaders.ByteArrayReader(desc)); + } + } + + @Override + @SuppressWarnings("CyclomaticComplexity") + public ParquetValueReader primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { + if (expected == null) { + return null; + } + + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); + if (logicalTypeAnnotation != null) { + return logicalTypeAnnotation + .accept(new LogicalTypeAnnotationParquetValueReaderVisitor(primitive, desc, expected)) + .orElseThrow( + () -> + new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return new ParquetValueReaders.ByteArrayReader(desc); + case INT32: + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { + return new ParquetValueReaders.IntAsLongReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case FLOAT: + if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { + return new ParquetValueReaders.FloatAsDoubleReader(desc); + } else { + return new ParquetValueReaders.UnboxedReader<>(desc); + } + case BOOLEAN: + case INT64: + case DOUBLE: + return new ParquetValueReaders.UnboxedReader<>(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static class BinaryDecimalReader + extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + Binary binary = column.nextBinary(); + BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale); + // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader + return DecimalData.fromBigDecimal(bigDecimal, precision, scale); + } + } + + private static class IntegerDecimalReader + extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale); + } + } + + private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { + private final int precision; + private final int scale; + + LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(DecimalData ignored) { + return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale); + } + } + + private static class MicrosToTimestampTzReader + extends ParquetValueReaders.UnboxedReader { + MicrosToTimestampTzReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long value = readLong(); + return TimestampData.fromLocalDateTime( + Instant.ofEpochSecond( + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L) + .atOffset(ZoneOffset.UTC) + .toLocalDateTime()); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MicrosToTimestampReader + extends ParquetValueReaders.UnboxedReader { + MicrosToTimestampReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long value = readLong(); + return TimestampData.fromInstant( + Instant.ofEpochSecond( + Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L)); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MillisToTimestampReader + extends ParquetValueReaders.UnboxedReader { + MillisToTimestampReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long millis = readLong(); + return TimestampData.fromEpochMillis(millis); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class MillisToTimestampTzReader + extends ParquetValueReaders.UnboxedReader { + MillisToTimestampTzReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public TimestampData read(TimestampData ignored) { + long millis = readLong(); + return TimestampData.fromLocalDateTime( + Instant.ofEpochMilli(millis).atOffset(ZoneOffset.UTC).toLocalDateTime()); + } + + @Override + public long readLong() { + return column.nextLong(); + } + } + + private static class StringReader extends ParquetValueReaders.PrimitiveReader { + StringReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public StringData read(StringData ignored) { + Binary binary = column.nextBinary(); + ByteBuffer buffer = binary.toByteBuffer(); + if (buffer.hasArray()) { + return StringData.fromBytes( + buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); + } else { + return StringData.fromBytes(binary.getBytes()); + } + } + } + + private static class LossyMicrosToMillisTimeReader + extends ParquetValueReaders.PrimitiveReader { + LossyMicrosToMillisTimeReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Integer read(Integer reuse) { + // Discard microseconds since Flink uses millisecond unit for TIME type. + return (int) Math.floorDiv(column.nextLong(), 1000L); + } + } + + private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader { + MillisTimeReader(ColumnDescriptor desc) { + super(desc); + } + + @Override + public Integer read(Integer reuse) { + return (int) column.nextLong(); + } + } + + private static class ArrayReader + extends ParquetValueReaders.RepeatedReader { + private int readPos = 0; + private int writePos = 0; + + ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { + super(definitionLevel, repetitionLevel, reader); + } + + @Override + protected ReusableArrayData newListData(ArrayData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableArrayData) { + return (ReusableArrayData) reuse; + } else { + return new ReusableArrayData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected E getElement(ReusableArrayData list) { + E value = null; + if (readPos < list.capacity()) { + value = (E) list.values[readPos]; + } + + readPos += 1; + + return value; + } + + @Override + protected void addElement(ReusableArrayData reused, E element) { + if (writePos >= reused.capacity()) { + reused.grow(); + } + + reused.values[writePos] = element; + + writePos += 1; + } + + @Override + protected ArrayData buildList(ReusableArrayData list) { + // Since ReusableArrayData is not accepted by Flink, use GenericArrayData temporarily to walk + // around it. + // Revert this to use ReusableArrayData once it is fixed in Flink. + // For your reference, https://issues.apache.org/jira/browse/FLINK-25238. + return new GenericArrayData(Arrays.copyOf(list.values, writePos)); + } + } + + private static class MapReader + extends ParquetValueReaders.RepeatedKeyValueReader { + private int readPos = 0; + private int writePos = 0; + + private final ParquetValueReaders.ReusableEntry entry = + new ParquetValueReaders.ReusableEntry<>(); + private final ParquetValueReaders.ReusableEntry nullEntry = + new ParquetValueReaders.ReusableEntry<>(); + + MapReader( + int definitionLevel, + int repetitionLevel, + ParquetValueReader keyReader, + ParquetValueReader valueReader) { + super(definitionLevel, repetitionLevel, keyReader, valueReader); + } + + @Override + protected ReusableMapData newMapData(MapData reuse) { + this.readPos = 0; + this.writePos = 0; + + if (reuse instanceof ReusableMapData) { + return (ReusableMapData) reuse; + } else { + return new ReusableMapData(); + } + } + + @Override + @SuppressWarnings("unchecked") + protected Map.Entry getPair(ReusableMapData map) { + Map.Entry kv = nullEntry; + if (readPos < map.capacity()) { + entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); + kv = entry; + } + + readPos += 1; + + return kv; + } + + @Override + protected void addPair(ReusableMapData map, K key, V value) { + if (writePos >= map.capacity()) { + map.grow(); + } + + map.keys.values[writePos] = key; + map.values.values[writePos] = value; + + writePos += 1; + } + + @Override + protected MapData buildMap(ReusableMapData map) { + map.setNumElements(writePos); + return map; + } + } + + private static class RowDataReader + extends ParquetValueReaders.StructReader { + private final int numFields; + + RowDataReader(List types, List> readers) { + super(types, readers); + this.numFields = readers.size(); + } + + @Override + protected GenericRowData newStructData(RowData reuse) { + if (reuse instanceof GenericRowData) { + return (GenericRowData) reuse; + } else { + return new GenericRowData(numFields); + } + } + + @Override + protected Object getField(GenericRowData intermediate, int pos) { + return intermediate.getField(pos); + } + + @Override + protected RowData buildStruct(GenericRowData struct) { + return struct; + } + + @Override + protected void set(GenericRowData row, int pos, Object value) { + row.setField(pos, value); + } + + @Override + protected void setNull(GenericRowData row, int pos) { + row.setField(pos, null); + } + + @Override + protected void setBoolean(GenericRowData row, int pos, boolean value) { + row.setField(pos, value); + } + + @Override + protected void setInteger(GenericRowData row, int pos, int value) { + row.setField(pos, value); + } + + @Override + protected void setLong(GenericRowData row, int pos, long value) { + row.setField(pos, value); + } + + @Override + protected void setFloat(GenericRowData row, int pos, float value) { + row.setField(pos, value); + } + + @Override + protected void setDouble(GenericRowData row, int pos, double value) { + row.setField(pos, value); + } + } + + private static class ReusableMapData implements MapData { + private final ReusableArrayData keys; + private final ReusableArrayData values; + + private int numElements; + + private ReusableMapData() { + this.keys = new ReusableArrayData(); + this.values = new ReusableArrayData(); + } + + private void grow() { + keys.grow(); + values.grow(); + } + + private int capacity() { + return keys.capacity(); + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + keys.setNumElements(numElements); + values.setNumElements(numElements); + } + + @Override + public int size() { + return numElements; + } + + @Override + public ReusableArrayData keyArray() { + return keys; + } + + @Override + public ReusableArrayData valueArray() { + return values; + } + } + + private static class ReusableArrayData implements ArrayData { + private static final Object[] EMPTY = new Object[0]; + + private Object[] values = EMPTY; + private int numElements = 0; + + private void grow() { + if (values.length == 0) { + this.values = new Object[20]; + } else { + Object[] old = values; + this.values = new Object[old.length << 1]; + // copy the old array in case it has values that can be reused + System.arraycopy(old, 0, values, 0, old.length); + } + } + + private int capacity() { + return values.length; + } + + public void setNumElements(int numElements) { + this.numElements = numElements; + } + + @Override + public int size() { + return numElements; + } + + @Override + public boolean isNullAt(int ordinal) { + return null == values[ordinal]; + } + + @Override + public boolean getBoolean(int ordinal) { + return (boolean) values[ordinal]; + } + + @Override + public byte getByte(int ordinal) { + return (byte) values[ordinal]; + } + + @Override + public short getShort(int ordinal) { + return (short) values[ordinal]; + } + + @Override + public int getInt(int ordinal) { + return (int) values[ordinal]; + } + + @Override + public long getLong(int ordinal) { + return (long) values[ordinal]; + } + + @Override + public float getFloat(int ordinal) { + return (float) values[ordinal]; + } + + @Override + public double getDouble(int ordinal) { + return (double) values[ordinal]; + } + + @Override + public StringData getString(int pos) { + return (StringData) values[pos]; + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return (DecimalData) values[pos]; + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return (TimestampData) values[pos]; + } + + @SuppressWarnings("unchecked") + @Override + public RawValueData getRawValue(int pos) { + return (RawValueData) values[pos]; + } + + @Override + public byte[] getBinary(int ordinal) { + return (byte[]) values[ordinal]; + } + + @Override + public ArrayData getArray(int ordinal) { + return (ArrayData) values[ordinal]; + } + + @Override + public MapData getMap(int ordinal) { + return (MapData) values[ordinal]; + } + + @Override + public RowData getRow(int pos, int numFields) { + return (RowData) values[pos]; + } + + @Override + public boolean[] toBooleanArray() { + return ArrayUtil.toPrimitive((Boolean[]) values); + } + + @Override + public byte[] toByteArray() { + return ArrayUtil.toPrimitive((Byte[]) values); + } + + @Override + public short[] toShortArray() { + return ArrayUtil.toPrimitive((Short[]) values); + } + + @Override + public int[] toIntArray() { + return ArrayUtil.toPrimitive((Integer[]) values); + } + + @Override + public long[] toLongArray() { + return ArrayUtil.toPrimitive((Long[]) values); + } + + @Override + public float[] toFloatArray() { + return ArrayUtil.toPrimitive((Float[]) values); + } + + @Override + public double[] toDoubleArray() { + return ArrayUtil.toPrimitive((Double[]) values); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java new file mode 100644 index 000000000000..db4f1730a134 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java @@ -0,0 +1,504 @@ +/* + * 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.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NoSuchElementException; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; +import org.apache.flink.table.types.logical.SmallIntType; +import org.apache.flink.table.types.logical.TinyIntType; +import org.apache.iceberg.parquet.ParquetValueReaders; +import org.apache.iceberg.parquet.ParquetValueWriter; +import org.apache.iceberg.parquet.ParquetValueWriters; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; +import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.io.api.Binary; +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; + +public class FlinkParquetWriters { + private FlinkParquetWriters() {} + + @SuppressWarnings("unchecked") + public static ParquetValueWriter buildWriter(LogicalType schema, MessageType type) { + return (ParquetValueWriter) + ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type)); + } + + private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor> { + private final MessageType type; + + WriteBuilder(MessageType type) { + this.type = type; + } + + @Override + public ParquetValueWriter message( + RowType sStruct, MessageType message, List> fields) { + return struct(sStruct, message.asGroupType(), fields); + } + + @Override + public ParquetValueWriter struct( + RowType sStruct, GroupType struct, List> fieldWriters) { + List fields = struct.getFields(); + List flinkFields = sStruct.getFields(); + List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); + List flinkTypes = Lists.newArrayList(); + for (int i = 0; i < fields.size(); i += 1) { + writers.add(newOption(struct.getType(i), fieldWriters.get(i))); + flinkTypes.add(flinkFields.get(i).getType()); + } + + return new RowDataWriter(writers, flinkTypes); + } + + @Override + public ParquetValueWriter list( + ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { + GroupType repeated = array.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new ArrayDataWriter<>( + repeatedD, + repeatedR, + newOption(repeated.getType(0), elementWriter), + sArray.getElementType()); + } + + @Override + public ParquetValueWriter map( + MapType sMap, + GroupType map, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter) { + GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); + String[] repeatedPath = currentPath(); + + int repeatedD = type.getMaxDefinitionLevel(repeatedPath); + int repeatedR = type.getMaxRepetitionLevel(repeatedPath); + + return new MapDataWriter<>( + repeatedD, + repeatedR, + newOption(repeatedKeyValue.getType(0), keyWriter), + newOption(repeatedKeyValue.getType(1), valueWriter), + sMap.getKeyType(), + sMap.getValueType()); + } + + private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { + int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); + return ParquetValueWriters.option(fieldType, maxD, writer); + } + + @Override + public ParquetValueWriter primitive(LogicalType fType, PrimitiveType primitive) { + ColumnDescriptor desc = type.getColumnDescription(currentPath()); + + if (primitive.getOriginalType() != null) { + switch (primitive.getOriginalType()) { + case ENUM: + case JSON: + case UTF8: + return strings(desc); + case DATE: + case INT_8: + case INT_16: + case INT_32: + return ints(fType, desc); + case INT_64: + return ParquetValueWriters.longs(desc); + case TIME_MICROS: + return timeMicros(desc); + case TIMESTAMP_MICROS: + return timestamps(desc); + case DECIMAL: + DecimalLogicalTypeAnnotation decimal = + (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); + switch (primitive.getPrimitiveTypeName()) { + case INT32: + return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); + case INT64: + return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); + case BINARY: + case FIXED_LEN_BYTE_ARRAY: + return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); + default: + throw new UnsupportedOperationException( + "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); + } + case BSON: + return byteArrays(desc); + default: + throw new UnsupportedOperationException( + "Unsupported logical type: " + primitive.getOriginalType()); + } + } + + switch (primitive.getPrimitiveTypeName()) { + case FIXED_LEN_BYTE_ARRAY: + case BINARY: + return byteArrays(desc); + case BOOLEAN: + return ParquetValueWriters.booleans(desc); + case INT32: + return ints(fType, desc); + case INT64: + return ParquetValueWriters.longs(desc); + case FLOAT: + return ParquetValueWriters.floats(desc); + case DOUBLE: + return ParquetValueWriters.doubles(desc); + default: + throw new UnsupportedOperationException("Unsupported type: " + primitive); + } + } + } + + private static ParquetValueWriters.PrimitiveWriter ints( + LogicalType type, ColumnDescriptor desc) { + if (type instanceof TinyIntType) { + return ParquetValueWriters.tinyints(desc); + } else if (type instanceof SmallIntType) { + return ParquetValueWriters.shorts(desc); + } + return ParquetValueWriters.ints(desc); + } + + private static ParquetValueWriters.PrimitiveWriter strings(ColumnDescriptor desc) { + return new StringDataWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter timeMicros(ColumnDescriptor desc) { + return new TimeMicrosWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( + ColumnDescriptor desc, int precision, int scale) { + Preconditions.checkArgument( + precision <= 9, + "Cannot write decimal value as integer with precision larger than 9," + + " wrong precision %s", + precision); + return new IntegerDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsLong( + ColumnDescriptor desc, int precision, int scale) { + Preconditions.checkArgument( + precision <= 18, + "Cannot write decimal value as long with precision larger than 18, " + + " wrong precision %s", + precision); + return new LongDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( + ColumnDescriptor desc, int precision, int scale) { + return new FixedDecimalWriter(desc, precision, scale); + } + + private static ParquetValueWriters.PrimitiveWriter timestamps( + ColumnDescriptor desc) { + return new TimestampDataWriter(desc); + } + + private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { + return new ByteArrayWriter(desc); + } + + private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter { + private StringDataWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, StringData value) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes())); + } + } + + private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter { + private TimeMicrosWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, Integer value) { + long micros = value.longValue() * 1000; + column.writeLong(repetitionLevel, micros); + } + } + + private static class IntegerDecimalWriter + extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + + private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); + } + } + + private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + + private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + Preconditions.checkArgument( + decimal.scale() == scale, + "Cannot write value as decimal(%s,%s), wrong scale: %s", + precision, + scale, + decimal); + Preconditions.checkArgument( + decimal.precision() <= precision, + "Cannot write value as decimal(%s,%s), too large: %s", + precision, + scale, + decimal); + + column.writeLong(repetitionLevel, decimal.toUnscaledLong()); + } + } + + private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { + super(desc); + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(int repetitionLevel, DecimalData decimal) { + byte[] binary = + DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get()); + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); + } + } + + private static class TimestampDataWriter + extends ParquetValueWriters.PrimitiveWriter { + private TimestampDataWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, TimestampData value) { + column.writeLong( + repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000); + } + } + + private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { + private ByteArrayWriter(ColumnDescriptor desc) { + super(desc); + } + + @Override + public void write(int repetitionLevel, byte[] bytes) { + column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); + } + } + + private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { + private final LogicalType elementType; + + private ArrayDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter writer, + LogicalType elementType) { + super(definitionLevel, repetitionLevel, writer); + this.elementType = elementType; + } + + @Override + protected Iterator elements(ArrayData list) { + return new ElementIterator<>(list); + } + + private class ElementIterator implements Iterator { + private final int size; + private final ArrayData list; + private final ArrayData.ElementGetter getter; + private int index; + + private ElementIterator(ArrayData list) { + this.list = list; + size = list.size(); + getter = ArrayData.createElementGetter(elementType); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public E next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + E element = (E) getter.getElementOrNull(list, index); + index += 1; + + return element; + } + } + } + + private static class MapDataWriter + extends ParquetValueWriters.RepeatedKeyValueWriter { + private final LogicalType keyType; + private final LogicalType valueType; + + private MapDataWriter( + int definitionLevel, + int repetitionLevel, + ParquetValueWriter keyWriter, + ParquetValueWriter valueWriter, + LogicalType keyType, + LogicalType valueType) { + super(definitionLevel, repetitionLevel, keyWriter, valueWriter); + this.keyType = keyType; + this.valueType = valueType; + } + + @Override + protected Iterator> pairs(MapData map) { + return new EntryIterator<>(map); + } + + private class EntryIterator implements Iterator> { + private final int size; + private final ArrayData keys; + private final ArrayData values; + private final ParquetValueReaders.ReusableEntry entry; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + private int index; + + private EntryIterator(MapData map) { + size = map.size(); + keys = map.keyArray(); + values = map.valueArray(); + entry = new ParquetValueReaders.ReusableEntry<>(); + keyGetter = ArrayData.createElementGetter(keyType); + valueGetter = ArrayData.createElementGetter(valueType); + index = 0; + } + + @Override + public boolean hasNext() { + return index != size; + } + + @Override + @SuppressWarnings("unchecked") + public Map.Entry next() { + if (index >= size) { + throw new NoSuchElementException(); + } + + entry.set( + (K) keyGetter.getElementOrNull(keys, index), + (V) valueGetter.getElementOrNull(values, index)); + index += 1; + + return entry; + } + } + } + + private static class RowDataWriter extends ParquetValueWriters.StructWriter { + private final RowData.FieldGetter[] fieldGetter; + + RowDataWriter(List> writers, List types) { + super(writers); + fieldGetter = new RowData.FieldGetter[types.size()]; + for (int i = 0; i < types.size(); i += 1) { + fieldGetter[i] = RowData.createFieldGetter(types.get(i), i); + } + } + + @Override + protected Object get(RowData struct, int index) { + return fieldGetter[index].getFieldOrNull(struct); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java new file mode 100644 index 000000000000..ba4e1a7a7aec --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java @@ -0,0 +1,161 @@ +/* + * 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.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +abstract class FlinkSchemaVisitor { + + static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { + return visit(flinkType, schema.asStruct(), visitor); + } + + private static T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor visitor) { + switch (iType.typeId()) { + case STRUCT: + return visitRecord(flinkType, iType.asStructType(), visitor); + + case MAP: + MapType mapType = (MapType) flinkType; + Types.MapType iMapType = iType.asMapType(); + T key; + T value; + + Types.NestedField keyField = iMapType.field(iMapType.keyId()); + visitor.beforeMapKey(keyField); + try { + key = visit(mapType.getKeyType(), iMapType.keyType(), visitor); + } finally { + visitor.afterMapKey(keyField); + } + + Types.NestedField valueField = iMapType.field(iMapType.valueId()); + visitor.beforeMapValue(valueField); + try { + value = visit(mapType.getValueType(), iMapType.valueType(), visitor); + } finally { + visitor.afterMapValue(valueField); + } + + return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType()); + + case LIST: + ArrayType listType = (ArrayType) flinkType; + Types.ListType iListType = iType.asListType(); + T element; + + Types.NestedField elementField = iListType.field(iListType.elementId()); + visitor.beforeListElement(elementField); + try { + element = visit(listType.getElementType(), iListType.elementType(), visitor); + } finally { + visitor.afterListElement(elementField); + } + + return visitor.list(iListType, element, listType.getElementType()); + + default: + return visitor.primitive(iType.asPrimitiveType(), flinkType); + } + } + + private static T visitRecord( + LogicalType flinkType, Types.StructType struct, FlinkSchemaVisitor visitor) { + Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType); + RowType rowType = (RowType) flinkType; + + int fieldSize = struct.fields().size(); + List results = Lists.newArrayListWithExpectedSize(fieldSize); + List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); + List nestedFields = struct.fields(); + + for (int i = 0; i < fieldSize; i++) { + Types.NestedField iField = nestedFields.get(i); + int fieldIndex = rowType.getFieldIndex(iField.name()); + Preconditions.checkArgument( + fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); + + LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); + + fieldTypes.add(fieldFlinkType); + + visitor.beforeField(iField); + try { + results.add(visit(fieldFlinkType, iField.type(), visitor)); + } finally { + visitor.afterField(iField); + } + } + + return visitor.record(struct, results, fieldTypes); + } + + public T record(Types.StructType iStruct, List results, List fieldTypes) { + return null; + } + + public T list(Types.ListType iList, T element, LogicalType elementType) { + return null; + } + + public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) { + return null; + } + + public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { + return null; + } + + public void beforeField(Types.NestedField field) {} + + public void afterField(Types.NestedField field) {} + + public void beforeListElement(Types.NestedField elementField) { + beforeField(elementField); + } + + public void afterListElement(Types.NestedField elementField) { + afterField(elementField); + } + + public void beforeMapKey(Types.NestedField keyField) { + beforeField(keyField); + } + + public void afterMapKey(Types.NestedField keyField) { + afterField(keyField); + } + + public void beforeMapValue(Types.NestedField valueField) { + beforeField(valueField); + } + + public void afterMapValue(Types.NestedField valueField) { + afterField(valueField); + } +} 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 new file mode 100644 index 000000000000..32f6c3a2ccfd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.data; + +import java.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.charset.StandardCharsets; +import java.util.List; +import java.util.Map; +import org.apache.avro.io.Decoder; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.avro.ValueReader; +import org.apache.iceberg.avro.ValueReaders; +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; + +public class FlinkValueReaders { + + private FlinkValueReaders() {} + + static ValueReader strings() { + return StringReader.INSTANCE; + } + + static ValueReader enums(List symbols) { + return new EnumReader(symbols); + } + + static ValueReader uuids() { + return ValueReaders.fixed(16); + } + + static ValueReader timeMicros() { + return TimeMicrosReader.INSTANCE; + } + + static ValueReader timestampMills() { + return TimestampMillsReader.INSTANCE; + } + + static ValueReader timestampMicros() { + return TimestampMicrosReader.INSTANCE; + } + + static ValueReader decimal( + ValueReader unscaledReader, int precision, int scale) { + return new DecimalReader(unscaledReader, precision, scale); + } + + static ValueReader array(ValueReader elementReader) { + return new ArrayReader(elementReader); + } + + static ValueReader arrayMap(ValueReader keyReader, ValueReader valueReader) { + return new ArrayMapReader(keyReader, valueReader); + } + + static ValueReader map(ValueReader keyReader, ValueReader valueReader) { + return new MapReader(keyReader, valueReader); + } + + static ValueReader struct( + List> readers, Types.StructType struct, Map idToConstant) { + return new StructReader(readers, struct, idToConstant); + } + + private static class StringReader implements ValueReader { + private static final StringReader INSTANCE = new StringReader(); + + private StringReader() {} + + @Override + public StringData read(Decoder decoder, Object reuse) throws IOException { + // use the decoder's readString(Utf8) method because it may be a resolving decoder + Utf8 utf8 = null; + if (reuse instanceof StringData) { + utf8 = new Utf8(((StringData) reuse).toBytes()); + } + + Utf8 string = decoder.readString(utf8); + return StringData.fromBytes(string.getBytes(), 0, string.getByteLength()); + } + } + + private static class EnumReader implements ValueReader { + private final StringData[] symbols; + + private EnumReader(List symbols) { + this.symbols = new StringData[symbols.size()]; + for (int i = 0; i < this.symbols.length; i += 1) { + this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); + } + } + + @Override + public StringData read(Decoder decoder, Object ignore) throws IOException { + int index = decoder.readEnum(); + return symbols[index]; + } + } + + private static class DecimalReader implements ValueReader { + private final ValueReader bytesReader; + private final int precision; + private final int scale; + + private DecimalReader(ValueReader bytesReader, int precision, int scale) { + this.bytesReader = bytesReader; + this.precision = precision; + this.scale = scale; + } + + @Override + public DecimalData read(Decoder decoder, Object reuse) throws IOException { + byte[] bytes = bytesReader.read(decoder, null); + return DecimalData.fromBigDecimal( + new BigDecimal(new BigInteger(bytes), scale), precision, scale); + } + } + + private static class TimeMicrosReader implements ValueReader { + private static final TimeMicrosReader INSTANCE = new TimeMicrosReader(); + + @Override + public Integer read(Decoder decoder, Object reuse) throws IOException { + long micros = decoder.readLong(); + // Flink only support time mills, just erase micros. + return (int) (micros / 1000); + } + } + + private static class TimestampMillsReader implements ValueReader { + private static final TimestampMillsReader INSTANCE = new TimestampMillsReader(); + + @Override + public TimestampData read(Decoder decoder, Object reuse) throws IOException { + return TimestampData.fromEpochMillis(decoder.readLong()); + } + } + + private static class TimestampMicrosReader implements ValueReader { + private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader(); + + @Override + public TimestampData read(Decoder decoder, Object reuse) throws IOException { + long micros = decoder.readLong(); + long mills = micros / 1000; + int nanos = ((int) (micros % 1000)) * 1000; + if (nanos < 0) { + nanos += 1_000_000; + mills -= 1; + } + return TimestampData.fromEpochMillis(mills, nanos); + } + } + + private static class ArrayReader implements ValueReader { + private final ValueReader elementReader; + private final List reusedList = Lists.newArrayList(); + + private ArrayReader(ValueReader elementReader) { + this.elementReader = elementReader; + } + + @Override + public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { + reusedList.clear(); + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedList.add(elementReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + // this will convert the list to an array so it is okay to reuse the list + return new GenericArrayData(reusedList.toArray()); + } + } + + private static MapData kvArrayToMap(List keyList, List valueList) { + Map map = Maps.newHashMap(); + Object[] keys = keyList.toArray(); + Object[] values = valueList.toArray(); + for (int i = 0; i < keys.length; i++) { + map.put(keys[i], values[i]); + } + + return new GenericMapData(map); + } + + private static class ArrayMapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readArrayStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.arrayNext(); + } + + return kvArrayToMap(reusedKeyList, reusedValueList); + } + } + + private static class MapReader implements ValueReader { + private final ValueReader keyReader; + private final ValueReader valueReader; + + private final List reusedKeyList = Lists.newArrayList(); + private final List reusedValueList = Lists.newArrayList(); + + private MapReader(ValueReader keyReader, ValueReader valueReader) { + this.keyReader = keyReader; + this.valueReader = valueReader; + } + + @Override + public MapData read(Decoder decoder, Object reuse) throws IOException { + reusedKeyList.clear(); + reusedValueList.clear(); + + long chunkLength = decoder.readMapStart(); + + while (chunkLength > 0) { + for (int i = 0; i < chunkLength; i += 1) { + reusedKeyList.add(keyReader.read(decoder, null)); + reusedValueList.add(valueReader.read(decoder, null)); + } + + chunkLength = decoder.mapNext(); + } + + return kvArrayToMap(reusedKeyList, reusedValueList); + } + } + + private static class StructReader extends ValueReaders.StructReader { + private final int numFields; + + private StructReader( + List> readers, Types.StructType struct, Map idToConstant) { + super(readers, struct, idToConstant); + this.numFields = readers.size(); + } + + @Override + protected RowData reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { + return (GenericRowData) 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); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java new file mode 100644 index 000000000000..4e86ecce28b5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java @@ -0,0 +1,253 @@ +/* + * 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.lang.reflect.Array; +import java.util.List; +import org.apache.avro.io.Encoder; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.avro.ValueWriter; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.DecimalUtil; + +public class FlinkValueWriters { + + private FlinkValueWriters() {} + + static ValueWriter strings() { + return StringWriter.INSTANCE; + } + + static ValueWriter timeMicros() { + return TimeMicrosWriter.INSTANCE; + } + + static ValueWriter timestampMicros() { + return TimestampMicrosWriter.INSTANCE; + } + + static ValueWriter decimal(int precision, int scale) { + return new DecimalWriter(precision, scale); + } + + static ValueWriter array(ValueWriter elementWriter, LogicalType elementType) { + return new ArrayWriter<>(elementWriter, elementType); + } + + static ValueWriter arrayMap( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter map( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); + } + + static ValueWriter row(List> writers, List types) { + return new RowWriter(writers, types); + } + + private static class StringWriter implements ValueWriter { + private static final StringWriter INSTANCE = new StringWriter(); + + private StringWriter() {} + + @Override + public void write(StringData s, Encoder encoder) throws IOException { + // toBytes is cheaper than Avro calling toString, which incurs encoding costs + encoder.writeString(new Utf8(s.toBytes())); + } + } + + private static class DecimalWriter implements ValueWriter { + private final int precision; + private final int scale; + private final ThreadLocal bytes; + + private DecimalWriter(int precision, int scale) { + this.precision = precision; + this.scale = scale; + this.bytes = + ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); + } + + @Override + public void write(DecimalData d, Encoder encoder) throws IOException { + encoder.writeFixed( + DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get())); + } + } + + private static class TimeMicrosWriter implements ValueWriter { + private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter(); + + @Override + public void write(Integer timeMills, Encoder encoder) throws IOException { + encoder.writeLong(timeMills * 1000L); + } + } + + private static class TimestampMicrosWriter implements ValueWriter { + private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter(); + + @Override + public void write(TimestampData timestampData, Encoder encoder) throws IOException { + long micros = + timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; + encoder.writeLong(micros); + } + } + + private static class ArrayWriter implements ValueWriter { + private final ValueWriter elementWriter; + private final ArrayData.ElementGetter elementGetter; + + private ArrayWriter(ValueWriter elementWriter, LogicalType elementType) { + this.elementWriter = elementWriter; + this.elementGetter = ArrayData.createElementGetter(elementType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(ArrayData array, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = array.size(); + encoder.setItemCount(numElements); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class ArrayMapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private ArrayMapWriter( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueWriter = valueWriter; + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeArrayStart(); + int numElements = map.size(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); + valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); + } + encoder.writeArrayEnd(); + } + } + + private static class MapWriter implements ValueWriter { + private final ValueWriter keyWriter; + private final ValueWriter valueWriter; + private final ArrayData.ElementGetter keyGetter; + private final ArrayData.ElementGetter valueGetter; + + private MapWriter( + ValueWriter keyWriter, + LogicalType keyType, + ValueWriter valueWriter, + LogicalType valueType) { + this.keyWriter = keyWriter; + this.keyGetter = ArrayData.createElementGetter(keyType); + this.valueWriter = valueWriter; + this.valueGetter = ArrayData.createElementGetter(valueType); + } + + @Override + @SuppressWarnings("unchecked") + public void write(MapData map, Encoder encoder) throws IOException { + encoder.writeMapStart(); + int numElements = map.size(); + encoder.setItemCount(numElements); + ArrayData keyArray = map.keyArray(); + ArrayData valueArray = map.valueArray(); + for (int i = 0; i < numElements; i += 1) { + encoder.startItem(); + keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); + valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); + } + encoder.writeMapEnd(); + } + } + + static class RowWriter implements ValueWriter { + private final ValueWriter[] writers; + private final RowData.FieldGetter[] getters; + + private RowWriter(List> writers, List types) { + this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); + this.getters = new RowData.FieldGetter[writers.size()]; + for (int i = 0; i < writers.size(); i += 1) { + this.writers[i] = writers.get(i); + this.getters[i] = RowData.createFieldGetter(types.get(i), i); + } + } + + @Override + public void write(RowData row, Encoder encoder) throws IOException { + for (int i = 0; i < writers.length; i += 1) { + if (row.isNullAt(i)) { + writers[i].write(null, encoder); + } else { + write(row, i, writers[i], encoder); + } + } + } + + @SuppressWarnings("unchecked") + private void write(RowData row, int pos, ValueWriter writer, Encoder encoder) + throws IOException { + writer.write((T) getters[pos].getFieldOrNull(row), encoder); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java new file mode 100644 index 000000000000..33feb2e32118 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java @@ -0,0 +1,222 @@ +/* + * 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.util.Deque; +import java.util.List; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.RowType.RowField; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.parquet.schema.GroupType; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.OriginalType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; + +public class ParquetWithFlinkSchemaVisitor { + private final Deque fieldNames = Lists.newLinkedList(); + + public static T visit( + LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor visitor) { + Preconditions.checkArgument(sType != null, "Invalid DataType: null"); + if (type instanceof MessageType) { + Preconditions.checkArgument( + sType instanceof RowType, "Invalid struct: %s is not a struct", sType); + RowType struct = (RowType) sType; + return visitor.message( + struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); + } else if (type.isPrimitive()) { + return visitor.primitive(sType, type.asPrimitiveType()); + } else { + // if not a primitive, the typeId must be a group + GroupType group = type.asGroupType(); + OriginalType annotation = group.getOriginalType(); + if (annotation != null) { + switch (annotation) { + case LIST: + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid list: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid list: does not contain single repeated field: %s", + group); + + GroupType repeatedElement = group.getFields().get(0).asGroupType(); + Preconditions.checkArgument( + repeatedElement.isRepetition(Type.Repetition.REPEATED), + "Invalid list: inner group is not repeated"); + Preconditions.checkArgument( + repeatedElement.getFieldCount() <= 1, + "Invalid list: repeated group is not a single field: %s", + group); + + Preconditions.checkArgument( + sType instanceof ArrayType, "Invalid list: %s is not an array", sType); + ArrayType array = (ArrayType) sType; + RowType.RowField element = + new RowField( + "element", array.getElementType(), "element of " + array.asSummaryString()); + + visitor.fieldNames.push(repeatedElement.getName()); + try { + T elementResult = null; + if (repeatedElement.getFieldCount() > 0) { + elementResult = visitField(element, repeatedElement.getType(0), visitor); + } + + return visitor.list(array, group, elementResult); + + } finally { + visitor.fieldNames.pop(); + } + + case MAP: + Preconditions.checkArgument( + !group.isRepetition(Type.Repetition.REPEATED), + "Invalid map: top-level group is repeated: %s", + group); + Preconditions.checkArgument( + group.getFieldCount() == 1, + "Invalid map: does not contain single repeated field: %s", + group); + + GroupType repeatedKeyValue = group.getType(0).asGroupType(); + Preconditions.checkArgument( + repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), + "Invalid map: inner group is not repeated"); + Preconditions.checkArgument( + repeatedKeyValue.getFieldCount() <= 2, + "Invalid map: repeated group does not have 2 fields"); + + Preconditions.checkArgument( + sType instanceof MapType, "Invalid map: %s is not a map", sType); + MapType map = (MapType) sType; + RowField keyField = + new RowField("key", map.getKeyType(), "key of " + map.asSummaryString()); + RowField valueField = + new RowField("value", map.getValueType(), "value of " + map.asSummaryString()); + + visitor.fieldNames.push(repeatedKeyValue.getName()); + try { + T keyResult = null; + T valueResult = null; + switch (repeatedKeyValue.getFieldCount()) { + case 2: + // if there are 2 fields, both key and value are projected + keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); + valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); + break; + case 1: + // if there is just one, use the name to determine what it is + Type keyOrValue = repeatedKeyValue.getType(0); + if (keyOrValue.getName().equalsIgnoreCase("key")) { + keyResult = visitField(keyField, keyOrValue, visitor); + // value result remains null + } else { + valueResult = visitField(valueField, keyOrValue, visitor); + // key result remains null + } + break; + default: + // both results will remain null + } + + return visitor.map(map, group, keyResult, valueResult); + + } finally { + visitor.fieldNames.pop(); + } + + default: + } + } + Preconditions.checkArgument( + sType instanceof RowType, "Invalid struct: %s is not a struct", sType); + RowType struct = (RowType) sType; + return visitor.struct(struct, group, visitFields(struct, group, visitor)); + } + } + + private static T visitField( + RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor visitor) { + visitor.fieldNames.push(field.getName()); + try { + return visit(sField.getType(), field, visitor); + } finally { + visitor.fieldNames.pop(); + } + } + + private static List visitFields( + RowType struct, GroupType group, ParquetWithFlinkSchemaVisitor visitor) { + List sFields = struct.getFields(); + Preconditions.checkArgument( + sFields.size() == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); + List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); + for (int i = 0; i < sFields.size(); i += 1) { + Type field = group.getFields().get(i); + RowType.RowField sField = sFields.get(i); + Preconditions.checkArgument( + field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())), + "Structs do not match: field %s != %s", + field.getName(), + sField.getName()); + results.add(visitField(sField, field, visitor)); + } + + return results; + } + + public T message(RowType sStruct, MessageType message, List fields) { + return null; + } + + public T struct(RowType sStruct, GroupType struct, List fields) { + return null; + } + + public T list(ArrayType sArray, GroupType array, T element) { + return null; + } + + public T map(MapType sMap, GroupType map, T key, T value) { + return null; + } + + public T primitive(LogicalType sPrimitive, PrimitiveType primitive) { + return null; + } + + protected String[] currentPath() { + return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); + } + + protected String[] path(String name) { + List list = Lists.newArrayList(fieldNames.descendingIterator()); + list.add(name); + return list.toArray(new String[0]); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java new file mode 100644 index 000000000000..33816c97ac29 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java @@ -0,0 +1,341 @@ +/* + * 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.util.Arrays; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +public class RowDataProjection implements RowData { + /** + * Creates a projecting wrapper for {@link RowData} rows. + * + *

    This projection will not project the nested children types of repeated types like lists and + * maps. + * + * @param schema schema of rows wrapped by this projection + * @param projectedSchema result schema of the projected rows + * @return a wrapper to project rows + */ + public static RowDataProjection create(Schema schema, Schema projectedSchema) { + return RowDataProjection.create( + FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct()); + } + + /** + * Creates a projecting wrapper for {@link RowData} rows. + * + *

    This projection will not project the nested children types of repeated types like lists and + * maps. + * + * @param rowType flink row type of rows wrapped by this projection + * @param schema schema of rows wrapped by this projection + * @param projectedSchema result schema of the projected rows + * @return a wrapper to project rows + */ + public static RowDataProjection create( + RowType rowType, Types.StructType schema, Types.StructType projectedSchema) { + return new RowDataProjection(rowType, schema, projectedSchema); + } + + private final RowData.FieldGetter[] getters; + private RowData rowData; + + private RowDataProjection( + RowType rowType, Types.StructType rowStruct, Types.StructType projectType) { + Map fieldIdToPosition = Maps.newHashMap(); + for (int i = 0; i < rowStruct.fields().size(); i++) { + fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i); + } + + this.getters = new RowData.FieldGetter[projectType.fields().size()]; + for (int i = 0; i < getters.length; i++) { + Types.NestedField projectField = projectType.fields().get(i); + Types.NestedField rowField = rowStruct.field(projectField.fieldId()); + + Preconditions.checkNotNull( + rowField, + "Cannot locate the project field <%s> in the iceberg struct <%s>", + projectField, + rowStruct); + + getters[i] = + createFieldGetter( + rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField); + } + } + + private static RowData.FieldGetter createFieldGetter( + RowType rowType, int position, Types.NestedField rowField, Types.NestedField projectField) { + Preconditions.checkArgument( + rowField.type().typeId() == projectField.type().typeId(), + "Different iceberg type between row field <%s> and project field <%s>", + rowField, + projectField); + + switch (projectField.type().typeId()) { + case STRUCT: + RowType nestedRowType = (RowType) rowType.getTypeAt(position); + return row -> { + // null nested struct value + if (row.isNullAt(position)) { + return null; + } + + RowData nestedRow = row.getRow(position, nestedRowType.getFieldCount()); + return RowDataProjection.create( + nestedRowType, rowField.type().asStructType(), projectField.type().asStructType()) + .wrap(nestedRow); + }; + + case MAP: + Types.MapType projectedMap = projectField.type().asMapType(); + Types.MapType originalMap = rowField.type().asMapType(); + + boolean keyProjectable = + !projectedMap.keyType().isNestedType() + || projectedMap.keyType().equals(originalMap.keyType()); + boolean valueProjectable = + !projectedMap.valueType().isNestedType() + || projectedMap.valueType().equals(originalMap.valueType()); + Preconditions.checkArgument( + keyProjectable && valueProjectable, + "Cannot project a partial map key or value with non-primitive type. Trying to project <%s> out of <%s>", + projectField, + rowField); + + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + + case LIST: + Types.ListType projectedList = projectField.type().asListType(); + Types.ListType originalList = rowField.type().asListType(); + + boolean elementProjectable = + !projectedList.elementType().isNestedType() + || projectedList.elementType().equals(originalList.elementType()); + Preconditions.checkArgument( + elementProjectable, + "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>", + projectField, + rowField); + + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + + default: + return RowData.createFieldGetter(rowType.getTypeAt(position), position); + } + } + + public RowData wrap(RowData row) { + // StructProjection allow wrapping null root struct object. + // See more discussions in https://github.com/apache/iceberg/pull/7517. + // RowDataProjection never allowed null root object to be wrapped. + // Hence, it is fine to enforce strict Preconditions check here. + Preconditions.checkArgument(row != null, "Invalid row data: null"); + this.rowData = row; + return this; + } + + private Object getValue(int pos) { + Preconditions.checkState(rowData != null, "Row data not wrapped"); + return getters[pos].getFieldOrNull(rowData); + } + + @Override + public int getArity() { + return getters.length; + } + + @Override + public RowKind getRowKind() { + Preconditions.checkState(rowData != null, "Row data not wrapped"); + return rowData.getRowKind(); + } + + @Override + public void setRowKind(RowKind kind) { + throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection"); + } + + @Override + public boolean isNullAt(int pos) { + return getValue(pos) == null; + } + + @Override + public boolean getBoolean(int pos) { + return (boolean) getValue(pos); + } + + @Override + public byte getByte(int pos) { + return (byte) getValue(pos); + } + + @Override + public short getShort(int pos) { + return (short) getValue(pos); + } + + @Override + public int getInt(int pos) { + return (int) getValue(pos); + } + + @Override + public long getLong(int pos) { + return (long) getValue(pos); + } + + @Override + public float getFloat(int pos) { + return (float) getValue(pos); + } + + @Override + public double getDouble(int pos) { + return (double) getValue(pos); + } + + @Override + public StringData getString(int pos) { + return (StringData) getValue(pos); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return (DecimalData) getValue(pos); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + return (TimestampData) getValue(pos); + } + + @Override + @SuppressWarnings("unchecked") + public RawValueData getRawValue(int pos) { + return (RawValueData) getValue(pos); + } + + @Override + public byte[] getBinary(int pos) { + return (byte[]) getValue(pos); + } + + @Override + public ArrayData getArray(int pos) { + return (ArrayData) getValue(pos); + } + + @Override + public MapData getMap(int pos) { + return (MapData) getValue(pos); + } + + @Override + public RowData getRow(int pos, int numFields) { + return (RowData) getValue(pos); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof RowDataProjection)) { + return false; + } + + RowDataProjection that = (RowDataProjection) o; + return deepEquals(that); + } + + @Override + public int hashCode() { + int result = Objects.hashCode(getRowKind()); + for (int pos = 0; pos < getArity(); pos++) { + if (!isNullAt(pos)) { + // Arrays.deepHashCode handles array object properly + result = 31 * result + Arrays.deepHashCode(new Object[] {getValue(pos)}); + } + } + + return result; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder(); + sb.append(getRowKind().shortString()).append("("); + for (int pos = 0; pos < getArity(); pos++) { + if (pos != 0) { + sb.append(","); + } + // copied the behavior from Flink GenericRowData + sb.append(StringUtils.arrayAwareToString(getValue(pos))); + } + + sb.append(")"); + return sb.toString(); + } + + private boolean deepEquals(RowDataProjection other) { + if (getRowKind() != other.getRowKind()) { + return false; + } + + if (getArity() != other.getArity()) { + return false; + } + + for (int pos = 0; pos < getArity(); ++pos) { + if (isNullAt(pos) && other.isNullAt(pos)) { + continue; + } + + if ((isNullAt(pos) && !other.isNullAt(pos)) || (!isNullAt(pos) && other.isNullAt(pos))) { + return false; + } + + // Objects.deepEquals handles array object properly + if (!Objects.deepEquals(getValue(pos), other.getValue(pos))) { + return false; + } + } + + return true; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java new file mode 100644 index 000000000000..3a8f5ccc6c03 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.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.flink.data; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; + +public class RowDataUtil { + + private RowDataUtil() {} + + public static Object convertConstant(Type type, Object value) { + if (value == null) { + return null; + } + + switch (type.typeId()) { + case DECIMAL: // DecimalData + Types.DecimalType decimal = (Types.DecimalType) type; + return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale()); + case STRING: // StringData + if (value instanceof Utf8) { + Utf8 utf8 = (Utf8) value; + return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); + } + return StringData.fromString(value.toString()); + case FIXED: // byte[] + if (value instanceof byte[]) { + return value; + } else if (value instanceof GenericData.Fixed) { + return ((GenericData.Fixed) value).bytes(); + } + return ByteBuffers.toByteArray((ByteBuffer) value); + case BINARY: // byte[] + return ByteBuffers.toByteArray((ByteBuffer) value); + case TIME: // int mills instead of long + return (int) ((Long) value / 1000); + case TIMESTAMP: // TimestampData + return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value)); + default: + } + return value; + } + + /** + * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method. This + * skips the check the arity of rowType and from, because the from RowData may contains additional + * column for position deletes. Using {@link RowDataSerializer#copy(RowData, RowData)} will fail + * the arity check. + */ + public static RowData clone( + RowData from, + RowData reuse, + RowType rowType, + TypeSerializer[] fieldSerializers, + RowData.FieldGetter[] fieldGetters) { + GenericRowData ret; + if (reuse instanceof GenericRowData) { + ret = (GenericRowData) reuse; + } else { + ret = new GenericRowData(from.getArity()); + } + + ret.setRowKind(from.getRowKind()); + for (int i = 0; i < rowType.getFieldCount(); i++) { + if (!from.isNullAt(i)) { + ret.setField(i, fieldSerializers[i].copy(fieldGetters[i].getFieldOrNull(from))); + } else { + ret.setField(i, null); + } + } + + return ret; + } + + /** + * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to + * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], + * RowData.FieldGetter[])} instead. + */ + @Deprecated + public static RowData clone( + RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + if (!from.isNullAt(i)) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + } + + return clone(from, reuse, rowType, fieldSerializers, fieldGetters); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java new file mode 100644 index 000000000000..1019285018d0 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -0,0 +1,300 @@ +/* + * 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.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.UUID; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RawValueData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +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.ByteBuffers; + +@Internal +public class StructRowData implements RowData { + private final Types.StructType type; + private RowKind kind; + private StructLike struct; + + public StructRowData(Types.StructType type) { + this(type, RowKind.INSERT); + } + + public StructRowData(Types.StructType type, RowKind kind) { + this(type, null, kind); + } + + private StructRowData(Types.StructType type, StructLike struct) { + this(type, struct, RowKind.INSERT); + } + + private StructRowData(Types.StructType type, StructLike struct, RowKind kind) { + this.type = type; + this.struct = struct; + this.kind = kind; + } + + public StructRowData setStruct(StructLike newStruct) { + this.struct = newStruct; + return this; + } + + @Override + public int getArity() { + return struct.size(); + } + + @Override + public RowKind getRowKind() { + return kind; + } + + @Override + public void setRowKind(RowKind newKind) { + Preconditions.checkNotNull(newKind, "kind can not be null"); + this.kind = newKind; + } + + @Override + public boolean isNullAt(int pos) { + return struct.get(pos, Object.class) == null; + } + + @Override + public boolean getBoolean(int pos) { + return struct.get(pos, Boolean.class); + } + + @Override + public byte getByte(int pos) { + return (byte) (int) struct.get(pos, Integer.class); + } + + @Override + public short getShort(int pos) { + return (short) (int) struct.get(pos, Integer.class); + } + + @Override + public int getInt(int pos) { + Object integer = struct.get(pos, Object.class); + + if (integer instanceof Integer) { + return (int) integer; + } else if (integer instanceof LocalDate) { + return (int) ((LocalDate) integer).toEpochDay(); + } else if (integer instanceof LocalTime) { + return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000); + } else { + throw new IllegalStateException( + "Unknown type for int field. Type name: " + integer.getClass().getName()); + } + } + + @Override + public long getLong(int pos) { + Object longVal = struct.get(pos, Object.class); + + if (longVal instanceof Long) { + return (long) longVal; + } else if (longVal instanceof OffsetDateTime) { + return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; + } else if (longVal instanceof LocalDate) { + return ((LocalDate) longVal).toEpochDay(); + } else if (longVal instanceof LocalTime) { + return ((LocalTime) longVal).toNanoOfDay(); + } else if (longVal instanceof LocalDateTime) { + return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC)) + .toNanos() + / 1000; + } else { + throw new IllegalStateException( + "Unknown type for long field. Type name: " + longVal.getClass().getName()); + } + } + + @Override + public float getFloat(int pos) { + return struct.get(pos, Float.class); + } + + @Override + public double getDouble(int pos) { + return struct.get(pos, Double.class); + } + + @Override + public StringData getString(int pos) { + return isNullAt(pos) ? null : getStringDataInternal(pos); + } + + private StringData getStringDataInternal(int pos) { + CharSequence seq = struct.get(pos, CharSequence.class); + return StringData.fromString(seq.toString()); + } + + @Override + public DecimalData getDecimal(int pos, int precision, int scale) { + return isNullAt(pos) + ? null + : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale); + } + + private BigDecimal getDecimalInternal(int pos) { + return struct.get(pos, BigDecimal.class); + } + + @Override + public TimestampData getTimestamp(int pos, int precision) { + long timeLong = getLong(pos); + return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000); + } + + @Override + public RawValueData getRawValue(int pos) { + throw new UnsupportedOperationException("Not supported yet."); + } + + @Override + public byte[] getBinary(int pos) { + return isNullAt(pos) ? null : getBinaryInternal(pos); + } + + private byte[] getBinaryInternal(int pos) { + Object bytes = struct.get(pos, Object.class); + + // should only be either ByteBuffer or byte[] + if (bytes instanceof ByteBuffer) { + return ByteBuffers.toByteArray((ByteBuffer) bytes); + } else if (bytes instanceof byte[]) { + return (byte[]) bytes; + } else if (bytes instanceof UUID) { + UUID uuid = (UUID) bytes; + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + return bb.array(); + } else { + throw new IllegalStateException( + "Unknown type for binary field. Type name: " + bytes.getClass().getName()); + } + } + + @Override + public ArrayData getArray(int pos) { + return isNullAt(pos) + ? null + : (ArrayData) + convertValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class)); + } + + @Override + public MapData getMap(int pos) { + return isNullAt(pos) + ? null + : (MapData) + convertValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class)); + } + + @Override + public RowData getRow(int pos, int numFields) { + return isNullAt(pos) ? null : getStructRowData(pos, numFields); + } + + private StructRowData getStructRowData(int pos, int numFields) { + return new StructRowData( + type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); + } + + private Object convertValue(Type elementType, Object value) { + switch (elementType.typeId()) { + case BOOLEAN: + case INTEGER: + case DATE: + case TIME: + case LONG: + case FLOAT: + case DOUBLE: + case DECIMAL: + return value; + case TIMESTAMP: + long millisecond = (long) value / 1000; + int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000; + return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); + case STRING: + return StringData.fromString(value.toString()); + case FIXED: + case BINARY: + return ByteBuffers.toByteArray((ByteBuffer) value); + case STRUCT: + return new StructRowData(elementType.asStructType(), (StructLike) value); + case LIST: + List list = (List) value; + Object[] array = new Object[list.size()]; + + int index = 0; + for (Object element : list) { + if (element == null) { + array[index] = null; + } else { + array[index] = convertValue(elementType.asListType().elementType(), element); + } + + index += 1; + } + return new GenericArrayData(array); + case MAP: + Types.MapType mapType = elementType.asMapType(); + Set> entries = ((Map) value).entrySet(); + Map result = Maps.newHashMap(); + for (Map.Entry entry : entries) { + final Object keyValue = convertValue(mapType.keyType(), entry.getKey()); + final Object valueValue = convertValue(mapType.valueType(), entry.getValue()); + result.put(keyValue, valueValue); + } + + return new GenericMapData(result); + default: + throw new UnsupportedOperationException("Unsupported element type: " + elementType); + } + } +} 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 new file mode 100644 index 000000000000..d74b2349b1de --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/MonitorSource.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.flink.maintenance.operator; + +import java.io.IOException; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimitedSourceReader; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiter; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +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; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Monitors an Iceberg table for changes */ +@Internal +public class MonitorSource extends SingleThreadedIteratorSource { + private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); + + private final TableLoader tableLoader; + private final RateLimiterStrategy rateLimiterStrategy; + private final long maxReadBack; + + /** + * Creates a {@link org.apache.flink.api.connector.source.Source} which monitors an Iceberg table + * for changes. + * + * @param tableLoader used for accessing the table + * @param rateLimiterStrategy limits the frequency the table is checked + * @param maxReadBack sets the number of snapshots read before stopping change collection + */ + 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"); + Preconditions.checkArgument(maxReadBack > 0, "Need to read at least 1 snapshot to work"); + + this.tableLoader = tableLoader; + this.rateLimiterStrategy = rateLimiterStrategy; + this.maxReadBack = maxReadBack; + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public TypeInformation getProducedType() { + return TypeInformation.of(TableChange.class); + } + + @Override + Iterator createIterator() { + return new TableChangeIterator(tableLoader, null, maxReadBack); + } + + @Override + SimpleVersionedSerializer> iteratorSerializer() { + return new TableChangeIteratorSerializer(tableLoader, maxReadBack); + } + + @Override + public SourceReader> createReader( + SourceReaderContext readerContext) throws Exception { + RateLimiter rateLimiter = rateLimiterStrategy.createRateLimiter(1); + return new RateLimitedSourceReader<>(super.createReader(readerContext), rateLimiter); + } + + /** The Iterator which returns the latest changes on an Iceberg table. */ + @VisibleForTesting + static class TableChangeIterator implements Iterator { + private Long lastSnapshotId; + private final long maxReadBack; + private final Table table; + + TableChangeIterator(TableLoader tableLoader, Long lastSnapshotId, long maxReadBack) { + this.lastSnapshotId = lastSnapshotId; + this.maxReadBack = maxReadBack; + tableLoader.open(); + this.table = tableLoader.loadTable(); + } + + @Override + public boolean hasNext() { + return true; + } + + @Override + public TableChange next() { + try { + table.refresh(); + Snapshot currentSnapshot = table.currentSnapshot(); + Long current = currentSnapshot != null ? currentSnapshot.snapshotId() : null; + Long checking = current; + TableChange event = TableChange.empty(); + long readBack = 0; + while (checking != null && !checking.equals(lastSnapshotId) && ++readBack <= maxReadBack) { + Snapshot snapshot = table.snapshot(checking); + if (snapshot != null) { + if (!DataOperations.REPLACE.equals(snapshot.operation())) { + LOG.debug("Reading snapshot {}", snapshot.snapshotId()); + event.merge(new TableChange(snapshot, table.io())); + } else { + LOG.debug("Skipping replace snapshot {}", snapshot.snapshotId()); + } + + checking = snapshot.parentId(); + } else { + // If the last snapshot has been removed from the history + checking = null; + } + } + + lastSnapshotId = current; + return event; + } catch (Exception e) { + LOG.warn("Failed to fetch table changes for {}", table, e); + return TableChange.empty(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("lastSnapshotId", lastSnapshotId) + .add("maxReadBack", maxReadBack) + .add("table", table) + .toString(); + } + } + + private static final class TableChangeIteratorSerializer + implements SimpleVersionedSerializer> { + + private static final int CURRENT_VERSION = 1; + private final TableLoader tableLoader; + private final long maxReadBack; + + TableChangeIteratorSerializer(TableLoader tableLoader, long maxReadBack) { + this.tableLoader = tableLoader; + this.maxReadBack = maxReadBack; + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(Iterator iterator) throws IOException { + Preconditions.checkArgument( + iterator instanceof TableChangeIterator, + "Use TableChangeIterator iterator. Found incompatible type: %s", + iterator.getClass()); + + TableChangeIterator tableChangeIterator = (TableChangeIterator) iterator; + DataOutputSerializer out = new DataOutputSerializer(8); + long toStore = + tableChangeIterator.lastSnapshotId != null ? tableChangeIterator.lastSnapshotId : -1L; + out.writeLong(toStore); + return out.getCopyOfBuffer(); + } + + @Override + public TableChangeIterator deserialize(int version, byte[] serialized) throws IOException { + if (version == CURRENT_VERSION) { + DataInputDeserializer in = new DataInputDeserializer(serialized); + long fromStore = in.readLong(); + return new TableChangeIterator( + tableLoader, fromStore != -1 ? fromStore : null, maxReadBack); + } else { + throw new IOException("Unrecognized version or corrupt state: " + version); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java new file mode 100644 index 000000000000..20c7684d9700 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/SingleThreadedIteratorSource.java @@ -0,0 +1,197 @@ +/* + * 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.IOException; +import java.util.Collection; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceEnumerator; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceReader; +import org.apache.flink.api.connector.source.lib.util.IteratorSourceSplit; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.SimpleVersionedSerializer; +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; + +/** + * Implementation of the Source V2 API which uses an iterator to read the elements, and uses a + * single thread to do so. + * + * @param The return type of the source + */ +@Internal +public abstract class SingleThreadedIteratorSource + implements Source< + T, + SingleThreadedIteratorSource.GlobalSplit, + Collection>>, + ResultTypeQueryable { + private static final String PARALLELISM_ERROR = "Parallelism should be set to 1"; + + /** + * Creates the iterator to return the elements which then emitted by the source. + * + * @return iterator for the elements + */ + abstract Iterator createIterator(); + + /** + * Serializes the iterator, which is used to save and restore the state of the source. + * + * @return serializer for the iterator + */ + abstract SimpleVersionedSerializer> iteratorSerializer(); + + @Override + public SplitEnumerator, Collection>> createEnumerator( + SplitEnumeratorContext> enumContext) { + Preconditions.checkArgument(enumContext.currentParallelism() == 1, PARALLELISM_ERROR); + return new IteratorSourceEnumerator<>( + enumContext, ImmutableList.of(new GlobalSplit<>(createIterator()))); + } + + @Override + public SplitEnumerator, Collection>> restoreEnumerator( + SplitEnumeratorContext> enumContext, Collection> checkpoint) { + Preconditions.checkArgument(enumContext.currentParallelism() == 1, PARALLELISM_ERROR); + return new IteratorSourceEnumerator<>(enumContext, checkpoint); + } + + @Override + public SimpleVersionedSerializer> getSplitSerializer() { + return new SplitSerializer<>(iteratorSerializer()); + } + + @Override + public SimpleVersionedSerializer>> getEnumeratorCheckpointSerializer() { + return new EnumeratorSerializer<>(iteratorSerializer()); + } + + @Override + public SourceReader> createReader(SourceReaderContext readerContext) + throws Exception { + Preconditions.checkArgument(readerContext.getIndexOfSubtask() == 0, PARALLELISM_ERROR); + return new IteratorSourceReader<>(readerContext); + } + + /** The single split of the {@link SingleThreadedIteratorSource}. */ + static class GlobalSplit implements IteratorSourceSplit> { + private final Iterator iterator; + + GlobalSplit(Iterator iterator) { + this.iterator = iterator; + } + + @Override + public String splitId() { + return "1"; + } + + @Override + public Iterator getIterator() { + return iterator; + } + + @Override + public IteratorSourceSplit> getUpdatedSplitForIterator( + final Iterator newIterator) { + return new GlobalSplit<>(newIterator); + } + + @Override + public String toString() { + return String.format("GlobalSplit (%s)", iterator); + } + } + + private static final class SplitSerializer + implements SimpleVersionedSerializer> { + private final SimpleVersionedSerializer> iteratorSerializer; + + SplitSerializer(SimpleVersionedSerializer> iteratorSerializer) { + this.iteratorSerializer = iteratorSerializer; + } + + private static final int CURRENT_VERSION = 1; + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(GlobalSplit split) throws IOException { + return iteratorSerializer.serialize(split.iterator); + } + + @Override + public GlobalSplit deserialize(int version, byte[] serialized) throws IOException { + return new GlobalSplit<>(iteratorSerializer.deserialize(version, serialized)); + } + } + + private static final class EnumeratorSerializer + implements SimpleVersionedSerializer>> { + private static final int CURRENT_VERSION = 1; + private final SimpleVersionedSerializer> iteratorSerializer; + + EnumeratorSerializer(SimpleVersionedSerializer> iteratorSerializer) { + this.iteratorSerializer = iteratorSerializer; + } + + @Override + public int getVersion() { + return CURRENT_VERSION; + } + + @Override + public byte[] serialize(Collection> checkpoint) throws IOException { + Preconditions.checkArgument(checkpoint.size() < 2, PARALLELISM_ERROR); + if (checkpoint.isEmpty()) { + return new byte[] {0}; + } else { + byte[] iterator = iteratorSerializer.serialize(checkpoint.iterator().next().getIterator()); + byte[] result = new byte[iterator.length + 1]; + result[0] = 1; + System.arraycopy(iterator, 0, result, 1, iterator.length); + return result; + } + } + + @Override + public Collection> deserialize(int version, byte[] serialized) + throws IOException { + if (serialized[0] == 0) { + return Lists.newArrayList(); + } else { + byte[] iterator = new byte[serialized.length - 1]; + System.arraycopy(serialized, 1, iterator, 0, serialized.length - 1); + return Lists.newArrayList( + new GlobalSplit<>(iteratorSerializer.deserialize(version, iterator))); + } + } + } +} 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 new file mode 100644 index 000000000000..452ed80ed0e5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableChange.java @@ -0,0 +1,133 @@ +/* + * 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.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** Event describing changes in an Iceberg table */ +@Internal +class TableChange { + private int dataFileNum; + private int deleteFileNum; + private long dataFileSize; + private long deleteFileSize; + private int commitNum; + + TableChange( + int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { + this.dataFileNum = dataFileNum; + this.deleteFileNum = deleteFileNum; + this.dataFileSize = dataFileSize; + this.deleteFileSize = deleteFileSize; + this.commitNum = commitNum; + } + + TableChange(Snapshot snapshot, FileIO io) { + Iterable dataFiles = snapshot.addedDataFiles(io); + Iterable deleteFiles = snapshot.addedDeleteFiles(io); + + dataFiles.forEach( + dataFile -> { + this.dataFileNum++; + this.dataFileSize += dataFile.fileSizeInBytes(); + }); + + deleteFiles.forEach( + deleteFile -> { + this.deleteFileNum++; + this.deleteFileSize += deleteFile.fileSizeInBytes(); + }); + + this.commitNum = 1; + } + + static TableChange empty() { + return new TableChange(0, 0, 0L, 0L, 0); + } + + int dataFileNum() { + return dataFileNum; + } + + int deleteFileNum() { + return deleteFileNum; + } + + long dataFileSize() { + return dataFileSize; + } + + long deleteFileSize() { + return deleteFileSize; + } + + public int commitNum() { + return commitNum; + } + + public void merge(TableChange other) { + this.dataFileNum += other.dataFileNum; + this.deleteFileNum += other.deleteFileNum; + this.dataFileSize += other.dataFileSize; + this.deleteFileSize += other.deleteFileSize; + this.commitNum += other.commitNum; + } + + TableChange copy() { + return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFileNum", dataFileNum) + .add("deleteFileNum", deleteFileNum) + .add("dataFileSize", dataFileSize) + .add("deleteFileSize", deleteFileSize) + .add("commitNum", commitNum) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (other == null || getClass() != other.getClass()) { + return false; + } + + TableChange that = (TableChange) other; + return dataFileNum == that.dataFileNum + && deleteFileNum == that.deleteFileNum + && dataFileSize == that.dataFileSize + && deleteFileSize == that.deleteFileSize + && commitNum == that.commitNum; + } + + @Override + public int hashCode() { + return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java new file mode 100644 index 000000000000..f7e8e0c884cf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java @@ -0,0 +1,61 @@ +/* + * 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 org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.formats.avro.AvroToRowDataConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; + +/** + * This util class converts Avro GenericRecord to Flink RowData.
    + *
    + * Internally it uses Flink {@link AvroToRowDataConverters}. Because of the precision difference + * between how Iceberg schema (micro) and Flink {@link AvroToRowDataConverters} (milli) deal with + * time type, we can't directly use the Avro Schema converted from Iceberg schema via {@link + * AvroSchemaUtil#convert(org.apache.iceberg.Schema, String)}. + */ +public class AvroGenericRecordToRowDataMapper implements MapFunction { + + private final AvroToRowDataConverters.AvroToRowDataConverter converter; + + AvroGenericRecordToRowDataMapper(RowType rowType) { + this.converter = AvroToRowDataConverters.createRowConverter(rowType); + } + + @Override + public RowData map(GenericRecord genericRecord) throws Exception { + return (RowData) converter.convert(genericRecord); + } + + /** Create a mapper based on Avro schema. */ + public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordToRowDataMapper(rowType); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java new file mode 100644 index 000000000000..e8a46c5becd7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.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.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.deletes.DeleteGranularity; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RowDataProjection; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; + +abstract class BaseDeltaTaskWriter extends BaseTaskWriter { + + private final Schema schema; + private final Schema deleteSchema; + private final RowDataWrapper wrapper; + private final RowDataWrapper keyWrapper; + private final RowDataProjection keyProjection; + private final boolean upsert; + + BaseDeltaTaskWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.schema = schema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + this.keyWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); + this.keyProjection = + RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); + this.upsert = upsert; + } + + abstract RowDataDeltaWriter route(RowData row); + + RowDataWrapper wrapper() { + return wrapper; + } + + @Override + public void write(RowData row) throws IOException { + RowDataDeltaWriter writer = route(row); + + switch (row.getRowKind()) { + case INSERT: + case UPDATE_AFTER: + if (upsert) { + writer.deleteKey(keyProjection.wrap(row)); + } + writer.write(row); + break; + + case UPDATE_BEFORE: + if (upsert) { + break; // UPDATE_BEFORE is not necessary for UPSERT, we do nothing to prevent delete one + // row twice + } + writer.delete(row); + break; + case DELETE: + if (upsert) { + writer.deleteKey(keyProjection.wrap(row)); + } else { + writer.delete(row); + } + break; + + default: + throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind()); + } + } + + protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { + RowDataDeltaWriter(PartitionKey partition) { + super(partition, schema, deleteSchema, DeleteGranularity.FILE); + } + + @Override + protected StructLike asStructLike(RowData data) { + return wrapper.wrap(data); + } + + @Override + protected StructLike asStructLikeKey(RowData data) { + return keyWrapper.wrap(data); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java new file mode 100644 index 000000000000..1cb6e013bd2c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java @@ -0,0 +1,70 @@ +/* + * 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.stream.IntStream; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; + +/** + * A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key. + * To be used with the {@link BucketPartitioner}. + */ +class BucketPartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + private final int bucketFieldPosition; + + private transient RowDataWrapper rowDataWrapper; + + BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { + this.schema = schema; + this.partitionKey = new PartitionKey(partitionSpec, schema); + this.flinkSchema = flinkSchema; + this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); + } + + private int getBucketFieldPosition(PartitionSpec partitionSpec) { + int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); + return IntStream.range(0, partitionSpec.fields().size()) + .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) + .toArray()[0]; + } + + private RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + return rowDataWrapper; + } + + @Override + public Integer getKey(RowData rowData) { + partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); + return partitionKey.get(bucketFieldPosition, Integer.class); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java new file mode 100644 index 000000000000..9c9a117906e2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.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.flink.sink; + +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This partitioner will redirect records to writers deterministically based on the Bucket partition + * spec. It'll attempt to optimize the file size written depending on whether numPartitions is + * greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE + * bucket in the partition spec. + */ +class BucketPartitioner implements Partitioner { + + static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null"; + static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE = + "Invalid bucket ID %s: must be non-negative."; + static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE = + "Invalid bucket ID %s: must be less than bucket limit: %s."; + + private final int maxNumBuckets; + + // To hold the OFFSET of the next writer to use for any bucket, only used when writers > the + // number of buckets + private final int[] currentBucketWriterOffset; + + BucketPartitioner(PartitionSpec partitionSpec) { + this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); + this.currentBucketWriterOffset = new int[maxNumBuckets]; + } + + /** + * Determine the partition id based on the following criteria: If the number of writers <= the + * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one + * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic + * is handled by the {@link #getPartitionWithMoreWritersThanBuckets + * getPartitionWritersGreaterThanBuckets} method. + * + * @param bucketId the bucketId for each request + * @param numPartitions the total number of partitions + * @return the partition id (writer) to use for each request + */ + @Override + public int partition(Integer bucketId, int numPartitions) { + Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); + Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); + Preconditions.checkArgument( + bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); + + if (numPartitions <= maxNumBuckets) { + return bucketId % numPartitions; + } else { + return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); + } + } + + /*- + * If the number of writers > the number of buckets each partitioner will keep a state of multiple + * writers per bucket as evenly as possible, and will round-robin the requests across them, in this + * case each writer will target only one bucket at all times (many writers -> one bucket). Example: + * Configuration: numPartitions (writers) = 5, maxBuckets = 2 + * Expected behavior: + * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 + * - Records for Bucket 1 will always use Writer 1 and 3 + * Notes: + * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId + * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). + * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. + * + * @return the destination partition index (writer subtask id) + */ + private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { + int currentOffset = currentBucketWriterOffset[bucketId]; + // Determine if this bucket requires an "extra writer" + int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; + // The max number of writers this bucket can have + int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; + + // Increment the writer offset or reset if it's reached the max for this bucket + int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; + currentBucketWriterOffset[bucketId] = nextOffset; + + return bucketId + (maxNumBuckets * currentOffset); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java new file mode 100644 index 000000000000..c33207728d3e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.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.sink; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.transforms.PartitionSpecVisitor; + +final class BucketPartitionerUtil { + static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = + "Invalid number of buckets: %s (must be 1)"; + + private BucketPartitionerUtil() {} + + /** + * Determines whether the PartitionSpec has one and only one Bucket definition + * + * @param partitionSpec the partition spec in question + * @return whether the PartitionSpec has only one Bucket + */ + static boolean hasOneBucketField(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + return bucketFields != null && bucketFields.size() == 1; + } + + /** + * Extracts the Bucket definition from a PartitionSpec. + * + * @param partitionSpec the partition spec in question + * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) + */ + private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { + List> bucketFields = getBucketFields(partitionSpec); + Preconditions.checkArgument( + bucketFields.size() == 1, + BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, + bucketFields.size()); + return bucketFields.get(0); + } + + static int getBucketFieldId(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f0; + } + + static int getMaxNumBuckets(PartitionSpec partitionSpec) { + return getBucketFieldInfo(partitionSpec).f1; + } + + private static List> getBucketFields(PartitionSpec spec) { + return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() + .filter(Objects::nonNull) + .collect(Collectors.toList()); + } + + private static class BucketPartitionSpecVisitor + implements PartitionSpecVisitor> { + @Override + public Tuple2 identity(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 bucket( + int fieldId, String sourceName, int sourceId, int numBuckets) { + return new Tuple2<>(fieldId, numBuckets); + } + + @Override + public Tuple2 truncate( + int fieldId, String sourceName, int sourceId, int width) { + return null; + } + + @Override + public Tuple2 year(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 month(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 day(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 hour(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { + return null; + } + + @Override + public Tuple2 unknown( + int fieldId, String sourceName, int sourceId, String transform) { + return null; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java new file mode 100644 index 000000000000..e9f9786f9190 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java @@ -0,0 +1,91 @@ +/* + * 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.time.Duration; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A table loader that will only reload a table after a certain interval has passed. WARNING: This + * table loader should be used carefully when used with writer tasks. It could result in heavy load + * on a catalog for jobs with many writers. + */ +class CachingTableSupplier implements SerializableSupplier

  • { + + private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); + + private final Table initialTable; + private final TableLoader tableLoader; + private final Duration tableRefreshInterval; + private long lastLoadTimeMillis; + private transient Table table; + + CachingTableSupplier( + SerializableTable initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { + Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); + Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); + Preconditions.checkArgument( + tableRefreshInterval != null, "tableRefreshInterval cannot be null"); + this.initialTable = initialTable; + this.table = initialTable; + this.tableLoader = tableLoader; + this.tableRefreshInterval = tableRefreshInterval; + this.lastLoadTimeMillis = System.currentTimeMillis(); + } + + @Override + public Table get() { + if (table == null) { + this.table = initialTable; + } + return table; + } + + Table initialTable() { + return initialTable; + } + + void refreshTable() { + if (System.currentTimeMillis() > lastLoadTimeMillis + tableRefreshInterval.toMillis()) { + try { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.lastLoadTimeMillis = System.currentTimeMillis(); + + LOG.info( + "Table {} reloaded, next min load time threshold is {}", + table.name(), + DateTimeUtil.formatTimestampMillis( + lastLoadTimeMillis + tableRefreshInterval.toMillis())); + } catch (Exception e) { + LOG.warn("An error occurred reloading table {}, table was not reloaded", table.name(), e); + } + } + } +} 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 new file mode 100644 index 000000000000..9a2f57181708 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java @@ -0,0 +1,93 @@ +/* + * 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.Arrays; +import java.util.NavigableMap; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class CommitSummary { + + private final AtomicLong dataFilesCount = new AtomicLong(); + private final AtomicLong dataFilesRecordCount = new AtomicLong(); + private final AtomicLong dataFilesByteCount = new AtomicLong(); + private final AtomicLong deleteFilesCount = new AtomicLong(); + private final AtomicLong deleteFilesRecordCount = new AtomicLong(); + private final AtomicLong deleteFilesByteCount = new AtomicLong(); + + CommitSummary(NavigableMap pendingResults) { + pendingResults + .values() + .forEach( + writeResult -> { + dataFilesCount.addAndGet(writeResult.dataFiles().length); + Arrays.stream(writeResult.dataFiles()) + .forEach( + dataFile -> { + dataFilesRecordCount.addAndGet(dataFile.recordCount()); + dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); + }); + deleteFilesCount.addAndGet(writeResult.deleteFiles().length); + Arrays.stream(writeResult.deleteFiles()) + .forEach( + deleteFile -> { + deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); + deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); + }); + }); + } + + long dataFilesCount() { + return dataFilesCount.get(); + } + + long dataFilesRecordCount() { + return dataFilesRecordCount.get(); + } + + long dataFilesByteCount() { + return dataFilesByteCount.get(); + } + + long deleteFilesCount() { + return deleteFilesCount.get(); + } + + long deleteFilesRecordCount() { + return deleteFilesRecordCount.get(); + } + + long deleteFilesByteCount() { + return deleteFilesByteCount.get(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("dataFilesCount", dataFilesCount) + .add("dataFilesRecordCount", dataFilesRecordCount) + .add("dataFilesByteCount", dataFilesByteCount) + .add("deleteFilesCount", deleteFilesCount) + .add("deleteFilesRecordCount", deleteFilesRecordCount) + .add("deleteFilesByteCount", deleteFilesByteCount) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java new file mode 100644 index 000000000000..036970c06d5b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java @@ -0,0 +1,71 @@ +/* + * 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 org.apache.iceberg.ManifestFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class DeltaManifests { + + private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; + + private final ManifestFile dataManifest; + private final ManifestFile deleteManifest; + private final CharSequence[] referencedDataFiles; + + DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { + this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); + } + + DeltaManifests( + ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { + Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); + + this.dataManifest = dataManifest; + this.deleteManifest = deleteManifest; + this.referencedDataFiles = referencedDataFiles; + } + + ManifestFile dataManifest() { + return dataManifest; + } + + ManifestFile deleteManifest() { + return deleteManifest; + } + + CharSequence[] referencedDataFiles() { + return referencedDataFiles; + } + + List manifests() { + List manifests = Lists.newArrayListWithCapacity(2); + if (dataManifest != null) { + manifests.add(dataManifest); + } + + if (deleteManifest != null) { + manifests.add(deleteManifest); + } + + return manifests; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java new file mode 100644 index 000000000000..92ca284b12ba --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.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.flink.sink; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class DeltaManifestsSerializer implements SimpleVersionedSerializer { + private static final int VERSION_1 = 1; + private static final int VERSION_2 = 2; + private static final byte[] EMPTY_BINARY = new byte[0]; + + static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); + + @Override + public int getVersion() { + return VERSION_2; + } + + @Override + public byte[] serialize(DeltaManifests deltaManifests) throws IOException { + Preconditions.checkNotNull( + deltaManifests, "DeltaManifests to be serialized should not be null"); + + ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); + DataOutputStream out = new DataOutputStream(binaryOut); + + byte[] dataManifestBinary = EMPTY_BINARY; + if (deltaManifests.dataManifest() != null) { + dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); + } + + out.writeInt(dataManifestBinary.length); + out.write(dataManifestBinary); + + byte[] deleteManifestBinary = EMPTY_BINARY; + if (deltaManifests.deleteManifest() != null) { + deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); + } + + out.writeInt(deleteManifestBinary.length); + out.write(deleteManifestBinary); + + CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); + out.writeInt(referencedDataFiles.length); + for (CharSequence referencedDataFile : referencedDataFiles) { + out.writeUTF(referencedDataFile.toString()); + } + + return binaryOut.toByteArray(); + } + + @Override + public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { + if (version == VERSION_1) { + return deserializeV1(serialized); + } else if (version == VERSION_2) { + return deserializeV2(serialized); + } else { + throw new RuntimeException("Unknown serialize version: " + version); + } + } + + private DeltaManifests deserializeV1(byte[] serialized) throws IOException { + return new DeltaManifests(ManifestFiles.decode(serialized), null); + } + + private DeltaManifests deserializeV2(byte[] serialized) throws IOException { + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); + DataInputStream in = new DataInputStream(binaryIn); + + int dataManifestSize = in.readInt(); + if (dataManifestSize > 0) { + byte[] dataManifestBinary = new byte[dataManifestSize]; + Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); + + dataManifest = ManifestFiles.decode(dataManifestBinary); + } + + int deleteManifestSize = in.readInt(); + if (deleteManifestSize > 0) { + byte[] deleteManifestBinary = new byte[deleteManifestSize]; + Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); + + deleteManifest = ManifestFiles.decode(deleteManifestBinary); + } + + int referenceDataFileNum = in.readInt(); + CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; + for (int i = 0; i < referenceDataFileNum; i++) { + referencedDataFiles[i] = in.readUTF(); + } + + return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java new file mode 100644 index 000000000000..18b269d6c3e9 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java @@ -0,0 +1,86 @@ +/* + * 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 org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.StructLikeWrapper; +import org.apache.iceberg.util.StructProjection; + +/** + * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record + * will be emitted to same writer in order. + */ +class EqualityFieldKeySelector implements KeySelector { + + private final Schema schema; + private final RowType flinkSchema; + private final Schema deleteSchema; + + private transient RowDataWrapper rowDataWrapper; + private transient StructProjection structProjection; + private transient StructLikeWrapper structLikeWrapper; + + EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { + this.schema = schema; + this.flinkSchema = flinkSchema; + this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); + } + + /** + * Construct the {@link RowDataWrapper} lazily here because few members in it are not + * serializable. In this way, we don't have to serialize them with forcing. + */ + protected RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + return rowDataWrapper; + } + + /** Construct the {@link StructProjection} lazily because it is not serializable. */ + protected StructProjection lazyStructProjection() { + if (structProjection == null) { + structProjection = StructProjection.create(schema, deleteSchema); + } + return structProjection; + } + + /** Construct the {@link StructLikeWrapper} lazily because it is not serializable. */ + protected StructLikeWrapper lazyStructLikeWrapper() { + if (structLikeWrapper == null) { + structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct()); + } + return structLikeWrapper; + } + + @Override + public Integer getKey(RowData row) { + RowDataWrapper wrappedRowData = lazyRowDataWrapper().wrap(row); + StructProjection projectedRowData = lazyStructProjection().wrap(wrappedRowData); + StructLikeWrapper wrapper = lazyStructLikeWrapper().set(projectedRowData); + return wrapper.hashCode(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java new file mode 100644 index 000000000000..b6f1392d1562 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java @@ -0,0 +1,274 @@ +/* + * 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.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkOrcWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +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; + +public class FlinkAppenderFactory implements FileAppenderFactory, Serializable { + private final Schema schema; + private final RowType flinkSchema; + private final Map props; + private final PartitionSpec spec; + private final int[] equalityFieldIds; + private final Schema eqDeleteRowSchema; + private final Schema posDeleteRowSchema; + private final Table table; + + private RowType eqDeleteFlinkSchema = null; + private RowType posDeleteFlinkSchema = null; + + public FlinkAppenderFactory( + Table table, + Schema schema, + RowType flinkSchema, + Map props, + PartitionSpec spec, + int[] equalityFieldIds, + Schema eqDeleteRowSchema, + Schema posDeleteRowSchema) { + Preconditions.checkNotNull(table, "Table shouldn't be null"); + this.table = table; + this.schema = schema; + this.flinkSchema = flinkSchema; + this.props = props; + this.spec = spec; + this.equalityFieldIds = equalityFieldIds; + this.eqDeleteRowSchema = eqDeleteRowSchema; + this.posDeleteRowSchema = posDeleteRowSchema; + } + + private RowType lazyEqDeleteFlinkSchema() { + if (eqDeleteFlinkSchema == null) { + Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); + this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema); + } + return eqDeleteFlinkSchema; + } + + private RowType lazyPosDeleteFlinkSchema() { + if (posDeleteFlinkSchema == null) { + Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null"); + this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema); + } + return this.posDeleteFlinkSchema; + } + + @Override + public FileAppender newAppender(OutputFile outputFile, FileFormat format) { + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + try { + switch (format) { + case AVRO: + return Avro.write(outputFile) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .setAll(props) + .schema(schema) + .metricsConfig(metricsConfig) + .overwrite() + .build(); + + case ORC: + return ORC.write(outputFile) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + case PARQUET: + return Parquet.write(outputFile) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) + .setAll(props) + .metricsConfig(metricsConfig) + .schema(schema) + .overwrite() + .build(); + + default: + throw new UnsupportedOperationException("Cannot write unknown file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(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 outputFile, 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"); + + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case ORC: + return ORC.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + case PARQUET: + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(eqDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .equalityFieldIds(equalityFieldIds) + .buildEqualityWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write equality-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public PositionDeleteWriter newPosDeleteWriter( + EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { + MetricsConfig metricsConfig = MetricsConfig.forPositionDelete(table); + try { + switch (format) { + case AVRO: + return Avro.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .buildPositionWriter(); + + case ORC: + RowType orcPosDeleteSchema = + FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); + return ORC.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .transformPaths(path -> StringData.fromString(path.toString())) + .buildPositionWriter(); + + case PARQUET: + RowType flinkPosDeleteSchema = + FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); + return Parquet.writeDeletes(outputFile.encryptingOutputFile()) + .createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) + .withPartition(partition) + .overwrite() + .setAll(props) + .metricsConfig(metricsConfig) + .rowSchema(posDeleteRowSchema) + .withSpec(spec) + .withKeyMetadata(outputFile.keyMetadata()) + .transformPaths(path -> StringData.fromString(path.toString())) + .buildPositionWriter(); + + default: + throw new UnsupportedOperationException( + "Cannot write pos-deletes for unsupported file format: " + format); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java new file mode 100644 index 000000000000..2183fe062af4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java @@ -0,0 +1,293 @@ +/* + * 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 static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; + +import java.io.Serializable; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.BaseFileWriterFactory; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.data.FlinkAvroWriter; +import org.apache.iceberg.flink.data.FlinkOrcWriter; +import org.apache.iceberg.flink.data.FlinkParquetWriters; +import org.apache.iceberg.io.DeleteSchemaUtil; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { + private RowType dataFlinkType; + private RowType equalityDeleteFlinkType; + private RowType positionDeleteFlinkType; + + FlinkFileWriterFactory( + Table table, + FileFormat dataFileFormat, + Schema dataSchema, + RowType dataFlinkType, + SortOrder dataSortOrder, + FileFormat deleteFileFormat, + int[] equalityFieldIds, + Schema equalityDeleteRowSchema, + RowType equalityDeleteFlinkType, + SortOrder equalityDeleteSortOrder, + Schema positionDeleteRowSchema, + RowType positionDeleteFlinkType) { + + super( + table, + dataFileFormat, + dataSchema, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteSortOrder, + positionDeleteRowSchema); + + this.dataFlinkType = dataFlinkType; + this.equalityDeleteFlinkType = equalityDeleteFlinkType; + this.positionDeleteFlinkType = positionDeleteFlinkType; + } + + static Builder builderFor(Table table) { + return new Builder(table); + } + + @Override + protected void configureDataWrite(Avro.DataWriteBuilder builder) { + builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); + } + + @Override + protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { + builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); + } + + @Override + protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { + int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME); + if (rowFieldIndex >= 0) { + // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos + RowType positionDeleteRowFlinkType = + (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex); + builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType)); + } + } + + @Override + protected void configureDataWrite(Parquet.DataWriteBuilder builder) { + builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); + } + + @Override + protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); + } + + @Override + protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { + builder.createWriterFunc( + msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType)); + builder.transformPaths(path -> StringData.fromString(path.toString())); + } + + @Override + protected void configureDataWrite(ORC.DataWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); + } + + @Override + protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); + } + + @Override + protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { + builder.createWriterFunc( + (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema)); + builder.transformPaths(path -> StringData.fromString(path.toString())); + } + + private RowType dataFlinkType() { + if (dataFlinkType == null) { + Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); + this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); + } + + return dataFlinkType; + } + + private RowType equalityDeleteFlinkType() { + if (equalityDeleteFlinkType == null) { + Preconditions.checkNotNull( + equalityDeleteRowSchema(), "Equality delete schema must not be null"); + this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); + } + + return equalityDeleteFlinkType; + } + + private RowType positionDeleteFlinkType() { + if (positionDeleteFlinkType == null) { + // wrap the optional row schema into the position delete schema that contains path and + // position + Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); + this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema); + } + + return positionDeleteFlinkType; + } + + static class Builder { + private final Table table; + private FileFormat dataFileFormat; + private Schema dataSchema; + private RowType dataFlinkType; + private SortOrder dataSortOrder; + private FileFormat deleteFileFormat; + private int[] equalityFieldIds; + private Schema equalityDeleteRowSchema; + private RowType equalityDeleteFlinkType; + private SortOrder equalityDeleteSortOrder; + private Schema positionDeleteRowSchema; + private RowType positionDeleteFlinkType; + + Builder(Table table) { + this.table = table; + + Map properties = table.properties(); + + String dataFileFormatName = + properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); + this.dataFileFormat = FileFormat.fromString(dataFileFormatName); + + String deleteFileFormatName = + properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); + this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); + } + + Builder dataFileFormat(FileFormat newDataFileFormat) { + this.dataFileFormat = newDataFileFormat; + return this; + } + + Builder dataSchema(Schema newDataSchema) { + this.dataSchema = newDataSchema; + return this; + } + + /** + * Sets a Flink type for data. + * + *

    If not set, the value is derived from the provided Iceberg schema. + */ + Builder dataFlinkType(RowType newDataFlinkType) { + this.dataFlinkType = newDataFlinkType; + return this; + } + + Builder dataSortOrder(SortOrder newDataSortOrder) { + this.dataSortOrder = newDataSortOrder; + return this; + } + + Builder deleteFileFormat(FileFormat newDeleteFileFormat) { + this.deleteFileFormat = newDeleteFileFormat; + return this; + } + + Builder equalityFieldIds(int[] newEqualityFieldIds) { + this.equalityFieldIds = newEqualityFieldIds; + return this; + } + + Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { + this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; + return this; + } + + /** + * Sets a Flink type for equality deletes. + * + *

    If not set, the value is derived from the provided Iceberg schema. + */ + Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) { + this.equalityDeleteFlinkType = newEqualityDeleteFlinkType; + return this; + } + + Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { + this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; + return this; + } + + Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { + this.positionDeleteRowSchema = newPositionDeleteRowSchema; + return this; + } + + /** + * Sets a Flink type for position deletes. + * + *

    If not set, the value is derived from the provided Iceberg schema. + */ + Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) { + this.positionDeleteFlinkType = newPositionDeleteFlinkType; + return this; + } + + FlinkFileWriterFactory build() { + boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; + boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; + Preconditions.checkArgument( + noEqualityDeleteConf || fullEqualityDeleteConf, + "Equality field IDs and equality delete row schema must be set together"); + + return new FlinkFileWriterFactory( + table, + dataFileFormat, + dataSchema, + dataFlinkType, + dataSortOrder, + deleteFileFormat, + equalityFieldIds, + equalityDeleteRowSchema, + equalityDeleteFlinkType, + equalityDeleteSortOrder, + positionDeleteRowSchema, + positionDeleteFlinkType); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java new file mode 100644 index 000000000000..c7e8a2dea7cb --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -0,0 +1,132 @@ +/* + * 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.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.function.Supplier; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class FlinkManifestUtil { + private static final int FORMAT_V2 = 2; + private static final Long DUMMY_SNAPSHOT_ID = 0L; + + private FlinkManifestUtil() {} + + static ManifestFile writeDataFiles( + OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { + ManifestWriter writer = + ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); + + try (ManifestWriter closeableWriter = writer) { + closeableWriter.addAll(dataFiles); + } + + return writer.toManifestFile(); + } + + static List readDataFiles( + ManifestFile manifestFile, FileIO io, Map specsById) + throws IOException { + try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io, specsById)) { + return Lists.newArrayList(dataFiles); + } + } + + static ManifestOutputFileFactory createOutputFileFactory( + Supplier

    tableSupplier, + Map tableProps, + String flinkJobId, + String operatorUniqueId, + int subTaskId, + long attemptNumber) { + return new ManifestOutputFileFactory( + tableSupplier, tableProps, flinkJobId, operatorUniqueId, subTaskId, attemptNumber); + } + + /** + * Write the {@link WriteResult} to temporary manifest files. + * + * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same + * partition spec + */ + static DeltaManifests writeCompletedFiles( + WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) + throws IOException { + + ManifestFile dataManifest = null; + ManifestFile deleteManifest = null; + + // Write the completed data files into a newly created data manifest file. + if (result.dataFiles() != null && result.dataFiles().length > 0) { + dataManifest = + writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); + } + + // Write the completed delete files into a newly created delete manifest file. + if (result.deleteFiles() != null && result.deleteFiles().length > 0) { + OutputFile deleteManifestFile = outputFileSupplier.get(); + + ManifestWriter deleteManifestWriter = + ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); + try (ManifestWriter writer = deleteManifestWriter) { + for (DeleteFile deleteFile : result.deleteFiles()) { + writer.add(deleteFile); + } + } + + deleteManifest = deleteManifestWriter.toManifestFile(); + } + + return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); + } + + static WriteResult readCompletedFiles( + DeltaManifests deltaManifests, FileIO io, Map specsById) + throws IOException { + WriteResult.Builder builder = WriteResult.builder(); + + // Read the completed data files from persisted data manifest file. + if (deltaManifests.dataManifest() != null) { + builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io, specsById)); + } + + // Read the completed delete files from persisted delete manifests file. + if (deltaManifests.deleteManifest() != null) { + try (CloseableIterable deleteFiles = + ManifestFiles.readDeleteManifest(deltaManifests.deleteManifest(), io, specsById)) { + builder.addDeleteFiles(deleteFiles); + } + } + + return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); + } +} 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 new file mode 100644 index 000000000000..769af7d77140 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -0,0 +1,654 @@ +/* + * 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 static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Function; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +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.streaming.api.datastream.SingleOutputStreamOperator; +import org.apache.flink.streaming.api.functions.sink.DiscardingSink; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +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.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class FlinkSink { + private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class); + + private static final String ICEBERG_STREAM_WRITER_NAME = + IcebergStreamWriter.class.getSimpleName(); + private static final String ICEBERG_FILES_COMMITTER_NAME = + IcebergFilesCommitter.class.getSimpleName(); + + private FlinkSink() {} + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } + + public static class Builder { + private Function> inputCreator = null; + private TableLoader tableLoader; + private Table table; + private TableSchema tableSchema; + private List equalityFieldColumns = null; + private String uidPrefix = null; + private final Map snapshotProperties = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private final Map writeOptions = Maps.newHashMap(); + private FlinkWriteConf flinkWriteConf = null; + + private Builder() {} + + private Builder forRowData(DataStream newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forMapperOutputType( + DataStream input, MapFunction mapper, TypeInformation outputType) { + this.inputCreator = + newUidPrefix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism + // of map operator same as its input to keep map operator chaining its input, and avoid + // rebalanced by default. + SingleOutputStreamOperator inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidPrefix != null) { + inputStream.name(operatorName(newUidPrefix)).uid(newUidPrefix + "-mapper"); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} + * which will write all the records into {@link DataFile}s and emit them to downstream operator. + * Providing a table would avoid so many table loading from each separate task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + /** + * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need + * this loader because {@link Table} is not serializable and could not just use the loaded table + * from Builder#table in the remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + /** + * Set the write properties for Flink sink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for Flink sink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of + * multiple operators (like writer, committer, dummy sink etc.) Actually operator uid will be + * appended with a suffix like "uidPrefix-writer".
    + *
    + * If provided, this prefix is also applied to operator names.
    + *
    + * Flink auto generates operator uid if not set explicitly. It is a recommended + * best-practice to set uid for all operators before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid.
    + *
    + * Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous Flink sink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore Flink sink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newPrefix prefix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidPrefix(String newPrefix) { + this.uidPrefix = newPrefix; + return this; + } + + public Builder setSnapshotProperties(Map properties) { + snapshotProperties.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotProperties.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + private DataStreamSink chainIcebergOperators() { + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); + + DataStream rowDataInput = inputCreator.apply(uidPrefix); + + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + this.table = loader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + // Find out the equality field id list based on the user-provided equality field column names. + List equalityFieldIds = checkAndGetEqualityFieldIds(); + + // Convert the requested flink table schema to flink row type. + RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + + // Distribute the records from input data stream based on the write.distribution-mode and + // equality fields. + DataStream distributeStream = + distributeDataStream( + rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + + // Add parallel writers that append rows to files + SingleOutputStreamOperator writerStream = + appendWriter(distributeStream, flinkRowType, equalityFieldIds); + + // Add single-parallelism committer that commits files + // after successful checkpoint or end of input + SingleOutputStreamOperator committerStream = appendCommitter(writerStream); + + // Add dummy discard sink + return appendDummySink(committerStream); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + return chainIcebergOperators(); + } + + private String operatorName(String suffix) { + return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; + } + + @VisibleForTesting + List checkAndGetEqualityFieldIds() { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = + Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + @SuppressWarnings("unchecked") + private DataStreamSink appendDummySink( + SingleOutputStreamOperator committerStream) { + DataStreamSink resultStream = + committerStream + .addSink(new DiscardingSink()) + .name(operatorName(String.format("IcebergSink %s", this.table.name()))) + .setParallelism(1); + if (uidPrefix != null) { + resultStream = resultStream.uid(uidPrefix + "-dummysink"); + } + return resultStream; + } + + private SingleOutputStreamOperator appendCommitter( + SingleOutputStreamOperator writerStream) { + IcebergFilesCommitter filesCommitter = + new IcebergFilesCommitter( + tableLoader, + flinkWriteConf.overwriteMode(), + snapshotProperties, + flinkWriteConf.workerPoolSize(), + flinkWriteConf.branch(), + table.spec()); + SingleOutputStreamOperator committerStream = + writerStream + .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) + .setParallelism(1) + .setMaxParallelism(1); + if (uidPrefix != null) { + committerStream = committerStream.uid(uidPrefix + "-committer"); + } + return committerStream; + } + + private SingleOutputStreamOperator appendWriter( + DataStream input, RowType flinkRowType, List equalityFieldIds) { + // Validate the equality fields and partition fields if we enable the upsert mode. + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !flinkWriteConf.overwriteMode(), + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + + SerializableSupplier
    tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = + new CachingTableSupplier(serializableTable, tableLoader, tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + IcebergStreamWriter streamWriter = + createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); + + int parallelism = + flinkWriteConf.writeParallelism() == null + ? input.getParallelism() + : flinkWriteConf.writeParallelism(); + SingleOutputStreamOperator writerStream = + input + .transform( + operatorName(ICEBERG_STREAM_WRITER_NAME), + TypeInformation.of(WriteResult.class), + streamWriter) + .setParallelism(parallelism); + if (uidPrefix != null) { + writerStream = writerStream.uid(uidPrefix + "-writer"); + } + return writerStream; + } + + private DataStream distributeDataStream( + DataStream input, + List equalityFieldIds, + PartitionSpec partitionSpec, + Schema iSchema, + RowType flinkRowType) { + DistributionMode writeMode = flinkWriteConf.distributionMode(); + + LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + switch (writeMode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (partitionSpec.isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } + } else { + if (partitionSpec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : partitionSpec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy( + new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); + } + } + } + + static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to + // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 + // 'byte'), we will + // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here + // we must use flink + // schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + static IcebergStreamWriter createStreamWriter( + SerializableSupplier
    tableSupplier, + FlinkWriteConf flinkWriteConf, + RowType flinkRowType, + List equalityFieldIds) { + Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); + + Table initTable = tableSupplier.get(); + FileFormat format = flinkWriteConf.dataFileFormat(); + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + flinkWriteConf.targetDataFileSize(), + format, + writeProperties(initTable, format, flinkWriteConf), + equalityFieldIds, + flinkWriteConf.upsertMode()); + + return new IcebergStreamWriter<>(initTable.name(), taskWriterFactory); + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java new file mode 100644 index 000000000000..b9bceaa9311d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -0,0 +1,516 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeinfo.BasicTypeInfo; +import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +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; +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.types.Comparators; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergFilesCommitter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + private static final long INITIAL_CHECKPOINT_ID = -1L; + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + + private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class); + private static final String FLINK_JOB_ID = "flink.job-id"; + private static final String OPERATOR_ID = "flink.operator-id"; + + // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always + // increasing, so we could correctly commit all the data files whose checkpoint id is greater than + // the max committed one to iceberg table, for avoiding committing the same data files twice. This + // id will be attached to iceberg's meta when committing the iceberg transaction. + private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + // TableLoader to load iceberg table lazily. + private final TableLoader tableLoader; + private final boolean replacePartitions; + private final Map snapshotProperties; + + // A sorted map to maintain the completed data files for each pending checkpointId (which have not + // been committed to iceberg table). We need a sorted map here because there's possible that few + // checkpoints snapshot failed, for example: the 1st checkpoint have 2 data files <1, >, the 2st checkpoint have 1 data files <2, >. Snapshot for checkpoint#1 + // interrupted because of network/disk failure etc, while we don't expect any data loss in iceberg + // table. So we keep the finished files <1, > in memory and retry to commit iceberg + // table when the next checkpoint happen. + private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); + + // The completed files cache for current checkpoint. Once the snapshot barrier received, it will + // be flushed to the 'dataFilesPerCheckpoint'. + private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final String branch; + + // It will have an unique identifier for one job. + private transient String flinkJobId; + private transient String operatorUniqueId; + private transient Table table; + private transient IcebergFilesCommitterMetrics committerMetrics; + private transient ManifestOutputFileFactory manifestOutputFileFactory; + private transient long maxCommittedCheckpointId; + private transient int continuousEmptyCheckpoints; + private transient int maxContinuousEmptyCommits; + // There're two cases that we restore from flink checkpoints: the first case is restoring from + // snapshot created by the same flink job; another case is restoring from snapshot created by + // another different job. For the second case, we need to maintain the old flink job's id in flink + // state backend to find the max-committed-checkpoint-id when traversing iceberg table's + // snapshots. + private static final ListStateDescriptor JOB_ID_DESCRIPTOR = + new ListStateDescriptor<>("iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); + private transient ListState jobIdState; + // All pending checkpoints states for this function. + private static final ListStateDescriptor> STATE_DESCRIPTOR = + buildStateDescriptor(); + private transient ListState> checkpointsState; + + private final Integer workerPoolSize; + private final PartitionSpec spec; + private transient ExecutorService workerPool; + + IcebergFilesCommitter( + TableLoader tableLoader, + boolean replacePartitions, + Map snapshotProperties, + Integer workerPoolSize, + String branch, + PartitionSpec spec) { + this.tableLoader = tableLoader; + this.replacePartitions = replacePartitions; + this.snapshotProperties = snapshotProperties; + this.workerPoolSize = workerPoolSize; + this.branch = branch; + this.spec = spec; + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + this.operatorUniqueId = getRuntimeContext().getOperatorUniqueID(); + + // Open the table loader and load the table. + this.tableLoader.open(); + this.table = tableLoader.loadTable(); + this.committerMetrics = new IcebergFilesCommitterMetrics(super.metrics, table.name()); + + maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + + int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + int attemptId = getRuntimeContext().getAttemptNumber(); + this.manifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); + this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); + this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); + if (context.isRestored()) { + Iterable jobIdIterable = jobIdState.get(); + if (jobIdIterable == null || !jobIdIterable.iterator().hasNext()) { + LOG.warn( + "Failed to restore committer state. This can happen when operator uid changed and Flink " + + "allowNonRestoredState is enabled. Best practice is to explicitly set the operator id " + + "via FlinkSink#Builder#uidPrefix() so that the committer operator uid is stable. " + + "Otherwise, Flink auto generate an operator uid based on job topology." + + "With that, operator uid is subjective to change upon topology change."); + return; + } + + String restoredFlinkJobId = jobIdIterable.iterator().next(); + Preconditions.checkState( + !Strings.isNullOrEmpty(restoredFlinkJobId), + "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); + + // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new + // flink job even if it's restored from a snapshot created by another different flink job, so + // it's safe to assign the max committed checkpoint id from restored flink job to the current + // flink job. + this.maxCommittedCheckpointId = + getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + + NavigableMap uncommittedDataFiles = + Maps.newTreeMap(checkpointsState.get().iterator().next()) + .tailMap(maxCommittedCheckpointId, false); + if (!uncommittedDataFiles.isEmpty()) { + // Committed all uncommitted data files from the old flink job to iceberg table. + long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey(); + commitUpToCheckpoint( + uncommittedDataFiles, restoredFlinkJobId, operatorUniqueId, maxUncommittedCheckpointId); + } + } + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + long checkpointId = context.getCheckpointId(); + LOG.info( + "Start to flush snapshot state to state backend, table: {}, checkpointId: {}", + table, + checkpointId); + + // Update the checkpoint state. + long startNano = System.nanoTime(); + dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + // Reset the snapshot state to the latest state. + checkpointsState.clear(); + checkpointsState.add(dataFilesPerCheckpoint); + + jobIdState.clear(); + jobIdState.add(flinkJobId); + + // Clear the local buffer for current checkpoint. + writeResultsOfCurrentCkpt.clear(); + committerMetrics.checkpointDuration( + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + super.notifyCheckpointComplete(checkpointId); + // It's possible that we have the following events: + // 1. snapshotState(ckpId); + // 2. snapshotState(ckpId+1); + // 3. notifyCheckpointComplete(ckpId+1); + // 4. notifyCheckpointComplete(ckpId); + // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all + // the files, + // Besides, we need to maintain the max-committed-checkpoint-id to be increasing. + if (checkpointId > maxCommittedCheckpointId) { + LOG.info("Checkpoint {} completed. Attempting commit.", checkpointId); + commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, checkpointId); + this.maxCommittedCheckpointId = checkpointId; + } else { + LOG.info( + "Skipping committing checkpoint {}. {} is already committed.", + checkpointId, + maxCommittedCheckpointId); + } + + // reload the table in case new configuration is needed + this.table = tableLoader.loadTable(); + } + + private void commitUpToCheckpoint( + NavigableMap deltaManifestsMap, + String newFlinkJobId, + String operatorId, + long checkpointId) + throws IOException { + NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); + List manifests = Lists.newArrayList(); + NavigableMap pendingResults = Maps.newTreeMap(); + for (Map.Entry e : pendingMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { + // Skip the empty flink manifest. + continue; + } + + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + committerMetrics.updateCommitSummary(summary); + pendingMap.clear(); + deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + } + + private void commitPendingResult( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); + } + continuousEmptyCheckpoints = 0; + } else { + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void deleteCommittedManifests( + List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(this) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } + + private void replacePartitions( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + + commitOperation( + dynamicOverwrite, + summary, + "dynamic partition overwrite", + newFlinkJobId, + operatorId, + checkpointId); + } + + private void commitDeltaTxn( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId, + long checkpointId) { + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + commitOperation(appendFiles, summary, "append", newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + commitOperation(rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate operation, + CommitSummary summary, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + LOG.info( + "Committing {} for checkpoint {} to table {} branch {} with summary: {}", + description, + checkpointId, + table.name(), + branch, + summary); + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(FLINK_JOB_ID, newFlinkJobId); + operation.set(OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + committerMetrics.commitDuration(durationMs); + } + + @Override + public void processElement(StreamRecord element) { + this.writeResultsOfCurrentCkpt.add(element.getValue()); + } + + @Override + public void endInput() throws IOException { + // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. + long currentCheckpointId = Long.MAX_VALUE; + dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); + writeResultsOfCurrentCkpt.clear(); + + commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); + } + + /** + * Write all the complete data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + private byte[] writeToManifest(long checkpointId) throws IOException { + if (writeResultsOfCurrentCkpt.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> manifestOutputFileFactory.create(checkpointId), spec); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void open() throws Exception { + super.open(); + + final String operatorID = getRuntimeContext().getOperatorUniqueID(); + this.workerPool = + ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); + } + + @Override + public void close() throws Exception { + if (tableLoader != null) { + tableLoader.close(); + } + + if (workerPool != null) { + workerPool.shutdown(); + } + } + + @VisibleForTesting + static ListStateDescriptor> buildStateDescriptor() { + Comparator longComparator = Comparators.forType(Types.LongType.get()); + // Construct a SortedMapTypeInfo. + SortedMapTypeInfo sortedMapTypeInfo = + new SortedMapTypeInfo<>( + BasicTypeInfo.LONG_TYPE_INFO, + PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, + longComparator); + return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java new file mode 100644 index 000000000000..5b28c4acb1c5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java @@ -0,0 +1,73 @@ +/* + * 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.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.flink.util.ElapsedTimeGauge; + +class IcebergFilesCommitterMetrics { + private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); + private final AtomicLong lastCommitDurationMs = new AtomicLong(); + private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; + private final Counter committedDataFilesCount; + private final Counter committedDataFilesRecordCount; + private final Counter committedDataFilesByteCount; + private final Counter committedDeleteFilesCount; + private final Counter committedDeleteFilesRecordCount; + private final Counter committedDeleteFilesByteCount; + + IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup committerMetrics = + metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); + committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); + committerMetrics.gauge("lastCommitDurationMs", lastCommitDurationMs::get); + this.elapsedSecondsSinceLastSuccessfulCommit = new ElapsedTimeGauge(TimeUnit.SECONDS); + committerMetrics.gauge( + "elapsedSecondsSinceLastSuccessfulCommit", elapsedSecondsSinceLastSuccessfulCommit); + this.committedDataFilesCount = committerMetrics.counter("committedDataFilesCount"); + this.committedDataFilesRecordCount = committerMetrics.counter("committedDataFilesRecordCount"); + this.committedDataFilesByteCount = committerMetrics.counter("committedDataFilesByteCount"); + this.committedDeleteFilesCount = committerMetrics.counter("committedDeleteFilesCount"); + this.committedDeleteFilesRecordCount = + committerMetrics.counter("committedDeleteFilesRecordCount"); + this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); + } + + void checkpointDuration(long checkpointDurationMs) { + lastCheckpointDurationMs.set(checkpointDurationMs); + } + + void commitDuration(long commitDurationMs) { + lastCommitDurationMs.set(commitDurationMs); + } + + /** This is called upon a successful commit. */ + void updateCommitSummary(CommitSummary stats) { + elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); + committedDataFilesCount.inc(stats.dataFilesCount()); + committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); + committedDataFilesByteCount.inc(stats.dataFilesByteCount()); + committedDeleteFilesCount.inc(stats.deleteFilesCount()); + committedDeleteFilesRecordCount.inc(stats.deleteFilesRecordCount()); + committedDeleteFilesByteCount.inc(stats.deleteFilesByteCount()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java new file mode 100644 index 000000000000..9ea0349fb057 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -0,0 +1,120 @@ +/* + * 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.io.IOException; +import java.util.concurrent.TimeUnit; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.ChainingStrategy; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { + + private static final long serialVersionUID = 1L; + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + + private transient TaskWriter writer; + private transient int subTaskId; + private transient int attemptId; + private transient IcebergStreamWriterMetrics writerMetrics; + + IcebergStreamWriter(String fullTableName, TaskWriterFactory taskWriterFactory) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + setChainingStrategy(ChainingStrategy.ALWAYS); + } + + @Override + public void open() { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); + + // Initialize the task writer factory. + this.taskWriterFactory.initialize(subTaskId, attemptId); + + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { + flush(); + this.writer = taskWriterFactory.create(); + } + + @Override + public void processElement(StreamRecord element) throws Exception { + writer.write(element.getValue()); + } + + @Override + public void close() throws Exception { + super.close(); + if (writer != null) { + writer.close(); + writer = null; + } + } + + @Override + public void endInput() throws IOException { + // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the + // remaining completed files to downstream before closing the writer so that we won't miss any + // of them. + // Note that if the task is not closed after calling endInput, checkpoint may be triggered again + // causing files to be sent repeatedly, the writer is marked as null after the last file is sent + // to guard against duplicated writes. + flush(); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("table_name", fullTableName) + .add("subtask_id", subTaskId) + .add("attempt_id", attemptId) + .toString(); + } + + /** close all open files and emit files to downstream committer operator */ + private void flush() throws IOException { + if (writer == null) { + return; + } + + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + writerMetrics.updateFlushResult(result); + output.collect(new StreamRecord<>(result)); + writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + + // Set writer to null to prevent duplicate flushes in the corner case of + // prepareSnapshotPreBarrier happening after endInput. + writer = null; + } +} 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 new file mode 100644 index 000000000000..ce2a6c583fdf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java @@ -0,0 +1,89 @@ +/* + * 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 com.codahale.metrics.SlidingWindowReservoir; +import java.util.Arrays; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Histogram; +import org.apache.flink.metrics.MetricGroup; +import org.apache.iceberg.io.WriteResult; + +class IcebergStreamWriterMetrics { + // 1,024 reservoir size should cost about 8KB, which is quite small. + // It should also produce good accuracy for histogram distribution (like percentiles). + private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; + + private final Counter flushedDataFiles; + private final Counter flushedDeleteFiles; + private final Counter flushedReferencedDataFiles; + private final AtomicLong lastFlushDurationMs; + private final Histogram dataFilesSizeHistogram; + private final Histogram deleteFilesSizeHistogram; + + IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup writerMetrics = + metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); + this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); + this.flushedDeleteFiles = writerMetrics.counter("flushedDeleteFiles"); + this.flushedReferencedDataFiles = writerMetrics.counter("flushedReferencedDataFiles"); + this.lastFlushDurationMs = new AtomicLong(); + writerMetrics.gauge("lastFlushDurationMs", lastFlushDurationMs::get); + + com.codahale.metrics.Histogram dropwizardDataFilesSizeHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); + this.dataFilesSizeHistogram = + writerMetrics.histogram( + "dataFilesSizeHistogram", + new DropwizardHistogramWrapper(dropwizardDataFilesSizeHistogram)); + com.codahale.metrics.Histogram dropwizardDeleteFilesSizeHistogram = + new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); + this.deleteFilesSizeHistogram = + writerMetrics.histogram( + "deleteFilesSizeHistogram", + new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); + } + + void updateFlushResult(WriteResult result) { + flushedDataFiles.inc(result.dataFiles().length); + flushedDeleteFiles.inc(result.deleteFiles().length); + flushedReferencedDataFiles.inc(result.referencedDataFiles().length); + + // For file size distribution histogram, we don't have to update them after successful commits. + // This should works equally well and we avoided the overhead of tracking the list of file sizes + // in the {@link CommitSummary}, which currently stores simple stats for counters and gauges + // metrics. + Arrays.stream(result.dataFiles()) + .forEach( + dataFile -> { + dataFilesSizeHistogram.update(dataFile.fileSizeInBytes()); + }); + Arrays.stream(result.deleteFiles()) + .forEach( + deleteFile -> { + deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); + }); + } + + void flushDuration(long flushDurationMs) { + lastFlushDurationMs.set(flushDurationMs); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java new file mode 100644 index 000000000000..da5e6e7627ae --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -0,0 +1,94 @@ +/* + * 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.Map; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.base.Strings; + +class ManifestOutputFileFactory { + // Users could define their own flink manifests directory by setting this value in table + // properties. + static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; + + private final Supplier
    tableSupplier; + private final Map props; + private final String flinkJobId; + private final String operatorUniqueId; + private final int subTaskId; + private final long attemptNumber; + private final AtomicInteger fileCount = new AtomicInteger(0); + + ManifestOutputFileFactory( + Supplier
    tableSupplier, + Map props, + String flinkJobId, + String operatorUniqueId, + int subTaskId, + long attemptNumber) { + this.tableSupplier = tableSupplier; + this.props = props; + this.flinkJobId = flinkJobId; + this.operatorUniqueId = operatorUniqueId; + this.subTaskId = subTaskId; + this.attemptNumber = attemptNumber; + } + + private String generatePath(long checkpointId) { + return FileFormat.AVRO.addExtension( + String.format( + "%s-%s-%05d-%d-%d-%05d", + flinkJobId, + operatorUniqueId, + subTaskId, + attemptNumber, + checkpointId, + fileCount.incrementAndGet())); + } + + OutputFile create(long checkpointId) { + String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); + TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); + + String newManifestFullPath; + if (Strings.isNullOrEmpty(flinkManifestDir)) { + // User don't specify any flink manifest directory, so just use the default metadata path. + newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); + } else { + newManifestFullPath = + String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); + } + + return tableSupplier.get().io().newOutputFile(newManifestFullPath); + } + + private static String stripTrailingSlash(String path) { + String result = path; + while (result.endsWith("/")) { + result = result.substring(0, result.length() - 1); + } + return result; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java new file mode 100644 index 000000000000..df951684b446 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.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.sink; + +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.RowDataWrapper; + +/** + * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be + * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy + * for {@link FlinkSink}. + */ +class PartitionKeySelector implements KeySelector { + + private final Schema schema; + private final PartitionKey partitionKey; + private final RowType flinkSchema; + + private transient RowDataWrapper rowDataWrapper; + + PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { + this.schema = schema; + this.partitionKey = new PartitionKey(spec, schema); + this.flinkSchema = flinkSchema; + } + + /** + * Construct the {@link RowDataWrapper} lazily here because few members in it are not + * serializable. In this way, we don't have to serialize them with forcing. + */ + private RowDataWrapper lazyRowDataWrapper() { + if (rowDataWrapper == null) { + rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + return rowDataWrapper; + } + + @Override + public String getKey(RowData row) { + partitionKey.partition(lazyRowDataWrapper().wrap(row)); + return partitionKey.toPath(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java new file mode 100644 index 000000000000..38062dd1a2c4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java @@ -0,0 +1,97 @@ +/* + * 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.io.IOException; +import java.io.UncheckedIOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Tasks; + +class PartitionedDeltaWriter extends BaseDeltaTaskWriter { + + private final PartitionKey partitionKey; + + private final Map writers = Maps.newHashMap(); + + PartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + flinkSchema, + equalityFieldIds, + upsert); + this.partitionKey = new PartitionKey(spec, schema); + } + + @Override + RowDataDeltaWriter route(RowData row) { + partitionKey.partition(wrapper().wrap(row)); + + RowDataDeltaWriter writer = writers.get(partitionKey); + if (writer == null) { + // NOTICE: we need to copy a new partition key here, in case of messing up the keys in + // writers. + PartitionKey copiedKey = partitionKey.copy(); + writer = new RowDataDeltaWriter(copiedKey); + writers.put(copiedKey, writer); + } + + return writer; + } + + @Override + public void close() { + try { + Tasks.foreach(writers.values()) + .throwFailureWhenFinished() + .noRetry() + .run(RowDataDeltaWriter::close, IOException.class); + + writers.clear(); + } catch (IOException e) { + throw new UncheckedIOException("Failed to close equality delta writer", e); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java new file mode 100644 index 000000000000..67422a1afeb1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java @@ -0,0 +1,244 @@ +/* + * 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 java.util.function.Supplier; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; +import org.apache.iceberg.io.PartitionedFanoutWriter; +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.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.SerializableSupplier; + +public class RowDataTaskWriterFactory implements TaskWriterFactory { + private final Supplier
    tableSupplier; + private final Schema schema; + private final RowType flinkSchema; + private final PartitionSpec spec; + private final long targetFileSizeBytes; + private final FileFormat format; + private final List equalityFieldIds; + private final boolean upsert; + private final FileAppenderFactory appenderFactory; + + private transient OutputFileFactory outputFileFactory; + + public RowDataTaskWriterFactory( + Table table, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert) { + this( + () -> table, + flinkSchema, + targetFileSizeBytes, + format, + writeProperties, + equalityFieldIds, + upsert); + } + + public RowDataTaskWriterFactory( + SerializableSupplier
    tableSupplier, + RowType flinkSchema, + long targetFileSizeBytes, + FileFormat format, + Map writeProperties, + List equalityFieldIds, + boolean upsert) { + this.tableSupplier = tableSupplier; + + Table table; + if (tableSupplier instanceof CachingTableSupplier) { + // rely on the initial table metadata for schema, etc., until schema evolution is supported + table = ((CachingTableSupplier) tableSupplier).initialTable(); + } else { + table = tableSupplier.get(); + } + + this.schema = table.schema(); + this.flinkSchema = flinkSchema; + this.spec = table.spec(); + this.targetFileSizeBytes = targetFileSizeBytes; + this.format = format; + this.equalityFieldIds = equalityFieldIds; + this.upsert = upsert; + + if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { + this.appenderFactory = + new FlinkAppenderFactory( + table, schema, flinkSchema, writeProperties, spec, null, null, null); + } else if (upsert) { + // In upsert mode, only the new row is emitted using INSERT row kind. Therefore, any column of + // the inserted row + // may differ from the deleted row other than the primary key fields, and the delete file must + // contain values + // that are correct for the deleted row. Therefore, only write the equality delete fields. + this.appenderFactory = + new FlinkAppenderFactory( + table, + schema, + flinkSchema, + writeProperties, + spec, + ArrayUtil.toIntArray(equalityFieldIds), + TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), + null); + } else { + this.appenderFactory = + new FlinkAppenderFactory( + table, + schema, + flinkSchema, + writeProperties, + spec, + ArrayUtil.toIntArray(equalityFieldIds), + schema, + null); + } + } + + @Override + public void initialize(int taskId, int attemptId) { + Table table; + if (tableSupplier instanceof CachingTableSupplier) { + // rely on the initial table metadata for schema, etc., until schema evolution is supported + table = ((CachingTableSupplier) tableSupplier).initialTable(); + } else { + table = tableSupplier.get(); + } + + refreshTable(); + + this.outputFileFactory = + OutputFileFactory.builderFor(table, taskId, attemptId) + .format(format) + .ioSupplier(() -> tableSupplier.get().io()) + .build(); + } + + @Override + public TaskWriter create() { + Preconditions.checkNotNull( + outputFileFactory, + "The outputFileFactory shouldn't be null if we have invoked the initialize()."); + + refreshTable(); + + if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { + // Initialize a task writer to write INSERT only. + if (spec.isUnpartitioned()) { + return new UnpartitionedWriter<>( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes); + } else { + return new RowDataPartitionedFanoutWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema); + } + } else { + // Initialize a task writer to write both INSERT and equality DELETE. + if (spec.isUnpartitioned()) { + return new UnpartitionedDeltaWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema, + equalityFieldIds, + upsert); + } else { + return new PartitionedDeltaWriter( + spec, + format, + appenderFactory, + outputFileFactory, + tableSupplier.get().io(), + targetFileSizeBytes, + schema, + flinkSchema, + equalityFieldIds, + upsert); + } + } + } + + void refreshTable() { + if (tableSupplier instanceof CachingTableSupplier) { + ((CachingTableSupplier) tableSupplier).refreshTable(); + } + } + + private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { + + private final PartitionKey partitionKey; + private final RowDataWrapper rowDataWrapper; + + RowDataPartitionedFanoutWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema) { + super(spec, format, appenderFactory, fileFactory, io, targetFileSize); + this.partitionKey = new PartitionKey(spec, schema); + this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); + } + + @Override + protected PartitionKey partition(RowData row) { + partitionKey.partition(rowDataWrapper.wrap(row)); + return partitionKey; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java new file mode 100644 index 000000000000..e3a1245e8cbd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java @@ -0,0 +1,45 @@ +/* + * 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.io.Serializable; +import org.apache.iceberg.io.TaskWriter; + +/** + * Factory to create {@link TaskWriter} + * + * @param data type of record. + */ +public interface TaskWriterFactory extends Serializable { + + /** + * Initialize the factory with a given taskId and attemptId. + * + * @param taskId the identifier of task. + * @param attemptId the attempt id of this task. + */ + void initialize(int taskId, int attemptId); + + /** + * Initialize a {@link TaskWriter} with given task id and attempt id. + * + * @return a newly created task writer. + */ + TaskWriter create(); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java new file mode 100644 index 000000000000..7680fb933b20 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.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.flink.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFileFactory; + +class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { + private final RowDataDeltaWriter writer; + + UnpartitionedDeltaWriter( + PartitionSpec spec, + FileFormat format, + FileAppenderFactory appenderFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSize, + Schema schema, + RowType flinkSchema, + List equalityFieldIds, + boolean upsert) { + super( + spec, + format, + appenderFactory, + fileFactory, + io, + targetFileSize, + schema, + flinkSchema, + equalityFieldIds, + upsert); + this.writer = new RowDataDeltaWriter(null); + } + + @Override + RowDataDeltaWriter route(RowData row) { + return writer; + } + + @Override + public void close() throws IOException { + writer.close(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java new file mode 100644 index 000000000000..5525f02c873e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -0,0 +1,262 @@ +/* + * 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.Map; +import java.util.NavigableMap; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.datasketches.sampling.ReservoirItemsUnion; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link + * DataStatisticsOperator} subtasks for every checkpoint. + */ +class AggregatedStatisticsTracker { + private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); + + private final String operatorName; + private final int parallelism; + private final TypeSerializer statisticsSerializer; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final int switchToSketchThreshold; + private final NavigableMap aggregationsPerCheckpoint; + + private CompletedStatistics completedStatistics; + + AggregatedStatisticsTracker( + String operatorName, + int parallelism, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + int switchToSketchThreshold, + @Nullable CompletedStatistics restoredStatistics) { + this.operatorName = operatorName; + this.parallelism = parallelism; + this.statisticsSerializer = + new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.switchToSketchThreshold = switchToSketchThreshold; + this.completedStatistics = restoredStatistics; + + this.aggregationsPerCheckpoint = Maps.newTreeMap(); + } + + CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { + long checkpointId = event.checkpointId(); + LOG.debug( + "Handling statistics event from subtask {} of operator {} for checkpoint {}", + subtask, + operatorName, + checkpointId); + + if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { + LOG.info( + "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " + + "Was expecting data statistics from checkpoint higher than {}", + operatorName, + subtask, + checkpointId, + completedStatistics.checkpointId()); + return null; + } + + Aggregation aggregation = + aggregationsPerCheckpoint.computeIfAbsent( + checkpointId, + ignored -> + new Aggregation( + parallelism, + downstreamParallelism, + switchToSketchThreshold, + statisticsType, + StatisticsUtil.collectType(statisticsType, completedStatistics))); + DataStatistics dataStatistics = + StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); + if (!aggregation.merge(subtask, dataStatistics)) { + LOG.debug( + "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", + operatorName, + subtask, + checkpointId); + } + + if (aggregation.isComplete()) { + this.completedStatistics = aggregation.completedStatistics(checkpointId); + // clean up aggregations up to the completed checkpoint id + aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); + return completedStatistics; + } + + return null; + } + + @VisibleForTesting + NavigableMap aggregationsPerCheckpoint() { + return aggregationsPerCheckpoint; + } + + static class Aggregation { + private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); + + private final Set subtaskSet; + private final int parallelism; + private final int downstreamParallelism; + private final int switchToSketchThreshold; + private final StatisticsType configuredType; + private StatisticsType currentType; + private Map mapStatistics; + private ReservoirItemsUnion sketchStatistics; + + Aggregation( + int parallelism, + int downstreamParallelism, + int switchToSketchThreshold, + StatisticsType configuredType, + StatisticsType currentType) { + this.subtaskSet = Sets.newHashSet(); + this.parallelism = parallelism; + this.downstreamParallelism = downstreamParallelism; + this.switchToSketchThreshold = switchToSketchThreshold; + this.configuredType = configuredType; + this.currentType = currentType; + + if (currentType == StatisticsType.Map) { + this.mapStatistics = Maps.newHashMap(); + this.sketchStatistics = null; + } else { + this.mapStatistics = null; + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + } + } + + @VisibleForTesting + Set subtaskSet() { + return subtaskSet; + } + + @VisibleForTesting + StatisticsType currentType() { + return currentType; + } + + @VisibleForTesting + Map mapStatistics() { + return mapStatistics; + } + + @VisibleForTesting + ReservoirItemsUnion sketchStatistics() { + return sketchStatistics; + } + + private boolean isComplete() { + return subtaskSet.size() == parallelism; + } + + /** @return false if duplicate */ + private boolean merge(int subtask, DataStatistics taskStatistics) { + if (subtaskSet.contains(subtask)) { + return false; + } + + subtaskSet.add(subtask); + merge(taskStatistics); + return true; + } + + @SuppressWarnings("unchecked") + private void merge(DataStatistics taskStatistics) { + if (taskStatistics.type() == StatisticsType.Map) { + Map taskMapStats = (Map) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); + if (configuredType == StatisticsType.Auto + && mapStatistics.size() > switchToSketchThreshold) { + convertCoordinatorToSketch(); + } + } else { + // convert task stats to sketch first + ReservoirItemsSketch taskSketch = + ReservoirItemsSketch.newInstance( + SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); + SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); + sketchStatistics.update(taskSketch); + } + } else { + ReservoirItemsSketch taskSketch = + (ReservoirItemsSketch) taskStatistics.result(); + if (currentType == StatisticsType.Map) { + // convert global stats to sketch first + convertCoordinatorToSketch(); + } + + if (taskSketch.getNumSamples() > 0) { + sketchStatistics.update(taskSketch); + } + } + } + + private void convertCoordinatorToSketch() { + this.sketchStatistics = + ReservoirItemsUnion.newInstance( + SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); + SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); + this.currentType = StatisticsType.Sketch; + this.mapStatistics = null; + } + + private CompletedStatistics completedStatistics(long checkpointId) { + if (currentType == StatisticsType.Map) { + LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); + return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); + } else { + ReservoirItemsSketch sketch = sketchStatistics.getResult(); + if (sketch != null) { + LOG.info( + "Completed sketch statistics aggregation: " + + "reservoir size = {}, number of items seen = {}, number of samples = {}", + sketch.getK(), + sketch.getN(), + sketch.getNumSamples()); + return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); + } else { + LOG.info("Empty sketch statistics."); + return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); + } + } + } + } +} 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 new file mode 100644 index 000000000000..e4cba174f0f2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.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.flink.sink.shuffle; + +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** + * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation + * from all subtasks. It contains the raw statistics (Map or reservoir samples). + */ +class CompletedStatistics { + private final long checkpointId; + private final StatisticsType type; + private final Map keyFrequency; + private final SortKey[] keySamples; + + static CompletedStatistics fromKeyFrequency(long checkpointId, Map stats) { + return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); + } + + static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { + return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); + } + + CompletedStatistics( + long checkpointId, + StatisticsType type, + Map keyFrequency, + SortKey[] keySamples) { + this.checkpointId = checkpointId; + this.type = type; + this.keyFrequency = keyFrequency; + this.keySamples = keySamples; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("keyFrequency", keyFrequency) + .add("keySamples", keySamples) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof CompletedStatistics)) { + return false; + } + + CompletedStatistics other = (CompletedStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(keyFrequency, other.keyFrequency()) + && Arrays.equals(keySamples, other.keySamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + Map keyFrequency() { + return keyFrequency; + } + + SortKey[] keySamples() { + return keySamples; + } + + boolean isEmpty() { + if (type == StatisticsType.Sketch) { + return keySamples.length == 0; + } else { + return keyFrequency().isEmpty(); + } + } +} 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 new file mode 100644 index 000000000000..1ac0e386a011 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java @@ -0,0 +1,176 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; + +class CompletedStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer keyFrequencySerializer; + private final ListSerializer keySamplesSerializer; + + CompletedStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new CompletedStatisticsSerializer(sortKeySerializer); + } + + @Override + public CompletedStatistics createInstance() { + return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from) { + return new CompletedStatistics( + from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); + } + + @Override + public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + keyFrequencySerializer.serialize(record.keyFrequency(), target); + } else { + keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); + } + } + + @Override + public CompletedStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + Map keyFrequency = keyFrequencySerializer.deserialize(source); + return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); + } else { + List sortKeys = keySamplesSerializer.deserialize(source); + SortKey[] keySamples = new SortKey[sortKeys.size()]; + keySamples = sortKeys.toArray(keySamples); + return CompletedStatistics.fromKeySamples(checkpointId, keySamples); + } + } + + @Override + public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new CompletedStatisticsSerializerSnapshot(this); + } + + public static class CompletedStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public CompletedStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + CompletedStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new CompletedStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java new file mode 100644 index 000000000000..76c59cd5f4b8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java @@ -0,0 +1,48 @@ +/* + * 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.Map; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SortKey; + +/** + * DataStatistics defines the interface to collect data distribution information. + * + *

    Data statistics tracks traffic volume distribution across data keys. For low-cardinality key, + * a simple map of (key, count) can be used. For high-cardinality key, probabilistic data structures + * (sketching) can be used. + */ +@Internal +interface DataStatistics { + + StatisticsType type(); + + boolean isEmpty(); + + /** Add row sortKey to data statistics. */ + void add(SortKey sortKey); + + /** + * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link + * ReservoirItemsSketch} (high cardinality) + */ + Object result(); +} 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 new file mode 100644 index 000000000000..4bfde7204acf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java @@ -0,0 +1,522 @@ +/* + * 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.Comparator; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.ThreadFactory; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.util.ExceptionUtils; +import org.apache.flink.util.FatalExitExceptionHandler; +import org.apache.flink.util.FlinkRuntimeException; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.ThrowableCatchingRunnable; +import org.apache.flink.util.function.ThrowingRunnable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +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.jetbrains.annotations.NotNull; +import org.jetbrains.annotations.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} + * every subtask and then merge them together. Once aggregation for all subtasks data statistics + * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link + * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the + * aggregated data statistics to improve data clustering. + */ +@Internal +class DataStatisticsCoordinator implements OperatorCoordinator { + private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); + + private final String operatorName; + private final OperatorCoordinator.Context context; + private final Schema schema; + private final SortOrder sortOrder; + private final Comparator comparator; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final double closeFileCostWeightPercentage; + + private final ExecutorService coordinatorExecutor; + private final SubtaskGateways subtaskGateways; + private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; + private final TypeSerializer completedStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; + + private transient boolean started; + private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; + private transient CompletedStatistics completedStatistics; + private transient GlobalStatistics globalStatistics; + + DataStatisticsCoordinator( + String operatorName, + OperatorCoordinator.Context context, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType statisticsType, + double closeFileCostWeightPercentage) { + this.operatorName = operatorName; + this.context = context; + this.schema = schema; + this.sortOrder = sortOrder; + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + + this.coordinatorThreadFactory = + new CoordinatorExecutorThreadFactory( + "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); + this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); + this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public void start() throws Exception { + LOG.info("Starting data statistics coordinator: {}.", operatorName); + this.started = true; + + // statistics are restored already in resetToCheckpoint() before start() called + this.aggregatedStatisticsTracker = + new AggregatedStatisticsTracker( + operatorName, + context.currentParallelism(), + schema, + sortOrder, + downstreamParallelism, + statisticsType, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + completedStatistics); + } + + @Override + public void close() throws Exception { + coordinatorExecutor.shutdown(); + this.aggregatedStatisticsTracker = null; + this.started = false; + LOG.info("Closed data statistics coordinator: {}.", operatorName); + } + + @VisibleForTesting + void callInCoordinatorThread(Callable callable, String errorMessage) { + ensureStarted(); + // Ensure the task is done by the coordinator executor. + if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { + try { + Callable guardedCallable = + () -> { + try { + return callable.call(); + } catch (Throwable t) { + LOG.error( + "Uncaught Exception in data statistics coordinator: {} executor", + operatorName, + t); + ExceptionUtils.rethrowException(t); + return null; + } + }; + + coordinatorExecutor.submit(guardedCallable).get(); + } catch (InterruptedException | ExecutionException e) { + throw new FlinkRuntimeException(errorMessage, e); + } + } else { + try { + callable.call(); + } catch (Throwable t) { + LOG.error( + "Uncaught Exception in data statistics coordinator: {} executor", operatorName, t); + throw new FlinkRuntimeException(errorMessage, t); + } + } + } + + public void runInCoordinatorThread(Runnable runnable) { + this.coordinatorExecutor.execute( + new ThrowableCatchingRunnable( + throwable -> + this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable), + runnable)); + } + + private void runInCoordinatorThread(ThrowingRunnable action, String actionString) { + ensureStarted(); + runInCoordinatorThread( + () -> { + try { + action.run(); + } catch (Throwable t) { + ExceptionUtils.rethrowIfFatalErrorOrOOM(t); + LOG.error( + "Uncaught exception in the data statistics coordinator: {} while {}. Triggering job failover", + operatorName, + actionString, + t); + context.failJob(t); + } + }); + } + + private void ensureStarted() { + Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); + } + + private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { + CompletedStatistics maybeCompletedStatistics = + aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); + + if (maybeCompletedStatistics != null) { + if (maybeCompletedStatistics.isEmpty()) { + LOG.info( + "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); + } else { + LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); + // completedStatistics contains the complete samples, which is needed to compute + // the range bounds in globalStatistics if downstreamParallelism changed. + this.completedStatistics = maybeCompletedStatistics; + // globalStatistics only contains assignment calculated based on Map or Sketch statistics + this.globalStatistics = + globalStatistics( + maybeCompletedStatistics, + downstreamParallelism, + comparator, + closeFileCostWeightPercentage); + sendGlobalStatisticsToSubtasks(globalStatistics); + } + } + } + + private static GlobalStatistics globalStatistics( + CompletedStatistics completedStatistics, + int downstreamParallelism, + Comparator comparator, + double closeFileCostWeightPercentage) { + if (completedStatistics.type() == StatisticsType.Sketch) { + // range bound is a much smaller array compared to the complete samples. + // It helps reduce the amount of data transfer from coordinator to operator subtasks. + return GlobalStatistics.fromRangeBounds( + completedStatistics.checkpointId(), + SketchUtil.rangeBounds( + downstreamParallelism, comparator, completedStatistics.keySamples())); + } else { + return GlobalStatistics.fromMapAssignment( + completedStatistics.checkpointId(), + MapAssignment.fromKeyFrequency( + downstreamParallelism, + completedStatistics.keyFrequency(), + closeFileCostWeightPercentage, + comparator)); + } + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { + runInCoordinatorThread( + () -> { + LOG.info( + "Broadcast latest global statistics from checkpoint {} to all subtasks", + statistics.checkpointId()); + // applyImmediately is set to false so that operator subtasks can + // apply the change at checkpoint boundary + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, globalStatisticsSerializer, false); + for (int i = 0; i < context.currentParallelism(); ++i) { + // Ignore future return value for potential error (e.g. subtask down). + // Upon restart, subtasks send request to coordinator to refresh statistics + // if there is any difference + subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics for checkpoint %d", + operatorName, statistics.checkpointId())); + } + + @SuppressWarnings("FutureReturnValueIgnored") + private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { + if (globalStatistics != null) { + runInCoordinatorThread( + () -> { + if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { + LOG.debug( + "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", + subtask); + } else { + LOG.info( + "Send latest global statistics from checkpoint {} to subtask {}", + globalStatistics.checkpointId(), + subtask); + StatisticsEvent statisticsEvent = + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, globalStatisticsSerializer, true); + subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); + } + }, + String.format( + "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", + operatorName, subtask, globalStatistics.checkpointId())); + } else { + LOG.info( + "Ignore global statistics request from subtask {} as statistics not available", subtask); + } + } + + @Override + public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Handling event from subtask {} (#{}) of {}: {}", + subtask, + attemptNumber, + operatorName, + event); + if (event instanceof StatisticsEvent) { + handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); + } else if (event instanceof RequestGlobalStatisticsEvent) { + handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); + } else { + throw new IllegalArgumentException( + "Invalid operator event type: " + event.getClass().getCanonicalName()); + } + }, + String.format( + "handling operator event %s from subtask %d (#%d)", + event.getClass(), subtask, attemptNumber)); + } + + @Override + public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) { + runInCoordinatorThread( + () -> { + LOG.debug( + "Snapshotting data statistics coordinator {} for checkpoint {}", + operatorName, + checkpointId); + if (completedStatistics == null) { + // null checkpoint result is not allowed, hence supply an empty byte array + resultFuture.complete(new byte[0]); + } else { + resultFuture.complete( + StatisticsUtil.serializeCompletedStatistics( + completedStatistics, completedStatisticsSerializer)); + } + }, + String.format("taking checkpoint %d", checkpointId)); + } + + @Override + public void notifyCheckpointComplete(long checkpointId) {} + + @Override + public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { + Preconditions.checkState( + !started, "The coordinator %s can only be reset if it was not yet started", operatorName); + if (checkpointData == null || checkpointData.length == 0) { + LOG.info( + "Data statistic coordinator {} has nothing to restore from checkpoint {}", + operatorName, + checkpointId); + return; + } + + LOG.info( + "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); + this.completedStatistics = + StatisticsUtil.deserializeCompletedStatistics( + checkpointData, completedStatisticsSerializer); + // recompute global statistics in case downstream parallelism changed + this.globalStatistics = + globalStatistics( + completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); + } + + @Override + public void subtaskReset(int subtask, long checkpointId) { + runInCoordinatorThread( + () -> { + LOG.info( + "Operator {} subtask {} is reset to checkpoint {}", + operatorName, + subtask, + checkpointId); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.reset(subtask); + }, + String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + } + + @Override + public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Throwable reason) { + runInCoordinatorThread( + () -> { + LOG.info( + "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", + subtask, + attemptNumber, + operatorName); + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); + }, + String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + } + + @Override + public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { + Preconditions.checkArgument(subtask == gateway.getSubtask()); + Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); + runInCoordinatorThread( + () -> { + Preconditions.checkState( + this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); + subtaskGateways.registerSubtaskGateway(gateway); + }, + String.format( + "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + } + + @VisibleForTesting + CompletedStatistics completedStatistics() { + return completedStatistics; + } + + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } + + private static class SubtaskGateways { + private final String operatorName; + private final Map[] gateways; + + @SuppressWarnings("unchecked") + private SubtaskGateways(String operatorName, int parallelism) { + this.operatorName = operatorName; + gateways = new Map[parallelism]; + + for (int i = 0; i < parallelism; ++i) { + gateways[i] = Maps.newHashMap(); + } + } + + private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway gateway) { + int subtaskIndex = gateway.getSubtask(); + int attemptNumber = gateway.getExecution().getAttemptNumber(); + Preconditions.checkState( + !gateways[subtaskIndex].containsKey(attemptNumber), + "Coordinator of %s already has a subtask gateway for %d (#%d)", + operatorName, + subtaskIndex, + attemptNumber); + LOG.debug( + "Coordinator of {} registers gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].put(attemptNumber, gateway); + } + + private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { + LOG.debug( + "Coordinator of {} unregisters gateway for subtask {} attempt {}", + operatorName, + subtaskIndex, + attemptNumber); + gateways[subtaskIndex].remove(attemptNumber); + } + + private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { + Preconditions.checkState( + !gateways[subtaskIndex].isEmpty(), + "Coordinator of %s subtask %d is not ready yet to receive events", + operatorName, + subtaskIndex); + return Iterables.getOnlyElement(gateways[subtaskIndex].values()); + } + + private void reset(int subtaskIndex) { + gateways[subtaskIndex].clear(); + } + } + + private static class CoordinatorExecutorThreadFactory + implements ThreadFactory, Thread.UncaughtExceptionHandler { + + private final String coordinatorThreadName; + private final ClassLoader classLoader; + private final Thread.UncaughtExceptionHandler errorHandler; + + @javax.annotation.Nullable private Thread thread; + + CoordinatorExecutorThreadFactory( + final String coordinatorThreadName, final ClassLoader contextClassLoader) { + this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE); + } + + @org.apache.flink.annotation.VisibleForTesting + CoordinatorExecutorThreadFactory( + final String coordinatorThreadName, + final ClassLoader contextClassLoader, + final Thread.UncaughtExceptionHandler errorHandler) { + this.coordinatorThreadName = coordinatorThreadName; + this.classLoader = contextClassLoader; + this.errorHandler = errorHandler; + } + + @Override + public synchronized Thread newThread(@NotNull Runnable runnable) { + thread = new Thread(runnable, coordinatorThreadName); + thread.setContextClassLoader(classLoader); + thread.setUncaughtExceptionHandler(this); + return thread; + } + + @Override + public synchronized void uncaughtException(Thread t, Throwable e) { + errorHandler.uncaughtException(t, e); + } + + boolean isCurrentThreadCoordinatorThread() { + return Thread.currentThread() == thread; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..9d7d989c298e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java @@ -0,0 +1,70 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +/** + * DataStatisticsCoordinatorProvider provides the method to create new {@link + * DataStatisticsCoordinator} + */ +@Internal +public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { + + private final String operatorName; + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsCoordinatorProvider( + String operatorName, + OperatorID operatorID, + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + super(operatorID); + this.operatorName = operatorName; + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { + return new DataStatisticsCoordinator( + operatorName, + context, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java new file mode 100644 index 000000000000..7995a8a5b181 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java @@ -0,0 +1,266 @@ +/* + * 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.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be + * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to + * shuffle record to improve data clustering while maintaining relative balanced traffic + * distribution to downstream subtasks. + */ +@Internal +public class DataStatisticsOperator extends AbstractStreamOperator + implements OneInputStreamOperator, OperatorEventHandler { + + private static final long serialVersionUID = 1L; + + private final String operatorName; + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final OperatorEventGateway operatorEventGateway; + private final int downstreamParallelism; + private final StatisticsType statisticsType; + private final TypeSerializer taskStatisticsSerializer; + private final TypeSerializer globalStatisticsSerializer; + + private transient int parallelism; + private transient int subtaskIndex; + private transient ListState globalStatisticsState; + // current statistics type may be different from the config due to possible + // migration from Map statistics to Sketch statistics when high cardinality detected + private transient volatile StatisticsType taskStatisticsType; + private transient volatile DataStatistics localStatistics; + private transient volatile GlobalStatistics globalStatistics; + + DataStatisticsOperator( + String operatorName, + Schema schema, + SortOrder sortOrder, + OperatorEventGateway operatorEventGateway, + int downstreamParallelism, + StatisticsType statisticsType) { + this.operatorName = operatorName; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.operatorEventGateway = operatorEventGateway; + this.downstreamParallelism = downstreamParallelism; + this.statisticsType = statisticsType; + + SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); + this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); + this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + this.parallelism = getRuntimeContext().getTaskInfo().getNumberOfParallelSubtasks(); + this.subtaskIndex = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + + // Use union state so that new subtasks can also restore global statistics during scale-up. + this.globalStatisticsState = + context + .getOperatorStateStore() + .getUnionListState( + new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); + + if (context.isRestored()) { + if (globalStatisticsState.get() == null + || !globalStatisticsState.get().iterator().hasNext()) { + LOG.info( + "Operator {} subtask {} doesn't have global statistics state to restore", + operatorName, + subtaskIndex); + // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be + // leveraged to request global statistics from coordinator if new subtasks (scale-up case) + // has nothing to restore from. + } else { + GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); + LOG.info( + "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); + this.globalStatistics = restoredStatistics; + } + + // Always request for new statistics from coordinator upon task initialization. + // There are a few scenarios this is needed + // 1. downstream writer parallelism changed due to rescale. + // 2. coordinator failed to send the aggregated statistics to subtask + // (e.g. due to subtask failure at the time). + // Records may flow before coordinator can respond. Range partitioner should be + // able to continue to operate with potentially suboptimal behavior (in sketch case). + LOG.info( + "Operator {} subtask {} requests new global statistics from coordinator ", + operatorName, + subtaskIndex); + // coordinator can use the hashCode (if available) in the request event to determine + // if operator already has the latest global statistics and respond can be skipped. + // This makes the handling cheap in most situations. + RequestGlobalStatisticsEvent event = + globalStatistics != null + ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) + : new RequestGlobalStatisticsEvent(); + operatorEventGateway.sendEventToCoordinator(event); + } + + this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); + this.localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @Override + public void open() throws Exception { + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } + } + + @Override + public void handleOperatorEvent(OperatorEvent event) { + Preconditions.checkArgument( + event instanceof StatisticsEvent, + String.format( + "Operator %s subtask %s received unexpected operator event %s", + operatorName, subtaskIndex, event.getClass())); + StatisticsEvent statisticsEvent = (StatisticsEvent) event; + LOG.info( + "Operator {} subtask {} received global data event from coordinator checkpoint {}", + operatorName, + subtaskIndex, + statisticsEvent.checkpointId()); + this.globalStatistics = + StatisticsUtil.deserializeGlobalStatistics( + statisticsEvent.statisticsBytes(), globalStatisticsSerializer); + checkStatisticsTypeMigration(); + // if applyImmediately not set, wait until the checkpoint time to switch + if (statisticsEvent.applyImmediately()) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } + } + + @Override + public void processElement(StreamRecord streamRecord) { + // collect data statistics + RowData record = streamRecord.getValue(); + StructLike struct = rowDataWrapper.wrap(record); + sortKey.wrap(struct); + localStatistics.add(sortKey); + + checkStatisticsTypeMigration(); + output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + long checkpointId = context.getCheckpointId(); + LOG.info( + "Operator {} subtask {} snapshotting data statistics for checkpoint {}", + operatorName, + subtaskIndex, + checkpointId); + + // Pass global statistics to partitioner so that all the operators refresh statistics + // at same checkpoint barrier + if (globalStatistics != null) { + output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); + } + + // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores + // an exact copy of globalStatistics + if (globalStatistics != null + && getRuntimeContext().getTaskInfo().getIndexOfThisSubtask() == 0) { + globalStatisticsState.clear(); + LOG.info( + "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); + globalStatisticsState.add(globalStatistics); + LOG.debug( + "Operator {} subtask {} saved global statistics to state: {}", + operatorName, + subtaskIndex, + globalStatistics); + } + + // For now, local statistics are sent to coordinator at checkpoint + LOG.info( + "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", + operatorName, + subtaskIndex, + checkpointId); + operatorEventGateway.sendEventToCoordinator( + StatisticsEvent.createTaskStatisticsEvent( + checkpointId, localStatistics, taskStatisticsSerializer)); + + // Recreate the local statistics + localStatistics = + StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); + } + + @SuppressWarnings("unchecked") + private void checkStatisticsTypeMigration() { + // only check if the statisticsType config is Auto and localStatistics is currently Map type + if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { + Map mapStatistics = (Map) localStatistics.result(); + // convert if local statistics has cardinality over the threshold or + // if received global statistics is already sketch type + if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { + LOG.info( + "Operator {} subtask {} switched local statistics from Map to Sketch.", + operatorName, + subtaskIndex); + this.taskStatisticsType = StatisticsType.Sketch; + this.localStatistics = + StatisticsUtil.createTaskStatistics( + taskStatisticsType, parallelism, downstreamParallelism); + SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); + } + } + } + + @VisibleForTesting + DataStatistics localStatistics() { + return localStatistics; + } + + @VisibleForTesting + GlobalStatistics globalStatistics() { + return globalStatistics; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java new file mode 100644 index 000000000000..8ce99073836d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java @@ -0,0 +1,204 @@ +/* + * 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.io.IOException; +import java.util.Map; +import java.util.Objects; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.api.common.typeutils.base.MapSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +@Internal +class DataStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final MapSerializer mapSerializer; + private final SortKeySketchSerializer sketchSerializer; + + DataStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); + this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer duplicate() { + TypeSerializer duplicateSortKeySerializer = sortKeySerializer.duplicate(); + return (duplicateSortKeySerializer == sortKeySerializer) + ? this + : new DataStatisticsSerializer(duplicateSortKeySerializer); + } + + @Override + public DataStatistics createInstance() { + return new MapDataStatistics(); + } + + @SuppressWarnings("unchecked") + @Override + public DataStatistics copy(DataStatistics obj) { + StatisticsType statisticsType = obj.type(); + if (statisticsType == StatisticsType.Map) { + MapDataStatistics from = (MapDataStatistics) obj; + Map fromStats = (Map) from.result(); + Map toStats = Maps.newHashMap(fromStats); + return new MapDataStatistics(toStats); + } else if (statisticsType == StatisticsType.Sketch) { + // because ReservoirItemsSketch doesn't expose enough public methods for cloning, + // this implementation adopted the less efficient serialization and deserialization. + SketchDataStatistics from = (SketchDataStatistics) obj; + ReservoirItemsSketch fromStats = (ReservoirItemsSketch) from.result(); + byte[] bytes = fromStats.toByteArray(sketchSerializer); + Memory memory = Memory.wrap(bytes); + ReservoirItemsSketch toStats = + ReservoirItemsSketch.heapify(memory, sketchSerializer); + return new SketchDataStatistics(toStats); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics copy(DataStatistics from, DataStatistics reuse) { + // not much benefit to reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @SuppressWarnings("unchecked") + @Override + public void serialize(DataStatistics obj, DataOutputView target) throws IOException { + StatisticsType statisticsType = obj.type(); + statisticsTypeSerializer.serialize(obj.type(), target); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = (Map) obj.result(); + mapSerializer.serialize(mapStatistics, target); + } else if (statisticsType == StatisticsType.Sketch) { + ReservoirItemsSketch sketch = (ReservoirItemsSketch) obj.result(); + byte[] sketchBytes = sketch.toByteArray(sketchSerializer); + target.writeInt(sketchBytes.length); + target.write(sketchBytes); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataInputView source) throws IOException { + StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); + if (statisticsType == StatisticsType.Map) { + Map mapStatistics = mapSerializer.deserialize(source); + return new MapDataStatistics(mapStatistics); + } else if (statisticsType == StatisticsType.Sketch) { + int numBytes = source.readInt(); + byte[] sketchBytes = new byte[numBytes]; + source.read(sketchBytes); + Memory sketchMemory = Memory.wrap(sketchBytes); + ReservoirItemsSketch sketch = + ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); + return new SketchDataStatistics(sketch); + } else { + throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); + } + } + + @Override + public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof DataStatisticsSerializer)) { + return false; + } + + DataStatisticsSerializer other = (DataStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new DataStatisticsSerializerSnapshot(this); + } + + public static class DataStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public DataStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new DataStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java new file mode 100644 index 000000000000..50ec23e9f7a2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java @@ -0,0 +1,114 @@ +/* + * 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.Arrays; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to + * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the + * complete raw samples. + */ +class GlobalStatistics { + private final long checkpointId; + private final StatisticsType type; + private final MapAssignment mapAssignment; + private final SortKey[] rangeBounds; + + private transient Integer hashCode; + + GlobalStatistics( + long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { + Preconditions.checkArgument( + (mapAssignment != null && rangeBounds == null) + || (mapAssignment == null && rangeBounds != null), + "Invalid key assignment or range bounds: both are non-null or null"); + this.checkpointId = checkpointId; + this.type = type; + this.mapAssignment = mapAssignment; + this.rangeBounds = rangeBounds; + } + + static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { + return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); + } + + static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { + return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("checkpointId", checkpointId) + .add("type", type) + .add("mapAssignment", mapAssignment) + .add("rangeBounds", rangeBounds) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof GlobalStatistics)) { + return false; + } + + GlobalStatistics other = (GlobalStatistics) o; + return Objects.equal(checkpointId, other.checkpointId) + && Objects.equal(type, other.type) + && Objects.equal(mapAssignment, other.mapAssignment()) + && Arrays.equals(rangeBounds, other.rangeBounds()); + } + + @Override + public int hashCode() { + // implemented caching because coordinator can call the hashCode many times. + // when subtasks request statistics refresh upon initialization for reconciliation purpose, + // hashCode is used to check if there is any difference btw coordinator and operator state. + if (hashCode == null) { + this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); + } + + return hashCode; + } + + long checkpointId() { + return checkpointId; + } + + StatisticsType type() { + return type; + } + + MapAssignment mapAssignment() { + return mapAssignment; + } + + SortKey[] rangeBounds() { + return rangeBounds; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java new file mode 100644 index 000000000000..a7fe2b30b865 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java @@ -0,0 +1,199 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.base.EnumSerializer; +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class GlobalStatisticsSerializer extends TypeSerializer { + private final TypeSerializer sortKeySerializer; + private final EnumSerializer statisticsTypeSerializer; + private final ListSerializer rangeBoundsSerializer; + private final ListSerializer intsSerializer; + private final ListSerializer longsSerializer; + + GlobalStatisticsSerializer(TypeSerializer sortKeySerializer) { + this.sortKeySerializer = sortKeySerializer; + this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); + this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); + this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); + this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new GlobalStatisticsSerializer(sortKeySerializer); + } + + @Override + public GlobalStatistics createInstance() { + return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from) { + return new GlobalStatistics( + from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); + } + + @Override + public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { + target.writeLong(record.checkpointId()); + statisticsTypeSerializer.serialize(record.type(), target); + if (record.type() == StatisticsType.Map) { + MapAssignment mapAssignment = record.mapAssignment(); + target.writeInt(mapAssignment.numPartitions()); + target.writeInt(mapAssignment.keyAssignments().size()); + for (Map.Entry entry : mapAssignment.keyAssignments().entrySet()) { + sortKeySerializer.serialize(entry.getKey(), target); + KeyAssignment keyAssignment = entry.getValue(); + intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); + longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); + target.writeLong(keyAssignment.closeFileCostWeight()); + } + } else { + rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); + } + } + + @Override + public GlobalStatistics deserialize(DataInputView source) throws IOException { + long checkpointId = source.readLong(); + StatisticsType type = statisticsTypeSerializer.deserialize(source); + if (type == StatisticsType.Map) { + int numPartitions = source.readInt(); + int mapSize = source.readInt(); + Map keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); + for (int i = 0; i < mapSize; ++i) { + SortKey sortKey = sortKeySerializer.deserialize(source); + List assignedSubtasks = intsSerializer.deserialize(source); + List subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); + long closeFileCostWeight = source.readLong(); + keyAssignments.put( + sortKey, + new KeyAssignment( + assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); + } + + return GlobalStatistics.fromMapAssignment( + checkpointId, new MapAssignment(numPartitions, keyAssignments)); + } else { + List sortKeys = rangeBoundsSerializer.deserialize(source); + SortKey[] rangeBounds = new SortKey[sortKeys.size()]; + return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); + } + } + + @Override + public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) + throws IOException { + // not much benefit to reuse + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + + if (obj == null || getClass() != obj.getClass()) { + return false; + } + + GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; + return Objects.equals(sortKeySerializer, other.sortKeySerializer); + } + + @Override + public int hashCode() { + return sortKeySerializer.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new GlobalStatisticsSerializerSnapshot(this); + } + + public static class GlobalStatisticsSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public GlobalStatisticsSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { + super(serializer); + } + + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { + return new TypeSerializer[] {outerSerializer.sortKeySerializer}; + } + + @Override + protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; + return new GlobalStatisticsSerializer(sortKeySerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java new file mode 100644 index 000000000000..781bcc646023 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java @@ -0,0 +1,155 @@ +/* + * 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.Arrays; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Subtask assignment for a key for Map statistics based */ +class KeyAssignment { + private final List assignedSubtasks; + private final List subtaskWeightsWithCloseFileCost; + private final long closeFileCostWeight; + private final long[] subtaskWeightsExcludingCloseCost; + private final long keyWeight; + private final long[] cumulativeWeights; + + /** + * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could + * also be multiple subtasks if the key has heavy weight that should be handled by multiple + * subtasks. + * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight + * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, + * 10, 7] for target weight of 10 per subtask. + */ + KeyAssignment( + List assignedSubtasks, + List subtaskWeightsWithCloseFileCost, + long closeFileCostWeight) { + Preconditions.checkArgument( + assignedSubtasks != null && !assignedSubtasks.isEmpty(), + "Invalid assigned subtasks: null or empty"); + Preconditions.checkArgument( + subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), + "Invalid assigned subtasks weights: null or empty"); + Preconditions.checkArgument( + assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), + "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", + assignedSubtasks.size(), + subtaskWeightsWithCloseFileCost.size()); + subtaskWeightsWithCloseFileCost.forEach( + weight -> + Preconditions.checkArgument( + weight > closeFileCostWeight, + "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", + weight, + closeFileCostWeight)); + + this.assignedSubtasks = assignedSubtasks; + this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; + this.closeFileCostWeight = closeFileCostWeight; + // Exclude the close file cost for key routing + this.subtaskWeightsExcludingCloseCost = + subtaskWeightsWithCloseFileCost.stream() + .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) + .toArray(); + this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); + this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; + long cumulativeWeight = 0; + for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { + cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; + cumulativeWeights[i] = cumulativeWeight; + } + } + + List assignedSubtasks() { + return assignedSubtasks; + } + + List subtaskWeightsWithCloseFileCost() { + return subtaskWeightsWithCloseFileCost; + } + + long closeFileCostWeight() { + return closeFileCostWeight; + } + + long[] subtaskWeightsExcludingCloseCost() { + return subtaskWeightsExcludingCloseCost; + } + + /** + * Select a subtask for the key. + * + * @return subtask id + */ + int select() { + if (assignedSubtasks.size() == 1) { + // only choice. no need to run random number generator. + return assignedSubtasks.get(0); + } else { + long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); + int index = Arrays.binarySearch(cumulativeWeights, randomNumber); + // choose the subtask where randomNumber < cumulativeWeights[pos]. + // this works regardless whether index is negative or not. + int position = Math.abs(index + 1); + Preconditions.checkState( + position < assignedSubtasks.size(), + "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", + keyWeight, + randomNumber, + cumulativeWeights); + return assignedSubtasks.get(position); + } + } + + @Override + public int hashCode() { + return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + KeyAssignment that = (KeyAssignment) o; + return Objects.equals(assignedSubtasks, that.assignedSubtasks) + && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) + && closeFileCostWeight == that.closeFileCostWeight; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("assignedSubtasks", assignedSubtasks) + .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) + .add("closeFileCostWeight", closeFileCostWeight) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java new file mode 100644 index 000000000000..9d8167460a1b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java @@ -0,0 +1,242 @@ +/* + * 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.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +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.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.Pair; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Key assignment to subtasks for Map statistics. */ +class MapAssignment { + private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); + + private final int numPartitions; + private final Map keyAssignments; + + MapAssignment(int numPartitions, Map keyAssignments) { + Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); + this.numPartitions = numPartitions; + this.keyAssignments = keyAssignments; + } + + static MapAssignment fromKeyFrequency( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + return new MapAssignment( + numPartitions, + assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); + } + + @Override + public int hashCode() { + return Objects.hashCode(numPartitions, keyAssignments); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + MapAssignment that = (MapAssignment) o; + return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("numPartitions", numPartitions) + .add("keyAssignments", keyAssignments) + .toString(); + } + + int numPartitions() { + return numPartitions; + } + + Map keyAssignments() { + return keyAssignments; + } + + /** + * Returns assignment summary for every subtask. + * + * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned + * to the subtask, number of keys assigned to the subtask) + */ + Map> assignmentInfo() { + Map> assignmentInfo = Maps.newTreeMap(); + keyAssignments.forEach( + (key, keyAssignment) -> { + for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { + int subtaskId = keyAssignment.assignedSubtasks().get(i); + long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; + Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); + assignmentInfo.put( + subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); + } + }); + + return assignmentInfo; + } + + static Map assignment( + int numPartitions, + Map mapStatistics, + double closeFileCostWeightPercentage, + Comparator comparator) { + mapStatistics.forEach( + (key, value) -> + Preconditions.checkArgument( + value > 0, "Invalid statistics: weight is 0 for key %s", key)); + + long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); + double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; + long closeFileCostWeight = + (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); + + NavigableMap sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); + mapStatistics.forEach( + (k, v) -> { + int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); + long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; + sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); + }); + + long totalWeightWithCloseFileCost = + sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); + long targetWeightPerSubtaskWithCloseFileCost = + (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); + return buildAssignment( + numPartitions, + sortedStatsWithCloseFileCost, + targetWeightPerSubtaskWithCloseFileCost, + closeFileCostWeight); + } + + private static Map buildAssignment( + int numPartitions, + NavigableMap sortedStatistics, + long targetWeightPerSubtask, + long closeFileCostWeight) { + Map assignmentMap = + Maps.newHashMapWithExpectedSize(sortedStatistics.size()); + Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); + int subtaskId = 0; + SortKey currentKey = null; + long keyRemainingWeight = 0L; + long subtaskRemainingWeight = targetWeightPerSubtask; + List assignedSubtasks = Lists.newArrayList(); + List subtaskWeights = Lists.newArrayList(); + while (mapKeyIterator.hasNext() || currentKey != null) { + // This should never happen because target weight is calculated using ceil function. + if (subtaskId >= numPartitions) { + LOG.error( + "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " + + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", + numPartitions, + targetWeightPerSubtask, + closeFileCostWeight, + sortedStatistics); + throw new IllegalStateException( + "Internal algorithm error: exhausted subtasks with unassigned keys left"); + } + + if (currentKey == null) { + currentKey = mapKeyIterator.next(); + keyRemainingWeight = sortedStatistics.get(currentKey); + } + + assignedSubtasks.add(subtaskId); + if (keyRemainingWeight < subtaskRemainingWeight) { + // assign the remaining weight of the key to the current subtask + subtaskWeights.add(keyRemainingWeight); + subtaskRemainingWeight -= keyRemainingWeight; + keyRemainingWeight = 0L; + } else { + // filled up the current subtask + long assignedWeight = subtaskRemainingWeight; + keyRemainingWeight -= subtaskRemainingWeight; + + // If assigned weight is less than close file cost, pad it up with close file cost. + // This might cause the subtask assigned weight over the target weight. + // But it should be no more than one close file cost. Small skew is acceptable. + if (assignedWeight <= closeFileCostWeight) { + long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); + keyRemainingWeight -= paddingWeight; + assignedWeight += paddingWeight; + } + + subtaskWeights.add(assignedWeight); + // move on to the next subtask + subtaskId += 1; + subtaskRemainingWeight = targetWeightPerSubtask; + } + + Preconditions.checkState( + assignedSubtasks.size() == subtaskWeights.size(), + "List size mismatch: assigned subtasks = %s, subtask weights = %s", + assignedSubtasks, + subtaskWeights); + + // If the remaining key weight is smaller than the close file cost, simply skip the residual + // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. + // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is + // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous + // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and + // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the + // close file cost), which is inaccurate as the true key weight should be 2. + // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is + // expected and acceptable. Traffic distribution should still be balanced. + if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { + keyRemainingWeight = 0; + } + + if (keyRemainingWeight == 0) { + // finishing up the assignment for the current key + KeyAssignment keyAssignment = + new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); + assignmentMap.put(currentKey, keyAssignment); + assignedSubtasks = Lists.newArrayList(); + subtaskWeights = Lists.newArrayList(); + currentKey = null; + } + } + + return assignmentMap; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java new file mode 100644 index 000000000000..05b943f6046f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.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.flink.sink.shuffle; + +import java.util.Map; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +/** MapDataStatistics uses map to count key frequency */ +class MapDataStatistics implements DataStatistics { + private final Map keyFrequency; + + MapDataStatistics() { + this.keyFrequency = Maps.newHashMap(); + } + + MapDataStatistics(Map keyFrequency) { + this.keyFrequency = keyFrequency; + } + + @Override + public StatisticsType type() { + return StatisticsType.Map; + } + + @Override + public boolean isEmpty() { + return keyFrequency.isEmpty(); + } + + @Override + public void add(SortKey sortKey) { + if (keyFrequency.containsKey(sortKey)) { + keyFrequency.merge(sortKey, 1L, Long::sum); + } else { + // clone the sort key before adding to map because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + keyFrequency.put(copiedKey, 1L); + } + } + + @Override + public Object result() { + return keyFrequency; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof MapDataStatistics)) { + return false; + } + + MapDataStatistics other = (MapDataStatistics) o; + return Objects.equal(keyFrequency, other.keyFrequency); + } + + @Override + public int hashCode() { + return Objects.hashCode(keyFrequency); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java new file mode 100644 index 000000000000..f36a078c94e0 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java @@ -0,0 +1,95 @@ +/* + * 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.concurrent.TimeUnit; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Internal partitioner implementation that supports MapDataStatistics, which is typically used for + * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used + * for high-cardinality use cases. Otherwise, the memory footprint is too high. + * + *

    It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always + * precise when calculating close cost for every file, target weight per subtask, padding residual + * weight, assigned weight without close cost. + * + *

    All actions should be executed in a single Flink mailbox thread. So there is no need to make + * it thread safe. + */ +class MapRangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); + + private final RowDataWrapper rowDataWrapper; + private final SortKey sortKey; + private final MapAssignment mapAssignment; + + // Counter that tracks how many times a new key encountered + // where there is no traffic statistics learned about it. + private long newSortKeyCounter; + private long lastNewSortKeyLogTimeMilli; + + MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + this.sortKey = new SortKey(schema, sortOrder); + this.mapAssignment = mapAssignment; + this.newSortKeyCounter = 0; + this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); + + int partition; + if (keyAssignment == null) { + LOG.trace( + "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", + sortKey); + // Ideally unknownKeyCounter should be published as a counter metric. + // It seems difficult to pass in MetricGroup into the partitioner. + // Just log an INFO message every minute. + newSortKeyCounter += 1; + long now = System.currentTimeMillis(); + if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { + LOG.info( + "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", + newSortKeyCounter); + lastNewSortKeyLogTimeMilli = now; + newSortKeyCounter = 0; + } + partition = (int) (newSortKeyCounter % numPartitions); + } else { + partition = keyAssignment.select(); + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, mapAssignment.numPartitions(), numPartitions); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java new file mode 100644 index 000000000000..6608b938f5a8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.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.flink.sink.shuffle; + +import java.util.Random; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ +@Internal +public class RangePartitioner implements Partitioner { + private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); + + private final Schema schema; + private final SortOrder sortOrder; + + private transient AtomicLong roundRobinCounter; + private transient Partitioner delegatePartitioner; + + public RangePartitioner(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int partition(StatisticsOrRecord wrapper, int numPartitions) { + if (wrapper.hasStatistics()) { + this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); + return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + } else { + if (delegatePartitioner != null) { + return delegatePartitioner.partition(wrapper.record(), numPartitions); + } else { + int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); + LOG.trace("Statistics not available. Round robin to partition {}", partition); + return partition; + } + } + } + + private AtomicLong roundRobinCounter(int numPartitions) { + if (roundRobinCounter == null) { + // randomize the starting point to avoid synchronization across subtasks + this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); + } + + return roundRobinCounter; + } + + private Partitioner delegatePartitioner(GlobalStatistics statistics) { + if (statistics.type() == StatisticsType.Map) { + return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); + } else if (statistics.type() == StatisticsType.Sketch) { + return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); + } else { + throw new IllegalArgumentException( + String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); + } + } + + /** + * Util method that handles rescale (write parallelism / numPartitions change). + * + * @param partition partition caculated based on the existing statistics + * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated + * based on + * @param numPartitions current number of partitions + * @return adjusted partition if necessary. + */ + static int adjustPartitionWithRescale( + int partition, int numPartitionsStatsCalculation, int numPartitions) { + if (numPartitionsStatsCalculation <= numPartitions) { + // no rescale or scale-up case. + // new subtasks are ignored and not assigned any keys, which is sub-optimal and only + // transient. when rescale is detected, operator requests new statistics from + // coordinator upon initialization. + return partition; + } else { + // scale-down case. + // Use mod % operation to distribution the over-range partitions. + // It can cause skew among subtasks. but the behavior is still better than + // discarding the statistics and falling back to round-robin (no clustering). + // Again, this is transient and stats refresh is requested when rescale is detected. + return partition % numPartitions; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java new file mode 100644 index 000000000000..ce17e1964392 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.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.sink.shuffle; + +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +class RequestGlobalStatisticsEvent implements OperatorEvent { + private final Integer signature; + + RequestGlobalStatisticsEvent() { + this.signature = null; + } + + /** @param signature hashCode of the subtask's existing global statistics */ + RequestGlobalStatisticsEvent(int signature) { + this.signature = signature; + } + + Integer signature() { + return signature; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java new file mode 100644 index 000000000000..35bbb27baf16 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java @@ -0,0 +1,87 @@ +/* + * 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.Arrays; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +/** MapDataStatistics uses map to count key frequency */ +class SketchDataStatistics implements DataStatistics { + + private final ReservoirItemsSketch sketch; + + SketchDataStatistics(int reservoirSize) { + this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); + } + + SketchDataStatistics(ReservoirItemsSketch sketchStats) { + this.sketch = sketchStats; + } + + @Override + public StatisticsType type() { + return StatisticsType.Sketch; + } + + @Override + public boolean isEmpty() { + return sketch.getNumSamples() == 0; + } + + @Override + public void add(SortKey sortKey) { + // clone the sort key first because input sortKey object can be reused + SortKey copiedKey = sortKey.copy(); + sketch.update(copiedKey); + } + + @Override + public Object result() { + return sketch; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof SketchDataStatistics)) { + return false; + } + + ReservoirItemsSketch otherSketch = ((SketchDataStatistics) o).sketch; + return Objects.equal(sketch.getK(), otherSketch.getK()) + && Objects.equal(sketch.getN(), otherSketch.getN()) + && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); + } + + @Override + public int hashCode() { + return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java new file mode 100644 index 000000000000..dddb0d8722c0 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java @@ -0,0 +1,51 @@ +/* + * 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.Comparator; +import org.apache.flink.api.common.functions.Partitioner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Comparators; + +class SketchRangePartitioner implements Partitioner { + private final SortKey sortKey; + private final Comparator comparator; + private final SortKey[] rangeBounds; + private final RowDataWrapper rowDataWrapper; + + SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { + this.sortKey = new SortKey(schema, sortOrder); + this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); + this.rangeBounds = rangeBounds; + this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + } + + @Override + public int partition(RowData row, int numPartitions) { + // reuse the sortKey and rowDataWrapper + sortKey.wrap(rowDataWrapper.wrap(row)); + return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java new file mode 100644 index 000000000000..871ef9ef1149 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java @@ -0,0 +1,159 @@ +/* + * 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.Arrays; +import java.util.Comparator; +import java.util.Map; +import java.util.function.Consumer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.StructLike; + +class SketchUtil { + static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; + static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; + static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; + static final int OPERATOR_OVER_SAMPLE_RATIO = 10; + + // switch the statistics tracking from map to sketch if the cardinality of the sort key is over + // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. + static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; + static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; + + private SketchUtil() {} + + /** + * The larger the reservoir size, the more accurate for range bounds calculation and the more + * balanced range distribution. + * + *

    Here are the heuristic rules + *

  • Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the + * range bounds + *
  • Min is 10K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 1M to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @return reservoir size + */ + static int determineCoordinatorReservoirSize(int numPartitions) { + int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; + + if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { + // adjust it up and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); + } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { + // adjust it down and still make reservoirSize divisible by numPartitions + int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; + reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; + } + + return reservoirSize; + } + + /** + * Determine the sampling reservoir size where operator subtasks collect data statistics. + * + *

    Here are the heuristic rules + *

  • Target size is "coordinator reservoir size * over sampling ration (10) / operator + * parallelism" + *
  • Min is 1K to achieve good accuracy while memory footprint is still relatively small + *
  • Max is 100K to cap the memory footprint on coordinator + * + * @param numPartitions number of range partitions which equals to downstream operator parallelism + * @param operatorParallelism data statistics operator parallelism + * @return reservoir size + */ + static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { + int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); + int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; + return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); + } + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + * + *

    Assumption is that a single key is not dominant enough to span multiple subtasks. + * + * @param numPartitions number of partitions which maps to downstream operator parallelism + * @param samples sampled keys + * @return array of range partition bounds. It should be a sorted list (ascending). Number of + * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list + */ + static SortKey[] rangeBounds( + int numPartitions, Comparator comparator, SortKey[] samples) { + // sort the keys first + Arrays.sort(samples, comparator); + int numCandidates = numPartitions - 1; + SortKey[] candidates = new SortKey[numCandidates]; + int step = (int) Math.ceil((double) samples.length / numPartitions); + int position = step - 1; + int numChosen = 0; + while (position < samples.length && numChosen < numCandidates) { + SortKey candidate = samples[position]; + // skip duplicate values + if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { + // linear probe for the next distinct value + position += 1; + } else { + candidates[numChosen] = candidate; + position += step; + numChosen += 1; + } + } + + return candidates; + } + + /** This can be a bit expensive since it is quadratic. */ + static void convertMapToSketch( + Map taskMapStats, Consumer sketchConsumer) { + taskMapStats.forEach( + (sortKey, count) -> { + for (int i = 0; i < count; ++i) { + sketchConsumer.accept(sortKey); + } + }); + } + + static int partition( + SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { + int partition = Arrays.binarySearch(rangeBounds, key, comparator); + + // binarySearch either returns the match location or -[insertion point]-1 + if (partition < 0) { + partition = -partition - 1; + } + + if (partition > rangeBounds.length) { + partition = rangeBounds.length; + } + + return RangePartitioner.adjustPartitionWithRescale( + partition, rangeBounds.length + 1, numPartitions); + } +} 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 new file mode 100644 index 000000000000..b82fc8250763 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java @@ -0,0 +1,359 @@ +/* + * 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.io.IOException; +import java.math.BigDecimal; +import java.math.BigInteger; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.Objects; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.StringUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderParser; +import org.apache.iceberg.types.CheckCompatibility; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.types.Types; + +class SortKeySerializer extends TypeSerializer { + private final Schema schema; + private final SortOrder sortOrder; + private final int size; + private final Types.NestedField[] transformedFields; + + private transient SortKey sortKey; + + SortKeySerializer(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + this.size = sortOrder.fields().size(); + + this.transformedFields = new Types.NestedField[size]; + for (int i = 0; i < size; ++i) { + SortField sortField = sortOrder.fields().get(i); + Types.NestedField sourceField = schema.findField(sortField.sourceId()); + Type resultType = sortField.transform().getResultType(sourceField.type()); + Types.NestedField transformedField = + Types.NestedField.of( + sourceField.fieldId(), + sourceField.isOptional(), + sourceField.name(), + resultType, + sourceField.doc()); + transformedFields[i] = transformedField; + } + } + + private SortKey lazySortKey() { + if (sortKey == null) { + this.sortKey = new SortKey(schema, sortOrder); + } + + return sortKey; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer duplicate() { + return new SortKeySerializer(schema, sortOrder); + } + + @Override + public SortKey createInstance() { + return new SortKey(schema, sortOrder); + } + + @Override + public SortKey copy(SortKey from) { + return from.copy(); + } + + @Override + public SortKey copy(SortKey from, SortKey reuse) { + // no benefit of reuse + return copy(from); + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(SortKey record, DataOutputView target) throws IOException { + Preconditions.checkArgument( + record.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + record.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + target.writeBoolean(record.get(i, Boolean.class)); + break; + case INTEGER: + case DATE: + target.writeInt(record.get(i, Integer.class)); + break; + case LONG: + case TIME: + case TIMESTAMP: + target.writeLong(record.get(i, Long.class)); + break; + case FLOAT: + target.writeFloat(record.get(i, Float.class)); + break; + case DOUBLE: + target.writeDouble(record.get(i, Double.class)); + break; + case STRING: + target.writeUTF(record.get(i, CharSequence.class).toString()); + break; + case UUID: + UUID uuid = record.get(i, UUID.class); + target.writeLong(uuid.getMostSignificantBits()); + target.writeLong(uuid.getLeastSignificantBits()); + break; + case FIXED: + case BINARY: + byte[] bytes = record.get(i, ByteBuffer.class).array(); + target.writeInt(bytes.length); + target.write(bytes); + break; + case DECIMAL: + BigDecimal decimal = record.get(i, BigDecimal.class); + byte[] decimalBytes = decimal.unscaledValue().toByteArray(); + target.writeInt(decimalBytes.length); + target.write(decimalBytes); + target.writeInt(decimal.scale()); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + } + + @Override + public SortKey deserialize(DataInputView source) throws IOException { + // copying is a little faster than constructing a new SortKey object + SortKey deserialized = lazySortKey().copy(); + deserialize(deserialized, source); + return deserialized; + } + + @Override + public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { + Preconditions.checkArgument( + reuse.size() == size, + "Invalid size of the sort key object: %s. Expected %s", + reuse.size(), + size); + for (int i = 0; i < size; ++i) { + int fieldId = transformedFields[i].fieldId(); + Type.TypeID typeId = transformedFields[i].type().typeId(); + switch (typeId) { + case BOOLEAN: + reuse.set(i, source.readBoolean()); + break; + case INTEGER: + case DATE: + reuse.set(i, source.readInt()); + break; + case LONG: + case TIME: + case TIMESTAMP: + reuse.set(i, source.readLong()); + break; + case FLOAT: + reuse.set(i, source.readFloat()); + break; + case DOUBLE: + reuse.set(i, source.readDouble()); + break; + case STRING: + reuse.set(i, source.readUTF()); + break; + case UUID: + long mostSignificantBits = source.readLong(); + long leastSignificantBits = source.readLong(); + reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); + break; + case FIXED: + case BINARY: + byte[] bytes = new byte[source.readInt()]; + source.read(bytes); + reuse.set(i, ByteBuffer.wrap(bytes)); + break; + case DECIMAL: + byte[] unscaledBytes = new byte[source.readInt()]; + source.read(unscaledBytes); + int scale = source.readInt(); + BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); + reuse.set(i, decimal); + break; + case STRUCT: + case MAP: + case LIST: + default: + // SortKey transformation is a flattened struct without list and map + throw new UnsupportedOperationException( + String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + } + } + + return reuse; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + // no optimization here + serialize(deserialize(source), target); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof SortKeySerializer)) { + return false; + } + + SortKeySerializer other = (SortKeySerializer) obj; + return Objects.equals(schema.asStruct(), other.schema.asStruct()) + && Objects.equals(sortOrder, other.sortOrder); + } + + @Override + public int hashCode() { + return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new SortKeySerializerSnapshot(schema, sortOrder); + } + + public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + private Schema schema; + private SortOrder sortOrder; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public SortKeySerializerSnapshot() { + // this constructor is used when restoring from a checkpoint. + } + + @SuppressWarnings("checkstyle:RedundantModifier") + public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { + this.schema = schema; + this.sortOrder = sortOrder; + } + + @Override + public int getCurrentVersion() { + return CURRENT_VERSION; + } + + @Override + public void writeSnapshot(DataOutputView out) throws IOException { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + + StringUtils.writeString(SchemaParser.toJson(schema), out); + StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); + } + + @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); + } + } + + @Override + public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( + TypeSerializerSnapshot oldSerializerSnapshot) { + if (!(oldSerializerSnapshot instanceof SortKeySerializerSnapshot)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + // Sort order should be identical + SortKeySerializerSnapshot oldSnapshot = (SortKeySerializerSnapshot) oldSerializerSnapshot; + if (!sortOrder.sameOrder(oldSnapshot.sortOrder)) { + return TypeSerializerSchemaCompatibility.incompatible(); + } + + Set sortFieldIds = + sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); + // only care about the schema related to sort fields + Schema sortSchema = TypeUtil.project(schema, sortFieldIds); + Schema oldSortSchema = TypeUtil.project(oldSnapshot.schema, sortFieldIds); + + List compatibilityErrors = + CheckCompatibility.writeCompatibilityErrors(sortSchema, oldSortSchema); + if (compatibilityErrors.isEmpty()) { + return TypeSerializerSchemaCompatibility.compatibleAsIs(); + } + + return TypeSerializerSchemaCompatibility.incompatible(); + } + + @Override + public TypeSerializer restoreSerializer() { + Preconditions.checkState(schema != null, "Invalid schema: null"); + Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); + return new SortKeySerializer(schema, sortOrder); + } + + private void readV1(DataInputView in) throws IOException { + String schemaJson = StringUtils.readString(in); + String sortOrderJson = StringUtils.readString(in); + this.schema = SchemaParser.fromJson(schemaJson); + this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java new file mode 100644 index 000000000000..d6c23f035015 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UncheckedIOException; +import java.util.Arrays; +import java.util.List; +import org.apache.datasketches.common.ArrayOfItemsSerDe; +import org.apache.datasketches.common.ArrayOfStringsSerDe; +import org.apache.datasketches.common.ByteArrayUtil; +import org.apache.datasketches.common.Util; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.base.ListSerializer; +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.base.Preconditions; + +/** + * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link + * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The + * implementation is modeled after {@link ArrayOfStringsSerDe} + */ +class SortKeySketchSerializer extends ArrayOfItemsSerDe implements Serializable { + private static final int DEFAULT_SORT_KEY_SIZE = 128; + + private final TypeSerializer itemSerializer; + private final ListSerializer listSerializer; + private final DataInputDeserializer input; + + SortKeySketchSerializer(TypeSerializer itemSerializer) { + this.itemSerializer = itemSerializer; + this.listSerializer = new ListSerializer<>(itemSerializer); + this.input = new DataInputDeserializer(); + } + + @Override + public byte[] serializeToByteArray(SortKey item) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); + itemSerializer.serialize(item, output); + byte[] itemBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[numBytes + Integer.BYTES]; + ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); + ByteArrayUtil.putIntLE(out, 0, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public byte[] serializeToByteArray(SortKey[] items) { + try { + DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); + listSerializer.serialize(Arrays.asList(items), output); + byte[] itemsBytes = output.getSharedBuffer(); + int numBytes = output.length(); + byte[] out = new byte[Integer.BYTES + numBytes]; + ByteArrayUtil.putIntLE(out, 0, numBytes); + System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); + return out; + } catch (IOException e) { + throw new UncheckedIOException("Failed to serialize sort key", e); + } + } + + @Override + public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return new SortKey[0]; + } + + long offset = startingOffset; + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + offset += Integer.BYTES; + + Util.checkBounds(offset, numBytes, mem.getCapacity()); + byte[] sortKeyBytes = new byte[numBytes]; + mem.getByteArray(offset, sortKeyBytes, 0, numBytes); + input.setBuffer(sortKeyBytes); + + try { + List sortKeys = listSerializer.deserialize(input); + SortKey[] array = new SortKey[numItems]; + sortKeys.toArray(array); + input.releaseArrays(); + return array; + } catch (IOException e) { + throw new UncheckedIOException("Failed to deserialize sort key sketch", e); + } + } + + @Override + public int sizeOf(SortKey item) { + return serializeToByteArray(item).length; + } + + @Override + public int sizeOf(Memory mem, long offset, int numItems) { + Preconditions.checkArgument(mem != null, "Invalid input memory: null"); + if (numItems <= 0) { + return 0; + } + + Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); + int numBytes = mem.getInt(offset); + return Integer.BYTES + numBytes; + } + + @Override + public String toString(SortKey item) { + return item.toString(); + } + + @Override + public Class getClassOfT() { + return SortKey.class; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java new file mode 100644 index 000000000000..1e5bdbbac3e4 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.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.sink.shuffle; + +import java.util.List; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortField; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +class SortKeyUtil { + private SortKeyUtil() {} + + /** Compute the result schema of {@code SortKey} transformation */ + static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { + List sortFields = sortOrder.fields(); + int size = sortFields.size(); + List transformedFields = Lists.newArrayListWithCapacity(size); + for (int i = 0; i < size; ++i) { + int sourceFieldId = sortFields.get(i).sourceId(); + Types.NestedField sourceField = schema.findField(sourceFieldId); + Preconditions.checkArgument( + sourceField != null, "Cannot find source field: %s", sourceFieldId); + Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); + // There could be multiple transformations on the same source column, like in the PartitionKey + // case. To resolve the collision, field id is set to transform index and field name is set to + // sourceFieldName_transformIndex + Types.NestedField transformedField = + Types.NestedField.of( + i, + sourceField.isOptional(), + sourceField.name() + '_' + i, + transformedType, + sourceField.doc()); + transformedFields.add(transformedField); + } + + return new Schema(transformedFields); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java new file mode 100644 index 000000000000..f6fcdb8b16ef --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.runtime.operators.coordination.OperatorEvent; + +/** + * DataStatisticsEvent is sent between data statistics coordinator and operator to transmit data + * statistics in bytes + */ +@Internal +class StatisticsEvent implements OperatorEvent { + + private static final long serialVersionUID = 1L; + private final long checkpointId; + private final byte[] statisticsBytes; + private final boolean applyImmediately; + + private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { + this.checkpointId = checkpointId; + this.statisticsBytes = statisticsBytes; + this.applyImmediately = applyImmediately; + } + + static StatisticsEvent createTaskStatisticsEvent( + long checkpointId, + DataStatistics statistics, + TypeSerializer statisticsSerializer) { + // applyImmediately is really only relevant for coordinator to operator event. + // task reported statistics is always merged immediately by the coordinator. + return new StatisticsEvent( + checkpointId, + StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), + true); + } + + static StatisticsEvent createGlobalStatisticsEvent( + GlobalStatistics statistics, + TypeSerializer statisticsSerializer, + boolean applyImmediately) { + return new StatisticsEvent( + statistics.checkpointId(), + StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), + applyImmediately); + } + + long checkpointId() { + return checkpointId; + } + + byte[] statisticsBytes() { + return statisticsBytes; + } + + boolean applyImmediately() { + return applyImmediately; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java new file mode 100644 index 000000000000..bc28df2b0e22 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.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.flink.sink.shuffle; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * The wrapper class for data statistics and record. It is the only way for data statistics operator + * to send global data statistics to custom partitioner to distribute data based on statistics + * + *

    DataStatisticsOrRecord contains either data statistics(globally aggregated) or a record. It is + * sent from {@link DataStatisticsOperator} to partitioner. Once partitioner receives the data + * statistics, it will use that to decide the coming record should send to which writer subtask. + * After shuffling, a filter and mapper are required to filter out the data distribution weight, + * unwrap the object and extract the original record type T. + */ +@Internal +public class StatisticsOrRecord implements Serializable { + + private static final long serialVersionUID = 1L; + + private GlobalStatistics statistics; + private RowData record; + + private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { + Preconditions.checkArgument( + record != null ^ statistics != null, "DataStatistics or record, not neither or both"); + this.statistics = statistics; + this.record = record; + } + + static StatisticsOrRecord fromRecord(RowData record) { + return new StatisticsOrRecord(null, record); + } + + static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { + return new StatisticsOrRecord(statistics, null); + } + + static StatisticsOrRecord reuseRecord( + StatisticsOrRecord reuse, TypeSerializer recordSerializer) { + if (reuse.hasRecord()) { + return reuse; + } else { + // not reusable + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + } + } + + static StatisticsOrRecord reuseStatistics( + StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { + if (reuse.hasStatistics()) { + return reuse; + } else { + // not reusable + return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); + } + } + + boolean hasStatistics() { + return statistics != null; + } + + public boolean hasRecord() { + return record != null; + } + + GlobalStatistics statistics() { + return statistics; + } + + void statistics(GlobalStatistics newStatistics) { + this.statistics = newStatistics; + } + + public RowData record() { + return record; + } + + void record(RowData newRecord) { + this.record = newRecord; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("statistics", statistics) + .add("record", record) + .toString(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java new file mode 100644 index 000000000000..d4ae2b359679 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import java.io.IOException; +import java.util.Objects; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.table.data.RowData; + +@Internal +class StatisticsOrRecordSerializer extends TypeSerializer { + private final TypeSerializer statisticsSerializer; + private final TypeSerializer recordSerializer; + + StatisticsOrRecordSerializer( + TypeSerializer statisticsSerializer, + TypeSerializer recordSerializer) { + this.statisticsSerializer = statisticsSerializer; + this.recordSerializer = recordSerializer; + } + + @Override + public boolean isImmutableType() { + return false; + } + + @SuppressWarnings("ReferenceEquality") + @Override + public TypeSerializer duplicate() { + TypeSerializer duplicateStatisticsSerializer = + statisticsSerializer.duplicate(); + TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); + if ((statisticsSerializer != duplicateStatisticsSerializer) + || (recordSerializer != duplicateRowDataSerializer)) { + return new StatisticsOrRecordSerializer( + duplicateStatisticsSerializer, duplicateRowDataSerializer); + } else { + return this; + } + } + + @Override + public StatisticsOrRecord createInstance() { + // arbitrarily always create RowData value instance + return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); + } + + @Override + public StatisticsOrRecord copy(StatisticsOrRecord from) { + if (from.hasRecord()) { + return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); + } else { + return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); + } + } + + @Override + public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { + StatisticsOrRecord to; + if (from.hasRecord()) { + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); + RowData record = recordSerializer.copy(from.record(), to.record()); + to.record(record); + } else { + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); + to.statistics(statistics); + } + + return to; + } + + @Override + public int getLength() { + return -1; + } + + @Override + public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) + throws IOException { + if (statisticsOrRecord.hasRecord()) { + target.writeBoolean(true); + recordSerializer.serialize(statisticsOrRecord.record(), target); + } else { + target.writeBoolean(false); + statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); + } + } + + @Override + public StatisticsOrRecord deserialize(DataInputView source) throws IOException { + boolean isRecord = source.readBoolean(); + if (isRecord) { + return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); + } else { + return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); + } + } + + @Override + public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) + throws IOException { + StatisticsOrRecord to; + boolean isRecord = source.readBoolean(); + if (isRecord) { + to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); + RowData record = recordSerializer.deserialize(to.record(), source); + to.record(record); + } else { + to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); + GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); + to.statistics(statistics); + } + + return to; + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + boolean hasRecord = source.readBoolean(); + target.writeBoolean(hasRecord); + if (hasRecord) { + recordSerializer.copy(source, target); + } else { + statisticsSerializer.copy(source, target); + } + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof StatisticsOrRecordSerializer)) { + return false; + } + + StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; + return Objects.equals(statisticsSerializer, other.statisticsSerializer) + && Objects.equals(recordSerializer, other.recordSerializer); + } + + @Override + public int hashCode() { + return Objects.hash(statisticsSerializer, recordSerializer); + } + + @Override + public TypeSerializerSnapshot snapshotConfiguration() { + return new StatisticsOrRecordSerializerSnapshot(this); + } + + public static class StatisticsOrRecordSerializerSnapshot + extends CompositeTypeSerializerSnapshot { + private static final int CURRENT_VERSION = 1; + + /** Constructor for read instantiation. */ + @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) + public StatisticsOrRecordSerializerSnapshot() {} + + @SuppressWarnings("checkstyle:RedundantModifier") + public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { + super(serializer); + } + + @SuppressWarnings("checkstyle:RedundantModifier") + @Override + protected int getCurrentOuterSnapshotVersion() { + return CURRENT_VERSION; + } + + @Override + protected TypeSerializer[] getNestedSerializers( + StatisticsOrRecordSerializer outerSerializer) { + return new TypeSerializer[] { + outerSerializer.statisticsSerializer, outerSerializer.recordSerializer + }; + } + + @SuppressWarnings("unchecked") + @Override + protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( + TypeSerializer[] nestedSerializers) { + TypeSerializer statisticsSerializer = + (TypeSerializer) nestedSerializers[0]; + TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; + return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java new file mode 100644 index 000000000000..43f72e336e06 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +/** + * Range distribution requires gathering statistics on the sort keys to determine proper range + * boundaries to distribute/cluster rows before writer operators. + */ +public enum StatisticsType { + /** + * Tracks the data statistics as {@code Map} frequency. It works better for + * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in + * hundreds or thousands. + * + *

      + *
    • Pro: accurate measurement on the statistics/weight of every key. + *
    • Con: memory footprint can be large if the key cardinality is high. + *
    + */ + Map, + + /** + * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds + * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, + * uuid etc.) where the cardinalities can be in millions or billions. + * + *
      + *
    • Pro: relatively low memory footprint for high-cardinality sort keys. + *
    • Con: non-precise approximation with potentially lower accuracy. + *
    + */ + Sketch, + + /** + * Initially use Map for statistics tracking. If key cardinality turns out to be high, + * automatically switch to sketch sampling. + */ + Auto +} 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 new file mode 100644 index 000000000000..5d48ec57ca49 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.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.flink.sink.shuffle; + +import java.io.IOException; +import java.io.UncheckedIOException; +import javax.annotation.Nullable; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class StatisticsUtil { + + private StatisticsUtil() {} + + static DataStatistics createTaskStatistics( + StatisticsType type, int operatorParallelism, int numPartitions) { + if (type == StatisticsType.Map) { + return new MapDataStatistics(); + } else { + return new SketchDataStatistics( + SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); + } + } + + static byte[] serializeDataStatistics( + DataStatistics dataStatistics, TypeSerializer statisticsSerializer) { + DataOutputSerializer out = new DataOutputSerializer(64); + try { + statisticsSerializer.serialize(dataStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize data statistics", e); + } + } + + static DataStatistics deserializeDataStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); + try { + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize data statistics", e); + } + } + + static byte[] serializeCompletedStatistics( + CompletedStatistics completedStatistics, + TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(completedStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static CompletedStatistics deserializeCompletedStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static byte[] serializeGlobalStatistics( + GlobalStatistics globalStatistics, TypeSerializer statisticsSerializer) { + try { + DataOutputSerializer out = new DataOutputSerializer(1024); + statisticsSerializer.serialize(globalStatistics, out); + return out.getCopyOfBuffer(); + } catch (IOException e) { + throw new UncheckedIOException("Fail to serialize aggregated statistics", e); + } + } + + static GlobalStatistics deserializeGlobalStatistics( + byte[] bytes, TypeSerializer statisticsSerializer) { + try { + DataInputDeserializer input = new DataInputDeserializer(bytes); + return statisticsSerializer.deserialize(input); + } catch (IOException e) { + throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); + } + } + + static StatisticsType collectType(StatisticsType config) { + return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; + } + + static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } + + static StatisticsType collectType( + StatisticsType config, @Nullable CompletedStatistics statistics) { + if (statistics != null) { + return statistics.type(); + } + + return collectType(config); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java new file mode 100644 index 000000000000..796434c45136 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import org.apache.avro.generic.GenericRecord; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; + +public class AvroGenericRecordFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataToAvroGenericRecordConverter converter; + + public AvroGenericRecordFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataToAvroGenericRecordConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java new file mode 100644 index 000000000000..91d975349b19 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -0,0 +1,156 @@ +/* + * 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.io.UncheckedIOException; +import java.util.Iterator; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Flink data iterator that reads {@link CombinedScanTask} into a {@link CloseableIterator} + * + * @param is the output data type returned by this iterator. + */ +@Internal +public class DataIterator implements CloseableIterator { + + private final FileScanTaskReader fileScanTaskReader; + + private final InputFilesDecryptor inputFilesDecryptor; + private final CombinedScanTask combinedTask; + + private Iterator tasks; + private CloseableIterator currentIterator; + private int fileOffset; + private long recordOffset; + + public DataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption) { + this.fileScanTaskReader = fileScanTaskReader; + + this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption); + this.combinedTask = task; + + this.tasks = task.files().iterator(); + this.currentIterator = CloseableIterator.empty(); + + // fileOffset starts at -1 because we started + // from an empty iterator that is not from the split files. + this.fileOffset = -1; + // record offset points to the record that next() should return when called + this.recordOffset = 0L; + } + + /** + * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume + * from. E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the + * 2nd row in file 0. When next() is called after seek, 2nd row from file 0 should be returned. + */ + public void seek(int startingFileOffset, long startingRecordOffset) { + Preconditions.checkState( + fileOffset == -1, "Seek should be called before any other iterator actions"); + // skip files + Preconditions.checkState( + startingFileOffset < combinedTask.files().size(), + "Invalid starting file offset %s for combined scan task with %s files: %s", + startingFileOffset, + combinedTask.files().size(), + combinedTask); + for (long i = 0L; i < startingFileOffset; ++i) { + tasks.next(); + } + + updateCurrentIterator(); + // skip records within the file + for (long i = 0; i < startingRecordOffset; ++i) { + if (currentFileHasNext() && hasNext()) { + next(); + } else { + throw new IllegalStateException( + String.format( + "Invalid starting record offset %d for file %d from CombinedScanTask: %s", + startingRecordOffset, startingFileOffset, combinedTask)); + } + } + + fileOffset = startingFileOffset; + recordOffset = startingRecordOffset; + } + + @Override + public boolean hasNext() { + updateCurrentIterator(); + return currentIterator.hasNext(); + } + + @Override + public T next() { + updateCurrentIterator(); + recordOffset += 1; + return currentIterator.next(); + } + + public boolean currentFileHasNext() { + return currentIterator.hasNext(); + } + + /** Updates the current iterator field to ensure that the current Iterator is not exhausted. */ + private void updateCurrentIterator() { + try { + while (!currentIterator.hasNext() && tasks.hasNext()) { + currentIterator.close(); + currentIterator = openTaskIterator(tasks.next()); + fileOffset += 1; + recordOffset = 0L; + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private CloseableIterator openTaskIterator(FileScanTask scanTask) { + return fileScanTaskReader.open(scanTask, inputFilesDecryptor); + } + + @Override + public void close() throws IOException { + // close the current iterator + currentIterator.close(); + tasks = null; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java new file mode 100644 index 000000000000..4394dab4d4cc --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java @@ -0,0 +1,47 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.flink.data.StructRowData; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; + +@Internal +public class DataTaskReader implements FileScanTaskReader { + + private final Schema readSchema; + + public DataTaskReader(Schema readSchema) { + this.readSchema = readSchema; + } + + @Override + public CloseableIterator open( + FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { + StructRowData row = new StructRowData(readSchema.asStruct()); + CloseableIterable iterable = + CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); + return iterable.iterator(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java new file mode 100644 index 000000000000..927a804a4792 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.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.flink.source; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Read a {@link FileScanTask} into a {@link CloseableIterator} + * + * @param is the output data type returned by this iterator. + */ +@Internal +public interface FileScanTaskReader extends Serializable { + CloseableIterator open(FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java new file mode 100644 index 000000000000..9a5123dc489e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java @@ -0,0 +1,141 @@ +/* + * 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.util.concurrent.ExecutorService; +import org.apache.flink.api.common.io.DefaultInputSplitAssigner; +import org.apache.flink.api.common.io.InputFormat; +import org.apache.flink.api.common.io.LocatableInputSplitAssigner; +import org.apache.flink.api.common.io.RichInputFormat; +import org.apache.flink.api.common.io.statistics.BaseStatistics; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.io.InputSplitAssigner; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.util.ThreadPools; + +/** Flink {@link InputFormat} for Iceberg. */ +public class FlinkInputFormat extends RichInputFormat { + + private static final long serialVersionUID = 1L; + + private final TableLoader tableLoader; + private final FileIO io; + private final EncryptionManager encryption; + private final ScanContext context; + private final FileScanTaskReader rowDataReader; + + private transient DataIterator iterator; + private transient long currentReadCount = 0L; + + FlinkInputFormat( + TableLoader tableLoader, + Schema tableSchema, + FileIO io, + EncryptionManager encryption, + ScanContext context) { + this.tableLoader = tableLoader; + this.io = io; + this.encryption = encryption; + this.context = context; + + tableLoader.open(); + Table table = tableLoader.loadTable(); + if (table instanceof BaseMetadataTable) { + this.rowDataReader = new DataTaskReader(context.project()); + } else { + this.rowDataReader = + new RowDataFileScanTaskReader( + tableSchema, + context.project(), + context.nameMapping(), + context.caseSensitive(), + context.filters()); + } + } + + @VisibleForTesting + Schema projectedSchema() { + return context.project(); + } + + @Override + public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { + // Legacy method, not be used. + return null; + } + + @Override + public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { + // Called in Job manager, so it is OK to load table from catalog. + tableLoader.open(); + final ExecutorService workerPool = + ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); + try (TableLoader loader = tableLoader) { + Table table = loader.loadTable(); + return FlinkSplitPlanner.planInputSplits(table, context, workerPool); + } finally { + workerPool.shutdown(); + } + } + + @Override + public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { + return context.exposeLocality() + ? new LocatableInputSplitAssigner(inputSplits) + : new DefaultInputSplitAssigner(inputSplits); + } + + @Override + public void configure(Configuration parameters) {} + + @Override + public void open(FlinkInputSplit split) { + this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption); + } + + @Override + public boolean reachedEnd() { + if (context.limit() > 0 && currentReadCount >= context.limit()) { + return true; + } else { + return !iterator.hasNext(); + } + } + + @Override + public RowData nextRecord(RowData reuse) { + currentReadCount++; + return iterator.next(); + } + + @Override + public void close() throws IOException { + if (iterator != null) { + iterator.close(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java new file mode 100644 index 000000000000..16fd4f39596c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java @@ -0,0 +1,48 @@ +/* + * 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.util.Arrays; +import javax.annotation.Nullable; +import org.apache.flink.core.io.LocatableInputSplit; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +public class FlinkInputSplit extends LocatableInputSplit { + + private final CombinedScanTask task; + + FlinkInputSplit(int splitNumber, CombinedScanTask task, @Nullable String[] hostnames) { + super(splitNumber, hostnames); + this.task = task; + } + + CombinedScanTask getTask() { + return task; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("splitNumber", getSplitNumber()) + .add("task", task) + .add("hosts", Arrays.toString(getHostnames())) + .toString(); + } +} 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 new file mode 100644 index 000000000000..b1431a32dd20 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java @@ -0,0 +1,307 @@ +/* + * 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.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.Configuration; +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.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.FileIO; +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.PropertyUtil; + +public class FlinkSource { + private FlinkSource() {} + + /** + * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link + * TableScan}. See more options in {@link ScanContext}. + * + *

    The Source can be read static data in bounded mode. It can also continuously check the + * arrival of new data and read records incrementally. + * + *

      + *
    • Without startSnapshotId: Bounded + *
    • With startSnapshotId and with endSnapshotId: Bounded + *
    • With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded + *
    + * + *

    + * + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData() { + return new Builder(); + } + + /** Source builder to build {@link DataStream}. */ + public static class Builder { + private StreamExecutionEnvironment env; + private Table table; + private TableLoader tableLoader; + private TableSchema projectedSchema; + private ReadableConfig readableConfig = new Configuration(); + private final ScanContext.Builder contextBuilder = ScanContext.builder(); + private Boolean exposeLocality; + + private final Map readOptions = Maps.newHashMap(); + + public Builder tableLoader(TableLoader newLoader) { + this.tableLoader = newLoader; + return this; + } + + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + public Builder env(StreamExecutionEnvironment newEnv) { + this.env = newEnv; + return this; + } + + public Builder filters(List filters) { + contextBuilder.filters(filters); + return this; + } + + public Builder project(TableSchema schema) { + this.projectedSchema = schema; + return this; + } + + public Builder limit(Long newLimit) { + if (newLimit != null) { + readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); + } + return this; + } + + public Builder set(String property, String value) { + readOptions.put(property, value); + return this; + } + + public Builder setAll(Map properties) { + readOptions.putAll(properties); + return this; + } + + /** @deprecated Use {@link #setAll} instead. */ + @Deprecated + public Builder properties(Map properties) { + readOptions.putAll(properties); + return this; + } + + public Builder caseSensitive(boolean caseSensitive) { + readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(caseSensitive)); + return this; + } + + public Builder snapshotId(Long snapshotId) { + readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(snapshotId)); + return this; + } + + public Builder branch(String branch) { + readOptions.put(FlinkReadOptions.BRANCH.key(), branch); + return this; + } + + public Builder tag(String tag) { + readOptions.put(FlinkReadOptions.TAG.key(), tag); + return this; + } + + public Builder startSnapshotId(Long startSnapshotId) { + readOptions.put(FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(startSnapshotId)); + return this; + } + + public Builder endSnapshotId(Long endSnapshotId) { + readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(endSnapshotId)); + return this; + } + + public Builder startTag(String startTag) { + readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); + return this; + } + + public Builder endTag(String endTag) { + readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); + return this; + } + + public Builder asOfTimestamp(Long asOfTimestamp) { + readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(asOfTimestamp)); + return this; + } + + public Builder splitSize(Long splitSize) { + readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(splitSize)); + return this; + } + + public Builder splitLookback(Integer splitLookback) { + readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(splitLookback)); + return this; + } + + public Builder splitOpenFileCost(Long splitOpenFileCost) { + readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(splitOpenFileCost)); + return this; + } + + public Builder streaming(boolean streaming) { + readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder nameMapping(String nameMapping) { + readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, nameMapping); + return this; + } + + public Builder monitorInterval(Duration interval) { + readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, interval.toNanos() + " ns"); + return this; + } + + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + readOptions.put( + FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT, + Integer.toString(newMaxPlanningSnapshotCount)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + public FlinkInputFormat buildFormat() { + Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); + + Schema icebergSchema; + FileIO io; + EncryptionManager encryption; + if (table == null) { + // load required fields by table loader. + tableLoader.open(); + try (TableLoader loader = tableLoader) { + table = loader.loadTable(); + icebergSchema = table.schema(); + io = table.io(); + encryption = table.encryption(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } else { + icebergSchema = table.schema(); + io = table.io(); + encryption = table.encryption(); + } + + if (projectedSchema == null) { + contextBuilder.project(icebergSchema); + } else { + contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); + } + + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, readableConfig, exposeLocality)); + contextBuilder.planParallelism( + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); + + contextBuilder.resolveConfig(table, readOptions, readableConfig); + + ScanContext context = contextBuilder.build(); + context.validate(); + return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption, context); + } + + public DataStream build() { + Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); + FlinkInputFormat format = buildFormat(); + + ScanContext context = contextBuilder.build(); + TypeInformation typeInfo = + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project())); + + if (!context.isStreaming()) { + int parallelism = + SourceUtil.inferParallelism( + readableConfig, + context.limit(), + () -> { + try { + return format.createInputSplits(0).length; + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to create iceberg input splits for table: " + table, e); + } + }); + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + return env.createInput(format, typeInfo).setParallelism(parallelism); + } else { + StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context); + + String monitorFunctionName = String.format("Iceberg table (%s) monitor", table); + String readerOperatorName = String.format("Iceberg table (%s) reader", table); + + return env.addSource(function, monitorFunctionName) + .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format)); + } + } + } + + public static boolean isBounded(Map properties) { + return !PropertyUtil.propertyAsBoolean(properties, FlinkReadOptions.STREAMING, false); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java new file mode 100644 index 000000000000..15078809714f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java @@ -0,0 +1,189 @@ +/* + * 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.io.UncheckedIOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.IncrementalAppendScan; +import org.apache.iceberg.Scan; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.io.CloseableIterable; +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.apache.iceberg.util.Tasks; + +@Internal +public class FlinkSplitPlanner { + private FlinkSplitPlanner() {} + + static FlinkInputSplit[] planInputSplits( + Table table, ScanContext context, ExecutorService workerPool) { + try (CloseableIterable tasksIterable = + planTasks(table, context, workerPool)) { + List tasks = Lists.newArrayList(tasksIterable); + FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()]; + boolean exposeLocality = context.exposeLocality(); + + Tasks.range(tasks.size()) + .stopOnFailure() + .executeWith(exposeLocality ? workerPool : null) + .run( + index -> { + CombinedScanTask task = tasks.get(index); + String[] hostnames = null; + if (exposeLocality) { + hostnames = Util.blockLocations(table.io(), task); + } + splits[index] = new FlinkInputSplit(index, task, hostnames); + }); + return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to process tasks iterable", e); + } + } + + /** This returns splits for the FLIP-27 source */ + public static List planIcebergSourceSplits( + Table table, ScanContext context, ExecutorService workerPool) { + try (CloseableIterable tasksIterable = + planTasks(table, context, workerPool)) { + return Lists.newArrayList( + CloseableIterable.transform(tasksIterable, IcebergSourceSplit::fromCombinedScanTask)); + } catch (IOException e) { + throw new UncheckedIOException("Failed to process task iterable: ", e); + } + } + + static CloseableIterable planTasks( + Table table, ScanContext context, ExecutorService workerPool) { + ScanMode scanMode = checkScanMode(context); + if (scanMode == ScanMode.INCREMENTAL_APPEND_SCAN) { + IncrementalAppendScan scan = table.newIncrementalAppendScan(); + scan = refineScanWithBaseConfigs(scan, context, workerPool); + + if (context.startTag() != null) { + Preconditions.checkArgument( + table.snapshot(context.startTag()) != null, + "Cannot find snapshot with tag %s", + context.startTag()); + scan = scan.fromSnapshotExclusive(table.snapshot(context.startTag()).snapshotId()); + } + + if (context.startSnapshotId() != null) { + Preconditions.checkArgument( + context.startTag() == null, "START_SNAPSHOT_ID and START_TAG cannot both be set"); + scan = scan.fromSnapshotExclusive(context.startSnapshotId()); + } + + if (context.endTag() != null) { + Preconditions.checkArgument( + table.snapshot(context.endTag()) != null, + "Cannot find snapshot with tag %s", + context.endTag()); + scan = scan.toSnapshot(table.snapshot(context.endTag()).snapshotId()); + } + + if (context.endSnapshotId() != null) { + Preconditions.checkArgument( + context.endTag() == null, "END_SNAPSHOT_ID and END_TAG cannot both be set"); + scan = scan.toSnapshot(context.endSnapshotId()); + } + + return scan.planTasks(); + } else { + TableScan scan = table.newScan(); + scan = refineScanWithBaseConfigs(scan, context, workerPool); + + if (context.snapshotId() != null) { + scan = scan.useSnapshot(context.snapshotId()); + } else if (context.tag() != null) { + scan = scan.useRef(context.tag()); + } else if (context.branch() != null) { + scan = scan.useRef(context.branch()); + } + + if (context.asOfTimestamp() != null) { + scan = scan.asOfTime(context.asOfTimestamp()); + } + + return scan.planTasks(); + } + } + + @VisibleForTesting + enum ScanMode { + BATCH, + INCREMENTAL_APPEND_SCAN + } + + @VisibleForTesting + static ScanMode checkScanMode(ScanContext context) { + if (context.startSnapshotId() != null + || context.endSnapshotId() != null + || context.startTag() != null + || context.endTag() != null) { + return ScanMode.INCREMENTAL_APPEND_SCAN; + } else { + return ScanMode.BATCH; + } + } + + /** refine scan with common configs */ + private static > T refineScanWithBaseConfigs( + T scan, ScanContext context, ExecutorService workerPool) { + T refinedScan = + scan.caseSensitive(context.caseSensitive()).project(context.project()).planWith(workerPool); + + if (context.includeColumnStats()) { + refinedScan = refinedScan.includeColumnStats(); + } + + if (context.includeStatsForColumns() != null) { + refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); + } + + refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); + + refinedScan = + refinedScan.option(TableProperties.SPLIT_LOOKBACK, context.splitLookback().toString()); + + refinedScan = + refinedScan.option( + TableProperties.SPLIT_OPEN_FILE_COST, context.splitOpenFileCost().toString()); + + if (context.filters() != null) { + for (Expression filter : context.filters()) { + refinedScan = refinedScan.filter(filter); + } + } + + return refinedScan; + } +} 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 new file mode 100644 index 000000000000..ccbd0d9997ed --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java @@ -0,0 +1,549 @@ +/* + * 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.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +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.connector.source.Boundedness; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.BaseMetadataTable; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadConf; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; +import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; +import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner; +import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl; +import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; +import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; +import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.IcebergSourceReader; +import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; +import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; +import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; +import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +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.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Experimental +public class IcebergSource implements Source { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); + + // This table loader can be closed, and it is only safe to use this instance for resource + // independent information (e.g. a table name). Copies of this are required to avoid lifecycle + // management conflicts with the user provided table loader. e.g. a copy of this is required for + // split planning, which uses the underlying io, and should be closed after split planning is + // complete. + private final TableLoader tableLoader; + private final ScanContext scanContext; + private final ReaderFunction readerFunction; + private final SplitAssignerFactory assignerFactory; + private final SerializableComparator splitComparator; + private final SerializableRecordEmitter emitter; + private final String tableName; + + IcebergSource( + TableLoader tableLoader, + ScanContext scanContext, + ReaderFunction readerFunction, + SplitAssignerFactory assignerFactory, + SerializableComparator splitComparator, + Table table, + SerializableRecordEmitter emitter) { + Preconditions.checkNotNull(tableLoader, "tableLoader is required."); + Preconditions.checkNotNull(readerFunction, "readerFunction is required."); + Preconditions.checkNotNull(assignerFactory, "assignerFactory is required."); + Preconditions.checkNotNull(table, "table is required."); + this.tableLoader = tableLoader; + this.scanContext = scanContext; + this.readerFunction = readerFunction; + this.assignerFactory = assignerFactory; + this.splitComparator = splitComparator; + this.emitter = emitter; + this.tableName = table.name(); + } + + String name() { + return "IcebergSource-" + tableName; + } + + private String planningThreadName() { + // Ideally, operatorId should be used as the threadPoolName as Flink guarantees its uniqueness + // within a job. SplitEnumeratorContext doesn't expose the OperatorCoordinator.Context, which + // would contain the OperatorID. Need to discuss with Flink community whether it is ok to expose + // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" + // from SourceCoordinatorContext implementation. For now,

  • - is used as + // the unique thread pool name. + return tableName + "-" + UUID.randomUUID(); + } + + private List planSplitsForBatch(String threadName) { + ExecutorService workerPool = + ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); + try (TableLoader loader = tableLoader.clone()) { + loader.open(); + List splits = + FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); + LOG.info( + "Discovered {} splits from table {} during job initialization", splits.size(), tableName); + return splits; + } catch (IOException e) { + throw new UncheckedIOException("Failed to close table loader", e); + } finally { + workerPool.shutdown(); + } + } + + @Override + public Boundedness getBoundedness() { + return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; + } + + @Override + public SourceReader createReader(SourceReaderContext readerContext) { + IcebergSourceReaderMetrics metrics = + new IcebergSourceReaderMetrics(readerContext.metricGroup(), tableName); + return new IcebergSourceReader<>( + emitter, metrics, readerFunction, splitComparator, readerContext); + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return createEnumerator(enumContext, null); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, IcebergEnumeratorState enumState) { + return createEnumerator(enumContext, enumState); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); + } + + private SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext, + @Nullable IcebergEnumeratorState enumState) { + SplitAssigner assigner; + if (enumState == null) { + assigner = assignerFactory.createAssigner(); + } else { + LOG.info( + "Iceberg source restored {} splits from state for table {}", + enumState.pendingSplits().size(), + tableName); + assigner = assignerFactory.createAssigner(enumState.pendingSplits()); + } + if (scanContext.isStreaming()) { + ContinuousSplitPlanner splitPlanner = + new ContinuousSplitPlannerImpl(tableLoader, scanContext, planningThreadName()); + return new ContinuousIcebergEnumerator( + enumContext, assigner, scanContext, splitPlanner, enumState); + } else { + if (enumState == null) { + // Only do scan planning if nothing is restored from checkpoint state + List splits = planSplitsForBatch(planningThreadName()); + assigner.onDiscoveredSplits(splits); + } + + return new StaticIcebergEnumerator(enumContext, assigner); + } + } + + public static Builder builder() { + return new Builder<>(); + } + + public static Builder forRowData() { + return new Builder<>(); + } + + public static class Builder { + private TableLoader tableLoader; + private Table table; + private SplitAssignerFactory splitAssignerFactory; + private SerializableComparator splitComparator; + private ReaderFunction readerFunction; + private ReadableConfig flinkConfig = new Configuration(); + private final ScanContext.Builder contextBuilder = ScanContext.builder(); + private TableSchema projectedFlinkSchema; + private Boolean exposeLocality; + + private final Map readOptions = Maps.newHashMap(); + + Builder() {} + + public Builder tableLoader(TableLoader loader) { + this.tableLoader = loader; + return this; + } + + public Builder table(Table newTable) { + this.table = newTable; + return this; + } + + public Builder assignerFactory(SplitAssignerFactory assignerFactory) { + this.splitAssignerFactory = assignerFactory; + return this; + } + + public Builder splitComparator( + SerializableComparator newSplitComparator) { + this.splitComparator = newSplitComparator; + return this; + } + + public Builder readerFunction(ReaderFunction newReaderFunction) { + this.readerFunction = newReaderFunction; + return this; + } + + public Builder flinkConfig(ReadableConfig config) { + this.flinkConfig = config; + return this; + } + + public Builder caseSensitive(boolean newCaseSensitive) { + readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(newCaseSensitive)); + return this; + } + + public Builder useSnapshotId(Long newSnapshotId) { + if (newSnapshotId != null) { + readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(newSnapshotId)); + } + return this; + } + + public Builder streamingStartingStrategy(StreamingStartingStrategy newStartingStrategy) { + readOptions.put(FlinkReadOptions.STARTING_STRATEGY, newStartingStrategy.name()); + return this; + } + + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + if (newStartSnapshotTimestamp != null) { + readOptions.put( + FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key(), + Long.toString(newStartSnapshotTimestamp)); + } + return this; + } + + public Builder startSnapshotId(Long newStartSnapshotId) { + if (newStartSnapshotId != null) { + readOptions.put( + FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(newStartSnapshotId)); + } + return this; + } + + public Builder tag(String tag) { + readOptions.put(FlinkReadOptions.TAG.key(), tag); + return this; + } + + public Builder branch(String branch) { + readOptions.put(FlinkReadOptions.BRANCH.key(), branch); + return this; + } + + public Builder startTag(String startTag) { + readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); + return this; + } + + public Builder endTag(String endTag) { + readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); + return this; + } + + public Builder endSnapshotId(Long newEndSnapshotId) { + if (newEndSnapshotId != null) { + readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(newEndSnapshotId)); + } + return this; + } + + public Builder asOfTimestamp(Long newAsOfTimestamp) { + if (newAsOfTimestamp != null) { + readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(newAsOfTimestamp)); + } + return this; + } + + public Builder splitSize(Long newSplitSize) { + if (newSplitSize != null) { + readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(newSplitSize)); + } + return this; + } + + public Builder splitLookback(Integer newSplitLookback) { + if (newSplitLookback != null) { + readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(newSplitLookback)); + } + return this; + } + + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { + if (newSplitOpenFileCost != null) { + readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(newSplitOpenFileCost)); + } + + return this; + } + + public Builder streaming(boolean streaming) { + readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); + return this; + } + + public Builder monitorInterval(Duration newMonitorInterval) { + if (newMonitorInterval != null) { + readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, newMonitorInterval.toNanos() + " ns"); + } + return this; + } + + public Builder nameMapping(String newNameMapping) { + readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, newNameMapping); + return this; + } + + public Builder project(Schema newProjectedSchema) { + this.contextBuilder.project(newProjectedSchema); + return this; + } + + public Builder project(TableSchema newProjectedFlinkSchema) { + this.projectedFlinkSchema = newProjectedFlinkSchema; + return this; + } + + public Builder filters(List newFilters) { + this.contextBuilder.filters(newFilters); + return this; + } + + public Builder limit(Long newLimit) { + if (newLimit != null) { + readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); + } + return this; + } + + public Builder includeColumnStats(boolean newIncludeColumnStats) { + readOptions.put( + FlinkReadOptions.INCLUDE_COLUMN_STATS, Boolean.toString(newIncludeColumnStats)); + return this; + } + + public Builder planParallelism(int planParallelism) { + readOptions.put( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key(), + Integer.toString(planParallelism)); + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder maxAllowedPlanningFailures(int maxAllowedPlanningFailures) { + readOptions.put( + FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.key(), + Integer.toString(maxAllowedPlanningFailures)); + return this; + } + + /** + * Set the read properties for Flink source. View the supported properties in {@link + * FlinkReadOptions} + */ + public Builder set(String property, String value) { + readOptions.put(property, value); + return this; + } + + /** + * Set the read properties for Flink source. View the supported properties in {@link + * FlinkReadOptions} + */ + public Builder setAll(Map properties) { + readOptions.putAll(properties); + return this; + } + + /** + * Emits watermarks once per split based on the min value of column statistics from files + * metadata in the given split. The generated watermarks are also used for ordering the splits + * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider + * setting {@link #watermarkColumnTimeUnit(TimeUnit)}. + * + *

    Consider setting `read.split.open-file-cost` to prevent combining small files to a single + * split when the watermark is used for watermark alignment. + */ + public Builder watermarkColumn(String columnName) { + Preconditions.checkArgument( + splitAssignerFactory == null, + "Watermark column and SplitAssigner should not be set in the same source"); + readOptions.put(FlinkReadOptions.WATERMARK_COLUMN, columnName); + return this; + } + + /** + * When the type of the {@link #watermarkColumn} is {@link + * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the + * value. The default value is {@link TimeUnit#MICROSECONDS}. + */ + public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { + readOptions.put(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT, timeUnit.name()); + return this; + } + + /** @deprecated Use {@link #setAll} instead. */ + @Deprecated + public Builder properties(Map properties) { + readOptions.putAll(properties); + return this; + } + + public IcebergSource build() { + if (table == null) { + try (TableLoader loader = tableLoader) { + loader.open(); + this.table = tableLoader.loadTable(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + contextBuilder.resolveConfig(table, readOptions, flinkConfig); + Schema icebergSchema = table.schema(); + if (projectedFlinkSchema != null) { + contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); + } + + SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); + FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, flinkConfig); + String watermarkColumn = flinkReadConf.watermarkColumn(); + TimeUnit watermarkTimeUnit = flinkReadConf.watermarkColumnTimeUnit(); + + if (watermarkColumn != null) { + // Column statistics is needed for watermark generation + contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); + + SplitWatermarkExtractor watermarkExtractor = + new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); + emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); + splitAssignerFactory = + new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); + } + + ScanContext context = contextBuilder.build(); + context.validate(); + if (readerFunction == null) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + this.readerFunction = (ReaderFunction) rowDataReaderFunction; + } else { + RowDataReaderFunction rowDataReaderFunction = + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + this.readerFunction = (ReaderFunction) rowDataReaderFunction; + } + } + + if (splitAssignerFactory == null) { + if (splitComparator == null) { + splitAssignerFactory = new SimpleSplitAssignerFactory(); + } else { + splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); + } + } + + // Since builder already load the table, pass it to the source to avoid double loading + return new IcebergSource<>( + tableLoader, + context, + readerFunction, + splitAssignerFactory, + splitComparator, + table, + emitter); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java new file mode 100644 index 000000000000..610657e8d47b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.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.flink.source; + +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; +import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.types.DataType; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkFilters; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.SplitAssignerType; +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; + +/** Flink Iceberg table source. */ +@Internal +public class IcebergTableSource + implements ScanTableSource, + SupportsProjectionPushDown, + SupportsFilterPushDown, + SupportsLimitPushDown { + + private int[] projectedFields; + private Long limit; + private List filters; + + private final TableLoader loader; + private final TableSchema schema; + private final Map properties; + private final boolean isLimitPushDown; + private final ReadableConfig readableConfig; + + private IcebergTableSource(IcebergTableSource toCopy) { + this.loader = toCopy.loader; + this.schema = toCopy.schema; + this.properties = toCopy.properties; + this.projectedFields = toCopy.projectedFields; + this.isLimitPushDown = toCopy.isLimitPushDown; + this.limit = toCopy.limit; + this.filters = toCopy.filters; + this.readableConfig = toCopy.readableConfig; + } + + public IcebergTableSource( + TableLoader loader, + TableSchema schema, + Map properties, + ReadableConfig readableConfig) { + this(loader, schema, properties, null, false, null, ImmutableList.of(), readableConfig); + } + + private IcebergTableSource( + TableLoader loader, + TableSchema schema, + Map properties, + int[] projectedFields, + boolean isLimitPushDown, + Long limit, + List filters, + ReadableConfig readableConfig) { + this.loader = loader; + this.schema = schema; + this.properties = properties; + this.projectedFields = projectedFields; + this.isLimitPushDown = isLimitPushDown; + this.limit = limit; + this.filters = filters; + this.readableConfig = readableConfig; + } + + @Override + public void applyProjection(int[][] projectFields) { + this.projectedFields = new int[projectFields.length]; + for (int i = 0; i < projectFields.length; i++) { + Preconditions.checkArgument( + projectFields[i].length == 1, "Don't support nested projection in iceberg source now."); + this.projectedFields[i] = projectFields[i][0]; + } + } + + private DataStream createDataStream(StreamExecutionEnvironment execEnv) { + return FlinkSource.forRowData() + .env(execEnv) + .tableLoader(loader) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConf(readableConfig) + .build(); + } + + private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + SplitAssignerType assignerType = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); + IcebergSource source = + IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .build(); + DataStreamSource stream = + env.fromSource( + source, + WatermarkStrategy.noWatermarks(), + source.name(), + TypeInformation.of(RowData.class)); + return stream; + } + + private TableSchema getProjectedSchema() { + if (projectedFields == null) { + return schema; + } else { + String[] fullNames = schema.getFieldNames(); + DataType[] fullTypes = schema.getFieldDataTypes(); + return TableSchema.builder() + .fields( + Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), + Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)) + .build(); + } + } + + @Override + public void applyLimit(long newLimit) { + this.limit = newLimit; + } + + @Override + public Result applyFilters(List flinkFilters) { + List acceptedFilters = Lists.newArrayList(); + List expressions = Lists.newArrayList(); + + for (ResolvedExpression resolvedExpression : flinkFilters) { + Optional icebergExpression = FlinkFilters.convert(resolvedExpression); + if (icebergExpression.isPresent()) { + expressions.add(icebergExpression.get()); + acceptedFilters.add(resolvedExpression); + } + } + + this.filters = expressions; + return Result.of(acceptedFilters, flinkFilters); + } + + @Override + public boolean supportsNestedProjection() { + // TODO: support nested projection + return false; + } + + @Override + public ChangelogMode getChangelogMode() { + return ChangelogMode.insertOnly(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment execEnv) { + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { + return createFLIP27Stream(execEnv); + } else { + return createDataStream(execEnv); + } + } + + @Override + public boolean isBounded() { + return FlinkSource.isBounded(properties); + } + }; + } + + @Override + public DynamicTableSource copy() { + return new IcebergTableSource(this); + } + + @Override + public String asSummaryString() { + return "Iceberg table source"; + } +} 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 new file mode 100644 index 000000000000..88364f4e87b1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java @@ -0,0 +1,243 @@ +/* + * 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.util.List; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DeleteFilter; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +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.RowDataProjection; +import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.mapping.NameMappingParser; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.PartitionUtil; + +@Internal +public class RowDataFileScanTaskReader implements FileScanTaskReader { + + private final Schema tableSchema; + private final Schema projectedSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FlinkSourceFilter rowFilter; + + public RowDataFileScanTaskReader( + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + List filters) { + this.tableSchema = tableSchema; + this.projectedSchema = projectedSchema; + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + + if (filters != null && !filters.isEmpty()) { + Expression combinedExpression = + filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); + this.rowFilter = + new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); + } else { + this.rowFilter = null; + } + } + + @Override + public CloseableIterator open( + FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { + Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds()); + + Map idToConstant = + partitionSchema.columns().isEmpty() + ? ImmutableMap.of() + : PartitionUtil.constantsMap(task, RowDataUtil::convertConstant); + + FlinkDeleteFilter deletes = + new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor); + CloseableIterable iterable = + deletes.filter( + newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor)); + + // Project the RowData to remove the extra meta columns. + if (!projectedSchema.sameSchema(deletes.requiredSchema())) { + RowDataProjection rowDataProjection = + RowDataProjection.create( + deletes.requiredRowType(), + deletes.requiredSchema().asStruct(), + projectedSchema.asStruct()); + iterable = CloseableIterable.transform(iterable, rowDataProjection::wrap); + } + + return iterable.iterator(); + } + + private CloseableIterable newIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + CloseableIterable iter; + if (task.isDataTask()) { + throw new UnsupportedOperationException("Cannot read data task."); + } else { + switch (task.file().format()) { + case PARQUET: + iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + case AVRO: + iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + case ORC: + iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); + break; + + default: + throw new UnsupportedOperationException( + "Cannot read unknown format: " + task.file().format()); + } + } + + if (rowFilter != null) { + return CloseableIterable.filter(iter, rowFilter::filter); + } + return iter; + } + + private CloseableIterable newAvroIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Avro.ReadBuilder builder = + Avro.read(inputFilesDecryptor.getInputFile(task)) + .reuseContainers() + .project(schema) + .split(task.start(), task.length()) + .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private CloseableIterable newParquetIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Parquet.ReadBuilder builder = + Parquet.read(inputFilesDecryptor.getInputFile(task)) + .split(task.start(), task.length()) + .project(schema) + .createReaderFunc( + fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) + .filter(task.residual()) + .caseSensitive(caseSensitive) + .reuseContainers(); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private CloseableIterable newOrcIterable( + FileScanTask task, + Schema schema, + Map idToConstant, + InputFilesDecryptor inputFilesDecryptor) { + Schema readSchemaWithoutConstantAndMetadataFields = + TypeUtil.selectNot( + schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); + + ORC.ReadBuilder builder = + ORC.read(inputFilesDecryptor.getInputFile(task)) + .project(readSchemaWithoutConstantAndMetadataFields) + .split(task.start(), task.length()) + .createReaderFunc( + readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) + .filter(task.residual()) + .caseSensitive(caseSensitive); + + if (nameMapping != null) { + builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); + } + + return builder.build(); + } + + private static class FlinkDeleteFilter extends DeleteFilter { + private final RowType requiredRowType; + private final RowDataWrapper asStructLike; + private final InputFilesDecryptor inputFilesDecryptor; + + FlinkDeleteFilter( + FileScanTask task, + Schema tableSchema, + Schema requestedSchema, + InputFilesDecryptor inputFilesDecryptor) { + super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); + this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); + this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); + this.inputFilesDecryptor = inputFilesDecryptor; + } + + public RowType requiredRowType() { + return requiredRowType; + } + + @Override + protected StructLike asStructLike(RowData row) { + return asStructLike.wrap(row); + } + + @Override + protected InputFile getInputFile(String location) { + return inputFilesDecryptor.getInputFile(location); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java new file mode 100644 index 000000000000..c958604c004a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -0,0 +1,172 @@ +/* + * 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 static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RowDataRewriter { + + private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); + + private final Schema schema; + private final String nameMapping; + private final FileIO io; + private final boolean caseSensitive; + private final EncryptionManager encryptionManager; + private final TaskWriterFactory taskWriterFactory; + private final String tableName; + + public RowDataRewriter( + Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { + this.schema = table.schema(); + this.caseSensitive = caseSensitive; + this.io = io; + this.encryptionManager = encryptionManager; + this.nameMapping = + PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); + this.tableName = table.name(); + + String formatString = + PropertyUtil.propertyAsString( + table.properties(), + TableProperties.DEFAULT_FILE_FORMAT, + TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); + FileFormat format = FileFormat.fromString(formatString); + RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); + this.taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkSchema, + Long.MAX_VALUE, + format, + table.properties(), + null, + false); + } + + public List rewriteDataForTasks( + DataStream dataStream, int parallelism) throws Exception { + RewriteMap map = + new RewriteMap( + schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); + DataStream> ds = dataStream.map(map).setParallelism(parallelism); + return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() + .flatMap(Collection::stream) + .collect(Collectors.toList()); + } + + public static class RewriteMap extends RichMapFunction> { + + private TaskWriter writer; + private int subTaskId; + private int attemptId; + + private final Schema schema; + private final String nameMapping; + private final FileIO io; + private final boolean caseSensitive; + private final EncryptionManager encryptionManager; + private final TaskWriterFactory taskWriterFactory; + private final RowDataFileScanTaskReader rowDataReader; + + public RewriteMap( + Schema schema, + String nameMapping, + FileIO io, + boolean caseSensitive, + EncryptionManager encryptionManager, + TaskWriterFactory taskWriterFactory) { + this.schema = schema; + this.nameMapping = nameMapping; + this.io = io; + this.caseSensitive = caseSensitive; + this.encryptionManager = encryptionManager; + this.taskWriterFactory = taskWriterFactory; + this.rowDataReader = + new RowDataFileScanTaskReader( + schema, schema, nameMapping, caseSensitive, Collections.emptyList()); + } + + @Override + public void open(Configuration parameters) { + this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); + this.attemptId = getRuntimeContext().getAttemptNumber(); + // Initialize the task writer factory. + this.taskWriterFactory.initialize(subTaskId, attemptId); + } + + @Override + public List map(CombinedScanTask task) throws Exception { + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + try (DataIterator iterator = + new DataIterator<>(rowDataReader, task, io, encryptionManager)) { + while (iterator.hasNext()) { + RowData rowData = iterator.next(); + writer.write(rowData); + } + return Lists.newArrayList(writer.dataFiles()); + } catch (Throwable originalThrowable) { + try { + LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); + writer.abort(); + LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); + } catch (Throwable inner) { + if (originalThrowable != inner) { + originalThrowable.addSuppressed(inner); + LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); + } + } + + if (originalThrowable instanceof Exception) { + throw originalThrowable; + } else { + throw new RuntimeException(originalThrowable); + } + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java new file mode 100644 index 000000000000..8ef1f1fbb833 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java @@ -0,0 +1,70 @@ +/* + * 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.Serializable; +import java.util.function.Function; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.annotation.Internal; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +/** + * This is not serializable because Avro {@link Schema} is not actually serializable, even though it + * implements {@link Serializable} interface. + */ +@Internal +public class RowDataToAvroGenericRecordConverter implements Function { + private final RowDataToAvroConverters.RowDataToAvroConverter converter; + private final Schema avroSchema; + + private RowDataToAvroGenericRecordConverter(RowType rowType, Schema avroSchema) { + this.converter = RowDataToAvroConverters.createConverter(rowType); + this.avroSchema = avroSchema; + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) converter.convert(avroSchema, rowData); + } + + /** Create a converter based on Iceberg schema */ + public static RowDataToAvroGenericRecordConverter fromIcebergSchema( + String tableName, org.apache.iceberg.Schema icebergSchema) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); + } + + /** Create a mapper based on Avro schema */ + public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java new file mode 100644 index 000000000000..ab79a3173933 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java @@ -0,0 +1,597 @@ +/* + * 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.Serializable; +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.util.Preconditions; +import org.apache.flink.util.TimeUtils; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadConf; +import org.apache.iceberg.flink.FlinkReadOptions; + +/** Context object with optional arguments for a Flink Scan. */ +@Internal +public class ScanContext implements Serializable { + + private static final long serialVersionUID = 1L; + + private final boolean caseSensitive; + private final boolean exposeLocality; + private final Long snapshotId; + private final String branch; + private final String tag; + private final StreamingStartingStrategy startingStrategy; + private final Long startSnapshotId; + private final Long startSnapshotTimestamp; + private final Long endSnapshotId; + private final Long asOfTimestamp; + private final String startTag; + private final String endTag; + private final Long splitSize; + private final Integer splitLookback; + private final Long splitOpenFileCost; + private final boolean isStreaming; + private final Duration monitorInterval; + + private final String nameMapping; + private final Schema schema; + private final List filters; + private final long limit; + private final boolean includeColumnStats; + private final Collection includeStatsForColumns; + private final Integer planParallelism; + private final int maxPlanningSnapshotCount; + private final int maxAllowedPlanningFailures; + private final String watermarkColumn; + private final TimeUnit watermarkColumnTimeUnit; + + private ScanContext( + boolean caseSensitive, + Long snapshotId, + StreamingStartingStrategy startingStrategy, + Long startSnapshotTimestamp, + Long startSnapshotId, + Long endSnapshotId, + Long asOfTimestamp, + Long splitSize, + Integer splitLookback, + Long splitOpenFileCost, + boolean isStreaming, + Duration monitorInterval, + String nameMapping, + Schema schema, + List filters, + long limit, + boolean includeColumnStats, + Collection includeStatsForColumns, + boolean exposeLocality, + Integer planParallelism, + int maxPlanningSnapshotCount, + int maxAllowedPlanningFailures, + String watermarkColumn, + TimeUnit watermarkColumnTimeUnit, + String branch, + String tag, + String startTag, + String endTag) { + this.caseSensitive = caseSensitive; + this.snapshotId = snapshotId; + this.tag = tag; + this.branch = branch; + this.startingStrategy = startingStrategy; + this.startSnapshotTimestamp = startSnapshotTimestamp; + this.startSnapshotId = startSnapshotId; + this.endSnapshotId = endSnapshotId; + this.asOfTimestamp = asOfTimestamp; + this.startTag = startTag; + this.endTag = endTag; + this.splitSize = splitSize; + this.splitLookback = splitLookback; + this.splitOpenFileCost = splitOpenFileCost; + this.isStreaming = isStreaming; + this.monitorInterval = monitorInterval; + + this.nameMapping = nameMapping; + this.schema = schema; + this.filters = filters; + this.limit = limit; + this.includeColumnStats = includeColumnStats; + this.includeStatsForColumns = includeStatsForColumns; + this.exposeLocality = exposeLocality; + this.planParallelism = planParallelism; + this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; + this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; + this.watermarkColumn = watermarkColumn; + this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; + } + + void validate() { + if (isStreaming) { + if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { + Preconditions.checkArgument( + startSnapshotId != null, + "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + Preconditions.checkArgument( + startSnapshotTimestamp == null, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) { + Preconditions.checkArgument( + startSnapshotTimestamp != null, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + Preconditions.checkArgument( + startSnapshotId == null, + "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + Preconditions.checkArgument( + tag == null, + String.format("Cannot scan table using ref %s configured for streaming reader", tag)); + Preconditions.checkArgument( + snapshotId == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + asOfTimestamp == null, "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + endSnapshotId == null, "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument(endTag == null, "Cannot set end-tag option for streaming reader"); + } + + Preconditions.checkArgument( + !(startTag != null && startSnapshotId() != null), + "START_SNAPSHOT_ID and START_TAG cannot both be set."); + + Preconditions.checkArgument( + !(endTag != null && endSnapshotId() != null), + "END_SNAPSHOT_ID and END_TAG cannot both be set."); + + Preconditions.checkArgument( + maxAllowedPlanningFailures >= -1, + "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + public boolean caseSensitive() { + return caseSensitive; + } + + public Long snapshotId() { + return snapshotId; + } + + public String branch() { + return branch; + } + + public String tag() { + return tag; + } + + public String startTag() { + return startTag; + } + + public String endTag() { + return endTag; + } + + public StreamingStartingStrategy streamingStartingStrategy() { + return startingStrategy; + } + + public Long startSnapshotTimestamp() { + return startSnapshotTimestamp; + } + + public Long startSnapshotId() { + return startSnapshotId; + } + + public Long endSnapshotId() { + return endSnapshotId; + } + + public Long asOfTimestamp() { + return asOfTimestamp; + } + + public Long splitSize() { + return splitSize; + } + + public Integer splitLookback() { + return splitLookback; + } + + public Long splitOpenFileCost() { + return splitOpenFileCost; + } + + public boolean isStreaming() { + return isStreaming; + } + + public Duration monitorInterval() { + return monitorInterval; + } + + public String nameMapping() { + return nameMapping; + } + + public Schema project() { + return schema; + } + + public List filters() { + return filters; + } + + public long limit() { + return limit; + } + + public boolean includeColumnStats() { + return includeColumnStats; + } + + public Collection includeStatsForColumns() { + return includeStatsForColumns; + } + + public boolean exposeLocality() { + return exposeLocality; + } + + public Integer planParallelism() { + return planParallelism; + } + + public int maxPlanningSnapshotCount() { + return maxPlanningSnapshotCount; + } + + public int maxAllowedPlanningFailures() { + return maxAllowedPlanningFailures; + } + + public String watermarkColumn() { + return watermarkColumn; + } + + public TimeUnit watermarkColumnTimeUnit() { + return watermarkColumnTimeUnit; + } + + public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { + return ScanContext.builder() + .caseSensitive(caseSensitive) + .useSnapshotId(null) + .useBranch(branch) + .useTag(null) + .startSnapshotId(newStartSnapshotId) + .endSnapshotId(newEndSnapshotId) + .startTag(null) + .endTag(null) + .asOfTimestamp(null) + .splitSize(splitSize) + .splitLookback(splitLookback) + .splitOpenFileCost(splitOpenFileCost) + .streaming(isStreaming) + .monitorInterval(monitorInterval) + .nameMapping(nameMapping) + .project(schema) + .filters(filters) + .limit(limit) + .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) + .exposeLocality(exposeLocality) + .planParallelism(planParallelism) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit) + .build(); + } + + public ScanContext copyWithSnapshotId(long newSnapshotId) { + return ScanContext.builder() + .caseSensitive(caseSensitive) + .useSnapshotId(newSnapshotId) + .useBranch(branch) + .useTag(tag) + .startSnapshotId(null) + .endSnapshotId(null) + .startTag(null) + .endTag(null) + .asOfTimestamp(null) + .splitSize(splitSize) + .splitLookback(splitLookback) + .splitOpenFileCost(splitOpenFileCost) + .streaming(isStreaming) + .monitorInterval(monitorInterval) + .nameMapping(nameMapping) + .project(schema) + .filters(filters) + .limit(limit) + .includeColumnStats(includeColumnStats) + .includeColumnStats(includeStatsForColumns) + .exposeLocality(exposeLocality) + .planParallelism(planParallelism) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(watermarkColumn) + .watermarkColumnTimeUnit(watermarkColumnTimeUnit) + .build(); + } + + public static Builder builder() { + return new Builder(); + } + + public static class Builder { + private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); + private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); + private String branch = FlinkReadOptions.BRANCH.defaultValue(); + private String tag = FlinkReadOptions.TAG.defaultValue(); + private String startTag = FlinkReadOptions.START_TAG.defaultValue(); + private String endTag = FlinkReadOptions.END_TAG.defaultValue(); + private StreamingStartingStrategy startingStrategy = + FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue(); + private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue(); + private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue(); + private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue(); + private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue(); + private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue(); + private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue(); + private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue(); + private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue(); + private Duration monitorInterval = + TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()); + private String nameMapping; + private Schema projectedSchema; + private List filters; + private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); + private boolean includeColumnStats = + FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); + private Collection includeStatsForColumns = null; + private boolean exposeLocality; + private Integer planParallelism = + FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); + private int maxPlanningSnapshotCount = + FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); + private int maxAllowedPlanningFailures = + FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); + private String watermarkColumn = FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue(); + private TimeUnit watermarkColumnTimeUnit = + FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue(); + + private Builder() {} + + public Builder caseSensitive(boolean newCaseSensitive) { + this.caseSensitive = newCaseSensitive; + return this; + } + + public Builder useSnapshotId(Long newSnapshotId) { + this.snapshotId = newSnapshotId; + return this; + } + + public Builder useTag(String newTag) { + this.tag = newTag; + return this; + } + + public Builder useBranch(String newBranch) { + this.branch = newBranch; + return this; + } + + public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { + this.startingStrategy = newStartingStrategy; + return this; + } + + public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { + this.startSnapshotTimestamp = newStartSnapshotTimestamp; + return this; + } + + public Builder startSnapshotId(Long newStartSnapshotId) { + this.startSnapshotId = newStartSnapshotId; + return this; + } + + public Builder endSnapshotId(Long newEndSnapshotId) { + this.endSnapshotId = newEndSnapshotId; + return this; + } + + public Builder startTag(String newStartTag) { + this.startTag = newStartTag; + return this; + } + + public Builder endTag(String newEndTag) { + this.endTag = newEndTag; + return this; + } + + public Builder asOfTimestamp(Long newAsOfTimestamp) { + this.asOfTimestamp = newAsOfTimestamp; + return this; + } + + public Builder splitSize(Long newSplitSize) { + this.splitSize = newSplitSize; + return this; + } + + public Builder splitLookback(Integer newSplitLookback) { + this.splitLookback = newSplitLookback; + return this; + } + + public Builder splitOpenFileCost(Long newSplitOpenFileCost) { + this.splitOpenFileCost = newSplitOpenFileCost; + return this; + } + + public Builder streaming(boolean streaming) { + this.isStreaming = streaming; + return this; + } + + public Builder monitorInterval(Duration newMonitorInterval) { + this.monitorInterval = newMonitorInterval; + return this; + } + + public Builder nameMapping(String newNameMapping) { + this.nameMapping = newNameMapping; + return this; + } + + public Builder project(Schema newProjectedSchema) { + this.projectedSchema = newProjectedSchema; + return this; + } + + public Builder filters(List newFilters) { + this.filters = newFilters; + return this; + } + + public Builder limit(long newLimit) { + this.limit = newLimit; + return this; + } + + public Builder includeColumnStats(boolean newIncludeColumnStats) { + this.includeColumnStats = newIncludeColumnStats; + return this; + } + + public Builder includeColumnStats(Collection newIncludeStatsForColumns) { + this.includeStatsForColumns = newIncludeStatsForColumns; + return this; + } + + public Builder exposeLocality(boolean newExposeLocality) { + this.exposeLocality = newExposeLocality; + return this; + } + + public Builder planParallelism(Integer parallelism) { + this.planParallelism = parallelism; + return this; + } + + public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { + this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; + return this; + } + + public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { + this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; + return this; + } + + public Builder watermarkColumn(String newWatermarkColumn) { + this.watermarkColumn = newWatermarkColumn; + return this; + } + + public Builder watermarkColumnTimeUnit(TimeUnit newWatermarkTimeUnit) { + this.watermarkColumnTimeUnit = newWatermarkTimeUnit; + return this; + } + + public Builder resolveConfig( + Table table, Map readOptions, ReadableConfig readableConfig) { + FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); + + return this.useSnapshotId(flinkReadConf.snapshotId()) + .useTag(flinkReadConf.tag()) + .useBranch(flinkReadConf.branch()) + .startTag(flinkReadConf.startTag()) + .endTag(flinkReadConf.endTag()) + .caseSensitive(flinkReadConf.caseSensitive()) + .asOfTimestamp(flinkReadConf.asOfTimestamp()) + .startingStrategy(flinkReadConf.startingStrategy()) + .startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp()) + .startSnapshotId(flinkReadConf.startSnapshotId()) + .endSnapshotId(flinkReadConf.endSnapshotId()) + .splitSize(flinkReadConf.splitSize()) + .splitLookback(flinkReadConf.splitLookback()) + .splitOpenFileCost(flinkReadConf.splitFileOpenCost()) + .streaming(flinkReadConf.streaming()) + .monitorInterval(flinkReadConf.monitorInterval()) + .nameMapping(flinkReadConf.nameMapping()) + .limit(flinkReadConf.limit()) + .planParallelism(flinkReadConf.workerPoolSize()) + .includeColumnStats(flinkReadConf.includeColumnStats()) + .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) + .maxAllowedPlanningFailures(maxAllowedPlanningFailures) + .watermarkColumn(flinkReadConf.watermarkColumn()) + .watermarkColumnTimeUnit(flinkReadConf.watermarkColumnTimeUnit()); + } + + public ScanContext build() { + return new ScanContext( + caseSensitive, + snapshotId, + startingStrategy, + startSnapshotTimestamp, + startSnapshotId, + endSnapshotId, + asOfTimestamp, + splitSize, + splitLookback, + splitOpenFileCost, + isStreaming, + monitorInterval, + nameMapping, + projectedSchema, + filters, + limit, + includeColumnStats, + includeStatsForColumns, + exposeLocality, + planParallelism, + maxPlanningSnapshotCount, + maxAllowedPlanningFailures, + watermarkColumn, + watermarkColumnTimeUnit, + branch, + tag, + startTag, + endTag); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java new file mode 100644 index 000000000000..7c3a69dbc141 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.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.flink.source; + +import java.util.function.Supplier; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.api.config.ExecutionConfigOptions; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.hadoop.Util; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class SourceUtil { + private SourceUtil() {} + + static boolean isLocalityEnabled( + Table table, ReadableConfig readableConfig, Boolean exposeLocality) { + Boolean localityEnabled = + exposeLocality != null + ? exposeLocality + : readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO); + + if (localityEnabled != null && !localityEnabled) { + return false; + } + + return Util.mayHaveBlockLocations(table.io(), table.location()); + } + + /** + * Infer source parallelism. + * + * @param readableConfig Flink config. + * @param splitCountProvider Split count supplier. As the computation may involve expensive split + * discover, lazy evaluation is performed if inferring parallelism is enabled. + * @param limitCount limited output count. + */ + static int inferParallelism( + ReadableConfig readableConfig, long limitCount, Supplier splitCountProvider) { + int parallelism = + readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); + if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { + int maxInferParallelism = + readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); + Preconditions.checkState( + maxInferParallelism >= 1, + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() + + " cannot be less than 1"); + parallelism = Math.min(splitCountProvider.get(), maxInferParallelism); + } + + if (limitCount > 0) { + int limit = limitCount >= Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limitCount; + parallelism = Math.min(parallelism, limit); + } + + // parallelism must be positive. + parallelism = Math.max(1, parallelism); + return parallelism; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java new file mode 100644 index 000000000000..a07613aee59b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java @@ -0,0 +1,269 @@ +/* + * 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.io.UncheckedIOException; +import java.util.List; +import java.util.concurrent.ExecutorService; +import org.apache.flink.api.common.functions.RuntimeContext; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.api.common.typeutils.base.LongSerializer; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; +import org.apache.flink.streaming.api.functions.source.RichSourceFunction; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This is the single (non-parallel) monitoring task which takes a {@link FlinkInputFormat}, it is + * responsible for: + * + *

      + *
    1. Monitoring snapshots of the Iceberg table. + *
    2. Creating the {@link FlinkInputSplit splits} corresponding to the incremental files + *
    3. Assigning them to downstream tasks for further processing. + *
    + * + *

    The splits to be read are forwarded to the downstream {@link StreamingReaderOperator} which + * can have parallelism greater than one. + */ +public class StreamingMonitorFunction extends RichSourceFunction + implements CheckpointedFunction { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingMonitorFunction.class); + + private static final long INIT_LAST_SNAPSHOT_ID = -1L; + + private final TableLoader tableLoader; + private final ScanContext scanContext; + + private volatile boolean isRunning = true; + + // The checkpoint thread is not the same thread that running the function for SourceStreamTask + // now. It's necessary to + // mark this as volatile. + private volatile long lastSnapshotId = INIT_LAST_SNAPSHOT_ID; + + private transient SourceContext sourceContext; + private transient Table table; + private transient ListState lastSnapshotIdState; + private transient ExecutorService workerPool; + + public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext) { + Preconditions.checkArgument( + scanContext.snapshotId() == null, "Cannot set snapshot-id option for streaming reader"); + Preconditions.checkArgument( + scanContext.asOfTimestamp() == null, + "Cannot set as-of-timestamp option for streaming reader"); + Preconditions.checkArgument( + scanContext.endSnapshotId() == null, + "Cannot set end-snapshot-id option for streaming reader"); + Preconditions.checkArgument( + scanContext.endTag() == null, "Cannot set end-tag option for streaming reader"); + Preconditions.checkArgument( + scanContext.maxPlanningSnapshotCount() > 0, + "The max-planning-snapshot-count must be greater than zero"); + this.tableLoader = tableLoader; + this.scanContext = scanContext; + } + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + + final RuntimeContext runtimeContext = getRuntimeContext(); + ValidationException.check( + runtimeContext instanceof StreamingRuntimeContext, + "context should be instance of StreamingRuntimeContext"); + final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + // Load iceberg table from table loader. + tableLoader.open(); + table = tableLoader.loadTable(); + + // Initialize the flink state for last snapshot id. + lastSnapshotIdState = + context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("snapshot-id-state", LongSerializer.INSTANCE)); + + // Restore the last-snapshot-id from flink's state if possible. + if (context.isRestored()) { + LOG.info("Restoring state for the {}.", getClass().getSimpleName()); + lastSnapshotId = lastSnapshotIdState.get().iterator().next(); + } else if (scanContext.startTag() != null || scanContext.startSnapshotId() != null) { + Preconditions.checkArgument( + !(scanContext.startTag() != null && scanContext.startSnapshotId() != null), + "START_SNAPSHOT_ID and START_TAG cannot both be set."); + Preconditions.checkNotNull( + table.currentSnapshot(), "Don't have any available snapshot in table."); + + long startSnapshotId; + if (scanContext.startTag() != null) { + Preconditions.checkArgument( + table.snapshot(scanContext.startTag()) != null, + "Cannot find snapshot with tag %s in table.", + scanContext.startTag()); + startSnapshotId = table.snapshot(scanContext.startTag()).snapshotId(); + } else { + startSnapshotId = scanContext.startSnapshotId(); + } + + long currentSnapshotId = table.currentSnapshot().snapshotId(); + Preconditions.checkState( + SnapshotUtil.isAncestorOf(table, currentSnapshotId, startSnapshotId), + "The option start-snapshot-id %s is not an ancestor of the current snapshot.", + startSnapshotId); + + lastSnapshotId = startSnapshotId; + } + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + lastSnapshotIdState.clear(); + lastSnapshotIdState.add(lastSnapshotId); + } + + @Override + public void run(SourceContext ctx) throws Exception { + this.sourceContext = ctx; + while (isRunning) { + monitorAndForwardSplits(); + Thread.sleep(scanContext.monitorInterval().toMillis()); + } + } + + private long toSnapshotIdInclusive( + long lastConsumedSnapshotId, long currentSnapshotId, int maxPlanningSnapshotCount) { + List snapshotIds = + SnapshotUtil.snapshotIdsBetween(table, lastConsumedSnapshotId, currentSnapshotId); + if (snapshotIds.size() <= maxPlanningSnapshotCount) { + return currentSnapshotId; + } else { + // It uses reverted index since snapshotIdsBetween returns Ids that are ordered by committed + // time descending. + return snapshotIds.get(snapshotIds.size() - maxPlanningSnapshotCount); + } + } + + @VisibleForTesting + void sourceContext(SourceContext ctx) { + this.sourceContext = ctx; + } + + @VisibleForTesting + void monitorAndForwardSplits() { + // Refresh the table to get the latest committed snapshot. + table.refresh(); + + Snapshot snapshot = + scanContext.branch() != null + ? table.snapshot(scanContext.branch()) + : table.currentSnapshot(); + if (snapshot != null && snapshot.snapshotId() != lastSnapshotId) { + long snapshotId = snapshot.snapshotId(); + + ScanContext newScanContext; + if (lastSnapshotId == INIT_LAST_SNAPSHOT_ID) { + newScanContext = scanContext.copyWithSnapshotId(snapshotId); + } else { + snapshotId = + toSnapshotIdInclusive( + lastSnapshotId, snapshotId, scanContext.maxPlanningSnapshotCount()); + newScanContext = scanContext.copyWithAppendsBetween(lastSnapshotId, snapshotId); + } + + LOG.debug( + "Start discovering splits from {} (exclusive) to {} (inclusive)", + lastSnapshotId, + snapshotId); + long start = System.currentTimeMillis(); + FlinkInputSplit[] splits = + FlinkSplitPlanner.planInputSplits(table, newScanContext, workerPool); + LOG.debug( + "Discovered {} splits, time elapsed {}ms", + splits.length, + System.currentTimeMillis() - start); + + // only need to hold the checkpoint lock when emitting the splits and updating lastSnapshotId + start = System.currentTimeMillis(); + synchronized (sourceContext.getCheckpointLock()) { + for (FlinkInputSplit split : splits) { + sourceContext.collect(split); + } + + lastSnapshotId = snapshotId; + } + LOG.debug( + "Forwarded {} splits, time elapsed {}ms", + splits.length, + System.currentTimeMillis() - start); + } + } + + @Override + public void cancel() { + // this is to cover the case where cancel() is called before the run() + if (sourceContext != null) { + synchronized (sourceContext.getCheckpointLock()) { + isRunning = false; + } + } else { + isRunning = false; + } + + // Release all the resources here. + if (tableLoader != null) { + try { + tableLoader.close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + } + + @Override + public void close() { + cancel(); + + if (workerPool != null) { + workerPool.shutdown(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java new file mode 100644 index 000000000000..ee6f7b63988d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java @@ -0,0 +1,246 @@ +/* + * 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.util.Queue; +import org.apache.flink.api.common.operators.MailboxExecutor; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.ListStateDescriptor; +import org.apache.flink.runtime.state.JavaSerializer; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateSnapshotContext; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.api.operators.StreamSourceContexts; +import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The operator that reads the {@link FlinkInputSplit splits} received from the preceding {@link + * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} which has a + * parallelism of 1, this operator can have multiple parallelism. + * + *

    As soon as a split descriptor is received, it is put in a queue, and use {@link + * MailboxExecutor} read the actual data of the split. This architecture allows the separation of + * the reading thread from the one split processing the checkpoint barriers, thus removing any + * potential back-pressure. + */ +public class StreamingReaderOperator extends AbstractStreamOperator + implements OneInputStreamOperator { + + private static final Logger LOG = LoggerFactory.getLogger(StreamingReaderOperator.class); + + // It's the same thread that is running this operator and checkpoint actions. we use this executor + // to schedule only + // one split for future reading, so that a new checkpoint could be triggered without blocking long + // time for exhausting + // all scheduled splits. + private final MailboxExecutor executor; + private FlinkInputFormat format; + + private transient SourceFunction.SourceContext sourceContext; + + private transient ListState inputSplitsState; + private transient Queue splits; + + // Splits are read by the same thread that calls processElement. Each read task is submitted to + // that thread by adding + // them to the executor. This state is used to ensure that only one read task is in that queue at + // a time, so that read + // tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this + // is set to RUNNING. + // When there are no more files to read, this will be set to IDLE. + private transient SplitState currentSplitState; + + private StreamingReaderOperator( + FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { + this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); + this.processingTimeService = timeService; + this.executor = + Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); + } + + @Override + public void initializeState(StateInitializationContext context) throws Exception { + super.initializeState(context); + + // TODO Replace Java serialization with Avro approach to keep state compatibility. + // See issue: https://github.com/apache/iceberg/issues/1698 + inputSplitsState = + context + .getOperatorStateStore() + .getListState(new ListStateDescriptor<>("splits", new JavaSerializer<>())); + + // Initialize the current split state to IDLE. + currentSplitState = SplitState.IDLE; + + // Recover splits state from flink state backend if possible. + splits = Lists.newLinkedList(); + if (context.isRestored()) { + int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); + LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); + + for (FlinkInputSplit split : inputSplitsState.get()) { + splits.add(split); + } + } + + this.sourceContext = + StreamSourceContexts.getSourceContext( + getOperatorConfig().getTimeCharacteristic(), + getProcessingTimeService(), + new Object(), // no actual locking needed + output, + getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), + -1, + true); + + // Enqueue to process the recovered input splits. + enqueueProcessSplits(); + } + + @Override + public void snapshotState(StateSnapshotContext context) throws Exception { + super.snapshotState(context); + + inputSplitsState.clear(); + inputSplitsState.addAll(Lists.newArrayList(splits)); + } + + @Override + public void processElement(StreamRecord element) { + splits.add(element.getValue()); + enqueueProcessSplits(); + } + + private void enqueueProcessSplits() { + if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) { + currentSplitState = SplitState.RUNNING; + executor.execute(this::processSplits, this.getClass().getSimpleName()); + } + } + + private void processSplits() throws IOException { + FlinkInputSplit split = splits.poll(); + if (split == null) { + currentSplitState = SplitState.IDLE; + return; + } + + format.open(split); + try { + RowData nextElement = null; + while (!format.reachedEnd()) { + nextElement = format.nextRecord(nextElement); + sourceContext.collect(nextElement); + } + } finally { + currentSplitState = SplitState.IDLE; + format.close(); + } + + // Re-schedule to process the next split. + enqueueProcessSplits(); + } + + @Override + public void processWatermark(Watermark mark) { + // we do nothing because we emit our own watermarks if needed. + } + + @Override + public void close() throws Exception { + super.close(); + + if (format != null) { + format.close(); + format.closeInputFormat(); + format = null; + } + + sourceContext = null; + } + + @Override + public void finish() throws Exception { + super.finish(); + output.close(); + if (sourceContext != null) { + sourceContext.emitWatermark(Watermark.MAX_WATERMARK); + sourceContext.close(); + sourceContext = null; + } + } + + static OneInputStreamOperatorFactory factory(FlinkInputFormat format) { + return new OperatorFactory(format); + } + + private enum SplitState { + IDLE, + RUNNING + } + + private static class OperatorFactory extends AbstractStreamOperatorFactory + implements YieldingOperatorFactory, + OneInputStreamOperatorFactory { + + private final FlinkInputFormat format; + + private transient MailboxExecutor mailboxExecutor; + + private OperatorFactory(FlinkInputFormat format) { + this.format = format; + } + + @Override + public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { + this.mailboxExecutor = mailboxExecutor; + } + + @SuppressWarnings("unchecked") + @Override + public > O createStreamOperator( + StreamOperatorParameters parameters) { + StreamingReaderOperator operator = + new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); + operator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + return (O) operator; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return StreamingReaderOperator.class; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java new file mode 100644 index 000000000000..11707bf82a0f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java @@ -0,0 +1,54 @@ +/* + * 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; + +/** Starting strategy for streaming execution. */ +public enum StreamingStartingStrategy { + /** + * Do a regular table scan then switch to the incremental mode. + * + *

    The incremental mode starts from the current snapshot exclusive. + */ + TABLE_SCAN_THEN_INCREMENTAL, + + /** + * Start incremental mode from the latest snapshot inclusive. + * + *

    If it is an empty map, all future append snapshots should be discovered. + */ + INCREMENTAL_FROM_LATEST_SNAPSHOT, + + /** + * Start incremental mode from the earliest snapshot inclusive. + * + *

    If it is an empty map, all future append snapshots should be discovered. + */ + INCREMENTAL_FROM_EARLIEST_SNAPSHOT, + + /** Start incremental mode from a snapshot with a specific id inclusive. */ + INCREMENTAL_FROM_SNAPSHOT_ID, + + /** + * Start incremental mode from a snapshot with a specific timestamp inclusive. + * + *

    If the timestamp is between two snapshots, it should start from the snapshot after the + * timestamp. + */ + INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java new file mode 100644 index 000000000000..e7447d08c985 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java @@ -0,0 +1,119 @@ +/* + * 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.assigner; + +import java.util.ArrayDeque; +import java.util.Collection; +import java.util.PriorityQueue; +import java.util.Queue; +import java.util.concurrent.CompletableFuture; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Since all methods are called in the source coordinator thread by enumerator, there is no need for + * locking. + */ +@Internal +public class DefaultSplitAssigner implements SplitAssigner { + + private final Queue pendingSplits; + private CompletableFuture availableFuture; + + public DefaultSplitAssigner(SerializableComparator comparator) { + this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); + } + + public DefaultSplitAssigner( + SerializableComparator comparator, + Collection assignerState) { + this(comparator); + // Because default assigner only tracks unassigned splits, + // there is no need to filter splits based on status (unassigned) here. + assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); + } + + @Override + public synchronized GetSplitResult getNext(@Nullable String hostname) { + if (pendingSplits.isEmpty()) { + return GetSplitResult.unavailable(); + } else { + IcebergSourceSplit split = pendingSplits.poll(); + return GetSplitResult.forSplit(split); + } + } + + @Override + public void onDiscoveredSplits(Collection splits) { + addSplits(splits); + } + + @Override + public void onUnassignedSplits(Collection splits) { + addSplits(splits); + } + + private synchronized void addSplits(Collection splits) { + if (!splits.isEmpty()) { + pendingSplits.addAll(splits); + // only complete pending future if new splits are discovered + completeAvailableFuturesIfNeeded(); + } + } + + /** Simple assigner only tracks unassigned splits */ + @Override + public synchronized Collection state() { + return pendingSplits.stream() + .map(split -> new IcebergSourceSplitState(split, IcebergSourceSplitStatus.UNASSIGNED)) + .collect(Collectors.toList()); + } + + @Override + public synchronized CompletableFuture isAvailable() { + if (availableFuture == null) { + availableFuture = new CompletableFuture<>(); + } + return availableFuture; + } + + @Override + public synchronized int pendingSplitCount() { + return pendingSplits.size(); + } + + @Override + public long pendingRecords() { + return pendingSplits.stream() + .map(split -> split.task().estimatedRowsCount()) + .reduce(0L, Long::sum); + } + + private synchronized void completeAvailableFuturesIfNeeded() { + if (availableFuture != null && !pendingSplits.isEmpty()) { + availableFuture.complete(null); + } + availableFuture = null; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java new file mode 100644 index 000000000000..72deaeb890f3 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.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.flink.source.assigner; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +public class GetSplitResult { + + public enum Status { + AVAILABLE, + + /** + * There are pending splits. But they can't be assigned due to constraints (like event time + * alignment) + */ + CONSTRAINED, + + /** Assigner doesn't have pending splits. */ + UNAVAILABLE + } + + private final Status status; + private final IcebergSourceSplit split; + + private GetSplitResult(Status status) { + this.status = status; + this.split = null; + } + + private GetSplitResult(IcebergSourceSplit split) { + Preconditions.checkNotNull(split, "Split cannot be null"); + this.status = Status.AVAILABLE; + this.split = split; + } + + public Status status() { + return status; + } + + public IcebergSourceSplit split() { + return split; + } + + private static final GetSplitResult UNAVAILABLE = new GetSplitResult(Status.UNAVAILABLE); + private static final GetSplitResult CONSTRAINED = new GetSplitResult(Status.CONSTRAINED); + + public static GetSplitResult unavailable() { + return UNAVAILABLE; + } + + public static GetSplitResult constrained() { + return CONSTRAINED; + } + + public static GetSplitResult forSplit(IcebergSourceSplit split) { + return new GetSplitResult(split); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java new file mode 100644 index 000000000000..e58478897aef --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.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.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.SerializableComparator; + +/** + * Create default assigner with a comparator that hands out splits where the order of the splits + * will be defined by the {@link SerializableComparator}. + */ +public class OrderedSplitAssignerFactory implements SplitAssignerFactory { + private final SerializableComparator comparator; + + public OrderedSplitAssignerFactory(SerializableComparator comparator) { + this.comparator = comparator; + } + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(comparator); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(comparator, assignerState); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java new file mode 100644 index 000000000000..a2e2ff364d46 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.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.flink.source.assigner; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** Create simple assigner that hands out splits without any guarantee in order or locality. */ +public class SimpleSplitAssignerFactory implements SplitAssignerFactory { + public SimpleSplitAssignerFactory() {} + + @Override + public SplitAssigner createAssigner() { + return new DefaultSplitAssigner(null); + } + + @Override + public SplitAssigner createAssigner(Collection assignerState) { + return new DefaultSplitAssigner(null, assignerState); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java new file mode 100644 index 000000000000..dae7c8cca70c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.assigner; + +import java.io.Closeable; +import java.util.Collection; +import java.util.concurrent.CompletableFuture; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** + * SplitAssigner interface is extracted out as a separate component so that we can plug in different + * split assignment strategy for different requirements. E.g. + * + *

      + *
    • Simple assigner with no ordering guarantee or locality aware optimization. + *
    • Locality aware assigner that prefer splits that are local. + *
    • Snapshot aware assigner that assign splits based on the order they are committed. + *
    • Event time alignment assigner that assign splits satisfying certain time ordering within a + * single source or across sources. + *
    + * + *

    Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from + * the coordinator thread. But enumerator may call the {@link SplitAssigner#pendingSplitCount()} + * from the I/O threads. + */ +public interface SplitAssigner extends Closeable { + + /** + * Some assigners may need to start background threads or perform other activity such as + * registering as listeners to updates from other event sources e.g., watermark tracker. + */ + default void start() {} + + /** + * Some assigners may need to perform certain actions when their corresponding enumerators are + * closed + */ + @Override + default void close() {} + + /** + * Request a new split from the assigner when enumerator trying to assign splits to awaiting + * readers. + * + *

    If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should + * call {@link SplitAssigner#onUnassignedSplits} to return the split. + */ + GetSplitResult getNext(@Nullable String hostname); + + /** Add new splits discovered by enumerator */ + void onDiscoveredSplits(Collection splits); + + /** Forward addSplitsBack event (for failed reader) to assigner */ + void onUnassignedSplits(Collection splits); + + /** + * Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon + * completed splits + */ + default void onCompletedSplits(Collection completedSplitIds) {} + + /** + * Get assigner state for checkpointing. This is a super-set API that works for all currently + * imagined assigners. + */ + Collection state(); + + /** + * Enumerator can get a notification via CompletableFuture when the assigner has more splits + * available later. Enumerator should schedule assignment in the thenAccept action of the future. + * + *

    Assigner will return the same future if this method is called again before the previous + * future is completed. + * + *

    The future can be completed from other thread, e.g. the coordinator thread from another + * thread for event time alignment. + * + *

    If enumerator need to trigger action upon the future completion, it may want to run it in + * the coordinator thread using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}. + */ + CompletableFuture isAvailable(); + + /** + * Return the number of pending splits that haven't been assigned yet. + * + *

    The enumerator can poll this API to publish a metric on the number of pending splits. + * + *

    The enumerator can also use this information to throttle split discovery for streaming read. + * If there are already many pending splits tracked by the assigner, it is undesirable to discover + * more splits and track them in the assigner. That will increase the memory footprint and + * enumerator checkpoint size. + * + *

    Throttling works better together with {@link ScanContext#maxPlanningSnapshotCount()}. + * Otherwise, the next split discovery after throttling will just discover all non-enumerated + * snapshots and splits, which defeats the purpose of throttling. + */ + int pendingSplitCount(); + + /** + * Return the number of pending records, which can act as a measure of the source lag. This value + * could be an estimation if the exact number of records cannot be accurately computed. + */ + long pendingRecords(); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java new file mode 100644 index 000000000000..6e02a556ffcd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java @@ -0,0 +1,30 @@ +/* + * 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.assigner; + +import java.io.Serializable; +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +public interface SplitAssignerFactory extends Serializable { + + SplitAssigner createAssigner(); + + SplitAssigner createAssigner(Collection assignerState); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java new file mode 100644 index 000000000000..03ba67a554f9 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.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.flink.source.assigner; + +import org.apache.flink.annotation.Internal; + +@Internal +public enum SplitAssignerType { + SIMPLE { + @Override + public SplitAssignerFactory factory() { + return new SimpleSplitAssignerFactory(); + } + }; + + public abstract SplitAssignerFactory factory(); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java new file mode 100644 index 000000000000..280a126a46ce --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -0,0 +1,184 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.flink.api.connector.source.SourceEvent; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; +import org.apache.iceberg.flink.source.assigner.GetSplitResult; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +abstract class AbstractIcebergEnumerator + implements SplitEnumerator, + SupportsHandleExecutionAttemptSourceEvent { + private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); + + private final SplitEnumeratorContext enumeratorContext; + private final SplitAssigner assigner; + private final Map readersAwaitingSplit; + private final AtomicReference> availableFuture; + + AbstractIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { + this.enumeratorContext = enumeratorContext; + this.assigner = assigner; + this.readersAwaitingSplit = new LinkedHashMap<>(); + this.availableFuture = new AtomicReference<>(); + this.enumeratorContext + .metricGroup() + // This number may not capture the entire backlog due to split discovery throttling to avoid + // excessive memory footprint. Some pending splits may not have been discovered yet. + .setUnassignedSplitsGauge(() -> Long.valueOf(assigner.pendingSplitCount())); + this.enumeratorContext.metricGroup().gauge("pendingRecords", assigner::pendingRecords); + } + + @Override + public void start() { + assigner.start(); + } + + @Override + public void close() throws IOException { + assigner.close(); + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // Iceberg source uses custom split request event to piggyback finished split ids. + throw new UnsupportedOperationException( + String.format( + "Received invalid default split request event " + + "from subtask %d as Iceberg source uses custom split request event", + subtaskId)); + } + + @Override + public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { + if (sourceEvent instanceof SplitRequestEvent) { + SplitRequestEvent splitRequestEvent = (SplitRequestEvent) sourceEvent; + LOG.info("Received request split event from subtask {}", subtaskId); + assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds()); + readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname()); + assignSplits(); + } else { + throw new IllegalArgumentException( + String.format( + "Received unknown event from subtask %d: %s", + subtaskId, sourceEvent.getClass().getCanonicalName())); + } + } + + // Flink's SourceCoordinator already keeps track of subTask to splits mapping. + // It already takes care of re-assigning splits to speculated attempts as well. + @Override + public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { + handleSourceEvent(subTaskId, sourceEvent); + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); + assigner.onUnassignedSplits(splits); + assignSplits(); + } + + @Override + public void addReader(int subtaskId) { + LOG.info("Added reader: {}", subtaskId); + } + + private void assignSplits() { + LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size()); + Iterator> awaitingReader = + readersAwaitingSplit.entrySet().iterator(); + while (awaitingReader.hasNext()) { + Map.Entry nextAwaiting = awaitingReader.next(); + // if the reader that requested another split has failed in the meantime, remove + // it from the list of waiting readers + if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) { + awaitingReader.remove(); + continue; + } + + int awaitingSubtask = nextAwaiting.getKey(); + String hostname = nextAwaiting.getValue(); + GetSplitResult getResult = assigner.getNext(hostname); + if (getResult.status() == GetSplitResult.Status.AVAILABLE) { + LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split()); + enumeratorContext.assignSplit(getResult.split(), awaitingSubtask); + awaitingReader.remove(); + } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) { + getAvailableFutureIfNeeded(); + break; + } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) { + if (shouldWaitForMoreSplits()) { + getAvailableFutureIfNeeded(); + break; + } else { + LOG.info("No more splits available for subtask {}", awaitingSubtask); + enumeratorContext.signalNoMoreSplits(awaitingSubtask); + awaitingReader.remove(); + } + } else { + throw new IllegalArgumentException("Unsupported status: " + getResult.status()); + } + } + } + + /** return true if enumerator should wait for splits like in the continuous enumerator case */ + protected abstract boolean shouldWaitForMoreSplits(); + + private synchronized void getAvailableFutureIfNeeded() { + if (availableFuture.get() != null) { + return; + } + + CompletableFuture future = + assigner + .isAvailable() + .thenAccept( + ignore -> + // Must run assignSplits in coordinator thread + // because the future may be completed from other threads. + // E.g., in event time alignment assigner, + // watermark advancement from another source may + // cause the available future to be completed + enumeratorContext.runInCoordinatorThread( + () -> { + LOG.debug("Executing callback of assignSplits"); + availableFuture.set(null); + assignSplits(); + })); + availableFuture.set(future); + LOG.debug("Registered callback for future available splits"); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java new file mode 100644 index 000000000000..41863ffee60b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.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.flink.source.enumerator; + +import java.util.Collection; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class ContinuousEnumerationResult { + private final Collection splits; + private final IcebergEnumeratorPosition fromPosition; + private final IcebergEnumeratorPosition toPosition; + + /** + * @param splits should never be null. But it can be an empty collection + * @param fromPosition can be null + * @param toPosition should never be null. But it can have null snapshotId and snapshotTimestampMs + */ + ContinuousEnumerationResult( + Collection splits, + IcebergEnumeratorPosition fromPosition, + IcebergEnumeratorPosition toPosition) { + Preconditions.checkArgument(splits != null, "Invalid to splits collection: null"); + Preconditions.checkArgument(toPosition != null, "Invalid end position: null"); + this.splits = splits; + this.fromPosition = fromPosition; + this.toPosition = toPosition; + } + + public Collection splits() { + return splits; + } + + public IcebergEnumeratorPosition fromPosition() { + return fromPosition; + } + + public IcebergEnumeratorPosition toPosition() { + return toPosition; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java new file mode 100644 index 000000000000..c7021b9c6847 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.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.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collections; +import java.util.Objects; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.util.ElapsedTimeGauge; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { + + private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** + * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to + * control the total number of snapshots worth of splits tracked by assigner. + */ + private static final int ENUMERATION_SPLIT_COUNT_HISTORY_SIZE = 3; + + private final SplitEnumeratorContext enumeratorContext; + private final SplitAssigner assigner; + private final ScanContext scanContext; + private final ContinuousSplitPlanner splitPlanner; + + /** + * snapshotId for the last enumerated snapshot. next incremental enumeration should be based off + * this as the starting position. + */ + private final AtomicReference enumeratorPosition; + + /** Track enumeration result history for split discovery throttling. */ + private final EnumerationHistory enumerationHistory; + + /** Count the consecutive failures and throw exception if the max allowed failres are reached */ + private transient int consecutiveFailures = 0; + + private final ElapsedTimeGauge elapsedSecondsSinceLastSplitDiscovery; + + public ContinuousIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, + SplitAssigner assigner, + ScanContext scanContext, + ContinuousSplitPlanner splitPlanner, + @Nullable IcebergEnumeratorState enumState) { + super(enumeratorContext, assigner); + + this.enumeratorContext = enumeratorContext; + this.assigner = assigner; + this.scanContext = scanContext; + this.splitPlanner = splitPlanner; + this.enumeratorPosition = new AtomicReference<>(); + this.enumerationHistory = new EnumerationHistory(ENUMERATION_SPLIT_COUNT_HISTORY_SIZE); + this.elapsedSecondsSinceLastSplitDiscovery = new ElapsedTimeGauge(TimeUnit.SECONDS); + this.enumeratorContext + .metricGroup() + .gauge("elapsedSecondsSinceLastSplitDiscovery", elapsedSecondsSinceLastSplitDiscovery); + + if (enumState != null) { + this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); + this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); + } + } + + @Override + public void start() { + super.start(); + enumeratorContext.callAsync( + this::discoverSplits, + this::processDiscoveredSplits, + 0L, + scanContext.monitorInterval().toMillis()); + } + + @Override + public void close() throws IOException { + splitPlanner.close(); + super.close(); + } + + @Override + protected boolean shouldWaitForMoreSplits() { + return true; + } + + @Override + public IcebergEnumeratorState snapshotState(long checkpointId) { + return new IcebergEnumeratorState( + enumeratorPosition.get(), assigner.state(), enumerationHistory.snapshot()); + } + + /** This method is executed in an IO thread pool. */ + private ContinuousEnumerationResult discoverSplits() { + int pendingSplitCountFromAssigner = assigner.pendingSplitCount(); + if (enumerationHistory.shouldPauseSplitDiscovery(pendingSplitCountFromAssigner)) { + // If the assigner already has many pending splits, it is better to pause split discovery. + // Otherwise, eagerly discovering more splits will just increase assigner memory footprint + // and enumerator checkpoint state size. + LOG.info( + "Pause split discovery as the assigner already has too many pending splits: {}", + pendingSplitCountFromAssigner); + return new ContinuousEnumerationResult( + Collections.emptyList(), enumeratorPosition.get(), enumeratorPosition.get()); + } else { + return splitPlanner.planSplits(enumeratorPosition.get()); + } + } + + /** This method is executed in a single coordinator thread. */ + private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwable error) { + if (error == null) { + consecutiveFailures = 0; + if (!Objects.equals(result.fromPosition(), enumeratorPosition.get())) { + // Multiple discoverSplits() may be triggered with the same starting snapshot to the I/O + // thread pool. E.g., the splitDiscoveryInterval is very short (like 10 ms in some unit + // tests) or the thread pool is busy and multiple discovery actions are executed + // concurrently. Discovery result should only be accepted if the starting position + // matches the enumerator position (like compare-and-swap). + LOG.info( + "Skip {} discovered splits because the scan starting position doesn't match " + + "the current enumerator position: enumerator position = {}, scan starting position = {}", + result.splits().size(), + enumeratorPosition.get(), + result.fromPosition()); + } else { + elapsedSecondsSinceLastSplitDiscovery.refreshLastRecordedTime(); + // Sometimes, enumeration may yield no splits for a few reasons. + // - upstream paused or delayed streaming writes to the Iceberg table. + // - enumeration frequency is higher than the upstream write frequency. + if (!result.splits().isEmpty()) { + assigner.onDiscoveredSplits(result.splits()); + // EnumerationHistory makes throttling decision on split discovery + // based on the total number of splits discovered in the last a few cycles. + // Only update enumeration history when there are some discovered splits. + enumerationHistory.add(result.splits().size()); + LOG.info( + "Added {} splits discovered between ({}, {}] to the assigner", + result.splits().size(), + result.fromPosition(), + result.toPosition()); + } else { + LOG.info( + "No new splits discovered between ({}, {}]", + result.fromPosition(), + result.toPosition()); + } + // update the enumerator position even if there is no split discovered + // or the toPosition is empty (e.g. for empty table). + enumeratorPosition.set(result.toPosition()); + LOG.info("Update enumerator position to {}", result.toPosition()); + } + } else { + consecutiveFailures++; + if (scanContext.maxAllowedPlanningFailures() < 0 + || consecutiveFailures <= scanContext.maxAllowedPlanningFailures()) { + LOG.error("Failed to discover new splits", error); + } else { + throw new RuntimeException("Failed to discover new splits", error); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java new file mode 100644 index 000000000000..2a1325178873 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java @@ -0,0 +1,30 @@ +/* + * 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.enumerator; + +import java.io.Closeable; +import org.apache.flink.annotation.Internal; + +/** This interface is introduced so that we can plug in different split planner for unit test */ +@Internal +public interface ContinuousSplitPlanner extends Closeable { + + /** Discover the files appended between {@code lastPosition} and current table snapshot */ + ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java new file mode 100644 index 000000000000..fef4ec45ed8a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.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.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ExecutorService; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.FlinkSplitPlanner; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { + private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); + + private final Table table; + private final ScanContext scanContext; + private final boolean isSharedPool; + private final ExecutorService workerPool; + private final TableLoader tableLoader; + + /** + * @param tableLoader A cloned tableLoader. + * @param threadName thread name prefix for worker pool to run the split planning. If null, a + * shared worker pool will be used. + */ + public ContinuousSplitPlannerImpl( + TableLoader tableLoader, ScanContext scanContext, String threadName) { + this.tableLoader = tableLoader.clone(); + this.tableLoader.open(); + this.table = this.tableLoader.loadTable(); + this.scanContext = scanContext; + this.isSharedPool = threadName == null; + this.workerPool = + isSharedPool + ? ThreadPools.getWorkerPool() + : ThreadPools.newWorkerPool( + "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); + } + + @Override + public void close() throws IOException { + if (!isSharedPool) { + workerPool.shutdown(); + } + tableLoader.close(); + } + + @Override + public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { + table.refresh(); + if (lastPosition != null) { + return discoverIncrementalSplits(lastPosition); + } else { + return discoverInitialSplits(); + } + } + + private Snapshot toSnapshotInclusive( + Long lastConsumedSnapshotId, Snapshot currentSnapshot, int maxPlanningSnapshotCount) { + // snapshots are in reverse order (latest snapshot first) + List snapshots = + Lists.newArrayList( + SnapshotUtil.ancestorsBetween( + table, currentSnapshot.snapshotId(), lastConsumedSnapshotId)); + if (snapshots.size() <= maxPlanningSnapshotCount) { + return currentSnapshot; + } else { + // Because snapshots are in reverse order of commit history, this index returns + // the max allowed number of snapshots from the lastConsumedSnapshotId. + return snapshots.get(snapshots.size() - maxPlanningSnapshotCount); + } + } + + private ContinuousEnumerationResult discoverIncrementalSplits( + IcebergEnumeratorPosition lastPosition) { + Snapshot currentSnapshot = + scanContext.branch() != null + ? table.snapshot(scanContext.branch()) + : table.currentSnapshot(); + + if (currentSnapshot == null) { + // empty table + Preconditions.checkArgument( + lastPosition.snapshotId() == null, + "Invalid last enumerated position for an empty table: not null"); + LOG.info("Skip incremental scan because table is empty"); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else if (lastPosition.snapshotId() != null + && currentSnapshot.snapshotId() == lastPosition.snapshotId()) { + LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); + return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); + } else { + Long lastConsumedSnapshotId = lastPosition.snapshotId(); + Snapshot toSnapshotInclusive = + toSnapshotInclusive( + lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); + IcebergEnumeratorPosition newPosition = + IcebergEnumeratorPosition.of( + toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis()); + ScanContext incrementalScan = + scanContext.copyWithAppendsBetween( + lastPosition.snapshotId(), toSnapshotInclusive.snapshotId()); + List splits = + FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); + LOG.info( + "Discovered {} splits from incremental scan: " + + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", + splits.size(), + lastPosition, + newPosition); + return new ContinuousEnumerationResult(splits, lastPosition, newPosition); + } + } + + /** + * Discovery initial set of splits based on {@link StreamingStartingStrategy}. + *

  • {@link ContinuousEnumerationResult#splits()} should contain initial splits discovered from + * table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. For all other + * strategies, splits collection should be empty. + *
  • {@link ContinuousEnumerationResult#toPosition()} points to the starting position for the + * next incremental split discovery with exclusive behavior. Meaning files committed by the + * snapshot from the position in {@code ContinuousEnumerationResult} won't be included in the + * next incremental scan. + */ + private ContinuousEnumerationResult discoverInitialSplits() { + Optional startSnapshotOptional = startSnapshot(table, scanContext); + if (!startSnapshotOptional.isPresent()) { + return new ContinuousEnumerationResult( + Collections.emptyList(), null, IcebergEnumeratorPosition.empty()); + } + + Snapshot startSnapshot = startSnapshotOptional.get(); + LOG.info( + "Get starting snapshot id {} based on strategy {}", + startSnapshot.snapshotId(), + scanContext.streamingStartingStrategy()); + List splits; + IcebergEnumeratorPosition toPosition; + if (scanContext.streamingStartingStrategy() + == StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) { + // do a batch table scan first + splits = + FlinkSplitPlanner.planIcebergSourceSplits( + table, scanContext.copyWithSnapshotId(startSnapshot.snapshotId()), workerPool); + LOG.info( + "Discovered {} splits from initial batch table scan with snapshot Id {}", + splits.size(), + startSnapshot.snapshotId()); + // For TABLE_SCAN_THEN_INCREMENTAL, incremental mode starts exclusive from the startSnapshot + toPosition = + IcebergEnumeratorPosition.of(startSnapshot.snapshotId(), startSnapshot.timestampMillis()); + } else { + // For all other modes, starting snapshot should be consumed inclusively. + // Use parentId to achieve the inclusive behavior. It is fine if parentId is null. + splits = Collections.emptyList(); + Long parentSnapshotId = startSnapshot.parentId(); + if (parentSnapshotId != null) { + Snapshot parentSnapshot = table.snapshot(parentSnapshotId); + Long parentSnapshotTimestampMs = + parentSnapshot != null ? parentSnapshot.timestampMillis() : null; + toPosition = IcebergEnumeratorPosition.of(parentSnapshotId, parentSnapshotTimestampMs); + } else { + toPosition = IcebergEnumeratorPosition.empty(); + } + + LOG.info( + "Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", + startSnapshot.snapshotId(), + startSnapshot.timestampMillis()); + } + + return new ContinuousEnumerationResult(splits, null, toPosition); + } + + /** + * Calculate the starting snapshot based on the {@link StreamingStartingStrategy} defined in + * {@code ScanContext}. + * + *

    If the {@link StreamingStartingStrategy} is not {@link + * StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}, the start snapshot should be consumed + * inclusively. + */ + @VisibleForTesting + static Optional startSnapshot(Table table, ScanContext scanContext) { + switch (scanContext.streamingStartingStrategy()) { + case TABLE_SCAN_THEN_INCREMENTAL: + case INCREMENTAL_FROM_LATEST_SNAPSHOT: + return Optional.ofNullable(table.currentSnapshot()); + case INCREMENTAL_FROM_EARLIEST_SNAPSHOT: + return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); + case INCREMENTAL_FROM_SNAPSHOT_ID: + Snapshot matchedSnapshotById = table.snapshot(scanContext.startSnapshotId()); + Preconditions.checkArgument( + matchedSnapshotById != null, + "Start snapshot id not found in history: " + scanContext.startSnapshotId()); + return Optional.of(matchedSnapshotById); + case INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: + Snapshot matchedSnapshotByTimestamp = + SnapshotUtil.oldestAncestorAfter(table, scanContext.startSnapshotTimestamp()); + Preconditions.checkArgument( + matchedSnapshotByTimestamp != null, + "Cannot find a snapshot after: " + scanContext.startSnapshotTimestamp()); + return Optional.of(matchedSnapshotByTimestamp); + default: + throw new IllegalArgumentException( + "Unknown starting strategy: " + scanContext.streamingStartingStrategy()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java new file mode 100644 index 000000000000..ec56a9ecdac1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java @@ -0,0 +1,100 @@ +/* + * 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.enumerator; + +import java.util.Arrays; +import javax.annotation.concurrent.ThreadSafe; +import org.apache.flink.annotation.VisibleForTesting; + +/** + * This enumeration history is used for split discovery throttling. It tracks the discovered split + * count per every non-empty enumeration. + */ +@ThreadSafe +class EnumerationHistory { + + private final int[] history; + // int (2B) should be enough without overflow for enumeration history + private int count; + + EnumerationHistory(int maxHistorySize) { + this.history = new int[maxHistorySize]; + } + + synchronized void restore(int[] restoredHistory) { + int startingOffset = 0; + int restoreSize = restoredHistory.length; + + if (restoredHistory.length > history.length) { + // keep the newest history + startingOffset = restoredHistory.length - history.length; + // only restore the latest history up to maxHistorySize + restoreSize = history.length; + } + + System.arraycopy(restoredHistory, startingOffset, history, 0, restoreSize); + count = restoreSize; + } + + synchronized int[] snapshot() { + int len = history.length; + if (count > len) { + int[] copy = new int[len]; + // this is like a circular buffer + int indexForOldest = count % len; + System.arraycopy(history, indexForOldest, copy, 0, len - indexForOldest); + System.arraycopy(history, 0, copy, len - indexForOldest, indexForOldest); + return copy; + } else { + return Arrays.copyOfRange(history, 0, count); + } + } + + /** Add the split count from the last enumeration result. */ + synchronized void add(int splitCount) { + int pos = count % history.length; + history[pos] = splitCount; + count += 1; + } + + @VisibleForTesting + synchronized boolean hasFullHistory() { + return count >= history.length; + } + + /** + * Checks whether split discovery should be paused. + * + * @return true if split discovery should pause because assigner has too many splits already. + */ + synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { + if (count < history.length) { + // only check throttling when full history is obtained. + return false; + } else { + // if ScanContext#maxPlanningSnapshotCount() is 10, each split enumeration can + // discovery splits up to 10 snapshots. if maxHistorySize is 3, the max number of + // splits tracked in assigner shouldn't be more than 10 * (3 + 1) snapshots + // worth of splits. +1 because there could be another enumeration when the + // pending splits fall just below the 10 * 3. + int totalSplitCountFromRecentDiscovery = Arrays.stream(history).reduce(0, Integer::sum); + return pendingSplitCountFromAssigner >= totalSplitCountFromRecentDiscovery; + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java new file mode 100644 index 000000000000..96aba296f8cf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java @@ -0,0 +1,79 @@ +/* + * 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.enumerator; + +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.base.Objects; + +class IcebergEnumeratorPosition { + private final Long snapshotId; + // Track snapshot timestamp mainly for info logging + private final Long snapshotTimestampMs; + + static IcebergEnumeratorPosition empty() { + return new IcebergEnumeratorPosition(null, null); + } + + static IcebergEnumeratorPosition of(long snapshotId, Long snapshotTimestampMs) { + return new IcebergEnumeratorPosition(snapshotId, snapshotTimestampMs); + } + + private IcebergEnumeratorPosition(Long snapshotId, Long snapshotTimestampMs) { + this.snapshotId = snapshotId; + this.snapshotTimestampMs = snapshotTimestampMs; + } + + boolean isEmpty() { + return snapshotId == null; + } + + Long snapshotId() { + return snapshotId; + } + + Long snapshotTimestampMs() { + return snapshotTimestampMs; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("snapshotId", snapshotId) + .add("snapshotTimestampMs", snapshotTimestampMs) + .toString(); + } + + @Override + public int hashCode() { + return Objects.hashCode(snapshotId, snapshotTimestampMs); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + IcebergEnumeratorPosition other = (IcebergEnumeratorPosition) o; + return Objects.equal(snapshotId, other.snapshotId()) + && Objects.equal(snapshotTimestampMs, other.snapshotTimestampMs()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java new file mode 100644 index 000000000000..1c63807361c5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java @@ -0,0 +1,90 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +class IcebergEnumeratorPositionSerializer + implements SimpleVersionedSerializer { + + public static final IcebergEnumeratorPositionSerializer INSTANCE = + new IcebergEnumeratorPositionSerializer(); + + private static final int VERSION = 1; + + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(128)); + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergEnumeratorPosition position) throws IOException { + return serializeV1(position); + } + + @Override + public IcebergEnumeratorPosition deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + default: + throw new IOException("Unknown version: " + version); + } + } + + private byte[] serializeV1(IcebergEnumeratorPosition position) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + out.writeBoolean(position.snapshotId() != null); + if (position.snapshotId() != null) { + out.writeLong(position.snapshotId()); + } + out.writeBoolean(position.snapshotTimestampMs() != null); + if (position.snapshotTimestampMs() != null) { + out.writeLong(position.snapshotTimestampMs()); + } + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + private IcebergEnumeratorPosition deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + Long snapshotId = null; + if (in.readBoolean()) { + snapshotId = in.readLong(); + } + + Long snapshotTimestampMs = null; + if (in.readBoolean()) { + snapshotTimestampMs = in.readLong(); + } + + if (snapshotId != null) { + return IcebergEnumeratorPosition.of(snapshotId, snapshotTimestampMs); + } else { + return IcebergEnumeratorPosition.empty(); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java new file mode 100644 index 000000000000..26fbad46c128 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.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.flink.source.enumerator; + +import java.io.Serializable; +import java.util.Collection; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; + +/** Enumerator state for checkpointing */ +@Internal +public class IcebergEnumeratorState implements Serializable { + @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; + private final Collection pendingSplits; + private final int[] enumerationSplitCountHistory; + + public IcebergEnumeratorState(Collection pendingSplits) { + this(null, pendingSplits); + } + + public IcebergEnumeratorState( + @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, + Collection pendingSplits) { + this(lastEnumeratedPosition, pendingSplits, new int[0]); + } + + public IcebergEnumeratorState( + @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, + Collection pendingSplits, + int[] enumerationSplitCountHistory) { + this.lastEnumeratedPosition = lastEnumeratedPosition; + this.pendingSplits = pendingSplits; + this.enumerationSplitCountHistory = enumerationSplitCountHistory; + } + + @Nullable + public IcebergEnumeratorPosition lastEnumeratedPosition() { + return lastEnumeratedPosition; + } + + public Collection pendingSplits() { + return pendingSplits; + } + + public int[] enumerationSplitCountHistory() { + return enumerationSplitCountHistory; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java new file mode 100644 index 000000000000..f76f8a69ff0e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.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.flink.source.enumerator; + +import java.io.IOException; +import java.util.Collection; +import org.apache.flink.annotation.Internal; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergEnumeratorStateSerializer + implements SimpleVersionedSerializer { + + private static final int VERSION = 2; + + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); + + private final IcebergEnumeratorPositionSerializer positionSerializer = + IcebergEnumeratorPositionSerializer.INSTANCE; + private final IcebergSourceSplitSerializer splitSerializer; + + public IcebergEnumeratorStateSerializer(boolean caseSensitive) { + this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergEnumeratorState enumState) throws IOException { + return serializeV2(enumState); + } + + @Override + public IcebergEnumeratorState deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return deserializeV1(serialized); + case 2: + return deserializeV2(serialized); + default: + throw new IOException("Unknown version: " + version); + } + } + + @VisibleForTesting + byte[] serializeV1(IcebergEnumeratorState enumState) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); + serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + @VisibleForTesting + IcebergEnumeratorState deserializeV1(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + IcebergEnumeratorPosition enumeratorPosition = + deserializeEnumeratorPosition(in, positionSerializer); + Collection pendingSplits = + deserializePendingSplits(in, splitSerializer); + return new IcebergEnumeratorState(enumeratorPosition, pendingSplits); + } + + @VisibleForTesting + byte[] serializeV2(IcebergEnumeratorState enumState) throws IOException { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); + serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); + serializeEnumerationSplitCountHistory(out, enumState.enumerationSplitCountHistory()); + byte[] result = out.getCopyOfBuffer(); + out.clear(); + return result; + } + + @VisibleForTesting + IcebergEnumeratorState deserializeV2(byte[] serialized) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + IcebergEnumeratorPosition enumeratorPosition = + deserializeEnumeratorPosition(in, positionSerializer); + Collection pendingSplits = + deserializePendingSplits(in, splitSerializer); + int[] enumerationSplitCountHistory = deserializeEnumerationSplitCountHistory(in); + return new IcebergEnumeratorState( + enumeratorPosition, pendingSplits, enumerationSplitCountHistory); + } + + private static void serializeEnumeratorPosition( + DataOutputSerializer out, + IcebergEnumeratorPosition enumeratorPosition, + IcebergEnumeratorPositionSerializer positionSerializer) + throws IOException { + out.writeBoolean(enumeratorPosition != null); + if (enumeratorPosition != null) { + out.writeInt(positionSerializer.getVersion()); + byte[] positionBytes = positionSerializer.serialize(enumeratorPosition); + out.writeInt(positionBytes.length); + out.write(positionBytes); + } + } + + private static IcebergEnumeratorPosition deserializeEnumeratorPosition( + DataInputDeserializer in, IcebergEnumeratorPositionSerializer positionSerializer) + throws IOException { + IcebergEnumeratorPosition enumeratorPosition = null; + if (in.readBoolean()) { + int version = in.readInt(); + byte[] positionBytes = new byte[in.readInt()]; + in.read(positionBytes); + enumeratorPosition = positionSerializer.deserialize(version, positionBytes); + } + return enumeratorPosition; + } + + private static void serializePendingSplits( + DataOutputSerializer out, + Collection pendingSplits, + IcebergSourceSplitSerializer splitSerializer) + throws IOException { + out.writeInt(splitSerializer.getVersion()); + out.writeInt(pendingSplits.size()); + for (IcebergSourceSplitState splitState : pendingSplits) { + byte[] splitBytes = splitSerializer.serialize(splitState.split()); + out.writeInt(splitBytes.length); + out.write(splitBytes); + out.writeUTF(splitState.status().name()); + } + } + + private static Collection deserializePendingSplits( + DataInputDeserializer in, IcebergSourceSplitSerializer splitSerializer) throws IOException { + int splitSerializerVersion = in.readInt(); + int splitCount = in.readInt(); + Collection pendingSplits = Lists.newArrayListWithCapacity(splitCount); + for (int i = 0; i < splitCount; ++i) { + byte[] splitBytes = new byte[in.readInt()]; + in.read(splitBytes); + IcebergSourceSplit split = splitSerializer.deserialize(splitSerializerVersion, splitBytes); + String statusName = in.readUTF(); + pendingSplits.add( + new IcebergSourceSplitState(split, IcebergSourceSplitStatus.valueOf(statusName))); + } + return pendingSplits; + } + + private static void serializeEnumerationSplitCountHistory( + DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { + out.writeInt(enumerationSplitCountHistory.length); + for (int enumerationSplitCount : enumerationSplitCountHistory) { + out.writeInt(enumerationSplitCount); + } + } + + private static int[] deserializeEnumerationSplitCountHistory(DataInputDeserializer in) + throws IOException { + int historySize = in.readInt(); + int[] history = new int[historySize]; + if (historySize > 0) { + for (int i = 0; i < historySize; ++i) { + history[i] = in.readInt(); + } + } + + return history; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java new file mode 100644 index 000000000000..4e55ea5d5fd6 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java @@ -0,0 +1,51 @@ +/* + * 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.enumerator; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.iceberg.flink.source.assigner.SplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** One-time split enumeration at the start-up for batch execution */ +@Internal +public class StaticIcebergEnumerator extends AbstractIcebergEnumerator { + private final SplitAssigner assigner; + + public StaticIcebergEnumerator( + SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { + super(enumeratorContext, assigner); + this.assigner = assigner; + } + + @Override + public void start() { + super.start(); + } + + @Override + protected boolean shouldWaitForMoreSplits() { + return false; + } + + @Override + public IcebergEnumeratorState snapshotState(long checkpointId) { + return new IcebergEnumeratorState(null, assigner.state(), new int[0]); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java new file mode 100644 index 000000000000..7b94c364c976 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.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.flink.source.reader; + +import java.util.Collections; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * {@link RecordsWithSplitIds} is used to pass a batch of records from fetcher to source reader. + * Batching is to improve the efficiency for records handover. + * + *

    {@link RecordsWithSplitIds} interface can encapsulate batches from multiple splits. This is + * the case for Kafka source where fetchers can retrieve records from multiple Kafka partitions at + * the same time. + * + *

    For file-based sources like Iceberg, readers always read one split/file at a time. Hence, we + * will only have a batch of records for one split here. + * + *

    This class uses array to store a batch of records from the same file (with the same + * fileOffset). + */ +class ArrayBatchRecords implements RecordsWithSplitIds> { + @Nullable private String splitId; + @Nullable private final Pool.Recycler recycler; + @Nullable private final T[] records; + private final int numberOfRecords; + private final Set finishedSplits; + private final RecordAndPosition recordAndPosition; + + // point to current read position within the records array + private int position; + + private ArrayBatchRecords( + @Nullable String splitId, + @Nullable Pool.Recycler recycler, + @Nullable T[] records, + int numberOfRecords, + int fileOffset, + long startingRecordOffset, + Set finishedSplits) { + Preconditions.checkArgument(numberOfRecords >= 0, "numberOfRecords can't be negative"); + Preconditions.checkArgument(fileOffset >= 0, "fileOffset can't be negative"); + Preconditions.checkArgument(startingRecordOffset >= 0, "numberOfRecords can't be negative"); + + this.splitId = splitId; + this.recycler = recycler; + this.records = records; + this.numberOfRecords = numberOfRecords; + this.finishedSplits = + Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); + this.recordAndPosition = new RecordAndPosition<>(); + + recordAndPosition.set(null, fileOffset, startingRecordOffset); + this.position = 0; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = this.splitId; + // set the splitId to null to indicate no more splits + // this class only contains record for one split + this.splitId = null; + return nextSplit; + } + + @Nullable + @Override + public RecordAndPosition nextRecordFromSplit() { + if (position < numberOfRecords) { + recordAndPosition.record(records[position]); + position++; + return recordAndPosition; + } else { + return null; + } + } + + /** + * This method is called when all records from this batch has been emitted. If recycler is set, it + * should be called to return the records array back to pool. + */ + @Override + public void recycle() { + if (recycler != null) { + recycler.recycle(records); + } + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + @VisibleForTesting + T[] records() { + return records; + } + + @VisibleForTesting + int numberOfRecords() { + return numberOfRecords; + } + + /** + * Create a ArrayBatchRecords backed up an array with records from the same file + * + * @param splitId Iceberg source only read from one split a time. We never have multiple records + * from multiple splits. + * @param recycler Because {@link DataIterator} with {@link RowData} returns an iterator of reused + * RowData object, we need to clone RowData eagerly when constructing a batch of records. We + * can use object pool to reuse the RowData array object which can be expensive to create. + * This recycler can be provided to recycle the array object back to pool after read is + * exhausted. If the {@link DataIterator} returns an iterator of non-reused objects, we don't + * need to clone objects. It is cheap to just create the batch array. Hence, we don't need + * object pool and recycler can be set to null. + * @param records an array (maybe reused) holding a batch of records + * @param numberOfRecords actual number of records in the array + * @param fileOffset fileOffset for all records in this batch + * @param startingRecordOffset starting recordOffset + * @param record type + */ + public static ArrayBatchRecords forRecords( + String splitId, + Pool.Recycler recycler, + T[] records, + int numberOfRecords, + int fileOffset, + long startingRecordOffset) { + return new ArrayBatchRecords<>( + splitId, + recycler, + records, + numberOfRecords, + fileOffset, + startingRecordOffset, + Collections.emptySet()); + } + + /** + * Create ab ArrayBatchRecords with only finished split id + * + * @param splitId for the split that is just exhausted + */ + public static ArrayBatchRecords finishedSplit(String splitId) { + return new ArrayBatchRecords<>(null, null, null, 0, 0, 0, Collections.singleton(splitId)); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java new file mode 100644 index 000000000000..306afd1811be --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.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.source.reader; + +import java.io.IOException; +import java.util.NoSuchElementException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderOptions; +import org.apache.flink.connector.file.src.util.Pool; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** This implementation stores record batch in array from recyclable pool */ +class ArrayPoolDataIteratorBatcher implements DataIteratorBatcher { + private final int batchSize; + private final int handoverQueueSize; + private final RecordFactory recordFactory; + + private transient Pool pool; + + ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory recordFactory) { + this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); + this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY); + this.recordFactory = recordFactory; + } + + @Override + public CloseableIterator>> batch( + String splitId, DataIterator inputIterator) { + Preconditions.checkArgument(inputIterator != null, "Input data iterator can't be null"); + // lazily create pool as it is not serializable + if (pool == null) { + this.pool = createPoolOfBatches(handoverQueueSize); + } + return new ArrayPoolBatchIterator(splitId, inputIterator, pool); + } + + private Pool createPoolOfBatches(int numBatches) { + Pool poolOfBatches = new Pool<>(numBatches); + for (int batchId = 0; batchId < numBatches; batchId++) { + T[] batch = recordFactory.createBatch(batchSize); + poolOfBatches.add(batch); + } + + return poolOfBatches; + } + + private class ArrayPoolBatchIterator + implements CloseableIterator>> { + + private final String splitId; + private final DataIterator inputIterator; + private final Pool pool; + + ArrayPoolBatchIterator(String splitId, DataIterator inputIterator, Pool pool) { + this.splitId = splitId; + this.inputIterator = inputIterator; + this.pool = pool; + } + + @Override + public boolean hasNext() { + return inputIterator.hasNext(); + } + + @Override + public RecordsWithSplitIds> next() { + if (!inputIterator.hasNext()) { + throw new NoSuchElementException(); + } + + T[] batch = getCachedEntry(); + int recordCount = 0; + while (inputIterator.hasNext() && recordCount < batchSize) { + // The record produced by inputIterator can be reused like for the RowData case. + // inputIterator.next() can't be called again until the copy is made + // since the record is not consumed immediately. + T nextRecord = inputIterator.next(); + recordFactory.clone(nextRecord, batch, recordCount); + recordCount++; + if (!inputIterator.currentFileHasNext()) { + // break early so that records in the ArrayResultIterator + // have the same fileOffset. + break; + } + } + + return ArrayBatchRecords.forRecords( + splitId, + pool.recycler(), + batch, + recordCount, + inputIterator.fileOffset(), + inputIterator.recordOffset() - recordCount); + } + + @Override + public void close() throws IOException { + inputIterator.close(); + } + + private T[] getCachedEntry() { + try { + return pool.pollEntry(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted while waiting for array pool entry", e); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java new file mode 100644 index 000000000000..66e59633fff2 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -0,0 +1,102 @@ +/* + * 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 java.util.List; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Read Iceberg rows as {@link GenericRecord}. */ +public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { + private final String tableName; + private final Schema readSchema; + private final FileIO io; + private final EncryptionManager encryption; + private final RowDataFileScanTaskReader rowDataReader; + + private transient RowDataToAvroGenericRecordConverter converter; + + /** + * Create a reader function without projection and name mapping. Column name is case-insensitive. + */ + public static AvroGenericRecordReaderFunction fromTable(Table table) { + return new AvroGenericRecordReaderFunction( + table.name(), + new Configuration(), + table.schema(), + null, + null, + false, + table.io(), + table.encryption(), + null); + } + + public AvroGenericRecordReaderFunction( + String tableName, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters) { + super(new ListDataIteratorBatcher<>(config)); + this.tableName = tableName; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.io = io; + this.encryption = encryption; + this.rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>( + new AvroGenericRecordFileScanTaskReader(rowDataReader, lazyConverter()), + split.task(), + io, + encryption); + } + + private RowDataToAvroGenericRecordConverter lazyConverter() { + if (converter == null) { + this.converter = RowDataToAvroGenericRecordConverter.fromIcebergSchema(tableName, readSchema); + } + return converter; + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..4bb6f0a98c4c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java @@ -0,0 +1,98 @@ +/* + * 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 java.io.Serializable; +import java.util.Comparator; +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.TypeID; +import org.apache.iceberg.types.Types; + +/** + * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics + * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link + * WatermarkExtractorRecordEmitter} along with the actual records. + */ +@Internal +public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { + private final int eventTimeFieldId; + private final String eventTimeFieldName; + private final TimeUnit timeUnit; + + /** + * Creates the extractor. + * + * @param schema The schema of the Table + * @param eventTimeFieldName The column which should be used as an event time + * @param timeUnit Used for converting the long value to epoch milliseconds + */ + public ColumnStatsWatermarkExtractor( + Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { + Types.NestedField field = schema.findField(eventTimeFieldName); + TypeID typeID = field.type().typeId(); + Preconditions.checkArgument( + typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), + "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", + typeID); + this.eventTimeFieldId = field.fieldId(); + this.eventTimeFieldName = eventTimeFieldName; + // Use the timeUnit only for Long columns. + this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; + } + + @VisibleForTesting + ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { + this.eventTimeFieldId = eventTimeFieldId; + this.eventTimeFieldName = eventTimeFieldName; + this.timeUnit = TimeUnit.MICROSECONDS; + } + + /** + * Get the watermark for a split using column statistics. + * + * @param split The split + * @return The watermark + * @throws IllegalArgumentException if there is no statistics for the column + */ + @Override + public long extractWatermark(IcebergSourceSplit split) { + return split.task().files().stream() + .map( + scanTask -> { + Preconditions.checkArgument( + scanTask.file().lowerBounds() != null + && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, + "Missing statistics for column name = %s in file = %s", + eventTimeFieldName, + eventTimeFieldId, + scanTask.file()); + return timeUnit.toMillis( + Conversions.fromByteBuffer( + Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); + }) + .min(Comparator.comparingLong(l -> l)) + .get(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java new file mode 100644 index 000000000000..c376e359c600 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.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.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; + +/** + * Batcher converts iterator of T into iterator of batched {@code + * RecordsWithSplitIds>}, as FLIP-27's {@link SplitReader#fetch()} returns + * batched records. + */ +@FunctionalInterface +public interface DataIteratorBatcher extends Serializable { + CloseableIterator>> batch( + String splitId, DataIterator inputIterator); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java new file mode 100644 index 000000000000..bbf797ef4aa8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java @@ -0,0 +1,43 @@ +/* + * 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.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +/** A {@link ReaderFunction} implementation that uses {@link DataIterator}. */ +public abstract class DataIteratorReaderFunction implements ReaderFunction { + private final DataIteratorBatcher batcher; + + public DataIteratorReaderFunction(DataIteratorBatcher batcher) { + this.batcher = batcher; + } + + protected abstract DataIterator createDataIterator(IcebergSourceSplit split); + + @Override + public CloseableIterator>> apply( + IcebergSourceSplit split) { + DataIterator inputIterator = createDataIterator(split); + inputIterator.seek(split.fileOffset(), split.recordOffset()); + return batcher.batch(split.splitId(), inputIterator); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java new file mode 100644 index 000000000000..f143b8d2df2e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.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.flink.source.reader; + +import java.util.Collection; +import java.util.Collections; +import java.util.Map; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergSourceReader + extends SingleThreadMultiplexSourceReaderBase< + RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { + + public IcebergSourceReader( + SerializableRecordEmitter emitter, + IcebergSourceReaderMetrics metrics, + ReaderFunction readerFunction, + SerializableComparator splitComparator, + SourceReaderContext context) { + super( + () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), + emitter, + context.getConfiguration(), + context); + } + + @Override + public void start() { + // We request a split only if we did not get splits during the checkpoint restore. + // Otherwise, reader restarts will keep requesting more and more splits. + if (getNumberOfCurrentlyAssignedSplits() == 0) { + requestSplit(Collections.emptyList()); + } + } + + @Override + protected void onSplitFinished(Map finishedSplitIds) { + requestSplit(Lists.newArrayList(finishedSplitIds.keySet())); + } + + @Override + protected IcebergSourceSplit initializedState(IcebergSourceSplit split) { + return split; + } + + @Override + protected IcebergSourceSplit toSplitType(String splitId, IcebergSourceSplit splitState) { + return splitState; + } + + private void requestSplit(Collection finishedSplitIds) { + context.sendSourceEventToCoordinator(new SplitRequestEvent(finishedSplitIds)); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java new file mode 100644 index 000000000000..2a3e1dd86b95 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java @@ -0,0 +1,61 @@ +/* + * 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.metrics.Counter; +import org.apache.flink.metrics.MetricGroup; + +public class IcebergSourceReaderMetrics { + private final Counter assignedSplits; + private final Counter assignedBytes; + private final Counter finishedSplits; + private final Counter finishedBytes; + private final Counter splitReaderFetchCalls; + + public IcebergSourceReaderMetrics(MetricGroup metrics, String fullTableName) { + MetricGroup readerMetrics = + metrics.addGroup("IcebergSourceReader").addGroup("table", fullTableName); + + this.assignedSplits = readerMetrics.counter("assignedSplits"); + this.assignedBytes = readerMetrics.counter("assignedBytes"); + this.finishedSplits = readerMetrics.counter("finishedSplits"); + this.finishedBytes = readerMetrics.counter("finishedBytes"); + this.splitReaderFetchCalls = readerMetrics.counter("splitReaderFetchCalls"); + } + + public void incrementAssignedSplits(long count) { + assignedSplits.inc(count); + } + + public void incrementAssignedBytes(long count) { + assignedBytes.inc(count); + } + + public void incrementFinishedSplits(long count) { + finishedSplits.inc(count); + } + + public void incrementFinishedBytes(long count) { + finishedBytes.inc(count); + } + + public void incrementSplitReaderFetchCalls(long count) { + splitReaderFetchCalls.inc(count); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java new file mode 100644 index 000000000000..9c20494fdbcd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -0,0 +1,167 @@ +/* + * 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 java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Queue; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.connector.base.source.reader.RecordsBySplits; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; +import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class IcebergSourceSplitReader implements SplitReader, IcebergSourceSplit> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceSplitReader.class); + + private final IcebergSourceReaderMetrics metrics; + private final ReaderFunction openSplitFunction; + private final SerializableComparator splitComparator; + private final int indexOfSubtask; + private final Queue splits; + + private CloseableIterator>> currentReader; + private IcebergSourceSplit currentSplit; + private String currentSplitId; + + IcebergSourceSplitReader( + IcebergSourceReaderMetrics metrics, + ReaderFunction openSplitFunction, + SerializableComparator splitComparator, + SourceReaderContext context) { + this.metrics = metrics; + this.openSplitFunction = openSplitFunction; + this.splitComparator = splitComparator; + this.indexOfSubtask = context.getIndexOfSubtask(); + this.splits = Queues.newArrayDeque(); + } + + /** + * The method reads a batch of records from the assigned splits. If all the records from the + * current split are returned then it will emit a {@link ArrayBatchRecords#finishedSplit(String)} + * batch to signal this event. In the next fetch loop the reader will continue with the next split + * (if any). + * + * @return The fetched records + * @throws IOException If there is an error during reading + */ + @Override + public RecordsWithSplitIds> fetch() throws IOException { + metrics.incrementSplitReaderFetchCalls(1); + if (currentReader == null) { + IcebergSourceSplit nextSplit = splits.poll(); + if (nextSplit != null) { + currentSplit = nextSplit; + currentSplitId = nextSplit.splitId(); + currentReader = openSplitFunction.apply(currentSplit); + } else { + // return an empty result, which will lead to split fetch to be idle. + // SplitFetcherManager will then close idle fetcher. + return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + } + } + + if (currentReader.hasNext()) { + // Because Iterator#next() doesn't support checked exception, + // we need to wrap and unwrap the checked IOException with UncheckedIOException + try { + return currentReader.next(); + } catch (UncheckedIOException e) { + throw e.getCause(); + } + } else { + return finishSplit(); + } + } + + @Override + public void handleSplitsChanges(SplitsChange splitsChange) { + if (!(splitsChange instanceof SplitsAddition)) { + throw new UnsupportedOperationException( + String.format("Unsupported split change: %s", splitsChange.getClass())); + } + + if (splitComparator != null) { + List newSplits = Lists.newArrayList(splitsChange.splits()); + newSplits.sort(splitComparator); + LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); + splits.addAll(newSplits); + } else { + LOG.info("Add {} splits to reader", splitsChange.splits().size()); + splits.addAll(splitsChange.splits()); + } + metrics.incrementAssignedSplits(splitsChange.splits().size()); + metrics.incrementAssignedBytes(calculateBytes(splitsChange)); + } + + @Override + public void wakeUp() {} + + @Override + public void close() throws Exception { + currentSplitId = null; + if (currentReader != null) { + currentReader.close(); + } + } + + @Override + public void pauseOrResumeSplits( + Collection splitsToPause, Collection splitsToResume) { + // IcebergSourceSplitReader only reads splits sequentially. When waiting for watermark alignment + // the SourceOperator will stop processing and recycling the fetched batches. This exhausts the + // {@link ArrayPoolDataIteratorBatcher#pool} and the `currentReader.next()` call will be + // blocked even without split-level watermark alignment. Based on this the + // `pauseOrResumeSplits` and the `wakeUp` are left empty. + } + + private long calculateBytes(IcebergSourceSplit split) { + return split.task().files().stream().map(FileScanTask::length).reduce(0L, Long::sum); + } + + private long calculateBytes(SplitsChange splitsChanges) { + return splitsChanges.splits().stream().map(this::calculateBytes).reduce(0L, Long::sum); + } + + private ArrayBatchRecords finishSplit() throws IOException { + if (currentReader != null) { + currentReader.close(); + currentReader = null; + } + + ArrayBatchRecords finishRecords = ArrayBatchRecords.finishedSplit(currentSplitId); + LOG.info("Split reader {} finished split: {}", indexOfSubtask, currentSplitId); + metrics.incrementFinishedSplits(1); + metrics.incrementFinishedBytes(calculateBytes(currentSplit)); + currentSplitId = null; + return finishRecords; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java new file mode 100644 index 000000000000..020e87646d05 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.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.flink.source.reader; + +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class LimitableDataIterator extends DataIterator { + private final RecordLimiter limiter; + + LimitableDataIterator( + FileScanTaskReader fileScanTaskReader, + CombinedScanTask task, + FileIO io, + EncryptionManager encryption, + RecordLimiter limiter) { + super(fileScanTaskReader, task, io, encryption); + Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); + this.limiter = limiter; + } + + @Override + public boolean hasNext() { + if (limiter.reachedLimit()) { + return false; + } + + return super.hasNext(); + } + + @Override + public T next() { + limiter.increment(); + return super.next(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java new file mode 100644 index 000000000000..1acb3df76102 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java @@ -0,0 +1,85 @@ +/* + * 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 java.util.Collections; +import java.util.List; +import java.util.Set; +import javax.annotation.Nullable; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +class ListBatchRecords implements RecordsWithSplitIds> { + private String splitId; + private final List records; + private final Set finishedSplits; + private final RecordAndPosition recordAndPosition; + + // point to current read position within the records list + private int position; + + ListBatchRecords( + String splitId, + List records, + int fileOffset, + long startingRecordOffset, + Set finishedSplits) { + this.splitId = splitId; + this.records = records; + this.finishedSplits = + Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); + this.recordAndPosition = new RecordAndPosition<>(); + this.recordAndPosition.set(null, fileOffset, startingRecordOffset); + + this.position = 0; + } + + @Nullable + @Override + public String nextSplit() { + String nextSplit = this.splitId; + // set the splitId to null to indicate no more splits + // this class only contains record for one split + this.splitId = null; + return nextSplit; + } + + @Nullable + @Override + public RecordAndPosition nextRecordFromSplit() { + if (position < records.size()) { + recordAndPosition.record(records.get(position)); + position++; + return recordAndPosition; + } else { + return null; + } + } + + @Override + public Set finishedSplits() { + return finishedSplits; + } + + public static ListBatchRecords forRecords( + String splitId, List records, int fileOffset, long startingRecordOffset) { + return new ListBatchRecords<>( + splitId, records, fileOffset, startingRecordOffset, Collections.emptySet()); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java new file mode 100644 index 000000000000..365416239d37 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java @@ -0,0 +1,94 @@ +/* + * 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 java.io.IOException; +import java.util.List; +import java.util.NoSuchElementException; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** + * FlinkRecordReaderFunction essentially cloned objects already. So there is no need to use array + * pool to clone objects. Simply create a new ArrayList for each batch. + */ +class ListDataIteratorBatcher implements DataIteratorBatcher { + + private final int batchSize; + + ListDataIteratorBatcher(ReadableConfig config) { + this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); + } + + @Override + public CloseableIterator>> batch( + String splitId, DataIterator dataIterator) { + return new ListBatchIterator(splitId, dataIterator); + } + + private class ListBatchIterator + implements CloseableIterator>> { + + private final String splitId; + private final DataIterator inputIterator; + + ListBatchIterator(String splitId, DataIterator inputIterator) { + this.splitId = splitId; + this.inputIterator = inputIterator; + } + + @Override + public boolean hasNext() { + return inputIterator.hasNext(); + } + + @Override + public RecordsWithSplitIds> next() { + if (!inputIterator.hasNext()) { + throw new NoSuchElementException(); + } + + final List batch = Lists.newArrayListWithCapacity(batchSize); + int recordCount = 0; + while (inputIterator.hasNext() && recordCount < batchSize) { + T nextRecord = inputIterator.next(); + batch.add(nextRecord); + recordCount++; + if (!inputIterator.currentFileHasNext()) { + // break early so that records have the same fileOffset. + break; + } + } + + return ListBatchRecords.forRecords( + splitId, batch, inputIterator.fileOffset(), inputIterator.recordOffset() - recordCount); + } + + @Override + public void close() throws IOException { + if (inputIterator != null) { + inputIterator.close(); + } + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java new file mode 100644 index 000000000000..fb4466913b90 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.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.flink.source.reader; + +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.DataTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** Reading metadata tables (like snapshots, manifests, etc.) */ +@Internal +public class MetaDataReaderFunction extends DataIteratorReaderFunction { + private final Schema readSchema; + private final FileIO io; + private final EncryptionManager encryption; + + public MetaDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + FileIO io, + EncryptionManager encryption) { + super( + new ArrayPoolDataIteratorBatcher<>( + config, + new RowDataRecordFactory( + FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); + this.readSchema = readSchema(tableSchema, projectedSchema); + this.io = io; + this.encryption = encryption; + } + + @Override + public DataIterator createDataIterator(IcebergSourceSplit split) { + return new DataIterator<>(new DataTaskReader(readSchema), split.task(), io, encryption); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java new file mode 100644 index 000000000000..1ea91f10b4e7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java @@ -0,0 +1,31 @@ +/* + * 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 java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; + +@FunctionalInterface +public interface ReaderFunction + extends Serializable, + Function< + IcebergSourceSplit, CloseableIterator>>> {} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java new file mode 100644 index 000000000000..6ac92592b6aa --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import org.apache.flink.annotation.Internal; + +/** + * A record along with the reader position to be stored in the checkpoint. + * + *

    The position defines the point in the reader AFTER the record. Record processing and updating + * checkpointed state happens atomically. The position points to where the reader should resume + * after this record is processed. + * + *

    This mutable object is useful in cases where only one instance of a {@code RecordAndPosition} + * is needed at a time. Then the same instance of RecordAndPosition can be reused. + */ +@Internal +public class RecordAndPosition { + private T record; + private int fileOffset; + private long recordOffset; + + public RecordAndPosition(T record, int fileOffset, long recordOffset) { + this.record = record; + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + public RecordAndPosition() {} + + // ------------------------------------------------------------------------ + + public T record() { + return record; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + /** Updates the record and position in this object. */ + public void set(T newRecord, int newFileOffset, long newRecordOffset) { + this.record = newRecord; + this.fileOffset = newFileOffset; + this.recordOffset = newRecordOffset; + } + + /** Sets the next record of a sequence. This increments the {@code recordOffset} by one. */ + public void record(T nextRecord) { + this.record = nextRecord; + this.recordOffset++; + } + + @Override + public String toString() { + return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java new file mode 100644 index 000000000000..ef92e2e6b81f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.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.source.reader; + +import java.io.Serializable; + +/** + * In FLIP-27 source, SplitReader#fetch() returns a batch of records. Since DataIterator for RowData + * returns an iterator of reused RowData objects, RecordFactory is needed to (1) create object array + * that is recyclable via pool. (2) clone RowData element from DataIterator to the batch array. + */ +interface RecordFactory extends Serializable { + /** Create a batch of records */ + T[] createBatch(int batchSize); + + /** Clone record into the specified position of the batch array */ + void clone(T from, T[] batch, int position); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java new file mode 100644 index 000000000000..f260a53089ff --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java @@ -0,0 +1,45 @@ +/* + * 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 java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.annotation.Internal; + +@Internal +class RecordLimiter { + private final long limit; + private final AtomicLong counter; + + static RecordLimiter create(long limit) { + return new RecordLimiter(limit); + } + + private RecordLimiter(long limit) { + this.limit = limit; + this.counter = new AtomicLong(0); + } + + public boolean reachedLimit() { + return limit > 0 && counter.get() >= limit; + } + + public void increment() { + counter.incrementAndGet(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java new file mode 100644 index 000000000000..c9208a0e1834 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.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.source.reader; + +import java.util.List; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class RowDataReaderFunction extends DataIteratorReaderFunction { + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters) { + this( + config, + tableSchema, + projectedSchema, + nameMapping, + caseSensitive, + io, + encryption, + filters, + -1L); + } + + public RowDataReaderFunction( + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super( + new ArrayPoolDataIteratorBatcher<>( + config, + new RowDataRecordFactory( + FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + public DataIterator createDataIterator(IcebergSourceSplit split) { + return new LimitableDataIterator<>( + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java new file mode 100644 index 000000000000..40d5c28d7bc7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java @@ -0,0 +1,73 @@ +/* + * 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.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.flink.data.RowDataUtil; + +class RowDataRecordFactory implements RecordFactory { + private final RowType rowType; + private final TypeSerializer[] fieldSerializers; + private final RowData.FieldGetter[] fieldGetters; + + RowDataRecordFactory(RowType rowType) { + this.rowType = rowType; + this.fieldSerializers = createFieldSerializers(rowType); + this.fieldGetters = createFieldGetters(rowType); + } + + static TypeSerializer[] createFieldSerializers(RowType rowType) { + return rowType.getChildren().stream() + .map(InternalSerializers::create) + .toArray(TypeSerializer[]::new); + } + + static RowData.FieldGetter[] createFieldGetters(RowType rowType) { + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return fieldGetters; + } + + @Override + public RowData[] createBatch(int batchSize) { + RowData[] arr = new RowData[batchSize]; + for (int i = 0; i < batchSize; ++i) { + arr[i] = new GenericRowData(rowType.getFieldCount()); + } + return arr; + } + + @Override + public void clone(RowData from, RowData[] batch, int position) { + // Set the return value from RowDataUtil.clone back to the array. + // Clone method returns same clone target object (reused) if it is a GenericRowData. + // Clone method will allocate a new GenericRowData object + // if the target object is NOT a GenericRowData. + // So we should always set the clone return value back to the array. + batch[position] = + RowDataUtil.clone(from, batch[position], rowType, fieldSerializers, fieldGetters); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java new file mode 100644 index 000000000000..a6e2c1dae243 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.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.flink.source.reader; + +import java.io.Serializable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.connector.base.source.reader.RecordEmitter; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +@Internal +@FunctionalInterface +public interface SerializableRecordEmitter + extends RecordEmitter, T, IcebergSourceSplit>, Serializable { + static SerializableRecordEmitter defaultEmitter() { + return (element, output, split) -> { + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + }; + } + + static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { + return new WatermarkExtractorRecordEmitter<>(extractor); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java new file mode 100644 index 000000000000..d1c50ac8ca52 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.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.flink.source.reader; + +import java.io.Serializable; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; + +/** The interface used to extract watermarks from splits. */ +public interface SplitWatermarkExtractor extends Serializable { + /** Get the watermark for a split. */ + long extractWatermark(IcebergSourceSplit split); +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java new file mode 100644 index 000000000000..02ef57d344b1 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.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.source.reader; + +import org.apache.flink.api.common.eventtime.Watermark; +import org.apache.flink.api.connector.source.SourceOutput; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Emitter which emits the watermarks, records and updates the split position. + * + *

    The Emitter emits watermarks at the beginning of every split provided by the {@link + * SplitWatermarkExtractor}. + */ +class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { + private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); + private final SplitWatermarkExtractor timeExtractor; + private String lastSplitId = null; + private long watermark; + + WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { + this.timeExtractor = timeExtractor; + } + + @Override + public void emitRecord( + RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { + if (!split.splitId().equals(lastSplitId)) { + long newWatermark = timeExtractor.extractWatermark(split); + if (newWatermark < watermark) { + LOG.info( + "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", + watermark, + newWatermark, + lastSplitId, + split.splitId()); + } else { + watermark = newWatermark; + output.emitWatermark(new Watermark(watermark)); + LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); + } + + lastSplitId = split.splitId(); + } + + output.collect(element.record()); + split.updatePosition(element.fileOffset(), element.recordOffset()); + } +} 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 new file mode 100644 index 000000000000..344f64833b62 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java @@ -0,0 +1,220 @@ +/* + * 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.split; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import javax.annotation.Nullable; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.ScanTaskParser; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +@Internal +public class IcebergSourceSplit implements SourceSplit, Serializable { + private static final long serialVersionUID = 1L; + private static final ThreadLocal SERIALIZER_CACHE = + ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); + + private final CombinedScanTask task; + + private int fileOffset; + private long recordOffset; + + // The splits are frequently serialized into checkpoints. + // Caching the byte representation makes repeated serialization cheap. + @Nullable private transient byte[] serializedBytesCache; + + private IcebergSourceSplit(CombinedScanTask task, int fileOffset, long recordOffset) { + this.task = task; + this.fileOffset = fileOffset; + this.recordOffset = recordOffset; + } + + public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) { + return fromCombinedScanTask(combinedScanTask, 0, 0L); + } + + public static IcebergSourceSplit fromCombinedScanTask( + CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) { + return new IcebergSourceSplit(combinedScanTask, fileOffset, recordOffset); + } + + public CombinedScanTask task() { + return task; + } + + public int fileOffset() { + return fileOffset; + } + + public long recordOffset() { + return recordOffset; + } + + @Override + public String splitId() { + return MoreObjects.toStringHelper(this).add("files", toString(task.files())).toString(); + } + + public void updatePosition(int newFileOffset, long newRecordOffset) { + // invalidate the cache after position change + serializedBytesCache = null; + fileOffset = newFileOffset; + recordOffset = newRecordOffset; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("files", toString(task.files())) + .add("fileOffset", fileOffset) + .add("recordOffset", recordOffset) + .toString(); + } + + private String toString(Collection files) { + return Iterables.toString( + files.stream() + .map( + fileScanTask -> + MoreObjects.toStringHelper(fileScanTask) + .add("file", fileScanTask.file().path().toString()) + .add("start", fileScanTask.start()) + .add("length", fileScanTask.length()) + .toString()) + .collect(Collectors.toList())); + } + + byte[] serializeV1() throws IOException { + if (serializedBytesCache == null) { + serializedBytesCache = InstantiationUtil.serializeObject(this); + } + + return serializedBytesCache; + } + + static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { + try { + return InstantiationUtil.deserializeObject( + serialized, IcebergSourceSplit.class.getClassLoader()); + } catch (ClassNotFoundException e) { + throw new RuntimeException("Failed to deserialize the split.", e); + } + } + + byte[] serializeV2() throws IOException { + return serialize(2); + } + + byte[] serializeV3() throws IOException { + return serialize(3); + } + + private byte[] serialize(int version) throws IOException { + if (serializedBytesCache == null) { + DataOutputSerializer out = SERIALIZER_CACHE.get(); + Collection fileScanTasks = task.tasks(); + Preconditions.checkArgument( + fileOffset >= 0 && fileOffset < fileScanTasks.size(), + "Invalid file offset: %s. Should be within the range of [0, %s)", + fileOffset, + fileScanTasks.size()); + + out.writeInt(fileOffset); + out.writeLong(recordOffset); + out.writeInt(fileScanTasks.size()); + + for (FileScanTask fileScanTask : fileScanTasks) { + String taskJson = ScanTaskParser.toJson(fileScanTask); + writeTaskJson(out, taskJson, version); + } + + serializedBytesCache = out.getCopyOfBuffer(); + out.clear(); + } + + return serializedBytesCache; + } + + private static void writeTaskJson(DataOutputSerializer out, String taskJson, int version) + throws IOException { + switch (version) { + case 2: + out.writeUTF(taskJson); + break; + case 3: + SerializerHelper.writeLongUTF(out, taskJson); + break; + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } + + static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) + throws IOException { + return deserialize(serialized, caseSensitive, 2); + } + + static IcebergSourceSplit deserializeV3(byte[] serialized, boolean caseSensitive) + throws IOException { + return deserialize(serialized, caseSensitive, 3); + } + + private static IcebergSourceSplit deserialize( + byte[] serialized, boolean caseSensitive, int version) throws IOException { + DataInputDeserializer in = new DataInputDeserializer(serialized); + int fileOffset = in.readInt(); + long recordOffset = in.readLong(); + int taskCount = in.readInt(); + + List tasks = Lists.newArrayListWithCapacity(taskCount); + for (int i = 0; i < taskCount; ++i) { + String taskJson = readTaskJson(in, version); + FileScanTask task = ScanTaskParser.fromJson(taskJson, caseSensitive); + tasks.add(task); + } + + CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); + return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); + } + + private static String readTaskJson(DataInputDeserializer in, int version) throws IOException { + switch (version) { + case 2: + return in.readUTF(); + case 3: + return SerializerHelper.readLongUTF(in); + default: + throw new IllegalArgumentException("Unsupported version: " + version); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java new file mode 100644 index 000000000000..d4b0f9e1977d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.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.flink.source.split; + +import java.io.IOException; +import org.apache.flink.annotation.Internal; +import org.apache.flink.core.io.SimpleVersionedSerializer; + +@Internal +public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 3; + + private final boolean caseSensitive; + + public IcebergSourceSplitSerializer(boolean caseSensitive) { + this.caseSensitive = caseSensitive; + } + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergSourceSplit split) throws IOException { + return split.serializeV3(); + } + + @Override + public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { + switch (version) { + case 1: + return IcebergSourceSplit.deserializeV1(serialized); + case 2: + return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); + case 3: + return IcebergSourceSplit.deserializeV3(serialized, caseSensitive); + default: + throw new IOException( + String.format( + "Failed to deserialize IcebergSourceSplit. " + + "Encountered unsupported version: %d. Supported version are [1]", + version)); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java new file mode 100644 index 000000000000..d9061e049e00 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.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.flink.source.split; + +public class IcebergSourceSplitState { + private final IcebergSourceSplit split; + private final IcebergSourceSplitStatus status; + + public IcebergSourceSplitState(IcebergSourceSplit split, IcebergSourceSplitStatus status) { + this.split = split; + this.status = status; + } + + public IcebergSourceSplit split() { + return split; + } + + public IcebergSourceSplitStatus status() { + return status; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java new file mode 100644 index 000000000000..d4a84a165e1a --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java @@ -0,0 +1,25 @@ +/* + * 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.split; + +public enum IcebergSourceSplitStatus { + UNASSIGNED, + ASSIGNED, + COMPLETED +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java new file mode 100644 index 000000000000..319648ca275c --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * 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.split; + +import java.io.Serializable; +import java.util.Comparator; + +public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java new file mode 100644 index 000000000000..841969666ee5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.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.flink.source.split; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UTFDataFormatException; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; + +/** + * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly + * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and + * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. + */ +class SerializerHelper implements Serializable { + + private SerializerHelper() {} + + /** + * Similar to {@link DataOutputSerializer#writeUTF(String)}. Except this supports larger payloads + * which is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param out the output stream to write the string to. + * @param str the string value to be written. + */ + public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { + int strlen = str.length(); + long utflen = 0; + int ch; + + /* use charAt instead of copying String to char array */ + for (int i = 0; i < strlen; i++) { + ch = str.charAt(i); + utflen += getUTFBytesSize(ch); + + if (utflen > Integer.MAX_VALUE) { + throw new UTFDataFormatException("Encoded string reached maximum length: " + utflen); + } + } + + if (utflen > Integer.MAX_VALUE - 4) { + throw new UTFDataFormatException("Encoded string is too long: " + utflen); + } + + out.writeInt((int) utflen); + writeUTFBytes(out, str, (int) utflen); + } + + /** + * Similar to {@link DataInputDeserializer#readUTF()}. Except this supports larger payloads which + * is up to max integer value. + * + *

    Note: This method can be removed when the method which does similar thing within the {@link + * DataOutputSerializer} already which does the same thing, so use that one instead once that is + * released on Flink version 1.20. + * + *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 + * + * @param in the input stream to read the string from. + * @return the string value read from the input stream. + * @throws IOException if an I/O error occurs when reading from the input stream. + */ + public static String readLongUTF(DataInputDeserializer in) throws IOException { + int utflen = in.readInt(); + byte[] bytearr = new byte[utflen]; + char[] chararr = new char[utflen]; + + int ch; + int char2; + int char3; + int count = 0; + int chararrCount = 0; + + in.readFully(bytearr, 0, utflen); + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + if (ch > 127) { + break; + } + count++; + chararr[chararrCount++] = (char) ch; + } + + while (count < utflen) { + ch = (int) bytearr[count] & 0xff; + switch (ch >> 4) { + case 0: + case 1: + case 2: + case 3: + case 4: + case 5: + case 6: + case 7: + /* 0xxxxxxx */ + count++; + chararr[chararrCount++] = (char) ch; + break; + case 12: + case 13: + /* 110x xxxx 10xx xxxx */ + count += 2; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = bytearr[count - 1]; + if ((char2 & 0xC0) != 0x80) { + throw new UTFDataFormatException("malformed input around byte " + count); + } + chararr[chararrCount++] = (char) (((ch & 0x1F) << 6) | (char2 & 0x3F)); + break; + case 14: + /* 1110 xxxx 10xx xxxx 10xx xxxx */ + count += 3; + if (count > utflen) { + throw new UTFDataFormatException("malformed input: partial character at end"); + } + char2 = bytearr[count - 2]; + char3 = bytearr[count - 1]; + if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { + throw new UTFDataFormatException("malformed input around byte " + (count - 1)); + } + chararr[chararrCount++] = + (char) (((ch & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F)); + break; + default: + /* 10xx xxxx, 1111 xxxx */ + throw new UTFDataFormatException("malformed input around byte " + count); + } + } + // The number of chars produced may be less than utflen + return new String(chararr, 0, chararrCount); + } + + private static int getUTFBytesSize(int ch) { + if ((ch >= 0x0001) && (ch <= 0x007F)) { + return 1; + } else if (ch > 0x07FF) { + return 3; + } else { + return 2; + } + } + + private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) + throws IOException { + int strlen = str.length(); + int ch; + + int len = Math.max(1024, utflen); + + byte[] bytearr = new byte[len]; + int count = 0; + + int index; + for (index = 0; index < strlen; index++) { + ch = str.charAt(index); + if (!((ch >= 0x0001) && (ch <= 0x007F))) { + break; + } + bytearr[count++] = (byte) ch; + } + + for (; index < strlen; index++) { + ch = str.charAt(index); + if ((ch >= 0x0001) && (ch <= 0x007F)) { + bytearr[count++] = (byte) ch; + } else if (ch > 0x07FF) { + bytearr[count++] = (byte) (0xE0 | ((ch >> 12) & 0x0F)); + bytearr[count++] = (byte) (0x80 | ((ch >> 6) & 0x3F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } else { + bytearr[count++] = (byte) (0xC0 | ((ch >> 6) & 0x1F)); + bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); + } + } + + out.write(bytearr, 0, count); + } +} 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 new file mode 100644 index 000000000000..56ee92014d12 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} + * which could be used for ordering splits. These are used by the {@link + * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link + * org.apache.iceberg.flink.source.reader.IcebergSourceReader} + */ +public class SplitComparators { + private SplitComparators() {} + + /** Comparator which orders the splits based on the file sequence number of the data files */ + 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"); + + Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); + Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); + + Preconditions.checkNotNull( + seq1, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o1); + Preconditions.checkNotNull( + seq2, + "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", + o2); + + int temp = Long.compare(seq1, seq2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } + + /** Comparator which orders the splits based on watermark of the splits */ + public static SerializableComparator watermark( + SplitWatermarkExtractor watermarkExtractor) { + return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { + long watermark1 = watermarkExtractor.extractWatermark(o1); + long watermark2 = watermarkExtractor.extractWatermark(o2); + + int temp = Long.compare(watermark1, watermark2); + if (temp != 0) { + return temp; + } else { + return o1.splitId().compareTo(o2.splitId()); + } + }; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java new file mode 100644 index 000000000000..eabd757aa638 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java @@ -0,0 +1,54 @@ +/* + * 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.split; + +import java.util.Collection; +import java.util.Collections; +import org.apache.flink.annotation.Internal; +import org.apache.flink.api.connector.source.SourceEvent; + +/** We can remove this class once FLINK-21364 is resolved. */ +@Internal +public class SplitRequestEvent implements SourceEvent { + private static final long serialVersionUID = 1L; + + private final Collection finishedSplitIds; + private final String requesterHostname; + + public SplitRequestEvent() { + this(Collections.emptyList()); + } + + public SplitRequestEvent(Collection finishedSplitIds) { + this(finishedSplitIds, null); + } + + public SplitRequestEvent(Collection finishedSplitIds, String requesterHostname) { + this.finishedSplitIds = finishedSplitIds; + this.requesterHostname = requesterHostname; + } + + public Collection finishedSplitIds() { + return finishedSplitIds; + } + + public String requesterHostname() { + return requesterHostname; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java new file mode 100644 index 000000000000..6306e82d5729 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/ElapsedTimeGauge.java @@ -0,0 +1,47 @@ +/* + * 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.util; + +import java.util.concurrent.TimeUnit; +import org.apache.flink.annotation.Internal; +import org.apache.flink.metrics.Gauge; + +/** + * This gauge measures the elapsed time between now and last recorded time set by {@link + * ElapsedTimeGauge#refreshLastRecordedTime()}. + */ +@Internal +public class ElapsedTimeGauge implements Gauge { + private final TimeUnit reportUnit; + private volatile long lastRecordedTimeNano; + + public ElapsedTimeGauge(TimeUnit timeUnit) { + this.reportUnit = timeUnit; + refreshLastRecordedTime(); + } + + public void refreshLastRecordedTime() { + this.lastRecordedTimeNano = System.nanoTime(); + } + + @Override + public Long getValue() { + return reportUnit.convert(System.nanoTime() - lastRecordedTimeNano, TimeUnit.NANOSECONDS); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java new file mode 100644 index 000000000000..2bbc9cf208fe --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java @@ -0,0 +1,248 @@ +/* + * 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.util; + +import java.util.List; +import java.util.Map; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.catalog.TableChange; +import org.apache.flink.table.catalog.UniqueConstraint; +import org.apache.iceberg.Table; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.UpdateProperties; +import org.apache.iceberg.UpdateSchema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.types.Type; + +public class FlinkAlterTableUtil { + private FlinkAlterTableUtil() {} + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + Map setProperties) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!setProperties.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + setProperties.forEach( + (k, v) -> { + if (v == null) { + updateProperties.remove(k); + } else { + updateProperties.set(k, v); + } + }); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitChanges( + Table table, + String setLocation, + String setSnapshotId, + String pickSnapshotId, + List schemaChanges, + List propertyChanges) { + commitManageSnapshots(table, setSnapshotId, pickSnapshotId); + + Transaction transaction = table.newTransaction(); + + if (setLocation != null) { + transaction.updateLocation().setLocation(setLocation).commit(); + } + + if (!schemaChanges.isEmpty()) { + UpdateSchema updateSchema = transaction.updateSchema(); + FlinkAlterTableUtil.applySchemaChanges(updateSchema, schemaChanges); + updateSchema.commit(); + } + + if (!propertyChanges.isEmpty()) { + UpdateProperties updateProperties = transaction.updateProperties(); + FlinkAlterTableUtil.applyPropertyChanges(updateProperties, propertyChanges); + updateProperties.commit(); + } + + transaction.commitTransaction(); + } + + public static void commitManageSnapshots( + Table table, String setSnapshotId, String cherrypickSnapshotId) { + // don't allow setting the snapshot and picking a commit at the same time because order is + // ambiguous and choosing one order leads to different results + Preconditions.checkArgument( + setSnapshotId == null || cherrypickSnapshotId == null, + "Cannot set the current snapshot ID and cherry-pick snapshot changes"); + + if (setSnapshotId != null) { + long newSnapshotId = Long.parseLong(setSnapshotId); + table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); + } + + // if updating the table snapshot, perform that update first in case it fails + if (cherrypickSnapshotId != null) { + long newSnapshotId = Long.parseLong(cherrypickSnapshotId); + table.manageSnapshots().cherrypick(newSnapshotId).commit(); + } + } + + /** + * Applies a list of Flink table changes to an {@link UpdateSchema} operation. + * + * @param pendingUpdate an uncommitted UpdateSchema operation to configure + * @param schemaChanges a list of Flink table changes + */ + public static void applySchemaChanges( + UpdateSchema pendingUpdate, List schemaChanges) { + for (TableChange change : schemaChanges) { + if (change instanceof TableChange.AddColumn) { + TableChange.AddColumn addColumn = (TableChange.AddColumn) change; + Column flinkColumn = addColumn.getColumn(); + Preconditions.checkArgument( + FlinkCompatibilityUtil.isPhysicalColumn(flinkColumn), + "Unsupported table change: Adding computed column %s.", + flinkColumn.getName()); + Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); + if (flinkColumn.getDataType().getLogicalType().isNullable()) { + pendingUpdate.addColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); + } else { + pendingUpdate.addRequiredColumn( + flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); + } + } else if (change instanceof TableChange.ModifyColumn) { + TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; + applyModifyColumn(pendingUpdate, modifyColumn); + } else if (change instanceof TableChange.DropColumn) { + TableChange.DropColumn dropColumn = (TableChange.DropColumn) change; + pendingUpdate.deleteColumn(dropColumn.getColumnName()); + } else if (change instanceof TableChange.AddWatermark) { + throw new UnsupportedOperationException("Unsupported table change: AddWatermark."); + } else if (change instanceof TableChange.ModifyWatermark) { + throw new UnsupportedOperationException("Unsupported table change: ModifyWatermark."); + } else if (change instanceof TableChange.DropWatermark) { + throw new UnsupportedOperationException("Unsupported table change: DropWatermark."); + } else if (change instanceof TableChange.AddUniqueConstraint) { + TableChange.AddUniqueConstraint addPk = (TableChange.AddUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, addPk.getConstraint()); + } else if (change instanceof TableChange.ModifyUniqueConstraint) { + TableChange.ModifyUniqueConstraint modifyPk = (TableChange.ModifyUniqueConstraint) change; + applyUniqueConstraint(pendingUpdate, modifyPk.getNewConstraint()); + } else if (change instanceof TableChange.DropConstraint) { + throw new UnsupportedOperationException("Unsupported table change: DropConstraint."); + } else { + throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); + } + } + } + + /** + * Applies a list of Flink table property changes to an {@link UpdateProperties} operation. + * + * @param pendingUpdate an uncommitted UpdateProperty operation to configure + * @param propertyChanges a list of Flink table changes + */ + public static void applyPropertyChanges( + UpdateProperties pendingUpdate, List propertyChanges) { + for (TableChange change : propertyChanges) { + if (change instanceof TableChange.SetOption) { + TableChange.SetOption setOption = (TableChange.SetOption) change; + pendingUpdate.set(setOption.getKey(), setOption.getValue()); + } else if (change instanceof TableChange.ResetOption) { + TableChange.ResetOption resetOption = (TableChange.ResetOption) change; + pendingUpdate.remove(resetOption.getKey()); + } else { + throw new UnsupportedOperationException( + "The given table change is not a property change: " + change); + } + } + } + + private static void applyModifyColumn( + UpdateSchema pendingUpdate, TableChange.ModifyColumn modifyColumn) { + if (modifyColumn instanceof TableChange.ModifyColumnName) { + TableChange.ModifyColumnName modifyName = (TableChange.ModifyColumnName) modifyColumn; + pendingUpdate.renameColumn(modifyName.getOldColumnName(), modifyName.getNewColumnName()); + } else if (modifyColumn instanceof TableChange.ModifyColumnPosition) { + TableChange.ModifyColumnPosition modifyPosition = + (TableChange.ModifyColumnPosition) modifyColumn; + applyModifyColumnPosition(pendingUpdate, modifyPosition); + } else if (modifyColumn instanceof TableChange.ModifyPhysicalColumnType) { + TableChange.ModifyPhysicalColumnType modifyType = + (TableChange.ModifyPhysicalColumnType) modifyColumn; + Type type = FlinkSchemaUtil.convert(modifyType.getNewType().getLogicalType()); + String columnName = modifyType.getOldColumn().getName(); + pendingUpdate.updateColumn(columnName, type.asPrimitiveType()); + if (modifyType.getNewColumn().getDataType().getLogicalType().isNullable()) { + pendingUpdate.makeColumnOptional(columnName); + } else { + pendingUpdate.requireColumn(columnName); + } + } else if (modifyColumn instanceof TableChange.ModifyColumnComment) { + TableChange.ModifyColumnComment modifyComment = + (TableChange.ModifyColumnComment) modifyColumn; + pendingUpdate.updateColumnDoc( + modifyComment.getOldColumn().getName(), modifyComment.getNewComment()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column change: " + modifyColumn); + } + } + + private static void applyModifyColumnPosition( + UpdateSchema pendingUpdate, TableChange.ModifyColumnPosition modifyColumnPosition) { + TableChange.ColumnPosition newPosition = modifyColumnPosition.getNewPosition(); + if (newPosition instanceof TableChange.First) { + pendingUpdate.moveFirst(modifyColumnPosition.getOldColumn().getName()); + } else if (newPosition instanceof TableChange.After) { + TableChange.After after = (TableChange.After) newPosition; + pendingUpdate.moveAfter(modifyColumnPosition.getOldColumn().getName(), after.column()); + } else { + throw new UnsupportedOperationException( + "Cannot apply unknown modify-column-position change: " + modifyColumnPosition); + } + } + + private static void applyUniqueConstraint( + UpdateSchema pendingUpdate, UniqueConstraint constraint) { + switch (constraint.getType()) { + case PRIMARY_KEY: + pendingUpdate.setIdentifierFields(constraint.getColumns()); + break; + case UNIQUE_KEY: + throw new UnsupportedOperationException( + "Unsupported table change: setting unique key constraints."); + default: + throw new UnsupportedOperationException( + "Cannot apply unknown unique constraint: " + constraint.getType().name()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java new file mode 100644 index 000000000000..f02af894e82b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java @@ -0,0 +1,47 @@ +/* + * 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.util; + +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.catalog.Column; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; +import org.apache.flink.table.types.logical.RowType; + +/** + * This is a small util class that try to hide calls to Flink Internal or PublicEvolve interfaces as + * Flink can change those APIs during minor version release. + */ +public class FlinkCompatibilityUtil { + + private FlinkCompatibilityUtil() {} + + public static TypeInformation toTypeInfo(RowType rowType) { + return InternalTypeInfo.of(rowType); + } + + public static boolean isPhysicalColumn(TableColumn column) { + return column.isPhysical(); + } + + public static boolean isPhysicalColumn(Column column) { + return column.isPhysical(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java new file mode 100644 index 000000000000..20b33e615e5f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java @@ -0,0 +1,61 @@ +/* + * 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.util; + +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; + +public class FlinkPackage { + + private static final AtomicReference VERSION = new AtomicReference<>(); + public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION"; + + private FlinkPackage() {} + + /** Returns Flink version string like x.y.z */ + public static String version() { + if (null == VERSION.get()) { + String detectedVersion; + try { + detectedVersion = versionFromJar(); + // use unknown version in case exact implementation version can't be found from the jar + // (this can happen if the DataStream class appears multiple times in the same classpath + // such as with shading) + detectedVersion = detectedVersion != null ? detectedVersion : FLINK_UNKNOWN_VERSION; + } catch (Exception e) { + detectedVersion = FLINK_UNKNOWN_VERSION; + } + VERSION.set(detectedVersion); + } + + return VERSION.get(); + } + + @VisibleForTesting + static String versionFromJar() { + // Choose {@link DataStream} class because it is one of the core Flink API + return DataStream.class.getPackage().getImplementationVersion(); + } + + @VisibleForTesting + static void setVersion(String version) { + VERSION.set(version); + } +} diff --git a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000000..29a9955a7e20 --- /dev/null +++ b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -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.flink.FlinkDynamicTableFactory diff --git a/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory new file mode 100644 index 000000000000..2b6bfa3cd579 --- /dev/null +++ b/flink/v1.19/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory @@ -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.flink.FlinkCatalogFactory diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java new file mode 100644 index 000000000000..4184526a6a1a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java @@ -0,0 +1,90 @@ +/* + * 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; + +import org.junit.jupiter.api.Test; + +public abstract class AvroGenericRecordConverterBase { + protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; + + @Test + public void testPrimitiveTypes() throws Exception { + testConverter(new DataGenerators.Primitives()); + } + + @Test + public void testStructOfPrimitive() throws Exception { + testConverter(new DataGenerators.StructOfPrimitive()); + } + + @Test + public void testStructOfArray() throws Exception { + testConverter(new DataGenerators.StructOfArray()); + } + + @Test + public void testStructOfMap() throws Exception { + testConverter(new DataGenerators.StructOfMap()); + } + + @Test + public void testStructOfStruct() throws Exception { + testConverter(new DataGenerators.StructOfStruct()); + } + + @Test + public void testArrayOfPrimitive() throws Exception { + testConverter(new DataGenerators.ArrayOfPrimitive()); + } + + @Test + public void testArrayOfArray() throws Exception { + testConverter(new DataGenerators.ArrayOfArray()); + } + + @Test + public void testArrayOfMap() throws Exception { + testConverter(new DataGenerators.ArrayOfMap()); + } + + @Test + public void testArrayOfStruct() throws Exception { + testConverter(new DataGenerators.ArrayOfStruct()); + } + + @Test + public void testMapOfPrimitives() throws Exception { + testConverter(new DataGenerators.MapOfPrimitives()); + } + + @Test + public void testMapOfArray() throws Exception { + testConverter(new DataGenerators.MapOfArray()); + } + + @Test + public void testMapOfMap() throws Exception { + testConverter(new DataGenerators.MapOfMap()); + } + + @Test + public void testMapOfStruct() throws Exception { + testConverter(new DataGenerators.MapOfStruct()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java new file mode 100644 index 000000000000..062ff68d5d85 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -0,0 +1,121 @@ +/* + * 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; + +import java.io.File; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import org.apache.flink.util.ArrayUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.SupportsNamespaces; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +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.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class CatalogTestBase extends TestBase { + + protected static final String DATABASE = "db"; + @TempDir protected File hiveWarehouse; + @TempDir protected File hadoopWarehouse; + + @Parameter(index = 0) + protected String catalogName; + + @Parameter(index = 1) + protected Namespace baseNamespace; + + protected Catalog validationCatalog; + protected SupportsNamespaces validationNamespaceCatalog; + protected Map config = Maps.newHashMap(); + + protected String flinkDatabase; + protected Namespace icebergNamespace; + protected boolean isHadoopCatalog; + + @Parameters(name = "catalogName={0}, baseNamespace={1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {"testhive", Namespace.empty()}, + new Object[] {"testhadoop", Namespace.empty()}, + new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); + } + + @BeforeEach + public void before() { + this.isHadoopCatalog = catalogName.startsWith("testhadoop"); + this.validationCatalog = + isHadoopCatalog + ? new HadoopCatalog(hiveConf, "file:" + hadoopWarehouse.getPath()) + : catalog; + this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; + + config.put("type", "iceberg"); + if (!baseNamespace.isEmpty()) { + config.put(FlinkCatalogFactory.BASE_NAMESPACE, baseNamespace.toString()); + } + if (isHadoopCatalog) { + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"); + } else { + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + config.put(CatalogProperties.URI, getURI(hiveConf)); + } + config.put(CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouseRoot())); + + this.flinkDatabase = catalogName + "." + DATABASE; + this.icebergNamespace = + Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE})); + sql("CREATE CATALOG %s WITH %s", catalogName, toWithClause(config)); + } + + @AfterEach + public void clean() { + dropCatalog(catalogName, true); + } + + protected String warehouseRoot() { + if (isHadoopCatalog) { + return hadoopWarehouse.getAbsolutePath(); + } else { + return hiveWarehouse.getAbsolutePath(); + } + } + + protected String getFullQualifiedTableName(String tableName) { + final List levels = Lists.newArrayList(icebergNamespace.levels()); + levels.add(tableName); + return Joiner.on('.').join(levels); + } + + static String getURI(HiveConf conf) { + return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java new file mode 100644 index 000000000000..b1e3b20ff7ac --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java @@ -0,0 +1,42 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericRecord; + +/** + * This interface defines test data generator. Different implementations for primitive and complex + * nested fields are defined in {@link DataGenerators}. + */ +public interface DataGenerator { + Schema icebergSchema(); + + RowType flinkRowType(); + + org.apache.avro.Schema avroSchema(); + + GenericRecord generateIcebergGenericRecord(); + + GenericRowData generateFlinkRowData(); + + org.apache.avro.generic.GenericRecord generateAvroGenericRecord(); +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java new file mode 100644 index 000000000000..e2cd411d7069 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java @@ -0,0 +1,1172 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import com.fasterxml.jackson.databind.node.IntNode; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Arrays; +import java.util.List; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import org.apache.avro.LogicalTypes; +import org.apache.avro.SchemaBuilder; +import org.apache.avro.generic.GenericData; +import org.apache.avro.util.Utf8; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.joda.time.DateTime; +import org.joda.time.DateTimeZone; +import org.joda.time.Days; + +/** + * Util class to generate test data with extensive coverage different field types: from primitives + * to complex nested types. + */ +public class DataGenerators { + + public static class Primitives implements DataGenerator { + private static final DateTime JODA_DATETIME_EPOC = + new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC); + private static final DateTime JODA_DATETIME_20220110 = + new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); + private static final int DAYS_BTW_EPOC_AND_20220110 = + Days.daysBetween(JODA_DATETIME_EPOC, JODA_DATETIME_20220110).getDays(); + private static final int HOUR_8_IN_MILLI = (int) TimeUnit.HOURS.toMillis(8); + + private static final LocalDate JAVA_LOCAL_DATE_20220110 = LocalDate.of(2022, 1, 10); + private static final LocalTime JAVA_LOCAL_TIME_HOUR8 = LocalTime.of(8, 0); + private static final OffsetDateTime JAVA_OFFSET_DATE_TIME_20220110 = + OffsetDateTime.of(2022, 1, 10, 0, 0, 0, 0, ZoneOffset.UTC); + private static final LocalDateTime JAVA_LOCAL_DATE_TIME_20220110 = + LocalDateTime.of(2022, 1, 10, 0, 0, 0); + private static final BigDecimal BIG_DECIMAL_NEGATIVE = new BigDecimal("-1.50"); + private static final byte[] FIXED_BYTES = "012345689012345".getBytes(StandardCharsets.UTF_8); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + // primitive types + Types.NestedField.optional(2, "boolean_field", Types.BooleanType.get()), + Types.NestedField.optional(3, "int_field", Types.IntegerType.get()), + Types.NestedField.optional(4, "long_field", Types.LongType.get()), + Types.NestedField.optional(5, "float_field", Types.FloatType.get()), + Types.NestedField.optional(6, "double_field", Types.DoubleType.get()), + Types.NestedField.required(7, "string_field", Types.StringType.get()), + Types.NestedField.required(8, "date_field", Types.DateType.get()), + Types.NestedField.required(9, "time_field", Types.TimeType.get()), + Types.NestedField.required(10, "ts_with_zone_field", Types.TimestampType.withZone()), + Types.NestedField.required( + 11, "ts_without_zone_field", Types.TimestampType.withoutZone()), + Types.NestedField.required(12, "uuid_field", Types.UUIDType.get()), + Types.NestedField.required(13, "binary_field", Types.BinaryType.get()), + Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2)), + Types.NestedField.required(15, "fixed_field", Types.FixedType.ofLength(16))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + /** + * Fix up Avro Schema that is converted from Iceberg Schema. + * + * @param schemaConvertedFromIceberg Avro Schema converted from Iceberg schema via {@link + * AvroSchemaUtil#convert(Schema, String)} + */ + private org.apache.avro.Schema fixupAvroSchemaConvertedFromIcebergSchema( + org.apache.avro.Schema schemaConvertedFromIceberg) { + List fixedFields = + schemaConvertedFromIceberg.getFields().stream() + .map( + field -> { + org.apache.avro.Schema.Field updatedField = field; + if (field.name().equals("time_field")) { + // Iceberg's AvroSchemaUtil uses timestamp-micros with Long value for time + // field, while AvroToRowDataConverters#convertToTime() always looks for + // Integer value assuming millis. The root problem is that + // AvroToRowDataConverters#createConverter() uses LogicalTypeRoot to + // determine converter and LogicalTypeRoot lost the timestamp precision + // carried by LogicalType like Time(6). + org.apache.avro.Schema fieldSchema = + LogicalTypes.timeMillis() + .addToSchema( + org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); + updatedField = new org.apache.avro.Schema.Field("time_field", fieldSchema); + } + + return new org.apache.avro.Schema.Field(updatedField, updatedField.schema()); + }) + .collect(Collectors.toList()); + return org.apache.avro.Schema.createRecord( + schemaConvertedFromIceberg.getName(), + schemaConvertedFromIceberg.getDoc(), + schemaConvertedFromIceberg.getNamespace(), + schemaConvertedFromIceberg.isError(), + fixedFields); + } + + private final org.apache.avro.Schema avroSchema = + fixupAvroSchemaConvertedFromIcebergSchema(AvroSchemaUtil.convert(icebergSchema, "table")); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("boolean_field", false); + genericRecord.setField("int_field", Integer.MAX_VALUE); + genericRecord.setField("long_field", Long.MAX_VALUE); + genericRecord.setField("float_field", Float.MAX_VALUE); + genericRecord.setField("double_field", Double.MAX_VALUE); + genericRecord.setField("string_field", "str"); + + genericRecord.setField("date_field", JAVA_LOCAL_DATE_20220110); + genericRecord.setField("time_field", JAVA_LOCAL_TIME_HOUR8); + genericRecord.setField("ts_with_zone_field", JAVA_OFFSET_DATE_TIME_20220110); + genericRecord.setField("ts_without_zone_field", JAVA_LOCAL_DATE_TIME_20220110); + + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + + genericRecord.setField("uuid_field", UUID.nameUUIDFromBytes(uuidBytes)); + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + genericRecord.setField("binary_field", ByteBuffer.wrap(binaryBytes)); + + genericRecord.setField("decimal_field", BIG_DECIMAL_NEGATIVE); + genericRecord.setField("fixed_field", FIXED_BYTES); + + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + + return GenericRowData.of( + StringData.fromString("row_id_value"), + false, + Integer.MAX_VALUE, + Long.MAX_VALUE, + Float.MAX_VALUE, + Double.MAX_VALUE, + StringData.fromString("str"), + DAYS_BTW_EPOC_AND_20220110, + HOUR_8_IN_MILLI, + // Although Avro logical type for timestamp fields are in micro seconds, + // AvroToRowDataConverters only looks for long value in milliseconds. + TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), + TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), + uuidBytes, + binaryBytes, + DecimalData.fromBigDecimal(BIG_DECIMAL_NEGATIVE, 9, 2), + FIXED_BYTES); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", new Utf8("row_id_value")); + genericRecord.put("boolean_field", false); + genericRecord.put("int_field", Integer.MAX_VALUE); + genericRecord.put("long_field", Long.MAX_VALUE); + genericRecord.put("float_field", Float.MAX_VALUE); + genericRecord.put("double_field", Double.MAX_VALUE); + genericRecord.put("string_field", new Utf8("str")); + + genericRecord.put("date_field", DAYS_BTW_EPOC_AND_20220110); + genericRecord.put("time_field", HOUR_8_IN_MILLI); + // Although Avro logical type for timestamp fields are in micro seconds, + // AvroToRowDataConverters only looks for long value in milliseconds. + genericRecord.put("ts_with_zone_field", JODA_DATETIME_20220110.getMillis()); + genericRecord.put("ts_without_zone_field", JODA_DATETIME_20220110.getMillis()); + + byte[] uuidBytes = new byte[16]; + for (int i = 0; i < 16; ++i) { + uuidBytes[i] = (byte) i; + } + genericRecord.put("uuid_field", ByteBuffer.wrap(uuidBytes)); + + byte[] binaryBytes = new byte[7]; + for (int i = 0; i < 7; ++i) { + binaryBytes[i] = (byte) i; + } + genericRecord.put("binary_field", ByteBuffer.wrap(binaryBytes)); + + BigDecimal bigDecimal = new BigDecimal("-1.50"); + // unscaledValue().toByteArray() is to match the behavior of RowDataToAvroConverters from + // Flink for decimal type + genericRecord.put("decimal_field", ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray())); + + genericRecord.put("fixed_field", ByteBuffer.wrap(FIXED_BYTES)); + + return genericRecord; + } + } + + public static class StructOfPrimitive implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_primitive", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required(102, "name", Types.StringType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_primitive").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("name", "Jane"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_primitive", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of(1, StringData.fromString("Jane"))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_primitive").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("name", "Jane"); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_primitive", struct); + return genericRecord; + } + } + + public static class StructOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_array", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, "names", Types.ListType.ofRequired(201, Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_array").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("names", Arrays.asList("Jane", "Joe")); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_array", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + StringData[] names = {StringData.fromString("Jane"), StringData.fromString("Joe")}; + return GenericRowData.of( + StringData.fromString("row_id_value"), GenericRowData.of(1, new GenericArrayData(names))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_array").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("names", Arrays.asList("Jane", "Joe")); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_array", struct); + return genericRecord; + } + } + + public static class StructOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_map", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, + "names", + Types.MapType.ofRequired( + 201, 202, Types.StringType.get(), Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_map").type().asStructType().fields()); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("names", ImmutableMap.of("Jane", "female", "Joe", "male")); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_map", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of( + 1, + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), + StringData.fromString("female"), + StringData.fromString("Joe"), + StringData.fromString("male"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_map").schema(); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("names", ImmutableMap.of("Jane", new Utf8("female"), "Joe", new Utf8("male"))); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_map", struct); + return genericRecord; + } + } + + public static class StructOfStruct implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "struct_of_struct", + Types.StructType.of( + required(101, "id", Types.IntegerType.get()), + required( + 102, + "person_struct", + Types.StructType.of( + Types.NestedField.required(201, "name", Types.StringType.get()), + Types.NestedField.required(202, "address", Types.StringType.get())))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + Schema structSchema = + new Schema(icebergSchema.findField("struct_of_struct").type().asStructType().fields()); + Schema personSchema = + new Schema(structSchema.findField("person_struct").type().asStructType().fields()); + GenericRecord person = GenericRecord.create(personSchema); + person.setField("name", "Jane"); + person.setField("address", "Apple Park"); + GenericRecord struct = GenericRecord.create(structSchema); + struct.setField("id", 1); + struct.setField("person_struct", person); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("struct_of_struct", struct); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + GenericRowData.of( + 1, + GenericRowData.of( + StringData.fromString("Jane"), StringData.fromString("Apple Park")))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_struct").schema(); + org.apache.avro.Schema personSchema = structSchema.getField("person_struct").schema(); + org.apache.avro.generic.GenericRecord person = new GenericData.Record(personSchema); + person.put("name", "Jane"); + person.put("address", "Apple Park"); + org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); + struct.put("id", 1); + struct.put("person_struct", person); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("struct_of_struct", struct); + return genericRecord; + } + } + + public static class ArrayOfPrimitive implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, "array_of_int", Types.ListType.ofOptional(101, Types.IntegerType.get()))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("array_of_int", Arrays.asList(1, 2, 3)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + Integer[] arr = {1, 2, 3}; + return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(arr)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("array_of_int", Arrays.asList(1, 2, 3)); + return genericRecord; + } + } + + public static class ArrayOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "array_of_array", + Types.ListType.ofRequired( + 101, Types.ListType.ofRequired(201, Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + // non-primitive + Integer[] array1 = {1, 2, 3}; + Integer[] array2 = {4, 5, 6}; + GenericArrayData[] arrayOfArrays = { + new GenericArrayData(array1), new GenericArrayData(array2) + }; + return GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(arrayOfArrays)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); + return genericRecord; + } + } + + public static class ArrayOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "array_of_map", + Types.ListType.ofRequired( + 101, + Types.MapType.ofRequired( + 201, 202, Types.StringType.get(), Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "array_of_map", + Arrays.asList( + ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + GenericMapData[] array = { + new GenericMapData( + ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2)), + new GenericMapData( + ImmutableMap.of(StringData.fromString("Alice"), 3, StringData.fromString("Bob"), 4)) + }; + return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(array)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "array_of_map", + Arrays.asList( + ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); + return genericRecord; + } + } + + public static class ArrayOfStruct implements DataGenerator { + private final Types.StructType structType = + Types.StructType.of( + required(201, "id", Types.IntegerType.get()), + required(202, "name", Types.StringType.get())); + private final Schema structIcebergSchema = new Schema(structType.fields()); + private final org.apache.avro.Schema structAvroSchema = + AvroSchemaUtil.convert(structIcebergSchema, "struct"); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, "array_of_struct", Types.ListType.ofRequired(101, structType))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord struct1 = GenericRecord.create(structIcebergSchema); + struct1.setField("id", 1); + struct1.setField("name", "Jane"); + GenericRecord struct2 = GenericRecord.create(structIcebergSchema); + struct2.setField("id", 2); + struct2.setField("name", "Joe"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("array_of_struct", Arrays.asList(struct1, struct2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + GenericRowData[] structArray = { + GenericRowData.of(1, StringData.fromString("Jane")), + GenericRowData.of(2, StringData.fromString("Joe")) + }; + return GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(structArray)); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); + struct1.put("id", 1); + struct1.put("name", "Jane"); + org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); + struct2.put("id", 2); + struct2.put("name", "Joe"); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("array_of_struct", Arrays.asList(struct1, struct2)); + return genericRecord; + } + } + + public static class MapOfPrimitives implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "map_of_primitives", + Types.MapType.ofRequired( + 101, 102, Types.StringType.get(), Types.IntegerType.get()))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); + return genericRecord; + } + } + + public static class MapOfArray implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_array", + Types.MapType.ofRequired( + 101, + 102, + Types.StringType.get(), + Types.ListType.ofRequired(201, Types.IntegerType.get())))); + + private final RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return rowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_array", + ImmutableMap.of( + "Jane", Arrays.asList(1, 2, 3), + "Joe", Arrays.asList(4, 5, 6))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + Integer[] janeArray = {1, 2, 3}; + Integer[] joeArray = {4, 5, 6}; + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), + new GenericArrayData(janeArray), + StringData.fromString("Joe"), + new GenericArrayData(joeArray)))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "map_of_array", + ImmutableMap.of( + "Jane", Arrays.asList(1, 2, 3), + "Joe", Arrays.asList(4, 5, 6))); + return genericRecord; + } + } + + public static class MapOfMap implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_map", + Types.MapType.ofRequired( + 101, + 102, + Types.StringType.get(), + Types.MapType.ofRequired( + 301, 302, Types.StringType.get(), Types.IntegerType.get())))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + private final org.apache.avro.Schema avroSchema = + AvroSchemaUtil.convert(icebergSchema, "table"); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_map", + ImmutableMap.of( + "female", ImmutableMap.of("Jane", 1, "Alice", 2), + "male", ImmutableMap.of("Joe", 3, "Bob", 4))); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("female"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Jane"), 1, StringData.fromString("Alice"), 2)), + StringData.fromString("male"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("Joe"), 3, StringData.fromString("Bob"), 4))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", "row_id_value"); + genericRecord.put( + "map_of_map", + ImmutableMap.of( + "female", ImmutableMap.of("Jane", 1, "Alice", 2), + "male", ImmutableMap.of("Joe", 3, "Bob", 4))); + return genericRecord; + } + } + + public static class MapOfStruct implements DataGenerator { + private org.apache.avro.Schema createAvroSchemaIdField() { + org.apache.avro.Schema schema = SchemaBuilder.builder().intType(); + // this is needed to match the converter generated schema props + schema.addProp("field-id", IntNode.valueOf(201)); + return schema; + } + + private org.apache.avro.Schema createAvroSchemaNameField() { + org.apache.avro.Schema schema = SchemaBuilder.builder().stringType(); + // this is needed to match the converter generated schema props + schema.addProp("field-id", IntNode.valueOf(202)); + return schema; + } + + private final Types.StructType structType = + Types.StructType.of( + required(201, "id", Types.IntegerType.get()), + required(202, "name", Types.StringType.get())); + private final Schema structIcebergSchema = new Schema(structType.fields()); + + private final org.apache.avro.Schema structAvroSchema = + SchemaBuilder.builder() + .record("struct") + .fields() + .name("id") + .type(createAvroSchemaIdField()) + .noDefault() + .name("name") + .type(createAvroSchemaNameField()) + .noDefault() + .endRecord(); + + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.required( + 2, + "map_of_struct", + Types.MapType.ofRequired(101, 102, Types.StringType.get(), structType))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + // Can't use AvroSchemaUtil.convert otherwise the nested schema will have generated name like + // `r102` not the specified name like `struct`. + org.apache.avro.Schema avroSchema = + SchemaBuilder.builder() + .record("table") + .fields() + .requiredString("row_id") + .name("map_of_struct") + .type(SchemaBuilder.builder().map().values(structAvroSchema)) + .noDefault() + .endRecord(); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + return avroSchema; + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + GenericRecord struct1 = GenericRecord.create(structIcebergSchema); + struct1.setField("id", 1); + struct1.setField("name", "Jane"); + GenericRecord struct2 = GenericRecord.create(structIcebergSchema); + struct2.setField("id", 2); + struct2.setField("name", "Joe"); + GenericRecord genericRecord = GenericRecord.create(icebergSchema); + genericRecord.setField("row_id", "row_id_value"); + genericRecord.setField( + "map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); + return genericRecord; + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + StringData.fromString("struct1"), + GenericRowData.of(1, StringData.fromString("Jane")), + StringData.fromString("struct2"), + GenericRowData.of(2, StringData.fromString("Joe"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); + struct1.put("id", 1); + struct1.put("name", new Utf8("Jane")); + org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); + struct2.put("id", 2); + struct2.put("name", new Utf8("Joe")); + org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); + genericRecord.put("row_id", new Utf8("row_id_value")); + genericRecord.put("map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); + return genericRecord; + } + } + + public static class MapOfStructStruct implements DataGenerator { + private final Schema icebergSchema = + new Schema( + Types.NestedField.required(1, "row_id", Types.StringType.get()), + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get()), + Types.NestedField.optional(202, "keyData", Types.StringType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()), + Types.NestedField.optional(204, "valueData", Types.StringType.get()))))); + + private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); + + @Override + public Schema icebergSchema() { + return icebergSchema; + } + + @Override + public RowType flinkRowType() { + return flinkRowType; + } + + @Override + public org.apache.avro.Schema avroSchema() { + throw new UnsupportedOperationException( + "Not applicable as Avro Map only support string key type"); + } + + @Override + public GenericRecord generateIcebergGenericRecord() { + throw new UnsupportedOperationException("Not implemented yet"); + } + + @Override + public GenericRowData generateFlinkRowData() { + return GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of( + GenericRowData.of(1L, StringData.fromString("key_data")), + GenericRowData.of(1L, StringData.fromString("value_data"))))); + } + + @Override + public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { + throw new UnsupportedOperationException("Avro Map only support string key type"); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java new file mode 100644 index 000000000000..fd5c6b76b683 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java @@ -0,0 +1,105 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.UUID; +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.extension.AfterAllCallback; +import org.junit.jupiter.api.extension.AfterEachCallback; +import org.junit.jupiter.api.extension.BeforeAllCallback; +import org.junit.jupiter.api.extension.BeforeEachCallback; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopCatalogExtension + implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, AfterEachCallback { + protected final String database; + protected final String tableName; + + protected Path temporaryFolder; + protected Catalog catalog; + protected CatalogLoader catalogLoader; + protected String warehouse; + protected TableLoader tableLoader; + + public HadoopCatalogExtension(String database, String tableName) { + this.database = database; + this.tableName = tableName; + } + + @Override + public void beforeAll(ExtensionContext context) throws Exception { + this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-"); + } + + @Override + public void afterAll(ExtensionContext context) throws Exception { + FileUtils.deleteDirectory(temporaryFolder.toFile()); + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + assertThat(temporaryFolder).exists().isDirectory(); + this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); + this.catalogLoader = + CatalogLoader.hadoop( + "hadoop", + new Configuration(), + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + this.catalog = catalogLoader.loadCatalog(); + this.tableLoader = + TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); + } + + @Override + public void afterEach(ExtensionContext context) throws Exception { + try { + catalog.dropTable(TableIdentifier.of(database, tableName)); + ((HadoopCatalog) catalog).close(); + tableLoader.close(); + } catch (Exception e) { + throw new RuntimeException("Failed to close catalog resource"); + } + } + + public TableLoader tableLoader() { + return tableLoader; + } + + public Catalog catalog() { + return catalog; + } + + public CatalogLoader catalogLoader() { + return catalogLoader; + } + + public String warehouse() { + return warehouse; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java new file mode 100644 index 000000000000..dc6ef400a4a9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.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; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.junit.jupiter.api.extension.ExtensionContext; + +public class HadoopTableExtension extends HadoopCatalogExtension { + private final Schema schema; + private final PartitionSpec partitionSpec; + + private Table table; + + public HadoopTableExtension(String database, String tableName, Schema schema) { + this(database, tableName, schema, null); + } + + public HadoopTableExtension( + String database, String tableName, Schema schema, PartitionSpec partitionSpec) { + super(database, tableName); + this.schema = schema; + this.partitionSpec = partitionSpec; + } + + @Override + public void beforeEach(ExtensionContext context) throws Exception { + super.beforeEach(context); + if (partitionSpec == null) { + this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); + } else { + this.table = + catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); + } + tableLoader.open(); + } + + public Table table() { + return table; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java new file mode 100644 index 000000000000..d2e086aa448e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.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; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; + +public class MiniFlinkClusterExtension { + + private static final int DEFAULT_TM_NUM = 1; + private static final int DEFAULT_PARALLELISM = 4; + + public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + private MiniFlinkClusterExtension() {} + + /** + * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't + * break the unit tests because of the class loader leak issue. In our iceberg integration tests, + * there're some that will assert the results after finished the flink jobs, so actually we may + * access the class loader that has been closed by the flink task managers if we enable the switch + * classloader.check-leaked-classloader by default. + */ + public static MiniClusterExtension createWithClassloaderCheckDisabled() { + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + } + + public static MiniClusterExtension createWithClassloaderCheckDisabled( + InMemoryReporter inMemoryReporter) { + Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); + inMemoryReporter.addToConfiguration(configuration); + + return new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(DEFAULT_TM_NUM) + .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) + .setConfiguration(configuration) + .build()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java new file mode 100644 index 000000000000..e532fb62615c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java @@ -0,0 +1,135 @@ +/* + * 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; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; + +public class RowDataConverter { + private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); + private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); + + private RowDataConverter() {} + + public static RowData convert(Schema iSchema, Record record) { + return convert(iSchema.asStruct(), record); + } + + private static RowData convert(Types.StructType struct, Record record) { + GenericRowData rowData = new GenericRowData(struct.fields().size()); + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Types.NestedField field = fields.get(i); + + Type fieldType = field.type(); + rowData.setField(i, convert(fieldType, record.get(i))); + } + return rowData; + } + + private static Object convert(Type type, Object object) { + if (object == null) { + return null; + } + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + case FIXED: + return object; + case DATE: + return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); + case TIME: + // Iceberg's time is in microseconds, while flink's time is in milliseconds. + LocalTime localTime = (LocalTime) object; + return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); + } else { + return TimestampData.fromLocalDateTime((LocalDateTime) object); + } + case STRING: + return StringData.fromString((String) object); + case UUID: + UUID uuid = (UUID) object; + ByteBuffer bb = ByteBuffer.allocate(16); + bb.putLong(uuid.getMostSignificantBits()); + bb.putLong(uuid.getLeastSignificantBits()); + return bb.array(); + case BINARY: + ByteBuffer buffer = (ByteBuffer) object; + return Arrays.copyOfRange( + buffer.array(), + buffer.arrayOffset() + buffer.position(), + buffer.arrayOffset() + buffer.remaining()); + case DECIMAL: + Types.DecimalType decimalType = (Types.DecimalType) type; + return DecimalData.fromBigDecimal( + (BigDecimal) object, decimalType.precision(), decimalType.scale()); + case STRUCT: + return convert(type.asStructType(), (Record) object); + case LIST: + List list = (List) object; + Object[] convertedArray = new Object[list.size()]; + for (int i = 0; i < convertedArray.length; i++) { + convertedArray[i] = convert(type.asListType().elementType(), list.get(i)); + } + return new GenericArrayData(convertedArray); + case MAP: + Map convertedMap = Maps.newLinkedHashMap(); + Map map = (Map) object; + for (Map.Entry entry : map.entrySet()) { + convertedMap.put( + convert(type.asMapType().keyType(), entry.getKey()), + convert(type.asMapType().valueType(), entry.getValue())); + } + return new GenericMapData(convertedMap); + default: + throw new UnsupportedOperationException("Not a supported type: " + type); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java new file mode 100644 index 000000000000..1767f774922a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java @@ -0,0 +1,439 @@ +/* + * 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; + +import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +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.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableScan; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.InternalRecordWrapper; +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.EncryptedOutputFile; +import org.apache.iceberg.flink.sink.FlinkAppenderFactory; +import org.apache.iceberg.hadoop.HadoopInputFile; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +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.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.StructLikeSet; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; + +public class SimpleDataUtil { + + private SimpleDataUtil() {} + + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + public static final TableSchema FLINK_SCHEMA = + TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); + + public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); + + public static final Record RECORD = GenericRecord.create(SCHEMA); + + public static Table createTable( + String path, Map properties, boolean partitioned) { + PartitionSpec spec; + if (partitioned) { + spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + return new HadoopTables().create(SCHEMA, spec, properties, path); + } + + public static Record createRecord(Integer id, String data) { + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + return record; + } + + public static RowData createRowData(Integer id, String data) { + return GenericRowData.of(id, StringData.fromString(data)); + } + + public static RowData createInsert(Integer id, String data) { + return GenericRowData.ofKind(RowKind.INSERT, id, StringData.fromString(data)); + } + + public static RowData createDelete(Integer id, String data) { + return GenericRowData.ofKind(RowKind.DELETE, id, StringData.fromString(data)); + } + + public static RowData createUpdateBefore(Integer id, String data) { + return GenericRowData.ofKind(RowKind.UPDATE_BEFORE, id, StringData.fromString(data)); + } + + public static RowData createUpdateAfter(Integer id, String data) { + return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); + } + + public static DataFile writeFile( + Table table, + Schema schema, + PartitionSpec spec, + Configuration conf, + String location, + String filename, + List rows) + throws IOException { + return writeFile(table, schema, spec, conf, location, filename, rows, null); + } + + /** Write the list of {@link RowData} to the given path and with the given partition data */ + public static DataFile writeFile( + Table table, + Schema schema, + PartitionSpec spec, + Configuration conf, + String location, + String filename, + List rows, + StructLike partition) + throws IOException { + Path path = new Path(location, filename); + FileFormat fileFormat = FileFormat.fromFileName(filename); + Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); + + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, schema, flinkSchema, ImmutableMap.of(), spec, null, null, null); + + FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); + try (FileAppender closeableAppender = appender) { + closeableAppender.addAll(rows); + } + + DataFiles.Builder builder = + DataFiles.builder(spec) + .withInputFile(HadoopInputFile.fromPath(path, conf)) + .withMetrics(appender.metrics()); + + if (partition != null) { + builder = builder.withPartition(partition); + } + + return builder.build(); + } + + public static DeleteFile writeEqDeleteFile( + Table table, + FileFormat format, + String filename, + FileAppenderFactory appenderFactory, + List deletes) + throws IOException { + EncryptedOutputFile outputFile = + table + .encryption() + .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); + + EqualityDeleteWriter eqWriter = + appenderFactory.newEqDeleteWriter(outputFile, format, null); + try (EqualityDeleteWriter writer = eqWriter) { + writer.write(deletes); + } + return eqWriter.toDeleteFile(); + } + + public static DeleteFile writePosDeleteFile( + Table table, + FileFormat format, + String filename, + FileAppenderFactory appenderFactory, + List> positions) + throws IOException { + EncryptedOutputFile outputFile = + table + .encryption() + .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); + + PositionDeleteWriter posWriter = + appenderFactory.newPosDeleteWriter(outputFile, format, null); + PositionDelete posDelete = PositionDelete.create(); + try (PositionDeleteWriter writer = posWriter) { + for (Pair p : positions) { + writer.write(posDelete.set(p.first(), p.second(), null)); + } + } + return posWriter.toDeleteFile(); + } + + private static List convertToRecords(List rows) { + List records = Lists.newArrayList(); + for (RowData row : rows) { + Integer id = row.isNullAt(0) ? null : row.getInt(0); + String data = row.isNullAt(1) ? null : row.getString(1).toString(); + records.add(createRecord(id, data)); + } + return records; + } + + public static void assertTableRows(String tablePath, List expected, String branch) + throws IOException { + assertTableRecords(tablePath, convertToRecords(expected), branch); + } + + public static void assertTableRows(Table table, List expected) throws IOException { + assertTableRecords(table, convertToRecords(expected), SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRows(Table table, List expected, String branch) + throws IOException { + assertTableRecords(table, convertToRecords(expected), branch); + } + + /** Get all rows for a table */ + public static List tableRecords(Table table) throws IOException { + table.refresh(); + List records = Lists.newArrayList(); + try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { + for (Record record : iterable) { + records.add(record); + } + } + return records; + } + + public static boolean equalsRecords(List expected, List actual, Schema schema) { + if (expected.size() != actual.size()) { + return false; + } + Types.StructType type = schema.asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + StructLikeSet actualSet = StructLikeSet.create(type); + actualSet.addAll(actual); + return expectedSet.equals(actualSet); + } + + public static void assertRecordsEqual(List expected, List actual, Schema schema) { + assertThat(actual).hasSameSizeAs(expected); + Types.StructType type = schema.asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + StructLikeSet actualSet = StructLikeSet.create(type); + actualSet.addAll(actual); + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); + } + + /** + * Assert table contains the expected list of records after waiting up to the configured {@code + * timeout} + */ + public static void assertTableRecords(Table table, List expected, Duration timeout) { + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted(() -> assertRecordsEqual(expected, tableRecords(table), table.schema())); + } + + public static void assertTableRecords(Table table, List expected) throws IOException { + assertTableRecords(table, expected, SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRecords(Table table, List expected, String branch) + throws IOException { + table.refresh(); + Snapshot snapshot = latestSnapshot(table, branch); + + if (snapshot == null) { + assertThat(expected).isEmpty(); + return; + } + + Types.StructType type = table.schema().asStruct(); + StructLikeSet expectedSet = StructLikeSet.create(type); + expectedSet.addAll(expected); + + try (CloseableIterable iterable = + IcebergGenerics.read(table).useSnapshot(snapshot.snapshotId()).build()) { + StructLikeSet actualSet = StructLikeSet.create(type); + + for (Record record : iterable) { + actualSet.add(record); + } + + assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); + } + } + + // Returns the latest snapshot of the given branch in the table + public static Snapshot latestSnapshot(Table table, String branch) { + // For the main branch, currentSnapshot() is used to validate that the API behavior has + // not changed since that was the API used for validation prior to addition of branches. + if (branch.equals(SnapshotRef.MAIN_BRANCH)) { + return table.currentSnapshot(); + } + + return table.snapshot(branch); + } + + public static void assertTableRecords(String tablePath, List expected) + throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + assertTableRecords(new HadoopTables().load(tablePath), expected, SnapshotRef.MAIN_BRANCH); + } + + public static void assertTableRecords(String tablePath, List expected, String branch) + throws IOException { + Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); + assertTableRecords(new HadoopTables().load(tablePath), expected, branch); + } + + public static StructLikeSet expectedRowSet(Table table, Record... records) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + for (Record record : records) { + set.add(wrapper.copyFor(record)); + } + return set; + } + + public static StructLikeSet actualRowSet(Table table, String... columns) throws IOException { + return actualRowSet(table, null, columns); + } + + public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... columns) + throws IOException { + table.refresh(); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table) + .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) + .select(columns) + .build()) { + reader.forEach(record -> set.add(wrapper.copyFor(record))); + } + return set; + } + + public static List partitionDataFiles(Table table, Map partitionValues) + throws IOException { + table.refresh(); + Types.StructType partitionType = table.spec().partitionType(); + + Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); + StructLikeWrapper expectedWrapper = + StructLikeWrapper.forType(partitionType).set(partitionRecord); + + List dataFiles = Lists.newArrayList(); + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + StructLikeWrapper wrapper = + StructLikeWrapper.forType(partitionType).set(scanTask.file().partition()); + + if (expectedWrapper.equals(wrapper)) { + dataFiles.add(scanTask.file()); + } + } + } + + return dataFiles; + } + + public static Map> snapshotToDataFiles(Table table) throws IOException { + table.refresh(); + + Map> result = Maps.newHashMap(); + Snapshot current = table.currentSnapshot(); + while (current != null) { + TableScan tableScan = table.newScan(); + if (current.parentId() != null) { + // Collect the data files that was added only in current snapshot. + tableScan = tableScan.appendsBetween(current.parentId(), current.snapshotId()); + } else { + // Collect the data files that was added in the oldest snapshot. + tableScan = tableScan.useSnapshot(current.snapshotId()); + } + try (CloseableIterable scanTasks = tableScan.planFiles()) { + result.put( + current.snapshotId(), + ImmutableList.copyOf(Iterables.transform(scanTasks, FileScanTask::file))); + } + + // Continue to traverse the parent snapshot if exists. + if (current.parentId() == null) { + break; + } + // Iterate to the parent snapshot. + current = table.snapshot(current.parentId()); + } + return result; + } + + public static List matchingPartitions( + List dataFiles, PartitionSpec partitionSpec, Map partitionValues) { + Types.StructType partitionType = partitionSpec.partitionType(); + Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); + StructLikeWrapper expected = StructLikeWrapper.forType(partitionType).set(partitionRecord); + return dataFiles.stream() + .filter( + df -> { + StructLikeWrapper wrapper = + StructLikeWrapper.forType(partitionType).set(df.partition()); + return wrapper.equals(expected); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/SqlBase.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.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +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.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } + + protected static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java new file mode 100644 index 000000000000..401960c3591b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -0,0 +1,145 @@ +/* + * 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; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.List; +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.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public abstract class TestBase extends SqlBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir protected Path temporaryDirectory; + + private static TestHiveMetastore metastore = null; + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + + private volatile TableEnvironment tEnv = null; + + @BeforeAll + public static void startMetastore() { + TestBase.metastore = new TestHiveMetastore(); + metastore.start(); + TestBase.hiveConf = metastore.hiveConf(); + TestBase.catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + TestBase.catalog = null; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); + + TableEnvironment env = TableEnvironment.create(settings); + env.getConfig() + .getConfiguration() + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + tEnv = env; + } + } + } + return tEnv; + } + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java new file mode 100644 index 000000000000..e8f65921c19a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.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; + +import static org.apache.iceberg.CatalogProperties.URI; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.entry; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.Map; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +/** Test for {@link CatalogLoader}. */ +public class TestCatalogLoader extends TestBase { + + private static File warehouse = null; + private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); + private static final Schema SCHEMA = + new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); + + @BeforeAll + public static void createWarehouse() throws IOException { + warehouse = File.createTempFile("warehouse", null); + assertThat(warehouse.delete()).isTrue(); + hiveConf.set("my_key", "my_value"); + } + + @AfterAll + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); + } + } + + @Test + public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException { + Map properties = Maps.newHashMap(); + properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse); + CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties); + validateCatalogLoader(loader); + } + + @Test + public void testHiveCatalogLoader() throws IOException, ClassNotFoundException { + CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + validateCatalogLoader(loader); + } + + @Test + public void testRESTCatalogLoader() { + Map properties = Maps.newHashMap(); + properties.put(URI, "http://localhost/"); + CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap()); + } + + private static void validateCatalogLoader(CatalogLoader loader) + throws IOException, ClassNotFoundException { + Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); + validateHadoopConf(table); + } + + private static void validateHadoopConf(Table table) { + FileIO io = table.io(); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); + HadoopFileIO hadoopIO = (HadoopFileIO) io; + assertThat(hadoopIO.conf()).contains(entry("my_key", "my_value")); + } + + @SuppressWarnings("unchecked") + private static T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(object); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + return (T) in.readObject(); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java new file mode 100644 index 000000000000..f719c7bc0001 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.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.flink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +/** Test for {@link TableLoader}. */ +public class TestCatalogTableLoader extends TestBase { + + private static File warehouse = null; + private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); + private static final Schema SCHEMA = + new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); + + @BeforeAll + public static void createWarehouse() throws IOException { + warehouse = File.createTempFile("warehouse", null); + assertThat(warehouse.delete()).isTrue(); + hiveConf.set("my_key", "my_value"); + } + + @AfterAll + public static void dropWarehouse() throws IOException { + if (warehouse != null && warehouse.exists()) { + Path warehousePath = new Path(warehouse.getAbsolutePath()); + FileSystem fs = warehousePath.getFileSystem(hiveConf); + assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); + } + } + + @Test + public void testHadoopTableLoader() throws IOException, ClassNotFoundException { + String location = "file:" + warehouse + "/my_table"; + new HadoopTables(hiveConf).create(SCHEMA, location); + validateTableLoader(TableLoader.fromHadoopTable(location, hiveConf)); + } + + @Test + public void testHiveCatalogTableLoader() throws IOException, ClassNotFoundException { + CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + javaSerdes(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); + + CatalogLoader catalogLoader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); + validateTableLoader(TableLoader.fromCatalog(catalogLoader, IDENTIFIER)); + } + + private static void validateTableLoader(TableLoader loader) + throws IOException, ClassNotFoundException { + TableLoader copied = javaSerdes(loader); + copied.open(); + try { + validateHadoopConf(copied.loadTable()); + } finally { + copied.close(); + } + } + + private static void validateHadoopConf(Table table) { + FileIO io = table.io(); + assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); + HadoopFileIO hadoopIO = (HadoopFileIO) io; + assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); + } + + @SuppressWarnings("unchecked") + private static T javaSerdes(T object) throws IOException, ClassNotFoundException { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(object); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + return (T) in.readObject(); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java new file mode 100644 index 000000000000..1997ef6998a2 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java @@ -0,0 +1,296 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.flink.source.ChangeLogTableTestBase; +import org.apache.iceberg.relocated.com.google.common.base.Joiner; +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.util.StructLikeSet; +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; + +/** + * In this test case, we mainly cover the impact of primary key selection, multiple operations + * within a single transaction, and multiple operations between different txn on the correctness of + * the data. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestChangeLogTable extends ChangeLogTableTestBase { + private static final Configuration CONF = new Configuration(); + private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; + + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String TABLE_NAME = "test_table"; + private String warehouse; + + @Parameter private boolean partitioned; + + @Parameters(name = "PartitionedTable={0}") + public static Iterable parameters() { + return ImmutableList.of(new Object[] {true}, new Object[] {false}); + } + + @BeforeEach + public void before() throws IOException { + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + // Set the table.exec.sink.upsert-materialize=NONE, so that downstream operators will receive + // the + // records with the same order as the source operator, bypassing Flink's inferred shuffle. + getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); + } + + @AfterEach + @Override + public void clean() { + sql("DROP TABLE IF EXISTS %s", TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + BoundedTableFactory.clearDataSets(); + } + + @TestTemplate + public void testSqlChangeLogOnIdKey() throws Exception { + List> inputRowsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(1, "bbb"), + insertRow(2, "aaa"), + deleteRow(2, "aaa"), + insertRow(2, "bbb")), + ImmutableList.of( + updateBeforeRow(2, "bbb"), + updateAfterRow(2, "ccc"), + deleteRow(2, "ccc"), + insertRow(2, "ddd")), + ImmutableList.of( + deleteRow(1, "bbb"), + insertRow(1, "ccc"), + deleteRow(1, "ccc"), + insertRow(1, "ddd"))); + + List> expectedRecordsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "bbb")), + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "ddd")), + ImmutableList.of(insertRow(1, "ddd"), insertRow(2, "ddd"))); + + testSqlChangeLog( + TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(1, "bbb"), + insertRow(2, "aaa")), + ImmutableList.of( + updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), + ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "ccc"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa")), + ImmutableList.of(insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(1, "ccc"), + insertRow(2, "aaa"), + insertRow(2, "ccc"))); + + testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(1, "bbb"), + insertRow(2, "aaa")), + ImmutableList.of( + updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), + ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(1, "ccc"), + insertRow(2, "aaa"), + insertRow(2, "bbb"))); + + testSqlChangeLog( + TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); + } + + @TestTemplate + public void testPureInsertOnIdKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), + ImmutableList.of(insertRow(3, "ccc"), insertRow(4, "ddd")), + ImmutableList.of(insertRow(5, "eee"), insertRow(6, "fff"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), + ImmutableList.of( + insertRow(1, "aaa"), insertRow(2, "bbb"), insertRow(3, "ccc"), insertRow(4, "ddd")), + ImmutableList.of( + insertRow(1, "aaa"), + insertRow(2, "bbb"), + insertRow(3, "ccc"), + insertRow(4, "ddd"), + insertRow(5, "eee"), + insertRow(6, "fff"))); + + testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); + } + + private static Record record(int id, String data) { + return SimpleDataUtil.createRecord(id, data); + } + + private Table createTable(String tableName, List key, boolean isPartitioned) { + String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : ""; + sql( + "CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) %s", + tableName, Joiner.on(',').join(key), partitionByCause); + + // Upgrade the iceberg table to format v2. + CatalogLoader loader = + CatalogLoader.hadoop( + "my_catalog", CONF, ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); + Table table = loader.loadCatalog().loadTable(TableIdentifier.of(DATABASE_NAME, TABLE_NAME)); + TableOperations ops = ((BaseTable) table).operations(); + TableMetadata meta = ops.current(); + ops.commit(meta, meta.upgradeToFormatVersion(2)); + + return table; + } + + private void testSqlChangeLog( + String tableName, + List key, + List> inputRowsPerCheckpoint, + List> expectedRecordsPerCheckpoint) + throws Exception { + String dataId = BoundedTableFactory.registerDataSet(inputRowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)).isEqualTo(listJoin(inputRowsPerCheckpoint)); + + Table table = createTable(tableName, key, partitioned); + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + table.refresh(); + List snapshots = findValidSnapshots(table); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots) + .as("Should have the expected snapshot number") + .hasSameSizeAs(expectedRecordsPerCheckpoint); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRows = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(table, snapshotId)) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(table, expectedRows)); + } + + if (expectedSnapshotNum > 0) { + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in the final table") + .containsExactlyInAnyOrderElementsOf( + expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)); + } + } + + private List findValidSnapshots(Table table) { + List validSnapshots = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.allManifests(table.io()).stream() + .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + validSnapshots.add(snapshot); + } + } + return validSnapshots; + } + + private static StructLikeSet expectedRowSet(Table table, List rows) { + Record[] records = new Record[rows.size()]; + for (int i = 0; i < records.length; i++) { + records[i] = record((int) rows.get(i).getField(0), (String) rows.get(i).getField(1)); + } + return SimpleDataUtil.expectedRowSet(table, records); + } + + private static StructLikeSet actualRowSet(Table table, long snapshotId) throws IOException { + return SimpleDataUtil.actualRowSet(table, snapshotId, "*"); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java new file mode 100644 index 000000000000..8992cbd75187 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java @@ -0,0 +1,203 @@ +/* + * 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; + +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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.util.Map; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestDataFileSerialization { + + private static final Schema DATE_SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec PARTITION_SPEC = + PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); + + private static final Map COLUMN_SIZES = Maps.newHashMap(); + private static final Map VALUE_COUNTS = Maps.newHashMap(); + private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); + private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); + private static final Map LOWER_BOUNDS = Maps.newHashMap(); + private static final Map UPPER_BOUNDS = Maps.newHashMap(); + + static { + COLUMN_SIZES.put(1, 2L); + COLUMN_SIZES.put(2, 3L); + VALUE_COUNTS.put(1, 5L); + VALUE_COUNTS.put(2, 3L); + VALUE_COUNTS.put(4, 2L); + NULL_VALUE_COUNTS.put(1, 0L); + NULL_VALUE_COUNTS.put(2, 2L); + NAN_VALUE_COUNTS.put(4, 1L); + LOWER_BOUNDS.put(1, longToBuffer(0L)); + UPPER_BOUNDS.put(1, longToBuffer(4L)); + } + + private static final Metrics METRICS = + new Metrics( + 5L, null, VALUE_COUNTS, NULL_VALUE_COUNTS, NAN_VALUE_COUNTS, LOWER_BOUNDS, UPPER_BOUNDS); + + private static final DataFile DATA_FILE = + DataFiles.builder(PARTITION_SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(1234) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withSplitOffsets(ImmutableList.of(4L)) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) + .withSortOrder(SortOrder.unsorted()) + .build(); + + private static final DeleteFile POS_DELETE_FILE = + FileMetadata.deleteFileBuilder(PARTITION_SPEC) + .ofPositionDeletes() + .withPath("/path/to/pos-delete.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .withRecordCount(23) + .build(); + + private static final DeleteFile EQ_DELETE_FILE = + FileMetadata.deleteFileBuilder(PARTITION_SPEC) + .ofEqualityDeletes(2, 3) + .withPath("/path/to/equality-delete.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("date=2018-06-08") + .withMetrics(METRICS) + .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) + .withRecordCount(23) + .withSortOrder(SortOrder.unsorted()) + .build(); + + @Test + public void testJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(DATA_FILE); + out.writeObject(DATA_FILE.copy()); + + out.writeObject(POS_DELETE_FILE); + out.writeObject(POS_DELETE_FILE.copy()); + + out.writeObject(EQ_DELETE_FILE); + out.writeObject(EQ_DELETE_FILE.copy()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); + TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); + } + + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a position DeleteFile").isInstanceOf(DeleteFile.class); + TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); + } + + for (int i = 0; i < 2; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a equality DeleteFile").isInstanceOf(DeleteFile.class); + TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); + } + } + } + + @Test + public void testDataFileKryoSerialization() throws IOException { + KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + kryo.serialize(DATA_FILE, outputView); + kryo.serialize(DATA_FILE.copy(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + DataFile dataFile1 = kryo.deserialize(inputView); + DataFile dataFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(DATA_FILE, dataFile1); + TestHelpers.assertEquals(DATA_FILE, dataFile2); + } + + @Test + public void testDeleteFileKryoSerialization() throws IOException { + KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + kryo.serialize(POS_DELETE_FILE, outputView); + kryo.serialize(POS_DELETE_FILE.copy(), outputView); + + kryo.serialize(EQ_DELETE_FILE, outputView); + kryo.serialize(EQ_DELETE_FILE.copy(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + + DeleteFile posDeleteFile1 = kryo.deserialize(inputView); + DeleteFile posDeleteFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile1); + TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile2); + + DeleteFile eqDeleteFile1 = kryo.deserialize(inputView); + DeleteFile eqDeleteFile2 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile1); + TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile2); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java new file mode 100644 index 000000000000..b9a7d5b1d589 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java @@ -0,0 +1,61 @@ +/* + * 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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; + +public class TestFixtures { + + private TestFixtures() {} + + public static final Schema SCHEMA = + new Schema( + required(1, "data", Types.StringType.get()), + required(2, "id", Types.LongType.get()), + required(3, "dt", Types.StringType.get())); + + public static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("dt").bucket("id", 1).build(); + + public static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + public static final String DATABASE = "default"; + public static final String TABLE = "t"; + public static final String SINK_TABLE = "t_sink"; + + public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); + public static final TableIdentifier SINK_TABLE_IDENTIFIER = + TableIdentifier.of(DATABASE, SINK_TABLE); + + public static final Schema TS_SCHEMA = + new Schema( + required(1, "ts", Types.TimestampType.withoutZone()), + required(2, "str", Types.StringType.get())); + + public static final PartitionSpec TS_SPEC = + PartitionSpec.builderFor(TS_SCHEMA).hour("ts").build(); + + public static final RowType TS_ROW_TYPE = FlinkSchemaUtil.convert(TS_SCHEMA); +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java new file mode 100644 index 000000000000..70c8043f8fbb --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.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.flink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Files; +import java.util.concurrent.TimeUnit; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Schema; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableDescriptor; +import org.apache.flink.table.api.TableEnvironment; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; + +public class TestFlinkAnonymousTable extends TestBase { + + @Test + public void testWriteAnonymousTable() throws Exception { + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); + TableEnvironment tEnv = getTableEnv(); + Table table = + tEnv.from( + TableDescriptor.forConnector("datagen") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("number-of-rows", "3") + .build()); + + TableDescriptor descriptor = + TableDescriptor.forConnector("iceberg") + .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) + .option("catalog-name", "hadoop_test") + .option("catalog-type", "hadoop") + .option("catalog-database", "test_db") + .option("catalog-table", "test") + .option("warehouse", warehouseDir.getAbsolutePath()) + .build(); + + table.insertInto(descriptor).execute(); + Awaitility.await() + .atMost(3, TimeUnit.SECONDS) + .untilAsserted( + () -> + assertThat(warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) + .exists()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java new file mode 100644 index 000000000000..bd07087756ad --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java @@ -0,0 +1,253 @@ +/* + * 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; + +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.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; +import org.apache.flink.types.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkCatalogDatabase extends CatalogTestBase { + + @AfterEach + @Override + public void clean() { + sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + @TestTemplate + public void testCreateNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should not already exist") + .isFalse(); + + sql("CREATE DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should exist") + .isTrue(); + + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should still exist") + .isTrue(); + + dropDatabase(flinkDatabase, true); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should be dropped") + .isFalse(); + + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Database should be created") + .isTrue(); + } + + @TestTemplate + public void testDropEmptyDatabase() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + dropDatabase(flinkDatabase, true); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should have been dropped") + .isFalse(); + } + + @TestTemplate + public void testDropNonEmptyNamespace() { + assumeThat(isHadoopCatalog) + .as("Hadoop catalog throws IOException: Directory is not empty.") + .isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) + .as("Table should exist") + .isTrue(); + assertThatThrownBy(() -> dropDatabase(flinkDatabase, true)) + .cause() + .isInstanceOf(DatabaseNotEmptyException.class) + .hasMessage( + String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); + sql("DROP TABLE %s.tl", flinkDatabase); + } + + @TestTemplate + public void testListTables() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + assertThat(sql("SHOW TABLES")).isEmpty(); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); + + List tables = sql("SHOW TABLES"); + assertThat(tables).hasSize(1); + assertThat("tl").as("Table name should match").isEqualTo(tables.get(0).getField(0)); + } + + @TestTemplate + public void testListNamespace() { + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + + List databases = sql("SHOW DATABASES"); + + if (isHadoopCatalog) { + assertThat(databases).hasSize(1); + assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); + if (!baseNamespace.isEmpty()) { + // test namespace not belongs to this catalog + validationNamespaceCatalog.createNamespace( + Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); + databases = sql("SHOW DATABASES"); + assertThat(databases).hasSize(1); + assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); + } + } else { + // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the + // creation for default + // database. See HiveMetaStore.HMSHandler.init. + assertThat(databases) + .as("Should have db database") + .anyMatch(d -> Objects.equals(d.getField(0), "db")); + } + } + + @TestTemplate + public void testCreateNamespaceWithMetadata() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("prop", "value"); + } + + @TestTemplate + public void testCreateNamespaceWithComment() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("comment", "namespace doc"); + } + + @TestTemplate + public void testCreateNamespaceWithLocation() throws Exception { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + + Path location = temporaryDirectory.getRoot(); + sql("CREATE DATABASE %s WITH ('location'='%s')", flinkDatabase, location); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("location", "file:" + location.getRoot()); + } + + @TestTemplate + public void testSetProperties() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + + sql("CREATE DATABASE %s", flinkDatabase); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should exist") + .isTrue(); + + Map defaultMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(defaultMetadata).doesNotContainKey("prop"); + sql("ALTER DATABASE %s SET ('prop'='value')", flinkDatabase); + Map nsMetadata = + validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); + assertThat(nsMetadata).containsEntry("prop", "value"); + } + + @TestTemplate + public void testHadoopNotSupportMeta() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isTrue(); + assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) + .as("Namespace should not already exist") + .isFalse(); + assertThatThrownBy(() -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) + .cause() + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage( + String.format( + "Cannot create namespace %s: metadata is not supported", icebergNamespace)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java new file mode 100644 index 000000000000..4c9e95b8fa82 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java @@ -0,0 +1,119 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.hive.HiveCatalog; +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.Test; + +public class TestFlinkCatalogFactory { + + private Map props; + + @BeforeEach + public void before() { + props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "/tmp/location"); + } + + @Test + public void testCreateCatalogHive() { + String catalogName = "hiveCatalog"; + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); + } + + @Test + public void testCreateCatalogHadoop() { + String catalogName = "hadoopCatalog"; + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); + } + + @Test + public void testCreateCatalogCustom() { + String catalogName = "customCatalog"; + props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); + + Catalog catalog = + FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) + .loadCatalog(); + + assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); + } + + @Test + public void testCreateCatalogCustomWithHiveCatalogTypeSet() { + String catalogName = "customCatalog"; + props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); + props.put( + FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); + + assertThatThrownBy( + () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith( + "Cannot create catalog customCatalog, both catalog-type and catalog-impl are set"); + } + + @Test + public void testLoadCatalogUnknown() { + String catalogName = "unknownCatalog"; + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); + + assertThatThrownBy( + () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessageStartingWith("Unknown catalog-type: fooType"); + } + + public static class CustomHadoopCatalog extends HadoopCatalog { + + public CustomHadoopCatalog() {} + + public CustomHadoopCatalog(Configuration conf, String warehouseLocation) { + setConf(conf); + initialize( + "custom", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation)); + } + } +} 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 new file mode 100644 index 000000000000..0b9c2193b4d5 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java @@ -0,0 +1,669 @@ +/* + * 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; + +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.Arrays; +import java.util.Collections; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.StreamSupport; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableException; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.api.constraints.UniqueConstraint; +import org.apache.flink.table.catalog.CatalogTable; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DataOperations; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkCatalogTable extends CatalogTestBase { + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @AfterEach + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); + sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + @TestTemplate + public void testGetTable() { + sql("CREATE TABLE tl(id BIGINT, strV STRING)"); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, "tl")); + Schema iSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "strV", Types.StringType.get())); + assertThat(table.schema().toString()) + .as("Should load the expected iceberg schema") + .isEqualTo(iSchema.toString()); + } + + @TestTemplate + public void testRenameTable() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support rename table").isFalse(); + final Schema tableSchema = + new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); + validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); + sql("ALTER TABLE tl RENAME TO tl2"); + + assertThatThrownBy(() -> getTableEnv().from("tl")) + .isInstanceOf(ValidationException.class) + .hasMessage("Table `tl` was not found."); + + Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); + assertThat(tableSchema.asStruct()).isEqualTo(actualSchema.asStruct()); + } + + @TestTemplate + public void testCreateTable() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT)"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + CatalogTable catalogTable = catalogTable("tl"); + assertThat(catalogTable.getSchema()) + .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); + } + + @TestTemplate + public void testCreateTableWithPrimaryKey() throws Exception { + sql("CREATE TABLE tl(id BIGINT, data STRING, key STRING PRIMARY KEY NOT ENFORCED)"); + + Table table = table("tl"); + assertThat(table.schema().identifierFieldIds()) + .as("Should have the expected row key.") + .isEqualTo(Sets.newHashSet(table.schema().findField("key").fieldId())); + CatalogTable catalogTable = catalogTable("tl"); + Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); + assertThat(uniqueConstraintOptional).isPresent(); + assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("key"); + } + + @TestTemplate + public void testCreateTableWithMultiColumnsInPrimaryKey() throws Exception { + sql( + "CREATE TABLE tl(id BIGINT, data STRING, CONSTRAINT pk_constraint PRIMARY KEY(data, id) NOT ENFORCED)"); + + Table table = table("tl"); + assertThat(table.schema().identifierFieldIds()) + .as("Should have the expected RowKey") + .isEqualTo( + Sets.newHashSet( + table.schema().findField("id").fieldId(), + table.schema().findField("data").fieldId())); + CatalogTable catalogTable = catalogTable("tl"); + Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); + assertThat(uniqueConstraintOptional).isPresent(); + assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("id", "data"); + } + + @TestTemplate + public void testCreateTableIfNotExists() { + sql("CREATE TABLE tl(id BIGINT)"); + + // Assert that table does exist. + assertThat(table("tl")).isNotNull(); + + sql("DROP TABLE tl"); + assertThatThrownBy(() -> table("tl")) + .isInstanceOf(NoSuchTableException.class) + .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); + + sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); + assertThat(table("tl").properties()).doesNotContainKey("key"); + + table("tl").updateProperties().set("key", "value").commit(); + assertThat(table("tl").properties()).containsEntry("key", "value"); + + sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); + assertThat(table("tl").properties()).containsEntry("key", "value"); + } + + @TestTemplate + public void testCreateTableLike() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT)"); + sql("CREATE TABLE tl2 LIKE tl"); + + Table table = table("tl2"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + CatalogTable catalogTable = catalogTable("tl2"); + assertThat(catalogTable.getSchema()) + .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); + } + + @TestTemplate + public void testCreateTableLocation() { + assumeThat(isHadoopCatalog) + .as("HadoopCatalog does not support creating table with location") + .isFalse(); + sql("CREATE TABLE tl(id BIGINT) WITH ('location'='file:///tmp/location')"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + assertThat(table.location()).isEqualTo("file:///tmp/location"); + } + + @TestTemplate + public void testCreatePartitionTable() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT, dt STRING) PARTITIONED BY(dt)"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + assertThat(table.spec()) + .isEqualTo(PartitionSpec.builderFor(table.schema()).identity("dt").build()); + CatalogTable catalogTable = catalogTable("tl"); + assertThat(catalogTable.getSchema()) + .isEqualTo( + TableSchema.builder() + .field("id", DataTypes.BIGINT()) + .field("dt", DataTypes.STRING()) + .build()); + assertThat(catalogTable.getPartitionKeys()).isEqualTo(Collections.singletonList("dt")); + } + + @TestTemplate + public void testCreateTableWithColumnComment() { + sql("CREATE TABLE tl(id BIGINT COMMENT 'comment - id', data STRING COMMENT 'comment - data')"); + + Table table = table("tl"); + assertThat(table.schema().asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get(), "comment - id"), + Types.NestedField.optional(2, "data", Types.StringType.get(), "comment - data")) + .asStruct()); + } + + @TestTemplate + public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); + + Table table = table("tl"); + assertThat(((BaseTable) table).operations().current().formatVersion()).isEqualTo(2); + } + + @TestTemplate + public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); + + Table table = table("tl"); + TableOperations ops = ((BaseTable) table).operations(); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v1") + .isEqualTo(1); + sql("ALTER TABLE tl SET('format-version'='2')"); + assertThat(ops.refresh().formatVersion()) + .as("should update table to use format v2") + .isEqualTo(2); + } + + @TestTemplate + public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { + sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); + + Table table = table("tl"); + TableOperations ops = ((BaseTable) table).operations(); + assertThat(ops.refresh().formatVersion()) + .as("should create table using format v2") + .isEqualTo(2); + assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) + .rootCause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot downgrade v2 table to v1"); + } + + @TestTemplate + public void testLoadTransformPartitionTable() throws TableNotExistException { + Schema schema = new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); + validationCatalog.createTable( + TableIdentifier.of(icebergNamespace, "tl"), + schema, + PartitionSpec.builderFor(schema).bucket("id", 100).build()); + + CatalogTable catalogTable = catalogTable("tl"); + assertThat(catalogTable.getSchema()) + .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); + assertThat(catalogTable.getPartitionKeys()).isEmpty(); + } + + @TestTemplate + public void testAlterTableProperties() throws TableNotExistException { + sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); + Map properties = Maps.newHashMap(); + properties.put("oldK", "oldV"); + + // new + sql("ALTER TABLE tl SET('newK'='newV')"); + properties.put("newK", "newV"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + + // update old + sql("ALTER TABLE tl SET('oldK'='oldV2')"); + properties.put("oldK", "oldV2"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + + // remove property + sql("ALTER TABLE tl RESET('oldK')"); + properties.remove("oldK"); + assertThat(table("tl").properties()).containsAllEntriesOf(properties); + } + + @TestTemplate + public void testAlterTableAddColumn() { + sql("CREATE TABLE tl(id BIGINT)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + sql("ALTER TABLE tl ADD (dt STRING)"); + Schema schemaAfter1 = table("tl").schema(); + assertThat(schemaAfter1.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Add multiple columns + sql("ALTER TABLE tl ADD (col1 STRING COMMENT 'comment for col1', col2 BIGINT)"); + Schema schemaAfter2 = table("tl").schema(); + assertThat(schemaAfter2.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional( + 3, "col1", Types.StringType.get(), "comment for col1"), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct()); + // Adding a required field should fail because Iceberg's SchemaUpdate does not allow + // incompatible changes. + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); + + // Adding an existing field should fail due to Flink's internal validation. + assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("Try to add a column `id` which already exists in the table."); + } + + @TestTemplate + public void testAlterTableDropColumn() { + sql("CREATE TABLE tl(id BIGINT, dt STRING, col1 STRING, col2 BIGINT)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct()); + sql("ALTER TABLE tl DROP (dt)"); + Schema schemaAfter1 = table("tl").schema(); + assertThat(schemaAfter1.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get()), + Types.NestedField.optional(4, "col2", Types.LongType.get())) + .asStruct()); + // Drop multiple columns + sql("ALTER TABLE tl DROP (col1, col2)"); + Schema schemaAfter2 = table("tl").schema(); + assertThat(schemaAfter2.asStruct()) + .isEqualTo( + new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); + // Dropping an non-existing field should fail due to Flink's internal validation. + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `foo` does not exist in the base table."); + + // Dropping an already-deleted field should fail due to Flink's internal validation. + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining("The column `dt` does not exist in the base table."); + } + + @TestTemplate + public void testAlterTableModifyColumnName() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + sql("ALTER TABLE tl RENAME dt TO data"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())) + .asStruct()); + } + + @TestTemplate + public void testAlterTableModifyColumnType() { + sql("CREATE TABLE tl(id INTEGER, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Promote type from Integer to Long + sql("ALTER TABLE tl MODIFY (id BIGINT)"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's + // validation. + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column type: dt: string -> int"); + } + + @TestTemplate + public void testAlterTableModifyColumnNullability() { + sql("CREATE TABLE tl(id INTEGER NOT NULL, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Changing nullability from optional to required should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); + + // Set nullability from required to optional + sql("ALTER TABLE tl MODIFY (id INTEGER)"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + } + + @TestTemplate + public void testAlterTableModifyColumnPosition() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING FIRST)"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(1, "id", Types.LongType.get())) + .asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING AFTER id)"); + Schema schemaAfterAfter = table("tl").schema(); + assertThat(schemaAfterAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + // Modifying the position of a non-existing column should fail due to Flink's internal + // validation. + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Try to modify a column `non_existing` which does not exist in the table."); + + // Moving a column after a non-existing column should fail due to Flink's internal validation. + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) + .isInstanceOf(ValidationException.class) + .hasMessageContaining( + "Referenced column `non_existing` by 'AFTER' does not exist in the table."); + } + + @TestTemplate + public void testAlterTableModifyColumnComment() { + sql("CREATE TABLE tl(id BIGINT, dt STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get())) + .asStruct()); + + sql("ALTER TABLE tl MODIFY (dt STRING COMMENT 'comment for dt field')"); + Schema schemaAfter = table("tl").schema(); + assertThat(schemaAfter.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional( + 2, "dt", Types.StringType.get(), "comment for dt field")) + .asStruct()); + } + + @TestTemplate + public void testAlterTableConstraint() { + sql("CREATE TABLE tl(id BIGINT NOT NULL, dt STRING NOT NULL, col1 STRING)"); + Schema schemaBefore = table("tl").schema(); + assertThat(schemaBefore.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct()); + assertThat(schemaBefore.identifierFieldNames()).isEmpty(); + sql("ALTER TABLE tl ADD (PRIMARY KEY (id) NOT ENFORCED)"); + Schema schemaAfterAdd = table("tl").schema(); + assertThat(schemaAfterAdd.identifierFieldNames()).containsExactly("id"); + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (dt) NOT ENFORCED)"); + Schema schemaAfterModify = table("tl").schema(); + assertThat(schemaAfterModify.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct()); + assertThat(schemaAfterModify.identifierFieldNames()).containsExactly("dt"); + // Composite primary key + sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, dt) NOT ENFORCED)"); + Schema schemaAfterComposite = table("tl").schema(); + assertThat(schemaAfterComposite.asStruct()) + .isEqualTo( + new Schema( + Types.NestedField.required(1, "id", Types.LongType.get()), + Types.NestedField.required(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "col1", Types.StringType.get())) + .asStruct()); + assertThat(schemaAfterComposite.identifierFieldNames()).containsExactlyInAnyOrder("id", "dt"); + // Setting an optional field as primary key should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Setting a composite key containing an optional field should fail + // because Iceberg's SchemaUpdate does not allow incompatible changes. + assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(IllegalArgumentException.class) + .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); + + // Dropping constraints is not supported yet + assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) + .isInstanceOf(TableException.class) + .hasRootCauseInstanceOf(UnsupportedOperationException.class) + .hasRootCauseMessage("Unsupported table change: DropConstraint."); + } + + @TestTemplate + public void testRelocateTable() { + assumeThat(isHadoopCatalog).as("HadoopCatalog does not support relocate table").isFalse(); + sql("CREATE TABLE tl(id BIGINT)"); + sql("ALTER TABLE tl SET('location'='file:///tmp/location')"); + assertThat(table("tl").location()).isEqualTo("file:///tmp/location"); + } + + @TestTemplate + public void testSetCurrentAndCherryPickSnapshotId() { + sql("CREATE TABLE tl(c1 INT, c2 STRING, c3 STRING) PARTITIONED BY (c1)"); + + Table table = table("tl"); + + DataFile fileA = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + DataFile fileB = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=1") // easy way to set partition data for now + .withRecordCount(1) + .build(); + DataFile replacementFile = + DataFiles.builder(table.spec()) + .withPath("/path/to/data-a-replacement.parquet") + .withFileSizeInBytes(10) + .withPartitionPath("c1=0") // easy way to set partition data for now + .withRecordCount(1) + .build(); + + table.newAppend().appendFile(fileA).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + + // stage an overwrite that replaces FILE_A + table.newReplacePartitions().addFile(replacementFile).stageOnly().commit(); + + Snapshot staged = Iterables.getLast(table.snapshots()); + assertThat(staged.operation()) + .as("Should find the staged overwrite snapshot") + .isEqualTo(DataOperations.OVERWRITE); + // add another append so that the original commit can't be fast-forwarded + table.newAppend().appendFile(fileB).commit(); + + // test cherry pick + sql("ALTER TABLE tl SET('cherry-pick-snapshot-id'='%s')", staged.snapshotId()); + validateTableFiles(table, fileB, replacementFile); + + // test set current snapshot + sql("ALTER TABLE tl SET('current-snapshot-id'='%s')", snapshotId); + validateTableFiles(table, fileA); + } + + private void validateTableFiles(Table tbl, DataFile... expectedFiles) { + tbl.refresh(); + Set expectedFilePaths = + Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); + Set actualFilePaths = + StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) + .map(FileScanTask::file) + .map(ContentFile::path) + .collect(Collectors.toSet()); + assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); + } + + private Table table(String name) { + return validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, name)); + } + + private CatalogTable catalogTable(String name) throws TableNotExistException { + return (CatalogTable) + getTableEnv() + .getCatalog(getTableEnv().getCurrentCatalog()) + .get() + .getTable(new ObjectPath(DATABASE, name)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java new file mode 100644 index 000000000000..e69e1ac4d713 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java @@ -0,0 +1,119 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.flink.table.catalog.CatalogPartitionSpec; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.table.catalog.exceptions.TableNotExistException; +import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkCatalogTablePartitions extends CatalogTestBase { + + private final String tableName = "test_table"; + + @Parameter(index = 2) + private FileFormat format; + + @Parameter(index = 3) + private Boolean cacheEnabled; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, cacheEnabled={3}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean cacheEnabled : new Boolean[] {true, false}) { + for (Object[] catalogParams : CatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, cacheEnabled}); + } + } + } + return parameters; + } + + @Override + @BeforeEach + public void before() { + super.before(); + config.put(CatalogProperties.CACHE_ENABLED, String.valueOf(cacheEnabled)); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @AfterEach + public void cleanNamespaces() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + @TestTemplate + public void testListPartitionsWithUnpartitionedTable() { + sql( + "CREATE TABLE %s (id INT, data VARCHAR) with ('write.format.default'='%s')", + tableName, format.name()); + sql("INSERT INTO %s SELECT 1,'a'", tableName); + + ObjectPath objectPath = new ObjectPath(DATABASE, tableName); + FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); + assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) + .isInstanceOf(TableNotPartitionedException.class) + .hasMessageStartingWith("Table db.test_table in catalog") + .hasMessageEndingWith("is not partitioned."); + } + + @TestTemplate + public void testListPartitionsWithPartitionedTable() + throws TableNotExistException, TableNotPartitionedException { + sql( + "CREATE TABLE %s (id INT, data VARCHAR) PARTITIONED BY (data) " + + "with ('write.format.default'='%s')", + tableName, format.name()); + sql("INSERT INTO %s SELECT 1,'a'", tableName); + sql("INSERT INTO %s SELECT 2,'b'", tableName); + + ObjectPath objectPath = new ObjectPath(DATABASE, tableName); + FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); + List list = flinkCatalog.listPartitions(objectPath); + assertThat(list).hasSize(2); + List expected = Lists.newArrayList(); + CatalogPartitionSpec partitionSpec1 = new CatalogPartitionSpec(ImmutableMap.of("data", "a")); + CatalogPartitionSpec partitionSpec2 = new CatalogPartitionSpec(ImmutableMap.of("data", "b")); + expected.add(partitionSpec1); + expected.add(partitionSpec2); + assertThat(list).as("Should produce the expected catalog partition specs.").isEqualTo(expected); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java new file mode 100644 index 000000000000..4b6ac25ab8e3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java @@ -0,0 +1,61 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.Map; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestFlinkConfParser { + + @Test + public void testDurationConf() { + Map writeOptions = ImmutableMap.of("write-prop", "111s"); + + ConfigOption configOption = + ConfigOptions.key("conf-prop").durationType().noDefaultValue(); + Configuration flinkConf = new Configuration(); + flinkConf.setString(configOption.key(), "222s"); + + Table table = mock(Table.class); + when(table.properties()).thenReturn(ImmutableMap.of("table-prop", "333s")); + + FlinkConfParser confParser = new FlinkConfParser(table, writeOptions, flinkConf); + Duration defaultVal = Duration.ofMillis(999); + + Duration result = + confParser.durationConf().option("write-prop").defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(111)); + + result = confParser.durationConf().flinkConfig(configOption).defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(222)); + + result = confParser.durationConf().tableProperty("table-prop").defaultValue(defaultVal).parse(); + assertThat(result).isEqualTo(Duration.ofSeconds(333)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java new file mode 100644 index 000000000000..838b0ea0e1a9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java @@ -0,0 +1,462 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.Instant; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.expressions.ApiExpressionUtils; +import org.apache.flink.table.expressions.CallExpression; +import org.apache.flink.table.expressions.Expression; +import org.apache.flink.table.expressions.FieldReferenceExpression; +import org.apache.flink.table.expressions.ResolvedExpression; +import org.apache.flink.table.expressions.UnresolvedCallExpression; +import org.apache.flink.table.expressions.UnresolvedReferenceExpression; +import org.apache.flink.table.expressions.ValueLiteralExpression; +import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; +import org.apache.flink.table.functions.BuiltInFunctionDefinitions; +import org.apache.iceberg.expressions.And; +import org.apache.iceberg.expressions.BoundLiteralPredicate; +import org.apache.iceberg.expressions.Not; +import org.apache.iceberg.expressions.Or; +import org.apache.iceberg.expressions.UnboundPredicate; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.DateTimeUtil; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.Test; + +public class TestFlinkFilters { + + private static final TableSchema TABLE_SCHEMA = + TableSchema.builder() + .field("field1", DataTypes.INT()) + .field("field2", DataTypes.BIGINT()) + .field("field3", DataTypes.FLOAT()) + .field("field4", DataTypes.DOUBLE()) + .field("field5", DataTypes.STRING()) + .field("field6", DataTypes.BOOLEAN()) + .field("field7", DataTypes.BINARY(2)) + .field("field8", DataTypes.DECIMAL(10, 2)) + .field("field9", DataTypes.DATE()) + .field("field10", DataTypes.TIME()) + .field("field11", DataTypes.TIMESTAMP()) + .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .build(); + + // A map list of fields and values used to verify the conversion of flink expression to iceberg + // expression + private static final List> FIELD_VALUE_LIST = + ImmutableList.of( + Pair.of("field1", 1), + Pair.of("field2", 2L), + Pair.of("field3", 3F), + Pair.of("field4", 4D), + Pair.of("field5", "iceberg"), + Pair.of("field6", true), + Pair.of("field7", new byte[] {'a', 'b'}), + Pair.of("field8", BigDecimal.valueOf(10.12)), + Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())), + Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())), + Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())), + Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))); + + @Test + public void testFlinkDataTypeEqual() { + matchLiteral("field1", 1, 1); + matchLiteral("field2", 10L, 10L); + matchLiteral("field3", 1.2F, 1.2F); + matchLiteral("field4", 3.4D, 3.4D); + matchLiteral("field5", "abcd", "abcd"); + matchLiteral("field6", true, true); + matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'})); + matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12)); + + LocalDate date = LocalDate.parse("2020-12-23"); + matchLiteral("field9", date, DateTimeUtil.daysFromDate(date)); + + LocalTime time = LocalTime.parse("12:13:14"); + matchLiteral("field10", time, DateTimeUtil.microsFromTime(time)); + + LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14"); + matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime)); + + Instant instant = Instant.parse("2020-12-23T12:13:14.00Z"); + matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant)); + } + + @Test + public void testEquals() { + for (Pair pair : FIELD_VALUE_LIST) { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second()); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + } + + @Test + public void testEqualsNaN() { + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNaN("field3"); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testNotEquals() { + for (Pair pair : FIELD_VALUE_LIST) { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second()); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + } + + @Test + public void testNotEqualsNaN() { + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNaN("field3"); + + Optional actual = + FlinkFilters.convert( + resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert( + resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testGreaterThan() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testGreaterThanEquals() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testLessThan() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.lessThan("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testLessThanEquals() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1); + + Optional actual = + FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + Optional actual1 = + FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); + assertThat(actual1).isPresent(); + assertPredicatesMatch(expected, actual1.get()); + } + + @Test + public void testIsNull() { + Expression expr = resolve(Expressions.$("field1").isNull()); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); + assertPredicatesMatch(expected, actual.get()); + } + + @Test + public void testIsNotNull() { + Expression expr = resolve(Expressions.$("field1").isNotNull()); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.notNull("field1"); + assertPredicatesMatch(expected, actual.get()); + } + + @Test + public void testAnd() { + Expression expr = + resolve( + Expressions.$("field1") + .isEqual(Expressions.lit(1)) + .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + And and = (And) actual.get(); + And expected = + (And) + org.apache.iceberg.expressions.Expressions.and( + org.apache.iceberg.expressions.Expressions.equal("field1", 1), + org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); + + assertPredicatesMatch(expected.left(), and.left()); + assertPredicatesMatch(expected.right(), and.right()); + } + + @Test + public void testOr() { + Expression expr = + resolve( + Expressions.$("field1") + .isEqual(Expressions.lit(1)) + .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + Or or = (Or) actual.get(); + Or expected = + (Or) + org.apache.iceberg.expressions.Expressions.or( + org.apache.iceberg.expressions.Expressions.equal("field1", 1), + org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); + + assertPredicatesMatch(expected.left(), or.left()); + assertPredicatesMatch(expected.right(), or.right()); + } + + @Test + public void testNot() { + Expression expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.NOT, + Expressions.$("field1").isEqual(Expressions.lit(1)))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + Not not = (Not) actual.get(); + Not expected = + (Not) + org.apache.iceberg.expressions.Expressions.not( + org.apache.iceberg.expressions.Expressions.equal("field1", 1)); + + assertThat(not.op()).as("Predicate operation should match").isEqualTo(expected.op()); + assertPredicatesMatch(expected.child(), not.child()); + } + + @Test + public void testLike() { + UnboundPredicate expected = + org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc"); + Expression expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + assertPredicatesMatch(expected, actual.get()); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, + Expressions.$("field5"), + Expressions.lit("%abc%"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, + Expressions.$("field5"), + Expressions.lit("abc%d"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + + expr = + resolve( + ApiExpressionUtils.unresolvedCall( + BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); + actual = FlinkFilters.convert(expr); + assertThat(actual).isNotPresent(); + } + + @SuppressWarnings("unchecked") + private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { + Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); + Optional actual = FlinkFilters.convert(expr); + assertThat(actual).isPresent(); + org.apache.iceberg.expressions.Expression expression = actual.get(); + assertThat(expression) + .as("The expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + UnboundPredicate unboundPredicate = (UnboundPredicate) expression; + + org.apache.iceberg.expressions.Expression expression1 = + unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); + assertThat(expression1) + .as("The expression should be a BoundLiteralPredicate") + .isInstanceOf(BoundLiteralPredicate.class); + + BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; + assertThat(predicate.test(icebergLiteral)).isTrue(); + } + + private static Expression resolve(Expression originalExpression) { + return originalExpression.accept( + new ApiExpressionDefaultVisitor() { + @Override + public Expression visit(UnresolvedReferenceExpression unresolvedReference) { + String name = unresolvedReference.getName(); + Optional field = TABLE_SCHEMA.getTableColumn(name); + if (field.isPresent()) { + int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get()); + return new FieldReferenceExpression(name, field.get().getType(), 0, index); + } else { + return null; + } + } + + @Override + public Expression visit(UnresolvedCallExpression unresolvedCall) { + List children = + unresolvedCall.getChildren().stream() + .map(e -> (ResolvedExpression) e.accept(this)) + .collect(Collectors.toList()); + return new CallExpression( + unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); + } + + @Override + public Expression visit(ValueLiteralExpression valueLiteral) { + return valueLiteral; + } + + @Override + protected Expression defaultMethod(Expression expression) { + throw new UnsupportedOperationException( + String.format("unsupported expression: %s", expression)); + } + }); + } + + private void assertPredicatesMatch( + org.apache.iceberg.expressions.Expression expected, + org.apache.iceberg.expressions.Expression actual) { + assertThat(expected) + .as("The expected expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + assertThat(actual) + .as("The actual expression should be a UnboundPredicate") + .isInstanceOf(UnboundPredicate.class); + UnboundPredicate predicateExpected = (UnboundPredicate) expected; + UnboundPredicate predicateActual = (UnboundPredicate) actual; + assertThat(predicateActual.op()).isEqualTo(predicateExpected.op()); + assertThat(predicateActual.literal()).isEqualTo(predicateExpected.literal()); + assertThat(predicateActual.ref().name()).isEqualTo(predicateExpected.ref().name()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java new file mode 100644 index 000000000000..91343ab1ee72 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.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.flink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +public class TestFlinkHiveCatalog extends TestBase { + + @Test + public void testCreateCatalogWithWarehouseLocation() throws IOException { + Map props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); + props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); + + checkSQLQuery(props, warehouseDir); + } + + @Test + public void testCreateCatalogWithHiveConfDir() throws IOException { + // Dump the hive conf into a local file. + File hiveConfDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); + File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); + File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); + try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { + Configuration newConf = new Configuration(hiveConf); + // Set another new directory which is different with the hive metastore's warehouse path. + newConf.set( + HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); + newConf.writeXml(fos); + } + assertThat(hiveSiteXML.toPath()).exists(); + + // Construct the catalog attributions. + Map props = Maps.newHashMap(); + props.put("type", "iceberg"); + props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); + props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + // Set the 'hive-conf-dir' instead of 'warehouse' + props.put(FlinkCatalogFactory.HIVE_CONF_DIR, hiveConfDir.getAbsolutePath()); + + checkSQLQuery(props, warehouseDir); + } + + private void checkSQLQuery(Map catalogProperties, File warehouseDir) + throws IOException { + sql("CREATE CATALOG test_catalog WITH %s", CatalogTestBase.toWithClause(catalogProperties)); + sql("USE CATALOG test_catalog"); + sql("CREATE DATABASE test_db"); + sql("USE test_db"); + sql("CREATE TABLE test_table(c1 INT, c2 STRING)"); + sql("INSERT INTO test_table SELECT 1, 'a'"); + + Path databasePath = warehouseDir.toPath().resolve("test_db.db"); + assertThat(databasePath).exists(); + + Path tablePath = databasePath.resolve("test_table"); + assertThat(tablePath).exists(); + + Path dataPath = tablePath.resolve("data"); + assertThat(dataPath).exists(); + assertThat(Files.list(dataPath).count()) + .as("Should have a .crc file and a .parquet file") + .isEqualTo(2); + + sql("DROP TABLE test_table"); + dropDatabase("test_db", false); + dropCatalog("test_catalog", false); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java new file mode 100644 index 000000000000..eab60d886ada --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java @@ -0,0 +1,416 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.api.ValidationException; +import org.apache.flink.table.types.logical.BinaryType; +import org.apache.flink.table.types.logical.CharType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.TimeType; +import org.apache.flink.table.types.logical.TimestampType; +import org.apache.flink.table.types.logical.VarBinaryType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +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.junit.jupiter.api.Test; + +public class TestFlinkSchemaUtil { + + @Test + public void testConvertFlinkSchemaToIcebergSchema() { + TableSchema flinkSchema = + TableSchema.builder() + .field("id", DataTypes.INT().notNull()) + .field("name", DataTypes.STRING()) /* optional by default */ + .field("salary", DataTypes.DOUBLE().notNull()) + .field( + "locations", + DataTypes.MAP( + DataTypes.STRING(), + DataTypes.ROW( + DataTypes.FIELD("posX", DataTypes.DOUBLE().notNull(), "X field"), + DataTypes.FIELD("posY", DataTypes.DOUBLE().notNull(), "Y field")))) + .field("strArray", DataTypes.ARRAY(DataTypes.STRING()).nullable()) + .field("intArray", DataTypes.ARRAY(DataTypes.INT()).nullable()) + .field("char", DataTypes.CHAR(10).notNull()) + .field("varchar", DataTypes.VARCHAR(10).notNull()) + .field("boolean", DataTypes.BOOLEAN().nullable()) + .field("tinyint", DataTypes.TINYINT()) + .field("smallint", DataTypes.SMALLINT()) + .field("bigint", DataTypes.BIGINT()) + .field("varbinary", DataTypes.VARBINARY(10)) + .field("binary", DataTypes.BINARY(10)) + .field("time", DataTypes.TIME()) + .field("timestampWithoutZone", DataTypes.TIMESTAMP()) + .field("timestampWithZone", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) + .field("date", DataTypes.DATE()) + .field("decimal", DataTypes.DECIMAL(2, 2)) + .field("decimal2", DataTypes.DECIMAL(38, 2)) + .field("decimal3", DataTypes.DECIMAL(10, 1)) + .field("multiset", DataTypes.MULTISET(DataTypes.STRING().notNull())) + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required(0, "id", Types.IntegerType.get(), null), + Types.NestedField.optional(1, "name", Types.StringType.get(), null), + Types.NestedField.required(2, "salary", Types.DoubleType.get(), null), + Types.NestedField.optional( + 3, + "locations", + Types.MapType.ofOptional( + 24, + 25, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(22, "posX", Types.DoubleType.get(), "X field"), + Types.NestedField.required( + 23, "posY", Types.DoubleType.get(), "Y field")))), + Types.NestedField.optional( + 4, "strArray", Types.ListType.ofOptional(26, Types.StringType.get())), + Types.NestedField.optional( + 5, "intArray", Types.ListType.ofOptional(27, Types.IntegerType.get())), + Types.NestedField.required(6, "char", Types.StringType.get()), + Types.NestedField.required(7, "varchar", Types.StringType.get()), + Types.NestedField.optional(8, "boolean", Types.BooleanType.get()), + Types.NestedField.optional(9, "tinyint", Types.IntegerType.get()), + Types.NestedField.optional(10, "smallint", Types.IntegerType.get()), + Types.NestedField.optional(11, "bigint", Types.LongType.get()), + Types.NestedField.optional(12, "varbinary", Types.BinaryType.get()), + Types.NestedField.optional(13, "binary", Types.FixedType.ofLength(10)), + Types.NestedField.optional(14, "time", Types.TimeType.get()), + Types.NestedField.optional( + 15, "timestampWithoutZone", Types.TimestampType.withoutZone()), + Types.NestedField.optional(16, "timestampWithZone", Types.TimestampType.withZone()), + Types.NestedField.optional(17, "date", Types.DateType.get()), + Types.NestedField.optional(18, "decimal", Types.DecimalType.of(2, 2)), + Types.NestedField.optional(19, "decimal2", Types.DecimalType.of(38, 2)), + Types.NestedField.optional(20, "decimal3", Types.DecimalType.of(10, 1)), + Types.NestedField.optional( + 21, + "multiset", + Types.MapType.ofRequired(28, 29, Types.StringType.get(), Types.IntegerType.get()))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testMapField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "map_int_long", + DataTypes.MAP(DataTypes.INT(), DataTypes.BIGINT()).notNull()) /* Required */ + .field( + "map_int_array_string", + DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING())) + .field( + "map_decimal_string", DataTypes.MAP(DataTypes.DECIMAL(10, 2), DataTypes.STRING())) + .field( + "map_fields_fields", + DataTypes.MAP( + DataTypes.ROW( + DataTypes.FIELD("field_int", DataTypes.INT(), "doc - int"), + DataTypes.FIELD("field_string", DataTypes.STRING(), "doc - string")) + .notNull(), /* Required */ + DataTypes.ROW( + DataTypes.FIELD( + "field_array", + DataTypes.ARRAY(DataTypes.STRING()), + "doc - array")) + .notNull() /* Required */) + .notNull() /* Required */) + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "map_int_long", + Types.MapType.ofOptional(4, 5, Types.IntegerType.get(), Types.LongType.get()), + null), + Types.NestedField.optional( + 1, + "map_int_array_string", + Types.MapType.ofOptional( + 7, + 8, + Types.ListType.ofOptional(6, Types.IntegerType.get()), + Types.StringType.get()), + null), + Types.NestedField.optional( + 2, + "map_decimal_string", + Types.MapType.ofOptional( + 9, 10, Types.DecimalType.of(10, 2), Types.StringType.get())), + Types.NestedField.required( + 3, + "map_fields_fields", + Types.MapType.ofRequired( + 15, + 16, + Types.StructType.of( + Types.NestedField.optional( + 11, "field_int", Types.IntegerType.get(), "doc - int"), + Types.NestedField.optional( + 12, "field_string", Types.StringType.get(), "doc - string")), + Types.StructType.of( + Types.NestedField.optional( + 14, + "field_array", + Types.ListType.ofOptional(13, Types.StringType.get()), + "doc - array"))))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testStructField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "struct_int_string_decimal", + DataTypes.ROW( + DataTypes.FIELD("field_int", DataTypes.INT()), + DataTypes.FIELD("field_string", DataTypes.STRING()), + DataTypes.FIELD("field_decimal", DataTypes.DECIMAL(19, 2)), + DataTypes.FIELD( + "field_struct", + DataTypes.ROW( + DataTypes.FIELD("inner_struct_int", DataTypes.INT()), + DataTypes.FIELD( + "inner_struct_float_array", + DataTypes.ARRAY(DataTypes.FLOAT()))) + .notNull()) /* Row is required */) + .notNull()) /* Required */ + .field( + "struct_map_int_int", + DataTypes.ROW( + DataTypes.FIELD( + "field_map", DataTypes.MAP(DataTypes.INT(), DataTypes.INT()))) + .nullable()) /* Optional */ + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "struct_int_string_decimal", + Types.StructType.of( + Types.NestedField.optional(5, "field_int", Types.IntegerType.get()), + Types.NestedField.optional(6, "field_string", Types.StringType.get()), + Types.NestedField.optional(7, "field_decimal", Types.DecimalType.of(19, 2)), + Types.NestedField.required( + 8, + "field_struct", + Types.StructType.of( + Types.NestedField.optional( + 3, "inner_struct_int", Types.IntegerType.get()), + Types.NestedField.optional( + 4, + "inner_struct_float_array", + Types.ListType.ofOptional(2, Types.FloatType.get())))))), + Types.NestedField.optional( + 1, + "struct_map_int_int", + Types.StructType.of( + Types.NestedField.optional( + 11, + "field_map", + Types.MapType.ofOptional( + 9, 10, Types.IntegerType.get(), Types.IntegerType.get()))))); + + checkSchema(flinkSchema, icebergSchema); + } + + @Test + public void testListField() { + TableSchema flinkSchema = + TableSchema.builder() + .field( + "list_struct_fields", + DataTypes.ARRAY(DataTypes.ROW(DataTypes.FIELD("field_int", DataTypes.INT()))) + .notNull()) /* Required */ + .field( + "list_optional_struct_fields", + DataTypes.ARRAY( + DataTypes.ROW( + DataTypes.FIELD( + "field_timestamp_with_local_time_zone", + DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()))) + .nullable()) /* Optional */ + .field( + "list_map_fields", + DataTypes.ARRAY( + DataTypes.MAP( + DataTypes.ARRAY( + DataTypes.INT().notNull()), /* Key of map must be required */ + DataTypes.ROW( + DataTypes.FIELD("field_0", DataTypes.INT(), "doc - int"))) + .notNull()) + .notNull()) /* Required */ + .build(); + + Schema icebergSchema = + new Schema( + Types.NestedField.required( + 0, + "list_struct_fields", + Types.ListType.ofOptional( + 4, + Types.StructType.of( + Types.NestedField.optional(3, "field_int", Types.IntegerType.get())))), + Types.NestedField.optional( + 1, + "list_optional_struct_fields", + Types.ListType.ofOptional( + 6, + Types.StructType.of( + Types.NestedField.optional( + 5, + "field_timestamp_with_local_time_zone", + Types.TimestampType.withZone())))), + Types.NestedField.required( + 2, + "list_map_fields", + Types.ListType.ofRequired( + 11, + Types.MapType.ofOptional( + 9, + 10, + Types.ListType.ofRequired(7, Types.IntegerType.get()), + Types.StructType.of( + Types.NestedField.optional( + 8, "field_0", Types.IntegerType.get(), "doc - int")))))); + + checkSchema(flinkSchema, icebergSchema); + } + + private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { + assertThat(FlinkSchemaUtil.convert(flinkSchema).asStruct()).isEqualTo(icebergSchema.asStruct()); + // The conversion is not a 1:1 mapping, so we just check iceberg types. + assertThat( + FlinkSchemaUtil.convert( + FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) + .asStruct()) + .isEqualTo(icebergSchema.asStruct()); + } + + @Test + public void testInconsistentTypes() { + checkInconsistentType( + Types.UUIDType.get(), new BinaryType(16), new BinaryType(16), Types.FixedType.ofLength(16)); + checkInconsistentType( + Types.StringType.get(), + new VarCharType(VarCharType.MAX_LENGTH), + new CharType(100), + Types.StringType.get()); + checkInconsistentType( + Types.BinaryType.get(), + new VarBinaryType(VarBinaryType.MAX_LENGTH), + new VarBinaryType(100), + Types.BinaryType.get()); + checkInconsistentType( + Types.TimeType.get(), new TimeType(), new TimeType(3), Types.TimeType.get()); + checkInconsistentType( + Types.TimestampType.withoutZone(), + new TimestampType(6), + new TimestampType(3), + Types.TimestampType.withoutZone()); + checkInconsistentType( + Types.TimestampType.withZone(), + new LocalZonedTimestampType(6), + new LocalZonedTimestampType(3), + Types.TimestampType.withZone()); + } + + private void checkInconsistentType( + Type icebergType, + LogicalType flinkExpectedType, + LogicalType flinkType, + Type icebergExpectedType) { + assertThat(FlinkSchemaUtil.convert(icebergType)).isEqualTo(flinkExpectedType); + assertThat(FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()) + .isEqualTo(Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType))); + } + + @Test + public void testConvertFlinkSchemaBaseOnIcebergSchema() { + Schema baseSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required(101, "int", Types.IntegerType.get()), + Types.NestedField.optional(102, "string", Types.StringType.get())), + Sets.newHashSet(101)); + + TableSchema flinkSchema = + TableSchema.builder() + .field("int", DataTypes.INT().notNull()) + .field("string", DataTypes.STRING().nullable()) + .primaryKey("int") + .build(); + Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); + assertThat(convertedSchema.asStruct()).isEqualTo(baseSchema.asStruct()); + assertThat(convertedSchema.identifierFieldIds()).containsExactly(101); + } + + @Test + public void testConvertFlinkSchemaWithPrimaryKeys() { + Schema icebergSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required(1, "int", Types.IntegerType.get()), + Types.NestedField.required(2, "string", Types.StringType.get())), + Sets.newHashSet(1, 2)); + + TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); + assertThat(tableSchema.getPrimaryKey()) + .isPresent() + .get() + .satisfies(k -> assertThat(k.getColumns()).containsExactly("int", "string")); + } + + @Test + public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { + Schema icebergSchema = + new Schema( + Lists.newArrayList( + Types.NestedField.required( + 1, + "struct", + Types.StructType.of( + Types.NestedField.required(2, "inner", Types.IntegerType.get())))), + Sets.newHashSet(2)); + + assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Could not create a PRIMARY KEY") + .hasMessageContaining("Column 'struct.inner' does not exist."); + } +} 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 new file mode 100644 index 000000000000..2978a92945a2 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -0,0 +1,244 @@ +/* + * 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; + +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.util.List; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.Expressions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +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; + + @Parameter(index = 2) + private FileFormat format; + + @Parameter(index = 3) + private boolean isStreamingJob; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + public static List parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + for (Object[] catalogParams : CatalogTestBase.parameters()) { + String catalogName = (String) catalogParams[0]; + Namespace baseNamespace = (Namespace) catalogParams[1]; + parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + } + } + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + } + return tEnv; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE_NAME, format.name()); + icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + dropDatabase(flinkDatabase, true); + BoundedTableFactory.clearDataSets(); + super.clean(); + } + + @TestTemplate + public void testInsertFromSourceTable() throws Exception { + // Register the rows into a temporary table. + getTableEnv() + .createTemporaryView( + "sourceTable", + getTableEnv() + .fromValues( + SimpleDataUtil.FLINK_SCHEMA.toRowDataType(), + Expressions.row(1, "hello"), + Expressions.row(2, "world"), + Expressions.row(3, (String) null), + Expressions.row(null, "bar"))); + + // Redirect the records from source table to destination table. + sql("INSERT INTO %s SELECT id,data from sourceTable", TABLE_NAME); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), + SimpleDataUtil.createRecord(2, "world"), + SimpleDataUtil.createRecord(3, null), + SimpleDataUtil.createRecord(null, "bar"))); + } + + @TestTemplate + public void testOverwriteTable() throws Exception { + assumeThat(isStreamingJob) + .as("Flink unbounded streaming does not support overwrite operation") + .isFalse(); + + sql("INSERT INTO %s SELECT 1, 'a'", TABLE_NAME); + SimpleDataUtil.assertTableRecords( + icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(1, "a"))); + + sql("INSERT OVERWRITE %s SELECT 2, 'b'", TABLE_NAME); + SimpleDataUtil.assertTableRecords( + icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); + } + + @TestTemplate + public void testReplacePartitions() throws Exception { + assumeThat(isStreamingJob) + .as("Flink unbounded streaming does not support overwrite operation") + .isFalse(); + String tableName = "test_partition"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", + tableName, format.name()); + + try { + Table partitionedTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + + sql("INSERT INTO %s SELECT 1, 'a'", tableName); + sql("INSERT INTO %s SELECT 2, 'b'", tableName); + sql("INSERT INTO %s SELECT 3, 'c'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"))); + + sql("INSERT OVERWRITE %s SELECT 4, 'b'", tableName); + sql("INSERT OVERWRITE %s SELECT 5, 'a'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(5, "a"), + SimpleDataUtil.createRecord(4, "b"), + SimpleDataUtil.createRecord(3, "c"))); + + sql("INSERT OVERWRITE %s PARTITION (data='a') SELECT 6", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(6, "a"), + SimpleDataUtil.createRecord(4, "b"), + SimpleDataUtil.createRecord(3, "c"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testInsertIntoPartition() throws Exception { + String tableName = "test_insert_into_partition"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", + tableName, format.name()); + + try { + Table partitionedTable = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); + + // Full partition. + sql("INSERT INTO %s PARTITION (data='a') SELECT 1", tableName); + sql("INSERT INTO %s PARTITION (data='a') SELECT 2", tableName); + sql("INSERT INTO %s PARTITION (data='b') SELECT 3", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "a"), + SimpleDataUtil.createRecord(3, "b"))); + + // Partial partition. + sql("INSERT INTO %s SELECT 4, 'c'", tableName); + sql("INSERT INTO %s SELECT 5, 'd'", tableName); + + SimpleDataUtil.assertTableRecords( + partitionedTable, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "a"), + SimpleDataUtil.createRecord(3, "b"), + SimpleDataUtil.createRecord(4, "c"), + SimpleDataUtil.createRecord(5, "d"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } +} 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 new file mode 100644 index 000000000000..482cfd110bde --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,244 @@ +/* + * 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; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +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.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.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List parameters() { + return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + } + + protected synchronized TableEnvironment getTableEnv() { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + return tEnv; + } + + @BeforeEach + public void before() { + String warehouseLocation = "file:" + warehouseRoot.getPath(); + this.catalog = new HadoopCatalog(new Configuration(), warehouseLocation); + Map config = Maps.newHashMap(); + config.put("type", "iceberg"); + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HADOOP); + config.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + sql("CREATE CATALOG %s WITH %s", CATALOG, toWithClause(config)); + + sql("CREATE DATABASE %s", FLINK_DATABASE); + sql("USE CATALOG %s", CATALOG); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE, FileFormat.PARQUET.name()); + } + + @AfterEach + public void clean() throws Exception { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, TABLE); + dropDatabase(FLINK_DATABASE, true); + BoundedTableFactory.clearDataSets(); + + dropCatalog(CATALOG, true); + catalog.close(); + } + + @TestTemplate + public void testWriteParallelism() { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + 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 /*+ 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)); + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java new file mode 100644 index 000000000000..c5becb6caca1 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java @@ -0,0 +1,334 @@ +/* + * 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; + +import java.time.LocalDate; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +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.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkUpsert extends CatalogTestBase { + + @Parameter(index = 2) + private FileFormat format; + + @Parameter(index = 3) + private boolean isStreamingJob; + + private final Map tableUpsertProps = Maps.newHashMap(); + private TableEnvironment tEnv; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") + public static List parameters() { + List parameters = Lists.newArrayList(); + for (FileFormat format : + new FileFormat[] {FileFormat.PARQUET, FileFormat.AVRO, FileFormat.ORC}) { + for (Boolean isStreaming : new Boolean[] {true, false}) { + // Only test with one catalog as this is a file operation concern. + // FlinkCatalogTestBase requires the catalog name start with testhadoop if using hadoop + // catalog. + String catalogName = "testhadoop"; + Namespace baseNamespace = Namespace.of("default"); + parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); + } + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + } + return tEnv; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + tableUpsertProps.put(TableProperties.FORMAT_VERSION, "2"); + tableUpsertProps.put(TableProperties.UPSERT_ENABLED, "true"); + tableUpsertProps.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + } + + @Override + @AfterEach + public void clean() { + dropDatabase(flinkDatabase, true); + super.clean(); + } + + @TestTemplate + public void testUpsertAndQuery() { + String tableName = "test_upsert_query"; + LocalDate dt20220301 = LocalDate.of(2022, 3, 1); + LocalDate dt20220302 = LocalDate.of(2022, 3, 2); + + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + try { + sql( + "INSERT INTO %s VALUES " + + "(1, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-01')," + + "(2, 'Jane', DATE '2022-03-01')", + tableName); + + sql( + "INSERT INTO %s VALUES " + + "(2, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-02')," + + "(2, 'Jane', DATE '2022-03-02')", + tableName); + + List rowsOn20220301 = + Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); + + List rowsOn20220302 = + Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testUpsertOptions() { + String tableName = "test_upsert_options"; + LocalDate dt20220301 = LocalDate.of(2022, 3, 1); + LocalDate dt20220302 = LocalDate.of(2022, 3, 2); + + Map optionsUpsertProps = Maps.newHashMap(tableUpsertProps); + optionsUpsertProps.remove(TableProperties.UPSERT_ENABLED); + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(optionsUpsertProps)); + + try { + sql( + "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " + + "(1, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-01')," + + "(2, 'Jane', DATE '2022-03-01')", + tableName); + + sql( + "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " + + "(2, 'Bill', DATE '2022-03-01')," + + "(1, 'Jane', DATE '2022-03-02')," + + "(2, 'Jane', DATE '2022-03-02')", + tableName); + + List rowsOn20220301 = + Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); + + List rowsOn20220302 = + Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); + TestHelpers.assertRows( + sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testPrimaryKeyEqualToPartitionKey() { + // This is an SQL based reproduction of TestFlinkIcebergSinkV2#testUpsertOnDataKey + String tableName = "upsert_on_id_key"; + try { + sql( + "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, PRIMARY KEY(id) NOT ENFORCED) " + + "PARTITIONED BY (id) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(1, 'Jane')," + "(2, 'Bill')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, "Jane"), Row.of(2, "Bill"))); + + sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(2, 'Jane')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, "Bill"), Row.of(2, "Jane"))); + + sql("INSERT INTO %s VALUES " + "(3, 'Bill')," + "(4, 'Jane')", tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of(1, "Bill"), Row.of(2, "Jane"), Row.of(3, "Bill"), Row.of(4, "Jane"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testPrimaryKeyFieldsAtBeginningOfSchema() { + String tableName = "upsert_on_pk_at_schema_start"; + LocalDate dt = LocalDate.of(2022, 3, 1); + try { + sql( + "CREATE TABLE %s(id INT, dt DATE NOT NULL, name STRING NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql( + "INSERT INTO %s VALUES " + + "(1, DATE '2022-03-01', 'Andy')," + + "(1, DATE '2022-03-01', 'Bill')," + + "(2, DATE '2022-03-01', 'Jane')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, dt, "Bill"), Row.of(2, dt, "Jane"))); + + sql( + "INSERT INTO %s VALUES " + + "(1, DATE '2022-03-01', 'Jane')," + + "(2, DATE '2022-03-01', 'Bill')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of(1, dt, "Jane"), Row.of(2, dt, "Bill"))); + + sql( + "INSERT INTO %s VALUES " + + "(3, DATE '2022-03-01', 'Duke')," + + "(4, DATE '2022-03-01', 'Leon')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of(1, dt, "Jane"), + Row.of(2, dt, "Bill"), + Row.of(3, dt, "Duke"), + Row.of(4, dt, "Leon"))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } + + @TestTemplate + public void testPrimaryKeyFieldsAtEndOfTableSchema() { + // This is the same test case as testPrimaryKeyFieldsAtBeginningOfSchema, but the primary key + // fields + // are located at the end of the flink schema. + String tableName = "upsert_on_pk_at_schema_end"; + LocalDate dt = LocalDate.of(2022, 3, 1); + try { + sql( + "CREATE TABLE %s(name STRING NOT NULL, id INT, dt DATE NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " + + "PARTITIONED BY (dt) WITH %s", + tableName, toWithClause(tableUpsertProps)); + + sql( + "INSERT INTO %s VALUES " + + "('Andy', 1, DATE '2022-03-01')," + + "('Bill', 1, DATE '2022-03-01')," + + "('Jane', 2, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of("Bill", 1, dt), Row.of("Jane", 2, dt))); + + sql( + "INSERT INTO %s VALUES " + + "('Jane', 1, DATE '2022-03-01')," + + "('Bill', 2, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList(Row.of("Jane", 1, dt), Row.of("Bill", 2, dt))); + + sql( + "INSERT INTO %s VALUES " + + "('Duke', 3, DATE '2022-03-01')," + + "('Leon', 4, DATE '2022-03-01')", + tableName); + + TestHelpers.assertRows( + sql("SELECT * FROM %s", tableName), + Lists.newArrayList( + Row.of("Jane", 1, dt), + Row.of("Bill", 2, dt), + Row.of("Duke", 3, dt), + Row.of("Leon", 4, dt))); + } finally { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); + } + } +} 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 new file mode 100644 index 000000000000..8cebf950c5f0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java @@ -0,0 +1,632 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.time.OffsetDateTime; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import org.apache.avro.generic.GenericData; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.ArrayData; +import org.apache.flink.table.data.DecimalData; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.MapData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.data.conversion.DataStructureConverter; +import org.apache.flink.table.data.conversion.DataStructureConverters; +import org.apache.flink.table.runtime.typeutils.InternalSerializers; +import org.apache.flink.table.types.logical.ArrayType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.MapType; +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.ContentFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RowDataUtil; +import org.apache.iceberg.flink.source.FlinkInputFormat; +import org.apache.iceberg.flink.source.FlinkInputSplit; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; + +public class TestHelpers { + private TestHelpers() {} + + public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { + KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + kryo.serialize(table, outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + return kryo.deserialize(inputView); + } + + public static RowData copyRowData(RowData from, RowType rowType) { + TypeSerializer[] fieldSerializers = + rowType.getChildren().stream() + .map((LogicalType type) -> InternalSerializers.create(type)) + .toArray(TypeSerializer[]::new); + RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; + for (int i = 0; i < rowType.getFieldCount(); ++i) { + fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); + } + + return RowDataUtil.clone(from, null, rowType, fieldSerializers, fieldGetters); + } + + public static void readRowData(FlinkInputFormat input, Consumer visitor) + throws IOException { + for (FlinkInputSplit s : input.createInputSplits(0)) { + input.open(s); + try { + while (!input.reachedEnd()) { + RowData row = input.nextRecord(null); + visitor.accept(row); + } + } finally { + input.close(); + } + } + } + + public static List readRowData(FlinkInputFormat inputFormat, RowType rowType) + throws IOException { + List results = Lists.newArrayList(); + readRowData(inputFormat, row -> results.add(copyRowData(row, rowType))); + return results; + } + + public static List readRows(FlinkInputFormat inputFormat, RowType rowType) + throws IOException { + return convertRowDataToRow(readRowData(inputFormat, rowType), rowType); + } + + public static List convertRowDataToRow(List rowDataList, RowType rowType) { + DataStructureConverter converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + return rowDataList.stream() + .map(converter::toExternal) + .map(Row.class::cast) + .collect(Collectors.toList()); + } + + private static List convertRecordToRow(List expectedRecords, Schema schema) { + List expected = Lists.newArrayList(); + @SuppressWarnings("unchecked") + DataStructureConverter converter = + (DataStructureConverter) + DataStructureConverters.getConverter( + TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(schema))); + expectedRecords.forEach( + r -> expected.add(converter.toExternal(RowDataConverter.convert(schema, r)))); + return expected; + } + + public static void assertRecordsWithOrder( + List results, List expectedRecords, Schema schema) { + List expected = convertRecordToRow(expectedRecords, schema); + assertRowsWithOrder(results, expected); + } + + public static void assertRecords(List results, List expectedRecords, Schema schema) { + List expected = convertRecordToRow(expectedRecords, schema); + assertRows(results, expected); + } + + public static void assertRows(List results, List expected, RowType rowType) { + assertRows(convertRowDataToRow(results, rowType), convertRowDataToRow(expected, rowType)); + } + + public static void assertRows(List results, List expected) { + assertThat(results).containsExactlyInAnyOrderElementsOf(expected); + } + + public static void assertRowsWithOrder(List results, List expected) { + assertThat(results).containsExactlyElementsOf(expected); + } + + public static void assertRowData(Schema schema, StructLike expected, RowData actual) { + assertRowData(schema.asStruct(), FlinkSchemaUtil.convert(schema), expected, actual); + } + + public static void assertRowData( + Types.StructType structType, + LogicalType rowType, + StructLike expectedRecord, + RowData actualRowData) { + if (expectedRecord == null && actualRowData == null) { + return; + } + + assertThat(expectedRecord).isNotNull(); + assertThat(actualRowData).isNotNull(); + + List types = Lists.newArrayList(); + for (Types.NestedField field : structType.fields()) { + types.add(field.type()); + } + + for (int i = 0; i < types.size(); i += 1) { + LogicalType logicalType = ((RowType) rowType).getTypeAt(i); + Object expected = expectedRecord.get(i, Object.class); + // The RowData.createFieldGetter won't return null for the required field. But in the + // projection case, if we are + // projecting a nested required field from an optional struct, then we should give a null for + // the projected field + // if the outer struct value is null. So we need to check the nullable for actualRowData here. + // For more details + // please see issue #2738. + Object actual = + actualRowData.isNullAt(i) + ? null + : RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData); + assertEquals(types.get(i), logicalType, expected, actual); + } + } + + private static void assertEquals( + Type type, LogicalType logicalType, Object expected, Object actual) { + + if (expected == null && actual == null) { + return; + } + + assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + + switch (type.typeId()) { + case BOOLEAN: + assertThat(actual).as("boolean value should be equal").isEqualTo(expected); + break; + case INTEGER: + assertThat(actual).as("int value should be equal").isEqualTo(expected); + break; + case LONG: + assertThat(actual).as("long value should be equal").isEqualTo(expected); + break; + case FLOAT: + assertThat(actual).as("float value should be equal").isEqualTo(expected); + break; + case DOUBLE: + assertThat(actual).as("double value should be equal").isEqualTo(expected); + break; + case STRING: + assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); + assertThat(actual.toString()) + .as("string should be equal") + .isEqualTo(String.valueOf(expected)); + break; + case DATE: + assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); + LocalDate date = DateTimeUtil.dateFromDays((int) actual); + assertThat(date).as("date should be equal").isEqualTo(expected); + break; + case TIME: + assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); + int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); + assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); + break; + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + assertThat(expected) + .as("Should expect a OffsetDataTime") + .isInstanceOf(OffsetDateTime.class); + OffsetDateTime ts = (OffsetDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("OffsetDataTime should be equal") + .isEqualTo(ts.toLocalDateTime()); + } else { + assertThat(expected) + .as("Should expect a LocalDataTime") + .isInstanceOf(LocalDateTime.class); + LocalDateTime ts = (LocalDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("LocalDataTime should be equal") + .isEqualTo(ts); + } + break; + case BINARY: + assertThat(ByteBuffer.wrap((byte[]) actual)) + .as("Should expect a ByteBuffer") + .isInstanceOf(ByteBuffer.class) + .isEqualTo(expected); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + BigDecimal bd = (BigDecimal) expected; + assertThat(((DecimalData) actual).toBigDecimal()) + .as("decimal value should be equal") + .isEqualTo(bd); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + Collection expectedArrayData = (Collection) expected; + ArrayData actualArrayData = (ArrayData) actual; + LogicalType elementType = ((ArrayType) logicalType).getElementType(); + assertThat(actualArrayData.size()) + .as("array length should be equal") + .isEqualTo(expectedArrayData.size()); + assertArrayValues( + type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); + break; + case MAP: + assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + assertMapValues(type.asMapType(), logicalType, (Map) expected, (MapData) actual); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(StructLike.class); + assertRowData(type.asStructType(), logicalType, (StructLike) expected, (RowData) actual); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); + long firstLong = bb.getLong(); + long secondLong = bb.getLong(); + assertThat(new UUID(firstLong, secondLong).toString()) + .as("UUID should be equal") + .isEqualTo(expected.toString()); + break; + case FIXED: + assertThat(actual) + .as("Should expect byte[]") + .isInstanceOf(byte[].class) + .isEqualTo(expected); + break; + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + public static void assertEquals(Schema schema, List records, List rows) { + Streams.forEachPair( + records.stream(), rows.stream(), (record, row) -> assertEquals(schema, record, row)); + } + + public static void assertEquals(Schema schema, GenericData.Record record, Row row) { + List fields = schema.asStruct().fields(); + assertThat(fields).hasSameSizeAs(record.getSchema().getFields()); + assertThat(fields).hasSize(row.getArity()); + + RowType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < fields.size(); ++i) { + Type fieldType = fields.get(i).type(); + Object expectedValue = record.get(i); + Object actualValue = row.getField(i); + LogicalType logicalType = rowType.getTypeAt(i); + assertAvroEquals(fieldType, logicalType, expectedValue, actualValue); + } + } + + private static void assertEquals(Types.StructType struct, GenericData.Record record, Row row) { + List fields = struct.fields(); + for (int i = 0; i < fields.size(); i += 1) { + Type fieldType = fields.get(i).type(); + Object expectedValue = record.get(i); + Object actualValue = row.getField(i); + assertAvroEquals(fieldType, null, expectedValue, actualValue); + } + } + + private static void assertAvroEquals( + Type type, LogicalType logicalType, Object expected, Object actual) { + + if (expected == null && actual == null) { + return; + } + assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + + switch (type.typeId()) { + case BOOLEAN: + case INTEGER: + case LONG: + case FLOAT: + case DOUBLE: + assertThat(expected) + .as("Should expect a " + type.typeId().javaClass()) + .isInstanceOf(type.typeId().javaClass()); + assertThat(actual) + .as("Should expect a " + type.typeId().javaClass()) + .isInstanceOf(type.typeId().javaClass()); + assertThat(actual).as(type.typeId() + " value should be equal").isEqualTo(expected); + break; + case STRING: + assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); + assertThat(actual).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); + assertThat(actual.toString()).as("string should be equal").isEqualTo(expected.toString()); + break; + case DATE: + assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); + LocalDate date = DateTimeUtil.dateFromDays((int) actual); + assertThat(date).as("date should be equal").isEqualTo(expected); + break; + case TIME: + assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); + int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); + assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); + break; + case TIMESTAMP: + if (((Types.TimestampType) type).shouldAdjustToUTC()) { + assertThat(expected) + .as("Should expect a OffsetDataTime") + .isInstanceOf(OffsetDateTime.class); + OffsetDateTime ts = (OffsetDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("OffsetDataTime should be equal") + .isEqualTo(ts.toLocalDateTime()); + } else { + assertThat(expected) + .as("Should expect a LocalDataTime") + .isInstanceOf(LocalDateTime.class); + LocalDateTime ts = (LocalDateTime) expected; + assertThat(((TimestampData) actual).toLocalDateTime()) + .as("LocalDataTime should be equal") + .isEqualTo(ts); + } + break; + case BINARY: + assertThat(ByteBuffer.wrap((byte[]) actual)) + .as("Should expect a ByteBuffer") + .isInstanceOf(ByteBuffer.class) + .isEqualTo(expected); + break; + case DECIMAL: + assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); + BigDecimal bd = (BigDecimal) expected; + assertThat(((DecimalData) actual).toBigDecimal()) + .as("decimal value should be equal") + .isEqualTo(bd); + break; + case LIST: + assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); + Collection expectedArrayData = (Collection) expected; + ArrayData actualArrayData; + try { + actualArrayData = (ArrayData) actual; + } catch (ClassCastException e) { + actualArrayData = new GenericArrayData((Object[]) actual); + } + LogicalType elementType = ((ArrayType) logicalType).getElementType(); + assertThat(actualArrayData.size()) + .as("array length should be equal") + .isEqualTo(expectedArrayData.size()); + assertArrayValues( + type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); + break; + case MAP: + assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); + MapData actualMap; + try { + actualMap = (MapData) actual; + } catch (ClassCastException e) { + actualMap = new GenericMapData((Map) actual); + } + assertMapValues(type.asMapType(), logicalType, (Map) expected, actualMap); + break; + case STRUCT: + assertThat(expected).as("Should expect a Record").isInstanceOf(GenericData.Record.class); + assertEquals( + type.asNestedType().asStructType(), (GenericData.Record) expected, (Row) actual); + break; + case UUID: + assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); + ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); + long firstLong = bb.getLong(); + long secondLong = bb.getLong(); + assertThat(new UUID(firstLong, secondLong).toString()) + .as("UUID should be equal") + .isEqualTo(expected.toString()); + break; + case FIXED: + assertThat(actual) + .as("Should expect byte[]") + .isInstanceOf(byte[].class) + .isEqualTo(expected); + break; + default: + throw new IllegalArgumentException("Not a supported type: " + type); + } + } + + private static void assertArrayValues( + Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { + List expectedElements = Lists.newArrayList(expectedArray); + for (int i = 0; i < expectedArray.size(); i += 1) { + if (expectedElements.get(i) == null) { + assertThat(actualArray.isNullAt(i)).isTrue(); + continue; + } + + Object expected = expectedElements.get(i); + + assertEquals( + type, + logicalType, + expected, + ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, i)); + } + } + + private static void assertMapValues( + Types.MapType mapType, LogicalType type, Map expected, MapData actual) { + assertThat(actual.size()).as("map size should be equal").isEqualTo(expected.size()); + + ArrayData actualKeyArrayData = actual.keyArray(); + ArrayData actualValueArrayData = actual.valueArray(); + LogicalType actualKeyType = ((MapType) type).getKeyType(); + LogicalType actualValueType = ((MapType) type).getValueType(); + Type keyType = mapType.keyType(); + Type valueType = mapType.valueType(); + + ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(actualKeyType); + ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(actualValueType); + + for (Map.Entry entry : expected.entrySet()) { + Object matchedActualKey = null; + int matchedKeyIndex = 0; + for (int i = 0; i < actual.size(); i += 1) { + try { + Object key = keyGetter.getElementOrNull(actualKeyArrayData, i); + assertEquals(keyType, actualKeyType, entry.getKey(), key); + matchedActualKey = key; + matchedKeyIndex = i; + break; + } catch (AssertionError e) { + // not found + } + } + assertThat(matchedActualKey).as("Should have a matching key").isNotNull(); + final int valueIndex = matchedKeyIndex; + assertEquals( + valueType, + actualValueType, + entry.getValue(), + valueGetter.getElementOrNull(actualValueArrayData, valueIndex)); + } + } + + public static void assertEquals(ManifestFile expected, ManifestFile actual) { + if (expected == actual) { + return; + } + assertThat(expected).isNotNull(); + assertThat(actual).isNotNull(); + assertThat(actual.path()).as("Path must match").isEqualTo(expected.path()); + assertThat(actual.length()).as("Length must match").isEqualTo(expected.length()); + assertThat(actual.partitionSpecId()) + .as("Spec id must match") + .isEqualTo(expected.partitionSpecId()); + assertThat(actual.content()).as("ManifestContent must match").isEqualTo(expected.content()); + assertThat(actual.sequenceNumber()) + .as("SequenceNumber must match") + .isEqualTo(expected.sequenceNumber()); + assertThat(actual.minSequenceNumber()) + .as("MinSequenceNumber must match") + .isEqualTo(expected.minSequenceNumber()); + assertThat(actual.snapshotId()).as("Snapshot id must match").isEqualTo(expected.snapshotId()); + assertThat(actual.hasAddedFiles()) + .as("Added files flag must match") + .isEqualTo(expected.hasAddedFiles()); + assertThat(actual.addedFilesCount()) + .as("Added files count must match") + .isEqualTo(expected.addedFilesCount()); + assertThat(actual.addedRowsCount()) + .as("Added rows count must match") + .isEqualTo(expected.addedRowsCount()); + assertThat(actual.hasExistingFiles()) + .as("Existing files flag must match") + .isEqualTo(expected.hasExistingFiles()); + assertThat(actual.existingFilesCount()) + .as("Existing files count must match") + .isEqualTo(expected.existingFilesCount()); + assertThat(actual.existingRowsCount()) + .as("Existing rows count must match") + .isEqualTo(expected.existingRowsCount()); + assertThat(actual.hasDeletedFiles()) + .as("Deleted files flag must match") + .isEqualTo(expected.hasDeletedFiles()); + assertThat(actual.deletedFilesCount()) + .as("Deleted files count must match") + .isEqualTo(expected.deletedFilesCount()); + assertThat(actual.deletedRowsCount()) + .as("Deleted rows count must match") + .isEqualTo(expected.deletedRowsCount()); + + List expectedSummaries = expected.partitions(); + List actualSummaries = actual.partitions(); + assertThat(actualSummaries) + .as("PartitionFieldSummary size does not match") + .hasSameSizeAs(expectedSummaries); + for (int i = 0; i < expectedSummaries.size(); i++) { + assertThat(actualSummaries.get(i).containsNull()) + .as("Null flag in partition must match") + .isEqualTo(expectedSummaries.get(i).containsNull()); + assertThat(actualSummaries.get(i).containsNaN()) + .as("NaN flag in partition must match") + .isEqualTo(expectedSummaries.get(i).containsNaN()); + assertThat(actualSummaries.get(i).lowerBound()) + .as("Lower bounds in partition must match") + .isEqualTo(expectedSummaries.get(i).lowerBound()); + assertThat(actualSummaries.get(i).upperBound()) + .as("Upper bounds in partition must match") + .isEqualTo(expectedSummaries.get(i).upperBound()); + } + } + + public static void assertEquals(ContentFile expected, ContentFile actual) { + if (expected == actual) { + return; + } + assertThat(expected).isNotNull(); + 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.format()).as("Format").isEqualTo(expected.format()); + assertThat(actual.partition().size()) + .as("Partition size") + .isEqualTo(expected.partition().size()); + for (int i = 0; i < expected.partition().size(); i++) { + assertThat(actual.partition().get(i, Object.class)) + .as("Partition data at index " + i) + .isEqualTo(expected.partition().get(i, Object.class)); + } + assertThat(actual.recordCount()).as("Record count").isEqualTo(expected.recordCount()); + assertThat(actual.fileSizeInBytes()) + .as("File size in bytes") + .isEqualTo(expected.fileSizeInBytes()); + assertThat(actual.columnSizes()).as("Column sizes").isEqualTo(expected.columnSizes()); + assertThat(actual.valueCounts()).as("Value counts").isEqualTo(expected.valueCounts()); + assertThat(actual.nullValueCounts()) + .as("Null value counts") + .isEqualTo(expected.nullValueCounts()); + assertThat(actual.lowerBounds()).as("Lower bounds").isEqualTo(expected.lowerBounds()); + assertThat(actual.upperBounds()).as("Upper bounds").isEqualTo(expected.upperBounds()); + assertThat(actual.keyMetadata()).as("Key metadata").isEqualTo(expected.keyMetadata()); + assertThat(actual.splitOffsets()).as("Split offsets").isEqualTo(expected.splitOffsets()); + assertThat(actual.equalityFieldIds()) + .as("Equality field id list") + .isEqualTo(expected.equalityFieldIds()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java new file mode 100644 index 000000000000..47f5485df879 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -0,0 +1,331 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.file.Files; +import java.util.Map; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.catalog.Catalog; +import org.apache.flink.table.catalog.ObjectPath; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +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.thrift.TException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergConnector extends TestBase { + + private static final String TABLE_NAME = "test_table"; + + @Parameter(index = 0) + private String catalogName; + + @Parameter(index = 1) + private Map properties; + + @Parameter(index = 2) + private boolean isStreaming; + + private volatile TableEnvironment tEnv; + + @Parameters(name = "catalogName = {0}, properties = {1}, isStreaming = {2}") + public static Iterable parameters() { + return Lists.newArrayList( + // Create iceberg table in the hadoop catalog and default database. + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop"), + false + }, + // Create iceberg table in the hadoop catalog and not_existing_db. + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhadoop", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hadoop", + "catalog-database", "not_existing_db"), + false + }, + // Create iceberg table in the hive catalog and default database. + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive"), + false + }, + // Create iceberg table in the hive catalog and not_existing_db. + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db", + "catalog-table", "not_existing_table"), + true + }, + new Object[] { + "testhive", + ImmutableMap.of( + "connector", "iceberg", + "catalog-type", "hive", + "catalog-database", "not_existing_db"), + false + }); + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreaming) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + // Set only one parallelism. + tEnv.getConfig() + .getConfiguration() + .set(CoreOptions.DEFAULT_PARALLELISM, 1) + .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + } + } + } + return tEnv; + } + + @AfterEach + public void after() throws TException { + sql("DROP TABLE IF EXISTS %s", TABLE_NAME); + + // Clean the created orphan databases and tables from hive-metastore. + if (isHiveCatalog()) { + HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf); + try { + metaStoreClient.dropTable(databaseName(), tableName()); + if (!isDefaultDatabaseName()) { + try { + metaStoreClient.dropDatabase(databaseName()); + } catch (Exception ignored) { + // Ignore + } + } + } finally { + metaStoreClient.close(); + } + } + } + + private void testCreateConnectorTable() { + Map tableProps = createTableProps(); + + // Create table under the flink's current database. + sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); + sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); + + FlinkCatalogFactory factory = new FlinkCatalogFactory(); + Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); + assertThat(flinkCatalog.databaseExists(databaseName())).isTrue(); + assertThat(flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))).isTrue(); + + // Drop and create it again. + sql("DROP TABLE %s", TABLE_NAME); + sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); + assertThat(sql("SELECT * FROM %s", TABLE_NAME)) + .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); + } + + @TestTemplate + public void testCreateTableUnderDefaultDatabase() { + testCreateConnectorTable(); + } + + @TestTemplate + public void testCatalogDatabaseConflictWithFlinkDatabase() { + sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); + sql("USE `%s`", databaseName()); + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); + } + + @TestTemplate + public void testConnectorTableInIcebergCatalog() { + // Create the catalog properties + Map catalogProps = Maps.newHashMap(); + catalogProps.put("type", "iceberg"); + if (isHiveCatalog()) { + catalogProps.put("catalog-type", "hive"); + catalogProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + } else { + catalogProps.put("catalog-type", "hadoop"); + } + catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); + + // Create the table properties + Map tableProps = createTableProps(); + + // Create a connector table in an iceberg catalog. + sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); + try { + assertThatThrownBy( + () -> + sql( + "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", + FlinkCatalogFactory.DEFAULT_DATABASE_NAME, + TABLE_NAME, + toWithClause(tableProps))) + .cause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " + + "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " + + "create table without 'connector'='iceberg' related properties in an iceberg table."); + } finally { + sql("DROP CATALOG IF EXISTS `test_catalog`"); + } + } + + private Map createTableProps() { + Map tableProps = Maps.newHashMap(properties); + tableProps.put("catalog-name", catalogName); + tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); + if (isHiveCatalog()) { + tableProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); + } + return tableProps; + } + + private boolean isHiveCatalog() { + return "testhive".equalsIgnoreCase(catalogName); + } + + private boolean isDefaultDatabaseName() { + return FlinkCatalogFactory.DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName()); + } + + private String tableName() { + return properties.getOrDefault("catalog-table", TABLE_NAME); + } + + private String databaseName() { + return properties.getOrDefault("catalog-database", "default_database"); + } + + private String createWarehouse() { + try { + return String.format( + "file://%s", + Files.createTempDirectory(temporaryDirectory, "junit").toFile().getAbsolutePath()); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java new file mode 100644 index 000000000000..8f1f129e183b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java @@ -0,0 +1,173 @@ +/* + * 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; + +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.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Path; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.ManifestWriter; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestManifestFileSerialization { + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + required(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("double").build(); + + private static final DataFile FILE_A = + DataFiles.builder(SPEC) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withPartition(org.apache.iceberg.TestHelpers.Row.of(1D)) + .withPartitionPath("double=1") + .withMetrics( + new Metrics( + 5L, + null, // no column sizes + ImmutableMap.of(1, 5L, 2, 3L), // value count + ImmutableMap.of(1, 0L, 2, 2L), // null count + ImmutableMap.of(), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private static final DataFile FILE_B = + DataFiles.builder(SPEC) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withPartition(org.apache.iceberg.TestHelpers.Row.of(Double.NaN)) + .withPartitionPath("double=NaN") + .withMetrics( + new Metrics( + 1L, + null, // no column sizes + ImmutableMap.of(1, 1L, 4, 1L), // value count + ImmutableMap.of(1, 0L, 2, 0L), // null count + ImmutableMap.of(4, 1L), // nan count + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(1L)) // upper bounds + )) + .build(); + + private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); + + @TempDir private Path temp; + + @Test + public void testKryoSerialization() throws IOException { + KryoSerializer kryo = + new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); + + DataOutputSerializer outputView = new DataOutputSerializer(1024); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + kryo.serialize(manifest, outputView); + kryo.serialize(manifest.copy(), outputView); + kryo.serialize(GenericManifestFile.copyOf(manifest).build(), outputView); + + DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); + ManifestFile m1 = kryo.deserialize(inputView); + ManifestFile m2 = kryo.deserialize(inputView); + ManifestFile m3 = kryo.deserialize(inputView); + + TestHelpers.assertEquals(manifest, m1); + TestHelpers.assertEquals(manifest, m2); + TestHelpers.assertEquals(manifest, m3); + } + + @Test + public void testJavaSerialization() throws Exception { + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + + ManifestFile manifest = writeManifest(FILE_A, FILE_B); + + try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { + out.writeObject(manifest); + out.writeObject(manifest.copy()); + out.writeObject(GenericManifestFile.copyOf(manifest).build()); + } + + try (ObjectInputStream in = + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { + for (int i = 0; i < 3; i += 1) { + Object obj = in.readObject(); + assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); + TestHelpers.assertEquals(manifest, (ManifestFile) obj); + } + } + } + + private ManifestFile writeManifest(DataFile... files) throws IOException { + File manifestFile = File.createTempFile("input", "m0.avro", temp.toFile()); + assertThat(manifestFile.delete()).isTrue(); + OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); + + ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); + try { + for (DataFile file : files) { + writer.add(file); + } + } finally { + writer.close(); + } + + return writer.toManifestFile(); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java new file mode 100644 index 000000000000..0af49e9e2365 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java @@ -0,0 +1,93 @@ +/* + * 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; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Iterator; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.RecordWrapperTest; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.util.StructLikeWrapper; + +public class TestRowDataWrapper extends RecordWrapperTest { + + /** + * Flink's time type has been truncated to millis seconds, so we need a customized assert method + * to check the values. + */ + @Override + public void testTime() { + generateAndValidate( + new Schema(TIME.fields()), + (message, expectedWrapper, actualWrapper) -> { + for (int pos = 0; pos < TIME.fields().size(); pos++) { + Object expected = expectedWrapper.get().get(pos, Object.class); + Object actual = actualWrapper.get().get(pos, Object.class); + if (expected == actual) { + return; + } + + assertThat(actual).isNotNull(); + assertThat(expected).isNotNull(); + + int expectedMilliseconds = (int) ((long) expected / 1000_000); + int actualMilliseconds = (int) ((long) actual / 1000_000); + assertThat(actualMilliseconds).as(message).isEqualTo(expectedMilliseconds); + } + }); + } + + @Override + protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { + int numRecords = 100; + Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); + Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); + + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + Iterator actual = recordList.iterator(); + Iterator expected = rowDataList.iterator(); + + StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); + StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); + for (int i = 0; i < numRecords; i++) { + assertThat(actual).hasNext(); + assertThat(expected).hasNext(); + + StructLike recordStructLike = recordWrapper.wrap(actual.next()); + StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); + + assertMethod.assertEquals( + "Should have expected StructLike values", + expectedWrapper.set(rowDataStructLike), + actualWrapper.set(recordStructLike)); + } + + assertThat(actual).isExhausted(); + assertThat(expected).isExhausted(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java new file mode 100644 index 000000000000..a7c58e551112 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.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.flink; + +import java.io.File; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestTables; + +public class TestTableLoader implements TableLoader { + private final File dir; + + public static TableLoader of(String dir) { + return new TestTableLoader(dir); + } + + public TestTableLoader(String dir) { + this.dir = new File(dir); + } + + @Override + public void open() {} + + @Override + public boolean isOpen() { + return true; + } + + @Override + public Table loadTable() { + return TestTables.load(dir, "test"); + } + + @Override + @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) + public TableLoader clone() { + return new TestTableLoader(dir.getAbsolutePath()); + } + + @Override + public void close() {} +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java new file mode 100644 index 000000000000..7f0e7acaa822 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.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.flink; + +import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; +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.Map; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.Transaction; +import org.apache.iceberg.hadoop.HadoopTables; +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; + +public class TestTableSerialization { + private static final HadoopTables TABLES = new HadoopTables(); + + private static final Schema SCHEMA = + new Schema( + required(1, "id", Types.LongType.get()), + optional(2, "data", Types.StringType.get()), + required(3, "date", Types.StringType.get()), + optional(4, "double", Types.DoubleType.get())); + + private static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("date").build(); + + private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + + @TempDir private Path temp; + private Table table; + + @BeforeEach + public void initTable() throws IOException { + Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); + + File tableLocation = File.createTempFile("junit", null, temp.toFile()); + assertThat(tableLocation.delete()).isTrue(); + + this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); + } + + @Test + public void testSerializableTableKryoSerialization() throws IOException { + SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); + TestHelpers.assertSerializedAndLoadedMetadata( + table, roundTripKryoSerialize(SerializableTable.class, serializableTable)); + } + + @Test + public void testSerializableMetadataTableKryoSerialization() throws IOException { + for (MetadataTableType type : MetadataTableType.values()) { + TableOperations ops = ((HasTableOperations) table).operations(); + Table metadataTable = + MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); + SerializableTable serializableMetadataTable = + (SerializableTable) SerializableTable.copyOf(metadataTable); + + TestHelpers.assertSerializedAndLoadedMetadata( + metadataTable, + roundTripKryoSerialize(SerializableTable.class, serializableMetadataTable)); + } + } + + @Test + public void testSerializableTransactionTableKryoSerialization() throws IOException { + Transaction txn = table.newTransaction(); + + txn.updateProperties().set("k1", "v1").commit(); + + Table txnTable = txn.table(); + SerializableTable serializableTxnTable = (SerializableTable) SerializableTable.copyOf(txnTable); + + TestHelpers.assertSerializedMetadata( + txnTable, roundTripKryoSerialize(SerializableTable.class, serializableTxnTable)); + } +} 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 new file mode 100644 index 000000000000..6b8399f666d4 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java @@ -0,0 +1,481 @@ +/* + * 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.actions; + +import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; +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.Path; +import java.util.Collection; +import java.util.List; +import java.util.Set; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.commons.lang3.StringUtils; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.RewriteDataFilesActionResult; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +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.types.Types; +import org.apache.iceberg.util.Pair; +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; + +public class TestRewriteDataFilesAction extends CatalogTestBase { + + private static final String TABLE_NAME_UNPARTITIONED = "test_table_unpartitioned"; + private static final String TABLE_NAME_PARTITIONED = "test_table_partitioned"; + private static final String TABLE_NAME_WITH_PK = "test_table_with_pk"; + + @Parameter(index = 2) + private FileFormat format; + + private Table icebergTableUnPartitioned; + private Table icebergTablePartitioned; + private Table icebergTableWithPk; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") + 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}); + } + } + return parameters; + } + + private @TempDir Path temp; + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + 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()); + 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()); + 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()); + icebergTableWithPk = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + @TestTemplate + public void testRewriteDataFilesEmptyTable() throws Exception { + assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); + Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); + assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); + } + + @TestTemplate + public void testRewriteDataFilesUnpartitionedTable() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(2); + RewriteDataFilesActionResult result = + Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); + + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFiles1).hasSize(1); + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTableUnPartitioned, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), SimpleDataUtil.createRecord(2, "world"))); + } + + @TestTemplate + public void testRewriteDataFilesPartitionedTable() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 3, 'world' ,'b'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(4); + RewriteDataFilesActionResult result = + Actions.forTable(icebergTablePartitioned).rewriteDataFiles().execute(); + + assertThat(result.deletedDataFiles()).hasSize(4); + assertThat(result.addedDataFiles()).hasSize(2); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFiles1).hasSize(2); + // Assert the table records as expected. + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "spec", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + SimpleDataUtil.assertTableRecords( + icebergTablePartitioned, + Lists.newArrayList( + record.copy("id", 1, "data", "hello", "spec", "a"), + record.copy("id", 2, "data", "hello", "spec", "a"), + record.copy("id", 3, "data", "world", "spec", "b"), + record.copy("id", 4, "data", "world", "spec", "b"))); + } + + @TestTemplate + public void testRewriteDataFilesWithFilter() throws Exception { + sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 3, 'world' ,'a'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); + sql("INSERT INTO %s SELECT 5, 'world' ,'b'", TABLE_NAME_PARTITIONED); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(5); + RewriteDataFilesActionResult result = + Actions.forTable(icebergTablePartitioned) + .rewriteDataFiles() + .filter(Expressions.equal("spec", "a")) + .filter(Expressions.startsWith("data", "he")) + .execute(); + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + + icebergTablePartitioned.refresh(); + + CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); + List dataFiles1 = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFiles1).hasSize(4); + // Assert the table records as expected. + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get()), + Types.NestedField.optional(3, "spec", Types.StringType.get())); + + Record record = GenericRecord.create(schema); + SimpleDataUtil.assertTableRecords( + icebergTablePartitioned, + Lists.newArrayList( + record.copy("id", 1, "data", "hello", "spec", "a"), + record.copy("id", 2, "data", "hello", "spec", "a"), + record.copy("id", 3, "data", "world", "spec", "a"), + record.copy("id", 4, "data", "world", "spec", "b"), + record.copy("id", 5, "data", "world", "spec", "b"))); + } + + @TestTemplate + public void testRewriteLargeTableHasResiduals() throws IOException { + // all records belong to the same partition + List records1 = Lists.newArrayList(); + List records2 = Lists.newArrayList(); + List expected = Lists.newArrayList(); + for (int i = 0; i < 100; i++) { + int id = i; + String data = String.valueOf(i % 3); + if (i % 2 == 0) { + records1.add("(" + id + ",'" + data + "')"); + } else { + records2.add("(" + id + ",'" + data + "')"); + } + Record record = RECORD.copy(); + record.setField("id", id); + record.setField("data", data); + expected.add(record); + } + + sql("INSERT INTO %s values " + StringUtils.join(records1, ","), TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s values " + StringUtils.join(records2, ","), TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = + icebergTableUnPartitioned + .newScan() + .ignoreResiduals() + .filter(Expressions.equal("data", "0")) + .planFiles(); + for (FileScanTask task : tasks) { + assertThat(task.residual()) + .as("Residuals must be ignored") + .isEqualTo(Expressions.alwaysTrue()); + } + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(2); + Actions actions = Actions.forTable(icebergTableUnPartitioned); + + RewriteDataFilesActionResult result = + actions.rewriteDataFiles().filter(Expressions.equal("data", "0")).execute(); + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); + } + + /** + * a test case to test avoid repeate compress + * + *

    If datafile cannot be combined to CombinedScanTask with other DataFiles, the size of the + * CombinedScanTask list size is 1, so we remove these CombinedScanTasks to avoid compressed + * repeatedly. + * + *

    In this test case,we generated 3 data files and set targetSizeInBytes greater than the + * largest file size so that it cannot be combined a CombinedScanTask with other datafiles. The + * datafile with the largest file size will not be compressed. + * + * @throws IOException IOException + */ + @TestTemplate + public void testRewriteAvoidRepeateCompress() throws IOException { + List expected = Lists.newArrayList(); + Schema schema = icebergTableUnPartitioned.schema(); + GenericAppenderFactory genericAppenderFactory = new GenericAppenderFactory(schema); + File file = File.createTempFile("junit", null, temp.toFile()); + int count = 0; + try (FileAppender fileAppender = + genericAppenderFactory.newAppender(Files.localOutput(file), format)) { + long filesize = 20000; + for (; fileAppender.length() < filesize; count++) { + Record record = SimpleDataUtil.createRecord(count, UUID.randomUUID().toString()); + fileAppender.add(record); + expected.add(record); + } + } + + DataFile dataFile = + DataFiles.builder(icebergTableUnPartitioned.spec()) + .withPath(file.getAbsolutePath()) + .withFileSizeInBytes(file.length()) + .withFormat(format) + .withRecordCount(count) + .build(); + + icebergTableUnPartitioned.newAppend().appendFile(dataFile).commit(); + + sql("INSERT INTO %s SELECT 1,'a' ", TABLE_NAME_UNPARTITIONED); + sql("INSERT INTO %s SELECT 2,'b' ", TABLE_NAME_UNPARTITIONED); + + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + assertThat(dataFiles).hasSize(3); + Actions actions = Actions.forTable(icebergTableUnPartitioned); + + long targetSizeInBytes = file.length() + 10; + RewriteDataFilesActionResult result = + actions + .rewriteDataFiles() + .targetSizeInBytes(targetSizeInBytes) + .splitOpenFileCost(1) + .execute(); + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + icebergTableUnPartitioned.refresh(); + + CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); + List dataFilesRewrote = + Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); + assertThat(dataFilesRewrote).hasSize(2); + // the biggest file do not be rewrote + List rewroteDataFileNames = + dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); + assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); + + // Assert the table records as expected. + expected.add(SimpleDataUtil.createRecord(1, "a")); + expected.add(SimpleDataUtil.createRecord(2, "b")); + SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); + } + + @TestTemplate + public void testRewriteNoConflictWithEqualityDeletes() throws IOException { + // Add 2 data files + sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_WITH_PK); + sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_WITH_PK); + + // Load 2 stale tables to pass to rewrite actions + // Since the first rewrite will refresh stale1, we need another stale2 for the second rewrite + Table stale1 = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + Table stale2 = + validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); + + // Add 1 data file and 1 equality-delete file + sql("INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ SELECT 1, 'hi'", TABLE_NAME_WITH_PK); + + icebergTableWithPk.refresh(); + assertThat(icebergTableWithPk.currentSnapshot().sequenceNumber()) + .as("The latest sequence number should be greater than that of the stale snapshot") + .isEqualTo(stale1.currentSnapshot().sequenceNumber() + 1); + CloseableIterable tasks = icebergTableWithPk.newScan().planFiles(); + List dataFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); + Set deleteFiles = + Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::deletes)).stream() + .flatMap(Collection::stream) + .collect(Collectors.toSet()); + assertThat(dataFiles).hasSize(3); + assertThat(deleteFiles).hasSize(1); + assertThat(Iterables.getOnlyElement(deleteFiles).content()) + .isEqualTo(FileContent.EQUALITY_DELETES); + shouldHaveDataAndFileSequenceNumbers( + TABLE_NAME_WITH_PK, + ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); + + assertThatThrownBy( + () -> + Actions.forTable(stale1) + .rewriteDataFiles() + .useStartingSequenceNumber(false) + .execute(), + "Rewrite using new sequence number should fail") + .isInstanceOf(ValidationException.class); + + // Rewrite using the starting sequence number should succeed + RewriteDataFilesActionResult result = + Actions.forTable(stale2).rewriteDataFiles().useStartingSequenceNumber(true).execute(); + + // Should not rewrite files from the new commit + assertThat(result.deletedDataFiles()).hasSize(2); + assertThat(result.addedDataFiles()).hasSize(1); + // The 2 older files with file-sequence-number <= 2 should be rewritten into a new file. + // The new file is the one with file-sequence-number == 4. + // The new file should use rewrite's starting-sequence-number 2 as its data-sequence-number. + shouldHaveDataAndFileSequenceNumbers( + TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(3L, 3L), Pair.of(3L, 3L), Pair.of(2L, 4L))); + + // Assert the table records as expected. + SimpleDataUtil.assertTableRecords( + icebergTableWithPk, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hi"), SimpleDataUtil.createRecord(2, "world"))); + } + + /** + * Assert that data files and delete files in the table should have expected data sequence numbers + * and file sequence numbers + * + * @param tableName table name + * @param expectedSequenceNumbers list of {@link Pair}'s. Each {@link Pair} contains + * (expectedDataSequenceNumber, expectedFileSequenceNumber) of a file. + */ + private void shouldHaveDataAndFileSequenceNumbers( + String tableName, List> expectedSequenceNumbers) { + // "status < 2" for added or existing entries + List liveEntries = sql("SELECT * FROM %s$entries WHERE status < 2", tableName); + + List> actualSequenceNumbers = + liveEntries.stream() + .map( + row -> + Pair.of( + row.getFieldAs("sequence_number"), row.getFieldAs("file_sequence_number"))) + .collect(Collectors.toList()); + assertThat(actualSequenceNumbers).hasSameElementsAs(expectedSequenceNumbers); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java new file mode 100644 index 000000000000..cc58d9817ac6 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.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 org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class RandomRowData { + private RandomRowData() {} + + public static Iterable generate(Schema schema, int numRecords, long seed) { + return convert(schema, RandomGenericData.generate(schema, numRecords, seed)); + } + + public static Iterable convert(Schema schema, Iterable records) { + return Iterables.transform(records, record -> RowDataConverter.convert(schema, record)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java new file mode 100644 index 000000000000..74b1da6007e6 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.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.flink.data; + +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.configuration.Configuration; +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; + +public class RowDataToRowMapper extends RichMapFunction { + + private final RowType rowType; + + private transient DataStructureConverter converter; + + public RowDataToRowMapper(RowType rowType) { + this.rowType = rowType; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.converter = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); + } + + @Override + public Row map(RowData value) throws Exception { + return (Row) converter.toExternal(value); + } +} 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/TestFlinkAvroReaderWriter.java new file mode 100644 index 000000000000..a1039d27d888 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java @@ -0,0 +1,185 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Time; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.DataTest; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +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.iceberg.util.DateTimeUtil; +import org.junit.jupiter.api.Test; + +public class TestFlinkAvroReaderWriter extends DataTest { + + private static final int NUM_RECORDS = 100; + + private static final Schema SCHEMA_NUM_TYPE = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "int", Types.IntegerType.get()), + Types.NestedField.optional(3, "float", Types.FloatType.get()), + Types.NestedField.optional(4, "double", Types.DoubleType.get()), + Types.NestedField.optional(5, "date", Types.DateType.get()), + Types.NestedField.optional(6, "time", Types.TimeType.get()), + Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone()), + Types.NestedField.optional(8, "bigint", Types.LongType.get()), + Types.NestedField.optional(9, "decimal", Types.DecimalType.of(4, 2))); + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L); + writeAndValidate(schema, expectedRecords, NUM_RECORDS); + } + + private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) + throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); + + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); + + // Write the expected records into AVRO file, then read them into RowData and assert with the + // expected Record list. + try (FileAppender writer = + Avro.write(Files.localOutput(recordsFile)) + .schema(schema) + .createWriterFunc(DataWriter::create) + .build()) { + writer.addAll(expectedRecords); + } + + try (CloseableIterable reader = + Avro.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(FlinkAvroReader::new) + .build()) { + Iterator expected = expectedRecords.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < numRecord; i++) { + assertThat(rows).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); + } + assertThat(rows).isExhausted(); + } + + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); + + // Write the expected RowData into AVRO file, then read them into Record and assert with the + // expected RowData list. + try (FileAppender writer = + Avro.write(Files.localOutput(rowDataFile)) + .schema(schema) + .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) + .build()) { + writer.addAll(expectedRows); + } + + try (CloseableIterable reader = + Avro.read(Files.localInput(rowDataFile)) + .project(schema) + .createReaderFunc(DataReader::create) + .build()) { + Iterator expected = expectedRows.iterator(); + Iterator records = reader.iterator(); + for (int i = 0; i < numRecord; i += 1) { + assertThat(records).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); + } + assertThat(records).isExhausted(); + } + } + + private Record recordNumType( + int id, + int intV, + float floatV, + double doubleV, + long date, + long time, + long timestamp, + long bigint, + double decimal) { + Record record = GenericRecord.create(SCHEMA_NUM_TYPE); + record.setField("id", id); + record.setField("int", intV); + record.setField("float", floatV); + record.setField("double", doubleV); + record.setField( + "date", DateTimeUtil.dateFromDays((int) new Date(date).toLocalDate().toEpochDay())); + record.setField("time", new Time(time).toLocalTime()); + record.setField("timestamp", DateTimeUtil.timestampFromMicros(timestamp * 1000)); + record.setField("bigint", bigint); + record.setField("decimal", BigDecimal.valueOf(decimal)); + return record; + } + + @Test + public void testNumericTypes() throws IOException { + + List expected = + ImmutableList.of( + recordNumType( + 2, + Integer.MAX_VALUE, + Float.MAX_VALUE, + Double.MAX_VALUE, + Long.MAX_VALUE, + 1643811742000L, + 1643811742000L, + 1643811742000L, + 10.24d), + recordNumType( + 2, + Integer.MIN_VALUE, + Float.MIN_VALUE, + Double.MIN_VALUE, + Long.MIN_VALUE, + 1643811742000L, + 1643811742000L, + 1643811742000L, + 10.24d)); + + writeAndValidate(SCHEMA_NUM_TYPE, expected, 2); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java new file mode 100644 index 000000000000..72f2ce4f4bce --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.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.flink.data; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.orc.GenericOrcReader; +import org.apache.iceberg.data.orc.GenericOrcWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.orc.ORC; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class TestFlinkOrcReaderWriter extends DataTest { + private static final int NUM_RECORDS = 100; + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + RowType flinkSchema = FlinkSchemaUtil.convert(schema); + List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); + List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); + + File recordsFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(recordsFile.delete()).isTrue(); + + // Write the expected records into ORC file, then read them into RowData and assert with the + // expected Record list. + try (FileAppender writer = + ORC.write(Files.localOutput(recordsFile)) + .schema(schema) + .createWriterFunc(GenericOrcWriter::buildWriter) + .build()) { + writer.addAll(expectedRecords); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(recordsFile)) + .project(schema) + .createReaderFunc(type -> new FlinkOrcReader(schema, type)) + .build()) { + Iterator expected = expectedRecords.iterator(); + Iterator rows = reader.iterator(); + for (int i = 0; i < NUM_RECORDS; i++) { + assertThat(rows).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); + } + assertThat(rows).isExhausted(); + } + + File rowDataFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(rowDataFile.delete()).isTrue(); + + // Write the expected RowData into ORC file, then read them into Record and assert with the + // expected RowData list. + RowType rowType = FlinkSchemaUtil.convert(schema); + try (FileAppender writer = + ORC.write(Files.localOutput(rowDataFile)) + .schema(schema) + .createWriterFunc((iSchema, typeDesc) -> FlinkOrcWriter.buildWriter(rowType, iSchema)) + .build()) { + writer.addAll(expectedRows); + } + + try (CloseableIterable reader = + ORC.read(Files.localInput(rowDataFile)) + .project(schema) + .createReaderFunc(type -> GenericOrcReader.buildReader(schema, type)) + .build()) { + Iterator expected = expectedRows.iterator(); + Iterator records = reader.iterator(); + for (int i = 0; i < NUM_RECORDS; i += 1) { + assertThat(records.hasNext()).isTrue(); + TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); + } + assertThat(records).isExhausted(); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java new file mode 100644 index 000000000000..4cfb24f62921 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java @@ -0,0 +1,239 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.parquet.schema.Types.primitive; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Iterator; +import java.util.List; +import org.apache.avro.generic.GenericData; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.GenericRecordBuilder; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.apache.iceberg.parquet.ParquetValueReader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.apache.parquet.avro.AvroParquetWriter; +import org.apache.parquet.hadoop.ParquetWriter; +import org.apache.parquet.schema.LogicalTypeAnnotation; +import org.apache.parquet.schema.MessageType; +import org.apache.parquet.schema.PrimitiveType; +import org.apache.parquet.schema.Type; +import org.junit.jupiter.api.Test; + +public class TestFlinkParquetReader extends DataTest { + private static final int NUM_RECORDS = 100; + + @Test + public void testBuildReader() { + MessageType fileSchema = + new MessageType( + "test", + // 0: required(100, "id", LongType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(100) + .named("id"), + // 1: optional(101, "data", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(101) + .named("data"), + // 2: required(102, "b", Types.BooleanType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, Type.Repetition.REQUIRED) + .id(102) + .named("b"), + // 3: optional(103, "i", Types.IntegerType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(103) + .named("i"), + // 4: optional(105, "f", Types.FloatType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(104) + .named("l"), + // 5: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.FLOAT, Type.Repetition.OPTIONAL) + .id(105) + .named("f"), + // 6: required(106, "d", Types.DoubleType.get()) + primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, Type.Repetition.REQUIRED) + .id(106) + .named("d"), + // 7: optional(107, "date", Types.DateType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) + .id(107) + .as(LogicalTypeAnnotation.dateType()) + .named("date"), + // 8: required(108, "ts_tz", Types.TimestampType.withZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(108) + .as( + LogicalTypeAnnotation.timestampType( + true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts_tz"), + // 9: required(109, "ts", Types.TimestampType.withoutZone()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(109) + .as( + LogicalTypeAnnotation.timestampType( + false, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("ts"), + // 10: required(110, "s", Types.StringType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) + .id(110) + .as(LogicalTypeAnnotation.stringType()) + .named("s"), + // 11: required(112, "fixed", Types.FixedType.ofLength(7)) + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(112) + .length(7) + .named("f"), + // 12: optional(113, "bytes", Types.BinaryType.get()) + primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) + .id(113) + .named("bytes"), + // 13: required(114, "dec_9_0", Types.DecimalType.of(9, 0)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(114) + .as(LogicalTypeAnnotation.decimalType(0, 9)) + .named("dec_9_0"), + // 14: required(115, "dec_11_2", Types.DecimalType.of(11, 2)) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) + .id(115) + .as(LogicalTypeAnnotation.decimalType(2, 11)) + .named("dec_11_2"), + // 15: required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision + primitive( + PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) + .id(116) + .length(16) + .as(LogicalTypeAnnotation.decimalType(10, 38)) + .named("dec_38_10"), + // 16: required(117, "time", Types.TimeType.get()) + primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) + .id(117) + .as(LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) + .named("time")); + ParquetValueReader reader = + FlinkParquetReaders.buildReader(new Schema(SUPPORTED_PRIMITIVES.fields()), fileSchema); + + assertThat(reader.columns().size()).isEqualTo(SUPPORTED_PRIMITIVES.fields().size()); + } + + @Test + public void testTwoLevelList() throws IOException { + Schema schema = + new Schema( + optional(1, "arraybytes", Types.ListType.ofRequired(3, Types.BinaryType.get())), + optional(2, "topbytes", Types.BinaryType.get())); + org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); + + ParquetWriter writer = + AvroParquetWriter.builder(new Path(testFile.toURI())) + .withDataModel(GenericData.get()) + .withSchema(avroSchema) + .config("parquet.avro.add-list-element-records", "true") + .config("parquet.avro.write-old-list-structure", "true") + .build(); + + GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema); + List expectedByteList = Lists.newArrayList(); + byte[] expectedByte = {0x00, 0x01}; + ByteBuffer expectedBinary = ByteBuffer.wrap(expectedByte); + expectedByteList.add(expectedBinary); + recordBuilder.set("arraybytes", expectedByteList); + recordBuilder.set("topbytes", expectedBinary); + GenericData.Record expectedRecord = recordBuilder.build(); + + writer.write(expectedRecord); + writer.close(); + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) + .build()) { + Iterator rows = reader.iterator(); + assertThat(rows).hasNext(); + RowData rowData = rows.next(); + assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); + assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); + assertThat(rows).isExhausted(); + } + } + + private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(GenericParquetWriter::buildWriter) + .build()) { + writer.addAll(iterable); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) + .build()) { + Iterator expected = iterable.iterator(); + Iterator rows = reader.iterator(); + LogicalType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < NUM_RECORDS; i += 1) { + assertThat(rows).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); + } + assertThat(rows).isExhausted(); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(RandomGenericData.generate(schema, NUM_RECORDS, 19981), schema); + writeAndValidate( + RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124), schema); + writeAndValidate( + RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), + schema); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java new file mode 100644 index 000000000000..b1e6f5aa00ff --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java @@ -0,0 +1,94 @@ +/* + * 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 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 org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.DataTest; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetReaders; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.parquet.Parquet; +import org.junit.jupiter.api.io.TempDir; + +public class TestFlinkParquetWriter extends DataTest { + private static final int NUM_RECORDS = 100; + + @TempDir private Path temp; + + private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { + File testFile = File.createTempFile("junit", null, temp.toFile()); + assertThat(testFile.delete()).isTrue(); + + LogicalType logicalType = FlinkSchemaUtil.convert(schema); + + try (FileAppender writer = + Parquet.write(Files.localOutput(testFile)) + .schema(schema) + .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(logicalType, msgType)) + .build()) { + writer.addAll(iterable); + } + + try (CloseableIterable reader = + Parquet.read(Files.localInput(testFile)) + .project(schema) + .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) + .build()) { + Iterator expected = iterable.iterator(); + Iterator actual = reader.iterator(); + LogicalType rowType = FlinkSchemaUtil.convert(schema); + for (int i = 0; i < NUM_RECORDS; i += 1) { + assertThat(actual).hasNext(); + TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); + } + assertThat(actual).isExhausted(); + } + } + + @Override + protected void writeAndValidate(Schema schema) throws IOException { + writeAndValidate(RandomRowData.generate(schema, NUM_RECORDS, 19981), schema); + + writeAndValidate( + RandomRowData.convert( + schema, + RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124)), + schema); + + writeAndValidate( + RandomRowData.convert( + schema, + RandomGenericData.generateFallbackRecords( + schema, NUM_RECORDS, 21124, NUM_RECORDS / 20)), + schema); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java new file mode 100644 index 000000000000..d078b2228456 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java @@ -0,0 +1,593 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatNoException; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.flink.table.data.GenericArrayData; +import org.apache.flink.table.data.GenericMapData; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.TestHelpers; +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.apache.iceberg.util.StructProjection; +import org.junit.jupiter.api.Test; + +public class TestRowDataProjection { + @Test + public void testNullRootRowData() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); + + assertThatThrownBy(() -> projection.wrap(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid row data: null"); + } + + @Test + public void testFullProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + generateAndValidate(schema, schema); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + } + + @Test + public void testReorderedFullProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + generateAndValidate(schema, reordered); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, reordered, rowData, copyRowData, otherRowData); + } + + @Test + public void testBasicProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + generateAndValidate(schema, idOnly); + generateAndValidate(schema, dataOnly); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, dataOnly, rowData, copyRowData, otherRowData); + } + + @Test + public void testEmptyProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + generateAndValidate(schema, schema.select()); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, schema.select(), rowData, copyRowData, otherRowData, true); + } + + @Test + public void testRename() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + Schema renamed = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + generateAndValidate(schema, renamed); + + GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); + GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); + testEqualsAndHashCode(schema, renamed, rowData, copyRowData, otherRowData); + } + + @Test + public void testNestedProjection() { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + GenericRowData rowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); + GenericRowData copyRowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); + GenericRowData otherRowData = GenericRowData.of(2L, GenericRowData.of(2.0f, 2.0f)); + + GenericRowData rowDataNullStruct = GenericRowData.of(1L, null); + GenericRowData copyRowDataNullStruct = GenericRowData.of(1L, null); + GenericRowData otherRowDataNullStruct = GenericRowData.of(2L, null); + + // Project id only. + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, idOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct); + + // Project lat only. + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + assertThat(latOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, latOnly); + testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, latOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); + + // Project long only. + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + assertThat(longOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, longOnly); + testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, longOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); + + // Project location. + Schema locationOnly = schema.select("location"); + assertThat(locationOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, locationOnly); + testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode( + schema, + locationOnly, + rowDataNullStruct, + copyRowDataNullStruct, + otherRowDataNullStruct, + true); + } + + @Test + public void testPrimitivesFullProjection() { + DataGenerator dataGenerator = new DataGenerators.Primitives(); + Schema schema = dataGenerator.icebergSchema(); + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + GenericRowData otherRowData = dataGenerator.generateFlinkRowData(); + // modify the string field value (position 6) + otherRowData.setField(6, StringData.fromString("foo_bar")); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + setOptionalFieldsNullForPrimitives(rowDataNullOptionalFields); + GenericRowData copyRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + setOptionalFieldsNullForPrimitives(copyRowDataNullOptionalFields); + GenericRowData otherRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); + // modify the string field value (position 6) + otherRowDataNullOptionalFields.setField(6, StringData.fromString("foo_bar")); + setOptionalFieldsNullForPrimitives(otherRowData); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + private void setOptionalFieldsNullForPrimitives(GenericRowData rowData) { + // fields from [1, 5] range are optional + for (int pos = 1; pos <= 5; ++pos) { + rowData.setField(pos, null); + } + } + + @Test + public void testMapOfPrimitivesProjection() { + DataGenerator dataGenerator = new DataGenerators.MapOfPrimitives(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project map only. + Schema mapOnly = schema.select("map_of_primitives"); + assertThat(mapOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, mapOnly); + + // Project all. + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(StringData.fromString("foo"), 1, StringData.fromString("bar"), 2))); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData, true); + testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("row_id_value"), null); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("row_id_value"), null); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of(StringData.fromString("other_row_id_value"), null); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + mapOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields, + true); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testMapOfStructStructProjection() { + DataGenerator dataGenerator = new DataGenerators.MapOfStructStruct(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project map only. + Schema mapOnly = schema.select("map"); + assertThat(mapOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, mapOnly); + + // Project all. + generateAndValidate(schema, schema); + + // Project partial map key. + Schema partialMapKey = + new Schema( + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()), + Types.NestedField.required(204, "valueData", Types.StringType.get()))))); + assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial map key or value struct."); + + // Project partial map key. + Schema partialMapValue = + new Schema( + Types.NestedField.optional( + 2, + "map", + Types.MapType.ofOptional( + 101, + 102, + Types.StructType.of( + Types.NestedField.required(201, "key", Types.LongType.get()), + Types.NestedField.required(202, "keyData", Types.StringType.get())), + Types.StructType.of( + Types.NestedField.required(203, "value", Types.LongType.get()))))); + assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial map key or value struct."); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericMapData( + ImmutableMap.of( + GenericRowData.of(1L, StringData.fromString("other_key_data")), + GenericRowData.of(1L, StringData.fromString("other_value_data"))))); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericMapData( + ImmutableMap.of(GenericRowData.of(2L, null), GenericRowData.of(2L, null)))); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + mapOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testArrayOfPrimitiveProjection() { + DataGenerator dataGenerator = new DataGenerators.ArrayOfPrimitive(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project list only. + Schema arrayOnly = schema.select("array_of_int"); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, arrayOnly); + + // Project all. + generateAndValidate(schema, schema); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericArrayData(new Integer[] {4, 5, 6})); + testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, arrayOnly, rowData, copyRowData, otherRowData); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("other_row_id_value"), + new GenericArrayData(new Integer[] {4, null, 6})); + testEqualsAndHashCode( + schema, + idOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + arrayOnly, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + @Test + public void testArrayOfStructProjection() { + DataGenerator dataGenerator = new DataGenerators.ArrayOfStruct(); + Schema schema = dataGenerator.icebergSchema(); + + // Project id only. + Schema idOnly = schema.select("row_id"); + assertThat(idOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, idOnly); + + // Project list only. + Schema arrayOnly = schema.select("array_of_struct"); + assertThat(arrayOnly.columns().size()).isGreaterThan(0); + generateAndValidate(schema, arrayOnly); + + // Project all. + generateAndValidate(schema, schema); + + // Project partial list value. + Schema partialList = + new Schema( + Types.NestedField.optional( + 2, + "array_of_struct", + Types.ListType.ofOptional( + 101, + Types.StructType.of( + Types.NestedField.required(202, "name", Types.StringType.get()))))); + + assertThatThrownBy(() -> generateAndValidate(schema, partialList)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Cannot project a partial list element struct."); + + GenericRowData rowData = dataGenerator.generateFlinkRowData(); + GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); + // modify the map field value + GenericRowData otherRowData = + GenericRowData.of( + StringData.fromString("row_id_value"), new GenericArrayData(new Integer[] {4, 5, 6})); + testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); + + GenericRowData rowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + GenericRowData copyRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {1, null, 3})); + // modify the map field value + GenericRowData otherRowDataNullOptionalFields = + GenericRowData.of( + StringData.fromString("row_id_value"), + new GenericArrayData(new Integer[] {4, null, 6})); + testEqualsAndHashCode( + schema, + schema, + rowDataNullOptionalFields, + copyRowDataNullOptionalFields, + otherRowDataNullOptionalFields); + } + + private void generateAndValidate(Schema schema, Schema projectSchema) { + int numRecords = 100; + List recordList = RandomGenericData.generate(schema, numRecords, 102L); + List rowDataList = + Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); + assertThat(rowDataList).hasSize(recordList.size()); + + StructProjection structProjection = StructProjection.create(schema, projectSchema); + RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); + + for (int i = 0; i < numRecords; i++) { + StructLike expected = structProjection.wrap(recordList.get(i)); + RowData projected = rowDataProjection.wrap(rowDataList.get(i)); + TestHelpers.assertRowData(projectSchema, expected, projected); + + assertThat(projected).isEqualTo(projected); + assertThat(projected).hasSameHashCodeAs(projected); + // make sure toString doesn't throw NPE for null values + assertThatNoException().isThrownBy(projected::toString); + } + } + + private void testEqualsAndHashCode( + Schema schema, + Schema projectionSchema, + RowData rowData, + RowData copyRowData, + RowData otherRowData) { + testEqualsAndHashCode(schema, projectionSchema, rowData, copyRowData, otherRowData, false); + } + + /** + * @param isOtherRowDataSameAsRowData sometimes projection on otherRowData can result in the same + * RowData, e.g. due to empty projection or null struct + */ + private void testEqualsAndHashCode( + Schema schema, + Schema projectionSchema, + RowData rowData, + RowData copyRowData, + RowData otherRowData, + boolean isOtherRowDataSameAsRowData) { + RowDataProjection projection = RowDataProjection.create(schema, projectionSchema); + RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); + RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); + + assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(copyProjection.wrap(copyRowData)); + + if (isOtherRowDataSameAsRowData) { + assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)).hasSameHashCodeAs(otherProjection.wrap(otherRowData)); + } else { + assertThat(projection.wrap(rowData)).isNotEqualTo(otherProjection.wrap(otherRowData)); + assertThat(projection.wrap(rowData)) + .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); + } + } +} 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 new file mode 100644 index 000000000000..7dd4e8759c0e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java @@ -0,0 +1,596 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.withPrecision; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +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; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Files; +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.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestRowProjection { + + @TempDir private Path temp; + + private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) + throws IOException { + File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); + assertThat(file.delete()).isTrue(); + + try (FileAppender appender = + Avro.write(Files.localOutput(file)) + .schema(writeSchema) + .createWriterFunc(ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(writeSchema))) + .build()) { + appender.add(row); + } + + Iterable records = + Avro.read(Files.localInput(file)) + .project(readSchema) + .createReaderFunc(FlinkAvroReader::new) + .build(); + + return Iterables.getOnlyElement(records); + } + + @Test + public void testFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData projected = writeAndRead("full_projection", schema, schema, row); + + assertThat(projected.getLong(0)).isEqualTo(34); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + } + + @Test + public void testSpecialCharacterProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "user id", Types.LongType.get()), + Types.NestedField.optional(1, "data%0", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData full = writeAndRead("special_chars", schema, schema, row); + + assertThat(full.getLong(0)).isEqualTo(34L); + assertThat(full.getString(1)).asString().isEqualTo("test"); + + RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); + + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + } + + @Test + public void testReorderedFullProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema reordered = + new Schema( + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("full_projection", schema, reordered, row); + + assertThat(projected.getString(0)).asString().isEqualTo("test"); + assertThat(projected.getLong(1)).isEqualTo(34); + } + + @Test + public void testReorderedProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema reordered = + new Schema( + Types.NestedField.optional(2, "missing_1", Types.StringType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(3, "missing_2", Types.LongType.get())); + + RowData projected = writeAndRead("full_projection", schema, reordered, row); + + assertThat(projected.isNullAt(0)).isTrue(); + assertThat(projected.getString(1)).asString().isEqualTo("test"); + assertThat(projected.isNullAt(2)).isTrue(); + } + + @Test + public void testRenamedAddedField() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(1, "a", Types.LongType.get()), + Types.NestedField.required(2, "b", Types.LongType.get()), + Types.NestedField.required(3, "d", Types.LongType.get())); + + RowData row = GenericRowData.of(100L, 200L, 300L); + + Schema renamedAdded = + new Schema( + Types.NestedField.optional(1, "a", Types.LongType.get()), + Types.NestedField.optional(2, "b", Types.LongType.get()), + Types.NestedField.optional(3, "c", Types.LongType.get()), + Types.NestedField.optional(4, "d", Types.LongType.get())); + + RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.getLong(1)) + .as("Should contain the correct value in column 2") + .isEqualTo(200L); + assertThat(projected.getLong(2)) + .as("Should contain the correct value in column 1") + .isEqualTo(300L); + assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); + } + + @Test + public void testEmptyProjection() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); + + assertThat(projected).isNotNull(); + assertThat(projected.getArity()).isEqualTo(0); + } + + @Test + public void testBasicProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); + assertThat(projected.getArity()).as("Should not project data").isEqualTo(1); + assertThat(projected.getLong(0)).isEqualTo(34L); + + Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); + + projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); + + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + int cmp = Comparators.charSequences().compare("test", projected.getString(0).toString()); + assertThat(projected.getString(0)).asString().isEqualTo("test"); + } + + @Test + public void testRename() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get())); + + RowData row = GenericRowData.of(34L, StringData.fromString("test")); + + Schema readSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "renamed", Types.StringType.get())); + + RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); + + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getString(1)) + .as("Should contain the correct data/renamed value") + .asString() + .isEqualTo("test"); + } + + @Test + public void testNestedStructProjection() throws Exception { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 3, + "location", + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get())))); + + RowData location = GenericRowData.of(52.995143f, -1.539054f); + RowData record = GenericRowData.of(34L, location); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); + assertThat(projected.getArity()).isEqualTo(1); + assertThat(projected.getLong(0)).as("Should contain the correct id value").isEqualTo(34L); + + Schema latOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); + + projected = writeAndRead("latitude_only", writeSchema, latOnly, record); + RowData projectedLocation = projected.getRow(0, 1); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project longitude").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + + Schema longOnly = + new Schema( + Types.NestedField.optional( + 3, + "location", + Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); + + projected = writeAndRead("longitude_only", writeSchema, longOnly, record); + projectedLocation = projected.getRow(0, 1); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getArity()).as("Should not project latitutde").isEqualTo(1); + assertThat(projectedLocation.getFloat(0)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); + + Schema locationOnly = writeSchema.select("location"); + projected = writeAndRead("location_only", writeSchema, locationOnly, record); + projectedLocation = projected.getRow(0, 1); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); + assertThat(projectedLocation.getFloat(0)) + .as("Should project latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedLocation.getFloat(1)) + .as("Should project longitude") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); + } + + @Test + public void testMapProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "properties", + Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); + + GenericMapData properties = + new GenericMapData( + ImmutableMap.of( + StringData.fromString("a"), + StringData.fromString("A"), + StringData.fromString("b"), + StringData.fromString("B"))); + + RowData row = GenericRowData.of(34L, properties); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project properties map").isEqualTo(1); + + Schema keyOnly = writeSchema.select("properties.key"); + projected = writeAndRead("key_only", writeSchema, keyOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); + + Schema valueOnly = writeSchema.select("properties.value"); + projected = writeAndRead("value_only", writeSchema, valueOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(properties); + + Schema mapOnly = writeSchema.select("properties"); + projected = writeAndRead("map_only", writeSchema, mapOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + 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; + } + + @Test + public void testMapOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "lat", Types.FloatType.get()), + Types.NestedField.required(2, "long", Types.FloatType.get()))))); + + RowData l1 = GenericRowData.of(53.992811f, -1.542616f); + RowData l2 = GenericRowData.of(52.995143f, -1.539054f); + GenericMapData map = + new GenericMapData( + ImmutableMap.of(StringData.fromString("L1"), l1, StringData.fromString("L2"), l2)); + RowData row = GenericRowData.of(34L, map); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project locations map").isEqualTo(1); + + projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getMap(0)).isEqualTo(row.getMap(1)); + + projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); + GenericMapData locations = (GenericMapData) projected.getMap(0); + assertThat(locations).isNotNull(); + GenericArrayData l1l2Array = + new GenericArrayData( + new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); + RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain lat") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + assertThat(projectedL1.getArity()).as("L1 should not contain long").isEqualTo(1); + RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain lat") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + assertThat(projectedL2.getArity()).as("L2 should not contain long").isEqualTo(1); + + projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + locations = (GenericMapData) projected.getMap(0); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); + projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getArity()).as("L1 should not contain lat").isEqualTo(1); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain long") + .isEqualTo(-1.542616f, withPrecision(0.000001f)); + projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getArity()).as("L2 should not contain lat").isEqualTo(1); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain long") + .isEqualTo(-1.539054f, withPrecision(0.000001f)); + + Schema latitiudeRenamed = + new Schema( + Types.NestedField.optional( + 5, + "locations", + Types.MapType.ofOptional( + 6, + 7, + Types.StringType.get(), + Types.StructType.of( + Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); + + projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + locations = (GenericMapData) projected.getMap(0); + assertThat(locations).isNotNull(); + assertThat(locations.keyArray()).isEqualTo(l1l2Array); + projectedL1 = (RowData) locations.get(StringData.fromString("L1")); + assertThat(projectedL1).isNotNull(); + assertThat(projectedL1.getFloat(0)) + .as("L1 should contain latitude") + .isEqualTo(53.992811f, withPrecision(0.000001f)); + projectedL2 = (RowData) locations.get(StringData.fromString("L2")); + assertThat(projectedL2).isNotNull(); + assertThat(projectedL2.getFloat(0)) + .as("L2 should contain latitude") + .isEqualTo(52.995143f, withPrecision(0.000001f)); + } + + @Test + public void testListProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); + + GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L}); + + RowData row = GenericRowData.of(34L, values); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).as("Should not project values list").isEqualTo(1); + + Schema elementOnly = writeSchema.select("values.element"); + projected = writeAndRead("element_only", writeSchema, elementOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); + + Schema listOnly = writeSchema.select("values"); + projected = writeAndRead("list_only", writeSchema, listOnly, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(values); + } + + @Test + @SuppressWarnings("unchecked") + public void testListOfStructsProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.required(19, "x", Types.IntegerType.get()), + Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); + + RowData p1 = GenericRowData.of(1, 2); + RowData p2 = GenericRowData.of(3, null); + GenericArrayData arrayData = new GenericArrayData(new RowData[] {p1, p2}); + RowData row = GenericRowData.of(34L, arrayData); + + Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); + + RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); + assertThat(projected.getLong(0)).isEqualTo(34L); + assertThat(projected.getArity()).isEqualTo(1); + + projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.getArray(0)).isEqualTo(row.getArray(1)); + + projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); + ArrayData points = projected.getArray(0); + assertThat(points.size()).isEqualTo(2); + RowData projectedP1 = points.getRow(0, 2); + assertThat(projectedP1.getInt(0)).as("Should project x").isEqualTo(1); + assertThat(projectedP1.getArity()).as("Should not project y").isEqualTo(1); + RowData projectedP2 = points.getRow(1, 2); + assertThat(projectedP2.getArity()).as("Should not project y").isEqualTo(1); + assertThat(projectedP2.getInt(0)).as("Should project x").isEqualTo(3); + + projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); + points = projected.getArray(0); + assertThat(points.size()).isEqualTo(2); + projectedP1 = points.getRow(0, 2); + assertThat(projectedP1.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project y").isEqualTo(2); + projectedP2 = points.getRow(1, 2); + assertThat(projectedP2.getArity()).as("Should not project x").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null y").isTrue(); + + Schema yRenamed = + new Schema( + Types.NestedField.optional( + 22, + "points", + Types.ListType.ofOptional( + 21, + Types.StructType.of( + Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); + + projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); + assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); + assertThat(projected.isNullAt(0)).isFalse(); + points = projected.getArray(0); + assertThat(points.size()).isEqualTo(2); + projectedP1 = points.getRow(0, 2); + assertThat(projectedP1.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP1.getInt(0)).as("Should project z").isEqualTo(2); + projectedP2 = points.getRow(1, 2); + assertThat(projectedP2.getArity()).as("Should not project x and y").isEqualTo(1); + assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); + } + + @Test + public void testAddedFieldsWithRequiredChildren() throws Exception { + Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); + + RowData row = GenericRowData.of(100L); + + Schema addedFields = + new Schema( + Types.NestedField.optional(1, "a", Types.LongType.get()), + Types.NestedField.optional( + 2, + "b", + Types.StructType.of(Types.NestedField.required(3, "c", Types.LongType.get()))), + Types.NestedField.optional(4, "d", Types.ListType.ofRequired(5, Types.LongType.get())), + Types.NestedField.optional( + 6, + "e", + Types.MapType.ofRequired(7, 8, Types.LongType.get(), Types.LongType.get()))); + + RowData projected = + writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); + assertThat(projected.getLong(0)) + .as("Should contain the correct value in column 1") + .isEqualTo(100L); + assertThat(projected.isNullAt(1)).as("Should contain empty value in new column 2").isTrue(); + assertThat(projected.isNullAt(2)).as("Should contain empty value in new column 4").isTrue(); + assertThat(projected.isNullAt(3)).as("Should contain empty value in new column 6").isTrue(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java new file mode 100644 index 000000000000..eccab20e04fc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java @@ -0,0 +1,100 @@ +/* + * 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 org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.TestHelpers; +import org.junit.jupiter.api.Test; + +public class TestStructRowData { + + protected void testConverter(DataGenerator dataGenerator) { + StructRowData converter = new StructRowData(dataGenerator.icebergSchema().asStruct()); + GenericRecord expected = dataGenerator.generateIcebergGenericRecord(); + StructRowData actual = converter.setStruct(expected); + TestHelpers.assertRowData(dataGenerator.icebergSchema(), expected, actual); + } + + @Test + public void testPrimitiveTypes() { + testConverter(new DataGenerators.Primitives()); + } + + @Test + public void testStructOfPrimitive() { + testConverter(new DataGenerators.StructOfPrimitive()); + } + + @Test + public void testStructOfArray() { + testConverter(new DataGenerators.StructOfArray()); + } + + @Test + public void testStructOfMap() { + testConverter(new DataGenerators.StructOfMap()); + } + + @Test + public void testStructOfStruct() { + testConverter(new DataGenerators.StructOfStruct()); + } + + @Test + public void testArrayOfPrimitive() { + testConverter(new DataGenerators.ArrayOfPrimitive()); + } + + @Test + public void testArrayOfArray() { + testConverter(new DataGenerators.ArrayOfArray()); + } + + @Test + public void testArrayOfMap() { + testConverter(new DataGenerators.ArrayOfMap()); + } + + @Test + public void testArrayOfStruct() { + testConverter(new DataGenerators.ArrayOfStruct()); + } + + @Test + public void testMapOfPrimitives() { + testConverter(new DataGenerators.MapOfPrimitives()); + } + + @Test + public void testMapOfArray() { + testConverter(new DataGenerators.MapOfArray()); + } + + @Test + public void testMapOfMap() { + testConverter(new DataGenerators.MapOfMap()); + } + + @Test + public void testMapOfStruct() { + testConverter(new DataGenerators.MapOfStruct()); + } +} 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 new file mode 100644 index 000000000000..9b6580fad0bf --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/CollectingSink.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.maintenance.operator; + +import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +/** Sink for collecting output during testing. */ +class CollectingSink implements Sink { + private static final long serialVersionUID = 1L; + private static final List> QUEUES = + Collections.synchronizedList(Lists.newArrayListWithExpectedSize(1)); + private static final AtomicInteger NUM_SINKS = new AtomicInteger(-1); + private final int index; + + /** Creates a new sink which collects the elements received. */ + CollectingSink() { + this.index = NUM_SINKS.incrementAndGet(); + QUEUES.add(new LinkedBlockingQueue<>()); + } + + /** + * Gets all the remaining output received by this {@link Sink}. + * + * @return all the remaining output + */ + List remainingOutput() { + return Lists.newArrayList((BlockingQueue) QUEUES.get(this.index)); + } + + /** + * Check if there is no remaining output received by this {@link Sink}. + * + * @return true if there is no remaining output + */ + boolean isEmpty() { + return QUEUES.get(this.index).isEmpty(); + } + + /** + * Wait until the next element received by the {@link Sink}. + * + * @param timeout for the poll + * @return The first element received by this {@link Sink} + * @throws TimeoutException if no element received until the timeout + */ + T poll(Duration timeout) throws TimeoutException { + Object element; + + try { + element = QUEUES.get(this.index).poll(timeout.toMillis(), TimeUnit.MILLISECONDS); + } catch (InterruptedException var4) { + throw new RuntimeException(var4); + } + + if (element == null) { + throw new TimeoutException(); + } else { + return (T) element; + } + } + + @Override + public SinkWriter createWriter(InitContext context) { + return new CollectingWriter<>(index); + } + + private static class CollectingWriter implements SinkWriter { + private final int index; + + CollectingWriter(int index) { + this.index = index; + } + + @Override + public void write(T element, Context context) { + QUEUES.get(index).add(element); + } + + @Override + public void flush(boolean endOfInput) { + // Nothing to do here + } + + @Override + public void close() { + // Nothing to do here + } + } +} 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 new file mode 100644 index 000000000000..91d36aa3e85d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkSqlExtension.java @@ -0,0 +1,135 @@ +/* + * 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 new file mode 100644 index 000000000000..9cdc55cb0cce --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/FlinkStreamingTestUtils.java @@ -0,0 +1,73 @@ +/* + * 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 new file mode 100644 index 000000000000..38bb9c393fa9 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ManualSource.java @@ -0,0 +1,316 @@ +/* + * 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.ArrayDeque; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.source.Boundedness; +import org.apache.flink.api.connector.source.ReaderOutput; +import org.apache.flink.api.connector.source.Source; +import org.apache.flink.api.connector.source.SourceReader; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.api.connector.source.SourceSplit; +import org.apache.flink.api.connector.source.SplitEnumerator; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.api.java.tuple.Tuple2; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.core.io.InputStatus; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Queues; +import org.jetbrains.annotations.Nullable; + +/** Testing source implementation for Flink sources which can be triggered manually. */ +class ManualSource + implements Source, + ResultTypeQueryable { + + private static final long serialVersionUID = 1L; + private static final List>> QUEUES = + Collections.synchronizedList(Lists.newArrayList()); + private static final List> AVAILABILITIES = + Collections.synchronizedList(Lists.newArrayList()); + private static int numSources = 0; + private final TypeInformation type; + private final int index; + private transient DataStream stream; + private final transient StreamExecutionEnvironment env; + + /** + * Creates a new source for testing. + * + * @param env to register the source + * @param type of the events returned by the source + */ + ManualSource(StreamExecutionEnvironment env, TypeInformation type) { + this.type = type; + this.env = env; + this.index = numSources++; + QUEUES.add(Queues.newArrayDeque()); + AVAILABILITIES.add(new CompletableFuture<>()); + } + + /** + * Emit a new record from the source. + * + * @param event to emit + */ + void sendRecord(T event) { + this.sendInternal(Tuple2.of(event, null)); + } + + /** + * Emit a new record with the given event time from the source. + * + * @param event to emit + * @param eventTime of the event + */ + void sendRecord(T event, long eventTime) { + this.sendInternal(Tuple2.of(event, eventTime)); + } + + /** + * Emit a watermark from the source. + * + * @param timeStamp of the watermark + */ + void sendWatermark(long timeStamp) { + this.sendInternal(Tuple2.of(null, timeStamp)); + } + + /** Mark the source as finished. */ + void markFinished() { + this.sendWatermark(Long.MAX_VALUE); + this.sendInternal(Tuple2.of(null, null)); + } + + /** + * Get the {@link DataStream} for this source. + * + * @return the stream emitted by this source + */ + DataStream dataStream() { + if (this.stream == null) { + this.stream = + this.env + .fromSource(this, WatermarkStrategy.noWatermarks(), "ManualSource-" + index, type) + .forceNonParallel(); + } + + return this.stream; + } + + private void sendInternal(Tuple2 tuple) { + QUEUES.get(index).offer(tuple); + AVAILABILITIES.get(index).complete(null); + } + + @Override + public Boundedness getBoundedness() { + return Boundedness.CONTINUOUS_UNBOUNDED; + } + + @Override + public SplitEnumerator createEnumerator( + SplitEnumeratorContext enumContext) { + return new DummyCheckpointEnumerator(); + } + + @Override + public SplitEnumerator restoreEnumerator( + SplitEnumeratorContext enumContext, DummyCheckpoint checkpoint) { + return new DummyCheckpointEnumerator(); + } + + @Override + public SimpleVersionedSerializer getSplitSerializer() { + return new NoOpDummySplitSerializer(); + } + + @Override + public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { + return new NoOpDummyCheckpointSerializer(); + } + + @Override + public SourceReader createReader(SourceReaderContext sourceReaderContext) { + return new SourceReader() { + @Override + public void start() { + // Do nothing + } + + @Override + public InputStatus pollNext(ReaderOutput output) { + Tuple2 next = (Tuple2) QUEUES.get(index).poll(); + + if (next != null) { + if (next.f0 == null) { + // No more input + return InputStatus.END_OF_INPUT; + } + + if (next.f1 == null) { + // No event time set + output.collect(next.f0); + } else { + // With event time + output.collect(next.f0, next.f1); + } + } + + AVAILABILITIES.set(index, new CompletableFuture<>()); + return QUEUES.get(index).isEmpty() + ? InputStatus.NOTHING_AVAILABLE + : InputStatus.MORE_AVAILABLE; + } + + @Override + public List snapshotState(long checkpointId) { + return Lists.newArrayList(new DummySplit()); + } + + @Override + public CompletableFuture isAvailable() { + return AVAILABILITIES.get(index); + } + + @Override + public void addSplits(List splits) { + // do nothing + } + + @Override + public void notifyNoMoreSplits() { + // do nothing + } + + @Override + public void close() { + // do nothing + } + }; + } + + @Override + public TypeInformation getProducedType() { + return this.type; + } + + /** + * Placeholder because the ManualSource itself implicitly represents the only split and does not + * require an actual split object. + */ + public static class DummySplit implements SourceSplit { + @Override + public String splitId() { + return "dummy"; + } + } + + /** + * Placeholder because the ManualSource does not support fault-tolerance and thus does not require + * actual checkpointing. + */ + public static class DummyCheckpoint {} + + /** Placeholder because the ManualSource does not need enumeration, but checkpointing needs it. */ + private static class DummyCheckpointEnumerator + implements SplitEnumerator { + + @Override + public void start() { + // do nothing + } + + @Override + public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { + // do nothing + } + + @Override + public void addSplitsBack(List splits, int subtaskId) { + // do nothing + } + + @Override + public void addReader(int subtaskId) { + // do nothing + } + + @Override + public DummyCheckpoint snapshotState(long checkpointId) { + return new DummyCheckpoint(); + } + + @Override + public void close() { + // do nothing + } + } + + /** + * Not used - only required to avoid NullPointerException. The split is not transferred from the + * enumerator, it is implicitly represented by the ManualSource. + */ + private static class NoOpDummySplitSerializer implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(DummySplit split) { + return new byte[0]; + } + + @Override + public DummySplit deserialize(int version, byte[] serialized) { + return new DummySplit(); + } + } + + /** + * Not used - only required to avoid NullPointerException. The split is not transferred from the + * enumerator, it is implicitly represented by the ManualSource. + */ + private static class NoOpDummyCheckpointSerializer + implements SimpleVersionedSerializer { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(DummyCheckpoint split) { + return new byte[0]; + } + + @Override + public DummyCheckpoint deserialize(int version, byte[] serialized) { + return new DummyCheckpoint(); + } + } +} 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 new file mode 100644 index 000000000000..272e0b693fd3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/OperatorTestBase.java @@ -0,0 +1,51 @@ +/* + * 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.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; + +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.flink.FlinkCatalogFactory; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.extension.RegisterExtension; + +class OperatorTestBase { + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + static final String TABLE_NAME = "test_table"; + + @RegisterExtension + protected static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .build()); + + @RegisterExtension + final FlinkSqlExtension sql = + new FlinkSqlExtension( + "catalog", + ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), + "db"); +} 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 new file mode 100644 index 000000000000..876d64214560 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestMonitorSource.java @@ -0,0 +1,362 @@ +/* + * 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.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.client.JobExecutionException; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.RewriteFiles; +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.flink.TableLoader; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +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; + +class TestMonitorSource extends OperatorTestBase { + private static final TableChange EMPTY_EVENT = TableChange.empty(); + private static final RateLimiterStrategy HIGH_RATE = RateLimiterStrategy.perSecond(100.0); + private static final RateLimiterStrategy LOW_RATE = RateLimiterStrategy.perSecond(1.0 / 10000.0); + + @TempDir private File checkpointDir; + + @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(); + + MonitorSource.TableChangeIterator iterator = + 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(); + 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(); + expected = tableChangeWithLastSnapshot(table, TableChange.empty()); + + sql.exec("INSERT INTO %s VALUES (3, 'c')", TABLE_NAME); + table.refresh(); + expected = tableChangeWithLastSnapshot(table, expected); + + assertThat(iterator.next()).isEqualTo(expected); + // Make sure that consecutive calls do not return the data again + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + /** + * Create a table and check that the source returns the data as new commits arrive to the table. + */ + @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(); + DataStream events = + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink result = new CollectingSink<>(); + events.sinkTo(result); + + JobClient jobClient = null; + try { + // First result is an empty event + jobClient = env.executeAsync("Table Change Source Test"); + assertThat(result.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + + // Insert some data + File dataDir = new File(new Path(table.location(), "data").toUri().getPath()); + dataDir.mkdir(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(table, FileFormat.PARQUET, dataDir.toPath()); + List batch1 = RandomGenericData.generate(table.schema(), 2, 1); + dataAppender.appendToTable(batch1); + + // Wait until the changes are committed + Awaitility.await() + .until( + () -> { + table.refresh(); + return table.currentSnapshot() != null; + }); + + table.refresh(); + long size = firstFileLength(table); + + // Wait until the first non-empty event has arrived, and check the expected result + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + }); + } finally { + closeJobClient(jobClient); + } + } + + /** 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); + + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + 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), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + + // Sink to collect the results + CollectingSink result = new CollectingSink<>(); + events.sinkTo(result); + + // Start the job + Configuration conf; + JobClient jobClient = null; + AtomicReference firstNonEmptyEvent = new AtomicReference<>(); + try { + jobClient = env.executeAsync("Table Change Source Test"); + + Awaitility.await() + .until( + () -> { + TableChange newEvent = result.poll(Duration.ofSeconds(5L)); + // Fetch every empty event from the beginning + while (newEvent.equals(EMPTY_EVENT)) { + newEvent = result.poll(Duration.ofSeconds(5L)); + } + + // The first non-empty event should contain the expected value + firstNonEmptyEvent.set(newEvent); + return true; + }); + } finally { + // Stop with savepoint + conf = closeJobClient(jobClient, savepointDir); + } + + // Restore from savepoint, create the same topology with a different env + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink resultWithSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithSavepoint); + + // Make sure that the job with restored source does not read new records from the table + JobClient clientWithSavepoint = null; + try { + clientWithSavepoint = env.executeAsync("Table Change Source test with savepoint"); + + assertThat(resultWithSavepoint.poll(Duration.ofSeconds(5L))).isEqualTo(EMPTY_EVENT); + } finally { + closeJobClient(clientWithSavepoint, null); + } + + // Restore without savepoint + env = StreamExecutionEnvironment.getExecutionEnvironment(); + events = + env.fromSource( + new MonitorSource(tableLoader, LOW_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .forceNonParallel(); + CollectingSink resultWithoutSavepoint = new CollectingSink<>(); + events.sinkTo(resultWithoutSavepoint); + + // Make sure that a new job without state reads the event as expected + JobClient clientWithoutSavepoint = null; + try { + clientWithoutSavepoint = env.executeAsync("Table Change Source Test without savepoint"); + assertThat(resultWithoutSavepoint.poll(Duration.ofSeconds(5L))) + .isEqualTo(firstNonEmptyEvent.get()); + } finally { + closeJobClient(clientWithoutSavepoint); + } + } + + @Test + void testNotOneParallelismThrows() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + env.fromSource( + new MonitorSource(tableLoader, HIGH_RATE, Long.MAX_VALUE), + WatermarkStrategy.noWatermarks(), + "TableChangeSource") + .setParallelism(2) + .print(); + + assertThatThrownBy(env::execute) + .isInstanceOf(JobExecutionException.class) + .rootCause() + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Parallelism should be set to 1"); + } + + @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); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + MonitorSource.TableChangeIterator iterator = + new MonitorSource.TableChangeIterator(tableLoader, null, 1); + + // For a single maxReadBack we only get a single change + assertThat(iterator.next().commitNum()).isEqualTo(1); + + iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); + + // Expecting 2 commits/snapshots for maxReadBack=2 + assertThat(iterator.next().commitNum()).isEqualTo(2); + + iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + + // For maxReadBack Long.MAX_VALUE we get every change + assertThat(iterator.next().commitNum()).isEqualTo(3); + } + + @Test + void testSkipReplace() { + sql.exec("CREATE TABLE %s (id int, data varchar)", TABLE_NAME); + sql.exec("INSERT INTO %s VALUES (1, 'a')", TABLE_NAME); + + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + tableLoader.open(); + + MonitorSource.TableChangeIterator iterator = + new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); + + // Read the current snapshot + assertThat(iterator.next().commitNum()).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(); + // Replace the file with itself for testing purposes + rewrite.deleteFile(dataFile); + rewrite.addFile(dataFile); + rewrite.commit(); + + // Check that the rewrite is ignored + assertThat(iterator.next()).isEqualTo(EMPTY_EVENT); + } + + private static long firstFileLength(Table table) { + return table.currentSnapshot().addedDataFiles(table.io()).iterator().next().fileSizeInBytes(); + } + + private static TableChange tableChangeWithLastSnapshot(Table table, TableChange previous) { + List dataFiles = + Lists.newArrayList(table.currentSnapshot().addedDataFiles(table.io()).iterator()); + List deleteFiles = + Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); + + long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); + long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); + boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); + + return new TableChange( + previous.dataFileNum() + dataFiles.size(), + previous.deleteFileNum() + deleteFiles.size(), + previous.dataFileSize() + dataSize, + previous.deleteFileSize() + deleteSize, + previous.commitNum() + 1); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java new file mode 100644 index 000000000000..44eb907a17aa --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.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.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.flink.AvroGenericRecordConverterBase; +import org.apache.iceberg.flink.DataGenerator; + +public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { + @Override + protected void testConverter(DataGenerator dataGenerator) throws Exception { + // Need to use avroSchema from DataGenerator because some primitive types have special Avro + // type handling. Hence the Avro schema converted from Iceberg schema won't work. + AvroGenericRecordToRowDataMapper mapper = + AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); + RowData expected = dataGenerator.generateFlinkRowData(); + RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java new file mode 100644 index 000000000000..abac605f81fd --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.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.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionKeySelector { + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testCorrectKeySelection(TableSchemaType tableSchemaType) { + int numBuckets = 60; + + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitionKeySelector keySelector = + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); + + TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) + .forEach( + rowData -> { + int expectedBucketId = + TestBucketPartitionerUtil.computeBucketId( + numBuckets, rowData.getString(1).toString()); + Integer key = keySelector.getKey(rowData); + assertThat(key).isEqualTo(expectedBucketId); + }); + } + + @Test + public void testKeySelectorMultipleBucketsFail() { + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); + + assertThatExceptionOfType(RuntimeException.class) + .isThrownBy( + () -> + new BucketPartitionKeySelector( + partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java new file mode 100644 index 000000000000..59bdba578ebb --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java @@ -0,0 +1,108 @@ +/* + * 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 static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; +import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatExceptionOfType; + +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.CsvSource; + +public class TestBucketPartitioner { + + static final int DEFAULT_NUM_BUCKETS = 60; + + @ParameterizedTest + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) + public void testPartitioningParallelismGreaterThanBuckets( + String schemaTypeStr, String numBucketsStr) { + int numPartitions = 500; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + int bucketId = 0; + for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + assertThat(actualPartitionIndex).isEqualTo(expectedIndex); + bucketId++; + if (bucketId == numBuckets) { + bucketId = 0; + } + } + } + + @ParameterizedTest + @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) + public void testPartitioningParallelismEqualLessThanBuckets( + String schemaTypeStr, String numBucketsStr) { + int numPartitions = 30; + TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); + int numBuckets = Integer.parseInt(numBucketsStr); + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); + assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); + } + } + + @Test + public void testPartitionerBucketIdNullFail() { + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) + .withMessage(BUCKET_NULL_MESSAGE); + } + + @Test + public void testPartitionerMultipleBucketsFail() { + PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); + + assertThatExceptionOfType(RuntimeException.class) + .isThrownBy(() -> new BucketPartitioner(partitionSpec)) + .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); + } + + @Test + public void testPartitionerBucketIdOutOfRangeFail() { + PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); + BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); + + int negativeBucketId = -1; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) + .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); + + int tooBigBucketId = DEFAULT_NUM_BUCKETS; + assertThatExceptionOfType(IllegalArgumentException.class) + .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) + .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); + } +} 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 new file mode 100644 index 000000000000..ba0ea867ffb7 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.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.sink; + +import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; +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 org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +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.sink.TestBucketPartitionerUtil.TableSchemaType; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.CloseableIterable; +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.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestBucketPartitionerFlinkIcebergSink { + + private static final int NUMBER_TASK_MANAGERS = 1; + private static final int SLOTS_PER_TASK_MANAGER = 8; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUMBER_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + private static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + // Parallelism = 8 (parallelism > numBuckets) throughout the test suite + private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; + private final FileFormat format = FileFormat.PARQUET; + private final int numBuckets = 4; + + private Table table; + private StreamExecutionEnvironment env; + private TableLoader tableLoader; + + private void setupEnvironment(TableSchemaType tableSchemaType) { + PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitionSpec, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + env = + StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism * 2); + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + private void appendRowsToTable(List allRows) throws Exception { + DataFormatConverters.RowConverter converter = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + DataStream dataStream = + env.addSource( + new BoundedTestSource<>( + allRows.stream().map(converter::toExternal).toArray(Row[]::new)), + ROW_TYPE_INFO) + .map(converter::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)) + .partitionCustom( + new BucketPartitioner(table.spec()), + new BucketPartitionKeySelector( + table.spec(), + table.schema(), + FlinkSink.toFlinkRowType(table.schema(), SimpleDataUtil.FLINK_SCHEMA))); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.NONE) + .append(); + + env.execute("Test Iceberg DataStream"); + + SimpleDataUtil.assertTableRows(table, allRows); + } + + @ParameterizedTest + @EnumSource( + value = TableSchemaType.class, + names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) + public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { + setupEnvironment(tableSchemaType); + List rows = generateTestDataRows(); + + appendRowsToTable(rows); + TableTestStats stats = extractPartitionResults(tableSchemaType); + + assertThat(stats.totalRowCount).isEqualTo(rows.size()); + // All 4 buckets should've been written to + assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); + assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); + // Writer expectation (2 writers per bucket): + // - Bucket0 -> Writers [0, 4] + // - Bucket1 -> Writers [1, 5] + // - Bucket2 -> Writers [2, 6] + // - Bucket3 -> Writers [3, 7] + for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { + assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); + // 2 files per bucket (one file is created by each writer) + assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); + // 2 rows per file (total of 16 rows across 8 files) + assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); + } + } + + /** + * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 + * buckets) + */ + private List generateTestDataRows() { + int totalNumRows = parallelism * 2; + int numRowsPerBucket = totalNumRows / numBuckets; + return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); + } + + private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) + throws IOException { + int totalRecordCount = 0; + Map> writersPerBucket = Maps.newHashMap(); // > + Map filesPerBucket = Maps.newHashMap(); // + Map rowsPerWriter = Maps.newHashMap(); // + + try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { + for (FileScanTask scanTask : fileScanTasks) { + long recordCountInFile = scanTask.file().recordCount(); + + String[] splitFilePath = scanTask.file().path().toString().split("/"); + // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet + // Writer ID: .......^^^^^ + String filename = splitFilePath[splitFilePath.length - 1]; + int writerId = Integer.parseInt(filename.split("-")[0]); + + totalRecordCount += recordCountInFile; + int bucketId = + scanTask + .file() + .partition() + .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); + writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); + writersPerBucket.get(bucketId).add(writerId); + filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); + rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); + } + } + + return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); + } + + /** DTO to hold Test Stats */ + private static class TableTestStats { + final int totalRowCount; + final Map> writersPerBucket; + final Map numFilesPerBucket; + final Map rowsPerWriter; + + TableTestStats( + int totalRecordCount, + Map> writersPerBucket, + Map numFilesPerBucket, + Map rowsPerWriter) { + this.totalRowCount = totalRecordCount; + this.writersPerBucket = writersPerBucket; + this.numFilesPerBucket = numFilesPerBucket; + this.rowsPerWriter = rowsPerWriter; + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java new file mode 100644 index 000000000000..e1309bfac6d5 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.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.flink.sink; + +import java.util.List; +import java.util.UUID; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.BucketUtil; + +final class TestBucketPartitionerUtil { + + enum TableSchemaType { + ONE_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 0; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); + } + }, + IDENTITY_AND_BUCKET { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .identity("id") + .bucket("data", numBuckets) + .build(); + } + }, + TWO_BUCKETS { + @Override + public int bucketPartitionColumnPosition() { + return 1; + } + + @Override + public PartitionSpec getPartitionSpec(int numBuckets) { + return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) + .bucket("id", numBuckets) + .bucket("data", numBuckets) + .build(); + } + }; + + public abstract int bucketPartitionColumnPosition(); + + public abstract PartitionSpec getPartitionSpec(int numBuckets); + } + + private TestBucketPartitionerUtil() {} + + /** + * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to + * numBuckets - 1) + * + * @param numRowsPerBucket how many different rows should be generated per bucket + * @param numBuckets max number of buckets to consider + * @return the list of rows whose data "hashes" to the desired bucketId + */ + static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { + List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); + // For some of our tests, this order of the generated rows matters + for (int i = 0; i < numRowsPerBucket; i++) { + for (int bucketId = 0; bucketId < numBuckets; bucketId++) { + String value = generateValueForBucketId(bucketId, numBuckets); + rows.add(GenericRowData.of(1, StringData.fromString(value))); + } + } + return rows; + } + + /** + * Utility method to generate a UUID string that will "hash" to a desired bucketId + * + * @param bucketId the desired bucketId + * @return the string data that "hashes" to the desired bucketId + */ + private static String generateValueForBucketId(int bucketId, int numBuckets) { + while (true) { + String uuid = UUID.randomUUID().toString(); + if (computeBucketId(numBuckets, uuid) == bucketId) { + return uuid; + } + } + } + + /** + * Utility that performs the same hashing/bucketing mechanism used by Bucket.java + * + * @param numBuckets max number of buckets to consider + * @param value the string to compute the bucketId from + * @return the computed bucketId + */ + static int computeBucketId(int numBuckets, String value) { + return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java new file mode 100644 index 000000000000..360db658cd2f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java @@ -0,0 +1,81 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import java.time.Duration; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; + +public class TestCachingTableSupplier { + + @Test + public void testCheckArguments() { + SerializableTable initialTable = mock(SerializableTable.class); + + Table loadedTable = mock(Table.class); + TableLoader tableLoader = mock(TableLoader.class); + when(tableLoader.loadTable()).thenReturn(loadedTable); + + new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); + + assertThatThrownBy(() -> new CachingTableSupplier(initialTable, tableLoader, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("tableRefreshInterval cannot be null"); + assertThatThrownBy(() -> new CachingTableSupplier(null, tableLoader, Duration.ofMillis(100))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("initialTable cannot be null"); + assertThatThrownBy(() -> new CachingTableSupplier(initialTable, null, Duration.ofMillis(100))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("tableLoader cannot be null"); + } + + @Test + public void testTableReload() { + SerializableTable initialTable = mock(SerializableTable.class); + + Table loadedTable = mock(Table.class); + TableLoader tableLoader = mock(TableLoader.class); + when(tableLoader.loadTable()).thenReturn(loadedTable); + + CachingTableSupplier cachingTableSupplier = + new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); + + // refresh shouldn't do anything as the min reload interval hasn't passed + cachingTableSupplier.refreshTable(); + assertThat(cachingTableSupplier.get()).isEqualTo(initialTable); + + // refresh after waiting past the min reload interval + Awaitility.await() + .atLeast(100, TimeUnit.MILLISECONDS) + .untilAsserted( + () -> { + cachingTableSupplier.refreshTable(); + assertThat(cachingTableSupplier.get()).isEqualTo(loadedTable); + }); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java new file mode 100644 index 000000000000..8faae1b05a4e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -0,0 +1,257 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Map; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.common.DynFields; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.BaseTaskWriter; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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 TestCompressionSettings { + @TempDir protected Path temporaryFolder; + + private Table table; + + @Parameter(index = 0) + private Map initProperties; + + @Parameters(name = "tableProperties = {0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {ImmutableMap.of()}, + new Object[] { + ImmutableMap.of( + TableProperties.AVRO_COMPRESSION, + "zstd", + TableProperties.AVRO_COMPRESSION_LEVEL, + "3", + TableProperties.PARQUET_COMPRESSION, + "zstd", + TableProperties.PARQUET_COMPRESSION_LEVEL, + "3", + TableProperties.ORC_COMPRESSION, + "zstd", + TableProperties.ORC_COMPRESSION_STRATEGY, + "compression") + } + }; + } + + @BeforeEach + public void before() throws IOException { + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); + } + + @TestTemplate + public void testCompressionAvro() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); + + if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) + .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); + } else { + assertThat(resultProperties) + .containsEntry( + TableProperties.AVRO_COMPRESSION, + initProperties.get(TableProperties.AVRO_COMPRESSION)) + .containsEntry( + TableProperties.AVRO_COMPRESSION_LEVEL, + initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); + } + + // Override compression to snappy and some random level + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "AVRO", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_LEVEL.key(), + "6")); + + assertThat(resultProperties) + .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") + .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); + } + + @TestTemplate + public void testCompressionParquet() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); + + if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) + .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); + } else { + assertThat(resultProperties) + .containsEntry( + TableProperties.PARQUET_COMPRESSION, + initProperties.get(TableProperties.PARQUET_COMPRESSION)) + .containsEntry( + TableProperties.PARQUET_COMPRESSION_LEVEL, + initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); + } + + // Override compression to snappy and some random level + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "PARQUET", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_LEVEL.key(), + "6")); + + assertThat(resultProperties) + .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") + .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); + } + + @TestTemplate + public void testCompressionOrc() throws Exception { + // No override provided + Map resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); + + if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); + } else { + assertThat(resultProperties) + .containsEntry( + TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) + .containsEntry( + TableProperties.ORC_COMPRESSION_STRATEGY, + initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); + } + + // Override compression to snappy and a different strategy + resultProperties = + appenderProperties( + table, + SimpleDataUtil.FLINK_SCHEMA, + ImmutableMap.of( + FlinkWriteOptions.WRITE_FORMAT.key(), + "ORC", + FlinkWriteOptions.COMPRESSION_CODEC.key(), + "snappy", + FlinkWriteOptions.COMPRESSION_STRATEGY.key(), + "speed")); + + assertThat(resultProperties) + .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") + .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); + } + + private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); + FlinkWriteConf flinkWriteConfig = + new FlinkWriteConf( + icebergTable, override, new org.apache.flink.configuration.Configuration()); + + IcebergStreamWriter streamWriter = + FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); + + harness.setup(); + harness.open(); + + return harness; + } + + private static Map appenderProperties( + Table table, TableSchema schema, Map override) throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter(table, schema, override)) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + + testHarness.prepareSnapshotPreBarrier(1L); + DynFields.BoundField operatorField = + DynFields.builder() + .hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator") + .build(testHarness.getOperatorFactory()); + DynFields.BoundField writerField = + DynFields.builder() + .hiddenImpl(IcebergStreamWriter.class, "writer") + .build(operatorField.get()); + DynFields.BoundField appenderField = + DynFields.builder() + .hiddenImpl(BaseTaskWriter.class, "appenderFactory") + .build(writerField.get()); + DynFields.BoundField> propsField = + DynFields.builder() + .hiddenImpl(FlinkAppenderFactory.class, "props") + .build(appenderField.get()); + return propsField.get(); + } + } +} 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 new file mode 100644 index 000000000000..21f3ee2c655a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java @@ -0,0 +1,429 @@ +/* + * 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 static org.apache.iceberg.flink.SimpleDataUtil.createDelete; +import static org.apache.iceberg.flink.SimpleDataUtil.createInsert; +import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; +import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; +import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; +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.nio.file.Path; +import java.nio.file.Paths; +import java.time.OffsetDateTime; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.LocalZonedTimestampType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.FileContent; +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.RowDelta; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.TestTables; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +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.types.Types; +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 class TestDeltaTaskWriter extends TestBase { + + @Parameter(index = 1) + private FileFormat format; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.ORC}, + new Object[] {2, FileFormat.PARQUET}); + } + + @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"); + } + + private int idFieldId() { + return table.schema().findField("id").fieldId(); + } + + private int dataFieldId() { + return table.schema().findField("data").fieldId(); + } + + private void testCdcEvents(boolean partitioned) throws IOException { + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + // Start the 1th transaction. + TaskWriter writer = taskWriterFactory.create(); + + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "bbb")); + writer.write(createInsert(3, "ccc")); + + // Update <2, 'bbb'> to <2, 'ddd'> + writer.write(createUpdateBefore(2, "bbb")); // 1 pos-delete and 1 eq-delete. + writer.write(createUpdateAfter(2, "ddd")); + + // Update <1, 'aaa'> to <1, 'eee'> + writer.write(createUpdateBefore(1, "aaa")); // 1 pos-delete and 1 eq-delete. + writer.write(createUpdateAfter(1, "eee")); + + // Insert <4, 'fff'> + writer.write(createInsert(4, "fff")); + // Insert <5, 'ggg'> + writer.write(createInsert(5, "ggg")); + + // Delete <3, 'ccc'> + writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(1, "eee"), + createRecord(2, "ddd"), + createRecord(4, "fff"), + createRecord(5, "ggg"))); + + // Start the 2nd transaction. + writer = taskWriterFactory.create(); + + // Update <2, 'ddd'> to <6, 'hhh'> - (Update both key and value) + writer.write(createUpdateBefore(2, "ddd")); // 1 eq-delete + writer.write(createUpdateAfter(6, "hhh")); + + // Update <5, 'ggg'> to <5, 'iii'> + writer.write(createUpdateBefore(5, "ggg")); // 1 eq-delete + writer.write(createUpdateAfter(5, "iii")); + + // Delete <4, 'fff'> + writer.write(createDelete(4, "fff")); // 1 eq-delete. + + result = writer.complete(); + assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); + } + + @TestTemplate + public void testUnpartitioned() throws IOException { + createAndInitTable(false); + testCdcEvents(false); + } + + @TestTemplate + public void testPartitioned() throws IOException { + createAndInitTable(true); + testCdcEvents(true); + } + + private void testWritePureEqDeletes(boolean partitioned) throws IOException { + createAndInitTable(partitioned); + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + writer.write(createDelete(1, "aaa")); + writer.write(createDelete(2, "bbb")); + writer.write(createDelete(3, "ccc")); + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).isEmpty(); + assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); + commitTransaction(result); + + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); + } + + @TestTemplate + public void testUnpartitionedPureEqDeletes() throws IOException { + testWritePureEqDeletes(false); + } + + @TestTemplate + public void testPartitionedPureEqDeletes() throws IOException { + testWritePureEqDeletes(true); + } + + private void testAbort(boolean partitioned) throws IOException { + createAndInitTable(partitioned); + List equalityFieldIds = Lists.newArrayList(idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + for (int i = 0; i < 8_000; i += 2) { + writer.write(createUpdateBefore(i + 1, "aaa")); + writer.write(createUpdateAfter(i + 1, "aaa")); + + writer.write(createUpdateBefore(i + 2, "bbb")); + writer.write(createUpdateAfter(i + 2, "bbb")); + } + + // Assert the current data/delete file count. + List files = + Files.walk(Paths.get(tableDir.getPath(), "data")) + .filter(p -> p.toFile().isFile()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(files).hasSize(partitioned ? 4 : 2); + + writer.abort(); + for (Path file : files) { + assertThat(file).doesNotExist(); + } + } + + @TestTemplate + public void testUnpartitionedAbort() throws IOException { + testAbort(false); + } + + @TestTemplate + public void testPartitionedAbort() throws IOException { + testAbort(true); + } + + @TestTemplate + public void testPartitionedTableWithDataAsKey() throws IOException { + createAndInitTable(true); + List equalityFieldIds = Lists.newArrayList(dataFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + // Start the 1th transaction. + TaskWriter writer = taskWriterFactory.create(); + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "aaa")); + writer.write(createInsert(3, "bbb")); + writer.write(createInsert(4, "ccc")); + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).hasSize(3); + assertThat(result.deleteFiles()).hasSize(1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); + + // Start the 2nd transaction. + writer = taskWriterFactory.create(); + writer.write(createInsert(5, "aaa")); + writer.write(createInsert(6, "bbb")); + writer.write(createDelete(7, "ccc")); // 1 eq-delete. + + result = writer.complete(); + assertThat(result.dataFiles()).hasSize(2); + assertThat(result.deleteFiles()).hasSize(1); + commitTransaction(result); + + assertThat(actualRowSet("*")) + .isEqualTo( + expectedRowSet( + createRecord(2, "aaa"), + createRecord(5, "aaa"), + createRecord(3, "bbb"), + createRecord(6, "bbb"))); + } + + @TestTemplate + public void testPartitionedTableWithDataAndIdAsKey() throws IOException { + createAndInitTable(true); + List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + writer.write(createInsert(1, "aaa")); + writer.write(createInsert(2, "aaa")); + + writer.write(createDelete(2, "aaa")); // 1 pos-delete. + + WriteResult result = writer.complete(); + assertThat(result.dataFiles()).hasSize(1); + assertThat(result.deleteFiles()).hasSize(1); + assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); + commitTransaction(result); + + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); + } + + @TestTemplate + public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { + Schema tableSchema = + new Schema( + required(3, "id", Types.IntegerType.get()), + required(4, "ts", Types.TimestampType.withZone())); + RowType flinkType = + new RowType( + false, + ImmutableList.of( + new RowType.RowField("id", new IntType()), + new RowType.RowField("ts", new LocalZonedTimestampType(3)))); + + this.table = create(tableSchema, PartitionSpec.unpartitioned()); + initTable(table); + + List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); + TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); + taskWriterFactory.initialize(1, 1); + + TaskWriter writer = taskWriterFactory.create(); + RowDataSerializer serializer = new RowDataSerializer(flinkType); + OffsetDateTime start = OffsetDateTime.now(); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + writer.write( + serializer.toBinaryRow( + GenericRowData.ofKind( + RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); + + WriteResult result = writer.complete(); + // One data file + assertThat(result.dataFiles()).hasSize(1); + // One eq delete file + one pos delete file + assertThat(result.deleteFiles()).hasSize(2); + assertThat( + Arrays.stream(result.deleteFiles()) + .map(ContentFile::content) + .collect(Collectors.toSet())) + .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); + commitTransaction(result); + + Record expectedRecord = GenericRecord.create(tableSchema); + expectedRecord.setField("id", 1); + int cutPrecisionNano = start.getNano() / 1000000 * 1000000; + expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); + + assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); + } + + private void commitTransaction(WriteResult result) { + RowDelta rowDelta = table.newRowDelta(); + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + rowDelta + .validateDeletedFiles() + .validateDataFilesExist(Lists.newArrayList(result.referencedDataFiles())) + .commit(); + } + + private StructLikeSet expectedRowSet(Record... records) { + return SimpleDataUtil.expectedRowSet(table, records); + } + + private StructLikeSet actualRowSet(String... columns) throws IOException { + return SimpleDataUtil.actualRowSet(table, columns); + } + + private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + FlinkSchemaUtil.convert(table.schema()), + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + false); + } + + private TaskWriterFactory createTaskWriterFactory( + RowType flinkType, List equalityFieldIds) { + return new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + flinkType, + 128 * 1024 * 1024, + format, + table.properties(), + equalityFieldIds, + true); + } + + private void createAndInitTable(boolean partitioned) { + if (partitioned) { + this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); + } else { + this.table = create(SCHEMA, PartitionSpec.unpartitioned()); + } + + initTable(table); + } + + private void initTable(TestTables.TestTable testTable) { + testTable + .updateProperties() + .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) + .defaultFormat(format) + .commit(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java new file mode 100644 index 000000000000..dd89f43483b0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.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.flink.sink; + +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.TestAppenderFactory; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkAppenderFactory extends TestAppenderFactory { + + private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); + + @Override + protected FileAppenderFactory createAppenderFactory( + List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { + return new FlinkAppenderFactory( + table, + table.schema(), + rowType, + table.properties(), + table.spec(), + ArrayUtil.toIntArray(equalityFieldIds), + eqDeleteSchema, + posDeleteRowSchema); + } + + @Override + protected RowData createRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet expectedRowSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(rowType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java new file mode 100644 index 000000000000..414ee40d1357 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestFileWriterFactory; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkFileWriterFactory extends TestFileWriterFactory { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java new file mode 100644 index 000000000000..b778037c559c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.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.sink; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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.TableProperties; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + public void testWriteRow() throws Exception { + testWriteRow(parallelism, null, DistributionMode.NONE); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java new file mode 100644 index 000000000000..9ce36cc1e8d0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.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.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; +import java.util.List; +import java.util.stream.Collectors; +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.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + protected static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + protected static final DataFormatConverters.RowConverter CONVERTER = + new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + + protected BoundedTestSource createBoundedSource(List rows) { + return new BoundedTestSource<>(rows.toArray(new Row[0])); + } + + protected List createRows(String prefix) { + return Lists.newArrayList( + Row.of(1, prefix + "aaa"), + Row.of(1, prefix + "bbb"), + Row.of(1, prefix + "ccc"), + Row.of(2, prefix + "aaa"), + Row.of(2, prefix + "bbb"), + Row.of(2, prefix + "ccc"), + Row.of(3, prefix + "aaa"), + Row.of(3, prefix + "bbb"), + Row.of(3, prefix + "ccc")); + } + + protected List convertToRowData(List rows) { + return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); + } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java new file mode 100644 index 000000000000..441b5ed2a4ae --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.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.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +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.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { + @RegisterExtension + public static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @Parameter(index = 0) + private String formatVersion; + + @Parameter(index = 1) + private String branch; + + private TableLoader tableLoader; + + @Parameters(name = "formatVersion = {0}, branch = {1}") + public static Object[][] parameters() { + return new Object[][] { + {"1", "main"}, + {"1", "testBranch"}, + {"2", "main"}, + {"2", "testBranch"} + }; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + formatVersion)); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} 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 new file mode 100644 index 000000000000..75e397d3f203 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,180 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * This tests the distribution mode of Flink sink. Extract them separately since it is unnecessary + * to test different file formats (Avro, Orc, Parquet) like in {@link TestFlinkIcebergSink}. + * Removing the file format dimension reduces the number of combinations from 12 to 4, which helps + * reduce test run time. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private final FileFormat format = FileFormat.PARQUET; + + @Parameter(index = 0) + private int parallelism; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "parallelism = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {1, true}, + {1, false}, + {2, true}, + {2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).isGreaterThan(3); + } + } + } + + @TestTemplate + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testPartitionWriteMode() throws Exception { + testWriteRow(parallelism, null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +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.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); + rightTable.refresh(); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} 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 new file mode 100644 index 000000000000..577c54976b9a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java @@ -0,0 +1,235 @@ +/* + * 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 static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use schema identifier field IDs as equality field id list by default + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrderElementsOf(table.schema().identifierFieldIds()); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + + builder.equalityFieldColumns(Lists.newArrayList("type")); + assertThat(builder.checkAndGetEqualityFieldIds()) + .containsExactlyInAnyOrder(table.schema().findField("type").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + 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()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java new file mode 100644 index 000000000000..fc33c2fea5e6 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -0,0 +1,389 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +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.types.RowKind; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.IcebergGenerics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.BoundedTestSource; +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.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkIcebergSinkV2Base { + + protected static final int FORMAT_V2 = 2; + protected static final TypeInformation ROW_TYPE_INFO = + new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); + + protected static final int ROW_ID_POS = 0; + protected static final int ROW_DATA_POS = 1; + + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + + @Parameter(index = 0) + protected FileFormat format; + + @Parameter(index = 1) + protected int parallelism = 1; + + @Parameter(index = 2) + protected boolean partitioned; + + @Parameter(index = 3) + protected String writeDistributionMode; + + @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.AVRO, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, + new Object[] {FileFormat.ORC, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.ORC, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, + new Object[] {FileFormat.PARQUET, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, + new Object[] {FileFormat.PARQUET, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} + }; + } + + protected static final Map ROW_KIND_MAP = + ImmutableMap.of( + "+I", RowKind.INSERT, + "-D", RowKind.DELETE, + "-U", RowKind.UPDATE_BEFORE, + "+U", RowKind.UPDATE_AFTER); + + protected Row row(String rowKind, int id, String data) { + RowKind kind = ROW_KIND_MAP.get(rowKind); + if (kind == null) { + throw new IllegalArgumentException("Unknown row kind: " + rowKind); + } + + return Row.ofKind(kind, id, data); + } + + protected void testUpsertOnIdDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 2, "bbb"), row("+I", 2, "ccc")), + ImmutableList.of(row("+U", 1, "bbb"), row("-U", 1, "ccc"), row("-D", 1, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "aaa"), record(2, "bbb")), + ImmutableList.of(record(1, "aaa"), record(2, "ccc")), + ImmutableList.of(record(1, "bbb"), record(2, "ccc"))); + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnIdDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 2, "bbb"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa")), + ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "aaa"), record(2, "bbb")), + ImmutableList.of( + record(1, "aaa"), record(1, "bbb"), record(1, "ccc"), record(2, "bbb")), + ImmutableList.of( + record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "bbb"))); + + testChangeLogs( + ImmutableList.of("data", "id"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnSameKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa")), + // Checkpoint #2 + ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa")), + // Checkpoint #3 + ImmutableList.of(row("-D", 1, "aaa"), row("+I", 1, "aaa")), + // Checkpoint #4 + ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa")), + ImmutableList.of(record(1, "aaa"), record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 2, "bbb"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa")), + ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"), row("+I", 2, "ccc"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "aaa")), + ImmutableList.of(record(1, "aaa"), record(1, "bbb"), record(1, "ccc")), + ImmutableList.of( + record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "ccc"))); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testUpsertOnDataKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), + ImmutableList.of(row("+U", 4, "aaa"), row("-U", 3, "bbb"), row("+U", 5, "bbb")), + ImmutableList.of(row("+I", 6, "aaa"), row("+U", 7, "bbb"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(2, "aaa"), record(3, "bbb")), + ImmutableList.of(record(4, "aaa"), record(5, "bbb")), + ImmutableList.of(record(6, "aaa"), record(7, "bbb"))); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } + + protected void testChangeLogOnIdKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of( + row("+I", 1, "aaa"), + row("-D", 1, "aaa"), + row("+I", 1, "bbb"), + row("+I", 2, "aaa"), + row("-D", 2, "aaa"), + row("+I", 2, "bbb")), + ImmutableList.of( + row("-U", 2, "bbb"), row("+U", 2, "ccc"), row("-D", 2, "ccc"), row("+I", 2, "ddd")), + ImmutableList.of( + row("-D", 1, "bbb"), + row("+I", 1, "ccc"), + row("-D", 1, "ccc"), + row("+I", 1, "ddd"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb"), record(2, "bbb")), + ImmutableList.of(record(1, "bbb"), record(2, "ddd")), + ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); + + if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { + assertThatThrownBy( + () -> + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch)) + .isInstanceOf(IllegalStateException.class) + .hasMessageStartingWith( + "In 'hash' distribution mode with equality fields set, partition field") + .hasMessageContaining("should be included in equality fields:"); + + } else { + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + false, + elementsPerCheckpoint, + expectedRecords, + branch); + } + } + + protected void testUpsertOnIdKey(String branch) throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), + ImmutableList.of(row("+I", 1, "ccc")), + ImmutableList.of(row("+U", 1, "ddd"), row("+I", 1, "eee"))); + + List> expectedRecords = + ImmutableList.of( + ImmutableList.of(record(1, "bbb")), + ImmutableList.of(record(1, "ccc")), + ImmutableList.of(record(1, "eee"))); + + if (!partitioned) { + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch); + } else { + assertThatThrownBy( + () -> + testChangeLogs( + ImmutableList.of("id"), + row -> row.getField(ROW_ID_POS), + true, + elementsPerCheckpoint, + expectedRecords, + branch)) + .isInstanceOf(IllegalStateException.class) + .hasMessageContaining("should be included in equality fields:"); + } + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } + + protected Record record(int id, String data) { + return SimpleDataUtil.createRecord(id, data); + } + + private List findValidSnapshots() { + List validSnapshots = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + if (snapshot.allManifests(table.io()).stream() + .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { + validSnapshots.add(snapshot); + } + } + return validSnapshots; + } + + private StructLikeSet expectedRowSet(Record... records) { + return SimpleDataUtil.expectedRowSet(table, records); + } + + private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + table.refresh(); + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + try (CloseableIterable reader = + IcebergGenerics.read(table).useSnapshot(snapshotId).select(columns).build()) { + reader.forEach(set::add); + } + return set; + } +} 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 new file mode 100644 index 000000000000..0b0c55f51c32 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.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.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +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; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java new file mode 100644 index 000000000000..53b7c4c0cc91 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -0,0 +1,312 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ManifestFiles; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestFlinkManifest { + private static final Configuration CONF = new Configuration(); + + @TempDir protected Path temporaryFolder; + + private Table table; + private FileAppenderFactory appenderFactory; + private final AtomicInteger fileCount = new AtomicInteger(0); + + @BeforeEach + public void before() throws IOException { + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); + String warehouse = folder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).isTrue(); + + // Construct the iceberg table. + table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); + + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + this.appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + @Test + public void testIO() throws IOException { + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + for (long checkpointId = 1; checkpointId <= 3; checkpointId++) { + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + final long curCkpId = checkpointId; + + List dataFiles = generateDataFiles(10); + List eqDeleteFiles = generateEqDeleteFiles(5); + List posDeleteFiles = generatePosDeleteFiles(5); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(eqDeleteFiles) + .addDeleteFiles(posDeleteFiles) + .build(), + () -> factory.create(curCkpId), + table.spec()); + + WriteResult result = + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); + assertThat(result.deleteFiles()).hasSize(10); + for (int i = 0; i < dataFiles.size(); i++) { + TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); + } + assertThat(result.deleteFiles()).hasSize(10); + for (int i = 0; i < 5; i++) { + TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); + } + for (int i = 0; i < 5; i++) { + TestHelpers.assertEquals(posDeleteFiles.get(i), result.deleteFiles()[5 + i]); + } + } + } + + @Test + public void testUserProvidedManifestLocation() throws IOException { + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); + Map props = + ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ManifestOutputFileFactory factory = + new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(5); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder().addDataFiles(dataFiles).build(), + () -> factory.create(checkpointId), + table.spec()); + + assertThat(deltaManifests.dataManifest()).isNotNull(); + assertThat(deltaManifests.deleteManifest()).isNull(); + assertThat(Paths.get(deltaManifests.dataManifest().path())) + .hasParent(userProvidedFolder.toPath()); + + WriteResult result = + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); + + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(5); + + assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); + for (int i = 0; i < dataFiles.size(); i++) { + TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); + } + } + + @Test + public void testVersionedSerializer() throws IOException { + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(10); + List eqDeleteFiles = generateEqDeleteFiles(10); + List posDeleteFiles = generatePosDeleteFiles(10); + DeltaManifests expected = + FlinkManifestUtil.writeCompletedFiles( + WriteResult.builder() + .addDataFiles(dataFiles) + .addDeleteFiles(eqDeleteFiles) + .addDeleteFiles(posDeleteFiles) + .build(), + () -> factory.create(checkpointId), + table.spec()); + + byte[] versionedSerializeData = + SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, expected); + DeltaManifests actual = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, versionedSerializeData); + TestHelpers.assertEquals(expected.dataManifest(), actual.dataManifest()); + TestHelpers.assertEquals(expected.deleteManifest(), actual.deleteManifest()); + + byte[] versionedSerializeData2 = + SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, actual); + assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); + } + + @Test + public void testCompatibility() throws IOException { + // The v2 deserializer should be able to deserialize the v1 binary. + long checkpointId = 1; + String flinkJobId = newFlinkJobId(); + String operatorId = newOperatorUniqueId(); + ManifestOutputFileFactory factory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, 1, 1); + + List dataFiles = generateDataFiles(10); + ManifestFile manifest = + FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); + byte[] dataV1 = + SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); + + DeltaManifests delta = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, dataV1); + assertThat(delta.deleteManifest()).isNull(); + assertThat(delta.dataManifest()).isNotNull(); + TestHelpers.assertEquals(manifest, delta.dataManifest()); + + List actualFiles = + FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); + assertThat(actualFiles).hasSize(10); + for (int i = 0; i < 10; i++) { + TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); + } + } + + private static class V1Serializer implements SimpleVersionedSerializer { + + @Override + public int getVersion() { + return 1; + } + + @Override + public byte[] serialize(ManifestFile m) throws IOException { + return ManifestFiles.encode(m); + } + + @Override + public ManifestFile deserialize(int version, byte[] serialized) throws IOException { + return ManifestFiles.decode(serialized); + } + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + CONF, + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private DeleteFile writePosDeleteFile(String filename, List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, positions); + } + + private List generateDataFiles(int fileNum) throws IOException { + List rowDataList = Lists.newArrayList(); + List dataFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + rowDataList.add(SimpleDataUtil.createRowData(i, "a" + i)); + dataFiles.add(writeDataFile("data-file-" + fileCount.incrementAndGet(), rowDataList)); + } + return dataFiles; + } + + private List generateEqDeleteFiles(int fileNum) throws IOException { + List rowDataList = Lists.newArrayList(); + List deleteFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + rowDataList.add(SimpleDataUtil.createDelete(i, "a" + i)); + deleteFiles.add( + writeEqDeleteFile("eq-delete-file-" + fileCount.incrementAndGet(), rowDataList)); + } + return deleteFiles; + } + + private List generatePosDeleteFiles(int fileNum) throws IOException { + List> positions = Lists.newArrayList(); + List deleteFiles = Lists.newArrayList(); + for (int i = 0; i < fileNum; i++) { + positions.add(Pair.of("data-file-1", (long) i)); + deleteFiles.add( + writePosDeleteFile("pos-delete-file-" + fileCount.incrementAndGet(), positions)); + } + return deleteFiles; + } + + private static String newFlinkJobId() { + return UUID.randomUUID().toString(); + } + + private static String newOperatorUniqueId() { + return UUID.randomUUID().toString(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java new file mode 100644 index 000000000000..939ed2be7dbc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.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.flink.sink; + +import java.util.Arrays; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPartitioningWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkPartitioningWriters extends TestPartitioningWriters { + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}") + protected static List parameters() { + return Arrays.asList( + new Object[] {2, FileFormat.AVRO}, + new Object[] {2, FileFormat.PARQUET}, + new Object[] {2, FileFormat.ORC}); + } + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java new file mode 100644 index 000000000000..3050752d1c24 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java @@ -0,0 +1,66 @@ +/* + * 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 org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestPositionDeltaWriters; +import org.apache.iceberg.util.ArrayUtil; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } + + @Override + protected StructLikeSet toSet(Iterable rows) { + StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); + RowType flinkType = FlinkSchemaUtil.convert(table.schema()); + for (RowData row : rows) { + RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); + set.add(wrapper.wrap(row)); + } + return set; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java new file mode 100644 index 000000000000..03051b69cf87 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java @@ -0,0 +1,51 @@ +/* + * 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 org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestRollingFileWriters; +import org.apache.iceberg.util.ArrayUtil; + +public class TestFlinkRollingFileWriters extends TestRollingFileWriters { + + @Override + protected FileWriterFactory newWriterFactory( + Schema dataSchema, + List equalityFieldIds, + Schema equalityDeleteRowSchema, + Schema positionDeleteRowSchema) { + return FlinkFileWriterFactory.builderFor(table) + .dataSchema(table.schema()) + .dataFileFormat(format()) + .deleteFileFormat(format()) + .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) + .equalityDeleteRowSchema(equalityDeleteRowSchema) + .positionDeleteRowSchema(positionDeleteRowSchema) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data) { + return SimpleDataUtil.createRowData(id, data); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java new file mode 100644 index 000000000000..e6d64ef2c720 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.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.flink.sink; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.io.FileWriterFactory; +import org.apache.iceberg.io.TestWriterMetrics; + +public class TestFlinkWriterMetrics extends TestWriterMetrics { + + public TestFlinkWriterMetrics(FileFormat fileFormat) { + super(fileFormat); + } + + @Override + protected FileWriterFactory newWriterFactory(Table sourceTable) { + return FlinkFileWriterFactory.builderFor(sourceTable) + .dataSchema(sourceTable.schema()) + .dataFileFormat(fileFormat) + .deleteFileFormat(fileFormat) + .positionDeleteRowSchema(sourceTable.schema()) + .build(); + } + + @Override + protected RowData toRow(Integer id, String data, boolean boolValue, Long longValue) { + GenericRowData nested = GenericRowData.of(boolValue, longValue); + GenericRowData row = GenericRowData.of(id, StringData.fromString(data), nested); + return row; + } + + @Override + public RowData toGenericRow(int value, int repeated) { + GenericRowData row = new GenericRowData(repeated); + for (int i = 0; i < repeated; i++) { + row.setField(i, value); + } + return row; + } +} 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 new file mode 100644 index 000000000000..948c7b31430c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java @@ -0,0 +1,1148 @@ +/* + * 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 static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; +import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +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.Files; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.NavigableMap; +import java.util.SortedMap; +import java.util.stream.Collectors; +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.state.ListState; +import org.apache.flink.api.common.state.OperatorStateStore; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.testutils.MockEnvironment; +import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionData; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +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.util.Pair; +import org.apache.iceberg.util.ThreadPools; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergFilesCommitter extends TestBase { + private static final Configuration CONF = new Configuration(); + + private File flinkManifestFolder; + + @Parameter(index = 1) + private FileFormat format; + + @Parameter(index = 2) + private String branch; + + @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") + protected static List parameters() { + return Arrays.asList( + new Object[] {1, FileFormat.AVRO, "main"}, + new Object[] {2, FileFormat.AVRO, "test-branch"}, + new Object[] {1, FileFormat.PARQUET, "main"}, + new Object[] {2, FileFormat.PARQUET, "test-branch"}, + new Object[] {1, FileFormat.ORC, "main"}, + new Object[] {2, FileFormat.ORC, "test-branch"}); + } + + @Override + @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()); + + table + .updateProperties() + .set(DEFAULT_FILE_FORMAT, format.name()) + .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) + .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") + .commit(); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + long checkpointId = 0; + long timestamp = 0; + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the + // future flink job + // failover won't fail. + for (int i = 1; i <= 3; i++) { + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + } + + @TestTemplate + public void testMaxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + + JobID jobId = new JobID(); + long checkpointId = 0; + long timestamp = 0; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + + assertSnapshotSize(0); + + for (int i = 1; i <= 9; i++) { + harness.snapshot(++checkpointId, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(i / 3); + } + } + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + @TestTemplate + public void testCommitTxn() throws Exception { + // Test with 3 continues checkpoints: + // 1. snapshotState for checkpoint#1 + // 2. notifyCheckpointComplete for checkpoint#1 + // 3. snapshotState for checkpoint#2 + // 4. notifyCheckpointComplete for checkpoint#2 + // 5. snapshotState for checkpoint#3 + // 6. notifyCheckpointComplete for checkpoint#3 + long timestamp = 0; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + + harness.snapshot(i, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(i); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobID, operatorId, i); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); + } + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + long timestamp = 0; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + harness.processElement(of(dataFile2), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that the two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + long timestamp = 0; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + harness.processElement(of(dataFile2), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testRecoveryFromValidSnapshot() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); + + harness.processElement(of(dataFile1), ++timestamp); + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's + // possible that we + // flink job will restore from a checkpoint with only step#1 finished. + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(newJobId)) { + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId, operatorId, -1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + harness.processElement(of(dataFile), ++timestamp); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId, operatorId, checkpointId); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(oldJobId)) { + harness.setup(); + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 0; + timestamp = 0; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(newJobId)) { + harness.setup(); + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, 3); + assertMaxCommittedCheckpointId(newJobId, newOperatorId, -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId, newOperatorId, checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobId = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId == 0 ? -1 : checkpointId); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + harness.processElement(of(dataFile), ++timestamp); + harness.snapshot(checkpointId + 1, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId + 1); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId + 1); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobId = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId1, -1L); + assertMaxCommittedCheckpointId(jobId, operatorId2, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + + harness1.processElement(of(dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + + harness2.processElement(of(dataFile2), ++timestamp); + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, -1); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + harness1.processElement(of(dataFile1), ++timestamp); + harness1.snapshot(++checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + harness2.processElement(of(dataFile2), ++timestamp); + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); + } + } + + @TestTemplate + public void testBoundedStream() throws Exception { + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertFlinkManifests(0); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); + + DataFile dataFile = writeDataFile("data-1", tableRows); + harness.processElement(of(dataFile), 1); + ((BoundedOneInput) harness.getOneInputOperator()).endInput(); + + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) + .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + final long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); + + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + harness.processElement(of(dataFile1), ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); + + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + + // 4. process both data files and delete files. + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + ++timestamp); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + + // 5. snapshotState for checkpoint#2 + harness.snapshot(++checkpoint, ++timestamp); + assertFlinkManifests(2); + + // 6. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testCommitTwoCheckpointsInSingleTxn() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); + RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); + RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + ++timestamp); + + // The 1th snapshotState. + harness.snapshot(checkpoint, ++timestamp); + + RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); + RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); + DeleteFile deleteFile2 = + writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); + harness.processElement( + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + ++timestamp); + + // The 2nd snapshotState. + harness.snapshot(++checkpoint, ++timestamp); + + // Notify the 2nd snapshot to complete. + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(2); + } + } + + @TestTemplate + public void testSpecEvolution() throws Exception { + long timestamp = 0; + int checkpointId = 0; + List rows = Lists.newArrayList(); + JobID jobId = new JobID(); + + OperatorID operatorId; + OperatorSubtaskState snapshot; + DataFile dataFile; + int specId; + + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + + checkpointId++; + RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); + // table unpartitioned + dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + harness.snapshot(checkpointId, ++timestamp); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(table.spec().specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + // Change partition spec + table.refresh(); + PartitionSpec oldSpec = table.spec(); + table.updateSpec().addField("id").commit(); + + checkpointId++; + rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); + // write data with old partition spec + dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); + harness.processElement(of(dataFile), ++timestamp); + rows.add(rowData); + snapshot = harness.snapshot(checkpointId, ++timestamp); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(oldSpec.specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.setup(); + harness.initializeState(snapshot); + harness.open(); + + SimpleDataUtil.assertTableRows(table, rows, branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + + checkpointId++; + RowData row = SimpleDataUtil.createRowData(checkpointId, "world" + checkpointId); + StructLike partition = new PartitionData(table.spec().partitionType()); + partition.set(0, checkpointId); + dataFile = + writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); + harness.processElement(of(dataFile), ++timestamp); + rows.add(row); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + specId = + getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); + assertThat(specId).isEqualTo(table.spec().specId()); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, rows, branch); + assertSnapshotSize(checkpointId); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); + } + } + + private int getStagingManifestSpecId(OperatorStateStore operatorStateStore, long checkPointId) + throws Exception { + ListState> checkpointsState = + operatorStateStore.getListState(IcebergFilesCommitter.buildStateDescriptor()); + NavigableMap statedDataFiles = + Maps.newTreeMap(checkpointsState.get().iterator().next()); + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, statedDataFiles.get(checkPointId)); + return deltaManifests.dataManifest().partitionSpecId(); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, deletes); + } + + private DeleteFile writePosDeleteFile( + FileAppenderFactory appenderFactory, + String filename, + List> positions) + throws IOException { + return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, positions); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + CONF, + table.location(), + format.addExtension(filename), + rows); + } + + private DataFile writeDataFile( + String filename, List rows, PartitionSpec spec, StructLike partition) + throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + spec, + CONF, + table.location(), + format.addExtension(filename), + rows, + partition); + } + + private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { + table.refresh(); + long actualId = + IcebergFilesCommitter.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toHexString(), branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + throws Exception { + TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); + return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); + } + + private static MockEnvironment createEnvironment(JobID jobID) { + return new MockEnvironmentBuilder() + .setTaskName("test task") + .setManagedMemorySize(32 * 1024) + .setInputSplitProvider(new MockInputSplitProvider()) + .setBufferSize(256) + .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) + .setExecutionConfig(new ExecutionConfig()) + .setMaxParallelism(16) + .setJobID(jobID) + .build(); + } + + private static class TestOperatorFactory extends AbstractStreamOperatorFactory + implements OneInputStreamOperatorFactory { + private final String tablePath; + private final String branch; + private final PartitionSpec spec; + + private TestOperatorFactory(String tablePath, String branch, PartitionSpec spec) { + this.tablePath = tablePath; + this.branch = branch; + this.spec = spec; + } + + private static TestOperatorFactory of(String tablePath, String branch, PartitionSpec spec) { + return new TestOperatorFactory(tablePath, branch, spec); + } + + @Override + @SuppressWarnings("unchecked") + public > T createStreamOperator( + StreamOperatorParameters param) { + IcebergFilesCommitter committer = + new IcebergFilesCommitter( + new TestTableLoader(tablePath), + false, + Collections.singletonMap("flink.test", TestIcebergFilesCommitter.class.getName()), + ThreadPools.WORKER_THREAD_POOL_SIZE, + branch, + spec); + committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); + return (T) committer; + } + + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return IcebergFilesCommitter.class; + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java new file mode 100644 index 000000000000..50283f7ad215 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -0,0 +1,390 @@ +/* + * 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 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 java.util.Locale; +import java.util.Map; +import java.util.Set; +import org.apache.flink.streaming.api.operators.BoundedOneInput; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +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.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.io.WriteResult; +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.types.Types; +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 TestIcebergStreamWriter { + @TempDir protected java.nio.file.Path temporaryFolder; + + private Table table; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} + }; + } + + @BeforeEach + public void before() throws IOException { + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); + // Construct the iceberg table. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); + } + + @TestTemplate + public void testWritingTable() throws Exception { + long checkpointId = 1L; + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + // The first checkpoint + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + + checkpointId = checkpointId + 1; + + // The second checkpoint + testHarness.processElement(SimpleDataUtil.createRowData(4, "foo"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(5, "bar"), 2); + + testHarness.prepareSnapshotPreBarrier(checkpointId); + expectedDataFiles = partitioned ? 4 : 2; + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + + // Commit the iceberg transaction. + AppendFiles appendFiles = table.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + + // Assert the table records. + SimpleDataUtil.assertTableRecords( + table, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "hello"), + SimpleDataUtil.createRecord(2, "world"), + SimpleDataUtil.createRecord(3, "hello"), + SimpleDataUtil.createRecord(4, "foo"), + SimpleDataUtil.createRecord(5, "bar"))); + } + } + + @TestTemplate + public void testSnapshotTwice() throws Exception { + long checkpointId = 1; + long timestamp = 1; + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); + + testHarness.prepareSnapshotPreBarrier(checkpointId++); + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + + // snapshot again immediately. + for (int i = 0; i < 5; i++) { + testHarness.prepareSnapshotPreBarrier(checkpointId++); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + } + } + } + + @TestTemplate + public void testTableWithoutSnapshot() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + assertThat(testHarness.extractOutputValues()).isEmpty(); + } + // Even if we closed the iceberg stream writer, there's no orphan data file. + assertThat(scanDataFiles()).isEmpty(); + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + // Still not emit the data file yet, because there is no checkpoint. + assertThat(testHarness.extractOutputValues()).isEmpty(); + } + // Once we closed the iceberg stream writer, there will left an orphan data file. + assertThat(scanDataFiles()).hasSize(1); + } + + private Set scanDataFiles() throws IOException { + Path dataDir = new Path(table.location(), "data"); + FileSystem fs = FileSystem.get(new Configuration()); + if (!fs.exists(dataDir)) { + return ImmutableSet.of(); + } else { + Set paths = Sets.newHashSet(); + RemoteIterator iterators = fs.listFiles(dataDir, true); + while (iterators.hasNext()) { + LocatedFileStatus status = iterators.next(); + if (status.isFile()) { + Path path = status.getPath(); + if (path.getName().endsWith("." + format.toString().toLowerCase(Locale.ROOT))) { + paths.add(path.toString()); + } + } + } + return paths; + } + } + + @TestTemplate + public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); + + assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); + ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); + + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + + ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + // Datafiles should not be sent again + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + } + } + + @TestTemplate + public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); + testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); + + testHarness.endInput(); + + int expectedDataFiles = partitioned ? 2 : 1; + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + + testHarness.prepareSnapshotPreBarrier(1L); + + result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier + // is triggered, write should only send WriteResult once + assertThat(result.dataFiles()).hasSize(expectedDataFiles); + } + } + + @TestTemplate + public void testTableWithTargetFileSize() throws Exception { + // Adjust the target-file-size in table properties. + table + .updateProperties() + .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger + .commit(); + + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(SimpleDataUtil.createRowData(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter()) { + for (RowData row : rows) { + testHarness.processElement(row, 1); + } + + // snapshot the operator. + testHarness.prepareSnapshotPreBarrier(1); + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(8); + + // Assert that the data file have the expected records. + for (DataFile dataFile : result.dataFiles()) { + assertThat(dataFile.recordCount()).isEqualTo(1000); + } + + // Commit the iceberg transaction. + AppendFiles appendFiles = table.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + // Assert the table records. + SimpleDataUtil.assertTableRecords(table, records); + } + + @TestTemplate + public void testPromotedFlinkDataType() throws Exception { + Schema iSchema = + new Schema( + Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), + Types.NestedField.required(2, "smallint", Types.IntegerType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + TableSchema flinkSchema = + TableSchema.builder() + .field("tinyint", DataTypes.TINYINT().notNull()) + .field("smallint", DataTypes.SMALLINT().notNull()) + .field("int", DataTypes.INT().nullable()) + .build(); + + PartitionSpec spec; + if (partitioned) { + spec = + PartitionSpec.builderFor(iSchema) + .identity("smallint") + .identity("tinyint") + .identity("int") + .build(); + } else { + spec = PartitionSpec.unpartitioned(); + } + + String location = + Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); + + List rows = + Lists.newArrayList( + GenericRowData.of((byte) 0x01, (short) -32768, 101), + GenericRowData.of((byte) 0x02, (short) 0, 102), + GenericRowData.of((byte) 0x03, (short) 32767, 103)); + + Record record = GenericRecord.create(iSchema); + List expected = + Lists.newArrayList( + record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), + record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), + record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); + + try (OneInputStreamOperatorTestHarness testHarness = + createIcebergStreamWriter(icebergTable, flinkSchema)) { + for (RowData row : rows) { + testHarness.processElement(row, 1); + } + testHarness.prepareSnapshotPreBarrier(1); + WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + assertThat(result.deleteFiles()).isEmpty(); + assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); + + // Commit the iceberg transaction. + AppendFiles appendFiles = icebergTable.newAppend(); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + SimpleDataUtil.assertTableRecords(location, expected); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + throws Exception { + return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema) throws Exception { + RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); + FlinkWriteConf flinkWriteConfig = + new FlinkWriteConf( + icebergTable, Maps.newHashMap(), new org.apache.flink.configuration.Configuration()); + + IcebergStreamWriter streamWriter = + FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); + + harness.setup(); + harness.open(); + + return harness; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java new file mode 100644 index 000000000000..919fef579ab0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.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.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.PartitionKey; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.InternalRecordWrapper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestRowDataPartitionKey { + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(0, "boolType", Types.BooleanType.get()), + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "longType", Types.LongType.get()), + Types.NestedField.required(3, "dateType", Types.DateType.get()), + Types.NestedField.required(4, "timeType", Types.TimeType.get()), + Types.NestedField.required(5, "stringType", Types.StringType.get()), + Types.NestedField.required(6, "timestampWithoutZone", Types.TimestampType.withoutZone()), + Types.NestedField.required(7, "timestampWithZone", Types.TimestampType.withZone()), + Types.NestedField.required(8, "fixedType", Types.FixedType.ofLength(5)), + Types.NestedField.required(9, "uuidType", Types.UUIDType.get()), + Types.NestedField.required(10, "binaryType", Types.BinaryType.get()), + Types.NestedField.required(11, "decimalType1", Types.DecimalType.of(18, 3)), + Types.NestedField.required(12, "decimalType2", Types.DecimalType.of(10, 5)), + Types.NestedField.required(13, "decimalType3", Types.DecimalType.of(38, 19)), + Types.NestedField.required(14, "floatType", Types.FloatType.get()), + Types.NestedField.required(15, "doubleType", Types.DoubleType.get())); + + private static final List SUPPORTED_PRIMITIVES = + SCHEMA.asStruct().fields().stream().map(Types.NestedField::name).collect(Collectors.toList()); + + private static final Schema NESTED_SCHEMA = + new Schema( + Types.NestedField.required( + 1, + "structType", + Types.StructType.of( + Types.NestedField.optional(2, "innerStringType", Types.StringType.get()), + Types.NestedField.optional(3, "innerIntegerType", Types.IntegerType.get())))); + + @Test + public void testNullPartitionValue() { + Schema schema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "data", Types.StringType.get())); + + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); + + List rows = + Lists.newArrayList( + GenericRowData.of(1, StringData.fromString("a")), + GenericRowData.of(2, StringData.fromString("b")), + GenericRowData.of(3, null)); + + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); + + for (RowData row : rows) { + PartitionKey partitionKey = new PartitionKey(spec, schema); + partitionKey.partition(rowWrapper.wrap(row)); + assertThat(partitionKey.size()).isEqualTo(1); + + String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); + assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); + } + } + + @Test + public void testPartitionWithOneNestedField() { + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); + List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1991); + List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); + + PartitionSpec spec1 = + PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerStringType").build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerIntegerType").build(); + + for (int i = 0; i < rows.size(); i++) { + RowData row = rows.get(i); + Record record = (Record) records.get(i).get(0); + + PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); + partitionKey1.partition(rowWrapper.wrap(row)); + assertThat(partitionKey1.size()).isEqualTo(1); + + assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); + + PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); + partitionKey2.partition(rowWrapper.wrap(row)); + assertThat(partitionKey2.size()).isEqualTo(1); + + assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); + } + } + + @Test + public void testPartitionMultipleNestedField() { + RowDataWrapper rowWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); + List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1992); + List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); + + PartitionSpec spec1 = + PartitionSpec.builderFor(NESTED_SCHEMA) + .identity("structType.innerIntegerType") + .identity("structType.innerStringType") + .build(); + PartitionSpec spec2 = + PartitionSpec.builderFor(NESTED_SCHEMA) + .identity("structType.innerStringType") + .identity("structType.innerIntegerType") + .build(); + + PartitionKey pk1 = new PartitionKey(spec1, NESTED_SCHEMA); + PartitionKey pk2 = new PartitionKey(spec2, NESTED_SCHEMA); + + for (int i = 0; i < rows.size(); i++) { + RowData row = rows.get(i); + Record record = (Record) records.get(i).get(0); + + pk1.partition(rowWrapper.wrap(row)); + assertThat(pk1.size()).isEqualTo(2); + + assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); + assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); + + pk2.partition(rowWrapper.wrap(row)); + assertThat(pk2.size()).isEqualTo(2); + + assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); + assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); + } + } + + @Test + public void testPartitionValueTypes() { + RowType rowType = FlinkSchemaUtil.convert(SCHEMA); + RowDataWrapper rowWrapper = new RowDataWrapper(rowType, SCHEMA.asStruct()); + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(SCHEMA.asStruct()); + + List records = RandomGenericData.generate(SCHEMA, 10, 1993); + List rows = Lists.newArrayList(RandomRowData.convert(SCHEMA, records)); + + for (String column : SUPPORTED_PRIMITIVES) { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity(column).build(); + Class[] javaClasses = spec.javaClasses(); + + PartitionKey pk = new PartitionKey(spec, SCHEMA); + PartitionKey expectedPK = new PartitionKey(spec, SCHEMA); + + for (int j = 0; j < rows.size(); j++) { + RowData row = rows.get(j); + Record record = records.get(j); + + pk.partition(rowWrapper.wrap(row)); + expectedPK.partition(recordWrapper.wrap(record)); + + assertThat(pk.size()) + .as("Partition with column " + column + " should have one field.") + .isEqualTo(1); + + if (column.equals("timeType")) { + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); + } else { + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with column " + column + " should have the expected values") + .isEqualTo(expectedPK.get(0, javaClasses[0])); + } + } + } + } + + @Test + public void testNestedPartitionValues() { + Schema nestedSchema = new Schema(Types.NestedField.optional(1001, "nested", SCHEMA.asStruct())); + RowType rowType = FlinkSchemaUtil.convert(nestedSchema); + + RowDataWrapper rowWrapper = new RowDataWrapper(rowType, nestedSchema.asStruct()); + InternalRecordWrapper recordWrapper = new InternalRecordWrapper(nestedSchema.asStruct()); + + List records = RandomGenericData.generate(nestedSchema, 10, 1994); + List rows = Lists.newArrayList(RandomRowData.convert(nestedSchema, records)); + + for (String supportedPrimitive : SUPPORTED_PRIMITIVES) { + String column = String.format("nested.%s", supportedPrimitive); + + PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity(column).build(); + Class[] javaClasses = spec.javaClasses(); + + PartitionKey pk = new PartitionKey(spec, nestedSchema); + PartitionKey expectedPK = new PartitionKey(spec, nestedSchema); + + for (int j = 0; j < rows.size(); j++) { + pk.partition(rowWrapper.wrap(rows.get(j))); + expectedPK.partition(recordWrapper.wrap(records.get(j))); + + assertThat(pk.size()) + .as("Partition with nested column " + column + " should have one field.") + .isEqualTo(1); + + if (column.equals("nested.timeType")) { + assertThat(pk.get(0, Long.class) / 1000) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, Long.class) / 1000); + } else { + assertThat(pk.get(0, javaClasses[0])) + .as("Partition with nested column " + column + " should have the expected values.") + .isEqualTo(expectedPK.get(0, javaClasses[0])); + } + } + } + } +} 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 new file mode 100644 index 000000000000..8bfd6cb3d043 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java @@ -0,0 +1,242 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.data.RandomRowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +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 TestTaskWriters { + private static final Configuration CONF = new Configuration(); + private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; + + @TempDir protected java.nio.file.Path temporaryFolder; + + @Parameters(name = "format = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, true}, + {FileFormat.AVRO, false}, + {FileFormat.ORC, true}, + {FileFormat.ORC, false}, + {FileFormat.PARQUET, true}, + {FileFormat.PARQUET, false} + }; + } + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private boolean partitioned; + + private Table table; + + @BeforeEach + public void before() throws IOException { + File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); + // Construct the iceberg table with the specified file format. + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); + } + + @TestTemplate + public void testWriteZeroRecord() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.close(); + + DataFile[] dataFiles = taskWriter.dataFiles(); + assertThat(dataFiles).isNotNull().isEmpty(); + + // Close again. + taskWriter.close(); + dataFiles = taskWriter.dataFiles(); + assertThat(dataFiles).isNotNull().isEmpty(); + } + } + + @TestTemplate + public void testCloseTwice() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + taskWriter.close(); // The first close + taskWriter.close(); // The second close + + int expectedFiles = partitioned ? 2 : 1; + DataFile[] dataFiles = taskWriter.dataFiles(); + assertThat(dataFiles).hasSize(expectedFiles); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + } + } + } + + @TestTemplate + public void testAbort() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); + taskWriter.write(SimpleDataUtil.createRowData(2, "world")); + + taskWriter.abort(); + DataFile[] dataFiles = taskWriter.dataFiles(); + + int expectedFiles = partitioned ? 2 : 1; + assertThat(dataFiles).hasSize(expectedFiles); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); + } + } + } + + @TestTemplate + public void testCompleteFiles() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + taskWriter.write(SimpleDataUtil.createRowData(1, "a")); + taskWriter.write(SimpleDataUtil.createRowData(2, "b")); + taskWriter.write(SimpleDataUtil.createRowData(3, "c")); + taskWriter.write(SimpleDataUtil.createRowData(4, "d")); + + DataFile[] dataFiles = taskWriter.dataFiles(); + int expectedFiles = partitioned ? 4 : 1; + assertThat(dataFiles).hasSize(expectedFiles); + + dataFiles = taskWriter.dataFiles(); + assertThat(dataFiles).hasSize(expectedFiles); + + FileSystem fs = FileSystem.get(CONF); + for (DataFile dataFile : dataFiles) { + assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); + } + + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords( + table, + Lists.newArrayList( + SimpleDataUtil.createRecord(1, "a"), + SimpleDataUtil.createRecord(2, "b"), + SimpleDataUtil.createRecord(3, "c"), + SimpleDataUtil.createRecord(4, "d"))); + } + } + + @TestTemplate + public void testRollingWithTargetFileSize() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(4)) { + List rows = Lists.newArrayListWithCapacity(8000); + List records = Lists.newArrayListWithCapacity(8000); + for (int i = 0; i < 2000; i++) { + for (String data : new String[] {"a", "b", "c", "d"}) { + rows.add(SimpleDataUtil.createRowData(i, data)); + records.add(SimpleDataUtil.createRecord(i, data)); + } + } + + for (RowData row : rows) { + taskWriter.write(row); + } + + DataFile[] dataFiles = taskWriter.dataFiles(); + assertThat(dataFiles).hasSize(8); + + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRecords(table, records); + } + } + + @TestTemplate + public void testRandomData() throws IOException { + try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { + Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); + for (RowData row : rows) { + taskWriter.write(row); + } + + taskWriter.close(); + DataFile[] dataFiles = taskWriter.dataFiles(); + AppendFiles appendFiles = table.newAppend(); + for (DataFile dataFile : dataFiles) { + appendFiles.appendFile(dataFile); + } + appendFiles.commit(); + + // Assert the data rows. + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(rows)); + } + } + + private TaskWriter createTaskWriter(long targetFileSize) { + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), + targetFileSize, + format, + table.properties(), + null, + false); + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java new file mode 100644 index 000000000000..5910bd685510 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java @@ -0,0 +1,100 @@ +/* + * 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.Comparator; +import java.util.Map; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.runtime.typeutils.RowDataSerializer; +import org.apache.flink.table.types.logical.IntType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.logical.VarCharType; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; + +class Fixtures { + private Fixtures() {} + + public static final int NUM_SUBTASKS = 2; + public static final Schema SCHEMA = + new Schema( + Types.NestedField.optional(1, "id", Types.StringType.get()), + Types.NestedField.optional(2, "number", Types.IntegerType.get())); + public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); + public static final TypeSerializer ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); + public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); + public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + public static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); + public static final SortKeySerializer SORT_KEY_SERIALIZER = + new SortKeySerializer(SCHEMA, SORT_ORDER); + public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = + new DataStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = + new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); + public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = + new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); + + public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); + public static final Map CHAR_KEYS = createCharKeys(); + + public static StatisticsEvent createStatisticsEvent( + StatisticsType type, + TypeSerializer statisticsSerializer, + long checkpointId, + SortKey... keys) { + DataStatistics statistics = createTaskStatistics(type, keys); + return StatisticsEvent.createTaskStatisticsEvent( + checkpointId, statistics, statisticsSerializer); + } + + public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { + DataStatistics statistics; + if (type == StatisticsType.Sketch) { + statistics = new SketchDataStatistics(128); + } else { + statistics = new MapDataStatistics(); + } + + for (SortKey key : keys) { + statistics.add(key); + } + + return statistics; + } + + private static Map createCharKeys() { + Map keys = Maps.newHashMap(); + for (char c = 'a'; c <= 'z'; ++c) { + String key = Character.toString(c); + SortKey sortKey = SORT_KEY.copy(); + sortKey.set(0, key); + keys.put(key, sortKey); + } + + return keys; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java new file mode 100644 index 000000000000..8322ce683768 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestAggregatedStatisticsTracker { + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveNewerStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + // checkpoint 1 is completed + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + // checkpoint 2 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveOlderStatisticsEventTest(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + // both checkpoints are tracked + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint3Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 2L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); + // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); + aggregation = tracker.aggregationsPerCheckpoint().get(3L); + assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); + } + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 2L, + CHAR_KEYS.get("b"), 4L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void receiveCompletedStatisticsEvent(StatisticsType type) { + AggregatedStatisticsTracker tracker = createTracker(type); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b")); + + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + createStatisticsEvent( + type, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + + // Receive data statistics from all subtasks at checkpoint 1 + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + assertThat(completedStatistics.checkpointId()).isEqualTo(1L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 3L, + CHAR_KEYS.get("b"), 3L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + } + + StatisticsEvent checkpoint2Subtask0DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); + completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); + aggregation = tracker.aggregationsPerCheckpoint().get(2L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + } else { + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); + } + + StatisticsEvent checkpoint2Subtask1DataStatisticEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); + // Receive data statistics from all subtasks at checkpoint 2 + completedStatistics = + tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.checkpointId()).isEqualTo(2L); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo( + ImmutableMap.of( + CHAR_KEYS.get("a"), 1L, + CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + } + + @Test + public void coordinatorSwitchToSketchOverThreshold() { + int parallelism = 3; + int downstreamParallelism = 3; + int switchToSketchThreshold = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + switchToSketchThreshold, + null); + + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + // converted to sketch statistics as map size is 4 (over the switch threshold of 3) + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + @Test + public void coordinatorMapOperatorSketch() { + int parallelism = 3; + int downstreamParallelism = 3; + AggregatedStatisticsTracker tracker = + new AggregatedStatisticsTracker( + "testOperator", + parallelism, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + StatisticsType.Auto, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + + // first operator event has map statistics + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b")); + CompletedStatistics completedStatistics = + tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + AggregatedStatisticsTracker.Aggregation aggregation = + tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); + assertThat(aggregation.sketchStatistics()).isNull(); + assertThat(aggregation.mapStatistics()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + + // second operator event contains sketch statistics + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent( + StatisticsType.Sketch, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d")); + completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); + assertThat(completedStatistics).isNull(); + assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); + aggregation = tracker.aggregationsPerCheckpoint().get(1L); + assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); + assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); + assertThat(aggregation.mapStatistics()).isNull(); + assertThat(aggregation.sketchStatistics().getResult().getSamples()) + .containsExactlyInAnyOrder( + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); + + // third operator event has Map statistics + StatisticsEvent checkpoint1Subtask2StatisticsEvent = + createStatisticsEvent( + StatisticsType.Map, + TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); + assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); + assertThat(completedStatistics.keySamples()) + .containsExactly( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f")); + } + + private AggregatedStatisticsTracker createTracker(StatisticsType type) { + return new AggregatedStatisticsTracker( + "testOperator", + Fixtures.NUM_SUBTASKS, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + Fixtures.NUM_SUBTASKS, + type, + SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, + null); + } +} 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 new file mode 100644 index 000000000000..4ee9888934a8 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java @@ -0,0 +1,54 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestCompletedStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.COMPLETED_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return CompletedStatistics.class; + } + + @Override + protected CompletedStatistics[] getTestData() { + + return new CompletedStatistics[] { + CompletedStatistics.fromKeyFrequency( + 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), + CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} 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 new file mode 100644 index 000000000000..a08a8a73e80c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java @@ -0,0 +1,246 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.Duration; +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.util.ExceptionUtils; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestDataStatisticsCoordinator { + private static final String OPERATOR_NAME = "TestCoordinator"; + private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); + + private EventReceivingTasks receivingTasks; + + @BeforeEach + public void before() throws Exception { + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + private void tasksReady(DataStatisticsCoordinator coordinator) { + setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + String failureMessage = "The coordinator of TestCoordinator has not started yet."; + assertThatThrownBy( + () -> + dataStatisticsCoordinator.handleEventFromOperator( + 0, + 0, + StatisticsEvent.createTaskStatisticsEvent( + 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) + .isInstanceOf(IllegalStateException.class) + .hasMessage(failureMessage); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { + try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + type, + Fixtures.TASK_STATISTICS_SERIALIZER, + 1L, + CHAR_KEYS.get("a"), + 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, + CHAR_KEYS.get("a"), + 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( + CHAR_KEYS.get("a"), 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( + CHAR_KEYS.get("a"), + CHAR_KEYS.get("a"), + 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 = + createCoordinator(StatisticsType.Sketch)) { + dataStatisticsCoordinator.start(); + tasksReady(dataStatisticsCoordinator); + + // receive request before global statistics is ready + dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); + assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); + assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); + + StatisticsEvent checkpoint1Subtask0DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + StatisticsEvent checkpoint1Subtask1DataStatisticEvent = + Fixtures.createStatisticsEvent( + StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + // Handle events from operators for checkpoint 1 + dataStatisticsCoordinator.handleEventFromOperator( + 0, 0, checkpoint1Subtask0DataStatisticEvent); + dataStatisticsCoordinator.handleEventFromOperator( + 1, 0, checkpoint1Subtask1DataStatisticEvent); + + waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) + .isInstanceOf(StatisticsEvent.class); + + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + + dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); + + // coordinator should send a response to subtask 1 + Awaitility.await("wait for statistics event") + .pollInterval(Duration.ofMillis(10)) + .atMost(Duration.ofSeconds(10)) + .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) + .isInstanceOf(StatisticsEvent.class); + assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) + .isInstanceOf(StatisticsEvent.class); + } + } + + static void setAllTasksReady( + int subtasks, + DataStatisticsCoordinator dataStatisticsCoordinator, + EventReceivingTasks receivingTasks) { + for (int i = 0; i < subtasks; i++) { + dataStatisticsCoordinator.executionAttemptReady( + i, 0, receivingTasks.createGatewayForSubtask(i, 0)); + } + } + + static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { + CompletableFuture future = new CompletableFuture<>(); + coordinator.callInCoordinatorThread( + () -> { + future.complete(null); + return null; + }, + "Coordinator fails to process action"); + + try { + future.get(); + } catch (InterruptedException e) { + throw new AssertionError("test interrupted"); + } catch (ExecutionException e) { + ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); + } + } + + private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { + return new DataStatisticsCoordinator( + OPERATOR_NAME, + new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + NUM_SUBTASKS, + type, + 0.0d); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java new file mode 100644 index 000000000000..6317f2bfde18 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; +import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; +import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.EnumSource; + +public class TestDataStatisticsCoordinatorProvider { + private static final OperatorID OPERATOR_ID = new OperatorID(); + + private EventReceivingTasks receivingTasks; + + @BeforeEach + public void before() { + receivingTasks = EventReceivingTasks.createForRunningTasks(); + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testCheckpointAndReset(StatisticsType type) throws Exception { + DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); + try (RecreateOnResetOperatorCoordinator coordinator = + (RecreateOnResetOperatorCoordinator) + provider.create( + new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { + DataStatisticsCoordinator dataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + + // Start the coordinator + coordinator.start(); + TestDataStatisticsCoordinator.setAllTasksReady( + Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); + + // Handle events from operators for checkpoint 1 + StatisticsEvent checkpoint1Subtask0StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); + coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint1Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); + coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + // Verify checkpoint 1 global data statistics + Map checkpoint1KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); + MapAssignment checkpoint1MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + + CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics).isNotNull(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + + byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask0StatisticsEvent = + createStatisticsEvent( + type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); + coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + StatisticsEvent checkpoint2Subtask1StatisticsEvent = + createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); + coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); + TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); + + // Verify checkpoint 2 global data statistics + Map checkpoint2KeyFrequency = + ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); + MapAssignment checkpoint2MapAssignment = + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + completedStatistics = dataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); + } + + globalStatistics = dataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); + } + + waitForCheckpoint(2L, dataStatisticsCoordinator); + + // Reset coordinator to checkpoint 1 + coordinator.resetToCheckpoint(1L, checkpoint1Bytes); + DataStatisticsCoordinator restoredDataStatisticsCoordinator = + (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); + assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); + + completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); + assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + // Verify restored data statistics + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(completedStatistics.keyFrequency()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); + } else { + assertThat(completedStatistics.keySamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); + assertThat(globalStatistics).isNotNull(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); + } else { + assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); + } + } + } + + private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) + throws InterruptedException, ExecutionException { + CompletableFuture future = new CompletableFuture<>(); + coordinator.checkpointCoordinator(checkpointId, future); + return future.get(); + } + + private static DataStatisticsCoordinatorProvider createProvider( + StatisticsType type, int downstreamParallelism) { + return new DataStatisticsCoordinatorProvider( + "DataStatisticsCoordinatorProvider", + OPERATOR_ID, + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + downstreamParallelism, + type, + 0.0); + } +} 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 new file mode 100644 index 000000000000..bc248b778184 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java @@ -0,0 +1,352 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.verify; + +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.api.common.ExecutionConfig; +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.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; +import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; +import org.apache.flink.runtime.state.AbstractStateBackend; +import org.apache.flink.runtime.state.OperatorStateBackendParametersImpl; +import org.apache.flink.runtime.state.StateInitializationContext; +import org.apache.flink.runtime.state.StateInitializationContextImpl; +import org.apache.flink.runtime.state.TestTaskStateManager; +import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; +import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; +import org.apache.flink.streaming.util.MockOutput; +import org.apache.flink.streaming.util.MockStreamConfig; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +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.junit.jupiter.api.BeforeEach; +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.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mockito; + +public class TestDataStatisticsOperator { + + private Environment env; + + @BeforeEach + public void before() throws Exception { + this.env = + new StreamMockEnvironment( + new Configuration(), + new Configuration(), + new ExecutionConfig(), + 1L, + new MockInputSplitProvider(), + 1, + new TestTaskStateManager()); + } + + private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) + throws Exception { + MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); + return createOperator(type, downstreamParallelism, mockGateway); + } + + private DataStatisticsOperator createOperator( + StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) + throws Exception { + DataStatisticsOperator operator = + new DataStatisticsOperator( + "testOperator", + Fixtures.SCHEMA, + Fixtures.SORT_ORDER, + mockGateway, + downstreamParallelism, + type); + operator.setup( + new OneInputStreamTask(env), + new MockStreamConfig(new Configuration(), 1), + new MockOutput<>(Lists.newArrayList())); + return operator; + } + + @SuppressWarnings("unchecked") + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testProcessElement(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(StringData.fromString("a"), 5))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); + + DataStatistics localStatistics = operator.localStatistics(); + assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + Map keyFrequency = (Map) localStatistics.result(); + assertThat(keyFrequency) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); + } else { + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) localStatistics.result(); + assertThat(sketch.getSamples()) + .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); + } + + testHarness.endInput(); + } + } + + @ParameterizedTest + @EnumSource(StatisticsType.class) + public void testOperatorOutput(StatisticsType type) throws Exception { + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); + testHarness.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); + + List recordsOutput = + testHarness.extractOutputValues().stream() + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record) + .collect(Collectors.toList()); + assertThat(recordsOutput) + .containsExactlyInAnyOrderElementsOf( + ImmutableList.of( + GenericRowData.of(StringData.fromString("a"), 2), + GenericRowData.of(StringData.fromString("b"), 3), + GenericRowData.of(StringData.fromString("b"), 1))); + } + } + + private static Stream provideRestoreStateParameters() { + return Stream.of( + Arguments.of(StatisticsType.Map, -1), + Arguments.of(StatisticsType.Map, 0), + Arguments.of(StatisticsType.Map, 1), + Arguments.of(StatisticsType.Sketch, -1), + Arguments.of(StatisticsType.Sketch, 0), + Arguments.of(StatisticsType.Sketch, 1)); + } + + @ParameterizedTest + @MethodSource("provideRestoreStateParameters") + public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { + Map keyFrequency = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); + SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); + DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); + OperatorSubtaskState snapshot; + try (OneInputStreamOperatorTestHarness testHarness1 = + createHarness(operator)) { + GlobalStatistics statistics; + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); + } else { + statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); + } + + StatisticsEvent event = + StatisticsEvent.createGlobalStatisticsEvent( + statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); + operator.handleOperatorEvent(event); + + GlobalStatistics globalStatistics = operator.globalStatistics(); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + + snapshot = testHarness1.snapshot(1L, 0); + } + + // Use the snapshot to initialize state for another new operator and then verify that the global + // statistics for the new operator is same as before + MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); + DataStatisticsOperator restoredOperator = + createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); + try (OneInputStreamOperatorTestHarness testHarness2 = + new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { + testHarness2.setup(); + testHarness2.initializeState(snapshot); + + GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); + // global statistics is always restored and used initially even if + // downstream parallelism changed. + assertThat(globalStatistics).isNotNull(); + // request is always sent to coordinator during initialization. + // coordinator would respond with a new global statistics that + // has range bound recomputed with new parallelism. + verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); + assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); + if (StatisticsUtil.collectType(type) == StatisticsType.Map) { + assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); + assertThat(globalStatistics.rangeBounds()).isNull(); + } else { + assertThat(globalStatistics.mapAssignment()).isNull(); + assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); + } + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithLocalStatsOverThreshold() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // add rows with unique keys + for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()).hasSize(i + 1); + } + + // one more item should trigger the migration to sketch stats + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); + // reservoir not full yet + assertThat(sketch.getN()).isLessThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); + + // add more items to saturate the reservoir + for (int i = 0; i < reservoirSize; ++i) { + operator.processElement( + new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); + } + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + sketch = (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()) + .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); + // reservoir is full now + assertThat(sketch.getN()).isGreaterThan(reservoirSize); + assertThat(sketch.getSamples()).hasSize(reservoirSize); + + testHarness.endInput(); + } + } + + @SuppressWarnings("unchecked") + @Test + public void testMigrationWithGlobalSketchStatistics() throws Exception { + DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); + try (OneInputStreamOperatorTestHarness testHarness = + createHarness(operator)) { + StateInitializationContext stateContext = getStateContext(); + operator.initializeState(stateContext); + + // started with Map stype + operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); + assertThat((Map) operator.localStatistics().result()) + .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); + + // received global statistics with sketch type + GlobalStatistics globalStatistics = + GlobalStatistics.fromRangeBounds( + 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); + operator.handleOperatorEvent( + StatisticsEvent.createGlobalStatisticsEvent( + globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); + + int reservoirSize = + SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); + + assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); + ReservoirItemsSketch sketch = + (ReservoirItemsSketch) operator.localStatistics().result(); + assertThat(sketch.getK()).isEqualTo(reservoirSize); + assertThat(sketch.getN()).isEqualTo(1); + assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); + + testHarness.endInput(); + } + } + + private StateInitializationContext getStateContext() throws Exception { + AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); + CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); + OperatorStateStore operatorStateStore = + abstractStateBackend.createOperatorStateBackend( + new OperatorStateBackendParametersImpl( + env, "test-operator", Collections.emptyList(), cancelStreamRegistry)); + return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); + } + + private OneInputStreamOperatorTestHarness createHarness( + DataStatisticsOperator dataStatisticsOperator) throws Exception { + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>( + dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); + harness.setup( + new StatisticsOrRecordSerializer( + Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); + harness.open(); + return harness; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java new file mode 100644 index 000000000000..59ce6df05d9d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +public class TestDataStatisticsSerializer extends SerializerTestBase { + @Override + protected TypeSerializer createSerializer() { + return Fixtures.TASK_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return DataStatistics.class; + } + + @Override + protected DataStatistics[] getTestData() { + return new DataStatistics[] { + new MapDataStatistics(), + Fixtures.createTaskStatistics( + StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), + new SketchDataStatistics(128), + Fixtures.createTaskStatistics( + StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java new file mode 100644 index 000000000000..7afaf239c668 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.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.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; + +public class TestGlobalStatisticsSerializer extends SerializerTestBase { + + @Override + protected TypeSerializer createSerializer() { + return Fixtures.GLOBAL_STATISTICS_SERIALIZER; + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return GlobalStatistics.class; + } + + @Override + protected GlobalStatistics[] getTestData() { + return new GlobalStatistics[] { + GlobalStatistics.fromMapAssignment( + 1L, + MapAssignment.fromKeyFrequency( + Fixtures.NUM_SUBTASKS, + ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), + 0.0d, + SORT_ORDER_COMPARTOR)), + GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) + }; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java new file mode 100644 index 000000000000..8a25c7ad9898 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.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.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.Test; + +public class TestMapDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + MapDataStatistics dataStatistics = new MapDataStatistics(); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("a")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + Map actual = (Map) dataStatistics.result(); + Map expected = + ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java new file mode 100644 index 000000000000..d5a0bebc74e7 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -0,0 +1,434 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +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.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.util.Pair; +import org.junit.jupiter.api.Test; + +public class TestMapRangePartitioner { + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); + + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final SortKey[] SORT_KEYS = initSortKeys(); + + private static SortKey[] initSortKeys() { + SortKey[] sortKeys = new SortKey[10]; + for (int i = 0; i < 10; ++i) { + RowData rowData = + GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = SORT_KEY.copy(); + sortKey.wrap(keyWrapper); + sortKeys[i] = sortKey; + } + return sortKeys; + } + + // Total weight is 800 + private final Map mapStatistics = + ImmutableMap.of( + SORT_KEYS[0], + 350L, + SORT_KEYS[1], + 230L, + SORT_KEYS[2], + 120L, + SORT_KEYS[3], + 40L, + SORT_KEYS[4], + 10L, + SORT_KEYS[5], + 10L, + SORT_KEYS[6], + 10L, + SORT_KEYS[7], + 10L, + SORT_KEYS[8], + 10L, + SORT_KEYS[9], + 10L); + + @Test + public void testEvenlyDividableNoClosingFileCost() { + int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); + + // each task should get targeted weight of 100 (=800/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), + SORT_KEYS[1], + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), + SORT_KEYS[2], + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), + SORT_KEYS[3], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), + SORT_KEYS[4], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(100L, 1), + 1, + Pair.of(100L, 1), + 2, + Pair.of(100L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(100L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 1), + 7, + Pair.of(100L, 7)); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); + Map>> partitionResults = + runPartitioner(partitioner, numPartitions, mapStatistics); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testEvenlyDividableWithClosingFileCost() { + int numPartitions = 8; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); + + // target subtask weight is 100 before close file cost factored in. + // close file cost is 5 = 5% * 100. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight with close cost per subtask is 110 (880/8) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), + SORT_KEYS[1], + new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), + SORT_KEYS[2], + new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), + SORT_KEYS[3], + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), + SORT_KEYS[4], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight (excluding close file cost) for the subtask, + // second is the number of keys assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(105L, 1), + 1, + Pair.of(105L, 1), + 2, + Pair.of(105L, 1), + 3, + Pair.of(100L, 2), + 4, + Pair.of(105L, 1), + 5, + Pair.of(100L, 2), + 6, + Pair.of(100L, 2), + 7, + Pair.of(75L, 7)); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); + Map>> partitionResults = + runPartitioner(partitioner, numPartitions, mapStatistics); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableNoClosingFileCost() { + int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); + + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // each task should get targeted weight of 89 = ceiling(800/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), + SORT_KEYS[1], + new KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), + SORT_KEYS[2], + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), + SORT_KEYS[3], + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), + SORT_KEYS[4], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[5], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[6], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[7], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[8], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), + SORT_KEYS[9], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight, second is the number of assigned keys + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(89L, 1), + 1, + Pair.of(89L, 1), + 2, + Pair.of(89L, 1), + 3, + Pair.of(89L, 2), + 4, + Pair.of(89L, 1), + 5, + Pair.of(89L, 1), + 6, + Pair.of(89L, 2), + 7, + Pair.of(89L, 2), + 8, + Pair.of(88L, 7)); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); + Map>> partitionResults = + runPartitioner(partitioner, numPartitions, mapStatistics); + validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); + } + + @Test + public void testNonDividableWithClosingFileCost() { + int numPartitions = 9; + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); + + // target subtask weight is 89 before close file cost factored in. + // close file cost is 5 (= 5% * 89) per file. + // key weights before and after close file cost factored in + // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 + // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 + // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 + // target subtask weight per subtask is 98 ceiling(880/9) + Map expectedAssignment = + ImmutableMap.of( + SORT_KEYS[0], + new KeyAssignment( + ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), + SORT_KEYS[1], + new KeyAssignment( + ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), + SORT_KEYS[2], + new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), + SORT_KEYS[3], + new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), + SORT_KEYS[4], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[5], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[6], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[7], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[8], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), + SORT_KEYS[9], + new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); + assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); + + // key: subtask id + // value pair: first is the assigned weight for the subtask, second is the number of keys + // assigned to the subtask + Map> expectedAssignmentInfo = + ImmutableMap.of( + 0, + Pair.of(93L, 1), + 1, + Pair.of(93L, 1), + 2, + Pair.of(93L, 1), + 3, + Pair.of(88L, 2), + 4, + Pair.of(93L, 1), + 5, + Pair.of(93L, 1), + 6, + Pair.of(88L, 2), + 7, + Pair.of(88L, 2), + 8, + Pair.of(61L, 7)); + assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); + + MapRangePartitioner partitioner = + new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); + Map>> partitionResults = + runPartitioner(partitioner, numPartitions, mapStatistics); + // drift threshold is high for non-dividable scenario with close cost + validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); + } + + private static Map>> runPartitioner( + MapRangePartitioner partitioner, int numPartitions, Map mapStatistics) { + // The Map key is the subtaskId. + // For the map value pair, the first element is the count of assigned and + // the second element of Set is for the set of assigned keys. + Map>> partitionResults = Maps.newHashMap(); + mapStatistics.forEach( + (sortKey, weight) -> { + String key = sortKey.get(0, String.class); + // run 100x times of the weight + long iterations = weight * 100; + for (int i = 0; i < iterations; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString(key), 1, StringData.fromString("2023-06-20")); + int subtaskId = partitioner.partition(rowData, numPartitions); + partitionResults.computeIfAbsent( + subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); + Pair> pair = partitionResults.get(subtaskId); + pair.first().incrementAndGet(); + pair.second().add(rowData); + } + }); + return partitionResults; + } + + /** @param expectedAssignmentInfo excluding closing cost */ + private void validatePartitionResults( + Map> expectedAssignmentInfo, + Map>> partitionResults, + double maxDriftPercentage) { + + assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); + + List expectedAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualAssignedKeyCounts = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + List expectedNormalizedWeights = + Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); + List actualNormalizedWeights = + Lists.newArrayListWithExpectedSize(partitionResults.size()); + + long expectedTotalWeight = + expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); + expectedAssignmentInfo.forEach( + (subtaskId, pair) -> { + expectedAssignedKeyCounts.add(pair.second()); + expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); + }); + + long actualTotalWeight = + partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); + partitionResults.forEach( + (subtaskId, pair) -> { + actualAssignedKeyCounts.add(pair.second().size()); + actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); + }); + + // number of assigned keys should match exactly + assertThat(actualAssignedKeyCounts) + .as("the number of assigned keys should match for every subtask") + .isEqualTo(expectedAssignedKeyCounts); + + // weight for every subtask shouldn't differ for more than some threshold relative to the + // expected weight + for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { + double expectedWeight = expectedNormalizedWeights.get(subtaskId); + double min = expectedWeight * (1 - maxDriftPercentage / 100); + double max = expectedWeight * (1 + maxDriftPercentage / 100); + assertThat(actualNormalizedWeights.get(subtaskId)) + .as( + "Subtask %d weight should within %.1f percent of the expected range %s", + subtaskId, maxDriftPercentage, expectedWeight) + .isBetween(min, max); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java new file mode 100644 index 000000000000..0485fdb7fa04 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Set; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.junit.jupiter.api.Test; + +public class TestRangePartitioner { + private final int numPartitions = 4; + + @Test + public void testRoundRobinRecordsBeforeStatisticsAvailable() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + results.add( + partitioner.partition( + StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), + numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } + + @Test + public void testRoundRobinStatisticsWrapper() { + RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); + Set results = Sets.newHashSetWithExpectedSize(numPartitions); + for (int i = 0; i < numPartitions; ++i) { + GlobalStatistics statistics = + GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); + results.add( + partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); + } + + // round-robin. every partition should get an assignment + assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java new file mode 100644 index 000000000000..396bfae2f13c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.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.flink.sink.shuffle; + +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.datasketches.sampling.ReservoirItemsSketch; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; + +public class TestSketchDataStatistics { + @SuppressWarnings("unchecked") + @Test + public void testAddsAndGet() { + SketchDataStatistics dataStatistics = new SketchDataStatistics(128); + + GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("c")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + reusedRow.setField(0, StringData.fromString("b")); + Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); + dataStatistics.add(Fixtures.SORT_KEY); + + ReservoirItemsSketch actual = (ReservoirItemsSketch) dataStatistics.result(); + assertThat(actual.getSamples()) + .isEqualTo( + new SortKey[] { + CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") + }); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java new file mode 100644 index 000000000000..378c6afff077 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.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.flink.sink.shuffle; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TestFixtures; +import org.junit.jupiter.api.Test; + +public class TestSketchRangePartitioner { + // sort on the long id field + private static final SortOrder SORT_ORDER = + SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); + private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final int NUM_PARTITIONS = 16; + private static final long RANGE_STEP = 1_000; + private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; + private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); + + /** + * To understand how range bounds are used in range partitioning, here is an example for human + * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be + * + *
      + *
    • age <= 15 + *
    • age > 15 && age <= 32 + *
    • age >32 && age <= 60 + *
    • age > 60 + *
    + */ + private static SortKey[] createRangeBounds() { + SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; + for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { + RowData rowData = + GenericRowData.of( + StringData.fromString("data"), + RANGE_STEP * (i + 1), + StringData.fromString("2023-06-20")); + RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); + keyWrapper.wrap(rowData); + SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); + sortKey.wrap(keyWrapper); + rangeBounds[i] = sortKey; + } + + return rangeBounds; + } + + @Test + public void testRangePartitioningWithRangeBounds() { + SketchRangePartitioner partitioner = + new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); + GenericRowData row = + GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); + for (long id = 0; id < MAX_ID; ++id) { + row.setField(1, id); + int partition = partitioner.partition(row, NUM_PARTITIONS); + assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); + int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); + assertThat(partition).isEqualTo(expectedPartition); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java new file mode 100644 index 000000000000..16202c075ea0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java @@ -0,0 +1,189 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.SortKey; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestSketchUtil { + @Test + public void testCoordinatorReservoirSize() { + // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); + // adjust to multiplier of 100 + assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); + // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) + assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) + .isEqualTo(1_000_000 - (1_000_000 % 10_123)); + } + + @Test + public void testOperatorReservoirSize() { + assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) + .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); + assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) + .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); + assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) + .isEqualTo( + (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); + assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) + .isEqualTo( + (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); + } + + @Test + public void testRangeBoundsOneChannel() { + assertThat( + SketchUtil.rangeBounds( + 1, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .isEmpty(); + } + + @Test + public void testRangeBoundsDivisible() { + assertThat( + SketchUtil.rangeBounds( + 3, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f") + })) + .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); + } + + @Test + public void testRangeBoundsNonDivisible() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("d"), + CHAR_KEYS.get("e"), + CHAR_KEYS.get("f"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); + } + + @Test + public void testRangeBoundsSkipDuplicates() { + // step is 3 = ceiling(11/4) + assertThat( + SketchUtil.rangeBounds( + 4, + SORT_ORDER_COMPARTOR, + new SortKey[] { + CHAR_KEYS.get("a"), + CHAR_KEYS.get("b"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("c"), + CHAR_KEYS.get("g"), + CHAR_KEYS.get("h"), + CHAR_KEYS.get("i"), + CHAR_KEYS.get("j"), + CHAR_KEYS.get("k"), + })) + // skipped duplicate c's + .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); + } + + @ParameterizedTest + @ValueSource(ints = {4, 6}) + public void testPartitioningAndScaleUp(int numPartitions) { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + @Test + public void testPartitionScaleDown() { + // Range bounds are calculated based on 4 partitions + SortKey[] rangeBounds = + new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; + int numPartitions = 3; + + // <= c + assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); + // > c && <= j + assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); + assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); + // > j && <= m + assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); + assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); + // > m + // reassigns out-of-range partitions via mod (% 3 in this case) + assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); + assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); + } + + private static void assertPartition( + int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { + assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) + .isEqualTo(expectedPartition); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java new file mode 100644 index 000000000000..c7fea015142c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.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.flink.sink.shuffle; + +import org.apache.flink.api.common.typeutils.SerializerTestBase; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; + +public abstract class TestSortKeySerializerBase extends SerializerTestBase { + + protected abstract Schema schema(); + + protected abstract SortOrder sortOrder(); + + protected abstract GenericRowData rowData(); + + @Override + protected TypeSerializer createSerializer() { + return new SortKeySerializer(schema(), sortOrder()); + } + + @Override + protected int getLength() { + return -1; + } + + @Override + protected Class getTypeClass() { + return SortKey.class; + } + + @Override + protected SortKey[] getTestData() { + return new SortKey[] {sortKey()}; + } + + private SortKey sortKey() { + RowDataWrapper rowDataWrapper = + new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); + SortKey sortKey = new SortKey(schema(), sortOrder()); + sortKey.wrap(rowDataWrapper.wrap(rowData())); + return sortKey; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java new file mode 100644 index 000000000000..0000688a8b55 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java @@ -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. + */ +package org.apache.iceberg.flink.sink.shuffle; + +import org.apache.flink.table.data.GenericRowData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; + +public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.StructOfStruct(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("row_id") + .sortBy( + Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy( + Expressions.truncate("struct_of_struct.person_struct.name", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } +} 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 new file mode 100644 index 000000000000..54cceae6e55b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java @@ -0,0 +1,90 @@ +/* + * 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 static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.DataGenerator; +import org.apache.iceberg.flink.DataGenerators; +import org.apache.iceberg.flink.RowDataWrapper; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { + private final DataGenerator generator = new DataGenerators.Primitives(); + + @Override + protected Schema schema() { + return generator.icebergSchema(); + } + + @Override + protected SortOrder sortOrder() { + return SortOrder.builderFor(schema()) + .asc("boolean_field") + .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) + .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) + // can not test HeapByteBuffer due to equality test inside SerializerTestBase + // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, + // NullOrder.NULLS_FIRST) + .build(); + } + + @Override + protected GenericRowData rowData() { + return generator.generateFlinkRowData(); + } + + @Test + public void testSerializationSize() throws Exception { + RowData rowData = + GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); + RowDataWrapper rowDataWrapper = + new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); + StructLike struct = rowDataWrapper.wrap(rowData); + SortKey sortKey = Fixtures.SORT_KEY.copy(); + sortKey.wrap(struct); + SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); + DataOutputSerializer output = new DataOutputSerializer(1024); + serializer.serialize(sortKey, output); + 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); + + DataInputDeserializer input = new DataInputDeserializer(serializedBytes); + SortKey deserialized = serializer.deserialize(input); + assertThat(deserialized).isEqualTo(sortKey); + } +} 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 new file mode 100644 index 000000000000..c0f688f2589e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java @@ -0,0 +1,213 @@ +/* + * 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 static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; +import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; +import static org.assertj.core.api.AssertionsForClassTypes.assertThat; + +import java.io.IOException; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputSerializer; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortKey; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeySerializerSnapshot { + private final Schema schema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); + + @Test + public void testRestoredSerializer() 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); + TypeSerializerSnapshot snapshot = + roundTrip(originalSerializer.snapshotConfiguration()); + TypeSerializer restoredSerializer = snapshot.restoreSerializer(); + + 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 = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed non-sort boolean field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // add a new non-sort float field + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.StringType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), + Types.NestedField.required(5, "float", Types.FloatType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // change str field to a long type + Schema newSchema = + new Schema( + Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(2, "str", Types.LongType.get()), + Types.NestedField.optional(3, "int", Types.IntegerType.get()), + Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); + SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); + // switch sort field order + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(newSchema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // removed str field from sort order + SortOrder newSortOrder = + SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // remove str field from sort order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + @Test + public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { + SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = + new SortKeySerializer.SortKeySerializerSnapshot(schema, sortOrder); + + // switch sort field order + SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); + SortKeySerializer.SortKeySerializerSnapshot newSnapshot = + roundTrip(new SortKeySerializer.SortKeySerializerSnapshot(schema, newSortOrder)); + + TypeSerializerSchemaCompatibility resultCompatibility = + newSnapshot.resolveSchemaCompatibility(oldSnapshot); + assertThat(resultCompatibility.isIncompatible()).isTrue(); + } + + /** Copied from Flink {@code AvroSerializerSnapshotTest} */ + private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( + TypeSerializerSnapshot original) throws IOException { + // writeSnapshot(); + DataOutputSerializer out = new DataOutputSerializer(1024); + original.writeSnapshot(out); + // init + SortKeySerializer.SortKeySerializerSnapshot restored = + new SortKeySerializer.SortKeySerializerSnapshot(); + // readSnapshot(); + DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); + restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); + return restored; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java new file mode 100644 index 000000000000..1be7e27f2c01 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java @@ -0,0 +1,73 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import org.apache.iceberg.NullOrder; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortDirection; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSortKeyUtil { + @Test + public void testResultSchema() { + Schema schema = + new Schema( + Types.NestedField.required(1, "id", Types.StringType.get()), + Types.NestedField.required(2, "ratio", Types.DoubleType.get()), + Types.NestedField.optional( + 3, + "user", + Types.StructType.of( + Types.NestedField.required(11, "name", Types.StringType.get()), + Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), + Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), + Types.NestedField.optional( + 14, + "location", + Types.StructType.of( + Types.NestedField.required(101, "lat", Types.FloatType.get()), + Types.NestedField.required(102, "long", Types.FloatType.get()), + Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); + + SortOrder sortOrder = + SortOrder.builderFor(schema) + .asc("ratio") + .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) + .sortBy( + Expressions.truncate("user.location.blob", 16), + SortDirection.ASC, + NullOrder.NULLS_FIRST) + .build(); + + assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) + .isEqualTo( + Types.StructType.of( + Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), + Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), + Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), + Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java new file mode 100644 index 000000000000..a08578a4c106 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java @@ -0,0 +1,170 @@ +/* + * 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.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Supplier; +import java.util.stream.Stream; +import org.apache.flink.api.java.typeutils.RowTypeInfo; +import org.apache.flink.configuration.ConfigOption; +import org.apache.flink.configuration.ConfigOptions; +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.streaming.api.functions.source.SourceFunction; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.connector.ChangelogMode; +import org.apache.flink.table.connector.ProviderContext; +import org.apache.flink.table.connector.source.DataStreamScanProvider; +import org.apache.flink.table.connector.source.DynamicTableSource; +import org.apache.flink.table.connector.source.ScanTableSource; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.factories.DynamicTableSourceFactory; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.utils.TableSchemaUtils; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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; + +public class BoundedTableFactory implements DynamicTableSourceFactory { + private static final AtomicInteger DATA_SET_ID = new AtomicInteger(0); + private static final Map>> DATA_SETS = Maps.newHashMap(); + + private static final ConfigOption DATA_ID = + ConfigOptions.key("data-id").stringType().noDefaultValue(); + + public static String registerDataSet(List> dataSet) { + String dataSetId = String.valueOf(DATA_SET_ID.incrementAndGet()); + DATA_SETS.put(dataSetId, dataSet); + return dataSetId; + } + + public static void clearDataSets() { + DATA_SETS.clear(); + } + + @Override + public DynamicTableSource createDynamicTableSource(Context context) { + TableSchema tableSchema = + TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); + + Configuration configuration = Configuration.fromMap(context.getCatalogTable().getOptions()); + String dataId = configuration.getString(DATA_ID); + Preconditions.checkArgument( + DATA_SETS.containsKey(dataId), "data-id %s does not found in registered data set.", dataId); + + return new BoundedTableSource(DATA_SETS.get(dataId), tableSchema); + } + + @Override + public String factoryIdentifier() { + return "BoundedSource"; + } + + @Override + public Set> requiredOptions() { + return ImmutableSet.of(); + } + + @Override + public Set> optionalOptions() { + return ImmutableSet.of(DATA_ID); + } + + private static class BoundedTableSource implements ScanTableSource { + + private final List> elementsPerCheckpoint; + private final TableSchema tableSchema; + + private BoundedTableSource(List> elementsPerCheckpoint, TableSchema tableSchema) { + this.elementsPerCheckpoint = elementsPerCheckpoint; + this.tableSchema = tableSchema; + } + + private BoundedTableSource(BoundedTableSource toCopy) { + this.elementsPerCheckpoint = toCopy.elementsPerCheckpoint; + this.tableSchema = toCopy.tableSchema; + } + + @Override + public ChangelogMode getChangelogMode() { + Supplier> supplier = () -> elementsPerCheckpoint.stream().flatMap(List::stream); + + // Add the INSERT row kind by default. + ChangelogMode.Builder builder = ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT); + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.DELETE)) { + builder.addContainedKind(RowKind.DELETE); + } + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_BEFORE)) { + builder.addContainedKind(RowKind.UPDATE_BEFORE); + } + + if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_AFTER)) { + builder.addContainedKind(RowKind.UPDATE_AFTER); + } + + return builder.build(); + } + + @Override + public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { + return new DataStreamScanProvider() { + @Override + public DataStream produceDataStream( + ProviderContext providerContext, StreamExecutionEnvironment env) { + boolean checkpointEnabled = env.getCheckpointConfig().isCheckpointingEnabled(); + SourceFunction source = + new BoundedTestSource<>(elementsPerCheckpoint, checkpointEnabled); + + RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); + // Converter to convert the Row to RowData. + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); + + return env.addSource(source, new RowTypeInfo(tableSchema.getFieldTypes())) + .map(rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)); + } + + @Override + public boolean isBounded() { + return true; + } + }; + } + + @Override + public DynamicTableSource copy() { + return new BoundedTableSource(this); + } + + @Override + public String asSummaryString() { + return "Bounded test table source"; + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java new file mode 100644 index 000000000000..7b435d059845 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java @@ -0,0 +1,108 @@ +/* + * 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.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.state.CheckpointListener; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * A stream source that: 1) emits the elements from elementsPerCheckpoint.get(0) without allowing + * checkpoints. 2) then waits for the checkpoint to complete. 3) emits the elements from + * elementsPerCheckpoint.get(1) without allowing checkpoints. 4) then waits for the checkpoint to + * complete. 5) ... + * + *

    Util all the list from elementsPerCheckpoint are exhausted. + */ +public final class BoundedTestSource implements SourceFunction, CheckpointListener { + + private final List> elementsPerCheckpoint; + private final boolean checkpointEnabled; + private volatile boolean running = true; + + private final AtomicInteger numCheckpointsComplete = new AtomicInteger(0); + + /** Emits all those elements in several checkpoints. */ + public BoundedTestSource(List> elementsPerCheckpoint, boolean checkpointEnabled) { + this.elementsPerCheckpoint = elementsPerCheckpoint; + this.checkpointEnabled = checkpointEnabled; + } + + public BoundedTestSource(List> elementsPerCheckpoint) { + this(elementsPerCheckpoint, true); + } + + /** Emits all those elements in a single checkpoint. */ + public BoundedTestSource(T... elements) { + this(Collections.singletonList(Arrays.asList(elements))); + } + + @Override + public void run(SourceContext ctx) throws Exception { + if (!checkpointEnabled) { + Preconditions.checkArgument( + elementsPerCheckpoint.size() <= 1, + "There should be at most one list in the elementsPerCheckpoint when checkpoint is disabled."); + elementsPerCheckpoint.stream().flatMap(List::stream).forEach(ctx::collect); + return; + } + + for (List elements : elementsPerCheckpoint) { + + final int checkpointToAwait; + synchronized (ctx.getCheckpointLock()) { + // Let's say checkpointToAwait = numCheckpointsComplete.get() + delta, in fact the value of + // delta should not + // affect the final table records because we only need to make sure that there will be + // exactly + // elementsPerCheckpoint.size() checkpoints to emit each records buffer from the original + // elementsPerCheckpoint. + // Even if the checkpoints that emitted results are not continuous, the correctness of the + // data should not be + // affected in the end. Setting the delta to be 2 is introducing the variable that produce + // un-continuous + // checkpoints that emit the records buffer from elementsPerCheckpoints. + checkpointToAwait = numCheckpointsComplete.get() + 2; + for (T element : elements) { + ctx.collect(element); + } + } + + synchronized (ctx.getCheckpointLock()) { + while (running && numCheckpointsComplete.get() < checkpointToAwait) { + ctx.getCheckpointLock().wait(1); + } + } + } + } + + @Override + public void notifyCheckpointComplete(long checkpointId) throws Exception { + numCheckpointsComplete.incrementAndGet(); + } + + @Override + public void cancel() { + running = false; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java new file mode 100644 index 000000000000..5dfbbe3abe73 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java @@ -0,0 +1,95 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.types.Row; +import org.apache.flink.types.RowKind; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestInfo; + +public class ChangeLogTableTestBase extends TestBase { + private volatile TableEnvironment tEnv = null; + + protected String tableName; + + @BeforeEach + public void setup(TestInfo testInfo) { + assertThat(testInfo.getTestMethod()).isPresent(); + this.tableName = testInfo.getTestMethod().get().getName(); + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s", tableName); + BoundedTableFactory.clearDataSets(); + } + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings settings = + EnvironmentSettings.newInstance().inStreamingMode().build(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(400) + .setMaxParallelism(1) + .setParallelism(1); + + tEnv = StreamTableEnvironment.create(env, settings); + } + } + } + return tEnv; + } + + protected static Row insertRow(Object... values) { + return Row.ofKind(RowKind.INSERT, values); + } + + protected static Row deleteRow(Object... values) { + return Row.ofKind(RowKind.DELETE, values); + } + + protected static Row updateBeforeRow(Object... values) { + return Row.ofKind(RowKind.UPDATE_BEFORE, values); + } + + protected static Row updateAfterRow(Object... values) { + return Row.ofKind(RowKind.UPDATE_AFTER, values); + } + + protected static List listJoin(List> lists) { + return lists.stream().flatMap(List::stream).collect(Collectors.toList()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java new file mode 100644 index 000000000000..540902f3cea5 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java @@ -0,0 +1,200 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.spy; + +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.hadoop.conf.Configuration; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.hadoop.HadoopCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.ThreadPools; + +public class SplitHelpers { + + private SplitHelpers() {} + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + *

    By default, v1 Iceberg table is created. For v2 table use {@link + * SplitHelpers#createSplitsFromTransientHadoopTable(Path, int, int, String)} + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + */ + public static List createSplitsFromTransientHadoopTable( + Path temporaryFolder, int fileCount, int filesPerSplit) throws Exception { + return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); + } + + /** + * This create a list of IcebergSourceSplit from real files + *

  • Create a new Hadoop table under the {@code temporaryFolder} + *
  • write {@code fileCount} number of files to the new Iceberg table + *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} + * limit + *
  • Delete the Hadoop table + * + *

    Since the table and data files are deleted before this method return, caller shouldn't + * attempt to read the data files. + * + * @param temporaryFolder Folder to place the data to + * @param fileCount The number of files to create and add to the table + * @param filesPerSplit The number of files used for a split + * @param version The table version to create + */ + public static List createSplitsFromTransientHadoopTable( + Path temporaryFolder, int fileCount, int filesPerSplit, String version) throws Exception { + final File warehouseFile = File.createTempFile("junit", null, temporaryFolder.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + final String warehouse = "file:" + warehouseFile; + Configuration hadoopConf = new Configuration(); + final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); + ImmutableMap properties = + ImmutableMap.of(TableProperties.FORMAT_VERSION, version); + try { + final Table table = + catalog.createTable( + TestFixtures.TABLE_IDENTIFIER, + TestFixtures.SCHEMA, + PartitionSpec.unpartitioned(), + null, + properties); + final GenericAppenderHelper dataAppender = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + for (int i = 0; i < fileCount; ++i) { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); + dataAppender.appendToTable(records); + } + + final ScanContext scanContext = ScanContext.builder().build(); + final List splits = + FlinkSplitPlanner.planIcebergSourceSplits( + table, scanContext, ThreadPools.getWorkerPool()); + return splits.stream() + .flatMap( + split -> { + List> filesList = + Lists.partition(Lists.newArrayList(split.task().files()), filesPerSplit); + return filesList.stream() + .map(files -> new BaseCombinedScanTask(files)) + .map( + combinedScanTask -> + IcebergSourceSplit.fromCombinedScanTask(combinedScanTask)); + }) + .collect(Collectors.toList()); + } finally { + catalog.dropTable(TestFixtures.TABLE_IDENTIFIER); + catalog.close(); + } + } + + /** + * This method will equip the {@code icebergSourceSplits} with mock delete files. + *

  • For each split, create {@code deleteFilesPerSplit} number of delete files + *
  • Replace the original {@code FileScanTask} with the new {@code FileScanTask} with mock + *
  • Caller should not attempt to read the deleted files since they are created as mock, and + * they are not real files + * + * @param icebergSourceSplits The real splits to equip with mock delete files + * @param temporaryFolder The temporary folder to create the mock delete files with + * @param deleteFilesPerSplit The number of delete files to create for each split + * @return The list of re-created splits with mock delete files + * @throws IOException If there is any error creating the mock delete files + */ + public static List equipSplitsWithMockDeleteFiles( + List icebergSourceSplits, Path temporaryFolder, int deleteFilesPerSplit) + throws IOException { + List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); + for (IcebergSourceSplit split : icebergSourceSplits) { + final CombinedScanTask combinedScanTask = spy(split.task()); + + final List deleteFiles = Lists.newArrayList(); + final PartitionSpec spec = + PartitionSpec.builderFor(TestFixtures.SCHEMA).withSpecId(0).build(); + + for (int i = 0; i < deleteFilesPerSplit; ++i) { + final DeleteFile deleteFile = + FileMetadata.deleteFileBuilder(spec) + .withFormat(FileFormat.PARQUET) + .withPath(File.createTempFile("junit", null, temporaryFolder.toFile()).getPath()) + .ofPositionDeletes() + .withFileSizeInBytes(1000) + .withRecordCount(1000) + .build(); + deleteFiles.add(deleteFile); + } + + List newFileScanTasks = Lists.newArrayList(); + for (FileScanTask task : combinedScanTask.tasks()) { + String schemaString = SchemaParser.toJson(task.schema()); + String specString = PartitionSpecParser.toJson(task.spec()); + + BaseFileScanTask baseFileScanTask = + new BaseFileScanTask( + task.file(), + deleteFiles.toArray(new DeleteFile[] {}), + schemaString, + specString, + ResidualEvaluator.unpartitioned(task.residual())); + newFileScanTasks.add(baseFileScanTask); + } + doReturn(newFileScanTasks).when(combinedScanTask).tasks(); + icebergSourceSplitsWithMockDeleteFiles.add( + IcebergSourceSplit.fromCombinedScanTask( + combinedScanTask, split.fileOffset(), split.recordOffset())); + } + return icebergSourceSplitsWithMockDeleteFiles; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java new file mode 100644 index 000000000000..e4e48ca67f66 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.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.flink.source; + +import java.util.List; +import java.util.Map; +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.iceberg.relocated.com.google.common.collect.Lists; + +public class SqlHelpers { + private SqlHelpers() {} + + public static List sql(TableEnvironment tableEnv, String query, Object... args) { + TableResult tableResult = tableEnv.executeSql(String.format(query, args)); + try (CloseableIterator iter = tableResult.collect()) { + List results = Lists.newArrayList(iter); + return results; + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + public static String sqlOptionsToString(Map sqlOptions) { + StringBuilder builder = new StringBuilder(); + sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(",")); + String optionStr = builder.toString(); + if (optionStr.endsWith(",")) { + optionStr = optionStr.substring(0, optionStr.length() - 1); + } + + if (!optionStr.isEmpty()) { + optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr); + } + + return optionStr; + } + + private static String optionToKv(String key, Object value) { + return "'" + key + "'='" + value + "'"; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java new file mode 100644 index 000000000000..f89d63ac73e3 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java @@ -0,0 +1,105 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.io.IOException; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.events.Listeners; +import org.apache.iceberg.events.ScanEvent; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TableSourceTestBase extends TestBase { + @Parameters(name = "useFlip27Source = {0}") + protected static Object[][] parameters() { + return new Object[][] { + {false}, {true}, + }; + } + + @Parameter(index = 0) + protected boolean useFlip27Source; + + protected static final String CATALOG_NAME = "test_catalog"; + protected static final String DATABASE_NAME = "test_db"; + protected static final String TABLE_NAME = "test_table"; + protected final FileFormat format = FileFormat.AVRO; + protected int scanEventCount = 0; + protected ScanEvent lastScanEvent = null; + + @Override + protected TableEnvironment getTableEnv() { + super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); + super.getTableEnv() + .getConfig() + .getConfiguration() + .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); + return super.getTableEnv(); + } + + @BeforeEach + public void before() throws IOException { + // register a scan event listener to validate pushdown + Listeners.register( + event -> { + scanEventCount += 1; + lastScanEvent = event; + }, + ScanEvent.class); + + File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); + assertThat(warehouseFile.delete()).isTrue(); + String warehouse = String.format("file:%s", warehouseFile); + + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + + this.scanEventCount = 0; + this.lastScanEvent = null; + } + + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java new file mode 100644 index 000000000000..bde751e1f87f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java @@ -0,0 +1,81 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Objects; +import java.util.stream.Collectors; +import org.apache.flink.types.Row; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.junit.jupiter.api.Test; + +public class TestBoundedTableFactory extends ChangeLogTableTestBase { + + @Test + public void testEmptyDataSet() { + List> emptyDataSet = ImmutableList.of(); + + String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); + sql( + "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", + tableName, dataId); + + assertThat(sql("SELECT * FROM %s", tableName)).isEmpty(); + } + + @Test + public void testBoundedTableFactory() { + List> dataSet = + ImmutableList.of( + ImmutableList.of( + insertRow(1, "aaa"), + deleteRow(1, "aaa"), + insertRow(1, "bbb"), + insertRow(2, "aaa"), + deleteRow(2, "aaa"), + insertRow(2, "bbb")), + ImmutableList.of( + updateBeforeRow(2, "bbb"), + updateAfterRow(2, "ccc"), + deleteRow(2, "ccc"), + insertRow(2, "ddd")), + ImmutableList.of( + deleteRow(1, "bbb"), + insertRow(1, "ccc"), + deleteRow(1, "ccc"), + insertRow(1, "ddd"))); + + String dataId = BoundedTableFactory.registerDataSet(dataSet); + sql( + "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", + tableName, dataId); + + List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); + assertThat(sql("SELECT * FROM %s", tableName)).isEqualTo(rowSet); + + assertThat(sql("SELECT * FROM %s WHERE data='aaa'", tableName)) + .isEqualTo( + rowSet.stream() + .filter(r -> Objects.equals(r.getField(1), "aaa")) + .collect(Collectors.toList())); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java new file mode 100644 index 000000000000..c8b65e131c33 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.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.flink.source; + +import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.DataTypes; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.TestTemplate; + +/** Test {@link FlinkInputFormat}. */ +public class TestFlinkInputFormat extends TestFlinkSource { + + @Override + protected List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + return runFormat(formatBuilder.tableLoader(tableLoader()).buildFormat()); + } + + @TestTemplate + public void testNestedProjection() throws Exception { + Schema schema = + new Schema( + required(1, "data", Types.StringType.get()), + required( + 2, + "nested", + Types.StructType.of( + Types.NestedField.required(3, "f1", Types.StringType.get()), + Types.NestedField.required(4, "f2", Types.StringType.get()), + Types.NestedField.required(5, "f3", Types.LongType.get()))), + required(6, "id", Types.LongType.get())); + + Table table = + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), schema); + + List writeRecords = RandomGenericData.generate(schema, 2, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); + + // Schema: [data, nested[f1, f2, f3], id] + // Projection: [nested.f2, data] + // The Flink SQL output: [f2, data] + // The FlinkInputFormat output: [nested[f2], data] + + TableSchema projectedSchema = + TableSchema.builder() + .field("nested", DataTypes.ROW(DataTypes.FIELD("f2", DataTypes.STRING()))) + .field("data", DataTypes.STRING()) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : writeRecords) { + Row nested = Row.of(((Record) record.get(1)).get(1)); + expected.add(Row.of(nested, record.get(0))); + } + + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testBasicProjection() throws IOException { + Schema writeSchema = + new Schema( + Types.NestedField.required(0, "id", Types.LongType.get()), + Types.NestedField.optional(1, "data", Types.StringType.get()), + Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); + + Table table = + CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); + + List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); + + TableSchema projectedSchema = + TableSchema.builder() + .field("id", DataTypes.BIGINT()) + .field("data", DataTypes.STRING()) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : writeRecords) { + expected.add(Row.of(record.get(0), record.get(1))); + } + + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testReadPartitionColumn() throws Exception { + assumeThat(fileFormat).as("Temporary skip ORC").isNotEqualTo(FileFormat.ORC); + + Schema nestedSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.LongType.get()), + Types.NestedField.optional( + 2, + "struct", + Types.StructType.of( + Types.NestedField.optional(3, "innerId", Types.LongType.get()), + Types.NestedField.optional(4, "innerName", Types.StringType.get())))); + PartitionSpec spec = + PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); + + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); + List records = RandomGenericData.generate(nestedSchema, 10, 0L); + GenericAppenderHelper appender = + new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + for (Record record : records) { + org.apache.iceberg.TestHelpers.Row partition = + org.apache.iceberg.TestHelpers.Row.of(record.get(1, Record.class).get(1)); + appender.appendToTable(partition, Collections.singletonList(record)); + } + + TableSchema projectedSchema = + TableSchema.builder() + .field("struct", DataTypes.ROW(DataTypes.FIELD("innerName", DataTypes.STRING()))) + .build(); + List result = + runFormat( + FlinkSource.forRowData() + .tableLoader(tableLoader()) + .project(projectedSchema) + .buildFormat()); + + List expected = Lists.newArrayList(); + for (Record record : records) { + Row nested = Row.of(((Record) record.get(1)).get(1)); + expected.add(Row.of(nested)); + } + + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testValidation() { + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + FlinkSource.forRowData() + .env(StreamExecutionEnvironment.getExecutionEnvironment()) + .tableLoader(tableLoader()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } + + private List runFormat(FlinkInputFormat inputFormat) throws IOException { + RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); + return TestHelpers.readRows(inputFormat, rowType); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java new file mode 100644 index 000000000000..1b4fc863631f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java @@ -0,0 +1,71 @@ +/* + * 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.util.Map; +import org.apache.flink.table.types.logical.RowType; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.StructLikeSet; + +public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { + + @Override + protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) + throws IOException { + Schema projected = testTable.schema().select(columns); + RowType rowType = FlinkSchemaUtil.convert(projected); + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.WAREHOUSE_LOCATION, + hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + properties.put( + CatalogProperties.CLIENT_POOL_SIZE, + Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); + CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); + FlinkInputFormat inputFormat = + FlinkSource.forRowData() + .tableLoader( + TableLoader.fromCatalog( + hiveCatalogLoader, TableIdentifier.of("default", tableName))) + .project(FlinkSchemaUtil.toSchema(rowType)) + .buildFormat(); + + StructLikeSet set = StructLikeSet.create(projected.asStruct()); + TestHelpers.readRowData(inputFormat, rowType) + .forEach( + rowData -> { + RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); + set.add(wrapper.wrap(rowData)); + }); + + return set; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java new file mode 100644 index 000000000000..59a4c3118cdf --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.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.source; + +import java.io.File; +import java.io.IOException; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestMergingMetrics; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.RowDataConverter; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.FlinkAppenderFactory; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestFlinkMergingMetrics extends TestMergingMetrics { + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension("test_db", "test_table"); + + @Override + protected FileAppender writeAndGetAppender(List records) throws IOException { + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); + FileAppender appender = + new FlinkAppenderFactory( + table, + SCHEMA, + flinkSchema, + ImmutableMap.of(), + PartitionSpec.unpartitioned(), + null, + null, + null) + .newAppender( + Files.localOutput(File.createTempFile("junit", null, tempDir)), fileFormat); + try (FileAppender fileAppender = appender) { + records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); + } + return appender; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java new file mode 100644 index 000000000000..8352924d042a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java @@ -0,0 +1,813 @@ +/* + * 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 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.time.Instant; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.avro.generic.GenericData; +import org.apache.commons.collections.ListUtils; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileContent; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Files; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.MetadataTableType; +import org.apache.iceberg.MetadataTableUtils; +import org.apache.iceberg.MetricsUtil; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.catalog.Namespace; +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.expressions.Expressions; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.CloseableIterable; +import org.apache.iceberg.io.InputFile; +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.SnapshotUtil; +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; + +public class TestFlinkMetaDataTable extends CatalogTestBase { + private static final String TABLE_NAME = "test_table"; + private final FileFormat format = FileFormat.AVRO; + private @TempDir Path temp; + + @Parameter(index = 2) + private Boolean isPartition; + + @Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + + for (Boolean isPartition : new Boolean[] {true, false}) { + String catalogName = "testhadoop"; + Namespace baseNamespace = Namespace.of("default"); + parameters.add(new Object[] {catalogName, baseNamespace, isPartition}); + } + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + @BeforeEach + public void before() { + super.before(); + sql("USE CATALOG %s", catalogName); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE %s", DATABASE); + if (isPartition) { + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')", + TABLE_NAME, format.name()); + sql("INSERT INTO %s VALUES (1,'a',10),(2,'a',20)", TABLE_NAME); + sql("INSERT INTO %s VALUES (1,'b',10),(2,'b',20)", TABLE_NAME); + } else { + sql( + "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')", + TABLE_NAME, format.name()); + sql( + "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", + TABLE_NAME); + sql("INSERT INTO %s VALUES (4,'iceberg',10)", TABLE_NAME); + } + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + @TestTemplate + public void testSnapshots() { + String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + Iterator snapshots = table.snapshots().iterator(); + for (Row row : result) { + Snapshot next = snapshots.next(); + assertThat(((Instant) row.getField(0)).toEpochMilli()) + .as("Should have expected timestamp") + .isEqualTo(next.timestampMillis()); + assertThat(next.snapshotId()) + .as("Should have expected snapshot id") + .isEqualTo(next.snapshotId()); + assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); + assertThat(row.getField(3)).as("Should have expected operation").isEqualTo(next.operation()); + assertThat(row.getField(4)) + .as("Should have expected manifest list location") + .isEqualTo(next.manifestListLocation()); + assertThat(row.getField(5)).as("Should have expected summary").isEqualTo(next.summary()); + } + } + + @TestTemplate + public void testHistory() { + String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + Iterator snapshots = table.snapshots().iterator(); + for (Row row : result) { + Snapshot next = snapshots.next(); + assertThat(((Instant) row.getField(0)).toEpochMilli()) + .as("Should have expected made_current_at") + .isEqualTo(next.timestampMillis()); + assertThat(row.getField(1)) + .as("Should have expected snapshot id") + .isEqualTo(next.snapshotId()); + assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); + assertThat(row.getField(3)) + .as("Should have expected is current ancestor") + .isEqualTo( + SnapshotUtil.isAncestorOf( + table, table.currentSnapshot().snapshotId(), next.snapshotId())); + } + } + + @TestTemplate + public void testManifests() { + String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME); + List result = sql(sql); + + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + List expectedDataManifests = dataManifests(table); + + for (int i = 0; i < result.size(); i++) { + Row row = result.get(i); + ManifestFile manifestFile = expectedDataManifests.get(i); + assertThat(row.getField(0)) + .as("Should have expected content") + .isEqualTo(manifestFile.content().id()); + assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); + assertThat(row.getField(2)) + .as("Should have expected length") + .isEqualTo(manifestFile.length()); + assertThat(row.getField(3)) + .as("Should have expected partition_spec_id") + .isEqualTo(manifestFile.partitionSpecId()); + assertThat(row.getField(4)) + .as("Should have expected added_snapshot_id") + .isEqualTo(manifestFile.snapshotId()); + assertThat(row.getField(5)) + .as("Should have expected added_data_files_count") + .isEqualTo(manifestFile.addedFilesCount()); + assertThat(row.getField(6)) + .as("Should have expected existing_data_files_count") + .isEqualTo(manifestFile.existingFilesCount()); + assertThat(row.getField(7)) + .as("Should have expected deleted_data_files_count") + .isEqualTo(manifestFile.deletedFilesCount()); + } + } + + @TestTemplate + public void testAllManifests() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + String sql = String.format("SELECT * FROM %s$all_manifests ", TABLE_NAME); + List result = sql(sql); + + List expectedDataManifests = allDataManifests(table); + + assertThat(expectedDataManifests).hasSize(result.size()); + for (int i = 0; i < result.size(); i++) { + Row row = result.get(i); + ManifestFile manifestFile = expectedDataManifests.get(i); + assertThat(row.getField(0)) + .as("Should have expected content") + .isEqualTo(manifestFile.content().id()); + assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); + assertThat(row.getField(2)) + .as("Should have expected length") + .isEqualTo(manifestFile.length()); + assertThat(row.getField(3)) + .as("Should have expected partition_spec_id") + .isEqualTo(manifestFile.partitionSpecId()); + assertThat(row.getField(4)) + .as("Should have expected added_snapshot_id") + .isEqualTo(manifestFile.snapshotId()); + assertThat(row.getField(5)) + .as("Should have expected added_data_files_count") + .isEqualTo(manifestFile.addedFilesCount()); + assertThat(row.getField(6)) + .as("Should have expected existing_data_files_count") + .isEqualTo(manifestFile.existingFilesCount()); + assertThat(row.getField(7)) + .as("Should have expected deleted_data_files_count") + .isEqualTo(manifestFile.deletedFilesCount()); + } + } + + @TestTemplate + public void testUnPartitionedTable() throws IOException { + assumeThat(isPartition).isFalse(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + List dataDeletes = Lists.newArrayList(dataDelete.copy("id", 1)); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, Files.localOutput(testFile), dataDeletes, deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + List expectedDataManifests = dataManifests(table); + List expectedDeleteManifests = deleteManifests(table); + + assertThat(expectedDataManifests).hasSize(2); + assertThat(expectedDeleteManifests).hasSize(1); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + + // check delete files table + Schema deleteFilesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("delete_files")) + .schema(); + + List deleteColumns = + deleteFilesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String deleteNames = + deleteColumns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + deleteFilesTableSchema = deleteFilesTableSchema.select(deleteColumns); + + List actualDeleteFiles = sql("SELECT %s FROM %s$delete_files", deleteNames, TABLE_NAME); + assertThat(actualDeleteFiles).hasSize(1); + assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); + + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); + assertThat(expectedDeleteFiles).as("Should be 1 delete file manifest entry").hasSize(1); + TestHelpers.assertEquals( + deleteFilesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check data files table + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + List actualDataFiles = sql("SELECT %s FROM %s$data_files", names, TABLE_NAME); + assertThat(actualDataFiles).as("Metadata table should return 2 data file").hasSize(2); + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + assertThat(expectedDataFiles).as("Should be 2 data file manifest entry").hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + // check all files table + List actualFiles = sql("SELECT %s FROM %s$files ORDER BY content", names, TABLE_NAME); + assertThat(actualFiles).as("Metadata table should return 3 files").hasSize(3); + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + assertThat(expectedFiles).as("Should have 3 files manifest entriess").hasSize(3); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + assumeThat(isPartition).isTrue(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id", "data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + deleteRow.put("data", "a"); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile), + org.apache.iceberg.TestHelpers.Row.of("a"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + deleteRow.put("data", "b"); + File testFile2 = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile2), + org.apache.iceberg.TestHelpers.Row.of("b"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes2).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + + List expectedDataManifests = dataManifests(table); + List expectedDeleteManifests = deleteManifests(table); + + assertThat(expectedDataManifests).hasSize(2); + assertThat(expectedDeleteManifests).hasSize(2); + Table deleteFilesTable = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("delete_files")); + Schema filesTableSchema = deleteFilesTable.schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check delete files table + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + assertThat(expectedDeleteFiles).hasSize(1); + List actualDeleteFiles = + sql("SELECT %s FROM %s$delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + + assertThat(actualDeleteFiles).hasSize(1); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check data files table + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + assertThat(expectedDataFiles).hasSize(1); + List actualDataFiles = + sql("SELECT %s FROM %s$data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + assertThat(actualDataFiles).hasSize(1); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + List actualPartitionsWithProjection = + sql("SELECT file_count FROM %s$partitions ", TABLE_NAME); + assertThat(actualPartitionsWithProjection).hasSize(2); + for (int i = 0; i < 2; ++i) { + assertThat(actualPartitionsWithProjection.get(i).getField(0)).isEqualTo(1); + } + + // Check files table + List expectedFiles = + Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) + .collect(Collectors.toList()); + assertThat(expectedFiles).hasSize(2); + List actualFiles = + sql( + "SELECT %s FROM %s$files WHERE `partition`.`data`='a' ORDER BY content", + names, TABLE_NAME); + assertThat(actualFiles).hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); + TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); + } + + @TestTemplate + public void testAllFilesUnpartitioned() throws Exception { + assumeThat(isPartition).isFalse(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Schema deleteRowSchema = table.schema().select("id", "data"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).commit(); + + List expectedDataManifests = dataManifests(table); + assertThat(expectedDataManifests).hasSize(2); + List expectedDeleteManifests = deleteManifests(table); + assertThat(expectedDeleteManifests).hasSize(1); + + // Clear table to test whether 'all_files' can read past files + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("all_data_files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check all data files table + List actualDataFiles = + sql("SELECT %s FROM %s$all_data_files order by record_count ", names, TABLE_NAME); + + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); + assertThat(expectedDataFiles).hasSize(2); + assertThat(actualDataFiles).hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles, actualDataFiles); + + // Check all delete files table + List actualDeleteFiles = sql("SELECT %s FROM %s$all_delete_files", names, TABLE_NAME); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); + assertThat(expectedDeleteFiles).hasSize(1); + assertThat(actualDeleteFiles).hasSize(1); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check all files table + List actualFiles = + sql("SELECT %s FROM %s$all_files ORDER BY content, record_count asc", names, TABLE_NAME); + List expectedFiles = + ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + assertThat(actualFiles).hasSize(3); + TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); + } + + @TestTemplate + public void testAllFilesPartitioned() throws Exception { + assumeThat(!isPartition).isFalse(); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + // Create delete file + Schema deleteRowSchema = table.schema().select("id"); + Record dataDelete = GenericRecord.create(deleteRowSchema); + + Map deleteRow = Maps.newHashMap(); + deleteRow.put("id", 1); + File testFile = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile), + org.apache.iceberg.TestHelpers.Row.of("a"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + File testFile2 = File.createTempFile("junit", null, temp.toFile()); + DeleteFile eqDeletes2 = + FileHelpers.writeDeleteFile( + table, + Files.localOutput(testFile2), + org.apache.iceberg.TestHelpers.Row.of("b"), + Lists.newArrayList(dataDelete.copy(deleteRow)), + deleteRowSchema); + table.newRowDelta().addDeletes(eqDeletes).addDeletes(eqDeletes2).commit(); + + List expectedDataManifests = dataManifests(table); + assertThat(expectedDataManifests).hasSize(2); + List expectedDeleteManifests = deleteManifests(table); + assertThat(expectedDeleteManifests).hasSize(1); + // Clear table to test whether 'all_files' can read past files + table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); + + Schema entriesTableSchema = + MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) + .schema(); + Schema filesTableSchema = + MetadataTableUtils.createMetadataTableInstance( + table, MetadataTableType.from("all_data_files")) + .schema(); + + List columns = + filesTableSchema.columns().stream() + .map(Types.NestedField::name) + .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) + .collect(Collectors.toList()); + String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); + + filesTableSchema = filesTableSchema.select(columns); + + // Check all data files table + List actualDataFiles = + sql("SELECT %s FROM %s$all_data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + List expectedDataFiles = + expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); + assertThat(expectedDataFiles).hasSize(1); + assertThat(actualDataFiles).hasSize(1); + TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); + + // Check all delete files table + List actualDeleteFiles = + sql("SELECT %s FROM %s$all_delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); + List expectedDeleteFiles = + expectedEntries( + table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); + assertThat(expectedDeleteFiles).hasSize(1); + assertThat(actualDeleteFiles).hasSize(1); + TestHelpers.assertEquals( + filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); + + // Check all files table + List actualFiles = + sql( + "SELECT %s FROM %s$all_files WHERE `partition`.`data`='a' ORDER BY content", + names, TABLE_NAME); + List expectedFiles = + ListUtils.union(expectedDataFiles, expectedDeleteFiles); + expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); + assertThat(actualFiles).hasSize(2); + TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); + } + + @TestTemplate + public void testMetadataLogEntries() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); + Snapshot currentSnapshot = tableMetadata.currentSnapshot(); + Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); + List metadataLogEntries = + Lists.newArrayList(tableMetadata.previousFiles()); + + // Check metadataLog table + List metadataLogs = sql("SELECT * FROM %s$metadata_log_entries", TABLE_NAME); + + assertThat(metadataLogs).hasSize(3); + Row metadataLog = metadataLogs.get(0); + assertThat(metadataLog.getField("timestamp")) + .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(0).timestampMillis())); + assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(0).file()); + assertThat(metadataLog.getField("latest_snapshot_id")).isNull(); + assertThat(metadataLog.getField("latest_schema_id")).isNull(); + assertThat(metadataLog.getField("latest_sequence_number")).isNull(); + + metadataLog = metadataLogs.get(1); + assertThat(metadataLog.getField("timestamp")) + .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(1).timestampMillis())); + assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(1).file()); + assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); + assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(parentSnapshot.schemaId()); + assertThat(metadataLog.getField("latest_sequence_number")) + .isEqualTo(parentSnapshot.sequenceNumber()); + assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); + + metadataLog = metadataLogs.get(2); + assertThat(metadataLog.getField("timestamp")) + .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); + assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); + assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); + assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); + assertThat(metadataLog.getField("latest_sequence_number")) + .isEqualTo(currentSnapshot.sequenceNumber()); + + // test filtering + List metadataLogWithFilters = + sql( + "SELECT * FROM %s$metadata_log_entries WHERE latest_snapshot_id = %s", + TABLE_NAME, currentSnapshotId); + assertThat(metadataLogWithFilters).hasSize(1); + metadataLog = metadataLogWithFilters.get(0); + assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) + .isEqualTo(metadataLog.getField("timestamp")); + + assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); + assertThat(metadataLog.getField("latest_snapshot_id")) + .isEqualTo(tableMetadata.currentSnapshot().snapshotId()); + assertThat(metadataLog.getField("latest_schema_id")) + .isEqualTo(tableMetadata.currentSnapshot().schemaId()); + assertThat(metadataLog.getField("latest_sequence_number")) + .isEqualTo(tableMetadata.currentSnapshot().sequenceNumber()); + + // test projection + List metadataFiles = + metadataLogEntries.stream() + .map(TableMetadata.MetadataLogEntry::file) + .collect(Collectors.toList()); + metadataFiles.add(tableMetadata.metadataFileLocation()); + List metadataLogWithProjection = + sql("SELECT file FROM %s$metadata_log_entries", TABLE_NAME); + assertThat(metadataLogWithProjection).hasSize(3); + for (int i = 0; i < metadataFiles.size(); i++) { + assertThat(metadataLogWithProjection.get(i).getField("file")).isEqualTo(metadataFiles.get(i)); + } + } + + @TestTemplate + public void testSnapshotReferencesMetatable() { + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); + + Long currentSnapshotId = table.currentSnapshot().snapshotId(); + + // Create branch + table + .manageSnapshots() + .createBranch("testBranch", currentSnapshotId) + .setMaxRefAgeMs("testBranch", 10) + .setMinSnapshotsToKeep("testBranch", 20) + .setMaxSnapshotAgeMs("testBranch", 30) + .commit(); + // Create Tag + table + .manageSnapshots() + .createTag("testTag", currentSnapshotId) + .setMaxRefAgeMs("testTag", 50) + .commit(); + // Check refs table + List references = sql("SELECT * FROM %s$refs", TABLE_NAME); + List branches = sql("SELECT * FROM %s$refs WHERE type='BRANCH'", TABLE_NAME); + assertThat(references).hasSize(3); + assertThat(branches).hasSize(2); + List tags = sql("SELECT * FROM %s$refs WHERE type='TAG'", TABLE_NAME); + assertThat(tags).hasSize(1); + // Check branch entries in refs table + List mainBranch = + sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); + assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); + List testBranch = + sql("SELECT * FROM %s$refs WHERE name='testBranch' AND type='BRANCH'", TABLE_NAME); + assertThat((String) testBranch.get(0).getFieldAs("name")).isEqualTo("testBranch"); + assertThat((String) testBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + assertThat((Long) testBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); + assertThat((Long) testBranch.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(10)); + assertThat((Integer) testBranch.get(0).getFieldAs("min_snapshots_to_keep")) + .isEqualTo(Integer.valueOf(20)); + assertThat((Long) testBranch.get(0).getFieldAs("max_snapshot_age_in_ms")) + .isEqualTo(Long.valueOf(30)); + + // Check tag entries in refs table + List testTag = + sql("SELECT * FROM %s$refs WHERE name='testTag' AND type='TAG'", TABLE_NAME); + assertThat((String) testTag.get(0).getFieldAs("name")).isEqualTo("testTag"); + assertThat((String) testTag.get(0).getFieldAs("type")).isEqualTo("TAG"); + assertThat((Long) testTag.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); + assertThat((Long) testTag.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(50)); + // Check projection in refs table + List testTagProjection = + sql( + "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM %s$refs where type='TAG'", + TABLE_NAME); + assertThat((String) testTagProjection.get(0).getFieldAs("name")).isEqualTo("testTag"); + assertThat((String) testTagProjection.get(0).getFieldAs("type")).isEqualTo("TAG"); + assertThat((Long) testTagProjection.get(0).getFieldAs("snapshot_id")) + .isEqualTo(currentSnapshotId); + assertThat((Long) testTagProjection.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(50)); + assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); + List mainBranchProjection = + sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); + assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); + assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + List testBranchProjection = + sql( + "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM %s$refs WHERE name='testBranch' AND type = 'BRANCH'", + TABLE_NAME); + assertThat((String) testBranchProjection.get(0).getFieldAs("name")).isEqualTo("testBranch"); + assertThat((String) testBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); + assertThat((Long) testBranchProjection.get(0).getFieldAs("snapshot_id")) + .isEqualTo(currentSnapshotId); + assertThat((Long) testBranchProjection.get(0).getFieldAs("max_reference_age_in_ms")) + .isEqualTo(Long.valueOf(10)); + } + + /** + * Find matching manifest entries of an Iceberg table + * + * @param table iceberg table + * @param expectedContent file content to populate on entries + * @param entriesTableSchema schema of Manifest entries + * @param manifestsToExplore manifests to explore of the table + * @param partValue partition value that manifest entries must match, or null to skip filtering + */ + private List expectedEntries( + Table table, + FileContent expectedContent, + Schema entriesTableSchema, + List manifestsToExplore, + String partValue) + throws IOException { + List expected = Lists.newArrayList(); + for (ManifestFile manifest : manifestsToExplore) { + InputFile in = table.io().newInputFile(manifest.path()); + try (CloseableIterable rows = + Avro.read(in).project(entriesTableSchema).build()) { + for (GenericData.Record record : rows) { + if ((Integer) record.get("status") < 2 /* added or existing */) { + GenericData.Record file = (GenericData.Record) record.get("data_file"); + if (partitionMatch(file, partValue)) { + asMetadataRecord(file, expectedContent); + expected.add(file); + } + } + } + } + } + return expected; + } + + // Populate certain fields derived in the metadata tables + private void asMetadataRecord(GenericData.Record file, FileContent content) { + file.put(0, content.id()); + file.put(3, 0); // specId + } + + private boolean partitionMatch(GenericData.Record file, String partValue) { + if (partValue == null) { + return true; + } + GenericData.Record partition = (GenericData.Record) file.get(4); + return partValue.equals(partition.get(0).toString()); + } + + private List dataManifests(Table table) { + return table.currentSnapshot().dataManifests(table.io()); + } + + private List allDataManifests(Table table) { + List manifests = Lists.newArrayList(); + for (Snapshot snapshot : table.snapshots()) { + manifests.addAll(snapshot.dataManifests(table.io())); + } + return manifests; + } + + private List deleteManifests(Table table) { + return table.currentSnapshot().deleteManifests(table.io()); + } +} 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 new file mode 100644 index 000000000000..0b5a8011ad3f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java @@ -0,0 +1,90 @@ +/* + * 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.util.Map; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ParameterizedTestExtension; +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.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.DeleteReadTests; +import org.apache.iceberg.hive.HiveCatalog; +import org.apache.iceberg.hive.TestHiveMetastore; +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.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { + + protected static String databaseName = "default"; + + protected static HiveConf hiveConf = null; + protected static HiveCatalog catalog = null; + private static TestHiveMetastore metastore = null; + + @BeforeAll + public static void startMetastore() { + metastore = new TestHiveMetastore(); + metastore.start(); + hiveConf = metastore.hiveConf(); + catalog = + (HiveCatalog) + CatalogUtil.loadCatalog( + HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); + } + + @AfterAll + public static void stopMetastore() throws Exception { + metastore.stop(); + catalog = null; + } + + @Override + protected Table createTable(String name, Schema schema, PartitionSpec spec) { + Map props = Maps.newHashMap(); + props.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + 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)); + + return table; + } + + @Override + protected void dropTable(String name) { + catalog.dropTable(TableIdentifier.of(databaseName, name)); + } + + @Override + protected boolean expectPruned() { + return false; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java new file mode 100644 index 000000000000..cf6b233dcec6 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java @@ -0,0 +1,540 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.nio.file.Path; +import java.time.LocalDate; +import java.time.LocalDateTime; +import java.time.LocalTime; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.DataFile; +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.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +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.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +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 TestFlinkScan { + @RegisterExtension + protected static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @TempDir protected Path temporaryDirectory; + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @Parameter protected FileFormat fileFormat; + + @Parameters(name = "format={0}") + public static Collection fileFormat() { + return Arrays.asList(FileFormat.AVRO, FileFormat.PARQUET, FileFormat.ORC); + } + + protected TableLoader tableLoader() { + return CATALOG_EXTENSION.tableLoader(); + } + + protected abstract List runWithProjection(String... projected) throws Exception; + + protected abstract List runWithFilter( + Expression filter, String sqlFilter, boolean caseSensitive) throws Exception; + + protected List runWithFilter(Expression filter, String sqlFilter) throws Exception { + return runWithFilter(filter, sqlFilter, true); + } + + protected abstract List runWithOptions(Map options) throws Exception; + + protected abstract List run() throws Exception; + + @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, temporaryDirectory).appendToTable(expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + Table table = + CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testProjection() throws Exception { + Table table = + CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + new GenericAppenderHelper(table, fileFormat, temporaryDirectory) + .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); + assertRows(runWithProjection("data"), Row.of(inputRecords.get(0).get(0))); + } + + @TestTemplate + public void testIdentityPartitionProjections() throws Exception { + Schema logSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "dt", Types.StringType.get()), + Types.NestedField.optional(3, "level", Types.StringType.get()), + Types.NestedField.optional(4, "message", Types.StringType.get())); + PartitionSpec spec = + PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); + + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); + List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); + + int idx = 0; + AppendFiles append = table.newAppend(); + for (Record record : inputRecords) { + record.set(1, "2020-03-2" + idx); + record.set(2, Integer.toString(idx)); + append.appendFile( + new GenericAppenderHelper(table, fileFormat, temporaryDirectory) + .writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), + ImmutableList.of(record))); + idx += 1; + } + append.commit(); + + // individual fields + validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); + validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); + // field pairs + validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); + // out-of-order pairs + validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); + validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); + // out-of-order triplets + validateIdentityPartitionProjections( + table, Arrays.asList("dt", "level", "message"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("level", "dt", "message"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("dt", "message", "level"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("level", "message", "dt"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("message", "dt", "level"), inputRecords); + validateIdentityPartitionProjections( + table, Arrays.asList("message", "level", "dt"), inputRecords); + } + + private void validateIdentityPartitionProjections( + Table table, List projectedFields, List inputRecords) throws Exception { + List rows = runWithProjection(projectedFields.toArray(new String[0])); + + for (int pos = 0; pos < inputRecords.size(); pos++) { + Record inputRecord = inputRecords.get(pos); + Row actualRecord = rows.get(pos); + + for (int i = 0; i < projectedFields.size(); i++) { + String name = projectedFields.get(i); + assertThat(inputRecord.getField(name)) + .as("Projected field " + name + " should match") + .isEqualTo(actualRecord.getField(i)); + } + } + } + + @TestTemplate + public void testSnapshotReads() throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords); + long snapshotId = table.currentSnapshot().snapshotId(); + + long timestampMillis = table.currentSnapshot().timestampMillis(); + + // produce another timestamp + waitUntilAfter(timestampMillis); + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L)); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("snapshot-id", Long.toString(snapshotId))), + expectedRecords, + TestFixtures.SCHEMA); + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("as-of-timestamp", Long.toString(timestampMillis))), + expectedRecords, + TestFixtures.SCHEMA); + } + + @TestTemplate + public void testTagReads() throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List expectedRecords1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords1); + long snapshotId = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().createTag("t1", snapshotId).commit(); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords1, TestFixtures.SCHEMA); + + List expectedRecords2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecords2); + snapshotId = table.currentSnapshot().snapshotId(); + + table.manageSnapshots().replaceTag("t1", snapshotId).commit(); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.addAll(expectedRecords1); + expectedRecords.addAll(expectedRecords2); + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testBranchReads() throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List expectedRecordsBase = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecordsBase); + long snapshotId = table.currentSnapshot().snapshotId(); + + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName, snapshotId).commit(); + + List expectedRecordsForBranch = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(branchName, expectedRecordsForBranch); + + List expectedRecordsForMain = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(expectedRecordsForMain); + + List branchExpectedRecords = Lists.newArrayList(); + branchExpectedRecords.addAll(expectedRecordsBase); + branchExpectedRecords.addAll(expectedRecordsForBranch); + + TestHelpers.assertRecords( + runWithOptions(ImmutableMap.of("branch", branchName)), + branchExpectedRecords, + TestFixtures.SCHEMA); + + List mainExpectedRecords = Lists.newArrayList(); + mainExpectedRecords.addAll(expectedRecordsBase); + mainExpectedRecords.addAll(expectedRecordsForMain); + + TestHelpers.assertRecords(run(), mainExpectedRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testIncrementalReadViaTag() throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(records1); + long snapshotId1 = table.currentSnapshot().snapshotId(); + String startTag = "t1"; + table.manageSnapshots().createTag(startTag, snapshotId1).commit(); + + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); + helper.appendToTable(records2); + + List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); + helper.appendToTable(records3); + long snapshotId3 = table.currentSnapshot().snapshotId(); + String endTag = "t2"; + table.manageSnapshots().createTag(endTag, snapshotId3).commit(); + + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); + + List expected = Lists.newArrayList(); + expected.addAll(records2); + expected.addAll(records3); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-snapshot-id", Long.toString(snapshotId1)) + .put("end-tag", endTag) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow()), + expected, + TestFixtures.SCHEMA); + + assertThatThrownBy( + () -> + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .put("start-snapshot-id", Long.toString(snapshotId1)) + .buildOrThrow())) + .isInstanceOf(Exception.class) + .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + + assertThatThrownBy( + () -> + runWithOptions( + ImmutableMap.builder() + .put("start-tag", startTag) + .put("end-tag", endTag) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow())) + .isInstanceOf(Exception.class) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @TestTemplate + public void testIncrementalRead() throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); + helper.appendToTable(records1); + long snapshotId1 = table.currentSnapshot().snapshotId(); + + // snapshot 2 + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); + helper.appendToTable(records2); + + List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); + helper.appendToTable(records3); + long snapshotId3 = table.currentSnapshot().snapshotId(); + + // snapshot 4 + helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); + + List expected2 = Lists.newArrayList(); + expected2.addAll(records2); + expected2.addAll(records3); + TestHelpers.assertRecords( + runWithOptions( + ImmutableMap.builder() + .put("start-snapshot-id", Long.toString(snapshotId1)) + .put("end-snapshot-id", Long.toString(snapshotId3)) + .buildOrThrow()), + expected2, + TestFixtures.SCHEMA); + } + + @TestTemplate + public void testFilterExpPartition() throws Exception { + Table table = + CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + expectedRecords.get(0).set(2, "2020-03-20"); + expectedRecords.get(1).set(2, "2020-03-20"); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + DataFile dataFile1 = + helper.writeFile(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + DataFile dataFile2 = + helper.writeFile( + org.apache.iceberg.TestHelpers.Row.of("2020-03-21", 0), + RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); + helper.appendToTable(dataFile1, dataFile2); + TestHelpers.assertRecords( + runWithFilter(Expressions.equal("dt", "2020-03-20"), "where dt='2020-03-20'", true), + expectedRecords, + TestFixtures.SCHEMA); + } + + private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); + expectedRecords.get(0).set(0, "a"); + expectedRecords.get(1).set(0, "b"); + expectedRecords.get(2).set(0, "c"); + + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + DataFile dataFile = helper.writeFile(expectedRecords); + helper.appendToTable(dataFile); + + List actual = + runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); + + TestHelpers.assertRecords(actual, expectedRecords.subList(1, 3), TestFixtures.SCHEMA); + } + + @TestTemplate + public void testFilterExp() throws Exception { + testFilterExp(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); + } + + @TestTemplate + public void testFilterExpCaseInsensitive() throws Exception { + // sqlFilter does not support case-insensitive filtering: + // https://issues.apache.org/jira/browse/FLINK-16175 + testFilterExp(Expressions.greaterThanOrEqual("DATA", "b"), "where data>='b'", false); + } + + @TestTemplate + public void testPartitionTypes() throws Exception { + Schema typesSchema = + new Schema( + Types.NestedField.optional(1, "id", Types.IntegerType.get()), + Types.NestedField.optional(2, "decimal", Types.DecimalType.of(38, 18)), + Types.NestedField.optional(3, "str", Types.StringType.get()), + Types.NestedField.optional(4, "binary", Types.BinaryType.get()), + Types.NestedField.optional(5, "date", Types.DateType.get()), + Types.NestedField.optional(6, "time", Types.TimeType.get()), + Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone())); + PartitionSpec spec = + PartitionSpec.builderFor(typesSchema) + .identity("decimal") + .identity("str") + .identity("binary") + .identity("date") + .identity("time") + .identity("timestamp") + .build(); + + Table table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); + List records = RandomGenericData.generate(typesSchema, 10, 0L); + GenericAppenderHelper appender = + new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + for (Record record : records) { + org.apache.iceberg.TestHelpers.Row partition = + org.apache.iceberg.TestHelpers.Row.of( + record.get(1), + record.get(2), + record.get(3), + record.get(4) == null ? null : DateTimeUtil.daysFromDate((LocalDate) record.get(4)), + record.get(5) == null ? null : DateTimeUtil.microsFromTime((LocalTime) record.get(5)), + record.get(6) == null + ? null + : DateTimeUtil.microsFromTimestamp((LocalDateTime) record.get(6))); + appender.appendToTable(partition, Collections.singletonList(record)); + } + + TestHelpers.assertRecords(run(), records, typesSchema); + } + + @TestTemplate + public void testCustomizedFlinkDataTypes() throws Exception { + Schema schema = + new Schema( + Types.NestedField.required( + 1, + "map", + Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), + Types.NestedField.required( + 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); + List records = RandomGenericData.generate(schema, 10, 0L); + GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); + helper.appendToTable(records); + TestHelpers.assertRecords(run(), records, schema); + } + + private static void assertRows(List results, Row... expected) { + TestHelpers.assertRows(results, Arrays.asList(expected)); + } + + private static void waitUntilAfter(long timestampMillis) { + long current = System.currentTimeMillis(); + while (current <= timestampMillis) { + current = System.currentTimeMillis(); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java new file mode 100644 index 000000000000..1493c0932044 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.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.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.junit.jupiter.api.BeforeEach; + +/** Test Flink SELECT SQLs. */ +public class TestFlinkScanSql extends TestFlinkSource { + private volatile TableEnvironment tEnv; + + @BeforeEach + public void before() throws IOException { + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_EXTENSION.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Override + protected List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(sqlOptions); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java new file mode 100644 index 000000000000..dd50170f0fd7 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java @@ -0,0 +1,90 @@ +/* + * 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.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import org.apache.flink.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public abstract class TestFlinkSource extends TestFlinkScan { + + @Override + protected List runWithProjection(String... projected) throws Exception { + TableSchema.Builder builder = TableSchema.builder(); + TableSchema schema = + FlinkSchemaUtil.toSchema( + FlinkSchemaUtil.convert( + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); + for (String field : projected) { + TableColumn column = schema.getTableColumn(field).get(); + builder.field(column.getName(), column.getType()); + } + return run(FlinkSource.forRowData().project(builder.build()), Maps.newHashMap(), "", projected); + } + + @Override + protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + FlinkSource.Builder builder = + FlinkSource.forRowData().filters(Collections.singletonList(filter)); + Map options = Maps.newHashMap(); + options.put("case-sensitive", Boolean.toString(caseSensitive)); + return run(builder, options, sqlFilter, "*"); + } + + @Override + protected List runWithOptions(Map options) throws Exception { + FlinkSource.Builder builder = FlinkSource.forRowData(); + Optional.ofNullable(options.get("case-sensitive")) + .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); + Optional.ofNullable(options.get("snapshot-id")) + .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); + Optional.ofNullable(options.get("branch")).ifPresent(value -> builder.branch(value)); + Optional.ofNullable(options.get("start-tag")).ifPresent(value -> builder.startTag(value)); + Optional.ofNullable(options.get("end-tag")).ifPresent(value -> builder.endTag(value)); + Optional.ofNullable(options.get("start-snapshot-id")) + .ifPresent(value -> builder.startSnapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("end-snapshot-id")) + .ifPresent(value -> builder.endSnapshotId(Long.parseLong(value))); + Optional.ofNullable(options.get("as-of-timestamp")) + .ifPresent(value -> builder.asOfTimestamp(Long.parseLong(value))); + return run(builder, options, "", "*"); + } + + @Override + protected List run() throws Exception { + return run(FlinkSource.forRowData(), Maps.newHashMap(), "", "*"); + } + + protected abstract List run( + FlinkSource.Builder formatBuilder, + Map sqlOptions, + String sqlFilter, + String... sqlSelectedFields) + throws Exception; +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java new file mode 100644 index 000000000000..14131d9e96d5 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java @@ -0,0 +1,61 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkSourceConfig extends TableSourceTestBase { + private static final String TABLE = "test_table"; + + @TestTemplate + public void testFlinkSessionConfig() { + getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); + assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot set as-of-timestamp option for streaming reader"); + } + + @TestTemplate + public void testFlinkHintConfig() { + List result = + sql( + "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", + TABLE, System.currentTimeMillis()); + assertThat(result).hasSize(3); + } + + @TestTemplate + public void testReadOptionHierarchy() { + getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); + List result = sql("SELECT * FROM %s", TABLE); + // Note that this query doesn't have the limit clause in the SQL. + // This assertions works because limit is pushed down to the reader and + // reader parallelism is 1. + assertThat(result).hasSize(1); + + result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); + assertThat(result).hasSize(3); + } +} 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 new file mode 100644 index 000000000000..e1162c3225b1 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java @@ -0,0 +1,85 @@ +/* + * 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.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.PipelineOptions; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +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.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.Test; + +/** Use the FlinkSource */ +public class TestFlinkSourceSql extends TestSqlBase { + @Override + public void before() throws IOException { + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_EXTENSION.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + @Test + public void testInferParallelismWithGlobalSetting() throws IOException { + Configuration cfg = getTableEnv().getConfig().getConfiguration(); + cfg.set(PipelineOptions.MAX_PARALLELISM, 1); + + Table table = + CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + List expectedRecords = Lists.newArrayList(); + long maxFileLen = 0; + for (int i = 0; i < 5; i++) { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); + DataFile dataFile = helper.writeFile(null, records); + helper.appendToTable(dataFile); + expectedRecords.addAll(records); + maxFileLen = Math.max(dataFile.fileSizeInBytes(), maxFileLen); + } + + // Make sure to generate multiple CombinedScanTasks + SqlHelpers.sql( + getTableEnv(), + "ALTER TABLE t SET ('read.split.open-file-cost'='1', 'read.split.target-size'='%s')", + maxFileLen); + + List results = run(Maps.newHashMap(), "", "*"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java new file mode 100644 index 000000000000..18528c789114 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java @@ -0,0 +1,561 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.flink.table.api.SqlParserException; +import org.apache.flink.types.Row; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.TestTemplate; + +public class TestFlinkTableSource extends TableSourceTestBase { + + @TestTemplate + public void testLimitPushDown() { + + assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) + .isInstanceOf(SqlParserException.class) + .hasMessageStartingWith("SQL parse failed."); + + assertThat(sql("SELECT * FROM %s LIMIT 0", TABLE_NAME)).isEmpty(); + + String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); + List resultExceed = sql(sqlLimitExceed); + assertThat(resultExceed).hasSize(3); + List expectedList = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedList, resultExceed); + + String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); + String explain = getTableEnv().explainSql(querySql); + String expectedExplain = "limit=[1]"; + assertThat(explain).as("Explain should contain LimitPushDown").contains(expectedExplain); + List result = sql(querySql); + assertThat(result).hasSize(1); + assertThat(result).containsAnyElementsOf(expectedList); + + String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); + List mixedResult = sql(sqlMixed); + assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + } + + @TestTemplate + public void testNoFilterPushDown() { + String sql = String.format("SELECT * FROM %s ", TABLE_NAME); + List result = sql(sql); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedRecords, result); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + } + + @TestTemplate + public void testFilterPushDownEqual() { + String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") == 1"; + + List result = sql(sqlLiteralRight); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownEqualNull() { + String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); + + List result = sql(sqlEqualNull); + assertThat(result).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); + } + + @TestTemplate + public void testFilterPushDownEqualLiteralOnLeft() { + String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") == 1"; + + List resultLeft = sql(sqlLiteralLeft); + assertThat(resultLeft).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownNoEqual() { + String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") != 1"; + + List resultNE = sql(sqlNE); + assertThat(resultNE).hasSize(2); + + List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedNE, resultNE); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownNoEqualNull() { + String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); + + List resultNE = sql(sqlNotEqualNull); + assertThat(resultNE).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); + } + + @TestTemplate + public void testFilterPushDownAnd() { + String sqlAnd = + String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); + + List resultAnd = sql(sqlAnd); + assertThat(resultAnd).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); + } + + @TestTemplate + public void testFilterPushDownOr() { + String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; + + List resultOr = sql(sqlOr); + assertThat(resultOr).hasSize(2); + + List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedOR, resultOr); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownGreaterThan() { + String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") > 1"; + + List resultGT = sql(sqlGT); + assertThat(resultGT).hasSize(2); + + List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedGT, resultGT); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownGreaterThanNull() { + String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); + + List resultGT = sql(sqlGT); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); + } + + @TestTemplate + public void testFilterPushDownGreaterThanLiteralOnLeft() { + String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") < 3"; + + List resultGT = sql(sqlGT); + assertThat(resultGT).hasSize(2); + + List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedGT, resultGT); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownGreaterThanEqual() { + String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") >= 2"; + + List resultGTE = sql(sqlGTE); + assertThat(resultGTE).hasSize(2); + + List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedGTE, resultGTE); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownGreaterThanEqualNull() { + String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); + + List resultGT = sql(sqlGTE); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); + } + + @TestTemplate + public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { + String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") <= 2"; + + List resultGTE = sql(sqlGTE); + assertThat(resultGTE).hasSize(2); + + List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedGTE, resultGTE); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownLessThan() { + String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") < 2"; + + List resultLT = sql(sqlLT); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownLessThanNull() { + String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); + + List resultGT = sql(sqlLT); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); + } + + @TestTemplate + public void testFilterPushDownLessThanLiteralOnLeft() { + String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") > 2"; + + List resultLT = sql(sqlLT); + assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownLessThanEqual() { + String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") <= 1"; + + List resultLTE = sql(sqlLTE); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownLessThanEqualNull() { + String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); + + List resultGT = sql(sqlLTE); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent).as("Should not push down a filter").isNull(); + } + + @TestTemplate + public void testFilterPushDownLessThanEqualLiteralOnLeft() { + String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); + String expectedFilter = "ref(name=\"id\") >= 3"; + + List resultLTE = sql(sqlLTE); + assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownIn() { + String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; + List resultIN = sql(sqlIN); + assertThat(resultIN).hasSize(2); + + List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedIN, resultIN); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownInNull() { + String sqlInNull = + String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); + + List result = sql(sqlInNull); + assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + + // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it + // and push the rest down. + String expectedScan = "ref(name=\"data\") == \"iceberg\""; + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); + } + + @TestTemplate + public void testFilterPushDownNotIn() { + String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); + + List resultNotIn = sql(sqlNotIn); + assertThat(resultNotIn).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); + } + + @TestTemplate + public void testFilterPushDownNotInNull() { + String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); + List resultGT = sql(sqlNotInNull); + assertThat(resultGT).isEmpty(); + assertThat(lastScanEvent) + .as( + "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.") + .isNull(); + } + + @TestTemplate + public void testFilterPushDownIsNotNull() { + String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); + String expectedFilter = "not_null(ref(name=\"data\"))"; + + List resultNotNull = sql(sqlNotNull); + assertThat(resultNotNull).hasSize(2); + + List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expected, resultNotNull); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownIsNull() { + String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); + String expectedFilter = "is_null(ref(name=\"data\"))"; + + List resultNull = sql(sqlNull); + assertThat(resultNull).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownNot() { + String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); + + List resultNot = sql(sqlNot); + assertThat(resultNot).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); + + assertThat(scanEventCount).isEqualTo(1); + String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownBetween() { + String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); + + List resultBetween = sql(sqlBetween); + assertThat(resultBetween).hasSize(2); + + List expectedBetween = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedBetween, resultBetween); + + assertThat(scanEventCount).isEqualTo(1); + String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expected); + } + + @TestTemplate + public void testFilterPushDownNotBetween() { + String sqlNotBetween = + String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); + String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; + + List resultNotBetween = sql(sqlNotBetween); + assertThat(resultNotBetween).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + } + + @TestTemplate + public void testFilterPushDownLike() { + String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; + + String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; + List resultLike = sql(sqlLike); + assertThat(resultLike).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); + assertThat(scanEventCount).isEqualTo(1); + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedFilter); + + // %% won't match the row with null value + sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; + resultLike = sql(sqlLike); + assertThat(resultLike).hasSize(2); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); + assertSameElements(expectedRecords, resultLike); + String expectedScan = "not_null(ref(name=\"data\"))"; + assertThat(lastScanEvent.filter()) + .as("Should contain the push down filter") + .asString() + .isEqualTo(expectedScan); + } + + @TestTemplate + public void testFilterNotPushDownLike() { + Row expectRecord = Row.of(1, "iceberg", 10.0); + String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; + List resultLike = sql(sqlNoPushDown); + assertThat(resultLike).isEmpty(); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; + resultLike = sql(sqlNoPushDown); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; + resultLike = sql(sqlNoPushDown); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; + resultLike = sql(sqlNoPushDown); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + + sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; + resultLike = sql(sqlNoPushDown); + assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + } + + @TestTemplate + public void testFilterPushDown2Literal() { + String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); + List result = sql(sql2Literal); + List expectedRecords = + Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); + assertSameElements(expectedRecords, result); + assertThat(lastScanEvent.filter()) + .as("Should not push down a filter") + .isEqualTo(Expressions.alwaysTrue()); + } + + @TestTemplate + public void testSqlParseNaN() { + // todo add some test case to test NaN + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java new file mode 100644 index 000000000000..b7447d15c05a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -0,0 +1,147 @@ +/* + * 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 static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +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.table.api.TableColumn; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +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.TestTemplate; + +public class TestIcebergSourceBounded extends TestFlinkScan { + @TestTemplate + public void testValidation() { + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); + + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(false) + .endTag("tag") + .endSnapshotId(1L) + .build()) + .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") + .isInstanceOf(IllegalArgumentException.class); + } + + @Override + protected List runWithProjection(String... projected) throws Exception { + Schema icebergTableSchema = + CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); + TableSchema.Builder builder = TableSchema.builder(); + TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); + for (String field : projected) { + TableColumn column = schema.getTableColumn(field).get(); + builder.field(column.getName(), column.getType()); + } + TableSchema flinkSchema = builder.build(); + Schema projectedSchema = FlinkSchemaUtil.convert(icebergTableSchema, flinkSchema); + return run(projectedSchema, Lists.newArrayList(), Maps.newHashMap(), "", projected); + } + + @Override + protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) + throws Exception { + Map options = Maps.newHashMap(); + options.put("case-sensitive", Boolean.toString(caseSensitive)); + return run(null, Collections.singletonList(filter), options, sqlFilter, "*"); + } + + @Override + protected List runWithOptions(Map options) throws Exception { + return run(null, Lists.newArrayList(), options, "", "*"); + } + + @Override + protected List run() throws Exception { + return run(null, Lists.newArrayList(), Maps.newHashMap(), "", "*"); + } + + protected List run( + Schema projectedSchema, + List filters, + Map options, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + 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(); + } + + IcebergSource.Builder sourceBuilder = + IcebergSource.forRowData() + .tableLoader(tableLoader()) + .table(table) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.properties(options); + + DataStream stream = + env.fromSource( + sourceBuilder.build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + TypeInformation.of(RowData.class)) + .map( + new RowDataToRowMapper( + FlinkSchemaUtil.convert( + projectedSchema == null ? table.schema() : projectedSchema))); + + try (CloseableIterator iter = stream.executeAndCollect()) { + return Lists.newArrayList(iter); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java new file mode 100644 index 000000000000..7bfed00a9eb4 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -0,0 +1,196 @@ +/* + * 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.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.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.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.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); + + @Parameters(name = "format={0}, parallelism = {1}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 2}, + {FileFormat.PARQUET, 2}, + {FileFormat.ORC, 2} + }; + } + + @Parameter(index = 0) + private FileFormat fileFormat; + + @Parameter(index = 1) + private int parallelism; + + @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 = + 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); + } + + @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(); + } + + AvroGenericRecordReaderFunction readerFunction = + new AvroGenericRecordReaderFunction( + TestFixtures.TABLE_IDENTIFIER.name(), + new Configuration(), + table.schema(), + null, + null, + false, + table.io(), + table.encryption(), + filters); + + IcebergSource.Builder sourceBuilder = + IcebergSource.builder() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .readerFunction(readerFunction) + .assignerFactory(new SimpleSplitAssignerFactory()) + .flinkConfig(config); + if (projectedSchema != null) { + sourceBuilder.project(projectedSchema); + } + + sourceBuilder.filters(filters); + sourceBuilder.setAll(options); + + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + RowType rowType = FlinkSchemaUtil.convert(readSchema); + 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); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java new file mode 100644 index 000000000000..0f41c5af4c95 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -0,0 +1,76 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.junit.jupiter.api.BeforeEach; + +public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { + private volatile TableEnvironment tEnv; + + @BeforeEach + public void before() throws IOException { + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + SqlHelpers.sql( + getTableEnv(), + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_EXTENSION.warehouse()); + SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); + getTableEnv() + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @Override + protected List run( + Schema projectedSchema, + List filters, + Map options, + String sqlFilter, + String... sqlSelectedFields) + throws Exception { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(options); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java new file mode 100644 index 000000000000..749cbf89338a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java @@ -0,0 +1,573 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.nio.file.Path; +import java.time.Duration; +import java.util.Collection; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.JobStatus; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.runtime.client.JobStatusMessage; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceContinuous { + + public static final InMemoryReporter METRIC_REPORTER = InMemoryReporter.create(); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(METRIC_REPORTER); + + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + private final AtomicLong randomSeed = new AtomicLong(0L); + + @Test + public void testTableScanThenIncremental() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + // snapshot1 + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testTableScanThenIncrementalAfterExpiration() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + // snapshot1 + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); + + assertThat(TABLE_EXTENSION.table().history()).hasSize(1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) + .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 4); + List initialRecords = Lists.newArrayList(); + initialRecords.addAll(batch1); + initialRecords.addAll(batch2); + TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + TABLE_EXTENSION.table().currentSnapshot().snapshotId(); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testEarliestSnapshot() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + // snapshot0 + List batch0 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + + // snapshot1 + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 4); + List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); + combinedBatch0AndBatch1.addAll(batch1); + TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testLatestSnapshot(@InjectClusterClient ClusterClient clusterClient) + throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + // snapshot0 + List batch0 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + + // snapshot1 + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + // we want to make sure job is running first so that enumerator can + // start from the latest snapshot before inserting the next batch2 below. + waitUntilJobIsRunning(clusterClient); + + // inclusive behavior for starting snapshot + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testSpecificSnapshotId() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + // snapshot0 + List batch0 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); + + // snapshot1 + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot1) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testSpecificSnapshotTimestamp() throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + // snapshot0 + List batch0 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch0); + long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); + + // sleep for 2 ms to make sure snapshot1 has a higher timestamp value + Thread.sleep(2); + + // snapshot1 + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch1); + long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot1Timestamp) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + // consume data from snapshot1 + List result1 = waitForResult(iter, 2); + TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); + + // snapshot2 + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch2); + + List result2 = waitForResult(iter, 2); + TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); + + // snapshot3 + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + assertThatIcebergEnumeratorMetricsExist(); + } + } + + @Test + public void testReadingFromBranch() throws Exception { + String branch = "b1"; + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); + + List batchBase = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batchBase); + + // create branch + TABLE_EXTENSION + .table() + .manageSnapshots() + .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) + .commit(); + + // snapshot1 to branch + List batch1 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch1); + + // snapshot2 to branch + List batch2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch2); + + List branchExpectedRecords = Lists.newArrayList(); + branchExpectedRecords.addAll(batchBase); + branchExpectedRecords.addAll(batch1); + branchExpectedRecords.addAll(batch2); + // reads from branch: it should contain the first snapshot (before the branch creation) followed + // by the next 2 snapshots added + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .useBranch(branch) + .build(); + + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List resultMain = waitForResult(iter, 6); + TestHelpers.assertRecords( + resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); + + // snapshot3 to branch + List batch3 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch3); + + List result3 = waitForResult(iter, 2); + TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); + + // snapshot4 to branch + List batch4 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(branch, batch4); + + List result4 = waitForResult(iter, 2); + TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); + } + + // read only from main branch. Should contain only the first snapshot + scanContext = + ScanContext.builder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10L)) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + try (CloseableIterator iter = + createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { + List resultMain = waitForResult(iter, 2); + TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); + + List batchMain2 = + RandomGenericData.generate( + TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); + dataAppender.appendToTable(batchMain2); + resultMain = waitForResult(iter, 2); + TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); + } + } + + @Test + public void testValidation() { + assertThatThrownBy( + () -> + IcebergSource.forRowData() + .tableLoader(TABLE_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(true) + .endTag("tag") + .build()) + .hasMessage("Cannot set end-tag option for streaming reader") + .isInstanceOf(IllegalArgumentException.class); + } + + private DataStream createStream(ScanContext scanContext) throws Exception { + // start the source and collect output + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream stream = + env.fromSource( + IcebergSource.forRowData() + .tableLoader(TABLE_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + .streaming(scanContext.isStreaming()) + .streamingStartingStrategy(scanContext.streamingStartingStrategy()) + .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) + .startSnapshotId(scanContext.startSnapshotId()) + .monitorInterval(Duration.ofMillis(10L)) + .branch(scanContext.branch()) + .build(), + WatermarkStrategy.noWatermarks(), + "icebergSource", + TypeInformation.of(RowData.class)) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); + return stream; + } + + public static List waitForResult(CloseableIterator iter, int limit) { + List results = Lists.newArrayListWithCapacity(limit); + while (results.size() < limit) { + if (iter.hasNext()) { + results.add(iter.next()); + } else { + break; + } + } + return results; + } + + public static void waitUntilJobIsRunning(ClusterClient client) { + Awaitility.await("job should be running") + .atMost(Duration.ofSeconds(30)) + .pollInterval(Duration.ofMillis(10)) + .untilAsserted(() -> assertThat(getRunningJobs(client)).isNotEmpty()); + } + + public static List getRunningJobs(ClusterClient client) throws Exception { + Collection statusMessages = client.listJobs().get(); + return statusMessages.stream() + .filter(status -> status.getJobState() == JobStatus.RUNNING) + .map(JobStatusMessage::getJobId) + .collect(Collectors.toList()); + } + + private static void assertThatIcebergEnumeratorMetricsExist() { + assertThatIcebergSourceMetricExists( + "enumerator", "coordinator.enumerator.elapsedSecondsSinceLastSplitDiscovery"); + assertThatIcebergSourceMetricExists("enumerator", "coordinator.enumerator.unassignedSplits"); + assertThatIcebergSourceMetricExists("enumerator", "coordinator.enumerator.pendingRecords"); + } + + private static void assertThatIcebergSourceMetricExists( + String metricGroupPattern, String metricName) { + Optional groups = METRIC_REPORTER.findGroup(metricGroupPattern); + assertThat(groups).isPresent(); + assertThat( + METRIC_REPORTER.getMetricsByGroup(groups.get()).keySet().stream() + .map(name -> groups.get().getMetricIdentifier(name))) + .satisfiesOnlyOnce( + fullMetricName -> assertThat(fullMetricName).containsSubsequence(metricName)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java new file mode 100644 index 000000000000..938ae4d9bb0a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java @@ -0,0 +1,394 @@ +/* + * 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 static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.time.Duration; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.restartstrategy.RestartStrategies; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.client.program.ClusterClient; +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.highavailability.nonha.embedded.HaLeadershipControl; +import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectClusterClient; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.test.util.MiniClusterWithClientResource; +import org.apache.flink.util.function.ThrowingConsumer; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.FlinkSink; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +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; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 120) +public class TestIcebergSourceFailover { + + // Parallelism higher than 1, but lower than the number of splits used by some of our tests + // The goal is to allow some splits to remain in the enumerator when restoring the state + private static final int PARALLELISM = 2; + private static final int DO_NOT_FAIL = Integer.MAX_VALUE; + protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .withHaLeadershipControl() + .build(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @RegisterExtension + protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); + + protected Table sourceTable; + protected Table sinkTable; + + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); + } + + protected IcebergSource.Builder sourceBuilder() { + Configuration config = new Configuration(); + return IcebergSource.forRowData() + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) + .assignerFactory(new SimpleSplitAssignerFactory()) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) + .flinkConfig(config); + } + + protected Schema schema() { + return TestFixtures.SCHEMA; + } + + protected List generateRecords(int numRecords, long seed) { + return RandomGenericData.generate(schema(), numRecords, seed); + } + + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); + } + + @Disabled("Disabled for now as it is flaky on CI") + @Test + public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) + throws Exception { + List expectedRecords = Lists.newArrayList(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + createBoundedStreams(env, 2); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Savepoint Test"); + JobID jobId = jobClient.getJobID(); + + // Write something, but do not finish before checkpoint is created + RecordCounterToWait.waitForCondition(); + CompletableFuture savepoint = + clusterClient.stopWithSavepoint( + jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); + RecordCounterToWait.continueProcessing(); + + // Wait for the job to stop with the savepoint + String savepointPath = savepoint.get(); + + // We expect that at least a few records has written + assertThat(tableRecords(sinkTable)).hasSizeGreaterThan(0); + + // New env from the savepoint + Configuration conf = new Configuration(); + conf.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); + env = StreamExecutionEnvironment.getExecutionEnvironment(conf); + createBoundedStreams(env, DO_NOT_FAIL); + + env.execute("Bounded Iceberg Source Savepoint Test"); + + // We expect no duplications + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); + } + + @Test + public void testBoundedWithTaskManagerFailover() throws Exception { + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); + } + + @Test + public void testBoundedWithJobManagerFailover() throws Exception { + runTestWithNewMiniCluster( + miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); + } + + private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { + List expectedRecords = Lists.newArrayList(); + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); + for (int i = 0; i < 4; ++i) { + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + } + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); + createBoundedStreams(env, 2); + + JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); + JobID jobId = jobClient.getJobID(); + + RecordCounterToWait.waitForCondition(); + triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); + + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); + } + + @Test + public void testContinuousWithTaskManagerFailover() throws Exception { + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); + } + + @Test + public void testContinuousWithJobManagerFailover() throws Exception { + runTestWithNewMiniCluster( + miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); + } + + private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) + throws Exception { + GenericAppenderHelper dataAppender = + new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); + List expectedRecords = Lists.newArrayList(); + + List batch = generateRecords(2, 0); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + env.enableCheckpointing(10L); + Configuration config = new Configuration(); + config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); + + DataStream stream = + env.fromSource( + sourceBuilder() + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(stream) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) + .append(); + + JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); + JobID jobId = jobClient.getJobID(); + + for (int i = 1; i < 5; i++) { + Thread.sleep(10); + List records = generateRecords(2, i); + expectedRecords.addAll(records); + dataAppender.appendToTable(records); + if (i == 2) { + triggerFailover(failoverType, jobId, () -> {}, miniCluster); + } + } + + // wait longer for continuous source to reduce flakiness + // because CI servers tend to be overloaded. + assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); + } + + private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { + env.setParallelism(PARALLELISM); + + DataStream stream = + env.fromSource( + sourceBuilder().build(), + WatermarkStrategy.noWatermarks(), + "IcebergSource", + TypeInformation.of(RowData.class)); + + DataStream streamFailingInTheMiddleOfReading = + RecordCounterToWait.wrapWithFailureAfter(stream, failAfter); + + // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee + // exactly-once behavior. When Iceberg sink, we can verify end-to-end + // exactly-once. Here we mainly about source exactly-once behavior. + FlinkSink.forRowData(streamFailingInTheMiddleOfReading) + .table(sinkTable) + .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) + .append(); + } + + // ------------------------------------------------------------------------ + // test utilities copied from Flink's FileSourceTextLinesITCase + // ------------------------------------------------------------------------ + + private static void runTestWithNewMiniCluster(ThrowingConsumer testMethod) + throws Exception { + MiniClusterWithClientResource miniCluster = null; + try { + miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); + miniCluster.before(); + testMethod.accept(miniCluster.getMiniCluster()); + } finally { + if (miniCluster != null) { + miniCluster.after(); + } + } + } + + private enum FailoverType { + NONE, + TM, + JM + } + + private static void triggerFailover( + FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + switch (type) { + case NONE: + afterFailAction.run(); + break; + case TM: + restartTaskManager(afterFailAction, miniCluster); + break; + case JM: + triggerJobManagerFailover(jobId, afterFailAction, miniCluster); + break; + } + } + + private static void triggerJobManagerFailover( + JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { + HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); + haLeadershipControl.revokeJobMasterLeadership(jobId).get(); + afterFailAction.run(); + haLeadershipControl.grantJobMasterLeadership(jobId).get(); + } + + private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) + throws Exception { + miniCluster.terminateTaskManager(0).get(); + afterFailAction.run(); + miniCluster.startTaskManager(); + } + + private static class RecordCounterToWait { + + private static AtomicInteger records; + private static CountDownLatch countDownLatch; + private static CompletableFuture continueProcessing; + + private static DataStream wrapWithFailureAfter(DataStream stream, int condition) { + + records = new AtomicInteger(); + continueProcessing = new CompletableFuture<>(); + countDownLatch = new CountDownLatch(stream.getParallelism()); + return stream.map( + record -> { + boolean reachedFailPoint = records.incrementAndGet() > condition; + boolean notFailedYet = countDownLatch.getCount() != 0; + if (notFailedYet && reachedFailPoint) { + countDownLatch.countDown(); + continueProcessing.get(); + } + return record; + }); + } + + private static void waitForCondition() throws InterruptedException { + countDownLatch.await(); + } + + private static void continueProcessing() { + continueProcessing.complete(null); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java new file mode 100644 index 000000000000..4f61d2f7308a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.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.source; + +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.time.ZoneOffset; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkReadOptions; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.util.StructLikeWrapper; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; + +public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { + // Increment ts by 15 minutes for each generateRecords batch + private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); + // Within a batch, increment ts by 1 second + private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); + + private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); + + @Override + @BeforeEach + protected void setupTable() { + this.sourceTable = + SOURCE_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + this.sinkTable = + SINK_CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); + } + + @Override + protected IcebergSource.Builder sourceBuilder() { + Configuration config = new Configuration(); + return IcebergSource.forRowData() + .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + // Prevent combining splits + .set( + FlinkReadOptions.SPLIT_FILE_OPEN_COST, + Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) + .flinkConfig(config); + } + + @Override + protected Schema schema() { + return TestFixtures.TS_SCHEMA; + } + + @Override + protected List generateRecords(int numRecords, long seed) { + // Override the ts field to create a more realistic situation for event time alignment + tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); + return RandomGenericData.generate(schema(), numRecords, seed).stream() + .peek( + record -> { + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), + ZoneId.of("Z")); + record.setField("ts", ts); + }) + .collect(Collectors.toList()); + } + + /** + * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves + * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates + * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the + * {@link LocalDateTime} to a Long type so that Comparators can continue to work. + */ + @Override + protected void assertRecords(Table table, List expectedRecords, Duration timeout) + throws Exception { + List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); + Awaitility.await("expected list of records should be produced") + .atMost(timeout) + .untilAsserted( + () -> + SimpleDataUtil.assertRecordsEqual( + expectedNormalized, + convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), + table.schema())); + } + + private List convertLocalDateTimeToMilli(List records) { + return records.stream() + .peek( + r -> { + LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); + r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); + }) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java new file mode 100644 index 000000000000..df148c212ebd --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java @@ -0,0 +1,102 @@ +/* + * 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.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.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.util.CloseableIterator; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.CatalogLoader; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.RowDataWrapper; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; +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.StructLikeSet; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSourceReaderDeletes extends TestFlinkReaderDeletesBase { + + private static final int PARALLELISM = 4; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @Override + protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) + throws IOException { + Schema projected = testTable.schema().select(columns); + RowType rowType = FlinkSchemaUtil.convert(projected); + + Map properties = Maps.newHashMap(); + properties.put( + CatalogProperties.WAREHOUSE_LOCATION, + hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); + properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); + properties.put( + CatalogProperties.CLIENT_POOL_SIZE, + Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); + CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); + TableLoader hiveTableLoader = + TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName)); + hiveTableLoader.open(); + try (TableLoader tableLoader = hiveTableLoader) { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + DataStream stream = + env.fromSource( + IcebergSource.builder() + .tableLoader(tableLoader) + .assignerFactory(new SimpleSplitAssignerFactory()) + .project(projected) + .build(), + WatermarkStrategy.noWatermarks(), + "testBasicRead", + TypeInformation.of(RowData.class)); + + try (CloseableIterator iter = stream.executeAndCollect()) { + List rowDataList = Lists.newArrayList(iter); + StructLikeSet set = StructLikeSet.create(projected.asStruct()); + rowDataList.forEach( + rowData -> { + RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); + set.add(wrapper.wrap(rowData)); + }); + return set; + } catch (Exception e) { + throw new IOException("Failed to collect result", e); + } + } + } +} 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 new file mode 100644 index 000000000000..75f0a785a8c5 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java @@ -0,0 +1,158 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.IOException; +import java.time.Instant; +import java.time.ZoneId; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +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.Test; + +/** Use the IcebergSource (FLIP-27) */ +public class TestIcebergSourceSql extends TestSqlBase { + private static final Schema SCHEMA_TS = + new Schema( + required(1, "t1", Types.TimestampType.withoutZone()), + required(2, "t2", Types.LongType.get())); + + @Override + public void before() throws IOException { + TableEnvironment tableEnvironment = getTableEnv(); + Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); + tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + + tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); + SqlHelpers.sql( + tableEnvironment, + "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_EXTENSION.warehouse()); + SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); + + tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + } + + private Record generateRecord(Instant t1, long t2) { + Record record = GenericRecord.create(SCHEMA_TS); + record.setField("t1", t1.atZone(ZoneId.systemDefault()).toLocalDateTime()); + record.setField("t2", t2); + return record; + } + + /** Generates the records in the expected order, with respect to their datafile */ + private List generateExpectedRecords(boolean ascending) throws Exception { + Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); + long baseTime = 1702382109000L; + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + + Record file1Record1 = + generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); + Record file1Record2 = + generateRecord( + Instant.ofEpochMilli(baseTime - 10 * 1000L), baseTime + (1000 * 60 * 60 * 24 * 35L)); + + List recordsDataFile1 = Lists.newArrayList(); + recordsDataFile1.add(file1Record1); + recordsDataFile1.add(file1Record2); + DataFile dataFile1 = helper.writeFile(recordsDataFile1); + + Record file2Record1 = + generateRecord( + Instant.ofEpochMilli(baseTime + 14 * 1000L), baseTime - (1000 * 60 * 60 * 24 * 30L)); + Record file2Record2 = + generateRecord( + Instant.ofEpochMilli(baseTime + 12 * 1000L), baseTime - (1000 * 60 * 61 * 24 * 35L)); + + List recordsDataFile2 = Lists.newArrayList(); + recordsDataFile2.add(file2Record1); + recordsDataFile2.add(file2Record2); + + DataFile dataFile2 = helper.writeFile(recordsDataFile2); + helper.appendToTable(dataFile1, dataFile2); + + // Expected records if the splits are ordered + // - ascending (watermark from t1) - records from the split with early timestamps, then + // records from the split with late timestamps + // - descending (watermark from t2) - records from the split with old longs, then records + // from the split with new longs + List expected = Lists.newArrayList(); + if (ascending) { + expected.addAll(recordsDataFile1); + expected.addAll(recordsDataFile2); + } else { + expected.addAll(recordsDataFile2); + expected.addAll(recordsDataFile1); + } + return expected; + } + + /** Tests the order of splits returned when setting the watermark-column options */ + @Test + public void testWatermarkOptionsAscending() throws Exception { + List expected = generateExpectedRecords(true); + TestHelpers.assertRecordsWithOrder( + run( + ImmutableMap.of("watermark-column", "t1", "split-file-open-cost", "128000000"), + "", + "*"), + expected, + SCHEMA_TS); + } + + /** + * Tests the order of splits returned when setting the watermark-column and + * watermark-column-time-unit" options + */ + @Test + public void testWatermarkOptionsDescending() throws Exception { + List expected = generateExpectedRecords(false); + TestHelpers.assertRecordsWithOrder( + run( + ImmutableMap.of( + "watermark-column", + "t2", + "watermark-column-time-unit", + "MILLISECONDS", + "split-file-open-cost", + "128000000"), + "", + "*"), + expected, + SCHEMA_TS); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java new file mode 100644 index 000000000000..70889f4f76aa --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java @@ -0,0 +1,408 @@ +/* + * 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 static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; + +import java.io.Serializable; +import java.nio.file.Path; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.Optional; +import java.util.concurrent.ConcurrentMap; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.runtime.metrics.MetricNames; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.minicluster.RpcServiceSharing; +import org.apache.flink.runtime.testutils.CommonTestUtils; +import org.apache.flink.runtime.testutils.InMemoryReporter; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; +import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.test.junit5.InjectMiniCluster; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.util.CloseableIterator; +import org.apache.flink.util.Collector; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.TestFixtures; +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.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceWithWatermarkExtractor implements Serializable { + private static final int PARALLELISM = 4; + private static final String SOURCE_NAME = "IcebergSource"; + private static final int RECORD_NUM_FOR_2_SPLITS = 200; + private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); + + @TempDir protected Path temporaryFolder; + + private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(1) + .setNumberSlotsPerTaskManager(PARALLELISM) + .setRpcServiceSharing(RpcServiceSharing.DEDICATED) + .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .withHaLeadershipControl() + .build()); + + @RegisterExtension + private static final HadoopTableExtension TABLE_EXTENSION = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); + + /** + * This is an integration test for watermark handling and windowing. Integration testing the + * following features: + * + *
      + *
    • - Ordering of the splits + *
    • - Emitting of watermarks + *
    • - Firing windows based on watermarks + *
    + * + *

    The test generates 4 splits + * + *

      + *
    • - Split 1 - Watermark 100 min + *
    • - Split 2, 3 - Watermark 0 min + *
    • - Split 4 - Watermark 6 min + *
    + * + *

    Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency + * issues). + * + *

    Checks that windows are handled correctly based on the emitted watermarks, and splits are + * read in the following order: + * + *

      + *
    • - Split 2, 3 + *
    • - Split 4 + *
    • - Split 1 + *
    + * + *

    As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. + * + *

    Add 2 more splits, so the task manager close the windows for the original 4 splits and emit + * the appropriate aggregated records. + */ + @Test + public void testWindowing() throws Exception { + GenericAppenderHelper dataAppender = appender(); + List expectedRecords = Lists.newArrayList(); + + // Generate records with the following pattern: + // - File 1 - Later records (Watermark 6000000) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + // - File 2 - First records (Watermark 0) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - File 3 - Parallel write for the first records (Watermark 360000) + // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") + List batch = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), + generateRecord(101, "file_1-recordTs_101"), + generateRecord(103, "file_1-recordTs_103")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + // Generate records where the timestamps are out of order, but still between 0-5 minutes + batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + batch = + ImmutableList.of( + generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); + expectedRecords.addAll(batch); + dataAppender.appendToTable(batch); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(1); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withTimestampAssigner(new RowDataTimestampAssigner()), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + stream + .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) + .apply( + new AllWindowFunction() { + @Override + public void apply( + TimeWindow window, Iterable values, Collector out) { + // Emit RowData which contains the window start time, and the record count in + // that window + AtomicInteger count = new AtomicInteger(0); + values.forEach(a -> count.incrementAndGet()); + out.collect(row(window.getStart(), count.get())); + WINDOWS.put(window.getStart(), count.get()); + } + }); + + // Use static variable to collect the windows, since other solutions were flaky + WINDOWS.clear(); + env.executeAsync("Iceberg Source Windowing Test"); + + // Wait for the 2 first windows from File 2 and File 3 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + WINDOWS.equals( + ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); + + // Write data so the windows containing test data are closed + dataAppender.appendToTable( + dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); + + // Wait for last test record window from File 1 + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + WINDOWS.equals( + ImmutableMap.of( + 0L, + RECORD_NUM_FOR_2_SPLITS, + TimeUnit.MINUTES.toMillis(5), + 2, + TimeUnit.MINUTES.toMillis(100), + 3))); + } + + /** + * This is an integration test for watermark handling and throttling. Integration testing the + * following: + * + *

      + *
    • - Emitting of watermarks + *
    • - Watermark alignment + *
    + * + *

    The test generates 3 splits + * + *

      + *
    • - Split 1 - Watermark 100 min + *
    • - Split 2, 3 - Watermark 0 min + *
    + * + * The splits are read in the following order: + * + *
      + *
    • - Split 2, 3 (Task Manager 1, Task Manager 2) + *
    • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) + *
    + * + * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. + * + *

    The status of the watermark alignment is checked by the alignment related metrics. + * + *

    Adding new records with old timestamps to the table will enable the running reader to + * continue reading the files, but the watermark alignment will still prevent the paused reader to + * continue. + * + *

    After adding some records with new timestamps the blocked reader is un-paused, and both ot + * the readers continue reading. + */ + @Test + public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { + GenericAppenderHelper dataAppender = appender(); + + // Generate records in advance + + // File 1 - Later records (Watermark 6.000.000 - 100 min) + // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") + List batch1 = + ImmutableList.of( + generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); + + // File 2 - First records (Watermark 0 - 0 min) + // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... + List batch2 = Lists.newArrayListWithCapacity(100); + for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { + batch2.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); + } + + // File 3 - Some records will be blocked (Watermark 900.000 - 15 min) + List batch3 = + ImmutableList.of( + generateRecord(15, "file_3-recordTs_15"), + generateRecord(16, "file_3-recordTs_16"), + generateRecord(17, "file_3-recordTs_17")); + + // File 4 - Some records will be blocked (Watermark 900.000 - 15 min) + List batch4 = + ImmutableList.of( + generateRecord(15, "file_4-recordTs_15"), + generateRecord(16, "file_4-recordTs_16"), + generateRecord(17, "file_4-recordTs_17")); + + // File 5 - Records which will remove the block (Watermark 5.400.000 - 90 min) + List batch5 = + ImmutableList.of( + generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(2); + + DataStream stream = + env.fromSource( + source(), + WatermarkStrategy.noWatermarks() + .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), + SOURCE_NAME, + TypeInformation.of(RowData.class)); + + try (CloseableIterator resultIterator = stream.collectAsync()) { + JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); + CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); + + // Insert the first data into the table + dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); + + // Get the drift metric, wait for it to be created and reach the expected state + // (100 min - 20 min - 0 min) + // Also this validates that the WatermarkAlignment is working + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until( + () -> + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) + .isPresent()); + Gauge drift = + findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); + + // Add some old records with 2 splits, so even if the blocked gets one split, the other reader + // one gets one as well + dataAppender.appendToTable(dataAppender.writeFile(batch3), dataAppender.writeFile(batch4)); + + // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 + // min - 15 min) + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); + + // Add some new records which should unblock the throttled reader + dataAppender.appendToTable(batch5); + + // Wait for the new drift to decrease below the allowed drift to signal the normal state + Awaitility.await() + .pollInterval(Duration.ofMillis(10)) + .atMost(30, TimeUnit.SECONDS) + .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); + } + } + + protected IcebergSource source() { + return IcebergSource.builder() + .tableLoader(TABLE_EXTENSION.tableLoader()) + .watermarkColumn("ts") + .project(TestFixtures.TS_SCHEMA) + .splitSize(100L) + .streaming(true) + .monitorInterval(Duration.ofMillis(10)) + .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + } + + protected Record generateRecord(int minutes, String str) { + // Override the ts field to create a more realistic situation for event time alignment + Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); + LocalDateTime ts = + LocalDateTime.ofInstant( + Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), + ZoneId.of("Z")); + record.setField("ts", ts); + record.setField("str", str); + return record; + } + + private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { + String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; + return REPORTER.findMetrics(jobID, metricsName).values().stream() + .map(m -> (Gauge) m) + .filter(m -> m.getValue() == withValue) + .findFirst(); + } + + private GenericAppenderHelper appender() { + // We need to create multiple splits, so we need to generate parquet files with multiple offsets + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set("write.parquet.page-size-bytes", "64"); + hadoopConf.set("write.parquet.row-group-size-bytes", "64"); + return new GenericAppenderHelper( + TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); + } + + private static RowData row(long time, long count) { + GenericRowData result = new GenericRowData(2); + result.setField(0, time); + result.setField(1, String.valueOf(count)); + return result; + } + + private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { + @Override + public long extractTimestamp(RowData element, long recordTimestamp) { + return element.getTimestamp(0, 0).getMillisecond(); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java new file mode 100644 index 000000000000..41b023b93617 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -0,0 +1,184 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.functions.RichMapFunction; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.BatchExecutionOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.configuration.JobManagerOptions; +import org.apache.flink.configuration.RestOptions; +import org.apache.flink.configuration.SlowTaskDetectorOptions; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.Table; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestBase; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestIcebergSpeculativeExecutionSupport extends TestBase { + private static final int NUM_TASK_MANAGERS = 1; + private static final int NUM_TASK_SLOTS = 3; + + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TASK_MANAGERS) + .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) + .setConfiguration(configure()) + .build()); + + private StreamTableEnvironment tEnv; + private static final String CATALOG_NAME = "test_catalog"; + private static final String DATABASE_NAME = "test_db"; + private static final String INPUT_TABLE_NAME = "test_table"; + private static final String OUTPUT_TABLE_NAME = "sink_table"; + + @Override + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment(configure()); + env.setRuntimeMode(RuntimeExecutionMode.BATCH); + tEnv = StreamTableEnvironment.create(env); + } + } + + return tEnv; + } + + @BeforeEach + public void before() throws IOException { + String warehouse = + String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); + sql( + "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", + CATALOG_NAME, warehouse); + sql("USE CATALOG %s", CATALOG_NAME); + sql("CREATE DATABASE %s", DATABASE_NAME); + sql("USE %s", DATABASE_NAME); + + sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); + sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); + sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); + } + + @AfterEach + public void after() { + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); + sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); + dropDatabase(DATABASE_NAME, true); + dropCatalog(CATALOG_NAME, true); + } + + @Test + public void testSpeculativeExecution() throws Exception { + Table table = + tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); + DataStream slowStream = + tEnv.toDataStream(table, Row.class) + .map(new TestingMap()) + .name("test_map") + .returns( + Types.ROW_NAMED( + new String[] {"i", "j", "subTask", "attempt"}, + Types.INT, + Types.INT, + Types.INT, + Types.INT)) + .setParallelism(NUM_TASK_SLOTS); + + tEnv.fromDataStream(slowStream) + .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) + .await(); + + List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); + + // Ensure that all subTasks has attemptNum > 0 + assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); + + // Ensure the test_table rows are returned exactly the same after the slow map task from the + // sink_table + assertSameElements( + output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), + Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); + } + + /** A testing map function that simulates the slow task. */ + private static class TestingMap extends RichMapFunction { + @Override + public Row map(Row row) throws Exception { + // Put the subtasks with the first attempt to sleep to trigger speculative + // execution + if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + Thread.sleep(Integer.MAX_VALUE); + } + + Row output = + Row.of( + row.getField(0), + row.getField(1), + getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(), + getRuntimeContext().getTaskInfo().getAttemptNumber()); + + return output; + } + } + + private static Configuration configure() { + Configuration configuration = new Configuration(); + configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + configuration.set(RestOptions.BIND_PORT, "0"); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + + // Use FLIP-27 source + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + + // for speculative execution + configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); + + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); + configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); + configuration.set( + SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); + configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); + + return configuration; + } +} 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 new file mode 100644 index 000000000000..9cf953342a18 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java @@ -0,0 +1,299 @@ +/* + * 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 static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import java.nio.ByteBuffer; +import java.nio.file.Path; +import java.util.Base64; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.Parameters; +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.TableIdentifier; +import org.apache.iceberg.data.FileHelpers; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.TestHelpers; +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.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.io.TempDir; + +public class TestMetadataTableReadableMetrics extends CatalogTestBase { + private static final String TABLE_NAME = "test_table"; + + @Parameters(name = "catalogName={0}, baseNamespace={1}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + String catalogName = "testhive"; + Namespace baseNamespace = Namespace.empty(); + parameters.add(new Object[] {catalogName, baseNamespace}); + return parameters; + } + + @Override + protected TableEnvironment getTableEnv() { + Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); + configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); + return super.getTableEnv(); + } + + private @TempDir Path temp; + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema PRIMITIVE_SCHEMA = + new Schema( + required(1, "booleanCol", Types.BooleanType.get()), + required(2, "intCol", Types.IntegerType.get()), + required(3, "longCol", Types.LongType.get()), + required(4, "floatCol", Types.FloatType.get()), + required(5, "doubleCol", Types.DoubleType.get()), + optional(6, "decimalCol", Types.DecimalType.of(10, 2)), + optional(7, "stringCol", Types.StringType.get()), + optional(8, "fixedCol", Types.FixedType.ofLength(3)), + optional(9, "binaryCol", Types.BinaryType.get())); + + private Table createPrimitiveTable() throws IOException { + Table table = + catalog.createTable( + TableIdentifier.of(DATABASE, TABLE_NAME), + PRIMITIVE_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + List records = + Lists.newArrayList( + createPrimitiveRecord( + false, + 1, + 1L, + 0, + 1.0D, + new BigDecimal("1.00"), + "1", + Base64.getDecoder().decode("1111"), + ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), + createPrimitiveRecord( + true, + 2, + 2L, + 0, + 2.0D, + new BigDecimal("2.00"), + "2", + Base64.getDecoder().decode("2222"), + ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), + createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), + createPrimitiveRecord( + false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); + table.newAppend().appendFile(dataFile).commit(); + return table; + } + + private void createNestedTable() throws IOException { + Table table = + validationCatalog.createTable( + TableIdentifier.of(DATABASE, TABLE_NAME), + NESTED_SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + + List records = + Lists.newArrayList( + createNestedRecord(0L, 0.0), + createNestedRecord(1L, Double.NaN), + createNestedRecord(null, null)); + + File testFile = File.createTempFile("junit", null, temp.toFile()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); + table.newAppend().appendFile(dataFile).commit(); + } + + @BeforeEach + public void before() { + super.before(); + sql("USE CATALOG %s", catalogName); + sql("CREATE DATABASE %s", DATABASE); + sql("USE %s", DATABASE); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + protected GenericRecord createPrimitiveRecord( + boolean booleanCol, + int intCol, + long longCol, + float floatCol, + double doubleCol, + BigDecimal decimalCol, + String stringCol, + byte[] fixedCol, + ByteBuffer binaryCol) { + GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); + record.set(0, booleanCol); + record.set(1, intCol); + record.set(2, longCol); + record.set(3, floatCol); + record.set(4, doubleCol); + record.set(5, decimalCol); + record.set(6, stringCol); + record.set(7, fixedCol); + record.set(8, binaryCol); + return record; + } + + private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { + GenericRecord record = GenericRecord.create(NESTED_SCHEMA); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, longCol); + leaf.set(1, doubleCol); + nested.set(0, leaf); + record.set(0, nested); + return record; + } + + protected Object[] row(Object... values) { + return values; + } + + @TestTemplate + public void testPrimitiveColumns() throws Exception { + createPrimitiveTable(); + List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); + + Row binaryCol = + Row.of( + 52L, + 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 fixedCol = + Row.of( + 44L, + 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"); + + List expected = + Lists.newArrayList( + Row.of( + Row.of( + binaryCol, + booleanCol, + decimalCol, + doubleCol, + fixedCol, + floatCol, + intCol, + longCol, + stringCol))); + TestHelpers.assertRows(result, expected); + } + + @TestTemplate + public void testSelectPrimitiveValues() throws Exception { + createPrimitiveTable(); + + TestHelpers.assertRows( + sql( + "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s$files", + TABLE_NAME), + ImmutableList.of(Row.of(1, true))); + + TestHelpers.assertRows( + sql("SELECT content, readable_metrics.longCol.value_count FROM %s$files", TABLE_NAME), + ImmutableList.of(Row.of(0, 4L))); + + TestHelpers.assertRows( + sql("SELECT readable_metrics.longCol.value_count, content FROM %s$files", TABLE_NAME), + ImmutableList.of(Row.of(4L, 0))); + } + + @TestTemplate + public void testSelectNestedValues() throws Exception { + createNestedTable(); + TestHelpers.assertRows( + sql( + "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " + + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s$files", + TABLE_NAME), + ImmutableList.of(Row.of(0L, 3L))); + } + + @TestTemplate + 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 metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); + + TestHelpers.assertRows( + sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java new file mode 100644 index 000000000000..ce9054ad49b6 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java @@ -0,0 +1,188 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.data.RowDataProjection; +import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; +import org.apache.iceberg.flink.sink.TaskWriterFactory; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +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.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestProjectMetaColumn { + + @TempDir protected Path temporaryFolder; + + @Parameter(index = 0) + private FileFormat format; + + @Parameters(name = "fileFormat={0}") + public static Iterable parameters() { + return Lists.newArrayList( + new Object[] {FileFormat.PARQUET}, + new Object[] {FileFormat.ORC}, + new Object[] {FileFormat.AVRO}); + } + + private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { + // Create the table with given format version. + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); + Table table = + SimpleDataUtil.createTable( + location, + ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), + false); + + List rows = + Lists.newArrayList( + SimpleDataUtil.createInsert(1, "AAA"), + SimpleDataUtil.createInsert(2, "BBB"), + SimpleDataUtil.createInsert(3, "CCC")); + writeAndCommit(table, ImmutableList.of(), false, rows); + + FlinkInputFormat input = + FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); + + List results = Lists.newArrayList(); + TestHelpers.readRowData( + input, + rowData -> { + // If project to remove the meta columns, it will get a RowDataProjection. + assertThat(rowData).isInstanceOf(GenericRowData.class); + results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); + }); + + // Assert the results. + TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); + } + + @TestTemplate + public void testV1SkipToRemoveMetaColumn() throws IOException { + testSkipToRemoveMetaColumn(1); + } + + @TestTemplate + public void testV2SkipToRemoveMetaColumn() throws IOException { + testSkipToRemoveMetaColumn(2); + } + + @TestTemplate + public void testV2RemoveMetaColumn() throws Exception { + // Create the v2 table. + String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); + Table table = + SimpleDataUtil.createTable( + location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); + + List rows = + Lists.newArrayList( + SimpleDataUtil.createInsert(1, "AAA"), + SimpleDataUtil.createDelete(1, "AAA"), + SimpleDataUtil.createInsert(2, "AAA"), + SimpleDataUtil.createInsert(2, "BBB")); + int eqFieldId = table.schema().findField("data").fieldId(); + writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); + + FlinkInputFormat input = + FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); + + List results = Lists.newArrayList(); + TestHelpers.readRowData( + input, + rowData -> { + // If project to remove the meta columns, it will get a RowDataProjection. + assertThat(rowData).isInstanceOf(RowDataProjection.class); + results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); + }); + + // Assert the results. + TestHelpers.assertRows( + ImmutableList.of( + SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")), + results, + SimpleDataUtil.ROW_TYPE); + } + + private void writeAndCommit( + Table table, List eqFieldIds, boolean upsert, List rows) + throws IOException { + TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); + try (TaskWriter io = writer) { + for (RowData row : rows) { + io.write(row); + } + } + + RowDelta delta = table.newRowDelta(); + WriteResult result = writer.complete(); + + for (DataFile dataFile : result.dataFiles()) { + delta.addRows(dataFile); + } + + for (DeleteFile deleteFile : result.deleteFiles()) { + delta.addDeletes(deleteFile); + } + + delta.commit(); + } + + private TaskWriter createTaskWriter( + Table table, List equalityFieldIds, boolean upsert) { + TaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + SerializableTable.copyOf(table), + SimpleDataUtil.ROW_TYPE, + TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, + format, + table.properties(), + equalityFieldIds, + upsert); + + taskWriterFactory.initialize(1, 1); + return taskWriterFactory.create(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java new file mode 100644 index 000000000000..6ef40693827e --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.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.flink.source; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.apache.avro.generic.GenericRecord; +import org.apache.iceberg.flink.AvroGenericRecordConverterBase; +import org.apache.iceberg.flink.DataGenerator; + +public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { + @Override + protected void testConverter(DataGenerator dataGenerator) { + RowDataToAvroGenericRecordConverter converter = + RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); + GenericRecord expected = dataGenerator.generateAvroGenericRecord(); + GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); + assertThat(actual).isEqualTo(expected); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java new file mode 100644 index 000000000000..5dd7de545e11 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.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.flink.source; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.junit.jupiter.api.Test; + +class TestScanContext { + @Test + void testIncrementalFromSnapshotId() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testIncrementalFromSnapshotTimestamp() { + ScanContext context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .build(); + assertException( + context, + "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); + + context = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotId(1L) + .startSnapshotTimestamp(1L) + .build(); + assertException( + context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); + } + + @Test + void testStreaming() { + ScanContext context = ScanContext.builder().streaming(true).useTag("tag").build(); + assertException(context, "Cannot scan table using ref tag configured for streaming reader"); + + context = ScanContext.builder().streaming(true).useSnapshotId(1L).build(); + assertException(context, "Cannot set snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build(); + assertException(context, "Cannot set as-of-timestamp option for streaming reader"); + + context = ScanContext.builder().streaming(true).endSnapshotId(1L).build(); + assertException(context, "Cannot set end-snapshot-id option for streaming reader"); + + context = ScanContext.builder().streaming(true).endTag("tag").build(); + assertException(context, "Cannot set end-tag option for streaming reader"); + } + + @Test + void testStartConflict() { + ScanContext context = ScanContext.builder().startTag("tag").startSnapshotId(1L).build(); + assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be set."); + } + + @Test + void testEndConflict() { + ScanContext context = ScanContext.builder().endTag("tag").endSnapshotId(1L).build(); + assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be set."); + } + + @Test + void testMaxAllowedPlanningFailures() { + ScanContext context = ScanContext.builder().maxAllowedPlanningFailures(-2).build(); + assertException( + context, "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); + } + + private void assertException(ScanContext context, String message) { + assertThatThrownBy(() -> context.validate()) + .hasMessage(message) + .isInstanceOf(IllegalArgumentException.class); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java new file mode 100644 index 000000000000..b701419a7499 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java @@ -0,0 +1,61 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import org.apache.flink.configuration.Configuration; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.junit.jupiter.api.Test; + +public class TestSourceUtil { + @Test + public void testInferedParallelism() throws IOException { + Configuration configuration = new Configuration(); + // Empty table, infer parallelism should be at least 1 + int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); + assertThat(parallelism).isEqualTo(1); + + // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits + // num : 2 + parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); + assertThat(parallelism).isEqualTo(2); + + // 2 splits and limit is 1 , max infer parallelism is default 100, + // which is greater than splits num and limit, the parallelism is the limit value : 1 + parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); + assertThat(parallelism).isEqualTo(1); + + // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 + configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); + parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); + assertThat(parallelism).isEqualTo(1); + + // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : + // 1 + parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); + assertThat(parallelism).isEqualTo(1); + + // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 + configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); + parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); + assertThat(parallelism).isEqualTo(1); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java new file mode 100644 index 000000000000..f9b776397cfc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java @@ -0,0 +1,160 @@ +/* + * 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 static org.apache.iceberg.flink.TestFixtures.DATABASE; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +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; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** Test other more advanced usage of SQL. They don't need to run for every file format. */ +public abstract class TestSqlBase { + @RegisterExtension + public static MiniClusterExtension miniClusterExtension = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + public static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @TempDir protected Path temporaryFolder; + + private volatile TableEnvironment tEnv; + + protected TableEnvironment getTableEnv() { + if (tEnv == null) { + synchronized (this) { + if (tEnv == null) { + this.tEnv = + TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); + } + } + } + return tEnv; + } + + @BeforeEach + public abstract void before() throws IOException; + + @Test + public void testResiduals() throws Exception { + Table table = + CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + List writeRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + writeRecords.get(0).set(1, 123L); + writeRecords.get(0).set(2, "2020-03-20"); + writeRecords.get(1).set(1, 456L); + writeRecords.get(1).set(2, "2020-03-20"); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + + List expectedRecords = Lists.newArrayList(); + expectedRecords.add(writeRecords.get(0)); + + DataFile dataFile1 = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), writeRecords); + DataFile dataFile2 = + helper.writeFile( + TestHelpers.Row.of("2020-03-21", 0), + RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); + helper.appendToTable(dataFile1, dataFile2); + + org.apache.iceberg.flink.TestHelpers.assertRecords( + run(Maps.newHashMap(), "where dt='2020-03-20' and id=123", "*"), + expectedRecords, + TestFixtures.SCHEMA); + } + + @Test + public void testExposeLocality() throws Exception { + Table table = + CATALOG_EXTENSION + .catalog() + .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); + + TableLoader tableLoader = TableLoader.fromHadoopTable(table.location()); + List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 10, 0L); + expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); + + GenericAppenderHelper helper = + new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); + DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); + helper.appendToTable(dataFile); + + // test sql api + Configuration tableConf = getTableEnv().getConfig().getConfiguration(); + tableConf.setBoolean( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); + + List results = SqlHelpers.sql(getTableEnv(), "select * from t"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + + // test table api + tableConf.setBoolean( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), true); + FlinkSource.Builder builder = FlinkSource.forRowData().tableLoader(tableLoader).table(table); + + // When running with CI or local, `localityEnabled` will be false even if this configuration is + // enabled + assertThat(SourceUtil.isLocalityEnabled(table, tableConf, true)) + .as("Expose split locality info should be false.") + .isFalse(); + + results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); + org.apache.iceberg.flink.TestHelpers.assertRecords( + results, expectedRecords, TestFixtures.SCHEMA); + } + + protected List run( + Map options, String sqlFilter, String... sqlSelectedFields) { + String select = String.join(",", sqlSelectedFields); + String optionStr = SqlHelpers.sqlOptionsToString(options); + return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java new file mode 100644 index 000000000000..97ed4ca1e93f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java @@ -0,0 +1,434 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Iterator; +import java.util.List; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.config.TableConfigOptions; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.Table; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.CatalogTestBase; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; + +@Timeout(60) +public class TestStreamScanSql extends CatalogTestBase { + private static final String TABLE = "test_table"; + private static final FileFormat FORMAT = FileFormat.PARQUET; + + private volatile TableEnvironment tEnv; + + @Override + protected TableEnvironment getTableEnv() { + TableEnvironment tableEnv = tEnv; + if (tableEnv != null) { + return tableEnv; + } + synchronized (this) { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = + EnvironmentSettings.newInstance().inStreamingMode(); + + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + + StreamTableEnvironment streamTableEnv = + StreamTableEnvironment.create(env, settingsBuilder.build()); + streamTableEnv + .getConfig() + .getConfiguration() + .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); + tEnv = streamTableEnv; + } + } + return tEnv; + } + + @Override + @BeforeEach + public void before() { + super.before(); + sql("CREATE DATABASE %s", flinkDatabase); + sql("USE CATALOG %s", catalogName); + sql("USE %s", DATABASE); + } + + @Override + @AfterEach + public void clean() { + sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); + dropDatabase(flinkDatabase, true); + super.clean(); + } + + private void insertRows(String partition, Table table, Row... rows) throws IOException { + insertRows(partition, SnapshotRef.MAIN_BRANCH, table, rows); + } + + private void insertRows(String partition, String branch, Table table, Row... rows) + throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, FORMAT, temporaryDirectory); + + GenericRecord gRecord = GenericRecord.create(table.schema()); + List records = Lists.newArrayList(); + for (Row row : rows) { + records.add( + gRecord.copy( + "id", row.getField(0), + "data", row.getField(1), + "dt", row.getField(2))); + } + + if (partition != null) { + appender.appendToTable(TestHelpers.Row.of(partition, 0), branch, records); + } else { + appender.appendToTable(branch, records); + } + } + + private void insertRowsInBranch(String branch, Table table, Row... rows) throws IOException { + insertRows(null, branch, table, rows); + } + + private void insertRows(Table table, Row... rows) throws IOException { + insertRows(null, table, rows); + } + + private void assertRows(List expectedRows, Iterator iterator) { + for (Row expectedRow : expectedRows) { + assertThat(iterator).hasNext(); + Row actualRow = iterator.next(); + assertThat(actualRow.getArity()).isEqualTo(3); + assertThat(actualRow.getField(0)).isEqualTo(expectedRow.getField(0)); + assertThat(actualRow.getField(1)).isEqualTo(expectedRow.getField(1)); + assertThat(actualRow.getField(2)).isEqualTo(expectedRow.getField(2)); + } + } + + @TestTemplate + public void testUnPartitionedTable() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + + Row row1 = Row.of(1, "aaa", "2021-01-01"); + insertRows(table, row1); + assertRows(ImmutableList.of(row1), iterator); + + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row2); + assertRows(ImmutableList.of(row2), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testPartitionedTable() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR) PARTITIONED BY (dt)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + Row row1 = Row.of(1, "aaa", "2021-01-01"); + insertRows("2021-01-01", table, row1); + assertRows(ImmutableList.of(row1), iterator); + + Row row2 = Row.of(2, "bbb", "2021-01-02"); + insertRows("2021-01-02", table, row2); + assertRows(ImmutableList.of(row2), iterator); + + Row row3 = Row.of(1, "aaa", "2021-01-02"); + insertRows("2021-01-02", table, row3); + assertRows(ImmutableList.of(row3), iterator); + + Row row4 = Row.of(2, "bbb", "2021-01-01"); + insertRows("2021-01-01", table, row4); + assertRows(ImmutableList.of(row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testConsumeFromBeginning() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1, row2); + + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + try (CloseableIterator iterator = result.collect()) { + assertRows(ImmutableList.of(row1, row2), iterator); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + insertRows(table, row3); + assertRows(ImmutableList.of(row3), iterator); + + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row4); + assertRows(ImmutableList.of(row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + /** + * Insert records on the main branch. Then, insert in a named branch. Reads from the main branch + * and assert that the only records from main are returned + */ + public void testConsumeFilesFromMainBranch() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots on main branch + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + + insertRows(table, row1, row2); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName).commit(); + + // insert on the 'b1' branch + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + + insertRowsInBranch(branchName, table, row3, row4); + + // read from main + TableResult result = + exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); + + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row1, row2), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + /** + * Insert records on the main branch. Creates a named branch. Insert record on named branch. Then + * select from the named branch and assert all the records are returned. + */ + public void testConsumeFilesFromBranch() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots on main branch + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + + insertRows(table, row1, row2); + String branchName = "b1"; + table.manageSnapshots().createBranch(branchName).commit(); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", + TABLE, branchName); + + try (CloseableIterator iterator = result.collect()) { + assertRows(ImmutableList.of(row1, row2), iterator); + // insert on the 'b1' branch + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRowsInBranch(branchName, table, row3, row4); + assertRows(ImmutableList.of(row3, row4), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + /** + * Insert records on branch b1. Then insert record on b2. Then select from each branch and assert + * the correct records are returned + */ + public void testConsumeFilesFromTwoBranches() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + String branch1 = "b1"; + String branch2 = "b2"; + table.manageSnapshots().createBranch(branch1).commit(); + table.manageSnapshots().createBranch(branch2).commit(); + + // Produce two snapshots on main branch + Row row1Branch1 = Row.of(1, "b1", "2021-01-01"); + Row row2Branch1 = Row.of(2, "b1", "2021-01-01"); + + Row row1Branch2 = Row.of(2, "b2", "2021-01-01"); + Row row2Branch2 = Row.of(3, "b3", "2021-01-01"); + + insertRowsInBranch(branch1, table, row1Branch1, row2Branch1); + insertRowsInBranch(branch2, table, row1Branch2, row2Branch2); + + TableResult resultBranch1 = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", + TABLE, branch1); + + try (CloseableIterator iterator = resultBranch1.collect()) { + assertRows(ImmutableList.of(row1Branch1, row2Branch1), iterator); + Row another = Row.of(4, "ccc", "2021-01-01"); + insertRowsInBranch(branch1, table, another); + assertRows(ImmutableList.of(another), iterator); + } + + TableResult resultBranch2 = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", + TABLE, branch2); + try (CloseableIterator iterator = resultBranch2.collect()) { + assertRows(ImmutableList.of(row1Branch2, row2Branch2), iterator); + Row another = Row.of(4, "ccc", "2021-01-01"); + insertRowsInBranch(branch2, table, another); + assertRows(ImmutableList.of(another), iterator); + } + + resultBranch1.getJobClient().ifPresent(JobClient::cancel); + resultBranch2.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testConsumeFromStartSnapshotId() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots. + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1); + insertRows(table, row2); + + long startSnapshotId = table.currentSnapshot().snapshotId(); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row3, row4); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " + + "'start-snapshot-id'='%d')*/", + TABLE, startSnapshotId); + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row3, row4), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + } + + @TestTemplate + public void testConsumeFromStartTag() throws Exception { + sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); + Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); + + // Produce two snapshots. + Row row1 = Row.of(1, "aaa", "2021-01-01"); + Row row2 = Row.of(2, "bbb", "2021-01-01"); + insertRows(table, row1); + insertRows(table, row2); + + String tagName = "t1"; + long startSnapshotId = table.currentSnapshot().snapshotId(); + table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); + + Row row3 = Row.of(3, "ccc", "2021-01-01"); + Row row4 = Row.of(4, "ddd", "2021-01-01"); + insertRows(table, row3, row4); + + TableResult result = + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " + + "'start-tag'='%s')*/", + TABLE, tagName); + try (CloseableIterator iterator = result.collect()) { + // the start snapshot(row2) is exclusive. + assertRows(ImmutableList.of(row3, row4), iterator); + + Row row5 = Row.of(5, "eee", "2021-01-01"); + Row row6 = Row.of(6, "fff", "2021-01-01"); + insertRows(table, row5, row6); + assertRows(ImmutableList.of(row5, row6), iterator); + + Row row7 = Row.of(7, "ggg", "2021-01-01"); + insertRows(table, row7); + assertRows(ImmutableList.of(row7), iterator); + } + result.getJobClient().ifPresent(JobClient::cancel); + + assertThatThrownBy( + () -> + exec( + "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " + + "'start-snapshot-id'='%d' )*/", + TABLE, tagName, startSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); + } +} 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 new file mode 100644 index 000000000000..9c4f476b02b4 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java @@ -0,0 +1,402 @@ +/* + * 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 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.time.Duration; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.operators.StreamSource; +import org.apache.flink.streaming.api.watermark.Watermark; +import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +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.types.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingMonitorFunction extends TestBase { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; + private static final long WAIT_TIME_MILLIS = 10 * 1000L; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); + } + + @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()); + } + + private void runSourceFunctionInTask( + TestSourceContext sourceContext, StreamingMonitorFunction function) { + Thread task = + new Thread( + () -> { + try { + function.run(sourceContext); + } catch (Exception e) { + throw new RuntimeException(e); + } + }); + task.start(); + } + + @TestTemplate + public void testConsumeWithoutStartSnapshotId() throws Exception { + List> recordsList = generateRecordsAndCommitTxn(10); + ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @TestTemplate + public void testConsumeFromStartSnapshotId() throws Exception { + // Commit the first five transactions. + generateRecordsAndCommitTxn(5); + long startSnapshotId = table.currentSnapshot().snapshotId(); + + // Commit the next five transactions. + List> recordsList = generateRecordsAndCommitTxn(5); + + ScanContext scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .startSnapshotId(startSnapshotId) + .build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @TestTemplate + public void testConsumeFromStartTag() throws Exception { + // Commit the first five transactions. + generateRecordsAndCommitTxn(5); + long startSnapshotId = table.currentSnapshot().snapshotId(); + String tagName = "t1"; + table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); + + // Commit the next five transactions. + List> recordsList = generateRecordsAndCommitTxn(5); + + ScanContext scanContext = + ScanContext.builder().monitorInterval(Duration.ofMillis(100)).startTag(tagName).build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, function); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + function.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + } + + @TestTemplate + public void testCheckpointRestore() throws Exception { + List> recordsList = generateRecordsAndCommitTxn(10); + ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); + + StreamingMonitorFunction func = createFunction(scanContext); + OperatorSubtaskState state; + try (AbstractStreamOperatorTestHarness harness = createHarness(func)) { + harness.setup(); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, func); + + awaitExpectedSplits(sourceContext); + + state = harness.snapshot(1, 1); + + // Stop the stream task. + func.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); + } + + List> newRecordsList = generateRecordsAndCommitTxn(10); + StreamingMonitorFunction newFunc = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(newFunc)) { + harness.setup(); + // Recover to process the remaining snapshots. + harness.initializeState(state); + harness.open(); + + TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); + runSourceFunctionInTask(sourceContext, newFunc); + + awaitExpectedSplits(sourceContext); + + // Stop the stream task. + newFunc.close(); + + TestHelpers.assertRecords( + sourceContext.toRows(), Lists.newArrayList(Iterables.concat(newRecordsList)), SCHEMA); + } + } + + private void awaitExpectedSplits(TestSourceContext sourceContext) { + Awaitility.await("expected splits should be produced") + .atMost(Duration.ofMillis(WAIT_TIME_MILLIS)) + .untilAsserted( + () -> { + assertThat(sourceContext.latch.getCount()).isEqualTo(0); + assertThat(sourceContext.splits).as("Should produce the expected splits").hasSize(1); + }); + } + + @TestTemplate + public void testInvalidMaxPlanningSnapshotCount() { + ScanContext scanContext1 = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .maxPlanningSnapshotCount(0) + .build(); + + assertThatThrownBy(() -> createFunction(scanContext1)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("The max-planning-snapshot-count must be greater than zero"); + + ScanContext scanContext2 = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .maxPlanningSnapshotCount(-10) + .build(); + + assertThatThrownBy(() -> createFunction(scanContext2)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("The max-planning-snapshot-count must be greater than zero"); + } + + @TestTemplate + public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { + generateRecordsAndCommitTxn(10); + + // Use the oldest snapshot as starting to avoid the initial case. + long oldestSnapshotId = SnapshotUtil.oldestAncestor(table).snapshotId(); + + ScanContext scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(100)) + .splitSize(1000L) + .startSnapshotId(oldestSnapshotId) + .maxPlanningSnapshotCount(Integer.MAX_VALUE) + .build(); + + FlinkInputSplit[] expectedSplits = + FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); + + assertThat(expectedSplits).hasSize(9); + + // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the + // total splits number + for (int maxPlanningSnapshotCount : ImmutableList.of(1, 9, 15)) { + scanContext = + ScanContext.builder() + .monitorInterval(Duration.ofMillis(500)) + .startSnapshotId(oldestSnapshotId) + .splitSize(1000L) + .maxPlanningSnapshotCount(maxPlanningSnapshotCount) + .build(); + + StreamingMonitorFunction function = createFunction(scanContext); + try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { + harness.setup(); + harness.open(); + + CountDownLatch latch = new CountDownLatch(1); + TestSourceContext sourceContext = new TestSourceContext(latch); + function.sourceContext(sourceContext); + function.monitorAndForwardSplits(); + + if (maxPlanningSnapshotCount < 10) { + assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); + } + } + } + } + + private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { + List> expectedRecords = Lists.newArrayList(); + for (int i = 0; i < commitTimes; i++) { + List records = RandomGenericData.generate(SCHEMA, 100, 0L); + expectedRecords.add(records); + + // Commit those records to iceberg table. + writeRecords(records); + } + return expectedRecords; + } + + private void writeRecords(List records) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); + appender.appendToTable(records); + } + + private StreamingMonitorFunction createFunction(ScanContext scanContext) { + return new StreamingMonitorFunction( + TestTableLoader.of(tableDir.getAbsolutePath()), scanContext); + } + + private AbstractStreamOperatorTestHarness createHarness( + StreamingMonitorFunction function) throws Exception { + StreamSource streamSource = + new StreamSource<>(function); + return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0); + } + + private class TestSourceContext implements SourceFunction.SourceContext { + private final List splits = Lists.newArrayList(); + private final Object checkpointLock = new Object(); + private final CountDownLatch latch; + + TestSourceContext(CountDownLatch latch) { + this.latch = latch; + } + + @Override + public void collect(FlinkInputSplit element) { + splits.add(element); + latch.countDown(); + } + + @Override + public void collectWithTimestamp(FlinkInputSplit element, long timestamp) { + collect(element); + } + + @Override + public void emitWatermark(Watermark mark) {} + + @Override + public void markAsTemporarilyIdle() {} + + @Override + public Object getCheckpointLock() { + return checkpointLock; + } + + @Override + public void close() {} + + private List toRows() throws IOException { + FlinkInputFormat format = + FlinkSource.forRowData() + .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) + .buildFormat(); + + List rows = Lists.newArrayList(); + for (FlinkInputSplit split : splits) { + format.open(split); + + RowData element = null; + try { + while (!format.reachedEnd()) { + element = format.nextRecord(element); + rows.add(Row.of(element.getInt(0), element.getString(1).toString())); + } + } finally { + format.close(); + } + } + + return rows; + } + } +} 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 new file mode 100644 index 000000000000..1606ee9f9648 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java @@ -0,0 +1,293 @@ +/* + * 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 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.Collections; +import java.util.List; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.TimeCharacteristic; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; +import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; +import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.TestTableLoader; +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.Types; +import org.apache.iceberg.util.SnapshotUtil; +import org.apache.iceberg.util.ThreadPools; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestStreamingReaderOperator extends TestBase { + + private static final Schema SCHEMA = + new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "data", Types.StringType.get())); + private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; + + @Parameters(name = "formatVersion = {0}") + protected static List parameters() { + return Arrays.asList(1, 2); + } + + @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()); + } + + @TestTemplate + public void testProcessAllRecords() throws Exception { + List> expectedRecords = generateRecordsAndCommitTxn(10); + + List splits = generateSplits(); + assertThat(splits).hasSize(10); + + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + SteppingMailboxProcessor processor = createLocalMailbox(harness); + + List expected = Lists.newArrayList(); + for (int i = 0; i < splits.size(); i++) { + // Process this element to enqueue to mail-box. + harness.processElement(splits.get(i), -1); + + // Run the mail-box once to read all records from the given split. + assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); + + // Assert the output has expected elements. + expected.addAll(expectedRecords.get(i)); + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + } + } + + @TestTemplate + public void testTriggerCheckpoint() throws Exception { + // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading + // records from + // split1. + List> expectedRecords = generateRecordsAndCommitTxn(3); + + List splits = generateSplits(); + assertThat(splits).hasSize(3); + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + SteppingMailboxProcessor processor = createLocalMailbox(harness); + + harness.processElement(splits.get(0), ++timestamp); + harness.processElement(splits.get(1), ++timestamp); + harness.processElement(splits.get(2), ++timestamp); + + // Trigger snapshot state, it will start to work once all records from split0 are read. + processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); + + assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); + assertThat(processor.runMailboxStep()) + .as("Should have processed the snapshot state action") + .isTrue(); + + TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); + + // Read records from split1. + assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); + + // Read records from split2. + assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); + + TestHelpers.assertRecords( + readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); + } + } + + @TestTemplate + public void testCheckpointRestore() throws Exception { + List> expectedRecords = generateRecordsAndCommitTxn(15); + + List splits = generateSplits(); + assertThat(splits).hasSize(15); + + OperatorSubtaskState state; + List expected = Lists.newArrayList(); + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + harness.open(); + + // Enqueue all the splits. + for (FlinkInputSplit split : splits) { + harness.processElement(split, -1); + } + + // Read all records from the first five splits. + SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); + for (int i = 0; i < 5; i++) { + expected.addAll(expectedRecords.get(i)); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); + + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + + // Snapshot state now, there're 10 splits left in the state. + state = harness.snapshot(1, 1); + } + + expected.clear(); + try (OneInputStreamOperatorTestHarness harness = createReader()) { + harness.setup(); + // Recover to process the remaining splits. + harness.initializeState(state); + harness.open(); + + SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); + + for (int i = 5; i < 10; i++) { + expected.addAll(expectedRecords.get(i)); + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); + + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + + // Let's process the final 5 splits now. + for (int i = 10; i < 15; i++) { + expected.addAll(expectedRecords.get(i)); + harness.processElement(splits.get(i), 1); + + assertThat(localMailbox.runMailboxStep()) + .as("Should have processed the split#" + i) + .isTrue(); + TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); + } + } + } + + private List readOutputValues( + OneInputStreamOperatorTestHarness harness) { + List results = Lists.newArrayList(); + for (RowData rowData : harness.extractOutputValues()) { + results.add(Row.of(rowData.getInt(0), rowData.getString(1).toString())); + } + return results; + } + + private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { + List> expectedRecords = Lists.newArrayList(); + for (int i = 0; i < commitTimes; i++) { + List records = RandomGenericData.generate(SCHEMA, 100, 0L); + expectedRecords.add(records); + + // Commit those records to iceberg table. + writeRecords(records); + } + return expectedRecords; + } + + private void writeRecords(List records) throws IOException { + GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); + appender.appendToTable(records); + } + + private List generateSplits() { + List inputSplits = Lists.newArrayList(); + + List snapshotIds = SnapshotUtil.currentAncestorIds(table); + for (int i = snapshotIds.size() - 1; i >= 0; i--) { + ScanContext scanContext; + if (i == snapshotIds.size() - 1) { + // Generate the splits from the first snapshot. + scanContext = ScanContext.builder().useSnapshotId(snapshotIds.get(i)).build(); + } else { + // Generate the splits between the previous snapshot and current snapshot. + scanContext = + ScanContext.builder() + .startSnapshotId(snapshotIds.get(i + 1)) + .endSnapshotId(snapshotIds.get(i)) + .build(); + } + + Collections.addAll( + inputSplits, + FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool())); + } + + return inputSplits; + } + + private OneInputStreamOperatorTestHarness createReader() + throws Exception { + // This input format is used to opening the emitted split. + FlinkInputFormat inputFormat = + FlinkSource.forRowData() + .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) + .buildFormat(); + + OneInputStreamOperatorFactory factory = + StreamingReaderOperator.factory(inputFormat); + OneInputStreamOperatorTestHarness harness = + new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); + harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); + + return harness; + } + + private SteppingMailboxProcessor createLocalMailbox( + OneInputStreamOperatorTestHarness harness) { + return new SteppingMailboxProcessor( + MailboxDefaultAction.Controller::suspendDefaultAction, + harness.getTaskMailbox(), + StreamTaskActionExecutor.IMMEDIATE); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java new file mode 100644 index 000000000000..1e612b0a2b2a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java @@ -0,0 +1,132 @@ +/* + * 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.assigner; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.fail; + +import java.nio.file.Path; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.atomic.AtomicBoolean; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public abstract class SplitAssignerTestBase { + @TempDir protected Path temporaryFolder; + + @Test + public void testEmptyInitialization() { + SplitAssigner assigner = splitAssigner(); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test a sequence of interactions for StaticEnumerator */ + @Test + public void testStaticEnumeratorSequence() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 1, "1")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assigner.onUnassignedSplits(createSplits(1, 1, "1")); + assertSnapshot(assigner, 2); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + /** Test a sequence of interactions for ContinuousEnumerator */ + @Test + public void testContinuousEnumeratorSequence() throws Exception { + SplitAssigner assigner = splitAssigner(); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + + List splits1 = createSplits(1, 1, "1"); + assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); + List splits2 = createSplits(1, 1, "1"); + assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); + + assigner.onDiscoveredSplits(createSplits(2, 1, "1")); + assertSnapshot(assigner, 2); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + private void assertAvailableFuture( + SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { + // register callback + AtomicBoolean futureCompleted = new AtomicBoolean(); + CompletableFuture future = assigner.isAvailable(); + future.thenAccept(ignored -> futureCompleted.set(true)); + // calling isAvailable again should return the same object reference + // note that thenAccept will return a new future. + // we want to assert the same instance on the assigner returned future + assertThat(assigner.isAvailable()).isSameAs(future); + + // now add some splits + addSplitsRunnable.run(); + assertThat(futureCompleted.get()).isTrue(); + + for (int i = 0; i < splitCount; ++i) { + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + } + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } + + protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { + GetSplitResult result = assigner.getNext(null); + assertThat(result.status()).isEqualTo(expectedStatus); + switch (expectedStatus) { + case AVAILABLE: + assertThat(result.split()).isNotNull(); + break; + case CONSTRAINED: + case UNAVAILABLE: + assertThat(result.split()).isNull(); + break; + default: + fail("Unknown status: %s", expectedStatus); + } + } + + protected void assertSnapshot(SplitAssigner assigner, int splitCount) { + Collection stateBeforeGet = assigner.state(); + assertThat(stateBeforeGet).hasSize(splitCount); + } + + protected List createSplits(int fileCount, int filesPerSplit, String version) + throws Exception { + return SplitHelpers.createSplitsFromTransientHadoopTable( + temporaryFolder, fileCount, filesPerSplit, version); + } + + protected abstract SplitAssigner splitAssigner(); +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java new file mode 100644 index 000000000000..17e64bbf0594 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java @@ -0,0 +1,43 @@ +/* + * 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.assigner; + +import org.apache.iceberg.flink.source.SplitHelpers; +import org.junit.jupiter.api.Test; + +public class TestDefaultSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new DefaultSplitAssigner(null); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInASplit() throws Exception { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits( + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 4, 2)); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertSnapshot(assigner, 1); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + assertSnapshot(assigner, 0); + } +} 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 new file mode 100644 index 000000000000..ff63ba8e58a0 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java @@ -0,0 +1,81 @@ +/* + * 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.assigner; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.List; +import org.apache.iceberg.ContentFile; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.jupiter.api.Test; + +public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assertThatThrownBy( + () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), + "Multiple files in a split is not allowed") + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Please use 'split-open-file-cost'"); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() throws Exception { + SplitAssigner assigner = splitAssigner(); + List splits = createSplits(5, 1, "2"); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, 1L); + assertGetNext(assigner, 2L); + assertGetNext(assigner, 3L); + assertGetNext(assigner, 4L); + assertGetNext(assigner, 5L); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + assertThat(comparator).isNotNull(); + } + + private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { + GetSplitResult result = assigner.getNext(null); + ContentFile file = result.split().task().files().iterator().next().file(); + assertThat(file.fileSequenceNumber()).isEqualTo(expectedSequenceNumber); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java new file mode 100644 index 000000000000..84f04d5a530a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java @@ -0,0 +1,146 @@ +/* + * 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.assigner; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneOffset; +import java.time.temporal.ChronoUnit; +import java.util.List; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Schema; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ReaderUtil; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.SerializationUtil; +import org.junit.jupiter.api.Test; + +public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { + public static final Schema SCHEMA = + new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + @Override + protected SplitAssigner splitAssigner() { + return new OrderedSplitAssignerFactory( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) + .createAssigner(); + } + + /** Test the assigner when multiple files are in a single split */ + @Test + public void testMultipleFilesInAnIcebergSplit() { + SplitAssigner assigner = splitAssigner(); + assigner.onDiscoveredSplits(createSplits(4, 2, "2")); + + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + /** Test sorted splits */ + @Test + public void testSplitSort() { + SplitAssigner assigner = splitAssigner(); + + Instant now = Instant.now(); + List splits = + IntStream.range(0, 5) + .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) + .collect(Collectors.toList()); + + assigner.onDiscoveredSplits(splits.subList(3, 5)); + assigner.onDiscoveredSplits(splits.subList(0, 1)); + assigner.onDiscoveredSplits(splits.subList(1, 3)); + + assertGetNext(assigner, splits.get(0)); + assertGetNext(assigner, splits.get(1)); + assertGetNext(assigner, splits.get(2)); + assertGetNext(assigner, splits.get(3)); + assertGetNext(assigner, splits.get(4)); + + assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); + } + + @Test + public void testSerializable() { + byte[] bytes = + SerializationUtil.serializeToBytes( + SplitComparators.watermark( + new ColumnStatsWatermarkExtractor( + TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); + SerializableComparator comparator = + SerializationUtil.deserializeFromBytes(bytes); + assertThat(comparator).isNotNull(); + } + + private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { + GetSplitResult result = assigner.getNext(null); + assertThat(split).isEqualTo(result.split()); + } + + @Override + protected List createSplits( + int fileCount, int filesPerSplit, String version) { + return IntStream.range(0, fileCount / filesPerSplit) + .mapToObj( + splitNum -> + splitFromRecords( + IntStream.range(0, filesPerSplit) + .mapToObj( + fileNum -> + RandomGenericData.generate( + SCHEMA, 2, (long) splitNum * filesPerSplit + fileNum)) + .collect(Collectors.toList()))) + .collect(Collectors.toList()); + } + + private IcebergSourceSplit splitFromInstant(Instant instant) { + Record record = GenericRecord.create(SCHEMA); + record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); + return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); + } + + private IcebergSourceSplit splitFromRecords(List> records) { + try { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + records, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); + } catch (IOException e) { + throw new RuntimeException("Split creation exception", e); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java new file mode 100644 index 000000000000..ebc92df02360 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java @@ -0,0 +1,97 @@ +/* + * 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.enumerator; + +import java.io.IOException; +import java.util.List; +import java.util.NavigableMap; +import java.util.TreeMap; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +class ManualContinuousSplitPlanner implements ContinuousSplitPlanner { + private final int maxPlanningSnapshotCount; + // track splits per snapshot + private final NavigableMap> splits; + private long latestSnapshotId; + private int remainingFailures; + + ManualContinuousSplitPlanner(ScanContext scanContext, int expectedFailures) { + this.maxPlanningSnapshotCount = scanContext.maxPlanningSnapshotCount(); + this.splits = new TreeMap<>(); + this.latestSnapshotId = 0L; + this.remainingFailures = expectedFailures; + } + + @Override + public synchronized ContinuousEnumerationResult planSplits( + IcebergEnumeratorPosition lastPosition) { + if (remainingFailures > 0) { + remainingFailures--; + throw new RuntimeException("Expected failure at planning"); + } + + long fromSnapshotIdExclusive = 0; + if (lastPosition != null && lastPosition.snapshotId() != null) { + fromSnapshotIdExclusive = lastPosition.snapshotId(); + } + + Preconditions.checkArgument( + fromSnapshotIdExclusive <= latestSnapshotId, + "last enumerated snapshotId is greater than the latestSnapshotId"); + if (fromSnapshotIdExclusive == latestSnapshotId) { + // already discovered everything. + return new ContinuousEnumerationResult(Lists.newArrayList(), lastPosition, lastPosition); + } + + // find the subset of snapshots to return discovered splits + long toSnapshotIdInclusive; + if (latestSnapshotId - fromSnapshotIdExclusive > maxPlanningSnapshotCount) { + toSnapshotIdInclusive = fromSnapshotIdExclusive + maxPlanningSnapshotCount; + } else { + toSnapshotIdInclusive = latestSnapshotId; + } + + List discoveredSplits = Lists.newArrayList(); + NavigableMap> discoveredView = + splits.subMap(fromSnapshotIdExclusive, false, toSnapshotIdInclusive, true); + discoveredView.forEach((snapshotId, snapshotSplits) -> discoveredSplits.addAll(snapshotSplits)); + ContinuousEnumerationResult result = + new ContinuousEnumerationResult( + discoveredSplits, + lastPosition, + // use the snapshot Id as snapshot timestamp. + IcebergEnumeratorPosition.of(toSnapshotIdInclusive, toSnapshotIdInclusive)); + return result; + } + + /** + * Add a collection of new splits. A monotonically increased snapshotId is assigned to each batch + * of splits added by this method. + */ + public synchronized void addSplits(List newSplits) { + latestSnapshotId += 1; + splits.put(latestSnapshotId, newSplits); + } + + @Override + public void close() throws IOException {} +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java new file mode 100644 index 000000000000..41a787762fda --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java @@ -0,0 +1,352 @@ +/* + * 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.enumerator; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.api.connector.source.SplitEnumeratorContext; +import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.flink.source.split.SplitRequestEvent; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestContinuousIcebergEnumerator { + @TempDir protected Path temporaryFolder; + + @Test + public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + Collection pendingSplitsEmpty = + enumerator.snapshotState(1).pendingSplits(); + assertThat(pendingSplitsEmpty).isEmpty(); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); + assertThat(pendingSplits).hasSize(1); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); + } + + @Test + public void testDiscoverWhenReaderRegistered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register one reader, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .contains(splits.get(0)); + } + + @Test + public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register one reader, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // remove the reader (like in a failure) + enumeratorContext.registeredReaders().remove(2); + + // make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + assertThat(splits).hasSize(1); + splitPlanner.addSplits(splits); + enumeratorContext.triggerAllActions(); + + assertThat(enumeratorContext.getSplitAssignments()).doesNotContainKey(2); + List pendingSplitIds = + enumerator.snapshotState(1).pendingSplits().stream() + .map(IcebergSourceSplitState::split) + .map(IcebergSourceSplit::splitId) + .collect(Collectors.toList()); + assertThat(pendingSplitIds).hasSameSizeAs(splits).first().isEqualTo(splits.get(0).splitId()); + + // register the reader again, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .contains(splits.get(0)); + } + + @Test + public void testThrottlingDiscovery() throws Exception { + // create 10 splits + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 1); + + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + // discover one snapshot at a time + .maxPlanningSnapshotCount(1) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // register reader-2, and let it request a split + enumeratorContext.registerReader(2, "localhost"); + enumerator.addReader(2); + enumerator.handleSourceEvent(2, new SplitRequestEvent()); + + // add splits[0] to the planner for next discovery + splitPlanner.addSplits(Arrays.asList(splits.get(0))); + enumeratorContext.triggerAllActions(); + + // because discovered split was assigned to reader, pending splits should be empty + assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); + // split assignment to reader-2 should contain splits[0, 1) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); + + // add the remaining 9 splits (one for every snapshot) + // run discovery cycles while reader-2 still processing the splits[0] + for (int i = 1; i < 10; ++i) { + splitPlanner.addSplits(Arrays.asList(splits.get(i))); + enumeratorContext.triggerAllActions(); + } + + // can only discover up to 3 snapshots/splits + assertThat(enumerator.snapshotState(2).pendingSplits()).hasSize(3); + // split assignment to reader-2 should be splits[0, 1) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 1)); + + // now reader-2 finished splits[0] + enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); + enumeratorContext.triggerAllActions(); + // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was + // discovered and added. + assertThat(enumerator.snapshotState(3).pendingSplits()).hasSize(3); + // split assignment to reader-2 should be splits[0, 2) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); + + // run 3 more split discovery cycles + for (int i = 0; i < 3; ++i) { + enumeratorContext.triggerAllActions(); + } + + // no more splits are discovered due to throttling + assertThat(enumerator.snapshotState(4).pendingSplits()).hasSize(3); + // split assignment to reader-2 should still be splits[0, 2) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 2)); + + // now reader-2 finished splits[1] + enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); + enumeratorContext.triggerAllActions(); + // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was + // discovered and added. + assertThat(enumerator.snapshotState(5).pendingSplits()).hasSize(3); + // split assignment to reader-2 should be splits[0, 3) + assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) + .containsExactlyElementsOf(splits.subList(0, 3)); + } + + @Test + public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(2) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 1); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + splitPlanner.addSplits(splits); + + // Trigger a planning and check that no splits returned due to the planning error + enumeratorContext.triggerAllActions(); + assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); + + // Second scan planning should succeed and discover the expected splits + enumeratorContext.triggerAllActions(); + Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); + assertThat(pendingSplits).hasSize(1); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); + } + + @Test + public void testOverMaxAllowedPlanningErrors() throws Exception { + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(1) + .build(); + ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 2); + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + splitPlanner.addSplits(splits); + + // Check that the scheduler response ignores the current error and continues to run until the + // failure limit is reached + enumeratorContext.triggerAllActions(); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isFalse(); + + // Check that the task has failed with the expected exception after the failure limit is reached + enumeratorContext.triggerAllActions(); + assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) + .isTrue(); + assertThatThrownBy( + () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) + .hasCauseInstanceOf(RuntimeException.class) + .hasMessageContaining("Failed to discover new split"); + } + + @Test + public void testPlanningIgnoringErrors() throws Exception { + int expectedFailures = 3; + TestingSplitEnumeratorContext enumeratorContext = + new TestingSplitEnumeratorContext<>(4); + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .maxPlanningSnapshotCount(1) + .maxAllowedPlanningFailures(-1) + .build(); + ManualContinuousSplitPlanner splitPlanner = + new ManualContinuousSplitPlanner(scanContext, expectedFailures); + ContinuousIcebergEnumerator enumerator = + createEnumerator(enumeratorContext, scanContext, splitPlanner); + + // Make one split available and trigger the periodic discovery + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + splitPlanner.addSplits(splits); + + Collection pendingSplits; + // Can not discover the new split with planning failures + for (int i = 0; i < expectedFailures; ++i) { + enumeratorContext.triggerAllActions(); + pendingSplits = enumerator.snapshotState(i).pendingSplits(); + assertThat(pendingSplits).isEmpty(); + } + + // Discovered the new split after a successful scan planning + enumeratorContext.triggerAllActions(); + pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); + assertThat(pendingSplits).hasSize(1); + IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); + assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); + assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); + } + + private static ContinuousIcebergEnumerator createEnumerator( + SplitEnumeratorContext context, + ScanContext scanContext, + ContinuousSplitPlanner splitPlanner) { + + ContinuousIcebergEnumerator enumerator = + new ContinuousIcebergEnumerator( + context, + new DefaultSplitAssigner(null, Collections.emptyList()), + scanContext, + splitPlanner, + null); + enumerator.start(); + return enumerator; + } +} 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 new file mode 100644 index 000000000000..0690b456e033 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java @@ -0,0 +1,692 @@ +/* + * 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.enumerator; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Collectors; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestContinuousSplitPlannerImpl { + @TempDir protected Path temporaryFolder; + + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private static final AtomicLong RANDOM_SEED = new AtomicLong(); + + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + private GenericAppenderHelper dataAppender; + private DataFile dataFile1; + private Snapshot snapshot1; + private DataFile dataFile2; + private Snapshot snapshot2; + + @BeforeEach + public void before() throws IOException { + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); + } + + private void appendTwoSnapshots() throws IOException { + // snapshot1 + List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + dataFile1 = dataAppender.writeFile(null, batch1); + dataAppender.appendToTable(dataFile1); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); + + // snapshot2 + List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); + dataFile2 = dataAppender.writeFile(null, batch2); + dataAppender.appendToTable(dataFile2); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); + } + + /** @return the last enumerated snapshot id */ + private CycleResult verifyOneCycle( + ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) + throws Exception { + List batch = + RandomGenericData.generate(TestFixtures.SCHEMA, 2, RANDOM_SEED.incrementAndGet()); + DataFile dataFile = dataAppender.writeFile(null, batch); + dataAppender.appendToTable(dataFile); + Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); + + ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); + assertThat(result.fromPosition().snapshotId()).isEqualTo(lastPosition.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs()) + .isEqualTo(lastPosition.snapshotTimestampMs()); + assertThat(result.toPosition().snapshotId().longValue()).isEqualTo(snapshot.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot.timestampMillis()); + assertThat(result.splits()).hasSize(1); + IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); + assertThat(split.task().files()) + .hasSize(1) + .first() + .satisfies( + fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); + return new CycleResult(result.toPosition(), split); + } + + @Test + public void testTableScanThenIncrementalWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.fromPosition()).isNull(); + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + assertThat(initialResult.splits()).hasSize(1); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + assertThat(split.task().files()).hasSize(2); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + Set expectedFiles = + ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .splitSize(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); + + // latest mode should discover both snapshots, as latest position is marked by when job starts + appendTwoSnapshots(); + ContinuousEnumerationResult afterTwoSnapshotsAppended = + splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); + assertThat(afterTwoSnapshotsAppended.splits()).hasSize(2); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.fromPosition()).isNull(); + // For inclusive behavior, the initial result should point to snapshot1 + // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + assertThat(split.task().files()).hasSize(1); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception { + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); + assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); + + ContinuousEnumerationResult emptyTableSecondDiscoveryResult = + splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); + assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotId()).isNull(); + assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); + + // next 3 snapshots + IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.fromPosition()).isNull(); + // For inclusive behavior, the initial result should point to snapshot1's parent, + // which leads to null snapshotId and snapshotTimestampMs. + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + assertThat(secondResult.fromPosition().snapshotId()).isNull(); + assertThat(secondResult.fromPosition().snapshotTimestampMs()).isNull(); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + assertThat(split.task().files()).hasSize(2); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover files appended in both snapshot1 and snapshot2 + Set expectedFiles = + ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); + assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromSnapshotIdWithEmptyTable() { + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: 1"); + } + + @Test + public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { + appendTwoSnapshots(); + + // find an invalid snapshotId + long invalidSnapshotId = 0L; + while (invalidSnapshotId == snapshot1.snapshotId() + || invalidSnapshotId == snapshot2.snapshotId()) { + invalidSnapshotId++; + } + + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(invalidSnapshotId) + .build(); + + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); + } + + @Test + public void testIncrementalFromSnapshotId() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot2.snapshotId()) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.fromPosition()).isNull(); + // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as + // snapshot2's parent) + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + assertThat(split.task().files()).hasSize(1); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testIncrementalFromSnapshotTimestampWithEmptyTable() { + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(1L) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot find a snapshot after: 1"); + } + + @Test + public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exception { + appendTwoSnapshots(); + + long invalidSnapshotTimestampMs = snapshot2.timestampMillis() + 1000L; + + ScanContext scanContextWithInvalidSnapshotId = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(invalidSnapshotTimestampMs) + .build(); + + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl( + TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); + + assertThatThrownBy(() -> splitPlanner.planSplits(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a snapshot after:"); + } + + @Test + public void testIncrementalFromSnapshotTimestamp() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis()) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.fromPosition()).isNull(); + // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). + assertThat(initialResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(initialResult.splits()).isEmpty(); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + assertThat(secondResult.fromPosition().snapshotId().longValue()) + .isEqualTo(snapshot1.snapshotId()); + assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot1.timestampMillis()); + assertThat(secondResult.toPosition().snapshotId().longValue()) + .isEqualTo(snapshot2.snapshotId()); + assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(snapshot2.timestampMillis()); + IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); + assertThat(split.task().files()).hasSize(1); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + // should discover dataFile2 appended in snapshot2 + Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); + + IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); + for (int i = 0; i < 3; ++i) { + lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; + } + } + + @Test + public void testMaxPlanningSnapshotCount() throws Exception { + appendTwoSnapshots(); + // append 3 more snapshots + for (int i = 2; i < 5; ++i) { + appendSnapshot(i, 2); + } + + ScanContext scanContext = + ScanContext.builder() + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + // limit to 1 snapshot per discovery + .maxPlanningSnapshotCount(1) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.fromPosition()).isNull(); + // For inclusive behavior, the initial result should point to snapshot1's parent, + // which leads to null snapshotId and snapshotTimestampMs. + assertThat(initialResult.toPosition().snapshotId()).isNull(); + assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); + assertThat(initialResult.splits()).isEmpty(); + + ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); + // should discover dataFile1 appended in snapshot1 + verifyMaxPlanningSnapshotCountResult( + secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); + + ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); + // should discover dataFile2 appended in snapshot2 + verifyMaxPlanningSnapshotCountResult( + thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); + } + + @Test + public void testTableScanNoStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(false) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.splits()).hasSize(1); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + assertThat(split.task().files()).hasSize(2); + verifyStatCount(split, 0); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 0); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanAllStats() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.splits()).hasSize(1); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + assertThat(split.task().files()).hasSize(2); + verifyStatCount(split, 3); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 3); + lastPosition = result.lastPosition; + } + } + + @Test + public void testTableScanSingleStat() throws Exception { + appendTwoSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .includeColumnStats(ImmutableSet.of("data")) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + ContinuousSplitPlannerImpl splitPlanner = + new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); + + ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); + assertThat(initialResult.splits()).hasSize(1); + IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); + assertThat(split.task().files()).hasSize(2); + verifyStatCount(split, 1); + + IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); + for (int i = 0; i < 3; ++i) { + CycleResult result = verifyOneCycle(splitPlanner, lastPosition); + verifyStatCount(result.split, 1); + lastPosition = result.lastPosition; + } + } + + private void verifyStatCount(IcebergSourceSplit split, int expected) { + if (expected == 0) { + split + .task() + .files() + .forEach( + f -> { + assertThat(f.file().valueCounts()).isNull(); + assertThat(f.file().columnSizes()).isNull(); + assertThat(f.file().lowerBounds()).isNull(); + assertThat(f.file().upperBounds()).isNull(); + assertThat(f.file().nanValueCounts()).isNull(); + assertThat(f.file().nullValueCounts()).isNull(); + }); + } else { + split + .task() + .files() + .forEach( + f -> { + assertThat(f.file().valueCounts()).hasSize(expected); + assertThat(f.file().columnSizes()).hasSize(expected); + assertThat(f.file().lowerBounds()).hasSize(expected); + assertThat(f.file().upperBounds()).hasSize(expected); + assertThat(f.file().nullValueCounts()).hasSize(expected); + // The nanValue is not counted for long and string fields + assertThat(f.file().nanValueCounts()).isEmpty(); + }); + } + } + + private void verifyMaxPlanningSnapshotCountResult( + ContinuousEnumerationResult result, + Snapshot fromSnapshotExclusive, + Snapshot toSnapshotInclusive, + Set expectedFiles) { + if (fromSnapshotExclusive == null) { + assertThat(result.fromPosition().snapshotId()).isNull(); + assertThat(result.fromPosition().snapshotTimestampMs()).isNull(); + } else { + assertThat(result.fromPosition().snapshotId().longValue()) + .isEqualTo(fromSnapshotExclusive.snapshotId()); + assertThat(result.fromPosition().snapshotTimestampMs().longValue()) + .isEqualTo(fromSnapshotExclusive.timestampMillis()); + } + assertThat(result.toPosition().snapshotId().longValue()) + .isEqualTo(toSnapshotInclusive.snapshotId()); + assertThat(result.toPosition().snapshotTimestampMs().longValue()) + .isEqualTo(toSnapshotInclusive.timestampMillis()); + // should only have one split with one data file, because split discover is limited to + // one snapshot and each snapshot has only one data file appended. + IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); + assertThat(split.task().files()).hasSize(1); + Set discoveredFiles = + split.task().files().stream() + .map(fileScanTask -> fileScanTask.file().path().toString()) + .collect(Collectors.toSet()); + assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); + } + + private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { + List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); + DataFile dataFile = dataAppender.writeFile(null, batch); + dataAppender.appendToTable(dataFile); + return TABLE_RESOURCE.table().currentSnapshot(); + } + + private static class CycleResult { + IcebergEnumeratorPosition lastPosition; + IcebergSourceSplit split; + + CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { + this.lastPosition = lastPosition; + this.split = split; + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java new file mode 100644 index 000000000000..b2185675340f --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java @@ -0,0 +1,200 @@ +/* + * 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.enumerator; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.data.GenericAppenderHelper; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.ScanContext; +import org.apache.iceberg.flink.source.StreamingStartingStrategy; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestContinuousSplitPlannerImplStartStrategy { + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + private static final HadoopTableExtension TABLE_RESOURCE = + new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); + + private GenericAppenderHelper dataAppender; + private Snapshot snapshot1; + private Snapshot snapshot2; + private Snapshot snapshot3; + + @BeforeEach + public void before() throws IOException { + dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); + } + + private void appendThreeSnapshots() throws IOException { + List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); + dataAppender.appendToTable(batch1); + snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); + + List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); + dataAppender.appendToTable(batch2); + snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); + + List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); + dataAppender.appendToTable(batch3); + snapshot3 = TABLE_RESOURCE.table().currentSnapshot(); + } + + @Test + public void testTableScanThenIncrementalStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) + .build(); + + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); + } + + @Test + public void testForLatestSnapshotStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) + .build(); + + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); + } + + @Test + public void testForEarliestSnapshotStrategy() throws IOException { + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) + .build(); + + assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) + .isNotPresent(); + + appendThreeSnapshots(); + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot1.snapshotId()); + } + + @Test + public void testForSpecificSnapshotIdStrategy() throws IOException { + ScanContext scanContextInvalidSnapshotId = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(1L) + .build(); + + assertThatThrownBy( + () -> + ContinuousSplitPlannerImpl.startSnapshot( + TABLE_RESOURCE.table(), scanContextInvalidSnapshotId)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Start snapshot id not found in history: 1"); + + appendThreeSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) + .startSnapshotId(snapshot2.snapshotId()) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); + } + + @Test + public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOException { + ScanContext scanContextInvalidSnapshotTimestamp = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(1L) + .build(); + + assertThatThrownBy( + () -> + ContinuousSplitPlannerImpl.startSnapshot( + TABLE_RESOURCE.table(), scanContextInvalidSnapshotTimestamp)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Cannot find a snapshot after: "); + + appendThreeSnapshots(); + + ScanContext scanContext = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis()) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); + } + + @Test + public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOException { + appendThreeSnapshots(); + + ScanContext config = + ScanContext.builder() + .streaming(true) + .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) + .startSnapshotTimestamp(snapshot2.timestampMillis() - 1L) + .build(); + + Snapshot startSnapshot = + ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), config).get(); + assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java new file mode 100644 index 000000000000..feefcb98646b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java @@ -0,0 +1,135 @@ +/* + * 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.enumerator; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; + +public class TestEnumerationHistory { + private static final int MAX_HISTORY_SIZE = 3; + private static final int FEW_PENDING_SPLITS = 2; + private static final int TOO_MANY_PENDING_SPLITS = 100; + + @Test + public void testEmptyHistory() { + EnumerationHistory history = new EnumerationHistory(MAX_HISTORY_SIZE); + int[] expectedHistorySnapshot = new int[0]; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testNotFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + int[] expectedHistorySnapshot = {1, 2}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testExactFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + int[] expectedHistorySnapshot = {1, 2, 3}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testOneMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + int[] expectedHistorySnapshot = {2, 3, 4}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testTwoMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + history.add(5); + int[] expectedHistorySnapshot = {3, 4, 5}; + testHistory(history, expectedHistorySnapshot); + } + + @Test + public void testThreeMoreThanFullHistory() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + history.add(4); + history.add(5); + history.add(6); + int[] expectedHistorySnapshot = {4, 5, 6}; + testHistory(history, expectedHistorySnapshot); + } + + private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); + if (history.hasFullHistory()) { + // throttle because pending split count is more than the sum of enumeration history + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); + } else { + // skipped throttling check because there is not enough history + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isFalse(); + } + + int[] historySnapshot = history.snapshot(); + assertThat(historySnapshot).containsExactly(expectedHistorySnapshot); + + EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); + restoredHistory.restore(historySnapshot); + + assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); + if (history.hasFullHistory()) { + // throttle because pending split count is more than the sum of enumeration history + assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); + } else { + // skipped throttling check because there is not enough history + assertThat(history.shouldPauseSplitDiscovery(30)).isFalse(); + } + } + + @Test + public void testRestoreDifferentSize() { + EnumerationHistory history = new EnumerationHistory(3); + history.add(1); + history.add(2); + history.add(3); + int[] historySnapshot = history.snapshot(); + + EnumerationHistory smallerHistory = new EnumerationHistory(2); + smallerHistory.restore(historySnapshot); + int[] expectedRestoredHistorySnapshot = {2, 3}; + assertThat(smallerHistory.snapshot()).containsExactly(expectedRestoredHistorySnapshot); + + EnumerationHistory largerHisotry = new EnumerationHistory(4); + largerHisotry.restore(historySnapshot); + assertThat(largerHisotry.snapshot()).containsExactly(historySnapshot); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java new file mode 100644 index 000000000000..2520a6b763e4 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java @@ -0,0 +1,146 @@ +/* + * 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.enumerator; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; +import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +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 TestIcebergEnumeratorStateSerializer { + @TempDir protected Path temporaryFolder; + + private final IcebergEnumeratorStateSerializer serializer = + new IcebergEnumeratorStateSerializer(true); + + @Parameter(index = 0) + protected int version; + + @Parameters(name = "version={0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {1}, new Object[] {2}}; + } + + @TestTemplate + public void testEmptySnapshotIdAndPendingSplits() throws Exception { + IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); + testSerializer(enumeratorState); + } + + @TestTemplate + public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); + + IcebergEnumeratorState enumeratorState = + new IcebergEnumeratorState(position, Collections.emptyList()); + testSerializer(enumeratorState); + } + + @TestTemplate + public void testSomeSnapshotIdAndPendingSplits() throws Exception { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); + Collection pendingSplits = Lists.newArrayList(); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); + + IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(position, pendingSplits); + testSerializer(enumeratorState); + } + + @TestTemplate + public void testEnumerationSplitCountHistory() throws Exception { + if (version == 2) { + IcebergEnumeratorPosition position = + IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); + List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); + Collection pendingSplits = Lists.newArrayList(); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); + pendingSplits.add( + new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); + int[] enumerationSplitCountHistory = {1, 2, 3}; + + IcebergEnumeratorState enumeratorState = + new IcebergEnumeratorState(position, pendingSplits, enumerationSplitCountHistory); + testSerializer(enumeratorState); + } + } + + private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOException { + byte[] result; + if (version == 1) { + result = serializer.serializeV1(enumeratorState); + } else { + result = serializer.serialize(enumeratorState); + } + + IcebergEnumeratorState deserialized = serializer.deserialize(version, result); + assertEnumeratorStateEquals(enumeratorState, deserialized); + } + + private void assertEnumeratorStateEquals( + IcebergEnumeratorState expected, IcebergEnumeratorState actual) { + assertThat(actual.lastEnumeratedPosition()).isEqualTo(expected.lastEnumeratedPosition()); + + assertThat(actual.pendingSplits()).hasSameSizeAs(expected.pendingSplits()); + Iterator expectedIterator = expected.pendingSplits().iterator(); + Iterator actualIterator = actual.pendingSplits().iterator(); + for (int i = 0; i < expected.pendingSplits().size(); ++i) { + IcebergSourceSplitState expectedSplitState = expectedIterator.next(); + IcebergSourceSplitState actualSplitState = actualIterator.next(); + assertThat(actualSplitState.split().splitId()) + .isEqualTo(expectedSplitState.split().splitId()); + assertThat(actualSplitState.split().fileOffset()) + .isEqualTo(expectedSplitState.split().fileOffset()); + assertThat(actualSplitState.split().recordOffset()) + .isEqualTo(expectedSplitState.split().recordOffset()); + assertThat(actualSplitState.status()).isEqualTo(expectedSplitState.status()); + } + + assertThat(actual.enumerationSplitCountHistory()) + .containsExactly(expected.enumerationSplitCountHistory()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java new file mode 100644 index 000000000000..0d1d0ce3217c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java @@ -0,0 +1,218 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.iceberg.CombinedScanTask; +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.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +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 ReaderFunctionTestBase { + + @Parameters(name = "fileFormat={0}") + public static Object[][] parameters() { + return new Object[][] { + new Object[] {FileFormat.AVRO}, + new Object[] {FileFormat.ORC}, + new Object[] {FileFormat.PARQUET} + }; + } + + @TempDir protected Path temporaryFolder; + + protected abstract ReaderFunction readerFunction(); + + protected abstract void assertRecords(List expected, List actual, Schema schema); + + @Parameter(index = 0) + private FileFormat fileFormat; + + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); + + private void assertRecordsAndPosition( + List expectedRecords, + int expectedFileOffset, + long startRecordOffset, + RecordsWithSplitIds> batch) { + batch.nextSplit(); + List actualRecords = Lists.newArrayList(); + long recordOffset = startRecordOffset; + RecordAndPosition recordAndPosition; + while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { + actualRecords.add(recordAndPosition.record()); + assertThat(recordAndPosition.fileOffset()).isEqualTo(expectedFileOffset); + assertThat(recordAndPosition.recordOffset() - 1).isEqualTo(recordOffset); + recordOffset++; + } + + assertThat(actualRecords).hasSameSizeAs(expectedRecords); + assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); + } + + @TestTemplate + public void testNoCheckpointedPosition() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @TestTemplate + public void testCheckpointedPositionBeforeFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @TestTemplate + public void testCheckpointedPositionMiddleFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch0 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(0).subList(1, 2), 0, 1L, batch0); + batch0.recycle(); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @TestTemplate + public void testCheckpointedPositionAfterFirstFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @TestTemplate + public void testCheckpointedPositionBeforeSecondFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } + + @TestTemplate + public void testCheckpointedPositionMidSecondFile() throws IOException { + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + CombinedScanTask combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, fileFormat, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); + CloseableIterator>> reader = + readerFunction().apply(split); + + RecordsWithSplitIds> batch1 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(1).subList(1, 2), 1, 1L, batch1); + batch1.recycle(); + + RecordsWithSplitIds> batch2 = reader.next(); + assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); + batch2.recycle(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java new file mode 100644 index 000000000000..0edf8ae009fe --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java @@ -0,0 +1,128 @@ +/* + * 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 java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.BaseFileScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.PartitionSpecParser; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.expressions.ResidualEvaluator; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; + +public class ReaderUtil { + + private ReaderUtil() {} + + public static FileScanTask createFileTask( + List records, + File file, + FileFormat fileFormat, + FileAppenderFactory appenderFactory) + throws IOException { + FileAppender appender = + appenderFactory.newAppender(Files.localOutput(file), fileFormat); + try { + appender.addAll(records); + } finally { + appender.close(); + } + + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withRecordCount(records.size()) + .withFileSizeInBytes(file.length()) + .withPath(file.toString()) + .withFormat(fileFormat) + .withMetrics(appender.metrics()) + .build(); + + ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); + return new BaseFileScanTask( + dataFile, + null, + SchemaParser.toJson(TestFixtures.SCHEMA), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + residuals); + } + + public static DataIterator createDataIterator(CombinedScanTask combinedTask) { + return new DataIterator<>( + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), + combinedTask, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + PlaintextEncryptionManager.instance()); + } + + public static List> createRecordBatchList( + Schema schema, int listSize, int batchCount) { + return createRecordBatchList(0L, schema, listSize, batchCount); + } + + public static List> createRecordBatchList( + long seed, Schema schema, int listSize, int batchCount) { + List records = RandomGenericData.generate(schema, listSize * batchCount, seed); + return Lists.partition(records, batchCount); + } + + public static CombinedScanTask createCombinedScanTask( + List> recordBatchList, + Path temporaryFolder, + FileFormat fileFormat, + GenericAppenderFactory appenderFactory) + throws IOException { + List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); + for (List recordBatch : recordBatchList) { + FileScanTask fileTask = + ReaderUtil.createFileTask( + recordBatch, + File.createTempFile("junit", null, temporaryFolder.toFile()), + fileFormat, + appenderFactory); + fileTasks.add(fileTask); + } + + return new BaseCombinedScanTask(fileTasks); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java new file mode 100644 index 000000000000..6f09bd9a56d6 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.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.flink.source.reader; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.concurrent.atomic.AtomicBoolean; +import org.junit.jupiter.api.Test; + +public class TestArrayBatchRecords { + + @Test + public void testFullRange() { + String[] elements = new String[] {"0", "1", "2", "3"}; + testArray(elements, elements.length, 2, 119); + } + + @Test + public void testSubRange() { + String[] elements = new String[] {"0", "1", "2", "3"}; + testArray(elements, 2, 0, 0); + } + + private void testArray( + String[] elements, int numberOfRecords, int fileOffset, long startingRecordOffset) { + String splitId = "iceberg_split_1"; + AtomicBoolean recycled = new AtomicBoolean(); + + ArrayBatchRecords recordsWithSplitIds = + ArrayBatchRecords.forRecords( + splitId, + ignored -> recycled.set(true), + elements, + numberOfRecords, + fileOffset, + startingRecordOffset); + + assertThat(recordsWithSplitIds.nextSplit()).isEqualTo(splitId); + + for (int i = 0; i < numberOfRecords; i++) { + RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); + assertThat(recAndPos.record()).isEqualTo(elements[i]); + assertThat(recAndPos.fileOffset()).isEqualTo(fileOffset); + // recordOffset points to the position after this one + assertThat(recAndPos.recordOffset()).isEqualTo(startingRecordOffset + i + 1); + } + + assertThat(recordsWithSplitIds.nextRecordFromSplit()).isNull(); + assertThat(recordsWithSplitIds.nextSplit()).isNull(); + recordsWithSplitIds.recycle(); + assertThat(recycled.get()).isTrue(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java new file mode 100644 index 000000000000..1a78bb1b0010 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java @@ -0,0 +1,360 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.nio.file.Path; +import java.util.Arrays; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; +import org.apache.flink.connector.base.source.reader.SourceReaderOptions; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.BaseCombinedScanTask; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.io.CloseableIterator; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestArrayPoolDataIteratorBatcherRowData { + + @TempDir protected Path temporaryFolder; + private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; + private final Configuration config = + new Configuration() + .set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1) + .set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); + + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); + private final DataIteratorBatcher batcher = + new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); + + /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ + @Test + public void testSingleFileLessThanOneFullBatch() throws Exception { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); + FileScanTask fileTask = + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); + CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch.finishedSplits()).isEmpty(); + assertThat(batch.nextSplit()).isEqualTo(splitId); + assertThat(batch.records()).hasSize(2); + assertThat(batch.numberOfRecords()).isEqualTo(1); + + RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); + + /////////////////////////////// + // assert first record + + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); + + assertThat(batch.nextRecordFromSplit()).isNull(); + assertThat(batch.nextSplit()).isNull(); + batch.recycle(); + + assertThat(recordBatchIterator).isExhausted(); + } + + /** + * Read a CombinedScanTask that contains a single file with multiple batches. + * + *

    Insert 5 records in a single file that should result in 3 batches + */ + @Test + public void testSingleFileWithMultipleBatches() throws Exception { + List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); + FileScanTask fileTask = + ReaderUtil.createFileTask( + records, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); + CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + /////////////////////////////// + // assert first batch with full batch of 2 records + + ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch0.finishedSplits()).isEmpty(); + assertThat(batch0.nextSplit()).isEqualTo(splitId); + assertThat(batch0.records()).hasSize(2); + assertThat(batch0.numberOfRecords()).isEqualTo(2); + + RecordAndPosition recordAndPosition; + + // assert first record + recordAndPosition = batch0.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); + + // assert second record + recordAndPosition = batch0.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); + + assertThat(batch0.nextRecordFromSplit()).isNull(); + assertThat(batch0.nextSplit()).isNull(); + batch0.recycle(); + + /////////////////////////////// + // assert second batch with full batch of 2 records + + ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch1.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch1.finishedSplits()).isEmpty(); + assertThat(batch1.nextSplit()).isEqualTo(splitId); + assertThat(batch1.records()).hasSize(2); + assertThat(batch1.numberOfRecords()).isEqualTo(2); + + // assert third record + recordAndPosition = batch1.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); + + // assert fourth record + recordAndPosition = batch1.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); + + assertThat(batch1.nextRecordFromSplit()).isNull(); + assertThat(batch1.nextSplit()).isNull(); + batch1.recycle(); + + /////////////////////////////// + // assert third batch with partial batch of 1 record + + ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch2.records()).containsExactlyInAnyOrder(batch0.records()); + assertThat(batch2.finishedSplits()).isEmpty(); + assertThat(batch2.nextSplit()).isEqualTo(splitId); + assertThat(batch2.records()).hasSize(2); + assertThat(batch2.numberOfRecords()).isEqualTo(1); + + // assert fifth record + recordAndPosition = batch2.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(0); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(5); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); + + assertThat(batch2.nextRecordFromSplit()).isNull(); + assertThat(batch2.nextSplit()).isNull(); + batch2.recycle(); + + assertThat(recordBatchIterator).isExhausted(); + } + + /** + * Read a CombinedScanTask that contains with multiple files. + * + *

    In this test, we also seek the iterator to starting position (1, 1). + */ + @Test + public void testMultipleFilesWithSeekPosition() throws Exception { + List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); + FileScanTask fileTask0 = + ReaderUtil.createFileTask( + records0, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); + List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); + FileScanTask fileTask1 = + ReaderUtil.createFileTask( + records1, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); + List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); + FileScanTask fileTask2 = + ReaderUtil.createFileTask( + records2, + File.createTempFile("junit", null, temporaryFolder.toFile()), + FILE_FORMAT, + appenderFactory); + CombinedScanTask combinedTask = + new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); + + DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); + dataIterator.seek(1, 1); + + String splitId = "someSplitId"; + CloseableIterator>> recordBatchIterator = + batcher.batch(splitId, dataIterator); + + /////////////////////////////// + // file0 is skipped by seek + + /////////////////////////////// + // file1 has 4 records. because the seek position, first record is skipped. + // we should read 3 remaining records in 2 batches: + // batch10 with 2 records and batch11 with 1 records. + + // assert first batch from file1 with full batch of 2 records + + // variable naming convention: batch + ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch10.finishedSplits()).isEmpty(); + assertThat(batch10.nextSplit()).isEqualTo(splitId); + assertThat(batch10.records()).hasSize(2); + assertThat(batch10.numberOfRecords()).isEqualTo(2); + + RecordAndPosition recordAndPosition; + + recordAndPosition = batch10.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("seek should skip the first record in file1. starting from the second record") + .isEqualTo(2); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); + + recordAndPosition = batch10.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); + + assertThat(batch10.nextRecordFromSplit()).isNull(); + assertThat(batch10.nextSplit()).isNull(); + batch10.recycle(); + + // assert second batch from file1 with partial batch of 1 record + + // variable naming convention: batch__ + ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch11.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch11.finishedSplits()).isEmpty(); + assertThat(batch11.nextSplit()).isEqualTo(splitId); + assertThat(batch11.records()).hasSize(2); + assertThat(batch11.numberOfRecords()).isEqualTo(1); + + recordAndPosition = batch11.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(1); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(4); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); + + assertThat(batch11.nextRecordFromSplit()).isNull(); + assertThat(batch11.nextSplit()).isNull(); + batch11.recycle(); + + /////////////////////////////// + // file2 has 3 records. + // we should read 3 records in 2 batches: + // batch20 with 2 records and batch21 with 1 records + + // assert first batch from file2 with full batch of 2 records + + // variable naming convention: batch__ + ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch20.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch20.finishedSplits()).isEmpty(); + assertThat(batch20.nextSplit()).isEqualTo(splitId); + assertThat(batch20.records()).hasSize(2); + assertThat(batch20.numberOfRecords()).isEqualTo(2); + + recordAndPosition = batch20.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(1); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); + + recordAndPosition = batch20.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(2); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); + + assertThat(batch20.nextRecordFromSplit()).isNull(); + assertThat(batch20.nextSplit()).isNull(); + batch20.recycle(); + + /////////////////////////////// + // assert second batch from file2 with partial batch of 1 record + + // variable naming convention: batch__ + ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); + assertThat(batch21.records()).containsExactlyInAnyOrder(batch10.records()); + assertThat(batch21.finishedSplits()).isEmpty(); + assertThat(batch21.nextSplit()).isEqualTo(splitId); + assertThat(batch21.records()).hasSize(2); + assertThat(batch21.numberOfRecords()).isEqualTo(1); + + recordAndPosition = batch21.nextRecordFromSplit(); + assertThat(recordAndPosition.fileOffset()).isEqualTo(2); + + assertThat(recordAndPosition.recordOffset()) + .as("The position points to where the reader should resume after this record is processed.") + .isEqualTo(3); + TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); + + assertThat(batch21.nextRecordFromSplit()).isNull(); + assertThat(batch21.nextSplit()).isNull(); + batch21.recycle(); + + assertThat(recordBatchIterator).isExhausted(); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java new file mode 100644 index 000000000000..af806d4c655d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java @@ -0,0 +1,176 @@ +/* + * 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 static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.IOException; +import java.nio.file.Path; +import java.time.LocalDateTime; +import java.time.OffsetDateTime; +import java.time.ZoneOffset; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeUnit; +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.data.GenericAppenderFactory; +import org.apache.iceberg.data.RandomGenericData; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopTableExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeAll; +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 class TestColumnStatsWatermarkExtractor { + public static final Schema SCHEMA = + new Schema( + required(1, "timestamp_column", Types.TimestampType.withoutZone()), + required(2, "timestamptz_column", Types.TimestampType.withZone()), + required(3, "long_column", Types.LongType.get()), + required(4, "string_column", Types.StringType.get())); + + private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); + + private static final List> TEST_RECORDS = + ImmutableList.of( + RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); + + private static final List> MIN_VALUES = + ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); + + @TempDir protected Path temporaryFolder; + + @RegisterExtension + private static final HadoopTableExtension SOURCE_TABLE_EXTENSION = + new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); + + @Parameter(index = 0) + private String columnName; + + @BeforeAll + public static void updateMinValue() { + for (int i = 0; i < TEST_RECORDS.size(); ++i) { + for (Record r : TEST_RECORDS.get(i)) { + Map minValues = MIN_VALUES.get(i); + + LocalDateTime localDateTime = (LocalDateTime) r.get(0); + minValues.merge( + "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); + + OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); + minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); + + minValues.merge("long_column", (Long) r.get(2), Math::min); + } + } + } + + @Parameters(name = "columnName = {0}") + public static Collection data() { + return ImmutableList.of( + new Object[] {"timestamp_column"}, + new Object[] {"timestamptz_column"}, + new Object[] {"long_column"}); + } + + @TestTemplate + public void testSingle() throws IOException { + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); + + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); + } + + @TestTemplate + public void testTimeUnit() throws IOException { + assumeThat(columnName).isEqualTo("long_column"); + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); + + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName) / 1000L); + } + + @TestTemplate + public void testMultipleFiles() throws IOException { + assumeThat(columnName).isEqualTo("timestamp_column"); + IcebergSourceSplit combinedSplit = + IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + TEST_RECORDS, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); + + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); + + assertThat(extractor.extractWatermark(split(0))) + .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); + assertThat(extractor.extractWatermark(split(1))) + .isEqualTo(MIN_VALUES.get(1).get(columnName).longValue()); + assertThat(extractor.extractWatermark(combinedSplit)) + .isEqualTo(Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName))); + } + + @TestTemplate + public void testWrongColumn() { + assumeThat(columnName).isEqualTo("string_column"); + assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); + } + + @TestTemplate + public void testEmptyStatistics() throws IOException { + assumeThat(columnName).isEqualTo("timestamp_column"); + + // Create an extractor for a column we do not have statistics + ColumnStatsWatermarkExtractor extractor = + new ColumnStatsWatermarkExtractor(10, "missing_field"); + assertThatThrownBy(() -> extractor.extractWatermark(split(0))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Missing statistics for column"); + } + + private IcebergSourceSplit split(int id) throws IOException { + return IcebergSourceSplit.fromCombinedScanTask( + ReaderUtil.createCombinedScanTask( + ImmutableList.of(TEST_RECORDS.get(id)), + temporaryFolder, + FileFormat.PARQUET, + APPENDER_FACTORY)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java new file mode 100644 index 000000000000..8d6782586676 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java @@ -0,0 +1,181 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.apache.flink.api.connector.source.SourceReaderContext; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; +import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.flink.source.split.SerializableComparator; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceReader { + @TempDir protected Path temporaryFolder; + + private final GenericAppenderFactory appenderFactory = + new GenericAppenderFactory(TestFixtures.SCHEMA); + + @Test + public void testReaderMetrics() throws Exception { + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + IcebergSourceReader reader = createReader(metricGroup, readerContext, null); + reader.start(); + + testOneSplitFetcher(reader, readerOutput, metricGroup, 1); + testOneSplitFetcher(reader, readerOutput, metricGroup, 2); + } + + @Test + public void testReaderOrder() throws Exception { + // Create 2 splits + List> recordBatchList1 = + ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task1 = + ReaderUtil.createCombinedScanTask( + recordBatchList1, temporaryFolder, FileFormat.PARQUET, appenderFactory); + + List> recordBatchList2 = + ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task2 = + ReaderUtil.createCombinedScanTask( + recordBatchList2, temporaryFolder, FileFormat.PARQUET, appenderFactory); + + // Sort the splits in one way + List rowDataList1 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task1), + IcebergSourceSplit.fromCombinedScanTask(task2)), + 2); + + // Reverse the splits + List rowDataList2 = + read( + Arrays.asList( + IcebergSourceSplit.fromCombinedScanTask(task2), + IcebergSourceSplit.fromCombinedScanTask(task1)), + 2); + + // Check that the order of the elements is not changed + assertThat(rowDataList1).containsExactlyElementsOf(rowDataList2); + } + + private List read(List splits, long expected) throws Exception { + TestingMetricGroup metricGroup = new TestingMetricGroup(); + TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); + // Using IdBasedComparator, so we can have a deterministic order of the splits + IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); + reader.start(); + + reader.addSplits(splits); + TestingReaderOutput readerOutput = new TestingReaderOutput<>(); + while (readerOutput.getEmittedRecords().size() < expected) { + reader.pollNext(readerOutput); + } + + reader.pollNext(readerOutput); + + assertThat(readerOutput.getEmittedRecords()).hasSize((int) expected); + return readerOutput.getEmittedRecords(); + } + + private void testOneSplitFetcher( + IcebergSourceReader reader, + TestingReaderOutput readerOutput, + TestingMetricGroup metricGroup, + int expectedCount) + throws Exception { + long seed = expectedCount; + // Each split should contain only one file with one record + List> recordBatchList = + ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); + CombinedScanTask task = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); + reader.addSplits(Collections.singletonList(split)); + + while (readerOutput.getEmittedRecords().size() < expectedCount) { + reader.pollNext(readerOutput); + } + + assertThat(readerOutput.getEmittedRecords()).hasSize(expectedCount); + TestHelpers.assertRowData( + TestFixtures.SCHEMA, + recordBatchList.get(0).get(0), + readerOutput.getEmittedRecords().get(expectedCount - 1)); + assertThat(metricGroup.counters().get("assignedSplits").getCount()).isEqualTo(expectedCount); + + // One more poll will get null record batch. + // That will finish the split and cause split fetcher to be closed due to idleness. + // Then next split will create a new split reader. + reader.pollNext(readerOutput); + } + + private IcebergSourceReader createReader( + MetricGroup metricGroup, + SourceReaderContext readerContext, + SerializableComparator splitComparator) { + IcebergSourceReaderMetrics readerMetrics = + new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); + RowDataReaderFunction readerFunction = + new RowDataReaderFunction( + new Configuration(), + TestFixtures.SCHEMA, + TestFixtures.SCHEMA, + null, + true, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + PlaintextEncryptionManager.instance(), + Collections.emptyList()); + return new IcebergSourceReader<>( + SerializableRecordEmitter.defaultEmitter(), + readerMetrics, + readerFunction, + splitComparator, + readerContext); + } + + private static class IdBasedComparator implements SerializableComparator { + @Override + public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { + return o1.splitId().compareTo(o2.splitId()); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java new file mode 100644 index 000000000000..36749d3ec2dc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java @@ -0,0 +1,84 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.Collections; +import java.util.List; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.CombinedScanTask; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.data.GenericAppenderFactory; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.io.TempDir; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestLimitableDataIterator { + @TempDir private static Path temporaryFolder; + + private final RowDataFileScanTaskReader reader = + new RowDataFileScanTaskReader( + TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); + private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); + private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + + private static CombinedScanTask combinedScanTask; + private static int totalRecords; + + @BeforeAll + public static void beforeClass() throws Exception { + GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); + List> recordBatchList = + ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); + combinedScanTask = + ReaderUtil.createCombinedScanTask( + recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); + totalRecords = 3 * 2; + } + + @ParameterizedTest + @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) + public void testUnlimited(long limit) { + LimitableDataIterator dataIterator = + new LimitableDataIterator<>( + reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); + + List result = Lists.newArrayList(); + while (dataIterator.hasNext()) { + result.add(dataIterator.next()); + } + + if (limit <= 0 || limit > totalRecords) { + // read all records + assertThat(result).hasSize(totalRecords); + } else { + assertThat(result).hasSize((int) limit); + } + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java new file mode 100644 index 000000000000..55f9c0af3a29 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.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.flink.source.reader; + +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.configuration.Configuration; +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.Schema; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.hadoop.HadoopFileIO; + +public class TestRowDataReaderFunction extends ReaderFunctionTestBase { + + protected static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); + private static final DataStructureConverter ROW_DATA_CONVERTER = + DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(ROW_TYPE)); + + @Override + protected ReaderFunction readerFunction() { + return new RowDataReaderFunction( + new Configuration(), + TestFixtures.SCHEMA, + TestFixtures.SCHEMA, + null, + true, + new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), + PlaintextEncryptionManager.instance(), + Collections.emptyList()); + } + + @Override + protected void assertRecords(List expected, List actual, Schema schema) { + List rows = toRows(actual); + TestHelpers.assertRecords(rows, expected, TestFixtures.SCHEMA); + } + + private List toRows(List actual) { + return actual.stream() + .map(rowData -> (Row) ROW_DATA_CONVERTER.toExternal(rowData)) + .collect(Collectors.toList()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java new file mode 100644 index 000000000000..290628c5fc90 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java @@ -0,0 +1,102 @@ +/* + * 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 java.util.Map; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.SimpleCounter; +import org.apache.flink.metrics.groups.OperatorIOMetricGroup; +import org.apache.flink.metrics.groups.SourceReaderMetricGroup; +import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +class TestingMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { + private final Map counters; + + TestingMetricGroup() { + this.counters = Maps.newHashMap(); + } + + /** Pass along the reference to share the map for child metric groups. */ + private TestingMetricGroup(Map counters) { + this.counters = counters; + } + + Map counters() { + return counters; + } + + @Override + public Counter counter(String name) { + Counter counter = new SimpleCounter(); + counters.put(name, counter); + return counter; + } + + @Override + public MetricGroup addGroup(String name) { + return new TestingMetricGroup(counters); + } + + @Override + public MetricGroup addGroup(String key, String value) { + return new TestingMetricGroup(counters); + } + + @Override + public OperatorIOMetricGroup getIOMetricGroup() { + return new TestingOperatorIOMetricGroup(); + } + + @Override + public Counter getNumRecordsInErrorsCounter() { + return new SimpleCounter(); + } + + @Override + public void setPendingBytesGauge(Gauge pendingBytesGauge) {} + + @Override + public void setPendingRecordsGauge(Gauge pendingRecordsGauge) {} + + private static class TestingOperatorIOMetricGroup extends UnregisteredMetricsGroup + implements OperatorIOMetricGroup { + @Override + public Counter getNumRecordsInCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumRecordsOutCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumBytesInCounter() { + return new SimpleCounter(); + } + + @Override + public Counter getNumBytesOutCounter() { + return new SimpleCounter(); + } + } +} 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 new file mode 100644 index 000000000000..12bacdcd074d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.split; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.file.Path; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.flink.source.SplitHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceSplitSerializer { + + @TempDir protected Path temporaryFolder; + + private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); + + @Test + public void testLatestVersion() throws Exception { + serializeAndDeserialize(1, 1); + serializeAndDeserialize(10, 2); + } + + private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + temporaryFolder, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = serializer.serialize(split); + IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); + assertSplitEquals(split, deserialized); + + byte[] cachedResult = serializer.serialize(split); + assertThat(cachedResult).isSameAs(result); + IcebergSourceSplit deserialized2 = + serializer.deserialize(serializer.getVersion(), cachedResult); + assertSplitEquals(split, deserialized2); + + split.updatePosition(0, 100); + byte[] resultAfterUpdatePosition = serializer.serialize(split); + // after position change, serialized bytes should have changed + assertThat(resultAfterUpdatePosition).isNotSameAs(cachedResult); + IcebergSourceSplit deserialized3 = + serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); + assertSplitEquals(split, deserialized3); + } + } + + @Test + public void testV1() throws Exception { + serializeAndDeserializeV1(1, 1); + serializeAndDeserializeV1(10, 2); + } + + private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + temporaryFolder, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testV2() throws Exception { + serializeAndDeserializeV2(1, 1); + serializeAndDeserializeV2(10, 2); + } + + private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + temporaryFolder, splitCount, filesPerSplit); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV2(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testV3WithTooManyDeleteFiles() throws Exception { + serializeAndDeserializeV3(1, 1, 5000); + } + + private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mockDeletesPerSplit) + throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable( + temporaryFolder, splitCount, filesPerSplit); + final List splitsWithMockDeleteFiles = + SplitHelpers.equipSplitsWithMockDeleteFiles(splits, temporaryFolder, mockDeletesPerSplit); + + for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { + byte[] result = split.serializeV3(); + IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV3(result, true); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testDeserializeV1() throws Exception { + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); + for (IcebergSourceSplit split : splits) { + byte[] result = split.serializeV1(); + IcebergSourceSplit deserialized = serializer.deserialize(1, result); + assertSplitEquals(split, deserialized); + } + } + + @Test + public void testCheckpointedPosition() throws Exception { + final AtomicInteger index = new AtomicInteger(); + final List splits = + SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 2).stream() + .map( + split -> { + IcebergSourceSplit result; + if (index.get() % 2 == 0) { + result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); + } else { + result = split; + } + index.incrementAndGet(); + return result; + }) + .collect(Collectors.toList()); + + for (IcebergSourceSplit split : splits) { + byte[] result = serializer.serialize(split); + IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); + assertSplitEquals(split, deserialized); + + byte[] cachedResult = serializer.serialize(split); + assertThat(cachedResult).isSameAs(result); + IcebergSourceSplit deserialized2 = + serializer.deserialize(serializer.getVersion(), cachedResult); + assertSplitEquals(split, deserialized2); + } + } + + private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { + List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); + List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); + assertThat(actualTasks).hasSameSizeAs(expectedTasks); + 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.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); + assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); + assertThat(actualTask.start()).isEqualTo(expectedTask.start()); + assertThat(actualTask.length()).isEqualTo(expectedTask.length()); + } + + assertThat(actual.fileOffset()).isEqualTo(expected.fileOffset()); + assertThat(actual.recordOffset()).isEqualTo(expected.recordOffset()); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java new file mode 100644 index 000000000000..4ba4f9d983dc --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -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. + */ +package org.apache.iceberg.flink.util; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +public class TestFlinkPackage { + + /** This unit test would need to be adjusted as new Flink version is supported. */ + @Test + public void testVersion() { + assertThat(FlinkPackage.version()).isEqualTo("1.19.0"); + } + + @Test + public void testDefaultVersion() { + // It's difficult to reproduce a reflection error in a unit test, so we just inject a mocked + // fault to test the default logic + + // First make sure we're not caching a version result from a previous test + FlinkPackage.setVersion(null); + try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { + mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class); + mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); + } + FlinkPackage.setVersion(null); + try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { + mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null); + mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); + FlinkPackage.setVersion(null); + assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); + } + } +} diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory new file mode 100644 index 000000000000..47a3c94aa991 --- /dev/null +++ b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory @@ -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.flink.source.BoundedTableFactory From dee0a689ce40482910c75f6a92585ddf02ab26c7 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Mon, 5 Aug 2024 08:58:45 -0700 Subject: [PATCH 0570/1019] Flink: remove v1.17 module --- flink/v1.17/build.gradle | 268 ---- flink/v1.17/flink-runtime/LICENSE | 502 ------- flink/v1.17/flink-runtime/NOTICE | 91 -- .../flink/IcebergConnectorSmokeTest.java | 21 - .../shuffle/MapRangePartitionerBenchmark.java | 207 --- .../apache/iceberg/flink/CatalogLoader.java | 215 --- .../apache/iceberg/flink/FlinkCatalog.java | 834 ------------ .../iceberg/flink/FlinkCatalogFactory.java | 213 --- .../apache/iceberg/flink/FlinkConfParser.java | 261 ---- .../iceberg/flink/FlinkConfigOptions.java | 107 -- .../flink/FlinkDynamicTableFactory.java | 208 --- .../flink/FlinkEnvironmentContext.java | 31 - .../apache/iceberg/flink/FlinkFilters.java | 266 ---- .../apache/iceberg/flink/FlinkFixupTypes.java | 50 - .../apache/iceberg/flink/FlinkReadConf.java | 213 --- .../iceberg/flink/FlinkReadOptions.java | 123 -- .../apache/iceberg/flink/FlinkSchemaUtil.java | 232 ---- .../iceberg/flink/FlinkSourceFilter.java | 49 - .../apache/iceberg/flink/FlinkTypeToType.java | 203 --- .../iceberg/flink/FlinkTypeVisitor.java | 80 -- .../apache/iceberg/flink/FlinkWriteConf.java | 205 --- .../iceberg/flink/FlinkWriteOptions.java | 73 - .../iceberg/flink/IcebergTableSink.java | 121 -- .../apache/iceberg/flink/RowDataWrapper.java | 142 -- .../org/apache/iceberg/flink/TableLoader.java | 159 --- .../apache/iceberg/flink/TypeToFlinkType.java | 134 -- .../apache/iceberg/flink/actions/Actions.java | 52 - .../flink/actions/RewriteDataFilesAction.java | 72 - .../data/AvroWithFlinkSchemaVisitor.java | 75 -- .../iceberg/flink/data/FlinkAvroReader.java | 169 --- .../iceberg/flink/data/FlinkAvroWriter.java | 165 --- .../iceberg/flink/data/FlinkOrcReader.java | 131 -- .../iceberg/flink/data/FlinkOrcReaders.java | 283 ---- .../iceberg/flink/data/FlinkOrcWriter.java | 163 --- .../iceberg/flink/data/FlinkOrcWriters.java | 317 ----- .../flink/data/FlinkParquetReaders.java | 905 ------------- .../flink/data/FlinkParquetWriters.java | 504 ------- .../flink/data/FlinkSchemaVisitor.java | 161 --- .../iceberg/flink/data/FlinkValueReaders.java | 312 ----- .../iceberg/flink/data/FlinkValueWriters.java | 253 ---- .../data/ParquetWithFlinkSchemaVisitor.java | 222 ---- .../iceberg/flink/data/RowDataProjection.java | 341 ----- .../iceberg/flink/data/RowDataUtil.java | 123 -- .../iceberg/flink/data/StructRowData.java | 300 ----- .../AvroGenericRecordToRowDataMapper.java | 61 - .../flink/sink/BaseDeltaTaskWriter.java | 126 -- .../sink/BucketPartitionKeySelector.java | 70 - .../iceberg/flink/sink/BucketPartitioner.java | 103 -- .../flink/sink/BucketPartitionerUtil.java | 125 -- .../flink/sink/CachingTableSupplier.java | 91 -- .../iceberg/flink/sink/CommitSummary.java | 93 -- .../iceberg/flink/sink/DeltaManifests.java | 71 - .../flink/sink/DeltaManifestsSerializer.java | 122 -- .../flink/sink/EqualityFieldKeySelector.java | 86 -- .../flink/sink/FlinkAppenderFactory.java | 280 ---- .../flink/sink/FlinkFileWriterFactory.java | 293 ----- .../iceberg/flink/sink/FlinkManifestUtil.java | 132 -- .../apache/iceberg/flink/sink/FlinkSink.java | 654 --------- .../flink/sink/IcebergFilesCommitter.java | 516 -------- .../sink/IcebergFilesCommitterMetrics.java | 96 -- .../flink/sink/IcebergStreamWriter.java | 120 -- .../sink/IcebergStreamWriterMetrics.java | 89 -- .../flink/sink/ManifestOutputFileFactory.java | 94 -- .../flink/sink/PartitionKeySelector.java | 64 - .../flink/sink/PartitionedDeltaWriter.java | 97 -- .../flink/sink/RowDataTaskWriterFactory.java | 244 ---- .../iceberg/flink/sink/TaskWriterFactory.java | 45 - .../flink/sink/UnpartitionedDeltaWriter.java | 69 - .../shuffle/AggregatedStatisticsTracker.java | 262 ---- .../sink/shuffle/CompletedStatistics.java | 111 -- .../CompletedStatisticsSerializer.java | 178 --- .../flink/sink/shuffle/DataStatistics.java | 48 - .../shuffle/DataStatisticsCoordinator.java | 522 -------- .../DataStatisticsCoordinatorProvider.java | 70 - .../sink/shuffle/DataStatisticsOperator.java | 265 ---- .../shuffle/DataStatisticsSerializer.java | 206 --- .../flink/sink/shuffle/GlobalStatistics.java | 114 -- .../shuffle/GlobalStatisticsSerializer.java | 201 --- .../flink/sink/shuffle/KeyAssignment.java | 155 --- .../flink/sink/shuffle/MapAssignment.java | 242 ---- .../flink/sink/shuffle/MapDataStatistics.java | 88 -- .../sink/shuffle/MapRangePartitioner.java | 95 -- .../flink/sink/shuffle/RangePartitioner.java | 110 -- .../shuffle/RequestGlobalStatisticsEvent.java | 38 - .../sink/shuffle/SketchDataStatistics.java | 87 -- .../sink/shuffle/SketchRangePartitioner.java | 51 - .../flink/sink/shuffle/SketchUtil.java | 159 --- .../flink/sink/shuffle/SortKeySerializer.java | 373 ------ .../sink/shuffle/SortKeySketchSerializer.java | 143 -- .../flink/sink/shuffle/SortKeyUtil.java | 59 - .../flink/sink/shuffle/StatisticsEvent.java | 76 -- .../sink/shuffle/StatisticsOrRecord.java | 112 -- .../shuffle/StatisticsOrRecordSerializer.java | 210 --- .../flink/sink/shuffle/StatisticsType.java | 55 - .../flink/sink/shuffle/StatisticsUtil.java | 126 -- .../AvroGenericRecordFileScanTaskReader.java | 42 - .../iceberg/flink/source/DataIterator.java | 156 --- .../iceberg/flink/source/DataTaskReader.java | 47 - .../flink/source/FileScanTaskReader.java | 35 - .../flink/source/FlinkInputFormat.java | 141 -- .../iceberg/flink/source/FlinkInputSplit.java | 48 - .../iceberg/flink/source/FlinkSource.java | 307 ----- .../flink/source/FlinkSplitPlanner.java | 189 --- .../iceberg/flink/source/IcebergSource.java | 549 -------- .../flink/source/IcebergTableSource.java | 229 ---- .../source/RowDataFileScanTaskReader.java | 243 ---- .../iceberg/flink/source/RowDataRewriter.java | 172 --- .../RowDataToAvroGenericRecordConverter.java | 70 - .../iceberg/flink/source/ScanContext.java | 597 --------- .../iceberg/flink/source/SourceUtil.java | 77 -- .../source/StreamingMonitorFunction.java | 269 ---- .../flink/source/StreamingReaderOperator.java | 246 ---- .../source/StreamingStartingStrategy.java | 54 - .../source/assigner/DefaultSplitAssigner.java | 112 -- .../flink/source/assigner/GetSplitResult.java | 77 -- .../assigner/OrderedSplitAssignerFactory.java | 46 - .../assigner/SimpleSplitAssignerFactory.java | 37 - .../flink/source/assigner/SplitAssigner.java | 118 -- .../source/assigner/SplitAssignerFactory.java | 30 - .../source/assigner/SplitAssignerType.java | 33 - .../enumerator/AbstractIcebergEnumerator.java | 182 --- .../ContinuousEnumerationResult.java | 57 - .../ContinuousIcebergEnumerator.java | 178 --- .../enumerator/ContinuousSplitPlanner.java | 30 - .../ContinuousSplitPlannerImpl.java | 240 ---- .../source/enumerator/EnumerationHistory.java | 100 -- .../enumerator/IcebergEnumeratorPosition.java | 79 -- .../IcebergEnumeratorPositionSerializer.java | 90 -- .../enumerator/IcebergEnumeratorState.java | 65 - .../IcebergEnumeratorStateSerializer.java | 194 --- .../enumerator/StaticIcebergEnumerator.java | 51 - .../source/reader/ArrayBatchRecords.java | 171 --- .../reader/ArrayPoolDataIteratorBatcher.java | 130 -- .../AvroGenericRecordReaderFunction.java | 102 -- .../reader/ColumnStatsWatermarkExtractor.java | 98 -- .../source/reader/DataIteratorBatcher.java | 36 - .../reader/DataIteratorReaderFunction.java | 43 - .../source/reader/IcebergSourceReader.java | 77 -- .../reader/IcebergSourceReaderMetrics.java | 61 - .../reader/IcebergSourceSplitReader.java | 167 --- .../source/reader/LimitableDataIterator.java | 56 - .../flink/source/reader/ListBatchRecords.java | 85 -- .../reader/ListDataIteratorBatcher.java | 94 -- .../source/reader/MetaDataReaderFunction.java | 65 - .../flink/source/reader/ReaderFunction.java | 31 - .../source/reader/RecordAndPosition.java | 78 -- .../flink/source/reader/RecordFactory.java | 34 - .../flink/source/reader/RecordLimiter.java | 45 - .../source/reader/RowDataReaderFunction.java | 115 -- .../source/reader/RowDataRecordFactory.java | 73 - .../reader/SerializableRecordEmitter.java | 40 - .../reader/SplitWatermarkExtractor.java | 28 - .../WatermarkExtractorRecordEmitter.java | 67 - .../source/split/IcebergSourceSplit.java | 220 ---- .../split/IcebergSourceSplitSerializer.java | 62 - .../source/split/IcebergSourceSplitState.java | 37 - .../split/IcebergSourceSplitStatus.java | 25 - .../source/split/SerializableComparator.java | 24 - .../flink/source/split/SerializerHelper.java | 206 --- .../flink/source/split/SplitComparators.java | 76 -- .../flink/source/split/SplitRequestEvent.java | 54 - .../flink/util/FlinkAlterTableUtil.java | 248 ---- .../flink/util/FlinkCompatibilityUtil.java | 47 - .../iceberg/flink/util/FlinkPackage.java | 61 - .../org.apache.flink.table.factories.Factory | 16 - ....apache.flink.table.factories.TableFactory | 16 - .../flink/AvroGenericRecordConverterBase.java | 90 -- .../apache/iceberg/flink/CatalogTestBase.java | 143 -- .../apache/iceberg/flink/DataGenerator.java | 42 - .../apache/iceberg/flink/DataGenerators.java | 1172 ----------------- .../iceberg/flink/HadoopCatalogExtension.java | 105 -- .../iceberg/flink/HadoopTableExtension.java | 59 - .../flink/MiniFlinkClusterExtension.java | 67 - .../iceberg/flink/RowDataConverter.java | 135 -- .../apache/iceberg/flink/SimpleDataUtil.java | 439 ------ .../org/apache/iceberg/flink/TestBase.java | 128 -- .../iceberg/flink/TestCatalogLoader.java | 116 -- .../iceberg/flink/TestCatalogTableLoader.java | 113 -- .../iceberg/flink/TestChangeLogTable.java | 296 ----- .../flink/TestDataFileSerialization.java | 203 --- .../apache/iceberg/flink/TestFixtures.java | 61 - .../flink/TestFlinkAnonymousTable.java | 65 - .../flink/TestFlinkCatalogDatabase.java | 253 ---- .../flink/TestFlinkCatalogFactory.java | 119 -- .../iceberg/flink/TestFlinkCatalogTable.java | 669 ---------- .../TestFlinkCatalogTablePartitions.java | 119 -- .../iceberg/flink/TestFlinkConfParser.java | 61 - .../iceberg/flink/TestFlinkFilters.java | 462 ------- .../iceberg/flink/TestFlinkHiveCatalog.java | 101 -- .../iceberg/flink/TestFlinkSchemaUtil.java | 416 ------ .../iceberg/flink/TestFlinkTableSink.java | 358 ----- .../apache/iceberg/flink/TestFlinkUpsert.java | 334 ----- .../org/apache/iceberg/flink/TestHelpers.java | 632 --------- .../iceberg/flink/TestIcebergConnector.java | 343 ----- .../flink/TestManifestFileSerialization.java | 173 --- .../iceberg/flink/TestRowDataWrapper.java | 93 -- .../apache/iceberg/flink/TestTableLoader.java | 57 - .../iceberg/flink/TestTableSerialization.java | 110 -- .../actions/TestRewriteDataFilesAction.java | 481 ------- .../iceberg/flink/data/RandomRowData.java | 38 - .../flink/data/RowDataToRowMapper.java | 50 - .../flink/data/TestFlinkAvroReaderWriter.java | 185 --- .../flink/data/TestFlinkOrcReaderWriter.java | 107 -- .../flink/data/TestFlinkParquetReader.java | 239 ---- .../flink/data/TestFlinkParquetWriter.java | 94 -- .../flink/data/TestRowDataProjection.java | 593 --------- .../iceberg/flink/data/TestRowProjection.java | 594 --------- .../iceberg/flink/data/TestStructRowData.java | 100 -- .../TestAvroGenericRecordToRowDataMapper.java | 38 - .../sink/TestBucketPartitionKeySelector.java | 67 - .../flink/sink/TestBucketPartitioner.java | 108 -- ...TestBucketPartitionerFlinkIcebergSink.java | 227 ---- .../flink/sink/TestBucketPartitionerUtil.java | 126 -- .../flink/sink/TestCachingTableSupplier.java | 81 -- .../flink/sink/TestCompressionSettings.java | 257 ---- .../flink/sink/TestDeltaTaskWriter.java | 429 ------ .../flink/sink/TestFlinkAppenderFactory.java | 65 - .../sink/TestFlinkFileWriterFactory.java | 66 - .../flink/sink/TestFlinkIcebergSink.java | 385 ------ .../flink/sink/TestFlinkIcebergSinkBase.java | 64 - .../sink/TestFlinkIcebergSinkBranch.java | 137 -- .../flink/sink/TestFlinkIcebergSinkV2.java | 235 ---- .../sink/TestFlinkIcebergSinkV2Base.java | 389 ------ .../sink/TestFlinkIcebergSinkV2Branch.java | 125 -- .../iceberg/flink/sink/TestFlinkManifest.java | 312 ----- .../sink/TestFlinkPartitioningWriters.java | 66 - .../sink/TestFlinkPositionDeltaWriters.java | 66 - .../sink/TestFlinkRollingFileWriters.java | 51 - .../flink/sink/TestFlinkWriterMetrics.java | 60 - .../flink/sink/TestIcebergFilesCommitter.java | 1148 ---------------- .../flink/sink/TestIcebergStreamWriter.java | 390 ------ .../flink/sink/TestRowDataPartitionKey.java | 251 ---- .../iceberg/flink/sink/TestTaskWriters.java | 242 ---- .../iceberg/flink/sink/shuffle/Fixtures.java | 100 -- .../TestAggregatedStatisticsTracker.java | 465 ------- .../TestCompletedStatisticsSerializer.java | 54 - .../TestDataStatisticsCoordinator.java | 246 ---- ...TestDataStatisticsCoordinatorProvider.java | 187 --- .../shuffle/TestDataStatisticsOperator.java | 350 ----- .../shuffle/TestDataStatisticsSerializer.java | 53 - .../TestGlobalStatisticsSerializer.java | 59 - .../sink/shuffle/TestMapDataStatistics.java | 67 - .../sink/shuffle/TestMapRangePartitioner.java | 434 ------ .../sink/shuffle/TestRangePartitioner.java | 65 - .../shuffle/TestSketchDataStatistics.java | 60 - .../shuffle/TestSketchRangePartitioner.java | 88 -- .../flink/sink/shuffle/TestSketchUtil.java | 189 --- .../shuffle/TestSortKeySerializerBase.java | 65 - .../TestSortKeySerializerNestedStruct.java | 55 - .../TestSortKeySerializerPrimitives.java | 90 -- .../TestSortKeySerializerSnapshot.java | 213 --- .../flink/sink/shuffle/TestSortKeyUtil.java | 73 - .../flink/source/BoundedTableFactory.java | 170 --- .../flink/source/BoundedTestSource.java | 108 -- .../flink/source/ChangeLogTableTestBase.java | 95 -- .../iceberg/flink/source/SplitHelpers.java | 200 --- .../iceberg/flink/source/SqlHelpers.java | 60 - .../flink/source/TableSourceTestBase.java | 104 -- .../flink/source/TestBoundedTableFactory.java | 81 -- .../flink/source/TestFlinkInputFormat.java | 211 --- .../TestFlinkInputFormatReaderDeletes.java | 70 - .../flink/source/TestFlinkMergingMetrics.java | 67 - .../flink/source/TestFlinkMetaDataTable.java | 813 ------------ .../source/TestFlinkReaderDeletesBase.java | 90 -- .../iceberg/flink/source/TestFlinkScan.java | 540 -------- .../flink/source/TestFlinkScanSql.java | 69 - .../iceberg/flink/source/TestFlinkSource.java | 90 -- .../flink/source/TestFlinkSourceConfig.java | 61 - .../flink/source/TestFlinkSourceSql.java | 85 -- .../flink/source/TestFlinkTableSource.java | 561 -------- .../source/TestIcebergSourceBounded.java | 147 --- ...TestIcebergSourceBoundedGenericRecord.java | 196 --- .../source/TestIcebergSourceBoundedSql.java | 76 -- .../source/TestIcebergSourceContinuous.java | 538 -------- .../source/TestIcebergSourceFailover.java | 394 ------ ...gSourceFailoverWithWatermarkExtractor.java | 130 -- .../TestIcebergSourceReaderDeletes.java | 102 -- .../flink/source/TestIcebergSourceSql.java | 158 --- ...stIcebergSourceWithWatermarkExtractor.java | 408 ------ ...estIcebergSpeculativeExecutionSupport.java | 184 --- .../TestMetadataTableReadableMetrics.java | 299 ----- .../flink/source/TestProjectMetaColumn.java | 188 --- ...stRowDataToAvroGenericRecordConverter.java | 36 - .../iceberg/flink/source/TestScanContext.java | 112 -- .../iceberg/flink/source/TestSourceUtil.java | 61 - .../iceberg/flink/source/TestSqlBase.java | 160 --- .../flink/source/TestStreamScanSql.java | 434 ------ .../source/TestStreamingMonitorFunction.java | 402 ------ .../source/TestStreamingReaderOperator.java | 293 ----- .../assigner/SplitAssignerTestBase.java | 132 -- .../assigner/TestDefaultSplitAssigner.java | 43 - ...tFileSequenceNumberBasedSplitAssigner.java | 81 -- .../TestWatermarkBasedSplitAssigner.java | 146 -- .../ManualContinuousSplitPlanner.java | 97 -- .../TestContinuousIcebergEnumerator.java | 352 ----- .../TestContinuousSplitPlannerImpl.java | 692 ---------- ...ntinuousSplitPlannerImplStartStrategy.java | 200 --- .../enumerator/TestEnumerationHistory.java | 135 -- .../TestIcebergEnumeratorStateSerializer.java | 146 -- .../source/reader/ReaderFunctionTestBase.java | 218 --- .../flink/source/reader/ReaderUtil.java | 128 -- .../source/reader/TestArrayBatchRecords.java | 69 - ...stArrayPoolDataIteratorBatcherRowData.java | 360 ----- .../TestColumnStatsWatermarkExtractor.java | 176 --- .../reader/TestIcebergSourceReader.java | 181 --- .../reader/TestLimitableDataIterator.java | 84 -- .../reader/TestRowDataReaderFunction.java | 69 - .../source/reader/TestingMetricGroup.java | 102 -- .../TestIcebergSourceSplitSerializer.java | 183 --- .../iceberg/flink/util/TestFlinkPackage.java | 55 - .../org.apache.flink.table.factories.Factory | 16 - 311 files changed, 55202 deletions(-) delete mode 100644 flink/v1.17/build.gradle delete mode 100644 flink/v1.17/flink-runtime/LICENSE delete mode 100644 flink/v1.17/flink-runtime/NOTICE delete mode 100644 flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java delete mode 100644 flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java delete mode 100644 flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java delete mode 100644 flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory delete mode 100644 flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java delete mode 100644 flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java delete mode 100644 flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory diff --git a/flink/v1.17/build.gradle b/flink/v1.17/build.gradle deleted file mode 100644 index 0278e4dc3b73..000000000000 --- a/flink/v1.17/build.gradle +++ /dev/null @@ -1,268 +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. - */ - -String flinkMajorVersion = '1.17' -String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") - -project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { - - dependencies { - implementation project(path: ':iceberg-bundled-guava', configuration: 'shadow') - api project(':iceberg-api') - implementation project(':iceberg-common') - implementation project(':iceberg-core') - api project(':iceberg-data') - implementation project(':iceberg-orc') - implementation project(':iceberg-parquet') - implementation project(':iceberg-hive-metastore') - - compileOnly libs.flink117.avro - // for dropwizard histogram metrics implementation - compileOnly libs.flink117.metrics.dropwizard - compileOnly libs.flink117.streaming.java - compileOnly "${libs.flink117.streaming.java.get().module}:${libs.flink117.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink117.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" - compileOnly libs.flink117.connector.base - compileOnly libs.flink117.connector.files - - compileOnly libs.hadoop2.hdfs - compileOnly libs.hadoop2.common - compileOnly(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - - 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.avro.avro - - implementation("${libs.orc.core.get().module}:${libs.versions.orc.get()}:nohive") { - exclude group: 'org.apache.hadoop' - exclude group: 'commons-lang' - // These artifacts are shaded and included in the orc-core fat jar - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.hive', module: 'hive-storage-api' - exclude group: 'org.slf4j' - } - - implementation libs.datasketches - - testImplementation libs.flink117.connector.test.utils - testImplementation libs.flink117.core - testImplementation libs.flink117.runtime - testImplementation(libs.flink117.test.utilsjunit) { - exclude group: 'junit' - } - testImplementation(libs.flink117.test.utils) { - exclude group: "org.apache.curator", module: 'curator-test' - exclude group: 'junit' - } - - testImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-core', configuration: 'testArtifacts') - testImplementation project(path: ':iceberg-data', configuration: 'testArtifacts') - - // By default, hive-exec is a fat/uber jar and it exports a guava library - // that's really old. We use the core classifier to be able to override our guava - // version. Luckily, hive-exec seems to work okay so far with this version of guava - // See: https://github.com/apache/hive/blob/master/ql/pom.xml#L911 for more context. - testImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hive', module: 'hive-llap-tez' - exclude group: 'org.apache.logging.log4j' - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.calcite' - exclude group: 'org.apache.calcite.avatica' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - - testImplementation(libs.hive2.metastore) { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hbase' - exclude group: 'org.apache.logging.log4j' - exclude group: 'co.cask.tephra' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' - exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' - exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' - exclude group: 'com.tdunning', module: 'json' - exclude group: 'javax.transaction', module: 'transaction-api' - exclude group: 'com.zaxxer', module: 'HikariCP' - exclude group: 'org.slf4j' - } - - testImplementation libs.awaitility - testImplementation libs.assertj.core - } - - test { - useJUnitPlatform() - } -} - -project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' - - tasks.jar.dependsOn tasks.shadowJar - - sourceSets { - integration { - java.srcDir "$projectDir/src/integration/java" - resources.srcDir "$projectDir/src/integration/resources" - } - } - - configurations { - implementation { - // included in Flink - exclude group: 'org.slf4j' - exclude group: 'org.apache.commons' - exclude group: 'commons-pool' - exclude group: 'commons-codec' - exclude group: 'org.xerial.snappy' - exclude group: 'javax.xml.bind' - exclude group: 'javax.annotation' - } - } - - dependencies { - implementation(project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}")) { - exclude group: 'org.apache.flink' - } - implementation project(':iceberg-aws') - implementation project(':iceberg-azure') - implementation(project(':iceberg-aliyun')) { - exclude group: 'edu.umd.cs.findbugs', module: 'findbugs' - exclude group: 'org.apache.httpcomponents', module: 'httpclient' - exclude group: 'commons-logging', module: 'commons-logging' - } - implementation project(':iceberg-gcp') - implementation(project(':iceberg-nessie')) { - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - - // for dropwizard histogram metrics implementation - implementation libs.flink117.metrics.dropwizard - - // for integration testing with the flink-runtime-jar - // all of those dependencies are required because the integration test extends FlinkTestBase - integrationCompileOnly project(':iceberg-api') - integrationImplementation libs.junit.vintage.engine - integrationImplementation libs.assertj.core - integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") - integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') - integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink117.test.utils) { - exclude group: "org.apache.curator", module: 'curator-test' - exclude group: 'junit' - } - - integrationImplementation libs.flink117.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink117.get()}" - - integrationImplementation libs.hadoop2.common - integrationImplementation libs.hadoop2.hdfs - integrationImplementation(libs.hadoop2.minicluster) { - exclude group: 'org.apache.avro', module: 'avro' - } - - integrationImplementation(libs.hive2.metastore) { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hbase' - exclude group: 'org.apache.logging.log4j' - exclude group: 'co.cask.tephra' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - exclude group: 'org.eclipse.jetty.aggregate', module: 'jetty-all' - exclude group: 'org.eclipse.jetty.orbit', module: 'javax.servlet' - exclude group: 'org.apache.parquet', module: 'parquet-hadoop-bundle' - exclude group: 'com.tdunning', module: 'json' - exclude group: 'javax.transaction', module: 'transaction-api' - exclude group: 'com.zaxxer', module: 'HikariCP' - exclude group: 'org.slf4j' - } - - integrationImplementation("${libs.hive2.exec.get().module}:${libs.hive2.exec.get().getVersion()}:core") { - exclude group: 'org.apache.avro', module: 'avro' - exclude group: 'org.slf4j', module: 'slf4j-log4j12' - exclude group: 'org.pentaho' // missing dependency - exclude group: 'org.apache.hive', module: 'hive-llap-tez' - exclude group: 'org.apache.logging.log4j' - exclude group: 'com.google.protobuf', module: 'protobuf-java' - exclude group: 'org.apache.calcite' - exclude group: 'org.apache.calcite.avatica' - exclude group: 'com.google.code.findbugs', module: 'jsr305' - } - } - - shadowJar { - configurations = [project.configurations.runtimeClasspath] - - zip64 true - - // include the LICENSE and NOTICE files for the shaded Jar - from(projectDir) { - include 'LICENSE' - include 'NOTICE' - } - - // Relocate dependencies to avoid conflicts - relocate 'org.apache.avro', 'org.apache.iceberg.shaded.org.apache.avro' - relocate 'org.apache.parquet', 'org.apache.iceberg.shaded.org.apache.parquet' - relocate 'com.google.errorprone', 'org.apache.iceberg.shaded.com.google.errorprone' - relocate 'com.google.flatbuffers', 'org.apache.iceberg.shaded.com.google.flatbuffers' - relocate 'com.fasterxml', 'org.apache.iceberg.shaded.com.fasterxml' - relocate 'com.github.benmanes', 'org.apache.iceberg.shaded.com.github.benmanes' - relocate 'org.checkerframework', 'org.apache.iceberg.shaded.org.checkerframework' - relocate 'shaded.parquet', 'org.apache.iceberg.shaded.org.apache.parquet.shaded' - relocate 'org.apache.orc', 'org.apache.iceberg.shaded.org.apache.orc' - relocate 'io.airlift', 'org.apache.iceberg.shaded.io.airlift' - relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' - relocate 'org.apache.hc.client5', 'org.apache.iceberg.shaded.org.apache.hc.client5' - relocate 'org.apache.hc.core5', 'org.apache.iceberg.shaded.org.apache.hc.core5' - - archiveClassifier.set(null) - } - - task integrationTest(type: Test) { - description = "Test Flink Runtime Jar against Flink ${flinkMajorVersion}" - group = "verification" - jvmArgs += project.property('extraJvmArgs') - testClassesDirs = sourceSets.integration.output.classesDirs - classpath = sourceSets.integration.runtimeClasspath + files(shadowJar.archiveFile.get().asFile.path) - inputs.file(shadowJar.archiveFile.get().asFile.path) - } - integrationTest.dependsOn shadowJar - check.dependsOn integrationTest - - jar { - enabled = false - } -} diff --git a/flink/v1.17/flink-runtime/LICENSE b/flink/v1.17/flink-runtime/LICENSE deleted file mode 100644 index 8ab53469eb87..000000000000 --- a/flink/v1.17/flink-runtime/LICENSE +++ /dev/null @@ -1,502 +0,0 @@ - - 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 Apache Avro. - -Copyright: 2014-2020 The Apache Software Foundation. -Home page: https://parquet.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains the Jackson JSON processor. - -Copyright: 2007-2020 Tatu Saloranta and other contributors -Home page: http://jackson.codehaus.org/ -License: http://www.apache.org/licenses/LICENSE-2.0.txt - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Parquet. - -Copyright: 2014-2020 The Apache Software Foundation. -Home page: https://parquet.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Thrift. - -Copyright: 2006-2010 The Apache Software Foundation. -Home page: https://thrift.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains fastutil. - -Copyright: 2002-2014 Sebastiano Vigna -Home page: http://fastutil.di.unimi.it/ -License: http://www.apache.org/licenses/LICENSE-2.0.html - --------------------------------------------------------------------------------- - -This binary artifact contains Apache ORC. - -Copyright: 2013-2020 The Apache Software Foundation. -Home page: https://orc.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Hive's storage API via ORC. - -Copyright: 2013-2020 The Apache Software Foundation. -Home page: https://hive.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Airlift Aircompressor. - -Copyright: 2011-2020 Aircompressor authors. -Home page: https://github.com/airlift/aircompressor -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Airlift Slice. - -Copyright: 2013-2020 Slice authors. -Home page: https://github.com/airlift/slice -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains JetBrains annotations. - -Copyright: 2000-2020 JetBrains s.r.o. -Home page: https://github.com/JetBrains/java-annotations -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google Guava. - -Copyright: 2006-2020 The Guava Authors -Home page: https://github.com/google/guava -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google Error Prone Annotations. - -Copyright: Copyright 2011-2019 The Error Prone Authors -Home page: https://github.com/google/error-prone -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains checkerframework checker-qual Annotations. - -Copyright: 2004-2020 the Checker Framework developers -Home page: https://github.com/typetools/checker-framework -License: https://github.com/typetools/checker-framework/blob/master/LICENSE.txt (MIT license) - -License text: -| The annotations are licensed under the MIT License. (The text of this -| license appears below.) More specifically, all the parts of the Checker -| Framework that you might want to include with your own program use the -| MIT License. This is the checker-qual.jar file and all the files that -| appear in it: every file in a qual/ directory, plus utility files such -| as NullnessUtil.java, RegexUtil.java, SignednessUtil.java, etc. -| In addition, the cleanroom implementations of third-party annotations, -| which the Checker Framework recognizes as aliases for its own -| annotations, are licensed under the MIT License. -| -| Permission is hereby granted, free of charge, to any person obtaining a copy -| of this software and associated documentation files (the "Software"), to deal -| in the Software without restriction, including without limitation the rights -| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -| copies of the Software, and to permit persons to whom the Software is -| furnished to do so, subject to the following conditions: -| -| The above copyright notice and this permission notice shall be included in -| all copies or substantial portions of the Software. -| -| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -| THE SOFTWARE. - --------------------------------------------------------------------------------- - -This binary artifact contains Animal Sniffer Annotations. - -Copyright: 2009-2018 codehaus.org -Home page: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/ -License: https://www.mojohaus.org/animal-sniffer/animal-sniffer-annotations/license.html (MIT license) - -License text: -| The MIT License -| -| Copyright (c) 2009 codehaus.org. -| -| Permission is hereby granted, free of charge, to any person obtaining a copy -| of this software and associated documentation files (the "Software"), to deal -| in the Software without restriction, including without limitation the rights -| to use, copy, modify, merge, publish, distribute, sublicense, and/or sell -| copies of the Software, and to permit persons to whom the Software is -| furnished to do so, subject to the following conditions: -| -| The above copyright notice and this permission notice shall be included in -| all copies or substantial portions of the Software. -| -| THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR -| IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, -| FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE -| AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER -| LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, -| OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN -| THE SOFTWARE. - --------------------------------------------------------------------------------- - -This binary artifact contains Caffeine by Ben Manes. - -Copyright: 2014-2020 Ben Manes and contributors -Home page: https://github.com/ben-manes/caffeine -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Apache Yetus audience annotations. - -Copyright: 2008-2020 The Apache Software Foundation. -Home page: https://yetus.apache.org/ -License: http://www.apache.org/licenses/LICENSE-2.0 - --------------------------------------------------------------------------------- - -This binary artifact contains Google protobuf. - -Copyright: 2008 Google Inc. -Home page: https://developers.google.com/protocol-buffers -License: https://github.com/protocolbuffers/protobuf/blob/master/LICENSE (BSD) - -License text: - -| Copyright 2008 Google Inc. All rights reserved. -| -| Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are -| met: -| -| * Redistributions of source code must retain the above copyright -| notice, this list of conditions and the following disclaimer. -| * Redistributions in binary form must reproduce the above -| copyright notice, this list of conditions and the following disclaimer -| in the documentation and/or other materials provided with the -| distribution. -| * Neither the name of Google Inc. nor the names of its -| contributors may be used to endorse or promote products derived from -| this software without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. -| -| Code generated by the Protocol Buffer compiler is owned by the owner -| of the input file used when generating it. This code is not -| standalone and requires a support library to be linked with it. This -| support library is itself covered by the above license. - --------------------------------------------------------------------------------- - -This binary artifact contains ThreeTen. - -Copyright: 2007-present, Stephen Colebourne & Michael Nascimento Santos. -Home page: https://www.threeten.org/threeten-extra/ -License: https://github.com/ThreeTen/threeten-extra/blob/master/LICENSE.txt (BSD 3-clause) - -License text: - -| All rights reserved. -| -| * Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are met: -| -| * Redistributions of source code must retain the above copyright notice, -| this list of conditions and the following disclaimer. -| -| * Redistributions in binary form must reproduce the above copyright notice, -| this list of conditions and the following disclaimer in the documentation -| and/or other materials provided with the distribution. -| -| * Neither the name of JSR-310 nor the names of its contributors -| may be used to endorse or promote products derived from this software -| without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT OWNER OR -| CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, -| EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT LIMITED TO, -| PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, DATA, OR -| PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF -| LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING -| NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS -| SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - -This binary artifact includes Project Nessie with the following in its NOTICE -file: - -| Dremio -| Copyright 2015-2017 Dremio Corporation -| -| This product includes software developed at -| The Apache Software Foundation (http://www.apache.org/). - --------------------------------------------------------------------------------- - -This binary 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 binary artifact contains Apache HttpComponents Client. - -Copyright: 1999-2022 The Apache Software Foundation. -Home page: https://hc.apache.org/ -License: http://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 diff --git a/flink/v1.17/flink-runtime/NOTICE b/flink/v1.17/flink-runtime/NOTICE deleted file mode 100644 index dc36f84c4ac5..000000000000 --- a/flink/v1.17/flink-runtime/NOTICE +++ /dev/null @@ -1,91 +0,0 @@ - -Apache Iceberg -Copyright 2017-2024 The Apache Software Foundation - -This product includes software developed at -The Apache Software Foundation (http://www.apache.org/). - --------------------------------------------------------------------------------- - -This binary artifact includes Apache ORC with the following in its NOTICE file: - -| Apache ORC -| Copyright 2013-2019 The Apache Software Foundation -| -| This product includes software developed by The Apache Software -| Foundation (http://www.apache.org/). -| -| This product includes software developed by Hewlett-Packard: -| (c) Copyright [2014-2015] Hewlett-Packard Development Company, L.P - --------------------------------------------------------------------------------- - -This binary artifact includes Airlift Aircompressor with the following in its -NOTICE file: - -| Snappy Copyright Notices -| ========================= -| -| * Copyright 2011 Dain Sundstrom -| * Copyright 2011, Google Inc. -| -| -| Snappy License -| =============== -| Copyright 2011, Google Inc. -| All rights reserved. -| -| Redistribution and use in source and binary forms, with or without -| modification, are permitted provided that the following conditions are -| met: -| -| * Redistributions of source code must retain the above copyright -| notice, this list of conditions and the following disclaimer. -| * Redistributions in binary form must reproduce the above -| copyright notice, this list of conditions and the following disclaimer -| in the documentation and/or other materials provided with the -| distribution. -| * Neither the name of Google Inc. nor the names of its -| contributors may be used to endorse or promote products derived from -| this software without specific prior written permission. -| -| THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS -| "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT -| LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR -| A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT -| OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, -| SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT -| LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE, -| DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY -| THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT -| (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE -| OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. - --------------------------------------------------------------------------------- - -This binary artifact includes Apache Yetus with the following in its NOTICE -file: - -| Apache Yetus -| Copyright 2008-2020 The Apache Software Foundation -| -| This product includes software developed at -| The Apache Software Foundation (https://www.apache.org/). -| -| --- -| Additional licenses for the Apache Yetus Source/Website: -| --- -| -| -| See LICENSE for terms. - --------------------------------------------------------------------------------- - -This binary artifact includes Project Nessie with the following in its NOTICE -file: - -| Dremio -| Copyright 2015-2017 Dremio Corporation -| -| This product includes software developed at -| The Apache Software Foundation (http://www.apache.org/). diff --git a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java b/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java deleted file mode 100644 index 0d9bbf9d3601..000000000000 --- a/flink/v1.17/flink-runtime/src/integration/java/org/apache/iceberg/flink/IcebergConnectorSmokeTest.java +++ /dev/null @@ -1,21 +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; - -public class IcebergConnectorSmokeTest extends TestIcebergConnector {} diff --git a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java deleted file mode 100644 index a9ad386a5a4a..000000000000 --- a/flink/v1.17/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ /dev/null @@ -1,207 +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.sink.shuffle; - -import java.nio.charset.StandardCharsets; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; -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.apache.iceberg.types.Types; -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.TearDown; -import org.openjdk.jmh.annotations.Threads; -import org.openjdk.jmh.annotations.Warmup; -import org.openjdk.jmh.infra.Blackhole; - -@Fork(1) -@State(Scope.Benchmark) -@Warmup(iterations = 3) -@Measurement(iterations = 5) -@BenchmarkMode(Mode.SingleShotTime) -public class MapRangePartitionerBenchmark { - private static final String CHARS = - "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ0123456789_-.!?"; - private static final int SAMPLE_SIZE = 100_000; - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "name2", Types.StringType.get()), - Types.NestedField.required(3, "name3", Types.StringType.get()), - Types.NestedField.required(4, "name4", Types.StringType.get()), - Types.NestedField.required(5, "name5", Types.StringType.get()), - Types.NestedField.required(6, "name6", Types.StringType.get()), - Types.NestedField.required(7, "name7", Types.StringType.get()), - Types.NestedField.required(8, "name8", Types.StringType.get()), - Types.NestedField.required(9, "name9", Types.StringType.get())); - - private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - private static final Comparator SORT_ORDER_COMPARTOR = - SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); - private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); - - private MapRangePartitioner partitioner; - private RowData[] rows; - - @Setup - public void setupBenchmark() { - NavigableMap weights = longTailDistribution(100_000, 24, 240, 100, 2.0); - Map mapStatistics = Maps.newHashMapWithExpectedSize(weights.size()); - weights.forEach( - (id, weight) -> { - SortKey sortKey = SORT_KEY.copy(); - sortKey.set(0, id); - mapStatistics.put(sortKey, weight); - }); - - MapAssignment mapAssignment = - MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); - this.partitioner = - new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); - - List keys = Lists.newArrayList(weights.keySet().iterator()); - long[] weightsCDF = new long[keys.size()]; - long totalWeight = 0; - for (int i = 0; i < keys.size(); ++i) { - totalWeight += weights.get(keys.get(i)); - weightsCDF[i] = totalWeight; - } - - // pre-calculate the samples for benchmark run - this.rows = new GenericRowData[SAMPLE_SIZE]; - for (int i = 0; i < SAMPLE_SIZE; ++i) { - long weight = ThreadLocalRandom.current().nextLong(totalWeight); - int index = binarySearchIndex(weightsCDF, weight); - rows[i] = - GenericRowData.of( - keys.get(index), - randomString("name2-"), - randomString("name3-"), - randomString("name4-"), - randomString("name5-"), - randomString("name6-"), - randomString("name7-"), - randomString("name8-"), - randomString("name9-")); - } - } - - @TearDown - public void tearDownBenchmark() {} - - @Benchmark - @Threads(1) - public void testPartitionerLongTailDistribution(Blackhole blackhole) { - for (int i = 0; i < SAMPLE_SIZE; ++i) { - blackhole.consume(partitioner.partition(rows[i], 128)); - } - } - - private static String randomString(String prefix) { - int length = ThreadLocalRandom.current().nextInt(200); - byte[] buffer = new byte[length]; - - for (int i = 0; i < length; i += 1) { - buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); - } - - // CHARS is all ASCII - return prefix + new String(buffer, StandardCharsets.US_ASCII); - } - - /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ - private static int binarySearchIndex(long[] weightsUDF, long target) { - Preconditions.checkArgument( - target < weightsUDF[weightsUDF.length - 1], - "weight is out of range: total weight = %s, search target = %s", - weightsUDF[weightsUDF.length - 1], - target); - int start = 0; - int end = weightsUDF.length - 1; - while (start < end) { - int mid = (start + end) / 2; - if (weightsUDF[mid] < target && weightsUDF[mid + 1] >= target) { - return mid; - } - - if (weightsUDF[mid] >= target) { - end = mid - 1; - } else if (weightsUDF[mid + 1] < target) { - start = mid + 1; - } - } - return start; - } - - /** Key is the id string and value is the weight in long value. */ - private static NavigableMap longTailDistribution( - long startingWeight, - int longTailStartingIndex, - int longTailLength, - long longTailBaseWeight, - double weightRandomJitterPercentage) { - - NavigableMap weights = Maps.newTreeMap(); - - // first part just decays the weight by half - long currentWeight = startingWeight; - for (int index = 0; index < longTailStartingIndex; ++index) { - double jitter = ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage / 100); - long weight = (long) (currentWeight * (1.0 + jitter)); - weight = weight > 0 ? weight : 1; - weights.put(index, weight); - if (currentWeight > longTailBaseWeight) { - currentWeight = currentWeight / 2; - } - } - - // long tail part - for (int index = longTailStartingIndex; - index < longTailStartingIndex + longTailLength; - ++index) { - long longTailWeight = - (long) - (longTailBaseWeight - * ThreadLocalRandom.current().nextDouble(weightRandomJitterPercentage)); - longTailWeight = longTailWeight > 0 ? longTailWeight : 1; - weights.put(index, longTailWeight); - } - - return weights; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java deleted file mode 100644 index 18473bf4f190..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/CatalogLoader.java +++ /dev/null @@ -1,215 +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; - -import java.io.Serializable; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hadoop.SerializableConfiguration; -import org.apache.iceberg.hive.HiveCatalog; -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.Maps; -import org.apache.iceberg.rest.RESTCatalog; - -/** Serializable loader to load an Iceberg {@link Catalog}. */ -public interface CatalogLoader extends Serializable, Cloneable { - - /** - * Create a new catalog with the provided properties. NOTICE: for flink, we may initialize the - * {@link CatalogLoader} at flink sql client side or job manager side, and then serialize this - * catalog loader to task manager, finally deserialize it and create a new catalog at task manager - * side. - * - * @return a newly created {@link Catalog} - */ - Catalog loadCatalog(); - - /** Clone a CatalogLoader. */ - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - CatalogLoader clone(); - - static CatalogLoader hadoop( - String name, Configuration hadoopConf, Map properties) { - return new HadoopCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader hive(String name, Configuration hadoopConf, Map properties) { - return new HiveCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader rest(String name, Configuration hadoopConf, Map properties) { - return new RESTCatalogLoader(name, hadoopConf, properties); - } - - static CatalogLoader custom( - String name, Map properties, Configuration hadoopConf, String impl) { - return new CustomCatalogLoader(name, properties, hadoopConf, impl); - } - - class HadoopCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final String warehouseLocation; - private final Map properties; - - private HadoopCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.warehouseLocation = properties.get(CatalogProperties.WAREHOUSE_LOCATION); - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - HadoopCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new HadoopCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("warehouseLocation", warehouseLocation) - .toString(); - } - } - - class HiveCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final String uri; - private final String warehouse; - private final int clientPoolSize; - private final Map properties; - - private HiveCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.uri = properties.get(CatalogProperties.URI); - this.warehouse = properties.get(CatalogProperties.WAREHOUSE_LOCATION); - this.clientPoolSize = - properties.containsKey(CatalogProperties.CLIENT_POOL_SIZE) - ? Integer.parseInt(properties.get(CatalogProperties.CLIENT_POOL_SIZE)) - : CatalogProperties.CLIENT_POOL_SIZE_DEFAULT; - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new HiveCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("uri", uri) - .add("warehouse", warehouse) - .add("clientPoolSize", clientPoolSize) - .toString(); - } - } - - class RESTCatalogLoader implements CatalogLoader { - private final String catalogName; - private final SerializableConfiguration hadoopConf; - private final Map properties; - - private RESTCatalogLoader( - String catalogName, Configuration conf, Map properties) { - this.catalogName = catalogName; - this.hadoopConf = new SerializableConfiguration(conf); - this.properties = Maps.newHashMap(properties); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog( - RESTCatalog.class.getName(), catalogName, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new RESTCatalogLoader(catalogName, new Configuration(hadoopConf.get()), properties); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("catalogName", catalogName) - .add("properties", properties) - .toString(); - } - } - - class CustomCatalogLoader implements CatalogLoader { - - private final SerializableConfiguration hadoopConf; - private final Map properties; - private final String name; - private final String impl; - - private CustomCatalogLoader( - String name, Map properties, Configuration conf, String impl) { - this.hadoopConf = new SerializableConfiguration(conf); - this.properties = Maps.newHashMap(properties); // wrap into a hashmap for serialization - this.name = name; - this.impl = - Preconditions.checkNotNull( - impl, "Cannot initialize custom Catalog, impl class name is null"); - } - - @Override - public Catalog loadCatalog() { - return CatalogUtil.loadCatalog(impl, name, properties, hadoopConf.get()); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public CatalogLoader clone() { - return new CustomCatalogLoader(name, properties, new Configuration(hadoopConf.get()), impl); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("name", name).add("impl", impl).toString(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java deleted file mode 100644 index 988465a38f4d..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalog.java +++ /dev/null @@ -1,834 +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; - -import java.io.Closeable; -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.AbstractCatalog; -import org.apache.flink.table.catalog.CatalogBaseTable; -import org.apache.flink.table.catalog.CatalogDatabase; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.CatalogFunction; -import org.apache.flink.table.catalog.CatalogPartition; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.CatalogTableImpl; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.TableChange; -import org.apache.flink.table.catalog.exceptions.CatalogException; -import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; -import org.apache.flink.table.catalog.exceptions.DatabaseNotExistException; -import org.apache.flink.table.catalog.exceptions.FunctionNotExistException; -import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; -import org.apache.flink.table.catalog.stats.CatalogColumnStatistics; -import org.apache.flink.table.catalog.stats.CatalogTableStatistics; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.factories.Factory; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.CachingCatalog; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -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.AlreadyExistsException; -import org.apache.iceberg.exceptions.NamespaceNotEmptyException; -import org.apache.iceberg.exceptions.NoSuchNamespaceException; -import org.apache.iceberg.flink.util.FlinkAlterTableUtil; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Splitter; -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.relocated.com.google.common.collect.Sets; - -/** - * A Flink Catalog implementation that wraps an Iceberg {@link Catalog}. - * - *

    The mapping between Flink database and Iceberg namespace: Supplying a base namespace for a - * given catalog, so if you have a catalog that supports a 2-level namespace, you would supply the - * first level in the catalog configuration and the second level would be exposed as Flink - * databases. - * - *

    The Iceberg table manages its partitions by itself. The partition of the Iceberg table is - * independent of the partition of Flink. - */ -public class FlinkCatalog extends AbstractCatalog { - private final CatalogLoader catalogLoader; - private final Catalog icebergCatalog; - private final Namespace baseNamespace; - private final SupportsNamespaces asNamespaceCatalog; - private final Closeable closeable; - private final boolean cacheEnabled; - - public FlinkCatalog( - String catalogName, - String defaultDatabase, - Namespace baseNamespace, - CatalogLoader catalogLoader, - boolean cacheEnabled, - long cacheExpirationIntervalMs) { - super(catalogName, defaultDatabase); - this.catalogLoader = catalogLoader; - this.baseNamespace = baseNamespace; - this.cacheEnabled = cacheEnabled; - - Catalog originalCatalog = catalogLoader.loadCatalog(); - icebergCatalog = - cacheEnabled - ? CachingCatalog.wrap(originalCatalog, cacheExpirationIntervalMs) - : originalCatalog; - asNamespaceCatalog = - originalCatalog instanceof SupportsNamespaces ? (SupportsNamespaces) originalCatalog : null; - closeable = originalCatalog instanceof Closeable ? (Closeable) originalCatalog : null; - - FlinkEnvironmentContext.init(); - } - - @Override - public void open() throws CatalogException {} - - @Override - public void close() throws CatalogException { - if (closeable != null) { - try { - closeable.close(); - } catch (IOException e) { - throw new CatalogException(e); - } - } - } - - public Catalog catalog() { - return icebergCatalog; - } - - /** Append a new level to the base namespace */ - private static Namespace appendLevel(Namespace baseNamespace, String newLevel) { - String[] namespace = new String[baseNamespace.levels().length + 1]; - System.arraycopy(baseNamespace.levels(), 0, namespace, 0, baseNamespace.levels().length); - namespace[baseNamespace.levels().length] = newLevel; - return Namespace.of(namespace); - } - - TableIdentifier toIdentifier(ObjectPath path) { - String objectName = path.getObjectName(); - List tableName = Splitter.on('$').splitToList(objectName); - - if (tableName.size() == 1) { - return TableIdentifier.of( - appendLevel(baseNamespace, path.getDatabaseName()), path.getObjectName()); - } else if (tableName.size() == 2 && MetadataTableType.from(tableName.get(1)) != null) { - return TableIdentifier.of( - appendLevel(appendLevel(baseNamespace, path.getDatabaseName()), tableName.get(0)), - tableName.get(1)); - } else { - throw new IllegalArgumentException("Illegal table name:" + objectName); - } - } - - @Override - public List listDatabases() throws CatalogException { - if (asNamespaceCatalog == null) { - return Collections.singletonList(getDefaultDatabase()); - } - - return asNamespaceCatalog.listNamespaces(baseNamespace).stream() - .map(n -> n.level(n.levels().length - 1)) - .collect(Collectors.toList()); - } - - @Override - public CatalogDatabase getDatabase(String databaseName) - throws DatabaseNotExistException, CatalogException { - if (asNamespaceCatalog == null) { - if (!getDefaultDatabase().equals(databaseName)) { - throw new DatabaseNotExistException(getName(), databaseName); - } else { - return new CatalogDatabaseImpl(Maps.newHashMap(), ""); - } - } else { - try { - Map metadata = - Maps.newHashMap( - asNamespaceCatalog.loadNamespaceMetadata(appendLevel(baseNamespace, databaseName))); - String comment = metadata.remove("comment"); - return new CatalogDatabaseImpl(metadata, comment); - } catch (NoSuchNamespaceException e) { - throw new DatabaseNotExistException(getName(), databaseName, e); - } - } - } - - @Override - public boolean databaseExists(String databaseName) throws CatalogException { - try { - getDatabase(databaseName); - return true; - } catch (DatabaseNotExistException ignore) { - return false; - } - } - - @Override - public void createDatabase(String name, CatalogDatabase database, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - createDatabase( - name, mergeComment(database.getProperties(), database.getComment()), ignoreIfExists); - } - - private void createDatabase( - String databaseName, Map metadata, boolean ignoreIfExists) - throws DatabaseAlreadyExistException, CatalogException { - if (asNamespaceCatalog != null) { - try { - asNamespaceCatalog.createNamespace(appendLevel(baseNamespace, databaseName), metadata); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new DatabaseAlreadyExistException(getName(), databaseName, e); - } - } - } else { - throw new UnsupportedOperationException( - "Namespaces are not supported by catalog: " + getName()); - } - } - - private Map mergeComment(Map metadata, String comment) { - Map ret = Maps.newHashMap(metadata); - if (metadata.containsKey("comment")) { - throw new CatalogException("Database properties should not contain key: 'comment'."); - } - - if (!StringUtils.isNullOrWhitespaceOnly(comment)) { - ret.put("comment", comment); - } - return ret; - } - - @Override - public void dropDatabase(String name, boolean ignoreIfNotExists, boolean cascade) - throws DatabaseNotExistException, DatabaseNotEmptyException, CatalogException { - if (asNamespaceCatalog != null) { - try { - boolean success = asNamespaceCatalog.dropNamespace(appendLevel(baseNamespace, name)); - if (!success && !ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } catch (NoSuchNamespaceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name, e); - } - } catch (NamespaceNotEmptyException e) { - throw new DatabaseNotEmptyException(getName(), name, e); - } - } else { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } - } - - @Override - public void alterDatabase(String name, CatalogDatabase newDatabase, boolean ignoreIfNotExists) - throws DatabaseNotExistException, CatalogException { - if (asNamespaceCatalog != null) { - Namespace namespace = appendLevel(baseNamespace, name); - Map updates = Maps.newHashMap(); - Set removals = Sets.newHashSet(); - - try { - Map oldProperties = asNamespaceCatalog.loadNamespaceMetadata(namespace); - Map newProperties = - mergeComment(newDatabase.getProperties(), newDatabase.getComment()); - - for (String key : oldProperties.keySet()) { - if (!newProperties.containsKey(key)) { - removals.add(key); - } - } - - for (Map.Entry entry : newProperties.entrySet()) { - if (!entry.getValue().equals(oldProperties.get(entry.getKey()))) { - updates.put(entry.getKey(), entry.getValue()); - } - } - - if (!updates.isEmpty()) { - asNamespaceCatalog.setProperties(namespace, updates); - } - - if (!removals.isEmpty()) { - asNamespaceCatalog.removeProperties(namespace, removals); - } - - } catch (NoSuchNamespaceException e) { - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name, e); - } - } - } else { - if (getDefaultDatabase().equals(name)) { - throw new CatalogException( - "Can not alter the default database when the iceberg catalog doesn't support namespaces."); - } - if (!ignoreIfNotExists) { - throw new DatabaseNotExistException(getName(), name); - } - } - } - - @Override - public List listTables(String databaseName) - throws DatabaseNotExistException, CatalogException { - try { - return icebergCatalog.listTables(appendLevel(baseNamespace, databaseName)).stream() - .map(TableIdentifier::name) - .collect(Collectors.toList()); - } catch (NoSuchNamespaceException e) { - throw new DatabaseNotExistException(getName(), databaseName, e); - } - } - - @Override - public CatalogTable getTable(ObjectPath tablePath) - throws TableNotExistException, CatalogException { - Table table = loadIcebergTable(tablePath); - return toCatalogTable(table); - } - - private Table loadIcebergTable(ObjectPath tablePath) throws TableNotExistException { - try { - Table table = icebergCatalog.loadTable(toIdentifier(tablePath)); - if (cacheEnabled) { - table.refresh(); - } - - return table; - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - throw new TableNotExistException(getName(), tablePath, e); - } - } - - @Override - public boolean tableExists(ObjectPath tablePath) throws CatalogException { - return icebergCatalog.tableExists(toIdentifier(tablePath)); - } - - @Override - public void dropTable(ObjectPath tablePath, boolean ignoreIfNotExists) - throws TableNotExistException, CatalogException { - try { - icebergCatalog.dropTable(toIdentifier(tablePath)); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - if (!ignoreIfNotExists) { - throw new TableNotExistException(getName(), tablePath, e); - } - } - } - - @Override - public void renameTable(ObjectPath tablePath, String newTableName, boolean ignoreIfNotExists) - throws TableNotExistException, TableAlreadyExistException, CatalogException { - try { - icebergCatalog.renameTable( - toIdentifier(tablePath), - toIdentifier(new ObjectPath(tablePath.getDatabaseName(), newTableName))); - } catch (org.apache.iceberg.exceptions.NoSuchTableException e) { - if (!ignoreIfNotExists) { - throw new TableNotExistException(getName(), tablePath, e); - } - } catch (AlreadyExistsException e) { - throw new TableAlreadyExistException(getName(), tablePath, e); - } - } - - @Override - public void createTable(ObjectPath tablePath, CatalogBaseTable table, boolean ignoreIfExists) - throws CatalogException, TableAlreadyExistException { - if (Objects.equals( - table.getOptions().get("connector"), FlinkDynamicTableFactory.FACTORY_IDENTIFIER)) { - throw new IllegalArgumentException( - "Cannot create the table with 'connector'='iceberg' table property in " - + "an iceberg catalog, Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " - + "create table without 'connector'='iceberg' related properties in an iceberg table."); - } - Preconditions.checkArgument(table instanceof ResolvedCatalogTable, "table should be resolved"); - createIcebergTable(tablePath, (ResolvedCatalogTable) table, ignoreIfExists); - } - - void createIcebergTable(ObjectPath tablePath, ResolvedCatalogTable table, boolean ignoreIfExists) - throws CatalogException, TableAlreadyExistException { - validateFlinkTable(table); - - Schema icebergSchema = FlinkSchemaUtil.convert(table.getResolvedSchema()); - PartitionSpec spec = toPartitionSpec(((CatalogTable) table).getPartitionKeys(), icebergSchema); - - ImmutableMap.Builder properties = ImmutableMap.builder(); - String location = null; - for (Map.Entry entry : table.getOptions().entrySet()) { - if ("location".equalsIgnoreCase(entry.getKey())) { - location = entry.getValue(); - } else { - properties.put(entry.getKey(), entry.getValue()); - } - } - - try { - icebergCatalog.createTable( - toIdentifier(tablePath), icebergSchema, spec, location, properties.build()); - } catch (AlreadyExistsException e) { - if (!ignoreIfExists) { - throw new TableAlreadyExistException(getName(), tablePath, e); - } - } - } - - private static void validateTableSchemaAndPartition(CatalogTable ct1, CatalogTable ct2) { - TableSchema ts1 = ct1.getSchema(); - TableSchema ts2 = ct2.getSchema(); - boolean equalsPrimary = false; - - if (ts1.getPrimaryKey().isPresent() && ts2.getPrimaryKey().isPresent()) { - equalsPrimary = - Objects.equals(ts1.getPrimaryKey().get().getType(), ts2.getPrimaryKey().get().getType()) - && Objects.equals( - ts1.getPrimaryKey().get().getColumns(), ts2.getPrimaryKey().get().getColumns()); - } else if (!ts1.getPrimaryKey().isPresent() && !ts2.getPrimaryKey().isPresent()) { - equalsPrimary = true; - } - - if (!(Objects.equals(ts1.getTableColumns(), ts2.getTableColumns()) - && Objects.equals(ts1.getWatermarkSpecs(), ts2.getWatermarkSpecs()) - && equalsPrimary)) { - throw new UnsupportedOperationException( - "Altering schema is not supported in the old alterTable API. " - + "To alter schema, use the other alterTable API and provide a list of TableChange's."); - } - - validateTablePartition(ct1, ct2); - } - - private static void validateTablePartition(CatalogTable ct1, CatalogTable ct2) { - if (!ct1.getPartitionKeys().equals(ct2.getPartitionKeys())) { - throw new UnsupportedOperationException("Altering partition keys is not supported yet."); - } - } - - /** - * This alterTable API only supports altering table properties. - * - *

    Support for adding/removing/renaming columns cannot be done by comparing CatalogTable - * instances, unless the Flink schema contains Iceberg column IDs. - * - *

    To alter columns, use the other alterTable API and provide a list of TableChange's. - * - * @param tablePath path of the table or view to be modified - * @param newTable the new table definition - * @param ignoreIfNotExists flag to specify behavior when the table or view does not exist: if set - * to false, throw an exception, if set to true, do nothing. - * @throws CatalogException in case of any runtime exception - * @throws TableNotExistException if the table does not exist - */ - @Override - public void alterTable(ObjectPath tablePath, CatalogBaseTable newTable, boolean ignoreIfNotExists) - throws CatalogException, TableNotExistException { - validateFlinkTable(newTable); - - Table icebergTable; - try { - icebergTable = loadIcebergTable(tablePath); - } catch (TableNotExistException e) { - if (!ignoreIfNotExists) { - throw e; - } else { - return; - } - } - - CatalogTable table = toCatalogTable(icebergTable); - validateTableSchemaAndPartition(table, (CatalogTable) newTable); - - Map oldProperties = table.getOptions(); - Map setProperties = Maps.newHashMap(); - - String setLocation = null; - String setSnapshotId = null; - String pickSnapshotId = null; - - for (Map.Entry entry : newTable.getOptions().entrySet()) { - String key = entry.getKey(); - String value = entry.getValue(); - - if (Objects.equals(value, oldProperties.get(key))) { - continue; - } - - if ("location".equalsIgnoreCase(key)) { - setLocation = value; - } else if ("current-snapshot-id".equalsIgnoreCase(key)) { - setSnapshotId = value; - } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(key)) { - pickSnapshotId = value; - } else { - setProperties.put(key, value); - } - } - - oldProperties - .keySet() - .forEach( - k -> { - if (!newTable.getOptions().containsKey(k)) { - setProperties.put(k, null); - } - }); - - FlinkAlterTableUtil.commitChanges( - icebergTable, setLocation, setSnapshotId, pickSnapshotId, setProperties); - } - - @Override - public void alterTable( - ObjectPath tablePath, - CatalogBaseTable newTable, - List tableChanges, - boolean ignoreIfNotExists) - throws TableNotExistException, CatalogException { - validateFlinkTable(newTable); - - Table icebergTable; - try { - icebergTable = loadIcebergTable(tablePath); - } catch (TableNotExistException e) { - if (!ignoreIfNotExists) { - throw e; - } else { - return; - } - } - - // Does not support altering partition yet. - validateTablePartition(toCatalogTable(icebergTable), (CatalogTable) newTable); - - String setLocation = null; - String setSnapshotId = null; - String cherrypickSnapshotId = null; - - List propertyChanges = Lists.newArrayList(); - List schemaChanges = Lists.newArrayList(); - for (TableChange change : tableChanges) { - if (change instanceof TableChange.SetOption) { - TableChange.SetOption set = (TableChange.SetOption) change; - - if ("location".equalsIgnoreCase(set.getKey())) { - setLocation = set.getValue(); - } else if ("current-snapshot-id".equalsIgnoreCase(set.getKey())) { - setSnapshotId = set.getValue(); - } else if ("cherry-pick-snapshot-id".equalsIgnoreCase(set.getKey())) { - cherrypickSnapshotId = set.getValue(); - } else { - propertyChanges.add(change); - } - } else if (change instanceof TableChange.ResetOption) { - propertyChanges.add(change); - } else { - schemaChanges.add(change); - } - } - - FlinkAlterTableUtil.commitChanges( - icebergTable, - setLocation, - setSnapshotId, - cherrypickSnapshotId, - schemaChanges, - propertyChanges); - } - - private static void validateFlinkTable(CatalogBaseTable table) { - Preconditions.checkArgument( - table instanceof CatalogTable, "The Table should be a CatalogTable."); - - TableSchema schema = table.getSchema(); - schema - .getTableColumns() - .forEach( - column -> { - if (!FlinkCompatibilityUtil.isPhysicalColumn(column)) { - throw new UnsupportedOperationException( - "Creating table with computed columns is not supported yet."); - } - }); - - if (!schema.getWatermarkSpecs().isEmpty()) { - throw new UnsupportedOperationException( - "Creating table with watermark specs is not supported yet."); - } - } - - private static PartitionSpec toPartitionSpec(List partitionKeys, Schema icebergSchema) { - PartitionSpec.Builder builder = PartitionSpec.builderFor(icebergSchema); - partitionKeys.forEach(builder::identity); - return builder.build(); - } - - private static List toPartitionKeys(PartitionSpec spec, Schema icebergSchema) { - ImmutableList.Builder partitionKeysBuilder = ImmutableList.builder(); - for (PartitionField field : spec.fields()) { - if (field.transform().isIdentity()) { - partitionKeysBuilder.add(icebergSchema.findColumnName(field.sourceId())); - } else { - // Not created by Flink SQL. - // For compatibility with iceberg tables, return empty. - // TODO modify this after Flink support partition transform. - return Collections.emptyList(); - } - } - return partitionKeysBuilder.build(); - } - - static CatalogTable toCatalogTable(Table table) { - TableSchema schema = FlinkSchemaUtil.toSchema(table.schema()); - List partitionKeys = toPartitionKeys(table.spec(), table.schema()); - - // NOTE: We can not create a IcebergCatalogTable extends CatalogTable, because Flink optimizer - // may use - // CatalogTableImpl to copy a new catalog table. - // Let's re-loading table from Iceberg catalog when creating source/sink operators. - // Iceberg does not have Table comment, so pass a null (Default comment value in Flink). - return new CatalogTableImpl(schema, partitionKeys, table.properties(), null); - } - - @Override - public Optional getFactory() { - return Optional.of(new FlinkDynamicTableFactory(this)); - } - - CatalogLoader getCatalogLoader() { - return catalogLoader; - } - - // ------------------------------ Unsupported methods - // --------------------------------------------- - - @Override - public List listViews(String databaseName) throws CatalogException { - return Collections.emptyList(); - } - - @Override - public CatalogPartition getPartition(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean partitionExists(ObjectPath tablePath, CatalogPartitionSpec partitionSpec) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void createPartition( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogPartition partition, - boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropPartition( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartition( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogPartition newPartition, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listFunctions(String dbName) throws CatalogException { - return Collections.emptyList(); - } - - @Override - public CatalogFunction getFunction(ObjectPath functionPath) - throws FunctionNotExistException, CatalogException { - throw new FunctionNotExistException(getName(), functionPath); - } - - @Override - public boolean functionExists(ObjectPath functionPath) throws CatalogException { - return false; - } - - @Override - public void createFunction( - ObjectPath functionPath, CatalogFunction function, boolean ignoreIfExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterFunction( - ObjectPath functionPath, CatalogFunction newFunction, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void dropFunction(ObjectPath functionPath, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTableStatistics( - ObjectPath tablePath, CatalogTableStatistics tableStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterTableColumnStatistics( - ObjectPath tablePath, CatalogColumnStatistics columnStatistics, boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartitionStatistics( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogTableStatistics partitionStatistics, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public void alterPartitionColumnStatistics( - ObjectPath tablePath, - CatalogPartitionSpec partitionSpec, - CatalogColumnStatistics columnStatistics, - boolean ignoreIfNotExists) - throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listPartitions(ObjectPath tablePath) - throws TableNotExistException, TableNotPartitionedException, CatalogException { - Table table = loadIcebergTable(tablePath); - - if (table.spec().isUnpartitioned()) { - throw new TableNotPartitionedException(icebergCatalog.name(), tablePath); - } - - Set set = Sets.newHashSet(); - try (CloseableIterable tasks = table.newScan().planFiles()) { - for (DataFile dataFile : CloseableIterable.transform(tasks, FileScanTask::file)) { - Map map = Maps.newHashMap(); - StructLike structLike = dataFile.partition(); - PartitionSpec spec = table.specs().get(dataFile.specId()); - for (int i = 0; i < structLike.size(); i++) { - map.put(spec.fields().get(i).name(), String.valueOf(structLike.get(i, Object.class))); - } - set.add(new CatalogPartitionSpec(map)); - } - } catch (IOException e) { - throw new CatalogException( - String.format("Failed to list partitions of table %s", tablePath), e); - } - - return Lists.newArrayList(set); - } - - @Override - public List listPartitions( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - throw new UnsupportedOperationException(); - } - - @Override - public List listPartitionsByFilter( - ObjectPath tablePath, List filters) throws CatalogException { - throw new UnsupportedOperationException(); - } - - // After partition pruning and filter push down, the statistics have become very inaccurate, so - // the statistics from - // here are of little significance. - // Flink will support something like SupportsReportStatistics in future. - - @Override - public CatalogTableStatistics getTableStatistics(ObjectPath tablePath) throws CatalogException { - return CatalogTableStatistics.UNKNOWN; - } - - @Override - public CatalogColumnStatistics getTableColumnStatistics(ObjectPath tablePath) - throws CatalogException { - return CatalogColumnStatistics.UNKNOWN; - } - - @Override - public CatalogTableStatistics getPartitionStatistics( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - return CatalogTableStatistics.UNKNOWN; - } - - @Override - public CatalogColumnStatistics getPartitionColumnStatistics( - ObjectPath tablePath, CatalogPartitionSpec partitionSpec) throws CatalogException { - return CatalogColumnStatistics.UNKNOWN; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java deleted file mode 100644 index 1453753849ec..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ /dev/null @@ -1,213 +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; - -import java.net.URL; -import java.nio.file.Files; -import java.nio.file.Paths; -import java.util.List; -import java.util.Locale; -import java.util.Map; -import org.apache.flink.configuration.GlobalConfiguration; -import org.apache.flink.runtime.util.HadoopUtils; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.factories.CatalogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.base.Strings; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.PropertyUtil; - -/** - * A Flink Catalog factory implementation that creates {@link FlinkCatalog}. - * - *

    This supports the following catalog configuration options: - * - *

      - *
    • type - Flink catalog factory key, should be "iceberg" - *
    • catalog-type - iceberg catalog type, "hive", "hadoop" or "rest" - *
    • uri - the Hive Metastore URI (Hive catalog only) - *
    • clients - the Hive Client Pool Size (Hive catalog only) - *
    • warehouse - the warehouse path (Hadoop catalog only) - *
    • default-database - a database name to use as the default - *
    • base-namespace - a base namespace as the prefix for all databases (Hadoop - * catalog only) - *
    • cache-enabled - whether to enable catalog cache - *
    - * - *

    To use a custom catalog that is not a Hive or Hadoop catalog, extend this class and override - * {@link #createCatalogLoader(String, Map, Configuration)}. - */ -public class FlinkCatalogFactory implements CatalogFactory { - - // Can not just use "type", it conflicts with CATALOG_TYPE. - public static final String ICEBERG_CATALOG_TYPE = "catalog-type"; - public static final String ICEBERG_CATALOG_TYPE_HADOOP = "hadoop"; - public static final String ICEBERG_CATALOG_TYPE_HIVE = "hive"; - public static final String ICEBERG_CATALOG_TYPE_REST = "rest"; - - public static final String HIVE_CONF_DIR = "hive-conf-dir"; - public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; - public static final String DEFAULT_DATABASE = "default-database"; - public static final String DEFAULT_DATABASE_NAME = "default"; - public static final String BASE_NAMESPACE = "base-namespace"; - - public static final String TYPE = "type"; - public static final String PROPERTY_VERSION = "property-version"; - - /** - * Create an Iceberg {@link org.apache.iceberg.catalog.Catalog} loader to be used by this Flink - * catalog adapter. - * - * @param name Flink's catalog name - * @param properties Flink's catalog properties - * @param hadoopConf Hadoop configuration for catalog - * @return an Iceberg catalog loader - */ - static CatalogLoader createCatalogLoader( - String name, Map properties, Configuration hadoopConf) { - String catalogImpl = properties.get(CatalogProperties.CATALOG_IMPL); - if (catalogImpl != null) { - String catalogType = properties.get(ICEBERG_CATALOG_TYPE); - Preconditions.checkArgument( - catalogType == null, - "Cannot create catalog %s, both catalog-type and catalog-impl are set: catalog-type=%s, catalog-impl=%s", - name, - catalogType, - catalogImpl); - return CatalogLoader.custom(name, properties, hadoopConf, catalogImpl); - } - - String catalogType = properties.getOrDefault(ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HIVE); - switch (catalogType.toLowerCase(Locale.ENGLISH)) { - case ICEBERG_CATALOG_TYPE_HIVE: - // The values of properties 'uri', 'warehouse', 'hive-conf-dir' are allowed to be null, in - // that case it will - // fallback to parse those values from hadoop configuration which is loaded from classpath. - String hiveConfDir = properties.get(HIVE_CONF_DIR); - String hadoopConfDir = properties.get(HADOOP_CONF_DIR); - Configuration newHadoopConf = mergeHiveConf(hadoopConf, hiveConfDir, hadoopConfDir); - return CatalogLoader.hive(name, newHadoopConf, properties); - - case ICEBERG_CATALOG_TYPE_HADOOP: - return CatalogLoader.hadoop(name, hadoopConf, properties); - - case ICEBERG_CATALOG_TYPE_REST: - return CatalogLoader.rest(name, hadoopConf, properties); - - default: - throw new UnsupportedOperationException( - "Unknown catalog-type: " + catalogType + " (Must be 'hive', 'hadoop' or 'rest')"); - } - } - - @Override - public Map requiredContext() { - Map context = Maps.newHashMap(); - context.put(TYPE, "iceberg"); - context.put(PROPERTY_VERSION, "1"); - return context; - } - - @Override - public List supportedProperties() { - return ImmutableList.of("*"); - } - - @Override - public Catalog createCatalog(String name, Map properties) { - return createCatalog(name, properties, clusterHadoopConf()); - } - - protected Catalog createCatalog( - String name, Map properties, Configuration hadoopConf) { - CatalogLoader catalogLoader = createCatalogLoader(name, properties, hadoopConf); - String defaultDatabase = properties.getOrDefault(DEFAULT_DATABASE, DEFAULT_DATABASE_NAME); - - Namespace baseNamespace = Namespace.empty(); - if (properties.containsKey(BASE_NAMESPACE)) { - baseNamespace = Namespace.of(properties.get(BASE_NAMESPACE).split("\\.")); - } - - boolean cacheEnabled = - PropertyUtil.propertyAsBoolean( - properties, CatalogProperties.CACHE_ENABLED, CatalogProperties.CACHE_ENABLED_DEFAULT); - - long cacheExpirationIntervalMs = - PropertyUtil.propertyAsLong( - properties, - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS, - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS_OFF); - Preconditions.checkArgument( - cacheExpirationIntervalMs != 0, - "%s is not allowed to be 0.", - CatalogProperties.CACHE_EXPIRATION_INTERVAL_MS); - - return new FlinkCatalog( - name, - defaultDatabase, - baseNamespace, - catalogLoader, - cacheEnabled, - cacheExpirationIntervalMs); - } - - private static Configuration mergeHiveConf( - Configuration hadoopConf, String hiveConfDir, String hadoopConfDir) { - Configuration newConf = new Configuration(hadoopConf); - if (!Strings.isNullOrEmpty(hiveConfDir)) { - Preconditions.checkState( - Files.exists(Paths.get(hiveConfDir, "hive-site.xml")), - "There should be a hive-site.xml file under the directory %s", - hiveConfDir); - newConf.addResource(new Path(hiveConfDir, "hive-site.xml")); - } else { - // If don't provide the hive-site.xml path explicitly, it will try to load resource from - // classpath. If still - // couldn't load the configuration file, then it will throw exception in HiveCatalog. - URL configFile = CatalogLoader.class.getClassLoader().getResource("hive-site.xml"); - if (configFile != null) { - newConf.addResource(configFile); - } - } - - if (!Strings.isNullOrEmpty(hadoopConfDir)) { - Preconditions.checkState( - Files.exists(Paths.get(hadoopConfDir, "hdfs-site.xml")), - "Failed to load Hadoop configuration: missing %s", - Paths.get(hadoopConfDir, "hdfs-site.xml")); - newConf.addResource(new Path(hadoopConfDir, "hdfs-site.xml")); - Preconditions.checkState( - Files.exists(Paths.get(hadoopConfDir, "core-site.xml")), - "Failed to load Hadoop configuration: missing %s", - Paths.get(hadoopConfDir, "core-site.xml")); - newConf.addResource(new Path(hadoopConfDir, "core-site.xml")); - } - - return newConf; - } - - public static Configuration clusterHadoopConf() { - return HadoopUtils.getHadoopConfiguration(GlobalConfiguration.loadConfiguration()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java deleted file mode 100644 index 7167859e600c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ /dev/null @@ -1,261 +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; - -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.function.Function; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class FlinkConfParser { - - private final Map tableProperties; - private final Map options; - private final ReadableConfig readableConfig; - - FlinkConfParser(Table table, Map options, ReadableConfig readableConfig) { - this.tableProperties = table.properties(); - this.options = options; - this.readableConfig = readableConfig; - } - - public BooleanConfParser booleanConf() { - return new BooleanConfParser(); - } - - public IntConfParser intConf() { - return new IntConfParser(); - } - - public LongConfParser longConf() { - return new LongConfParser(); - } - - public > EnumConfParser enumConfParser(Class enumClass) { - return new EnumConfParser<>(enumClass); - } - - public StringConfParser stringConf() { - return new StringConfParser(); - } - - public DurationConfParser durationConf() { - return new DurationConfParser(); - } - - class BooleanConfParser extends ConfParser { - private Boolean defaultValue; - - @Override - protected BooleanConfParser self() { - return this; - } - - public BooleanConfParser defaultValue(boolean value) { - this.defaultValue = value; - return self(); - } - - public BooleanConfParser defaultValue(String value) { - this.defaultValue = Boolean.parseBoolean(value); - return self(); - } - - public boolean parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Boolean::parseBoolean, defaultValue); - } - } - - class IntConfParser extends ConfParser { - private Integer defaultValue; - - @Override - protected IntConfParser self() { - return this; - } - - public IntConfParser defaultValue(int value) { - this.defaultValue = value; - return self(); - } - - public int parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Integer::parseInt, defaultValue); - } - - public Integer parseOptional() { - return parse(Integer::parseInt, null); - } - } - - class LongConfParser extends ConfParser { - private Long defaultValue; - - @Override - protected LongConfParser self() { - return this; - } - - public LongConfParser defaultValue(long value) { - this.defaultValue = value; - return self(); - } - - public long parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Long::parseLong, defaultValue); - } - - public Long parseOptional() { - return parse(Long::parseLong, null); - } - } - - class StringConfParser extends ConfParser { - private String defaultValue; - - @Override - protected StringConfParser self() { - return this; - } - - public StringConfParser defaultValue(String value) { - this.defaultValue = value; - return self(); - } - - public String parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(Function.identity(), defaultValue); - } - - public String parseOptional() { - return parse(Function.identity(), null); - } - } - - class EnumConfParser> extends ConfParser, E> { - private E defaultValue; - private final Class enumClass; - - EnumConfParser(Class enumClass) { - this.enumClass = enumClass; - } - - @Override - protected EnumConfParser self() { - return this; - } - - public EnumConfParser defaultValue(E value) { - this.defaultValue = value; - return self(); - } - - public E parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(s -> Enum.valueOf(enumClass, s), defaultValue); - } - - public E parseOptional() { - return parse(s -> Enum.valueOf(enumClass, s), null); - } - } - - class DurationConfParser extends ConfParser { - private Duration defaultValue; - - @Override - protected DurationConfParser self() { - return this; - } - - public DurationConfParser defaultValue(Duration value) { - this.defaultValue = value; - return self(); - } - - public Duration parse() { - Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); - return parse(TimeUtils::parseDuration, defaultValue); - } - - public Duration parseOptional() { - return parse(TimeUtils::parseDuration, null); - } - } - - abstract class ConfParser { - private final List optionNames = Lists.newArrayList(); - private String tablePropertyName; - private ConfigOption configOption; - - protected abstract ThisT self(); - - public ThisT option(String name) { - this.optionNames.add(name); - return self(); - } - - public ThisT flinkConfig(ConfigOption newConfigOption) { - this.configOption = newConfigOption; - return self(); - } - - public ThisT tableProperty(String name) { - this.tablePropertyName = name; - return self(); - } - - protected T parse(Function conversion, T defaultValue) { - if (!optionNames.isEmpty()) { - for (String optionName : optionNames) { - String optionValue = options.get(optionName); - if (optionValue != null) { - return conversion.apply(optionValue); - } - } - } - - if (configOption != null) { - T propertyValue = readableConfig.get(configOption); - if (propertyValue != null) { - return propertyValue; - } - } - - if (tablePropertyName != null) { - String propertyValue = tableProperties.get(tablePropertyName); - if (propertyValue != null) { - return conversion.apply(propertyValue); - } - } - - return defaultValue; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java deleted file mode 100644 index 7c7afd24ed8e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkConfigOptions.java +++ /dev/null @@ -1,107 +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; - -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.description.Description; -import org.apache.flink.configuration.description.TextElement; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.iceberg.flink.source.assigner.SplitAssignerType; -import org.apache.iceberg.util.ThreadPools; - -/** - * When constructing Flink Iceberg source via Java API, configs can be set in {@link Configuration} - * passed to source builder. E.g. - * - *

    - *   configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    - *   FlinkSource.forRowData()
    - *       .flinkConf(configuration)
    - *       ...
    - * 
    - * - *

    When using Flink SQL/table API, connector options can be set in Flink's {@link - * TableEnvironment}. - * - *

    - *   TableEnvironment tEnv = createTableEnv();
    - *   tEnv.getConfig()
    - *        .getConfiguration()
    - *        .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true);
    - * 
    - */ -public class FlinkConfigOptions { - - private FlinkConfigOptions() {} - - public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM = - ConfigOptions.key("table.exec.iceberg.infer-source-parallelism") - .booleanType() - .defaultValue(true) - .withDescription( - "If is false, parallelism of source are set by config.\n" - + "If is true, source parallelism is inferred according to splits number.\n"); - - public static final ConfigOption TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX = - ConfigOptions.key("table.exec.iceberg.infer-source-parallelism.max") - .intType() - .defaultValue(100) - .withDescription("Sets max infer parallelism for source operator."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO = - ConfigOptions.key("table.exec.iceberg.expose-split-locality-info") - .booleanType() - .noDefaultValue() - .withDescription( - "Expose split host information to use Flink's locality aware split assigner."); - - public static final ConfigOption SOURCE_READER_FETCH_BATCH_RECORD_COUNT = - ConfigOptions.key("table.exec.iceberg.fetch-batch-record-count") - .intType() - .defaultValue(2048) - .withDescription("The target number of records for Iceberg reader fetch batch."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE = - ConfigOptions.key("table.exec.iceberg.worker-pool-size") - .intType() - .defaultValue(ThreadPools.WORKER_THREAD_POOL_SIZE) - .withDescription("The size of workers pool used to plan or scan manifests."); - - public static final ConfigOption TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE = - ConfigOptions.key("table.exec.iceberg.use-flip27-source") - .booleanType() - .defaultValue(false) - .withDescription("Use the FLIP-27 based Iceberg source implementation."); - - public static final ConfigOption TABLE_EXEC_SPLIT_ASSIGNER_TYPE = - ConfigOptions.key("table.exec.iceberg.split-assigner-type") - .enumType(SplitAssignerType.class) - .defaultValue(SplitAssignerType.SIMPLE) - .withDescription( - Description.builder() - .text("Split assigner type that determine how splits are assigned to readers.") - .linebreak() - .list( - TextElement.text( - SplitAssignerType.SIMPLE - + ": simple assigner that doesn't provide any guarantee on order or locality.")) - .build()); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java deleted file mode 100644 index b7f1be4b93fb..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkDynamicTableFactory.java +++ /dev/null @@ -1,208 +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; - -import java.util.Map; -import java.util.Set; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.CatalogDatabaseImpl; -import org.apache.flink.table.catalog.ObjectIdentifier; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.ResolvedCatalogTable; -import org.apache.flink.table.catalog.exceptions.DatabaseAlreadyExistException; -import org.apache.flink.table.catalog.exceptions.TableAlreadyExistException; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.factories.DynamicTableSinkFactory; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.AlreadyExistsException; -import org.apache.iceberg.flink.source.IcebergTableSource; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; - -public class FlinkDynamicTableFactory - implements DynamicTableSinkFactory, DynamicTableSourceFactory { - static final String FACTORY_IDENTIFIER = "iceberg"; - - private static final ConfigOption CATALOG_NAME = - ConfigOptions.key("catalog-name") - .stringType() - .noDefaultValue() - .withDescription("Catalog name"); - - private static final ConfigOption CATALOG_TYPE = - ConfigOptions.key(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE) - .stringType() - .noDefaultValue() - .withDescription("Catalog type, the optional types are: custom, hadoop, hive."); - - private static final ConfigOption CATALOG_DATABASE = - ConfigOptions.key("catalog-database") - .stringType() - .defaultValue(FlinkCatalogFactory.DEFAULT_DATABASE_NAME) - .withDescription("Database name managed in the iceberg catalog."); - - private static final ConfigOption CATALOG_TABLE = - ConfigOptions.key("catalog-table") - .stringType() - .noDefaultValue() - .withDescription("Table name managed in the underlying iceberg catalog and database."); - - private final FlinkCatalog catalog; - - public FlinkDynamicTableFactory() { - this.catalog = null; - } - - public FlinkDynamicTableFactory(FlinkCatalog catalog) { - this.catalog = catalog; - } - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); - Map tableProps = resolvedCatalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); - - TableLoader tableLoader; - if (catalog != null) { - tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); - } else { - tableLoader = - createTableLoader( - resolvedCatalogTable, - tableProps, - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - } - - return new IcebergTableSource(tableLoader, tableSchema, tableProps, context.getConfiguration()); - } - - @Override - public DynamicTableSink createDynamicTableSink(Context context) { - ObjectIdentifier objectIdentifier = context.getObjectIdentifier(); - ResolvedCatalogTable resolvedCatalogTable = context.getCatalogTable(); - Map writeProps = resolvedCatalogTable.getOptions(); - TableSchema tableSchema = TableSchemaUtils.getPhysicalSchema(resolvedCatalogTable.getSchema()); - - TableLoader tableLoader; - if (catalog != null) { - tableLoader = createTableLoader(catalog, objectIdentifier.toObjectPath()); - } else { - tableLoader = - createTableLoader( - resolvedCatalogTable, - writeProps, - objectIdentifier.getDatabaseName(), - objectIdentifier.getObjectName()); - } - - return new IcebergTableSink(tableLoader, tableSchema, context.getConfiguration(), writeProps); - } - - @Override - public Set> requiredOptions() { - Set> options = Sets.newHashSet(); - options.add(CATALOG_TYPE); - options.add(CATALOG_NAME); - return options; - } - - @Override - public Set> optionalOptions() { - Set> options = Sets.newHashSet(); - options.add(CATALOG_DATABASE); - options.add(CATALOG_TABLE); - return options; - } - - @Override - public String factoryIdentifier() { - return FACTORY_IDENTIFIER; - } - - private static TableLoader createTableLoader( - ResolvedCatalogTable resolvedCatalogTable, - Map tableProps, - String databaseName, - String tableName) { - Configuration flinkConf = new Configuration(); - tableProps.forEach(flinkConf::setString); - - String catalogName = flinkConf.getString(CATALOG_NAME); - Preconditions.checkNotNull( - catalogName, "Table property '%s' cannot be null", CATALOG_NAME.key()); - - String catalogDatabase = flinkConf.getString(CATALOG_DATABASE, databaseName); - Preconditions.checkNotNull(catalogDatabase, "The iceberg database name cannot be null"); - - String catalogTable = flinkConf.getString(CATALOG_TABLE, tableName); - Preconditions.checkNotNull(catalogTable, "The iceberg table name cannot be null"); - - org.apache.hadoop.conf.Configuration hadoopConf = FlinkCatalogFactory.clusterHadoopConf(); - FlinkCatalogFactory factory = new FlinkCatalogFactory(); - FlinkCatalog flinkCatalog = - (FlinkCatalog) factory.createCatalog(catalogName, tableProps, hadoopConf); - ObjectPath objectPath = new ObjectPath(catalogDatabase, catalogTable); - - // Create database if not exists in the external catalog. - if (!flinkCatalog.databaseExists(catalogDatabase)) { - try { - flinkCatalog.createDatabase( - catalogDatabase, new CatalogDatabaseImpl(Maps.newHashMap(), null), true); - } catch (DatabaseAlreadyExistException e) { - throw new AlreadyExistsException( - e, - "Database %s already exists in the iceberg catalog %s.", - catalogName, - catalogDatabase); - } - } - - // Create table if not exists in the external catalog. - if (!flinkCatalog.tableExists(objectPath)) { - try { - flinkCatalog.createIcebergTable(objectPath, resolvedCatalogTable, true); - } catch (TableAlreadyExistException e) { - throw new AlreadyExistsException( - e, - "Table %s already exists in the database %s and catalog %s", - catalogTable, - catalogDatabase, - catalogName); - } - } - - return TableLoader.fromCatalog( - flinkCatalog.getCatalogLoader(), TableIdentifier.of(catalogDatabase, catalogTable)); - } - - private static TableLoader createTableLoader(FlinkCatalog catalog, ObjectPath objectPath) { - Preconditions.checkNotNull(catalog, "Flink catalog cannot be null"); - return TableLoader.fromCatalog(catalog.getCatalogLoader(), catalog.toIdentifier(objectPath)); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java deleted file mode 100644 index f35bb577fbba..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkEnvironmentContext.java +++ /dev/null @@ -1,31 +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; - -import org.apache.iceberg.EnvironmentContext; -import org.apache.iceberg.flink.util.FlinkPackage; - -class FlinkEnvironmentContext { - private FlinkEnvironmentContext() {} - - public static void init() { - EnvironmentContext.put(EnvironmentContext.ENGINE_NAME, "flink"); - EnvironmentContext.put(EnvironmentContext.ENGINE_VERSION, FlinkPackage.version()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java deleted file mode 100644 index f2244d5137a1..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFilters.java +++ /dev/null @@ -1,266 +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; - -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import java.util.function.BiFunction; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.FieldReferenceExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.flink.table.functions.FunctionDefinition; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expression.Operation; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.NaNUtil; - -public class FlinkFilters { - private FlinkFilters() {} - - private static final Pattern STARTS_WITH_PATTERN = Pattern.compile("([^%]+)%"); - - private static final Map FILTERS = - ImmutableMap.builder() - .put(BuiltInFunctionDefinitions.EQUALS, Operation.EQ) - .put(BuiltInFunctionDefinitions.NOT_EQUALS, Operation.NOT_EQ) - .put(BuiltInFunctionDefinitions.GREATER_THAN, Operation.GT) - .put(BuiltInFunctionDefinitions.GREATER_THAN_OR_EQUAL, Operation.GT_EQ) - .put(BuiltInFunctionDefinitions.LESS_THAN, Operation.LT) - .put(BuiltInFunctionDefinitions.LESS_THAN_OR_EQUAL, Operation.LT_EQ) - .put(BuiltInFunctionDefinitions.IS_NULL, Operation.IS_NULL) - .put(BuiltInFunctionDefinitions.IS_NOT_NULL, Operation.NOT_NULL) - .put(BuiltInFunctionDefinitions.AND, Operation.AND) - .put(BuiltInFunctionDefinitions.OR, Operation.OR) - .put(BuiltInFunctionDefinitions.NOT, Operation.NOT) - .put(BuiltInFunctionDefinitions.LIKE, Operation.STARTS_WITH) - .buildOrThrow(); - - /** - * Convert flink expression to iceberg expression. - * - *

    the BETWEEN, NOT_BETWEEN, IN expression will be converted by flink automatically. the - * BETWEEN will be converted to (GT_EQ AND LT_EQ), the NOT_BETWEEN will be converted to (LT_EQ OR - * GT_EQ), the IN will be converted to OR, so we do not add the conversion here - * - * @param flinkExpression the flink expression - * @return the iceberg expression - */ - public static Optional convert( - org.apache.flink.table.expressions.Expression flinkExpression) { - if (!(flinkExpression instanceof CallExpression)) { - return Optional.empty(); - } - - CallExpression call = (CallExpression) flinkExpression; - Operation op = FILTERS.get(call.getFunctionDefinition()); - if (op != null) { - switch (op) { - case IS_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(Expressions::isNull); - - case NOT_NULL: - return onlyChildAs(call, FieldReferenceExpression.class) - .map(FieldReferenceExpression::getName) - .map(Expressions::notNull); - - case LT: - return convertFieldAndLiteral(Expressions::lessThan, Expressions::greaterThan, call); - - case LT_EQ: - return convertFieldAndLiteral( - Expressions::lessThanOrEqual, Expressions::greaterThanOrEqual, call); - - case GT: - return convertFieldAndLiteral(Expressions::greaterThan, Expressions::lessThan, call); - - case GT_EQ: - return convertFieldAndLiteral( - Expressions::greaterThanOrEqual, Expressions::lessThanOrEqual, call); - - case EQ: - return convertFieldAndLiteral( - (ref, lit) -> { - if (NaNUtil.isNaN(lit)) { - return Expressions.isNaN(ref); - } else { - return Expressions.equal(ref, lit); - } - }, - call); - - case NOT_EQ: - return convertFieldAndLiteral( - (ref, lit) -> { - if (NaNUtil.isNaN(lit)) { - return Expressions.notNaN(ref); - } else { - return Expressions.notEqual(ref, lit); - } - }, - call); - - case NOT: - return onlyChildAs(call, CallExpression.class) - .flatMap(FlinkFilters::convert) - .map(Expressions::not); - - case AND: - return convertLogicExpression(Expressions::and, call); - - case OR: - return convertLogicExpression(Expressions::or, call); - - case STARTS_WITH: - return convertLike(call); - } - } - - return Optional.empty(); - } - - private static Optional onlyChildAs( - CallExpression call, Class expectedChildClass) { - List children = call.getResolvedChildren(); - if (children.size() != 1) { - return Optional.empty(); - } - - ResolvedExpression child = children.get(0); - if (!expectedChildClass.isInstance(child)) { - return Optional.empty(); - } - - return Optional.of(expectedChildClass.cast(child)); - } - - private static Optional convertLike(CallExpression call) { - List args = call.getResolvedChildren(); - if (args.size() != 2) { - return Optional.empty(); - } - - org.apache.flink.table.expressions.Expression left = args.get(0); - org.apache.flink.table.expressions.Expression right = args.get(1); - - if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - return convertLiteral((ValueLiteralExpression) right) - .flatMap( - lit -> { - if (lit instanceof String) { - String pattern = (String) lit; - Matcher matcher = STARTS_WITH_PATTERN.matcher(pattern); - // exclude special char of LIKE - // '_' is the wildcard of the SQL LIKE - if (!pattern.contains("_") && matcher.matches()) { - return Optional.of(Expressions.startsWith(name, matcher.group(1))); - } - } - - return Optional.empty(); - }); - } - - return Optional.empty(); - } - - private static Optional convertLogicExpression( - BiFunction function, CallExpression call) { - List args = call.getResolvedChildren(); - if (args == null || args.size() != 2) { - return Optional.empty(); - } - - Optional left = convert(args.get(0)); - Optional right = convert(args.get(1)); - if (left.isPresent() && right.isPresent()) { - return Optional.of(function.apply(left.get(), right.get())); - } - - return Optional.empty(); - } - - private static Optional convertLiteral(ValueLiteralExpression expression) { - Optional value = - expression.getValueAs( - expression.getOutputDataType().getLogicalType().getDefaultConversion()); - return value.map( - o -> { - if (o instanceof LocalDateTime) { - return DateTimeUtil.microsFromTimestamp((LocalDateTime) o); - } else if (o instanceof Instant) { - return DateTimeUtil.microsFromInstant((Instant) o); - } else if (o instanceof LocalTime) { - return DateTimeUtil.microsFromTime((LocalTime) o); - } else if (o instanceof LocalDate) { - return DateTimeUtil.daysFromDate((LocalDate) o); - } - - return o; - }); - } - - private static Optional convertFieldAndLiteral( - BiFunction expr, CallExpression call) { - return convertFieldAndLiteral(expr, expr, call); - } - - private static Optional convertFieldAndLiteral( - BiFunction convertLR, - BiFunction convertRL, - CallExpression call) { - List args = call.getResolvedChildren(); - if (args.size() != 2) { - return Optional.empty(); - } - - org.apache.flink.table.expressions.Expression left = args.get(0); - org.apache.flink.table.expressions.Expression right = args.get(1); - - if (left instanceof FieldReferenceExpression && right instanceof ValueLiteralExpression) { - String name = ((FieldReferenceExpression) left).getName(); - Optional lit = convertLiteral((ValueLiteralExpression) right); - if (lit.isPresent()) { - return Optional.of(convertLR.apply(name, lit.get())); - } - } else if (left instanceof ValueLiteralExpression - && right instanceof FieldReferenceExpression) { - Optional lit = convertLiteral((ValueLiteralExpression) left); - String name = ((FieldReferenceExpression) right).getName(); - if (lit.isPresent()) { - return Optional.of(convertRL.apply(name, lit.get())); - } - } - - return Optional.empty(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java deleted file mode 100644 index 767d4497ac91..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkFixupTypes.java +++ /dev/null @@ -1,50 +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; - -import org.apache.iceberg.Schema; -import org.apache.iceberg.types.FixupTypes; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -/** - * The uuid and fixed are converted to the same Flink type. Conversion back can produce only one, - * which may not be correct. - */ -class FlinkFixupTypes extends FixupTypes { - - private FlinkFixupTypes(Schema referenceSchema) { - super(referenceSchema); - } - - static Schema fixup(Schema schema, Schema referenceSchema) { - return new Schema( - TypeUtil.visit(schema, new FlinkFixupTypes(referenceSchema)).asStructType().fields()); - } - - @Override - protected boolean fixupPrimitive(Type.PrimitiveType type, Type source) { - if (type instanceof Types.FixedType) { - int length = ((Types.FixedType) type).length(); - return source.typeId() == Type.TypeID.UUID && length == 16; - } - return false; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java deleted file mode 100644 index 804a956ec9b9..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadConf.java +++ /dev/null @@ -1,213 +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; - -import java.time.Duration; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; - -public class FlinkReadConf { - - private final FlinkConfParser confParser; - - public FlinkReadConf( - Table table, Map readOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(table, readOptions, readableConfig); - } - - public Long snapshotId() { - return confParser.longConf().option(FlinkReadOptions.SNAPSHOT_ID.key()).parseOptional(); - } - - public String tag() { - return confParser.stringConf().option(FlinkReadOptions.TAG.key()).parseOptional(); - } - - public String startTag() { - return confParser.stringConf().option(FlinkReadOptions.START_TAG.key()).parseOptional(); - } - - public String endTag() { - return confParser.stringConf().option(FlinkReadOptions.END_TAG.key()).parseOptional(); - } - - public String branch() { - return confParser.stringConf().option(FlinkReadOptions.BRANCH.key()).parseOptional(); - } - - public boolean caseSensitive() { - return confParser - .booleanConf() - .option(FlinkReadOptions.CASE_SENSITIVE) - .flinkConfig(FlinkReadOptions.CASE_SENSITIVE_OPTION) - .defaultValue(FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue()) - .parse(); - } - - public Long asOfTimestamp() { - return confParser.longConf().option(FlinkReadOptions.AS_OF_TIMESTAMP.key()).parseOptional(); - } - - public StreamingStartingStrategy startingStrategy() { - return confParser - .enumConfParser(StreamingStartingStrategy.class) - .option(FlinkReadOptions.STARTING_STRATEGY) - .flinkConfig(FlinkReadOptions.STARTING_STRATEGY_OPTION) - .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .parse(); - } - - public Long startSnapshotTimestamp() { - return confParser - .longConf() - .option(FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key()) - .parseOptional(); - } - - public Long startSnapshotId() { - return confParser.longConf().option(FlinkReadOptions.START_SNAPSHOT_ID.key()).parseOptional(); - } - - public Long endSnapshotId() { - return confParser.longConf().option(FlinkReadOptions.END_SNAPSHOT_ID.key()).parseOptional(); - } - - public long splitSize() { - return confParser - .longConf() - .option(FlinkReadOptions.SPLIT_SIZE) - .flinkConfig(FlinkReadOptions.SPLIT_SIZE_OPTION) - .tableProperty(TableProperties.SPLIT_SIZE) - .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT) - .parse(); - } - - public int splitLookback() { - return confParser - .intConf() - .option(FlinkReadOptions.SPLIT_LOOKBACK) - .flinkConfig(FlinkReadOptions.SPLIT_LOOKBACK_OPTION) - .tableProperty(TableProperties.SPLIT_LOOKBACK) - .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT) - .parse(); - } - - public long splitFileOpenCost() { - return confParser - .longConf() - .option(FlinkReadOptions.SPLIT_FILE_OPEN_COST) - .flinkConfig(FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION) - .tableProperty(TableProperties.SPLIT_OPEN_FILE_COST) - .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT) - .parse(); - } - - public boolean streaming() { - return confParser - .booleanConf() - .option(FlinkReadOptions.STREAMING) - .flinkConfig(FlinkReadOptions.STREAMING_OPTION) - .defaultValue(FlinkReadOptions.STREAMING_OPTION.defaultValue()) - .parse(); - } - - public Duration monitorInterval() { - String duration = - confParser - .stringConf() - .option(FlinkReadOptions.MONITOR_INTERVAL) - .flinkConfig(FlinkReadOptions.MONITOR_INTERVAL_OPTION) - .defaultValue(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()) - .parse(); - - return TimeUtils.parseDuration(duration); - } - - public boolean includeColumnStats() { - return confParser - .booleanConf() - .option(FlinkReadOptions.INCLUDE_COLUMN_STATS) - .flinkConfig(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION) - .defaultValue(FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue()) - .parse(); - } - - public int maxPlanningSnapshotCount() { - return confParser - .intConf() - .option(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT) - .flinkConfig(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION) - .defaultValue(FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue()) - .parse(); - } - - public String nameMapping() { - return confParser.stringConf().option(TableProperties.DEFAULT_NAME_MAPPING).parseOptional(); - } - - public long limit() { - return confParser - .longConf() - .option(FlinkReadOptions.LIMIT) - .flinkConfig(FlinkReadOptions.LIMIT_OPTION) - .defaultValue(FlinkReadOptions.LIMIT_OPTION.defaultValue()) - .parse(); - } - - public int workerPoolSize() { - return confParser - .intConf() - .option(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key()) - .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) - .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) - .parse(); - } - - public int maxAllowedPlanningFailures() { - return confParser - .intConf() - .option(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES) - .flinkConfig(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION) - .defaultValue(FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue()) - .parse(); - } - - public String watermarkColumn() { - return confParser - .stringConf() - .option(FlinkReadOptions.WATERMARK_COLUMN) - .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_OPTION) - .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue()) - .parseOptional(); - } - - public TimeUnit watermarkColumnTimeUnit() { - return confParser - .enumConfParser(TimeUnit.class) - .option(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT) - .flinkConfig(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION) - .defaultValue(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue()) - .parse(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java deleted file mode 100644 index 1bbd88146c8f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkReadOptions.java +++ /dev/null @@ -1,123 +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; - -import java.util.concurrent.TimeUnit; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; - -/** Flink source read options */ -public class FlinkReadOptions { - private static final String PREFIX = "connector.iceberg."; - - private FlinkReadOptions() {} - - public static final ConfigOption SNAPSHOT_ID = - ConfigOptions.key("snapshot-id").longType().defaultValue(null); - - public static final ConfigOption TAG = - ConfigOptions.key("tag").stringType().defaultValue(null); - - public static final ConfigOption BRANCH = - ConfigOptions.key("branch").stringType().defaultValue(null); - - public static final ConfigOption START_TAG = - ConfigOptions.key("start-tag").stringType().defaultValue(null); - - public static final ConfigOption END_TAG = - ConfigOptions.key("end-tag").stringType().defaultValue(null); - - public static final String CASE_SENSITIVE = "case-sensitive"; - public static final ConfigOption CASE_SENSITIVE_OPTION = - ConfigOptions.key(PREFIX + CASE_SENSITIVE).booleanType().defaultValue(false); - - public static final ConfigOption AS_OF_TIMESTAMP = - ConfigOptions.key("as-of-timestamp").longType().defaultValue(null); - - public static final String STARTING_STRATEGY = "starting-strategy"; - public static final ConfigOption STARTING_STRATEGY_OPTION = - ConfigOptions.key(PREFIX + STARTING_STRATEGY) - .enumType(StreamingStartingStrategy.class) - .defaultValue(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT); - - public static final ConfigOption START_SNAPSHOT_TIMESTAMP = - ConfigOptions.key("start-snapshot-timestamp").longType().defaultValue(null); - - public static final ConfigOption START_SNAPSHOT_ID = - ConfigOptions.key("start-snapshot-id").longType().defaultValue(null); - - public static final ConfigOption END_SNAPSHOT_ID = - ConfigOptions.key("end-snapshot-id").longType().defaultValue(null); - - public static final String SPLIT_SIZE = "split-size"; - public static final ConfigOption SPLIT_SIZE_OPTION = - ConfigOptions.key(PREFIX + SPLIT_SIZE) - .longType() - .defaultValue(TableProperties.SPLIT_SIZE_DEFAULT); - - public static final String SPLIT_LOOKBACK = "split-lookback"; - public static final ConfigOption SPLIT_LOOKBACK_OPTION = - ConfigOptions.key(PREFIX + SPLIT_LOOKBACK) - .intType() - .defaultValue(TableProperties.SPLIT_LOOKBACK_DEFAULT); - - public static final String SPLIT_FILE_OPEN_COST = "split-file-open-cost"; - public static final ConfigOption SPLIT_FILE_OPEN_COST_OPTION = - ConfigOptions.key(PREFIX + SPLIT_FILE_OPEN_COST) - .longType() - .defaultValue(TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT); - - public static final String STREAMING = "streaming"; - public static final ConfigOption STREAMING_OPTION = - ConfigOptions.key(PREFIX + STREAMING).booleanType().defaultValue(false); - - public static final String MONITOR_INTERVAL = "monitor-interval"; - public static final ConfigOption MONITOR_INTERVAL_OPTION = - ConfigOptions.key(PREFIX + MONITOR_INTERVAL).stringType().defaultValue("60s"); - - public static final String INCLUDE_COLUMN_STATS = "include-column-stats"; - public static final ConfigOption INCLUDE_COLUMN_STATS_OPTION = - ConfigOptions.key(PREFIX + INCLUDE_COLUMN_STATS).booleanType().defaultValue(false); - - public static final String MAX_PLANNING_SNAPSHOT_COUNT = "max-planning-snapshot-count"; - public static final ConfigOption MAX_PLANNING_SNAPSHOT_COUNT_OPTION = - ConfigOptions.key(PREFIX + MAX_PLANNING_SNAPSHOT_COUNT) - .intType() - .defaultValue(Integer.MAX_VALUE); - - public static final String LIMIT = "limit"; - public static final ConfigOption LIMIT_OPTION = - ConfigOptions.key(PREFIX + LIMIT).longType().defaultValue(-1L); - - public static final String MAX_ALLOWED_PLANNING_FAILURES = "max-allowed-planning-failures"; - public static final ConfigOption MAX_ALLOWED_PLANNING_FAILURES_OPTION = - ConfigOptions.key(PREFIX + MAX_ALLOWED_PLANNING_FAILURES).intType().defaultValue(3); - - public static final String WATERMARK_COLUMN = "watermark-column"; - public static final ConfigOption WATERMARK_COLUMN_OPTION = - ConfigOptions.key(PREFIX + WATERMARK_COLUMN).stringType().noDefaultValue(); - - public static final String WATERMARK_COLUMN_TIME_UNIT = "watermark-column-time-unit"; - public static final ConfigOption WATERMARK_COLUMN_TIME_UNIT_OPTION = - ConfigOptions.key(PREFIX + WATERMARK_COLUMN_TIME_UNIT) - .enumType(TimeUnit.class) - .defaultValue(TimeUnit.MICROSECONDS); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java deleted file mode 100644 index 4790dc85bf28..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ /dev/null @@ -1,232 +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; - -import java.util.List; -import java.util.Set; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.ResolvedSchema; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.Schema; -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.Sets; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -/** - * Converter between Flink types and Iceberg type. The conversion is not a 1:1 mapping that not - * allows back-and-forth conversion. So some information might get lost during the back-and-forth - * conversion. - * - *

    This inconsistent types: - * - *

      - *
    • map Iceberg UUID type to Flink BinaryType(16) - *
    • map Flink VarCharType(_) and CharType(_) to Iceberg String type - *
    • map Flink VarBinaryType(_) to Iceberg Binary type - *
    • map Flink TimeType(_) to Iceberg Time type (microseconds) - *
    • map Flink TimestampType(_) to Iceberg Timestamp without zone type (microseconds) - *
    • map Flink LocalZonedTimestampType(_) to Iceberg Timestamp with zone type (microseconds) - *
    • map Flink MultiSetType to Iceberg Map type(element, int) - *
    - * - *

    - */ -public class FlinkSchemaUtil { - - private FlinkSchemaUtil() {} - - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ - @Deprecated - public static Schema convert(TableSchema schema) { - LogicalType schemaType = schema.toRowDataType().getLogicalType(); - Preconditions.checkArgument( - schemaType instanceof RowType, "Schema logical type should be row type."); - - RowType root = (RowType) schemaType; - Type converted = root.accept(new FlinkTypeToType(root)); - - Schema icebergSchema = new Schema(converted.asStructType().fields()); - if (schema.getPrimaryKey().isPresent()) { - return freshIdentifierFieldIds(icebergSchema, schema.getPrimaryKey().get().getColumns()); - } else { - return icebergSchema; - } - } - - /** Convert the flink table schema to apache iceberg schema with column comment. */ - public static Schema convert(ResolvedSchema flinkSchema) { - List tableColumns = flinkSchema.getColumns(); - // copy from org.apache.flink.table.api.Schema#toRowDataType - DataTypes.Field[] fields = - tableColumns.stream() - .map( - column -> { - if (column.getComment().isPresent()) { - return DataTypes.FIELD( - column.getName(), column.getDataType(), column.getComment().get()); - } else { - return DataTypes.FIELD(column.getName(), column.getDataType()); - } - }) - .toArray(DataTypes.Field[]::new); - - LogicalType schemaType = DataTypes.ROW(fields).notNull().getLogicalType(); - Preconditions.checkArgument( - schemaType instanceof RowType, "Schema logical type should be row type."); - - RowType root = (RowType) schemaType; - Type converted = root.accept(new FlinkTypeToType(root)); - Schema icebergSchema = new Schema(converted.asStructType().fields()); - if (flinkSchema.getPrimaryKey().isPresent()) { - return freshIdentifierFieldIds(icebergSchema, flinkSchema.getPrimaryKey().get().getColumns()); - } else { - return icebergSchema; - } - } - - private static Schema freshIdentifierFieldIds(Schema icebergSchema, List primaryKeys) { - // Locate the identifier field id list. - Set identifierFieldIds = Sets.newHashSet(); - for (String primaryKey : primaryKeys) { - Types.NestedField field = icebergSchema.findField(primaryKey); - Preconditions.checkNotNull( - field, - "Cannot find field ID for the primary key column %s in schema %s", - primaryKey, - icebergSchema); - identifierFieldIds.add(field.fieldId()); - } - return new Schema( - icebergSchema.schemaId(), icebergSchema.asStruct().fields(), identifierFieldIds); - } - - /** - * Convert a Flink {@link TableSchema} to a {@link Schema} based on the given schema. - * - *

    This conversion does not assign new ids; it uses ids from the base schema. - * - *

    Data types, field order, and nullability will match the Flink type. This conversion may - * return a schema that is not compatible with base schema. - * - * @param baseSchema a Schema on which conversion is based - * @param flinkSchema a Flink TableSchema - * @return the equivalent Schema - * @throws IllegalArgumentException if the type cannot be converted or there are missing ids - */ - public static Schema convert(Schema baseSchema, TableSchema flinkSchema) { - // convert to a type with fresh ids - Types.StructType struct = convert(flinkSchema).asStruct(); - // reassign ids to match the base schema - Schema schema = TypeUtil.reassignIds(new Schema(struct.fields()), baseSchema); - // reassign doc to match the base schema - schema = TypeUtil.reassignDoc(schema, baseSchema); - - // fix types that can't be represented in Flink (UUID) - Schema fixedSchema = FlinkFixupTypes.fixup(schema, baseSchema); - if (flinkSchema.getPrimaryKey().isPresent()) { - return freshIdentifierFieldIds(fixedSchema, flinkSchema.getPrimaryKey().get().getColumns()); - } else { - return fixedSchema; - } - } - - /** - * Convert a {@link Schema} to a {@link RowType Flink type}. - * - * @param schema a Schema - * @return the equivalent Flink type - * @throws IllegalArgumentException if the type cannot be converted to Flink - */ - public static RowType convert(Schema schema) { - return (RowType) TypeUtil.visit(schema, new TypeToFlinkType()); - } - - /** - * Convert a {@link Type} to a {@link LogicalType Flink type}. - * - * @param type a Type - * @return the equivalent Flink type - * @throws IllegalArgumentException if the type cannot be converted to Flink - */ - public static LogicalType convert(Type type) { - return TypeUtil.visit(type, new TypeToFlinkType()); - } - - /** - * Convert a {@link LogicalType Flink type} to a {@link Type}. - * - * @param flinkType a FlinkType - * @return the equivalent Iceberg type - */ - public static Type convert(LogicalType flinkType) { - return flinkType.accept(new FlinkTypeToType()); - } - - /** - * Convert a {@link RowType} to a {@link TableSchema}. - * - * @param rowType a RowType - * @return Flink TableSchema - */ - public static TableSchema toSchema(RowType rowType) { - TableSchema.Builder builder = TableSchema.builder(); - for (RowType.RowField field : rowType.getFields()) { - builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); - } - return builder.build(); - } - - /** - * Convert a {@link Schema} to a {@link TableSchema}. - * - * @param schema iceberg schema to convert. - * @return Flink TableSchema. - */ - public static TableSchema toSchema(Schema schema) { - TableSchema.Builder builder = TableSchema.builder(); - - // Add columns. - for (RowType.RowField field : convert(schema).getFields()) { - builder.field(field.getName(), TypeConversions.fromLogicalToDataType(field.getType())); - } - - // Add primary key. - Set identifierFieldIds = schema.identifierFieldIds(); - if (!identifierFieldIds.isEmpty()) { - List columns = Lists.newArrayListWithExpectedSize(identifierFieldIds.size()); - for (Integer identifierFieldId : identifierFieldIds) { - String columnName = schema.findColumnName(identifierFieldId); - Preconditions.checkNotNull( - columnName, "Cannot find field with id %s in schema %s", identifierFieldId, schema); - - columns.add(columnName); - } - builder.primaryKey(columns.toArray(new String[0])); - } - - return builder.build(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java deleted file mode 100644 index 5fbd84909d69..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkSourceFilter.java +++ /dev/null @@ -1,49 +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; - -import org.apache.flink.api.common.functions.FilterFunction; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Evaluator; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.types.Types; - -public class FlinkSourceFilter implements FilterFunction { - - private final RowType rowType; - private final Evaluator evaluator; - private final Types.StructType struct; - private volatile RowDataWrapper wrapper; - - public FlinkSourceFilter(Schema schema, Expression expr, boolean caseSensitive) { - this.rowType = FlinkSchemaUtil.convert(schema); - this.struct = schema.asStruct(); - this.evaluator = new Evaluator(struct, expr, caseSensitive); - } - - @Override - public boolean filter(RowData value) { - if (wrapper == null) { - this.wrapper = new RowDataWrapper(rowType, struct); - } - return evaluator.eval(wrapper.wrap(value)); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java deleted file mode 100644 index 408065f06057..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeToType.java +++ /dev/null @@ -1,203 +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; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.MultisetType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -class FlinkTypeToType extends FlinkTypeVisitor { - - private final RowType root; - private int nextId; - - FlinkTypeToType() { - this.root = null; - } - - FlinkTypeToType(RowType root) { - this.root = root; - // the root struct's fields use the first ids - this.nextId = root.getFieldCount(); - } - - private int getNextId() { - int next = nextId; - nextId += 1; - return next; - } - - @Override - public Type visit(CharType charType) { - return Types.StringType.get(); - } - - @Override - public Type visit(VarCharType varCharType) { - return Types.StringType.get(); - } - - @Override - public Type visit(BooleanType booleanType) { - return Types.BooleanType.get(); - } - - @Override - public Type visit(BinaryType binaryType) { - return Types.FixedType.ofLength(binaryType.getLength()); - } - - @Override - public Type visit(VarBinaryType varBinaryType) { - return Types.BinaryType.get(); - } - - @Override - public Type visit(DecimalType decimalType) { - return Types.DecimalType.of(decimalType.getPrecision(), decimalType.getScale()); - } - - @Override - public Type visit(TinyIntType tinyIntType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(SmallIntType smallIntType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(IntType intType) { - return Types.IntegerType.get(); - } - - @Override - public Type visit(BigIntType bigIntType) { - return Types.LongType.get(); - } - - @Override - public Type visit(FloatType floatType) { - return Types.FloatType.get(); - } - - @Override - public Type visit(DoubleType doubleType) { - return Types.DoubleType.get(); - } - - @Override - public Type visit(DateType dateType) { - return Types.DateType.get(); - } - - @Override - public Type visit(TimeType timeType) { - return Types.TimeType.get(); - } - - @Override - public Type visit(TimestampType timestampType) { - return Types.TimestampType.withoutZone(); - } - - @Override - public Type visit(LocalZonedTimestampType localZonedTimestampType) { - return Types.TimestampType.withZone(); - } - - @Override - public Type visit(ArrayType arrayType) { - Type elementType = arrayType.getElementType().accept(this); - if (arrayType.getElementType().isNullable()) { - return Types.ListType.ofOptional(getNextId(), elementType); - } else { - return Types.ListType.ofRequired(getNextId(), elementType); - } - } - - @Override - public Type visit(MultisetType multisetType) { - Type elementType = multisetType.getElementType().accept(this); - return Types.MapType.ofRequired(getNextId(), getNextId(), elementType, Types.IntegerType.get()); - } - - @Override - public Type visit(MapType mapType) { - // keys in map are not allowed to be null. - Type keyType = mapType.getKeyType().accept(this); - Type valueType = mapType.getValueType().accept(this); - if (mapType.getValueType().isNullable()) { - return Types.MapType.ofOptional(getNextId(), getNextId(), keyType, valueType); - } else { - return Types.MapType.ofRequired(getNextId(), getNextId(), keyType, valueType); - } - } - - @Override - @SuppressWarnings("ReferenceEquality") - public Type visit(RowType rowType) { - List newFields = Lists.newArrayListWithExpectedSize(rowType.getFieldCount()); - boolean isRoot = root == rowType; - - List types = - rowType.getFields().stream() - .map(f -> f.getType().accept(this)) - .collect(Collectors.toList()); - - for (int i = 0; i < rowType.getFieldCount(); i++) { - int id = isRoot ? i : getNextId(); - - RowType.RowField field = rowType.getFields().get(i); - String name = field.getName(); - String comment = field.getDescription().orElse(null); - - if (field.getType().isNullable()) { - newFields.add(Types.NestedField.optional(id, name, types.get(i), comment)); - } else { - newFields.add(Types.NestedField.required(id, name, types.get(i), comment)); - } - } - - return Types.StructType.of(newFields); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java deleted file mode 100644 index f3de2416088c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkTypeVisitor.java +++ /dev/null @@ -1,80 +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; - -import org.apache.flink.table.types.logical.DayTimeIntervalType; -import org.apache.flink.table.types.logical.DistinctType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeVisitor; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RawType; -import org.apache.flink.table.types.logical.StructuredType; -import org.apache.flink.table.types.logical.SymbolType; -import org.apache.flink.table.types.logical.YearMonthIntervalType; -import org.apache.flink.table.types.logical.ZonedTimestampType; - -public abstract class FlinkTypeVisitor implements LogicalTypeVisitor { - - // ------------------------- Unsupported types ------------------------------ - - @Override - public T visit(ZonedTimestampType zonedTimestampType) { - throw new UnsupportedOperationException("Unsupported ZonedTimestampType."); - } - - @Override - public T visit(YearMonthIntervalType yearMonthIntervalType) { - throw new UnsupportedOperationException("Unsupported YearMonthIntervalType."); - } - - @Override - public T visit(DayTimeIntervalType dayTimeIntervalType) { - throw new UnsupportedOperationException("Unsupported DayTimeIntervalType."); - } - - @Override - public T visit(DistinctType distinctType) { - throw new UnsupportedOperationException("Unsupported DistinctType."); - } - - @Override - public T visit(StructuredType structuredType) { - throw new UnsupportedOperationException("Unsupported StructuredType."); - } - - @Override - public T visit(NullType nullType) { - throw new UnsupportedOperationException("Unsupported NullType."); - } - - @Override - public T visit(RawType rawType) { - throw new UnsupportedOperationException("Unsupported RawType."); - } - - @Override - public T visit(SymbolType symbolType) { - throw new UnsupportedOperationException("Unsupported SymbolType."); - } - - @Override - public T visit(LogicalType other) { - throw new UnsupportedOperationException("Unsupported type: " + other); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java deleted file mode 100644 index ca7b1120bc81..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ /dev/null @@ -1,205 +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; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; - -/** - * A class for common Iceberg configs for Flink writes. - * - *

    If a config is set at multiple levels, the following order of precedence is used (top to - * bottom): - * - *

      - *
    1. Write options - *
    2. flink ReadableConfig - *
    3. Table metadata - *
    - * - * The most specific value is set in write options and takes precedence over all other configs. If - * no write option is provided, this class checks the flink configuration for any overrides. If no - * applicable value is found in the write options, this class uses the table metadata. - * - *

    Note this class is NOT meant to be serialized. - */ -public class FlinkWriteConf { - - private final FlinkConfParser confParser; - - public FlinkWriteConf( - Table table, Map writeOptions, ReadableConfig readableConfig) { - this.confParser = new FlinkConfParser(table, writeOptions, readableConfig); - } - - public boolean overwriteMode() { - return confParser - .booleanConf() - .option(FlinkWriteOptions.OVERWRITE_MODE.key()) - .flinkConfig(FlinkWriteOptions.OVERWRITE_MODE) - .defaultValue(FlinkWriteOptions.OVERWRITE_MODE.defaultValue()) - .parse(); - } - - public boolean upsertMode() { - return confParser - .booleanConf() - .option(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key()) - .flinkConfig(FlinkWriteOptions.WRITE_UPSERT_ENABLED) - .tableProperty(TableProperties.UPSERT_ENABLED) - .defaultValue(TableProperties.UPSERT_ENABLED_DEFAULT) - .parse(); - } - - public FileFormat dataFileFormat() { - String valueAsString = - confParser - .stringConf() - .option(FlinkWriteOptions.WRITE_FORMAT.key()) - .flinkConfig(FlinkWriteOptions.WRITE_FORMAT) - .tableProperty(TableProperties.DEFAULT_FILE_FORMAT) - .defaultValue(TableProperties.DEFAULT_FILE_FORMAT_DEFAULT) - .parse(); - return FileFormat.fromString(valueAsString); - } - - public long targetDataFileSize() { - return confParser - .longConf() - .option(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES.key()) - .flinkConfig(FlinkWriteOptions.TARGET_FILE_SIZE_BYTES) - .tableProperty(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES) - .defaultValue(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT) - .parse(); - } - - public String parquetCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.PARQUET_COMPRESSION) - .defaultValue(TableProperties.PARQUET_COMPRESSION_DEFAULT) - .parse(); - } - - public String parquetCompressionLevel() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) - .tableProperty(TableProperties.PARQUET_COMPRESSION_LEVEL) - .defaultValue(TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT) - .parseOptional(); - } - - public String avroCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.AVRO_COMPRESSION) - .defaultValue(TableProperties.AVRO_COMPRESSION_DEFAULT) - .parse(); - } - - public String avroCompressionLevel() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_LEVEL.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_LEVEL) - .tableProperty(TableProperties.AVRO_COMPRESSION_LEVEL) - .defaultValue(TableProperties.AVRO_COMPRESSION_LEVEL_DEFAULT) - .parseOptional(); - } - - public String orcCompressionCodec() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_CODEC.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_CODEC) - .tableProperty(TableProperties.ORC_COMPRESSION) - .defaultValue(TableProperties.ORC_COMPRESSION_DEFAULT) - .parse(); - } - - public String orcCompressionStrategy() { - return confParser - .stringConf() - .option(FlinkWriteOptions.COMPRESSION_STRATEGY.key()) - .flinkConfig(FlinkWriteOptions.COMPRESSION_STRATEGY) - .tableProperty(TableProperties.ORC_COMPRESSION_STRATEGY) - .defaultValue(TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT) - .parse(); - } - - public DistributionMode distributionMode() { - String modeName = - confParser - .stringConf() - .option(FlinkWriteOptions.DISTRIBUTION_MODE.key()) - .flinkConfig(FlinkWriteOptions.DISTRIBUTION_MODE) - .tableProperty(TableProperties.WRITE_DISTRIBUTION_MODE) - .defaultValue(TableProperties.WRITE_DISTRIBUTION_MODE_NONE) - .parse(); - return DistributionMode.fromName(modeName); - } - - public int workerPoolSize() { - return confParser - .intConf() - .flinkConfig(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE) - .defaultValue(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue()) - .parse(); - } - - public String branch() { - return confParser - .stringConf() - .option(FlinkWriteOptions.BRANCH.key()) - .defaultValue(FlinkWriteOptions.BRANCH.defaultValue()) - .parse(); - } - - public Integer writeParallelism() { - return confParser.intConf().option(FlinkWriteOptions.WRITE_PARALLELISM.key()).parseOptional(); - } - - /** - * NOTE: This may be removed or changed in a future release. This value specifies the interval for - * refreshing the table instances in sink writer subtasks. If not specified then the default - * behavior is to not refresh the table. - * - * @return the interval for refreshing the table in sink writer subtasks - */ - @Experimental - public Duration tableRefreshInterval() { - return confParser - .durationConf() - .option(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key()) - .flinkConfig(FlinkWriteOptions.TABLE_REFRESH_INTERVAL) - .parseOptional(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java deleted file mode 100644 index df73f2e09cac..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.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; - -import java.time.Duration; -import org.apache.flink.annotation.Experimental; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.iceberg.SnapshotRef; - -/** Flink sink write options */ -public class FlinkWriteOptions { - - private FlinkWriteOptions() {} - - // File format for write operations(default: Table write.format.default ) - public static final ConfigOption WRITE_FORMAT = - ConfigOptions.key("write-format").stringType().noDefaultValue(); - - // Overrides this table's write.target-file-size-bytes - public static final ConfigOption TARGET_FILE_SIZE_BYTES = - ConfigOptions.key("target-file-size-bytes").longType().noDefaultValue(); - - // Overrides this table's write..compression-codec - public static final ConfigOption COMPRESSION_CODEC = - ConfigOptions.key("compression-codec").stringType().noDefaultValue(); - - // Overrides this table's write..compression-level - public static final ConfigOption COMPRESSION_LEVEL = - ConfigOptions.key("compression-level").stringType().noDefaultValue(); - - // Overrides this table's write..compression-strategy - public static final ConfigOption COMPRESSION_STRATEGY = - ConfigOptions.key("compression-strategy").stringType().noDefaultValue(); - - // Overrides this table's write.upsert.enabled - public static final ConfigOption WRITE_UPSERT_ENABLED = - ConfigOptions.key("upsert-enabled").booleanType().noDefaultValue(); - - public static final ConfigOption OVERWRITE_MODE = - ConfigOptions.key("overwrite-enabled").booleanType().defaultValue(false); - - // Overrides the table's write.distribution-mode - public static final ConfigOption DISTRIBUTION_MODE = - ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); - - // Branch to write to - public static final ConfigOption BRANCH = - ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); - - public static final ConfigOption WRITE_PARALLELISM = - ConfigOptions.key("write-parallelism").intType().noDefaultValue(); - - @Experimental - public static final ConfigOption TABLE_REFRESH_INTERVAL = - ConfigOptions.key("table-refresh-interval").durationType().noDefaultValue(); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java deleted file mode 100644 index 1b9268569d9a..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/IcebergTableSink.java +++ /dev/null @@ -1,121 +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; - -import java.util.List; -import java.util.Map; -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.api.constraints.UniqueConstraint; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.sink.DataStreamSinkProvider; -import org.apache.flink.table.connector.sink.DynamicTableSink; -import org.apache.flink.table.connector.sink.abilities.SupportsOverwrite; -import org.apache.flink.table.connector.sink.abilities.SupportsPartitioning; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.flink.sink.FlinkSink; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; - -public class IcebergTableSink implements DynamicTableSink, SupportsPartitioning, SupportsOverwrite { - private final TableLoader tableLoader; - private final TableSchema tableSchema; - private final ReadableConfig readableConfig; - private final Map writeProps; - - private boolean overwrite = false; - - private IcebergTableSink(IcebergTableSink toCopy) { - this.tableLoader = toCopy.tableLoader; - this.tableSchema = toCopy.tableSchema; - this.overwrite = toCopy.overwrite; - this.readableConfig = toCopy.readableConfig; - this.writeProps = toCopy.writeProps; - } - - public IcebergTableSink( - TableLoader tableLoader, - TableSchema tableSchema, - ReadableConfig readableConfig, - Map writeProps) { - this.tableLoader = tableLoader; - this.tableSchema = tableSchema; - this.readableConfig = readableConfig; - this.writeProps = writeProps; - } - - @Override - public SinkRuntimeProvider getSinkRuntimeProvider(Context context) { - Preconditions.checkState( - !overwrite || context.isBounded(), - "Unbounded data stream doesn't support overwrite operation."); - - List equalityColumns = - tableSchema.getPrimaryKey().map(UniqueConstraint::getColumns).orElseGet(ImmutableList::of); - - return new DataStreamSinkProvider() { - @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(); - } - }; - } - - @Override - public void applyStaticPartition(Map partition) { - // The flink's PartitionFanoutWriter will handle the static partition write policy - // automatically. - } - - @Override - public ChangelogMode getChangelogMode(ChangelogMode requestedMode) { - ChangelogMode.Builder builder = ChangelogMode.newBuilder(); - for (RowKind kind : requestedMode.getContainedKinds()) { - builder.addContainedKind(kind); - } - return builder.build(); - } - - @Override - public DynamicTableSink copy() { - return new IcebergTableSink(this); - } - - @Override - public String asSummaryString() { - return "Iceberg table sink"; - } - - @Override - public void applyOverwrite(boolean newOverwrite) { - this.overwrite = newOverwrite; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java deleted file mode 100644 index d4cec7a3e80b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/RowDataWrapper.java +++ /dev/null @@ -1,142 +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; - -import java.lang.reflect.Array; -import java.nio.ByteBuffer; -import java.time.LocalDateTime; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.UUIDUtil; - -public class RowDataWrapper implements StructLike { - - private final LogicalType[] types; - private final PositionalGetter[] getters; - private RowData rowData = null; - - public RowDataWrapper(RowType rowType, Types.StructType struct) { - int size = rowType.getFieldCount(); - - types = (LogicalType[]) Array.newInstance(LogicalType.class, size); - getters = (PositionalGetter[]) Array.newInstance(PositionalGetter.class, size); - - for (int i = 0; i < size; i++) { - types[i] = rowType.getTypeAt(i); - getters[i] = buildGetter(types[i], struct.fields().get(i).type()); - } - } - - public RowDataWrapper wrap(RowData data) { - this.rowData = data; - return this; - } - - @Override - public int size() { - return types.length; - } - - @Override - public T get(int pos, Class javaClass) { - if (rowData.isNullAt(pos)) { - return null; - } else if (getters[pos] != null) { - return javaClass.cast(getters[pos].get(rowData, pos)); - } - - Object value = RowData.createFieldGetter(types[pos], pos).getFieldOrNull(rowData); - return javaClass.cast(value); - } - - @Override - public void set(int pos, T value) { - throw new UnsupportedOperationException( - "Could not set a field in the RowDataWrapper because rowData is read-only"); - } - - private interface PositionalGetter { - T get(RowData data, int pos); - } - - private static PositionalGetter buildGetter(LogicalType logicalType, Type type) { - switch (logicalType.getTypeRoot()) { - case TINYINT: - return (row, pos) -> (int) row.getByte(pos); - case SMALLINT: - return (row, pos) -> (int) row.getShort(pos); - case CHAR: - case VARCHAR: - return (row, pos) -> row.getString(pos).toString(); - - case BINARY: - case VARBINARY: - if (Type.TypeID.UUID == type.typeId()) { - return (row, pos) -> UUIDUtil.convert(row.getBinary(pos)); - } else { - return (row, pos) -> ByteBuffer.wrap(row.getBinary(pos)); - } - - case DECIMAL: - DecimalType decimalType = (DecimalType) logicalType; - return (row, pos) -> - row.getDecimal(pos, decimalType.getPrecision(), decimalType.getScale()).toBigDecimal(); - - case TIME_WITHOUT_TIME_ZONE: - // Time in RowData is in milliseconds (Integer), while iceberg's time is microseconds - // (Long). - return (row, pos) -> ((long) row.getInt(pos)) * 1_000; - - case TIMESTAMP_WITHOUT_TIME_ZONE: - TimestampType timestampType = (TimestampType) logicalType; - return (row, pos) -> { - LocalDateTime localDateTime = - row.getTimestamp(pos, timestampType.getPrecision()).toLocalDateTime(); - return DateTimeUtil.microsFromTimestamp(localDateTime); - }; - - case TIMESTAMP_WITH_LOCAL_TIME_ZONE: - LocalZonedTimestampType lzTs = (LocalZonedTimestampType) logicalType; - return (row, pos) -> { - TimestampData timestampData = row.getTimestamp(pos, lzTs.getPrecision()); - return timestampData.getMillisecond() * 1000 - + timestampData.getNanoOfMillisecond() / 1000; - }; - - case ROW: - RowType rowType = (RowType) logicalType; - Types.StructType structType = (Types.StructType) type; - - RowDataWrapper nestedWrapper = new RowDataWrapper(rowType, structType); - return (row, pos) -> nestedWrapper.wrap(row.getRow(pos, rowType.getFieldCount())); - - default: - return null; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java deleted file mode 100644 index da509451fee7..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TableLoader.java +++ /dev/null @@ -1,159 +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; - -import java.io.Closeable; -import java.io.IOException; -import java.io.Serializable; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.hadoop.SerializableConfiguration; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -/** - * Serializable loader to load an Iceberg {@link Table}. Flink needs to get {@link Table} objects in - * the cluster (for example, to get splits), not just on the client side. So we need an Iceberg - * table loader to get the {@link Table} object. - */ -public interface TableLoader extends Closeable, Serializable, Cloneable { - - void open(); - - boolean isOpen(); - - Table loadTable(); - - /** Clone a TableLoader */ - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - TableLoader clone(); - - static TableLoader fromCatalog(CatalogLoader catalogLoader, TableIdentifier identifier) { - return new CatalogTableLoader(catalogLoader, identifier); - } - - static TableLoader fromHadoopTable(String location) { - return fromHadoopTable(location, FlinkCatalogFactory.clusterHadoopConf()); - } - - static TableLoader fromHadoopTable(String location, Configuration hadoopConf) { - return new HadoopTableLoader(location, hadoopConf); - } - - class HadoopTableLoader implements TableLoader { - - private static final long serialVersionUID = 1L; - - private final String location; - private final SerializableConfiguration hadoopConf; - - private transient HadoopTables tables; - - private HadoopTableLoader(String location, Configuration conf) { - this.location = location; - this.hadoopConf = new SerializableConfiguration(conf); - } - - @Override - public void open() { - tables = new HadoopTables(hadoopConf.get()); - } - - @Override - public boolean isOpen() { - return tables != null; - } - - @Override - public Table loadTable() { - FlinkEnvironmentContext.init(); - return tables.load(location); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new HadoopTableLoader(location, new Configuration(hadoopConf.get())); - } - - @Override - public void close() {} - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("location", location).toString(); - } - } - - class CatalogTableLoader implements TableLoader { - - private static final long serialVersionUID = 1L; - - private final CatalogLoader catalogLoader; - private final String identifier; - - private transient Catalog catalog; - - private CatalogTableLoader(CatalogLoader catalogLoader, TableIdentifier tableIdentifier) { - this.catalogLoader = catalogLoader; - this.identifier = tableIdentifier.toString(); - } - - @Override - public void open() { - catalog = catalogLoader.loadCatalog(); - } - - @Override - public boolean isOpen() { - return catalog != null; - } - - @Override - public Table loadTable() { - FlinkEnvironmentContext.init(); - return catalog.loadTable(TableIdentifier.parse(identifier)); - } - - @Override - public void close() throws IOException { - if (catalog instanceof Closeable) { - ((Closeable) catalog).close(); - } - - catalog = null; - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new CatalogTableLoader(catalogLoader.clone(), TableIdentifier.parse(identifier)); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("tableIdentifier", identifier) - .add("catalogLoader", catalogLoader) - .toString(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java deleted file mode 100644 index f8f1b74b1ceb..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/TypeToFlinkType.java +++ /dev/null @@ -1,134 +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; - -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.BigIntType; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.BooleanType; -import org.apache.flink.table.types.logical.DateType; -import org.apache.flink.table.types.logical.DecimalType; -import org.apache.flink.table.types.logical.DoubleType; -import org.apache.flink.table.types.logical.FloatType; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -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.TypeUtil; -import org.apache.iceberg.types.Types; - -class TypeToFlinkType extends TypeUtil.SchemaVisitor { - TypeToFlinkType() {} - - @Override - public LogicalType schema(Schema schema, LogicalType structType) { - return structType; - } - - @Override - public LogicalType struct(Types.StructType struct, List fieldResults) { - List fields = struct.fields(); - - List flinkFields = Lists.newArrayListWithExpectedSize(fieldResults.size()); - for (int i = 0; i < fields.size(); i += 1) { - Types.NestedField field = fields.get(i); - LogicalType type = fieldResults.get(i); - RowType.RowField flinkField = - new RowType.RowField(field.name(), type.copy(field.isOptional()), field.doc()); - flinkFields.add(flinkField); - } - - return new RowType(flinkFields); - } - - @Override - public LogicalType field(Types.NestedField field, LogicalType fieldResult) { - return fieldResult; - } - - @Override - public LogicalType list(Types.ListType list, LogicalType elementResult) { - return new ArrayType(elementResult.copy(list.isElementOptional())); - } - - @Override - public LogicalType map(Types.MapType map, LogicalType keyResult, LogicalType valueResult) { - // keys in map are not allowed to be null. - return new MapType(keyResult.copy(false), valueResult.copy(map.isValueOptional())); - } - - @Override - public LogicalType primitive(Type.PrimitiveType primitive) { - switch (primitive.typeId()) { - case BOOLEAN: - return new BooleanType(); - case INTEGER: - return new IntType(); - case LONG: - return new BigIntType(); - case FLOAT: - return new FloatType(); - case DOUBLE: - return new DoubleType(); - case DATE: - return new DateType(); - case TIME: - // For the type: Flink only support TimeType with default precision (second) now. The - // precision of time is - // not supported in Flink, so we can think of it as a simple time type directly. - // For the data: Flink uses int that support mills to represent time data, so it supports - // mills precision. - return new TimeType(); - case TIMESTAMP: - Types.TimestampType timestamp = (Types.TimestampType) primitive; - if (timestamp.shouldAdjustToUTC()) { - // MICROS - return new LocalZonedTimestampType(6); - } else { - // MICROS - return new TimestampType(6); - } - case STRING: - return new VarCharType(VarCharType.MAX_LENGTH); - case UUID: - // UUID length is 16 - return new BinaryType(16); - case FIXED: - Types.FixedType fixedType = (Types.FixedType) primitive; - return new BinaryType(fixedType.length()); - case BINARY: - return new VarBinaryType(VarBinaryType.MAX_LENGTH); - case DECIMAL: - Types.DecimalType decimal = (Types.DecimalType) primitive; - return new DecimalType(decimal.precision(), decimal.scale()); - default: - throw new UnsupportedOperationException( - "Cannot convert unknown type to Flink: " + primitive); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java deleted file mode 100644 index b96b47c5a785..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/Actions.java +++ /dev/null @@ -1,52 +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.actions; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.Table; - -public class Actions { - - public static final Configuration CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private final StreamExecutionEnvironment env; - private final Table table; - - private Actions(StreamExecutionEnvironment env, Table table) { - this.env = env; - this.table = table; - } - - public static Actions forTable(StreamExecutionEnvironment env, Table table) { - return new Actions(env, table); - } - - public static Actions forTable(Table table) { - return new Actions(StreamExecutionEnvironment.getExecutionEnvironment(CONFIG), table); - } - - public RewriteDataFilesAction rewriteDataFiles() { - return new RewriteDataFilesAction(env, table); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java deleted file mode 100644 index 670abebcb58a..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/actions/RewriteDataFilesAction.java +++ /dev/null @@ -1,72 +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.actions; - -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.BaseRewriteDataFilesAction; -import org.apache.iceberg.flink.source.RowDataRewriter; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class RewriteDataFilesAction extends BaseRewriteDataFilesAction { - - private final StreamExecutionEnvironment env; - private int maxParallelism; - - public RewriteDataFilesAction(StreamExecutionEnvironment env, Table table) { - super(table); - this.env = env; - this.maxParallelism = env.getParallelism(); - } - - @Override - protected FileIO fileIO() { - return table().io(); - } - - @Override - protected List rewriteDataForTasks(List combinedScanTasks) { - int size = combinedScanTasks.size(); - int parallelism = Math.min(size, maxParallelism); - DataStream dataStream = env.fromCollection(combinedScanTasks); - RowDataRewriter rowDataRewriter = - new RowDataRewriter(table(), caseSensitive(), fileIO(), encryptionManager()); - try { - return rowDataRewriter.rewriteDataForTasks(dataStream, parallelism); - } catch (Exception e) { - throw new RuntimeException("Rewrite data file error.", e); - } - } - - @Override - protected RewriteDataFilesAction self() { - return this; - } - - public RewriteDataFilesAction maxParallelism(int parallelism) { - Preconditions.checkArgument(parallelism > 0, "Invalid max parallelism %s", parallelism); - this.maxParallelism = parallelism; - return this; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java deleted file mode 100644 index 8103224a0b6c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/AvroWithFlinkSchemaVisitor.java +++ /dev/null @@ -1,75 +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 org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.LogicalTypeFamily; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.NullType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.avro.AvroWithPartnerByStructureVisitor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.Pair; - -public abstract class AvroWithFlinkSchemaVisitor - extends AvroWithPartnerByStructureVisitor { - - @Override - protected boolean isStringType(LogicalType logicalType) { - return logicalType.getTypeRoot().getFamilies().contains(LogicalTypeFamily.CHARACTER_STRING); - } - - @Override - protected boolean isMapType(LogicalType logicalType) { - return logicalType instanceof MapType; - } - - @Override - protected LogicalType arrayElementType(LogicalType arrayType) { - Preconditions.checkArgument( - arrayType instanceof ArrayType, "Invalid array: %s is not an array", arrayType); - return ((ArrayType) arrayType).getElementType(); - } - - @Override - protected LogicalType mapKeyType(LogicalType mapType) { - Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); - return ((MapType) mapType).getKeyType(); - } - - @Override - protected LogicalType mapValueType(LogicalType mapType) { - Preconditions.checkArgument(isMapType(mapType), "Invalid map: %s is not a map", mapType); - return ((MapType) mapType).getValueType(); - } - - @Override - protected Pair fieldNameAndType(LogicalType structType, int pos) { - Preconditions.checkArgument( - structType instanceof RowType, "Invalid struct: %s is not a struct", structType); - RowType.RowField field = ((RowType) structType).getFields().get(pos); - return Pair.of(field.getName(), field.getType()); - } - - @Override - protected LogicalType nullType() { - return new NullType(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java deleted file mode 100644 index 86404959735a..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroReader.java +++ /dev/null @@ -1,169 +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; - -public class FlinkAvroReader implements DatumReader, SupportsRowPosition { - - private final Schema readSchema; - private final ValueReader reader; - private Schema fileSchema = null; - - public FlinkAvroReader(org.apache.iceberg.Schema expectedSchema, Schema readSchema) { - this(expectedSchema, readSchema, ImmutableMap.of()); - } - - @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.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java deleted file mode 100644 index 873e65783119..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkAvroWriter.java +++ /dev/null @@ -1,165 +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.stream.Collectors; -import java.util.stream.IntStream; -import java.util.stream.Stream; -import org.apache.avro.LogicalTypes; -import org.apache.avro.Schema; -import org.apache.avro.io.Encoder; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.avro.MetricsAwareDatumWriter; -import org.apache.iceberg.avro.ValueWriter; -import org.apache.iceberg.avro.ValueWriters; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class FlinkAvroWriter implements MetricsAwareDatumWriter { - private final RowType rowType; - private ValueWriter writer = null; - - public FlinkAvroWriter(RowType rowType) { - this.rowType = rowType; - } - - @Override - @SuppressWarnings("unchecked") - public void setSchema(Schema schema) { - this.writer = - (ValueWriter) - AvroWithFlinkSchemaVisitor.visit(rowType, schema, new WriteBuilder()); - } - - @Override - public void write(RowData datum, Encoder out) throws IOException { - writer.write(datum, out); - } - - @Override - public Stream metrics() { - return writer.metrics(); - } - - private static class WriteBuilder extends AvroWithFlinkSchemaVisitor> { - @Override - public ValueWriter record( - LogicalType struct, Schema record, List names, List> fields) { - return FlinkValueWriters.row( - fields, - IntStream.range(0, names.size()) - .mapToObj(i -> fieldNameAndType(struct, i).second()) - .collect(Collectors.toList())); - } - - @Override - public ValueWriter union(LogicalType type, Schema union, List> options) { - Preconditions.checkArgument( - options.contains(ValueWriters.nulls()), - "Cannot create writer for non-option union: %s", - union); - Preconditions.checkArgument( - options.size() == 2, "Cannot create writer for non-option union: %s", union); - if (union.getTypes().get(0).getType() == Schema.Type.NULL) { - return ValueWriters.option(0, options.get(1)); - } else { - return ValueWriters.option(1, options.get(0)); - } - } - - @Override - public ValueWriter array(LogicalType sArray, Schema array, ValueWriter elementWriter) { - return FlinkValueWriters.array(elementWriter, arrayElementType(sArray)); - } - - @Override - public ValueWriter map(LogicalType sMap, Schema map, ValueWriter valueReader) { - return FlinkValueWriters.map( - FlinkValueWriters.strings(), mapKeyType(sMap), valueReader, mapValueType(sMap)); - } - - @Override - public ValueWriter map( - LogicalType sMap, Schema map, ValueWriter keyWriter, ValueWriter valueWriter) { - return FlinkValueWriters.arrayMap( - keyWriter, mapKeyType(sMap), valueWriter, mapValueType(sMap)); - } - - @Override - public ValueWriter primitive(LogicalType type, Schema primitive) { - org.apache.avro.LogicalType logicalType = primitive.getLogicalType(); - if (logicalType != null) { - switch (logicalType.getName()) { - case "date": - return ValueWriters.ints(); - - case "time-micros": - return FlinkValueWriters.timeMicros(); - - case "timestamp-micros": - return FlinkValueWriters.timestampMicros(); - - case "decimal": - LogicalTypes.Decimal decimal = (LogicalTypes.Decimal) logicalType; - return FlinkValueWriters.decimal(decimal.getPrecision(), decimal.getScale()); - - case "uuid": - return ValueWriters.uuids(); - - default: - throw new IllegalArgumentException("Unsupported logical type: " + logicalType); - } - } - - switch (primitive.getType()) { - case NULL: - return ValueWriters.nulls(); - case BOOLEAN: - return ValueWriters.booleans(); - case INT: - switch (type.getTypeRoot()) { - case TINYINT: - return ValueWriters.tinyints(); - case SMALLINT: - return ValueWriters.shorts(); - default: - return ValueWriters.ints(); - } - case LONG: - return ValueWriters.longs(); - case FLOAT: - return ValueWriters.floats(); - case DOUBLE: - return ValueWriters.doubles(); - case STRING: - return FlinkValueWriters.strings(); - case FIXED: - return ValueWriters.fixed(primitive.getFixedSize()); - case BYTES: - return ValueWriters.bytes(); - default: - throw new IllegalArgumentException("Unsupported type: " + primitive); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java deleted file mode 100644 index 65b9d44ad4b8..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReader.java +++ /dev/null @@ -1,131 +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.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.orc.OrcRowReader; -import org.apache.iceberg.orc.OrcSchemaWithTypeVisitor; -import org.apache.iceberg.orc.OrcValueReader; -import org.apache.iceberg.orc.OrcValueReaders; -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.orc.TypeDescription; -import org.apache.orc.storage.ql.exec.vector.StructColumnVector; -import org.apache.orc.storage.ql.exec.vector.VectorizedRowBatch; - -public class FlinkOrcReader implements OrcRowReader { - private final OrcValueReader reader; - - public FlinkOrcReader(Schema iSchema, TypeDescription readSchema) { - this(iSchema, readSchema, ImmutableMap.of()); - } - - public FlinkOrcReader(Schema iSchema, TypeDescription readSchema, Map idToConstant) { - this.reader = - OrcSchemaWithTypeVisitor.visit(iSchema, readSchema, new ReadBuilder(idToConstant)); - } - - @Override - public RowData read(VectorizedRowBatch batch, int row) { - return (RowData) reader.read(new StructColumnVector(batch.size, batch.cols), row); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - reader.setBatchContext(batchOffsetInFile); - } - - private static class ReadBuilder extends OrcSchemaWithTypeVisitor> { - private final Map idToConstant; - - private ReadBuilder(Map idToConstant) { - this.idToConstant = idToConstant; - } - - @Override - public OrcValueReader record( - Types.StructType iStruct, - TypeDescription record, - List names, - List> fields) { - return FlinkOrcReaders.struct(fields, iStruct, idToConstant); - } - - @Override - public OrcValueReader list( - Types.ListType iList, TypeDescription array, OrcValueReader elementReader) { - return FlinkOrcReaders.array(elementReader); - } - - @Override - public OrcValueReader map( - Types.MapType iMap, - TypeDescription map, - OrcValueReader keyReader, - OrcValueReader valueReader) { - return FlinkOrcReaders.map(keyReader, valueReader); - } - - @Override - public OrcValueReader primitive(Type.PrimitiveType iPrimitive, TypeDescription primitive) { - switch (iPrimitive.typeId()) { - case BOOLEAN: - return OrcValueReaders.booleans(); - case INTEGER: - return OrcValueReaders.ints(); - case LONG: - return OrcValueReaders.longs(); - case FLOAT: - return OrcValueReaders.floats(); - case DOUBLE: - return OrcValueReaders.doubles(); - case DATE: - return FlinkOrcReaders.dates(); - case TIME: - return FlinkOrcReaders.times(); - case TIMESTAMP: - Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; - if (timestampType.shouldAdjustToUTC()) { - return FlinkOrcReaders.timestampTzs(); - } else { - return FlinkOrcReaders.timestamps(); - } - case STRING: - return FlinkOrcReaders.strings(); - case UUID: - case FIXED: - case BINARY: - return OrcValueReaders.bytes(); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; - return FlinkOrcReaders.decimals(decimalType.precision(), decimalType.scale()); - default: - throw new IllegalArgumentException( - String.format( - "Invalid iceberg type %s corresponding to ORC type %s", iPrimitive, primitive)); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java deleted file mode 100644 index 7a4a15c7e600..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcReaders.java +++ /dev/null @@ -1,283 +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.math.BigDecimal; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.orc.OrcValueReader; -import org.apache.iceberg.orc.OrcValueReaders; -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.apache.iceberg.types.Types; -import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; -import org.apache.orc.storage.ql.exec.vector.ColumnVector; -import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; -import org.apache.orc.storage.ql.exec.vector.ListColumnVector; -import org.apache.orc.storage.ql.exec.vector.LongColumnVector; -import org.apache.orc.storage.ql.exec.vector.MapColumnVector; -import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; -import org.apache.orc.storage.serde2.io.HiveDecimalWritable; - -class FlinkOrcReaders { - private FlinkOrcReaders() {} - - static OrcValueReader strings() { - return StringReader.INSTANCE; - } - - static OrcValueReader dates() { - return DateReader.INSTANCE; - } - - static OrcValueReader decimals(int precision, int scale) { - if (precision <= 18) { - return new Decimal18Reader(precision, scale); - } else if (precision <= 38) { - return new Decimal38Reader(precision, scale); - } else { - throw new IllegalArgumentException("Invalid precision: " + precision); - } - } - - static OrcValueReader times() { - return TimeReader.INSTANCE; - } - - static OrcValueReader timestamps() { - return TimestampReader.INSTANCE; - } - - static OrcValueReader timestampTzs() { - return TimestampTzReader.INSTANCE; - } - - static OrcValueReader array(OrcValueReader elementReader) { - return new ArrayReader<>(elementReader); - } - - public static OrcValueReader map( - OrcValueReader keyReader, OrcValueReader valueReader) { - return new MapReader<>(keyReader, valueReader); - } - - public static OrcValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); - } - - private static class StringReader implements OrcValueReader { - private static final StringReader INSTANCE = new StringReader(); - - @Override - public StringData nonNullRead(ColumnVector vector, int row) { - BytesColumnVector bytesVector = (BytesColumnVector) vector; - return StringData.fromBytes( - bytesVector.vector[row], bytesVector.start[row], bytesVector.length[row]); - } - } - - private static class DateReader implements OrcValueReader { - private static final DateReader INSTANCE = new DateReader(); - - @Override - public Integer nonNullRead(ColumnVector vector, int row) { - return (int) ((LongColumnVector) vector).vector[row]; - } - } - - private static class Decimal18Reader implements OrcValueReader { - private final int precision; - private final int scale; - - Decimal18Reader(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData nonNullRead(ColumnVector vector, int row) { - HiveDecimalWritable value = ((DecimalColumnVector) vector).vector[row]; - - // The hive ORC writer may will adjust the scale of decimal data. - Preconditions.checkArgument( - value.precision() <= precision, - "Cannot read value as decimal(%s,%s), too large: %s", - precision, - scale, - value); - - return DecimalData.fromUnscaledLong(value.serialize64(scale), precision, scale); - } - } - - private static class Decimal38Reader implements OrcValueReader { - private final int precision; - private final int scale; - - Decimal38Reader(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData nonNullRead(ColumnVector vector, int row) { - BigDecimal value = - ((DecimalColumnVector) vector).vector[row].getHiveDecimal().bigDecimalValue(); - - Preconditions.checkArgument( - value.precision() <= precision, - "Cannot read value as decimal(%s,%s), too large: %s", - precision, - scale, - value); - - return DecimalData.fromBigDecimal(value, precision, scale); - } - } - - private static class TimeReader implements OrcValueReader { - private static final TimeReader INSTANCE = new TimeReader(); - - @Override - public Integer nonNullRead(ColumnVector vector, int row) { - long micros = ((LongColumnVector) vector).vector[row]; - // Flink only support time mills, just erase micros. - return (int) (micros / 1000); - } - } - - private static class TimestampReader implements OrcValueReader { - private static final TimestampReader INSTANCE = new TimestampReader(); - - @Override - public TimestampData nonNullRead(ColumnVector vector, int row) { - TimestampColumnVector tcv = (TimestampColumnVector) vector; - LocalDateTime localDate = - Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) - .atOffset(ZoneOffset.UTC) - .toLocalDateTime(); - return TimestampData.fromLocalDateTime(localDate); - } - } - - private static class TimestampTzReader implements OrcValueReader { - private static final TimestampTzReader INSTANCE = new TimestampTzReader(); - - @Override - public TimestampData nonNullRead(ColumnVector vector, int row) { - TimestampColumnVector tcv = (TimestampColumnVector) vector; - Instant instant = - Instant.ofEpochSecond(Math.floorDiv(tcv.time[row], 1_000), tcv.nanos[row]) - .atOffset(ZoneOffset.UTC) - .toInstant(); - return TimestampData.fromInstant(instant); - } - } - - private static class ArrayReader implements OrcValueReader { - private final OrcValueReader elementReader; - - private ArrayReader(OrcValueReader elementReader) { - this.elementReader = elementReader; - } - - @Override - public ArrayData nonNullRead(ColumnVector vector, int row) { - ListColumnVector listVector = (ListColumnVector) vector; - int offset = (int) listVector.offsets[row]; - int length = (int) listVector.lengths[row]; - List elements = Lists.newArrayListWithExpectedSize(length); - for (int c = 0; c < length; ++c) { - elements.add(elementReader.read(listVector.child, offset + c)); - } - return new GenericArrayData(elements.toArray()); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - elementReader.setBatchContext(batchOffsetInFile); - } - } - - private static class MapReader implements OrcValueReader { - private final OrcValueReader keyReader; - private final OrcValueReader valueReader; - - private MapReader(OrcValueReader keyReader, OrcValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData nonNullRead(ColumnVector vector, int row) { - MapColumnVector mapVector = (MapColumnVector) vector; - int offset = (int) mapVector.offsets[row]; - long length = mapVector.lengths[row]; - - Map map = Maps.newHashMap(); - for (int c = 0; c < length; c++) { - K key = keyReader.read(mapVector.keys, offset + c); - V value = valueReader.read(mapVector.values, offset + c); - map.put(key, value); - } - - return new GenericMapData(map); - } - - @Override - public void setBatchContext(long batchOffsetInFile) { - keyReader.setBatchContext(batchOffsetInFile); - valueReader.setBatchContext(batchOffsetInFile); - } - } - - private static class StructReader extends OrcValueReaders.StructReader { - private final int numFields; - - StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); - this.numFields = struct.fields().size(); - } - - @Override - protected RowData create() { - return new GenericRowData(numFields); - } - - @Override - protected void set(RowData struct, int pos, Object value) { - ((GenericRowData) struct).setField(pos, value); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java deleted file mode 100644 index 6a31accffd22..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriter.java +++ /dev/null @@ -1,163 +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.util.Deque; -import java.util.List; -import java.util.stream.Stream; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.orc.GenericOrcWriters; -import org.apache.iceberg.orc.OrcRowWriter; -import org.apache.iceberg.orc.OrcValueWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -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.orc.storage.ql.exec.vector.VectorizedRowBatch; - -public class FlinkOrcWriter implements OrcRowWriter { - private final FlinkOrcWriters.RowDataWriter writer; - - private FlinkOrcWriter(RowType rowType, Schema iSchema) { - this.writer = - (FlinkOrcWriters.RowDataWriter) - FlinkSchemaVisitor.visit(rowType, iSchema, new WriteBuilder()); - } - - public static OrcRowWriter buildWriter(RowType rowType, Schema iSchema) { - return new FlinkOrcWriter(rowType, iSchema); - } - - @Override - public void write(RowData row, VectorizedRowBatch output) { - Preconditions.checkArgument(row != null, "value must not be null"); - writer.writeRow(row, output); - } - - @Override - public List> writers() { - return writer.writers(); - } - - @Override - public Stream> metrics() { - return writer.metrics(); - } - - private static class WriteBuilder extends FlinkSchemaVisitor> { - private final Deque fieldIds = Lists.newLinkedList(); - - private WriteBuilder() {} - - @Override - public void beforeField(Types.NestedField field) { - fieldIds.push(field.fieldId()); - } - - @Override - public void afterField(Types.NestedField field) { - fieldIds.pop(); - } - - @Override - public OrcValueWriter record( - Types.StructType iStruct, List> results, List fieldType) { - return FlinkOrcWriters.struct(results, fieldType); - } - - @Override - public OrcValueWriter map( - Types.MapType iMap, - OrcValueWriter key, - OrcValueWriter value, - LogicalType keyType, - LogicalType valueType) { - return FlinkOrcWriters.map(key, value, keyType, valueType); - } - - @Override - public OrcValueWriter list( - Types.ListType iList, OrcValueWriter element, LogicalType elementType) { - return FlinkOrcWriters.list(element, elementType); - } - - @Override - public OrcValueWriter primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { - switch (iPrimitive.typeId()) { - case BOOLEAN: - return GenericOrcWriters.booleans(); - case INTEGER: - switch (flinkPrimitive.getTypeRoot()) { - case TINYINT: - return GenericOrcWriters.bytes(); - case SMALLINT: - return GenericOrcWriters.shorts(); - } - return GenericOrcWriters.ints(); - case LONG: - return GenericOrcWriters.longs(); - case FLOAT: - Preconditions.checkArgument( - fieldIds.peek() != null, - String.format( - "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " - + "information is not properly pushed during schema visiting.", - iPrimitive)); - return GenericOrcWriters.floats(fieldIds.peek()); - case DOUBLE: - Preconditions.checkArgument( - fieldIds.peek() != null, - String.format( - "[BUG] Cannot find field id for primitive field with type %s. This is likely because id " - + "information is not properly pushed during schema visiting.", - iPrimitive)); - return GenericOrcWriters.doubles(fieldIds.peek()); - case DATE: - return FlinkOrcWriters.dates(); - case TIME: - return FlinkOrcWriters.times(); - case TIMESTAMP: - Types.TimestampType timestampType = (Types.TimestampType) iPrimitive; - if (timestampType.shouldAdjustToUTC()) { - return FlinkOrcWriters.timestampTzs(); - } else { - return FlinkOrcWriters.timestamps(); - } - case STRING: - return FlinkOrcWriters.strings(); - case UUID: - case FIXED: - case BINARY: - return GenericOrcWriters.byteArrays(); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) iPrimitive; - return FlinkOrcWriters.decimals(decimalType.precision(), decimalType.scale()); - default: - throw new IllegalArgumentException( - String.format( - "Invalid iceberg type %s corresponding to Flink logical type %s", - iPrimitive, flinkPrimitive)); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java deleted file mode 100644 index da2f95cf822f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkOrcWriters.java +++ /dev/null @@ -1,317 +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.time.Instant; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.stream.Stream; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.FieldMetrics; -import org.apache.iceberg.data.orc.GenericOrcWriters; -import org.apache.iceberg.orc.OrcValueWriter; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.orc.storage.common.type.HiveDecimal; -import org.apache.orc.storage.ql.exec.vector.BytesColumnVector; -import org.apache.orc.storage.ql.exec.vector.ColumnVector; -import org.apache.orc.storage.ql.exec.vector.DecimalColumnVector; -import org.apache.orc.storage.ql.exec.vector.ListColumnVector; -import org.apache.orc.storage.ql.exec.vector.LongColumnVector; -import org.apache.orc.storage.ql.exec.vector.MapColumnVector; -import org.apache.orc.storage.ql.exec.vector.TimestampColumnVector; - -class FlinkOrcWriters { - - private FlinkOrcWriters() {} - - static OrcValueWriter strings() { - return StringWriter.INSTANCE; - } - - static OrcValueWriter dates() { - return DateWriter.INSTANCE; - } - - static OrcValueWriter times() { - return TimeWriter.INSTANCE; - } - - static OrcValueWriter timestamps() { - return TimestampWriter.INSTANCE; - } - - static OrcValueWriter timestampTzs() { - return TimestampTzWriter.INSTANCE; - } - - static OrcValueWriter decimals(int precision, int scale) { - if (precision <= 18) { - return new Decimal18Writer(precision, scale); - } else if (precision <= 38) { - return new Decimal38Writer(precision, scale); - } else { - throw new IllegalArgumentException("Invalid precision: " + precision); - } - } - - static OrcValueWriter list( - OrcValueWriter elementWriter, LogicalType elementType) { - return new ListWriter<>(elementWriter, elementType); - } - - static OrcValueWriter map( - OrcValueWriter keyWriter, - OrcValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - return new MapWriter<>(keyWriter, valueWriter, keyType, valueType); - } - - static OrcValueWriter struct(List> writers, List types) { - return new RowDataWriter(writers, types); - } - - private static class StringWriter implements OrcValueWriter { - private static final StringWriter INSTANCE = new StringWriter(); - - @Override - public void nonNullWrite(int rowId, StringData data, ColumnVector output) { - byte[] value = data.toBytes(); - ((BytesColumnVector) output).setRef(rowId, value, 0, value.length); - } - } - - private static class DateWriter implements OrcValueWriter { - private static final DateWriter INSTANCE = new DateWriter(); - - @Override - public void nonNullWrite(int rowId, Integer data, ColumnVector output) { - ((LongColumnVector) output).vector[rowId] = data; - } - } - - private static class TimeWriter implements OrcValueWriter { - private static final TimeWriter INSTANCE = new TimeWriter(); - - @Override - public void nonNullWrite(int rowId, Integer millis, ColumnVector output) { - // The time in flink is in millisecond, while the standard time in iceberg is microsecond. - // So we need to transform it to microsecond. - ((LongColumnVector) output).vector[rowId] = millis * 1000L; - } - } - - private static class TimestampWriter implements OrcValueWriter { - private static final TimestampWriter INSTANCE = new TimestampWriter(); - - @Override - public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { - TimestampColumnVector cv = (TimestampColumnVector) output; - cv.setIsUTC(true); - // millis - OffsetDateTime offsetDateTime = data.toInstant().atOffset(ZoneOffset.UTC); - cv.time[rowId] = - offsetDateTime.toEpochSecond() * 1_000 + offsetDateTime.getNano() / 1_000_000; - // truncate nanos to only keep microsecond precision. - cv.nanos[rowId] = (offsetDateTime.getNano() / 1_000) * 1_000; - } - } - - private static class TimestampTzWriter implements OrcValueWriter { - private static final TimestampTzWriter INSTANCE = new TimestampTzWriter(); - - @SuppressWarnings("JavaInstantGetSecondsGetNano") - @Override - public void nonNullWrite(int rowId, TimestampData data, ColumnVector output) { - TimestampColumnVector cv = (TimestampColumnVector) output; - // millis - Instant instant = data.toInstant(); - cv.time[rowId] = instant.toEpochMilli(); - // truncate nanos to only keep microsecond precision. - cv.nanos[rowId] = (instant.getNano() / 1_000) * 1_000; - } - } - - private static class Decimal18Writer implements OrcValueWriter { - private final int precision; - private final int scale; - - Decimal18Writer(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { - Preconditions.checkArgument( - scale == data.scale(), - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - data); - Preconditions.checkArgument( - data.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - data); - - ((DecimalColumnVector) output) - .vector[rowId].setFromLongAndScale(data.toUnscaledLong(), data.scale()); - } - } - - private static class Decimal38Writer implements OrcValueWriter { - private final int precision; - private final int scale; - - Decimal38Writer(int precision, int scale) { - this.precision = precision; - this.scale = scale; - } - - @Override - public void nonNullWrite(int rowId, DecimalData data, ColumnVector output) { - Preconditions.checkArgument( - scale == data.scale(), - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - data); - Preconditions.checkArgument( - data.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - data); - - ((DecimalColumnVector) output) - .vector[rowId].set(HiveDecimal.create(data.toBigDecimal(), false)); - } - } - - static class ListWriter implements OrcValueWriter { - private final OrcValueWriter elementWriter; - private final ArrayData.ElementGetter elementGetter; - - ListWriter(OrcValueWriter elementWriter, LogicalType elementType) { - this.elementWriter = elementWriter; - this.elementGetter = ArrayData.createElementGetter(elementType); - } - - @Override - @SuppressWarnings("unchecked") - public void nonNullWrite(int rowId, ArrayData data, ColumnVector output) { - ListColumnVector cv = (ListColumnVector) output; - cv.lengths[rowId] = data.size(); - cv.offsets[rowId] = cv.childCount; - cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); - // make sure the child is big enough. - growColumnVector(cv.child, cv.childCount); - - for (int e = 0; e < cv.lengths[rowId]; ++e) { - Object value = elementGetter.getElementOrNull(data, e); - elementWriter.write((int) (e + cv.offsets[rowId]), (T) value, cv.child); - } - } - - @Override - public Stream> metrics() { - return elementWriter.metrics(); - } - } - - static class MapWriter implements OrcValueWriter { - private final OrcValueWriter keyWriter; - private final OrcValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - MapWriter( - OrcValueWriter keyWriter, - OrcValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.valueWriter = valueWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void nonNullWrite(int rowId, MapData data, ColumnVector output) { - MapColumnVector cv = (MapColumnVector) output; - ArrayData keyArray = data.keyArray(); - ArrayData valArray = data.valueArray(); - - // record the length and start of the list elements - cv.lengths[rowId] = data.size(); - cv.offsets[rowId] = cv.childCount; - cv.childCount = (int) (cv.childCount + cv.lengths[rowId]); - // make sure the child is big enough - growColumnVector(cv.keys, cv.childCount); - growColumnVector(cv.values, cv.childCount); - // Add each element - for (int e = 0; e < cv.lengths[rowId]; ++e) { - int pos = (int) (e + cv.offsets[rowId]); - keyWriter.write(pos, (K) keyGetter.getElementOrNull(keyArray, e), cv.keys); - valueWriter.write(pos, (V) valueGetter.getElementOrNull(valArray, e), cv.values); - } - } - - @Override - public Stream> metrics() { - return Stream.concat(keyWriter.metrics(), valueWriter.metrics()); - } - } - - static class RowDataWriter extends GenericOrcWriters.StructWriter { - private final List fieldGetters; - - RowDataWriter(List> writers, List types) { - super(writers); - - this.fieldGetters = Lists.newArrayListWithExpectedSize(types.size()); - for (int i = 0; i < types.size(); i++) { - fieldGetters.add(RowData.createFieldGetter(types.get(i), i)); - } - } - - @Override - protected Object get(RowData struct, int index) { - return fieldGetters.get(index).getFieldOrNull(struct); - } - } - - private static void growColumnVector(ColumnVector cv, int requestedSize) { - if (cv.isNull.length < requestedSize) { - // Use growth factor of 3 to avoid frequent array allocations - cv.ensureSize(requestedSize * 3, true); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java deleted file mode 100644 index a5f2bb738960..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetReaders.java +++ /dev/null @@ -1,905 +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.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.MetadataColumns; -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.TypeWithSchemaVisitor; -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.types.Types; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.LogicalTypeAnnotation; -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; - -public class FlinkParquetReaders { - private FlinkParquetReaders() {} - - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema) { - return buildReader(expectedSchema, fileSchema, ImmutableMap.of()); - } - - @SuppressWarnings("unchecked") - public static ParquetValueReader buildReader( - Schema expectedSchema, MessageType fileSchema, Map idToConstant) { - return (ParquetValueReader) - TypeWithSchemaVisitor.visit( - expectedSchema.asStruct(), fileSchema, new ReadBuilder(fileSchema, idToConstant)); - } - - private static class ReadBuilder extends TypeWithSchemaVisitor> { - private final MessageType type; - private final Map idToConstant; - - ReadBuilder(MessageType type, Map idToConstant) { - this.type = type; - this.idToConstant = idToConstant; - } - - @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); - if (fieldReaders.get(i) != null) { - int fieldD = type.getMaxDefinitionLevel(path(fieldType.getName())) - 1; - if (fieldType.getId() != null) { - int id = fieldType.getId().intValue(); - readersById.put(id, ParquetValueReaders.option(fieldType, fieldD, fieldReaders.get(i))); - typesById.put(id, fieldType); - if (idToConstant.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 (idToConstant.containsKey(id)) { - // containsKey is used because the constant may be null - int fieldMaxDefinitionLevel = - maxDefinitionLevelsById.getOrDefault(id, defaultMaxDefinitionLevel); - reorderedFields.add( - ParquetValueReaders.constant(idToConstant.get(id), fieldMaxDefinitionLevel)); - types.add(null); - } else if (id == MetadataColumns.ROW_POSITION.fieldId()) { - reorderedFields.add(ParquetValueReaders.position()); - types.add(null); - } else if (id == MetadataColumns.IS_DELETED.fieldId()) { - reorderedFields.add(ParquetValueReaders.constant(false)); - 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 RowDataReader(types, reorderedFields); - } - - @Override - public ParquetValueReader list( - Types.ListType expectedList, GroupType array, ParquetValueReader elementReader) { - if (expectedList == null) { - return null; - } - - 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) { - if (expectedMap == null) { - return null; - } - - 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)); - } - - private static class LogicalTypeAnnotationParquetValueReaderVisitor - implements LogicalTypeAnnotation.LogicalTypeAnnotationVisitor> { - - private final PrimitiveType primitive; - private final ColumnDescriptor desc; - private final org.apache.iceberg.types.Type.PrimitiveType expected; - - LogicalTypeAnnotationParquetValueReaderVisitor( - PrimitiveType primitive, - ColumnDescriptor desc, - org.apache.iceberg.types.Type.PrimitiveType expected) { - this.primitive = primitive; - this.desc = desc; - this.expected = expected; - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.StringLogicalTypeAnnotation stringLogicalType) { - return Optional.of(new StringReader(desc)); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.EnumLogicalTypeAnnotation enumLogicalType) { - return Optional.of(new StringReader(desc)); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.JsonLogicalTypeAnnotation jsonLogicalType) { - return Optional.of(new StringReader(desc)); - } - - @Override - public Optional> visit( - DecimalLogicalTypeAnnotation decimalLogicalType) { - switch (primitive.getPrimitiveTypeName()) { - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return Optional.of( - new BinaryDecimalReader( - desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - case INT64: - return Optional.of( - new LongDecimalReader( - desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - case INT32: - return Optional.of( - new IntegerDecimalReader( - desc, decimalLogicalType.getPrecision(), decimalLogicalType.getScale())); - } - - return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(decimalLogicalType); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.DateLogicalTypeAnnotation dateLogicalType) { - return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.TimeLogicalTypeAnnotation timeLogicalType) { - if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { - return Optional.of(new MillisTimeReader(desc)); - } else if (timeLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { - return Optional.of(new LossyMicrosToMillisTimeReader(desc)); - } - - return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timeLogicalType); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.TimestampLogicalTypeAnnotation timestampLogicalType) { - if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MILLIS) { - if (timestampLogicalType.isAdjustedToUTC()) { - return Optional.of(new MillisToTimestampTzReader(desc)); - } else { - return Optional.of(new MillisToTimestampReader(desc)); - } - } else if (timestampLogicalType.getUnit() == LogicalTypeAnnotation.TimeUnit.MICROS) { - if (timestampLogicalType.isAdjustedToUTC()) { - return Optional.of(new MicrosToTimestampTzReader(desc)); - } else { - return Optional.of(new MicrosToTimestampReader(desc)); - } - } - - return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(timestampLogicalType); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.IntLogicalTypeAnnotation intLogicalType) { - int width = intLogicalType.getBitWidth(); - if (width <= 32) { - if (expected.typeId() == Types.LongType.get().typeId()) { - return Optional.of(new ParquetValueReaders.IntAsLongReader(desc)); - } else { - return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); - } - } else if (width <= 64) { - return Optional.of(new ParquetValueReaders.UnboxedReader<>(desc)); - } - - return LogicalTypeAnnotation.LogicalTypeAnnotationVisitor.super.visit(intLogicalType); - } - - @Override - public Optional> visit( - LogicalTypeAnnotation.BsonLogicalTypeAnnotation bsonLogicalType) { - return Optional.of(new ParquetValueReaders.ByteArrayReader(desc)); - } - } - - @Override - @SuppressWarnings("CyclomaticComplexity") - public ParquetValueReader primitive( - org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { - if (expected == null) { - return null; - } - - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - LogicalTypeAnnotation logicalTypeAnnotation = primitive.getLogicalTypeAnnotation(); - if (logicalTypeAnnotation != null) { - return logicalTypeAnnotation - .accept(new LogicalTypeAnnotationParquetValueReaderVisitor(primitive, desc, expected)) - .orElseThrow( - () -> - new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getLogicalTypeAnnotation())); - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return new ParquetValueReaders.ByteArrayReader(desc); - case INT32: - if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.LONG) { - return new ParquetValueReaders.IntAsLongReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case FLOAT: - if (expected.typeId() == org.apache.iceberg.types.Type.TypeID.DOUBLE) { - return new ParquetValueReaders.FloatAsDoubleReader(desc); - } else { - return new ParquetValueReaders.UnboxedReader<>(desc); - } - case BOOLEAN: - case INT64: - case DOUBLE: - return new ParquetValueReaders.UnboxedReader<>(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static class BinaryDecimalReader - extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - BinaryDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - Binary binary = column.nextBinary(); - BigDecimal bigDecimal = new BigDecimal(new BigInteger(binary.getBytes()), scale); - // TODO: need a unit test to write-read-validate decimal via FlinkParquetWrite/Reader - return DecimalData.fromBigDecimal(bigDecimal, precision, scale); - } - } - - private static class IntegerDecimalReader - extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - IntegerDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - return DecimalData.fromUnscaledLong(column.nextInteger(), precision, scale); - } - } - - private static class LongDecimalReader extends ParquetValueReaders.PrimitiveReader { - private final int precision; - private final int scale; - - LongDecimalReader(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(DecimalData ignored) { - return DecimalData.fromUnscaledLong(column.nextLong(), precision, scale); - } - } - - private static class MicrosToTimestampTzReader - extends ParquetValueReaders.UnboxedReader { - MicrosToTimestampTzReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long value = readLong(); - return TimestampData.fromLocalDateTime( - Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L) - .atOffset(ZoneOffset.UTC) - .toLocalDateTime()); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MicrosToTimestampReader - extends ParquetValueReaders.UnboxedReader { - MicrosToTimestampReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long value = readLong(); - return TimestampData.fromInstant( - Instant.ofEpochSecond( - Math.floorDiv(value, 1000_000L), Math.floorMod(value, 1000_000L) * 1000L)); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MillisToTimestampReader - extends ParquetValueReaders.UnboxedReader { - MillisToTimestampReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long millis = readLong(); - return TimestampData.fromEpochMillis(millis); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class MillisToTimestampTzReader - extends ParquetValueReaders.UnboxedReader { - MillisToTimestampTzReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public TimestampData read(TimestampData ignored) { - long millis = readLong(); - return TimestampData.fromLocalDateTime( - Instant.ofEpochMilli(millis).atOffset(ZoneOffset.UTC).toLocalDateTime()); - } - - @Override - public long readLong() { - return column.nextLong(); - } - } - - private static class StringReader extends ParquetValueReaders.PrimitiveReader { - StringReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public StringData read(StringData ignored) { - Binary binary = column.nextBinary(); - ByteBuffer buffer = binary.toByteBuffer(); - if (buffer.hasArray()) { - return StringData.fromBytes( - buffer.array(), buffer.arrayOffset() + buffer.position(), buffer.remaining()); - } else { - return StringData.fromBytes(binary.getBytes()); - } - } - } - - private static class LossyMicrosToMillisTimeReader - extends ParquetValueReaders.PrimitiveReader { - LossyMicrosToMillisTimeReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Integer read(Integer reuse) { - // Discard microseconds since Flink uses millisecond unit for TIME type. - return (int) Math.floorDiv(column.nextLong(), 1000L); - } - } - - private static class MillisTimeReader extends ParquetValueReaders.PrimitiveReader { - MillisTimeReader(ColumnDescriptor desc) { - super(desc); - } - - @Override - public Integer read(Integer reuse) { - return (int) column.nextLong(); - } - } - - private static class ArrayReader - extends ParquetValueReaders.RepeatedReader { - private int readPos = 0; - private int writePos = 0; - - ArrayReader(int definitionLevel, int repetitionLevel, ParquetValueReader reader) { - super(definitionLevel, repetitionLevel, reader); - } - - @Override - protected ReusableArrayData newListData(ArrayData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableArrayData) { - return (ReusableArrayData) reuse; - } else { - return new ReusableArrayData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected E getElement(ReusableArrayData list) { - E value = null; - if (readPos < list.capacity()) { - value = (E) list.values[readPos]; - } - - readPos += 1; - - return value; - } - - @Override - protected void addElement(ReusableArrayData reused, E element) { - if (writePos >= reused.capacity()) { - reused.grow(); - } - - reused.values[writePos] = element; - - writePos += 1; - } - - @Override - protected ArrayData buildList(ReusableArrayData list) { - // Since ReusableArrayData is not accepted by Flink, use GenericArrayData temporarily to walk - // around it. - // Revert this to use ReusableArrayData once it is fixed in Flink. - // For your reference, https://issues.apache.org/jira/browse/FLINK-25238. - return new GenericArrayData(Arrays.copyOf(list.values, writePos)); - } - } - - private static class MapReader - extends ParquetValueReaders.RepeatedKeyValueReader { - private int readPos = 0; - private int writePos = 0; - - private final ParquetValueReaders.ReusableEntry entry = - new ParquetValueReaders.ReusableEntry<>(); - private final ParquetValueReaders.ReusableEntry nullEntry = - new ParquetValueReaders.ReusableEntry<>(); - - MapReader( - int definitionLevel, - int repetitionLevel, - ParquetValueReader keyReader, - ParquetValueReader valueReader) { - super(definitionLevel, repetitionLevel, keyReader, valueReader); - } - - @Override - protected ReusableMapData newMapData(MapData reuse) { - this.readPos = 0; - this.writePos = 0; - - if (reuse instanceof ReusableMapData) { - return (ReusableMapData) reuse; - } else { - return new ReusableMapData(); - } - } - - @Override - @SuppressWarnings("unchecked") - protected Map.Entry getPair(ReusableMapData map) { - Map.Entry kv = nullEntry; - if (readPos < map.capacity()) { - entry.set((K) map.keys.values[readPos], (V) map.values.values[readPos]); - kv = entry; - } - - readPos += 1; - - return kv; - } - - @Override - protected void addPair(ReusableMapData map, K key, V value) { - if (writePos >= map.capacity()) { - map.grow(); - } - - map.keys.values[writePos] = key; - map.values.values[writePos] = value; - - writePos += 1; - } - - @Override - protected MapData buildMap(ReusableMapData map) { - map.setNumElements(writePos); - return map; - } - } - - private static class RowDataReader - extends ParquetValueReaders.StructReader { - private final int numFields; - - RowDataReader(List types, List> readers) { - super(types, readers); - this.numFields = readers.size(); - } - - @Override - protected GenericRowData newStructData(RowData reuse) { - if (reuse instanceof GenericRowData) { - return (GenericRowData) reuse; - } else { - return new GenericRowData(numFields); - } - } - - @Override - protected Object getField(GenericRowData intermediate, int pos) { - return intermediate.getField(pos); - } - - @Override - protected RowData buildStruct(GenericRowData struct) { - return struct; - } - - @Override - protected void set(GenericRowData row, int pos, Object value) { - row.setField(pos, value); - } - - @Override - protected void setNull(GenericRowData row, int pos) { - row.setField(pos, null); - } - - @Override - protected void setBoolean(GenericRowData row, int pos, boolean value) { - row.setField(pos, value); - } - - @Override - protected void setInteger(GenericRowData row, int pos, int value) { - row.setField(pos, value); - } - - @Override - protected void setLong(GenericRowData row, int pos, long value) { - row.setField(pos, value); - } - - @Override - protected void setFloat(GenericRowData row, int pos, float value) { - row.setField(pos, value); - } - - @Override - protected void setDouble(GenericRowData row, int pos, double value) { - row.setField(pos, value); - } - } - - private static class ReusableMapData implements MapData { - private final ReusableArrayData keys; - private final ReusableArrayData values; - - private int numElements; - - private ReusableMapData() { - this.keys = new ReusableArrayData(); - this.values = new ReusableArrayData(); - } - - private void grow() { - keys.grow(); - values.grow(); - } - - private int capacity() { - return keys.capacity(); - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - keys.setNumElements(numElements); - values.setNumElements(numElements); - } - - @Override - public int size() { - return numElements; - } - - @Override - public ReusableArrayData keyArray() { - return keys; - } - - @Override - public ReusableArrayData valueArray() { - return values; - } - } - - private static class ReusableArrayData implements ArrayData { - private static final Object[] EMPTY = new Object[0]; - - private Object[] values = EMPTY; - private int numElements = 0; - - private void grow() { - if (values.length == 0) { - this.values = new Object[20]; - } else { - Object[] old = values; - this.values = new Object[old.length << 1]; - // copy the old array in case it has values that can be reused - System.arraycopy(old, 0, values, 0, old.length); - } - } - - private int capacity() { - return values.length; - } - - public void setNumElements(int numElements) { - this.numElements = numElements; - } - - @Override - public int size() { - return numElements; - } - - @Override - public boolean isNullAt(int ordinal) { - return null == values[ordinal]; - } - - @Override - public boolean getBoolean(int ordinal) { - return (boolean) values[ordinal]; - } - - @Override - public byte getByte(int ordinal) { - return (byte) values[ordinal]; - } - - @Override - public short getShort(int ordinal) { - return (short) values[ordinal]; - } - - @Override - public int getInt(int ordinal) { - return (int) values[ordinal]; - } - - @Override - public long getLong(int ordinal) { - return (long) values[ordinal]; - } - - @Override - public float getFloat(int ordinal) { - return (float) values[ordinal]; - } - - @Override - public double getDouble(int ordinal) { - return (double) values[ordinal]; - } - - @Override - public StringData getString(int pos) { - return (StringData) values[pos]; - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return (DecimalData) values[pos]; - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return (TimestampData) values[pos]; - } - - @SuppressWarnings("unchecked") - @Override - public RawValueData getRawValue(int pos) { - return (RawValueData) values[pos]; - } - - @Override - public byte[] getBinary(int ordinal) { - return (byte[]) values[ordinal]; - } - - @Override - public ArrayData getArray(int ordinal) { - return (ArrayData) values[ordinal]; - } - - @Override - public MapData getMap(int ordinal) { - return (MapData) values[ordinal]; - } - - @Override - public RowData getRow(int pos, int numFields) { - return (RowData) values[pos]; - } - - @Override - public boolean[] toBooleanArray() { - return ArrayUtil.toPrimitive((Boolean[]) values); - } - - @Override - public byte[] toByteArray() { - return ArrayUtil.toPrimitive((Byte[]) values); - } - - @Override - public short[] toShortArray() { - return ArrayUtil.toPrimitive((Short[]) values); - } - - @Override - public int[] toIntArray() { - return ArrayUtil.toPrimitive((Integer[]) values); - } - - @Override - public long[] toLongArray() { - return ArrayUtil.toPrimitive((Long[]) values); - } - - @Override - public float[] toFloatArray() { - return ArrayUtil.toPrimitive((Float[]) values); - } - - @Override - public double[] toDoubleArray() { - return ArrayUtil.toPrimitive((Double[]) values); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java deleted file mode 100644 index db4f1730a134..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkParquetWriters.java +++ /dev/null @@ -1,504 +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.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NoSuchElementException; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.RowType.RowField; -import org.apache.flink.table.types.logical.SmallIntType; -import org.apache.flink.table.types.logical.TinyIntType; -import org.apache.iceberg.parquet.ParquetValueReaders; -import org.apache.iceberg.parquet.ParquetValueWriter; -import org.apache.iceberg.parquet.ParquetValueWriters; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; -import org.apache.parquet.column.ColumnDescriptor; -import org.apache.parquet.io.api.Binary; -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; - -public class FlinkParquetWriters { - private FlinkParquetWriters() {} - - @SuppressWarnings("unchecked") - public static ParquetValueWriter buildWriter(LogicalType schema, MessageType type) { - return (ParquetValueWriter) - ParquetWithFlinkSchemaVisitor.visit(schema, type, new WriteBuilder(type)); - } - - private static class WriteBuilder extends ParquetWithFlinkSchemaVisitor> { - private final MessageType type; - - WriteBuilder(MessageType type) { - this.type = type; - } - - @Override - public ParquetValueWriter message( - RowType sStruct, MessageType message, List> fields) { - return struct(sStruct, message.asGroupType(), fields); - } - - @Override - public ParquetValueWriter struct( - RowType sStruct, GroupType struct, List> fieldWriters) { - List fields = struct.getFields(); - List flinkFields = sStruct.getFields(); - List> writers = Lists.newArrayListWithExpectedSize(fieldWriters.size()); - List flinkTypes = Lists.newArrayList(); - for (int i = 0; i < fields.size(); i += 1) { - writers.add(newOption(struct.getType(i), fieldWriters.get(i))); - flinkTypes.add(flinkFields.get(i).getType()); - } - - return new RowDataWriter(writers, flinkTypes); - } - - @Override - public ParquetValueWriter list( - ArrayType sArray, GroupType array, ParquetValueWriter elementWriter) { - GroupType repeated = array.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new ArrayDataWriter<>( - repeatedD, - repeatedR, - newOption(repeated.getType(0), elementWriter), - sArray.getElementType()); - } - - @Override - public ParquetValueWriter map( - MapType sMap, - GroupType map, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter) { - GroupType repeatedKeyValue = map.getFields().get(0).asGroupType(); - String[] repeatedPath = currentPath(); - - int repeatedD = type.getMaxDefinitionLevel(repeatedPath); - int repeatedR = type.getMaxRepetitionLevel(repeatedPath); - - return new MapDataWriter<>( - repeatedD, - repeatedR, - newOption(repeatedKeyValue.getType(0), keyWriter), - newOption(repeatedKeyValue.getType(1), valueWriter), - sMap.getKeyType(), - sMap.getValueType()); - } - - private ParquetValueWriter newOption(Type fieldType, ParquetValueWriter writer) { - int maxD = type.getMaxDefinitionLevel(path(fieldType.getName())); - return ParquetValueWriters.option(fieldType, maxD, writer); - } - - @Override - public ParquetValueWriter primitive(LogicalType fType, PrimitiveType primitive) { - ColumnDescriptor desc = type.getColumnDescription(currentPath()); - - if (primitive.getOriginalType() != null) { - switch (primitive.getOriginalType()) { - case ENUM: - case JSON: - case UTF8: - return strings(desc); - case DATE: - case INT_8: - case INT_16: - case INT_32: - return ints(fType, desc); - case INT_64: - return ParquetValueWriters.longs(desc); - case TIME_MICROS: - return timeMicros(desc); - case TIMESTAMP_MICROS: - return timestamps(desc); - case DECIMAL: - DecimalLogicalTypeAnnotation decimal = - (DecimalLogicalTypeAnnotation) primitive.getLogicalTypeAnnotation(); - switch (primitive.getPrimitiveTypeName()) { - case INT32: - return decimalAsInteger(desc, decimal.getPrecision(), decimal.getScale()); - case INT64: - return decimalAsLong(desc, decimal.getPrecision(), decimal.getScale()); - case BINARY: - case FIXED_LEN_BYTE_ARRAY: - return decimalAsFixed(desc, decimal.getPrecision(), decimal.getScale()); - default: - throw new UnsupportedOperationException( - "Unsupported base type for decimal: " + primitive.getPrimitiveTypeName()); - } - case BSON: - return byteArrays(desc); - default: - throw new UnsupportedOperationException( - "Unsupported logical type: " + primitive.getOriginalType()); - } - } - - switch (primitive.getPrimitiveTypeName()) { - case FIXED_LEN_BYTE_ARRAY: - case BINARY: - return byteArrays(desc); - case BOOLEAN: - return ParquetValueWriters.booleans(desc); - case INT32: - return ints(fType, desc); - case INT64: - return ParquetValueWriters.longs(desc); - case FLOAT: - return ParquetValueWriters.floats(desc); - case DOUBLE: - return ParquetValueWriters.doubles(desc); - default: - throw new UnsupportedOperationException("Unsupported type: " + primitive); - } - } - } - - private static ParquetValueWriters.PrimitiveWriter ints( - LogicalType type, ColumnDescriptor desc) { - if (type instanceof TinyIntType) { - return ParquetValueWriters.tinyints(desc); - } else if (type instanceof SmallIntType) { - return ParquetValueWriters.shorts(desc); - } - return ParquetValueWriters.ints(desc); - } - - private static ParquetValueWriters.PrimitiveWriter strings(ColumnDescriptor desc) { - return new StringDataWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter timeMicros(ColumnDescriptor desc) { - return new TimeMicrosWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsInteger( - ColumnDescriptor desc, int precision, int scale) { - Preconditions.checkArgument( - precision <= 9, - "Cannot write decimal value as integer with precision larger than 9," - + " wrong precision %s", - precision); - return new IntegerDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsLong( - ColumnDescriptor desc, int precision, int scale) { - Preconditions.checkArgument( - precision <= 18, - "Cannot write decimal value as long with precision larger than 18, " - + " wrong precision %s", - precision); - return new LongDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter decimalAsFixed( - ColumnDescriptor desc, int precision, int scale) { - return new FixedDecimalWriter(desc, precision, scale); - } - - private static ParquetValueWriters.PrimitiveWriter timestamps( - ColumnDescriptor desc) { - return new TimestampDataWriter(desc); - } - - private static ParquetValueWriters.PrimitiveWriter byteArrays(ColumnDescriptor desc) { - return new ByteArrayWriter(desc); - } - - private static class StringDataWriter extends ParquetValueWriters.PrimitiveWriter { - private StringDataWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, StringData value) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(value.toBytes())); - } - } - - private static class TimeMicrosWriter extends ParquetValueWriters.PrimitiveWriter { - private TimeMicrosWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, Integer value) { - long micros = value.longValue() * 1000; - column.writeLong(repetitionLevel, micros); - } - } - - private static class IntegerDecimalWriter - extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private IntegerDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeInteger(repetitionLevel, (int) decimal.toUnscaledLong()); - } - } - - private static class LongDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - - private LongDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - Preconditions.checkArgument( - decimal.scale() == scale, - "Cannot write value as decimal(%s,%s), wrong scale: %s", - precision, - scale, - decimal); - Preconditions.checkArgument( - decimal.precision() <= precision, - "Cannot write value as decimal(%s,%s), too large: %s", - precision, - scale, - decimal); - - column.writeLong(repetitionLevel, decimal.toUnscaledLong()); - } - } - - private static class FixedDecimalWriter extends ParquetValueWriters.PrimitiveWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private FixedDecimalWriter(ColumnDescriptor desc, int precision, int scale) { - super(desc); - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(int repetitionLevel, DecimalData decimal) { - byte[] binary = - DecimalUtil.toReusedFixLengthBytes(precision, scale, decimal.toBigDecimal(), bytes.get()); - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(binary)); - } - } - - private static class TimestampDataWriter - extends ParquetValueWriters.PrimitiveWriter { - private TimestampDataWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, TimestampData value) { - column.writeLong( - repetitionLevel, value.getMillisecond() * 1000 + value.getNanoOfMillisecond() / 1000); - } - } - - private static class ByteArrayWriter extends ParquetValueWriters.PrimitiveWriter { - private ByteArrayWriter(ColumnDescriptor desc) { - super(desc); - } - - @Override - public void write(int repetitionLevel, byte[] bytes) { - column.writeBinary(repetitionLevel, Binary.fromReusedByteArray(bytes)); - } - } - - private static class ArrayDataWriter extends ParquetValueWriters.RepeatedWriter { - private final LogicalType elementType; - - private ArrayDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter writer, - LogicalType elementType) { - super(definitionLevel, repetitionLevel, writer); - this.elementType = elementType; - } - - @Override - protected Iterator elements(ArrayData list) { - return new ElementIterator<>(list); - } - - private class ElementIterator implements Iterator { - private final int size; - private final ArrayData list; - private final ArrayData.ElementGetter getter; - private int index; - - private ElementIterator(ArrayData list) { - this.list = list; - size = list.size(); - getter = ArrayData.createElementGetter(elementType); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public E next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - E element = (E) getter.getElementOrNull(list, index); - index += 1; - - return element; - } - } - } - - private static class MapDataWriter - extends ParquetValueWriters.RepeatedKeyValueWriter { - private final LogicalType keyType; - private final LogicalType valueType; - - private MapDataWriter( - int definitionLevel, - int repetitionLevel, - ParquetValueWriter keyWriter, - ParquetValueWriter valueWriter, - LogicalType keyType, - LogicalType valueType) { - super(definitionLevel, repetitionLevel, keyWriter, valueWriter); - this.keyType = keyType; - this.valueType = valueType; - } - - @Override - protected Iterator> pairs(MapData map) { - return new EntryIterator<>(map); - } - - private class EntryIterator implements Iterator> { - private final int size; - private final ArrayData keys; - private final ArrayData values; - private final ParquetValueReaders.ReusableEntry entry; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - private int index; - - private EntryIterator(MapData map) { - size = map.size(); - keys = map.keyArray(); - values = map.valueArray(); - entry = new ParquetValueReaders.ReusableEntry<>(); - keyGetter = ArrayData.createElementGetter(keyType); - valueGetter = ArrayData.createElementGetter(valueType); - index = 0; - } - - @Override - public boolean hasNext() { - return index != size; - } - - @Override - @SuppressWarnings("unchecked") - public Map.Entry next() { - if (index >= size) { - throw new NoSuchElementException(); - } - - entry.set( - (K) keyGetter.getElementOrNull(keys, index), - (V) valueGetter.getElementOrNull(values, index)); - index += 1; - - return entry; - } - } - } - - private static class RowDataWriter extends ParquetValueWriters.StructWriter { - private final RowData.FieldGetter[] fieldGetter; - - RowDataWriter(List> writers, List types) { - super(writers); - fieldGetter = new RowData.FieldGetter[types.size()]; - for (int i = 0; i < types.size(); i += 1) { - fieldGetter[i] = RowData.createFieldGetter(types.get(i), i); - } - } - - @Override - protected Object get(RowData struct, int index) { - return fieldGetter[index].getFieldOrNull(struct); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java deleted file mode 100644 index ba4e1a7a7aec..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkSchemaVisitor.java +++ /dev/null @@ -1,161 +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.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -abstract class FlinkSchemaVisitor { - - static T visit(RowType flinkType, Schema schema, FlinkSchemaVisitor visitor) { - return visit(flinkType, schema.asStruct(), visitor); - } - - private static T visit(LogicalType flinkType, Type iType, FlinkSchemaVisitor visitor) { - switch (iType.typeId()) { - case STRUCT: - return visitRecord(flinkType, iType.asStructType(), visitor); - - case MAP: - MapType mapType = (MapType) flinkType; - Types.MapType iMapType = iType.asMapType(); - T key; - T value; - - Types.NestedField keyField = iMapType.field(iMapType.keyId()); - visitor.beforeMapKey(keyField); - try { - key = visit(mapType.getKeyType(), iMapType.keyType(), visitor); - } finally { - visitor.afterMapKey(keyField); - } - - Types.NestedField valueField = iMapType.field(iMapType.valueId()); - visitor.beforeMapValue(valueField); - try { - value = visit(mapType.getValueType(), iMapType.valueType(), visitor); - } finally { - visitor.afterMapValue(valueField); - } - - return visitor.map(iMapType, key, value, mapType.getKeyType(), mapType.getValueType()); - - case LIST: - ArrayType listType = (ArrayType) flinkType; - Types.ListType iListType = iType.asListType(); - T element; - - Types.NestedField elementField = iListType.field(iListType.elementId()); - visitor.beforeListElement(elementField); - try { - element = visit(listType.getElementType(), iListType.elementType(), visitor); - } finally { - visitor.afterListElement(elementField); - } - - return visitor.list(iListType, element, listType.getElementType()); - - default: - return visitor.primitive(iType.asPrimitiveType(), flinkType); - } - } - - private static T visitRecord( - LogicalType flinkType, Types.StructType struct, FlinkSchemaVisitor visitor) { - Preconditions.checkArgument(flinkType instanceof RowType, "%s is not a RowType.", flinkType); - RowType rowType = (RowType) flinkType; - - int fieldSize = struct.fields().size(); - List results = Lists.newArrayListWithExpectedSize(fieldSize); - List fieldTypes = Lists.newArrayListWithExpectedSize(fieldSize); - List nestedFields = struct.fields(); - - for (int i = 0; i < fieldSize; i++) { - Types.NestedField iField = nestedFields.get(i); - int fieldIndex = rowType.getFieldIndex(iField.name()); - Preconditions.checkArgument( - fieldIndex >= 0, "NestedField: %s is not found in flink RowType: %s", iField, rowType); - - LogicalType fieldFlinkType = rowType.getTypeAt(fieldIndex); - - fieldTypes.add(fieldFlinkType); - - visitor.beforeField(iField); - try { - results.add(visit(fieldFlinkType, iField.type(), visitor)); - } finally { - visitor.afterField(iField); - } - } - - return visitor.record(struct, results, fieldTypes); - } - - public T record(Types.StructType iStruct, List results, List fieldTypes) { - return null; - } - - public T list(Types.ListType iList, T element, LogicalType elementType) { - return null; - } - - public T map(Types.MapType iMap, T key, T value, LogicalType keyType, LogicalType valueType) { - return null; - } - - public T primitive(Type.PrimitiveType iPrimitive, LogicalType flinkPrimitive) { - return null; - } - - public void beforeField(Types.NestedField field) {} - - public void afterField(Types.NestedField field) {} - - public void beforeListElement(Types.NestedField elementField) { - beforeField(elementField); - } - - public void afterListElement(Types.NestedField elementField) { - afterField(elementField); - } - - public void beforeMapKey(Types.NestedField keyField) { - beforeField(keyField); - } - - public void afterMapKey(Types.NestedField keyField) { - afterField(keyField); - } - - public void beforeMapValue(Types.NestedField valueField) { - beforeField(valueField); - } - - public void afterMapValue(Types.NestedField valueField) { - afterField(valueField); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java deleted file mode 100644 index 32f6c3a2ccfd..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueReaders.java +++ /dev/null @@ -1,312 +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.math.BigDecimal; -import java.math.BigInteger; -import java.nio.charset.StandardCharsets; -import java.util.List; -import java.util.Map; -import org.apache.avro.io.Decoder; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.avro.ValueReader; -import org.apache.iceberg.avro.ValueReaders; -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; - -public class FlinkValueReaders { - - private FlinkValueReaders() {} - - static ValueReader strings() { - return StringReader.INSTANCE; - } - - static ValueReader enums(List symbols) { - return new EnumReader(symbols); - } - - static ValueReader uuids() { - return ValueReaders.fixed(16); - } - - static ValueReader timeMicros() { - return TimeMicrosReader.INSTANCE; - } - - static ValueReader timestampMills() { - return TimestampMillsReader.INSTANCE; - } - - static ValueReader timestampMicros() { - return TimestampMicrosReader.INSTANCE; - } - - static ValueReader decimal( - ValueReader unscaledReader, int precision, int scale) { - return new DecimalReader(unscaledReader, precision, scale); - } - - static ValueReader array(ValueReader elementReader) { - return new ArrayReader(elementReader); - } - - static ValueReader arrayMap(ValueReader keyReader, ValueReader valueReader) { - return new ArrayMapReader(keyReader, valueReader); - } - - static ValueReader map(ValueReader keyReader, ValueReader valueReader) { - return new MapReader(keyReader, valueReader); - } - - static ValueReader struct( - List> readers, Types.StructType struct, Map idToConstant) { - return new StructReader(readers, struct, idToConstant); - } - - private static class StringReader implements ValueReader { - private static final StringReader INSTANCE = new StringReader(); - - private StringReader() {} - - @Override - public StringData read(Decoder decoder, Object reuse) throws IOException { - // use the decoder's readString(Utf8) method because it may be a resolving decoder - Utf8 utf8 = null; - if (reuse instanceof StringData) { - utf8 = new Utf8(((StringData) reuse).toBytes()); - } - - Utf8 string = decoder.readString(utf8); - return StringData.fromBytes(string.getBytes(), 0, string.getByteLength()); - } - } - - private static class EnumReader implements ValueReader { - private final StringData[] symbols; - - private EnumReader(List symbols) { - this.symbols = new StringData[symbols.size()]; - for (int i = 0; i < this.symbols.length; i += 1) { - this.symbols[i] = StringData.fromBytes(symbols.get(i).getBytes(StandardCharsets.UTF_8)); - } - } - - @Override - public StringData read(Decoder decoder, Object ignore) throws IOException { - int index = decoder.readEnum(); - return symbols[index]; - } - } - - private static class DecimalReader implements ValueReader { - private final ValueReader bytesReader; - private final int precision; - private final int scale; - - private DecimalReader(ValueReader bytesReader, int precision, int scale) { - this.bytesReader = bytesReader; - this.precision = precision; - this.scale = scale; - } - - @Override - public DecimalData read(Decoder decoder, Object reuse) throws IOException { - byte[] bytes = bytesReader.read(decoder, null); - return DecimalData.fromBigDecimal( - new BigDecimal(new BigInteger(bytes), scale), precision, scale); - } - } - - private static class TimeMicrosReader implements ValueReader { - private static final TimeMicrosReader INSTANCE = new TimeMicrosReader(); - - @Override - public Integer read(Decoder decoder, Object reuse) throws IOException { - long micros = decoder.readLong(); - // Flink only support time mills, just erase micros. - return (int) (micros / 1000); - } - } - - private static class TimestampMillsReader implements ValueReader { - private static final TimestampMillsReader INSTANCE = new TimestampMillsReader(); - - @Override - public TimestampData read(Decoder decoder, Object reuse) throws IOException { - return TimestampData.fromEpochMillis(decoder.readLong()); - } - } - - private static class TimestampMicrosReader implements ValueReader { - private static final TimestampMicrosReader INSTANCE = new TimestampMicrosReader(); - - @Override - public TimestampData read(Decoder decoder, Object reuse) throws IOException { - long micros = decoder.readLong(); - long mills = micros / 1000; - int nanos = ((int) (micros % 1000)) * 1000; - if (nanos < 0) { - nanos += 1_000_000; - mills -= 1; - } - return TimestampData.fromEpochMillis(mills, nanos); - } - } - - private static class ArrayReader implements ValueReader { - private final ValueReader elementReader; - private final List reusedList = Lists.newArrayList(); - - private ArrayReader(ValueReader elementReader) { - this.elementReader = elementReader; - } - - @Override - public GenericArrayData read(Decoder decoder, Object reuse) throws IOException { - reusedList.clear(); - long chunkLength = decoder.readArrayStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedList.add(elementReader.read(decoder, null)); - } - - chunkLength = decoder.arrayNext(); - } - - // this will convert the list to an array so it is okay to reuse the list - return new GenericArrayData(reusedList.toArray()); - } - } - - private static MapData kvArrayToMap(List keyList, List valueList) { - Map map = Maps.newHashMap(); - Object[] keys = keyList.toArray(); - Object[] values = valueList.toArray(); - for (int i = 0; i < keys.length; i++) { - map.put(keys[i], values[i]); - } - - return new GenericMapData(map); - } - - private static class ArrayMapReader implements ValueReader { - private final ValueReader keyReader; - private final ValueReader valueReader; - - private final List reusedKeyList = Lists.newArrayList(); - private final List reusedValueList = Lists.newArrayList(); - - private ArrayMapReader(ValueReader keyReader, ValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData read(Decoder decoder, Object reuse) throws IOException { - reusedKeyList.clear(); - reusedValueList.clear(); - - long chunkLength = decoder.readArrayStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedKeyList.add(keyReader.read(decoder, null)); - reusedValueList.add(valueReader.read(decoder, null)); - } - - chunkLength = decoder.arrayNext(); - } - - return kvArrayToMap(reusedKeyList, reusedValueList); - } - } - - private static class MapReader implements ValueReader { - private final ValueReader keyReader; - private final ValueReader valueReader; - - private final List reusedKeyList = Lists.newArrayList(); - private final List reusedValueList = Lists.newArrayList(); - - private MapReader(ValueReader keyReader, ValueReader valueReader) { - this.keyReader = keyReader; - this.valueReader = valueReader; - } - - @Override - public MapData read(Decoder decoder, Object reuse) throws IOException { - reusedKeyList.clear(); - reusedValueList.clear(); - - long chunkLength = decoder.readMapStart(); - - while (chunkLength > 0) { - for (int i = 0; i < chunkLength; i += 1) { - reusedKeyList.add(keyReader.read(decoder, null)); - reusedValueList.add(valueReader.read(decoder, null)); - } - - chunkLength = decoder.mapNext(); - } - - return kvArrayToMap(reusedKeyList, reusedValueList); - } - } - - private static class StructReader extends ValueReaders.StructReader { - private final int numFields; - - private StructReader( - List> readers, Types.StructType struct, Map idToConstant) { - super(readers, struct, idToConstant); - this.numFields = readers.size(); - } - - @Override - protected RowData reuseOrCreate(Object reuse) { - if (reuse instanceof GenericRowData && ((GenericRowData) reuse).getArity() == numFields) { - return (GenericRowData) 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); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java deleted file mode 100644 index 4e86ecce28b5..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/FlinkValueWriters.java +++ /dev/null @@ -1,253 +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.lang.reflect.Array; -import java.util.List; -import org.apache.avro.io.Encoder; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.avro.ValueWriter; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.DecimalUtil; - -public class FlinkValueWriters { - - private FlinkValueWriters() {} - - static ValueWriter strings() { - return StringWriter.INSTANCE; - } - - static ValueWriter timeMicros() { - return TimeMicrosWriter.INSTANCE; - } - - static ValueWriter timestampMicros() { - return TimestampMicrosWriter.INSTANCE; - } - - static ValueWriter decimal(int precision, int scale) { - return new DecimalWriter(precision, scale); - } - - static ValueWriter array(ValueWriter elementWriter, LogicalType elementType) { - return new ArrayWriter<>(elementWriter, elementType); - } - - static ValueWriter arrayMap( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - return new ArrayMapWriter<>(keyWriter, keyType, valueWriter, valueType); - } - - static ValueWriter map( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - return new MapWriter<>(keyWriter, keyType, valueWriter, valueType); - } - - static ValueWriter row(List> writers, List types) { - return new RowWriter(writers, types); - } - - private static class StringWriter implements ValueWriter { - private static final StringWriter INSTANCE = new StringWriter(); - - private StringWriter() {} - - @Override - public void write(StringData s, Encoder encoder) throws IOException { - // toBytes is cheaper than Avro calling toString, which incurs encoding costs - encoder.writeString(new Utf8(s.toBytes())); - } - } - - private static class DecimalWriter implements ValueWriter { - private final int precision; - private final int scale; - private final ThreadLocal bytes; - - private DecimalWriter(int precision, int scale) { - this.precision = precision; - this.scale = scale; - this.bytes = - ThreadLocal.withInitial(() -> new byte[TypeUtil.decimalRequiredBytes(precision)]); - } - - @Override - public void write(DecimalData d, Encoder encoder) throws IOException { - encoder.writeFixed( - DecimalUtil.toReusedFixLengthBytes(precision, scale, d.toBigDecimal(), bytes.get())); - } - } - - private static class TimeMicrosWriter implements ValueWriter { - private static final TimeMicrosWriter INSTANCE = new TimeMicrosWriter(); - - @Override - public void write(Integer timeMills, Encoder encoder) throws IOException { - encoder.writeLong(timeMills * 1000L); - } - } - - private static class TimestampMicrosWriter implements ValueWriter { - private static final TimestampMicrosWriter INSTANCE = new TimestampMicrosWriter(); - - @Override - public void write(TimestampData timestampData, Encoder encoder) throws IOException { - long micros = - timestampData.getMillisecond() * 1000 + timestampData.getNanoOfMillisecond() / 1000; - encoder.writeLong(micros); - } - } - - private static class ArrayWriter implements ValueWriter { - private final ValueWriter elementWriter; - private final ArrayData.ElementGetter elementGetter; - - private ArrayWriter(ValueWriter elementWriter, LogicalType elementType) { - this.elementWriter = elementWriter; - this.elementGetter = ArrayData.createElementGetter(elementType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(ArrayData array, Encoder encoder) throws IOException { - encoder.writeArrayStart(); - int numElements = array.size(); - encoder.setItemCount(numElements); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - elementWriter.write((T) elementGetter.getElementOrNull(array, i), encoder); - } - encoder.writeArrayEnd(); - } - } - - private static class ArrayMapWriter implements ValueWriter { - private final ValueWriter keyWriter; - private final ValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - private ArrayMapWriter( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueWriter = valueWriter; - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(MapData map, Encoder encoder) throws IOException { - encoder.writeArrayStart(); - int numElements = map.size(); - encoder.setItemCount(numElements); - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); - valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); - } - encoder.writeArrayEnd(); - } - } - - private static class MapWriter implements ValueWriter { - private final ValueWriter keyWriter; - private final ValueWriter valueWriter; - private final ArrayData.ElementGetter keyGetter; - private final ArrayData.ElementGetter valueGetter; - - private MapWriter( - ValueWriter keyWriter, - LogicalType keyType, - ValueWriter valueWriter, - LogicalType valueType) { - this.keyWriter = keyWriter; - this.keyGetter = ArrayData.createElementGetter(keyType); - this.valueWriter = valueWriter; - this.valueGetter = ArrayData.createElementGetter(valueType); - } - - @Override - @SuppressWarnings("unchecked") - public void write(MapData map, Encoder encoder) throws IOException { - encoder.writeMapStart(); - int numElements = map.size(); - encoder.setItemCount(numElements); - ArrayData keyArray = map.keyArray(); - ArrayData valueArray = map.valueArray(); - for (int i = 0; i < numElements; i += 1) { - encoder.startItem(); - keyWriter.write((K) keyGetter.getElementOrNull(keyArray, i), encoder); - valueWriter.write((V) valueGetter.getElementOrNull(valueArray, i), encoder); - } - encoder.writeMapEnd(); - } - } - - static class RowWriter implements ValueWriter { - private final ValueWriter[] writers; - private final RowData.FieldGetter[] getters; - - private RowWriter(List> writers, List types) { - this.writers = (ValueWriter[]) Array.newInstance(ValueWriter.class, writers.size()); - this.getters = new RowData.FieldGetter[writers.size()]; - for (int i = 0; i < writers.size(); i += 1) { - this.writers[i] = writers.get(i); - this.getters[i] = RowData.createFieldGetter(types.get(i), i); - } - } - - @Override - public void write(RowData row, Encoder encoder) throws IOException { - for (int i = 0; i < writers.length; i += 1) { - if (row.isNullAt(i)) { - writers[i].write(null, encoder); - } else { - write(row, i, writers[i], encoder); - } - } - } - - @SuppressWarnings("unchecked") - private void write(RowData row, int pos, ValueWriter writer, Encoder encoder) - throws IOException { - writer.write((T) getters[pos].getFieldOrNull(row), encoder); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java deleted file mode 100644 index 33feb2e32118..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/ParquetWithFlinkSchemaVisitor.java +++ /dev/null @@ -1,222 +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.util.Deque; -import java.util.List; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.RowType.RowField; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.parquet.schema.GroupType; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.OriginalType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; - -public class ParquetWithFlinkSchemaVisitor { - private final Deque fieldNames = Lists.newLinkedList(); - - public static T visit( - LogicalType sType, Type type, ParquetWithFlinkSchemaVisitor visitor) { - Preconditions.checkArgument(sType != null, "Invalid DataType: null"); - if (type instanceof MessageType) { - Preconditions.checkArgument( - sType instanceof RowType, "Invalid struct: %s is not a struct", sType); - RowType struct = (RowType) sType; - return visitor.message( - struct, (MessageType) type, visitFields(struct, type.asGroupType(), visitor)); - } else if (type.isPrimitive()) { - return visitor.primitive(sType, type.asPrimitiveType()); - } else { - // if not a primitive, the typeId must be a group - GroupType group = type.asGroupType(); - OriginalType annotation = group.getOriginalType(); - if (annotation != null) { - switch (annotation) { - case LIST: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid list: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid list: does not contain single repeated field: %s", - group); - - GroupType repeatedElement = group.getFields().get(0).asGroupType(); - Preconditions.checkArgument( - repeatedElement.isRepetition(Type.Repetition.REPEATED), - "Invalid list: inner group is not repeated"); - Preconditions.checkArgument( - repeatedElement.getFieldCount() <= 1, - "Invalid list: repeated group is not a single field: %s", - group); - - Preconditions.checkArgument( - sType instanceof ArrayType, "Invalid list: %s is not an array", sType); - ArrayType array = (ArrayType) sType; - RowType.RowField element = - new RowField( - "element", array.getElementType(), "element of " + array.asSummaryString()); - - visitor.fieldNames.push(repeatedElement.getName()); - try { - T elementResult = null; - if (repeatedElement.getFieldCount() > 0) { - elementResult = visitField(element, repeatedElement.getType(0), visitor); - } - - return visitor.list(array, group, elementResult); - - } finally { - visitor.fieldNames.pop(); - } - - case MAP: - Preconditions.checkArgument( - !group.isRepetition(Type.Repetition.REPEATED), - "Invalid map: top-level group is repeated: %s", - group); - Preconditions.checkArgument( - group.getFieldCount() == 1, - "Invalid map: does not contain single repeated field: %s", - group); - - GroupType repeatedKeyValue = group.getType(0).asGroupType(); - Preconditions.checkArgument( - repeatedKeyValue.isRepetition(Type.Repetition.REPEATED), - "Invalid map: inner group is not repeated"); - Preconditions.checkArgument( - repeatedKeyValue.getFieldCount() <= 2, - "Invalid map: repeated group does not have 2 fields"); - - Preconditions.checkArgument( - sType instanceof MapType, "Invalid map: %s is not a map", sType); - MapType map = (MapType) sType; - RowField keyField = - new RowField("key", map.getKeyType(), "key of " + map.asSummaryString()); - RowField valueField = - new RowField("value", map.getValueType(), "value of " + map.asSummaryString()); - - visitor.fieldNames.push(repeatedKeyValue.getName()); - try { - T keyResult = null; - T valueResult = null; - switch (repeatedKeyValue.getFieldCount()) { - case 2: - // if there are 2 fields, both key and value are projected - keyResult = visitField(keyField, repeatedKeyValue.getType(0), visitor); - valueResult = visitField(valueField, repeatedKeyValue.getType(1), visitor); - break; - case 1: - // if there is just one, use the name to determine what it is - Type keyOrValue = repeatedKeyValue.getType(0); - if (keyOrValue.getName().equalsIgnoreCase("key")) { - keyResult = visitField(keyField, keyOrValue, visitor); - // value result remains null - } else { - valueResult = visitField(valueField, keyOrValue, visitor); - // key result remains null - } - break; - default: - // both results will remain null - } - - return visitor.map(map, group, keyResult, valueResult); - - } finally { - visitor.fieldNames.pop(); - } - - default: - } - } - Preconditions.checkArgument( - sType instanceof RowType, "Invalid struct: %s is not a struct", sType); - RowType struct = (RowType) sType; - return visitor.struct(struct, group, visitFields(struct, group, visitor)); - } - } - - private static T visitField( - RowType.RowField sField, Type field, ParquetWithFlinkSchemaVisitor visitor) { - visitor.fieldNames.push(field.getName()); - try { - return visit(sField.getType(), field, visitor); - } finally { - visitor.fieldNames.pop(); - } - } - - private static List visitFields( - RowType struct, GroupType group, ParquetWithFlinkSchemaVisitor visitor) { - List sFields = struct.getFields(); - Preconditions.checkArgument( - sFields.size() == group.getFieldCount(), "Structs do not match: %s and %s", struct, group); - List results = Lists.newArrayListWithExpectedSize(group.getFieldCount()); - for (int i = 0; i < sFields.size(); i += 1) { - Type field = group.getFields().get(i); - RowType.RowField sField = sFields.get(i); - Preconditions.checkArgument( - field.getName().equals(AvroSchemaUtil.makeCompatibleName(sField.getName())), - "Structs do not match: field %s != %s", - field.getName(), - sField.getName()); - results.add(visitField(sField, field, visitor)); - } - - return results; - } - - public T message(RowType sStruct, MessageType message, List fields) { - return null; - } - - public T struct(RowType sStruct, GroupType struct, List fields) { - return null; - } - - public T list(ArrayType sArray, GroupType array, T element) { - return null; - } - - public T map(MapType sMap, GroupType map, T key, T value) { - return null; - } - - public T primitive(LogicalType sPrimitive, PrimitiveType primitive) { - return null; - } - - protected String[] currentPath() { - return Lists.newArrayList(fieldNames.descendingIterator()).toArray(new String[0]); - } - - protected String[] path(String name) { - List list = Lists.newArrayList(fieldNames.descendingIterator()); - list.add(name); - return list.toArray(new String[0]); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java deleted file mode 100644 index 33816c97ac29..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataProjection.java +++ /dev/null @@ -1,341 +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.util.Arrays; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; - -public class RowDataProjection implements RowData { - /** - * Creates a projecting wrapper for {@link RowData} rows. - * - *

    This projection will not project the nested children types of repeated types like lists and - * maps. - * - * @param schema schema of rows wrapped by this projection - * @param projectedSchema result schema of the projected rows - * @return a wrapper to project rows - */ - public static RowDataProjection create(Schema schema, Schema projectedSchema) { - return RowDataProjection.create( - FlinkSchemaUtil.convert(schema), schema.asStruct(), projectedSchema.asStruct()); - } - - /** - * Creates a projecting wrapper for {@link RowData} rows. - * - *

    This projection will not project the nested children types of repeated types like lists and - * maps. - * - * @param rowType flink row type of rows wrapped by this projection - * @param schema schema of rows wrapped by this projection - * @param projectedSchema result schema of the projected rows - * @return a wrapper to project rows - */ - public static RowDataProjection create( - RowType rowType, Types.StructType schema, Types.StructType projectedSchema) { - return new RowDataProjection(rowType, schema, projectedSchema); - } - - private final RowData.FieldGetter[] getters; - private RowData rowData; - - private RowDataProjection( - RowType rowType, Types.StructType rowStruct, Types.StructType projectType) { - Map fieldIdToPosition = Maps.newHashMap(); - for (int i = 0; i < rowStruct.fields().size(); i++) { - fieldIdToPosition.put(rowStruct.fields().get(i).fieldId(), i); - } - - this.getters = new RowData.FieldGetter[projectType.fields().size()]; - for (int i = 0; i < getters.length; i++) { - Types.NestedField projectField = projectType.fields().get(i); - Types.NestedField rowField = rowStruct.field(projectField.fieldId()); - - Preconditions.checkNotNull( - rowField, - "Cannot locate the project field <%s> in the iceberg struct <%s>", - projectField, - rowStruct); - - getters[i] = - createFieldGetter( - rowType, fieldIdToPosition.get(projectField.fieldId()), rowField, projectField); - } - } - - private static RowData.FieldGetter createFieldGetter( - RowType rowType, int position, Types.NestedField rowField, Types.NestedField projectField) { - Preconditions.checkArgument( - rowField.type().typeId() == projectField.type().typeId(), - "Different iceberg type between row field <%s> and project field <%s>", - rowField, - projectField); - - switch (projectField.type().typeId()) { - case STRUCT: - RowType nestedRowType = (RowType) rowType.getTypeAt(position); - return row -> { - // null nested struct value - if (row.isNullAt(position)) { - return null; - } - - RowData nestedRow = row.getRow(position, nestedRowType.getFieldCount()); - return RowDataProjection.create( - nestedRowType, rowField.type().asStructType(), projectField.type().asStructType()) - .wrap(nestedRow); - }; - - case MAP: - Types.MapType projectedMap = projectField.type().asMapType(); - Types.MapType originalMap = rowField.type().asMapType(); - - boolean keyProjectable = - !projectedMap.keyType().isNestedType() - || projectedMap.keyType().equals(originalMap.keyType()); - boolean valueProjectable = - !projectedMap.valueType().isNestedType() - || projectedMap.valueType().equals(originalMap.valueType()); - Preconditions.checkArgument( - keyProjectable && valueProjectable, - "Cannot project a partial map key or value with non-primitive type. Trying to project <%s> out of <%s>", - projectField, - rowField); - - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - - case LIST: - Types.ListType projectedList = projectField.type().asListType(); - Types.ListType originalList = rowField.type().asListType(); - - boolean elementProjectable = - !projectedList.elementType().isNestedType() - || projectedList.elementType().equals(originalList.elementType()); - Preconditions.checkArgument( - elementProjectable, - "Cannot project a partial list element with non-primitive type. Trying to project <%s> out of <%s>", - projectField, - rowField); - - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - - default: - return RowData.createFieldGetter(rowType.getTypeAt(position), position); - } - } - - public RowData wrap(RowData row) { - // StructProjection allow wrapping null root struct object. - // See more discussions in https://github.com/apache/iceberg/pull/7517. - // RowDataProjection never allowed null root object to be wrapped. - // Hence, it is fine to enforce strict Preconditions check here. - Preconditions.checkArgument(row != null, "Invalid row data: null"); - this.rowData = row; - return this; - } - - private Object getValue(int pos) { - Preconditions.checkState(rowData != null, "Row data not wrapped"); - return getters[pos].getFieldOrNull(rowData); - } - - @Override - public int getArity() { - return getters.length; - } - - @Override - public RowKind getRowKind() { - Preconditions.checkState(rowData != null, "Row data not wrapped"); - return rowData.getRowKind(); - } - - @Override - public void setRowKind(RowKind kind) { - throw new UnsupportedOperationException("Cannot set row kind in the RowDataProjection"); - } - - @Override - public boolean isNullAt(int pos) { - return getValue(pos) == null; - } - - @Override - public boolean getBoolean(int pos) { - return (boolean) getValue(pos); - } - - @Override - public byte getByte(int pos) { - return (byte) getValue(pos); - } - - @Override - public short getShort(int pos) { - return (short) getValue(pos); - } - - @Override - public int getInt(int pos) { - return (int) getValue(pos); - } - - @Override - public long getLong(int pos) { - return (long) getValue(pos); - } - - @Override - public float getFloat(int pos) { - return (float) getValue(pos); - } - - @Override - public double getDouble(int pos) { - return (double) getValue(pos); - } - - @Override - public StringData getString(int pos) { - return (StringData) getValue(pos); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return (DecimalData) getValue(pos); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - return (TimestampData) getValue(pos); - } - - @Override - @SuppressWarnings("unchecked") - public RawValueData getRawValue(int pos) { - return (RawValueData) getValue(pos); - } - - @Override - public byte[] getBinary(int pos) { - return (byte[]) getValue(pos); - } - - @Override - public ArrayData getArray(int pos) { - return (ArrayData) getValue(pos); - } - - @Override - public MapData getMap(int pos) { - return (MapData) getValue(pos); - } - - @Override - public RowData getRow(int pos, int numFields) { - return (RowData) getValue(pos); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof RowDataProjection)) { - return false; - } - - RowDataProjection that = (RowDataProjection) o; - return deepEquals(that); - } - - @Override - public int hashCode() { - int result = Objects.hashCode(getRowKind()); - for (int pos = 0; pos < getArity(); pos++) { - if (!isNullAt(pos)) { - // Arrays.deepHashCode handles array object properly - result = 31 * result + Arrays.deepHashCode(new Object[] {getValue(pos)}); - } - } - - return result; - } - - @Override - public String toString() { - StringBuilder sb = new StringBuilder(); - sb.append(getRowKind().shortString()).append("("); - for (int pos = 0; pos < getArity(); pos++) { - if (pos != 0) { - sb.append(","); - } - // copied the behavior from Flink GenericRowData - sb.append(StringUtils.arrayAwareToString(getValue(pos))); - } - - sb.append(")"); - return sb.toString(); - } - - private boolean deepEquals(RowDataProjection other) { - if (getRowKind() != other.getRowKind()) { - return false; - } - - if (getArity() != other.getArity()) { - return false; - } - - for (int pos = 0; pos < getArity(); ++pos) { - if (isNullAt(pos) && other.isNullAt(pos)) { - continue; - } - - if ((isNullAt(pos) && !other.isNullAt(pos)) || (!isNullAt(pos) && other.isNullAt(pos))) { - return false; - } - - // Objects.deepEquals handles array object properly - if (!Objects.deepEquals(getValue(pos), other.getValue(pos))) { - return false; - } - } - - return true; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java deleted file mode 100644 index 3a8f5ccc6c03..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ /dev/null @@ -1,123 +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.math.BigDecimal; -import java.nio.ByteBuffer; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.ByteBuffers; -import org.apache.iceberg.util.DateTimeUtil; - -public class RowDataUtil { - - private RowDataUtil() {} - - public static Object convertConstant(Type type, Object value) { - if (value == null) { - return null; - } - - switch (type.typeId()) { - case DECIMAL: // DecimalData - Types.DecimalType decimal = (Types.DecimalType) type; - return DecimalData.fromBigDecimal((BigDecimal) value, decimal.precision(), decimal.scale()); - case STRING: // StringData - if (value instanceof Utf8) { - Utf8 utf8 = (Utf8) value; - return StringData.fromBytes(utf8.getBytes(), 0, utf8.getByteLength()); - } - return StringData.fromString(value.toString()); - case FIXED: // byte[] - if (value instanceof byte[]) { - return value; - } else if (value instanceof GenericData.Fixed) { - return ((GenericData.Fixed) value).bytes(); - } - return ByteBuffers.toByteArray((ByteBuffer) value); - case BINARY: // byte[] - return ByteBuffers.toByteArray((ByteBuffer) value); - case TIME: // int mills instead of long - return (int) ((Long) value / 1000); - case TIMESTAMP: // TimestampData - return TimestampData.fromLocalDateTime(DateTimeUtil.timestampFromMicros((Long) value)); - default: - } - return value; - } - - /** - * Similar to the private {@link RowDataSerializer#copyRowData(RowData, RowData)} method. This - * skips the check the arity of rowType and from, because the from RowData may contains additional - * column for position deletes. Using {@link RowDataSerializer#copy(RowData, RowData)} will fail - * the arity check. - */ - public static RowData clone( - RowData from, - RowData reuse, - RowType rowType, - TypeSerializer[] fieldSerializers, - RowData.FieldGetter[] fieldGetters) { - GenericRowData ret; - if (reuse instanceof GenericRowData) { - ret = (GenericRowData) reuse; - } else { - ret = new GenericRowData(from.getArity()); - } - - ret.setRowKind(from.getRowKind()); - for (int i = 0; i < rowType.getFieldCount(); i++) { - if (!from.isNullAt(i)) { - ret.setField(i, fieldSerializers[i].copy(fieldGetters[i].getFieldOrNull(from))); - } else { - ret.setField(i, null); - } - } - - return ret; - } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java deleted file mode 100644 index 1019285018d0..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ /dev/null @@ -1,300 +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.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.UUID; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RawValueData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -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.ByteBuffers; - -@Internal -public class StructRowData implements RowData { - private final Types.StructType type; - private RowKind kind; - private StructLike struct; - - public StructRowData(Types.StructType type) { - this(type, RowKind.INSERT); - } - - public StructRowData(Types.StructType type, RowKind kind) { - this(type, null, kind); - } - - private StructRowData(Types.StructType type, StructLike struct) { - this(type, struct, RowKind.INSERT); - } - - private StructRowData(Types.StructType type, StructLike struct, RowKind kind) { - this.type = type; - this.struct = struct; - this.kind = kind; - } - - public StructRowData setStruct(StructLike newStruct) { - this.struct = newStruct; - return this; - } - - @Override - public int getArity() { - return struct.size(); - } - - @Override - public RowKind getRowKind() { - return kind; - } - - @Override - public void setRowKind(RowKind newKind) { - Preconditions.checkNotNull(newKind, "kind can not be null"); - this.kind = newKind; - } - - @Override - public boolean isNullAt(int pos) { - return struct.get(pos, Object.class) == null; - } - - @Override - public boolean getBoolean(int pos) { - return struct.get(pos, Boolean.class); - } - - @Override - public byte getByte(int pos) { - return (byte) (int) struct.get(pos, Integer.class); - } - - @Override - public short getShort(int pos) { - return (short) (int) struct.get(pos, Integer.class); - } - - @Override - public int getInt(int pos) { - Object integer = struct.get(pos, Object.class); - - if (integer instanceof Integer) { - return (int) integer; - } else if (integer instanceof LocalDate) { - return (int) ((LocalDate) integer).toEpochDay(); - } else if (integer instanceof LocalTime) { - return (int) (((LocalTime) integer).toNanoOfDay() / 1000_000); - } else { - throw new IllegalStateException( - "Unknown type for int field. Type name: " + integer.getClass().getName()); - } - } - - @Override - public long getLong(int pos) { - Object longVal = struct.get(pos, Object.class); - - if (longVal instanceof Long) { - return (long) longVal; - } else if (longVal instanceof OffsetDateTime) { - return Duration.between(Instant.EPOCH, (OffsetDateTime) longVal).toNanos() / 1000; - } else if (longVal instanceof LocalDate) { - return ((LocalDate) longVal).toEpochDay(); - } else if (longVal instanceof LocalTime) { - return ((LocalTime) longVal).toNanoOfDay(); - } else if (longVal instanceof LocalDateTime) { - return Duration.between(Instant.EPOCH, ((LocalDateTime) longVal).atOffset(ZoneOffset.UTC)) - .toNanos() - / 1000; - } else { - throw new IllegalStateException( - "Unknown type for long field. Type name: " + longVal.getClass().getName()); - } - } - - @Override - public float getFloat(int pos) { - return struct.get(pos, Float.class); - } - - @Override - public double getDouble(int pos) { - return struct.get(pos, Double.class); - } - - @Override - public StringData getString(int pos) { - return isNullAt(pos) ? null : getStringDataInternal(pos); - } - - private StringData getStringDataInternal(int pos) { - CharSequence seq = struct.get(pos, CharSequence.class); - return StringData.fromString(seq.toString()); - } - - @Override - public DecimalData getDecimal(int pos, int precision, int scale) { - return isNullAt(pos) - ? null - : DecimalData.fromBigDecimal(getDecimalInternal(pos), precision, scale); - } - - private BigDecimal getDecimalInternal(int pos) { - return struct.get(pos, BigDecimal.class); - } - - @Override - public TimestampData getTimestamp(int pos, int precision) { - long timeLong = getLong(pos); - return TimestampData.fromEpochMillis(timeLong / 1000, (int) (timeLong % 1000) * 1000); - } - - @Override - public RawValueData getRawValue(int pos) { - throw new UnsupportedOperationException("Not supported yet."); - } - - @Override - public byte[] getBinary(int pos) { - return isNullAt(pos) ? null : getBinaryInternal(pos); - } - - private byte[] getBinaryInternal(int pos) { - Object bytes = struct.get(pos, Object.class); - - // should only be either ByteBuffer or byte[] - if (bytes instanceof ByteBuffer) { - return ByteBuffers.toByteArray((ByteBuffer) bytes); - } else if (bytes instanceof byte[]) { - return (byte[]) bytes; - } else if (bytes instanceof UUID) { - UUID uuid = (UUID) bytes; - ByteBuffer bb = ByteBuffer.allocate(16); - bb.putLong(uuid.getMostSignificantBits()); - bb.putLong(uuid.getLeastSignificantBits()); - return bb.array(); - } else { - throw new IllegalStateException( - "Unknown type for binary field. Type name: " + bytes.getClass().getName()); - } - } - - @Override - public ArrayData getArray(int pos) { - return isNullAt(pos) - ? null - : (ArrayData) - convertValue(type.fields().get(pos).type().asListType(), struct.get(pos, List.class)); - } - - @Override - public MapData getMap(int pos) { - return isNullAt(pos) - ? null - : (MapData) - convertValue(type.fields().get(pos).type().asMapType(), struct.get(pos, Map.class)); - } - - @Override - public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); - } - - private StructRowData getStructRowData(int pos, int numFields) { - return new StructRowData( - type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); - } - - private Object convertValue(Type elementType, Object value) { - switch (elementType.typeId()) { - case BOOLEAN: - case INTEGER: - case DATE: - case TIME: - case LONG: - case FLOAT: - case DOUBLE: - case DECIMAL: - return value; - case TIMESTAMP: - long millisecond = (long) value / 1000; - int nanoOfMillisecond = (int) ((Long) value % 1000) * 1000; - return TimestampData.fromEpochMillis(millisecond, nanoOfMillisecond); - case STRING: - return StringData.fromString(value.toString()); - case FIXED: - case BINARY: - return ByteBuffers.toByteArray((ByteBuffer) value); - case STRUCT: - return new StructRowData(elementType.asStructType(), (StructLike) value); - case LIST: - List list = (List) value; - Object[] array = new Object[list.size()]; - - int index = 0; - for (Object element : list) { - if (element == null) { - array[index] = null; - } else { - array[index] = convertValue(elementType.asListType().elementType(), element); - } - - index += 1; - } - return new GenericArrayData(array); - case MAP: - Types.MapType mapType = elementType.asMapType(); - Set> entries = ((Map) value).entrySet(); - Map result = Maps.newHashMap(); - for (Map.Entry entry : entries) { - final Object keyValue = convertValue(mapType.keyType(), entry.getKey()); - final Object valueValue = convertValue(mapType.valueType(), entry.getValue()); - result.put(keyValue, valueValue); - } - - return new GenericMapData(result); - default: - throw new UnsupportedOperationException("Unsupported element type: " + elementType); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java deleted file mode 100644 index f7e8e0c884cf..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/AvroGenericRecordToRowDataMapper.java +++ /dev/null @@ -1,61 +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.sink; - -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.formats.avro.AvroToRowDataConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.avro.AvroSchemaUtil; - -/** - * This util class converts Avro GenericRecord to Flink RowData.
    - *
    - * Internally it uses Flink {@link AvroToRowDataConverters}. Because of the precision difference - * between how Iceberg schema (micro) and Flink {@link AvroToRowDataConverters} (milli) deal with - * time type, we can't directly use the Avro Schema converted from Iceberg schema via {@link - * AvroSchemaUtil#convert(org.apache.iceberg.Schema, String)}. - */ -public class AvroGenericRecordToRowDataMapper implements MapFunction { - - private final AvroToRowDataConverters.AvroToRowDataConverter converter; - - AvroGenericRecordToRowDataMapper(RowType rowType) { - this.converter = AvroToRowDataConverters.createRowConverter(rowType); - } - - @Override - public RowData map(GenericRecord genericRecord) throws Exception { - return (RowData) converter.convert(genericRecord); - } - - /** Create a mapper based on Avro schema. */ - public static AvroGenericRecordToRowDataMapper forAvroSchema(Schema avroSchema) { - DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); - return new AvroGenericRecordToRowDataMapper(rowType); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java deleted file mode 100644 index e8a46c5becd7..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BaseDeltaTaskWriter.java +++ /dev/null @@ -1,126 +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.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.deletes.DeleteGranularity; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.RowDataProjection; -import org.apache.iceberg.io.BaseTaskWriter; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; - -abstract class BaseDeltaTaskWriter extends BaseTaskWriter { - - private final Schema schema; - private final Schema deleteSchema; - private final RowDataWrapper wrapper; - private final RowDataWrapper keyWrapper; - private final RowDataProjection keyProjection; - private final boolean upsert; - - BaseDeltaTaskWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.schema = schema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); - this.wrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - this.keyWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(deleteSchema), deleteSchema.asStruct()); - this.keyProjection = - RowDataProjection.create(flinkSchema, schema.asStruct(), deleteSchema.asStruct()); - this.upsert = upsert; - } - - abstract RowDataDeltaWriter route(RowData row); - - RowDataWrapper wrapper() { - return wrapper; - } - - @Override - public void write(RowData row) throws IOException { - RowDataDeltaWriter writer = route(row); - - switch (row.getRowKind()) { - case INSERT: - case UPDATE_AFTER: - if (upsert) { - writer.deleteKey(keyProjection.wrap(row)); - } - writer.write(row); - break; - - case UPDATE_BEFORE: - if (upsert) { - break; // UPDATE_BEFORE is not necessary for UPSERT, we do nothing to prevent delete one - // row twice - } - writer.delete(row); - break; - case DELETE: - if (upsert) { - writer.deleteKey(keyProjection.wrap(row)); - } else { - writer.delete(row); - } - break; - - default: - throw new UnsupportedOperationException("Unknown row kind: " + row.getRowKind()); - } - } - - protected class RowDataDeltaWriter extends BaseEqualityDeltaWriter { - RowDataDeltaWriter(PartitionKey partition) { - super(partition, schema, deleteSchema, DeleteGranularity.FILE); - } - - @Override - protected StructLike asStructLike(RowData data) { - return wrapper.wrap(data); - } - - @Override - protected StructLike asStructLikeKey(RowData data) { - return keyWrapper.wrap(data); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.java deleted file mode 100644 index 1cb6e013bd2c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionKeySelector.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.flink.sink; - -import java.util.stream.IntStream; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; - -/** - * A {@link KeySelector} that extracts the bucketId from a data row's bucket partition as the key. - * To be used with the {@link BucketPartitioner}. - */ -class BucketPartitionKeySelector implements KeySelector { - - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - private final int bucketFieldPosition; - - private transient RowDataWrapper rowDataWrapper; - - BucketPartitionKeySelector(PartitionSpec partitionSpec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(partitionSpec, schema); - this.flinkSchema = flinkSchema; - this.bucketFieldPosition = getBucketFieldPosition(partitionSpec); - } - - private int getBucketFieldPosition(PartitionSpec partitionSpec) { - int bucketFieldId = BucketPartitionerUtil.getBucketFieldId(partitionSpec); - return IntStream.range(0, partitionSpec.fields().size()) - .filter(i -> partitionSpec.fields().get(i).fieldId() == bucketFieldId) - .toArray()[0]; - } - - private RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - return rowDataWrapper; - } - - @Override - public Integer getKey(RowData rowData) { - partitionKey.partition(lazyRowDataWrapper().wrap(rowData)); - return partitionKey.get(bucketFieldPosition, Integer.class); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java deleted file mode 100644 index 9c9a117906e2..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitioner.java +++ /dev/null @@ -1,103 +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.sink; - -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * This partitioner will redirect records to writers deterministically based on the Bucket partition - * spec. It'll attempt to optimize the file size written depending on whether numPartitions is - * greater, less or equal than the maxNumBuckets. Note: The current implementation only supports ONE - * bucket in the partition spec. - */ -class BucketPartitioner implements Partitioner { - - static final String BUCKET_NULL_MESSAGE = "bucketId cannot be null"; - static final String BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE = - "Invalid bucket ID %s: must be non-negative."; - static final String BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE = - "Invalid bucket ID %s: must be less than bucket limit: %s."; - - private final int maxNumBuckets; - - // To hold the OFFSET of the next writer to use for any bucket, only used when writers > the - // number of buckets - private final int[] currentBucketWriterOffset; - - BucketPartitioner(PartitionSpec partitionSpec) { - this.maxNumBuckets = BucketPartitionerUtil.getMaxNumBuckets(partitionSpec); - this.currentBucketWriterOffset = new int[maxNumBuckets]; - } - - /** - * Determine the partition id based on the following criteria: If the number of writers <= the - * number of buckets, an evenly distributed number of buckets will be assigned to each writer (one - * writer -> many buckets). Conversely, if the number of writers > the number of buckets the logic - * is handled by the {@link #getPartitionWithMoreWritersThanBuckets - * getPartitionWritersGreaterThanBuckets} method. - * - * @param bucketId the bucketId for each request - * @param numPartitions the total number of partitions - * @return the partition id (writer) to use for each request - */ - @Override - public int partition(Integer bucketId, int numPartitions) { - Preconditions.checkNotNull(bucketId, BUCKET_NULL_MESSAGE); - Preconditions.checkArgument(bucketId >= 0, BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, bucketId); - Preconditions.checkArgument( - bucketId < maxNumBuckets, BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, bucketId, maxNumBuckets); - - if (numPartitions <= maxNumBuckets) { - return bucketId % numPartitions; - } else { - return getPartitionWithMoreWritersThanBuckets(bucketId, numPartitions); - } - } - - /*- - * If the number of writers > the number of buckets each partitioner will keep a state of multiple - * writers per bucket as evenly as possible, and will round-robin the requests across them, in this - * case each writer will target only one bucket at all times (many writers -> one bucket). Example: - * Configuration: numPartitions (writers) = 5, maxBuckets = 2 - * Expected behavior: - * - Records for Bucket 0 will be "round robin" between Writers 0, 2 and 4 - * - Records for Bucket 1 will always use Writer 1 and 3 - * Notes: - * - maxNumWritersPerBucket determines when to reset the currentBucketWriterOffset to 0 for this bucketId - * - When numPartitions is not evenly divisible by maxBuckets, some buckets will have one more writer (extraWriter). - * In this example Bucket 0 has an "extra writer" to consider before resetting its offset to 0. - * - * @return the destination partition index (writer subtask id) - */ - private int getPartitionWithMoreWritersThanBuckets(int bucketId, int numPartitions) { - int currentOffset = currentBucketWriterOffset[bucketId]; - // Determine if this bucket requires an "extra writer" - int extraWriter = bucketId < (numPartitions % maxNumBuckets) ? 1 : 0; - // The max number of writers this bucket can have - int maxNumWritersPerBucket = (numPartitions / maxNumBuckets) + extraWriter; - - // Increment the writer offset or reset if it's reached the max for this bucket - int nextOffset = currentOffset == maxNumWritersPerBucket - 1 ? 0 : currentOffset + 1; - currentBucketWriterOffset[bucketId] = nextOffset; - - return bucketId + (maxNumBuckets * currentOffset); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java deleted file mode 100644 index c33207728d3e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/BucketPartitionerUtil.java +++ /dev/null @@ -1,125 +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.sink; - -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import org.apache.flink.api.java.tuple.Tuple2; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.transforms.PartitionSpecVisitor; - -final class BucketPartitionerUtil { - static final String BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE = - "Invalid number of buckets: %s (must be 1)"; - - private BucketPartitionerUtil() {} - - /** - * Determines whether the PartitionSpec has one and only one Bucket definition - * - * @param partitionSpec the partition spec in question - * @return whether the PartitionSpec has only one Bucket - */ - static boolean hasOneBucketField(PartitionSpec partitionSpec) { - List> bucketFields = getBucketFields(partitionSpec); - return bucketFields != null && bucketFields.size() == 1; - } - - /** - * Extracts the Bucket definition from a PartitionSpec. - * - * @param partitionSpec the partition spec in question - * @return the Bucket definition in the form of a tuple (fieldId, maxNumBuckets) - */ - private static Tuple2 getBucketFieldInfo(PartitionSpec partitionSpec) { - List> bucketFields = getBucketFields(partitionSpec); - Preconditions.checkArgument( - bucketFields.size() == 1, - BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, - bucketFields.size()); - return bucketFields.get(0); - } - - static int getBucketFieldId(PartitionSpec partitionSpec) { - return getBucketFieldInfo(partitionSpec).f0; - } - - static int getMaxNumBuckets(PartitionSpec partitionSpec) { - return getBucketFieldInfo(partitionSpec).f1; - } - - private static List> getBucketFields(PartitionSpec spec) { - return PartitionSpecVisitor.visit(spec, new BucketPartitionSpecVisitor()).stream() - .filter(Objects::nonNull) - .collect(Collectors.toList()); - } - - private static class BucketPartitionSpecVisitor - implements PartitionSpecVisitor> { - @Override - public Tuple2 identity(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 bucket( - int fieldId, String sourceName, int sourceId, int numBuckets) { - return new Tuple2<>(fieldId, numBuckets); - } - - @Override - public Tuple2 truncate( - int fieldId, String sourceName, int sourceId, int width) { - return null; - } - - @Override - public Tuple2 year(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 month(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 day(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 hour(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 alwaysNull(int fieldId, String sourceName, int sourceId) { - return null; - } - - @Override - public Tuple2 unknown( - int fieldId, String sourceName, int sourceId, String transform) { - return null; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java deleted file mode 100644 index e9f9786f9190..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CachingTableSupplier.java +++ /dev/null @@ -1,91 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink; - -import java.time.Duration; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.SerializableSupplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * A table loader that will only reload a table after a certain interval has passed. WARNING: This - * table loader should be used carefully when used with writer tasks. It could result in heavy load - * on a catalog for jobs with many writers. - */ -class CachingTableSupplier implements SerializableSupplier

  • { - - private static final Logger LOG = LoggerFactory.getLogger(CachingTableSupplier.class); - - private final Table initialTable; - private final TableLoader tableLoader; - private final Duration tableRefreshInterval; - private long lastLoadTimeMillis; - private transient Table table; - - CachingTableSupplier( - SerializableTable initialTable, TableLoader tableLoader, Duration tableRefreshInterval) { - Preconditions.checkArgument(initialTable != null, "initialTable cannot be null"); - Preconditions.checkArgument(tableLoader != null, "tableLoader cannot be null"); - Preconditions.checkArgument( - tableRefreshInterval != null, "tableRefreshInterval cannot be null"); - this.initialTable = initialTable; - this.table = initialTable; - this.tableLoader = tableLoader; - this.tableRefreshInterval = tableRefreshInterval; - this.lastLoadTimeMillis = System.currentTimeMillis(); - } - - @Override - public Table get() { - if (table == null) { - this.table = initialTable; - } - return table; - } - - Table initialTable() { - return initialTable; - } - - void refreshTable() { - if (System.currentTimeMillis() > lastLoadTimeMillis + tableRefreshInterval.toMillis()) { - try { - if (!tableLoader.isOpen()) { - tableLoader.open(); - } - - this.table = tableLoader.loadTable(); - this.lastLoadTimeMillis = System.currentTimeMillis(); - - LOG.info( - "Table {} reloaded, next min load time threshold is {}", - table.name(), - DateTimeUtil.formatTimestampMillis( - lastLoadTimeMillis + tableRefreshInterval.toMillis())); - } catch (Exception e) { - LOG.warn("An error occurred reloading table {}, table was not reloaded", table.name(), e); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java deleted file mode 100644 index 9a2f57181708..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/CommitSummary.java +++ /dev/null @@ -1,93 +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.sink; - -import java.util.Arrays; -import java.util.NavigableMap; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -class CommitSummary { - - private final AtomicLong dataFilesCount = new AtomicLong(); - private final AtomicLong dataFilesRecordCount = new AtomicLong(); - private final AtomicLong dataFilesByteCount = new AtomicLong(); - private final AtomicLong deleteFilesCount = new AtomicLong(); - private final AtomicLong deleteFilesRecordCount = new AtomicLong(); - private final AtomicLong deleteFilesByteCount = new AtomicLong(); - - CommitSummary(NavigableMap pendingResults) { - pendingResults - .values() - .forEach( - writeResult -> { - dataFilesCount.addAndGet(writeResult.dataFiles().length); - Arrays.stream(writeResult.dataFiles()) - .forEach( - dataFile -> { - dataFilesRecordCount.addAndGet(dataFile.recordCount()); - dataFilesByteCount.addAndGet(dataFile.fileSizeInBytes()); - }); - deleteFilesCount.addAndGet(writeResult.deleteFiles().length); - Arrays.stream(writeResult.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesRecordCount.addAndGet(deleteFile.recordCount()); - deleteFilesByteCount.addAndGet(deleteFile.fileSizeInBytes()); - }); - }); - } - - long dataFilesCount() { - return dataFilesCount.get(); - } - - long dataFilesRecordCount() { - return dataFilesRecordCount.get(); - } - - long dataFilesByteCount() { - return dataFilesByteCount.get(); - } - - long deleteFilesCount() { - return deleteFilesCount.get(); - } - - long deleteFilesRecordCount() { - return deleteFilesRecordCount.get(); - } - - long deleteFilesByteCount() { - return deleteFilesByteCount.get(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("dataFilesCount", dataFilesCount) - .add("dataFilesRecordCount", dataFilesRecordCount) - .add("dataFilesByteCount", dataFilesByteCount) - .add("deleteFilesCount", deleteFilesCount) - .add("deleteFilesRecordCount", deleteFilesRecordCount) - .add("deleteFilesByteCount", deleteFilesByteCount) - .toString(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.java deleted file mode 100644 index 036970c06d5b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifests.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.flink.sink; - -import java.util.List; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class DeltaManifests { - - private static final CharSequence[] EMPTY_REF_DATA_FILES = new CharSequence[0]; - - private final ManifestFile dataManifest; - private final ManifestFile deleteManifest; - private final CharSequence[] referencedDataFiles; - - DeltaManifests(ManifestFile dataManifest, ManifestFile deleteManifest) { - this(dataManifest, deleteManifest, EMPTY_REF_DATA_FILES); - } - - DeltaManifests( - ManifestFile dataManifest, ManifestFile deleteManifest, CharSequence[] referencedDataFiles) { - Preconditions.checkNotNull(referencedDataFiles, "Referenced data files shouldn't be null."); - - this.dataManifest = dataManifest; - this.deleteManifest = deleteManifest; - this.referencedDataFiles = referencedDataFiles; - } - - ManifestFile dataManifest() { - return dataManifest; - } - - ManifestFile deleteManifest() { - return deleteManifest; - } - - CharSequence[] referencedDataFiles() { - return referencedDataFiles; - } - - List manifests() { - List manifests = Lists.newArrayListWithCapacity(2); - if (dataManifest != null) { - manifests.add(dataManifest); - } - - if (deleteManifest != null) { - manifests.add(deleteManifest); - } - - return manifests; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java deleted file mode 100644 index 92ca284b12ba..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/DeltaManifestsSerializer.java +++ /dev/null @@ -1,122 +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.sink; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.DataInputStream; -import java.io.DataOutputStream; -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class DeltaManifestsSerializer implements SimpleVersionedSerializer { - private static final int VERSION_1 = 1; - private static final int VERSION_2 = 2; - private static final byte[] EMPTY_BINARY = new byte[0]; - - static final DeltaManifestsSerializer INSTANCE = new DeltaManifestsSerializer(); - - @Override - public int getVersion() { - return VERSION_2; - } - - @Override - public byte[] serialize(DeltaManifests deltaManifests) throws IOException { - Preconditions.checkNotNull( - deltaManifests, "DeltaManifests to be serialized should not be null"); - - ByteArrayOutputStream binaryOut = new ByteArrayOutputStream(); - DataOutputStream out = new DataOutputStream(binaryOut); - - byte[] dataManifestBinary = EMPTY_BINARY; - if (deltaManifests.dataManifest() != null) { - dataManifestBinary = ManifestFiles.encode(deltaManifests.dataManifest()); - } - - out.writeInt(dataManifestBinary.length); - out.write(dataManifestBinary); - - byte[] deleteManifestBinary = EMPTY_BINARY; - if (deltaManifests.deleteManifest() != null) { - deleteManifestBinary = ManifestFiles.encode(deltaManifests.deleteManifest()); - } - - out.writeInt(deleteManifestBinary.length); - out.write(deleteManifestBinary); - - CharSequence[] referencedDataFiles = deltaManifests.referencedDataFiles(); - out.writeInt(referencedDataFiles.length); - for (CharSequence referencedDataFile : referencedDataFiles) { - out.writeUTF(referencedDataFile.toString()); - } - - return binaryOut.toByteArray(); - } - - @Override - public DeltaManifests deserialize(int version, byte[] serialized) throws IOException { - if (version == VERSION_1) { - return deserializeV1(serialized); - } else if (version == VERSION_2) { - return deserializeV2(serialized); - } else { - throw new RuntimeException("Unknown serialize version: " + version); - } - } - - private DeltaManifests deserializeV1(byte[] serialized) throws IOException { - return new DeltaManifests(ManifestFiles.decode(serialized), null); - } - - private DeltaManifests deserializeV2(byte[] serialized) throws IOException { - ManifestFile dataManifest = null; - ManifestFile deleteManifest = null; - - ByteArrayInputStream binaryIn = new ByteArrayInputStream(serialized); - DataInputStream in = new DataInputStream(binaryIn); - - int dataManifestSize = in.readInt(); - if (dataManifestSize > 0) { - byte[] dataManifestBinary = new byte[dataManifestSize]; - Preconditions.checkState(in.read(dataManifestBinary) == dataManifestSize); - - dataManifest = ManifestFiles.decode(dataManifestBinary); - } - - int deleteManifestSize = in.readInt(); - if (deleteManifestSize > 0) { - byte[] deleteManifestBinary = new byte[deleteManifestSize]; - Preconditions.checkState(in.read(deleteManifestBinary) == deleteManifestSize); - - deleteManifest = ManifestFiles.decode(deleteManifestBinary); - } - - int referenceDataFileNum = in.readInt(); - CharSequence[] referencedDataFiles = new CharSequence[referenceDataFileNum]; - for (int i = 0; i < referenceDataFileNum; i++) { - referencedDataFiles[i] = in.readUTF(); - } - - return new DeltaManifests(dataManifest, deleteManifest, referencedDataFiles); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java deleted file mode 100644 index 18b269d6c3e9..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/EqualityFieldKeySelector.java +++ /dev/null @@ -1,86 +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.sink; - -import java.util.List; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.StructLikeWrapper; -import org.apache.iceberg.util.StructProjection; - -/** - * Create a {@link KeySelector} to shuffle by equality fields, to ensure same equality fields record - * will be emitted to same writer in order. - */ -class EqualityFieldKeySelector implements KeySelector { - - private final Schema schema; - private final RowType flinkSchema; - private final Schema deleteSchema; - - private transient RowDataWrapper rowDataWrapper; - private transient StructProjection structProjection; - private transient StructLikeWrapper structLikeWrapper; - - EqualityFieldKeySelector(Schema schema, RowType flinkSchema, List equalityFieldIds) { - this.schema = schema; - this.flinkSchema = flinkSchema; - this.deleteSchema = TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)); - } - - /** - * Construct the {@link RowDataWrapper} lazily here because few members in it are not - * serializable. In this way, we don't have to serialize them with forcing. - */ - protected RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - return rowDataWrapper; - } - - /** Construct the {@link StructProjection} lazily because it is not serializable. */ - protected StructProjection lazyStructProjection() { - if (structProjection == null) { - structProjection = StructProjection.create(schema, deleteSchema); - } - return structProjection; - } - - /** Construct the {@link StructLikeWrapper} lazily because it is not serializable. */ - protected StructLikeWrapper lazyStructLikeWrapper() { - if (structLikeWrapper == null) { - structLikeWrapper = StructLikeWrapper.forType(deleteSchema.asStruct()); - } - return structLikeWrapper; - } - - @Override - public Integer getKey(RowData row) { - RowDataWrapper wrappedRowData = lazyRowDataWrapper().wrap(row); - StructProjection projectedRowData = lazyStructProjection().wrap(wrappedRowData); - StructLikeWrapper wrapper = lazyStructLikeWrapper().set(projectedRowData); - return wrapper.hashCode(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java deleted file mode 100644 index eacef58a8d5d..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkAppenderFactory.java +++ /dev/null @@ -1,280 +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.sink; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -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.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -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; - -public class FlinkAppenderFactory implements FileAppenderFactory, Serializable { - private final Schema schema; - private final RowType flinkSchema; - private final Map props; - private final PartitionSpec spec; - private final int[] equalityFieldIds; - private final Schema eqDeleteRowSchema; - private final Schema posDeleteRowSchema; - private final Table table; - - private RowType eqDeleteFlinkSchema = null; - private RowType posDeleteFlinkSchema = null; - - public FlinkAppenderFactory( - Table table, - Schema schema, - RowType flinkSchema, - Map props, - PartitionSpec spec, - int[] equalityFieldIds, - Schema eqDeleteRowSchema, - Schema posDeleteRowSchema) { - Preconditions.checkNotNull(table, "Table shouldn't be null"); - this.table = table; - this.schema = schema; - this.flinkSchema = flinkSchema; - this.props = props; - this.spec = spec; - this.equalityFieldIds = equalityFieldIds; - this.eqDeleteRowSchema = eqDeleteRowSchema; - this.posDeleteRowSchema = posDeleteRowSchema; - } - - private RowType lazyEqDeleteFlinkSchema() { - if (eqDeleteFlinkSchema == null) { - Preconditions.checkNotNull(eqDeleteRowSchema, "Equality delete row schema shouldn't be null"); - this.eqDeleteFlinkSchema = FlinkSchemaUtil.convert(eqDeleteRowSchema); - } - return eqDeleteFlinkSchema; - } - - private RowType lazyPosDeleteFlinkSchema() { - if (posDeleteFlinkSchema == null) { - Preconditions.checkNotNull(posDeleteRowSchema, "Pos-delete row schema shouldn't be null"); - this.posDeleteFlinkSchema = FlinkSchemaUtil.convert(posDeleteRowSchema); - } - return this.posDeleteFlinkSchema; - } - - @Override - public FileAppender newAppender(OutputFile outputFile, FileFormat format) { - return newAppender(EncryptionUtil.plainAsEncryptedOutput(outputFile), format); - } - - @Override - public FileAppender newAppender(EncryptedOutputFile outputFile, FileFormat format) { - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - try { - switch (format) { - case AVRO: - return Avro.write(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .setAll(props) - .schema(schema) - .metricsConfig(metricsConfig) - .overwrite() - .build(); - - case ORC: - return ORC.write(outputFile) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - case PARQUET: - return Parquet.write(outputFile) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(flinkSchema, msgType)) - .setAll(props) - .metricsConfig(metricsConfig) - .schema(schema) - .overwrite() - .build(); - - default: - throw new UnsupportedOperationException("Cannot write unknown file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(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 outputFile, 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"); - - MetricsConfig metricsConfig = MetricsConfig.forTable(table); - try { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(lazyEqDeleteFlinkSchema())) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case ORC: - return ORC.writeDeletes(outputFile) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(flinkSchema, iSchema)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - case PARQUET: - return Parquet.writeDeletes(outputFile) - .createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(lazyEqDeleteFlinkSchema(), msgType)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(eqDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .equalityFieldIds(equalityFieldIds) - .buildEqualityWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write equality-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - @Override - public PositionDeleteWriter newPosDeleteWriter( - EncryptedOutputFile outputFile, FileFormat format, StructLike partition) { - MetricsConfig metricsConfig = MetricsConfig.forPositionDelete(table); - try { - switch (format) { - case AVRO: - return Avro.writeDeletes(outputFile) - .createWriterFunc(ignore -> new FlinkAvroWriter(lazyPosDeleteFlinkSchema())) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .buildPositionWriter(); - - case ORC: - RowType orcPosDeleteSchema = - FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return ORC.writeDeletes(outputFile) - .createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(orcPosDeleteSchema, iSchema)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .transformPaths(path -> StringData.fromString(path.toString())) - .buildPositionWriter(); - - case PARQUET: - RowType flinkPosDeleteSchema = - FlinkSchemaUtil.convert(DeleteSchemaUtil.posDeleteSchema(posDeleteRowSchema)); - return Parquet.writeDeletes(outputFile) - .createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(flinkPosDeleteSchema, msgType)) - .withPartition(partition) - .overwrite() - .setAll(props) - .metricsConfig(metricsConfig) - .rowSchema(posDeleteRowSchema) - .withSpec(spec) - .withKeyMetadata(outputFile.keyMetadata()) - .transformPaths(path -> StringData.fromString(path.toString())) - .buildPositionWriter(); - - default: - throw new UnsupportedOperationException( - "Cannot write pos-deletes for unsupported file format: " + format); - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java deleted file mode 100644 index 2183fe062af4..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkFileWriterFactory.java +++ /dev/null @@ -1,293 +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.sink; - -import static org.apache.iceberg.MetadataColumns.DELETE_FILE_ROW_FIELD_NAME; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; -import static org.apache.iceberg.TableProperties.DELETE_DEFAULT_FILE_FORMAT; - -import java.io.Serializable; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.BaseFileWriterFactory; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.data.FlinkAvroWriter; -import org.apache.iceberg.flink.data.FlinkOrcWriter; -import org.apache.iceberg.flink.data.FlinkParquetWriters; -import org.apache.iceberg.io.DeleteSchemaUtil; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class FlinkFileWriterFactory extends BaseFileWriterFactory implements Serializable { - private RowType dataFlinkType; - private RowType equalityDeleteFlinkType; - private RowType positionDeleteFlinkType; - - FlinkFileWriterFactory( - Table table, - FileFormat dataFileFormat, - Schema dataSchema, - RowType dataFlinkType, - SortOrder dataSortOrder, - FileFormat deleteFileFormat, - int[] equalityFieldIds, - Schema equalityDeleteRowSchema, - RowType equalityDeleteFlinkType, - SortOrder equalityDeleteSortOrder, - Schema positionDeleteRowSchema, - RowType positionDeleteFlinkType) { - - super( - table, - dataFileFormat, - dataSchema, - dataSortOrder, - deleteFileFormat, - equalityFieldIds, - equalityDeleteRowSchema, - equalityDeleteSortOrder, - positionDeleteRowSchema); - - this.dataFlinkType = dataFlinkType; - this.equalityDeleteFlinkType = equalityDeleteFlinkType; - this.positionDeleteFlinkType = positionDeleteFlinkType; - } - - static Builder builderFor(Table table) { - return new Builder(table); - } - - @Override - protected void configureDataWrite(Avro.DataWriteBuilder builder) { - builder.createWriterFunc(ignore -> new FlinkAvroWriter(dataFlinkType())); - } - - @Override - protected void configureEqualityDelete(Avro.DeleteWriteBuilder builder) { - builder.createWriterFunc(ignored -> new FlinkAvroWriter(equalityDeleteFlinkType())); - } - - @Override - protected void configurePositionDelete(Avro.DeleteWriteBuilder builder) { - int rowFieldIndex = positionDeleteFlinkType().getFieldIndex(DELETE_FILE_ROW_FIELD_NAME); - if (rowFieldIndex >= 0) { - // FlinkAvroWriter accepts just the Flink type of the row ignoring the path and pos - RowType positionDeleteRowFlinkType = - (RowType) positionDeleteFlinkType().getTypeAt(rowFieldIndex); - builder.createWriterFunc(ignored -> new FlinkAvroWriter(positionDeleteRowFlinkType)); - } - } - - @Override - protected void configureDataWrite(Parquet.DataWriteBuilder builder) { - builder.createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(dataFlinkType(), msgType)); - } - - @Override - protected void configureEqualityDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(equalityDeleteFlinkType(), msgType)); - } - - @Override - protected void configurePositionDelete(Parquet.DeleteWriteBuilder builder) { - builder.createWriterFunc( - msgType -> FlinkParquetWriters.buildWriter(positionDeleteFlinkType(), msgType)); - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - @Override - protected void configureDataWrite(ORC.DataWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(dataFlinkType(), iSchema)); - } - - @Override - protected void configureEqualityDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(equalityDeleteFlinkType(), iSchema)); - } - - @Override - protected void configurePositionDelete(ORC.DeleteWriteBuilder builder) { - builder.createWriterFunc( - (iSchema, typDesc) -> FlinkOrcWriter.buildWriter(positionDeleteFlinkType(), iSchema)); - builder.transformPaths(path -> StringData.fromString(path.toString())); - } - - private RowType dataFlinkType() { - if (dataFlinkType == null) { - Preconditions.checkNotNull(dataSchema(), "Data schema must not be null"); - this.dataFlinkType = FlinkSchemaUtil.convert(dataSchema()); - } - - return dataFlinkType; - } - - private RowType equalityDeleteFlinkType() { - if (equalityDeleteFlinkType == null) { - Preconditions.checkNotNull( - equalityDeleteRowSchema(), "Equality delete schema must not be null"); - this.equalityDeleteFlinkType = FlinkSchemaUtil.convert(equalityDeleteRowSchema()); - } - - return equalityDeleteFlinkType; - } - - private RowType positionDeleteFlinkType() { - if (positionDeleteFlinkType == null) { - // wrap the optional row schema into the position delete schema that contains path and - // position - Schema positionDeleteSchema = DeleteSchemaUtil.posDeleteSchema(positionDeleteRowSchema()); - this.positionDeleteFlinkType = FlinkSchemaUtil.convert(positionDeleteSchema); - } - - return positionDeleteFlinkType; - } - - static class Builder { - private final Table table; - private FileFormat dataFileFormat; - private Schema dataSchema; - private RowType dataFlinkType; - private SortOrder dataSortOrder; - private FileFormat deleteFileFormat; - private int[] equalityFieldIds; - private Schema equalityDeleteRowSchema; - private RowType equalityDeleteFlinkType; - private SortOrder equalityDeleteSortOrder; - private Schema positionDeleteRowSchema; - private RowType positionDeleteFlinkType; - - Builder(Table table) { - this.table = table; - - Map properties = table.properties(); - - String dataFileFormatName = - properties.getOrDefault(DEFAULT_FILE_FORMAT, DEFAULT_FILE_FORMAT_DEFAULT); - this.dataFileFormat = FileFormat.fromString(dataFileFormatName); - - String deleteFileFormatName = - properties.getOrDefault(DELETE_DEFAULT_FILE_FORMAT, dataFileFormatName); - this.deleteFileFormat = FileFormat.fromString(deleteFileFormatName); - } - - Builder dataFileFormat(FileFormat newDataFileFormat) { - this.dataFileFormat = newDataFileFormat; - return this; - } - - Builder dataSchema(Schema newDataSchema) { - this.dataSchema = newDataSchema; - return this; - } - - /** - * Sets a Flink type for data. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder dataFlinkType(RowType newDataFlinkType) { - this.dataFlinkType = newDataFlinkType; - return this; - } - - Builder dataSortOrder(SortOrder newDataSortOrder) { - this.dataSortOrder = newDataSortOrder; - return this; - } - - Builder deleteFileFormat(FileFormat newDeleteFileFormat) { - this.deleteFileFormat = newDeleteFileFormat; - return this; - } - - Builder equalityFieldIds(int[] newEqualityFieldIds) { - this.equalityFieldIds = newEqualityFieldIds; - return this; - } - - Builder equalityDeleteRowSchema(Schema newEqualityDeleteRowSchema) { - this.equalityDeleteRowSchema = newEqualityDeleteRowSchema; - return this; - } - - /** - * Sets a Flink type for equality deletes. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder equalityDeleteFlinkType(RowType newEqualityDeleteFlinkType) { - this.equalityDeleteFlinkType = newEqualityDeleteFlinkType; - return this; - } - - Builder equalityDeleteSortOrder(SortOrder newEqualityDeleteSortOrder) { - this.equalityDeleteSortOrder = newEqualityDeleteSortOrder; - return this; - } - - Builder positionDeleteRowSchema(Schema newPositionDeleteRowSchema) { - this.positionDeleteRowSchema = newPositionDeleteRowSchema; - return this; - } - - /** - * Sets a Flink type for position deletes. - * - *

    If not set, the value is derived from the provided Iceberg schema. - */ - Builder positionDeleteFlinkType(RowType newPositionDeleteFlinkType) { - this.positionDeleteFlinkType = newPositionDeleteFlinkType; - return this; - } - - FlinkFileWriterFactory build() { - boolean noEqualityDeleteConf = equalityFieldIds == null && equalityDeleteRowSchema == null; - boolean fullEqualityDeleteConf = equalityFieldIds != null && equalityDeleteRowSchema != null; - Preconditions.checkArgument( - noEqualityDeleteConf || fullEqualityDeleteConf, - "Equality field IDs and equality delete row schema must be set together"); - - return new FlinkFileWriterFactory( - table, - dataFileFormat, - dataSchema, - dataFlinkType, - dataSortOrder, - deleteFileFormat, - equalityFieldIds, - equalityDeleteRowSchema, - equalityDeleteFlinkType, - equalityDeleteSortOrder, - positionDeleteRowSchema, - positionDeleteFlinkType); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java deleted file mode 100644 index c7e8a2dea7cb..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ /dev/null @@ -1,132 +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.sink; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class FlinkManifestUtil { - private static final int FORMAT_V2 = 2; - private static final Long DUMMY_SNAPSHOT_ID = 0L; - - private FlinkManifestUtil() {} - - static ManifestFile writeDataFiles( - OutputFile outputFile, PartitionSpec spec, List dataFiles) throws IOException { - ManifestWriter writer = - ManifestFiles.write(FORMAT_V2, spec, outputFile, DUMMY_SNAPSHOT_ID); - - try (ManifestWriter closeableWriter = writer) { - closeableWriter.addAll(dataFiles); - } - - return writer.toManifestFile(); - } - - static List readDataFiles( - ManifestFile manifestFile, FileIO io, Map specsById) - throws IOException { - try (CloseableIterable dataFiles = ManifestFiles.read(manifestFile, io, specsById)) { - return Lists.newArrayList(dataFiles); - } - } - - static ManifestOutputFileFactory createOutputFileFactory( - Supplier

    tableSupplier, - Map tableProps, - String flinkJobId, - String operatorUniqueId, - int subTaskId, - long attemptNumber) { - return new ManifestOutputFileFactory( - tableSupplier, tableProps, flinkJobId, operatorUniqueId, subTaskId, attemptNumber); - } - - /** - * Write the {@link WriteResult} to temporary manifest files. - * - * @param result all those DataFiles/DeleteFiles in this WriteResult should be written with same - * partition spec - */ - static DeltaManifests writeCompletedFiles( - WriteResult result, Supplier outputFileSupplier, PartitionSpec spec) - throws IOException { - - ManifestFile dataManifest = null; - ManifestFile deleteManifest = null; - - // Write the completed data files into a newly created data manifest file. - if (result.dataFiles() != null && result.dataFiles().length > 0) { - dataManifest = - writeDataFiles(outputFileSupplier.get(), spec, Lists.newArrayList(result.dataFiles())); - } - - // Write the completed delete files into a newly created delete manifest file. - if (result.deleteFiles() != null && result.deleteFiles().length > 0) { - OutputFile deleteManifestFile = outputFileSupplier.get(); - - ManifestWriter deleteManifestWriter = - ManifestFiles.writeDeleteManifest(FORMAT_V2, spec, deleteManifestFile, DUMMY_SNAPSHOT_ID); - try (ManifestWriter writer = deleteManifestWriter) { - for (DeleteFile deleteFile : result.deleteFiles()) { - writer.add(deleteFile); - } - } - - deleteManifest = deleteManifestWriter.toManifestFile(); - } - - return new DeltaManifests(dataManifest, deleteManifest, result.referencedDataFiles()); - } - - static WriteResult readCompletedFiles( - DeltaManifests deltaManifests, FileIO io, Map specsById) - throws IOException { - WriteResult.Builder builder = WriteResult.builder(); - - // Read the completed data files from persisted data manifest file. - if (deltaManifests.dataManifest() != null) { - builder.addDataFiles(readDataFiles(deltaManifests.dataManifest(), io, specsById)); - } - - // Read the completed delete files from persisted delete manifests file. - if (deltaManifests.deleteManifest() != null) { - try (CloseableIterable deleteFiles = - ManifestFiles.readDeleteManifest(deltaManifests.deleteManifest(), io, specsById)) { - builder.addDeleteFiles(deleteFiles); - } - } - - return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java deleted file mode 100644 index 769af7d77140..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ /dev/null @@ -1,654 +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.sink; - -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; -import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; -import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; -import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; -import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Function; -import org.apache.flink.api.common.functions.MapFunction; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.Configuration; -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.streaming.api.datastream.SingleOutputStreamOperator; -import org.apache.flink.streaming.api.functions.sink.DiscardingSink; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionField; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; -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.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.SerializableSupplier; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class FlinkSink { - private static final Logger LOG = LoggerFactory.getLogger(FlinkSink.class); - - private static final String ICEBERG_STREAM_WRITER_NAME = - IcebergStreamWriter.class.getSimpleName(); - private static final String ICEBERG_FILES_COMMITTER_NAME = - IcebergFilesCommitter.class.getSimpleName(); - - private FlinkSink() {} - - /** - * Initialize a {@link Builder} to export the data from generic input data stream into iceberg - * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper - * function and a {@link TypeInformation} to convert those generic records to a RowData - * DataStream. - * - * @param input the generic source input data stream. - * @param mapper function to convert the generic data to {@link RowData} - * @param outputType to define the {@link TypeInformation} for the input data. - * @param the data type of records. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder builderFor( - DataStream input, MapFunction mapper, TypeInformation outputType) { - return new Builder().forMapperOutputType(input, mapper, outputType); - } - - /** - * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into - * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a - * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. - * - * @param input the source input data stream with {@link Row}s. - * @param tableSchema defines the {@link TypeInformation} for input data. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRow(DataStream input, TableSchema tableSchema) { - RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - DataType[] fieldDataTypes = tableSchema.getFieldDataTypes(); - - DataFormatConverters.RowConverter rowConverter = - new DataFormatConverters.RowConverter(fieldDataTypes); - return builderFor(input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) - .tableSchema(tableSchema); - } - - /** - * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s - * into iceberg table. - * - * @param input the source input data stream with {@link RowData}s. - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRowData(DataStream input) { - return new Builder().forRowData(input); - } - - public static class Builder { - private Function> inputCreator = null; - private TableLoader tableLoader; - private Table table; - private TableSchema tableSchema; - private List equalityFieldColumns = null; - private String uidPrefix = null; - private final Map snapshotProperties = Maps.newHashMap(); - private ReadableConfig readableConfig = new Configuration(); - private final Map writeOptions = Maps.newHashMap(); - private FlinkWriteConf flinkWriteConf = null; - - private Builder() {} - - private Builder forRowData(DataStream newRowDataInput) { - this.inputCreator = ignored -> newRowDataInput; - return this; - } - - private Builder forMapperOutputType( - DataStream input, MapFunction mapper, TypeInformation outputType) { - this.inputCreator = - newUidPrefix -> { - // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we - // need to set the parallelism - // of map operator same as its input to keep map operator chaining its input, and avoid - // rebalanced by default. - SingleOutputStreamOperator inputStream = - input.map(mapper, outputType).setParallelism(input.getParallelism()); - if (newUidPrefix != null) { - inputStream.name(operatorName(newUidPrefix)).uid(newUidPrefix + "-mapper"); - } - return inputStream; - }; - return this; - } - - /** - * This iceberg {@link Table} instance is used for initializing {@link IcebergStreamWriter} - * which will write all the records into {@link DataFile}s and emit them to downstream operator. - * Providing a table would avoid so many table loading from each separate task. - * - * @param newTable the loaded iceberg table instance. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - /** - * The table loader is used for loading tables in {@link IcebergFilesCommitter} lazily, we need - * this loader because {@link Table} is not serializable and could not just use the loaded table - * from Builder#table in the remote task manager. - * - * @param newTableLoader to load iceberg table inside tasks. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder tableLoader(TableLoader newTableLoader) { - this.tableLoader = newTableLoader; - return this; - } - - /** - * Set the write properties for Flink sink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder set(String property, String value) { - writeOptions.put(property, value); - return this; - } - - /** - * Set the write properties for Flink sink. View the supported properties in {@link - * FlinkWriteOptions} - */ - public Builder setAll(Map properties) { - writeOptions.putAll(properties); - return this; - } - - public Builder tableSchema(TableSchema newTableSchema) { - this.tableSchema = newTableSchema; - return this; - } - - public Builder overwrite(boolean newOverwrite) { - writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - /** - * Configure the write {@link DistributionMode} that the flink sink will use. Currently, flink - * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. - * - * @param mode to specify the write distribution mode. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); - if (mode != null) { - writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); - } - return this; - } - - /** - * Configuring the write parallel number for iceberg stream writer. - * - * @param newWriteParallelism the number of parallel iceberg stream writer. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder writeParallelism(int newWriteParallelism) { - writeOptions.put( - FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); - return this; - } - - /** - * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which - * means it will DELETE the old records and then INSERT the new records. In partitioned table, - * the partition fields should be a subset of equality fields, otherwise the old row that - * located in partition-A could not be deleted by the new row that located in partition-B. - * - * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder upsert(boolean enabled) { - writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); - return this; - } - - /** - * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. - * - * @param columns defines the iceberg table's key. - * @return {@link Builder} to connect the iceberg table. - */ - public Builder equalityFieldColumns(List columns) { - this.equalityFieldColumns = columns; - return this; - } - - /** - * Set the uid prefix for FlinkSink operators. Note that FlinkSink internally consists of - * multiple operators (like writer, committer, dummy sink etc.) Actually operator uid will be - * appended with a suffix like "uidPrefix-writer".
    - *
    - * If provided, this prefix is also applied to operator names.
    - *
    - * Flink auto generates operator uid if not set explicitly. It is a recommended - * best-practice to set uid for all operators before deploying to production. Flink has an - * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force - * explicit setting of all operator uid.
    - *
    - * Be careful with setting this for an existing job, because now we are changing the operator - * uid from an auto-generated one to this new value. When deploying the change with a - * checkpoint, Flink won't be able to restore the previous Flink sink operator state (more - * specifically the committer operator state). You need to use {@code --allowNonRestoredState} - * to ignore the previous sink state. During restore Flink sink state is used to check if last - * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss - * if the Iceberg commit failed in the last completed checkpoint. - * - * @param newPrefix prefix for Flink sink operator uid and name - * @return {@link Builder} to connect the iceberg table. - */ - public Builder uidPrefix(String newPrefix) { - this.uidPrefix = newPrefix; - return this; - } - - public Builder setSnapshotProperties(Map properties) { - snapshotProperties.putAll(properties); - return this; - } - - public Builder setSnapshotProperty(String property, String value) { - snapshotProperties.put(property, value); - return this; - } - - public Builder toBranch(String branch) { - writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); - return this; - } - - private DataStreamSink chainIcebergOperators() { - Preconditions.checkArgument( - inputCreator != null, - "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); - Preconditions.checkNotNull(tableLoader, "Table loader shouldn't be null"); - - DataStream rowDataInput = inputCreator.apply(uidPrefix); - - if (table == null) { - if (!tableLoader.isOpen()) { - tableLoader.open(); - } - - try (TableLoader loader = tableLoader) { - this.table = loader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to load iceberg table from table loader: " + tableLoader, e); - } - } - - flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); - - // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); - - // Convert the requested flink table schema to flink row type. - RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); - - // Distribute the records from input data stream based on the write.distribution-mode and - // equality fields. - DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); - - // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); - - // Add single-parallelism committer that commits files - // after successful checkpoint or end of input - SingleOutputStreamOperator committerStream = appendCommitter(writerStream); - - // Add dummy discard sink - return appendDummySink(committerStream); - } - - /** - * Append the iceberg sink operators to write records to iceberg table. - * - * @return {@link DataStreamSink} for sink. - */ - public DataStreamSink append() { - return chainIcebergOperators(); - } - - private String operatorName(String suffix) { - return uidPrefix != null ? uidPrefix + "-" + suffix : suffix; - } - - @VisibleForTesting - List checkAndGetEqualityFieldIds() { - List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); - if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { - Set equalityFieldSet = - Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); - for (String column : equalityFieldColumns) { - org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); - Preconditions.checkNotNull( - field, - "Missing required equality field column '%s' in table schema %s", - column, - table.schema()); - equalityFieldSet.add(field.fieldId()); - } - - if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { - LOG.warn( - "The configured equality field column IDs {} are not matched with the schema identifier field IDs" - + " {}, use job specified equality field columns as the equality fields by default.", - equalityFieldSet, - table.schema().identifierFieldIds()); - } - equalityFieldIds = Lists.newArrayList(equalityFieldSet); - } - return equalityFieldIds; - } - - @SuppressWarnings("unchecked") - private DataStreamSink appendDummySink( - SingleOutputStreamOperator committerStream) { - DataStreamSink resultStream = - committerStream - .addSink(new DiscardingSink()) - .name(operatorName(String.format("IcebergSink %s", this.table.name()))) - .setParallelism(1); - if (uidPrefix != null) { - resultStream = resultStream.uid(uidPrefix + "-dummysink"); - } - return resultStream; - } - - private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { - IcebergFilesCommitter filesCommitter = - new IcebergFilesCommitter( - tableLoader, - flinkWriteConf.overwriteMode(), - snapshotProperties, - flinkWriteConf.workerPoolSize(), - flinkWriteConf.branch(), - table.spec()); - SingleOutputStreamOperator committerStream = - writerStream - .transform(operatorName(ICEBERG_FILES_COMMITTER_NAME), Types.VOID, filesCommitter) - .setParallelism(1) - .setMaxParallelism(1); - if (uidPrefix != null) { - committerStream = committerStream.uid(uidPrefix + "-committer"); - } - return committerStream; - } - - private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { - // Validate the equality fields and partition fields if we enable the upsert mode. - if (flinkWriteConf.upsertMode()) { - Preconditions.checkState( - !flinkWriteConf.overwriteMode(), - "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - Preconditions.checkState( - !equalityFieldIds.isEmpty(), - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); - if (!table.spec().isUnpartitioned()) { - for (PartitionField partitionField : table.spec().fields()) { - Preconditions.checkState( - equalityFieldIds.contains(partitionField.sourceId()), - "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", - partitionField, - equalityFieldColumns); - } - } - } - - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); - - SerializableSupplier
    tableSupplier; - if (tableRefreshInterval != null) { - tableSupplier = - new CachingTableSupplier(serializableTable, tableLoader, tableRefreshInterval); - } else { - tableSupplier = () -> serializableTable; - } - - IcebergStreamWriter streamWriter = - createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); - SingleOutputStreamOperator writerStream = - input - .transform( - operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), - streamWriter) - .setParallelism(parallelism); - if (uidPrefix != null) { - writerStream = writerStream.uid(uidPrefix + "-writer"); - } - return writerStream; - } - - private DataStream distributeDataStream( - DataStream input, - List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { - DistributionMode writeMode = flinkWriteConf.distributionMode(); - - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); - switch (writeMode) { - case NONE: - if (equalityFieldIds.isEmpty()) { - return input; - } else { - LOG.info("Distribute rows by equality fields, because there are equality fields set"); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } - - case HASH: - if (equalityFieldIds.isEmpty()) { - if (partitionSpec.isUnpartitioned()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and table is unpartitioned"); - return input; - } else { - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); - } - } else { - if (partitionSpec.isUnpartitioned()) { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and table is unpartitioned"); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } else { - for (PartitionField partitionField : partitionSpec.fields()) { - Preconditions.checkState( - equalityFieldIds.contains(partitionField.sourceId()), - "In 'hash' distribution mode with equality fields set, partition field '%s' " - + "should be included in equality fields: '%s'", - partitionField, - equalityFieldColumns); - } - return input.keyBy(new PartitionKeySelector(partitionSpec, iSchema, flinkRowType)); - } - } - - case RANGE: - if (equalityFieldIds.isEmpty()) { - LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input.keyBy( - new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); - } - - default: - throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); - } - } - } - - static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { - if (requestedSchema != null) { - // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing - // iceberg schema. - Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); - TypeUtil.validateWriteSchema(schema, writeSchema, true, true); - - // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. - return (RowType) requestedSchema.toRowDataType().getLogicalType(); - } else { - return FlinkSchemaUtil.convert(schema); - } - } - - static IcebergStreamWriter createStreamWriter( - SerializableSupplier
    tableSupplier, - FlinkWriteConf flinkWriteConf, - RowType flinkRowType, - List equalityFieldIds) { - Preconditions.checkArgument(tableSupplier != null, "Iceberg table supplier shouldn't be null"); - - Table initTable = tableSupplier.get(); - FileFormat format = flinkWriteConf.dataFileFormat(); - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - tableSupplier, - flinkRowType, - flinkWriteConf.targetDataFileSize(), - format, - writeProperties(initTable, format, flinkWriteConf), - equalityFieldIds, - flinkWriteConf.upsertMode()); - - return new IcebergStreamWriter<>(initTable.name(), taskWriterFactory); - } - - /** - * Based on the {@link FileFormat} overwrites the table level compression properties for the table - * write. - * - * @param table The table to get the table level settings - * @param format The FileFormat to use - * @param conf The write configuration - * @return The properties to use for writing - */ - private static Map writeProperties( - Table table, FileFormat format, FlinkWriteConf conf) { - Map writeProperties = Maps.newHashMap(table.properties()); - - switch (format) { - case PARQUET: - writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); - String parquetCompressionLevel = conf.parquetCompressionLevel(); - if (parquetCompressionLevel != null) { - writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); - } - - break; - case AVRO: - writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); - String avroCompressionLevel = conf.avroCompressionLevel(); - if (avroCompressionLevel != null) { - writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); - } - - break; - case ORC: - writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); - writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); - break; - default: - throw new IllegalArgumentException(String.format("Unknown file format %s", format)); - } - - return writeProperties; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java deleted file mode 100644 index b9bceaa9311d..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ /dev/null @@ -1,516 +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.sink; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Comparator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import java.util.SortedMap; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeinfo.BasicTypeInfo; -import org.apache.flink.api.common.typeinfo.PrimitiveArrayTypeInfo; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.runtime.typeutils.SortedMapTypeInfo; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.ReplacePartitions; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotUpdate; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.WriteResult; -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; -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.types.Comparators; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private static final long serialVersionUID = 1L; - private static final long INITIAL_CHECKPOINT_ID = -1L; - private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; - - private static final Logger LOG = LoggerFactory.getLogger(IcebergFilesCommitter.class); - private static final String FLINK_JOB_ID = "flink.job-id"; - private static final String OPERATOR_ID = "flink.operator-id"; - - // The max checkpoint id we've committed to iceberg table. As the flink's checkpoint is always - // increasing, so we could correctly commit all the data files whose checkpoint id is greater than - // the max committed one to iceberg table, for avoiding committing the same data files twice. This - // id will be attached to iceberg's meta when committing the iceberg transaction. - private static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; - static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; - - // TableLoader to load iceberg table lazily. - private final TableLoader tableLoader; - private final boolean replacePartitions; - private final Map snapshotProperties; - - // A sorted map to maintain the completed data files for each pending checkpointId (which have not - // been committed to iceberg table). We need a sorted map here because there's possible that few - // checkpoints snapshot failed, for example: the 1st checkpoint have 2 data files <1, >, the 2st checkpoint have 1 data files <2, >. Snapshot for checkpoint#1 - // interrupted because of network/disk failure etc, while we don't expect any data loss in iceberg - // table. So we keep the finished files <1, > in memory and retry to commit iceberg - // table when the next checkpoint happen. - private final NavigableMap dataFilesPerCheckpoint = Maps.newTreeMap(); - - // The completed files cache for current checkpoint. Once the snapshot barrier received, it will - // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); - private final String branch; - - // It will have an unique identifier for one job. - private transient String flinkJobId; - private transient String operatorUniqueId; - private transient Table table; - private transient IcebergFilesCommitterMetrics committerMetrics; - private transient ManifestOutputFileFactory manifestOutputFileFactory; - private transient long maxCommittedCheckpointId; - private transient int continuousEmptyCheckpoints; - private transient int maxContinuousEmptyCommits; - // There're two cases that we restore from flink checkpoints: the first case is restoring from - // snapshot created by the same flink job; another case is restoring from snapshot created by - // another different job. For the second case, we need to maintain the old flink job's id in flink - // state backend to find the max-committed-checkpoint-id when traversing iceberg table's - // snapshots. - private static final ListStateDescriptor JOB_ID_DESCRIPTOR = - new ListStateDescriptor<>("iceberg-flink-job-id", BasicTypeInfo.STRING_TYPE_INFO); - private transient ListState jobIdState; - // All pending checkpoints states for this function. - private static final ListStateDescriptor> STATE_DESCRIPTOR = - buildStateDescriptor(); - private transient ListState> checkpointsState; - - private final Integer workerPoolSize; - private final PartitionSpec spec; - private transient ExecutorService workerPool; - - IcebergFilesCommitter( - TableLoader tableLoader, - boolean replacePartitions, - Map snapshotProperties, - Integer workerPoolSize, - String branch, - PartitionSpec spec) { - this.tableLoader = tableLoader; - this.replacePartitions = replacePartitions; - this.snapshotProperties = snapshotProperties; - this.workerPoolSize = workerPoolSize; - this.branch = branch; - this.spec = spec; - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - this.flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); - this.operatorUniqueId = getRuntimeContext().getOperatorUniqueID(); - - // Open the table loader and load the table. - this.tableLoader.open(); - this.table = tableLoader.loadTable(); - this.committerMetrics = new IcebergFilesCommitterMetrics(super.metrics, table.name()); - - maxContinuousEmptyCommits = - PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); - Preconditions.checkArgument( - maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); - - int subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - int attemptId = getRuntimeContext().getAttemptNumber(); - this.manifestOutputFileFactory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorUniqueId, subTaskId, attemptId); - this.maxCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - this.checkpointsState = context.getOperatorStateStore().getListState(STATE_DESCRIPTOR); - this.jobIdState = context.getOperatorStateStore().getListState(JOB_ID_DESCRIPTOR); - if (context.isRestored()) { - Iterable jobIdIterable = jobIdState.get(); - if (jobIdIterable == null || !jobIdIterable.iterator().hasNext()) { - LOG.warn( - "Failed to restore committer state. This can happen when operator uid changed and Flink " - + "allowNonRestoredState is enabled. Best practice is to explicitly set the operator id " - + "via FlinkSink#Builder#uidPrefix() so that the committer operator uid is stable. " - + "Otherwise, Flink auto generate an operator uid based on job topology." - + "With that, operator uid is subjective to change upon topology change."); - return; - } - - String restoredFlinkJobId = jobIdIterable.iterator().next(); - Preconditions.checkState( - !Strings.isNullOrEmpty(restoredFlinkJobId), - "Flink job id parsed from checkpoint snapshot shouldn't be null or empty"); - - // Since flink's checkpoint id will start from the max-committed-checkpoint-id + 1 in the new - // flink job even if it's restored from a snapshot created by another different flink job, so - // it's safe to assign the max committed checkpoint id from restored flink job to the current - // flink job. - this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); - - NavigableMap uncommittedDataFiles = - Maps.newTreeMap(checkpointsState.get().iterator().next()) - .tailMap(maxCommittedCheckpointId, false); - if (!uncommittedDataFiles.isEmpty()) { - // Committed all uncommitted data files from the old flink job to iceberg table. - long maxUncommittedCheckpointId = uncommittedDataFiles.lastKey(); - commitUpToCheckpoint( - uncommittedDataFiles, restoredFlinkJobId, operatorUniqueId, maxUncommittedCheckpointId); - } - } - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - long checkpointId = context.getCheckpointId(); - LOG.info( - "Start to flush snapshot state to state backend, table: {}, checkpointId: {}", - table, - checkpointId); - - // Update the checkpoint state. - long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); - // Reset the snapshot state to the latest state. - checkpointsState.clear(); - checkpointsState.add(dataFilesPerCheckpoint); - - jobIdState.clear(); - jobIdState.add(flinkJobId); - - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); - committerMetrics.checkpointDuration( - TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - super.notifyCheckpointComplete(checkpointId); - // It's possible that we have the following events: - // 1. snapshotState(ckpId); - // 2. snapshotState(ckpId+1); - // 3. notifyCheckpointComplete(ckpId+1); - // 4. notifyCheckpointComplete(ckpId); - // For step#4, we don't need to commit iceberg table again because in step#3 we've committed all - // the files, - // Besides, we need to maintain the max-committed-checkpoint-id to be increasing. - if (checkpointId > maxCommittedCheckpointId) { - LOG.info("Checkpoint {} completed. Attempting commit.", checkpointId); - commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, checkpointId); - this.maxCommittedCheckpointId = checkpointId; - } else { - LOG.info( - "Skipping committing checkpoint {}. {} is already committed.", - checkpointId, - maxCommittedCheckpointId); - } - - // reload the table in case new configuration is needed - this.table = tableLoader.loadTable(); - } - - private void commitUpToCheckpoint( - NavigableMap deltaManifestsMap, - String newFlinkJobId, - String operatorId, - long checkpointId) - throws IOException { - NavigableMap pendingMap = deltaManifestsMap.headMap(checkpointId, true); - List manifests = Lists.newArrayList(); - NavigableMap pendingResults = Maps.newTreeMap(); - for (Map.Entry e : pendingMap.entrySet()) { - if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue())) { - // Skip the empty flink manifest. - continue; - } - - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, e.getValue()); - pendingResults.put( - e.getKey(), - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); - manifests.addAll(deltaManifests.manifests()); - } - - CommitSummary summary = new CommitSummary(pendingResults); - commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - committerMetrics.updateCommitSummary(summary); - pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); - } - - private void commitPendingResult( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); - continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; - if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { - if (replacePartitions) { - replacePartitions(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - } else { - commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); - } - continuousEmptyCheckpoints = 0; - } else { - LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); - } - } - - private void deleteCommittedManifests( - List manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - - private void replacePartitions( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - Preconditions.checkState( - summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); - // Commit the overwrite transaction. - ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); - for (WriteResult result : pendingResults.values()) { - Preconditions.checkState( - result.referencedDataFiles().length == 0, "Should have no referenced data files."); - Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); - } - - commitOperation( - dynamicOverwrite, - summary, - "dynamic partition overwrite", - newFlinkJobId, - operatorId, - checkpointId); - } - - private void commitDeltaTxn( - NavigableMap pendingResults, - CommitSummary summary, - String newFlinkJobId, - String operatorId, - long checkpointId) { - if (summary.deleteFilesCount() == 0) { - // To be compatible with iceberg format V1. - AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); - for (WriteResult result : pendingResults.values()) { - Preconditions.checkState( - result.referencedDataFiles().length == 0, - "Should have no referenced data files for append."); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - } - commitOperation(appendFiles, summary, "append", newFlinkJobId, operatorId, checkpointId); - } else { - // To be compatible with iceberg format V2. - for (Map.Entry e : pendingResults.entrySet()) { - // We don't commit the merged result into a single transaction because for the sequential - // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied - // to data files from txn1. Committing the merged one will lead to the incorrect delete - // semantic. - WriteResult result = e.getValue(); - - // Row delta validations are not needed for streaming changes that write equality deletes. - // Equality deletes are applied to data in all previous sequence numbers, so retries may - // push deletes further in the future, but do not affect correctness. Position deletes - // committed to the table in this path are used only to delete rows from data files that are - // being added in this commit. There is no way for data files added along with the delete - // files to be concurrently removed, so there is no need to validate the files referenced by - // the position delete files that are being committed. - RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); - - Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); - Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); - commitOperation(rowDelta, summary, "rowDelta", newFlinkJobId, operatorId, e.getKey()); - } - } - } - - private void commitOperation( - SnapshotUpdate operation, - CommitSummary summary, - String description, - String newFlinkJobId, - String operatorId, - long checkpointId) { - LOG.info( - "Committing {} for checkpoint {} to table {} branch {} with summary: {}", - description, - checkpointId, - table.name(), - branch, - summary); - snapshotProperties.forEach(operation::set); - // custom snapshot metadata properties will be overridden if they conflict with internal ones - // used by the sink. - operation.set(MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); - operation.set(FLINK_JOB_ID, newFlinkJobId); - operation.set(OPERATOR_ID, operatorId); - operation.toBranch(branch); - - long startNano = System.nanoTime(); - operation.commit(); // abort is automatically called if this fails. - long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); - LOG.info( - "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", - description, - table.name(), - branch, - checkpointId, - durationMs); - committerMetrics.commitDuration(durationMs); - } - - @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); - } - - @Override - public void endInput() throws IOException { - // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - - commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); - } - - /** - * Write all the complete data files to a newly created manifest file and return the manifest's - * avro serialized bytes. - */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - result, () -> manifestOutputFileFactory.create(checkpointId), spec); - - return SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, deltaManifests); - } - - @Override - public void open() throws Exception { - super.open(); - - final String operatorID = getRuntimeContext().getOperatorUniqueID(); - this.workerPool = - ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); - } - - @Override - public void close() throws Exception { - if (tableLoader != null) { - tableLoader.close(); - } - - if (workerPool != null) { - workerPool.shutdown(); - } - } - - @VisibleForTesting - static ListStateDescriptor> buildStateDescriptor() { - Comparator longComparator = Comparators.forType(Types.LongType.get()); - // Construct a SortedMapTypeInfo. - SortedMapTypeInfo sortedMapTypeInfo = - new SortedMapTypeInfo<>( - BasicTypeInfo.LONG_TYPE_INFO, - PrimitiveArrayTypeInfo.BYTE_PRIMITIVE_ARRAY_TYPE_INFO, - longComparator); - return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); - } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java deleted file mode 100644 index 9de0d6aaa551..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitterMetrics.java +++ /dev/null @@ -1,96 +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.sink; - -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; - -class IcebergFilesCommitterMetrics { - private final AtomicLong lastCheckpointDurationMs = new AtomicLong(); - private final AtomicLong lastCommitDurationMs = new AtomicLong(); - private final ElapsedTimeGauge elapsedSecondsSinceLastSuccessfulCommit; - private final Counter committedDataFilesCount; - private final Counter committedDataFilesRecordCount; - private final Counter committedDataFilesByteCount; - private final Counter committedDeleteFilesCount; - private final Counter committedDeleteFilesRecordCount; - private final Counter committedDeleteFilesByteCount; - - IcebergFilesCommitterMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup committerMetrics = - metrics.addGroup("IcebergFilesCommitter").addGroup("table", fullTableName); - committerMetrics.gauge("lastCheckpointDurationMs", lastCheckpointDurationMs::get); - committerMetrics.gauge("lastCommitDurationMs", lastCommitDurationMs::get); - this.elapsedSecondsSinceLastSuccessfulCommit = new ElapsedTimeGauge(TimeUnit.SECONDS); - committerMetrics.gauge( - "elapsedSecondsSinceLastSuccessfulCommit", elapsedSecondsSinceLastSuccessfulCommit); - this.committedDataFilesCount = committerMetrics.counter("committedDataFilesCount"); - this.committedDataFilesRecordCount = committerMetrics.counter("committedDataFilesRecordCount"); - this.committedDataFilesByteCount = committerMetrics.counter("committedDataFilesByteCount"); - this.committedDeleteFilesCount = committerMetrics.counter("committedDeleteFilesCount"); - this.committedDeleteFilesRecordCount = - committerMetrics.counter("committedDeleteFilesRecordCount"); - this.committedDeleteFilesByteCount = committerMetrics.counter("committedDeleteFilesByteCount"); - } - - void checkpointDuration(long checkpointDurationMs) { - lastCheckpointDurationMs.set(checkpointDurationMs); - } - - void commitDuration(long commitDurationMs) { - lastCommitDurationMs.set(commitDurationMs); - } - - /** This is called upon a successful commit. */ - void updateCommitSummary(CommitSummary stats) { - elapsedSecondsSinceLastSuccessfulCommit.refreshLastRecordedTime(); - committedDataFilesCount.inc(stats.dataFilesCount()); - committedDataFilesRecordCount.inc(stats.dataFilesRecordCount()); - committedDataFilesByteCount.inc(stats.dataFilesByteCount()); - committedDeleteFilesCount.inc(stats.deleteFilesCount()); - committedDeleteFilesRecordCount.inc(stats.deleteFilesRecordCount()); - committedDeleteFilesByteCount.inc(stats.deleteFilesByteCount()); - } - - /** - * This gauge measures the elapsed time between now and last recorded time set by {@link - * ElapsedTimeGauge#refreshLastRecordedTime()}. - */ - private static class ElapsedTimeGauge implements Gauge { - private final TimeUnit reportUnit; - private volatile long lastRecordedTimeNano; - - ElapsedTimeGauge(TimeUnit timeUnit) { - this.reportUnit = timeUnit; - this.lastRecordedTimeNano = System.nanoTime(); - } - - void refreshLastRecordedTime() { - this.lastRecordedTimeNano = System.nanoTime(); - } - - @Override - public Long getValue() { - return reportUnit.convert(System.nanoTime() - lastRecordedTimeNano, TimeUnit.NANOSECONDS); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java deleted file mode 100644 index 9ea0349fb057..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ /dev/null @@ -1,120 +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.sink; - -import java.io.IOException; -import java.util.concurrent.TimeUnit; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.ChainingStrategy; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { - - private static final long serialVersionUID = 1L; - - private final String fullTableName; - private final TaskWriterFactory taskWriterFactory; - - private transient TaskWriter writer; - private transient int subTaskId; - private transient int attemptId; - private transient IcebergStreamWriterMetrics writerMetrics; - - IcebergStreamWriter(String fullTableName, TaskWriterFactory taskWriterFactory) { - this.fullTableName = fullTableName; - this.taskWriterFactory = taskWriterFactory; - setChainingStrategy(ChainingStrategy.ALWAYS); - } - - @Override - public void open() { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); - this.writerMetrics = new IcebergStreamWriterMetrics(super.metrics, fullTableName); - - // Initialize the task writer factory. - this.taskWriterFactory.initialize(subTaskId, attemptId); - - // Initialize the task writer. - this.writer = taskWriterFactory.create(); - } - - @Override - public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); - this.writer = taskWriterFactory.create(); - } - - @Override - public void processElement(StreamRecord element) throws Exception { - writer.write(element.getValue()); - } - - @Override - public void close() throws Exception { - super.close(); - if (writer != null) { - writer.close(); - writer = null; - } - } - - @Override - public void endInput() throws IOException { - // For bounded stream, it may don't enable the checkpoint mechanism so we'd better to emit the - // remaining completed files to downstream before closing the writer so that we won't miss any - // of them. - // Note that if the task is not closed after calling endInput, checkpoint may be triggered again - // causing files to be sent repeatedly, the writer is marked as null after the last file is sent - // to guard against duplicated writes. - flush(); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) - .toString(); - } - - /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { - if (writer == null) { - return; - } - - long startNano = System.nanoTime(); - WriteResult result = writer.complete(); - writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); - writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); - - // Set writer to null to prevent duplicate flushes in the corner case of - // prepareSnapshotPreBarrier happening after endInput. - writer = null; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java deleted file mode 100644 index ce2a6c583fdf..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriterMetrics.java +++ /dev/null @@ -1,89 +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.sink; - -import com.codahale.metrics.SlidingWindowReservoir; -import java.util.Arrays; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.dropwizard.metrics.DropwizardHistogramWrapper; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Histogram; -import org.apache.flink.metrics.MetricGroup; -import org.apache.iceberg.io.WriteResult; - -class IcebergStreamWriterMetrics { - // 1,024 reservoir size should cost about 8KB, which is quite small. - // It should also produce good accuracy for histogram distribution (like percentiles). - private static final int HISTOGRAM_RESERVOIR_SIZE = 1024; - - private final Counter flushedDataFiles; - private final Counter flushedDeleteFiles; - private final Counter flushedReferencedDataFiles; - private final AtomicLong lastFlushDurationMs; - private final Histogram dataFilesSizeHistogram; - private final Histogram deleteFilesSizeHistogram; - - IcebergStreamWriterMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup writerMetrics = - metrics.addGroup("IcebergStreamWriter").addGroup("table", fullTableName); - this.flushedDataFiles = writerMetrics.counter("flushedDataFiles"); - this.flushedDeleteFiles = writerMetrics.counter("flushedDeleteFiles"); - this.flushedReferencedDataFiles = writerMetrics.counter("flushedReferencedDataFiles"); - this.lastFlushDurationMs = new AtomicLong(); - writerMetrics.gauge("lastFlushDurationMs", lastFlushDurationMs::get); - - com.codahale.metrics.Histogram dropwizardDataFilesSizeHistogram = - new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); - this.dataFilesSizeHistogram = - writerMetrics.histogram( - "dataFilesSizeHistogram", - new DropwizardHistogramWrapper(dropwizardDataFilesSizeHistogram)); - com.codahale.metrics.Histogram dropwizardDeleteFilesSizeHistogram = - new com.codahale.metrics.Histogram(new SlidingWindowReservoir(HISTOGRAM_RESERVOIR_SIZE)); - this.deleteFilesSizeHistogram = - writerMetrics.histogram( - "deleteFilesSizeHistogram", - new DropwizardHistogramWrapper(dropwizardDeleteFilesSizeHistogram)); - } - - void updateFlushResult(WriteResult result) { - flushedDataFiles.inc(result.dataFiles().length); - flushedDeleteFiles.inc(result.deleteFiles().length); - flushedReferencedDataFiles.inc(result.referencedDataFiles().length); - - // For file size distribution histogram, we don't have to update them after successful commits. - // This should works equally well and we avoided the overhead of tracking the list of file sizes - // in the {@link CommitSummary}, which currently stores simple stats for counters and gauges - // metrics. - Arrays.stream(result.dataFiles()) - .forEach( - dataFile -> { - dataFilesSizeHistogram.update(dataFile.fileSizeInBytes()); - }); - Arrays.stream(result.deleteFiles()) - .forEach( - deleteFile -> { - deleteFilesSizeHistogram.update(deleteFile.fileSizeInBytes()); - }); - } - - void flushDuration(long flushDurationMs) { - lastFlushDurationMs.set(flushDurationMs); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java deleted file mode 100644 index da5e6e7627ae..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ /dev/null @@ -1,94 +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.sink; - -import java.util.Map; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.base.Strings; - -class ManifestOutputFileFactory { - // Users could define their own flink manifests directory by setting this value in table - // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - - private final Supplier
    tableSupplier; - private final Map props; - private final String flinkJobId; - private final String operatorUniqueId; - private final int subTaskId; - private final long attemptNumber; - private final AtomicInteger fileCount = new AtomicInteger(0); - - ManifestOutputFileFactory( - Supplier
    tableSupplier, - Map props, - String flinkJobId, - String operatorUniqueId, - int subTaskId, - long attemptNumber) { - this.tableSupplier = tableSupplier; - this.props = props; - this.flinkJobId = flinkJobId; - this.operatorUniqueId = operatorUniqueId; - this.subTaskId = subTaskId; - this.attemptNumber = attemptNumber; - } - - private String generatePath(long checkpointId) { - return FileFormat.AVRO.addExtension( - String.format( - "%s-%s-%05d-%d-%d-%05d", - flinkJobId, - operatorUniqueId, - subTaskId, - attemptNumber, - checkpointId, - fileCount.incrementAndGet())); - } - - OutputFile create(long checkpointId) { - String flinkManifestDir = props.get(FLINK_MANIFEST_LOCATION); - TableOperations ops = ((HasTableOperations) tableSupplier.get()).operations(); - - String newManifestFullPath; - if (Strings.isNullOrEmpty(flinkManifestDir)) { - // User don't specify any flink manifest directory, so just use the default metadata path. - newManifestFullPath = ops.metadataFileLocation(generatePath(checkpointId)); - } else { - newManifestFullPath = - String.format("%s/%s", stripTrailingSlash(flinkManifestDir), generatePath(checkpointId)); - } - - return tableSupplier.get().io().newOutputFile(newManifestFullPath); - } - - private static String stripTrailingSlash(String path) { - String result = path; - while (result.endsWith("/")) { - result = result.substring(0, result.length() - 1); - } - return result; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.java deleted file mode 100644 index df951684b446..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionKeySelector.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.flink.sink; - -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.RowDataWrapper; - -/** - * Create a {@link KeySelector} to shuffle by partition key, then each partition/bucket will be - * wrote by only one task. That will reduce lots of small files in partitioned fanout write policy - * for {@link FlinkSink}. - */ -class PartitionKeySelector implements KeySelector { - - private final Schema schema; - private final PartitionKey partitionKey; - private final RowType flinkSchema; - - private transient RowDataWrapper rowDataWrapper; - - PartitionKeySelector(PartitionSpec spec, Schema schema, RowType flinkSchema) { - this.schema = schema; - this.partitionKey = new PartitionKey(spec, schema); - this.flinkSchema = flinkSchema; - } - - /** - * Construct the {@link RowDataWrapper} lazily here because few members in it are not - * serializable. In this way, we don't have to serialize them with forcing. - */ - private RowDataWrapper lazyRowDataWrapper() { - if (rowDataWrapper == null) { - rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - return rowDataWrapper; - } - - @Override - public String getKey(RowData row) { - partitionKey.partition(lazyRowDataWrapper().wrap(row)); - return partitionKey.toPath(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java deleted file mode 100644 index 38062dd1a2c4..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/PartitionedDeltaWriter.java +++ /dev/null @@ -1,97 +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.sink; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Tasks; - -class PartitionedDeltaWriter extends BaseDeltaTaskWriter { - - private final PartitionKey partitionKey; - - private final Map writers = Maps.newHashMap(); - - PartitionedDeltaWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super( - spec, - format, - appenderFactory, - fileFactory, - io, - targetFileSize, - schema, - flinkSchema, - equalityFieldIds, - upsert); - this.partitionKey = new PartitionKey(spec, schema); - } - - @Override - RowDataDeltaWriter route(RowData row) { - partitionKey.partition(wrapper().wrap(row)); - - RowDataDeltaWriter writer = writers.get(partitionKey); - if (writer == null) { - // NOTICE: we need to copy a new partition key here, in case of messing up the keys in - // writers. - PartitionKey copiedKey = partitionKey.copy(); - writer = new RowDataDeltaWriter(copiedKey); - writers.put(copiedKey, writer); - } - - return writer; - } - - @Override - public void close() { - try { - Tasks.foreach(writers.values()) - .throwFailureWhenFinished() - .noRetry() - .run(RowDataDeltaWriter::close, IOException.class); - - writers.clear(); - } catch (IOException e) { - throw new UncheckedIOException("Failed to close equality delta writer", e); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java deleted file mode 100644 index 67422a1afeb1..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/RowDataTaskWriterFactory.java +++ /dev/null @@ -1,244 +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.sink; - -import java.util.List; -import java.util.Map; -import java.util.function.Supplier; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; -import org.apache.iceberg.io.PartitionedFanoutWriter; -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.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.SerializableSupplier; - -public class RowDataTaskWriterFactory implements TaskWriterFactory { - private final Supplier
    tableSupplier; - private final Schema schema; - private final RowType flinkSchema; - private final PartitionSpec spec; - private final long targetFileSizeBytes; - private final FileFormat format; - private final List equalityFieldIds; - private final boolean upsert; - private final FileAppenderFactory appenderFactory; - - private transient OutputFileFactory outputFileFactory; - - public RowDataTaskWriterFactory( - Table table, - RowType flinkSchema, - long targetFileSizeBytes, - FileFormat format, - Map writeProperties, - List equalityFieldIds, - boolean upsert) { - this( - () -> table, - flinkSchema, - targetFileSizeBytes, - format, - writeProperties, - equalityFieldIds, - upsert); - } - - public RowDataTaskWriterFactory( - SerializableSupplier
    tableSupplier, - RowType flinkSchema, - long targetFileSizeBytes, - FileFormat format, - Map writeProperties, - List equalityFieldIds, - boolean upsert) { - this.tableSupplier = tableSupplier; - - Table table; - if (tableSupplier instanceof CachingTableSupplier) { - // rely on the initial table metadata for schema, etc., until schema evolution is supported - table = ((CachingTableSupplier) tableSupplier).initialTable(); - } else { - table = tableSupplier.get(); - } - - this.schema = table.schema(); - this.flinkSchema = flinkSchema; - this.spec = table.spec(); - this.targetFileSizeBytes = targetFileSizeBytes; - this.format = format; - this.equalityFieldIds = equalityFieldIds; - this.upsert = upsert; - - if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { - this.appenderFactory = - new FlinkAppenderFactory( - table, schema, flinkSchema, writeProperties, spec, null, null, null); - } else if (upsert) { - // In upsert mode, only the new row is emitted using INSERT row kind. Therefore, any column of - // the inserted row - // may differ from the deleted row other than the primary key fields, and the delete file must - // contain values - // that are correct for the deleted row. Therefore, only write the equality delete fields. - this.appenderFactory = - new FlinkAppenderFactory( - table, - schema, - flinkSchema, - writeProperties, - spec, - ArrayUtil.toIntArray(equalityFieldIds), - TypeUtil.select(schema, Sets.newHashSet(equalityFieldIds)), - null); - } else { - this.appenderFactory = - new FlinkAppenderFactory( - table, - schema, - flinkSchema, - writeProperties, - spec, - ArrayUtil.toIntArray(equalityFieldIds), - schema, - null); - } - } - - @Override - public void initialize(int taskId, int attemptId) { - Table table; - if (tableSupplier instanceof CachingTableSupplier) { - // rely on the initial table metadata for schema, etc., until schema evolution is supported - table = ((CachingTableSupplier) tableSupplier).initialTable(); - } else { - table = tableSupplier.get(); - } - - refreshTable(); - - this.outputFileFactory = - OutputFileFactory.builderFor(table, taskId, attemptId) - .format(format) - .ioSupplier(() -> tableSupplier.get().io()) - .build(); - } - - @Override - public TaskWriter create() { - Preconditions.checkNotNull( - outputFileFactory, - "The outputFileFactory shouldn't be null if we have invoked the initialize()."); - - refreshTable(); - - if (equalityFieldIds == null || equalityFieldIds.isEmpty()) { - // Initialize a task writer to write INSERT only. - if (spec.isUnpartitioned()) { - return new UnpartitionedWriter<>( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes); - } else { - return new RowDataPartitionedFanoutWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema); - } - } else { - // Initialize a task writer to write both INSERT and equality DELETE. - if (spec.isUnpartitioned()) { - return new UnpartitionedDeltaWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema, - equalityFieldIds, - upsert); - } else { - return new PartitionedDeltaWriter( - spec, - format, - appenderFactory, - outputFileFactory, - tableSupplier.get().io(), - targetFileSizeBytes, - schema, - flinkSchema, - equalityFieldIds, - upsert); - } - } - } - - void refreshTable() { - if (tableSupplier instanceof CachingTableSupplier) { - ((CachingTableSupplier) tableSupplier).refreshTable(); - } - } - - private static class RowDataPartitionedFanoutWriter extends PartitionedFanoutWriter { - - private final PartitionKey partitionKey; - private final RowDataWrapper rowDataWrapper; - - RowDataPartitionedFanoutWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema) { - super(spec, format, appenderFactory, fileFactory, io, targetFileSize); - this.partitionKey = new PartitionKey(spec, schema); - this.rowDataWrapper = new RowDataWrapper(flinkSchema, schema.asStruct()); - } - - @Override - protected PartitionKey partition(RowData row) { - partitionKey.partition(rowDataWrapper.wrap(row)); - return partitionKey; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java deleted file mode 100644 index e3a1245e8cbd..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/TaskWriterFactory.java +++ /dev/null @@ -1,45 +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.sink; - -import java.io.Serializable; -import org.apache.iceberg.io.TaskWriter; - -/** - * Factory to create {@link TaskWriter} - * - * @param data type of record. - */ -public interface TaskWriterFactory extends Serializable { - - /** - * Initialize the factory with a given taskId and attemptId. - * - * @param taskId the identifier of task. - * @param attemptId the attempt id of this task. - */ - void initialize(int taskId, int attemptId); - - /** - * Initialize a {@link TaskWriter} with given task id and attempt id. - * - * @return a newly created task writer. - */ - TaskWriter create(); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java deleted file mode 100644 index 7680fb933b20..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/UnpartitionedDeltaWriter.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.sink; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFileFactory; - -class UnpartitionedDeltaWriter extends BaseDeltaTaskWriter { - private final RowDataDeltaWriter writer; - - UnpartitionedDeltaWriter( - PartitionSpec spec, - FileFormat format, - FileAppenderFactory appenderFactory, - OutputFileFactory fileFactory, - FileIO io, - long targetFileSize, - Schema schema, - RowType flinkSchema, - List equalityFieldIds, - boolean upsert) { - super( - spec, - format, - appenderFactory, - fileFactory, - io, - targetFileSize, - schema, - flinkSchema, - equalityFieldIds, - upsert); - this.writer = new RowDataDeltaWriter(null); - } - - @Override - RowDataDeltaWriter route(RowData row) { - return writer; - } - - @Override - public void close() throws IOException { - writer.close(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java deleted file mode 100644 index 5525f02c873e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ /dev/null @@ -1,262 +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.sink.shuffle; - -import java.util.Map; -import java.util.NavigableMap; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.datasketches.sampling.ReservoirItemsUnion; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * AggregatedStatisticsTracker tracks the statistics aggregation received from {@link - * DataStatisticsOperator} subtasks for every checkpoint. - */ -class AggregatedStatisticsTracker { - private static final Logger LOG = LoggerFactory.getLogger(AggregatedStatisticsTracker.class); - - private final String operatorName; - private final int parallelism; - private final TypeSerializer statisticsSerializer; - private final int downstreamParallelism; - private final StatisticsType statisticsType; - private final int switchToSketchThreshold; - private final NavigableMap aggregationsPerCheckpoint; - - private CompletedStatistics completedStatistics; - - AggregatedStatisticsTracker( - String operatorName, - int parallelism, - Schema schema, - SortOrder sortOrder, - int downstreamParallelism, - StatisticsType statisticsType, - int switchToSketchThreshold, - @Nullable CompletedStatistics restoredStatistics) { - this.operatorName = operatorName; - this.parallelism = parallelism; - this.statisticsSerializer = - new DataStatisticsSerializer(new SortKeySerializer(schema, sortOrder)); - this.downstreamParallelism = downstreamParallelism; - this.statisticsType = statisticsType; - this.switchToSketchThreshold = switchToSketchThreshold; - this.completedStatistics = restoredStatistics; - - this.aggregationsPerCheckpoint = Maps.newTreeMap(); - } - - CompletedStatistics updateAndCheckCompletion(int subtask, StatisticsEvent event) { - long checkpointId = event.checkpointId(); - LOG.debug( - "Handling statistics event from subtask {} of operator {} for checkpoint {}", - subtask, - operatorName, - checkpointId); - - if (completedStatistics != null && completedStatistics.checkpointId() > checkpointId) { - LOG.info( - "Ignore stale statistics event from operator {} subtask {} for older checkpoint {}. " - + "Was expecting data statistics from checkpoint higher than {}", - operatorName, - subtask, - checkpointId, - completedStatistics.checkpointId()); - return null; - } - - Aggregation aggregation = - aggregationsPerCheckpoint.computeIfAbsent( - checkpointId, - ignored -> - new Aggregation( - parallelism, - downstreamParallelism, - switchToSketchThreshold, - statisticsType, - StatisticsUtil.collectType(statisticsType, completedStatistics))); - DataStatistics dataStatistics = - StatisticsUtil.deserializeDataStatistics(event.statisticsBytes(), statisticsSerializer); - if (!aggregation.merge(subtask, dataStatistics)) { - LOG.debug( - "Ignore duplicate data statistics from operator {} subtask {} for checkpoint {}.", - operatorName, - subtask, - checkpointId); - } - - if (aggregation.isComplete()) { - this.completedStatistics = aggregation.completedStatistics(checkpointId); - // clean up aggregations up to the completed checkpoint id - aggregationsPerCheckpoint.headMap(checkpointId, true).clear(); - return completedStatistics; - } - - return null; - } - - @VisibleForTesting - NavigableMap aggregationsPerCheckpoint() { - return aggregationsPerCheckpoint; - } - - static class Aggregation { - private static final Logger LOG = LoggerFactory.getLogger(Aggregation.class); - - private final Set subtaskSet; - private final int parallelism; - private final int downstreamParallelism; - private final int switchToSketchThreshold; - private final StatisticsType configuredType; - private StatisticsType currentType; - private Map mapStatistics; - private ReservoirItemsUnion sketchStatistics; - - Aggregation( - int parallelism, - int downstreamParallelism, - int switchToSketchThreshold, - StatisticsType configuredType, - StatisticsType currentType) { - this.subtaskSet = Sets.newHashSet(); - this.parallelism = parallelism; - this.downstreamParallelism = downstreamParallelism; - this.switchToSketchThreshold = switchToSketchThreshold; - this.configuredType = configuredType; - this.currentType = currentType; - - if (currentType == StatisticsType.Map) { - this.mapStatistics = Maps.newHashMap(); - this.sketchStatistics = null; - } else { - this.mapStatistics = null; - this.sketchStatistics = - ReservoirItemsUnion.newInstance( - SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); - } - } - - @VisibleForTesting - Set subtaskSet() { - return subtaskSet; - } - - @VisibleForTesting - StatisticsType currentType() { - return currentType; - } - - @VisibleForTesting - Map mapStatistics() { - return mapStatistics; - } - - @VisibleForTesting - ReservoirItemsUnion sketchStatistics() { - return sketchStatistics; - } - - private boolean isComplete() { - return subtaskSet.size() == parallelism; - } - - /** @return false if duplicate */ - private boolean merge(int subtask, DataStatistics taskStatistics) { - if (subtaskSet.contains(subtask)) { - return false; - } - - subtaskSet.add(subtask); - merge(taskStatistics); - return true; - } - - @SuppressWarnings("unchecked") - private void merge(DataStatistics taskStatistics) { - if (taskStatistics.type() == StatisticsType.Map) { - Map taskMapStats = (Map) taskStatistics.result(); - if (currentType == StatisticsType.Map) { - taskMapStats.forEach((key, count) -> mapStatistics.merge(key, count, Long::sum)); - if (configuredType == StatisticsType.Auto - && mapStatistics.size() > switchToSketchThreshold) { - convertCoordinatorToSketch(); - } - } else { - // convert task stats to sketch first - ReservoirItemsSketch taskSketch = - ReservoirItemsSketch.newInstance( - SketchUtil.determineOperatorReservoirSize(parallelism, downstreamParallelism)); - SketchUtil.convertMapToSketch(taskMapStats, taskSketch::update); - sketchStatistics.update(taskSketch); - } - } else { - ReservoirItemsSketch taskSketch = - (ReservoirItemsSketch) taskStatistics.result(); - if (currentType == StatisticsType.Map) { - // convert global stats to sketch first - convertCoordinatorToSketch(); - } - - if (taskSketch.getNumSamples() > 0) { - sketchStatistics.update(taskSketch); - } - } - } - - private void convertCoordinatorToSketch() { - this.sketchStatistics = - ReservoirItemsUnion.newInstance( - SketchUtil.determineCoordinatorReservoirSize(downstreamParallelism)); - SketchUtil.convertMapToSketch(mapStatistics, sketchStatistics::update); - this.currentType = StatisticsType.Sketch; - this.mapStatistics = null; - } - - private CompletedStatistics completedStatistics(long checkpointId) { - if (currentType == StatisticsType.Map) { - LOG.info("Completed map statistics aggregation with {} keys", mapStatistics.size()); - return CompletedStatistics.fromKeyFrequency(checkpointId, mapStatistics); - } else { - ReservoirItemsSketch sketch = sketchStatistics.getResult(); - if (sketch != null) { - LOG.info( - "Completed sketch statistics aggregation: " - + "reservoir size = {}, number of items seen = {}, number of samples = {}", - sketch.getK(), - sketch.getN(), - sketch.getNumSamples()); - return CompletedStatistics.fromKeySamples(checkpointId, sketch.getSamples()); - } else { - LOG.info("Empty sketch statistics."); - return CompletedStatistics.fromKeySamples(checkpointId, new SortKey[0]); - } - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java deleted file mode 100644 index e4cba174f0f2..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatistics.java +++ /dev/null @@ -1,111 +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.sink.shuffle; - -import java.util.Arrays; -import java.util.Map; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; - -/** - * This is what {@link AggregatedStatisticsTracker} returns upon a completed statistics aggregation - * from all subtasks. It contains the raw statistics (Map or reservoir samples). - */ -class CompletedStatistics { - private final long checkpointId; - private final StatisticsType type; - private final Map keyFrequency; - private final SortKey[] keySamples; - - static CompletedStatistics fromKeyFrequency(long checkpointId, Map stats) { - return new CompletedStatistics(checkpointId, StatisticsType.Map, stats, null); - } - - static CompletedStatistics fromKeySamples(long checkpointId, SortKey[] keySamples) { - return new CompletedStatistics(checkpointId, StatisticsType.Sketch, null, keySamples); - } - - CompletedStatistics( - long checkpointId, - StatisticsType type, - Map keyFrequency, - SortKey[] keySamples) { - this.checkpointId = checkpointId; - this.type = type; - this.keyFrequency = keyFrequency; - this.keySamples = keySamples; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("type", type) - .add("keyFrequency", keyFrequency) - .add("keySamples", keySamples) - .toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof CompletedStatistics)) { - return false; - } - - CompletedStatistics other = (CompletedStatistics) o; - return Objects.equal(checkpointId, other.checkpointId) - && Objects.equal(type, other.type) - && Objects.equal(keyFrequency, other.keyFrequency()) - && Arrays.equals(keySamples, other.keySamples()); - } - - @Override - public int hashCode() { - return Objects.hashCode(checkpointId, type, keyFrequency, keySamples); - } - - long checkpointId() { - return checkpointId; - } - - StatisticsType type() { - return type; - } - - Map keyFrequency() { - return keyFrequency; - } - - SortKey[] keySamples() { - return keySamples; - } - - boolean isEmpty() { - if (type == StatisticsType.Sketch) { - return keySamples.length == 0; - } else { - return keyFrequency().isEmpty(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java deleted file mode 100644 index 7f55188e7f8c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/CompletedStatisticsSerializer.java +++ /dev/null @@ -1,178 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.EnumSerializer; -import org.apache.flink.api.common.typeutils.base.ListSerializer; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.iceberg.SortKey; - -class CompletedStatisticsSerializer extends TypeSerializer { - private final TypeSerializer sortKeySerializer; - private final EnumSerializer statisticsTypeSerializer; - private final MapSerializer keyFrequencySerializer; - private final ListSerializer keySamplesSerializer; - - CompletedStatisticsSerializer(TypeSerializer sortKeySerializer) { - this.sortKeySerializer = sortKeySerializer; - this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); - this.keyFrequencySerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); - this.keySamplesSerializer = new ListSerializer<>(sortKeySerializer); - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public TypeSerializer duplicate() { - return new CompletedStatisticsSerializer(sortKeySerializer); - } - - @Override - public CompletedStatistics createInstance() { - return CompletedStatistics.fromKeyFrequency(0L, Collections.emptyMap()); - } - - @Override - public CompletedStatistics copy(CompletedStatistics from) { - return new CompletedStatistics( - from.checkpointId(), from.type(), from.keyFrequency(), from.keySamples()); - } - - @Override - public CompletedStatistics copy(CompletedStatistics from, CompletedStatistics reuse) { - // no benefit of reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(CompletedStatistics record, DataOutputView target) throws IOException { - target.writeLong(record.checkpointId()); - statisticsTypeSerializer.serialize(record.type(), target); - if (record.type() == StatisticsType.Map) { - keyFrequencySerializer.serialize(record.keyFrequency(), target); - } else { - keySamplesSerializer.serialize(Arrays.asList(record.keySamples()), target); - } - } - - @Override - public CompletedStatistics deserialize(DataInputView source) throws IOException { - long checkpointId = source.readLong(); - StatisticsType type = statisticsTypeSerializer.deserialize(source); - if (type == StatisticsType.Map) { - Map keyFrequency = keyFrequencySerializer.deserialize(source); - return CompletedStatistics.fromKeyFrequency(checkpointId, keyFrequency); - } else { - List sortKeys = keySamplesSerializer.deserialize(source); - SortKey[] keySamples = new SortKey[sortKeys.size()]; - keySamples = sortKeys.toArray(keySamples); - return CompletedStatistics.fromKeySamples(checkpointId, keySamples); - } - } - - @Override - public CompletedStatistics deserialize(CompletedStatistics reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - serialize(deserialize(source), target); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - CompletedStatisticsSerializer other = (CompletedStatisticsSerializer) obj; - return Objects.equals(sortKeySerializer, other.sortKeySerializer); - } - - @Override - public int hashCode() { - return sortKeySerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new CompletedStatisticsSerializerSnapshot(this); - } - - public static class CompletedStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; - - /** Constructor for read instantiation. */ - @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public CompletedStatisticsSerializerSnapshot() { - super(CompletedStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public CompletedStatisticsSerializerSnapshot(CompletedStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - CompletedStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.sortKeySerializer}; - } - - @Override - protected CompletedStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; - return new CompletedStatisticsSerializer(sortKeySerializer); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java deleted file mode 100644 index 76c59cd5f4b8..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatistics.java +++ /dev/null @@ -1,48 +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.sink.shuffle; - -import java.util.Map; -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.SortKey; - -/** - * DataStatistics defines the interface to collect data distribution information. - * - *

    Data statistics tracks traffic volume distribution across data keys. For low-cardinality key, - * a simple map of (key, count) can be used. For high-cardinality key, probabilistic data structures - * (sketching) can be used. - */ -@Internal -interface DataStatistics { - - StatisticsType type(); - - boolean isEmpty(); - - /** Add row sortKey to data statistics. */ - void add(SortKey sortKey); - - /** - * Get the collected statistics. Could be a {@link Map} (low cardinality) or {@link - * ReservoirItemsSketch} (high cardinality) - */ - Object result(); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.java deleted file mode 100644 index 4bfde7204acf..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinator.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.flink.sink.shuffle; - -import java.util.Comparator; -import java.util.Map; -import java.util.concurrent.Callable; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.ExecutorService; -import java.util.concurrent.Executors; -import java.util.concurrent.ThreadFactory; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.util.ExceptionUtils; -import org.apache.flink.util.FatalExitExceptionHandler; -import org.apache.flink.util.FlinkRuntimeException; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.ThrowableCatchingRunnable; -import org.apache.flink.util.function.ThrowingRunnable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -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.jetbrains.annotations.NotNull; -import org.jetbrains.annotations.Nullable; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * DataStatisticsCoordinator receives {@link StatisticsEvent} from {@link DataStatisticsOperator} - * every subtask and then merge them together. Once aggregation for all subtasks data statistics - * completes, DataStatisticsCoordinator will send the aggregated data statistics back to {@link - * DataStatisticsOperator}. In the end a custom partitioner will distribute traffic based on the - * aggregated data statistics to improve data clustering. - */ -@Internal -class DataStatisticsCoordinator implements OperatorCoordinator { - private static final Logger LOG = LoggerFactory.getLogger(DataStatisticsCoordinator.class); - - private final String operatorName; - private final OperatorCoordinator.Context context; - private final Schema schema; - private final SortOrder sortOrder; - private final Comparator comparator; - private final int downstreamParallelism; - private final StatisticsType statisticsType; - private final double closeFileCostWeightPercentage; - - private final ExecutorService coordinatorExecutor; - private final SubtaskGateways subtaskGateways; - private final CoordinatorExecutorThreadFactory coordinatorThreadFactory; - private final TypeSerializer completedStatisticsSerializer; - private final TypeSerializer globalStatisticsSerializer; - - private transient boolean started; - private transient AggregatedStatisticsTracker aggregatedStatisticsTracker; - private transient CompletedStatistics completedStatistics; - private transient GlobalStatistics globalStatistics; - - DataStatisticsCoordinator( - String operatorName, - OperatorCoordinator.Context context, - Schema schema, - SortOrder sortOrder, - int downstreamParallelism, - StatisticsType statisticsType, - double closeFileCostWeightPercentage) { - this.operatorName = operatorName; - this.context = context; - this.schema = schema; - this.sortOrder = sortOrder; - this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); - this.downstreamParallelism = downstreamParallelism; - this.statisticsType = statisticsType; - this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; - - this.coordinatorThreadFactory = - new CoordinatorExecutorThreadFactory( - "DataStatisticsCoordinator-" + operatorName, context.getUserCodeClassloader()); - this.coordinatorExecutor = Executors.newSingleThreadExecutor(coordinatorThreadFactory); - this.subtaskGateways = new SubtaskGateways(operatorName, context.currentParallelism()); - SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); - this.completedStatisticsSerializer = new CompletedStatisticsSerializer(sortKeySerializer); - this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); - } - - @Override - public void start() throws Exception { - LOG.info("Starting data statistics coordinator: {}.", operatorName); - this.started = true; - - // statistics are restored already in resetToCheckpoint() before start() called - this.aggregatedStatisticsTracker = - new AggregatedStatisticsTracker( - operatorName, - context.currentParallelism(), - schema, - sortOrder, - downstreamParallelism, - statisticsType, - SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, - completedStatistics); - } - - @Override - public void close() throws Exception { - coordinatorExecutor.shutdown(); - this.aggregatedStatisticsTracker = null; - this.started = false; - LOG.info("Closed data statistics coordinator: {}.", operatorName); - } - - @VisibleForTesting - void callInCoordinatorThread(Callable callable, String errorMessage) { - ensureStarted(); - // Ensure the task is done by the coordinator executor. - if (!coordinatorThreadFactory.isCurrentThreadCoordinatorThread()) { - try { - Callable guardedCallable = - () -> { - try { - return callable.call(); - } catch (Throwable t) { - LOG.error( - "Uncaught Exception in data statistics coordinator: {} executor", - operatorName, - t); - ExceptionUtils.rethrowException(t); - return null; - } - }; - - coordinatorExecutor.submit(guardedCallable).get(); - } catch (InterruptedException | ExecutionException e) { - throw new FlinkRuntimeException(errorMessage, e); - } - } else { - try { - callable.call(); - } catch (Throwable t) { - LOG.error( - "Uncaught Exception in data statistics coordinator: {} executor", operatorName, t); - throw new FlinkRuntimeException(errorMessage, t); - } - } - } - - public void runInCoordinatorThread(Runnable runnable) { - this.coordinatorExecutor.execute( - new ThrowableCatchingRunnable( - throwable -> - this.coordinatorThreadFactory.uncaughtException(Thread.currentThread(), throwable), - runnable)); - } - - private void runInCoordinatorThread(ThrowingRunnable action, String actionString) { - ensureStarted(); - runInCoordinatorThread( - () -> { - try { - action.run(); - } catch (Throwable t) { - ExceptionUtils.rethrowIfFatalErrorOrOOM(t); - LOG.error( - "Uncaught exception in the data statistics coordinator: {} while {}. Triggering job failover", - operatorName, - actionString, - t); - context.failJob(t); - } - }); - } - - private void ensureStarted() { - Preconditions.checkState(started, "The coordinator of %s has not started yet.", operatorName); - } - - private void handleDataStatisticRequest(int subtask, StatisticsEvent event) { - CompletedStatistics maybeCompletedStatistics = - aggregatedStatisticsTracker.updateAndCheckCompletion(subtask, event); - - if (maybeCompletedStatistics != null) { - if (maybeCompletedStatistics.isEmpty()) { - LOG.info( - "Skip aggregated statistics for checkpoint {} as it is empty.", event.checkpointId()); - } else { - LOG.info("Completed statistics aggregation for checkpoint {}", event.checkpointId()); - // completedStatistics contains the complete samples, which is needed to compute - // the range bounds in globalStatistics if downstreamParallelism changed. - this.completedStatistics = maybeCompletedStatistics; - // globalStatistics only contains assignment calculated based on Map or Sketch statistics - this.globalStatistics = - globalStatistics( - maybeCompletedStatistics, - downstreamParallelism, - comparator, - closeFileCostWeightPercentage); - sendGlobalStatisticsToSubtasks(globalStatistics); - } - } - } - - private static GlobalStatistics globalStatistics( - CompletedStatistics completedStatistics, - int downstreamParallelism, - Comparator comparator, - double closeFileCostWeightPercentage) { - if (completedStatistics.type() == StatisticsType.Sketch) { - // range bound is a much smaller array compared to the complete samples. - // It helps reduce the amount of data transfer from coordinator to operator subtasks. - return GlobalStatistics.fromRangeBounds( - completedStatistics.checkpointId(), - SketchUtil.rangeBounds( - downstreamParallelism, comparator, completedStatistics.keySamples())); - } else { - return GlobalStatistics.fromMapAssignment( - completedStatistics.checkpointId(), - MapAssignment.fromKeyFrequency( - downstreamParallelism, - completedStatistics.keyFrequency(), - closeFileCostWeightPercentage, - comparator)); - } - } - - @SuppressWarnings("FutureReturnValueIgnored") - private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { - runInCoordinatorThread( - () -> { - LOG.info( - "Broadcast latest global statistics from checkpoint {} to all subtasks", - statistics.checkpointId()); - // applyImmediately is set to false so that operator subtasks can - // apply the change at checkpoint boundary - StatisticsEvent statisticsEvent = - StatisticsEvent.createGlobalStatisticsEvent( - statistics, globalStatisticsSerializer, false); - for (int i = 0; i < context.currentParallelism(); ++i) { - // Ignore future return value for potential error (e.g. subtask down). - // Upon restart, subtasks send request to coordinator to refresh statistics - // if there is any difference - subtaskGateways.getSubtaskGateway(i).sendEvent(statisticsEvent); - } - }, - String.format( - "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); - } - - @SuppressWarnings("FutureReturnValueIgnored") - private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatisticsEvent event) { - if (globalStatistics != null) { - runInCoordinatorThread( - () -> { - if (event.signature() != null && event.signature() != globalStatistics.hashCode()) { - LOG.debug( - "Skip responding to statistics request from subtask {}, as hashCode matches or not included in the request", - subtask); - } else { - LOG.info( - "Send latest global statistics from checkpoint {} to subtask {}", - globalStatistics.checkpointId(), - subtask); - StatisticsEvent statisticsEvent = - StatisticsEvent.createGlobalStatisticsEvent( - globalStatistics, globalStatisticsSerializer, true); - subtaskGateways.getSubtaskGateway(subtask).sendEvent(statisticsEvent); - } - }, - String.format( - "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); - } else { - LOG.info( - "Ignore global statistics request from subtask {} as statistics not available", subtask); - } - } - - @Override - public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEvent event) { - runInCoordinatorThread( - () -> { - LOG.debug( - "Handling event from subtask {} (#{}) of {}: {}", - subtask, - attemptNumber, - operatorName, - event); - if (event instanceof StatisticsEvent) { - handleDataStatisticRequest(subtask, ((StatisticsEvent) event)); - } else if (event instanceof RequestGlobalStatisticsEvent) { - handleRequestGlobalStatisticsEvent(subtask, (RequestGlobalStatisticsEvent) event); - } else { - throw new IllegalArgumentException( - "Invalid operator event type: " + event.getClass().getCanonicalName()); - } - }, - String.format( - "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); - } - - @Override - public void checkpointCoordinator(long checkpointId, CompletableFuture resultFuture) { - runInCoordinatorThread( - () -> { - LOG.debug( - "Snapshotting data statistics coordinator {} for checkpoint {}", - operatorName, - checkpointId); - if (completedStatistics == null) { - // null checkpoint result is not allowed, hence supply an empty byte array - resultFuture.complete(new byte[0]); - } else { - resultFuture.complete( - StatisticsUtil.serializeCompletedStatistics( - completedStatistics, completedStatisticsSerializer)); - } - }, - String.format("taking checkpoint %d", checkpointId)); - } - - @Override - public void notifyCheckpointComplete(long checkpointId) {} - - @Override - public void resetToCheckpoint(long checkpointId, byte[] checkpointData) { - Preconditions.checkState( - !started, "The coordinator %s can only be reset if it was not yet started", operatorName); - if (checkpointData == null || checkpointData.length == 0) { - LOG.info( - "Data statistic coordinator {} has nothing to restore from checkpoint {}", - operatorName, - checkpointId); - return; - } - - LOG.info( - "Restoring data statistic coordinator {} from checkpoint {}", operatorName, checkpointId); - this.completedStatistics = - StatisticsUtil.deserializeCompletedStatistics( - checkpointData, completedStatisticsSerializer); - // recompute global statistics in case downstream parallelism changed - this.globalStatistics = - globalStatistics( - completedStatistics, downstreamParallelism, comparator, closeFileCostWeightPercentage); - } - - @Override - public void subtaskReset(int subtask, long checkpointId) { - runInCoordinatorThread( - () -> { - LOG.info( - "Operator {} subtask {} is reset to checkpoint {}", - operatorName, - subtask, - checkpointId); - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways.reset(subtask); - }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); - } - - @Override - public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Throwable reason) { - runInCoordinatorThread( - () -> { - LOG.info( - "Unregistering gateway after failure for subtask {} (#{}) of data statistics {}", - subtask, - attemptNumber, - operatorName); - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); - }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); - } - - @Override - public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway gateway) { - Preconditions.checkArgument(subtask == gateway.getSubtask()); - Preconditions.checkArgument(attemptNumber == gateway.getExecution().getAttemptNumber()); - runInCoordinatorThread( - () -> { - Preconditions.checkState( - this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); - subtaskGateways.registerSubtaskGateway(gateway); - }, - String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); - } - - @VisibleForTesting - CompletedStatistics completedStatistics() { - return completedStatistics; - } - - @VisibleForTesting - GlobalStatistics globalStatistics() { - return globalStatistics; - } - - private static class SubtaskGateways { - private final String operatorName; - private final Map[] gateways; - - @SuppressWarnings("unchecked") - private SubtaskGateways(String operatorName, int parallelism) { - this.operatorName = operatorName; - gateways = new Map[parallelism]; - - for (int i = 0; i < parallelism; ++i) { - gateways[i] = Maps.newHashMap(); - } - } - - private void registerSubtaskGateway(OperatorCoordinator.SubtaskGateway gateway) { - int subtaskIndex = gateway.getSubtask(); - int attemptNumber = gateway.getExecution().getAttemptNumber(); - Preconditions.checkState( - !gateways[subtaskIndex].containsKey(attemptNumber), - "Coordinator of %s already has a subtask gateway for %d (#%d)", - operatorName, - subtaskIndex, - attemptNumber); - LOG.debug( - "Coordinator of {} registers gateway for subtask {} attempt {}", - operatorName, - subtaskIndex, - attemptNumber); - gateways[subtaskIndex].put(attemptNumber, gateway); - } - - private void unregisterSubtaskGateway(int subtaskIndex, int attemptNumber) { - LOG.debug( - "Coordinator of {} unregisters gateway for subtask {} attempt {}", - operatorName, - subtaskIndex, - attemptNumber); - gateways[subtaskIndex].remove(attemptNumber); - } - - private OperatorCoordinator.SubtaskGateway getSubtaskGateway(int subtaskIndex) { - Preconditions.checkState( - !gateways[subtaskIndex].isEmpty(), - "Coordinator of %s subtask %d is not ready yet to receive events", - operatorName, - subtaskIndex); - return Iterables.getOnlyElement(gateways[subtaskIndex].values()); - } - - private void reset(int subtaskIndex) { - gateways[subtaskIndex].clear(); - } - } - - private static class CoordinatorExecutorThreadFactory - implements ThreadFactory, Thread.UncaughtExceptionHandler { - - private final String coordinatorThreadName; - private final ClassLoader classLoader; - private final Thread.UncaughtExceptionHandler errorHandler; - - @javax.annotation.Nullable private Thread thread; - - CoordinatorExecutorThreadFactory( - final String coordinatorThreadName, final ClassLoader contextClassLoader) { - this(coordinatorThreadName, contextClassLoader, FatalExitExceptionHandler.INSTANCE); - } - - @org.apache.flink.annotation.VisibleForTesting - CoordinatorExecutorThreadFactory( - final String coordinatorThreadName, - final ClassLoader contextClassLoader, - final Thread.UncaughtExceptionHandler errorHandler) { - this.coordinatorThreadName = coordinatorThreadName; - this.classLoader = contextClassLoader; - this.errorHandler = errorHandler; - } - - @Override - public synchronized Thread newThread(@NotNull Runnable runnable) { - thread = new Thread(runnable, coordinatorThreadName); - thread.setContextClassLoader(classLoader); - thread.setUncaughtExceptionHandler(this); - return thread; - } - - @Override - public synchronized void uncaughtException(Thread t, Throwable e) { - errorHandler.uncaughtException(t, e); - } - - boolean isCurrentThreadCoordinatorThread() { - return Thread.currentThread() == thread; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.java deleted file mode 100644 index 9d7d989c298e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsCoordinatorProvider.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.flink.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; - -/** - * DataStatisticsCoordinatorProvider provides the method to create new {@link - * DataStatisticsCoordinator} - */ -@Internal -public class DataStatisticsCoordinatorProvider extends RecreateOnResetOperatorCoordinator.Provider { - - private final String operatorName; - private final Schema schema; - private final SortOrder sortOrder; - private final int downstreamParallelism; - private final StatisticsType type; - private final double closeFileCostWeightPercentage; - - public DataStatisticsCoordinatorProvider( - String operatorName, - OperatorID operatorID, - Schema schema, - SortOrder sortOrder, - int downstreamParallelism, - StatisticsType type, - double closeFileCostWeightPercentage) { - super(operatorID); - this.operatorName = operatorName; - this.schema = schema; - this.sortOrder = sortOrder; - this.downstreamParallelism = downstreamParallelism; - this.type = type; - this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; - } - - @Override - public OperatorCoordinator getCoordinator(OperatorCoordinator.Context context) { - return new DataStatisticsCoordinator( - operatorName, - context, - schema, - sortOrder, - downstreamParallelism, - type, - closeFileCostWeightPercentage); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java deleted file mode 100644 index 59c38b239725..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperator.java +++ /dev/null @@ -1,265 +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.sink.shuffle; - -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; -import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; -import org.apache.flink.runtime.operators.coordination.OperatorEventHandler; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * DataStatisticsOperator collects traffic distribution statistics. A custom partitioner shall be - * attached to the DataStatisticsOperator output. The custom partitioner leverages the statistics to - * shuffle record to improve data clustering while maintaining relative balanced traffic - * distribution to downstream subtasks. - */ -@Internal -public class DataStatisticsOperator extends AbstractStreamOperator - implements OneInputStreamOperator, OperatorEventHandler { - - private static final long serialVersionUID = 1L; - - private final String operatorName; - private final RowDataWrapper rowDataWrapper; - private final SortKey sortKey; - private final OperatorEventGateway operatorEventGateway; - private final int downstreamParallelism; - private final StatisticsType statisticsType; - private final TypeSerializer taskStatisticsSerializer; - private final TypeSerializer globalStatisticsSerializer; - - private transient int parallelism; - private transient int subtaskIndex; - private transient ListState globalStatisticsState; - // current statistics type may be different from the config due to possible - // migration from Map statistics to Sketch statistics when high cardinality detected - private transient volatile StatisticsType taskStatisticsType; - private transient volatile DataStatistics localStatistics; - private transient volatile GlobalStatistics globalStatistics; - - DataStatisticsOperator( - String operatorName, - Schema schema, - SortOrder sortOrder, - OperatorEventGateway operatorEventGateway, - int downstreamParallelism, - StatisticsType statisticsType) { - this.operatorName = operatorName; - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - this.sortKey = new SortKey(schema, sortOrder); - this.operatorEventGateway = operatorEventGateway; - this.downstreamParallelism = downstreamParallelism; - this.statisticsType = statisticsType; - - SortKeySerializer sortKeySerializer = new SortKeySerializer(schema, sortOrder); - this.taskStatisticsSerializer = new DataStatisticsSerializer(sortKeySerializer); - this.globalStatisticsSerializer = new GlobalStatisticsSerializer(sortKeySerializer); - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - this.parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); - this.subtaskIndex = getRuntimeContext().getIndexOfThisSubtask(); - - // Use union state so that new subtasks can also restore global statistics during scale-up. - this.globalStatisticsState = - context - .getOperatorStateStore() - .getUnionListState( - new ListStateDescriptor<>("globalStatisticsState", globalStatisticsSerializer)); - - if (context.isRestored()) { - if (globalStatisticsState.get() == null - || !globalStatisticsState.get().iterator().hasNext()) { - LOG.info( - "Operator {} subtask {} doesn't have global statistics state to restore", - operatorName, - subtaskIndex); - // If Flink deprecates union state in the future, RequestGlobalStatisticsEvent can be - // leveraged to request global statistics from coordinator if new subtasks (scale-up case) - // has nothing to restore from. - } else { - GlobalStatistics restoredStatistics = globalStatisticsState.get().iterator().next(); - LOG.info( - "Operator {} subtask {} restored global statistics state", operatorName, subtaskIndex); - this.globalStatistics = restoredStatistics; - } - - // Always request for new statistics from coordinator upon task initialization. - // There are a few scenarios this is needed - // 1. downstream writer parallelism changed due to rescale. - // 2. coordinator failed to send the aggregated statistics to subtask - // (e.g. due to subtask failure at the time). - // Records may flow before coordinator can respond. Range partitioner should be - // able to continue to operate with potentially suboptimal behavior (in sketch case). - LOG.info( - "Operator {} subtask {} requests new global statistics from coordinator ", - operatorName, - subtaskIndex); - // coordinator can use the hashCode (if available) in the request event to determine - // if operator already has the latest global statistics and respond can be skipped. - // This makes the handling cheap in most situations. - RequestGlobalStatisticsEvent event = - globalStatistics != null - ? new RequestGlobalStatisticsEvent(globalStatistics.hashCode()) - : new RequestGlobalStatisticsEvent(); - operatorEventGateway.sendEventToCoordinator(event); - } - - this.taskStatisticsType = StatisticsUtil.collectType(statisticsType, globalStatistics); - this.localStatistics = - StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); - } - - @Override - public void open() throws Exception { - if (globalStatistics != null) { - output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); - } - } - - @Override - public void handleOperatorEvent(OperatorEvent event) { - Preconditions.checkArgument( - event instanceof StatisticsEvent, - String.format( - "Operator %s subtask %s received unexpected operator event %s", - operatorName, subtaskIndex, event.getClass())); - StatisticsEvent statisticsEvent = (StatisticsEvent) event; - LOG.info( - "Operator {} subtask {} received global data event from coordinator checkpoint {}", - operatorName, - subtaskIndex, - statisticsEvent.checkpointId()); - this.globalStatistics = - StatisticsUtil.deserializeGlobalStatistics( - statisticsEvent.statisticsBytes(), globalStatisticsSerializer); - checkStatisticsTypeMigration(); - // if applyImmediately not set, wait until the checkpoint time to switch - if (statisticsEvent.applyImmediately()) { - output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); - } - } - - @Override - public void processElement(StreamRecord streamRecord) { - // collect data statistics - RowData record = streamRecord.getValue(); - StructLike struct = rowDataWrapper.wrap(record); - sortKey.wrap(struct); - localStatistics.add(sortKey); - - checkStatisticsTypeMigration(); - output.collect(new StreamRecord<>(StatisticsOrRecord.fromRecord(record))); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - long checkpointId = context.getCheckpointId(); - LOG.info( - "Operator {} subtask {} snapshotting data statistics for checkpoint {}", - operatorName, - subtaskIndex, - checkpointId); - - // Pass global statistics to partitioner so that all the operators refresh statistics - // at same checkpoint barrier - if (globalStatistics != null) { - output.collect(new StreamRecord<>(StatisticsOrRecord.fromStatistics(globalStatistics))); - } - - // Only subtask 0 saves the state so that globalStatisticsState(UnionListState) stores - // an exact copy of globalStatistics - if (globalStatistics != null && getRuntimeContext().getIndexOfThisSubtask() == 0) { - globalStatisticsState.clear(); - LOG.info( - "Operator {} subtask {} saving global statistics to state", operatorName, subtaskIndex); - globalStatisticsState.add(globalStatistics); - LOG.debug( - "Operator {} subtask {} saved global statistics to state: {}", - operatorName, - subtaskIndex, - globalStatistics); - } - - // For now, local statistics are sent to coordinator at checkpoint - LOG.info( - "Operator {} Subtask {} sending local statistics to coordinator for checkpoint {}", - operatorName, - subtaskIndex, - checkpointId); - operatorEventGateway.sendEventToCoordinator( - StatisticsEvent.createTaskStatisticsEvent( - checkpointId, localStatistics, taskStatisticsSerializer)); - - // Recreate the local statistics - localStatistics = - StatisticsUtil.createTaskStatistics(taskStatisticsType, parallelism, downstreamParallelism); - } - - @SuppressWarnings("unchecked") - private void checkStatisticsTypeMigration() { - // only check if the statisticsType config is Auto and localStatistics is currently Map type - if (statisticsType == StatisticsType.Auto && localStatistics.type() == StatisticsType.Map) { - Map mapStatistics = (Map) localStatistics.result(); - // convert if local statistics has cardinality over the threshold or - // if received global statistics is already sketch type - if (mapStatistics.size() > SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD - || (globalStatistics != null && globalStatistics.type() == StatisticsType.Sketch)) { - LOG.info( - "Operator {} subtask {} switched local statistics from Map to Sketch.", - operatorName, - subtaskIndex); - this.taskStatisticsType = StatisticsType.Sketch; - this.localStatistics = - StatisticsUtil.createTaskStatistics( - taskStatisticsType, parallelism, downstreamParallelism); - SketchUtil.convertMapToSketch(mapStatistics, localStatistics::add); - } - } - } - - @VisibleForTesting - DataStatistics localStatistics() { - return localStatistics; - } - - @VisibleForTesting - GlobalStatistics globalStatistics() { - return globalStatistics; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java deleted file mode 100644 index c25481b3c1f2..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsSerializer.java +++ /dev/null @@ -1,206 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Map; -import java.util.Objects; -import org.apache.datasketches.memory.Memory; -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.EnumSerializer; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.api.common.typeutils.base.MapSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -@Internal -class DataStatisticsSerializer extends TypeSerializer { - private final TypeSerializer sortKeySerializer; - private final EnumSerializer statisticsTypeSerializer; - private final MapSerializer mapSerializer; - private final SortKeySketchSerializer sketchSerializer; - - DataStatisticsSerializer(TypeSerializer sortKeySerializer) { - this.sortKeySerializer = sortKeySerializer; - this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); - this.mapSerializer = new MapSerializer<>(sortKeySerializer, LongSerializer.INSTANCE); - this.sketchSerializer = new SortKeySketchSerializer(sortKeySerializer); - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer duplicate() { - TypeSerializer duplicateSortKeySerializer = sortKeySerializer.duplicate(); - return (duplicateSortKeySerializer == sortKeySerializer) - ? this - : new DataStatisticsSerializer(duplicateSortKeySerializer); - } - - @Override - public DataStatistics createInstance() { - return new MapDataStatistics(); - } - - @SuppressWarnings("unchecked") - @Override - public DataStatistics copy(DataStatistics obj) { - StatisticsType statisticsType = obj.type(); - if (statisticsType == StatisticsType.Map) { - MapDataStatistics from = (MapDataStatistics) obj; - Map fromStats = (Map) from.result(); - Map toStats = Maps.newHashMap(fromStats); - return new MapDataStatistics(toStats); - } else if (statisticsType == StatisticsType.Sketch) { - // because ReservoirItemsSketch doesn't expose enough public methods for cloning, - // this implementation adopted the less efficient serialization and deserialization. - SketchDataStatistics from = (SketchDataStatistics) obj; - ReservoirItemsSketch fromStats = (ReservoirItemsSketch) from.result(); - byte[] bytes = fromStats.toByteArray(sketchSerializer); - Memory memory = Memory.wrap(bytes); - ReservoirItemsSketch toStats = - ReservoirItemsSketch.heapify(memory, sketchSerializer); - return new SketchDataStatistics(toStats); - } else { - throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); - } - } - - @Override - public DataStatistics copy(DataStatistics from, DataStatistics reuse) { - // not much benefit to reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @SuppressWarnings("unchecked") - @Override - public void serialize(DataStatistics obj, DataOutputView target) throws IOException { - StatisticsType statisticsType = obj.type(); - statisticsTypeSerializer.serialize(obj.type(), target); - if (statisticsType == StatisticsType.Map) { - Map mapStatistics = (Map) obj.result(); - mapSerializer.serialize(mapStatistics, target); - } else if (statisticsType == StatisticsType.Sketch) { - ReservoirItemsSketch sketch = (ReservoirItemsSketch) obj.result(); - byte[] sketchBytes = sketch.toByteArray(sketchSerializer); - target.writeInt(sketchBytes.length); - target.write(sketchBytes); - } else { - throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); - } - } - - @Override - public DataStatistics deserialize(DataInputView source) throws IOException { - StatisticsType statisticsType = statisticsTypeSerializer.deserialize(source); - if (statisticsType == StatisticsType.Map) { - Map mapStatistics = mapSerializer.deserialize(source); - return new MapDataStatistics(mapStatistics); - } else if (statisticsType == StatisticsType.Sketch) { - int numBytes = source.readInt(); - byte[] sketchBytes = new byte[numBytes]; - source.read(sketchBytes); - Memory sketchMemory = Memory.wrap(sketchBytes); - ReservoirItemsSketch sketch = - ReservoirItemsSketch.heapify(sketchMemory, sketchSerializer); - return new SketchDataStatistics(sketch); - } else { - throw new IllegalArgumentException("Unsupported data statistics type: " + statisticsType); - } - } - - @Override - public DataStatistics deserialize(DataStatistics reuse, DataInputView source) throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - serialize(deserialize(source), target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof DataStatisticsSerializer)) { - return false; - } - - DataStatisticsSerializer other = (DataStatisticsSerializer) obj; - return Objects.equals(sortKeySerializer, other.sortKeySerializer); - } - - @Override - public int hashCode() { - return sortKeySerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new DataStatisticsSerializerSnapshot(this); - } - - public static class DataStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; - - /** Constructor for read instantiation. */ - @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public DataStatisticsSerializerSnapshot() { - super(DataStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public DataStatisticsSerializerSnapshot(DataStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers(DataStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.sortKeySerializer}; - } - - @Override - protected DataStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; - return new DataStatisticsSerializer(sortKeySerializer); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java deleted file mode 100644 index 50ec23e9f7a2..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatistics.java +++ /dev/null @@ -1,114 +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.sink.shuffle; - -import java.util.Arrays; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * This is used by {@link RangePartitioner} for guiding range partitioning. This is what is sent to - * the operator subtasks. For sketch statistics, it only contains much smaller range bounds than the - * complete raw samples. - */ -class GlobalStatistics { - private final long checkpointId; - private final StatisticsType type; - private final MapAssignment mapAssignment; - private final SortKey[] rangeBounds; - - private transient Integer hashCode; - - GlobalStatistics( - long checkpointId, StatisticsType type, MapAssignment mapAssignment, SortKey[] rangeBounds) { - Preconditions.checkArgument( - (mapAssignment != null && rangeBounds == null) - || (mapAssignment == null && rangeBounds != null), - "Invalid key assignment or range bounds: both are non-null or null"); - this.checkpointId = checkpointId; - this.type = type; - this.mapAssignment = mapAssignment; - this.rangeBounds = rangeBounds; - } - - static GlobalStatistics fromMapAssignment(long checkpointId, MapAssignment mapAssignment) { - return new GlobalStatistics(checkpointId, StatisticsType.Map, mapAssignment, null); - } - - static GlobalStatistics fromRangeBounds(long checkpointId, SortKey[] rangeBounds) { - return new GlobalStatistics(checkpointId, StatisticsType.Sketch, null, rangeBounds); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("checkpointId", checkpointId) - .add("type", type) - .add("mapAssignment", mapAssignment) - .add("rangeBounds", rangeBounds) - .toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof GlobalStatistics)) { - return false; - } - - GlobalStatistics other = (GlobalStatistics) o; - return Objects.equal(checkpointId, other.checkpointId) - && Objects.equal(type, other.type) - && Objects.equal(mapAssignment, other.mapAssignment()) - && Arrays.equals(rangeBounds, other.rangeBounds()); - } - - @Override - public int hashCode() { - // implemented caching because coordinator can call the hashCode many times. - // when subtasks request statistics refresh upon initialization for reconciliation purpose, - // hashCode is used to check if there is any difference btw coordinator and operator state. - if (hashCode == null) { - this.hashCode = Objects.hashCode(checkpointId, type, mapAssignment, rangeBounds); - } - - return hashCode; - } - - long checkpointId() { - return checkpointId; - } - - StatisticsType type() { - return type; - } - - MapAssignment mapAssignment() { - return mapAssignment; - } - - SortKey[] rangeBounds() { - return rangeBounds; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java deleted file mode 100644 index dfb947a84a0c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/GlobalStatisticsSerializer.java +++ /dev/null @@ -1,201 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.base.EnumSerializer; -import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.api.common.typeutils.base.ListSerializer; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -class GlobalStatisticsSerializer extends TypeSerializer { - private final TypeSerializer sortKeySerializer; - private final EnumSerializer statisticsTypeSerializer; - private final ListSerializer rangeBoundsSerializer; - private final ListSerializer intsSerializer; - private final ListSerializer longsSerializer; - - GlobalStatisticsSerializer(TypeSerializer sortKeySerializer) { - this.sortKeySerializer = sortKeySerializer; - this.statisticsTypeSerializer = new EnumSerializer<>(StatisticsType.class); - this.rangeBoundsSerializer = new ListSerializer<>(sortKeySerializer); - this.intsSerializer = new ListSerializer<>(IntSerializer.INSTANCE); - this.longsSerializer = new ListSerializer<>(LongSerializer.INSTANCE); - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public TypeSerializer duplicate() { - return new GlobalStatisticsSerializer(sortKeySerializer); - } - - @Override - public GlobalStatistics createInstance() { - return GlobalStatistics.fromRangeBounds(0L, new SortKey[0]); - } - - @Override - public GlobalStatistics copy(GlobalStatistics from) { - return new GlobalStatistics( - from.checkpointId(), from.type(), from.mapAssignment(), from.rangeBounds()); - } - - @Override - public GlobalStatistics copy(GlobalStatistics from, GlobalStatistics reuse) { - // no benefit of reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(GlobalStatistics record, DataOutputView target) throws IOException { - target.writeLong(record.checkpointId()); - statisticsTypeSerializer.serialize(record.type(), target); - if (record.type() == StatisticsType.Map) { - MapAssignment mapAssignment = record.mapAssignment(); - target.writeInt(mapAssignment.numPartitions()); - target.writeInt(mapAssignment.keyAssignments().size()); - for (Map.Entry entry : mapAssignment.keyAssignments().entrySet()) { - sortKeySerializer.serialize(entry.getKey(), target); - KeyAssignment keyAssignment = entry.getValue(); - intsSerializer.serialize(keyAssignment.assignedSubtasks(), target); - longsSerializer.serialize(keyAssignment.subtaskWeightsWithCloseFileCost(), target); - target.writeLong(keyAssignment.closeFileCostWeight()); - } - } else { - rangeBoundsSerializer.serialize(Arrays.asList(record.rangeBounds()), target); - } - } - - @Override - public GlobalStatistics deserialize(DataInputView source) throws IOException { - long checkpointId = source.readLong(); - StatisticsType type = statisticsTypeSerializer.deserialize(source); - if (type == StatisticsType.Map) { - int numPartitions = source.readInt(); - int mapSize = source.readInt(); - Map keyAssignments = Maps.newHashMapWithExpectedSize(mapSize); - for (int i = 0; i < mapSize; ++i) { - SortKey sortKey = sortKeySerializer.deserialize(source); - List assignedSubtasks = intsSerializer.deserialize(source); - List subtaskWeightsWithCloseFileCost = longsSerializer.deserialize(source); - long closeFileCostWeight = source.readLong(); - keyAssignments.put( - sortKey, - new KeyAssignment( - assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight)); - } - - return GlobalStatistics.fromMapAssignment( - checkpointId, new MapAssignment(numPartitions, keyAssignments)); - } else { - List sortKeys = rangeBoundsSerializer.deserialize(source); - SortKey[] rangeBounds = new SortKey[sortKeys.size()]; - return GlobalStatistics.fromRangeBounds(checkpointId, sortKeys.toArray(rangeBounds)); - } - } - - @Override - public GlobalStatistics deserialize(GlobalStatistics reuse, DataInputView source) - throws IOException { - // not much benefit to reuse - return deserialize(source); - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - serialize(deserialize(source), target); - } - - @Override - public boolean equals(Object obj) { - if (this == obj) { - return true; - } - - if (obj == null || getClass() != obj.getClass()) { - return false; - } - - GlobalStatisticsSerializer other = (GlobalStatisticsSerializer) obj; - return Objects.equals(sortKeySerializer, other.sortKeySerializer); - } - - @Override - public int hashCode() { - return sortKeySerializer.hashCode(); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new GlobalStatisticsSerializerSnapshot(this); - } - - public static class GlobalStatisticsSerializerSnapshot - extends CompositeTypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; - - /** Constructor for read instantiation. */ - @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public GlobalStatisticsSerializerSnapshot() { - super(GlobalStatisticsSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public GlobalStatisticsSerializerSnapshot(GlobalStatisticsSerializer serializer) { - super(serializer); - } - - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers(GlobalStatisticsSerializer outerSerializer) { - return new TypeSerializer[] {outerSerializer.sortKeySerializer}; - } - - @Override - protected GlobalStatisticsSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - SortKeySerializer sortKeySerializer = (SortKeySerializer) nestedSerializers[0]; - return new GlobalStatisticsSerializer(sortKeySerializer); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java deleted file mode 100644 index 781bcc646023..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/KeyAssignment.java +++ /dev/null @@ -1,155 +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.sink.shuffle; - -import java.util.Arrays; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.ThreadLocalRandom; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Subtask assignment for a key for Map statistics based */ -class KeyAssignment { - private final List assignedSubtasks; - private final List subtaskWeightsWithCloseFileCost; - private final long closeFileCostWeight; - private final long[] subtaskWeightsExcludingCloseCost; - private final long keyWeight; - private final long[] cumulativeWeights; - - /** - * @param assignedSubtasks assigned subtasks for this key. It could be a single subtask. It could - * also be multiple subtasks if the key has heavy weight that should be handled by multiple - * subtasks. - * @param subtaskWeightsWithCloseFileCost assigned weight for each subtask. E.g., if the keyWeight - * is 27 and the key is assigned to 3 subtasks, subtaskWeights could contain values as [10, - * 10, 7] for target weight of 10 per subtask. - */ - KeyAssignment( - List assignedSubtasks, - List subtaskWeightsWithCloseFileCost, - long closeFileCostWeight) { - Preconditions.checkArgument( - assignedSubtasks != null && !assignedSubtasks.isEmpty(), - "Invalid assigned subtasks: null or empty"); - Preconditions.checkArgument( - subtaskWeightsWithCloseFileCost != null && !subtaskWeightsWithCloseFileCost.isEmpty(), - "Invalid assigned subtasks weights: null or empty"); - Preconditions.checkArgument( - assignedSubtasks.size() == subtaskWeightsWithCloseFileCost.size(), - "Invalid assignment: size mismatch (tasks length = %s, weights length = %s)", - assignedSubtasks.size(), - subtaskWeightsWithCloseFileCost.size()); - subtaskWeightsWithCloseFileCost.forEach( - weight -> - Preconditions.checkArgument( - weight > closeFileCostWeight, - "Invalid weight: should be larger than close file cost: weight = %s, close file cost = %s", - weight, - closeFileCostWeight)); - - this.assignedSubtasks = assignedSubtasks; - this.subtaskWeightsWithCloseFileCost = subtaskWeightsWithCloseFileCost; - this.closeFileCostWeight = closeFileCostWeight; - // Exclude the close file cost for key routing - this.subtaskWeightsExcludingCloseCost = - subtaskWeightsWithCloseFileCost.stream() - .mapToLong(weightWithCloseFileCost -> weightWithCloseFileCost - closeFileCostWeight) - .toArray(); - this.keyWeight = Arrays.stream(subtaskWeightsExcludingCloseCost).sum(); - this.cumulativeWeights = new long[subtaskWeightsExcludingCloseCost.length]; - long cumulativeWeight = 0; - for (int i = 0; i < subtaskWeightsExcludingCloseCost.length; ++i) { - cumulativeWeight += subtaskWeightsExcludingCloseCost[i]; - cumulativeWeights[i] = cumulativeWeight; - } - } - - List assignedSubtasks() { - return assignedSubtasks; - } - - List subtaskWeightsWithCloseFileCost() { - return subtaskWeightsWithCloseFileCost; - } - - long closeFileCostWeight() { - return closeFileCostWeight; - } - - long[] subtaskWeightsExcludingCloseCost() { - return subtaskWeightsExcludingCloseCost; - } - - /** - * Select a subtask for the key. - * - * @return subtask id - */ - int select() { - if (assignedSubtasks.size() == 1) { - // only choice. no need to run random number generator. - return assignedSubtasks.get(0); - } else { - long randomNumber = ThreadLocalRandom.current().nextLong(keyWeight); - int index = Arrays.binarySearch(cumulativeWeights, randomNumber); - // choose the subtask where randomNumber < cumulativeWeights[pos]. - // this works regardless whether index is negative or not. - int position = Math.abs(index + 1); - Preconditions.checkState( - position < assignedSubtasks.size(), - "Invalid selected position: out of range. key weight = %s, random number = %s, cumulative weights array = %s", - keyWeight, - randomNumber, - cumulativeWeights); - return assignedSubtasks.get(position); - } - } - - @Override - public int hashCode() { - return Objects.hash(assignedSubtasks, subtaskWeightsWithCloseFileCost, closeFileCostWeight); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - KeyAssignment that = (KeyAssignment) o; - return Objects.equals(assignedSubtasks, that.assignedSubtasks) - && Objects.equals(subtaskWeightsWithCloseFileCost, that.subtaskWeightsWithCloseFileCost) - && closeFileCostWeight == that.closeFileCostWeight; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("assignedSubtasks", assignedSubtasks) - .add("subtaskWeightsWithCloseFileCost", subtaskWeightsWithCloseFileCost) - .add("closeFileCostWeight", closeFileCostWeight) - .toString(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java deleted file mode 100644 index 9d8167460a1b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapAssignment.java +++ /dev/null @@ -1,242 +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.sink.shuffle; - -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.NavigableMap; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -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.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.Pair; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** Key assignment to subtasks for Map statistics. */ -class MapAssignment { - private static final Logger LOG = LoggerFactory.getLogger(MapAssignment.class); - - private final int numPartitions; - private final Map keyAssignments; - - MapAssignment(int numPartitions, Map keyAssignments) { - Preconditions.checkArgument(keyAssignments != null, "Invalid key assignments: null"); - this.numPartitions = numPartitions; - this.keyAssignments = keyAssignments; - } - - static MapAssignment fromKeyFrequency( - int numPartitions, - Map mapStatistics, - double closeFileCostWeightPercentage, - Comparator comparator) { - return new MapAssignment( - numPartitions, - assignment(numPartitions, mapStatistics, closeFileCostWeightPercentage, comparator)); - } - - @Override - public int hashCode() { - return Objects.hashCode(numPartitions, keyAssignments); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (o == null || getClass() != o.getClass()) { - return false; - } - - MapAssignment that = (MapAssignment) o; - return numPartitions == that.numPartitions && keyAssignments.equals(that.keyAssignments); - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("numPartitions", numPartitions) - .add("keyAssignments", keyAssignments) - .toString(); - } - - int numPartitions() { - return numPartitions; - } - - Map keyAssignments() { - return keyAssignments; - } - - /** - * Returns assignment summary for every subtask. - * - * @return assignment summary for every subtask. Key is subtaskId. Value pair is (weight assigned - * to the subtask, number of keys assigned to the subtask) - */ - Map> assignmentInfo() { - Map> assignmentInfo = Maps.newTreeMap(); - keyAssignments.forEach( - (key, keyAssignment) -> { - for (int i = 0; i < keyAssignment.assignedSubtasks().size(); ++i) { - int subtaskId = keyAssignment.assignedSubtasks().get(i); - long subtaskWeight = keyAssignment.subtaskWeightsExcludingCloseCost()[i]; - Pair oldValue = assignmentInfo.getOrDefault(subtaskId, Pair.of(0L, 0)); - assignmentInfo.put( - subtaskId, Pair.of(oldValue.first() + subtaskWeight, oldValue.second() + 1)); - } - }); - - return assignmentInfo; - } - - static Map assignment( - int numPartitions, - Map mapStatistics, - double closeFileCostWeightPercentage, - Comparator comparator) { - mapStatistics.forEach( - (key, value) -> - Preconditions.checkArgument( - value > 0, "Invalid statistics: weight is 0 for key %s", key)); - - long totalWeight = mapStatistics.values().stream().mapToLong(l -> l).sum(); - double targetWeightPerSubtask = ((double) totalWeight) / numPartitions; - long closeFileCostWeight = - (long) Math.ceil(targetWeightPerSubtask * closeFileCostWeightPercentage / 100); - - NavigableMap sortedStatsWithCloseFileCost = Maps.newTreeMap(comparator); - mapStatistics.forEach( - (k, v) -> { - int estimatedSplits = (int) Math.ceil(v / targetWeightPerSubtask); - long estimatedCloseFileCost = closeFileCostWeight * estimatedSplits; - sortedStatsWithCloseFileCost.put(k, v + estimatedCloseFileCost); - }); - - long totalWeightWithCloseFileCost = - sortedStatsWithCloseFileCost.values().stream().mapToLong(l -> l).sum(); - long targetWeightPerSubtaskWithCloseFileCost = - (long) Math.ceil(((double) totalWeightWithCloseFileCost) / numPartitions); - return buildAssignment( - numPartitions, - sortedStatsWithCloseFileCost, - targetWeightPerSubtaskWithCloseFileCost, - closeFileCostWeight); - } - - private static Map buildAssignment( - int numPartitions, - NavigableMap sortedStatistics, - long targetWeightPerSubtask, - long closeFileCostWeight) { - Map assignmentMap = - Maps.newHashMapWithExpectedSize(sortedStatistics.size()); - Iterator mapKeyIterator = sortedStatistics.keySet().iterator(); - int subtaskId = 0; - SortKey currentKey = null; - long keyRemainingWeight = 0L; - long subtaskRemainingWeight = targetWeightPerSubtask; - List assignedSubtasks = Lists.newArrayList(); - List subtaskWeights = Lists.newArrayList(); - while (mapKeyIterator.hasNext() || currentKey != null) { - // This should never happen because target weight is calculated using ceil function. - if (subtaskId >= numPartitions) { - LOG.error( - "Internal algorithm error: exhausted subtasks with unassigned keys left. number of partitions: {}, " - + "target weight per subtask: {}, close file cost in weight: {}, data statistics: {}", - numPartitions, - targetWeightPerSubtask, - closeFileCostWeight, - sortedStatistics); - throw new IllegalStateException( - "Internal algorithm error: exhausted subtasks with unassigned keys left"); - } - - if (currentKey == null) { - currentKey = mapKeyIterator.next(); - keyRemainingWeight = sortedStatistics.get(currentKey); - } - - assignedSubtasks.add(subtaskId); - if (keyRemainingWeight < subtaskRemainingWeight) { - // assign the remaining weight of the key to the current subtask - subtaskWeights.add(keyRemainingWeight); - subtaskRemainingWeight -= keyRemainingWeight; - keyRemainingWeight = 0L; - } else { - // filled up the current subtask - long assignedWeight = subtaskRemainingWeight; - keyRemainingWeight -= subtaskRemainingWeight; - - // If assigned weight is less than close file cost, pad it up with close file cost. - // This might cause the subtask assigned weight over the target weight. - // But it should be no more than one close file cost. Small skew is acceptable. - if (assignedWeight <= closeFileCostWeight) { - long paddingWeight = Math.min(keyRemainingWeight, closeFileCostWeight); - keyRemainingWeight -= paddingWeight; - assignedWeight += paddingWeight; - } - - subtaskWeights.add(assignedWeight); - // move on to the next subtask - subtaskId += 1; - subtaskRemainingWeight = targetWeightPerSubtask; - } - - Preconditions.checkState( - assignedSubtasks.size() == subtaskWeights.size(), - "List size mismatch: assigned subtasks = %s, subtask weights = %s", - assignedSubtasks, - subtaskWeights); - - // If the remaining key weight is smaller than the close file cost, simply skip the residual - // as it doesn't make sense to assign a weight smaller than close file cost to a new subtask. - // this might lead to some inaccuracy in weight calculation. E.g., assuming the key weight is - // 2 and close file cost is 2. key weight with close cost is 4. Let's assume the previous - // task has a weight of 3 available. So weight of 3 for this key is assigned to the task and - // the residual weight of 1 is dropped. Then the routing weight for this key is 1 (minus the - // close file cost), which is inaccurate as the true key weight should be 2. - // Again, this greedy algorithm is not intended to be perfect. Some small inaccuracy is - // expected and acceptable. Traffic distribution should still be balanced. - if (keyRemainingWeight > 0 && keyRemainingWeight <= closeFileCostWeight) { - keyRemainingWeight = 0; - } - - if (keyRemainingWeight == 0) { - // finishing up the assignment for the current key - KeyAssignment keyAssignment = - new KeyAssignment(assignedSubtasks, subtaskWeights, closeFileCostWeight); - assignmentMap.put(currentKey, keyAssignment); - assignedSubtasks = Lists.newArrayList(); - subtaskWeights = Lists.newArrayList(); - currentKey = null; - } - } - - return assignmentMap; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java deleted file mode 100644 index 05b943f6046f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapDataStatistics.java +++ /dev/null @@ -1,88 +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.sink.shuffle; - -import java.util.Map; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -/** MapDataStatistics uses map to count key frequency */ -class MapDataStatistics implements DataStatistics { - private final Map keyFrequency; - - MapDataStatistics() { - this.keyFrequency = Maps.newHashMap(); - } - - MapDataStatistics(Map keyFrequency) { - this.keyFrequency = keyFrequency; - } - - @Override - public StatisticsType type() { - return StatisticsType.Map; - } - - @Override - public boolean isEmpty() { - return keyFrequency.isEmpty(); - } - - @Override - public void add(SortKey sortKey) { - if (keyFrequency.containsKey(sortKey)) { - keyFrequency.merge(sortKey, 1L, Long::sum); - } else { - // clone the sort key before adding to map because input sortKey object can be reused - SortKey copiedKey = sortKey.copy(); - keyFrequency.put(copiedKey, 1L); - } - } - - @Override - public Object result() { - return keyFrequency; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("map", keyFrequency).toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof MapDataStatistics)) { - return false; - } - - MapDataStatistics other = (MapDataStatistics) o; - return Objects.equal(keyFrequency, other.keyFrequency); - } - - @Override - public int hashCode() { - return Objects.hashCode(keyFrequency); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java deleted file mode 100644 index f36a078c94e0..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitioner.java +++ /dev/null @@ -1,95 +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.sink.shuffle; - -import java.util.concurrent.TimeUnit; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Internal partitioner implementation that supports MapDataStatistics, which is typically used for - * low-cardinality use cases. While MapDataStatistics can keep accurate counters, it can't be used - * for high-cardinality use cases. Otherwise, the memory footprint is too high. - * - *

    It is a greedy algorithm for bin packing. With close file cost, the calculation isn't always - * precise when calculating close cost for every file, target weight per subtask, padding residual - * weight, assigned weight without close cost. - * - *

    All actions should be executed in a single Flink mailbox thread. So there is no need to make - * it thread safe. - */ -class MapRangePartitioner implements Partitioner { - private static final Logger LOG = LoggerFactory.getLogger(MapRangePartitioner.class); - - private final RowDataWrapper rowDataWrapper; - private final SortKey sortKey; - private final MapAssignment mapAssignment; - - // Counter that tracks how many times a new key encountered - // where there is no traffic statistics learned about it. - private long newSortKeyCounter; - private long lastNewSortKeyLogTimeMilli; - - MapRangePartitioner(Schema schema, SortOrder sortOrder, MapAssignment mapAssignment) { - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - this.sortKey = new SortKey(schema, sortOrder); - this.mapAssignment = mapAssignment; - this.newSortKeyCounter = 0; - this.lastNewSortKeyLogTimeMilli = System.currentTimeMillis(); - } - - @Override - public int partition(RowData row, int numPartitions) { - // reuse the sortKey and rowDataWrapper - sortKey.wrap(rowDataWrapper.wrap(row)); - KeyAssignment keyAssignment = mapAssignment.keyAssignments().get(sortKey); - - int partition; - if (keyAssignment == null) { - LOG.trace( - "Encountered new sort key: {}. Fall back to round robin as statistics not learned yet.", - sortKey); - // Ideally unknownKeyCounter should be published as a counter metric. - // It seems difficult to pass in MetricGroup into the partitioner. - // Just log an INFO message every minute. - newSortKeyCounter += 1; - long now = System.currentTimeMillis(); - if (now - lastNewSortKeyLogTimeMilli > TimeUnit.MINUTES.toMillis(1)) { - LOG.info( - "Encounter new sort keys {} times. Fall back to round robin as statistics not learned yet", - newSortKeyCounter); - lastNewSortKeyLogTimeMilli = now; - newSortKeyCounter = 0; - } - partition = (int) (newSortKeyCounter % numPartitions); - } else { - partition = keyAssignment.select(); - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, mapAssignment.numPartitions(), numPartitions); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java deleted file mode 100644 index 6608b938f5a8..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RangePartitioner.java +++ /dev/null @@ -1,110 +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.sink.shuffle; - -import java.util.Random; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** This custom partitioner implements the {@link DistributionMode#RANGE} for Flink sink. */ -@Internal -public class RangePartitioner implements Partitioner { - private static final Logger LOG = LoggerFactory.getLogger(RangePartitioner.class); - - private final Schema schema; - private final SortOrder sortOrder; - - private transient AtomicLong roundRobinCounter; - private transient Partitioner delegatePartitioner; - - public RangePartitioner(Schema schema, SortOrder sortOrder) { - this.schema = schema; - this.sortOrder = sortOrder; - } - - @Override - public int partition(StatisticsOrRecord wrapper, int numPartitions) { - if (wrapper.hasStatistics()) { - this.delegatePartitioner = delegatePartitioner(wrapper.statistics()); - return (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); - } else { - if (delegatePartitioner != null) { - return delegatePartitioner.partition(wrapper.record(), numPartitions); - } else { - int partition = (int) (roundRobinCounter(numPartitions).getAndIncrement() % numPartitions); - LOG.trace("Statistics not available. Round robin to partition {}", partition); - return partition; - } - } - } - - private AtomicLong roundRobinCounter(int numPartitions) { - if (roundRobinCounter == null) { - // randomize the starting point to avoid synchronization across subtasks - this.roundRobinCounter = new AtomicLong(new Random().nextInt(numPartitions)); - } - - return roundRobinCounter; - } - - private Partitioner delegatePartitioner(GlobalStatistics statistics) { - if (statistics.type() == StatisticsType.Map) { - return new MapRangePartitioner(schema, sortOrder, statistics.mapAssignment()); - } else if (statistics.type() == StatisticsType.Sketch) { - return new SketchRangePartitioner(schema, sortOrder, statistics.rangeBounds()); - } else { - throw new IllegalArgumentException( - String.format("Invalid statistics type: %s. Should be Map or Sketch", statistics.type())); - } - } - - /** - * Util method that handles rescale (write parallelism / numPartitions change). - * - * @param partition partition caculated based on the existing statistics - * @param numPartitionsStatsCalculation number of partitions when the assignment was calculated - * based on - * @param numPartitions current number of partitions - * @return adjusted partition if necessary. - */ - static int adjustPartitionWithRescale( - int partition, int numPartitionsStatsCalculation, int numPartitions) { - if (numPartitionsStatsCalculation <= numPartitions) { - // no rescale or scale-up case. - // new subtasks are ignored and not assigned any keys, which is sub-optimal and only - // transient. when rescale is detected, operator requests new statistics from - // coordinator upon initialization. - return partition; - } else { - // scale-down case. - // Use mod % operation to distribution the over-range partitions. - // It can cause skew among subtasks. but the behavior is still better than - // discarding the statistics and falling back to round-robin (no clustering). - // Again, this is transient and stats refresh is requested when rescale is detected. - return partition % numPartitions; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java deleted file mode 100644 index ce17e1964392..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.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.sink.shuffle; - -import org.apache.flink.runtime.operators.coordination.OperatorEvent; - -class RequestGlobalStatisticsEvent implements OperatorEvent { - private final Integer signature; - - RequestGlobalStatisticsEvent() { - this.signature = null; - } - - /** @param signature hashCode of the subtask's existing global statistics */ - RequestGlobalStatisticsEvent(int signature) { - this.signature = signature; - } - - Integer signature() { - return signature; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java deleted file mode 100644 index 35bbb27baf16..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchDataStatistics.java +++ /dev/null @@ -1,87 +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.sink.shuffle; - -import java.util.Arrays; -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; - -/** MapDataStatistics uses map to count key frequency */ -class SketchDataStatistics implements DataStatistics { - - private final ReservoirItemsSketch sketch; - - SketchDataStatistics(int reservoirSize) { - this.sketch = ReservoirItemsSketch.newInstance(reservoirSize); - } - - SketchDataStatistics(ReservoirItemsSketch sketchStats) { - this.sketch = sketchStats; - } - - @Override - public StatisticsType type() { - return StatisticsType.Sketch; - } - - @Override - public boolean isEmpty() { - return sketch.getNumSamples() == 0; - } - - @Override - public void add(SortKey sortKey) { - // clone the sort key first because input sortKey object can be reused - SortKey copiedKey = sortKey.copy(); - sketch.update(copiedKey); - } - - @Override - public Object result() { - return sketch; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this).add("sketch", sketch).toString(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!(o instanceof SketchDataStatistics)) { - return false; - } - - ReservoirItemsSketch otherSketch = ((SketchDataStatistics) o).sketch; - return Objects.equal(sketch.getK(), otherSketch.getK()) - && Objects.equal(sketch.getN(), otherSketch.getN()) - && Arrays.deepEquals(sketch.getSamples(), otherSketch.getSamples()); - } - - @Override - public int hashCode() { - return Objects.hashCode(sketch.getK(), sketch.getN(), sketch.getSamples()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java deleted file mode 100644 index dddb0d8722c0..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchRangePartitioner.java +++ /dev/null @@ -1,51 +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.sink.shuffle; - -import java.util.Comparator; -import org.apache.flink.api.common.functions.Partitioner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.types.Comparators; - -class SketchRangePartitioner implements Partitioner { - private final SortKey sortKey; - private final Comparator comparator; - private final SortKey[] rangeBounds; - private final RowDataWrapper rowDataWrapper; - - SketchRangePartitioner(Schema schema, SortOrder sortOrder, SortKey[] rangeBounds) { - this.sortKey = new SortKey(schema, sortOrder); - this.comparator = Comparators.forType(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()); - this.rangeBounds = rangeBounds; - this.rowDataWrapper = new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - } - - @Override - public int partition(RowData row, int numPartitions) { - // reuse the sortKey and rowDataWrapper - sortKey.wrap(rowDataWrapper.wrap(row)); - return SketchUtil.partition(sortKey, numPartitions, rangeBounds, comparator); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java deleted file mode 100644 index 871ef9ef1149..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SketchUtil.java +++ /dev/null @@ -1,159 +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.sink.shuffle; - -import java.util.Arrays; -import java.util.Comparator; -import java.util.Map; -import java.util.function.Consumer; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.StructLike; - -class SketchUtil { - static final int COORDINATOR_MIN_RESERVOIR_SIZE = 10_000; - static final int COORDINATOR_MAX_RESERVOIR_SIZE = 1_000_000; - static final int COORDINATOR_TARGET_PARTITIONS_MULTIPLIER = 100; - static final int OPERATOR_OVER_SAMPLE_RATIO = 10; - - // switch the statistics tracking from map to sketch if the cardinality of the sort key is over - // this threshold. It is hardcoded for now, we can revisit in the future if config is needed. - static final int OPERATOR_SKETCH_SWITCH_THRESHOLD = 10_000; - static final int COORDINATOR_SKETCH_SWITCH_THRESHOLD = 100_000; - - private SketchUtil() {} - - /** - * The larger the reservoir size, the more accurate for range bounds calculation and the more - * balanced range distribution. - * - *

    Here are the heuristic rules - *

  • Target size: numPartitions x 100 to achieve good accuracy and is easier to calculate the - * range bounds - *
  • Min is 10K to achieve good accuracy while memory footprint is still relatively small - *
  • Max is 1M to cap the memory footprint on coordinator - * - * @param numPartitions number of range partitions which equals to downstream operator parallelism - * @return reservoir size - */ - static int determineCoordinatorReservoirSize(int numPartitions) { - int reservoirSize = numPartitions * COORDINATOR_TARGET_PARTITIONS_MULTIPLIER; - - if (reservoirSize < COORDINATOR_MIN_RESERVOIR_SIZE) { - // adjust it up and still make reservoirSize divisible by numPartitions - int remainder = COORDINATOR_MIN_RESERVOIR_SIZE % numPartitions; - reservoirSize = COORDINATOR_MIN_RESERVOIR_SIZE + (numPartitions - remainder); - } else if (reservoirSize > COORDINATOR_MAX_RESERVOIR_SIZE) { - // adjust it down and still make reservoirSize divisible by numPartitions - int remainder = COORDINATOR_MAX_RESERVOIR_SIZE % numPartitions; - reservoirSize = COORDINATOR_MAX_RESERVOIR_SIZE - remainder; - } - - return reservoirSize; - } - - /** - * Determine the sampling reservoir size where operator subtasks collect data statistics. - * - *

    Here are the heuristic rules - *

  • Target size is "coordinator reservoir size * over sampling ration (10) / operator - * parallelism" - *
  • Min is 1K to achieve good accuracy while memory footprint is still relatively small - *
  • Max is 100K to cap the memory footprint on coordinator - * - * @param numPartitions number of range partitions which equals to downstream operator parallelism - * @param operatorParallelism data statistics operator parallelism - * @return reservoir size - */ - static int determineOperatorReservoirSize(int operatorParallelism, int numPartitions) { - int coordinatorReservoirSize = determineCoordinatorReservoirSize(numPartitions); - int totalOperatorSamples = coordinatorReservoirSize * OPERATOR_OVER_SAMPLE_RATIO; - return (int) Math.ceil((double) totalOperatorSamples / operatorParallelism); - } - - /** - * To understand how range bounds are used in range partitioning, here is an example for human - * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be - * - *
      - *
    • age <= 15 - *
    • age > 15 && age <= 32 - *
    • age >32 && age <= 60 - *
    • age > 60 - *
    - * - *

    Assumption is that a single key is not dominant enough to span multiple subtasks. - * - * @param numPartitions number of partitions which maps to downstream operator parallelism - * @param samples sampled keys - * @return array of range partition bounds. It should be a sorted list (ascending). Number of - * items should be {@code numPartitions - 1}. if numPartitions is 1, return an empty list - */ - static SortKey[] rangeBounds( - int numPartitions, Comparator comparator, SortKey[] samples) { - // sort the keys first - Arrays.sort(samples, comparator); - int numCandidates = numPartitions - 1; - SortKey[] candidates = new SortKey[numCandidates]; - int step = (int) Math.ceil((double) samples.length / numPartitions); - int position = step - 1; - int numChosen = 0; - while (position < samples.length && numChosen < numCandidates) { - SortKey candidate = samples[position]; - // skip duplicate values - if (numChosen > 0 && candidate.equals(candidates[numChosen - 1])) { - // linear probe for the next distinct value - position += 1; - } else { - candidates[numChosen] = candidate; - position += step; - numChosen += 1; - } - } - - return candidates; - } - - /** This can be a bit expensive since it is quadratic. */ - static void convertMapToSketch( - Map taskMapStats, Consumer sketchConsumer) { - taskMapStats.forEach( - (sortKey, count) -> { - for (int i = 0; i < count; ++i) { - sketchConsumer.accept(sortKey); - } - }); - } - - static int partition( - SortKey key, int numPartitions, SortKey[] rangeBounds, Comparator comparator) { - int partition = Arrays.binarySearch(rangeBounds, key, comparator); - - // binarySearch either returns the match location or -[insertion point]-1 - if (partition < 0) { - partition = -partition - 1; - } - - if (partition > rangeBounds.length) { - partition = rangeBounds.length; - } - - return RangePartitioner.adjustPartitionWithRescale( - partition, rangeBounds.length + 1, numPartitions); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java deleted file mode 100644 index d1d75019fa2e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySerializer.java +++ /dev/null @@ -1,373 +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.sink.shuffle; - -import java.io.IOException; -import java.math.BigDecimal; -import java.math.BigInteger; -import java.nio.ByteBuffer; -import java.util.List; -import java.util.Objects; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.StringUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.SortField; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderParser; -import org.apache.iceberg.types.CheckCompatibility; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.types.Types; - -class SortKeySerializer extends TypeSerializer { - private final Schema schema; - private final SortOrder sortOrder; - private final int size; - private final Types.NestedField[] transformedFields; - - private transient SortKey sortKey; - - SortKeySerializer(Schema schema, SortOrder sortOrder) { - this.schema = schema; - this.sortOrder = sortOrder; - this.size = sortOrder.fields().size(); - - this.transformedFields = new Types.NestedField[size]; - for (int i = 0; i < size; ++i) { - SortField sortField = sortOrder.fields().get(i); - Types.NestedField sourceField = schema.findField(sortField.sourceId()); - Type resultType = sortField.transform().getResultType(sourceField.type()); - Types.NestedField transformedField = - Types.NestedField.of( - sourceField.fieldId(), - sourceField.isOptional(), - sourceField.name(), - resultType, - sourceField.doc()); - transformedFields[i] = transformedField; - } - } - - private SortKey lazySortKey() { - if (sortKey == null) { - this.sortKey = new SortKey(schema, sortOrder); - } - - return sortKey; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @Override - public TypeSerializer duplicate() { - return new SortKeySerializer(schema, sortOrder); - } - - @Override - public SortKey createInstance() { - return new SortKey(schema, sortOrder); - } - - @Override - public SortKey copy(SortKey from) { - return from.copy(); - } - - @Override - public SortKey copy(SortKey from, SortKey reuse) { - // no benefit of reuse - return copy(from); - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(SortKey record, DataOutputView target) throws IOException { - Preconditions.checkArgument( - record.size() == size, - "Invalid size of the sort key object: %s. Expected %s", - record.size(), - size); - for (int i = 0; i < size; ++i) { - int fieldId = transformedFields[i].fieldId(); - Type.TypeID typeId = transformedFields[i].type().typeId(); - switch (typeId) { - case BOOLEAN: - target.writeBoolean(record.get(i, Boolean.class)); - break; - case INTEGER: - case DATE: - target.writeInt(record.get(i, Integer.class)); - break; - case LONG: - case TIME: - case TIMESTAMP: - target.writeLong(record.get(i, Long.class)); - break; - case FLOAT: - target.writeFloat(record.get(i, Float.class)); - break; - case DOUBLE: - target.writeDouble(record.get(i, Double.class)); - break; - case STRING: - target.writeUTF(record.get(i, CharSequence.class).toString()); - break; - case UUID: - UUID uuid = record.get(i, UUID.class); - target.writeLong(uuid.getMostSignificantBits()); - target.writeLong(uuid.getLeastSignificantBits()); - break; - case FIXED: - case BINARY: - byte[] bytes = record.get(i, ByteBuffer.class).array(); - target.writeInt(bytes.length); - target.write(bytes); - break; - case DECIMAL: - BigDecimal decimal = record.get(i, BigDecimal.class); - byte[] decimalBytes = decimal.unscaledValue().toByteArray(); - target.writeInt(decimalBytes.length); - target.write(decimalBytes); - target.writeInt(decimal.scale()); - break; - case STRUCT: - case MAP: - case LIST: - default: - // SortKey transformation is a flattened struct without list and map - throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); - } - } - } - - @Override - public SortKey deserialize(DataInputView source) throws IOException { - // copying is a little faster than constructing a new SortKey object - SortKey deserialized = lazySortKey().copy(); - deserialize(deserialized, source); - return deserialized; - } - - @Override - public SortKey deserialize(SortKey reuse, DataInputView source) throws IOException { - Preconditions.checkArgument( - reuse.size() == size, - "Invalid size of the sort key object: %s. Expected %s", - reuse.size(), - size); - for (int i = 0; i < size; ++i) { - int fieldId = transformedFields[i].fieldId(); - Type.TypeID typeId = transformedFields[i].type().typeId(); - switch (typeId) { - case BOOLEAN: - reuse.set(i, source.readBoolean()); - break; - case INTEGER: - case DATE: - reuse.set(i, source.readInt()); - break; - case LONG: - case TIME: - case TIMESTAMP: - reuse.set(i, source.readLong()); - break; - case FLOAT: - reuse.set(i, source.readFloat()); - break; - case DOUBLE: - reuse.set(i, source.readDouble()); - break; - case STRING: - reuse.set(i, source.readUTF()); - break; - case UUID: - long mostSignificantBits = source.readLong(); - long leastSignificantBits = source.readLong(); - reuse.set(i, new UUID(mostSignificantBits, leastSignificantBits)); - break; - case FIXED: - case BINARY: - byte[] bytes = new byte[source.readInt()]; - source.read(bytes); - reuse.set(i, ByteBuffer.wrap(bytes)); - break; - case DECIMAL: - byte[] unscaledBytes = new byte[source.readInt()]; - source.read(unscaledBytes); - int scale = source.readInt(); - BigDecimal decimal = new BigDecimal(new BigInteger(unscaledBytes), scale); - reuse.set(i, decimal); - break; - case STRUCT: - case MAP: - case LIST: - default: - // SortKey transformation is a flattened struct without list and map - throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); - } - } - - return reuse; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - // no optimization here - serialize(deserialize(source), target); - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof SortKeySerializer)) { - return false; - } - - SortKeySerializer other = (SortKeySerializer) obj; - return Objects.equals(schema.asStruct(), other.schema.asStruct()) - && Objects.equals(sortOrder, other.sortOrder); - } - - @Override - public int hashCode() { - return schema.asStruct().hashCode() * 31 + sortOrder.hashCode(); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new SortKeySerializerSnapshot(schema, sortOrder); - } - - public static class SortKeySerializerSnapshot implements TypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; - - private Schema schema; - private SortOrder sortOrder; - - /** Constructor for read instantiation. */ - @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public SortKeySerializerSnapshot() { - // this constructor is used when restoring from a checkpoint. - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public SortKeySerializerSnapshot(Schema schema, SortOrder sortOrder) { - this.schema = schema; - this.sortOrder = sortOrder; - } - - @Override - public int getCurrentVersion() { - return CURRENT_VERSION; - } - - @Override - public void writeSnapshot(DataOutputView out) throws IOException { - Preconditions.checkState(schema != null, "Invalid schema: null"); - Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - - StringUtils.writeString(SchemaParser.toJson(schema), out); - StringUtils.writeString(SortOrderParser.toJson(sortOrder), out); - } - - @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); - } - } - - @Override - public TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - TypeSerializer newSerializer) { - if (!(newSerializer instanceof SortKeySerializer)) { - return TypeSerializerSchemaCompatibility.incompatible(); - } - - // Sort order should be identical - SortKeySerializerSnapshot newSnapshot = - (SortKeySerializerSnapshot) newSerializer.snapshotConfiguration(); - if (!sortOrder.sameOrder(newSnapshot.sortOrder)) { - return TypeSerializerSchemaCompatibility.incompatible(); - } - - Set sortFieldIds = - sortOrder.fields().stream().map(SortField::sourceId).collect(Collectors.toSet()); - // only care about the schema related to sort fields - Schema sortSchema = TypeUtil.project(schema, sortFieldIds); - Schema newSortSchema = TypeUtil.project(newSnapshot.schema, sortFieldIds); - - List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(sortSchema, newSortSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } - - @Override - public TypeSerializer restoreSerializer() { - Preconditions.checkState(schema != null, "Invalid schema: null"); - Preconditions.checkState(sortOrder != null, "Invalid sort order: null"); - return new SortKeySerializer(schema, sortOrder); - } - - private void readV1(DataInputView in) throws IOException { - String schemaJson = StringUtils.readString(in); - String sortOrderJson = StringUtils.readString(in); - this.schema = SchemaParser.fromJson(schemaJson); - this.sortOrder = SortOrderParser.fromJson(sortOrderJson).bind(schema); - } - - @VisibleForTesting - static TypeSerializerSchemaCompatibility resolveSchemaCompatibility( - Schema readSchema, Schema writeSchema) { - List compatibilityErrors = - CheckCompatibility.writeCompatibilityErrors(readSchema, writeSchema); - if (compatibilityErrors.isEmpty()) { - return TypeSerializerSchemaCompatibility.compatibleAsIs(); - } - - return TypeSerializerSchemaCompatibility.incompatible(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java deleted file mode 100644 index d6c23f035015..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeySketchSerializer.java +++ /dev/null @@ -1,143 +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.sink.shuffle; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UncheckedIOException; -import java.util.Arrays; -import java.util.List; -import org.apache.datasketches.common.ArrayOfItemsSerDe; -import org.apache.datasketches.common.ArrayOfStringsSerDe; -import org.apache.datasketches.common.ByteArrayUtil; -import org.apache.datasketches.common.Util; -import org.apache.datasketches.memory.Memory; -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.base.ListSerializer; -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.base.Preconditions; - -/** - * Only way to implement {@link ReservoirItemsSketch} serializer is to extend from {@link - * ArrayOfItemsSerDe}, as deserialization uses a private constructor from ReservoirItemsSketch. The - * implementation is modeled after {@link ArrayOfStringsSerDe} - */ -class SortKeySketchSerializer extends ArrayOfItemsSerDe implements Serializable { - private static final int DEFAULT_SORT_KEY_SIZE = 128; - - private final TypeSerializer itemSerializer; - private final ListSerializer listSerializer; - private final DataInputDeserializer input; - - SortKeySketchSerializer(TypeSerializer itemSerializer) { - this.itemSerializer = itemSerializer; - this.listSerializer = new ListSerializer<>(itemSerializer); - this.input = new DataInputDeserializer(); - } - - @Override - public byte[] serializeToByteArray(SortKey item) { - try { - DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE); - itemSerializer.serialize(item, output); - byte[] itemBytes = output.getSharedBuffer(); - int numBytes = output.length(); - byte[] out = new byte[numBytes + Integer.BYTES]; - ByteArrayUtil.copyBytes(itemBytes, 0, out, 4, numBytes); - ByteArrayUtil.putIntLE(out, 0, numBytes); - return out; - } catch (IOException e) { - throw new UncheckedIOException("Failed to serialize sort key", e); - } - } - - @Override - public byte[] serializeToByteArray(SortKey[] items) { - try { - DataOutputSerializer output = new DataOutputSerializer(DEFAULT_SORT_KEY_SIZE * items.length); - listSerializer.serialize(Arrays.asList(items), output); - byte[] itemsBytes = output.getSharedBuffer(); - int numBytes = output.length(); - byte[] out = new byte[Integer.BYTES + numBytes]; - ByteArrayUtil.putIntLE(out, 0, numBytes); - System.arraycopy(itemsBytes, 0, out, Integer.BYTES, numBytes); - return out; - } catch (IOException e) { - throw new UncheckedIOException("Failed to serialize sort key", e); - } - } - - @Override - public SortKey[] deserializeFromMemory(Memory mem, long startingOffset, int numItems) { - Preconditions.checkArgument(mem != null, "Invalid input memory: null"); - if (numItems <= 0) { - return new SortKey[0]; - } - - long offset = startingOffset; - Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); - int numBytes = mem.getInt(offset); - offset += Integer.BYTES; - - Util.checkBounds(offset, numBytes, mem.getCapacity()); - byte[] sortKeyBytes = new byte[numBytes]; - mem.getByteArray(offset, sortKeyBytes, 0, numBytes); - input.setBuffer(sortKeyBytes); - - try { - List sortKeys = listSerializer.deserialize(input); - SortKey[] array = new SortKey[numItems]; - sortKeys.toArray(array); - input.releaseArrays(); - return array; - } catch (IOException e) { - throw new UncheckedIOException("Failed to deserialize sort key sketch", e); - } - } - - @Override - public int sizeOf(SortKey item) { - return serializeToByteArray(item).length; - } - - @Override - public int sizeOf(Memory mem, long offset, int numItems) { - Preconditions.checkArgument(mem != null, "Invalid input memory: null"); - if (numItems <= 0) { - return 0; - } - - Util.checkBounds(offset, Integer.BYTES, mem.getCapacity()); - int numBytes = mem.getInt(offset); - return Integer.BYTES + numBytes; - } - - @Override - public String toString(SortKey item) { - return item.toString(); - } - - @Override - public Class getClassOfT() { - return SortKey.class; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java deleted file mode 100644 index 1e5bdbbac3e4..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/SortKeyUtil.java +++ /dev/null @@ -1,59 +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.sink.shuffle; - -import java.util.List; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortField; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -class SortKeyUtil { - private SortKeyUtil() {} - - /** Compute the result schema of {@code SortKey} transformation */ - static Schema sortKeySchema(Schema schema, SortOrder sortOrder) { - List sortFields = sortOrder.fields(); - int size = sortFields.size(); - List transformedFields = Lists.newArrayListWithCapacity(size); - for (int i = 0; i < size; ++i) { - int sourceFieldId = sortFields.get(i).sourceId(); - Types.NestedField sourceField = schema.findField(sourceFieldId); - Preconditions.checkArgument( - sourceField != null, "Cannot find source field: %s", sourceFieldId); - Type transformedType = sortFields.get(i).transform().getResultType(sourceField.type()); - // There could be multiple transformations on the same source column, like in the PartitionKey - // case. To resolve the collision, field id is set to transform index and field name is set to - // sourceFieldName_transformIndex - Types.NestedField transformedField = - Types.NestedField.of( - i, - sourceField.isOptional(), - sourceField.name() + '_' + i, - transformedType, - sourceField.doc()); - transformedFields.add(transformedField); - } - - return new Schema(transformedFields); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java deleted file mode 100644 index f6fcdb8b16ef..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsEvent.java +++ /dev/null @@ -1,76 +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.sink.shuffle; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.runtime.operators.coordination.OperatorEvent; - -/** - * DataStatisticsEvent is sent between data statistics coordinator and operator to transmit data - * statistics in bytes - */ -@Internal -class StatisticsEvent implements OperatorEvent { - - private static final long serialVersionUID = 1L; - private final long checkpointId; - private final byte[] statisticsBytes; - private final boolean applyImmediately; - - private StatisticsEvent(long checkpointId, byte[] statisticsBytes, boolean applyImmediately) { - this.checkpointId = checkpointId; - this.statisticsBytes = statisticsBytes; - this.applyImmediately = applyImmediately; - } - - static StatisticsEvent createTaskStatisticsEvent( - long checkpointId, - DataStatistics statistics, - TypeSerializer statisticsSerializer) { - // applyImmediately is really only relevant for coordinator to operator event. - // task reported statistics is always merged immediately by the coordinator. - return new StatisticsEvent( - checkpointId, - StatisticsUtil.serializeDataStatistics(statistics, statisticsSerializer), - true); - } - - static StatisticsEvent createGlobalStatisticsEvent( - GlobalStatistics statistics, - TypeSerializer statisticsSerializer, - boolean applyImmediately) { - return new StatisticsEvent( - statistics.checkpointId(), - StatisticsUtil.serializeGlobalStatistics(statistics, statisticsSerializer), - applyImmediately); - } - - long checkpointId() { - return checkpointId; - } - - byte[] statisticsBytes() { - return statisticsBytes; - } - - boolean applyImmediately() { - return applyImmediately; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java deleted file mode 100644 index bc28df2b0e22..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecord.java +++ /dev/null @@ -1,112 +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.sink.shuffle; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * The wrapper class for data statistics and record. It is the only way for data statistics operator - * to send global data statistics to custom partitioner to distribute data based on statistics - * - *

    DataStatisticsOrRecord contains either data statistics(globally aggregated) or a record. It is - * sent from {@link DataStatisticsOperator} to partitioner. Once partitioner receives the data - * statistics, it will use that to decide the coming record should send to which writer subtask. - * After shuffling, a filter and mapper are required to filter out the data distribution weight, - * unwrap the object and extract the original record type T. - */ -@Internal -public class StatisticsOrRecord implements Serializable { - - private static final long serialVersionUID = 1L; - - private GlobalStatistics statistics; - private RowData record; - - private StatisticsOrRecord(GlobalStatistics statistics, RowData record) { - Preconditions.checkArgument( - record != null ^ statistics != null, "DataStatistics or record, not neither or both"); - this.statistics = statistics; - this.record = record; - } - - static StatisticsOrRecord fromRecord(RowData record) { - return new StatisticsOrRecord(null, record); - } - - static StatisticsOrRecord fromStatistics(GlobalStatistics statistics) { - return new StatisticsOrRecord(statistics, null); - } - - static StatisticsOrRecord reuseRecord( - StatisticsOrRecord reuse, TypeSerializer recordSerializer) { - if (reuse.hasRecord()) { - return reuse; - } else { - // not reusable - return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); - } - } - - static StatisticsOrRecord reuseStatistics( - StatisticsOrRecord reuse, TypeSerializer statisticsSerializer) { - if (reuse.hasStatistics()) { - return reuse; - } else { - // not reusable - return StatisticsOrRecord.fromStatistics(statisticsSerializer.createInstance()); - } - } - - boolean hasStatistics() { - return statistics != null; - } - - public boolean hasRecord() { - return record != null; - } - - GlobalStatistics statistics() { - return statistics; - } - - void statistics(GlobalStatistics newStatistics) { - this.statistics = newStatistics; - } - - public RowData record() { - return record; - } - - void record(RowData newRecord) { - this.record = newRecord; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("statistics", statistics) - .add("record", record) - .toString(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java deleted file mode 100644 index 6e403425938d..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsOrRecordSerializer.java +++ /dev/null @@ -1,210 +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.sink.shuffle; - -import java.io.IOException; -import java.util.Objects; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.typeutils.CompositeTypeSerializerSnapshot; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; -import org.apache.flink.table.data.RowData; - -@Internal -class StatisticsOrRecordSerializer extends TypeSerializer { - private final TypeSerializer statisticsSerializer; - private final TypeSerializer recordSerializer; - - StatisticsOrRecordSerializer( - TypeSerializer statisticsSerializer, - TypeSerializer recordSerializer) { - this.statisticsSerializer = statisticsSerializer; - this.recordSerializer = recordSerializer; - } - - @Override - public boolean isImmutableType() { - return false; - } - - @SuppressWarnings("ReferenceEquality") - @Override - public TypeSerializer duplicate() { - TypeSerializer duplicateStatisticsSerializer = - statisticsSerializer.duplicate(); - TypeSerializer duplicateRowDataSerializer = recordSerializer.duplicate(); - if ((statisticsSerializer != duplicateStatisticsSerializer) - || (recordSerializer != duplicateRowDataSerializer)) { - return new StatisticsOrRecordSerializer( - duplicateStatisticsSerializer, duplicateRowDataSerializer); - } else { - return this; - } - } - - @Override - public StatisticsOrRecord createInstance() { - // arbitrarily always create RowData value instance - return StatisticsOrRecord.fromRecord(recordSerializer.createInstance()); - } - - @Override - public StatisticsOrRecord copy(StatisticsOrRecord from) { - if (from.hasRecord()) { - return StatisticsOrRecord.fromRecord(recordSerializer.copy(from.record())); - } else { - return StatisticsOrRecord.fromStatistics(statisticsSerializer.copy(from.statistics())); - } - } - - @Override - public StatisticsOrRecord copy(StatisticsOrRecord from, StatisticsOrRecord reuse) { - StatisticsOrRecord to; - if (from.hasRecord()) { - to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); - RowData record = recordSerializer.copy(from.record(), to.record()); - to.record(record); - } else { - to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - GlobalStatistics statistics = statisticsSerializer.copy(from.statistics(), to.statistics()); - to.statistics(statistics); - } - - return to; - } - - @Override - public int getLength() { - return -1; - } - - @Override - public void serialize(StatisticsOrRecord statisticsOrRecord, DataOutputView target) - throws IOException { - if (statisticsOrRecord.hasRecord()) { - target.writeBoolean(true); - recordSerializer.serialize(statisticsOrRecord.record(), target); - } else { - target.writeBoolean(false); - statisticsSerializer.serialize(statisticsOrRecord.statistics(), target); - } - } - - @Override - public StatisticsOrRecord deserialize(DataInputView source) throws IOException { - boolean isRecord = source.readBoolean(); - if (isRecord) { - return StatisticsOrRecord.fromRecord(recordSerializer.deserialize(source)); - } else { - return StatisticsOrRecord.fromStatistics(statisticsSerializer.deserialize(source)); - } - } - - @Override - public StatisticsOrRecord deserialize(StatisticsOrRecord reuse, DataInputView source) - throws IOException { - StatisticsOrRecord to; - boolean isRecord = source.readBoolean(); - if (isRecord) { - to = StatisticsOrRecord.reuseRecord(reuse, recordSerializer); - RowData record = recordSerializer.deserialize(to.record(), source); - to.record(record); - } else { - to = StatisticsOrRecord.reuseStatistics(reuse, statisticsSerializer); - GlobalStatistics statistics = statisticsSerializer.deserialize(to.statistics(), source); - to.statistics(statistics); - } - - return to; - } - - @Override - public void copy(DataInputView source, DataOutputView target) throws IOException { - boolean hasRecord = source.readBoolean(); - target.writeBoolean(hasRecord); - if (hasRecord) { - recordSerializer.copy(source, target); - } else { - statisticsSerializer.copy(source, target); - } - } - - @Override - public boolean equals(Object obj) { - if (!(obj instanceof StatisticsOrRecordSerializer)) { - return false; - } - - StatisticsOrRecordSerializer other = (StatisticsOrRecordSerializer) obj; - return Objects.equals(statisticsSerializer, other.statisticsSerializer) - && Objects.equals(recordSerializer, other.recordSerializer); - } - - @Override - public int hashCode() { - return Objects.hash(statisticsSerializer, recordSerializer); - } - - @Override - public TypeSerializerSnapshot snapshotConfiguration() { - return new StatisticsOrRecordSerializerSnapshot(this); - } - - public static class StatisticsOrRecordSerializerSnapshot - extends CompositeTypeSerializerSnapshot { - private static final int CURRENT_VERSION = 1; - - /** Constructor for read instantiation. */ - @SuppressWarnings({"unused", "checkstyle:RedundantModifier"}) - public StatisticsOrRecordSerializerSnapshot() { - super(StatisticsOrRecordSerializer.class); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - public StatisticsOrRecordSerializerSnapshot(StatisticsOrRecordSerializer serializer) { - super(serializer); - } - - @SuppressWarnings("checkstyle:RedundantModifier") - @Override - protected int getCurrentOuterSnapshotVersion() { - return CURRENT_VERSION; - } - - @Override - protected TypeSerializer[] getNestedSerializers( - StatisticsOrRecordSerializer outerSerializer) { - return new TypeSerializer[] { - outerSerializer.statisticsSerializer, outerSerializer.recordSerializer - }; - } - - @SuppressWarnings("unchecked") - @Override - protected StatisticsOrRecordSerializer createOuterSerializerWithNestedSerializers( - TypeSerializer[] nestedSerializers) { - TypeSerializer statisticsSerializer = - (TypeSerializer) nestedSerializers[0]; - TypeSerializer recordSerializer = (TypeSerializer) nestedSerializers[1]; - return new StatisticsOrRecordSerializer(statisticsSerializer, recordSerializer); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java deleted file mode 100644 index 43f72e336e06..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsType.java +++ /dev/null @@ -1,55 +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.sink.shuffle; - -/** - * Range distribution requires gathering statistics on the sort keys to determine proper range - * boundaries to distribute/cluster rows before writer operators. - */ -public enum StatisticsType { - /** - * Tracks the data statistics as {@code Map} frequency. It works better for - * low-cardinality scenarios (like country, event_type, etc.) where the cardinalities are in - * hundreds or thousands. - * - *

      - *
    • Pro: accurate measurement on the statistics/weight of every key. - *
    • Con: memory footprint can be large if the key cardinality is high. - *
    - */ - Map, - - /** - * Sample the sort keys via reservoir sampling. Then split the range partitions via range bounds - * from sampled values. It works better for high-cardinality scenarios (like device_id, user_id, - * uuid etc.) where the cardinalities can be in millions or billions. - * - *
      - *
    • Pro: relatively low memory footprint for high-cardinality sort keys. - *
    • Con: non-precise approximation with potentially lower accuracy. - *
    - */ - Sketch, - - /** - * Initially use Map for statistics tracking. If key cardinality turns out to be high, - * automatically switch to sketch sampling. - */ - Auto -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java deleted file mode 100644 index 5d48ec57ca49..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/StatisticsUtil.java +++ /dev/null @@ -1,126 +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.sink.shuffle; - -import java.io.IOException; -import java.io.UncheckedIOException; -import javax.annotation.Nullable; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -class StatisticsUtil { - - private StatisticsUtil() {} - - static DataStatistics createTaskStatistics( - StatisticsType type, int operatorParallelism, int numPartitions) { - if (type == StatisticsType.Map) { - return new MapDataStatistics(); - } else { - return new SketchDataStatistics( - SketchUtil.determineOperatorReservoirSize(operatorParallelism, numPartitions)); - } - } - - static byte[] serializeDataStatistics( - DataStatistics dataStatistics, TypeSerializer statisticsSerializer) { - DataOutputSerializer out = new DataOutputSerializer(64); - try { - statisticsSerializer.serialize(dataStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new UncheckedIOException("Fail to serialize data statistics", e); - } - } - - static DataStatistics deserializeDataStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { - DataInputDeserializer input = new DataInputDeserializer(bytes, 0, bytes.length); - try { - return statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new UncheckedIOException("Fail to deserialize data statistics", e); - } - } - - static byte[] serializeCompletedStatistics( - CompletedStatistics completedStatistics, - TypeSerializer statisticsSerializer) { - try { - DataOutputSerializer out = new DataOutputSerializer(1024); - statisticsSerializer.serialize(completedStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new UncheckedIOException("Fail to serialize aggregated statistics", e); - } - } - - static CompletedStatistics deserializeCompletedStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { - try { - DataInputDeserializer input = new DataInputDeserializer(bytes); - return statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); - } - } - - static byte[] serializeGlobalStatistics( - GlobalStatistics globalStatistics, TypeSerializer statisticsSerializer) { - try { - DataOutputSerializer out = new DataOutputSerializer(1024); - statisticsSerializer.serialize(globalStatistics, out); - return out.getCopyOfBuffer(); - } catch (IOException e) { - throw new UncheckedIOException("Fail to serialize aggregated statistics", e); - } - } - - static GlobalStatistics deserializeGlobalStatistics( - byte[] bytes, TypeSerializer statisticsSerializer) { - try { - DataInputDeserializer input = new DataInputDeserializer(bytes); - return statisticsSerializer.deserialize(input); - } catch (IOException e) { - throw new UncheckedIOException("Fail to deserialize aggregated statistics", e); - } - } - - static StatisticsType collectType(StatisticsType config) { - return config == StatisticsType.Sketch ? StatisticsType.Sketch : StatisticsType.Map; - } - - static StatisticsType collectType(StatisticsType config, @Nullable GlobalStatistics statistics) { - if (statistics != null) { - return statistics.type(); - } - - return collectType(config); - } - - static StatisticsType collectType( - StatisticsType config, @Nullable CompletedStatistics statistics) { - if (statistics != null) { - return statistics.type(); - } - - return collectType(config); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java deleted file mode 100644 index 796434c45136..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/AvroGenericRecordFileScanTaskReader.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source; - -import org.apache.avro.generic.GenericRecord; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; - -public class AvroGenericRecordFileScanTaskReader implements FileScanTaskReader { - private final RowDataFileScanTaskReader rowDataReader; - private final RowDataToAvroGenericRecordConverter converter; - - public AvroGenericRecordFileScanTaskReader( - RowDataFileScanTaskReader rowDataReader, RowDataToAvroGenericRecordConverter converter) { - this.rowDataReader = rowDataReader; - this.converter = converter; - } - - @Override - public CloseableIterator open( - FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { - return CloseableIterator.transform( - rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java deleted file mode 100644 index 91d975349b19..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ /dev/null @@ -1,156 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Iterator; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Flink data iterator that reads {@link CombinedScanTask} into a {@link CloseableIterator} - * - * @param is the output data type returned by this iterator. - */ -@Internal -public class DataIterator implements CloseableIterator { - - private final FileScanTaskReader fileScanTaskReader; - - private final InputFilesDecryptor inputFilesDecryptor; - private final CombinedScanTask combinedTask; - - private Iterator tasks; - private CloseableIterator currentIterator; - private int fileOffset; - private long recordOffset; - - public DataIterator( - FileScanTaskReader fileScanTaskReader, - CombinedScanTask task, - FileIO io, - EncryptionManager encryption) { - this.fileScanTaskReader = fileScanTaskReader; - - this.inputFilesDecryptor = new InputFilesDecryptor(task, io, encryption); - this.combinedTask = task; - - this.tasks = task.files().iterator(); - this.currentIterator = CloseableIterator.empty(); - - // fileOffset starts at -1 because we started - // from an empty iterator that is not from the split files. - this.fileOffset = -1; - // record offset points to the record that next() should return when called - this.recordOffset = 0L; - } - - /** - * (startingFileOffset, startingRecordOffset) points to the next row that reader should resume - * from. E.g., if the seek position is (file=0, record=1), seek moves the iterator position to the - * 2nd row in file 0. When next() is called after seek, 2nd row from file 0 should be returned. - */ - public void seek(int startingFileOffset, long startingRecordOffset) { - Preconditions.checkState( - fileOffset == -1, "Seek should be called before any other iterator actions"); - // skip files - Preconditions.checkState( - startingFileOffset < combinedTask.files().size(), - "Invalid starting file offset %s for combined scan task with %s files: %s", - startingFileOffset, - combinedTask.files().size(), - combinedTask); - for (long i = 0L; i < startingFileOffset; ++i) { - tasks.next(); - } - - updateCurrentIterator(); - // skip records within the file - for (long i = 0; i < startingRecordOffset; ++i) { - if (currentFileHasNext() && hasNext()) { - next(); - } else { - throw new IllegalStateException( - String.format( - "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); - } - } - - fileOffset = startingFileOffset; - recordOffset = startingRecordOffset; - } - - @Override - public boolean hasNext() { - updateCurrentIterator(); - return currentIterator.hasNext(); - } - - @Override - public T next() { - updateCurrentIterator(); - recordOffset += 1; - return currentIterator.next(); - } - - public boolean currentFileHasNext() { - return currentIterator.hasNext(); - } - - /** Updates the current iterator field to ensure that the current Iterator is not exhausted. */ - private void updateCurrentIterator() { - try { - while (!currentIterator.hasNext() && tasks.hasNext()) { - currentIterator.close(); - currentIterator = openTaskIterator(tasks.next()); - fileOffset += 1; - recordOffset = 0L; - } - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - private CloseableIterator openTaskIterator(FileScanTask scanTask) { - return fileScanTaskReader.open(scanTask, inputFilesDecryptor); - } - - @Override - public void close() throws IOException { - // close the current iterator - currentIterator.close(); - tasks = null; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java deleted file mode 100644 index 4394dab4d4cc..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/DataTaskReader.java +++ /dev/null @@ -1,47 +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.source; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.flink.data.StructRowData; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; - -@Internal -public class DataTaskReader implements FileScanTaskReader { - - private final Schema readSchema; - - public DataTaskReader(Schema readSchema) { - this.readSchema = readSchema; - } - - @Override - public CloseableIterator open( - FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { - StructRowData row = new StructRowData(readSchema.asStruct()); - CloseableIterable iterable = - CloseableIterable.transform(task.asDataTask().rows(), row::setStruct); - return iterable.iterator(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java deleted file mode 100644 index 927a804a4792..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FileScanTaskReader.java +++ /dev/null @@ -1,35 +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.source; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.io.CloseableIterator; - -/** - * Read a {@link FileScanTask} into a {@link CloseableIterator} - * - * @param is the output data type returned by this iterator. - */ -@Internal -public interface FileScanTaskReader extends Serializable { - CloseableIterator open(FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java deleted file mode 100644 index 9a5123dc489e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java +++ /dev/null @@ -1,141 +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.source; - -import java.io.IOException; -import java.util.concurrent.ExecutorService; -import org.apache.flink.api.common.io.DefaultInputSplitAssigner; -import org.apache.flink.api.common.io.InputFormat; -import org.apache.flink.api.common.io.LocatableInputSplitAssigner; -import org.apache.flink.api.common.io.RichInputFormat; -import org.apache.flink.api.common.io.statistics.BaseStatistics; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.core.io.InputSplitAssigner; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.util.ThreadPools; - -/** Flink {@link InputFormat} for Iceberg. */ -public class FlinkInputFormat extends RichInputFormat { - - private static final long serialVersionUID = 1L; - - private final TableLoader tableLoader; - private final FileIO io; - private final EncryptionManager encryption; - private final ScanContext context; - private final FileScanTaskReader rowDataReader; - - private transient DataIterator iterator; - private transient long currentReadCount = 0L; - - FlinkInputFormat( - TableLoader tableLoader, - Schema tableSchema, - FileIO io, - EncryptionManager encryption, - ScanContext context) { - this.tableLoader = tableLoader; - this.io = io; - this.encryption = encryption; - this.context = context; - - tableLoader.open(); - Table table = tableLoader.loadTable(); - if (table instanceof BaseMetadataTable) { - this.rowDataReader = new DataTaskReader(context.project()); - } else { - this.rowDataReader = - new RowDataFileScanTaskReader( - tableSchema, - context.project(), - context.nameMapping(), - context.caseSensitive(), - context.filters()); - } - } - - @VisibleForTesting - Schema projectedSchema() { - return context.project(); - } - - @Override - public BaseStatistics getStatistics(BaseStatistics cachedStatistics) { - // Legacy method, not be used. - return null; - } - - @Override - public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException { - // Called in Job manager, so it is OK to load table from catalog. - tableLoader.open(); - final ExecutorService workerPool = - ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); - try (TableLoader loader = tableLoader) { - Table table = loader.loadTable(); - return FlinkSplitPlanner.planInputSplits(table, context, workerPool); - } finally { - workerPool.shutdown(); - } - } - - @Override - public InputSplitAssigner getInputSplitAssigner(FlinkInputSplit[] inputSplits) { - return context.exposeLocality() - ? new LocatableInputSplitAssigner(inputSplits) - : new DefaultInputSplitAssigner(inputSplits); - } - - @Override - public void configure(Configuration parameters) {} - - @Override - public void open(FlinkInputSplit split) { - this.iterator = new DataIterator<>(rowDataReader, split.getTask(), io, encryption); - } - - @Override - public boolean reachedEnd() { - if (context.limit() > 0 && currentReadCount >= context.limit()) { - return true; - } else { - return !iterator.hasNext(); - } - } - - @Override - public RowData nextRecord(RowData reuse) { - currentReadCount++; - return iterator.next(); - } - - @Override - public void close() throws IOException { - if (iterator != null) { - iterator.close(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java deleted file mode 100644 index 16fd4f39596c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputSplit.java +++ /dev/null @@ -1,48 +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.source; - -import java.util.Arrays; -import javax.annotation.Nullable; -import org.apache.flink.core.io.LocatableInputSplit; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; - -public class FlinkInputSplit extends LocatableInputSplit { - - private final CombinedScanTask task; - - FlinkInputSplit(int splitNumber, CombinedScanTask task, @Nullable String[] hostnames) { - super(splitNumber, hostnames); - this.task = task; - } - - CombinedScanTask getTask() { - return task; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("splitNumber", getSplitNumber()) - .add("task", task) - .add("hosts", Arrays.toString(getHostnames())) - .toString(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java deleted file mode 100644 index b1431a32dd20..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSource.java +++ /dev/null @@ -1,307 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.Configuration; -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.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.FileIO; -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.PropertyUtil; - -public class FlinkSource { - private FlinkSource() {} - - /** - * Initialize a {@link Builder} to read the data from iceberg table. Equivalent to {@link - * TableScan}. See more options in {@link ScanContext}. - * - *

    The Source can be read static data in bounded mode. It can also continuously check the - * arrival of new data and read records incrementally. - * - *

      - *
    • Without startSnapshotId: Bounded - *
    • With startSnapshotId and with endSnapshotId: Bounded - *
    • With startSnapshotId (-1 means unbounded preceding) and Without endSnapshotId: Unbounded - *
    - * - *

    - * - * @return {@link Builder} to connect the iceberg table. - */ - public static Builder forRowData() { - return new Builder(); - } - - /** Source builder to build {@link DataStream}. */ - public static class Builder { - private StreamExecutionEnvironment env; - private Table table; - private TableLoader tableLoader; - private TableSchema projectedSchema; - private ReadableConfig readableConfig = new Configuration(); - private final ScanContext.Builder contextBuilder = ScanContext.builder(); - private Boolean exposeLocality; - - private final Map readOptions = Maps.newHashMap(); - - public Builder tableLoader(TableLoader newLoader) { - this.tableLoader = newLoader; - return this; - } - - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - public Builder env(StreamExecutionEnvironment newEnv) { - this.env = newEnv; - return this; - } - - public Builder filters(List filters) { - contextBuilder.filters(filters); - return this; - } - - public Builder project(TableSchema schema) { - this.projectedSchema = schema; - return this; - } - - public Builder limit(Long newLimit) { - if (newLimit != null) { - readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); - } - return this; - } - - public Builder set(String property, String value) { - readOptions.put(property, value); - return this; - } - - public Builder setAll(Map properties) { - readOptions.putAll(properties); - return this; - } - - /** @deprecated Use {@link #setAll} instead. */ - @Deprecated - public Builder properties(Map properties) { - readOptions.putAll(properties); - return this; - } - - public Builder caseSensitive(boolean caseSensitive) { - readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(caseSensitive)); - return this; - } - - public Builder snapshotId(Long snapshotId) { - readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(snapshotId)); - return this; - } - - public Builder branch(String branch) { - readOptions.put(FlinkReadOptions.BRANCH.key(), branch); - return this; - } - - public Builder tag(String tag) { - readOptions.put(FlinkReadOptions.TAG.key(), tag); - return this; - } - - public Builder startSnapshotId(Long startSnapshotId) { - readOptions.put(FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(startSnapshotId)); - return this; - } - - public Builder endSnapshotId(Long endSnapshotId) { - readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(endSnapshotId)); - return this; - } - - public Builder startTag(String startTag) { - readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); - return this; - } - - public Builder endTag(String endTag) { - readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); - return this; - } - - public Builder asOfTimestamp(Long asOfTimestamp) { - readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(asOfTimestamp)); - return this; - } - - public Builder splitSize(Long splitSize) { - readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(splitSize)); - return this; - } - - public Builder splitLookback(Integer splitLookback) { - readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(splitLookback)); - return this; - } - - public Builder splitOpenFileCost(Long splitOpenFileCost) { - readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(splitOpenFileCost)); - return this; - } - - public Builder streaming(boolean streaming) { - readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder nameMapping(String nameMapping) { - readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, nameMapping); - return this; - } - - public Builder monitorInterval(Duration interval) { - readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, interval.toNanos() + " ns"); - return this; - } - - public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { - readOptions.put( - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT, - Integer.toString(newMaxPlanningSnapshotCount)); - return this; - } - - public Builder flinkConf(ReadableConfig config) { - this.readableConfig = config; - return this; - } - - public FlinkInputFormat buildFormat() { - Preconditions.checkNotNull(tableLoader, "TableLoader should not be null"); - - Schema icebergSchema; - FileIO io; - EncryptionManager encryption; - if (table == null) { - // load required fields by table loader. - tableLoader.open(); - try (TableLoader loader = tableLoader) { - table = loader.loadTable(); - icebergSchema = table.schema(); - io = table.io(); - encryption = table.encryption(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } else { - icebergSchema = table.schema(); - io = table.io(); - encryption = table.encryption(); - } - - if (projectedSchema == null) { - contextBuilder.project(icebergSchema); - } else { - contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedSchema)); - } - - contextBuilder.exposeLocality( - SourceUtil.isLocalityEnabled(table, readableConfig, exposeLocality)); - contextBuilder.planParallelism( - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); - - contextBuilder.resolveConfig(table, readOptions, readableConfig); - - ScanContext context = contextBuilder.build(); - context.validate(); - return new FlinkInputFormat(tableLoader, icebergSchema, io, encryption, context); - } - - public DataStream build() { - Preconditions.checkNotNull(env, "StreamExecutionEnvironment should not be null"); - FlinkInputFormat format = buildFormat(); - - ScanContext context = contextBuilder.build(); - TypeInformation typeInfo = - FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(context.project())); - - if (!context.isStreaming()) { - int parallelism = - SourceUtil.inferParallelism( - readableConfig, - context.limit(), - () -> { - try { - return format.createInputSplits(0).length; - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to create iceberg input splits for table: " + table, e); - } - }); - if (env.getMaxParallelism() > 0) { - parallelism = Math.min(parallelism, env.getMaxParallelism()); - } - return env.createInput(format, typeInfo).setParallelism(parallelism); - } else { - StreamingMonitorFunction function = new StreamingMonitorFunction(tableLoader, context); - - String monitorFunctionName = String.format("Iceberg table (%s) monitor", table); - String readerOperatorName = String.format("Iceberg table (%s) reader", table); - - return env.addSource(function, monitorFunctionName) - .transform(readerOperatorName, typeInfo, StreamingReaderOperator.factory(format)); - } - } - } - - public static boolean isBounded(Map properties) { - return !PropertyUtil.propertyAsBoolean(properties, FlinkReadOptions.STREAMING, false); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java deleted file mode 100644 index 15078809714f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/FlinkSplitPlanner.java +++ /dev/null @@ -1,189 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.IncrementalAppendScan; -import org.apache.iceberg.Scan; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.io.CloseableIterable; -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.apache.iceberg.util.Tasks; - -@Internal -public class FlinkSplitPlanner { - private FlinkSplitPlanner() {} - - static FlinkInputSplit[] planInputSplits( - Table table, ScanContext context, ExecutorService workerPool) { - try (CloseableIterable tasksIterable = - planTasks(table, context, workerPool)) { - List tasks = Lists.newArrayList(tasksIterable); - FlinkInputSplit[] splits = new FlinkInputSplit[tasks.size()]; - boolean exposeLocality = context.exposeLocality(); - - Tasks.range(tasks.size()) - .stopOnFailure() - .executeWith(exposeLocality ? workerPool : null) - .run( - index -> { - CombinedScanTask task = tasks.get(index); - String[] hostnames = null; - if (exposeLocality) { - hostnames = Util.blockLocations(table.io(), task); - } - splits[index] = new FlinkInputSplit(index, task, hostnames); - }); - return splits; - } catch (IOException e) { - throw new UncheckedIOException("Failed to process tasks iterable", e); - } - } - - /** This returns splits for the FLIP-27 source */ - public static List planIcebergSourceSplits( - Table table, ScanContext context, ExecutorService workerPool) { - try (CloseableIterable tasksIterable = - planTasks(table, context, workerPool)) { - return Lists.newArrayList( - CloseableIterable.transform(tasksIterable, IcebergSourceSplit::fromCombinedScanTask)); - } catch (IOException e) { - throw new UncheckedIOException("Failed to process task iterable: ", e); - } - } - - static CloseableIterable planTasks( - Table table, ScanContext context, ExecutorService workerPool) { - ScanMode scanMode = checkScanMode(context); - if (scanMode == ScanMode.INCREMENTAL_APPEND_SCAN) { - IncrementalAppendScan scan = table.newIncrementalAppendScan(); - scan = refineScanWithBaseConfigs(scan, context, workerPool); - - if (context.startTag() != null) { - Preconditions.checkArgument( - table.snapshot(context.startTag()) != null, - "Cannot find snapshot with tag %s", - context.startTag()); - scan = scan.fromSnapshotExclusive(table.snapshot(context.startTag()).snapshotId()); - } - - if (context.startSnapshotId() != null) { - Preconditions.checkArgument( - context.startTag() == null, "START_SNAPSHOT_ID and START_TAG cannot both be set"); - scan = scan.fromSnapshotExclusive(context.startSnapshotId()); - } - - if (context.endTag() != null) { - Preconditions.checkArgument( - table.snapshot(context.endTag()) != null, - "Cannot find snapshot with tag %s", - context.endTag()); - scan = scan.toSnapshot(table.snapshot(context.endTag()).snapshotId()); - } - - if (context.endSnapshotId() != null) { - Preconditions.checkArgument( - context.endTag() == null, "END_SNAPSHOT_ID and END_TAG cannot both be set"); - scan = scan.toSnapshot(context.endSnapshotId()); - } - - return scan.planTasks(); - } else { - TableScan scan = table.newScan(); - scan = refineScanWithBaseConfigs(scan, context, workerPool); - - if (context.snapshotId() != null) { - scan = scan.useSnapshot(context.snapshotId()); - } else if (context.tag() != null) { - scan = scan.useRef(context.tag()); - } else if (context.branch() != null) { - scan = scan.useRef(context.branch()); - } - - if (context.asOfTimestamp() != null) { - scan = scan.asOfTime(context.asOfTimestamp()); - } - - return scan.planTasks(); - } - } - - @VisibleForTesting - enum ScanMode { - BATCH, - INCREMENTAL_APPEND_SCAN - } - - @VisibleForTesting - static ScanMode checkScanMode(ScanContext context) { - if (context.startSnapshotId() != null - || context.endSnapshotId() != null - || context.startTag() != null - || context.endTag() != null) { - return ScanMode.INCREMENTAL_APPEND_SCAN; - } else { - return ScanMode.BATCH; - } - } - - /** refine scan with common configs */ - private static > T refineScanWithBaseConfigs( - T scan, ScanContext context, ExecutorService workerPool) { - T refinedScan = - scan.caseSensitive(context.caseSensitive()).project(context.project()).planWith(workerPool); - - if (context.includeColumnStats()) { - refinedScan = refinedScan.includeColumnStats(); - } - - if (context.includeStatsForColumns() != null) { - refinedScan = refinedScan.includeColumnStats(context.includeStatsForColumns()); - } - - refinedScan = refinedScan.option(TableProperties.SPLIT_SIZE, context.splitSize().toString()); - - refinedScan = - refinedScan.option(TableProperties.SPLIT_LOOKBACK, context.splitLookback().toString()); - - refinedScan = - refinedScan.option( - TableProperties.SPLIT_OPEN_FILE_COST, context.splitOpenFileCost().toString()); - - if (context.filters() != null) { - for (Expression filter : context.filters()) { - refinedScan = refinedScan.filter(filter); - } - } - - return refinedScan; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java deleted file mode 100644 index ccbd0d9997ed..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergSource.java +++ /dev/null @@ -1,549 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.UUID; -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.connector.source.Boundedness; -import org.apache.flink.api.connector.source.Source; -import org.apache.flink.api.connector.source.SourceReader; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.BaseMetadataTable; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadConf; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.assigner.SplitAssignerFactory; -import org.apache.iceberg.flink.source.enumerator.ContinuousIcebergEnumerator; -import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlanner; -import org.apache.iceberg.flink.source.enumerator.ContinuousSplitPlannerImpl; -import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorState; -import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; -import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; -import org.apache.iceberg.flink.source.reader.IcebergSourceReader; -import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; -import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; -import org.apache.iceberg.flink.source.reader.ReaderFunction; -import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; -import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -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.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Experimental -public class IcebergSource implements Source { - private static final Logger LOG = LoggerFactory.getLogger(IcebergSource.class); - - // This table loader can be closed, and it is only safe to use this instance for resource - // independent information (e.g. a table name). Copies of this are required to avoid lifecycle - // management conflicts with the user provided table loader. e.g. a copy of this is required for - // split planning, which uses the underlying io, and should be closed after split planning is - // complete. - private final TableLoader tableLoader; - private final ScanContext scanContext; - private final ReaderFunction readerFunction; - private final SplitAssignerFactory assignerFactory; - private final SerializableComparator splitComparator; - private final SerializableRecordEmitter emitter; - private final String tableName; - - IcebergSource( - TableLoader tableLoader, - ScanContext scanContext, - ReaderFunction readerFunction, - SplitAssignerFactory assignerFactory, - SerializableComparator splitComparator, - Table table, - SerializableRecordEmitter emitter) { - Preconditions.checkNotNull(tableLoader, "tableLoader is required."); - Preconditions.checkNotNull(readerFunction, "readerFunction is required."); - Preconditions.checkNotNull(assignerFactory, "assignerFactory is required."); - Preconditions.checkNotNull(table, "table is required."); - this.tableLoader = tableLoader; - this.scanContext = scanContext; - this.readerFunction = readerFunction; - this.assignerFactory = assignerFactory; - this.splitComparator = splitComparator; - this.emitter = emitter; - this.tableName = table.name(); - } - - String name() { - return "IcebergSource-" + tableName; - } - - private String planningThreadName() { - // Ideally, operatorId should be used as the threadPoolName as Flink guarantees its uniqueness - // within a job. SplitEnumeratorContext doesn't expose the OperatorCoordinator.Context, which - // would contain the OperatorID. Need to discuss with Flink community whether it is ok to expose - // a public API like the protected method "OperatorCoordinator.Context getCoordinatorContext()" - // from SourceCoordinatorContext implementation. For now,

  • - is used as - // the unique thread pool name. - return tableName + "-" + UUID.randomUUID(); - } - - private List planSplitsForBatch(String threadName) { - ExecutorService workerPool = - ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); - try (TableLoader loader = tableLoader.clone()) { - loader.open(); - List splits = - FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); - LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; - } catch (IOException e) { - throw new UncheckedIOException("Failed to close table loader", e); - } finally { - workerPool.shutdown(); - } - } - - @Override - public Boundedness getBoundedness() { - return scanContext.isStreaming() ? Boundedness.CONTINUOUS_UNBOUNDED : Boundedness.BOUNDED; - } - - @Override - public SourceReader createReader(SourceReaderContext readerContext) { - IcebergSourceReaderMetrics metrics = - new IcebergSourceReaderMetrics(readerContext.metricGroup(), tableName); - return new IcebergSourceReader<>( - emitter, metrics, readerFunction, splitComparator, readerContext); - } - - @Override - public SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext) { - return createEnumerator(enumContext, null); - } - - @Override - public SplitEnumerator restoreEnumerator( - SplitEnumeratorContext enumContext, IcebergEnumeratorState enumState) { - return createEnumerator(enumContext, enumState); - } - - @Override - public SimpleVersionedSerializer getSplitSerializer() { - return new IcebergSourceSplitSerializer(scanContext.caseSensitive()); - } - - @Override - public SimpleVersionedSerializer getEnumeratorCheckpointSerializer() { - return new IcebergEnumeratorStateSerializer(scanContext.caseSensitive()); - } - - private SplitEnumerator createEnumerator( - SplitEnumeratorContext enumContext, - @Nullable IcebergEnumeratorState enumState) { - SplitAssigner assigner; - if (enumState == null) { - assigner = assignerFactory.createAssigner(); - } else { - LOG.info( - "Iceberg source restored {} splits from state for table {}", - enumState.pendingSplits().size(), - tableName); - assigner = assignerFactory.createAssigner(enumState.pendingSplits()); - } - if (scanContext.isStreaming()) { - ContinuousSplitPlanner splitPlanner = - new ContinuousSplitPlannerImpl(tableLoader, scanContext, planningThreadName()); - return new ContinuousIcebergEnumerator( - enumContext, assigner, scanContext, splitPlanner, enumState); - } else { - if (enumState == null) { - // Only do scan planning if nothing is restored from checkpoint state - List splits = planSplitsForBatch(planningThreadName()); - assigner.onDiscoveredSplits(splits); - } - - return new StaticIcebergEnumerator(enumContext, assigner); - } - } - - public static Builder builder() { - return new Builder<>(); - } - - public static Builder forRowData() { - return new Builder<>(); - } - - public static class Builder { - private TableLoader tableLoader; - private Table table; - private SplitAssignerFactory splitAssignerFactory; - private SerializableComparator splitComparator; - private ReaderFunction readerFunction; - private ReadableConfig flinkConfig = new Configuration(); - private final ScanContext.Builder contextBuilder = ScanContext.builder(); - private TableSchema projectedFlinkSchema; - private Boolean exposeLocality; - - private final Map readOptions = Maps.newHashMap(); - - Builder() {} - - public Builder tableLoader(TableLoader loader) { - this.tableLoader = loader; - return this; - } - - public Builder table(Table newTable) { - this.table = newTable; - return this; - } - - public Builder assignerFactory(SplitAssignerFactory assignerFactory) { - this.splitAssignerFactory = assignerFactory; - return this; - } - - public Builder splitComparator( - SerializableComparator newSplitComparator) { - this.splitComparator = newSplitComparator; - return this; - } - - public Builder readerFunction(ReaderFunction newReaderFunction) { - this.readerFunction = newReaderFunction; - return this; - } - - public Builder flinkConfig(ReadableConfig config) { - this.flinkConfig = config; - return this; - } - - public Builder caseSensitive(boolean newCaseSensitive) { - readOptions.put(FlinkReadOptions.CASE_SENSITIVE, Boolean.toString(newCaseSensitive)); - return this; - } - - public Builder useSnapshotId(Long newSnapshotId) { - if (newSnapshotId != null) { - readOptions.put(FlinkReadOptions.SNAPSHOT_ID.key(), Long.toString(newSnapshotId)); - } - return this; - } - - public Builder streamingStartingStrategy(StreamingStartingStrategy newStartingStrategy) { - readOptions.put(FlinkReadOptions.STARTING_STRATEGY, newStartingStrategy.name()); - return this; - } - - public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { - if (newStartSnapshotTimestamp != null) { - readOptions.put( - FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.key(), - Long.toString(newStartSnapshotTimestamp)); - } - return this; - } - - public Builder startSnapshotId(Long newStartSnapshotId) { - if (newStartSnapshotId != null) { - readOptions.put( - FlinkReadOptions.START_SNAPSHOT_ID.key(), Long.toString(newStartSnapshotId)); - } - return this; - } - - public Builder tag(String tag) { - readOptions.put(FlinkReadOptions.TAG.key(), tag); - return this; - } - - public Builder branch(String branch) { - readOptions.put(FlinkReadOptions.BRANCH.key(), branch); - return this; - } - - public Builder startTag(String startTag) { - readOptions.put(FlinkReadOptions.START_TAG.key(), startTag); - return this; - } - - public Builder endTag(String endTag) { - readOptions.put(FlinkReadOptions.END_TAG.key(), endTag); - return this; - } - - public Builder endSnapshotId(Long newEndSnapshotId) { - if (newEndSnapshotId != null) { - readOptions.put(FlinkReadOptions.END_SNAPSHOT_ID.key(), Long.toString(newEndSnapshotId)); - } - return this; - } - - public Builder asOfTimestamp(Long newAsOfTimestamp) { - if (newAsOfTimestamp != null) { - readOptions.put(FlinkReadOptions.AS_OF_TIMESTAMP.key(), Long.toString(newAsOfTimestamp)); - } - return this; - } - - public Builder splitSize(Long newSplitSize) { - if (newSplitSize != null) { - readOptions.put(FlinkReadOptions.SPLIT_SIZE, Long.toString(newSplitSize)); - } - return this; - } - - public Builder splitLookback(Integer newSplitLookback) { - if (newSplitLookback != null) { - readOptions.put(FlinkReadOptions.SPLIT_LOOKBACK, Integer.toString(newSplitLookback)); - } - return this; - } - - public Builder splitOpenFileCost(Long newSplitOpenFileCost) { - if (newSplitOpenFileCost != null) { - readOptions.put(FlinkReadOptions.SPLIT_FILE_OPEN_COST, Long.toString(newSplitOpenFileCost)); - } - - return this; - } - - public Builder streaming(boolean streaming) { - readOptions.put(FlinkReadOptions.STREAMING, Boolean.toString(streaming)); - return this; - } - - public Builder monitorInterval(Duration newMonitorInterval) { - if (newMonitorInterval != null) { - readOptions.put(FlinkReadOptions.MONITOR_INTERVAL, newMonitorInterval.toNanos() + " ns"); - } - return this; - } - - public Builder nameMapping(String newNameMapping) { - readOptions.put(TableProperties.DEFAULT_NAME_MAPPING, newNameMapping); - return this; - } - - public Builder project(Schema newProjectedSchema) { - this.contextBuilder.project(newProjectedSchema); - return this; - } - - public Builder project(TableSchema newProjectedFlinkSchema) { - this.projectedFlinkSchema = newProjectedFlinkSchema; - return this; - } - - public Builder filters(List newFilters) { - this.contextBuilder.filters(newFilters); - return this; - } - - public Builder limit(Long newLimit) { - if (newLimit != null) { - readOptions.put(FlinkReadOptions.LIMIT, Long.toString(newLimit)); - } - return this; - } - - public Builder includeColumnStats(boolean newIncludeColumnStats) { - readOptions.put( - FlinkReadOptions.INCLUDE_COLUMN_STATS, Boolean.toString(newIncludeColumnStats)); - return this; - } - - public Builder planParallelism(int planParallelism) { - readOptions.put( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.key(), - Integer.toString(planParallelism)); - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder maxAllowedPlanningFailures(int maxAllowedPlanningFailures) { - readOptions.put( - FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.key(), - Integer.toString(maxAllowedPlanningFailures)); - return this; - } - - /** - * Set the read properties for Flink source. View the supported properties in {@link - * FlinkReadOptions} - */ - public Builder set(String property, String value) { - readOptions.put(property, value); - return this; - } - - /** - * Set the read properties for Flink source. View the supported properties in {@link - * FlinkReadOptions} - */ - public Builder setAll(Map properties) { - readOptions.putAll(properties); - return this; - } - - /** - * Emits watermarks once per split based on the min value of column statistics from files - * metadata in the given split. The generated watermarks are also used for ordering the splits - * for read. Accepted column types are timestamp/timestamptz/long. For long columns consider - * setting {@link #watermarkColumnTimeUnit(TimeUnit)}. - * - *

    Consider setting `read.split.open-file-cost` to prevent combining small files to a single - * split when the watermark is used for watermark alignment. - */ - public Builder watermarkColumn(String columnName) { - Preconditions.checkArgument( - splitAssignerFactory == null, - "Watermark column and SplitAssigner should not be set in the same source"); - readOptions.put(FlinkReadOptions.WATERMARK_COLUMN, columnName); - return this; - } - - /** - * When the type of the {@link #watermarkColumn} is {@link - * org.apache.iceberg.types.Types.LongType}, then sets the {@link TimeUnit} to convert the - * value. The default value is {@link TimeUnit#MICROSECONDS}. - */ - public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { - readOptions.put(FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT, timeUnit.name()); - return this; - } - - /** @deprecated Use {@link #setAll} instead. */ - @Deprecated - public Builder properties(Map properties) { - readOptions.putAll(properties); - return this; - } - - public IcebergSource build() { - if (table == null) { - try (TableLoader loader = tableLoader) { - loader.open(); - this.table = tableLoader.loadTable(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - - contextBuilder.resolveConfig(table, readOptions, flinkConfig); - Schema icebergSchema = table.schema(); - if (projectedFlinkSchema != null) { - contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); - } - - SerializableRecordEmitter emitter = SerializableRecordEmitter.defaultEmitter(); - FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, flinkConfig); - String watermarkColumn = flinkReadConf.watermarkColumn(); - TimeUnit watermarkTimeUnit = flinkReadConf.watermarkColumnTimeUnit(); - - if (watermarkColumn != null) { - // Column statistics is needed for watermark generation - contextBuilder.includeColumnStats(Sets.newHashSet(watermarkColumn)); - - SplitWatermarkExtractor watermarkExtractor = - new ColumnStatsWatermarkExtractor(icebergSchema, watermarkColumn, watermarkTimeUnit); - emitter = SerializableRecordEmitter.emitterWithWatermark(watermarkExtractor); - splitAssignerFactory = - new OrderedSplitAssignerFactory(SplitComparators.watermark(watermarkExtractor)); - } - - ScanContext context = contextBuilder.build(); - context.validate(); - if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } - } - - if (splitAssignerFactory == null) { - if (splitComparator == null) { - splitAssignerFactory = new SimpleSplitAssignerFactory(); - } else { - splitAssignerFactory = new OrderedSplitAssignerFactory(splitComparator); - } - } - - // Since builder already load the table, pass it to the source to avoid double loading - return new IcebergSource<>( - tableLoader, - context, - readerFunction, - splitAssignerFactory, - splitComparator, - table, - emitter); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java deleted file mode 100644 index 610657e8d47b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ /dev/null @@ -1,229 +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.source; - -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.connector.source.abilities.SupportsFilterPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsLimitPushDown; -import org.apache.flink.table.connector.source.abilities.SupportsProjectionPushDown; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.types.DataType; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkFilters; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.SplitAssignerType; -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; - -/** Flink Iceberg table source. */ -@Internal -public class IcebergTableSource - implements ScanTableSource, - SupportsProjectionPushDown, - SupportsFilterPushDown, - SupportsLimitPushDown { - - private int[] projectedFields; - private Long limit; - private List filters; - - private final TableLoader loader; - private final TableSchema schema; - private final Map properties; - private final boolean isLimitPushDown; - private final ReadableConfig readableConfig; - - private IcebergTableSource(IcebergTableSource toCopy) { - this.loader = toCopy.loader; - this.schema = toCopy.schema; - this.properties = toCopy.properties; - this.projectedFields = toCopy.projectedFields; - this.isLimitPushDown = toCopy.isLimitPushDown; - this.limit = toCopy.limit; - this.filters = toCopy.filters; - this.readableConfig = toCopy.readableConfig; - } - - public IcebergTableSource( - TableLoader loader, - TableSchema schema, - Map properties, - ReadableConfig readableConfig) { - this(loader, schema, properties, null, false, null, ImmutableList.of(), readableConfig); - } - - private IcebergTableSource( - TableLoader loader, - TableSchema schema, - Map properties, - int[] projectedFields, - boolean isLimitPushDown, - Long limit, - List filters, - ReadableConfig readableConfig) { - this.loader = loader; - this.schema = schema; - this.properties = properties; - this.projectedFields = projectedFields; - this.isLimitPushDown = isLimitPushDown; - this.limit = limit; - this.filters = filters; - this.readableConfig = readableConfig; - } - - @Override - public void applyProjection(int[][] projectFields) { - this.projectedFields = new int[projectFields.length]; - for (int i = 0; i < projectFields.length; i++) { - Preconditions.checkArgument( - projectFields[i].length == 1, "Don't support nested projection in iceberg source now."); - this.projectedFields[i] = projectFields[i][0]; - } - } - - private DataStream createDataStream(StreamExecutionEnvironment execEnv) { - return FlinkSource.forRowData() - .env(execEnv) - .tableLoader(loader) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConf(readableConfig) - .build(); - } - - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { - SplitAssignerType assignerType = - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; - } - - private TableSchema getProjectedSchema() { - if (projectedFields == null) { - return schema; - } else { - String[] fullNames = schema.getFieldNames(); - DataType[] fullTypes = schema.getFieldDataTypes(); - return TableSchema.builder() - .fields( - Arrays.stream(projectedFields).mapToObj(i -> fullNames[i]).toArray(String[]::new), - Arrays.stream(projectedFields).mapToObj(i -> fullTypes[i]).toArray(DataType[]::new)) - .build(); - } - } - - @Override - public void applyLimit(long newLimit) { - this.limit = newLimit; - } - - @Override - public Result applyFilters(List flinkFilters) { - List acceptedFilters = Lists.newArrayList(); - List expressions = Lists.newArrayList(); - - for (ResolvedExpression resolvedExpression : flinkFilters) { - Optional icebergExpression = FlinkFilters.convert(resolvedExpression); - if (icebergExpression.isPresent()) { - expressions.add(icebergExpression.get()); - acceptedFilters.add(resolvedExpression); - } - } - - this.filters = expressions; - return Result.of(acceptedFilters, flinkFilters); - } - - @Override - public boolean supportsNestedProjection() { - // TODO: support nested projection - return false; - } - - @Override - public ChangelogMode getChangelogMode() { - return ChangelogMode.insertOnly(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return new DataStreamScanProvider() { - @Override - public DataStream produceDataStream( - ProviderContext providerContext, StreamExecutionEnvironment execEnv) { - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE)) { - return createFLIP27Stream(execEnv); - } else { - return createDataStream(execEnv); - } - } - - @Override - public boolean isBounded() { - return FlinkSource.isBounded(properties); - } - }; - } - - @Override - public DynamicTableSource copy() { - return new IcebergTableSource(this); - } - - @Override - public String asSummaryString() { - return "Iceberg table source"; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java deleted file mode 100644 index 88364f4e87b1..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataFileScanTaskReader.java +++ /dev/null @@ -1,243 +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.source; - -import java.util.List; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.DeleteFilter; -import org.apache.iceberg.encryption.InputFilesDecryptor; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -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.RowDataProjection; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.InputFile; -import org.apache.iceberg.mapping.NameMappingParser; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.TypeUtil; -import org.apache.iceberg.util.PartitionUtil; - -@Internal -public class RowDataFileScanTaskReader implements FileScanTaskReader { - - private final Schema tableSchema; - private final Schema projectedSchema; - private final String nameMapping; - private final boolean caseSensitive; - private final FlinkSourceFilter rowFilter; - - public RowDataFileScanTaskReader( - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - List filters) { - this.tableSchema = tableSchema; - this.projectedSchema = projectedSchema; - this.nameMapping = nameMapping; - this.caseSensitive = caseSensitive; - - if (filters != null && !filters.isEmpty()) { - Expression combinedExpression = - filters.stream().reduce(Expressions.alwaysTrue(), Expressions::and); - this.rowFilter = - new FlinkSourceFilter(this.projectedSchema, combinedExpression, this.caseSensitive); - } else { - this.rowFilter = null; - } - } - - @Override - public CloseableIterator open( - FileScanTask task, InputFilesDecryptor inputFilesDecryptor) { - Schema partitionSchema = TypeUtil.select(projectedSchema, task.spec().identitySourceIds()); - - Map idToConstant = - partitionSchema.columns().isEmpty() - ? ImmutableMap.of() - : PartitionUtil.constantsMap(task, RowDataUtil::convertConstant); - - FlinkDeleteFilter deletes = - new FlinkDeleteFilter(task, tableSchema, projectedSchema, inputFilesDecryptor); - CloseableIterable iterable = - deletes.filter( - newIterable(task, deletes.requiredSchema(), idToConstant, inputFilesDecryptor)); - - // Project the RowData to remove the extra meta columns. - if (!projectedSchema.sameSchema(deletes.requiredSchema())) { - RowDataProjection rowDataProjection = - RowDataProjection.create( - deletes.requiredRowType(), - deletes.requiredSchema().asStruct(), - projectedSchema.asStruct()); - iterable = CloseableIterable.transform(iterable, rowDataProjection::wrap); - } - - return iterable.iterator(); - } - - private CloseableIterable newIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - CloseableIterable iter; - if (task.isDataTask()) { - throw new UnsupportedOperationException("Cannot read data task."); - } else { - switch (task.file().format()) { - case PARQUET: - iter = newParquetIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case AVRO: - iter = newAvroIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - case ORC: - iter = newOrcIterable(task, schema, idToConstant, inputFilesDecryptor); - break; - - default: - throw new UnsupportedOperationException( - "Cannot read unknown format: " + task.file().format()); - } - } - - if (rowFilter != null) { - return CloseableIterable.filter(iter, rowFilter::filter); - } - return iter; - } - - private CloseableIterable newAvroIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Avro.ReadBuilder builder = - Avro.read(inputFilesDecryptor.getInputFile(task)) - .reuseContainers() - .project(schema) - .split(task.start(), task.length()) - .createReaderFunc(readSchema -> new FlinkAvroReader(schema, readSchema, idToConstant)); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newParquetIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Parquet.ReadBuilder builder = - Parquet.read(inputFilesDecryptor.getInputFile(task)) - .split(task.start(), task.length()) - .project(schema) - .createReaderFunc( - fileSchema -> FlinkParquetReaders.buildReader(schema, fileSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive) - .reuseContainers(); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private CloseableIterable newOrcIterable( - FileScanTask task, - Schema schema, - Map idToConstant, - InputFilesDecryptor inputFilesDecryptor) { - Schema readSchemaWithoutConstantAndMetadataFields = - TypeUtil.selectNot( - schema, Sets.union(idToConstant.keySet(), MetadataColumns.metadataFieldIds())); - - ORC.ReadBuilder builder = - ORC.read(inputFilesDecryptor.getInputFile(task)) - .project(readSchemaWithoutConstantAndMetadataFields) - .split(task.start(), task.length()) - .createReaderFunc( - readOrcSchema -> new FlinkOrcReader(schema, readOrcSchema, idToConstant)) - .filter(task.residual()) - .caseSensitive(caseSensitive); - - if (nameMapping != null) { - builder.withNameMapping(NameMappingParser.fromJson(nameMapping)); - } - - return builder.build(); - } - - private static class FlinkDeleteFilter extends DeleteFilter { - private final RowType requiredRowType; - private final RowDataWrapper asStructLike; - private final InputFilesDecryptor inputFilesDecryptor; - - FlinkDeleteFilter( - FileScanTask task, - Schema tableSchema, - Schema requestedSchema, - InputFilesDecryptor inputFilesDecryptor) { - super(task.file().path().toString(), task.deletes(), tableSchema, requestedSchema); - this.requiredRowType = FlinkSchemaUtil.convert(requiredSchema()); - this.asStructLike = new RowDataWrapper(requiredRowType, requiredSchema().asStruct()); - this.inputFilesDecryptor = inputFilesDecryptor; - } - - public RowType requiredRowType() { - return requiredRowType; - } - - @Override - protected StructLike asStructLike(RowData row) { - return asStructLike.wrap(row); - } - - @Override - protected InputFile getInputFile(String location) { - return inputFilesDecryptor.getInputFile(location); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java deleted file mode 100644 index c958604c004a..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ /dev/null @@ -1,172 +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.source; - -import static org.apache.iceberg.TableProperties.DEFAULT_NAME_MAPPING; - -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; -import org.apache.iceberg.flink.sink.TaskWriterFactory; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -public class RowDataRewriter { - - private static final Logger LOG = LoggerFactory.getLogger(RowDataRewriter.class); - - private final Schema schema; - private final String nameMapping; - private final FileIO io; - private final boolean caseSensitive; - private final EncryptionManager encryptionManager; - private final TaskWriterFactory taskWriterFactory; - private final String tableName; - - public RowDataRewriter( - Table table, boolean caseSensitive, FileIO io, EncryptionManager encryptionManager) { - this.schema = table.schema(); - this.caseSensitive = caseSensitive; - this.io = io; - this.encryptionManager = encryptionManager; - this.nameMapping = - PropertyUtil.propertyAsString(table.properties(), DEFAULT_NAME_MAPPING, null); - this.tableName = table.name(); - - String formatString = - PropertyUtil.propertyAsString( - table.properties(), - TableProperties.DEFAULT_FILE_FORMAT, - TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); - FileFormat format = FileFormat.fromString(formatString); - RowType flinkSchema = FlinkSchemaUtil.convert(table.schema()); - this.taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - flinkSchema, - Long.MAX_VALUE, - format, - table.properties(), - null, - false); - } - - public List rewriteDataForTasks( - DataStream dataStream, int parallelism) throws Exception { - RewriteMap map = - new RewriteMap( - schema, nameMapping, io, caseSensitive, encryptionManager, taskWriterFactory); - DataStream> ds = dataStream.map(map).setParallelism(parallelism); - return Lists.newArrayList(ds.executeAndCollect("Rewrite table :" + tableName)).stream() - .flatMap(Collection::stream) - .collect(Collectors.toList()); - } - - public static class RewriteMap extends RichMapFunction> { - - private TaskWriter writer; - private int subTaskId; - private int attemptId; - - private final Schema schema; - private final String nameMapping; - private final FileIO io; - private final boolean caseSensitive; - private final EncryptionManager encryptionManager; - private final TaskWriterFactory taskWriterFactory; - private final RowDataFileScanTaskReader rowDataReader; - - public RewriteMap( - Schema schema, - String nameMapping, - FileIO io, - boolean caseSensitive, - EncryptionManager encryptionManager, - TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; - this.io = io; - this.caseSensitive = caseSensitive; - this.encryptionManager = encryptionManager; - this.taskWriterFactory = taskWriterFactory; - this.rowDataReader = - new RowDataFileScanTaskReader( - schema, schema, nameMapping, caseSensitive, Collections.emptyList()); - } - - @Override - public void open(Configuration parameters) { - this.subTaskId = getRuntimeContext().getIndexOfThisSubtask(); - this.attemptId = getRuntimeContext().getAttemptNumber(); - // Initialize the task writer factory. - this.taskWriterFactory.initialize(subTaskId, attemptId); - } - - @Override - public List map(CombinedScanTask task) throws Exception { - // Initialize the task writer. - this.writer = taskWriterFactory.create(); - try (DataIterator iterator = - new DataIterator<>(rowDataReader, task, io, encryptionManager)) { - while (iterator.hasNext()) { - RowData rowData = iterator.next(); - writer.write(rowData); - } - return Lists.newArrayList(writer.dataFiles()); - } catch (Throwable originalThrowable) { - try { - LOG.error("Aborting commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); - writer.abort(); - LOG.error("Aborted commit for (subTaskId {}, attemptId {})", subTaskId, attemptId); - } catch (Throwable inner) { - if (originalThrowable != inner) { - originalThrowable.addSuppressed(inner); - LOG.warn("Suppressing exception in catch: {}", inner.getMessage(), inner); - } - } - - if (originalThrowable instanceof Exception) { - throw originalThrowable; - } else { - throw new RuntimeException(originalThrowable); - } - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.java deleted file mode 100644 index 8ef1f1fbb833..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/RowDataToAvroGenericRecordConverter.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.flink.source; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.annotation.Internal; -import org.apache.flink.formats.avro.RowDataToAvroConverters; -import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.DataType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.utils.TypeConversions; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.flink.FlinkSchemaUtil; - -/** - * This is not serializable because Avro {@link Schema} is not actually serializable, even though it - * implements {@link Serializable} interface. - */ -@Internal -public class RowDataToAvroGenericRecordConverter implements Function { - private final RowDataToAvroConverters.RowDataToAvroConverter converter; - private final Schema avroSchema; - - private RowDataToAvroGenericRecordConverter(RowType rowType, Schema avroSchema) { - this.converter = RowDataToAvroConverters.createConverter(rowType); - this.avroSchema = avroSchema; - } - - @Override - public GenericRecord apply(RowData rowData) { - return (GenericRecord) converter.convert(avroSchema, rowData); - } - - /** Create a converter based on Iceberg schema */ - public static RowDataToAvroGenericRecordConverter fromIcebergSchema( - String tableName, org.apache.iceberg.Schema icebergSchema) { - RowType rowType = FlinkSchemaUtil.convert(icebergSchema); - Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); - return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); - } - - /** Create a mapper based on Avro schema */ - public static RowDataToAvroGenericRecordConverter fromAvroSchema(Schema avroSchema) { - DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); - LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); - RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); - return new RowDataToAvroGenericRecordConverter(rowType, avroSchema); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java deleted file mode 100644 index ab79a3173933..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java +++ /dev/null @@ -1,597 +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.source; - -import java.io.Serializable; -import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.util.Preconditions; -import org.apache.flink.util.TimeUtils; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadConf; -import org.apache.iceberg.flink.FlinkReadOptions; - -/** Context object with optional arguments for a Flink Scan. */ -@Internal -public class ScanContext implements Serializable { - - private static final long serialVersionUID = 1L; - - private final boolean caseSensitive; - private final boolean exposeLocality; - private final Long snapshotId; - private final String branch; - private final String tag; - private final StreamingStartingStrategy startingStrategy; - private final Long startSnapshotId; - private final Long startSnapshotTimestamp; - private final Long endSnapshotId; - private final Long asOfTimestamp; - private final String startTag; - private final String endTag; - private final Long splitSize; - private final Integer splitLookback; - private final Long splitOpenFileCost; - private final boolean isStreaming; - private final Duration monitorInterval; - - private final String nameMapping; - private final Schema schema; - private final List filters; - private final long limit; - private final boolean includeColumnStats; - private final Collection includeStatsForColumns; - private final Integer planParallelism; - private final int maxPlanningSnapshotCount; - private final int maxAllowedPlanningFailures; - private final String watermarkColumn; - private final TimeUnit watermarkColumnTimeUnit; - - private ScanContext( - boolean caseSensitive, - Long snapshotId, - StreamingStartingStrategy startingStrategy, - Long startSnapshotTimestamp, - Long startSnapshotId, - Long endSnapshotId, - Long asOfTimestamp, - Long splitSize, - Integer splitLookback, - Long splitOpenFileCost, - boolean isStreaming, - Duration monitorInterval, - String nameMapping, - Schema schema, - List filters, - long limit, - boolean includeColumnStats, - Collection includeStatsForColumns, - boolean exposeLocality, - Integer planParallelism, - int maxPlanningSnapshotCount, - int maxAllowedPlanningFailures, - String watermarkColumn, - TimeUnit watermarkColumnTimeUnit, - String branch, - String tag, - String startTag, - String endTag) { - this.caseSensitive = caseSensitive; - this.snapshotId = snapshotId; - this.tag = tag; - this.branch = branch; - this.startingStrategy = startingStrategy; - this.startSnapshotTimestamp = startSnapshotTimestamp; - this.startSnapshotId = startSnapshotId; - this.endSnapshotId = endSnapshotId; - this.asOfTimestamp = asOfTimestamp; - this.startTag = startTag; - this.endTag = endTag; - this.splitSize = splitSize; - this.splitLookback = splitLookback; - this.splitOpenFileCost = splitOpenFileCost; - this.isStreaming = isStreaming; - this.monitorInterval = monitorInterval; - - this.nameMapping = nameMapping; - this.schema = schema; - this.filters = filters; - this.limit = limit; - this.includeColumnStats = includeColumnStats; - this.includeStatsForColumns = includeStatsForColumns; - this.exposeLocality = exposeLocality; - this.planParallelism = planParallelism; - this.maxPlanningSnapshotCount = maxPlanningSnapshotCount; - this.maxAllowedPlanningFailures = maxAllowedPlanningFailures; - this.watermarkColumn = watermarkColumn; - this.watermarkColumnTimeUnit = watermarkColumnTimeUnit; - } - - void validate() { - if (isStreaming) { - if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) { - Preconditions.checkArgument( - startSnapshotId != null, - "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); - Preconditions.checkArgument( - startSnapshotTimestamp == null, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - if (startingStrategy == StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) { - Preconditions.checkArgument( - startSnapshotTimestamp != null, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); - Preconditions.checkArgument( - startSnapshotId == null, - "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - - Preconditions.checkArgument( - tag == null, - String.format("Cannot scan table using ref %s configured for streaming reader", tag)); - Preconditions.checkArgument( - snapshotId == null, "Cannot set snapshot-id option for streaming reader"); - Preconditions.checkArgument( - asOfTimestamp == null, "Cannot set as-of-timestamp option for streaming reader"); - Preconditions.checkArgument( - endSnapshotId == null, "Cannot set end-snapshot-id option for streaming reader"); - Preconditions.checkArgument(endTag == null, "Cannot set end-tag option for streaming reader"); - } - - Preconditions.checkArgument( - !(startTag != null && startSnapshotId() != null), - "START_SNAPSHOT_ID and START_TAG cannot both be set."); - - Preconditions.checkArgument( - !(endTag != null && endSnapshotId() != null), - "END_SNAPSHOT_ID and END_TAG cannot both be set."); - - Preconditions.checkArgument( - maxAllowedPlanningFailures >= -1, - "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); - } - - public boolean caseSensitive() { - return caseSensitive; - } - - public Long snapshotId() { - return snapshotId; - } - - public String branch() { - return branch; - } - - public String tag() { - return tag; - } - - public String startTag() { - return startTag; - } - - public String endTag() { - return endTag; - } - - public StreamingStartingStrategy streamingStartingStrategy() { - return startingStrategy; - } - - public Long startSnapshotTimestamp() { - return startSnapshotTimestamp; - } - - public Long startSnapshotId() { - return startSnapshotId; - } - - public Long endSnapshotId() { - return endSnapshotId; - } - - public Long asOfTimestamp() { - return asOfTimestamp; - } - - public Long splitSize() { - return splitSize; - } - - public Integer splitLookback() { - return splitLookback; - } - - public Long splitOpenFileCost() { - return splitOpenFileCost; - } - - public boolean isStreaming() { - return isStreaming; - } - - public Duration monitorInterval() { - return monitorInterval; - } - - public String nameMapping() { - return nameMapping; - } - - public Schema project() { - return schema; - } - - public List filters() { - return filters; - } - - public long limit() { - return limit; - } - - public boolean includeColumnStats() { - return includeColumnStats; - } - - public Collection includeStatsForColumns() { - return includeStatsForColumns; - } - - public boolean exposeLocality() { - return exposeLocality; - } - - public Integer planParallelism() { - return planParallelism; - } - - public int maxPlanningSnapshotCount() { - return maxPlanningSnapshotCount; - } - - public int maxAllowedPlanningFailures() { - return maxAllowedPlanningFailures; - } - - public String watermarkColumn() { - return watermarkColumn; - } - - public TimeUnit watermarkColumnTimeUnit() { - return watermarkColumnTimeUnit; - } - - public ScanContext copyWithAppendsBetween(Long newStartSnapshotId, long newEndSnapshotId) { - return ScanContext.builder() - .caseSensitive(caseSensitive) - .useSnapshotId(null) - .useBranch(branch) - .useTag(null) - .startSnapshotId(newStartSnapshotId) - .endSnapshotId(newEndSnapshotId) - .startTag(null) - .endTag(null) - .asOfTimestamp(null) - .splitSize(splitSize) - .splitLookback(splitLookback) - .splitOpenFileCost(splitOpenFileCost) - .streaming(isStreaming) - .monitorInterval(monitorInterval) - .nameMapping(nameMapping) - .project(schema) - .filters(filters) - .limit(limit) - .includeColumnStats(includeColumnStats) - .includeColumnStats(includeStatsForColumns) - .exposeLocality(exposeLocality) - .planParallelism(planParallelism) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .watermarkColumn(watermarkColumn) - .watermarkColumnTimeUnit(watermarkColumnTimeUnit) - .build(); - } - - public ScanContext copyWithSnapshotId(long newSnapshotId) { - return ScanContext.builder() - .caseSensitive(caseSensitive) - .useSnapshotId(newSnapshotId) - .useBranch(branch) - .useTag(tag) - .startSnapshotId(null) - .endSnapshotId(null) - .startTag(null) - .endTag(null) - .asOfTimestamp(null) - .splitSize(splitSize) - .splitLookback(splitLookback) - .splitOpenFileCost(splitOpenFileCost) - .streaming(isStreaming) - .monitorInterval(monitorInterval) - .nameMapping(nameMapping) - .project(schema) - .filters(filters) - .limit(limit) - .includeColumnStats(includeColumnStats) - .includeColumnStats(includeStatsForColumns) - .exposeLocality(exposeLocality) - .planParallelism(planParallelism) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .watermarkColumn(watermarkColumn) - .watermarkColumnTimeUnit(watermarkColumnTimeUnit) - .build(); - } - - public static Builder builder() { - return new Builder(); - } - - public static class Builder { - private boolean caseSensitive = FlinkReadOptions.CASE_SENSITIVE_OPTION.defaultValue(); - private Long snapshotId = FlinkReadOptions.SNAPSHOT_ID.defaultValue(); - private String branch = FlinkReadOptions.BRANCH.defaultValue(); - private String tag = FlinkReadOptions.TAG.defaultValue(); - private String startTag = FlinkReadOptions.START_TAG.defaultValue(); - private String endTag = FlinkReadOptions.END_TAG.defaultValue(); - private StreamingStartingStrategy startingStrategy = - FlinkReadOptions.STARTING_STRATEGY_OPTION.defaultValue(); - private Long startSnapshotTimestamp = FlinkReadOptions.START_SNAPSHOT_TIMESTAMP.defaultValue(); - private Long startSnapshotId = FlinkReadOptions.START_SNAPSHOT_ID.defaultValue(); - private Long endSnapshotId = FlinkReadOptions.END_SNAPSHOT_ID.defaultValue(); - private Long asOfTimestamp = FlinkReadOptions.AS_OF_TIMESTAMP.defaultValue(); - private Long splitSize = FlinkReadOptions.SPLIT_SIZE_OPTION.defaultValue(); - private Integer splitLookback = FlinkReadOptions.SPLIT_LOOKBACK_OPTION.defaultValue(); - private Long splitOpenFileCost = FlinkReadOptions.SPLIT_FILE_OPEN_COST_OPTION.defaultValue(); - private boolean isStreaming = FlinkReadOptions.STREAMING_OPTION.defaultValue(); - private Duration monitorInterval = - TimeUtils.parseDuration(FlinkReadOptions.MONITOR_INTERVAL_OPTION.defaultValue()); - private String nameMapping; - private Schema projectedSchema; - private List filters; - private long limit = FlinkReadOptions.LIMIT_OPTION.defaultValue(); - private boolean includeColumnStats = - FlinkReadOptions.INCLUDE_COLUMN_STATS_OPTION.defaultValue(); - private Collection includeStatsForColumns = null; - private boolean exposeLocality; - private Integer planParallelism = - FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE.defaultValue(); - private int maxPlanningSnapshotCount = - FlinkReadOptions.MAX_PLANNING_SNAPSHOT_COUNT_OPTION.defaultValue(); - private int maxAllowedPlanningFailures = - FlinkReadOptions.MAX_ALLOWED_PLANNING_FAILURES_OPTION.defaultValue(); - private String watermarkColumn = FlinkReadOptions.WATERMARK_COLUMN_OPTION.defaultValue(); - private TimeUnit watermarkColumnTimeUnit = - FlinkReadOptions.WATERMARK_COLUMN_TIME_UNIT_OPTION.defaultValue(); - - private Builder() {} - - public Builder caseSensitive(boolean newCaseSensitive) { - this.caseSensitive = newCaseSensitive; - return this; - } - - public Builder useSnapshotId(Long newSnapshotId) { - this.snapshotId = newSnapshotId; - return this; - } - - public Builder useTag(String newTag) { - this.tag = newTag; - return this; - } - - public Builder useBranch(String newBranch) { - this.branch = newBranch; - return this; - } - - public Builder startingStrategy(StreamingStartingStrategy newStartingStrategy) { - this.startingStrategy = newStartingStrategy; - return this; - } - - public Builder startSnapshotTimestamp(Long newStartSnapshotTimestamp) { - this.startSnapshotTimestamp = newStartSnapshotTimestamp; - return this; - } - - public Builder startSnapshotId(Long newStartSnapshotId) { - this.startSnapshotId = newStartSnapshotId; - return this; - } - - public Builder endSnapshotId(Long newEndSnapshotId) { - this.endSnapshotId = newEndSnapshotId; - return this; - } - - public Builder startTag(String newStartTag) { - this.startTag = newStartTag; - return this; - } - - public Builder endTag(String newEndTag) { - this.endTag = newEndTag; - return this; - } - - public Builder asOfTimestamp(Long newAsOfTimestamp) { - this.asOfTimestamp = newAsOfTimestamp; - return this; - } - - public Builder splitSize(Long newSplitSize) { - this.splitSize = newSplitSize; - return this; - } - - public Builder splitLookback(Integer newSplitLookback) { - this.splitLookback = newSplitLookback; - return this; - } - - public Builder splitOpenFileCost(Long newSplitOpenFileCost) { - this.splitOpenFileCost = newSplitOpenFileCost; - return this; - } - - public Builder streaming(boolean streaming) { - this.isStreaming = streaming; - return this; - } - - public Builder monitorInterval(Duration newMonitorInterval) { - this.monitorInterval = newMonitorInterval; - return this; - } - - public Builder nameMapping(String newNameMapping) { - this.nameMapping = newNameMapping; - return this; - } - - public Builder project(Schema newProjectedSchema) { - this.projectedSchema = newProjectedSchema; - return this; - } - - public Builder filters(List newFilters) { - this.filters = newFilters; - return this; - } - - public Builder limit(long newLimit) { - this.limit = newLimit; - return this; - } - - public Builder includeColumnStats(boolean newIncludeColumnStats) { - this.includeColumnStats = newIncludeColumnStats; - return this; - } - - public Builder includeColumnStats(Collection newIncludeStatsForColumns) { - this.includeStatsForColumns = newIncludeStatsForColumns; - return this; - } - - public Builder exposeLocality(boolean newExposeLocality) { - this.exposeLocality = newExposeLocality; - return this; - } - - public Builder planParallelism(Integer parallelism) { - this.planParallelism = parallelism; - return this; - } - - public Builder maxPlanningSnapshotCount(int newMaxPlanningSnapshotCount) { - this.maxPlanningSnapshotCount = newMaxPlanningSnapshotCount; - return this; - } - - public Builder maxAllowedPlanningFailures(int newMaxAllowedPlanningFailures) { - this.maxAllowedPlanningFailures = newMaxAllowedPlanningFailures; - return this; - } - - public Builder watermarkColumn(String newWatermarkColumn) { - this.watermarkColumn = newWatermarkColumn; - return this; - } - - public Builder watermarkColumnTimeUnit(TimeUnit newWatermarkTimeUnit) { - this.watermarkColumnTimeUnit = newWatermarkTimeUnit; - return this; - } - - public Builder resolveConfig( - Table table, Map readOptions, ReadableConfig readableConfig) { - FlinkReadConf flinkReadConf = new FlinkReadConf(table, readOptions, readableConfig); - - return this.useSnapshotId(flinkReadConf.snapshotId()) - .useTag(flinkReadConf.tag()) - .useBranch(flinkReadConf.branch()) - .startTag(flinkReadConf.startTag()) - .endTag(flinkReadConf.endTag()) - .caseSensitive(flinkReadConf.caseSensitive()) - .asOfTimestamp(flinkReadConf.asOfTimestamp()) - .startingStrategy(flinkReadConf.startingStrategy()) - .startSnapshotTimestamp(flinkReadConf.startSnapshotTimestamp()) - .startSnapshotId(flinkReadConf.startSnapshotId()) - .endSnapshotId(flinkReadConf.endSnapshotId()) - .splitSize(flinkReadConf.splitSize()) - .splitLookback(flinkReadConf.splitLookback()) - .splitOpenFileCost(flinkReadConf.splitFileOpenCost()) - .streaming(flinkReadConf.streaming()) - .monitorInterval(flinkReadConf.monitorInterval()) - .nameMapping(flinkReadConf.nameMapping()) - .limit(flinkReadConf.limit()) - .planParallelism(flinkReadConf.workerPoolSize()) - .includeColumnStats(flinkReadConf.includeColumnStats()) - .maxPlanningSnapshotCount(flinkReadConf.maxPlanningSnapshotCount()) - .maxAllowedPlanningFailures(maxAllowedPlanningFailures) - .watermarkColumn(flinkReadConf.watermarkColumn()) - .watermarkColumnTimeUnit(flinkReadConf.watermarkColumnTimeUnit()); - } - - public ScanContext build() { - return new ScanContext( - caseSensitive, - snapshotId, - startingStrategy, - startSnapshotTimestamp, - startSnapshotId, - endSnapshotId, - asOfTimestamp, - splitSize, - splitLookback, - splitOpenFileCost, - isStreaming, - monitorInterval, - nameMapping, - projectedSchema, - filters, - limit, - includeColumnStats, - includeStatsForColumns, - exposeLocality, - planParallelism, - maxPlanningSnapshotCount, - maxAllowedPlanningFailures, - watermarkColumn, - watermarkColumnTimeUnit, - branch, - tag, - startTag, - endTag); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java deleted file mode 100644 index 7c3a69dbc141..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/SourceUtil.java +++ /dev/null @@ -1,77 +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.source; - -import java.util.function.Supplier; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.api.config.ExecutionConfigOptions; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.hadoop.Util; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class SourceUtil { - private SourceUtil() {} - - static boolean isLocalityEnabled( - Table table, ReadableConfig readableConfig, Boolean exposeLocality) { - Boolean localityEnabled = - exposeLocality != null - ? exposeLocality - : readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO); - - if (localityEnabled != null && !localityEnabled) { - return false; - } - - return Util.mayHaveBlockLocations(table.io(), table.location()); - } - - /** - * Infer source parallelism. - * - * @param readableConfig Flink config. - * @param splitCountProvider Split count supplier. As the computation may involve expensive split - * discover, lazy evaluation is performed if inferring parallelism is enabled. - * @param limitCount limited output count. - */ - static int inferParallelism( - ReadableConfig readableConfig, long limitCount, Supplier splitCountProvider) { - int parallelism = - readableConfig.get(ExecutionConfigOptions.TABLE_EXEC_RESOURCE_DEFAULT_PARALLELISM); - if (readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM)) { - int maxInferParallelism = - readableConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX); - Preconditions.checkState( - maxInferParallelism >= 1, - FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX.key() - + " cannot be less than 1"); - parallelism = Math.min(splitCountProvider.get(), maxInferParallelism); - } - - if (limitCount > 0) { - int limit = limitCount >= Integer.MAX_VALUE ? Integer.MAX_VALUE : (int) limitCount; - parallelism = Math.min(parallelism, limit); - } - - // parallelism must be positive. - parallelism = Math.max(1, parallelism); - return parallelism; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java deleted file mode 100644 index a07613aee59b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java +++ /dev/null @@ -1,269 +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.source; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.List; -import java.util.concurrent.ExecutorService; -import org.apache.flink.api.common.functions.RuntimeContext; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.api.common.typeutils.base.LongSerializer; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.runtime.state.FunctionInitializationContext; -import org.apache.flink.runtime.state.FunctionSnapshotContext; -import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; -import org.apache.flink.streaming.api.functions.source.RichSourceFunction; -import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * This is the single (non-parallel) monitoring task which takes a {@link FlinkInputFormat}, it is - * responsible for: - * - *

      - *
    1. Monitoring snapshots of the Iceberg table. - *
    2. Creating the {@link FlinkInputSplit splits} corresponding to the incremental files - *
    3. Assigning them to downstream tasks for further processing. - *
    - * - *

    The splits to be read are forwarded to the downstream {@link StreamingReaderOperator} which - * can have parallelism greater than one. - */ -public class StreamingMonitorFunction extends RichSourceFunction - implements CheckpointedFunction { - - private static final Logger LOG = LoggerFactory.getLogger(StreamingMonitorFunction.class); - - private static final long INIT_LAST_SNAPSHOT_ID = -1L; - - private final TableLoader tableLoader; - private final ScanContext scanContext; - - private volatile boolean isRunning = true; - - // The checkpoint thread is not the same thread that running the function for SourceStreamTask - // now. It's necessary to - // mark this as volatile. - private volatile long lastSnapshotId = INIT_LAST_SNAPSHOT_ID; - - private transient SourceContext sourceContext; - private transient Table table; - private transient ListState lastSnapshotIdState; - private transient ExecutorService workerPool; - - public StreamingMonitorFunction(TableLoader tableLoader, ScanContext scanContext) { - Preconditions.checkArgument( - scanContext.snapshotId() == null, "Cannot set snapshot-id option for streaming reader"); - Preconditions.checkArgument( - scanContext.asOfTimestamp() == null, - "Cannot set as-of-timestamp option for streaming reader"); - Preconditions.checkArgument( - scanContext.endSnapshotId() == null, - "Cannot set end-snapshot-id option for streaming reader"); - Preconditions.checkArgument( - scanContext.endTag() == null, "Cannot set end-tag option for streaming reader"); - Preconditions.checkArgument( - scanContext.maxPlanningSnapshotCount() > 0, - "The max-planning-snapshot-count must be greater than zero"); - this.tableLoader = tableLoader; - this.scanContext = scanContext; - } - - @Override - public void open(Configuration parameters) throws Exception { - super.open(parameters); - - final RuntimeContext runtimeContext = getRuntimeContext(); - ValidationException.check( - runtimeContext instanceof StreamingRuntimeContext, - "context should be instance of StreamingRuntimeContext"); - final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); - this.workerPool = - ThreadPools.newWorkerPool( - "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); - } - - @Override - public void initializeState(FunctionInitializationContext context) throws Exception { - // Load iceberg table from table loader. - tableLoader.open(); - table = tableLoader.loadTable(); - - // Initialize the flink state for last snapshot id. - lastSnapshotIdState = - context - .getOperatorStateStore() - .getListState(new ListStateDescriptor<>("snapshot-id-state", LongSerializer.INSTANCE)); - - // Restore the last-snapshot-id from flink's state if possible. - if (context.isRestored()) { - LOG.info("Restoring state for the {}.", getClass().getSimpleName()); - lastSnapshotId = lastSnapshotIdState.get().iterator().next(); - } else if (scanContext.startTag() != null || scanContext.startSnapshotId() != null) { - Preconditions.checkArgument( - !(scanContext.startTag() != null && scanContext.startSnapshotId() != null), - "START_SNAPSHOT_ID and START_TAG cannot both be set."); - Preconditions.checkNotNull( - table.currentSnapshot(), "Don't have any available snapshot in table."); - - long startSnapshotId; - if (scanContext.startTag() != null) { - Preconditions.checkArgument( - table.snapshot(scanContext.startTag()) != null, - "Cannot find snapshot with tag %s in table.", - scanContext.startTag()); - startSnapshotId = table.snapshot(scanContext.startTag()).snapshotId(); - } else { - startSnapshotId = scanContext.startSnapshotId(); - } - - long currentSnapshotId = table.currentSnapshot().snapshotId(); - Preconditions.checkState( - SnapshotUtil.isAncestorOf(table, currentSnapshotId, startSnapshotId), - "The option start-snapshot-id %s is not an ancestor of the current snapshot.", - startSnapshotId); - - lastSnapshotId = startSnapshotId; - } - } - - @Override - public void snapshotState(FunctionSnapshotContext context) throws Exception { - lastSnapshotIdState.clear(); - lastSnapshotIdState.add(lastSnapshotId); - } - - @Override - public void run(SourceContext ctx) throws Exception { - this.sourceContext = ctx; - while (isRunning) { - monitorAndForwardSplits(); - Thread.sleep(scanContext.monitorInterval().toMillis()); - } - } - - private long toSnapshotIdInclusive( - long lastConsumedSnapshotId, long currentSnapshotId, int maxPlanningSnapshotCount) { - List snapshotIds = - SnapshotUtil.snapshotIdsBetween(table, lastConsumedSnapshotId, currentSnapshotId); - if (snapshotIds.size() <= maxPlanningSnapshotCount) { - return currentSnapshotId; - } else { - // It uses reverted index since snapshotIdsBetween returns Ids that are ordered by committed - // time descending. - return snapshotIds.get(snapshotIds.size() - maxPlanningSnapshotCount); - } - } - - @VisibleForTesting - void sourceContext(SourceContext ctx) { - this.sourceContext = ctx; - } - - @VisibleForTesting - void monitorAndForwardSplits() { - // Refresh the table to get the latest committed snapshot. - table.refresh(); - - Snapshot snapshot = - scanContext.branch() != null - ? table.snapshot(scanContext.branch()) - : table.currentSnapshot(); - if (snapshot != null && snapshot.snapshotId() != lastSnapshotId) { - long snapshotId = snapshot.snapshotId(); - - ScanContext newScanContext; - if (lastSnapshotId == INIT_LAST_SNAPSHOT_ID) { - newScanContext = scanContext.copyWithSnapshotId(snapshotId); - } else { - snapshotId = - toSnapshotIdInclusive( - lastSnapshotId, snapshotId, scanContext.maxPlanningSnapshotCount()); - newScanContext = scanContext.copyWithAppendsBetween(lastSnapshotId, snapshotId); - } - - LOG.debug( - "Start discovering splits from {} (exclusive) to {} (inclusive)", - lastSnapshotId, - snapshotId); - long start = System.currentTimeMillis(); - FlinkInputSplit[] splits = - FlinkSplitPlanner.planInputSplits(table, newScanContext, workerPool); - LOG.debug( - "Discovered {} splits, time elapsed {}ms", - splits.length, - System.currentTimeMillis() - start); - - // only need to hold the checkpoint lock when emitting the splits and updating lastSnapshotId - start = System.currentTimeMillis(); - synchronized (sourceContext.getCheckpointLock()) { - for (FlinkInputSplit split : splits) { - sourceContext.collect(split); - } - - lastSnapshotId = snapshotId; - } - LOG.debug( - "Forwarded {} splits, time elapsed {}ms", - splits.length, - System.currentTimeMillis() - start); - } - } - - @Override - public void cancel() { - // this is to cover the case where cancel() is called before the run() - if (sourceContext != null) { - synchronized (sourceContext.getCheckpointLock()) { - isRunning = false; - } - } else { - isRunning = false; - } - - // Release all the resources here. - if (tableLoader != null) { - try { - tableLoader.close(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } - } - - @Override - public void close() { - cancel(); - - if (workerPool != null) { - workerPool.shutdown(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java deleted file mode 100644 index ee6f7b63988d..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingReaderOperator.java +++ /dev/null @@ -1,246 +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.source; - -import java.io.IOException; -import java.util.Queue; -import org.apache.flink.api.common.operators.MailboxExecutor; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.ListStateDescriptor; -import org.apache.flink.runtime.state.JavaSerializer; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateSnapshotContext; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.AbstractStreamOperator; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.OneInputStreamOperator; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.api.operators.StreamSourceContexts; -import org.apache.flink.streaming.api.operators.YieldingOperatorFactory; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.ProcessingTimeService; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The operator that reads the {@link FlinkInputSplit splits} received from the preceding {@link - * StreamingMonitorFunction}. Contrary to the {@link StreamingMonitorFunction} which has a - * parallelism of 1, this operator can have multiple parallelism. - * - *

    As soon as a split descriptor is received, it is put in a queue, and use {@link - * MailboxExecutor} read the actual data of the split. This architecture allows the separation of - * the reading thread from the one split processing the checkpoint barriers, thus removing any - * potential back-pressure. - */ -public class StreamingReaderOperator extends AbstractStreamOperator - implements OneInputStreamOperator { - - private static final Logger LOG = LoggerFactory.getLogger(StreamingReaderOperator.class); - - // It's the same thread that is running this operator and checkpoint actions. we use this executor - // to schedule only - // one split for future reading, so that a new checkpoint could be triggered without blocking long - // time for exhausting - // all scheduled splits. - private final MailboxExecutor executor; - private FlinkInputFormat format; - - private transient SourceFunction.SourceContext sourceContext; - - private transient ListState inputSplitsState; - private transient Queue splits; - - // Splits are read by the same thread that calls processElement. Each read task is submitted to - // that thread by adding - // them to the executor. This state is used to ensure that only one read task is in that queue at - // a time, so that read - // tasks do not accumulate ahead of checkpoint tasks. When there is a read task in the queue, this - // is set to RUNNING. - // When there are no more files to read, this will be set to IDLE. - private transient SplitState currentSplitState; - - private StreamingReaderOperator( - FlinkInputFormat format, ProcessingTimeService timeService, MailboxExecutor mailboxExecutor) { - this.format = Preconditions.checkNotNull(format, "The InputFormat should not be null."); - this.processingTimeService = timeService; - this.executor = - Preconditions.checkNotNull(mailboxExecutor, "The mailboxExecutor should not be null."); - } - - @Override - public void initializeState(StateInitializationContext context) throws Exception { - super.initializeState(context); - - // TODO Replace Java serialization with Avro approach to keep state compatibility. - // See issue: https://github.com/apache/iceberg/issues/1698 - inputSplitsState = - context - .getOperatorStateStore() - .getListState(new ListStateDescriptor<>("splits", new JavaSerializer<>())); - - // Initialize the current split state to IDLE. - currentSplitState = SplitState.IDLE; - - // Recover splits state from flink state backend if possible. - splits = Lists.newLinkedList(); - if (context.isRestored()) { - int subtaskIdx = getRuntimeContext().getIndexOfThisSubtask(); - LOG.info("Restoring state for the {} (taskIdx: {}).", getClass().getSimpleName(), subtaskIdx); - - for (FlinkInputSplit split : inputSplitsState.get()) { - splits.add(split); - } - } - - this.sourceContext = - StreamSourceContexts.getSourceContext( - getOperatorConfig().getTimeCharacteristic(), - getProcessingTimeService(), - new Object(), // no actual locking needed - output, - getRuntimeContext().getExecutionConfig().getAutoWatermarkInterval(), - -1, - true); - - // Enqueue to process the recovered input splits. - enqueueProcessSplits(); - } - - @Override - public void snapshotState(StateSnapshotContext context) throws Exception { - super.snapshotState(context); - - inputSplitsState.clear(); - inputSplitsState.addAll(Lists.newArrayList(splits)); - } - - @Override - public void processElement(StreamRecord element) { - splits.add(element.getValue()); - enqueueProcessSplits(); - } - - private void enqueueProcessSplits() { - if (currentSplitState == SplitState.IDLE && !splits.isEmpty()) { - currentSplitState = SplitState.RUNNING; - executor.execute(this::processSplits, this.getClass().getSimpleName()); - } - } - - private void processSplits() throws IOException { - FlinkInputSplit split = splits.poll(); - if (split == null) { - currentSplitState = SplitState.IDLE; - return; - } - - format.open(split); - try { - RowData nextElement = null; - while (!format.reachedEnd()) { - nextElement = format.nextRecord(nextElement); - sourceContext.collect(nextElement); - } - } finally { - currentSplitState = SplitState.IDLE; - format.close(); - } - - // Re-schedule to process the next split. - enqueueProcessSplits(); - } - - @Override - public void processWatermark(Watermark mark) { - // we do nothing because we emit our own watermarks if needed. - } - - @Override - public void close() throws Exception { - super.close(); - - if (format != null) { - format.close(); - format.closeInputFormat(); - format = null; - } - - sourceContext = null; - } - - @Override - public void finish() throws Exception { - super.finish(); - output.close(); - if (sourceContext != null) { - sourceContext.emitWatermark(Watermark.MAX_WATERMARK); - sourceContext.close(); - sourceContext = null; - } - } - - static OneInputStreamOperatorFactory factory(FlinkInputFormat format) { - return new OperatorFactory(format); - } - - private enum SplitState { - IDLE, - RUNNING - } - - private static class OperatorFactory extends AbstractStreamOperatorFactory - implements YieldingOperatorFactory, - OneInputStreamOperatorFactory { - - private final FlinkInputFormat format; - - private transient MailboxExecutor mailboxExecutor; - - private OperatorFactory(FlinkInputFormat format) { - this.format = format; - } - - @Override - public void setMailboxExecutor(MailboxExecutor mailboxExecutor) { - this.mailboxExecutor = mailboxExecutor; - } - - @SuppressWarnings("unchecked") - @Override - public > O createStreamOperator( - StreamOperatorParameters parameters) { - StreamingReaderOperator operator = - new StreamingReaderOperator(format, processingTimeService, mailboxExecutor); - operator.setup( - parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); - return (O) operator; - } - - @Override - public Class getStreamOperatorClass(ClassLoader classLoader) { - return StreamingReaderOperator.class; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java deleted file mode 100644 index 11707bf82a0f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java +++ /dev/null @@ -1,54 +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.source; - -/** Starting strategy for streaming execution. */ -public enum StreamingStartingStrategy { - /** - * Do a regular table scan then switch to the incremental mode. - * - *

    The incremental mode starts from the current snapshot exclusive. - */ - TABLE_SCAN_THEN_INCREMENTAL, - - /** - * Start incremental mode from the latest snapshot inclusive. - * - *

    If it is an empty map, all future append snapshots should be discovered. - */ - INCREMENTAL_FROM_LATEST_SNAPSHOT, - - /** - * Start incremental mode from the earliest snapshot inclusive. - * - *

    If it is an empty map, all future append snapshots should be discovered. - */ - INCREMENTAL_FROM_EARLIEST_SNAPSHOT, - - /** Start incremental mode from a snapshot with a specific id inclusive. */ - INCREMENTAL_FROM_SNAPSHOT_ID, - - /** - * Start incremental mode from a snapshot with a specific timestamp inclusive. - * - *

    If the timestamp is between two snapshots, it should start from the snapshot after the - * timestamp. - */ - INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java deleted file mode 100644 index 37a0f1a6055f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/DefaultSplitAssigner.java +++ /dev/null @@ -1,112 +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.source.assigner; - -import java.util.ArrayDeque; -import java.util.Collection; -import java.util.PriorityQueue; -import java.util.Queue; -import java.util.concurrent.CompletableFuture; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.flink.source.split.SerializableComparator; - -/** - * Since all methods are called in the source coordinator thread by enumerator, there is no need for - * locking. - */ -@Internal -public class DefaultSplitAssigner implements SplitAssigner { - - private final Queue pendingSplits; - private CompletableFuture availableFuture; - - public DefaultSplitAssigner(SerializableComparator comparator) { - this.pendingSplits = comparator == null ? new ArrayDeque<>() : new PriorityQueue<>(comparator); - } - - public DefaultSplitAssigner( - SerializableComparator comparator, - Collection assignerState) { - this(comparator); - // Because default assigner only tracks unassigned splits, - // there is no need to filter splits based on status (unassigned) here. - assignerState.forEach(splitState -> pendingSplits.add(splitState.split())); - } - - @Override - public synchronized GetSplitResult getNext(@Nullable String hostname) { - if (pendingSplits.isEmpty()) { - return GetSplitResult.unavailable(); - } else { - IcebergSourceSplit split = pendingSplits.poll(); - return GetSplitResult.forSplit(split); - } - } - - @Override - public void onDiscoveredSplits(Collection splits) { - addSplits(splits); - } - - @Override - public void onUnassignedSplits(Collection splits) { - addSplits(splits); - } - - private synchronized void addSplits(Collection splits) { - if (!splits.isEmpty()) { - pendingSplits.addAll(splits); - // only complete pending future if new splits are discovered - completeAvailableFuturesIfNeeded(); - } - } - - /** Simple assigner only tracks unassigned splits */ - @Override - public synchronized Collection state() { - return pendingSplits.stream() - .map(split -> new IcebergSourceSplitState(split, IcebergSourceSplitStatus.UNASSIGNED)) - .collect(Collectors.toList()); - } - - @Override - public synchronized CompletableFuture isAvailable() { - if (availableFuture == null) { - availableFuture = new CompletableFuture<>(); - } - return availableFuture; - } - - @Override - public synchronized int pendingSplitCount() { - return pendingSplits.size(); - } - - private synchronized void completeAvailableFuturesIfNeeded() { - if (availableFuture != null && !pendingSplits.isEmpty()) { - availableFuture.complete(null); - } - availableFuture = null; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java deleted file mode 100644 index 72deaeb890f3..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/GetSplitResult.java +++ /dev/null @@ -1,77 +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.source.assigner; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -@Internal -public class GetSplitResult { - - public enum Status { - AVAILABLE, - - /** - * There are pending splits. But they can't be assigned due to constraints (like event time - * alignment) - */ - CONSTRAINED, - - /** Assigner doesn't have pending splits. */ - UNAVAILABLE - } - - private final Status status; - private final IcebergSourceSplit split; - - private GetSplitResult(Status status) { - this.status = status; - this.split = null; - } - - private GetSplitResult(IcebergSourceSplit split) { - Preconditions.checkNotNull(split, "Split cannot be null"); - this.status = Status.AVAILABLE; - this.split = split; - } - - public Status status() { - return status; - } - - public IcebergSourceSplit split() { - return split; - } - - private static final GetSplitResult UNAVAILABLE = new GetSplitResult(Status.UNAVAILABLE); - private static final GetSplitResult CONSTRAINED = new GetSplitResult(Status.CONSTRAINED); - - public static GetSplitResult unavailable() { - return UNAVAILABLE; - } - - public static GetSplitResult constrained() { - return CONSTRAINED; - } - - public static GetSplitResult forSplit(IcebergSourceSplit split) { - return new GetSplitResult(split); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java deleted file mode 100644 index e58478897aef..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/OrderedSplitAssignerFactory.java +++ /dev/null @@ -1,46 +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.source.assigner; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.SerializableComparator; - -/** - * Create default assigner with a comparator that hands out splits where the order of the splits - * will be defined by the {@link SerializableComparator}. - */ -public class OrderedSplitAssignerFactory implements SplitAssignerFactory { - private final SerializableComparator comparator; - - public OrderedSplitAssignerFactory(SerializableComparator comparator) { - this.comparator = comparator; - } - - @Override - public SplitAssigner createAssigner() { - return new DefaultSplitAssigner(comparator); - } - - @Override - public SplitAssigner createAssigner(Collection assignerState) { - return new DefaultSplitAssigner(comparator, assignerState); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java deleted file mode 100644 index a2e2ff364d46..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SimpleSplitAssignerFactory.java +++ /dev/null @@ -1,37 +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.source.assigner; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** Create simple assigner that hands out splits without any guarantee in order or locality. */ -public class SimpleSplitAssignerFactory implements SplitAssignerFactory { - public SimpleSplitAssignerFactory() {} - - @Override - public SplitAssigner createAssigner() { - return new DefaultSplitAssigner(null); - } - - @Override - public SplitAssigner createAssigner(Collection assignerState) { - return new DefaultSplitAssigner(null, assignerState); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java deleted file mode 100644 index ca60612f0ec9..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssigner.java +++ /dev/null @@ -1,118 +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.source.assigner; - -import java.io.Closeable; -import java.util.Collection; -import java.util.concurrent.CompletableFuture; -import javax.annotation.Nullable; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** - * SplitAssigner interface is extracted out as a separate component so that we can plug in different - * split assignment strategy for different requirements. E.g. - * - *

      - *
    • Simple assigner with no ordering guarantee or locality aware optimization. - *
    • Locality aware assigner that prefer splits that are local. - *
    • Snapshot aware assigner that assign splits based on the order they are committed. - *
    • Event time alignment assigner that assign splits satisfying certain time ordering within a - * single source or across sources. - *
    - * - *

    Assigner implementation needs to be thread safe. Enumerator call the assigner APIs mostly from - * the coordinator thread. But enumerator may call the {@link SplitAssigner#pendingSplitCount()} - * from the I/O threads. - */ -public interface SplitAssigner extends Closeable { - - /** - * Some assigners may need to start background threads or perform other activity such as - * registering as listeners to updates from other event sources e.g., watermark tracker. - */ - default void start() {} - - /** - * Some assigners may need to perform certain actions when their corresponding enumerators are - * closed - */ - @Override - default void close() {} - - /** - * Request a new split from the assigner when enumerator trying to assign splits to awaiting - * readers. - * - *

    If enumerator wasn't able to assign the split (e.g., reader disconnected), enumerator should - * call {@link SplitAssigner#onUnassignedSplits} to return the split. - */ - GetSplitResult getNext(@Nullable String hostname); - - /** Add new splits discovered by enumerator */ - void onDiscoveredSplits(Collection splits); - - /** Forward addSplitsBack event (for failed reader) to assigner */ - void onUnassignedSplits(Collection splits); - - /** - * Some assigner (like event time alignment) may rack in-progress splits to advance watermark upon - * completed splits - */ - default void onCompletedSplits(Collection completedSplitIds) {} - - /** - * Get assigner state for checkpointing. This is a super-set API that works for all currently - * imagined assigners. - */ - Collection state(); - - /** - * Enumerator can get a notification via CompletableFuture when the assigner has more splits - * available later. Enumerator should schedule assignment in the thenAccept action of the future. - * - *

    Assigner will return the same future if this method is called again before the previous - * future is completed. - * - *

    The future can be completed from other thread, e.g. the coordinator thread from another - * thread for event time alignment. - * - *

    If enumerator need to trigger action upon the future completion, it may want to run it in - * the coordinator thread using {@link SplitEnumeratorContext#runInCoordinatorThread(Runnable)}. - */ - CompletableFuture isAvailable(); - - /** - * Return the number of pending splits that haven't been assigned yet. - * - *

    The enumerator can poll this API to publish a metric on the number of pending splits. - * - *

    The enumerator can also use this information to throttle split discovery for streaming read. - * If there are already many pending splits tracked by the assigner, it is undesirable to discover - * more splits and track them in the assigner. That will increase the memory footprint and - * enumerator checkpoint size. - * - *

    Throttling works better together with {@link ScanContext#maxPlanningSnapshotCount()}. - * Otherwise, the next split discovery after throttling will just discover all non-enumerated - * snapshots and splits, which defeats the purpose of throttling. - */ - int pendingSplitCount(); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java deleted file mode 100644 index 6e02a556ffcd..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerFactory.java +++ /dev/null @@ -1,30 +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.source.assigner; - -import java.io.Serializable; -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -public interface SplitAssignerFactory extends Serializable { - - SplitAssigner createAssigner(); - - SplitAssigner createAssigner(Collection assignerState); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java deleted file mode 100644 index 03ba67a554f9..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/assigner/SplitAssignerType.java +++ /dev/null @@ -1,33 +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.source.assigner; - -import org.apache.flink.annotation.Internal; - -@Internal -public enum SplitAssignerType { - SIMPLE { - @Override - public SplitAssignerFactory factory() { - return new SimpleSplitAssignerFactory(); - } - }; - - public abstract SplitAssignerFactory factory(); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java deleted file mode 100644 index 801baf77a612..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ /dev/null @@ -1,182 +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.source.enumerator; - -import java.io.IOException; -import java.util.Iterator; -import java.util.LinkedHashMap; -import java.util.List; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.flink.api.connector.source.SourceEvent; -import org.apache.flink.api.connector.source.SplitEnumerator; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.api.connector.source.SupportsHandleExecutionAttemptSourceEvent; -import org.apache.iceberg.flink.source.assigner.GetSplitResult; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * TODO: publish enumerator monitor metrics like number of pending metrics after FLINK-21000 is - * resolved - */ -abstract class AbstractIcebergEnumerator - implements SplitEnumerator, - SupportsHandleExecutionAttemptSourceEvent { - private static final Logger LOG = LoggerFactory.getLogger(AbstractIcebergEnumerator.class); - - private final SplitEnumeratorContext enumeratorContext; - private final SplitAssigner assigner; - private final Map readersAwaitingSplit; - private final AtomicReference> availableFuture; - - AbstractIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { - this.enumeratorContext = enumeratorContext; - this.assigner = assigner; - this.readersAwaitingSplit = new LinkedHashMap<>(); - this.availableFuture = new AtomicReference<>(); - } - - @Override - public void start() { - assigner.start(); - } - - @Override - public void close() throws IOException { - assigner.close(); - } - - @Override - public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname) { - // Iceberg source uses custom split request event to piggyback finished split ids. - throw new UnsupportedOperationException( - String.format( - "Received invalid default split request event " - + "from subtask %d as Iceberg source uses custom split request event", - subtaskId)); - } - - @Override - public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { - if (sourceEvent instanceof SplitRequestEvent) { - SplitRequestEvent splitRequestEvent = (SplitRequestEvent) sourceEvent; - LOG.info("Received request split event from subtask {}", subtaskId); - assigner.onCompletedSplits(splitRequestEvent.finishedSplitIds()); - readersAwaitingSplit.put(subtaskId, splitRequestEvent.requesterHostname()); - assignSplits(); - } else { - throw new IllegalArgumentException( - String.format( - "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); - } - } - - // Flink's SourceCoordinator already keeps track of subTask to splits mapping. - // It already takes care of re-assigning splits to speculated attempts as well. - @Override - public void handleSourceEvent(int subTaskId, int attemptNumber, SourceEvent sourceEvent) { - handleSourceEvent(subTaskId, sourceEvent); - } - - @Override - public void addSplitsBack(List splits, int subtaskId) { - LOG.info("Add {} splits back to the pool for failed subtask {}", splits.size(), subtaskId); - assigner.onUnassignedSplits(splits); - assignSplits(); - } - - @Override - public void addReader(int subtaskId) { - LOG.info("Added reader: {}", subtaskId); - } - - private void assignSplits() { - LOG.info("Assigning splits for {} awaiting readers", readersAwaitingSplit.size()); - Iterator> awaitingReader = - readersAwaitingSplit.entrySet().iterator(); - while (awaitingReader.hasNext()) { - Map.Entry nextAwaiting = awaitingReader.next(); - // if the reader that requested another split has failed in the meantime, remove - // it from the list of waiting readers - if (!enumeratorContext.registeredReaders().containsKey(nextAwaiting.getKey())) { - awaitingReader.remove(); - continue; - } - - int awaitingSubtask = nextAwaiting.getKey(); - String hostname = nextAwaiting.getValue(); - GetSplitResult getResult = assigner.getNext(hostname); - if (getResult.status() == GetSplitResult.Status.AVAILABLE) { - LOG.info("Assign split to subtask {}: {}", awaitingSubtask, getResult.split()); - enumeratorContext.assignSplit(getResult.split(), awaitingSubtask); - awaitingReader.remove(); - } else if (getResult.status() == GetSplitResult.Status.CONSTRAINED) { - getAvailableFutureIfNeeded(); - break; - } else if (getResult.status() == GetSplitResult.Status.UNAVAILABLE) { - if (shouldWaitForMoreSplits()) { - getAvailableFutureIfNeeded(); - break; - } else { - LOG.info("No more splits available for subtask {}", awaitingSubtask); - enumeratorContext.signalNoMoreSplits(awaitingSubtask); - awaitingReader.remove(); - } - } else { - throw new IllegalArgumentException("Unsupported status: " + getResult.status()); - } - } - } - - /** return true if enumerator should wait for splits like in the continuous enumerator case */ - protected abstract boolean shouldWaitForMoreSplits(); - - private synchronized void getAvailableFutureIfNeeded() { - if (availableFuture.get() != null) { - return; - } - - CompletableFuture future = - assigner - .isAvailable() - .thenAccept( - ignore -> - // Must run assignSplits in coordinator thread - // because the future may be completed from other threads. - // E.g., in event time alignment assigner, - // watermark advancement from another source may - // cause the available future to be completed - enumeratorContext.runInCoordinatorThread( - () -> { - LOG.debug("Executing callback of assignSplits"); - availableFuture.set(null); - assignSplits(); - })); - availableFuture.set(future); - LOG.debug("Registered callback for future available splits"); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java deleted file mode 100644 index 41863ffee60b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousEnumerationResult.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.enumerator; - -import java.util.Collection; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class ContinuousEnumerationResult { - private final Collection splits; - private final IcebergEnumeratorPosition fromPosition; - private final IcebergEnumeratorPosition toPosition; - - /** - * @param splits should never be null. But it can be an empty collection - * @param fromPosition can be null - * @param toPosition should never be null. But it can have null snapshotId and snapshotTimestampMs - */ - ContinuousEnumerationResult( - Collection splits, - IcebergEnumeratorPosition fromPosition, - IcebergEnumeratorPosition toPosition) { - Preconditions.checkArgument(splits != null, "Invalid to splits collection: null"); - Preconditions.checkArgument(toPosition != null, "Invalid end position: null"); - this.splits = splits; - this.fromPosition = fromPosition; - this.toPosition = toPosition; - } - - public Collection splits() { - return splits; - } - - public IcebergEnumeratorPosition fromPosition() { - return fromPosition; - } - - public IcebergEnumeratorPosition toPosition() { - return toPosition; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java deleted file mode 100644 index 55451b105885..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ /dev/null @@ -1,178 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collections; -import java.util.Objects; -import java.util.concurrent.atomic.AtomicReference; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { - - private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); - /** - * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to - * control the total number of snapshots worth of splits tracked by assigner. - */ - private static final int ENUMERATION_SPLIT_COUNT_HISTORY_SIZE = 3; - - private final SplitEnumeratorContext enumeratorContext; - private final SplitAssigner assigner; - private final ScanContext scanContext; - private final ContinuousSplitPlanner splitPlanner; - - /** - * snapshotId for the last enumerated snapshot. next incremental enumeration should be based off - * this as the starting position. - */ - private final AtomicReference enumeratorPosition; - - /** Track enumeration result history for split discovery throttling. */ - private final EnumerationHistory enumerationHistory; - - /** Count the consecutive failures and throw exception if the max allowed failres are reached */ - private transient int consecutiveFailures = 0; - - public ContinuousIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, - SplitAssigner assigner, - ScanContext scanContext, - ContinuousSplitPlanner splitPlanner, - @Nullable IcebergEnumeratorState enumState) { - super(enumeratorContext, assigner); - - this.enumeratorContext = enumeratorContext; - this.assigner = assigner; - this.scanContext = scanContext; - this.splitPlanner = splitPlanner; - this.enumeratorPosition = new AtomicReference<>(); - this.enumerationHistory = new EnumerationHistory(ENUMERATION_SPLIT_COUNT_HISTORY_SIZE); - - if (enumState != null) { - this.enumeratorPosition.set(enumState.lastEnumeratedPosition()); - this.enumerationHistory.restore(enumState.enumerationSplitCountHistory()); - } - } - - @Override - public void start() { - super.start(); - enumeratorContext.callAsync( - this::discoverSplits, - this::processDiscoveredSplits, - 0L, - scanContext.monitorInterval().toMillis()); - } - - @Override - public void close() throws IOException { - splitPlanner.close(); - super.close(); - } - - @Override - protected boolean shouldWaitForMoreSplits() { - return true; - } - - @Override - public IcebergEnumeratorState snapshotState(long checkpointId) { - return new IcebergEnumeratorState( - enumeratorPosition.get(), assigner.state(), enumerationHistory.snapshot()); - } - - /** This method is executed in an IO thread pool. */ - private ContinuousEnumerationResult discoverSplits() { - int pendingSplitCountFromAssigner = assigner.pendingSplitCount(); - if (enumerationHistory.shouldPauseSplitDiscovery(pendingSplitCountFromAssigner)) { - // If the assigner already has many pending splits, it is better to pause split discovery. - // Otherwise, eagerly discovering more splits will just increase assigner memory footprint - // and enumerator checkpoint state size. - LOG.info( - "Pause split discovery as the assigner already has too many pending splits: {}", - pendingSplitCountFromAssigner); - return new ContinuousEnumerationResult( - Collections.emptyList(), enumeratorPosition.get(), enumeratorPosition.get()); - } else { - return splitPlanner.planSplits(enumeratorPosition.get()); - } - } - - /** This method is executed in a single coordinator thread. */ - private void processDiscoveredSplits(ContinuousEnumerationResult result, Throwable error) { - if (error == null) { - consecutiveFailures = 0; - if (!Objects.equals(result.fromPosition(), enumeratorPosition.get())) { - // Multiple discoverSplits() may be triggered with the same starting snapshot to the I/O - // thread pool. E.g., the splitDiscoveryInterval is very short (like 10 ms in some unit - // tests) or the thread pool is busy and multiple discovery actions are executed - // concurrently. Discovery result should only be accepted if the starting position - // matches the enumerator position (like compare-and-swap). - LOG.info( - "Skip {} discovered splits because the scan starting position doesn't match " - + "the current enumerator position: enumerator position = {}, scan starting position = {}", - result.splits().size(), - enumeratorPosition.get(), - result.fromPosition()); - } else { - // Sometimes, enumeration may yield no splits for a few reasons. - // - upstream paused or delayed streaming writes to the Iceberg table. - // - enumeration frequency is higher than the upstream write frequency. - if (!result.splits().isEmpty()) { - assigner.onDiscoveredSplits(result.splits()); - // EnumerationHistory makes throttling decision on split discovery - // based on the total number of splits discovered in the last a few cycles. - // Only update enumeration history when there are some discovered splits. - enumerationHistory.add(result.splits().size()); - LOG.info( - "Added {} splits discovered between ({}, {}] to the assigner", - result.splits().size(), - result.fromPosition(), - result.toPosition()); - } else { - LOG.info( - "No new splits discovered between ({}, {}]", - result.fromPosition(), - result.toPosition()); - } - // update the enumerator position even if there is no split discovered - // or the toPosition is empty (e.g. for empty table). - enumeratorPosition.set(result.toPosition()); - LOG.info("Update enumerator position to {}", result.toPosition()); - } - } else { - consecutiveFailures++; - if (scanContext.maxAllowedPlanningFailures() < 0 - || consecutiveFailures <= scanContext.maxAllowedPlanningFailures()) { - LOG.error("Failed to discover new splits", error); - } else { - throw new RuntimeException("Failed to discover new splits", error); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java deleted file mode 100644 index 2a1325178873..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlanner.java +++ /dev/null @@ -1,30 +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.source.enumerator; - -import java.io.Closeable; -import org.apache.flink.annotation.Internal; - -/** This interface is introduced so that we can plug in different split planner for unit test */ -@Internal -public interface ContinuousSplitPlanner extends Closeable { - - /** Discover the files appended between {@code lastPosition} and current table snapshot */ - ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java deleted file mode 100644 index fef4ec45ed8a..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ /dev/null @@ -1,240 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ExecutorService; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.util.Preconditions; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.FlinkSplitPlanner; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -@Internal -public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner { - private static final Logger LOG = LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class); - - private final Table table; - private final ScanContext scanContext; - private final boolean isSharedPool; - private final ExecutorService workerPool; - private final TableLoader tableLoader; - - /** - * @param tableLoader A cloned tableLoader. - * @param threadName thread name prefix for worker pool to run the split planning. If null, a - * shared worker pool will be used. - */ - public ContinuousSplitPlannerImpl( - TableLoader tableLoader, ScanContext scanContext, String threadName) { - this.tableLoader = tableLoader.clone(); - this.tableLoader.open(); - this.table = this.tableLoader.loadTable(); - this.scanContext = scanContext; - this.isSharedPool = threadName == null; - this.workerPool = - isSharedPool - ? ThreadPools.getWorkerPool() - : ThreadPools.newWorkerPool( - "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); - } - - @Override - public void close() throws IOException { - if (!isSharedPool) { - workerPool.shutdown(); - } - tableLoader.close(); - } - - @Override - public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition lastPosition) { - table.refresh(); - if (lastPosition != null) { - return discoverIncrementalSplits(lastPosition); - } else { - return discoverInitialSplits(); - } - } - - private Snapshot toSnapshotInclusive( - Long lastConsumedSnapshotId, Snapshot currentSnapshot, int maxPlanningSnapshotCount) { - // snapshots are in reverse order (latest snapshot first) - List snapshots = - Lists.newArrayList( - SnapshotUtil.ancestorsBetween( - table, currentSnapshot.snapshotId(), lastConsumedSnapshotId)); - if (snapshots.size() <= maxPlanningSnapshotCount) { - return currentSnapshot; - } else { - // Because snapshots are in reverse order of commit history, this index returns - // the max allowed number of snapshots from the lastConsumedSnapshotId. - return snapshots.get(snapshots.size() - maxPlanningSnapshotCount); - } - } - - private ContinuousEnumerationResult discoverIncrementalSplits( - IcebergEnumeratorPosition lastPosition) { - Snapshot currentSnapshot = - scanContext.branch() != null - ? table.snapshot(scanContext.branch()) - : table.currentSnapshot(); - - if (currentSnapshot == null) { - // empty table - Preconditions.checkArgument( - lastPosition.snapshotId() == null, - "Invalid last enumerated position for an empty table: not null"); - LOG.info("Skip incremental scan because table is empty"); - return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); - } else if (lastPosition.snapshotId() != null - && currentSnapshot.snapshotId() == lastPosition.snapshotId()) { - LOG.info("Current table snapshot is already enumerated: {}", currentSnapshot.snapshotId()); - return new ContinuousEnumerationResult(Collections.emptyList(), lastPosition, lastPosition); - } else { - Long lastConsumedSnapshotId = lastPosition.snapshotId(); - Snapshot toSnapshotInclusive = - toSnapshotInclusive( - lastConsumedSnapshotId, currentSnapshot, scanContext.maxPlanningSnapshotCount()); - IcebergEnumeratorPosition newPosition = - IcebergEnumeratorPosition.of( - toSnapshotInclusive.snapshotId(), toSnapshotInclusive.timestampMillis()); - ScanContext incrementalScan = - scanContext.copyWithAppendsBetween( - lastPosition.snapshotId(), toSnapshotInclusive.snapshotId()); - List splits = - FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool); - LOG.info( - "Discovered {} splits from incremental scan: " - + "from snapshot (exclusive) is {}, to snapshot (inclusive) is {}", - splits.size(), - lastPosition, - newPosition); - return new ContinuousEnumerationResult(splits, lastPosition, newPosition); - } - } - - /** - * Discovery initial set of splits based on {@link StreamingStartingStrategy}. - *

  • {@link ContinuousEnumerationResult#splits()} should contain initial splits discovered from - * table scan for {@link StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}. For all other - * strategies, splits collection should be empty. - *
  • {@link ContinuousEnumerationResult#toPosition()} points to the starting position for the - * next incremental split discovery with exclusive behavior. Meaning files committed by the - * snapshot from the position in {@code ContinuousEnumerationResult} won't be included in the - * next incremental scan. - */ - private ContinuousEnumerationResult discoverInitialSplits() { - Optional startSnapshotOptional = startSnapshot(table, scanContext); - if (!startSnapshotOptional.isPresent()) { - return new ContinuousEnumerationResult( - Collections.emptyList(), null, IcebergEnumeratorPosition.empty()); - } - - Snapshot startSnapshot = startSnapshotOptional.get(); - LOG.info( - "Get starting snapshot id {} based on strategy {}", - startSnapshot.snapshotId(), - scanContext.streamingStartingStrategy()); - List splits; - IcebergEnumeratorPosition toPosition; - if (scanContext.streamingStartingStrategy() - == StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) { - // do a batch table scan first - splits = - FlinkSplitPlanner.planIcebergSourceSplits( - table, scanContext.copyWithSnapshotId(startSnapshot.snapshotId()), workerPool); - LOG.info( - "Discovered {} splits from initial batch table scan with snapshot Id {}", - splits.size(), - startSnapshot.snapshotId()); - // For TABLE_SCAN_THEN_INCREMENTAL, incremental mode starts exclusive from the startSnapshot - toPosition = - IcebergEnumeratorPosition.of(startSnapshot.snapshotId(), startSnapshot.timestampMillis()); - } else { - // For all other modes, starting snapshot should be consumed inclusively. - // Use parentId to achieve the inclusive behavior. It is fine if parentId is null. - splits = Collections.emptyList(); - Long parentSnapshotId = startSnapshot.parentId(); - if (parentSnapshotId != null) { - Snapshot parentSnapshot = table.snapshot(parentSnapshotId); - Long parentSnapshotTimestampMs = - parentSnapshot != null ? parentSnapshot.timestampMillis() : null; - toPosition = IcebergEnumeratorPosition.of(parentSnapshotId, parentSnapshotTimestampMs); - } else { - toPosition = IcebergEnumeratorPosition.empty(); - } - - LOG.info( - "Start incremental scan with start snapshot (inclusive): id = {}, timestamp = {}", - startSnapshot.snapshotId(), - startSnapshot.timestampMillis()); - } - - return new ContinuousEnumerationResult(splits, null, toPosition); - } - - /** - * Calculate the starting snapshot based on the {@link StreamingStartingStrategy} defined in - * {@code ScanContext}. - * - *

    If the {@link StreamingStartingStrategy} is not {@link - * StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}, the start snapshot should be consumed - * inclusively. - */ - @VisibleForTesting - static Optional startSnapshot(Table table, ScanContext scanContext) { - switch (scanContext.streamingStartingStrategy()) { - case TABLE_SCAN_THEN_INCREMENTAL: - case INCREMENTAL_FROM_LATEST_SNAPSHOT: - return Optional.ofNullable(table.currentSnapshot()); - case INCREMENTAL_FROM_EARLIEST_SNAPSHOT: - return Optional.ofNullable(SnapshotUtil.oldestAncestor(table)); - case INCREMENTAL_FROM_SNAPSHOT_ID: - Snapshot matchedSnapshotById = table.snapshot(scanContext.startSnapshotId()); - Preconditions.checkArgument( - matchedSnapshotById != null, - "Start snapshot id not found in history: " + scanContext.startSnapshotId()); - return Optional.of(matchedSnapshotById); - case INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP: - Snapshot matchedSnapshotByTimestamp = - SnapshotUtil.oldestAncestorAfter(table, scanContext.startSnapshotTimestamp()); - Preconditions.checkArgument( - matchedSnapshotByTimestamp != null, - "Cannot find a snapshot after: " + scanContext.startSnapshotTimestamp()); - return Optional.of(matchedSnapshotByTimestamp); - default: - throw new IllegalArgumentException( - "Unknown starting strategy: " + scanContext.streamingStartingStrategy()); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java deleted file mode 100644 index ec56a9ecdac1..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/EnumerationHistory.java +++ /dev/null @@ -1,100 +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.source.enumerator; - -import java.util.Arrays; -import javax.annotation.concurrent.ThreadSafe; -import org.apache.flink.annotation.VisibleForTesting; - -/** - * This enumeration history is used for split discovery throttling. It tracks the discovered split - * count per every non-empty enumeration. - */ -@ThreadSafe -class EnumerationHistory { - - private final int[] history; - // int (2B) should be enough without overflow for enumeration history - private int count; - - EnumerationHistory(int maxHistorySize) { - this.history = new int[maxHistorySize]; - } - - synchronized void restore(int[] restoredHistory) { - int startingOffset = 0; - int restoreSize = restoredHistory.length; - - if (restoredHistory.length > history.length) { - // keep the newest history - startingOffset = restoredHistory.length - history.length; - // only restore the latest history up to maxHistorySize - restoreSize = history.length; - } - - System.arraycopy(restoredHistory, startingOffset, history, 0, restoreSize); - count = restoreSize; - } - - synchronized int[] snapshot() { - int len = history.length; - if (count > len) { - int[] copy = new int[len]; - // this is like a circular buffer - int indexForOldest = count % len; - System.arraycopy(history, indexForOldest, copy, 0, len - indexForOldest); - System.arraycopy(history, 0, copy, len - indexForOldest, indexForOldest); - return copy; - } else { - return Arrays.copyOfRange(history, 0, count); - } - } - - /** Add the split count from the last enumeration result. */ - synchronized void add(int splitCount) { - int pos = count % history.length; - history[pos] = splitCount; - count += 1; - } - - @VisibleForTesting - synchronized boolean hasFullHistory() { - return count >= history.length; - } - - /** - * Checks whether split discovery should be paused. - * - * @return true if split discovery should pause because assigner has too many splits already. - */ - synchronized boolean shouldPauseSplitDiscovery(int pendingSplitCountFromAssigner) { - if (count < history.length) { - // only check throttling when full history is obtained. - return false; - } else { - // if ScanContext#maxPlanningSnapshotCount() is 10, each split enumeration can - // discovery splits up to 10 snapshots. if maxHistorySize is 3, the max number of - // splits tracked in assigner shouldn't be more than 10 * (3 + 1) snapshots - // worth of splits. +1 because there could be another enumeration when the - // pending splits fall just below the 10 * 3. - int totalSplitCountFromRecentDiscovery = Arrays.stream(history).reduce(0, Integer::sum); - return pendingSplitCountFromAssigner >= totalSplitCountFromRecentDiscovery; - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java deleted file mode 100644 index 96aba296f8cf..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPosition.java +++ /dev/null @@ -1,79 +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.source.enumerator; - -import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -import org.apache.iceberg.relocated.com.google.common.base.Objects; - -class IcebergEnumeratorPosition { - private final Long snapshotId; - // Track snapshot timestamp mainly for info logging - private final Long snapshotTimestampMs; - - static IcebergEnumeratorPosition empty() { - return new IcebergEnumeratorPosition(null, null); - } - - static IcebergEnumeratorPosition of(long snapshotId, Long snapshotTimestampMs) { - return new IcebergEnumeratorPosition(snapshotId, snapshotTimestampMs); - } - - private IcebergEnumeratorPosition(Long snapshotId, Long snapshotTimestampMs) { - this.snapshotId = snapshotId; - this.snapshotTimestampMs = snapshotTimestampMs; - } - - boolean isEmpty() { - return snapshotId == null; - } - - Long snapshotId() { - return snapshotId; - } - - Long snapshotTimestampMs() { - return snapshotTimestampMs; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("snapshotId", snapshotId) - .add("snapshotTimestampMs", snapshotTimestampMs) - .toString(); - } - - @Override - public int hashCode() { - return Objects.hashCode(snapshotId, snapshotTimestampMs); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - IcebergEnumeratorPosition other = (IcebergEnumeratorPosition) o; - return Objects.equal(snapshotId, other.snapshotId()) - && Objects.equal(snapshotTimestampMs, other.snapshotTimestampMs()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java deleted file mode 100644 index 1c63807361c5..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorPositionSerializer.java +++ /dev/null @@ -1,90 +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.source.enumerator; - -import java.io.IOException; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -class IcebergEnumeratorPositionSerializer - implements SimpleVersionedSerializer { - - public static final IcebergEnumeratorPositionSerializer INSTANCE = - new IcebergEnumeratorPositionSerializer(); - - private static final int VERSION = 1; - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(128)); - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergEnumeratorPosition position) throws IOException { - return serializeV1(position); - } - - @Override - public IcebergEnumeratorPosition deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return deserializeV1(serialized); - default: - throw new IOException("Unknown version: " + version); - } - } - - private byte[] serializeV1(IcebergEnumeratorPosition position) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - out.writeBoolean(position.snapshotId() != null); - if (position.snapshotId() != null) { - out.writeLong(position.snapshotId()); - } - out.writeBoolean(position.snapshotTimestampMs() != null); - if (position.snapshotTimestampMs() != null) { - out.writeLong(position.snapshotTimestampMs()); - } - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - private IcebergEnumeratorPosition deserializeV1(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - Long snapshotId = null; - if (in.readBoolean()) { - snapshotId = in.readLong(); - } - - Long snapshotTimestampMs = null; - if (in.readBoolean()) { - snapshotTimestampMs = in.readLong(); - } - - if (snapshotId != null) { - return IcebergEnumeratorPosition.of(snapshotId, snapshotTimestampMs); - } else { - return IcebergEnumeratorPosition.empty(); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java deleted file mode 100644 index 26fbad46c128..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorState.java +++ /dev/null @@ -1,65 +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.source.enumerator; - -import java.io.Serializable; -import java.util.Collection; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; - -/** Enumerator state for checkpointing */ -@Internal -public class IcebergEnumeratorState implements Serializable { - @Nullable private final IcebergEnumeratorPosition lastEnumeratedPosition; - private final Collection pendingSplits; - private final int[] enumerationSplitCountHistory; - - public IcebergEnumeratorState(Collection pendingSplits) { - this(null, pendingSplits); - } - - public IcebergEnumeratorState( - @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, - Collection pendingSplits) { - this(lastEnumeratedPosition, pendingSplits, new int[0]); - } - - public IcebergEnumeratorState( - @Nullable IcebergEnumeratorPosition lastEnumeratedPosition, - Collection pendingSplits, - int[] enumerationSplitCountHistory) { - this.lastEnumeratedPosition = lastEnumeratedPosition; - this.pendingSplits = pendingSplits; - this.enumerationSplitCountHistory = enumerationSplitCountHistory; - } - - @Nullable - public IcebergEnumeratorPosition lastEnumeratedPosition() { - return lastEnumeratedPosition; - } - - public Collection pendingSplits() { - return pendingSplits; - } - - public int[] enumerationSplitCountHistory() { - return enumerationSplitCountHistory; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java deleted file mode 100644 index f76f8a69ff0e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/IcebergEnumeratorStateSerializer.java +++ /dev/null @@ -1,194 +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.source.enumerator; - -import java.io.IOException; -import java.util.Collection; -import org.apache.flink.annotation.Internal; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergEnumeratorStateSerializer - implements SimpleVersionedSerializer { - - private static final int VERSION = 2; - - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); - - private final IcebergEnumeratorPositionSerializer positionSerializer = - IcebergEnumeratorPositionSerializer.INSTANCE; - private final IcebergSourceSplitSerializer splitSerializer; - - public IcebergEnumeratorStateSerializer(boolean caseSensitive) { - this.splitSerializer = new IcebergSourceSplitSerializer(caseSensitive); - } - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergEnumeratorState enumState) throws IOException { - return serializeV2(enumState); - } - - @Override - public IcebergEnumeratorState deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return deserializeV1(serialized); - case 2: - return deserializeV2(serialized); - default: - throw new IOException("Unknown version: " + version); - } - } - - @VisibleForTesting - byte[] serializeV1(IcebergEnumeratorState enumState) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); - serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - @VisibleForTesting - IcebergEnumeratorState deserializeV1(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - IcebergEnumeratorPosition enumeratorPosition = - deserializeEnumeratorPosition(in, positionSerializer); - Collection pendingSplits = - deserializePendingSplits(in, splitSerializer); - return new IcebergEnumeratorState(enumeratorPosition, pendingSplits); - } - - @VisibleForTesting - byte[] serializeV2(IcebergEnumeratorState enumState) throws IOException { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - serializeEnumeratorPosition(out, enumState.lastEnumeratedPosition(), positionSerializer); - serializePendingSplits(out, enumState.pendingSplits(), splitSerializer); - serializeEnumerationSplitCountHistory(out, enumState.enumerationSplitCountHistory()); - byte[] result = out.getCopyOfBuffer(); - out.clear(); - return result; - } - - @VisibleForTesting - IcebergEnumeratorState deserializeV2(byte[] serialized) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - IcebergEnumeratorPosition enumeratorPosition = - deserializeEnumeratorPosition(in, positionSerializer); - Collection pendingSplits = - deserializePendingSplits(in, splitSerializer); - int[] enumerationSplitCountHistory = deserializeEnumerationSplitCountHistory(in); - return new IcebergEnumeratorState( - enumeratorPosition, pendingSplits, enumerationSplitCountHistory); - } - - private static void serializeEnumeratorPosition( - DataOutputSerializer out, - IcebergEnumeratorPosition enumeratorPosition, - IcebergEnumeratorPositionSerializer positionSerializer) - throws IOException { - out.writeBoolean(enumeratorPosition != null); - if (enumeratorPosition != null) { - out.writeInt(positionSerializer.getVersion()); - byte[] positionBytes = positionSerializer.serialize(enumeratorPosition); - out.writeInt(positionBytes.length); - out.write(positionBytes); - } - } - - private static IcebergEnumeratorPosition deserializeEnumeratorPosition( - DataInputDeserializer in, IcebergEnumeratorPositionSerializer positionSerializer) - throws IOException { - IcebergEnumeratorPosition enumeratorPosition = null; - if (in.readBoolean()) { - int version = in.readInt(); - byte[] positionBytes = new byte[in.readInt()]; - in.read(positionBytes); - enumeratorPosition = positionSerializer.deserialize(version, positionBytes); - } - return enumeratorPosition; - } - - private static void serializePendingSplits( - DataOutputSerializer out, - Collection pendingSplits, - IcebergSourceSplitSerializer splitSerializer) - throws IOException { - out.writeInt(splitSerializer.getVersion()); - out.writeInt(pendingSplits.size()); - for (IcebergSourceSplitState splitState : pendingSplits) { - byte[] splitBytes = splitSerializer.serialize(splitState.split()); - out.writeInt(splitBytes.length); - out.write(splitBytes); - out.writeUTF(splitState.status().name()); - } - } - - private static Collection deserializePendingSplits( - DataInputDeserializer in, IcebergSourceSplitSerializer splitSerializer) throws IOException { - int splitSerializerVersion = in.readInt(); - int splitCount = in.readInt(); - Collection pendingSplits = Lists.newArrayListWithCapacity(splitCount); - for (int i = 0; i < splitCount; ++i) { - byte[] splitBytes = new byte[in.readInt()]; - in.read(splitBytes); - IcebergSourceSplit split = splitSerializer.deserialize(splitSerializerVersion, splitBytes); - String statusName = in.readUTF(); - pendingSplits.add( - new IcebergSourceSplitState(split, IcebergSourceSplitStatus.valueOf(statusName))); - } - return pendingSplits; - } - - private static void serializeEnumerationSplitCountHistory( - DataOutputSerializer out, int[] enumerationSplitCountHistory) throws IOException { - out.writeInt(enumerationSplitCountHistory.length); - for (int enumerationSplitCount : enumerationSplitCountHistory) { - out.writeInt(enumerationSplitCount); - } - } - - private static int[] deserializeEnumerationSplitCountHistory(DataInputDeserializer in) - throws IOException { - int historySize = in.readInt(); - int[] history = new int[historySize]; - if (historySize > 0) { - for (int i = 0; i < historySize; ++i) { - history[i] = in.readInt(); - } - } - - return history; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java deleted file mode 100644 index 4e55ea5d5fd6..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/StaticIcebergEnumerator.java +++ /dev/null @@ -1,51 +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.source.enumerator; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.iceberg.flink.source.assigner.SplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** One-time split enumeration at the start-up for batch execution */ -@Internal -public class StaticIcebergEnumerator extends AbstractIcebergEnumerator { - private final SplitAssigner assigner; - - public StaticIcebergEnumerator( - SplitEnumeratorContext enumeratorContext, SplitAssigner assigner) { - super(enumeratorContext, assigner); - this.assigner = assigner; - } - - @Override - public void start() { - super.start(); - } - - @Override - protected boolean shouldWaitForMoreSplits() { - return false; - } - - @Override - public IcebergEnumeratorState snapshotState(long checkpointId) { - return new IcebergEnumeratorState(null, assigner.state(), new int[0]); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.java deleted file mode 100644 index 7b94c364c976..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayBatchRecords.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.flink.source.reader; - -import java.util.Collections; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.flink.annotation.VisibleForTesting; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.file.src.util.Pool; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * {@link RecordsWithSplitIds} is used to pass a batch of records from fetcher to source reader. - * Batching is to improve the efficiency for records handover. - * - *

    {@link RecordsWithSplitIds} interface can encapsulate batches from multiple splits. This is - * the case for Kafka source where fetchers can retrieve records from multiple Kafka partitions at - * the same time. - * - *

    For file-based sources like Iceberg, readers always read one split/file at a time. Hence, we - * will only have a batch of records for one split here. - * - *

    This class uses array to store a batch of records from the same file (with the same - * fileOffset). - */ -class ArrayBatchRecords implements RecordsWithSplitIds> { - @Nullable private String splitId; - @Nullable private final Pool.Recycler recycler; - @Nullable private final T[] records; - private final int numberOfRecords; - private final Set finishedSplits; - private final RecordAndPosition recordAndPosition; - - // point to current read position within the records array - private int position; - - private ArrayBatchRecords( - @Nullable String splitId, - @Nullable Pool.Recycler recycler, - @Nullable T[] records, - int numberOfRecords, - int fileOffset, - long startingRecordOffset, - Set finishedSplits) { - Preconditions.checkArgument(numberOfRecords >= 0, "numberOfRecords can't be negative"); - Preconditions.checkArgument(fileOffset >= 0, "fileOffset can't be negative"); - Preconditions.checkArgument(startingRecordOffset >= 0, "numberOfRecords can't be negative"); - - this.splitId = splitId; - this.recycler = recycler; - this.records = records; - this.numberOfRecords = numberOfRecords; - this.finishedSplits = - Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); - this.recordAndPosition = new RecordAndPosition<>(); - - recordAndPosition.set(null, fileOffset, startingRecordOffset); - this.position = 0; - } - - @Nullable - @Override - public String nextSplit() { - String nextSplit = this.splitId; - // set the splitId to null to indicate no more splits - // this class only contains record for one split - this.splitId = null; - return nextSplit; - } - - @Nullable - @Override - public RecordAndPosition nextRecordFromSplit() { - if (position < numberOfRecords) { - recordAndPosition.record(records[position]); - position++; - return recordAndPosition; - } else { - return null; - } - } - - /** - * This method is called when all records from this batch has been emitted. If recycler is set, it - * should be called to return the records array back to pool. - */ - @Override - public void recycle() { - if (recycler != null) { - recycler.recycle(records); - } - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - - @VisibleForTesting - T[] records() { - return records; - } - - @VisibleForTesting - int numberOfRecords() { - return numberOfRecords; - } - - /** - * Create a ArrayBatchRecords backed up an array with records from the same file - * - * @param splitId Iceberg source only read from one split a time. We never have multiple records - * from multiple splits. - * @param recycler Because {@link DataIterator} with {@link RowData} returns an iterator of reused - * RowData object, we need to clone RowData eagerly when constructing a batch of records. We - * can use object pool to reuse the RowData array object which can be expensive to create. - * This recycler can be provided to recycle the array object back to pool after read is - * exhausted. If the {@link DataIterator} returns an iterator of non-reused objects, we don't - * need to clone objects. It is cheap to just create the batch array. Hence, we don't need - * object pool and recycler can be set to null. - * @param records an array (maybe reused) holding a batch of records - * @param numberOfRecords actual number of records in the array - * @param fileOffset fileOffset for all records in this batch - * @param startingRecordOffset starting recordOffset - * @param record type - */ - public static ArrayBatchRecords forRecords( - String splitId, - Pool.Recycler recycler, - T[] records, - int numberOfRecords, - int fileOffset, - long startingRecordOffset) { - return new ArrayBatchRecords<>( - splitId, - recycler, - records, - numberOfRecords, - fileOffset, - startingRecordOffset, - Collections.emptySet()); - } - - /** - * Create ab ArrayBatchRecords with only finished split id - * - * @param splitId for the split that is just exhausted - */ - public static ArrayBatchRecords finishedSplit(String splitId) { - return new ArrayBatchRecords<>(null, null, null, 0, 0, 0, Collections.singleton(splitId)); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java deleted file mode 100644 index 306afd1811be..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ArrayPoolDataIteratorBatcher.java +++ /dev/null @@ -1,130 +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.source.reader; - -import java.io.IOException; -import java.util.NoSuchElementException; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SourceReaderOptions; -import org.apache.flink.connector.file.src.util.Pool; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** This implementation stores record batch in array from recyclable pool */ -class ArrayPoolDataIteratorBatcher implements DataIteratorBatcher { - private final int batchSize; - private final int handoverQueueSize; - private final RecordFactory recordFactory; - - private transient Pool pool; - - ArrayPoolDataIteratorBatcher(ReadableConfig config, RecordFactory recordFactory) { - this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); - this.handoverQueueSize = config.get(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY); - this.recordFactory = recordFactory; - } - - @Override - public CloseableIterator>> batch( - String splitId, DataIterator inputIterator) { - Preconditions.checkArgument(inputIterator != null, "Input data iterator can't be null"); - // lazily create pool as it is not serializable - if (pool == null) { - this.pool = createPoolOfBatches(handoverQueueSize); - } - return new ArrayPoolBatchIterator(splitId, inputIterator, pool); - } - - private Pool createPoolOfBatches(int numBatches) { - Pool poolOfBatches = new Pool<>(numBatches); - for (int batchId = 0; batchId < numBatches; batchId++) { - T[] batch = recordFactory.createBatch(batchSize); - poolOfBatches.add(batch); - } - - return poolOfBatches; - } - - private class ArrayPoolBatchIterator - implements CloseableIterator>> { - - private final String splitId; - private final DataIterator inputIterator; - private final Pool pool; - - ArrayPoolBatchIterator(String splitId, DataIterator inputIterator, Pool pool) { - this.splitId = splitId; - this.inputIterator = inputIterator; - this.pool = pool; - } - - @Override - public boolean hasNext() { - return inputIterator.hasNext(); - } - - @Override - public RecordsWithSplitIds> next() { - if (!inputIterator.hasNext()) { - throw new NoSuchElementException(); - } - - T[] batch = getCachedEntry(); - int recordCount = 0; - while (inputIterator.hasNext() && recordCount < batchSize) { - // The record produced by inputIterator can be reused like for the RowData case. - // inputIterator.next() can't be called again until the copy is made - // since the record is not consumed immediately. - T nextRecord = inputIterator.next(); - recordFactory.clone(nextRecord, batch, recordCount); - recordCount++; - if (!inputIterator.currentFileHasNext()) { - // break early so that records in the ArrayResultIterator - // have the same fileOffset. - break; - } - } - - return ArrayBatchRecords.forRecords( - splitId, - pool.recycler(), - batch, - recordCount, - inputIterator.fileOffset(), - inputIterator.recordOffset() - recordCount); - } - - @Override - public void close() throws IOException { - inputIterator.close(); - } - - private T[] getCachedEntry() { - try { - return pool.pollEntry(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException("Interrupted while waiting for array pool entry", e); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java deleted file mode 100644 index 66e59633fff2..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ /dev/null @@ -1,102 +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.source.reader; - -import java.util.List; -import org.apache.avro.generic.GenericRecord; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Read Iceberg rows as {@link GenericRecord}. */ -public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { - private final String tableName; - private final Schema readSchema; - private final FileIO io; - private final EncryptionManager encryption; - private final RowDataFileScanTaskReader rowDataReader; - - private transient RowDataToAvroGenericRecordConverter converter; - - /** - * Create a reader function without projection and name mapping. Column name is case-insensitive. - */ - public static AvroGenericRecordReaderFunction fromTable(Table table) { - return new AvroGenericRecordReaderFunction( - table.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - null); - } - - public AvroGenericRecordReaderFunction( - String tableName, - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters) { - super(new ListDataIteratorBatcher<>(config)); - this.tableName = tableName; - this.readSchema = readSchema(tableSchema, projectedSchema); - this.io = io; - this.encryption = encryption; - this.rowDataReader = - new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); - } - - @Override - protected DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>( - new AvroGenericRecordFileScanTaskReader(rowDataReader, lazyConverter()), - split.task(), - io, - encryption); - } - - private RowDataToAvroGenericRecordConverter lazyConverter() { - if (converter == null) { - this.converter = RowDataToAvroGenericRecordConverter.fromIcebergSchema(tableName, readSchema); - } - return converter; - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java deleted file mode 100644 index 4bb6f0a98c4c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ColumnStatsWatermarkExtractor.java +++ /dev/null @@ -1,98 +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.source.reader; - -import java.io.Serializable; -import java.util.Comparator; -import java.util.concurrent.TimeUnit; -import org.apache.flink.annotation.Internal; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Type.TypeID; -import org.apache.iceberg.types.Types; - -/** - * {@link SplitWatermarkExtractor} implementation which uses an Iceberg timestamp column statistics - * to get the watermarks for the {@link IcebergSourceSplit}. This watermark is emitted by the {@link - * WatermarkExtractorRecordEmitter} along with the actual records. - */ -@Internal -public class ColumnStatsWatermarkExtractor implements SplitWatermarkExtractor, Serializable { - private final int eventTimeFieldId; - private final String eventTimeFieldName; - private final TimeUnit timeUnit; - - /** - * Creates the extractor. - * - * @param schema The schema of the Table - * @param eventTimeFieldName The column which should be used as an event time - * @param timeUnit Used for converting the long value to epoch milliseconds - */ - public ColumnStatsWatermarkExtractor( - Schema schema, String eventTimeFieldName, TimeUnit timeUnit) { - Types.NestedField field = schema.findField(eventTimeFieldName); - TypeID typeID = field.type().typeId(); - Preconditions.checkArgument( - typeID.equals(TypeID.LONG) || typeID.equals(TypeID.TIMESTAMP), - "Found %s, expected a LONG or TIMESTAMP column for watermark generation.", - typeID); - this.eventTimeFieldId = field.fieldId(); - this.eventTimeFieldName = eventTimeFieldName; - // Use the timeUnit only for Long columns. - this.timeUnit = typeID.equals(TypeID.LONG) ? timeUnit : TimeUnit.MICROSECONDS; - } - - @VisibleForTesting - ColumnStatsWatermarkExtractor(int eventTimeFieldId, String eventTimeFieldName) { - this.eventTimeFieldId = eventTimeFieldId; - this.eventTimeFieldName = eventTimeFieldName; - this.timeUnit = TimeUnit.MICROSECONDS; - } - - /** - * Get the watermark for a split using column statistics. - * - * @param split The split - * @return The watermark - * @throws IllegalArgumentException if there is no statistics for the column - */ - @Override - public long extractWatermark(IcebergSourceSplit split) { - return split.task().files().stream() - .map( - scanTask -> { - Preconditions.checkArgument( - scanTask.file().lowerBounds() != null - && scanTask.file().lowerBounds().get(eventTimeFieldId) != null, - "Missing statistics for column name = %s in file = %s", - eventTimeFieldName, - eventTimeFieldId, - scanTask.file()); - return timeUnit.toMillis( - Conversions.fromByteBuffer( - Types.LongType.get(), scanTask.file().lowerBounds().get(eventTimeFieldId))); - }) - .min(Comparator.comparingLong(l -> l)) - .get(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java deleted file mode 100644 index c376e359c600..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorBatcher.java +++ /dev/null @@ -1,36 +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.source.reader; - -import java.io.Serializable; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; - -/** - * Batcher converts iterator of T into iterator of batched {@code - * RecordsWithSplitIds>}, as FLIP-27's {@link SplitReader#fetch()} returns - * batched records. - */ -@FunctionalInterface -public interface DataIteratorBatcher extends Serializable { - CloseableIterator>> batch( - String splitId, DataIterator inputIterator); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.java deleted file mode 100644 index bbf797ef4aa8..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/DataIteratorReaderFunction.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.flink.source.reader; - -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; - -/** A {@link ReaderFunction} implementation that uses {@link DataIterator}. */ -public abstract class DataIteratorReaderFunction implements ReaderFunction { - private final DataIteratorBatcher batcher; - - public DataIteratorReaderFunction(DataIteratorBatcher batcher) { - this.batcher = batcher; - } - - protected abstract DataIterator createDataIterator(IcebergSourceSplit split); - - @Override - public CloseableIterator>> apply( - IcebergSourceSplit split) { - DataIterator inputIterator = createDataIterator(split); - inputIterator.seek(split.fileOffset(), split.recordOffset()); - return batcher.batch(split.splitId(), inputIterator); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java deleted file mode 100644 index f143b8d2df2e..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReader.java +++ /dev/null @@ -1,77 +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.source.reader; - -import java.util.Collection; -import java.util.Collections; -import java.util.Map; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.SingleThreadMultiplexSourceReaderBase; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergSourceReader - extends SingleThreadMultiplexSourceReaderBase< - RecordAndPosition, T, IcebergSourceSplit, IcebergSourceSplit> { - - public IcebergSourceReader( - SerializableRecordEmitter emitter, - IcebergSourceReaderMetrics metrics, - ReaderFunction readerFunction, - SerializableComparator splitComparator, - SourceReaderContext context) { - super( - () -> new IcebergSourceSplitReader<>(metrics, readerFunction, splitComparator, context), - emitter, - context.getConfiguration(), - context); - } - - @Override - public void start() { - // We request a split only if we did not get splits during the checkpoint restore. - // Otherwise, reader restarts will keep requesting more and more splits. - if (getNumberOfCurrentlyAssignedSplits() == 0) { - requestSplit(Collections.emptyList()); - } - } - - @Override - protected void onSplitFinished(Map finishedSplitIds) { - requestSplit(Lists.newArrayList(finishedSplitIds.keySet())); - } - - @Override - protected IcebergSourceSplit initializedState(IcebergSourceSplit split) { - return split; - } - - @Override - protected IcebergSourceSplit toSplitType(String splitId, IcebergSourceSplit splitState) { - return splitState; - } - - private void requestSplit(Collection finishedSplitIds) { - context.sendSourceEventToCoordinator(new SplitRequestEvent(finishedSplitIds)); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java deleted file mode 100644 index 2a3e1dd86b95..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceReaderMetrics.java +++ /dev/null @@ -1,61 +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.source.reader; - -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.MetricGroup; - -public class IcebergSourceReaderMetrics { - private final Counter assignedSplits; - private final Counter assignedBytes; - private final Counter finishedSplits; - private final Counter finishedBytes; - private final Counter splitReaderFetchCalls; - - public IcebergSourceReaderMetrics(MetricGroup metrics, String fullTableName) { - MetricGroup readerMetrics = - metrics.addGroup("IcebergSourceReader").addGroup("table", fullTableName); - - this.assignedSplits = readerMetrics.counter("assignedSplits"); - this.assignedBytes = readerMetrics.counter("assignedBytes"); - this.finishedSplits = readerMetrics.counter("finishedSplits"); - this.finishedBytes = readerMetrics.counter("finishedBytes"); - this.splitReaderFetchCalls = readerMetrics.counter("splitReaderFetchCalls"); - } - - public void incrementAssignedSplits(long count) { - assignedSplits.inc(count); - } - - public void incrementAssignedBytes(long count) { - assignedBytes.inc(count); - } - - public void incrementFinishedSplits(long count) { - finishedSplits.inc(count); - } - - public void incrementFinishedBytes(long count) { - finishedBytes.inc(count); - } - - public void incrementSplitReaderFetchCalls(long count) { - splitReaderFetchCalls.inc(count); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java deleted file mode 100644 index 9c20494fdbcd..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ /dev/null @@ -1,167 +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.source.reader; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Queue; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.connector.base.source.reader.RecordsBySplits; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.splitreader.SplitReader; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsAddition; -import org.apache.flink.connector.base.source.reader.splitreader.SplitsChange; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Queues; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -class IcebergSourceSplitReader implements SplitReader, IcebergSourceSplit> { - private static final Logger LOG = LoggerFactory.getLogger(IcebergSourceSplitReader.class); - - private final IcebergSourceReaderMetrics metrics; - private final ReaderFunction openSplitFunction; - private final SerializableComparator splitComparator; - private final int indexOfSubtask; - private final Queue splits; - - private CloseableIterator>> currentReader; - private IcebergSourceSplit currentSplit; - private String currentSplitId; - - IcebergSourceSplitReader( - IcebergSourceReaderMetrics metrics, - ReaderFunction openSplitFunction, - SerializableComparator splitComparator, - SourceReaderContext context) { - this.metrics = metrics; - this.openSplitFunction = openSplitFunction; - this.splitComparator = splitComparator; - this.indexOfSubtask = context.getIndexOfSubtask(); - this.splits = Queues.newArrayDeque(); - } - - /** - * The method reads a batch of records from the assigned splits. If all the records from the - * current split are returned then it will emit a {@link ArrayBatchRecords#finishedSplit(String)} - * batch to signal this event. In the next fetch loop the reader will continue with the next split - * (if any). - * - * @return The fetched records - * @throws IOException If there is an error during reading - */ - @Override - public RecordsWithSplitIds> fetch() throws IOException { - metrics.incrementSplitReaderFetchCalls(1); - if (currentReader == null) { - IcebergSourceSplit nextSplit = splits.poll(); - if (nextSplit != null) { - currentSplit = nextSplit; - currentSplitId = nextSplit.splitId(); - currentReader = openSplitFunction.apply(currentSplit); - } else { - // return an empty result, which will lead to split fetch to be idle. - // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); - } - } - - if (currentReader.hasNext()) { - // Because Iterator#next() doesn't support checked exception, - // we need to wrap and unwrap the checked IOException with UncheckedIOException - try { - return currentReader.next(); - } catch (UncheckedIOException e) { - throw e.getCause(); - } - } else { - return finishSplit(); - } - } - - @Override - public void handleSplitsChanges(SplitsChange splitsChange) { - if (!(splitsChange instanceof SplitsAddition)) { - throw new UnsupportedOperationException( - String.format("Unsupported split change: %s", splitsChange.getClass())); - } - - if (splitComparator != null) { - List newSplits = Lists.newArrayList(splitsChange.splits()); - newSplits.sort(splitComparator); - LOG.info("Add {} splits to reader: {}", newSplits.size(), newSplits); - splits.addAll(newSplits); - } else { - LOG.info("Add {} splits to reader", splitsChange.splits().size()); - splits.addAll(splitsChange.splits()); - } - metrics.incrementAssignedSplits(splitsChange.splits().size()); - metrics.incrementAssignedBytes(calculateBytes(splitsChange)); - } - - @Override - public void wakeUp() {} - - @Override - public void close() throws Exception { - currentSplitId = null; - if (currentReader != null) { - currentReader.close(); - } - } - - @Override - public void pauseOrResumeSplits( - Collection splitsToPause, Collection splitsToResume) { - // IcebergSourceSplitReader only reads splits sequentially. When waiting for watermark alignment - // the SourceOperator will stop processing and recycling the fetched batches. This exhausts the - // {@link ArrayPoolDataIteratorBatcher#pool} and the `currentReader.next()` call will be - // blocked even without split-level watermark alignment. Based on this the - // `pauseOrResumeSplits` and the `wakeUp` are left empty. - } - - private long calculateBytes(IcebergSourceSplit split) { - return split.task().files().stream().map(FileScanTask::length).reduce(0L, Long::sum); - } - - private long calculateBytes(SplitsChange splitsChanges) { - return splitsChanges.splits().stream().map(this::calculateBytes).reduce(0L, Long::sum); - } - - private ArrayBatchRecords finishSplit() throws IOException { - if (currentReader != null) { - currentReader.close(); - currentReader = null; - } - - ArrayBatchRecords finishRecords = ArrayBatchRecords.finishedSplit(currentSplitId); - LOG.info("Split reader {} finished split: {}", indexOfSubtask, currentSplitId); - metrics.incrementFinishedSplits(1); - metrics.incrementFinishedBytes(calculateBytes(currentSplit)); - currentSplitId = null; - return finishRecords; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java deleted file mode 100644 index 020e87646d05..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/LimitableDataIterator.java +++ /dev/null @@ -1,56 +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.source.reader; - -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.FileScanTaskReader; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class LimitableDataIterator extends DataIterator { - private final RecordLimiter limiter; - - LimitableDataIterator( - FileScanTaskReader fileScanTaskReader, - CombinedScanTask task, - FileIO io, - EncryptionManager encryption, - RecordLimiter limiter) { - super(fileScanTaskReader, task, io, encryption); - Preconditions.checkArgument(limiter != null, "Invalid record limiter: null"); - this.limiter = limiter; - } - - @Override - public boolean hasNext() { - if (limiter.reachedLimit()) { - return false; - } - - return super.hasNext(); - } - - @Override - public T next() { - limiter.increment(); - return super.next(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java deleted file mode 100644 index 1acb3df76102..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListBatchRecords.java +++ /dev/null @@ -1,85 +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.source.reader; - -import java.util.Collections; -import java.util.List; -import java.util.Set; -import javax.annotation.Nullable; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -class ListBatchRecords implements RecordsWithSplitIds> { - private String splitId; - private final List records; - private final Set finishedSplits; - private final RecordAndPosition recordAndPosition; - - // point to current read position within the records list - private int position; - - ListBatchRecords( - String splitId, - List records, - int fileOffset, - long startingRecordOffset, - Set finishedSplits) { - this.splitId = splitId; - this.records = records; - this.finishedSplits = - Preconditions.checkNotNull(finishedSplits, "finishedSplits can be empty but not null"); - this.recordAndPosition = new RecordAndPosition<>(); - this.recordAndPosition.set(null, fileOffset, startingRecordOffset); - - this.position = 0; - } - - @Nullable - @Override - public String nextSplit() { - String nextSplit = this.splitId; - // set the splitId to null to indicate no more splits - // this class only contains record for one split - this.splitId = null; - return nextSplit; - } - - @Nullable - @Override - public RecordAndPosition nextRecordFromSplit() { - if (position < records.size()) { - recordAndPosition.record(records.get(position)); - position++; - return recordAndPosition; - } else { - return null; - } - } - - @Override - public Set finishedSplits() { - return finishedSplits; - } - - public static ListBatchRecords forRecords( - String splitId, List records, int fileOffset, long startingRecordOffset) { - return new ListBatchRecords<>( - splitId, records, fileOffset, startingRecordOffset, Collections.emptySet()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java deleted file mode 100644 index 365416239d37..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ListDataIteratorBatcher.java +++ /dev/null @@ -1,94 +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.source.reader; - -import java.io.IOException; -import java.util.List; -import java.util.NoSuchElementException; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -/** - * FlinkRecordReaderFunction essentially cloned objects already. So there is no need to use array - * pool to clone objects. Simply create a new ArrayList for each batch. - */ -class ListDataIteratorBatcher implements DataIteratorBatcher { - - private final int batchSize; - - ListDataIteratorBatcher(ReadableConfig config) { - this.batchSize = config.get(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT); - } - - @Override - public CloseableIterator>> batch( - String splitId, DataIterator dataIterator) { - return new ListBatchIterator(splitId, dataIterator); - } - - private class ListBatchIterator - implements CloseableIterator>> { - - private final String splitId; - private final DataIterator inputIterator; - - ListBatchIterator(String splitId, DataIterator inputIterator) { - this.splitId = splitId; - this.inputIterator = inputIterator; - } - - @Override - public boolean hasNext() { - return inputIterator.hasNext(); - } - - @Override - public RecordsWithSplitIds> next() { - if (!inputIterator.hasNext()) { - throw new NoSuchElementException(); - } - - final List batch = Lists.newArrayListWithCapacity(batchSize); - int recordCount = 0; - while (inputIterator.hasNext() && recordCount < batchSize) { - T nextRecord = inputIterator.next(); - batch.add(nextRecord); - recordCount++; - if (!inputIterator.currentFileHasNext()) { - // break early so that records have the same fileOffset. - break; - } - } - - return ListBatchRecords.forRecords( - splitId, batch, inputIterator.fileOffset(), inputIterator.recordOffset() - recordCount); - } - - @Override - public void close() throws IOException { - if (inputIterator != null) { - inputIterator.close(); - } - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java deleted file mode 100644 index fb4466913b90..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/MetaDataReaderFunction.java +++ /dev/null @@ -1,65 +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.source.reader; - -import org.apache.flink.annotation.Internal; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.DataTaskReader; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** Reading metadata tables (like snapshots, manifests, etc.) */ -@Internal -public class MetaDataReaderFunction extends DataIteratorReaderFunction { - private final Schema readSchema; - private final FileIO io; - private final EncryptionManager encryption; - - public MetaDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - FileIO io, - EncryptionManager encryption) { - super( - new ArrayPoolDataIteratorBatcher<>( - config, - new RowDataRecordFactory( - FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); - this.readSchema = readSchema(tableSchema, projectedSchema); - this.io = io; - this.encryption = encryption; - } - - @Override - public DataIterator createDataIterator(IcebergSourceSplit split) { - return new DataIterator<>(new DataTaskReader(readSchema), split.task(), io, encryption); - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java deleted file mode 100644 index 1ea91f10b4e7..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/ReaderFunction.java +++ /dev/null @@ -1,31 +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.source.reader; - -import java.io.Serializable; -import java.util.function.Function; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; - -@FunctionalInterface -public interface ReaderFunction - extends Serializable, - Function< - IcebergSourceSplit, CloseableIterator>>> {} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java deleted file mode 100644 index 6ac92592b6aa..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ /dev/null @@ -1,78 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.reader; - -import org.apache.flink.annotation.Internal; - -/** - * A record along with the reader position to be stored in the checkpoint. - * - *

    The position defines the point in the reader AFTER the record. Record processing and updating - * checkpointed state happens atomically. The position points to where the reader should resume - * after this record is processed. - * - *

    This mutable object is useful in cases where only one instance of a {@code RecordAndPosition} - * is needed at a time. Then the same instance of RecordAndPosition can be reused. - */ -@Internal -public class RecordAndPosition { - private T record; - private int fileOffset; - private long recordOffset; - - public RecordAndPosition(T record, int fileOffset, long recordOffset) { - this.record = record; - this.fileOffset = fileOffset; - this.recordOffset = recordOffset; - } - - public RecordAndPosition() {} - - // ------------------------------------------------------------------------ - - public T record() { - return record; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } - - /** Updates the record and position in this object. */ - public void set(T newRecord, int newFileOffset, long newRecordOffset) { - this.record = newRecord; - this.fileOffset = newFileOffset; - this.recordOffset = newRecordOffset; - } - - /** Sets the next record of a sequence. This increments the {@code recordOffset} by one. */ - public void record(T nextRecord) { - this.record = nextRecord; - this.recordOffset++; - } - - @Override - public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java deleted file mode 100644 index ef92e2e6b81f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordFactory.java +++ /dev/null @@ -1,34 +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.source.reader; - -import java.io.Serializable; - -/** - * In FLIP-27 source, SplitReader#fetch() returns a batch of records. Since DataIterator for RowData - * returns an iterator of reused RowData objects, RecordFactory is needed to (1) create object array - * that is recyclable via pool. (2) clone RowData element from DataIterator to the batch array. - */ -interface RecordFactory extends Serializable { - /** Create a batch of records */ - T[] createBatch(int batchSize); - - /** Clone record into the specified position of the batch array */ - void clone(T from, T[] batch, int position); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java deleted file mode 100644 index f260a53089ff..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordLimiter.java +++ /dev/null @@ -1,45 +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.source.reader; - -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.annotation.Internal; - -@Internal -class RecordLimiter { - private final long limit; - private final AtomicLong counter; - - static RecordLimiter create(long limit) { - return new RecordLimiter(limit); - } - - private RecordLimiter(long limit) { - this.limit = limit; - this.counter = new AtomicLong(0); - } - - public boolean reachedLimit() { - return limit > 0 && counter.get() >= limit; - } - - public void increment() { - counter.incrementAndGet(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java deleted file mode 100644 index c9208a0e1834..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataReaderFunction.java +++ /dev/null @@ -1,115 +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.source.reader; - -import java.util.List; -import org.apache.flink.configuration.ReadableConfig; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -public class RowDataReaderFunction extends DataIteratorReaderFunction { - private final Schema tableSchema; - private final Schema readSchema; - private final String nameMapping; - private final boolean caseSensitive; - private final FileIO io; - private final EncryptionManager encryption; - private final List filters; - private final long limit; - - private transient RecordLimiter recordLimiter = null; - - public RowDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters) { - this( - config, - tableSchema, - projectedSchema, - nameMapping, - caseSensitive, - io, - encryption, - filters, - -1L); - } - - public RowDataReaderFunction( - ReadableConfig config, - Schema tableSchema, - Schema projectedSchema, - String nameMapping, - boolean caseSensitive, - FileIO io, - EncryptionManager encryption, - List filters, - long limit) { - super( - new ArrayPoolDataIteratorBatcher<>( - config, - new RowDataRecordFactory( - FlinkSchemaUtil.convert(readSchema(tableSchema, projectedSchema))))); - this.tableSchema = tableSchema; - this.readSchema = readSchema(tableSchema, projectedSchema); - this.nameMapping = nameMapping; - this.caseSensitive = caseSensitive; - this.io = io; - this.encryption = encryption; - this.filters = filters; - this.limit = limit; - } - - @Override - public DataIterator createDataIterator(IcebergSourceSplit split) { - return new LimitableDataIterator<>( - new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters), - split.task(), - io, - encryption, - lazyLimiter()); - } - - private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { - Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); - return projectedSchema == null ? tableSchema : projectedSchema; - } - - /** Lazily create RecordLimiter to avoid the need to make it serializable */ - private RecordLimiter lazyLimiter() { - if (recordLimiter == null) { - this.recordLimiter = RecordLimiter.create(limit); - } - - return recordLimiter; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.java deleted file mode 100644 index 40d5c28d7bc7..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataRecordFactory.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.source.reader; - -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.flink.data.RowDataUtil; - -class RowDataRecordFactory implements RecordFactory { - private final RowType rowType; - private final TypeSerializer[] fieldSerializers; - private final RowData.FieldGetter[] fieldGetters; - - RowDataRecordFactory(RowType rowType) { - this.rowType = rowType; - this.fieldSerializers = createFieldSerializers(rowType); - this.fieldGetters = createFieldGetters(rowType); - } - - static TypeSerializer[] createFieldSerializers(RowType rowType) { - return rowType.getChildren().stream() - .map(InternalSerializers::create) - .toArray(TypeSerializer[]::new); - } - - static RowData.FieldGetter[] createFieldGetters(RowType rowType) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - - return fieldGetters; - } - - @Override - public RowData[] createBatch(int batchSize) { - RowData[] arr = new RowData[batchSize]; - for (int i = 0; i < batchSize; ++i) { - arr[i] = new GenericRowData(rowType.getFieldCount()); - } - return arr; - } - - @Override - public void clone(RowData from, RowData[] batch, int position) { - // Set the return value from RowDataUtil.clone back to the array. - // Clone method returns same clone target object (reused) if it is a GenericRowData. - // Clone method will allocate a new GenericRowData object - // if the target object is NOT a GenericRowData. - // So we should always set the clone return value back to the array. - batch[position] = - RowDataUtil.clone(from, batch[position], rowType, fieldSerializers, fieldGetters); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java deleted file mode 100644 index a6e2c1dae243..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SerializableRecordEmitter.java +++ /dev/null @@ -1,40 +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.source.reader; - -import java.io.Serializable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.connector.base.source.reader.RecordEmitter; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -@Internal -@FunctionalInterface -public interface SerializableRecordEmitter - extends RecordEmitter, T, IcebergSourceSplit>, Serializable { - static SerializableRecordEmitter defaultEmitter() { - return (element, output, split) -> { - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - }; - } - - static SerializableRecordEmitter emitterWithWatermark(SplitWatermarkExtractor extractor) { - return new WatermarkExtractorRecordEmitter<>(extractor); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java deleted file mode 100644 index d1c50ac8ca52..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/SplitWatermarkExtractor.java +++ /dev/null @@ -1,28 +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.source.reader; - -import java.io.Serializable; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; - -/** The interface used to extract watermarks from splits. */ -public interface SplitWatermarkExtractor extends Serializable { - /** Get the watermark for a split. */ - long extractWatermark(IcebergSourceSplit split); -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java deleted file mode 100644 index 02ef57d344b1..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/reader/WatermarkExtractorRecordEmitter.java +++ /dev/null @@ -1,67 +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.source.reader; - -import org.apache.flink.api.common.eventtime.Watermark; -import org.apache.flink.api.connector.source.SourceOutput; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * Emitter which emits the watermarks, records and updates the split position. - * - *

    The Emitter emits watermarks at the beginning of every split provided by the {@link - * SplitWatermarkExtractor}. - */ -class WatermarkExtractorRecordEmitter implements SerializableRecordEmitter { - private static final Logger LOG = LoggerFactory.getLogger(WatermarkExtractorRecordEmitter.class); - private final SplitWatermarkExtractor timeExtractor; - private String lastSplitId = null; - private long watermark; - - WatermarkExtractorRecordEmitter(SplitWatermarkExtractor timeExtractor) { - this.timeExtractor = timeExtractor; - } - - @Override - public void emitRecord( - RecordAndPosition element, SourceOutput output, IcebergSourceSplit split) { - if (!split.splitId().equals(lastSplitId)) { - long newWatermark = timeExtractor.extractWatermark(split); - if (newWatermark < watermark) { - LOG.info( - "Received a new split with lower watermark. Previous watermark = {}, current watermark = {}, previous split = {}, current split = {}", - watermark, - newWatermark, - lastSplitId, - split.splitId()); - } else { - watermark = newWatermark; - output.emitWatermark(new Watermark(watermark)); - LOG.debug("Watermark = {} emitted based on split = {}", watermark, lastSplitId); - } - - lastSplitId = split.splitId(); - } - - output.collect(element.record()); - split.updatePosition(element.fileOffset(), element.recordOffset()); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java deleted file mode 100644 index 344f64833b62..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplit.java +++ /dev/null @@ -1,220 +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.source.split; - -import java.io.IOException; -import java.io.Serializable; -import java.util.Collection; -import java.util.List; -import java.util.stream.Collectors; -import javax.annotation.Nullable; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceSplit; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.util.InstantiationUtil; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.ScanTaskParser; -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.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -@Internal -public class IcebergSourceSplit implements SourceSplit, Serializable { - private static final long serialVersionUID = 1L; - private static final ThreadLocal SERIALIZER_CACHE = - ThreadLocal.withInitial(() -> new DataOutputSerializer(1024)); - - private final CombinedScanTask task; - - private int fileOffset; - private long recordOffset; - - // The splits are frequently serialized into checkpoints. - // Caching the byte representation makes repeated serialization cheap. - @Nullable private transient byte[] serializedBytesCache; - - private IcebergSourceSplit(CombinedScanTask task, int fileOffset, long recordOffset) { - this.task = task; - this.fileOffset = fileOffset; - this.recordOffset = recordOffset; - } - - public static IcebergSourceSplit fromCombinedScanTask(CombinedScanTask combinedScanTask) { - return fromCombinedScanTask(combinedScanTask, 0, 0L); - } - - public static IcebergSourceSplit fromCombinedScanTask( - CombinedScanTask combinedScanTask, int fileOffset, long recordOffset) { - return new IcebergSourceSplit(combinedScanTask, fileOffset, recordOffset); - } - - public CombinedScanTask task() { - return task; - } - - public int fileOffset() { - return fileOffset; - } - - public long recordOffset() { - return recordOffset; - } - - @Override - public String splitId() { - return MoreObjects.toStringHelper(this).add("files", toString(task.files())).toString(); - } - - public void updatePosition(int newFileOffset, long newRecordOffset) { - // invalidate the cache after position change - serializedBytesCache = null; - fileOffset = newFileOffset; - recordOffset = newRecordOffset; - } - - @Override - public String toString() { - return MoreObjects.toStringHelper(this) - .add("files", toString(task.files())) - .add("fileOffset", fileOffset) - .add("recordOffset", recordOffset) - .toString(); - } - - private String toString(Collection files) { - return Iterables.toString( - files.stream() - .map( - fileScanTask -> - MoreObjects.toStringHelper(fileScanTask) - .add("file", fileScanTask.file().path().toString()) - .add("start", fileScanTask.start()) - .add("length", fileScanTask.length()) - .toString()) - .collect(Collectors.toList())); - } - - byte[] serializeV1() throws IOException { - if (serializedBytesCache == null) { - serializedBytesCache = InstantiationUtil.serializeObject(this); - } - - return serializedBytesCache; - } - - static IcebergSourceSplit deserializeV1(byte[] serialized) throws IOException { - try { - return InstantiationUtil.deserializeObject( - serialized, IcebergSourceSplit.class.getClassLoader()); - } catch (ClassNotFoundException e) { - throw new RuntimeException("Failed to deserialize the split.", e); - } - } - - byte[] serializeV2() throws IOException { - return serialize(2); - } - - byte[] serializeV3() throws IOException { - return serialize(3); - } - - private byte[] serialize(int version) throws IOException { - if (serializedBytesCache == null) { - DataOutputSerializer out = SERIALIZER_CACHE.get(); - Collection fileScanTasks = task.tasks(); - Preconditions.checkArgument( - fileOffset >= 0 && fileOffset < fileScanTasks.size(), - "Invalid file offset: %s. Should be within the range of [0, %s)", - fileOffset, - fileScanTasks.size()); - - out.writeInt(fileOffset); - out.writeLong(recordOffset); - out.writeInt(fileScanTasks.size()); - - for (FileScanTask fileScanTask : fileScanTasks) { - String taskJson = ScanTaskParser.toJson(fileScanTask); - writeTaskJson(out, taskJson, version); - } - - serializedBytesCache = out.getCopyOfBuffer(); - out.clear(); - } - - return serializedBytesCache; - } - - private static void writeTaskJson(DataOutputSerializer out, String taskJson, int version) - throws IOException { - switch (version) { - case 2: - out.writeUTF(taskJson); - break; - case 3: - SerializerHelper.writeLongUTF(out, taskJson); - break; - default: - throw new IllegalArgumentException("Unsupported version: " + version); - } - } - - static IcebergSourceSplit deserializeV2(byte[] serialized, boolean caseSensitive) - throws IOException { - return deserialize(serialized, caseSensitive, 2); - } - - static IcebergSourceSplit deserializeV3(byte[] serialized, boolean caseSensitive) - throws IOException { - return deserialize(serialized, caseSensitive, 3); - } - - private static IcebergSourceSplit deserialize( - byte[] serialized, boolean caseSensitive, int version) throws IOException { - DataInputDeserializer in = new DataInputDeserializer(serialized); - int fileOffset = in.readInt(); - long recordOffset = in.readLong(); - int taskCount = in.readInt(); - - List tasks = Lists.newArrayListWithCapacity(taskCount); - for (int i = 0; i < taskCount; ++i) { - String taskJson = readTaskJson(in, version); - FileScanTask task = ScanTaskParser.fromJson(taskJson, caseSensitive); - tasks.add(task); - } - - CombinedScanTask combinedScanTask = new BaseCombinedScanTask(tasks); - return IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, fileOffset, recordOffset); - } - - private static String readTaskJson(DataInputDeserializer in, int version) throws IOException { - switch (version) { - case 2: - return in.readUTF(); - case 3: - return SerializerHelper.readLongUTF(in); - default: - throw new IllegalArgumentException("Unsupported version: " + version); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java deleted file mode 100644 index d4b0f9e1977d..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ /dev/null @@ -1,62 +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.source.split; - -import java.io.IOException; -import org.apache.flink.annotation.Internal; -import org.apache.flink.core.io.SimpleVersionedSerializer; - -@Internal -public class IcebergSourceSplitSerializer implements SimpleVersionedSerializer { - private static final int VERSION = 3; - - private final boolean caseSensitive; - - public IcebergSourceSplitSerializer(boolean caseSensitive) { - this.caseSensitive = caseSensitive; - } - - @Override - public int getVersion() { - return VERSION; - } - - @Override - public byte[] serialize(IcebergSourceSplit split) throws IOException { - return split.serializeV3(); - } - - @Override - public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOException { - switch (version) { - case 1: - return IcebergSourceSplit.deserializeV1(serialized); - case 2: - return IcebergSourceSplit.deserializeV2(serialized, caseSensitive); - case 3: - return IcebergSourceSplit.deserializeV3(serialized, caseSensitive); - default: - throw new IOException( - String.format( - "Failed to deserialize IcebergSourceSplit. " - + "Encountered unsupported version: %d. Supported version are [1]", - version)); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java deleted file mode 100644 index d9061e049e00..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitState.java +++ /dev/null @@ -1,37 +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.source.split; - -public class IcebergSourceSplitState { - private final IcebergSourceSplit split; - private final IcebergSourceSplitStatus status; - - public IcebergSourceSplitState(IcebergSourceSplit split, IcebergSourceSplitStatus status) { - this.split = split; - this.status = status; - } - - public IcebergSourceSplit split() { - return split; - } - - public IcebergSourceSplitStatus status() { - return status; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java deleted file mode 100644 index d4a84a165e1a..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitStatus.java +++ /dev/null @@ -1,25 +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.source.split; - -public enum IcebergSourceSplitStatus { - UNASSIGNED, - ASSIGNED, - COMPLETED -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java deleted file mode 100644 index 319648ca275c..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializableComparator.java +++ /dev/null @@ -1,24 +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.source.split; - -import java.io.Serializable; -import java.util.Comparator; - -public interface SerializableComparator extends Comparator, Serializable {} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java deleted file mode 100644 index 841969666ee5..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SerializerHelper.java +++ /dev/null @@ -1,206 +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.source.split; - -import java.io.IOException; -import java.io.Serializable; -import java.io.UTFDataFormatException; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; - -/** - * Helper class to serialize and deserialize strings longer than 65K. The inspiration is mostly - * taken from the class org.apache.flink.core.memory.DataInputSerializer.readUTF and - * org.apache.flink.core.memory.DataOutputSerializer.writeUTF. - */ -class SerializerHelper implements Serializable { - - private SerializerHelper() {} - - /** - * Similar to {@link DataOutputSerializer#writeUTF(String)}. Except this supports larger payloads - * which is up to max integer value. - * - *

    Note: This method can be removed when the method which does similar thing within the {@link - * DataOutputSerializer} already which does the same thing, so use that one instead once that is - * released on Flink version 1.20. - * - *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 - * - * @param out the output stream to write the string to. - * @param str the string value to be written. - */ - public static void writeLongUTF(DataOutputSerializer out, String str) throws IOException { - int strlen = str.length(); - long utflen = 0; - int ch; - - /* use charAt instead of copying String to char array */ - for (int i = 0; i < strlen; i++) { - ch = str.charAt(i); - utflen += getUTFBytesSize(ch); - - if (utflen > Integer.MAX_VALUE) { - throw new UTFDataFormatException("Encoded string reached maximum length: " + utflen); - } - } - - if (utflen > Integer.MAX_VALUE - 4) { - throw new UTFDataFormatException("Encoded string is too long: " + utflen); - } - - out.writeInt((int) utflen); - writeUTFBytes(out, str, (int) utflen); - } - - /** - * Similar to {@link DataInputDeserializer#readUTF()}. Except this supports larger payloads which - * is up to max integer value. - * - *

    Note: This method can be removed when the method which does similar thing within the {@link - * DataOutputSerializer} already which does the same thing, so use that one instead once that is - * released on Flink version 1.20. - * - *

    See * FLINK-34228 * https://github.com/apache/flink/pull/24191 - * - * @param in the input stream to read the string from. - * @return the string value read from the input stream. - * @throws IOException if an I/O error occurs when reading from the input stream. - */ - public static String readLongUTF(DataInputDeserializer in) throws IOException { - int utflen = in.readInt(); - byte[] bytearr = new byte[utflen]; - char[] chararr = new char[utflen]; - - int ch; - int char2; - int char3; - int count = 0; - int chararrCount = 0; - - in.readFully(bytearr, 0, utflen); - - while (count < utflen) { - ch = (int) bytearr[count] & 0xff; - if (ch > 127) { - break; - } - count++; - chararr[chararrCount++] = (char) ch; - } - - while (count < utflen) { - ch = (int) bytearr[count] & 0xff; - switch (ch >> 4) { - case 0: - case 1: - case 2: - case 3: - case 4: - case 5: - case 6: - case 7: - /* 0xxxxxxx */ - count++; - chararr[chararrCount++] = (char) ch; - break; - case 12: - case 13: - /* 110x xxxx 10xx xxxx */ - count += 2; - if (count > utflen) { - throw new UTFDataFormatException("malformed input: partial character at end"); - } - char2 = bytearr[count - 1]; - if ((char2 & 0xC0) != 0x80) { - throw new UTFDataFormatException("malformed input around byte " + count); - } - chararr[chararrCount++] = (char) (((ch & 0x1F) << 6) | (char2 & 0x3F)); - break; - case 14: - /* 1110 xxxx 10xx xxxx 10xx xxxx */ - count += 3; - if (count > utflen) { - throw new UTFDataFormatException("malformed input: partial character at end"); - } - char2 = bytearr[count - 2]; - char3 = bytearr[count - 1]; - if (((char2 & 0xC0) != 0x80) || ((char3 & 0xC0) != 0x80)) { - throw new UTFDataFormatException("malformed input around byte " + (count - 1)); - } - chararr[chararrCount++] = - (char) (((ch & 0x0F) << 12) | ((char2 & 0x3F) << 6) | (char3 & 0x3F)); - break; - default: - /* 10xx xxxx, 1111 xxxx */ - throw new UTFDataFormatException("malformed input around byte " + count); - } - } - // The number of chars produced may be less than utflen - return new String(chararr, 0, chararrCount); - } - - private static int getUTFBytesSize(int ch) { - if ((ch >= 0x0001) && (ch <= 0x007F)) { - return 1; - } else if (ch > 0x07FF) { - return 3; - } else { - return 2; - } - } - - private static void writeUTFBytes(DataOutputSerializer out, String str, int utflen) - throws IOException { - int strlen = str.length(); - int ch; - - int len = Math.max(1024, utflen); - - byte[] bytearr = new byte[len]; - int count = 0; - - int index; - for (index = 0; index < strlen; index++) { - ch = str.charAt(index); - if (!((ch >= 0x0001) && (ch <= 0x007F))) { - break; - } - bytearr[count++] = (byte) ch; - } - - for (; index < strlen; index++) { - ch = str.charAt(index); - if ((ch >= 0x0001) && (ch <= 0x007F)) { - bytearr[count++] = (byte) ch; - } else if (ch > 0x07FF) { - bytearr[count++] = (byte) (0xE0 | ((ch >> 12) & 0x0F)); - bytearr[count++] = (byte) (0x80 | ((ch >> 6) & 0x3F)); - bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); - } else { - bytearr[count++] = (byte) (0xC0 | ((ch >> 6) & 0x1F)); - bytearr[count++] = (byte) (0x80 | (ch & 0x3F)); - } - } - - out.write(bytearr, 0, count); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java deleted file mode 100644 index 56ee92014d12..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitComparators.java +++ /dev/null @@ -1,76 +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.source.split; - -import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * Provides implementations of {@link org.apache.iceberg.flink.source.split.SerializableComparator} - * which could be used for ordering splits. These are used by the {@link - * org.apache.iceberg.flink.source.assigner.OrderedSplitAssignerFactory} and the {@link - * org.apache.iceberg.flink.source.reader.IcebergSourceReader} - */ -public class SplitComparators { - private SplitComparators() {} - - /** Comparator which orders the splits based on the file sequence number of the data files */ - 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"); - - Long seq1 = o1.task().files().iterator().next().file().fileSequenceNumber(); - Long seq2 = o2.task().files().iterator().next().file().fileSequenceNumber(); - - Preconditions.checkNotNull( - seq1, - "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", - o1); - Preconditions.checkNotNull( - seq2, - "Invalid file sequence number: null. Doesn't support splits written with V1 format: %s", - o2); - - int temp = Long.compare(seq1, seq2); - if (temp != 0) { - return temp; - } else { - return o1.splitId().compareTo(o2.splitId()); - } - }; - } - - /** Comparator which orders the splits based on watermark of the splits */ - public static SerializableComparator watermark( - SplitWatermarkExtractor watermarkExtractor) { - return (IcebergSourceSplit o1, IcebergSourceSplit o2) -> { - long watermark1 = watermarkExtractor.extractWatermark(o1); - long watermark2 = watermarkExtractor.extractWatermark(o2); - - int temp = Long.compare(watermark1, watermark2); - if (temp != 0) { - return temp; - } else { - return o1.splitId().compareTo(o2.splitId()); - } - }; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java deleted file mode 100644 index eabd757aa638..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/source/split/SplitRequestEvent.java +++ /dev/null @@ -1,54 +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.source.split; - -import java.util.Collection; -import java.util.Collections; -import org.apache.flink.annotation.Internal; -import org.apache.flink.api.connector.source.SourceEvent; - -/** We can remove this class once FLINK-21364 is resolved. */ -@Internal -public class SplitRequestEvent implements SourceEvent { - private static final long serialVersionUID = 1L; - - private final Collection finishedSplitIds; - private final String requesterHostname; - - public SplitRequestEvent() { - this(Collections.emptyList()); - } - - public SplitRequestEvent(Collection finishedSplitIds) { - this(finishedSplitIds, null); - } - - public SplitRequestEvent(Collection finishedSplitIds, String requesterHostname) { - this.finishedSplitIds = finishedSplitIds; - this.requesterHostname = requesterHostname; - } - - public Collection finishedSplitIds() { - return finishedSplitIds; - } - - public String requesterHostname() { - return requesterHostname; - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java deleted file mode 100644 index 2bbc9cf208fe..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkAlterTableUtil.java +++ /dev/null @@ -1,248 +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.util; - -import java.util.List; -import java.util.Map; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.catalog.TableChange; -import org.apache.flink.table.catalog.UniqueConstraint; -import org.apache.iceberg.Table; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.UpdateProperties; -import org.apache.iceberg.UpdateSchema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.types.Type; - -public class FlinkAlterTableUtil { - private FlinkAlterTableUtil() {} - - public static void commitChanges( - Table table, - String setLocation, - String setSnapshotId, - String pickSnapshotId, - Map setProperties) { - commitManageSnapshots(table, setSnapshotId, pickSnapshotId); - - Transaction transaction = table.newTransaction(); - - if (setLocation != null) { - transaction.updateLocation().setLocation(setLocation).commit(); - } - - if (!setProperties.isEmpty()) { - UpdateProperties updateProperties = transaction.updateProperties(); - setProperties.forEach( - (k, v) -> { - if (v == null) { - updateProperties.remove(k); - } else { - updateProperties.set(k, v); - } - }); - updateProperties.commit(); - } - - transaction.commitTransaction(); - } - - public static void commitChanges( - Table table, - String setLocation, - String setSnapshotId, - String pickSnapshotId, - List schemaChanges, - List propertyChanges) { - commitManageSnapshots(table, setSnapshotId, pickSnapshotId); - - Transaction transaction = table.newTransaction(); - - if (setLocation != null) { - transaction.updateLocation().setLocation(setLocation).commit(); - } - - if (!schemaChanges.isEmpty()) { - UpdateSchema updateSchema = transaction.updateSchema(); - FlinkAlterTableUtil.applySchemaChanges(updateSchema, schemaChanges); - updateSchema.commit(); - } - - if (!propertyChanges.isEmpty()) { - UpdateProperties updateProperties = transaction.updateProperties(); - FlinkAlterTableUtil.applyPropertyChanges(updateProperties, propertyChanges); - updateProperties.commit(); - } - - transaction.commitTransaction(); - } - - public static void commitManageSnapshots( - Table table, String setSnapshotId, String cherrypickSnapshotId) { - // don't allow setting the snapshot and picking a commit at the same time because order is - // ambiguous and choosing one order leads to different results - Preconditions.checkArgument( - setSnapshotId == null || cherrypickSnapshotId == null, - "Cannot set the current snapshot ID and cherry-pick snapshot changes"); - - if (setSnapshotId != null) { - long newSnapshotId = Long.parseLong(setSnapshotId); - table.manageSnapshots().setCurrentSnapshot(newSnapshotId).commit(); - } - - // if updating the table snapshot, perform that update first in case it fails - if (cherrypickSnapshotId != null) { - long newSnapshotId = Long.parseLong(cherrypickSnapshotId); - table.manageSnapshots().cherrypick(newSnapshotId).commit(); - } - } - - /** - * Applies a list of Flink table changes to an {@link UpdateSchema} operation. - * - * @param pendingUpdate an uncommitted UpdateSchema operation to configure - * @param schemaChanges a list of Flink table changes - */ - public static void applySchemaChanges( - UpdateSchema pendingUpdate, List schemaChanges) { - for (TableChange change : schemaChanges) { - if (change instanceof TableChange.AddColumn) { - TableChange.AddColumn addColumn = (TableChange.AddColumn) change; - Column flinkColumn = addColumn.getColumn(); - Preconditions.checkArgument( - FlinkCompatibilityUtil.isPhysicalColumn(flinkColumn), - "Unsupported table change: Adding computed column %s.", - flinkColumn.getName()); - Type icebergType = FlinkSchemaUtil.convert(flinkColumn.getDataType().getLogicalType()); - if (flinkColumn.getDataType().getLogicalType().isNullable()) { - pendingUpdate.addColumn( - flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); - } else { - pendingUpdate.addRequiredColumn( - flinkColumn.getName(), icebergType, flinkColumn.getComment().orElse(null)); - } - } else if (change instanceof TableChange.ModifyColumn) { - TableChange.ModifyColumn modifyColumn = (TableChange.ModifyColumn) change; - applyModifyColumn(pendingUpdate, modifyColumn); - } else if (change instanceof TableChange.DropColumn) { - TableChange.DropColumn dropColumn = (TableChange.DropColumn) change; - pendingUpdate.deleteColumn(dropColumn.getColumnName()); - } else if (change instanceof TableChange.AddWatermark) { - throw new UnsupportedOperationException("Unsupported table change: AddWatermark."); - } else if (change instanceof TableChange.ModifyWatermark) { - throw new UnsupportedOperationException("Unsupported table change: ModifyWatermark."); - } else if (change instanceof TableChange.DropWatermark) { - throw new UnsupportedOperationException("Unsupported table change: DropWatermark."); - } else if (change instanceof TableChange.AddUniqueConstraint) { - TableChange.AddUniqueConstraint addPk = (TableChange.AddUniqueConstraint) change; - applyUniqueConstraint(pendingUpdate, addPk.getConstraint()); - } else if (change instanceof TableChange.ModifyUniqueConstraint) { - TableChange.ModifyUniqueConstraint modifyPk = (TableChange.ModifyUniqueConstraint) change; - applyUniqueConstraint(pendingUpdate, modifyPk.getNewConstraint()); - } else if (change instanceof TableChange.DropConstraint) { - throw new UnsupportedOperationException("Unsupported table change: DropConstraint."); - } else { - throw new UnsupportedOperationException("Cannot apply unknown table change: " + change); - } - } - } - - /** - * Applies a list of Flink table property changes to an {@link UpdateProperties} operation. - * - * @param pendingUpdate an uncommitted UpdateProperty operation to configure - * @param propertyChanges a list of Flink table changes - */ - public static void applyPropertyChanges( - UpdateProperties pendingUpdate, List propertyChanges) { - for (TableChange change : propertyChanges) { - if (change instanceof TableChange.SetOption) { - TableChange.SetOption setOption = (TableChange.SetOption) change; - pendingUpdate.set(setOption.getKey(), setOption.getValue()); - } else if (change instanceof TableChange.ResetOption) { - TableChange.ResetOption resetOption = (TableChange.ResetOption) change; - pendingUpdate.remove(resetOption.getKey()); - } else { - throw new UnsupportedOperationException( - "The given table change is not a property change: " + change); - } - } - } - - private static void applyModifyColumn( - UpdateSchema pendingUpdate, TableChange.ModifyColumn modifyColumn) { - if (modifyColumn instanceof TableChange.ModifyColumnName) { - TableChange.ModifyColumnName modifyName = (TableChange.ModifyColumnName) modifyColumn; - pendingUpdate.renameColumn(modifyName.getOldColumnName(), modifyName.getNewColumnName()); - } else if (modifyColumn instanceof TableChange.ModifyColumnPosition) { - TableChange.ModifyColumnPosition modifyPosition = - (TableChange.ModifyColumnPosition) modifyColumn; - applyModifyColumnPosition(pendingUpdate, modifyPosition); - } else if (modifyColumn instanceof TableChange.ModifyPhysicalColumnType) { - TableChange.ModifyPhysicalColumnType modifyType = - (TableChange.ModifyPhysicalColumnType) modifyColumn; - Type type = FlinkSchemaUtil.convert(modifyType.getNewType().getLogicalType()); - String columnName = modifyType.getOldColumn().getName(); - pendingUpdate.updateColumn(columnName, type.asPrimitiveType()); - if (modifyType.getNewColumn().getDataType().getLogicalType().isNullable()) { - pendingUpdate.makeColumnOptional(columnName); - } else { - pendingUpdate.requireColumn(columnName); - } - } else if (modifyColumn instanceof TableChange.ModifyColumnComment) { - TableChange.ModifyColumnComment modifyComment = - (TableChange.ModifyColumnComment) modifyColumn; - pendingUpdate.updateColumnDoc( - modifyComment.getOldColumn().getName(), modifyComment.getNewComment()); - } else { - throw new UnsupportedOperationException( - "Cannot apply unknown modify-column change: " + modifyColumn); - } - } - - private static void applyModifyColumnPosition( - UpdateSchema pendingUpdate, TableChange.ModifyColumnPosition modifyColumnPosition) { - TableChange.ColumnPosition newPosition = modifyColumnPosition.getNewPosition(); - if (newPosition instanceof TableChange.First) { - pendingUpdate.moveFirst(modifyColumnPosition.getOldColumn().getName()); - } else if (newPosition instanceof TableChange.After) { - TableChange.After after = (TableChange.After) newPosition; - pendingUpdate.moveAfter(modifyColumnPosition.getOldColumn().getName(), after.column()); - } else { - throw new UnsupportedOperationException( - "Cannot apply unknown modify-column-position change: " + modifyColumnPosition); - } - } - - private static void applyUniqueConstraint( - UpdateSchema pendingUpdate, UniqueConstraint constraint) { - switch (constraint.getType()) { - case PRIMARY_KEY: - pendingUpdate.setIdentifierFields(constraint.getColumns()); - break; - case UNIQUE_KEY: - throw new UnsupportedOperationException( - "Unsupported table change: setting unique key constraints."); - default: - throw new UnsupportedOperationException( - "Cannot apply unknown unique constraint: " + constraint.getType().name()); - } - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java deleted file mode 100644 index f02af894e82b..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkCompatibilityUtil.java +++ /dev/null @@ -1,47 +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.util; - -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.catalog.Column; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.InternalTypeInfo; -import org.apache.flink.table.types.logical.RowType; - -/** - * This is a small util class that try to hide calls to Flink Internal or PublicEvolve interfaces as - * Flink can change those APIs during minor version release. - */ -public class FlinkCompatibilityUtil { - - private FlinkCompatibilityUtil() {} - - public static TypeInformation toTypeInfo(RowType rowType) { - return InternalTypeInfo.of(rowType); - } - - public static boolean isPhysicalColumn(TableColumn column) { - return column.isPhysical(); - } - - public static boolean isPhysicalColumn(Column column) { - return column.isPhysical(); - } -} diff --git a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java b/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java deleted file mode 100644 index 20b33e615e5f..000000000000 --- a/flink/v1.17/flink/src/main/java/org/apache/iceberg/flink/util/FlinkPackage.java +++ /dev/null @@ -1,61 +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.util; - -import java.util.concurrent.atomic.AtomicReference; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; - -public class FlinkPackage { - - private static final AtomicReference VERSION = new AtomicReference<>(); - public static final String FLINK_UNKNOWN_VERSION = "FLINK-UNKNOWN-VERSION"; - - private FlinkPackage() {} - - /** Returns Flink version string like x.y.z */ - public static String version() { - if (null == VERSION.get()) { - String detectedVersion; - try { - detectedVersion = versionFromJar(); - // use unknown version in case exact implementation version can't be found from the jar - // (this can happen if the DataStream class appears multiple times in the same classpath - // such as with shading) - detectedVersion = detectedVersion != null ? detectedVersion : FLINK_UNKNOWN_VERSION; - } catch (Exception e) { - detectedVersion = FLINK_UNKNOWN_VERSION; - } - VERSION.set(detectedVersion); - } - - return VERSION.get(); - } - - @VisibleForTesting - static String versionFromJar() { - // Choose {@link DataStream} class because it is one of the core Flink API - return DataStream.class.getPackage().getImplementationVersion(); - } - - @VisibleForTesting - static void setVersion(String version) { - VERSION.set(version); - } -} diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 29a9955a7e20..000000000000 --- a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.FlinkDynamicTableFactory diff --git a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory b/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory deleted file mode 100644 index 2b6bfa3cd579..000000000000 --- a/flink/v1.17/flink/src/main/resources/META-INF/services/org.apache.flink.table.factories.TableFactory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.FlinkCatalogFactory diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java deleted file mode 100644 index 4184526a6a1a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/AvroGenericRecordConverterBase.java +++ /dev/null @@ -1,90 +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; - -import org.junit.jupiter.api.Test; - -public abstract class AvroGenericRecordConverterBase { - protected abstract void testConverter(DataGenerator dataGenerator) throws Exception; - - @Test - public void testPrimitiveTypes() throws Exception { - testConverter(new DataGenerators.Primitives()); - } - - @Test - public void testStructOfPrimitive() throws Exception { - testConverter(new DataGenerators.StructOfPrimitive()); - } - - @Test - public void testStructOfArray() throws Exception { - testConverter(new DataGenerators.StructOfArray()); - } - - @Test - public void testStructOfMap() throws Exception { - testConverter(new DataGenerators.StructOfMap()); - } - - @Test - public void testStructOfStruct() throws Exception { - testConverter(new DataGenerators.StructOfStruct()); - } - - @Test - public void testArrayOfPrimitive() throws Exception { - testConverter(new DataGenerators.ArrayOfPrimitive()); - } - - @Test - public void testArrayOfArray() throws Exception { - testConverter(new DataGenerators.ArrayOfArray()); - } - - @Test - public void testArrayOfMap() throws Exception { - testConverter(new DataGenerators.ArrayOfMap()); - } - - @Test - public void testArrayOfStruct() throws Exception { - testConverter(new DataGenerators.ArrayOfStruct()); - } - - @Test - public void testMapOfPrimitives() throws Exception { - testConverter(new DataGenerators.MapOfPrimitives()); - } - - @Test - public void testMapOfArray() throws Exception { - testConverter(new DataGenerators.MapOfArray()); - } - - @Test - public void testMapOfMap() throws Exception { - testConverter(new DataGenerators.MapOfMap()); - } - - @Test - public void testMapOfStruct() throws Exception { - testConverter(new DataGenerators.MapOfStruct()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java deleted file mode 100644 index 91ed3c4adea3..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ /dev/null @@ -1,143 +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; - -import java.io.File; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import org.apache.flink.util.ArrayUtils; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.SupportsNamespaces; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; -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.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; - -@ExtendWith(ParameterizedTestExtension.class) -public abstract class CatalogTestBase extends TestBase { - - protected static final String DATABASE = "db"; - @TempDir protected File hiveWarehouse; - @TempDir protected File hadoopWarehouse; - - @Parameter(index = 0) - protected String catalogName; - - @Parameter(index = 1) - protected Namespace baseNamespace; - - protected Catalog validationCatalog; - protected SupportsNamespaces validationNamespaceCatalog; - protected Map config = Maps.newHashMap(); - - protected String flinkDatabase; - protected Namespace icebergNamespace; - protected boolean isHadoopCatalog; - - @Parameters(name = "catalogName={0}, baseNamespace={1}") - protected static List parameters() { - return Arrays.asList( - new Object[] {"testhive", Namespace.empty()}, - new Object[] {"testhadoop", Namespace.empty()}, - new Object[] {"testhadoop_basenamespace", Namespace.of("l0", "l1")}); - } - - @BeforeEach - public void before() { - this.isHadoopCatalog = catalogName.startsWith("testhadoop"); - this.validationCatalog = - isHadoopCatalog - ? new HadoopCatalog(hiveConf, "file:" + hadoopWarehouse.getPath()) - : catalog; - this.validationNamespaceCatalog = (SupportsNamespaces) validationCatalog; - - config.put("type", "iceberg"); - if (!baseNamespace.isEmpty()) { - config.put(FlinkCatalogFactory.BASE_NAMESPACE, baseNamespace.toString()); - } - if (isHadoopCatalog) { - config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"); - } else { - config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - config.put(CatalogProperties.URI, getURI(hiveConf)); - } - config.put(CatalogProperties.WAREHOUSE_LOCATION, String.format("file://%s", warehouseRoot())); - - this.flinkDatabase = catalogName + "." + DATABASE; - this.icebergNamespace = - Namespace.of(ArrayUtils.concat(baseNamespace.levels(), new String[] {DATABASE})); - sql("CREATE CATALOG %s WITH %s", catalogName, toWithClause(config)); - } - - @AfterEach - public void clean() { - dropCatalog(catalogName, true); - } - - protected String warehouseRoot() { - if (isHadoopCatalog) { - return hadoopWarehouse.getAbsolutePath(); - } else { - return hiveWarehouse.getAbsolutePath(); - } - } - - protected String getFullQualifiedTableName(String tableName) { - final List levels = Lists.newArrayList(icebergNamespace.levels()); - levels.add(tableName); - return Joiner.on('.').join(levels); - } - - static String getURI(HiveConf conf) { - return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); - } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java deleted file mode 100644 index b1e3b20ff7ac..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerator.java +++ /dev/null @@ -1,42 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericRecord; - -/** - * This interface defines test data generator. Different implementations for primitive and complex - * nested fields are defined in {@link DataGenerators}. - */ -public interface DataGenerator { - Schema icebergSchema(); - - RowType flinkRowType(); - - org.apache.avro.Schema avroSchema(); - - GenericRecord generateIcebergGenericRecord(); - - GenericRowData generateFlinkRowData(); - - org.apache.avro.generic.GenericRecord generateAvroGenericRecord(); -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java deleted file mode 100644 index e2cd411d7069..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/DataGenerators.java +++ /dev/null @@ -1,1172 +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; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import com.fasterxml.jackson.databind.node.IntNode; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.charset.StandardCharsets; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.Arrays; -import java.util.List; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import org.apache.avro.LogicalTypes; -import org.apache.avro.SchemaBuilder; -import org.apache.avro.generic.GenericData; -import org.apache.avro.util.Utf8; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.joda.time.DateTime; -import org.joda.time.DateTimeZone; -import org.joda.time.Days; - -/** - * Util class to generate test data with extensive coverage different field types: from primitives - * to complex nested types. - */ -public class DataGenerators { - - public static class Primitives implements DataGenerator { - private static final DateTime JODA_DATETIME_EPOC = - new DateTime(1970, 1, 1, 0, 0, 0, 0, DateTimeZone.UTC); - private static final DateTime JODA_DATETIME_20220110 = - new DateTime(2022, 1, 10, 0, 0, 0, 0, DateTimeZone.UTC); - private static final int DAYS_BTW_EPOC_AND_20220110 = - Days.daysBetween(JODA_DATETIME_EPOC, JODA_DATETIME_20220110).getDays(); - private static final int HOUR_8_IN_MILLI = (int) TimeUnit.HOURS.toMillis(8); - - private static final LocalDate JAVA_LOCAL_DATE_20220110 = LocalDate.of(2022, 1, 10); - private static final LocalTime JAVA_LOCAL_TIME_HOUR8 = LocalTime.of(8, 0); - private static final OffsetDateTime JAVA_OFFSET_DATE_TIME_20220110 = - OffsetDateTime.of(2022, 1, 10, 0, 0, 0, 0, ZoneOffset.UTC); - private static final LocalDateTime JAVA_LOCAL_DATE_TIME_20220110 = - LocalDateTime.of(2022, 1, 10, 0, 0, 0); - private static final BigDecimal BIG_DECIMAL_NEGATIVE = new BigDecimal("-1.50"); - private static final byte[] FIXED_BYTES = "012345689012345".getBytes(StandardCharsets.UTF_8); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - // primitive types - Types.NestedField.optional(2, "boolean_field", Types.BooleanType.get()), - Types.NestedField.optional(3, "int_field", Types.IntegerType.get()), - Types.NestedField.optional(4, "long_field", Types.LongType.get()), - Types.NestedField.optional(5, "float_field", Types.FloatType.get()), - Types.NestedField.optional(6, "double_field", Types.DoubleType.get()), - Types.NestedField.required(7, "string_field", Types.StringType.get()), - Types.NestedField.required(8, "date_field", Types.DateType.get()), - Types.NestedField.required(9, "time_field", Types.TimeType.get()), - Types.NestedField.required(10, "ts_with_zone_field", Types.TimestampType.withZone()), - Types.NestedField.required( - 11, "ts_without_zone_field", Types.TimestampType.withoutZone()), - Types.NestedField.required(12, "uuid_field", Types.UUIDType.get()), - Types.NestedField.required(13, "binary_field", Types.BinaryType.get()), - Types.NestedField.required(14, "decimal_field", Types.DecimalType.of(9, 2)), - Types.NestedField.required(15, "fixed_field", Types.FixedType.ofLength(16))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - /** - * Fix up Avro Schema that is converted from Iceberg Schema. - * - * @param schemaConvertedFromIceberg Avro Schema converted from Iceberg schema via {@link - * AvroSchemaUtil#convert(Schema, String)} - */ - private org.apache.avro.Schema fixupAvroSchemaConvertedFromIcebergSchema( - org.apache.avro.Schema schemaConvertedFromIceberg) { - List fixedFields = - schemaConvertedFromIceberg.getFields().stream() - .map( - field -> { - org.apache.avro.Schema.Field updatedField = field; - if (field.name().equals("time_field")) { - // Iceberg's AvroSchemaUtil uses timestamp-micros with Long value for time - // field, while AvroToRowDataConverters#convertToTime() always looks for - // Integer value assuming millis. The root problem is that - // AvroToRowDataConverters#createConverter() uses LogicalTypeRoot to - // determine converter and LogicalTypeRoot lost the timestamp precision - // carried by LogicalType like Time(6). - org.apache.avro.Schema fieldSchema = - LogicalTypes.timeMillis() - .addToSchema( - org.apache.avro.Schema.create(org.apache.avro.Schema.Type.INT)); - updatedField = new org.apache.avro.Schema.Field("time_field", fieldSchema); - } - - return new org.apache.avro.Schema.Field(updatedField, updatedField.schema()); - }) - .collect(Collectors.toList()); - return org.apache.avro.Schema.createRecord( - schemaConvertedFromIceberg.getName(), - schemaConvertedFromIceberg.getDoc(), - schemaConvertedFromIceberg.getNamespace(), - schemaConvertedFromIceberg.isError(), - fixedFields); - } - - private final org.apache.avro.Schema avroSchema = - fixupAvroSchemaConvertedFromIcebergSchema(AvroSchemaUtil.convert(icebergSchema, "table")); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("boolean_field", false); - genericRecord.setField("int_field", Integer.MAX_VALUE); - genericRecord.setField("long_field", Long.MAX_VALUE); - genericRecord.setField("float_field", Float.MAX_VALUE); - genericRecord.setField("double_field", Double.MAX_VALUE); - genericRecord.setField("string_field", "str"); - - genericRecord.setField("date_field", JAVA_LOCAL_DATE_20220110); - genericRecord.setField("time_field", JAVA_LOCAL_TIME_HOUR8); - genericRecord.setField("ts_with_zone_field", JAVA_OFFSET_DATE_TIME_20220110); - genericRecord.setField("ts_without_zone_field", JAVA_LOCAL_DATE_TIME_20220110); - - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - - genericRecord.setField("uuid_field", UUID.nameUUIDFromBytes(uuidBytes)); - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - genericRecord.setField("binary_field", ByteBuffer.wrap(binaryBytes)); - - genericRecord.setField("decimal_field", BIG_DECIMAL_NEGATIVE); - genericRecord.setField("fixed_field", FIXED_BYTES); - - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - - return GenericRowData.of( - StringData.fromString("row_id_value"), - false, - Integer.MAX_VALUE, - Long.MAX_VALUE, - Float.MAX_VALUE, - Double.MAX_VALUE, - StringData.fromString("str"), - DAYS_BTW_EPOC_AND_20220110, - HOUR_8_IN_MILLI, - // Although Avro logical type for timestamp fields are in micro seconds, - // AvroToRowDataConverters only looks for long value in milliseconds. - TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), - TimestampData.fromEpochMillis(JODA_DATETIME_20220110.getMillis()), - uuidBytes, - binaryBytes, - DecimalData.fromBigDecimal(BIG_DECIMAL_NEGATIVE, 9, 2), - FIXED_BYTES); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", new Utf8("row_id_value")); - genericRecord.put("boolean_field", false); - genericRecord.put("int_field", Integer.MAX_VALUE); - genericRecord.put("long_field", Long.MAX_VALUE); - genericRecord.put("float_field", Float.MAX_VALUE); - genericRecord.put("double_field", Double.MAX_VALUE); - genericRecord.put("string_field", new Utf8("str")); - - genericRecord.put("date_field", DAYS_BTW_EPOC_AND_20220110); - genericRecord.put("time_field", HOUR_8_IN_MILLI); - // Although Avro logical type for timestamp fields are in micro seconds, - // AvroToRowDataConverters only looks for long value in milliseconds. - genericRecord.put("ts_with_zone_field", JODA_DATETIME_20220110.getMillis()); - genericRecord.put("ts_without_zone_field", JODA_DATETIME_20220110.getMillis()); - - byte[] uuidBytes = new byte[16]; - for (int i = 0; i < 16; ++i) { - uuidBytes[i] = (byte) i; - } - genericRecord.put("uuid_field", ByteBuffer.wrap(uuidBytes)); - - byte[] binaryBytes = new byte[7]; - for (int i = 0; i < 7; ++i) { - binaryBytes[i] = (byte) i; - } - genericRecord.put("binary_field", ByteBuffer.wrap(binaryBytes)); - - BigDecimal bigDecimal = new BigDecimal("-1.50"); - // unscaledValue().toByteArray() is to match the behavior of RowDataToAvroConverters from - // Flink for decimal type - genericRecord.put("decimal_field", ByteBuffer.wrap(bigDecimal.unscaledValue().toByteArray())); - - genericRecord.put("fixed_field", ByteBuffer.wrap(FIXED_BYTES)); - - return genericRecord; - } - } - - public static class StructOfPrimitive implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_primitive", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required(102, "name", Types.StringType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_primitive").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("name", "Jane"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_primitive", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of(1, StringData.fromString("Jane"))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_primitive").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("name", "Jane"); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_primitive", struct); - return genericRecord; - } - } - - public static class StructOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_array", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, "names", Types.ListType.ofRequired(201, Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_array").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("names", Arrays.asList("Jane", "Joe")); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_array", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - StringData[] names = {StringData.fromString("Jane"), StringData.fromString("Joe")}; - return GenericRowData.of( - StringData.fromString("row_id_value"), GenericRowData.of(1, new GenericArrayData(names))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_array").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("names", Arrays.asList("Jane", "Joe")); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_array", struct); - return genericRecord; - } - } - - public static class StructOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_map", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, - "names", - Types.MapType.ofRequired( - 201, 202, Types.StringType.get(), Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_map").type().asStructType().fields()); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("names", ImmutableMap.of("Jane", "female", "Joe", "male")); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_map", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of( - 1, - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), - StringData.fromString("female"), - StringData.fromString("Joe"), - StringData.fromString("male"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_map").schema(); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("names", ImmutableMap.of("Jane", new Utf8("female"), "Joe", new Utf8("male"))); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_map", struct); - return genericRecord; - } - } - - public static class StructOfStruct implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "struct_of_struct", - Types.StructType.of( - required(101, "id", Types.IntegerType.get()), - required( - 102, - "person_struct", - Types.StructType.of( - Types.NestedField.required(201, "name", Types.StringType.get()), - Types.NestedField.required(202, "address", Types.StringType.get())))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - Schema structSchema = - new Schema(icebergSchema.findField("struct_of_struct").type().asStructType().fields()); - Schema personSchema = - new Schema(structSchema.findField("person_struct").type().asStructType().fields()); - GenericRecord person = GenericRecord.create(personSchema); - person.setField("name", "Jane"); - person.setField("address", "Apple Park"); - GenericRecord struct = GenericRecord.create(structSchema); - struct.setField("id", 1); - struct.setField("person_struct", person); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("struct_of_struct", struct); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - GenericRowData.of( - 1, - GenericRowData.of( - StringData.fromString("Jane"), StringData.fromString("Apple Park")))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.Schema structSchema = avroSchema.getField("struct_of_struct").schema(); - org.apache.avro.Schema personSchema = structSchema.getField("person_struct").schema(); - org.apache.avro.generic.GenericRecord person = new GenericData.Record(personSchema); - person.put("name", "Jane"); - person.put("address", "Apple Park"); - org.apache.avro.generic.GenericRecord struct = new GenericData.Record(structSchema); - struct.put("id", 1); - struct.put("person_struct", person); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("struct_of_struct", struct); - return genericRecord; - } - } - - public static class ArrayOfPrimitive implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, "array_of_int", Types.ListType.ofOptional(101, Types.IntegerType.get()))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("array_of_int", Arrays.asList(1, 2, 3)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - Integer[] arr = {1, 2, 3}; - return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(arr)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("array_of_int", Arrays.asList(1, 2, 3)); - return genericRecord; - } - } - - public static class ArrayOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "array_of_array", - Types.ListType.ofRequired( - 101, Types.ListType.ofRequired(201, Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - // non-primitive - Integer[] array1 = {1, 2, 3}; - Integer[] array2 = {4, 5, 6}; - GenericArrayData[] arrayOfArrays = { - new GenericArrayData(array1), new GenericArrayData(array2) - }; - return GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(arrayOfArrays)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "array_of_array", Arrays.asList(Arrays.asList(1, 2, 3), Arrays.asList(4, 5, 6))); - return genericRecord; - } - } - - public static class ArrayOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "array_of_map", - Types.ListType.ofRequired( - 101, - Types.MapType.ofRequired( - 201, 202, Types.StringType.get(), Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "array_of_map", - Arrays.asList( - ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - GenericMapData[] array = { - new GenericMapData( - ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2)), - new GenericMapData( - ImmutableMap.of(StringData.fromString("Alice"), 3, StringData.fromString("Bob"), 4)) - }; - return GenericRowData.of(StringData.fromString("row_id_value"), new GenericArrayData(array)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "array_of_map", - Arrays.asList( - ImmutableMap.of("Jane", 1, "Joe", 2), ImmutableMap.of("Alice", 3, "Bob", 4))); - return genericRecord; - } - } - - public static class ArrayOfStruct implements DataGenerator { - private final Types.StructType structType = - Types.StructType.of( - required(201, "id", Types.IntegerType.get()), - required(202, "name", Types.StringType.get())); - private final Schema structIcebergSchema = new Schema(structType.fields()); - private final org.apache.avro.Schema structAvroSchema = - AvroSchemaUtil.convert(structIcebergSchema, "struct"); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, "array_of_struct", Types.ListType.ofRequired(101, structType))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord struct1 = GenericRecord.create(structIcebergSchema); - struct1.setField("id", 1); - struct1.setField("name", "Jane"); - GenericRecord struct2 = GenericRecord.create(structIcebergSchema); - struct2.setField("id", 2); - struct2.setField("name", "Joe"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("array_of_struct", Arrays.asList(struct1, struct2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - GenericRowData[] structArray = { - GenericRowData.of(1, StringData.fromString("Jane")), - GenericRowData.of(2, StringData.fromString("Joe")) - }; - return GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(structArray)); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); - struct1.put("id", 1); - struct1.put("name", "Jane"); - org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); - struct2.put("id", 2); - struct2.put("name", "Joe"); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("array_of_struct", Arrays.asList(struct1, struct2)); - return genericRecord; - } - } - - public static class MapOfPrimitives implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, - "map_of_primitives", - Types.MapType.ofRequired( - 101, 102, Types.StringType.get(), Types.IntegerType.get()))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(StringData.fromString("Jane"), 1, StringData.fromString("Joe"), 2))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put("map_of_primitives", ImmutableMap.of("Jane", 1, "Joe", 2)); - return genericRecord; - } - } - - public static class MapOfArray implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_array", - Types.MapType.ofRequired( - 101, - 102, - Types.StringType.get(), - Types.ListType.ofRequired(201, Types.IntegerType.get())))); - - private final RowType rowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return rowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_array", - ImmutableMap.of( - "Jane", Arrays.asList(1, 2, 3), - "Joe", Arrays.asList(4, 5, 6))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - Integer[] janeArray = {1, 2, 3}; - Integer[] joeArray = {4, 5, 6}; - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), - new GenericArrayData(janeArray), - StringData.fromString("Joe"), - new GenericArrayData(joeArray)))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "map_of_array", - ImmutableMap.of( - "Jane", Arrays.asList(1, 2, 3), - "Joe", Arrays.asList(4, 5, 6))); - return genericRecord; - } - } - - public static class MapOfMap implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_map", - Types.MapType.ofRequired( - 101, - 102, - Types.StringType.get(), - Types.MapType.ofRequired( - 301, 302, Types.StringType.get(), Types.IntegerType.get())))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - private final org.apache.avro.Schema avroSchema = - AvroSchemaUtil.convert(icebergSchema, "table"); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_map", - ImmutableMap.of( - "female", ImmutableMap.of("Jane", 1, "Alice", 2), - "male", ImmutableMap.of("Joe", 3, "Bob", 4))); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("female"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Jane"), 1, StringData.fromString("Alice"), 2)), - StringData.fromString("male"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("Joe"), 3, StringData.fromString("Bob"), 4))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", "row_id_value"); - genericRecord.put( - "map_of_map", - ImmutableMap.of( - "female", ImmutableMap.of("Jane", 1, "Alice", 2), - "male", ImmutableMap.of("Joe", 3, "Bob", 4))); - return genericRecord; - } - } - - public static class MapOfStruct implements DataGenerator { - private org.apache.avro.Schema createAvroSchemaIdField() { - org.apache.avro.Schema schema = SchemaBuilder.builder().intType(); - // this is needed to match the converter generated schema props - schema.addProp("field-id", IntNode.valueOf(201)); - return schema; - } - - private org.apache.avro.Schema createAvroSchemaNameField() { - org.apache.avro.Schema schema = SchemaBuilder.builder().stringType(); - // this is needed to match the converter generated schema props - schema.addProp("field-id", IntNode.valueOf(202)); - return schema; - } - - private final Types.StructType structType = - Types.StructType.of( - required(201, "id", Types.IntegerType.get()), - required(202, "name", Types.StringType.get())); - private final Schema structIcebergSchema = new Schema(structType.fields()); - - private final org.apache.avro.Schema structAvroSchema = - SchemaBuilder.builder() - .record("struct") - .fields() - .name("id") - .type(createAvroSchemaIdField()) - .noDefault() - .name("name") - .type(createAvroSchemaNameField()) - .noDefault() - .endRecord(); - - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.required( - 2, - "map_of_struct", - Types.MapType.ofRequired(101, 102, Types.StringType.get(), structType))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - // Can't use AvroSchemaUtil.convert otherwise the nested schema will have generated name like - // `r102` not the specified name like `struct`. - org.apache.avro.Schema avroSchema = - SchemaBuilder.builder() - .record("table") - .fields() - .requiredString("row_id") - .name("map_of_struct") - .type(SchemaBuilder.builder().map().values(structAvroSchema)) - .noDefault() - .endRecord(); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - return avroSchema; - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - GenericRecord struct1 = GenericRecord.create(structIcebergSchema); - struct1.setField("id", 1); - struct1.setField("name", "Jane"); - GenericRecord struct2 = GenericRecord.create(structIcebergSchema); - struct2.setField("id", 2); - struct2.setField("name", "Joe"); - GenericRecord genericRecord = GenericRecord.create(icebergSchema); - genericRecord.setField("row_id", "row_id_value"); - genericRecord.setField( - "map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); - return genericRecord; - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - StringData.fromString("struct1"), - GenericRowData.of(1, StringData.fromString("Jane")), - StringData.fromString("struct2"), - GenericRowData.of(2, StringData.fromString("Joe"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - org.apache.avro.generic.GenericRecord struct1 = new GenericData.Record(structAvroSchema); - struct1.put("id", 1); - struct1.put("name", new Utf8("Jane")); - org.apache.avro.generic.GenericRecord struct2 = new GenericData.Record(structAvroSchema); - struct2.put("id", 2); - struct2.put("name", new Utf8("Joe")); - org.apache.avro.generic.GenericRecord genericRecord = new GenericData.Record(avroSchema); - genericRecord.put("row_id", new Utf8("row_id_value")); - genericRecord.put("map_of_struct", ImmutableMap.of("struct1", struct1, "struct2", struct2)); - return genericRecord; - } - } - - public static class MapOfStructStruct implements DataGenerator { - private final Schema icebergSchema = - new Schema( - Types.NestedField.required(1, "row_id", Types.StringType.get()), - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get()), - Types.NestedField.optional(202, "keyData", Types.StringType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()), - Types.NestedField.optional(204, "valueData", Types.StringType.get()))))); - - private final RowType flinkRowType = FlinkSchemaUtil.convert(icebergSchema); - - @Override - public Schema icebergSchema() { - return icebergSchema; - } - - @Override - public RowType flinkRowType() { - return flinkRowType; - } - - @Override - public org.apache.avro.Schema avroSchema() { - throw new UnsupportedOperationException( - "Not applicable as Avro Map only support string key type"); - } - - @Override - public GenericRecord generateIcebergGenericRecord() { - throw new UnsupportedOperationException("Not implemented yet"); - } - - @Override - public GenericRowData generateFlinkRowData() { - return GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of( - GenericRowData.of(1L, StringData.fromString("key_data")), - GenericRowData.of(1L, StringData.fromString("value_data"))))); - } - - @Override - public org.apache.avro.generic.GenericRecord generateAvroGenericRecord() { - throw new UnsupportedOperationException("Avro Map only support string key type"); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java deleted file mode 100644 index fd5c6b76b683..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopCatalogExtension.java +++ /dev/null @@ -1,105 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.UUID; -import org.apache.commons.io.FileUtils; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.extension.AfterAllCallback; -import org.junit.jupiter.api.extension.AfterEachCallback; -import org.junit.jupiter.api.extension.BeforeAllCallback; -import org.junit.jupiter.api.extension.BeforeEachCallback; -import org.junit.jupiter.api.extension.ExtensionContext; - -public class HadoopCatalogExtension - implements BeforeAllCallback, BeforeEachCallback, AfterAllCallback, AfterEachCallback { - protected final String database; - protected final String tableName; - - protected Path temporaryFolder; - protected Catalog catalog; - protected CatalogLoader catalogLoader; - protected String warehouse; - protected TableLoader tableLoader; - - public HadoopCatalogExtension(String database, String tableName) { - this.database = database; - this.tableName = tableName; - } - - @Override - public void beforeAll(ExtensionContext context) throws Exception { - this.temporaryFolder = Files.createTempDirectory("junit5_hadoop_catalog-"); - } - - @Override - public void afterAll(ExtensionContext context) throws Exception { - FileUtils.deleteDirectory(temporaryFolder.toFile()); - } - - @Override - public void beforeEach(ExtensionContext context) throws Exception { - assertThat(temporaryFolder).exists().isDirectory(); - this.warehouse = "file:" + temporaryFolder + "/" + UUID.randomUUID(); - this.catalogLoader = - CatalogLoader.hadoop( - "hadoop", - new Configuration(), - ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - this.catalog = catalogLoader.loadCatalog(); - this.tableLoader = - TableLoader.fromCatalog(catalogLoader, TableIdentifier.of(database, tableName)); - } - - @Override - public void afterEach(ExtensionContext context) throws Exception { - try { - catalog.dropTable(TableIdentifier.of(database, tableName)); - ((HadoopCatalog) catalog).close(); - tableLoader.close(); - } catch (Exception e) { - throw new RuntimeException("Failed to close catalog resource"); - } - } - - public TableLoader tableLoader() { - return tableLoader; - } - - public Catalog catalog() { - return catalog; - } - - public CatalogLoader catalogLoader() { - return catalogLoader; - } - - public String warehouse() { - return warehouse; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java deleted file mode 100644 index dc6ef400a4a9..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/HadoopTableExtension.java +++ /dev/null @@ -1,59 +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; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.junit.jupiter.api.extension.ExtensionContext; - -public class HadoopTableExtension extends HadoopCatalogExtension { - private final Schema schema; - private final PartitionSpec partitionSpec; - - private Table table; - - public HadoopTableExtension(String database, String tableName, Schema schema) { - this(database, tableName, schema, null); - } - - public HadoopTableExtension( - String database, String tableName, Schema schema, PartitionSpec partitionSpec) { - super(database, tableName); - this.schema = schema; - this.partitionSpec = partitionSpec; - } - - @Override - public void beforeEach(ExtensionContext context) throws Exception { - super.beforeEach(context); - if (partitionSpec == null) { - this.table = catalog.createTable(TableIdentifier.of(database, tableName), schema); - } else { - this.table = - catalog.createTable(TableIdentifier.of(database, tableName), schema, partitionSpec); - } - tableLoader.open(); - } - - public Table table() { - return table; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java deleted file mode 100644 index d2e086aa448e..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/MiniFlinkClusterExtension.java +++ /dev/null @@ -1,67 +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; - -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.test.junit5.MiniClusterExtension; - -public class MiniFlinkClusterExtension { - - private static final int DEFAULT_TM_NUM = 1; - private static final int DEFAULT_PARALLELISM = 4; - - public static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = - new Configuration() - // disable classloader check as Avro may cache class/object in the serializers. - .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - - private MiniFlinkClusterExtension() {} - - /** - * It will start a mini cluster with classloader.check-leaked-classloader=false, so that we won't - * break the unit tests because of the class loader leak issue. In our iceberg integration tests, - * there're some that will assert the results after finished the flink jobs, so actually we may - * access the class loader that has been closed by the flink task managers if we enable the switch - * classloader.check-leaked-classloader by default. - */ - public static MiniClusterExtension createWithClassloaderCheckDisabled() { - return new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - } - - public static MiniClusterExtension createWithClassloaderCheckDisabled( - InMemoryReporter inMemoryReporter) { - Configuration configuration = new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG); - inMemoryReporter.addToConfiguration(configuration); - - return new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(DEFAULT_TM_NUM) - .setNumberSlotsPerTaskManager(DEFAULT_PARALLELISM) - .setConfiguration(configuration) - .build()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.java deleted file mode 100644 index e532fb62615c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/RowDataConverter.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; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.Arrays; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.TimeUnit; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.data.TimestampData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; - -public class RowDataConverter { - private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); - private static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); - - private RowDataConverter() {} - - public static RowData convert(Schema iSchema, Record record) { - return convert(iSchema.asStruct(), record); - } - - private static RowData convert(Types.StructType struct, Record record) { - GenericRowData rowData = new GenericRowData(struct.fields().size()); - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Types.NestedField field = fields.get(i); - - Type fieldType = field.type(); - rowData.setField(i, convert(fieldType, record.get(i))); - } - return rowData; - } - - private static Object convert(Type type, Object object) { - if (object == null) { - return null; - } - - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - case FIXED: - return object; - case DATE: - return (int) ChronoUnit.DAYS.between(EPOCH_DAY, (LocalDate) object); - case TIME: - // Iceberg's time is in microseconds, while flink's time is in milliseconds. - LocalTime localTime = (LocalTime) object; - return (int) TimeUnit.NANOSECONDS.toMillis(localTime.toNanoOfDay()); - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - return TimestampData.fromInstant(((OffsetDateTime) object).toInstant()); - } else { - return TimestampData.fromLocalDateTime((LocalDateTime) object); - } - case STRING: - return StringData.fromString((String) object); - case UUID: - UUID uuid = (UUID) object; - ByteBuffer bb = ByteBuffer.allocate(16); - bb.putLong(uuid.getMostSignificantBits()); - bb.putLong(uuid.getLeastSignificantBits()); - return bb.array(); - case BINARY: - ByteBuffer buffer = (ByteBuffer) object; - return Arrays.copyOfRange( - buffer.array(), - buffer.arrayOffset() + buffer.position(), - buffer.arrayOffset() + buffer.remaining()); - case DECIMAL: - Types.DecimalType decimalType = (Types.DecimalType) type; - return DecimalData.fromBigDecimal( - (BigDecimal) object, decimalType.precision(), decimalType.scale()); - case STRUCT: - return convert(type.asStructType(), (Record) object); - case LIST: - List list = (List) object; - Object[] convertedArray = new Object[list.size()]; - for (int i = 0; i < convertedArray.length; i++) { - convertedArray[i] = convert(type.asListType().elementType(), list.get(i)); - } - return new GenericArrayData(convertedArray); - case MAP: - Map convertedMap = Maps.newLinkedHashMap(); - Map map = (Map) object; - for (Map.Entry entry : map.entrySet()) { - convertedMap.put( - convert(type.asMapType().keyType(), entry.getKey()), - convert(type.asMapType().valueType(), entry.getValue())); - } - return new GenericMapData(convertedMap); - default: - throw new UnsupportedOperationException("Not a supported type: " + type); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java deleted file mode 100644 index 1767f774922a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/SimpleDataUtil.java +++ /dev/null @@ -1,439 +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; - -import static org.apache.iceberg.hadoop.HadoopOutputFile.fromPath; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.time.Duration; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; -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.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableScan; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.InternalRecordWrapper; -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.EncryptedOutputFile; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.hadoop.HadoopInputFile; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -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.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.Pair; -import org.apache.iceberg.util.StructLikeSet; -import org.apache.iceberg.util.StructLikeWrapper; -import org.awaitility.Awaitility; - -public class SimpleDataUtil { - - private SimpleDataUtil() {} - - public static final Schema SCHEMA = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())); - - public static final TableSchema FLINK_SCHEMA = - TableSchema.builder().field("id", DataTypes.INT()).field("data", DataTypes.STRING()).build(); - - public static final RowType ROW_TYPE = (RowType) FLINK_SCHEMA.toRowDataType().getLogicalType(); - - public static final Record RECORD = GenericRecord.create(SCHEMA); - - public static Table createTable( - String path, Map properties, boolean partitioned) { - PartitionSpec spec; - if (partitioned) { - spec = PartitionSpec.builderFor(SCHEMA).identity("data").build(); - } else { - spec = PartitionSpec.unpartitioned(); - } - return new HadoopTables().create(SCHEMA, spec, properties, path); - } - - public static Record createRecord(Integer id, String data) { - Record record = RECORD.copy(); - record.setField("id", id); - record.setField("data", data); - return record; - } - - public static RowData createRowData(Integer id, String data) { - return GenericRowData.of(id, StringData.fromString(data)); - } - - public static RowData createInsert(Integer id, String data) { - return GenericRowData.ofKind(RowKind.INSERT, id, StringData.fromString(data)); - } - - public static RowData createDelete(Integer id, String data) { - return GenericRowData.ofKind(RowKind.DELETE, id, StringData.fromString(data)); - } - - public static RowData createUpdateBefore(Integer id, String data) { - return GenericRowData.ofKind(RowKind.UPDATE_BEFORE, id, StringData.fromString(data)); - } - - public static RowData createUpdateAfter(Integer id, String data) { - return GenericRowData.ofKind(RowKind.UPDATE_AFTER, id, StringData.fromString(data)); - } - - public static DataFile writeFile( - Table table, - Schema schema, - PartitionSpec spec, - Configuration conf, - String location, - String filename, - List rows) - throws IOException { - return writeFile(table, schema, spec, conf, location, filename, rows, null); - } - - /** Write the list of {@link RowData} to the given path and with the given partition data */ - public static DataFile writeFile( - Table table, - Schema schema, - PartitionSpec spec, - Configuration conf, - String location, - String filename, - List rows, - StructLike partition) - throws IOException { - Path path = new Path(location, filename); - FileFormat fileFormat = FileFormat.fromFileName(filename); - Preconditions.checkNotNull(fileFormat, "Cannot determine format for file: %s", filename); - - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - FileAppenderFactory appenderFactory = - new FlinkAppenderFactory( - table, schema, flinkSchema, ImmutableMap.of(), spec, null, null, null); - - FileAppender appender = appenderFactory.newAppender(fromPath(path, conf), fileFormat); - try (FileAppender closeableAppender = appender) { - closeableAppender.addAll(rows); - } - - DataFiles.Builder builder = - DataFiles.builder(spec) - .withInputFile(HadoopInputFile.fromPath(path, conf)) - .withMetrics(appender.metrics()); - - if (partition != null) { - builder = builder.withPartition(partition); - } - - return builder.build(); - } - - public static DeleteFile writeEqDeleteFile( - Table table, - FileFormat format, - String filename, - FileAppenderFactory appenderFactory, - List deletes) - throws IOException { - EncryptedOutputFile outputFile = - table - .encryption() - .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); - - EqualityDeleteWriter eqWriter = - appenderFactory.newEqDeleteWriter(outputFile, format, null); - try (EqualityDeleteWriter writer = eqWriter) { - writer.write(deletes); - } - return eqWriter.toDeleteFile(); - } - - public static DeleteFile writePosDeleteFile( - Table table, - FileFormat format, - String filename, - FileAppenderFactory appenderFactory, - List> positions) - throws IOException { - EncryptedOutputFile outputFile = - table - .encryption() - .encrypt(fromPath(new Path(table.location(), filename), new Configuration())); - - PositionDeleteWriter posWriter = - appenderFactory.newPosDeleteWriter(outputFile, format, null); - PositionDelete posDelete = PositionDelete.create(); - try (PositionDeleteWriter writer = posWriter) { - for (Pair p : positions) { - writer.write(posDelete.set(p.first(), p.second(), null)); - } - } - return posWriter.toDeleteFile(); - } - - private static List convertToRecords(List rows) { - List records = Lists.newArrayList(); - for (RowData row : rows) { - Integer id = row.isNullAt(0) ? null : row.getInt(0); - String data = row.isNullAt(1) ? null : row.getString(1).toString(); - records.add(createRecord(id, data)); - } - return records; - } - - public static void assertTableRows(String tablePath, List expected, String branch) - throws IOException { - assertTableRecords(tablePath, convertToRecords(expected), branch); - } - - public static void assertTableRows(Table table, List expected) throws IOException { - assertTableRecords(table, convertToRecords(expected), SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRows(Table table, List expected, String branch) - throws IOException { - assertTableRecords(table, convertToRecords(expected), branch); - } - - /** Get all rows for a table */ - public static List tableRecords(Table table) throws IOException { - table.refresh(); - List records = Lists.newArrayList(); - try (CloseableIterable iterable = IcebergGenerics.read(table).build()) { - for (Record record : iterable) { - records.add(record); - } - } - return records; - } - - public static boolean equalsRecords(List expected, List actual, Schema schema) { - if (expected.size() != actual.size()) { - return false; - } - Types.StructType type = schema.asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - StructLikeSet actualSet = StructLikeSet.create(type); - actualSet.addAll(actual); - return expectedSet.equals(actualSet); - } - - public static void assertRecordsEqual(List expected, List actual, Schema schema) { - assertThat(actual).hasSameSizeAs(expected); - Types.StructType type = schema.asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - StructLikeSet actualSet = StructLikeSet.create(type); - actualSet.addAll(actual); - assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); - } - - /** - * Assert table contains the expected list of records after waiting up to the configured {@code - * timeout} - */ - public static void assertTableRecords(Table table, List expected, Duration timeout) { - Awaitility.await("expected list of records should be produced") - .atMost(timeout) - .untilAsserted(() -> assertRecordsEqual(expected, tableRecords(table), table.schema())); - } - - public static void assertTableRecords(Table table, List expected) throws IOException { - assertTableRecords(table, expected, SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRecords(Table table, List expected, String branch) - throws IOException { - table.refresh(); - Snapshot snapshot = latestSnapshot(table, branch); - - if (snapshot == null) { - assertThat(expected).isEmpty(); - return; - } - - Types.StructType type = table.schema().asStruct(); - StructLikeSet expectedSet = StructLikeSet.create(type); - expectedSet.addAll(expected); - - try (CloseableIterable iterable = - IcebergGenerics.read(table).useSnapshot(snapshot.snapshotId()).build()) { - StructLikeSet actualSet = StructLikeSet.create(type); - - for (Record record : iterable) { - actualSet.add(record); - } - - assertThat(actualSet).containsExactlyInAnyOrderElementsOf(expectedSet); - } - } - - // Returns the latest snapshot of the given branch in the table - public static Snapshot latestSnapshot(Table table, String branch) { - // For the main branch, currentSnapshot() is used to validate that the API behavior has - // not changed since that was the API used for validation prior to addition of branches. - if (branch.equals(SnapshotRef.MAIN_BRANCH)) { - return table.currentSnapshot(); - } - - return table.snapshot(branch); - } - - public static void assertTableRecords(String tablePath, List expected) - throws IOException { - Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); - assertTableRecords(new HadoopTables().load(tablePath), expected, SnapshotRef.MAIN_BRANCH); - } - - public static void assertTableRecords(String tablePath, List expected, String branch) - throws IOException { - Preconditions.checkArgument(expected != null, "expected records shouldn't be null"); - assertTableRecords(new HadoopTables().load(tablePath), expected, branch); - } - - public static StructLikeSet expectedRowSet(Table table, Record... records) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); - for (Record record : records) { - set.add(wrapper.copyFor(record)); - } - return set; - } - - public static StructLikeSet actualRowSet(Table table, String... columns) throws IOException { - return actualRowSet(table, null, columns); - } - - public static StructLikeSet actualRowSet(Table table, Long snapshotId, String... columns) - throws IOException { - table.refresh(); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - InternalRecordWrapper wrapper = new InternalRecordWrapper(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table) - .useSnapshot(snapshotId == null ? table.currentSnapshot().snapshotId() : snapshotId) - .select(columns) - .build()) { - reader.forEach(record -> set.add(wrapper.copyFor(record))); - } - return set; - } - - public static List partitionDataFiles(Table table, Map partitionValues) - throws IOException { - table.refresh(); - Types.StructType partitionType = table.spec().partitionType(); - - Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); - StructLikeWrapper expectedWrapper = - StructLikeWrapper.forType(partitionType).set(partitionRecord); - - List dataFiles = Lists.newArrayList(); - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - StructLikeWrapper wrapper = - StructLikeWrapper.forType(partitionType).set(scanTask.file().partition()); - - if (expectedWrapper.equals(wrapper)) { - dataFiles.add(scanTask.file()); - } - } - } - - return dataFiles; - } - - public static Map> snapshotToDataFiles(Table table) throws IOException { - table.refresh(); - - Map> result = Maps.newHashMap(); - Snapshot current = table.currentSnapshot(); - while (current != null) { - TableScan tableScan = table.newScan(); - if (current.parentId() != null) { - // Collect the data files that was added only in current snapshot. - tableScan = tableScan.appendsBetween(current.parentId(), current.snapshotId()); - } else { - // Collect the data files that was added in the oldest snapshot. - tableScan = tableScan.useSnapshot(current.snapshotId()); - } - try (CloseableIterable scanTasks = tableScan.planFiles()) { - result.put( - current.snapshotId(), - ImmutableList.copyOf(Iterables.transform(scanTasks, FileScanTask::file))); - } - - // Continue to traverse the parent snapshot if exists. - if (current.parentId() == null) { - break; - } - // Iterate to the parent snapshot. - current = table.snapshot(current.parentId()); - } - return result; - } - - public static List matchingPartitions( - List dataFiles, PartitionSpec partitionSpec, Map partitionValues) { - Types.StructType partitionType = partitionSpec.partitionType(); - Record partitionRecord = GenericRecord.create(partitionType).copy(partitionValues); - StructLikeWrapper expected = StructLikeWrapper.forType(partitionType).set(partitionRecord); - return dataFiles.stream() - .filter( - df -> { - StructLikeWrapper wrapper = - StructLikeWrapper.forType(partitionType).set(df.partition()); - return wrapper.equals(expected); - }) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java deleted file mode 100644 index a74226092f38..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ /dev/null @@ -1,128 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.nio.file.Path; -import java.util.List; -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.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public abstract class TestBase extends TestBaseUtils { - - @RegisterExtension - public static MiniClusterExtension miniClusterExtension = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @TempDir protected Path temporaryDirectory; - - private static TestHiveMetastore metastore = null; - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - - private volatile TableEnvironment tEnv = null; - - @BeforeAll - public static void startMetastore() { - TestBase.metastore = new TestHiveMetastore(); - metastore.start(); - TestBase.hiveConf = metastore.hiveConf(); - TestBase.catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterAll - public static void stopMetastore() throws Exception { - metastore.stop(); - TestBase.catalog = null; - } - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = EnvironmentSettings.newInstance().inBatchMode().build(); - - TableEnvironment env = TableEnvironment.create(settings); - env.getConfig() - .getConfiguration() - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - tEnv = env; - } - } - } - return tEnv; - } - - protected static TableResult exec(TableEnvironment env, String query, Object... args) { - return env.executeSql(String.format(query, args)); - } - - protected TableResult exec(String query, Object... args) { - return exec(getTableEnv(), query, args); - } - - protected List sql(String query, Object... args) { - TableResult tableResult = exec(query, args); - try (CloseableIterator iter = tableResult.collect()) { - return Lists.newArrayList(iter); - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - protected void assertSameElements(Iterable expected, Iterable actual) { - assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); - } - - protected void assertSameElements(String message, Iterable expected, Iterable actual) { - assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); - } - - /** - * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not - * use the current catalog before dropping it. This method switches to the 'default_catalog' and - * drops the one requested. - * - * @param catalogName The catalog to drop - * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog - */ - protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); - sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java deleted file mode 100644 index e8f65921c19a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogLoader.java +++ /dev/null @@ -1,116 +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; - -import static org.apache.iceberg.CatalogProperties.URI; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.entry; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.util.Map; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -/** Test for {@link CatalogLoader}. */ -public class TestCatalogLoader extends TestBase { - - private static File warehouse = null; - private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); - private static final Schema SCHEMA = - new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - - @BeforeAll - public static void createWarehouse() throws IOException { - warehouse = File.createTempFile("warehouse", null); - assertThat(warehouse.delete()).isTrue(); - hiveConf.set("my_key", "my_value"); - } - - @AfterAll - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); - } - } - - @Test - public void testHadoopCatalogLoader() throws IOException, ClassNotFoundException { - Map properties = Maps.newHashMap(); - properties.put(CatalogProperties.WAREHOUSE_LOCATION, "file:" + warehouse); - CatalogLoader loader = CatalogLoader.hadoop("my_catalog", hiveConf, properties); - validateCatalogLoader(loader); - } - - @Test - public void testHiveCatalogLoader() throws IOException, ClassNotFoundException { - CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - validateCatalogLoader(loader); - } - - @Test - public void testRESTCatalogLoader() { - Map properties = Maps.newHashMap(); - properties.put(URI, "http://localhost/"); - CatalogLoader.rest("my_catalog", hiveConf, Maps.newHashMap()); - } - - private static void validateCatalogLoader(CatalogLoader loader) - throws IOException, ClassNotFoundException { - Table table = javaSerAndDeSer(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); - validateHadoopConf(table); - } - - private static void validateHadoopConf(Table table) { - FileIO io = table.io(); - assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); - HadoopFileIO hadoopIO = (HadoopFileIO) io; - assertThat(hadoopIO.conf()).contains(entry("my_key", "my_value")); - } - - @SuppressWarnings("unchecked") - private static T javaSerAndDeSer(T object) throws IOException, ClassNotFoundException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(object); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - return (T) in.readObject(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java deleted file mode 100644 index f719c7bc0001..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestCatalogTableLoader.java +++ /dev/null @@ -1,113 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.Test; - -/** Test for {@link TableLoader}. */ -public class TestCatalogTableLoader extends TestBase { - - private static File warehouse = null; - private static final TableIdentifier IDENTIFIER = TableIdentifier.of("default", "my_table"); - private static final Schema SCHEMA = - new Schema(Types.NestedField.required(1, "f1", Types.StringType.get())); - - @BeforeAll - public static void createWarehouse() throws IOException { - warehouse = File.createTempFile("warehouse", null); - assertThat(warehouse.delete()).isTrue(); - hiveConf.set("my_key", "my_value"); - } - - @AfterAll - public static void dropWarehouse() throws IOException { - if (warehouse != null && warehouse.exists()) { - Path warehousePath = new Path(warehouse.getAbsolutePath()); - FileSystem fs = warehousePath.getFileSystem(hiveConf); - assertThat(fs.delete(warehousePath, true)).as("Failed to delete " + warehousePath).isTrue(); - } - } - - @Test - public void testHadoopTableLoader() throws IOException, ClassNotFoundException { - String location = "file:" + warehouse + "/my_table"; - new HadoopTables(hiveConf).create(SCHEMA, location); - validateTableLoader(TableLoader.fromHadoopTable(location, hiveConf)); - } - - @Test - public void testHiveCatalogTableLoader() throws IOException, ClassNotFoundException { - CatalogLoader loader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - javaSerdes(loader).loadCatalog().createTable(IDENTIFIER, SCHEMA); - - CatalogLoader catalogLoader = CatalogLoader.hive("my_catalog", hiveConf, Maps.newHashMap()); - validateTableLoader(TableLoader.fromCatalog(catalogLoader, IDENTIFIER)); - } - - private static void validateTableLoader(TableLoader loader) - throws IOException, ClassNotFoundException { - TableLoader copied = javaSerdes(loader); - copied.open(); - try { - validateHadoopConf(copied.loadTable()); - } finally { - copied.close(); - } - } - - private static void validateHadoopConf(Table table) { - FileIO io = table.io(); - assertThat(io).as("FileIO should be a HadoopFileIO").isInstanceOf(HadoopFileIO.class); - HadoopFileIO hadoopIO = (HadoopFileIO) io; - assertThat(hadoopIO.conf().get("my_key")).isEqualTo("my_value"); - } - - @SuppressWarnings("unchecked") - private static T javaSerdes(T object) throws IOException, ClassNotFoundException { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(object); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - return (T) in.readObject(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java deleted file mode 100644 index 7fc6ab82490d..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestChangeLogTable.java +++ /dev/null @@ -1,296 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import org.apache.flink.types.Row; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.source.BoundedTableFactory; -import org.apache.iceberg.flink.source.ChangeLogTableTestBase; -import org.apache.iceberg.relocated.com.google.common.base.Joiner; -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.util.StructLikeSet; -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; - -/** - * In this test case, we mainly cover the impact of primary key selection, multiple operations - * within a single transaction, and multiple operations between different txn on the correctness of - * the data. - */ -@ExtendWith(ParameterizedTestExtension.class) -public class TestChangeLogTable extends ChangeLogTableTestBase { - private static final Configuration CONF = new Configuration(); - private static final String SOURCE_TABLE = "default_catalog.default_database.source_change_logs"; - - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String TABLE_NAME = "test_table"; - private static String warehouse; - - @Parameter private boolean partitioned; - - @Parameters(name = "PartitionedTable={0}") - public static Iterable parameters() { - return ImmutableList.of(new Object[] {true}, new Object[] {false}); - } - - @BeforeEach - public void before() throws IOException { - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - // Set the table.exec.sink.upsert-materialize=NONE, so that downstream operators will receive - // the - // records with the same order as the source operator, bypassing Flink's inferred shuffle. - getTableEnv().getConfig().set("table.exec.sink.upsert-materialize", "NONE"); - } - - @AfterEach - @Override - public void clean() { - sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - BoundedTableFactory.clearDataSets(); - } - - @TestTemplate - public void testSqlChangeLogOnIdKey() throws Exception { - List> inputRowsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(1, "bbb"), - insertRow(2, "aaa"), - deleteRow(2, "aaa"), - insertRow(2, "bbb")), - ImmutableList.of( - updateBeforeRow(2, "bbb"), - updateAfterRow(2, "ccc"), - deleteRow(2, "ccc"), - insertRow(2, "ddd")), - ImmutableList.of( - deleteRow(1, "bbb"), - insertRow(1, "ccc"), - deleteRow(1, "ccc"), - insertRow(1, "ddd"))); - - List> expectedRecordsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "bbb")), - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "ddd")), - ImmutableList.of(insertRow(1, "ddd"), insertRow(2, "ddd"))); - - testSqlChangeLog( - TABLE_NAME, ImmutableList.of("id"), inputRowsPerCheckpoint, expectedRecordsPerCheckpoint); - } - - @TestTemplate - public void testChangeLogOnDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(1, "bbb"), - insertRow(2, "aaa")), - ImmutableList.of( - updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), - ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "ccc"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa")), - ImmutableList.of(insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(1, "ccc"), - insertRow(2, "aaa"), - insertRow(2, "ccc"))); - - testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); - } - - @TestTemplate - public void testChangeLogOnIdDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(1, "bbb"), - insertRow(2, "aaa")), - ImmutableList.of( - updateBeforeRow(2, "aaa"), updateAfterRow(1, "ccc"), insertRow(1, "aaa")), - ImmutableList.of(deleteRow(1, "bbb"), insertRow(2, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "bbb"), insertRow(2, "aaa"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), insertRow(1, "bbb"), insertRow(1, "ccc"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(1, "ccc"), - insertRow(2, "aaa"), - insertRow(2, "bbb"))); - - testSqlChangeLog( - TABLE_NAME, ImmutableList.of("data", "id"), elementsPerCheckpoint, expectedRecords); - } - - @TestTemplate - public void testPureInsertOnIdKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), - ImmutableList.of(insertRow(3, "ccc"), insertRow(4, "ddd")), - ImmutableList.of(insertRow(5, "eee"), insertRow(6, "fff"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(insertRow(1, "aaa"), insertRow(2, "bbb")), - ImmutableList.of( - insertRow(1, "aaa"), insertRow(2, "bbb"), insertRow(3, "ccc"), insertRow(4, "ddd")), - ImmutableList.of( - insertRow(1, "aaa"), - insertRow(2, "bbb"), - insertRow(3, "ccc"), - insertRow(4, "ddd"), - insertRow(5, "eee"), - insertRow(6, "fff"))); - - testSqlChangeLog(TABLE_NAME, ImmutableList.of("data"), elementsPerCheckpoint, expectedRecords); - } - - private static Record record(int id, String data) { - return SimpleDataUtil.createRecord(id, data); - } - - private Table createTable(String tableName, List key, boolean isPartitioned) { - String partitionByCause = isPartitioned ? "PARTITIONED BY (data)" : ""; - sql( - "CREATE TABLE %s(id INT, data VARCHAR, PRIMARY KEY(%s) NOT ENFORCED) %s", - tableName, Joiner.on(',').join(key), partitionByCause); - - // Upgrade the iceberg table to format v2. - CatalogLoader loader = - CatalogLoader.hadoop( - "my_catalog", CONF, ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse)); - Table table = loader.loadCatalog().loadTable(TableIdentifier.of(DATABASE_NAME, TABLE_NAME)); - TableOperations ops = ((BaseTable) table).operations(); - TableMetadata meta = ops.current(); - ops.commit(meta, meta.upgradeToFormatVersion(2)); - - return table; - } - - private void testSqlChangeLog( - String tableName, - List key, - List> inputRowsPerCheckpoint, - List> expectedRecordsPerCheckpoint) - throws Exception { - String dataId = BoundedTableFactory.registerDataSet(inputRowsPerCheckpoint); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)).isEqualTo(listJoin(inputRowsPerCheckpoint)); - - Table table = createTable(tableName, key, partitioned); - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - table.refresh(); - List snapshots = findValidSnapshots(table); - int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - assertThat(snapshots) - .as("Should have the expected snapshot number") - .hasSameSizeAs(expectedRecordsPerCheckpoint); - - for (int i = 0; i < expectedSnapshotNum; i++) { - long snapshotId = snapshots.get(i).snapshotId(); - List expectedRows = expectedRecordsPerCheckpoint.get(i); - assertThat(actualRowSet(table, snapshotId)) - .as("Should have the expected records for the checkpoint#" + i) - .isEqualTo(expectedRowSet(table, expectedRows)); - } - - if (expectedSnapshotNum > 0) { - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in the final table") - .containsExactlyInAnyOrderElementsOf( - expectedRecordsPerCheckpoint.get(expectedSnapshotNum - 1)); - } - } - - private List findValidSnapshots(Table table) { - List validSnapshots = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests(table.io()).stream() - .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { - validSnapshots.add(snapshot); - } - } - return validSnapshots; - } - - private static StructLikeSet expectedRowSet(Table table, List rows) { - Record[] records = new Record[rows.size()]; - for (int i = 0; i < records.length; i++) { - records[i] = record((int) rows.get(i).getField(0), (String) rows.get(i).getField(1)); - } - return SimpleDataUtil.expectedRowSet(table, records); - } - - private static StructLikeSet actualRowSet(Table table, long snapshotId) throws IOException { - return SimpleDataUtil.actualRowSet(table, snapshotId, "*"); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java deleted file mode 100644 index 8992cbd75187..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestDataFileSerialization.java +++ /dev/null @@ -1,203 +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; - -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.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.util.Map; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestDataFileSerialization { - - private static final Schema DATE_SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - optional(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec PARTITION_SPEC = - PartitionSpec.builderFor(DATE_SCHEMA).identity("date").build(); - - private static final Map COLUMN_SIZES = Maps.newHashMap(); - private static final Map VALUE_COUNTS = Maps.newHashMap(); - private static final Map NULL_VALUE_COUNTS = Maps.newHashMap(); - private static final Map NAN_VALUE_COUNTS = Maps.newHashMap(); - private static final Map LOWER_BOUNDS = Maps.newHashMap(); - private static final Map UPPER_BOUNDS = Maps.newHashMap(); - - static { - COLUMN_SIZES.put(1, 2L); - COLUMN_SIZES.put(2, 3L); - VALUE_COUNTS.put(1, 5L); - VALUE_COUNTS.put(2, 3L); - VALUE_COUNTS.put(4, 2L); - NULL_VALUE_COUNTS.put(1, 0L); - NULL_VALUE_COUNTS.put(2, 2L); - NAN_VALUE_COUNTS.put(4, 1L); - LOWER_BOUNDS.put(1, longToBuffer(0L)); - UPPER_BOUNDS.put(1, longToBuffer(4L)); - } - - private static final Metrics METRICS = - new Metrics( - 5L, null, VALUE_COUNTS, NULL_VALUE_COUNTS, NAN_VALUE_COUNTS, LOWER_BOUNDS, UPPER_BOUNDS); - - private static final DataFile DATA_FILE = - DataFiles.builder(PARTITION_SPEC) - .withPath("/path/to/data-1.parquet") - .withFileSizeInBytes(1234) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withSplitOffsets(ImmutableList.of(4L)) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(34)) - .withSortOrder(SortOrder.unsorted()) - .build(); - - private static final DeleteFile POS_DELETE_FILE = - FileMetadata.deleteFileBuilder(PARTITION_SPEC) - .ofPositionDeletes() - .withPath("/path/to/pos-delete.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) - .withRecordCount(23) - .build(); - - private static final DeleteFile EQ_DELETE_FILE = - FileMetadata.deleteFileBuilder(PARTITION_SPEC) - .ofEqualityDeletes(2, 3) - .withPath("/path/to/equality-delete.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("date=2018-06-08") - .withMetrics(METRICS) - .withEncryptionKeyMetadata(ByteBuffer.allocate(4).putInt(35)) - .withRecordCount(23) - .withSortOrder(SortOrder.unsorted()) - .build(); - - @Test - public void testJavaSerialization() throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(DATA_FILE); - out.writeObject(DATA_FILE.copy()); - - out.writeObject(POS_DELETE_FILE); - out.writeObject(POS_DELETE_FILE.copy()); - - out.writeObject(EQ_DELETE_FILE); - out.writeObject(EQ_DELETE_FILE.copy()); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - assertThat(obj).as("Should be a DataFile").isInstanceOf(DataFile.class); - TestHelpers.assertEquals(DATA_FILE, (DataFile) obj); - } - - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - assertThat(obj).as("Should be a position DeleteFile").isInstanceOf(DeleteFile.class); - TestHelpers.assertEquals(POS_DELETE_FILE, (DeleteFile) obj); - } - - for (int i = 0; i < 2; i += 1) { - Object obj = in.readObject(); - assertThat(obj).as("Should be a equality DeleteFile").isInstanceOf(DeleteFile.class); - TestHelpers.assertEquals(EQ_DELETE_FILE, (DeleteFile) obj); - } - } - } - - @Test - public void testDataFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DataFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - kryo.serialize(DATA_FILE, outputView); - kryo.serialize(DATA_FILE.copy(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - DataFile dataFile1 = kryo.deserialize(inputView); - DataFile dataFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(DATA_FILE, dataFile1); - TestHelpers.assertEquals(DATA_FILE, dataFile2); - } - - @Test - public void testDeleteFileKryoSerialization() throws IOException { - KryoSerializer kryo = new KryoSerializer<>(DeleteFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - kryo.serialize(POS_DELETE_FILE, outputView); - kryo.serialize(POS_DELETE_FILE.copy(), outputView); - - kryo.serialize(EQ_DELETE_FILE, outputView); - kryo.serialize(EQ_DELETE_FILE.copy(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - - DeleteFile posDeleteFile1 = kryo.deserialize(inputView); - DeleteFile posDeleteFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile1); - TestHelpers.assertEquals(POS_DELETE_FILE, posDeleteFile2); - - DeleteFile eqDeleteFile1 = kryo.deserialize(inputView); - DeleteFile eqDeleteFile2 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile1); - TestHelpers.assertEquals(EQ_DELETE_FILE, eqDeleteFile2); - } - - private static ByteBuffer longToBuffer(long value) { - return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java deleted file mode 100644 index b9a7d5b1d589..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFixtures.java +++ /dev/null @@ -1,61 +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; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; - -public class TestFixtures { - - private TestFixtures() {} - - public static final Schema SCHEMA = - new Schema( - required(1, "data", Types.StringType.get()), - required(2, "id", Types.LongType.get()), - required(3, "dt", Types.StringType.get())); - - public static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("dt").bucket("id", 1).build(); - - public static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); - - public static final String DATABASE = "default"; - public static final String TABLE = "t"; - public static final String SINK_TABLE = "t_sink"; - - public static final TableIdentifier TABLE_IDENTIFIER = TableIdentifier.of(DATABASE, TABLE); - public static final TableIdentifier SINK_TABLE_IDENTIFIER = - TableIdentifier.of(DATABASE, SINK_TABLE); - - public static final Schema TS_SCHEMA = - new Schema( - required(1, "ts", Types.TimestampType.withoutZone()), - required(2, "str", Types.StringType.get())); - - public static final PartitionSpec TS_SPEC = - PartitionSpec.builderFor(TS_SCHEMA).hour("ts").build(); - - public static final RowType TS_ROW_TYPE = FlinkSchemaUtil.convert(TS_SCHEMA); -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java deleted file mode 100644 index 70c8043f8fbb..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkAnonymousTable.java +++ /dev/null @@ -1,65 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.nio.file.Files; -import java.util.concurrent.TimeUnit; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Schema; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableDescriptor; -import org.apache.flink.table.api.TableEnvironment; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; - -public class TestFlinkAnonymousTable extends TestBase { - - @Test - public void testWriteAnonymousTable() throws Exception { - File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); - TableEnvironment tEnv = getTableEnv(); - Table table = - tEnv.from( - TableDescriptor.forConnector("datagen") - .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) - .option("number-of-rows", "3") - .build()); - - TableDescriptor descriptor = - TableDescriptor.forConnector("iceberg") - .schema(Schema.newBuilder().column("f0", DataTypes.STRING()).build()) - .option("catalog-name", "hadoop_test") - .option("catalog-type", "hadoop") - .option("catalog-database", "test_db") - .option("catalog-table", "test") - .option("warehouse", warehouseDir.getAbsolutePath()) - .build(); - - table.insertInto(descriptor).execute(); - Awaitility.await() - .atMost(3, TimeUnit.SECONDS) - .untilAsserted( - () -> - assertThat(warehouseDir.toPath().resolve("test_db").resolve("test").toFile()) - .exists()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java deleted file mode 100644 index 6850060e8fc8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogDatabase.java +++ /dev/null @@ -1,253 +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; - -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.nio.file.Path; -import java.util.List; -import java.util.Map; -import java.util.Objects; -import org.apache.flink.table.catalog.exceptions.DatabaseNotEmptyException; -import org.apache.flink.types.Row; -import org.apache.iceberg.Schema; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkCatalogDatabase extends CatalogTestBase { - - @AfterEach - @Override - public void clean() { - sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testCreateNamespace() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should not already exist") - .isFalse(); - - sql("CREATE DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should exist") - .isTrue(); - - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should still exist") - .isTrue(); - - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should be dropped") - .isFalse(); - - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Database should be created") - .isTrue(); - } - - @TestTemplate - public void testDropEmptyDatabase() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - sql("DROP DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should have been dropped") - .isFalse(); - } - - @TestTemplate - public void testDropNonEmptyNamespace() { - assumeThat(isHadoopCatalog) - .as("Hadoop catalog throws IOException: Directory is not empty.") - .isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - assertThat(validationCatalog.tableExists(TableIdentifier.of(icebergNamespace, "tl"))) - .as("Table should exist") - .isTrue(); - assertThatThrownBy(() -> sql("DROP DATABASE %s", flinkDatabase)) - .cause() - .isInstanceOf(DatabaseNotEmptyException.class) - .hasMessage( - String.format("Database %s in catalog %s is not empty.", DATABASE, catalogName)); - sql("DROP TABLE %s.tl", flinkDatabase); - } - - @TestTemplate - public void testListTables() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - assertThat(sql("SHOW TABLES")).isEmpty(); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get()))); - - List tables = sql("SHOW TABLES"); - assertThat(tables).hasSize(1); - assertThat("tl").as("Table name should match").isEqualTo(tables.get(0).getField(0)); - } - - @TestTemplate - public void testListNamespace() { - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - - List databases = sql("SHOW DATABASES"); - - if (isHadoopCatalog) { - assertThat(databases).hasSize(1); - assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); - if (!baseNamespace.isEmpty()) { - // test namespace not belongs to this catalog - validationNamespaceCatalog.createNamespace( - Namespace.of(baseNamespace.level(0), "UNKNOWN_NAMESPACE")); - databases = sql("SHOW DATABASES"); - assertThat(databases).hasSize(1); - assertThat(databases.get(0).getField(0)).as("Should have db database").isEqualTo("db"); - } - } else { - // If there are multiple classes extends FlinkTestBase, TestHiveMetastore may loose the - // creation for default - // database. See HiveMetaStore.HMSHandler.init. - assertThat(databases) - .as("Should have db database") - .anyMatch(d -> Objects.equals(d.getField(0), "db")); - } - } - - @TestTemplate - public void testCreateNamespaceWithMetadata() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("prop", "value"); - } - - @TestTemplate - public void testCreateNamespaceWithComment() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - - sql("CREATE DATABASE %s COMMENT 'namespace doc'", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("comment", "namespace doc"); - } - - @TestTemplate - public void testCreateNamespaceWithLocation() throws Exception { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - - Path location = temporaryDirectory.getRoot(); - sql("CREATE DATABASE %s WITH ('location'='%s')", flinkDatabase, location); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("location", "file:" + location.getRoot()); - } - - @TestTemplate - public void testSetProperties() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isFalse(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - - sql("CREATE DATABASE %s", flinkDatabase); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should exist") - .isTrue(); - - Map defaultMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(defaultMetadata).doesNotContainKey("prop"); - sql("ALTER DATABASE %s SET ('prop'='value')", flinkDatabase); - Map nsMetadata = - validationNamespaceCatalog.loadNamespaceMetadata(icebergNamespace); - assertThat(nsMetadata).containsEntry("prop", "value"); - } - - @TestTemplate - public void testHadoopNotSupportMeta() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support namespace metadata").isTrue(); - assertThat(validationNamespaceCatalog.namespaceExists(icebergNamespace)) - .as("Namespace should not already exist") - .isFalse(); - assertThatThrownBy(() -> sql("CREATE DATABASE %s WITH ('prop'='value')", flinkDatabase)) - .cause() - .isInstanceOf(UnsupportedOperationException.class) - .hasMessage( - String.format( - "Cannot create namespace %s: metadata is not supported", icebergNamespace)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java deleted file mode 100644 index 4c9e95b8fa82..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogFactory.java +++ /dev/null @@ -1,119 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.catalog.Catalog; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.hive.HiveCatalog; -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.Test; - -public class TestFlinkCatalogFactory { - - private Map props; - - @BeforeEach - public void before() { - props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(CatalogProperties.WAREHOUSE_LOCATION, "/tmp/location"); - } - - @Test - public void testCreateCatalogHive() { - String catalogName = "hiveCatalog"; - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - assertThat(catalog).isNotNull().isInstanceOf(HiveCatalog.class); - } - - @Test - public void testCreateCatalogHadoop() { - String catalogName = "hadoopCatalog"; - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - assertThat(catalog).isNotNull().isInstanceOf(HadoopCatalog.class); - } - - @Test - public void testCreateCatalogCustom() { - String catalogName = "customCatalog"; - props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); - - Catalog catalog = - FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration()) - .loadCatalog(); - - assertThat(catalog).isNotNull().isInstanceOf(CustomHadoopCatalog.class); - } - - @Test - public void testCreateCatalogCustomWithHiveCatalogTypeSet() { - String catalogName = "customCatalog"; - props.put(CatalogProperties.CATALOG_IMPL, CustomHadoopCatalog.class.getName()); - props.put( - FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HIVE); - - assertThatThrownBy( - () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith( - "Cannot create catalog customCatalog, both catalog-type and catalog-impl are set"); - } - - @Test - public void testLoadCatalogUnknown() { - String catalogName = "unknownCatalog"; - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "fooType"); - - assertThatThrownBy( - () -> FlinkCatalogFactory.createCatalogLoader(catalogName, props, new Configuration())) - .isInstanceOf(UnsupportedOperationException.class) - .hasMessageStartingWith("Unknown catalog-type: fooType"); - } - - public static class CustomHadoopCatalog extends HadoopCatalog { - - public CustomHadoopCatalog() {} - - public CustomHadoopCatalog(Configuration conf, String warehouseLocation) { - setConf(conf); - initialize( - "custom", ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation)); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java deleted file mode 100644 index 7a364b856398..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTable.java +++ /dev/null @@ -1,669 +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; - -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.Arrays; -import java.util.Collections; -import java.util.Map; -import java.util.Optional; -import java.util.Set; -import java.util.stream.Collectors; -import java.util.stream.StreamSupport; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableException; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.api.constraints.UniqueConstraint; -import org.apache.flink.table.catalog.CatalogTable; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DataOperations; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.exceptions.NoSuchTableException; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkCatalogTable extends CatalogTestBase { - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @AfterEach - public void cleanNamespaces() { - sql("DROP TABLE IF EXISTS %s.tl", flinkDatabase); - sql("DROP TABLE IF EXISTS %s.tl2", flinkDatabase); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testGetTable() { - sql("CREATE TABLE tl(id BIGINT, strV STRING)"); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, "tl")); - Schema iSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "strV", Types.StringType.get())); - assertThat(table.schema().toString()) - .as("Should load the expected iceberg schema") - .isEqualTo(iSchema.toString()); - } - - @TestTemplate - public void testRenameTable() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support rename table").isFalse(); - final Schema tableSchema = - new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); - validationCatalog.createTable(TableIdentifier.of(icebergNamespace, "tl"), tableSchema); - sql("ALTER TABLE tl RENAME TO tl2"); - - assertThatThrownBy(() -> getTableEnv().from("tl")) - .isInstanceOf(ValidationException.class) - .hasMessage("Table `tl` was not found."); - - Schema actualSchema = FlinkSchemaUtil.convert(getTableEnv().from("tl2").getSchema()); - assertThat(tableSchema.asStruct()).isEqualTo(actualSchema.asStruct()); - } - - @TestTemplate - public void testCreateTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT)"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - CatalogTable catalogTable = catalogTable("tl"); - assertThat(catalogTable.getSchema()) - .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); - } - - @TestTemplate - public void testCreateTableWithPrimaryKey() throws Exception { - sql("CREATE TABLE tl(id BIGINT, data STRING, key STRING PRIMARY KEY NOT ENFORCED)"); - - Table table = table("tl"); - assertThat(table.schema().identifierFieldIds()) - .as("Should have the expected row key.") - .isEqualTo(Sets.newHashSet(table.schema().findField("key").fieldId())); - CatalogTable catalogTable = catalogTable("tl"); - Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); - assertThat(uniqueConstraintOptional).isPresent(); - assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("key"); - } - - @TestTemplate - public void testCreateTableWithMultiColumnsInPrimaryKey() throws Exception { - sql( - "CREATE TABLE tl(id BIGINT, data STRING, CONSTRAINT pk_constraint PRIMARY KEY(data, id) NOT ENFORCED)"); - - Table table = table("tl"); - assertThat(table.schema().identifierFieldIds()) - .as("Should have the expected RowKey") - .isEqualTo( - Sets.newHashSet( - table.schema().findField("id").fieldId(), - table.schema().findField("data").fieldId())); - CatalogTable catalogTable = catalogTable("tl"); - Optional uniqueConstraintOptional = catalogTable.getSchema().getPrimaryKey(); - assertThat(uniqueConstraintOptional).isPresent(); - assertThat(uniqueConstraintOptional.get().getColumns()).containsExactly("id", "data"); - } - - @TestTemplate - public void testCreateTableIfNotExists() { - sql("CREATE TABLE tl(id BIGINT)"); - - // Assert that table does exist. - assertThat(table("tl")).isNotNull(); - - sql("DROP TABLE tl"); - assertThatThrownBy(() -> table("tl")) - .isInstanceOf(NoSuchTableException.class) - .hasMessage("Table does not exist: " + getFullQualifiedTableName("tl")); - - sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); - assertThat(table("tl").properties()).doesNotContainKey("key"); - - table("tl").updateProperties().set("key", "value").commit(); - assertThat(table("tl").properties()).containsEntry("key", "value"); - - sql("CREATE TABLE IF NOT EXISTS tl(id BIGINT)"); - assertThat(table("tl").properties()).containsEntry("key", "value"); - } - - @TestTemplate - public void testCreateTableLike() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT)"); - sql("CREATE TABLE tl2 LIKE tl"); - - Table table = table("tl2"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - CatalogTable catalogTable = catalogTable("tl2"); - assertThat(catalogTable.getSchema()) - .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); - } - - @TestTemplate - public void testCreateTableLocation() { - assumeThat(isHadoopCatalog) - .as("HadoopCatalog does not support creating table with location") - .isFalse(); - sql("CREATE TABLE tl(id BIGINT) WITH ('location'='file:///tmp/location')"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - assertThat(table.location()).isEqualTo("file:///tmp/location"); - } - - @TestTemplate - public void testCreatePartitionTable() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT, dt STRING) PARTITIONED BY(dt)"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - assertThat(table.spec()) - .isEqualTo(PartitionSpec.builderFor(table.schema()).identity("dt").build()); - CatalogTable catalogTable = catalogTable("tl"); - assertThat(catalogTable.getSchema()) - .isEqualTo( - TableSchema.builder() - .field("id", DataTypes.BIGINT()) - .field("dt", DataTypes.STRING()) - .build()); - assertThat(catalogTable.getPartitionKeys()).isEqualTo(Collections.singletonList("dt")); - } - - @TestTemplate - public void testCreateTableWithColumnComment() { - sql("CREATE TABLE tl(id BIGINT COMMENT 'comment - id', data STRING COMMENT 'comment - data')"); - - Table table = table("tl"); - assertThat(table.schema().asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get(), "comment - id"), - Types.NestedField.optional(2, "data", Types.StringType.get(), "comment - data")) - .asStruct()); - } - - @TestTemplate - public void testCreateTableWithFormatV2ThroughTableProperty() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); - - Table table = table("tl"); - assertThat(((BaseTable) table).operations().current().formatVersion()).isEqualTo(2); - } - - @TestTemplate - public void testUpgradeTableWithFormatV2ThroughTableProperty() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='1')"); - - Table table = table("tl"); - TableOperations ops = ((BaseTable) table).operations(); - assertThat(ops.refresh().formatVersion()) - .as("should create table using format v1") - .isEqualTo(1); - sql("ALTER TABLE tl SET('format-version'='2')"); - assertThat(ops.refresh().formatVersion()) - .as("should update table to use format v2") - .isEqualTo(2); - } - - @TestTemplate - public void testDowngradeTableToFormatV1ThroughTablePropertyFails() throws Exception { - sql("CREATE TABLE tl(id BIGINT) WITH ('format-version'='2')"); - - Table table = table("tl"); - TableOperations ops = ((BaseTable) table).operations(); - assertThat(ops.refresh().formatVersion()) - .as("should create table using format v2") - .isEqualTo(2); - assertThatThrownBy(() -> sql("ALTER TABLE tl SET('format-version'='1')")) - .rootCause() - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot downgrade v2 table to v1"); - } - - @TestTemplate - public void testLoadTransformPartitionTable() throws TableNotExistException { - Schema schema = new Schema(Types.NestedField.optional(0, "id", Types.LongType.get())); - validationCatalog.createTable( - TableIdentifier.of(icebergNamespace, "tl"), - schema, - PartitionSpec.builderFor(schema).bucket("id", 100).build()); - - CatalogTable catalogTable = catalogTable("tl"); - assertThat(catalogTable.getSchema()) - .isEqualTo(TableSchema.builder().field("id", DataTypes.BIGINT()).build()); - assertThat(catalogTable.getPartitionKeys()).isEmpty(); - } - - @TestTemplate - public void testAlterTableProperties() throws TableNotExistException { - sql("CREATE TABLE tl(id BIGINT) WITH ('oldK'='oldV')"); - Map properties = Maps.newHashMap(); - properties.put("oldK", "oldV"); - - // new - sql("ALTER TABLE tl SET('newK'='newV')"); - properties.put("newK", "newV"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // update old - sql("ALTER TABLE tl SET('oldK'='oldV2')"); - properties.put("oldK", "oldV2"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - - // remove property - sql("ALTER TABLE tl RESET('oldK')"); - properties.remove("oldK"); - assertThat(table("tl").properties()).containsAllEntriesOf(properties); - } - - @TestTemplate - public void testAlterTableAddColumn() { - sql("CREATE TABLE tl(id BIGINT)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - sql("ALTER TABLE tl ADD (dt STRING)"); - Schema schemaAfter1 = table("tl").schema(); - assertThat(schemaAfter1.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - // Add multiple columns - sql("ALTER TABLE tl ADD (col1 STRING COMMENT 'comment for col1', col2 BIGINT)"); - Schema schemaAfter2 = table("tl").schema(); - assertThat(schemaAfter2.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional( - 3, "col1", Types.StringType.get(), "comment for col1"), - Types.NestedField.optional(4, "col2", Types.LongType.get())) - .asStruct()); - // Adding a required field should fail because Iceberg's SchemaUpdate does not allow - // incompatible changes. - assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (pk STRING NOT NULL)")) - .hasRootCauseInstanceOf(IllegalArgumentException.class) - .hasRootCauseMessage("Incompatible change: cannot add required column: pk"); - - // Adding an existing field should fail due to Flink's internal validation. - assertThatThrownBy(() -> sql("ALTER TABLE tl ADD (id STRING)")) - .isInstanceOf(ValidationException.class) - .hasMessageContaining("Try to add a column `id` which already exists in the table."); - } - - @TestTemplate - public void testAlterTableDropColumn() { - sql("CREATE TABLE tl(id BIGINT, dt STRING, col1 STRING, col2 BIGINT)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get()), - Types.NestedField.optional(4, "col2", Types.LongType.get())) - .asStruct()); - sql("ALTER TABLE tl DROP (dt)"); - Schema schemaAfter1 = table("tl").schema(); - assertThat(schemaAfter1.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get()), - Types.NestedField.optional(4, "col2", Types.LongType.get())) - .asStruct()); - // Drop multiple columns - sql("ALTER TABLE tl DROP (col1, col2)"); - Schema schemaAfter2 = table("tl").schema(); - assertThat(schemaAfter2.asStruct()) - .isEqualTo( - new Schema(Types.NestedField.optional(1, "id", Types.LongType.get())).asStruct()); - // Dropping an non-existing field should fail due to Flink's internal validation. - assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (foo)")) - .isInstanceOf(ValidationException.class) - .hasMessageContaining("The column `foo` does not exist in the base table."); - - // Dropping an already-deleted field should fail due to Flink's internal validation. - assertThatThrownBy(() -> sql("ALTER TABLE tl DROP (dt)")) - .isInstanceOf(ValidationException.class) - .hasMessageContaining("The column `dt` does not exist in the base table."); - } - - @TestTemplate - public void testAlterTableModifyColumnName() { - sql("CREATE TABLE tl(id BIGINT, dt STRING)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - sql("ALTER TABLE tl RENAME dt TO data"); - Schema schemaAfter = table("tl").schema(); - assertThat(schemaAfter.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())) - .asStruct()); - } - - @TestTemplate - public void testAlterTableModifyColumnType() { - sql("CREATE TABLE tl(id INTEGER, dt STRING)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - // Promote type from Integer to Long - sql("ALTER TABLE tl MODIFY (id BIGINT)"); - Schema schemaAfter = table("tl").schema(); - assertThat(schemaAfter.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - // Type change that doesn't follow the type-promotion rule should fail due to Iceberg's - // validation. - assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt INTEGER)")) - .isInstanceOf(TableException.class) - .hasRootCauseInstanceOf(IllegalArgumentException.class) - .hasRootCauseMessage("Cannot change column type: dt: string -> int"); - } - - @TestTemplate - public void testAlterTableModifyColumnNullability() { - sql("CREATE TABLE tl(id INTEGER NOT NULL, dt STRING)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - // Changing nullability from optional to required should fail - // because Iceberg's SchemaUpdate does not allow incompatible changes. - assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING NOT NULL)")) - .isInstanceOf(TableException.class) - .hasRootCauseInstanceOf(IllegalArgumentException.class) - .hasRootCauseMessage("Cannot change column nullability: dt: optional -> required"); - - // Set nullability from required to optional - sql("ALTER TABLE tl MODIFY (id INTEGER)"); - Schema schemaAfter = table("tl").schema(); - assertThat(schemaAfter.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - } - - @TestTemplate - public void testAlterTableModifyColumnPosition() { - sql("CREATE TABLE tl(id BIGINT, dt STRING)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - - sql("ALTER TABLE tl MODIFY (dt STRING FIRST)"); - Schema schemaAfter = table("tl").schema(); - assertThat(schemaAfter.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(1, "id", Types.LongType.get())) - .asStruct()); - - sql("ALTER TABLE tl MODIFY (dt STRING AFTER id)"); - Schema schemaAfterAfter = table("tl").schema(); - assertThat(schemaAfterAfter.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - // Modifying the position of a non-existing column should fail due to Flink's internal - // validation. - assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (non_existing STRING FIRST)")) - .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "Try to modify a column `non_existing` which does not exist in the table."); - - // Moving a column after a non-existing column should fail due to Flink's internal validation. - assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (dt STRING AFTER non_existing)")) - .isInstanceOf(ValidationException.class) - .hasMessageContaining( - "Referenced column `non_existing` by 'AFTER' does not exist in the table."); - } - - @TestTemplate - public void testAlterTableModifyColumnComment() { - sql("CREATE TABLE tl(id BIGINT, dt STRING)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get())) - .asStruct()); - - sql("ALTER TABLE tl MODIFY (dt STRING COMMENT 'comment for dt field')"); - Schema schemaAfter = table("tl").schema(); - assertThat(schemaAfter.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional( - 2, "dt", Types.StringType.get(), "comment for dt field")) - .asStruct()); - } - - @TestTemplate - public void testAlterTableConstraint() { - sql("CREATE TABLE tl(id BIGINT NOT NULL, dt STRING NOT NULL, col1 STRING)"); - Schema schemaBefore = table("tl").schema(); - assertThat(schemaBefore.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get())) - .asStruct()); - assertThat(schemaBefore.identifierFieldNames()).isEmpty(); - sql("ALTER TABLE tl ADD (PRIMARY KEY (id) NOT ENFORCED)"); - Schema schemaAfterAdd = table("tl").schema(); - assertThat(schemaAfterAdd.identifierFieldNames()).containsExactly("id"); - sql("ALTER TABLE tl MODIFY (PRIMARY KEY (dt) NOT ENFORCED)"); - Schema schemaAfterModify = table("tl").schema(); - assertThat(schemaAfterModify.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get())) - .asStruct()); - assertThat(schemaAfterModify.identifierFieldNames()).containsExactly("dt"); - // Composite primary key - sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, dt) NOT ENFORCED)"); - Schema schemaAfterComposite = table("tl").schema(); - assertThat(schemaAfterComposite.asStruct()) - .isEqualTo( - new Schema( - Types.NestedField.required(1, "id", Types.LongType.get()), - Types.NestedField.required(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "col1", Types.StringType.get())) - .asStruct()); - assertThat(schemaAfterComposite.identifierFieldNames()).containsExactlyInAnyOrder("id", "dt"); - // Setting an optional field as primary key should fail - // because Iceberg's SchemaUpdate does not allow incompatible changes. - assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (col1) NOT ENFORCED)")) - .isInstanceOf(TableException.class) - .hasRootCauseInstanceOf(IllegalArgumentException.class) - .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); - - // Setting a composite key containing an optional field should fail - // because Iceberg's SchemaUpdate does not allow incompatible changes. - assertThatThrownBy(() -> sql("ALTER TABLE tl MODIFY (PRIMARY KEY (id, col1) NOT ENFORCED)")) - .isInstanceOf(TableException.class) - .hasRootCauseInstanceOf(IllegalArgumentException.class) - .hasRootCauseMessage("Cannot add field col1 as an identifier field: not a required field"); - - // Dropping constraints is not supported yet - assertThatThrownBy(() -> sql("ALTER TABLE tl DROP PRIMARY KEY")) - .isInstanceOf(TableException.class) - .hasRootCauseInstanceOf(UnsupportedOperationException.class) - .hasRootCauseMessage("Unsupported table change: DropConstraint."); - } - - @TestTemplate - public void testRelocateTable() { - assumeThat(isHadoopCatalog).as("HadoopCatalog does not support relocate table").isFalse(); - sql("CREATE TABLE tl(id BIGINT)"); - sql("ALTER TABLE tl SET('location'='file:///tmp/location')"); - assertThat(table("tl").location()).isEqualTo("file:///tmp/location"); - } - - @TestTemplate - public void testSetCurrentAndCherryPickSnapshotId() { - sql("CREATE TABLE tl(c1 INT, c2 STRING, c3 STRING) PARTITIONED BY (c1)"); - - Table table = table("tl"); - - DataFile fileA = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-a.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - DataFile fileB = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-b.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=1") // easy way to set partition data for now - .withRecordCount(1) - .build(); - DataFile replacementFile = - DataFiles.builder(table.spec()) - .withPath("/path/to/data-a-replacement.parquet") - .withFileSizeInBytes(10) - .withPartitionPath("c1=0") // easy way to set partition data for now - .withRecordCount(1) - .build(); - - table.newAppend().appendFile(fileA).commit(); - long snapshotId = table.currentSnapshot().snapshotId(); - - // stage an overwrite that replaces FILE_A - table.newReplacePartitions().addFile(replacementFile).stageOnly().commit(); - - Snapshot staged = Iterables.getLast(table.snapshots()); - assertThat(staged.operation()) - .as("Should find the staged overwrite snapshot") - .isEqualTo(DataOperations.OVERWRITE); - // add another append so that the original commit can't be fast-forwarded - table.newAppend().appendFile(fileB).commit(); - - // test cherry pick - sql("ALTER TABLE tl SET('cherry-pick-snapshot-id'='%s')", staged.snapshotId()); - validateTableFiles(table, fileB, replacementFile); - - // test set current snapshot - sql("ALTER TABLE tl SET('current-snapshot-id'='%s')", snapshotId); - validateTableFiles(table, fileA); - } - - private void validateTableFiles(Table tbl, DataFile... expectedFiles) { - tbl.refresh(); - Set expectedFilePaths = - Arrays.stream(expectedFiles).map(DataFile::path).collect(Collectors.toSet()); - Set actualFilePaths = - StreamSupport.stream(tbl.newScan().planFiles().spliterator(), false) - .map(FileScanTask::file) - .map(ContentFile::path) - .collect(Collectors.toSet()); - assertThat(actualFilePaths).as("Files should match").isEqualTo(expectedFilePaths); - } - - private Table table(String name) { - return validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, name)); - } - - private CatalogTable catalogTable(String name) throws TableNotExistException { - return (CatalogTable) - getTableEnv() - .getCatalog(getTableEnv().getCurrentCatalog()) - .get() - .getTable(new ObjectPath(DATABASE, name)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java deleted file mode 100644 index e395414e925d..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkCatalogTablePartitions.java +++ /dev/null @@ -1,119 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.List; -import org.apache.flink.table.catalog.CatalogPartitionSpec; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.table.catalog.exceptions.TableNotExistException; -import org.apache.flink.table.catalog.exceptions.TableNotPartitionedException; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkCatalogTablePartitions extends CatalogTestBase { - - private final String tableName = "test_table"; - - @Parameter(index = 2) - private FileFormat format; - - @Parameter(index = 3) - private Boolean cacheEnabled; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, cacheEnabled={3}") - protected static List parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { - for (Boolean cacheEnabled : new Boolean[] {true, false}) { - for (Object[] catalogParams : CatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, cacheEnabled}); - } - } - } - return parameters; - } - - @Override - @BeforeEach - public void before() { - super.before(); - config.put(CatalogProperties.CACHE_ENABLED, String.valueOf(cacheEnabled)); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @AfterEach - public void cleanNamespaces() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testListPartitionsWithUnpartitionedTable() { - sql( - "CREATE TABLE %s (id INT, data VARCHAR) with ('write.format.default'='%s')", - tableName, format.name()); - sql("INSERT INTO %s SELECT 1,'a'", tableName); - - ObjectPath objectPath = new ObjectPath(DATABASE, tableName); - FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - assertThatThrownBy(() -> flinkCatalog.listPartitions(objectPath)) - .isInstanceOf(TableNotPartitionedException.class) - .hasMessageStartingWith("Table db.test_table in catalog") - .hasMessageEndingWith("is not partitioned."); - } - - @TestTemplate - public void testListPartitionsWithPartitionedTable() - throws TableNotExistException, TableNotPartitionedException { - sql( - "CREATE TABLE %s (id INT, data VARCHAR) PARTITIONED BY (data) " - + "with ('write.format.default'='%s')", - tableName, format.name()); - sql("INSERT INTO %s SELECT 1,'a'", tableName); - sql("INSERT INTO %s SELECT 2,'b'", tableName); - - ObjectPath objectPath = new ObjectPath(DATABASE, tableName); - FlinkCatalog flinkCatalog = (FlinkCatalog) getTableEnv().getCatalog(catalogName).get(); - List list = flinkCatalog.listPartitions(objectPath); - assertThat(list).hasSize(2); - List expected = Lists.newArrayList(); - CatalogPartitionSpec partitionSpec1 = new CatalogPartitionSpec(ImmutableMap.of("data", "a")); - CatalogPartitionSpec partitionSpec2 = new CatalogPartitionSpec(ImmutableMap.of("data", "b")); - expected.add(partitionSpec1); - expected.add(partitionSpec2); - assertThat(list).as("Should produce the expected catalog partition specs.").isEqualTo(expected); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java deleted file mode 100644 index 4b6ac25ab8e3..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkConfParser.java +++ /dev/null @@ -1,61 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.Map; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.iceberg.Table; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; - -public class TestFlinkConfParser { - - @Test - public void testDurationConf() { - Map writeOptions = ImmutableMap.of("write-prop", "111s"); - - ConfigOption configOption = - ConfigOptions.key("conf-prop").durationType().noDefaultValue(); - Configuration flinkConf = new Configuration(); - flinkConf.setString(configOption.key(), "222s"); - - Table table = mock(Table.class); - when(table.properties()).thenReturn(ImmutableMap.of("table-prop", "333s")); - - FlinkConfParser confParser = new FlinkConfParser(table, writeOptions, flinkConf); - Duration defaultVal = Duration.ofMillis(999); - - Duration result = - confParser.durationConf().option("write-prop").defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(111)); - - result = confParser.durationConf().flinkConfig(configOption).defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(222)); - - result = confParser.durationConf().tableProperty("table-prop").defaultValue(defaultVal).parse(); - assertThat(result).isEqualTo(Duration.ofSeconds(333)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.java deleted file mode 100644 index 838b0ea0e1a9..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkFilters.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.flink; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.Instant; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.List; -import java.util.Optional; -import java.util.stream.Collectors; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.Expressions; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.expressions.ApiExpressionUtils; -import org.apache.flink.table.expressions.CallExpression; -import org.apache.flink.table.expressions.Expression; -import org.apache.flink.table.expressions.FieldReferenceExpression; -import org.apache.flink.table.expressions.ResolvedExpression; -import org.apache.flink.table.expressions.UnresolvedCallExpression; -import org.apache.flink.table.expressions.UnresolvedReferenceExpression; -import org.apache.flink.table.expressions.ValueLiteralExpression; -import org.apache.flink.table.expressions.utils.ApiExpressionDefaultVisitor; -import org.apache.flink.table.functions.BuiltInFunctionDefinitions; -import org.apache.iceberg.expressions.And; -import org.apache.iceberg.expressions.BoundLiteralPredicate; -import org.apache.iceberg.expressions.Not; -import org.apache.iceberg.expressions.Or; -import org.apache.iceberg.expressions.UnboundPredicate; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.util.DateTimeUtil; -import org.apache.iceberg.util.Pair; -import org.junit.jupiter.api.Test; - -public class TestFlinkFilters { - - private static final TableSchema TABLE_SCHEMA = - TableSchema.builder() - .field("field1", DataTypes.INT()) - .field("field2", DataTypes.BIGINT()) - .field("field3", DataTypes.FLOAT()) - .field("field4", DataTypes.DOUBLE()) - .field("field5", DataTypes.STRING()) - .field("field6", DataTypes.BOOLEAN()) - .field("field7", DataTypes.BINARY(2)) - .field("field8", DataTypes.DECIMAL(10, 2)) - .field("field9", DataTypes.DATE()) - .field("field10", DataTypes.TIME()) - .field("field11", DataTypes.TIMESTAMP()) - .field("field12", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .build(); - - // A map list of fields and values used to verify the conversion of flink expression to iceberg - // expression - private static final List> FIELD_VALUE_LIST = - ImmutableList.of( - Pair.of("field1", 1), - Pair.of("field2", 2L), - Pair.of("field3", 3F), - Pair.of("field4", 4D), - Pair.of("field5", "iceberg"), - Pair.of("field6", true), - Pair.of("field7", new byte[] {'a', 'b'}), - Pair.of("field8", BigDecimal.valueOf(10.12)), - Pair.of("field9", DateTimeUtil.daysFromDate(LocalDate.now())), - Pair.of("field10", DateTimeUtil.microsFromTime(LocalTime.now())), - Pair.of("field11", DateTimeUtil.microsFromTimestamp(LocalDateTime.now())), - Pair.of("field12", DateTimeUtil.microsFromInstant(Instant.now()))); - - @Test - public void testFlinkDataTypeEqual() { - matchLiteral("field1", 1, 1); - matchLiteral("field2", 10L, 10L); - matchLiteral("field3", 1.2F, 1.2F); - matchLiteral("field4", 3.4D, 3.4D); - matchLiteral("field5", "abcd", "abcd"); - matchLiteral("field6", true, true); - matchLiteral("field7", new byte[] {'a', 'b'}, ByteBuffer.wrap(new byte[] {'a', 'b'})); - matchLiteral("field8", BigDecimal.valueOf(10.12), BigDecimal.valueOf(10.12)); - - LocalDate date = LocalDate.parse("2020-12-23"); - matchLiteral("field9", date, DateTimeUtil.daysFromDate(date)); - - LocalTime time = LocalTime.parse("12:13:14"); - matchLiteral("field10", time, DateTimeUtil.microsFromTime(time)); - - LocalDateTime dateTime = LocalDateTime.parse("2020-12-23T12:13:14"); - matchLiteral("field11", dateTime, DateTimeUtil.microsFromTimestamp(dateTime)); - - Instant instant = Instant.parse("2020-12-23T12:13:14.00Z"); - matchLiteral("field12", instant, DateTimeUtil.microsFromInstant(instant)); - } - - @Test - public void testEquals() { - for (Pair pair : FIELD_VALUE_LIST) { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.equal(pair.first(), pair.second()); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$(pair.first()).isEqual(Expressions.lit(pair.second())))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(pair.second()).isEqual(Expressions.$(pair.first())))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - } - - @Test - public void testEqualsNaN() { - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNaN("field3"); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field3").isEqual(Expressions.lit(Float.NaN)))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(Float.NaN).isEqual(Expressions.$("field3")))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testNotEquals() { - for (Pair pair : FIELD_VALUE_LIST) { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.notEqual(pair.first(), pair.second()); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$(pair.first()).isNotEqual(Expressions.lit(pair.second())))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(pair.second()).isNotEqual(Expressions.$(pair.first())))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - } - - @Test - public void testNotEqualsNaN() { - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.notNaN("field3"); - - Optional actual = - FlinkFilters.convert( - resolve(Expressions.$("field3").isNotEqual(Expressions.lit(Float.NaN)))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert( - resolve(Expressions.lit(Float.NaN).isNotEqual(Expressions.$("field3")))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testGreaterThan() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.greaterThan("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isGreater(Expressions.lit(1)))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isLess(Expressions.$("field1")))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testGreaterThanEquals() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.greaterThanOrEqual("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isGreaterOrEqual(Expressions.lit(1)))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isLessOrEqual(Expressions.$("field1")))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testLessThan() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.lessThan("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isLess(Expressions.lit(1)))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isGreater(Expressions.$("field1")))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testLessThanEquals() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.lessThanOrEqual("field1", 1); - - Optional actual = - FlinkFilters.convert(resolve(Expressions.$("field1").isLessOrEqual(Expressions.lit(1)))); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - Optional actual1 = - FlinkFilters.convert(resolve(Expressions.lit(1).isGreaterOrEqual(Expressions.$("field1")))); - assertThat(actual1).isPresent(); - assertPredicatesMatch(expected, actual1.get()); - } - - @Test - public void testIsNull() { - Expression expr = resolve(Expressions.$("field1").isNull()); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - UnboundPredicate expected = org.apache.iceberg.expressions.Expressions.isNull("field1"); - assertPredicatesMatch(expected, actual.get()); - } - - @Test - public void testIsNotNull() { - Expression expr = resolve(Expressions.$("field1").isNotNull()); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.notNull("field1"); - assertPredicatesMatch(expected, actual.get()); - } - - @Test - public void testAnd() { - Expression expr = - resolve( - Expressions.$("field1") - .isEqual(Expressions.lit(1)) - .and(Expressions.$("field2").isEqual(Expressions.lit(2L)))); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - And and = (And) actual.get(); - And expected = - (And) - org.apache.iceberg.expressions.Expressions.and( - org.apache.iceberg.expressions.Expressions.equal("field1", 1), - org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); - - assertPredicatesMatch(expected.left(), and.left()); - assertPredicatesMatch(expected.right(), and.right()); - } - - @Test - public void testOr() { - Expression expr = - resolve( - Expressions.$("field1") - .isEqual(Expressions.lit(1)) - .or(Expressions.$("field2").isEqual(Expressions.lit(2L)))); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - Or or = (Or) actual.get(); - Or expected = - (Or) - org.apache.iceberg.expressions.Expressions.or( - org.apache.iceberg.expressions.Expressions.equal("field1", 1), - org.apache.iceberg.expressions.Expressions.equal("field2", 2L)); - - assertPredicatesMatch(expected.left(), or.left()); - assertPredicatesMatch(expected.right(), or.right()); - } - - @Test - public void testNot() { - Expression expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.NOT, - Expressions.$("field1").isEqual(Expressions.lit(1)))); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - Not not = (Not) actual.get(); - Not expected = - (Not) - org.apache.iceberg.expressions.Expressions.not( - org.apache.iceberg.expressions.Expressions.equal("field1", 1)); - - assertThat(not.op()).as("Predicate operation should match").isEqualTo(expected.op()); - assertPredicatesMatch(expected.child(), not.child()); - } - - @Test - public void testLike() { - UnboundPredicate expected = - org.apache.iceberg.expressions.Expressions.startsWith("field5", "abc"); - Expression expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("abc%"))); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - assertPredicatesMatch(expected, actual.get()); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%abc"))); - actual = FlinkFilters.convert(expr); - assertThat(actual).isNotPresent(); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, - Expressions.$("field5"), - Expressions.lit("%abc%"))); - actual = FlinkFilters.convert(expr); - assertThat(actual).isNotPresent(); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, - Expressions.$("field5"), - Expressions.lit("abc%d"))); - actual = FlinkFilters.convert(expr); - assertThat(actual).isNotPresent(); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("%"))); - actual = FlinkFilters.convert(expr); - assertThat(actual).isNotPresent(); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a_"))); - actual = FlinkFilters.convert(expr); - assertThat(actual).isNotPresent(); - - expr = - resolve( - ApiExpressionUtils.unresolvedCall( - BuiltInFunctionDefinitions.LIKE, Expressions.$("field5"), Expressions.lit("a%b"))); - actual = FlinkFilters.convert(expr); - assertThat(actual).isNotPresent(); - } - - @SuppressWarnings("unchecked") - private void matchLiteral(String fieldName, Object flinkLiteral, T icebergLiteral) { - Expression expr = resolve(Expressions.$(fieldName).isEqual(Expressions.lit(flinkLiteral))); - Optional actual = FlinkFilters.convert(expr); - assertThat(actual).isPresent(); - org.apache.iceberg.expressions.Expression expression = actual.get(); - assertThat(expression) - .as("The expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - UnboundPredicate unboundPredicate = (UnboundPredicate) expression; - - org.apache.iceberg.expressions.Expression expression1 = - unboundPredicate.bind(FlinkSchemaUtil.convert(TABLE_SCHEMA).asStruct(), false); - assertThat(expression1) - .as("The expression should be a BoundLiteralPredicate") - .isInstanceOf(BoundLiteralPredicate.class); - - BoundLiteralPredicate predicate = (BoundLiteralPredicate) expression1; - assertThat(predicate.test(icebergLiteral)).isTrue(); - } - - private static Expression resolve(Expression originalExpression) { - return originalExpression.accept( - new ApiExpressionDefaultVisitor() { - @Override - public Expression visit(UnresolvedReferenceExpression unresolvedReference) { - String name = unresolvedReference.getName(); - Optional field = TABLE_SCHEMA.getTableColumn(name); - if (field.isPresent()) { - int index = TABLE_SCHEMA.getTableColumns().indexOf(field.get()); - return new FieldReferenceExpression(name, field.get().getType(), 0, index); - } else { - return null; - } - } - - @Override - public Expression visit(UnresolvedCallExpression unresolvedCall) { - List children = - unresolvedCall.getChildren().stream() - .map(e -> (ResolvedExpression) e.accept(this)) - .collect(Collectors.toList()); - return new CallExpression( - unresolvedCall.getFunctionDefinition(), children, DataTypes.STRING()); - } - - @Override - public Expression visit(ValueLiteralExpression valueLiteral) { - return valueLiteral; - } - - @Override - protected Expression defaultMethod(Expression expression) { - throw new UnsupportedOperationException( - String.format("unsupported expression: %s", expression)); - } - }); - } - - private void assertPredicatesMatch( - org.apache.iceberg.expressions.Expression expected, - org.apache.iceberg.expressions.Expression actual) { - assertThat(expected) - .as("The expected expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - assertThat(actual) - .as("The actual expression should be a UnboundPredicate") - .isInstanceOf(UnboundPredicate.class); - UnboundPredicate predicateExpected = (UnboundPredicate) expected; - UnboundPredicate predicateActual = (UnboundPredicate) actual; - assertThat(predicateActual.op()).isEqualTo(predicateExpected.op()); - assertThat(predicateActual.literal()).isEqualTo(predicateExpected.literal()); - assertThat(predicateActual.ref().name()).isEqualTo(predicateExpected.ref().name()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java deleted file mode 100644 index f1de267cf29b..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkHiveCatalog.java +++ /dev/null @@ -1,101 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.FileOutputStream; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Map; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.junit.jupiter.api.Test; - -public class TestFlinkHiveCatalog extends TestBase { - - @Test - public void testCreateCatalogWithWarehouseLocation() throws IOException { - Map props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - - File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); - props.put(CatalogProperties.WAREHOUSE_LOCATION, "file://" + warehouseDir.getAbsolutePath()); - - checkSQLQuery(props, warehouseDir); - } - - @Test - public void testCreateCatalogWithHiveConfDir() throws IOException { - // Dump the hive conf into a local file. - File hiveConfDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); - File hiveSiteXML = new File(hiveConfDir, "hive-site.xml"); - File warehouseDir = Files.createTempDirectory(temporaryDirectory, "junit").toFile(); - try (FileOutputStream fos = new FileOutputStream(hiveSiteXML)) { - Configuration newConf = new Configuration(hiveConf); - // Set another new directory which is different with the hive metastore's warehouse path. - newConf.set( - HiveConf.ConfVars.METASTOREWAREHOUSE.varname, "file://" + warehouseDir.getAbsolutePath()); - newConf.writeXml(fos); - } - assertThat(hiveSiteXML.toPath()).exists(); - - // Construct the catalog attributions. - Map props = Maps.newHashMap(); - props.put("type", "iceberg"); - props.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hive"); - props.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - // Set the 'hive-conf-dir' instead of 'warehouse' - props.put(FlinkCatalogFactory.HIVE_CONF_DIR, hiveConfDir.getAbsolutePath()); - - checkSQLQuery(props, warehouseDir); - } - - private void checkSQLQuery(Map catalogProperties, File warehouseDir) - throws IOException { - sql("CREATE CATALOG test_catalog WITH %s", CatalogTestBase.toWithClause(catalogProperties)); - sql("USE CATALOG test_catalog"); - sql("CREATE DATABASE test_db"); - sql("USE test_db"); - sql("CREATE TABLE test_table(c1 INT, c2 STRING)"); - sql("INSERT INTO test_table SELECT 1, 'a'"); - - Path databasePath = warehouseDir.toPath().resolve("test_db.db"); - assertThat(databasePath).exists(); - - Path tablePath = databasePath.resolve("test_table"); - assertThat(tablePath).exists(); - - Path dataPath = tablePath.resolve("data"); - assertThat(dataPath).exists(); - assertThat(Files.list(dataPath).count()) - .as("Should have a .crc file and a .parquet file") - .isEqualTo(2); - - sql("DROP TABLE test_table"); - sql("DROP DATABASE test_db"); - dropCatalog("test_catalog", false); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java deleted file mode 100644 index eab60d886ada..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkSchemaUtil.java +++ /dev/null @@ -1,416 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.api.ValidationException; -import org.apache.flink.table.types.logical.BinaryType; -import org.apache.flink.table.types.logical.CharType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.TimeType; -import org.apache.flink.table.types.logical.TimestampType; -import org.apache.flink.table.types.logical.VarBinaryType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -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.junit.jupiter.api.Test; - -public class TestFlinkSchemaUtil { - - @Test - public void testConvertFlinkSchemaToIcebergSchema() { - TableSchema flinkSchema = - TableSchema.builder() - .field("id", DataTypes.INT().notNull()) - .field("name", DataTypes.STRING()) /* optional by default */ - .field("salary", DataTypes.DOUBLE().notNull()) - .field( - "locations", - DataTypes.MAP( - DataTypes.STRING(), - DataTypes.ROW( - DataTypes.FIELD("posX", DataTypes.DOUBLE().notNull(), "X field"), - DataTypes.FIELD("posY", DataTypes.DOUBLE().notNull(), "Y field")))) - .field("strArray", DataTypes.ARRAY(DataTypes.STRING()).nullable()) - .field("intArray", DataTypes.ARRAY(DataTypes.INT()).nullable()) - .field("char", DataTypes.CHAR(10).notNull()) - .field("varchar", DataTypes.VARCHAR(10).notNull()) - .field("boolean", DataTypes.BOOLEAN().nullable()) - .field("tinyint", DataTypes.TINYINT()) - .field("smallint", DataTypes.SMALLINT()) - .field("bigint", DataTypes.BIGINT()) - .field("varbinary", DataTypes.VARBINARY(10)) - .field("binary", DataTypes.BINARY(10)) - .field("time", DataTypes.TIME()) - .field("timestampWithoutZone", DataTypes.TIMESTAMP()) - .field("timestampWithZone", DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()) - .field("date", DataTypes.DATE()) - .field("decimal", DataTypes.DECIMAL(2, 2)) - .field("decimal2", DataTypes.DECIMAL(38, 2)) - .field("decimal3", DataTypes.DECIMAL(10, 1)) - .field("multiset", DataTypes.MULTISET(DataTypes.STRING().notNull())) - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required(0, "id", Types.IntegerType.get(), null), - Types.NestedField.optional(1, "name", Types.StringType.get(), null), - Types.NestedField.required(2, "salary", Types.DoubleType.get(), null), - Types.NestedField.optional( - 3, - "locations", - Types.MapType.ofOptional( - 24, - 25, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(22, "posX", Types.DoubleType.get(), "X field"), - Types.NestedField.required( - 23, "posY", Types.DoubleType.get(), "Y field")))), - Types.NestedField.optional( - 4, "strArray", Types.ListType.ofOptional(26, Types.StringType.get())), - Types.NestedField.optional( - 5, "intArray", Types.ListType.ofOptional(27, Types.IntegerType.get())), - Types.NestedField.required(6, "char", Types.StringType.get()), - Types.NestedField.required(7, "varchar", Types.StringType.get()), - Types.NestedField.optional(8, "boolean", Types.BooleanType.get()), - Types.NestedField.optional(9, "tinyint", Types.IntegerType.get()), - Types.NestedField.optional(10, "smallint", Types.IntegerType.get()), - Types.NestedField.optional(11, "bigint", Types.LongType.get()), - Types.NestedField.optional(12, "varbinary", Types.BinaryType.get()), - Types.NestedField.optional(13, "binary", Types.FixedType.ofLength(10)), - Types.NestedField.optional(14, "time", Types.TimeType.get()), - Types.NestedField.optional( - 15, "timestampWithoutZone", Types.TimestampType.withoutZone()), - Types.NestedField.optional(16, "timestampWithZone", Types.TimestampType.withZone()), - Types.NestedField.optional(17, "date", Types.DateType.get()), - Types.NestedField.optional(18, "decimal", Types.DecimalType.of(2, 2)), - Types.NestedField.optional(19, "decimal2", Types.DecimalType.of(38, 2)), - Types.NestedField.optional(20, "decimal3", Types.DecimalType.of(10, 1)), - Types.NestedField.optional( - 21, - "multiset", - Types.MapType.ofRequired(28, 29, Types.StringType.get(), Types.IntegerType.get()))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testMapField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "map_int_long", - DataTypes.MAP(DataTypes.INT(), DataTypes.BIGINT()).notNull()) /* Required */ - .field( - "map_int_array_string", - DataTypes.MAP(DataTypes.ARRAY(DataTypes.INT()), DataTypes.STRING())) - .field( - "map_decimal_string", DataTypes.MAP(DataTypes.DECIMAL(10, 2), DataTypes.STRING())) - .field( - "map_fields_fields", - DataTypes.MAP( - DataTypes.ROW( - DataTypes.FIELD("field_int", DataTypes.INT(), "doc - int"), - DataTypes.FIELD("field_string", DataTypes.STRING(), "doc - string")) - .notNull(), /* Required */ - DataTypes.ROW( - DataTypes.FIELD( - "field_array", - DataTypes.ARRAY(DataTypes.STRING()), - "doc - array")) - .notNull() /* Required */) - .notNull() /* Required */) - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "map_int_long", - Types.MapType.ofOptional(4, 5, Types.IntegerType.get(), Types.LongType.get()), - null), - Types.NestedField.optional( - 1, - "map_int_array_string", - Types.MapType.ofOptional( - 7, - 8, - Types.ListType.ofOptional(6, Types.IntegerType.get()), - Types.StringType.get()), - null), - Types.NestedField.optional( - 2, - "map_decimal_string", - Types.MapType.ofOptional( - 9, 10, Types.DecimalType.of(10, 2), Types.StringType.get())), - Types.NestedField.required( - 3, - "map_fields_fields", - Types.MapType.ofRequired( - 15, - 16, - Types.StructType.of( - Types.NestedField.optional( - 11, "field_int", Types.IntegerType.get(), "doc - int"), - Types.NestedField.optional( - 12, "field_string", Types.StringType.get(), "doc - string")), - Types.StructType.of( - Types.NestedField.optional( - 14, - "field_array", - Types.ListType.ofOptional(13, Types.StringType.get()), - "doc - array"))))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testStructField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "struct_int_string_decimal", - DataTypes.ROW( - DataTypes.FIELD("field_int", DataTypes.INT()), - DataTypes.FIELD("field_string", DataTypes.STRING()), - DataTypes.FIELD("field_decimal", DataTypes.DECIMAL(19, 2)), - DataTypes.FIELD( - "field_struct", - DataTypes.ROW( - DataTypes.FIELD("inner_struct_int", DataTypes.INT()), - DataTypes.FIELD( - "inner_struct_float_array", - DataTypes.ARRAY(DataTypes.FLOAT()))) - .notNull()) /* Row is required */) - .notNull()) /* Required */ - .field( - "struct_map_int_int", - DataTypes.ROW( - DataTypes.FIELD( - "field_map", DataTypes.MAP(DataTypes.INT(), DataTypes.INT()))) - .nullable()) /* Optional */ - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "struct_int_string_decimal", - Types.StructType.of( - Types.NestedField.optional(5, "field_int", Types.IntegerType.get()), - Types.NestedField.optional(6, "field_string", Types.StringType.get()), - Types.NestedField.optional(7, "field_decimal", Types.DecimalType.of(19, 2)), - Types.NestedField.required( - 8, - "field_struct", - Types.StructType.of( - Types.NestedField.optional( - 3, "inner_struct_int", Types.IntegerType.get()), - Types.NestedField.optional( - 4, - "inner_struct_float_array", - Types.ListType.ofOptional(2, Types.FloatType.get())))))), - Types.NestedField.optional( - 1, - "struct_map_int_int", - Types.StructType.of( - Types.NestedField.optional( - 11, - "field_map", - Types.MapType.ofOptional( - 9, 10, Types.IntegerType.get(), Types.IntegerType.get()))))); - - checkSchema(flinkSchema, icebergSchema); - } - - @Test - public void testListField() { - TableSchema flinkSchema = - TableSchema.builder() - .field( - "list_struct_fields", - DataTypes.ARRAY(DataTypes.ROW(DataTypes.FIELD("field_int", DataTypes.INT()))) - .notNull()) /* Required */ - .field( - "list_optional_struct_fields", - DataTypes.ARRAY( - DataTypes.ROW( - DataTypes.FIELD( - "field_timestamp_with_local_time_zone", - DataTypes.TIMESTAMP_WITH_LOCAL_TIME_ZONE()))) - .nullable()) /* Optional */ - .field( - "list_map_fields", - DataTypes.ARRAY( - DataTypes.MAP( - DataTypes.ARRAY( - DataTypes.INT().notNull()), /* Key of map must be required */ - DataTypes.ROW( - DataTypes.FIELD("field_0", DataTypes.INT(), "doc - int"))) - .notNull()) - .notNull()) /* Required */ - .build(); - - Schema icebergSchema = - new Schema( - Types.NestedField.required( - 0, - "list_struct_fields", - Types.ListType.ofOptional( - 4, - Types.StructType.of( - Types.NestedField.optional(3, "field_int", Types.IntegerType.get())))), - Types.NestedField.optional( - 1, - "list_optional_struct_fields", - Types.ListType.ofOptional( - 6, - Types.StructType.of( - Types.NestedField.optional( - 5, - "field_timestamp_with_local_time_zone", - Types.TimestampType.withZone())))), - Types.NestedField.required( - 2, - "list_map_fields", - Types.ListType.ofRequired( - 11, - Types.MapType.ofOptional( - 9, - 10, - Types.ListType.ofRequired(7, Types.IntegerType.get()), - Types.StructType.of( - Types.NestedField.optional( - 8, "field_0", Types.IntegerType.get(), "doc - int")))))); - - checkSchema(flinkSchema, icebergSchema); - } - - private void checkSchema(TableSchema flinkSchema, Schema icebergSchema) { - assertThat(FlinkSchemaUtil.convert(flinkSchema).asStruct()).isEqualTo(icebergSchema.asStruct()); - // The conversion is not a 1:1 mapping, so we just check iceberg types. - assertThat( - FlinkSchemaUtil.convert( - FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergSchema))) - .asStruct()) - .isEqualTo(icebergSchema.asStruct()); - } - - @Test - public void testInconsistentTypes() { - checkInconsistentType( - Types.UUIDType.get(), new BinaryType(16), new BinaryType(16), Types.FixedType.ofLength(16)); - checkInconsistentType( - Types.StringType.get(), - new VarCharType(VarCharType.MAX_LENGTH), - new CharType(100), - Types.StringType.get()); - checkInconsistentType( - Types.BinaryType.get(), - new VarBinaryType(VarBinaryType.MAX_LENGTH), - new VarBinaryType(100), - Types.BinaryType.get()); - checkInconsistentType( - Types.TimeType.get(), new TimeType(), new TimeType(3), Types.TimeType.get()); - checkInconsistentType( - Types.TimestampType.withoutZone(), - new TimestampType(6), - new TimestampType(3), - Types.TimestampType.withoutZone()); - checkInconsistentType( - Types.TimestampType.withZone(), - new LocalZonedTimestampType(6), - new LocalZonedTimestampType(3), - Types.TimestampType.withZone()); - } - - private void checkInconsistentType( - Type icebergType, - LogicalType flinkExpectedType, - LogicalType flinkType, - Type icebergExpectedType) { - assertThat(FlinkSchemaUtil.convert(icebergType)).isEqualTo(flinkExpectedType); - assertThat(FlinkSchemaUtil.convert(FlinkSchemaUtil.toSchema(RowType.of(flinkType))).asStruct()) - .isEqualTo(Types.StructType.of(Types.NestedField.optional(0, "f0", icebergExpectedType))); - } - - @Test - public void testConvertFlinkSchemaBaseOnIcebergSchema() { - Schema baseSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required(101, "int", Types.IntegerType.get()), - Types.NestedField.optional(102, "string", Types.StringType.get())), - Sets.newHashSet(101)); - - TableSchema flinkSchema = - TableSchema.builder() - .field("int", DataTypes.INT().notNull()) - .field("string", DataTypes.STRING().nullable()) - .primaryKey("int") - .build(); - Schema convertedSchema = FlinkSchemaUtil.convert(baseSchema, flinkSchema); - assertThat(convertedSchema.asStruct()).isEqualTo(baseSchema.asStruct()); - assertThat(convertedSchema.identifierFieldIds()).containsExactly(101); - } - - @Test - public void testConvertFlinkSchemaWithPrimaryKeys() { - Schema icebergSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required(1, "int", Types.IntegerType.get()), - Types.NestedField.required(2, "string", Types.StringType.get())), - Sets.newHashSet(1, 2)); - - TableSchema tableSchema = FlinkSchemaUtil.toSchema(icebergSchema); - assertThat(tableSchema.getPrimaryKey()) - .isPresent() - .get() - .satisfies(k -> assertThat(k.getColumns()).containsExactly("int", "string")); - } - - @Test - public void testConvertFlinkSchemaWithNestedColumnInPrimaryKeys() { - Schema icebergSchema = - new Schema( - Lists.newArrayList( - Types.NestedField.required( - 1, - "struct", - Types.StructType.of( - Types.NestedField.required(2, "inner", Types.IntegerType.get())))), - Sets.newHashSet(2)); - - assertThatThrownBy(() -> FlinkSchemaUtil.toSchema(icebergSchema)) - .isInstanceOf(ValidationException.class) - .hasMessageStartingWith("Could not create a PRIMARY KEY") - .hasMessageContaining("Column 'struct.inner' does not exist."); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java deleted file mode 100644 index b73300e3f170..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ /dev/null @@ -1,358 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.Expressions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.source.BoundedTableFactory; -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.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -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; - - @Parameter(index = 2) - private FileFormat format; - - @Parameter(index = 3) - private boolean isStreamingJob; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") - public static List parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.ORC, FileFormat.AVRO, FileFormat.PARQUET}) { - for (Boolean isStreaming : new Boolean[] {true, false}) { - for (Object[] catalogParams : CatalogTestBase.parameters()) { - String catalogName = (String) catalogParams[0]; - Namespace baseNamespace = (Namespace) catalogParams[1]; - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); - } - } - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreamingJob) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - } - } - return tEnv; - } - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - sql( - "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", - TABLE_NAME, format.name()); - icebergTable = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - BoundedTableFactory.clearDataSets(); - super.clean(); - } - - @TestTemplate - public void testInsertFromSourceTable() throws Exception { - // Register the rows into a temporary table. - getTableEnv() - .createTemporaryView( - "sourceTable", - getTableEnv() - .fromValues( - SimpleDataUtil.FLINK_SCHEMA.toRowDataType(), - Expressions.row(1, "hello"), - Expressions.row(2, "world"), - Expressions.row(3, (String) null), - Expressions.row(null, "bar"))); - - // Redirect the records from source table to destination table. - sql("INSERT INTO %s SELECT id,data from sourceTable", TABLE_NAME); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), - SimpleDataUtil.createRecord(2, "world"), - SimpleDataUtil.createRecord(3, null), - SimpleDataUtil.createRecord(null, "bar"))); - } - - @TestTemplate - public void testOverwriteTable() throws Exception { - assumeThat(isStreamingJob) - .as("Flink unbounded streaming does not support overwrite operation") - .isFalse(); - - sql("INSERT INTO %s SELECT 1, 'a'", TABLE_NAME); - SimpleDataUtil.assertTableRecords( - icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(1, "a"))); - - sql("INSERT OVERWRITE %s SELECT 2, 'b'", TABLE_NAME); - SimpleDataUtil.assertTableRecords( - icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); - } - - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - 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 /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, 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)); - } - - @TestTemplate - public void testReplacePartitions() throws Exception { - assumeThat(isStreamingJob) - .as("Flink unbounded streaming does not support overwrite operation") - .isFalse(); - String tableName = "test_partition"; - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", - tableName, format.name()); - - try { - Table partitionedTable = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - - sql("INSERT INTO %s SELECT 1, 'a'", tableName); - sql("INSERT INTO %s SELECT 2, 'b'", tableName); - sql("INSERT INTO %s SELECT 3, 'c'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "b"), - SimpleDataUtil.createRecord(3, "c"))); - - sql("INSERT OVERWRITE %s SELECT 4, 'b'", tableName); - sql("INSERT OVERWRITE %s SELECT 5, 'a'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(5, "a"), - SimpleDataUtil.createRecord(4, "b"), - SimpleDataUtil.createRecord(3, "c"))); - - sql("INSERT OVERWRITE %s PARTITION (data='a') SELECT 6", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(6, "a"), - SimpleDataUtil.createRecord(4, "b"), - SimpleDataUtil.createRecord(3, "c"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testInsertIntoPartition() throws Exception { - String tableName = "test_insert_into_partition"; - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH ('write.format.default'='%s')", - tableName, format.name()); - - try { - Table partitionedTable = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - - // Full partition. - sql("INSERT INTO %s PARTITION (data='a') SELECT 1", tableName); - sql("INSERT INTO %s PARTITION (data='a') SELECT 2", tableName); - sql("INSERT INTO %s PARTITION (data='b') SELECT 3", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "a"), - SimpleDataUtil.createRecord(3, "b"))); - - // Partial partition. - sql("INSERT INTO %s SELECT 4, 'c'", tableName); - sql("INSERT INTO %s SELECT 5, 'd'", tableName); - - SimpleDataUtil.assertTableRecords( - partitionedTable, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "a"), - SimpleDataUtil.createRecord(3, "b"), - SimpleDataUtil.createRecord(4, "c"), - SimpleDataUtil.createRecord(5, "d"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java deleted file mode 100644 index d52d54e159e6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestFlinkUpsert.java +++ /dev/null @@ -1,334 +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; - -import java.time.LocalDate; -import java.util.List; -import java.util.Map; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.Namespace; -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.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkUpsert extends CatalogTestBase { - - @Parameter(index = 2) - private FileFormat format; - - @Parameter(index = 3) - private boolean isStreamingJob; - - private final Map tableUpsertProps = Maps.newHashMap(); - private TableEnvironment tEnv; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}, isStreaming={3}") - public static List parameters() { - List parameters = Lists.newArrayList(); - for (FileFormat format : - new FileFormat[] {FileFormat.PARQUET, FileFormat.AVRO, FileFormat.ORC}) { - for (Boolean isStreaming : new Boolean[] {true, false}) { - // Only test with one catalog as this is a file operation concern. - // FlinkCatalogTestBase requires the catalog name start with testhadoop if using hadoop - // catalog. - String catalogName = "testhadoop"; - Namespace baseNamespace = Namespace.of("default"); - parameters.add(new Object[] {catalogName, baseNamespace, format, isStreaming}); - } - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreamingJob) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - } - } - return tEnv; - } - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE IF NOT EXISTS %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - tableUpsertProps.put(TableProperties.FORMAT_VERSION, "2"); - tableUpsertProps.put(TableProperties.UPSERT_ENABLED, "true"); - tableUpsertProps.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - } - - @Override - @AfterEach - public void clean() { - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testUpsertAndQuery() { - String tableName = "test_upsert_query"; - LocalDate dt20220301 = LocalDate.of(2022, 3, 1); - LocalDate dt20220302 = LocalDate.of(2022, 3, 2); - - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - try { - sql( - "INSERT INTO %s VALUES " - + "(1, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-01')," - + "(2, 'Jane', DATE '2022-03-01')", - tableName); - - sql( - "INSERT INTO %s VALUES " - + "(2, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-02')," - + "(2, 'Jane', DATE '2022-03-02')", - tableName); - - List rowsOn20220301 = - Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); - - List rowsOn20220302 = - Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testUpsertOptions() { - String tableName = "test_upsert_options"; - LocalDate dt20220301 = LocalDate.of(2022, 3, 1); - LocalDate dt20220302 = LocalDate.of(2022, 3, 2); - - Map optionsUpsertProps = Maps.newHashMap(tableUpsertProps); - optionsUpsertProps.remove(TableProperties.UPSERT_ENABLED); - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, dt DATE, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(optionsUpsertProps)); - - try { - sql( - "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " - + "(1, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-01')," - + "(2, 'Jane', DATE '2022-03-01')", - tableName); - - sql( - "INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ VALUES " - + "(2, 'Bill', DATE '2022-03-01')," - + "(1, 'Jane', DATE '2022-03-02')," - + "(2, 'Jane', DATE '2022-03-02')", - tableName); - - List rowsOn20220301 = - Lists.newArrayList(Row.of(1, "Jane", dt20220301), Row.of(2, "Bill", dt20220301)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt < '2022-03-02'", tableName), rowsOn20220301); - - List rowsOn20220302 = - Lists.newArrayList(Row.of(1, "Jane", dt20220302), Row.of(2, "Jane", dt20220302)); - TestHelpers.assertRows( - sql("SELECT * FROM %s WHERE dt = '2022-03-02'", tableName), rowsOn20220302); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Iterables.concat(rowsOn20220301, rowsOn20220302))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testPrimaryKeyEqualToPartitionKey() { - // This is an SQL based reproduction of TestFlinkIcebergSinkV2#testUpsertOnDataKey - String tableName = "upsert_on_id_key"; - try { - sql( - "CREATE TABLE %s(id INT NOT NULL, name STRING NOT NULL, PRIMARY KEY(id) NOT ENFORCED) " - + "PARTITIONED BY (id) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(1, 'Jane')," + "(2, 'Bill')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, "Jane"), Row.of(2, "Bill"))); - - sql("INSERT INTO %s VALUES " + "(1, 'Bill')," + "(2, 'Jane')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, "Bill"), Row.of(2, "Jane"))); - - sql("INSERT INTO %s VALUES " + "(3, 'Bill')," + "(4, 'Jane')", tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of(1, "Bill"), Row.of(2, "Jane"), Row.of(3, "Bill"), Row.of(4, "Jane"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testPrimaryKeyFieldsAtBeginningOfSchema() { - String tableName = "upsert_on_pk_at_schema_start"; - LocalDate dt = LocalDate.of(2022, 3, 1); - try { - sql( - "CREATE TABLE %s(id INT, dt DATE NOT NULL, name STRING NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql( - "INSERT INTO %s VALUES " - + "(1, DATE '2022-03-01', 'Andy')," - + "(1, DATE '2022-03-01', 'Bill')," - + "(2, DATE '2022-03-01', 'Jane')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, dt, "Bill"), Row.of(2, dt, "Jane"))); - - sql( - "INSERT INTO %s VALUES " - + "(1, DATE '2022-03-01', 'Jane')," - + "(2, DATE '2022-03-01', 'Bill')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of(1, dt, "Jane"), Row.of(2, dt, "Bill"))); - - sql( - "INSERT INTO %s VALUES " - + "(3, DATE '2022-03-01', 'Duke')," - + "(4, DATE '2022-03-01', 'Leon')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of(1, dt, "Jane"), - Row.of(2, dt, "Bill"), - Row.of(3, dt, "Duke"), - Row.of(4, dt, "Leon"))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } - - @TestTemplate - public void testPrimaryKeyFieldsAtEndOfTableSchema() { - // This is the same test case as testPrimaryKeyFieldsAtBeginningOfSchema, but the primary key - // fields - // are located at the end of the flink schema. - String tableName = "upsert_on_pk_at_schema_end"; - LocalDate dt = LocalDate.of(2022, 3, 1); - try { - sql( - "CREATE TABLE %s(name STRING NOT NULL, id INT, dt DATE NOT NULL, PRIMARY KEY(id,dt) NOT ENFORCED) " - + "PARTITIONED BY (dt) WITH %s", - tableName, toWithClause(tableUpsertProps)); - - sql( - "INSERT INTO %s VALUES " - + "('Andy', 1, DATE '2022-03-01')," - + "('Bill', 1, DATE '2022-03-01')," - + "('Jane', 2, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of("Bill", 1, dt), Row.of("Jane", 2, dt))); - - sql( - "INSERT INTO %s VALUES " - + "('Jane', 1, DATE '2022-03-01')," - + "('Bill', 2, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList(Row.of("Jane", 1, dt), Row.of("Bill", 2, dt))); - - sql( - "INSERT INTO %s VALUES " - + "('Duke', 3, DATE '2022-03-01')," - + "('Leon', 4, DATE '2022-03-01')", - tableName); - - TestHelpers.assertRows( - sql("SELECT * FROM %s", tableName), - Lists.newArrayList( - Row.of("Jane", 1, dt), - Row.of("Bill", 2, dt), - Row.of("Duke", 3, dt), - Row.of("Leon", 4, dt))); - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java deleted file mode 100644 index 8cebf950c5f0..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestHelpers.java +++ /dev/null @@ -1,632 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.time.OffsetDateTime; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import org.apache.avro.generic.GenericData; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.table.data.ArrayData; -import org.apache.flink.table.data.DecimalData; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.MapData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.data.conversion.DataStructureConverter; -import org.apache.flink.table.data.conversion.DataStructureConverters; -import org.apache.flink.table.runtime.typeutils.InternalSerializers; -import org.apache.flink.table.types.logical.ArrayType; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.flink.table.types.logical.MapType; -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.ContentFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.data.RowDataUtil; -import org.apache.iceberg.flink.source.FlinkInputFormat; -import org.apache.iceberg.flink.source.FlinkInputSplit; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.apache.iceberg.types.Type; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.DateTimeUtil; - -public class TestHelpers { - private TestHelpers() {} - - public static T roundTripKryoSerialize(Class clazz, T table) throws IOException { - KryoSerializer kryo = new KryoSerializer<>(clazz, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - kryo.serialize(table, outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - return kryo.deserialize(inputView); - } - - public static RowData copyRowData(RowData from, RowType rowType) { - TypeSerializer[] fieldSerializers = - rowType.getChildren().stream() - .map((LogicalType type) -> InternalSerializers.create(type)) - .toArray(TypeSerializer[]::new); - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - - return RowDataUtil.clone(from, null, rowType, fieldSerializers, fieldGetters); - } - - public static void readRowData(FlinkInputFormat input, Consumer visitor) - throws IOException { - for (FlinkInputSplit s : input.createInputSplits(0)) { - input.open(s); - try { - while (!input.reachedEnd()) { - RowData row = input.nextRecord(null); - visitor.accept(row); - } - } finally { - input.close(); - } - } - } - - public static List readRowData(FlinkInputFormat inputFormat, RowType rowType) - throws IOException { - List results = Lists.newArrayList(); - readRowData(inputFormat, row -> results.add(copyRowData(row, rowType))); - return results; - } - - public static List readRows(FlinkInputFormat inputFormat, RowType rowType) - throws IOException { - return convertRowDataToRow(readRowData(inputFormat, rowType), rowType); - } - - public static List convertRowDataToRow(List rowDataList, RowType rowType) { - DataStructureConverter converter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - return rowDataList.stream() - .map(converter::toExternal) - .map(Row.class::cast) - .collect(Collectors.toList()); - } - - private static List convertRecordToRow(List expectedRecords, Schema schema) { - List expected = Lists.newArrayList(); - @SuppressWarnings("unchecked") - DataStructureConverter converter = - (DataStructureConverter) - DataStructureConverters.getConverter( - TypeConversions.fromLogicalToDataType(FlinkSchemaUtil.convert(schema))); - expectedRecords.forEach( - r -> expected.add(converter.toExternal(RowDataConverter.convert(schema, r)))); - return expected; - } - - public static void assertRecordsWithOrder( - List results, List expectedRecords, Schema schema) { - List expected = convertRecordToRow(expectedRecords, schema); - assertRowsWithOrder(results, expected); - } - - public static void assertRecords(List results, List expectedRecords, Schema schema) { - List expected = convertRecordToRow(expectedRecords, schema); - assertRows(results, expected); - } - - public static void assertRows(List results, List expected, RowType rowType) { - assertRows(convertRowDataToRow(results, rowType), convertRowDataToRow(expected, rowType)); - } - - public static void assertRows(List results, List expected) { - assertThat(results).containsExactlyInAnyOrderElementsOf(expected); - } - - public static void assertRowsWithOrder(List results, List expected) { - assertThat(results).containsExactlyElementsOf(expected); - } - - public static void assertRowData(Schema schema, StructLike expected, RowData actual) { - assertRowData(schema.asStruct(), FlinkSchemaUtil.convert(schema), expected, actual); - } - - public static void assertRowData( - Types.StructType structType, - LogicalType rowType, - StructLike expectedRecord, - RowData actualRowData) { - if (expectedRecord == null && actualRowData == null) { - return; - } - - assertThat(expectedRecord).isNotNull(); - assertThat(actualRowData).isNotNull(); - - List types = Lists.newArrayList(); - for (Types.NestedField field : structType.fields()) { - types.add(field.type()); - } - - for (int i = 0; i < types.size(); i += 1) { - LogicalType logicalType = ((RowType) rowType).getTypeAt(i); - Object expected = expectedRecord.get(i, Object.class); - // The RowData.createFieldGetter won't return null for the required field. But in the - // projection case, if we are - // projecting a nested required field from an optional struct, then we should give a null for - // the projected field - // if the outer struct value is null. So we need to check the nullable for actualRowData here. - // For more details - // please see issue #2738. - Object actual = - actualRowData.isNullAt(i) - ? null - : RowData.createFieldGetter(logicalType, i).getFieldOrNull(actualRowData); - assertEquals(types.get(i), logicalType, expected, actual); - } - } - - private static void assertEquals( - Type type, LogicalType logicalType, Object expected, Object actual) { - - if (expected == null && actual == null) { - return; - } - - assertThat(expected).isNotNull(); - assertThat(actual).isNotNull(); - - switch (type.typeId()) { - case BOOLEAN: - assertThat(actual).as("boolean value should be equal").isEqualTo(expected); - break; - case INTEGER: - assertThat(actual).as("int value should be equal").isEqualTo(expected); - break; - case LONG: - assertThat(actual).as("long value should be equal").isEqualTo(expected); - break; - case FLOAT: - assertThat(actual).as("float value should be equal").isEqualTo(expected); - break; - case DOUBLE: - assertThat(actual).as("double value should be equal").isEqualTo(expected); - break; - case STRING: - assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); - assertThat(actual.toString()) - .as("string should be equal") - .isEqualTo(String.valueOf(expected)); - break; - case DATE: - assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); - LocalDate date = DateTimeUtil.dateFromDays((int) actual); - assertThat(date).as("date should be equal").isEqualTo(expected); - break; - case TIME: - assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); - int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); - assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); - break; - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - assertThat(expected) - .as("Should expect a OffsetDataTime") - .isInstanceOf(OffsetDateTime.class); - OffsetDateTime ts = (OffsetDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("OffsetDataTime should be equal") - .isEqualTo(ts.toLocalDateTime()); - } else { - assertThat(expected) - .as("Should expect a LocalDataTime") - .isInstanceOf(LocalDateTime.class); - LocalDateTime ts = (LocalDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("LocalDataTime should be equal") - .isEqualTo(ts); - } - break; - case BINARY: - assertThat(ByteBuffer.wrap((byte[]) actual)) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class) - .isEqualTo(expected); - break; - case DECIMAL: - assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); - BigDecimal bd = (BigDecimal) expected; - assertThat(((DecimalData) actual).toBigDecimal()) - .as("decimal value should be equal") - .isEqualTo(bd); - break; - case LIST: - assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); - Collection expectedArrayData = (Collection) expected; - ArrayData actualArrayData = (ArrayData) actual; - LogicalType elementType = ((ArrayType) logicalType).getElementType(); - assertThat(actualArrayData.size()) - .as("array length should be equal") - .isEqualTo(expectedArrayData.size()); - assertArrayValues( - type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); - break; - case MAP: - assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - assertMapValues(type.asMapType(), logicalType, (Map) expected, (MapData) actual); - break; - case STRUCT: - assertThat(expected).as("Should expect a Record").isInstanceOf(StructLike.class); - assertRowData(type.asStructType(), logicalType, (StructLike) expected, (RowData) actual); - break; - case UUID: - assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); - long firstLong = bb.getLong(); - long secondLong = bb.getLong(); - assertThat(new UUID(firstLong, secondLong).toString()) - .as("UUID should be equal") - .isEqualTo(expected.toString()); - break; - case FIXED: - assertThat(actual) - .as("Should expect byte[]") - .isInstanceOf(byte[].class) - .isEqualTo(expected); - break; - default: - throw new IllegalArgumentException("Not a supported type: " + type); - } - } - - public static void assertEquals(Schema schema, List records, List rows) { - Streams.forEachPair( - records.stream(), rows.stream(), (record, row) -> assertEquals(schema, record, row)); - } - - public static void assertEquals(Schema schema, GenericData.Record record, Row row) { - List fields = schema.asStruct().fields(); - assertThat(fields).hasSameSizeAs(record.getSchema().getFields()); - assertThat(fields).hasSize(row.getArity()); - - RowType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < fields.size(); ++i) { - Type fieldType = fields.get(i).type(); - Object expectedValue = record.get(i); - Object actualValue = row.getField(i); - LogicalType logicalType = rowType.getTypeAt(i); - assertAvroEquals(fieldType, logicalType, expectedValue, actualValue); - } - } - - private static void assertEquals(Types.StructType struct, GenericData.Record record, Row row) { - List fields = struct.fields(); - for (int i = 0; i < fields.size(); i += 1) { - Type fieldType = fields.get(i).type(); - Object expectedValue = record.get(i); - Object actualValue = row.getField(i); - assertAvroEquals(fieldType, null, expectedValue, actualValue); - } - } - - private static void assertAvroEquals( - Type type, LogicalType logicalType, Object expected, Object actual) { - - if (expected == null && actual == null) { - return; - } - assertThat(expected).isNotNull(); - assertThat(actual).isNotNull(); - - switch (type.typeId()) { - case BOOLEAN: - case INTEGER: - case LONG: - case FLOAT: - case DOUBLE: - assertThat(expected) - .as("Should expect a " + type.typeId().javaClass()) - .isInstanceOf(type.typeId().javaClass()); - assertThat(actual) - .as("Should expect a " + type.typeId().javaClass()) - .isInstanceOf(type.typeId().javaClass()); - assertThat(actual).as(type.typeId() + " value should be equal").isEqualTo(expected); - break; - case STRING: - assertThat(expected).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); - assertThat(actual).as("Should expect a CharSequence").isInstanceOf(CharSequence.class); - assertThat(actual.toString()).as("string should be equal").isEqualTo(expected.toString()); - break; - case DATE: - assertThat(expected).as("Should expect a Date").isInstanceOf(LocalDate.class); - LocalDate date = DateTimeUtil.dateFromDays((int) actual); - assertThat(date).as("date should be equal").isEqualTo(expected); - break; - case TIME: - assertThat(expected).as("Should expect a LocalTime").isInstanceOf(LocalTime.class); - int milliseconds = (int) (((LocalTime) expected).toNanoOfDay() / 1000_000); - assertThat(actual).as("time millis should be equal").isEqualTo(milliseconds); - break; - case TIMESTAMP: - if (((Types.TimestampType) type).shouldAdjustToUTC()) { - assertThat(expected) - .as("Should expect a OffsetDataTime") - .isInstanceOf(OffsetDateTime.class); - OffsetDateTime ts = (OffsetDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("OffsetDataTime should be equal") - .isEqualTo(ts.toLocalDateTime()); - } else { - assertThat(expected) - .as("Should expect a LocalDataTime") - .isInstanceOf(LocalDateTime.class); - LocalDateTime ts = (LocalDateTime) expected; - assertThat(((TimestampData) actual).toLocalDateTime()) - .as("LocalDataTime should be equal") - .isEqualTo(ts); - } - break; - case BINARY: - assertThat(ByteBuffer.wrap((byte[]) actual)) - .as("Should expect a ByteBuffer") - .isInstanceOf(ByteBuffer.class) - .isEqualTo(expected); - break; - case DECIMAL: - assertThat(expected).as("Should expect a BigDecimal").isInstanceOf(BigDecimal.class); - BigDecimal bd = (BigDecimal) expected; - assertThat(((DecimalData) actual).toBigDecimal()) - .as("decimal value should be equal") - .isEqualTo(bd); - break; - case LIST: - assertThat(expected).as("Should expect a Collection").isInstanceOf(Collection.class); - Collection expectedArrayData = (Collection) expected; - ArrayData actualArrayData; - try { - actualArrayData = (ArrayData) actual; - } catch (ClassCastException e) { - actualArrayData = new GenericArrayData((Object[]) actual); - } - LogicalType elementType = ((ArrayType) logicalType).getElementType(); - assertThat(actualArrayData.size()) - .as("array length should be equal") - .isEqualTo(expectedArrayData.size()); - assertArrayValues( - type.asListType().elementType(), elementType, expectedArrayData, actualArrayData); - break; - case MAP: - assertThat(expected).as("Should expect a Map").isInstanceOf(Map.class); - MapData actualMap; - try { - actualMap = (MapData) actual; - } catch (ClassCastException e) { - actualMap = new GenericMapData((Map) actual); - } - assertMapValues(type.asMapType(), logicalType, (Map) expected, actualMap); - break; - case STRUCT: - assertThat(expected).as("Should expect a Record").isInstanceOf(GenericData.Record.class); - assertEquals( - type.asNestedType().asStructType(), (GenericData.Record) expected, (Row) actual); - break; - case UUID: - assertThat(expected).as("Should expect a UUID").isInstanceOf(UUID.class); - ByteBuffer bb = ByteBuffer.wrap((byte[]) actual); - long firstLong = bb.getLong(); - long secondLong = bb.getLong(); - assertThat(new UUID(firstLong, secondLong).toString()) - .as("UUID should be equal") - .isEqualTo(expected.toString()); - break; - case FIXED: - assertThat(actual) - .as("Should expect byte[]") - .isInstanceOf(byte[].class) - .isEqualTo(expected); - break; - default: - throw new IllegalArgumentException("Not a supported type: " + type); - } - } - - private static void assertArrayValues( - Type type, LogicalType logicalType, Collection expectedArray, ArrayData actualArray) { - List expectedElements = Lists.newArrayList(expectedArray); - for (int i = 0; i < expectedArray.size(); i += 1) { - if (expectedElements.get(i) == null) { - assertThat(actualArray.isNullAt(i)).isTrue(); - continue; - } - - Object expected = expectedElements.get(i); - - assertEquals( - type, - logicalType, - expected, - ArrayData.createElementGetter(logicalType).getElementOrNull(actualArray, i)); - } - } - - private static void assertMapValues( - Types.MapType mapType, LogicalType type, Map expected, MapData actual) { - assertThat(actual.size()).as("map size should be equal").isEqualTo(expected.size()); - - ArrayData actualKeyArrayData = actual.keyArray(); - ArrayData actualValueArrayData = actual.valueArray(); - LogicalType actualKeyType = ((MapType) type).getKeyType(); - LogicalType actualValueType = ((MapType) type).getValueType(); - Type keyType = mapType.keyType(); - Type valueType = mapType.valueType(); - - ArrayData.ElementGetter keyGetter = ArrayData.createElementGetter(actualKeyType); - ArrayData.ElementGetter valueGetter = ArrayData.createElementGetter(actualValueType); - - for (Map.Entry entry : expected.entrySet()) { - Object matchedActualKey = null; - int matchedKeyIndex = 0; - for (int i = 0; i < actual.size(); i += 1) { - try { - Object key = keyGetter.getElementOrNull(actualKeyArrayData, i); - assertEquals(keyType, actualKeyType, entry.getKey(), key); - matchedActualKey = key; - matchedKeyIndex = i; - break; - } catch (AssertionError e) { - // not found - } - } - assertThat(matchedActualKey).as("Should have a matching key").isNotNull(); - final int valueIndex = matchedKeyIndex; - assertEquals( - valueType, - actualValueType, - entry.getValue(), - valueGetter.getElementOrNull(actualValueArrayData, valueIndex)); - } - } - - public static void assertEquals(ManifestFile expected, ManifestFile actual) { - if (expected == actual) { - return; - } - assertThat(expected).isNotNull(); - assertThat(actual).isNotNull(); - assertThat(actual.path()).as("Path must match").isEqualTo(expected.path()); - assertThat(actual.length()).as("Length must match").isEqualTo(expected.length()); - assertThat(actual.partitionSpecId()) - .as("Spec id must match") - .isEqualTo(expected.partitionSpecId()); - assertThat(actual.content()).as("ManifestContent must match").isEqualTo(expected.content()); - assertThat(actual.sequenceNumber()) - .as("SequenceNumber must match") - .isEqualTo(expected.sequenceNumber()); - assertThat(actual.minSequenceNumber()) - .as("MinSequenceNumber must match") - .isEqualTo(expected.minSequenceNumber()); - assertThat(actual.snapshotId()).as("Snapshot id must match").isEqualTo(expected.snapshotId()); - assertThat(actual.hasAddedFiles()) - .as("Added files flag must match") - .isEqualTo(expected.hasAddedFiles()); - assertThat(actual.addedFilesCount()) - .as("Added files count must match") - .isEqualTo(expected.addedFilesCount()); - assertThat(actual.addedRowsCount()) - .as("Added rows count must match") - .isEqualTo(expected.addedRowsCount()); - assertThat(actual.hasExistingFiles()) - .as("Existing files flag must match") - .isEqualTo(expected.hasExistingFiles()); - assertThat(actual.existingFilesCount()) - .as("Existing files count must match") - .isEqualTo(expected.existingFilesCount()); - assertThat(actual.existingRowsCount()) - .as("Existing rows count must match") - .isEqualTo(expected.existingRowsCount()); - assertThat(actual.hasDeletedFiles()) - .as("Deleted files flag must match") - .isEqualTo(expected.hasDeletedFiles()); - assertThat(actual.deletedFilesCount()) - .as("Deleted files count must match") - .isEqualTo(expected.deletedFilesCount()); - assertThat(actual.deletedRowsCount()) - .as("Deleted rows count must match") - .isEqualTo(expected.deletedRowsCount()); - - List expectedSummaries = expected.partitions(); - List actualSummaries = actual.partitions(); - assertThat(actualSummaries) - .as("PartitionFieldSummary size does not match") - .hasSameSizeAs(expectedSummaries); - for (int i = 0; i < expectedSummaries.size(); i++) { - assertThat(actualSummaries.get(i).containsNull()) - .as("Null flag in partition must match") - .isEqualTo(expectedSummaries.get(i).containsNull()); - assertThat(actualSummaries.get(i).containsNaN()) - .as("NaN flag in partition must match") - .isEqualTo(expectedSummaries.get(i).containsNaN()); - assertThat(actualSummaries.get(i).lowerBound()) - .as("Lower bounds in partition must match") - .isEqualTo(expectedSummaries.get(i).lowerBound()); - assertThat(actualSummaries.get(i).upperBound()) - .as("Upper bounds in partition must match") - .isEqualTo(expectedSummaries.get(i).upperBound()); - } - } - - public static void assertEquals(ContentFile expected, ContentFile actual) { - if (expected == actual) { - return; - } - assertThat(expected).isNotNull(); - 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.format()).as("Format").isEqualTo(expected.format()); - assertThat(actual.partition().size()) - .as("Partition size") - .isEqualTo(expected.partition().size()); - for (int i = 0; i < expected.partition().size(); i++) { - assertThat(actual.partition().get(i, Object.class)) - .as("Partition data at index " + i) - .isEqualTo(expected.partition().get(i, Object.class)); - } - assertThat(actual.recordCount()).as("Record count").isEqualTo(expected.recordCount()); - assertThat(actual.fileSizeInBytes()) - .as("File size in bytes") - .isEqualTo(expected.fileSizeInBytes()); - assertThat(actual.columnSizes()).as("Column sizes").isEqualTo(expected.columnSizes()); - assertThat(actual.valueCounts()).as("Value counts").isEqualTo(expected.valueCounts()); - assertThat(actual.nullValueCounts()) - .as("Null value counts") - .isEqualTo(expected.nullValueCounts()); - assertThat(actual.lowerBounds()).as("Lower bounds").isEqualTo(expected.lowerBounds()); - assertThat(actual.upperBounds()).as("Upper bounds").isEqualTo(expected.upperBounds()); - assertThat(actual.keyMetadata()).as("Key metadata").isEqualTo(expected.keyMetadata()); - assertThat(actual.splitOffsets()).as("Split offsets").isEqualTo(expected.splitOffsets()); - assertThat(actual.equalityFieldIds()) - .as("Equality field id list") - .isEqualTo(expected.equalityFieldIds()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java deleted file mode 100644 index b709c0058f7d..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ /dev/null @@ -1,343 +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; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import java.io.UncheckedIOException; -import java.nio.file.Files; -import java.util.Map; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.catalog.Catalog; -import org.apache.flink.table.catalog.ObjectPath; -import org.apache.flink.types.Row; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -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.thrift.TException; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestIcebergConnector extends TestBase { - - private static final String TABLE_NAME = "test_table"; - - @Parameter(index = 0) - private String catalogName; - - @Parameter(index = 1) - private Map properties; - - @Parameter(index = 2) - private boolean isStreaming; - - private volatile TableEnvironment tEnv; - - @Parameters(name = "catalogName = {0}, properties = {1}, isStreaming = {2}") - public static Iterable parameters() { - return Lists.newArrayList( - // Create iceberg table in the hadoop catalog and default database. - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop"), - false - }, - // Create iceberg table in the hadoop catalog and not_existing_db. - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhadoop", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hadoop", - "catalog-database", "not_existing_db"), - false - }, - // Create iceberg table in the hive catalog and default database. - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive"), - false - }, - // Create iceberg table in the hive catalog and not_existing_db. - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db", - "catalog-table", "not_existing_table"), - true - }, - new Object[] { - "testhive", - ImmutableMap.of( - "connector", "iceberg", - "catalog-type", "hive", - "catalog-database", "not_existing_db"), - false - }); - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); - if (isStreaming) { - settingsBuilder.inStreamingMode(); - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - env.setMaxParallelism(2); - env.setParallelism(2); - tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); - } else { - settingsBuilder.inBatchMode(); - tEnv = TableEnvironment.create(settingsBuilder.build()); - } - // Set only one parallelism. - tEnv.getConfig() - .getConfiguration() - .set(CoreOptions.DEFAULT_PARALLELISM, 1) - .set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - } - } - } - return tEnv; - } - - @AfterEach - public void after() throws TException { - sql("DROP TABLE IF EXISTS %s", TABLE_NAME); - - // Clean the created orphan databases and tables from hive-metastore. - if (isHiveCatalog()) { - HiveMetaStoreClient metaStoreClient = new HiveMetaStoreClient(hiveConf); - try { - metaStoreClient.dropTable(databaseName(), tableName()); - if (!isDefaultDatabaseName()) { - try { - metaStoreClient.dropDatabase(databaseName()); - } catch (Exception ignored) { - // Ignore - } - } - } finally { - metaStoreClient.close(); - } - } - } - - private void testCreateConnectorTable() { - Map tableProps = createTableProps(); - - // Create table under the flink's current database. - sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - sql("INSERT INTO %s VALUES (1, 'AAA'), (2, 'BBB'), (3, 'CCC')", TABLE_NAME); - assertThat(sql("SELECT * FROM %s", TABLE_NAME)) - .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); - - FlinkCatalogFactory factory = new FlinkCatalogFactory(); - Catalog flinkCatalog = factory.createCatalog(catalogName, tableProps, new Configuration()); - assertThat(flinkCatalog.databaseExists(databaseName())).isTrue(); - assertThat(flinkCatalog.tableExists(new ObjectPath(databaseName(), tableName()))).isTrue(); - - // Drop and create it again. - sql("DROP TABLE %s", TABLE_NAME); - sql("CREATE TABLE %s (id BIGINT, data STRING) WITH %s", TABLE_NAME, toWithClause(tableProps)); - assertThat(sql("SELECT * FROM %s", TABLE_NAME)) - .containsExactlyInAnyOrder(Row.of(1L, "AAA"), Row.of(2L, "BBB"), Row.of(3L, "CCC")); - } - - @TestTemplate - public void testCreateTableUnderDefaultDatabase() { - testCreateConnectorTable(); - } - - @TestTemplate - public void testCatalogDatabaseConflictWithFlinkDatabase() { - sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); - sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } - } - - @TestTemplate - public void testConnectorTableInIcebergCatalog() { - // Create the catalog properties - Map catalogProps = Maps.newHashMap(); - catalogProps.put("type", "iceberg"); - if (isHiveCatalog()) { - catalogProps.put("catalog-type", "hive"); - catalogProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - } else { - catalogProps.put("catalog-type", "hadoop"); - } - catalogProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); - - // Create the table properties - Map tableProps = createTableProps(); - - // Create a connector table in an iceberg catalog. - sql("CREATE CATALOG `test_catalog` WITH %s", toWithClause(catalogProps)); - try { - assertThatThrownBy( - () -> - sql( - "CREATE TABLE `test_catalog`.`%s`.`%s` (id BIGINT, data STRING) WITH %s", - FlinkCatalogFactory.DEFAULT_DATABASE_NAME, - TABLE_NAME, - toWithClause(tableProps))) - .cause() - .isInstanceOf(IllegalArgumentException.class) - .hasMessage( - "Cannot create the table with 'connector'='iceberg' table property in an iceberg catalog, " - + "Please create table with 'connector'='iceberg' property in a non-iceberg catalog or " - + "create table without 'connector'='iceberg' related properties in an iceberg table."); - } finally { - sql("DROP CATALOG IF EXISTS `test_catalog`"); - } - } - - private Map createTableProps() { - Map tableProps = Maps.newHashMap(properties); - tableProps.put("catalog-name", catalogName); - tableProps.put(CatalogProperties.WAREHOUSE_LOCATION, createWarehouse()); - if (isHiveCatalog()) { - tableProps.put(CatalogProperties.URI, CatalogTestBase.getURI(hiveConf)); - } - return tableProps; - } - - private boolean isHiveCatalog() { - return "testhive".equalsIgnoreCase(catalogName); - } - - private boolean isDefaultDatabaseName() { - return FlinkCatalogFactory.DEFAULT_DATABASE_NAME.equalsIgnoreCase(databaseName()); - } - - private String tableName() { - return properties.getOrDefault("catalog-table", TABLE_NAME); - } - - private String databaseName() { - return properties.getOrDefault("catalog-database", "default_database"); - } - - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - - private String createWarehouse() { - try { - return String.format( - "file://%s", - Files.createTempDirectory(temporaryDirectory, "junit").toFile().getAbsolutePath()); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java deleted file mode 100644 index 8f1f129e183b..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestManifestFileSerialization.java +++ /dev/null @@ -1,173 +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; - -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.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.File; -import java.io.IOException; -import java.io.ObjectInputStream; -import java.io.ObjectOutputStream; -import java.nio.ByteBuffer; -import java.nio.ByteOrder; -import java.nio.file.Path; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.java.typeutils.runtime.kryo.KryoSerializer; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.GenericManifestFile; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.ManifestWriter; -import org.apache.iceberg.Metrics; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.io.OutputFile; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestManifestFileSerialization { - - private static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - required(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("double").build(); - - private static final DataFile FILE_A = - DataFiles.builder(SPEC) - .withPath("/path/to/data-1.parquet") - .withFileSizeInBytes(0) - .withPartition(org.apache.iceberg.TestHelpers.Row.of(1D)) - .withPartitionPath("double=1") - .withMetrics( - new Metrics( - 5L, - null, // no column sizes - ImmutableMap.of(1, 5L, 2, 3L), // value count - ImmutableMap.of(1, 0L, 2, 2L), // null count - ImmutableMap.of(), // nan count - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(4L)) // upper bounds - )) - .build(); - - private static final DataFile FILE_B = - DataFiles.builder(SPEC) - .withPath("/path/to/data-2.parquet") - .withFileSizeInBytes(0) - .withPartition(org.apache.iceberg.TestHelpers.Row.of(Double.NaN)) - .withPartitionPath("double=NaN") - .withMetrics( - new Metrics( - 1L, - null, // no column sizes - ImmutableMap.of(1, 1L, 4, 1L), // value count - ImmutableMap.of(1, 0L, 2, 0L), // null count - ImmutableMap.of(4, 1L), // nan count - ImmutableMap.of(1, longToBuffer(0L)), // lower bounds - ImmutableMap.of(1, longToBuffer(1L)) // upper bounds - )) - .build(); - - private static final FileIO FILE_IO = new HadoopFileIO(new Configuration()); - - @TempDir private Path temp; - - @Test - public void testKryoSerialization() throws IOException { - KryoSerializer kryo = - new KryoSerializer<>(ManifestFile.class, new ExecutionConfig()); - - DataOutputSerializer outputView = new DataOutputSerializer(1024); - - ManifestFile manifest = writeManifest(FILE_A, FILE_B); - - kryo.serialize(manifest, outputView); - kryo.serialize(manifest.copy(), outputView); - kryo.serialize(GenericManifestFile.copyOf(manifest).build(), outputView); - - DataInputDeserializer inputView = new DataInputDeserializer(outputView.getCopyOfBuffer()); - ManifestFile m1 = kryo.deserialize(inputView); - ManifestFile m2 = kryo.deserialize(inputView); - ManifestFile m3 = kryo.deserialize(inputView); - - TestHelpers.assertEquals(manifest, m1); - TestHelpers.assertEquals(manifest, m2); - TestHelpers.assertEquals(manifest, m3); - } - - @Test - public void testJavaSerialization() throws Exception { - ByteArrayOutputStream bytes = new ByteArrayOutputStream(); - - ManifestFile manifest = writeManifest(FILE_A, FILE_B); - - try (ObjectOutputStream out = new ObjectOutputStream(bytes)) { - out.writeObject(manifest); - out.writeObject(manifest.copy()); - out.writeObject(GenericManifestFile.copyOf(manifest).build()); - } - - try (ObjectInputStream in = - new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray()))) { - for (int i = 0; i < 3; i += 1) { - Object obj = in.readObject(); - assertThat(obj).as("Should be a ManifestFile").isInstanceOf(ManifestFile.class); - TestHelpers.assertEquals(manifest, (ManifestFile) obj); - } - } - } - - private ManifestFile writeManifest(DataFile... files) throws IOException { - File manifestFile = File.createTempFile("input", "m0.avro", temp.toFile()); - assertThat(manifestFile.delete()).isTrue(); - OutputFile outputFile = FILE_IO.newOutputFile(manifestFile.getCanonicalPath()); - - ManifestWriter writer = ManifestFiles.write(SPEC, outputFile); - try { - for (DataFile file : files) { - writer.add(file); - } - } finally { - writer.close(); - } - - return writer.toManifestFile(); - } - - private static ByteBuffer longToBuffer(long value) { - return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java deleted file mode 100644 index 0af49e9e2365..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestRowDataWrapper.java +++ /dev/null @@ -1,93 +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; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Iterator; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.RecordWrapperTest; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.util.StructLikeWrapper; - -public class TestRowDataWrapper extends RecordWrapperTest { - - /** - * Flink's time type has been truncated to millis seconds, so we need a customized assert method - * to check the values. - */ - @Override - public void testTime() { - generateAndValidate( - new Schema(TIME.fields()), - (message, expectedWrapper, actualWrapper) -> { - for (int pos = 0; pos < TIME.fields().size(); pos++) { - Object expected = expectedWrapper.get().get(pos, Object.class); - Object actual = actualWrapper.get().get(pos, Object.class); - if (expected == actual) { - return; - } - - assertThat(actual).isNotNull(); - assertThat(expected).isNotNull(); - - int expectedMilliseconds = (int) ((long) expected / 1000_000); - int actualMilliseconds = (int) ((long) actual / 1000_000); - assertThat(actualMilliseconds).as(message).isEqualTo(expectedMilliseconds); - } - }); - } - - @Override - protected void generateAndValidate(Schema schema, RecordWrapperTest.AssertMethod assertMethod) { - int numRecords = 100; - Iterable recordList = RandomGenericData.generate(schema, numRecords, 101L); - Iterable rowDataList = RandomRowData.generate(schema, numRecords, 101L); - - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(schema.asStruct()); - RowDataWrapper rowDataWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - - Iterator actual = recordList.iterator(); - Iterator expected = rowDataList.iterator(); - - StructLikeWrapper actualWrapper = StructLikeWrapper.forType(schema.asStruct()); - StructLikeWrapper expectedWrapper = StructLikeWrapper.forType(schema.asStruct()); - for (int i = 0; i < numRecords; i++) { - assertThat(actual).hasNext(); - assertThat(expected).hasNext(); - - StructLike recordStructLike = recordWrapper.wrap(actual.next()); - StructLike rowDataStructLike = rowDataWrapper.wrap(expected.next()); - - assertMethod.assertEquals( - "Should have expected StructLike values", - expectedWrapper.set(rowDataStructLike), - actualWrapper.set(recordStructLike)); - } - - assertThat(actual).isExhausted(); - assertThat(expected).isExhausted(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java deleted file mode 100644 index a7c58e551112..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableLoader.java +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink; - -import java.io.File; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestTables; - -public class TestTableLoader implements TableLoader { - private final File dir; - - public static TableLoader of(String dir) { - return new TestTableLoader(dir); - } - - public TestTableLoader(String dir) { - this.dir = new File(dir); - } - - @Override - public void open() {} - - @Override - public boolean isOpen() { - return true; - } - - @Override - public Table loadTable() { - return TestTables.load(dir, "test"); - } - - @Override - @SuppressWarnings({"checkstyle:NoClone", "checkstyle:SuperClone"}) - public TableLoader clone() { - return new TestTableLoader(dir.getAbsolutePath()); - } - - @Override - public void close() {} -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java deleted file mode 100644 index 7f0e7acaa822..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/TestTableSerialization.java +++ /dev/null @@ -1,110 +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; - -import static org.apache.iceberg.flink.TestHelpers.roundTripKryoSerialize; -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.Map; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableOperations; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.Transaction; -import org.apache.iceberg.hadoop.HadoopTables; -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; - -public class TestTableSerialization { - private static final HadoopTables TABLES = new HadoopTables(); - - private static final Schema SCHEMA = - new Schema( - required(1, "id", Types.LongType.get()), - optional(2, "data", Types.StringType.get()), - required(3, "date", Types.StringType.get()), - optional(4, "double", Types.DoubleType.get())); - - private static final PartitionSpec SPEC = - PartitionSpec.builderFor(SCHEMA).identity("date").build(); - - private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - - @TempDir private Path temp; - private Table table; - - @BeforeEach - public void initTable() throws IOException { - Map props = ImmutableMap.of("k1", "v1", "k2", "v2"); - - File tableLocation = File.createTempFile("junit", null, temp.toFile()); - assertThat(tableLocation.delete()).isTrue(); - - this.table = TABLES.create(SCHEMA, SPEC, SORT_ORDER, props, tableLocation.toString()); - } - - @Test - public void testSerializableTableKryoSerialization() throws IOException { - SerializableTable serializableTable = (SerializableTable) SerializableTable.copyOf(table); - TestHelpers.assertSerializedAndLoadedMetadata( - table, roundTripKryoSerialize(SerializableTable.class, serializableTable)); - } - - @Test - public void testSerializableMetadataTableKryoSerialization() throws IOException { - for (MetadataTableType type : MetadataTableType.values()) { - TableOperations ops = ((HasTableOperations) table).operations(); - Table metadataTable = - MetadataTableUtils.createMetadataTableInstance(ops, table.name(), "meta", type); - SerializableTable serializableMetadataTable = - (SerializableTable) SerializableTable.copyOf(metadataTable); - - TestHelpers.assertSerializedAndLoadedMetadata( - metadataTable, - roundTripKryoSerialize(SerializableTable.class, serializableMetadataTable)); - } - } - - @Test - public void testSerializableTransactionTableKryoSerialization() throws IOException { - Transaction txn = table.newTransaction(); - - txn.updateProperties().set("k1", "v1").commit(); - - Table txnTable = txn.table(); - SerializableTable serializableTxnTable = (SerializableTable) SerializableTable.copyOf(txnTable); - - TestHelpers.assertSerializedMetadata( - txnTable, roundTripKryoSerialize(SerializableTable.class, serializableTxnTable)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java deleted file mode 100644 index 800cce96edac..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/actions/TestRewriteDataFilesAction.java +++ /dev/null @@ -1,481 +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.actions; - -import static org.apache.iceberg.flink.SimpleDataUtil.RECORD; -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.Path; -import java.util.Collection; -import java.util.List; -import java.util.Set; -import java.util.UUID; -import java.util.stream.Collectors; -import org.apache.commons.lang3.StringUtils; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Files; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.actions.RewriteDataFilesActionResult; -import org.apache.iceberg.catalog.Namespace; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.exceptions.ValidationException; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -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.types.Types; -import org.apache.iceberg.util.Pair; -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; - -public class TestRewriteDataFilesAction extends CatalogTestBase { - - private static final String TABLE_NAME_UNPARTITIONED = "test_table_unpartitioned"; - private static final String TABLE_NAME_PARTITIONED = "test_table_partitioned"; - private static final String TABLE_NAME_WITH_PK = "test_table_with_pk"; - - @Parameter(index = 2) - private FileFormat format; - - private Table icebergTableUnPartitioned; - private Table icebergTablePartitioned; - private Table icebergTableWithPk; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @Parameters(name = "catalogName={0}, baseNamespace={1}, format={2}") - 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}); - } - } - return parameters; - } - - private @TempDir Path temp; - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - 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()); - 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()); - 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()); - icebergTableWithPk = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_UNPARTITIONED); - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_PARTITIONED); - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME_WITH_PK); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testRewriteDataFilesEmptyTable() throws Exception { - assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); - Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); - assertThat(icebergTableUnPartitioned.currentSnapshot()).isNull(); - } - - @TestTemplate - public void testRewriteDataFilesUnpartitionedTable() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(2); - RewriteDataFilesActionResult result = - Actions.forTable(icebergTableUnPartitioned).rewriteDataFiles().execute(); - - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFiles1).hasSize(1); - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTableUnPartitioned, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), SimpleDataUtil.createRecord(2, "world"))); - } - - @TestTemplate - public void testRewriteDataFilesPartitionedTable() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 3, 'world' ,'b'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(4); - RewriteDataFilesActionResult result = - Actions.forTable(icebergTablePartitioned).rewriteDataFiles().execute(); - - assertThat(result.deletedDataFiles()).hasSize(4); - assertThat(result.addedDataFiles()).hasSize(2); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFiles1).hasSize(2); - // Assert the table records as expected. - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "spec", Types.StringType.get())); - - Record record = GenericRecord.create(schema); - SimpleDataUtil.assertTableRecords( - icebergTablePartitioned, - Lists.newArrayList( - record.copy("id", 1, "data", "hello", "spec", "a"), - record.copy("id", 2, "data", "hello", "spec", "a"), - record.copy("id", 3, "data", "world", "spec", "b"), - record.copy("id", 4, "data", "world", "spec", "b"))); - } - - @TestTemplate - public void testRewriteDataFilesWithFilter() throws Exception { - sql("INSERT INTO %s SELECT 1, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 2, 'hello' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 3, 'world' ,'a'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 4, 'world' ,'b'", TABLE_NAME_PARTITIONED); - sql("INSERT INTO %s SELECT 5, 'world' ,'b'", TABLE_NAME_PARTITIONED); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks = icebergTablePartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(5); - RewriteDataFilesActionResult result = - Actions.forTable(icebergTablePartitioned) - .rewriteDataFiles() - .filter(Expressions.equal("spec", "a")) - .filter(Expressions.startsWith("data", "he")) - .execute(); - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - - icebergTablePartitioned.refresh(); - - CloseableIterable tasks1 = icebergTablePartitioned.newScan().planFiles(); - List dataFiles1 = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFiles1).hasSize(4); - // Assert the table records as expected. - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get()), - Types.NestedField.optional(3, "spec", Types.StringType.get())); - - Record record = GenericRecord.create(schema); - SimpleDataUtil.assertTableRecords( - icebergTablePartitioned, - Lists.newArrayList( - record.copy("id", 1, "data", "hello", "spec", "a"), - record.copy("id", 2, "data", "hello", "spec", "a"), - record.copy("id", 3, "data", "world", "spec", "a"), - record.copy("id", 4, "data", "world", "spec", "b"), - record.copy("id", 5, "data", "world", "spec", "b"))); - } - - @TestTemplate - public void testRewriteLargeTableHasResiduals() throws IOException { - // all records belong to the same partition - List records1 = Lists.newArrayList(); - List records2 = Lists.newArrayList(); - List expected = Lists.newArrayList(); - for (int i = 0; i < 100; i++) { - int id = i; - String data = String.valueOf(i % 3); - if (i % 2 == 0) { - records1.add("(" + id + ",'" + data + "')"); - } else { - records2.add("(" + id + ",'" + data + "')"); - } - Record record = RECORD.copy(); - record.setField("id", id); - record.setField("data", data); - expected.add(record); - } - - sql("INSERT INTO %s values " + StringUtils.join(records1, ","), TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s values " + StringUtils.join(records2, ","), TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = - icebergTableUnPartitioned - .newScan() - .ignoreResiduals() - .filter(Expressions.equal("data", "0")) - .planFiles(); - for (FileScanTask task : tasks) { - assertThat(task.residual()) - .as("Residuals must be ignored") - .isEqualTo(Expressions.alwaysTrue()); - } - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(2); - Actions actions = Actions.forTable(icebergTableUnPartitioned); - - RewriteDataFilesActionResult result = - actions.rewriteDataFiles().filter(Expressions.equal("data", "0")).execute(); - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); - } - - /** - * a test case to test avoid repeate compress - * - *

    If datafile cannot be combined to CombinedScanTask with other DataFiles, the size of the - * CombinedScanTask list size is 1, so we remove these CombinedScanTasks to avoid compressed - * repeatedly. - * - *

    In this test case,we generated 3 data files and set targetSizeInBytes greater than the - * largest file size so that it cannot be combined a CombinedScanTask with other datafiles. The - * datafile with the largest file size will not be compressed. - * - * @throws IOException IOException - */ - @TestTemplate - public void testRewriteAvoidRepeateCompress() throws IOException { - List expected = Lists.newArrayList(); - Schema schema = icebergTableUnPartitioned.schema(); - GenericAppenderFactory genericAppenderFactory = new GenericAppenderFactory(schema); - File file = File.createTempFile("junit", null, temp.toFile()); - int count = 0; - try (FileAppender fileAppender = - genericAppenderFactory.newAppender(Files.localOutput(file), format)) { - long filesize = 20000; - for (; fileAppender.length() < filesize; count++) { - Record record = SimpleDataUtil.createRecord(count, UUID.randomUUID().toString()); - fileAppender.add(record); - expected.add(record); - } - } - - DataFile dataFile = - DataFiles.builder(icebergTableUnPartitioned.spec()) - .withPath(file.getAbsolutePath()) - .withFileSizeInBytes(file.length()) - .withFormat(format) - .withRecordCount(count) - .build(); - - icebergTableUnPartitioned.newAppend().appendFile(dataFile).commit(); - - sql("INSERT INTO %s SELECT 1,'a' ", TABLE_NAME_UNPARTITIONED); - sql("INSERT INTO %s SELECT 2,'b' ", TABLE_NAME_UNPARTITIONED); - - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks = icebergTableUnPartitioned.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - assertThat(dataFiles).hasSize(3); - Actions actions = Actions.forTable(icebergTableUnPartitioned); - - long targetSizeInBytes = file.length() + 10; - RewriteDataFilesActionResult result = - actions - .rewriteDataFiles() - .targetSizeInBytes(targetSizeInBytes) - .splitOpenFileCost(1) - .execute(); - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - icebergTableUnPartitioned.refresh(); - - CloseableIterable tasks1 = icebergTableUnPartitioned.newScan().planFiles(); - List dataFilesRewrote = - Lists.newArrayList(CloseableIterable.transform(tasks1, FileScanTask::file)); - assertThat(dataFilesRewrote).hasSize(2); - // the biggest file do not be rewrote - List rewroteDataFileNames = - dataFilesRewrote.stream().map(ContentFile::path).collect(Collectors.toList()); - assertThat(rewroteDataFileNames).contains(file.getAbsolutePath()); - - // Assert the table records as expected. - expected.add(SimpleDataUtil.createRecord(1, "a")); - expected.add(SimpleDataUtil.createRecord(2, "b")); - SimpleDataUtil.assertTableRecords(icebergTableUnPartitioned, expected); - } - - @TestTemplate - public void testRewriteNoConflictWithEqualityDeletes() throws IOException { - // Add 2 data files - sql("INSERT INTO %s SELECT 1, 'hello'", TABLE_NAME_WITH_PK); - sql("INSERT INTO %s SELECT 2, 'world'", TABLE_NAME_WITH_PK); - - // Load 2 stale tables to pass to rewrite actions - // Since the first rewrite will refresh stale1, we need another stale2 for the second rewrite - Table stale1 = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - Table stale2 = - validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME_WITH_PK)); - - // Add 1 data file and 1 equality-delete file - sql("INSERT INTO %s /*+ OPTIONS('upsert-enabled'='true')*/ SELECT 1, 'hi'", TABLE_NAME_WITH_PK); - - icebergTableWithPk.refresh(); - assertThat(icebergTableWithPk.currentSnapshot().sequenceNumber()) - .as("The latest sequence number should be greater than that of the stale snapshot") - .isEqualTo(stale1.currentSnapshot().sequenceNumber() + 1); - CloseableIterable tasks = icebergTableWithPk.newScan().planFiles(); - List dataFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::file)); - Set deleteFiles = - Lists.newArrayList(CloseableIterable.transform(tasks, FileScanTask::deletes)).stream() - .flatMap(Collection::stream) - .collect(Collectors.toSet()); - assertThat(dataFiles).hasSize(3); - assertThat(deleteFiles).hasSize(1); - assertThat(Iterables.getOnlyElement(deleteFiles).content()) - .isEqualTo(FileContent.EQUALITY_DELETES); - shouldHaveDataAndFileSequenceNumbers( - TABLE_NAME_WITH_PK, - ImmutableList.of(Pair.of(1L, 1L), Pair.of(2L, 2L), Pair.of(3L, 3L), Pair.of(3L, 3L))); - - assertThatThrownBy( - () -> - Actions.forTable(stale1) - .rewriteDataFiles() - .useStartingSequenceNumber(false) - .execute(), - "Rewrite using new sequence number should fail") - .isInstanceOf(ValidationException.class); - - // Rewrite using the starting sequence number should succeed - RewriteDataFilesActionResult result = - Actions.forTable(stale2).rewriteDataFiles().useStartingSequenceNumber(true).execute(); - - // Should not rewrite files from the new commit - assertThat(result.deletedDataFiles()).hasSize(2); - assertThat(result.addedDataFiles()).hasSize(1); - // The 2 older files with file-sequence-number <= 2 should be rewritten into a new file. - // The new file is the one with file-sequence-number == 4. - // The new file should use rewrite's starting-sequence-number 2 as its data-sequence-number. - shouldHaveDataAndFileSequenceNumbers( - TABLE_NAME_WITH_PK, ImmutableList.of(Pair.of(3L, 3L), Pair.of(3L, 3L), Pair.of(2L, 4L))); - - // Assert the table records as expected. - SimpleDataUtil.assertTableRecords( - icebergTableWithPk, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hi"), SimpleDataUtil.createRecord(2, "world"))); - } - - /** - * Assert that data files and delete files in the table should have expected data sequence numbers - * and file sequence numbers - * - * @param tableName table name - * @param expectedSequenceNumbers list of {@link Pair}'s. Each {@link Pair} contains - * (expectedDataSequenceNumber, expectedFileSequenceNumber) of a file. - */ - private void shouldHaveDataAndFileSequenceNumbers( - String tableName, List> expectedSequenceNumbers) { - // "status < 2" for added or existing entries - List liveEntries = sql("SELECT * FROM %s$entries WHERE status < 2", tableName); - - List> actualSequenceNumbers = - liveEntries.stream() - .map( - row -> - Pair.of( - row.getFieldAs("sequence_number"), row.getFieldAs("file_sequence_number"))) - .collect(Collectors.toList()); - assertThat(actualSequenceNumbers).hasSameElementsAs(expectedSequenceNumbers); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.java deleted file mode 100644 index cc58d9817ac6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RandomRowData.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 org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; - -public class RandomRowData { - private RandomRowData() {} - - public static Iterable generate(Schema schema, int numRecords, long seed) { - return convert(schema, RandomGenericData.generate(schema, numRecords, seed)); - } - - public static Iterable convert(Schema schema, Iterable records) { - return Iterables.transform(records, record -> RowDataConverter.convert(schema, record)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java deleted file mode 100644 index 74b1da6007e6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/RowDataToRowMapper.java +++ /dev/null @@ -1,50 +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 org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.configuration.Configuration; -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; - -public class RowDataToRowMapper extends RichMapFunction { - - private final RowType rowType; - - private transient DataStructureConverter converter; - - public RowDataToRowMapper(RowType rowType) { - this.rowType = rowType; - } - - @Override - public void open(Configuration parameters) throws Exception { - this.converter = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(rowType)); - } - - @Override - public Row map(RowData value) throws Exception { - return (Row) converter.toExternal(value); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java deleted file mode 100644 index a1039d27d888..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkAvroReaderWriter.java +++ /dev/null @@ -1,185 +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 static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.sql.Date; -import java.sql.Time; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.data.DataTest; -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.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -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.iceberg.util.DateTimeUtil; -import org.junit.jupiter.api.Test; - -public class TestFlinkAvroReaderWriter extends DataTest { - - private static final int NUM_RECORDS = 100; - - private static final Schema SCHEMA_NUM_TYPE = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "int", Types.IntegerType.get()), - Types.NestedField.optional(3, "float", Types.FloatType.get()), - Types.NestedField.optional(4, "double", Types.DoubleType.get()), - Types.NestedField.optional(5, "date", Types.DateType.get()), - Types.NestedField.optional(6, "time", Types.TimeType.get()), - Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone()), - Types.NestedField.optional(8, "bigint", Types.LongType.get()), - Types.NestedField.optional(9, "decimal", Types.DecimalType.of(4, 2))); - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1991L); - writeAndValidate(schema, expectedRecords, NUM_RECORDS); - } - - private void writeAndValidate(Schema schema, List expectedRecords, int numRecord) - throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - - File recordsFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(recordsFile.delete()).isTrue(); - - // Write the expected records into AVRO file, then read them into RowData and assert with the - // expected Record list. - try (FileAppender writer = - Avro.write(Files.localOutput(recordsFile)) - .schema(schema) - .createWriterFunc(DataWriter::create) - .build()) { - writer.addAll(expectedRecords); - } - - try (CloseableIterable reader = - Avro.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(FlinkAvroReader::new) - .build()) { - Iterator expected = expectedRecords.iterator(); - Iterator rows = reader.iterator(); - for (int i = 0; i < numRecord; i++) { - assertThat(rows).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); - } - assertThat(rows).isExhausted(); - } - - File rowDataFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(rowDataFile.delete()).isTrue(); - - // Write the expected RowData into AVRO file, then read them into Record and assert with the - // expected RowData list. - try (FileAppender writer = - Avro.write(Files.localOutput(rowDataFile)) - .schema(schema) - .createWriterFunc(ignore -> new FlinkAvroWriter(flinkSchema)) - .build()) { - writer.addAll(expectedRows); - } - - try (CloseableIterable reader = - Avro.read(Files.localInput(rowDataFile)) - .project(schema) - .createReaderFunc(DataReader::create) - .build()) { - Iterator expected = expectedRows.iterator(); - Iterator records = reader.iterator(); - for (int i = 0; i < numRecord; i += 1) { - assertThat(records).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); - } - assertThat(records).isExhausted(); - } - } - - private Record recordNumType( - int id, - int intV, - float floatV, - double doubleV, - long date, - long time, - long timestamp, - long bigint, - double decimal) { - Record record = GenericRecord.create(SCHEMA_NUM_TYPE); - record.setField("id", id); - record.setField("int", intV); - record.setField("float", floatV); - record.setField("double", doubleV); - record.setField( - "date", DateTimeUtil.dateFromDays((int) new Date(date).toLocalDate().toEpochDay())); - record.setField("time", new Time(time).toLocalTime()); - record.setField("timestamp", DateTimeUtil.timestampFromMicros(timestamp * 1000)); - record.setField("bigint", bigint); - record.setField("decimal", BigDecimal.valueOf(decimal)); - return record; - } - - @Test - public void testNumericTypes() throws IOException { - - List expected = - ImmutableList.of( - recordNumType( - 2, - Integer.MAX_VALUE, - Float.MAX_VALUE, - Double.MAX_VALUE, - Long.MAX_VALUE, - 1643811742000L, - 1643811742000L, - 1643811742000L, - 10.24d), - recordNumType( - 2, - Integer.MIN_VALUE, - Float.MIN_VALUE, - Double.MIN_VALUE, - Long.MIN_VALUE, - 1643811742000L, - 1643811742000L, - 1643811742000L, - 10.24d)); - - writeAndValidate(SCHEMA_NUM_TYPE, expected, 2); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java deleted file mode 100644 index 91ee017238ac..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkOrcReaderWriter.java +++ /dev/null @@ -1,107 +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 static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.orc.GenericOrcReader; -import org.apache.iceberg.data.orc.GenericOrcWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.orc.ORC; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class TestFlinkOrcReaderWriter extends DataTest { - private static final int NUM_RECORDS = 100; - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - RowType flinkSchema = FlinkSchemaUtil.convert(schema); - List expectedRecords = RandomGenericData.generate(schema, NUM_RECORDS, 1990L); - List expectedRows = Lists.newArrayList(RandomRowData.convert(schema, expectedRecords)); - - File recordsFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(recordsFile.delete()).isTrue(); - - // Write the expected records into ORC file, then read them into RowData and assert with the - // expected Record list. - try (FileAppender writer = - ORC.write(Files.localOutput(recordsFile)) - .schema(schema) - .createWriterFunc(GenericOrcWriter::buildWriter) - .build()) { - writer.addAll(expectedRecords); - } - - try (CloseableIterable reader = - ORC.read(Files.localInput(recordsFile)) - .project(schema) - .createReaderFunc(type -> new FlinkOrcReader(schema, type)) - .build()) { - Iterator expected = expectedRecords.iterator(); - Iterator rows = reader.iterator(); - for (int i = 0; i < NUM_RECORDS; i++) { - assertThat(rows.hasNext()).isTrue(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, expected.next(), rows.next()); - } - assertThat(rows).isExhausted(); - } - - File rowDataFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(rowDataFile.delete()).isTrue(); - - // Write the expected RowData into ORC file, then read them into Record and assert with the - // expected RowData list. - RowType rowType = FlinkSchemaUtil.convert(schema); - try (FileAppender writer = - ORC.write(Files.localOutput(rowDataFile)) - .schema(schema) - .createWriterFunc((iSchema, typeDesc) -> FlinkOrcWriter.buildWriter(rowType, iSchema)) - .build()) { - writer.addAll(expectedRows); - } - - try (CloseableIterable reader = - ORC.read(Files.localInput(rowDataFile)) - .project(schema) - .createReaderFunc(type -> GenericOrcReader.buildReader(schema, type)) - .build()) { - Iterator expected = expectedRows.iterator(); - Iterator records = reader.iterator(); - for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(records).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), flinkSchema, records.next(), expected.next()); - } - assertThat(records).isExhausted(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java deleted file mode 100644 index 4cfb24f62921..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetReader.java +++ /dev/null @@ -1,239 +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 static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.parquet.schema.Types.primitive; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.ByteBuffer; -import java.util.Iterator; -import java.util.List; -import org.apache.avro.generic.GenericData; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.GenericRecordBuilder; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.avro.AvroSchemaUtil; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.GenericParquetWriter; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.parquet.Parquet; -import org.apache.iceberg.parquet.ParquetValueReader; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.apache.parquet.avro.AvroParquetWriter; -import org.apache.parquet.hadoop.ParquetWriter; -import org.apache.parquet.schema.LogicalTypeAnnotation; -import org.apache.parquet.schema.MessageType; -import org.apache.parquet.schema.PrimitiveType; -import org.apache.parquet.schema.Type; -import org.junit.jupiter.api.Test; - -public class TestFlinkParquetReader extends DataTest { - private static final int NUM_RECORDS = 100; - - @Test - public void testBuildReader() { - MessageType fileSchema = - new MessageType( - "test", - // 0: required(100, "id", LongType.get()) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) - .id(100) - .named("id"), - // 1: optional(101, "data", Types.StringType.get()) - primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) - .id(101) - .named("data"), - // 2: required(102, "b", Types.BooleanType.get()) - primitive(PrimitiveType.PrimitiveTypeName.BOOLEAN, Type.Repetition.REQUIRED) - .id(102) - .named("b"), - // 3: optional(103, "i", Types.IntegerType.get()) - primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) - .id(103) - .named("i"), - // 4: optional(105, "f", Types.FloatType.get()) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) - .id(104) - .named("l"), - // 5: required(106, "d", Types.DoubleType.get()) - primitive(PrimitiveType.PrimitiveTypeName.FLOAT, Type.Repetition.OPTIONAL) - .id(105) - .named("f"), - // 6: required(106, "d", Types.DoubleType.get()) - primitive(PrimitiveType.PrimitiveTypeName.DOUBLE, Type.Repetition.REQUIRED) - .id(106) - .named("d"), - // 7: optional(107, "date", Types.DateType.get()) - primitive(PrimitiveType.PrimitiveTypeName.INT32, Type.Repetition.OPTIONAL) - .id(107) - .as(LogicalTypeAnnotation.dateType()) - .named("date"), - // 8: required(108, "ts_tz", Types.TimestampType.withZone()) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) - .id(108) - .as( - LogicalTypeAnnotation.timestampType( - true, LogicalTypeAnnotation.TimeUnit.MICROS)) - .named("ts_tz"), - // 9: required(109, "ts", Types.TimestampType.withoutZone()) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) - .id(109) - .as( - LogicalTypeAnnotation.timestampType( - false, LogicalTypeAnnotation.TimeUnit.MICROS)) - .named("ts"), - // 10: required(110, "s", Types.StringType.get()) - primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.REQUIRED) - .id(110) - .as(LogicalTypeAnnotation.stringType()) - .named("s"), - // 11: required(112, "fixed", Types.FixedType.ofLength(7)) - primitive( - PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) - .id(112) - .length(7) - .named("f"), - // 12: optional(113, "bytes", Types.BinaryType.get()) - primitive(PrimitiveType.PrimitiveTypeName.BINARY, Type.Repetition.OPTIONAL) - .id(113) - .named("bytes"), - // 13: required(114, "dec_9_0", Types.DecimalType.of(9, 0)) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) - .id(114) - .as(LogicalTypeAnnotation.decimalType(0, 9)) - .named("dec_9_0"), - // 14: required(115, "dec_11_2", Types.DecimalType.of(11, 2)) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.REQUIRED) - .id(115) - .as(LogicalTypeAnnotation.decimalType(2, 11)) - .named("dec_11_2"), - // 15: required(116, "dec_38_10", Types.DecimalType.of(38, 10)) // maximum precision - primitive( - PrimitiveType.PrimitiveTypeName.FIXED_LEN_BYTE_ARRAY, Type.Repetition.REQUIRED) - .id(116) - .length(16) - .as(LogicalTypeAnnotation.decimalType(10, 38)) - .named("dec_38_10"), - // 16: required(117, "time", Types.TimeType.get()) - primitive(PrimitiveType.PrimitiveTypeName.INT64, Type.Repetition.OPTIONAL) - .id(117) - .as(LogicalTypeAnnotation.timeType(true, LogicalTypeAnnotation.TimeUnit.MICROS)) - .named("time")); - ParquetValueReader reader = - FlinkParquetReaders.buildReader(new Schema(SUPPORTED_PRIMITIVES.fields()), fileSchema); - - assertThat(reader.columns().size()).isEqualTo(SUPPORTED_PRIMITIVES.fields().size()); - } - - @Test - public void testTwoLevelList() throws IOException { - Schema schema = - new Schema( - optional(1, "arraybytes", Types.ListType.ofRequired(3, Types.BinaryType.get())), - optional(2, "topbytes", Types.BinaryType.get())); - org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(schema.asStruct()); - - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).isTrue(); - - ParquetWriter writer = - AvroParquetWriter.builder(new Path(testFile.toURI())) - .withDataModel(GenericData.get()) - .withSchema(avroSchema) - .config("parquet.avro.add-list-element-records", "true") - .config("parquet.avro.write-old-list-structure", "true") - .build(); - - GenericRecordBuilder recordBuilder = new GenericRecordBuilder(avroSchema); - List expectedByteList = Lists.newArrayList(); - byte[] expectedByte = {0x00, 0x01}; - ByteBuffer expectedBinary = ByteBuffer.wrap(expectedByte); - expectedByteList.add(expectedBinary); - recordBuilder.set("arraybytes", expectedByteList); - recordBuilder.set("topbytes", expectedBinary); - GenericData.Record expectedRecord = recordBuilder.build(); - - writer.write(expectedRecord); - writer.close(); - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) - .build()) { - Iterator rows = reader.iterator(); - assertThat(rows).hasNext(); - RowData rowData = rows.next(); - assertThat(rowData.getArray(0).getBinary(0)).isEqualTo(expectedByte); - assertThat(rowData.getBinary(1)).isEqualTo(expectedByte); - assertThat(rows).isExhausted(); - } - } - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).isTrue(); - - try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(GenericParquetWriter::buildWriter) - .build()) { - writer.addAll(iterable); - } - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(type -> FlinkParquetReaders.buildReader(schema, type)) - .build()) { - Iterator expected = iterable.iterator(); - Iterator rows = reader.iterator(); - LogicalType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(rows).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), rowType, expected.next(), rows.next()); - } - assertThat(rows).isExhausted(); - } - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(RandomGenericData.generate(schema, NUM_RECORDS, 19981), schema); - writeAndValidate( - RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124), schema); - writeAndValidate( - RandomGenericData.generateFallbackRecords(schema, NUM_RECORDS, 21124, NUM_RECORDS / 20), - schema); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java deleted file mode 100644 index b1e6f5aa00ff..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestFlinkParquetWriter.java +++ /dev/null @@ -1,94 +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 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 org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.LogicalType; -import org.apache.iceberg.Files; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.DataTest; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.data.parquet.GenericParquetReaders; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.parquet.Parquet; -import org.junit.jupiter.api.io.TempDir; - -public class TestFlinkParquetWriter extends DataTest { - private static final int NUM_RECORDS = 100; - - @TempDir private Path temp; - - private void writeAndValidate(Iterable iterable, Schema schema) throws IOException { - File testFile = File.createTempFile("junit", null, temp.toFile()); - assertThat(testFile.delete()).isTrue(); - - LogicalType logicalType = FlinkSchemaUtil.convert(schema); - - try (FileAppender writer = - Parquet.write(Files.localOutput(testFile)) - .schema(schema) - .createWriterFunc(msgType -> FlinkParquetWriters.buildWriter(logicalType, msgType)) - .build()) { - writer.addAll(iterable); - } - - try (CloseableIterable reader = - Parquet.read(Files.localInput(testFile)) - .project(schema) - .createReaderFunc(msgType -> GenericParquetReaders.buildReader(schema, msgType)) - .build()) { - Iterator expected = iterable.iterator(); - Iterator actual = reader.iterator(); - LogicalType rowType = FlinkSchemaUtil.convert(schema); - for (int i = 0; i < NUM_RECORDS; i += 1) { - assertThat(actual).hasNext(); - TestHelpers.assertRowData(schema.asStruct(), rowType, actual.next(), expected.next()); - } - assertThat(actual).isExhausted(); - } - } - - @Override - protected void writeAndValidate(Schema schema) throws IOException { - writeAndValidate(RandomRowData.generate(schema, NUM_RECORDS, 19981), schema); - - writeAndValidate( - RandomRowData.convert( - schema, - RandomGenericData.generateDictionaryEncodableRecords(schema, NUM_RECORDS, 21124)), - schema); - - writeAndValidate( - RandomRowData.convert( - schema, - RandomGenericData.generateFallbackRecords( - schema, NUM_RECORDS, 21124, NUM_RECORDS / 20)), - schema); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java deleted file mode 100644 index d078b2228456..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowDataProjection.java +++ /dev/null @@ -1,593 +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 static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatNoException; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.List; -import org.apache.flink.table.data.GenericArrayData; -import org.apache.flink.table.data.GenericMapData; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.TestHelpers; -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.apache.iceberg.util.StructProjection; -import org.junit.jupiter.api.Test; - -public class TestRowDataProjection { - @Test - public void testNullRootRowData() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowDataProjection projection = RowDataProjection.create(schema, schema.select("id")); - - assertThatThrownBy(() -> projection.wrap(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid row data: null"); - } - - @Test - public void testFullProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - generateAndValidate(schema, schema); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - } - - @Test - public void testReorderedFullProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - generateAndValidate(schema, reordered); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, reordered, rowData, copyRowData, otherRowData); - } - - @Test - public void testBasicProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - generateAndValidate(schema, idOnly); - generateAndValidate(schema, dataOnly); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, dataOnly, rowData, copyRowData, otherRowData); - } - - @Test - public void testEmptyProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - generateAndValidate(schema, schema.select()); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, schema.select(), rowData, copyRowData, otherRowData, true); - } - - @Test - public void testRename() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - Schema renamed = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - generateAndValidate(schema, renamed); - - GenericRowData rowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData copyRowData = GenericRowData.of(1L, StringData.fromString("a")); - GenericRowData otherRowData = GenericRowData.of(2L, StringData.fromString("b")); - testEqualsAndHashCode(schema, renamed, rowData, copyRowData, otherRowData); - } - - @Test - public void testNestedProjection() { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - GenericRowData rowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); - GenericRowData copyRowData = GenericRowData.of(1L, GenericRowData.of(1.0f, 1.0f)); - GenericRowData otherRowData = GenericRowData.of(2L, GenericRowData.of(2.0f, 2.0f)); - - GenericRowData rowDataNullStruct = GenericRowData.of(1L, null); - GenericRowData copyRowDataNullStruct = GenericRowData.of(1L, null); - GenericRowData otherRowDataNullStruct = GenericRowData.of(2L, null); - - // Project id only. - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, idOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct); - - // Project lat only. - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - assertThat(latOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, latOnly); - testEqualsAndHashCode(schema, latOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, latOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); - - // Project long only. - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - assertThat(longOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, longOnly); - testEqualsAndHashCode(schema, longOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, longOnly, rowDataNullStruct, copyRowDataNullStruct, otherRowDataNullStruct, true); - - // Project location. - Schema locationOnly = schema.select("location"); - assertThat(locationOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, locationOnly); - testEqualsAndHashCode(schema, locationOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode( - schema, - locationOnly, - rowDataNullStruct, - copyRowDataNullStruct, - otherRowDataNullStruct, - true); - } - - @Test - public void testPrimitivesFullProjection() { - DataGenerator dataGenerator = new DataGenerators.Primitives(); - Schema schema = dataGenerator.icebergSchema(); - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - GenericRowData otherRowData = dataGenerator.generateFlinkRowData(); - // modify the string field value (position 6) - otherRowData.setField(6, StringData.fromString("foo_bar")); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - setOptionalFieldsNullForPrimitives(rowDataNullOptionalFields); - GenericRowData copyRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - setOptionalFieldsNullForPrimitives(copyRowDataNullOptionalFields); - GenericRowData otherRowDataNullOptionalFields = dataGenerator.generateFlinkRowData(); - // modify the string field value (position 6) - otherRowDataNullOptionalFields.setField(6, StringData.fromString("foo_bar")); - setOptionalFieldsNullForPrimitives(otherRowData); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - private void setOptionalFieldsNullForPrimitives(GenericRowData rowData) { - // fields from [1, 5] range are optional - for (int pos = 1; pos <= 5; ++pos) { - rowData.setField(pos, null); - } - } - - @Test - public void testMapOfPrimitivesProjection() { - DataGenerator dataGenerator = new DataGenerators.MapOfPrimitives(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project map only. - Schema mapOnly = schema.select("map_of_primitives"); - assertThat(mapOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, mapOnly); - - // Project all. - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(StringData.fromString("foo"), 1, StringData.fromString("bar"), 2))); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData, true); - testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("row_id_value"), null); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("row_id_value"), null); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of(StringData.fromString("other_row_id_value"), null); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - mapOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields, - true); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testMapOfStructStructProjection() { - DataGenerator dataGenerator = new DataGenerators.MapOfStructStruct(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project map only. - Schema mapOnly = schema.select("map"); - assertThat(mapOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, mapOnly); - - // Project all. - generateAndValidate(schema, schema); - - // Project partial map key. - Schema partialMapKey = - new Schema( - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()), - Types.NestedField.required(204, "valueData", Types.StringType.get()))))); - assertThatThrownBy(() -> generateAndValidate(schema, partialMapKey)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial map key or value struct."); - - // Project partial map key. - Schema partialMapValue = - new Schema( - Types.NestedField.optional( - 2, - "map", - Types.MapType.ofOptional( - 101, - 102, - Types.StructType.of( - Types.NestedField.required(201, "key", Types.LongType.get()), - Types.NestedField.required(202, "keyData", Types.StringType.get())), - Types.StructType.of( - Types.NestedField.required(203, "value", Types.LongType.get()))))); - assertThatThrownBy(() -> generateAndValidate(schema, partialMapValue)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial map key or value struct."); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericMapData( - ImmutableMap.of( - GenericRowData.of(1L, StringData.fromString("other_key_data")), - GenericRowData.of(1L, StringData.fromString("other_value_data"))))); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, mapOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(1L, null), GenericRowData.of(1L, null)))); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericMapData( - ImmutableMap.of(GenericRowData.of(2L, null), GenericRowData.of(2L, null)))); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - mapOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testArrayOfPrimitiveProjection() { - DataGenerator dataGenerator = new DataGenerators.ArrayOfPrimitive(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project list only. - Schema arrayOnly = schema.select("array_of_int"); - assertThat(arrayOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, arrayOnly); - - // Project all. - generateAndValidate(schema, schema); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericArrayData(new Integer[] {4, 5, 6})); - testEqualsAndHashCode(schema, idOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, arrayOnly, rowData, copyRowData, otherRowData); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("other_row_id_value"), - new GenericArrayData(new Integer[] {4, null, 6})); - testEqualsAndHashCode( - schema, - idOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - arrayOnly, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - @Test - public void testArrayOfStructProjection() { - DataGenerator dataGenerator = new DataGenerators.ArrayOfStruct(); - Schema schema = dataGenerator.icebergSchema(); - - // Project id only. - Schema idOnly = schema.select("row_id"); - assertThat(idOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, idOnly); - - // Project list only. - Schema arrayOnly = schema.select("array_of_struct"); - assertThat(arrayOnly.columns().size()).isGreaterThan(0); - generateAndValidate(schema, arrayOnly); - - // Project all. - generateAndValidate(schema, schema); - - // Project partial list value. - Schema partialList = - new Schema( - Types.NestedField.optional( - 2, - "array_of_struct", - Types.ListType.ofOptional( - 101, - Types.StructType.of( - Types.NestedField.required(202, "name", Types.StringType.get()))))); - - assertThatThrownBy(() -> generateAndValidate(schema, partialList)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Cannot project a partial list element struct."); - - GenericRowData rowData = dataGenerator.generateFlinkRowData(); - GenericRowData copyRowData = dataGenerator.generateFlinkRowData(); - // modify the map field value - GenericRowData otherRowData = - GenericRowData.of( - StringData.fromString("row_id_value"), new GenericArrayData(new Integer[] {4, 5, 6})); - testEqualsAndHashCode(schema, schema, rowData, copyRowData, otherRowData); - - GenericRowData rowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - GenericRowData copyRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {1, null, 3})); - // modify the map field value - GenericRowData otherRowDataNullOptionalFields = - GenericRowData.of( - StringData.fromString("row_id_value"), - new GenericArrayData(new Integer[] {4, null, 6})); - testEqualsAndHashCode( - schema, - schema, - rowDataNullOptionalFields, - copyRowDataNullOptionalFields, - otherRowDataNullOptionalFields); - } - - private void generateAndValidate(Schema schema, Schema projectSchema) { - int numRecords = 100; - List recordList = RandomGenericData.generate(schema, numRecords, 102L); - List rowDataList = - Lists.newArrayList(RandomRowData.generate(schema, numRecords, 102L).iterator()); - assertThat(rowDataList).hasSize(recordList.size()); - - StructProjection structProjection = StructProjection.create(schema, projectSchema); - RowDataProjection rowDataProjection = RowDataProjection.create(schema, projectSchema); - - for (int i = 0; i < numRecords; i++) { - StructLike expected = structProjection.wrap(recordList.get(i)); - RowData projected = rowDataProjection.wrap(rowDataList.get(i)); - TestHelpers.assertRowData(projectSchema, expected, projected); - - assertThat(projected).isEqualTo(projected); - assertThat(projected).hasSameHashCodeAs(projected); - // make sure toString doesn't throw NPE for null values - assertThatNoException().isThrownBy(projected::toString); - } - } - - private void testEqualsAndHashCode( - Schema schema, - Schema projectionSchema, - RowData rowData, - RowData copyRowData, - RowData otherRowData) { - testEqualsAndHashCode(schema, projectionSchema, rowData, copyRowData, otherRowData, false); - } - - /** - * @param isOtherRowDataSameAsRowData sometimes projection on otherRowData can result in the same - * RowData, e.g. due to empty projection or null struct - */ - private void testEqualsAndHashCode( - Schema schema, - Schema projectionSchema, - RowData rowData, - RowData copyRowData, - RowData otherRowData, - boolean isOtherRowDataSameAsRowData) { - RowDataProjection projection = RowDataProjection.create(schema, projectionSchema); - RowDataProjection copyProjection = RowDataProjection.create(schema, projectionSchema); - RowDataProjection otherProjection = RowDataProjection.create(schema, projectionSchema); - - assertThat(projection.wrap(rowData)).isEqualTo(copyProjection.wrap(copyRowData)); - assertThat(projection.wrap(rowData)).hasSameHashCodeAs(copyProjection.wrap(copyRowData)); - - if (isOtherRowDataSameAsRowData) { - assertThat(projection.wrap(rowData)).isEqualTo(otherProjection.wrap(otherRowData)); - assertThat(projection.wrap(rowData)).hasSameHashCodeAs(otherProjection.wrap(otherRowData)); - } else { - assertThat(projection.wrap(rowData)).isNotEqualTo(otherProjection.wrap(otherRowData)); - assertThat(projection.wrap(rowData)) - .doesNotHaveSameHashCodeAs(otherProjection.wrap(otherRowData)); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java deleted file mode 100644 index e76452b7cea0..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestRowProjection.java +++ /dev/null @@ -1,594 +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 static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.withPrecision; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -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; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Files; -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.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestRowProjection { - - @TempDir private Path temp; - - private RowData writeAndRead(String desc, Schema writeSchema, Schema readSchema, RowData row) - throws IOException { - File file = File.createTempFile("junit", desc + ".avro", temp.toFile()); - assertThat(file.delete()).isTrue(); - - try (FileAppender appender = - Avro.write(Files.localOutput(file)) - .schema(writeSchema) - .createWriterFunc(ignore -> new FlinkAvroWriter(FlinkSchemaUtil.convert(writeSchema))) - .build()) { - appender.add(row); - } - - Iterable records = - Avro.read(Files.localInput(file)) - .project(readSchema) - .createReaderFunc(FlinkAvroReader::new) - .build(); - - return Iterables.getOnlyElement(records); - } - - @Test - public void testFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData projected = writeAndRead("full_projection", schema, schema, row); - - assertThat(projected.getLong(0)).isEqualTo(34); - assertThat(projected.getString(1)).asString().isEqualTo("test"); - } - - @Test - public void testSpecialCharacterProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "user id", Types.LongType.get()), - Types.NestedField.optional(1, "data%0", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData full = writeAndRead("special_chars", schema, schema, row); - - assertThat(full.getLong(0)).isEqualTo(34L); - assertThat(full.getString(1)).asString().isEqualTo("test"); - - RowData projected = writeAndRead("special_characters", schema, schema.select("data%0"), full); - - assertThat(projected.getArity()).isEqualTo(1); - assertThat(projected.getString(0)).asString().isEqualTo("test"); - } - - @Test - public void testReorderedFullProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema reordered = - new Schema( - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("full_projection", schema, reordered, row); - - assertThat(projected.getString(0)).asString().isEqualTo("test"); - assertThat(projected.getLong(1)).isEqualTo(34); - } - - @Test - public void testReorderedProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema reordered = - new Schema( - Types.NestedField.optional(2, "missing_1", Types.StringType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(3, "missing_2", Types.LongType.get())); - - RowData projected = writeAndRead("full_projection", schema, reordered, row); - - assertThat(projected.isNullAt(0)).isTrue(); - assertThat(projected.getString(1)).asString().isEqualTo("test"); - assertThat(projected.isNullAt(2)).isTrue(); - } - - @Test - public void testRenamedAddedField() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(1, "a", Types.LongType.get()), - Types.NestedField.required(2, "b", Types.LongType.get()), - Types.NestedField.required(3, "d", Types.LongType.get())); - - RowData row = GenericRowData.of(100L, 200L, 300L); - - Schema renamedAdded = - new Schema( - Types.NestedField.optional(1, "a", Types.LongType.get()), - Types.NestedField.optional(2, "b", Types.LongType.get()), - Types.NestedField.optional(3, "c", Types.LongType.get()), - Types.NestedField.optional(4, "d", Types.LongType.get())); - - RowData projected = writeAndRead("rename_and_add_column_projection", schema, renamedAdded, row); - assertThat(projected.getLong(0)) - .as("Should contain the correct value in column 1") - .isEqualTo(100L); - assertThat(projected.getLong(1)) - .as("Should contain the correct value in column 2") - .isEqualTo(200L); - assertThat(projected.getLong(2)) - .as("Should contain the correct value in column 1") - .isEqualTo(300L); - assertThat(projected.isNullAt(3)).as("Should contain empty value on new column 4").isTrue(); - } - - @Test - public void testEmptyProjection() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - RowData projected = writeAndRead("empty_projection", schema, schema.select(), row); - - assertThat(projected).isNotNull(); - assertThat(projected.getArity()).isEqualTo(0); - } - - @Test - public void testBasicProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("basic_projection_id", writeSchema, idOnly, row); - assertThat(projected.getArity()).as("Should not project data").isEqualTo(1); - assertThat(projected.getLong(0)).isEqualTo(34L); - - Schema dataOnly = new Schema(Types.NestedField.optional(1, "data", Types.StringType.get())); - - projected = writeAndRead("basic_projection_data", writeSchema, dataOnly, row); - - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getString(0)).asString().isEqualTo("test"); - } - - @Test - public void testRename() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get())); - - RowData row = GenericRowData.of(34L, StringData.fromString("test")); - - Schema readSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "renamed", Types.StringType.get())); - - RowData projected = writeAndRead("project_and_rename", writeSchema, readSchema, row); - - assertThat(projected.getLong(0)).isEqualTo(34L); - assertThat(projected.getString(1)) - .as("Should contain the correct data/renamed value") - .asString() - .isEqualTo("test"); - } - - @Test - public void testNestedStructProjection() throws Exception { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 3, - "location", - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get())))); - - RowData location = GenericRowData.of(52.995143f, -1.539054f); - RowData record = GenericRowData.of(34L, location); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, record); - assertThat(projected.getArity()).isEqualTo(1); - assertThat(projected.getLong(0)).as("Should contain the correct id value").isEqualTo(34L); - - Schema latOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(1, "lat", Types.FloatType.get())))); - - projected = writeAndRead("latitude_only", writeSchema, latOnly, record); - RowData projectedLocation = projected.getRow(0, 1); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); - assertThat(projectedLocation.getArity()).as("Should not project longitude").isEqualTo(1); - assertThat(projectedLocation.getFloat(0)) - .as("Should project latitude") - .isEqualTo(52.995143f, withPrecision(0.000001f)); - - Schema longOnly = - new Schema( - Types.NestedField.optional( - 3, - "location", - Types.StructType.of(Types.NestedField.required(2, "long", Types.FloatType.get())))); - - projected = writeAndRead("longitude_only", writeSchema, longOnly, record); - projectedLocation = projected.getRow(0, 1); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); - assertThat(projectedLocation.getArity()).as("Should not project latitutde").isEqualTo(1); - assertThat(projectedLocation.getFloat(0)) - .as("Should project longitude") - .isEqualTo(-1.539054f, withPrecision(0.000001f)); - - Schema locationOnly = writeSchema.select("location"); - projected = writeAndRead("location_only", writeSchema, locationOnly, record); - projectedLocation = projected.getRow(0, 1); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.isNullAt(0)).as("Should project location").isFalse(); - assertThat(projectedLocation.getFloat(0)) - .as("Should project latitude") - .isEqualTo(52.995143f, withPrecision(0.000001f)); - assertThat(projectedLocation.getFloat(1)) - .as("Should project longitude") - .isEqualTo(-1.539054f, withPrecision(0.000001f)); - } - - @Test - public void testMapProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "properties", - Types.MapType.ofOptional(6, 7, Types.StringType.get(), Types.StringType.get()))); - - GenericMapData properties = - new GenericMapData( - ImmutableMap.of( - StringData.fromString("a"), - StringData.fromString("A"), - StringData.fromString("b"), - StringData.fromString("B"))); - - RowData row = GenericRowData.of(34L, properties); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - assertThat(projected.getLong(0)).isEqualTo(34L); - assertThat(projected.getArity()).as("Should not project properties map").isEqualTo(1); - - Schema keyOnly = writeSchema.select("properties.key"); - projected = writeAndRead("key_only", writeSchema, keyOnly, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getMap(0)).isEqualTo(properties); - - Schema valueOnly = writeSchema.select("properties.value"); - projected = writeAndRead("value_only", writeSchema, valueOnly, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getMap(0)).isEqualTo(properties); - - Schema mapOnly = writeSchema.select("properties"); - projected = writeAndRead("map_only", writeSchema, mapOnly, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - 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; - } - - @Test - public void testMapOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "lat", Types.FloatType.get()), - Types.NestedField.required(2, "long", Types.FloatType.get()))))); - - RowData l1 = GenericRowData.of(53.992811f, -1.542616f); - RowData l2 = GenericRowData.of(52.995143f, -1.539054f); - GenericMapData map = - new GenericMapData( - ImmutableMap.of(StringData.fromString("L1"), l1, StringData.fromString("L2"), l2)); - RowData row = GenericRowData.of(34L, map); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - assertThat(projected.getLong(0)).isEqualTo(34L); - assertThat(projected.getArity()).as("Should not project locations map").isEqualTo(1); - - projected = writeAndRead("all_locations", writeSchema, writeSchema.select("locations"), row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getMap(0)).isEqualTo(row.getMap(1)); - - projected = writeAndRead("lat_only", writeSchema, writeSchema.select("locations.lat"), row); - GenericMapData locations = (GenericMapData) projected.getMap(0); - assertThat(locations).isNotNull(); - GenericArrayData l1l2Array = - new GenericArrayData( - new Object[] {StringData.fromString("L2"), StringData.fromString("L1")}); - assertThat(locations.keyArray()).isEqualTo(l1l2Array); - RowData projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - assertThat(projectedL1).isNotNull(); - assertThat(projectedL1.getFloat(0)) - .as("L1 should contain lat") - .isEqualTo(53.992811f, withPrecision(0.000001f)); - assertThat(projectedL1.getArity()).as("L1 should not contain long").isEqualTo(1); - RowData projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - assertThat(projectedL2).isNotNull(); - assertThat(projectedL2.getFloat(0)) - .as("L2 should contain lat") - .isEqualTo(52.995143f, withPrecision(0.000001f)); - assertThat(projectedL2.getArity()).as("L2 should not contain long").isEqualTo(1); - - projected = writeAndRead("long_only", writeSchema, writeSchema.select("locations.long"), row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - locations = (GenericMapData) projected.getMap(0); - assertThat(locations).isNotNull(); - assertThat(locations.keyArray()).isEqualTo(l1l2Array); - projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - assertThat(projectedL1).isNotNull(); - assertThat(projectedL1.getArity()).as("L1 should not contain lat").isEqualTo(1); - assertThat(projectedL1.getFloat(0)) - .as("L1 should contain long") - .isEqualTo(-1.542616f, withPrecision(0.000001f)); - projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - assertThat(projectedL2).isNotNull(); - assertThat(projectedL2.getArity()).as("L2 should not contain lat").isEqualTo(1); - assertThat(projectedL2.getFloat(0)) - .as("L2 should contain long") - .isEqualTo(-1.539054f, withPrecision(0.000001f)); - - Schema latitiudeRenamed = - new Schema( - Types.NestedField.optional( - 5, - "locations", - Types.MapType.ofOptional( - 6, - 7, - Types.StringType.get(), - Types.StructType.of( - Types.NestedField.required(1, "latitude", Types.FloatType.get()))))); - - projected = writeAndRead("latitude_renamed", writeSchema, latitiudeRenamed, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - locations = (GenericMapData) projected.getMap(0); - assertThat(locations).isNotNull(); - assertThat(locations.keyArray()).isEqualTo(l1l2Array); - projectedL1 = (RowData) locations.get(StringData.fromString("L1")); - assertThat(projectedL1).isNotNull(); - assertThat(projectedL1.getFloat(0)) - .as("L1 should contain latitude") - .isEqualTo(53.992811f, withPrecision(0.000001f)); - projectedL2 = (RowData) locations.get(StringData.fromString("L2")); - assertThat(projectedL2).isNotNull(); - assertThat(projectedL2.getFloat(0)) - .as("L2 should contain latitude") - .isEqualTo(52.995143f, withPrecision(0.000001f)); - } - - @Test - public void testListProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 10, "values", Types.ListType.ofOptional(11, Types.LongType.get()))); - - GenericArrayData values = new GenericArrayData(new Long[] {56L, 57L, 58L}); - - RowData row = GenericRowData.of(34L, values); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - assertThat(projected.getLong(0)).isEqualTo(34L); - assertThat(projected.getArity()).as("Should not project values list").isEqualTo(1); - - Schema elementOnly = writeSchema.select("values.element"); - projected = writeAndRead("element_only", writeSchema, elementOnly, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getArray(0)).isEqualTo(values); - - Schema listOnly = writeSchema.select("values"); - projected = writeAndRead("list_only", writeSchema, listOnly, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getArray(0)).isEqualTo(values); - } - - @Test - @SuppressWarnings("unchecked") - public void testListOfStructsProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.required(19, "x", Types.IntegerType.get()), - Types.NestedField.optional(18, "y", Types.IntegerType.get()))))); - - RowData p1 = GenericRowData.of(1, 2); - RowData p2 = GenericRowData.of(3, null); - GenericArrayData arrayData = new GenericArrayData(new RowData[] {p1, p2}); - RowData row = GenericRowData.of(34L, arrayData); - - Schema idOnly = new Schema(Types.NestedField.required(0, "id", Types.LongType.get())); - - RowData projected = writeAndRead("id_only", writeSchema, idOnly, row); - assertThat(projected.getLong(0)).isEqualTo(34L); - assertThat(projected.getArity()).isEqualTo(1); - - projected = writeAndRead("all_points", writeSchema, writeSchema.select("points"), row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.getArray(0)).isEqualTo(row.getArray(1)); - - projected = writeAndRead("x_only", writeSchema, writeSchema.select("points.x"), row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.isNullAt(0)).isFalse(); - ArrayData points = projected.getArray(0); - assertThat(points.size()).isEqualTo(2); - RowData projectedP1 = points.getRow(0, 2); - assertThat(projectedP1.getInt(0)).as("Should project x").isEqualTo(1); - assertThat(projectedP1.getArity()).as("Should not project y").isEqualTo(1); - RowData projectedP2 = points.getRow(1, 2); - assertThat(projectedP2.getArity()).as("Should not project y").isEqualTo(1); - assertThat(projectedP2.getInt(0)).as("Should project x").isEqualTo(3); - - projected = writeAndRead("y_only", writeSchema, writeSchema.select("points.y"), row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.isNullAt(0)).isFalse(); - points = projected.getArray(0); - assertThat(points.size()).isEqualTo(2); - projectedP1 = points.getRow(0, 2); - assertThat(projectedP1.getArity()).as("Should not project x").isEqualTo(1); - assertThat(projectedP1.getInt(0)).as("Should project y").isEqualTo(2); - projectedP2 = points.getRow(1, 2); - assertThat(projectedP2.getArity()).as("Should not project x").isEqualTo(1); - assertThat(projectedP2.isNullAt(0)).as("Should project null y").isTrue(); - - Schema yRenamed = - new Schema( - Types.NestedField.optional( - 22, - "points", - Types.ListType.ofOptional( - 21, - Types.StructType.of( - Types.NestedField.optional(18, "z", Types.IntegerType.get()))))); - - projected = writeAndRead("y_renamed", writeSchema, yRenamed, row); - assertThat(projected.getArity()).as("Should not project id").isEqualTo(1); - assertThat(projected.isNullAt(0)).isFalse(); - points = projected.getArray(0); - assertThat(points.size()).isEqualTo(2); - projectedP1 = points.getRow(0, 2); - assertThat(projectedP1.getArity()).as("Should not project x and y").isEqualTo(1); - assertThat(projectedP1.getInt(0)).as("Should project z").isEqualTo(2); - projectedP2 = points.getRow(1, 2); - assertThat(projectedP2.getArity()).as("Should not project x and y").isEqualTo(1); - assertThat(projectedP2.isNullAt(0)).as("Should project null z").isTrue(); - } - - @Test - public void testAddedFieldsWithRequiredChildren() throws Exception { - Schema schema = new Schema(Types.NestedField.required(1, "a", Types.LongType.get())); - - RowData row = GenericRowData.of(100L); - - Schema addedFields = - new Schema( - Types.NestedField.optional(1, "a", Types.LongType.get()), - Types.NestedField.optional( - 2, - "b", - Types.StructType.of(Types.NestedField.required(3, "c", Types.LongType.get()))), - Types.NestedField.optional(4, "d", Types.ListType.ofRequired(5, Types.LongType.get())), - Types.NestedField.optional( - 6, - "e", - Types.MapType.ofRequired(7, 8, Types.LongType.get(), Types.LongType.get()))); - - RowData projected = - writeAndRead("add_fields_with_required_children_projection", schema, addedFields, row); - assertThat(projected.getLong(0)) - .as("Should contain the correct value in column 1") - .isEqualTo(100L); - assertThat(projected.isNullAt(1)).as("Should contain empty value in new column 2").isTrue(); - assertThat(projected.isNullAt(2)).as("Should contain empty value in new column 4").isTrue(); - assertThat(projected.isNullAt(3)).as("Should contain empty value in new column 6").isTrue(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java deleted file mode 100644 index eccab20e04fc..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/data/TestStructRowData.java +++ /dev/null @@ -1,100 +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 org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.TestHelpers; -import org.junit.jupiter.api.Test; - -public class TestStructRowData { - - protected void testConverter(DataGenerator dataGenerator) { - StructRowData converter = new StructRowData(dataGenerator.icebergSchema().asStruct()); - GenericRecord expected = dataGenerator.generateIcebergGenericRecord(); - StructRowData actual = converter.setStruct(expected); - TestHelpers.assertRowData(dataGenerator.icebergSchema(), expected, actual); - } - - @Test - public void testPrimitiveTypes() { - testConverter(new DataGenerators.Primitives()); - } - - @Test - public void testStructOfPrimitive() { - testConverter(new DataGenerators.StructOfPrimitive()); - } - - @Test - public void testStructOfArray() { - testConverter(new DataGenerators.StructOfArray()); - } - - @Test - public void testStructOfMap() { - testConverter(new DataGenerators.StructOfMap()); - } - - @Test - public void testStructOfStruct() { - testConverter(new DataGenerators.StructOfStruct()); - } - - @Test - public void testArrayOfPrimitive() { - testConverter(new DataGenerators.ArrayOfPrimitive()); - } - - @Test - public void testArrayOfArray() { - testConverter(new DataGenerators.ArrayOfArray()); - } - - @Test - public void testArrayOfMap() { - testConverter(new DataGenerators.ArrayOfMap()); - } - - @Test - public void testArrayOfStruct() { - testConverter(new DataGenerators.ArrayOfStruct()); - } - - @Test - public void testMapOfPrimitives() { - testConverter(new DataGenerators.MapOfPrimitives()); - } - - @Test - public void testMapOfArray() { - testConverter(new DataGenerators.MapOfArray()); - } - - @Test - public void testMapOfMap() { - testConverter(new DataGenerators.MapOfMap()); - } - - @Test - public void testMapOfStruct() { - testConverter(new DataGenerators.MapOfStruct()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.java deleted file mode 100644 index 44eb907a17aa..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestAvroGenericRecordToRowDataMapper.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.sink; - -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.flink.AvroGenericRecordConverterBase; -import org.apache.iceberg.flink.DataGenerator; - -public class TestAvroGenericRecordToRowDataMapper extends AvroGenericRecordConverterBase { - @Override - protected void testConverter(DataGenerator dataGenerator) throws Exception { - // Need to use avroSchema from DataGenerator because some primitive types have special Avro - // type handling. Hence the Avro schema converted from Iceberg schema won't work. - AvroGenericRecordToRowDataMapper mapper = - AvroGenericRecordToRowDataMapper.forAvroSchema(dataGenerator.avroSchema()); - RowData expected = dataGenerator.generateFlinkRowData(); - RowData actual = mapper.map(dataGenerator.generateAvroGenericRecord()); - assertThat(actual).isEqualTo(expected); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java deleted file mode 100644 index abac605f81fd..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionKeySelector.java +++ /dev/null @@ -1,67 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatExceptionOfType; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestBucketPartitionKeySelector { - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testCorrectKeySelection(TableSchemaType tableSchemaType) { - int numBuckets = 60; - - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitionKeySelector keySelector = - new BucketPartitionKeySelector( - partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE); - - TestBucketPartitionerUtil.generateRowsForBucketIdRange(2, numBuckets) - .forEach( - rowData -> { - int expectedBucketId = - TestBucketPartitionerUtil.computeBucketId( - numBuckets, rowData.getString(1).toString()); - Integer key = keySelector.getKey(rowData); - assertThat(key).isEqualTo(expectedBucketId); - }); - } - - @Test - public void testKeySelectorMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(1); - - assertThatExceptionOfType(RuntimeException.class) - .isThrownBy( - () -> - new BucketPartitionKeySelector( - partitionSpec, SimpleDataUtil.SCHEMA, SimpleDataUtil.ROW_TYPE)) - .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java deleted file mode 100644 index 59bdba578ebb..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitioner.java +++ /dev/null @@ -1,108 +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.sink; - -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE; -import static org.apache.iceberg.flink.sink.BucketPartitioner.BUCKET_NULL_MESSAGE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatExceptionOfType; - -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.CsvSource; - -public class TestBucketPartitioner { - - static final int DEFAULT_NUM_BUCKETS = 60; - - @ParameterizedTest - @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) - public void testPartitioningParallelismGreaterThanBuckets( - String schemaTypeStr, String numBucketsStr) { - int numPartitions = 500; - TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - int bucketId = 0; - for (int expectedIndex = 0; expectedIndex < numPartitions; expectedIndex++) { - int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - assertThat(actualPartitionIndex).isEqualTo(expectedIndex); - bucketId++; - if (bucketId == numBuckets) { - bucketId = 0; - } - } - } - - @ParameterizedTest - @CsvSource({"ONE_BUCKET,50", "IDENTITY_AND_BUCKET,50", "ONE_BUCKET,60", "IDENTITY_AND_BUCKET,60"}) - public void testPartitioningParallelismEqualLessThanBuckets( - String schemaTypeStr, String numBucketsStr) { - int numPartitions = 30; - TableSchemaType tableSchemaType = TableSchemaType.valueOf(schemaTypeStr); - int numBuckets = Integer.parseInt(numBucketsStr); - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - for (int bucketId = 0; bucketId < numBuckets; bucketId++) { - int actualPartitionIndex = bucketPartitioner.partition(bucketId, numPartitions); - assertThat(actualPartitionIndex).isEqualTo(bucketId % numPartitions); - } - } - - @Test - public void testPartitionerBucketIdNullFail() { - PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> bucketPartitioner.partition(null, DEFAULT_NUM_BUCKETS)) - .withMessage(BUCKET_NULL_MESSAGE); - } - - @Test - public void testPartitionerMultipleBucketsFail() { - PartitionSpec partitionSpec = TableSchemaType.TWO_BUCKETS.getPartitionSpec(DEFAULT_NUM_BUCKETS); - - assertThatExceptionOfType(RuntimeException.class) - .isThrownBy(() -> new BucketPartitioner(partitionSpec)) - .withMessage(BucketPartitionerUtil.BAD_NUMBER_OF_BUCKETS_ERROR_MESSAGE, 2); - } - - @Test - public void testPartitionerBucketIdOutOfRangeFail() { - PartitionSpec partitionSpec = TableSchemaType.ONE_BUCKET.getPartitionSpec(DEFAULT_NUM_BUCKETS); - BucketPartitioner bucketPartitioner = new BucketPartitioner(partitionSpec); - - int negativeBucketId = -1; - assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(negativeBucketId, 1)) - .withMessage(BUCKET_LESS_THAN_LOWER_BOUND_MESSAGE, negativeBucketId); - - int tooBigBucketId = DEFAULT_NUM_BUCKETS; - assertThatExceptionOfType(IllegalArgumentException.class) - .isThrownBy(() -> bucketPartitioner.partition(tooBigBucketId, 1)) - .withMessage(BUCKET_GREATER_THAN_UPPER_BOUND_MESSAGE, tooBigBucketId, DEFAULT_NUM_BUCKETS); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java deleted file mode 100644 index ba0ea867ffb7..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerFlinkIcebergSink.java +++ /dev/null @@ -1,227 +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.sink; - -import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.apache.iceberg.flink.TestFixtures.TABLE_IDENTIFIER; -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 org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -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.sink.TestBucketPartitionerUtil.TableSchemaType; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.CloseableIterable; -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.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestBucketPartitionerFlinkIcebergSink { - - private static final int NUMBER_TASK_MANAGERS = 1; - private static final int SLOTS_PER_TASK_MANAGER = 8; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(NUMBER_TASK_MANAGERS) - .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) - .build()); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - // Parallelism = 8 (parallelism > numBuckets) throughout the test suite - private final int parallelism = NUMBER_TASK_MANAGERS * SLOTS_PER_TASK_MANAGER; - private final FileFormat format = FileFormat.PARQUET; - private final int numBuckets = 4; - - private Table table; - private StreamExecutionEnvironment env; - private TableLoader tableLoader; - - private void setupEnvironment(TableSchemaType tableSchemaType) { - PartitionSpec partitionSpec = tableSchemaType.getPartitionSpec(numBuckets); - table = - CATALOG_EXTENSION - .catalog() - .createTable( - TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitionSpec, - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = - StreamExecutionEnvironment.getExecutionEnvironment(DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism * 2); - tableLoader = CATALOG_EXTENSION.tableLoader(); - } - - private void appendRowsToTable(List allRows) throws Exception { - DataFormatConverters.RowConverter converter = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - DataStream dataStream = - env.addSource( - new BoundedTestSource<>( - allRows.stream().map(converter::toExternal).toArray(Row[]::new)), - ROW_TYPE_INFO) - .map(converter::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)) - .partitionCustom( - new BucketPartitioner(table.spec()), - new BucketPartitionKeySelector( - table.spec(), - table.schema(), - FlinkSink.toFlinkRowType(table.schema(), SimpleDataUtil.FLINK_SCHEMA))); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.NONE) - .append(); - - env.execute("Test Iceberg DataStream"); - - SimpleDataUtil.assertTableRows(table, allRows); - } - - @ParameterizedTest - @EnumSource( - value = TableSchemaType.class, - names = {"ONE_BUCKET", "IDENTITY_AND_BUCKET"}) - public void testSendRecordsToAllBucketsEvenly(TableSchemaType tableSchemaType) throws Exception { - setupEnvironment(tableSchemaType); - List rows = generateTestDataRows(); - - appendRowsToTable(rows); - TableTestStats stats = extractPartitionResults(tableSchemaType); - - assertThat(stats.totalRowCount).isEqualTo(rows.size()); - // All 4 buckets should've been written to - assertThat(stats.writersPerBucket.size()).isEqualTo(numBuckets); - assertThat(stats.numFilesPerBucket.size()).isEqualTo(numBuckets); - // Writer expectation (2 writers per bucket): - // - Bucket0 -> Writers [0, 4] - // - Bucket1 -> Writers [1, 5] - // - Bucket2 -> Writers [2, 6] - // - Bucket3 -> Writers [3, 7] - for (int i = 0, j = numBuckets; i < numBuckets; i++, j++) { - assertThat(stats.writersPerBucket.get(i)).hasSameElementsAs(Arrays.asList(i, j)); - // 2 files per bucket (one file is created by each writer) - assertThat(stats.numFilesPerBucket.get(i)).isEqualTo(2); - // 2 rows per file (total of 16 rows across 8 files) - assertThat(stats.rowsPerWriter.get(i)).isEqualTo(2); - } - } - - /** - * Generating 16 rows to be sent uniformly to all writers (round-robin across 8 writers -> 4 - * buckets) - */ - private List generateTestDataRows() { - int totalNumRows = parallelism * 2; - int numRowsPerBucket = totalNumRows / numBuckets; - return TestBucketPartitionerUtil.generateRowsForBucketIdRange(numRowsPerBucket, numBuckets); - } - - private TableTestStats extractPartitionResults(TableSchemaType tableSchemaType) - throws IOException { - int totalRecordCount = 0; - Map> writersPerBucket = Maps.newHashMap(); // > - Map filesPerBucket = Maps.newHashMap(); // - Map rowsPerWriter = Maps.newHashMap(); // - - try (CloseableIterable fileScanTasks = table.newScan().planFiles()) { - for (FileScanTask scanTask : fileScanTasks) { - long recordCountInFile = scanTask.file().recordCount(); - - String[] splitFilePath = scanTask.file().path().toString().split("/"); - // Filename example: 00007-0-a7d3a29a-33e9-4740-88f4-0f494397d60c-00001.parquet - // Writer ID: .......^^^^^ - String filename = splitFilePath[splitFilePath.length - 1]; - int writerId = Integer.parseInt(filename.split("-")[0]); - - totalRecordCount += recordCountInFile; - int bucketId = - scanTask - .file() - .partition() - .get(tableSchemaType.bucketPartitionColumnPosition(), Integer.class); - writersPerBucket.computeIfAbsent(bucketId, k -> Lists.newArrayList()); - writersPerBucket.get(bucketId).add(writerId); - filesPerBucket.put(bucketId, filesPerBucket.getOrDefault(bucketId, 0) + 1); - rowsPerWriter.put(writerId, rowsPerWriter.getOrDefault(writerId, 0L) + recordCountInFile); - } - } - - return new TableTestStats(totalRecordCount, writersPerBucket, filesPerBucket, rowsPerWriter); - } - - /** DTO to hold Test Stats */ - private static class TableTestStats { - final int totalRowCount; - final Map> writersPerBucket; - final Map numFilesPerBucket; - final Map rowsPerWriter; - - TableTestStats( - int totalRecordCount, - Map> writersPerBucket, - Map numFilesPerBucket, - Map rowsPerWriter) { - this.totalRowCount = totalRecordCount; - this.writersPerBucket = writersPerBucket; - this.numFilesPerBucket = numFilesPerBucket; - this.rowsPerWriter = rowsPerWriter; - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java deleted file mode 100644 index e1309bfac6d5..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestBucketPartitionerUtil.java +++ /dev/null @@ -1,126 +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.sink; - -import java.util.List; -import java.util.UUID; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.BucketUtil; - -final class TestBucketPartitionerUtil { - - enum TableSchemaType { - ONE_BUCKET { - @Override - public int bucketPartitionColumnPosition() { - return 0; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).bucket("data", numBuckets).build(); - } - }, - IDENTITY_AND_BUCKET { - @Override - public int bucketPartitionColumnPosition() { - return 1; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .identity("id") - .bucket("data", numBuckets) - .build(); - } - }, - TWO_BUCKETS { - @Override - public int bucketPartitionColumnPosition() { - return 1; - } - - @Override - public PartitionSpec getPartitionSpec(int numBuckets) { - return PartitionSpec.builderFor(SimpleDataUtil.SCHEMA) - .bucket("id", numBuckets) - .bucket("data", numBuckets) - .build(); - } - }; - - public abstract int bucketPartitionColumnPosition(); - - public abstract PartitionSpec getPartitionSpec(int numBuckets); - } - - private TestBucketPartitionerUtil() {} - - /** - * Utility method to generate rows whose values will "hash" to a range of bucketIds (from 0 to - * numBuckets - 1) - * - * @param numRowsPerBucket how many different rows should be generated per bucket - * @param numBuckets max number of buckets to consider - * @return the list of rows whose data "hashes" to the desired bucketId - */ - static List generateRowsForBucketIdRange(int numRowsPerBucket, int numBuckets) { - List rows = Lists.newArrayListWithCapacity(numBuckets * numRowsPerBucket); - // For some of our tests, this order of the generated rows matters - for (int i = 0; i < numRowsPerBucket; i++) { - for (int bucketId = 0; bucketId < numBuckets; bucketId++) { - String value = generateValueForBucketId(bucketId, numBuckets); - rows.add(GenericRowData.of(1, StringData.fromString(value))); - } - } - return rows; - } - - /** - * Utility method to generate a UUID string that will "hash" to a desired bucketId - * - * @param bucketId the desired bucketId - * @return the string data that "hashes" to the desired bucketId - */ - private static String generateValueForBucketId(int bucketId, int numBuckets) { - while (true) { - String uuid = UUID.randomUUID().toString(); - if (computeBucketId(numBuckets, uuid) == bucketId) { - return uuid; - } - } - } - - /** - * Utility that performs the same hashing/bucketing mechanism used by Bucket.java - * - * @param numBuckets max number of buckets to consider - * @param value the string to compute the bucketId from - * @return the computed bucketId - */ - static int computeBucketId(int numBuckets, String value) { - return (BucketUtil.hash(value) & Integer.MAX_VALUE) % numBuckets; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java deleted file mode 100644 index 360db658cd2f..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCachingTableSupplier.java +++ /dev/null @@ -1,81 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.mockito.Mockito.mock; -import static org.mockito.Mockito.when; - -import java.time.Duration; -import java.util.concurrent.TimeUnit; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.TableLoader; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; - -public class TestCachingTableSupplier { - - @Test - public void testCheckArguments() { - SerializableTable initialTable = mock(SerializableTable.class); - - Table loadedTable = mock(Table.class); - TableLoader tableLoader = mock(TableLoader.class); - when(tableLoader.loadTable()).thenReturn(loadedTable); - - new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); - - assertThatThrownBy(() -> new CachingTableSupplier(initialTable, tableLoader, null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("tableRefreshInterval cannot be null"); - assertThatThrownBy(() -> new CachingTableSupplier(null, tableLoader, Duration.ofMillis(100))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("initialTable cannot be null"); - assertThatThrownBy(() -> new CachingTableSupplier(initialTable, null, Duration.ofMillis(100))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("tableLoader cannot be null"); - } - - @Test - public void testTableReload() { - SerializableTable initialTable = mock(SerializableTable.class); - - Table loadedTable = mock(Table.class); - TableLoader tableLoader = mock(TableLoader.class); - when(tableLoader.loadTable()).thenReturn(loadedTable); - - CachingTableSupplier cachingTableSupplier = - new CachingTableSupplier(initialTable, tableLoader, Duration.ofMillis(100)); - - // refresh shouldn't do anything as the min reload interval hasn't passed - cachingTableSupplier.refreshTable(); - assertThat(cachingTableSupplier.get()).isEqualTo(initialTable); - - // refresh after waiting past the min reload interval - Awaitility.await() - .atLeast(100, TimeUnit.MILLISECONDS) - .untilAsserted( - () -> { - cachingTableSupplier.refreshTable(); - assertThat(cachingTableSupplier.get()).isEqualTo(loadedTable); - }); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java deleted file mode 100644 index 8faae1b05a4e..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ /dev/null @@ -1,257 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.Map; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.common.DynFields; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.BaseTaskWriter; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -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 TestCompressionSettings { - @TempDir protected Path temporaryFolder; - - private Table table; - - @Parameter(index = 0) - private Map initProperties; - - @Parameters(name = "tableProperties = {0}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {ImmutableMap.of()}, - new Object[] { - ImmutableMap.of( - TableProperties.AVRO_COMPRESSION, - "zstd", - TableProperties.AVRO_COMPRESSION_LEVEL, - "3", - TableProperties.PARQUET_COMPRESSION, - "zstd", - TableProperties.PARQUET_COMPRESSION_LEVEL, - "3", - TableProperties.ORC_COMPRESSION, - "zstd", - TableProperties.ORC_COMPRESSION_STRATEGY, - "compression") - } - }; - } - - @BeforeEach - public void before() throws IOException { - File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), initProperties, false); - } - - @TestTemplate - public void testCompressionAvro() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "AVRO")); - - if (initProperties.get(TableProperties.AVRO_COMPRESSION) == null) { - assertThat(resultProperties) - .containsEntry(TableProperties.AVRO_COMPRESSION, TableProperties.AVRO_COMPRESSION_DEFAULT) - .doesNotContainKey(TableProperties.AVRO_COMPRESSION_LEVEL); - } else { - assertThat(resultProperties) - .containsEntry( - TableProperties.AVRO_COMPRESSION, - initProperties.get(TableProperties.AVRO_COMPRESSION)) - .containsEntry( - TableProperties.AVRO_COMPRESSION_LEVEL, - initProperties.get(TableProperties.AVRO_COMPRESSION_LEVEL)); - } - - // Override compression to snappy and some random level - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "AVRO", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_LEVEL.key(), - "6")); - - assertThat(resultProperties) - .containsEntry(TableProperties.AVRO_COMPRESSION, "snappy") - .containsEntry(TableProperties.AVRO_COMPRESSION_LEVEL, "6"); - } - - @TestTemplate - public void testCompressionParquet() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "PARQUET")); - - if (initProperties.get(TableProperties.PARQUET_COMPRESSION) == null) { - assertThat(resultProperties) - .containsEntry( - TableProperties.PARQUET_COMPRESSION, - TableProperties.PARQUET_COMPRESSION_DEFAULT_SINCE_1_4_0) - .doesNotContainKey(TableProperties.PARQUET_COMPRESSION_LEVEL); - } else { - assertThat(resultProperties) - .containsEntry( - TableProperties.PARQUET_COMPRESSION, - initProperties.get(TableProperties.PARQUET_COMPRESSION)) - .containsEntry( - TableProperties.PARQUET_COMPRESSION_LEVEL, - initProperties.get(TableProperties.PARQUET_COMPRESSION_LEVEL)); - } - - // Override compression to snappy and some random level - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "PARQUET", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_LEVEL.key(), - "6")); - - assertThat(resultProperties) - .containsEntry(TableProperties.PARQUET_COMPRESSION, "snappy") - .containsEntry(TableProperties.PARQUET_COMPRESSION_LEVEL, "6"); - } - - @TestTemplate - public void testCompressionOrc() throws Exception { - // No override provided - Map resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of(FlinkWriteOptions.WRITE_FORMAT.key(), "ORC")); - - if (initProperties.get(TableProperties.ORC_COMPRESSION) == null) { - assertThat(resultProperties) - .containsEntry(TableProperties.ORC_COMPRESSION, TableProperties.ORC_COMPRESSION_DEFAULT) - .containsEntry( - TableProperties.ORC_COMPRESSION_STRATEGY, - TableProperties.ORC_COMPRESSION_STRATEGY_DEFAULT); - } else { - assertThat(resultProperties) - .containsEntry( - TableProperties.ORC_COMPRESSION, initProperties.get(TableProperties.ORC_COMPRESSION)) - .containsEntry( - TableProperties.ORC_COMPRESSION_STRATEGY, - initProperties.get(TableProperties.ORC_COMPRESSION_STRATEGY)); - } - - // Override compression to snappy and a different strategy - resultProperties = - appenderProperties( - table, - SimpleDataUtil.FLINK_SCHEMA, - ImmutableMap.of( - FlinkWriteOptions.WRITE_FORMAT.key(), - "ORC", - FlinkWriteOptions.COMPRESSION_CODEC.key(), - "snappy", - FlinkWriteOptions.COMPRESSION_STRATEGY.key(), - "speed")); - - assertThat(resultProperties) - .containsEntry(TableProperties.ORC_COMPRESSION, "snappy") - .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); - } - - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { - RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); - FlinkWriteConf flinkWriteConfig = - new FlinkWriteConf( - icebergTable, override, new org.apache.flink.configuration.Configuration()); - - IcebergStreamWriter streamWriter = - FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); - - harness.setup(); - harness.open(); - - return harness; - } - - private static Map appenderProperties( - Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter(table, schema, override)) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - - testHarness.prepareSnapshotPreBarrier(1L); - DynFields.BoundField operatorField = - DynFields.builder() - .hiddenImpl(testHarness.getOperatorFactory().getClass(), "operator") - .build(testHarness.getOperatorFactory()); - DynFields.BoundField writerField = - DynFields.builder() - .hiddenImpl(IcebergStreamWriter.class, "writer") - .build(operatorField.get()); - DynFields.BoundField appenderField = - DynFields.builder() - .hiddenImpl(BaseTaskWriter.class, "appenderFactory") - .build(writerField.get()); - DynFields.BoundField> propsField = - DynFields.builder() - .hiddenImpl(FlinkAppenderFactory.class, "props") - .build(appenderField.get()); - return propsField.get(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java deleted file mode 100644 index 21f3ee2c655a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestDeltaTaskWriter.java +++ /dev/null @@ -1,429 +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.sink; - -import static org.apache.iceberg.flink.SimpleDataUtil.createDelete; -import static org.apache.iceberg.flink.SimpleDataUtil.createInsert; -import static org.apache.iceberg.flink.SimpleDataUtil.createRecord; -import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateAfter; -import static org.apache.iceberg.flink.SimpleDataUtil.createUpdateBefore; -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.nio.file.Path; -import java.nio.file.Paths; -import java.time.OffsetDateTime; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.TimestampData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.LocalZonedTimestampType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.FileContent; -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.RowDelta; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.TestTables; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -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.types.Types; -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 class TestDeltaTaskWriter extends TestBase { - - @Parameter(index = 1) - private FileFormat format; - - @Parameters(name = "formatVersion = {0}, fileFormat = {1}") - protected static List parameters() { - return Arrays.asList( - new Object[] {2, FileFormat.AVRO}, - new Object[] {2, FileFormat.ORC}, - new Object[] {2, FileFormat.PARQUET}); - } - - @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"); - } - - private int idFieldId() { - return table.schema().findField("id").fieldId(); - } - - private int dataFieldId() { - return table.schema().findField("data").fieldId(); - } - - private void testCdcEvents(boolean partitioned) throws IOException { - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - // Start the 1th transaction. - TaskWriter writer = taskWriterFactory.create(); - - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "bbb")); - writer.write(createInsert(3, "ccc")); - - // Update <2, 'bbb'> to <2, 'ddd'> - writer.write(createUpdateBefore(2, "bbb")); // 1 pos-delete and 1 eq-delete. - writer.write(createUpdateAfter(2, "ddd")); - - // Update <1, 'aaa'> to <1, 'eee'> - writer.write(createUpdateBefore(1, "aaa")); // 1 pos-delete and 1 eq-delete. - writer.write(createUpdateAfter(1, "eee")); - - // Insert <4, 'fff'> - writer.write(createInsert(4, "fff")); - // Insert <5, 'ggg'> - writer.write(createInsert(5, "ggg")); - - // Delete <3, 'ccc'> - writer.write(createDelete(3, "ccc")); // 1 pos-delete and 1 eq-delete. - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).hasSize(partitioned ? 7 : 1); - assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet( - createRecord(1, "eee"), - createRecord(2, "ddd"), - createRecord(4, "fff"), - createRecord(5, "ggg"))); - - // Start the 2nd transaction. - writer = taskWriterFactory.create(); - - // Update <2, 'ddd'> to <6, 'hhh'> - (Update both key and value) - writer.write(createUpdateBefore(2, "ddd")); // 1 eq-delete - writer.write(createUpdateAfter(6, "hhh")); - - // Update <5, 'ggg'> to <5, 'iii'> - writer.write(createUpdateBefore(5, "ggg")); // 1 eq-delete - writer.write(createUpdateAfter(5, "iii")); - - // Delete <4, 'fff'> - writer.write(createDelete(4, "fff")); // 1 eq-delete. - - result = writer.complete(); - assertThat(result.dataFiles()).hasSize(partitioned ? 2 : 1); - assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet(createRecord(1, "eee"), createRecord(5, "iii"), createRecord(6, "hhh"))); - } - - @TestTemplate - public void testUnpartitioned() throws IOException { - createAndInitTable(false); - testCdcEvents(false); - } - - @TestTemplate - public void testPartitioned() throws IOException { - createAndInitTable(true); - testCdcEvents(true); - } - - private void testWritePureEqDeletes(boolean partitioned) throws IOException { - createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - writer.write(createDelete(1, "aaa")); - writer.write(createDelete(2, "bbb")); - writer.write(createDelete(3, "ccc")); - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).isEmpty(); - assertThat(result.deleteFiles()).hasSize(partitioned ? 3 : 1); - commitTransaction(result); - - assertThat(actualRowSet("*")).isEqualTo(expectedRowSet()); - } - - @TestTemplate - public void testUnpartitionedPureEqDeletes() throws IOException { - testWritePureEqDeletes(false); - } - - @TestTemplate - public void testPartitionedPureEqDeletes() throws IOException { - testWritePureEqDeletes(true); - } - - private void testAbort(boolean partitioned) throws IOException { - createAndInitTable(partitioned); - List equalityFieldIds = Lists.newArrayList(idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - for (int i = 0; i < 8_000; i += 2) { - writer.write(createUpdateBefore(i + 1, "aaa")); - writer.write(createUpdateAfter(i + 1, "aaa")); - - writer.write(createUpdateBefore(i + 2, "bbb")); - writer.write(createUpdateAfter(i + 2, "bbb")); - } - - // Assert the current data/delete file count. - List files = - Files.walk(Paths.get(tableDir.getPath(), "data")) - .filter(p -> p.toFile().isFile()) - .filter(p -> !p.toString().endsWith(".crc")) - .collect(Collectors.toList()); - assertThat(files).hasSize(partitioned ? 4 : 2); - - writer.abort(); - for (Path file : files) { - assertThat(file).doesNotExist(); - } - } - - @TestTemplate - public void testUnpartitionedAbort() throws IOException { - testAbort(false); - } - - @TestTemplate - public void testPartitionedAbort() throws IOException { - testAbort(true); - } - - @TestTemplate - public void testPartitionedTableWithDataAsKey() throws IOException { - createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - // Start the 1th transaction. - TaskWriter writer = taskWriterFactory.create(); - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "aaa")); - writer.write(createInsert(3, "bbb")); - writer.write(createInsert(4, "ccc")); - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).hasSize(3); - assertThat(result.deleteFiles()).hasSize(1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet(createRecord(2, "aaa"), createRecord(3, "bbb"), createRecord(4, "ccc"))); - - // Start the 2nd transaction. - writer = taskWriterFactory.create(); - writer.write(createInsert(5, "aaa")); - writer.write(createInsert(6, "bbb")); - writer.write(createDelete(7, "ccc")); // 1 eq-delete. - - result = writer.complete(); - assertThat(result.dataFiles()).hasSize(2); - assertThat(result.deleteFiles()).hasSize(1); - commitTransaction(result); - - assertThat(actualRowSet("*")) - .isEqualTo( - expectedRowSet( - createRecord(2, "aaa"), - createRecord(5, "aaa"), - createRecord(3, "bbb"), - createRecord(6, "bbb"))); - } - - @TestTemplate - public void testPartitionedTableWithDataAndIdAsKey() throws IOException { - createAndInitTable(true); - List equalityFieldIds = Lists.newArrayList(dataFieldId(), idFieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(equalityFieldIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - writer.write(createInsert(1, "aaa")); - writer.write(createInsert(2, "aaa")); - - writer.write(createDelete(2, "aaa")); // 1 pos-delete. - - WriteResult result = writer.complete(); - assertThat(result.dataFiles()).hasSize(1); - assertThat(result.deleteFiles()).hasSize(1); - assertThat(result.deleteFiles()[0].content()).isEqualTo(FileContent.POSITION_DELETES); - commitTransaction(result); - - assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(createRecord(1, "aaa"))); - } - - @TestTemplate - public void testEqualityColumnOnCustomPrecisionTSColumn() throws IOException { - Schema tableSchema = - new Schema( - required(3, "id", Types.IntegerType.get()), - required(4, "ts", Types.TimestampType.withZone())); - RowType flinkType = - new RowType( - false, - ImmutableList.of( - new RowType.RowField("id", new IntType()), - new RowType.RowField("ts", new LocalZonedTimestampType(3)))); - - this.table = create(tableSchema, PartitionSpec.unpartitioned()); - initTable(table); - - List equalityIds = ImmutableList.of(table.schema().findField("ts").fieldId()); - TaskWriterFactory taskWriterFactory = createTaskWriterFactory(flinkType, equalityIds); - taskWriterFactory.initialize(1, 1); - - TaskWriter writer = taskWriterFactory.create(); - RowDataSerializer serializer = new RowDataSerializer(flinkType); - OffsetDateTime start = OffsetDateTime.now(); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.INSERT, 1, TimestampData.fromInstant(start.toInstant())))); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.INSERT, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); - writer.write( - serializer.toBinaryRow( - GenericRowData.ofKind( - RowKind.DELETE, 2, TimestampData.fromInstant(start.plusSeconds(1).toInstant())))); - - WriteResult result = writer.complete(); - // One data file - assertThat(result.dataFiles()).hasSize(1); - // One eq delete file + one pos delete file - assertThat(result.deleteFiles()).hasSize(2); - assertThat( - Arrays.stream(result.deleteFiles()) - .map(ContentFile::content) - .collect(Collectors.toSet())) - .isEqualTo(Sets.newHashSet(FileContent.POSITION_DELETES, FileContent.EQUALITY_DELETES)); - commitTransaction(result); - - Record expectedRecord = GenericRecord.create(tableSchema); - expectedRecord.setField("id", 1); - int cutPrecisionNano = start.getNano() / 1000000 * 1000000; - expectedRecord.setField("ts", start.withNano(cutPrecisionNano)); - - assertThat(actualRowSet("*")).isEqualTo(expectedRowSet(expectedRecord)); - } - - private void commitTransaction(WriteResult result) { - RowDelta rowDelta = table.newRowDelta(); - Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); - Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); - rowDelta - .validateDeletedFiles() - .validateDataFilesExist(Lists.newArrayList(result.referencedDataFiles())) - .commit(); - } - - private StructLikeSet expectedRowSet(Record... records) { - return SimpleDataUtil.expectedRowSet(table, records); - } - - private StructLikeSet actualRowSet(String... columns) throws IOException { - return SimpleDataUtil.actualRowSet(table, columns); - } - - private TaskWriterFactory createTaskWriterFactory(List equalityFieldIds) { - return new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - FlinkSchemaUtil.convert(table.schema()), - 128 * 1024 * 1024, - format, - table.properties(), - equalityFieldIds, - false); - } - - private TaskWriterFactory createTaskWriterFactory( - RowType flinkType, List equalityFieldIds) { - return new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - flinkType, - 128 * 1024 * 1024, - format, - table.properties(), - equalityFieldIds, - true); - } - - private void createAndInitTable(boolean partitioned) { - if (partitioned) { - this.table = create(SCHEMA, PartitionSpec.builderFor(SCHEMA).identity("data").build()); - } else { - this.table = create(SCHEMA, PartitionSpec.unpartitioned()); - } - - initTable(table); - } - - private void initTable(TestTables.TestTable testTable) { - testTable - .updateProperties() - .set(TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES, String.valueOf(8 * 1024)) - .defaultFormat(format) - .commit(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java deleted file mode 100644 index dd89f43483b0..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkAppenderFactory.java +++ /dev/null @@ -1,65 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.TestAppenderFactory; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkAppenderFactory extends TestAppenderFactory { - - private final RowType rowType = FlinkSchemaUtil.convert(SCHEMA); - - @Override - protected FileAppenderFactory createAppenderFactory( - List equalityFieldIds, Schema eqDeleteSchema, Schema posDeleteRowSchema) { - return new FlinkAppenderFactory( - table, - table.schema(), - rowType, - table.properties(), - table.spec(), - ArrayUtil.toIntArray(equalityFieldIds), - eqDeleteSchema, - posDeleteRowSchema); - } - - @Override - protected RowData createRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet expectedRowSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(rowType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java deleted file mode 100644 index 414ee40d1357..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkFileWriterFactory.java +++ /dev/null @@ -1,66 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestFileWriterFactory; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkFileWriterFactory extends TestFileWriterFactory { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java deleted file mode 100644 index 61ab087f2ca3..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ /dev/null @@ -1,385 +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.sink; - -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -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.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -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.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -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.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - - @Parameter(index = 0) - private FileFormat format; - - @Parameter(index = 1) - private int parallelism; - - @Parameter(index = 2) - private boolean partitioned; - - @Parameters(name = "format={0}, parallelism = {1}, partitioned = {2}") - public static Object[][] parameters() { - return new Object[][] { - {FileFormat.AVRO, 1, true}, - {FileFormat.AVRO, 1, false}, - {FileFormat.AVRO, 2, true}, - {FileFormat.AVRO, 2, false}, - {FileFormat.ORC, 1, true}, - {FileFormat.ORC, 1, false}, - {FileFormat.ORC, 2, true}, - {FileFormat.ORC, 2, false}, - {FileFormat.PARQUET, 1, true}, - {FileFormat.PARQUET, 1, false}, - {FileFormat.PARQUET, 2, true}, - {FileFormat.PARQUET, 2, false} - }; - } - - @BeforeEach - public void before() throws IOException { - table = - CATALOG_EXTENSION - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - - tableLoader = CATALOG_EXTENSION.tableLoader(); - } - - @TestTemplate - public void testWriteRowData() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - - @TestTemplate - public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); - } - - @TestTemplate - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @TestTemplate - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); - } - } - } - - @TestTemplate - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @TestTemplate - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); - rightTable.refresh(); - assertThat(rightTable.currentSnapshot().summary()) - .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) - .containsEntry("direction", "rightTable"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @TestTemplate - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java deleted file mode 100644 index b38aa6b50ce6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.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.flink.sink; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.types.Row; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.source.BoundedTestSource; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class TestFlinkIcebergSinkBase { - - protected Table table; - protected StreamExecutionEnvironment env; - protected static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - protected static final DataFormatConverters.RowConverter CONVERTER = - new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); - - protected BoundedTestSource createBoundedSource(List rows) { - return new BoundedTestSource<>(rows.toArray(new Row[0])); - } - - protected List createRows(String prefix) { - return Lists.newArrayList( - Row.of(1, prefix + "aaa"), - Row.of(1, prefix + "bbb"), - Row.of(1, prefix + "ccc"), - Row.of(2, prefix + "aaa"), - Row.of(2, prefix + "bbb"), - Row.of(2, prefix + "ccc"), - Row.of(3, prefix + "aaa"), - Row.of(3, prefix + "bbb"), - Row.of(3, prefix + "ccc")); - } - - protected List convertToRowData(List rows) { - return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java deleted file mode 100644 index 441b5ed2a4ae..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBranch.java +++ /dev/null @@ -1,137 +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.sink; - -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; -import org.apache.iceberg.DistributionMode; -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; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -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.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestFlinkIcebergSinkBranch extends TestFlinkIcebergSinkBase { - @RegisterExtension - public static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - @Parameter(index = 0) - private String formatVersion; - - @Parameter(index = 1) - private String branch; - - private TableLoader tableLoader; - - @Parameters(name = "formatVersion = {0}, branch = {1}") - public static Object[][] parameters() { - return new Object[][] { - {"1", "main"}, - {"1", "testBranch"}, - {"2", "main"}, - {"2", "testBranch"} - }; - } - - @BeforeEach - public void before() throws IOException { - table = - CATALOG_EXTENSION - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - FileFormat.AVRO.name(), - TableProperties.FORMAT_VERSION, - formatVersion)); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100); - - tableLoader = CATALOG_EXTENSION.tableLoader(); - } - - @TestTemplate - public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - verifyOtherBranchUnmodified(); - } - - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .toBranch(branch) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); - SimpleDataUtil.assertTableRows( - table, - ImmutableList.of(), - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); - - verifyOtherBranchUnmodified(); - } - - private void verifyOtherBranchUnmodified() { - String otherBranch = - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; - if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - assertThat(table.currentSnapshot()).isNull(); - } - - assertThat(table.snapshot(otherBranch)).isNull(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java deleted file mode 100644 index 577c54976b9a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2.java +++ /dev/null @@ -1,235 +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.sink; - -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -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.List; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.MetadataColumns; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -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.types.Types; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.Timeout; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; - -@ExtendWith(ParameterizedTestExtension.class) -@Timeout(value = 60) -public class TestFlinkIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - @BeforeEach - public void setupTable() { - table = - CATALOG_EXTENSION - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - format.name(), - TableProperties.FORMAT_VERSION, - String.valueOf(FORMAT_V2))); - - table - .updateProperties() - .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) - .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) - .commit(); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100L) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - - tableLoader = CATALOG_EXTENSION.tableLoader(); - } - - @TestTemplate - public void testCheckAndGetEqualityFieldIds() { - table - .updateSchema() - .allowIncompatibleChanges() - .addRequiredColumn("type", Types.StringType.get()) - .setIdentifierFields("type") - .commit(); - - DataStream dataStream = - env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); - - // Use schema identifier field IDs as equality field id list by default - assertThat(builder.checkAndGetEqualityFieldIds()) - .containsExactlyInAnyOrderElementsOf(table.schema().identifierFieldIds()); - - // Use user-provided equality field column as equality field id list - builder.equalityFieldColumns(Lists.newArrayList("id")); - assertThat(builder.checkAndGetEqualityFieldIds()) - .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); - - builder.equalityFieldColumns(Lists.newArrayList("type")); - assertThat(builder.checkAndGetEqualityFieldIds()) - .containsExactlyInAnyOrder(table.schema().findField("type").fieldId()); - } - - @TestTemplate - public void testChangeLogOnIdKey() throws Exception { - testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testUpsertOnlyDeletesOnDataKey() throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); - - List> expectedRecords = - ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - true, - elementsPerCheckpoint, - expectedRecords, - SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testChangeLogOnDataKey() throws Exception { - testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testChangeLogOnIdDataKey() throws Exception { - testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testChangeLogOnSameKey() throws Exception { - testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testUpsertModeCheck() throws Exception { - DataStream dataStream = - env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .tableLoader(tableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .upsert(true); - - assertThatThrownBy( - () -> - builder - .equalityFieldColumns(ImmutableList.of("id", "data")) - .overwrite(true) - .append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); - } - - @TestTemplate - public void testUpsertOnIdKey() throws Exception { - testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testUpsertOnDataKey() throws Exception { - testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testUpsertOnIdDataKey() throws Exception { - testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); - } - - @TestTemplate - public void testDeleteStats() throws Exception { - assumeThat(format).isNotEqualTo(FileFormat.AVRO); - - List> elementsPerCheckpoint = - ImmutableList.of( - // Checkpoint #1 - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); - - List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); - - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - "main"); - - DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); - String fromStat = - 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()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java deleted file mode 100644 index fc33c2fea5e6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ /dev/null @@ -1,389 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.api.java.functions.KeySelector; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -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.types.RowKind; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.IcebergGenerics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.BoundedTestSource; -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.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkIcebergSinkV2Base { - - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = - new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; - - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; - - @Parameter(index = 0) - protected FileFormat format; - - @Parameter(index = 1) - protected int parallelism = 1; - - @Parameter(index = 2) - protected boolean partitioned; - - @Parameter(index = 3) - protected String writeDistributionMode; - - @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {FileFormat.AVRO, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {FileFormat.AVRO, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {FileFormat.AVRO, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_NONE}, - new Object[] {FileFormat.ORC, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {FileFormat.ORC, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {FileFormat.ORC, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {FileFormat.ORC, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_HASH}, - new Object[] {FileFormat.PARQUET, 1, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {FileFormat.PARQUET, 1, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {FileFormat.PARQUET, 4, true, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE}, - new Object[] {FileFormat.PARQUET, 4, false, TableProperties.WRITE_DISTRIBUTION_MODE_RANGE} - }; - } - - protected static final Map ROW_KIND_MAP = - ImmutableMap.of( - "+I", RowKind.INSERT, - "-D", RowKind.DELETE, - "-U", RowKind.UPDATE_BEFORE, - "+U", RowKind.UPDATE_AFTER); - - protected Row row(String rowKind, int id, String data) { - RowKind kind = ROW_KIND_MAP.get(rowKind); - if (kind == null) { - throw new IllegalArgumentException("Unknown row kind: " + rowKind); - } - - return Row.ofKind(kind, id, data); - } - - protected void testUpsertOnIdDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 2, "bbb"), row("+I", 2, "ccc")), - ImmutableList.of(row("+U", 1, "bbb"), row("-U", 1, "ccc"), row("-D", 1, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "aaa"), record(2, "bbb")), - ImmutableList.of(record(1, "aaa"), record(2, "ccc")), - ImmutableList.of(record(1, "bbb"), record(2, "ccc"))); - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnIdDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 2, "bbb"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa")), - ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "aaa"), record(2, "bbb")), - ImmutableList.of( - record(1, "aaa"), record(1, "bbb"), record(1, "ccc"), record(2, "bbb")), - ImmutableList.of( - record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "bbb"))); - - testChangeLogs( - ImmutableList.of("data", "id"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnSameKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - // Checkpoint #1 - ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa")), - // Checkpoint #2 - ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa")), - // Checkpoint #3 - ImmutableList.of(row("-D", 1, "aaa"), row("+I", 1, "aaa")), - // Checkpoint #4 - ImmutableList.of(row("-U", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 1, "aaa"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa")), - ImmutableList.of(record(1, "aaa"), record(1, "aaa"))); - - testChangeLogs( - ImmutableList.of("id", "data"), - row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 2, "bbb"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa")), - ImmutableList.of(row("-U", 2, "aaa"), row("+U", 1, "ccc"), row("+I", 1, "aaa")), - ImmutableList.of(row("-D", 1, "bbb"), row("+I", 2, "aaa"), row("+I", 2, "ccc"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "aaa")), - ImmutableList.of(record(1, "aaa"), record(1, "bbb"), record(1, "ccc")), - ImmutableList.of( - record(1, "aaa"), record(1, "ccc"), record(2, "aaa"), record(2, "ccc"))); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testUpsertOnDataKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), - ImmutableList.of(row("+U", 4, "aaa"), row("-U", 3, "bbb"), row("+U", 5, "bbb")), - ImmutableList.of(row("+I", 6, "aaa"), row("+U", 7, "bbb"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(2, "aaa"), record(3, "bbb")), - ImmutableList.of(record(4, "aaa"), record(5, "bbb")), - ImmutableList.of(record(6, "aaa"), record(7, "bbb"))); - - testChangeLogs( - ImmutableList.of("data"), - row -> row.getField(ROW_DATA_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } - - protected void testChangeLogOnIdKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of( - row("+I", 1, "aaa"), - row("-D", 1, "aaa"), - row("+I", 1, "bbb"), - row("+I", 2, "aaa"), - row("-D", 2, "aaa"), - row("+I", 2, "bbb")), - ImmutableList.of( - row("-U", 2, "bbb"), row("+U", 2, "ccc"), row("-D", 2, "ccc"), row("+I", 2, "ddd")), - ImmutableList.of( - row("-D", 1, "bbb"), - row("+I", 1, "ccc"), - row("-D", 1, "ccc"), - row("+I", 1, "ddd"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb"), record(2, "bbb")), - ImmutableList.of(record(1, "bbb"), record(2, "ddd")), - ImmutableList.of(record(1, "ddd"), record(2, "ddd"))); - - if (partitioned && writeDistributionMode.equals(TableProperties.WRITE_DISTRIBUTION_MODE_HASH)) { - assertThatThrownBy( - () -> - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch)) - .isInstanceOf(IllegalStateException.class) - .hasMessageStartingWith( - "In 'hash' distribution mode with equality fields set, partition field") - .hasMessageContaining("should be included in equality fields:"); - - } else { - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - false, - elementsPerCheckpoint, - expectedRecords, - branch); - } - } - - protected void testUpsertOnIdKey(String branch) throws Exception { - List> elementsPerCheckpoint = - ImmutableList.of( - ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), - ImmutableList.of(row("+I", 1, "ccc")), - ImmutableList.of(row("+U", 1, "ddd"), row("+I", 1, "eee"))); - - List> expectedRecords = - ImmutableList.of( - ImmutableList.of(record(1, "bbb")), - ImmutableList.of(record(1, "ccc")), - ImmutableList.of(record(1, "eee"))); - - if (!partitioned) { - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch); - } else { - assertThatThrownBy( - () -> - testChangeLogs( - ImmutableList.of("id"), - row -> row.getField(ROW_ID_POS), - true, - elementsPerCheckpoint, - expectedRecords, - branch)) - .isInstanceOf(IllegalStateException.class) - .hasMessageContaining("should be included in equality fields:"); - } - } - - protected void testChangeLogs( - List equalityFieldColumns, - KeySelector keySelector, - boolean insertAsUpsert, - List> elementsPerCheckpoint, - List> expectedRecordsPerCheckpoint, - String branch) - throws Exception { - DataStream dataStream = - env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .tableLoader(tableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .equalityFieldColumns(equalityFieldColumns) - .upsert(insertAsUpsert) - .toBranch(branch) - .append(); - - // Execute the program. - env.execute("Test Iceberg Change-Log DataStream."); - - table.refresh(); - List snapshots = findValidSnapshots(); - int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); - assertThat(snapshots).hasSize(expectedSnapshotNum); - - for (int i = 0; i < expectedSnapshotNum; i++) { - long snapshotId = snapshots.get(i).snapshotId(); - List expectedRecords = expectedRecordsPerCheckpoint.get(i); - assertThat(actualRowSet(snapshotId, "*")) - .as("Should have the expected records for the checkpoint#" + i) - .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); - } - } - - protected Record record(int id, String data) { - return SimpleDataUtil.createRecord(id, data); - } - - private List findValidSnapshots() { - List validSnapshots = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - if (snapshot.allManifests(table.io()).stream() - .anyMatch(m -> snapshot.snapshotId() == m.snapshotId())) { - validSnapshots.add(snapshot); - } - } - return validSnapshots; - } - - private StructLikeSet expectedRowSet(Record... records) { - return SimpleDataUtil.expectedRowSet(table, records); - } - - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { - table.refresh(); - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - try (CloseableIterable reader = - IcebergGenerics.read(table).useSnapshot(snapshotId).select(columns).build()) { - reader.forEach(set::add); - } - return set; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java deleted file mode 100644 index 0b0c55f51c32..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Branch.java +++ /dev/null @@ -1,125 +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.sink; - -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; - -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; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - @Parameter(index = 0) - private String branch; - - @Parameters(name = "branch = {0}") - public static Object[][] parameters() { - return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; - } - - @BeforeEach - public void before() throws IOException { - table = - CATALOG_EXTENSION - .catalog() - .createTable( - TestFixtures.TABLE_IDENTIFIER, - SimpleDataUtil.SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of( - TableProperties.DEFAULT_FILE_FORMAT, - FileFormat.AVRO.name(), - TableProperties.FORMAT_VERSION, - "2")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100); - - tableLoader = CATALOG_EXTENSION.tableLoader(); - } - - @TestTemplate - public void testChangeLogOnIdKey() throws Exception { - testChangeLogOnIdKey(branch); - verifyOtherBranchUnmodified(); - } - - @TestTemplate - public void testChangeLogOnDataKey() throws Exception { - testChangeLogOnDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @TestTemplate - public void testChangeLogOnIdDataKey() throws Exception { - testChangeLogOnIdDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @TestTemplate - public void testUpsertOnIdKey() throws Exception { - testUpsertOnIdKey(branch); - verifyOtherBranchUnmodified(); - } - - @TestTemplate - public void testUpsertOnDataKey() throws Exception { - testUpsertOnDataKey(branch); - verifyOtherBranchUnmodified(); - } - - @TestTemplate - public void testUpsertOnIdDataKey() throws Exception { - testUpsertOnIdDataKey(branch); - verifyOtherBranchUnmodified(); - } - - private void verifyOtherBranchUnmodified() { - String otherBranch = - branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; - if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { - assertThat(table.currentSnapshot()).isNull(); - } - - assertThat(table.snapshot(otherBranch)).isNull(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java deleted file mode 100644 index 53b7c4c0cc91..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ /dev/null @@ -1,312 +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.sink; - -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.nio.file.Paths; -import java.util.List; -import java.util.Map; -import java.util.UUID; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.core.io.SimpleVersionedSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.ManifestFiles; -import org.apache.iceberg.Table; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.WriteResult; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.Pair; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestFlinkManifest { - private static final Configuration CONF = new Configuration(); - - @TempDir protected Path temporaryFolder; - - private Table table; - private FileAppenderFactory appenderFactory; - private final AtomicInteger fileCount = new AtomicInteger(0); - - @BeforeEach - public void before() throws IOException { - File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); - String warehouse = folder.getAbsolutePath(); - - String tablePath = warehouse.concat("/test"); - assertThat(new File(tablePath).mkdir()).isTrue(); - - // Construct the iceberg table. - table = SimpleDataUtil.createTable(tablePath, ImmutableMap.of(), false); - - int[] equalityFieldIds = - new int[] { - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() - }; - this.appenderFactory = - new FlinkAppenderFactory( - table, - table.schema(), - FlinkSchemaUtil.convert(table.schema()), - table.properties(), - table.spec(), - equalityFieldIds, - table.schema(), - null); - } - - @Test - public void testIO() throws IOException { - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - for (long checkpointId = 1; checkpointId <= 3; checkpointId++) { - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - final long curCkpId = checkpointId; - - List dataFiles = generateDataFiles(10); - List eqDeleteFiles = generateEqDeleteFiles(5); - List posDeleteFiles = generatePosDeleteFiles(5); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder() - .addDataFiles(dataFiles) - .addDeleteFiles(eqDeleteFiles) - .addDeleteFiles(posDeleteFiles) - .build(), - () -> factory.create(curCkpId), - table.spec()); - - WriteResult result = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - assertThat(result.deleteFiles()).hasSize(10); - for (int i = 0; i < dataFiles.size(); i++) { - TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); - } - assertThat(result.deleteFiles()).hasSize(10); - for (int i = 0; i < 5; i++) { - TestHelpers.assertEquals(eqDeleteFiles.get(i), result.deleteFiles()[i]); - } - for (int i = 0; i < 5; i++) { - TestHelpers.assertEquals(posDeleteFiles.get(i), result.deleteFiles()[5 + i]); - } - } - } - - @Test - public void testUserProvidedManifestLocation() throws IOException { - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); - Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); - ManifestOutputFileFactory factory = - new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(5); - DeltaManifests deltaManifests = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder().addDataFiles(dataFiles).build(), - () -> factory.create(checkpointId), - table.spec()); - - assertThat(deltaManifests.dataManifest()).isNotNull(); - assertThat(deltaManifests.deleteManifest()).isNull(); - assertThat(Paths.get(deltaManifests.dataManifest().path())) - .hasParent(userProvidedFolder.toPath()); - - WriteResult result = - FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs()); - - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(5); - - assertThat(result.dataFiles()).hasSameSizeAs(dataFiles); - for (int i = 0; i < dataFiles.size(); i++) { - TestHelpers.assertEquals(dataFiles.get(i), result.dataFiles()[i]); - } - } - - @Test - public void testVersionedSerializer() throws IOException { - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(10); - List eqDeleteFiles = generateEqDeleteFiles(10); - List posDeleteFiles = generatePosDeleteFiles(10); - DeltaManifests expected = - FlinkManifestUtil.writeCompletedFiles( - WriteResult.builder() - .addDataFiles(dataFiles) - .addDeleteFiles(eqDeleteFiles) - .addDeleteFiles(posDeleteFiles) - .build(), - () -> factory.create(checkpointId), - table.spec()); - - byte[] versionedSerializeData = - SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, expected); - DeltaManifests actual = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, versionedSerializeData); - TestHelpers.assertEquals(expected.dataManifest(), actual.dataManifest()); - TestHelpers.assertEquals(expected.deleteManifest(), actual.deleteManifest()); - - byte[] versionedSerializeData2 = - SimpleVersionedSerialization.writeVersionAndSerialize( - DeltaManifestsSerializer.INSTANCE, actual); - assertThat(versionedSerializeData2).containsExactly(versionedSerializeData); - } - - @Test - public void testCompatibility() throws IOException { - // The v2 deserializer should be able to deserialize the v1 binary. - long checkpointId = 1; - String flinkJobId = newFlinkJobId(); - String operatorId = newOperatorUniqueId(); - ManifestOutputFileFactory factory = - FlinkManifestUtil.createOutputFileFactory( - () -> table, table.properties(), flinkJobId, operatorId, 1, 1); - - List dataFiles = generateDataFiles(10); - ManifestFile manifest = - FlinkManifestUtil.writeDataFiles(factory.create(checkpointId), table.spec(), dataFiles); - byte[] dataV1 = - SimpleVersionedSerialization.writeVersionAndSerialize(new V1Serializer(), manifest); - - DeltaManifests delta = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, dataV1); - assertThat(delta.deleteManifest()).isNull(); - assertThat(delta.dataManifest()).isNotNull(); - TestHelpers.assertEquals(manifest, delta.dataManifest()); - - List actualFiles = - FlinkManifestUtil.readDataFiles(delta.dataManifest(), table.io(), table.specs()); - assertThat(actualFiles).hasSize(10); - for (int i = 0; i < 10; i++) { - TestHelpers.assertEquals(dataFiles.get(i), actualFiles.get(i)); - } - } - - private static class V1Serializer implements SimpleVersionedSerializer { - - @Override - public int getVersion() { - return 1; - } - - @Override - public byte[] serialize(ManifestFile m) throws IOException { - return ManifestFiles.encode(m); - } - - @Override - public ManifestFile deserialize(int version, byte[] serialized) throws IOException { - return ManifestFiles.decode(serialized); - } - } - - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - table.spec(), - CONF, - table.location(), - FileFormat.PARQUET.addExtension(filename), - rows); - } - - private DeleteFile writeEqDeleteFile(String filename, List deletes) throws IOException { - return SimpleDataUtil.writeEqDeleteFile( - table, FileFormat.PARQUET, filename, appenderFactory, deletes); - } - - private DeleteFile writePosDeleteFile(String filename, List> positions) - throws IOException { - return SimpleDataUtil.writePosDeleteFile( - table, FileFormat.PARQUET, filename, appenderFactory, positions); - } - - private List generateDataFiles(int fileNum) throws IOException { - List rowDataList = Lists.newArrayList(); - List dataFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - rowDataList.add(SimpleDataUtil.createRowData(i, "a" + i)); - dataFiles.add(writeDataFile("data-file-" + fileCount.incrementAndGet(), rowDataList)); - } - return dataFiles; - } - - private List generateEqDeleteFiles(int fileNum) throws IOException { - List rowDataList = Lists.newArrayList(); - List deleteFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - rowDataList.add(SimpleDataUtil.createDelete(i, "a" + i)); - deleteFiles.add( - writeEqDeleteFile("eq-delete-file-" + fileCount.incrementAndGet(), rowDataList)); - } - return deleteFiles; - } - - private List generatePosDeleteFiles(int fileNum) throws IOException { - List> positions = Lists.newArrayList(); - List deleteFiles = Lists.newArrayList(); - for (int i = 0; i < fileNum; i++) { - positions.add(Pair.of("data-file-1", (long) i)); - deleteFiles.add( - writePosDeleteFile("pos-delete-file-" + fileCount.incrementAndGet(), positions)); - } - return deleteFiles; - } - - private static String newFlinkJobId() { - return UUID.randomUUID().toString(); - } - - private static String newOperatorUniqueId() { - return UUID.randomUUID().toString(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java deleted file mode 100644 index f79a3e634071..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPartitioningWriters.java +++ /dev/null @@ -1,66 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestPartitioningWriters; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkPartitioningWriters extends TestPartitioningWriters { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java deleted file mode 100644 index 3050752d1c24..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkPositionDeltaWriters.java +++ /dev/null @@ -1,66 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestPositionDeltaWriters; -import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkPositionDeltaWriters extends TestPositionDeltaWriters { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } - - @Override - protected StructLikeSet toSet(Iterable rows) { - StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); - RowType flinkType = FlinkSchemaUtil.convert(table.schema()); - for (RowData row : rows) { - RowDataWrapper wrapper = new RowDataWrapper(flinkType, table.schema().asStruct()); - set.add(wrapper.wrap(row)); - } - return set; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java deleted file mode 100644 index 03051b69cf87..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkRollingFileWriters.java +++ /dev/null @@ -1,51 +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.sink; - -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestRollingFileWriters; -import org.apache.iceberg.util.ArrayUtil; - -public class TestFlinkRollingFileWriters extends TestRollingFileWriters { - - @Override - protected FileWriterFactory newWriterFactory( - Schema dataSchema, - List equalityFieldIds, - Schema equalityDeleteRowSchema, - Schema positionDeleteRowSchema) { - return FlinkFileWriterFactory.builderFor(table) - .dataSchema(table.schema()) - .dataFileFormat(format()) - .deleteFileFormat(format()) - .equalityFieldIds(ArrayUtil.toIntArray(equalityFieldIds)) - .equalityDeleteRowSchema(equalityDeleteRowSchema) - .positionDeleteRowSchema(positionDeleteRowSchema) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data) { - return SimpleDataUtil.createRowData(id, data); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java deleted file mode 100644 index e6d64ef2c720..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkWriterMetrics.java +++ /dev/null @@ -1,60 +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.sink; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.io.FileWriterFactory; -import org.apache.iceberg.io.TestWriterMetrics; - -public class TestFlinkWriterMetrics extends TestWriterMetrics { - - public TestFlinkWriterMetrics(FileFormat fileFormat) { - super(fileFormat); - } - - @Override - protected FileWriterFactory newWriterFactory(Table sourceTable) { - return FlinkFileWriterFactory.builderFor(sourceTable) - .dataSchema(sourceTable.schema()) - .dataFileFormat(fileFormat) - .deleteFileFormat(fileFormat) - .positionDeleteRowSchema(sourceTable.schema()) - .build(); - } - - @Override - protected RowData toRow(Integer id, String data, boolean boolValue, Long longValue) { - GenericRowData nested = GenericRowData.of(boolValue, longValue); - GenericRowData row = GenericRowData.of(id, StringData.fromString(data), nested); - return row; - } - - @Override - public RowData toGenericRow(int value, int repeated) { - GenericRowData row = new GenericRowData(repeated); - for (int i = 0; i < repeated; i++) { - row.setField(i, value); - } - return row; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java deleted file mode 100644 index 948c7b31430c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergFilesCommitter.java +++ /dev/null @@ -1,1148 +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.sink; - -import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; -import static org.apache.iceberg.flink.sink.IcebergFilesCommitter.MAX_CONTINUOUS_EMPTY_COMMITS; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; -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.Files; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.NavigableMap; -import java.util.SortedMap; -import java.util.stream.Collectors; -import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.state.ListState; -import org.apache.flink.api.common.state.OperatorStateStore; -import org.apache.flink.core.io.SimpleVersionedSerialization; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.testutils.MockEnvironment; -import org.apache.flink.runtime.operators.testutils.MockEnvironmentBuilder; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.api.operators.StreamOperator; -import org.apache.flink.streaming.api.operators.StreamOperatorParameters; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.GenericManifestFile; -import org.apache.iceberg.ManifestContent; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionData; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.io.WriteResult; -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.util.Pair; -import org.apache.iceberg.util.ThreadPools; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestIcebergFilesCommitter extends TestBase { - private static final Configuration CONF = new Configuration(); - - private File flinkManifestFolder; - - @Parameter(index = 1) - private FileFormat format; - - @Parameter(index = 2) - private String branch; - - @Parameters(name = "formatVersion = {0}, fileFormat = {1}, branch = {2}") - protected static List parameters() { - return Arrays.asList( - new Object[] {1, FileFormat.AVRO, "main"}, - new Object[] {2, FileFormat.AVRO, "test-branch"}, - new Object[] {1, FileFormat.PARQUET, "main"}, - new Object[] {2, FileFormat.PARQUET, "test-branch"}, - new Object[] {1, FileFormat.ORC, "main"}, - new Object[] {2, FileFormat.ORC, "test-branch"}); - } - - @Override - @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()); - - table - .updateProperties() - .set(DEFAULT_FILE_FORMAT, format.name()) - .set(FLINK_MANIFEST_LOCATION, flinkManifestFolder.getAbsolutePath()) - .set(MAX_CONTINUOUS_EMPTY_COMMITS, "1") - .commit(); - } - - @TestTemplate - public void testCommitTxnWithoutDataFiles() throws Exception { - long checkpointId = 0; - long timestamp = 0; - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. - for (int i = 1; i <= 3; i++) { - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(0); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - } - - @TestTemplate - public void testMaxContinuousEmptyCommits() throws Exception { - table.updateProperties().set(MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); - - JobID jobId = new JobID(); - long checkpointId = 0; - long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - - assertSnapshotSize(0); - - for (int i = 1; i <= 9; i++) { - harness.snapshot(++checkpointId, ++timestamp); - harness.notifyOfCompletedCheckpoint(checkpointId); - - assertSnapshotSize(i / 3); - } - } - } - - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); - } - - @TestTemplate - public void testCommitTxn() throws Exception { - // Test with 3 continues checkpoints: - // 1. snapshotState for checkpoint#1 - // 2. notifyCheckpointComplete for checkpoint#1 - // 3. snapshotState for checkpoint#2 - // 4. notifyCheckpointComplete for checkpoint#2 - // 5. snapshotState for checkpoint#3 - // 6. notifyCheckpointComplete for checkpoint#3 - long timestamp = 0; - - JobID jobID = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - - List rows = Lists.newArrayListWithExpectedSize(3); - for (int i = 1; i <= 3; i++) { - RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); - DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - - harness.snapshot(i, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(i); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobID, operatorId, i); - assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) - .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); - } - } - } - - @TestTemplate - public void testOrderedEventsBetweenCheckpoints() throws Exception { - // It's possible that two checkpoints happen in the following orders: - // 1. snapshotState for checkpoint#1; - // 2. snapshotState for checkpoint#2; - // 3. notifyCheckpointComplete for checkpoint#1; - // 4. notifyCheckpointComplete for checkpoint#2; - long timestamp = 0; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; - harness.snapshot(firstCheckpointId, ++timestamp); - assertFlinkManifests(1); - - RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; - harness.snapshot(secondCheckpointId, ++timestamp); - assertFlinkManifests(2); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(firstCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, firstCheckpointId); - assertFlinkManifests(1); - - // 4. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(secondCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testDisorderedEventsBetweenCheckpoints() throws Exception { - // It's possible that the two checkpoints happen in the following orders: - // 1. snapshotState for checkpoint#1; - // 2. snapshotState for checkpoint#2; - // 3. notifyCheckpointComplete for checkpoint#2; - // 4. notifyCheckpointComplete for checkpoint#1; - long timestamp = 0; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; - harness.snapshot(firstCheckpointId, ++timestamp); - assertFlinkManifests(1); - - RowData row2 = SimpleDataUtil.createRowData(2, "world"); - DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; - harness.snapshot(secondCheckpointId, ++timestamp); - assertFlinkManifests(2); - - // 3. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(secondCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - - // 4. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(firstCheckpointId); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, secondCheckpointId); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testRecoveryFromValidSnapshot() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List expectedRows = Lists.newArrayList(); - OperatorSubtaskState snapshot; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row = SimpleDataUtil.createRowData(1, "hello"); - expectedRows.add(row); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row), branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - RowData row = SimpleDataUtil.createRowData(2, "world"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - - @TestTemplate - public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { - // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). It's - // possible that we - // flink job will restore from a checkpoint with only step#1 finished. - long checkpointId = 0; - long timestamp = 0; - OperatorSubtaskState snapshot; - List expectedRows = Lists.newArrayList(); - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row = SimpleDataUtil.createRowData(1, "hello"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - snapshot = harness.snapshot(++checkpointId, ++timestamp); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - assertFlinkManifests(1); - } - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - harness.snapshot(++checkpointId, ++timestamp); - // Did not write any new record, so it won't generate new manifest. - assertFlinkManifests(0); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - RowData row = SimpleDataUtil.createRowData(2, "world"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - snapshot = harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - } - - // Redeploying flink job from external checkpoint. - JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(newJobId)) { - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - assertMaxCommittedCheckpointId(newJobId, operatorId, -1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(3); - - RowData row = SimpleDataUtil.createRowData(3, "foo"); - expectedRows.add(row); - DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); - - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(newJobId, operatorId, checkpointId); - } - } - - @TestTemplate - public void testStartAnotherJobToWriteSameTable() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List rows = Lists.newArrayList(); - List tableRows = Lists.newArrayList(); - - JobID oldJobId = new JobID(); - OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(oldJobId)) { - harness.setup(); - harness.open(); - oldOperatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, -1L); - - for (int i = 1; i <= 3; i++) { - rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, checkpointId); - } - } - - // The new started job will start with checkpoint = 1 again. - checkpointId = 0; - timestamp = 0; - JobID newJobId = new JobID(); - OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = - createStreamSink(newJobId)) { - harness.setup(); - harness.open(); - newOperatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(3); - assertMaxCommittedCheckpointId(oldJobId, oldOperatorId, 3); - assertMaxCommittedCheckpointId(newJobId, newOperatorId, -1); - - rows.add(SimpleDataUtil.createRowData(2, "world")); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(newJobId, newOperatorId, checkpointId); - } - } - - @TestTemplate - public void testMultipleJobsWriteSameTable() throws Exception { - long timestamp = 0; - List tableRows = Lists.newArrayList(); - - JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; - OperatorID[] operatorIds = - new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; - for (int i = 0; i < 20; i++) { - int jobIndex = i % 3; - int checkpointId = i / 3; - JobID jobId = jobs[jobIndex]; - OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.open(); - - assertSnapshotSize(i); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId == 0 ? -1 : checkpointId); - - List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); - tableRows.addAll(rows); - - DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(checkpointId + 1, ++timestamp); - assertFlinkManifests(1); - - harness.notifyOfCompletedCheckpoint(checkpointId + 1); - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(i + 1); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId + 1); - } - } - } - - @TestTemplate - public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { - long checkpointId = 0; - long timestamp = 0; - List expectedRows = Lists.newArrayList(); - OperatorSubtaskState snapshot1; - OperatorSubtaskState snapshot2; - - JobID jobId = new JobID(); - OperatorID operatorId1 = new OperatorID(); - OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { - harness1.getStreamConfig().setOperatorID(operatorId1); - harness1.setup(); - harness1.open(); - harness2.getStreamConfig().setOperatorID(operatorId2); - harness2.setup(); - harness2.open(); - - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId1, -1L); - assertMaxCommittedCheckpointId(jobId, operatorId2, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); - expectedRows.add(row1); - DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); - - RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); - expectedRows.add(row2); - DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - - harness2.processElement(of(dataFile2), ++timestamp); - snapshot2 = harness2.snapshot(checkpointId, ++timestamp); - assertFlinkManifests(2); - - // Only notify one of the committers - harness1.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(1); - - // Only the first row is committed at this point - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, -1); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { - harness1.getStreamConfig().setOperatorID(operatorId1); - harness1.setup(); - harness1.initializeState(snapshot1); - harness1.open(); - - harness2.getStreamConfig().setOperatorID(operatorId2); - harness2.setup(); - harness2.initializeState(snapshot2); - harness2.open(); - - // All flink manifests should be cleaned because it has committed the unfinished iceberg - // transaction. - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(2); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); - - RowData row1 = SimpleDataUtil.createRowData(2, "world1"); - expectedRows.add(row1); - DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); - - RowData row2 = SimpleDataUtil.createRowData(2, "world2"); - expectedRows.add(row2); - DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); - harness2.snapshot(checkpointId, ++timestamp); - - assertFlinkManifests(2); - - harness1.notifyOfCompletedCheckpoint(checkpointId); - harness2.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, expectedRows, branch); - assertSnapshotSize(4); - assertMaxCommittedCheckpointId(jobId, operatorId1, checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId2, checkpointId); - } - } - - @TestTemplate - public void testBoundedStream() throws Exception { - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertFlinkManifests(0); - assertSnapshotSize(0); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); - - DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); - ((BoundedOneInput) harness.getOneInputOperator()).endInput(); - - assertFlinkManifests(0); - SimpleDataUtil.assertTableRows(table, tableRows, branch); - assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); - assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) - .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); - } - } - - @TestTemplate - public void testFlinkManifests() throws Exception { - long timestamp = 0; - final long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createRowData(1, "hello"); - DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - harness.snapshot(checkpoint, ++timestamp); - List manifestPaths = assertFlinkManifests(1); - Path manifestPath = manifestPaths.get(0); - assertThat(manifestPath.getFileName()) - .asString() - .isEqualTo( - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); - - // 2. Read the data files from manifests and assert. - List dataFiles = - FlinkManifestUtil.readDataFiles( - createTestingManifestFile(manifestPath), table.io(), table.specs()); - assertThat(dataFiles).hasSize(1); - TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testDeleteFiles() throws Exception { - assumeThat(formatVersion) - .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(1); - - long timestamp = 0; - long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - // 1. snapshotState for checkpoint#1 - harness.snapshot(checkpoint, ++timestamp); - List manifestPaths = assertFlinkManifests(1); - Path manifestPath = manifestPaths.get(0); - assertThat(manifestPath.getFileName()) - .asString() - .isEqualTo( - String.format("%s-%s-%05d-%d-%d-%05d.avro", jobId, operatorId, 0, 0, checkpoint, 1)); - - // 2. Read the data files from manifests and assert. - List dataFiles = - FlinkManifestUtil.readDataFiles( - createTestingManifestFile(manifestPath), table.io(), table.specs()); - assertThat(dataFiles).hasSize(1); - TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); - - // 3. notifyCheckpointComplete for checkpoint#1 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - - // 4. process both data files and delete files. - RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); - - RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); - DeleteFile deleteFile1 = - writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), - ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - - // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); - assertFlinkManifests(2); - - // 6. notifyCheckpointComplete for checkpoint#2 - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - } - } - - @TestTemplate - public void testCommitTwoCheckpointsInSingleTxn() throws Exception { - assumeThat(formatVersion) - .as("Only support equality-delete in format v2 or later.") - .isGreaterThan(1); - - long timestamp = 0; - long checkpoint = 10; - - JobID jobId = new JobID(); - OperatorID operatorId; - FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertMaxCommittedCheckpointId(jobId, operatorId, -1L); - - RowData insert1 = SimpleDataUtil.createInsert(1, "aaa"); - RowData insert2 = SimpleDataUtil.createInsert(2, "bbb"); - RowData delete3 = SimpleDataUtil.createDelete(3, "ccc"); - DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(insert1, insert2)); - DeleteFile deleteFile1 = - writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), - ++timestamp); - - // The 1th snapshotState. - harness.snapshot(checkpoint, ++timestamp); - - RowData insert4 = SimpleDataUtil.createInsert(4, "ddd"); - RowData delete2 = SimpleDataUtil.createDelete(2, "bbb"); - DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(insert4)); - DeleteFile deleteFile2 = - writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); - harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), - ++timestamp); - - // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); - - // Notify the 2nd snapshot to complete. - harness.notifyOfCompletedCheckpoint(checkpoint); - SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert4), branch); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); - assertFlinkManifests(0); - assertThat(table.snapshots()).hasSize(2); - } - } - - @TestTemplate - public void testSpecEvolution() throws Exception { - long timestamp = 0; - int checkpointId = 0; - List rows = Lists.newArrayList(); - JobID jobId = new JobID(); - - OperatorID operatorId; - OperatorSubtaskState snapshot; - DataFile dataFile; - int specId; - - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.setup(); - harness.open(); - operatorId = harness.getOperator().getOperatorID(); - - assertSnapshotSize(0); - - checkpointId++; - RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); - // table unpartitioned - dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - harness.snapshot(checkpointId, ++timestamp); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(table.spec().specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - - // Change partition spec - table.refresh(); - PartitionSpec oldSpec = table.spec(); - table.updateSpec().addField("id").commit(); - - checkpointId++; - rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); - // write data with old partition spec - dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); - rows.add(rowData); - snapshot = harness.snapshot(checkpointId, ++timestamp); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(oldSpec.specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - - // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { - harness.getStreamConfig().setOperatorID(operatorId); - harness.setup(); - harness.initializeState(snapshot); - harness.open(); - - SimpleDataUtil.assertTableRows(table, rows, branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - - checkpointId++; - RowData row = SimpleDataUtil.createRowData(checkpointId, "world" + checkpointId); - StructLike partition = new PartitionData(table.spec().partitionType()); - partition.set(0, checkpointId); - dataFile = - writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); - rows.add(row); - harness.snapshot(checkpointId, ++timestamp); - assertFlinkManifests(1); - - specId = - getStagingManifestSpecId(harness.getOperator().getOperatorStateBackend(), checkpointId); - assertThat(specId).isEqualTo(table.spec().specId()); - - harness.notifyOfCompletedCheckpoint(checkpointId); - assertFlinkManifests(0); - - SimpleDataUtil.assertTableRows(table, rows, branch); - assertSnapshotSize(checkpointId); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpointId); - } - } - - private int getStagingManifestSpecId(OperatorStateStore operatorStateStore, long checkPointId) - throws Exception { - ListState> checkpointsState = - operatorStateStore.getListState(IcebergFilesCommitter.buildStateDescriptor()); - NavigableMap statedDataFiles = - Maps.newTreeMap(checkpointsState.get().iterator().next()); - DeltaManifests deltaManifests = - SimpleVersionedSerialization.readVersionAndDeSerialize( - DeltaManifestsSerializer.INSTANCE, statedDataFiles.get(checkPointId)); - return deltaManifests.dataManifest().partitionSpecId(); - } - - private DeleteFile writeEqDeleteFile( - FileAppenderFactory appenderFactory, String filename, List deletes) - throws IOException { - return SimpleDataUtil.writeEqDeleteFile(table, format, filename, appenderFactory, deletes); - } - - private DeleteFile writePosDeleteFile( - FileAppenderFactory appenderFactory, - String filename, - List> positions) - throws IOException { - return SimpleDataUtil.writePosDeleteFile(table, format, filename, appenderFactory, positions); - } - - private FileAppenderFactory createDeletableAppenderFactory() { - int[] equalityFieldIds = - new int[] { - table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() - }; - return new FlinkAppenderFactory( - table, - table.schema(), - FlinkSchemaUtil.convert(table.schema()), - table.properties(), - table.spec(), - equalityFieldIds, - table.schema(), - null); - } - - private ManifestFile createTestingManifestFile(Path manifestPath) { - return new GenericManifestFile( - manifestPath.toAbsolutePath().toString(), - manifestPath.toFile().length(), - 0, - ManifestContent.DATA, - 0, - 0, - 0L, - 0, - 0, - 0, - 0, - 0, - 0, - null, - null); - } - - private List assertFlinkManifests(int expectedCount) throws IOException { - List manifests = - Files.list(flinkManifestFolder.toPath()) - .filter(p -> !p.toString().endsWith(".crc")) - .collect(Collectors.toList()); - assertThat(manifests).hasSize(expectedCount); - return manifests; - } - - private DataFile writeDataFile(String filename, List rows) throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - table.spec(), - CONF, - table.location(), - format.addExtension(filename), - rows); - } - - private DataFile writeDataFile( - String filename, List rows, PartitionSpec spec, StructLike partition) - throws IOException { - return SimpleDataUtil.writeFile( - table, - table.schema(), - spec, - CONF, - table.location(), - format.addExtension(filename), - rows, - partition); - } - - private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { - table.refresh(); - long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); - assertThat(actualId).isEqualTo(expectedId); - } - - private void assertSnapshotSize(int expectedSnapshotSize) { - table.refresh(); - assertThat(table.snapshots()).hasSize(expectedSnapshotSize); - } - - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) - throws Exception { - TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); - return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); - } - - private static MockEnvironment createEnvironment(JobID jobID) { - return new MockEnvironmentBuilder() - .setTaskName("test task") - .setManagedMemorySize(32 * 1024) - .setInputSplitProvider(new MockInputSplitProvider()) - .setBufferSize(256) - .setTaskConfiguration(new org.apache.flink.configuration.Configuration()) - .setExecutionConfig(new ExecutionConfig()) - .setMaxParallelism(16) - .setJobID(jobID) - .build(); - } - - private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { - private final String tablePath; - private final String branch; - private final PartitionSpec spec; - - private TestOperatorFactory(String tablePath, String branch, PartitionSpec spec) { - this.tablePath = tablePath; - this.branch = branch; - this.spec = spec; - } - - private static TestOperatorFactory of(String tablePath, String branch, PartitionSpec spec) { - return new TestOperatorFactory(tablePath, branch, spec); - } - - @Override - @SuppressWarnings("unchecked") - public > T createStreamOperator( - StreamOperatorParameters param) { - IcebergFilesCommitter committer = - new IcebergFilesCommitter( - new TestTableLoader(tablePath), - false, - Collections.singletonMap("flink.test", TestIcebergFilesCommitter.class.getName()), - ThreadPools.WORKER_THREAD_POOL_SIZE, - branch, - spec); - committer.setup(param.getContainingTask(), param.getStreamConfig(), param.getOutput()); - return (T) committer; - } - - @Override - public Class getStreamOperatorClass(ClassLoader classLoader) { - return IcebergFilesCommitter.class; - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java deleted file mode 100644 index 50283f7ad215..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ /dev/null @@ -1,390 +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.sink; - -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 java.util.Locale; -import java.util.Map; -import java.util.Set; -import org.apache.flink.streaming.api.operators.BoundedOneInput; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -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.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkWriteConf; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.hadoop.HadoopTables; -import org.apache.iceberg.io.WriteResult; -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.types.Types; -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 TestIcebergStreamWriter { - @TempDir protected java.nio.file.Path temporaryFolder; - - private Table table; - - @Parameter(index = 0) - private FileFormat format; - - @Parameter(index = 1) - private boolean partitioned; - - @Parameters(name = "format = {0}, partitioned = {1}") - public static Object[][] parameters() { - return new Object[][] { - {FileFormat.AVRO, true}, - {FileFormat.AVRO, false}, - {FileFormat.ORC, true}, - {FileFormat.ORC, false}, - {FileFormat.PARQUET, true}, - {FileFormat.PARQUET, false} - }; - } - - @BeforeEach - public void before() throws IOException { - File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); - // Construct the iceberg table. - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); - } - - @TestTemplate - public void testWritingTable() throws Exception { - long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - // The first checkpoint - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(3, "hello"), 1); - - testHarness.prepareSnapshotPreBarrier(checkpointId); - int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - - checkpointId = checkpointId + 1; - - // The second checkpoint - testHarness.processElement(SimpleDataUtil.createRowData(4, "foo"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(5, "bar"), 2); - - testHarness.prepareSnapshotPreBarrier(checkpointId); - expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - - // Commit the iceberg transaction. - AppendFiles appendFiles = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - - // Assert the table records. - SimpleDataUtil.assertTableRecords( - table, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "hello"), - SimpleDataUtil.createRecord(2, "world"), - SimpleDataUtil.createRecord(3, "hello"), - SimpleDataUtil.createRecord(4, "foo"), - SimpleDataUtil.createRecord(5, "bar"))); - } - } - - @TestTemplate - public void testSnapshotTwice() throws Exception { - long checkpointId = 1; - long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); - - testHarness.prepareSnapshotPreBarrier(checkpointId++); - int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - - // snapshot again immediately. - for (int i = 0; i < 5; i++) { - testHarness.prepareSnapshotPreBarrier(checkpointId++); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - } - } - } - - @TestTemplate - public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - assertThat(testHarness.extractOutputValues()).isEmpty(); - } - // Even if we closed the iceberg stream writer, there's no orphan data file. - assertThat(scanDataFiles()).isEmpty(); - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - // Still not emit the data file yet, because there is no checkpoint. - assertThat(testHarness.extractOutputValues()).isEmpty(); - } - // Once we closed the iceberg stream writer, there will left an orphan data file. - assertThat(scanDataFiles()).hasSize(1); - } - - private Set scanDataFiles() throws IOException { - Path dataDir = new Path(table.location(), "data"); - FileSystem fs = FileSystem.get(new Configuration()); - if (!fs.exists(dataDir)) { - return ImmutableSet.of(); - } else { - Set paths = Sets.newHashSet(); - RemoteIterator iterators = fs.listFiles(dataDir, true); - while (iterators.hasNext()) { - LocatedFileStatus status = iterators.next(); - if (status.isFile()) { - Path path = status.getPath(); - if (path.getName().endsWith("." + format.toString().toLowerCase(Locale.ROOT))) { - paths.add(path.toString()); - } - } - } - return paths; - } - } - - @TestTemplate - public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - - assertThat(testHarness.getOneInputOperator()).isInstanceOf(BoundedOneInput.class); - ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - - int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - - ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - // Datafiles should not be sent again - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - } - } - - @TestTemplate - public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); - testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); - - testHarness.endInput(); - - int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - - testHarness.prepareSnapshotPreBarrier(1L); - - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier - // is triggered, write should only send WriteResult once - assertThat(result.dataFiles()).hasSize(expectedDataFiles); - } - } - - @TestTemplate - public void testTableWithTargetFileSize() throws Exception { - // Adjust the target-file-size in table properties. - table - .updateProperties() - .set(TableProperties.WRITE_TARGET_FILE_SIZE_BYTES, "4") // ~4 bytes; low enough to trigger - .commit(); - - List rows = Lists.newArrayListWithCapacity(8000); - List records = Lists.newArrayListWithCapacity(8000); - for (int i = 0; i < 2000; i++) { - for (String data : new String[] {"a", "b", "c", "d"}) { - rows.add(SimpleDataUtil.createRowData(i, data)); - records.add(SimpleDataUtil.createRecord(i, data)); - } - } - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter()) { - for (RowData row : rows) { - testHarness.processElement(row, 1); - } - - // snapshot the operator. - testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(8); - - // Assert that the data file have the expected records. - for (DataFile dataFile : result.dataFiles()) { - assertThat(dataFile.recordCount()).isEqualTo(1000); - } - - // Commit the iceberg transaction. - AppendFiles appendFiles = table.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - } - - // Assert the table records. - SimpleDataUtil.assertTableRecords(table, records); - } - - @TestTemplate - public void testPromotedFlinkDataType() throws Exception { - Schema iSchema = - new Schema( - Types.NestedField.required(1, "tinyint", Types.IntegerType.get()), - Types.NestedField.required(2, "smallint", Types.IntegerType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get())); - TableSchema flinkSchema = - TableSchema.builder() - .field("tinyint", DataTypes.TINYINT().notNull()) - .field("smallint", DataTypes.SMALLINT().notNull()) - .field("int", DataTypes.INT().nullable()) - .build(); - - PartitionSpec spec; - if (partitioned) { - spec = - PartitionSpec.builderFor(iSchema) - .identity("smallint") - .identity("tinyint") - .identity("int") - .build(); - } else { - spec = PartitionSpec.unpartitioned(); - } - - String location = - Files.createTempDirectory(temporaryFolder, "junit").toFile().getAbsolutePath(); - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - Table icebergTable = new HadoopTables().create(iSchema, spec, props, location); - - List rows = - Lists.newArrayList( - GenericRowData.of((byte) 0x01, (short) -32768, 101), - GenericRowData.of((byte) 0x02, (short) 0, 102), - GenericRowData.of((byte) 0x03, (short) 32767, 103)); - - Record record = GenericRecord.create(iSchema); - List expected = - Lists.newArrayList( - record.copy(ImmutableMap.of("tinyint", 1, "smallint", -32768, "int", 101)), - record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), - record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - - try (OneInputStreamOperatorTestHarness testHarness = - createIcebergStreamWriter(icebergTable, flinkSchema)) { - for (RowData row : rows) { - testHarness.processElement(row, 1); - } - testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); - assertThat(result.deleteFiles()).isEmpty(); - assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); - - // Commit the iceberg transaction. - AppendFiles appendFiles = icebergTable.newAppend(); - Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); - appendFiles.commit(); - } - - SimpleDataUtil.assertTableRecords(location, expected); - } - - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() - throws Exception { - return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); - } - - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema) throws Exception { - RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); - FlinkWriteConf flinkWriteConfig = - new FlinkWriteConf( - icebergTable, Maps.newHashMap(), new org.apache.flink.configuration.Configuration()); - - IcebergStreamWriter streamWriter = - FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); - - harness.setup(); - harness.open(); - - return harness; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java deleted file mode 100644 index 919fef579ab0..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestRowDataPartitionKey.java +++ /dev/null @@ -1,251 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionKey; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.InternalRecordWrapper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestRowDataPartitionKey { - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(0, "boolType", Types.BooleanType.get()), - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "longType", Types.LongType.get()), - Types.NestedField.required(3, "dateType", Types.DateType.get()), - Types.NestedField.required(4, "timeType", Types.TimeType.get()), - Types.NestedField.required(5, "stringType", Types.StringType.get()), - Types.NestedField.required(6, "timestampWithoutZone", Types.TimestampType.withoutZone()), - Types.NestedField.required(7, "timestampWithZone", Types.TimestampType.withZone()), - Types.NestedField.required(8, "fixedType", Types.FixedType.ofLength(5)), - Types.NestedField.required(9, "uuidType", Types.UUIDType.get()), - Types.NestedField.required(10, "binaryType", Types.BinaryType.get()), - Types.NestedField.required(11, "decimalType1", Types.DecimalType.of(18, 3)), - Types.NestedField.required(12, "decimalType2", Types.DecimalType.of(10, 5)), - Types.NestedField.required(13, "decimalType3", Types.DecimalType.of(38, 19)), - Types.NestedField.required(14, "floatType", Types.FloatType.get()), - Types.NestedField.required(15, "doubleType", Types.DoubleType.get())); - - private static final List SUPPORTED_PRIMITIVES = - SCHEMA.asStruct().fields().stream().map(Types.NestedField::name).collect(Collectors.toList()); - - private static final Schema NESTED_SCHEMA = - new Schema( - Types.NestedField.required( - 1, - "structType", - Types.StructType.of( - Types.NestedField.optional(2, "innerStringType", Types.StringType.get()), - Types.NestedField.optional(3, "innerIntegerType", Types.IntegerType.get())))); - - @Test - public void testNullPartitionValue() { - Schema schema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "data", Types.StringType.get())); - - PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").build(); - - List rows = - Lists.newArrayList( - GenericRowData.of(1, StringData.fromString("a")), - GenericRowData.of(2, StringData.fromString("b")), - GenericRowData.of(3, null)); - - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema), schema.asStruct()); - - for (RowData row : rows) { - PartitionKey partitionKey = new PartitionKey(spec, schema); - partitionKey.partition(rowWrapper.wrap(row)); - assertThat(partitionKey.size()).isEqualTo(1); - - String expectedStr = row.isNullAt(1) ? null : row.getString(1).toString(); - assertThat(partitionKey.get(0, String.class)).isEqualTo(expectedStr); - } - } - - @Test - public void testPartitionWithOneNestedField() { - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); - List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1991); - List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); - - PartitionSpec spec1 = - PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerStringType").build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(NESTED_SCHEMA).identity("structType.innerIntegerType").build(); - - for (int i = 0; i < rows.size(); i++) { - RowData row = rows.get(i); - Record record = (Record) records.get(i).get(0); - - PartitionKey partitionKey1 = new PartitionKey(spec1, NESTED_SCHEMA); - partitionKey1.partition(rowWrapper.wrap(row)); - assertThat(partitionKey1.size()).isEqualTo(1); - - assertThat(partitionKey1.get(0, String.class)).isEqualTo(record.get(0)); - - PartitionKey partitionKey2 = new PartitionKey(spec2, NESTED_SCHEMA); - partitionKey2.partition(rowWrapper.wrap(row)); - assertThat(partitionKey2.size()).isEqualTo(1); - - assertThat(partitionKey2.get(0, Integer.class)).isEqualTo(record.get(1)); - } - } - - @Test - public void testPartitionMultipleNestedField() { - RowDataWrapper rowWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(NESTED_SCHEMA), NESTED_SCHEMA.asStruct()); - List records = RandomGenericData.generate(NESTED_SCHEMA, 10, 1992); - List rows = Lists.newArrayList(RandomRowData.convert(NESTED_SCHEMA, records)); - - PartitionSpec spec1 = - PartitionSpec.builderFor(NESTED_SCHEMA) - .identity("structType.innerIntegerType") - .identity("structType.innerStringType") - .build(); - PartitionSpec spec2 = - PartitionSpec.builderFor(NESTED_SCHEMA) - .identity("structType.innerStringType") - .identity("structType.innerIntegerType") - .build(); - - PartitionKey pk1 = new PartitionKey(spec1, NESTED_SCHEMA); - PartitionKey pk2 = new PartitionKey(spec2, NESTED_SCHEMA); - - for (int i = 0; i < rows.size(); i++) { - RowData row = rows.get(i); - Record record = (Record) records.get(i).get(0); - - pk1.partition(rowWrapper.wrap(row)); - assertThat(pk1.size()).isEqualTo(2); - - assertThat(pk1.get(0, Integer.class)).isEqualTo(record.get(1)); - assertThat(pk1.get(1, String.class)).isEqualTo(record.get(0)); - - pk2.partition(rowWrapper.wrap(row)); - assertThat(pk2.size()).isEqualTo(2); - - assertThat(pk2.get(0, String.class)).isEqualTo(record.get(0)); - assertThat(pk2.get(1, Integer.class)).isEqualTo(record.get(1)); - } - } - - @Test - public void testPartitionValueTypes() { - RowType rowType = FlinkSchemaUtil.convert(SCHEMA); - RowDataWrapper rowWrapper = new RowDataWrapper(rowType, SCHEMA.asStruct()); - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(SCHEMA.asStruct()); - - List records = RandomGenericData.generate(SCHEMA, 10, 1993); - List rows = Lists.newArrayList(RandomRowData.convert(SCHEMA, records)); - - for (String column : SUPPORTED_PRIMITIVES) { - PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity(column).build(); - Class[] javaClasses = spec.javaClasses(); - - PartitionKey pk = new PartitionKey(spec, SCHEMA); - PartitionKey expectedPK = new PartitionKey(spec, SCHEMA); - - for (int j = 0; j < rows.size(); j++) { - RowData row = rows.get(j); - Record record = records.get(j); - - pk.partition(rowWrapper.wrap(row)); - expectedPK.partition(recordWrapper.wrap(record)); - - assertThat(pk.size()) - .as("Partition with column " + column + " should have one field.") - .isEqualTo(1); - - if (column.equals("timeType")) { - assertThat(pk.get(0, Long.class) / 1000) - .as("Partition with column " + column + " should have the expected values") - .isEqualTo(expectedPK.get(0, Long.class) / 1000); - } else { - assertThat(pk.get(0, javaClasses[0])) - .as("Partition with column " + column + " should have the expected values") - .isEqualTo(expectedPK.get(0, javaClasses[0])); - } - } - } - } - - @Test - public void testNestedPartitionValues() { - Schema nestedSchema = new Schema(Types.NestedField.optional(1001, "nested", SCHEMA.asStruct())); - RowType rowType = FlinkSchemaUtil.convert(nestedSchema); - - RowDataWrapper rowWrapper = new RowDataWrapper(rowType, nestedSchema.asStruct()); - InternalRecordWrapper recordWrapper = new InternalRecordWrapper(nestedSchema.asStruct()); - - List records = RandomGenericData.generate(nestedSchema, 10, 1994); - List rows = Lists.newArrayList(RandomRowData.convert(nestedSchema, records)); - - for (String supportedPrimitive : SUPPORTED_PRIMITIVES) { - String column = String.format("nested.%s", supportedPrimitive); - - PartitionSpec spec = PartitionSpec.builderFor(nestedSchema).identity(column).build(); - Class[] javaClasses = spec.javaClasses(); - - PartitionKey pk = new PartitionKey(spec, nestedSchema); - PartitionKey expectedPK = new PartitionKey(spec, nestedSchema); - - for (int j = 0; j < rows.size(); j++) { - pk.partition(rowWrapper.wrap(rows.get(j))); - expectedPK.partition(recordWrapper.wrap(records.get(j))); - - assertThat(pk.size()) - .as("Partition with nested column " + column + " should have one field.") - .isEqualTo(1); - - if (column.equals("nested.timeType")) { - assertThat(pk.get(0, Long.class) / 1000) - .as("Partition with nested column " + column + " should have the expected values.") - .isEqualTo(expectedPK.get(0, Long.class) / 1000); - } else { - assertThat(pk.get(0, javaClasses[0])) - .as("Partition with nested column " + column + " should have the expected values.") - .isEqualTo(expectedPK.get(0, javaClasses[0])); - } - } - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java deleted file mode 100644 index 8bfd6cb3d043..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/TestTaskWriters.java +++ /dev/null @@ -1,242 +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.sink; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Files; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.data.RandomRowData; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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 TestTaskWriters { - private static final Configuration CONF = new Configuration(); - private static final long TARGET_FILE_SIZE = 128 * 1024 * 1024; - - @TempDir protected java.nio.file.Path temporaryFolder; - - @Parameters(name = "format = {0}, partitioned = {1}") - public static Object[][] parameters() { - return new Object[][] { - {FileFormat.AVRO, true}, - {FileFormat.AVRO, false}, - {FileFormat.ORC, true}, - {FileFormat.ORC, false}, - {FileFormat.PARQUET, true}, - {FileFormat.PARQUET, false} - }; - } - - @Parameter(index = 0) - private FileFormat format; - - @Parameter(index = 1) - private boolean partitioned; - - private Table table; - - @BeforeEach - public void before() throws IOException { - File folder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); - // Construct the iceberg table with the specified file format. - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - table = SimpleDataUtil.createTable(folder.getAbsolutePath(), props, partitioned); - } - - @TestTemplate - public void testWriteZeroRecord() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.close(); - - DataFile[] dataFiles = taskWriter.dataFiles(); - assertThat(dataFiles).isNotNull().isEmpty(); - - // Close again. - taskWriter.close(); - dataFiles = taskWriter.dataFiles(); - assertThat(dataFiles).isNotNull().isEmpty(); - } - } - - @TestTemplate - public void testCloseTwice() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - taskWriter.close(); // The first close - taskWriter.close(); // The second close - - int expectedFiles = partitioned ? 2 : 1; - DataFile[] dataFiles = taskWriter.dataFiles(); - assertThat(dataFiles).hasSize(expectedFiles); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); - } - } - } - - @TestTemplate - public void testAbort() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "hello")); - taskWriter.write(SimpleDataUtil.createRowData(2, "world")); - - taskWriter.abort(); - DataFile[] dataFiles = taskWriter.dataFiles(); - - int expectedFiles = partitioned ? 2 : 1; - assertThat(dataFiles).hasSize(expectedFiles); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isFalse(); - } - } - } - - @TestTemplate - public void testCompleteFiles() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - taskWriter.write(SimpleDataUtil.createRowData(1, "a")); - taskWriter.write(SimpleDataUtil.createRowData(2, "b")); - taskWriter.write(SimpleDataUtil.createRowData(3, "c")); - taskWriter.write(SimpleDataUtil.createRowData(4, "d")); - - DataFile[] dataFiles = taskWriter.dataFiles(); - int expectedFiles = partitioned ? 4 : 1; - assertThat(dataFiles).hasSize(expectedFiles); - - dataFiles = taskWriter.dataFiles(); - assertThat(dataFiles).hasSize(expectedFiles); - - FileSystem fs = FileSystem.get(CONF); - for (DataFile dataFile : dataFiles) { - assertThat(fs.exists(new Path(dataFile.path().toString()))).isTrue(); - } - - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRecords( - table, - Lists.newArrayList( - SimpleDataUtil.createRecord(1, "a"), - SimpleDataUtil.createRecord(2, "b"), - SimpleDataUtil.createRecord(3, "c"), - SimpleDataUtil.createRecord(4, "d"))); - } - } - - @TestTemplate - public void testRollingWithTargetFileSize() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(4)) { - List rows = Lists.newArrayListWithCapacity(8000); - List records = Lists.newArrayListWithCapacity(8000); - for (int i = 0; i < 2000; i++) { - for (String data : new String[] {"a", "b", "c", "d"}) { - rows.add(SimpleDataUtil.createRowData(i, data)); - records.add(SimpleDataUtil.createRecord(i, data)); - } - } - - for (RowData row : rows) { - taskWriter.write(row); - } - - DataFile[] dataFiles = taskWriter.dataFiles(); - assertThat(dataFiles).hasSize(8); - - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRecords(table, records); - } - } - - @TestTemplate - public void testRandomData() throws IOException { - try (TaskWriter taskWriter = createTaskWriter(TARGET_FILE_SIZE)) { - Iterable rows = RandomRowData.generate(SimpleDataUtil.SCHEMA, 100, 1996); - for (RowData row : rows) { - taskWriter.write(row); - } - - taskWriter.close(); - DataFile[] dataFiles = taskWriter.dataFiles(); - AppendFiles appendFiles = table.newAppend(); - for (DataFile dataFile : dataFiles) { - appendFiles.appendFile(dataFile); - } - appendFiles.commit(); - - // Assert the data rows. - SimpleDataUtil.assertTableRows(table, Lists.newArrayList(rows)); - } - } - - private TaskWriter createTaskWriter(long targetFileSize) { - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - (RowType) SimpleDataUtil.FLINK_SCHEMA.toRowDataType().getLogicalType(), - targetFileSize, - format, - table.properties(), - null, - false); - taskWriterFactory.initialize(1, 1); - return taskWriterFactory.create(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java deleted file mode 100644 index 5910bd685510..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/Fixtures.java +++ /dev/null @@ -1,100 +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.sink.shuffle; - -import java.util.Comparator; -import java.util.Map; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.runtime.typeutils.RowDataSerializer; -import org.apache.flink.table.types.logical.IntType; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.types.logical.VarCharType; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.SortOrderComparators; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; - -class Fixtures { - private Fixtures() {} - - public static final int NUM_SUBTASKS = 2; - public static final Schema SCHEMA = - new Schema( - Types.NestedField.optional(1, "id", Types.StringType.get()), - Types.NestedField.optional(2, "number", Types.IntegerType.get())); - public static final RowType ROW_TYPE = RowType.of(new VarCharType(), new IntType()); - public static final TypeSerializer ROW_SERIALIZER = new RowDataSerializer(ROW_TYPE); - public static final RowDataWrapper ROW_WRAPPER = new RowDataWrapper(ROW_TYPE, SCHEMA.asStruct()); - public static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); - public static final Comparator SORT_ORDER_COMPARTOR = - SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); - public static final SortKeySerializer SORT_KEY_SERIALIZER = - new SortKeySerializer(SCHEMA, SORT_ORDER); - public static final DataStatisticsSerializer TASK_STATISTICS_SERIALIZER = - new DataStatisticsSerializer(SORT_KEY_SERIALIZER); - public static final GlobalStatisticsSerializer GLOBAL_STATISTICS_SERIALIZER = - new GlobalStatisticsSerializer(SORT_KEY_SERIALIZER); - public static final CompletedStatisticsSerializer COMPLETED_STATISTICS_SERIALIZER = - new CompletedStatisticsSerializer(SORT_KEY_SERIALIZER); - - public static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); - public static final Map CHAR_KEYS = createCharKeys(); - - public static StatisticsEvent createStatisticsEvent( - StatisticsType type, - TypeSerializer statisticsSerializer, - long checkpointId, - SortKey... keys) { - DataStatistics statistics = createTaskStatistics(type, keys); - return StatisticsEvent.createTaskStatisticsEvent( - checkpointId, statistics, statisticsSerializer); - } - - public static DataStatistics createTaskStatistics(StatisticsType type, SortKey... keys) { - DataStatistics statistics; - if (type == StatisticsType.Sketch) { - statistics = new SketchDataStatistics(128); - } else { - statistics = new MapDataStatistics(); - } - - for (SortKey key : keys) { - statistics.add(key); - } - - return statistics; - } - - private static Map createCharKeys() { - Map keys = Maps.newHashMap(); - for (char c = 'a'; c <= 'z'; ++c) { - String key = Character.toString(c); - SortKey sortKey = SORT_KEY.copy(); - sortKey.set(0, key); - keys.put(key, sortKey); - } - - return keys; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java deleted file mode 100644 index 8322ce683768..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestAggregatedStatisticsTracker.java +++ /dev/null @@ -1,465 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestAggregatedStatisticsTracker { - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void receiveNewerStatisticsEvent(StatisticsType type) { - AggregatedStatisticsTracker tracker = createTracker(type); - - StatisticsEvent checkpoint1Subtask0StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); - CompletedStatistics completedStatistics = - tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); - AggregatedStatisticsTracker.Aggregation aggregation = - tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); - } - - StatisticsEvent checkpoint2Subtask0StatisticsEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 2L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b")); - completedStatistics = tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); - assertThat(completedStatistics).isNull(); - // both checkpoints are tracked - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); - aggregation = tracker.aggregationsPerCheckpoint().get(2L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - } - - StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); - completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); - // checkpoint 1 is completed - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - assertThat(completedStatistics.checkpointId()).isEqualTo(1L); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), 1L, - CHAR_KEYS.get("b"), 1L)); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - } - - // checkpoint 2 remains - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); - aggregation = tracker.aggregationsPerCheckpoint().get(2L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - } - } - - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void receiveOlderStatisticsEventTest(StatisticsType type) { - AggregatedStatisticsTracker tracker = createTracker(type); - - StatisticsEvent checkpoint2Subtask0StatisticsEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 2L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b")); - CompletedStatistics completedStatistics = - tracker.updateAndCheckCompletion(0, checkpoint2Subtask0StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); - AggregatedStatisticsTracker.Aggregation aggregation = - tracker.aggregationsPerCheckpoint().get(2L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - } - - StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); - completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); - assertThat(completedStatistics).isNull(); - // both checkpoints are tracked - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L); - aggregation = tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("b"), 1L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("b")); - } - - StatisticsEvent checkpoint3Subtask0StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 3L, CHAR_KEYS.get("x")); - completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint3Subtask0StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L, 2L, 3L); - aggregation = tracker.aggregationsPerCheckpoint().get(3L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); - } - - StatisticsEvent checkpoint2Subtask1StatisticsEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 2L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b")); - completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint2Subtask1StatisticsEvent); - // checkpoint 1 is cleared along with checkpoint 2. checkpoint 3 remains - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(3L); - aggregation = tracker.aggregationsPerCheckpoint().get(3L); - assertThat(aggregation.currentType()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("x"), 1L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("x")); - } - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - assertThat(completedStatistics.checkpointId()).isEqualTo(2L); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), 2L, - CHAR_KEYS.get("b"), 4L)); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly( - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b")); - } - } - - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void receiveCompletedStatisticsEvent(StatisticsType type) { - AggregatedStatisticsTracker tracker = createTracker(type); - - StatisticsEvent checkpoint1Subtask0DataStatisticEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b")); - - CompletedStatistics completedStatistics = - tracker.updateAndCheckCompletion(0, checkpoint1Subtask0DataStatisticEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); - AggregatedStatisticsTracker.Aggregation aggregation = - tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("b")); - } - - StatisticsEvent checkpoint1Subtask1DataStatisticEvent = - createStatisticsEvent( - type, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b")); - - // Receive data statistics from all subtasks at checkpoint 1 - completedStatistics = - tracker.updateAndCheckCompletion(1, checkpoint1Subtask1DataStatisticEvent); - assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - assertThat(completedStatistics.checkpointId()).isEqualTo(1L); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), 3L, - CHAR_KEYS.get("b"), 3L)); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly( - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("a"), - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b")); - } - - StatisticsEvent checkpoint2Subtask0DataStatisticEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("a")); - completedStatistics = - tracker.updateAndCheckCompletion(0, checkpoint2Subtask0DataStatisticEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(2L); - aggregation = tracker.aggregationsPerCheckpoint().get(2L); - assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(aggregation.mapStatistics()).isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); - } else { - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder(CHAR_KEYS.get("a")); - } - - StatisticsEvent checkpoint2Subtask1DataStatisticEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("b")); - // Receive data statistics from all subtasks at checkpoint 2 - completedStatistics = - tracker.updateAndCheckCompletion(1, checkpoint2Subtask1DataStatisticEvent); - assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); - - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.checkpointId()).isEqualTo(2L); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()) - .isEqualTo( - ImmutableMap.of( - CHAR_KEYS.get("a"), 1L, - CHAR_KEYS.get("b"), 1L)); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - } - } - - @Test - public void coordinatorSwitchToSketchOverThreshold() { - int parallelism = 3; - int downstreamParallelism = 3; - int switchToSketchThreshold = 3; - AggregatedStatisticsTracker tracker = - new AggregatedStatisticsTracker( - "testOperator", - parallelism, - Fixtures.SCHEMA, - Fixtures.SORT_ORDER, - downstreamParallelism, - StatisticsType.Auto, - switchToSketchThreshold, - null); - - StatisticsEvent checkpoint1Subtask0StatisticsEvent = - createStatisticsEvent( - StatisticsType.Map, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b")); - CompletedStatistics completedStatistics = - tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); - AggregatedStatisticsTracker.Aggregation aggregation = - tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); - assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); - assertThat(aggregation.sketchStatistics()).isNull(); - assertThat(aggregation.mapStatistics()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); - - StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent( - StatisticsType.Map, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d")); - completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); - aggregation = tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); - // converted to sketch statistics as map size is 4 (over the switch threshold of 3) - assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); - assertThat(aggregation.mapStatistics()).isNull(); - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder( - CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); - - StatisticsEvent checkpoint1Subtask2StatisticsEvent = - createStatisticsEvent( - StatisticsType.Map, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f")); - completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); - assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); - assertThat(completedStatistics.keySamples()) - .containsExactly( - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d"), - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f")); - } - - @Test - public void coordinatorMapOperatorSketch() { - int parallelism = 3; - int downstreamParallelism = 3; - AggregatedStatisticsTracker tracker = - new AggregatedStatisticsTracker( - "testOperator", - parallelism, - Fixtures.SCHEMA, - Fixtures.SORT_ORDER, - downstreamParallelism, - StatisticsType.Auto, - SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, - null); - - // first operator event has map statistics - StatisticsEvent checkpoint1Subtask0StatisticsEvent = - createStatisticsEvent( - StatisticsType.Map, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b")); - CompletedStatistics completedStatistics = - tracker.updateAndCheckCompletion(0, checkpoint1Subtask0StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); - AggregatedStatisticsTracker.Aggregation aggregation = - tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0); - assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Map); - assertThat(aggregation.sketchStatistics()).isNull(); - assertThat(aggregation.mapStatistics()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); - - // second operator event contains sketch statistics - StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent( - StatisticsType.Sketch, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d")); - completedStatistics = tracker.updateAndCheckCompletion(1, checkpoint1Subtask1StatisticsEvent); - assertThat(completedStatistics).isNull(); - assertThat(tracker.aggregationsPerCheckpoint().keySet()).containsExactlyInAnyOrder(1L); - aggregation = tracker.aggregationsPerCheckpoint().get(1L); - assertThat(aggregation.subtaskSet()).containsExactlyInAnyOrder(0, 1); - assertThat(aggregation.currentType()).isEqualTo(StatisticsType.Sketch); - assertThat(aggregation.mapStatistics()).isNull(); - assertThat(aggregation.sketchStatistics().getResult().getSamples()) - .containsExactlyInAnyOrder( - CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("d")); - - // third operator event has Map statistics - StatisticsEvent checkpoint1Subtask2StatisticsEvent = - createStatisticsEvent( - StatisticsType.Map, - TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f")); - completedStatistics = tracker.updateAndCheckCompletion(2, checkpoint1Subtask2StatisticsEvent); - assertThat(tracker.aggregationsPerCheckpoint()).isEmpty(); - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsType.Sketch); - assertThat(completedStatistics.keySamples()) - .containsExactly( - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d"), - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f")); - } - - private AggregatedStatisticsTracker createTracker(StatisticsType type) { - return new AggregatedStatisticsTracker( - "testOperator", - Fixtures.NUM_SUBTASKS, - Fixtures.SCHEMA, - Fixtures.SORT_ORDER, - Fixtures.NUM_SUBTASKS, - type, - SketchUtil.COORDINATOR_SKETCH_SWITCH_THRESHOLD, - null); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java deleted file mode 100644 index 4ee9888934a8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestCompletedStatisticsSerializer.java +++ /dev/null @@ -1,54 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; - -import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; - -public class TestCompletedStatisticsSerializer extends SerializerTestBase { - - @Override - protected TypeSerializer createSerializer() { - return Fixtures.COMPLETED_STATISTICS_SERIALIZER; - } - - @Override - protected int getLength() { - return -1; - } - - @Override - protected Class getTypeClass() { - return CompletedStatistics.class; - } - - @Override - protected CompletedStatistics[] getTestData() { - - return new CompletedStatistics[] { - CompletedStatistics.fromKeyFrequency( - 1L, ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L)), - CompletedStatistics.fromKeySamples(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) - }; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java deleted file mode 100644 index a08a8a73e80c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinator.java +++ /dev/null @@ -1,246 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.NUM_SUBTASKS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.time.Duration; -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.util.ExceptionUtils; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestDataStatisticsCoordinator { - private static final String OPERATOR_NAME = "TestCoordinator"; - private static final OperatorID TEST_OPERATOR_ID = new OperatorID(1234L, 5678L); - - private EventReceivingTasks receivingTasks; - - @BeforeEach - public void before() throws Exception { - receivingTasks = EventReceivingTasks.createForRunningTasks(); - } - - private void tasksReady(DataStatisticsCoordinator coordinator) { - setAllTasksReady(NUM_SUBTASKS, coordinator, receivingTasks); - } - - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testThrowExceptionWhenNotStarted(StatisticsType type) throws Exception { - try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { - String failureMessage = "The coordinator of TestCoordinator has not started yet."; - assertThatThrownBy( - () -> - dataStatisticsCoordinator.handleEventFromOperator( - 0, - 0, - StatisticsEvent.createTaskStatisticsEvent( - 0, new MapDataStatistics(), Fixtures.TASK_STATISTICS_SERIALIZER))) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.executionAttemptFailed(0, 0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - assertThatThrownBy(() -> dataStatisticsCoordinator.checkpointCoordinator(0, null)) - .isInstanceOf(IllegalStateException.class) - .hasMessage(failureMessage); - } - } - - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testDataStatisticsEventHandling(StatisticsType type) throws Exception { - try (DataStatisticsCoordinator dataStatisticsCoordinator = createCoordinator(type)) { - dataStatisticsCoordinator.start(); - tasksReady(dataStatisticsCoordinator); - - StatisticsEvent checkpoint1Subtask0DataStatisticEvent = - Fixtures.createStatisticsEvent( - type, - Fixtures.TASK_STATISTICS_SERIALIZER, - 1L, - CHAR_KEYS.get("a"), - 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, - CHAR_KEYS.get("a"), - 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( - CHAR_KEYS.get("a"), 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( - CHAR_KEYS.get("a"), - CHAR_KEYS.get("a"), - 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 = - createCoordinator(StatisticsType.Sketch)) { - dataStatisticsCoordinator.start(); - tasksReady(dataStatisticsCoordinator); - - // receive request before global statistics is ready - dataStatisticsCoordinator.handleEventFromOperator(0, 0, new RequestGlobalStatisticsEvent()); - assertThat(receivingTasks.getSentEventsForSubtask(0)).isEmpty(); - assertThat(receivingTasks.getSentEventsForSubtask(1)).isEmpty(); - - StatisticsEvent checkpoint1Subtask0DataStatisticEvent = - Fixtures.createStatisticsEvent( - StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); - StatisticsEvent checkpoint1Subtask1DataStatisticEvent = - Fixtures.createStatisticsEvent( - StatisticsType.Sketch, Fixtures.TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); - // Handle events from operators for checkpoint 1 - dataStatisticsCoordinator.handleEventFromOperator( - 0, 0, checkpoint1Subtask0DataStatisticEvent); - dataStatisticsCoordinator.handleEventFromOperator( - 1, 0, checkpoint1Subtask1DataStatisticEvent); - - waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - Awaitility.await("wait for statistics event") - .pollInterval(Duration.ofMillis(10)) - .atMost(Duration.ofSeconds(10)) - .until(() -> receivingTasks.getSentEventsForSubtask(0).size() == 1); - assertThat(receivingTasks.getSentEventsForSubtask(0).get(0)) - .isInstanceOf(StatisticsEvent.class); - - Awaitility.await("wait for statistics event") - .pollInterval(Duration.ofMillis(10)) - .atMost(Duration.ofSeconds(10)) - .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 1); - assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) - .isInstanceOf(StatisticsEvent.class); - - dataStatisticsCoordinator.handleEventFromOperator(1, 0, new RequestGlobalStatisticsEvent()); - - // coordinator should send a response to subtask 1 - Awaitility.await("wait for statistics event") - .pollInterval(Duration.ofMillis(10)) - .atMost(Duration.ofSeconds(10)) - .until(() -> receivingTasks.getSentEventsForSubtask(1).size() == 2); - assertThat(receivingTasks.getSentEventsForSubtask(1).get(0)) - .isInstanceOf(StatisticsEvent.class); - assertThat(receivingTasks.getSentEventsForSubtask(1).get(1)) - .isInstanceOf(StatisticsEvent.class); - } - } - - static void setAllTasksReady( - int subtasks, - DataStatisticsCoordinator dataStatisticsCoordinator, - EventReceivingTasks receivingTasks) { - for (int i = 0; i < subtasks; i++) { - dataStatisticsCoordinator.executionAttemptReady( - i, 0, receivingTasks.createGatewayForSubtask(i, 0)); - } - } - - static void waitForCoordinatorToProcessActions(DataStatisticsCoordinator coordinator) { - CompletableFuture future = new CompletableFuture<>(); - coordinator.callInCoordinatorThread( - () -> { - future.complete(null); - return null; - }, - "Coordinator fails to process action"); - - try { - future.get(); - } catch (InterruptedException e) { - throw new AssertionError("test interrupted"); - } catch (ExecutionException e) { - ExceptionUtils.rethrow(ExceptionUtils.stripExecutionException(e)); - } - } - - private static DataStatisticsCoordinator createCoordinator(StatisticsType type) { - return new DataStatisticsCoordinator( - OPERATOR_NAME, - new MockOperatorCoordinatorContext(TEST_OPERATOR_ID, NUM_SUBTASKS), - Fixtures.SCHEMA, - Fixtures.SORT_ORDER, - NUM_SUBTASKS, - type, - 0.0d); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java deleted file mode 100644 index 6317f2bfde18..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsCoordinatorProvider.java +++ /dev/null @@ -1,187 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.TASK_STATISTICS_SERIALIZER; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.createStatisticsEvent; -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.ExecutionException; -import org.apache.flink.runtime.jobgraph.OperatorID; -import org.apache.flink.runtime.operators.coordination.EventReceivingTasks; -import org.apache.flink.runtime.operators.coordination.MockOperatorCoordinatorContext; -import org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.EnumSource; - -public class TestDataStatisticsCoordinatorProvider { - private static final OperatorID OPERATOR_ID = new OperatorID(); - - private EventReceivingTasks receivingTasks; - - @BeforeEach - public void before() { - receivingTasks = EventReceivingTasks.createForRunningTasks(); - } - - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testCheckpointAndReset(StatisticsType type) throws Exception { - DataStatisticsCoordinatorProvider provider = createProvider(type, Fixtures.NUM_SUBTASKS); - try (RecreateOnResetOperatorCoordinator coordinator = - (RecreateOnResetOperatorCoordinator) - provider.create( - new MockOperatorCoordinatorContext(OPERATOR_ID, Fixtures.NUM_SUBTASKS))) { - DataStatisticsCoordinator dataStatisticsCoordinator = - (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); - - // Start the coordinator - coordinator.start(); - TestDataStatisticsCoordinator.setAllTasksReady( - Fixtures.NUM_SUBTASKS, dataStatisticsCoordinator, receivingTasks); - - // Handle events from operators for checkpoint 1 - StatisticsEvent checkpoint1Subtask0StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("a")); - coordinator.handleEventFromOperator(0, 0, checkpoint1Subtask0StatisticsEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - StatisticsEvent checkpoint1Subtask1StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 1L, CHAR_KEYS.get("b")); - coordinator.handleEventFromOperator(1, 0, checkpoint1Subtask1StatisticsEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify checkpoint 1 global data statistics - Map checkpoint1KeyFrequency = - ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L); - MapAssignment checkpoint1MapAssignment = - MapAssignment.fromKeyFrequency( - Fixtures.NUM_SUBTASKS, checkpoint1KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); - - CompletedStatistics completedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(completedStatistics).isNotNull(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint1KeyFrequency); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - } - - GlobalStatistics globalStatistics = dataStatisticsCoordinator.globalStatistics(); - assertThat(globalStatistics).isNotNull(); - assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); - } else { - assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); - } - - byte[] checkpoint1Bytes = waitForCheckpoint(1L, dataStatisticsCoordinator); - - StatisticsEvent checkpoint2Subtask0StatisticsEvent = - createStatisticsEvent( - type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("d"), CHAR_KEYS.get("e")); - coordinator.handleEventFromOperator(0, 0, checkpoint2Subtask0StatisticsEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - StatisticsEvent checkpoint2Subtask1StatisticsEvent = - createStatisticsEvent(type, TASK_STATISTICS_SERIALIZER, 2L, CHAR_KEYS.get("f")); - coordinator.handleEventFromOperator(1, 0, checkpoint2Subtask1StatisticsEvent); - TestDataStatisticsCoordinator.waitForCoordinatorToProcessActions(dataStatisticsCoordinator); - - // Verify checkpoint 2 global data statistics - Map checkpoint2KeyFrequency = - ImmutableMap.of(CHAR_KEYS.get("d"), 1L, CHAR_KEYS.get("e"), 1L, CHAR_KEYS.get("f"), 1L); - MapAssignment checkpoint2MapAssignment = - MapAssignment.fromKeyFrequency( - Fixtures.NUM_SUBTASKS, checkpoint2KeyFrequency, 0.0d, SORT_ORDER_COMPARTOR); - completedStatistics = dataStatisticsCoordinator.completedStatistics(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()).isEqualTo(checkpoint2KeyFrequency); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly(CHAR_KEYS.get("d"), CHAR_KEYS.get("e"), CHAR_KEYS.get("f")); - } - - globalStatistics = dataStatisticsCoordinator.globalStatistics(); - assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint2MapAssignment); - } else { - assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("e")); - } - - waitForCheckpoint(2L, dataStatisticsCoordinator); - - // Reset coordinator to checkpoint 1 - coordinator.resetToCheckpoint(1L, checkpoint1Bytes); - DataStatisticsCoordinator restoredDataStatisticsCoordinator = - (DataStatisticsCoordinator) coordinator.getInternalCoordinator(); - assertThat(dataStatisticsCoordinator).isNotSameAs(restoredDataStatisticsCoordinator); - - completedStatistics = restoredDataStatisticsCoordinator.completedStatistics(); - assertThat(completedStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - // Verify restored data statistics - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(completedStatistics.keyFrequency()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 1L)); - } else { - assertThat(completedStatistics.keySamples()) - .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - } - - globalStatistics = restoredDataStatisticsCoordinator.globalStatistics(); - assertThat(globalStatistics).isNotNull(); - assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.mapAssignment()).isEqualTo(checkpoint1MapAssignment); - } else { - assertThat(globalStatistics.rangeBounds()).containsExactly(CHAR_KEYS.get("a")); - } - } - } - - private byte[] waitForCheckpoint(long checkpointId, DataStatisticsCoordinator coordinator) - throws InterruptedException, ExecutionException { - CompletableFuture future = new CompletableFuture<>(); - coordinator.checkpointCoordinator(checkpointId, future); - return future.get(); - } - - private static DataStatisticsCoordinatorProvider createProvider( - StatisticsType type, int downstreamParallelism) { - return new DataStatisticsCoordinatorProvider( - "DataStatisticsCoordinatorProvider", - OPERATOR_ID, - Fixtures.SCHEMA, - Fixtures.SORT_ORDER, - downstreamParallelism, - type, - 0.0); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java deleted file mode 100644 index c760f1ba96d3..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsOperator.java +++ /dev/null @@ -1,350 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.ArgumentMatchers.any; -import static org.mockito.Mockito.verify; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.flink.api.common.ExecutionConfig; -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.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.runtime.execution.Environment; -import org.apache.flink.runtime.operators.coordination.MockOperatorEventGateway; -import org.apache.flink.runtime.operators.testutils.MockInputSplitProvider; -import org.apache.flink.runtime.state.AbstractStateBackend; -import org.apache.flink.runtime.state.StateInitializationContext; -import org.apache.flink.runtime.state.StateInitializationContextImpl; -import org.apache.flink.runtime.state.TestTaskStateManager; -import org.apache.flink.runtime.state.hashmap.HashMapStateBackend; -import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; -import org.apache.flink.streaming.runtime.tasks.OneInputStreamTask; -import org.apache.flink.streaming.runtime.tasks.StreamMockEnvironment; -import org.apache.flink.streaming.util.MockOutput; -import org.apache.flink.streaming.util.MockStreamConfig; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.SortKey; -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.junit.jupiter.api.BeforeEach; -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.EnumSource; -import org.junit.jupiter.params.provider.MethodSource; -import org.mockito.Mockito; - -public class TestDataStatisticsOperator { - - private Environment env; - - @BeforeEach - public void before() throws Exception { - this.env = - new StreamMockEnvironment( - new Configuration(), - new Configuration(), - new ExecutionConfig(), - 1L, - new MockInputSplitProvider(), - 1, - new TestTaskStateManager()); - } - - private DataStatisticsOperator createOperator(StatisticsType type, int downstreamParallelism) - throws Exception { - MockOperatorEventGateway mockGateway = new MockOperatorEventGateway(); - return createOperator(type, downstreamParallelism, mockGateway); - } - - private DataStatisticsOperator createOperator( - StatisticsType type, int downstreamParallelism, MockOperatorEventGateway mockGateway) - throws Exception { - DataStatisticsOperator operator = - new DataStatisticsOperator( - "testOperator", - Fixtures.SCHEMA, - Fixtures.SORT_ORDER, - mockGateway, - downstreamParallelism, - type); - operator.setup( - new OneInputStreamTask(env), - new MockStreamConfig(new Configuration(), 1), - new MockOutput<>(Lists.newArrayList())); - return operator; - } - - @SuppressWarnings("unchecked") - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testProcessElement(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(StringData.fromString("a"), 5))); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 3))); - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - - DataStatistics localStatistics = operator.localStatistics(); - assertThat(localStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - Map keyFrequency = (Map) localStatistics.result(); - assertThat(keyFrequency) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L)); - } else { - ReservoirItemsSketch sketch = - (ReservoirItemsSketch) localStatistics.result(); - assertThat(sketch.getSamples()) - .containsExactly(CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")); - } - - testHarness.endInput(); - } - } - - @ParameterizedTest - @EnumSource(StatisticsType.class) - public void testOperatorOutput(StatisticsType type) throws Exception { - DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); - try (OneInputStreamOperatorTestHarness testHarness = - createHarness(operator)) { - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 2))); - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 3))); - testHarness.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("b"), 1))); - - List recordsOutput = - testHarness.extractOutputValues().stream() - .filter(StatisticsOrRecord::hasRecord) - .map(StatisticsOrRecord::record) - .collect(Collectors.toList()); - assertThat(recordsOutput) - .containsExactlyInAnyOrderElementsOf( - ImmutableList.of( - GenericRowData.of(StringData.fromString("a"), 2), - GenericRowData.of(StringData.fromString("b"), 3), - GenericRowData.of(StringData.fromString("b"), 1))); - } - } - - private static Stream provideRestoreStateParameters() { - return Stream.of( - Arguments.of(StatisticsType.Map, -1), - Arguments.of(StatisticsType.Map, 0), - Arguments.of(StatisticsType.Map, 1), - Arguments.of(StatisticsType.Sketch, -1), - Arguments.of(StatisticsType.Sketch, 0), - Arguments.of(StatisticsType.Sketch, 1)); - } - - @ParameterizedTest - @MethodSource("provideRestoreStateParameters") - public void testRestoreState(StatisticsType type, int parallelismAdjustment) throws Exception { - Map keyFrequency = - ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 1L, CHAR_KEYS.get("c"), 1L); - SortKey[] rangeBounds = new SortKey[] {CHAR_KEYS.get("a")}; - MapAssignment mapAssignment = - MapAssignment.fromKeyFrequency(2, keyFrequency, 0.0d, SORT_ORDER_COMPARTOR); - DataStatisticsOperator operator = createOperator(type, Fixtures.NUM_SUBTASKS); - OperatorSubtaskState snapshot; - try (OneInputStreamOperatorTestHarness testHarness1 = - createHarness(operator)) { - GlobalStatistics statistics; - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - statistics = GlobalStatistics.fromMapAssignment(1L, mapAssignment); - } else { - statistics = GlobalStatistics.fromRangeBounds(1L, rangeBounds); - } - - StatisticsEvent event = - StatisticsEvent.createGlobalStatisticsEvent( - statistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false); - operator.handleOperatorEvent(event); - - GlobalStatistics globalStatistics = operator.globalStatistics(); - assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); - assertThat(globalStatistics.rangeBounds()).isNull(); - } else { - assertThat(globalStatistics.mapAssignment()).isNull(); - assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); - } - - snapshot = testHarness1.snapshot(1L, 0); - } - - // Use the snapshot to initialize state for another new operator and then verify that the global - // statistics for the new operator is same as before - MockOperatorEventGateway spyGateway = Mockito.spy(new MockOperatorEventGateway()); - DataStatisticsOperator restoredOperator = - createOperator(type, Fixtures.NUM_SUBTASKS + parallelismAdjustment, spyGateway); - try (OneInputStreamOperatorTestHarness testHarness2 = - new OneInputStreamOperatorTestHarness<>(restoredOperator, 2, 2, 1)) { - testHarness2.setup(); - testHarness2.initializeState(snapshot); - - GlobalStatistics globalStatistics = restoredOperator.globalStatistics(); - // global statistics is always restored and used initially even if - // downstream parallelism changed. - assertThat(globalStatistics).isNotNull(); - // request is always sent to coordinator during initialization. - // coordinator would respond with a new global statistics that - // has range bound recomputed with new parallelism. - verify(spyGateway).sendEventToCoordinator(any(RequestGlobalStatisticsEvent.class)); - assertThat(globalStatistics.type()).isEqualTo(StatisticsUtil.collectType(type)); - if (StatisticsUtil.collectType(type) == StatisticsType.Map) { - assertThat(globalStatistics.mapAssignment()).isEqualTo(mapAssignment); - assertThat(globalStatistics.rangeBounds()).isNull(); - } else { - assertThat(globalStatistics.mapAssignment()).isNull(); - assertThat(globalStatistics.rangeBounds()).isEqualTo(rangeBounds); - } - } - } - - @SuppressWarnings("unchecked") - @Test - public void testMigrationWithLocalStatsOverThreshold() throws Exception { - DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); - try (OneInputStreamOperatorTestHarness testHarness = - createHarness(operator)) { - StateInitializationContext stateContext = getStateContext(); - operator.initializeState(stateContext); - - // add rows with unique keys - for (int i = 0; i < SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD; ++i) { - operator.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); - assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); - assertThat((Map) operator.localStatistics().result()).hasSize(i + 1); - } - - // one more item should trigger the migration to sketch stats - operator.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString("key-trigger-migration"), 1))); - - int reservoirSize = - SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); - - assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); - ReservoirItemsSketch sketch = - (ReservoirItemsSketch) operator.localStatistics().result(); - assertThat(sketch.getK()).isEqualTo(reservoirSize); - assertThat(sketch.getN()).isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1); - // reservoir not full yet - assertThat(sketch.getN()).isLessThan(reservoirSize); - assertThat(sketch.getSamples()).hasSize((int) sketch.getN()); - - // add more items to saturate the reservoir - for (int i = 0; i < reservoirSize; ++i) { - operator.processElement( - new StreamRecord<>(GenericRowData.of(StringData.fromString(String.valueOf(i)), i))); - } - - assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); - sketch = (ReservoirItemsSketch) operator.localStatistics().result(); - assertThat(sketch.getK()).isEqualTo(reservoirSize); - assertThat(sketch.getN()) - .isEqualTo(SketchUtil.OPERATOR_SKETCH_SWITCH_THRESHOLD + 1 + reservoirSize); - // reservoir is full now - assertThat(sketch.getN()).isGreaterThan(reservoirSize); - assertThat(sketch.getSamples()).hasSize(reservoirSize); - - testHarness.endInput(); - } - } - - @SuppressWarnings("unchecked") - @Test - public void testMigrationWithGlobalSketchStatistics() throws Exception { - DataStatisticsOperator operator = createOperator(StatisticsType.Auto, Fixtures.NUM_SUBTASKS); - try (OneInputStreamOperatorTestHarness testHarness = - createHarness(operator)) { - StateInitializationContext stateContext = getStateContext(); - operator.initializeState(stateContext); - - // started with Map stype - operator.processElement(new StreamRecord<>(GenericRowData.of(StringData.fromString("a"), 1))); - assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Map); - assertThat((Map) operator.localStatistics().result()) - .isEqualTo(ImmutableMap.of(CHAR_KEYS.get("a"), 1L)); - - // received global statistics with sketch type - GlobalStatistics globalStatistics = - GlobalStatistics.fromRangeBounds( - 1L, new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("f")}); - operator.handleOperatorEvent( - StatisticsEvent.createGlobalStatisticsEvent( - globalStatistics, Fixtures.GLOBAL_STATISTICS_SERIALIZER, false)); - - int reservoirSize = - SketchUtil.determineOperatorReservoirSize(Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS); - - assertThat(operator.localStatistics().type()).isEqualTo(StatisticsType.Sketch); - ReservoirItemsSketch sketch = - (ReservoirItemsSketch) operator.localStatistics().result(); - assertThat(sketch.getK()).isEqualTo(reservoirSize); - assertThat(sketch.getN()).isEqualTo(1); - assertThat(sketch.getSamples()).isEqualTo(new SortKey[] {CHAR_KEYS.get("a")}); - - testHarness.endInput(); - } - } - - private StateInitializationContext getStateContext() throws Exception { - AbstractStateBackend abstractStateBackend = new HashMapStateBackend(); - CloseableRegistry cancelStreamRegistry = new CloseableRegistry(); - OperatorStateStore operatorStateStore = - abstractStateBackend.createOperatorStateBackend( - env, "test-operator", Collections.emptyList(), cancelStreamRegistry); - return new StateInitializationContextImpl(null, operatorStateStore, null, null, null); - } - - private OneInputStreamOperatorTestHarness createHarness( - DataStatisticsOperator dataStatisticsOperator) throws Exception { - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>( - dataStatisticsOperator, Fixtures.NUM_SUBTASKS, Fixtures.NUM_SUBTASKS, 0); - harness.setup( - new StatisticsOrRecordSerializer( - Fixtures.GLOBAL_STATISTICS_SERIALIZER, Fixtures.ROW_SERIALIZER)); - harness.open(); - return harness; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java deleted file mode 100644 index 59ce6df05d9d..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestDataStatisticsSerializer.java +++ /dev/null @@ -1,53 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; - -import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.api.common.typeutils.TypeSerializer; - -public class TestDataStatisticsSerializer extends SerializerTestBase { - @Override - protected TypeSerializer createSerializer() { - return Fixtures.TASK_STATISTICS_SERIALIZER; - } - - @Override - protected int getLength() { - return -1; - } - - @Override - protected Class getTypeClass() { - return DataStatistics.class; - } - - @Override - protected DataStatistics[] getTestData() { - return new DataStatistics[] { - new MapDataStatistics(), - Fixtures.createTaskStatistics( - StatisticsType.Map, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")), - new SketchDataStatistics(128), - Fixtures.createTaskStatistics( - StatisticsType.Sketch, CHAR_KEYS.get("a"), CHAR_KEYS.get("a"), CHAR_KEYS.get("b")) - }; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java deleted file mode 100644 index 7afaf239c668..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestGlobalStatisticsSerializer.java +++ /dev/null @@ -1,59 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; - -import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; - -public class TestGlobalStatisticsSerializer extends SerializerTestBase { - - @Override - protected TypeSerializer createSerializer() { - return Fixtures.GLOBAL_STATISTICS_SERIALIZER; - } - - @Override - protected int getLength() { - return -1; - } - - @Override - protected Class getTypeClass() { - return GlobalStatistics.class; - } - - @Override - protected GlobalStatistics[] getTestData() { - return new GlobalStatistics[] { - GlobalStatistics.fromMapAssignment( - 1L, - MapAssignment.fromKeyFrequency( - Fixtures.NUM_SUBTASKS, - ImmutableMap.of(CHAR_KEYS.get("a"), 1L, CHAR_KEYS.get("b"), 2L), - 0.0d, - SORT_ORDER_COMPARTOR)), - GlobalStatistics.fromRangeBounds(2L, new SortKey[] {CHAR_KEYS.get("a"), CHAR_KEYS.get("b")}) - }; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java deleted file mode 100644 index 8a25c7ad9898..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapDataStatistics.java +++ /dev/null @@ -1,67 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Map; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.Test; - -public class TestMapDataStatistics { - @SuppressWarnings("unchecked") - @Test - public void testAddsAndGet() { - MapDataStatistics dataStatistics = new MapDataStatistics(); - - GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("b")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("c")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("b")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("a")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("b")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - Map actual = (Map) dataStatistics.result(); - Map expected = - ImmutableMap.of(CHAR_KEYS.get("a"), 2L, CHAR_KEYS.get("b"), 3L, CHAR_KEYS.get("c"), 1L); - assertThat(actual).isEqualTo(expected); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java deleted file mode 100644 index d5a0bebc74e7..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ /dev/null @@ -1,434 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; -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.relocated.com.google.common.collect.Sets; -import org.apache.iceberg.util.Pair; -import org.junit.jupiter.api.Test; - -public class TestMapRangePartitioner { - private static final SortOrder SORT_ORDER = - SortOrder.builderFor(TestFixtures.SCHEMA).asc("data").build(); - - private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); - private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final SortKey[] SORT_KEYS = initSortKeys(); - - private static SortKey[] initSortKeys() { - SortKey[] sortKeys = new SortKey[10]; - for (int i = 0; i < 10; ++i) { - RowData rowData = - GenericRowData.of(StringData.fromString("k" + i), i, StringData.fromString("2023-06-20")); - RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); - keyWrapper.wrap(rowData); - SortKey sortKey = SORT_KEY.copy(); - sortKey.wrap(keyWrapper); - sortKeys[i] = sortKey; - } - return sortKeys; - } - - // Total weight is 800 - private final Map mapStatistics = - ImmutableMap.of( - SORT_KEYS[0], - 350L, - SORT_KEYS[1], - 230L, - SORT_KEYS[2], - 120L, - SORT_KEYS[3], - 40L, - SORT_KEYS[4], - 10L, - SORT_KEYS[5], - 10L, - SORT_KEYS[6], - 10L, - SORT_KEYS[7], - 10L, - SORT_KEYS[8], - 10L, - SORT_KEYS[9], - 10L); - - @Test - public void testEvenlyDividableNoClosingFileCost() { - int numPartitions = 8; - MapAssignment mapAssignment = - MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); - - // each task should get targeted weight of 100 (=800/8) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(100L, 100L, 100L, 50L), 0L), - SORT_KEYS[1], - new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(50L, 100L, 80L), 0L), - SORT_KEYS[2], - new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(20L, 100L), 0L), - SORT_KEYS[3], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(40L), 0L), - SORT_KEYS[4], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[5], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[6], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[7], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[8], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L), - SORT_KEYS[9], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(10L), 0L)); - assertThat(mapAssignment).isEqualTo(new MapAssignment(numPartitions, expectedAssignment)); - - // key: subtask id - // value pair: first is the assigned weight, second is the number of assigned keys - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(100L, 1), - 1, - Pair.of(100L, 1), - 2, - Pair.of(100L, 1), - 3, - Pair.of(100L, 2), - 4, - Pair.of(100L, 1), - 5, - Pair.of(100L, 2), - 6, - Pair.of(100L, 1), - 7, - Pair.of(100L, 7)); - assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); - - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); - Map>> partitionResults = - runPartitioner(partitioner, numPartitions, mapStatistics); - validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); - } - - @Test - public void testEvenlyDividableWithClosingFileCost() { - int numPartitions = 8; - MapAssignment mapAssignment = - MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); - - // target subtask weight is 100 before close file cost factored in. - // close file cost is 5 = 5% * 100. - // key weights before and after close file cost factored in - // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 - // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 - // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 - // target subtask weight with close cost per subtask is 110 (880/8) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(110L, 110L, 110L, 40L), 5L), - SORT_KEYS[1], - new KeyAssignment(ImmutableList.of(3, 4, 5), ImmutableList.of(70L, 110L, 65L), 5L), - SORT_KEYS[2], - new KeyAssignment(ImmutableList.of(5, 6), ImmutableList.of(45L, 85L), 5L), - SORT_KEYS[3], - new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(25L, 20L), 5L), - SORT_KEYS[4], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[5], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[6], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[7], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[8], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L), - SORT_KEYS[9], - new KeyAssignment(ImmutableList.of(7), ImmutableList.of(15L), 5L)); - assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight (excluding close file cost) for the subtask, - // second is the number of keys assigned to the subtask - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(105L, 1), - 1, - Pair.of(105L, 1), - 2, - Pair.of(105L, 1), - 3, - Pair.of(100L, 2), - 4, - Pair.of(105L, 1), - 5, - Pair.of(100L, 2), - 6, - Pair.of(100L, 2), - 7, - Pair.of(75L, 7)); - assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); - - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); - Map>> partitionResults = - runPartitioner(partitioner, numPartitions, mapStatistics); - validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); - } - - @Test - public void testNonDividableNoClosingFileCost() { - int numPartitions = 9; - MapAssignment mapAssignment = - MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); - - // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 - // each task should get targeted weight of 89 = ceiling(800/9) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(89L, 89L, 89L, 83L), 0L), - SORT_KEYS[1], - new KeyAssignment( - ImmutableList.of(3, 4, 5, 6), ImmutableList.of(6L, 89L, 89L, 46L), 0L), - SORT_KEYS[2], - new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(43L, 77L), 0L), - SORT_KEYS[3], - new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(12L, 28L), 0L), - SORT_KEYS[4], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[5], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[6], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[7], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[8], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L), - SORT_KEYS[9], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(10L), 0L)); - assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight, second is the number of assigned keys - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(89L, 1), - 1, - Pair.of(89L, 1), - 2, - Pair.of(89L, 1), - 3, - Pair.of(89L, 2), - 4, - Pair.of(89L, 1), - 5, - Pair.of(89L, 1), - 6, - Pair.of(89L, 2), - 7, - Pair.of(89L, 2), - 8, - Pair.of(88L, 7)); - assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); - - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); - Map>> partitionResults = - runPartitioner(partitioner, numPartitions, mapStatistics); - validatePartitionResults(expectedAssignmentInfo, partitionResults, 5.0); - } - - @Test - public void testNonDividableWithClosingFileCost() { - int numPartitions = 9; - MapAssignment mapAssignment = - MapAssignment.fromKeyFrequency(numPartitions, mapStatistics, 5.0, SORT_ORDER_COMPARTOR); - - // target subtask weight is 89 before close file cost factored in. - // close file cost is 5 (= 5% * 89) per file. - // key weights before and after close file cost factored in - // before: 350, 230, 120, 40, 10, 10, 10, 10, 10, 10 - // close-cost: 20, 15, 10, 5, 5, 5, 5, 5, 5, 5 - // after: 370, 245, 130, 45, 15, 15, 15, 15, 15, 15 - // target subtask weight per subtask is 98 ceiling(880/9) - Map expectedAssignment = - ImmutableMap.of( - SORT_KEYS[0], - new KeyAssignment( - ImmutableList.of(0, 1, 2, 3), ImmutableList.of(98L, 98L, 98L, 76L), 5L), - SORT_KEYS[1], - new KeyAssignment( - ImmutableList.of(3, 4, 5, 6), ImmutableList.of(22L, 98L, 98L, 27L), 5L), - SORT_KEYS[2], - new KeyAssignment(ImmutableList.of(6, 7), ImmutableList.of(71L, 59L), 5L), - SORT_KEYS[3], - new KeyAssignment(ImmutableList.of(7, 8), ImmutableList.of(39L, 6L), 5L), - SORT_KEYS[4], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[5], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[6], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[7], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[8], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L), - SORT_KEYS[9], - new KeyAssignment(ImmutableList.of(8), ImmutableList.of(15L), 5L)); - assertThat(mapAssignment.keyAssignments()).isEqualTo(expectedAssignment); - - // key: subtask id - // value pair: first is the assigned weight for the subtask, second is the number of keys - // assigned to the subtask - Map> expectedAssignmentInfo = - ImmutableMap.of( - 0, - Pair.of(93L, 1), - 1, - Pair.of(93L, 1), - 2, - Pair.of(93L, 1), - 3, - Pair.of(88L, 2), - 4, - Pair.of(93L, 1), - 5, - Pair.of(93L, 1), - 6, - Pair.of(88L, 2), - 7, - Pair.of(88L, 2), - 8, - Pair.of(61L, 7)); - assertThat(mapAssignment.assignmentInfo()).isEqualTo(expectedAssignmentInfo); - - MapRangePartitioner partitioner = - new MapRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, mapAssignment); - Map>> partitionResults = - runPartitioner(partitioner, numPartitions, mapStatistics); - // drift threshold is high for non-dividable scenario with close cost - validatePartitionResults(expectedAssignmentInfo, partitionResults, 10.0); - } - - private static Map>> runPartitioner( - MapRangePartitioner partitioner, int numPartitions, Map mapStatistics) { - // The Map key is the subtaskId. - // For the map value pair, the first element is the count of assigned and - // the second element of Set is for the set of assigned keys. - Map>> partitionResults = Maps.newHashMap(); - mapStatistics.forEach( - (sortKey, weight) -> { - String key = sortKey.get(0, String.class); - // run 100x times of the weight - long iterations = weight * 100; - for (int i = 0; i < iterations; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString(key), 1, StringData.fromString("2023-06-20")); - int subtaskId = partitioner.partition(rowData, numPartitions); - partitionResults.computeIfAbsent( - subtaskId, k -> Pair.of(new AtomicLong(0), Sets.newHashSet())); - Pair> pair = partitionResults.get(subtaskId); - pair.first().incrementAndGet(); - pair.second().add(rowData); - } - }); - return partitionResults; - } - - /** @param expectedAssignmentInfo excluding closing cost */ - private void validatePartitionResults( - Map> expectedAssignmentInfo, - Map>> partitionResults, - double maxDriftPercentage) { - - assertThat(partitionResults.size()).isEqualTo(expectedAssignmentInfo.size()); - - List expectedAssignedKeyCounts = - Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); - List actualAssignedKeyCounts = - Lists.newArrayListWithExpectedSize(partitionResults.size()); - List expectedNormalizedWeights = - Lists.newArrayListWithExpectedSize(expectedAssignmentInfo.size()); - List actualNormalizedWeights = - Lists.newArrayListWithExpectedSize(partitionResults.size()); - - long expectedTotalWeight = - expectedAssignmentInfo.values().stream().mapToLong(Pair::first).sum(); - expectedAssignmentInfo.forEach( - (subtaskId, pair) -> { - expectedAssignedKeyCounts.add(pair.second()); - expectedNormalizedWeights.add(pair.first().doubleValue() / expectedTotalWeight); - }); - - long actualTotalWeight = - partitionResults.values().stream().mapToLong(pair -> pair.first().longValue()).sum(); - partitionResults.forEach( - (subtaskId, pair) -> { - actualAssignedKeyCounts.add(pair.second().size()); - actualNormalizedWeights.add(pair.first().doubleValue() / actualTotalWeight); - }); - - // number of assigned keys should match exactly - assertThat(actualAssignedKeyCounts) - .as("the number of assigned keys should match for every subtask") - .isEqualTo(expectedAssignedKeyCounts); - - // weight for every subtask shouldn't differ for more than some threshold relative to the - // expected weight - for (int subtaskId = 0; subtaskId < expectedNormalizedWeights.size(); ++subtaskId) { - double expectedWeight = expectedNormalizedWeights.get(subtaskId); - double min = expectedWeight * (1 - maxDriftPercentage / 100); - double max = expectedWeight * (1 + maxDriftPercentage / 100); - assertThat(actualNormalizedWeights.get(subtaskId)) - .as( - "Subtask %d weight should within %.1f percent of the expected range %s", - subtaskId, maxDriftPercentage, expectedWeight) - .isBetween(min, max); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java deleted file mode 100644 index 0485fdb7fa04..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestRangePartitioner.java +++ /dev/null @@ -1,65 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.Set; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.relocated.com.google.common.collect.Sets; -import org.junit.jupiter.api.Test; - -public class TestRangePartitioner { - private final int numPartitions = 4; - - @Test - public void testRoundRobinRecordsBeforeStatisticsAvailable() { - RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); - Set results = Sets.newHashSetWithExpectedSize(numPartitions); - for (int i = 0; i < numPartitions; ++i) { - results.add( - partitioner.partition( - StatisticsOrRecord.fromRecord(GenericRowData.of(StringData.fromString("a"), 1)), - numPartitions)); - } - - // round-robin. every partition should get an assignment - assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); - } - - @Test - public void testRoundRobinStatisticsWrapper() { - RangePartitioner partitioner = new RangePartitioner(SCHEMA, SORT_ORDER); - Set results = Sets.newHashSetWithExpectedSize(numPartitions); - for (int i = 0; i < numPartitions; ++i) { - GlobalStatistics statistics = - GlobalStatistics.fromRangeBounds(1L, new SortKey[] {CHAR_KEYS.get("a")}); - results.add( - partitioner.partition(StatisticsOrRecord.fromStatistics(statistics), numPartitions)); - } - - // round-robin. every partition should get an assignment - assertThat(results).containsExactlyInAnyOrder(0, 1, 2, 3); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java deleted file mode 100644 index 396bfae2f13c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchDataStatistics.java +++ /dev/null @@ -1,60 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_WRAPPER; -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.datasketches.sampling.ReservoirItemsSketch; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.SortKey; -import org.junit.jupiter.api.Test; - -public class TestSketchDataStatistics { - @SuppressWarnings("unchecked") - @Test - public void testAddsAndGet() { - SketchDataStatistics dataStatistics = new SketchDataStatistics(128); - - GenericRowData reusedRow = GenericRowData.of(StringData.fromString("a"), 1); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("b")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("c")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - reusedRow.setField(0, StringData.fromString("b")); - Fixtures.SORT_KEY.wrap(ROW_WRAPPER.wrap(reusedRow)); - dataStatistics.add(Fixtures.SORT_KEY); - - ReservoirItemsSketch actual = (ReservoirItemsSketch) dataStatistics.result(); - assertThat(actual.getSamples()) - .isEqualTo( - new SortKey[] { - CHAR_KEYS.get("a"), CHAR_KEYS.get("b"), CHAR_KEYS.get("c"), CHAR_KEYS.get("b") - }); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java deleted file mode 100644 index 378c6afff077..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchRangePartitioner.java +++ /dev/null @@ -1,88 +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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TestFixtures; -import org.junit.jupiter.api.Test; - -public class TestSketchRangePartitioner { - // sort on the long id field - private static final SortOrder SORT_ORDER = - SortOrder.builderFor(TestFixtures.SCHEMA).asc("id").build(); - private static final SortKey SORT_KEY = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); - private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final int NUM_PARTITIONS = 16; - private static final long RANGE_STEP = 1_000; - private static final long MAX_ID = RANGE_STEP * NUM_PARTITIONS; - private static final SortKey[] RANGE_BOUNDS = createRangeBounds(); - - /** - * To understand how range bounds are used in range partitioning, here is an example for human - * ages with 4 partitions: [15, 32, 60]. The 4 ranges would be - * - *
      - *
    • age <= 15 - *
    • age > 15 && age <= 32 - *
    • age >32 && age <= 60 - *
    • age > 60 - *
    - */ - private static SortKey[] createRangeBounds() { - SortKey[] rangeBounds = new SortKey[NUM_PARTITIONS - 1]; - for (int i = 0; i < NUM_PARTITIONS - 1; ++i) { - RowData rowData = - GenericRowData.of( - StringData.fromString("data"), - RANGE_STEP * (i + 1), - StringData.fromString("2023-06-20")); - RowDataWrapper keyWrapper = new RowDataWrapper(ROW_TYPE, TestFixtures.SCHEMA.asStruct()); - keyWrapper.wrap(rowData); - SortKey sortKey = new SortKey(TestFixtures.SCHEMA, SORT_ORDER); - sortKey.wrap(keyWrapper); - rangeBounds[i] = sortKey; - } - - return rangeBounds; - } - - @Test - public void testRangePartitioningWithRangeBounds() { - SketchRangePartitioner partitioner = - new SketchRangePartitioner(TestFixtures.SCHEMA, SORT_ORDER, RANGE_BOUNDS); - GenericRowData row = - GenericRowData.of(StringData.fromString("data"), 0L, StringData.fromString("2023-06-20")); - for (long id = 0; id < MAX_ID; ++id) { - row.setField(1, id); - int partition = partitioner.partition(row, NUM_PARTITIONS); - assertThat(partition).isGreaterThanOrEqualTo(0).isLessThan(NUM_PARTITIONS); - int expectedPartition = id == 0L ? 0 : (int) ((id - 1) / RANGE_STEP); - assertThat(partition).isEqualTo(expectedPartition); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java deleted file mode 100644 index 16202c075ea0..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSketchUtil.java +++ /dev/null @@ -1,189 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.CHAR_KEYS; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER_COMPARTOR; -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.iceberg.SortKey; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -public class TestSketchUtil { - @Test - public void testCoordinatorReservoirSize() { - // adjusted to over min threshold of 10_000 and is divisible by number of partitions (3) - assertThat(SketchUtil.determineCoordinatorReservoirSize(3)).isEqualTo(10_002); - // adjust to multiplier of 100 - assertThat(SketchUtil.determineCoordinatorReservoirSize(123)).isEqualTo(123_00); - // adjusted to below max threshold of 1_000_000 and is divisible by number of partitions (3) - assertThat(SketchUtil.determineCoordinatorReservoirSize(10_123)) - .isEqualTo(1_000_000 - (1_000_000 % 10_123)); - } - - @Test - public void testOperatorReservoirSize() { - assertThat(SketchUtil.determineOperatorReservoirSize(5, 3)) - .isEqualTo((10_002 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5); - assertThat(SketchUtil.determineOperatorReservoirSize(123, 123)) - .isEqualTo((123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 123); - assertThat(SketchUtil.determineOperatorReservoirSize(256, 123)) - .isEqualTo( - (int) Math.ceil((double) (123_00 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 256)); - assertThat(SketchUtil.determineOperatorReservoirSize(5_120, 10_123)) - .isEqualTo( - (int) Math.ceil((double) (992_054 * SketchUtil.OPERATOR_OVER_SAMPLE_RATIO) / 5_120)); - } - - @Test - public void testRangeBoundsOneChannel() { - assertThat( - SketchUtil.rangeBounds( - 1, - SORT_ORDER_COMPARTOR, - new SortKey[] { - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d"), - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f") - })) - .isEmpty(); - } - - @Test - public void testRangeBoundsDivisible() { - assertThat( - SketchUtil.rangeBounds( - 3, - SORT_ORDER_COMPARTOR, - new SortKey[] { - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d"), - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f") - })) - .containsExactly(CHAR_KEYS.get("b"), CHAR_KEYS.get("d")); - } - - @Test - public void testRangeBoundsNonDivisible() { - // step is 3 = ceiling(11/4) - assertThat( - SketchUtil.rangeBounds( - 4, - SORT_ORDER_COMPARTOR, - new SortKey[] { - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("d"), - CHAR_KEYS.get("e"), - CHAR_KEYS.get("f"), - CHAR_KEYS.get("g"), - CHAR_KEYS.get("h"), - CHAR_KEYS.get("i"), - CHAR_KEYS.get("j"), - CHAR_KEYS.get("k"), - })) - .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("f"), CHAR_KEYS.get("i")); - } - - @Test - public void testRangeBoundsSkipDuplicates() { - // step is 3 = ceiling(11/4) - assertThat( - SketchUtil.rangeBounds( - 4, - SORT_ORDER_COMPARTOR, - new SortKey[] { - CHAR_KEYS.get("a"), - CHAR_KEYS.get("b"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("c"), - CHAR_KEYS.get("g"), - CHAR_KEYS.get("h"), - CHAR_KEYS.get("i"), - CHAR_KEYS.get("j"), - CHAR_KEYS.get("k"), - })) - // skipped duplicate c's - .containsExactly(CHAR_KEYS.get("c"), CHAR_KEYS.get("g"), CHAR_KEYS.get("j")); - } - - @ParameterizedTest - @ValueSource(ints = {4, 6}) - public void testPartitioningAndScaleUp(int numPartitions) { - // Range bounds are calculated based on 4 partitions - SortKey[] rangeBounds = - new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; - - // <= c - assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); - assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); - // > c && <= j - assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); - assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); - assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); - // > j && <= m - assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); - assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); - assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); - // > m - assertPartition(3, CHAR_KEYS.get("n"), numPartitions, rangeBounds); - assertPartition(3, CHAR_KEYS.get("z"), numPartitions, rangeBounds); - } - - @Test - public void testPartitionScaleDown() { - // Range bounds are calculated based on 4 partitions - SortKey[] rangeBounds = - new SortKey[] {CHAR_KEYS.get("c"), CHAR_KEYS.get("j"), CHAR_KEYS.get("m")}; - int numPartitions = 3; - - // <= c - assertPartition(0, CHAR_KEYS.get("a"), numPartitions, rangeBounds); - assertPartition(0, CHAR_KEYS.get("c"), numPartitions, rangeBounds); - // > c && <= j - assertPartition(1, CHAR_KEYS.get("d"), numPartitions, rangeBounds); - assertPartition(1, CHAR_KEYS.get("i"), numPartitions, rangeBounds); - assertPartition(1, CHAR_KEYS.get("j"), numPartitions, rangeBounds); - // > j && <= m - assertPartition(2, CHAR_KEYS.get("k"), numPartitions, rangeBounds); - assertPartition(2, CHAR_KEYS.get("l"), numPartitions, rangeBounds); - assertPartition(2, CHAR_KEYS.get("m"), numPartitions, rangeBounds); - // > m - // reassigns out-of-range partitions via mod (% 3 in this case) - assertPartition(0, CHAR_KEYS.get("n"), numPartitions, rangeBounds); - assertPartition(0, CHAR_KEYS.get("z"), numPartitions, rangeBounds); - } - - private static void assertPartition( - int expectedPartition, SortKey key, int numPartitions, SortKey[] rangeBounds) { - assertThat(SketchUtil.partition(key, numPartitions, rangeBounds, SORT_ORDER_COMPARTOR)) - .isEqualTo(expectedPartition); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java deleted file mode 100644 index c7fea015142c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerBase.java +++ /dev/null @@ -1,65 +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.sink.shuffle; - -import org.apache.flink.api.common.typeutils.SerializerTestBase; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; - -public abstract class TestSortKeySerializerBase extends SerializerTestBase { - - protected abstract Schema schema(); - - protected abstract SortOrder sortOrder(); - - protected abstract GenericRowData rowData(); - - @Override - protected TypeSerializer createSerializer() { - return new SortKeySerializer(schema(), sortOrder()); - } - - @Override - protected int getLength() { - return -1; - } - - @Override - protected Class getTypeClass() { - return SortKey.class; - } - - @Override - protected SortKey[] getTestData() { - return new SortKey[] {sortKey()}; - } - - private SortKey sortKey() { - RowDataWrapper rowDataWrapper = - new RowDataWrapper(FlinkSchemaUtil.convert(schema()), schema().asStruct()); - SortKey sortKey = new SortKey(schema(), sortOrder()); - sortKey.wrap(rowDataWrapper.wrap(rowData())); - return sortKey; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java deleted file mode 100644 index 0000688a8b55..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerNestedStruct.java +++ /dev/null @@ -1,55 +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.sink.shuffle; - -import org.apache.flink.table.data.GenericRowData; -import org.apache.iceberg.NullOrder; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortDirection; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; - -public class TestSortKeySerializerNestedStruct extends TestSortKeySerializerBase { - private final DataGenerator generator = new DataGenerators.StructOfStruct(); - - @Override - protected Schema schema() { - return generator.icebergSchema(); - } - - @Override - protected SortOrder sortOrder() { - return SortOrder.builderFor(schema()) - .asc("row_id") - .sortBy( - Expressions.bucket("struct_of_struct.id", 4), SortDirection.DESC, NullOrder.NULLS_LAST) - .sortBy( - Expressions.truncate("struct_of_struct.person_struct.name", 16), - SortDirection.ASC, - NullOrder.NULLS_FIRST) - .build(); - } - - @Override - protected GenericRowData rowData() { - return generator.generateFlinkRowData(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java deleted file mode 100644 index 54cceae6e55b..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerPrimitives.java +++ /dev/null @@ -1,90 +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.sink.shuffle; - -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; - -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.NullOrder; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortDirection; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.DataGenerator; -import org.apache.iceberg.flink.DataGenerators; -import org.apache.iceberg.flink.RowDataWrapper; -import org.junit.jupiter.api.Test; - -public class TestSortKeySerializerPrimitives extends TestSortKeySerializerBase { - private final DataGenerator generator = new DataGenerators.Primitives(); - - @Override - protected Schema schema() { - return generator.icebergSchema(); - } - - @Override - protected SortOrder sortOrder() { - return SortOrder.builderFor(schema()) - .asc("boolean_field") - .sortBy(Expressions.bucket("int_field", 4), SortDirection.DESC, NullOrder.NULLS_LAST) - .sortBy(Expressions.truncate("string_field", 2), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy(Expressions.bucket("uuid_field", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy(Expressions.hour("ts_with_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy(Expressions.day("ts_without_zone_field"), SortDirection.ASC, NullOrder.NULLS_FIRST) - // can not test HeapByteBuffer due to equality test inside SerializerTestBase - // .sortBy(Expressions.truncate("binary_field", 2), SortDirection.ASC, - // NullOrder.NULLS_FIRST) - .build(); - } - - @Override - protected GenericRowData rowData() { - return generator.generateFlinkRowData(); - } - - @Test - public void testSerializationSize() throws Exception { - RowData rowData = - GenericRowData.of(StringData.fromString("550e8400-e29b-41d4-a716-446655440000"), 1L); - RowDataWrapper rowDataWrapper = - new RowDataWrapper(Fixtures.ROW_TYPE, Fixtures.SCHEMA.asStruct()); - StructLike struct = rowDataWrapper.wrap(rowData); - SortKey sortKey = Fixtures.SORT_KEY.copy(); - sortKey.wrap(struct); - SortKeySerializer serializer = new SortKeySerializer(Fixtures.SCHEMA, Fixtures.SORT_ORDER); - DataOutputSerializer output = new DataOutputSerializer(1024); - serializer.serialize(sortKey, output); - 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); - - DataInputDeserializer input = new DataInputDeserializer(serializedBytes); - SortKey deserialized = serializer.deserialize(input); - assertThat(deserialized).isEqualTo(sortKey); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java deleted file mode 100644 index 012654603b04..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeySerializerSnapshot.java +++ /dev/null @@ -1,213 +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.sink.shuffle; - -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.ROW_TYPE; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SCHEMA; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_KEY; -import static org.apache.iceberg.flink.sink.shuffle.Fixtures.SORT_ORDER; -import static org.assertj.core.api.AssertionsForClassTypes.assertThat; - -import java.io.IOException; -import org.apache.flink.api.common.typeutils.TypeSerializer; -import org.apache.flink.api.common.typeutils.TypeSerializerSchemaCompatibility; -import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; -import org.apache.flink.core.memory.DataInputDeserializer; -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputSerializer; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.StringData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortKey; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.StructLike; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestSortKeySerializerSnapshot { - private final Schema schema = - new Schema( - Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.optional(2, "str", Types.StringType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get()), - Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); - private final SortOrder sortOrder = SortOrder.builderFor(schema).asc("str").asc("int").build(); - - @Test - public void testRestoredSerializer() 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); - TypeSerializerSnapshot snapshot = - roundTrip(originalSerializer.snapshotConfiguration()); - TypeSerializer restoredSerializer = snapshot.restoreSerializer(); - - 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); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - SortKeySerializer newSerializer = new SortKeySerializer(schema, sortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); - } - - @Test - public void testSnapshotIsCompatibleWithRemoveNonSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - // removed non-sort boolean field - Schema newSchema = - new Schema( - Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.optional(2, "str", Types.StringType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get())); - SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); - SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); - } - - @Test - public void testSnapshotIsCompatibleWithAddNonSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - // add a new non-sort float field - Schema newSchema = - new Schema( - Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.optional(2, "str", Types.StringType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get()), - Types.NestedField.optional(4, "boolean", Types.BooleanType.get()), - Types.NestedField.required(5, "float", Types.FloatType.get())); - SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); - SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isCompatibleAsIs()).isTrue(); - } - - @Test - public void testSnapshotIsIncompatibleWithIncompatibleSchema() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - // change str field to a long type - Schema newSchema = - new Schema( - Types.NestedField.optional(1, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.optional(2, "str", Types.LongType.get()), - Types.NestedField.optional(3, "int", Types.IntegerType.get()), - Types.NestedField.optional(4, "boolean", Types.BooleanType.get())); - SortOrder newSortOrder = SortOrder.builderFor(newSchema).asc("str").asc("int").build(); - // switch sort field order - SortKeySerializer newSerializer = new SortKeySerializer(newSchema, newSortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isIncompatible()).isTrue(); - } - - @Test - public void testSnapshotIsIncompatibleWithAddSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - // removed str field from sort order - SortOrder newSortOrder = - SortOrder.builderFor(schema).asc("str").asc("int").desc("boolean").build(); - SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isIncompatible()).isTrue(); - } - - @Test - public void testSnapshotIsIncompatibleWithRemoveSortField() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - // remove str field from sort order - SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").build(); - SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isIncompatible()).isTrue(); - } - - @Test - public void testSnapshotIsIncompatibleWithSortFieldsOrderChange() throws Exception { - SortKeySerializer oldSerializer = new SortKeySerializer(schema, sortOrder); - SortKeySerializer.SortKeySerializerSnapshot oldSnapshot = - roundTrip(oldSerializer.snapshotConfiguration()); - - // switch sort field order - SortOrder newSortOrder = SortOrder.builderFor(schema).asc("int").asc("str").build(); - SortKeySerializer newSerializer = new SortKeySerializer(schema, newSortOrder); - - TypeSerializerSchemaCompatibility resultCompatibility = - oldSnapshot.resolveSchemaCompatibility(newSerializer); - assertThat(resultCompatibility.isIncompatible()).isTrue(); - } - - /** Copied from Flink {@code AvroSerializerSnapshotTest} */ - private static SortKeySerializer.SortKeySerializerSnapshot roundTrip( - TypeSerializerSnapshot original) throws IOException { - // writeSnapshot(); - DataOutputSerializer out = new DataOutputSerializer(1024); - original.writeSnapshot(out); - // init - SortKeySerializer.SortKeySerializerSnapshot restored = - new SortKeySerializer.SortKeySerializerSnapshot(); - // readSnapshot(); - DataInputView in = new DataInputDeserializer(out.wrapAsByteBuffer()); - restored.readSnapshot(restored.getCurrentVersion(), in, original.getClass().getClassLoader()); - return restored; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.java deleted file mode 100644 index 1be7e27f2c01..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestSortKeyUtil.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.sink.shuffle; - -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.iceberg.NullOrder; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SortDirection; -import org.apache.iceberg.SortOrder; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.Test; - -public class TestSortKeyUtil { - @Test - public void testResultSchema() { - Schema schema = - new Schema( - Types.NestedField.required(1, "id", Types.StringType.get()), - Types.NestedField.required(2, "ratio", Types.DoubleType.get()), - Types.NestedField.optional( - 3, - "user", - Types.StructType.of( - Types.NestedField.required(11, "name", Types.StringType.get()), - Types.NestedField.required(12, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.optional(13, "device_id", Types.UUIDType.get()), - Types.NestedField.optional( - 14, - "location", - Types.StructType.of( - Types.NestedField.required(101, "lat", Types.FloatType.get()), - Types.NestedField.required(102, "long", Types.FloatType.get()), - Types.NestedField.required(103, "blob", Types.BinaryType.get())))))); - - SortOrder sortOrder = - SortOrder.builderFor(schema) - .asc("ratio") - .sortBy(Expressions.hour("user.ts"), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy( - Expressions.bucket("user.device_id", 16), SortDirection.ASC, NullOrder.NULLS_FIRST) - .sortBy( - Expressions.truncate("user.location.blob", 16), - SortDirection.ASC, - NullOrder.NULLS_FIRST) - .build(); - - assertThat(SortKeyUtil.sortKeySchema(schema, sortOrder).asStruct()) - .isEqualTo( - Types.StructType.of( - Types.NestedField.required(0, "ratio_0", Types.DoubleType.get()), - Types.NestedField.required(1, "ts_1", Types.IntegerType.get()), - Types.NestedField.optional(2, "device_id_2", Types.IntegerType.get()), - Types.NestedField.required(3, "blob_3", Types.BinaryType.get()))); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java deleted file mode 100644 index a08578a4c106..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTableFactory.java +++ /dev/null @@ -1,170 +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.source; - -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.function.Supplier; -import java.util.stream.Stream; -import org.apache.flink.api.java.typeutils.RowTypeInfo; -import org.apache.flink.configuration.ConfigOption; -import org.apache.flink.configuration.ConfigOptions; -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.streaming.api.functions.source.SourceFunction; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.connector.ChangelogMode; -import org.apache.flink.table.connector.ProviderContext; -import org.apache.flink.table.connector.source.DataStreamScanProvider; -import org.apache.flink.table.connector.source.DynamicTableSource; -import org.apache.flink.table.connector.source.ScanTableSource; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.data.util.DataFormatConverters; -import org.apache.flink.table.factories.DynamicTableSourceFactory; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.table.utils.TableSchemaUtils; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -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; - -public class BoundedTableFactory implements DynamicTableSourceFactory { - private static final AtomicInteger DATA_SET_ID = new AtomicInteger(0); - private static final Map>> DATA_SETS = Maps.newHashMap(); - - private static final ConfigOption DATA_ID = - ConfigOptions.key("data-id").stringType().noDefaultValue(); - - public static String registerDataSet(List> dataSet) { - String dataSetId = String.valueOf(DATA_SET_ID.incrementAndGet()); - DATA_SETS.put(dataSetId, dataSet); - return dataSetId; - } - - public static void clearDataSets() { - DATA_SETS.clear(); - } - - @Override - public DynamicTableSource createDynamicTableSource(Context context) { - TableSchema tableSchema = - TableSchemaUtils.getPhysicalSchema(context.getCatalogTable().getSchema()); - - Configuration configuration = Configuration.fromMap(context.getCatalogTable().getOptions()); - String dataId = configuration.getString(DATA_ID); - Preconditions.checkArgument( - DATA_SETS.containsKey(dataId), "data-id %s does not found in registered data set.", dataId); - - return new BoundedTableSource(DATA_SETS.get(dataId), tableSchema); - } - - @Override - public String factoryIdentifier() { - return "BoundedSource"; - } - - @Override - public Set> requiredOptions() { - return ImmutableSet.of(); - } - - @Override - public Set> optionalOptions() { - return ImmutableSet.of(DATA_ID); - } - - private static class BoundedTableSource implements ScanTableSource { - - private final List> elementsPerCheckpoint; - private final TableSchema tableSchema; - - private BoundedTableSource(List> elementsPerCheckpoint, TableSchema tableSchema) { - this.elementsPerCheckpoint = elementsPerCheckpoint; - this.tableSchema = tableSchema; - } - - private BoundedTableSource(BoundedTableSource toCopy) { - this.elementsPerCheckpoint = toCopy.elementsPerCheckpoint; - this.tableSchema = toCopy.tableSchema; - } - - @Override - public ChangelogMode getChangelogMode() { - Supplier> supplier = () -> elementsPerCheckpoint.stream().flatMap(List::stream); - - // Add the INSERT row kind by default. - ChangelogMode.Builder builder = ChangelogMode.newBuilder().addContainedKind(RowKind.INSERT); - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.DELETE)) { - builder.addContainedKind(RowKind.DELETE); - } - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_BEFORE)) { - builder.addContainedKind(RowKind.UPDATE_BEFORE); - } - - if (supplier.get().anyMatch(r -> r.getKind() == RowKind.UPDATE_AFTER)) { - builder.addContainedKind(RowKind.UPDATE_AFTER); - } - - return builder.build(); - } - - @Override - public ScanRuntimeProvider getScanRuntimeProvider(ScanContext runtimeProviderContext) { - return new DataStreamScanProvider() { - @Override - public DataStream produceDataStream( - ProviderContext providerContext, StreamExecutionEnvironment env) { - boolean checkpointEnabled = env.getCheckpointConfig().isCheckpointingEnabled(); - SourceFunction source = - new BoundedTestSource<>(elementsPerCheckpoint, checkpointEnabled); - - RowType rowType = (RowType) tableSchema.toRowDataType().getLogicalType(); - // Converter to convert the Row to RowData. - DataFormatConverters.RowConverter rowConverter = - new DataFormatConverters.RowConverter(tableSchema.getFieldDataTypes()); - - return env.addSource(source, new RowTypeInfo(tableSchema.getFieldTypes())) - .map(rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)); - } - - @Override - public boolean isBounded() { - return true; - } - }; - } - - @Override - public DynamicTableSource copy() { - return new BoundedTableSource(this); - } - - @Override - public String asSummaryString() { - return "Bounded test table source"; - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java deleted file mode 100644 index 7b435d059845..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/BoundedTestSource.java +++ /dev/null @@ -1,108 +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.source; - -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.state.CheckpointListener; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - -/** - * A stream source that: 1) emits the elements from elementsPerCheckpoint.get(0) without allowing - * checkpoints. 2) then waits for the checkpoint to complete. 3) emits the elements from - * elementsPerCheckpoint.get(1) without allowing checkpoints. 4) then waits for the checkpoint to - * complete. 5) ... - * - *

    Util all the list from elementsPerCheckpoint are exhausted. - */ -public final class BoundedTestSource implements SourceFunction, CheckpointListener { - - private final List> elementsPerCheckpoint; - private final boolean checkpointEnabled; - private volatile boolean running = true; - - private final AtomicInteger numCheckpointsComplete = new AtomicInteger(0); - - /** Emits all those elements in several checkpoints. */ - public BoundedTestSource(List> elementsPerCheckpoint, boolean checkpointEnabled) { - this.elementsPerCheckpoint = elementsPerCheckpoint; - this.checkpointEnabled = checkpointEnabled; - } - - public BoundedTestSource(List> elementsPerCheckpoint) { - this(elementsPerCheckpoint, true); - } - - /** Emits all those elements in a single checkpoint. */ - public BoundedTestSource(T... elements) { - this(Collections.singletonList(Arrays.asList(elements))); - } - - @Override - public void run(SourceContext ctx) throws Exception { - if (!checkpointEnabled) { - Preconditions.checkArgument( - elementsPerCheckpoint.size() <= 1, - "There should be at most one list in the elementsPerCheckpoint when checkpoint is disabled."); - elementsPerCheckpoint.stream().flatMap(List::stream).forEach(ctx::collect); - return; - } - - for (List elements : elementsPerCheckpoint) { - - final int checkpointToAwait; - synchronized (ctx.getCheckpointLock()) { - // Let's say checkpointToAwait = numCheckpointsComplete.get() + delta, in fact the value of - // delta should not - // affect the final table records because we only need to make sure that there will be - // exactly - // elementsPerCheckpoint.size() checkpoints to emit each records buffer from the original - // elementsPerCheckpoint. - // Even if the checkpoints that emitted results are not continuous, the correctness of the - // data should not be - // affected in the end. Setting the delta to be 2 is introducing the variable that produce - // un-continuous - // checkpoints that emit the records buffer from elementsPerCheckpoints. - checkpointToAwait = numCheckpointsComplete.get() + 2; - for (T element : elements) { - ctx.collect(element); - } - } - - synchronized (ctx.getCheckpointLock()) { - while (running && numCheckpointsComplete.get() < checkpointToAwait) { - ctx.getCheckpointLock().wait(1); - } - } - } - } - - @Override - public void notifyCheckpointComplete(long checkpointId) throws Exception { - numCheckpointsComplete.incrementAndGet(); - } - - @Override - public void cancel() { - running = false; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java deleted file mode 100644 index 5dfbbe3abe73..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/ChangeLogTableTestBase.java +++ /dev/null @@ -1,95 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.types.Row; -import org.apache.flink.types.RowKind; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.TestBase; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestInfo; - -public class ChangeLogTableTestBase extends TestBase { - private volatile TableEnvironment tEnv = null; - - protected String tableName; - - @BeforeEach - public void setup(TestInfo testInfo) { - assertThat(testInfo.getTestMethod()).isPresent(); - this.tableName = testInfo.getTestMethod().get().getName(); - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s", tableName); - BoundedTableFactory.clearDataSets(); - } - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings settings = - EnvironmentSettings.newInstance().inStreamingMode().build(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(400) - .setMaxParallelism(1) - .setParallelism(1); - - tEnv = StreamTableEnvironment.create(env, settings); - } - } - } - return tEnv; - } - - protected static Row insertRow(Object... values) { - return Row.ofKind(RowKind.INSERT, values); - } - - protected static Row deleteRow(Object... values) { - return Row.ofKind(RowKind.DELETE, values); - } - - protected static Row updateBeforeRow(Object... values) { - return Row.ofKind(RowKind.UPDATE_BEFORE, values); - } - - protected static Row updateAfterRow(Object... values) { - return Row.ofKind(RowKind.UPDATE_AFTER, values); - } - - protected static List listJoin(List> lists) { - return lists.stream().flatMap(List::stream).collect(Collectors.toList()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java deleted file mode 100644 index 540902f3cea5..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SplitHelpers.java +++ /dev/null @@ -1,200 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.mockito.Mockito.doReturn; -import static org.mockito.Mockito.spy; - -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.hadoop.conf.Configuration; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.BaseFileScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileMetadata; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.ResidualEvaluator; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.hadoop.HadoopCatalog; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.util.ThreadPools; - -public class SplitHelpers { - - private SplitHelpers() {} - - /** - * This create a list of IcebergSourceSplit from real files - *

  • Create a new Hadoop table under the {@code temporaryFolder} - *
  • write {@code fileCount} number of files to the new Iceberg table - *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} - * limit - *
  • Delete the Hadoop table - * - *

    Since the table and data files are deleted before this method return, caller shouldn't - * attempt to read the data files. - * - *

    By default, v1 Iceberg table is created. For v2 table use {@link - * SplitHelpers#createSplitsFromTransientHadoopTable(Path, int, int, String)} - * - * @param temporaryFolder Folder to place the data to - * @param fileCount The number of files to create and add to the table - * @param filesPerSplit The number of files used for a split - */ - public static List createSplitsFromTransientHadoopTable( - Path temporaryFolder, int fileCount, int filesPerSplit) throws Exception { - return createSplitsFromTransientHadoopTable(temporaryFolder, fileCount, filesPerSplit, "1"); - } - - /** - * This create a list of IcebergSourceSplit from real files - *

  • Create a new Hadoop table under the {@code temporaryFolder} - *
  • write {@code fileCount} number of files to the new Iceberg table - *
  • Discover the splits from the table and partition the splits by the {@code filePerSplit} - * limit - *
  • Delete the Hadoop table - * - *

    Since the table and data files are deleted before this method return, caller shouldn't - * attempt to read the data files. - * - * @param temporaryFolder Folder to place the data to - * @param fileCount The number of files to create and add to the table - * @param filesPerSplit The number of files used for a split - * @param version The table version to create - */ - public static List createSplitsFromTransientHadoopTable( - Path temporaryFolder, int fileCount, int filesPerSplit, String version) throws Exception { - final File warehouseFile = File.createTempFile("junit", null, temporaryFolder.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - final String warehouse = "file:" + warehouseFile; - Configuration hadoopConf = new Configuration(); - final HadoopCatalog catalog = new HadoopCatalog(hadoopConf, warehouse); - ImmutableMap properties = - ImmutableMap.of(TableProperties.FORMAT_VERSION, version); - try { - final Table table = - catalog.createTable( - TestFixtures.TABLE_IDENTIFIER, - TestFixtures.SCHEMA, - PartitionSpec.unpartitioned(), - null, - properties); - final GenericAppenderHelper dataAppender = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - for (int i = 0; i < fileCount; ++i) { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); - dataAppender.appendToTable(records); - } - - final ScanContext scanContext = ScanContext.builder().build(); - final List splits = - FlinkSplitPlanner.planIcebergSourceSplits( - table, scanContext, ThreadPools.getWorkerPool()); - return splits.stream() - .flatMap( - split -> { - List> filesList = - Lists.partition(Lists.newArrayList(split.task().files()), filesPerSplit); - return filesList.stream() - .map(files -> new BaseCombinedScanTask(files)) - .map( - combinedScanTask -> - IcebergSourceSplit.fromCombinedScanTask(combinedScanTask)); - }) - .collect(Collectors.toList()); - } finally { - catalog.dropTable(TestFixtures.TABLE_IDENTIFIER); - catalog.close(); - } - } - - /** - * This method will equip the {@code icebergSourceSplits} with mock delete files. - *

  • For each split, create {@code deleteFilesPerSplit} number of delete files - *
  • Replace the original {@code FileScanTask} with the new {@code FileScanTask} with mock - *
  • Caller should not attempt to read the deleted files since they are created as mock, and - * they are not real files - * - * @param icebergSourceSplits The real splits to equip with mock delete files - * @param temporaryFolder The temporary folder to create the mock delete files with - * @param deleteFilesPerSplit The number of delete files to create for each split - * @return The list of re-created splits with mock delete files - * @throws IOException If there is any error creating the mock delete files - */ - public static List equipSplitsWithMockDeleteFiles( - List icebergSourceSplits, Path temporaryFolder, int deleteFilesPerSplit) - throws IOException { - List icebergSourceSplitsWithMockDeleteFiles = Lists.newArrayList(); - for (IcebergSourceSplit split : icebergSourceSplits) { - final CombinedScanTask combinedScanTask = spy(split.task()); - - final List deleteFiles = Lists.newArrayList(); - final PartitionSpec spec = - PartitionSpec.builderFor(TestFixtures.SCHEMA).withSpecId(0).build(); - - for (int i = 0; i < deleteFilesPerSplit; ++i) { - final DeleteFile deleteFile = - FileMetadata.deleteFileBuilder(spec) - .withFormat(FileFormat.PARQUET) - .withPath(File.createTempFile("junit", null, temporaryFolder.toFile()).getPath()) - .ofPositionDeletes() - .withFileSizeInBytes(1000) - .withRecordCount(1000) - .build(); - deleteFiles.add(deleteFile); - } - - List newFileScanTasks = Lists.newArrayList(); - for (FileScanTask task : combinedScanTask.tasks()) { - String schemaString = SchemaParser.toJson(task.schema()); - String specString = PartitionSpecParser.toJson(task.spec()); - - BaseFileScanTask baseFileScanTask = - new BaseFileScanTask( - task.file(), - deleteFiles.toArray(new DeleteFile[] {}), - schemaString, - specString, - ResidualEvaluator.unpartitioned(task.residual())); - newFileScanTasks.add(baseFileScanTask); - } - doReturn(newFileScanTasks).when(combinedScanTask).tasks(); - icebergSourceSplitsWithMockDeleteFiles.add( - IcebergSourceSplit.fromCombinedScanTask( - combinedScanTask, split.fileOffset(), split.recordOffset())); - } - return icebergSourceSplitsWithMockDeleteFiles; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java deleted file mode 100644 index e4e48ca67f66..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/SqlHelpers.java +++ /dev/null @@ -1,60 +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.source; - -import java.util.List; -import java.util.Map; -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.iceberg.relocated.com.google.common.collect.Lists; - -public class SqlHelpers { - private SqlHelpers() {} - - public static List sql(TableEnvironment tableEnv, String query, Object... args) { - TableResult tableResult = tableEnv.executeSql(String.format(query, args)); - try (CloseableIterator iter = tableResult.collect()) { - List results = Lists.newArrayList(iter); - return results; - } catch (Exception e) { - throw new RuntimeException("Failed to collect table result", e); - } - } - - public static String sqlOptionsToString(Map sqlOptions) { - StringBuilder builder = new StringBuilder(); - sqlOptions.forEach((key, value) -> builder.append(optionToKv(key, value)).append(",")); - String optionStr = builder.toString(); - if (optionStr.endsWith(",")) { - optionStr = optionStr.substring(0, optionStr.length() - 1); - } - - if (!optionStr.isEmpty()) { - optionStr = String.format("/*+ OPTIONS(%s)*/", optionStr); - } - - return optionStr; - } - - private static String optionToKv(String key, Object value) { - return "'" + key + "'='" + value + "'"; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java deleted file mode 100644 index 32c81d9465a4..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TableSourceTestBase.java +++ /dev/null @@ -1,104 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.io.IOException; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.events.Listeners; -import org.apache.iceberg.events.ScanEvent; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestBase; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public abstract class TableSourceTestBase extends TestBase { - @Parameters(name = "useFlip27Source = {0}") - protected static Object[][] parameters() { - return new Object[][] { - {false}, {true}, - }; - } - - @Parameter(index = 0) - protected boolean useFlip27Source; - - protected static final String CATALOG_NAME = "test_catalog"; - protected static final String DATABASE_NAME = "test_db"; - protected static final String TABLE_NAME = "test_table"; - protected final FileFormat format = FileFormat.AVRO; - protected int scanEventCount = 0; - protected ScanEvent lastScanEvent = null; - - @Override - protected TableEnvironment getTableEnv() { - super.getTableEnv().getConfig().getConfiguration().set(CoreOptions.DEFAULT_PARALLELISM, 1); - super.getTableEnv() - .getConfig() - .getConfiguration() - .setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), useFlip27Source); - return super.getTableEnv(); - } - - @BeforeEach - public void before() throws IOException { - // register a scan event listener to validate pushdown - Listeners.register( - event -> { - scanEventCount += 1; - lastScanEvent = event; - }, - ScanEvent.class); - - File warehouseFile = File.createTempFile("junit", null, temporaryDirectory.toFile()); - assertThat(warehouseFile.delete()).isTrue(); - String warehouse = String.format("file:%s", warehouseFile); - - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - - this.scanEventCount = 0; - this.lastScanEvent = null; - } - - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, TABLE_NAME); - dropCatalog(CATALOG_NAME, true); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java deleted file mode 100644 index bde751e1f87f..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestBoundedTableFactory.java +++ /dev/null @@ -1,81 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.List; -import java.util.Objects; -import java.util.stream.Collectors; -import org.apache.flink.types.Row; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Streams; -import org.junit.jupiter.api.Test; - -public class TestBoundedTableFactory extends ChangeLogTableTestBase { - - @Test - public void testEmptyDataSet() { - List> emptyDataSet = ImmutableList.of(); - - String dataId = BoundedTableFactory.registerDataSet(emptyDataSet); - sql( - "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - tableName, dataId); - - assertThat(sql("SELECT * FROM %s", tableName)).isEmpty(); - } - - @Test - public void testBoundedTableFactory() { - List> dataSet = - ImmutableList.of( - ImmutableList.of( - insertRow(1, "aaa"), - deleteRow(1, "aaa"), - insertRow(1, "bbb"), - insertRow(2, "aaa"), - deleteRow(2, "aaa"), - insertRow(2, "bbb")), - ImmutableList.of( - updateBeforeRow(2, "bbb"), - updateAfterRow(2, "ccc"), - deleteRow(2, "ccc"), - insertRow(2, "ddd")), - ImmutableList.of( - deleteRow(1, "bbb"), - insertRow(1, "ccc"), - deleteRow(1, "ccc"), - insertRow(1, "ddd"))); - - String dataId = BoundedTableFactory.registerDataSet(dataSet); - sql( - "CREATE TABLE %s(id INT, data STRING) WITH ('connector'='BoundedSource', 'data-id'='%s')", - tableName, dataId); - - List rowSet = dataSet.stream().flatMap(Streams::stream).collect(Collectors.toList()); - assertThat(sql("SELECT * FROM %s", tableName)).isEqualTo(rowSet); - - assertThat(sql("SELECT * FROM %s WHERE data='aaa'", tableName)) - .isEqualTo( - rowSet.stream() - .filter(r -> Objects.equals(r.getField(1), "aaa")) - .collect(Collectors.toList())); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java deleted file mode 100644 index c8b65e131c33..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormat.java +++ /dev/null @@ -1,211 +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.source; - -import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.assertj.core.api.Assertions.assertThatThrownBy; -import static org.assertj.core.api.Assumptions.assumeThat; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.DataTypes; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.TestTemplate; - -/** Test {@link FlinkInputFormat}. */ -public class TestFlinkInputFormat extends TestFlinkSource { - - @Override - protected List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - return runFormat(formatBuilder.tableLoader(tableLoader()).buildFormat()); - } - - @TestTemplate - public void testNestedProjection() throws Exception { - Schema schema = - new Schema( - required(1, "data", Types.StringType.get()), - required( - 2, - "nested", - Types.StructType.of( - Types.NestedField.required(3, "f1", Types.StringType.get()), - Types.NestedField.required(4, "f2", Types.StringType.get()), - Types.NestedField.required(5, "f3", Types.LongType.get()))), - required(6, "id", Types.LongType.get())); - - Table table = - CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), schema); - - List writeRecords = RandomGenericData.generate(schema, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); - - // Schema: [data, nested[f1, f2, f3], id] - // Projection: [nested.f2, data] - // The Flink SQL output: [f2, data] - // The FlinkInputFormat output: [nested[f2], data] - - TableSchema projectedSchema = - TableSchema.builder() - .field("nested", DataTypes.ROW(DataTypes.FIELD("f2", DataTypes.STRING()))) - .field("data", DataTypes.STRING()) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : writeRecords) { - Row nested = Row.of(((Record) record.get(1)).get(1)); - expected.add(Row.of(nested, record.get(0))); - } - - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testBasicProjection() throws IOException { - Schema writeSchema = - new Schema( - Types.NestedField.required(0, "id", Types.LongType.get()), - Types.NestedField.optional(1, "data", Types.StringType.get()), - Types.NestedField.optional(2, "time", Types.TimestampType.withZone())); - - Table table = - CATALOG_EXTENSION.catalog().createTable(TableIdentifier.of("default", "t"), writeSchema); - - List writeRecords = RandomGenericData.generate(writeSchema, 2, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory).appendToTable(writeRecords); - - TableSchema projectedSchema = - TableSchema.builder() - .field("id", DataTypes.BIGINT()) - .field("data", DataTypes.STRING()) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : writeRecords) { - expected.add(Row.of(record.get(0), record.get(1))); - } - - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testReadPartitionColumn() throws Exception { - assumeThat(fileFormat).as("Temporary skip ORC").isNotEqualTo(FileFormat.ORC); - - Schema nestedSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.LongType.get()), - Types.NestedField.optional( - 2, - "struct", - Types.StructType.of( - Types.NestedField.optional(3, "innerId", Types.LongType.get()), - Types.NestedField.optional(4, "innerName", Types.StringType.get())))); - PartitionSpec spec = - PartitionSpec.builderFor(nestedSchema).identity("struct.innerName").build(); - - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, nestedSchema, spec); - List records = RandomGenericData.generate(nestedSchema, 10, 0L); - GenericAppenderHelper appender = - new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - for (Record record : records) { - org.apache.iceberg.TestHelpers.Row partition = - org.apache.iceberg.TestHelpers.Row.of(record.get(1, Record.class).get(1)); - appender.appendToTable(partition, Collections.singletonList(record)); - } - - TableSchema projectedSchema = - TableSchema.builder() - .field("struct", DataTypes.ROW(DataTypes.FIELD("innerName", DataTypes.STRING()))) - .build(); - List result = - runFormat( - FlinkSource.forRowData() - .tableLoader(tableLoader()) - .project(projectedSchema) - .buildFormat()); - - List expected = Lists.newArrayList(); - for (Record record : records) { - Row nested = Row.of(((Record) record.get(1)).get(1)); - expected.add(Row.of(nested)); - } - - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testValidation() { - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); - - assertThatThrownBy( - () -> - FlinkSource.forRowData() - .env(StreamExecutionEnvironment.getExecutionEnvironment()) - .tableLoader(tableLoader()) - .streaming(false) - .endTag("tag") - .endSnapshotId(1L) - .build()) - .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") - .isInstanceOf(IllegalArgumentException.class); - } - - private List runFormat(FlinkInputFormat inputFormat) throws IOException { - RowType rowType = FlinkSchemaUtil.convert(inputFormat.projectedSchema()); - return TestHelpers.readRows(inputFormat, rowType); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.java deleted file mode 100644 index 226da5813ad8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkInputFormatReaderDeletes.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.flink.source; - -import java.io.IOException; -import java.util.Map; -import org.apache.flink.table.types.logical.RowType; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.util.StructLikeSet; - -public class TestFlinkInputFormatReaderDeletes extends TestFlinkReaderDeletesBase { - @Override - protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) - throws IOException { - Schema projected = testTable.schema().select(columns); - RowType rowType = FlinkSchemaUtil.convert(projected); - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.WAREHOUSE_LOCATION, - hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); - properties.put( - CatalogProperties.CLIENT_POOL_SIZE, - Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); - CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); - FlinkInputFormat inputFormat = - FlinkSource.forRowData() - .tableLoader( - TableLoader.fromCatalog( - hiveCatalogLoader, TableIdentifier.of("default", tableName))) - .project(FlinkSchemaUtil.toSchema(rowType)) - .buildFormat(); - - StructLikeSet set = StructLikeSet.create(projected.asStruct()); - TestHelpers.readRowData(inputFormat, rowType) - .forEach( - rowData -> { - RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); - set.add(wrapper.wrap(rowData)); - }); - - return set; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java deleted file mode 100644 index 5be4a31b4ac8..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMergingMetrics.java +++ /dev/null @@ -1,67 +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.source; - -import java.io.File; -import java.io.IOException; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestMergingMetrics; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.RowDataConverter; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.sink.FlinkAppenderFactory; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; -import org.junit.jupiter.api.extension.RegisterExtension; - -public class TestFlinkMergingMetrics extends TestMergingMetrics { - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension("test_db", "test_table"); - - @Override - protected FileAppender writeAndGetAppender(List records) throws IOException { - Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); - RowType flinkSchema = FlinkSchemaUtil.convert(SCHEMA); - FileAppender appender = - new FlinkAppenderFactory( - table, - SCHEMA, - flinkSchema, - ImmutableMap.of(), - PartitionSpec.unpartitioned(), - null, - null, - null) - .newAppender( - org.apache.iceberg.Files.localOutput(File.createTempFile("junit", null, tempDir)), - fileFormat); - try (FileAppender fileAppender = appender) { - records.stream().map(r -> RowDataConverter.convert(SCHEMA, r)).forEach(fileAppender::add); - } - return appender; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java deleted file mode 100644 index f58cc87c6a29..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkMetaDataTable.java +++ /dev/null @@ -1,813 +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.source; - -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.time.Instant; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.Stream; -import org.apache.avro.generic.GenericData; -import org.apache.commons.collections.ListUtils; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileContent; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Files; -import org.apache.iceberg.HasTableOperations; -import org.apache.iceberg.ManifestFile; -import org.apache.iceberg.MetadataTableType; -import org.apache.iceberg.MetadataTableUtils; -import org.apache.iceberg.MetricsUtil; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.avro.Avro; -import org.apache.iceberg.catalog.Namespace; -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.expressions.Expressions; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.InputFile; -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.SnapshotUtil; -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; - -public class TestFlinkMetaDataTable extends CatalogTestBase { - private static final String TABLE_NAME = "test_table"; - private final FileFormat format = FileFormat.AVRO; - private @TempDir Path temp; - - @Parameter(index = 2) - private Boolean isPartition; - - @Parameters(name = "catalogName={0}, baseNamespace={1}, isPartition={2}") - protected static List parameters() { - List parameters = Lists.newArrayList(); - - for (Boolean isPartition : new Boolean[] {true, false}) { - String catalogName = "testhadoop"; - Namespace baseNamespace = Namespace.of("default"); - parameters.add(new Object[] {catalogName, baseNamespace, isPartition}); - } - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); - configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - @BeforeEach - public void before() { - super.before(); - sql("USE CATALOG %s", catalogName); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE %s", DATABASE); - if (isPartition) { - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) PARTITIONED BY (data) WITH ('format-version'='2', 'write.format.default'='%s')", - TABLE_NAME, format.name()); - sql("INSERT INTO %s VALUES (1,'a',10),(2,'a',20)", TABLE_NAME); - sql("INSERT INTO %s VALUES (1,'b',10),(2,'b',20)", TABLE_NAME); - } else { - sql( - "CREATE TABLE %s (id INT, data VARCHAR,d DOUBLE) WITH ('format-version'='2', 'write.format.default'='%s')", - TABLE_NAME, format.name()); - sql( - "INSERT INTO %s VALUES (1,'iceberg',10),(2,'b',20),(3,CAST(NULL AS VARCHAR),30)", - TABLE_NAME); - sql("INSERT INTO %s VALUES (4,'iceberg',10)", TABLE_NAME); - } - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - @TestTemplate - public void testSnapshots() { - String sql = String.format("SELECT * FROM %s$snapshots ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - Iterator snapshots = table.snapshots().iterator(); - for (Row row : result) { - Snapshot next = snapshots.next(); - assertThat(((Instant) row.getField(0)).toEpochMilli()) - .as("Should have expected timestamp") - .isEqualTo(next.timestampMillis()); - assertThat(next.snapshotId()) - .as("Should have expected snapshot id") - .isEqualTo(next.snapshotId()); - assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); - assertThat(row.getField(3)).as("Should have expected operation").isEqualTo(next.operation()); - assertThat(row.getField(4)) - .as("Should have expected manifest list location") - .isEqualTo(next.manifestListLocation()); - assertThat(row.getField(5)).as("Should have expected summary").isEqualTo(next.summary()); - } - } - - @TestTemplate - public void testHistory() { - String sql = String.format("SELECT * FROM %s$history ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - Iterator snapshots = table.snapshots().iterator(); - for (Row row : result) { - Snapshot next = snapshots.next(); - assertThat(((Instant) row.getField(0)).toEpochMilli()) - .as("Should have expected made_current_at") - .isEqualTo(next.timestampMillis()); - assertThat(row.getField(1)) - .as("Should have expected snapshot id") - .isEqualTo(next.snapshotId()); - assertThat(row.getField(2)).as("Should have expected parent id").isEqualTo(next.parentId()); - assertThat(row.getField(3)) - .as("Should have expected is current ancestor") - .isEqualTo( - SnapshotUtil.isAncestorOf( - table, table.currentSnapshot().snapshotId(), next.snapshotId())); - } - } - - @TestTemplate - public void testManifests() { - String sql = String.format("SELECT * FROM %s$manifests ", TABLE_NAME); - List result = sql(sql); - - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - List expectedDataManifests = dataManifests(table); - - for (int i = 0; i < result.size(); i++) { - Row row = result.get(i); - ManifestFile manifestFile = expectedDataManifests.get(i); - assertThat(row.getField(0)) - .as("Should have expected content") - .isEqualTo(manifestFile.content().id()); - assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); - assertThat(row.getField(2)) - .as("Should have expected length") - .isEqualTo(manifestFile.length()); - assertThat(row.getField(3)) - .as("Should have expected partition_spec_id") - .isEqualTo(manifestFile.partitionSpecId()); - assertThat(row.getField(4)) - .as("Should have expected added_snapshot_id") - .isEqualTo(manifestFile.snapshotId()); - assertThat(row.getField(5)) - .as("Should have expected added_data_files_count") - .isEqualTo(manifestFile.addedFilesCount()); - assertThat(row.getField(6)) - .as("Should have expected existing_data_files_count") - .isEqualTo(manifestFile.existingFilesCount()); - assertThat(row.getField(7)) - .as("Should have expected deleted_data_files_count") - .isEqualTo(manifestFile.deletedFilesCount()); - } - } - - @TestTemplate - public void testAllManifests() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - String sql = String.format("SELECT * FROM %s$all_manifests ", TABLE_NAME); - List result = sql(sql); - - List expectedDataManifests = allDataManifests(table); - - assertThat(expectedDataManifests).hasSize(result.size()); - for (int i = 0; i < result.size(); i++) { - Row row = result.get(i); - ManifestFile manifestFile = expectedDataManifests.get(i); - assertThat(row.getField(0)) - .as("Should have expected content") - .isEqualTo(manifestFile.content().id()); - assertThat(row.getField(1)).as("Should have expected path").isEqualTo(manifestFile.path()); - assertThat(row.getField(2)) - .as("Should have expected length") - .isEqualTo(manifestFile.length()); - assertThat(row.getField(3)) - .as("Should have expected partition_spec_id") - .isEqualTo(manifestFile.partitionSpecId()); - assertThat(row.getField(4)) - .as("Should have expected added_snapshot_id") - .isEqualTo(manifestFile.snapshotId()); - assertThat(row.getField(5)) - .as("Should have expected added_data_files_count") - .isEqualTo(manifestFile.addedFilesCount()); - assertThat(row.getField(6)) - .as("Should have expected existing_data_files_count") - .isEqualTo(manifestFile.existingFilesCount()); - assertThat(row.getField(7)) - .as("Should have expected deleted_data_files_count") - .isEqualTo(manifestFile.deletedFilesCount()); - } - } - - @TestTemplate - public void testUnPartitionedTable() throws IOException { - assumeThat(isPartition).isFalse(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - List dataDeletes = Lists.newArrayList(dataDelete.copy("id", 1)); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, Files.localOutput(testFile), dataDeletes, deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - List expectedDataManifests = dataManifests(table); - List expectedDeleteManifests = deleteManifests(table); - - assertThat(expectedDataManifests).hasSize(2); - assertThat(expectedDeleteManifests).hasSize(1); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - - // check delete files table - Schema deleteFilesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("delete_files")) - .schema(); - - List deleteColumns = - deleteFilesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String deleteNames = - deleteColumns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - deleteFilesTableSchema = deleteFilesTableSchema.select(deleteColumns); - - List actualDeleteFiles = sql("SELECT %s FROM %s$delete_files", deleteNames, TABLE_NAME); - assertThat(actualDeleteFiles).hasSize(1); - assertThat(expectedDeleteManifests).as("Should have 1 delete manifest").hasSize(1); - - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); - assertThat(expectedDeleteFiles).as("Should be 1 delete file manifest entry").hasSize(1); - TestHelpers.assertEquals( - deleteFilesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check data files table - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - List actualDataFiles = sql("SELECT %s FROM %s$data_files", names, TABLE_NAME); - assertThat(actualDataFiles).as("Metadata table should return 2 data file").hasSize(2); - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - assertThat(expectedDataFiles).as("Should be 2 data file manifest entry").hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - // check all files table - List actualFiles = sql("SELECT %s FROM %s$files ORDER BY content", names, TABLE_NAME); - assertThat(actualFiles).as("Metadata table should return 3 files").hasSize(3); - List expectedFiles = - Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) - .collect(Collectors.toList()); - assertThat(expectedFiles).as("Should have 3 files manifest entriess").hasSize(3); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); - } - - @TestTemplate - public void testPartitionedTable() throws Exception { - assumeThat(isPartition).isTrue(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id", "data"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - deleteRow.put("data", "a"); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile), - org.apache.iceberg.TestHelpers.Row.of("a"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - deleteRow.put("data", "b"); - File testFile2 = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes2 = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile2), - org.apache.iceberg.TestHelpers.Row.of("b"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes2).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - - List expectedDataManifests = dataManifests(table); - List expectedDeleteManifests = deleteManifests(table); - - assertThat(expectedDataManifests).hasSize(2); - assertThat(expectedDeleteManifests).hasSize(2); - Table deleteFilesTable = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("delete_files")); - Schema filesTableSchema = deleteFilesTable.schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check delete files table - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - assertThat(expectedDeleteFiles).hasSize(1); - List actualDeleteFiles = - sql("SELECT %s FROM %s$delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - - assertThat(actualDeleteFiles).hasSize(1); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check data files table - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - assertThat(expectedDataFiles).hasSize(1); - List actualDataFiles = - sql("SELECT %s FROM %s$data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - assertThat(actualDataFiles).hasSize(1); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - List actualPartitionsWithProjection = - sql("SELECT file_count FROM %s$partitions ", TABLE_NAME); - assertThat(actualPartitionsWithProjection).hasSize(2); - for (int i = 0; i < 2; ++i) { - assertThat(actualPartitionsWithProjection.get(i).getField(0)).isEqualTo(1); - } - - // Check files table - List expectedFiles = - Stream.concat(expectedDataFiles.stream(), expectedDeleteFiles.stream()) - .collect(Collectors.toList()); - assertThat(expectedFiles).hasSize(2); - List actualFiles = - sql( - "SELECT %s FROM %s$files WHERE `partition`.`data`='a' ORDER BY content", - names, TABLE_NAME); - assertThat(actualFiles).hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(0), actualFiles.get(0)); - TestHelpers.assertEquals(filesTableSchema, expectedFiles.get(1), actualFiles.get(1)); - } - - @TestTemplate - public void testAllFilesUnpartitioned() throws Exception { - assumeThat(isPartition).isFalse(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Schema deleteRowSchema = table.schema().select("id", "data"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).commit(); - - List expectedDataManifests = dataManifests(table); - assertThat(expectedDataManifests).hasSize(2); - List expectedDeleteManifests = deleteManifests(table); - assertThat(expectedDeleteManifests).hasSize(1); - - // Clear table to test whether 'all_files' can read past files - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("all_data_files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check all data files table - List actualDataFiles = - sql("SELECT %s FROM %s$all_data_files order by record_count ", names, TABLE_NAME); - - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, null); - assertThat(expectedDataFiles).hasSize(2); - assertThat(actualDataFiles).hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles, actualDataFiles); - - // Check all delete files table - List actualDeleteFiles = sql("SELECT %s FROM %s$all_delete_files", names, TABLE_NAME); - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, null); - assertThat(expectedDeleteFiles).hasSize(1); - assertThat(actualDeleteFiles).hasSize(1); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check all files table - List actualFiles = - sql("SELECT %s FROM %s$all_files ORDER BY content, record_count asc", names, TABLE_NAME); - List expectedFiles = - ListUtils.union(expectedDataFiles, expectedDeleteFiles); - expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - assertThat(actualFiles).hasSize(3); - TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); - } - - @TestTemplate - public void testAllFilesPartitioned() throws Exception { - assumeThat(!isPartition).isFalse(); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - // Create delete file - Schema deleteRowSchema = table.schema().select("id"); - Record dataDelete = GenericRecord.create(deleteRowSchema); - - Map deleteRow = Maps.newHashMap(); - deleteRow.put("id", 1); - File testFile = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile), - org.apache.iceberg.TestHelpers.Row.of("a"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - File testFile2 = File.createTempFile("junit", null, temp.toFile()); - DeleteFile eqDeletes2 = - FileHelpers.writeDeleteFile( - table, - Files.localOutput(testFile2), - org.apache.iceberg.TestHelpers.Row.of("b"), - Lists.newArrayList(dataDelete.copy(deleteRow)), - deleteRowSchema); - table.newRowDelta().addDeletes(eqDeletes).addDeletes(eqDeletes2).commit(); - - List expectedDataManifests = dataManifests(table); - assertThat(expectedDataManifests).hasSize(2); - List expectedDeleteManifests = deleteManifests(table); - assertThat(expectedDeleteManifests).hasSize(1); - // Clear table to test whether 'all_files' can read past files - table.newDelete().deleteFromRowFilter(Expressions.alwaysTrue()).commit(); - - Schema entriesTableSchema = - MetadataTableUtils.createMetadataTableInstance(table, MetadataTableType.from("entries")) - .schema(); - Schema filesTableSchema = - MetadataTableUtils.createMetadataTableInstance( - table, MetadataTableType.from("all_data_files")) - .schema(); - - List columns = - filesTableSchema.columns().stream() - .map(Types.NestedField::name) - .filter(c -> !c.equals(MetricsUtil.READABLE_METRICS)) - .collect(Collectors.toList()); - String names = columns.stream().map(n -> "`" + n + "`").collect(Collectors.joining(",")); - - filesTableSchema = filesTableSchema.select(columns); - - // Check all data files table - List actualDataFiles = - sql("SELECT %s FROM %s$all_data_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - List expectedDataFiles = - expectedEntries(table, FileContent.DATA, entriesTableSchema, expectedDataManifests, "a"); - assertThat(expectedDataFiles).hasSize(1); - assertThat(actualDataFiles).hasSize(1); - TestHelpers.assertEquals(filesTableSchema, expectedDataFiles.get(0), actualDataFiles.get(0)); - - // Check all delete files table - List actualDeleteFiles = - sql("SELECT %s FROM %s$all_delete_files WHERE `partition`.`data`='a'", names, TABLE_NAME); - List expectedDeleteFiles = - expectedEntries( - table, FileContent.EQUALITY_DELETES, entriesTableSchema, expectedDeleteManifests, "a"); - assertThat(expectedDeleteFiles).hasSize(1); - assertThat(actualDeleteFiles).hasSize(1); - TestHelpers.assertEquals( - filesTableSchema, expectedDeleteFiles.get(0), actualDeleteFiles.get(0)); - - // Check all files table - List actualFiles = - sql( - "SELECT %s FROM %s$all_files WHERE `partition`.`data`='a' ORDER BY content", - names, TABLE_NAME); - List expectedFiles = - ListUtils.union(expectedDataFiles, expectedDeleteFiles); - expectedFiles.sort(Comparator.comparing(r -> ((Integer) r.get("content")))); - assertThat(actualFiles).hasSize(2); - TestHelpers.assertEquals(filesTableSchema, expectedFiles, actualFiles); - } - - @TestTemplate - public void testMetadataLogEntries() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Long currentSnapshotId = table.currentSnapshot().snapshotId(); - TableMetadata tableMetadata = ((HasTableOperations) table).operations().current(); - Snapshot currentSnapshot = tableMetadata.currentSnapshot(); - Snapshot parentSnapshot = table.snapshot(currentSnapshot.parentId()); - List metadataLogEntries = - Lists.newArrayList(tableMetadata.previousFiles()); - - // Check metadataLog table - List metadataLogs = sql("SELECT * FROM %s$metadata_log_entries", TABLE_NAME); - - assertThat(metadataLogs).hasSize(3); - Row metadataLog = metadataLogs.get(0); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(0).timestampMillis())); - assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(0).file()); - assertThat(metadataLog.getField("latest_snapshot_id")).isNull(); - assertThat(metadataLog.getField("latest_schema_id")).isNull(); - assertThat(metadataLog.getField("latest_sequence_number")).isNull(); - - metadataLog = metadataLogs.get(1); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(metadataLogEntries.get(1).timestampMillis())); - assertThat(metadataLog.getField("file")).isEqualTo(metadataLogEntries.get(1).file()); - assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); - assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(parentSnapshot.schemaId()); - assertThat(metadataLog.getField("latest_sequence_number")) - .isEqualTo(parentSnapshot.sequenceNumber()); - assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(parentSnapshot.snapshotId()); - - metadataLog = metadataLogs.get(2); - assertThat(metadataLog.getField("timestamp")) - .isEqualTo(Instant.ofEpochMilli(currentSnapshot.timestampMillis())); - assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); - assertThat(metadataLog.getField("latest_snapshot_id")).isEqualTo(currentSnapshot.snapshotId()); - assertThat(metadataLog.getField("latest_schema_id")).isEqualTo(currentSnapshot.schemaId()); - assertThat(metadataLog.getField("latest_sequence_number")) - .isEqualTo(currentSnapshot.sequenceNumber()); - - // test filtering - List metadataLogWithFilters = - sql( - "SELECT * FROM %s$metadata_log_entries WHERE latest_snapshot_id = %s", - TABLE_NAME, currentSnapshotId); - assertThat(metadataLogWithFilters).hasSize(1); - metadataLog = metadataLogWithFilters.get(0); - assertThat(Instant.ofEpochMilli(tableMetadata.currentSnapshot().timestampMillis())) - .isEqualTo(metadataLog.getField("timestamp")); - - assertThat(metadataLog.getField("file")).isEqualTo(tableMetadata.metadataFileLocation()); - assertThat(metadataLog.getField("latest_snapshot_id")) - .isEqualTo(tableMetadata.currentSnapshot().snapshotId()); - assertThat(metadataLog.getField("latest_schema_id")) - .isEqualTo(tableMetadata.currentSnapshot().schemaId()); - assertThat(metadataLog.getField("latest_sequence_number")) - .isEqualTo(tableMetadata.currentSnapshot().sequenceNumber()); - - // test projection - List metadataFiles = - metadataLogEntries.stream() - .map(TableMetadata.MetadataLogEntry::file) - .collect(Collectors.toList()); - metadataFiles.add(tableMetadata.metadataFileLocation()); - List metadataLogWithProjection = - sql("SELECT file FROM %s$metadata_log_entries", TABLE_NAME); - assertThat(metadataLogWithProjection).hasSize(3); - for (int i = 0; i < metadataFiles.size(); i++) { - assertThat(metadataLogWithProjection.get(i).getField("file")).isEqualTo(metadataFiles.get(i)); - } - } - - @TestTemplate - public void testSnapshotReferencesMetatable() { - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE_NAME)); - - Long currentSnapshotId = table.currentSnapshot().snapshotId(); - - // Create branch - table - .manageSnapshots() - .createBranch("testBranch", currentSnapshotId) - .setMaxRefAgeMs("testBranch", 10) - .setMinSnapshotsToKeep("testBranch", 20) - .setMaxSnapshotAgeMs("testBranch", 30) - .commit(); - // Create Tag - table - .manageSnapshots() - .createTag("testTag", currentSnapshotId) - .setMaxRefAgeMs("testTag", 50) - .commit(); - // Check refs table - List references = sql("SELECT * FROM %s$refs", TABLE_NAME); - List branches = sql("SELECT * FROM %s$refs WHERE type='BRANCH'", TABLE_NAME); - assertThat(references).hasSize(3); - assertThat(branches).hasSize(2); - List tags = sql("SELECT * FROM %s$refs WHERE type='TAG'", TABLE_NAME); - assertThat(tags).hasSize(1); - // Check branch entries in refs table - List mainBranch = - sql("SELECT * FROM %s$refs WHERE name='main' AND type='BRANCH'", TABLE_NAME); - assertThat((String) mainBranch.get(0).getFieldAs("name")).isEqualTo("main"); - assertThat((String) mainBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - assertThat((Long) mainBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); - List testBranch = - sql("SELECT * FROM %s$refs WHERE name='testBranch' AND type='BRANCH'", TABLE_NAME); - assertThat((String) testBranch.get(0).getFieldAs("name")).isEqualTo("testBranch"); - assertThat((String) testBranch.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - assertThat((Long) testBranch.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); - assertThat((Long) testBranch.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(10)); - assertThat((Integer) testBranch.get(0).getFieldAs("min_snapshots_to_keep")) - .isEqualTo(Integer.valueOf(20)); - assertThat((Long) testBranch.get(0).getFieldAs("max_snapshot_age_in_ms")) - .isEqualTo(Long.valueOf(30)); - - // Check tag entries in refs table - List testTag = - sql("SELECT * FROM %s$refs WHERE name='testTag' AND type='TAG'", TABLE_NAME); - assertThat((String) testTag.get(0).getFieldAs("name")).isEqualTo("testTag"); - assertThat((String) testTag.get(0).getFieldAs("type")).isEqualTo("TAG"); - assertThat((Long) testTag.get(0).getFieldAs("snapshot_id")).isEqualTo(currentSnapshotId); - assertThat((Long) testTag.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(50)); - // Check projection in refs table - List testTagProjection = - sql( - "SELECT name,type,snapshot_id,max_reference_age_in_ms,min_snapshots_to_keep FROM %s$refs where type='TAG'", - TABLE_NAME); - assertThat((String) testTagProjection.get(0).getFieldAs("name")).isEqualTo("testTag"); - assertThat((String) testTagProjection.get(0).getFieldAs("type")).isEqualTo("TAG"); - assertThat((Long) testTagProjection.get(0).getFieldAs("snapshot_id")) - .isEqualTo(currentSnapshotId); - assertThat((Long) testTagProjection.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(50)); - assertThat((String) testTagProjection.get(0).getFieldAs("min_snapshots_to_keep")).isNull(); - List mainBranchProjection = - sql("SELECT name, type FROM %s$refs WHERE name='main' AND type = 'BRANCH'", TABLE_NAME); - assertThat((String) mainBranchProjection.get(0).getFieldAs("name")).isEqualTo("main"); - assertThat((String) mainBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - List testBranchProjection = - sql( - "SELECT type, name, max_reference_age_in_ms, snapshot_id FROM %s$refs WHERE name='testBranch' AND type = 'BRANCH'", - TABLE_NAME); - assertThat((String) testBranchProjection.get(0).getFieldAs("name")).isEqualTo("testBranch"); - assertThat((String) testBranchProjection.get(0).getFieldAs("type")).isEqualTo("BRANCH"); - assertThat((Long) testBranchProjection.get(0).getFieldAs("snapshot_id")) - .isEqualTo(currentSnapshotId); - assertThat((Long) testBranchProjection.get(0).getFieldAs("max_reference_age_in_ms")) - .isEqualTo(Long.valueOf(10)); - } - - /** - * Find matching manifest entries of an Iceberg table - * - * @param table iceberg table - * @param expectedContent file content to populate on entries - * @param entriesTableSchema schema of Manifest entries - * @param manifestsToExplore manifests to explore of the table - * @param partValue partition value that manifest entries must match, or null to skip filtering - */ - private List expectedEntries( - Table table, - FileContent expectedContent, - Schema entriesTableSchema, - List manifestsToExplore, - String partValue) - throws IOException { - List expected = Lists.newArrayList(); - for (ManifestFile manifest : manifestsToExplore) { - InputFile in = table.io().newInputFile(manifest.path()); - try (CloseableIterable rows = - Avro.read(in).project(entriesTableSchema).build()) { - for (GenericData.Record record : rows) { - if ((Integer) record.get("status") < 2 /* added or existing */) { - GenericData.Record file = (GenericData.Record) record.get("data_file"); - if (partitionMatch(file, partValue)) { - asMetadataRecord(file, expectedContent); - expected.add(file); - } - } - } - } - } - return expected; - } - - // Populate certain fields derived in the metadata tables - private void asMetadataRecord(GenericData.Record file, FileContent content) { - file.put(0, content.id()); - file.put(3, 0); // specId - } - - private boolean partitionMatch(GenericData.Record file, String partValue) { - if (partValue == null) { - return true; - } - GenericData.Record partition = (GenericData.Record) file.get(4); - return partValue.equals(partition.get(0).toString()); - } - - private List dataManifests(Table table) { - return table.currentSnapshot().dataManifests(table.io()); - } - - private List allDataManifests(Table table) { - List manifests = Lists.newArrayList(); - for (Snapshot snapshot : table.snapshots()) { - manifests.addAll(snapshot.dataManifests(table.io())); - } - return manifests; - } - - private List deleteManifests(Table table) { - return table.currentSnapshot().deleteManifests(table.io()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java deleted file mode 100644 index 0b5a8011ad3f..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkReaderDeletesBase.java +++ /dev/null @@ -1,90 +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.source; - -import java.util.Map; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.BaseTable; -import org.apache.iceberg.CatalogUtil; -import org.apache.iceberg.ParameterizedTestExtension; -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.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.DeleteReadTests; -import org.apache.iceberg.hive.HiveCatalog; -import org.apache.iceberg.hive.TestHiveMetastore; -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.AfterAll; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public abstract class TestFlinkReaderDeletesBase extends DeleteReadTests { - - protected static String databaseName = "default"; - - protected static HiveConf hiveConf = null; - protected static HiveCatalog catalog = null; - private static TestHiveMetastore metastore = null; - - @BeforeAll - public static void startMetastore() { - metastore = new TestHiveMetastore(); - metastore.start(); - hiveConf = metastore.hiveConf(); - catalog = - (HiveCatalog) - CatalogUtil.loadCatalog( - HiveCatalog.class.getName(), "hive", ImmutableMap.of(), hiveConf); - } - - @AfterAll - public static void stopMetastore() throws Exception { - metastore.stop(); - catalog = null; - } - - @Override - protected Table createTable(String name, Schema schema, PartitionSpec spec) { - Map props = Maps.newHashMap(); - props.put(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - 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)); - - return table; - } - - @Override - protected void dropTable(String name) { - catalog.dropTable(TableIdentifier.of(databaseName, name)); - } - - @Override - protected boolean expectPruned() { - return false; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java deleted file mode 100644 index cf6b233dcec6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScan.java +++ /dev/null @@ -1,540 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.nio.file.Path; -import java.time.LocalDate; -import java.time.LocalDateTime; -import java.time.LocalTime; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.AppendFiles; -import org.apache.iceberg.DataFile; -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.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -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.types.Types; -import org.apache.iceberg.util.DateTimeUtil; -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 TestFlinkScan { - @RegisterExtension - protected static MiniClusterExtension miniClusterExtension = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @TempDir protected Path temporaryDirectory; - - @RegisterExtension - protected static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - - @Parameter protected FileFormat fileFormat; - - @Parameters(name = "format={0}") - public static Collection fileFormat() { - return Arrays.asList(FileFormat.AVRO, FileFormat.PARQUET, FileFormat.ORC); - } - - protected TableLoader tableLoader() { - return CATALOG_EXTENSION.tableLoader(); - } - - protected abstract List runWithProjection(String... projected) throws Exception; - - protected abstract List runWithFilter( - Expression filter, String sqlFilter, boolean caseSensitive) throws Exception; - - protected List runWithFilter(Expression filter, String sqlFilter) throws Exception { - return runWithFilter(filter, sqlFilter, true); - } - - protected abstract List runWithOptions(Map options) throws Exception; - - protected abstract List run() throws Exception; - - @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, temporaryDirectory).appendToTable(expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testPartitionedTable() throws Exception { - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - expectedRecords.get(0).set(2, "2020-03-20"); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - TestHelpers.assertRecords(run(), expectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testProjection() throws Exception { - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - List inputRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - new GenericAppenderHelper(table, fileFormat, temporaryDirectory) - .appendToTable(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), inputRecords); - assertRows(runWithProjection("data"), Row.of(inputRecords.get(0).get(0))); - } - - @TestTemplate - public void testIdentityPartitionProjections() throws Exception { - Schema logSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "dt", Types.StringType.get()), - Types.NestedField.optional(3, "level", Types.StringType.get()), - Types.NestedField.optional(4, "message", Types.StringType.get())); - PartitionSpec spec = - PartitionSpec.builderFor(logSchema).identity("dt").identity("level").build(); - - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, logSchema, spec); - List inputRecords = RandomGenericData.generate(logSchema, 10, 0L); - - int idx = 0; - AppendFiles append = table.newAppend(); - for (Record record : inputRecords) { - record.set(1, "2020-03-2" + idx); - record.set(2, Integer.toString(idx)); - append.appendFile( - new GenericAppenderHelper(table, fileFormat, temporaryDirectory) - .writeFile( - org.apache.iceberg.TestHelpers.Row.of("2020-03-2" + idx, Integer.toString(idx)), - ImmutableList.of(record))); - idx += 1; - } - append.commit(); - - // individual fields - validateIdentityPartitionProjections(table, Collections.singletonList("dt"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("level"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("message"), inputRecords); - validateIdentityPartitionProjections(table, Collections.singletonList("id"), inputRecords); - // field pairs - validateIdentityPartitionProjections(table, Arrays.asList("dt", "message"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("level", "message"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("dt", "level"), inputRecords); - // out-of-order pairs - validateIdentityPartitionProjections(table, Arrays.asList("message", "dt"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("message", "level"), inputRecords); - validateIdentityPartitionProjections(table, Arrays.asList("level", "dt"), inputRecords); - // out-of-order triplets - validateIdentityPartitionProjections( - table, Arrays.asList("dt", "level", "message"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("level", "dt", "message"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("dt", "message", "level"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("level", "message", "dt"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("message", "dt", "level"), inputRecords); - validateIdentityPartitionProjections( - table, Arrays.asList("message", "level", "dt"), inputRecords); - } - - private void validateIdentityPartitionProjections( - Table table, List projectedFields, List inputRecords) throws Exception { - List rows = runWithProjection(projectedFields.toArray(new String[0])); - - for (int pos = 0; pos < inputRecords.size(); pos++) { - Record inputRecord = inputRecords.get(pos); - Row actualRecord = rows.get(pos); - - for (int i = 0; i < projectedFields.size(); i++) { - String name = projectedFields.get(i); - assertThat(inputRecord.getField(name)) - .as("Projected field " + name + " should match") - .isEqualTo(actualRecord.getField(i)); - } - } - } - - @TestTemplate - public void testSnapshotReads() throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords); - long snapshotId = table.currentSnapshot().snapshotId(); - - long timestampMillis = table.currentSnapshot().timestampMillis(); - - // produce another timestamp - waitUntilAfter(timestampMillis); - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L)); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("snapshot-id", Long.toString(snapshotId))), - expectedRecords, - TestFixtures.SCHEMA); - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("as-of-timestamp", Long.toString(timestampMillis))), - expectedRecords, - TestFixtures.SCHEMA); - } - - @TestTemplate - public void testTagReads() throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List expectedRecords1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords1); - long snapshotId = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().createTag("t1", snapshotId).commit(); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords1, TestFixtures.SCHEMA); - - List expectedRecords2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecords2); - snapshotId = table.currentSnapshot().snapshotId(); - - table.manageSnapshots().replaceTag("t1", snapshotId).commit(); - - List expectedRecords = Lists.newArrayList(); - expectedRecords.addAll(expectedRecords1); - expectedRecords.addAll(expectedRecords2); - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("tag", "t1")), expectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testBranchReads() throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List expectedRecordsBase = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecordsBase); - long snapshotId = table.currentSnapshot().snapshotId(); - - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName, snapshotId).commit(); - - List expectedRecordsForBranch = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(branchName, expectedRecordsForBranch); - - List expectedRecordsForMain = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(expectedRecordsForMain); - - List branchExpectedRecords = Lists.newArrayList(); - branchExpectedRecords.addAll(expectedRecordsBase); - branchExpectedRecords.addAll(expectedRecordsForBranch); - - TestHelpers.assertRecords( - runWithOptions(ImmutableMap.of("branch", branchName)), - branchExpectedRecords, - TestFixtures.SCHEMA); - - List mainExpectedRecords = Lists.newArrayList(); - mainExpectedRecords.addAll(expectedRecordsBase); - mainExpectedRecords.addAll(expectedRecordsForMain); - - TestHelpers.assertRecords(run(), mainExpectedRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testIncrementalReadViaTag() throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(records1); - long snapshotId1 = table.currentSnapshot().snapshotId(); - String startTag = "t1"; - table.manageSnapshots().createTag(startTag, snapshotId1).commit(); - - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); - helper.appendToTable(records2); - - List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); - helper.appendToTable(records3); - long snapshotId3 = table.currentSnapshot().snapshotId(); - String endTag = "t2"; - table.manageSnapshots().createTag(endTag, snapshotId3).commit(); - - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); - - List expected = Lists.newArrayList(); - expected.addAll(records2); - expected.addAll(records3); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-snapshot-id", Long.toString(snapshotId1)) - .put("end-tag", endTag) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow()), - expected, - TestFixtures.SCHEMA); - - assertThatThrownBy( - () -> - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .put("start-snapshot-id", Long.toString(snapshotId1)) - .buildOrThrow())) - .isInstanceOf(Exception.class) - .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); - - assertThatThrownBy( - () -> - runWithOptions( - ImmutableMap.builder() - .put("start-tag", startTag) - .put("end-tag", endTag) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow())) - .isInstanceOf(Exception.class) - .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set."); - } - - @TestTemplate - public void testIncrementalRead() throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 0L); - helper.appendToTable(records1); - long snapshotId1 = table.currentSnapshot().snapshotId(); - - // snapshot 2 - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1L); - helper.appendToTable(records2); - - List records3 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 2L); - helper.appendToTable(records3); - long snapshotId3 = table.currentSnapshot().snapshotId(); - - // snapshot 4 - helper.appendToTable(RandomGenericData.generate(TestFixtures.SCHEMA, 1, 3L)); - - List expected2 = Lists.newArrayList(); - expected2.addAll(records2); - expected2.addAll(records3); - TestHelpers.assertRecords( - runWithOptions( - ImmutableMap.builder() - .put("start-snapshot-id", Long.toString(snapshotId1)) - .put("end-snapshot-id", Long.toString(snapshotId3)) - .buildOrThrow()), - expected2, - TestFixtures.SCHEMA); - } - - @TestTemplate - public void testFilterExpPartition() throws Exception { - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - expectedRecords.get(0).set(2, "2020-03-20"); - expectedRecords.get(1).set(2, "2020-03-20"); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - DataFile dataFile1 = - helper.writeFile(org.apache.iceberg.TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - DataFile dataFile2 = - helper.writeFile( - org.apache.iceberg.TestHelpers.Row.of("2020-03-21", 0), - RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); - helper.appendToTable(dataFile1, dataFile2); - TestHelpers.assertRecords( - runWithFilter(Expressions.equal("dt", "2020-03-20"), "where dt='2020-03-20'", true), - expectedRecords, - TestFixtures.SCHEMA); - } - - private void testFilterExp(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 0L); - expectedRecords.get(0).set(0, "a"); - expectedRecords.get(1).set(0, "b"); - expectedRecords.get(2).set(0, "c"); - - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - DataFile dataFile = helper.writeFile(expectedRecords); - helper.appendToTable(dataFile); - - List actual = - runWithFilter(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); - - TestHelpers.assertRecords(actual, expectedRecords.subList(1, 3), TestFixtures.SCHEMA); - } - - @TestTemplate - public void testFilterExp() throws Exception { - testFilterExp(Expressions.greaterThanOrEqual("data", "b"), "where data>='b'", true); - } - - @TestTemplate - public void testFilterExpCaseInsensitive() throws Exception { - // sqlFilter does not support case-insensitive filtering: - // https://issues.apache.org/jira/browse/FLINK-16175 - testFilterExp(Expressions.greaterThanOrEqual("DATA", "b"), "where data>='b'", false); - } - - @TestTemplate - public void testPartitionTypes() throws Exception { - Schema typesSchema = - new Schema( - Types.NestedField.optional(1, "id", Types.IntegerType.get()), - Types.NestedField.optional(2, "decimal", Types.DecimalType.of(38, 18)), - Types.NestedField.optional(3, "str", Types.StringType.get()), - Types.NestedField.optional(4, "binary", Types.BinaryType.get()), - Types.NestedField.optional(5, "date", Types.DateType.get()), - Types.NestedField.optional(6, "time", Types.TimeType.get()), - Types.NestedField.optional(7, "timestamp", Types.TimestampType.withoutZone())); - PartitionSpec spec = - PartitionSpec.builderFor(typesSchema) - .identity("decimal") - .identity("str") - .identity("binary") - .identity("date") - .identity("time") - .identity("timestamp") - .build(); - - Table table = - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, typesSchema, spec); - List records = RandomGenericData.generate(typesSchema, 10, 0L); - GenericAppenderHelper appender = - new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - for (Record record : records) { - org.apache.iceberg.TestHelpers.Row partition = - org.apache.iceberg.TestHelpers.Row.of( - record.get(1), - record.get(2), - record.get(3), - record.get(4) == null ? null : DateTimeUtil.daysFromDate((LocalDate) record.get(4)), - record.get(5) == null ? null : DateTimeUtil.microsFromTime((LocalTime) record.get(5)), - record.get(6) == null - ? null - : DateTimeUtil.microsFromTimestamp((LocalDateTime) record.get(6))); - appender.appendToTable(partition, Collections.singletonList(record)); - } - - TestHelpers.assertRecords(run(), records, typesSchema); - } - - @TestTemplate - public void testCustomizedFlinkDataTypes() throws Exception { - Schema schema = - new Schema( - Types.NestedField.required( - 1, - "map", - Types.MapType.ofRequired(2, 3, Types.StringType.get(), Types.StringType.get())), - Types.NestedField.required( - 4, "arr", Types.ListType.ofRequired(5, Types.StringType.get()))); - Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, schema); - List records = RandomGenericData.generate(schema, 10, 0L); - GenericAppenderHelper helper = new GenericAppenderHelper(table, fileFormat, temporaryDirectory); - helper.appendToTable(records); - TestHelpers.assertRecords(run(), records, schema); - } - - private static void assertRows(List results, Row... expected) { - TestHelpers.assertRows(results, Arrays.asList(expected)); - } - - private static void waitUntilAfter(long timestampMillis) { - long current = System.currentTimeMillis(); - while (current <= timestampMillis) { - current = System.currentTimeMillis(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java deleted file mode 100644 index 1493c0932044..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkScanSql.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.junit.jupiter.api.BeforeEach; - -/** Test Flink SELECT SQLs. */ -public class TestFlinkScanSql extends TestFlinkSource { - private volatile TableEnvironment tEnv; - - @BeforeEach - public void before() throws IOException { - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_EXTENSION.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Override - protected List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(sqlOptions); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java deleted file mode 100644 index dd50170f0fd7..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSource.java +++ /dev/null @@ -1,90 +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.source; - -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Optional; -import org.apache.flink.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.types.Row; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -public abstract class TestFlinkSource extends TestFlinkScan { - - @Override - protected List runWithProjection(String... projected) throws Exception { - TableSchema.Builder builder = TableSchema.builder(); - TableSchema schema = - FlinkSchemaUtil.toSchema( - FlinkSchemaUtil.convert( - CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema())); - for (String field : projected) { - TableColumn column = schema.getTableColumn(field).get(); - builder.field(column.getName(), column.getType()); - } - return run(FlinkSource.forRowData().project(builder.build()), Maps.newHashMap(), "", projected); - } - - @Override - protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - FlinkSource.Builder builder = - FlinkSource.forRowData().filters(Collections.singletonList(filter)); - Map options = Maps.newHashMap(); - options.put("case-sensitive", Boolean.toString(caseSensitive)); - return run(builder, options, sqlFilter, "*"); - } - - @Override - protected List runWithOptions(Map options) throws Exception { - FlinkSource.Builder builder = FlinkSource.forRowData(); - Optional.ofNullable(options.get("case-sensitive")) - .ifPresent(value -> builder.caseSensitive(Boolean.parseBoolean(value))); - Optional.ofNullable(options.get("snapshot-id")) - .ifPresent(value -> builder.snapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("tag")).ifPresent(value -> builder.tag(value)); - Optional.ofNullable(options.get("branch")).ifPresent(value -> builder.branch(value)); - Optional.ofNullable(options.get("start-tag")).ifPresent(value -> builder.startTag(value)); - Optional.ofNullable(options.get("end-tag")).ifPresent(value -> builder.endTag(value)); - Optional.ofNullable(options.get("start-snapshot-id")) - .ifPresent(value -> builder.startSnapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("end-snapshot-id")) - .ifPresent(value -> builder.endSnapshotId(Long.parseLong(value))); - Optional.ofNullable(options.get("as-of-timestamp")) - .ifPresent(value -> builder.asOfTimestamp(Long.parseLong(value))); - return run(builder, options, "", "*"); - } - - @Override - protected List run() throws Exception { - return run(FlinkSource.forRowData(), Maps.newHashMap(), "", "*"); - } - - protected abstract List run( - FlinkSource.Builder formatBuilder, - Map sqlOptions, - String sqlFilter, - String... sqlSelectedFields) - throws Exception; -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java deleted file mode 100644 index 14131d9e96d5..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceConfig.java +++ /dev/null @@ -1,61 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.List; -import org.apache.flink.types.Row; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkSourceConfig extends TableSourceTestBase { - private static final String TABLE = "test_table"; - - @TestTemplate - public void testFlinkSessionConfig() { - getTableEnv().getConfig().set(FlinkReadOptions.STREAMING_OPTION, true); - assertThatThrownBy(() -> sql("SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='1')*/", TABLE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot set as-of-timestamp option for streaming reader"); - } - - @TestTemplate - public void testFlinkHintConfig() { - List result = - sql( - "SELECT * FROM %s /*+ OPTIONS('as-of-timestamp'='%d','streaming'='false')*/", - TABLE, System.currentTimeMillis()); - assertThat(result).hasSize(3); - } - - @TestTemplate - public void testReadOptionHierarchy() { - getTableEnv().getConfig().set(FlinkReadOptions.LIMIT_OPTION, 1L); - List result = sql("SELECT * FROM %s", TABLE); - // Note that this query doesn't have the limit clause in the SQL. - // This assertions works because limit is pushed down to the reader and - // reader parallelism is 1. - assertThat(result).hasSize(1); - - result = sql("SELECT * FROM %s /*+ OPTIONS('limit'='3')*/", TABLE); - assertThat(result).hasSize(3); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java deleted file mode 100644 index e1162c3225b1..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkSourceSql.java +++ /dev/null @@ -1,85 +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.source; - -import java.io.IOException; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.PipelineOptions; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -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.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.Test; - -/** Use the FlinkSource */ -public class TestFlinkSourceSql extends TestSqlBase { - @Override - public void before() throws IOException { - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_EXTENSION.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - @Test - public void testInferParallelismWithGlobalSetting() throws IOException { - Configuration cfg = getTableEnv().getConfig().getConfiguration(); - cfg.set(PipelineOptions.MAX_PARALLELISM, 1); - - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, null); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - List expectedRecords = Lists.newArrayList(); - long maxFileLen = 0; - for (int i = 0; i < 5; i++) { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 2, i); - DataFile dataFile = helper.writeFile(null, records); - helper.appendToTable(dataFile); - expectedRecords.addAll(records); - maxFileLen = Math.max(dataFile.fileSizeInBytes(), maxFileLen); - } - - // Make sure to generate multiple CombinedScanTasks - SqlHelpers.sql( - getTableEnv(), - "ALTER TABLE t SET ('read.split.open-file-cost'='1', 'read.split.target-size'='%s')", - maxFileLen); - - List results = run(Maps.newHashMap(), "", "*"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java deleted file mode 100644 index 18528c789114..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestFlinkTableSource.java +++ /dev/null @@ -1,561 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.List; -import org.apache.flink.table.api.SqlParserException; -import org.apache.flink.types.Row; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.TestTemplate; - -public class TestFlinkTableSource extends TableSourceTestBase { - - @TestTemplate - public void testLimitPushDown() { - - assertThatThrownBy(() -> sql("SELECT * FROM %s LIMIT -1", TABLE_NAME)) - .isInstanceOf(SqlParserException.class) - .hasMessageStartingWith("SQL parse failed."); - - assertThat(sql("SELECT * FROM %s LIMIT 0", TABLE_NAME)).isEmpty(); - - String sqlLimitExceed = String.format("SELECT * FROM %s LIMIT 4", TABLE_NAME); - List resultExceed = sql(sqlLimitExceed); - assertThat(resultExceed).hasSize(3); - List expectedList = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedList, resultExceed); - - String querySql = String.format("SELECT * FROM %s LIMIT 1", TABLE_NAME); - String explain = getTableEnv().explainSql(querySql); - String expectedExplain = "limit=[1]"; - assertThat(explain).as("Explain should contain LimitPushDown").contains(expectedExplain); - List result = sql(querySql); - assertThat(result).hasSize(1); - assertThat(result).containsAnyElementsOf(expectedList); - - String sqlMixed = String.format("SELECT * FROM %s WHERE id = 1 LIMIT 2", TABLE_NAME); - List mixedResult = sql(sqlMixed); - assertThat(mixedResult).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - } - - @TestTemplate - public void testNoFilterPushDown() { - String sql = String.format("SELECT * FROM %s ", TABLE_NAME); - List result = sql(sql); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, result); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - } - - @TestTemplate - public void testFilterPushDownEqual() { - String sqlLiteralRight = String.format("SELECT * FROM %s WHERE id = 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") == 1"; - - List result = sql(sqlLiteralRight); - assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownEqualNull() { - String sqlEqualNull = String.format("SELECT * FROM %s WHERE data = NULL ", TABLE_NAME); - - List result = sql(sqlEqualNull); - assertThat(result).isEmpty(); - assertThat(lastScanEvent).as("Should not push down a filter").isNull(); - } - - @TestTemplate - public void testFilterPushDownEqualLiteralOnLeft() { - String sqlLiteralLeft = String.format("SELECT * FROM %s WHERE 1 = id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") == 1"; - - List resultLeft = sql(sqlLiteralLeft); - assertThat(resultLeft).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownNoEqual() { - String sqlNE = String.format("SELECT * FROM %s WHERE id <> 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") != 1"; - - List resultNE = sql(sqlNE); - assertThat(resultNE).hasSize(2); - - List expectedNE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedNE, resultNE); - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownNoEqualNull() { - String sqlNotEqualNull = String.format("SELECT * FROM %s WHERE data <> NULL ", TABLE_NAME); - - List resultNE = sql(sqlNotEqualNull); - assertThat(resultNE).isEmpty(); - assertThat(lastScanEvent).as("Should not push down a filter").isNull(); - } - - @TestTemplate - public void testFilterPushDownAnd() { - String sqlAnd = - String.format("SELECT * FROM %s WHERE id = 1 AND data = 'iceberg' ", TABLE_NAME); - - List resultAnd = sql(sqlAnd); - assertThat(resultAnd).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - assertThat(scanEventCount).isEqualTo(1); - String expected = "(ref(name=\"id\") == 1 and ref(name=\"data\") == \"iceberg\")"; - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expected); - } - - @TestTemplate - public void testFilterPushDownOr() { - String sqlOr = String.format("SELECT * FROM %s WHERE id = 1 OR data = 'b' ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"data\") == \"b\")"; - - List resultOr = sql(sqlOr); - assertThat(resultOr).hasSize(2); - - List expectedOR = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedOR, resultOr); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownGreaterThan() { - String sqlGT = String.format("SELECT * FROM %s WHERE id > 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") > 1"; - - List resultGT = sql(sqlGT); - assertThat(resultGT).hasSize(2); - - List expectedGT = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedGT, resultGT); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownGreaterThanNull() { - String sqlGT = String.format("SELECT * FROM %s WHERE data > null ", TABLE_NAME); - - List resultGT = sql(sqlGT); - assertThat(resultGT).isEmpty(); - assertThat(lastScanEvent).as("Should not push down a filter").isNull(); - } - - @TestTemplate - public void testFilterPushDownGreaterThanLiteralOnLeft() { - String sqlGT = String.format("SELECT * FROM %s WHERE 3 > id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") < 3"; - - List resultGT = sql(sqlGT); - assertThat(resultGT).hasSize(2); - - List expectedGT = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedGT, resultGT); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownGreaterThanEqual() { - String sqlGTE = String.format("SELECT * FROM %s WHERE id >= 2 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") >= 2"; - - List resultGTE = sql(sqlGTE); - assertThat(resultGTE).hasSize(2); - - List expectedGTE = Lists.newArrayList(Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedGTE, resultGTE); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownGreaterThanEqualNull() { - String sqlGTE = String.format("SELECT * FROM %s WHERE data >= null ", TABLE_NAME); - - List resultGT = sql(sqlGTE); - assertThat(resultGT).isEmpty(); - assertThat(lastScanEvent).as("Should not push down a filter").isNull(); - } - - @TestTemplate - public void testFilterPushDownGreaterThanEqualLiteralOnLeft() { - String sqlGTE = String.format("SELECT * FROM %s WHERE 2 >= id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") <= 2"; - - List resultGTE = sql(sqlGTE); - assertThat(resultGTE).hasSize(2); - - List expectedGTE = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedGTE, resultGTE); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownLessThan() { - String sqlLT = String.format("SELECT * FROM %s WHERE id < 2 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") < 2"; - - List resultLT = sql(sqlLT); - assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownLessThanNull() { - String sqlLT = String.format("SELECT * FROM %s WHERE data < null ", TABLE_NAME); - - List resultGT = sql(sqlLT); - assertThat(resultGT).isEmpty(); - assertThat(lastScanEvent).as("Should not push down a filter").isNull(); - } - - @TestTemplate - public void testFilterPushDownLessThanLiteralOnLeft() { - String sqlLT = String.format("SELECT * FROM %s WHERE 2 < id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") > 2"; - - List resultLT = sql(sqlLT); - assertThat(resultLT).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownLessThanEqual() { - String sqlLTE = String.format("SELECT * FROM %s WHERE id <= 1 ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") <= 1"; - - List resultLTE = sql(sqlLTE); - assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownLessThanEqualNull() { - String sqlLTE = String.format("SELECT * FROM %s WHERE data <= null ", TABLE_NAME); - - List resultGT = sql(sqlLTE); - assertThat(resultGT).isEmpty(); - assertThat(lastScanEvent).as("Should not push down a filter").isNull(); - } - - @TestTemplate - public void testFilterPushDownLessThanEqualLiteralOnLeft() { - String sqlLTE = String.format("SELECT * FROM %s WHERE 3 <= id ", TABLE_NAME); - String expectedFilter = "ref(name=\"id\") >= 3"; - - List resultLTE = sql(sqlLTE); - assertThat(resultLTE).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownIn() { - String sqlIN = String.format("SELECT * FROM %s WHERE id IN (1,2) ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") == 1 or ref(name=\"id\") == 2)"; - List resultIN = sql(sqlIN); - assertThat(resultIN).hasSize(2); - - List expectedIN = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedIN, resultIN); - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownInNull() { - String sqlInNull = - String.format("SELECT * FROM %s WHERE data IN ('iceberg',NULL) ", TABLE_NAME); - - List result = sql(sqlInNull); - assertThat(result).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - - // In SQL, null check can only be done as IS NULL or IS NOT NULL, so it's correct to ignore it - // and push the rest down. - String expectedScan = "ref(name=\"data\") == \"iceberg\""; - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedScan); - } - - @TestTemplate - public void testFilterPushDownNotIn() { - String sqlNotIn = String.format("SELECT * FROM %s WHERE id NOT IN (3,2) ", TABLE_NAME); - - List resultNotIn = sql(sqlNotIn); - assertThat(resultNotIn).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - assertThat(scanEventCount).isEqualTo(1); - String expectedScan = "(ref(name=\"id\") != 2 and ref(name=\"id\") != 3)"; - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedScan); - } - - @TestTemplate - public void testFilterPushDownNotInNull() { - String sqlNotInNull = String.format("SELECT * FROM %s WHERE id NOT IN (1,2,NULL) ", TABLE_NAME); - List resultGT = sql(sqlNotInNull); - assertThat(resultGT).isEmpty(); - assertThat(lastScanEvent) - .as( - "As the predicate pushdown filter out all rows, Flink did not create scan plan, so it doesn't publish any ScanEvent.") - .isNull(); - } - - @TestTemplate - public void testFilterPushDownIsNotNull() { - String sqlNotNull = String.format("SELECT * FROM %s WHERE data IS NOT NULL", TABLE_NAME); - String expectedFilter = "not_null(ref(name=\"data\"))"; - - List resultNotNull = sql(sqlNotNull); - assertThat(resultNotNull).hasSize(2); - - List expected = Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expected, resultNotNull); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownIsNull() { - String sqlNull = String.format("SELECT * FROM %s WHERE data IS NULL", TABLE_NAME); - String expectedFilter = "is_null(ref(name=\"data\"))"; - - List resultNull = sql(sqlNull); - assertThat(resultNull).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownNot() { - String sqlNot = String.format("SELECT * FROM %s WHERE NOT (id = 1 OR id = 2 ) ", TABLE_NAME); - - List resultNot = sql(sqlNot); - assertThat(resultNot).hasSize(1).first().isEqualTo(Row.of(3, null, 30.0)); - - assertThat(scanEventCount).isEqualTo(1); - String expectedFilter = "(ref(name=\"id\") != 1 and ref(name=\"id\") != 2)"; - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownBetween() { - String sqlBetween = String.format("SELECT * FROM %s WHERE id BETWEEN 1 AND 2 ", TABLE_NAME); - - List resultBetween = sql(sqlBetween); - assertThat(resultBetween).hasSize(2); - - List expectedBetween = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedBetween, resultBetween); - - assertThat(scanEventCount).isEqualTo(1); - String expected = "(ref(name=\"id\") >= 1 and ref(name=\"id\") <= 2)"; - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expected); - } - - @TestTemplate - public void testFilterPushDownNotBetween() { - String sqlNotBetween = - String.format("SELECT * FROM %s WHERE id NOT BETWEEN 2 AND 3 ", TABLE_NAME); - String expectedFilter = "(ref(name=\"id\") < 2 or ref(name=\"id\") > 3)"; - - List resultNotBetween = sql(sqlNotBetween); - assertThat(resultNotBetween).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - } - - @TestTemplate - public void testFilterPushDownLike() { - String expectedFilter = "ref(name=\"data\") startsWith \"\"ice\"\""; - - String sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'ice%%' "; - List resultLike = sql(sqlLike); - assertThat(resultLike).hasSize(1).first().isEqualTo(Row.of(1, "iceberg", 10.0)); - assertThat(scanEventCount).isEqualTo(1); - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedFilter); - - // %% won't match the row with null value - sqlLike = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%' "; - resultLike = sql(sqlLike); - assertThat(resultLike).hasSize(2); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0)); - assertSameElements(expectedRecords, resultLike); - String expectedScan = "not_null(ref(name=\"data\"))"; - assertThat(lastScanEvent.filter()) - .as("Should contain the push down filter") - .asString() - .isEqualTo(expectedScan); - } - - @TestTemplate - public void testFilterNotPushDownLike() { - Row expectRecord = Row.of(1, "iceberg", 10.0); - String sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i' "; - List resultLike = sql(sqlNoPushDown); - assertThat(resultLike).isEmpty(); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%i%%' "; - resultLike = sql(sqlNoPushDown); - assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE '%%ice%%g' "; - resultLike = sql(sqlNoPushDown); - assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'iceber_' "; - resultLike = sql(sqlNoPushDown); - assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - - sqlNoPushDown = "SELECT * FROM " + TABLE_NAME + " WHERE data LIKE 'i%%g' "; - resultLike = sql(sqlNoPushDown); - assertThat(resultLike).hasSize(1).first().isEqualTo(expectRecord); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - } - - @TestTemplate - public void testFilterPushDown2Literal() { - String sql2Literal = String.format("SELECT * FROM %s WHERE 1 > 0 ", TABLE_NAME); - List result = sql(sql2Literal); - List expectedRecords = - Lists.newArrayList(Row.of(1, "iceberg", 10.0), Row.of(2, "b", 20.0), Row.of(3, null, 30.0)); - assertSameElements(expectedRecords, result); - assertThat(lastScanEvent.filter()) - .as("Should not push down a filter") - .isEqualTo(Expressions.alwaysTrue()); - } - - @TestTemplate - public void testSqlParseNaN() { - // todo add some test case to test NaN - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java deleted file mode 100644 index b7447d15c05a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ /dev/null @@ -1,147 +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.source; - -import static org.apache.iceberg.flink.SimpleDataUtil.SCHEMA; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -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.table.api.TableColumn; -import org.apache.flink.table.api.TableSchema; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -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.TestTemplate; - -public class TestIcebergSourceBounded extends TestFlinkScan { - @TestTemplate - public void testValidation() { - CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA); - - assertThatThrownBy( - () -> - IcebergSource.forRowData() - .tableLoader(tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .streaming(false) - .endTag("tag") - .endSnapshotId(1L) - .build()) - .hasMessage("END_SNAPSHOT_ID and END_TAG cannot both be set.") - .isInstanceOf(IllegalArgumentException.class); - } - - @Override - protected List runWithProjection(String... projected) throws Exception { - Schema icebergTableSchema = - CATALOG_EXTENSION.catalog().loadTable(TestFixtures.TABLE_IDENTIFIER).schema(); - TableSchema.Builder builder = TableSchema.builder(); - TableSchema schema = FlinkSchemaUtil.toSchema(FlinkSchemaUtil.convert(icebergTableSchema)); - for (String field : projected) { - TableColumn column = schema.getTableColumn(field).get(); - builder.field(column.getName(), column.getType()); - } - TableSchema flinkSchema = builder.build(); - Schema projectedSchema = FlinkSchemaUtil.convert(icebergTableSchema, flinkSchema); - return run(projectedSchema, Lists.newArrayList(), Maps.newHashMap(), "", projected); - } - - @Override - protected List runWithFilter(Expression filter, String sqlFilter, boolean caseSensitive) - throws Exception { - Map options = Maps.newHashMap(); - options.put("case-sensitive", Boolean.toString(caseSensitive)); - return run(null, Collections.singletonList(filter), options, sqlFilter, "*"); - } - - @Override - protected List runWithOptions(Map options) throws Exception { - return run(null, Lists.newArrayList(), options, "", "*"); - } - - @Override - protected List run() throws Exception { - return run(null, Lists.newArrayList(), Maps.newHashMap(), "", "*"); - } - - protected List run( - Schema projectedSchema, - List filters, - Map options, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - 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(); - } - - IcebergSource.Builder sourceBuilder = - IcebergSource.forRowData() - .tableLoader(tableLoader()) - .table(table) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.properties(options); - - DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) - .map( - new RowDataToRowMapper( - FlinkSchemaUtil.convert( - projectedSchema == null ? table.schema() : projectedSchema))); - - try (CloseableIterator iter = stream.executeAndCollect()) { - return Lists.newArrayList(iter); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java deleted file mode 100644 index 7bfed00a9eb4..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ /dev/null @@ -1,196 +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.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.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.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.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); - - @Parameters(name = "format={0}, parallelism = {1}") - public static Object[][] parameters() { - return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} - }; - } - - @Parameter(index = 0) - private FileFormat fileFormat; - - @Parameter(index = 1) - private int parallelism; - - @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 = - 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); - } - - @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(); - } - - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); - - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); - if (projectedSchema != null) { - sourceBuilder.project(projectedSchema); - } - - sourceBuilder.filters(filters); - sourceBuilder.setAll(options); - - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); - RowType rowType = FlinkSchemaUtil.convert(readSchema); - 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); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java deleted file mode 100644 index 0f41c5af4c95..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ /dev/null @@ -1,76 +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.source; - -import java.io.IOException; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.iceberg.Schema; -import org.apache.iceberg.expressions.Expression; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.jupiter.api.BeforeEach; - -public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { - private volatile TableEnvironment tEnv; - - @BeforeEach - public void before() throws IOException { - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); - SqlHelpers.sql( - getTableEnv(), - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_EXTENSION.warehouse()); - SqlHelpers.sql(getTableEnv(), "use catalog iceberg_catalog"); - getTableEnv() - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @Override - protected List run( - Schema projectedSchema, - List filters, - Map options, - String sqlFilter, - String... sqlSelectedFields) - throws Exception { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(options); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java deleted file mode 100644 index 9c7006e16b8e..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceContinuous.java +++ /dev/null @@ -1,538 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.nio.file.Path; -import java.time.Duration; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.JobStatus; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.client.program.ClusterClient; -import org.apache.flink.runtime.client.JobStatusMessage; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.InjectClusterClient; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.HadoopTableExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataToRowMapper; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public class TestIcebergSourceContinuous { - - @TempDir protected Path temporaryFolder; - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopTableExtension TABLE_EXTENSION = - new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - private final AtomicLong randomSeed = new AtomicLong(0L); - - @Test - public void testTableScanThenIncremental() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - // snapshot1 - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - TABLE_EXTENSION.table().currentSnapshot().snapshotId(); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - TABLE_EXTENSION.table().currentSnapshot().snapshotId(); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testTableScanThenIncrementalAfterExpiration() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - // snapshot1 - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshotId = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - TABLE_EXTENSION.table().expireSnapshots().expireSnapshotId(snapshotId).commit(); - - assertThat(TABLE_EXTENSION.table().history()).hasSize(1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - assertThat(FlinkSplitPlanner.checkScanMode(scanContext)) - .isEqualTo(FlinkSplitPlanner.ScanMode.BATCH); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 4); - List initialRecords = Lists.newArrayList(); - initialRecords.addAll(batch1); - initialRecords.addAll(batch2); - TestHelpers.assertRecords(result1, initialRecords, TABLE_EXTENSION.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - TABLE_EXTENSION.table().currentSnapshot().snapshotId(); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testEarliestSnapshot() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - // snapshot0 - List batch0 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - - // snapshot1 - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 4); - List combinedBatch0AndBatch1 = Lists.newArrayList(batch0); - combinedBatch0AndBatch1.addAll(batch1); - TestHelpers.assertRecords(result1, combinedBatch0AndBatch1, TABLE_EXTENSION.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testLatestSnapshot(@InjectClusterClient ClusterClient clusterClient) - throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - // snapshot0 - List batch0 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - - // snapshot1 - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - // we want to make sure job is running first so that enumerator can - // start from the latest snapshot before inserting the next batch2 below. - waitUntilJobIsRunning(clusterClient); - - // inclusive behavior for starting snapshot - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testSpecificSnapshotId() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - // snapshot0 - List batch0 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - long snapshot0 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); - - // snapshot1 - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshot1 = TABLE_EXTENSION.table().currentSnapshot().snapshotId(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot1) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testSpecificSnapshotTimestamp() throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - // snapshot0 - List batch0 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch0); - long snapshot0Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); - - // sleep for 2 ms to make sure snapshot1 has a higher timestamp value - Thread.sleep(2); - - // snapshot1 - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch1); - long snapshot1Timestamp = TABLE_EXTENSION.table().currentSnapshot().timestampMillis(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot1Timestamp) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - // consume data from snapshot1 - List result1 = waitForResult(iter, 2); - TestHelpers.assertRecords(result1, batch1, TABLE_EXTENSION.table().schema()); - - // snapshot2 - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch2); - - List result2 = waitForResult(iter, 2); - TestHelpers.assertRecords(result2, batch2, TABLE_EXTENSION.table().schema()); - - // snapshot3 - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testReadingFromBranch() throws Exception { - String branch = "b1"; - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder); - - List batchBase = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batchBase); - - // create branch - TABLE_EXTENSION - .table() - .manageSnapshots() - .createBranch(branch, TABLE_EXTENSION.table().currentSnapshot().snapshotId()) - .commit(); - - // snapshot1 to branch - List batch1 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch1); - - // snapshot2 to branch - List batch2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch2); - - List branchExpectedRecords = Lists.newArrayList(); - branchExpectedRecords.addAll(batchBase); - branchExpectedRecords.addAll(batch1); - branchExpectedRecords.addAll(batch2); - // reads from branch: it should contain the first snapshot (before the branch creation) followed - // by the next 2 snapshots added - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .useBranch(branch) - .build(); - - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List resultMain = waitForResult(iter, 6); - TestHelpers.assertRecords( - resultMain, branchExpectedRecords, TABLE_EXTENSION.table().schema()); - - // snapshot3 to branch - List batch3 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch3); - - List result3 = waitForResult(iter, 2); - TestHelpers.assertRecords(result3, batch3, TABLE_EXTENSION.table().schema()); - - // snapshot4 to branch - List batch4 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(branch, batch4); - - List result4 = waitForResult(iter, 2); - TestHelpers.assertRecords(result4, batch4, TABLE_EXTENSION.table().schema()); - } - - // read only from main branch. Should contain only the first snapshot - scanContext = - ScanContext.builder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10L)) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - try (CloseableIterator iter = - createStream(scanContext).executeAndCollect(getClass().getSimpleName())) { - List resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchBase, TABLE_EXTENSION.table().schema()); - - List batchMain2 = - RandomGenericData.generate( - TABLE_EXTENSION.table().schema(), 2, randomSeed.incrementAndGet()); - dataAppender.appendToTable(batchMain2); - resultMain = waitForResult(iter, 2); - TestHelpers.assertRecords(resultMain, batchMain2, TABLE_EXTENSION.table().schema()); - } - } - - @Test - public void testValidation() { - assertThatThrownBy( - () -> - IcebergSource.forRowData() - .tableLoader(TABLE_EXTENSION.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .streaming(true) - .endTag("tag") - .build()) - .hasMessage("Cannot set end-tag option for streaming reader") - .isInstanceOf(IllegalArgumentException.class); - } - - private DataStream createStream(ScanContext scanContext) throws Exception { - // start the source and collect output - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - DataStream stream = - env.fromSource( - IcebergSource.forRowData() - .tableLoader(TABLE_EXTENSION.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - .streaming(scanContext.isStreaming()) - .streamingStartingStrategy(scanContext.streamingStartingStrategy()) - .startSnapshotTimestamp(scanContext.startSnapshotTimestamp()) - .startSnapshotId(scanContext.startSnapshotId()) - .monitorInterval(Duration.ofMillis(10L)) - .branch(scanContext.branch()) - .build(), - WatermarkStrategy.noWatermarks(), - "icebergSource", - TypeInformation.of(RowData.class)) - .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(TABLE_EXTENSION.table().schema()))); - return stream; - } - - public static List waitForResult(CloseableIterator iter, int limit) { - List results = Lists.newArrayListWithCapacity(limit); - while (results.size() < limit) { - if (iter.hasNext()) { - results.add(iter.next()); - } else { - break; - } - } - return results; - } - - public static void waitUntilJobIsRunning(ClusterClient client) { - Awaitility.await("job should be running") - .atMost(Duration.ofSeconds(30)) - .pollInterval(Duration.ofMillis(10)) - .untilAsserted(() -> assertThat(getRunningJobs(client)).isNotEmpty()); - } - - public static List getRunningJobs(ClusterClient client) throws Exception { - Collection statusMessages = client.listJobs().get(); - return statusMessages.stream() - .filter(status -> status.getJobState() == JobStatus.RUNNING) - .map(JobStatusMessage::getJobId) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java deleted file mode 100644 index 938ae4d9bb0a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailover.java +++ /dev/null @@ -1,394 +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.source; - -import static org.apache.iceberg.flink.SimpleDataUtil.tableRecords; -import static org.assertj.core.api.Assertions.assertThat; - -import java.nio.file.Path; -import java.time.Duration; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.restartstrategy.RestartStrategies; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.client.program.ClusterClient; -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.highavailability.nonha.embedded.HaLeadershipControl; -import org.apache.flink.runtime.jobgraph.SavepointConfigOptions; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.InjectClusterClient; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.MiniClusterWithClientResource; -import org.apache.flink.util.function.ThrowingConsumer; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.sink.FlinkSink; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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; -import org.junit.jupiter.api.io.TempDir; - -@Timeout(value = 120) -public class TestIcebergSourceFailover { - - // Parallelism higher than 1, but lower than the number of splits used by some of our tests - // The goal is to allow some splits to remain in the enumerator when restoring the state - private static final int PARALLELISM = 2; - private static final int DO_NOT_FAIL = Integer.MAX_VALUE; - protected static final MiniClusterResourceConfiguration MINI_CLUSTER_RESOURCE_CONFIG = - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .withHaLeadershipControl() - .build(); - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - new MiniClusterExtension(MINI_CLUSTER_RESOURCE_CONFIG); - - @TempDir protected Path temporaryFolder; - - @RegisterExtension - protected static final HadoopCatalogExtension SOURCE_CATALOG_EXTENSION = - new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - - @RegisterExtension - protected static final HadoopCatalogExtension SINK_CATALOG_EXTENSION = - new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.SINK_TABLE); - - protected Table sourceTable; - protected Table sinkTable; - - @BeforeEach - protected void setupTable() { - this.sourceTable = - SOURCE_CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); - this.sinkTable = - SINK_CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.SCHEMA); - } - - protected IcebergSource.Builder sourceBuilder() { - Configuration config = new Configuration(); - return IcebergSource.forRowData() - .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) - .assignerFactory(new SimpleSplitAssignerFactory()) - // Prevent combining splits - .set( - FlinkReadOptions.SPLIT_FILE_OPEN_COST, - Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) - .flinkConfig(config); - } - - protected Schema schema() { - return TestFixtures.SCHEMA; - } - - protected List generateRecords(int numRecords, long seed) { - return RandomGenericData.generate(schema(), numRecords, seed); - } - - protected void assertRecords(Table table, List expectedRecords, Duration timeout) - throws Exception { - SimpleDataUtil.assertTableRecords(table, expectedRecords, timeout); - } - - @Disabled("Disabled for now as it is flaky on CI") - @Test - public void testBoundedWithSavepoint(@InjectClusterClient ClusterClient clusterClient) - throws Exception { - List expectedRecords = Lists.newArrayList(); - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); - for (int i = 0; i < 4; ++i) { - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - } - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - createBoundedStreams(env, 2); - - JobClient jobClient = env.executeAsync("Bounded Iceberg Source Savepoint Test"); - JobID jobId = jobClient.getJobID(); - - // Write something, but do not finish before checkpoint is created - RecordCounterToWait.waitForCondition(); - CompletableFuture savepoint = - clusterClient.stopWithSavepoint( - jobId, false, temporaryFolder.toString(), SavepointFormatType.CANONICAL); - RecordCounterToWait.continueProcessing(); - - // Wait for the job to stop with the savepoint - String savepointPath = savepoint.get(); - - // We expect that at least a few records has written - assertThat(tableRecords(sinkTable)).hasSizeGreaterThan(0); - - // New env from the savepoint - Configuration conf = new Configuration(); - conf.set(SavepointConfigOptions.SAVEPOINT_PATH, savepointPath); - env = StreamExecutionEnvironment.getExecutionEnvironment(conf); - createBoundedStreams(env, DO_NOT_FAIL); - - env.execute("Bounded Iceberg Source Savepoint Test"); - - // We expect no duplications - assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); - } - - @Test - public void testBoundedWithTaskManagerFailover() throws Exception { - runTestWithNewMiniCluster( - miniCluster -> testBoundedIcebergSource(FailoverType.TM, miniCluster)); - } - - @Test - public void testBoundedWithJobManagerFailover() throws Exception { - runTestWithNewMiniCluster( - miniCluster -> testBoundedIcebergSource(FailoverType.JM, miniCluster)); - } - - private void testBoundedIcebergSource(FailoverType failoverType, MiniCluster miniCluster) - throws Exception { - List expectedRecords = Lists.newArrayList(); - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); - for (int i = 0; i < 4; ++i) { - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - } - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setRestartStrategy(RestartStrategies.fixedDelayRestart(1, 0)); - createBoundedStreams(env, 2); - - JobClient jobClient = env.executeAsync("Bounded Iceberg Source Failover Test"); - JobID jobId = jobClient.getJobID(); - - RecordCounterToWait.waitForCondition(); - triggerFailover(failoverType, jobId, RecordCounterToWait::continueProcessing, miniCluster); - - assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); - } - - @Test - public void testContinuousWithTaskManagerFailover() throws Exception { - runTestWithNewMiniCluster( - miniCluster -> testContinuousIcebergSource(FailoverType.TM, miniCluster)); - } - - @Test - public void testContinuousWithJobManagerFailover() throws Exception { - runTestWithNewMiniCluster( - miniCluster -> testContinuousIcebergSource(FailoverType.JM, miniCluster)); - } - - private void testContinuousIcebergSource(FailoverType failoverType, MiniCluster miniCluster) - throws Exception { - GenericAppenderHelper dataAppender = - new GenericAppenderHelper(sourceTable, FileFormat.PARQUET, temporaryFolder); - List expectedRecords = Lists.newArrayList(); - - List batch = generateRecords(2, 0); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(PARALLELISM); - env.enableCheckpointing(10L); - Configuration config = new Configuration(); - config.setInteger(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 128); - - DataStream stream = - env.fromSource( - sourceBuilder() - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(stream) - .table(sinkTable) - .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) - .append(); - - JobClient jobClient = env.executeAsync("Continuous Iceberg Source Failover Test"); - JobID jobId = jobClient.getJobID(); - - for (int i = 1; i < 5; i++) { - Thread.sleep(10); - List records = generateRecords(2, i); - expectedRecords.addAll(records); - dataAppender.appendToTable(records); - if (i == 2) { - triggerFailover(failoverType, jobId, () -> {}, miniCluster); - } - } - - // wait longer for continuous source to reduce flakiness - // because CI servers tend to be overloaded. - assertRecords(sinkTable, expectedRecords, Duration.ofSeconds(120)); - } - - private void createBoundedStreams(StreamExecutionEnvironment env, int failAfter) { - env.setParallelism(PARALLELISM); - - DataStream stream = - env.fromSource( - sourceBuilder().build(), - WatermarkStrategy.noWatermarks(), - "IcebergSource", - TypeInformation.of(RowData.class)); - - DataStream streamFailingInTheMiddleOfReading = - RecordCounterToWait.wrapWithFailureAfter(stream, failAfter); - - // CollectStreamSink from DataStream#executeAndCollect() doesn't guarantee - // exactly-once behavior. When Iceberg sink, we can verify end-to-end - // exactly-once. Here we mainly about source exactly-once behavior. - FlinkSink.forRowData(streamFailingInTheMiddleOfReading) - .table(sinkTable) - .tableLoader(SINK_CATALOG_EXTENSION.tableLoader()) - .append(); - } - - // ------------------------------------------------------------------------ - // test utilities copied from Flink's FileSourceTextLinesITCase - // ------------------------------------------------------------------------ - - private static void runTestWithNewMiniCluster(ThrowingConsumer testMethod) - throws Exception { - MiniClusterWithClientResource miniCluster = null; - try { - miniCluster = new MiniClusterWithClientResource(MINI_CLUSTER_RESOURCE_CONFIG); - miniCluster.before(); - testMethod.accept(miniCluster.getMiniCluster()); - } finally { - if (miniCluster != null) { - miniCluster.after(); - } - } - } - - private enum FailoverType { - NONE, - TM, - JM - } - - private static void triggerFailover( - FailoverType type, JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) - throws Exception { - switch (type) { - case NONE: - afterFailAction.run(); - break; - case TM: - restartTaskManager(afterFailAction, miniCluster); - break; - case JM: - triggerJobManagerFailover(jobId, afterFailAction, miniCluster); - break; - } - } - - private static void triggerJobManagerFailover( - JobID jobId, Runnable afterFailAction, MiniCluster miniCluster) throws Exception { - HaLeadershipControl haLeadershipControl = miniCluster.getHaLeadershipControl().get(); - haLeadershipControl.revokeJobMasterLeadership(jobId).get(); - afterFailAction.run(); - haLeadershipControl.grantJobMasterLeadership(jobId).get(); - } - - private static void restartTaskManager(Runnable afterFailAction, MiniCluster miniCluster) - throws Exception { - miniCluster.terminateTaskManager(0).get(); - afterFailAction.run(); - miniCluster.startTaskManager(); - } - - private static class RecordCounterToWait { - - private static AtomicInteger records; - private static CountDownLatch countDownLatch; - private static CompletableFuture continueProcessing; - - private static DataStream wrapWithFailureAfter(DataStream stream, int condition) { - - records = new AtomicInteger(); - continueProcessing = new CompletableFuture<>(); - countDownLatch = new CountDownLatch(stream.getParallelism()); - return stream.map( - record -> { - boolean reachedFailPoint = records.incrementAndGet() > condition; - boolean notFailedYet = countDownLatch.getCount() != 0; - if (notFailedYet && reachedFailPoint) { - countDownLatch.countDown(); - continueProcessing.get(); - } - return record; - }); - } - - private static void waitForCondition() throws InterruptedException { - countDownLatch.await(); - } - - private static void continueProcessing() { - continueProcessing.complete(null); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java deleted file mode 100644 index 4f61d2f7308a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceFailoverWithWatermarkExtractor.java +++ /dev/null @@ -1,130 +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.source; - -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.time.ZoneOffset; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkReadOptions; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.types.Comparators; -import org.apache.iceberg.util.StructLikeWrapper; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.BeforeEach; - -public class TestIcebergSourceFailoverWithWatermarkExtractor extends TestIcebergSourceFailover { - // Increment ts by 15 minutes for each generateRecords batch - private static final long RECORD_BATCH_TS_INCREMENT_MILLI = TimeUnit.MINUTES.toMillis(15); - // Within a batch, increment ts by 1 second - private static final long RECORD_TS_INCREMENT_MILLI = TimeUnit.SECONDS.toMillis(1); - - private final AtomicLong tsMilli = new AtomicLong(System.currentTimeMillis()); - - @Override - @BeforeEach - protected void setupTable() { - this.sourceTable = - SOURCE_CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); - this.sinkTable = - SINK_CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.SINK_TABLE_IDENTIFIER, TestFixtures.TS_SCHEMA); - } - - @Override - protected IcebergSource.Builder sourceBuilder() { - Configuration config = new Configuration(); - return IcebergSource.forRowData() - .tableLoader(SOURCE_CATALOG_EXTENSION.tableLoader()) - .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA) - // Prevent combining splits - .set( - FlinkReadOptions.SPLIT_FILE_OPEN_COST, - Long.toString(TableProperties.SPLIT_SIZE_DEFAULT)) - .flinkConfig(config); - } - - @Override - protected Schema schema() { - return TestFixtures.TS_SCHEMA; - } - - @Override - protected List generateRecords(int numRecords, long seed) { - // Override the ts field to create a more realistic situation for event time alignment - tsMilli.addAndGet(RECORD_BATCH_TS_INCREMENT_MILLI); - return RandomGenericData.generate(schema(), numRecords, seed).stream() - .peek( - record -> { - LocalDateTime ts = - LocalDateTime.ofInstant( - Instant.ofEpochMilli(tsMilli.addAndGet(RECORD_TS_INCREMENT_MILLI)), - ZoneId.of("Z")); - record.setField("ts", ts); - }) - .collect(Collectors.toList()); - } - - /** - * This override is needed because {@link Comparators} used by {@link StructLikeWrapper} retrieves - * Timestamp type using Long type as inner class, while the {@link RandomGenericData} generates - * {@link LocalDateTime} for {@code TimestampType.withoutZone()}. This method normalizes the - * {@link LocalDateTime} to a Long type so that Comparators can continue to work. - */ - @Override - protected void assertRecords(Table table, List expectedRecords, Duration timeout) - throws Exception { - List expectedNormalized = convertLocalDateTimeToMilli(expectedRecords); - Awaitility.await("expected list of records should be produced") - .atMost(timeout) - .untilAsserted( - () -> - SimpleDataUtil.assertRecordsEqual( - expectedNormalized, - convertLocalDateTimeToMilli(SimpleDataUtil.tableRecords(table)), - table.schema())); - } - - private List convertLocalDateTimeToMilli(List records) { - return records.stream() - .peek( - r -> { - LocalDateTime localDateTime = ((LocalDateTime) r.getField("ts")); - r.setField("ts", localDateTime.atZone(ZoneOffset.UTC).toInstant().toEpochMilli()); - }) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java deleted file mode 100644 index df148c212ebd..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceReaderDeletes.java +++ /dev/null @@ -1,102 +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.source; - -import java.io.IOException; -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.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.data.RowData; -import org.apache.flink.table.types.logical.RowType; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.util.CloseableIterator; -import org.apache.hadoop.hive.conf.HiveConf; -import org.apache.iceberg.CatalogProperties; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.CatalogLoader; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.RowDataWrapper; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.source.assigner.SimpleSplitAssignerFactory; -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.StructLikeSet; -import org.junit.jupiter.api.extension.RegisterExtension; - -public class TestIcebergSourceReaderDeletes extends TestFlinkReaderDeletesBase { - - private static final int PARALLELISM = 4; - - @RegisterExtension - private static final MiniClusterExtension MINI_CLUSTER = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @Override - protected StructLikeSet rowSet(String tableName, Table testTable, String... columns) - throws IOException { - Schema projected = testTable.schema().select(columns); - RowType rowType = FlinkSchemaUtil.convert(projected); - - Map properties = Maps.newHashMap(); - properties.put( - CatalogProperties.WAREHOUSE_LOCATION, - hiveConf.get(HiveConf.ConfVars.METASTOREWAREHOUSE.varname)); - properties.put(CatalogProperties.URI, hiveConf.get(HiveConf.ConfVars.METASTOREURIS.varname)); - properties.put( - CatalogProperties.CLIENT_POOL_SIZE, - Integer.toString(hiveConf.getInt("iceberg.hive.client-pool-size", 5))); - CatalogLoader hiveCatalogLoader = CatalogLoader.hive(catalog.name(), hiveConf, properties); - TableLoader hiveTableLoader = - TableLoader.fromCatalog(hiveCatalogLoader, TableIdentifier.of("default", tableName)); - hiveTableLoader.open(); - try (TableLoader tableLoader = hiveTableLoader) { - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - DataStream stream = - env.fromSource( - IcebergSource.builder() - .tableLoader(tableLoader) - .assignerFactory(new SimpleSplitAssignerFactory()) - .project(projected) - .build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)); - - try (CloseableIterator iter = stream.executeAndCollect()) { - List rowDataList = Lists.newArrayList(iter); - StructLikeSet set = StructLikeSet.create(projected.asStruct()); - rowDataList.forEach( - rowData -> { - RowDataWrapper wrapper = new RowDataWrapper(rowType, projected.asStruct()); - set.add(wrapper.wrap(rowData)); - }); - return set; - } catch (Exception e) { - throw new IOException("Failed to collect result", e); - } - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.java deleted file mode 100644 index 75f0a785a8c5..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceSql.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.flink.source; - -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.IOException; -import java.time.Instant; -import java.time.ZoneId; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.Table; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -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.Test; - -/** Use the IcebergSource (FLIP-27) */ -public class TestIcebergSourceSql extends TestSqlBase { - private static final Schema SCHEMA_TS = - new Schema( - required(1, "t1", Types.TimestampType.withoutZone()), - required(2, "t2", Types.LongType.get())); - - @Override - public void before() throws IOException { - TableEnvironment tableEnvironment = getTableEnv(); - Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); - - tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); - SqlHelpers.sql( - tableEnvironment, - "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_EXTENSION.warehouse()); - SqlHelpers.sql(tableEnvironment, "use catalog iceberg_catalog"); - - tableConf.set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - } - - private Record generateRecord(Instant t1, long t2) { - Record record = GenericRecord.create(SCHEMA_TS); - record.setField("t1", t1.atZone(ZoneId.systemDefault()).toLocalDateTime()); - record.setField("t2", t2); - return record; - } - - /** Generates the records in the expected order, with respect to their datafile */ - private List generateExpectedRecords(boolean ascending) throws Exception { - Table table = CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, SCHEMA_TS); - long baseTime = 1702382109000L; - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - - Record file1Record1 = - generateRecord(Instant.ofEpochMilli(baseTime), baseTime + (1000 * 60 * 60 * 24 * 30L)); - Record file1Record2 = - generateRecord( - Instant.ofEpochMilli(baseTime - 10 * 1000L), baseTime + (1000 * 60 * 60 * 24 * 35L)); - - List recordsDataFile1 = Lists.newArrayList(); - recordsDataFile1.add(file1Record1); - recordsDataFile1.add(file1Record2); - DataFile dataFile1 = helper.writeFile(recordsDataFile1); - - Record file2Record1 = - generateRecord( - Instant.ofEpochMilli(baseTime + 14 * 1000L), baseTime - (1000 * 60 * 60 * 24 * 30L)); - Record file2Record2 = - generateRecord( - Instant.ofEpochMilli(baseTime + 12 * 1000L), baseTime - (1000 * 60 * 61 * 24 * 35L)); - - List recordsDataFile2 = Lists.newArrayList(); - recordsDataFile2.add(file2Record1); - recordsDataFile2.add(file2Record2); - - DataFile dataFile2 = helper.writeFile(recordsDataFile2); - helper.appendToTable(dataFile1, dataFile2); - - // Expected records if the splits are ordered - // - ascending (watermark from t1) - records from the split with early timestamps, then - // records from the split with late timestamps - // - descending (watermark from t2) - records from the split with old longs, then records - // from the split with new longs - List expected = Lists.newArrayList(); - if (ascending) { - expected.addAll(recordsDataFile1); - expected.addAll(recordsDataFile2); - } else { - expected.addAll(recordsDataFile2); - expected.addAll(recordsDataFile1); - } - return expected; - } - - /** Tests the order of splits returned when setting the watermark-column options */ - @Test - public void testWatermarkOptionsAscending() throws Exception { - List expected = generateExpectedRecords(true); - TestHelpers.assertRecordsWithOrder( - run( - ImmutableMap.of("watermark-column", "t1", "split-file-open-cost", "128000000"), - "", - "*"), - expected, - SCHEMA_TS); - } - - /** - * Tests the order of splits returned when setting the watermark-column and - * watermark-column-time-unit" options - */ - @Test - public void testWatermarkOptionsDescending() throws Exception { - List expected = generateExpectedRecords(false); - TestHelpers.assertRecordsWithOrder( - run( - ImmutableMap.of( - "watermark-column", - "t2", - "watermark-column-time-unit", - "MILLISECONDS", - "split-file-open-cost", - "128000000"), - "", - "*"), - expected, - SCHEMA_TS); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java deleted file mode 100644 index 70889f4f76aa..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceWithWatermarkExtractor.java +++ /dev/null @@ -1,408 +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.source; - -import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; - -import java.io.Serializable; -import java.nio.file.Path; -import java.time.Duration; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneId; -import java.util.List; -import java.util.Optional; -import java.util.concurrent.ConcurrentMap; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicInteger; -import org.apache.flink.api.common.JobID; -import org.apache.flink.api.common.eventtime.SerializableTimestampAssigner; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.runtime.metrics.MetricNames; -import org.apache.flink.runtime.minicluster.MiniCluster; -import org.apache.flink.runtime.minicluster.RpcServiceSharing; -import org.apache.flink.runtime.testutils.CommonTestUtils; -import org.apache.flink.runtime.testutils.InMemoryReporter; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.streaming.api.functions.windowing.AllWindowFunction; -import org.apache.flink.streaming.api.windowing.assigners.TumblingEventTimeWindows; -import org.apache.flink.streaming.api.windowing.time.Time; -import org.apache.flink.streaming.api.windowing.windows.TimeWindow; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.InjectMiniCluster; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.util.CloseableIterator; -import org.apache.flink.util.Collector; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableExtension; -import org.apache.iceberg.flink.TestFixtures; -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.awaitility.Awaitility; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public class TestIcebergSourceWithWatermarkExtractor implements Serializable { - private static final int PARALLELISM = 4; - private static final String SOURCE_NAME = "IcebergSource"; - private static final int RECORD_NUM_FOR_2_SPLITS = 200; - private static final ConcurrentMap WINDOWS = Maps.newConcurrentMap(); - - @TempDir protected Path temporaryFolder; - - private static final InMemoryReporter REPORTER = InMemoryReporter.createWithRetainedMetrics(); - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(1) - .setNumberSlotsPerTaskManager(PARALLELISM) - .setRpcServiceSharing(RpcServiceSharing.DEDICATED) - .setConfiguration(REPORTER.addToConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG)) - .withHaLeadershipControl() - .build()); - - @RegisterExtension - private static final HadoopTableExtension TABLE_EXTENSION = - new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.TS_SCHEMA); - - /** - * This is an integration test for watermark handling and windowing. Integration testing the - * following features: - * - *
      - *
    • - Ordering of the splits - *
    • - Emitting of watermarks - *
    • - Firing windows based on watermarks - *
    - * - *

    The test generates 4 splits - * - *

      - *
    • - Split 1 - Watermark 100 min - *
    • - Split 2, 3 - Watermark 0 min - *
    • - Split 4 - Watermark 6 min - *
    - * - *

    Creates a source with 5 minutes tumbling window with parallelism 1 (to prevent concurrency - * issues). - * - *

    Checks that windows are handled correctly based on the emitted watermarks, and splits are - * read in the following order: - * - *

      - *
    • - Split 2, 3 - *
    • - Split 4 - *
    • - Split 1 - *
    - * - *

    As a result the window aggregator emits the records based on in Split 2-3, and Split 4 data. - * - *

    Add 2 more splits, so the task manager close the windows for the original 4 splits and emit - * the appropriate aggregated records. - */ - @Test - public void testWindowing() throws Exception { - GenericAppenderHelper dataAppender = appender(); - List expectedRecords = Lists.newArrayList(); - - // Generate records with the following pattern: - // - File 1 - Later records (Watermark 6000000) - // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") - // - File 2 - First records (Watermark 0) - // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - File 3 - Parallel write for the first records (Watermark 360000) - // - Split 1 - 2 records (6, "file_3-recordTs_6"), (7, "file_3-recordTs_7") - List batch = - ImmutableList.of( - generateRecord(100, "file_1-recordTs_100"), - generateRecord(101, "file_1-recordTs_101"), - generateRecord(103, "file_1-recordTs_103")); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - batch = Lists.newArrayListWithCapacity(100); - for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - // Generate records where the timestamps are out of order, but still between 0-5 minutes - batch.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); - } - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - batch = - ImmutableList.of( - generateRecord(6, "file_3-recordTs_6"), generateRecord(7, "file_3-recordTs_7")); - expectedRecords.addAll(batch); - dataAppender.appendToTable(batch); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(1); - - DataStream stream = - env.fromSource( - source(), - WatermarkStrategy.noWatermarks() - .withTimestampAssigner(new RowDataTimestampAssigner()), - SOURCE_NAME, - TypeInformation.of(RowData.class)); - - stream - .windowAll(TumblingEventTimeWindows.of(Time.minutes(5))) - .apply( - new AllWindowFunction() { - @Override - public void apply( - TimeWindow window, Iterable values, Collector out) { - // Emit RowData which contains the window start time, and the record count in - // that window - AtomicInteger count = new AtomicInteger(0); - values.forEach(a -> count.incrementAndGet()); - out.collect(row(window.getStart(), count.get())); - WINDOWS.put(window.getStart(), count.get()); - } - }); - - // Use static variable to collect the windows, since other solutions were flaky - WINDOWS.clear(); - env.executeAsync("Iceberg Source Windowing Test"); - - // Wait for the 2 first windows from File 2 and File 3 - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - WINDOWS.equals( - ImmutableMap.of(0L, RECORD_NUM_FOR_2_SPLITS, TimeUnit.MINUTES.toMillis(5), 2))); - - // Write data so the windows containing test data are closed - dataAppender.appendToTable( - dataAppender.writeFile(ImmutableList.of(generateRecord(1500, "last-record")))); - - // Wait for last test record window from File 1 - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - WINDOWS.equals( - ImmutableMap.of( - 0L, - RECORD_NUM_FOR_2_SPLITS, - TimeUnit.MINUTES.toMillis(5), - 2, - TimeUnit.MINUTES.toMillis(100), - 3))); - } - - /** - * This is an integration test for watermark handling and throttling. Integration testing the - * following: - * - *

      - *
    • - Emitting of watermarks - *
    • - Watermark alignment - *
    - * - *

    The test generates 3 splits - * - *

      - *
    • - Split 1 - Watermark 100 min - *
    • - Split 2, 3 - Watermark 0 min - *
    - * - * The splits are read in the following order: - * - *
      - *
    • - Split 2, 3 (Task Manager 1, Task Manager 2) - *
    • - Split 1 (Task Manager 1 or ask Manager 2 depending on scheduling) - *
    - * - * Reading split 1 will cause the watermark alignment to pause reading for the given task manager. - * - *

    The status of the watermark alignment is checked by the alignment related metrics. - * - *

    Adding new records with old timestamps to the table will enable the running reader to - * continue reading the files, but the watermark alignment will still prevent the paused reader to - * continue. - * - *

    After adding some records with new timestamps the blocked reader is un-paused, and both ot - * the readers continue reading. - */ - @Test - public void testThrottling(@InjectMiniCluster MiniCluster miniCluster) throws Exception { - GenericAppenderHelper dataAppender = appender(); - - // Generate records in advance - - // File 1 - Later records (Watermark 6.000.000 - 100 min) - // - Split 1 - 2 records (100, "file_1-recordTs_100"), (103, "file_1-recordTs_103") - List batch1 = - ImmutableList.of( - generateRecord(100, "file_1-recordTs_100"), generateRecord(103, "file_1-recordTs_103")); - - // File 2 - First records (Watermark 0 - 0 min) - // - Split 1 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - // - Split 2 - 100 records (0, "file_2-recordTs_0"), (1, "file_2-recordTs_1"),... - List batch2 = Lists.newArrayListWithCapacity(100); - for (int i = 0; i < RECORD_NUM_FOR_2_SPLITS; ++i) { - batch2.add(generateRecord(4 - i % 5, "file_2-recordTs_" + i)); - } - - // File 3 - Some records will be blocked (Watermark 900.000 - 15 min) - List batch3 = - ImmutableList.of( - generateRecord(15, "file_3-recordTs_15"), - generateRecord(16, "file_3-recordTs_16"), - generateRecord(17, "file_3-recordTs_17")); - - // File 4 - Some records will be blocked (Watermark 900.000 - 15 min) - List batch4 = - ImmutableList.of( - generateRecord(15, "file_4-recordTs_15"), - generateRecord(16, "file_4-recordTs_16"), - generateRecord(17, "file_4-recordTs_17")); - - // File 5 - Records which will remove the block (Watermark 5.400.000 - 90 min) - List batch5 = - ImmutableList.of( - generateRecord(90, "file_5-recordTs_90"), generateRecord(91, "file_5-recordTs_91")); - - StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); - env.setParallelism(2); - - DataStream stream = - env.fromSource( - source(), - WatermarkStrategy.noWatermarks() - .withWatermarkAlignment("iceberg", Duration.ofMinutes(20), Duration.ofMillis(10)), - SOURCE_NAME, - TypeInformation.of(RowData.class)); - - try (CloseableIterator resultIterator = stream.collectAsync()) { - JobClient jobClient = env.executeAsync("Iceberg Source Throttling Test"); - CommonTestUtils.waitForAllTaskRunning(miniCluster, jobClient.getJobID(), false); - - // Insert the first data into the table - dataAppender.appendToTable(dataAppender.writeFile(batch1), dataAppender.writeFile(batch2)); - - // Get the drift metric, wait for it to be created and reach the expected state - // (100 min - 20 min - 0 min) - // Also this validates that the WatermarkAlignment is working - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until( - () -> - findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)) - .isPresent()); - Gauge drift = - findAlignmentDriftMetric(jobClient.getJobID(), TimeUnit.MINUTES.toMillis(80)).get(); - - // Add some old records with 2 splits, so even if the blocked gets one split, the other reader - // one gets one as well - dataAppender.appendToTable(dataAppender.writeFile(batch3), dataAppender.writeFile(batch4)); - - // Get the drift metric, wait for it to be created and reach the expected state (100 min - 20 - // min - 15 min) - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until(() -> drift.getValue() == TimeUnit.MINUTES.toMillis(65)); - - // Add some new records which should unblock the throttled reader - dataAppender.appendToTable(batch5); - - // Wait for the new drift to decrease below the allowed drift to signal the normal state - Awaitility.await() - .pollInterval(Duration.ofMillis(10)) - .atMost(30, TimeUnit.SECONDS) - .until(() -> drift.getValue() < TimeUnit.MINUTES.toMillis(20)); - } - } - - protected IcebergSource source() { - return IcebergSource.builder() - .tableLoader(TABLE_EXTENSION.tableLoader()) - .watermarkColumn("ts") - .project(TestFixtures.TS_SCHEMA) - .splitSize(100L) - .streaming(true) - .monitorInterval(Duration.ofMillis(10)) - .streamingStartingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - } - - protected Record generateRecord(int minutes, String str) { - // Override the ts field to create a more realistic situation for event time alignment - Record record = GenericRecord.create(TestFixtures.TS_SCHEMA); - LocalDateTime ts = - LocalDateTime.ofInstant( - Instant.ofEpochMilli(Time.of(minutes, TimeUnit.MINUTES).toMilliseconds()), - ZoneId.of("Z")); - record.setField("ts", ts); - record.setField("str", str); - return record; - } - - private Optional> findAlignmentDriftMetric(JobID jobID, long withValue) { - String metricsName = SOURCE_NAME + ".*" + MetricNames.WATERMARK_ALIGNMENT_DRIFT; - return REPORTER.findMetrics(jobID, metricsName).values().stream() - .map(m -> (Gauge) m) - .filter(m -> m.getValue() == withValue) - .findFirst(); - } - - private GenericAppenderHelper appender() { - // We need to create multiple splits, so we need to generate parquet files with multiple offsets - org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); - hadoopConf.set("write.parquet.page-size-bytes", "64"); - hadoopConf.set("write.parquet.row-group-size-bytes", "64"); - return new GenericAppenderHelper( - TABLE_EXTENSION.table(), FileFormat.PARQUET, temporaryFolder, hadoopConf); - } - - private static RowData row(long time, long count) { - GenericRowData result = new GenericRowData(2); - result.setField(0, time); - result.setField(1, String.valueOf(count)); - return result; - } - - private static class RowDataTimestampAssigner implements SerializableTimestampAssigner { - @Override - public long extractTimestamp(RowData element, long recordTimestamp) { - return element.getTimestamp(0, 0).getMillisecond(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java deleted file mode 100644 index 95d0b90b6ca9..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ /dev/null @@ -1,184 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.nio.file.Files; -import java.time.Duration; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.common.RuntimeExecutionMode; -import org.apache.flink.api.common.functions.RichMapFunction; -import org.apache.flink.api.common.typeinfo.Types; -import org.apache.flink.configuration.BatchExecutionOptions; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.configuration.JobManagerOptions; -import org.apache.flink.configuration.RestOptions; -import org.apache.flink.configuration.SlowTaskDetectorOptions; -import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; -import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -import org.apache.flink.table.api.Table; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestBase; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; - -public class TestIcebergSpeculativeExecutionSupport extends TestBase { - private static final int NUM_TASK_MANAGERS = 1; - private static final int NUM_TASK_SLOTS = 3; - - @RegisterExtension - public static MiniClusterExtension miniClusterResource = - new MiniClusterExtension( - new MiniClusterResourceConfiguration.Builder() - .setNumberTaskManagers(NUM_TASK_MANAGERS) - .setNumberSlotsPerTaskManager(NUM_TASK_SLOTS) - .setConfiguration(configure()) - .build()); - - private StreamTableEnvironment tEnv; - private static final String CATALOG_NAME = "test_catalog"; - private static final String DATABASE_NAME = "test_db"; - private static final String INPUT_TABLE_NAME = "test_table"; - private static final String OUTPUT_TABLE_NAME = "sink_table"; - - @Override - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment(configure()); - env.setRuntimeMode(RuntimeExecutionMode.BATCH); - tEnv = StreamTableEnvironment.create(env); - } - } - - return tEnv; - } - - @BeforeEach - public void before() throws IOException { - String warehouse = - String.format("file:%s", Files.createTempDirectory(temporaryDirectory, "junit").toString()); - sql( - "CREATE CATALOG %s WITH ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", - CATALOG_NAME, warehouse); - sql("USE CATALOG %s", CATALOG_NAME); - sql("CREATE DATABASE %s", DATABASE_NAME); - sql("USE %s", DATABASE_NAME); - - sql("CREATE TABLE %s (i INT, j INT)", INPUT_TABLE_NAME); - sql("INSERT INTO %s VALUES (1, -1),(2, -1),(3, -1)", INPUT_TABLE_NAME); - sql("CREATE TABLE %s (i INT, j INT, subTask INT, attempt INT)", OUTPUT_TABLE_NAME); - } - - @AfterEach - public void after() { - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); - sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); - sql("DROP DATABASE %s", DATABASE_NAME); - dropCatalog(CATALOG_NAME, true); - } - - @Test - public void testSpeculativeExecution() throws Exception { - Table table = - tEnv.sqlQuery(String.format("SELECT * FROM %s.%s", DATABASE_NAME, INPUT_TABLE_NAME)); - DataStream slowStream = - tEnv.toDataStream(table, Row.class) - .map(new TestingMap()) - .name("test_map") - .returns( - Types.ROW_NAMED( - new String[] {"i", "j", "subTask", "attempt"}, - Types.INT, - Types.INT, - Types.INT, - Types.INT)) - .setParallelism(NUM_TASK_SLOTS); - - tEnv.fromDataStream(slowStream) - .executeInsert(String.format("%s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)) - .await(); - - List output = sql(String.format("SELECT * FROM %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME)); - - // Ensure that all subTasks has attemptNum > 0 - assertThat(output.stream().map(x -> x.getField(3)).collect(Collectors.toSet())).contains(1); - - // Ensure the test_table rows are returned exactly the same after the slow map task from the - // sink_table - assertSameElements( - output.stream().map(x -> Row.of(x.getField(0), x.getField(1))).collect(Collectors.toList()), - Arrays.asList(Row.of(1, -1), Row.of(2, -1), Row.of(3, -1))); - } - - /** A testing map function that simulates the slow task. */ - private static class TestingMap extends RichMapFunction { - @Override - public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getAttemptNumber() <= 0) { - Thread.sleep(Integer.MAX_VALUE); - } - - Row output = - Row.of( - row.getField(0), - row.getField(1), - getRuntimeContext().getIndexOfThisSubtask(), - getRuntimeContext().getAttemptNumber()); - - return output; - } - } - - private static Configuration configure() { - Configuration configuration = new Configuration(); - configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); - configuration.set(RestOptions.BIND_PORT, "0"); - configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); - - // Use FLIP-27 source - configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); - - // for speculative execution - configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); - - configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_MULTIPLIER, 1.0); - configuration.set(SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_RATIO, 0.2); - configuration.set( - SlowTaskDetectorOptions.EXECUTION_TIME_BASELINE_LOWER_BOUND, Duration.ofMillis(0)); - configuration.set(BatchExecutionOptions.BLOCK_SLOW_NODE_DURATION, Duration.ofMillis(0)); - - return configuration; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java deleted file mode 100644 index 40dfda723749..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestMetadataTableReadableMetrics.java +++ /dev/null @@ -1,299 +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.source; - -import static org.apache.iceberg.types.Types.NestedField.optional; -import static org.apache.iceberg.types.Types.NestedField.required; - -import java.io.File; -import java.io.IOException; -import java.math.BigDecimal; -import java.nio.ByteBuffer; -import java.nio.file.Path; -import java.util.Base64; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.configuration.CoreOptions; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.Files; -import org.apache.iceberg.Parameters; -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.TableIdentifier; -import org.apache.iceberg.data.FileHelpers; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.TestHelpers; -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.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.io.TempDir; - -public class TestMetadataTableReadableMetrics extends CatalogTestBase { - private static final String TABLE_NAME = "test_table"; - - @Parameters(name = "catalogName={0}, baseNamespace={1}") - protected static List parameters() { - List parameters = Lists.newArrayList(); - String catalogName = "testhive"; - Namespace baseNamespace = Namespace.empty(); - parameters.add(new Object[] {catalogName, baseNamespace}); - return parameters; - } - - @Override - protected TableEnvironment getTableEnv() { - Configuration configuration = super.getTableEnv().getConfig().getConfiguration(); - configuration.set(CoreOptions.DEFAULT_PARALLELISM, 1); - return super.getTableEnv(); - } - - private @TempDir Path temp; - - private static final Types.StructType LEAF_STRUCT_TYPE = - Types.StructType.of( - optional(1, "leafLongCol", Types.LongType.get()), - optional(2, "leafDoubleCol", Types.DoubleType.get())); - - private static final Types.StructType NESTED_STRUCT_TYPE = - Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); - - private static final Schema NESTED_SCHEMA = - new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); - - private static final Schema PRIMITIVE_SCHEMA = - new Schema( - required(1, "booleanCol", Types.BooleanType.get()), - required(2, "intCol", Types.IntegerType.get()), - required(3, "longCol", Types.LongType.get()), - required(4, "floatCol", Types.FloatType.get()), - required(5, "doubleCol", Types.DoubleType.get()), - optional(6, "decimalCol", Types.DecimalType.of(10, 2)), - optional(7, "stringCol", Types.StringType.get()), - optional(8, "fixedCol", Types.FixedType.ofLength(3)), - optional(9, "binaryCol", Types.BinaryType.get())); - - private Table createPrimitiveTable() throws IOException { - Table table = - catalog.createTable( - TableIdentifier.of(DATABASE, TABLE_NAME), - PRIMITIVE_SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of()); - List records = - Lists.newArrayList( - createPrimitiveRecord( - false, - 1, - 1L, - 0, - 1.0D, - new BigDecimal("1.00"), - "1", - Base64.getDecoder().decode("1111"), - ByteBuffer.wrap(Base64.getDecoder().decode("1111"))), - createPrimitiveRecord( - true, - 2, - 2L, - 0, - 2.0D, - new BigDecimal("2.00"), - "2", - Base64.getDecoder().decode("2222"), - ByteBuffer.wrap(Base64.getDecoder().decode("2222"))), - createPrimitiveRecord(false, 1, 1, Float.NaN, Double.NaN, null, "1", null, null), - createPrimitiveRecord( - false, 2, 2L, Float.NaN, 2.0D, new BigDecimal("2.00"), "2", null, null)); - - File testFile = File.createTempFile("junit", null, temp.toFile()); - DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); - table.newAppend().appendFile(dataFile).commit(); - return table; - } - - private void createNestedTable() throws IOException { - Table table = - validationCatalog.createTable( - TableIdentifier.of(DATABASE, TABLE_NAME), - NESTED_SCHEMA, - PartitionSpec.unpartitioned(), - ImmutableMap.of()); - - List records = - Lists.newArrayList( - createNestedRecord(0L, 0.0), - createNestedRecord(1L, Double.NaN), - createNestedRecord(null, null)); - - File testFile = File.createTempFile("junit", null, temp.toFile()); - DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(testFile), records); - table.newAppend().appendFile(dataFile).commit(); - } - - @BeforeEach - public void before() { - super.before(); - sql("USE CATALOG %s", catalogName); - sql("CREATE DATABASE %s", DATABASE); - sql("USE %s", DATABASE); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE_NAME); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - protected GenericRecord createPrimitiveRecord( - boolean booleanCol, - int intCol, - long longCol, - float floatCol, - double doubleCol, - BigDecimal decimalCol, - String stringCol, - byte[] fixedCol, - ByteBuffer binaryCol) { - GenericRecord record = GenericRecord.create(PRIMITIVE_SCHEMA); - record.set(0, booleanCol); - record.set(1, intCol); - record.set(2, longCol); - record.set(3, floatCol); - record.set(4, doubleCol); - record.set(5, decimalCol); - record.set(6, stringCol); - record.set(7, fixedCol); - record.set(8, binaryCol); - return record; - } - - private GenericRecord createNestedRecord(Long longCol, Double doubleCol) { - GenericRecord record = GenericRecord.create(NESTED_SCHEMA); - GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); - GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); - leaf.set(0, longCol); - leaf.set(1, doubleCol); - nested.set(0, leaf); - record.set(0, nested); - return record; - } - - protected Object[] row(Object... values) { - return values; - } - - @TestTemplate - public void testPrimitiveColumns() throws Exception { - createPrimitiveTable(); - List result = sql("SELECT readable_metrics FROM %s$files", TABLE_NAME); - - Row binaryCol = - Row.of( - 52L, - 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 fixedCol = - Row.of( - 44L, - 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"); - - List expected = - Lists.newArrayList( - Row.of( - Row.of( - binaryCol, - booleanCol, - decimalCol, - doubleCol, - fixedCol, - floatCol, - intCol, - longCol, - stringCol))); - TestHelpers.assertRows(result, expected); - } - - @TestTemplate - public void testSelectPrimitiveValues() throws Exception { - createPrimitiveTable(); - - TestHelpers.assertRows( - sql( - "SELECT readable_metrics.intCol.lower_bound, readable_metrics.booleanCol.upper_bound FROM %s$files", - TABLE_NAME), - ImmutableList.of(Row.of(1, true))); - - TestHelpers.assertRows( - sql("SELECT content, readable_metrics.longCol.value_count FROM %s$files", TABLE_NAME), - ImmutableList.of(Row.of(0, 4L))); - - TestHelpers.assertRows( - sql("SELECT readable_metrics.longCol.value_count, content FROM %s$files", TABLE_NAME), - ImmutableList.of(Row.of(4L, 0))); - } - - @TestTemplate - public void testSelectNestedValues() throws Exception { - createNestedTable(); - TestHelpers.assertRows( - sql( - "SELECT readable_metrics.`nestedStructCol.leafStructCol.leafLongCol`.lower_bound, " - + "readable_metrics.`nestedStructCol.leafStructCol.leafDoubleCol`.value_count FROM %s$files", - TABLE_NAME), - ImmutableList.of(Row.of(0L, 3L))); - } - - @TestTemplate - 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 metrics = Row.of(Row.of(leafDoubleCol, leafLongCol)); - - TestHelpers.assertRows( - sql("SELECT readable_metrics FROM %s$files", TABLE_NAME), ImmutableList.of(metrics)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java deleted file mode 100644 index ce9054ad49b6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestProjectMetaColumn.java +++ /dev/null @@ -1,188 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.nio.file.Files; -import java.nio.file.Path; -import java.util.List; -import org.apache.flink.table.data.GenericRowData; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DeleteFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.RowDelta; -import org.apache.iceberg.SerializableTable; -import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; -import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.data.RowDataProjection; -import org.apache.iceberg.flink.sink.RowDataTaskWriterFactory; -import org.apache.iceberg.flink.sink.TaskWriterFactory; -import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; -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.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.io.TempDir; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestProjectMetaColumn { - - @TempDir protected Path temporaryFolder; - - @Parameter(index = 0) - private FileFormat format; - - @Parameters(name = "fileFormat={0}") - public static Iterable parameters() { - return Lists.newArrayList( - new Object[] {FileFormat.PARQUET}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.AVRO}); - } - - private void testSkipToRemoveMetaColumn(int formatVersion) throws IOException { - // Create the table with given format version. - String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); - Table table = - SimpleDataUtil.createTable( - location, - ImmutableMap.of(TableProperties.FORMAT_VERSION, String.valueOf(formatVersion)), - false); - - List rows = - Lists.newArrayList( - SimpleDataUtil.createInsert(1, "AAA"), - SimpleDataUtil.createInsert(2, "BBB"), - SimpleDataUtil.createInsert(3, "CCC")); - writeAndCommit(table, ImmutableList.of(), false, rows); - - FlinkInputFormat input = - FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); - - List results = Lists.newArrayList(); - TestHelpers.readRowData( - input, - rowData -> { - // If project to remove the meta columns, it will get a RowDataProjection. - assertThat(rowData).isInstanceOf(GenericRowData.class); - results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); - }); - - // Assert the results. - TestHelpers.assertRows(rows, results, SimpleDataUtil.ROW_TYPE); - } - - @TestTemplate - public void testV1SkipToRemoveMetaColumn() throws IOException { - testSkipToRemoveMetaColumn(1); - } - - @TestTemplate - public void testV2SkipToRemoveMetaColumn() throws IOException { - testSkipToRemoveMetaColumn(2); - } - - @TestTemplate - public void testV2RemoveMetaColumn() throws Exception { - // Create the v2 table. - String location = Files.createTempDirectory(temporaryFolder, "junit").toFile().toString(); - Table table = - SimpleDataUtil.createTable( - location, ImmutableMap.of(TableProperties.FORMAT_VERSION, "2"), false); - - List rows = - Lists.newArrayList( - SimpleDataUtil.createInsert(1, "AAA"), - SimpleDataUtil.createDelete(1, "AAA"), - SimpleDataUtil.createInsert(2, "AAA"), - SimpleDataUtil.createInsert(2, "BBB")); - int eqFieldId = table.schema().findField("data").fieldId(); - writeAndCommit(table, ImmutableList.of(eqFieldId), true, rows); - - FlinkInputFormat input = - FlinkSource.forRowData().tableLoader(TableLoader.fromHadoopTable(location)).buildFormat(); - - List results = Lists.newArrayList(); - TestHelpers.readRowData( - input, - rowData -> { - // If project to remove the meta columns, it will get a RowDataProjection. - assertThat(rowData).isInstanceOf(RowDataProjection.class); - results.add(TestHelpers.copyRowData(rowData, SimpleDataUtil.ROW_TYPE)); - }); - - // Assert the results. - TestHelpers.assertRows( - ImmutableList.of( - SimpleDataUtil.createInsert(2, "AAA"), SimpleDataUtil.createInsert(2, "BBB")), - results, - SimpleDataUtil.ROW_TYPE); - } - - private void writeAndCommit( - Table table, List eqFieldIds, boolean upsert, List rows) - throws IOException { - TaskWriter writer = createTaskWriter(table, eqFieldIds, upsert); - try (TaskWriter io = writer) { - for (RowData row : rows) { - io.write(row); - } - } - - RowDelta delta = table.newRowDelta(); - WriteResult result = writer.complete(); - - for (DataFile dataFile : result.dataFiles()) { - delta.addRows(dataFile); - } - - for (DeleteFile deleteFile : result.deleteFiles()) { - delta.addDeletes(deleteFile); - } - - delta.commit(); - } - - private TaskWriter createTaskWriter( - Table table, List equalityFieldIds, boolean upsert) { - TaskWriterFactory taskWriterFactory = - new RowDataTaskWriterFactory( - SerializableTable.copyOf(table), - SimpleDataUtil.ROW_TYPE, - TableProperties.WRITE_TARGET_FILE_SIZE_BYTES_DEFAULT, - format, - table.properties(), - equalityFieldIds, - upsert); - - taskWriterFactory.initialize(1, 1); - return taskWriterFactory.create(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java deleted file mode 100644 index 6ef40693827e..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestRowDataToAvroGenericRecordConverter.java +++ /dev/null @@ -1,36 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import org.apache.avro.generic.GenericRecord; -import org.apache.iceberg.flink.AvroGenericRecordConverterBase; -import org.apache.iceberg.flink.DataGenerator; - -public class TestRowDataToAvroGenericRecordConverter extends AvroGenericRecordConverterBase { - @Override - protected void testConverter(DataGenerator dataGenerator) { - RowDataToAvroGenericRecordConverter converter = - RowDataToAvroGenericRecordConverter.fromAvroSchema(dataGenerator.avroSchema()); - GenericRecord expected = dataGenerator.generateAvroGenericRecord(); - GenericRecord actual = converter.apply(dataGenerator.generateFlinkRowData()); - assertThat(actual).isEqualTo(expected); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java deleted file mode 100644 index 5dd7de545e11..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestScanContext.java +++ /dev/null @@ -1,112 +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.source; - -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import org.junit.jupiter.api.Test; - -class TestScanContext { - @Test - void testIncrementalFromSnapshotId() { - ScanContext context = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .build(); - assertException( - context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: null"); - - context = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .startSnapshotTimestamp(1L) - .build(); - assertException( - context, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - - @Test - void testIncrementalFromSnapshotTimestamp() { - ScanContext context = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .build(); - assertException( - context, - "Invalid starting snapshot timestamp for SPECIFIC_START_SNAPSHOT_TIMESTAMP strategy: null"); - - context = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotId(1L) - .startSnapshotTimestamp(1L) - .build(); - assertException( - context, "Invalid starting snapshot id for SPECIFIC_START_SNAPSHOT_ID strategy: not null"); - } - - @Test - void testStreaming() { - ScanContext context = ScanContext.builder().streaming(true).useTag("tag").build(); - assertException(context, "Cannot scan table using ref tag configured for streaming reader"); - - context = ScanContext.builder().streaming(true).useSnapshotId(1L).build(); - assertException(context, "Cannot set snapshot-id option for streaming reader"); - - context = ScanContext.builder().streaming(true).asOfTimestamp(1L).build(); - assertException(context, "Cannot set as-of-timestamp option for streaming reader"); - - context = ScanContext.builder().streaming(true).endSnapshotId(1L).build(); - assertException(context, "Cannot set end-snapshot-id option for streaming reader"); - - context = ScanContext.builder().streaming(true).endTag("tag").build(); - assertException(context, "Cannot set end-tag option for streaming reader"); - } - - @Test - void testStartConflict() { - ScanContext context = ScanContext.builder().startTag("tag").startSnapshotId(1L).build(); - assertException(context, "START_SNAPSHOT_ID and START_TAG cannot both be set."); - } - - @Test - void testEndConflict() { - ScanContext context = ScanContext.builder().endTag("tag").endSnapshotId(1L).build(); - assertException(context, "END_SNAPSHOT_ID and END_TAG cannot both be set."); - } - - @Test - void testMaxAllowedPlanningFailures() { - ScanContext context = ScanContext.builder().maxAllowedPlanningFailures(-2).build(); - assertException( - context, "Cannot set maxAllowedPlanningFailures to a negative number other than -1."); - } - - private void assertException(ScanContext context, String message) { - assertThatThrownBy(() -> context.validate()) - .hasMessage(message) - .isInstanceOf(IllegalArgumentException.class); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java deleted file mode 100644 index b701419a7499..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSourceUtil.java +++ /dev/null @@ -1,61 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import org.apache.flink.configuration.Configuration; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.junit.jupiter.api.Test; - -public class TestSourceUtil { - @Test - public void testInferedParallelism() throws IOException { - Configuration configuration = new Configuration(); - // Empty table, infer parallelism should be at least 1 - int parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 0); - assertThat(parallelism).isEqualTo(1); - - // 2 splits (max infer is the default value 100 , max > splits num), the parallelism is splits - // num : 2 - parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - assertThat(parallelism).isEqualTo(2); - - // 2 splits and limit is 1 , max infer parallelism is default 100, - // which is greater than splits num and limit, the parallelism is the limit value : 1 - parallelism = SourceUtil.inferParallelism(configuration, 1, () -> 2); - assertThat(parallelism).isEqualTo(1); - - // 2 splits and max infer parallelism is 1 (max < splits num), the parallelism is 1 - configuration.setInteger(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, 1); - parallelism = SourceUtil.inferParallelism(configuration, -1L, () -> 2); - assertThat(parallelism).isEqualTo(1); - - // 2 splits, max infer parallelism is 1, limit is 3, the parallelism is max infer parallelism : - // 1 - parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - assertThat(parallelism).isEqualTo(1); - - // 2 splits, infer parallelism is disabled, the parallelism is flink default parallelism 1 - configuration.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); - parallelism = SourceUtil.inferParallelism(configuration, 3, () -> 2); - assertThat(parallelism).isEqualTo(1); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java deleted file mode 100644 index f9b776397cfc..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestSqlBase.java +++ /dev/null @@ -1,160 +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.source; - -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.List; -import java.util.Map; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.table.api.EnvironmentSettings; -import org.apache.flink.table.api.TableEnvironment; -import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.flink.TableLoader; -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; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -/** Test other more advanced usage of SQL. They don't need to run for every file format. */ -public abstract class TestSqlBase { - @RegisterExtension - public static MiniClusterExtension miniClusterExtension = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - public static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - @TempDir protected Path temporaryFolder; - - private volatile TableEnvironment tEnv; - - protected TableEnvironment getTableEnv() { - if (tEnv == null) { - synchronized (this) { - if (tEnv == null) { - this.tEnv = - TableEnvironment.create(EnvironmentSettings.newInstance().inBatchMode().build()); - } - } - } - return tEnv; - } - - @BeforeEach - public abstract void before() throws IOException; - - @Test - public void testResiduals() throws Exception { - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - List writeRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - writeRecords.get(0).set(1, 123L); - writeRecords.get(0).set(2, "2020-03-20"); - writeRecords.get(1).set(1, 456L); - writeRecords.get(1).set(2, "2020-03-20"); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - - List expectedRecords = Lists.newArrayList(); - expectedRecords.add(writeRecords.get(0)); - - DataFile dataFile1 = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), writeRecords); - DataFile dataFile2 = - helper.writeFile( - TestHelpers.Row.of("2020-03-21", 0), - RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L)); - helper.appendToTable(dataFile1, dataFile2); - - org.apache.iceberg.flink.TestHelpers.assertRecords( - run(Maps.newHashMap(), "where dt='2020-03-20' and id=123", "*"), - expectedRecords, - TestFixtures.SCHEMA); - } - - @Test - public void testExposeLocality() throws Exception { - Table table = - CATALOG_EXTENSION - .catalog() - .createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA, TestFixtures.SPEC); - - TableLoader tableLoader = TableLoader.fromHadoopTable(table.location()); - List expectedRecords = RandomGenericData.generate(TestFixtures.SCHEMA, 10, 0L); - expectedRecords.forEach(expectedRecord -> expectedRecord.set(2, "2020-03-20")); - - GenericAppenderHelper helper = - new GenericAppenderHelper(table, FileFormat.PARQUET, temporaryFolder); - DataFile dataFile = helper.writeFile(TestHelpers.Row.of("2020-03-20", 0), expectedRecords); - helper.appendToTable(dataFile); - - // test sql api - Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), false); - - List results = SqlHelpers.sql(getTableEnv(), "select * from t"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - - // test table api - tableConf.setBoolean( - FlinkConfigOptions.TABLE_EXEC_ICEBERG_EXPOSE_SPLIT_LOCALITY_INFO.key(), true); - FlinkSource.Builder builder = FlinkSource.forRowData().tableLoader(tableLoader).table(table); - - // When running with CI or local, `localityEnabled` will be false even if this configuration is - // enabled - assertThat(SourceUtil.isLocalityEnabled(table, tableConf, true)) - .as("Expose split locality info should be false.") - .isFalse(); - - results = run(Maps.newHashMap(), "where dt='2020-03-20'", "*"); - org.apache.iceberg.flink.TestHelpers.assertRecords( - results, expectedRecords, TestFixtures.SCHEMA); - } - - protected List run( - Map options, String sqlFilter, String... sqlSelectedFields) { - String select = String.join(",", sqlSelectedFields); - String optionStr = SqlHelpers.sqlOptionsToString(options); - return SqlHelpers.sql(getTableEnv(), "select %s from t %s %s", select, optionStr, sqlFilter); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java deleted file mode 100644 index 57ee7baf202c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamScanSql.java +++ /dev/null @@ -1,434 +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.source; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import java.util.Iterator; -import java.util.List; -import org.apache.flink.core.execution.JobClient; -import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; -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.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.config.TableConfigOptions; -import org.apache.flink.types.Row; -import org.apache.flink.util.CloseableIterator; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.SnapshotRef; -import org.apache.iceberg.Table; -import org.apache.iceberg.TestHelpers; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.CatalogTestBase; -import org.apache.iceberg.flink.MiniFlinkClusterExtension; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.Timeout; - -@Timeout(60) -public class TestStreamScanSql extends CatalogTestBase { - private static final String TABLE = "test_table"; - private static final FileFormat FORMAT = FileFormat.PARQUET; - - private TableEnvironment tEnv; - - @Override - protected TableEnvironment getTableEnv() { - TableEnvironment tableEnv = tEnv; - if (tableEnv != null) { - return tableEnv; - } - synchronized (this) { - if (tEnv == null) { - EnvironmentSettings.Builder settingsBuilder = - EnvironmentSettings.newInstance().inStreamingMode(); - - StreamExecutionEnvironment env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); - env.enableCheckpointing(400); - - StreamTableEnvironment streamTableEnv = - StreamTableEnvironment.create(env, settingsBuilder.build()); - streamTableEnv - .getConfig() - .getConfiguration() - .set(TableConfigOptions.TABLE_DYNAMIC_TABLE_OPTIONS_ENABLED, true); - tEnv = streamTableEnv; - } - } - return tEnv; - } - - @Override - @BeforeEach - public void before() { - super.before(); - sql("CREATE DATABASE %s", flinkDatabase); - sql("USE CATALOG %s", catalogName); - sql("USE %s", DATABASE); - } - - @Override - @AfterEach - public void clean() { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, TABLE); - sql("DROP DATABASE IF EXISTS %s", flinkDatabase); - super.clean(); - } - - private void insertRows(String partition, Table table, Row... rows) throws IOException { - insertRows(partition, SnapshotRef.MAIN_BRANCH, table, rows); - } - - private void insertRows(String partition, String branch, Table table, Row... rows) - throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, FORMAT, temporaryDirectory); - - GenericRecord gRecord = GenericRecord.create(table.schema()); - List records = Lists.newArrayList(); - for (Row row : rows) { - records.add( - gRecord.copy( - "id", row.getField(0), - "data", row.getField(1), - "dt", row.getField(2))); - } - - if (partition != null) { - appender.appendToTable(TestHelpers.Row.of(partition, 0), branch, records); - } else { - appender.appendToTable(branch, records); - } - } - - private void insertRowsInBranch(String branch, Table table, Row... rows) throws IOException { - insertRows(null, branch, table, rows); - } - - private void insertRows(Table table, Row... rows) throws IOException { - insertRows(null, table, rows); - } - - private void assertRows(List expectedRows, Iterator iterator) { - for (Row expectedRow : expectedRows) { - assertThat(iterator).hasNext(); - Row actualRow = iterator.next(); - assertThat(actualRow.getArity()).isEqualTo(3); - assertThat(actualRow.getField(0)).isEqualTo(expectedRow.getField(0)); - assertThat(actualRow.getField(1)).isEqualTo(expectedRow.getField(1)); - assertThat(actualRow.getField(2)).isEqualTo(expectedRow.getField(2)); - } - } - - @TestTemplate - public void testUnPartitionedTable() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - - Row row1 = Row.of(1, "aaa", "2021-01-01"); - insertRows(table, row1); - assertRows(ImmutableList.of(row1), iterator); - - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row2); - assertRows(ImmutableList.of(row2), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testPartitionedTable() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR) PARTITIONED BY (dt)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - Row row1 = Row.of(1, "aaa", "2021-01-01"); - insertRows("2021-01-01", table, row1); - assertRows(ImmutableList.of(row1), iterator); - - Row row2 = Row.of(2, "bbb", "2021-01-02"); - insertRows("2021-01-02", table, row2); - assertRows(ImmutableList.of(row2), iterator); - - Row row3 = Row.of(1, "aaa", "2021-01-02"); - insertRows("2021-01-02", table, row3); - assertRows(ImmutableList.of(row3), iterator); - - Row row4 = Row.of(2, "bbb", "2021-01-01"); - insertRows("2021-01-01", table, row4); - assertRows(ImmutableList.of(row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testConsumeFromBeginning() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1, row2); - - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - try (CloseableIterator iterator = result.collect()) { - assertRows(ImmutableList.of(row1, row2), iterator); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - insertRows(table, row3); - assertRows(ImmutableList.of(row3), iterator); - - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row4); - assertRows(ImmutableList.of(row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - /** - * Insert records on the main branch. Then, insert in a named branch. Reads from the main branch - * and assert that the only records from main are returned - */ - public void testConsumeFilesFromMainBranch() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots on main branch - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - - insertRows(table, row1, row2); - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName).commit(); - - // insert on the 'b1' branch - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - - insertRowsInBranch(branchName, table, row3, row4); - - // read from main - TableResult result = - exec("SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s')*/", TABLE); - - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row1, row2), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - /** - * Insert records on the main branch. Creates a named branch. Insert record on named branch. Then - * select from the named branch and assert all the records are returned. - */ - public void testConsumeFilesFromBranch() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots on main branch - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - - insertRows(table, row1, row2); - String branchName = "b1"; - table.manageSnapshots().createBranch(branchName).commit(); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", - TABLE, branchName); - - try (CloseableIterator iterator = result.collect()) { - assertRows(ImmutableList.of(row1, row2), iterator); - // insert on the 'b1' branch - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRowsInBranch(branchName, table, row3, row4); - assertRows(ImmutableList.of(row3, row4), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - /** - * Insert records on branch b1. Then insert record on b2. Then select from each branch and assert - * the correct records are returned - */ - public void testConsumeFilesFromTwoBranches() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - String branch1 = "b1"; - String branch2 = "b2"; - table.manageSnapshots().createBranch(branch1).commit(); - table.manageSnapshots().createBranch(branch2).commit(); - - // Produce two snapshots on main branch - Row row1Branch1 = Row.of(1, "b1", "2021-01-01"); - Row row2Branch1 = Row.of(2, "b1", "2021-01-01"); - - Row row1Branch2 = Row.of(2, "b2", "2021-01-01"); - Row row2Branch2 = Row.of(3, "b3", "2021-01-01"); - - insertRowsInBranch(branch1, table, row1Branch1, row2Branch1); - insertRowsInBranch(branch2, table, row1Branch2, row2Branch2); - - TableResult resultBranch1 = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", - TABLE, branch1); - - try (CloseableIterator iterator = resultBranch1.collect()) { - assertRows(ImmutableList.of(row1Branch1, row2Branch1), iterator); - Row another = Row.of(4, "ccc", "2021-01-01"); - insertRowsInBranch(branch1, table, another); - assertRows(ImmutableList.of(another), iterator); - } - - TableResult resultBranch2 = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'branch'='%s')*/ ", - TABLE, branch2); - try (CloseableIterator iterator = resultBranch2.collect()) { - assertRows(ImmutableList.of(row1Branch2, row2Branch2), iterator); - Row another = Row.of(4, "ccc", "2021-01-01"); - insertRowsInBranch(branch2, table, another); - assertRows(ImmutableList.of(another), iterator); - } - - resultBranch1.getJobClient().ifPresent(JobClient::cancel); - resultBranch2.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testConsumeFromStartSnapshotId() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots. - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1); - insertRows(table, row2); - - long startSnapshotId = table.currentSnapshot().snapshotId(); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row3, row4); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " - + "'start-snapshot-id'='%d')*/", - TABLE, startSnapshotId); - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row3, row4), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - } - - @TestTemplate - public void testConsumeFromStartTag() throws Exception { - sql("CREATE TABLE %s (id INT, data VARCHAR, dt VARCHAR)", TABLE); - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, TABLE)); - - // Produce two snapshots. - Row row1 = Row.of(1, "aaa", "2021-01-01"); - Row row2 = Row.of(2, "bbb", "2021-01-01"); - insertRows(table, row1); - insertRows(table, row2); - - String tagName = "t1"; - long startSnapshotId = table.currentSnapshot().snapshotId(); - table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); - - Row row3 = Row.of(3, "ccc", "2021-01-01"); - Row row4 = Row.of(4, "ddd", "2021-01-01"); - insertRows(table, row3, row4); - - TableResult result = - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', " - + "'start-tag'='%s')*/", - TABLE, tagName); - try (CloseableIterator iterator = result.collect()) { - // the start snapshot(row2) is exclusive. - assertRows(ImmutableList.of(row3, row4), iterator); - - Row row5 = Row.of(5, "eee", "2021-01-01"); - Row row6 = Row.of(6, "fff", "2021-01-01"); - insertRows(table, row5, row6); - assertRows(ImmutableList.of(row5, row6), iterator); - - Row row7 = Row.of(7, "ggg", "2021-01-01"); - insertRows(table, row7); - assertRows(ImmutableList.of(row7), iterator); - } - result.getJobClient().ifPresent(JobClient::cancel); - - assertThatThrownBy( - () -> - exec( - "SELECT * FROM %s /*+ OPTIONS('streaming'='true', 'monitor-interval'='1s', 'start-tag'='%s', " - + "'start-snapshot-id'='%d' )*/", - TABLE, tagName, startSnapshotId)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("START_SNAPSHOT_ID and START_TAG cannot both be set."); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java deleted file mode 100644 index 9c4f476b02b4..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingMonitorFunction.java +++ /dev/null @@ -1,402 +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.source; - -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.time.Duration; -import java.util.Arrays; -import java.util.List; -import java.util.concurrent.CountDownLatch; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.functions.source.SourceFunction; -import org.apache.flink.streaming.api.operators.StreamSource; -import org.apache.flink.streaming.api.watermark.Watermark; -import org.apache.flink.streaming.util.AbstractStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -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.types.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.awaitility.Awaitility; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestStreamingMonitorFunction extends TestBase { - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - private static final long WAIT_TIME_MILLIS = 10 * 1000L; - - @Parameters(name = "formatVersion = {0}") - protected static List parameters() { - return Arrays.asList(1, 2); - } - - @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()); - } - - private void runSourceFunctionInTask( - TestSourceContext sourceContext, StreamingMonitorFunction function) { - Thread task = - new Thread( - () -> { - try { - function.run(sourceContext); - } catch (Exception e) { - throw new RuntimeException(e); - } - }); - task.start(); - } - - @TestTemplate - public void testConsumeWithoutStartSnapshotId() throws Exception { - List> recordsList = generateRecordsAndCommitTxn(10); - ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @TestTemplate - public void testConsumeFromStartSnapshotId() throws Exception { - // Commit the first five transactions. - generateRecordsAndCommitTxn(5); - long startSnapshotId = table.currentSnapshot().snapshotId(); - - // Commit the next five transactions. - List> recordsList = generateRecordsAndCommitTxn(5); - - ScanContext scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .startSnapshotId(startSnapshotId) - .build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @TestTemplate - public void testConsumeFromStartTag() throws Exception { - // Commit the first five transactions. - generateRecordsAndCommitTxn(5); - long startSnapshotId = table.currentSnapshot().snapshotId(); - String tagName = "t1"; - table.manageSnapshots().createTag(tagName, startSnapshotId).commit(); - - // Commit the next five transactions. - List> recordsList = generateRecordsAndCommitTxn(5); - - ScanContext scanContext = - ScanContext.builder().monitorInterval(Duration.ofMillis(100)).startTag(tagName).build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, function); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - function.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - } - - @TestTemplate - public void testCheckpointRestore() throws Exception { - List> recordsList = generateRecordsAndCommitTxn(10); - ScanContext scanContext = ScanContext.builder().monitorInterval(Duration.ofMillis(100)).build(); - - StreamingMonitorFunction func = createFunction(scanContext); - OperatorSubtaskState state; - try (AbstractStreamOperatorTestHarness harness = createHarness(func)) { - harness.setup(); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, func); - - awaitExpectedSplits(sourceContext); - - state = harness.snapshot(1, 1); - - // Stop the stream task. - func.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(recordsList)), SCHEMA); - } - - List> newRecordsList = generateRecordsAndCommitTxn(10); - StreamingMonitorFunction newFunc = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(newFunc)) { - harness.setup(); - // Recover to process the remaining snapshots. - harness.initializeState(state); - harness.open(); - - TestSourceContext sourceContext = new TestSourceContext(new CountDownLatch(1)); - runSourceFunctionInTask(sourceContext, newFunc); - - awaitExpectedSplits(sourceContext); - - // Stop the stream task. - newFunc.close(); - - TestHelpers.assertRecords( - sourceContext.toRows(), Lists.newArrayList(Iterables.concat(newRecordsList)), SCHEMA); - } - } - - private void awaitExpectedSplits(TestSourceContext sourceContext) { - Awaitility.await("expected splits should be produced") - .atMost(Duration.ofMillis(WAIT_TIME_MILLIS)) - .untilAsserted( - () -> { - assertThat(sourceContext.latch.getCount()).isEqualTo(0); - assertThat(sourceContext.splits).as("Should produce the expected splits").hasSize(1); - }); - } - - @TestTemplate - public void testInvalidMaxPlanningSnapshotCount() { - ScanContext scanContext1 = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .maxPlanningSnapshotCount(0) - .build(); - - assertThatThrownBy(() -> createFunction(scanContext1)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("The max-planning-snapshot-count must be greater than zero"); - - ScanContext scanContext2 = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .maxPlanningSnapshotCount(-10) - .build(); - - assertThatThrownBy(() -> createFunction(scanContext2)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("The max-planning-snapshot-count must be greater than zero"); - } - - @TestTemplate - public void testConsumeWithMaxPlanningSnapshotCount() throws Exception { - generateRecordsAndCommitTxn(10); - - // Use the oldest snapshot as starting to avoid the initial case. - long oldestSnapshotId = SnapshotUtil.oldestAncestor(table).snapshotId(); - - ScanContext scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(100)) - .splitSize(1000L) - .startSnapshotId(oldestSnapshotId) - .maxPlanningSnapshotCount(Integer.MAX_VALUE) - .build(); - - FlinkInputSplit[] expectedSplits = - FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool()); - - assertThat(expectedSplits).hasSize(9); - - // This covers three cases that maxPlanningSnapshotCount is less than, equal or greater than the - // total splits number - for (int maxPlanningSnapshotCount : ImmutableList.of(1, 9, 15)) { - scanContext = - ScanContext.builder() - .monitorInterval(Duration.ofMillis(500)) - .startSnapshotId(oldestSnapshotId) - .splitSize(1000L) - .maxPlanningSnapshotCount(maxPlanningSnapshotCount) - .build(); - - StreamingMonitorFunction function = createFunction(scanContext); - try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { - harness.setup(); - harness.open(); - - CountDownLatch latch = new CountDownLatch(1); - TestSourceContext sourceContext = new TestSourceContext(latch); - function.sourceContext(sourceContext); - function.monitorAndForwardSplits(); - - if (maxPlanningSnapshotCount < 10) { - assertThat(sourceContext.splits).hasSize(maxPlanningSnapshotCount); - } - } - } - } - - private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { - List> expectedRecords = Lists.newArrayList(); - for (int i = 0; i < commitTimes; i++) { - List records = RandomGenericData.generate(SCHEMA, 100, 0L); - expectedRecords.add(records); - - // Commit those records to iceberg table. - writeRecords(records); - } - return expectedRecords; - } - - private void writeRecords(List records) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); - appender.appendToTable(records); - } - - private StreamingMonitorFunction createFunction(ScanContext scanContext) { - return new StreamingMonitorFunction( - TestTableLoader.of(tableDir.getAbsolutePath()), scanContext); - } - - private AbstractStreamOperatorTestHarness createHarness( - StreamingMonitorFunction function) throws Exception { - StreamSource streamSource = - new StreamSource<>(function); - return new AbstractStreamOperatorTestHarness<>(streamSource, 1, 1, 0); - } - - private class TestSourceContext implements SourceFunction.SourceContext { - private final List splits = Lists.newArrayList(); - private final Object checkpointLock = new Object(); - private final CountDownLatch latch; - - TestSourceContext(CountDownLatch latch) { - this.latch = latch; - } - - @Override - public void collect(FlinkInputSplit element) { - splits.add(element); - latch.countDown(); - } - - @Override - public void collectWithTimestamp(FlinkInputSplit element, long timestamp) { - collect(element); - } - - @Override - public void emitWatermark(Watermark mark) {} - - @Override - public void markAsTemporarilyIdle() {} - - @Override - public Object getCheckpointLock() { - return checkpointLock; - } - - @Override - public void close() {} - - private List toRows() throws IOException { - FlinkInputFormat format = - FlinkSource.forRowData() - .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) - .buildFormat(); - - List rows = Lists.newArrayList(); - for (FlinkInputSplit split : splits) { - format.open(split); - - RowData element = null; - try { - while (!format.reachedEnd()) { - element = format.nextRecord(element); - rows.add(Row.of(element.getInt(0), element.getString(1).toString())); - } - } finally { - format.close(); - } - } - - return rows; - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java deleted file mode 100644 index 1606ee9f9648..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/TestStreamingReaderOperator.java +++ /dev/null @@ -1,293 +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.source; - -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.Collections; -import java.util.List; -import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; -import org.apache.flink.streaming.api.TimeCharacteristic; -import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; -import org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor; -import org.apache.flink.streaming.runtime.tasks.mailbox.MailboxDefaultAction; -import org.apache.flink.streaming.runtime.tasks.mailbox.SteppingMailboxProcessor; -import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; -import org.apache.flink.table.data.RowData; -import org.apache.flink.types.Row; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Schema; -import org.apache.iceberg.TestBase; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.TestTableLoader; -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.Types; -import org.apache.iceberg.util.SnapshotUtil; -import org.apache.iceberg.util.ThreadPools; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.TestTemplate; -import org.junit.jupiter.api.extension.ExtendWith; - -@ExtendWith(ParameterizedTestExtension.class) -public class TestStreamingReaderOperator extends TestBase { - - private static final Schema SCHEMA = - new Schema( - Types.NestedField.required(1, "id", Types.IntegerType.get()), - Types.NestedField.required(2, "data", Types.StringType.get())); - private static final FileFormat DEFAULT_FORMAT = FileFormat.PARQUET; - - @Parameters(name = "formatVersion = {0}") - protected static List parameters() { - return Arrays.asList(1, 2); - } - - @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()); - } - - @TestTemplate - public void testProcessAllRecords() throws Exception { - List> expectedRecords = generateRecordsAndCommitTxn(10); - - List splits = generateSplits(); - assertThat(splits).hasSize(10); - - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - SteppingMailboxProcessor processor = createLocalMailbox(harness); - - List expected = Lists.newArrayList(); - for (int i = 0; i < splits.size(); i++) { - // Process this element to enqueue to mail-box. - harness.processElement(splits.get(i), -1); - - // Run the mail-box once to read all records from the given split. - assertThat(processor.runMailboxStep()).as("Should processed 1 split").isTrue(); - - // Assert the output has expected elements. - expected.addAll(expectedRecords.get(i)); - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - } - } - - @TestTemplate - public void testTriggerCheckpoint() throws Exception { - // Received emitted splits: split1, split2, split3, checkpoint request is triggered when reading - // records from - // split1. - List> expectedRecords = generateRecordsAndCommitTxn(3); - - List splits = generateSplits(); - assertThat(splits).hasSize(3); - - long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - SteppingMailboxProcessor processor = createLocalMailbox(harness); - - harness.processElement(splits.get(0), ++timestamp); - harness.processElement(splits.get(1), ++timestamp); - harness.processElement(splits.get(2), ++timestamp); - - // Trigger snapshot state, it will start to work once all records from split0 are read. - processor.getMainMailboxExecutor().execute(() -> harness.snapshot(1, 3), "Trigger snapshot"); - - assertThat(processor.runMailboxStep()).as("Should have processed the split0").isTrue(); - assertThat(processor.runMailboxStep()) - .as("Should have processed the snapshot state action") - .isTrue(); - - TestHelpers.assertRecords(readOutputValues(harness), expectedRecords.get(0), SCHEMA); - - // Read records from split1. - assertThat(processor.runMailboxStep()).as("Should have processed the split1").isTrue(); - - // Read records from split2. - assertThat(processor.runMailboxStep()).as("Should have processed the split2").isTrue(); - - TestHelpers.assertRecords( - readOutputValues(harness), Lists.newArrayList(Iterables.concat(expectedRecords)), SCHEMA); - } - } - - @TestTemplate - public void testCheckpointRestore() throws Exception { - List> expectedRecords = generateRecordsAndCommitTxn(15); - - List splits = generateSplits(); - assertThat(splits).hasSize(15); - - OperatorSubtaskState state; - List expected = Lists.newArrayList(); - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - harness.open(); - - // Enqueue all the splits. - for (FlinkInputSplit split : splits) { - harness.processElement(split, -1); - } - - // Read all records from the first five splits. - SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); - for (int i = 0; i < 5; i++) { - expected.addAll(expectedRecords.get(i)); - assertThat(localMailbox.runMailboxStep()) - .as("Should have processed the split#" + i) - .isTrue(); - - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - - // Snapshot state now, there're 10 splits left in the state. - state = harness.snapshot(1, 1); - } - - expected.clear(); - try (OneInputStreamOperatorTestHarness harness = createReader()) { - harness.setup(); - // Recover to process the remaining splits. - harness.initializeState(state); - harness.open(); - - SteppingMailboxProcessor localMailbox = createLocalMailbox(harness); - - for (int i = 5; i < 10; i++) { - expected.addAll(expectedRecords.get(i)); - assertThat(localMailbox.runMailboxStep()) - .as("Should have processed the split#" + i) - .isTrue(); - - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - - // Let's process the final 5 splits now. - for (int i = 10; i < 15; i++) { - expected.addAll(expectedRecords.get(i)); - harness.processElement(splits.get(i), 1); - - assertThat(localMailbox.runMailboxStep()) - .as("Should have processed the split#" + i) - .isTrue(); - TestHelpers.assertRecords(readOutputValues(harness), expected, SCHEMA); - } - } - } - - private List readOutputValues( - OneInputStreamOperatorTestHarness harness) { - List results = Lists.newArrayList(); - for (RowData rowData : harness.extractOutputValues()) { - results.add(Row.of(rowData.getInt(0), rowData.getString(1).toString())); - } - return results; - } - - private List> generateRecordsAndCommitTxn(int commitTimes) throws IOException { - List> expectedRecords = Lists.newArrayList(); - for (int i = 0; i < commitTimes; i++) { - List records = RandomGenericData.generate(SCHEMA, 100, 0L); - expectedRecords.add(records); - - // Commit those records to iceberg table. - writeRecords(records); - } - return expectedRecords; - } - - private void writeRecords(List records) throws IOException { - GenericAppenderHelper appender = new GenericAppenderHelper(table, DEFAULT_FORMAT, temp); - appender.appendToTable(records); - } - - private List generateSplits() { - List inputSplits = Lists.newArrayList(); - - List snapshotIds = SnapshotUtil.currentAncestorIds(table); - for (int i = snapshotIds.size() - 1; i >= 0; i--) { - ScanContext scanContext; - if (i == snapshotIds.size() - 1) { - // Generate the splits from the first snapshot. - scanContext = ScanContext.builder().useSnapshotId(snapshotIds.get(i)).build(); - } else { - // Generate the splits between the previous snapshot and current snapshot. - scanContext = - ScanContext.builder() - .startSnapshotId(snapshotIds.get(i + 1)) - .endSnapshotId(snapshotIds.get(i)) - .build(); - } - - Collections.addAll( - inputSplits, - FlinkSplitPlanner.planInputSplits(table, scanContext, ThreadPools.getWorkerPool())); - } - - return inputSplits; - } - - private OneInputStreamOperatorTestHarness createReader() - throws Exception { - // This input format is used to opening the emitted split. - FlinkInputFormat inputFormat = - FlinkSource.forRowData() - .tableLoader(TestTableLoader.of(tableDir.getAbsolutePath())) - .buildFormat(); - - OneInputStreamOperatorFactory factory = - StreamingReaderOperator.factory(inputFormat); - OneInputStreamOperatorTestHarness harness = - new OneInputStreamOperatorTestHarness<>(factory, 1, 1, 0); - harness.getStreamConfig().setTimeCharacteristic(TimeCharacteristic.ProcessingTime); - - return harness; - } - - private SteppingMailboxProcessor createLocalMailbox( - OneInputStreamOperatorTestHarness harness) { - return new SteppingMailboxProcessor( - MailboxDefaultAction.Controller::suspendDefaultAction, - harness.getTaskMailbox(), - StreamTaskActionExecutor.IMMEDIATE); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java deleted file mode 100644 index 1e612b0a2b2a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/SplitAssignerTestBase.java +++ /dev/null @@ -1,132 +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.source.assigner; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.fail; - -import java.nio.file.Path; -import java.util.Collection; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public abstract class SplitAssignerTestBase { - @TempDir protected Path temporaryFolder; - - @Test - public void testEmptyInitialization() { - SplitAssigner assigner = splitAssigner(); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - /** Test a sequence of interactions for StaticEnumerator */ - @Test - public void testStaticEnumeratorSequence() throws Exception { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits(createSplits(4, 1, "1")); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertSnapshot(assigner, 1); - assigner.onUnassignedSplits(createSplits(1, 1, "1")); - assertSnapshot(assigner, 2); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - /** Test a sequence of interactions for ContinuousEnumerator */ - @Test - public void testContinuousEnumeratorSequence() throws Exception { - SplitAssigner assigner = splitAssigner(); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - - List splits1 = createSplits(1, 1, "1"); - assertAvailableFuture(assigner, 1, () -> assigner.onDiscoveredSplits(splits1)); - List splits2 = createSplits(1, 1, "1"); - assertAvailableFuture(assigner, 1, () -> assigner.onUnassignedSplits(splits2)); - - assigner.onDiscoveredSplits(createSplits(2, 1, "1")); - assertSnapshot(assigner, 2); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - private void assertAvailableFuture( - SplitAssigner assigner, int splitCount, Runnable addSplitsRunnable) { - // register callback - AtomicBoolean futureCompleted = new AtomicBoolean(); - CompletableFuture future = assigner.isAvailable(); - future.thenAccept(ignored -> futureCompleted.set(true)); - // calling isAvailable again should return the same object reference - // note that thenAccept will return a new future. - // we want to assert the same instance on the assigner returned future - assertThat(assigner.isAvailable()).isSameAs(future); - - // now add some splits - addSplitsRunnable.run(); - assertThat(futureCompleted.get()).isTrue(); - - for (int i = 0; i < splitCount; ++i) { - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - } - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } - - protected void assertGetNext(SplitAssigner assigner, GetSplitResult.Status expectedStatus) { - GetSplitResult result = assigner.getNext(null); - assertThat(result.status()).isEqualTo(expectedStatus); - switch (expectedStatus) { - case AVAILABLE: - assertThat(result.split()).isNotNull(); - break; - case CONSTRAINED: - case UNAVAILABLE: - assertThat(result.split()).isNull(); - break; - default: - fail("Unknown status: %s", expectedStatus); - } - } - - protected void assertSnapshot(SplitAssigner assigner, int splitCount) { - Collection stateBeforeGet = assigner.state(); - assertThat(stateBeforeGet).hasSize(splitCount); - } - - protected List createSplits(int fileCount, int filesPerSplit, String version) - throws Exception { - return SplitHelpers.createSplitsFromTransientHadoopTable( - temporaryFolder, fileCount, filesPerSplit, version); - } - - protected abstract SplitAssigner splitAssigner(); -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.java deleted file mode 100644 index 17e64bbf0594..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestDefaultSplitAssigner.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.flink.source.assigner; - -import org.apache.iceberg.flink.source.SplitHelpers; -import org.junit.jupiter.api.Test; - -public class TestDefaultSplitAssigner extends SplitAssignerTestBase { - @Override - protected SplitAssigner splitAssigner() { - return new DefaultSplitAssigner(null); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInASplit() throws Exception { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits( - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 4, 2)); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertSnapshot(assigner, 1); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - assertSnapshot(assigner, 0); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java deleted file mode 100644 index ff63ba8e58a0..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestFileSequenceNumberBasedSplitAssigner.java +++ /dev/null @@ -1,81 +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.source.assigner; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.util.List; -import org.apache.iceberg.ContentFile; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.util.SerializationUtil; -import org.junit.jupiter.api.Test; - -public class TestFileSequenceNumberBasedSplitAssigner extends SplitAssignerTestBase { - @Override - protected SplitAssigner splitAssigner() { - return new OrderedSplitAssignerFactory(SplitComparators.fileSequenceNumber()).createAssigner(); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInAnIcebergSplit() { - SplitAssigner assigner = splitAssigner(); - assertThatThrownBy( - () -> assigner.onDiscoveredSplits(createSplits(4, 2, "2")), - "Multiple files in a split is not allowed") - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Please use 'split-open-file-cost'"); - } - - /** Test sorted splits */ - @Test - public void testSplitSort() throws Exception { - SplitAssigner assigner = splitAssigner(); - List splits = createSplits(5, 1, "2"); - - assigner.onDiscoveredSplits(splits.subList(3, 5)); - assigner.onDiscoveredSplits(splits.subList(0, 1)); - assigner.onDiscoveredSplits(splits.subList(1, 3)); - - assertGetNext(assigner, 1L); - assertGetNext(assigner, 2L); - assertGetNext(assigner, 3L); - assertGetNext(assigner, 4L); - assertGetNext(assigner, 5L); - - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - @Test - public void testSerializable() { - byte[] bytes = SerializationUtil.serializeToBytes(SplitComparators.fileSequenceNumber()); - SerializableComparator comparator = - SerializationUtil.deserializeFromBytes(bytes); - assertThat(comparator).isNotNull(); - } - - private void assertGetNext(SplitAssigner assigner, Long expectedSequenceNumber) { - GetSplitResult result = assigner.getNext(null); - ContentFile file = result.split().task().files().iterator().next().file(); - assertThat(file.fileSequenceNumber()).isEqualTo(expectedSequenceNumber); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java deleted file mode 100644 index 84f04d5a530a..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/assigner/TestWatermarkBasedSplitAssigner.java +++ /dev/null @@ -1,146 +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.source.assigner; - -import static org.apache.iceberg.types.Types.NestedField.required; -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.time.Instant; -import java.time.LocalDateTime; -import java.time.ZoneOffset; -import java.time.temporal.ChronoUnit; -import java.util.List; -import java.util.concurrent.TimeUnit; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Schema; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.GenericRecord; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; -import org.apache.iceberg.flink.source.reader.ReaderUtil; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.flink.source.split.SplitComparators; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.types.Types; -import org.apache.iceberg.util.SerializationUtil; -import org.junit.jupiter.api.Test; - -public class TestWatermarkBasedSplitAssigner extends SplitAssignerTestBase { - public static final Schema SCHEMA = - new Schema(required(1, "timestamp_column", Types.TimestampType.withoutZone())); - private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); - - @Override - protected SplitAssigner splitAssigner() { - return new OrderedSplitAssignerFactory( - SplitComparators.watermark( - new ColumnStatsWatermarkExtractor(SCHEMA, "timestamp_column", null))) - .createAssigner(); - } - - /** Test the assigner when multiple files are in a single split */ - @Test - public void testMultipleFilesInAnIcebergSplit() { - SplitAssigner assigner = splitAssigner(); - assigner.onDiscoveredSplits(createSplits(4, 2, "2")); - - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.AVAILABLE); - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - /** Test sorted splits */ - @Test - public void testSplitSort() { - SplitAssigner assigner = splitAssigner(); - - Instant now = Instant.now(); - List splits = - IntStream.range(0, 5) - .mapToObj(i -> splitFromInstant(now.plus(i, ChronoUnit.MINUTES))) - .collect(Collectors.toList()); - - assigner.onDiscoveredSplits(splits.subList(3, 5)); - assigner.onDiscoveredSplits(splits.subList(0, 1)); - assigner.onDiscoveredSplits(splits.subList(1, 3)); - - assertGetNext(assigner, splits.get(0)); - assertGetNext(assigner, splits.get(1)); - assertGetNext(assigner, splits.get(2)); - assertGetNext(assigner, splits.get(3)); - assertGetNext(assigner, splits.get(4)); - - assertGetNext(assigner, GetSplitResult.Status.UNAVAILABLE); - } - - @Test - public void testSerializable() { - byte[] bytes = - SerializationUtil.serializeToBytes( - SplitComparators.watermark( - new ColumnStatsWatermarkExtractor( - TestFixtures.SCHEMA, "id", TimeUnit.MILLISECONDS))); - SerializableComparator comparator = - SerializationUtil.deserializeFromBytes(bytes); - assertThat(comparator).isNotNull(); - } - - private void assertGetNext(SplitAssigner assigner, IcebergSourceSplit split) { - GetSplitResult result = assigner.getNext(null); - assertThat(split).isEqualTo(result.split()); - } - - @Override - protected List createSplits( - int fileCount, int filesPerSplit, String version) { - return IntStream.range(0, fileCount / filesPerSplit) - .mapToObj( - splitNum -> - splitFromRecords( - IntStream.range(0, filesPerSplit) - .mapToObj( - fileNum -> - RandomGenericData.generate( - SCHEMA, 2, (long) splitNum * filesPerSplit + fileNum)) - .collect(Collectors.toList()))) - .collect(Collectors.toList()); - } - - private IcebergSourceSplit splitFromInstant(Instant instant) { - Record record = GenericRecord.create(SCHEMA); - record.set(0, LocalDateTime.ofInstant(instant, ZoneOffset.UTC)); - return splitFromRecords(ImmutableList.of(ImmutableList.of(record))); - } - - private IcebergSourceSplit splitFromRecords(List> records) { - try { - return IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - records, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); - } catch (IOException e) { - throw new RuntimeException("Split creation exception", e); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java deleted file mode 100644 index ebc92df02360..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/ManualContinuousSplitPlanner.java +++ /dev/null @@ -1,97 +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.source.enumerator; - -import java.io.IOException; -import java.util.List; -import java.util.NavigableMap; -import java.util.TreeMap; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -class ManualContinuousSplitPlanner implements ContinuousSplitPlanner { - private final int maxPlanningSnapshotCount; - // track splits per snapshot - private final NavigableMap> splits; - private long latestSnapshotId; - private int remainingFailures; - - ManualContinuousSplitPlanner(ScanContext scanContext, int expectedFailures) { - this.maxPlanningSnapshotCount = scanContext.maxPlanningSnapshotCount(); - this.splits = new TreeMap<>(); - this.latestSnapshotId = 0L; - this.remainingFailures = expectedFailures; - } - - @Override - public synchronized ContinuousEnumerationResult planSplits( - IcebergEnumeratorPosition lastPosition) { - if (remainingFailures > 0) { - remainingFailures--; - throw new RuntimeException("Expected failure at planning"); - } - - long fromSnapshotIdExclusive = 0; - if (lastPosition != null && lastPosition.snapshotId() != null) { - fromSnapshotIdExclusive = lastPosition.snapshotId(); - } - - Preconditions.checkArgument( - fromSnapshotIdExclusive <= latestSnapshotId, - "last enumerated snapshotId is greater than the latestSnapshotId"); - if (fromSnapshotIdExclusive == latestSnapshotId) { - // already discovered everything. - return new ContinuousEnumerationResult(Lists.newArrayList(), lastPosition, lastPosition); - } - - // find the subset of snapshots to return discovered splits - long toSnapshotIdInclusive; - if (latestSnapshotId - fromSnapshotIdExclusive > maxPlanningSnapshotCount) { - toSnapshotIdInclusive = fromSnapshotIdExclusive + maxPlanningSnapshotCount; - } else { - toSnapshotIdInclusive = latestSnapshotId; - } - - List discoveredSplits = Lists.newArrayList(); - NavigableMap> discoveredView = - splits.subMap(fromSnapshotIdExclusive, false, toSnapshotIdInclusive, true); - discoveredView.forEach((snapshotId, snapshotSplits) -> discoveredSplits.addAll(snapshotSplits)); - ContinuousEnumerationResult result = - new ContinuousEnumerationResult( - discoveredSplits, - lastPosition, - // use the snapshot Id as snapshot timestamp. - IcebergEnumeratorPosition.of(toSnapshotIdInclusive, toSnapshotIdInclusive)); - return result; - } - - /** - * Add a collection of new splits. A monotonically increased snapshotId is assigned to each batch - * of splits added by this method. - */ - public synchronized void addSplits(List newSplits) { - latestSnapshotId += 1; - splits.put(latestSnapshotId, newSplits); - } - - @Override - public void close() throws IOException {} -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java deleted file mode 100644 index 41a787762fda..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousIcebergEnumerator.java +++ /dev/null @@ -1,352 +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.source.enumerator; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.api.connector.source.SplitEnumeratorContext; -import org.apache.flink.connector.testutils.source.reader.TestingSplitEnumeratorContext; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.assigner.DefaultSplitAssigner; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.flink.source.split.SplitRequestEvent; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestContinuousIcebergEnumerator { - @TempDir protected Path temporaryFolder; - - @Test - public void testDiscoverSplitWhenNoReaderRegistered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - Collection pendingSplitsEmpty = - enumerator.snapshotState(1).pendingSplits(); - assertThat(pendingSplitsEmpty).isEmpty(); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - Collection pendingSplits = enumerator.snapshotState(2).pendingSplits(); - assertThat(pendingSplits).hasSize(1); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); - assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); - } - - @Test - public void testDiscoverWhenReaderRegistered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register one reader, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .contains(splits.get(0)); - } - - @Test - public void testRequestingReaderUnavailableWhenSplitDiscovered() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register one reader, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // remove the reader (like in a failure) - enumeratorContext.registeredReaders().remove(2); - - // make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - assertThat(splits).hasSize(1); - splitPlanner.addSplits(splits); - enumeratorContext.triggerAllActions(); - - assertThat(enumeratorContext.getSplitAssignments()).doesNotContainKey(2); - List pendingSplitIds = - enumerator.snapshotState(1).pendingSplits().stream() - .map(IcebergSourceSplitState::split) - .map(IcebergSourceSplit::splitId) - .collect(Collectors.toList()); - assertThat(pendingSplitIds).hasSameSizeAs(splits).first().isEqualTo(splits.get(0).splitId()); - - // register the reader again, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .contains(splits.get(0)); - } - - @Test - public void testThrottlingDiscovery() throws Exception { - // create 10 splits - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 1); - - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - // discover one snapshot at a time - .maxPlanningSnapshotCount(1) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 0); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // register reader-2, and let it request a split - enumeratorContext.registerReader(2, "localhost"); - enumerator.addReader(2); - enumerator.handleSourceEvent(2, new SplitRequestEvent()); - - // add splits[0] to the planner for next discovery - splitPlanner.addSplits(Arrays.asList(splits.get(0))); - enumeratorContext.triggerAllActions(); - - // because discovered split was assigned to reader, pending splits should be empty - assertThat(enumerator.snapshotState(1).pendingSplits()).isEmpty(); - // split assignment to reader-2 should contain splits[0, 1) - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .containsExactlyElementsOf(splits.subList(0, 1)); - - // add the remaining 9 splits (one for every snapshot) - // run discovery cycles while reader-2 still processing the splits[0] - for (int i = 1; i < 10; ++i) { - splitPlanner.addSplits(Arrays.asList(splits.get(i))); - enumeratorContext.triggerAllActions(); - } - - // can only discover up to 3 snapshots/splits - assertThat(enumerator.snapshotState(2).pendingSplits()).hasSize(3); - // split assignment to reader-2 should be splits[0, 1) - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .containsExactlyElementsOf(splits.subList(0, 1)); - - // now reader-2 finished splits[0] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(0).splitId()))); - enumeratorContext.triggerAllActions(); - // still have 3 pending splits. After assigned splits[1] to reader-2, one more split was - // discovered and added. - assertThat(enumerator.snapshotState(3).pendingSplits()).hasSize(3); - // split assignment to reader-2 should be splits[0, 2) - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .containsExactlyElementsOf(splits.subList(0, 2)); - - // run 3 more split discovery cycles - for (int i = 0; i < 3; ++i) { - enumeratorContext.triggerAllActions(); - } - - // no more splits are discovered due to throttling - assertThat(enumerator.snapshotState(4).pendingSplits()).hasSize(3); - // split assignment to reader-2 should still be splits[0, 2) - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .containsExactlyElementsOf(splits.subList(0, 2)); - - // now reader-2 finished splits[1] - enumerator.handleSourceEvent(2, new SplitRequestEvent(Arrays.asList(splits.get(1).splitId()))); - enumeratorContext.triggerAllActions(); - // still have 3 pending splits. After assigned new splits[2] to reader-2, one more split was - // discovered and added. - assertThat(enumerator.snapshotState(5).pendingSplits()).hasSize(3); - // split assignment to reader-2 should be splits[0, 3) - assertThat(enumeratorContext.getSplitAssignments().get(2).getAssignedSplits()) - .containsExactlyElementsOf(splits.subList(0, 3)); - } - - @Test - public void testTransientPlanningErrorsWithSuccessfulRetry() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(2) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 1); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - splitPlanner.addSplits(splits); - - // Trigger a planning and check that no splits returned due to the planning error - enumeratorContext.triggerAllActions(); - assertThat(enumerator.snapshotState(2).pendingSplits()).isEmpty(); - - // Second scan planning should succeed and discover the expected splits - enumeratorContext.triggerAllActions(); - Collection pendingSplits = enumerator.snapshotState(3).pendingSplits(); - assertThat(pendingSplits).hasSize(1); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); - assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); - } - - @Test - public void testOverMaxAllowedPlanningErrors() throws Exception { - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(1) - .build(); - ManualContinuousSplitPlanner splitPlanner = new ManualContinuousSplitPlanner(scanContext, 2); - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - splitPlanner.addSplits(splits); - - // Check that the scheduler response ignores the current error and continues to run until the - // failure limit is reached - enumeratorContext.triggerAllActions(); - assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) - .isFalse(); - - // Check that the task has failed with the expected exception after the failure limit is reached - enumeratorContext.triggerAllActions(); - assertThat(enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).isDone()) - .isTrue(); - assertThatThrownBy( - () -> enumeratorContext.getExecutorService().getAllScheduledTasks().get(0).get()) - .hasCauseInstanceOf(RuntimeException.class) - .hasMessageContaining("Failed to discover new split"); - } - - @Test - public void testPlanningIgnoringErrors() throws Exception { - int expectedFailures = 3; - TestingSplitEnumeratorContext enumeratorContext = - new TestingSplitEnumeratorContext<>(4); - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .maxPlanningSnapshotCount(1) - .maxAllowedPlanningFailures(-1) - .build(); - ManualContinuousSplitPlanner splitPlanner = - new ManualContinuousSplitPlanner(scanContext, expectedFailures); - ContinuousIcebergEnumerator enumerator = - createEnumerator(enumeratorContext, scanContext, splitPlanner); - - // Make one split available and trigger the periodic discovery - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - splitPlanner.addSplits(splits); - - Collection pendingSplits; - // Can not discover the new split with planning failures - for (int i = 0; i < expectedFailures; ++i) { - enumeratorContext.triggerAllActions(); - pendingSplits = enumerator.snapshotState(i).pendingSplits(); - assertThat(pendingSplits).isEmpty(); - } - - // Discovered the new split after a successful scan planning - enumeratorContext.triggerAllActions(); - pendingSplits = enumerator.snapshotState(expectedFailures + 1).pendingSplits(); - assertThat(pendingSplits).hasSize(1); - IcebergSourceSplitState pendingSplit = pendingSplits.iterator().next(); - assertThat(pendingSplit.split().splitId()).isEqualTo(splits.get(0).splitId()); - assertThat(pendingSplit.status()).isEqualTo(IcebergSourceSplitStatus.UNASSIGNED); - } - - private static ContinuousIcebergEnumerator createEnumerator( - SplitEnumeratorContext context, - ScanContext scanContext, - ContinuousSplitPlanner splitPlanner) { - - ContinuousIcebergEnumerator enumerator = - new ContinuousIcebergEnumerator( - context, - new DefaultSplitAssigner(null, Collections.emptyList()), - scanContext, - splitPlanner, - null); - enumerator.start(); - return enumerator; - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java deleted file mode 100644 index 0690b456e033..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImpl.java +++ /dev/null @@ -1,692 +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.source.enumerator; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.List; -import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; -import java.util.stream.Collectors; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableExtension; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; -import org.apache.iceberg.relocated.com.google.common.collect.Iterables; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public class TestContinuousSplitPlannerImpl { - @TempDir protected Path temporaryFolder; - - private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - private static final AtomicLong RANDOM_SEED = new AtomicLong(); - - @RegisterExtension - private static final HadoopTableExtension TABLE_RESOURCE = - new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - private GenericAppenderHelper dataAppender; - private DataFile dataFile1; - private Snapshot snapshot1; - private DataFile dataFile2; - private Snapshot snapshot2; - - @BeforeEach - public void before() throws IOException { - dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); - } - - private void appendTwoSnapshots() throws IOException { - // snapshot1 - List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - dataFile1 = dataAppender.writeFile(null, batch1); - dataAppender.appendToTable(dataFile1); - snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); - - // snapshot2 - List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); - dataFile2 = dataAppender.writeFile(null, batch2); - dataAppender.appendToTable(dataFile2); - snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); - } - - /** @return the last enumerated snapshot id */ - private CycleResult verifyOneCycle( - ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) - throws Exception { - List batch = - RandomGenericData.generate(TestFixtures.SCHEMA, 2, RANDOM_SEED.incrementAndGet()); - DataFile dataFile = dataAppender.writeFile(null, batch); - dataAppender.appendToTable(dataFile); - Snapshot snapshot = TABLE_RESOURCE.table().currentSnapshot(); - - ContinuousEnumerationResult result = splitPlanner.planSplits(lastPosition); - assertThat(result.fromPosition().snapshotId()).isEqualTo(lastPosition.snapshotId()); - assertThat(result.fromPosition().snapshotTimestampMs()) - .isEqualTo(lastPosition.snapshotTimestampMs()); - assertThat(result.toPosition().snapshotId().longValue()).isEqualTo(snapshot.snapshotId()); - assertThat(result.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot.timestampMillis()); - assertThat(result.splits()).hasSize(1); - IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - assertThat(split.task().files()) - .hasSize(1) - .first() - .satisfies( - fileScanTask -> assertThat(fileScanTask.file().path()).isEqualTo(dataFile.path())); - return new CycleResult(result.toPosition(), split); - } - - @Test - public void testTableScanThenIncrementalWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); - assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); - assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); - assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); - assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); - assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); - assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); - assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testTableScanThenIncrementalWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.fromPosition()).isNull(); - assertThat(initialResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot2.snapshotId()); - assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot2.timestampMillis()); - assertThat(initialResult.splits()).hasSize(1); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - assertThat(split.task().files()).hasSize(2); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromLatestSnapshotWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .splitSize(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); - assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); - assertThat(emptyTableInitialDiscoveryResult.toPosition().isEmpty()).isTrue(); - assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); - assertThat(emptyTableSecondDiscoveryResult.fromPosition().isEmpty()).isTrue(); - assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); - assertThat(emptyTableSecondDiscoveryResult.toPosition().isEmpty()).isTrue(); - assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); - - // latest mode should discover both snapshots, as latest position is marked by when job starts - appendTwoSnapshots(); - ContinuousEnumerationResult afterTwoSnapshotsAppended = - splitPlanner.planSplits(emptyTableSecondDiscoveryResult.toPosition()); - assertThat(afterTwoSnapshotsAppended.splits()).hasSize(2); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = afterTwoSnapshotsAppended.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromLatestSnapshotWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.fromPosition()).isNull(); - // For inclusive behavior, the initial result should point to snapshot1 - // Then the next incremental scan shall discover files from latest snapshot2 (inclusive) - assertThat(initialResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot1.snapshotId()); - assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot1.timestampMillis()); - assertThat(initialResult.splits()).isEmpty(); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - assertThat(secondResult.fromPosition().snapshotId().longValue()) - .isEqualTo(snapshot1.snapshotId()); - assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot1.timestampMillis()); - assertThat(secondResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot2.snapshotId()); - assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot2.timestampMillis()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - assertThat(split.task().files()).hasSize(1); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromEarliestSnapshotWithEmptyTable() throws Exception { - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult emptyTableInitialDiscoveryResult = splitPlanner.planSplits(null); - assertThat(emptyTableInitialDiscoveryResult.splits()).isEmpty(); - assertThat(emptyTableInitialDiscoveryResult.fromPosition()).isNull(); - assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotId()).isNull(); - assertThat(emptyTableInitialDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); - - ContinuousEnumerationResult emptyTableSecondDiscoveryResult = - splitPlanner.planSplits(emptyTableInitialDiscoveryResult.toPosition()); - assertThat(emptyTableSecondDiscoveryResult.splits()).isEmpty(); - assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotId()).isNull(); - assertThat(emptyTableSecondDiscoveryResult.fromPosition().snapshotTimestampMs()).isNull(); - assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotId()).isNull(); - assertThat(emptyTableSecondDiscoveryResult.toPosition().snapshotTimestampMs()).isNull(); - - // next 3 snapshots - IcebergEnumeratorPosition lastPosition = emptyTableSecondDiscoveryResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromEarliestSnapshotWithNonEmptyTable() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.fromPosition()).isNull(); - // For inclusive behavior, the initial result should point to snapshot1's parent, - // which leads to null snapshotId and snapshotTimestampMs. - assertThat(initialResult.toPosition().snapshotId()).isNull(); - assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); - assertThat(initialResult.splits()).isEmpty(); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - assertThat(secondResult.fromPosition().snapshotId()).isNull(); - assertThat(secondResult.fromPosition().snapshotTimestampMs()).isNull(); - assertThat(secondResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot2.snapshotId()); - assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot2.timestampMillis()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - assertThat(split.task().files()).hasSize(2); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover files appended in both snapshot1 and snapshot2 - Set expectedFiles = - ImmutableSet.of(dataFile1.path().toString(), dataFile2.path().toString()); - assertThat(discoveredFiles).containsExactlyInAnyOrderElementsOf(expectedFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromSnapshotIdWithEmptyTable() { - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - - assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: 1"); - } - - @Test - public void testIncrementalFromSnapshotIdWithInvalidIds() throws Exception { - appendTwoSnapshots(); - - // find an invalid snapshotId - long invalidSnapshotId = 0L; - while (invalidSnapshotId == snapshot1.snapshotId() - || invalidSnapshotId == snapshot2.snapshotId()) { - invalidSnapshotId++; - } - - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(invalidSnapshotId) - .build(); - - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - - assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: " + invalidSnapshotId); - } - - @Test - public void testIncrementalFromSnapshotId() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot2.snapshotId()) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.fromPosition()).isNull(); - // For inclusive behavior of snapshot2, the initial result should point to snapshot1 (as - // snapshot2's parent) - assertThat(initialResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot1.snapshotId()); - assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot1.timestampMillis()); - assertThat(initialResult.splits()).isEmpty(); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - assertThat(secondResult.fromPosition().snapshotId().longValue()) - .isEqualTo(snapshot1.snapshotId()); - assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot1.timestampMillis()); - assertThat(secondResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot2.snapshotId()); - assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot2.timestampMillis()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - assertThat(split.task().files()).hasSize(1); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testIncrementalFromSnapshotTimestampWithEmptyTable() { - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(1L) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - - assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot find a snapshot after: 1"); - } - - @Test - public void testIncrementalFromSnapshotTimestampWithInvalidIds() throws Exception { - appendTwoSnapshots(); - - long invalidSnapshotTimestampMs = snapshot2.timestampMillis() + 1000L; - - ScanContext scanContextWithInvalidSnapshotId = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(invalidSnapshotTimestampMs) - .build(); - - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl( - TABLE_RESOURCE.tableLoader().clone(), scanContextWithInvalidSnapshotId, null); - - assertThatThrownBy(() -> splitPlanner.planSplits(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after:"); - } - - @Test - public void testIncrementalFromSnapshotTimestamp() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis()) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.fromPosition()).isNull(); - // For inclusive behavior, the initial result should point to snapshot1 (as snapshot2's parent). - assertThat(initialResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot1.snapshotId()); - assertThat(initialResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot1.timestampMillis()); - assertThat(initialResult.splits()).isEmpty(); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - assertThat(secondResult.fromPosition().snapshotId().longValue()) - .isEqualTo(snapshot1.snapshotId()); - assertThat(secondResult.fromPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot1.timestampMillis()); - assertThat(secondResult.toPosition().snapshotId().longValue()) - .isEqualTo(snapshot2.snapshotId()); - assertThat(secondResult.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(snapshot2.timestampMillis()); - IcebergSourceSplit split = Iterables.getOnlyElement(secondResult.splits()); - assertThat(split.task().files()).hasSize(1); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - // should discover dataFile2 appended in snapshot2 - Set expectedFiles = ImmutableSet.of(dataFile2.path().toString()); - assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); - - IcebergEnumeratorPosition lastPosition = secondResult.toPosition(); - for (int i = 0; i < 3; ++i) { - lastPosition = verifyOneCycle(splitPlanner, lastPosition).lastPosition; - } - } - - @Test - public void testMaxPlanningSnapshotCount() throws Exception { - appendTwoSnapshots(); - // append 3 more snapshots - for (int i = 2; i < 5; ++i) { - appendSnapshot(i, 2); - } - - ScanContext scanContext = - ScanContext.builder() - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - // limit to 1 snapshot per discovery - .maxPlanningSnapshotCount(1) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.fromPosition()).isNull(); - // For inclusive behavior, the initial result should point to snapshot1's parent, - // which leads to null snapshotId and snapshotTimestampMs. - assertThat(initialResult.toPosition().snapshotId()).isNull(); - assertThat(initialResult.toPosition().snapshotTimestampMs()).isNull(); - assertThat(initialResult.splits()).isEmpty(); - - ContinuousEnumerationResult secondResult = splitPlanner.planSplits(initialResult.toPosition()); - // should discover dataFile1 appended in snapshot1 - verifyMaxPlanningSnapshotCountResult( - secondResult, null, snapshot1, ImmutableSet.of(dataFile1.path().toString())); - - ContinuousEnumerationResult thirdResult = splitPlanner.planSplits(secondResult.toPosition()); - // should discover dataFile2 appended in snapshot2 - verifyMaxPlanningSnapshotCountResult( - thirdResult, snapshot1, snapshot2, ImmutableSet.of(dataFile2.path().toString())); - } - - @Test - public void testTableScanNoStats() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(false) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.splits()).hasSize(1); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - assertThat(split.task().files()).hasSize(2); - verifyStatCount(split, 0); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 0); - lastPosition = result.lastPosition; - } - } - - @Test - public void testTableScanAllStats() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.splits()).hasSize(1); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - assertThat(split.task().files()).hasSize(2); - verifyStatCount(split, 3); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 3); - lastPosition = result.lastPosition; - } - } - - @Test - public void testTableScanSingleStat() throws Exception { - appendTwoSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .includeColumnStats(ImmutableSet.of("data")) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - ContinuousSplitPlannerImpl splitPlanner = - new ContinuousSplitPlannerImpl(TABLE_RESOURCE.tableLoader().clone(), scanContext, null); - - ContinuousEnumerationResult initialResult = splitPlanner.planSplits(null); - assertThat(initialResult.splits()).hasSize(1); - IcebergSourceSplit split = Iterables.getOnlyElement(initialResult.splits()); - assertThat(split.task().files()).hasSize(2); - verifyStatCount(split, 1); - - IcebergEnumeratorPosition lastPosition = initialResult.toPosition(); - for (int i = 0; i < 3; ++i) { - CycleResult result = verifyOneCycle(splitPlanner, lastPosition); - verifyStatCount(result.split, 1); - lastPosition = result.lastPosition; - } - } - - private void verifyStatCount(IcebergSourceSplit split, int expected) { - if (expected == 0) { - split - .task() - .files() - .forEach( - f -> { - assertThat(f.file().valueCounts()).isNull(); - assertThat(f.file().columnSizes()).isNull(); - assertThat(f.file().lowerBounds()).isNull(); - assertThat(f.file().upperBounds()).isNull(); - assertThat(f.file().nanValueCounts()).isNull(); - assertThat(f.file().nullValueCounts()).isNull(); - }); - } else { - split - .task() - .files() - .forEach( - f -> { - assertThat(f.file().valueCounts()).hasSize(expected); - assertThat(f.file().columnSizes()).hasSize(expected); - assertThat(f.file().lowerBounds()).hasSize(expected); - assertThat(f.file().upperBounds()).hasSize(expected); - assertThat(f.file().nullValueCounts()).hasSize(expected); - // The nanValue is not counted for long and string fields - assertThat(f.file().nanValueCounts()).isEmpty(); - }); - } - } - - private void verifyMaxPlanningSnapshotCountResult( - ContinuousEnumerationResult result, - Snapshot fromSnapshotExclusive, - Snapshot toSnapshotInclusive, - Set expectedFiles) { - if (fromSnapshotExclusive == null) { - assertThat(result.fromPosition().snapshotId()).isNull(); - assertThat(result.fromPosition().snapshotTimestampMs()).isNull(); - } else { - assertThat(result.fromPosition().snapshotId().longValue()) - .isEqualTo(fromSnapshotExclusive.snapshotId()); - assertThat(result.fromPosition().snapshotTimestampMs().longValue()) - .isEqualTo(fromSnapshotExclusive.timestampMillis()); - } - assertThat(result.toPosition().snapshotId().longValue()) - .isEqualTo(toSnapshotInclusive.snapshotId()); - assertThat(result.toPosition().snapshotTimestampMs().longValue()) - .isEqualTo(toSnapshotInclusive.timestampMillis()); - // should only have one split with one data file, because split discover is limited to - // one snapshot and each snapshot has only one data file appended. - IcebergSourceSplit split = Iterables.getOnlyElement(result.splits()); - assertThat(split.task().files()).hasSize(1); - Set discoveredFiles = - split.task().files().stream() - .map(fileScanTask -> fileScanTask.file().path().toString()) - .collect(Collectors.toSet()); - assertThat(discoveredFiles).containsExactlyElementsOf(expectedFiles); - } - - private Snapshot appendSnapshot(long seed, int numRecords) throws Exception { - List batch = RandomGenericData.generate(TestFixtures.SCHEMA, numRecords, seed); - DataFile dataFile = dataAppender.writeFile(null, batch); - dataAppender.appendToTable(dataFile); - return TABLE_RESOURCE.table().currentSnapshot(); - } - - private static class CycleResult { - IcebergEnumeratorPosition lastPosition; - IcebergSourceSplit split; - - CycleResult(IcebergEnumeratorPosition lastPosition, IcebergSourceSplit split) { - this.lastPosition = lastPosition; - this.split = split; - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java deleted file mode 100644 index b2185675340f..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestContinuousSplitPlannerImplStartStrategy.java +++ /dev/null @@ -1,200 +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.source.enumerator; - -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.List; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.Snapshot; -import org.apache.iceberg.data.GenericAppenderHelper; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableExtension; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.ScanContext; -import org.apache.iceberg.flink.source.StreamingStartingStrategy; -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.RegisterExtension; -import org.junit.jupiter.api.io.TempDir; - -public class TestContinuousSplitPlannerImplStartStrategy { - private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - - @TempDir protected Path temporaryFolder; - - @RegisterExtension - private static final HadoopTableExtension TABLE_RESOURCE = - new HadoopTableExtension(TestFixtures.DATABASE, TestFixtures.TABLE, TestFixtures.SCHEMA); - - private GenericAppenderHelper dataAppender; - private Snapshot snapshot1; - private Snapshot snapshot2; - private Snapshot snapshot3; - - @BeforeEach - public void before() throws IOException { - dataAppender = new GenericAppenderHelper(TABLE_RESOURCE.table(), FILE_FORMAT, temporaryFolder); - } - - private void appendThreeSnapshots() throws IOException { - List batch1 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 0L); - dataAppender.appendToTable(batch1); - snapshot1 = TABLE_RESOURCE.table().currentSnapshot(); - - List batch2 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 1L); - dataAppender.appendToTable(batch2); - snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); - - List batch3 = RandomGenericData.generate(TestFixtures.SCHEMA, 2, 2L); - dataAppender.appendToTable(batch3); - snapshot3 = TABLE_RESOURCE.table().currentSnapshot(); - } - - @Test - public void testTableScanThenIncrementalStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) - .build(); - - assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) - .isNotPresent(); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); - assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); - } - - @Test - public void testForLatestSnapshotStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_LATEST_SNAPSHOT) - .build(); - - assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) - .isNotPresent(); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); - assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot3.snapshotId()); - } - - @Test - public void testForEarliestSnapshotStrategy() throws IOException { - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_EARLIEST_SNAPSHOT) - .build(); - - assertThat(ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext)) - .isNotPresent(); - - appendThreeSnapshots(); - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); - assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot1.snapshotId()); - } - - @Test - public void testForSpecificSnapshotIdStrategy() throws IOException { - ScanContext scanContextInvalidSnapshotId = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(1L) - .build(); - - assertThatThrownBy( - () -> - ContinuousSplitPlannerImpl.startSnapshot( - TABLE_RESOURCE.table(), scanContextInvalidSnapshotId)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Start snapshot id not found in history: 1"); - - appendThreeSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_ID) - .startSnapshotId(snapshot2.snapshotId()) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); - assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); - } - - @Test - public void testForSpecificSnapshotTimestampStrategySnapshot2() throws IOException { - ScanContext scanContextInvalidSnapshotTimestamp = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(1L) - .build(); - - assertThatThrownBy( - () -> - ContinuousSplitPlannerImpl.startSnapshot( - TABLE_RESOURCE.table(), scanContextInvalidSnapshotTimestamp)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageStartingWith("Cannot find a snapshot after: "); - - appendThreeSnapshots(); - - ScanContext scanContext = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis()) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), scanContext).get(); - assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); - } - - @Test - public void testForSpecificSnapshotTimestampStrategySnapshot2Minus1() throws IOException { - appendThreeSnapshots(); - - ScanContext config = - ScanContext.builder() - .streaming(true) - .startingStrategy(StreamingStartingStrategy.INCREMENTAL_FROM_SNAPSHOT_TIMESTAMP) - .startSnapshotTimestamp(snapshot2.timestampMillis() - 1L) - .build(); - - Snapshot startSnapshot = - ContinuousSplitPlannerImpl.startSnapshot(TABLE_RESOURCE.table(), config).get(); - assertThat(startSnapshot.snapshotId()).isEqualTo(snapshot2.snapshotId()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.java deleted file mode 100644 index feefcb98646b..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestEnumerationHistory.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.source.enumerator; - -import static org.assertj.core.api.Assertions.assertThat; - -import org.junit.jupiter.api.Test; - -public class TestEnumerationHistory { - private static final int MAX_HISTORY_SIZE = 3; - private static final int FEW_PENDING_SPLITS = 2; - private static final int TOO_MANY_PENDING_SPLITS = 100; - - @Test - public void testEmptyHistory() { - EnumerationHistory history = new EnumerationHistory(MAX_HISTORY_SIZE); - int[] expectedHistorySnapshot = new int[0]; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testNotFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - int[] expectedHistorySnapshot = {1, 2}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testExactFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - int[] expectedHistorySnapshot = {1, 2, 3}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testOneMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - int[] expectedHistorySnapshot = {2, 3, 4}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testTwoMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - history.add(5); - int[] expectedHistorySnapshot = {3, 4, 5}; - testHistory(history, expectedHistorySnapshot); - } - - @Test - public void testThreeMoreThanFullHistory() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - history.add(4); - history.add(5); - history.add(6); - int[] expectedHistorySnapshot = {4, 5, 6}; - testHistory(history, expectedHistorySnapshot); - } - - private void testHistory(EnumerationHistory history, int[] expectedHistorySnapshot) { - assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); - if (history.hasFullHistory()) { - // throttle because pending split count is more than the sum of enumeration history - assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); - } else { - // skipped throttling check because there is not enough history - assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isFalse(); - } - - int[] historySnapshot = history.snapshot(); - assertThat(historySnapshot).containsExactly(expectedHistorySnapshot); - - EnumerationHistory restoredHistory = new EnumerationHistory(MAX_HISTORY_SIZE); - restoredHistory.restore(historySnapshot); - - assertThat(history.shouldPauseSplitDiscovery(FEW_PENDING_SPLITS)).isFalse(); - if (history.hasFullHistory()) { - // throttle because pending split count is more than the sum of enumeration history - assertThat(history.shouldPauseSplitDiscovery(TOO_MANY_PENDING_SPLITS)).isTrue(); - } else { - // skipped throttling check because there is not enough history - assertThat(history.shouldPauseSplitDiscovery(30)).isFalse(); - } - } - - @Test - public void testRestoreDifferentSize() { - EnumerationHistory history = new EnumerationHistory(3); - history.add(1); - history.add(2); - history.add(3); - int[] historySnapshot = history.snapshot(); - - EnumerationHistory smallerHistory = new EnumerationHistory(2); - smallerHistory.restore(historySnapshot); - int[] expectedRestoredHistorySnapshot = {2, 3}; - assertThat(smallerHistory.snapshot()).containsExactly(expectedRestoredHistorySnapshot); - - EnumerationHistory largerHisotry = new EnumerationHistory(4); - largerHisotry.restore(historySnapshot); - assertThat(largerHisotry.snapshot()).containsExactly(historySnapshot); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java deleted file mode 100644 index 2520a6b763e4..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/enumerator/TestIcebergEnumeratorStateSerializer.java +++ /dev/null @@ -1,146 +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.source.enumerator; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.Collection; -import java.util.Collections; -import java.util.Iterator; -import java.util.List; -import org.apache.iceberg.Parameter; -import org.apache.iceberg.ParameterizedTestExtension; -import org.apache.iceberg.Parameters; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitState; -import org.apache.iceberg.flink.source.split.IcebergSourceSplitStatus; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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 TestIcebergEnumeratorStateSerializer { - @TempDir protected Path temporaryFolder; - - private final IcebergEnumeratorStateSerializer serializer = - new IcebergEnumeratorStateSerializer(true); - - @Parameter(index = 0) - protected int version; - - @Parameters(name = "version={0}") - public static Object[][] parameters() { - return new Object[][] {new Object[] {1}, new Object[] {2}}; - } - - @TestTemplate - public void testEmptySnapshotIdAndPendingSplits() throws Exception { - IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(Collections.emptyList()); - testSerializer(enumeratorState); - } - - @TestTemplate - public void testSomeSnapshotIdAndEmptyPendingSplits() throws Exception { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(1L, System.currentTimeMillis()); - - IcebergEnumeratorState enumeratorState = - new IcebergEnumeratorState(position, Collections.emptyList()); - testSerializer(enumeratorState); - } - - @TestTemplate - public void testSomeSnapshotIdAndPendingSplits() throws Exception { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); - Collection pendingSplits = Lists.newArrayList(); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); - - IcebergEnumeratorState enumeratorState = new IcebergEnumeratorState(position, pendingSplits); - testSerializer(enumeratorState); - } - - @TestTemplate - public void testEnumerationSplitCountHistory() throws Exception { - if (version == 2) { - IcebergEnumeratorPosition position = - IcebergEnumeratorPosition.of(2L, System.currentTimeMillis()); - List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 3, 1); - Collection pendingSplits = Lists.newArrayList(); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(0), IcebergSourceSplitStatus.UNASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(1), IcebergSourceSplitStatus.ASSIGNED)); - pendingSplits.add( - new IcebergSourceSplitState(splits.get(2), IcebergSourceSplitStatus.COMPLETED)); - int[] enumerationSplitCountHistory = {1, 2, 3}; - - IcebergEnumeratorState enumeratorState = - new IcebergEnumeratorState(position, pendingSplits, enumerationSplitCountHistory); - testSerializer(enumeratorState); - } - } - - private void testSerializer(IcebergEnumeratorState enumeratorState) throws IOException { - byte[] result; - if (version == 1) { - result = serializer.serializeV1(enumeratorState); - } else { - result = serializer.serialize(enumeratorState); - } - - IcebergEnumeratorState deserialized = serializer.deserialize(version, result); - assertEnumeratorStateEquals(enumeratorState, deserialized); - } - - private void assertEnumeratorStateEquals( - IcebergEnumeratorState expected, IcebergEnumeratorState actual) { - assertThat(actual.lastEnumeratedPosition()).isEqualTo(expected.lastEnumeratedPosition()); - - assertThat(actual.pendingSplits()).hasSameSizeAs(expected.pendingSplits()); - Iterator expectedIterator = expected.pendingSplits().iterator(); - Iterator actualIterator = actual.pendingSplits().iterator(); - for (int i = 0; i < expected.pendingSplits().size(); ++i) { - IcebergSourceSplitState expectedSplitState = expectedIterator.next(); - IcebergSourceSplitState actualSplitState = actualIterator.next(); - assertThat(actualSplitState.split().splitId()) - .isEqualTo(expectedSplitState.split().splitId()); - assertThat(actualSplitState.split().fileOffset()) - .isEqualTo(expectedSplitState.split().fileOffset()); - assertThat(actualSplitState.split().recordOffset()) - .isEqualTo(expectedSplitState.split().recordOffset()); - assertThat(actualSplitState.status()).isEqualTo(expectedSplitState.status()); - } - - assertThat(actual.enumerationSplitCountHistory()) - .containsExactly(expected.enumerationSplitCountHistory()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java deleted file mode 100644 index 0d1d0ce3217c..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderFunctionTestBase.java +++ /dev/null @@ -1,218 +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.source.reader; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.IOException; -import java.nio.file.Path; -import java.util.List; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.iceberg.CombinedScanTask; -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.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -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 ReaderFunctionTestBase { - - @Parameters(name = "fileFormat={0}") - public static Object[][] parameters() { - return new Object[][] { - new Object[] {FileFormat.AVRO}, - new Object[] {FileFormat.ORC}, - new Object[] {FileFormat.PARQUET} - }; - } - - @TempDir protected Path temporaryFolder; - - protected abstract ReaderFunction readerFunction(); - - protected abstract void assertRecords(List expected, List actual, Schema schema); - - @Parameter(index = 0) - private FileFormat fileFormat; - - private final GenericAppenderFactory appenderFactory = - new GenericAppenderFactory(TestFixtures.SCHEMA); - - private void assertRecordsAndPosition( - List expectedRecords, - int expectedFileOffset, - long startRecordOffset, - RecordsWithSplitIds> batch) { - batch.nextSplit(); - List actualRecords = Lists.newArrayList(); - long recordOffset = startRecordOffset; - RecordAndPosition recordAndPosition; - while ((recordAndPosition = batch.nextRecordFromSplit()) != null) { - actualRecords.add(recordAndPosition.record()); - assertThat(recordAndPosition.fileOffset()).isEqualTo(expectedFileOffset); - assertThat(recordAndPosition.recordOffset() - 1).isEqualTo(recordOffset); - recordOffset++; - } - - assertThat(actualRecords).hasSameSizeAs(expectedRecords); - assertRecords(expectedRecords, actualRecords, TestFixtures.SCHEMA); - } - - @TestTemplate - public void testNoCheckpointedPosition() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @TestTemplate - public void testCheckpointedPositionBeforeFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 0L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0), 0, 0L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @TestTemplate - public void testCheckpointedPositionMiddleFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 1L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch0 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(0).subList(1, 2), 0, 1L, batch0); - batch0.recycle(); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @TestTemplate - public void testCheckpointedPositionAfterFirstFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 0, 2L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @TestTemplate - public void testCheckpointedPositionBeforeSecondFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 0L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1), 1, 0L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } - - @TestTemplate - public void testCheckpointedPositionMidSecondFile() throws IOException { - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - CombinedScanTask combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, fileFormat, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(combinedScanTask, 1, 1L); - CloseableIterator>> reader = - readerFunction().apply(split); - - RecordsWithSplitIds> batch1 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(1).subList(1, 2), 1, 1L, batch1); - batch1.recycle(); - - RecordsWithSplitIds> batch2 = reader.next(); - assertRecordsAndPosition(recordBatchList.get(2), 2, 0L, batch2); - batch2.recycle(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java deleted file mode 100644 index 0edf8ae009fe..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/ReaderUtil.java +++ /dev/null @@ -1,128 +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.source.reader; - -import java.io.File; -import java.io.IOException; -import java.nio.file.Path; -import java.util.Collections; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.BaseFileScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DataFiles; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.Files; -import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.PartitionSpecParser; -import org.apache.iceberg.Schema; -import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.expressions.Expressions; -import org.apache.iceberg.expressions.ResidualEvaluator; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.io.FileAppender; -import org.apache.iceberg.io.FileAppenderFactory; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; - -public class ReaderUtil { - - private ReaderUtil() {} - - public static FileScanTask createFileTask( - List records, - File file, - FileFormat fileFormat, - FileAppenderFactory appenderFactory) - throws IOException { - FileAppender appender = - appenderFactory.newAppender(Files.localOutput(file), fileFormat); - try { - appender.addAll(records); - } finally { - appender.close(); - } - - DataFile dataFile = - DataFiles.builder(PartitionSpec.unpartitioned()) - .withRecordCount(records.size()) - .withFileSizeInBytes(file.length()) - .withPath(file.toString()) - .withFormat(fileFormat) - .withMetrics(appender.metrics()) - .build(); - - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(Expressions.alwaysTrue()); - return new BaseFileScanTask( - dataFile, - null, - SchemaParser.toJson(TestFixtures.SCHEMA), - PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), - residuals); - } - - public static DataIterator createDataIterator(CombinedScanTask combinedTask) { - return new DataIterator<>( - new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()), - combinedTask, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - PlaintextEncryptionManager.instance()); - } - - public static List> createRecordBatchList( - Schema schema, int listSize, int batchCount) { - return createRecordBatchList(0L, schema, listSize, batchCount); - } - - public static List> createRecordBatchList( - long seed, Schema schema, int listSize, int batchCount) { - List records = RandomGenericData.generate(schema, listSize * batchCount, seed); - return Lists.partition(records, batchCount); - } - - public static CombinedScanTask createCombinedScanTask( - List> recordBatchList, - Path temporaryFolder, - FileFormat fileFormat, - GenericAppenderFactory appenderFactory) - throws IOException { - List fileTasks = Lists.newArrayListWithCapacity(recordBatchList.size()); - for (List recordBatch : recordBatchList) { - FileScanTask fileTask = - ReaderUtil.createFileTask( - recordBatch, - File.createTempFile("junit", null, temporaryFolder.toFile()), - fileFormat, - appenderFactory); - fileTasks.add(fileTask); - } - - return new BaseCombinedScanTask(fileTasks); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java deleted file mode 100644 index 6f09bd9a56d6..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayBatchRecords.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.reader; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.util.concurrent.atomic.AtomicBoolean; -import org.junit.jupiter.api.Test; - -public class TestArrayBatchRecords { - - @Test - public void testFullRange() { - String[] elements = new String[] {"0", "1", "2", "3"}; - testArray(elements, elements.length, 2, 119); - } - - @Test - public void testSubRange() { - String[] elements = new String[] {"0", "1", "2", "3"}; - testArray(elements, 2, 0, 0); - } - - private void testArray( - String[] elements, int numberOfRecords, int fileOffset, long startingRecordOffset) { - String splitId = "iceberg_split_1"; - AtomicBoolean recycled = new AtomicBoolean(); - - ArrayBatchRecords recordsWithSplitIds = - ArrayBatchRecords.forRecords( - splitId, - ignored -> recycled.set(true), - elements, - numberOfRecords, - fileOffset, - startingRecordOffset); - - assertThat(recordsWithSplitIds.nextSplit()).isEqualTo(splitId); - - for (int i = 0; i < numberOfRecords; i++) { - RecordAndPosition recAndPos = recordsWithSplitIds.nextRecordFromSplit(); - assertThat(recAndPos.record()).isEqualTo(elements[i]); - assertThat(recAndPos.fileOffset()).isEqualTo(fileOffset); - // recordOffset points to the position after this one - assertThat(recAndPos.recordOffset()).isEqualTo(startingRecordOffset + i + 1); - } - - assertThat(recordsWithSplitIds.nextRecordFromSplit()).isNull(); - assertThat(recordsWithSplitIds.nextSplit()).isNull(); - recordsWithSplitIds.recycle(); - assertThat(recycled.get()).isTrue(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java deleted file mode 100644 index 1a78bb1b0010..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestArrayPoolDataIteratorBatcherRowData.java +++ /dev/null @@ -1,360 +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.source.reader; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.io.File; -import java.nio.file.Path; -import java.util.Arrays; -import java.util.List; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.base.source.reader.RecordsWithSplitIds; -import org.apache.flink.connector.base.source.reader.SourceReaderOptions; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.BaseCombinedScanTask; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.FlinkConfigOptions; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.source.DataIterator; -import org.apache.iceberg.io.CloseableIterator; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestArrayPoolDataIteratorBatcherRowData { - - @TempDir protected Path temporaryFolder; - private static final FileFormat FILE_FORMAT = FileFormat.PARQUET; - private final Configuration config = - new Configuration() - .set(SourceReaderOptions.ELEMENT_QUEUE_CAPACITY, 1) - .set(FlinkConfigOptions.SOURCE_READER_FETCH_BATCH_RECORD_COUNT, 2); - - private final GenericAppenderFactory appenderFactory = - new GenericAppenderFactory(TestFixtures.SCHEMA); - private final DataIteratorBatcher batcher = - new ArrayPoolDataIteratorBatcher<>(config, new RowDataRecordFactory(TestFixtures.ROW_TYPE)); - - /** Read a CombinedScanTask that contains a single file with less than a full batch of records */ - @Test - public void testSingleFileLessThanOneFullBatch() throws Exception { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); - FileScanTask fileTask = - ReaderUtil.createFileTask( - records, - File.createTempFile("junit", null, temporaryFolder.toFile()), - FILE_FORMAT, - appenderFactory); - CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - ArrayBatchRecords batch = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch.finishedSplits()).isEmpty(); - assertThat(batch.nextSplit()).isEqualTo(splitId); - assertThat(batch.records()).hasSize(2); - assertThat(batch.numberOfRecords()).isEqualTo(1); - - RecordAndPosition recordAndPosition = batch.nextRecordFromSplit(); - - /////////////////////////////// - // assert first record - - assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(1); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - - assertThat(batch.nextRecordFromSplit()).isNull(); - assertThat(batch.nextSplit()).isNull(); - batch.recycle(); - - assertThat(recordBatchIterator).isExhausted(); - } - - /** - * Read a CombinedScanTask that contains a single file with multiple batches. - * - *

    Insert 5 records in a single file that should result in 3 batches - */ - @Test - public void testSingleFileWithMultipleBatches() throws Exception { - List records = RandomGenericData.generate(TestFixtures.SCHEMA, 5, 1); - FileScanTask fileTask = - ReaderUtil.createFileTask( - records, - File.createTempFile("junit", null, temporaryFolder.toFile()), - FILE_FORMAT, - appenderFactory); - CombinedScanTask combinedTask = new BaseCombinedScanTask(fileTask); - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - /////////////////////////////// - // assert first batch with full batch of 2 records - - ArrayBatchRecords batch0 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch0.finishedSplits()).isEmpty(); - assertThat(batch0.nextSplit()).isEqualTo(splitId); - assertThat(batch0.records()).hasSize(2); - assertThat(batch0.numberOfRecords()).isEqualTo(2); - - RecordAndPosition recordAndPosition; - - // assert first record - recordAndPosition = batch0.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(1); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(0), recordAndPosition.record()); - - // assert second record - recordAndPosition = batch0.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(2); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(1), recordAndPosition.record()); - - assertThat(batch0.nextRecordFromSplit()).isNull(); - assertThat(batch0.nextSplit()).isNull(); - batch0.recycle(); - - /////////////////////////////// - // assert second batch with full batch of 2 records - - ArrayBatchRecords batch1 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch1.records()).containsExactlyInAnyOrder(batch0.records()); - assertThat(batch1.finishedSplits()).isEmpty(); - assertThat(batch1.nextSplit()).isEqualTo(splitId); - assertThat(batch1.records()).hasSize(2); - assertThat(batch1.numberOfRecords()).isEqualTo(2); - - // assert third record - recordAndPosition = batch1.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(3); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(2), recordAndPosition.record()); - - // assert fourth record - recordAndPosition = batch1.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(4); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(3), recordAndPosition.record()); - - assertThat(batch1.nextRecordFromSplit()).isNull(); - assertThat(batch1.nextSplit()).isNull(); - batch1.recycle(); - - /////////////////////////////// - // assert third batch with partial batch of 1 record - - ArrayBatchRecords batch2 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch2.records()).containsExactlyInAnyOrder(batch0.records()); - assertThat(batch2.finishedSplits()).isEmpty(); - assertThat(batch2.nextSplit()).isEqualTo(splitId); - assertThat(batch2.records()).hasSize(2); - assertThat(batch2.numberOfRecords()).isEqualTo(1); - - // assert fifth record - recordAndPosition = batch2.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(0); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(5); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records.get(4), recordAndPosition.record()); - - assertThat(batch2.nextRecordFromSplit()).isNull(); - assertThat(batch2.nextSplit()).isNull(); - batch2.recycle(); - - assertThat(recordBatchIterator).isExhausted(); - } - - /** - * Read a CombinedScanTask that contains with multiple files. - * - *

    In this test, we also seek the iterator to starting position (1, 1). - */ - @Test - public void testMultipleFilesWithSeekPosition() throws Exception { - List records0 = RandomGenericData.generate(TestFixtures.SCHEMA, 1, 1); - FileScanTask fileTask0 = - ReaderUtil.createFileTask( - records0, - File.createTempFile("junit", null, temporaryFolder.toFile()), - FILE_FORMAT, - appenderFactory); - List records1 = RandomGenericData.generate(TestFixtures.SCHEMA, 4, 2); - FileScanTask fileTask1 = - ReaderUtil.createFileTask( - records1, - File.createTempFile("junit", null, temporaryFolder.toFile()), - FILE_FORMAT, - appenderFactory); - List records2 = RandomGenericData.generate(TestFixtures.SCHEMA, 3, 3); - FileScanTask fileTask2 = - ReaderUtil.createFileTask( - records2, - File.createTempFile("junit", null, temporaryFolder.toFile()), - FILE_FORMAT, - appenderFactory); - CombinedScanTask combinedTask = - new BaseCombinedScanTask(Arrays.asList(fileTask0, fileTask1, fileTask2)); - - DataIterator dataIterator = ReaderUtil.createDataIterator(combinedTask); - dataIterator.seek(1, 1); - - String splitId = "someSplitId"; - CloseableIterator>> recordBatchIterator = - batcher.batch(splitId, dataIterator); - - /////////////////////////////// - // file0 is skipped by seek - - /////////////////////////////// - // file1 has 4 records. because the seek position, first record is skipped. - // we should read 3 remaining records in 2 batches: - // batch10 with 2 records and batch11 with 1 records. - - // assert first batch from file1 with full batch of 2 records - - // variable naming convention: batch - ArrayBatchRecords batch10 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch10.finishedSplits()).isEmpty(); - assertThat(batch10.nextSplit()).isEqualTo(splitId); - assertThat(batch10.records()).hasSize(2); - assertThat(batch10.numberOfRecords()).isEqualTo(2); - - RecordAndPosition recordAndPosition; - - recordAndPosition = batch10.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(1); - assertThat(recordAndPosition.recordOffset()) - .as("seek should skip the first record in file1. starting from the second record") - .isEqualTo(2); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(1), recordAndPosition.record()); - - recordAndPosition = batch10.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(1); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(3); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(2), recordAndPosition.record()); - - assertThat(batch10.nextRecordFromSplit()).isNull(); - assertThat(batch10.nextSplit()).isNull(); - batch10.recycle(); - - // assert second batch from file1 with partial batch of 1 record - - // variable naming convention: batch__ - ArrayBatchRecords batch11 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch11.records()).containsExactlyInAnyOrder(batch10.records()); - assertThat(batch11.finishedSplits()).isEmpty(); - assertThat(batch11.nextSplit()).isEqualTo(splitId); - assertThat(batch11.records()).hasSize(2); - assertThat(batch11.numberOfRecords()).isEqualTo(1); - - recordAndPosition = batch11.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(1); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(4); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records1.get(3), recordAndPosition.record()); - - assertThat(batch11.nextRecordFromSplit()).isNull(); - assertThat(batch11.nextSplit()).isNull(); - batch11.recycle(); - - /////////////////////////////// - // file2 has 3 records. - // we should read 3 records in 2 batches: - // batch20 with 2 records and batch21 with 1 records - - // assert first batch from file2 with full batch of 2 records - - // variable naming convention: batch__ - ArrayBatchRecords batch20 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch20.records()).containsExactlyInAnyOrder(batch10.records()); - assertThat(batch20.finishedSplits()).isEmpty(); - assertThat(batch20.nextSplit()).isEqualTo(splitId); - assertThat(batch20.records()).hasSize(2); - assertThat(batch20.numberOfRecords()).isEqualTo(2); - - recordAndPosition = batch20.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(2); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(1); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(0), recordAndPosition.record()); - - recordAndPosition = batch20.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(2); - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(2); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(1), recordAndPosition.record()); - - assertThat(batch20.nextRecordFromSplit()).isNull(); - assertThat(batch20.nextSplit()).isNull(); - batch20.recycle(); - - /////////////////////////////// - // assert second batch from file2 with partial batch of 1 record - - // variable naming convention: batch__ - ArrayBatchRecords batch21 = (ArrayBatchRecords) recordBatchIterator.next(); - assertThat(batch21.records()).containsExactlyInAnyOrder(batch10.records()); - assertThat(batch21.finishedSplits()).isEmpty(); - assertThat(batch21.nextSplit()).isEqualTo(splitId); - assertThat(batch21.records()).hasSize(2); - assertThat(batch21.numberOfRecords()).isEqualTo(1); - - recordAndPosition = batch21.nextRecordFromSplit(); - assertThat(recordAndPosition.fileOffset()).isEqualTo(2); - - assertThat(recordAndPosition.recordOffset()) - .as("The position points to where the reader should resume after this record is processed.") - .isEqualTo(3); - TestHelpers.assertRowData(TestFixtures.SCHEMA, records2.get(2), recordAndPosition.record()); - - assertThat(batch21.nextRecordFromSplit()).isNull(); - assertThat(batch21.nextSplit()).isNull(); - batch21.recycle(); - - assertThat(recordBatchIterator).isExhausted(); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java deleted file mode 100644 index 8bd1214bd960..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestColumnStatsWatermarkExtractor.java +++ /dev/null @@ -1,176 +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.source.reader; - -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -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.IOException; -import java.nio.file.Path; -import java.time.LocalDateTime; -import java.time.OffsetDateTime; -import java.time.ZoneOffset; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.TimeUnit; -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.data.GenericAppenderFactory; -import org.apache.iceberg.data.RandomGenericData; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.flink.HadoopTableExtension; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; -import org.apache.iceberg.types.Types; -import org.junit.jupiter.api.BeforeAll; -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 class TestColumnStatsWatermarkExtractor { - public static final Schema SCHEMA = - new Schema( - required(1, "timestamp_column", Types.TimestampType.withoutZone()), - required(2, "timestamptz_column", Types.TimestampType.withZone()), - required(3, "long_column", Types.LongType.get()), - required(4, "string_column", Types.StringType.get())); - - private static final GenericAppenderFactory APPENDER_FACTORY = new GenericAppenderFactory(SCHEMA); - - private static final List> TEST_RECORDS = - ImmutableList.of( - RandomGenericData.generate(SCHEMA, 3, 2L), RandomGenericData.generate(SCHEMA, 3, 19L)); - - private static final List> MIN_VALUES = - ImmutableList.of(Maps.newHashMapWithExpectedSize(3), Maps.newHashMapWithExpectedSize(3)); - - @TempDir protected Path temporaryFolder; - - @RegisterExtension - private static final HadoopTableExtension SOURCE_TABLE_RESOURCE = - new HadoopTableExtension(DATABASE, TestFixtures.TABLE, SCHEMA); - - @Parameter(index = 0) - private String columnName; - - @BeforeAll - public static void updateMinValue() { - for (int i = 0; i < TEST_RECORDS.size(); ++i) { - for (Record r : TEST_RECORDS.get(i)) { - Map minValues = MIN_VALUES.get(i); - - LocalDateTime localDateTime = (LocalDateTime) r.get(0); - minValues.merge( - "timestamp_column", localDateTime.toInstant(ZoneOffset.UTC).toEpochMilli(), Math::min); - - OffsetDateTime offsetDateTime = (OffsetDateTime) r.get(1); - minValues.merge("timestamptz_column", offsetDateTime.toInstant().toEpochMilli(), Math::min); - - minValues.merge("long_column", (Long) r.get(2), Math::min); - } - } - } - - @Parameters(name = "columnName = {0}") - public static Collection data() { - return ImmutableList.of( - new Object[] {"timestamp_column"}, - new Object[] {"timestamptz_column"}, - new Object[] {"long_column"}); - } - - @TestTemplate - public void testSingle() throws IOException { - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MILLISECONDS); - - assertThat(extractor.extractWatermark(split(0))) - .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); - } - - @TestTemplate - public void testTimeUnit() throws IOException { - assumeThat(columnName).isEqualTo("long_column"); - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, TimeUnit.MICROSECONDS); - - assertThat(extractor.extractWatermark(split(0))) - .isEqualTo(MIN_VALUES.get(0).get(columnName) / 1000L); - } - - @TestTemplate - public void testMultipleFiles() throws IOException { - assumeThat(columnName).isEqualTo("timestamp_column"); - IcebergSourceSplit combinedSplit = - IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - TEST_RECORDS, temporaryFolder, FileFormat.PARQUET, APPENDER_FACTORY)); - - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null); - - assertThat(extractor.extractWatermark(split(0))) - .isEqualTo(MIN_VALUES.get(0).get(columnName).longValue()); - assertThat(extractor.extractWatermark(split(1))) - .isEqualTo(MIN_VALUES.get(1).get(columnName).longValue()); - assertThat(extractor.extractWatermark(combinedSplit)) - .isEqualTo(Math.min(MIN_VALUES.get(0).get(columnName), MIN_VALUES.get(1).get(columnName))); - } - - @TestTemplate - public void testWrongColumn() { - assumeThat(columnName).isEqualTo("string_column"); - assertThatThrownBy(() -> new ColumnStatsWatermarkExtractor(SCHEMA, columnName, null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining( - "Found STRING, expected a LONG or TIMESTAMP column for watermark generation."); - } - - @TestTemplate - public void testEmptyStatistics() throws IOException { - assumeThat(columnName).isEqualTo("timestamp_column"); - - // Create an extractor for a column we do not have statistics - ColumnStatsWatermarkExtractor extractor = - new ColumnStatsWatermarkExtractor(10, "missing_field"); - assertThatThrownBy(() -> extractor.extractWatermark(split(0))) - .isInstanceOf(IllegalArgumentException.class) - .hasMessageContaining("Missing statistics for column"); - } - - private IcebergSourceSplit split(int id) throws IOException { - return IcebergSourceSplit.fromCombinedScanTask( - ReaderUtil.createCombinedScanTask( - ImmutableList.of(TEST_RECORDS.get(id)), - temporaryFolder, - FileFormat.PARQUET, - APPENDER_FACTORY)); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.java deleted file mode 100644 index 8d6782586676..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestIcebergSourceReader.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.source.reader; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.nio.file.Path; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import org.apache.flink.api.connector.source.SourceReaderContext; -import org.apache.flink.configuration.Configuration; -import org.apache.flink.connector.testutils.source.reader.TestingReaderContext; -import org.apache.flink.connector.testutils.source.reader.TestingReaderOutput; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.flink.source.split.IcebergSourceSplit; -import org.apache.iceberg.flink.source.split.SerializableComparator; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestIcebergSourceReader { - @TempDir protected Path temporaryFolder; - - private final GenericAppenderFactory appenderFactory = - new GenericAppenderFactory(TestFixtures.SCHEMA); - - @Test - public void testReaderMetrics() throws Exception { - TestingReaderOutput readerOutput = new TestingReaderOutput<>(); - TestingMetricGroup metricGroup = new TestingMetricGroup(); - TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - IcebergSourceReader reader = createReader(metricGroup, readerContext, null); - reader.start(); - - testOneSplitFetcher(reader, readerOutput, metricGroup, 1); - testOneSplitFetcher(reader, readerOutput, metricGroup, 2); - } - - @Test - public void testReaderOrder() throws Exception { - // Create 2 splits - List> recordBatchList1 = - ReaderUtil.createRecordBatchList(0L, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task1 = - ReaderUtil.createCombinedScanTask( - recordBatchList1, temporaryFolder, FileFormat.PARQUET, appenderFactory); - - List> recordBatchList2 = - ReaderUtil.createRecordBatchList(1L, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task2 = - ReaderUtil.createCombinedScanTask( - recordBatchList2, temporaryFolder, FileFormat.PARQUET, appenderFactory); - - // Sort the splits in one way - List rowDataList1 = - read( - Arrays.asList( - IcebergSourceSplit.fromCombinedScanTask(task1), - IcebergSourceSplit.fromCombinedScanTask(task2)), - 2); - - // Reverse the splits - List rowDataList2 = - read( - Arrays.asList( - IcebergSourceSplit.fromCombinedScanTask(task2), - IcebergSourceSplit.fromCombinedScanTask(task1)), - 2); - - // Check that the order of the elements is not changed - assertThat(rowDataList1).containsExactlyElementsOf(rowDataList2); - } - - private List read(List splits, long expected) throws Exception { - TestingMetricGroup metricGroup = new TestingMetricGroup(); - TestingReaderContext readerContext = new TestingReaderContext(new Configuration(), metricGroup); - // Using IdBasedComparator, so we can have a deterministic order of the splits - IcebergSourceReader reader = createReader(metricGroup, readerContext, new IdBasedComparator()); - reader.start(); - - reader.addSplits(splits); - TestingReaderOutput readerOutput = new TestingReaderOutput<>(); - while (readerOutput.getEmittedRecords().size() < expected) { - reader.pollNext(readerOutput); - } - - reader.pollNext(readerOutput); - - assertThat(readerOutput.getEmittedRecords()).hasSize((int) expected); - return readerOutput.getEmittedRecords(); - } - - private void testOneSplitFetcher( - IcebergSourceReader reader, - TestingReaderOutput readerOutput, - TestingMetricGroup metricGroup, - int expectedCount) - throws Exception { - long seed = expectedCount; - // Each split should contain only one file with one record - List> recordBatchList = - ReaderUtil.createRecordBatchList(seed, TestFixtures.SCHEMA, 1, 1); - CombinedScanTask task = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); - IcebergSourceSplit split = IcebergSourceSplit.fromCombinedScanTask(task); - reader.addSplits(Collections.singletonList(split)); - - while (readerOutput.getEmittedRecords().size() < expectedCount) { - reader.pollNext(readerOutput); - } - - assertThat(readerOutput.getEmittedRecords()).hasSize(expectedCount); - TestHelpers.assertRowData( - TestFixtures.SCHEMA, - recordBatchList.get(0).get(0), - readerOutput.getEmittedRecords().get(expectedCount - 1)); - assertThat(metricGroup.counters().get("assignedSplits").getCount()).isEqualTo(expectedCount); - - // One more poll will get null record batch. - // That will finish the split and cause split fetcher to be closed due to idleness. - // Then next split will create a new split reader. - reader.pollNext(readerOutput); - } - - private IcebergSourceReader createReader( - MetricGroup metricGroup, - SourceReaderContext readerContext, - SerializableComparator splitComparator) { - IcebergSourceReaderMetrics readerMetrics = - new IcebergSourceReaderMetrics(metricGroup, "db.tbl"); - RowDataReaderFunction readerFunction = - new RowDataReaderFunction( - new Configuration(), - TestFixtures.SCHEMA, - TestFixtures.SCHEMA, - null, - true, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - PlaintextEncryptionManager.instance(), - Collections.emptyList()); - return new IcebergSourceReader<>( - SerializableRecordEmitter.defaultEmitter(), - readerMetrics, - readerFunction, - splitComparator, - readerContext); - } - - private static class IdBasedComparator implements SerializableComparator { - @Override - public int compare(IcebergSourceSplit o1, IcebergSourceSplit o2) { - return o1.splitId().compareTo(o2.splitId()); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java deleted file mode 100644 index 36749d3ec2dc..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestLimitableDataIterator.java +++ /dev/null @@ -1,84 +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.source.reader; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.nio.file.Path; -import java.util.Collections; -import java.util.List; -import org.apache.flink.table.data.RowData; -import org.apache.iceberg.CombinedScanTask; -import org.apache.iceberg.FileFormat; -import org.apache.iceberg.data.GenericAppenderFactory; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.EncryptionManager; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; -import org.apache.iceberg.hadoop.HadoopFileIO; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.BeforeAll; -import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; - -public class TestLimitableDataIterator { - @TempDir private static Path temporaryFolder; - - private final RowDataFileScanTaskReader reader = - new RowDataFileScanTaskReader( - TestFixtures.SCHEMA, TestFixtures.SCHEMA, null, true, Collections.emptyList()); - private final HadoopFileIO fileIO = new HadoopFileIO(new org.apache.hadoop.conf.Configuration()); - private final EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); - - private static CombinedScanTask combinedScanTask; - private static int totalRecords; - - @BeforeAll - public static void beforeClass() throws Exception { - GenericAppenderFactory appenderFactory = new GenericAppenderFactory(TestFixtures.SCHEMA); - List> recordBatchList = - ReaderUtil.createRecordBatchList(TestFixtures.SCHEMA, 3, 2); - combinedScanTask = - ReaderUtil.createCombinedScanTask( - recordBatchList, temporaryFolder, FileFormat.PARQUET, appenderFactory); - totalRecords = 3 * 2; - } - - @ParameterizedTest - @ValueSource(longs = {-1L, 0L, 1L, 6L, 7L}) - public void testUnlimited(long limit) { - LimitableDataIterator dataIterator = - new LimitableDataIterator<>( - reader, combinedScanTask, fileIO, encryptionManager, RecordLimiter.create(limit)); - - List result = Lists.newArrayList(); - while (dataIterator.hasNext()) { - result.add(dataIterator.next()); - } - - if (limit <= 0 || limit > totalRecords) { - // read all records - assertThat(result).hasSize(totalRecords); - } else { - assertThat(result).hasSize((int) limit); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java deleted file mode 100644 index 55f9c0af3a29..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestRowDataReaderFunction.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ -package org.apache.iceberg.flink.source.reader; - -import java.util.Collections; -import java.util.List; -import java.util.stream.Collectors; -import org.apache.flink.configuration.Configuration; -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.Schema; -import org.apache.iceberg.data.Record; -import org.apache.iceberg.encryption.PlaintextEncryptionManager; -import org.apache.iceberg.flink.FlinkSchemaUtil; -import org.apache.iceberg.flink.TestFixtures; -import org.apache.iceberg.flink.TestHelpers; -import org.apache.iceberg.hadoop.HadoopFileIO; - -public class TestRowDataReaderFunction extends ReaderFunctionTestBase { - - protected static final RowType ROW_TYPE = FlinkSchemaUtil.convert(TestFixtures.SCHEMA); - private static final DataStructureConverter ROW_DATA_CONVERTER = - DataStructureConverters.getConverter(TypeConversions.fromLogicalToDataType(ROW_TYPE)); - - @Override - protected ReaderFunction readerFunction() { - return new RowDataReaderFunction( - new Configuration(), - TestFixtures.SCHEMA, - TestFixtures.SCHEMA, - null, - true, - new HadoopFileIO(new org.apache.hadoop.conf.Configuration()), - PlaintextEncryptionManager.instance(), - Collections.emptyList()); - } - - @Override - protected void assertRecords(List expected, List actual, Schema schema) { - List rows = toRows(actual); - TestHelpers.assertRecords(rows, expected, TestFixtures.SCHEMA); - } - - private List toRows(List actual) { - return actual.stream() - .map(rowData -> (Row) ROW_DATA_CONVERTER.toExternal(rowData)) - .collect(Collectors.toList()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java deleted file mode 100644 index 290628c5fc90..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/reader/TestingMetricGroup.java +++ /dev/null @@ -1,102 +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.source.reader; - -import java.util.Map; -import org.apache.flink.metrics.Counter; -import org.apache.flink.metrics.Gauge; -import org.apache.flink.metrics.MetricGroup; -import org.apache.flink.metrics.SimpleCounter; -import org.apache.flink.metrics.groups.OperatorIOMetricGroup; -import org.apache.flink.metrics.groups.SourceReaderMetricGroup; -import org.apache.flink.metrics.groups.UnregisteredMetricsGroup; -import org.apache.iceberg.relocated.com.google.common.collect.Maps; - -class TestingMetricGroup extends UnregisteredMetricsGroup implements SourceReaderMetricGroup { - private final Map counters; - - TestingMetricGroup() { - this.counters = Maps.newHashMap(); - } - - /** Pass along the reference to share the map for child metric groups. */ - private TestingMetricGroup(Map counters) { - this.counters = counters; - } - - Map counters() { - return counters; - } - - @Override - public Counter counter(String name) { - Counter counter = new SimpleCounter(); - counters.put(name, counter); - return counter; - } - - @Override - public MetricGroup addGroup(String name) { - return new TestingMetricGroup(counters); - } - - @Override - public MetricGroup addGroup(String key, String value) { - return new TestingMetricGroup(counters); - } - - @Override - public OperatorIOMetricGroup getIOMetricGroup() { - return new TestingOperatorIOMetricGroup(); - } - - @Override - public Counter getNumRecordsInErrorsCounter() { - return new SimpleCounter(); - } - - @Override - public void setPendingBytesGauge(Gauge pendingBytesGauge) {} - - @Override - public void setPendingRecordsGauge(Gauge pendingRecordsGauge) {} - - private static class TestingOperatorIOMetricGroup extends UnregisteredMetricsGroup - implements OperatorIOMetricGroup { - @Override - public Counter getNumRecordsInCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumRecordsOutCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumBytesInCounter() { - return new SimpleCounter(); - } - - @Override - public Counter getNumBytesOutCounter() { - return new SimpleCounter(); - } - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java deleted file mode 100644 index 12bacdcd074d..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/source/split/TestIcebergSourceSplitSerializer.java +++ /dev/null @@ -1,183 +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.source.split; - -import static org.assertj.core.api.Assertions.assertThat; - -import java.nio.file.Path; -import java.util.List; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.stream.Collectors; -import org.apache.iceberg.FileScanTask; -import org.apache.iceberg.flink.source.SplitHelpers; -import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.io.TempDir; - -public class TestIcebergSourceSplitSerializer { - - @TempDir protected Path temporaryFolder; - - private final IcebergSourceSplitSerializer serializer = new IcebergSourceSplitSerializer(true); - - @Test - public void testLatestVersion() throws Exception { - serializeAndDeserialize(1, 1); - serializeAndDeserialize(10, 2); - } - - private void serializeAndDeserialize(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - temporaryFolder, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = serializer.serialize(split); - IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); - assertSplitEquals(split, deserialized); - - byte[] cachedResult = serializer.serialize(split); - assertThat(cachedResult).isSameAs(result); - IcebergSourceSplit deserialized2 = - serializer.deserialize(serializer.getVersion(), cachedResult); - assertSplitEquals(split, deserialized2); - - split.updatePosition(0, 100); - byte[] resultAfterUpdatePosition = serializer.serialize(split); - // after position change, serialized bytes should have changed - assertThat(resultAfterUpdatePosition).isNotSameAs(cachedResult); - IcebergSourceSplit deserialized3 = - serializer.deserialize(serializer.getVersion(), resultAfterUpdatePosition); - assertSplitEquals(split, deserialized3); - } - } - - @Test - public void testV1() throws Exception { - serializeAndDeserializeV1(1, 1); - serializeAndDeserializeV1(10, 2); - } - - private void serializeAndDeserializeV1(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - temporaryFolder, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV1(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV1(result); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testV2() throws Exception { - serializeAndDeserializeV2(1, 1); - serializeAndDeserializeV2(10, 2); - } - - private void serializeAndDeserializeV2(int splitCount, int filesPerSplit) throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - temporaryFolder, splitCount, filesPerSplit); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV2(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV2(result, true); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testV3WithTooManyDeleteFiles() throws Exception { - serializeAndDeserializeV3(1, 1, 5000); - } - - private void serializeAndDeserializeV3(int splitCount, int filesPerSplit, int mockDeletesPerSplit) - throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable( - temporaryFolder, splitCount, filesPerSplit); - final List splitsWithMockDeleteFiles = - SplitHelpers.equipSplitsWithMockDeleteFiles(splits, temporaryFolder, mockDeletesPerSplit); - - for (IcebergSourceSplit split : splitsWithMockDeleteFiles) { - byte[] result = split.serializeV3(); - IcebergSourceSplit deserialized = IcebergSourceSplit.deserializeV3(result, true); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testDeserializeV1() throws Exception { - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 1, 1); - for (IcebergSourceSplit split : splits) { - byte[] result = split.serializeV1(); - IcebergSourceSplit deserialized = serializer.deserialize(1, result); - assertSplitEquals(split, deserialized); - } - } - - @Test - public void testCheckpointedPosition() throws Exception { - final AtomicInteger index = new AtomicInteger(); - final List splits = - SplitHelpers.createSplitsFromTransientHadoopTable(temporaryFolder, 10, 2).stream() - .map( - split -> { - IcebergSourceSplit result; - if (index.get() % 2 == 0) { - result = IcebergSourceSplit.fromCombinedScanTask(split.task(), 1, 1); - } else { - result = split; - } - index.incrementAndGet(); - return result; - }) - .collect(Collectors.toList()); - - for (IcebergSourceSplit split : splits) { - byte[] result = serializer.serialize(split); - IcebergSourceSplit deserialized = serializer.deserialize(serializer.getVersion(), result); - assertSplitEquals(split, deserialized); - - byte[] cachedResult = serializer.serialize(split); - assertThat(cachedResult).isSameAs(result); - IcebergSourceSplit deserialized2 = - serializer.deserialize(serializer.getVersion(), cachedResult); - assertSplitEquals(split, deserialized2); - } - } - - private void assertSplitEquals(IcebergSourceSplit expected, IcebergSourceSplit actual) { - List expectedTasks = Lists.newArrayList(expected.task().tasks().iterator()); - List actualTasks = Lists.newArrayList(actual.task().tasks().iterator()); - assertThat(actualTasks).hasSameSizeAs(expectedTasks); - 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.sizeBytes()).isEqualTo(expectedTask.sizeBytes()); - assertThat(actualTask.filesCount()).isEqualTo(expectedTask.filesCount()); - assertThat(actualTask.start()).isEqualTo(expectedTask.start()); - assertThat(actualTask.length()).isEqualTo(expectedTask.length()); - } - - assertThat(actual.fileOffset()).isEqualTo(expected.fileOffset()); - assertThat(actual.recordOffset()).isEqualTo(expected.recordOffset()); - } -} diff --git a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java deleted file mode 100644 index 079c70bae070..000000000000 --- a/flink/v1.17/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ /dev/null @@ -1,55 +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.util; - -import static org.assertj.core.api.Assertions.assertThat; - -import org.junit.jupiter.api.Test; -import org.mockito.MockedStatic; -import org.mockito.Mockito; - -public class TestFlinkPackage { - - /** This unit test would need to be adjusted as new Flink version is supported. */ - @Test - public void testVersion() { - assertThat(FlinkPackage.version()).isEqualTo("1.17.2"); - } - - @Test - public void testDefaultVersion() { - // It's difficult to reproduce a reflection error in a unit test, so we just inject a mocked - // fault to test the default logic - - // First make sure we're not caching a version result from a previous test - FlinkPackage.setVersion(null); - try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { - mockedStatic.when(FlinkPackage::versionFromJar).thenThrow(RuntimeException.class); - mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); - assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); - } - FlinkPackage.setVersion(null); - try (MockedStatic mockedStatic = Mockito.mockStatic(FlinkPackage.class)) { - mockedStatic.when(FlinkPackage::versionFromJar).thenReturn(null); - mockedStatic.when(FlinkPackage::version).thenCallRealMethod(); - FlinkPackage.setVersion(null); - assertThat(FlinkPackage.version()).isEqualTo(FlinkPackage.FLINK_UNKNOWN_VERSION); - } - } -} diff --git a/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory b/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory deleted file mode 100644 index 47a3c94aa991..000000000000 --- a/flink/v1.17/flink/src/test/resources/META-INF/services/org.apache.flink.table.factories.Factory +++ /dev/null @@ -1,16 +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. - -org.apache.iceberg.flink.source.BoundedTableFactory From 59ac3b6ebafaae4f98a96c31a049b5fb01a8c956 Mon Sep 17 00:00:00 2001 From: Steven Wu Date: Mon, 5 Aug 2024 10:09:36 -0700 Subject: [PATCH 0571/1019] Flink: adjust code for the new 1.20 module. also fixed the bug of missing jmh in the 1.19 module. --- .github/workflows/flink-ci.yml | 9 +---- flink/build.gradle | 9 +++-- .../shuffle/MapRangePartitionerBenchmark.java | 10 ++++-- flink/v1.20/build.gradle | 36 +++++++++---------- .../shuffle/MapRangePartitionerBenchmark.java | 10 ++++-- ...estIcebergSpeculativeExecutionSupport.java | 2 +- .../iceberg/flink/util/TestFlinkPackage.java | 2 +- gradle.properties | 4 +-- gradle/libs.versions.toml | 24 ++++++------- jmh.gradle | 12 +++---- settings.gradle | 18 +++++----- 11 files changed, 70 insertions(+), 66 deletions(-) diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 3df36e2be86a..370375783cc2 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -74,14 +74,7 @@ jobs: strategy: matrix: jvm: [11, 17, 21] - flink: ['1.17', '1.18', '1.19'] - exclude: - # Flink 1.17 does not support Java 17. - - jvm: 17 - flink: '1.17' - # Flink 1.17 does not support Java 21. - - jvm: 21 - flink: '1.17' + flink: ['1.18', '1.19', '1.20'] env: SPARK_LOCAL_IP: localhost steps: diff --git a/flink/build.gradle b/flink/build.gradle index f049ff69b059..17ed630cc235 100644 --- a/flink/build.gradle +++ b/flink/build.gradle @@ -19,11 +19,6 @@ def flinkVersions = (System.getProperty("flinkVersions") != null ? System.getProperty("flinkVersions") : System.getProperty("defaultFlinkVersions")).split(",") - -if (flinkVersions.contains("1.17")) { - apply from: file("$projectDir/v1.17/build.gradle") -} - if (flinkVersions.contains("1.18")) { apply from: file("$projectDir/v1.18/build.gradle") } @@ -31,3 +26,7 @@ if (flinkVersions.contains("1.18")) { if (flinkVersions.contains("1.19")) { apply from: file("$projectDir/v1.19/build.gradle") } + +if (flinkVersions.contains("1.20")) { + apply from: file("$projectDir/v1.20/build.gradle") +} \ No newline at end of file diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index c3917165753d..007b423e592a 100644 --- a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -27,6 +28,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; 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; @@ -66,6 +69,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -82,10 +87,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; diff --git a/flink/v1.20/build.gradle b/flink/v1.20/build.gradle index 392a1cb124f0..f2e1fb51a1f4 100644 --- a/flink/v1.20/build.gradle +++ b/flink/v1.20/build.gradle @@ -17,7 +17,7 @@ * under the License. */ -String flinkMajorVersion = '1.19' +String flinkMajorVersion = '1.20' String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { @@ -32,15 +32,15 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-hive-metastore') - compileOnly libs.flink119.avro + compileOnly libs.flink120.avro // for dropwizard histogram metrics implementation - compileOnly libs.flink119.metrics.dropwizard - compileOnly libs.flink119.streaming.java - compileOnly "${libs.flink119.streaming.java.get().module}:${libs.flink119.streaming.java.get().getVersion()}:tests" - compileOnly libs.flink119.table.api.java.bridge - compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" - compileOnly libs.flink119.connector.base - compileOnly libs.flink119.connector.files + compileOnly libs.flink120.metrics.dropwizard + compileOnly libs.flink120.streaming.java + compileOnly "${libs.flink120.streaming.java.get().module}:${libs.flink120.streaming.java.get().getVersion()}:tests" + compileOnly libs.flink120.table.api.java.bridge + compileOnly "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink120.get()}" + compileOnly libs.flink120.connector.base + compileOnly libs.flink120.connector.files compileOnly libs.hadoop2.hdfs compileOnly libs.hadoop2.common @@ -68,13 +68,13 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { implementation libs.datasketches - testImplementation libs.flink119.connector.test.utils - testImplementation libs.flink119.core - testImplementation libs.flink119.runtime - testImplementation(libs.flink119.test.utilsjunit) { + testImplementation libs.flink120.connector.test.utils + testImplementation libs.flink120.core + testImplementation libs.flink120.runtime + testImplementation(libs.flink120.test.utilsjunit) { exclude group: 'junit' } - testImplementation(libs.flink119.test.utils) { + testImplementation(libs.flink120.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } @@ -168,7 +168,7 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { } // for dropwizard histogram metrics implementation - implementation libs.flink119.metrics.dropwizard + implementation libs.flink120.metrics.dropwizard // for integration testing with the flink-runtime-jar // all of those dependencies are required because the integration test extends FlinkTestBase @@ -178,13 +178,13 @@ project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { integrationImplementation project(path: ":iceberg-flink:iceberg-flink-${flinkMajorVersion}", configuration: "testArtifacts") integrationImplementation project(path: ':iceberg-api', configuration: 'testArtifacts') integrationImplementation project(path: ':iceberg-hive-metastore', configuration: 'testArtifacts') - integrationImplementation(libs.flink119.test.utils) { + integrationImplementation(libs.flink120.test.utils) { exclude group: "org.apache.curator", module: 'curator-test' exclude group: 'junit' } - integrationImplementation libs.flink119.table.api.java.bridge - integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink119.get()}" + integrationImplementation libs.flink120.table.api.java.bridge + integrationImplementation "org.apache.flink:flink-table-planner_${scalaVersion}:${libs.versions.flink120.get()}" integrationImplementation libs.hadoop2.common integrationImplementation libs.hadoop2.hdfs diff --git a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index c3917165753d..007b423e592a 100644 --- a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.NavigableMap; @@ -27,6 +28,8 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SortKey; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.SortOrderComparators; +import org.apache.iceberg.StructLike; 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; @@ -66,6 +69,8 @@ public class MapRangePartitionerBenchmark { Types.NestedField.required(9, "name9", Types.StringType.get())); private static final SortOrder SORT_ORDER = SortOrder.builderFor(SCHEMA).asc("id").build(); + private static final Comparator SORT_ORDER_COMPARTOR = + SortOrderComparators.forSchema(SCHEMA, SORT_ORDER); private static final SortKey SORT_KEY = new SortKey(SCHEMA, SORT_ORDER); private MapRangePartitioner partitioner; @@ -82,10 +87,11 @@ public void setupBenchmark() { mapStatistics.put(sortKey, weight); }); - MapDataStatistics dataStatistics = new MapDataStatistics(mapStatistics); + MapAssignment mapAssignment = + MapAssignment.fromKeyFrequency(2, mapStatistics, 0.0, SORT_ORDER_COMPARTOR); this.partitioner = new MapRangePartitioner( - SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), dataStatistics, 2); + SCHEMA, SortOrder.builderFor(SCHEMA).asc("id").build(), mapAssignment); List keys = Lists.newArrayList(weights.keySet().iterator()); long[] weightsCDF = new long[keys.size()]; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 41b023b93617..992b712d9d69 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -165,7 +165,7 @@ private static Configuration configure() { Configuration configuration = new Configuration(); configuration.set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); configuration.set(RestOptions.BIND_PORT, "0"); - configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, 5000L); + configuration.set(JobManagerOptions.SLOT_REQUEST_TIMEOUT, Duration.ofSeconds(5)); // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java index 4ba4f9d983dc..65f21f7d050c 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/util/TestFlinkPackage.java @@ -29,7 +29,7 @@ public class TestFlinkPackage { /** This unit test would need to be adjusted as new Flink version is supported. */ @Test public void testVersion() { - assertThat(FlinkPackage.version()).isEqualTo("1.19.0"); + assertThat(FlinkPackage.version()).isEqualTo("1.20.0"); } @Test diff --git a/gradle.properties b/gradle.properties index c6b8dec17bc5..fcbe7d8de012 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,8 +16,8 @@ jmhOutputPath=build/reports/jmh/human-readable-output.txt jmhJsonOutputPath=build/reports/jmh/results.json jmhIncludeRegex=.* -systemProp.defaultFlinkVersions=1.19 -systemProp.knownFlinkVersions=1.17,1.18,1.19 +systemProp.defaultFlinkVersions=1.20 +systemProp.knownFlinkVersions=1.18,1.19,1.20 systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 12caeda95407..77e610e885f6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -39,9 +39,9 @@ delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.29.2" findbugs-jsr305 = "3.0.2" -flink117 = { strictly = "1.17.2"} flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} +flink120 = { strictly = "1.20.0"} google-libraries-bom = "26.43.0" guava = "33.2.1-jre" hadoop2 = "2.7.3" @@ -108,12 +108,6 @@ datasketches = { module = "org.apache.datasketches:datasketches-java", version.r delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } -flink117-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink117" } -flink117-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink117" } -flink117-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink117" } -flink117-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink117" } -flink117-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink117" } -flink117-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink117" } flink118-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink118" } flink118-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink118" } flink118-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink118" } @@ -126,6 +120,12 @@ flink119-connector-files = { module = "org.apache.flink:flink-connector-files", flink119-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink119" } flink119-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink119" } flink119-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink119" } +flink120-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink120" } +flink120-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink120" } +flink120-connector-files = { module = "org.apache.flink:flink-connector-files", version.ref = "flink120" } +flink120-metrics-dropwizard = { module = "org.apache.flink:flink-metrics-dropwizard", version.ref = "flink120" } +flink120-streaming-java = { module = "org.apache.flink:flink-streaming-java", version.ref = "flink120" } +flink120-table-api-java-bridge = { module = "org.apache.flink:flink-table-api-java-bridge", version.ref = "flink120" } google-libraries-bom = { module = "com.google.cloud:libraries-bom", version.ref = "google-libraries-bom" } guava-guava = { module = "com.google.guava:guava", version.ref = "guava" } hadoop2-client = { module = "org.apache.hadoop:hadoop-client", version.ref = "hadoop2" } @@ -180,11 +180,6 @@ assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-cor awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } esotericsoftware-kryo = { module = "com.esotericsoftware:kryo", version.ref = "esotericsoftware-kryo" } -flink117-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink117" } -flink117-core = { module = "org.apache.flink:flink-core", version.ref = "flink117" } -flink117-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink117" } -flink117-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink117" } -flink117-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink117" } flink118-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink118" } flink118-core = { module = "org.apache.flink:flink-core", version.ref = "flink118" } flink118-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink118" } @@ -195,6 +190,11 @@ flink119-core = { module = "org.apache.flink:flink-core", version.ref = "flink11 flink119-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink119" } flink119-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink119" } flink119-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink119" } +flink120-connector-test-utils = { module = "org.apache.flink:flink-connector-test-utils", version.ref = "flink120" } +flink120-core = { module = "org.apache.flink:flink-core", version.ref = "flink120" } +flink120-runtime = { module = "org.apache.flink:flink-runtime", version.ref = "flink120" } +flink120-test-utils = { module = "org.apache.flink:flink-test-utils", version.ref = "flink120" } +flink120-test-utilsjunit = { module = "org.apache.flink:flink-test-utils-junit", version.ref = "flink120" } guava-testlib = { module = "com.google.guava:guava-testlib", version.ref = "guava" } jakarta-el-api = { module = "jakarta.el:jakarta.el-api", version.ref = "jakarta-el-api" } jakarta-servlet = {module = "jakarta.servlet:jakarta.servlet-api", version.ref = "jakarta-servlet-api"} diff --git a/jmh.gradle b/jmh.gradle index 5e5e0151219f..a5d8d624270d 100644 --- a/jmh.gradle +++ b/jmh.gradle @@ -26,16 +26,16 @@ def sparkVersions = (System.getProperty("sparkVersions") != null ? System.getPro def scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") def jmhProjects = [project(":iceberg-core"), project(":iceberg-data")] -if (flinkVersions.contains("1.16")) { - jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.16")) +if (flinkVersions.contains("1.18")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.18")) } -if (flinkVersions.contains("1.17")) { - jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.17")) +if (flinkVersions.contains("1.19")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.19")) } -if (flinkVersions.contains("1.18")) { - jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.18")) +if (flinkVersions.contains("1.20")) { + jmhProjects.add(project(":iceberg-flink:iceberg-flink-1.20")) } if (sparkVersions.contains("3.3")) { diff --git a/settings.gradle b/settings.gradle index cdc69b0e2071..1e6d92bf1e1f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -112,15 +112,6 @@ if (!flinkVersions.isEmpty()) { project(':flink').name = 'iceberg-flink' } -if (flinkVersions.contains("1.17")) { - include ":iceberg-flink:flink-1.17" - include ":iceberg-flink:flink-runtime-1.17" - project(":iceberg-flink:flink-1.17").projectDir = file('flink/v1.17/flink') - project(":iceberg-flink:flink-1.17").name = "iceberg-flink-1.17" - project(":iceberg-flink:flink-runtime-1.17").projectDir = file('flink/v1.17/flink-runtime') - project(":iceberg-flink:flink-runtime-1.17").name = "iceberg-flink-runtime-1.17" -} - if (flinkVersions.contains("1.18")) { include ":iceberg-flink:flink-1.18" include ":iceberg-flink:flink-runtime-1.18" @@ -139,6 +130,15 @@ if (flinkVersions.contains("1.19")) { project(":iceberg-flink:flink-runtime-1.19").name = "iceberg-flink-runtime-1.19" } +if (flinkVersions.contains("1.20")) { + include ":iceberg-flink:flink-1.20" + include ":iceberg-flink:flink-runtime-1.20" + project(":iceberg-flink:flink-1.20").projectDir = file('flink/v1.20/flink') + project(":iceberg-flink:flink-1.20").name = "iceberg-flink-1.20" + project(":iceberg-flink:flink-runtime-1.20").projectDir = file('flink/v1.20/flink-runtime') + project(":iceberg-flink:flink-runtime-1.20").name = "iceberg-flink-runtime-1.20" +} + if (sparkVersions.contains("3.3")) { include ":iceberg-spark:spark-3.3_${scalaVersion}" include ":iceberg-spark:spark-extensions-3.3_${scalaVersion}" From 20f85137fbf26ad722f59be917aff375b1312524 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 6 Aug 2024 18:19:12 +0200 Subject: [PATCH 0572/1019] Build: Add checkstyle rule to ban assert usage (#10886) --- .baseline/checkstyle/checkstyle.xml | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 1491a35db194..570fb84d70a4 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -414,6 +414,9 @@ + + + From ac904647e468cce433e4daf9171b76a84c98a162 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 7 Aug 2024 11:11:27 +0200 Subject: [PATCH 0573/1019] Build: Bump Apache Avro to 1.12.0 (#10879) --- 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 77e610e885f6..b2946163f38b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -25,7 +25,7 @@ aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.27" arrow = "15.0.2" -avro = "1.11.3" +avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.1" awssdk-bom = "2.26.29" From 43485cf587ce22f0a815859f88ff562e0f8460ec Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 7 Aug 2024 13:05:44 +0200 Subject: [PATCH 0574/1019] Spec: Fix rendering of unified partition struct (#10896) The angle brackets were without any escapes so docs renderer treated them as HTML. The resulting text on the website looked like an unfinished sentence: The unified partition type looks like Struct. Putting the angle brackets in backticks prevent them from being interpreted as HTML. Surrounding names like spec#0, field#1 are also put inside backticks for consistence. --- format/spec.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/format/spec.md b/format/spec.md index c3321fa6991c..c322f8174fe2 100644 --- a/format/spec.md +++ b/format/spec.md @@ -770,13 +770,13 @@ The unified partition type is a struct containing all fields that have ever been and sorted by the field ids in ascending order. In other words, the struct fields represent a union of all known partition fields sorted in ascending order by the field ids. For example, -1) spec#0 has two fields {field#1, field#2} -and then the table has evolved into spec#1 which has three fields {field#1, field#2, field#3}. -The unified partition type looks like Struct. +1) `spec#0` has two fields `{field#1, field#2}` +and then the table has evolved into `spec#1` which has three fields `{field#1, field#2, field#3}`. +The unified partition type looks like `Struct`. -2) spec#0 has two fields {field#1, field#2} -and then the table has evolved into spec#1 which has just one field {field#2}. -The unified partition type looks like Struct. +2) `spec#0` has two fields `{field#1, field#2}` +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 From 8aa8acbc2ad07b9ec0bfb48de6c132b25870e560 Mon Sep 17 00:00:00 2001 From: Tom Tanaka <43331405+tomtongue@users.noreply.github.com> Date: Wed, 7 Aug 2024 20:08:23 +0900 Subject: [PATCH 0575/1019] Docs: Fix catalog name for S3 MRAP example (#10897) --- docs/docs/aws.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/docs/aws.md b/docs/docs/aws.md index bba968fa5586..43e540c20673 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -468,8 +468,8 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata --conf spark.sql.catalog.my_catalog.type=glue \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.use-arn-region-enabled=false \ - --conf spark.sql.catalog.test.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ - --conf spark.sql.catalog.test.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap ``` For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap` access-point for all S3 operations. From ffe76fcdfb83c136b6820a76d35a4c97e3ea9500 Mon Sep 17 00:00:00 2001 From: Robert Stupp Date: Wed, 7 Aug 2024 16:07:38 +0200 Subject: [PATCH 0576/1019] Add Flink 1.20 & remove Flink 1.17 in stage-binaries.sh and docs (#10888) This is a follow-up to #10881 --- dev/stage-binaries.sh | 2 +- site/docs/multi-engine-support.md | 3 ++- site/docs/releases.md | 4 ++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index 05bf3c4253f2..29cf31e5f423 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -19,7 +19,7 @@ # SCALA_VERSION=2.12 -FLINK_VERSIONS=1.17,1.18,1.19 +FLINK_VERSIONS=1.18,1.19,1.20 SPARK_VERSIONS=3.3,3.4,3.5 HIVE_VERSIONS=2,3 diff --git a/site/docs/multi-engine-support.md b/site/docs/multi-engine-support.md index e4123cc579b3..a3c63276bfdb 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -90,9 +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 | {{ icebergVersion }} | [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 | 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.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 | {{ 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) | diff --git a/site/docs/releases.md b/site/docs/releases.md index cc29857ed802..6b48e31a0728 100644 --- a/site/docs/releases.md +++ b/site/docs/releases.md @@ -31,9 +31,9 @@ The latest version of Iceberg is [{{ icebergVersion }}](https://github.com/apach * [{{ icebergVersion }} Spark 3.4\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.4_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.4_2.13-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Spark 3.3\_with Scala 2.12 runtime Jar](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) * [{{ icebergVersion }} Spark 3.3\_with Scala 2.13 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-spark-runtime-3.3_2.13/{{ icebergVersion }}/iceberg-spark-runtime-3.3_2.13-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Flink 1.20 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.20/{{ icebergVersion }}/iceberg-flink-runtime-1.20-{{ icebergVersion }}.jar) +* [{{ icebergVersion }} Flink 1.19 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.19/{{ icebergVersion }}/iceberg-flink-runtime-1.19-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Flink 1.18 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.18/{{ icebergVersion }}/iceberg-flink-runtime-1.18-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Flink 1.17 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.17/{{ icebergVersion }}/iceberg-flink-runtime-1.17-{{ icebergVersion }}.jar) -* [{{ icebergVersion }} Flink 1.16 runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-flink-runtime-1.16/{{ icebergVersion }}/iceberg-flink-runtime-1.16-{{ icebergVersion }}.jar) * [{{ icebergVersion }} Hive runtime Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-hive-runtime/{{ icebergVersion }}/iceberg-hive-runtime-{{ icebergVersion }}.jar) * [{{ icebergVersion }} aws-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-aws-bundle/{{ icebergVersion }}/iceberg-aws-bundle-{{ icebergVersion }}.jar) * [{{ icebergVersion }} gcp-bundle Jar](https://search.maven.org/remotecontent?filepath=org/apache/iceberg/iceberg-gcp-bundle/{{ icebergVersion }}/iceberg-gcp-bundle-{{ icebergVersion }}.jar) From 68e5d96a2b9853955ff1a36902194937ef291dc4 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 7 Aug 2024 16:08:16 +0200 Subject: [PATCH 0577/1019] Flink: Remove deprecated RowDataUtil.clone method (#10902) Scheduled for removal in 1.7.0. --- .../apache/iceberg/flink/data/RowDataUtil.java | 18 ------------------ .../apache/iceberg/flink/data/RowDataUtil.java | 18 ------------------ .../apache/iceberg/flink/data/RowDataUtil.java | 18 ------------------ 3 files changed, 54 deletions(-) diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index 3a8f5ccc6c03..4bd85bbd97b4 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -102,22 +102,4 @@ public static RowData clone( return ret; } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index 3a8f5ccc6c03..4bd85bbd97b4 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -102,22 +102,4 @@ public static RowData clone( return ret; } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java index 3a8f5ccc6c03..4bd85bbd97b4 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/RowDataUtil.java @@ -102,22 +102,4 @@ public static RowData clone( return ret; } - - /** - * @deprecated will be removed in 1.7.0; Not reusing FieldGetter in this method could lead to - * performance degradation, use {@link #clone(RowData, RowData, RowType, TypeSerializer[], - * RowData.FieldGetter[])} instead. - */ - @Deprecated - public static RowData clone( - RowData from, RowData reuse, RowType rowType, TypeSerializer[] fieldSerializers) { - RowData.FieldGetter[] fieldGetters = new RowData.FieldGetter[rowType.getFieldCount()]; - for (int i = 0; i < rowType.getFieldCount(); ++i) { - if (!from.isNullAt(i)) { - fieldGetters[i] = RowData.createFieldGetter(rowType.getTypeAt(i), i); - } - } - - return clone(from, reuse, rowType, fieldSerializers, fieldGetters); - } } From 1b4cbe022c4a774bdce6de772aab1e30b6f3a8d3 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 8 Aug 2024 09:59:12 +0200 Subject: [PATCH 0578/1019] AWS: Fix flaky TestS3RestSigner (#10898) --- .../org/apache/iceberg/aws/s3/signer/S3SignerServlet.java | 4 ++-- .../apache/iceberg/aws/s3/signer/TestS3RestSigner.java | 8 +++++++- 2 files changed, 9 insertions(+), 3 deletions(-) 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 06c099e3be5e..ce7527af765c 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 @@ -148,7 +148,7 @@ private OAuthTokenResponse handleOAuth(Map requestMap) { .withToken("client-credentials-token:sub=" + requestMap.get("client_id")) .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") .withTokenType("Bearer") - .setExpirationInSeconds(100) + .setExpirationInSeconds(10000) .build()); case "urn:ietf:params:oauth:grant-type:token-exchange": @@ -163,7 +163,7 @@ private OAuthTokenResponse handleOAuth(Map requestMap) { .withToken(token) .withIssuedTokenType("urn:ietf:params:oauth:token-type:access_token") .withTokenType("Bearer") - .setExpirationInSeconds(100) + .setExpirationInSeconds(10000) .build()); default: 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 5e20b71e438c..08f356ca7ab1 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 @@ -107,7 +107,13 @@ public static void afterClass() throws Exception { ScheduledThreadPoolExecutor executor = ((ScheduledThreadPoolExecutor) validatingSigner.icebergSigner.tokenRefreshExecutor()); - // token expiration is set to 100s so there should be exactly one token scheduled for refresh + // token expiration is set to 10000s by the S3SignerServlet so there should be exactly one token + // scheduled for refresh. Such a high token expiration value is explicitly selected to be much + // larger than TestS3RestSigner would need to execute all tests. + // The reason why this check is done here with a high token expiration is to make sure that + // there aren't other token refreshes being scheduled after every sign request and after + // TestS3RestSigner completes all tests, there should be only this single token in the queue + // that is scheduled for refresh assertThat(executor.getPoolSize()).isEqualTo(1); assertThat(executor.getQueue()) .as("should only have a single token scheduled for refresh") From f24437f4fb65364417f1dff777c590ce4268e547 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 8 Aug 2024 14:35:17 -0700 Subject: [PATCH 0579/1019] AWS: Implement SupportsRecoveryOperations mixin for S3FileIO (#10721) --- .../apache/iceberg/aws/AwsIntegTestUtil.java | 62 ++++++++++++------- .../aws/s3/TestS3FileIOIntegration.java | 38 ++++++++++++ .../org/apache/iceberg/aws/s3/S3FileIO.java | 50 ++++++++++++++- 3 files changed, 128 insertions(+), 22 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 bbe062d5db48..7e0ca6ed10b2 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; @@ -30,9 +31,10 @@ import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.Delete; import software.amazon.awssdk.services.s3.model.DeleteObjectsRequest; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Request; -import software.amazon.awssdk.services.s3.model.ListObjectsV2Response; +import software.amazon.awssdk.services.s3.model.ListObjectVersionsRequest; 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.s3control.S3ControlClient; import software.amazon.awssdk.services.s3control.model.CreateAccessPointRequest; import software.amazon.awssdk.services.s3control.model.DeleteAccessPointRequest; @@ -94,28 +96,46 @@ public static String testAccountId() { } public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { - boolean hasContent = true; - while (hasContent) { - ListObjectsV2Response response = - s3.listObjectsV2( - ListObjectsV2Request.builder().bucket(bucketName).prefix(prefix).build()); - hasContent = response.hasContents(); - if (hasContent) { - s3.deleteObjects( - DeleteObjectsRequest.builder() - .bucket(bucketName) - .delete( - Delete.builder() - .objects( - response.contents().stream() - .map(obj -> ObjectIdentifier.builder().key(obj.key()).build()) - .collect(Collectors.toList())) - .build()) - .build()); - } + 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) { + deleteObjectVersions(s3, bucketName, versionsToDelete); + versionsToDelete.clear(); + } + }); + + if (!versionsToDelete.isEmpty()) { + deleteObjectVersions(s3, bucketName, versionsToDelete); } } + private static void deleteObjectVersions( + S3Client s3, String bucket, List objectVersions) { + s3.deleteObjects( + DeleteObjectsRequest.builder() + .bucket(bucket) + .delete( + Delete.builder() + .objects( + objectVersions.stream() + .map( + obj -> + ObjectIdentifier.builder() + .key(obj.key()) + .versionId(obj.versionId()) + .build()) + .collect(Collectors.toList())) + .build()) + .build()); + } + public static void cleanGlueCatalog(GlueClient glue, List namespaces) { for (String namespace : namespaces) { try { 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 18abb82ce74a..cacf04891896 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 @@ -53,14 +53,17 @@ import software.amazon.awssdk.services.kms.model.ListAliasesResponse; import software.amazon.awssdk.services.kms.model.ScheduleKeyDeletionRequest; import software.amazon.awssdk.services.s3.S3Client; +import software.amazon.awssdk.services.s3.model.BucketVersioningStatus; import software.amazon.awssdk.services.s3.model.GetObjectAclRequest; import software.amazon.awssdk.services.s3.model.GetObjectAclResponse; import software.amazon.awssdk.services.s3.model.GetObjectRequest; import software.amazon.awssdk.services.s3.model.GetObjectResponse; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; import software.amazon.awssdk.services.s3.model.Permission; +import software.amazon.awssdk.services.s3.model.PutBucketVersioningRequest; import software.amazon.awssdk.services.s3.model.PutObjectRequest; import software.amazon.awssdk.services.s3.model.ServerSideEncryption; +import software.amazon.awssdk.services.s3.model.VersioningConfiguration; import software.amazon.awssdk.services.s3control.S3ControlClient; import software.amazon.awssdk.utils.ImmutableMap; import software.amazon.awssdk.utils.IoUtils; @@ -106,6 +109,12 @@ public static void beforeClass() { AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); AwsIntegTestUtil.createAccessPoint( crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.ENABLED).build()) + .build()); } @AfterAll @@ -445,6 +454,35 @@ public void testPrefixDelete() { }); } + @Test + public void testFileRecoveryHappyPath() throws Exception { + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); + String filePath = String.format("s3://%s/%s/%s", bucketName, prefix, "someFile.parquet"); + write(s3FileIO, filePath); + s3FileIO.deleteFile(filePath); + assertThat(s3FileIO.newInputFile(filePath).exists()).isFalse(); + + assertThat(s3FileIO.recoverFile(filePath)).isTrue(); + assertThat(s3FileIO.newInputFile(filePath).exists()).isTrue(); + } + + @Test + public void testFileRecoveryFailsToRecover() throws Exception { + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3, new S3FileIOProperties()); + s3.putBucketVersioning( + PutBucketVersioningRequest.builder() + .bucket(bucketName) + .versioningConfiguration( + VersioningConfiguration.builder().status(BucketVersioningStatus.SUSPENDED).build()) + .build()); + String filePath = String.format("s3://%s/%s/%s", bucketName, prefix, "unversionedFile.parquet"); + write(s3FileIO, filePath); + s3FileIO.deleteFile(filePath); + assertThat(s3FileIO.newInputFile(filePath).exists()).isFalse(); + + assertThat(s3FileIO.recoverFile(filePath)).isFalse(); + } + private S3FileIOProperties getDeletionTestProperties() { S3FileIOProperties properties = new S3FileIOProperties(); properties.setDeleteBatchSize(deletionBatchSize); 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 dd13e13f01a6..f7d2da5eb907 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 @@ -20,8 +20,10 @@ import java.util.Arrays; import java.util.Collection; +import java.util.Comparator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; @@ -37,6 +39,7 @@ import org.apache.iceberg.io.FileInfo; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SupportsRecoveryOperations; import org.apache.iceberg.metrics.MetricsContext; import org.apache.iceberg.relocated.com.google.common.base.Joiner; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -52,6 +55,7 @@ import org.apache.iceberg.util.ThreadPools; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import software.amazon.awssdk.core.exception.SdkException; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.model.Delete; import software.amazon.awssdk.services.s3.model.DeleteObjectRequest; @@ -61,10 +65,12 @@ import software.amazon.awssdk.services.s3.model.GetObjectTaggingResponse; 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.model.PutObjectTaggingRequest; import software.amazon.awssdk.services.s3.model.S3Exception; import software.amazon.awssdk.services.s3.model.Tag; import software.amazon.awssdk.services.s3.model.Tagging; +import software.amazon.awssdk.services.s3.paginators.ListObjectVersionsIterable; /** * FileIO implementation backed by S3. @@ -73,7 +79,7 @@ * schemes s3a, s3n, https are also treated as s3 file paths. Using this FileIO with other schemes * will result in {@link org.apache.iceberg.exceptions.ValidationException}. */ -public class S3FileIO implements CredentialSupplier, DelegateFileIO { +public class S3FileIO implements CredentialSupplier, DelegateFileIO, SupportsRecoveryOperations { private static final Logger LOG = LoggerFactory.getLogger(S3FileIO.class); private static final String DEFAULT_METRICS_IMPL = "org.apache.iceberg.hadoop.HadoopMetricsContext"; @@ -420,4 +426,46 @@ protected void finalize() throws Throwable { } } } + + @Override + public boolean recoverFile(String path) { + S3URI location = new S3URI(path, s3FileIOProperties.bucketToAccessPointMapping()); + ListObjectVersionsIterable response = + client() + .listObjectVersionsPaginator( + builder -> builder.bucket(location.bucket()).prefix(location.key())); + + // Recover to the last modified version, not isLatest, + // since isLatest is true for deletion markers. + Optional recoverVersion = + response.versions().stream().max(Comparator.comparing(ObjectVersion::lastModified)); + + return recoverVersion.map(version -> recoverObject(version, location.bucket())).orElse(false); + } + + private boolean recoverObject(ObjectVersion version, String bucket) { + if (version.isLatest()) { + return true; + } + + LOG.info("Attempting to recover object {}", version.key()); + try { + // Perform a copy instead of deleting the delete marker + // so that recovery does not rely on delete permissions + client() + .copyObject( + builder -> + builder + .sourceBucket(bucket) + .sourceKey(version.key()) + .sourceVersionId(version.versionId()) + .destinationBucket(bucket) + .destinationKey(version.key())); + } catch (SdkException e) { + LOG.warn("Failed to recover object {}", version.key(), e); + return false; + } + + return true; + } } From 80232d10900214d64d22d1c0944bf373643be45e Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Fri, 9 Aug 2024 15:04:36 +0530 Subject: [PATCH 0580/1019] Core: Remove deprecated APIs for 1.7.0 (#10818) --- .palantir/revapi.yml | 75 +++++++++++++++++++ .../iceberg/common/DynConstructors.java | 18 +---- .../org/apache/iceberg/common/DynFields.java | 15 ---- .../org/apache/iceberg/common/DynMethods.java | 38 +--------- .../iceberg/BaseMetastoreTableOperations.java | 11 --- .../apache/iceberg/FileScanTaskParser.java | 29 ------- .../org/apache/iceberg/SnapshotProducer.java | 12 --- .../org/apache/iceberg/io/ContentCache.java | 25 +------ .../apache/iceberg/rest/auth/OAuth2Util.java | 20 ----- .../iceberg/TestFileScanTaskParser.java | 30 -------- .../iceberg/hive/HiveOperationsBase.java | 10 --- 11 files changed, 81 insertions(+), 202 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 3018840b4513..e58ce70ded7a 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1056,6 +1056,81 @@ acceptedBreaks: - code: "java.method.removed" old: "method org.apache.iceberg.DataFiles.Builder org.apache.iceberg.DataFiles.Builder::withEqualityFieldIds(java.util.List)" justification: "Deprecations for 1.6.0 release" + "1.6.0": + org.apache.iceberg:iceberg-common: + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynFields.StaticField org.apache.iceberg.common.DynFields.Builder::buildStaticChecked()\ + \ throws java.lang.NoSuchFieldException" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method java.lang.Class org.apache.iceberg.common.DynConstructors.Ctor::getConstructedClass()" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynConstructors.Builder org.apache.iceberg.common.DynConstructors.Builder::hiddenImpl(java.lang.Class[])" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynMethods.Builder org.apache.iceberg.common.DynMethods.Builder::ctorImpl(java.lang.Class,\ + \ java.lang.Class[])" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.common.DynMethods.Builder org.apache.iceberg.common.DynMethods.Builder::ctorImpl(java.lang.String,\ + \ java.lang.Class[])" + justification: "Removing deprecated code" + - code: "java.method.visibilityReduced" + old: "method R org.apache.iceberg.common.DynMethods.UnboundMethod::invokeChecked(java.lang.Object,\ + \ java.lang.Object[]) throws java.lang.Exception" + new: "method R org.apache.iceberg.common.DynMethods.UnboundMethod::invokeChecked(java.lang.Object,\ + \ java.lang.Object[]) throws java.lang.Exception" + justification: "Reduced visibility and scoped to package" + org.apache.iceberg:iceberg-core: + - code: "java.class.removed" + old: "enum org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method java.lang.String org.apache.iceberg.FileScanTaskParser::toJson(org.apache.iceberg.FileScanTask)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.FileScanTask org.apache.iceberg.FileScanTaskParser::fromJson(java.lang.String,\ + \ boolean)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.ContentCache.CacheEntry org.apache.iceberg.io.ContentCache::get(java.lang.String,\ + \ java.util.function.Function)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.ContentCache.CacheEntry org.apache.iceberg.io.ContentCache::getIfPresent(java.lang.String)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.InputFile org.apache.iceberg.io.ContentCache::tryCache(org.apache.iceberg.io.FileIO,\ + \ java.lang.String, long)" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.BaseOverwriteFiles" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.BaseReplacePartitions" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.BaseRewriteManifests" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method org.apache.iceberg.io.OutputFile org.apache.iceberg.SnapshotProducer::newManifestOutput()\ + \ @ org.apache.iceberg.StreamingDelete" + justification: "Removing deprecated code" + - code: "java.method.removed" + old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, java.lang.String, java.lang.String, java.lang.String,\ + \ java.lang.String, java.lang.String)" + justification: "Removing deprecated code" + - code: "java.method.returnTypeChanged" + old: "method org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\ + \ org.apache.iceberg.TableMetadata)" + new: "method org.apache.iceberg.BaseMetastoreOperations.CommitStatus org.apache.iceberg.BaseMetastoreTableOperations::checkCommitStatus(java.lang.String,\ + \ org.apache.iceberg.TableMetadata)" + justification: "Removing deprecated code" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java index 7c777112871a..7ec8716c86a4 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java +++ b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java @@ -43,12 +43,6 @@ private Ctor(Constructor constructor, Class constructed) { this.constructed = constructed; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Class getConstructedClass() { - return constructed; - } - public C newInstanceChecked(Object... args) throws Exception { try { if (args.length > ctor.getParameterCount()) { @@ -82,6 +76,8 @@ public R invoke(Object target, Object... args) { return (R) newInstance(args); } + /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + @Deprecated // will become package-private @Override @SuppressWarnings("unchecked") public R invokeChecked(Object target, Object... args) throws Exception { @@ -172,16 +168,6 @@ public Builder impl(Class targetClass, Class... types) { return this; } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; This varargs method conflicts with {@link - * #hiddenImpl(Class, Class...)}. Use {@link #builder(Class)} instead. - */ - @Deprecated - public Builder hiddenImpl(Class... types) { - hiddenImpl(baseClass, types); - return this; - } - public Builder hiddenImpl(String className, Class... types) { // don't do any work if an implementation has been found if (ctor != null) { diff --git a/common/src/main/java/org/apache/iceberg/common/DynFields.java b/common/src/main/java/org/apache/iceberg/common/DynFields.java index e88affa0cbdd..cc397d329e94 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynFields.java +++ b/common/src/main/java/org/apache/iceberg/common/DynFields.java @@ -386,21 +386,6 @@ public BoundField build(Object target) { return this.build().bind(target); } - /** - * Returns the first valid implementation as a StaticField or throws a NoSuchFieldException if - * there is none. - * - * @param Java class stored in the field - * @return a {@link StaticField} with a valid implementation - * @throws IllegalStateException if the method is not static - * @throws NoSuchFieldException if no implementation was found - * @deprecated since 1.6.0, will be removed in 1.7.0 - */ - @Deprecated - public StaticField buildStaticChecked() throws NoSuchFieldException { - return this.buildChecked().asStatic(); - } - /** * Returns the first valid implementation as a StaticField or throws a RuntimeException if there * is none. diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index fc0e578c7dd1..65a69bd0e12c 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -51,10 +51,8 @@ public static class UnboundMethod { (method == null || method.isVarArgs()) ? -1 : method.getParameterTypes().length; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated // will become private @SuppressWarnings("unchecked") - public R invokeChecked(Object target, Object... args) throws Exception { + R invokeChecked(Object target, Object... args) throws Exception { try { if (argLength < 0) { return (R) method.invoke(target, args); @@ -127,6 +125,8 @@ public String toString() { /** Singleton {@link UnboundMethod}, performs no operation and returns null. */ private static final UnboundMethod NOOP = new UnboundMethod(null, "NOOP") { + /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + @Deprecated // will become package-private @Override public R invokeChecked(Object target, Object... args) { return null; @@ -315,38 +315,6 @@ public Builder impl(Class targetClass, Class... argClasses) { return this; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Builder ctorImpl(Class targetClass, Class... argClasses) { - // don't do any work if an implementation has been found - if (method != null) { - return this; - } - - try { - this.method = new DynConstructors.Builder().impl(targetClass, argClasses).buildChecked(); - } catch (NoSuchMethodException e) { - // not the right implementation - } - return this; - } - - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public Builder ctorImpl(String className, Class... argClasses) { - // don't do any work if an implementation has been found - if (method != null) { - return this; - } - - try { - this.method = new DynConstructors.Builder().impl(className, argClasses).buildChecked(); - } catch (NoSuchMethodException e) { - // not the right implementation - } - return this; - } - /** * Checks for an implementation, first finding the given class by name. * diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 90d435811020..5c82bc877a15 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -284,17 +284,6 @@ public long newSnapshotId() { }; } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link - * BaseMetastoreOperations.CommitStatus} instead - */ - @Deprecated - protected enum CommitStatus { - FAILURE, - SUCCESS, - UNKNOWN - } - /** * Attempt to load the table and see if any current or past metadata location matches the one we * were attempting to set. This is used as a last resort when we are dealing with exceptions that diff --git a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java index a6ea41319f4e..7ae7dc74a72e 100644 --- a/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java +++ b/core/src/main/java/org/apache/iceberg/FileScanTaskParser.java @@ -40,35 +40,6 @@ public class FileScanTaskParser { private FileScanTaskParser() {} - /** - * Serialize file scan task to JSON string - * - * @deprecated will be removed in 1.7.0; use {@link ScanTaskParser#toJson(FileScanTask)} instead - */ - @Deprecated - public static String toJson(FileScanTask fileScanTask) { - Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); - return JsonUtil.generate( - generator -> { - generator.writeStartObject(); - toJson(fileScanTask, generator); - generator.writeEndObject(); - }, - false); - } - - /** - * Deserialize file scan task from JSON string - * - * @deprecated will be removed in 1.7.0; use {@link ScanTaskParser#fromJson(String, boolean)} - * instead - */ - @Deprecated - public static FileScanTask fromJson(String json, boolean caseSensitive) { - Preconditions.checkArgument(json != null, "Invalid JSON string for file scan task: null"); - return JsonUtil.parse(json, node -> fromJson(node, caseSensitive)); - } - static void toJson(FileScanTask fileScanTask, JsonGenerator generator) throws IOException { Preconditions.checkArgument(fileScanTask != null, "Invalid file scan task: null"); Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 0a040fe34471..74997cc89849 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -500,18 +500,6 @@ protected OutputFile manifestListPath() { "snap-%d-%d-%s", snapshotId(), attempt.incrementAndGet(), commitUUID)))); } - /** - * @deprecated will be removed in 1.7.0; Use {@link SnapshotProducer#newManifestOutputFile} - * instead - */ - @Deprecated - protected OutputFile newManifestOutput() { - return ops.io() - .newOutputFile( - ops.metadataFileLocation( - FileFormat.AVRO.addExtension(commitUUID + "-m" + manifestCount.getAndIncrement()))); - } - protected EncryptedOutputFile newManifestOutputFile() { String manifestFileLocation = ops.metadataFileLocation( 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 7942c69d5d77..ce37cfb08934 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -28,7 +28,6 @@ import java.nio.ByteBuffer; import java.time.Duration; import java.util.List; -import java.util.function.Function; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -111,24 +110,6 @@ public CacheStats stats() { return cache.stats(); } - /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ - @Deprecated - public CacheEntry get(String key, Function mappingFunction) { - return cache.get(key, mappingFunction); - } - - /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ - @Deprecated - public CacheEntry getIfPresent(String location) { - return cache.getIfPresent(location); - } - - /** @deprecated will be removed in 1.7; use {@link #tryCache(InputFile)} instead */ - @Deprecated - public InputFile tryCache(FileIO io, String location, long length) { - return tryCache(io.newInputFile(location, length)); - } - /** * Try cache the file-content of file in the given location upon stream reading. * @@ -173,11 +154,7 @@ public String toString() { .toString(); } - /** @deprecated will be removed in 1.7; use {@link FileContent} instead. */ - @Deprecated - private static class CacheEntry {} - - private static class FileContent extends CacheEntry { + private static class FileContent { private final long length; private final List buffers; diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 189e5fde2cad..52c89af9d474 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -465,26 +465,6 @@ public AuthSession(Map baseHeaders, AuthConfig config) { this.config = config; } - /** @deprecated since 1.6.0, will be removed in 1.7.0 */ - @Deprecated - public AuthSession( - Map baseHeaders, - String token, - String tokenType, - String credential, - String scope, - String oauth2ServerUri) { - this( - baseHeaders, - AuthConfig.builder() - .token(token) - .tokenType(tokenType) - .credential(credential) - .scope(scope) - .oauth2ServerUri(oauth2ServerUri) - .build()); - } - public Map headers() { return headers; } diff --git a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java index 137e7897385b..c4a9fdf2340a 100644 --- a/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java +++ b/core/src/test/java/org/apache/iceberg/TestFileScanTaskParser.java @@ -31,14 +31,6 @@ public class TestFileScanTaskParser { @Test public void testNullArguments() { - assertThatThrownBy(() -> FileScanTaskParser.toJson(null)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file scan task: null"); - - assertThatThrownBy(() -> FileScanTaskParser.fromJson((String) null, true)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid JSON string for file scan task: null"); - assertThatThrownBy(() -> ScanTaskParser.toJson(null)) .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid scan task: null"); @@ -48,28 +40,6 @@ public void testNullArguments() { .hasMessage("Invalid JSON string for scan task: null"); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testFileScanTaskParser(boolean caseSensitive) { - PartitionSpec spec = TestBase.SPEC; - FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); - String jsonStr = FileScanTaskParser.toJson(fileScanTask); - assertThat(jsonStr).isEqualTo(fileScanTaskJsonWithoutTaskType()); - FileScanTask deserializedTask = FileScanTaskParser.fromJson(jsonStr, caseSensitive); - assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); - } - - /** Test backward compatibility where task-type field is absent from the JSON string */ - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testFileScanTaskParserWithoutTaskTypeField(boolean caseSensitive) { - PartitionSpec spec = TestBase.SPEC; - FileScanTask fileScanTask = createFileScanTask(spec, caseSensitive); - FileScanTask deserializedTask = - FileScanTaskParser.fromJson(fileScanTaskJsonWithoutTaskType(), caseSensitive); - assertFileScanTaskEquals(fileScanTask, deserializedTask, spec, caseSensitive); - } - @ParameterizedTest @ValueSource(booleans = {true, false}) public void testScanTaskParser(boolean caseSensitive) { diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java index 055a14246e77..6500e724a4f0 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -31,7 +31,6 @@ import org.apache.iceberg.ClientPool; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; -import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.NoSuchIcebergTableException; import org.apache.iceberg.io.FileIO; @@ -150,15 +149,6 @@ default void persistTable(Table hmsTable, boolean updateHiveTable, String metada } } - /** - * @deprecated since 1.6.0, will be removed in 1.7.0; Use {@link #storageDescriptor(Schema, - * String, boolean)} instead - */ - @Deprecated - static StorageDescriptor storageDescriptor(TableMetadata metadata, boolean hiveEngineEnabled) { - return storageDescriptor(metadata.schema(), metadata.location(), hiveEngineEnabled); - } - static StorageDescriptor storageDescriptor( Schema schema, String location, boolean hiveEngineEnabled) { final StorageDescriptor storageDescriptor = new StorageDescriptor(); From 4c7bdc5ad2d07410862c6b678443a5e8d17eb38d Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Fri, 9 Aug 2024 20:21:00 +0530 Subject: [PATCH 0581/1019] Core, Flink: Fix build warnings (#10899) --- core/src/main/java/org/apache/iceberg/BaseEntriesTable.java | 2 +- .../org/apache/iceberg/actions/SizeBasedFileRewriter.java | 3 ++- .../iceberg/rest/ExponentialHttpRequestRetryStrategy.java | 2 +- core/src/main/java/org/apache/iceberg/util/Pair.java | 5 ----- .../main/java/org/apache/iceberg/util/ParallelIterable.java | 1 + core/src/main/java/org/apache/iceberg/util/Tasks.java | 4 +++- .../java/org/apache/iceberg/view/BaseViewOperations.java | 1 + .../flink/sink/shuffle/MapRangePartitionerBenchmark.java | 3 ++- 8 files changed, 11 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index f4019d688cb8..526bb42ea687 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -262,7 +262,7 @@ private boolean fileContent(BoundReference ref) { return ref.fieldId() == DataFile.CONTENT.fieldId(); } - private boolean contentMatch(Integer fileContentId) { + private boolean contentMatch(Integer fileContentId) { if (FileContent.DATA.id() == fileContentId) { return ManifestContent.DATA.id() == manifestContentId; } else if (FileContent.EQUALITY_DELETES.id() == fileContentId diff --git a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java index fb3c27220cb2..cea7003c1a38 100644 --- a/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java +++ b/core/src/main/java/org/apache/iceberg/actions/SizeBasedFileRewriter.java @@ -229,7 +229,8 @@ protected long numOutputFiles(long inputSize) { // the remainder file is of a valid size for this rewrite so keep it return numFilesWithRemainder; - } else if (avgFileSizeWithoutRemainder < Math.min(1.1 * targetFileSize, writeMaxFileSize())) { + } else if (avgFileSizeWithoutRemainder + < Math.min(1.1 * targetFileSize, (double) writeMaxFileSize())) { // if the reminder is distributed amongst other files, // the average file size will be no more than 10% bigger than the target file size // so round down and distribute remainder amongst other files diff --git a/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java b/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java index aadb97bc7112..263b3c305af0 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java +++ b/core/src/main/java/org/apache/iceberg/rest/ExponentialHttpRequestRetryStrategy.java @@ -149,7 +149,7 @@ public TimeValue getRetryInterval(HttpResponse response, int execCount, HttpCont } } - int delayMillis = 1000 * (int) Math.min(Math.pow(2.0, (long) execCount - 1), 64.0); + int delayMillis = 1000 * (int) Math.min(Math.pow(2.0, (long) execCount - 1.0), 64.0); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMillis * 0.1))); return TimeValue.ofMilliseconds(delayMillis + jitter); diff --git a/core/src/main/java/org/apache/iceberg/util/Pair.java b/core/src/main/java/org/apache/iceberg/util/Pair.java index bd3a934f6f04..e36321c8e2c9 100644 --- a/core/src/main/java/org/apache/iceberg/util/Pair.java +++ b/core/src/main/java/org/apache/iceberg/util/Pair.java @@ -58,11 +58,6 @@ public Schema load(Pair, Class> key) { private X first; private Y second; - /** Constructor used by Avro */ - private Pair(Schema schema) { - this.schema = schema; - } - private Pair(X first, Y second) { this.first = first; this.second = second; diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 16fa6f3d8537..27cd96a39733 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -101,6 +101,7 @@ private ParallelIterator( } @Override + @SuppressWarnings("FutureReturnValueIgnored") public void close() { // close first, avoid new task submit this.closed.set(true); 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 02d2b834311f..14804e040755 100644 --- a/core/src/main/java/org/apache/iceberg/util/Tasks.java +++ b/core/src/main/java/org/apache/iceberg/util/Tasks.java @@ -450,7 +450,9 @@ private void runTaskWithRetry(Task task, I item) thr } int delayMs = - (int) Math.min(minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), maxSleepTimeMs); + (int) + Math.min( + minSleepTimeMs * Math.pow(scaleFactor, attempt - 1), (double) maxSleepTimeMs); int jitter = ThreadLocalRandom.current().nextInt(Math.max(1, (int) (delayMs * 0.1))); LOG.warn("Retrying task after failure: {}", e.getMessage(), e); diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java index 18b452f98367..df96b90eb728 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -102,6 +102,7 @@ public ViewMetadata refresh() { } @Override + @SuppressWarnings("ImmutablesReferenceEquality") public void commit(ViewMetadata base, ViewMetadata metadata) { // if the metadata is already out of date, reject it if (base != current()) { diff --git a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 007b423e592a..592e7ff16241 100644 --- a/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.20/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -139,7 +140,7 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + return prefix + new String(buffer, StandardCharsets.UTF_8); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ From 1454ea83152014ee41c23af82254322082dea81a Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 12 Aug 2024 14:12:20 +0800 Subject: [PATCH 0582/1019] Build: Bump Spark 3.5 to 3.5.2 (#10918) --- 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 b2946163f38b..1cc38d44ac9d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -81,7 +81,7 @@ slf4j = "1.7.36" snowflake-jdbc = "3.18.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" -spark-hive35 = "3.5.1" +spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.37" sqlite-jdbc = "3.46.0.1" From c604d1c291e2298d463aa1a6d80ed075c3222111 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 15:50:50 +0200 Subject: [PATCH 0583/1019] Build: Bump com.google.errorprone:error_prone_annotations (#10915) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.29.2 to 2.30.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.29.2...v2.30.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 1cc38d44ac9d..52b5ec2b8863 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -37,7 +37,7 @@ datasketches = "6.0.0" delta-standalone = "3.2.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.29.2" +errorprone-annotations = "2.30.0" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} From c969313bc5b5c17ef893f7af7ccb01722f54aa88 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 15:51:15 +0200 Subject: [PATCH 0584/1019] Build: Bump org.awaitility:awaitility from 4.2.1 to 4.2.2 (#10912) Bumps [org.awaitility:awaitility](https://github.com/awaitility/awaitility) from 4.2.1 to 4.2.2. - [Changelog](https://github.com/awaitility/awaitility/blob/master/changelog.txt) - [Commits](https://github.com/awaitility/awaitility/compare/awaitility-4.2.1...awaitility-4.2.2) --- updated-dependencies: - dependency-name: org.awaitility:awaitility 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 52b5ec2b8863..f7456dae7f1d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -27,7 +27,7 @@ aircompressor = "0.27" arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" -awaitility = "4.2.1" +awaitility = "4.2.2" awssdk-bom = "2.26.29" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" From 06cea8b05e2f3649746a882c150b33b02dd2be90 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 17:58:48 +0200 Subject: [PATCH 0585/1019] Build: Bump datamodel-code-generator from 0.25.8 to 0.25.9 (#10917) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.8 to 0.25.9. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.8...0.25.9) --- 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 f6233ff2ace4..77d4f18331ca 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.25.8 +datamodel-code-generator==0.25.9 From 09fde7ae131a5af738e67f3330c3e9fe48f14c27 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 17:59:27 +0200 Subject: [PATCH 0586/1019] Build: Bump nessie from 0.94.4 to 0.95.0 (#10910) Bumps `nessie` from 0.94.4 to 0.95.0. Updates `org.projectnessie.nessie:nessie-client` from 0.94.4 to 0.95.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.94.4 to 0.95.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.94.4 to 0.95.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.94.4 to 0.95.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 f7456dae7f1d..c963ee2c82a4 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -67,7 +67,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.94.4" +nessie = "0.95.0" netty-buffer = "4.1.112.Final" netty-buffer-compat = "4.1.112.Final" object-client-bundle = "3.3.2" From 88fe4b7fb0f222a1b65681119b5457faa8460e65 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 12 Aug 2024 20:51:34 +0200 Subject: [PATCH 0587/1019] Docs: Add Trademark symbol where appropriate (#10921) --- site/docs/hive-quickstart.md | 5 +---- site/docs/spark-quickstart.md | 4 +--- site/mkdocs.yml | 2 +- site/overrides/home.html | 4 ++-- site/overrides/partials/header.html | 7 ++----- 5 files changed, 7 insertions(+), 15 deletions(-) diff --git a/site/docs/hive-quickstart.md b/site/docs/hive-quickstart.md index c110d56f6310..7db266d66624 100644 --- a/site/docs/hive-quickstart.md +++ b/site/docs/hive-quickstart.md @@ -18,10 +18,7 @@ title: "Hive and Iceberg Quickstart" - limitations under the License. --> - -## Hive and Iceberg Quickstart - -This guide will get you up and running with an Iceberg and Hive environment, including sample code to +This guide will get you up and running with Apache Iceberg™ using Apache Hive™, including sample code to highlight some powerful features. You can learn more about Iceberg's Hive runtime by checking out the [Hive](docs/latest/hive.md) section. - [Docker Images](#docker-images) diff --git a/site/docs/spark-quickstart.md b/site/docs/spark-quickstart.md index 5a940009f9a3..e98bedb49825 100644 --- a/site/docs/spark-quickstart.md +++ b/site/docs/spark-quickstart.md @@ -18,9 +18,7 @@ title: "Spark and Iceberg Quickstart" - limitations under the License. --> -## Spark and Iceberg Quickstart - -This guide will get you up and running with an Iceberg and Spark environment, including sample code to +This guide will get you up and running with Apache Iceberg™ using Apache Spark™, including sample code to highlight some powerful features. You can learn more about Iceberg's Spark runtime by checking out the [Spark](docs/latest/spark-ddl.md) section. - [Docker-Compose](#docker-compose) diff --git a/site/mkdocs.yml b/site/mkdocs.yml index 3c6782bc850a..d652ec3f0830 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -17,7 +17,7 @@ INHERIT: ./nav.yml -site_name: Apache Iceberg +site_name: "Apache Iceberg™" theme: custom_dir: overrides diff --git a/site/overrides/home.html b/site/overrides/home.html index f6479d46ea9b..65d971e0a134 100644 --- a/site/overrides/home.html +++ b/site/overrides/home.html @@ -34,7 +34,7 @@

    -

    Apache Iceberg

    +

    Apache Iceberg™

    The open table format for analytic datasets.


      @@ -61,7 +61,7 @@

      The open table format for analytic datasets.



      -

      What is Iceberg?

      +

      What is Apache Iceberg™?


      diff --git a/site/overrides/partials/header.html b/site/overrides/partials/header.html index 6d943e0c0888..62b02698b62e 100644 --- a/site/overrides/partials/header.html +++ b/site/overrides/partials/header.html @@ -50,13 +50,10 @@

      - + {{ config.site_name }}
      - - {% if page.meta and page.meta.title %} {{ page.meta.title }} {% else - %} {{ page.title }} {% endif %} - + {{ config.site_name }}
      From dd570871c8fb4f396621b39ef26ec5cb360578b6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 23:53:48 +0200 Subject: [PATCH 0588/1019] Build: Bump com.google.cloud:libraries-bom from 26.43.0 to 26.44.0 (#10916) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.43.0 to 26.44.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.43.0...v26.44.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 c963ee2c82a4..647f92dbbc28 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -42,7 +42,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.43.0" +google-libraries-bom = "26.44.0" guava = "33.2.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" From 75487cb0065a9c8514b39bb9104620480d5789f7 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 23:54:25 +0200 Subject: [PATCH 0589/1019] Build: Bump org.apache.commons:commons-compress from 1.26.2 to 1.27.0 (#10914) Bumps org.apache.commons:commons-compress from 1.26.2 to 1.27.0. --- updated-dependencies: - dependency-name: org.apache.commons:commons-compress 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> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index a0e55f23405d..8dbbfc729e11 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -71,7 +71,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' force 'org.xerial.snappy:snappy-java:1.1.10.5' - force 'org.apache.commons:commons-compress:1.26.2' + force 'org.apache.commons:commons-compress:1.27.0' force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' } } From 7d436f7db332005195bdf2089d0e75ab8a7a595b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Aug 2024 23:55:59 +0200 Subject: [PATCH 0590/1019] Build: Bump software.amazon.awssdk:bom from 2.26.29 to 2.27.2 (#10913) Bumps software.amazon.awssdk:bom from 2.26.29 to 2.27.2. --- 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 647f92dbbc28..8ae69c566f2e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.26.29" +awssdk-bom = "2.27.2" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 0344c93b67a4d5b105c0aefd76f6bd47ed766d04 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 13 Aug 2024 09:25:34 +0200 Subject: [PATCH 0591/1019] Build: Bump org.xerial.snappy:snappy-java from 1.1.10.5 to 1.1.10.6 (#10911) Bumps [org.xerial.snappy:snappy-java](https://github.com/xerial/snappy-java) from 1.1.10.5 to 1.1.10.6. - [Release notes](https://github.com/xerial/snappy-java/releases) - [Commits](https://github.com/xerial/snappy-java/compare/v1.1.10.5...v1.1.10.6) --- updated-dependencies: - dependency-name: org.xerial.snappy:snappy-java 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> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 8dbbfc729e11..60aaeee8c6f1 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -70,7 +70,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { // force upgrades for dependencies with known vulnerabilities... resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' - force 'org.xerial.snappy:snappy-java:1.1.10.5' + force 'org.xerial.snappy:snappy-java:1.1.10.6' force 'org.apache.commons:commons-compress:1.27.0' force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' } From d569828d1811d349acf52bed40c6c76a7986e681 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Tue, 13 Aug 2024 12:58:48 +0200 Subject: [PATCH 0592/1019] Docs, Infra: Mount local versioned doc branch for testing (#10838) The versioned docs are pulled from the remote 'iceberg_docs' repo. When making local changes to versioned docs it seems reasonable to mount some local git branch instead of the remote one so that we can test the changes. This patch add support for setting a local branch to be mounted for the versioned docs using environment variables. --- site/README.md | 10 ++++++++++ site/dev/common.sh | 9 ++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/site/README.md b/site/README.md index 39bc1d931b81..97ddddc5cd69 100644 --- a/site/README.md +++ b/site/README.md @@ -113,6 +113,16 @@ To clear all build files, run `clean`. make clean ``` +#### Testing local changes on versioned docs + +When you build the docs as described above, by default the versioned docs are mounted from the upstream remote repositiory called `iceberg_docs`. One exception is the `nightly` version that is a soft link to the local `docs/` folder. + +When you make changes to some of the historical versioned docs in a local git branch you can mount this git branch instead of the remote one by setting the following environment variables: + +`ICEBERG_VERSIONED_DOCS_BRANCH` for the `docs/` folder + +`ICEBERG_VERSIONED_JAVADOC_BRANCH` for the `javadoc/` folder + #### Offline mode One of the great advantages to the MkDocs material plugin is the [offline feature](https://squidfunk.github.io/mkdocs-material/plugins/offline). You can view the Iceberg docs without the need of a server. To enable OFFLINE builds, add theOFFLINE environment variable to either `build` or `serve` recipes. diff --git a/site/dev/common.sh b/site/dev/common.sh index 481628aa4dbd..6fc045560f39 100755 --- a/site/dev/common.sh +++ b/site/dev/common.sh @@ -197,9 +197,12 @@ pull_versioned_docs () { # Ensure the remote repository for documentation exists and is up-to-date create_or_update_docs_remote - # Add local worktrees for documentation and javadoc from the remote repository - git worktree add -f docs/docs "${REMOTE}/docs" - git worktree add -f docs/javadoc "${REMOTE}/javadoc" + # Add local worktrees for documentation and javadoc either from the remote repository + # or from a local branch. + local docs_branch="${ICEBERG_VERSIONED_DOCS_BRANCH:-${REMOTE}/docs}" + local javadoc_branch="${ICEBERG_VERSIONED_JAVADOC_BRANCH:-${REMOTE}/javadoc}" + git worktree add -f docs/docs "${docs_branch}" + git worktree add -f docs/javadoc "${javadoc_branch}" # Retrieve the latest version of documentation for processing local latest_version=$(get_latest_version) From b3d23956d58dab82d47c8b6c9bdd8e937a1cd485 Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 13 Aug 2024 22:35:41 +0800 Subject: [PATCH 0593/1019] API: Fix JavaDoc typos in Transaction API --- api/src/main/java/org/apache/iceberg/Transaction.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/Transaction.java b/api/src/main/java/org/apache/iceberg/Transaction.java index fd84a974013d..77e19e45e451 100644 --- a/api/src/main/java/org/apache/iceberg/Transaction.java +++ b/api/src/main/java/org/apache/iceberg/Transaction.java @@ -52,7 +52,7 @@ public interface Transaction { UpdateProperties updateProperties(); /** - * Create a new {@link ReplaceSortOrder} to set a table sort order and commit the change. + * Create a new {@link ReplaceSortOrder} to set a table sort order. * * @return a new {@link ReplaceSortOrder} */ @@ -131,7 +131,7 @@ default AppendFiles newFastAppend() { ReplacePartitions newReplacePartitions(); /** - * Create a new {@link DeleteFiles delete API} to replace files in this table. + * Create a new {@link DeleteFiles delete API} to delete files in this table. * * @return a new {@link DeleteFiles} */ @@ -160,7 +160,7 @@ default UpdatePartitionStatistics updatePartitionStatistics() { } /** - * Create a new {@link ExpireSnapshots expire API} to manage snapshots in this table. + * Create a new {@link ExpireSnapshots expire API} to expire snapshots in this table. * * @return a new {@link ExpireSnapshots} */ From 86d7eb20c832856f3f6e060fe1e1f0ecb45f31a1 Mon Sep 17 00:00:00 2001 From: hsiang-c <137842490+hsiang-c@users.noreply.github.com> Date: Wed, 14 Aug 2024 02:32:38 +0800 Subject: [PATCH 0594/1019] Core: Fix metadata table test to set partition to the right PartitionKey (#10925) --- .../java/org/apache/iceberg/TestMetadataTableFilters.java | 6 +++--- .../java/org/apache/iceberg/TestMetadataTableScans.java | 4 ++-- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index 9e535dd77747..bf6456e85aec 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -317,7 +317,7 @@ public void testPartitionSpecEvolutionRemovalV1() { .withPartition(data10Key) .build(); PartitionKey data11Key = new PartitionKey(newSpec, table.schema()); - data10Key.set(1, 11); + data11Key.set(1, 11); DataFile data11 = DataFiles.builder(newSpec) .withPath("/path/to/data-11.parquet") @@ -465,8 +465,8 @@ public void testPartitionSpecEvolutionAdditiveV1() { .withPartition(data10Key) .build(); PartitionKey data11Key = new PartitionKey(newSpec, table.schema()); - data11Key.set(0, 1); // data=0 - data10Key.set(1, 11); // id=11 + data11Key.set(0, 1); // data=1 + data11Key.set(1, 11); // id=11 DataFile data11 = DataFiles.builder(newSpec) .withPath("/path/to/data-11.parquet") diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java index 755eb57de8e8..146b859bef00 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableScans.java @@ -939,8 +939,8 @@ public void testPartitionSpecEvolutionAdditive() { .withPartition(data10Key) .build(); PartitionKey data11Key = new PartitionKey(newSpec, table.schema()); - data11Key.set(0, 1); // data=0 - data10Key.set(1, 11); // id=11 + data11Key.set(0, 1); // data=1 + data11Key.set(1, 11); // id=11 DataFile data11 = DataFiles.builder(newSpec) .withPath("/path/to/data-11.parquet") From b3436f4c78a08baa69a7be6d1f9d6cf42c1b3a55 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 14 Aug 2024 01:27:32 +0200 Subject: [PATCH 0595/1019] AWS, Core, Hive: Extract FileIO closing into separate FileIOTracker class (#10893) --- .../apache/iceberg/aws/glue/GlueCatalog.java | 31 ++------ .../org/apache/iceberg/io/FileIOTracker.java | 65 +++++++++++++++++ .../iceberg/rest/RESTSessionCatalog.java | 27 ++----- .../java/org/apache/iceberg/TestTables.java | 21 +++++- .../apache/iceberg/io/TestFileIOTracker.java | 72 +++++++++++++++++++ .../org/apache/iceberg/hive/HiveCatalog.java | 28 ++------ 6 files changed, 173 insertions(+), 71 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/io/FileIOTracker.java create mode 100644 core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.java diff --git a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java index c6b157bb5c79..47807a2b9f37 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java +++ b/aws/src/main/java/org/apache/iceberg/aws/glue/GlueCatalog.java @@ -18,9 +18,6 @@ */ package org.apache.iceberg.aws.glue; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.RemovalListener; import java.io.IOException; import java.util.List; import java.util.Map; @@ -51,7 +48,7 @@ import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; -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.Preconditions; import org.apache.iceberg.relocated.com.google.common.base.Strings; @@ -97,7 +94,7 @@ public class GlueCatalog extends BaseMetastoreCatalog private LockManager lockManager; private CloseableGroup closeableGroup; private Map catalogProperties; - private Cache fileIOCloser; + private FileIOTracker fileIOTracker; // Attempt to set versionId if available on the path private static final DynMethods.UnboundMethod SET_VERSION_ID = @@ -194,11 +191,12 @@ void initialize( this.lockManager = lock; this.closeableGroup = new CloseableGroup(); + this.fileIOTracker = new FileIOTracker(); closeableGroup.addCloseable(glue); closeableGroup.addCloseable(lockManager); closeableGroup.addCloseable(metricsReporter()); + closeableGroup.addCloseable(fileIOTracker); closeableGroup.setSuppressCloseFailure(true); - this.fileIOCloser = newFileIOCloser(); } @Override @@ -243,7 +241,7 @@ protected TableOperations newTableOps(TableIdentifier tableIdentifier) { tableSpecificCatalogPropertiesBuilder.buildOrThrow(), hadoopConf, tableIdentifier); - fileIOCloser.put(glueTableOperations, glueTableOperations.io()); + fileIOTracker.track(glueTableOperations); return glueTableOperations; } @@ -256,7 +254,7 @@ protected TableOperations newTableOps(TableIdentifier tableIdentifier) { catalogProperties, hadoopConf, tableIdentifier); - fileIOCloser.put(glueTableOperations, glueTableOperations.io()); + fileIOTracker.track(glueTableOperations); return glueTableOperations; } @@ -634,10 +632,6 @@ public String name() { @Override public void close() throws IOException { closeableGroup.close(); - if (fileIOCloser != null) { - fileIOCloser.invalidateAll(); - fileIOCloser.cleanUp(); - } } @Override @@ -649,17 +643,4 @@ public void setConf(Configuration conf) { protected Map properties() { return catalogProperties == null ? ImmutableMap.of() : catalogProperties; } - - private Cache newFileIOCloser() { - return Caffeine.newBuilder() - .weakKeys() - .removalListener( - (RemovalListener) - (ops, fileIO, cause) -> { - if (null != fileIO) { - fileIO.close(); - } - }) - .build(); - } } diff --git a/core/src/main/java/org/apache/iceberg/io/FileIOTracker.java b/core/src/main/java/org/apache/iceberg/io/FileIOTracker.java new file mode 100644 index 000000000000..9d8630e79b14 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/io/FileIOTracker.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 com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import com.github.benmanes.caffeine.cache.RemovalListener; +import java.io.Closeable; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +/** + * Keeps track of the {@link FileIO} instance of the given {@link TableOperations} instance and + * closes the {@link FileIO} when {@link FileIOTracker#close()} gets called + */ +public class FileIOTracker implements Closeable { + private final Cache tracker; + + public FileIOTracker() { + this.tracker = + Caffeine.newBuilder() + .weakKeys() + .removalListener( + (RemovalListener) + (ops, fileIO, cause) -> { + if (null != fileIO) { + fileIO.close(); + } + }) + .build(); + } + + public void track(TableOperations ops) { + Preconditions.checkArgument(null != ops, "Invalid table ops: null"); + tracker.put(ops, ops.io()); + } + + @VisibleForTesting + Cache tracker() { + return tracker; + } + + @Override + public void close() { + tracker.invalidateAll(); + tracker.cleanUp(); + } +} 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 1c607e3b0220..53ce45bb0a3f 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -47,7 +47,6 @@ import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.TableMetadata; -import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; import org.apache.iceberg.Transactions; import org.apache.iceberg.catalog.BaseViewSessionCatalog; @@ -63,6 +62,7 @@ import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOTracker; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.MetricsReporters; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -136,7 +136,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private final BiFunction, FileIO> ioBuilder; private Cache sessions = null; private Cache tableSessions = null; - private Cache fileIOCloser; + private FileIOTracker fileIOTracker = null; private AuthSession catalogAuth = null; private boolean keepTokenRefreshed = true; private RESTClient client = null; @@ -268,10 +268,11 @@ public void initialize(String name, Map unresolved) { this.io = newFileIO(SessionContext.createEmpty(), mergedProps); - this.fileIOCloser = newFileIOCloser(); + this.fileIOTracker = new FileIOTracker(); this.closeables = new CloseableGroup(); this.closeables.addCloseable(this.io); this.closeables.addCloseable(this.client); + this.closeables.addCloseable(fileIOTracker); this.closeables.setSuppressCloseFailure(true); this.snapshotMode = @@ -465,7 +466,7 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { private void trackFileIO(RESTTableOperations ops) { if (io != ops.io()) { - fileIOCloser.put(ops, ops.io()); + fileIOTracker.track(ops); } } @@ -641,11 +642,6 @@ public void close() throws IOException { if (closeables != null) { closeables.close(); } - - if (fileIOCloser != null) { - fileIOCloser.invalidateAll(); - fileIOCloser.cleanUp(); - } } private void shutdownRefreshExecutor() { @@ -1088,19 +1084,6 @@ private static Cache newSessionCache(Map pr .build(); } - private Cache newFileIOCloser() { - return Caffeine.newBuilder() - .weakKeys() - .removalListener( - (RemovalListener) - (ops, fileIO, cause) -> { - if (null != fileIO) { - fileIO.close(); - } - }) - .build(); - } - public void commitTransaction(SessionContext context, List commits) { List tableChanges = Lists.newArrayListWithCapacity(commits.size()); diff --git a/core/src/test/java/org/apache/iceberg/TestTables.java b/core/src/test/java/org/apache/iceberg/TestTables.java index de05e85c3c77..eeff5db8e5a6 100644 --- a/core/src/test/java/org/apache/iceberg/TestTables.java +++ b/core/src/test/java/org/apache/iceberg/TestTables.java @@ -216,6 +216,7 @@ public static class TestTableOperations implements TableOperations { private final String tableName; private final File metadata; + private final FileIO fileIO; private TableMetadata current = null; private long lastSnapshotId = 0; private int failCommits = 0; @@ -223,6 +224,22 @@ public static class TestTableOperations implements TableOperations { public TestTableOperations(String tableName, File location) { this.tableName = tableName; this.metadata = new File(location, "metadata"); + this.fileIO = new LocalFileIO(); + metadata.mkdirs(); + refresh(); + if (current != null) { + for (Snapshot snap : current.snapshots()) { + this.lastSnapshotId = Math.max(lastSnapshotId, snap.snapshotId()); + } + } else { + this.lastSnapshotId = 0; + } + } + + public TestTableOperations(String tableName, File location, FileIO fileIO) { + this.tableName = tableName; + this.metadata = new File(location, "metadata"); + this.fileIO = fileIO; metadata.mkdirs(); refresh(); if (current != null) { @@ -277,7 +294,7 @@ public void commit(TableMetadata base, TableMetadata updatedMetadata) { @Override public FileIO io() { - return new LocalFileIO(); + return fileIO; } @Override @@ -300,7 +317,7 @@ public long newSnapshotId() { } } - static class LocalFileIO implements FileIO { + public static class LocalFileIO implements FileIO { @Override public InputFile newInputFile(String path) { diff --git a/core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.java b/core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.java new file mode 100644 index 000000000000..e6225d886cee --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/io/TestFileIOTracker.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.io; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.Mockito.times; + +import java.io.File; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.TestTables; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.mockito.Mockito; + +public class TestFileIOTracker { + + @TempDir private File tableDir; + + @SuppressWarnings("resource") + @Test + public void nullTableOps() { + assertThatThrownBy(() -> new FileIOTracker().track(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid table ops: null"); + } + + @SuppressWarnings("unchecked") + @Test + public void fileIOGetsClosed() throws NoSuchFieldException, IllegalAccessException { + FileIOTracker fileIOTracker = new FileIOTracker(); + + FileIO firstFileIO = Mockito.spy(new TestTables.LocalFileIO()); + TestTables.TestTableOperations firstOps = + new TestTables.TestTableOperations("x", tableDir, firstFileIO); + fileIOTracker.track(firstOps); + assertThat(fileIOTracker.tracker().estimatedSize()).isEqualTo(1); + + FileIO secondFileIO = Mockito.spy(new TestTables.LocalFileIO()); + TestTables.TestTableOperations secondOps = + new TestTables.TestTableOperations("y", tableDir, secondFileIO); + fileIOTracker.track(secondOps); + assertThat(fileIOTracker.tracker().estimatedSize()).isEqualTo(2); + + fileIOTracker.close(); + Awaitility.await("FileIO gets closed") + .atMost(5, TimeUnit.SECONDS) + .untilAsserted( + () -> { + assertThat(fileIOTracker.tracker().estimatedSize()).isEqualTo(0); + Mockito.verify(firstFileIO, times(1)).close(); + Mockito.verify(secondFileIO, times(1)).close(); + }); + } +} 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 8944cf93947b..5c58222f0c01 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,9 +18,6 @@ */ package org.apache.iceberg.hive; -import com.github.benmanes.caffeine.cache.Cache; -import com.github.benmanes.caffeine.cache.Caffeine; -import com.github.benmanes.caffeine.cache.RemovalListener; import java.io.IOException; import java.util.List; import java.util.Map; @@ -53,6 +50,7 @@ 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; @@ -83,7 +81,7 @@ public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespa private ClientPool clients; private boolean listAllTables = false; private Map catalogProperties; - private Cache fileIOCloser; + private FileIOTracker fileIOTracker; public HiveCatalog() {} @@ -116,20 +114,7 @@ public void initialize(String inputName, Map properties) { : CatalogUtil.loadFileIO(fileIOImpl, properties, conf); this.clients = new CachedClientPool(conf, properties); - this.fileIOCloser = newFileIOCloser(); - } - - private Cache newFileIOCloser() { - return Caffeine.newBuilder() - .weakKeys() - .removalListener( - (RemovalListener) - (ops, fileIOInstance, cause) -> { - if (null != fileIOInstance) { - fileIOInstance.close(); - } - }) - .build(); + this.fileIOTracker = new FileIOTracker(); } @Override @@ -533,7 +518,7 @@ public TableOperations newTableOps(TableIdentifier tableIdentifier) { String tableName = tableIdentifier.name(); HiveTableOperations ops = new HiveTableOperations(conf, clients, fileIO, name, dbName, tableName); - fileIOCloser.put(ops, ops.io()); + fileIOTracker.track(ops); return ops; } @@ -661,9 +646,8 @@ protected Map properties() { @Override public void close() throws IOException { super.close(); - if (fileIOCloser != null) { - fileIOCloser.invalidateAll(); - fileIOCloser.cleanUp(); + if (fileIOTracker != null) { + fileIOTracker.close(); } } From a3a43053f5f7fc23d6fcb5c5df34a9769a0f1520 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Thu, 15 Aug 2024 18:21:14 +0530 Subject: [PATCH 0596/1019] Build: Suppress various build warnings (#10938) --- .../java/org/apache/iceberg/aliyun/oss/OSSInputStream.java | 2 +- .../java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java | 2 +- api/src/main/java/org/apache/iceberg/Metrics.java | 2 ++ aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIO.java | 2 +- .../main/java/org/apache/iceberg/aws/s3/S3InputStream.java | 2 +- .../main/java/org/apache/iceberg/aws/s3/S3OutputStream.java | 2 +- .../org/apache/iceberg/azure/adlsv2/ADLSInputStream.java | 2 +- .../org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java | 2 +- baseline.gradle | 6 ++---- .../main/java/org/apache/iceberg/hadoop/HadoopStreams.java | 4 ++-- .../main/java/org/apache/iceberg/io/ResolvingFileIO.java | 2 +- .../java/org/apache/iceberg/util/SerializationUtil.java | 2 +- .../java/org/apache/iceberg/gcp/gcs/GCSInputStream.java | 2 +- .../java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java | 2 +- .../apache/iceberg/mr/hive/HiveIcebergOutputCommitter.java | 1 + 15 files changed, 18 insertions(+), 17 deletions(-) diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java index b161bfcaf767..64e625997760 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSInputStream.java @@ -157,7 +157,7 @@ private void closeStream() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java index 9644dab82285..8c226d00123c 100644 --- a/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java +++ b/aliyun/src/main/java/org/apache/iceberg/aliyun/oss/OSSOutputStream.java @@ -165,7 +165,7 @@ private void cleanUpStagingFiles() { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/api/src/main/java/org/apache/iceberg/Metrics.java b/api/src/main/java/org/apache/iceberg/Metrics.java index 2f2cf89cdadd..d1a7da6d9d95 100644 --- a/api/src/main/java/org/apache/iceberg/Metrics.java +++ b/api/src/main/java/org/apache/iceberg/Metrics.java @@ -179,6 +179,7 @@ private static void writeByteBufferMap( * @throws IOException On serialization error * @throws ClassNotFoundException If the class is not found */ + @SuppressWarnings("DangerousJavaDeserialization") private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundException { rowCount = (Long) in.readObject(); columnSizes = (Map) in.readObject(); @@ -190,6 +191,7 @@ private void readObject(ObjectInputStream in) throws IOException, ClassNotFoundE upperBounds = readByteBufferMap(in); } + @SuppressWarnings("DangerousJavaDeserialization") private static Map readByteBufferMap(ObjectInputStream in) throws IOException, ClassNotFoundException { int size = in.readInt(); 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 f7d2da5eb907..040d3012ca7a 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 @@ -412,7 +412,7 @@ public void close() { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); 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 f1d6c30a27a5..f442a0f04a1c 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 @@ -227,7 +227,7 @@ public void setSkipSize(int skipSize) { this.skipSize = skipSize; } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java index 2a9275045d5a..ef0b4debf66b 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java @@ -475,7 +475,7 @@ private void createStagingDirectoryIfNotExists() throws IOException, SecurityExc } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java index 261ede62db15..d9e1f93bea67 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSInputStream.java @@ -190,7 +190,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java index f38a408896be..b507d5b54115 100644 --- a/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java +++ b/azure/src/main/java/org/apache/iceberg/azure/adlsv2/ADLSOutputStream.java @@ -110,7 +110,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/baseline.gradle b/baseline.gradle index 42be6d8a24c6..486625636b59 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -91,16 +91,14 @@ subprojects { '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', - // TODO (https://github.com/apache/iceberg/issues/10853) this is a recently added check. Figure out whether we adjust the code or suppress for good - '-Xep:DangerousJavaDeserialization:WARN', + '-Xep:DangerousJavaDeserialization:ERROR', '-Xep:DangerousThreadPoolExecutorUsage:OFF', // subclasses are not equal '-Xep:EqualsGetClass:OFF', // specific to Palantir '-Xep:FinalClass:OFF', '-Xep:FormatStringAnnotation:ERROR', - // TODO (https://github.com/apache/iceberg/issues/10855) this is a recently added check. Figure out whether we adjust the code or suppress for good - '-Xep:ImmutablesReferenceEquality:WARN', + '-Xep:ImmutablesReferenceEquality:ERROR', '-Xep:IntLongMath:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java index c817b1d90afb..6a48b14c227d 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopStreams.java @@ -122,7 +122,7 @@ public int read(byte[] b, int off, int len) throws IOException { return stream.read(b, off, len); } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); @@ -195,7 +195,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); 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 e5c7c5163129..b5232960dcfb 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -240,7 +240,7 @@ private static String scheme(String location) { return null; } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java b/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java index ddba0b0bbea4..216f55eae30c 100644 --- a/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/SerializationUtil.java @@ -69,7 +69,7 @@ public static byte[] serializeToBytes( } } - @SuppressWarnings("unchecked") + @SuppressWarnings({"DangerousJavaDeserialization", "unchecked"}) public static T deserializeFromBytes(byte[] bytes) { if (bytes == null) { return null; diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java index fea51d50cfd0..4483b0233669 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSInputStream.java @@ -186,7 +186,7 @@ public void close() throws IOException { } } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); diff --git a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java index bcd6099ac987..e52a1638827a 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/gcs/GCSOutputStream.java @@ -130,7 +130,7 @@ public void close() throws IOException { stream.close(); } - @SuppressWarnings("checkstyle:NoFinalizer") + @SuppressWarnings({"checkstyle:NoFinalizer", "Finalize"}) @Override protected void finalize() throws Throwable { super.finalize(); 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 da2735774c95..8b9cd3d3d4db 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 @@ -508,6 +508,7 @@ private static void createFileForCommit(DataFile[] closedFiles, String location, LOG.debug("Iceberg committed file is created {}", fileForCommit); } + @SuppressWarnings("DangerousJavaDeserialization") private static DataFile[] readFileForCommit(String fileForCommitLocation, FileIO io) { try (ObjectInputStream ois = new ObjectInputStream(io.newInputFile(fileForCommitLocation).newStream())) { From 2d36e9641b5af851f1bdb093841c08c8ed1ccac9 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 15 Aug 2024 09:40:33 -0700 Subject: [PATCH 0597/1019] Core: add JSON serialization for BaseFilesTable.ManifestReadTask, AllManifestsTable.ManifestListReadTask, and BaseEntriesTable.ManifestReadTask (#10735) --- .palantir/revapi.yml | 15 +- .../org/apache/iceberg/AllManifestsTable.java | 41 ++- .../iceberg/AllManifestsTableTaskParser.java | 107 +++++++ .../org/apache/iceberg/BaseEntriesTable.java | 52 ++-- .../org/apache/iceberg/BaseFilesTable.java | 51 ++-- .../apache/iceberg/FilesTableTaskParser.java | 99 +++++++ .../apache/iceberg/GenericManifestFile.java | 36 +++ .../ManifestEntriesTableTaskParser.java | 98 +++++++ .../apache/iceberg/ManifestFileParser.java | 271 ++++++++++++++++++ .../org/apache/iceberg/ScanTaskParser.java | 28 +- .../org/apache/iceberg/TableMetadata.java | 13 +- .../org/apache/iceberg/io/FileIOParser.java | 4 +- .../apache/iceberg/util/PartitionUtil.java | 10 + .../TestAllManifestsTableTaskParser.java | 152 ++++++++++ .../iceberg/TestFilesTableTaskParser.java | 132 +++++++++ .../iceberg/TestManifestFileParser.java | 111 +++++++ 16 files changed, 1167 insertions(+), 53 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java create mode 100644 core/src/main/java/org/apache/iceberg/FilesTableTaskParser.java create mode 100644 core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java create mode 100644 core/src/main/java/org/apache/iceberg/ManifestFileParser.java create mode 100644 core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java create mode 100644 core/src/test/java/org/apache/iceberg/TestFilesTableTaskParser.java create mode 100644 core/src/test/java/org/apache/iceberg/TestManifestFileParser.java diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index e58ce70ded7a..9b8017f0beec 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1025,14 +1025,11 @@ acceptedBreaks: new: "class org.apache.iceberg.types.Types.NestedField" justification: "new Constructor added" org.apache.iceberg:iceberg-core: - - code: "java.method.visibilityReduced" - old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" - new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" - justification: "Deprecations for 1.6.0 release" - code: "java.element.noLongerDeprecated" old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" - justification: "Constructor became private as part of deprecations cleanup for 1.6.0 release" + justification: "Constructor became private as part of deprecations cleanup for\ + \ 1.6.0 release" - code: "java.element.noLongerDeprecated" old: "method void org.apache.iceberg.rest.auth.OAuth2Util.AuthSession::(java.util.Map, java.lang.String, java.lang.String, java.lang.String,\ @@ -1056,6 +1053,10 @@ acceptedBreaks: - code: "java.method.removed" old: "method org.apache.iceberg.DataFiles.Builder org.apache.iceberg.DataFiles.Builder::withEqualityFieldIds(java.util.List)" justification: "Deprecations for 1.6.0 release" + - code: "java.method.visibilityReduced" + old: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" + new: "method void org.apache.iceberg.encryption.PlaintextEncryptionManager::()" + justification: "Deprecations for 1.6.0 release" "1.6.0": org.apache.iceberg:iceberg-common: - code: "java.method.removed" @@ -1083,6 +1084,10 @@ acceptedBreaks: \ java.lang.Object[]) throws java.lang.Exception" justification: "Reduced visibility and scoped to package" org.apache.iceberg:iceberg-core: + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.GenericManifestFile" + new: "class org.apache.iceberg.GenericManifestFile" + justification: "Serialization across versions is not supported" - code: "java.class.removed" old: "enum org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus" justification: "Removing deprecated code" diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java index a9c6f50e0b36..641a7a3c9aec 100644 --- a/core/src/main/java/org/apache/iceberg/AllManifestsTable.java +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTable.java @@ -34,6 +34,7 @@ import org.apache.iceberg.expressions.Literal; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; 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.Maps; @@ -52,7 +53,8 @@ public class AllManifestsTable extends BaseMetadataTable { public static final Types.NestedField REF_SNAPSHOT_ID = Types.NestedField.required(18, "reference_snapshot_id", Types.LongType.get()); - private static final Schema MANIFEST_FILE_SCHEMA = + @VisibleForTesting + static final Schema MANIFEST_FILE_SCHEMA = new Schema( Types.NestedField.required(14, "content", Types.IntegerType.get()), Types.NestedField.required(1, "path", Types.StringType.get()), @@ -119,6 +121,7 @@ protected TableScan newRefinedScan(Table table, Schema schema, TableScanContext protected CloseableIterable doPlanFiles() { FileIO io = table().io(); Map specs = Maps.newHashMap(table().specs()); + Schema dataTableSchema = table().schema(); Expression filter = shouldIgnoreResiduals() ? Expressions.alwaysTrue() : filter(); SnapshotEvaluator snapshotEvaluator = @@ -132,7 +135,13 @@ protected CloseableIterable doPlanFiles() { snap -> { if (snap.manifestListLocation() != null) { return new ManifestListReadTask( - io, schema(), specs, snap.manifestListLocation(), filter, snap.snapshotId()); + dataTableSchema, + io, + schema(), + specs, + snap.manifestListLocation(), + filter, + snap.snapshotId()); } else { return StaticDataTask.of( io.newInputFile( @@ -149,6 +158,7 @@ protected CloseableIterable doPlanFiles() { } static class ManifestListReadTask implements DataTask { + private final Schema dataTableSchema; private final FileIO io; private final Schema schema; private final Map specs; @@ -158,12 +168,14 @@ static class ManifestListReadTask implements DataTask { private DataFile lazyDataFile = null; ManifestListReadTask( + Schema dataTableSchema, FileIO io, Schema schema, Map specs, String manifestListLocation, Expression residual, long referenceSnapshotId) { + this.dataTableSchema = dataTableSchema; this.io = io; this.schema = schema; this.specs = specs; @@ -244,6 +256,31 @@ public Expression residual() { public Iterable split(long splitSize) { return ImmutableList.of(this); // don't split } + + @Override + public Schema schema() { + return schema; + } + + Schema dataTableSchema() { + return dataTableSchema; + } + + FileIO io() { + return io; + } + + Map specsById() { + return specs; + } + + String manifestListLocation() { + return manifestListLocation; + } + + long referenceSnapshotId() { + return referenceSnapshotId; + } } static StaticDataTask.Row manifestFileToRow( diff --git a/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java b/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.java new file mode 100644 index 000000000000..24c28d95466c --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/AllManifestsTableTaskParser.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; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; + +class AllManifestsTableTaskParser { + private static final String DATA_TABLE_SCHEMA = "data-table-schema"; + private static final String FILE_IO = "file-io"; + private static final String SCHEMA = "schema"; + private static final String SPECS = "partition-specs"; + private static final String MANIFEST_LIST_LOCATION = "manifest-list-Location"; + private static final String RESIDUAL = "residual-filter"; + private static final String REFERENCE_SNAPSHOT_ID = "reference-snapshot-id"; + + private AllManifestsTableTaskParser() {} + + static void toJson(AllManifestsTable.ManifestListReadTask task, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(task != null, "Invalid manifest task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(DATA_TABLE_SCHEMA); + SchemaParser.toJson(task.dataTableSchema(), generator); + + generator.writeFieldName(FILE_IO); + FileIOParser.toJson(task.io(), generator); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(task.schema(), generator); + + generator.writeArrayFieldStart(SPECS); + for (PartitionSpec spec : task.specsById().values()) { + PartitionSpecParser.toJson(spec, generator); + } + + generator.writeEndArray(); + + generator.writeStringField(MANIFEST_LIST_LOCATION, task.manifestListLocation()); + + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(task.residual(), generator); + + generator.writeNumberField(REFERENCE_SNAPSHOT_ID, task.referenceSnapshotId()); + } + + static AllManifestsTable.ManifestListReadTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for manifest task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for manifest task: non-object (%s)", jsonNode); + + Schema dataTableSchema = SchemaParser.fromJson(JsonUtil.get(DATA_TABLE_SCHEMA, jsonNode)); + FileIO fileIO = FileIOParser.fromJson(JsonUtil.get(FILE_IO, jsonNode), null); + Schema schema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + + JsonNode specsArray = JsonUtil.get(SPECS, jsonNode); + Preconditions.checkArgument( + specsArray.isArray(), "Invalid JSON node for partition specs: non-array (%s)", specsArray); + + ImmutableList.Builder specsBuilder = ImmutableList.builder(); + for (JsonNode specNode : specsArray) { + PartitionSpec spec = PartitionSpecParser.fromJson(dataTableSchema, specNode); + specsBuilder.add(spec); + } + + Map specsById = PartitionUtil.indexSpecs(specsBuilder.build()); + String manifestListLocation = JsonUtil.getString(MANIFEST_LIST_LOCATION, jsonNode); + Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); + long referenceSnapshotId = JsonUtil.getLong(REFERENCE_SNAPSHOT_ID, jsonNode); + + return new AllManifestsTable.ManifestListReadTask( + dataTableSchema, + fileIO, + schema, + specsById, + manifestListLocation, + residualFilter, + referenceSnapshotId); + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java index 526bb42ea687..ae77bff7036d 100644 --- a/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseEntriesTable.java @@ -33,7 +33,6 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -92,15 +91,9 @@ static CloseableIterable planFiles( evalCache.get(manifest.partitionSpecId()).eval(manifest) && manifestContentEvaluator.eval(manifest)); - String schemaString = SchemaParser.toJson(projectedSchema); - String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter); - return CloseableIterable.transform( filteredManifests, - manifest -> - new ManifestReadTask( - table, manifest, projectedSchema, schemaString, specString, residuals)); + manifest -> new ManifestReadTask(table, manifest, projectedSchema, filter)); } /** @@ -283,19 +276,29 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask { private final ManifestFile manifest; private final Map specsById; + private ManifestReadTask( + Table table, ManifestFile manifest, Schema projection, Expression filter) { + this(table.schema(), table.io(), table.specs(), manifest, projection, filter); + } + ManifestReadTask( - Table table, + Schema dataTableSchema, + FileIO io, + Map specsById, ManifestFile manifest, Schema projection, - String schemaString, - String specString, - ResidualEvaluator residuals) { - super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals); + Expression filter) { + super( + DataFiles.fromManifest(manifest), + null, + SchemaParser.toJson(projection), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + ResidualEvaluator.unpartitioned(filter)); this.projection = projection; - this.io = table.io(); + this.io = io; this.manifest = manifest; - this.specsById = Maps.newHashMap(table.specs()); - this.dataTableSchema = table.schema(); + this.specsById = Maps.newHashMap(specsById); + this.dataTableSchema = dataTableSchema; Type fileProjectionType = projection.findType("data_file"); this.fileProjection = @@ -311,7 +314,6 @@ public long estimatedRowsCount() { + (long) manifest.existingFilesCount(); } - @VisibleForTesting ManifestFile manifest() { return manifest; } @@ -403,5 +405,21 @@ private MetricsUtil.ReadableMetricsStruct readableMetrics( public Iterable split(long splitSize) { return ImmutableList.of(this); // don't split } + + FileIO io() { + return io; + } + + Map specsById() { + return specsById; + } + + Schema dataTableSchema() { + return dataTableSchema; + } + + Schema projection() { + return projection; + } } } diff --git a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java index 149edf950032..b71744f739c7 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFilesTable.java +++ b/core/src/main/java/org/apache/iceberg/BaseFilesTable.java @@ -29,7 +29,6 @@ import org.apache.iceberg.expressions.ResidualEvaluator; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.FileIO; -import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Sets; @@ -80,16 +79,11 @@ private static CloseableIterable planFiles( CloseableIterable.filter( manifests, manifest -> evalCache.get(manifest.partitionSpecId()).eval(manifest)); - String schemaString = SchemaParser.toJson(projectedSchema); - String specString = PartitionSpecParser.toJson(PartitionSpec.unpartitioned()); Expression filter = ignoreResiduals ? Expressions.alwaysTrue() : rowFilter; - ResidualEvaluator residuals = ResidualEvaluator.unpartitioned(filter); return CloseableIterable.transform( filteredManifests, - manifest -> - new ManifestReadTask( - table, manifest, projectedSchema, schemaString, specString, residuals)); + manifest -> new ManifestReadTask(table, manifest, projectedSchema, filter)); } abstract static class BaseFilesTableScan extends BaseMetadataTableScan { @@ -140,18 +134,28 @@ static class ManifestReadTask extends BaseFileScanTask implements DataTask { private final Schema dataTableSchema; private final Schema projection; + private ManifestReadTask( + Table table, ManifestFile manifest, Schema projection, Expression filter) { + this(table.schema(), table.io(), table.specs(), manifest, projection, filter); + } + ManifestReadTask( - Table table, + Schema dataTableSchema, + FileIO io, + Map specsById, ManifestFile manifest, Schema projection, - String schemaString, - String specString, - ResidualEvaluator residuals) { - super(DataFiles.fromManifest(manifest), null, schemaString, specString, residuals); - this.io = table.io(); - this.specsById = Maps.newHashMap(table.specs()); + Expression filter) { + super( + DataFiles.fromManifest(manifest), + null, + SchemaParser.toJson(projection), + PartitionSpecParser.toJson(PartitionSpec.unpartitioned()), + ResidualEvaluator.unpartitioned(filter)); + this.io = io; + this.specsById = Maps.newHashMap(specsById); this.manifest = manifest; - this.dataTableSchema = table.schema(); + this.dataTableSchema = dataTableSchema; this.projection = projection; } @@ -240,9 +244,24 @@ public Iterable split(long splitSize) { return ImmutableList.of(this); // don't split } - @VisibleForTesting + FileIO io() { + return io; + } + + Map specsById() { + return specsById; + } + ManifestFile manifest() { return manifest; } + + Schema dataTableSchema() { + return dataTableSchema; + } + + Schema projection() { + return projection; + } } } diff --git a/core/src/main/java/org/apache/iceberg/FilesTableTaskParser.java b/core/src/main/java/org/apache/iceberg/FilesTableTaskParser.java new file mode 100644 index 000000000000..365deacebf10 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/FilesTableTaskParser.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; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; + +class FilesTableTaskParser { + private static final String FILE_IO = "file-io"; + private static final String SPECS = "partition-specs"; + private static final String SCHEMA = "schema"; + private static final String PROJECTION = "projection"; + private static final String RESIDUAL = "residual-filter"; + private static final String MANIFEST = "manifest-file"; + + private FilesTableTaskParser() {} + + static void toJson(BaseFilesTable.ManifestReadTask task, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(task != null, "Invalid files task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(task.dataTableSchema(), generator); + + generator.writeFieldName(PROJECTION); + SchemaParser.toJson(task.projection(), generator); + + generator.writeFieldName(FILE_IO); + FileIOParser.toJson(task.io(), generator); + + generator.writeArrayFieldStart(SPECS); + for (PartitionSpec spec : task.specsById().values()) { + PartitionSpecParser.toJson(spec, generator); + } + + generator.writeEndArray(); + + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(task.residual(), generator); + + generator.writeFieldName(MANIFEST); + ManifestFileParser.toJson(task.manifest(), generator); + } + + static BaseFilesTable.ManifestReadTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for files task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for files task: non-object (%s)", jsonNode); + + Schema dataTableSchema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + Schema projection = SchemaParser.fromJson(JsonUtil.get(PROJECTION, jsonNode)); + + FileIO fileIO = FileIOParser.fromJson(JsonUtil.get(FILE_IO, jsonNode), null); + + JsonNode specsArray = JsonUtil.get(SPECS, jsonNode); + Preconditions.checkArgument( + specsArray.isArray(), "Invalid JSON node for partition specs: non-array (%s)", specsArray); + + ImmutableList.Builder specsBuilder = ImmutableList.builder(); + for (JsonNode specNode : specsArray) { + PartitionSpec spec = PartitionSpecParser.fromJson(dataTableSchema, specNode); + specsBuilder.add(spec); + } + + Map specsById = PartitionUtil.indexSpecs(specsBuilder.build()); + Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); + ManifestFile manifestFile = ManifestFileParser.fromJson(JsonUtil.get(MANIFEST, jsonNode)); + + return new BaseFilesTable.ManifestReadTask( + dataTableSchema, fileIO, specsById, manifestFile, projection, residualFilter); + } +} diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index ed94da5e1791..d081e0bdd568 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -105,6 +105,42 @@ public GenericManifestFile(Schema avroSchema) { this.keyMetadata = null; } + /** Adjust the arg order to avoid conflict with the public constructor below */ + GenericManifestFile( + String path, + long length, + int specId, + ManifestContent content, + long sequenceNumber, + long minSequenceNumber, + Long snapshotId, + List partitions, + ByteBuffer keyMetadata, + Integer addedFilesCount, + Long addedRowsCount, + Integer existingFilesCount, + Long existingRowsCount, + Integer deletedFilesCount, + Long deletedRowsCount) { + this.avroSchema = AVRO_SCHEMA; + this.manifestPath = path; + this.length = length; + this.specId = specId; + this.content = content; + this.sequenceNumber = sequenceNumber; + this.minSequenceNumber = minSequenceNumber; + this.snapshotId = snapshotId; + this.addedFilesCount = addedFilesCount; + this.addedRowsCount = addedRowsCount; + this.existingFilesCount = existingFilesCount; + this.existingRowsCount = existingRowsCount; + this.deletedFilesCount = deletedFilesCount; + this.deletedRowsCount = deletedRowsCount; + this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); + this.fromProjectionPos = null; + this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); + } + public GenericManifestFile( String path, long length, diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java b/core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java new file mode 100644 index 000000000000..37f84d09ae91 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ManifestEntriesTableTaskParser.java @@ -0,0 +1,98 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.util.Map; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.expressions.ExpressionParser; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.FileIOParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; + +class ManifestEntriesTableTaskParser { + private static final String FILE_IO = "file-io"; + private static final String SPECS = "partition-specs"; + private static final String SCHEMA = "schema"; + private static final String PROJECTION = "projection"; + private static final String RESIDUAL = "residual-filter"; + private static final String MANIFEST = "manifest-file"; + + private ManifestEntriesTableTaskParser() {} + + static void toJson(BaseEntriesTable.ManifestReadTask task, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(task != null, "Invalid manifest task: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeFieldName(SCHEMA); + SchemaParser.toJson(task.dataTableSchema(), generator); + + generator.writeFieldName(FILE_IO); + FileIOParser.toJson(task.io(), generator); + + generator.writeArrayFieldStart(SPECS); + for (PartitionSpec spec : task.specsById().values()) { + PartitionSpecParser.toJson(spec, generator); + } + + generator.writeEndArray(); + + generator.writeFieldName(MANIFEST); + ManifestFileParser.toJson(task.manifest(), generator); + + generator.writeFieldName(PROJECTION); + SchemaParser.toJson(task.projection(), generator); + + generator.writeFieldName(RESIDUAL); + ExpressionParser.toJson(task.residual(), generator); + } + + static BaseEntriesTable.ManifestReadTask fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for manifest task: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for manifest task: non-object (%s)", jsonNode); + + Schema dataTableSchema = SchemaParser.fromJson(JsonUtil.get(SCHEMA, jsonNode)); + FileIO fileIO = FileIOParser.fromJson(JsonUtil.get(FILE_IO, jsonNode), null); + + JsonNode specsArray = JsonUtil.get(SPECS, jsonNode); + Preconditions.checkArgument( + specsArray.isArray(), "Invalid JSON node for partition specs: non-array (%s)", specsArray); + ImmutableList.Builder specsBuilder = ImmutableList.builder(); + for (JsonNode specNode : specsArray) { + PartitionSpec spec = PartitionSpecParser.fromJson(dataTableSchema, specNode); + specsBuilder.add(spec); + } + + Map specsById = PartitionUtil.indexSpecs(specsBuilder.build()); + + ManifestFile manifestFile = ManifestFileParser.fromJson(JsonUtil.get(MANIFEST, jsonNode)); + Schema projection = SchemaParser.fromJson(JsonUtil.get(PROJECTION, jsonNode)); + Expression residualFilter = ExpressionParser.fromJson(JsonUtil.get(RESIDUAL, jsonNode)); + + return new BaseEntriesTable.ManifestReadTask( + dataTableSchema, fileIO, specsById, manifestFile, projection, residualFilter); + } +} diff --git a/core/src/main/java/org/apache/iceberg/ManifestFileParser.java b/core/src/main/java/org/apache/iceberg/ManifestFileParser.java new file mode 100644 index 000000000000..80f1a0fcf7fc --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/ManifestFileParser.java @@ -0,0 +1,271 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; + +class ManifestFileParser { + private static final String PATH = "path"; + private static final String LENGTH = "length"; + private static final String SPEC_ID = "partition-spec-id"; + private static final String CONTENT = "content"; + private static final String SEQUENCE_NUMBER = "sequence-number"; + private static final String MIN_SEQUENCE_NUMBER = "min-sequence-number"; + private static final String ADDED_SNAPSHOT_ID = "added-snapshot-id"; + private static final String ADDED_FILES_COUNT = "added-files-count"; + private static final String EXISTING_FILES_COUNT = "existing-files-count"; + private static final String DELETED_FILES_COUNT = "deleted-files-count"; + private static final String ADDED_ROWS_COUNT = "added-rows-count"; + private static final String EXISTING_ROWS_COUNT = "existing-rows-count"; + private static final String DELETED_ROWS_COUNT = "deleted-rows-count"; + private static final String PARTITION_FIELD_SUMMARY = "partition-field-summary"; + private static final String KEY_METADATA = "key-metadata"; + + private ManifestFileParser() {} + + static void toJson(ManifestFile manifestFile, JsonGenerator generator) throws IOException { + Preconditions.checkArgument(manifestFile != null, "Invalid manifest file: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeStartObject(); + + generator.writeStringField(PATH, manifestFile.path()); + generator.writeNumberField(LENGTH, manifestFile.length()); + generator.writeNumberField(SPEC_ID, manifestFile.partitionSpecId()); + + if (manifestFile.content() != null) { + generator.writeNumberField(CONTENT, manifestFile.content().id()); + } + + generator.writeNumberField(SEQUENCE_NUMBER, manifestFile.sequenceNumber()); + generator.writeNumberField(MIN_SEQUENCE_NUMBER, manifestFile.minSequenceNumber()); + + if (manifestFile.snapshotId() != null) { + generator.writeNumberField(ADDED_SNAPSHOT_ID, manifestFile.snapshotId()); + } + + if (manifestFile.addedFilesCount() != null) { + generator.writeNumberField(ADDED_FILES_COUNT, manifestFile.addedFilesCount()); + } + + if (manifestFile.existingFilesCount() != null) { + generator.writeNumberField(EXISTING_FILES_COUNT, manifestFile.existingFilesCount()); + } + + if (manifestFile.deletedFilesCount() != null) { + generator.writeNumberField(DELETED_FILES_COUNT, manifestFile.deletedFilesCount()); + } + + if (manifestFile.addedRowsCount() != null) { + generator.writeNumberField(ADDED_ROWS_COUNT, manifestFile.addedRowsCount()); + } + + if (manifestFile.existingRowsCount() != null) { + generator.writeNumberField(EXISTING_ROWS_COUNT, manifestFile.existingRowsCount()); + } + + if (manifestFile.deletedRowsCount() != null) { + generator.writeNumberField(DELETED_ROWS_COUNT, manifestFile.deletedRowsCount()); + } + + if (manifestFile.partitions() != null) { + generator.writeArrayFieldStart(PARTITION_FIELD_SUMMARY); + for (ManifestFile.PartitionFieldSummary summary : manifestFile.partitions()) { + PartitionFieldSummaryParser.toJson(summary, generator); + } + + generator.writeEndArray(); + } + + if (manifestFile.keyMetadata() != null) { + generator.writeFieldName(KEY_METADATA); + SingleValueParser.toJson(DataFile.KEY_METADATA.type(), manifestFile.keyMetadata(), generator); + } + + generator.writeEndObject(); + } + + static ManifestFile fromJson(JsonNode jsonNode) { + Preconditions.checkArgument(jsonNode != null, "Invalid JSON node for manifest file: null"); + Preconditions.checkArgument( + jsonNode.isObject(), "Invalid JSON node for manifest file: non-object (%s)", jsonNode); + + String path = JsonUtil.getString(PATH, jsonNode); + long length = JsonUtil.getLong(LENGTH, jsonNode); + int specId = JsonUtil.getInt(SPEC_ID, jsonNode); + + ManifestContent manifestContent = null; + if (jsonNode.has(CONTENT)) { + manifestContent = ManifestContent.fromId(JsonUtil.getInt(CONTENT, jsonNode)); + } + + long sequenceNumber = JsonUtil.getLong(SEQUENCE_NUMBER, jsonNode); + long minSequenceNumber = JsonUtil.getLong(MIN_SEQUENCE_NUMBER, jsonNode); + + Long addedSnapshotId = null; + if (jsonNode.has(ADDED_SNAPSHOT_ID)) { + addedSnapshotId = JsonUtil.getLong(ADDED_SNAPSHOT_ID, jsonNode); + } + + Integer addedFilesCount = null; + if (jsonNode.has(ADDED_FILES_COUNT)) { + addedFilesCount = JsonUtil.getInt(ADDED_FILES_COUNT, jsonNode); + } + + Integer existingFilesCount = null; + if (jsonNode.has(EXISTING_FILES_COUNT)) { + existingFilesCount = JsonUtil.getInt(EXISTING_FILES_COUNT, jsonNode); + } + + Integer deletedFilesCount = null; + if (jsonNode.has(DELETED_FILES_COUNT)) { + deletedFilesCount = JsonUtil.getInt(DELETED_FILES_COUNT, jsonNode); + } + + Long addedRowsCount = null; + if (jsonNode.has(ADDED_ROWS_COUNT)) { + addedRowsCount = JsonUtil.getLong(ADDED_ROWS_COUNT, jsonNode); + } + + Long existingRowsCount = null; + if (jsonNode.has(EXISTING_ROWS_COUNT)) { + existingRowsCount = JsonUtil.getLong(EXISTING_ROWS_COUNT, jsonNode); + } + + Long deletedRowsCount = null; + if (jsonNode.has(DELETED_ROWS_COUNT)) { + deletedRowsCount = JsonUtil.getLong(DELETED_ROWS_COUNT, jsonNode); + } + + List partitionFieldSummaries = null; + if (jsonNode.has(PARTITION_FIELD_SUMMARY)) { + JsonNode summaryArray = JsonUtil.get(PARTITION_FIELD_SUMMARY, jsonNode); + Preconditions.checkArgument( + summaryArray.isArray(), + "Invalid JSON node for partition field summaries: non-array (%s)", + summaryArray); + + ImmutableList.Builder builder = ImmutableList.builder(); + for (JsonNode summaryNode : summaryArray) { + ManifestFile.PartitionFieldSummary summary = + PartitionFieldSummaryParser.fromJson(summaryNode); + builder.add(summary); + } + + partitionFieldSummaries = builder.build(); + } + + ByteBuffer keyMetadata = JsonUtil.getByteBufferOrNull(KEY_METADATA, jsonNode); + + return new GenericManifestFile( + path, + length, + specId, + manifestContent, + sequenceNumber, + minSequenceNumber, + addedSnapshotId, + partitionFieldSummaries, + keyMetadata, + addedFilesCount, + addedRowsCount, + existingFilesCount, + existingRowsCount, + deletedFilesCount, + deletedRowsCount); + } + + private static class PartitionFieldSummaryParser { + private static final String CONTAINS_NULL = "contains-null"; + private static final String CONTAINS_NAN = "contains-nan"; + private static final String LOWER_BOUND = "lower-bound"; + private static final String UPPER_BOUND = "upper-bound"; + + private PartitionFieldSummaryParser() {} + + static void toJson(ManifestFile.PartitionFieldSummary summary, JsonGenerator generator) + throws IOException { + Preconditions.checkArgument(summary != null, "Invalid partition field summary: null"); + Preconditions.checkArgument(generator != null, "Invalid JSON generator: null"); + + generator.writeStartObject(); + + generator.writeBooleanField(CONTAINS_NULL, summary.containsNull()); + + if (summary.containsNaN() != null) { + generator.writeBooleanField(CONTAINS_NAN, summary.containsNaN()); + } + + if (summary.lowerBound() != null) { + generator.writeFieldName(LOWER_BOUND); + SingleValueParser.toJson(Types.BinaryType.get(), summary.lowerBound(), generator); + } + + if (summary.upperBound() != null) { + generator.writeFieldName(UPPER_BOUND); + SingleValueParser.toJson(Types.BinaryType.get(), summary.upperBound(), generator); + } + + generator.writeEndObject(); + } + + static ManifestFile.PartitionFieldSummary fromJson(JsonNode jsonNode) { + Preconditions.checkArgument( + jsonNode != null, "Invalid JSON node for partition field summary: null"); + Preconditions.checkArgument( + jsonNode.isObject(), + "Invalid JSON node for partition field summary: non-object (%s)", + jsonNode); + + boolean containsNull = JsonUtil.getBool(CONTAINS_NULL, jsonNode); + Boolean containsNaN = null; + if (jsonNode.has(CONTAINS_NAN)) { + containsNaN = JsonUtil.getBool(CONTAINS_NAN, jsonNode); + } + + ByteBuffer lowerBound = null; + if (jsonNode.has(LOWER_BOUND)) { + lowerBound = + (ByteBuffer) + SingleValueParser.fromJson(Types.BinaryType.get(), jsonNode.get(LOWER_BOUND)); + } + + ByteBuffer upperBound = null; + if (jsonNode.has(UPPER_BOUND)) { + upperBound = + (ByteBuffer) + SingleValueParser.fromJson(Types.BinaryType.get(), jsonNode.get(UPPER_BOUND)); + } + + if (containsNaN != null) { + return new GenericPartitionFieldSummary(containsNull, containsNaN, lowerBound, upperBound); + } else { + return new GenericPartitionFieldSummary(containsNull, lowerBound, upperBound); + } + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/ScanTaskParser.java b/core/src/main/java/org/apache/iceberg/ScanTaskParser.java index 9447d0668a1f..67e44cea7d07 100644 --- a/core/src/main/java/org/apache/iceberg/ScanTaskParser.java +++ b/core/src/main/java/org/apache/iceberg/ScanTaskParser.java @@ -30,7 +30,10 @@ public class ScanTaskParser { private enum TaskType { FILE_SCAN_TASK("file-scan-task"), - DATA_TASK("data-task"); + DATA_TASK("data-task"), + FILES_TABLE_TASK("files-table-task"), + ALL_MANIFESTS_TABLE_TASK("all-manifests-table-task"), + MANIFEST_ENTRIES_TABLE_TASK("manifest-entries-task"); private final String value; @@ -45,6 +48,12 @@ public static TaskType fromTypeName(String value) { return FILE_SCAN_TASK; } else if (DATA_TASK.typeName().equalsIgnoreCase(value)) { return DATA_TASK; + } else if (FILES_TABLE_TASK.typeName().equalsIgnoreCase(value)) { + return FILES_TABLE_TASK; + } else if (ALL_MANIFESTS_TABLE_TASK.typeName().equalsIgnoreCase(value)) { + return ALL_MANIFESTS_TABLE_TASK; + } else if (MANIFEST_ENTRIES_TABLE_TASK.typeName().equalsIgnoreCase(value)) { + return MANIFEST_ENTRIES_TABLE_TASK; } else { throw new IllegalArgumentException("Unknown task type: " + value); } @@ -74,6 +83,17 @@ private static void toJson(FileScanTask fileScanTask, JsonGenerator generator) if (fileScanTask instanceof StaticDataTask) { generator.writeStringField(TASK_TYPE, TaskType.DATA_TASK.typeName()); DataTaskParser.toJson((StaticDataTask) fileScanTask, generator); + } else if (fileScanTask instanceof BaseFilesTable.ManifestReadTask) { + generator.writeStringField(TASK_TYPE, TaskType.FILES_TABLE_TASK.typeName()); + FilesTableTaskParser.toJson((BaseFilesTable.ManifestReadTask) fileScanTask, generator); + } else if (fileScanTask instanceof AllManifestsTable.ManifestListReadTask) { + generator.writeStringField(TASK_TYPE, TaskType.ALL_MANIFESTS_TABLE_TASK.typeName()); + AllManifestsTableTaskParser.toJson( + (AllManifestsTable.ManifestListReadTask) fileScanTask, generator); + } else if (fileScanTask instanceof BaseEntriesTable.ManifestReadTask) { + generator.writeStringField(TASK_TYPE, TaskType.MANIFEST_ENTRIES_TABLE_TASK.typeName()); + ManifestEntriesTableTaskParser.toJson( + (BaseEntriesTable.ManifestReadTask) fileScanTask, generator); } else if (fileScanTask instanceof BaseFileScanTask || fileScanTask instanceof BaseFileScanTask.SplitScanTask) { generator.writeStringField(TASK_TYPE, TaskType.FILE_SCAN_TASK.typeName()); @@ -98,6 +118,12 @@ private static FileScanTask fromJson(JsonNode jsonNode, boolean caseSensitive) { return FileScanTaskParser.fromJson(jsonNode, caseSensitive); case DATA_TASK: return DataTaskParser.fromJson(jsonNode); + case FILES_TABLE_TASK: + return FilesTableTaskParser.fromJson(jsonNode); + case ALL_MANIFESTS_TABLE_TASK: + return AllManifestsTableTaskParser.fromJson(jsonNode); + case MANIFEST_ENTRIES_TABLE_TASK: + return ManifestEntriesTableTaskParser.fromJson(jsonNode); default: throw new UnsupportedOperationException("Unsupported task type: " + taskType.typeName()); } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index 923db6bbd68f..abb2ee6cc3e9 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -43,6 +43,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.util.LocationUtil; import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.PartitionUtil; import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.SerializableSupplier; @@ -331,7 +332,7 @@ public String toString() { this.snapshotsById = indexAndValidateSnapshots(snapshots, lastSequenceNumber); this.schemasById = indexSchemas(); - this.specsById = indexSpecs(specs); + this.specsById = PartitionUtil.indexSpecs(specs); this.sortOrdersById = indexSortOrders(sortOrders); this.refs = validateRefs(currentSnapshotId, refs, snapshotsById); this.statisticsFiles = ImmutableList.copyOf(statisticsFiles); @@ -810,14 +811,6 @@ private Map indexSchemas() { return builder.build(); } - private static Map indexSpecs(List specs) { - ImmutableMap.Builder builder = ImmutableMap.builder(); - for (PartitionSpec spec : specs) { - builder.put(spec.specId(), spec); - } - return builder.build(); - } - private static Map indexSortOrders(List sortOrders) { ImmutableMap.Builder builder = ImmutableMap.builder(); for (SortOrder sortOrder : sortOrders) { @@ -1054,7 +1047,7 @@ public Builder setCurrentSchema(int schemaId) { this.specs = Lists.newArrayList(Iterables.transform(specs, spec -> updateSpecSchema(schema, spec))); specsById.clear(); - specsById.putAll(indexSpecs(specs)); + specsById.putAll(PartitionUtil.indexSpecs(specs)); this.sortOrders = Lists.newArrayList( diff --git a/core/src/main/java/org/apache/iceberg/io/FileIOParser.java b/core/src/main/java/org/apache/iceberg/io/FileIOParser.java index 90ace239c2b7..2743b7b616cd 100644 --- a/core/src/main/java/org/apache/iceberg/io/FileIOParser.java +++ b/core/src/main/java/org/apache/iceberg/io/FileIOParser.java @@ -40,7 +40,7 @@ public static String toJson(FileIO io, boolean pretty) { return JsonUtil.generate(gen -> toJson(io, gen), pretty); } - private static void toJson(FileIO io, JsonGenerator generator) throws IOException { + public static void toJson(FileIO io, JsonGenerator generator) throws IOException { String impl = io.getClass().getName(); Map properties; try { @@ -72,7 +72,7 @@ public static FileIO fromJson(String json, Object conf) { return JsonUtil.parse(json, node -> fromJson(node, conf)); } - private static FileIO fromJson(JsonNode json, Object conf) { + public static FileIO fromJson(JsonNode json, Object conf) { Preconditions.checkArgument(json.isObject(), "Cannot parse FileIO from non-object: %s", json); String impl = JsonUtil.getString(FILE_IO_IMPL, json); Map properties = JsonUtil.getStringMap(PROPERTIES, json); 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 4a5f0c792baa..83735939317b 100644 --- a/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/PartitionUtil.java @@ -26,6 +26,7 @@ import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -98,4 +99,13 @@ public static StructLike coercePartition( projection.wrap(partition); return projection; } + + public static Map indexSpecs(List specs) { + ImmutableMap.Builder builder = ImmutableMap.builder(); + for (PartitionSpec spec : specs) { + builder.put(spec.specId(), spec); + } + + return builder.build(); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java b/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java new file mode 100644 index 000000000000..2f057d7bd5a8 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestAllManifestsTableTaskParser.java @@ -0,0 +1,152 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.junit.jupiter.api.Test; + +public class TestAllManifestsTableTaskParser { + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid manifest task: null"); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.toJson(createTask(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for manifest task: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest task: non-object "); + + assertThatThrownBy(() -> AllManifestsTableTaskParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest task: non-object "); + } + + @Test + public void testParser() { + AllManifestsTable.ManifestListReadTask task = createTask(); + String jsonStr = ScanTaskParser.toJson(task); + assertThat(jsonStr).isEqualTo(taskJson()); + AllManifestsTable.ManifestListReadTask deserializedTask = + (AllManifestsTable.ManifestListReadTask) ScanTaskParser.fromJson(jsonStr, false); + assertTaskEquals(task, deserializedTask); + } + + private AllManifestsTable.ManifestListReadTask createTask() { + Schema dataTableSchema = TestBase.SCHEMA; + HadoopFileIO fileIO = new HadoopFileIO(); + fileIO.initialize(ImmutableMap.of("k1", "v1", "k2", "v2")); + Map specsById = + PartitionUtil.indexSpecs( + Arrays.asList(PartitionSpec.builderFor(dataTableSchema).bucket("data", 16).build())); + + return new AllManifestsTable.ManifestListReadTask( + dataTableSchema, + fileIO, + AllManifestsTable.MANIFEST_FILE_SCHEMA, + specsById, + "/path/manifest-list-file.avro", + Expressions.equal("id", 1), + 1L); + } + + private String taskJson() { + return "{\"task-type\":\"all-manifests-table-task\"," + + "\"data-table-schema\":{\"type\":\"struct\",\"schema-id\":0," + + "\"fields\":[{\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"file-io\":{\"io-impl\":\"org.apache.iceberg.hadoop.HadoopFileIO\"," + + "\"properties\":{\"k1\":\"v1\",\"k2\":\"v2\"}}," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{" + + "\"id\":14,\"name\":\"content\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":1,\"name\":\"path\",\"required\":true,\"type\":\"string\"}," + + "{\"id\":2,\"name\":\"length\",\"required\":true,\"type\":\"long\"}," + + "{\"id\":3,\"name\":\"partition_spec_id\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"added_snapshot_id\",\"required\":false,\"type\":\"long\"}," + + "{\"id\":5,\"name\":\"added_data_files_count\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":6,\"name\":\"existing_data_files_count\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":7,\"name\":\"deleted_data_files_count\",\"required\":false,\"type\":\"int\"}," + + "{\"id\":15,\"name\":\"added_delete_files_count\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":16,\"name\":\"existing_delete_files_count\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":17,\"name\":\"deleted_delete_files_count\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":8,\"name\":\"partition_summaries\",\"required\":false,\"type\":" + + "{\"type\":\"list\",\"element-id\":9,\"element\":{\"type\":\"struct\",\"fields\":[{" + + "\"id\":10,\"name\":\"contains_null\",\"required\":true,\"type\":\"boolean\"}," + + "{\"id\":11,\"name\":\"contains_nan\",\"required\":true,\"type\":\"boolean\"}," + + "{\"id\":12,\"name\":\"lower_bound\",\"required\":false,\"type\":\"string\"}," + + "{\"id\":13,\"name\":\"upper_bound\",\"required\":false,\"type\":\"string\"}]},\"element-required\":true}}," + + "{\"id\":18,\"name\":\"reference_snapshot_id\",\"required\":true,\"type\":\"long\"}]}," + + "\"partition-specs\":[{\"spec-id\":0,\"fields\":[{\"name\":\"data_bucket\"," + + "\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}]," + + "\"manifest-list-Location\":\"/path/manifest-list-file.avro\"," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}," + + "\"reference-snapshot-id\":1}"; + } + + private void assertTaskEquals( + AllManifestsTable.ManifestListReadTask expected, + AllManifestsTable.ManifestListReadTask actual) { + + HadoopFileIO expectedIO = (HadoopFileIO) expected.io(); + HadoopFileIO actualIO = (HadoopFileIO) expected.io(); + assertThat(actualIO.properties()).isEqualTo(expectedIO.properties()); + + assertThat(actual.dataTableSchema().asStruct()) + .as("Data table schema should match") + .isEqualTo(expected.dataTableSchema().asStruct()); + assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); + + assertThat(actual.specsById()).isEqualTo(expected.specsById()); + assertThat(actual.manifestListLocation()).isEqualTo(expected.manifestListLocation()); + assertThat(actual.residual().toString()).isEqualTo(expected.residual().toString()); + assertThat(actual.referenceSnapshotId()).isEqualTo(expected.referenceSnapshotId()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestFilesTableTaskParser.java b/core/src/test/java/org/apache/iceberg/TestFilesTableTaskParser.java new file mode 100644 index 000000000000..bea60601377e --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestFilesTableTaskParser.java @@ -0,0 +1,132 @@ +/* + * 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 com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.util.Arrays; +import java.util.Map; +import org.apache.iceberg.expressions.Expressions; +import org.apache.iceberg.hadoop.HadoopFileIO; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.util.JsonUtil; +import org.apache.iceberg.util.PartitionUtil; +import org.junit.jupiter.api.Test; + +public class TestFilesTableTaskParser { + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> FilesTableTaskParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid files task: null"); + + assertThatThrownBy(() -> FilesTableTaskParser.toJson(createTask(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> FilesTableTaskParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for files task: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> FilesTableTaskParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for files task: non-object "); + + assertThatThrownBy(() -> FilesTableTaskParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for files task: non-object "); + } + + @Test + public void testParser() { + BaseFilesTable.ManifestReadTask task = createTask(); + String jsonStr = ScanTaskParser.toJson(task); + assertThat(jsonStr).isEqualTo(taskJson()); + BaseFilesTable.ManifestReadTask deserializedTask = + (BaseFilesTable.ManifestReadTask) ScanTaskParser.fromJson(jsonStr, false); + assertTaskEquals(task, deserializedTask); + } + + private BaseFilesTable.ManifestReadTask createTask() { + Schema schema = TestBase.SCHEMA; + HadoopFileIO fileIO = new HadoopFileIO(); + fileIO.initialize(ImmutableMap.of("k1", "v1", "k2", "v2")); + Map specsById = + PartitionUtil.indexSpecs( + Arrays.asList(PartitionSpec.builderFor(schema).bucket("data", 16).build())); + ManifestFile manifestFile = TestManifestFileParser.createManifestFile(); + return new BaseFilesTable.ManifestReadTask( + schema, fileIO, specsById, manifestFile, schema, Expressions.equal("id", 1)); + } + + private String taskJson() { + return "{\"task-type\":\"files-table-task\"," + + "\"schema\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{" + + "\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"projection\":{\"type\":\"struct\",\"schema-id\":0,\"fields\":[{" + + "\"id\":3,\"name\":\"id\",\"required\":true,\"type\":\"int\"}," + + "{\"id\":4,\"name\":\"data\",\"required\":true,\"type\":\"string\"}]}," + + "\"file-io\":{\"io-impl\":\"org.apache.iceberg.hadoop.HadoopFileIO\"," + + "\"properties\":{\"k1\":\"v1\",\"k2\":\"v2\"}}," + + "\"partition-specs\":[{\"spec-id\":0,\"fields\":[{" + + "\"name\":\"data_bucket\",\"transform\":\"bucket[16]\",\"source-id\":4,\"field-id\":1000}]}]," + + "\"residual-filter\":{\"type\":\"eq\",\"term\":\"id\",\"value\":1}," + + "\"manifest-file\":{\"path\":\"/path/input.m0.avro\"," + + "\"length\":5878,\"partition-spec-id\":0,\"content\":0,\"sequence-number\":1,\"min-sequence-number\":2," + + "\"added-snapshot-id\":12345678901234567," + + "\"added-files-count\":1,\"existing-files-count\":3,\"deleted-files-count\":0," + + "\"added-rows-count\":10,\"existing-rows-count\":30,\"deleted-rows-count\":0," + + "\"partition-field-summary\":[{\"contains-null\":true,\"contains-nan\":false," + + "\"lower-bound\":\"0A000000\",\"upper-bound\":\"64000000\"}],\"key-metadata\":\"DB030000\"}}"; + } + + private void assertTaskEquals( + BaseFilesTable.ManifestReadTask expected, BaseFilesTable.ManifestReadTask actual) { + assertThat(actual.schema().asStruct()) + .as("Schema should match") + .isEqualTo(expected.schema().asStruct()); + assertThat(actual.projection().asStruct()) + .as("Projected schema should match") + .isEqualTo(expected.projection().asStruct()); + + HadoopFileIO expectedIO = (HadoopFileIO) expected.io(); + HadoopFileIO actualIO = (HadoopFileIO) expected.io(); + assertThat(actualIO.properties()).isEqualTo(expectedIO.properties()); + + assertThat(actual.specsById()).isEqualTo(expected.specsById()); + assertThat(actual.residual().toString()).isEqualTo(expected.residual().toString()); + assertThat(actual.manifest()).isEqualTo(expected.manifest()); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestManifestFileParser.java b/core/src/test/java/org/apache/iceberg/TestManifestFileParser.java new file mode 100644 index 000000000000..5a6e99c984f0 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestManifestFileParser.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.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import com.fasterxml.jackson.core.JsonGenerator; +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; +import java.io.StringWriter; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.JsonUtil; +import org.junit.jupiter.api.Test; + +public class TestManifestFileParser { + @Test + public void nullCheck() throws Exception { + StringWriter writer = new StringWriter(); + JsonGenerator generator = JsonUtil.factory().createGenerator(writer); + + assertThatThrownBy(() -> ManifestFileParser.toJson(null, generator)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid manifest file: null"); + + assertThatThrownBy(() -> ManifestFileParser.toJson(createManifestFile(), null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON generator: null"); + + assertThatThrownBy(() -> ManifestFileParser.fromJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid JSON node for manifest file: null"); + } + + @Test + public void invalidJsonNode() throws Exception { + String jsonStr = "{\"str\":\"1\", \"arr\":[]}"; + ObjectMapper mapper = new ObjectMapper(); + JsonNode rootNode = mapper.reader().readTree(jsonStr); + + assertThatThrownBy(() -> ManifestFileParser.fromJson(rootNode.get("str"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest file: non-object "); + + assertThatThrownBy(() -> ManifestFileParser.fromJson(rootNode.get("arr"))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid JSON node for manifest file: non-object "); + } + + @Test + public void testParser() throws Exception { + ManifestFile manifest = createManifestFile(); + String jsonStr = JsonUtil.generate(gen -> ManifestFileParser.toJson(manifest, gen), false); + assertThat(jsonStr).isEqualTo(manifestFileJson()); + } + + static ManifestFile createManifestFile() { + ByteBuffer lowerBound = Conversions.toByteBuffer(Types.IntegerType.get(), 10); + ByteBuffer upperBound = Conversions.toByteBuffer(Types.IntegerType.get(), 100); + List partitionFieldSummaries = + Arrays.asList(new GenericPartitionFieldSummary(true, false, lowerBound, upperBound)); + ByteBuffer keyMetadata = Conversions.toByteBuffer(Types.IntegerType.get(), 987); + + return new GenericManifestFile( + "/path/input.m0.avro", + 5878L, + 0, + ManifestContent.DATA, + 1L, + 2L, + 12345678901234567L, + 1, + 10L, + 3, + 30L, + 0, + 0L, + partitionFieldSummaries, + keyMetadata); + } + + private String manifestFileJson() { + return "{\"path\":\"/path/input.m0.avro\"," + + "\"length\":5878,\"partition-spec-id\":0,\"content\":0,\"sequence-number\":1,\"min-sequence-number\":2," + + "\"added-snapshot-id\":12345678901234567,\"added-files-count\":1,\"existing-files-count\":3,\"deleted-files-count\":0," + + "\"added-rows-count\":10,\"existing-rows-count\":30,\"deleted-rows-count\":0," + + "\"partition-field-summary\":[{\"contains-null\":true,\"contains-nan\":false," + + "\"lower-bound\":\"0A000000\",\"upper-bound\":\"64000000\"}]," + + "\"key-metadata\":\"DB030000\"}"; + } +} From 1d7a50a31d4325d76b929dca76a52638e7070ff0 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 15 Aug 2024 22:40:43 +0200 Subject: [PATCH 0598/1019] AWS, Core: Slim down Jetty config for tests (#10945) --- .../apache/iceberg/aws/s3/signer/TestS3RestSigner.java | 8 ++------ .../java/org/apache/iceberg/rest/TestRESTCatalog.java | 9 ++------- .../org/apache/iceberg/rest/TestRESTViewCatalog.java | 8 ++------ 3 files changed, 6 insertions(+), 19 deletions(-) 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 08f356ca7ab1..f2a70aee29e5 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 @@ -171,12 +171,8 @@ private static Server initHttpServer() throws Exception { new S3SignerServlet(S3ObjectMapper.mapper(), ImmutableList.of(deleteObjectsWithBody)); ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); - servletContext.setContextPath("/"); - ServletHolder servletHolder = new ServletHolder(servlet); - servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); - servletContext.addServlet(servletHolder, "/*"); - servletContext.setVirtualHosts(null); - servletContext.setGzipHandler(new GzipHandler()); + servletContext.addServlet(new ServletHolder(servlet), "/*"); + servletContext.setHandler(new GzipHandler()); Server server = new Server(0); server.setHandler(servletContext); 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 ced15b865115..1c15cfab43a3 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -155,15 +155,10 @@ public T execute( } }; - RESTCatalogServlet servlet = new RESTCatalogServlet(adaptor); ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); - servletContext.setContextPath("/"); - ServletHolder servletHolder = new ServletHolder(servlet); - servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); - servletContext.addServlet(servletHolder, "/*"); - servletContext.setVirtualHosts(null); - servletContext.setGzipHandler(new GzipHandler()); + servletContext.addServlet(new ServletHolder(new RESTCatalogServlet(adaptor)), "/*"); + servletContext.setHandler(new GzipHandler()); this.httpServer = new Server(0); httpServer.setHandler(servletContext); 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 f67c4b078e5d..db0969620dc9 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -92,15 +92,11 @@ public T execute( } }; - RESTCatalogServlet servlet = new RESTCatalogServlet(adaptor); ServletContextHandler servletContext = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); servletContext.setContextPath("/"); - ServletHolder servletHolder = new ServletHolder(servlet); - servletHolder.setInitParameter("javax.ws.rs.Application", "ServiceListPublic"); - servletContext.addServlet(servletHolder, "/*"); - servletContext.setVirtualHosts(null); - servletContext.setGzipHandler(new GzipHandler()); + servletContext.addServlet(new ServletHolder(new RESTCatalogServlet(adaptor)), "/*"); + servletContext.setHandler(new GzipHandler()); this.httpServer = new Server(0); httpServer.setHandler(servletContext); From b01708f708743e0b20e45c4af34e16fcd5d8500c Mon Sep 17 00:00:00 2001 From: SaketaChalamchala Date: Thu, 15 Aug 2024 22:08:49 -0700 Subject: [PATCH 0599/1019] Docs: Cloudera blog in February 2023 (#10947) --- site/docs/blogs.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 73eac3cf710a..00b0688e86a0 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -215,6 +215,12 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Ron Ortloff](https://www.linkedin.com/in/ron-ortloff/), [Dennis Huo](https://www.linkedin.com/in/dennis-huo-2aaba92a/) + +### [Open Data Lakehouse powered by Apache Iceberg on Apache Ozone](https://medium.com/engineering-cloudera/open-data-lakehouse-powered-by-apache-iceberg-on-apache-ozone-a225d5dcfe98/) +**Date**: February 28th, 2023, **Company**: Cloudera + +**Authors**: [Saketa Chalamchala](https://www.linkedin.com/in/saketa-chalamchala-3602026a) + ### [Dealing with Data Incidents Using the Rollback Feature in Apache Iceberg](https://www.dremio.com/blog/dealing-with-data-incidents-using-the-rollback-feature-in-apache-iceberg/) **Date**: February 24th, 2022, **Company**: Dremio From 78d96ffff354f0e84da2bb228045e08d295cb00e Mon Sep 17 00:00:00 2001 From: Jonathan Leang Date: Fri, 16 Aug 2024 16:08:25 -0700 Subject: [PATCH 0600/1019] Core: V3 Metadata Upgrade Validation and Testing (#10861) Co-authored-by: Jonathan Leang --- .../apache/iceberg/TestFormatVersions.java | 66 ++++++++++++++----- .../org/apache/iceberg/TestTableMetadata.java | 43 +++++++++--- .../iceberg/TestUpdateRequirements.java | 9 ++- 3 files changed, 88 insertions(+), 30 deletions(-) diff --git a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java index 4a9f18581d09..3414f1858e98 100644 --- a/core/src/test/java/org/apache/iceberg/TestFormatVersions.java +++ b/core/src/test/java/org/apache/iceberg/TestFormatVersions.java @@ -28,51 +28,83 @@ public class TestFormatVersions extends TestBase { @Parameters(name = "formatVersion = {0}") protected static List parameters() { - return Arrays.asList(1); + return Arrays.asList(1, 2); } @TestTemplate public void testDefaultFormatVersion() { - assertThat(table.ops().current().formatVersion()).isEqualTo(1); + assertThat(table.ops().current().formatVersion()).isEqualTo(formatVersion); } @TestTemplate public void testFormatVersionUpgrade() { TableOperations ops = table.ops(); - TableMetadata base = ops.current(); - ops.commit(base, base.upgradeToFormatVersion(2)); + int newFormatVersion = formatVersion + 1; + + TableMetadata newTableMetadata = ops.current().upgradeToFormatVersion(newFormatVersion); + + assertThat( + newTableMetadata.changes().stream() + .filter(MetadataUpdate.UpgradeFormatVersion.class::isInstance) + .map(MetadataUpdate.UpgradeFormatVersion.class::cast) + .map(MetadataUpdate.UpgradeFormatVersion::formatVersion)) + .containsExactly(newFormatVersion); + + ops.commit(ops.current(), newTableMetadata); - assertThat(ops.current().formatVersion()).isEqualTo(2); + assertThat(ops.current().formatVersion()).isEqualTo(newFormatVersion); + } + + @TestTemplate + public void testFormatVersionUpgradeToLatest() { + TableOperations ops = table.ops(); + + TableMetadata newTableMetadata = + ops.current().upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION); + + assertThat( + newTableMetadata.changes().stream() + .filter(MetadataUpdate.UpgradeFormatVersion.class::isInstance) + .map(MetadataUpdate.UpgradeFormatVersion.class::cast) + .map(MetadataUpdate.UpgradeFormatVersion::formatVersion)) + .isEqualTo(List.of(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION)); + + ops.commit(ops.current(), newTableMetadata); + + assertThat(ops.current().formatVersion()) + .isEqualTo(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION); } @TestTemplate public void testFormatVersionDowngrade() { TableOperations ops = table.ops(); - TableMetadata base = ops.current(); - ops.commit(base, base.upgradeToFormatVersion(2)); + int newFormatVersion = formatVersion + 1; + ops.commit(ops.current(), ops.current().upgradeToFormatVersion(newFormatVersion)); - assertThat(ops.current().formatVersion()).isEqualTo(2); + assertThat(ops.current().formatVersion()).isEqualTo(newFormatVersion); - assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(1)) + assertThatThrownBy(() -> ops.current().upgradeToFormatVersion(formatVersion)) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot downgrade v2 table to v1"); + .hasMessage( + String.format("Cannot downgrade v%d table to v%d", newFormatVersion, formatVersion)); - assertThat(ops.current().formatVersion()).isEqualTo(2); + assertThat(ops.current().formatVersion()).isEqualTo(newFormatVersion); } @TestTemplate public void testFormatVersionUpgradeNotSupported() { TableOperations ops = table.ops(); TableMetadata base = ops.current(); + int unsupportedFormatVersion = TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1; assertThatThrownBy( - () -> - ops.commit( - base, - base.upgradeToFormatVersion(TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION + 1))) + () -> ops.commit(base, base.upgradeToFormatVersion(unsupportedFormatVersion))) .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Cannot upgrade table to unsupported format version: v4 (supported: v3)"); + .hasMessage( + String.format( + "Cannot upgrade table to unsupported format version: v%d (supported: v%d)", + unsupportedFormatVersion, TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION)); - assertThat(ops.current().formatVersion()).isEqualTo(1); + assertThat(ops.current().formatVersion()).isEqualTo(formatVersion); } } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index e11cc500df55..809b1ba963c7 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -32,6 +32,7 @@ import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assertions.entry; +import static org.junit.jupiter.params.provider.Arguments.arguments; import com.fasterxml.jackson.core.JsonGenerator; import java.io.File; @@ -49,6 +50,8 @@ import java.util.Set; import java.util.SortedSet; import java.util.UUID; +import java.util.stream.IntStream; +import java.util.stream.Stream; import org.apache.iceberg.TableMetadata.MetadataLogEntry; import org.apache.iceberg.TableMetadata.SnapshotLogEntry; import org.apache.iceberg.exceptions.ValidationException; @@ -63,6 +66,9 @@ import org.apache.iceberg.util.JsonUtil; 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 class TestTableMetadata { private static final String TEST_LOCATION = "s3://bucket/test/location"; @@ -1451,8 +1457,20 @@ public void testCreateV2MetadataThroughTableProperty() { .doesNotContainKey(TableProperties.FORMAT_VERSION); } - @Test - public void testReplaceV1MetadataToV2ThroughTableProperty() { + private static Stream upgradeFormatVersionProvider() { + // return a stream of all valid upgrade paths + return IntStream.range(1, TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION) + .boxed() + .flatMap( + baseFormatVersion -> + IntStream.rangeClosed( + baseFormatVersion + 1, TableMetadata.SUPPORTED_TABLE_FORMAT_VERSION) + .mapToObj(newFormatVersion -> arguments(baseFormatVersion, newFormatVersion))); + } + + @ParameterizedTest + @MethodSource("upgradeFormatVersionProvider") + public void testReplaceMetadataThroughTableProperty(int baseFormatVersion, int newFormatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = @@ -1460,7 +1478,8 @@ public void testReplaceV1MetadataToV2ThroughTableProperty() { schema, PartitionSpec.unpartitioned(), null, - ImmutableMap.of(TableProperties.FORMAT_VERSION, "1", "key", "val")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(baseFormatVersion), "key", "val")); meta = meta.buildReplacement( @@ -1468,17 +1487,19 @@ public void testReplaceV1MetadataToV2ThroughTableProperty() { meta.spec(), meta.sortOrder(), meta.location(), - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key2", "val2")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(newFormatVersion), "key2", "val2")); - assertThat(meta.formatVersion()).isEqualTo(2); + assertThat(meta.formatVersion()).isEqualTo(newFormatVersion); assertThat(meta.properties()) .containsEntry("key", "val") .containsEntry("key2", "val2") .doesNotContainKey(TableProperties.FORMAT_VERSION); } - @Test - public void testUpgradeV1MetadataToV2ThroughTableProperty() { + @ParameterizedTest + @MethodSource("upgradeFormatVersionProvider") + public void testUpgradeMetadataThroughTableProperty(int baseFormatVersion, int newFormatVersion) { Schema schema = new Schema(Types.NestedField.required(10, "x", Types.StringType.get())); TableMetadata meta = @@ -1486,15 +1507,17 @@ public void testUpgradeV1MetadataToV2ThroughTableProperty() { schema, PartitionSpec.unpartitioned(), null, - ImmutableMap.of(TableProperties.FORMAT_VERSION, "1", "key", "val")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(baseFormatVersion), "key", "val")); meta = meta.replaceProperties( - ImmutableMap.of(TableProperties.FORMAT_VERSION, "2", "key2", "val2")); + ImmutableMap.of( + TableProperties.FORMAT_VERSION, String.valueOf(newFormatVersion), "key2", "val2")); assertThat(meta.formatVersion()) .as("format version should be configured based on the format-version key") - .isEqualTo(2); + .isEqualTo(newFormatVersion); assertThat(meta.properties()) .as("should not contain format-version but should contain new properties") .containsExactly(entry("key2", "val2")); diff --git a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java index ed1142441736..1a6c289ea241 100644 --- a/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java +++ b/core/src/test/java/org/apache/iceberg/TestUpdateRequirements.java @@ -37,6 +37,8 @@ import org.assertj.core.api.InstanceOfAssertFactories; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; public class TestUpdateRequirements { private final TableMetadata metadata = mock(TableMetadata.class); @@ -184,11 +186,12 @@ public void assignUUIDToViewFailure() { updatedViewMetadata.uuid(), viewMetadata.uuid())); } - @Test - public void upgradeFormatVersion() { + @ParameterizedTest + @ValueSource(ints = {2, 3}) + public void upgradeFormatVersion(int formatVersion) { List requirements = UpdateRequirements.forUpdateTable( - metadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(2))); + metadata, ImmutableList.of(new MetadataUpdate.UpgradeFormatVersion(formatVersion))); requirements.forEach(req -> req.validate(metadata)); assertThat(requirements) From c28084c43d78843a6733bd4a94acaf2a2a2c00cf Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 16 Aug 2024 17:25:34 -0600 Subject: [PATCH 0601/1019] Spec: Clarify in REST spec that server implementations of commit endpoints must fail with 400 if any unknown updates or requirements are received (#10848) --- open-api/rest-catalog-open-api.yaml | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 661af11efc4b..7f0619aff4f9 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -686,6 +686,8 @@ paths: Commits have two parts, requirements and updates. Requirements are assertions that will be validated before attempting to make and commit changes. For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value. + Server implementations are required to fail with a 400 status code + if any unknown updates or requirements are received. Updates are changes to make to table metadata. For example, after asserting that the current main ref @@ -986,7 +988,8 @@ paths: A commit for a single table consists of a table identifier with requirements and updates. Requirements are assertions that will be validated before attempting to make and commit changes. For example, `assert-ref-snapshot-id` will check that a named ref's snapshot ID has a certain value. - + Server implementations are required to fail with a 400 status code + if any unknown updates or requirements are received. Updates are changes to make to table metadata. For example, after asserting that the current main ref is at the expected snapshot, a commit may add a new child snapshot and set the ref to the new From 5bdae99137f24aa0e1b42d8e8791edcb43c4c71f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 18 Aug 2024 12:44:58 +0200 Subject: [PATCH 0602/1019] Build: Bump guava from 33.2.1-jre to 33.3.0-jre (#10960) Bumps `guava` from 33.2.1-jre to 33.3.0-jre. Updates `com.google.guava:guava` from 33.2.1-jre to 33.3.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.2.1-jre to 33.3.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 8ae69c566f2e..5398094b8a2d 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -43,7 +43,7 @@ flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} flink120 = { strictly = "1.20.0"} google-libraries-bom = "26.44.0" -guava = "33.2.1-jre" +guava = "33.3.0-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" httpcomponents-httpclient5 = "5.3.1" From 625ab2d54080c3afd55629e7489beb93ce6658ac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 18 Aug 2024 12:45:31 +0200 Subject: [PATCH 0603/1019] Build: Bump org.springframework:spring-web from 5.3.37 to 5.3.39 (#10959) Bumps [org.springframework:spring-web](https://github.com/spring-projects/spring-framework) from 5.3.37 to 5.3.39. - [Release notes](https://github.com/spring-projects/spring-framework/releases) - [Commits](https://github.com/spring-projects/spring-framework/compare/v5.3.37...v5.3.39) --- updated-dependencies: - dependency-name: org.springframework:spring-web 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 5398094b8a2d..4002623dfa0e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -83,7 +83,7 @@ spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.2" spring-boot = "2.7.18" -spring-web = "5.3.37" +spring-web = "5.3.39" sqlite-jdbc = "3.46.0.1" testcontainers = "1.20.1" tez010 = "0.10.3" From 2e8a32b273b83504cf0a04ac91b612bec491dd76 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Aug 2024 16:47:49 +0200 Subject: [PATCH 0604/1019] Build: Bump software.amazon.awssdk:bom from 2.27.2 to 2.27.7 (#10961) Bumps software.amazon.awssdk:bom from 2.27.2 to 2.27.7. --- 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 4002623dfa0e..09196493a284 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.27.2" +awssdk-bom = "2.27.7" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 656a58149d40df999db76ef7bc521632b80f3d35 Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Mon, 19 Aug 2024 09:46:20 -0700 Subject: [PATCH 0605/1019] Docs: Update MRAP endpoint and add notebook link (#9362) Co-authored-by: Prashant Singh --- docs/docs/aws.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 43e540c20673..973248a2b87c 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -468,13 +468,13 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata --conf spark.sql.catalog.my_catalog.type=glue \ --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ --conf spark.sql.catalog.my_catalog.s3.use-arn-region-enabled=false \ - --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket1=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap \ - --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket1=arn:aws:s3:::accesspoint/ \ + --conf spark.sql.catalog.my_catalog.s3.access-points.my-bucket2=arn:aws:s3:::accesspoint/ ``` -For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3::123456789012:accesspoint:mfzwi23gnjvgw.mrap` +For the above example, the objects in S3 on `my-bucket1` and `my-bucket2` buckets will use `arn:aws:s3:::accesspoint/` access-point for all S3 operations. -For more details on using access-points, please refer [Using access points with compatible Amazon S3 operations](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html). +For more details on using access-points, please refer [Using access points with compatible Amazon S3 operations](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-points-usage-examples.html), [Sample notebook](https://github.com/aws-samples/quant-research/tree/main) . ### S3 Access Grants From 58c231458851999e16b32dac2642bbf056a3863e Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 19 Aug 2024 14:49:06 -0700 Subject: [PATCH 0606/1019] Flink: put everything together for range distribution in Flink sink (#10859) --- docs/docs/flink-configuration.md | 64 +++- docs/docs/flink-writes.md | 101 ++++++ .../apache/iceberg/flink/FlinkConfParser.java | 27 ++ .../apache/iceberg/flink/FlinkWriteConf.java | 21 ++ .../iceberg/flink/FlinkWriteOptions.java | 14 + .../apache/iceberg/flink/sink/FlinkSink.java | 155 +++++++-- .../DataStatisticsOperatorFactory.java | 98 ++++++ .../flink/TestFlinkTableSinkExtended.java | 92 +++++ .../TestFlinkIcebergSinkDistributionMode.java | 314 ++++++++++++++++++ .../flink/sink/TestFlinkIcebergSinkV2.java | 21 +- 10 files changed, 859 insertions(+), 48 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java diff --git a/docs/docs/flink-configuration.md b/docs/docs/flink-configuration.md index 42dc15f5b3d2..1ac16d7fc3e9 100644 --- a/docs/docs/flink-configuration.md +++ b/docs/docs/flink-configuration.md @@ -146,14 +146,56 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ ... ``` -| Flink option | Default | Description | -| ---------------------- | ------------------------------------------ | ------------------------------------------------------------ | -| write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | -| target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | -| upsert-enabled | Table write.upsert.enabled | Overrides this table's write.upsert.enabled | -| overwrite-enabled | false | Overwrite the table's data, overwrite mode shouldn't be enable when configuring to use UPSERT data stream. | -| distribution-mode | Table write.distribution-mode | Overrides this table's write.distribution-mode | -| compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | -| compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | -| compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | -| write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | +| Flink option | Default | Description | +|-----------------------------------------|--------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------------| +| write-format | Table write.format.default | File format to use for this write operation; parquet, avro, or orc | +| target-file-size-bytes | As per table property | Overrides this table's write.target-file-size-bytes | +| upsert-enabled | Table write.upsert.enabled | Overrides this table's write.upsert.enabled | +| overwrite-enabled | false | Overwrite the table's data, overwrite mode shouldn't be enable when configuring to use UPSERT data stream. | +| distribution-mode | Table write.distribution-mode | Overrides this table's write.distribution-mode. RANGE distribution is in experimental status. | +| range-distribution-statistics-type | Auto | Range distribution data statistics collection type: Map, Sketch, Auto. See details [here](#range-distribution-statistics-type). | +| range-distribution-sort-key-base-weight | 0.0 (double) | Base weight for every sort key relative to target traffic weight per writer task. See details [here](#range-distribution-sort-key-base-weight). | +| compression-codec | Table write.(fileformat).compression-codec | Overrides this table's compression codec for this write | +| compression-level | Table write.(fileformat).compression-level | Overrides this table's compression level for Parquet and Avro tables for this write | +| compression-strategy | Table write.orc.compression-strategy | Overrides this table's compression strategy for ORC tables for this write | +| write-parallelism | Upstream operator parallelism | Overrides the writer parallelism | + +#### Range distribution statistics type + +Config value is a enum type: `Map`, `Sketch`, `Auto`. +
        +
      • Map: collects accurate sampling count for every single key. +It should be used for low cardinality scenarios (like hundreds or thousands). +
      • Sketch: constructs a uniform random sampling via reservoir sampling. +It fits well for high cardinality scenarios (like millions), as memory footprint is kept low. +
      • Auto: starts with Maps statistics. But if cardinality is detected higher +than a threshold (currently 10,000), statistics are automatically switched to Sketch. +
      + +#### Range distribution sort key base weight + +`range-distribution-sort-key-base-weight`: `0.0`. + +If sort order contains partition columns, each sort key would map to one partition and data +file. This relative weight can avoid placing too many small files for sort keys with low +traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means +each key has a base weight of `2%` of the targeted traffic weight per writer task. + +E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream +contains events from now up to 180 days ago. With event time, traffic weight distribution +across different days typically has a long tail pattern. Current day contains the most +traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism +is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer +task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, +the range partitioner would put all the oldest 150 days in one writer task. That writer task +would write to 150 small files (one per day). Keeping 150 open files can potentially consume +large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time +can also be potentially slow. If this config is set to `0.02`. It means every sort key has a +base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially +avoid placing more than `50` data files (one per day) on one writer task no matter how small +they are. + +This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For +{@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as +partition columns. Otherwise, too many partitions and small files may be generated during +write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. \ No newline at end of file diff --git a/docs/docs/flink-writes.md b/docs/docs/flink-writes.md index b916a5f9b7b0..f53b5d832efe 100644 --- a/docs/docs/flink-writes.md +++ b/docs/docs/flink-writes.md @@ -262,6 +262,107 @@ INSERT INTO tableName /*+ OPTIONS('upsert-enabled'='true') */ Check out all the options here: [write-options](flink-configuration.md#write-options) +## Distribution mode + +Flink streaming writer supports both `HASH` and `RANGE` distribution mode. +You can enable it via `FlinkSink#Builder#distributionMode(DistributionMode)` +or via [write-options](flink-configuration.md#write-options). + +### Hash distribution + +HASH distribution shuffles data by partition key (partitioned table) or +equality fields (non-partitioned table). It simply leverages Flink's +`DataStream#keyBy` to distribute the data. + +HASH distribution has a few limitations. +
        +
      • It doesn't handle skewed data well. E.g. some partitions have a lot more data than others. +
      • It can result in unbalanced traffic distribution if cardinality of the partition key or +equality fields is low as demonstrated by [PR 4228](https://github.com/apache/iceberg/pull/4228). +
      • Writer parallelism is limited to the cardinality of the hash key. +If the cardinality is 10, only at most 10 writer tasks would get the traffic. +Having higher writer parallelism (even if traffic volume requires) won't help. +
      + +### Range distribution (experimental) + +RANGE distribution shuffles data by partition key or sort order via a custom range partitioner. +Range distribution collects traffic statistics to guide the range partitioner to +evenly distribute traffic to writer tasks. + +Range distribution only shuffle the data via range partitioner. Rows are *not* sorted within +a data file, which Flink streaming writer doesn't support yet. + +#### Use cases + +RANGE distribution can be applied to an Iceberg table that either is partitioned or +has SortOrder defined. For a partitioned table without SortOrder, partition columns +are used as sort order. If SortOrder is explicitly defined for the table, it is used by +the range partitioner. + +Range distribution can handle skewed data. E.g. +
        +
      • Table is partitioned by event time. Typically, recent hours have more data, +while the long-tail hours have less and less data. +
      • Table is partitioned by country code, where some countries (like US) have +a lot more traffic and smaller countries have a lot less data +
      • Table is partitioned by event type, where some types have a lot more data than others. +
      + +Range distribution can also cluster data on non-partition columns. +E.g., table is partitioned hourly on ingestion time. Queries often include +predicate on a non-partition column like `device_id` or `country_code`. +Range partition would improve the query performance by clustering on the non-partition column +when table `SortOrder` is defined with the non-partition column. + +#### Traffic statistics + +Statistics are collected by every shuffle operator subtask and aggregated by the coordinator +for every checkpoint cycle. Aggregated statistics are broadcast to all subtasks and +applied to the range partitioner in the next checkpoint. So it may take up to two checkpoint +cycles to detect traffic distribution change and apply the new statistics to range partitioner. + +Range distribution can work with low cardinality (like `country_code`) +or high cardinality (like `device_id`) scenarios. +
        +
      • For low cardinality scenario (like hundreds or thousands), +HashMap is used to track traffic distribution for every key. +If a new sort key value shows up, range partitioner would just +round-robin it to the writer tasks before traffic distribution has been learned +about the new key. +
      • For high cardinality scenario (like millions or billions), +uniform random sampling (reservoir sampling) is used to compute range bounds +that split the sort key space evenly. +It keeps the memory footprint and network exchange low. +Reservoir sampling work well if key distribution is relatively even. +If a single hot key has unbalanced large share of the traffic, +range split by uniform sampling probably won't work very well. +
      + +#### Usage + +Here is how to enable range distribution in Java. There are two optional advanced configs. Default should +work well for most cases. See [write-options](flink-configuration.md#write-options) for details. +```java +FlinkSink.forRowData(input) + ... + .distributionMode(DistributionMode.RANGE) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .rangeDistributionSortKeyBaseWeight(0.0d) + .append(); +``` + +### Overhead + +Data shuffling (hash or range) has computational overhead of serialization/deserialization +and network I/O. Expect some increase of CPU utilization. + +Range distribution also collect and aggregate data distribution statistics. +That would also incur some CPU overhead. Memory overhead is typically +small if using default statistics type of `Auto`. Don't use `Map` statistics +type if key cardinality is high. That could result in significant memory footprint +and large network exchange for statistics aggregation. + ## Notes Flink streaming write jobs rely on snapshot summary to keep the last committed checkpoint ID, and diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public > EnumConfParser enumConfParser(Class enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser { private String defaultValue; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); 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 769af7d77140..2256d1e874ce 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 @@ -53,13 +53,19 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +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.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -233,15 +239,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + *

      Default is {@link StatisticsType#Auto} where initially Map statistics is used. But if + * cardinality is higher than the threshold (currently 10K) as defined in {@code + * SketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD}, statistics collection automatically switches to + * the sketch reservoir sampling. + * + *

      Explicit set the statistics type if the default behavior doesn't work. + * + * @param type to specify the statistics type for range distribution. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rangeDistributionStatisticsType(StatisticsType type) { + if (type != null) { + writeOptions.put(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key(), type.name()); + } + return this; + } + + /** + * If sort order contains partition columns, each sort key would map to one partition and data + * file. This relative weight can avoid placing too many small files for sort keys with low + * traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means + * each key has a base weight of `2%` of the targeted traffic weight per writer task. + * + *

      E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream + * contains events from now up to 180 days ago. With event time, traffic weight distribution + * across different days typically has a long tail pattern. Current day contains the most + * traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism + * is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer + * task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, + * the range partitioner would put all the oldest 150 days in one writer task. That writer task + * would write to 150 small files (one per day). Keeping 150 open files can potentially consume + * large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time + * can also be potentially slow. If this config is set to `0.02`. It means every sort key has a + * base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially + * avoid placing more than `50` data files (one per day) on one writer task no matter how small + * they are. + * + *

      This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For + * {@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as + * partition columns. Otherwise, too many partitions and small files may be generated during + * write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. + * + *

      Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -349,18 +408,20 @@ private DataStreamSink chainIcebergOperators() { // Find out the equality field id list based on the user-provided equality field column names. List equalityFieldIds = checkAndGetEqualityFieldIds(); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -447,7 +508,10 @@ private SingleOutputStreamOperator appendCommitter( } private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { + DataStream input, + RowType flinkRowType, + List equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +545,13 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), TypeInformation.of(WriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +561,15 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +611,52 @@ private DataStream distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +671,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory + implements CoordinatedOperatorFactory, + OneInputStreamOperatorFactory { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public > T createStreamOperator( + StreamOperatorParameters parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} 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 482cfd110bde..b63547d433a4 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.util.Arrays; @@ -46,6 +47,7 @@ import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; @@ -54,6 +56,7 @@ import org.apache.iceberg.hadoop.HadoopCatalog; 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.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -241,4 +244,93 @@ public void testHashDistributeMode() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); } } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // 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(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } } 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 75e397d3f203..df8c3c79d3e3 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 @@ -20,28 +20,37 @@ 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.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; 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.Snapshot; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; 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.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.types.Conversions; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -177,4 +186,309 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + 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 + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource createRangeDistributionBoundedSource( + List> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List> createCharRows(int numOfCheckpoints, int countPerChar) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List> createIntRows(int numOfCheckpoints, int maxId) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } } 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 577c54976b9a..b283b8390a2b 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 @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -184,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate From 6798dac3ff7354d5330f601b96e805d121139572 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 19 Aug 2024 15:29:47 -0700 Subject: [PATCH 0607/1019] Flink: FLIP-27 IcebergSource builder missed a couple of configs compared to old FlinkSource: expose locality and plan parallelism (#10957) --- .../java/org/apache/iceberg/flink/source/IcebergSource.java | 4 ++++ .../java/org/apache/iceberg/flink/source/IcebergSource.java | 4 ++++ .../java/org/apache/iceberg/flink/source/IcebergSource.java | 4 ++++ 3 files changed, 12 insertions(+) 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 ccbd0d9997ed..e629cc19bbf2 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 @@ -482,6 +482,10 @@ public IcebergSource build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); 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 ccbd0d9997ed..e629cc19bbf2 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 @@ -482,6 +482,10 @@ public IcebergSource build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); 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 ccbd0d9997ed..e629cc19bbf2 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 @@ -482,6 +482,10 @@ public IcebergSource build() { } contextBuilder.resolveConfig(table, readOptions, flinkConfig); + contextBuilder.exposeLocality( + SourceUtil.isLocalityEnabled(table, flinkConfig, exposeLocality)); + contextBuilder.planParallelism( + flinkConfig.get(FlinkConfigOptions.TABLE_EXEC_ICEBERG_WORKER_POOL_SIZE)); Schema icebergSchema = table.schema(); if (projectedFlinkSchema != null) { contextBuilder.project(FlinkSchemaUtil.convert(icebergSchema, projectedFlinkSchema)); From 08111dee47bbd8b5bb50a8a0cf98e00f05b74a7d Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 20 Aug 2024 13:50:00 +0200 Subject: [PATCH 0608/1019] Prevent implicit default locale/charset usage (#10969) --- baseline.gradle | 2 ++ .../flink/sink/shuffle/MapRangePartitionerBenchmark.java | 3 ++- .../main/java/org/apache/iceberg/connect/data/SinkWriter.java | 3 ++- 3 files changed, 6 insertions(+), 2 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index 486625636b59..ba7c2d6c3d09 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -93,6 +93,8 @@ subprojects { '-Xep:ConsistentLoggerName:OFF', '-Xep:DangerousJavaDeserialization:ERROR', '-Xep:DangerousThreadPoolExecutorUsage:OFF', + '-Xep:DefaultCharset:ERROR', + '-Xep:DefaultLocale:ERROR', // subclasses are not equal '-Xep:EqualsGetClass:OFF', // specific to Palantir diff --git a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java index 007b423e592a..24cad2669dbe 100644 --- a/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java +++ b/flink/v1.19/flink/src/jmh/java/org/apache/iceberg/flink/sink/shuffle/MapRangePartitionerBenchmark.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink.shuffle; +import java.nio.charset.StandardCharsets; import java.util.Comparator; import java.util.List; import java.util.Map; @@ -139,7 +140,7 @@ private static String randomString(String prefix) { buffer[i] = (byte) CHARS.charAt(ThreadLocalRandom.current().nextInt(CHARS.length())); } - return prefix + new String(buffer); + return prefix + new String(buffer, StandardCharsets.US_ASCII); } /** find the index where weightsUDF[index] < weight && weightsUDF[index+1] >= weight */ diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java index 35a2957f0122..f81155e13777 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/SinkWriter.java @@ -23,6 +23,7 @@ import java.time.ZoneOffset; import java.util.Collection; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.regex.Pattern; import java.util.stream.Collectors; @@ -119,7 +120,7 @@ private void routeRecordDynamically(SinkRecord record) { String routeValue = extractRouteValue(record.value(), routeField); if (routeValue != null) { - String tableName = routeValue.toLowerCase(); + String tableName = routeValue.toLowerCase(Locale.ROOT); writerForTable(tableName, record, true).write(record); } } From 4352935b1c1796313614a324658d14a4bcadf101 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Tue, 20 Aug 2024 15:08:43 +0200 Subject: [PATCH 0609/1019] Core: Add ManifestWrite benchmark (#8637) * Core: Add ManifestWrite benchmark * Thanks for the review! * Cleanup * Set timeout * Remove public Co-authored-by: Hongyue/Steve Zhang * Make ErrorProne happy --------- Co-authored-by: Hongyue/Steve Zhang --- .../apache/iceberg/ManifestReadBenchmark.java | 7 +- .../iceberg/ManifestWriteBenchmark.java | 170 ++++++++++++++++++ 2 files changed, 172 insertions(+), 5 deletions(-) create mode 100644 core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java index 6677e5d8b651..e4a63beceaa6 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java @@ -69,10 +69,9 @@ public void before() { manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); Random random = new Random(System.currentTimeMillis()); - ManifestListWriter listWriter = - ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0); - try { + try (ManifestListWriter listWriter = + ManifestLists.write(1, org.apache.iceberg.Files.localOutput(manifestListFile), 0, 1L, 0)) { for (int i = 0; i < NUM_FILES; i++) { OutputFile manifestFile = org.apache.iceberg.Files.localOutput( @@ -98,8 +97,6 @@ public void before() { listWriter.add(writer.toManifestFile()); } - - listWriter.close(); } catch (IOException e) { throw new UncheckedIOException(e); } diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java new file mode 100644 index 000000000000..c175248241e9 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ManifestWriteBenchmark.java @@ -0,0 +1,170 @@ +/* + * 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 java.io.File; +import java.io.IOException; +import java.io.UncheckedIOException; +import java.nio.ByteBuffer; +import java.util.Map; +import java.util.Random; +import java.util.UUID; +import java.util.concurrent.TimeUnit; +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; +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; + +/** + * A benchmark that evaluates the performance of writing manifest files + * + *

      To run this benchmark: + * ./gradlew :iceberg-core:jmh -PjmhIncludeRegex=ManifestWriteBenchmark + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 5, timeUnit = TimeUnit.MINUTES) +public class ManifestWriteBenchmark { + + private static final int NUM_FILES = 10; + private static final int NUM_ROWS = 100000; + private static final int NUM_COLS = 100; + + private String baseDir; + private String manifestListFile; + + private Metrics metrics; + + @Setup + public void before() { + Random random = new Random(System.currentTimeMillis()); + // Pre-create the metrics to avoid doing this in the benchmark itself + metrics = randomMetrics(random); + } + + @TearDown + public void after() { + if (baseDir != null) { + FileUtils.deleteQuietly(new File(baseDir)); + baseDir = null; + } + + manifestListFile = null; + } + + @State(Scope.Benchmark) + public static class BenchmarkState { + @Param({"1", "2"}) + private int formatVersion; + + public int getFormatVersion() { + return formatVersion; + } + } + + @Benchmark + @Threads(1) + public void writeManifestFile(BenchmarkState state) throws IOException { + this.baseDir = Files.createTempDir().getAbsolutePath(); + this.manifestListFile = String.format("%s/%s.avro", baseDir, UUID.randomUUID()); + + try (ManifestListWriter listWriter = + ManifestLists.write( + state.getFormatVersion(), + org.apache.iceberg.Files.localOutput(manifestListFile), + 0, + 1L, + 0)) { + for (int i = 0; i < NUM_FILES; i++) { + OutputFile manifestFile = + org.apache.iceberg.Files.localOutput( + String.format("%s/%s.avro", baseDir, UUID.randomUUID())); + + ManifestWriter writer = + ManifestFiles.write( + state.formatVersion, PartitionSpec.unpartitioned(), manifestFile, 1L); + try (ManifestWriter finalWriter = writer) { + for (int j = 0; j < NUM_ROWS; j++) { + DataFile dataFile = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withFormat(FileFormat.PARQUET) + .withPath(String.format("/path/to/data-%s-%s.parquet", i, j)) + .withFileSizeInBytes(j) + .withRecordCount(j) + .withMetrics(metrics) + .build(); + finalWriter.add(dataFile); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + + listWriter.add(writer.toManifestFile()); + } + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private Metrics randomMetrics(Random random) { + long rowCount = 100000L + random.nextInt(1000); + Map columnSizes = Maps.newHashMap(); + Map valueCounts = Maps.newHashMap(); + Map nullValueCounts = Maps.newHashMap(); + Map nanValueCounts = Maps.newHashMap(); + Map lowerBounds = Maps.newHashMap(); + Map upperBounds = Maps.newHashMap(); + for (int i = 0; i < NUM_COLS; i++) { + columnSizes.put(i, 1000000L + random.nextInt(100000)); + valueCounts.put(i, 100000L + random.nextInt(100)); + nullValueCounts.put(i, (long) random.nextInt(5)); + nanValueCounts.put(i, (long) random.nextInt(5)); + byte[] lower = new byte[8]; + random.nextBytes(lower); + lowerBounds.put(i, ByteBuffer.wrap(lower)); + byte[] upper = new byte[8]; + random.nextBytes(upper); + upperBounds.put(i, ByteBuffer.wrap(upper)); + } + + return new Metrics( + rowCount, + columnSizes, + valueCounts, + nullValueCounts, + nanValueCounts, + lowerBounds, + upperBounds); + } +} From 907994da30156c747c5d4316b2897dd73ce7238c Mon Sep 17 00:00:00 2001 From: Jason Date: Tue, 20 Aug 2024 18:24:08 +0300 Subject: [PATCH 0610/1019] S3OutputStream: Don't complete multipart upload on finalize (#10874) --- .../java/org/apache/iceberg/aws/s3/S3OutputStream.java | 10 ++++++++-- 1 file changed, 8 insertions(+), 2 deletions(-) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java index ef0b4debf66b..4e35c77d0557 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3OutputStream.java @@ -253,6 +253,10 @@ private void newStream() throws IOException { @Override public void close() throws IOException { + close(true); + } + + private void close(boolean completeUploads) throws IOException { if (closed) { return; } @@ -262,7 +266,9 @@ public void close() throws IOException { try { stream.close(); - completeUploads(); + if (completeUploads) { + completeUploads(); + } } finally { cleanUpStagingFiles(); } @@ -480,7 +486,7 @@ private void createStagingDirectoryIfNotExists() throws IOException, SecurityExc protected void finalize() throws Throwable { super.finalize(); if (!closed) { - close(); // releasing resources is more important than printing the warning + close(false); // releasing resources is more important than printing the warning String trace = Joiner.on("\n\t").join(Arrays.copyOfRange(createStack, 1, createStack.length)); LOG.warn("Unclosed output stream created by:\n\t{}", trace); } From 102494a34c6e28f51aac2d2ea348bbac4758bae7 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 20 Aug 2024 19:21:09 +0200 Subject: [PATCH 0611/1019] Enable UnusedMethod error-prone check (#10968) --- baseline.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/baseline.gradle b/baseline.gradle index ba7c2d6c3d09..8bf2dd3500e6 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -134,6 +134,7 @@ subprojects { '-Xep:TypeParameterUnusedInFormals:OFF', // Palantir's UnnecessarilyQualified may throw during analysis '-Xep:UnnecessarilyQualified:OFF', + '-Xep:UnusedMethod:ERROR', ) } } From e17bf34b96762a01bce9f63eeff6f648f63095e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 20 Aug 2024 19:54:52 +0200 Subject: [PATCH 0612/1019] Build: Upgrade to Gradle 8.10 (#10976) --- 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 68e8816d71c9..2b189974c29a 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=d725d707bfabd4dfdc958c624003b3c80accc03f7037b5122c4b1d0ef15cecab -distributionUrl=https\://services.gradle.org/distributions/gradle-8.9-bin.zip +distributionSha256Sum=5b9c5eb3f9fc2c94abaea57d90bd78747ca117ddbbf96c859d3741181a12bf2a +distributionUrl=https\://services.gradle.org/distributions/gradle-8.10-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index f7166e1f63af..64084f24de2c 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.9.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.10.0/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From 9e186ab594fa33ff3fbd7d46ba9c255ba9a51bd3 Mon Sep 17 00:00:00 2001 From: Steve Lessard Date: Tue, 20 Aug 2024 10:57:38 -0700 Subject: [PATCH 0613/1019] Core: Support case-insensitivity for column names in PartitionSpec (#10678) --- .../org/apache/iceberg/PartitionSpec.java | 85 +- .../org/apache/iceberg/types/TypeUtil.java | 28 +- .../iceberg/TestSchemaCaseSensitivity.java | 65 ++ ...stPartitionSpecBuilderCaseSensitivity.java | 873 ++++++++++++++++++ .../apache/iceberg/TestPartitionSpecInfo.java | 25 + .../org/apache/iceberg/TestPartitioning.java | 36 + 6 files changed, 1093 insertions(+), 19 deletions(-) create mode 100644 api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java create mode 100644 core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java diff --git a/api/src/main/java/org/apache/iceberg/PartitionSpec.java b/api/src/main/java/org/apache/iceberg/PartitionSpec.java index 08a1c4f9ecfd..9b74893f1831 100644 --- a/api/src/main/java/org/apache/iceberg/PartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/PartitionSpec.java @@ -371,6 +371,7 @@ public static class Builder { new AtomicInteger(unpartitionedLastAssignedId()); // check if there are conflicts between partition and schema field name private boolean checkConflicts = true; + private boolean caseSensitive = true; private Builder(Schema schema) { this.schema = schema; @@ -390,7 +391,8 @@ Builder checkConflicts(boolean check) { } private void checkAndAddPartitionName(String name, Integer sourceColumnId) { - Types.NestedField schemaField = schema.findField(name); + Types.NestedField schemaField = + this.caseSensitive ? schema.findField(name) : schema.caseInsensitiveFindField(name); if (checkConflicts) { if (sourceColumnId != null) { // for identity transform case we allow conflicts between partition and schema field name @@ -427,20 +429,31 @@ private void checkForRedundantPartitions(PartitionField field) { dedupFields.put(dedupKey, field); } + public Builder caseSensitive(boolean sensitive) { + this.caseSensitive = sensitive; + return this; + } + public Builder withSpecId(int newSpecId) { this.specId = newSpecId; return this; } private Types.NestedField findSourceColumn(String sourceName) { - Types.NestedField sourceColumn = schema.findField(sourceName); + Types.NestedField sourceColumn = + this.caseSensitive + ? schema.findField(sourceName) + : schema.caseInsensitiveFindField(sourceName); Preconditions.checkArgument( sourceColumn != null, "Cannot find source column: %s", sourceName); return sourceColumn; } Builder identity(String sourceName, String targetName) { - Types.NestedField sourceColumn = findSourceColumn(sourceName); + return identity(findSourceColumn(sourceName), targetName); + } + + private Builder identity(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName, sourceColumn.fieldId()); PartitionField field = new PartitionField( @@ -451,12 +464,16 @@ Builder identity(String sourceName, String targetName) { } public Builder identity(String sourceName) { - return identity(sourceName, sourceName); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + return identity(sourceColumn, schema.findColumnName(sourceColumn.fieldId())); } public Builder year(String sourceName, String targetName) { + return year(findSourceColumn(sourceName), targetName); + } + + private Builder year(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.year()); checkForRedundantPartitions(field); @@ -465,12 +482,17 @@ public Builder year(String sourceName, String targetName) { } public Builder year(String sourceName) { - return year(sourceName, sourceName + "_year"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return year(sourceColumn, columnName + "_year"); } public Builder month(String sourceName, String targetName) { + return month(findSourceColumn(sourceName), targetName); + } + + private Builder month(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.month()); checkForRedundantPartitions(field); @@ -479,12 +501,17 @@ public Builder month(String sourceName, String targetName) { } public Builder month(String sourceName) { - return month(sourceName, sourceName + "_month"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return month(sourceColumn, columnName + "_month"); } public Builder day(String sourceName, String targetName) { + return day(findSourceColumn(sourceName), targetName); + } + + private Builder day(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.day()); checkForRedundantPartitions(field); @@ -493,12 +520,17 @@ public Builder day(String sourceName, String targetName) { } public Builder day(String sourceName) { - return day(sourceName, sourceName + "_day"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return day(sourceColumn, columnName + "_day"); } public Builder hour(String sourceName, String targetName) { + return hour(findSourceColumn(sourceName), targetName); + } + + private Builder hour(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); PartitionField field = new PartitionField(sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.hour()); checkForRedundantPartitions(field); @@ -507,12 +539,17 @@ public Builder hour(String sourceName, String targetName) { } public Builder hour(String sourceName) { - return hour(sourceName, sourceName + "_hour"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return hour(sourceColumn, columnName + "_hour"); } public Builder bucket(String sourceName, int numBuckets, String targetName) { + return bucket(findSourceColumn(sourceName), numBuckets, targetName); + } + + private Builder bucket(Types.NestedField sourceColumn, int numBuckets, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); fields.add( new PartitionField( sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.bucket(numBuckets))); @@ -520,12 +557,17 @@ public Builder bucket(String sourceName, int numBuckets, String targetName) { } public Builder bucket(String sourceName, int numBuckets) { - return bucket(sourceName, numBuckets, sourceName + "_bucket"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return bucket(sourceColumn, numBuckets, columnName + "_bucket"); } public Builder truncate(String sourceName, int width, String targetName) { + return truncate(findSourceColumn(sourceName), width, targetName); + } + + private Builder truncate(Types.NestedField sourceColumn, int width, String targetName) { checkAndAddPartitionName(targetName); - Types.NestedField sourceColumn = findSourceColumn(sourceName); fields.add( new PartitionField( sourceColumn.fieldId(), nextFieldId(), targetName, Transforms.truncate(width))); @@ -533,11 +575,16 @@ public Builder truncate(String sourceName, int width, String targetName) { } public Builder truncate(String sourceName, int width) { - return truncate(sourceName, width, sourceName + "_trunc"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return truncate(sourceColumn, width, columnName + "_trunc"); } public Builder alwaysNull(String sourceName, String targetName) { - Types.NestedField sourceColumn = findSourceColumn(sourceName); + return alwaysNull(findSourceColumn(sourceName), targetName); + } + + private Builder alwaysNull(Types.NestedField sourceColumn, String targetName) { checkAndAddPartitionName( targetName, sourceColumn.fieldId()); // can duplicate a source column name fields.add( @@ -547,7 +594,9 @@ public Builder alwaysNull(String sourceName, String targetName) { } public Builder alwaysNull(String sourceName) { - return alwaysNull(sourceName, sourceName + "_null"); + Types.NestedField sourceColumn = findSourceColumn(sourceName); + String columnName = schema.findColumnName(sourceColumn.fieldId()); + return alwaysNull(sourceColumn, columnName + "_null"); } // add a partition field with an auto-increment partition field id starting from 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 07d06dcc5a89..7298dfca4c03 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -181,11 +181,37 @@ public static Map indexQuotedNameById( return indexer.byId(); } + /** + * Creates a mapping from lower-case field names to their corresponding field IDs. + * + *

      This method iterates over the fields of the provided struct and maps each field's name + * (converted to lower-case) to its ID. If two fields have the same lower-case name, an + * `IllegalArgumentException` is thrown. + * + * @param struct the struct type whose fields are to be indexed + * @return a map where the keys are lower-case field names and the values are field IDs + * @throws IllegalArgumentException if two fields have the same lower-case name + */ public static Map indexByLowerCaseName(Types.StructType struct) { Map indexByLowerCaseName = Maps.newHashMap(); + + IndexByName indexer = new IndexByName(); + visit(struct, indexer); + Map byName = indexer.byName(); + Map byId = indexer.byId(); + indexByName(struct) .forEach( - (name, integer) -> indexByLowerCaseName.put(name.toLowerCase(Locale.ROOT), integer)); + (name, fieldId) -> { + String key = name.toLowerCase(Locale.ROOT); + Integer existingId = indexByLowerCaseName.put(key, fieldId); + Preconditions.checkArgument( + existingId == null || existingId.equals(fieldId), + "Cannot build lower case index: %s and %s collide", + byId.get(existingId), + byId.get(fieldId)); + indexByLowerCaseName.put(key, fieldId); + }); return indexByLowerCaseName; } diff --git a/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java b/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.java new file mode 100644 index 000000000000..bdb73374bf21 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/TestSchemaCaseSensitivity.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; + +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatIllegalArgumentException; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestSchemaCaseSensitivity { + + @Test + public void testCaseInsensitiveFieldCollision() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get())); + assertThatIllegalArgumentException() + .isThrownBy(() -> schema.caseInsensitiveFindField("DATA")) + .withMessage("Cannot build lower case index: data and DATA collide"); + } + + @Test + public void testCaseSensitiveFindField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get())); + + Types.NestedField actual1 = schema.findField("data"); + assertThat(actual1).isEqualTo(Types.NestedField.required(2, "data", Types.StringType.get())); + Types.NestedField actual2 = schema.findField("DATA"); + assertThat(actual2).isEqualTo(Types.NestedField.required(3, "DATA", Types.StringType.get())); + } + + @Test + public void testCaseInsensitiveField() { + Schema schema = + new Schema( + required(1, "id", Types.LongType.get()), required(2, "data", Types.StringType.get())); + + Types.NestedField actual1 = schema.caseInsensitiveFindField("DATA"); + assertThat(actual1).isEqualTo(Types.NestedField.required(2, "data", Types.StringType.get())); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java new file mode 100644 index 000000000000..349912bd78df --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecBuilderCaseSensitivity.java @@ -0,0 +1,873 @@ +/* + * 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.required; +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 { + + private static final int V2_FORMAT_VERSION = 2; + private static final Schema SCHEMA_WITHOUT_NAME_CONFLICTS = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "category", Types.StringType.get()), + required(4, "order_date", Types.DateType.get()), + required(5, "order_time", Types.TimestampType.withoutZone()), + required(6, "ship_date", Types.DateType.get()), + required(7, "ship_time", Types.TimestampType.withoutZone())); + + private static final Schema SCHEMA_WITH_NAME_CONFLICTS = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get()), + required(4, "order_date", Types.DateType.get()), + required(5, "ORDER_DATE", Types.DateType.get()), + 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(); + } + + @Test + public void testPartitionTypeWithColumnNamesThatDifferOnlyInLetterCase() { + Schema schema = + new Schema( + required(1, "id", Types.IntegerType.get()), + required(2, "data", Types.StringType.get()), + required(3, "DATA", Types.StringType.get()), + required(4, "order_date", Types.DateType.get())); + PartitionSpec spec = PartitionSpec.builderFor(schema).identity("data").identity("DATA").build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "data", Types.StringType.get()), + NestedField.optional(1001, "DATA", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testPartitionTypeWithIdentityTargetName() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .identity("data", "partition1") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "partition1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .bucket("data", 10, "partition1") + .bucket("data", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).bucket("data", 10).build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_bucket", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("DATA", 10) + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_bucket", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("DATA", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("category", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testBucketTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .bucket("data", 10, "partition1") + .bucket("category", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testBucketTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .bucket("data", 10, "partition1") + .bucket("DATA", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testTruncateTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).truncate("data", 10).build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_trunc", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("DATA", 10) + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_trunc", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .truncate("data", 10, "partition1") + .truncate("data", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("DATA", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("category", 10, "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testTruncateTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .truncate("data", 10, "partition1") + .truncate("category", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testTruncateTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .truncate("data", 10, "partition1") + .truncate("DATA", 10, "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testIdentityTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).identity("data").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentityTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("DATA") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentitySourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .identity("data", "partition1") + .identity("data", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: identity(2) conflicts with 1001: PARTITION1: identity(2)"); + } + + @Test + public void testIdentitySourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("DATA", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: identity(2) conflicts with 1001: PARTITION1: identity(2)"); + } + + @Test + public void testIdentityTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("category", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testIdentityTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .identity("data", "partition1") + .identity("category", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testIdentityTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .identity("data", "partition1") + .identity("DATA", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testAlwaysNullTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).alwaysNull("data").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_null", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("DATA") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "data_null", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullSourceNameAllowsExactDuplicateWhenCaseSensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .alwaysNull("data", "partition1") + .alwaysNull("data", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullSourceNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("DATA", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("category", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.StringType.get()), + NestedField.optional(1001, "PARTITION1", Types.StringType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testAlwaysNullTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .alwaysNull("data", "partition1") + .alwaysNull("category", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testAlwaysNullTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .alwaysNull("data", "partition1") + .alwaysNull("DATA", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testYearTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).year("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_year", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_year", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .year("order_date", "partition1") + .year("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: year(4) conflicts with 1001: PARTITION1: year(4)"); + } + + @Test + public void testYearSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: year(4) conflicts with 1001: PARTITION1: year(4)"); + } + + @Test + public void testYearTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testYearTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .year("order_date", "partition1") + .year("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testYearTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .year("order_date", "partition1") + .year("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testMonthTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).month("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_month", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_month", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .month("order_date", "partition1") + .month("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: month(4) conflicts with 1001: PARTITION1: month(4)"); + } + + @Test + public void testMonthSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: month(4) conflicts with 1001: PARTITION1: month(4)"); + } + + @Test + public void testMonthTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testMonthTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .month("order_date", "partition1") + .month("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testMonthTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .month("order_date", "partition1") + .month("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testDayTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).day("order_date").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_day", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDayTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("ORDER_DATE") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_date_day", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDaySourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .day("order_date", "partition1") + .day("order_date", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: day(4) conflicts with 1001: PARTITION1: day(4)"); + } + + @Test + public void testDaySourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ORDER_DATE", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: day(4) conflicts with 1001: PARTITION1: day(4)"); + } + + @Test + public void testDayTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ship_date", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.DateType.get()), + NestedField.optional(1001, "PARTITION1", Types.DateType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testDayTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .day("order_date", "partition1") + .day("ship_date", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testDayTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .day("order_date", "partition1") + .day("ORDER_DATE", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testHourTargetNameDefaultValue() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS).hour("order_time").build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_time_hour", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourTargetNameDefaultValueCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("ORDER_TIME") + .build(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "order_time_hour", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourSourceNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .hour("order_time", "partition1") + .hour("order_time", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: hour(6) conflicts with 1001: PARTITION1: hour(6)"); + } + + @Test + public void testHourSourceNameDoesNotAllowInexactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ORDER_TIME", "PARTITION1") + .build()) + .withMessage( + "Cannot add redundant partition: 1000: partition1: hour(5) conflicts with 1001: PARTITION1: hour(5)"); + } + + @Test + public void testHourTargetNameAllowsInexactDuplicateWhenCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ship_time", "PARTITION1") + .build(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "partition1", Types.IntegerType.get()), + NestedField.optional(1001, "PARTITION1", Types.IntegerType.get())); + StructType actualType = spec.partitionType(); + assertThat(actualType).isEqualTo(expectedType); + } + + @Test + public void testHourTargetNameDoesNotAllowExactDuplicateWhenCaseInsensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITHOUT_NAME_CONFLICTS) + .caseSensitive(false) + .hour("order_time", "partition1") + .hour("ship_time", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } + + @Test + public void testHourTargetNameDoesNotAllowExactDuplicateWhenCaseSensitive() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> + PartitionSpec.builderFor(SCHEMA_WITH_NAME_CONFLICTS) + .hour("order_time", "partition1") + .hour("ORDER_TIME", "partition1") + .build()) + .withMessage("Cannot use partition name more than once: partition1"); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java index a324b8af2e70..f5636a77be61 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitionSpecInfo.java @@ -20,6 +20,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.assertThatIllegalArgumentException; import static org.assertj.core.api.Assertions.entry; import java.io.File; @@ -95,6 +96,30 @@ public void testSpecInfoPartitionedTable() { .doesNotContainKey(Integer.MAX_VALUE); } + @TestTemplate + public void testSpecInfoPartitionedTableCaseInsensitive() { + PartitionSpec spec = + PartitionSpec.builderFor(schema).caseSensitive(false).identity("DATA").build(); + TestTables.TestTable table = TestTables.create(tableDir, "test", schema, spec, formatVersion); + + assertThat(table.spec()).isEqualTo(spec); + assertThat(table.spec().lastAssignedFieldId()).isEqualTo(spec.lastAssignedFieldId()); + assertThat(table.specs()) + .containsExactly(entry(spec.specId(), spec)) + .doesNotContainKey(Integer.MAX_VALUE); + } + + @TestTemplate + public void testSpecInfoPartitionedTableCaseSensitiveFails() { + assertThatIllegalArgumentException() + .isThrownBy( + () -> { + PartitionSpec spec = + PartitionSpec.builderFor(schema).caseSensitive(true).identity("DATA").build(); + }) + .withMessage("Cannot find source column: DATA"); + } + @TestTemplate public void testColumnDropWithPartitionSpecEvolution() { PartitionSpec spec = PartitionSpec.builderFor(schema).identity("id").build(); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitioning.java b/core/src/test/java/org/apache/iceberg/TestPartitioning.java index 91f0fe95c2fa..a4df125f1de2 100644 --- a/core/src/test/java/org/apache/iceberg/TestPartitioning.java +++ b/core/src/test/java/org/apache/iceberg/TestPartitioning.java @@ -122,6 +122,25 @@ public void testPartitionTypeWithRenamesInV1Table() { assertThat(actualType).isEqualTo(expectedType); } + @Test + public void testPartitionTypeWithRenamesInV1TableCaseInsensitive() { + PartitionSpec initialSpec = + PartitionSpec.builderFor(SCHEMA).caseSensitive(false).identity("DATA", "p1").build(); + TestTables.TestTable table = + TestTables.create(tableDir, "test", SCHEMA, initialSpec, V1_FORMAT_VERSION); + + table.updateSpec().addField("category").commit(); + + table.updateSpec().renameField("p1", "p2").commit(); + + StructType expectedType = + StructType.of( + NestedField.optional(1000, "p2", Types.StringType.get()), + NestedField.optional(1001, "category", Types.StringType.get())); + StructType actualType = Partitioning.partitionType(table); + assertThat(actualType).isEqualTo(expectedType); + } + @Test public void testPartitionTypeWithAddingBackSamePartitionFieldInV1Table() { TestTables.TestTable table = @@ -252,6 +271,23 @@ public void testGroupingKeyTypeWithRenamesInV1Table() { assertThat(actualType).isEqualTo(expectedType); } + @Test + public void testGroupingKeyTypeWithRenamesInV1TableCaseInsensitive() { + PartitionSpec initialSpec = + PartitionSpec.builderFor(SCHEMA).caseSensitive(false).identity("DATA", "p1").build(); + TestTables.TestTable table = + TestTables.create(tableDir, "test", SCHEMA, initialSpec, V1_FORMAT_VERSION); + + table.updateSpec().addField("category").commit(); + + table.updateSpec().renameField("p1", "p2").commit(); + + StructType expectedType = + StructType.of(NestedField.optional(1000, "p2", Types.StringType.get())); + StructType actualType = Partitioning.groupingKeyType(table.schema(), table.specs().values()); + assertThat(actualType).isEqualTo(expectedType); + } + @Test public void testGroupingKeyTypeWithRenamesInV2Table() { PartitionSpec initialSpec = PartitionSpec.builderFor(SCHEMA).identity("data", "p1").build(); From e917cf26b19849c3dfa3cd1a5f108456ed9d4839 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 21 Aug 2024 10:09:23 +0200 Subject: [PATCH 0614/1019] Core: Remove unused throws declarations (#10974) --- .../org/apache/iceberg/util/TestParallelIterable.java | 10 +++------- 1 file changed, 3 insertions(+), 7 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 c259bbd0a7e2..ee16f3c1fb5a 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -20,7 +20,6 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -43,8 +42,7 @@ public class TestParallelIterable { @Test - public void closeParallelIteratorWithoutCompleteIteration() - throws IOException, IllegalAccessException, NoSuchFieldException { + public void closeParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); Iterable> transform = @@ -76,8 +74,7 @@ public CloseableIterator iterator() { } @Test - public void closeMoreDataParallelIteratorWithoutCompleteIteration() - throws IOException, IllegalAccessException, NoSuchFieldException { + public void closeMoreDataParallelIteratorWithoutCompleteIteration() { ExecutorService executor = Executors.newFixedThreadPool(1); Iterator integerIterator = new Iterator() { @@ -137,8 +134,7 @@ public CloseableIterator iterator() { } @Test - public void limitQueueSize() throws IOException, IllegalAccessException, NoSuchFieldException { - + public void limitQueueSize() { List> iterables = ImmutableList.of( () -> IntStream.range(0, 100).iterator(), From a24407a75306ea137c782cd4a11bdb8f7d5a4f96 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Wed, 21 Aug 2024 14:56:32 -0700 Subject: [PATCH 0615/1019] Flink: deprecate ReaderFunction with a new Converter interface to simplify user experience (#10956) --- .../iceberg/flink/source/IcebergSource.java | 94 +++++++++++--- .../reader/AvroGenericRecordConverter.java | 69 +++++++++++ .../AvroGenericRecordReaderFunction.java | 10 +- .../reader/ConverterReaderFunction.java | 117 ++++++++++++++++++ .../reader/IcebergSourceSplitReader.java | 2 +- .../flink/source/reader/RowDataConverter.java | 32 +++++ ...TestIcebergSourceBoundedGenericRecord.java | 70 +++++++---- 7 files changed, 351 insertions(+), 43 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java 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 e629cc19bbf2..351ba54e5c7c 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 @@ -61,10 +61,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -211,20 +213,40 @@ private SplitEnumerator createEnumer } } + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static Builder builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param output type + * @return an IcebergSource builder + */ + public static Builder forOutputType(RowDataConverter converter) { + return new Builder().converter(converter); + } + public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private ReaderFunction readerFunction; + private RowDataConverter converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +277,28 @@ public Builder splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder readerFunction(ReaderFunction newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder converter(RowDataConverter newConverter) { + this.converter = newConverter; + return this; + } + public Builder flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -510,25 +549,7 @@ public IcebergSource build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -549,5 +570,40 @@ public IcebergSource build() { table, emitter); } + + private ReaderFunction readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.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.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * 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 java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction extends DataIteratorReaderFunction { + private final RowDataConverter converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param output type + */ +public interface RowDataConverter + extends Function, ResultTypeQueryable, Serializable {} 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 7bfed00a9eb4..4e649d15b1ce 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 @@ -52,6 +52,7 @@ 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; @@ -69,12 +70,13 @@ public class TestIcebergSourceBoundedGenericRecord { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } @@ -84,6 +86,9 @@ public static Object[][] parameters() { @Parameter(index = 1) private int parallelism; + @Parameter(index = 2) + private boolean useConverter; + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = @@ -147,24 +152,15 @@ private List run( table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -172,7 +168,6 @@ private List run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -193,4 +188,35 @@ private List run( return Lists.newArrayList(iter); } } + + 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); + } } From 5eb7c7be0fad64928506ff6ad05766962fae7a4e Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 22 Aug 2024 00:04:57 +0200 Subject: [PATCH 0616/1019] Drop ParallelIterable's queue low water mark (#10978) As part of the change in commit 7831a8dfc3a2de546ca069f4fc1e7afd03777554, queue low water mark was introduced. However, it resulted in increased number of manifests being read when planning LIMIT queries in Trino Iceberg connector. To avoid increased I/O, back out the change for now. --- .../java/org/apache/iceberg/util/ParallelIterable.java | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 27cd96a39733..40bdf1e0c4f8 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -195,12 +195,9 @@ public synchronized boolean hasNext() { // If the consumer is processing records more slowly than the producers, the producers will // eventually fill the queue and yield, returning continuations. Continuations and new tasks // are started by checkTasks(). The check here prevents us from restarting continuations or - // starting new tasks too early (when queue is almost full) or too late (when queue is already - // emptied). Restarting too early would lead to tasks yielding very quickly (CPU waste on - // scheduling). Restarting too late would mean the consumer may need to wait for the tasks - // to produce new items. A consumer slower than producers shouldn't need to wait. - int queueLowWaterMark = maxQueueSize / 2; - if (queue.size() > queueLowWaterMark) { + // starting new tasks before the queue is emptied. Restarting too early would lead to tasks + // yielding very quickly (CPU waste on scheduling). + if (!queue.isEmpty()) { return true; } From e5ad3212f1812fe3c4295b88defceb141b867979 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 22 Aug 2024 00:06:05 +0200 Subject: [PATCH 0617/1019] Check for minimal queue size in ParallelIterable (#10977) --- .../apache/iceberg/util/ParallelIterable.java | 1 + .../iceberg/util/TestParallelIterable.java | 35 +++++++++++++++++++ 2 files changed, 36 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index 40bdf1e0c4f8..f86e5ddc5a8a 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -95,6 +95,7 @@ private ParallelIterator( iterables, iterable -> new Task<>(iterable, queue, closed, maxQueueSize)) .iterator(); this.workerPool = workerPool; + Preconditions.checkArgument(maxQueueSize > 0, "Max queue size must be greater than 0"); this.maxQueueSize = maxQueueSize; // submit 2 tasks per worker at a time this.taskFutures = new CompletableFuture[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; 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 ee16f3c1fb5a..5e37e0390db9 100644 --- a/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java +++ b/core/src/test/java/org/apache/iceberg/util/TestParallelIterable.java @@ -170,6 +170,41 @@ public void limitQueueSize() { 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()); + } + + assertThat(actualValues) + .as("multiset of values returned by the iterator") + .isEqualTo(expectedValues); + + iterator.close(); + executor.shutdownNow(); + } + private void queueHasElements(ParallelIterator iterator) { assertThat(iterator.hasNext()).isTrue(); assertThat(iterator.next()).isNotNull(); From 66997abef4377d62bfe0e752061c65c6cb902afd Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Wed, 21 Aug 2024 20:23:03 -0700 Subject: [PATCH 0618/1019] API, Spark 3.5: Action to compute table stats (#10288) --- .../iceberg/actions/ActionsProvider.java | 6 + .../iceberg/actions/ComputeTableStats.java | 47 ++ .../apache/iceberg/GenericBlobMetadata.java | 8 + .../actions/BaseComputeTableStats.java | 39 ++ spark/v3.5/build.gradle | 2 + .../actions/ComputeTableStatsSparkAction.java | 179 ++++++++ .../iceberg/spark/actions/NDVSketchUtil.java | 93 ++++ .../iceberg/spark/actions/SparkActions.java | 6 + .../spark/sql/stats/ThetaSketchAgg.scala | 121 +++++ .../actions/TestComputeTableStatsAction.java | 417 ++++++++++++++++++ 10 files changed, 918 insertions(+) create mode 100644 api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java create mode 100644 core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java create mode 100644 spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java create mode 100644 spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala create mode 100644 spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.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 2d6ff2679a17..85773febae17 100644 --- a/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java +++ b/api/src/main/java/org/apache/iceberg/actions/ActionsProvider.java @@ -70,4 +70,10 @@ default RewritePositionDeleteFiles rewritePositionDeletes(Table table) { throw new UnsupportedOperationException( this.getClass().getName() + " does not implement rewritePositionDeletes"); } + + /** Instantiates an action to compute table stats. */ + default ComputeTableStats computeTableStats(Table table) { + throw new UnsupportedOperationException( + this.getClass().getName() + " does not implement computeTableStats"); + } } diff --git a/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java b/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java new file mode 100644 index 000000000000..04449d591657 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/actions/ComputeTableStats.java @@ -0,0 +1,47 @@ +/* + * 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.StatisticsFile; + +/** An action that collects statistics of an Iceberg table and writes to Puffin files. */ +public interface ComputeTableStats extends Action { + /** + * Choose the set of columns to collect stats, by default all columns are chosen. + * + * @param columns a set of column names to be analyzed + * @return this for method chaining + */ + ComputeTableStats columns(String... columns); + + /** + * Choose the table snapshot to compute stats, by default the current snapshot is used. + * + * @param snapshotId long ID of the snapshot for which stats need to be computed + * @return this for method chaining + */ + ComputeTableStats snapshot(long snapshotId); + + /** The result of table statistics collection. */ + interface Result { + + /** Returns statistics file or none if no statistics were collected. */ + StatisticsFile statisticsFile(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java b/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java index 46bedfa01753..d3ac399556c3 100644 --- a/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java +++ b/core/src/main/java/org/apache/iceberg/GenericBlobMetadata.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -37,6 +38,13 @@ public static BlobMetadata from(org.apache.iceberg.puffin.BlobMetadata puffinMet puffinMetadata.properties()); } + public static List from( + Collection puffinMetadataList) { + return puffinMetadataList.stream() + .map(GenericBlobMetadata::from) + .collect(ImmutableList.toImmutableList()); + } + private final String type; private final long sourceSnapshotId; private final long sourceSnapshotSequenceNumber; diff --git a/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java b/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java new file mode 100644 index 000000000000..71941af1d7f0 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/actions/BaseComputeTableStats.java @@ -0,0 +1,39 @@ +/* + * 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 javax.annotation.Nullable; +import org.apache.iceberg.StatisticsFile; +import org.immutables.value.Value; + +@Value.Enclosing +@SuppressWarnings("ImmutablesStyle") +@Value.Style( + typeImmutableEnclosing = "ImmutableComputeTableStats", + visibilityString = "PUBLIC", + builderVisibilityString = "PUBLIC") +interface BaseComputeTableStats extends ComputeTableStats { + + @Value.Immutable + interface Result extends ComputeTableStats.Result { + @Override + @Nullable + StatisticsFile statisticsFile(); + } +} diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index 2ba5d493c6cd..c8d8bbf396a0 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -59,6 +59,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation("org.apache.datasketches:datasketches-java:${libs.versions.datasketches.get()}") if (scalaVersion == '2.12') { // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support implementation 'org.scala-lang:scala-library:2.12.18' @@ -292,6 +293,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap' + relocate 'org.apache.datasketches', 'org.apache.iceberg.shaded.org.apache.datasketches' archiveClassifier.set(null) } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java new file mode 100644 index 000000000000..a508021c1040 --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.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 java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ImmutableComputeTableStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +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.spark.JobGroupInfo; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Computes the statistics of the given columns and stores it as Puffin files. */ +public class ComputeTableStatsSparkAction extends BaseSparkAction + implements ComputeTableStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputeTableStatsSparkAction.class); + private static final Result EMPTY_RESULT = ImmutableComputeTableStats.Result.builder().build(); + + private final Table table; + private List columns; + private Snapshot snapshot; + + ComputeTableStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputeTableStatsSparkAction self() { + return this; + } + + @Override + public ComputeTableStats columns(String... newColumns) { + Preconditions.checkArgument( + newColumns != null && newColumns.length > 0, "Columns cannot be null/empty"); + this.columns = ImmutableList.copyOf(ImmutableSet.copyOf(newColumns)); + return this; + } + + @Override + public ComputeTableStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute stats for table {}", table.name()); + return EMPTY_RESULT; + } + validateColumns(); + JobGroupInfo info = newJobGroupInfo("COMPUTE-TABLE-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info( + "Computing stats for columns {} in {} (snapshot {})", + columns(), + table.name(), + snapshotId()); + List blobs = generateNDVBlobs(); + StatisticsFile statisticsFile = writeStatsFile(blobs); + table.updateStatistics().setStatistics(snapshotId(), statisticsFile).commit(); + return ImmutableComputeTableStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private StatisticsFile writeStatsFile(List blobs) { + LOG.info("Writing stats for table {} for snapshot {}", table.name(), snapshotId()); + OutputFile outputFile = table.io().newOutputFile(outputPath()); + try (PuffinWriter writer = Puffin.write(outputFile).createdBy(appIdentifier()).build()) { + blobs.forEach(writer::add); + writer.finish(); + return new GenericStatisticsFile( + snapshotId(), + outputFile.location(), + writer.fileSize(), + writer.footerSize(), + GenericBlobMetadata.from(writer.writtenBlobsMetadata())); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private List generateNDVBlobs() { + return NDVSketchUtil.generateBlobs(spark(), table, snapshot, columns()); + } + + private List columns() { + if (columns == null) { + Schema schema = table.schemas().get(snapshot.schemaId()); + this.columns = + schema.columns().stream() + .filter(nestedField -> nestedField.type().isPrimitiveType()) + .map(Types.NestedField::name) + .collect(Collectors.toList()); + } + return columns; + } + + private void validateColumns() { + Schema schema = table.schemas().get(snapshot.schemaId()); + Preconditions.checkArgument(!columns().isEmpty(), "No columns found to compute stats"); + for (String columnName : columns()) { + Types.NestedField field = schema.findField(columnName); + Preconditions.checkArgument(field != null, "Can't find column %s in %s", columnName, schema); + Preconditions.checkArgument( + field.type().isPrimitiveType(), + "Can't compute stats on non-primitive type column: %s (%s)", + columnName, + field.type()); + } + } + + private String appIdentifier() { + String icebergVersion = IcebergBuild.fullVersion(); + String sparkVersion = spark().version(); + return String.format("Iceberg %s Spark %s", icebergVersion, sparkVersion); + } + + private long snapshotId() { + return snapshot.snapshotId(); + } + + private String jobDesc() { + return String.format( + "Computing table stats for %s (snapshot_id=%s, columns=%s)", + table.name(), snapshotId(), columns()); + } + + private String outputPath() { + TableOperations operations = ((HasTableOperations) table).operations(); + String fileName = String.format("%s-%s.stats", snapshotId(), UUID.randomUUID()); + return operations.metadataFileLocation(fileName); + } +} diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java new file mode 100644 index 000000000000..22055a161e4e --- /dev/null +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.theta.CompactSketch; +import org.apache.datasketches.theta.Sketch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.PuffinCompressionCodec; +import org.apache.iceberg.puffin.StandardBlobTypes; +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.SparkReadOptions; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.stats.ThetaSketchAgg; + +public class NDVSketchUtil { + + private NDVSketchUtil() {} + + public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv"; + + static List generateBlobs( + SparkSession spark, Table table, Snapshot snapshot, List columns) { + Row sketches = computeNDVSketches(spark, table, snapshot, columns); + Schema schema = table.schemas().get(snapshot.schemaId()); + List blobs = Lists.newArrayList(); + for (int i = 0; i < columns.size(); i++) { + Types.NestedField field = schema.findField(columns.get(i)); + Sketch sketch = CompactSketch.wrap(Memory.wrap((byte[]) sketches.get(i))); + blobs.add(toBlob(field, sketch, snapshot)); + } + return blobs; + } + + private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snapshot) { + return new Blob( + StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + ByteBuffer.wrap(sketch.toByteArray()), + PuffinCompressionCodec.ZSTD, + ImmutableMap.of( + APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, + String.valueOf((long) sketch.getEstimate()))); + } + + private static Row computeNDVSketches( + SparkSession spark, Table table, Snapshot snapshot, List colNames) { + return spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshot.snapshotId()) + .load(table.name()) + .select(toAggColumns(colNames)) + .first(); + } + + private static Column[] toAggColumns(List colNames) { + return colNames.stream().map(NDVSketchUtil::toAggColumn).toArray(Column[]::new); + } + + private static Column toAggColumn(String colName) { + ThetaSketchAgg agg = new ThetaSketchAgg(colName); + return new Column(agg.toAggregateExpression()); + } +} 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 fb67ded96e35..f845386d30c4 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 @@ -20,6 +20,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.ComputeTableStats; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -96,4 +97,9 @@ public DeleteReachableFilesSparkAction deleteReachableFiles(String metadataLocat public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) { return new RewritePositionDeleteFilesSparkAction(spark, table); } + + @Override + public ComputeTableStats computeTableStats(Table table) { + return new ComputeTableStatsSparkAction(spark, table); + } } diff --git a/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala new file mode 100644 index 000000000000..cca16960f434 --- /dev/null +++ b/spark/v3.5/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -0,0 +1,121 @@ +/* + * 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.spark.sql.stats + +import java.nio.ByteBuffer +import org.apache.datasketches.common.Family +import org.apache.datasketches.memory.Memory +import org.apache.datasketches.theta.CompactSketch +import org.apache.datasketches.theta.SetOperationBuilder +import org.apache.datasketches.theta.Sketch +import org.apache.datasketches.theta.UpdateSketch +import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.Conversions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.UTF8String + +/** + * ThetaSketchAgg generates Alpha family sketch with default seed. + * The values fed to the sketch are converted to bytes using Iceberg's single value serialization. + * The result returned is an array of bytes of Compact Theta sketch of Datasketches library, + * which should be deserialized to Compact sketch before using. + * + * See [[https://iceberg.apache.org/puffin-spec/]] for more information. + * + */ +case class ThetaSketchAgg( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + + private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) + + def this(colName: String) = { + this(col(colName).expr, 0, 0) + } + + override def dataType: DataType = BinaryType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): Sketch = { + UpdateSketch.builder.setFamily(Family.ALPHA).build() + } + + override def update(buffer: Sketch, input: InternalRow): Sketch = { + val value = child.eval(input) + if (value != null) { + val icebergValue = toIcebergValue(value) + val byteBuffer = Conversions.toByteBuffer(icebergType, icebergValue) + buffer.asInstanceOf[UpdateSketch].update(byteBuffer) + } + buffer + } + + private def toIcebergValue(value: Any): Any = { + value match { + case s: UTF8String => s.toString + case d: Decimal => d.toJavaBigDecimal + case b: Array[Byte] => ByteBuffer.wrap(b) + case _ => value + } + } + + override def merge(buffer: Sketch, input: Sketch): Sketch = { + new SetOperationBuilder().buildUnion.union(buffer, input) + } + + override def eval(buffer: Sketch): Any = { + toBytes(buffer) + } + + override def serialize(buffer: Sketch): Array[Byte] = { + toBytes(buffer) + } + + override def deserialize(storageFormat: Array[Byte]): Sketch = { + CompactSketch.wrap(Memory.wrap(storageFormat)) + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + + private def toBytes(sketch: Sketch): Array[Byte] = { + val compactSketch = sketch.compact() + compactSketch.toByteArray + } +} 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 new file mode 100644 index 000000000000..588bb29f47dd --- /dev/null +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -0,0 +1,417 @@ +/* + * 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.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +import static org.apache.iceberg.types.Types.NestedField.optional; +import static org.apache.iceberg.types.Types.NestedField.required; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import java.io.IOException; +import java.util.List; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +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.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.CatalogTestBase; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.StructType; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.TestTemplate; + +public class TestComputeTableStatsAction extends CatalogTestBase { + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema SCHEMA_WITH_NESTED_COLUMN = + new Schema( + required(4, "nestedStructCol", NESTED_STRUCT_TYPE), + required(5, "stringCol", Types.StringType.get())); + + @TestTemplate + public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + // To create multiple splits on the mapper + table + .updateProperties() + .set("read.split.target-size", "100") + .set("write.parquet.row-group-size-bytes", "100") + .commit(); + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns("id", "data").execute(); + assertNotNull(results); + + List statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 2); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertEquals( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), + String.valueOf(4)); + } + + @TestTemplate + public void testComputeTableStatsActionWithoutExplicitColumns() + throws NoSuchTableException, ParseException { + 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, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + assertNotNull(results); + + Assertions.assertEquals(1, table.statisticsFiles().size()); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + Assertions.assertEquals(2, statisticsFile.blobMetadata().size()); + assertNotEquals(0, statisticsFile.fileSizeInBytes()); + Assertions.assertEquals( + 4, + Long.parseLong( + statisticsFile + .blobMetadata() + .get(0) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))); + Assertions.assertEquals( + 4, + Long.parseLong( + statisticsFile + .blobMetadata() + .get(1) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))); + } + + @TestTemplate + public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> actions.computeTableStats(table).columns("id1").execute()); + String message = exception.getMessage(); + assertTrue(message.contains("Can't find column id1 in table")); + } + + @TestTemplate + public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); + Assertions.assertNull(result.statisticsFile()); + } + + @TestTemplate + public void testComputeTableStatsWithNullValues() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + List records = + Lists.newArrayList( + new SimpleRecord(1, null), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + assertNotNull(results); + + List statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertEquals( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), + String.valueOf(4)); + } + + @TestTemplate + public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() + throws NoSuchTableException, ParseException { + SparkActions actions = SparkActions.get(); + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + long snapshotId1 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + // Snapshot id not specified + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + assertDoesNotThrow(() -> actions.computeTableStats(table).columns("data").execute()); + + sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); + // Append data to create snapshot + sql("INSERT into %s values(1)", tableName); + table.refresh(); + long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + + // Snapshot id specified + assertDoesNotThrow( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()); + + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()); + String message = exception.getMessage(); + assertTrue(message.contains("Can't find column data in table")); + } + + @TestTemplate + public void testComputeTableStatsWhenSnapshotIdNotSpecified() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + + assertNotNull(results); + + List statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertEquals( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), + String.valueOf(1)); + } + + @TestTemplate + public void testComputeTableStatsWithNestedSchema() + throws NoSuchTableException, ParseException, IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, + SCHEMA_WITH_NESTED_COLUMN, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + Table tbl = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + actions.computeTableStats(tbl).execute(); + + tbl.refresh(); + List statisticsFiles = tbl.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + } + + @TestTemplate + public void testComputeTableStatsWithNoComputableColumns() throws IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, NESTED_SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); + DataFile dataFile = FileHelpers.writeDataFile(table, Files.localOutput(temp.toFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + table.refresh(); + SparkActions actions = SparkActions.get(); + IllegalArgumentException exception = + assertThrows( + IllegalArgumentException.class, () -> actions.computeTableStats(table).execute()); + Assertions.assertEquals(exception.getMessage(), "No columns found to compute stats"); + } + + @TestTemplate + public void testComputeTableStatsOnByteColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("byte_col", "TINYINT"); + } + + @TestTemplate + public void testComputeTableStatsOnShortColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("short_col", "SMALLINT"); + } + + @TestTemplate + public void testComputeTableStatsOnIntColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("int_col", "INT"); + } + + @TestTemplate + public void testComputeTableStatsOnLongColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("long_col", "BIGINT"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP"); + } + + @TestTemplate + public void testComputeTableStatsOnTimestampNtzColumn() + throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP_NTZ"); + } + + @TestTemplate + public void testComputeTableStatsOnDateColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("date_col", "DATE"); + } + + @TestTemplate + public void testComputeTableStatsOnDecimalColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("decimal_col", "DECIMAL(20, 2)"); + } + + @TestTemplate + public void testComputeTableStatsOnBinaryColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("binary_col", "BINARY"); + } + + public void testComputeTableStats(String columnName, String type) + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, %s %s) USING iceberg", tableName, columnName, type); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + Dataset dataDF = randomDataDF(table.schema()); + append(tableName, dataDF); + + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns(columnName).execute(); + assertNotNull(results); + + List statisticsFiles = table.statisticsFiles(); + Assertions.assertEquals(statisticsFiles.size(), 1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + Assertions.assertNotNull( + blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)); + } + + private GenericRecord createNestedRecord() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_NESTED_COLUMN); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, 0L); + leaf.set(1, 0.0); + nested.set(0, leaf); + record.set(0, nested); + record.set(1, "data"); + return record; + } + + private Dataset randomDataDF(Schema schema) { + Iterable rows = RandomData.generateSpark(schema, 10, 0); + JavaRDD rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } + + @AfterEach + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } +} From 3aac099b17394815637650bbf2333d5ec4a9c7bd Mon Sep 17 00:00:00 2001 From: S N Munendra <9696252+munendrasn@users.noreply.github.com> Date: Thu, 22 Aug 2024 09:35:23 +0530 Subject: [PATCH 0619/1019] Core,AWS: Fix NPE in ResolvingFileIO when HadoopConf is not set (#10872) --- .../java/org/apache/iceberg/aws/s3/TestS3FileIO.java | 12 ++++++++++++ .../java/org/apache/iceberg/io/ResolvingFileIO.java | 10 ++++++---- 2 files changed, 18 insertions(+), 4 deletions(-) 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 38489e367434..1cc34a59cc1b 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 @@ -384,6 +384,18 @@ public void testResolvingFileIOLoad() { assertThat(result).isInstanceOf(S3FileIO.class); } + @Test + public void testResolvingFileIOLoadWithoutConf() { + ResolvingFileIO resolvingFileIO = new ResolvingFileIO(); + resolvingFileIO.initialize(ImmutableMap.of()); + FileIO result = + DynMethods.builder("io") + .hiddenImpl(ResolvingFileIO.class, String.class) + .build(resolvingFileIO) + .invoke("s3://foo/bar"); + assertThat(result).isInstanceOf(S3FileIO.class); + } + @Test public void testInputFileWithDataFile() throws IOException { String location = "s3://bucket/path/to/data-file.parquet"; 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 b5232960dcfb..a858045aab8b 100644 --- a/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java +++ b/core/src/main/java/org/apache/iceberg/io/ResolvingFileIO.java @@ -21,8 +21,10 @@ import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.concurrent.atomic.AtomicBoolean; import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.CatalogUtil; @@ -141,7 +143,7 @@ public void close() { @Override public void serializeConfWith( Function> confSerializer) { - this.hadoopConf = confSerializer.apply(hadoopConf.get()); + this.hadoopConf = confSerializer.apply(getConf()); } @Override @@ -151,7 +153,7 @@ public void setConf(Configuration conf) { @Override public Configuration getConf() { - return hadoopConf.get(); + return Optional.ofNullable(hadoopConf).map(Supplier::get).orElse(null); } @VisibleForTesting @@ -163,7 +165,7 @@ DelegateFileIO io(String location) { synchronized (io) { if (((HadoopConfigurable) io).getConf() == null) { // re-apply the config in case it's null after Kryo serialization - ((HadoopConfigurable) io).setConf(hadoopConf.get()); + ((HadoopConfigurable) io).setConf(getConf()); } } } @@ -174,7 +176,7 @@ DelegateFileIO io(String location) { return ioInstances.computeIfAbsent( impl, key -> { - Configuration conf = hadoopConf.get(); + Configuration conf = getConf(); FileIO fileIO; try { From 4f7db301eedd09514af89477fd0d230c96c27e70 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 22 Aug 2024 21:58:12 +0800 Subject: [PATCH 0620/1019] Spark 3.5: Fix incorrect catalog loaded in TestCreateActions (#10952) --- .../iceberg/spark/actions/TestCreateActions.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) 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 a0f1fba0434c..e2ebf7e8817c 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 @@ -186,6 +186,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @TestTemplate @@ -746,6 +751,8 @@ public void testStructOfThreeLevelLists() throws Exception { @TestTemplate public void testTwoLevelList() throws IOException { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -829,6 +836,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -851,6 +860,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -876,6 +887,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -903,6 +916,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + assumeThat(type).as("Cannot migrate to a hadoop based catalog").isNotEqualTo("hadoop"); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); From 692a6a03448354bdd2fe60c9c457a0b4db5f18ba Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 22 Aug 2024 16:19:31 +0200 Subject: [PATCH 0621/1019] Flink: Maintenance - TriggerManager (#10484) --- flink/v1.19/build.gradle | 1 + .../maintenance/operator/JdbcLockFactory.java | 321 +++++++++ .../maintenance/operator/MonitorSource.java | 4 +- .../maintenance/operator/TableChange.java | 43 +- .../operator/TableMaintenanceMetrics.java | 34 + .../flink/maintenance/operator/Trigger.java | 72 ++ .../operator/TriggerEvaluator.java | 128 ++++ .../operator/TriggerLockFactory.java | 63 ++ .../maintenance/operator/TriggerManager.java | 339 ++++++++++ .../operator/ConstantsForTests.java | 29 + .../MetricsReporterFactoryForTests.java | 153 +++++ .../operator/OperatorTestBase.java | 107 ++- .../operator/TestJdbcLockFactory.java | 57 ++ .../operator/TestLockFactoryBase.java | 80 +++ .../operator/TestMonitorSource.java | 28 +- .../operator/TestTriggerManager.java | 622 ++++++++++++++++++ ...ink.metrics.reporter.MetricReporterFactory | 16 + 17 files changed, 2081 insertions(+), 16 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java create mode 100644 flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index 392a1cb124f0..6327ebf99dbd 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -119,6 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.sqlite.jdbc } test { 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/operator/JdbcLockFactory.java new file mode 100644 index 000000000000..21c8935abede --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java @@ -0,0 +1,321 @@ +/* + * 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.IOException; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +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; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JdbcLockFactory implements TriggerLockFactory { + private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); + + @VisibleForTesting + static final String INIT_LOCK_TABLES_PROPERTY = "flink-maintenance.lock.jdbc.init-lock-tables"; + + private static final String LOCK_TABLE_NAME = "flink_maintenance_lock"; + private static final int LOCK_ID_MAX_LENGTH = 100; + private static final String CREATE_LOCK_TABLE_SQL = + String.format( + "CREATE TABLE %s " + + "(LOCK_TYPE CHAR(1) NOT NULL, " + + "LOCK_ID VARCHAR(%s) NOT NULL, " + + "INSTANCE_ID CHAR(36) NOT NULL, PRIMARY KEY (LOCK_TYPE, LOCK_ID))", + LOCK_TABLE_NAME, LOCK_ID_MAX_LENGTH); + + private static final String CREATE_LOCK_SQL = + String.format( + "INSERT INTO %s (LOCK_TYPE, LOCK_ID, INSTANCE_ID) VALUES (?, ?, ?)", LOCK_TABLE_NAME); + private static final String GET_LOCK_SQL = + String.format("SELECT INSTANCE_ID FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=?", LOCK_TABLE_NAME); + private static final String DELETE_LOCK_SQL = + String.format( + "DELETE FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=? AND INSTANCE_ID=?", LOCK_TABLE_NAME); + + private final String uri; + private final String lockId; + private final Map properties; + private transient JdbcClientPool pool; + + /** + * Creates a new {@link TriggerLockFactory}. The lockId should be unique between the users of the + * same uri. + * + * @param uri of the jdbc connection + * @param lockId which should indentify the job and the table + * @param properties used for creating the jdbc connection pool + */ + public JdbcLockFactory(String uri, String lockId, Map properties) { + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + Preconditions.checkNotNull(properties, "Properties map is required"); + Preconditions.checkArgument( + lockId.length() < LOCK_ID_MAX_LENGTH, + "Invalid prefix length: lockId should be shorter than %s", + LOCK_ID_MAX_LENGTH); + this.uri = uri; + this.lockId = lockId; + this.properties = properties; + } + + @Override + public void open() { + this.pool = new JdbcClientPool(1, uri, properties); + + if (PropertyUtil.propertyAsBoolean(properties, INIT_LOCK_TABLES_PROPERTY, false)) { + initializeLockTables(); + } + } + + /** Only used in testing to share the jdbc pool */ + @VisibleForTesting + void open(JdbcLockFactory other) { + this.pool = other.pool; + } + + @Override + public Lock createLock() { + return new Lock(pool, lockId, Type.MAINTENANCE); + } + + @Override + public Lock createRecoveryLock() { + return new Lock(pool, lockId, Type.RECOVERY); + } + + @Override + public void close() throws IOException { + pool.close(); + } + + private void initializeLockTables() { + LOG.debug("Creating database tables (if missing) to store table maintenance locks"); + try { + pool.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + LOCK_TABLE_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + LOG.debug("Flink maintenance lock table already exists"); + return true; + } + + LOG.info("Creating Flink maintenance lock table {}", LOCK_TABLE_NAME); + return conn.prepareStatement(CREATE_LOCK_TABLE_SQL).execute(); + }); + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC table maintenance lock"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + public static class Lock implements TriggerLockFactory.Lock { + private final JdbcClientPool pool; + private final String lockId; + private final Type type; + + public Lock(JdbcClientPool pool, String lockId, Type type) { + this.pool = pool; + this.lockId = lockId; + this.type = type; + } + + @Override + public boolean tryLock() { + if (isHeld()) { + LOG.info("Lock is already held for {}", this); + return false; + } + + String newInstanceId = UUID.randomUUID().toString(); + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(CREATE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, newInstanceId); + int count = sql.executeUpdate(); + LOG.info( + "Created {} lock with instanceId {} with row count {}", + this, + newInstanceId, + count); + return count == 1; + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during tryLock"); + } catch (SQLException e) { + // SQL exception happened when creating the lock. Check if the lock creation was + // successful behind the scenes. + if (newInstanceId.equals(instanceId())) { + return true; + } else { + throw new UncheckedSQLException(e, "Failed to create %s lock", this); + } + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public boolean isHeld() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + return rs.next(); + } + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", this); + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public void unlock() { + try { + // Possible concurrency issue: + // - `unlock` and `tryLock` happens at the same time when there is an existing lock + // + // Steps: + // 1. `unlock` removes the lock in the database, but there is a temporary connection failure + // 2. `lock` founds that there is no lock, so creates a new lock + // 3. `unlock` retires the lock removal and removes the new lock + // + // To prevent the situation above we fetch the current lockId, and remove the lock + // only with the given id. + String instanceId = instanceId(); + + if (instanceId != null) { + pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(DELETE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, instanceId); + long count = sql.executeUpdate(); + LOG.info( + "Deleted {} lock with instanceId {} with row count {}", + this, + instanceId, + count); + } catch (SQLException e) { + // SQL exception happened when deleting lock information + throw new UncheckedSQLException( + e, "Failed to delete %s lock with instanceId %s", this, instanceId); + } + + return null; + }); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (UncheckedSQLException e) { + throw e; + } catch (SQLException e) { + // SQL exception happened when getting/updating lock information + throw new UncheckedSQLException(e, "Failed to remove lock %s", this); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("type", type).add("lockId", lockId).toString(); + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + private String instanceId() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + if (rs.next()) { + return rs.getString(1); + } else { + return null; + } + } + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + } + } + + private enum Type { + MAINTENANCE("m"), + RECOVERY("r"); + + private String key; + + Type(String key) { + this.key = key; + } + } +} 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 d74b2349b1de..89efffa15f16 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 -public class MonitorSource extends SingleThreadedIteratorSource { +class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ public 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 */ - public MonitorSource( + 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 452ed80ed0e5..7d0b94e97da7 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 @@ -35,7 +35,7 @@ class TableChange { private long deleteFileSize; private int commitNum; - TableChange( + private TableChange( int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { this.dataFileNum = dataFileNum; this.deleteFileNum = deleteFileNum; @@ -67,6 +67,10 @@ static TableChange empty() { return new TableChange(0, 0, 0L, 0L, 0); } + static Builder builder() { + return new Builder(); + } + int dataFileNum() { return dataFileNum; } @@ -130,4 +134,41 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); } + + static class Builder { + private int dataFileNum = 0; + private int deleteFileNum = 0; + private long dataFileSize = 0L; + private long deleteFileSize = 0L; + private int commitNum = 0; + + public Builder dataFileNum(int newDataFileNum) { + this.dataFileNum = newDataFileNum; + return this; + } + + public Builder deleteFileNum(int newDeleteFileNum) { + this.deleteFileNum = newDeleteFileNum; + return this; + } + + public Builder dataFileSize(long newDataFileSize) { + this.dataFileSize = newDataFileSize; + return this; + } + + public Builder deleteFileSize(long newDeleteFileSize) { + this.deleteFileSize = newDeleteFileSize; + return this; + } + + public Builder commitNum(int newCommitNum) { + this.commitNum = newCommitNum; + return this; + } + + public TableChange build() { + return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + } + } } 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 new file mode 100644 index 000000000000..ec0fd920c34b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.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.maintenance.operator; + +public class TableMaintenanceMetrics { + public static final String GROUP_KEY = "maintenanceTask"; + public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + + // TriggerManager metrics + public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; + public static final String CONCURRENT_RUN_THROTTLED = "concurrentRunThrottled"; + public static final String TRIGGERED = "triggered"; + public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + + private TableMaintenanceMetrics() { + // do not instantiate + } +} 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/operator/Trigger.java new file mode 100644 index 000000000000..85c6c8dbdd55 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.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.flink.maintenance.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +@Internal +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) { + 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); + } + + static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, null, true); + } + + long timestamp() { + return timestamp; + } + + SerializableTable table() { + return table; + } + + Integer taskId() { + return taskId; + } + + boolean isRecovery() { + return isRecovery; + } + + @Override + 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/TriggerEvaluator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java new file mode 100644 index 000000000000..37e4e3afd4bd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -0,0 +1,128 @@ +/* + * 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.Serializable; +import java.time.Duration; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TriggerEvaluator implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); + private final List predicates; + + private TriggerEvaluator(List predicates) { + Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 condition."); + + this.predicates = predicates; + } + + boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { + boolean result = + predicates.stream() + .anyMatch( + p -> { + try { + return p.evaluate(event, lastTimeMs, currentTimeMs); + } catch (Exception e) { + throw new RuntimeException("Error accessing state", e); + } + }); + LOG.debug( + "Checking event: {}, at {}, last: {} with result: {}", + event, + currentTimeMs, + lastTimeMs, + result); + return result; + } + + static class Builder implements Serializable { + private Integer commitNumber; + private Integer fileNumber; + private Long fileSize; + private Integer deleteFileNumber; + private Duration timeout; + + Builder commitNumber(int newCommitNumber) { + this.commitNumber = newCommitNumber; + return this; + } + + Builder fileNumber(int newFileNumber) { + this.fileNumber = newFileNumber; + return this; + } + + Builder fileSize(long newFileSize) { + this.fileSize = newFileSize; + return this; + } + + Builder deleteFileNumber(int newDeleteFileNumber) { + this.deleteFileNumber = newDeleteFileNumber; + return this; + } + + Builder timeout(Duration newTimeout) { + this.timeout = newTimeout; + return this; + } + + TriggerEvaluator build() { + List predicates = Lists.newArrayList(); + if (commitNumber != null) { + predicates.add((change, unused, unused2) -> change.commitNum() >= commitNumber); + } + + if (fileNumber != null) { + predicates.add( + (change, unused, unused2) -> + change.dataFileNum() + change.deleteFileNum() >= fileNumber); + } + + if (fileSize != null) { + predicates.add( + (change, unused, unused2) -> + change.dataFileSize() + change.deleteFileSize() >= fileSize); + } + + if (deleteFileNumber != null) { + predicates.add((change, unused, unused2) -> change.deleteFileNum() >= deleteFileNumber); + } + + if (timeout != null) { + predicates.add( + (change, lastTimeMs, currentTimeMs) -> + currentTimeMs - lastTimeMs >= timeout.toMillis()); + } + + return new TriggerEvaluator(predicates); + } + } + + private interface Predicate extends Serializable { + boolean evaluate(TableChange event, long lastTimeMs, long currentTimeMs); + } +} 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/operator/TriggerLockFactory.java new file mode 100644 index 000000000000..329223d27ccf --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.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.flink.maintenance.operator; + +import java.io.Closeable; +import java.io.Serializable; +import org.apache.flink.annotation.Experimental; + +/** Lock interface for handling locks for the Flink Table Maintenance jobs. */ +@Experimental +public interface TriggerLockFactory extends Serializable, Closeable { + void open(); + + Lock createLock(); + + Lock createRecoveryLock(); + + interface Lock { + /** + * Tries to acquire a lock with a given key. Anyone already holding a lock would prevent + * acquiring this lock. Not reentrant. + * + *

      Called by {@link TriggerManager}. Implementations could assume that are no concurrent + * calls for this method. + * + * @return true if the lock is acquired by this job, false if the lock + * is already held by someone + */ + boolean tryLock(); + + /** + * Checks if the lock is already taken. + * + * @return true if the lock is held by someone + */ + 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. + */ + void unlock(); + } +} 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 new file mode 100644 index 000000000000..f4c3c1d47cf7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -0,0 +1,339 @@ +/* + * 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.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; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.TimerService; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} messages which are + * calculated based on the incoming {@link TableChange} messages. The TriggerManager keeps track of + * the changes since the last run of the Maintenance Tasks and triggers a new run based on the + * result of the {@link TriggerEvaluator}. + * + *

      The TriggerManager prevents overlapping Maintenance Task runs using {@link + * TriggerLockFactory.Lock}. The current implementation only handles conflicts within a single job. + * Users should avoid scheduling maintenance for the same table in different Flink jobs. + * + *

      The TriggerManager should run as a global operator. {@link KeyedProcessFunction} is used, so + * the timer functions are available, but the key is not used. + */ +@Internal +class TriggerManager extends KeyedProcessFunction + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); + + private final TableLoader tableLoader; + private final TriggerLockFactory lockFactory; + private final List taskNames; + private final List evaluators; + private final long minFireDelayMs; + private final long lockCheckDelayMs; + private transient Counter rateLimiterTriggeredCounter; + private transient Counter concurrentRunThrottledCounter; + private transient Counter nothingToTriggerCounter; + private transient List triggerCounters; + private transient ValueState nextEvaluationTimeState; + private transient ListState accumulatedChangesState; + private transient ListState lastTriggerTimesState; + private transient Long nextEvaluationTime; + private transient List accumulatedChanges; + private transient List lastTriggerTimes; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient boolean shouldRestoreTasks = false; + private transient boolean inited = false; + // To keep the task scheduling fair we keep the last triggered task position in memory. + // If we find a task to trigger, then we run it, but after it is finished, we start from the given + // position to prevent "starvation" of the tasks. + // When there is nothing to trigger, we start from the beginning, as the order of the tasks might + // be important (RewriteDataFiles first, and then RewriteManifestFiles later) + private transient int startsFrom = 0; + private transient boolean triggered = false; + + TriggerManager( + TableLoader tableLoader, + TriggerLockFactory lockFactory, + List taskNames, + List evaluators, + long minFireDelayMs, + long lockCheckDelayMs) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + Preconditions.checkArgument( + evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); + Preconditions.checkArgument( + taskNames.size() == evaluators.size(), "Provide a name and evaluator for all of the tasks"); + Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); + Preconditions.checkArgument( + lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); + + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskNames = taskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.rateLimiterTriggeredCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); + this.concurrentRunThrottledCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); + this.nothingToTriggerCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); + this.triggerCounters = + taskNames.stream() + .map( + name -> + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.TRIGGERED)) + .collect(Collectors.toList()); + + this.nextEvaluationTimeState = + getRuntimeContext() + .getState(new ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG)); + this.accumulatedChangesState = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>( + "triggerManagerAccumulatedChange", TypeInformation.of(TableChange.class))); + this.lastTriggerTimesState = + getRuntimeContext() + .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); + + tableLoader.open(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (inited) { + // Only store state if initialized + nextEvaluationTimeState.update(nextEvaluationTime); + accumulatedChangesState.update(accumulatedChanges); + lastTriggerTimesState.update(lastTriggerTimes); + LOG.info( + "Storing state: nextEvaluationTime {}, accumulatedChanges {}, lastTriggerTimes {}", + nextEvaluationTime, + accumulatedChanges, + lastTriggerTimes); + } else { + LOG.info("Not initialized, state is not stored"); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOG.info("Initializing state restored: {}", context.isRestored()); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + if (context.isRestored()) { + shouldRestoreTasks = true; + } + } + + @Override + public void processElement(TableChange change, Context ctx, Collector out) + throws Exception { + init(out, ctx.timerService()); + + accumulatedChanges.forEach(tableChange -> tableChange.merge(change)); + + long current = ctx.timerService().currentProcessingTime(); + if (nextEvaluationTime == null) { + checkAndFire(current, ctx.timerService(), out); + } else { + LOG.info( + "Trigger manager rate limiter triggered current: {}, next: {}, accumulated changes: {}", + current, + nextEvaluationTime, + accumulatedChanges); + rateLimiterTriggeredCounter.inc(); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + init(out, ctx.timerService()); + this.nextEvaluationTime = null; + checkAndFire(ctx.timerService().currentProcessingTime(), ctx.timerService(), out); + } + + @Override + public void close() throws IOException { + tableLoader.close(); + lockFactory.close(); + } + + private void checkAndFire(long current, TimerService timerService, Collector out) { + if (shouldRestoreTasks) { + if (recoveryLock.isHeld()) { + // Recovered tasks in progress. Skip trigger check + LOG.debug("The recovery lock is still held at {}", current); + schedule(timerService, current + lockCheckDelayMs); + return; + } else { + LOG.info("The recovery is finished at {}", current); + shouldRestoreTasks = false; + } + } + + Integer taskToStart = + nextTrigger(evaluators, accumulatedChanges, lastTriggerTimes, current, startsFrom); + if (taskToStart == null) { + // Nothing to execute + if (!triggered) { + nothingToTriggerCounter.inc(); + LOG.debug("Nothing to execute at {} for collected: {}", current, accumulatedChanges); + } else { + LOG.debug("Execution check finished"); + } + + // Next time start from the beginning + startsFrom = 0; + triggered = false; + return; + } + + if (lock.tryLock()) { + TableChange change = accumulatedChanges.get(taskToStart); + SerializableTable table = + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()); + out.collect(Trigger.create(current, table, taskToStart)); + LOG.debug("Fired event with time: {}, collected: {} for {}", current, change, table.name()); + triggerCounters.get(taskToStart).inc(); + accumulatedChanges.set(taskToStart, TableChange.empty()); + lastTriggerTimes.set(taskToStart, current); + schedule(timerService, current + minFireDelayMs); + startsFrom = (taskToStart + 1) % evaluators.size(); + triggered = true; + } else { + // A task is already running, waiting for it to finish + LOG.info("Failed to acquire lock. Delaying task to {}", current + lockCheckDelayMs); + + startsFrom = taskToStart; + concurrentRunThrottledCounter.inc(); + schedule(timerService, current + lockCheckDelayMs); + } + + timerService.registerProcessingTimeTimer(nextEvaluationTime); + } + + private void schedule(TimerService timerService, long time) { + this.nextEvaluationTime = time; + timerService.registerProcessingTimeTimer(time); + } + + private static Integer nextTrigger( + List evaluators, + List changes, + List lastTriggerTimes, + long currentTime, + int startPos) { + int current = startPos; + do { + if (evaluators + .get(current) + .check(changes.get(current), lastTriggerTimes.get(current), currentTime)) { + return current; + } + + current = (current + 1) % evaluators.size(); + } while (current != startPos); + + return null; + } + + private void init(Collector out, TimerService timerService) throws Exception { + if (!inited) { + long current = timerService.currentProcessingTime(); + + // Initialize from state + this.nextEvaluationTime = nextEvaluationTimeState.value(); + this.accumulatedChanges = Lists.newArrayList(accumulatedChangesState.get()); + this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get()); + + // Initialize if the state was empty + if (accumulatedChanges.isEmpty()) { + for (int i = 0; i < evaluators.size(); ++i) { + accumulatedChanges.add(TableChange.empty()); + lastTriggerTimes.add(current); + } + } + + if (shouldRestoreTasks) { + // When the job state is restored, there could be ongoing tasks. + // To prevent collision with the new triggers the following is done: + // - add a recovery lock + // - fire a recovery trigger + // This ensures that the tasks of the previous trigger are executed, and the lock is removed + // in the end. The result of the 'tryLock' is ignored as an already existing lock prevents + // collisions as well. + recoveryLock.tryLock(); + out.collect(Trigger.recovery(current)); + if (nextEvaluationTime == null) { + schedule(timerService, current + minFireDelayMs); + } + } + + inited = true; + } + } +} 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 new file mode 100644 index 000000000000..36e162d4f068 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.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.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/MetricsReporterFactoryForTests.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java new file mode 100644 index 000000000000..7a523035b7fb --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -0,0 +1,153 @@ +/* + * 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.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class MetricsReporterFactoryForTests implements MetricReporterFactory { + private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); + private static final Pattern FULL_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static Map counters = Maps.newConcurrentMap(); + private static Map gauges = Maps.newConcurrentMap(); + private static Set monitoredMetricNames; + + public MetricsReporterFactoryForTests() { + monitoredMetricNames = + Arrays.stream(TableMaintenanceMetrics.class.getDeclaredFields()) + .map( + f -> { + try { + return f.get(null).toString(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + } + + @Override + public MetricReporter createMetricReporter(Properties properties) { + return INSTANCE; + } + + public static void reset() { + counters = Maps.newConcurrentMap(); + gauges = Maps.newConcurrentMap(); + } + + public static Long counter(String name) { + return counterValues().get(name); + } + + public static Long gauge(String name) { + return gaugeValues().get(name); + } + + public static void assertGauges(Map expected) { + assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + } + + public static void assertCounters(Map expected) { + assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + } + + private static Map gaugeValues() { + return gauges.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> (Long) entry.getValue().getValue())); + } + + private static Map counterValues() { + return counters.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> entry.getValue().getCount())); + } + + private static Map filter(Map original, Map filter) { + return original.entrySet().stream() + .filter( + entry -> { + Long filterValue = filter.get(entry.getKey()); + return filterValue == null || filterValue != -1; + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static String longName(String fullName) { + Matcher matcher = FULL_METRIC_NAME.matcher(fullName); + if (!matcher.matches()) { + throw new RuntimeException(String.format("Can't parse simplified metrics name %s", fullName)); + } + + return matcher.group(1) + "." + matcher.group(2) + "." + matcher.group(3); + } + + private static class TestMetricsReporter implements MetricReporter { + @Override + public void open(MetricConfig config) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + if (monitoredMetricNames.contains(metricName)) { + if (metric instanceof Counter) { + counters.put(group.getMetricIdentifier(metricName), (Counter) metric); + } + + if (metric instanceof Gauge) { + gauges.put(group.getMetricIdentifier(metricName), (Gauge) metric); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + // do nothing + } + } +} 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 272e0b693fd3..225853086545 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 @@ -20,16 +20,24 @@ import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import java.io.File; 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.test.junit5.MiniClusterExtension; import org.apache.iceberg.flink.FlinkCatalogFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; 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(); static final String TABLE_NAME = "test_table"; @@ -39,7 +47,7 @@ class OperatorTestBase { new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(config()) .build()); @RegisterExtension @@ -48,4 +56,101 @@ class OperatorTestBase { "catalog", ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), "db"); + + 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; + } + + protected static TriggerLockFactory lockFactory() { + return new TriggerLockFactory() { + @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 + } + }; + } + + /** + * 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 + */ + public 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 + */ + public static void closeJobClient(JobClient jobClient) { + closeJobClient(jobClient, null); + } + + private static class MemoryLock implements TriggerLockFactory.Lock { + boolean locked = false; + + @Override + public boolean tryLock() { + if (locked) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } } 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/operator/TestJdbcLockFactory.java new file mode 100644 index 000000000000..051d09d92bad --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.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.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +class TestJdbcLockFactory extends TestLockFactoryBase { + @Override + TriggerLockFactory lockFactory() { + return lockFactory("tableName"); + } + + @Test + void testMultiTableLock() { + JdbcLockFactory other = lockFactory("tableName2"); + other.open((JdbcLockFactory) this.lockFactory); + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = other.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + private JdbcLockFactory lockFactory(String tableName) { + Map properties = Maps.newHashMap(); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(INIT_LOCK_TABLES_PROPERTY, "true"); + + return new JdbcLockFactory( + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), + tableName, + properties); + } +} 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/operator/TestLockFactoryBase.java new file mode 100644 index 000000000000..bf9e86f2534d --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.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.io.IOException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +abstract class TestLockFactoryBase { + protected TriggerLockFactory lockFactory; + + abstract TriggerLockFactory lockFactory(); + + @BeforeEach + void before() { + this.lockFactory = lockFactory(); + lockFactory.open(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testTryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock1.tryLock()).isFalse(); + assertThat(lock2.tryLock()).isFalse(); + } + + @Test + void testUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + } + + @Test + void testNoConflictWithRecoveryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createRecoveryLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + @Test + void testDoubleUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + assertThat(lock.tryLock()).isFalse(); + } +} 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 876d64214560..8c02601025ae 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 @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -35,6 +34,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -161,7 +161,8 @@ void testSource() throws Exception { } // The first non-empty event should contain the expected value - return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + return newEvent.equals( + TableChange.builder().dataFileNum(1).dataFileSize(size).commitNum(1).build()); }); } finally { closeJobClient(jobClient); @@ -348,15 +349,18 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange List deleteFiles = Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); - long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); - - return new TableChange( - previous.dataFileNum() + dataFiles.size(), - previous.deleteFileNum() + deleteFiles.size(), - previous.dataFileSize() + dataSize, - previous.deleteFileSize() + deleteSize, - previous.commitNum() + 1); + long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteSize = deleteFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + + TableChange newChange = previous.copy(); + newChange.merge( + TableChange.builder() + .dataFileNum(dataFiles.size()) + .dataFileSize(dataSize) + .deleteFileNum(deleteFiles.size()) + .deleteFileSize(deleteSize) + .commitNum(1) + .build()); + return newChange; } } 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 new file mode 100644 index 000000000000..55e64f3e84bf --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -0,0 +1,622 @@ +/* + * 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.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; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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.flink.TableLoader; +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; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +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 long processingTime = 0L; + private TriggerLockFactory lockFactory; + private TriggerLockFactory.Lock lock; + private TriggerLockFactory.Lock recoveringLock; + + @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(); + } + + @Test + void testCommitNumber() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitNumber(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 4); + } + } + + @Test + void testFileNumber() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileNumber(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 0); + + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileNum(1).deleteFileNum(1).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileNum(5).deleteFileNum(7).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 4); + } + } + + @Test + void testFileSize() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileSize(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 0); + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileSize(1L).deleteFileSize(1L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(3L).build(), 2); + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileSize(5L).deleteFileSize(7L).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(1L).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 4); + } + } + + @Test + void testDeleteFileNumber() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().deleteFileNumber(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileNum(3).deleteFileNum(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 4); + } + } + + @Test + void testTimeout() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + TableChange event = TableChange.builder().dataFileSize(1).commitNum(1).build(); + + // Wait for some time + testHarness.processElement(event, EVENT_TIME); + assertThat(testHarness.extractOutputValues()).isEmpty(); + + // Wait for the timeout to expire + long newTime = EVENT_TIME + Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // Send a new event + testHarness.setProcessingTime(newTime + 1); + testHarness.processElement(event, newTime); + + // No trigger yet + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Send a new event + newTime += Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + + // New trigger should arrive + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testStateRestore() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + testHarness.processElement( + TableChange.builder().dataFileSize(1).commitNum(1).build(), EVENT_TIME); + + assertThat(testHarness.extractOutputValues()).isEmpty(); + + state = testHarness.snapshot(1, EVENT_TIME); + } + + // Restore the state, write some more data, create a checkpoint, check the data which is written + manager = manager(tableLoader); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + // Arrives the first real change which triggers the recovery process + testHarness.processElement(TableChange.builder().commitNum(1).build(), EVENT_TIME_2); + assertTriggers( + testHarness.extractOutputValues(), + Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); + + // Remove the lock to allow the next trigger + recoveringLock.unlock(); + testHarness.setProcessingTime(EVENT_TIME_2); + // At this point the output contains the recovery trigger and the real trigger + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testMinFireDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, DELAY, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // No new fire yet + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testLockCheckDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, 1, DELAY); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + + // Create a lock to prevent execution, and check that there is no result + assertThat(lock.tryLock()).isTrue(); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // Remove the lock, and still no trigger + lock.unlock(); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + /** + * Simulating recovery scenarios where there is a leftover table lock, and ongoing maintenance + * task. + * + * @param locked if a lock exists on the table on job recovery + * @param runningTask is running and continues to run after job recovery + */ + @ParameterizedTest + @MethodSource("parametersForTestRecovery") + void testRecovery(boolean locked, boolean runningTask) throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + state = testHarness.snapshot(1, EVENT_TIME); + } + + if (locked) { + assertThat(lock.tryLock()).isTrue(); + } + + manager = manager(tableLoader); + List expected = Lists.newArrayListWithExpectedSize(3); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + ++processingTime; + expected.add(Trigger.recovery(processingTime)); + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Nothing happens until the recovery is finished + ++processingTime; + testHarness.setProcessingTime(processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + if (runningTask) { + // Simulate the action of the recovered maintenance task lock removal when it finishes + lock.unlock(); + } + + // Still no results as the recovery is ongoing + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it + // received recovery trigger + lock.unlock(); + recoveringLock.unlock(); + + // Emit only a single trigger + ++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)); + assertTriggers(testHarness.extractOutputValues(), expected); + } + } + + @Test + void testTriggerMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + TriggerManager manager = + new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1, NAME_2), + Lists.newArrayList( + new TriggerEvaluator.Builder().commitNumber(2).build(), + new TriggerEvaluator.Builder().commitNumber(4).build()), + 1L, + 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // This one doesn't trigger - tests NOTHING_TO_TRIGGER + source.sendRecord(TableChange.builder().commitNum(1).build()); + + Awaitility.await() + .until( + () -> { + Long notingCounter = + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + return notingCounter != null && notingCounter.equals(1L); + }); + + // Trigger one of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitNum(1).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(1L); + lock.unlock(); + + // Trigger both of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitNum(2).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(2L); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + 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) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testRateLimiterMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1_000_000L, 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger + source.sendRecord(TableChange.builder().commitNum(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // The second trigger will be blocked + source.sendRecord(TableChange.builder().commitNum(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + .equals(1L)); + + // Final check all the counters + assertCounters(1L, 0L); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testConcurrentRunMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1L, 1_000_000L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger - notice that we do not remove the lock after the trigger + source.sendRecord(TableChange.builder().commitNum(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // The second trigger will be blocked by the lock + source.sendRecord(TableChange.builder().commitNum(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + .equals(1L)); + + // Final check all the counters + assertCounters(0L, 1L); + } finally { + closeJobClient(jobClient); + } + } + + private static Stream parametersForTestRecovery() { + return Stream.of( + Arguments.of(true, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false)); + } + + private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + rateLimiterTrigger) + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + concurrentRunTrigger) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .build()); + } + + private KeyedOneInputStreamOperatorTestHarness harness( + TriggerManager manager) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new KeyedProcessOperator<>(manager), value -> true, Types.BOOLEAN); + } + + private void addEventAndCheckResult( + OneInputStreamOperatorTestHarness testHarness, + TableChange event, + int expectedSize) + throws Exception { + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(event, processingTime); + assertThat(testHarness.extractOutputValues()).hasSize(expectedSize); + // Remove the lock to allow the next trigger + lock.unlock(); + } + + private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { + return new TriggerManager( + tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + } + + private TriggerManager manager( + TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { + return new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1), + Lists.newArrayList(new TriggerEvaluator.Builder().commitNumber(2).build()), + minFireDelayMs, + lockCheckDelayMs); + } + + private TriggerManager manager(TableLoader tableLoader) { + return manager(tableLoader, new TriggerEvaluator.Builder().commitNumber(2).build()); + } + + private static void assertTriggers(List expected, List actual) { + assertThat(actual).hasSize(expected.size()); + for (int i = 0; i < expected.size(); ++i) { + Trigger expectedTrigger = expected.get(i); + Trigger actualTrigger = actual.get(i); + 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()); + } + } + } + } +} diff --git a/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 000000000000..952255a52b7c --- /dev/null +++ b/flink/v1.19/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -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.flink.maintenance.operator.MetricsReporterFactoryForTests From a0eb0130d3e9ab3e2fb0a7229254ee2ac83dc7f3 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 22 Aug 2024 07:39:07 -0700 Subject: [PATCH 0622/1019] Flink: backport PR #10956 for converter interface that deprecates ReaderFunction (#10985) --- .../iceberg/flink/source/IcebergSource.java | 94 +++++++++++--- .../reader/AvroGenericRecordConverter.java | 69 +++++++++++ .../AvroGenericRecordReaderFunction.java | 10 +- .../reader/ConverterReaderFunction.java | 117 ++++++++++++++++++ .../reader/IcebergSourceSplitReader.java | 2 +- .../flink/source/reader/RowDataConverter.java | 32 +++++ ...TestIcebergSourceBoundedGenericRecord.java | 70 +++++++---- .../iceberg/flink/source/IcebergSource.java | 94 +++++++++++--- .../reader/AvroGenericRecordConverter.java | 69 +++++++++++ .../AvroGenericRecordReaderFunction.java | 10 +- .../reader/ConverterReaderFunction.java | 117 ++++++++++++++++++ .../reader/IcebergSourceSplitReader.java | 2 +- .../flink/source/reader/RowDataConverter.java | 32 +++++ ...TestIcebergSourceBoundedGenericRecord.java | 70 +++++++---- 14 files changed, 702 insertions(+), 86 deletions(-) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java 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 e629cc19bbf2..351ba54e5c7c 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 @@ -61,10 +61,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -211,20 +213,40 @@ private SplitEnumerator createEnumer } } + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static Builder builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param output type + * @return an IcebergSource builder + */ + public static Builder forOutputType(RowDataConverter converter) { + return new Builder().converter(converter); + } + public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private ReaderFunction readerFunction; + private RowDataConverter converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +277,28 @@ public Builder splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder readerFunction(ReaderFunction newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder converter(RowDataConverter newConverter) { + this.converter = newConverter; + return this; + } + public Builder flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -510,25 +549,7 @@ public IcebergSource build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -549,5 +570,40 @@ public IcebergSource build() { table, emitter); } + + private ReaderFunction readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.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.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * 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 java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction extends DataIteratorReaderFunction { + private final RowDataConverter converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param output type + */ +public interface RowDataConverter + extends Function, ResultTypeQueryable, Serializable {} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 7bfed00a9eb4..4e649d15b1ce 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -52,6 +52,7 @@ 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; @@ -69,12 +70,13 @@ public class TestIcebergSourceBoundedGenericRecord { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } @@ -84,6 +86,9 @@ public static Object[][] parameters() { @Parameter(index = 1) private int parallelism; + @Parameter(index = 2) + private boolean useConverter; + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = @@ -147,24 +152,15 @@ private List run( table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -172,7 +168,6 @@ private List run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -193,4 +188,35 @@ private List run( return Lists.newArrayList(iter); } } + + 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); + } } 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 e629cc19bbf2..351ba54e5c7c 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 @@ -61,10 +61,12 @@ import org.apache.iceberg.flink.source.enumerator.IcebergEnumeratorStateSerializer; import org.apache.iceberg.flink.source.enumerator.StaticIcebergEnumerator; import org.apache.iceberg.flink.source.reader.ColumnStatsWatermarkExtractor; +import org.apache.iceberg.flink.source.reader.ConverterReaderFunction; import org.apache.iceberg.flink.source.reader.IcebergSourceReader; import org.apache.iceberg.flink.source.reader.IcebergSourceReaderMetrics; import org.apache.iceberg.flink.source.reader.MetaDataReaderFunction; import org.apache.iceberg.flink.source.reader.ReaderFunction; +import org.apache.iceberg.flink.source.reader.RowDataConverter; import org.apache.iceberg.flink.source.reader.RowDataReaderFunction; import org.apache.iceberg.flink.source.reader.SerializableRecordEmitter; import org.apache.iceberg.flink.source.reader.SplitWatermarkExtractor; @@ -211,20 +213,40 @@ private SplitEnumerator createEnumer } } + /** + * Create a source builder. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link IcebergSource#forRowData()} or + * {@link IcebergSource#forOutputType(RowDataConverter)} instead + */ + @Deprecated public static Builder builder() { return new Builder<>(); } + /** Create a source builder for RowData output type. */ public static Builder forRowData() { return new Builder<>(); } + /** + * Create a source builder that would convert {@link RowData} to the output type {@code T}. + * + * @param converter convert {@link RowData} to output type {@code T} + * @param output type + * @return an IcebergSource builder + */ + public static Builder forOutputType(RowDataConverter converter) { + return new Builder().converter(converter); + } + public static class Builder { private TableLoader tableLoader; private Table table; private SplitAssignerFactory splitAssignerFactory; private SerializableComparator splitComparator; private ReaderFunction readerFunction; + private RowDataConverter converter; private ReadableConfig flinkConfig = new Configuration(); private final ScanContext.Builder contextBuilder = ScanContext.builder(); private TableSchema projectedFlinkSchema; @@ -255,11 +277,28 @@ public Builder splitComparator( return this; } + /** + * @deprecated since 1.7.0. Will be removed in 2.0.0; use{@link + * IcebergSource#forOutputType(RowDataConverter)} instead to produce output type other than + * {@link RowData}. + */ + @Deprecated public Builder readerFunction(ReaderFunction newReaderFunction) { + Preconditions.checkState( + converter == null, + "Cannot set reader function when builder was created via IcebergSource.forOutputType(Converter)"); this.readerFunction = newReaderFunction; return this; } + /** + * Don't need to be public. It is set by {@link IcebergSource#forOutputType(RowDataConverter)}. + */ + private Builder converter(RowDataConverter newConverter) { + this.converter = newConverter; + return this; + } + public Builder flinkConfig(ReadableConfig config) { this.flinkConfig = config; return this; @@ -510,25 +549,7 @@ public IcebergSource build() { ScanContext context = contextBuilder.build(); context.validate(); if (readerFunction == null) { - if (table instanceof BaseMetadataTable) { - MetaDataReaderFunction rowDataReaderFunction = - new MetaDataReaderFunction( - flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } else { - RowDataReaderFunction rowDataReaderFunction = - new RowDataReaderFunction( - flinkConfig, - table.schema(), - context.project(), - context.nameMapping(), - context.caseSensitive(), - table.io(), - table.encryption(), - context.filters(), - context.limit()); - this.readerFunction = (ReaderFunction) rowDataReaderFunction; - } + this.readerFunction = readerFunction(context); } if (splitAssignerFactory == null) { @@ -549,5 +570,40 @@ public IcebergSource build() { table, emitter); } + + private ReaderFunction readerFunction(ScanContext context) { + if (table instanceof BaseMetadataTable) { + MetaDataReaderFunction rowDataReaderFunction = + new MetaDataReaderFunction( + flinkConfig, table.schema(), context.project(), table.io(), table.encryption()); + return (ReaderFunction) rowDataReaderFunction; + } else { + if (converter == null) { + return (ReaderFunction) + new RowDataReaderFunction( + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } else { + return new ConverterReaderFunction<>( + converter, + flinkConfig, + table.schema(), + context.project(), + context.nameMapping(), + context.caseSensitive(), + table.io(), + table.encryption(), + context.filters(), + context.limit()); + } + } + } } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.java new file mode 100644 index 000000000000..b158b0871a53 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordConverter.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.flink.source.reader; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.formats.avro.RowDataToAvroConverters; +import org.apache.flink.formats.avro.typeutils.AvroSchemaConverter; +import org.apache.flink.formats.avro.typeutils.GenericRecordAvroTypeInfo; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.LogicalType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.table.types.utils.TypeConversions; +import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.flink.FlinkSchemaUtil; + +public class AvroGenericRecordConverter implements RowDataConverter { + private final Schema avroSchema; + private final RowDataToAvroConverters.RowDataToAvroConverter flinkConverter; + private final TypeInformation outputTypeInfo; + + private AvroGenericRecordConverter(Schema avroSchema, RowType rowType) { + this.avroSchema = avroSchema; + this.flinkConverter = RowDataToAvroConverters.createConverter(rowType); + this.outputTypeInfo = new GenericRecordAvroTypeInfo(avroSchema); + } + + public static AvroGenericRecordConverter fromIcebergSchema( + org.apache.iceberg.Schema icebergSchema, String tableName) { + RowType rowType = FlinkSchemaUtil.convert(icebergSchema); + Schema avroSchema = AvroSchemaUtil.convert(icebergSchema, tableName); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + public static AvroGenericRecordConverter fromAvroSchema(Schema avroSchema, String tableName) { + DataType dataType = AvroSchemaConverter.convertToDataType(avroSchema.toString()); + LogicalType logicalType = TypeConversions.fromDataToLogicalType(dataType); + RowType rowType = RowType.of(logicalType.getChildren().toArray(new LogicalType[0])); + return new AvroGenericRecordConverter(avroSchema, rowType); + } + + @Override + public GenericRecord apply(RowData rowData) { + return (GenericRecord) flinkConverter.convert(avroSchema, rowData); + } + + @Override + public TypeInformation getProducedType() { + return outputTypeInfo; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java index 66e59633fff2..f89e5ce13474 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/AvroGenericRecordReaderFunction.java @@ -28,13 +28,21 @@ import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.flink.source.AvroGenericRecordFileScanTaskReader; import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.IcebergSource; import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; import org.apache.iceberg.flink.source.RowDataToAvroGenericRecordConverter; import org.apache.iceberg.flink.source.split.IcebergSourceSplit; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -/** Read Iceberg rows as {@link GenericRecord}. */ +/** + * Read Iceberg rows as {@link GenericRecord}. + * + * @deprecated since 1.7.0. Will be removed in 2.0.0; use {@link + * IcebergSource#forOutputType(RowDataConverter)} and {@link AvroGenericRecordConverter} + * instead. + */ +@Deprecated public class AvroGenericRecordReaderFunction extends DataIteratorReaderFunction { private final String tableName; private final Schema readSchema; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java new file mode 100644 index 000000000000..e1e7c17d63c5 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/ConverterReaderFunction.java @@ -0,0 +1,117 @@ +/* + * 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 java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.InputFilesDecryptor; +import org.apache.iceberg.expressions.Expression; +import org.apache.iceberg.flink.source.DataIterator; +import org.apache.iceberg.flink.source.FileScanTaskReader; +import org.apache.iceberg.flink.source.RowDataFileScanTaskReader; +import org.apache.iceberg.flink.source.split.IcebergSourceSplit; +import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +@Internal +public class ConverterReaderFunction extends DataIteratorReaderFunction { + private final RowDataConverter converter; + private final Schema tableSchema; + private final Schema readSchema; + private final String nameMapping; + private final boolean caseSensitive; + private final FileIO io; + private final EncryptionManager encryption; + private final List filters; + private final long limit; + + private transient RecordLimiter recordLimiter = null; + + public ConverterReaderFunction( + RowDataConverter converter, + ReadableConfig config, + Schema tableSchema, + Schema projectedSchema, + String nameMapping, + boolean caseSensitive, + FileIO io, + EncryptionManager encryption, + List filters, + long limit) { + super(new ListDataIteratorBatcher<>(config)); + this.converter = converter; + this.tableSchema = tableSchema; + this.readSchema = readSchema(tableSchema, projectedSchema); + this.nameMapping = nameMapping; + this.caseSensitive = caseSensitive; + this.io = io; + this.encryption = encryption; + this.filters = filters; + this.limit = limit; + } + + @Override + protected DataIterator createDataIterator(IcebergSourceSplit split) { + RowDataFileScanTaskReader rowDataReader = + new RowDataFileScanTaskReader(tableSchema, readSchema, nameMapping, caseSensitive, filters); + return new LimitableDataIterator<>( + new ConverterFileScanTaskReader<>(rowDataReader, converter), + split.task(), + io, + encryption, + lazyLimiter()); + } + + private static Schema readSchema(Schema tableSchema, Schema projectedSchema) { + Preconditions.checkNotNull(tableSchema, "Table schema can't be null"); + return projectedSchema == null ? tableSchema : projectedSchema; + } + + /** Lazily create RecordLimiter to avoid the need to make it serializable */ + private RecordLimiter lazyLimiter() { + if (recordLimiter == null) { + this.recordLimiter = RecordLimiter.create(limit); + } + + return recordLimiter; + } + + private static class ConverterFileScanTaskReader implements FileScanTaskReader { + private final RowDataFileScanTaskReader rowDataReader; + private final RowDataConverter converter; + + ConverterFileScanTaskReader( + RowDataFileScanTaskReader rowDataReader, RowDataConverter converter) { + this.rowDataReader = rowDataReader; + this.converter = converter; + } + + @Override + public CloseableIterator open( + FileScanTask fileScanTask, InputFilesDecryptor inputFilesDecryptor) { + return CloseableIterator.transform( + rowDataReader.open(fileScanTask, inputFilesDecryptor), converter); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java index 9c20494fdbcd..bcd72e25036b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/IcebergSourceSplitReader.java @@ -85,7 +85,7 @@ public RecordsWithSplitIds> fetch() throws IOException { } else { // return an empty result, which will lead to split fetch to be idle. // SplitFetcherManager will then close idle fetcher. - return new RecordsBySplits(Collections.emptyMap(), Collections.emptySet()); + return new RecordsBySplits<>(Collections.emptyMap(), Collections.emptySet()); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java new file mode 100644 index 000000000000..98bb7e981840 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RowDataConverter.java @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.source.reader; + +import java.io.Serializable; +import java.util.function.Function; +import org.apache.flink.api.java.typeutils.ResultTypeQueryable; +import org.apache.flink.table.data.RowData; + +/** + * Convert RowData to a different output type. + * + * @param output type + */ +public interface RowDataConverter + extends Function, ResultTypeQueryable, Serializable {} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java index 7bfed00a9eb4..4e649d15b1ce 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedGenericRecord.java @@ -52,6 +52,7 @@ 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; @@ -69,12 +70,13 @@ public class TestIcebergSourceBoundedGenericRecord { private static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); - @Parameters(name = "format={0}, parallelism = {1}") + @Parameters(name = "format={0}, parallelism = {1}, useConverter = {2}") public static Object[][] parameters() { return new Object[][] { - {FileFormat.AVRO, 2}, - {FileFormat.PARQUET, 2}, - {FileFormat.ORC, 2} + {FileFormat.AVRO, 2, true}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false}, + {FileFormat.ORC, 2, true} }; } @@ -84,6 +86,9 @@ public static Object[][] parameters() { @Parameter(index = 1) private int parallelism; + @Parameter(index = 2) + private boolean useConverter; + @TestTemplate public void testUnpartitionedTable() throws Exception { Table table = @@ -147,24 +152,15 @@ private List run( table = tableLoader.loadTable(); } - AvroGenericRecordReaderFunction readerFunction = - new AvroGenericRecordReaderFunction( - TestFixtures.TABLE_IDENTIFIER.name(), - new Configuration(), - table.schema(), - null, - null, - false, - table.io(), - table.encryption(), - filters); + Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); + IcebergSource.Builder sourceBuilder; + if (useConverter) { + sourceBuilder = createSourceBuilderWithConverter(table, readSchema, config); + } else { + sourceBuilder = + createSourceBuilderWithReaderFunction(table, projectedSchema, filters, config); + } - IcebergSource.Builder sourceBuilder = - IcebergSource.builder() - .tableLoader(CATALOG_EXTENSION.tableLoader()) - .readerFunction(readerFunction) - .assignerFactory(new SimpleSplitAssignerFactory()) - .flinkConfig(config); if (projectedSchema != null) { sourceBuilder.project(projectedSchema); } @@ -172,7 +168,6 @@ private List run( sourceBuilder.filters(filters); sourceBuilder.setAll(options); - Schema readSchema = projectedSchema != null ? projectedSchema : table.schema(); RowType rowType = FlinkSchemaUtil.convert(readSchema); org.apache.avro.Schema avroSchema = AvroSchemaUtil.convert(readSchema, TestFixtures.TABLE_IDENTIFIER.name()); @@ -193,4 +188,35 @@ private List run( return Lists.newArrayList(iter); } } + + 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); + } } From c2c31978e4f5205bd18df2a33fc6af0d1b46169e Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 22 Aug 2024 13:11:16 -0700 Subject: [PATCH 0623/1019] Flink: backport PR #10777 from 1.19 to 1.18 for sink test refactoring. (#10965) --- .../iceberg/flink/FlinkCatalogFactory.java | 1 + .../apache/iceberg/flink/CatalogTestBase.java | 22 -- .../org/apache/iceberg/flink/SqlBase.java | 110 ++++++ .../org/apache/iceberg/flink/TestBase.java | 23 +- .../iceberg/flink/TestFlinkTableSink.java | 116 +----- .../flink/TestFlinkTableSinkExtended.java | 336 ++++++++++++++++++ .../iceberg/flink/TestIcebergConnector.java | 24 +- .../flink/sink/TestFlinkIcebergSink.java | 270 +------------- .../flink/sink/TestFlinkIcebergSinkBase.java | 51 ++- .../TestFlinkIcebergSinkDistributionMode.java | 180 ++++++++++ .../sink/TestFlinkIcebergSinkExtended.java | 208 +++++++++++ ...estIcebergSpeculativeExecutionSupport.java | 2 +- 12 files changed, 917 insertions(+), 426 deletions(-) create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java index 1453753849ec..9b0c7a938920 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkCatalogFactory.java @@ -70,6 +70,7 @@ public class FlinkCatalogFactory implements CatalogFactory { public static final String HADOOP_CONF_DIR = "hadoop-conf-dir"; public static final String DEFAULT_DATABASE = "default-database"; public static final String DEFAULT_DATABASE_NAME = "default"; + public static final String DEFAULT_CATALOG_NAME = "default_catalog"; public static final String BASE_NAMESPACE = "base-namespace"; public static final String TYPE = "type"; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java index 91ed3c4adea3..062ff68d5d85 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/CatalogTestBase.java @@ -118,26 +118,4 @@ protected String getFullQualifiedTableName(String tableName) { static String getURI(HiveConf conf) { return conf.get(HiveConf.ConfVars.METASTOREURIS.varname); } - - static String toWithClause(Map props) { - StringBuilder builder = new StringBuilder(); - builder.append("("); - int propCount = 0; - for (Map.Entry entry : props.entrySet()) { - if (propCount > 0) { - builder.append(","); - } - builder - .append("'") - .append(entry.getKey()) - .append("'") - .append("=") - .append("'") - .append(entry.getValue()) - .append("'"); - propCount++; - } - builder.append(")"); - return builder.toString(); - } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.java new file mode 100644 index 000000000000..9411ea4f7d71 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/SqlBase.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.flink; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import java.util.Map; +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.iceberg.relocated.com.google.common.collect.Lists; + +public abstract class SqlBase { + protected abstract TableEnvironment getTableEnv(); + + protected static TableResult exec(TableEnvironment env, String query, Object... args) { + return env.executeSql(String.format(query, args)); + } + + protected TableResult exec(String query, Object... args) { + return exec(getTableEnv(), query, args); + } + + protected List sql(String query, Object... args) { + TableResult tableResult = exec(query, args); + try (CloseableIterator iter = tableResult.collect()) { + return Lists.newArrayList(iter); + } catch (Exception e) { + throw new RuntimeException("Failed to collect table result", e); + } + } + + protected void assertSameElements(Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().containsExactlyInAnyOrderElementsOf(expected); + } + + protected void assertSameElements(String message, Iterable expected, Iterable actual) { + assertThat(actual).isNotNull().as(message).containsExactlyInAnyOrderElementsOf(expected); + } + + /** + * We can not drop currently used catalog after FLINK-29677, so we have make sure that we do not + * use the current catalog before dropping it. This method switches to the 'default_catalog' and + * drops the one requested. + * + * @param catalogName The catalog to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog + */ + protected void dropCatalog(String catalogName, boolean ifExists) { + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); + } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } + + protected static String toWithClause(Map props) { + StringBuilder builder = new StringBuilder(); + builder.append("("); + int propCount = 0; + for (Map.Entry entry : props.entrySet()) { + if (propCount > 0) { + builder.append(","); + } + builder + .append("'") + .append(entry.getKey()) + .append("'") + .append("=") + .append("'") + .append(entry.getValue()) + .append("'"); + propCount++; + } + builder.append(")"); + return builder.toString(); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java index a74226092f38..401960c3591b 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestBase.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink; +import static org.apache.iceberg.flink.FlinkCatalogFactory.DEFAULT_CATALOG_NAME; import static org.assertj.core.api.Assertions.assertThat; import java.nio.file.Path; @@ -26,7 +27,6 @@ import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.TableResult; import org.apache.flink.test.junit5.MiniClusterExtension; -import org.apache.flink.test.util.TestBaseUtils; import org.apache.flink.types.Row; import org.apache.flink.util.CloseableIterator; import org.apache.hadoop.hive.conf.HiveConf; @@ -40,7 +40,7 @@ import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.api.io.TempDir; -public abstract class TestBase extends TestBaseUtils { +public abstract class TestBase extends SqlBase { @RegisterExtension public static MiniClusterExtension miniClusterExtension = @@ -71,6 +71,7 @@ public static void stopMetastore() throws Exception { TestBase.catalog = null; } + @Override protected TableEnvironment getTableEnv() { if (tEnv == null) { synchronized (this) { @@ -122,7 +123,23 @@ protected void assertSameElements(String message, Iterable expected, Iterab * @param ifExists If we should use the 'IF EXISTS' when dropping the catalog */ protected void dropCatalog(String catalogName, boolean ifExists) { - sql("USE CATALOG default_catalog"); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); sql("DROP CATALOG %s %s", ifExists ? "IF EXISTS" : "", catalogName); } + + /** + * We can not drop currently used database after FLINK-33226, so we have make sure that we do not + * use the current database before dropping it. This method switches to the default database in + * the default catalog, and then it and drops the one requested. + * + * @param database The database to drop + * @param ifExists If we should use the 'IF EXISTS' when dropping the database + */ + protected void dropDatabase(String database, boolean ifExists) { + String currentCatalog = getTableEnv().getCurrentCatalog(); + sql("USE CATALOG %s", DEFAULT_CATALOG_NAME); + sql("USE %s", getTableEnv().listDatabases()[0]); + sql("USE CATALOG %s", currentCatalog); + sql("DROP DATABASE %s %s", ifExists ? "IF EXISTS" : "", database); + } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java index b73300e3f170..2978a92945a2 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSink.java @@ -18,36 +18,21 @@ */ package org.apache.iceberg.flink; -import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assumptions.assumeThat; -import java.util.Collections; import java.util.List; -import java.util.Map; -import java.util.stream.Collectors; -import java.util.stream.IntStream; -import org.apache.flink.api.dag.Transformation; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.EnvironmentSettings; import org.apache.flink.table.api.Expressions; import org.apache.flink.table.api.TableEnvironment; import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; -import org.apache.flink.table.api.internal.TableEnvironmentImpl; -import org.apache.flink.table.operations.ModifyOperation; -import org.apache.flink.table.planner.delegation.PlannerBase; -import org.apache.flink.types.Row; -import org.apache.iceberg.DataFile; -import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.Parameter; import org.apache.iceberg.Parameters; import org.apache.iceberg.Table; -import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.flink.source.BoundedTableFactory; -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.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -122,7 +107,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); BoundedTableFactory.clearDataSets(); super.clean(); } @@ -169,39 +154,6 @@ public void testOverwriteTable() throws Exception { icebergTable, Lists.newArrayList(SimpleDataUtil.createRecord(2, "b"))); } - @TestTemplate - public void testWriteParallelism() throws Exception { - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - 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 /*+ OPTIONS('write-parallelism'='1') */ SELECT * FROM %s", - TABLE_NAME, 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)); - } - @TestTemplate public void testReplacePartitions() throws Exception { assumeThat(isStreamingJob) @@ -289,70 +241,4 @@ public void testInsertIntoPartition() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); } } - - @TestTemplate - public void testHashDistributeMode() throws Exception { - String tableName = "test_hash_distribution_mode"; - Map tableProps = - ImmutableMap.of( - "write.format.default", - format.name(), - TableProperties.WRITE_DISTRIBUTION_MODE, - DistributionMode.HASH.modeName()); - - // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. - List dataSet = - IntStream.range(1, 1000) - .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) - .flatMap(List::stream) - .collect(Collectors.toList()); - String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); - sql( - "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" - + " WITH ('connector'='BoundedSource', 'data-id'='%s')", - SOURCE_TABLE, dataId); - - assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) - .as("Should have the expected rows in source table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - sql( - "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", - tableName, toWithClause(tableProps)); - - try { - // Insert data set. - sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); - - assertThat(sql("SELECT * FROM %s", tableName)) - .as("Should have the expected rows in sink table.") - .containsExactlyInAnyOrderElementsOf(dataSet); - - // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, - // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per - // partition. - Table table = validationCatalog.loadTable(TableIdentifier.of(icebergNamespace, tableName)); - Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); - for (List dataFiles : snapshotToDataFiles.values()) { - if (dataFiles.isEmpty()) { - continue; - } - - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) - .hasSize(1); - assertThat( - SimpleDataUtil.matchingPartitions( - dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) - .hasSize(1); - } - } finally { - sql("DROP TABLE IF EXISTS %s.%s", flinkDatabase, tableName); - } - } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java new file mode 100644 index 000000000000..b63547d433a4 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestFlinkTableSinkExtended.java @@ -0,0 +1,336 @@ +/* + * 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; + +import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; + +import java.io.File; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import java.util.stream.IntStream; +import org.apache.flink.api.dag.Transformation; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.EnvironmentSettings; +import org.apache.flink.table.api.TableEnvironment; +import org.apache.flink.table.api.bridge.java.StreamTableEnvironment; +import org.apache.flink.table.api.internal.TableEnvironmentImpl; +import org.apache.flink.table.operations.ModifyOperation; +import org.apache.flink.table.planner.delegation.PlannerBase; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.source.BoundedTableFactory; +import org.apache.iceberg.hadoop.HadoopCatalog; +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.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.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkTableSink}, like catalog + * types, namespaces, file format, streaming/batch. Those combinations explode exponentially. Each + * test method in {@link TestFlinkTableSink} runs 21 combinations, which are expensive and slow. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkTableSinkExtended extends SqlBase { + protected static final String CATALOG = "testhadoop"; + protected static final String DATABASE = "db"; + protected static final String TABLE = "tbl"; + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + private static final String SOURCE_TABLE = "default_catalog.default_database.bounded_source"; + private static final String FLINK_DATABASE = CATALOG + "." + DATABASE; + private static final Namespace ICEBERG_NAMESPACE = Namespace.of(new String[] {DATABASE}); + + @TempDir protected File warehouseRoot; + + protected HadoopCatalog catalog = null; + + private TableEnvironment tEnv; + + @Parameter protected boolean isStreamingJob; + + @Parameters(name = "isStreamingJob={0}") + protected static List parameters() { + return Arrays.asList(new Boolean[] {true}, new Boolean[] {false}); + } + + protected synchronized TableEnvironment getTableEnv() { + if (tEnv == null) { + EnvironmentSettings.Builder settingsBuilder = EnvironmentSettings.newInstance(); + if (isStreamingJob) { + settingsBuilder.inStreamingMode(); + StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG); + env.enableCheckpointing(400); + env.setMaxParallelism(2); + env.setParallelism(2); + tEnv = StreamTableEnvironment.create(env, settingsBuilder.build()); + } else { + settingsBuilder.inBatchMode(); + tEnv = TableEnvironment.create(settingsBuilder.build()); + } + } + return tEnv; + } + + @BeforeEach + public void before() { + String warehouseLocation = "file:" + warehouseRoot.getPath(); + this.catalog = new HadoopCatalog(new Configuration(), warehouseLocation); + Map config = Maps.newHashMap(); + config.put("type", "iceberg"); + config.put(FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, ICEBERG_CATALOG_TYPE_HADOOP); + config.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + sql("CREATE CATALOG %s WITH %s", CATALOG, toWithClause(config)); + + sql("CREATE DATABASE %s", FLINK_DATABASE); + sql("USE CATALOG %s", CATALOG); + sql("USE %s", DATABASE); + sql( + "CREATE TABLE %s (id int, data varchar) with ('write.format.default'='%s')", + TABLE, FileFormat.PARQUET.name()); + } + + @AfterEach + public void clean() throws Exception { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, TABLE); + dropDatabase(FLINK_DATABASE, true); + BoundedTableFactory.clearDataSets(); + + dropCatalog(CATALOG, true); + catalog.close(); + } + + @TestTemplate + public void testWriteParallelism() { + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + 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 /*+ 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)); + } + + @TestTemplate + public void testHashDistributeMode() throws Exception { + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List dataSet = + IntStream.range(1, 1000) + .mapToObj(i -> ImmutableList.of(Row.of(i, "aaa"), Row.of(i, "bbb"), Row.of(i, "ccc"))) + .flatMap(List::stream) + .collect(Collectors.toList()); + String dataId = BoundedTableFactory.registerDataSet(ImmutableList.of(dataSet)); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.HASH.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(dataSet); + + // Sometimes we will have more than one checkpoint if we pass the auto checkpoint interval, + // thus producing multiple snapshots. Here we assert that each snapshot has only 1 file per + // partition. + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + Map> snapshotToDataFiles = SimpleDataUtil.snapshotToDataFiles(table); + for (List dataFiles : snapshotToDataFiles.values()) { + if (dataFiles.isEmpty()) { + continue; + } + + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "aaa"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "bbb"))) + .hasSize(1); + assertThat( + SimpleDataUtil.matchingPartitions( + dataFiles, table.spec(), ImmutableMap.of("data", "ccc"))) + .hasSize(1); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // 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(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java index b709c0058f7d..47f5485df879 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/TestIcebergConnector.java @@ -248,20 +248,12 @@ public void testCreateTableUnderDefaultDatabase() { public void testCatalogDatabaseConflictWithFlinkDatabase() { sql("CREATE DATABASE IF NOT EXISTS `%s`", databaseName()); sql("USE `%s`", databaseName()); - - try { - testCreateConnectorTable(); - // Ensure that the table was created under the specific database. - assertThatThrownBy( - () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) - .isInstanceOf(org.apache.flink.table.api.TableException.class) - .hasMessageStartingWith("Could not execute CreateTable in path"); - } finally { - sql("DROP TABLE IF EXISTS `%s`.`%s`", databaseName(), TABLE_NAME); - if (!isDefaultDatabaseName()) { - sql("DROP DATABASE `%s`", databaseName()); - } - } + testCreateConnectorTable(); + // Ensure that the table was created under the specific database. + assertThatThrownBy( + () -> sql("CREATE TABLE `default_catalog`.`%s`.`%s`", databaseName(), TABLE_NAME)) + .isInstanceOf(org.apache.flink.table.api.TableException.class) + .hasMessageStartingWith("Could not execute CreateTable in path"); } @TestTemplate @@ -327,10 +319,6 @@ private String databaseName() { return properties.getOrDefault("catalog-database", "default_database"); } - private String toWithClause(Map props) { - return CatalogTestBase.toWithClause(props); - } - private String createWarehouse() { try { return String.format( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java index 61ab087f2ca3..b778037c559c 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSink.java @@ -18,20 +18,11 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.TestFixtures.DATABASE; -import static org.assertj.core.api.Assertions.assertThat; -import static org.assertj.core.api.Assertions.assertThatThrownBy; - import java.io.IOException; -import java.util.Collections; import java.util.List; -import java.util.Map; -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.table.api.TableSchema; import org.apache.flink.table.data.RowData; -import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; @@ -39,37 +30,19 @@ import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; import org.apache.iceberg.PartitionSpec; -import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; -import org.apache.iceberg.catalog.TableIdentifier; -import org.apache.iceberg.flink.FlinkWriteOptions; -import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; -import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; 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.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; -import org.junit.jupiter.api.extension.RegisterExtension; @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSink extends TestFlinkIcebergSinkBase { - - @RegisterExtension - public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = - MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); - - @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = - new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); - - private TableLoader tableLoader; - @Parameter(index = 0) private FileFormat format; @@ -99,7 +72,7 @@ public static Object[][] parameters() { @BeforeEach public void before() throws IOException { - table = + this.table = CATALOG_EXTENSION .catalog() .createTable( @@ -110,14 +83,14 @@ public void before() throws IOException { : PartitionSpec.unpartitioned(), ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); - env = + this.env = StreamExecutionEnvironment.getExecutionEnvironment( MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) .enableCheckpointing(100) .setParallelism(parallelism) .setMaxParallelism(parallelism); - tableLoader = CATALOG_EXTENSION.tableLoader(); + this.tableLoader = CATALOG_EXTENSION.tableLoader(); } @TestTemplate @@ -140,246 +113,13 @@ public void testWriteRowData() throws Exception { SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); } - private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) - throws Exception { - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .tableSchema(tableSchema) - .writeParallelism(parallelism) - .distributionMode(distributionMode) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); - } - - private int partitionFiles(String partition) throws IOException { - return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); - } - @TestTemplate public void testWriteRow() throws Exception { - testWriteRow(null, DistributionMode.NONE); + testWriteRow(parallelism, null, DistributionMode.NONE); } @TestTemplate public void testWriteRowWithTableSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); - } - - @TestTemplate - public void testJobNoneDistributeMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, DistributionMode.NONE); - - if (parallelism > 1) { - if (partitioned) { - int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); - assertThat(files).isGreaterThan(3); - } - } - } - - @TestTemplate - public void testJobHashDistributionMode() { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Flink does not support 'range' write distribution mode now."); - } - - @TestTemplate - public void testJobNullDistributionMode() throws Exception { - table - .updateProperties() - .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) - .commit(); - - testWriteRow(null, null); - - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testPartitionWriteMode() throws Exception { - testWriteRow(null, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testShuffleByPartitionWithSchema() throws Exception { - testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); - if (partitioned) { - assertThat(partitionFiles("aaa")).isEqualTo(1); - assertThat(partitionFiles("bbb")).isEqualTo(1); - assertThat(partitionFiles("ccc")).isEqualTo(1); - } - } - - @TestTemplate - public void testTwoSinksInDisjointedDAG() throws Exception { - Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); - - Table leftTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("left"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader leftTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); - - Table rightTable = - CATALOG_EXTENSION - .catalog() - .createTable( - TableIdentifier.of("right"), - SimpleDataUtil.SCHEMA, - partitioned - ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() - : PartitionSpec.unpartitioned(), - props); - TableLoader rightTableLoader = - TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); - - env = - StreamExecutionEnvironment.getExecutionEnvironment( - MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) - .enableCheckpointing(100) - .setParallelism(parallelism) - .setMaxParallelism(parallelism); - env.getConfig().disableAutoGeneratedUIDs(); - - List leftRows = createRows("left-"); - DataStream leftStream = - env.fromCollection(leftRows, ROW_TYPE_INFO) - .name("leftCustomSource") - .uid("leftCustomSource"); - FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) - .table(leftTable) - .tableLoader(leftTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .distributionMode(DistributionMode.NONE) - .uidPrefix("leftIcebergSink") - .append(); - - List rightRows = createRows("right-"); - DataStream rightStream = - env.fromCollection(rightRows, ROW_TYPE_INFO) - .name("rightCustomSource") - .uid("rightCustomSource"); - FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) - .table(rightTable) - .tableLoader(rightTableLoader) - .tableSchema(SimpleDataUtil.FLINK_SCHEMA) - .writeParallelism(parallelism) - .distributionMode(DistributionMode.HASH) - .uidPrefix("rightIcebergSink") - .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) - .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream."); - - SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); - SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); - - leftTable.refresh(); - assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); - rightTable.refresh(); - assertThat(rightTable.currentSnapshot().summary()) - .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) - .containsEntry("direction", "rightTable"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownDistributionMode() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid distribution mode: UNRECOGNIZED"); - } - - @TestTemplate - public void testOverrideWriteConfigWithUnknownFileFormat() { - Map newProps = Maps.newHashMap(); - newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); - - List rows = createRows(""); - DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); - - FlinkSink.Builder builder = - FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) - .table(table) - .tableLoader(tableLoader) - .writeParallelism(parallelism) - .setAll(newProps); - - assertThatThrownBy(builder::append) - .isInstanceOf(IllegalArgumentException.class) - .hasMessage("Invalid file format: UNRECOGNIZED"); - } - - @TestTemplate - public void testWriteRowWithTableRefreshInterval() throws Exception { - List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); - DataStream dataStream = - env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) - .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); - - Configuration flinkConf = new Configuration(); - flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); - - FlinkSink.forRowData(dataStream) - .table(table) - .tableLoader(tableLoader) - .flinkConf(flinkConf) - .writeParallelism(parallelism) - .append(); - - // Execute the program. - env.execute("Test Iceberg DataStream"); - - // Assert the iceberg table's records. - SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); } } diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java index b38aa6b50ce6..9ce36cc1e8d0 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkBase.java @@ -18,29 +18,52 @@ */ package org.apache.iceberg.flink.sink; +import static org.apache.iceberg.flink.TestFixtures.DATABASE; + +import java.io.IOException; import java.util.List; import java.util.stream.Collectors; 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.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.Table; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; import org.apache.iceberg.flink.source.BoundedTestSource; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.extension.RegisterExtension; public class TestFlinkIcebergSinkBase { - protected Table table; - protected StreamExecutionEnvironment env; + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + protected static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); protected static final DataFormatConverters.RowConverter CONVERTER = new DataFormatConverters.RowConverter(SimpleDataUtil.FLINK_SCHEMA.getFieldDataTypes()); + protected TableLoader tableLoader; + protected Table table; + protected StreamExecutionEnvironment env; + protected BoundedTestSource createBoundedSource(List rows) { return new BoundedTestSource<>(rows.toArray(new Row[0])); } @@ -61,4 +84,28 @@ protected List createRows(String prefix) { protected List convertToRowData(List rows) { return rows.stream().map(CONVERTER::toInternal).collect(Collectors.toList()); } + + protected void testWriteRow( + int writerParallelism, TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(writerParallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + protected int partitionFiles(String partition) throws IOException { + return SimpleDataUtil.partitionDataFiles(table, ImmutableMap.of("data", partition)).size(); + } } 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 new file mode 100644 index 000000000000..75e397d3f203 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkDistributionMode.java @@ -0,0 +1,180 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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.TableProperties; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * This tests the distribution mode of Flink sink. Extract them separately since it is unnecessary + * to test different file formats (Avro, Orc, Parquet) like in {@link TestFlinkIcebergSink}. + * Removing the file format dimension reduces the number of combinations from 12 to 4, which helps + * reduce test run time. + */ +@ExtendWith(ParameterizedTestExtension.class) +public class TestFlinkIcebergSinkDistributionMode extends TestFlinkIcebergSinkBase { + + @RegisterExtension + public static MiniClusterExtension miniClusterResource = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private final FileFormat format = FileFormat.PARQUET; + + @Parameter(index = 0) + private int parallelism; + + @Parameter(index = 1) + private boolean partitioned; + + @Parameters(name = "parallelism = {0}, partitioned = {1}") + public static Object[][] parameters() { + return new Object[][] { + {1, true}, + {1, false}, + {2, true}, + {2, false} + }; + } + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(parallelism, SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).isGreaterThan(3); + } + } + } + + @TestTemplate + public void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(parallelism, null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testPartitionWriteMode() throws Exception { + testWriteRow(parallelism, null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")).isEqualTo(1); + assertThat(partitionFiles("bbb")).isEqualTo(1); + assertThat(partitionFiles("ccc")).isEqualTo(1); + } + } + + @TestTemplate + public void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java new file mode 100644 index 000000000000..36a59b20431c --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkExtended.java @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +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.table.data.RowData; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +/** + * This class tests the more extended features of Flink sink. Extract them separately since it is + * unnecessary to test all the parameters combinations in {@link TestFlinkIcebergSink}. Each test + * method in {@link TestFlinkIcebergSink} runs 12 combinations, which are expensive and slow. + */ +public class TestFlinkIcebergSinkExtended extends TestFlinkIcebergSinkBase { + private final boolean partitioned = true; + private final int parallelism = 2; + private final FileFormat format = FileFormat.PARQUET; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + this.env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @Test + public void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + FlinkSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidPrefix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + FlinkSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidPrefix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestFlinkIcebergSink.class.getName()) + .setSnapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + assertThat(leftTable.currentSnapshot().summary()).doesNotContainKeys("flink.test", "direction"); + rightTable.refresh(); + assertThat(rightTable.currentSnapshot().summary()) + .containsEntry("flink.test", TestFlinkIcebergSink.class.getName()) + .containsEntry("direction", "rightTable"); + } + + @Test + public void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps); + + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @Test + public void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 95d0b90b6ca9..51f9025b4159 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -103,7 +103,7 @@ public void before() throws IOException { public void after() { sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, INPUT_TABLE_NAME); sql("DROP TABLE IF EXISTS %s.%s", DATABASE_NAME, OUTPUT_TABLE_NAME); - sql("DROP DATABASE %s", DATABASE_NAME); + dropDatabase(DATABASE_NAME, true); dropCatalog(CATALOG_NAME, true); } From 59b30bbf8983bbe4aa1988fdeed438414348de08 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 22 Aug 2024 13:16:05 -0700 Subject: [PATCH 0624/1019] Spec: Minor modifications for v3 (#10948) --- format/spec.md | 119 +++++++++++++++++++++++++++---------------------- 1 file changed, 66 insertions(+), 53 deletions(-) diff --git a/format/spec.md b/format/spec.md index c322f8174fe2..b8da0454c648 100644 --- a/format/spec.md +++ b/format/spec.md @@ -44,6 +44,14 @@ 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 of the Iceberg spec extends data types and existing metadata structures to add new capabilities: + +* New data types: nanosecond timestamp(tz) +* Default value support for columns +* Multi-argument transforms for partitioning and sorting + ## Goals @@ -113,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. @@ -125,9 +133,9 @@ 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 v1 and v2 tables. These requirements are intended for writers when adding metadata files (including manifests files and manifest lists) to a table with the given version. +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. | Requirement | Write behavior | |-------------|----------------| @@ -135,10 +143,10 @@ Some tables in this spec have columns that specify requirements for v1 and v2 ta | _optional_ | The field can be written or omitted | | _required_ | The field must be written | -Readers should be more permissive because v1 metadata files are allowed in v2 tables so that tables can be upgraded to v2 without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected v2 read behavior: +Readers should be more permissive because v1 metadata files are allowed in v2 tables (or later) so that tables can be upgraded to without rewriting the metadata tree. For manifest list and manifest files, this table shows the expected read behavior for later versions: -| v1 | v2 | v2 read behavior | -|------------|------------|------------------| +| v1 | v2 | v2+ read behavior | +|------------|------------|-------------------| | | _optional_ | Read the field as _optional_ | | | _required_ | Read the field as _optional_; it may be missing in v1 files | | _optional_ | | Ignore the field | @@ -148,19 +156,21 @@ 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 | -Readers may be more strict for metadata JSON files because the JSON files are not reused and will always match the table version. Required v2 fields that were not present in v1 or optional in v1 may be handled as required fields. For example, a v2 table that is missing `last-sequence-number` can throw an exception. +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. -### Schemas and Data Types +Writers are not allowed to commit files with a partition spec that contains a field with an unknown transform. + +## 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). @@ -168,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. @@ -179,12 +189,12 @@ Supported primitive types are defined in the table below. Primitive types added | | **`long`** | 64-bit signed integers | | | | **`float`** | [32-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | Can promote to double | | | **`double`** | [64-bit IEEE 754](https://en.wikipedia.org/wiki/IEEE_754) floating point | | -| | **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed [1], precision must be 38 or less | +| | **`decimal(P,S)`** | Fixed-point decimal; precision P, scale S | Scale is fixed, precision must be 38 or less | | | **`date`** | Calendar date without timezone or time | | -| | **`time`** | Time of day without date, timezone | Microsecond precision [2] | -| | **`timestamp`** | Timestamp, microsecond precision, without timezone | [2] | +| | **`time`** | Time of day, microsecond precision, without date, timezone | | +| | **`timestamp`** | Timestamp, microsecond precision, without timezone | [1] | | | **`timestamptz`** | Timestamp, microsecond precision, with timezone | [2] | -| [v3](#version-3) | **`timestamp_ns`** | Timestamp, nanosecond precision, without timezone | [2] | +| [v3](#version-3) | **`timestamp_ns`** | Timestamp, nanosecond precision, without timezone | [1] | | [v3](#version-3) | **`timestamptz_ns`** | Timestamp, nanosecond precision, with timezone | [2] | | | **`string`** | Arbitrary-length character sequences | Encoded with UTF-8 [3] | | | **`uuid`** | Universally unique identifiers | Should use 16-byte fixed | @@ -193,16 +203,14 @@ Supported primitive types are defined in the table below. Primitive types added Notes: -1. Decimal scale is fixed and cannot be changed by schema evolution. Precision can only be widened. -2. `time`, `timestamp`, and `timestamptz` values are represented with _microsecond precision_. `timestamp_ns` and `timstamptz_ns` values are represented with _nanosecond precision_. - - Timestamp values _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). - - Timestamp values _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). +1. Timestamp values _without time zone_ represent a date and time of day regardless of zone: the time value is independent of zone adjustments (`2017-11-16 17:10:34` is always retrieved as `2017-11-16 17:10:34`). +2. Timestamp values _with time zone_ represent a point in time: values are stored as UTC and do not retain a source time zone (`2017-11-16 17:10:34 PST` is stored/retrieved as `2017-11-17 01:10:34 UTC` and these values are considered identical). 3. Character strings must be stored as UTF-8 encoded byte arrays. 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: @@ -216,7 +224,7 @@ The `initial-default` and `write-default` produce SQL default value behavior, wi 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 +283,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). @@ -284,7 +292,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. @@ -302,7 +310,7 @@ The set of metadata columns is: | **`2147483544 row`** | `struct<...>` | Deleted row values, used in position-based delete files | -### 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.) @@ -317,11 +325,13 @@ The source columns, selected by ids, must be a primitive type and cannot be cont Partition specs capture the transform from table data to partition values. This is used to transform predicates to partition predicates, in addition to transforming data values. Deriving partition predicates from column predicates on the table data is used to separate the logical queries from physical storage: the partitioning can change and the correct partition filters are always derived from column predicates. This simplifies queries because users don’t have to supply both logical predicates and partition predicates. For more information, see Scan Planning below. -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 ID, 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. +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. 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 | |-------------------|--------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|-------------| @@ -339,7 +349,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. @@ -356,7 +366,7 @@ Notes: For hash function details by type, see Appendix B. -#### Truncate Transform Details +### Truncate Transform Details | **Type** | **Config** | **Truncate specification** | **Examples** | |---------------|-----------------------|------------------------------------------------------------------|----------------------------------| @@ -374,7 +384,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. @@ -391,7 +401,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**. @@ -411,7 +421,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. @@ -479,7 +489,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. @@ -497,7 +507,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. @@ -511,7 +521,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. -### Snapshots +## Snapshots A snapshot consists of the following fields: @@ -544,7 +554,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. -#### Manifest Lists +### Manifest Lists Snapshots are embedded in table metadata, but the list of manifests for a snapshot are stored in a separate manifest list file. @@ -588,7 +598,7 @@ 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. -#### Scan Planning +### 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. @@ -600,6 +610,8 @@ Scan predicates are converted to partition predicates using an _inclusive projec For example, an `events` table with a timestamp column named `ts` that is partitioned by `ts_day=day(ts)` is queried by users with ranges over the timestamp column: `ts > X`. The inclusive projection is `ts_day >= day(X)`, which is used to select files that may have matching rows. Note that, in most cases, timestamps just before `X` will be included in the scan because the file contains rows that match the predicate and rows that do not match the predicate. +The inclusive projection for an unknown partition transform is _true_ because the partition field is ignored and not used in filtering. + Scan predicates are also used to filter data and delete files using column bounds and counts that are stored by field id in manifests. The same filter logic can be used for both data and delete files because both store metrics of the rows either inserted or deleted. If metrics show that a delete file has no rows that match a scan predicate, it may be ignored just as a data file would be ignored [2]. Data files that match the query filter must be read by the scan. @@ -611,10 +623,10 @@ Delete files that match the query filter must be applied to data files at read t * A _position_ delete file must be applied to a data file when all of the following are true: - 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 to the delete file's partition + - The data file's partition (both spec and partition values) is equal [4] to the delete file's partition * 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 to the delete file's partition _or_ the delete file's partition spec is unpartitioned + - 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 In general, deletes are applied only to data files that are older and in the same partition, except for two special cases: @@ -627,8 +639,9 @@ 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. +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 Reference +### Snapshot References Iceberg tables keep track of branches and tags using snapshot references. Tags are labels for individual snapshots. Branches are mutable named references that can be updated by committing a new snapshot as the branch's referenced snapshot using the [Commit Conflict Resolution and Retry](#commit-conflict-resolution-and-retry) procedures. @@ -645,7 +658,7 @@ The snapshot reference object records all the information of a reference includi Valid snapshot references are stored as the values of the `refs` map in table metadata. For serialization, see Appendix C. -#### Snapshot Retention Policy +### Snapshot Retention Policy Table snapshots expire and are removed from metadata to allow removed or replaced data files to be physically deleted. The snapshot expiration procedure removes snapshots from table metadata and applies the table's retention policy. @@ -661,13 +674,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: @@ -699,7 +712,7 @@ Table metadata consists of the following fields: For serialization details, see Appendix C. -#### 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 @@ -727,7 +740,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. @@ -778,7 +791,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. @@ -788,7 +801,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._ @@ -807,7 +820,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. @@ -824,7 +837,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. @@ -835,7 +848,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. @@ -860,7 +873,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. @@ -912,7 +925,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. @@ -1308,7 +1321,7 @@ Default values are added to struct fields in v3. Types `timestamp_ns` and `timestamptz_ns` are added in v3. -All readers are required to read tables with unknown partition transforms, ignoring them. +All readers are required to read tables with unknown partition transforms, ignoring the unsupported partition fields when filtering. Writing v3 metadata: From 8b0aea6ae6f531217af2feeee471ab5aff5195fd Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 22 Aug 2024 14:59:48 -0700 Subject: [PATCH 0625/1019] Flink: backport PR #10859 for range distribution (#10990) --- .../apache/iceberg/flink/FlinkConfParser.java | 27 ++ .../apache/iceberg/flink/FlinkWriteConf.java | 21 ++ .../iceberg/flink/FlinkWriteOptions.java | 14 + .../apache/iceberg/flink/sink/FlinkSink.java | 155 +++++++-- .../DataStatisticsOperatorFactory.java | 98 ++++++ .../TestFlinkIcebergSinkDistributionMode.java | 314 ++++++++++++++++++ .../flink/sink/TestFlinkIcebergSinkV2.java | 21 +- .../apache/iceberg/flink/FlinkConfParser.java | 27 ++ .../apache/iceberg/flink/FlinkWriteConf.java | 21 ++ .../iceberg/flink/FlinkWriteOptions.java | 14 + .../apache/iceberg/flink/sink/FlinkSink.java | 155 +++++++-- .../DataStatisticsOperatorFactory.java | 98 ++++++ .../flink/TestFlinkTableSinkExtended.java | 92 +++++ .../TestFlinkIcebergSinkDistributionMode.java | 314 ++++++++++++++++++ .../flink/sink/TestFlinkIcebergSinkV2.java | 21 +- 15 files changed, 1318 insertions(+), 74 deletions(-) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public > EnumConfParser enumConfParser(Class enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser { private String defaultValue; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 769af7d77140..2256d1e874ce 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -53,13 +53,19 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +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.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -233,15 +239,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + *

      Default is {@link StatisticsType#Auto} where initially Map statistics is used. But if + * cardinality is higher than the threshold (currently 10K) as defined in {@code + * SketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD}, statistics collection automatically switches to + * the sketch reservoir sampling. + * + *

      Explicit set the statistics type if the default behavior doesn't work. + * + * @param type to specify the statistics type for range distribution. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rangeDistributionStatisticsType(StatisticsType type) { + if (type != null) { + writeOptions.put(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key(), type.name()); + } + return this; + } + + /** + * If sort order contains partition columns, each sort key would map to one partition and data + * file. This relative weight can avoid placing too many small files for sort keys with low + * traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means + * each key has a base weight of `2%` of the targeted traffic weight per writer task. + * + *

      E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream + * contains events from now up to 180 days ago. With event time, traffic weight distribution + * across different days typically has a long tail pattern. Current day contains the most + * traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism + * is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer + * task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, + * the range partitioner would put all the oldest 150 days in one writer task. That writer task + * would write to 150 small files (one per day). Keeping 150 open files can potentially consume + * large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time + * can also be potentially slow. If this config is set to `0.02`. It means every sort key has a + * base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially + * avoid placing more than `50` data files (one per day) on one writer task no matter how small + * they are. + * + *

      This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For + * {@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as + * partition columns. Otherwise, too many partitions and small files may be generated during + * write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. + * + *

      Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -349,18 +408,20 @@ private DataStreamSink chainIcebergOperators() { // Find out the equality field id list based on the user-provided equality field column names. List equalityFieldIds = checkAndGetEqualityFieldIds(); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -447,7 +508,10 @@ private SingleOutputStreamOperator appendCommitter( } private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { + DataStream input, + RowType flinkRowType, + List equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +545,13 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), TypeInformation.of(WriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +561,15 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +611,52 @@ private DataStream distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +671,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory + implements CoordinatedOperatorFactory, + OneInputStreamOperatorFactory { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public > T createStreamOperator( + StreamOperatorParameters parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} 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 75e397d3f203..df8c3c79d3e3 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 @@ -20,28 +20,37 @@ 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.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; 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.Snapshot; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; 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.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.types.Conversions; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -177,4 +186,309 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + 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 + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource createRangeDistributionBoundedSource( + List> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List> createCharRows(int numOfCheckpoints, int countPerChar) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List> createIntRows(int numOfCheckpoints, int maxId) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } } 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 577c54976b9a..b283b8390a2b 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 @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -184,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java index 7167859e600c..d5eea6706b39 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkConfParser.java @@ -53,6 +53,10 @@ public LongConfParser longConf() { return new LongConfParser(); } + public DoubleConfParser doubleConf() { + return new DoubleConfParser(); + } + public > EnumConfParser enumConfParser(Class enumClass) { return new EnumConfParser<>(enumClass); } @@ -135,6 +139,29 @@ public Long parseOptional() { } } + class DoubleConfParser extends ConfParser { + private Double defaultValue; + + @Override + protected DoubleConfParser self() { + return this; + } + + public DoubleConfParser defaultValue(double value) { + this.defaultValue = value; + return self(); + } + + public double parse() { + Preconditions.checkArgument(defaultValue != null, "Default value cannot be null"); + return parse(Double::parseDouble, defaultValue); + } + + public Double parseOptional() { + return parse(Double::parseDouble, null); + } + } + class StringConfParser extends ConfParser { private String defaultValue; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java index ca7b1120bc81..a31902d49a8b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteConf.java @@ -26,6 +26,7 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** * A class for common Iceberg configs for Flink writes. @@ -167,6 +168,26 @@ public DistributionMode distributionMode() { return DistributionMode.fromName(modeName); } + public StatisticsType rangeDistributionStatisticsType() { + String name = + confParser + .stringConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.defaultValue()) + .parse(); + return StatisticsType.valueOf(name); + } + + public double rangeDistributionSortKeyBaseWeight() { + return confParser + .doubleConf() + .option(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key()) + .flinkConfig(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT) + .defaultValue(FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.defaultValue()) + .parse(); + } + public int workerPoolSize() { return confParser .intConf() diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java index df73f2e09cac..c35286774874 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkWriteOptions.java @@ -23,6 +23,7 @@ import org.apache.flink.configuration.ConfigOption; import org.apache.flink.configuration.ConfigOptions; import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.flink.sink.shuffle.StatisticsType; /** Flink sink write options */ public class FlinkWriteOptions { @@ -60,6 +61,19 @@ private FlinkWriteOptions() {} public static final ConfigOption DISTRIBUTION_MODE = ConfigOptions.key("distribution-mode").stringType().noDefaultValue(); + public static final ConfigOption RANGE_DISTRIBUTION_STATISTICS_TYPE = + ConfigOptions.key("range-distribution-statistics-type") + .stringType() + .defaultValue(StatisticsType.Auto.name()) + .withDescription("Type of statistics collection: Auto, Map, Sketch"); + + public static final ConfigOption RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT = + ConfigOptions.key("range-distribution-sort-key-base-weight") + .doubleType() + .defaultValue(0.0d) + .withDescription( + "Base weight for every sort key relative to target weight per writer task"); + // Branch to write to public static final ConfigOption BRANCH = ConfigOptions.key("branch").stringType().defaultValue(SnapshotRef.MAIN_BRANCH); 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 769af7d77140..2256d1e874ce 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 @@ -53,13 +53,19 @@ import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionField; import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Partitioning; import org.apache.iceberg.Schema; import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.SortOrder; import org.apache.iceberg.Table; import org.apache.iceberg.flink.FlinkSchemaUtil; import org.apache.iceberg.flink.FlinkWriteConf; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.TableLoader; +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.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -233,15 +239,68 @@ public Builder flinkConf(ReadableConfig config) { * @return {@link Builder} to connect the iceberg table. */ public Builder distributionMode(DistributionMode mode) { - Preconditions.checkArgument( - !DistributionMode.RANGE.equals(mode), - "Flink does not support 'range' write distribution mode now."); if (mode != null) { writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); } return this; } + /** + * Range distribution needs to collect statistics about data distribution to properly shuffle + * the records in relatively balanced way. In general, low cardinality should use {@link + * StatisticsType#Map} and high cardinality should use {@link StatisticsType#Sketch} Refer to + * {@link StatisticsType} Javadoc for more details. + * + *

      Default is {@link StatisticsType#Auto} where initially Map statistics is used. But if + * cardinality is higher than the threshold (currently 10K) as defined in {@code + * SketchUtil#OPERATOR_SKETCH_SWITCH_THRESHOLD}, statistics collection automatically switches to + * the sketch reservoir sampling. + * + *

      Explicit set the statistics type if the default behavior doesn't work. + * + * @param type to specify the statistics type for range distribution. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder rangeDistributionStatisticsType(StatisticsType type) { + if (type != null) { + writeOptions.put(FlinkWriteOptions.RANGE_DISTRIBUTION_STATISTICS_TYPE.key(), type.name()); + } + return this; + } + + /** + * If sort order contains partition columns, each sort key would map to one partition and data + * file. This relative weight can avoid placing too many small files for sort keys with low + * traffic. It is a double value that defines the minimal weight for each sort key. `0.02` means + * each key has a base weight of `2%` of the targeted traffic weight per writer task. + * + *

      E.g. the sink Iceberg table is partitioned daily by event time. Assume the data stream + * contains events from now up to 180 days ago. With event time, traffic weight distribution + * across different days typically has a long tail pattern. Current day contains the most + * traffic. The older days (long tail) contain less and less traffic. Assume writer parallelism + * is `10`. The total weight across all 180 days is `10,000`. Target traffic weight per writer + * task would be `1,000`. Assume the weight sum for the oldest 150 days is `1,000`. Normally, + * the range partitioner would put all the oldest 150 days in one writer task. That writer task + * would write to 150 small files (one per day). Keeping 150 open files can potentially consume + * large amount of memory. Flushing and uploading 150 files (however small) at checkpoint time + * can also be potentially slow. If this config is set to `0.02`. It means every sort key has a + * base weight of `2%` of targeted weight of `1,000` for every write task. It would essentially + * avoid placing more than `50` data files (one per day) on one writer task no matter how small + * they are. + * + *

      This is only applicable to {@link StatisticsType#Map} for low-cardinality scenario. For + * {@link StatisticsType#Sketch} high-cardinality sort columns, they are usually not used as + * partition columns. Otherwise, too many partitions and small files may be generated during + * write. Sketch range partitioner simply splits high-cardinality keys into ordered ranges. + * + *

      Default is {@code 0.0%}. + */ + public Builder rangeDistributionSortKeyBaseWeight(double weight) { + writeOptions.put( + FlinkWriteOptions.RANGE_DISTRIBUTION_SORT_KEY_BASE_WEIGHT.key(), Double.toString(weight)); + return this; + } + /** * Configuring the write parallel number for iceberg stream writer. * @@ -349,18 +408,20 @@ private DataStreamSink chainIcebergOperators() { // Find out the equality field id list based on the user-provided equality field column names. List equalityFieldIds = checkAndGetEqualityFieldIds(); - // Convert the requested flink table schema to flink row type. RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); + int writerParallelism = + flinkWriteConf.writeParallelism() == null + ? rowDataInput.getParallelism() + : flinkWriteConf.writeParallelism(); // Distribute the records from input data stream based on the write.distribution-mode and // equality fields. DataStream distributeStream = - distributeDataStream( - rowDataInput, equalityFieldIds, table.spec(), table.schema(), flinkRowType); + distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files SingleOutputStreamOperator writerStream = - appendWriter(distributeStream, flinkRowType, equalityFieldIds); + appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files // after successful checkpoint or end of input @@ -447,7 +508,10 @@ private SingleOutputStreamOperator appendCommitter( } private SingleOutputStreamOperator appendWriter( - DataStream input, RowType flinkRowType, List equalityFieldIds) { + DataStream input, + RowType flinkRowType, + List equalityFieldIds, + int writerParallelism) { // Validate the equality fields and partition fields if we enable the upsert mode. if (flinkWriteConf.upsertMode()) { Preconditions.checkState( @@ -481,17 +545,13 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - int parallelism = - flinkWriteConf.writeParallelism() == null - ? input.getParallelism() - : flinkWriteConf.writeParallelism(); SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), TypeInformation.of(WriteResult.class), streamWriter) - .setParallelism(parallelism); + .setParallelism(writerParallelism); if (uidPrefix != null) { writerStream = writerStream.uid(uidPrefix + "-writer"); } @@ -501,12 +561,15 @@ private SingleOutputStreamOperator appendWriter( private DataStream distributeDataStream( DataStream input, List equalityFieldIds, - PartitionSpec partitionSpec, - Schema iSchema, - RowType flinkRowType) { + RowType flinkRowType, + int writerParallelism) { DistributionMode writeMode = flinkWriteConf.distributionMode(); - LOG.info("Write distribution mode is '{}'", writeMode.modeName()); + + Schema iSchema = table.schema(); + PartitionSpec partitionSpec = table.spec(); + SortOrder sortOrder = table.sortOrder(); + switch (writeMode) { case NONE: if (equalityFieldIds.isEmpty()) { @@ -548,21 +611,52 @@ private DataStream distributeDataStream( } case RANGE: - if (equalityFieldIds.isEmpty()) { + // Ideally, exception should be thrown in the combination of range distribution and + // equality fields. Primary key case should use hash distribution mode. + // Keep the current behavior of falling back to keyBy for backward compatibility. + if (!equalityFieldIds.isEmpty()) { LOG.warn( - "Fallback to use 'none' distribution mode, because there are no equality fields set " - + "and {}=range is not supported yet in flink", - WRITE_DISTRIBUTION_MODE); - return input; - } else { - LOG.info( - "Distribute rows by equality fields, because there are equality fields set " - + "and{}=range is not supported yet in flink", + "Hash distribute rows by equality fields, even though {}=range is set. " + + "Range distribution for primary keys are not always safe in " + + "Flink streaming writer.", WRITE_DISTRIBUTION_MODE); return input.keyBy( new EqualityFieldKeySelector(iSchema, flinkRowType, equalityFieldIds)); } + // range distribute by partition key or sort key if table has an SortOrder + Preconditions.checkState( + sortOrder.isSorted() || partitionSpec.isPartitioned(), + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + if (sortOrder.isUnsorted()) { + sortOrder = Partitioning.sortOrderFor(partitionSpec); + LOG.info("Construct sort order from partition spec"); + } + + LOG.info("Range distribute rows by sort order: {}", sortOrder); + StatisticsType statisticsType = flinkWriteConf.rangeDistributionStatisticsType(); + SingleOutputStreamOperator shuffleStream = + input + .transform( + operatorName("range-shuffle"), + TypeInformation.of(StatisticsOrRecord.class), + new DataStatisticsOperatorFactory( + iSchema, + sortOrder, + writerParallelism, + statisticsType, + flinkWriteConf.rangeDistributionSortKeyBaseWeight())) + // Set the parallelism same as input operator to encourage chaining + .setParallelism(input.getParallelism()); + if (uidPrefix != null) { + shuffleStream = shuffleStream.uid(uidPrefix + "-shuffle"); + } + + return shuffleStream + .partitionCustom(new RangePartitioner(iSchema, sortOrder), r -> r) + .filter(StatisticsOrRecord::hasRecord) + .map(StatisticsOrRecord::record); + default: throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + writeMode); } @@ -577,12 +671,9 @@ static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { TypeUtil.validateWriteSchema(schema, writeSchema, true, true); // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will - // be promoted to - // iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT (backend by 1 - // 'byte'), we will - // read 4 bytes rather than 1 byte, it will mess up the byte array in BinaryRowData. So here - // we must use flink - // schema. + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the + // byte array in BinaryRowData. So here we must use flink schema. return (RowType) requestedSchema.toRowDataType().getLogicalType(); } else { return FlinkSchemaUtil.convert(schema); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java new file mode 100644 index 000000000000..dc147bf36d13 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/DataStatisticsOperatorFactory.java @@ -0,0 +1,98 @@ +/* + * 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.annotation.Internal; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.runtime.operators.coordination.OperatorCoordinator; +import org.apache.flink.runtime.operators.coordination.OperatorEventGateway; +import org.apache.flink.streaming.api.operators.AbstractStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.CoordinatedOperatorFactory; +import org.apache.flink.streaming.api.operators.OneInputStreamOperatorFactory; +import org.apache.flink.streaming.api.operators.StreamOperator; +import org.apache.flink.streaming.api.operators.StreamOperatorParameters; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; + +@Internal +public class DataStatisticsOperatorFactory extends AbstractStreamOperatorFactory + implements CoordinatedOperatorFactory, + OneInputStreamOperatorFactory { + + private final Schema schema; + private final SortOrder sortOrder; + private final int downstreamParallelism; + private final StatisticsType type; + private final double closeFileCostWeightPercentage; + + public DataStatisticsOperatorFactory( + Schema schema, + SortOrder sortOrder, + int downstreamParallelism, + StatisticsType type, + double closeFileCostWeightPercentage) { + this.schema = schema; + this.sortOrder = sortOrder; + this.downstreamParallelism = downstreamParallelism; + this.type = type; + this.closeFileCostWeightPercentage = closeFileCostWeightPercentage; + } + + @Override + public OperatorCoordinator.Provider getCoordinatorProvider( + String operatorName, OperatorID operatorID) { + return new DataStatisticsCoordinatorProvider( + operatorName, + operatorID, + schema, + sortOrder, + downstreamParallelism, + type, + closeFileCostWeightPercentage); + } + + @SuppressWarnings("unchecked") + @Override + public > T createStreamOperator( + StreamOperatorParameters parameters) { + OperatorID operatorId = parameters.getStreamConfig().getOperatorID(); + String operatorName = parameters.getStreamConfig().getOperatorName(); + OperatorEventGateway gateway = + parameters.getOperatorEventDispatcher().getOperatorEventGateway(operatorId); + + DataStatisticsOperator rangeStatisticsOperator = + new DataStatisticsOperator( + operatorName, schema, sortOrder, gateway, downstreamParallelism, type); + + rangeStatisticsOperator.setup( + parameters.getContainingTask(), parameters.getStreamConfig(), parameters.getOutput()); + parameters + .getOperatorEventDispatcher() + .registerEventHandler(operatorId, rangeStatisticsOperator); + + return (T) rangeStatisticsOperator; + } + + @SuppressWarnings("rawtypes") + @Override + public Class getStreamOperatorClass(ClassLoader classLoader) { + return DataStatisticsOperator.class; + } +} 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 482cfd110bde..b63547d433a4 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 @@ -20,6 +20,7 @@ import static org.apache.iceberg.flink.FlinkCatalogFactory.ICEBERG_CATALOG_TYPE_HADOOP; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.util.Arrays; @@ -46,6 +47,7 @@ import org.apache.iceberg.Parameter; import org.apache.iceberg.ParameterizedTestExtension; import org.apache.iceberg.Parameters; +import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; import org.apache.iceberg.catalog.Namespace; @@ -54,6 +56,7 @@ import org.apache.iceberg.hadoop.HadoopCatalog; 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.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; @@ -241,4 +244,93 @@ public void testHashDistributeMode() throws Exception { sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); } } + + @TestTemplate + public void testRangeDistributionPartitionColumn() { + // Range partitioner currently only works with streaming writes (with checkpoints) + assumeThat(isStreamingJob).isTrue(); + + // Initialize a BoundedSource table to precisely emit those rows in only one checkpoint. + List> rowsPerCheckpoint = + IntStream.range(1, 6) + .mapToObj( + checkpointId -> { + List charRows = Lists.newArrayList(); + // emit 26x10 rows for each checkpoint cycle + for (int i = 0; i < 10; ++i) { + for (char c = 'a'; c <= 'z'; c++) { + charRows.add(Row.of(c - 'a', String.valueOf(c))); + } + } + return charRows; + }) + .collect(Collectors.toList()); + List flattenedRows = + rowsPerCheckpoint.stream().flatMap(List::stream).collect(Collectors.toList()); + + String dataId = BoundedTableFactory.registerDataSet(rowsPerCheckpoint); + sql( + "CREATE TABLE %s(id INT NOT NULL, data STRING NOT NULL)" + + " WITH ('connector'='BoundedSource', 'data-id'='%s')", + SOURCE_TABLE, dataId); + + assertThat(sql("SELECT * FROM %s", SOURCE_TABLE)) + .as("Should have the expected rows in source table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Map tableProps = + ImmutableMap.of( + "write.format.default", + FileFormat.PARQUET.name(), + TableProperties.WRITE_DISTRIBUTION_MODE, + DistributionMode.RANGE.modeName()); + + String tableName = "test_hash_distribution_mode"; + sql( + "CREATE TABLE %s(id INT, data VARCHAR) PARTITIONED BY (data) WITH %s", + tableName, toWithClause(tableProps)); + + try { + // Insert data set. + sql("INSERT INTO %s SELECT * FROM %s", tableName, SOURCE_TABLE); + + assertThat(sql("SELECT * FROM %s", tableName)) + .as("Should have the expected rows in sink table.") + .containsExactlyInAnyOrderElementsOf(flattenedRows); + + Table table = catalog.loadTable(TableIdentifier.of(ICEBERG_NAMESPACE, tableName)); + // 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(5); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // range partition results in each partition only assigned to one writer task + // maybe less than 26 partitions as BoundedSource doesn't always precisely + // control the checkpoint boundary. + // It is hard to precisely control the test condition in SQL tests. + // Here only minimal safe assertions are applied to avoid flakiness. + // If there are no shuffling, the number of data files could be as high as + // 26 * 4 as the default parallelism is set to 4 for the mini cluster. + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } finally { + sql("DROP TABLE IF EXISTS %s.%s", FLINK_DATABASE, tableName); + } + } } 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 75e397d3f203..df8c3c79d3e3 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 @@ -20,28 +20,37 @@ 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.List; import java.util.Map; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.test.junit5.MiniClusterExtension; import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; import org.apache.iceberg.DistributionMode; 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.Snapshot; import org.apache.iceberg.TableProperties; import org.apache.iceberg.flink.FlinkWriteOptions; import org.apache.iceberg.flink.HadoopCatalogExtension; import org.apache.iceberg.flink.MiniFlinkClusterExtension; import org.apache.iceberg.flink.SimpleDataUtil; 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.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.types.Conversions; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -177,4 +186,309 @@ public void testOverrideWriteConfigWithUnknownDistributionMode() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Invalid distribution mode: UNRECOGNIZED"); } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderUnpartitioned() throws Exception { + assumeThat(partitioned).isFalse(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.Builder builder = + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism); + + // Range distribution requires either sort order or partition spec defined + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } + + @TestTemplate + public void testRangeDistributionWithoutSortOrderPartitioned() throws Exception { + assumeThat(partitioned).isTrue(); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + 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 + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("data").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createCharRows(numOfCheckpoints, 10)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Map) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + if (partitioned) { + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + // up to 26 partitions + assertThat(addedDataFiles).hasSizeLessThanOrEqualTo(26); + } + } else { + for (Snapshot snapshot : rangePartitionedCycles) { + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + } + + @TestTemplate + public void testRangeDistributionSketchWithSortOrder() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 6; + DataStream dataStream = + env.addSource( + createRangeDistributionBoundedSource(createIntRows(numOfCheckpoints, 1_000)), + ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Sketch) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + /** Test migration from Map stats to Sketch stats */ + @TestTemplate + public void testRangeDistributionStatisticsMigration() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + table.replaceSortOrder().asc("id").commit(); + + int numOfCheckpoints = 4; + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + // checkpointId 2 would emit 11_000 records which is larger than + // the OPERATOR_SKETCH_SWITCH_THRESHOLD of 10_000. + // This should trigger the stats migration. + int maxId = checkpointId < 1 ? 1_000 : 11_000; + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + DataStream dataStream = + env.addSource(createRangeDistributionBoundedSource(rowsPerCheckpoint), ROW_TYPE_INFO); + FlinkSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .rangeDistributionStatisticsType(StatisticsType.Auto) + .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); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + // since the input has a single value for the data column, + // it is always the same partition. Hence there is no difference + // for partitioned or not + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + 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); + // verify there is no overlap in min-max stats range + if (parallelism == 2) { + assertIdColumnStatsNoRangeOverlap(addedDataFiles.get(0), addedDataFiles.get(1)); + } + } + } + + private BoundedTestSource createRangeDistributionBoundedSource( + List> rowsPerCheckpoint) { + return new BoundedTestSource<>(rowsPerCheckpoint); + } + + private List> createCharRows(int numOfCheckpoints, int countPerChar) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(26 * countPerChar); + for (int j = 0; j < countPerChar; ++j) { + for (char c = 'a'; c <= 'z'; ++c) { + rows.add(Row.of(1, String.valueOf(c))); + } + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private List> createIntRows(int numOfCheckpoints, int maxId) { + List> rowsPerCheckpoint = Lists.newArrayListWithCapacity(numOfCheckpoints); + for (int checkpointId = 0; checkpointId < numOfCheckpoints; ++checkpointId) { + List rows = Lists.newArrayListWithCapacity(maxId); + for (int j = 0; j < maxId; ++j) { + // fixed value "a" for the data (possible partition column) + rows.add(Row.of(j, "a")); + } + + rowsPerCheckpoint.add(rows); + } + + return rowsPerCheckpoint; + } + + private void assertIdColumnStatsNoRangeOverlap(DataFile file1, DataFile file2) { + // id column has fieldId 1 + int file1LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.lowerBounds().get(1)); + int file1UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file1.upperBounds().get(1)); + int file2LowerBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.lowerBounds().get(1)); + int file2UpperBound = + Conversions.fromByteBuffer(Types.IntegerType.get(), file2.upperBounds().get(1)); + + if (file1LowerBound < file2LowerBound) { + assertThat(file1UpperBound).isLessThanOrEqualTo(file2LowerBound); + } else { + assertThat(file2UpperBound).isLessThanOrEqualTo(file1LowerBound); + } + } } 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 577c54976b9a..b283b8390a2b 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 @@ -30,6 +30,7 @@ import org.apache.flink.types.Row; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.DistributionMode; import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.ParameterizedTestExtension; @@ -184,11 +185,21 @@ public void testUpsertModeCheck() throws Exception { .hasMessage( "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); - assertThatThrownBy( - () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) - .isInstanceOf(IllegalStateException.class) - .hasMessage( - "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (writeDistributionMode.equals(DistributionMode.RANGE.modeName()) && !partitioned) { + // validation error thrown from distributeDataStream + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Invalid write distribution mode: range. Need to define sort order or partition spec."); + } else { + // validation error thrown from appendWriter + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } } @TestTemplate From 9aa0bcb70122fa39aa0488f2a88138a3c6ceb5de Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 23 Aug 2024 15:42:24 +0200 Subject: [PATCH 0626/1019] Flink: Port #10484 to v1.20 (#10989) --- flink/v1.20/build.gradle | 1 + .../maintenance/operator/JdbcLockFactory.java | 321 +++++++++ .../maintenance/operator/MonitorSource.java | 4 +- .../maintenance/operator/TableChange.java | 43 +- .../operator/TableMaintenanceMetrics.java | 34 + .../flink/maintenance/operator/Trigger.java | 72 ++ .../operator/TriggerEvaluator.java | 128 ++++ .../operator/TriggerLockFactory.java | 63 ++ .../maintenance/operator/TriggerManager.java | 339 ++++++++++ .../operator/ConstantsForTests.java | 29 + .../MetricsReporterFactoryForTests.java | 153 +++++ .../operator/OperatorTestBase.java | 107 ++- .../operator/TestJdbcLockFactory.java | 57 ++ .../operator/TestLockFactoryBase.java | 80 +++ .../operator/TestMonitorSource.java | 28 +- .../operator/TestTriggerManager.java | 622 ++++++++++++++++++ ...ink.metrics.reporter.MetricReporterFactory | 16 + 17 files changed, 2081 insertions(+), 16 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java create mode 100644 flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory diff --git a/flink/v1.20/build.gradle b/flink/v1.20/build.gradle index f2e1fb51a1f4..294c88d90709 100644 --- a/flink/v1.20/build.gradle +++ b/flink/v1.20/build.gradle @@ -119,6 +119,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { testImplementation libs.awaitility testImplementation libs.assertj.core + testImplementation libs.sqlite.jdbc } test { 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/operator/JdbcLockFactory.java new file mode 100644 index 000000000000..21c8935abede --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/JdbcLockFactory.java @@ -0,0 +1,321 @@ +/* + * 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.IOException; +import java.sql.DatabaseMetaData; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.SQLNonTransientConnectionException; +import java.sql.SQLTimeoutException; +import java.sql.SQLTransientConnectionException; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcClientPool; +import org.apache.iceberg.jdbc.UncheckedInterruptedException; +import org.apache.iceberg.jdbc.UncheckedSQLException; +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; +import org.apache.iceberg.util.PropertyUtil; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JdbcLockFactory implements TriggerLockFactory { + private static final Logger LOG = LoggerFactory.getLogger(JdbcLockFactory.class); + + @VisibleForTesting + static final String INIT_LOCK_TABLES_PROPERTY = "flink-maintenance.lock.jdbc.init-lock-tables"; + + private static final String LOCK_TABLE_NAME = "flink_maintenance_lock"; + private static final int LOCK_ID_MAX_LENGTH = 100; + private static final String CREATE_LOCK_TABLE_SQL = + String.format( + "CREATE TABLE %s " + + "(LOCK_TYPE CHAR(1) NOT NULL, " + + "LOCK_ID VARCHAR(%s) NOT NULL, " + + "INSTANCE_ID CHAR(36) NOT NULL, PRIMARY KEY (LOCK_TYPE, LOCK_ID))", + LOCK_TABLE_NAME, LOCK_ID_MAX_LENGTH); + + private static final String CREATE_LOCK_SQL = + String.format( + "INSERT INTO %s (LOCK_TYPE, LOCK_ID, INSTANCE_ID) VALUES (?, ?, ?)", LOCK_TABLE_NAME); + private static final String GET_LOCK_SQL = + String.format("SELECT INSTANCE_ID FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=?", LOCK_TABLE_NAME); + private static final String DELETE_LOCK_SQL = + String.format( + "DELETE FROM %s WHERE LOCK_TYPE=? AND LOCK_ID=? AND INSTANCE_ID=?", LOCK_TABLE_NAME); + + private final String uri; + private final String lockId; + private final Map properties; + private transient JdbcClientPool pool; + + /** + * Creates a new {@link TriggerLockFactory}. The lockId should be unique between the users of the + * same uri. + * + * @param uri of the jdbc connection + * @param lockId which should indentify the job and the table + * @param properties used for creating the jdbc connection pool + */ + public JdbcLockFactory(String uri, String lockId, Map properties) { + Preconditions.checkNotNull(uri, "JDBC connection URI is required"); + Preconditions.checkNotNull(properties, "Properties map is required"); + Preconditions.checkArgument( + lockId.length() < LOCK_ID_MAX_LENGTH, + "Invalid prefix length: lockId should be shorter than %s", + LOCK_ID_MAX_LENGTH); + this.uri = uri; + this.lockId = lockId; + this.properties = properties; + } + + @Override + public void open() { + this.pool = new JdbcClientPool(1, uri, properties); + + if (PropertyUtil.propertyAsBoolean(properties, INIT_LOCK_TABLES_PROPERTY, false)) { + initializeLockTables(); + } + } + + /** Only used in testing to share the jdbc pool */ + @VisibleForTesting + void open(JdbcLockFactory other) { + this.pool = other.pool; + } + + @Override + public Lock createLock() { + return new Lock(pool, lockId, Type.MAINTENANCE); + } + + @Override + public Lock createRecoveryLock() { + return new Lock(pool, lockId, Type.RECOVERY); + } + + @Override + public void close() throws IOException { + pool.close(); + } + + private void initializeLockTables() { + LOG.debug("Creating database tables (if missing) to store table maintenance locks"); + try { + pool.run( + conn -> { + DatabaseMetaData dbMeta = conn.getMetaData(); + ResultSet tableExists = + dbMeta.getTables( + null /* catalog name */, + null /* schemaPattern */, + LOCK_TABLE_NAME /* tableNamePattern */, + null /* types */); + if (tableExists.next()) { + LOG.debug("Flink maintenance lock table already exists"); + return true; + } + + LOG.info("Creating Flink maintenance lock table {}", LOCK_TABLE_NAME); + return conn.prepareStatement(CREATE_LOCK_TABLE_SQL).execute(); + }); + + } catch (SQLTimeoutException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Query timed out"); + } catch (SQLTransientConnectionException | SQLNonTransientConnectionException e) { + throw new UncheckedSQLException( + e, "Cannot initialize JDBC table maintenance lock: Connection failed"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Cannot initialize JDBC table maintenance lock"); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted in call to initialize"); + } + } + + public static class Lock implements TriggerLockFactory.Lock { + private final JdbcClientPool pool; + private final String lockId; + private final Type type; + + public Lock(JdbcClientPool pool, String lockId, Type type) { + this.pool = pool; + this.lockId = lockId; + this.type = type; + } + + @Override + public boolean tryLock() { + if (isHeld()) { + LOG.info("Lock is already held for {}", this); + return false; + } + + String newInstanceId = UUID.randomUUID().toString(); + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(CREATE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, newInstanceId); + int count = sql.executeUpdate(); + LOG.info( + "Created {} lock with instanceId {} with row count {}", + this, + newInstanceId, + count); + return count == 1; + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during tryLock"); + } catch (SQLException e) { + // SQL exception happened when creating the lock. Check if the lock creation was + // successful behind the scenes. + if (newInstanceId.equals(instanceId())) { + return true; + } else { + throw new UncheckedSQLException(e, "Failed to create %s lock", this); + } + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public boolean isHeld() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + return rs.next(); + } + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", this); + } + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + @Override + public void unlock() { + try { + // Possible concurrency issue: + // - `unlock` and `tryLock` happens at the same time when there is an existing lock + // + // Steps: + // 1. `unlock` removes the lock in the database, but there is a temporary connection failure + // 2. `lock` founds that there is no lock, so creates a new lock + // 3. `unlock` retires the lock removal and removes the new lock + // + // To prevent the situation above we fetch the current lockId, and remove the lock + // only with the given id. + String instanceId = instanceId(); + + if (instanceId != null) { + pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(DELETE_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + sql.setString(3, instanceId); + long count = sql.executeUpdate(); + LOG.info( + "Deleted {} lock with instanceId {} with row count {}", + this, + instanceId, + count); + } catch (SQLException e) { + // SQL exception happened when deleting lock information + throw new UncheckedSQLException( + e, "Failed to delete %s lock with instanceId %s", this, instanceId); + } + + return null; + }); + } + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (UncheckedSQLException e) { + throw e; + } catch (SQLException e) { + // SQL exception happened when getting/updating lock information + throw new UncheckedSQLException(e, "Failed to remove lock %s", this); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this).add("type", type).add("lockId", lockId).toString(); + } + + @SuppressWarnings("checkstyle:NestedTryDepth") + private String instanceId() { + try { + return pool.run( + conn -> { + try (PreparedStatement sql = conn.prepareStatement(GET_LOCK_SQL)) { + sql.setString(1, type.key); + sql.setString(2, lockId); + try (ResultSet rs = sql.executeQuery()) { + if (rs.next()) { + return rs.getString(1); + } else { + return null; + } + } + } catch (SQLException e) { + // SQL exception happened when getting lock information + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + }); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new UncheckedInterruptedException(e, "Interrupted during unlock"); + } catch (SQLException e) { + throw new UncheckedSQLException(e, "Failed to get lock information for %s", type); + } + } + } + + private enum Type { + MAINTENANCE("m"), + RECOVERY("r"); + + private String key; + + Type(String key) { + this.key = key; + } + } +} 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 d74b2349b1de..89efffa15f16 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 -public class MonitorSource extends SingleThreadedIteratorSource { +class MonitorSource extends SingleThreadedIteratorSource { private static final Logger LOG = LoggerFactory.getLogger(MonitorSource.class); private final TableLoader tableLoader; @@ -58,7 +58,7 @@ public 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 */ - public MonitorSource( + 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 452ed80ed0e5..7d0b94e97da7 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 @@ -35,7 +35,7 @@ class TableChange { private long deleteFileSize; private int commitNum; - TableChange( + private TableChange( int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { this.dataFileNum = dataFileNum; this.deleteFileNum = deleteFileNum; @@ -67,6 +67,10 @@ static TableChange empty() { return new TableChange(0, 0, 0L, 0L, 0); } + static Builder builder() { + return new Builder(); + } + int dataFileNum() { return dataFileNum; } @@ -130,4 +134,41 @@ public boolean equals(Object other) { public int hashCode() { return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); } + + static class Builder { + private int dataFileNum = 0; + private int deleteFileNum = 0; + private long dataFileSize = 0L; + private long deleteFileSize = 0L; + private int commitNum = 0; + + public Builder dataFileNum(int newDataFileNum) { + this.dataFileNum = newDataFileNum; + return this; + } + + public Builder deleteFileNum(int newDeleteFileNum) { + this.deleteFileNum = newDeleteFileNum; + return this; + } + + public Builder dataFileSize(long newDataFileSize) { + this.dataFileSize = newDataFileSize; + return this; + } + + public Builder deleteFileSize(long newDeleteFileSize) { + this.deleteFileSize = newDeleteFileSize; + return this; + } + + public Builder commitNum(int newCommitNum) { + this.commitNum = newCommitNum; + return this; + } + + public TableChange build() { + return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + } + } } 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 new file mode 100644 index 000000000000..ec0fd920c34b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TableMaintenanceMetrics.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.maintenance.operator; + +public class TableMaintenanceMetrics { + public static final String GROUP_KEY = "maintenanceTask"; + public static final String GROUP_VALUE_DEFAULT = "maintenanceTask"; + + // TriggerManager metrics + public static final String RATE_LIMITER_TRIGGERED = "rateLimiterTriggered"; + public static final String CONCURRENT_RUN_THROTTLED = "concurrentRunThrottled"; + public static final String TRIGGERED = "triggered"; + public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + + private TableMaintenanceMetrics() { + // do not instantiate + } +} 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/operator/Trigger.java new file mode 100644 index 000000000000..85c6c8dbdd55 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/Trigger.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.flink.maintenance.operator; + +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +@Internal +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) { + 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); + } + + static Trigger recovery(long timestamp) { + return new Trigger(timestamp, null, null, true); + } + + long timestamp() { + return timestamp; + } + + SerializableTable table() { + return table; + } + + Integer taskId() { + return taskId; + } + + boolean isRecovery() { + return isRecovery; + } + + @Override + 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/TriggerEvaluator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java new file mode 100644 index 000000000000..37e4e3afd4bd --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerEvaluator.java @@ -0,0 +1,128 @@ +/* + * 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.Serializable; +import java.time.Duration; +import java.util.List; +import org.apache.flink.annotation.Internal; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@Internal +class TriggerEvaluator implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(TriggerEvaluator.class); + private final List predicates; + + private TriggerEvaluator(List predicates) { + Preconditions.checkArgument(!predicates.isEmpty(), "Provide at least 1 condition."); + + this.predicates = predicates; + } + + boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { + boolean result = + predicates.stream() + .anyMatch( + p -> { + try { + return p.evaluate(event, lastTimeMs, currentTimeMs); + } catch (Exception e) { + throw new RuntimeException("Error accessing state", e); + } + }); + LOG.debug( + "Checking event: {}, at {}, last: {} with result: {}", + event, + currentTimeMs, + lastTimeMs, + result); + return result; + } + + static class Builder implements Serializable { + private Integer commitNumber; + private Integer fileNumber; + private Long fileSize; + private Integer deleteFileNumber; + private Duration timeout; + + Builder commitNumber(int newCommitNumber) { + this.commitNumber = newCommitNumber; + return this; + } + + Builder fileNumber(int newFileNumber) { + this.fileNumber = newFileNumber; + return this; + } + + Builder fileSize(long newFileSize) { + this.fileSize = newFileSize; + return this; + } + + Builder deleteFileNumber(int newDeleteFileNumber) { + this.deleteFileNumber = newDeleteFileNumber; + return this; + } + + Builder timeout(Duration newTimeout) { + this.timeout = newTimeout; + return this; + } + + TriggerEvaluator build() { + List predicates = Lists.newArrayList(); + if (commitNumber != null) { + predicates.add((change, unused, unused2) -> change.commitNum() >= commitNumber); + } + + if (fileNumber != null) { + predicates.add( + (change, unused, unused2) -> + change.dataFileNum() + change.deleteFileNum() >= fileNumber); + } + + if (fileSize != null) { + predicates.add( + (change, unused, unused2) -> + change.dataFileSize() + change.deleteFileSize() >= fileSize); + } + + if (deleteFileNumber != null) { + predicates.add((change, unused, unused2) -> change.deleteFileNum() >= deleteFileNumber); + } + + if (timeout != null) { + predicates.add( + (change, lastTimeMs, currentTimeMs) -> + currentTimeMs - lastTimeMs >= timeout.toMillis()); + } + + return new TriggerEvaluator(predicates); + } + } + + private interface Predicate extends Serializable { + boolean evaluate(TableChange event, long lastTimeMs, long currentTimeMs); + } +} 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/operator/TriggerLockFactory.java new file mode 100644 index 000000000000..329223d27ccf --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerLockFactory.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.flink.maintenance.operator; + +import java.io.Closeable; +import java.io.Serializable; +import org.apache.flink.annotation.Experimental; + +/** Lock interface for handling locks for the Flink Table Maintenance jobs. */ +@Experimental +public interface TriggerLockFactory extends Serializable, Closeable { + void open(); + + Lock createLock(); + + Lock createRecoveryLock(); + + interface Lock { + /** + * Tries to acquire a lock with a given key. Anyone already holding a lock would prevent + * acquiring this lock. Not reentrant. + * + *

      Called by {@link TriggerManager}. Implementations could assume that are no concurrent + * calls for this method. + * + * @return true if the lock is acquired by this job, false if the lock + * is already held by someone + */ + boolean tryLock(); + + /** + * Checks if the lock is already taken. + * + * @return true if the lock is held by someone + */ + 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. + */ + void unlock(); + } +} 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 new file mode 100644 index 000000000000..f4c3c1d47cf7 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TriggerManager.java @@ -0,0 +1,339 @@ +/* + * 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.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; +import org.apache.flink.api.common.state.ValueState; +import org.apache.flink.api.common.state.ValueStateDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.metrics.Counter; +import org.apache.flink.runtime.state.FunctionInitializationContext; +import org.apache.flink.runtime.state.FunctionSnapshotContext; +import org.apache.flink.streaming.api.TimerService; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * TriggerManager starts the Maintenance Tasks by emitting {@link Trigger} messages which are + * calculated based on the incoming {@link TableChange} messages. The TriggerManager keeps track of + * the changes since the last run of the Maintenance Tasks and triggers a new run based on the + * result of the {@link TriggerEvaluator}. + * + *

      The TriggerManager prevents overlapping Maintenance Task runs using {@link + * TriggerLockFactory.Lock}. The current implementation only handles conflicts within a single job. + * Users should avoid scheduling maintenance for the same table in different Flink jobs. + * + *

      The TriggerManager should run as a global operator. {@link KeyedProcessFunction} is used, so + * the timer functions are available, but the key is not used. + */ +@Internal +class TriggerManager extends KeyedProcessFunction + implements CheckpointedFunction { + private static final Logger LOG = LoggerFactory.getLogger(TriggerManager.class); + + private final TableLoader tableLoader; + private final TriggerLockFactory lockFactory; + private final List taskNames; + private final List evaluators; + private final long minFireDelayMs; + private final long lockCheckDelayMs; + private transient Counter rateLimiterTriggeredCounter; + private transient Counter concurrentRunThrottledCounter; + private transient Counter nothingToTriggerCounter; + private transient List triggerCounters; + private transient ValueState nextEvaluationTimeState; + private transient ListState accumulatedChangesState; + private transient ListState lastTriggerTimesState; + private transient Long nextEvaluationTime; + private transient List accumulatedChanges; + private transient List lastTriggerTimes; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient boolean shouldRestoreTasks = false; + private transient boolean inited = false; + // To keep the task scheduling fair we keep the last triggered task position in memory. + // If we find a task to trigger, then we run it, but after it is finished, we start from the given + // position to prevent "starvation" of the tasks. + // When there is nothing to trigger, we start from the beginning, as the order of the tasks might + // be important (RewriteDataFiles first, and then RewriteManifestFiles later) + private transient int startsFrom = 0; + private transient boolean triggered = false; + + TriggerManager( + TableLoader tableLoader, + TriggerLockFactory lockFactory, + List taskNames, + List evaluators, + long minFireDelayMs, + long lockCheckDelayMs) { + Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); + Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); + Preconditions.checkArgument( + taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + Preconditions.checkArgument( + evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); + Preconditions.checkArgument( + taskNames.size() == evaluators.size(), "Provide a name and evaluator for all of the tasks"); + Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); + Preconditions.checkArgument( + lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); + + this.tableLoader = tableLoader; + this.lockFactory = lockFactory; + this.taskNames = taskNames; + this.evaluators = evaluators; + this.minFireDelayMs = minFireDelayMs; + this.lockCheckDelayMs = lockCheckDelayMs; + } + + @Override + public void open(Configuration parameters) throws Exception { + this.rateLimiterTriggeredCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.RATE_LIMITER_TRIGGERED); + this.concurrentRunThrottledCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.CONCURRENT_RUN_THROTTLED); + this.nothingToTriggerCounter = + getRuntimeContext() + .getMetricGroup() + .addGroup( + TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) + .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); + this.triggerCounters = + taskNames.stream() + .map( + name -> + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.TRIGGERED)) + .collect(Collectors.toList()); + + this.nextEvaluationTimeState = + getRuntimeContext() + .getState(new ValueStateDescriptor<>("triggerManagerNextTriggerTime", Types.LONG)); + this.accumulatedChangesState = + getRuntimeContext() + .getListState( + new ListStateDescriptor<>( + "triggerManagerAccumulatedChange", TypeInformation.of(TableChange.class))); + this.lastTriggerTimesState = + getRuntimeContext() + .getListState(new ListStateDescriptor<>("triggerManagerLastTriggerTime", Types.LONG)); + + tableLoader.open(); + } + + @Override + public void snapshotState(FunctionSnapshotContext context) throws Exception { + if (inited) { + // Only store state if initialized + nextEvaluationTimeState.update(nextEvaluationTime); + accumulatedChangesState.update(accumulatedChanges); + lastTriggerTimesState.update(lastTriggerTimes); + LOG.info( + "Storing state: nextEvaluationTime {}, accumulatedChanges {}, lastTriggerTimes {}", + nextEvaluationTime, + accumulatedChanges, + lastTriggerTimes); + } else { + LOG.info("Not initialized, state is not stored"); + } + } + + @Override + public void initializeState(FunctionInitializationContext context) throws Exception { + LOG.info("Initializing state restored: {}", context.isRestored()); + lockFactory.open(); + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + if (context.isRestored()) { + shouldRestoreTasks = true; + } + } + + @Override + public void processElement(TableChange change, Context ctx, Collector out) + throws Exception { + init(out, ctx.timerService()); + + accumulatedChanges.forEach(tableChange -> tableChange.merge(change)); + + long current = ctx.timerService().currentProcessingTime(); + if (nextEvaluationTime == null) { + checkAndFire(current, ctx.timerService(), out); + } else { + LOG.info( + "Trigger manager rate limiter triggered current: {}, next: {}, accumulated changes: {}", + current, + nextEvaluationTime, + accumulatedChanges); + rateLimiterTriggeredCounter.inc(); + } + } + + @Override + public void onTimer(long timestamp, OnTimerContext ctx, Collector out) throws Exception { + init(out, ctx.timerService()); + this.nextEvaluationTime = null; + checkAndFire(ctx.timerService().currentProcessingTime(), ctx.timerService(), out); + } + + @Override + public void close() throws IOException { + tableLoader.close(); + lockFactory.close(); + } + + private void checkAndFire(long current, TimerService timerService, Collector out) { + if (shouldRestoreTasks) { + if (recoveryLock.isHeld()) { + // Recovered tasks in progress. Skip trigger check + LOG.debug("The recovery lock is still held at {}", current); + schedule(timerService, current + lockCheckDelayMs); + return; + } else { + LOG.info("The recovery is finished at {}", current); + shouldRestoreTasks = false; + } + } + + Integer taskToStart = + nextTrigger(evaluators, accumulatedChanges, lastTriggerTimes, current, startsFrom); + if (taskToStart == null) { + // Nothing to execute + if (!triggered) { + nothingToTriggerCounter.inc(); + LOG.debug("Nothing to execute at {} for collected: {}", current, accumulatedChanges); + } else { + LOG.debug("Execution check finished"); + } + + // Next time start from the beginning + startsFrom = 0; + triggered = false; + return; + } + + if (lock.tryLock()) { + TableChange change = accumulatedChanges.get(taskToStart); + SerializableTable table = + (SerializableTable) SerializableTable.copyOf(tableLoader.loadTable()); + out.collect(Trigger.create(current, table, taskToStart)); + LOG.debug("Fired event with time: {}, collected: {} for {}", current, change, table.name()); + triggerCounters.get(taskToStart).inc(); + accumulatedChanges.set(taskToStart, TableChange.empty()); + lastTriggerTimes.set(taskToStart, current); + schedule(timerService, current + minFireDelayMs); + startsFrom = (taskToStart + 1) % evaluators.size(); + triggered = true; + } else { + // A task is already running, waiting for it to finish + LOG.info("Failed to acquire lock. Delaying task to {}", current + lockCheckDelayMs); + + startsFrom = taskToStart; + concurrentRunThrottledCounter.inc(); + schedule(timerService, current + lockCheckDelayMs); + } + + timerService.registerProcessingTimeTimer(nextEvaluationTime); + } + + private void schedule(TimerService timerService, long time) { + this.nextEvaluationTime = time; + timerService.registerProcessingTimeTimer(time); + } + + private static Integer nextTrigger( + List evaluators, + List changes, + List lastTriggerTimes, + long currentTime, + int startPos) { + int current = startPos; + do { + if (evaluators + .get(current) + .check(changes.get(current), lastTriggerTimes.get(current), currentTime)) { + return current; + } + + current = (current + 1) % evaluators.size(); + } while (current != startPos); + + return null; + } + + private void init(Collector out, TimerService timerService) throws Exception { + if (!inited) { + long current = timerService.currentProcessingTime(); + + // Initialize from state + this.nextEvaluationTime = nextEvaluationTimeState.value(); + this.accumulatedChanges = Lists.newArrayList(accumulatedChangesState.get()); + this.lastTriggerTimes = Lists.newArrayList(lastTriggerTimesState.get()); + + // Initialize if the state was empty + if (accumulatedChanges.isEmpty()) { + for (int i = 0; i < evaluators.size(); ++i) { + accumulatedChanges.add(TableChange.empty()); + lastTriggerTimes.add(current); + } + } + + if (shouldRestoreTasks) { + // When the job state is restored, there could be ongoing tasks. + // To prevent collision with the new triggers the following is done: + // - add a recovery lock + // - fire a recovery trigger + // This ensures that the tasks of the previous trigger are executed, and the lock is removed + // in the end. The result of the 'tryLock' is ignored as an already existing lock prevents + // collisions as well. + recoveryLock.tryLock(); + out.collect(Trigger.recovery(current)); + if (nextEvaluationTime == null) { + schedule(timerService, current + minFireDelayMs); + } + } + + inited = true; + } + } +} 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 new file mode 100644 index 000000000000..36e162d4f068 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/ConstantsForTests.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.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/MetricsReporterFactoryForTests.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java new file mode 100644 index 000000000000..7a523035b7fb --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/MetricsReporterFactoryForTests.java @@ -0,0 +1,153 @@ +/* + * 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.Arrays; +import java.util.Map; +import java.util.Properties; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; +import org.apache.flink.metrics.Counter; +import org.apache.flink.metrics.Gauge; +import org.apache.flink.metrics.Metric; +import org.apache.flink.metrics.MetricConfig; +import org.apache.flink.metrics.MetricGroup; +import org.apache.flink.metrics.reporter.MetricReporter; +import org.apache.flink.metrics.reporter.MetricReporterFactory; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; + +public class MetricsReporterFactoryForTests implements MetricReporterFactory { + private static final TestMetricsReporter INSTANCE = new TestMetricsReporter(); + private static final Pattern FULL_METRIC_NAME = + Pattern.compile( + "\\.taskmanager\\.[^.]+\\.[^.]+\\.([^.]+)\\.\\d+\\." + + TableMaintenanceMetrics.GROUP_KEY + + "\\.([^.]+)\\.([^.]+)"); + + private static Map counters = Maps.newConcurrentMap(); + private static Map gauges = Maps.newConcurrentMap(); + private static Set monitoredMetricNames; + + public MetricsReporterFactoryForTests() { + monitoredMetricNames = + Arrays.stream(TableMaintenanceMetrics.class.getDeclaredFields()) + .map( + f -> { + try { + return f.get(null).toString(); + } catch (IllegalAccessException e) { + throw new RuntimeException(e); + } + }) + .collect(Collectors.toSet()); + } + + @Override + public MetricReporter createMetricReporter(Properties properties) { + return INSTANCE; + } + + public static void reset() { + counters = Maps.newConcurrentMap(); + gauges = Maps.newConcurrentMap(); + } + + public static Long counter(String name) { + return counterValues().get(name); + } + + public static Long gauge(String name) { + return gaugeValues().get(name); + } + + public static void assertGauges(Map expected) { + assertThat(filter(gaugeValues(), expected)).isEqualTo(filter(expected, expected)); + } + + public static void assertCounters(Map expected) { + assertThat(filter(counterValues(), expected)).isEqualTo(filter(expected, expected)); + } + + private static Map gaugeValues() { + return gauges.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> (Long) entry.getValue().getValue())); + } + + private static Map counterValues() { + return counters.entrySet().stream() + .collect( + Collectors.toMap( + entry -> longName(entry.getKey()), entry -> entry.getValue().getCount())); + } + + private static Map filter(Map original, Map filter) { + return original.entrySet().stream() + .filter( + entry -> { + Long filterValue = filter.get(entry.getKey()); + return filterValue == null || filterValue != -1; + }) + .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + } + + private static String longName(String fullName) { + Matcher matcher = FULL_METRIC_NAME.matcher(fullName); + if (!matcher.matches()) { + throw new RuntimeException(String.format("Can't parse simplified metrics name %s", fullName)); + } + + return matcher.group(1) + "." + matcher.group(2) + "." + matcher.group(3); + } + + private static class TestMetricsReporter implements MetricReporter { + @Override + public void open(MetricConfig config) { + // do nothing + } + + @Override + public void close() { + // do nothing + } + + @Override + public void notifyOfAddedMetric(Metric metric, String metricName, MetricGroup group) { + if (monitoredMetricNames.contains(metricName)) { + if (metric instanceof Counter) { + counters.put(group.getMetricIdentifier(metricName), (Counter) metric); + } + + if (metric instanceof Gauge) { + gauges.put(group.getMetricIdentifier(metricName), (Gauge) metric); + } + } + } + + @Override + public void notifyOfRemovedMetric(Metric metric, String metricName, MetricGroup group) { + // do nothing + } + } +} 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 272e0b693fd3..225853086545 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 @@ -20,16 +20,24 @@ import static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import java.io.File; 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.test.junit5.MiniClusterExtension; import org.apache.iceberg.flink.FlinkCatalogFactory; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.awaitility.Awaitility; import org.junit.jupiter.api.extension.RegisterExtension; 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(); static final String TABLE_NAME = "test_table"; @@ -39,7 +47,7 @@ class OperatorTestBase { new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUMBER_TASK_MANAGERS) .setNumberSlotsPerTaskManager(SLOTS_PER_TASK_MANAGER) - .setConfiguration(new Configuration(DISABLE_CLASSLOADER_CHECK_CONFIG)) + .setConfiguration(config()) .build()); @RegisterExtension @@ -48,4 +56,101 @@ class OperatorTestBase { "catalog", ImmutableMap.of("type", "iceberg", FlinkCatalogFactory.ICEBERG_CATALOG_TYPE, "hadoop"), "db"); + + 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; + } + + protected static TriggerLockFactory lockFactory() { + return new TriggerLockFactory() { + @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 + } + }; + } + + /** + * 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 + */ + public 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 + */ + public static void closeJobClient(JobClient jobClient) { + closeJobClient(jobClient, null); + } + + private static class MemoryLock implements TriggerLockFactory.Lock { + boolean locked = false; + + @Override + public boolean tryLock() { + if (locked) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } } 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/operator/TestJdbcLockFactory.java new file mode 100644 index 000000000000..051d09d92bad --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestJdbcLockFactory.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.flink.maintenance.operator; + +import static org.apache.iceberg.flink.maintenance.operator.JdbcLockFactory.INIT_LOCK_TABLES_PROPERTY; +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.junit.jupiter.api.Test; + +class TestJdbcLockFactory extends TestLockFactoryBase { + @Override + TriggerLockFactory lockFactory() { + return lockFactory("tableName"); + } + + @Test + void testMultiTableLock() { + JdbcLockFactory other = lockFactory("tableName2"); + other.open((JdbcLockFactory) this.lockFactory); + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = other.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + private JdbcLockFactory lockFactory(String tableName) { + Map properties = Maps.newHashMap(); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "username", "user"); + properties.put(JdbcCatalog.PROPERTY_PREFIX + "password", "password"); + properties.put(INIT_LOCK_TABLES_PROPERTY, "true"); + + return new JdbcLockFactory( + "jdbc:sqlite:file::memory:?ic" + UUID.randomUUID().toString().replace("-", ""), + tableName, + properties); + } +} 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/operator/TestLockFactoryBase.java new file mode 100644 index 000000000000..bf9e86f2534d --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockFactoryBase.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.io.IOException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +abstract class TestLockFactoryBase { + protected TriggerLockFactory lockFactory; + + abstract TriggerLockFactory lockFactory(); + + @BeforeEach + void before() { + this.lockFactory = lockFactory(); + lockFactory.open(); + } + + @AfterEach + void after() throws IOException { + lockFactory.close(); + } + + @Test + void testTryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock1.tryLock()).isFalse(); + assertThat(lock2.tryLock()).isFalse(); + } + + @Test + void testUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + } + + @Test + void testNoConflictWithRecoveryLock() { + TriggerLockFactory.Lock lock1 = lockFactory.createLock(); + TriggerLockFactory.Lock lock2 = lockFactory.createRecoveryLock(); + assertThat(lock1.tryLock()).isTrue(); + assertThat(lock2.tryLock()).isTrue(); + } + + @Test + void testDoubleUnLock() { + TriggerLockFactory.Lock lock = lockFactory.createLock(); + assertThat(lock.tryLock()).isTrue(); + + lock.unlock(); + lock.unlock(); + assertThat(lock.tryLock()).isTrue(); + assertThat(lock.tryLock()).isFalse(); + } +} 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 876d64214560..8c02601025ae 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 @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.maintenance.operator; -import static org.apache.iceberg.flink.maintenance.operator.FlinkStreamingTestUtils.closeJobClient; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; @@ -35,6 +34,7 @@ import org.apache.flink.streaming.api.datastream.DataStream; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.hadoop.fs.Path; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DataFile; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.FileFormat; @@ -161,7 +161,8 @@ void testSource() throws Exception { } // The first non-empty event should contain the expected value - return newEvent.equals(new TableChange(1, 0, size, 0L, 1)); + return newEvent.equals( + TableChange.builder().dataFileNum(1).dataFileSize(size).commitNum(1).build()); }); } finally { closeJobClient(jobClient); @@ -348,15 +349,18 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange List deleteFiles = Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); - long dataSize = dataFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - long deleteSize = deleteFiles.stream().mapToLong(d -> d.fileSizeInBytes()).sum(); - boolean hasDelete = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().hasNext(); - - return new TableChange( - previous.dataFileNum() + dataFiles.size(), - previous.deleteFileNum() + deleteFiles.size(), - previous.dataFileSize() + dataSize, - previous.deleteFileSize() + deleteSize, - previous.commitNum() + 1); + long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteSize = deleteFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + + TableChange newChange = previous.copy(); + newChange.merge( + TableChange.builder() + .dataFileNum(dataFiles.size()) + .dataFileSize(dataSize) + .deleteFileNum(deleteFiles.size()) + .deleteFileSize(deleteSize) + .commitNum(1) + .build()); + return newChange; } } 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 new file mode 100644 index 000000000000..55e64f3e84bf --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestTriggerManager.java @@ -0,0 +1,622 @@ +/* + * 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.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; +import org.apache.flink.api.common.typeinfo.Types; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +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.flink.TableLoader; +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; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +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 long processingTime = 0L; + private TriggerLockFactory lockFactory; + private TriggerLockFactory.Lock lock; + private TriggerLockFactory.Lock recoveringLock; + + @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(); + } + + @Test + void testCommitNumber() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitNumber(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 4); + } + } + + @Test + void testFileNumber() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileNumber(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 0); + + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileNum(1).deleteFileNum(1).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileNum(5).deleteFileNum(7).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 4); + } + } + + @Test + void testFileSize() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileSize(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 0); + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileSize(1L).deleteFileSize(1L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(3L).build(), 2); + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileSize(5L).deleteFileSize(7L).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(1L).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 4); + } + } + + @Test + void testDeleteFileNumber() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().deleteFileNumber(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult( + testHarness, TableChange.builder().dataFileNum(3).deleteFileNum(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 4); + } + } + + @Test + void testTimeout() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().timeout(Duration.ofSeconds(1)).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + TableChange event = TableChange.builder().dataFileSize(1).commitNum(1).build(); + + // Wait for some time + testHarness.processElement(event, EVENT_TIME); + assertThat(testHarness.extractOutputValues()).isEmpty(); + + // Wait for the timeout to expire + long newTime = EVENT_TIME + Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // Send a new event + testHarness.setProcessingTime(newTime + 1); + testHarness.processElement(event, newTime); + + // No trigger yet + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Send a new event + newTime += Duration.ofSeconds(1).toMillis(); + testHarness.setProcessingTime(newTime); + testHarness.processElement(event, newTime); + + // New trigger should arrive + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testStateRestore() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + testHarness.processElement( + TableChange.builder().dataFileSize(1).commitNum(1).build(), EVENT_TIME); + + assertThat(testHarness.extractOutputValues()).isEmpty(); + + state = testHarness.snapshot(1, EVENT_TIME); + } + + // Restore the state, write some more data, create a checkpoint, check the data which is written + manager = manager(tableLoader); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + // Arrives the first real change which triggers the recovery process + testHarness.processElement(TableChange.builder().commitNum(1).build(), EVENT_TIME_2); + assertTriggers( + testHarness.extractOutputValues(), + Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); + + // Remove the lock to allow the next trigger + recoveringLock.unlock(); + testHarness.setProcessingTime(EVENT_TIME_2); + // At this point the output contains the recovery trigger and the real trigger + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testMinFireDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, DELAY, 1); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // No new fire yet + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + @Test + void testLockCheckDelay() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader, 1, DELAY); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + + // Create a lock to prevent execution, and check that there is no result + assertThat(lock.tryLock()).isTrue(); + addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + long currentTime = testHarness.getProcessingTime(); + + // Remove the lock, and still no trigger + lock.unlock(); + assertThat(testHarness.extractOutputValues()).hasSize(1); + + // Check that the trigger fired after the delay + testHarness.setProcessingTime(currentTime + DELAY); + assertThat(testHarness.extractOutputValues()).hasSize(2); + } + } + + /** + * Simulating recovery scenarios where there is a leftover table lock, and ongoing maintenance + * task. + * + * @param locked if a lock exists on the table on job recovery + * @param runningTask is running and continues to run after job recovery + */ + @ParameterizedTest + @MethodSource("parametersForTestRecovery") + void testRecovery(boolean locked, boolean runningTask) throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + TriggerManager manager = manager(tableLoader); + OperatorSubtaskState state; + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + state = testHarness.snapshot(1, EVENT_TIME); + } + + if (locked) { + assertThat(lock.tryLock()).isTrue(); + } + + manager = manager(tableLoader); + List expected = Lists.newArrayListWithExpectedSize(3); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.initializeState(state); + testHarness.open(); + + ++processingTime; + expected.add(Trigger.recovery(processingTime)); + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Nothing happens until the recovery is finished + ++processingTime; + testHarness.setProcessingTime(processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + if (runningTask) { + // Simulate the action of the recovered maintenance task lock removal when it finishes + lock.unlock(); + } + + // Still no results as the recovery is ongoing + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + assertTriggers(testHarness.extractOutputValues(), expected); + + // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it + // received recovery trigger + lock.unlock(); + recoveringLock.unlock(); + + // Emit only a single trigger + ++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)); + assertTriggers(testHarness.extractOutputValues(), expected); + } + } + + @Test + void testTriggerMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + TriggerManager manager = + new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1, NAME_2), + Lists.newArrayList( + new TriggerEvaluator.Builder().commitNumber(2).build(), + new TriggerEvaluator.Builder().commitNumber(4).build()), + 1L, + 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // This one doesn't trigger - tests NOTHING_TO_TRIGGER + source.sendRecord(TableChange.builder().commitNum(1).build()); + + Awaitility.await() + .until( + () -> { + Long notingCounter = + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER); + return notingCounter != null && notingCounter.equals(1L); + }); + + // Trigger one of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitNum(1).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(1L); + lock.unlock(); + + // Trigger both of the tasks - tests TRIGGERED + source.sendRecord(TableChange.builder().commitNum(2).build()); + // Wait until we receive the trigger + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + lock.unlock(); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED)) + .isEqualTo(2L); + assertThat( + MetricsReporterFactoryForTests.counter(DUMMY_NAME + "." + NAME_2 + "." + 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) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testRateLimiterMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1_000_000L, 1L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger + source.sendRecord(TableChange.builder().commitNum(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // Remove the lock to allow the next trigger + lock.unlock(); + + // The second trigger will be blocked + source.sendRecord(TableChange.builder().commitNum(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED) + .equals(1L)); + + // Final check all the counters + assertCounters(1L, 0L); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testConcurrentRunMetrics() throws Exception { + TableLoader tableLoader = sql.tableLoader(TABLE_NAME); + + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = + new ManualSource<>(env, TypeInformation.of(TableChange.class)); + CollectingSink sink = new CollectingSink<>(); + + // High delay, so only triggered once + TriggerManager manager = manager(tableLoader, 1L, 1_000_000L); + source + .dataStream() + .keyBy(unused -> true) + .process(manager) + .name(DUMMY_NAME) + .forceNonParallel() + .sinkTo(sink); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + // Start the first trigger - notice that we do not remove the lock after the trigger + source.sendRecord(TableChange.builder().commitNum(2).build()); + assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); + + // The second trigger will be blocked by the lock + source.sendRecord(TableChange.builder().commitNum(2).build()); + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED) + .equals(1L)); + + // Final check all the counters + assertCounters(0L, 1L); + } finally { + closeJobClient(jobClient); + } + } + + private static Stream parametersForTestRecovery() { + return Stream.of( + Arguments.of(true, false), + Arguments.of(true, false), + Arguments.of(false, true), + Arguments.of(false, false)); + } + + private void assertCounters(long rateLimiterTrigger, long concurrentRunTrigger) { + MetricsReporterFactoryForTests.assertCounters( + new ImmutableMap.Builder() + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + RATE_LIMITER_TRIGGERED, + rateLimiterTrigger) + .put( + DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + CONCURRENT_RUN_THROTTLED, + concurrentRunTrigger) + .put(DUMMY_NAME + "." + NAME_1 + "." + TRIGGERED, 1L) + .put(DUMMY_NAME + "." + GROUP_VALUE_DEFAULT + "." + NOTHING_TO_TRIGGER, 0L) + .build()); + } + + private KeyedOneInputStreamOperatorTestHarness harness( + TriggerManager manager) throws Exception { + return new KeyedOneInputStreamOperatorTestHarness<>( + new KeyedProcessOperator<>(manager), value -> true, Types.BOOLEAN); + } + + private void addEventAndCheckResult( + OneInputStreamOperatorTestHarness testHarness, + TableChange event, + int expectedSize) + throws Exception { + ++processingTime; + testHarness.setProcessingTime(processingTime); + testHarness.processElement(event, processingTime); + assertThat(testHarness.extractOutputValues()).hasSize(expectedSize); + // Remove the lock to allow the next trigger + lock.unlock(); + } + + private TriggerManager manager(TableLoader tableLoader, TriggerEvaluator evaluator) { + return new TriggerManager( + tableLoader, lockFactory, Lists.newArrayList(NAME_1), Lists.newArrayList(evaluator), 1, 1); + } + + private TriggerManager manager( + TableLoader tableLoader, long minFireDelayMs, long lockCheckDelayMs) { + return new TriggerManager( + tableLoader, + lockFactory, + Lists.newArrayList(NAME_1), + Lists.newArrayList(new TriggerEvaluator.Builder().commitNumber(2).build()), + minFireDelayMs, + lockCheckDelayMs); + } + + private TriggerManager manager(TableLoader tableLoader) { + return manager(tableLoader, new TriggerEvaluator.Builder().commitNumber(2).build()); + } + + private static void assertTriggers(List expected, List actual) { + assertThat(actual).hasSize(expected.size()); + for (int i = 0; i < expected.size(); ++i) { + Trigger expectedTrigger = expected.get(i); + Trigger actualTrigger = actual.get(i); + 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()); + } + } + } + } +} diff --git a/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory new file mode 100644 index 000000000000..952255a52b7c --- /dev/null +++ b/flink/v1.20/flink/src/test/resources/META-INF/services/org.apache.flink.metrics.reporter.MetricReporterFactory @@ -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.flink.maintenance.operator.MetricsReporterFactoryForTests From 6daaba240219eac3554de678c6ee7fb7c6279364 Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 23 Aug 2024 17:19:58 +0200 Subject: [PATCH 0627/1019] Flink: Maintenance - TableChange refactor (#10992) --- .../maintenance/operator/JdbcLockFactory.java | 14 +- .../maintenance/operator/TableChange.java | 193 ++++++++++++------ .../operator/TriggerEvaluator.java | 85 +++++--- .../operator/TestMonitorSource.java | 27 ++- .../operator/TestTriggerManager.java | 182 +++++++++++------ 5 files changed, 329 insertions(+), 172 deletions(-) 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/operator/JdbcLockFactory.java index 21c8935abede..f22be33aeae6 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/operator/JdbcLockFactory.java @@ -105,12 +105,12 @@ void open(JdbcLockFactory other) { @Override public Lock createLock() { - return new Lock(pool, lockId, Type.MAINTENANCE); + return new JdbcLock(pool, lockId, Type.MAINTENANCE); } @Override public Lock createRecoveryLock() { - return new Lock(pool, lockId, Type.RECOVERY); + return new JdbcLock(pool, lockId, Type.RECOVERY); } @Override @@ -153,12 +153,12 @@ private void initializeLockTables() { } } - public static class Lock implements TriggerLockFactory.Lock { + private static class JdbcLock implements TriggerLockFactory.Lock { private final JdbcClientPool pool; private final String lockId; private final Type type; - public Lock(JdbcClientPool pool, String lockId, Type type) { + private JdbcLock(JdbcClientPool pool, String lockId, Type type) { this.pool = pool; this.lockId = lockId; this.type = type; @@ -221,7 +221,7 @@ public boolean isHeld() { throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); } catch (SQLException e) { // SQL exception happened when getting lock information - throw new UncheckedSQLException(e, "Failed to get lock information for %s", this); + throw new UncheckedSQLException(e, "Failed to check the state of the lock %s", this); } } @@ -266,8 +266,6 @@ public void unlock() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new UncheckedInterruptedException(e, "Interrupted during unlock"); - } catch (UncheckedSQLException e) { - throw e; } catch (SQLException e) { // SQL exception happened when getting/updating lock information throw new UncheckedSQLException(e, "Failed to remove lock %s", this); @@ -312,7 +310,7 @@ private enum Type { MAINTENANCE("m"), RECOVERY("r"); - private String key; + private final String key; Type(String key) { this.key = key; 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 7d0b94e97da7..5252cf61b031 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 @@ -29,19 +29,29 @@ /** Event describing changes in an Iceberg table */ @Internal class TableChange { - private int dataFileNum; - private int deleteFileNum; - private long dataFileSize; - private long deleteFileSize; - private int commitNum; - - private TableChange( - int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { - this.dataFileNum = dataFileNum; - this.deleteFileNum = deleteFileNum; - this.dataFileSize = dataFileSize; - this.deleteFileSize = deleteFileSize; - this.commitNum = commitNum; + private int dataFileCount; + private long dataFileSizeInBytes; + private int posDeleteFileCount; + private long posDeleteRecordCount; + private int eqDeleteFileCount; + private long eqDeleteRecordCount; + private int commitCount; + + TableChange( + int dataFileCount, + long dataFileSizeInBytes, + int posDeleteFileCount, + long posDeleteRecordCount, + int eqDeleteFileCount, + long eqDeleteRecordCount, + int commitCount) { + this.dataFileCount = dataFileCount; + this.dataFileSizeInBytes = dataFileSizeInBytes; + this.posDeleteFileCount = posDeleteFileCount; + this.posDeleteRecordCount = posDeleteRecordCount; + this.eqDeleteFileCount = eqDeleteFileCount; + this.eqDeleteRecordCount = eqDeleteRecordCount; + this.commitCount = commitCount; } TableChange(Snapshot snapshot, FileIO io) { @@ -50,67 +60,96 @@ private TableChange( dataFiles.forEach( dataFile -> { - this.dataFileNum++; - this.dataFileSize += dataFile.fileSizeInBytes(); + this.dataFileCount++; + this.dataFileSizeInBytes += dataFile.fileSizeInBytes(); }); deleteFiles.forEach( deleteFile -> { - this.deleteFileNum++; - this.deleteFileSize += deleteFile.fileSizeInBytes(); + switch (deleteFile.content()) { + case POSITION_DELETES: + this.posDeleteFileCount++; + this.posDeleteRecordCount += deleteFile.recordCount(); + break; + case EQUALITY_DELETES: + this.eqDeleteFileCount++; + this.eqDeleteRecordCount += deleteFile.recordCount(); + break; + default: + throw new IllegalArgumentException("Unexpected delete file content: " + deleteFile); + } }); - this.commitNum = 1; + this.commitCount = 1; } static TableChange empty() { - return new TableChange(0, 0, 0L, 0L, 0); + return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } static Builder builder() { return new Builder(); } - int dataFileNum() { - return dataFileNum; + int dataFileCount() { + return dataFileCount; + } + + long dataFileSizeInBytes() { + return dataFileSizeInBytes; } - int deleteFileNum() { - return deleteFileNum; + int posDeleteFileCount() { + return posDeleteFileCount; } - long dataFileSize() { - return dataFileSize; + long posDeleteRecordCount() { + return posDeleteRecordCount; } - long deleteFileSize() { - return deleteFileSize; + int eqDeleteFileCount() { + return eqDeleteFileCount; } - public int commitNum() { - return commitNum; + long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public int commitCount() { + return commitCount; } public void merge(TableChange other) { - this.dataFileNum += other.dataFileNum; - this.deleteFileNum += other.deleteFileNum; - this.dataFileSize += other.dataFileSize; - this.deleteFileSize += other.deleteFileSize; - this.commitNum += other.commitNum; + this.dataFileCount += other.dataFileCount; + this.dataFileSizeInBytes += other.dataFileSizeInBytes; + this.posDeleteFileCount += other.posDeleteFileCount; + this.posDeleteRecordCount += other.posDeleteRecordCount; + this.eqDeleteFileCount += other.eqDeleteFileCount; + this.eqDeleteRecordCount += other.eqDeleteRecordCount; + this.commitCount += other.commitCount; } TableChange copy() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("dataFileNum", dataFileNum) - .add("deleteFileNum", deleteFileNum) - .add("dataFileSize", dataFileSize) - .add("deleteFileSize", deleteFileSize) - .add("commitNum", commitNum) + .add("dataFileCount", dataFileCount) + .add("dataFileSizeInBytes", dataFileSizeInBytes) + .add("posDeleteFileCount", posDeleteFileCount) + .add("posDeleteRecordCount", posDeleteRecordCount) + .add("eqDeleteFileCount", eqDeleteFileCount) + .add("eqDeleteRecordCount", eqDeleteRecordCount) + .add("commitCount", commitCount) .toString(); } @@ -123,52 +162,80 @@ public boolean equals(Object other) { } TableChange that = (TableChange) other; - return dataFileNum == that.dataFileNum - && deleteFileNum == that.deleteFileNum - && dataFileSize == that.dataFileSize - && deleteFileSize == that.deleteFileSize - && commitNum == that.commitNum; + return dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes + && posDeleteFileCount == that.posDeleteFileCount + && posDeleteRecordCount == that.posDeleteRecordCount + && eqDeleteFileCount == that.eqDeleteFileCount + && eqDeleteRecordCount == that.eqDeleteRecordCount + && commitCount == that.commitCount; } @Override public int hashCode() { - return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return Objects.hash( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } static class Builder { - private int dataFileNum = 0; - private int deleteFileNum = 0; - private long dataFileSize = 0L; - private long deleteFileSize = 0L; - private int commitNum = 0; - - public Builder dataFileNum(int newDataFileNum) { - this.dataFileNum = newDataFileNum; + private int dataFileCount = 0; + private long dataFileSizeInBytes = 0L; + private int posDeleteFileCount = 0; + private long posDeleteRecordCount = 0L; + private int eqDeleteFileCount = 0; + private long eqDeleteRecordCount = 0L; + private int commitCount = 0; + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; return this; } - public Builder deleteFileNum(int newDeleteFileNum) { - this.deleteFileNum = newDeleteFileNum; + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; return this; } - public Builder dataFileSize(long newDataFileSize) { - this.dataFileSize = newDataFileSize; + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; return this; } - public Builder deleteFileSize(long newDeleteFileSize) { - this.deleteFileSize = newDeleteFileSize; + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; return this; } - public Builder commitNum(int newCommitNum) { - this.commitNum = newCommitNum; + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; return this; } public TableChange build() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } } } 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 37e4e3afd4bd..dba33b22a42a 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 @@ -40,15 +40,7 @@ private TriggerEvaluator(List predicates) { boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { boolean result = - predicates.stream() - .anyMatch( - p -> { - try { - return p.evaluate(event, lastTimeMs, currentTimeMs); - } catch (Exception e) { - throw new RuntimeException("Error accessing state", e); - } - }); + predicates.stream().anyMatch(p -> p.evaluate(event, lastTimeMs, currentTimeMs)); LOG.debug( "Checking event: {}, at {}, last: {} with result: {}", event, @@ -59,29 +51,47 @@ boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { } static class Builder implements Serializable { - private Integer commitNumber; - private Integer fileNumber; - private Long fileSize; - private Integer deleteFileNumber; + private Integer dataFileCount; + private Long dataFileSizeInBytes; + private Integer posDeleteFileCount; + private Long posDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer commitCount; private Duration timeout; - Builder commitNumber(int newCommitNumber) { - this.commitNumber = newCommitNumber; + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; return this; } - Builder fileNumber(int newFileNumber) { - this.fileNumber = newFileNumber; + public Builder dataFileSizeInBytes(long neDataFileSizeInBytes) { + this.dataFileSizeInBytes = neDataFileSizeInBytes; return this; } - Builder fileSize(long newFileSize) { - this.fileSize = newFileSize; + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; return this; } - Builder deleteFileNumber(int newDeleteFileNumber) { - this.deleteFileNumber = newDeleteFileNumber; + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; return this; } @@ -92,24 +102,37 @@ Builder timeout(Duration newTimeout) { TriggerEvaluator build() { List predicates = Lists.newArrayList(); - if (commitNumber != null) { - predicates.add((change, unused, unused2) -> change.commitNum() >= commitNumber); + if (dataFileCount != null) { + predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); + } + + if (dataFileSizeInBytes != null) { + predicates.add( + (change, unused, unused2) -> change.dataFileSizeInBytes() >= dataFileSizeInBytes); + } + + if (posDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteFileCount() >= posDeleteFileCount); + } + + if (posDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteRecordCount() >= posDeleteRecordCount); } - if (fileNumber != null) { + if (eqDeleteFileCount != null) { predicates.add( - (change, unused, unused2) -> - change.dataFileNum() + change.deleteFileNum() >= fileNumber); + (change, unused, unused2) -> change.eqDeleteFileCount() >= eqDeleteFileCount); } - if (fileSize != null) { + if (eqDeleteRecordCount != null) { predicates.add( - (change, unused, unused2) -> - change.dataFileSize() + change.deleteFileSize() >= fileSize); + (change, unused, unused2) -> change.eqDeleteRecordCount() >= eqDeleteRecordCount); } - if (deleteFileNumber != null) { - predicates.add((change, unused, unused2) -> change.deleteFileNum() >= deleteFileNumber); + if (commitCount != null) { + predicates.add((change, unused, unused2) -> change.commitCount() >= commitCount); } if (timeout != null) { 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 8c02601025ae..3aee05322561 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 @@ -162,7 +162,11 @@ void testSource() throws Exception { // The first non-empty event should contain the expected value return newEvent.equals( - TableChange.builder().dataFileNum(1).dataFileSize(size).commitNum(1).build()); + TableChange.builder() + .dataFileCount(1) + .dataFileSizeInBytes(size) + .commitCount(1) + .build()); }); } finally { closeJobClient(jobClient); @@ -298,17 +302,17 @@ void testMaxReadBack() { new MonitorSource.TableChangeIterator(tableLoader, null, 1); // For a single maxReadBack we only get a single change - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); // Expecting 2 commits/snapshots for maxReadBack=2 - assertThat(iterator.next().commitNum()).isEqualTo(2); + assertThat(iterator.next().commitCount()).isEqualTo(2); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // For maxReadBack Long.MAX_VALUE we get every change - assertThat(iterator.next().commitNum()).isEqualTo(3); + assertThat(iterator.next().commitCount()).isEqualTo(3); } @Test @@ -323,7 +327,7 @@ void testSkipReplace() { new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // Read the current snapshot - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot Table table = tableLoader.loadTable(); @@ -350,16 +354,17 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); - long deleteSize = deleteFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); TableChange newChange = previous.copy(); newChange.merge( TableChange.builder() - .dataFileNum(dataFiles.size()) - .dataFileSize(dataSize) - .deleteFileNum(deleteFiles.size()) - .deleteFileSize(deleteSize) - .commitNum(1) + .dataFileCount(dataFiles.size()) + .dataFileSizeInBytes(dataSize) + // Currently we only test with equality deletes + .eqDeleteFileCount(deleteFiles.size()) + .eqDeleteRecordCount(deleteRecordCount) + .commitCount(1) .build()); return newChange; } 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 55e64f3e84bf..fba4a12d9c6b 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 @@ -81,95 +81,159 @@ void after() throws IOException { } @Test - void testCommitNumber() throws Exception { + void testCommitCount() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 4); + } + } + + @Test + void testDataFileCount() throws Exception { TriggerManager manager = manager( - sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitNumber(3).build()); + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 0); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(3).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(10).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 0); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(5).build(), 3); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 4); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 4); } } @Test - void testFileNumber() throws Exception { + void testDataFileSizeInBytes() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileNumber(3).build()); + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(5L).build(), 2); - addEventAndCheckResult( - testHarness, TableChange.builder().dataFileNum(1).deleteFileNum(1).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); - addEventAndCheckResult( - testHarness, TableChange.builder().dataFileNum(5).deleteFileNum(7).build(), 3); + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 3); + } + } + + @Test + void testPosDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(10).build(), 3); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 4); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 4); } } @Test - void testFileSize() throws Exception { + void testPosDeleteRecordCount() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileSize(3).build()); + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 0); addEventAndCheckResult( - testHarness, TableChange.builder().dataFileSize(1L).deleteFileSize(1L).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(3L).build(), 2); + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 0); addEventAndCheckResult( - testHarness, TableChange.builder().dataFileSize(5L).deleteFileSize(7L).build(), 3); + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(5L).build(), 2); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(1L).build(), 3); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 4); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 3); } } @Test - void testDeleteFileNumber() throws Exception { + void testEqDeleteFileCount() throws Exception { TriggerManager manager = manager( sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().deleteFileNumber(3).build()); + new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult( - testHarness, TableChange.builder().dataFileNum(3).deleteFileNum(1).build(), 0); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(2).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(10).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 4); + } + } + + @Test + void testEqDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(5L).build(), 2); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 4); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 3); } } @@ -183,7 +247,7 @@ void testTimeout() throws Exception { harness(manager)) { testHarness.open(); - TableChange event = TableChange.builder().dataFileSize(1).commitNum(1).build(); + TableChange event = TableChange.builder().dataFileCount(1).commitCount(1).build(); // Wait for some time testHarness.processElement(event, EVENT_TIME); @@ -225,7 +289,7 @@ void testStateRestore() throws Exception { testHarness.open(); testHarness.processElement( - TableChange.builder().dataFileSize(1).commitNum(1).build(), EVENT_TIME); + TableChange.builder().dataFileCount(1).commitCount(1).build(), EVENT_TIME); assertThat(testHarness.extractOutputValues()).isEmpty(); @@ -240,7 +304,7 @@ void testStateRestore() throws Exception { testHarness.open(); // Arrives the first real change which triggers the recovery process - testHarness.processElement(TableChange.builder().commitNum(1).build(), EVENT_TIME_2); + testHarness.processElement(TableChange.builder().commitCount(1).build(), EVENT_TIME_2); assertTriggers( testHarness.extractOutputValues(), Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); @@ -261,11 +325,11 @@ void testMinFireDelay() throws Exception { harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); long currentTime = testHarness.getProcessingTime(); // No new fire yet - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); // Check that the trigger fired after the delay testHarness.setProcessingTime(currentTime + DELAY); @@ -281,11 +345,11 @@ void testLockCheckDelay() throws Exception { harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); // Create a lock to prevent execution, and check that there is no result assertThat(lock.tryLock()).isTrue(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); long currentTime = testHarness.getProcessingTime(); // Remove the lock, and still no trigger @@ -331,7 +395,7 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { ++processingTime; expected.add(Trigger.recovery(processingTime)); testHarness.setProcessingTime(processingTime); - testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); assertTriggers(testHarness.extractOutputValues(), expected); // Nothing happens until the recovery is finished @@ -347,7 +411,7 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { // Still no results as the recovery is ongoing ++processingTime; testHarness.setProcessingTime(processingTime); - testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); assertTriggers(testHarness.extractOutputValues(), expected); // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it @@ -383,8 +447,8 @@ void testTriggerMetrics() throws Exception { lockFactory, Lists.newArrayList(NAME_1, NAME_2), Lists.newArrayList( - new TriggerEvaluator.Builder().commitNumber(2).build(), - new TriggerEvaluator.Builder().commitNumber(4).build()), + new TriggerEvaluator.Builder().commitCount(2).build(), + new TriggerEvaluator.Builder().commitCount(4).build()), 1L, 1L); source @@ -400,7 +464,7 @@ void testTriggerMetrics() throws Exception { jobClient = env.executeAsync(); // This one doesn't trigger - tests NOTHING_TO_TRIGGER - source.sendRecord(TableChange.builder().commitNum(1).build()); + source.sendRecord(TableChange.builder().commitCount(1).build()); Awaitility.await() .until( @@ -412,7 +476,7 @@ void testTriggerMetrics() throws Exception { }); // Trigger one of the tasks - tests TRIGGERED - source.sendRecord(TableChange.builder().commitNum(1).build()); + source.sendRecord(TableChange.builder().commitCount(1).build()); // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); assertThat( @@ -421,7 +485,7 @@ void testTriggerMetrics() throws Exception { lock.unlock(); // Trigger both of the tasks - tests TRIGGERED - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); lock.unlock(); @@ -472,14 +536,14 @@ void testRateLimiterMetrics() throws Exception { jobClient = env.executeAsync(); // Start the first trigger - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); // Remove the lock to allow the next trigger lock.unlock(); // The second trigger will be blocked - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); Awaitility.await() .until( () -> @@ -518,11 +582,11 @@ void testConcurrentRunMetrics() throws Exception { jobClient = env.executeAsync(); // Start the first trigger - notice that we do not remove the lock after the trigger - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); // The second trigger will be blocked by the lock - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); Awaitility.await() .until( () -> @@ -589,13 +653,13 @@ private TriggerManager manager( tableLoader, lockFactory, Lists.newArrayList(NAME_1), - Lists.newArrayList(new TriggerEvaluator.Builder().commitNumber(2).build()), + Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), minFireDelayMs, lockCheckDelayMs); } private TriggerManager manager(TableLoader tableLoader) { - return manager(tableLoader, new TriggerEvaluator.Builder().commitNumber(2).build()); + return manager(tableLoader, new TriggerEvaluator.Builder().commitCount(2).build()); } private static void assertTriggers(List expected, List actual) { From 5b3debac7bcf822687a65d183e0a260a1f966d3a Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 23 Aug 2024 17:51:00 +0200 Subject: [PATCH 0628/1019] Flink: Port #10992 to v1.19 (#10994) --- .../maintenance/operator/JdbcLockFactory.java | 14 +- .../maintenance/operator/TableChange.java | 193 ++++++++++++------ .../operator/TriggerEvaluator.java | 85 +++++--- .../operator/TestMonitorSource.java | 27 ++- .../operator/TestTriggerManager.java | 182 +++++++++++------ 5 files changed, 329 insertions(+), 172 deletions(-) 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/operator/JdbcLockFactory.java index 21c8935abede..f22be33aeae6 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/operator/JdbcLockFactory.java @@ -105,12 +105,12 @@ void open(JdbcLockFactory other) { @Override public Lock createLock() { - return new Lock(pool, lockId, Type.MAINTENANCE); + return new JdbcLock(pool, lockId, Type.MAINTENANCE); } @Override public Lock createRecoveryLock() { - return new Lock(pool, lockId, Type.RECOVERY); + return new JdbcLock(pool, lockId, Type.RECOVERY); } @Override @@ -153,12 +153,12 @@ private void initializeLockTables() { } } - public static class Lock implements TriggerLockFactory.Lock { + private static class JdbcLock implements TriggerLockFactory.Lock { private final JdbcClientPool pool; private final String lockId; private final Type type; - public Lock(JdbcClientPool pool, String lockId, Type type) { + private JdbcLock(JdbcClientPool pool, String lockId, Type type) { this.pool = pool; this.lockId = lockId; this.type = type; @@ -221,7 +221,7 @@ public boolean isHeld() { throw new UncheckedInterruptedException(e, "Interrupted during isHeld"); } catch (SQLException e) { // SQL exception happened when getting lock information - throw new UncheckedSQLException(e, "Failed to get lock information for %s", this); + throw new UncheckedSQLException(e, "Failed to check the state of the lock %s", this); } } @@ -266,8 +266,6 @@ public void unlock() { } catch (InterruptedException e) { Thread.currentThread().interrupt(); throw new UncheckedInterruptedException(e, "Interrupted during unlock"); - } catch (UncheckedSQLException e) { - throw e; } catch (SQLException e) { // SQL exception happened when getting/updating lock information throw new UncheckedSQLException(e, "Failed to remove lock %s", this); @@ -312,7 +310,7 @@ private enum Type { MAINTENANCE("m"), RECOVERY("r"); - private String key; + private final String key; Type(String key) { this.key = key; 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 7d0b94e97da7..5252cf61b031 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 @@ -29,19 +29,29 @@ /** Event describing changes in an Iceberg table */ @Internal class TableChange { - private int dataFileNum; - private int deleteFileNum; - private long dataFileSize; - private long deleteFileSize; - private int commitNum; - - private TableChange( - int dataFileNum, int deleteFileNum, long dataFileSize, long deleteFileSize, int commitNum) { - this.dataFileNum = dataFileNum; - this.deleteFileNum = deleteFileNum; - this.dataFileSize = dataFileSize; - this.deleteFileSize = deleteFileSize; - this.commitNum = commitNum; + private int dataFileCount; + private long dataFileSizeInBytes; + private int posDeleteFileCount; + private long posDeleteRecordCount; + private int eqDeleteFileCount; + private long eqDeleteRecordCount; + private int commitCount; + + TableChange( + int dataFileCount, + long dataFileSizeInBytes, + int posDeleteFileCount, + long posDeleteRecordCount, + int eqDeleteFileCount, + long eqDeleteRecordCount, + int commitCount) { + this.dataFileCount = dataFileCount; + this.dataFileSizeInBytes = dataFileSizeInBytes; + this.posDeleteFileCount = posDeleteFileCount; + this.posDeleteRecordCount = posDeleteRecordCount; + this.eqDeleteFileCount = eqDeleteFileCount; + this.eqDeleteRecordCount = eqDeleteRecordCount; + this.commitCount = commitCount; } TableChange(Snapshot snapshot, FileIO io) { @@ -50,67 +60,96 @@ private TableChange( dataFiles.forEach( dataFile -> { - this.dataFileNum++; - this.dataFileSize += dataFile.fileSizeInBytes(); + this.dataFileCount++; + this.dataFileSizeInBytes += dataFile.fileSizeInBytes(); }); deleteFiles.forEach( deleteFile -> { - this.deleteFileNum++; - this.deleteFileSize += deleteFile.fileSizeInBytes(); + switch (deleteFile.content()) { + case POSITION_DELETES: + this.posDeleteFileCount++; + this.posDeleteRecordCount += deleteFile.recordCount(); + break; + case EQUALITY_DELETES: + this.eqDeleteFileCount++; + this.eqDeleteRecordCount += deleteFile.recordCount(); + break; + default: + throw new IllegalArgumentException("Unexpected delete file content: " + deleteFile); + } }); - this.commitNum = 1; + this.commitCount = 1; } static TableChange empty() { - return new TableChange(0, 0, 0L, 0L, 0); + return new TableChange(0, 0L, 0, 0L, 0, 0L, 0); } static Builder builder() { return new Builder(); } - int dataFileNum() { - return dataFileNum; + int dataFileCount() { + return dataFileCount; + } + + long dataFileSizeInBytes() { + return dataFileSizeInBytes; } - int deleteFileNum() { - return deleteFileNum; + int posDeleteFileCount() { + return posDeleteFileCount; } - long dataFileSize() { - return dataFileSize; + long posDeleteRecordCount() { + return posDeleteRecordCount; } - long deleteFileSize() { - return deleteFileSize; + int eqDeleteFileCount() { + return eqDeleteFileCount; } - public int commitNum() { - return commitNum; + long eqDeleteRecordCount() { + return eqDeleteRecordCount; + } + + public int commitCount() { + return commitCount; } public void merge(TableChange other) { - this.dataFileNum += other.dataFileNum; - this.deleteFileNum += other.deleteFileNum; - this.dataFileSize += other.dataFileSize; - this.deleteFileSize += other.deleteFileSize; - this.commitNum += other.commitNum; + this.dataFileCount += other.dataFileCount; + this.dataFileSizeInBytes += other.dataFileSizeInBytes; + this.posDeleteFileCount += other.posDeleteFileCount; + this.posDeleteRecordCount += other.posDeleteRecordCount; + this.eqDeleteFileCount += other.eqDeleteFileCount; + this.eqDeleteRecordCount += other.eqDeleteRecordCount; + this.commitCount += other.commitCount; } TableChange copy() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("dataFileNum", dataFileNum) - .add("deleteFileNum", deleteFileNum) - .add("dataFileSize", dataFileSize) - .add("deleteFileSize", deleteFileSize) - .add("commitNum", commitNum) + .add("dataFileCount", dataFileCount) + .add("dataFileSizeInBytes", dataFileSizeInBytes) + .add("posDeleteFileCount", posDeleteFileCount) + .add("posDeleteRecordCount", posDeleteRecordCount) + .add("eqDeleteFileCount", eqDeleteFileCount) + .add("eqDeleteRecordCount", eqDeleteRecordCount) + .add("commitCount", commitCount) .toString(); } @@ -123,52 +162,80 @@ public boolean equals(Object other) { } TableChange that = (TableChange) other; - return dataFileNum == that.dataFileNum - && deleteFileNum == that.deleteFileNum - && dataFileSize == that.dataFileSize - && deleteFileSize == that.deleteFileSize - && commitNum == that.commitNum; + return dataFileCount == that.dataFileCount + && dataFileSizeInBytes == that.dataFileSizeInBytes + && posDeleteFileCount == that.posDeleteFileCount + && posDeleteRecordCount == that.posDeleteRecordCount + && eqDeleteFileCount == that.eqDeleteFileCount + && eqDeleteRecordCount == that.eqDeleteRecordCount + && commitCount == that.commitCount; } @Override public int hashCode() { - return Objects.hash(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return Objects.hash( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } static class Builder { - private int dataFileNum = 0; - private int deleteFileNum = 0; - private long dataFileSize = 0L; - private long deleteFileSize = 0L; - private int commitNum = 0; - - public Builder dataFileNum(int newDataFileNum) { - this.dataFileNum = newDataFileNum; + private int dataFileCount = 0; + private long dataFileSizeInBytes = 0L; + private int posDeleteFileCount = 0; + private long posDeleteRecordCount = 0L; + private int eqDeleteFileCount = 0; + private long eqDeleteRecordCount = 0L; + private int commitCount = 0; + + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; + return this; + } + + public Builder dataFileSizeInBytes(long newDataFileSizeInBytes) { + this.dataFileSizeInBytes = newDataFileSizeInBytes; + return this; + } + + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; return this; } - public Builder deleteFileNum(int newDeleteFileNum) { - this.deleteFileNum = newDeleteFileNum; + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; return this; } - public Builder dataFileSize(long newDataFileSize) { - this.dataFileSize = newDataFileSize; + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; return this; } - public Builder deleteFileSize(long newDeleteFileSize) { - this.deleteFileSize = newDeleteFileSize; + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; return this; } - public Builder commitNum(int newCommitNum) { - this.commitNum = newCommitNum; + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; return this; } public TableChange build() { - return new TableChange(dataFileNum, deleteFileNum, dataFileSize, deleteFileSize, commitNum); + return new TableChange( + dataFileCount, + dataFileSizeInBytes, + posDeleteFileCount, + posDeleteRecordCount, + eqDeleteFileCount, + eqDeleteRecordCount, + commitCount); } } } 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 37e4e3afd4bd..dba33b22a42a 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 @@ -40,15 +40,7 @@ private TriggerEvaluator(List predicates) { boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { boolean result = - predicates.stream() - .anyMatch( - p -> { - try { - return p.evaluate(event, lastTimeMs, currentTimeMs); - } catch (Exception e) { - throw new RuntimeException("Error accessing state", e); - } - }); + predicates.stream().anyMatch(p -> p.evaluate(event, lastTimeMs, currentTimeMs)); LOG.debug( "Checking event: {}, at {}, last: {} with result: {}", event, @@ -59,29 +51,47 @@ boolean check(TableChange event, long lastTimeMs, long currentTimeMs) { } static class Builder implements Serializable { - private Integer commitNumber; - private Integer fileNumber; - private Long fileSize; - private Integer deleteFileNumber; + private Integer dataFileCount; + private Long dataFileSizeInBytes; + private Integer posDeleteFileCount; + private Long posDeleteRecordCount; + private Integer eqDeleteFileCount; + private Long eqDeleteRecordCount; + private Integer commitCount; private Duration timeout; - Builder commitNumber(int newCommitNumber) { - this.commitNumber = newCommitNumber; + public Builder dataFileCount(int newDataFileCount) { + this.dataFileCount = newDataFileCount; return this; } - Builder fileNumber(int newFileNumber) { - this.fileNumber = newFileNumber; + public Builder dataFileSizeInBytes(long neDataFileSizeInBytes) { + this.dataFileSizeInBytes = neDataFileSizeInBytes; return this; } - Builder fileSize(long newFileSize) { - this.fileSize = newFileSize; + public Builder posDeleteFileCount(int newPosDeleteFileCount) { + this.posDeleteFileCount = newPosDeleteFileCount; return this; } - Builder deleteFileNumber(int newDeleteFileNumber) { - this.deleteFileNumber = newDeleteFileNumber; + public Builder posDeleteRecordCount(long newPosDeleteRecordCount) { + this.posDeleteRecordCount = newPosDeleteRecordCount; + return this; + } + + public Builder eqDeleteFileCount(int newEqDeleteFileCount) { + this.eqDeleteFileCount = newEqDeleteFileCount; + return this; + } + + public Builder eqDeleteRecordCount(long newEqDeleteRecordCount) { + this.eqDeleteRecordCount = newEqDeleteRecordCount; + return this; + } + + public Builder commitCount(int newCommitCount) { + this.commitCount = newCommitCount; return this; } @@ -92,24 +102,37 @@ Builder timeout(Duration newTimeout) { TriggerEvaluator build() { List predicates = Lists.newArrayList(); - if (commitNumber != null) { - predicates.add((change, unused, unused2) -> change.commitNum() >= commitNumber); + if (dataFileCount != null) { + predicates.add((change, unused, unused2) -> change.dataFileCount() >= dataFileCount); + } + + if (dataFileSizeInBytes != null) { + predicates.add( + (change, unused, unused2) -> change.dataFileSizeInBytes() >= dataFileSizeInBytes); + } + + if (posDeleteFileCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteFileCount() >= posDeleteFileCount); + } + + if (posDeleteRecordCount != null) { + predicates.add( + (change, unused, unused2) -> change.posDeleteRecordCount() >= posDeleteRecordCount); } - if (fileNumber != null) { + if (eqDeleteFileCount != null) { predicates.add( - (change, unused, unused2) -> - change.dataFileNum() + change.deleteFileNum() >= fileNumber); + (change, unused, unused2) -> change.eqDeleteFileCount() >= eqDeleteFileCount); } - if (fileSize != null) { + if (eqDeleteRecordCount != null) { predicates.add( - (change, unused, unused2) -> - change.dataFileSize() + change.deleteFileSize() >= fileSize); + (change, unused, unused2) -> change.eqDeleteRecordCount() >= eqDeleteRecordCount); } - if (deleteFileNumber != null) { - predicates.add((change, unused, unused2) -> change.deleteFileNum() >= deleteFileNumber); + if (commitCount != null) { + predicates.add((change, unused, unused2) -> change.commitCount() >= commitCount); } if (timeout != null) { 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 8c02601025ae..3aee05322561 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 @@ -162,7 +162,11 @@ void testSource() throws Exception { // The first non-empty event should contain the expected value return newEvent.equals( - TableChange.builder().dataFileNum(1).dataFileSize(size).commitNum(1).build()); + TableChange.builder() + .dataFileCount(1) + .dataFileSizeInBytes(size) + .commitCount(1) + .build()); }); } finally { closeJobClient(jobClient); @@ -298,17 +302,17 @@ void testMaxReadBack() { new MonitorSource.TableChangeIterator(tableLoader, null, 1); // For a single maxReadBack we only get a single change - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, 2); // Expecting 2 commits/snapshots for maxReadBack=2 - assertThat(iterator.next().commitNum()).isEqualTo(2); + assertThat(iterator.next().commitCount()).isEqualTo(2); iterator = new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // For maxReadBack Long.MAX_VALUE we get every change - assertThat(iterator.next().commitNum()).isEqualTo(3); + assertThat(iterator.next().commitCount()).isEqualTo(3); } @Test @@ -323,7 +327,7 @@ void testSkipReplace() { new MonitorSource.TableChangeIterator(tableLoader, null, Long.MAX_VALUE); // Read the current snapshot - assertThat(iterator.next().commitNum()).isEqualTo(1); + assertThat(iterator.next().commitCount()).isEqualTo(1); // Create a DataOperations.REPLACE snapshot Table table = tableLoader.loadTable(); @@ -350,16 +354,17 @@ private static TableChange tableChangeWithLastSnapshot(Table table, TableChange Lists.newArrayList(table.currentSnapshot().addedDeleteFiles(table.io()).iterator()); long dataSize = dataFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); - long deleteSize = deleteFiles.stream().mapToLong(ContentFile::fileSizeInBytes).sum(); + long deleteRecordCount = deleteFiles.stream().mapToLong(DeleteFile::recordCount).sum(); TableChange newChange = previous.copy(); newChange.merge( TableChange.builder() - .dataFileNum(dataFiles.size()) - .dataFileSize(dataSize) - .deleteFileNum(deleteFiles.size()) - .deleteFileSize(deleteSize) - .commitNum(1) + .dataFileCount(dataFiles.size()) + .dataFileSizeInBytes(dataSize) + // Currently we only test with equality deletes + .eqDeleteFileCount(deleteFiles.size()) + .eqDeleteRecordCount(deleteRecordCount) + .commitCount(1) .build()); return newChange; } 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 55e64f3e84bf..fba4a12d9c6b 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 @@ -81,95 +81,159 @@ void after() throws IOException { } @Test - void testCommitNumber() throws Exception { + void testCommitCount() throws Exception { + TriggerManager manager = + manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(1).build(), 4); + } + } + + @Test + void testDataFileCount() throws Exception { TriggerManager manager = manager( - sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().commitNumber(3).build()); + sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().dataFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 0); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(3).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(10).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 0); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(5).build(), 3); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(1).build(), 4); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileCount(2).build(), 4); } } @Test - void testFileNumber() throws Exception { + void testDataFileSizeInBytes() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileNumber(3).build()); + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().dataFileSizeInBytes(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(5L).build(), 2); - addEventAndCheckResult( - testHarness, TableChange.builder().dataFileNum(1).deleteFileNum(1).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); - addEventAndCheckResult( - testHarness, TableChange.builder().dataFileNum(5).deleteFileNum(7).build(), 3); + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(1L).build(), 2); + + addEventAndCheckResult(testHarness, TableChange.builder().dataFileSizeInBytes(2L).build(), 3); + } + } + + @Test + void testPosDeleteFileCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteFileCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(10).build(), 3); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileNum(1).build(), 4); + addEventAndCheckResult(testHarness, TableChange.builder().posDeleteFileCount(1).build(), 4); } } @Test - void testFileSize() throws Exception { + void testPosDeleteRecordCount() throws Exception { TriggerManager manager = - manager(sql.tableLoader(TABLE_NAME), new TriggerEvaluator.Builder().fileSize(3).build()); + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().posDeleteRecordCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 0); addEventAndCheckResult( - testHarness, TableChange.builder().dataFileSize(1L).deleteFileSize(1L).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(3L).build(), 2); + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 0); addEventAndCheckResult( - testHarness, TableChange.builder().dataFileSize(5L).deleteFileSize(7L).build(), 3); + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(5L).build(), 2); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileSize(1L).build(), 3); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(1L).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().dataFileSize(1L).build(), 4); + addEventAndCheckResult( + testHarness, TableChange.builder().posDeleteRecordCount(2L).build(), 3); } } @Test - void testDeleteFileNumber() throws Exception { + void testEqDeleteFileCount() throws Exception { TriggerManager manager = manager( sql.tableLoader(TABLE_NAME), - new TriggerEvaluator.Builder().deleteFileNumber(3).build()); + new TriggerEvaluator.Builder().eqDeleteFileCount(3).build()); try (KeyedOneInputStreamOperatorTestHarness testHarness = harness(manager)) { testHarness.open(); - addEventAndCheckResult( - testHarness, TableChange.builder().dataFileNum(3).deleteFileNum(1).build(), 0); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(2).build(), 1); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(3).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(10).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(3).build(), 2); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(10).build(), 3); + + // No trigger in this case + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 3); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteFileCount(1).build(), 4); + } + } + + @Test + void testEqDeleteRecordCount() throws Exception { + TriggerManager manager = + manager( + sql.tableLoader(TABLE_NAME), + new TriggerEvaluator.Builder().eqDeleteRecordCount(3).build()); + try (KeyedOneInputStreamOperatorTestHarness testHarness = + harness(manager)) { + testHarness.open(); + + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 0); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(5L).build(), 2); // No trigger in this case - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 3); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(1L).build(), 2); - addEventAndCheckResult(testHarness, TableChange.builder().deleteFileNum(1).build(), 4); + addEventAndCheckResult(testHarness, TableChange.builder().eqDeleteRecordCount(2L).build(), 3); } } @@ -183,7 +247,7 @@ void testTimeout() throws Exception { harness(manager)) { testHarness.open(); - TableChange event = TableChange.builder().dataFileSize(1).commitNum(1).build(); + TableChange event = TableChange.builder().dataFileCount(1).commitCount(1).build(); // Wait for some time testHarness.processElement(event, EVENT_TIME); @@ -225,7 +289,7 @@ void testStateRestore() throws Exception { testHarness.open(); testHarness.processElement( - TableChange.builder().dataFileSize(1).commitNum(1).build(), EVENT_TIME); + TableChange.builder().dataFileCount(1).commitCount(1).build(), EVENT_TIME); assertThat(testHarness.extractOutputValues()).isEmpty(); @@ -240,7 +304,7 @@ void testStateRestore() throws Exception { testHarness.open(); // Arrives the first real change which triggers the recovery process - testHarness.processElement(TableChange.builder().commitNum(1).build(), EVENT_TIME_2); + testHarness.processElement(TableChange.builder().commitCount(1).build(), EVENT_TIME_2); assertTriggers( testHarness.extractOutputValues(), Lists.newArrayList(Trigger.recovery(testHarness.getProcessingTime()))); @@ -261,11 +325,11 @@ void testMinFireDelay() throws Exception { harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); long currentTime = testHarness.getProcessingTime(); // No new fire yet - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); // Check that the trigger fired after the delay testHarness.setProcessingTime(currentTime + DELAY); @@ -281,11 +345,11 @@ void testLockCheckDelay() throws Exception { harness(manager)) { testHarness.open(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); // Create a lock to prevent execution, and check that there is no result assertThat(lock.tryLock()).isTrue(); - addEventAndCheckResult(testHarness, TableChange.builder().commitNum(2).build(), 1); + addEventAndCheckResult(testHarness, TableChange.builder().commitCount(2).build(), 1); long currentTime = testHarness.getProcessingTime(); // Remove the lock, and still no trigger @@ -331,7 +395,7 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { ++processingTime; expected.add(Trigger.recovery(processingTime)); testHarness.setProcessingTime(processingTime); - testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); assertTriggers(testHarness.extractOutputValues(), expected); // Nothing happens until the recovery is finished @@ -347,7 +411,7 @@ void testRecovery(boolean locked, boolean runningTask) throws Exception { // Still no results as the recovery is ongoing ++processingTime; testHarness.setProcessingTime(processingTime); - testHarness.processElement(TableChange.builder().commitNum(2).build(), processingTime); + testHarness.processElement(TableChange.builder().commitCount(2).build(), processingTime); assertTriggers(testHarness.extractOutputValues(), expected); // Simulate the action of removing lock and recoveryLock by downstream lock cleaner when it @@ -383,8 +447,8 @@ void testTriggerMetrics() throws Exception { lockFactory, Lists.newArrayList(NAME_1, NAME_2), Lists.newArrayList( - new TriggerEvaluator.Builder().commitNumber(2).build(), - new TriggerEvaluator.Builder().commitNumber(4).build()), + new TriggerEvaluator.Builder().commitCount(2).build(), + new TriggerEvaluator.Builder().commitCount(4).build()), 1L, 1L); source @@ -400,7 +464,7 @@ void testTriggerMetrics() throws Exception { jobClient = env.executeAsync(); // This one doesn't trigger - tests NOTHING_TO_TRIGGER - source.sendRecord(TableChange.builder().commitNum(1).build()); + source.sendRecord(TableChange.builder().commitCount(1).build()); Awaitility.await() .until( @@ -412,7 +476,7 @@ void testTriggerMetrics() throws Exception { }); // Trigger one of the tasks - tests TRIGGERED - source.sendRecord(TableChange.builder().commitNum(1).build()); + source.sendRecord(TableChange.builder().commitCount(1).build()); // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); assertThat( @@ -421,7 +485,7 @@ void testTriggerMetrics() throws Exception { lock.unlock(); // Trigger both of the tasks - tests TRIGGERED - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); // Wait until we receive the trigger assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); lock.unlock(); @@ -472,14 +536,14 @@ void testRateLimiterMetrics() throws Exception { jobClient = env.executeAsync(); // Start the first trigger - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); // Remove the lock to allow the next trigger lock.unlock(); // The second trigger will be blocked - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); Awaitility.await() .until( () -> @@ -518,11 +582,11 @@ void testConcurrentRunMetrics() throws Exception { jobClient = env.executeAsync(); // Start the first trigger - notice that we do not remove the lock after the trigger - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); assertThat(sink.poll(Duration.ofSeconds(5))).isNotNull(); // The second trigger will be blocked by the lock - source.sendRecord(TableChange.builder().commitNum(2).build()); + source.sendRecord(TableChange.builder().commitCount(2).build()); Awaitility.await() .until( () -> @@ -589,13 +653,13 @@ private TriggerManager manager( tableLoader, lockFactory, Lists.newArrayList(NAME_1), - Lists.newArrayList(new TriggerEvaluator.Builder().commitNumber(2).build()), + Lists.newArrayList(new TriggerEvaluator.Builder().commitCount(2).build()), minFireDelayMs, lockCheckDelayMs); } private TriggerManager manager(TableLoader tableLoader) { - return manager(tableLoader, new TriggerEvaluator.Builder().commitNumber(2).build()); + return manager(tableLoader, new TriggerEvaluator.Builder().commitCount(2).build()); } private static void assertTriggers(List expected, List actual) { From 90825d90119915b57a8c3a4276701e088c2c87b9 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 23 Aug 2024 20:26:29 +0200 Subject: [PATCH 0629/1019] OpenAPI: Add endpoint field to CatalogConfig (#10928) --- open-api/rest-catalog-open-api.py | 11 ++++++ open-api/rest-catalog-open-api.yaml | 54 ++++++++++++++++++++++++++++- 2 files changed, 64 insertions(+), 1 deletion(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 56212f1ac0b5..cc89207b9f2a 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -54,6 +54,17 @@ class CatalogConfig(BaseModel): ..., description='Properties that should be used as default configuration; applied before client configuration.', ) + endpoints: Optional[List[str]] = Field( + None, + description='A list of endpoints that the server supports. The format of each endpoint must be " ". The HTTP verb and the resource path must be separated by a space character.', + example=[ + 'GET /v1/{prefix}/namespaces/{namespace}', + 'GET /v1/{prefix}/namespaces', + 'POST /v1/{prefix}/namespaces', + 'GET /v1/{prefix}/namespaces/{namespace}/tables/{table}', + 'GET /v1/{prefix}/namespaces/{namespace}/views/{view}', + ], + ) class UpdateNamespacePropertiesRequest(BaseModel): diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 7f0619aff4f9..6236b8d45086 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -100,6 +100,38 @@ paths: Common catalog configuration settings are documented at https://iceberg.apache.org/docs/latest/configuration/#catalog-properties + + + The catalog configuration also holds an optional `endpoints` field that contains information about the endpoints + supported by the server. If a server does not send the `endpoints` field, a default set of endpoints is assumed: + + - GET /v1/{prefix}/namespaces + + - POST /v1/{prefix}/namespaces + + - GET /v1/{prefix}/namespaces/{namespace} + + - DELETE /v1/{prefix}/namespaces/{namespace} + + - POST /v1/{prefix}/namespaces/{namespace}/properties + + - GET /v1/{prefix}/namespaces/{namespace}/tables + + - POST /v1/{prefix}/namespaces/{namespace}/tables + + - GET /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - POST /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - DELETE /v1/{prefix}/namespaces/{namespace}/tables/{table} + + - POST /v1/{prefix}/namespaces/{namespace}/register + + - POST /v1/{prefix}/namespaces/{namespace}/tables/{table}/metrics + + - POST /v1/{prefix}/tables/rename + + - POST /v1/{prefix}/transactions/commit " responses: 200: @@ -114,7 +146,14 @@ paths: }, "defaults": { "clients": "4" - } + }, + "endpoints": [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] } 400: $ref: '#/components/responses/BadRequestErrorResponse' @@ -1583,6 +1622,19 @@ components: type: string description: Properties that should be used as default configuration; applied before client configuration. + endpoints: + type: array + items: + type: string + description: A list of endpoints that the server supports. The format of each endpoint must be " ". + The HTTP verb and the resource path must be separated by a space character. + example: [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] CreateNamespaceRequest: type: object From 8636c17ba8387ffe11d9398c740387cff96545e4 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 23 Aug 2024 17:31:20 -0700 Subject: [PATCH 0630/1019] Spark 3.5: Add utility to load table state reliably (#10984) --- .../apache/iceberg/spark/SparkTableUtil.java | 19 +++++++++++++++---- .../iceberg/spark/actions/NDVSketchUtil.java | 12 ++++-------- .../actions/TestComputeTableStatsAction.java | 14 ++++++++++++++ 3 files changed, 33 insertions(+), 12 deletions(-) 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 7a96e97fb98a..e103104171f6 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 @@ -851,6 +851,12 @@ private static void deleteManifests(FileIO io, List manifests) { .run(item -> io.deleteFile(item.path())); } + public static Dataset loadTable(SparkSession spark, Table table, long snapshotId) { + SparkTable sparkTable = new SparkTable(table, snapshotId, false); + DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); + return Dataset.ofRows(spark, relation); + } + public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type) { return loadMetadataTable(spark, table, type, ImmutableMap.of()); @@ -858,11 +864,16 @@ public static Dataset loadMetadataTable( public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type, Map extraOptions) { - SparkTable metadataTable = - new SparkTable(MetadataTableUtils.createMetadataTableInstance(table, type), false); + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); + return Dataset.ofRows(spark, relation); + } + + private static DataSourceV2Relation createRelation( + SparkTable sparkTable, Map extraOptions) { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); - return Dataset.ofRows( - spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); + return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); } /** diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java index 22055a161e4e..c8a20d3cca73 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -32,9 +32,10 @@ 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.SparkReadOptions; +import org.apache.iceberg.spark.SparkTableUtil; 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.stats.ThetaSketchAgg; @@ -73,13 +74,8 @@ private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snap private static Row computeNDVSketches( SparkSession spark, Table table, Snapshot snapshot, List colNames) { - return spark - .read() - .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshot.snapshotId()) - .load(table.name()) - .select(toAggColumns(colNames)) - .first(); + Dataset inputDF = SparkTableUtil.loadTable(spark, table, snapshot.snapshotId()); + return inputDF.select(toAggColumns(colNames)).first(); } private static Column[] toAggColumns(List colNames) { 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 588bb29f47dd..4cab0d42815c 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 @@ -79,6 +79,20 @@ public class TestComputeTableStatsAction extends CatalogTestBase { required(4, "nestedStructCol", NESTED_STRUCT_TYPE), required(5, "stringCol", Types.StringType.get())); + @TestTemplate + public void testLoadingTableDirectly() { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("INSERT into %s values(1, 'abcd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + StatisticsFile statisticsFile = results.statisticsFile(); + Assertions.assertNotEquals(statisticsFile.fileSizeInBytes(), 0); + Assertions.assertEquals(statisticsFile.blobMetadata().size(), 2); + } + @TestTemplate public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); From 9f87ecf51debdbdd7e9b8f30525e6d9d562efc5e Mon Sep 17 00:00:00 2001 From: Akira Ajisaka Date: Sun, 25 Aug 2024 11:47:35 +0900 Subject: [PATCH 0631/1019] AWS: Include http-auth-aws-crt module into iceberg-aws-bundle (#10972) --- aws-bundle/LICENSE | 140 +++++++++++++----- aws-bundle/NOTICE | 67 +++++---- aws-bundle/build.gradle | 1 + .../apache/iceberg/aws/AwsIntegTestUtil.java | 11 ++ .../aws/s3/TestS3FileIOIntegration.java | 90 ++++++++--- build.gradle | 1 + kafka-connect/build.gradle | 1 + 7 files changed, 221 insertions(+), 90 deletions(-) diff --git a/aws-bundle/LICENSE b/aws-bundle/LICENSE index a4ca14bf8f3e..0a9656c2c7c9 100644 --- a/aws-bundle/LICENSE +++ b/aws-bundle/LICENSE @@ -207,7 +207,7 @@ This binary artifact contains code from the following projects: -------------------------------------------------------------------------------- -Group: commons-codec Name: commons-codec Version: 1.15 +Group: commons-codec Name: commons-codec Version: 1.17.1 Project URL: https://commons.apache.org/proper/commons-codec/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0.txt @@ -219,61 +219,61 @@ License: The Apache Software License, Version 2.0 - http://www.apache.org/licens -------------------------------------------------------------------------------- -Group: io.netty Name: netty-buffer Version: 4.1.86.Final +Group: io.netty Name: netty-buffer Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec Version: 4.1.86.Final +Group: io.netty Name: netty-codec Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec-http Version: 4.1.86.Final +Group: io.netty Name: netty-codec-http Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-codec-http2 Version: 4.1.86.Final +Group: io.netty Name: netty-codec-http2 Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-common Version: 4.1.86.Final +Group: io.netty Name: netty-common Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-handler Version: 4.1.86.Final +Group: io.netty Name: netty-handler Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-resolver Version: 4.1.86.Final +Group: io.netty Name: netty-resolver Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport Version: 4.1.86.Final +Group: io.netty Name: netty-transport Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 -------------------------------------------------------------------------------- -Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.86.Final +Group: io.netty Name: netty-transport-classes-epoll Version: 4.1.112.Final Project URL: https://netty.io/ License: 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.86.Final +Group: io.netty Name: netty-transport-native-unix-common Version: 4.1.112.Final Project URL: https://netty.io/ License: Apache License, Version 2.0 - https://www.apache.org/licenses/LICENSE-2.0 @@ -285,13 +285,13 @@ License: Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2. -------------------------------------------------------------------------------- -Group: org.apache.httpcomponents Name: httpcore Version: 4.4.13 +Group: org.apache.httpcomponents Name: httpcore Version: 4.4.16 Project URL: http://hc.apache.org/httpcomponents-core-ga License: Apache License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt -------------------------------------------------------------------------------- -Group: org.reactivestreams Name: reactive-streams Version: 1.0.3 +Group: org.reactivestreams Name: reactive-streams Version: 1.0.4 Project URL: http://reactive-streams.org License: CC0 - http://creativecommons.org/publicdomain/zero/1.0/ @@ -303,160 +303,220 @@ License: MIT License - http://www.opensource.org/licenses/mit-license.php -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: annotations Version: 2.20.131 +Group: software.amazon.awssdk Name: annotations Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: apache-client Version: 2.20.131 +Group: software.amazon.awssdk Name: apache-client Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: arns Version: 2.20.131 +Group: software.amazon.awssdk Name: arns Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: auth Version: 2.20.131 +Group: software.amazon.awssdk Name: auth Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-core Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-core Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.20.131 +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: dynamodb Version: 2.20.131 +Group: software.amazon.awssdk Name: checksums Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: endpoints-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: checksums-spi Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: glue Version: 2.20.131 +Group: software.amazon.awssdk Name: crt-core Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-client-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: dynamodb Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: iam Version: 2.20.131 +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: glue Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: json-utils Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: kms Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: lakeformation Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: metrics-spi Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: netty-nio-client Version: 2.20.131 +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: http-client-spi Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: profiles Version: 2.20.131 +Group: software.amazon.awssdk Name: iam Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: protocol-core Version: 2.20.131 +Group: software.amazon.awssdk Name: identity-spi Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: regions Version: 2.20.131 +Group: software.amazon.awssdk Name: json-utils Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: s3 Version: 2.20.131 +Group: software.amazon.awssdk Name: kms Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sdk-core Version: 2.20.131 +Group: software.amazon.awssdk Name: lakeformation Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sso Version: 2.20.131 +Group: software.amazon.awssdk Name: metrics-spi Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: profiles Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sts Version: 2.20.131 +Group: software.amazon.awssdk Name: protocol-core Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.20.131 +Group: software.amazon.awssdk Name: regions Version: 2.27.7 +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: s3 Version: 2.27.7 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: utils Version: 2.20.131 +Group: software.amazon.awssdk Name: sdk-core Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sso Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: sts Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.27.7 +Project URL: https://aws.amazon.com/sdkforjava +License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 + +-------------------------------------------------------------------------------- + +Group: software.amazon.awssdk Name: utils Version: 2.27.7 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- +Group: software.amazon.awssdk.crt Name: aws-crt Version: 0.30.6 +Project URL: https://github.com/awslabs/aws-crt-java +License: The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt + +-------------------------------------------------------------------------------- + Group: software.amazon.eventstream Name: eventstream Version: 1.0.1 Project URL: https://github.com/awslabs/aws-eventstream-java License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 diff --git a/aws-bundle/NOTICE b/aws-bundle/NOTICE index ed353940ba38..070d109c8e5f 100644 --- a/aws-bundle/NOTICE +++ b/aws-bundle/NOTICE @@ -7,7 +7,7 @@ The Apache Software Foundation (http://www.apache.org/). -------------------------------------------------------------------------------- -NOTICE for Group: commons-codec Name: commons-codec Version: 1.15 +NOTICE for Group: commons-codec Name: commons-codec Version: 1.17.1 src/test/org/apache/commons/codec/language/DoubleMetaphoneTest.java contains test data from http://aspell.net/test/orig/batch0.tab. @@ -23,32 +23,43 @@ Copyright (c) 2008 Alexander Beider & Stephen P. Morse. -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.20.131 -NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.20.131 +NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: checksums Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: checksums-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: crt-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-auth-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: identity-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: retries Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: retries-spi Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.27.7 AWS SDK for Java 2.0 Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. @@ -71,7 +82,7 @@ The licenses for these third party components are included in LICENSE.txt -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.20.18 +NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.27.7 # Jackson JSON processor diff --git a/aws-bundle/build.gradle b/aws-bundle/build.gradle index eea591a7b76d..16952d78d8e5 100644 --- a/aws-bundle/build.gradle +++ b/aws-bundle/build.gradle @@ -27,6 +27,7 @@ project(":iceberg-aws-bundle") { implementation platform(libs.awssdk.bom) implementation "software.amazon.awssdk:apache-client" implementation "software.amazon.awssdk:auth" + implementation "software.amazon.awssdk:http-auth-aws-crt" implementation "software.amazon.awssdk:iam" implementation "software.amazon.awssdk:sso" implementation "software.amazon.awssdk:s3" 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 7e0ca6ed10b2..e9cf474addfa 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/AwsIntegTestUtil.java @@ -95,6 +95,17 @@ public static String testAccountId() { return System.getenv("AWS_TEST_ACCOUNT_ID"); } + /** + * Set the environment variable AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS for a default account to + * use for testing. Developers need to create a S3 multi region access point before running + * integration tests because creating it takes a few minutes + * + * @return The alias of S3 multi region access point route to the default S3 bucket + */ + public static String testMultiRegionAccessPointAlias() { + return System.getenv("AWS_TEST_MULTI_REGION_ACCESS_POINT_ALIAS"); + } + public static void cleanS3Bucket(S3Client s3, String bucketName, String prefix) { ListObjectVersionsIterable response = s3.listObjectVersionsPaginator( 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 cacf04891896..fc2b9f97f1e6 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 @@ -41,6 +41,7 @@ import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.assertj.core.api.Assumptions; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; @@ -75,11 +76,13 @@ public class TestS3FileIOIntegration { private static S3Client s3; private static S3ControlClient s3Control; private static S3ControlClient crossRegionS3Control; + private static S3ControlClient multiRegionS3Control; private static KmsClient kms; private static String bucketName; private static String crossRegionBucketName; private static String accessPointName; private static String crossRegionAccessPointName; + private static String multiRegionAccessPointAlias; private static String prefix; private static byte[] contentBytes; private static String content; @@ -109,6 +112,7 @@ public static void beforeClass() { AwsIntegTestUtil.createAccessPoint(s3Control, accessPointName, bucketName); AwsIntegTestUtil.createAccessPoint( crossRegionS3Control, crossRegionAccessPointName, crossRegionBucketName); + multiRegionAccessPointAlias = AwsIntegTestUtil.testMultiRegionAccessPointAlias(); s3.putBucketVersioning( PutBucketVersioningRequest.builder() .bucket(bucketName) @@ -202,15 +206,32 @@ public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception { validateRead(s3FileIO); } + @Test + public void testNewInputStreamWithMultiRegionAccessPoint() throws Exception { + Assumptions.assumeThat(multiRegionAccessPointAlias).isNotEmpty(); + clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); + S3Client s3Client = clientFactory.s3(); + s3Client.putObject( + PutObjectRequest.builder().bucket(bucketName).key(objectKey).build(), + RequestBody.fromBytes(contentBytes)); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + s3FileIO.initialize( + ImmutableMap.of( + S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, + testMultiRegionAccessPointARN( + AwsIntegTestUtil.testRegion(), multiRegionAccessPointAlias))); + validateRead(s3FileIO); + } + @Test public void testNewOutputStream() throws Exception { S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); write(s3FileIO); - InputStream stream = - s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -221,11 +242,11 @@ public void testNewOutputStreamWithAccessPoint() throws Exception { S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, testAccessPointARN(AwsIntegTestUtil.testRegion(), accessPointName))); write(s3FileIO); - InputStream stream = - s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -238,17 +259,35 @@ public void testNewOutputStreamWithCrossRegionAccessPoint() throws Exception { S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, testAccessPointARN(AwsIntegTestUtil.testCrossRegion(), crossRegionAccessPointName))); write(s3FileIO); - InputStream stream = + try (InputStream stream = s3Client.getObject( GetObjectRequest.builder() .bucket( testAccessPointARN( AwsIntegTestUtil.testCrossRegion(), crossRegionAccessPointName)) .key(objectKey) - .build()); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + .build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } + } + + @Test + public void testNewOutputStreamWithMultiRegionAccessPoint() throws Exception { + Assumptions.assumeThat(multiRegionAccessPointAlias).isNotEmpty(); + clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + s3FileIO.initialize( + ImmutableMap.of( + S3FileIOProperties.ACCESS_POINTS_PREFIX + bucketName, + testMultiRegionAccessPointARN( + AwsIntegTestUtil.testRegion(), multiRegionAccessPointAlias))); + write(s3FileIO); + try (InputStream stream = + s3.getObject(GetObjectRequest.builder().bucket(bucketName).key(objectKey).build())) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } @Test @@ -508,18 +547,18 @@ private void write(S3FileIO s3FileIO) throws Exception { private void write(S3FileIO s3FileIO, String uri) throws Exception { OutputFile outputFile = s3FileIO.newOutputFile(uri); - OutputStream outputStream = outputFile.create(); - IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream); - outputStream.close(); + try (OutputStream outputStream = outputFile.create()) { + IoUtils.copy(new ByteArrayInputStream(contentBytes), outputStream); + } } private void validateRead(S3FileIO s3FileIO) throws Exception { InputFile file = s3FileIO.newInputFile(objectUri); assertThat(file.getLength()).isEqualTo(contentBytes.length); - InputStream stream = file.newStream(); - String result = IoUtils.toUtf8String(stream); - stream.close(); - assertThat(result).isEqualTo(content); + try (InputStream stream = file.newStream()) { + String result = IoUtils.toUtf8String(stream); + assertThat(result).isEqualTo(content); + } } private String testAccessPointARN(String region, String accessPoint) { @@ -532,6 +571,13 @@ private String testAccessPointARN(String region, String accessPoint) { accessPoint); } + private String testMultiRegionAccessPointARN(String region, String alias) { + // format: arn:aws:s3::account-id:accesspoint/MultiRegionAccessPoint_alias + return String.format( + "arn:%s:s3::%s:accesspoint/%s", + PartitionMetadata.of(Region.of(region)).id(), AwsIntegTestUtil.testAccountId(), alias); + } + private void createRandomObjects(String objectPrefix, int count) { S3URI s3URI = new S3URI(objectPrefix); random diff --git a/build.gradle b/build.gradle index 7a11943cf8be..3b89dc4e05ad 100644 --- a/build.gradle +++ b/build.gradle @@ -471,6 +471,7 @@ project(':iceberg-aws') { compileOnly("software.amazon.awssdk:url-connection-client") compileOnly("software.amazon.awssdk:apache-client") compileOnly("software.amazon.awssdk:auth") + compileOnly("software.amazon.awssdk:http-auth-aws-crt") compileOnly("software.amazon.awssdk:s3") compileOnly("software.amazon.awssdk:kms") compileOnly("software.amazon.awssdk:glue") diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 60aaeee8c6f1..c687bb4aea87 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -115,6 +115,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { implementation platform(libs.awssdk.bom) implementation 'software.amazon.awssdk:apache-client' implementation 'software.amazon.awssdk:auth' + implementation "software.amazon.awssdk:http-auth-aws-crt" implementation 'software.amazon.awssdk:iam' implementation 'software.amazon.awssdk:sso' implementation 'software.amazon.awssdk:s3' From c69f1dae56946444dccc751a6d0c5f9ef969da0b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 25 Aug 2024 13:58:53 +0200 Subject: [PATCH 0632/1019] Build: Bump software.amazon.awssdk:bom from 2.27.7 to 2.27.12 (#11006) Bumps software.amazon.awssdk:bom from 2.27.7 to 2.27.12. --- 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 09196493a284..f19094bf1266 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -28,7 +28,7 @@ arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" awaitility = "4.2.2" -awssdk-bom = "2.27.7" +awssdk-bom = "2.27.12" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 9b62b97fd257792f18b9f5d5119a6427a05acb9c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 25 Aug 2024 14:00:52 +0200 Subject: [PATCH 0633/1019] Build: Bump org.apache.commons:commons-compress from 1.27.0 to 1.27.1 (#11005) Bumps org.apache.commons:commons-compress from 1.27.0 to 1.27.1. --- updated-dependencies: - dependency-name: org.apache.commons:commons-compress 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> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index c687bb4aea87..c7f20e563577 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -71,7 +71,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' force 'org.xerial.snappy:snappy-java:1.1.10.6' - force 'org.apache.commons:commons-compress:1.27.0' + force 'org.apache.commons:commons-compress:1.27.1' force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.2.0' } } From f50173f039dce010c2b06a73efb8fe3714b6b9e5 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Aug 2024 10:47:38 +0200 Subject: [PATCH 0634/1019] Build: Bump jetty from 11.0.22 to 11.0.23 (#11003) Bumps `jetty` from 11.0.22 to 11.0.23. Updates `org.eclipse.jetty:jetty-server` from 11.0.22 to 11.0.23 Updates `org.eclipse.jetty:jetty-servlet` from 11.0.22 to 11.0.23 --- updated-dependencies: - dependency-name: org.eclipse.jetty:jetty-server dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.eclipse.jetty:jetty-servlet 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 f19094bf1266..bf7b9077d0ca 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -60,7 +60,7 @@ jakarta-el-api = "3.0.3" jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" -jetty = "11.0.22" +jetty = "11.0.23" junit = "5.10.1" kafka = "3.8.0" kryo-shaded = "4.0.3" From 5c2bb8ae71b0dd487c40ad53685b67e1fa6e8595 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Aug 2024 10:47:57 +0200 Subject: [PATCH 0635/1019] Build: Bump mkdocs-material from 9.5.31 to 9.5.33 (#11002) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.31 to 9.5.33. - [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.31...9.5.33) --- 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 cb85511b7d96..b07ec2b4d975 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.31 +mkdocs-material==9.5.33 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 efed85c403e98740a5672ba301031ba252a93872 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 26 Aug 2024 17:49:36 +0900 Subject: [PATCH 0636/1019] Docs: `_commit_snapshot_id` instead of `_change_snapshot_id` (#11000) --- docs/docs/spark-procedures.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 31172fb531bd..1cd14dd1888e 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -819,7 +819,7 @@ that provide additional information about the changes being tracked. These colum Here is an example of corresponding results. It shows that the first snapshot inserted 2 records, and the second snapshot deleted 1 record. -| id | name |_change_type | _change_ordinal | _change_snapshot_id | +| id | name |_change_type | _change_ordinal | _commit_snapshot_id | |---|--------|---|---|---| |1 | Alice |INSERT |0 |5390529835796506035| |2 | Bob |INSERT |0 |5390529835796506035| @@ -839,7 +839,7 @@ CALL spark_catalog.system.create_changelog_view( With the net changes, the above changelog view only contains the following row since Alice was inserted in the first snapshot and deleted in the second snapshot. -| id | name |_change_type | _change_ordinal | _change_snapshot_id | +| id | name |_change_type | _change_ordinal | _commit_snapshot_id | |---|--------|---|---|---| |2 | Bob |INSERT |0 |5390529835796506035| From 069d21dd549462a94699d799f4670d2b1e8f18bc Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 26 Aug 2024 17:50:17 +0900 Subject: [PATCH 0637/1019] Docs: Rename Clickhouse to ClickHouse (#10998) --- docs/mkdocs.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 5e43aa1d26d9..e915af4f3fce 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -53,7 +53,7 @@ nav: - hive.md - Trino: https://trino.io/docs/current/connector/iceberg.html - Daft: daft.md - - Clickhouse: https://clickhouse.com/docs/en/engines/table-engines/integrations/iceberg + - 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/ - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog From e67acb13a1acacc1ddd17e6f4150bb46233c9a0e Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 26 Aug 2024 08:01:13 -0700 Subject: [PATCH 0638/1019] Flink: infer source parallelism for FLIP-27 source in batch execution mode (#10832) --- .../iceberg/flink/source/IcebergSource.java | 85 +++++++- .../flink/source/IcebergTableSource.java | 31 +-- .../source/TestIcebergSourceBounded.java | 9 +- .../source/TestIcebergSourceBoundedSql.java | 2 +- .../TestIcebergSourceInferParallelism.java | 181 ++++++++++++++++++ .../flink/source/TestIcebergSourceSql.java | 7 +- ...estIcebergSpeculativeExecutionSupport.java | 14 +- 7 files changed, 293 insertions(+), 36 deletions(-) create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java 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 351ba54e5c7c..5718f4b93825 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 @@ -28,6 +28,8 @@ 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; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +39,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -74,6 +79,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; 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.ThreadPools; @@ -97,6 +103,11 @@ public class IcebergSource implements Source emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -132,16 +143,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -207,12 +228,35 @@ private SplitEnumerator createEnumer // Only do scan planning if nothing is restored from checkpoint state List splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + /** * Create a source builder. * @@ -571,6 +615,41 @@ public IcebergSource build() { emitter); } + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource source = build(); + TypeInformation outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static TypeInformation outputTypeInfo( + RowDataConverter converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + private ReaderFunction readerFunction(ScanContext context) { if (table instanceof BaseMetadataTable) { MetaDataReaderFunction rowDataReaderFunction = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ 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; @@ -130,11 +128,8 @@ protected List run( sourceBuilder.properties(options); DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * 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 static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +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.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator iterator = collector.getOutput()) { + List result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} 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 75f0a785a8c5..548940a842ce 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 @@ -53,7 +53,12 @@ public class TestIcebergSourceSql extends TestSqlBase { public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 992b712d9d69..f0d083060c1d 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.BatchExecutionOptions; @@ -48,8 +49,14 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_MANAGERS = 1; private static final int NUM_TASK_SLOTS = 3; @@ -144,9 +151,9 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + // Simulate slow subtask 0 with attempt 0 + TaskInfo taskInfo = getRuntimeContext().getTaskInfo(); + if (taskInfo.getIndexOfThisSubtask() == 0 && taskInfo.getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); } @@ -169,6 +176,7 @@ private static Configuration configure() { // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); // for speculative execution configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); From 95f53bd7567aad1ea9eba7f06084af99360add4c Mon Sep 17 00:00:00 2001 From: Qishang Zhong Date: Tue, 27 Aug 2024 01:05:16 +0800 Subject: [PATCH 0639/1019] Flink: Fix duplicate data with upsert writer in case of aborted checkpoints (#10526) --- .../apache/iceberg/flink/sink/FlinkSink.java | 11 +- .../iceberg/flink/sink/FlinkWriteResult.java | 40 +++ .../flink/sink/IcebergFilesCommitter.java | 48 ++-- .../flink/sink/IcebergStreamWriter.java | 13 +- .../flink/sink/TestCompressionSettings.java | 11 +- .../flink/sink/TestIcebergFilesCommitter.java | 240 +++++++++++++----- .../flink/sink/TestIcebergStreamWriter.java | 61 +++-- 7 files changed, 299 insertions(+), 125 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java 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 2256d1e874ce..5cd43a46de37 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 @@ -67,7 +67,6 @@ import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; 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; @@ -420,7 +419,7 @@ private DataStreamSink chainIcebergOperators() { distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files @@ -487,7 +486,7 @@ private DataStreamSink appendDummySink( } private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -507,7 +506,7 @@ private SingleOutputStreamOperator appendCommitter( return committerStream; } - private SingleOutputStreamOperator appendWriter( + private SingleOutputStreamOperator appendWriter( DataStream input, RowType flinkRowType, List equalityFieldIds, @@ -545,11 +544,11 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) .setParallelism(writerParallelism); if (uidPrefix != null) { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.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.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..7108c2008341 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -63,7 +63,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -96,7 +96,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -212,7 +212,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -220,8 +221,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -426,30 +425,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..bb5efe982ee1 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,7 +90,7 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override @@ -102,7 +103,7 @@ public String toString() { } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 8faae1b05a4e..3299e7a97776 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -210,8 +209,10 @@ public void testCompressionOrc() throws Exception { .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + private static OneInputStreamOperatorTestHarness + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -219,7 +220,7 @@ private static OneInputStreamOperatorTestHarness createIce IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -230,7 +231,7 @@ private static OneInputStreamOperatorTestHarness createIce private static Map appenderProperties( Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); 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 948c7b31430c..ac5babe11943 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 @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -161,7 +162,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +178,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +195,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +207,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +236,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +247,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +290,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +301,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +342,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +355,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +368,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +382,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +407,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +419,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +455,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +482,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +505,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +519,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +537,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +551,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +576,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +589,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +613,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +632,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +655,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +682,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +708,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +721,14 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +741,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +752,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +792,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +802,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +834,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +866,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +881,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +895,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +912,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + *

        + *
      • A specific row is updated + *
      • The prepareSnapshotPreBarrier triggered + *
      • Checkpoint failed for reasons outside of the Iceberg connector + *
      • The specific row is updated again in the second checkpoint as well + *
      • Second snapshot is triggered, and finished + *
      + * + *

      Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +997,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1009,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1028,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1046,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1063,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1089,7 +1189,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1209,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 50283f7ad215..e13721a9f170 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -28,6 +28,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -102,7 +103,7 @@ public void before() throws IOException { @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,7 +112,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -123,7 +125,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -148,14 +151,15 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -163,7 +167,10 @@ public void testSnapshotTwice() throws Exception { for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); } @@ -172,14 +179,14 @@ public void testSnapshotTwice() throws Exception { @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. @@ -212,7 +219,7 @@ private Set scanDataFiles() throws IOException { @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,13 +228,15 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -236,7 +245,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -244,13 +253,15 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once @@ -275,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -283,7 +294,8 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(8); @@ -346,13 +358,14 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + private static List getWriteResults(List flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness createIcebergStr IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); From 6b490b81880938c274e76191426e7bab09e6838a Mon Sep 17 00:00:00 2001 From: Rodrigo Date: Mon, 26 Aug 2024 13:28:46 -0700 Subject: [PATCH 0640/1019] Introduces the new IcebergSink based on the new V2 Flink Sink Abstraction (#10179) Co-authored-by: Liwei Li Co-authored-by: Kyle Bendickson Co-authored-by: Peter Vary --- .../iceberg/flink/sink/FlinkManifestUtil.java | 27 + .../apache/iceberg/flink/sink/FlinkSink.java | 3 +- .../flink/sink/IcebergCommittable.java | 95 ++ .../sink/IcebergCommittableSerializer.java | 68 + .../iceberg/flink/sink/IcebergCommitter.java | 311 ++++ .../flink/sink/IcebergFilesCommitter.java | 51 +- .../iceberg/flink/sink/IcebergSink.java | 742 +++++++++ .../iceberg/flink/sink/IcebergSinkWriter.java | 113 ++ .../flink/sink/IcebergStreamWriter.java | 6 +- .../flink/sink/IcebergWriteAggregator.java | 127 ++ .../flink/sink/ManifestOutputFileFactory.java | 4 +- .../apache/iceberg/flink/sink/SinkUtil.java | 94 ++ .../flink/sink/WriteResultSerializer.java | 61 + .../iceberg/flink/sink/SinkTestUtil.java | 62 + .../sink/TestFlinkIcebergSinkV2Base.java | 52 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 2 +- .../iceberg/flink/sink/TestFlinkManifest.java | 5 +- .../flink/sink/TestIcebergCommitter.java | 1445 +++++++++++++++++ .../flink/sink/TestIcebergFilesCommitter.java | 7 +- .../iceberg/flink/sink/TestIcebergSink.java | 436 +++++ .../flink/sink/TestIcebergSinkBranch.java | 120 ++ .../iceberg/flink/sink/TestIcebergSinkV2.java | 267 +++ .../flink/sink/TestIcebergSinkV2Branch.java | 119 ++ 23 files changed, 4129 insertions(+), 88 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index c7e8a2dea7cb..9571efdc5268 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } 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 5cd43a46de37..c53431490984 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 @@ -405,7 +405,8 @@ private DataStreamSink chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); int writerParallelism = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * 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.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + *

      {@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * 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.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + *

      In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..2245b36f15a7 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,311 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +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.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + *

        + *
      • There is a single {@link IcebergCommittable} for every checkpoint + *
      • There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + *
      • There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + *
      + */ +class IcebergCommitter implements Committer { + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + public static final WriteResult EMPTY_WRITE_RESULT = + WriteResult.builder() + .addDataFiles(Lists.newArrayList()) + .addDeleteFiles(Lists.newArrayList()) + .build(); + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + private final String branch; + private final Map snapshotProperties; + private final boolean replacePartitions; + private IcebergFilesCommitterMetrics committerMetrics; + private Table table; + private final TableLoader tableLoader; + private int maxContinuousEmptyCommits; + private ExecutorService workerPool; + private int continuousEmptyCheckpoints = 0; + + IcebergCommitter( + TableLoader tableLoader, + String branch, + Map snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + this.tableLoader = tableLoader; + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection> commitRequests) + throws IOException, InterruptedException { + if (commitRequests.isEmpty()) { + return; + } + + NavigableMap> commitRequestMap = Maps.newTreeMap(); + for (CommitRequest request : commitRequests) { + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + long maxCommittedCheckpointId = + SinkUtil.getMaxCommittedCheckpointId(table, last.jobId(), last.operatorId(), branch); + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + NavigableMap> uncommitted = + commitRequestMap.tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests(uncommitted, last.jobId(), last.operatorId()); + } + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link + * org.apache.iceberg.flink.sink.DeltaManifests} ordered by the checkpointId, and writing the new + * snapshot to the Iceberg table. The {@link org.apache.iceberg.SnapshotSummary} will contain the + * jobId, snapshotId, checkpointId so in case of job restart we can identify which changes are + * committed, and which are still waiting for the commit. + * + * @param commitRequestMap The checkpointId to {@link CommitRequest} map of the changes to commit + * @param newFlinkJobId The jobId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @param operatorId The operatorId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @throws IOException On commit failure + */ + private void commitPendingRequests( + NavigableMap> commitRequestMap, + String newFlinkJobId, + String operatorId) + throws IOException { + long checkpointId = commitRequestMap.lastKey(); + List manifests = Lists.newArrayList(); + NavigableMap pendingResults = Maps.newTreeMap(); + for (Map.Entry> e : commitRequestMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue().getCommittable().manifest())) { + pendingResults.put(e.getKey(), EMPTY_WRITE_RESULT); + } else { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue().getCommittable().manifest()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(summary); + } + + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void logCommitSummary(CommitSummary summary, String description) { + LOG.info( + "Preparing for commit: {} on table: {} branch: {} with summary: {}.", + description, + table, + branch, + summary); + } + + private void commitPendingResult( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId); + } + continuousEmptyCheckpoints = 0; + } else { + long checkpointId = pendingResults.lastKey(); + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void replacePartitions( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + String description = "dynamic partition overwrite"; + + logCommitSummary(summary, description); + commitOperation(dynamicOverwrite, description, newFlinkJobId, operatorId, checkpointId); + } + + private void commitDeltaTxn( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + String description = "append"; + logCommitSummary(summary, description); + // fail all commits as really its only one + commitOperation(appendFiles, description, newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + + String description = "rowDelta"; + logCommitSummary(summary, description); + commitOperation(rowDelta, description, newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate operation, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(SinkUtil.MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(SinkUtil.FLINK_JOB_ID, newFlinkJobId); + operation.set(SinkUtil.OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + if (committerMetrics != null) { + committerMetrics.commitDuration(durationMs); + } + } + + @Override + public void close() throws IOException { + tableLoader.close(); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index 7108c2008341..609deb621f43 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -44,13 +44,11 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.WriteResult; 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; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -187,7 +185,7 @@ public void initializeState(StateInitializationContext context) throws Exception // it's safe to assign the max committed checkpoint id from restored flink job to the current // flink job. this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + SinkUtil.getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); NavigableMap uncommittedDataFiles = Maps.newTreeMap(checkpointsState.get().iterator().next()) @@ -279,7 +277,7 @@ private void commitUpToCheckpoint( commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); committerMetrics.updateCommitSummary(summary); pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); } private void commitPendingResult( @@ -302,27 +300,6 @@ private void commitPendingResult( } } - private void deleteCommittedManifests( - List manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - private void replacePartitions( NavigableMap pendingResults, CommitSummary summary, @@ -503,28 +480,4 @@ static ListStateDescriptor> buildStateDescriptor() { longComparator); return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java new file mode 100644 index 000000000000..d080169544cd --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -0,0 +1,742 @@ +/* + * 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 static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +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.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. We will use the + * following: + * + *
        + *
      • {@link SupportsPreWriteTopology} which redistributes the data to the writers based on the + * {@link DistributionMode} + *
      • {@link org.apache.flink.api.connector.sink2.SinkWriter} which writes data/delete files, and + * generates the {@link org.apache.iceberg.io.WriteResult} objects for the files + *
      • {@link SupportsPreCommitTopology} which we use to place the {@link + * org.apache.iceberg.flink.sink.IcebergWriteAggregator} which merges the individual {@link + * org.apache.flink.api.connector.sink2.SinkWriter}'s {@link + * org.apache.iceberg.io.WriteResult}s to a single {@link + * org.apache.iceberg.flink.sink.IcebergCommittable} + *
      • {@link org.apache.iceberg.flink.sink.IcebergCommitter} which commits the incoming{@link + * org.apache.iceberg.flink.sink.IcebergCommittable}s to the Iceberg table + *
      • {@link SupportsPostCommitTopology} we could use for incremental compaction later. This is + * not implemented yet. + *
      + * + * The job graph looks like below: + * + *
      {@code
      + *                            Flink sink
      + *               +-----------------------------------------------------------------------------------+
      + *               |                                                                                   |
      + * +-------+     | +----------+                               +-------------+      +---------------+ |
      + * | Map 1 | ==> | | writer 1 |                               | committer 1 | ---> | post commit 1 | |
      + * +-------+     | +----------+                               +-------------+      +---------------+ |
      + *               |             \                             /                \                      |
      + *               |              \                           /                  \                     |
      + *               |               \                         /                    \                    |
      + * +-------+     | +----------+   \ +-------------------+ /   +-------------+    \ +---------------+ |
      + * | Map 2 | ==> | | writer 2 | --->| commit aggregator |     | committer 2 |      | post commit 2 | |
      + * +-------+     | +----------+     +-------------------+     +-------------+      +---------------+ |
      + *               |                                             Commit only on                        |
      + *               |                                             committer 1                           |
      + *               +-----------------------------------------------------------------------------------+
      + * }
      + */ +@Experimental +public class IcebergSink + implements Sink, + SupportsPreWriteTopology, + SupportsCommitter, + SupportsPreCommitTopology, + SupportsPostCommitTopology { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + private final TableLoader tableLoader; + private final Map snapshotProperties; + private final String uidSuffix; + private final String sinkId; + private final Map writeProperties; + private final RowType flinkRowType; + private final SerializableSupplier

  • tableSupplier; + private final transient FlinkWriteConf flinkWriteConf; + private final List equalityFieldIds; + private final boolean upsertMode; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final String branch; + private final boolean overwriteMode; + private final int workerPoolSize; + + private final Table table; + private final List equalityFieldColumns = null; + + private IcebergSink( + TableLoader tableLoader, + Table table, + Map snapshotProperties, + String uidSuffix, + Map writeProperties, + RowType flinkRowType, + SerializableSupplier
    tableSupplier, + FlinkWriteConf flinkWriteConf, + List equalityFieldIds, + String branch, + boolean overwriteMode) { + this.tableLoader = tableLoader; + this.snapshotProperties = snapshotProperties; + this.uidSuffix = uidSuffix; + this.writeProperties = writeProperties; + this.flinkRowType = flinkRowType; + this.tableSupplier = tableSupplier; + this.flinkWriteConf = flinkWriteConf; + this.equalityFieldIds = equalityFieldIds; + this.branch = branch; + this.overwriteMode = overwriteMode; + this.table = table; + this.upsertMode = flinkWriteConf.upsertMode(); + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter createWriter(InitContext context) { + RowDataTaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + targetDataFileSize, + dataFileFormat, + writeProperties, + equalityFieldIds, + upsertMode); + IcebergStreamWriterMetrics metrics = + new IcebergStreamWriterMetrics(context.metricGroup(), table.name()); + return new IcebergSinkWriter( + tableSupplier.get().name(), + taskWriterFactory, + metrics, + context.getSubtaskId(), + context.getAttemptNumber()); + } + + @Override + public Committer createCommitter(CommitterInitContext context) { + IcebergFilesCommitterMetrics metrics = + new IcebergFilesCommitterMetrics(context.metricGroup(), table.name()); + return new IcebergCommitter( + tableLoader, branch, snapshotProperties, overwriteMode, workerPoolSize, sinkId, metrics); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new IcebergCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream> committables) { + // TODO Support small file compaction + } + + @Override + public DataStream addPreWriteTopology(DataStream inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream> addPreCommitTopology( + DataStream> writeResults) { + TypeInformation> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + String suffix = defaultSuffix(uidSuffix, table.name()); + String preCommitAggregatorUid = String.format("Sink pre-commit aggregator: %s", suffix); + + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + return writeResults + .global() + .transform(preCommitAggregatorUid, typeInformation, new IcebergWriteAggregator(tableLoader)) + .uid(preCommitAggregatorUid) + .setParallelism(1) + .setMaxParallelism(1) + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + .global(); + } + + @Override + public SimpleVersionedSerializer getWriteResultSerializer() { + return new WriteResultSerializer(); + } + + public static class Builder { + private TableLoader tableLoader; + private String uidSuffix = ""; + private Function> inputCreator = null; + private TableSchema tableSchema; + private SerializableTable table; + private final Map writeOptions = Maps.newHashMap(); + private final Map snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private List equalityFieldColumns = null; + + private Builder() {} + + private Builder forRowData(DataStream newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forRow(DataStream input, TableSchema inputTableSchema) { + RowType rowType = (RowType) inputTableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = inputTableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return forMapperOutputType( + input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(inputTableSchema); + } + + private Builder forMapperOutputType( + DataStream input, MapFunction mapper, TypeInformation outputType) { + this.inputCreator = + newUidSuffix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism of map operator same as its input to keep map operator + // chaining its input, and avoid rebalanced by default. + SingleOutputStreamOperator inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidSuffix != null) { + String uid = String.format("Sink pre-writer mapper: %s", newUidSuffix); + inputStream.name(uid).uid(uid); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link SerializableTable} instance is used for initializing {@link + * IcebergStreamWriter} which will write all the records into {@link DataFile}s and emit them to + * downstream operator. Providing a table would avoid so many table loading from each separate + * task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = (SerializableTable) SerializableTable.copyOf(newTable); + return this; + } + + /** + * The table loader is used for loading tables in {@link + * org.apache.iceberg.flink.sink.IcebergCommitter} lazily, we need this loader because {@link + * Table} is not serializable and could not just use the loaded table from Builder#table in the + * remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + TableLoader tableLoader() { + return tableLoader; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the IcebergSink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid suffix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator). Actual operator uid will be appended + * with a suffix like "Sink Committer: $uidSuffix". + * + *

    Flink auto generates operator uid if not set explicitly. It is a recommended + * best-practice to set uid for all operators before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid. + * + *

    Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newSuffix suffix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidSuffix(String newSuffix) { + this.uidSuffix = newSuffix; + return this; + } + + public Builder snapshotProperties(Map properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + IcebergSink build() { + + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader(), "Table loader shouldn't be null"); + + // Set the table if it is not yet set in the builder, so we can do the equalityId checks + SerializableTable serializableTable = checkAndGetTable(tableLoader(), table); + this.table = serializableTable; + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + SerializableSupplier

    tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: + DataStreamSink rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream distributeDataStream(DataStream input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.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.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index bb5efe982ee1..412d6c7081bf 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -96,9 +96,9 @@ public void endInput() throws IOException { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * 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.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +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; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator> + implements OneInputStreamOperator< + CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage) element.getValue()).getCommittable()); + } + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..1cb7f4dea1e8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -26,13 +26,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier
    tableSupplier; private final Map props; private final String flinkJobId; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * 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 java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String OPERATOR_ID = "flink.operator-id"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * 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.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +class WriteResultSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.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.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List transformsToStreamElement(Collection elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary) value; + } + + static CommittableWithLineage extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage) value; + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index fc33c2fea5e6..da4ac2ada677 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map ROW_KIND_MAP = + static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List equalityFieldColumns, KeySelector keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List findValidSnapshots() { + List findValidSnapshots() { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable reader = 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 0b0c55f51c32..a2e9be9303c3 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 @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 53b7c4c0cc91..c21c3d5cc21b 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -134,7 +133,9 @@ public void testUserProvidedManifestLocation() throws IOException { String operatorId = newOperatorUniqueId(); File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..abdb2b5c79ae --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * 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 static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +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.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest buildCommitRequestFor( + String myJobID, long checkpoint, Collection writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer> { + + CommittableMessageSerializer serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer> duplicate() { + return null; + } + + @Override + public CommittableMessage createInstance() { + return null; + } + + @Override + public CommittableMessage copy( + CommittableMessage from) { + return from; + } + + @Override + public CommittableMessage copy( + CommittableMessage from, CommittableMessage reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage deserialize( + CommittableMessage reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return null; + } + }; +} 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 ac5babe11943..7808771d9887 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 @@ -140,8 +140,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -1179,8 +1178,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java new file mode 100644 index 000000000000..7f355c1e8403 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -0,0 +1,436 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.dag.Transformation; +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.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.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.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.IcebergSink.Builder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSink extends TestFlinkIcebergSinkBase { + + private TableLoader tableLoader; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @TestTemplate + void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @TestTemplate + void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); + } + } + } + + @TestTemplate + void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @TestTemplate + void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidSuffix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + IcebergSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); + + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .uidSuffix("ingestion"); + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @TestTemplate + void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} 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 new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.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.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + 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()); + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} 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 new file mode 100644 index 000000000000..93da40ab5c9a --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,119 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +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; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@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 + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} From 4717c862459ffd8d2f905aaa5fffc5db57e3f0d5 Mon Sep 17 00:00:00 2001 From: Rodrigo Date: Mon, 26 Aug 2024 15:57:57 -0700 Subject: [PATCH 0641/1019] Flink: Backport PR #10179 to Flink 1.20 for v2 sink (#11011) --- .../iceberg/flink/sink/FlinkManifestUtil.java | 27 + .../apache/iceberg/flink/sink/FlinkSink.java | 3 +- .../flink/sink/IcebergCommittable.java | 95 ++ .../sink/IcebergCommittableSerializer.java | 68 + .../iceberg/flink/sink/IcebergCommitter.java | 311 ++++ .../flink/sink/IcebergFilesCommitter.java | 51 +- .../iceberg/flink/sink/IcebergSink.java | 742 +++++++++ .../iceberg/flink/sink/IcebergSinkWriter.java | 113 ++ .../flink/sink/IcebergStreamWriter.java | 6 +- .../flink/sink/IcebergWriteAggregator.java | 127 ++ .../flink/sink/ManifestOutputFileFactory.java | 4 +- .../apache/iceberg/flink/sink/SinkUtil.java | 94 ++ .../flink/sink/WriteResultSerializer.java | 61 + .../iceberg/flink/sink/SinkTestUtil.java | 62 + .../sink/TestFlinkIcebergSinkV2Base.java | 52 +- .../sink/TestFlinkIcebergSinkV2Branch.java | 2 +- .../iceberg/flink/sink/TestFlinkManifest.java | 5 +- .../flink/sink/TestIcebergCommitter.java | 1445 +++++++++++++++++ .../flink/sink/TestIcebergFilesCommitter.java | 7 +- .../iceberg/flink/sink/TestIcebergSink.java | 436 +++++ .../flink/sink/TestIcebergSinkBranch.java | 120 ++ .../iceberg/flink/sink/TestIcebergSinkV2.java | 267 +++ .../flink/sink/TestIcebergSinkV2Branch.java | 119 ++ 23 files changed, 4129 insertions(+), 88 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java index c7e8a2dea7cb..9571efdc5268 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkManifestUtil.java @@ -33,9 +33,14 @@ import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; class FlinkManifestUtil { + + private static final Logger LOG = LoggerFactory.getLogger(FlinkManifestUtil.class); private static final int FORMAT_V2 = 2; private static final Long DUMMY_SNAPSHOT_ID = 0L; @@ -129,4 +134,26 @@ static WriteResult readCompletedFiles( return builder.addReferencedDataFiles(deltaManifests.referencedDataFiles()).build(); } + + static void deleteCommittedManifests( + Table table, List manifests, String newFlinkJobId, long checkpointId) { + for (ManifestFile manifest : manifests) { + try { + table.io().deleteFile(manifest.path()); + } catch (Exception e) { + // The flink manifests cleaning failure shouldn't abort the completed checkpoint. + String details = + MoreObjects.toStringHelper(FlinkManifestUtil.class) + .add("tableName", table.name()) + .add("flinkJobId", newFlinkJobId) + .add("checkpointId", checkpointId) + .add("manifestPath", manifest.path()) + .toString(); + LOG.warn( + "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", + details, + e); + } + } + } } 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 2256d1e874ce..be2a8db03097 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 @@ -406,7 +406,8 @@ private DataStreamSink chainIcebergOperators() { flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); // Find out the equality field id list based on the user-provided equality field column names. - List equalityFieldIds = checkAndGetEqualityFieldIds(); + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); RowType flinkRowType = toFlinkRowType(table.schema(), tableSchema); int writerParallelism = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java new file mode 100644 index 000000000000..408c3e9a9d5f --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittable.java @@ -0,0 +1,95 @@ +/* + * 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.io.Serializable; +import java.util.Arrays; +import java.util.Objects; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; + +/** + * The aggregated results of a single checkpoint which should be committed. Containing the + * serialized {@link org.apache.iceberg.flink.sink.DeltaManifests} file - which contains the commit + * data, and the jobId, operatorId, checkpointId triplet which helps identifying the specific commit + * + *

    {@link IcebergCommittableSerializer} is used for serializing the objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + */ +class IcebergCommittable implements Serializable { + private final byte[] manifest; + private final String jobId; + private final String operatorId; + private final long checkpointId; + + IcebergCommittable(byte[] manifest, String jobId, String operatorId, long checkpointId) { + this.manifest = manifest; + this.jobId = jobId; + this.operatorId = operatorId; + this.checkpointId = checkpointId; + } + + byte[] manifest() { + return manifest; + } + + String jobId() { + return jobId; + } + + String operatorId() { + return operatorId; + } + + Long checkpointId() { + return checkpointId; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("jobId", jobId) + .add("checkpointId", checkpointId) + .add("operatorId", operatorId) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IcebergCommittable that = (IcebergCommittable) o; + return checkpointId == that.checkpointId + && Arrays.equals(manifest, that.manifest) + && Objects.equals(jobId, that.jobId) + && Objects.equals(operatorId, that.operatorId); + } + + @Override + public int hashCode() { + int result = Objects.hash(jobId, operatorId, checkpointId); + result = 31 * result + Arrays.hashCode(manifest); + return result; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java new file mode 100644 index 000000000000..e2b388a83c75 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommittableSerializer.java @@ -0,0 +1,68 @@ +/* + * 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.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +/** + * This serializer is used for serializing the {@link IcebergCommittable} objects between the Writer + * and the Aggregator operator and between the Aggregator and the Committer as well. + * + *

    In both cases only the respective part is serialized. + */ +class IcebergCommittableSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(IcebergCommittable committable) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + view.writeUTF(committable.jobId()); + view.writeUTF(committable.operatorId()); + view.writeLong(committable.checkpointId()); + view.writeInt(committable.manifest().length); + view.write(committable.manifest()); + return out.toByteArray(); + } + + @Override + public IcebergCommittable deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + String jobId = view.readUTF(); + String operatorId = view.readUTF(); + long checkpointId = view.readLong(); + int manifestLen = view.readInt(); + byte[] manifestBuf; + manifestBuf = new byte[manifestLen]; + view.read(manifestBuf); + return new IcebergCommittable(manifestBuf, jobId, operatorId, checkpointId); + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java new file mode 100644 index 000000000000..2245b36f15a7 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -0,0 +1,311 @@ +/* + * 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.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.iceberg.AppendFiles; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.ReplacePartitions; +import org.apache.iceberg.RowDelta; +import org.apache.iceberg.SnapshotUpdate; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +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.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.ThreadPools; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class implements the Flink SinkV2 {@link Committer} interface to implement the Iceberg + * commits. The implementation builds on the following assumptions: + * + *

      + *
    • There is a single {@link IcebergCommittable} for every checkpoint + *
    • There is no late checkpoint - if checkpoint 'x' has received in one call, then after a + * successful run only checkpoints > x will arrive + *
    • There is no other writer which would generate another commit to the same branch with the + * same jobId-operatorId-checkpointId triplet + *
    + */ +class IcebergCommitter implements Committer { + private static final Logger LOG = LoggerFactory.getLogger(IcebergCommitter.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + public static final WriteResult EMPTY_WRITE_RESULT = + WriteResult.builder() + .addDataFiles(Lists.newArrayList()) + .addDeleteFiles(Lists.newArrayList()) + .build(); + + @VisibleForTesting + static final String MAX_CONTINUOUS_EMPTY_COMMITS = "flink.max-continuous-empty-commits"; + + private final String branch; + private final Map snapshotProperties; + private final boolean replacePartitions; + private IcebergFilesCommitterMetrics committerMetrics; + private Table table; + private final TableLoader tableLoader; + private int maxContinuousEmptyCommits; + private ExecutorService workerPool; + private int continuousEmptyCheckpoints = 0; + + IcebergCommitter( + TableLoader tableLoader, + String branch, + Map snapshotProperties, + boolean replacePartitions, + int workerPoolSize, + String sinkId, + IcebergFilesCommitterMetrics committerMetrics) { + this.branch = branch; + this.snapshotProperties = snapshotProperties; + this.replacePartitions = replacePartitions; + this.committerMetrics = committerMetrics; + this.tableLoader = tableLoader; + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + this.table = tableLoader.loadTable(); + this.maxContinuousEmptyCommits = + PropertyUtil.propertyAsInt(table.properties(), MAX_CONTINUOUS_EMPTY_COMMITS, 10); + Preconditions.checkArgument( + maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); + this.workerPool = + ThreadPools.newWorkerPool( + "iceberg-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize); + this.continuousEmptyCheckpoints = 0; + } + + @Override + public void commit(Collection> commitRequests) + throws IOException, InterruptedException { + if (commitRequests.isEmpty()) { + return; + } + + NavigableMap> commitRequestMap = Maps.newTreeMap(); + for (CommitRequest request : commitRequests) { + commitRequestMap.put(request.getCommittable().checkpointId(), request); + } + + IcebergCommittable last = commitRequestMap.lastEntry().getValue().getCommittable(); + long maxCommittedCheckpointId = + SinkUtil.getMaxCommittedCheckpointId(table, last.jobId(), last.operatorId(), branch); + // Mark the already committed FilesCommittable(s) as finished + commitRequestMap + .headMap(maxCommittedCheckpointId, true) + .values() + .forEach(CommitRequest::signalAlreadyCommitted); + NavigableMap> uncommitted = + commitRequestMap.tailMap(maxCommittedCheckpointId, false); + if (!uncommitted.isEmpty()) { + commitPendingRequests(uncommitted, last.jobId(), last.operatorId()); + } + } + + /** + * Commits the data to the Iceberg table by reading the file data from the {@link + * org.apache.iceberg.flink.sink.DeltaManifests} ordered by the checkpointId, and writing the new + * snapshot to the Iceberg table. The {@link org.apache.iceberg.SnapshotSummary} will contain the + * jobId, snapshotId, checkpointId so in case of job restart we can identify which changes are + * committed, and which are still waiting for the commit. + * + * @param commitRequestMap The checkpointId to {@link CommitRequest} map of the changes to commit + * @param newFlinkJobId The jobId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @param operatorId The operatorId to store in the {@link org.apache.iceberg.SnapshotSummary} + * @throws IOException On commit failure + */ + private void commitPendingRequests( + NavigableMap> commitRequestMap, + String newFlinkJobId, + String operatorId) + throws IOException { + long checkpointId = commitRequestMap.lastKey(); + List manifests = Lists.newArrayList(); + NavigableMap pendingResults = Maps.newTreeMap(); + for (Map.Entry> e : commitRequestMap.entrySet()) { + if (Arrays.equals(EMPTY_MANIFEST_DATA, e.getValue().getCommittable().manifest())) { + pendingResults.put(e.getKey(), EMPTY_WRITE_RESULT); + } else { + DeltaManifests deltaManifests = + SimpleVersionedSerialization.readVersionAndDeSerialize( + DeltaManifestsSerializer.INSTANCE, e.getValue().getCommittable().manifest()); + pendingResults.put( + e.getKey(), + FlinkManifestUtil.readCompletedFiles(deltaManifests, table.io(), table.specs())); + manifests.addAll(deltaManifests.manifests()); + } + } + + CommitSummary summary = new CommitSummary(pendingResults); + commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId); + if (committerMetrics != null) { + committerMetrics.updateCommitSummary(summary); + } + + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); + } + + private void logCommitSummary(CommitSummary summary, String description) { + LOG.info( + "Preparing for commit: {} on table: {} branch: {} with summary: {}.", + description, + table, + branch, + summary); + } + + private void commitPendingResult( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long totalFiles = summary.dataFilesCount() + summary.deleteFilesCount(); + continuousEmptyCheckpoints = totalFiles == 0 ? continuousEmptyCheckpoints + 1 : 0; + if (totalFiles != 0 || continuousEmptyCheckpoints % maxContinuousEmptyCommits == 0) { + if (replacePartitions) { + replacePartitions(pendingResults, summary, newFlinkJobId, operatorId); + } else { + commitDeltaTxn(pendingResults, summary, newFlinkJobId, operatorId); + } + continuousEmptyCheckpoints = 0; + } else { + long checkpointId = pendingResults.lastKey(); + LOG.info("Skip commit for checkpoint {} due to no data files or delete files.", checkpointId); + } + } + + private void replacePartitions( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + Preconditions.checkState( + summary.deleteFilesCount() == 0, "Cannot overwrite partitions with delete files."); + // Commit the overwrite transaction. + ReplacePartitions dynamicOverwrite = table.newReplacePartitions().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, "Should have no referenced data files."); + Arrays.stream(result.dataFiles()).forEach(dynamicOverwrite::addFile); + } + String description = "dynamic partition overwrite"; + + logCommitSummary(summary, description); + commitOperation(dynamicOverwrite, description, newFlinkJobId, operatorId, checkpointId); + } + + private void commitDeltaTxn( + NavigableMap pendingResults, + CommitSummary summary, + String newFlinkJobId, + String operatorId) { + long checkpointId = pendingResults.lastKey(); + if (summary.deleteFilesCount() == 0) { + // To be compatible with iceberg format V1. + AppendFiles appendFiles = table.newAppend().scanManifestsWith(workerPool); + for (WriteResult result : pendingResults.values()) { + Preconditions.checkState( + result.referencedDataFiles().length == 0, + "Should have no referenced data files for append."); + Arrays.stream(result.dataFiles()).forEach(appendFiles::appendFile); + } + String description = "append"; + logCommitSummary(summary, description); + // fail all commits as really its only one + commitOperation(appendFiles, description, newFlinkJobId, operatorId, checkpointId); + } else { + // To be compatible with iceberg format V2. + for (Map.Entry e : pendingResults.entrySet()) { + // We don't commit the merged result into a single transaction because for the sequential + // transaction txn1 and txn2, the equality-delete files of txn2 are required to be applied + // to data files from txn1. Committing the merged one will lead to the incorrect delete + // semantic. + WriteResult result = e.getValue(); + + // Row delta validations are not needed for streaming changes that write equality deletes. + // Equality deletes are applied to data in all previous sequence numbers, so retries may + // push deletes further in the future, but do not affect correctness. Position deletes + // committed to the table in this path are used only to delete rows from data files that are + // being added in this commit. There is no way for data files added along with the delete + // files to be concurrently removed, so there is no need to validate the files referenced by + // the position delete files that are being committed. + RowDelta rowDelta = table.newRowDelta().scanManifestsWith(workerPool); + + Arrays.stream(result.dataFiles()).forEach(rowDelta::addRows); + Arrays.stream(result.deleteFiles()).forEach(rowDelta::addDeletes); + + String description = "rowDelta"; + logCommitSummary(summary, description); + commitOperation(rowDelta, description, newFlinkJobId, operatorId, e.getKey()); + } + } + } + + private void commitOperation( + SnapshotUpdate operation, + String description, + String newFlinkJobId, + String operatorId, + long checkpointId) { + + snapshotProperties.forEach(operation::set); + // custom snapshot metadata properties will be overridden if they conflict with internal ones + // used by the sink. + operation.set(SinkUtil.MAX_COMMITTED_CHECKPOINT_ID, Long.toString(checkpointId)); + operation.set(SinkUtil.FLINK_JOB_ID, newFlinkJobId); + operation.set(SinkUtil.OPERATOR_ID, operatorId); + operation.toBranch(branch); + + long startNano = System.nanoTime(); + operation.commit(); // abort is automatically called if this fails. + long durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano); + LOG.info( + "Committed {} to table: {}, branch: {}, checkpointId {} in {} ms", + description, + table.name(), + branch, + checkpointId, + durationMs); + if (committerMetrics != null) { + committerMetrics.commitDuration(durationMs); + } + } + + @Override + public void close() throws IOException { + tableLoader.close(); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..622daa808897 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -44,13 +44,11 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.ReplacePartitions; import org.apache.iceberg.RowDelta; -import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotUpdate; import org.apache.iceberg.Table; import org.apache.iceberg.flink.TableLoader; import org.apache.iceberg.io.WriteResult; 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; import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Lists; @@ -187,7 +185,7 @@ public void initializeState(StateInitializationContext context) throws Exception // it's safe to assign the max committed checkpoint id from restored flink job to the current // flink job. this.maxCommittedCheckpointId = - getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); + SinkUtil.getMaxCommittedCheckpointId(table, restoredFlinkJobId, operatorUniqueId, branch); NavigableMap uncommittedDataFiles = Maps.newTreeMap(checkpointsState.get().iterator().next()) @@ -280,7 +278,7 @@ private void commitUpToCheckpoint( commitPendingResult(pendingResults, summary, newFlinkJobId, operatorId, checkpointId); committerMetrics.updateCommitSummary(summary); pendingMap.clear(); - deleteCommittedManifests(manifests, newFlinkJobId, checkpointId); + FlinkManifestUtil.deleteCommittedManifests(table, manifests, newFlinkJobId, checkpointId); } private void commitPendingResult( @@ -303,27 +301,6 @@ private void commitPendingResult( } } - private void deleteCommittedManifests( - List manifests, String newFlinkJobId, long checkpointId) { - for (ManifestFile manifest : manifests) { - try { - table.io().deleteFile(manifest.path()); - } catch (Exception e) { - // The flink manifests cleaning failure shouldn't abort the completed checkpoint. - String details = - MoreObjects.toStringHelper(this) - .add("flinkJobId", newFlinkJobId) - .add("checkpointId", checkpointId) - .add("manifestPath", manifest.path()) - .toString(); - LOG.warn( - "The iceberg transaction has been committed, but we failed to clean the temporary flink manifests: {}", - details, - e); - } - } - } - private void replacePartitions( NavigableMap pendingResults, CommitSummary summary, @@ -489,28 +466,4 @@ static ListStateDescriptor> buildStateDescriptor() { longComparator); return new ListStateDescriptor<>("iceberg-files-committer-state", sortedMapTypeInfo); } - - static long getMaxCommittedCheckpointId( - Table table, String flinkJobId, String operatorId, String branch) { - Snapshot snapshot = table.snapshot(branch); - long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; - - while (snapshot != null) { - Map summary = snapshot.summary(); - String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); - String snapshotOperatorId = summary.get(OPERATOR_ID); - if (flinkJobId.equals(snapshotFlinkJobId) - && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { - String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); - if (value != null) { - lastCommittedCheckpointId = Long.parseLong(value); - break; - } - } - Long parentSnapshotId = snapshot.parentId(); - snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; - } - - return lastCommittedCheckpointId; - } } 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 new file mode 100644 index 000000000000..d080169544cd --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSink.java @@ -0,0 +1,742 @@ +/* + * 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 static org.apache.iceberg.TableProperties.AVRO_COMPRESSION; +import static org.apache.iceberg.TableProperties.AVRO_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION; +import static org.apache.iceberg.TableProperties.ORC_COMPRESSION_STRATEGY; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION; +import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL; +import static org.apache.iceberg.TableProperties.WRITE_DISTRIBUTION_MODE; + +import java.io.IOException; +import java.io.UncheckedIOException; +import java.time.Duration; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.function.Function; +import org.apache.flink.annotation.Experimental; +import org.apache.flink.api.common.functions.MapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.ReadableConfig; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageTypeInfo; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreCommitTopology; +import org.apache.flink.streaming.api.connector.sink2.SupportsPreWriteTopology; +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.table.api.TableSchema; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.util.DataFormatConverters; +import org.apache.flink.table.types.DataType; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.types.Row; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.PartitionField; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SerializableTable; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.FlinkWriteConf; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.types.TypeUtil; +import org.apache.iceberg.util.SerializableSupplier; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Flink v2 sink offer different hooks to insert custom topologies into the sink. We will use the + * following: + * + *
      + *
    • {@link SupportsPreWriteTopology} which redistributes the data to the writers based on the + * {@link DistributionMode} + *
    • {@link org.apache.flink.api.connector.sink2.SinkWriter} which writes data/delete files, and + * generates the {@link org.apache.iceberg.io.WriteResult} objects for the files + *
    • {@link SupportsPreCommitTopology} which we use to place the {@link + * org.apache.iceberg.flink.sink.IcebergWriteAggregator} which merges the individual {@link + * org.apache.flink.api.connector.sink2.SinkWriter}'s {@link + * org.apache.iceberg.io.WriteResult}s to a single {@link + * org.apache.iceberg.flink.sink.IcebergCommittable} + *
    • {@link org.apache.iceberg.flink.sink.IcebergCommitter} which commits the incoming{@link + * org.apache.iceberg.flink.sink.IcebergCommittable}s to the Iceberg table + *
    • {@link SupportsPostCommitTopology} we could use for incremental compaction later. This is + * not implemented yet. + *
    + * + * The job graph looks like below: + * + *
    {@code
    + *                            Flink sink
    + *               +-----------------------------------------------------------------------------------+
    + *               |                                                                                   |
    + * +-------+     | +----------+                               +-------------+      +---------------+ |
    + * | Map 1 | ==> | | writer 1 |                               | committer 1 | ---> | post commit 1 | |
    + * +-------+     | +----------+                               +-------------+      +---------------+ |
    + *               |             \                             /                \                      |
    + *               |              \                           /                  \                     |
    + *               |               \                         /                    \                    |
    + * +-------+     | +----------+   \ +-------------------+ /   +-------------+    \ +---------------+ |
    + * | Map 2 | ==> | | writer 2 | --->| commit aggregator |     | committer 2 |      | post commit 2 | |
    + * +-------+     | +----------+     +-------------------+     +-------------+      +---------------+ |
    + *               |                                             Commit only on                        |
    + *               |                                             committer 1                           |
    + *               +-----------------------------------------------------------------------------------+
    + * }
    + */ +@Experimental +public class IcebergSink + implements Sink, + SupportsPreWriteTopology, + SupportsCommitter, + SupportsPreCommitTopology, + SupportsPostCommitTopology { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSink.class); + private final TableLoader tableLoader; + private final Map snapshotProperties; + private final String uidSuffix; + private final String sinkId; + private final Map writeProperties; + private final RowType flinkRowType; + private final SerializableSupplier
    tableSupplier; + private final transient FlinkWriteConf flinkWriteConf; + private final List equalityFieldIds; + private final boolean upsertMode; + private final FileFormat dataFileFormat; + private final long targetDataFileSize; + private final String branch; + private final boolean overwriteMode; + private final int workerPoolSize; + + private final Table table; + private final List equalityFieldColumns = null; + + private IcebergSink( + TableLoader tableLoader, + Table table, + Map snapshotProperties, + String uidSuffix, + Map writeProperties, + RowType flinkRowType, + SerializableSupplier
    tableSupplier, + FlinkWriteConf flinkWriteConf, + List equalityFieldIds, + String branch, + boolean overwriteMode) { + this.tableLoader = tableLoader; + this.snapshotProperties = snapshotProperties; + this.uidSuffix = uidSuffix; + this.writeProperties = writeProperties; + this.flinkRowType = flinkRowType; + this.tableSupplier = tableSupplier; + this.flinkWriteConf = flinkWriteConf; + this.equalityFieldIds = equalityFieldIds; + this.branch = branch; + this.overwriteMode = overwriteMode; + this.table = table; + this.upsertMode = flinkWriteConf.upsertMode(); + this.dataFileFormat = flinkWriteConf.dataFileFormat(); + this.targetDataFileSize = flinkWriteConf.targetDataFileSize(); + this.workerPoolSize = flinkWriteConf.workerPoolSize(); + // We generate a random UUID every time when a sink is created. + // This is used to separate files generated by different sinks writing the same table. + // Also used to generate the aggregator operator name + this.sinkId = UUID.randomUUID().toString(); + } + + @Override + public SinkWriter createWriter(InitContext context) { + RowDataTaskWriterFactory taskWriterFactory = + new RowDataTaskWriterFactory( + tableSupplier, + flinkRowType, + targetDataFileSize, + dataFileFormat, + writeProperties, + equalityFieldIds, + upsertMode); + IcebergStreamWriterMetrics metrics = + new IcebergStreamWriterMetrics(context.metricGroup(), table.name()); + return new IcebergSinkWriter( + tableSupplier.get().name(), + taskWriterFactory, + metrics, + context.getSubtaskId(), + context.getAttemptNumber()); + } + + @Override + public Committer createCommitter(CommitterInitContext context) { + IcebergFilesCommitterMetrics metrics = + new IcebergFilesCommitterMetrics(context.metricGroup(), table.name()); + return new IcebergCommitter( + tableLoader, branch, snapshotProperties, overwriteMode, workerPoolSize, sinkId, metrics); + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new IcebergCommittableSerializer(); + } + + @Override + public void addPostCommitTopology( + DataStream> committables) { + // TODO Support small file compaction + } + + @Override + public DataStream addPreWriteTopology(DataStream inputDataStream) { + return distributeDataStream(inputDataStream); + } + + @Override + public DataStream> addPreCommitTopology( + DataStream> writeResults) { + TypeInformation> typeInformation = + CommittableMessageTypeInfo.of(this::getCommittableSerializer); + + String suffix = defaultSuffix(uidSuffix, table.name()); + String preCommitAggregatorUid = String.format("Sink pre-commit aggregator: %s", suffix); + + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + return writeResults + .global() + .transform(preCommitAggregatorUid, typeInformation, new IcebergWriteAggregator(tableLoader)) + .uid(preCommitAggregatorUid) + .setParallelism(1) + .setMaxParallelism(1) + // global forces all output records send to subtask 0 of the downstream committer operator. + // This is to ensure commit only happen in one committer subtask. + // Once upstream Flink provides the capability of setting committer operator + // parallelism to 1, this can be removed. + .global(); + } + + @Override + public SimpleVersionedSerializer getWriteResultSerializer() { + return new WriteResultSerializer(); + } + + public static class Builder { + private TableLoader tableLoader; + private String uidSuffix = ""; + private Function> inputCreator = null; + private TableSchema tableSchema; + private SerializableTable table; + private final Map writeOptions = Maps.newHashMap(); + private final Map snapshotSummary = Maps.newHashMap(); + private ReadableConfig readableConfig = new Configuration(); + private List equalityFieldColumns = null; + + private Builder() {} + + private Builder forRowData(DataStream newRowDataInput) { + this.inputCreator = ignored -> newRowDataInput; + return this; + } + + private Builder forRow(DataStream input, TableSchema inputTableSchema) { + RowType rowType = (RowType) inputTableSchema.toRowDataType().getLogicalType(); + DataType[] fieldDataTypes = inputTableSchema.getFieldDataTypes(); + + DataFormatConverters.RowConverter rowConverter = + new DataFormatConverters.RowConverter(fieldDataTypes); + return forMapperOutputType( + input, rowConverter::toInternal, FlinkCompatibilityUtil.toTypeInfo(rowType)) + .tableSchema(inputTableSchema); + } + + private Builder forMapperOutputType( + DataStream input, MapFunction mapper, TypeInformation outputType) { + this.inputCreator = + newUidSuffix -> { + // Input stream order is crucial for some situation(e.g. in cdc case). Therefore, we + // need to set the parallelism of map operator same as its input to keep map operator + // chaining its input, and avoid rebalanced by default. + SingleOutputStreamOperator inputStream = + input.map(mapper, outputType).setParallelism(input.getParallelism()); + if (newUidSuffix != null) { + String uid = String.format("Sink pre-writer mapper: %s", newUidSuffix); + inputStream.name(uid).uid(uid); + } + return inputStream; + }; + return this; + } + + /** + * This iceberg {@link SerializableTable} instance is used for initializing {@link + * IcebergStreamWriter} which will write all the records into {@link DataFile}s and emit them to + * downstream operator. Providing a table would avoid so many table loading from each separate + * task. + * + * @param newTable the loaded iceberg table instance. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder table(Table newTable) { + this.table = (SerializableTable) SerializableTable.copyOf(newTable); + return this; + } + + /** + * The table loader is used for loading tables in {@link + * org.apache.iceberg.flink.sink.IcebergCommitter} lazily, we need this loader because {@link + * Table} is not serializable and could not just use the loaded table from Builder#table in the + * remote task manager. + * + * @param newTableLoader to load iceberg table inside tasks. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder tableLoader(TableLoader newTableLoader) { + this.tableLoader = newTableLoader; + return this; + } + + TableLoader tableLoader() { + return tableLoader; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder set(String property, String value) { + writeOptions.put(property, value); + return this; + } + + /** + * Set the write properties for IcebergSink. View the supported properties in {@link + * FlinkWriteOptions} + */ + public Builder setAll(Map properties) { + writeOptions.putAll(properties); + return this; + } + + public Builder tableSchema(TableSchema newTableSchema) { + this.tableSchema = newTableSchema; + return this; + } + + public Builder overwrite(boolean newOverwrite) { + writeOptions.put(FlinkWriteOptions.OVERWRITE_MODE.key(), Boolean.toString(newOverwrite)); + return this; + } + + public Builder flinkConf(ReadableConfig config) { + this.readableConfig = config; + return this; + } + + /** + * Configure the write {@link DistributionMode} that the IcebergSink will use. Currently, flink + * support {@link DistributionMode#NONE} and {@link DistributionMode#HASH}. + * + * @param mode to specify the write distribution mode. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder distributionMode(DistributionMode mode) { + Preconditions.checkArgument( + !DistributionMode.RANGE.equals(mode), + "Flink does not support 'range' write distribution mode now."); + if (mode != null) { + writeOptions.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), mode.modeName()); + } + return this; + } + + /** + * Configuring the write parallel number for iceberg stream writer. + * + * @param newWriteParallelism the number of parallel iceberg stream writer. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder writeParallelism(int newWriteParallelism) { + writeOptions.put( + FlinkWriteOptions.WRITE_PARALLELISM.key(), Integer.toString(newWriteParallelism)); + return this; + } + + /** + * All INSERT/UPDATE_AFTER events from input stream will be transformed to UPSERT events, which + * means it will DELETE the old records and then INSERT the new records. In partitioned table, + * the partition fields should be a subset of equality fields, otherwise the old row that + * located in partition-A could not be deleted by the new row that located in partition-B. + * + * @param enabled indicate whether it should transform all INSERT/UPDATE_AFTER events to UPSERT. + * @return {@link IcebergSink.Builder} to connect the iceberg table. + */ + public Builder upsert(boolean enabled) { + writeOptions.put(FlinkWriteOptions.WRITE_UPSERT_ENABLED.key(), Boolean.toString(enabled)); + return this; + } + + /** + * Configuring the equality field columns for iceberg table that accept CDC or UPSERT events. + * + * @param columns defines the iceberg table's key. + * @return {@link Builder} to connect the iceberg table. + */ + public Builder equalityFieldColumns(List columns) { + this.equalityFieldColumns = columns; + return this; + } + + /** + * Set the uid suffix for IcebergSink operators. Note that IcebergSink internally consists of + * multiple operators (like writer, committer, aggregator). Actual operator uid will be appended + * with a suffix like "Sink Committer: $uidSuffix". + * + *

    Flink auto generates operator uid if not set explicitly. It is a recommended + * best-practice to set uid for all operators before deploying to production. Flink has an + * option to {@code pipeline.auto-generate-uid=false} to disable auto-generation and force + * explicit setting of all operator uid. + * + *

    Be careful with setting this for an existing job, because now we are changing the operator + * uid from an auto-generated one to this new value. When deploying the change with a + * checkpoint, Flink won't be able to restore the previous IcebergSink operator state (more + * specifically the committer operator state). You need to use {@code --allowNonRestoredState} + * to ignore the previous sink state. During restore IcebergSink state is used to check if last + * commit was actually successful or not. {@code --allowNonRestoredState} can lead to data loss + * if the Iceberg commit failed in the last completed checkpoint. + * + * @param newSuffix suffix for Flink sink operator uid and name + * @return {@link Builder} to connect the iceberg table. + */ + public Builder uidSuffix(String newSuffix) { + this.uidSuffix = newSuffix; + return this; + } + + public Builder snapshotProperties(Map properties) { + snapshotSummary.putAll(properties); + return this; + } + + public Builder setSnapshotProperty(String property, String value) { + snapshotSummary.put(property, value); + return this; + } + + public Builder toBranch(String branch) { + writeOptions.put(FlinkWriteOptions.BRANCH.key(), branch); + return this; + } + + IcebergSink build() { + + Preconditions.checkArgument( + inputCreator != null, + "Please use forRowData() or forMapperOutputType() to initialize the input DataStream."); + Preconditions.checkNotNull(tableLoader(), "Table loader shouldn't be null"); + + // Set the table if it is not yet set in the builder, so we can do the equalityId checks + SerializableTable serializableTable = checkAndGetTable(tableLoader(), table); + this.table = serializableTable; + // Init the `flinkWriteConf` here, so we can do the checks + FlinkWriteConf flinkWriteConf = new FlinkWriteConf(table, writeOptions, readableConfig); + + Duration tableRefreshInterval = flinkWriteConf.tableRefreshInterval(); + SerializableSupplier

    tableSupplier; + if (tableRefreshInterval != null) { + tableSupplier = new CachingTableSupplier(table, tableLoader(), tableRefreshInterval); + } else { + tableSupplier = () -> serializableTable; + } + + boolean overwriteMode = flinkWriteConf.overwriteMode(); + + // Validate the equality fields and partition fields if we enable the upsert mode. + List equalityFieldIds = + SinkUtil.checkAndGetEqualityFieldIds(table, equalityFieldColumns); + + if (flinkWriteConf.upsertMode()) { + Preconditions.checkState( + !overwriteMode, + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + Preconditions.checkState( + !equalityFieldIds.isEmpty(), + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + if (!table.spec().isUnpartitioned()) { + for (PartitionField partitionField : table.spec().fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In UPSERT mode, partition field '%s' should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + } + } + + return new IcebergSink( + tableLoader, + table, + snapshotSummary, + uidSuffix, + writeProperties(table, flinkWriteConf.dataFileFormat(), flinkWriteConf), + toFlinkRowType(table.schema(), tableSchema), + tableSupplier, + flinkWriteConf, + equalityFieldIds, + flinkWriteConf.branch(), + overwriteMode); + } + + /** + * Append the iceberg sink operators to write records to iceberg table. + * + * @return {@link DataStreamSink} for sink. + */ + public DataStreamSink append() { + IcebergSink sink = build(); + String suffix = defaultSuffix(uidSuffix, table.name()); + DataStream rowDataInput = inputCreator.apply(suffix); + // Please note that V2 sink framework will apply the uid here to the framework created + // operators like writer, + // committer. E.g. "Sink writer: + DataStreamSink rowDataDataStreamSink = + rowDataInput.sinkTo(sink).uid(suffix).name(suffix); + + // Note that IcebergSink internally consists o multiple operators (like writer, committer, + // aggregator). + // The following parallelism will be propagated to all of the above operators. + if (sink.flinkWriteConf.writeParallelism() != null) { + rowDataDataStreamSink.setParallelism(sink.flinkWriteConf.writeParallelism()); + } + return rowDataDataStreamSink; + } + } + + private static String defaultSuffix(String uidSuffix, String defaultSuffix) { + if (uidSuffix == null || uidSuffix.isEmpty()) { + return defaultSuffix; + } + return uidSuffix; + } + + private static SerializableTable checkAndGetTable(TableLoader tableLoader, Table table) { + if (table == null) { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + try (TableLoader loader = tableLoader) { + return (SerializableTable) SerializableTable.copyOf(loader.loadTable()); + } catch (IOException e) { + throw new UncheckedIOException( + "Failed to load iceberg table from table loader: " + tableLoader, e); + } + } + + return (SerializableTable) SerializableTable.copyOf(table); + } + + private static RowType toFlinkRowType(Schema schema, TableSchema requestedSchema) { + if (requestedSchema != null) { + // Convert the flink schema to iceberg schema firstly, then reassign ids to match the existing + // iceberg schema. + Schema writeSchema = TypeUtil.reassignIds(FlinkSchemaUtil.convert(requestedSchema), schema); + TypeUtil.validateWriteSchema(schema, writeSchema, true, true); + + // We use this flink schema to read values from RowData. The flink's TINYINT and SMALLINT will + // be promoted to iceberg INTEGER, that means if we use iceberg's table schema to read TINYINT + // (backend by 1 'byte'), we will read 4 bytes rather than 1 byte, it will mess up the byte + // array in BinaryRowData. So here we must use flink schema. + return (RowType) requestedSchema.toRowDataType().getLogicalType(); + } else { + return FlinkSchemaUtil.convert(schema); + } + } + + /** + * Based on the {@link FileFormat} overwrites the table level compression properties for the table + * write. + * + * @param table The table to get the table level settings + * @param format The FileFormat to use + * @param conf The write configuration + * @return The properties to use for writing + */ + private static Map writeProperties( + Table table, FileFormat format, FlinkWriteConf conf) { + Map writeProperties = Maps.newHashMap(table.properties()); + + switch (format) { + case PARQUET: + writeProperties.put(PARQUET_COMPRESSION, conf.parquetCompressionCodec()); + String parquetCompressionLevel = conf.parquetCompressionLevel(); + if (parquetCompressionLevel != null) { + writeProperties.put(PARQUET_COMPRESSION_LEVEL, parquetCompressionLevel); + } + + break; + case AVRO: + writeProperties.put(AVRO_COMPRESSION, conf.avroCompressionCodec()); + String avroCompressionLevel = conf.avroCompressionLevel(); + if (avroCompressionLevel != null) { + writeProperties.put(AVRO_COMPRESSION_LEVEL, conf.avroCompressionLevel()); + } + + break; + case ORC: + writeProperties.put(ORC_COMPRESSION, conf.orcCompressionCodec()); + writeProperties.put(ORC_COMPRESSION_STRATEGY, conf.orcCompressionStrategy()); + break; + default: + throw new IllegalArgumentException(String.format("Unknown file format %s", format)); + } + + return writeProperties; + } + + private DataStream distributeDataStream(DataStream input) { + DistributionMode mode = flinkWriteConf.distributionMode(); + Schema schema = table.schema(); + PartitionSpec spec = table.spec(); + LOG.info("Write distribution mode is '{}'", mode.modeName()); + switch (mode) { + case NONE: + if (equalityFieldIds.isEmpty()) { + return input; + } else { + LOG.info("Distribute rows by equality fields, because there are equality fields set"); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + case HASH: + if (equalityFieldIds.isEmpty()) { + if (table.spec().isUnpartitioned()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and table is unpartitioned"); + return input; + } else { + if (BucketPartitionerUtil.hasOneBucketField(spec)) { + return input.partitionCustom( + new BucketPartitioner(spec), + new BucketPartitionKeySelector(spec, schema, flinkRowType)); + } else { + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + } else { + if (spec.isUnpartitioned()) { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and table is unpartitioned"); + return input.keyBy( + new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } else { + for (PartitionField partitionField : spec.fields()) { + Preconditions.checkState( + equalityFieldIds.contains(partitionField.sourceId()), + "In 'hash' distribution mode with equality fields set, partition field '%s' " + + "should be included in equality fields: '%s'", + partitionField, + equalityFieldColumns); + } + return input.keyBy(new PartitionKeySelector(spec, schema, flinkRowType)); + } + } + + case RANGE: + if (equalityFieldIds.isEmpty()) { + LOG.warn( + "Fallback to use 'none' distribution mode, because there are no equality fields set " + + "and {}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input; + } else { + LOG.info( + "Distribute rows by equality fields, because there are equality fields set " + + "and{}=range is not supported yet in flink", + WRITE_DISTRIBUTION_MODE); + return input.keyBy(new EqualityFieldKeySelector(schema, flinkRowType, equalityFieldIds)); + } + + default: + throw new RuntimeException("Unrecognized " + WRITE_DISTRIBUTION_MODE + ": " + mode); + } + } + + /** + * Initialize a {@link Builder} to export the data from generic input data stream into iceberg + * table. We use {@link RowData} inside the sink connector, so users need to provide a mapper + * function and a {@link TypeInformation} to convert those generic records to a RowData + * DataStream. + * + * @param input the generic source input data stream. + * @param mapper function to convert the generic data to {@link RowData} + * @param outputType to define the {@link TypeInformation} for the input data. + * @param the data type of records. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder builderFor( + DataStream input, MapFunction mapper, TypeInformation outputType) { + return new Builder().forMapperOutputType(input, mapper, outputType); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link Row}s into + * iceberg table. We use {@link RowData} inside the sink connector, so users need to provide a + * {@link TableSchema} for builder to convert those {@link Row}s to a {@link RowData} DataStream. + * + * @param input the source input data stream with {@link Row}s. + * @param tableSchema defines the {@link TypeInformation} for input data. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRow(DataStream input, TableSchema tableSchema) { + return new Builder().forRow(input, tableSchema); + } + + /** + * Initialize a {@link Builder} to export the data from input data stream with {@link RowData}s + * into iceberg table. + * + * @param input the source input data stream with {@link RowData}s. + * @return {@link Builder} to connect the iceberg table. + */ + public static Builder forRowData(DataStream input) { + return new Builder().forRowData(input); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.java new file mode 100644 index 000000000000..7234cf74020e --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergSinkWriter.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.flink.sink; + +import java.io.IOException; +import java.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.table.data.RowData; +import org.apache.iceberg.io.TaskWriter; +import org.apache.iceberg.io.WriteResult; +import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Iceberg writer implementation for the {@link SinkWriter} interface. Used by the {@link + * org.apache.iceberg.flink.sink.IcebergSink} (SinkV2). Writes out the data to the final place, and + * emits a single {@link WriteResult} at every checkpoint for every data/delete file created by this + * writer. + */ +class IcebergSinkWriter implements CommittingSinkWriter { + private static final Logger LOG = LoggerFactory.getLogger(IcebergSinkWriter.class); + + private final String fullTableName; + private final TaskWriterFactory taskWriterFactory; + private final IcebergStreamWriterMetrics metrics; + private TaskWriter writer; + private final int subTaskId; + private final int attemptId; + + IcebergSinkWriter( + String fullTableName, + TaskWriterFactory taskWriterFactory, + IcebergStreamWriterMetrics metrics, + int subTaskId, + int attemptId) { + this.fullTableName = fullTableName; + this.taskWriterFactory = taskWriterFactory; + // Initialize the task writer factory. + taskWriterFactory.initialize(subTaskId, attemptId); + // Initialize the task writer. + this.writer = taskWriterFactory.create(); + this.metrics = metrics; + this.subTaskId = subTaskId; + this.attemptId = attemptId; + LOG.debug( + "Created Stream Writer for table {} subtask {} attemptId {}", + fullTableName, + subTaskId, + attemptId); + } + + @Override + public void write(RowData element, Context context) throws IOException, InterruptedException { + writer.write(element); + } + + @Override + public void flush(boolean endOfInput) { + // flush is used to handle flush/endOfInput, so no action is taken here. + } + + @Override + public void close() throws Exception { + if (writer != null) { + writer.close(); + } + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) + .toString(); + } + + @Override + public Collection prepareCommit() throws IOException { + long startNano = System.nanoTime(); + WriteResult result = writer.complete(); + this.writer = taskWriterFactory.create(); + metrics.updateFlushResult(result); + metrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); + LOG.debug( + "Iceberg writer subtask {} attempt {} flushed {} data files and {} delete files", + subTaskId, + attemptId, + result.dataFiles().length, + result.deleteFiles().length); + return Lists.newArrayList(result); + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..7d86baa14fc2 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -95,9 +95,9 @@ public void endInput() throws IOException { @Override public String toString() { return MoreObjects.toStringHelper(this) - .add("table_name", fullTableName) - .add("subtask_id", subTaskId) - .add("attempt_id", attemptId) + .add("tableName", fullTableName) + .add("subTaskId", subTaskId) + .add("attemptId", attemptId) .toString(); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java new file mode 100644 index 000000000000..794ade577976 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergWriteAggregator.java @@ -0,0 +1,127 @@ +/* + * 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.io.IOException; +import java.util.Collection; +import org.apache.flink.core.io.SimpleVersionedSerialization; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.operators.AbstractStreamOperator; +import org.apache.flink.streaming.api.operators.OneInputStreamOperator; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.iceberg.Table; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.io.WriteResult; +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; + +/** + * Operator which aggregates the individual {@link WriteResult} objects) to a single {@link + * IcebergCommittable} per checkpoint (storing the serialized {@link + * org.apache.iceberg.flink.sink.DeltaManifests}, jobId, operatorId, checkpointId) + */ +class IcebergWriteAggregator extends AbstractStreamOperator> + implements OneInputStreamOperator< + CommittableMessage, CommittableMessage> { + private static final Logger LOG = LoggerFactory.getLogger(IcebergWriteAggregator.class); + private static final byte[] EMPTY_MANIFEST_DATA = new byte[0]; + private final Collection results; + private transient ManifestOutputFileFactory icebergManifestOutputFileFactory; + private transient Table table; + private final TableLoader tableLoader; + + IcebergWriteAggregator(TableLoader tableLoader) { + this.results = Sets.newHashSet(); + this.tableLoader = tableLoader; + } + + @Override + public void open() throws Exception { + if (!tableLoader.isOpen()) { + tableLoader.open(); + } + + String flinkJobId = getContainingTask().getEnvironment().getJobID().toString(); + String operatorId = getOperatorID().toString(); + int subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask(); + Preconditions.checkArgument( + subTaskId == 0, "The subTaskId must be zero in the IcebergWriteAggregator"); + int attemptId = getRuntimeContext().getTaskInfo().getAttemptNumber(); + this.table = tableLoader.loadTable(); + + this.icebergManifestOutputFileFactory = + FlinkManifestUtil.createOutputFileFactory( + () -> table, table.properties(), flinkJobId, operatorId, subTaskId, attemptId); + } + + @Override + public void finish() throws IOException { + prepareSnapshotPreBarrier(Long.MAX_VALUE); + } + + @Override + public void prepareSnapshotPreBarrier(long checkpointId) throws IOException { + IcebergCommittable committable = + new IcebergCommittable( + writeToManifest(results, checkpointId), + getContainingTask().getEnvironment().getJobID().toString(), + getRuntimeContext().getOperatorUniqueID(), + checkpointId); + CommittableMessage summary = + new CommittableSummary<>(0, 1, checkpointId, 1, 1, 0); + output.collect(new StreamRecord<>(summary)); + CommittableMessage message = + new CommittableWithLineage<>(committable, checkpointId, 0); + output.collect(new StreamRecord<>(message)); + LOG.info("Emitted commit message to downstream committer operator"); + results.clear(); + } + + /** + * Write all the completed data files to a newly created manifest file and return the manifest's + * avro serialized bytes. + */ + public byte[] writeToManifest(Collection writeResults, long checkpointId) + throws IOException { + if (writeResults.isEmpty()) { + return EMPTY_MANIFEST_DATA; + } + + WriteResult result = WriteResult.builder().addAll(writeResults).build(); + DeltaManifests deltaManifests = + FlinkManifestUtil.writeCompletedFiles( + result, () -> icebergManifestOutputFileFactory.create(checkpointId), table.spec()); + + return SimpleVersionedSerialization.writeVersionAndSerialize( + DeltaManifestsSerializer.INSTANCE, deltaManifests); + } + + @Override + public void processElement(StreamRecord> element) + throws Exception { + + if (element.isRecord() && element.getValue() instanceof CommittableWithLineage) { + results.add(((CommittableWithLineage) element.getValue()).getCommittable()); + } + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..1cb7f4dea1e8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -26,13 +26,13 @@ import org.apache.iceberg.Table; import org.apache.iceberg.TableOperations; import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; import org.apache.iceberg.relocated.com.google.common.base.Strings; class ManifestOutputFileFactory { // Users could define their own flink manifests directory by setting this value in table // properties. - static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; - + @VisibleForTesting static final String FLINK_MANIFEST_LOCATION = "flink.manifests.location"; private final Supplier
    tableSupplier; private final Map props; private final String flinkJobId; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java new file mode 100644 index 000000000000..7f28a50ecaa8 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/SinkUtil.java @@ -0,0 +1,94 @@ +/* + * 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 java.util.Set; +import org.apache.flink.util.Preconditions; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class SinkUtil { + + private static final long INITIAL_CHECKPOINT_ID = -1L; + + public static final String FLINK_JOB_ID = "flink.job-id"; + + public static final String OPERATOR_ID = "flink.operator-id"; + public static final String MAX_COMMITTED_CHECKPOINT_ID = "flink.max-committed-checkpoint-id"; + + private SinkUtil() {} + + private static final Logger LOG = LoggerFactory.getLogger(SinkUtil.class); + + static List checkAndGetEqualityFieldIds(Table table, List equalityFieldColumns) { + List equalityFieldIds = Lists.newArrayList(table.schema().identifierFieldIds()); + if (equalityFieldColumns != null && !equalityFieldColumns.isEmpty()) { + Set equalityFieldSet = Sets.newHashSetWithExpectedSize(equalityFieldColumns.size()); + for (String column : equalityFieldColumns) { + org.apache.iceberg.types.Types.NestedField field = table.schema().findField(column); + Preconditions.checkNotNull( + field, + "Missing required equality field column '%s' in table schema %s", + column, + table.schema()); + equalityFieldSet.add(field.fieldId()); + } + + if (!equalityFieldSet.equals(table.schema().identifierFieldIds())) { + LOG.warn( + "The configured equality field column IDs {} are not matched with the schema identifier field IDs" + + " {}, use job specified equality field columns as the equality fields by default.", + equalityFieldSet, + table.schema().identifierFieldIds()); + } + equalityFieldIds = Lists.newArrayList(equalityFieldSet); + } + return equalityFieldIds; + } + + static long getMaxCommittedCheckpointId( + Table table, String flinkJobId, String operatorId, String branch) { + Snapshot snapshot = table.snapshot(branch); + long lastCommittedCheckpointId = INITIAL_CHECKPOINT_ID; + + while (snapshot != null) { + Map summary = snapshot.summary(); + String snapshotFlinkJobId = summary.get(FLINK_JOB_ID); + String snapshotOperatorId = summary.get(OPERATOR_ID); + if (flinkJobId.equals(snapshotFlinkJobId) + && (snapshotOperatorId == null || snapshotOperatorId.equals(operatorId))) { + String value = summary.get(MAX_COMMITTED_CHECKPOINT_ID); + if (value != null) { + lastCommittedCheckpointId = Long.parseLong(value); + break; + } + } + Long parentSnapshotId = snapshot.parentId(); + snapshot = parentSnapshotId != null ? table.snapshot(parentSnapshotId) : null; + } + + return lastCommittedCheckpointId; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java new file mode 100644 index 000000000000..5a44373cccaa --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/WriteResultSerializer.java @@ -0,0 +1,61 @@ +/* + * 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.io.ByteArrayOutputStream; +import java.io.IOException; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.core.memory.DataInputDeserializer; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.util.InstantiationUtil; +import org.apache.iceberg.io.WriteResult; + +class WriteResultSerializer implements SimpleVersionedSerializer { + private static final int VERSION = 1; + + @Override + public int getVersion() { + return VERSION; + } + + @Override + public byte[] serialize(WriteResult writeResult) throws IOException { + ByteArrayOutputStream out = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper view = new DataOutputViewStreamWrapper(out); + byte[] result = InstantiationUtil.serializeObject(writeResult); + view.write(result); + return out.toByteArray(); + } + + @Override + public WriteResult deserialize(int version, byte[] serialized) throws IOException { + if (version == 1) { + DataInputDeserializer view = new DataInputDeserializer(serialized); + byte[] resultBuf = new byte[serialized.length]; + view.read(resultBuf); + try { + return InstantiationUtil.deserializeObject( + resultBuf, IcebergCommittableSerializer.class.getClassLoader()); + } catch (ClassNotFoundException cnc) { + throw new IOException("Could not deserialize the WriteResult object", cnc); + } + } + throw new IOException("Unrecognized version or corrupt state: " + version); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.java new file mode 100644 index 000000000000..1cf55bcdc817 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/SinkTestUtil.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.flink.sink; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.Collection; +import java.util.List; +import java.util.stream.Collectors; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; + +class SinkTestUtil { + + private SinkTestUtil() {} + + @SuppressWarnings("unchecked") + static List transformsToStreamElement(Collection elements) { + return elements.stream() + .map( + element -> { + if (element instanceof StreamRecord) { + return new StreamRecord<>( + ((StreamRecord>) element).getValue()); + } + return (StreamElement) element; + }) + .collect(Collectors.toList()); + } + + static CommittableSummary extractAndAssertCommittableSummary(StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableSummary.class); + return (CommittableSummary) value; + } + + static CommittableWithLineage extractAndAssertCommittableWithLineage( + StreamElement element) { + final Object value = element.asRecord().getValue(); + assertThat(value).isInstanceOf(CommittableWithLineage.class); + return (CommittableWithLineage) value; + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java index fc33c2fea5e6..da4ac2ada677 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkV2Base.java @@ -48,30 +48,30 @@ import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.StructLikeSet; -public class TestFlinkIcebergSinkV2Base { +class TestFlinkIcebergSinkV2Base { - protected static final int FORMAT_V2 = 2; - protected static final TypeInformation ROW_TYPE_INFO = + static final int FORMAT_V2 = 2; + static final TypeInformation ROW_TYPE_INFO = new RowTypeInfo(SimpleDataUtil.FLINK_SCHEMA.getFieldTypes()); - protected static final int ROW_ID_POS = 0; - protected static final int ROW_DATA_POS = 1; + static final int ROW_ID_POS = 0; + static final int ROW_DATA_POS = 1; - protected TableLoader tableLoader; - protected Table table; - protected StreamExecutionEnvironment env; + TableLoader tableLoader; + Table table; + StreamExecutionEnvironment env; @Parameter(index = 0) - protected FileFormat format; + FileFormat format; @Parameter(index = 1) - protected int parallelism = 1; + int parallelism = 1; @Parameter(index = 2) - protected boolean partitioned; + boolean partitioned; @Parameter(index = 3) - protected String writeDistributionMode; + String writeDistributionMode; @Parameters(name = "FileFormat={0}, Parallelism={1}, Partitioned={2}, WriteDistributionMode={3}") public static Object[][] parameters() { @@ -91,14 +91,14 @@ public static Object[][] parameters() { }; } - protected static final Map ROW_KIND_MAP = + static final Map ROW_KIND_MAP = ImmutableMap.of( "+I", RowKind.INSERT, "-D", RowKind.DELETE, "-U", RowKind.UPDATE_BEFORE, "+U", RowKind.UPDATE_AFTER); - protected Row row(String rowKind, int id, String data) { + Row row(String rowKind, int id, String data) { RowKind kind = ROW_KIND_MAP.get(rowKind); if (kind == null) { throw new IllegalArgumentException("Unknown row kind: " + rowKind); @@ -107,7 +107,7 @@ protected Row row(String rowKind, int id, String data) { return Row.ofKind(kind, id, data); } - protected void testUpsertOnIdDataKey(String branch) throws Exception { + void testUpsertOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "aaa"), row("+I", 2, "bbb")), @@ -128,7 +128,7 @@ protected void testUpsertOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdDataKey(String branch) throws Exception { + void testChangeLogOnIdDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -157,7 +157,7 @@ protected void testChangeLogOnIdDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnSameKey(String branch) throws Exception { + void testChangeLogOnSameKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( // Checkpoint #1 @@ -185,7 +185,7 @@ protected void testChangeLogOnSameKey(String branch) throws Exception { branch); } - protected void testChangeLogOnDataKey(String branch) throws Exception { + void testChangeLogOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -213,7 +213,7 @@ protected void testChangeLogOnDataKey(String branch) throws Exception { branch); } - protected void testUpsertOnDataKey(String branch) throws Exception { + void testUpsertOnDataKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+I", 2, "aaa"), row("+I", 3, "bbb")), @@ -235,7 +235,7 @@ protected void testUpsertOnDataKey(String branch) throws Exception { branch); } - protected void testChangeLogOnIdKey(String branch) throws Exception { + void testChangeLogOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of( @@ -285,7 +285,7 @@ protected void testChangeLogOnIdKey(String branch) throws Exception { } } - protected void testUpsertOnIdKey(String branch) throws Exception { + void testUpsertOnIdKey(String branch) throws Exception { List> elementsPerCheckpoint = ImmutableList.of( ImmutableList.of(row("+I", 1, "aaa"), row("+U", 1, "bbb")), @@ -321,7 +321,7 @@ protected void testUpsertOnIdKey(String branch) throws Exception { } } - protected void testChangeLogs( + void testChangeLogs( List equalityFieldColumns, KeySelector keySelector, boolean insertAsUpsert, @@ -358,11 +358,11 @@ protected void testChangeLogs( } } - protected Record record(int id, String data) { + Record record(int id, String data) { return SimpleDataUtil.createRecord(id, data); } - private List findValidSnapshots() { + List findValidSnapshots() { List validSnapshots = Lists.newArrayList(); for (Snapshot snapshot : table.snapshots()) { if (snapshot.allManifests(table.io()).stream() @@ -373,11 +373,11 @@ private List findValidSnapshots() { return validSnapshots; } - private StructLikeSet expectedRowSet(Record... records) { + StructLikeSet expectedRowSet(Record... records) { return SimpleDataUtil.expectedRowSet(table, records); } - private StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { + StructLikeSet actualRowSet(long snapshotId, String... columns) throws IOException { table.refresh(); StructLikeSet set = StructLikeSet.create(table.schema().asStruct()); try (CloseableIterable reader = 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 0b0c55f51c32..a2e9be9303c3 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 @@ -43,7 +43,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestFlinkIcebergSinkV2Branch extends TestFlinkIcebergSinkV2Base { @RegisterExtension - private static final HadoopCatalogExtension CATALOG_EXTENSION = + static final HadoopCatalogExtension CATALOG_EXTENSION = new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); @Parameter(index = 0) diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java index 53b7c4c0cc91..c21c3d5cc21b 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkManifest.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.flink.sink; -import static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; import static org.assertj.core.api.Assertions.assertThat; import java.io.File; @@ -134,7 +133,9 @@ public void testUserProvidedManifestLocation() throws IOException { String operatorId = newOperatorUniqueId(); File userProvidedFolder = Files.createTempDirectory(temporaryFolder, "junit").toFile(); Map props = - ImmutableMap.of(FLINK_MANIFEST_LOCATION, userProvidedFolder.getAbsolutePath() + "///"); + ImmutableMap.of( + ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION, + userProvidedFolder.getAbsolutePath() + "///"); ManifestOutputFileFactory factory = new ManifestOutputFileFactory(() -> table, props, flinkJobId, operatorId, 1, 1); diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java new file mode 100644 index 000000000000..abdb2b5c79ae --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -0,0 +1,1445 @@ +/* + * 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 static org.apache.iceberg.flink.sink.ManifestOutputFileFactory.FLINK_MANIFEST_LOCATION; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableSummary; +import static org.apache.iceberg.flink.sink.SinkTestUtil.extractAndAssertCommittableWithLineage; +import static org.apache.iceberg.flink.sink.SinkTestUtil.transformsToStreamElement; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assumptions.assumeThat; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.spy; + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.flink.api.common.JobID; +import org.apache.flink.api.common.TaskInfo; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.api.common.typeutils.TypeSerializerSnapshot; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; +import org.apache.flink.runtime.execution.Environment; +import org.apache.flink.runtime.jobgraph.OperatorID; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessageSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableSummary; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SinkV2Assertions; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.runtime.operators.sink.CommitterOperatorFactory; +import org.apache.flink.streaming.runtime.streamrecord.StreamElement; +import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; +import org.apache.flink.streaming.runtime.tasks.StreamTask; +import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; +import org.apache.flink.table.data.RowData; +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.FileFormat; +import org.apache.iceberg.GenericManifestFile; +import org.apache.iceberg.ManifestContent; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.Parameter; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.Parameters; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.SnapshotSummary; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestBase; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestHelpers; +import org.apache.iceberg.io.FileAppenderFactory; +import org.apache.iceberg.io.WriteResult; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(ParameterizedTestExtension.class) +class TestIcebergCommitter extends TestBase { + private static final Logger LOG = LoggerFactory.getLogger(TestIcebergCommitter.class); + public static final String OPERATOR_ID = "flink-sink"; + @TempDir File temporaryFolder; + + @TempDir File flinkManifestFolder; + + private Table table; + + private TableLoader tableLoader; + + @Parameter(index = 1) + private Boolean isStreamingMode; + + @Parameter(index = 2) + private String branch; + + private final String jobId = "jobId"; + private final long dataFIleRowCount = 5L; + + private final TestCommittableMessageTypeSerializer committableMessageTypeSerializer = + new TestCommittableMessageTypeSerializer(); + + private final DataFile dataFileTest1 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-1.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + private final DataFile dataFileTest2 = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-2.parquet") + .withFileSizeInBytes(0) + .withMetrics( + new Metrics( + dataFIleRowCount, + null, // no column sizes + ImmutableMap.of(1, 5L), // value count + ImmutableMap.of(1, 0L), // null count + null, + ImmutableMap.of(1, longToBuffer(0L)), // lower bounds + ImmutableMap.of(1, longToBuffer(4L)) // upper bounds + )) + .build(); + + @SuppressWarnings("checkstyle:NestedForDepth") + @Parameters(name = "formatVersion={0} isStreaming={1}, branch={2}") + protected static List parameters() { + List parameters = Lists.newArrayList(); + for (Boolean isStreamingMode : new Boolean[] {true, false}) { + for (int formatVersion : new int[] {1, 2}) { + parameters.add(new Object[] {formatVersion, isStreamingMode, SnapshotRef.MAIN_BRANCH}); + parameters.add(new Object[] {formatVersion, isStreamingMode, "test-branch"}); + } + } + return parameters; + } + + @BeforeEach + public void before() throws Exception { + String warehouse = temporaryFolder.getAbsolutePath(); + + String tablePath = warehouse.concat("/test"); + assertThat(new File(tablePath).mkdir()).as("Should create the table path correctly.").isTrue(); + + Map props = + ImmutableMap.of( + TableProperties.FORMAT_VERSION, + String.valueOf(formatVersion), + FLINK_MANIFEST_LOCATION, + flinkManifestFolder.getAbsolutePath(), + IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, + "1"); + table = SimpleDataUtil.createTable(tablePath, props, false); + tableLoader = TableLoader.fromHadoopTable(tablePath); + } + + @TestTemplate + public void testCommitTxnWithoutDataFiles() throws Exception { + IcebergCommitter committer = getCommitter(); + SimpleDataUtil.assertTableRows(table, Lists.newArrayList(), branch); + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1); + + for (long i = 1; i <= 3; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertMaxCommittedCheckpointId(jobId, i); + assertSnapshotSize((int) i); + } + } + + @TestTemplate + public void testMxContinuousEmptyCommits() throws Exception { + table.updateProperties().set(IcebergCommitter.MAX_CONTINUOUS_EMPTY_COMMITS, "3").commit(); + IcebergCommitter committer = getCommitter(); + for (int i = 1; i <= 9; i++) { + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList()); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + assertSnapshotSize(i / 3); + } + } + + @TestTemplate + public void testCommitTxn() throws Exception { + IcebergCommitter committer = getCommitter(); + assertSnapshotSize(0); + List rows = Lists.newArrayListWithExpectedSize(3); + for (int i = 1; i <= 3; i++) { + RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); + DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); + rows.add(rowData); + WriteResult writeResult = of(dataFile); + Committer.CommitRequest commitRequest = + buildCommitRequestFor(jobId, i, Lists.newArrayList(writeResult)); + committer.commit(Lists.newArrayList(commitRequest)); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.copyOf(rows), branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(jobId, i); + Map summary = SimpleDataUtil.latestSnapshot(table, branch).summary(); + assertThat(summary) + .containsEntry("flink.test", "org.apache.iceberg.flink.sink.TestIcebergCommitter") + .containsEntry("added-data-files", "1") + .containsEntry("flink.operator-id", OPERATOR_ID) + .containsEntry("flink.job-id", "jobId"); + } + } + + @TestTemplate + public void testOrderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#1; + // 4. notifyCheckpointComplete for checkpoint#2; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + OperatorSubtaskState snapshot = harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobId, firstCheckpointId); + assertFlinkManifests(1); + + // 4. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testDisorderedEventsBetweenCheckpoints() throws Exception { + // It's possible that two checkpoints happen in the following orders: + // 1. snapshotState for checkpoint#1; + // 2. snapshotState for checkpoint#2; + // 3. notifyCheckpointComplete for checkpoint#2; + // 4. notifyCheckpointComplete for checkpoint#1; + + long timestamp = 0; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + processElement(jobId, 1, harness, 1, OPERATOR_ID, dataFile1); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 1. snapshotState for checkpoint#1 + long firstCheckpointId = 1; + harness.snapshot(firstCheckpointId, ++timestamp); + assertFlinkManifests(1); + + RowData row2 = SimpleDataUtil.createRowData(2, "world"); + DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); + processElement(jobId, 2, harness, 1, OPERATOR_ID, dataFile2); + assertMaxCommittedCheckpointId(jobId, -1L); + + // 2. snapshotState for checkpoint#2 + long secondCheckpointId = 2; + harness.snapshot(secondCheckpointId, ++timestamp); + assertFlinkManifests(2); + + // 3. notifyCheckpointComplete for checkpoint#2 + harness.notifyOfCompletedCheckpoint(secondCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + + // 4. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(firstCheckpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2), branch); + assertMaxCommittedCheckpointId(jobId, secondCheckpointId); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testSingleCommit() throws Exception { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + testHarness.open(); + + long checkpointId = 1; + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + CommittableSummary committableSummary = + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // Trigger commit + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, 1L); + + List output = transformsToStreamElement(testHarness.getOutput()); + + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasSubtaskId(0) + .hasCheckpointId(checkpointId); + } + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, "1") + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1"); + } + + /** The data was not committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithUncommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + CommittableSummary committableSummary; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + snapshot = preJobTestHarness.snapshot(checkpointId, ++timestamp); + + assertThat(preJobTestHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(committableMessageTypeSerializer); + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + + table.refresh(); + + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + checkpointId++; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + + SinkV2Assertions.assertThat(extractAndAssertCommittableWithLineage(output2.get(1))) + .hasCheckpointId(0L) + .hasSubtaskId(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + /** The data was committed in the previous job. */ + @TestTemplate + public void testStateRestoreFromPreJobWithCommitted() throws Exception { + String jobId1 = "jobId1"; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + preJobTestHarness = getTestHarness()) { + + preJobTestHarness.open(); + + CommittableSummary committableSummary = + processElement(jobId1, checkpointId, preJobTestHarness, 1, OPERATOR_ID, dataFileTest1); + + assertFlinkManifests(1); + snapshot = preJobTestHarness.snapshot(checkpointId, 2L); + // commit snapshot + preJobTestHarness.notifyOfCompletedCheckpoint(checkpointId); + + List output = transformsToStreamElement(preJobTestHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId1, checkpointId); + } + + table.refresh(); + long preJobSnapshotId = table.snapshot(branch).snapshotId(); + + String jobId2 = "jobId2"; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + restored.setup(); + restored.initializeState(snapshot); + restored.open(); + + // Makes sure that data committed in the previous job is available in this job + List output2 = transformsToStreamElement(restored.getOutput()); + assertThat(output2).hasSize(2); + + table.refresh(); + long restoredSnapshotId = table.snapshot(branch).snapshotId(); + + assertThat(restoredSnapshotId) + .as("The table does not generate a new snapshot without data being committed.") + .isEqualTo(preJobSnapshotId); + + assertThat(table.snapshot(branch).summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + // Commit new data file + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output3 = transformsToStreamElement(restored.getOutput()); + assertThat(output3).hasSize(4); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + } + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(Long.parseLong(currentSnapshot2.summary().get(SnapshotSummary.TOTAL_RECORDS_PROP))) + .isEqualTo(dataFIleRowCount * 2); + + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + + @TestTemplate + public void testStateRestoreFromCurrJob() throws Exception { + String jobId1 = "jobId1"; + CommittableSummary committableSummary; + OperatorSubtaskState snapshot; + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + committableSummary = + processElement(jobId1, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + snapshot = testHarness.snapshot(checkpointId, 2L); + + assertThat(testHarness.getOutput()).isEmpty(); + } + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId1, -1L); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + restored = getTestHarness()) { + + restored.setup(committableMessageTypeSerializer); + + restored.initializeState(snapshot); + restored.open(); + + // Previous committables are immediately committed if possible + List output = transformsToStreamElement(restored.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasFailedCommittables(committableSummary.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary.getNumberOfCommittables()) + .hasPendingCommittables(0); + + table.refresh(); + Snapshot currentSnapshot = table.snapshot(branch); + + assertThat(currentSnapshot.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "1") + .containsEntry("flink.job-id", jobId1); + + String jobId2 = "jobId2"; + checkpointId = 1; + CommittableSummary committableSummary2 = + processElement(jobId2, checkpointId, restored, 1, OPERATOR_ID, dataFileTest2); + + // Trigger commit + restored.notifyOfCompletedCheckpoint(checkpointId); + + List output2 = transformsToStreamElement(restored.getOutput()); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(0))) + .hasFailedCommittables(committableSummary2.getNumberOfFailedCommittables()) + .hasOverallCommittables(committableSummary2.getNumberOfCommittables()) + .hasPendingCommittables(0); + restored.close(); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId2, 1L); + + table.refresh(); + Snapshot currentSnapshot2 = table.snapshot(branch); + assertThat(currentSnapshot2.summary()) + .containsEntry(SnapshotSummary.TOTAL_RECORDS_PROP, String.valueOf(dataFIleRowCount * 2)) + .containsEntry(SnapshotSummary.TOTAL_DATA_FILES_PROP, "2") + .containsEntry("flink.job-id", jobId2); + } + } + + @TestTemplate + public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Exception { + // We've two steps in checkpoint: 1. snapshotState(ckp); 2. notifyCheckpointComplete(ckp). + // The Flink job should be able to restore from a checkpoint with only step#1 finished. + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + OperatorSubtaskState snapshot; + List expectedRows = Lists.newArrayList(); + + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobId, -1L); + + RowData row = SimpleDataUtil.createRowData(1, "hello"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); + assertMaxCommittedCheckpointId(jobId, -1L); + assertFlinkManifests(1); + } + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0L); + + harness.snapshot(++checkpointId, ++timestamp); + // Did not write any new record, so it won't generate new manifest. + assertFlinkManifests(0); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(1); + + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 0); + + RowData row = SimpleDataUtil.createRowData(2, "world"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); + processElement(jobId, checkpointId, harness, 1, operatorId.toString(), dataFile); + + snapshot = harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(0); + } + + // Redeploying flink job from external checkpoint. + JobID newJobId = new JobID(); + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + harness.initializeState(snapshot); + harness.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), -1); + assertMaxCommittedCheckpointId(jobId, operatorId.toString(), 2); + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + + RowData row = SimpleDataUtil.createRowData(3, "foo"); + expectedRows.add(row); + DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); + processElement( + newJobId.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(++checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(newJobId.toString(), operatorId.toString(), 3); + } + } + + @TestTemplate + public void testStartAnotherJobToWriteSameTable() throws Exception { + long checkpointId = 1; + long timestamp = 0; + + List rows = Lists.newArrayList(); + List tableRows = Lists.newArrayList(); + + JobID oldJobId = new JobID(); + OperatorID oldOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + oldOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), -1L); + + for (int i = 1; i <= 3; i++) { + rows.add(SimpleDataUtil.createRowData(i, "hello" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + processElement( + oldJobId.toString(), ++checkpointId, harness, 1, oldOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), checkpointId); + } + } + + // The new started job will start with checkpoint = 1 again. + checkpointId = 1; + JobID newJobId = new JobID(); + OperatorID newOperatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.open(); + newOperatorId = harness.getOperator().getOperatorID(); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(oldJobId.toString(), oldOperatorId.toString(), 4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), -1); + + rows.add(SimpleDataUtil.createRowData(2, "world")); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile("data-new-1", rows); + processElement( + newJobId.toString(), checkpointId, harness, 1, newOperatorId.toString(), dataFile); + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(newJobId.toString(), newOperatorId.toString(), checkpointId); + } + } + + @TestTemplate + public void testMultipleJobsWriteSameTable() throws Exception { + long timestamp = 0; + List tableRows = Lists.newArrayList(); + + JobID[] jobs = new JobID[] {new JobID(), new JobID(), new JobID()}; + OperatorID[] operatorIds = + new OperatorID[] {new OperatorID(), new OperatorID(), new OperatorID()}; + for (int i = 0; i < 20; i++) { + int jobIndex = i % 3; + int checkpointId = i / 3; + JobID jobID = jobs[jobIndex]; + OperatorID operatorId = operatorIds[jobIndex]; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + harness.getStreamConfig().setOperatorID(operatorId); + + harness.open(); + + assertSnapshotSize(i); + assertMaxCommittedCheckpointId( + jobID.toString(), operatorId.toString(), checkpointId == 0 ? -1 : checkpointId - 1); + + List rows = Lists.newArrayList(SimpleDataUtil.createRowData(i, "word-" + i)); + tableRows.addAll(rows); + + DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); + + processElement(jobID.toString(), checkpointId, harness, 1, operatorId.toString(), dataFile); + + harness.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(1); + + harness.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + SimpleDataUtil.assertTableRows(table, tableRows, branch); + assertSnapshotSize(i + 1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpointId); + } + } + } + + @TestTemplate + public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { + + // We cannot test a different checkpoint thant 0 because when using the OperatorTestHarness + // for recovery the lastCompleted checkpoint is always reset to 0. + // see: https://github.com/apache/iceberg/issues/10942 + long checkpointId = 0; + long timestamp = 0; + List expectedRows = Lists.newArrayList(); + OperatorSubtaskState snapshot1; + OperatorSubtaskState snapshot2; + + JobID jobID = new JobID(); + OperatorID operatorId1 = new OperatorID(); + OperatorID operatorId2 = new OperatorID(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness()) { + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.open(); + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.open(); + + assertSnapshotSize(0); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), -1L); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + snapshot2 = harness2.snapshot(checkpointId, ++timestamp); + assertFlinkManifests(2); + + // Only notify one of the committers + harness1.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(1); + + // Only the first row is committed at this point + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), -1); + } + } + + // Restore from the given snapshot + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness1 = getTestHarness(); + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness2 = getTestHarness()) { + harness1.getStreamConfig().setOperatorID(operatorId1); + harness1.setup(); + harness1.initializeState(snapshot1); + harness1.open(); + + harness2.getStreamConfig().setOperatorID(operatorId2); + harness2.setup(); + harness2.initializeState(snapshot2); + harness2.open(); + + // All flink manifests should be cleaned because it has committed the unfinished iceberg + // transaction. + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + + RowData row1 = SimpleDataUtil.createRowData(2, "world1"); + expectedRows.add(row1); + DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); + + checkpointId++; + processElement( + jobID.toString(), checkpointId, harness1, 1, operatorId1.toString(), dataFile1); + + harness1.snapshot(checkpointId, ++timestamp); + + RowData row2 = SimpleDataUtil.createRowData(2, "world2"); + expectedRows.add(row2); + DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); + processElement( + jobID.toString(), checkpointId, harness2, 1, operatorId2.toString(), dataFile2); + + harness2.snapshot(checkpointId, ++timestamp); + + assertFlinkManifests(2); + + harness1.notifyOfCompletedCheckpoint(checkpointId); + harness2.notifyOfCompletedCheckpoint(checkpointId); + assertFlinkManifests(0); + + SimpleDataUtil.assertTableRows(table, expectedRows, branch); + assertSnapshotSize(4); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId1.toString(), checkpointId); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId2.toString(), checkpointId); + } + } + + @TestTemplate + public void testFlinkManifests() throws Exception { + long timestamp = 0; + long checkpoint = 1; + + JobID jobID = new JobID(); + OperatorID operatorId; + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + harness = getTestHarness()) { + + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + RowData row1 = SimpleDataUtil.createRowData(1, "hello"); + DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); + + // harness.processElement(of(dataFile1), ++timestamp); + processElement(jobID.toString(), checkpoint, harness, 1, operatorId.toString(), dataFile1); + + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), -1L); + + // 1. snapshotState for checkpoint#1 + harness.snapshot(checkpoint, ++timestamp); + List manifestPaths = assertFlinkManifests(1); + Path manifestPath = manifestPaths.get(0); + assertThat(manifestPath.getFileName()) + .asString() + .isEqualTo( + String.format("%s-%s-%05d-%d-%d-%05d.avro", jobID, operatorId, 0, 0, checkpoint, 1)); + // + // 2. Read the data files from manifests and assert. + List dataFiles = + FlinkManifestUtil.readDataFiles( + createTestingManifestFile(manifestPath), table.io(), table.specs()); + assertThat(dataFiles).hasSize(1); + TestHelpers.assertEquals(dataFile1, dataFiles.get(0)); + + // 3. notifyCheckpointComplete for checkpoint#1 + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + assertMaxCommittedCheckpointId(jobID.toString(), operatorId.toString(), checkpoint); + assertFlinkManifests(0); + } + } + + @TestTemplate + public void testHandleEndInput() throws Exception { + assumeThat(isStreamingMode).as("Only support batch mode").isFalse(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = Long.MAX_VALUE; + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFileTest1); + + testHarness.endInput(); + + assertMaxCommittedCheckpointId(jobId, OPERATOR_ID, Long.MAX_VALUE); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // endInput is idempotent + testHarness.endInput(); + assertThat(testHarness.getOutput()).hasSize(2); + } + } + + @TestTemplate + public void testDeleteFiles() throws Exception { + + assumeThat(formatVersion).as("Only support delete in format v2").isGreaterThanOrEqualTo(2); + + FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); + + try (OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = getTestHarness()) { + + testHarness.open(); + + long checkpointId = 1; + RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); + DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); + processElement(jobId, checkpointId, testHarness, 1, OPERATOR_ID, dataFile1); + + // testHarness.snapshot(checkpointId, 0); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(1); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + List output = transformsToStreamElement(testHarness.getOutput()); + assertThat(output).hasSize(2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output.get(0))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1), branch); + + // The 2. commit + checkpointId = 2; + RowData row2 = SimpleDataUtil.createInsert(2, "bbb"); + DataFile dataFile2 = writeDataFile("data-file-2", ImmutableList.of(row2)); + + RowData row3 = SimpleDataUtil.createInsert(3, "ccc"); + DataFile dataFile3 = writeDataFile("data-file-3", ImmutableList.of(row3)); + processElement(jobId, checkpointId, testHarness, 2, OPERATOR_ID, dataFile2, dataFile3); + + // testHarness.snapshot(checkpointId, 1); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(2); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row1, row2, row3), branch); + + List output2 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output2).hasSize(2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output2.get(2))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + + // The 3. commit + checkpointId = 3; + RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); + DeleteFile deleteFile1 = + writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + RowData row4 = SimpleDataUtil.createInsert(4, "ddd"); + DataFile dataFile4 = writeDataFile("data-file-4", ImmutableList.of(row4)); + + RowData row5 = SimpleDataUtil.createInsert(5, "eee"); + DataFile dataFile5 = writeDataFile("data-file-5", ImmutableList.of(row5)); + WriteResult withRecord4 = + WriteResult.builder() + .addDataFiles(dataFile4, dataFile5) + .addDeleteFiles(deleteFile1) + .build(); + processElement(withRecord4, jobId, checkpointId, testHarness, 2, OPERATOR_ID); + + // testHarness.snapshot(checkpointId, 3); + testHarness.notifyOfCompletedCheckpoint(checkpointId); + + assertSnapshotSize(3); + assertMaxCommittedCheckpointId(jobId, checkpointId); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(row2, row3, row4, row5), branch); + + List output3 = transformsToStreamElement(testHarness.getOutput()); + assertThat(output3).hasSize(2 + 2 + 2); + SinkV2Assertions.assertThat(extractAndAssertCommittableSummary(output3.get(4))) + .hasCheckpointId(checkpointId) + .hasPendingCommittables(0) + .hasOverallCommittables(1) + .hasFailedCommittables(0); + } + } + + private ManifestFile createTestingManifestFile(Path manifestPath) { + return new GenericManifestFile( + manifestPath.toAbsolutePath().toString(), + manifestPath.toFile().length(), + 0, + ManifestContent.DATA, + 0, + 0, + 0L, + 0, + 0, + 0, + 0, + 0, + 0, + null, + null); + } + + private IcebergWriteAggregator buildIcebergWriteAggregator(String myJobId, String operatorId) { + IcebergWriteAggregator icebergWriteAggregator = spy(new IcebergWriteAggregator(tableLoader)); + StreamTask ctx = mock(StreamTask.class); + Environment env = mock(Environment.class); + StreamingRuntimeContext streamingRuntimeContext = mock(StreamingRuntimeContext.class); + TaskInfo taskInfo = mock(TaskInfo.class); + JobID myJobID = mock(JobID.class); + OperatorID operatorID = mock(OperatorID.class); + doReturn(myJobId).when(myJobID).toString(); + doReturn(myJobID).when(env).getJobID(); + doReturn(env).when(ctx).getEnvironment(); + doReturn(ctx).when(icebergWriteAggregator).getContainingTask(); + doReturn(operatorId).when(operatorID).toString(); + doReturn(operatorID).when(icebergWriteAggregator).getOperatorID(); + doReturn(0).when(taskInfo).getAttemptNumber(); + doReturn(taskInfo).when(streamingRuntimeContext).getTaskInfo(); + doReturn(streamingRuntimeContext).when(icebergWriteAggregator).getRuntimeContext(); + + try { + icebergWriteAggregator.open(); + } catch (Exception e) { + throw new RuntimeException(e); + } + return icebergWriteAggregator; + } + + private CommittableSummary processElement( + WriteResult withRecord, + String myJobId, + long checkpointId, + OneInputStreamOperatorTestHarness testHarness, + int subTaskId, + String operatorId) + throws Exception { + + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobId, operatorId) + .writeToManifest(Lists.newArrayList(withRecord), checkpointId), + myJobId, + operatorId, + checkpointId); + + CommittableSummary committableSummary = + new CommittableSummary<>(subTaskId, 1, checkpointId, 1, 1, 0); + testHarness.processElement(new StreamRecord<>(committableSummary)); + + CommittableWithLineage committable = + new CommittableWithLineage<>(commit, checkpointId, subTaskId); + testHarness.processElement(new StreamRecord<>(committable)); + + return committableSummary; + } + + private CommittableSummary processElement( + String myJobID, + long checkpointId, + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness, + int subTaskId, + String operatorId, + DataFile... dataFile) + throws Exception { + WriteResult withRecord = WriteResult.builder().addDataFiles(dataFile).build(); + return processElement(withRecord, myJobID, checkpointId, testHarness, subTaskId, operatorId); + } + + private FileAppenderFactory createDeletableAppenderFactory() { + int[] equalityFieldIds = + new int[] { + table.schema().findField("id").fieldId(), table.schema().findField("data").fieldId() + }; + return new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + equalityFieldIds, + table.schema(), + null); + } + + private List assertFlinkManifests(int expectedCount) throws IOException { + List manifests = + Files.list(flinkManifestFolder.toPath()) + .filter(p -> !p.toString().endsWith(".crc")) + .collect(Collectors.toList()); + assertThat(manifests).hasSize(expectedCount); + return manifests; + } + + private DataFile writeDataFile(String filename, List rows) throws IOException { + return SimpleDataUtil.writeFile( + table, + table.schema(), + table.spec(), + new Configuration(), + table.location(), + FileFormat.PARQUET.addExtension(filename), + rows); + } + + private DeleteFile writeEqDeleteFile( + FileAppenderFactory appenderFactory, String filename, List deletes) + throws IOException { + return SimpleDataUtil.writeEqDeleteFile( + table, FileFormat.PARQUET, filename, appenderFactory, deletes); + } + + private OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + getTestHarness() throws Exception { + IcebergSink sink = + IcebergSink.forRowData(null).table(table).toBranch(branch).tableLoader(tableLoader).build(); + + OneInputStreamOperatorTestHarness< + CommittableMessage, CommittableMessage> + testHarness = + new OneInputStreamOperatorTestHarness<>( + new CommitterOperatorFactory<>(sink, !isStreamingMode, true)); + testHarness.setup(committableMessageTypeSerializer); + return testHarness; + } + + // ------------------------------- Utility Methods -------------------------------- + + private IcebergCommitter getCommitter() { + IcebergFilesCommitterMetrics metric = mock(IcebergFilesCommitterMetrics.class); + return new IcebergCommitter( + tableLoader, + branch, + Collections.singletonMap("flink.test", TestIcebergCommitter.class.getName()), + false, + 10, + "sinkId", + metric); + } + + private Committer.CommitRequest buildCommitRequestFor( + String myJobID, long checkpoint, Collection writeResults) throws IOException { + IcebergCommittable commit = + new IcebergCommittable( + buildIcebergWriteAggregator(myJobID, OPERATOR_ID) + .writeToManifest(writeResults, checkpoint), + myJobID, + OPERATOR_ID, + checkpoint); + + CommittableWithLineage committableWithLineage = + new CommittableWithLineage(commit, checkpoint, 1); + Committer.CommitRequest commitRequest = mock(Committer.CommitRequest.class); + + doReturn(committableWithLineage.getCommittable()).when(commitRequest).getCommittable(); + + return commitRequest; + } + + private WriteResult of(DataFile dataFile) { + return WriteResult.builder().addDataFiles(dataFile).build(); + } + + private void assertMaxCommittedCheckpointId(String myJobID, String operatorId, long expectedId) { + table.refresh(); + long actualId = SinkUtil.getMaxCommittedCheckpointId(table, myJobID, operatorId, branch); + assertThat(actualId).isEqualTo(expectedId); + } + + private void assertMaxCommittedCheckpointId(String myJobID, long expectedId) { + assertMaxCommittedCheckpointId(myJobID, OPERATOR_ID, expectedId); + } + + private void assertSnapshotSize(int expectedSnapshotSize) { + table.refresh(); + assertThat(table.snapshots()).hasSize(expectedSnapshotSize); + } + + private static ByteBuffer longToBuffer(long value) { + return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, value); + } + + private static class TestCommittableMessageTypeSerializer + extends TypeSerializer> { + + CommittableMessageSerializer serializer = + new CommittableMessageSerializer<>(new IcebergCommittableSerializer()); + + @Override + public boolean isImmutableType() { + return false; + } + + @Override + public TypeSerializer> duplicate() { + return null; + } + + @Override + public CommittableMessage createInstance() { + return null; + } + + @Override + public CommittableMessage copy( + CommittableMessage from) { + return from; + } + + @Override + public CommittableMessage copy( + CommittableMessage from, CommittableMessage reuse) { + return from; + } + + @Override + public int getLength() { + return 0; + } + + @Override + public void serialize(CommittableMessage record, DataOutputView target) + throws IOException { + byte[] serialize = serializer.serialize(record); + target.writeInt(serialize.length); + target.write(serialize); + } + + @Override + public CommittableMessage deserialize(DataInputView source) + throws IOException { + int length = source.readInt(); + byte[] bytes = new byte[length]; + source.read(bytes); + return serializer.deserialize(1, bytes); + } + + @Override + public CommittableMessage deserialize( + CommittableMessage reuse, DataInputView source) throws IOException { + return deserialize(source); + } + + @Override + public void copy(DataInputView source, DataOutputView target) throws IOException { + CommittableMessage deserialize = deserialize(source); + serialize(deserialize, target); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public int hashCode() { + return 0; + } + + @Override + public TypeSerializerSnapshot> snapshotConfiguration() { + return null; + } + }; +} 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 948c7b31430c..67ca9d08b206 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 @@ -139,8 +139,7 @@ public void testCommitTxnWithoutDataFiles() throws Exception { assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // It's better to advance the max-committed-checkpoint-id in iceberg snapshot, so that the - // future flink job - // failover won't fail. + // future flink job failover won't fail. for (int i = 1; i <= 3; i++) { harness.snapshot(++checkpointId, ++timestamp); assertFlinkManifests(0); @@ -1079,8 +1078,8 @@ private DataFile writeDataFile( private void assertMaxCommittedCheckpointId(JobID jobID, OperatorID operatorID, long expectedId) { table.refresh(); long actualId = - IcebergFilesCommitter.getMaxCommittedCheckpointId( - table, jobID.toString(), operatorID.toHexString(), branch); + SinkUtil.getMaxCommittedCheckpointId( + table, jobID.toString(), operatorID.toString(), branch); assertThat(actualId).isEqualTo(expectedId); } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java new file mode 100644 index 000000000000..7f355c1e8403 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSink.java @@ -0,0 +1,436 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import org.apache.flink.api.dag.Transformation; +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.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.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.Table; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.flink.FlinkWriteOptions; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.sink.IcebergSink.Builder; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSink extends TestFlinkIcebergSinkBase { + + private TableLoader tableLoader; + + @Parameter(index = 0) + private FileFormat format; + + @Parameter(index = 1) + private int parallelism; + + @Parameter(index = 2) + private boolean partitioned; + + @Parameters(name = "format={0}, parallelism={1}, partitioned={2}") + public static Object[][] parameters() { + return new Object[][] { + {FileFormat.AVRO, 1, true}, + {FileFormat.AVRO, 1, false}, + {FileFormat.AVRO, 2, true}, + {FileFormat.AVRO, 2, false}, + {FileFormat.ORC, 1, true}, + {FileFormat.ORC, 1, false}, + {FileFormat.ORC, 2, true}, + {FileFormat.ORC, 2, false}, + {FileFormat.PARQUET, 1, true}, + {FileFormat.PARQUET, 1, false}, + {FileFormat.PARQUET, 2, true}, + {FileFormat.PARQUET, 2, false} + }; + } + + @BeforeEach + void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name())); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + void testWriteRowData() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testWriteRow() throws Exception { + testWriteRow(null, DistributionMode.NONE); + } + + @TestTemplate + void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + } + + @TestTemplate + void testJobNoneDistributeMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, DistributionMode.NONE); + + if (parallelism > 1) { + if (partitioned) { + int files = partitionFiles("aaa") + partitionFiles("bbb") + partitionFiles("ccc"); + assertThat(files).as("Should have more than 3 files in iceberg table.").isGreaterThan(3); + } + } + } + + @TestTemplate + void testJobHashDistributionMode() { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + assertThatThrownBy(() -> testWriteRow(null, DistributionMode.RANGE)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Flink does not support 'range' write distribution mode now."); + } + + @TestTemplate + void testJobNullDistributionMode() throws Exception { + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.HASH.modeName()) + .commit(); + + testWriteRow(null, null); + + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testPartitionWriteMode() throws Exception { + testWriteRow(null, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testShuffleByPartitionWithSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.HASH); + if (partitioned) { + assertThat(partitionFiles("aaa")) + .as("There should be only 1 data file in partition 'aaa'") + .isEqualTo(1); + assertThat(partitionFiles("bbb")) + .as("There should be only 1 data file in partition 'bbb'") + .isEqualTo(1); + assertThat(partitionFiles("ccc")) + .as("There should be only 1 data file in partition 'ccc'") + .isEqualTo(1); + } + } + + @TestTemplate + void testTwoSinksInDisjointedDAG() throws Exception { + Map props = ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, format.name()); + + Table leftTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("left"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader leftTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("left")); + + Table rightTable = + CATALOG_EXTENSION + .catalog() + .createTable( + TableIdentifier.of("right"), + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + props); + TableLoader rightTableLoader = + TableLoader.fromCatalog(CATALOG_EXTENSION.catalogLoader(), TableIdentifier.of("right")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + env.getConfig().disableAutoGeneratedUIDs(); + + List leftRows = createRows("left-"); + DataStream leftStream = + env.fromCollection(leftRows, ROW_TYPE_INFO) + .name("leftCustomSource") + .uid("leftCustomSource"); + IcebergSink.forRow(leftStream, SimpleDataUtil.FLINK_SCHEMA) + .table(leftTable) + .tableLoader(leftTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .distributionMode(DistributionMode.NONE) + .uidSuffix("leftIcebergSink") + .append(); + + List rightRows = createRows("right-"); + DataStream rightStream = + env.fromCollection(rightRows, ROW_TYPE_INFO) + .name("rightCustomSource") + .uid("rightCustomSource"); + IcebergSink.forRow(rightStream, SimpleDataUtil.FLINK_SCHEMA) + .table(rightTable) + .tableLoader(rightTableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("rightIcebergSink") + .setSnapshotProperty("flink.test", TestIcebergSink.class.getName()) + .snapshotProperties(Collections.singletonMap("direction", "rightTable")) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(leftTable, convertToRowData(leftRows)); + SimpleDataUtil.assertTableRows(rightTable, convertToRowData(rightRows)); + + leftTable.refresh(); + + assertThat(leftTable.currentSnapshot().summary().get("flink.test")).isNull(); + assertThat(leftTable.currentSnapshot().summary().get("direction")).isNull(); + + assertThat(rightTable.currentSnapshot().summary().get("flink.test")) + .isEqualTo(TestIcebergSink.class.getName()); + assertThat(rightTable.currentSnapshot().summary().get("direction")).isEqualTo("rightTable"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownDistributionMode() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.DISTRIBUTION_MODE.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .append(); + + assertThatThrownBy(() -> env.execute("Test Iceberg DataStream")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid distribution mode: UNRECOGNIZED"); + } + + @TestTemplate + void testOverrideWriteConfigWithUnknownFileFormat() { + Map newProps = Maps.newHashMap(); + newProps.put(FlinkWriteOptions.WRITE_FORMAT.key(), "UNRECOGNIZED"); + + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .setAll(newProps) + .uidSuffix("ingestion"); + assertThatThrownBy(builder::append) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid file format: UNRECOGNIZED"); + } + + @TestTemplate + void testWriteRowWithTableRefreshInterval() throws Exception { + List rows = Lists.newArrayList(Row.of(1, "hello"), Row.of(2, "world"), Row.of(3, "foo")); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO) + .map(CONVERTER::toInternal, FlinkCompatibilityUtil.toTypeInfo(SimpleDataUtil.ROW_TYPE)); + + Configuration flinkConf = new Configuration(); + flinkConf.setString(FlinkWriteOptions.TABLE_REFRESH_INTERVAL.key(), "100ms"); + + IcebergSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .flinkConf(flinkConf) + .writeParallelism(parallelism) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream"); + + // Assert the iceberg table's records. + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } + + @TestTemplate + void testOperatorsUidNameNoUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: hadoop.default.t"); + assertThat(secondTransformation.getUid()).isEqualTo("hadoop.default.t"); + assertThat(secondTransformation.getName()).isEqualTo("hadoop.default.t"); + } + + @TestTemplate + void testOperatorsUidNameWitUidSuffix() throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .distributionMode(DistributionMode.HASH) + .uidSuffix("data-ingestion") + .append(); + + Transformation firstTransformation = env.getTransformations().get(0); + Transformation secondTransformation = env.getTransformations().get(1); + assertThat(firstTransformation.getUid()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(firstTransformation.getName()).isEqualTo("Sink pre-writer mapper: data-ingestion"); + assertThat(secondTransformation.getUid()).isEqualTo("data-ingestion"); + assertThat(secondTransformation.getName()).isEqualTo("data-ingestion"); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = + env.addSource(createBoundedSource(rows), ROW_TYPE_INFO).uid("mySourceId"); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .writeParallelism(parallelism) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows)); + } +} diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java new file mode 100644 index 000000000000..a8c8892af11c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkBranch.java @@ -0,0 +1,120 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.util.List; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.api.TableSchema; +import org.apache.flink.types.Row; +import org.apache.iceberg.DistributionMode; +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; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@ExtendWith(ParameterizedTestExtension.class) +public class TestIcebergSinkBranch extends TestFlinkIcebergSinkBase { + + @Parameter(index = 0) + private String branch; + + @Parameters(name = "branch = {0}") + public static Object[][] parameters() { + return new Object[][] {new Object[] {"main"}, new Object[] {"testBranch"}}; + } + + @BeforeEach + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "1")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testWriteRowWithTableSchema() throws Exception { + testWriteRow(SimpleDataUtil.FLINK_SCHEMA, DistributionMode.NONE); + verifyOtherBranchUnmodified(); + } + + private void testWriteRow(TableSchema tableSchema, DistributionMode distributionMode) + throws Exception { + List rows = createRows(""); + DataStream dataStream = env.addSource(createBoundedSource(rows), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .table(table) + .tableLoader(tableLoader) + .tableSchema(tableSchema) + .toBranch(branch) + .distributionMode(distributionMode) + .append(); + + // Execute the program. + env.execute("Test Iceberg DataStream."); + + SimpleDataUtil.assertTableRows(table, convertToRowData(rows), branch); + SimpleDataUtil.assertTableRows( + table, + ImmutableList.of(), + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH); + + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()).isNull(); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} 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 new file mode 100644 index 000000000000..007d5f314946 --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2.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.flink.sink; + +import static org.apache.iceberg.flink.TestFixtures.DATABASE; +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.List; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.MetadataColumns; +import org.apache.iceberg.ParameterizedTestExtension; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.SnapshotRef; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +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.types.Types; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.api.extension.RegisterExtension; + +@ExtendWith(ParameterizedTestExtension.class) +@Timeout(value = 60) +public class TestIcebergSinkV2 extends TestFlinkIcebergSinkV2Base { + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + MiniFlinkClusterExtension.createWithClassloaderCheckDisabled(); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(DATABASE, TestFixtures.TABLE); + + @BeforeEach + public void setupTable() { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + partitioned + ? PartitionSpec.builderFor(SimpleDataUtil.SCHEMA).identity("data").build() + : PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + format.name(), + TableProperties.FORMAT_VERSION, + String.valueOf(FORMAT_V2))); + + table + .updateProperties() + .set(TableProperties.DEFAULT_FILE_FORMAT, format.name()) + .set(TableProperties.WRITE_DISTRIBUTION_MODE, writeDistributionMode) + .commit(); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100L) + .setParallelism(parallelism) + .setMaxParallelism(parallelism); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testCheckAndGetEqualityFieldIds() { + table + .updateSchema() + .allowIncompatibleChanges() + .addRequiredColumn("type", Types.StringType.get()) + .setIdentifierFields("type") + .commit(); + + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA).table(table); + + // Use user-provided equality field column as equality field id list + builder.equalityFieldColumns(Lists.newArrayList("id")); + assertThat(SinkUtil.checkAndGetEqualityFieldIds(table, Lists.newArrayList("id"))) + .containsExactlyInAnyOrder(table.schema().findField("id").fieldId()); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnlyDeletesOnDataKey() throws Exception { + List> elementsPerCheckpoint = + ImmutableList.of( + ImmutableList.of(row("+I", 1, "aaa")), + ImmutableList.of(row("-D", 1, "aaa"), row("-D", 2, "bbb"))); + + List> expectedRecords = + ImmutableList.of(ImmutableList.of(record(1, "aaa")), ImmutableList.of()); + + testChangeLogs( + ImmutableList.of("data"), + row -> row.getField(ROW_DATA_POS), + true, + elementsPerCheckpoint, + expectedRecords, + SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testChangeLogOnSameKey() throws Exception { + testChangeLogOnSameKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertModeCheck() throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(ImmutableList.of()), ROW_TYPE_INFO); + IcebergSink.Builder builder = + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .upsert(true); + + assertThatThrownBy( + () -> + builder + .equalityFieldColumns(ImmutableList.of("id", "data")) + .overwrite(true) + .append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "OVERWRITE mode shouldn't be enable when configuring to use UPSERT data stream."); + + assertThatThrownBy( + () -> builder.equalityFieldColumns(ImmutableList.of()).overwrite(false).append()) + .isInstanceOf(IllegalStateException.class) + .hasMessage( + "Equality field columns shouldn't be empty when configuring to use UPSERT data stream."); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(SnapshotRef.MAIN_BRANCH); + } + + @TestTemplate + public void testDeleteStats() throws Exception { + assumeThat(format).isNotEqualTo(FileFormat.AVRO); + + List> elementsPerCheckpoint = + ImmutableList.of( + // Checkpoint #1 + ImmutableList.of(row("+I", 1, "aaa"), row("-D", 1, "aaa"), row("+I", 1, "aaa"))); + + List> expectedRecords = ImmutableList.of(ImmutableList.of(record(1, "aaa"))); + + testChangeLogs( + ImmutableList.of("id", "data"), + row -> Row.of(row.getField(ROW_ID_POS), row.getField(ROW_DATA_POS)), + false, + elementsPerCheckpoint, + expectedRecords, + "main"); + + DeleteFile deleteFile = table.currentSnapshot().addedDeleteFiles(table.io()).iterator().next(); + String fromStat = + 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()); + } + + protected void testChangeLogs( + List equalityFieldColumns, + KeySelector keySelector, + boolean insertAsUpsert, + List> elementsPerCheckpoint, + List> expectedRecordsPerCheckpoint, + String branch) + throws Exception { + DataStream dataStream = + env.addSource(new BoundedTestSource<>(elementsPerCheckpoint), ROW_TYPE_INFO); + + IcebergSink.forRow(dataStream, SimpleDataUtil.FLINK_SCHEMA) + .tableLoader(tableLoader) + .tableSchema(SimpleDataUtil.FLINK_SCHEMA) + .writeParallelism(parallelism) + .equalityFieldColumns(equalityFieldColumns) + .upsert(insertAsUpsert) + .toBranch(branch) + .uidSuffix("sink") + .append(); + + // Execute the program. + env.execute("Test Iceberg Change-Log DataStream."); + + table.refresh(); + List snapshots = findValidSnapshots(); + int expectedSnapshotNum = expectedRecordsPerCheckpoint.size(); + assertThat(snapshots).hasSize(expectedSnapshotNum); + + for (int i = 0; i < expectedSnapshotNum; i++) { + long snapshotId = snapshots.get(i).snapshotId(); + List expectedRecords = expectedRecordsPerCheckpoint.get(i); + assertThat(actualRowSet(snapshotId, "*")) + .as("Should have the expected records for the checkpoint#" + i) + .isEqualTo(expectedRowSet(expectedRecords.toArray(new Record[0]))); + } + } +} 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 new file mode 100644 index 000000000000..93da40ab5c9a --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergSinkV2Branch.java @@ -0,0 +1,119 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThat; + +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; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.SimpleDataUtil; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.TestTemplate; +import org.junit.jupiter.api.extension.ExtendWith; + +@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 + public void before() throws IOException { + table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SimpleDataUtil.SCHEMA, + PartitionSpec.unpartitioned(), + ImmutableMap.of( + TableProperties.DEFAULT_FILE_FORMAT, + FileFormat.AVRO.name(), + TableProperties.FORMAT_VERSION, + "2")); + + env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100); + + tableLoader = CATALOG_EXTENSION.tableLoader(); + } + + @TestTemplate + public void testChangeLogOnIdKey() throws Exception { + testChangeLogOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnDataKey() throws Exception { + testChangeLogOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testChangeLogOnIdDataKey() throws Exception { + testChangeLogOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdKey() throws Exception { + testUpsertOnIdKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnDataKey() throws Exception { + testUpsertOnDataKey(branch); + verifyOtherBranchUnmodified(); + } + + @TestTemplate + public void testUpsertOnIdDataKey() throws Exception { + testUpsertOnIdDataKey(branch); + verifyOtherBranchUnmodified(); + } + + private void verifyOtherBranchUnmodified() { + String otherBranch = + branch.equals(SnapshotRef.MAIN_BRANCH) ? "test-branch" : SnapshotRef.MAIN_BRANCH; + if (otherBranch.equals(SnapshotRef.MAIN_BRANCH)) { + assertThat(table.currentSnapshot()); + } + + assertThat(table.snapshot(otherBranch)).isNull(); + } +} From bcde1ef43af95a2008a71473478f8f0f8fbca3cc Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 26 Aug 2024 17:30:47 -0700 Subject: [PATCH 0642/1019] Core: Project data file stats only if there are equality deletes (#11013) --- .../java/org/apache/iceberg/DeleteFileIndex.java | 16 +++++++++++++--- .../java/org/apache/iceberg/ManifestGroup.java | 2 +- .../apache/iceberg/DeleteFileIndexTestBase.java | 8 ++++++++ 3 files changed, 22 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index c26716481836..794044373471 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -71,6 +71,8 @@ class DeleteFileIndex { private final PartitionMap eqDeletesByPartition; private final PartitionMap posDeletesByPartition; private final CharSequenceMap posDeletesByPath; + private final boolean hasEqDeletes; + private final boolean hasPosDeletes; private final boolean isEmpty; private DeleteFileIndex( @@ -82,15 +84,23 @@ private DeleteFileIndex( this.eqDeletesByPartition = eqDeletesByPartition; this.posDeletesByPartition = posDeletesByPartition; this.posDeletesByPath = posDeletesByPath; - boolean noEqDeletes = globalDeletes == null && eqDeletesByPartition == null; - boolean noPosDeletes = posDeletesByPartition == null && posDeletesByPath == null; - this.isEmpty = noEqDeletes && noPosDeletes; + this.hasEqDeletes = globalDeletes != null || eqDeletesByPartition != null; + this.hasPosDeletes = posDeletesByPartition != null || posDeletesByPath != null; + this.isEmpty = !hasEqDeletes && !hasPosDeletes; } public boolean isEmpty() { return isEmpty; } + public boolean hasEqualityDeletes() { + return hasEqDeletes; + } + + public boolean hasPositionDeletes() { + return hasPosDeletes; + } + public Iterable referencedDeleteFiles() { Iterable deleteFiles = Collections.emptyList(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestGroup.java b/core/src/main/java/org/apache/iceberg/ManifestGroup.java index 38ad2661a8a3..10cf6bd3bfbd 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestGroup.java +++ b/core/src/main/java/org/apache/iceberg/ManifestGroup.java @@ -184,7 +184,7 @@ public CloseableIterable plan(CreateTasksFunction cre DeleteFileIndex deleteFiles = deleteIndexBuilder.scanMetrics(scanMetrics).build(); boolean dropStats = ManifestReader.dropStats(columns); - if (!deleteFiles.isEmpty()) { + if (deleteFiles.hasEqualityDeletes()) { select(ManifestReader.withStatsColumns(columns)); } diff --git a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java index 836a1ddd80f5..986e8608c082 100644 --- a/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DeleteFileIndexTestBase.java @@ -139,6 +139,8 @@ public void testMinSequenceNumberFilteringForFiles() { DataFile file = unpartitionedFile(partSpec); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isFalse(); assertThat(index.forDataFile(0, file)).as("Only one delete file should apply").hasSize(1); } @@ -158,6 +160,9 @@ public void testUnpartitionedDeletes() { .specsById(ImmutableMap.of(partSpec.specId(), partSpec, 1, SPEC)) .build(); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isTrue(); + DataFile unpartitionedFile = unpartitionedFile(partSpec); assertThat(index.forDataFile(0, unpartitionedFile)) .as("All deletes should apply to seq 0") @@ -213,6 +218,9 @@ public void testPartitionedDeleteIndex() { .specsById(ImmutableMap.of(SPEC.specId(), SPEC, 1, PartitionSpec.unpartitioned())) .build(); + assertThat(index.hasEqualityDeletes()).isTrue(); + assertThat(index.hasPositionDeletes()).isTrue(); + assertThat(index.forDataFile(0, FILE_A)) .as("All deletes should apply to seq 0") .isEqualTo(deleteFiles); From fc92a66d836c3a8b015b091aff3a0ad558fed101 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Aug 2024 20:38:25 -0600 Subject: [PATCH 0643/1019] Build: Bump org.xerial:sqlite-jdbc from 3.46.0.1 to 3.46.1.0 (#11007) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.46.0.1 to 3.46.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.46.0.1...3.46.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 bf7b9077d0ca..29c605db77d2 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -84,7 +84,7 @@ spark-hive34 = "3.4.3" spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.39" -sqlite-jdbc = "3.46.0.1" +sqlite-jdbc = "3.46.1.0" testcontainers = "1.20.1" tez010 = "0.10.3" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 1fce583d4758d799e9b2646c58084f1f8efcdeb8 Mon Sep 17 00:00:00 2001 From: Charles Smith Date: Mon, 26 Aug 2024 19:40:20 -0700 Subject: [PATCH 0644/1019] Docs: Add Druid docs url to sidebar (#10997) --- docs/mkdocs.yml | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index e915af4f3fce..604fede583d8 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -63,6 +63,7 @@ nav: - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - Doris: https://doris.apache.org/docs/dev/lakehouse/datalake-analytics/iceberg + - Druid: https://druid.apache.org/docs/latest/development/extensions-contrib/iceberg/ - Integrations: - aws.md - dell.md From d5cafafeddb0eb2beb5366096ac4ce2844d92435 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 27 Aug 2024 08:33:42 -0700 Subject: [PATCH 0645/1019] Flink: backport PR #10832 of inferring parallelism in FLIP-27 source (#11009) --- .../iceberg/flink/source/IcebergSource.java | 85 +++++++- .../flink/source/IcebergTableSource.java | 31 +-- .../source/TestIcebergSourceBounded.java | 9 +- .../source/TestIcebergSourceBoundedSql.java | 2 +- .../TestIcebergSourceInferParallelism.java | 181 ++++++++++++++++++ .../flink/source/TestIcebergSourceSql.java | 7 +- ...estIcebergSpeculativeExecutionSupport.java | 15 +- .../iceberg/flink/source/IcebergSource.java | 85 +++++++- .../flink/source/IcebergTableSource.java | 31 +-- .../source/TestIcebergSourceBounded.java | 9 +- .../source/TestIcebergSourceBoundedSql.java | 2 +- .../TestIcebergSourceInferParallelism.java | 181 ++++++++++++++++++ .../flink/source/TestIcebergSourceSql.java | 7 +- ...estIcebergSpeculativeExecutionSupport.java | 14 +- 14 files changed, 586 insertions(+), 73 deletions(-) create mode 100644 flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java 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 351ba54e5c7c..5718f4b93825 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 @@ -28,6 +28,8 @@ 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; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +39,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -74,6 +79,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; 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.ThreadPools; @@ -97,6 +103,11 @@ public class IcebergSource implements Source emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -132,16 +143,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -207,12 +228,35 @@ private SplitEnumerator createEnumer // Only do scan planning if nothing is restored from checkpoint state List splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + /** * Create a source builder. * @@ -571,6 +615,41 @@ public IcebergSource build() { emitter); } + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource source = build(); + TypeInformation outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static TypeInformation outputTypeInfo( + RowDataConverter converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + private ReaderFunction readerFunction(ScanContext context) { if (table instanceof BaseMetadataTable) { MetaDataReaderFunction rowDataReaderFunction = diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ 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; @@ -130,11 +128,8 @@ protected List run( sourceBuilder.properties(options); DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * 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 static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +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.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator iterator = collector.getOutput()) { + List result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} 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 75f0a785a8c5..548940a842ce 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 @@ -53,7 +53,12 @@ public class TestIcebergSourceSql extends TestSqlBase { public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 51f9025b4159..564e8139e6cc 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -48,14 +48,20 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_MANAGERS = 1; private static final int NUM_TASK_SLOTS = 3; @RegisterExtension - public static MiniClusterExtension miniClusterResource = + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = new MiniClusterExtension( new MiniClusterResourceConfiguration.Builder() .setNumberTaskManagers(NUM_TASK_MANAGERS) @@ -144,9 +150,9 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getAttemptNumber() <= 0) { + // Simulate slow subtask 0 with attempt 0 + if (getRuntimeContext().getIndexOfThisSubtask() == 0 + && getRuntimeContext().getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); } @@ -169,6 +175,7 @@ private static Configuration configure() { // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); // for speculative execution configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); 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 351ba54e5c7c..5718f4b93825 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 @@ -28,6 +28,8 @@ 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; import org.apache.flink.api.connector.source.Source; import org.apache.flink.api.connector.source.SourceReader; @@ -37,6 +39,9 @@ import org.apache.flink.configuration.Configuration; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.datastream.DataStreamSource; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.data.RowData; import org.apache.flink.util.Preconditions; @@ -74,6 +79,7 @@ import org.apache.iceberg.flink.source.split.IcebergSourceSplitSerializer; import org.apache.iceberg.flink.source.split.SerializableComparator; import org.apache.iceberg.flink.source.split.SplitComparators; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; 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.ThreadPools; @@ -97,6 +103,11 @@ public class IcebergSource implements Source emitter; private final String tableName; + // cache the discovered splits by planSplitsForBatch, which can be called twice. And they come + // from two different threads: (1) source/stream construction by main thread (2) enumerator + // creation. Hence need volatile here. + private volatile List batchSplits; + IcebergSource( TableLoader tableLoader, ScanContext scanContext, @@ -132,16 +143,26 @@ private String planningThreadName() { return tableName + "-" + UUID.randomUUID(); } + /** + * Cache the enumerated splits for batch execution to avoid double planning as there are two code + * paths obtaining splits: (1) infer parallelism (2) enumerator creation. + */ private List planSplitsForBatch(String threadName) { + if (batchSplits != null) { + return batchSplits; + } + ExecutorService workerPool = ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); - List splits = + this.batchSplits = FlinkSplitPlanner.planIcebergSourceSplits(loader.loadTable(), scanContext, workerPool); LOG.info( - "Discovered {} splits from table {} during job initialization", splits.size(), tableName); - return splits; + "Discovered {} splits from table {} during job initialization", + batchSplits.size(), + tableName); + return batchSplits; } catch (IOException e) { throw new UncheckedIOException("Failed to close table loader", e); } finally { @@ -207,12 +228,35 @@ private SplitEnumerator createEnumer // Only do scan planning if nothing is restored from checkpoint state List splits = planSplitsForBatch(planningThreadName()); assigner.onDiscoveredSplits(splits); + // clear the cached splits after enumerator creation as they won't be needed anymore + this.batchSplits = null; } return new StaticIcebergEnumerator(enumContext, assigner); } } + private boolean shouldInferParallelism() { + return !scanContext.isStreaming(); + } + + private int inferParallelism(ReadableConfig flinkConf, StreamExecutionEnvironment env) { + int parallelism = + SourceUtil.inferParallelism( + flinkConf, + scanContext.limit(), + () -> { + List splits = planSplitsForBatch(planningThreadName()); + return splits.size(); + }); + + if (env.getMaxParallelism() > 0) { + parallelism = Math.min(parallelism, env.getMaxParallelism()); + } + + return parallelism; + } + /** * Create a source builder. * @@ -571,6 +615,41 @@ public IcebergSource build() { emitter); } + /** + * Build the {@link IcebergSource} and create a {@link DataStream} from the source. Watermark + * strategy is set to {@link WatermarkStrategy#noWatermarks()}. + * + * @return data stream from the Iceberg source + */ + public DataStream buildStream(StreamExecutionEnvironment env) { + // buildStream should only be called with RowData or Converter paths. + Preconditions.checkState( + readerFunction == null, + "Cannot set reader function when building a data stream from the source"); + IcebergSource source = build(); + TypeInformation outputTypeInfo = + outputTypeInfo(converter, table.schema(), source.scanContext.project()); + DataStreamSource stream = + env.fromSource(source, WatermarkStrategy.noWatermarks(), source.name(), outputTypeInfo); + if (source.shouldInferParallelism()) { + stream = stream.setParallelism(source.inferParallelism(flinkConfig, env)); + } + + return stream; + } + + private static TypeInformation outputTypeInfo( + RowDataConverter converter, Schema tableSchema, Schema projected) { + if (converter != null) { + return converter.getProducedType(); + } else { + // output type is RowData + Schema readSchema = projected != null ? projected : tableSchema; + return (TypeInformation) + FlinkCompatibilityUtil.toTypeInfo(FlinkSchemaUtil.convert(readSchema)); + } + } + private ReaderFunction readerFunction(ScanContext context) { if (table instanceof BaseMetadataTable) { MetaDataReaderFunction rowDataReaderFunction = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java index 610657e8d47b..65adce77d9f9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/IcebergTableSource.java @@ -23,11 +23,8 @@ import java.util.Map; import java.util.Optional; import org.apache.flink.annotation.Internal; -import org.apache.flink.api.common.eventtime.WatermarkStrategy; -import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.configuration.ReadableConfig; import org.apache.flink.streaming.api.datastream.DataStream; -import org.apache.flink.streaming.api.datastream.DataStreamSource; import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.table.api.TableSchema; import org.apache.flink.table.connector.ChangelogMode; @@ -128,26 +125,18 @@ private DataStream createDataStream(StreamExecutionEnvironment execEnv) .build(); } - private DataStreamSource createFLIP27Stream(StreamExecutionEnvironment env) { + private DataStream createFLIP27Stream(StreamExecutionEnvironment env) { SplitAssignerType assignerType = readableConfig.get(FlinkConfigOptions.TABLE_EXEC_SPLIT_ASSIGNER_TYPE); - IcebergSource source = - IcebergSource.forRowData() - .tableLoader(loader) - .assignerFactory(assignerType.factory()) - .properties(properties) - .project(getProjectedSchema()) - .limit(limit) - .filters(filters) - .flinkConfig(readableConfig) - .build(); - DataStreamSource stream = - env.fromSource( - source, - WatermarkStrategy.noWatermarks(), - source.name(), - TypeInformation.of(RowData.class)); - return stream; + return IcebergSource.forRowData() + .tableLoader(loader) + .assignerFactory(assignerType.factory()) + .properties(properties) + .project(getProjectedSchema()) + .limit(limit) + .filters(filters) + .flinkConfig(readableConfig) + .buildStream(env); } private TableSchema getProjectedSchema() { diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java index b7447d15c05a..db8647f054ae 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBounded.java @@ -24,8 +24,6 @@ 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; @@ -130,11 +128,8 @@ protected List run( sourceBuilder.properties(options); DataStream stream = - env.fromSource( - sourceBuilder.build(), - WatermarkStrategy.noWatermarks(), - "testBasicRead", - TypeInformation.of(RowData.class)) + sourceBuilder + .buildStream(env) .map( new RowDataToRowMapper( FlinkSchemaUtil.convert( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java index 0f41c5af4c95..d3713e296014 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceBoundedSql.java @@ -37,7 +37,7 @@ public class TestIcebergSourceBoundedSql extends TestIcebergSourceBounded { @BeforeEach public void before() throws IOException { Configuration tableConf = getTableEnv().getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); SqlHelpers.sql( getTableEnv(), "create catalog iceberg_catalog with ('type'='iceberg', 'catalog-type'='hadoop', 'warehouse'='%s')", diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java new file mode 100644 index 000000000000..2908cb927269 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSourceInferParallelism.java @@ -0,0 +1,181 @@ +/* + * 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 static org.apache.iceberg.flink.MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.file.Path; +import java.util.List; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.runtime.executiongraph.AccessExecutionGraph; +import org.apache.flink.runtime.executiongraph.AccessExecutionJobVertex; +import org.apache.flink.runtime.minicluster.MiniCluster; +import org.apache.flink.runtime.testutils.InternalMiniClusterExtension; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.flink.types.Row; +import org.apache.flink.util.CloseableIterator; +import org.apache.iceberg.FileFormat; +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.flink.FlinkConfigOptions; +import org.apache.iceberg.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.data.RowDataToRowMapper; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.api.io.TempDir; + +public class TestIcebergSourceInferParallelism { + private static final int NUM_TMS = 2; + private static final int SLOTS_PER_TM = 2; + private static final int PARALLELISM = NUM_TMS * SLOTS_PER_TM; + private static final int MAX_INFERRED_PARALLELISM = 3; + + @RegisterExtension + private static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(NUM_TMS) + .setNumberSlotsPerTaskManager(SLOTS_PER_TM) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + protected static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + @TempDir private Path tmpDir; + + private Table table; + private GenericAppenderHelper dataAppender; + + @BeforeEach + public void before() throws IOException { + this.table = + CATALOG_EXTENSION.catalog().createTable(TestFixtures.TABLE_IDENTIFIER, TestFixtures.SCHEMA); + this.dataAppender = new GenericAppenderHelper(table, FileFormat.PARQUET, tmpDir); + } + + @AfterEach + public void after() { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + @Test + public void testEmptyTable() throws Exception { + // Inferred parallelism should be at least 1 even if table is empty + test(1, 0); + } + + @Test + public void testTableWithFilesLessThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < 2; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should equal to 2 splits + test(2, 2); + } + + @Test + public void testTableWithFilesMoreThanMaxInferredParallelism() throws Exception { + // Append files to the table + for (int i = 0; i < MAX_INFERRED_PARALLELISM + 1; ++i) { + List batch = RandomGenericData.generate(table.schema(), 1, 0); + dataAppender.appendToTable(batch); + } + + // Inferred parallelism should be capped by the MAX_INFERRED_PARALLELISM + test(MAX_INFERRED_PARALLELISM, MAX_INFERRED_PARALLELISM + 1); + } + + private void test(int expectedParallelism, int expectedRecords) throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + env.setParallelism(PARALLELISM); + + Configuration config = new Configuration(); + config.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, true); + config.set( + FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM_MAX, + MAX_INFERRED_PARALLELISM); + + DataStream dataStream = + IcebergSource.forRowData() + .tableLoader(CATALOG_EXTENSION.tableLoader()) + .table(table) + .flinkConfig(config) + // force one file per split + .splitSize(1L) + .buildStream(env) + .map(new RowDataToRowMapper(FlinkSchemaUtil.convert(table.schema()))); + + DataStream.Collector collector = new DataStream.Collector<>(); + dataStream.collectAsync(collector); + JobClient jobClient = env.executeAsync(); + try (CloseableIterator iterator = collector.getOutput()) { + List result = Lists.newArrayList(); + while (iterator.hasNext()) { + result.add(iterator.next()); + } + + assertThat(result).hasSize(expectedRecords); + verifySourceParallelism( + expectedParallelism, miniCluster().getExecutionGraph(jobClient.getJobID()).get()); + } + } + + /** + * Borrowed this approach from Flink {@code FileSourceTextLinesITCase} to get source parallelism + * from execution graph. + */ + private static void verifySourceParallelism( + int expectedParallelism, AccessExecutionGraph executionGraph) { + AccessExecutionJobVertex sourceVertex = + executionGraph.getVerticesTopologically().iterator().next(); + assertThat(sourceVertex.getParallelism()).isEqualTo(expectedParallelism); + } + + /** + * Use reflection to get {@code InternalMiniClusterExtension} and {@code MiniCluster} to get + * execution graph and source parallelism. Haven't find other way via public APIS. + */ + private static MiniCluster miniCluster() throws Exception { + Field privateField = + MiniClusterExtension.class.getDeclaredField("internalMiniClusterExtension"); + privateField.setAccessible(true); + InternalMiniClusterExtension internalExtension = + (InternalMiniClusterExtension) privateField.get(MINI_CLUSTER_EXTENSION); + return internalExtension.getMiniCluster(); + } +} 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 75f0a785a8c5..548940a842ce 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 @@ -53,7 +53,12 @@ public class TestIcebergSourceSql extends TestSqlBase { public void before() throws IOException { TableEnvironment tableEnvironment = getTableEnv(); Configuration tableConf = tableEnvironment.getConfig().getConfiguration(); - tableConf.setBoolean(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE.key(), true); + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + // Disable inferring parallelism to avoid interfering watermark tests + // that check split assignment is ordered by the watermark column. + // The tests assumes default parallelism of 1 with single reader task + // in order to check the order of read records. + tableConf.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); tableEnvironment.getConfig().set("table.exec.resource.default-parallelism", "1"); SqlHelpers.sql( diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java index 41b023b93617..05a08c24d8d0 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/source/TestIcebergSpeculativeExecutionSupport.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.stream.Collectors; import org.apache.flink.api.common.RuntimeExecutionMode; +import org.apache.flink.api.common.TaskInfo; import org.apache.flink.api.common.functions.RichMapFunction; import org.apache.flink.api.common.typeinfo.Types; import org.apache.flink.configuration.BatchExecutionOptions; @@ -48,8 +49,14 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; import org.junit.jupiter.api.extension.RegisterExtension; +/** + * There is a infinite sleep in the test. Add a timeout to the test to avoid stuck situation in case + * anything goes wrong unexpectedly. + */ +@Timeout(value = 60) public class TestIcebergSpeculativeExecutionSupport extends TestBase { private static final int NUM_TASK_MANAGERS = 1; private static final int NUM_TASK_SLOTS = 3; @@ -144,9 +151,9 @@ public void testSpeculativeExecution() throws Exception { private static class TestingMap extends RichMapFunction { @Override public Row map(Row row) throws Exception { - // Put the subtasks with the first attempt to sleep to trigger speculative - // execution - if (getRuntimeContext().getTaskInfo().getAttemptNumber() <= 0) { + // Simulate slow subtask 0 with attempt 0 + TaskInfo taskInfo = getRuntimeContext().getTaskInfo(); + if (taskInfo.getIndexOfThisSubtask() == 0 && taskInfo.getAttemptNumber() <= 0) { Thread.sleep(Integer.MAX_VALUE); } @@ -169,6 +176,7 @@ private static Configuration configure() { // Use FLIP-27 source configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_USE_FLIP27_SOURCE, true); + configuration.set(FlinkConfigOptions.TABLE_EXEC_ICEBERG_INFER_SOURCE_PARALLELISM, false); // for speculative execution configuration.set(BatchExecutionOptions.SPECULATIVE_ENABLED, true); From be382697e659ae4fc5acb16aa2458c892630424d Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Tue, 27 Aug 2024 09:29:55 -0700 Subject: [PATCH 0646/1019] Add REST Compatibility Kit (#10908) * Add REST Compatibility Kit * Publish test artifacts for open api module --- build.gradle | 45 +++++++ deploy.gradle | 9 +- gradle/libs.versions.toml | 3 + open-api/README.md | 63 +++++++++ .../RESTCompatibilityKitCatalogTests.java | 87 +++++++++++++ .../rest/RESTCompatibilityKitSuite.java | 45 +++++++ .../RESTCompatibilityKitViewCatalogTests.java | 91 +++++++++++++ .../org/apache/iceberg/rest/RCKUtils.java | 110 ++++++++++++++++ .../iceberg/rest/RESTCatalogServer.java | 123 ++++++++++++++++++ .../rest/RESTServerCatalogAdapter.java | 85 ++++++++++++ .../iceberg/rest/RESTServerExtension.java | 45 +++++++ 11 files changed, 701 insertions(+), 5 deletions(-) create mode 100644 open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java create mode 100644 open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java create mode 100644 open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java create mode 100644 open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.java create mode 100644 open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.java create mode 100644 open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java create mode 100644 open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java diff --git a/build.gradle b/build.gradle index 3b89dc4e05ad..6857a8443ef7 100644 --- a/build.gradle +++ b/build.gradle @@ -953,6 +953,51 @@ project(':iceberg-snowflake') { } project(':iceberg-open-api') { + apply plugin: 'java-test-fixtures' + + dependencies { + testImplementation project(':iceberg-api') + testImplementation project(':iceberg-core') + testImplementation project(':iceberg-core').sourceSets.test.runtimeClasspath + testImplementation(testFixtures(project(':iceberg-open-api'))) + + testImplementation libs.junit.jupiter + testImplementation libs.junit.suite.api + testImplementation libs.junit.suite.engine + testImplementation libs.assertj.core + + testImplementation project(':iceberg-aws-bundle') + testImplementation project(':iceberg-gcp-bundle') + testImplementation project(':iceberg-azure-bundle') + + 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.jetty.servlet + testFixturesImplementation libs.jetty.server + testFixturesImplementation libs.sqlite.jdbc + } + + test { + useJUnitPlatform() + + // Always rerun the compatibility tests + outputs.upToDateWhen {false} + maxParallelForks = 1 + + // Pass through any system properties that start with "rck" (REST Compatibility Kit) + // Note: only pass through specific properties so they do not affect other build/test + // configurations + systemProperties System.properties + .findAll { k, v -> k.startsWith("rck") } + .collectEntries { k, v -> { [(k):v, (k.replaceFirst("rck.", "")):v] }} // strip prefix + } + def restCatalogSpec = "$projectDir/rest-catalog-open-api.yaml" tasks.register('validateRESTCatalogSpec', org.openapitools.generator.gradle.plugin.tasks.ValidateTask) { inputSpec.set(restCatalogSpec) diff --git a/deploy.gradle b/deploy.gradle index 462448303253..37a6262f1efe 100644 --- a/deploy.gradle +++ b/deploy.gradle @@ -22,12 +22,8 @@ if (project.hasProperty('release') && jdkVersion != '11') { } subprojects { - if (it.name == 'iceberg-open-api') { - // don't publish iceberg-open-api - return - } - def isBom = it.name == 'iceberg-bom' + def isOpenApi = it.name == 'iceberg-open-api' apply plugin: 'maven-publish' apply plugin: 'signing' @@ -76,6 +72,9 @@ subprojects { apache(MavenPublication) { if (isBom) { from components.javaPlatform + } else if (isOpenApi) { + artifact testJar + artifact testFixturesJar } else { if (tasks.matching({task -> task.name == 'shadowJar'}).isEmpty()) { from components.java diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 29c605db77d2..ba7011312fa0 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -62,6 +62,7 @@ jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" jetty = "11.0.23" junit = "5.10.1" +junit-platform = "1.10.3" kafka = "3.8.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" @@ -202,6 +203,8 @@ jetty-server = { module = "org.eclipse.jetty:jetty-server", version.ref = "jetty jetty-servlet = { module = "org.eclipse.jetty:jetty-servlet", version.ref = "jetty" } junit-jupiter = { module = "org.junit.jupiter:junit-jupiter", version.ref = "junit" } junit-jupiter-engine = { module = "org.junit.jupiter:junit-jupiter-engine", version.ref = "junit" } +junit-suite-api = { module = "org.junit.platform:junit-platform-suite-api", version.ref = "junit-platform" } +junit-suite-engine = { module = "org.junit.platform:junit-platform-suite-engine", version.ref = "junit-platform" } junit-vintage-engine = { module = "org.junit.vintage:junit-vintage-engine", version.ref = "junit" } kryo-shaded = { module = "com.esotericsoftware:kryo-shaded", version.ref = "kryo-shaded" } mockito-core = { module = "org.mockito:mockito-core", version.ref = "mockito" } diff --git a/open-api/README.md b/open-api/README.md index 6f906ec5acef..cf13df60f7d2 100644 --- a/open-api/README.md +++ b/open-api/README.md @@ -40,3 +40,66 @@ make generate ``` The generated code is not being used in the project, but helps to see what the changes in the open-API definition are in the generated code. + +# REST Compatibility Kit (RCK) + +The REST Compatibility Kit (RCK) is a Technology Compatibility Kit (TCK) implementation for the +Iceberg REST Specification. This includes a series of tests based on the Java reference +implementation of the REST Catalog that can be executed against any REST server that implements the +spec. + +## Test Configuration + +The RCK can be configured using either environment variables or java system properties and allows +for configuring both the tests and the REST client. Environment variables prefixed by `CATALOG_` +are passed through the catalog configuring with the following mutations: + +1. The `CATALOG_` prefix is stripped from the key name +2. Single underscore (`_`) is replaced with a dot (`.`) +3. Double underscore (`__`) is replaced with a dash (`-`) +4. The key names are converted to lowercase + +A basic environment configuration would look like the following: + +```shell +CATALOG_URI=https://my_rest_server.io/ ## -> uri=https://my_rest_server.io/ +CATALOG_WAREHOUSE=test_warehouse ## -> warehouse=test_warehouse +CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO ## -> io-impl=org.apache.iceberg.aws.s3.S3FileIO +CATALOG_CREDENTIAL=: ## -> credential=: +``` + +Java properties passed to the test must be prefixed with `rck.`, which can be used to configure some +test configurations described below and any catalog client properties. + +An example of the same configuration using java system properties would look like the following: +```shell +rck.uri=https://my_rest_server.io/ ## -> uri=https://my_rest_server.io/ +rck.warehouse=test_warehouse ## -> warehouse=test_warehouse +rck.io-impl=org.apache.iceberg.aws.s3.S3FileIO ## -> io-impl=org.apache.iceberg.aws.s3.S3FileIO +rck.credential=: ## -> credential=: +``` + +Some test behaviors are configurable depending on the catalog implementations. Not all behaviors +are strictly defined by the REST Specification. The following are currently configurable: + +| config | default | +|-------------------------------|---------| +| rck.requires-namespace-create | true | +| rck.supports-serverside-retry | true | + + +## Running Compatibility Tests + +The compatibility tests can be invoked via gradle with the following: + +Note: The default behavior is to run a local http server with a jdbc backend for testing purposes, +so `-Drck.local=false` must be set to point to an external REST server. + +```shell +./gradlew :iceberg-open-api:test --tests RESTCompatibilityKitSuite \ + -Drck.local=false \ + -Drck.requires-namespace-create=true \ + -Drck.uri=https://my_rest_server.io/ \ + -Drck.warehouse=test_warehouse \ + -Drck.credential=: +``` \ No newline at end of file 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 new file mode 100644 index 000000000000..4c4860e88a19 --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitCatalogTests.java @@ -0,0 +1,87 @@ +/* + * 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 org.apache.iceberg.catalog.CatalogTests; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(RESTServerExtension.class) +public class RESTCompatibilityKitCatalogTests extends CatalogTests { + private static final Logger LOG = LoggerFactory.getLogger(RESTCompatibilityKitCatalogTests.class); + + private static RESTCatalog restCatalog; + + @BeforeAll + static void beforeClass() throws Exception { + restCatalog = RCKUtils.initCatalogClient(); + + assertThat(restCatalog.listNamespaces()) + .withFailMessage("Namespaces list should not contain: %s", RCKUtils.TEST_NAMESPACES) + .doesNotContainAnyElementsOf(RCKUtils.TEST_NAMESPACES); + } + + @BeforeEach + void before() { + try { + RCKUtils.purgeCatalogTestEntries(restCatalog); + } catch (Exception e) { + LOG.warn("Failure during test setup", e); + } + } + + @AfterAll + static void afterClass() throws Exception { + restCatalog.close(); + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_REQUIRES_NAMESPACE_CREATE, + super.requiresNamespaceCreate()); + } + + @Override + protected boolean supportsServerSideRetry() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_SUPPORTS_SERVERSIDE_RETRY, true); + } + + @Override + protected boolean overridesRequestedLocation() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_OVERRIDES_REQUESTED_LOCATION, + false); + } +} diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java new file mode 100644 index 000000000000..a7bbe64c0650 --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitSuite.java @@ -0,0 +1,45 @@ +/* + * 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 org.junit.platform.suite.api.SelectClasses; +import org.junit.platform.suite.api.Suite; +import org.junit.platform.suite.api.SuiteDisplayName; + +/** + * Iceberg REST Compatibility Kit + * + *

    This test suite provides the ability to run the Iceberg catalog tests against a remote REST + * catalog implementation to verify the behaviors against the reference implementation catalog + * tests. + * + *

    The tests can be configured through environment variables or system properties. By default, + * the tests will run using a local http server using a servlet implementation that leverages the + * {@link RESTCatalogAdapter}. + */ +@Suite +@SuiteDisplayName("Iceberg REST Compatibility Kit") +@SelectClasses({RESTCompatibilityKitCatalogTests.class, RESTCompatibilityKitViewCatalogTests.class}) +public class RESTCompatibilityKitSuite { + static final String RCK_REQUIRES_NAMESPACE_CREATE = "rck.requires-namespace-create"; + static final String RCK_SUPPORTS_SERVERSIDE_RETRY = "rck.supports-serverside-retry"; + static final String RCK_OVERRIDES_REQUESTED_LOCATION = "rck.overrides-requested-location"; + + protected RESTCompatibilityKitSuite() {} +} diff --git a/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java new file mode 100644 index 000000000000..7a18531bf6af --- /dev/null +++ b/open-api/src/test/java/org/apache/iceberg/rest/RESTCompatibilityKitViewCatalogTests.java @@ -0,0 +1,91 @@ +/* + * 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 org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.ViewCatalogTests; +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.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@ExtendWith(RESTServerExtension.class) +public class RESTCompatibilityKitViewCatalogTests extends ViewCatalogTests { + private static final Logger LOG = + LoggerFactory.getLogger(RESTCompatibilityKitViewCatalogTests.class); + private static RESTCatalog restCatalog; + + @BeforeAll + static void beforeClass() throws Exception { + restCatalog = RCKUtils.initCatalogClient(); + + assertThat(restCatalog.listNamespaces()) + .withFailMessage("Namespaces list should not contain: %s", RCKUtils.TEST_NAMESPACES) + .doesNotContainAnyElementsOf(RCKUtils.TEST_NAMESPACES); + } + + @BeforeEach + void before() { + try { + RCKUtils.purgeCatalogTestEntries(restCatalog); + } catch (Exception e) { + LOG.warn("Failure during test setup", e); + } + } + + @AfterAll + static void afterClass() throws Exception { + restCatalog.close(); + } + + @Override + protected RESTCatalog catalog() { + return restCatalog; + } + + @Override + protected Catalog tableCatalog() { + return restCatalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_REQUIRES_NAMESPACE_CREATE, true); + } + + @Override + protected boolean supportsServerSideRetry() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), RESTCompatibilityKitSuite.RCK_SUPPORTS_SERVERSIDE_RETRY, true); + } + + @Override + protected boolean overridesRequestedLocation() { + return PropertyUtil.propertyAsBoolean( + restCatalog.properties(), + RESTCompatibilityKitSuite.RCK_OVERRIDES_REQUESTED_LOCATION, + false); + } +} 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 new file mode 100644 index 000000000000..0f1571d362e3 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RCKUtils.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.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; + +class RCKUtils { + private static final String CATALOG_ENV_PREFIX = "CATALOG_"; + static final String RCK_LOCAL = "rck.local"; + static final String RCK_PURGE_TEST_NAMESPACES = "rck.purge-test-namespaces"; + + static final List TEST_NAMESPACES = List.of(Namespace.of("ns"), Namespace.of("newdb")); + + private RCKUtils() {} + + /** + * Utility method that allows configuring catalog properties via environment variables. + * + *

    Returns a property map for all environment variables that start with CATALOG_ + * replacing double-underscore (__) with dash (-) and replacing single + * underscore (_) with dot (.) to allow for common catalog property + * conventions. All characters in the name are converted to lowercase and values are unmodified. + * + *

    Examples: + * + *

    
    +   *     CATALOG_CATALOG__IMPL=org.apache.iceberg.jdbc.JdbcCatalog -> catalog-impl=org.apache.iceberg.jdbc.JdbcCatalog
    +   *     CATALOG_URI=jdbc:sqlite:memory: -> uri=jdbc:sqlite:memory:
    +   *     CATALOG_WAREHOUSE=test_warehouse -> warehouse=test_warehouse
    +   *     CATALOG_IO__IMPL=org.apache.iceberg.aws.s3.S3FileIO -> io-impl=org.apache.iceberg.aws.s3.S3FileIO
    +   *     CATALOG_JDBC_USER=ice_user -> jdbc.user=ice_user
    +   * 
    + * + * @return configuration map + */ + static Map environmentCatalogConfig() { + return System.getenv().entrySet().stream() + .filter(e -> e.getKey().startsWith(CATALOG_ENV_PREFIX)) + .collect( + Collectors.toMap( + e -> + e.getKey() + .replaceFirst(CATALOG_ENV_PREFIX, "") + .replaceAll("__", "-") + .replaceAll("_", ".") + .toLowerCase(Locale.ROOT), + Map.Entry::getValue, + (m1, m2) -> { + throw new IllegalArgumentException("Duplicate key: " + m1); + }, + HashMap::new)); + } + + static RESTCatalog initCatalogClient() { + Map catalogProperties = Maps.newHashMap(); + catalogProperties.putAll(RCKUtils.environmentCatalogConfig()); + catalogProperties.putAll(Maps.fromProperties(System.getProperties())); + + // Set defaults + catalogProperties.putIfAbsent( + CatalogProperties.URI, + String.format("http://localhost:%s/", RESTCatalogServer.REST_PORT_DEFAULT)); + catalogProperties.putIfAbsent(CatalogProperties.WAREHOUSE_LOCATION, "rck_warehouse"); + + RESTCatalog catalog = new RESTCatalog(); + catalog.setConf(new Configuration()); + catalog.initialize("rck_catalog", catalogProperties); + return catalog; + } + + static void purgeCatalogTestEntries(RESTCatalog catalog) { + if (!PropertyUtil.propertyAsBoolean(catalog.properties(), RCK_PURGE_TEST_NAMESPACES, true)) { + return; + } + + TEST_NAMESPACES.stream() + .filter(catalog::namespaceExists) + .forEach( + namespace -> { + catalog.listTables(namespace).forEach(catalog::dropTable); + catalog.listViews(namespace).forEach(catalog::dropView); + catalog.dropNamespace(namespace); + }); + } +} 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 new file mode 100644 index 000000000000..a71d7f13ce79 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTCatalogServer.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.rest; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.hadoop.conf.Configuration; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.jdbc.JdbcCatalog; +import org.apache.iceberg.util.PropertyUtil; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class RESTCatalogServer { + private static final Logger LOG = LoggerFactory.getLogger(RESTCatalogServer.class); + + static final String REST_PORT = "rest.port"; + static final int REST_PORT_DEFAULT = 8181; + + private Server httpServer; + + RESTCatalogServer() {} + + static class CatalogContext { + private final Catalog catalog; + private final Map configuration; + + CatalogContext(Catalog catalog, Map configuration) { + this.catalog = catalog; + this.configuration = configuration; + } + + public Catalog catalog() { + return catalog; + } + + public Map configuration() { + return configuration; + } + } + + private CatalogContext initializeBackendCatalog() throws IOException { + // Translate environment variables to catalog properties + Map catalogProperties = RCKUtils.environmentCatalogConfig(); + + // Fallback to a JDBCCatalog impl if one is not set + catalogProperties.putIfAbsent(CatalogProperties.CATALOG_IMPL, JdbcCatalog.class.getName()); + catalogProperties.putIfAbsent(CatalogProperties.URI, "jdbc:sqlite::memory:"); + catalogProperties.putIfAbsent("jdbc.schema-version", "V1"); + + // Configure a default location if one is not specified + String warehouseLocation = catalogProperties.get(CatalogProperties.WAREHOUSE_LOCATION); + + if (warehouseLocation == null) { + File tmp = java.nio.file.Files.createTempDirectory("iceberg_warehouse").toFile(); + tmp.deleteOnExit(); + warehouseLocation = tmp.toPath().resolve("iceberg_data").toFile().getAbsolutePath(); + catalogProperties.put(CatalogProperties.WAREHOUSE_LOCATION, warehouseLocation); + + LOG.info("No warehouse location set. Defaulting to temp location: {}", warehouseLocation); + } + + LOG.info("Creating catalog with properties: {}", catalogProperties); + return new CatalogContext( + CatalogUtil.buildIcebergCatalog("rest_backend", catalogProperties, new Configuration()), + catalogProperties); + } + + public void start(boolean join) throws Exception { + CatalogContext catalogContext = initializeBackendCatalog(); + + RESTCatalogAdapter adapter = new RESTServerCatalogAdapter(catalogContext); + RESTCatalogServlet servlet = new RESTCatalogServlet(adapter); + + ServletContextHandler context = new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + ServletHolder servletHolder = new ServletHolder(servlet); + context.addServlet(servletHolder, "/*"); + context.insertHandler(new GzipHandler()); + + this.httpServer = + new Server( + PropertyUtil.propertyAsInt(catalogContext.configuration, REST_PORT, REST_PORT_DEFAULT)); + httpServer.setHandler(context); + httpServer.start(); + + if(join) { + httpServer.join(); + } + } + + public void stop() throws Exception { + if (httpServer != null) { + httpServer.stop(); + } + } + + public static void main(String[] args) throws Exception { + new RESTCatalogServer().start(true); + } +} diff --git a/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java new file mode 100644 index 000000000000..612ed926f54d --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerCatalogAdapter.java @@ -0,0 +1,85 @@ +/* + * 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.aws.s3.S3FileIOProperties; +import org.apache.iceberg.azure.AzureProperties; +import org.apache.iceberg.gcp.GCPProperties; +import org.apache.iceberg.rest.RESTCatalogServer.CatalogContext; +import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.util.PropertyUtil; + +class RESTServerCatalogAdapter extends RESTCatalogAdapter { + private static final String INCLUDE_CREDENTIALS = "include-credentials"; + + private final CatalogContext catalogContext; + + RESTServerCatalogAdapter(CatalogContext catalogContext) { + super(catalogContext.catalog()); + this.catalogContext = catalogContext; + } + + @Override + public T handleRequest( + Route route, Map vars, Object body, Class responseType) { + T restResponse = super.handleRequest(route, vars, body, responseType); + + if (restResponse instanceof LoadTableResponse) { + if (PropertyUtil.propertyAsBoolean( + catalogContext.configuration(), INCLUDE_CREDENTIALS, false)) { + applyCredentials( + catalogContext.configuration(), ((LoadTableResponse) restResponse).config()); + } + } + + return restResponse; + } + + private void applyCredentials( + Map catalogConfig, Map tableConfig) { + if (catalogConfig.containsKey(S3FileIOProperties.ACCESS_KEY_ID)) { + tableConfig.put( + S3FileIOProperties.ACCESS_KEY_ID, catalogConfig.get(S3FileIOProperties.ACCESS_KEY_ID)); + } + + if (catalogConfig.containsKey(S3FileIOProperties.SECRET_ACCESS_KEY)) { + tableConfig.put( + S3FileIOProperties.SECRET_ACCESS_KEY, + catalogConfig.get(S3FileIOProperties.SECRET_ACCESS_KEY)); + } + + if (catalogConfig.containsKey(S3FileIOProperties.SESSION_TOKEN)) { + tableConfig.put( + S3FileIOProperties.SESSION_TOKEN, catalogConfig.get(S3FileIOProperties.SESSION_TOKEN)); + } + + if (catalogConfig.containsKey(GCPProperties.GCS_OAUTH2_TOKEN)) { + tableConfig.put( + GCPProperties.GCS_OAUTH2_TOKEN, catalogConfig.get(GCPProperties.GCS_OAUTH2_TOKEN)); + } + + catalogConfig.entrySet().stream() + .filter( + entry -> + entry.getKey().startsWith(AzureProperties.ADLS_SAS_TOKEN_PREFIX) + || entry.getKey().startsWith(AzureProperties.ADLS_CONNECTION_STRING_PREFIX)) + .forEach(entry -> tableConfig.put(entry.getKey(), entry.getValue())); + } +} 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 new file mode 100644 index 000000000000..14a8ed73c958 --- /dev/null +++ b/open-api/src/testFixtures/java/org/apache/iceberg/rest/RESTServerExtension.java @@ -0,0 +1,45 @@ +/* + * 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 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 { + private RESTCatalogServer localServer; + + @Override + public void beforeAll(ExtensionContext extensionContext) throws Exception { + if (Boolean.parseBoolean( + extensionContext + .getConfigurationParameter(RCKUtils.RCK_LOCAL) + .orElse("true"))) { + this.localServer = new RESTCatalogServer(); + this.localServer.start(false); + } + } + + @Override + public void afterAll(ExtensionContext extensionContext) throws Exception { + if (localServer != null) { + localServer.stop(); + } + } +} From eb9d9dfcfd9d0b6cf1f838a29aecdc46122f9460 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 27 Aug 2024 13:47:53 -0700 Subject: [PATCH 0647/1019] Core: Generate realistic bounds in benchmarks (#11022) --- .../apache/iceberg/FileGenerationUtil.java | 76 ++++++++++-- .../iceberg/TestFileGenerationUtil.java | 108 ++++++++++++++++++ 2 files changed, 176 insertions(+), 8 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index 98a6eafaf8f6..e48f23ff9a0b 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -19,27 +19,48 @@ package org.apache.iceberg; import java.nio.ByteBuffer; +import java.util.Comparator; import java.util.Map; import java.util.Random; import java.util.UUID; import java.util.concurrent.ThreadLocalRandom; +import org.apache.iceberg.MetricsModes.Counts; +import org.apache.iceberg.MetricsModes.MetricsMode; +import org.apache.iceberg.MetricsModes.None; +import org.apache.iceberg.MetricsModes.Truncate; import org.apache.iceberg.io.DeleteSchemaUtil; import org.apache.iceberg.io.LocationProvider; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.transforms.Transform; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Comparators; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.PrimitiveType; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; +import org.apache.iceberg.util.RandomUtil; public class FileGenerationUtil { private FileGenerationUtil() {} public static DataFile generateDataFile(Table table, StructLike partition) { + return generateDataFile(table, partition, ImmutableMap.of(), ImmutableMap.of()); + } + + public static DataFile generateDataFile( + Table table, + StructLike partition, + Map lowerBounds, + Map upperBounds) { Schema schema = table.schema(); PartitionSpec spec = table.spec(); LocationProvider locations = table.locationProvider(); String path = locations.newDataLocation(spec, partition, generateFileName()); long fileSize = generateFileSize(); - Metrics metrics = generateRandomMetrics(schema); + MetricsConfig metricsConfig = MetricsConfig.forTable(table); + Metrics metrics = generateRandomMetrics(schema, metricsConfig, lowerBounds, upperBounds); return DataFiles.builder(spec) .withPath(path) .withPartition(partition) @@ -91,7 +112,11 @@ public static String generateFileName() { return String.format("%d-%d-%s-%d.parquet", partitionId, taskId, operationId, fileCount); } - public static Metrics generateRandomMetrics(Schema schema) { + public static Metrics generateRandomMetrics( + Schema schema, + MetricsConfig metricsConfig, + Map knownLowerBounds, + Map knownUpperBounds) { long rowCount = generateRowCount(); Map columnSizes = Maps.newHashMap(); Map valueCounts = Maps.newHashMap(); @@ -106,12 +131,16 @@ public static Metrics generateRandomMetrics(Schema schema) { valueCounts.put(fieldId, generateValueCount()); nullValueCounts.put(fieldId, (long) random().nextInt(5)); nanValueCounts.put(fieldId, (long) random().nextInt(5)); - byte[] lower = new byte[16]; - random().nextBytes(lower); - lowerBounds.put(fieldId, ByteBuffer.wrap(lower)); - byte[] upper = new byte[16]; - random().nextBytes(upper); - upperBounds.put(fieldId, ByteBuffer.wrap(upper)); + if (knownLowerBounds.containsKey(fieldId) && knownUpperBounds.containsKey(fieldId)) { + lowerBounds.put(fieldId, knownLowerBounds.get(fieldId)); + upperBounds.put(fieldId, knownUpperBounds.get(fieldId)); + } else if (column.type().isPrimitiveType()) { + PrimitiveType type = column.type().asPrimitiveType(); + MetricsMode metricsMode = metricsConfig.columnMode(column.name()); + Pair bounds = generateBounds(type, metricsMode); + lowerBounds.put(fieldId, bounds.first()); + upperBounds.put(fieldId, bounds.second()); + } } return new Metrics( @@ -185,6 +214,37 @@ private static long generateFileSize() { return random().nextInt(50_000); } + private static Pair generateBounds(PrimitiveType type, MetricsMode mode) { + Comparator cmp = Comparators.forType(type); + Object value1 = generateBound(type, mode); + Object value2 = generateBound(type, mode); + if (cmp.compare(value1, value2) > 0) { + ByteBuffer lowerBuffer = Conversions.toByteBuffer(type, value2); + ByteBuffer upperBuffer = Conversions.toByteBuffer(type, value1); + return Pair.of(lowerBuffer, upperBuffer); + } else { + ByteBuffer lowerBuffer = Conversions.toByteBuffer(type, value1); + ByteBuffer upperBuffer = Conversions.toByteBuffer(type, value2); + return Pair.of(lowerBuffer, upperBuffer); + } + } + + private static Object generateBound(PrimitiveType type, MetricsMode mode) { + if (mode instanceof None || mode instanceof Counts) { + return null; + } else if (mode instanceof Truncate) { + Object value = RandomUtil.generatePrimitive(type, random()); + Transform truncate = Transforms.truncate(((Truncate) mode).length()); + if (truncate.canTransform(type)) { + return truncate.bind(type).apply(value); + } else { + return value; + } + } else { + return RandomUtil.generatePrimitive(type, random()); + } + } + private static Random random() { return ThreadLocalRandom.current(); } diff --git a/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java new file mode 100644 index 000000000000..ea44aa73c6d6 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestFileGenerationUtil.java @@ -0,0 +1,108 @@ +/* + * 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.required; +import static org.assertj.core.api.Assertions.assertThat; + +import java.nio.ByteBuffer; +import java.util.Comparator; +import org.apache.iceberg.MetricsModes.MetricsMode; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Type.PrimitiveType; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.types.Types.NestedField; +import org.junit.jupiter.api.Test; + +public class TestFileGenerationUtil { + + public static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + + @Test + public void testBoundsWithDefaultMetricsConfig() { + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + Metrics metrics = + FileGenerationUtil.generateRandomMetrics( + SCHEMA, + metricsConfig, + ImmutableMap.of() /* no known lower bounds */, + ImmutableMap.of() /* no known upper bounds */); + + assertThat(metrics.lowerBounds()).hasSize(SCHEMA.columns().size()); + assertThat(metrics.upperBounds()).hasSize(SCHEMA.columns().size()); + + checkBounds(metrics, metricsConfig); + } + + @Test + public void testBoundsWithSpecificValues() { + MetricsConfig metricsConfig = MetricsConfig.getDefault(); + NestedField intField = SCHEMA.findField("int_col"); + PrimitiveType type = intField.type().asPrimitiveType(); + ByteBuffer intLower = Conversions.toByteBuffer(type, 0); + ByteBuffer intUpper = Conversions.toByteBuffer(type, Integer.MAX_VALUE); + Metrics metrics = + FileGenerationUtil.generateRandomMetrics( + SCHEMA, + metricsConfig, + ImmutableMap.of(intField.fieldId(), intLower), + ImmutableMap.of(intField.fieldId(), intUpper)); + + assertThat(metrics.lowerBounds()).hasSize(SCHEMA.columns().size()); + assertThat(metrics.upperBounds()).hasSize(SCHEMA.columns().size()); + + checkBounds(metrics, metricsConfig); + + ByteBuffer actualIntLower = metrics.lowerBounds().get(intField.fieldId()); + ByteBuffer actualIntUpper = metrics.upperBounds().get(intField.fieldId()); + assertThat(actualIntLower).isEqualTo(intLower); + assertThat(actualIntUpper).isEqualTo(intUpper); + } + + private void checkBounds(Metrics metrics, MetricsConfig metricsConfig) { + for (NestedField field : SCHEMA.columns()) { + MetricsMode mode = metricsConfig.columnMode(field.name()); + ByteBuffer lowerBuffer = metrics.lowerBounds().get(field.fieldId()); + ByteBuffer upperBuffer = metrics.upperBounds().get(field.fieldId()); + if (mode.equals(MetricsModes.None.get()) || mode.equals(MetricsModes.Counts.get())) { + assertThat(lowerBuffer).isNull(); + assertThat(upperBuffer).isNull(); + } else { + checkBounds(field.type().asPrimitiveType(), lowerBuffer, upperBuffer); + } + } + } + + private void checkBounds(PrimitiveType type, ByteBuffer lowerBuffer, ByteBuffer upperBuffer) { + Object lower = Conversions.fromByteBuffer(type, lowerBuffer); + Object upper = Conversions.fromByteBuffer(type, upperBuffer); + Comparator cmp = Comparators.forType(type); + assertThat(cmp.compare(lower, upper)).isLessThanOrEqualTo(0); + } +} From ae2c18ab101c469e0098458f33714e3a5dbc9434 Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Tue, 27 Aug 2024 14:23:09 -0700 Subject: [PATCH 0648/1019] OpenAPI, Build: Apply spotless to testFixtures source code (#11024) --- baseline.gradle | 2 +- .../java/org/apache/iceberg/rest/RESTCatalogServer.java | 2 +- .../java/org/apache/iceberg/rest/RESTServerExtension.java | 4 +--- 3 files changed, 3 insertions(+), 5 deletions(-) diff --git a/baseline.gradle b/baseline.gradle index 8bf2dd3500e6..be2dc1198e94 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -73,7 +73,7 @@ subprojects { pluginManager.withPlugin('com.diffplug.spotless') { spotless { java { - target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' + target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/testFixtures/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' // we use an older version of google-java-format that is compatible with JDK 8 googleJavaFormat("1.7") removeUnusedImports() 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 a71d7f13ce79..b3d12f74e4b0 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 @@ -106,7 +106,7 @@ public void start(boolean join) throws Exception { httpServer.setHandler(context); httpServer.start(); - if(join) { + if (join) { httpServer.join(); } } 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 14a8ed73c958..fc3648055694 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 @@ -28,9 +28,7 @@ public class RESTServerExtension implements BeforeAllCallback, AfterAllCallback @Override public void beforeAll(ExtensionContext extensionContext) throws Exception { if (Boolean.parseBoolean( - extensionContext - .getConfigurationParameter(RCKUtils.RCK_LOCAL) - .orElse("true"))) { + extensionContext.getConfigurationParameter(RCKUtils.RCK_LOCAL).orElse("true"))) { this.localServer = new RESTCatalogServer(); this.localServer.start(false); } From 0ca30248375c78ae148bc5c3e5c3e8e8a5bf1a17 Mon Sep 17 00:00:00 2001 From: Carl Steinbach Date: Wed, 28 Aug 2024 06:21:28 -0700 Subject: [PATCH 0649/1019] Docs: bump latest version to 1.6.1 (#11036) * Docs: update latest version in doap and templates * Add missing files --- .github/ISSUE_TEMPLATE/iceberg_bug_report.yml | 3 ++- doap.rdf | 6 +++--- site/mkdocs.yml | 2 +- site/nav.yml | 1 + 4 files changed, 7 insertions(+), 5 deletions(-) diff --git a/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml b/.github/ISSUE_TEMPLATE/iceberg_bug_report.yml index 9e7d6034fe2e..a04263e8c943 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.0 (latest release)" + - "1.6.1 (latest release)" + - "1.6.0" - "1.5.2" - "1.5.1" - "1.5.0" diff --git a/doap.rdf b/doap.rdf index 4f2bd4222c1e..51896c6a8037 100644 --- a/doap.rdf +++ b/doap.rdf @@ -41,9 +41,9 @@ - 1.6.0 - 2024-07-23 - 1.6.0 + 1.6.1 + 2024-08-27 + 1.6.1 diff --git a/site/mkdocs.yml b/site/mkdocs.yml index d652ec3f0830..db9bafb00f05 100644 --- a/site/mkdocs.yml +++ b/site/mkdocs.yml @@ -78,7 +78,7 @@ markdown_extensions: permalink: 🔗 extra: - icebergVersion: '1.6.0' + icebergVersion: '1.6.1' nessieVersion: '0.92.1' flinkVersion: '1.19.0' flinkVersionMajor: '1.19' diff --git a/site/nav.yml b/site/nav.yml index 91f1494a1065..cad34766be4c 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.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' - 1.5.1: '!include docs/docs/1.5.1/mkdocs.yml' From 6b61406d32b150059186e8a152cbbf148fc4caae Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 28 Aug 2024 09:40:32 -0600 Subject: [PATCH 0650/1019] Spec: Add RemovePartitionSpecsUpdate REST update type (#10846) --- open-api/rest-catalog-open-api.py | 6 ++++++ open-api/rest-catalog-open-api.yaml | 16 ++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index cc89207b9f2a..46d8f2aa7718 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -371,6 +371,11 @@ class RemovePartitionStatisticsUpdate(BaseUpdate): snapshot_id: int = Field(..., alias='snapshot-id') +class RemovePartitionSpecsUpdate(BaseUpdate): + action: Optional[Literal['remove-partition-specs']] = None + spec_ids: List[int] = Field(..., alias='spec-ids') + + class AssertCreate(BaseModel): """ The table must not already exist; used for create transactions @@ -1081,6 +1086,7 @@ class TableUpdate(BaseModel): RemovePropertiesUpdate, SetStatisticsUpdate, RemoveStatisticsUpdate, + RemovePartitionSpecsUpdate, ] diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index 6236b8d45086..eec594eb89fb 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -2320,6 +2320,7 @@ components: remove-statistics: '#/components/schemas/RemoveStatisticsUpdate' set-partition-statistics: '#/components/schemas/SetPartitionStatisticsUpdate' remove-partition-statistics: '#/components/schemas/RemovePartitionStatisticsUpdate' + remove-partition-specs: '#/components/schemas/RemovePartitionSpecsUpdate' type: object required: - action @@ -2622,6 +2623,20 @@ components: type: integer format: int64 + RemovePartitionSpecsUpdate: + allOf: + - $ref: '#/components/schemas/BaseUpdate' + required: + - spec-ids + properties: + action: + type: string + enum: [ "remove-partition-specs" ] + spec-ids: + type: array + items: + type: integer + TableUpdate: anyOf: - $ref: '#/components/schemas/AssignUUIDUpdate' @@ -2641,6 +2656,7 @@ components: - $ref: '#/components/schemas/RemovePropertiesUpdate' - $ref: '#/components/schemas/SetStatisticsUpdate' - $ref: '#/components/schemas/RemoveStatisticsUpdate' + - $ref: '#/components/schemas/RemovePartitionSpecsUpdate' ViewUpdate: anyOf: From fbfd4f1671609a2411dcfbc5cf9b39df91b66aa2 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 28 Aug 2024 16:50:57 -0700 Subject: [PATCH 0651/1019] Build: Ignore benchmark output folders across all modules (#11030) --- .gitignore | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 6740a061e346..e89662a10c6d 100644 --- a/.gitignore +++ b/.gitignore @@ -37,7 +37,7 @@ spark/v3.4/spark/benchmark/* spark/v3.4/spark-extensions/benchmark/* spark/v3.5/spark/benchmark/* spark/v3.5/spark-extensions/benchmark/* -data/benchmark/* +*/benchmark/* __pycache__/ *.py[cod] From 7012773797d7529581e4f3c8124076c72d6d1078 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 28 Aug 2024 16:51:38 -0700 Subject: [PATCH 0652/1019] Core: Add benchmark for appending files (#11029) --- .../org/apache/iceberg/AppendBenchmark.java | 123 ++++++++++++++++++ 1 file changed, 123 insertions(+) create mode 100644 core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java diff --git a/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java new file mode 100644 index 000000000000..a8bafe413c6f --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.List; +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.types.Types; +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 evaluates the performance of appending files to the table. + * + *

    To run this benchmark: + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=AppendBenchmark + * -PjmhOutputPath=benchmark/append-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 10, timeUnit = TimeUnit.MINUTES) +public class AppendBenchmark { + + private static final String TABLE_IDENT = "tbl"; + private static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final HadoopTables TABLES = new HadoopTables(); + + private Table table; + private List dataFiles; + + @Param({"500000", "1000000", "2500000"}) + private int numFiles; + + @Param({"true", "false"}) + private boolean fast; + + @Setup + public void setupBenchmark() { + initTable(); + initDataFiles(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + } + + @Benchmark + @Threads(1) + public void appendFiles(Blackhole blackhole) { + AppendFiles append = fast ? table.newFastAppend() : table.newAppend(); + + for (DataFile dataFile : dataFiles) { + append.appendFile(dataFile); + } + + append.commit(); + } + + private void initTable() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + } + + private void dropTable() { + TABLES.dropTable(TABLE_IDENT); + } + + private void initDataFiles() { + List generatedDataFiles = Lists.newArrayListWithExpectedSize(numFiles); + + for (int ordinal = 0; ordinal < numFiles; ordinal++) { + DataFile dataFile = FileGenerationUtil.generateDataFile(table, null); + generatedDataFiles.add(dataFile); + } + + this.dataFiles = generatedDataFiles; + } +} From c791f37aee396aa634bdf3a459dff4485cab0e75 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 29 Aug 2024 08:38:32 -0700 Subject: [PATCH 0653/1019] Spark 3.5: Use FileGenerationUtil in PlanningBenchmark (#11027) --- .../iceberg/spark/PlanningBenchmark.java | 118 ++++-------------- 1 file changed, 26 insertions(+), 92 deletions(-) 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 e2ce5e956348..ed97e6b08414 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 @@ -20,49 +20,41 @@ import static org.apache.iceberg.PlanningMode.DISTRIBUTED; import static org.apache.iceberg.PlanningMode.LOCAL; -import static org.apache.spark.sql.functions.lit; import com.google.errorprone.annotations.FormatMethod; import com.google.errorprone.annotations.FormatString; import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.ByteBuffer; import java.util.List; +import java.util.Map; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.hadoop.conf.Configuration; -import org.apache.iceberg.AppendFiles; import org.apache.iceberg.BatchScan; 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.FileGenerationUtil; import org.apache.iceberg.PlanningMode; import org.apache.iceberg.RowDelta; import org.apache.iceberg.RowLevelOperationMode; import org.apache.iceberg.ScanTask; -import org.apache.iceberg.Schema; import org.apache.iceberg.SparkDistributedDataScan; +import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; import org.apache.iceberg.TableProperties; +import org.apache.iceberg.TestHelpers; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.LocationProvider; 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.spark.data.RandomData; import org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.sql.Dataset; -import org.apache.spark.sql.Row; +import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.types.Types; 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.parser.ParseException; -import org.apache.spark.sql.types.StructType; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.BenchmarkMode; import org.openjdk.jmh.annotations.Fork; @@ -108,10 +100,8 @@ public class PlanningBenchmark { Expressions.and(PARTITION_PREDICATE, SORT_KEY_PREDICATE); private static final int NUM_PARTITIONS = 30; - private static final int NUM_REAL_DATA_FILES_PER_PARTITION = 25; - private static final int NUM_REPLICA_DATA_FILES_PER_PARTITION = 50_000; + private static final int NUM_DATA_FILES_PER_PARTITION = 50_000; private static final int NUM_DELETE_FILES_PER_PARTITION = 50; - private static final int NUM_ROWS_PER_DATA_FILE = 500; private final Configuration hadoopConf = new Configuration(); private SparkSession spark; @@ -285,99 +275,43 @@ private void dropTable() { sql("DROP TABLE IF EXISTS %s PURGE", TABLE_NAME); } - private DataFile loadAddedDataFile() { - table.refresh(); - - Iterable dataFiles = table.currentSnapshot().addedDataFiles(table.io()); - return Iterables.getOnlyElement(dataFiles); - } - - private DeleteFile loadAddedDeleteFile() { - table.refresh(); - - Iterable deleteFiles = table.currentSnapshot().addedDeleteFiles(table.io()); - return Iterables.getOnlyElement(deleteFiles); - } - - private void initDataAndDeletes() throws NoSuchTableException { - Schema schema = table.schema(); - PartitionSpec spec = table.spec(); - LocationProvider locations = table.locationProvider(); - + private void initDataAndDeletes() { for (int partitionOrdinal = 0; partitionOrdinal < NUM_PARTITIONS; partitionOrdinal++) { - Dataset inputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)) - .drop(SORT_KEY_COLUMN) - .withColumn(SORT_KEY_COLUMN, lit(Integer.MIN_VALUE)); - - for (int fileOrdinal = 0; fileOrdinal < NUM_REAL_DATA_FILES_PER_PARTITION; fileOrdinal++) { - appendAsFile(inputDF); - } + StructLike partition = TestHelpers.Row.of(partitionOrdinal); - DataFile dataFile = loadAddedDataFile(); - - sql( - "DELETE FROM %s WHERE ss_item_sk IS NULL AND %s = %d", - TABLE_NAME, PARTITION_COLUMN, partitionOrdinal); - - DeleteFile deleteFile = loadAddedDeleteFile(); - - AppendFiles append = table.newFastAppend(); + RowDelta rowDelta = table.newRowDelta(); - for (int fileOrdinal = 0; fileOrdinal < NUM_REPLICA_DATA_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DataFile replicaDataFile = - DataFiles.builder(spec) - .copy(dataFile) - .withPath(locations.newDataLocation(spec, dataFile.partition(), replicaFileName)) - .build(); - append.appendFile(replicaDataFile); + for (int fileOrdinal = 0; fileOrdinal < NUM_DATA_FILES_PER_PARTITION; fileOrdinal++) { + DataFile dataFile = generateDataFile(partition, Integer.MIN_VALUE, Integer.MIN_VALUE); + rowDelta.addRows(dataFile); } - append.commit(); - - RowDelta rowDelta = table.newRowDelta(); + // add one data file that would match the sort key predicate + DataFile sortKeyDataFile = generateDataFile(partition, SORT_KEY_VALUE, SORT_KEY_VALUE); + rowDelta.addRows(sortKeyDataFile); for (int fileOrdinal = 0; fileOrdinal < NUM_DELETE_FILES_PER_PARTITION; fileOrdinal++) { - String replicaFileName = UUID.randomUUID() + "-replica.parquet"; - DeleteFile replicaDeleteFile = - FileMetadata.deleteFileBuilder(spec) - .copy(deleteFile) - .withPath(locations.newDataLocation(spec, deleteFile.partition(), replicaFileName)) - .build(); - rowDelta.addDeletes(replicaDeleteFile); + DeleteFile deleteFile = FileGenerationUtil.generatePositionDeleteFile(table, partition); + rowDelta.addDeletes(deleteFile); } rowDelta.commit(); - - Dataset sortedInputDF = - randomDataDF(schema, NUM_ROWS_PER_DATA_FILE) - .drop(SORT_KEY_COLUMN) - .withColumn(SORT_KEY_COLUMN, lit(SORT_KEY_VALUE)) - .drop(PARTITION_COLUMN) - .withColumn(PARTITION_COLUMN, lit(partitionOrdinal)); - appendAsFile(sortedInputDF); } } - private void appendAsFile(Dataset df) throws NoSuchTableException { - df.coalesce(1).writeTo(TABLE_NAME).append(); + 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); + Map lowerBounds = ImmutableMap.of(sortKeyFieldId, lower); + ByteBuffer upper = Conversions.toByteBuffer(Types.IntegerType.get(), sortKeyMax); + Map upperBounds = ImmutableMap.of(sortKeyFieldId, upper); + return FileGenerationUtil.generateDataFile(table, partition, lowerBounds, upperBounds); } private String newWarehouseDir() { return hadoopConf.get("hadoop.tmp.dir") + UUID.randomUUID(); } - private Dataset randomDataDF(Schema schema, int numRows) { - Iterable rows = RandomData.generateSpark(schema, numRows, 0); - JavaSparkContext context = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD rowRDD = context.parallelize(Lists.newArrayList(rows)); - StructType rowSparkType = SparkSchemaUtil.convert(schema); - return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); - } - private List planFilesWithoutColumnStats(BatchScan scan, Expression predicate) { return planFiles(scan, predicate, false); } From 3701b4b0406b9a316319532d99989032f38a194c Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 29 Aug 2024 09:27:23 -0700 Subject: [PATCH 0654/1019] Flink: add unit tests for range distribution on bucket partition column (#11033) --- ...IcebergSinkRangeDistributionBucketing.java | 253 ++++++++++++++++++ ...IcebergSinkRangeDistributionBucketing.java | 253 ++++++++++++++++++ 2 files changed, 506 insertions(+) create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java 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 new file mode 100644 index 000000000000..a5f24e09a60b --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -0,0 +1,253 @@ +/* + * 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 static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test range distribution with bucketing partition column. Compared to hash distribution, range + * distribution is more general to handle bucketing column while achieving even distribution of + * traffic to writer tasks. + * + *

      + *
    • keyBy on low cardinality (e.g. + * 60) may not achieve balanced data distribution. + *
    • number of buckets (e.g. 60) is not divisible by the writer parallelism (e.g. 40). + *
    • number of buckets (e.g. 60) is smaller than the writer parallelism (e.g. 120). + *
    + */ +@Timeout(value = 30) +public class TestFlinkIcebergSinkRangeDistributionBucketing { + private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + // max supported parallelism is 16 (= 4 x 4) + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(4) + .setNumberSlotsPerTaskManager(4) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private static final int NUM_BUCKETS = 4; + private static final int NUM_OF_CHECKPOINTS = 4; + private static final int ROW_COUNT_PER_CHECKPOINT = 200; + 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 PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("uuid", NUM_BUCKETS).build(); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + private TableLoader tableLoader; + private Table table; + + @BeforeEach + public void before() throws IOException { + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name())); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + // Assuming ts is on ingestion/processing time. Writer only writes to 1 or 2 hours concurrently. + // Only sort on the bucket column to avoid each writer task writes to 60 buckets/files + // concurrently. + table.replaceSortOrder().asc(bucket("uuid", NUM_BUCKETS)).commit(); + } + + @AfterEach + public void after() throws Exception { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + /** number of buckets 4 matches writer parallelism of 4 */ + @Test + public void testBucketNumberEqualsToWriterParallelism() throws Exception { + testParallelism(4); + } + + /** number of buckets 4 is less than writer parallelism of 6 */ + @Test + public void testBucketNumberLessThanWriterParallelismNotDivisible() throws Exception { + testParallelism(6); + } + + /** number of buckets 4 is less than writer parallelism of 8 */ + @Test + public void testBucketNumberLessThanWriterParallelismDivisible() throws Exception { + testParallelism(8); + } + + /** number of buckets 4 is greater than writer parallelism of 3 */ + @Test + public void testBucketNumberHigherThanWriterParallelismNotDivisible() throws Exception { + testParallelism(3); + } + + /** number of buckets 4 is greater than writer parallelism of 2 */ + @Test + public void testBucketNumberHigherThanWriterParallelismDivisible() throws Exception { + testParallelism(2); + } + + private void testParallelism(int parallelism) throws Exception { + try (StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism)) { + + DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + new RowGenerator(), + ROW_COUNT_PER_CHECKPOINT * NUM_OF_CHECKPOINTS, + RateLimiterStrategy.perCheckpoint(ROW_COUNT_PER_CHECKPOINT), + FlinkCompatibilityUtil.toTypeInfo(ROW_TYPE)); + DataStream dataStream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the oldest snapshot to the newest 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()); + + // Source rate limit per checkpoint cycle may not be super precise. + // There could be more checkpoint cycles and commits than planned. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(NUM_OF_CHECKPOINTS); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + assertThat(addedDataFiles) + .hasSizeLessThanOrEqualTo(maxAddedDataFilesPerCheckpoint(parallelism)); + } + } + } + + /** + * Traffic is not perfectly balanced across all buckets in the small sample size Range + * distribution of the bucket id may cross subtask boundary. Hence the number of committed data + * files per checkpoint maybe larger than writer parallelism or the number of buckets. But it + * should not be more than the sum of those two. Without range distribution, the number of data + * files per commit can be 4x of parallelism (as the number of buckets is 4). + */ + private int maxAddedDataFilesPerCheckpoint(int parallelism) { + return NUM_BUCKETS + parallelism; + } + + private static class RowGenerator implements GeneratorFunction { + // use constant timestamp so that all rows go to the same hourly partition + private final long ts = System.currentTimeMillis(); + + @Override + public RowData map(Long index) throws Exception { + // random uuid should result in relatively balanced distribution across buckets + UUID uuid = UUID.randomUUID(); + ByteBuffer uuidByteBuffer = ByteBuffer.allocate(16); + uuidByteBuffer.putLong(uuid.getMostSignificantBits()); + uuidByteBuffer.putLong(uuid.getLeastSignificantBits()); + return GenericRowData.of( + TimestampData.fromEpochMillis(ts), + uuidByteBuffer.array(), + StringData.fromString("row-" + index)); + } + } +} 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 new file mode 100644 index 000000000000..a5f24e09a60b --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestFlinkIcebergSinkRangeDistributionBucketing.java @@ -0,0 +1,253 @@ +/* + * 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 static org.apache.iceberg.expressions.Expressions.bucket; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.flink.api.common.eventtime.WatermarkStrategy; +import org.apache.flink.api.connector.source.util.ratelimit.RateLimiterStrategy; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.configuration.CoreOptions; +import org.apache.flink.connector.datagen.source.DataGeneratorSource; +import org.apache.flink.connector.datagen.source.GeneratorFunction; +import org.apache.flink.runtime.testutils.MiniClusterResourceConfiguration; +import org.apache.flink.streaming.api.datastream.DataStream; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.table.data.GenericRowData; +import org.apache.flink.table.data.RowData; +import org.apache.flink.table.data.StringData; +import org.apache.flink.table.data.TimestampData; +import org.apache.flink.table.types.logical.RowType; +import org.apache.flink.test.junit5.MiniClusterExtension; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DistributionMode; +import org.apache.iceberg.FileFormat; +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.flink.FlinkSchemaUtil; +import org.apache.iceberg.flink.HadoopCatalogExtension; +import org.apache.iceberg.flink.MiniFlinkClusterExtension; +import org.apache.iceberg.flink.TableLoader; +import org.apache.iceberg.flink.TestFixtures; +import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; +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.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.Timeout; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** + * Test range distribution with bucketing partition column. Compared to hash distribution, range + * distribution is more general to handle bucketing column while achieving even distribution of + * traffic to writer tasks. + * + *
      + *
    • keyBy on low cardinality (e.g. + * 60) may not achieve balanced data distribution. + *
    • number of buckets (e.g. 60) is not divisible by the writer parallelism (e.g. 40). + *
    • number of buckets (e.g. 60) is smaller than the writer parallelism (e.g. 120). + *
    + */ +@Timeout(value = 30) +public class TestFlinkIcebergSinkRangeDistributionBucketing { + private static final Configuration DISABLE_CLASSLOADER_CHECK_CONFIG = + new Configuration() + // disable classloader check as Avro may cache class/object in the serializers. + .set(CoreOptions.CHECK_LEAKED_CLASSLOADER, false); + + // max supported parallelism is 16 (= 4 x 4) + @RegisterExtension + public static final MiniClusterExtension MINI_CLUSTER_EXTENSION = + new MiniClusterExtension( + new MiniClusterResourceConfiguration.Builder() + .setNumberTaskManagers(4) + .setNumberSlotsPerTaskManager(4) + .setConfiguration(DISABLE_CLASSLOADER_CHECK_CONFIG) + .build()); + + @RegisterExtension + private static final HadoopCatalogExtension CATALOG_EXTENSION = + new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); + + private static final int NUM_BUCKETS = 4; + private static final int NUM_OF_CHECKPOINTS = 4; + private static final int ROW_COUNT_PER_CHECKPOINT = 200; + 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 PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("uuid", NUM_BUCKETS).build(); + private static final RowType ROW_TYPE = FlinkSchemaUtil.convert(SCHEMA); + + private TableLoader tableLoader; + private Table table; + + @BeforeEach + public void before() throws IOException { + this.tableLoader = CATALOG_EXTENSION.tableLoader(); + this.table = + CATALOG_EXTENSION + .catalog() + .createTable( + TestFixtures.TABLE_IDENTIFIER, + SCHEMA, + SPEC, + ImmutableMap.of(TableProperties.DEFAULT_FILE_FORMAT, FileFormat.PARQUET.name())); + + table + .updateProperties() + .set(TableProperties.WRITE_DISTRIBUTION_MODE, DistributionMode.RANGE.modeName()) + .commit(); + + // Assuming ts is on ingestion/processing time. Writer only writes to 1 or 2 hours concurrently. + // Only sort on the bucket column to avoid each writer task writes to 60 buckets/files + // concurrently. + table.replaceSortOrder().asc(bucket("uuid", NUM_BUCKETS)).commit(); + } + + @AfterEach + public void after() throws Exception { + CATALOG_EXTENSION.catalog().dropTable(TestFixtures.TABLE_IDENTIFIER); + } + + /** number of buckets 4 matches writer parallelism of 4 */ + @Test + public void testBucketNumberEqualsToWriterParallelism() throws Exception { + testParallelism(4); + } + + /** number of buckets 4 is less than writer parallelism of 6 */ + @Test + public void testBucketNumberLessThanWriterParallelismNotDivisible() throws Exception { + testParallelism(6); + } + + /** number of buckets 4 is less than writer parallelism of 8 */ + @Test + public void testBucketNumberLessThanWriterParallelismDivisible() throws Exception { + testParallelism(8); + } + + /** number of buckets 4 is greater than writer parallelism of 3 */ + @Test + public void testBucketNumberHigherThanWriterParallelismNotDivisible() throws Exception { + testParallelism(3); + } + + /** number of buckets 4 is greater than writer parallelism of 2 */ + @Test + public void testBucketNumberHigherThanWriterParallelismDivisible() throws Exception { + testParallelism(2); + } + + private void testParallelism(int parallelism) throws Exception { + try (StreamExecutionEnvironment env = + StreamExecutionEnvironment.getExecutionEnvironment( + MiniFlinkClusterExtension.DISABLE_CLASSLOADER_CHECK_CONFIG) + .enableCheckpointing(100) + .setParallelism(parallelism) + .setMaxParallelism(parallelism)) { + + DataGeneratorSource generatorSource = + new DataGeneratorSource<>( + new RowGenerator(), + ROW_COUNT_PER_CHECKPOINT * NUM_OF_CHECKPOINTS, + RateLimiterStrategy.perCheckpoint(ROW_COUNT_PER_CHECKPOINT), + FlinkCompatibilityUtil.toTypeInfo(ROW_TYPE)); + DataStream dataStream = + env.fromSource(generatorSource, WatermarkStrategy.noWatermarks(), "Data Generator"); + + FlinkSink.forRowData(dataStream) + .table(table) + .tableLoader(tableLoader) + .writeParallelism(parallelism) + .append(); + env.execute(getClass().getSimpleName()); + + table.refresh(); + // ordered in reverse timeline from the oldest snapshot to the newest 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()); + + // Source rate limit per checkpoint cycle may not be super precise. + // There could be more checkpoint cycles and commits than planned. + assertThat(snapshots).hasSizeGreaterThanOrEqualTo(NUM_OF_CHECKPOINTS); + + // It takes 2 checkpoint cycle for statistics collection and application + // of the globally aggregated statistics in the range partitioner. + // The last two checkpoints should have range shuffle applied + List rangePartitionedCycles = + snapshots.subList(snapshots.size() - 2, snapshots.size()); + + for (Snapshot snapshot : rangePartitionedCycles) { + List addedDataFiles = + Lists.newArrayList(snapshot.addedDataFiles(table.io()).iterator()); + assertThat(addedDataFiles) + .hasSizeLessThanOrEqualTo(maxAddedDataFilesPerCheckpoint(parallelism)); + } + } + } + + /** + * Traffic is not perfectly balanced across all buckets in the small sample size Range + * distribution of the bucket id may cross subtask boundary. Hence the number of committed data + * files per checkpoint maybe larger than writer parallelism or the number of buckets. But it + * should not be more than the sum of those two. Without range distribution, the number of data + * files per commit can be 4x of parallelism (as the number of buckets is 4). + */ + private int maxAddedDataFilesPerCheckpoint(int parallelism) { + return NUM_BUCKETS + parallelism; + } + + private static class RowGenerator implements GeneratorFunction { + // use constant timestamp so that all rows go to the same hourly partition + private final long ts = System.currentTimeMillis(); + + @Override + public RowData map(Long index) throws Exception { + // random uuid should result in relatively balanced distribution across buckets + UUID uuid = UUID.randomUUID(); + ByteBuffer uuidByteBuffer = ByteBuffer.allocate(16); + uuidByteBuffer.putLong(uuid.getMostSignificantBits()); + uuidByteBuffer.putLong(uuid.getLeastSignificantBits()); + return GenericRowData.of( + TimestampData.fromEpochMillis(ts), + uuidByteBuffer.array(), + StringData.fromString("row-" + index)); + } + } +} From 5e1eb7ced61438a74470af45308742490a6f6e0a Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Thu, 29 Aug 2024 20:48:25 -0700 Subject: [PATCH 0655/1019] Kafka Connect: Disable publish tasks in runtime project (#11032) --- kafka-connect/build.gradle | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index c7f20e563577..785dc0a8c1ee 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -221,6 +221,11 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { } } + // there are no Maven artifacts so disable publishing tasks... + project.afterEvaluate { + project.tasks.matching { it.group == 'publishing' }.each {it.enabled = false} + } + tasks.jar.enabled = false tasks.distTar.enabled = false From d3e66df8261bb2493e18773a2d6bc972ba2d6ea9 Mon Sep 17 00:00:00 2001 From: Qishang Zhong Date: Sat, 31 Aug 2024 02:53:47 +0800 Subject: [PATCH 0656/1019] Flink: Backport PR #10526 to v1.18 and v1.20 (#11018) --- .../apache/iceberg/flink/sink/FlinkSink.java | 11 +- .../iceberg/flink/sink/FlinkWriteResult.java | 40 +++ .../flink/sink/IcebergFilesCommitter.java | 48 ++-- .../flink/sink/IcebergStreamWriter.java | 13 +- .../flink/sink/TestCompressionSettings.java | 11 +- .../flink/sink/TestIcebergFilesCommitter.java | 240 +++++++++++++----- .../flink/sink/TestIcebergStreamWriter.java | 61 +++-- .../apache/iceberg/flink/sink/FlinkSink.java | 11 +- .../iceberg/flink/sink/FlinkWriteResult.java | 40 +++ .../flink/sink/IcebergFilesCommitter.java | 48 ++-- .../flink/sink/IcebergStreamWriter.java | 13 +- .../flink/sink/TestCompressionSettings.java | 11 +- .../flink/sink/TestIcebergFilesCommitter.java | 240 +++++++++++++----- .../flink/sink/TestIcebergStreamWriter.java | 61 +++-- 14 files changed, 598 insertions(+), 250 deletions(-) create mode 100644 flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java index 2256d1e874ce..5cd43a46de37 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkSink.java @@ -67,7 +67,6 @@ import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; 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; @@ -420,7 +419,7 @@ private DataStreamSink chainIcebergOperators() { distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files @@ -487,7 +486,7 @@ private DataStreamSink appendDummySink( } private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -507,7 +506,7 @@ private SingleOutputStreamOperator appendCommitter( return committerStream; } - private SingleOutputStreamOperator appendWriter( + private SingleOutputStreamOperator appendWriter( DataStream input, RowType flinkRowType, List equalityFieldIds, @@ -545,11 +544,11 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) .setParallelism(writerParallelism); if (uidPrefix != null) { diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.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.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index b9bceaa9311d..7108c2008341 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -63,7 +63,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -96,7 +96,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -212,7 +212,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -220,8 +221,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -426,30 +425,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 9ea0349fb057..bb5efe982ee1 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,7 +90,7 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override @@ -102,7 +103,7 @@ public String toString() { } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 8faae1b05a4e..3299e7a97776 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -210,8 +209,10 @@ public void testCompressionOrc() throws Exception { .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + private static OneInputStreamOperatorTestHarness + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -219,7 +220,7 @@ private static OneInputStreamOperatorTestHarness createIce IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -230,7 +231,7 @@ private static OneInputStreamOperatorTestHarness createIce private static Map appenderProperties( Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); 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 948c7b31430c..ac5babe11943 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 @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -161,7 +162,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -176,8 +178,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -193,7 +195,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -204,7 +207,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -233,7 +236,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -243,21 +247,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -286,7 +290,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -296,21 +301,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -337,7 +342,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -349,8 +355,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -362,7 +368,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -375,9 +382,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -400,7 +407,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -411,15 +419,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -446,15 +455,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -473,9 +482,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -496,7 +505,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -510,8 +519,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -528,7 +537,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -542,8 +551,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -567,7 +576,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -579,7 +589,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -603,8 +613,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -620,14 +632,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -643,8 +655,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -668,13 +682,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -694,7 +708,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -706,13 +721,14 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -725,7 +741,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -735,7 +752,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -775,7 +792,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -784,7 +802,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -816,13 +834,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -846,7 +866,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -860,7 +881,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -872,11 +895,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -887,6 +912,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + *
      + *
    • A specific row is updated + *
    • The prepareSnapshotPreBarrier triggered + *
    • Checkpoint failed for reasons outside of the Iceberg connector + *
    • The specific row is updated again in the second checkpoint as well + *
    • Second snapshot is triggered, and finished + *
    + * + *

    Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -899,7 +997,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -910,7 +1009,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -929,7 +1028,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -947,7 +1046,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -963,7 +1063,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1089,7 +1189,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1109,7 +1209,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 50283f7ad215..e13721a9f170 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -28,6 +28,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -102,7 +103,7 @@ public void before() throws IOException { @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,7 +112,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -123,7 +125,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -148,14 +151,15 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -163,7 +167,10 @@ public void testSnapshotTwice() throws Exception { for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); } @@ -172,14 +179,14 @@ public void testSnapshotTwice() throws Exception { @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. @@ -212,7 +219,7 @@ private Set scanDataFiles() throws IOException { @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,13 +228,15 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -236,7 +245,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -244,13 +253,15 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once @@ -275,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -283,7 +294,8 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(8); @@ -346,13 +358,14 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + private static List getWriteResults(List flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness createIcebergStr IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); 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 be2a8db03097..c53431490984 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 @@ -67,7 +67,6 @@ import org.apache.iceberg.flink.sink.shuffle.StatisticsOrRecord; import org.apache.iceberg.flink.sink.shuffle.StatisticsType; import org.apache.iceberg.flink.util.FlinkCompatibilityUtil; -import org.apache.iceberg.io.WriteResult; 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; @@ -421,7 +420,7 @@ private DataStreamSink chainIcebergOperators() { distributeDataStream(rowDataInput, equalityFieldIds, flinkRowType, writerParallelism); // Add parallel writers that append rows to files - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = appendWriter(distributeStream, flinkRowType, equalityFieldIds, writerParallelism); // Add single-parallelism committer that commits files @@ -488,7 +487,7 @@ private DataStreamSink appendDummySink( } private SingleOutputStreamOperator appendCommitter( - SingleOutputStreamOperator writerStream) { + SingleOutputStreamOperator writerStream) { IcebergFilesCommitter filesCommitter = new IcebergFilesCommitter( tableLoader, @@ -508,7 +507,7 @@ private SingleOutputStreamOperator appendCommitter( return committerStream; } - private SingleOutputStreamOperator appendWriter( + private SingleOutputStreamOperator appendWriter( DataStream input, RowType flinkRowType, List equalityFieldIds, @@ -546,11 +545,11 @@ private SingleOutputStreamOperator appendWriter( IcebergStreamWriter streamWriter = createStreamWriter(tableSupplier, flinkWriteConf, flinkRowType, equalityFieldIds); - SingleOutputStreamOperator writerStream = + SingleOutputStreamOperator writerStream = input .transform( operatorName(ICEBERG_STREAM_WRITER_NAME), - TypeInformation.of(WriteResult.class), + TypeInformation.of(FlinkWriteResult.class), streamWriter) .setParallelism(writerParallelism); if (uidPrefix != null) { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.java new file mode 100644 index 000000000000..317fb169ae1b --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/FlinkWriteResult.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.flink.sink; + +import java.io.Serializable; +import org.apache.iceberg.io.WriteResult; + +public class FlinkWriteResult implements Serializable { + private final long checkpointId; + private final WriteResult writeResult; + + public FlinkWriteResult(long checkpointId, WriteResult writeResult) { + this.checkpointId = checkpointId; + this.writeResult = writeResult; + } + + public long checkpointId() { + return checkpointId; + } + + public WriteResult writeResult() { + return writeResult; + } +} diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index 622daa808897..609deb621f43 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -61,7 +61,7 @@ import org.slf4j.LoggerFactory; class IcebergFilesCommitter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; private static final long INITIAL_CHECKPOINT_ID = -1L; @@ -94,7 +94,7 @@ class IcebergFilesCommitter extends AbstractStreamOperator // The completed files cache for current checkpoint. Once the snapshot barrier received, it will // be flushed to the 'dataFilesPerCheckpoint'. - private final List writeResultsOfCurrentCkpt = Lists.newArrayList(); + private final Map> writeResultsSinceLastSnapshot = Maps.newHashMap(); private final String branch; // It will have an unique identifier for one job. @@ -210,7 +210,8 @@ public void snapshotState(StateSnapshotContext context) throws Exception { // Update the checkpoint state. long startNano = System.nanoTime(); - dataFilesPerCheckpoint.put(checkpointId, writeToManifest(checkpointId)); + writeToManifestUptoLatestCheckpoint(checkpointId); + // Reset the snapshot state to the latest state. checkpointsState.clear(); checkpointsState.add(dataFilesPerCheckpoint); @@ -218,8 +219,6 @@ public void snapshotState(StateSnapshotContext context) throws Exception { jobIdState.clear(); jobIdState.add(flinkJobId); - // Clear the local buffer for current checkpoint. - writeResultsOfCurrentCkpt.clear(); committerMetrics.checkpointDuration( TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); } @@ -403,30 +402,45 @@ private void commitOperation( } @Override - public void processElement(StreamRecord element) { - this.writeResultsOfCurrentCkpt.add(element.getValue()); + public void processElement(StreamRecord element) { + FlinkWriteResult flinkWriteResult = element.getValue(); + List writeResults = + writeResultsSinceLastSnapshot.computeIfAbsent( + flinkWriteResult.checkpointId(), k -> Lists.newArrayList()); + writeResults.add(flinkWriteResult.writeResult()); } @Override public void endInput() throws IOException { // Flush the buffered data files into 'dataFilesPerCheckpoint' firstly. - long currentCheckpointId = Long.MAX_VALUE; - dataFilesPerCheckpoint.put(currentCheckpointId, writeToManifest(currentCheckpointId)); - writeResultsOfCurrentCkpt.clear(); - + long currentCheckpointId = IcebergStreamWriter.END_INPUT_CHECKPOINT_ID; + writeToManifestUptoLatestCheckpoint(currentCheckpointId); commitUpToCheckpoint(dataFilesPerCheckpoint, flinkJobId, operatorUniqueId, currentCheckpointId); } + private void writeToManifestUptoLatestCheckpoint(long checkpointId) throws IOException { + if (!writeResultsSinceLastSnapshot.containsKey(checkpointId)) { + dataFilesPerCheckpoint.put(checkpointId, EMPTY_MANIFEST_DATA); + } + + for (Map.Entry> writeResultsOfCheckpoint : + writeResultsSinceLastSnapshot.entrySet()) { + dataFilesPerCheckpoint.put( + writeResultsOfCheckpoint.getKey(), + writeToManifest(writeResultsOfCheckpoint.getKey(), writeResultsOfCheckpoint.getValue())); + } + + // Clear the local buffer for current checkpoint. + writeResultsSinceLastSnapshot.clear(); + } + /** * Write all the complete data files to a newly created manifest file and return the manifest's * avro serialized bytes. */ - private byte[] writeToManifest(long checkpointId) throws IOException { - if (writeResultsOfCurrentCkpt.isEmpty()) { - return EMPTY_MANIFEST_DATA; - } - - WriteResult result = WriteResult.builder().addAll(writeResultsOfCurrentCkpt).build(); + private byte[] writeToManifest(long checkpointId, List writeResults) + throws IOException { + WriteResult result = WriteResult.builder().addAll(writeResults).build(); DeltaManifests deltaManifests = FlinkManifestUtil.writeCompletedFiles( result, () -> manifestOutputFileFactory.create(checkpointId), spec); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java index 7d86baa14fc2..412d6c7081bf 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergStreamWriter.java @@ -29,10 +29,11 @@ import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -class IcebergStreamWriter extends AbstractStreamOperator - implements OneInputStreamOperator, BoundedOneInput { +class IcebergStreamWriter extends AbstractStreamOperator + implements OneInputStreamOperator, BoundedOneInput { private static final long serialVersionUID = 1L; + static final long END_INPUT_CHECKPOINT_ID = Long.MAX_VALUE; private final String fullTableName; private final TaskWriterFactory taskWriterFactory; @@ -63,7 +64,7 @@ public void open() { @Override public void prepareSnapshotPreBarrier(long checkpointId) throws Exception { - flush(); + flush(checkpointId); this.writer = taskWriterFactory.create(); } @@ -89,7 +90,7 @@ public void endInput() throws IOException { // Note that if the task is not closed after calling endInput, checkpoint may be triggered again // causing files to be sent repeatedly, the writer is marked as null after the last file is sent // to guard against duplicated writes. - flush(); + flush(END_INPUT_CHECKPOINT_ID); } @Override @@ -102,7 +103,7 @@ public String toString() { } /** close all open files and emit files to downstream committer operator */ - private void flush() throws IOException { + private void flush(long checkpointId) throws IOException { if (writer == null) { return; } @@ -110,7 +111,7 @@ private void flush() throws IOException { long startNano = System.nanoTime(); WriteResult result = writer.complete(); writerMetrics.updateFlushResult(result); - output.collect(new StreamRecord<>(result)); + output.collect(new StreamRecord<>(new FlinkWriteResult(checkpointId, result))); writerMetrics.flushDuration(TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startNano)); // Set writer to null to prevent duplicate flushes in the corner case of diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java index 8faae1b05a4e..3299e7a97776 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestCompressionSettings.java @@ -40,7 +40,6 @@ import org.apache.iceberg.flink.SimpleDataUtil; import org.apache.iceberg.io.BaseTaskWriter; import org.apache.iceberg.io.TaskWriter; -import org.apache.iceberg.io.WriteResult; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -210,8 +209,10 @@ public void testCompressionOrc() throws Exception { .containsEntry(TableProperties.ORC_COMPRESSION_STRATEGY, "speed"); } - private static OneInputStreamOperatorTestHarness createIcebergStreamWriter( - Table icebergTable, TableSchema flinkSchema, Map override) throws Exception { + private static OneInputStreamOperatorTestHarness + createIcebergStreamWriter( + Table icebergTable, TableSchema flinkSchema, Map override) + throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = new FlinkWriteConf( @@ -219,7 +220,7 @@ private static OneInputStreamOperatorTestHarness createIce IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); @@ -230,7 +231,7 @@ private static OneInputStreamOperatorTestHarness createIce private static Map appenderProperties( Table table, TableSchema schema, Map override) throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(table, schema, override)) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); 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 67ca9d08b206..7808771d9887 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 @@ -129,7 +129,8 @@ public void testCommitTxnWithoutDataFiles() throws Exception { long timestamp = 0; JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -160,7 +161,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { JobID jobId = new JobID(); long checkpointId = 0; long timestamp = 0; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); @@ -175,8 +177,8 @@ public void testMaxContinuousEmptyCommits() throws Exception { } } - private WriteResult of(DataFile dataFile) { - return WriteResult.builder().addDataFiles(dataFile).build(); + private FlinkWriteResult of(long checkpointId, DataFile dataFile) { + return new FlinkWriteResult(checkpointId, WriteResult.builder().addDataFiles(dataFile).build()); } @TestTemplate @@ -192,7 +194,8 @@ public void testCommitTxn() throws Exception { JobID jobID = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobID)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobID)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -203,7 +206,7 @@ public void testCommitTxn() throws Exception { for (int i = 1; i <= 3; i++) { RowData rowData = SimpleDataUtil.createRowData(i, "hello" + i); DataFile dataFile = writeDataFile("data-" + i, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(i, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(i, ++timestamp); @@ -232,7 +235,8 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -242,21 +246,21 @@ public void testOrderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -285,7 +289,8 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -295,21 +300,21 @@ public void testDisorderedEventsBetweenCheckpoints() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + long firstCheckpointId = 1; + harness.processElement(of(firstCheckpointId, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 - long firstCheckpointId = 1; harness.snapshot(firstCheckpointId, ++timestamp); assertFlinkManifests(1); RowData row2 = SimpleDataUtil.createRowData(2, "world"); DataFile dataFile2 = writeDataFile("data-2", ImmutableList.of(row2)); - harness.processElement(of(dataFile2), ++timestamp); + long secondCheckpointId = 2; + harness.processElement(of(secondCheckpointId, dataFile2), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 2. snapshotState for checkpoint#2 - long secondCheckpointId = 2; harness.snapshot(secondCheckpointId, ++timestamp); assertFlinkManifests(2); @@ -336,7 +341,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -348,8 +354,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile1), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -361,7 +367,8 @@ public void testRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -374,9 +381,9 @@ public void testRecoveryFromValidSnapshot() throws Exception { RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -399,7 +406,8 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except List expectedRows = Lists.newArrayList(); JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -410,15 +418,16 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(1, "hello"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-1", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); SimpleDataUtil.assertTableRows(table, ImmutableList.of(), branch); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); assertFlinkManifests(1); } - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -445,15 +454,15 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(2, "world"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-2", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - snapshot = harness.snapshot(++checkpointId, ++timestamp); + snapshot = harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); } // Redeploying flink job from external checkpoint. JobID newJobId = new JobID(); - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.initializeState(snapshot); @@ -472,9 +481,9 @@ public void testRecoveryFromSnapshotWithoutCompletedNotification() throws Except RowData row = SimpleDataUtil.createRowData(3, "foo"); expectedRows.add(row); DataFile dataFile = writeDataFile("data-3", ImmutableList.of(row)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -495,7 +504,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { JobID oldJobId = new JobID(); OperatorID oldOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(oldJobId)) { harness.setup(); harness.open(); @@ -509,8 +518,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -527,7 +536,7 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { timestamp = 0; JobID newJobId = new JobID(); OperatorID newOperatorId; - try (OneInputStreamOperatorTestHarness harness = + try (OneInputStreamOperatorTestHarness harness = createStreamSink(newJobId)) { harness.setup(); harness.open(); @@ -541,8 +550,8 @@ public void testStartAnotherJobToWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile("data-new-1", rows); - harness.processElement(of(dataFile), ++timestamp); - harness.snapshot(++checkpointId, ++timestamp); + harness.processElement(of(++checkpointId, dataFile), ++timestamp); + harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); harness.notifyOfCompletedCheckpoint(checkpointId); @@ -566,7 +575,8 @@ public void testMultipleJobsWriteSameTable() throws Exception { int checkpointId = i / 3; JobID jobId = jobs[jobIndex]; OperatorID operatorId = operatorIds[jobIndex]; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.open(); @@ -578,7 +588,7 @@ public void testMultipleJobsWriteSameTable() throws Exception { tableRows.addAll(rows); DataFile dataFile = writeDataFile(String.format("data-%d", i), rows); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId + 1, dataFile), ++timestamp); harness.snapshot(checkpointId + 1, ++timestamp); assertFlinkManifests(1); @@ -602,8 +612,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { JobID jobId = new JobID(); OperatorID operatorId1 = new OperatorID(); OperatorID operatorId2 = new OperatorID(); - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.open(); @@ -619,14 +631,14 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-1-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - snapshot1 = harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + snapshot1 = harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(1, "hello2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-1-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); snapshot2 = harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -642,8 +654,10 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness1 = createStreamSink(jobId); - OneInputStreamOperatorTestHarness harness2 = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness1 = + createStreamSink(jobId); + OneInputStreamOperatorTestHarness harness2 = + createStreamSink(jobId)) { harness1.getStreamConfig().setOperatorID(operatorId1); harness1.setup(); harness1.initializeState(snapshot1); @@ -667,13 +681,13 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { expectedRows.add(row1); DataFile dataFile1 = writeDataFile("data-2-1", ImmutableList.of(row1)); - harness1.processElement(of(dataFile1), ++timestamp); - harness1.snapshot(++checkpointId, ++timestamp); + harness1.processElement(of(++checkpointId, dataFile1), ++timestamp); + harness1.snapshot(checkpointId, ++timestamp); RowData row2 = SimpleDataUtil.createRowData(2, "world2"); expectedRows.add(row2); DataFile dataFile2 = writeDataFile("data-2-2", ImmutableList.of(row2)); - harness2.processElement(of(dataFile2), ++timestamp); + harness2.processElement(of(checkpointId, dataFile2), ++timestamp); harness2.snapshot(checkpointId, ++timestamp); assertFlinkManifests(2); @@ -693,7 +707,8 @@ public void testMultipleSinksRecoveryFromValidSnapshot() throws Exception { public void testBoundedStream() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -705,13 +720,14 @@ public void testBoundedStream() throws Exception { List tableRows = Lists.newArrayList(SimpleDataUtil.createRowData(1, "word-1")); DataFile dataFile = writeDataFile("data-1", tableRows); - harness.processElement(of(dataFile), 1); + harness.processElement(of(IcebergStreamWriter.END_INPUT_CHECKPOINT_ID, dataFile), 1); ((BoundedOneInput) harness.getOneInputOperator()).endInput(); assertFlinkManifests(0); SimpleDataUtil.assertTableRows(table, tableRows, branch); assertSnapshotSize(1); - assertMaxCommittedCheckpointId(jobId, operatorId, Long.MAX_VALUE); + assertMaxCommittedCheckpointId( + jobId, operatorId, IcebergStreamWriter.END_INPUT_CHECKPOINT_ID); assertThat(SimpleDataUtil.latestSnapshot(table, branch).summary()) .containsEntry("flink.test", TestIcebergFilesCommitter.class.getName()); } @@ -724,7 +740,8 @@ public void testFlinkManifests() throws Exception { JobID jobId = new JobID(); OperatorID operatorId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -734,7 +751,7 @@ public void testFlinkManifests() throws Exception { RowData row1 = SimpleDataUtil.createRowData(1, "hello"); DataFile dataFile1 = writeDataFile("data-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -774,7 +791,8 @@ public void testDeleteFiles() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -783,7 +801,7 @@ public void testDeleteFiles() throws Exception { RowData row1 = SimpleDataUtil.createInsert(1, "aaa"); DataFile dataFile1 = writeDataFile("data-file-1", ImmutableList.of(row1)); - harness.processElement(of(dataFile1), ++timestamp); + harness.processElement(of(checkpoint, dataFile1), ++timestamp); assertMaxCommittedCheckpointId(jobId, operatorId, -1L); // 1. snapshotState for checkpoint#1 @@ -815,13 +833,15 @@ public void testDeleteFiles() throws Exception { RowData delete1 = SimpleDataUtil.createDelete(1, "aaa"); DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete1)); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile1).build()), ++timestamp); - assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); // 5. snapshotState for checkpoint#2 - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); assertFlinkManifests(2); // 6. notifyCheckpointComplete for checkpoint#2 @@ -845,7 +865,8 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { OperatorID operatorId; FileAppenderFactory appenderFactory = createDeletableAppenderFactory(); - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -859,7 +880,9 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile1 = writeEqDeleteFile(appenderFactory, "delete-file-1", ImmutableList.of(delete3)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build(), + new FlinkWriteResult( + checkpoint, + WriteResult.builder().addDataFiles(dataFile1).addDeleteFiles(deleteFile1).build()), ++timestamp); // The 1th snapshotState. @@ -871,11 +894,13 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { DeleteFile deleteFile2 = writeEqDeleteFile(appenderFactory, "delete-file-2", ImmutableList.of(delete2)); harness.processElement( - WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build(), + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile2).addDeleteFiles(deleteFile2).build()), ++timestamp); // The 2nd snapshotState. - harness.snapshot(++checkpoint, ++timestamp); + harness.snapshot(checkpoint, ++timestamp); // Notify the 2nd snapshot to complete. harness.notifyOfCompletedCheckpoint(checkpoint); @@ -886,6 +911,79 @@ public void testCommitTwoCheckpointsInSingleTxn() throws Exception { } } + /** + * The testcase is to simulate upserting to an Iceberg V2 table, and facing the following + * scenario: + * + *

      + *
    • A specific row is updated + *
    • The prepareSnapshotPreBarrier triggered + *
    • Checkpoint failed for reasons outside of the Iceberg connector + *
    • The specific row is updated again in the second checkpoint as well + *
    • Second snapshot is triggered, and finished + *
    + * + *

    Previously the files from the 2 snapshots were committed in a single Iceberg commit, as a + * results duplicate rows were created in the table. + * + * @throws Exception Exception + */ + @TestTemplate + public void testCommitMultipleCheckpointsForV2Table() throws Exception { + assumeThat(formatVersion) + .as("Only support equality-delete in format v2 or later.") + .isGreaterThan(1); + + long timestamp = 0; + long checkpoint = 10; + + JobID jobId = new JobID(); + OperatorID operatorId; + + FileAppenderFactory appenderFactory = + new FlinkAppenderFactory( + table, + table.schema(), + FlinkSchemaUtil.convert(table.schema()), + table.properties(), + table.spec(), + new int[] {table.schema().findField("id").fieldId()}, + table.schema(), + null); + + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { + harness.setup(); + harness.open(); + operatorId = harness.getOperator().getOperatorID(); + + assertMaxCommittedCheckpointId(jobId, operatorId, -1L); + + RowData insert1 = null; + RowData insert2 = null; + for (int i = 1; i <= 3; i++) { + insert1 = SimpleDataUtil.createInsert(1, "aaa" + i); + insert2 = SimpleDataUtil.createInsert(2, "bbb" + i); + DataFile dataFile = writeDataFile("data-file-" + i, ImmutableList.of(insert1, insert2)); + DeleteFile deleteFile = + writeEqDeleteFile( + appenderFactory, "delete-file-" + i, ImmutableList.of(insert1, insert2)); + harness.processElement( + new FlinkWriteResult( + ++checkpoint, + WriteResult.builder().addDataFiles(dataFile).addDeleteFiles(deleteFile).build()), + ++timestamp); + } + + harness.snapshot(checkpoint, ++timestamp); + harness.notifyOfCompletedCheckpoint(checkpoint); + SimpleDataUtil.assertTableRows(table, ImmutableList.of(insert1, insert2), branch); + assertMaxCommittedCheckpointId(jobId, operatorId, checkpoint); + assertFlinkManifests(0); + assertThat(table.snapshots()).hasSize(3); + } + } + @TestTemplate public void testSpecEvolution() throws Exception { long timestamp = 0; @@ -898,7 +996,8 @@ public void testSpecEvolution() throws Exception { DataFile dataFile; int specId; - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.setup(); harness.open(); operatorId = harness.getOperator().getOperatorID(); @@ -909,7 +1008,7 @@ public void testSpecEvolution() throws Exception { RowData rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // table unpartitioned dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData)); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); harness.snapshot(checkpointId, ++timestamp); @@ -928,7 +1027,7 @@ public void testSpecEvolution() throws Exception { rowData = SimpleDataUtil.createRowData(checkpointId, "hello" + checkpointId); // write data with old partition spec dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(rowData), oldSpec, null); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(rowData); snapshot = harness.snapshot(checkpointId, ++timestamp); @@ -946,7 +1045,8 @@ public void testSpecEvolution() throws Exception { } // Restore from the given snapshot - try (OneInputStreamOperatorTestHarness harness = createStreamSink(jobId)) { + try (OneInputStreamOperatorTestHarness harness = + createStreamSink(jobId)) { harness.getStreamConfig().setOperatorID(operatorId); harness.setup(); harness.initializeState(snapshot); @@ -962,7 +1062,7 @@ public void testSpecEvolution() throws Exception { partition.set(0, checkpointId); dataFile = writeDataFile("data-" + checkpointId, ImmutableList.of(row), table.spec(), partition); - harness.processElement(of(dataFile), ++timestamp); + harness.processElement(of(checkpointId, dataFile), ++timestamp); rows.add(row); harness.snapshot(checkpointId, ++timestamp); assertFlinkManifests(1); @@ -1088,7 +1188,7 @@ private void assertSnapshotSize(int expectedSnapshotSize) { assertThat(table.snapshots()).hasSize(expectedSnapshotSize); } - private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) + private OneInputStreamOperatorTestHarness createStreamSink(JobID jobID) throws Exception { TestOperatorFactory factory = TestOperatorFactory.of(table.location(), branch, table.spec()); return new OneInputStreamOperatorTestHarness<>(factory, createEnvironment(jobID)); @@ -1108,7 +1208,7 @@ private static MockEnvironment createEnvironment(JobID jobID) { } private static class TestOperatorFactory extends AbstractStreamOperatorFactory - implements OneInputStreamOperatorFactory { + implements OneInputStreamOperatorFactory { private final String tablePath; private final String branch; private final PartitionSpec spec; diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java index 50283f7ad215..e13721a9f170 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergStreamWriter.java @@ -28,6 +28,7 @@ import java.util.Locale; import java.util.Map; import java.util.Set; +import java.util.stream.Collectors; import org.apache.flink.streaming.api.operators.BoundedOneInput; import org.apache.flink.streaming.util.OneInputStreamOperatorTestHarness; import org.apache.flink.table.api.DataTypes; @@ -102,7 +103,7 @@ public void before() throws IOException { @TestTemplate public void testWritingTable() throws Exception { long checkpointId = 1L; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { // The first checkpoint testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); @@ -111,7 +112,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -123,7 +125,8 @@ public void testWritingTable() throws Exception { testHarness.prepareSnapshotPreBarrier(checkpointId); expectedDataFiles = partitioned ? 4 : 2; - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -148,14 +151,15 @@ public void testWritingTable() throws Exception { public void testSnapshotTwice() throws Exception { long checkpointId = 1; long timestamp = 1; - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), timestamp++); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), timestamp); testHarness.prepareSnapshotPreBarrier(checkpointId++); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -163,7 +167,10 @@ public void testSnapshotTwice() throws Exception { for (int i = 0; i < 5; i++) { testHarness.prepareSnapshotPreBarrier(checkpointId++); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder() + .addAll(getWriteResults(testHarness.extractOutputValues())) + .build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); } @@ -172,14 +179,14 @@ public void testSnapshotTwice() throws Exception { @TestTemplate public void testTableWithoutSnapshot() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { assertThat(testHarness.extractOutputValues()).isEmpty(); } // Even if we closed the iceberg stream writer, there's no orphan data file. assertThat(scanDataFiles()).isEmpty(); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); // Still not emit the data file yet, because there is no checkpoint. @@ -212,7 +219,7 @@ private Set scanDataFiles() throws IOException { @TestTemplate public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -221,13 +228,15 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); ((BoundedOneInput) testHarness.getOneInputOperator()).endInput(); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // Datafiles should not be sent again assertThat(result.dataFiles()).hasSize(expectedDataFiles); @@ -236,7 +245,7 @@ public void testBoundedStreamCloseWithEmittingDataFiles() throws Exception { @TestTemplate public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throws Exception { - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { testHarness.processElement(SimpleDataUtil.createRowData(1, "hello"), 1); testHarness.processElement(SimpleDataUtil.createRowData(2, "world"), 2); @@ -244,13 +253,15 @@ public void testBoundedStreamTriggeredEndInputBeforeTriggeringCheckpoint() throw testHarness.endInput(); int expectedDataFiles = partitioned ? 2 : 1; - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(expectedDataFiles); testHarness.prepareSnapshotPreBarrier(1L); - result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); // It should be ensured that after endInput is triggered, when prepareSnapshotPreBarrier // is triggered, write should only send WriteResult once @@ -275,7 +286,7 @@ public void testTableWithTargetFileSize() throws Exception { } } - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter()) { for (RowData row : rows) { testHarness.processElement(row, 1); @@ -283,7 +294,8 @@ public void testTableWithTargetFileSize() throws Exception { // snapshot the operator. testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(8); @@ -346,13 +358,14 @@ public void testPromotedFlinkDataType() throws Exception { record.copy(ImmutableMap.of("tinyint", 2, "smallint", 0, "int", 102)), record.copy(ImmutableMap.of("tinyint", 3, "smallint", 32767, "int", 103))); - try (OneInputStreamOperatorTestHarness testHarness = + try (OneInputStreamOperatorTestHarness testHarness = createIcebergStreamWriter(icebergTable, flinkSchema)) { for (RowData row : rows) { testHarness.processElement(row, 1); } testHarness.prepareSnapshotPreBarrier(1); - WriteResult result = WriteResult.builder().addAll(testHarness.extractOutputValues()).build(); + WriteResult result = + WriteResult.builder().addAll(getWriteResults(testHarness.extractOutputValues())).build(); assertThat(result.deleteFiles()).isEmpty(); assertThat(result.dataFiles()).hasSize(partitioned ? 3 : 1); @@ -365,12 +378,18 @@ public void testPromotedFlinkDataType() throws Exception { SimpleDataUtil.assertTableRecords(location, expected); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter() + private static List getWriteResults(List flinkWriteResults) { + return flinkWriteResults.stream() + .map(FlinkWriteResult::writeResult) + .collect(Collectors.toList()); + } + + private OneInputStreamOperatorTestHarness createIcebergStreamWriter() throws Exception { return createIcebergStreamWriter(table, SimpleDataUtil.FLINK_SCHEMA); } - private OneInputStreamOperatorTestHarness createIcebergStreamWriter( + private OneInputStreamOperatorTestHarness createIcebergStreamWriter( Table icebergTable, TableSchema flinkSchema) throws Exception { RowType flinkRowType = FlinkSink.toFlinkRowType(icebergTable.schema(), flinkSchema); FlinkWriteConf flinkWriteConfig = @@ -379,7 +398,7 @@ private OneInputStreamOperatorTestHarness createIcebergStr IcebergStreamWriter streamWriter = FlinkSink.createStreamWriter(() -> icebergTable, flinkWriteConfig, flinkRowType, null); - OneInputStreamOperatorTestHarness harness = + OneInputStreamOperatorTestHarness harness = new OneInputStreamOperatorTestHarness<>(streamWriter, 1, 1, 0); harness.setup(); From 950e5c202ea4c0cdb0f24788a46e96db115ddc7d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 1 Sep 2024 06:39:10 +0200 Subject: [PATCH 0657/1019] Build: Bump mkdocs-material from 9.5.33 to 9.5.34 (#11062) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.33 to 9.5.34. - [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.33...9.5.34) --- 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 b07ec2b4d975..377fdc43fef2 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.0.5 -mkdocs-material==9.5.33 +mkdocs-material==9.5.34 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 ac6a7f4e2a658e8f228c89e7884476738e80ded6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sun, 1 Sep 2024 07:22:59 +0200 Subject: [PATCH 0658/1019] Build: Bump com.google.errorprone:error_prone_annotations (#11055) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.30.0 to 2.31.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.30.0...v2.31.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 ba7011312fa0..8b1feda17109 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -37,7 +37,7 @@ datasketches = "6.0.0" delta-standalone = "3.2.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.30.0" +errorprone-annotations = "2.31.0" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} From 69a1c072e96ad6bf663bf242a55bf9ecdd9edf8f Mon Sep 17 00:00:00 2001 From: Jacob Marble Date: Tue, 3 Sep 2024 08:44:43 -0700 Subject: [PATCH 0659/1019] API: implement types timestamp_ns and timestamptz_ns (#9008) --- .../main/java/org/apache/iceberg/Schema.java | 25 ++ .../expressions/BoundLiteralPredicate.java | 1 + .../iceberg/expressions/ExpressionUtil.java | 19 + .../apache/iceberg/expressions/Literals.java | 57 ++- .../org/apache/iceberg/transforms/Bucket.java | 18 + .../org/apache/iceberg/transforms/Dates.java | 14 +- .../org/apache/iceberg/transforms/Days.java | 34 +- .../org/apache/iceberg/transforms/Hours.java | 33 +- .../org/apache/iceberg/transforms/Months.java | 34 +- .../transforms/PartitionSpecVisitor.java | 15 +- .../iceberg/transforms/SortOrderVisitor.java | 13 +- .../iceberg/transforms/TimeTransform.java | 41 +- .../apache/iceberg/transforms/Timestamps.java | 182 ++++++-- .../apache/iceberg/transforms/Transform.java | 6 + .../iceberg/transforms/TransformUtil.java | 19 +- .../apache/iceberg/transforms/Transforms.java | 64 +-- .../org/apache/iceberg/transforms/Years.java | 34 +- .../org/apache/iceberg/types/Comparators.java | 2 + .../org/apache/iceberg/types/Conversions.java | 2 + .../java/org/apache/iceberg/types/Type.java | 1 + .../org/apache/iceberg/types/TypeUtil.java | 1 + .../java/org/apache/iceberg/types/Types.java | 55 +++ .../org/apache/iceberg/util/DateTimeUtil.java | 87 +++- .../apache/iceberg/PartitionSpecTestBase.java | 9 +- .../org/apache/iceberg/TestAccessors.java | 2 + .../apache/iceberg/TestPartitionPaths.java | 38 ++ .../expressions/TestExpressionUtil.java | 71 +++- .../expressions/TestLiteralSerialization.java | 6 +- .../TestMiscLiteralConversions.java | 67 ++- .../TestStringLiteralConversions.java | 64 ++- .../TestTimestampLiteralConversions.java | 245 +++++++++++ .../iceberg/transforms/TestBucketing.java | 56 +++ .../apache/iceberg/transforms/TestDates.java | 69 +++ .../iceberg/transforms/TestIdentity.java | 2 +- .../transforms/TestTimeTransforms.java | 124 ++++++ .../iceberg/transforms/TestTimestamps.java | 398 +++++++++++++++++- .../apache/iceberg/types/TestComparators.java | 6 + .../apache/iceberg/types/TestConversions.java | 13 +- .../iceberg/types/TestReadabilityChecks.java | 2 + .../iceberg/types/TestSerializableTypes.java | 2 + .../org/apache/iceberg/types/TestTypes.java | 5 + .../apache/iceberg/util/TestDateTimeUtil.java | 71 +++- .../org/apache/iceberg/TableMetadata.java | 2 + .../org/apache/iceberg/TestTableMetadata.java | 51 +++ format/spec.md | 15 +- 45 files changed, 1807 insertions(+), 268 deletions(-) create mode 100644 api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java create mode 100644 api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java diff --git a/api/src/main/java/org/apache/iceberg/Schema.java b/api/src/main/java/org/apache/iceberg/Schema.java index 7ff712b62790..9bcf691f5a03 100644 --- a/api/src/main/java/org/apache/iceberg/Schema.java +++ b/api/src/main/java/org/apache/iceberg/Schema.java @@ -54,6 +54,8 @@ 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 Map MIN_FORMAT_VERSIONS = + ImmutableMap.of(Type.TypeID.TIMESTAMP_NANO, 3); private final StructType struct; private final int schemaId; @@ -573,4 +575,27 @@ private List reassignIds(List columns, TypeUtil.GetID }); return res.asStructType().fields(); } + + /** + * Check the compatibility of the schema with a format version. + * + *

    This validates that the schema does not contain types that were released in later format + * versions. + * + * @param schema a Schema + * @param formatVersion table format version + */ + public static void checkCompatibility(Schema schema, int formatVersion) { + // check the type in each field + 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); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java b/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java index 02dc31c6a6c5..127d46e6a48f 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundLiteralPredicate.java @@ -31,6 +31,7 @@ public class BoundLiteralPredicate extends BoundPredicate { Type.TypeID.LONG, Type.TypeID.DATE, Type.TypeID.TIME, + Type.TypeID.TIMESTAMP_NANO, Type.TypeID.TIMESTAMP); private static long toLong(Literal lit) { 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 bf72e03bc406..bd0adb228bd3 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -36,6 +36,7 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; /** Expression utility methods. */ public class ExpressionUtil { @@ -43,6 +44,7 @@ public class ExpressionUtil { Transforms.bucket(Integer.MAX_VALUE).bind(Types.StringType.get()); private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); private static final long FIVE_MINUTES_IN_MICROS = TimeUnit.MINUTES.toMicros(5); + private static final long FIVE_MINUTES_IN_NANOS = TimeUnit.MINUTES.toNanos(5); private static final long THREE_DAYS_IN_HOURS = TimeUnit.DAYS.toHours(3); private static final long NINETY_DAYS_IN_HOURS = TimeUnit.DAYS.toHours(90); private static final Pattern DATE = Pattern.compile("\\d{4}-\\d{2}-\\d{2}"); @@ -52,6 +54,12 @@ public class ExpressionUtil { private static final Pattern TIMESTAMPTZ = Pattern.compile( "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{1,9})?)?([-+]\\d{2}:\\d{2}|Z)"); + private static final Pattern TIMESTAMPNS = + Pattern.compile("\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?"); + private static final Pattern TIMESTAMPTZNS = + Pattern.compile( + "\\d{4}-\\d{2}-\\d{2}T\\d{2}:\\d{2}(:\\d{2}(.\\d{7,9})?)?([-+]\\d{2}:\\d{2}|Z)"); + static final int LONG_IN_PREDICATE_ABBREVIATION_THRESHOLD = 10; private static final int LONG_IN_PREDICATE_ABBREVIATION_MIN_GAIN = 5; @@ -515,6 +523,8 @@ private static String sanitize(Type type, Object value, long now, int today) { return "(time)"; case TIMESTAMP: return sanitizeTimestamp((long) value, now); + case TIMESTAMP_NANO: + return sanitizeTimestamp(DateTimeUtil.nanosToMicros((long) value / 1000), now); case STRING: return sanitizeString((CharSequence) value, now, today); case BOOLEAN: @@ -536,6 +546,9 @@ private static String sanitize(Literal literal, long now, int today) { return sanitizeDate(((Literals.DateLiteral) literal).value(), today); } else if (literal instanceof Literals.TimestampLiteral) { return sanitizeTimestamp(((Literals.TimestampLiteral) literal).value(), now); + } else if (literal instanceof Literals.TimestampNanoLiteral) { + return sanitizeTimestamp( + DateTimeUtil.nanosToMicros(((Literals.TimestampNanoLiteral) literal).value()), now); } else if (literal instanceof Literals.TimeLiteral) { return "(time)"; } else if (literal instanceof Literals.IntegerLiteral) { @@ -594,6 +607,12 @@ private static String sanitizeString(CharSequence value, long now, int today) { if (DATE.matcher(value).matches()) { Literal date = Literal.of(value).to(Types.DateType.get()); return sanitizeDate(date.value(), today); + } else if (TIMESTAMPNS.matcher(value).matches()) { + Literal ts = Literal.of(value).to(Types.TimestampNanoType.withoutZone()); + return sanitizeTimestamp(DateTimeUtil.nanosToMicros(ts.value()), now); + } else if (TIMESTAMPTZNS.matcher(value).matches()) { + Literal ts = Literal.of(value).to(Types.TimestampNanoType.withZone()); + return sanitizeTimestamp(DateTimeUtil.nanosToMicros(ts.value()), now); } else if (TIMESTAMP.matcher(value).matches()) { Literal ts = Literal.of(value).to(Types.TimestampType.withoutZone()); return sanitizeTimestamp(ts.value(), now); diff --git a/api/src/main/java/org/apache/iceberg/expressions/Literals.java b/api/src/main/java/org/apache/iceberg/expressions/Literals.java index 79d7190c49df..ee47035b1e72 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Literals.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Literals.java @@ -24,7 +24,6 @@ import java.nio.ByteBuffer; import java.time.Instant; import java.time.LocalDate; -import java.time.LocalDateTime; import java.time.LocalTime; import java.time.OffsetDateTime; import java.time.ZoneOffset; @@ -40,6 +39,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.NaNUtil; class Literals { @@ -299,6 +299,9 @@ public Literal to(Type type) { return (Literal) new TimeLiteral(value()); case TIMESTAMP: return (Literal) new TimestampLiteral(value()); + case TIMESTAMP_NANO: + // assume micros and convert to nanos to match the behavior in the timestamp case above + return new TimestampLiteral(value()).to(type); case DATE: if ((long) Integer.MAX_VALUE < value()) { return aboveMax(); @@ -437,11 +440,9 @@ public Literal to(Type type) { case TIMESTAMP: return (Literal) this; case DATE: - return (Literal) - new DateLiteral( - (int) - ChronoUnit.DAYS.between( - EPOCH_DAY, EPOCH.plus(value(), ChronoUnit.MICROS).toLocalDate())); + return (Literal) new DateLiteral(DateTimeUtil.microsToDays(value())); + case TIMESTAMP_NANO: + return (Literal) new TimestampNanoLiteral(DateTimeUtil.microsToNanos(value())); default: } return null; @@ -453,6 +454,32 @@ protected Type.TypeID typeId() { } } + static class TimestampNanoLiteral extends ComparableLiteral { + TimestampNanoLiteral(Long value) { + super(value); + } + + @Override + @SuppressWarnings("unchecked") + public Literal to(Type type) { + switch (type.typeId()) { + case DATE: + return (Literal) new DateLiteral(DateTimeUtil.nanosToDays(value())); + case TIMESTAMP: + return (Literal) new TimestampLiteral(DateTimeUtil.nanosToMicros(value())); + case TIMESTAMP_NANO: + return (Literal) this; + default: + } + return null; + } + + @Override + protected Type.TypeID typeId() { + return Type.TypeID.TIMESTAMP_NANO; + } + } + static class DecimalLiteral extends ComparableLiteral { DecimalLiteral(BigDecimal value) { super(value); @@ -502,19 +529,21 @@ public Literal to(Type type) { case TIMESTAMP: if (((Types.TimestampType) type).shouldAdjustToUTC()) { - long timestampMicros = - ChronoUnit.MICROS.between( - EPOCH, OffsetDateTime.parse(value(), DateTimeFormatter.ISO_DATE_TIME)); + long timestampMicros = DateTimeUtil.isoTimestamptzToMicros(value().toString()); return (Literal) new TimestampLiteral(timestampMicros); } else { - long timestampMicros = - ChronoUnit.MICROS.between( - EPOCH, - LocalDateTime.parse(value(), DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .atOffset(ZoneOffset.UTC)); + long timestampMicros = DateTimeUtil.isoTimestampToMicros(value().toString()); return (Literal) new TimestampLiteral(timestampMicros); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return (Literal) + new TimestampNanoLiteral(DateTimeUtil.isoTimestamptzToNanos(value())); + } else { + return (Literal) new TimestampNanoLiteral(DateTimeUtil.isoTimestampToNanos(value())); + } + case STRING: return (Literal) this; 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 912bcd271725..0e4e782cc110 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Bucket.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Bucket.java @@ -33,6 +33,7 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.BucketUtil; +import org.apache.iceberg.util.DateTimeUtil; import org.apache.iceberg.util.SerializableFunction; class Bucket implements Transform, Serializable { @@ -63,6 +64,8 @@ static & SerializableFunction> B get( case FIXED: case BINARY: return (B) new BucketByteBuffer(numBuckets); + case TIMESTAMP_NANO: + return (B) new BucketTimestampNano(numBuckets); case UUID: return (B) new BucketUUID(numBuckets); default: @@ -107,6 +110,7 @@ public boolean canTransform(Type type) { case DATE: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: case STRING: case BINARY: case FIXED: @@ -214,6 +218,20 @@ protected int hash(Long value) { } } + // In order to bucket TimestampNano the same as Timestamp, convert to micros before hashing. + private static class BucketTimestampNano extends Bucket + implements SerializableFunction { + + private BucketTimestampNano(int numBuckets) { + super(numBuckets); + } + + @Override + protected int hash(Long nanos) { + return BucketUtil.hash(DateTimeUtil.nanosToMicros(nanos)); + } + } + private static class BucketString extends Bucket implements SerializableFunction { 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 3d26b542be7b..88db16797867 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Dates.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Dates.java @@ -97,6 +97,10 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } + ChronoUnit granularity() { + return granularity; + } + @Override public boolean preservesOrder() { return true; @@ -109,11 +113,11 @@ public boolean satisfiesOrderOf(Transform other) { } if (other instanceof Dates) { - // test the granularity, in days. day(ts) => 1 day, months(ts) => 30 days, and day satisfies - // the order of months - Dates otherTransform = (Dates) other; - return granularity.getDuration().toDays() - <= otherTransform.granularity.getDuration().toDays(); + return TransformUtil.satisfiesOrderOf(granularity, ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity, ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf(granularity, ((TimeTransform) other).granularity()); } return false; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Days.java b/api/src/main/java/org/apache/iceberg/transforms/Days.java index f69d5d6110ed..e2b829b86662 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Days.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Days.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static Days get() { } @Override - @SuppressWarnings("unchecked") - protected Transform toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.DAY; - case TIMESTAMP: - return (Transform) Timestamps.DAY; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.DAYS; } @Override - public Type getResultType(Type sourceType) { - return Types.DateType.get(); + protected Transform toEnum(Type type) { + return (Transform) + fromSourceType(type, Dates.DAY, Timestamps.MICROS_TO_DAY, Timestamps.NANOS_TO_DAY); } @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.DAY.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.DAY.satisfiesOrderOf(other); - } else if (other instanceof Days || other instanceof Months || other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.DateType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/Hours.java b/api/src/main/java/org/apache/iceberg/transforms/Hours.java index afc14516f3cd..2ff79f6a66a7 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Hours.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Hours.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -30,19 +31,21 @@ static Hours get() { return (Hours) INSTANCE; } + @Override + protected ChronoUnit granularity() { + return ChronoUnit.HOURS; + } + @Override @SuppressWarnings("unchecked") protected Transform toEnum(Type type) { - if (type.typeId() == Type.TypeID.TIMESTAMP) { - return (Transform) Timestamps.HOUR; - } - - throw new IllegalArgumentException("Unsupported type: " + type); + return (Transform) + fromSourceType(type, null, Timestamps.MICROS_TO_HOUR, Timestamps.NANOS_TO_HOUR); } @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.TIMESTAMP || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override @@ -50,24 +53,6 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } - @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return other == Timestamps.HOUR; - } else if (other instanceof Hours - || other instanceof Days - || other instanceof Months - || other instanceof Years) { - return true; - } - - return false; - } - @Override public String toHumanString(Type alwaysInt, Integer value) { return value != null ? TransformUtil.humanHour(value) : "null"; diff --git a/api/src/main/java/org/apache/iceberg/transforms/Months.java b/api/src/main/java/org/apache/iceberg/transforms/Months.java index 8fa4d42385f7..73ec50e5dd9a 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Months.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Months.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static Months get() { } @Override - @SuppressWarnings("unchecked") - protected Transform toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.MONTH; - case TIMESTAMP: - return (Transform) Timestamps.MONTH; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.MONTHS; } @Override - public Type getResultType(Type sourceType) { - return Types.IntegerType.get(); + protected Transform toEnum(Type type) { + return (Transform) + fromSourceType(type, Dates.MONTH, Timestamps.MICROS_TO_MONTH, Timestamps.NANOS_TO_MONTH); } @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.MONTH.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.MONTH.satisfiesOrderOf(other); - } else if (other instanceof Months || other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.IntegerType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java index e4796478bf28..0d80ef88a296 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/PartitionSpecVisitor.java @@ -122,16 +122,23 @@ static R visit(Schema schema, PartitionField field, PartitionSpecVisitor int width = ((Truncate) transform).width(); return visitor.truncate(field.fieldId(), sourceName, field.sourceId(), width); } else if (transform == Dates.YEAR - || transform == Timestamps.YEAR + || transform == Timestamps.MICROS_TO_YEAR + || transform == Timestamps.NANOS_TO_YEAR || transform instanceof Years) { return visitor.year(field.fieldId(), sourceName, field.sourceId()); } else if (transform == Dates.MONTH - || transform == Timestamps.MONTH + || transform == Timestamps.MICROS_TO_MONTH + || transform == Timestamps.NANOS_TO_MONTH || transform instanceof Months) { return visitor.month(field.fieldId(), sourceName, field.sourceId()); - } else if (transform == Dates.DAY || transform == Timestamps.DAY || transform instanceof Days) { + } else if (transform == Dates.DAY + || transform == Timestamps.MICROS_TO_DAY + || transform == Timestamps.NANOS_TO_DAY + || transform instanceof Days) { return visitor.day(field.fieldId(), sourceName, field.sourceId()); - } else if (transform == Timestamps.HOUR || transform instanceof Hours) { + } else if (transform == Timestamps.MICROS_TO_HOUR + || transform == Timestamps.NANOS_TO_HOUR + || transform instanceof Hours) { return visitor.hour(field.fieldId(), sourceName, field.sourceId()); } else if (transform instanceof VoidTransform) { return visitor.alwaysNull(field.fieldId(), sourceName, field.sourceId()); diff --git a/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java b/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java index 680e095270fb..62cc9d3cdb33 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java +++ b/api/src/main/java/org/apache/iceberg/transforms/SortOrderVisitor.java @@ -85,21 +85,26 @@ static List visit(SortOrder sortOrder, SortOrderVisitor visitor) { visitor.truncate( sourceName, field.sourceId(), width, field.direction(), field.nullOrder())); } else if (transform == Dates.YEAR - || transform == Timestamps.YEAR + || transform == Timestamps.MICROS_TO_YEAR + || transform == Timestamps.NANOS_TO_YEAR || transform instanceof Years) { results.add( visitor.year(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform == Dates.MONTH - || transform == Timestamps.MONTH + || transform == Timestamps.MICROS_TO_MONTH + || transform == Timestamps.NANOS_TO_MONTH || transform instanceof Months) { results.add( visitor.month(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform == Dates.DAY - || transform == Timestamps.DAY + || transform == Timestamps.MICROS_TO_DAY + || transform == Timestamps.NANOS_TO_DAY || transform instanceof Days) { results.add( visitor.day(sourceName, field.sourceId(), field.direction(), field.nullOrder())); - } else if (transform == Timestamps.HOUR || transform instanceof Hours) { + } else if (transform == Timestamps.MICROS_TO_HOUR + || transform == Timestamps.NANOS_TO_HOUR + || transform instanceof Hours) { results.add( visitor.hour(sourceName, field.sourceId(), field.direction(), field.nullOrder())); } else if (transform instanceof UnknownTransform) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java b/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java index 01ea8130aa60..c348fda52b02 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TimeTransform.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.transforms; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.expressions.BoundPredicate; import org.apache.iceberg.expressions.BoundTransform; import org.apache.iceberg.expressions.UnboundPredicate; @@ -25,6 +26,24 @@ import org.apache.iceberg.util.SerializableFunction; abstract class TimeTransform implements Transform { + protected static R fromSourceType(Type type, R dateResult, R microsResult, R nanosResult) { + switch (type.typeId()) { + case DATE: + if (dateResult != null) { + return dateResult; + } + break; + case TIMESTAMP: + return microsResult; + case TIMESTAMP_NANO: + return nanosResult; + } + + throw new IllegalArgumentException("Unsupported type: " + type); + } + + protected abstract ChronoUnit granularity(); + protected abstract Transform toEnum(Type type); @Override @@ -37,9 +56,29 @@ public boolean preservesOrder() { return true; } + @Override + public boolean satisfiesOrderOf(Transform other) { + if (this == other) { + return true; + } + + if (other instanceof Dates) { + return TransformUtil.satisfiesOrderOf(granularity(), ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity(), ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf( + granularity(), ((TimeTransform) other).granularity()); + } + + return false; + } + @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.DATE || type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.DATE + || type.typeId() == Type.TypeID.TIMESTAMP + || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override 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 b5b50e9d42b2..8b8c2ca0a96b 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Timestamps.java @@ -32,53 +32,29 @@ import org.apache.iceberg.util.SerializableFunction; enum Timestamps implements Transform { - YEAR(ChronoUnit.YEARS, "year"), - MONTH(ChronoUnit.MONTHS, "month"), - DAY(ChronoUnit.DAYS, "day"), - HOUR(ChronoUnit.HOURS, "hour"); + MICROS_TO_YEAR(ChronoUnit.YEARS, "year", MicrosToYears.INSTANCE), + MICROS_TO_MONTH(ChronoUnit.MONTHS, "month", MicrosToMonths.INSTANCE), + MICROS_TO_DAY(ChronoUnit.DAYS, "day", MicrosToDays.INSTANCE), + MICROS_TO_HOUR(ChronoUnit.HOURS, "hour", MicrosToHours.INSTANCE), - @Immutable - static class Apply implements SerializableFunction { - private final ChronoUnit granularity; - - Apply(ChronoUnit granularity) { - this.granularity = granularity; - } - - @Override - public Integer apply(Long timestampMicros) { - if (timestampMicros == null) { - return null; - } - - switch (granularity) { - case YEARS: - return DateTimeUtil.microsToYears(timestampMicros); - case MONTHS: - return DateTimeUtil.microsToMonths(timestampMicros); - case DAYS: - return DateTimeUtil.microsToDays(timestampMicros); - case HOURS: - return DateTimeUtil.microsToHours(timestampMicros); - default: - throw new UnsupportedOperationException("Unsupported time unit: " + granularity); - } - } - } + NANOS_TO_YEAR(ChronoUnit.YEARS, "year", NanosToYears.INSTANCE), + NANOS_TO_MONTH(ChronoUnit.MONTHS, "month", NanosToMonths.INSTANCE), + NANOS_TO_DAY(ChronoUnit.DAYS, "day", NanosToDays.INSTANCE), + NANOS_TO_HOUR(ChronoUnit.HOURS, "hour", NanosToHours.INSTANCE); private final ChronoUnit granularity; private final String name; - private final Apply apply; + private final SerializableFunction apply; - Timestamps(ChronoUnit granularity, String name) { - this.granularity = granularity; + Timestamps(ChronoUnit granularity, String name, SerializableFunction apply) { this.name = name; - this.apply = new Apply(granularity); + this.granularity = granularity; + this.apply = apply; } @Override - public Integer apply(Long timestampMicros) { - return apply.apply(timestampMicros); + public Integer apply(Long timestamp) { + return apply.apply(timestamp); } @Override @@ -89,7 +65,7 @@ public SerializableFunction bind(Type type) { @Override public boolean canTransform(Type type) { - return type.typeId() == Type.TypeID.TIMESTAMP; + return type.typeId() == Type.TypeID.TIMESTAMP || type.typeId() == Type.TypeID.TIMESTAMP_NANO; } @Override @@ -100,6 +76,10 @@ public Type getResultType(Type sourceType) { return Types.IntegerType.get(); } + ChronoUnit granularity() { + return granularity; + } + @Override public boolean preservesOrder() { return true; @@ -111,12 +91,12 @@ public boolean satisfiesOrderOf(Transform other) { return true; } - if (other instanceof Timestamps) { - // test the granularity, in hours. hour(ts) => 1 hour, day(ts) => 24 hours, and hour satisfies - // the order of day - Timestamps otherTransform = (Timestamps) other; - return granularity.getDuration().toHours() - <= otherTransform.granularity.getDuration().toHours(); + if (other instanceof Dates) { + return TransformUtil.satisfiesOrderOf(granularity, ((Dates) other).granularity()); + } else if (other instanceof Timestamps) { + return TransformUtil.satisfiesOrderOf(granularity, ((Timestamps) other).granularity()); + } else if (other instanceof TimeTransform) { + return TransformUtil.satisfiesOrderOf(granularity, ((TimeTransform) other).granularity()); } return false; @@ -197,4 +177,116 @@ public String toString() { public String dedupName() { return "time"; } + + @Immutable + static class MicrosToYears implements SerializableFunction { + static final MicrosToYears INSTANCE = new MicrosToYears(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToYears(micros); + } + } + + @Immutable + static class MicrosToMonths implements SerializableFunction { + static final MicrosToMonths INSTANCE = new MicrosToMonths(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToMonths(micros); + } + } + + @Immutable + static class MicrosToDays implements SerializableFunction { + static final MicrosToDays INSTANCE = new MicrosToDays(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToDays(micros); + } + } + + @Immutable + static class MicrosToHours implements SerializableFunction { + static final MicrosToHours INSTANCE = new MicrosToHours(); + + @Override + public Integer apply(Long micros) { + if (micros == null) { + return null; + } + + return DateTimeUtil.microsToHours(micros); + } + } + + @Immutable + static class NanosToYears implements SerializableFunction { + static final NanosToYears INSTANCE = new NanosToYears(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToYears(nanos); + } + } + + @Immutable + static class NanosToMonths implements SerializableFunction { + static final NanosToMonths INSTANCE = new NanosToMonths(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToMonths(nanos); + } + } + + @Immutable + static class NanosToDays implements SerializableFunction { + static final NanosToDays INSTANCE = new NanosToDays(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToDays(nanos); + } + } + + @Immutable + static class NanosToHours implements SerializableFunction { + static final NanosToHours INSTANCE = new NanosToHours(); + + @Override + public Integer apply(Long nanos) { + if (nanos == null) { + return null; + } + + return DateTimeUtil.nanosToHours(nanos); + } + } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transform.java b/api/src/main/java/org/apache/iceberg/transforms/Transform.java index 5a56b672b1b1..78312b58b12f 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transform.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transform.java @@ -181,6 +181,12 @@ default String toHumanString(Type type, T value) { } else { return TransformUtil.humanTimestampWithoutZone((Long) value); } + case TIMESTAMP_NANO: + if (((Types.TimestampNanoType) type).shouldAdjustToUTC()) { + return TransformUtil.humanTimestampNanoWithZone((Long) value); + } else { + return TransformUtil.humanTimestampNanoWithoutZone((Long) value); + } case FIXED: case BINARY: if (value instanceof ByteBuffer) { diff --git a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java index 53bc23a49888..dd7f97e950e8 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java @@ -26,6 +26,7 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.Base64; +import org.apache.iceberg.util.DateTimeUtil; class TransformUtil { @@ -55,11 +56,19 @@ static String humanTime(Long microsFromMidnight) { } static String humanTimestampWithZone(Long timestampMicros) { - return ChronoUnit.MICROS.addTo(EPOCH, timestampMicros).toString(); + return DateTimeUtil.microsToIsoTimestamptz(timestampMicros); } static String humanTimestampWithoutZone(Long timestampMicros) { - return ChronoUnit.MICROS.addTo(EPOCH, timestampMicros).toLocalDateTime().toString(); + return DateTimeUtil.microsToIsoTimestamp(timestampMicros); + } + + static String humanTimestampNanoWithZone(Long timestampNanos) { + return DateTimeUtil.nanosToIsoTimestamptz(timestampNanos); + } + + static String humanTimestampNanoWithoutZone(Long timestampNanos) { + return DateTimeUtil.nanosToIsoTimestamp(timestampNanos); } static String humanHour(int hourOrdinal) { @@ -73,4 +82,10 @@ static String base64encode(ByteBuffer buffer) { // use direct encoding because all of the encoded bytes are in ASCII return StandardCharsets.ISO_8859_1.decode(Base64.getEncoder().encode(buffer)).toString(); } + + static boolean satisfiesOrderOf(ChronoUnit leftGranularity, ChronoUnit rightGranularity) { + // test the granularity, in hours. hour(ts) => 1 hour, day(ts) => 24 hours, and hour satisfies + // the order of day + return leftGranularity.getDuration().toHours() <= rightGranularity.getDuration().toHours(); + } } diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java index a1ce33ddd6da..11282efdefb1 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java @@ -23,7 +23,6 @@ import java.util.regex.Pattern; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.Schema; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Type; /** @@ -68,6 +67,8 @@ private Transforms() {} return new UnknownTransform<>(transform); } + /** @deprecated use {@link #identity()} instead; will be removed in 2.0.0 */ + @Deprecated public static Transform fromString(Type type, String transform) { Matcher widthMatcher = HAS_WIDTH.matcher(transform); if (widthMatcher.matches()) { @@ -80,22 +81,20 @@ private Transforms() {} } } - if (transform.equalsIgnoreCase("identity")) { - return Identity.get(type); - } - - try { - if (type.typeId() == Type.TypeID.TIMESTAMP) { - return Timestamps.valueOf(transform.toUpperCase(Locale.ENGLISH)); - } else if (type.typeId() == Type.TypeID.DATE) { - return Dates.valueOf(transform.toUpperCase(Locale.ENGLISH)); - } - } catch (IllegalArgumentException ignored) { - // fall through to return unknown transform - } - - if (transform.equalsIgnoreCase("void")) { - return VoidTransform.get(); + String lowerTransform = transform.toLowerCase(Locale.ENGLISH); + switch (lowerTransform) { + case "identity": + return Identity.get(type); + case "year": + return Years.get().toEnum(type); + case "month": + return Months.get().toEnum(type); + case "day": + return Days.get().toEnum(type); + case "hour": + return Hours.get().toEnum(type); + case "void": + return VoidTransform.get(); } return new UnknownTransform<>(transform); @@ -125,14 +124,7 @@ public static Transform identity(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform year(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.YEAR; - case TIMESTAMP: - return (Transform) Timestamps.YEAR; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by year"); - } + return (Transform) Years.get().toEnum(type); } /** @@ -146,14 +138,7 @@ public static Transform year(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform month(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.MONTH; - case TIMESTAMP: - return (Transform) Timestamps.MONTH; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by month"); - } + return (Transform) Months.get().toEnum(type); } /** @@ -167,14 +152,7 @@ public static Transform month(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform day(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.DAY; - case TIMESTAMP: - return (Transform) Timestamps.DAY; - default: - throw new IllegalArgumentException("Cannot partition type " + type + " by day"); - } + return (Transform) Days.get().toEnum(type); } /** @@ -188,9 +166,7 @@ public static Transform day(Type type) { @Deprecated @SuppressWarnings("unchecked") public static Transform hour(Type type) { - Preconditions.checkArgument( - type.typeId() == Type.TypeID.TIMESTAMP, "Cannot partition type %s by hour", type); - return (Transform) Timestamps.HOUR; + return (Transform) Hours.get().toEnum(type); } /** diff --git a/api/src/main/java/org/apache/iceberg/transforms/Years.java b/api/src/main/java/org/apache/iceberg/transforms/Years.java index 6c1eee578506..2920a37dc692 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Years.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Years.java @@ -19,6 +19,7 @@ package org.apache.iceberg.transforms; import java.io.ObjectStreamException; +import java.time.temporal.ChronoUnit; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; @@ -31,38 +32,19 @@ static Years get() { } @Override - @SuppressWarnings("unchecked") - protected Transform toEnum(Type type) { - switch (type.typeId()) { - case DATE: - return (Transform) Dates.YEAR; - case TIMESTAMP: - return (Transform) Timestamps.YEAR; - default: - throw new IllegalArgumentException("Unsupported type: " + type); - } + protected ChronoUnit granularity() { + return ChronoUnit.YEARS; } @Override - public Type getResultType(Type sourceType) { - return Types.IntegerType.get(); + protected Transform toEnum(Type type) { + return (Transform) + fromSourceType(type, Dates.YEAR, Timestamps.MICROS_TO_YEAR, Timestamps.NANOS_TO_YEAR); } @Override - public boolean satisfiesOrderOf(Transform other) { - if (this == other) { - return true; - } - - if (other instanceof Timestamps) { - return Timestamps.YEAR.satisfiesOrderOf(other); - } else if (other instanceof Dates) { - return Dates.YEAR.satisfiesOrderOf(other); - } else if (other instanceof Years) { - return true; - } - - return false; + public Type getResultType(Type sourceType) { + return Types.IntegerType.get(); } @Override diff --git a/api/src/main/java/org/apache/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java index a803afac104f..bfbffc64b673 100644 --- a/api/src/main/java/org/apache/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -41,6 +41,8 @@ private Comparators() {} .put(Types.TimeType.get(), Comparator.naturalOrder()) .put(Types.TimestampType.withZone(), Comparator.naturalOrder()) .put(Types.TimestampType.withoutZone(), Comparator.naturalOrder()) + .put(Types.TimestampNanoType.withZone(), Comparator.naturalOrder()) + .put(Types.TimestampNanoType.withoutZone(), Comparator.naturalOrder()) .put(Types.StringType.get(), Comparators.charSequences()) .put(Types.UUIDType.get(), Comparator.naturalOrder()) .put(Types.BinaryType.get(), Comparators.unsignedBytes()) diff --git a/api/src/main/java/org/apache/iceberg/types/Conversions.java b/api/src/main/java/org/apache/iceberg/types/Conversions.java index 1d2539514954..e18c7b4362e6 100644 --- a/api/src/main/java/org/apache/iceberg/types/Conversions.java +++ b/api/src/main/java/org/apache/iceberg/types/Conversions.java @@ -97,6 +97,7 @@ public static ByteBuffer toByteBuffer(Type.TypeID typeId, Object value) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: return ByteBuffer.allocate(8).order(ByteOrder.LITTLE_ENDIAN).putLong(0, (long) value); case FLOAT: return ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN).putFloat(0, (float) value); @@ -146,6 +147,7 @@ private static Object internalFromByteBuffer(Type type, ByteBuffer buffer) { case LONG: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: if (tmp.remaining() < 8) { // type was later promoted to long return (long) tmp.getInt(); 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 5062b54d10e1..571bf9a14e43 100644 --- a/api/src/main/java/org/apache/iceberg/types/Type.java +++ b/api/src/main/java/org/apache/iceberg/types/Type.java @@ -37,6 +37,7 @@ enum TypeID { DATE(Integer.class), TIME(Long.class), TIMESTAMP(Long.class), + TIMESTAMP_NANO(Long.class), STRING(CharSequence.class), UUID(java.util.UUID.class), FIXED(ByteBuffer.class), 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 7298dfca4c03..9d4b217f1595 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -522,6 +522,7 @@ private static int estimateSize(Type type) { case DOUBLE: case TIME: case TIMESTAMP: + case TIMESTAMP_NANO: // longs and doubles occupy 8 bytes // times and timestamps are internally represented as longs return 8; 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 ce6caa4721df..2352b9b52f13 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -49,6 +49,8 @@ private Types() {} .put(TimeType.get().toString(), TimeType.get()) .put(TimestampType.withZone().toString(), TimestampType.withZone()) .put(TimestampType.withoutZone().toString(), TimestampType.withoutZone()) + .put(TimestampNanoType.withZone().toString(), TimestampNanoType.withZone()) + .put(TimestampNanoType.withoutZone().toString(), TimestampNanoType.withoutZone()) .put(StringType.get().toString(), StringType.get()) .put(UUIDType.get().toString(), UUIDType.get()) .put(BinaryType.get().toString(), BinaryType.get()) @@ -259,6 +261,59 @@ public int hashCode() { } } + public static class TimestampNanoType extends PrimitiveType { + private static final TimestampNanoType INSTANCE_WITH_ZONE = new TimestampNanoType(true); + private static final TimestampNanoType INSTANCE_WITHOUT_ZONE = new TimestampNanoType(false); + + public static TimestampNanoType withZone() { + return INSTANCE_WITH_ZONE; + } + + public static TimestampNanoType withoutZone() { + return INSTANCE_WITHOUT_ZONE; + } + + private final boolean adjustToUTC; + + private TimestampNanoType(boolean adjustToUTC) { + this.adjustToUTC = adjustToUTC; + } + + public boolean shouldAdjustToUTC() { + return adjustToUTC; + } + + @Override + public TypeID typeId() { + return TypeID.TIMESTAMP_NANO; + } + + @Override + public String toString() { + if (shouldAdjustToUTC()) { + return "timestamptz_ns"; + } else { + return "timestamp_ns"; + } + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof TimestampNanoType)) { + return false; + } + + return adjustToUTC == ((TimestampNanoType) other).adjustToUTC; + } + + @Override + public int hashCode() { + return Objects.hash(TimestampNanoType.class, adjustToUTC); + } + } + public static class StringType extends PrimitiveType { private static final StringType INSTANCE = new StringType(); diff --git a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index a2f5301f44a9..3c312486be00 100644 --- a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -35,6 +35,15 @@ private DateTimeUtil() {} public static final LocalDate EPOCH_DAY = EPOCH.toLocalDate(); public static final long MICROS_PER_MILLIS = 1000L; public static final long MICROS_PER_SECOND = 1_000_000L; + private static final long NANOS_PER_SECOND = 1_000_000_000L; + private static final long NANOS_PER_MICRO = 1_000L; + + private static final DateTimeFormatter FORMATTER = + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) + .appendOffset("+HH:MM:ss", "+00:00") + .toFormatter(); public static LocalDate dateFromDays(int daysFromEpoch) { return ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); @@ -60,6 +69,10 @@ public static LocalDateTime timestampFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch).toLocalDateTime(); } + public static LocalDateTime timestampFromNanos(long nanosFromEpoch) { + return ChronoUnit.NANOS.addTo(EPOCH, nanosFromEpoch).toLocalDateTime(); + } + public static long microsFromInstant(Instant instant) { return ChronoUnit.MICROS.between(EPOCH, instant.atOffset(ZoneOffset.UTC)); } @@ -68,6 +81,10 @@ public static long microsFromTimestamp(LocalDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); } + public static long nanosFromTimestamp(LocalDateTime dateTime) { + return ChronoUnit.NANOS.between(EPOCH, dateTime.atOffset(ZoneOffset.UTC)); + } + public static long microsToMillis(long micros) { // When the timestamp is negative, i.e before 1970, we need to adjust the milliseconds portion. // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. @@ -75,6 +92,14 @@ public static long microsToMillis(long micros) { return Math.floorDiv(micros, MICROS_PER_MILLIS); } + public static long nanosToMicros(long nanos) { + return Math.floorDiv(nanos, NANOS_PER_MICRO); + } + + public static long microsToNanos(long micros) { + return Math.multiplyExact(micros, NANOS_PER_MICRO); + } + public static OffsetDateTime timestamptzFromMicros(long microsFromEpoch) { return ChronoUnit.MICROS.addTo(EPOCH, microsFromEpoch); } @@ -83,6 +108,10 @@ public static long microsFromTimestamptz(OffsetDateTime dateTime) { return ChronoUnit.MICROS.between(EPOCH, dateTime); } + public static long nanosFromTimestamptz(OffsetDateTime dateTime) { + return ChronoUnit.NANOS.between(EPOCH, dateTime); + } + public static String formatTimestampMillis(long millis) { return Instant.ofEpochMilli(millis).toString().replace("Z", "+00:00"); } @@ -97,13 +126,12 @@ public static String microsToIsoTime(long micros) { public static String microsToIsoTimestamptz(long micros) { LocalDateTime localDateTime = timestampFromMicros(micros); - DateTimeFormatter zeroOffsetFormatter = - new DateTimeFormatterBuilder() - .parseCaseInsensitive() - .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) - .appendOffset("+HH:MM:ss", "+00:00") - .toFormatter(); - return localDateTime.atOffset(ZoneOffset.UTC).format(zeroOffsetFormatter); + return localDateTime.atOffset(ZoneOffset.UTC).format(FORMATTER); + } + + public static String nanosToIsoTimestamptz(long nanos) { + LocalDateTime localDateTime = timestampFromNanos(nanos); + return localDateTime.atOffset(ZoneOffset.UTC).format(FORMATTER); } public static String microsToIsoTimestamp(long micros) { @@ -111,6 +139,11 @@ public static String microsToIsoTimestamp(long micros) { return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); } + public static String nanosToIsoTimestamp(long nanos) { + LocalDateTime localDateTime = timestampFromNanos(nanos); + return localDateTime.format(DateTimeFormatter.ISO_LOCAL_DATE_TIME); + } + public static int isoDateToDays(String dateString) { return daysFromDate(LocalDate.parse(dateString, DateTimeFormatter.ISO_LOCAL_DATE)); } @@ -124,6 +157,11 @@ public static long isoTimestamptzToMicros(String timestampString) { OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); } + public static long isoTimestamptzToNanos(CharSequence timestampString) { + return nanosFromTimestamptz( + OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME)); + } + public static boolean isUTCTimestamptz(String timestampString) { OffsetDateTime offsetDateTime = OffsetDateTime.parse(timestampString, DateTimeFormatter.ISO_DATE_TIME); @@ -135,6 +173,11 @@ public static long isoTimestampToMicros(String timestampString) { LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); } + public static long isoTimestampToNanos(CharSequence timestampString) { + return nanosFromTimestamp( + LocalDateTime.parse(timestampString, DateTimeFormatter.ISO_LOCAL_DATE_TIME)); + } + public static int daysToYears(int days) { return convertDays(days, ChronoUnit.YEARS); } @@ -185,6 +228,36 @@ private static int convertMicros(long micros, ChronoUnit granularity) { } } + public static int nanosToYears(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.YEARS)); + } + + public static int nanosToMonths(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.MONTHS)); + } + + public static int nanosToDays(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.DAYS)); + } + + public static int nanosToHours(long nanos) { + return Math.toIntExact(convertNanos(nanos, ChronoUnit.HOURS)); + } + + private static long convertNanos(long nanos, ChronoUnit granularity) { + if (nanos >= 0) { + long epochSecond = Math.floorDiv(nanos, NANOS_PER_SECOND); + long nanoAdjustment = Math.floorMod(nanos, NANOS_PER_SECOND); + return granularity.between(EPOCH, toOffsetDateTime(epochSecond, nanoAdjustment)); + } else { + // add 1 nano to the value to account for the case where there is exactly 1 unit between + // the timestamp and epoch because the result will always be decremented. + long epochSecond = Math.floorDiv(nanos, NANOS_PER_SECOND); + long nanoAdjustment = Math.floorMod(nanos + 1, NANOS_PER_SECOND); + return granularity.between(EPOCH, toOffsetDateTime(epochSecond, nanoAdjustment)) - 1; + } + } + private static OffsetDateTime toOffsetDateTime(long epochSecond, long nanoAdjustment) { return Instant.ofEpochSecond(epochSecond, nanoAdjustment).atOffset(ZoneOffset.UTC); } diff --git a/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java b/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java index 5e4ca1fb11be..11f2cb353880 100644 --- a/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java +++ b/api/src/test/java/org/apache/iceberg/PartitionSpecTestBase.java @@ -34,7 +34,8 @@ public class PartitionSpecTestBase { Types.NestedField.required(7, "s", Types.StringType.get()), Types.NestedField.required(8, "u", Types.UUIDType.get()), Types.NestedField.required(9, "f", Types.FixedType.ofLength(3)), - Types.NestedField.required(10, "b", Types.BinaryType.get())); + Types.NestedField.required(10, "b", Types.BinaryType.get()), + Types.NestedField.required(11, "tsn", Types.TimestampNanoType.withoutZone())); // a spec with all of the allowed transform/type pairs public static final PartitionSpec[] SPECS = @@ -49,6 +50,7 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).identity("u").build(), PartitionSpec.builderFor(SCHEMA).identity("f").build(), PartitionSpec.builderFor(SCHEMA).identity("b").build(), + PartitionSpec.builderFor(SCHEMA).identity("tsn").build(), PartitionSpec.builderFor(SCHEMA).bucket("i", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("l", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("d", 128).build(), @@ -59,6 +61,7 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).bucket("u", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("f", 128).build(), PartitionSpec.builderFor(SCHEMA).bucket("b", 128).build(), + PartitionSpec.builderFor(SCHEMA).bucket("tsn", 128).build(), PartitionSpec.builderFor(SCHEMA).year("d").build(), PartitionSpec.builderFor(SCHEMA).month("d").build(), PartitionSpec.builderFor(SCHEMA).day("d").build(), @@ -66,6 +69,10 @@ public class PartitionSpecTestBase { PartitionSpec.builderFor(SCHEMA).month("ts").build(), PartitionSpec.builderFor(SCHEMA).day("ts").build(), PartitionSpec.builderFor(SCHEMA).hour("ts").build(), + PartitionSpec.builderFor(SCHEMA).year("tsn").build(), + PartitionSpec.builderFor(SCHEMA).month("tsn").build(), + PartitionSpec.builderFor(SCHEMA).day("tsn").build(), + PartitionSpec.builderFor(SCHEMA).hour("tsn").build(), PartitionSpec.builderFor(SCHEMA).truncate("i", 10).build(), PartitionSpec.builderFor(SCHEMA).truncate("l", 10).build(), PartitionSpec.builderFor(SCHEMA).truncate("dec", 10).build(), diff --git a/api/src/test/java/org/apache/iceberg/TestAccessors.java b/api/src/test/java/org/apache/iceberg/TestAccessors.java index 332556e474c7..7b4feb845f12 100644 --- a/api/src/test/java/org/apache/iceberg/TestAccessors.java +++ b/api/src/test/java/org/apache/iceberg/TestAccessors.java @@ -180,6 +180,8 @@ public void testTime() { public void testTimestamp() { assertAccessorReturns(Types.TimestampType.withoutZone(), 123L); assertAccessorReturns(Types.TimestampType.withZone(), 123L); + assertAccessorReturns(Types.TimestampNanoType.withoutZone(), 123L); + assertAccessorReturns(Types.TimestampNanoType.withZone(), 123L); } @Test diff --git a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java index 5455415da015..31a6c486bf6e 100644 --- a/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java +++ b/api/src/test/java/org/apache/iceberg/TestPartitionPaths.java @@ -54,6 +54,44 @@ public void testPartitionPath() { .isEqualTo("ts_hour=2017-12-01-10/id_bucket=" + idBucket); } + @Test + public void testPartitionPathWithNanoseconds() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("id", 10).build(); + + Transform hour = Transforms.hour(); + Transform bucket = Transforms.bucket(10); + + Literal ts = + Literal.of("2017-12-01T10:12:55.038194789").to(Types.TimestampNanoType.withoutZone()); + Object tsHour = hour.bind(Types.TimestampNanoType.withoutZone()).apply(ts.value()); + Object idBucket = bucket.bind(Types.IntegerType.get()).apply(1); + + Row partition = Row.of(tsHour, idBucket); + + assertThat(spec.partitionToPath(partition)) + .as("Should produce expected partition key") + .isEqualTo("ts_hour=2017-12-01-10/id_bucket=" + idBucket); + } + + @Test + public void testPartitionPathWithNanosecondsTz() { + PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).hour("ts").bucket("id", 10).build(); + + Transform hour = Transforms.hour(); + Transform bucket = Transforms.bucket(10); + + Literal ts = + Literal.of("2017-12-01T10:12:55.038194789-08:00").to(Types.TimestampNanoType.withZone()); + Object tsTzHour = hour.bind(Types.TimestampNanoType.withZone()).apply(ts.value()); + Object idBucket = bucket.bind(Types.IntegerType.get()).apply(1); + + Row partition = Row.of(tsTzHour, idBucket); + + assertThat(spec.partitionToPath(partition)) + .as("Should produce expected partition key") + .isEqualTo("ts_hour=2017-12-01-18/id_bucket=" + idBucket); + } + @Test public void testEscapedStrings() { PartitionSpec spec = diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java index 894989f38e7d..10d3b6d0adfa 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestExpressionUtil.java @@ -42,11 +42,12 @@ public class TestExpressionUtil { Types.NestedField.required(2, "val", Types.IntegerType.get()), Types.NestedField.required(3, "val2", Types.IntegerType.get()), Types.NestedField.required(4, "ts", Types.TimestampType.withoutZone()), - Types.NestedField.required(5, "date", Types.DateType.get()), - Types.NestedField.required(6, "time", Types.DateType.get()), - Types.NestedField.optional(7, "data", Types.StringType.get()), - Types.NestedField.optional(8, "measurement", Types.DoubleType.get()), - Types.NestedField.optional(9, "test", Types.IntegerType.get())); + Types.NestedField.required(5, "tsns", Types.TimestampNanoType.withoutZone()), + Types.NestedField.required(6, "date", Types.DateType.get()), + Types.NestedField.required(7, "time", Types.DateType.get()), + Types.NestedField.optional(8, "data", Types.StringType.get()), + Types.NestedField.optional(9, "measurement", Types.DoubleType.get()), + Types.NestedField.optional(10, "test", Types.IntegerType.get())); private static final Types.StructType STRUCT = SCHEMA.asStruct(); @@ -460,7 +461,9 @@ public void testSanitizeTimestamp() { "2022-04-29T23:49:51", "2022-04-29T23:49:51.123456", "2022-04-29T23:49:51-07:00", - "2022-04-29T23:49:51.123456+01:00")) { + "2022-04-29T23:49:51.123456+01:00", + "2022-04-29T23:49:51.123456789", + "2022-04-29T23:49:51.123456789+01:00")) { assertEquals( Expressions.equal("test", "(timestamp)"), ExpressionUtil.sanitize(Expressions.equal("test", timestamp))); @@ -496,6 +499,13 @@ public void testSanitizeTimestampAboutNow() { Expression.Operation.EQ, "test", Literal.of(nowLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-about-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(nowLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", nowLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -522,6 +532,13 @@ public void testSanitizeTimestampPast() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesAgoLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesAgoLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesAgoLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -548,6 +565,13 @@ public void testSanitizeTimestampLastWeek() { Expression.Operation.EQ, "test", Literal.of(lastWeekLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-7-days-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(lastWeekLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", lastWeekLocal))) .as("Sanitized string should be identical except for descriptive literal") @@ -574,6 +598,13 @@ public void testSanitizeTimestampFuture() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesFromNowLocal).to(Types.TimestampType.withoutZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-from-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesFromNowLocal).to(Types.TimestampNanoType.withoutZone())))); assertThat( ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesFromNowLocal))) @@ -597,6 +628,13 @@ public void testSanitizeTimestamptzAboutNow() { Expression.Operation.EQ, "test", Literal.of(nowUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-about-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(nowUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", nowUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -618,6 +656,13 @@ public void testSanitizeTimestamptzPast() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesAgoUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesAgoUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesAgoUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -639,6 +684,13 @@ public void testSanitizeTimestamptzLastWeek() { Expression.Operation.EQ, "test", Literal.of(lastWeekUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-7-days-ago)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(lastWeekUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", lastWeekUtc))) .as("Sanitized string should be identical except for descriptive literal") @@ -660,6 +712,13 @@ public void testSanitizeTimestamptzFuture() { Expression.Operation.EQ, "test", Literal.of(ninetyMinutesFromNowUtc).to(Types.TimestampType.withZone())))); + assertEquals( + Expressions.equal("test", "(timestamp-1-hours-from-now)"), + ExpressionUtil.sanitize( + Expressions.predicate( + Expression.Operation.EQ, + "test", + Literal.of(ninetyMinutesFromNowUtc).to(Types.TimestampNanoType.withZone())))); assertThat(ExpressionUtil.toSanitizedString(Expressions.equal("test", ninetyMinutesFromNowUtc))) .as("Sanitized string should be identical except for descriptive literal") diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java index d5aa251ffb50..24fc458b37b4 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestLiteralSerialization.java @@ -38,8 +38,10 @@ public void testLiterals() throws Exception { Literal.of(8.75D), Literal.of("2017-11-29").to(Types.DateType.get()), Literal.of("11:30:07").to(Types.TimeType.get()), - Literal.of("2017-11-29T11:30:07.123").to(Types.TimestampType.withoutZone()), - Literal.of("2017-11-29T11:30:07.123+01:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T11:30:07.123456").to(Types.TimestampType.withoutZone()), + Literal.of("2017-11-29T11:30:07.123456+01:00").to(Types.TimestampType.withZone()), + Literal.of("2017-11-29T11:30:07.123456789").to(Types.TimestampNanoType.withoutZone()), + Literal.of("2017-11-29T11:30:07.123456789+01:00").to(Types.TimestampNanoType.withZone()), Literal.of("abc"), Literal.of(UUID.randomUUID()), Literal.of(new byte[] {1, 2, 3}).to(Types.FixedType.ofLength(3)), diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java index f8d2cd49d969..e2611ddb281f 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestMiscLiteralConversions.java @@ -42,7 +42,9 @@ public void testIdentityConversions() { Pair.of(Literal.of("34.55"), Types.DecimalType.of(9, 2)), Pair.of(Literal.of("2017-08-18"), Types.DateType.get()), Pair.of(Literal.of("14:21:01.919"), Types.TimeType.get()), - Pair.of(Literal.of("2017-08-18T14:21:01.919"), Types.TimestampType.withoutZone()), + Pair.of(Literal.of("2017-08-18T14:21:01.919432"), Types.TimestampType.withoutZone()), + Pair.of( + Literal.of("2017-08-18T14:21:01.919432755"), Types.TimestampNanoType.withoutZone()), Pair.of(Literal.of("abc"), Types.StringType.get()), Pair.of(Literal.of(UUID.randomUUID()), Types.UUIDType.get()), Pair.of(Literal.of(new byte[] {0, 1, 2}), Types.FixedType.ofLength(3)), @@ -62,6 +64,22 @@ public void testIdentityConversions() { } } + @Test + public void testTimestampWithMicrosecondsToDate() { + final Literal micros = + Literal.of("2017-08-18T14:21:01.919432755").to(Types.TimestampType.withoutZone()); + final Literal dateOfNanos = micros.to(Types.DateType.get()); + assertThat(dateOfNanos).isEqualTo(Literal.of("2017-08-18").to(Types.DateType.get())); + } + + @Test + public void testTimestampWithNanosecondsToDate() { + final Literal nanos = + Literal.of("2017-08-18T14:21:01.919432755").to(Types.TimestampNanoType.withoutZone()); + final Literal dateOfNanos = nanos.to(Types.DateType.get()); + assertThat(dateOfNanos).isEqualTo(Literal.of("2017-08-18").to(Types.DateType.get())); + } + @Test public void testBinaryToFixed() { Literal lit = Literal.of(ByteBuffer.wrap(new byte[] {0, 1, 2})); @@ -101,6 +119,8 @@ public void testInvalidBooleanConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), @@ -116,6 +136,8 @@ public void testInvalidIntegerConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -144,6 +166,8 @@ public void testInvalidFloatConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -161,6 +185,8 @@ public void testInvalidDoubleConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -179,6 +205,8 @@ public void testInvalidDateConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 4), Types.StringType.get(), Types.UUIDType.get(), @@ -198,6 +226,8 @@ public void testInvalidTimeConversions() { Types.DateType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 4), Types.StringType.get(), Types.UUIDType.get(), @@ -206,9 +236,26 @@ public void testInvalidTimeConversions() { } @Test - public void testInvalidTimestampConversions() { + public void testInvalidTimestampMicrosConversions() { testInvalidConversions( - Literal.of("2017-08-18T14:21:01.919").to(Types.TimestampType.withoutZone()), + Literal.of("2017-08-18T14:21:01.919123").to(Types.TimestampType.withoutZone()), + Types.BooleanType.get(), + Types.IntegerType.get(), + Types.LongType.get(), + Types.FloatType.get(), + Types.DoubleType.get(), + Types.TimeType.get(), + Types.DecimalType.of(9, 4), + Types.StringType.get(), + Types.UUIDType.get(), + Types.FixedType.ofLength(1), + Types.BinaryType.get()); + } + + @Test + public void testInvalidTimestampNanosConversions() { + testInvalidConversions( + Literal.of("2017-08-18T14:21:01.919123456").to(Types.TimestampNanoType.withoutZone()), Types.BooleanType.get(), Types.IntegerType.get(), Types.LongType.get(), @@ -233,8 +280,10 @@ public void testInvalidDecimalConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(1), @@ -267,8 +316,10 @@ public void testInvalidUUIDConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.FixedType.ofLength(1), @@ -286,8 +337,10 @@ public void testInvalidFixedConversions() { Types.DoubleType.get(), Types.DateType.get(), Types.TimeType.get(), - Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), @@ -307,6 +360,8 @@ public void testInvalidBinaryConversions() { Types.TimeType.get(), Types.TimestampType.withZone(), Types.TimestampType.withoutZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.DecimalType.of(9, 2), Types.StringType.get(), Types.UUIDType.get(), diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java index 1dc2225b3805..45d6654f6aaf 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStringLiteralConversions.java @@ -132,6 +132,51 @@ public void testStringToTimestampLiteral() { .isEqualTo(avroValue); } + @Test + public void testStringToTimestampLiteralWithMicrosecondPrecisionFromNanoseconds() { + // use Avro's timestamp conversion to validate the result + Schema avroSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + TimeConversions.TimestampMicrosConversion avroConversion = + new TimeConversions.TimestampMicrosConversion(); + + Literal timestampStr = Literal.of("2017-08-18T14:21:01.123456789"); + Literal timestamp = timestampStr.to(Types.TimestampType.withoutZone()); + long avroValue = + avroConversion.toLong( + LocalDateTime.of(2017, 8, 18, 14, 21, 1, 123456000).toInstant(ZoneOffset.UTC), + avroSchema, + avroSchema.getLogicalType()); + + assertThat((long) timestamp.value()) + .as("Timestamp without zone should match UTC") + .isEqualTo(avroValue); + } + + @Test + public void testStringToTimestampLiteralWithNanosecondPrecisionFromNanoseconds() { + Literal timestampStr = Literal.of("2017-08-18T14:21:01.123456789"); + Literal timestamp = timestampStr.to(Types.TimestampNanoType.withoutZone()); + + // Not only using Avro's timestamp conversion as it has no timestampNanos(). + long expected = 1503066061123456789L; + assertThat((long) timestamp.value()) + .as("Timestamp without zone should match UTC") + .isEqualTo(expected); + + // use Avro's timestamp conversion to validate the result within one microsecond + Schema avroSchema = LogicalTypes.timestampMicros().addToSchema(Schema.create(Schema.Type.LONG)); + TimeConversions.TimestampMicrosConversion avroConversion = + new TimeConversions.TimestampMicrosConversion(); + long avroValue = + avroConversion.toLong( + LocalDateTime.of(2017, 8, 18, 14, 21, 1, 123456000).toInstant(ZoneOffset.UTC), + avroSchema, + avroSchema.getLogicalType()); + assertThat(timestamp.value() - avroValue * 1000) + .as("Timestamp without zone should match UTC") + .isEqualTo(789L); + } + @Test public void testNegativeStringToTimestampLiteral() { // use Avro's timestamp conversion to validate the result @@ -181,8 +226,13 @@ public void testNegativeStringToTimestampLiteral() { @Test public void testTimestampWithZoneWithoutZoneInLiteral() { // Zone must be present in literals when converting to timestamp with zone - Literal timestampStr = Literal.of("2017-08-18T14:21:01.919"); - assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withZone())) + assertThatThrownBy( + () -> Literal.of("2017-08-18T14:21:01.919").to(Types.TimestampType.withZone())) + .isInstanceOf(DateTimeException.class) + .hasMessageContaining("could not be parsed"); + assertThatThrownBy( + () -> + Literal.of("2017-08-18T14:21:01.919123456").to(Types.TimestampNanoType.withZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } @@ -190,8 +240,14 @@ public void testTimestampWithZoneWithoutZoneInLiteral() { @Test public void testTimestampWithoutZoneWithZoneInLiteral() { // Zone must not be present in literals when converting to timestamp without zone - Literal timestampStr = Literal.of("2017-08-18T14:21:01.919+07:00"); - assertThatThrownBy(() -> timestampStr.to(Types.TimestampType.withoutZone())) + assertThatThrownBy( + () -> Literal.of("2017-08-18T14:21:01.919+07:00").to(Types.TimestampType.withoutZone())) + .isInstanceOf(DateTimeException.class) + .hasMessageContaining("could not be parsed"); + assertThatThrownBy( + () -> + Literal.of("2017-08-18T14:21:01.919123456+07:00") + .to(Types.TimestampNanoType.withoutZone())) .isInstanceOf(DateTimeException.class) .hasMessageContaining("could not be parsed"); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java b/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java new file mode 100644 index 000000000000..379ad4db5e97 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/expressions/TestTimestampLiteralConversions.java @@ -0,0 +1,245 @@ +/* + * 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.expressions; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.time.format.DateTimeParseException; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DateTimeUtil; +import org.junit.jupiter.api.Test; + +public class TestTimestampLiteralConversions { + @Test + public void testTimestampToTimestampNanoConversion() { + Literal timestamp = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1510842668000000L); + + Literal timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(1510842668000000000L); + + timestamp = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(0L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(0L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1L); + + timestampNano = timestamp.to(Types.TimestampNanoType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1000L); + } + + @Test + public void testTimestampToDateConversion() { + Literal ts = + Literal.of("2017-11-16T14:31:08.000001").to(Types.TimestampType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + + ts = Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0).isEqualTo(DateTimeUtil.isoDateToDays("1970-01-01")); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0).isEqualTo(DateTimeUtil.isoDateToDays("1970-01-01")); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1).isEqualTo(DateTimeUtil.isoDateToDays("1969-12-31")); + } + + @Test + public void testTimestampMicrosToDateConversion() { + Literal ts = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + } + + @Test + public void testTimestampNanoToTimestampConversion() { + Literal timestamp = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1510842668000000001L); + + Literal timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(1510842668000000L); + + timestamp = + Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(1L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(0L); + + timestamp = + Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1L); + + timestamp = + Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampNanoType.withoutZone()); + assertThat(timestamp.value()).isEqualTo(-1000L); + + timestampNano = timestamp.to(Types.TimestampType.withoutZone()); + assertThat(timestampNano.value()).isEqualTo(-1L); + } + + @Test + public void testTimestampNanosToDateConversion() { + Literal ts = + Literal.of("2017-11-16T14:31:08.000000001").to(Types.TimestampNanoType.withoutZone()); + int dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(DateTimeUtil.isoDateToDays("2017-11-16")); + + ts = Literal.of("1970-01-01T00:00:00.000000001").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(0); + + ts = Literal.of("1969-12-31T23:59:59.999999999").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + + ts = Literal.of("1969-12-31T23:59:59.999999000").to(Types.TimestampNanoType.withoutZone()); + dateOrdinal = (Integer) ts.to(Types.DateType.get()).value(); + assertThat(dateOrdinal).isEqualTo(-1); + } + + @Test + public void testTimestampNanosWithZoneConversion() { + Literal isoTimestampNanosWithZoneOffset = + Literal.of("2017-11-16T14:31:08.000000001+00:00"); + + assertThatThrownBy(() -> isoTimestampNanosWithZoneOffset.to(Types.TimestampType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampNanosWithZoneOffset.to(Types.TimestampNanoType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampNanosWithZoneOffset.to(Types.TimestampType.withZone()).value()) + .isEqualTo(1510842668000000L); + + assertThat(isoTimestampNanosWithZoneOffset.to(Types.TimestampNanoType.withZone()).value()) + .isEqualTo(1510842668000000001L); + } + + @Test + public void testTimestampMicrosWithZoneConversion() { + Literal isoTimestampMicrosWithZoneOffset = + Literal.of("2017-11-16T14:31:08.000001+00:00"); + + assertThatThrownBy(() -> isoTimestampMicrosWithZoneOffset.to(Types.TimestampType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampMicrosWithZoneOffset.to(Types.TimestampNanoType.withoutZone())) + .as("Should not convert timestamp with offset to a timestamp without zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampMicrosWithZoneOffset.to(Types.TimestampType.withZone()).value()) + .isEqualTo(1510842668000001L); + + assertThat(isoTimestampMicrosWithZoneOffset.to(Types.TimestampNanoType.withZone()).value()) + .isEqualTo(1510842668000001000L); + } + + @Test + public void testTimestampNanosWithoutZoneConversion() { + Literal isoTimestampNanosWithoutZoneOffset = + Literal.of("2017-11-16T14:31:08.000000001"); + + assertThatThrownBy(() -> isoTimestampNanosWithoutZoneOffset.to(Types.TimestampType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampNanosWithoutZoneOffset.to(Types.TimestampNanoType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampNanosWithoutZoneOffset.to(Types.TimestampType.withoutZone()).value()) + .isEqualTo(1510842668000000L); + + assertThat(isoTimestampNanosWithoutZoneOffset.to(Types.TimestampNanoType.withoutZone()).value()) + .isEqualTo(1510842668000000001L); + } + + @Test + public void testTimestampMicrosWithoutZoneConversion() { + Literal isoTimestampMicrosWithoutZoneOffset = + Literal.of("2017-11-16T14:31:08.000001"); + + assertThatThrownBy(() -> isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThatThrownBy( + () -> isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampNanoType.withZone())) + .as("Should not convert timestamp without offset to a timestamp with zone") + .isInstanceOf(DateTimeParseException.class); + + assertThat(isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampType.withoutZone()).value()) + .isEqualTo(1510842668000001L); + + assertThat( + isoTimestampMicrosWithoutZoneOffset.to(Types.TimestampNanoType.withoutZone()).value()) + .isEqualTo(1510842668000001000L); + } +} 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 28d01efa6d3b..fc4333d7c6c5 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestBucketing.java @@ -165,6 +165,62 @@ public void testLong() { .isEqualTo(hashBytes(buffer.array())); } + @Test + public void testTimestampNanoPromotion() { + Types.TimestampType tsType = Types.TimestampType.withoutZone(); + Types.TimestampNanoType tsNsType = Types.TimestampNanoType.withoutZone(); + Bucket tsNsBucket = Bucket.get(tsNsType, 1); + Bucket tsBucket = Bucket.get(tsType, 1); + + // Values from spec Appendix B: 32-bit Hash Requirements + assertThat(tsBucket.hash(Literal.of("2017-11-16T22:31:08").to(tsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + assertThat(tsNsBucket.hash(Literal.of("2017-11-16T22:31:08").to(tsNsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + + assertThat(tsBucket.hash(Literal.of("2017-11-16T22:31:08.000001").to(tsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08.000001) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + assertThat(tsNsBucket.hash(Literal.of("2017-11-16T22:31:08.000001001").to(tsNsType).value())) + .as( + "Spec example: hash(2017-11-16T22:31:08.000001) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + } + + @Test + public void testTimestampTzNanoPromotion() { + Types.TimestampType tsTzType = Types.TimestampType.withZone(); + Types.TimestampNanoType tsTzNsType = Types.TimestampNanoType.withZone(); + Bucket tsTzNsBucket = Bucket.get(tsTzNsType, 1); + Bucket tsTzBucket = Bucket.get(tsTzType, 1); + + // Values from spec Appendix B: 32-bit Hash Requirements + assertThat(tsTzBucket.hash(Literal.of("2017-11-16T14:31:08-08:00").to(tsTzType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08-08:00) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + assertThat(tsTzNsBucket.hash(Literal.of("2017-11-16T14:31:08-08:00").to(tsTzNsType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08-08:00) = -2047944441 for Timestamp and TimestampNano should match") + .isEqualTo(-2047944441); + + assertThat(tsTzBucket.hash(Literal.of("2017-11-16T14:31:08.000001-08:00").to(tsTzType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08.000001-08:00) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + assertThat( + tsTzNsBucket.hash( + Literal.of("2017-11-16T14:31:08.000001001-08:00").to(tsTzNsType).value())) + .as( + "Spec example: hash(2017-11-16T14:31:08.000001-08:00) = -1207196810 for Timestamp and TimestampNano should match") + .isEqualTo(-1207196810); + } + @Test public void testIntegerTypePromotion() { int randomInt = testRandom.nextInt(); diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestDates.java b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java index b9c380244666..c899b4cfa1cb 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestDates.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestDates.java @@ -26,6 +26,75 @@ import org.junit.jupiter.api.Test; public class TestDates { + @Test + public void testSatisfiesOrderOfDates() { + assertThat(Dates.DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimestamps() { + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimestampNanos() { + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testSatisfiesOrderOfTimeTransforms() { + assertThat(Dates.DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Dates.DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Dates.MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Dates.MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Dates.MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Dates.YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Dates.YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + @Test @SuppressWarnings("deprecation") public void testDeprecatedDateTransform() { 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 6101fdf0986d..93d3281411f3 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestIdentity.java @@ -106,7 +106,7 @@ public void testTimestampWithZoneHumanString() { // value will always be in UTC assertThat(identity.toHumanString(timestamptz, ts.value())) .as("Should produce timestamp with time zone adjusted to UTC") - .isEqualTo("2017-12-01T18:12:55.038194Z"); + .isEqualTo("2017-12-01T18:12:55.038194+00:00"); } @Test diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java new file mode 100644 index 000000000000..2a161f9bc822 --- /dev/null +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimeTransforms.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.transforms; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestTimeTransforms { + @Test + public void testMicrosSatisfiesOrderOfDates() { + assertThat(Hours.get().satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestamps() { + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestampNanos() { + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimeTransforms() { + assertThat(Hours.get().satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Hours.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Days.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Days.get().satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Days.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Months.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Months.get().satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Months.get().satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Years.get().satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Years.get().satisfiesOrderOf(Years.get())).isTrue(); + } + + @Test + public void testHoursToEnum() { + Hours hours = Hours.get(); + Types.DateType type = Types.DateType.get(); + assertThatThrownBy(() -> hours.toEnum(type)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageMatching("Unsupported type: date"); + } +} diff --git a/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java index 3c37e643eb95..78b0e67c686b 100644 --- a/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java +++ b/api/src/test/java/org/apache/iceberg/transforms/TestTimestamps.java @@ -26,13 +26,222 @@ import org.junit.jupiter.api.Test; public class TestTimestamps { + @Test + public void testMicrosSatisfiesOrderOfDates() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestamps() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimestampNanos() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testMicrosSatisfiesOrderOfTimeTransforms() { + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_HOUR.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.MICROS_TO_MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Timestamps.MICROS_TO_YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfDates() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Dates.YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Dates.YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimestamps() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.MICROS_TO_YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimestampNanos() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_HOUR)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_DAY)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_MONTH)).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Timestamps.NANOS_TO_YEAR)).isTrue(); + } + + @Test + public void testNanosSatisfiesOrderOfTimeTransforms() { + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Hours.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_HOUR.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Days.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_DAY.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Months.get())).isTrue(); + assertThat(Timestamps.NANOS_TO_MONTH.satisfiesOrderOf(Years.get())).isTrue(); + + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Hours.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Days.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Months.get())).isFalse(); + assertThat(Timestamps.NANOS_TO_YEAR.satisfiesOrderOf(Years.get())).isTrue(); + } + @Test @SuppressWarnings("deprecation") public void testDeprecatedTimestampTransform() { Types.TimestampType type = Types.TimestampType.withoutZone(); Literal ts = Literal.of("2017-12-01T10:12:55.038194").to(type); Literal pts = Literal.of("1970-01-01T00:00:01.000001").to(type); - Literal nts = Literal.of("1969-12-31T23:59:58.999999").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999").to(type); + + Transform years = Transforms.year(type); + assertThat((int) years.apply(ts.value())).as("Should produce 2017 - 1970 = 47").isEqualTo(47); + assertThat((int) years.apply(pts.value())).as("Should produce 1970 - 1970 = 0").isZero(); + assertThat((int) years.apply(nts.value())).as("Should produce 1969 - 1970 = -1").isEqualTo(-1); + + Transform months = Transforms.month(type); + assertThat((int) months.apply(ts.value())) + .as("Should produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat((int) months.apply(pts.value())).as("Should produce 0 * 12 + 0 = 0").isZero(); + assertThat((int) months.apply(nts.value())).isEqualTo(-1); + + Transform days = Transforms.day(type); + assertThat((int) days.apply(ts.value())).as("Should produce 17501").isEqualTo(17501); + assertThat((int) days.apply(pts.value())).as("Should produce 0 * 365 + 0 = 0").isZero(); + assertThat((int) days.apply(nts.value())).isEqualTo(-1); + + Transform hours = Transforms.hour(type); + assertThat((int) hours.apply(ts.value())) + .as("Should produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat((int) hours.apply(pts.value())).as("Should produce 0 * 24 + 0 = 0").isZero(); + assertThat((int) hours.apply(nts.value())).isEqualTo(-1); + } + + @Test + @SuppressWarnings("deprecation") + public void testDeprecatedTimestampNanoTransform() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal ts = Literal.of("2017-12-01T10:12:55.038194789").to(type); + Literal pts = Literal.of("1970-01-01T00:00:01.000000001").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999999").to(type); Transform years = Transforms.year(type); assertThat((int) years.apply(ts.value())).as("Should produce 2017 - 1970 = 47").isEqualTo(47); @@ -64,7 +273,51 @@ public void testTimestampTransform() { Types.TimestampType type = Types.TimestampType.withoutZone(); Literal ts = Literal.of("2017-12-01T10:12:55.038194").to(type); Literal pts = Literal.of("1970-01-01T00:00:01.000001").to(type); - Literal nts = Literal.of("1969-12-31T23:59:58.999999").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999").to(type); + + Transform years = Transforms.year(); + assertThat((int) years.bind(type).apply(ts.value())) + .as("Should produce 2017 - 1970 = 47") + .isEqualTo(47); + assertThat((int) years.bind(type).apply(pts.value())) + .as("Should produce 1970 - 1970 = 0") + .isZero(); + assertThat((int) years.bind(type).apply(nts.value())) + .as("Should produce 1969 - 1970 = -1") + .isEqualTo(-1); + + Transform months = Transforms.month(); + assertThat((int) months.bind(type).apply(ts.value())) + .as("Should produce 47 * 12 + 11 = 575") + .isEqualTo(575); + assertThat((int) months.bind(type).apply(pts.value())) + .as("Should produce 0 * 12 + 0 = 0") + .isZero(); + assertThat((int) months.bind(type).apply(nts.value())).isEqualTo(-1); + + Transform days = Transforms.day(); + assertThat((int) days.bind(type).apply(ts.value())).as("Should produce 17501").isEqualTo(17501); + assertThat((int) days.bind(type).apply(pts.value())) + .as("Should produce 0 * 365 + 0 = 0") + .isZero(); + assertThat((int) days.bind(type).apply(nts.value())).isEqualTo(-1); + + Transform hours = Transforms.hour(); + assertThat((int) hours.bind(type).apply(ts.value())) + .as("Should produce 17501 * 24 + 10") + .isEqualTo(420034); + assertThat((int) hours.bind(type).apply(pts.value())) + .as("Should produce 0 * 24 + 0 = 0") + .isZero(); + assertThat((int) hours.bind(type).apply(nts.value())).isEqualTo(-1); + } + + @Test + public void testTimestampNanoTransform() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal ts = Literal.of("2017-12-01T10:12:55.038194789").to(type); + Literal pts = Literal.of("1970-01-01T00:00:01.000000001").to(type); + Literal nts = Literal.of("1969-12-31T23:59:59.999999999").to(type); Transform years = Transforms.year(); assertThat((int) years.bind(type).apply(ts.value())) @@ -123,6 +376,26 @@ public void testTimestampWithoutZoneToHumanString() { .isEqualTo("2017-12-01-10"); } + @Test + public void testTimestampNanoWithoutZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("2017-12-01T10:12:55.038194789").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("2017"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("2017-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("2017-12-01"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("2017-12-01-10"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -143,6 +416,26 @@ public void testNegativeTimestampWithoutZoneToHumanString() { .isEqualTo("1969-12-30-10"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("1969-12-30T10:12:55.038194789").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-30"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-30-10"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanStringLowerBound() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -163,6 +456,26 @@ public void testNegativeTimestampWithoutZoneToHumanStringLowerBound() { .isEqualTo("1969-12-30-00"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanStringLowerBound() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("1969-12-30T00:00:00.000000000").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-30"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-30-00"); + } + @Test public void testNegativeTimestampWithoutZoneToHumanStringUpperBound() { Types.TimestampType type = Types.TimestampType.withoutZone(); @@ -183,6 +496,26 @@ public void testNegativeTimestampWithoutZoneToHumanStringUpperBound() { .isEqualTo("1969-12-31-23"); } + @Test + public void testNegativeTimestampNanoWithoutZoneToHumanStringUpperBound() { + Types.TimestampNanoType type = Types.TimestampNanoType.withoutZone(); + Literal date = Literal.of("1969-12-31T23:59:59.999999999").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("1969"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("1969-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("1969-12-31"); + + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("1969-12-31-23"); + } + @Test public void testTimestampWithZoneToHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); @@ -205,7 +538,28 @@ public void testTimestampWithZoneToHumanString() { } @Test - public void testNullHumanString() { + public void testTimestampNanoWithZoneToHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + Literal date = Literal.of("2017-12-01T10:12:55.038194789-08:00").to(type); + + Transform year = Transforms.year(); + assertThat(year.toHumanString(type, year.bind(type).apply(date.value()))).isEqualTo("2017"); + + Transform month = Transforms.month(); + assertThat(month.toHumanString(type, month.bind(type).apply(date.value()))) + .isEqualTo("2017-12"); + + Transform day = Transforms.day(); + assertThat(day.toHumanString(type, day.bind(type).apply(date.value()))).isEqualTo("2017-12-01"); + + // the hour is 18 because the value is always UTC + Transform hour = Transforms.hour(); + assertThat(hour.toHumanString(type, hour.bind(type).apply(date.value()))) + .isEqualTo("2017-12-01-18"); + } + + @Test + public void testTimestampNullHumanString() { Types.TimestampType type = Types.TimestampType.withZone(); assertThat(Transforms.year().toHumanString(type, null)) .as("Should produce \"null\" for null") @@ -221,6 +575,23 @@ public void testNullHumanString() { .isEqualTo("null"); } + @Test + public void testTimestampNanoNullHumanString() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + assertThat(Transforms.year().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.month().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.day().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + assertThat(Transforms.hour().toHumanString(type, null)) + .as("Should produce \"null\" for null") + .isEqualTo("null"); + } + @Test public void testTimestampsReturnType() { Types.TimestampType type = Types.TimestampType.withZone(); @@ -241,4 +612,25 @@ public void testTimestampsReturnType() { Type hourResultType = hour.getResultType(type); assertThat(hourResultType).isEqualTo(Types.IntegerType.get()); } + + @Test + public void testTimestampNanosReturnType() { + Types.TimestampNanoType type = Types.TimestampNanoType.withZone(); + + Transform year = Transforms.year(); + Type yearResultType = year.getResultType(type); + assertThat(yearResultType).isEqualTo(Types.IntegerType.get()); + + Transform month = Transforms.month(); + Type monthResultType = month.getResultType(type); + assertThat(monthResultType).isEqualTo(Types.IntegerType.get()); + + Transform day = Transforms.day(); + Type dayResultType = day.getResultType(type); + assertThat(dayResultType).isEqualTo(Types.DateType.get()); + + Transform hour = Transforms.hour(); + Type hourResultType = hour.getResultType(type); + assertThat(hourResultType).isEqualTo(Types.IntegerType.get()); + } } diff --git a/api/src/test/java/org/apache/iceberg/types/TestComparators.java b/api/src/test/java/org/apache/iceberg/types/TestComparators.java index 165d96c029cc..07653ba3c8a8 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestComparators.java +++ b/api/src/test/java/org/apache/iceberg/types/TestComparators.java @@ -79,6 +79,12 @@ public void testTimestamp() { assertComparesCorrectly(Comparators.forType(Types.TimestampType.withZone()), 111, 222); } + @Test + public void testTimestampNanos() { + assertComparesCorrectly(Comparators.forType(Types.TimestampNanoType.withoutZone()), 111, 222); + assertComparesCorrectly(Comparators.forType(Types.TimestampNanoType.withZone()), 111, 222); + } + @Test public void testString() { assertComparesCorrectly(Comparators.forType(Types.StringType.get()), "a", "b"); diff --git a/api/src/test/java/org/apache/iceberg/types/TestConversions.java b/api/src/test/java/org/apache/iceberg/types/TestConversions.java index 6c7a884a5839..e207cfd8d59a 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestConversions.java +++ b/api/src/test/java/org/apache/iceberg/types/TestConversions.java @@ -37,6 +37,7 @@ import org.apache.iceberg.types.Types.LongType; import org.apache.iceberg.types.Types.StringType; import org.apache.iceberg.types.Types.TimeType; +import org.apache.iceberg.types.Types.TimestampNanoType; import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.types.Types.UUIDType; import org.junit.jupiter.api.Test; @@ -93,7 +94,7 @@ public void testByteBufferConversions() { assertThat(Literal.of(10000L).to(TimeType.get()).toByteBuffer().array()) .isEqualTo(new byte[] {16, 39, 0, 0, 0, 0, 0, 0}); - // timestamps are stored as microseconds from 1970-01-01 00:00:00.000000 in an 8-byte + // timestamps are stored as micro|nanoseconds from 1970-01-01 00:00:00 in an 8-byte // little-endian long // 400000L is 0...110|00011010|10000000 in binary // 10000000 -> -128, 00011010 -> 26, 00000110 -> 6, ... , 00000000 -> 0 @@ -103,6 +104,16 @@ public void testByteBufferConversions() { .isEqualTo(new byte[] {-128, 26, 6, 0, 0, 0, 0, 0}); assertThat(Literal.of(400000L).to(TimestampType.withZone()).toByteBuffer().array()) .isEqualTo(new byte[] {-128, 26, 6, 0, 0, 0, 0, 0}); + // values passed to assertConversion and Literal.of differ because Literal.of(...) assumes + // the value is in micros, which gets converted when to(TimestampNanoType) is called + assertConversion( + 400000000L, TimestampNanoType.withoutZone(), new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertConversion( + 400000000L, TimestampNanoType.withZone(), new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertThat(Literal.of(400000L).to(TimestampNanoType.withoutZone()).toByteBuffer().array()) + .isEqualTo(new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); + assertThat(Literal.of(400000L).to(TimestampNanoType.withZone()).toByteBuffer().array()) + .isEqualTo(new byte[] {0, -124, -41, 23, 0, 0, 0, 0}); // strings are stored as UTF-8 bytes (without length) // 'A' -> 65, 'B' -> 66, 'C' -> 67 diff --git a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java index 7f5948bd5838..2d02da5346a7 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java +++ b/api/src/test/java/org/apache/iceberg/types/TestReadabilityChecks.java @@ -39,6 +39,8 @@ public class TestReadabilityChecks { Types.TimeType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.FixedType.ofLength(3), 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 d981b5a26789..96c330d6eb43 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestSerializableTypes.java @@ -41,6 +41,8 @@ public void testIdentityTypes() throws Exception { Types.TimeType.get(), Types.TimestampType.withoutZone(), Types.TimestampType.withZone(), + Types.TimestampNanoType.withoutZone(), + Types.TimestampNanoType.withZone(), Types.StringType.get(), Types.UUIDType.get(), Types.BinaryType.get() diff --git a/api/src/test/java/org/apache/iceberg/types/TestTypes.java b/api/src/test/java/org/apache/iceberg/types/TestTypes.java index 5db7ca7cd1bc..226c53f1e9ce 100644 --- a/api/src/test/java/org/apache/iceberg/types/TestTypes.java +++ b/api/src/test/java/org/apache/iceberg/types/TestTypes.java @@ -31,6 +31,11 @@ public void fromPrimitiveString() { assertThat(Types.fromPrimitiveString("BooLean")).isSameAs(Types.BooleanType.get()); assertThat(Types.fromPrimitiveString("timestamp")).isSameAs(Types.TimestampType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamptz")).isSameAs(Types.TimestampType.withZone()); + assertThat(Types.fromPrimitiveString("timestamp_ns")) + .isSameAs(Types.TimestampNanoType.withoutZone()); + assertThat(Types.fromPrimitiveString("timestamptz_ns")) + .isSameAs(Types.TimestampNanoType.withZone()); assertThat(Types.fromPrimitiveString("Fixed[ 3 ]")).isEqualTo(Types.FixedType.ofLength(3)); diff --git a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java index 42da3a3dd774..6088fe51b57a 100644 --- a/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java +++ b/api/src/test/java/org/apache/iceberg/util/TestDateTimeUtil.java @@ -20,19 +20,72 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.time.ZonedDateTime; +import org.apache.iceberg.transforms.Transforms; +import org.apache.iceberg.types.Types; import org.junit.jupiter.api.Test; public class TestDateTimeUtil { + @Test + public void nanosToMicros() { + assertThat(DateTimeUtil.nanosToMicros(1510871468000001001L)).isEqualTo(1510871468000001L); + assertThat(DateTimeUtil.nanosToMicros(-1510871468000001001L)).isEqualTo(-1510871468000002L); + } + + @Test + public void microsToNanos() { + assertThat(DateTimeUtil.microsToNanos(1510871468000001L)).isEqualTo(1510871468000001000L); + assertThat(DateTimeUtil.microsToNanos(-1510871468000001L)).isEqualTo(-1510871468000001000L); + } + + @Test + public void isoTimestampToNanos() { + assertThat(DateTimeUtil.isoTimestampToNanos("2017-11-16T22:31:08.000001001")) + .isEqualTo(1510871468000001001L); + assertThat(DateTimeUtil.isoTimestampToNanos("1922-02-15T01:28:51.999998999")) + .isEqualTo(-1510871468000001001L); + } + + @Test + public void isoTimestamptzToNanos() { + assertThat(DateTimeUtil.isoTimestamptzToNanos("2017-11-16T14:31:08.000001001-08:00")) + .isEqualTo(1510871468000001001L); + assertThat(DateTimeUtil.isoTimestamptzToNanos("1922-02-15T01:28:51.999998999+00:00")) + .isEqualTo(-1510871468000001001L); + } + + @Test + public void convertNanos() { + assertThat( + Transforms.identity() + .toHumanString(Types.TimestampNanoType.withoutZone(), 1510871468000001001L)) + .isEqualTo("2017-11-16T22:31:08.000001001"); + assertThat(DateTimeUtil.nanosToYears(1510871468000001001L)).isEqualTo(47); + assertThat(Transforms.year().toHumanString(Types.IntegerType.get(), 47)).isEqualTo("2017"); + assertThat(DateTimeUtil.nanosToMonths(1510871468000001001L)).isEqualTo(574); + assertThat(Transforms.month().toHumanString(Types.IntegerType.get(), 574)).isEqualTo("2017-11"); + assertThat(DateTimeUtil.nanosToDays(1510871468000001001L)).isEqualTo(17486); + assertThat(Transforms.day().toHumanString(Types.IntegerType.get(), 17486)) + .isEqualTo("2017-11-16"); + assertThat(DateTimeUtil.nanosToHours(1510871468000001001L)).isEqualTo(419686); + assertThat(Transforms.hour().toHumanString(Types.IntegerType.get(), 419686)) + .isEqualTo("2017-11-16-22"); + } @Test - public void formatTimestampMillis() { - String timestamp = "1970-01-01T00:00:00.001+00:00"; - assertThat(DateTimeUtil.formatTimestampMillis(1L)).isEqualTo(timestamp); - assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1L); - - timestamp = "1970-01-01T00:16:40+00:00"; - assertThat(DateTimeUtil.formatTimestampMillis(1000000L)).isEqualTo(timestamp); - assertThat(ZonedDateTime.parse(timestamp).toInstant().toEpochMilli()).isEqualTo(1000000L); + public void convertNanosNegative() { + assertThat( + Transforms.identity() + .toHumanString(Types.TimestampNanoType.withZone(), -1510871468000001001L)) + .isEqualTo("1922-02-15T01:28:51.999998999+00:00"); + assertThat(DateTimeUtil.nanosToYears(-1510871468000001001L)).isEqualTo(-48); + assertThat(Transforms.year().toHumanString(Types.IntegerType.get(), 47)).isEqualTo("2017"); + assertThat(DateTimeUtil.nanosToMonths(-1510871468000001001L)).isEqualTo(-575); + assertThat(Transforms.month().toHumanString(Types.IntegerType.get(), 574)).isEqualTo("2017-11"); + assertThat(DateTimeUtil.nanosToDays(-1510871468000001001L)).isEqualTo(-17487); + assertThat(Transforms.day().toHumanString(Types.IntegerType.get(), 17486)) + .isEqualTo("2017-11-16"); + assertThat(DateTimeUtil.nanosToHours(-1510871468000001001L)).isEqualTo(-419687); + assertThat(Transforms.hour().toHumanString(Types.IntegerType.get(), 419686)) + .isEqualTo("2017-11-16-22"); } } diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index abb2ee6cc3e9..c0bb16bc109d 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -1494,6 +1494,8 @@ private int addSchemaInternal(Schema schema, int newLastColumnId) { newLastColumnId, lastColumnId); + Schema.checkCompatibility(schema, formatVersion); + int newSchemaId = reuseOrCreateNewSchemaId(schema); boolean schemaFound = schemasById.containsKey(newSchemaId); if (schemaFound && newLastColumnId == lastColumnId) { diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 809b1ba963c7..5be8187a923e 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -64,6 +64,7 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -1635,4 +1636,54 @@ public void buildReplacementKeepsSnapshotLog() throws Exception { .hasSize(2) .containsExactlyElementsOf(metadata.snapshotLog()); } + + @Test + public void testConstructV3Metadata() { + TableMetadata.newTableMetadata( + TEST_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + 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)) { + Assertions.assertThrows( + IllegalStateException.class, + () -> + TableMetadata.newTableMetadata( + v3Schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + unsupportedFormatVersion), + String.format( + "Invalid type in v%s schema: 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); + } } diff --git a/format/spec.md b/format/spec.md index b8da0454c648..1297ec0365b2 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1076,10 +1076,10 @@ The 32-bit hash implementation is 32-bit Murmur3 hash, x86 variant, seeded with | **`time`** | `hashLong(microsecsFromMidnight(v))` | `22:31:08` → `-662762989` | | **`timestamp`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-2047944441`
    `2017-11-16T22:31:08.000001` → `-1207196810` | | **`timestamptz`** | `hashLong(microsecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00` → `-2047944441`
    `2017-11-16T14:31:08.000001-08:00` → `-1207196810` | -| **`timestamp_ns`** | `hashLong(nanosecsFromUnixEpoch(v))` | `2017-11-16T22:31:08` → `-737750069`
    `2017-11-16T22:31:08.000001` → `-976603392`
    `2017-11-16T22:31:08.000000001` → `-160215926` | -| **`timestamptz_ns`** | `hashLong(nanosecsFromUnixEpoch(v))` | `2017-11-16T14:31:08-08:00` → `-737750069`
    `2017-11-16T14:31:08.000001-08:00` → `-976603392`
    `2017-11-16T14:31:08.000000001-08:00` → `-160215926` | +| **`timestamp_ns`** | `hashLong(microsecsFromUnixEpoch(v))` [3] | `2017-11-16T22:31:08` → `-2047944441`
    `2017-11-16T22:31:08.000001001` → `-1207196810` | +| **`timestamptz_ns`** | `hashLong(microsecsFromUnixEpoch(v))` [3]| `2017-11-16T14:31:08-08:00` → `-2047944441`
    `2017-11-16T14:31:08.000001001-08:00` → `-1207196810` | | **`string`** | `hashBytes(utf8Bytes(v))` | `iceberg` → `1210000089` | -| **`uuid`** | `hashBytes(uuidBytes(v))` [3] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | +| **`uuid`** | `hashBytes(uuidBytes(v))` [4] | `f79c3e09-677c-4bbd-a479-3f349cb785e7` → `1488055340` | | **`fixed(L)`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | | **`binary`** | `hashBytes(v)` | `00 01 02 03` → `-188683207` | @@ -1088,17 +1088,18 @@ The types below are not currently valid for bucketing, and so are not hashed. Ho | Primitive type | Hash specification | Test value | |--------------------|-------------------------------------------|--------------------------------------------| | **`boolean`** | `false: hashInt(0)`, `true: hashInt(1)` | `true` → `1392991556` | -| **`float`** | `hashLong(doubleToLongBits(double(v))` [4]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | -| **`double`** | `hashLong(doubleToLongBits(v))` [4]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | +| **`float`** | `hashLong(doubleToLongBits(double(v))` [5]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | +| **`double`** | `hashLong(doubleToLongBits(v))` [5]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | Notes: 1. Integer and long hash results must be identical for all integer values. This ensures that schema evolution does not change bucket partition values if integer types are promoted. 2. Decimal values are hashed using the minimum number of bytes required to hold the unscaled value as a two’s complement big-endian; this representation does not include padding bytes required for storage in a fixed-length array. Hash results are not dependent on decimal scale, which is part of the type, not the data value. -3. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: +3. Nanosecond timestamps must be converted to microsecond precision before hashing to ensure timestamps have the same hash value. +4. UUIDs are encoded using big endian. The test UUID for the example above is: `f79c3e09-677c-4bbd-a479-3f349cb785e7`. This UUID encoded as a byte array is: `F7 9C 3E 09 67 7C 4B BD A4 79 3F 34 9C B7 85 E7` -4. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. +5. `doubleToLongBits` must give the IEEE 754 compliant bit representation of the double value. All `NaN` bit patterns must be canonicalized to `0x7ff8000000000000L`. Negative zero (`-0.0`) must be canonicalized to positive zero (`0.0`). Float hash values are the result of hashing the float cast to double to ensure that schema evolution does not change hash values if float types are promoted. ## Appendix C: JSON serialization From a01ea69b6dd0a80e0439b6ee99bf7d9fea2f4900 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 3 Sep 2024 23:55:51 +0530 Subject: [PATCH 0660/1019] Core: Refactor ZOrderByteUtils (#10624) --- .../apache/iceberg/util/ZOrderByteUtils.java | 59 +++++++++---------- 1 file changed, 29 insertions(+), 30 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java index 4a5805742443..7edca59731a5 100644 --- a/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java +++ b/core/src/main/java/org/apache/iceberg/util/ZOrderByteUtils.java @@ -50,41 +50,45 @@ static ByteBuffer allocatePrimitiveBuffer() { return ByteBuffer.allocate(PRIMITIVE_BUFFER_SIZE); } - /** - * Signed ints do not have their bytes in magnitude order because of the sign bit. To fix this, - * flip the sign bit so that all negatives are ordered before positives. This essentially shifts - * the 0 value so that we don't break our ordering when we cross the new 0 value. - */ + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ public static ByteBuffer intToOrderedBytes(int val, ByteBuffer reuse) { - ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); - bytes.putLong(((long) val) ^ 0x8000000000000000L); - return bytes; + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); + } + + /** Internally just calls {@link #wholeNumberOrderedBytes(long, ByteBuffer)} */ + public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { + return wholeNumberOrderedBytes(val, reuse); } /** - * Signed longs are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} + * Signed longs do not have their bytes in magnitude order because of the sign bit. To fix this, + * flip the sign bit so that all negatives are ordered before positives. This essentially shifts + * the 0 value so that we don't break our ordering when we cross the new 0 value. */ - public static ByteBuffer longToOrderedBytes(long val, ByteBuffer reuse) { + public static ByteBuffer wholeNumberOrderedBytes(long val, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); bytes.putLong(val ^ 0x8000000000000000L); return bytes; } - /** - * Signed shorts are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} - */ - public static ByteBuffer shortToOrderedBytes(short val, ByteBuffer reuse) { - return intToOrderedBytes(val, reuse); + /** Internally just calls {@link #floatingPointOrderedBytes(double, ByteBuffer)} */ + public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { + return floatingPointOrderedBytes(val, reuse); } - /** - * Signed tiny ints are treated the same as the signed ints in {@link #intToOrderedBytes(int, - * ByteBuffer)} - */ - public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { - return intToOrderedBytes(val, reuse); + /** Internally just calls {@link #floatingPointOrderedBytes(double, ByteBuffer)} */ + public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) { + return floatingPointOrderedBytes(val, reuse); } /** @@ -92,15 +96,10 @@ public static ByteBuffer tinyintToOrderedBytes(byte val, ByteBuffer reuse) { * y), they are ordered the same way when their bits are reinterpreted as sign-magnitude * integers.” * - *

    Which means floats can be treated as sign magnitude integers which can then be converted + *

    Which means doubles can be treated as sign magnitude integers which can then be converted * into lexicographically comparable bytes */ - public static ByteBuffer floatToOrderedBytes(float val, ByteBuffer reuse) { - return doubleToOrderedBytes(val, reuse); - } - - /** Doubles are treated the same as floats in {@link #floatToOrderedBytes(float, ByteBuffer)} */ - public static ByteBuffer doubleToOrderedBytes(double val, ByteBuffer reuse) { + public static ByteBuffer floatingPointOrderedBytes(double val, ByteBuffer reuse) { ByteBuffer bytes = ByteBuffers.reuse(reuse, PRIMITIVE_BUFFER_SIZE); long lval = Double.doubleToLongBits(val); lval ^= ((lval >> (Integer.SIZE - 1)) | Long.MIN_VALUE); From e80a97c8cede4ffe7aa9bf68c342910773c1f60c Mon Sep 17 00:00:00 2001 From: emkornfield Date: Tue, 3 Sep 2024 11:49:45 -0700 Subject: [PATCH 0661/1019] Docs: Initial committer guidelines and requirements for merging (#10780) --- site/docs/contribute.md | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/site/docs/contribute.md b/site/docs/contribute.md index 60bc89f9537f..beda3cfc507e 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -45,6 +45,18 @@ The Iceberg community prefers to receive contributions as [Github pull requests] * If a PR is related to an issue, adding `Closes #1234` in the PR description will automatically close the issue and helps keep the project clean * If a PR is posted for visibility and isn't necessarily ready for review or merging, be sure to convert the PR to a draft +### Merging Pull Requests + +Most pull requests can be merged once a single [committer](https://www.apache.org/foundation/how-it-works/#committers) other than the author is satisfied with the code in the PR (exceptions that require additional input from the community are detailed below). [Committers are trusted](https://infra.apache.org/new-committers-guide.html#the-committers-way) to act in the best [interest of the project](https://community.apache.org/projectIndependence.html#apache-projects-are-managed-independently). + +Before merging all review comments should be addressed either by making changes or agreeing the request is out of scope for the PR. For additions to public APIs committers should wait at least 24 hours before merging to ensure there is no additional feedback from members of the community. + +Requesting changes on a PR indicates a reviewer believes the PR has merit but still needs issues addressed before merging. If a reviewer believes the change should not be merged at all and there is nothing the author could do to address the reviewers concerns, the reviewer should explicitly state this on the PR. In the rare event that a PR author and reviewers cannot come to a consensus on a PR, the disagreement should be raised to the developer mailing list for further discussion. In this context, a reviewer is anyone leaving comments on the PR including contributors, committers and PMC members. + +There are several exceptions to a single committer being able to merge a PR: + +* Behavioral and functional changes to a specification must go through the [Iceberg improvement proposal](#apache-iceberg-improvement-proposals) before any code can be merged. +* Changes to files under the `format` directory and `open-api/rest-catalog*` are considered specification changes. Unless already covered under an Iceberg improvement proposal, specification changes require their own vote (e.g. bug fixes or specification clarifications). The vote follows the ASF [code modification](https://www.apache.org/foundation/voting.html#votes-on-code-modification) model and no lazy consensus modifier. Grammar, spelling and minor formatting fixes are exempted from this rule. Draft specifications (new independent specifications that are going through the Iceberg improvement process) do not require a vote but authors should provide notice on the developer mailing list about substantive changes (the final draft will be subject to a vote). ## Apache Iceberg Improvement Proposals From 034b403ec888cff3cb41df6245742e53bf8cef0e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 4 Sep 2024 00:54:32 +0530 Subject: [PATCH 0662/1019] Flink: Fix compile warning (#11072) --- .../apache/iceberg/flink/maintenance/operator/TableChange.java | 2 ++ .../apache/iceberg/flink/maintenance/operator/TableChange.java | 2 ++ 2 files changed, 4 insertions(+) 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 5252cf61b031..773b34b6c495 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 @@ -192,6 +192,8 @@ static class Builder { private long eqDeleteRecordCount = 0L; private int commitCount = 0; + private Builder() {} + public Builder dataFileCount(int newDataFileCount) { this.dataFileCount = newDataFileCount; return this; 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 5252cf61b031..773b34b6c495 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 @@ -192,6 +192,8 @@ static class Builder { private long eqDeleteRecordCount = 0L; private int commitCount = 0; + private Builder() {} + public Builder dataFileCount(int newDataFileCount) { this.dataFileCount = newDataFileCount; return this; From e056212fa57cf645e8f2bbf624e971a6d949e8ee Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Wed, 4 Sep 2024 05:07:33 +0800 Subject: [PATCH 0663/1019] Docs: Fix Flink 1.20 support versions (#11065) --- 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 a3c63276bfdb..ad67ba53cda3 100644 --- a/site/docs/multi-engine-support.md +++ b/site/docs/multi-engine-support.md @@ -93,7 +93,7 @@ Users should continuously upgrade their Flink version to stay up-to-date. | 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.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 | {{ 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) | +| 1.20 | Maintained | 1.7.0 (to be released) | - | - | From 8e067ff602b3186ffc23353e389d73d3aed6c8ad Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 5 Sep 2024 08:19:36 +0200 Subject: [PATCH 0664/1019] Build: Enable more error-prone checks (#11078) Enable a number of useful error-prone checks. --- baseline.gradle | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/baseline.gradle b/baseline.gradle index be2dc1198e94..be6e267697f0 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -88,6 +88,9 @@ subprojects { // error-prone is slow, don't run on tests/generated-src/generated '-XepExcludedPaths:.*/(test|generated-src|generated)/.*', '-Xep:AnnotateFormatMethod:ERROR', + '-Xep:BadComparable:ERROR', + '-Xep:BadInstanceof:ERROR', + '-Xep:CatchFail:ERROR', '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', @@ -97,11 +100,20 @@ subprojects { '-Xep:DefaultLocale:ERROR', // subclasses are not equal '-Xep:EqualsGetClass:OFF', + '-Xep:EqualsUnsafeCast:ERROR', + '-Xep:EqualsUsingHashCode:ERROR', + '-Xep:ExtendsObject:ERROR', // specific to Palantir '-Xep:FinalClass:OFF', '-Xep:FormatStringAnnotation:ERROR', + '-Xep:GetClassOnEnum:ERROR', + '-Xep:HidingField:ERROR', + '-Xep:ImmutableSetForContains:ERROR', '-Xep:ImmutablesReferenceEquality:ERROR', + '-Xep:InconsistentCapitalization:ERROR', + '-Xep:InconsistentHashCode:ERROR', '-Xep:IntLongMath:ERROR', + '-Xep:JdkObsolete:ERROR', // prefer method references over lambdas '-Xep:LambdaMethodReference:ERROR', // enforce logging conventions @@ -111,8 +123,18 @@ subprojects { // Enforce missing override '-Xep:MissingOverride:ERROR', '-Xep:MissingSummary:ERROR', + '-Xep:ModifiedButNotUsed:ERROR', + '-Xep:MutablePublicArray:ERROR', + '-Xep:NarrowCalculation:ERROR', + '-Xep:NarrowingCompoundAssignment:ERROR', + '-Xep:NullOptional:ERROR', + '-Xep:NullableOptional:ERROR', + '-Xep:NullablePrimitive:ERROR', + '-Xep:ObjectEqualsForPrimitives:ERROR', // Enforce hashCode over hash '-Xep:ObjectsHashCodeUnnecessaryVarargs:ERROR', + '-Xep:OrphanedFormatString:ERROR', + '-Xep:Overrides:ERROR', // Triggers false-positives whenever relocated @VisibleForTesting is used '-Xep:PreferCommonAnnotations:OFF', // specific to Palantir From 7dbd29ee95453bfc5cb2a32af6f23b438d9b0d6e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 5 Sep 2024 21:38:03 +0530 Subject: [PATCH 0665/1019] open-api: Fix compile warnings for testFixtures (#11071) --- build.gradle | 2 ++ gradle/libs.versions.toml | 2 ++ 2 files changed, 4 insertions(+) diff --git a/build.gradle b/build.gradle index 6857a8443ef7..fab949bf06a6 100644 --- a/build.gradle +++ b/build.gradle @@ -981,6 +981,8 @@ project(':iceberg-open-api') { testFixturesImplementation libs.jetty.servlet testFixturesImplementation libs.jetty.server testFixturesImplementation libs.sqlite.jdbc + + testFixturesCompileOnly libs.apiguardian } test { diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 8b1feda17109..55edaf281489 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -24,6 +24,7 @@ activation = "1.1.1" aliyun-sdk-oss = "3.10.2" antlr = "4.9.3" aircompressor = "0.27" +apiguardian = "1.1.2" arrow = "15.0.2" avro = "1.12.0" assertj-core = "3.26.3" @@ -177,6 +178,7 @@ slf4j-simple = { module = "org.slf4j:slf4j-simple", version.ref = "slf4j" } snowflake-jdbc = { module = "net.snowflake:snowflake-jdbc", version.ref = "snowflake-jdbc" } # test libraries +apiguardian = { module = "org.apiguardian:apiguardian-api", version.ref = "apiguardian" } assertj-core = { module = "org.assertj:assertj-core", version.ref = "assertj-core" } awaitility = { module = "org.awaitility:awaitility", version.ref = "awaitility" } delta-spark = { module = "io.delta:delta-spark_2.12", version.ref = "delta-spark" } From ecb3e4eeb139df15bc33cdd982357cb8c2ff22ed Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 6 Sep 2024 01:15:49 +0800 Subject: [PATCH 0666/1019] Spark 3.3, 3.4: Parallelize reading files in migrate procedures (#11043) Back-port of https://github.com/apache/iceberg/pull/9274 Back-port of https://github.com/apache/iceberg/pull/10037 --- .../extensions/TestAddFilesProcedure.java | 22 ++ .../apache/iceberg/spark/SparkTableUtil.java | 220 ++++++++++++++++-- .../actions/MigrateTableSparkAction.java | 11 +- .../actions/SnapshotTableSparkAction.java | 11 +- .../spark/procedures/AddFilesProcedure.java | 54 ++++- .../procedures/MigrateTableProcedure.java | 17 +- .../spark/procedures/ProcedureInput.java | 12 + .../procedures/SnapshotTableProcedure.java | 9 +- .../spark/actions/TestMigrateTableAction.java | 70 ++++++ .../actions/TestSnapshotTableAction.java | 69 ++++++ .../extensions/TestAddFilesProcedure.java | 22 ++ .../apache/iceberg/spark/SparkTableUtil.java | 218 ++++++++++++++++- .../actions/MigrateTableSparkAction.java | 11 +- .../actions/SnapshotTableSparkAction.java | 11 +- .../spark/procedures/AddFilesProcedure.java | 54 ++++- .../procedures/MigrateTableProcedure.java | 10 +- .../spark/procedures/ProcedureInput.java | 12 + .../procedures/SnapshotTableProcedure.java | 9 +- .../spark/actions/TestMigrateTableAction.java | 70 ++++++ .../actions/TestSnapshotTableAction.java | 69 ++++++ 20 files changed, 920 insertions(+), 61 deletions(-) create mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java create mode 100644 spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 44ee2ebdb646..b24ebfcf7977 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -939,6 +939,28 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testAddFilesWithParallelism() { + createUnpartitionedHiveTable(); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertEquals("Procedure output must match", ImmutableList.of(row(2L, 1L)), result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * 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.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 af1e99df71d3..dfd0b58ffbee 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -291,7 +292,7 @@ public static List listPartition( PartitionSpec spec, SerializableConfiguration conf, MetricsConfig metricsConfig) { - return listPartition(partition, spec, conf, metricsConfig, null); + return listPartition(partition, spec, conf, metricsConfig, null, 1); } /** @@ -314,7 +315,8 @@ public static List listPartition( PartitionSpec spec, SerializableConfiguration conf, MetricsConfig metricsConfig, - NameMapping mapping) { + NameMapping mapping, + int parallelism) { return TableMigrationUtil.listPartition( partition.values, partition.uri, @@ -322,7 +324,26 @@ public static List listPartition( spec, conf.get(), metricsConfig, - mapping); + mapping, + parallelism); + } + + private static List listPartition( + SparkPartition partition, + PartitionSpec spec, + SerializableConfiguration conf, + MetricsConfig metricsConfig, + NameMapping mapping, + ExecutorService service) { + return TableMigrationUtil.listPartition( + partition.values, + partition.uri, + partition.format, + spec, + conf.get(), + metricsConfig, + mapping, + service); } private static SparkPartition toSparkPartition( @@ -419,6 +440,114 @@ public static void importSparkTable( String stagingDir, Map partitionFilter, boolean checkDuplicateFiles) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, partitionFilter, checkDuplicateFiles, 1); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param parallelism number of threads to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + ExecutorService service) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, service); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + partitionFilter, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + ExecutorService service) { SessionCatalog catalog = spark.sessionState().catalog(); String db = @@ -439,7 +568,7 @@ public static void importSparkTable( if (Objects.equal(spec, PartitionSpec.unpartitioned())) { importUnpartitionedSparkTable( - spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles); + spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, service); } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); @@ -447,7 +576,13 @@ public static void importSparkTable( targetTable.newAppend().commit(); } else { importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + spark, + sourceTablePartitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + service); } } } catch (AnalysisException e) { @@ -480,7 +615,8 @@ public static void importSparkTable( targetTable, stagingDir, Collections.emptyMap(), - checkDuplicateFiles); + checkDuplicateFiles, + 1); } /** @@ -497,14 +633,15 @@ public static void importSparkTable( public static void importSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) { importSparkTable( - spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false); + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, 1); } private static void importUnpartitionedSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + ExecutorService service) { try { CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); Option format = @@ -529,7 +666,8 @@ private static void importUnpartitionedSparkTable( spec, conf, metricsConfig, - nameMapping); + nameMapping, + service); if (checkDuplicateFiles) { Dataset importedFiles = @@ -577,9 +715,60 @@ public static void importSparkPartitions( PartitionSpec spec, String stagingDir, boolean checkDuplicateFiles) { + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, checkDuplicateFiles, 1); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + int parallelism) { + importSparkPartitions( + spark, + partitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + ExecutorService service) { Configuration conf = spark.sessionState().newHadoopConf(); SerializableConfiguration serializableConf = new SerializableConfiguration(conf); - int parallelism = + int listingParallelism = Math.min( partitions.size(), spark.sessionState().conf().parallelPartitionDiscoveryParallelism()); int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); @@ -589,7 +778,7 @@ public static void importSparkPartitions( nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD partitionRDD = sparkContext.parallelize(partitions, parallelism); + JavaRDD partitionRDD = sparkContext.parallelize(partitions, listingParallelism); Dataset partitionDS = spark.createDataset(partitionRDD.rdd(), Encoders.javaSerialization(SparkPartition.class)); @@ -599,7 +788,12 @@ public static void importSparkPartitions( (FlatMapFunction) sparkPartition -> listPartition( - sparkPartition, spec, serializableConf, metricsConfig, nameMapping) + sparkPartition, + spec, + serializableConf, + metricsConfig, + nameMapping, + service) .iterator(), Encoders.javaSerialization(DataFile.class)); @@ -672,7 +866,7 @@ public static void importSparkPartitions( Table targetTable, PartitionSpec spec, String stagingDir) { - importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false); + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false, 1); } public static List filterPartitions( diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java index fe8acf0157d3..0eb2a99f2f49 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; @@ -59,6 +60,7 @@ public class MigrateTableSparkAction extends BaseTableCreationSparkAction partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { return modifyIcebergTable( destIdent, table -> { @@ -153,9 +164,16 @@ private InternalRow[] importToIceberg( Path sourcePath = new Path(sourceIdent.name()); String format = sourceIdent.namespace()[0]; importFileTable( - table, sourcePath, format, partitionFilter, checkDuplicateFiles, table.spec()); + table, + sourcePath, + format, + partitionFilter, + checkDuplicateFiles, + table.spec(), + parallelism); } else { - importCatalogTable(table, sourceIdent, partitionFilter, checkDuplicateFiles); + importCatalogTable( + table, sourceIdent, partitionFilter, checkDuplicateFiles, parallelism); } Snapshot snapshot = table.currentSnapshot(); @@ -178,7 +196,8 @@ private void importFileTable( String format, Map partitionFilter, boolean checkDuplicateFiles, - PartitionSpec spec) { + PartitionSpec spec, + int parallelism) { // List Partitions via Spark InMemory file search interface List partitions = Spark3Util.getPartitions(spark(), tableLocation, format, partitionFilter, spec); @@ -193,11 +212,11 @@ private void importFileTable( // Build a Global Partition for the source SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format); - importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles); + importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles, parallelism); } else { Preconditions.checkArgument( !partitions.isEmpty(), "Cannot find any matching partitions in table %s", table.name()); - importPartitions(table, partitions, checkDuplicateFiles); + importPartitions(table, partitions, checkDuplicateFiles, parallelism); } } @@ -205,7 +224,8 @@ private void importCatalogTable( Table table, Identifier sourceIdent, Map partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent); SparkTableUtil.importSparkTable( @@ -214,14 +234,24 @@ private void importCatalogTable( table, stagingLocation, partitionFilter, - checkDuplicateFiles); + checkDuplicateFiles, + parallelism); } private void importPartitions( - Table table, List partitions, boolean checkDuplicateFiles) { + Table table, + List partitions, + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); SparkTableUtil.importSparkPartitions( - spark(), partitions, table, table.spec(), stagingLocation, checkDuplicateFiles); + spark(), + partitions, + table, + table.spec(), + stagingLocation, + checkDuplicateFiles, + parallelism); } private String getMetadataLocation(Table table) { diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java index aaa6d2cb238d..69e4ef20ea50 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java @@ -39,7 +39,8 @@ class MigrateTableProcedure extends BaseProcedure { new ProcedureParameter[] { ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("properties", STRING_MAP), - ProcedureParameter.optional("drop_backup", DataTypes.BooleanType) + ProcedureParameter.optional("drop_backup", DataTypes.BooleanType), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -95,13 +96,19 @@ public InternalRow[] call(InternalRow args) { MigrateTableSparkAction migrateTableSparkAction = SparkActions.get().migrateTable(tableName).tableProperties(properties); - MigrateTable.Result result; if (dropBackup) { - result = migrateTableSparkAction.dropBackup().execute(); - } else { - result = migrateTableSparkAction.execute(); + migrateTableSparkAction = migrateTableSparkAction.dropBackup(); } + if (!args.isNullAt(3)) { + int parallelism = args.getInt(3); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + migrateTableSparkAction = + migrateTableSparkAction.executeWith(executorService(parallelism, "table-migration")); + } + + MigrateTable.Result result = migrateTableSparkAction.execute(); + return new InternalRow[] {newInternalRow(result.migratedDataFilesCount())}; } diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java index 42e4d8ba0603..0be4b38de79c 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java @@ -68,6 +68,18 @@ public Boolean asBoolean(ProcedureParameter param, Boolean defaultValue) { return args.isNullAt(ordinal) ? defaultValue : (Boolean) args.getBoolean(ordinal); } + public Integer asInt(ProcedureParameter param) { + Integer value = asInt(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Integer asInt(ProcedureParameter param, Integer defaultValue) { + validateParamType(param, DataTypes.IntegerType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Integer) args.getInt(ordinal); + } + public long asLong(ProcedureParameter param) { Long value = asLong(param, null); Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); diff --git a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java index 7a015a51e8ed..f709f64ebf62 100644 --- a/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java +++ b/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java @@ -38,7 +38,8 @@ class SnapshotTableProcedure extends BaseProcedure { ProcedureParameter.required("source_table", DataTypes.StringType), ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("location", DataTypes.StringType), - ProcedureParameter.optional("properties", STRING_MAP) + ProcedureParameter.optional("properties", STRING_MAP), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -102,6 +103,12 @@ public InternalRow[] call(InternalRow args) { action.tableLocation(snapshotLocation); } + 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")); + } + SnapshotTable.Result result = action.tableProperties(properties).execute(); return new InternalRow[] {newInternalRow(result.importedDataFilesCount())}; } diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java new file mode 100644 index 000000000000..7bed72b7cc2c --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java @@ -0,0 +1,70 @@ +/* + * 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.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestMigrateTableAction extends SparkCatalogTestBase { + + public TestMigrateTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @Test + public void testMigrateWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + AtomicInteger migrationThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .migrateTable(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-migration-" + migrationThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(migrationThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java new file mode 100644 index 000000000000..8e6358f51bcd --- /dev/null +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.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.actions; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestSnapshotTableAction extends SparkCatalogTestBase { + private static final String SOURCE_NAME = "spark_catalog.default.source"; + + public TestSnapshotTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); + } + + @Test + public void testSnapshotWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-snapshot-" + snapshotThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(snapshotThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java index 5f995a7776c3..ae2062691d77 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestAddFilesProcedure.java @@ -935,6 +935,28 @@ public void testPartitionedImportFromEmptyPartitionDoesNotThrow() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testAddFilesWithParallelism() { + createUnpartitionedHiveTable(); + + String createIceberg = + "CREATE TABLE %s (id Integer, name String, dept String, subdept String) USING iceberg"; + + sql(createIceberg, tableName); + + List result = + sql( + "CALL %s.system.add_files(table => '%s', source_table => '%s', parallelism => 2)", + catalogName, tableName, sourceTableName); + + assertEquals("Procedure output must match", ImmutableList.of(row(2L, 1L)), result); + + assertEquals( + "Iceberg table contains correct data", + sql("SELECT * FROM %s ORDER BY id", sourceTableName), + sql("SELECT * 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.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 6f57c7ae376c..7a96e97fb98a 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 @@ -28,6 +28,7 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.ExecutorService; import java.util.stream.Collectors; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.Path; @@ -277,7 +278,8 @@ private static List listPartition( PartitionSpec spec, SerializableConfiguration conf, MetricsConfig metricsConfig, - NameMapping mapping) { + NameMapping mapping, + int parallelism) { return TableMigrationUtil.listPartition( partition.values, partition.uri, @@ -285,7 +287,26 @@ private static List listPartition( spec, conf.get(), metricsConfig, - mapping); + mapping, + parallelism); + } + + private static List listPartition( + SparkPartition partition, + PartitionSpec spec, + SerializableConfiguration conf, + MetricsConfig metricsConfig, + NameMapping mapping, + ExecutorService service) { + return TableMigrationUtil.listPartition( + partition.values, + partition.uri, + partition.format, + spec, + conf.get(), + metricsConfig, + mapping, + service); } private static SparkPartition toSparkPartition( @@ -382,6 +403,114 @@ public static void importSparkTable( String stagingDir, Map partitionFilter, boolean checkDuplicateFiles) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, partitionFilter, checkDuplicateFiles, 1); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param parallelism number of threads to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + ExecutorService service) { + importSparkTable( + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, service); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + int parallelism) { + importSparkTable( + spark, + sourceTableIdent, + targetTable, + stagingDir, + partitionFilter, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from an existing Spark table to an Iceberg table. + * + *

    The import uses the Spark session to get table metadata. It assumes no operation is going on + * the original and target table and thus is not thread-safe. + * + * @param spark a Spark session + * @param sourceTableIdent an identifier of the source Spark table + * @param targetTable an Iceberg table where to import the data + * @param stagingDir a staging directory to store temporary manifest files + * @param partitionFilter only import partitions whose values match those in the map, can be + * partially defined + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkTable( + SparkSession spark, + TableIdentifier sourceTableIdent, + Table targetTable, + String stagingDir, + Map partitionFilter, + boolean checkDuplicateFiles, + ExecutorService service) { SessionCatalog catalog = spark.sessionState().catalog(); String db = @@ -402,7 +531,7 @@ public static void importSparkTable( if (Objects.equal(spec, PartitionSpec.unpartitioned())) { importUnpartitionedSparkTable( - spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles); + spark, sourceTableIdentWithDB, targetTable, checkDuplicateFiles, service); } else { List sourceTablePartitions = getPartitions(spark, sourceTableIdent, partitionFilter); @@ -410,7 +539,13 @@ public static void importSparkTable( targetTable.newAppend().commit(); } else { importSparkPartitions( - spark, sourceTablePartitions, targetTable, spec, stagingDir, checkDuplicateFiles); + spark, + sourceTablePartitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + service); } } } catch (AnalysisException e) { @@ -443,7 +578,8 @@ public static void importSparkTable( targetTable, stagingDir, Collections.emptyMap(), - checkDuplicateFiles); + checkDuplicateFiles, + 1); } /** @@ -460,14 +596,15 @@ public static void importSparkTable( public static void importSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, String stagingDir) { importSparkTable( - spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false); + spark, sourceTableIdent, targetTable, stagingDir, Collections.emptyMap(), false, 1); } private static void importUnpartitionedSparkTable( SparkSession spark, TableIdentifier sourceTableIdent, Table targetTable, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + ExecutorService service) { try { CatalogTable sourceTable = spark.sessionState().catalog().getTableMetadata(sourceTableIdent); Option format = @@ -492,7 +629,8 @@ private static void importUnpartitionedSparkTable( spec, conf, metricsConfig, - nameMapping); + nameMapping, + service); if (checkDuplicateFiles) { Dataset importedFiles = @@ -540,9 +678,60 @@ public static void importSparkPartitions( PartitionSpec spec, String stagingDir, boolean checkDuplicateFiles) { + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, checkDuplicateFiles, 1); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param parallelism number of threads to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + int parallelism) { + importSparkPartitions( + spark, + partitions, + targetTable, + spec, + stagingDir, + checkDuplicateFiles, + TableMigrationUtil.migrationService(parallelism)); + } + + /** + * Import files from given partitions to an Iceberg table. + * + * @param spark a Spark session + * @param partitions partitions to import + * @param targetTable an Iceberg table where to import the data + * @param spec a partition spec + * @param stagingDir a staging directory to store temporary manifest files + * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file + * @param service executor service to use for file reading + */ + public static void importSparkPartitions( + SparkSession spark, + List partitions, + Table targetTable, + PartitionSpec spec, + String stagingDir, + boolean checkDuplicateFiles, + ExecutorService service) { Configuration conf = spark.sessionState().newHadoopConf(); SerializableConfiguration serializableConf = new SerializableConfiguration(conf); - int parallelism = + int listingParallelism = Math.min( partitions.size(), spark.sessionState().conf().parallelPartitionDiscoveryParallelism()); int numShufflePartitions = spark.sessionState().conf().numShufflePartitions(); @@ -552,7 +741,7 @@ public static void importSparkPartitions( nameMappingString != null ? NameMappingParser.fromJson(nameMappingString) : null; JavaSparkContext sparkContext = JavaSparkContext.fromSparkContext(spark.sparkContext()); - JavaRDD partitionRDD = sparkContext.parallelize(partitions, parallelism); + JavaRDD partitionRDD = sparkContext.parallelize(partitions, listingParallelism); Dataset partitionDS = spark.createDataset(partitionRDD.rdd(), Encoders.javaSerialization(SparkPartition.class)); @@ -562,7 +751,12 @@ public static void importSparkPartitions( (FlatMapFunction) sparkPartition -> listPartition( - sparkPartition, spec, serializableConf, metricsConfig, nameMapping) + sparkPartition, + spec, + serializableConf, + metricsConfig, + nameMapping, + service) .iterator(), Encoders.javaSerialization(DataFile.class)); @@ -635,7 +829,7 @@ public static void importSparkPartitions( Table targetTable, PartitionSpec spec, String stagingDir) { - importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false); + importSparkPartitions(spark, partitions, targetTable, spec, stagingDir, false, 1); } public static List filterPartitions( diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java index 5f3cdd3f035c..bdffeb465405 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/MigrateTableSparkAction.java @@ -19,6 +19,7 @@ package org.apache.iceberg.spark.actions; import java.util.Map; +import java.util.concurrent.ExecutorService; import org.apache.iceberg.Snapshot; import org.apache.iceberg.SnapshotSummary; import org.apache.iceberg.Table; @@ -59,6 +60,7 @@ public class MigrateTableSparkAction extends BaseTableCreationSparkAction partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { return modifyIcebergTable( destIdent, table -> { @@ -153,9 +164,16 @@ private InternalRow[] importToIceberg( Path sourcePath = new Path(sourceIdent.name()); String format = sourceIdent.namespace()[0]; importFileTable( - table, sourcePath, format, partitionFilter, checkDuplicateFiles, table.spec()); + table, + sourcePath, + format, + partitionFilter, + checkDuplicateFiles, + table.spec(), + parallelism); } else { - importCatalogTable(table, sourceIdent, partitionFilter, checkDuplicateFiles); + importCatalogTable( + table, sourceIdent, partitionFilter, checkDuplicateFiles, parallelism); } Snapshot snapshot = table.currentSnapshot(); @@ -178,7 +196,8 @@ private void importFileTable( String format, Map partitionFilter, boolean checkDuplicateFiles, - PartitionSpec spec) { + PartitionSpec spec, + int parallelism) { // List Partitions via Spark InMemory file search interface List partitions = Spark3Util.getPartitions(spark(), tableLocation, format, partitionFilter, spec); @@ -193,11 +212,11 @@ private void importFileTable( // Build a Global Partition for the source SparkPartition partition = new SparkPartition(Collections.emptyMap(), tableLocation.toString(), format); - importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles); + importPartitions(table, ImmutableList.of(partition), checkDuplicateFiles, parallelism); } else { Preconditions.checkArgument( !partitions.isEmpty(), "Cannot find any matching partitions in table %s", table.name()); - importPartitions(table, partitions, checkDuplicateFiles); + importPartitions(table, partitions, checkDuplicateFiles, parallelism); } } @@ -205,7 +224,8 @@ private void importCatalogTable( Table table, Identifier sourceIdent, Map partitionFilter, - boolean checkDuplicateFiles) { + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); TableIdentifier sourceTableIdentifier = Spark3Util.toV1TableIdentifier(sourceIdent); SparkTableUtil.importSparkTable( @@ -214,14 +234,24 @@ private void importCatalogTable( table, stagingLocation, partitionFilter, - checkDuplicateFiles); + checkDuplicateFiles, + parallelism); } private void importPartitions( - Table table, List partitions, boolean checkDuplicateFiles) { + Table table, + List partitions, + boolean checkDuplicateFiles, + int parallelism) { String stagingLocation = getMetadataLocation(table); SparkTableUtil.importSparkPartitions( - spark(), partitions, table, table.spec(), stagingLocation, checkDuplicateFiles); + spark(), + partitions, + table, + table.spec(), + stagingLocation, + checkDuplicateFiles, + parallelism); } private String getMetadataLocation(Table table) { diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java index 37b1e3bf0195..a0bd04dd997e 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/MigrateTableProcedure.java @@ -40,7 +40,8 @@ class MigrateTableProcedure extends BaseProcedure { ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("properties", STRING_MAP), ProcedureParameter.optional("drop_backup", DataTypes.BooleanType), - ProcedureParameter.optional("backup_table_name", DataTypes.StringType) + ProcedureParameter.optional("backup_table_name", DataTypes.StringType), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -105,6 +106,13 @@ public InternalRow[] call(InternalRow args) { migrateTableSparkAction = migrateTableSparkAction.backupTableName(backupTableName); } + if (!args.isNullAt(4)) { + int parallelism = args.getInt(4); + Preconditions.checkArgument(parallelism > 0, "Parallelism should be larger than 0"); + migrateTableSparkAction = + migrateTableSparkAction.executeWith(executorService(parallelism, "table-migration")); + } + MigrateTable.Result result = migrateTableSparkAction.execute(); return new InternalRow[] {newInternalRow(result.migratedDataFilesCount())}; } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java index 42e4d8ba0603..0be4b38de79c 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/ProcedureInput.java @@ -68,6 +68,18 @@ public Boolean asBoolean(ProcedureParameter param, Boolean defaultValue) { return args.isNullAt(ordinal) ? defaultValue : (Boolean) args.getBoolean(ordinal); } + public Integer asInt(ProcedureParameter param) { + Integer value = asInt(param, null); + Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); + return value; + } + + public Integer asInt(ProcedureParameter param, Integer defaultValue) { + validateParamType(param, DataTypes.IntegerType); + int ordinal = ordinal(param); + return args.isNullAt(ordinal) ? defaultValue : (Integer) args.getInt(ordinal); + } + public long asLong(ProcedureParameter param) { Long value = asLong(param, null); Preconditions.checkArgument(value != null, "Parameter '%s' is not set", param.name()); diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java index 7a015a51e8ed..f709f64ebf62 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/procedures/SnapshotTableProcedure.java @@ -38,7 +38,8 @@ class SnapshotTableProcedure extends BaseProcedure { ProcedureParameter.required("source_table", DataTypes.StringType), ProcedureParameter.required("table", DataTypes.StringType), ProcedureParameter.optional("location", DataTypes.StringType), - ProcedureParameter.optional("properties", STRING_MAP) + ProcedureParameter.optional("properties", STRING_MAP), + ProcedureParameter.optional("parallelism", DataTypes.IntegerType) }; private static final StructType OUTPUT_TYPE = @@ -102,6 +103,12 @@ public InternalRow[] call(InternalRow args) { action.tableLocation(snapshotLocation); } + 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")); + } + SnapshotTable.Result result = action.tableProperties(properties).execute(); return new InternalRow[] {newInternalRow(result.importedDataFilesCount())}; } diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java new file mode 100644 index 000000000000..7bed72b7cc2c --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestMigrateTableAction.java @@ -0,0 +1,70 @@ +/* + * 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.assertj.core.api.Assumptions.assumeThat; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestMigrateTableAction extends SparkCatalogTestBase { + + public TestMigrateTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + assumeThat(catalogName).isEqualToIgnoringCase("spark_catalog"); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s_BACKUP_", tableName); + } + + @Test + public void testMigrateWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + tableName, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", tableName); + + AtomicInteger migrationThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .migrateTable(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-migration-" + migrationThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(migrationThreadsIndex.get(), 2); + } +} diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.java new file mode 100644 index 000000000000..8e6358f51bcd --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestSnapshotTableAction.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.actions; + +import java.io.IOException; +import java.util.Map; +import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.junit.After; +import org.junit.Assert; +import org.junit.Test; + +public class TestSnapshotTableAction extends SparkCatalogTestBase { + private static final String SOURCE_NAME = "spark_catalog.default.source"; + + public TestSnapshotTableAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @After + public void removeTables() { + sql("DROP TABLE IF EXISTS %s", tableName); + sql("DROP TABLE IF EXISTS %s PURGE", SOURCE_NAME); + } + + @Test + public void testSnapshotWithParallelTasks() throws IOException { + String location = temp.newFolder().toURI().toString(); + sql( + "CREATE TABLE %s (id bigint NOT NULL, data string) USING parquet LOCATION '%s'", + SOURCE_NAME, location); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", SOURCE_NAME); + sql("INSERT INTO TABLE %s VALUES (2, 'b')", SOURCE_NAME); + + AtomicInteger snapshotThreadsIndex = new AtomicInteger(0); + SparkActions.get() + .snapshotTable(SOURCE_NAME) + .as(tableName) + .executeWith( + Executors.newFixedThreadPool( + 4, + runnable -> { + Thread thread = new Thread(runnable); + thread.setName("table-snapshot-" + snapshotThreadsIndex.getAndIncrement()); + thread.setDaemon(true); + return thread; + })) + .execute(); + Assert.assertEquals(snapshotThreadsIndex.get(), 2); + } +} From 77a053f6a7440ab9ac9b98348dc1fd9bbcefa52d Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 5 Sep 2024 11:12:24 -0700 Subject: [PATCH 0667/1019] Spark 3.5: Mandate identifier fields when create_changelog_view for table contain unsortable columns (#11045) --- .../TestCreateChangelogViewProcedure.java | 14 ++++++++++++++ .../procedures/CreateChangelogViewProcedure.java | 15 ++++++++++++++- 2 files changed, 28 insertions(+), 1 deletion(-) diff --git a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java index bc60759bd038..3fd760c67c4a 100644 --- a/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java +++ b/spark/v3.5/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java @@ -436,4 +436,18 @@ public void testNetChangesWithComputeUpdates() { .isInstanceOf(IllegalArgumentException.class) .hasMessageContaining("Not support net changes with update images"); } + + @TestTemplate + public void testUpdateWithInComparableType() { + sql( + "CREATE TABLE %s (id INT NOT NULL, data MAP, age INT) USING iceberg", + tableName); + + assertThatThrownBy( + () -> + sql("CALL %s.system.create_changelog_view(table => '%s')", catalogName, tableName)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining( + "Identifier field is required as table contains unorderable columns: [data]"); + } } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java index b4594d91c0ef..ae77b69133f3 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java @@ -23,6 +23,7 @@ import java.util.Map; import java.util.Set; import java.util.function.Predicate; +import java.util.stream.Collectors; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Table; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -37,6 +38,7 @@ import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.OrderUtils; 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; @@ -146,10 +148,21 @@ public InternalRow[] call(InternalRow args) { Dataset df = loadRows(changelogTableIdent, options(input)); boolean netChanges = input.asBoolean(NET_CHANGES, false); + String[] identifierColumns = identifierColumns(input, tableIdent); + Set unorderableColumnNames = + Arrays.stream(df.schema().fields()) + .filter(field -> !OrderUtils.isOrderable(field.dataType())) + .map(StructField::name) + .collect(Collectors.toSet()); + + Preconditions.checkArgument( + identifierColumns.length > 0 || unorderableColumnNames.isEmpty(), + "Identifier field is required as table contains unorderable columns: %s", + unorderableColumnNames); if (shouldComputeUpdateImages(input)) { Preconditions.checkArgument(!netChanges, "Not support net changes with update images"); - df = computeUpdateImages(identifierColumns(input, tableIdent), df); + df = computeUpdateImages(identifierColumns, df); } else { df = removeCarryoverRows(df, netChanges); } From da593b91c30277cb746643659f46271efdc0f074 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 6 Sep 2024 21:29:47 -0700 Subject: [PATCH 0668/1019] Core: Fix setting hasNewDataFile flag in MergingSnapshotProducer (#11088) --- .../main/java/org/apache/iceberg/MergingSnapshotProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index b4c0567ab73a..d1eb5d89da41 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -984,11 +984,11 @@ private List newDataFilesAsManifests() { writer.close(); } this.cachedNewDataManifests.addAll(writer.toManifestFiles()); - this.hasNewDataFiles = false; } catch (IOException e) { throw new RuntimeIOException(e, "Failed to close manifest writer"); } }); + this.hasNewDataFiles = false; } return cachedNewDataManifests; From b1adfe2d057f9a9406ef1b79c2bc4fb36e532fc7 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 6 Sep 2024 21:33:07 -0700 Subject: [PATCH 0669/1019] Docs: Document accessing instance variables (#11087) --- site/docs/contribute.md | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) diff --git a/site/docs/contribute.md b/site/docs/contribute.md index beda3cfc507e..94b4679da78c 100644 --- a/site/docs/contribute.md +++ b/site/docs/contribute.md @@ -388,6 +388,34 @@ When passing boolean arguments to existing or external methods, use inline comme dropTable(identifier, purge); ``` +#### Accessing instance variables + +Use `this` when assigning values to instance variables, making it clear when the object's state is being changed. Omit `this` when reading instance variables to keep lines shorter. + +```java + private String value; + + // BAD: unnecessary `this` during reads + public String value() { + return this.value; + } + + // GOOD: no `this` when reading instance variables + public String value() { + return value; + } + + // BAD: missing `this` in assignments + public void value(String newValue) { + value = newValue; + } + + // GOOD: use `this` in assignments + public void value(String newValue) { + this.value = newValue; + } +``` + #### Config naming 1. Use `-` to link words in one concept From 8a91c09107944cbb643b6096ddd8ab506e51f994 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Sep 2024 07:53:58 +0200 Subject: [PATCH 0670/1019] Build: Bump jetty from 11.0.23 to 11.0.24 (#11096) Bumps `jetty` from 11.0.23 to 11.0.24. Updates `org.eclipse.jetty:jetty-server` from 11.0.23 to 11.0.24 Updates `org.eclipse.jetty:jetty-servlet` from 11.0.23 to 11.0.24 --- updated-dependencies: - dependency-name: org.eclipse.jetty:jetty-server dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.eclipse.jetty:jetty-servlet 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 55edaf281489..5f233aa5c3f1 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -61,7 +61,7 @@ jakarta-el-api = "3.0.3" jakarta-servlet-api = "6.1.0" jaxb-api = "2.3.1" jaxb-runtime = "2.3.9" -jetty = "11.0.23" +jetty = "11.0.24" junit = "5.10.1" junit-platform = "1.10.3" kafka = "3.8.0" From a4d6c60b92409ef6bc01c9b32a4906f7e13c1ca8 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 9 Sep 2024 12:28:26 +0530 Subject: [PATCH 0671/1019] Spec: Fix rendering of partition stats file section (#11068) --- format/spec.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/format/spec.md b/format/spec.md index 1297ec0365b2..dcddd80de42f 100644 --- a/format/spec.md +++ b/format/spec.md @@ -782,12 +782,14 @@ Note that partition data tuple's schema is based on the partition spec output us The unified partition type is a struct containing all fields that have ever been a part of any spec in the table and sorted by the field ids in ascending order. In other words, the struct fields represent a union of all known partition fields sorted in ascending order by the field ids. + For example, -1) `spec#0` has two fields `{field#1, field#2}` + +1. `spec#0` has two fields `{field#1, field#2}` and then the table has evolved into `spec#1` which has three fields `{field#1, field#2, field#3}`. The unified partition type looks like `Struct`. -2) `spec#0` has two fields `{field#1, field#2}` +2. `spec#0` has two fields `{field#1, field#2}` and then the table has evolved into `spec#1` which has just one field `{field#2}`. The unified partition type looks like `Struct`. From 151dfd85b9e08779f2d87fc03c9435a7daec26d6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Sep 2024 10:44:36 +0200 Subject: [PATCH 0672/1019] Build: Bump software.amazon.awssdk:bom from 2.27.12 to 2.27.21 (#11098) Bumps software.amazon.awssdk:bom from 2.27.12 to 2.27.21. --- 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 5f233aa5c3f1..768f3c8e1eb7 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.27.12" +awssdk-bom = "2.27.21" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 1bea409794ad99a2cc851c833d83d69937363e4f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Sep 2024 10:44:56 +0200 Subject: [PATCH 0673/1019] Build: Bump io.netty:netty-buffer from 4.1.112.Final to 4.1.113.Final (#11097) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.112.Final to 4.1.113.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.112.Final...netty-4.1.113.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 768f3c8e1eb7..c7e949fddcc2 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -70,8 +70,8 @@ microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.95.0" -netty-buffer = "4.1.112.Final" -netty-buffer-compat = "4.1.112.Final" +netty-buffer = "4.1.113.Final" +netty-buffer-compat = "4.1.113.Final" object-client-bundle = "3.3.2" orc = "1.9.4" parquet = "1.13.1" From c18138b6dc89f40de116a5ec4658958e99cb8cc3 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 9 Sep 2024 22:21:32 +0800 Subject: [PATCH 0674/1019] Spark 3.3, 3.4: Fix incorrect catalog loaded in TestCreateActions (#11049) Back-port of #10952 --- .../iceberg/spark/actions/TestCreateActions.java | 15 +++++++++++++++ .../iceberg/spark/actions/TestCreateActions.java | 15 +++++++++++++++ 2 files changed, 30 insertions(+) diff --git a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index 028d495b894d..06f118be0c6f 100644 --- a/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -183,6 +183,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @Test @@ -728,6 +733,8 @@ public void testStructOfThreeLevelLists() throws Exception { @Test public void testTwoLevelList() throws IOException { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -811,6 +818,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -833,6 +842,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -858,6 +869,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -885,6 +898,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); diff --git a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java index 028d495b894d..06f118be0c6f 100644 --- a/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java @@ -183,6 +183,11 @@ public void before() { public void after() throws IOException { // Drop the hive table. spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName)); + spark.sessionState().catalogManager().reset(); + spark.conf().unset("spark.sql.catalog.spark_catalog.type"); + spark.conf().unset("spark.sql.catalog.spark_catalog.default-namespace"); + spark.conf().unset("spark.sql.catalog.spark_catalog.parquet-enabled"); + spark.conf().unset("spark.sql.catalog.spark_catalog.cache-enabled"); } @Test @@ -728,6 +733,8 @@ public void testStructOfThreeLevelLists() throws Exception { @Test public void testTwoLevelList() throws IOException { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", true); String tableName = sourceName("testTwoLevelList"); @@ -811,6 +818,8 @@ public boolean accept(File dir, String name) { } private void threeLevelList(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelList_%s", useLegacyMode)); @@ -833,6 +842,8 @@ private void threeLevelList(boolean useLegacyMode) throws Exception { } private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = @@ -858,6 +869,8 @@ private void threeLevelListWithNestedStruct(boolean useLegacyMode) throws Except } private void threeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("threeLevelLists_%s", useLegacyMode)); @@ -885,6 +898,8 @@ private void threeLevelLists(boolean useLegacyMode) throws Exception { } private void structOfThreeLevelLists(boolean useLegacyMode) throws Exception { + Assume.assumeTrue("Cannot migrate to a hadoop based catalog", !type.equals("hadoop")); + spark.conf().set("spark.sql.parquet.writeLegacyFormat", useLegacyMode); String tableName = sourceName(String.format("structOfThreeLevelLists_%s", useLegacyMode)); From 41cc89ffaf0f628e9e7fd3a8380b716554ae53f9 Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 10 Sep 2024 00:03:05 +0800 Subject: [PATCH 0675/1019] Core: Prevent incremental file cleanup when expiring specified snapshots (#10983) --- .../org/apache/iceberg/RemoveSnapshots.java | 11 ++++++++ .../apache/iceberg/TestRemoveSnapshots.java | 25 +++++++++++++++++-- 2 files changed, 34 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java index fa6fcdf41442..7558ea7d8a3e 100644 --- a/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java +++ b/core/src/main/java/org/apache/iceberg/RemoveSnapshots.java @@ -84,6 +84,7 @@ public void accept(String file) { private ExecutorService deleteExecutorService = DEFAULT_DELETE_EXECUTOR_SERVICE; private ExecutorService planExecutorService = ThreadPools.getWorkerPool(); private Boolean incrementalCleanup; + private boolean specifiedSnapshotId = false; RemoveSnapshots(TableOperations ops) { this.ops = ops; @@ -116,6 +117,7 @@ public ExpireSnapshots cleanExpiredFiles(boolean clean) { public ExpireSnapshots expireSnapshotId(long expireSnapshotId) { LOG.info("Expiring snapshot with id: {}", expireSnapshotId); idsToRemove.add(expireSnapshotId); + specifiedSnapshotId = true; return this; } @@ -321,6 +323,15 @@ ExpireSnapshots withIncrementalCleanup(boolean useIncrementalCleanup) { private void cleanExpiredSnapshots() { TableMetadata current = ops.refresh(); + if (specifiedSnapshotId) { + if (incrementalCleanup != null && incrementalCleanup) { + throw new UnsupportedOperationException( + "Cannot clean files incrementally when snapshot IDs are specified"); + } + + incrementalCleanup = false; + } + if (incrementalCleanup == null) { incrementalCleanup = current.refs().size() == 1; } diff --git a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java index 6a47a24fb494..f95fe6191e43 100644 --- a/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java +++ b/core/src/test/java/org/apache/iceberg/TestRemoveSnapshots.java @@ -370,7 +370,7 @@ public void testRetainLastWithExpireById() { } // Retain last 3 snapshots, but explicitly remove the first snapshot - removeSnapshots(table).expireSnapshotId(firstSnapshotId).retainLast(3).commit(); + table.expireSnapshots().expireSnapshotId(firstSnapshotId).retainLast(3).commit(); assertThat(table.snapshots()).hasSize(2); assertThat(table.snapshot(firstSnapshotId)).isNull(); @@ -956,7 +956,8 @@ public void testWithExpiringStagedThenCherrypick() { List deletedFiles = Lists.newArrayList(); // Expire `B` commit. - removeSnapshots(table) + table + .expireSnapshots() .deleteWith(deletedFiles::add) .expireSnapshotId(snapshotB.snapshotId()) .commit(); @@ -1171,6 +1172,26 @@ public void testBranchExpiration() { assertThat(table.ops().current().ref(SnapshotRef.MAIN_BRANCH)).isNotNull(); } + @TestTemplate + public void testIncrementalCleanupFailsWhenExpiringSnapshotId() { + table.newAppend().appendFile(FILE_A).commit(); + table.newDelete().deleteFile(FILE_A).commit(); + long snapshotId = table.currentSnapshot().snapshotId(); + table.newAppend().appendFile(FILE_B).commit(); + waitUntilAfter(table.currentSnapshot().timestampMillis()); + RemoveSnapshots removeSnapshots = (RemoveSnapshots) table.expireSnapshots(); + + assertThatThrownBy( + () -> + removeSnapshots + .withIncrementalCleanup(true) + .expireSnapshotId(snapshotId) + .cleanExpiredFiles(true) + .commit()) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Cannot clean files incrementally when snapshot IDs are specified"); + } + @TestTemplate public void testMultipleRefsAndCleanExpiredFilesFailsForIncrementalCleanup() { table.newAppend().appendFile(FILE_A).commit(); From 71b1aedd14c671527aeee20e9d04a6293c3b1687 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 10 Sep 2024 08:05:11 +0200 Subject: [PATCH 0676/1019] Core, Kafka, Spark: Use AssertJ instead of JUnit assertions (#11102) --- .baseline/checkstyle/checkstyle.xml | 5 + .../org/apache/iceberg/TestTableMetadata.java | 25 ++-- .../connect/channel/CoordinatorTest.java | 46 +++--- .../actions/TestComputeTableStatsAction.java | 139 +++++++----------- 4 files changed, 95 insertions(+), 120 deletions(-) diff --git a/.baseline/checkstyle/checkstyle.xml b/.baseline/checkstyle/checkstyle.xml index 570fb84d70a4..30dfabba8690 100644 --- a/.baseline/checkstyle/checkstyle.xml +++ b/.baseline/checkstyle/checkstyle.xml @@ -427,6 +427,11 @@ + + + + + diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index 5be8187a923e..bcde7a7f31d3 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -64,7 +64,6 @@ import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.JsonUtil; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; import org.junit.jupiter.params.ParameterizedTest; @@ -1662,19 +1661,19 @@ public void testV3TimestampNanoTypeSupport() { 6, "ts_nanos", Types.TimestampNanoType.withZone())))); for (int unsupportedFormatVersion : ImmutableList.of(1, 2)) { - Assertions.assertThrows( - IllegalStateException.class, - () -> - TableMetadata.newTableMetadata( - v3Schema, - PartitionSpec.unpartitioned(), - SortOrder.unsorted(), - TEST_LOCATION, - ImmutableMap.of(), - unsupportedFormatVersion), - String.format( + assertThatThrownBy( + () -> + TableMetadata.newTableMetadata( + v3Schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + TEST_LOCATION, + ImmutableMap.of(), + unsupportedFormatVersion)) + .isInstanceOf(IllegalStateException.class) + .hasMessage( "Invalid type in v%s schema: struct.ts_nanos timestamptz_ns is not supported until v3", - unsupportedFormatVersion)); + unsupportedFormatVersion); } // should be allowed in v3 diff --git a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java index 9c0b8122ae42..f3d0cf3dcf55 100644 --- a/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java +++ b/kafka-connect/kafka-connect/src/test/java/org/apache/iceberg/connect/channel/CoordinatorTest.java @@ -24,7 +24,6 @@ import java.time.OffsetDateTime; import java.util.List; -import java.util.Map; import java.util.UUID; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -47,14 +46,13 @@ import org.apache.iceberg.types.Types.StructType; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.connect.sink.SinkTaskContext; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; public class CoordinatorTest extends ChannelTestBase { @Test public void testCommitAppend() { - Assertions.assertEquals(0, ImmutableList.copyOf(table.snapshots().iterator()).size()); + assertThat(table.snapshots()).isEmpty(); OffsetDateTime ts = EventTestUtil.now(); UUID commitId = @@ -66,17 +64,17 @@ public void testCommitAppend() { assertCommitComplete(2, commitId, ts); List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(1, snapshots.size()); + assertThat(snapshots).hasSize(1); Snapshot snapshot = snapshots.get(0); - Assertions.assertEquals(DataOperations.APPEND, snapshot.operation()); - Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); - Assertions.assertEquals(0, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); - - Map summary = snapshot.summary(); - Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); - Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); - Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + assertThat(snapshot.operation()).isEqualTo(DataOperations.APPEND); + assertThat(snapshot.addedDataFiles(table.io())).hasSize(1); + assertThat(snapshot.addedDeleteFiles(table.io())).isEmpty(); + + assertThat(snapshot.summary()) + .containsEntry(COMMIT_ID_SNAPSHOT_PROP, commitId.toString()) + .containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":3}") + .containsEntry(VALID_THROUGH_TS_SNAPSHOT_PROP, ts.toString()); } @Test @@ -93,17 +91,17 @@ public void testCommitDelta() { assertCommitComplete(2, commitId, ts); List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(1, snapshots.size()); + assertThat(snapshots).hasSize(1); Snapshot snapshot = snapshots.get(0); - Assertions.assertEquals(DataOperations.OVERWRITE, snapshot.operation()); - Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDataFiles(table.io())).size()); - Assertions.assertEquals(1, ImmutableList.copyOf(snapshot.addedDeleteFiles(table.io())).size()); - - Map summary = snapshot.summary(); - Assertions.assertEquals(commitId.toString(), summary.get(COMMIT_ID_SNAPSHOT_PROP)); - Assertions.assertEquals("{\"0\":3}", summary.get(OFFSETS_SNAPSHOT_PROP)); - Assertions.assertEquals(ts.toString(), summary.get(VALID_THROUGH_TS_SNAPSHOT_PROP)); + assertThat(snapshot.operation()).isEqualTo(DataOperations.OVERWRITE); + assertThat(snapshot.addedDataFiles(table.io())).hasSize(1); + assertThat(snapshot.addedDeleteFiles(table.io())).hasSize(1); + + assertThat(snapshot.summary()) + .containsEntry(COMMIT_ID_SNAPSHOT_PROP, commitId.toString()) + .containsEntry(OFFSETS_SNAPSHOT_PROP, "{\"0\":3}") + .containsEntry(VALID_THROUGH_TS_SNAPSHOT_PROP, ts.toString()); } @Test @@ -114,8 +112,7 @@ public void testCommitNoFiles() { assertThat(producer.history()).hasSize(2); assertCommitComplete(1, commitId, ts); - List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(0, snapshots.size()); + assertThat(table.snapshots()).isEmpty(); } @Test @@ -136,8 +133,7 @@ public void testCommitError() { // no commit messages sent assertThat(producer.history()).hasSize(1); - List snapshots = ImmutableList.copyOf(table.snapshots()); - Assertions.assertEquals(0, snapshots.size()); + assertThat(table.snapshots()).isEmpty(); } private void assertCommitTable(int idx, UUID commitId, OffsetDateTime ts) { 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 4cab0d42815c..7aa849d0bba8 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 @@ -21,11 +21,9 @@ import static org.apache.iceberg.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; import static org.apache.iceberg.types.Types.NestedField.optional; import static org.apache.iceberg.types.Types.NestedField.required; -import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; -import static org.junit.jupiter.api.Assertions.assertNotEquals; -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; +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 java.io.IOException; import java.util.List; @@ -58,7 +56,6 @@ import org.apache.spark.sql.catalyst.parser.ParseException; import org.apache.spark.sql.types.StructType; import org.junit.jupiter.api.AfterEach; -import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.TestTemplate; public class TestComputeTableStatsAction extends CatalogTestBase { @@ -89,8 +86,8 @@ public void testLoadingTableDirectly() { SparkActions actions = SparkActions.get(); ComputeTableStats.Result results = actions.computeTableStats(table).execute(); StatisticsFile statisticsFile = results.statisticsFile(); - Assertions.assertNotEquals(statisticsFile.fileSizeInBytes(), 0); - Assertions.assertEquals(statisticsFile.blobMetadata().size(), 2); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); } @TestTemplate @@ -115,19 +112,18 @@ public void testComputeTableStatsAction() throws NoSuchTableException, ParseExce SparkActions actions = SparkActions.get(); ComputeTableStats.Result results = actions.computeTableStats(table).columns("id", "data").execute(); - assertNotNull(results); + assertThat(results).isNotNull(); List statisticsFiles = table.statisticsFiles(); - Assertions.assertEquals(statisticsFiles.size(), 1); + assertThat(statisticsFiles).hasSize(1); StatisticsFile statisticsFile = statisticsFiles.get(0); - assertNotEquals(statisticsFile.fileSizeInBytes(), 0); - Assertions.assertEquals(statisticsFile.blobMetadata().size(), 2); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); - Assertions.assertEquals( - blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), - String.valueOf(4)); + assertThat(blobMetadata.properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); } @TestTemplate @@ -149,28 +145,16 @@ public void testComputeTableStatsActionWithoutExplicitColumns() Table table = Spark3Util.loadIcebergTable(spark, tableName); SparkActions actions = SparkActions.get(); ComputeTableStats.Result results = actions.computeTableStats(table).execute(); - assertNotNull(results); + assertThat(results).isNotNull(); - Assertions.assertEquals(1, table.statisticsFiles().size()); + assertThat(table.statisticsFiles()).hasSize(1); StatisticsFile statisticsFile = table.statisticsFiles().get(0); - Assertions.assertEquals(2, statisticsFile.blobMetadata().size()); - assertNotEquals(0, statisticsFile.fileSizeInBytes()); - Assertions.assertEquals( - 4, - Long.parseLong( - statisticsFile - .blobMetadata() - .get(0) - .properties() - .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))); - Assertions.assertEquals( - 4, - Long.parseLong( - statisticsFile - .blobMetadata() - .get(1) - .properties() - .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(2); + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); + assertThat(statisticsFile.blobMetadata().get(1).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); } @TestTemplate @@ -180,12 +164,9 @@ public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException sql("INSERT into %s values(1, 'abcd')", tableName); Table table = Spark3Util.loadIcebergTable(spark, tableName); SparkActions actions = SparkActions.get(); - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, - () -> actions.computeTableStats(table).columns("id1").execute()); - String message = exception.getMessage(); - assertTrue(message.contains("Can't find column id1 in table")); + assertThatThrownBy(() -> actions.computeTableStats(table).columns("id1").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can't find column id1 in table"); } @TestTemplate @@ -194,7 +175,7 @@ public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, Table table = Spark3Util.loadIcebergTable(spark, tableName); SparkActions actions = SparkActions.get(); ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); - Assertions.assertNull(result.statisticsFile()); + assertThat(result.statisticsFile()).isNull(); } @TestTemplate @@ -215,19 +196,17 @@ public void testComputeTableStatsWithNullValues() throws NoSuchTableException, P Table table = Spark3Util.loadIcebergTable(spark, tableName); SparkActions actions = SparkActions.get(); ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); - assertNotNull(results); + assertThat(results).isNotNull(); List statisticsFiles = table.statisticsFiles(); - Assertions.assertEquals(statisticsFiles.size(), 1); + assertThat(statisticsFiles).hasSize(1); StatisticsFile statisticsFile = statisticsFiles.get(0); - assertNotEquals(statisticsFile.fileSizeInBytes(), 0); - Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); - BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); - Assertions.assertEquals( - blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), - String.valueOf(4)); + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "4"); } @TestTemplate @@ -241,7 +220,8 @@ public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() // Snapshot id not specified Table table = Spark3Util.loadIcebergTable(spark, tableName); - assertDoesNotThrow(() -> actions.computeTableStats(table).columns("data").execute()); + assertThatNoException() + .isThrownBy(() -> actions.computeTableStats(table).columns("data").execute()); sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); // Append data to create snapshot @@ -250,15 +230,14 @@ public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); // Snapshot id specified - assertDoesNotThrow( - () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()); - - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, - () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()); - String message = exception.getMessage(); - assertTrue(message.contains("Can't find column data in table")); + assertThatNoException() + .isThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()); + + assertThatThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageStartingWith("Can't find column data in table"); } @TestTemplate @@ -271,19 +250,17 @@ public void testComputeTableStatsWhenSnapshotIdNotSpecified() SparkActions actions = SparkActions.get(); ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); - assertNotNull(results); + assertThat(results).isNotNull(); List statisticsFiles = table.statisticsFiles(); - Assertions.assertEquals(statisticsFiles.size(), 1); + assertThat(statisticsFiles).hasSize(1); StatisticsFile statisticsFile = statisticsFiles.get(0); - assertNotEquals(statisticsFile.fileSizeInBytes(), 0); - Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); - BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); - Assertions.assertEquals( - blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY), - String.valueOf(1)); + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsEntry(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, "1"); } @TestTemplate @@ -305,10 +282,10 @@ public void testComputeTableStatsWithNestedSchema() tbl.refresh(); List statisticsFiles = tbl.statisticsFiles(); - Assertions.assertEquals(statisticsFiles.size(), 1); + assertThat(statisticsFiles).hasSize(1); StatisticsFile statisticsFile = statisticsFiles.get(0); - assertNotEquals(statisticsFile.fileSizeInBytes(), 0); - Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); } @TestTemplate @@ -322,10 +299,9 @@ public void testComputeTableStatsWithNoComputableColumns() throws IOException { table.refresh(); SparkActions actions = SparkActions.get(); - IllegalArgumentException exception = - assertThrows( - IllegalArgumentException.class, () -> actions.computeTableStats(table).execute()); - Assertions.assertEquals(exception.getMessage(), "No columns found to compute stats"); + assertThatThrownBy(() -> actions.computeTableStats(table).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("No columns found to compute stats"); } @TestTemplate @@ -386,18 +362,17 @@ public void testComputeTableStats(String columnName, String type) table.refresh(); ComputeTableStats.Result results = actions.computeTableStats(table).columns(columnName).execute(); - assertNotNull(results); + assertThat(results).isNotNull(); List statisticsFiles = table.statisticsFiles(); - Assertions.assertEquals(statisticsFiles.size(), 1); + assertThat(statisticsFiles).hasSize(1); StatisticsFile statisticsFile = statisticsFiles.get(0); - assertNotEquals(statisticsFile.fileSizeInBytes(), 0); - Assertions.assertEquals(statisticsFile.blobMetadata().size(), 1); + assertThat(statisticsFile.fileSizeInBytes()).isGreaterThan(0); + assertThat(statisticsFile.blobMetadata()).hasSize(1); - BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); - Assertions.assertNotNull( - blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)); + assertThat(statisticsFile.blobMetadata().get(0).properties()) + .containsKey(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY); } private GenericRecord createNestedRecord() { From 1b0abc3461263e904eb37b818e060080805c3d97 Mon Sep 17 00:00:00 2001 From: Robin Moffatt Date: Tue, 10 Sep 2024 10:13:06 +0100 Subject: [PATCH 0677/1019] Docs: Add blogs written by rmoff (#11069) --- site/docs/blogs.md | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/site/docs/blogs.md b/site/docs/blogs.md index 00b0688e86a0..819d4f8b9e92 100644 --- a/site/docs/blogs.md +++ b/site/docs/blogs.md @@ -23,6 +23,18 @@ title: "Blogs" Here is a list of company blogs that talk about Iceberg. The blogs are ordered from most recent to oldest. + +### [Sending Data to Apache Iceberg from Apache Kafka with Apache Flink](https://www.decodable.co/blog/kafka-to-iceberg-with-flink) +**Date**: July 18th, 2024, **Company**: Decodable + +**Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) + + +### [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 + +**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) **Date**: April 1st, 2024, **Company**: Dremio @@ -149,6 +161,12 @@ Here is a list of company blogs that talk about Iceberg. The blogs are ordered f **Authors**: [Riza Suminto](https://www.linkedin.com/in/rizasuminto/) + +### [lakeFS ♥️ Apache Iceberg](https://lakefs.io/blog/using-lakefs-with-apache-iceberg/) +**Date**: June 26th, 2023, **Company**: LakeFS + +**Author**: [Robin Moffatt](https://www.linkedin.com/in/robinmoffatt) + ### [How Bilibili Builds OLAP Data Lakehouse with Apache Iceberg](https://medium.com/@lirui.fudan/how-bilibili-builds-olap-data-lakehouse-with-apache-iceberg-9f3408e53f9) **Date**: June 14th, 2023, **Company**: Bilibili From b4c525e27f2ccdd70fc1e892067a57a2b036dee2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Tue, 10 Sep 2024 11:13:39 +0200 Subject: [PATCH 0678/1019] Build: Upgrade to Gradle 8.10.1 (#11104) --- 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 2b189974c29a..8e876e1c5571 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=5b9c5eb3f9fc2c94abaea57d90bd78747ca117ddbbf96c859d3741181a12bf2a -distributionUrl=https\://services.gradle.org/distributions/gradle-8.10-bin.zip +distributionSha256Sum=1541fa36599e12857140465f3c91a97409b4512501c26f9631fb113e392c5bd1 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.10.1-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 64084f24de2c..85a01a52e645 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.0/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.10.1/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From 2d35932944d4dbdf028de9b6eb626bd0e2c10e67 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 10 Sep 2024 11:59:26 +0200 Subject: [PATCH 0679/1019] Build: Remove unused variables, fields and parameters (#11101) * Remove unused variables, fields and parameters --- .../org/apache/iceberg/expressions/ExpressionUtil.java | 1 - .../main/java/org/apache/iceberg/types/TypeUtil.java | 1 - .../iceberg/aws/TestAssumeRoleAwsClientFactory.java | 4 ---- .../java/org/apache/iceberg/aws/glue/GlueTestBase.java | 4 ---- .../apache/iceberg/aws/glue/TestGlueCatalogTable.java | 2 -- .../aws/lakeformation/LakeFormationTestBase.java | 8 +++----- .../TestLakeFormationAwsClientFactory.java | 1 - .../apache/iceberg/aws/s3/TestS3FileIOIntegration.java | 1 - .../java/org/apache/iceberg/aws/AwsProperties.java | 5 ----- baseline.gradle | 1 + .../java/org/apache/iceberg/ManifestReadBenchmark.java | 3 +-- .../java/org/apache/iceberg/UpdateRequirements.java | 10 +++++----- .../org/apache/iceberg/actions/RewriteFileGroup.java | 2 +- .../iceberg/actions/RewritePositionDeletesGroup.java | 2 +- .../java/org/apache/iceberg/rest/auth/OAuth2Util.java | 2 -- .../java/org/apache/iceberg/util/ParallelIterable.java | 4 +--- .../org/apache/iceberg/flink/data/StructRowData.java | 4 ++-- .../apache/iceberg/flink/source/RowDataRewriter.java | 6 ------ .../org/apache/iceberg/flink/data/StructRowData.java | 4 ++-- .../apache/iceberg/flink/source/RowDataRewriter.java | 6 ------ .../org/apache/iceberg/flink/data/StructRowData.java | 4 ++-- .../apache/iceberg/flink/source/RowDataRewriter.java | 6 ------ mr/src/main/java/org/apache/iceberg/mr/Catalogs.java | 10 +++------- .../apache/iceberg/mr/hive/HiveIcebergMetaHook.java | 4 ++-- .../java/org/apache/iceberg/orc/OrcFileAppender.java | 3 +++ .../java/org/apache/iceberg/spark/SmokeTest.java | 2 +- .../java/org/apache/iceberg/spark/SmokeTest.java | 2 +- .../java/org/apache/iceberg/spark/SmokeTest.java | 2 +- 28 files changed, 30 insertions(+), 74 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 bd0adb228bd3..43c97c50df46 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -44,7 +44,6 @@ public class ExpressionUtil { Transforms.bucket(Integer.MAX_VALUE).bind(Types.StringType.get()); private static final OffsetDateTime EPOCH = Instant.ofEpochSecond(0).atOffset(ZoneOffset.UTC); private static final long FIVE_MINUTES_IN_MICROS = TimeUnit.MINUTES.toMicros(5); - private static final long FIVE_MINUTES_IN_NANOS = TimeUnit.MINUTES.toNanos(5); private static final long THREE_DAYS_IN_HOURS = TimeUnit.DAYS.toHours(3); private static final long NINETY_DAYS_IN_HOURS = TimeUnit.DAYS.toHours(90); private static final Pattern DATE = Pattern.compile("\\d{4}-\\d{2}-\\d{2}"); 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 9d4b217f1595..8a9184569aec 100644 --- a/api/src/main/java/org/apache/iceberg/types/TypeUtil.java +++ b/api/src/main/java/org/apache/iceberg/types/TypeUtil.java @@ -197,7 +197,6 @@ public static Map indexByLowerCaseName(Types.StructType struct) IndexByName indexer = new IndexByName(); visit(struct, indexer); - Map byName = indexer.byName(); Map byId = indexer.byId(); indexByName(struct) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java index 9845d31021d8..fc6f2da9c68f 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/TestAssumeRoleAwsClientFactory.java @@ -34,8 +34,6 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.core.exception.SdkServiceException; import software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient; import software.amazon.awssdk.regions.Region; @@ -51,8 +49,6 @@ public class TestAssumeRoleAwsClientFactory { - private static final Logger LOG = LoggerFactory.getLogger(TestAssumeRoleAwsClientFactory.class); - private IamClient iam; private String roleName; private Map assumeRoleProperties; 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 ecf589d7c07a..495c5ca98e94 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 @@ -38,8 +38,6 @@ import org.apache.iceberg.types.Types; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.services.glue.GlueClient; import software.amazon.awssdk.services.glue.model.Column; import software.amazon.awssdk.services.glue.model.GetTableRequest; @@ -52,8 +50,6 @@ @SuppressWarnings({"VisibilityModifier", "HideUtilityClassConstructor"}) public class GlueTestBase { - private static final Logger LOG = LoggerFactory.getLogger(GlueTestBase.class); - // the integration test requires the following env variables static final String TEST_BUCKET_NAME = AwsIntegTestUtil.testBucketName(); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java index 6bd6a4ad383d..6ca2fe021612 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogTable.java @@ -333,8 +333,6 @@ public void testRenameTableFailsToCreateNewTable() { public void testRenameTableFailsToDeleteOldTable() { String namespace = createNamespace(); String tableName = createTable(namespace); - TableIdentifier id = TableIdentifier.of(namespace, tableName); - Table table = glueCatalog.loadTable(id); // delete the old table metadata, so that drop old table will fail String newTableName = tableName + "_2"; GLUE.updateTable( diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java index 630db7a5a1b4..5e34c200cf5e 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/LakeFormationTestBase.java @@ -78,7 +78,6 @@ import software.amazon.awssdk.services.lakeformation.model.GrantPermissionsRequest; import software.amazon.awssdk.services.lakeformation.model.Permission; import software.amazon.awssdk.services.lakeformation.model.PutDataLakeSettingsRequest; -import software.amazon.awssdk.services.lakeformation.model.PutDataLakeSettingsResponse; import software.amazon.awssdk.services.lakeformation.model.RegisterResourceRequest; import software.amazon.awssdk.services.lakeformation.model.Resource; import software.amazon.awssdk.services.lakeformation.model.TableResource; @@ -217,10 +216,9 @@ public static void beforeClass() throws Exception { // put lf data lake settings GetDataLakeSettingsResponse getDataLakeSettingsResponse = lakeformation.getDataLakeSettings(GetDataLakeSettingsRequest.builder().build()); - PutDataLakeSettingsResponse putDataLakeSettingsResponse = - lakeformation.putDataLakeSettings( - putDataLakeSettingsRequest( - lfRegisterPathRoleArn, getDataLakeSettingsResponse.dataLakeSettings(), true)); + lakeformation.putDataLakeSettings( + putDataLakeSettingsRequest( + lfRegisterPathRoleArn, getDataLakeSettingsResponse.dataLakeSettings(), true)); // Build test glueCatalog with lfPrivilegedRole glueCatalogPrivilegedRole = new GlueCatalog(); diff --git a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java index 609dbf1a6acc..8a3bb71ce362 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/lakeformation/TestLakeFormationAwsClientFactory.java @@ -52,7 +52,6 @@ public class TestLakeFormationAwsClientFactory { private static final Logger LOG = LoggerFactory.getLogger(TestLakeFormationAwsClientFactory.class); - private static final int IAM_PROPAGATION_DELAY = 10000; private static final int ASSUME_ROLE_SESSION_DURATION = 3600; private IamClient iam; 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 fc2b9f97f1e6..b8b86fddc596 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 @@ -76,7 +76,6 @@ public class TestS3FileIOIntegration { private static S3Client s3; private static S3ControlClient s3Control; private static S3ControlClient crossRegionS3Control; - private static S3ControlClient multiRegionS3Control; private static KmsClient kms; private static String bucketName; private static String crossRegionBucketName; diff --git a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java index 5c3afc28a98b..1a8db990578a 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsProperties.java @@ -31,8 +31,6 @@ import org.apache.iceberg.relocated.com.google.common.base.Strings; import org.apache.iceberg.relocated.com.google.common.collect.Sets; import org.apache.iceberg.util.PropertyUtil; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider; import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; @@ -46,8 +44,6 @@ public class AwsProperties implements Serializable { - private static final Logger LOG = LoggerFactory.getLogger(AwsProperties.class); - /** * The ID of the Glue Data Catalog where the tables reside. If none is provided, Glue * automatically uses the caller's AWS account ID by default. @@ -210,7 +206,6 @@ public class AwsProperties implements Serializable { */ public static final String REST_SESSION_TOKEN = "rest.session-token"; - private static final String HTTP_CLIENT_PREFIX = "http-client."; private final Set stsClientAssumeRoleTags; private final String clientAssumeRoleArn; diff --git a/baseline.gradle b/baseline.gradle index be6e267697f0..5e2710be1749 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -157,6 +157,7 @@ subprojects { // Palantir's UnnecessarilyQualified may throw during analysis '-Xep:UnnecessarilyQualified:OFF', '-Xep:UnusedMethod:ERROR', + '-Xep:UnusedVariable:ERROR', ) } } diff --git a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java index e4a63beceaa6..7b4e2b9ec523 100644 --- a/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/ManifestReadBenchmark.java @@ -122,12 +122,11 @@ public void readManifestFile() throws IOException { TestTables.LocalFileIO fileIO = new TestTables.LocalFileIO(); Map specs = ImmutableMap.of(PartitionSpec.unpartitioned().specId(), PartitionSpec.unpartitioned()); - long recordCount = 0L; for (ManifestFile manifestFile : manifests) { ManifestReader reader = ManifestFiles.read(manifestFile, fileIO, specs); try (CloseableIterator it = reader.iterator()) { while (it.hasNext()) { - recordCount += it.next().recordCount(); + it.next().recordCount(); } } } diff --git a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java index 6a5d07d7813d..d92c1a3742fe 100644 --- a/core/src/main/java/org/apache/iceberg/UpdateRequirements.java +++ b/core/src/main/java/org/apache/iceberg/UpdateRequirements.java @@ -124,7 +124,7 @@ private void update(MetadataUpdate.SetSnapshotRef setRef) { } } - private void update(MetadataUpdate.AddSchema update) { + private void update(MetadataUpdate.AddSchema unused) { if (!addedSchema) { if (base != null) { require(new UpdateRequirement.AssertLastAssignedFieldId(base.lastColumnId())); @@ -133,7 +133,7 @@ private void update(MetadataUpdate.AddSchema update) { } } - private void update(MetadataUpdate.SetCurrentSchema update) { + private void update(MetadataUpdate.SetCurrentSchema unused) { if (!setSchemaId) { if (base != null && !isReplace) { // require that the current schema has not changed @@ -143,7 +143,7 @@ private void update(MetadataUpdate.SetCurrentSchema update) { } } - private void update(MetadataUpdate.AddPartitionSpec update) { + private void update(MetadataUpdate.AddPartitionSpec unused) { if (!addedSpec) { if (base != null) { require( @@ -153,7 +153,7 @@ private void update(MetadataUpdate.AddPartitionSpec update) { } } - private void update(MetadataUpdate.SetDefaultPartitionSpec update) { + private void update(MetadataUpdate.SetDefaultPartitionSpec unused) { if (!setSpecId) { if (base != null && !isReplace) { // require that the default spec has not changed @@ -163,7 +163,7 @@ private void update(MetadataUpdate.SetDefaultPartitionSpec update) { } } - private void update(MetadataUpdate.SetDefaultSortOrder update) { + private void update(MetadataUpdate.SetDefaultSortOrder unused) { if (!setOrderId) { if (base != null && !isReplace) { // require that the default write order has not changed diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java index f816b5d7a4f6..dd1358f2ed40 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java @@ -106,7 +106,7 @@ public static Comparator comparator(RewriteJobOrder rewriteJob case FILES_DESC: return Comparator.comparing(RewriteFileGroup::numFiles, Comparator.reverseOrder()); default: - return (fileGroupOne, fileGroupTwo) -> 0; + return (unused, unused2) -> 0; } } } diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java index b8ebe528deea..2be7145bcd34 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java @@ -127,7 +127,7 @@ public static Comparator comparator(RewriteJobOrder return Comparator.comparing( RewritePositionDeletesGroup::numRewrittenDeleteFiles, Comparator.reverseOrder()); default: - return (fileGroupOne, fileGroupTwo) -> 0; + return (unused, unused2) -> 0; } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java index 52c89af9d474..1757ae653cc9 100644 --- a/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java +++ b/core/src/main/java/org/apache/iceberg/rest/auth/OAuth2Util.java @@ -72,7 +72,6 @@ private OAuth2Util() {} private static final String CLIENT_CREDENTIALS = "client_credentials"; private static final String TOKEN_EXCHANGE = "urn:ietf:params:oauth:grant-type:token-exchange"; private static final String SCOPE = "scope"; - private static final String CATALOG = "catalog"; // Client credentials flow private static final String CLIENT_ID = "client_id"; @@ -97,7 +96,6 @@ private OAuth2Util() {} private static final String TOKEN_TYPE = "token_type"; private static final String EXPIRES_IN = "expires_in"; private static final String ISSUED_TOKEN_TYPE = "issued_token_type"; - private static final String REFRESH_TOKEN = "refresh_token"; public static Map authHeaders(String token) { if (token != null) { diff --git a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java index f86e5ddc5a8a..d40f64844797 100644 --- a/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java +++ b/core/src/main/java/org/apache/iceberg/util/ParallelIterable.java @@ -85,18 +85,16 @@ static class ParallelIterator implements CloseableIterator { private final ExecutorService workerPool; private final CompletableFuture>>[] taskFutures; private final ConcurrentLinkedQueue queue = new ConcurrentLinkedQueue<>(); - private final int maxQueueSize; private final AtomicBoolean closed = new AtomicBoolean(false); private ParallelIterator( Iterable> iterables, ExecutorService workerPool, int maxQueueSize) { + Preconditions.checkArgument(maxQueueSize > 0, "Max queue size must be greater than 0"); this.tasks = Iterables.transform( iterables, iterable -> new Task<>(iterable, queue, closed, maxQueueSize)) .iterator(); this.workerPool = workerPool; - Preconditions.checkArgument(maxQueueSize > 0, "Max queue size must be greater than 0"); - this.maxQueueSize = maxQueueSize; // submit 2 tasks per worker at a time this.taskFutures = new CompletableFuture[2 * ThreadPools.WORKER_THREAD_POOL_SIZE]; } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 1019285018d0..34576a1e5c0b 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -237,10 +237,10 @@ public MapData getMap(int pos) { @Override public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); + return isNullAt(pos) ? null : getStructRowData(pos); } - private StructRowData getStructRowData(int pos, int numFields) { + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index c958604c004a..391633924264 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -104,10 +104,7 @@ public static class RewriteMap extends RichMapFunction taskWriterFactory; private final RowDataFileScanTaskReader rowDataReader; @@ -119,10 +116,7 @@ public RewriteMap( boolean caseSensitive, EncryptionManager encryptionManager, TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; this.io = io; - this.caseSensitive = caseSensitive; this.encryptionManager = encryptionManager; this.taskWriterFactory = taskWriterFactory; this.rowDataReader = diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 1019285018d0..34576a1e5c0b 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -237,10 +237,10 @@ public MapData getMap(int pos) { @Override public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); + return isNullAt(pos) ? null : getStructRowData(pos); } - private StructRowData getStructRowData(int pos, int numFields) { + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index c958604c004a..391633924264 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -104,10 +104,7 @@ public static class RewriteMap extends RichMapFunction taskWriterFactory; private final RowDataFileScanTaskReader rowDataReader; @@ -119,10 +116,7 @@ public RewriteMap( boolean caseSensitive, EncryptionManager encryptionManager, TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; this.io = io; - this.caseSensitive = caseSensitive; this.encryptionManager = encryptionManager; this.taskWriterFactory = taskWriterFactory; this.rowDataReader = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java index 1019285018d0..34576a1e5c0b 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/data/StructRowData.java @@ -237,10 +237,10 @@ public MapData getMap(int pos) { @Override public RowData getRow(int pos, int numFields) { - return isNullAt(pos) ? null : getStructRowData(pos, numFields); + return isNullAt(pos) ? null : getStructRowData(pos); } - private StructRowData getStructRowData(int pos, int numFields) { + private StructRowData getStructRowData(int pos) { return new StructRowData( type.fields().get(pos).type().asStructType(), struct.get(pos, StructLike.class)); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java index c958604c004a..391633924264 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/RowDataRewriter.java @@ -104,10 +104,7 @@ public static class RewriteMap extends RichMapFunction taskWriterFactory; private final RowDataFileScanTaskReader rowDataReader; @@ -119,10 +116,7 @@ public RewriteMap( boolean caseSensitive, EncryptionManager encryptionManager, TaskWriterFactory taskWriterFactory) { - this.schema = schema; - this.nameMapping = nameMapping; this.io = io; - this.caseSensitive = caseSensitive; this.encryptionManager = encryptionManager; this.taskWriterFactory = taskWriterFactory; this.rowDataReader = diff --git a/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java b/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java index e8a4b5da3583..2cd0e5ef4cc6 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java +++ b/mr/src/main/java/org/apache/iceberg/mr/Catalogs.java @@ -225,8 +225,7 @@ public static boolean hiveCatalog(Configuration conf, Properties props) { if (catalogType != null) { return CatalogUtil.ICEBERG_CATALOG_TYPE_HIVE.equalsIgnoreCase(catalogType); } - return getCatalogProperties(conf, catalogName, catalogType).get(CatalogProperties.CATALOG_IMPL) - == null; + return getCatalogProperties(conf, catalogName).get(CatalogProperties.CATALOG_IMPL) == null; } @VisibleForTesting @@ -237,8 +236,7 @@ static Optional loadCatalog(Configuration conf, String catalogName) { } else { String name = catalogName == null ? ICEBERG_DEFAULT_CATALOG_NAME : catalogName; return Optional.of( - CatalogUtil.buildIcebergCatalog( - name, getCatalogProperties(conf, name, catalogType), conf)); + CatalogUtil.buildIcebergCatalog(name, getCatalogProperties(conf, name), conf)); } } @@ -247,11 +245,9 @@ static Optional loadCatalog(Configuration conf, String catalogName) { * * @param conf a Hadoop configuration * @param catalogName name of the catalog - * @param catalogType type of the catalog * @return complete map of catalog properties */ - private static Map getCatalogProperties( - Configuration conf, String catalogName, String catalogType) { + private static Map getCatalogProperties(Configuration conf, String catalogName) { String keyPrefix = InputFormatConfig.CATALOG_CONFIG_PREFIX + catalogName; return Streams.stream(conf.iterator()) diff --git a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java index 49f5b86aa031..637dc6d15df1 100644 --- a/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java +++ b/mr/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java @@ -118,7 +118,7 @@ public void preCreateTable(org.apache.hadoop.hive.metastore.api.Table hmsTable) // Iceberg schema and specification generated by the code Schema schema = schema(catalogProperties, hmsTable); - PartitionSpec spec = spec(schema, catalogProperties, hmsTable); + PartitionSpec spec = spec(schema, hmsTable); // If there are partition keys specified remove them from the HMS table and add them to the // column list @@ -290,7 +290,7 @@ private Schema schema( } private static PartitionSpec spec( - Schema schema, Properties properties, org.apache.hadoop.hive.metastore.api.Table hmsTable) { + Schema schema, org.apache.hadoop.hive.metastore.api.Table hmsTable) { if (hmsTable.getParameters().get(InputFormatConfig.PARTITION_SPEC) != null) { Preconditions.checkArgument( diff --git a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java index b8a48645efb5..842bd99f4680 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java +++ b/orc/src/main/java/org/apache/iceberg/orc/OrcFileAppender.java @@ -53,7 +53,10 @@ class OrcFileAppender implements FileAppender { private final int avgRowByteSize; private final OrcRowWriter valueWriter; private boolean isClosed = false; + + @SuppressWarnings("unused") // Currently used in tests TODO remove this redundant field private final Configuration conf; + private final MetricsConfig metricsConfig; OrcFileAppender( diff --git a/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index d6fab897d62d..51e3721aea4f 100644 --- a/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.3/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -101,7 +101,7 @@ public void testAlterTable() throws NoSuchTableException { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); - Table table = getTable(); + Table table; // Add examples sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); diff --git a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 59bc70854cbe..20be98d17bb2 100644 --- a/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.4/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -103,7 +103,7 @@ public void testAlterTable() throws NoSuchTableException { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); - Table table = getTable(); + Table table; // Add examples sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); diff --git a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java index 89f7ad5dc62f..ec445774a452 100644 --- a/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java +++ b/spark/v3.5/spark-runtime/src/integration/java/org/apache/iceberg/spark/SmokeTest.java @@ -101,7 +101,7 @@ public void testAlterTable() { sql( "CREATE TABLE %s (category int, id bigint, data string, ts timestamp) USING iceberg", tableName); - Table table = getTable(); + Table table; // Add examples sql("ALTER TABLE %s ADD PARTITION FIELD bucket(16, id)", tableName); sql("ALTER TABLE %s ADD PARTITION FIELD truncate(data, 4)", tableName); From 6fcb18b4b07a7a61364e5ea7292c2c2370a5b5a8 Mon Sep 17 00:00:00 2001 From: Rahil C <32500120+rahil-c@users.noreply.github.com> Date: Tue, 10 Sep 2024 07:34:33 -0700 Subject: [PATCH 0680/1019] OpenAPI: Add Scan Planning Endpoints to REST spec (#9695) --- open-api/rest-catalog-open-api.py | 195 +++++++++- open-api/rest-catalog-open-api.yaml | 577 +++++++++++++++++++++++++++- 2 files changed, 767 insertions(+), 5 deletions(-) diff --git a/open-api/rest-catalog-open-api.py b/open-api/rest-catalog-open-api.py index 46d8f2aa7718..e890604c38cd 100644 --- a/open-api/rest-catalog-open-api.py +++ b/open-api/rest-catalog-open-api.py @@ -108,6 +108,8 @@ class ExpressionType(BaseModel): __root__: str = Field( ..., example=[ + 'true', + 'false', 'eq', 'and', 'or', @@ -129,6 +131,14 @@ class ExpressionType(BaseModel): ) +class TrueExpression(BaseModel): + type: ExpressionType + + +class FalseExpression(BaseModel): + type: ExpressionType + + class Reference(BaseModel): __root__: str = Field(..., example=['column-name']) @@ -457,6 +467,12 @@ class AssertViewUUID(BaseModel): uuid: str +class PlanStatus(BaseModel): + __root__: Literal['completed', 'submitted', 'cancelled', 'failed'] = Field( + ..., description='Status of a server-side planning operation' + ) + + class RegisterTableRequest(BaseModel): name: str metadata_location: str = Field(..., alias='metadata-location') @@ -808,8 +824,8 @@ class ContentFile(BaseModel): file_path: str = Field(..., alias='file-path') file_format: FileFormat = Field(..., alias='file-format') spec_id: int = Field(..., alias='spec-id') - partition: Optional[List[PrimitiveTypeValue]] = Field( - None, + partition: List[PrimitiveTypeValue] = Field( + ..., description='A list of partition field values ordered based on the fields of the partition spec specified by the `spec-id`', example=[1, 'bar'], ) @@ -839,6 +855,20 @@ class EqualityDeleteFile(ContentFile): ) +class FieldName(BaseModel): + __root__: str = Field( + ..., + description='A full field name (including parent field names), such as those passed in APIs like Java `Schema#findField(String name)`.\nThe nested field name follows these rules - Nested struct fields are named by concatenating field names at each struct level using dot (`.`) delimiter, e.g. employer.contact_info.address.zip_code - Nested fields in a map key are named using the keyword `key`, e.g. employee_address_map.key.first_name - Nested fields in a map value are named using the keyword `value`, e.g. employee_address_map.value.zip_code - Nested fields in a list are named using the keyword `element`, e.g. employees.element.first_name', + ) + + +class PlanTask(BaseModel): + __root__: str = Field( + ..., + description='An opaque string provided by the REST server that represents a unit of work to produce file scan tasks for scan planning. This allows clients to fetch tasks across multiple requests to accommodate large result sets.', + ) + + class CreateNamespaceRequest(BaseModel): namespace: Namespace properties: Optional[Dict[str, str]] = Field( @@ -883,6 +913,29 @@ class ViewRequirement(BaseModel): __root__: AssertViewUUID = Field(..., discriminator='type') +class FailedPlanningResult(IcebergErrorResponse): + """ + Failed server-side planning result + """ + + status: Literal['failed'] + + +class AsyncPlanningResult(BaseModel): + status: Literal['submitted'] + plan_id: Optional[str] = Field( + None, alias='plan-id', description='ID used to track a planning request' + ) + + +class EmptyPlanningResult(BaseModel): + """ + Empty server-side planning result + """ + + status: Literal['cancelled'] + + class ReportMetricsRequest2(CommitReport): report_type: str = Field(..., alias='report-type') @@ -936,6 +989,16 @@ class DataFile(ContentFile): ) +class DeleteFile(BaseModel): + __root__: Union[PositionDeleteFile, EqualityDeleteFile] = Field( + ..., discriminator='content' + ) + + +class FetchScanTasksRequest(BaseModel): + plan_task: PlanTask = Field(..., alias='plan-task') + + class Term(BaseModel): __root__: Union[Reference, TransformTerm] @@ -999,6 +1062,8 @@ class Type(BaseModel): class Expression(BaseModel): __root__: Union[ + TrueExpression, + FalseExpression, AndOrExpression, NotExpression, SetExpression, @@ -1141,6 +1206,52 @@ class LoadTableResult(BaseModel): config: Optional[Dict[str, str]] = None +class ScanTasks(BaseModel): + """ + Scan and planning tasks for server-side scan planning + + - `plan-tasks` contains opaque units of planning work + - `file-scan-tasks` contains a partial or complete list of table scan tasks + - `delete-files` contains delete files referenced by file scan tasks + + Each plan task must be passed to the fetchScanTasks endpoint to fetch the file scan tasks for the plan task. + + The list of delete files must contain all delete files referenced by the file scan tasks. + + """ + + delete_files: Optional[List[DeleteFile]] = Field( + None, + alias='delete-files', + description='Delete files referenced by file scan tasks', + ) + file_scan_tasks: Optional[List[FileScanTask]] = Field(None, alias='file-scan-tasks') + plan_tasks: Optional[List[PlanTask]] = Field(None, alias='plan-tasks') + + +class FetchPlanningResult(BaseModel): + __root__: Union[ + CompletedPlanningResult, FailedPlanningResult, EmptyPlanningResult + ] = Field( + ..., + description='Result of server-side scan planning for fetchPlanningResult', + discriminator='status', + ) + + +class PlanTableScanResult(BaseModel): + __root__: Union[ + CompletedPlanningWithIDResult, + FailedPlanningResult, + AsyncPlanningResult, + EmptyPlanningResult, + ] = Field( + ..., + description='Result of server-side scan planning for planTableScan', + discriminator='status', + ) + + class CommitTableRequest(BaseModel): identifier: Optional[TableIdentifier] = Field( None, @@ -1227,6 +1338,59 @@ class CommitTableResponse(BaseModel): metadata: TableMetadata +class PlanTableScanRequest(BaseModel): + snapshot_id: Optional[int] = Field( + None, + alias='snapshot-id', + description='Identifier for the snapshot to scan in a point-in-time scan', + ) + select: Optional[List[FieldName]] = Field( + None, description='List of selected schema fields' + ) + filter: Optional[Expression] = Field( + None, description='Expression used to filter the table data' + ) + case_sensitive: Optional[bool] = Field( + True, + alias='case-sensitive', + description='Enables case sensitive field matching for filter and select', + ) + use_snapshot_schema: Optional[bool] = Field( + False, + alias='use-snapshot-schema', + description='Whether to use the schema at the time the snapshot was written.\nWhen time travelling, the snapshot schema should be used (true). When scanning a branch, the table schema should be used (false).', + ) + start_snapshot_id: Optional[int] = Field( + None, + alias='start-snapshot-id', + description='Starting snapshot ID for an incremental scan (exclusive)', + ) + end_snapshot_id: Optional[int] = Field( + None, + alias='end-snapshot-id', + description='Ending snapshot ID for an incremental scan (inclusive).\nRequired when start-snapshot-id is specified.', + ) + stats_fields: Optional[List[FieldName]] = Field( + None, + alias='stats-fields', + description='List of fields for which the service should send column stats.', + ) + + +class FileScanTask(BaseModel): + data_file: DataFile = Field(..., alias='data-file') + delete_file_references: Optional[List[int]] = Field( + None, + alias='delete-file-references', + description='A list of indices in the delete files array (0-based)', + ) + residual_filter: Optional[Expression] = Field( + None, + alias='residual-filter', + description='An optional filter to be applied to rows in this file scan task.\nIf the residual is not present, the client must produce the residual or use the original filter.', + ) + + class Schema(StructType): schema_id: Optional[int] = Field(None, alias='schema-id') identifier_field_ids: Optional[List[int]] = Field( @@ -1234,10 +1398,31 @@ class Schema(StructType): ) +class CompletedPlanningResult(ScanTasks): + """ + Completed server-side planning result + """ + + status: Literal['completed'] + + +class FetchScanTasksResult(ScanTasks): + """ + Response schema for fetchScanTasks + """ + + class ReportMetricsRequest1(ScanReport): report_type: str = Field(..., alias='report-type') +class CompletedPlanningWithIDResult(CompletedPlanningResult): + plan_id: Optional[str] = Field( + None, alias='plan-id', description='ID used to track a planning request' + ) + status: Literal['completed'] + + StructField.update_forward_refs() ListType.update_forward_refs() MapType.update_forward_refs() @@ -1245,6 +1430,12 @@ class ReportMetricsRequest1(ScanReport): TableMetadata.update_forward_refs() ViewMetadata.update_forward_refs() AddSchemaUpdate.update_forward_refs() +ScanTasks.update_forward_refs() +FetchPlanningResult.update_forward_refs() +PlanTableScanResult.update_forward_refs() CreateTableRequest.update_forward_refs() CreateViewRequest.update_forward_refs() ReportMetricsRequest.update_forward_refs() +CompletedPlanningResult.update_forward_refs() +FetchScanTasksResult.update_forward_refs() +CompletedPlanningWithIDResult.update_forward_refs() diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index eec594eb89fb..abd937933290 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -593,6 +593,261 @@ paths: 5XX: $ref: '#/components/responses/ServerErrorResponse' + /v1/{prefix}/namespaces/{namespace}/tables/{table}/plan: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + post: + tags: + - Catalog API + summary: Submit a scan for planning + description: > + Submits a scan for server-side planning. + + + Point-in-time scans are planned by passing snapshot-id to identify the + table snapshot to scan. Incremental scans are planned by passing both + start-snapshot-id and end-snapshot-id. Requests that include both point + in time config properties and incremental config properties are + invalid. If the request does not include either incremental or + point-in-time config properties, scan planning should produce a + point-in-time scan of the latest snapshot in the table's main branch. + + + Responses must include a valid status listed below. A "cancelled" status is considered invalid for this endpoint. + + - When "completed" the planning operation has produced plan tasks and + file scan tasks that must be returned in the response (not fetched + later by calling fetchPlanningResult) + + - When "submitted" the response must include a plan-id used to poll + fetchPlanningResult to fetch the planning result when it is ready + + - When "failed" the response must be a valid error response + + The response for a "completed" planning operation includes two types of + tasks (file scan tasks and plan tasks) and both may be included in the + response. Tasks must not be included for any other response status. + + + Responses that include a plan-id indicate that the service is holding + state or performing work for the client. + + + - Clients should use the plan-id to fetch results from + fetchPlanningResult when the response status is "submitted" + + - Clients should inform the service if planning results are no longer + needed by calling cancelPlanning. Cancellation is not necessary after + fetchScanTasks has been used to fetch scan tasks for each plan task. + operationId: planTableScan + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/PlanTableScanRequest' + responses: + 200: + $ref: '#/components/responses/PlanTableScanResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 406: + $ref: '#/components/responses/UnsupportedOperationResponse' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + /v1/{prefix}/namespaces/{namespace}/tables/{table}/plan/{plan-id}: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + - $ref: '#/components/parameters/plan-id' + + get: + tags: + - Catalog API + summary: Fetches the result of scan planning for a plan-id + operationId: fetchPlanningResult + description: > + Fetches the result of scan planning for a plan-id. + + + Responses must include a valid status + + - When "completed" the planning operation has produced plan-tasks and + file-scan-tasks that must be returned in the response + + - When "submitted" the planning operation has not completed; the client + should wait to call this endpoint again to fetch a completed response + + - When "failed" the response must be a valid error response + + - When "cancelled" the plan-id is invalid and should be discarded + + + The response for a "completed" planning operation includes two types of + tasks (file scan tasks and plan tasks) and both may be included in the + response. Tasks must not be included for any other response status. + responses: + 200: + $ref: '#/components/responses/FetchPlanningResultResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchPlanIdException, the plan-id does not exist + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + PlanIdDoesNotExist: + $ref: '#/components/examples/NoSuchPlanIdError' + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + delete: + tags: + - Catalog API + summary: Cancels scan planning for a plan-id + operationId: cancelPlanning + description: > + Cancels scan planning for a plan-id. + + + This notifies the service that it can release resources held for the + scan. Clients should cancel scans that are no longer needed, either + while the plan-id returns a "submitted" status or while there are + remaining plan tasks that have not been fetched. + + + Cancellation is not necessary when + + - Scan tasks for each plan task have been fetched using fetchScanTasks + + - A plan-id has produced a "failed" or "cancelled" status from + planTableScan or fetchPlanningResult + responses: + 204: + description: Success, no content + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + + /v1/{prefix}/namespaces/{namespace}/tables/{table}/tasks: + parameters: + - $ref: '#/components/parameters/prefix' + - $ref: '#/components/parameters/namespace' + - $ref: '#/components/parameters/table' + + post: + tags: + - Catalog API + summary: Fetches result tasks for a plan task + operationId: fetchScanTasks + description: Fetches result tasks for a plan task. + requestBody: + content: + application/json: + schema: + $ref: '#/components/schemas/FetchScanTasksRequest' + responses: + 200: + $ref: '#/components/responses/FetchScanTasksResponse' + 400: + $ref: '#/components/responses/BadRequestErrorResponse' + 401: + $ref: '#/components/responses/UnauthorizedResponse' + 403: + $ref: '#/components/responses/ForbiddenResponse' + 404: + description: + Not Found + - NoSuchPlanTaskException, the plan-task does not exist + - NoSuchTableException, the table does not exist + - NoSuchNamespaceException, the namespace does not exist + content: + application/json: + schema: + $ref: '#/components/schemas/IcebergErrorResponse' + examples: + PlanTaskDoesNotExist: + $ref: '#/components/examples/NoSuchPlanTaskError' + TableDoesNotExist: + $ref: '#/components/examples/NoSuchTableError' + NamespaceDoesNotExist: + $ref: '#/components/examples/NoSuchNamespaceError' + 419: + $ref: '#/components/responses/AuthenticationTimeoutResponse' + 503: + $ref: '#/components/responses/ServiceUnavailableResponse' + 5XX: + $ref: '#/components/responses/ServerErrorResponse' + + + /v1/{prefix}/namespaces/{namespace}/register: parameters: - $ref: '#/components/parameters/prefix' @@ -681,7 +936,7 @@ paths: The snapshots to return in the body of the metadata. Setting the value to `all` would return the full set of snapshots currently valid for the table. Setting the value to `refs` would load all snapshots referenced by branches or tags. - + Default if no param is provided is `all`. required: false schema: @@ -1518,6 +1773,14 @@ components: type: string example: "sales" + plan-id: + name: plan-id + in: path + description: ID used to track a planning request + required: true + schema: + type: string + view: name: view in: path @@ -1826,6 +2089,8 @@ components: Expression: oneOf: + - $ref: '#/components/schemas/TrueExpression' + - $ref: '#/components/schemas/FalseExpression' - $ref: '#/components/schemas/AndOrExpression' - $ref: '#/components/schemas/NotExpression' - $ref: '#/components/schemas/SetExpression' @@ -1835,6 +2100,8 @@ components: ExpressionType: type: string example: + - "true" + - "false" - "eq" - "and" - "or" @@ -1853,6 +2120,24 @@ components: - "is-nan" - "not-nan" + TrueExpression: + type: object + required: + - type + properties: + type: + $ref: '#/components/schemas/ExpressionType' + enum: ["true"] + + FalseExpression: + type: object + required: + - type + properties: + type: + $ref: '#/components/schemas/ExpressionType' + enum: ["false"] + AndOrExpression: type: object required: @@ -2858,6 +3143,140 @@ components: additionalProperties: type: string + ScanTasks: + type: object + description: > + Scan and planning tasks for server-side scan planning + + + - `plan-tasks` contains opaque units of planning work + + - `file-scan-tasks` contains a partial or complete list of table scan tasks + + - `delete-files` contains delete files referenced by file scan tasks + + + Each plan task must be passed to the fetchScanTasks endpoint to fetch + the file scan tasks for the plan task. + + + The list of delete files must contain all delete files referenced by + the file scan tasks. + properties: + delete-files: + description: Delete files referenced by file scan tasks + type: array + items: + $ref: '#/components/schemas/DeleteFile' + file-scan-tasks: + type: array + items: + $ref: '#/components/schemas/FileScanTask' + plan-tasks: + type: array + items: + $ref: '#/components/schemas/PlanTask' + + CompletedPlanningResult: + type: object + description: Completed server-side planning result + allOf: + - $ref: '#/components/schemas/ScanTasks' + - type: object + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["completed"] + + CompletedPlanningWithIDResult: + type: object + allOf: + - $ref: '#/components/schemas/CompletedPlanningResult' + - type: object + properties: + plan-id: + description: ID used to track a planning request + type: string + + FailedPlanningResult: + type: object + description: Failed server-side planning result + allOf: + - $ref: '#/components/schemas/IcebergErrorResponse' + - type: object + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["failed"] + + AsyncPlanningResult: + type: object + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["submitted"] + plan-id: + description: ID used to track a planning request + type: string + + EmptyPlanningResult: + type: object + description: Empty server-side planning result + required: + - status + properties: + status: + $ref: '#/components/schemas/PlanStatus' + enum: ["submitted", "cancelled"] + + PlanStatus: + description: Status of a server-side planning operation + type: string + enum: ["completed", "submitted", "cancelled", "failed"] + + FetchPlanningResult: + type: object + description: Result of server-side scan planning for fetchPlanningResult + discriminator: + propertyName: status + mapping: + completed: '#/components/schemas/CompletedPlanningResult' + submitted: '#/components/schemas/EmptyPlanningResult' + cancelled: '#/components/schemas/EmptyPlanningResult' + failed: '#/components/schemas/FailedPlanningResult' + oneOf: + - $ref: '#/components/schemas/CompletedPlanningResult' + - $ref: '#/components/schemas/FailedPlanningResult' + - $ref: '#/components/schemas/EmptyPlanningResult' + + PlanTableScanResult: + type: object + description: Result of server-side scan planning for planTableScan + discriminator: + propertyName: status + mapping: + completed: '#/components/schemas/CompletedPlanningWithIDResult' + submitted: '#/components/schemas/AsyncPlanningResult' + cancelled: '#/components/schemas/EmptyPlanningResult' + failed: '#/components/schemas/FailedPlanningResult' + oneOf: + - $ref: '#/components/schemas/CompletedPlanningWithIDResult' + - $ref: '#/components/schemas/FailedPlanningResult' + - $ref: '#/components/schemas/AsyncPlanningResult' + - $ref: '#/components/schemas/EmptyPlanningResult' + + FetchScanTasksResult: + type: object + description: Response schema for fetchScanTasks + allOf: + - $ref: '#/components/schemas/ScanTasks' + CommitTableRequest: type: object required: @@ -3652,6 +4071,7 @@ components: type: object required: - spec-id + - partition - content - file-path - file-format @@ -3671,8 +4091,8 @@ components: items: $ref: '#/components/schemas/PrimitiveTypeValue' description: - "A list of partition field values ordered based on the fields of the partition spec specified by the - `spec-id`" + A list of partition field values ordered based on the fields of + the partition spec specified by the `spec-id` example: [1, "bar"] file-size-in-bytes: type: integer @@ -3730,6 +4150,16 @@ components: - $ref: '#/components/schemas/ValueMap' description: "Map of column id to upper bound primitive type values" + DeleteFile: + discriminator: + propertyName: content + mapping: + position-deletes: '#/components/schemas/PositionDeleteFile' + equality-deletes: '#/components/schemas/EqualityDeleteFile' + oneOf: + - $ref: '#/components/schemas/PositionDeleteFile' + - $ref: '#/components/schemas/EqualityDeleteFile' + PositionDeleteFile: allOf: - $ref: '#/components/schemas/ContentFile' @@ -3755,6 +4185,106 @@ components: type: integer description: "List of equality field IDs" + PlanTableScanRequest: + type: object + properties: + snapshot-id: + description: + Identifier for the snapshot to scan in a point-in-time scan + type: integer + format: int64 + select: + description: List of selected schema fields + type: array + items: + $ref: '#/components/schemas/FieldName' + filter: + description: + Expression used to filter the table data + $ref: '#/components/schemas/Expression' + case-sensitive: + description: Enables case sensitive field matching for filter and select + type: boolean + default: true + use-snapshot-schema: + description: + Whether to use the schema at the time the snapshot was written. + + When time travelling, the snapshot schema should be used (true). + When scanning a branch, the table schema should be used (false). + type: boolean + default: false + start-snapshot-id: + description: Starting snapshot ID for an incremental scan (exclusive) + type: integer + format: int64 + end-snapshot-id: + description: + Ending snapshot ID for an incremental scan (inclusive). + + Required when start-snapshot-id is specified. + type: integer + format: int64 + stats-fields: + description: + List of fields for which the service should send column stats. + type: array + items: + $ref: '#/components/schemas/FieldName' + + FieldName: + description: + A full field name (including parent field names), such as those passed + in APIs like Java `Schema#findField(String name)`. + + The nested field name follows these rules + - Nested struct fields are named by concatenating field names at each + struct level using dot (`.`) delimiter, e.g. + employer.contact_info.address.zip_code + - Nested fields in a map key are named using the keyword `key`, e.g. + employee_address_map.key.first_name + - Nested fields in a map value are named using the keyword `value`, + e.g. employee_address_map.value.zip_code + - Nested fields in a list are named using the keyword `element`, e.g. + employees.element.first_name + type: string + + FetchScanTasksRequest: + type: object + required: + - plan-task + properties: + plan-task: + $ref: '#/components/schemas/PlanTask' + + PlanTask: + description: + An opaque string provided by the REST server that represents a + unit of work to produce file scan tasks for scan planning. This allows + clients to fetch tasks across multiple requests to accommodate large result sets. + type: string + + FileScanTask: + type: object + required: + - data-file + properties: + data-file: + $ref: '#/components/schemas/DataFile' + delete-file-references: + description: A list of indices in the delete files array (0-based) + type: array + items: + type: integer + residual-filter: + description: + An optional filter to be applied to rows in this file scan task. + + If the residual is not present, the client must produce the + residual or use the original filter. + allOf: + - $ref: '#/components/schemas/Expression' + ############################# # Reusable Response Objects # ############################# @@ -3971,6 +4501,27 @@ components: schema: $ref: '#/components/schemas/LoadTableResult' + PlanTableScanResponse: + description: Result of submitting a table scan to plan + content: + application/json: + schema: + $ref: '#/components/schemas/PlanTableScanResult' + + FetchPlanningResultResponse: + description: Result of fetching a submitted scan planning operation + content: + application/json: + schema: + $ref: '#/components/schemas/FetchPlanningResult' + + FetchScanTasksResponse: + description: Result of retrieving additional plan tasks and file scan tasks. + content: + application/json: + schema: + $ref: '#/components/schemas/FetchScanTasksResult' + LoadTableResponse: description: Table metadata result when loading a table content: @@ -4048,6 +4599,26 @@ components: } } + NoSuchPlanIdError: + summary: The plan id does not exist + value: { + "error": { + "message": "The plan id does not exist", + "type": "NoSuchPlanIdException", + "code": 404 + } + } + + NoSuchPlanTaskError: + summary: The plan task does not exist + value: { + "error": { + "message": "The plan task does not exist", + "type": "NoSuchPlanTaskException", + "code": 404 + } + } + NoSuchTableError: summary: The requested table does not exist value: { From 29d60667e7c86420a975e6e28a42d465c000d8c8 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Tue, 10 Sep 2024 09:42:31 -0700 Subject: [PATCH 0681/1019] Kafka Connect: Docs on configuring the sink (#10746) * Kafka Connect: Docs on configuring the sink * Update docs/docs/kafka-connect.md Co-authored-by: Daniel Weeks * correct docs * added blurb about GCS ADC * add note about Kafka 2.5 requirement * Update docs/docs/kafka-connect.md Co-authored-by: Ajantha Bhat * Update docs/docs/kafka-connect.md Co-authored-by: Ajantha Bhat * Update docs/docs/kafka-connect.md Co-authored-by: Ajantha Bhat * document force lowercase config * Revert "document force lowercase config" This reverts commit 4213a0dcf20eeca9bb609a762c5536df19e30400. --------- Co-authored-by: Daniel Weeks Co-authored-by: Ajantha Bhat --- docs/docs/kafka-connect.md | 352 +++++++++++++++++++++++++++++++++++++ docs/mkdocs.yml | 1 + 2 files changed, 353 insertions(+) create mode 100644 docs/docs/kafka-connect.md diff --git a/docs/docs/kafka-connect.md b/docs/docs/kafka-connect.md new file mode 100644 index 000000000000..a904a17a9968 --- /dev/null +++ b/docs/docs/kafka-connect.md @@ -0,0 +1,352 @@ +--- +title: "Kafka Connect" +--- + + +# Kafka Connect + +[Kafka Connect](https://docs.confluent.io/platform/current/connect/index.html) is a popular framework for moving data +in and out of Kafka via connectors. There are many different connectors available, such as the S3 sink +for writing data from Kafka to S3 and Debezium source connectors for writing change data capture records from relational +databases to Kafka. + +It has a straightforward, decentralized, distributed architecture. A cluster consists of a number of worker processes, +and a connector runs tasks on these processes to perform the work. Connector deployment is configuration driven, so +generally no code needs to be written to run a connector. + +## Apache Iceberg Sink Connector + +The Apache Iceberg Sink Connector for Kafka Connect is a sink connector for writing data from Kafka into Iceberg tables. + +## Features + +* Commit coordination for centralized Iceberg commits +* Exactly-once delivery semantics +* Multi-table fan-out +* Automatic table creation and schema evolution +* Field name mapping via Iceberg’s column mapping functionality + +## Installation + +The connector zip archive is created as part of the Iceberg build. You can run the build via: +```bash +./gradlew -x test -x integrationTest clean build +``` +The zip archive will be found under `./kafka-connect/kafka-connect-runtime/build/distributions`. There is +one distribution that bundles the Hive Metastore client and related dependencies, and one that does not. +Copy the distribution archive into the Kafka Connect plugins directory on all nodes. + +## Requirements + +The sink relies on [KIP-447](https://cwiki.apache.org/confluence/display/KAFKA/KIP-447%3A+Producer+scalability+for+exactly+once+semantics) +for exactly-once semantics. This requires Kafka 2.5 or later. + +## Configuration + +| Property | Description | +|--------------------------------------------|------------------------------------------------------------------------------------------------------------------| +| iceberg.tables | Comma-separated list of destination tables | +| iceberg.tables.dynamic-enabled | Set to `true` to route to a table specified in `routeField` instead of using `routeRegex`, default is `false` | +| iceberg.tables.route-field | For multi-table fan-out, the name of the field used to route records to tables | +| iceberg.tables.default-commit-branch | Default branch for commits, main is used if not specified | +| iceberg.tables.default-id-columns | Default comma-separated list of columns that identify a row in tables (primary key) | +| iceberg.tables.default-partition-by | Default comma-separated list of partition field names to use when creating tables | +| iceberg.tables.auto-create-enabled | Set to `true` to automatically create destination tables, default is `false` | +| iceberg.tables.evolve-schema-enabled | Set to `true` to add any missing record fields to the table schema, default is `false` | +| iceberg.tables.schema-force-optional | Set to `true` to set columns as optional during table create and evolution, default is `false` to respect schema | +| iceberg.tables.schema-case-insensitive | Set to `true` to look up table columns by case-insensitive name, default is `false` for case-sensitive | +| iceberg.tables.auto-create-props.* | Properties set on new tables during auto-create | +| iceberg.tables.write-props.* | Properties passed through to Iceberg writer initialization, these take precedence | +| iceberg.table.\

    .commit-branch | Table-specific branch for commits, use `iceberg.tables.default-commit-branch` if not specified | +| iceberg.table.\
    .id-columns | Comma-separated list of columns that identify a row in the table (primary key) | +| 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.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) | +| iceberg.catalog | Name of the catalog, default is `iceberg` | +| iceberg.catalog.* | Properties passed through to Iceberg catalog initialization | +| iceberg.hadoop-conf-dir | If specified, Hadoop config files in this directory will be loaded | +| iceberg.hadoop.* | Properties passed through to the Hadoop configuration | +| iceberg.kafka.* | Properties passed through to control topic Kafka client initialization | + +If `iceberg.tables.dynamic-enabled` is `false` (the default) then you must specify `iceberg.tables`. If +`iceberg.tables.dynamic-enabled` is `true` then you must specify `iceberg.tables.route-field` which will +contain the name of the table. + +### Kafka configuration + +By default the connector will attempt to use Kafka client config from the worker properties for connecting to +the control topic. If that config cannot be read for some reason, Kafka client settings +can be set explicitly using `iceberg.kafka.*` properties. + +#### Message format + +Messages should be converted to a struct or map using the appropriate Kafka Connect converter. + +### Catalog configuration + +The `iceberg.catalog.*` properties are required for connecting to the Iceberg catalog. The core catalog +types are included in the default distribution, including REST, Glue, DynamoDB, Hadoop, Nessie, +JDBC, and Hive. JDBC drivers are not included in the default distribution, so you will need to include +those if needed. When using a Hive catalog, you can use the distribution that includes the Hive metastore client, +otherwise you will need to include that yourself. + +To set the catalog type, you can set `iceberg.catalog.type` to `rest`, `hive`, or `hadoop`. For other +catalog types, you need to instead set `iceberg.catalog.catalog-impl` to the name of the catalog class. + +#### REST example + +``` +"iceberg.catalog.type": "rest", +"iceberg.catalog.uri": "https://catalog-service", +"iceberg.catalog.credential": "", +"iceberg.catalog.warehouse": "", +``` + +#### Hive example + +NOTE: Use the distribution that includes the HMS client (or include the HMS client yourself). Use `S3FileIO` when +using S3 for storage (the default is `HadoopFileIO` with `HiveCatalog`). +``` +"iceberg.catalog.type": "hive", +"iceberg.catalog.uri": "thrift://hive:9083", +"iceberg.catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO", +"iceberg.catalog.warehouse": "s3a://bucket/warehouse", +"iceberg.catalog.client.region": "us-east-1", +"iceberg.catalog.s3.access-key-id": "", +"iceberg.catalog.s3.secret-access-key": "", +``` + +#### Glue example + +``` +"iceberg.catalog.catalog-impl": "org.apache.iceberg.aws.glue.GlueCatalog", +"iceberg.catalog.warehouse": "s3a://bucket/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO", +``` + +#### Nessie example + +``` +"iceberg.catalog.catalog-impl": "org.apache.iceberg.nessie.NessieCatalog", +"iceberg.catalog.uri": "http://localhost:19120/api/v2", +"iceberg.catalog.ref": "main", +"iceberg.catalog.warehouse": "s3a://bucket/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.aws.s3.S3FileIO", +``` + +#### Notes + +Depending on your setup, you may need to also set `iceberg.catalog.s3.endpoint`, `iceberg.catalog.s3.staging-dir`, +or `iceberg.catalog.s3.path-style-access`. See the [Iceberg docs](https://iceberg.apache.org/docs/latest/) for +full details on configuring catalogs. + +### Azure ADLS configuration example + +When using ADLS, Azure requires the passing of AZURE_CLIENT_ID, AZURE_TENANT_ID, and AZURE_CLIENT_SECRET for its Java SDK. +If you're running Kafka Connect in a container, be sure to inject those values as environment variables. See the +[Azure Identity Client library for Java](https://learn.microsoft.com/en-us/java/api/overview/azure/identity-readme?view=azure-java-stable) for more information. + +An example of these would be: +``` +AZURE_CLIENT_ID=e564f687-7b89-4b48-80b8-111111111111 +AZURE_TENANT_ID=95f2f365-f5b7-44b1-88a1-111111111111 +AZURE_CLIENT_SECRET="XXX" +``` +Where the CLIENT_ID is the Application ID of a registered application under +[App Registrations](https://portal.azure.com/#view/Microsoft_AAD_RegisteredApps/ApplicationsListBlade), the TENANT_ID is +from your [Azure Tenant Properties](https://portal.azure.com/#view/Microsoft_AAD_IAM/TenantProperties.ReactView), and +the CLIENT_SECRET is created within the "Certificates & Secrets" section, under "Manage" after choosing your specific +App Registration. You might have to choose "Client secrets" in the middle panel and the "+" in front of "New client secret" +to generate one. Be sure to set this variable to the Value and not the Id. + +It's also important that the App Registration is granted the Role Assignment "Storage Blob Data Contributor" in your +Storage Account's Access Control (IAM), or it won't be able to write new files there. + +Then, within the Connector's configuration, you'll want to include the following: + +``` +"iceberg.catalog.type": "rest", +"iceberg.catalog.uri": "https://catalog:8181", +"iceberg.catalog.warehouse": "abfss://storage-container-name@storageaccount.dfs.core.windows.net/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.azure.adlsv2.ADLSFileIO", +"iceberg.catalog.include-credentials": "true" +``` + +Where `storage-container-name` is the container name within your Azure Storage Account, `/warehouse` is the location +within that container where your Apache Iceberg files will be written by default (or if iceberg.tables.auto-create-enabled=true), +and the `include-credentials` parameter passes along the Azure Java client credentials along. This will configure the +Iceberg Sink connector to connect to the REST catalog implementation at `iceberg.catalog.uri` to obtain the required +Connection String for the ADLSv2 client + +### Google GCS configuration example + +By default, Application Default Credentials (ADC) will be used to connect to GCS. Details on how ADC works can +be found in the [Google Cloud documentation](https://cloud.google.com/docs/authentication/application-default-credentials). + +``` +"iceberg.catalog.type": "rest", +"iceberg.catalog.uri": "https://catalog:8181", +"iceberg.catalog.warehouse": "gs://bucket-name/warehouse", +"iceberg.catalog.io-impl": "org.apache.iceberg.google.gcs.GCSFileIO" +``` + +### Hadoop configuration + +When using HDFS or Hive, the sink will initialize the Hadoop configuration. First, config files +from the classpath are loaded. Next, if `iceberg.hadoop-conf-dir` is specified, config files +are loaded from that location. Finally, any `iceberg.hadoop.*` properties from the sink config are +applied. When merging these, the order of precedence is sink config > config dir > classpath. + +## Examples + +### Initial setup + +#### Source topic + +This assumes the source topic already exists and is named `events`. + +#### Control topic + +If your Kafka cluster has `auto.create.topics.enable` set to `true` (the default), then the control topic will be +automatically created. If not, then you will need to create the topic first. The default topic name is `control-iceberg`: +```bash +bin/kafka-topics \ + --command-config command-config.props \ + --bootstrap-server ${CONNECT_BOOTSTRAP_SERVERS} \ + --create \ + --topic control-iceberg \ + --partitions 1 +``` +*NOTE: Clusters running on Confluent Cloud have `auto.create.topics.enable` set to `false` by default.* + +#### Iceberg catalog configuration + +Configuration properties with the prefix `iceberg.catalog.` will be passed to Iceberg catalog initialization. +See the [Iceberg docs](https://iceberg.apache.org/docs/latest/) for details on how to configure +a particular catalog. + +### Single destination table + +This example writes all incoming records to a single table. + +#### Create the destination table + +```sql +CREATE TABLE default.events ( + id STRING, + type STRING, + ts TIMESTAMP, + payload STRING) +PARTITIONED BY (hours(ts)) +``` + +#### Connector config + +This example config connects to a Iceberg REST catalog. +```json +{ +"name": "events-sink", +"config": { + "connector.class": "org.apache.iceberg.connect.IcebergSinkConnector", + "tasks.max": "2", + "topics": "events", + "iceberg.tables": "default.events", + "iceberg.catalog.type": "rest", + "iceberg.catalog.uri": "https://localhost", + "iceberg.catalog.credential": "", + "iceberg.catalog.warehouse": "" + } +} +``` + +### Multi-table fan-out, static routing + +This example writes records with `type` set to `list` to the table `default.events_list`, and +writes records with `type` set to `create` to the table `default.events_create`. Other records +will be skipped. + +#### Create two destination tables + +```sql +CREATE TABLE default.events_list ( + id STRING, + type STRING, + ts TIMESTAMP, + payload STRING) +PARTITIONED BY (hours(ts)); + +CREATE TABLE default.events_create ( + id STRING, + type STRING, + ts TIMESTAMP, + payload STRING) +PARTITIONED BY (hours(ts)); +``` + +#### Connector config + +```json +{ +"name": "events-sink", +"config": { + "connector.class": "org.apache.iceberg.connect.IcebergSinkConnector", + "tasks.max": "2", + "topics": "events", + "iceberg.tables": "default.events_list,default.events_create", + "iceberg.tables.route-field": "type", + "iceberg.table.default.events_list.route-regex": "list", + "iceberg.table.default.events_create.route-regex": "create", + "iceberg.catalog.type": "rest", + "iceberg.catalog.uri": "https://localhost", + "iceberg.catalog.credential": "", + "iceberg.catalog.warehouse": "" + } +} +``` + +### Multi-table fan-out, dynamic routing + +This example writes to tables with names from the value in the `db_table` field. If a table with +the name does not exist, then the record will be skipped. For example, if the record's `db_table` +field is set to `default.events_list`, then the record is written to the `default.events_list` table. + +#### Create two destination tables + +See above for creating two tables. + +#### Connector config + +```json +{ +"name": "events-sink", +"config": { + "connector.class": "org.apache.iceberg.connect.IcebergSinkConnector", + "tasks.max": "2", + "topics": "events", + "iceberg.tables.dynamic-enabled": "true", + "iceberg.tables.route-field": "db_table", + "iceberg.catalog.type": "rest", + "iceberg.catalog.uri": "https://localhost", + "iceberg.catalog.credential": "", + "iceberg.catalog.warehouse": "" + } +} +``` diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index 604fede583d8..edafb727d3d8 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -64,6 +64,7 @@ nav: - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html - Doris: https://doris.apache.org/docs/dev/lakehouse/datalake-analytics/iceberg - Druid: https://druid.apache.org/docs/latest/development/extensions-contrib/iceberg/ + - Kafka Connect: kafka-connect.md - Integrations: - aws.md - dell.md From 779685d4dc1329986d9824aa5baf8bc94a410279 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Tue, 10 Sep 2024 15:54:20 -0700 Subject: [PATCH 0682/1019] Kafka Connect: Terminate commits on coordinator stop (#10814) --- .../iceberg/connect/channel/Coordinator.java | 19 ++++++++++++------- .../connect/channel/CoordinatorThread.java | 10 +++++----- .../iceberg/connect/data/RecordConverter.java | 9 +++++---- 3 files changed, 22 insertions(+), 16 deletions(-) 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 7274f77e0c85..2a00928310b2 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 @@ -53,6 +53,7 @@ import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.kafka.clients.admin.MemberDescription; +import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.sink.SinkTaskContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -71,6 +72,7 @@ class Coordinator extends Channel { private final String snapshotOffsetsProp; private final ExecutorService exec; private final CommitState commitState; + private volatile boolean terminated; Coordinator( Catalog catalog, @@ -218,6 +220,10 @@ private void commitToTable( .filter(distinctByKey(deleteFile -> deleteFile.path().toString())) .collect(Collectors.toList()); + if (terminated) { + throw new ConnectException("Coordinator is terminated, commit aborted"); + } + if (dataFiles.isEmpty() && deleteFiles.isEmpty()) { LOG.info("Nothing to commit to table {}, skipping", tableIdentifier); } else { @@ -296,19 +302,18 @@ private Map lastCommittedOffsetsForTable(Table table, String bran return ImmutableMap.of(); } - @Override - void stop() { + void terminate() { + this.terminated = true; + exec.shutdownNow(); - // ensure coordinator tasks are shut down, else cause the sink worker to fail + // wait for coordinator termination, else cause the sink task to fail try { if (!exec.awaitTermination(1, TimeUnit.MINUTES)) { - throw new RuntimeException("Timed out waiting for coordinator shutdown"); + throw new ConnectException("Timed out waiting for coordinator shutdown"); } } catch (InterruptedException e) { - throw new RuntimeException("Interrupted while waiting for coordinator shutdown", e); + throw new ConnectException("Interrupted while waiting for coordinator shutdown", e); } - - super.stop(); } } diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java index 6a31b17fc606..b1a34d0474d9 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/channel/CoordinatorThread.java @@ -25,7 +25,7 @@ class CoordinatorThread extends Thread { private static final Logger LOG = LoggerFactory.getLogger(CoordinatorThread.class); private static final String THREAD_NAME = "iceberg-coord"; - private Coordinator coordinator; + private final Coordinator coordinator; private volatile boolean terminated; CoordinatorThread(Coordinator coordinator) { @@ -39,7 +39,7 @@ public void run() { coordinator.start(); } catch (Exception e) { LOG.error("Coordinator error during start, exiting thread", e); - terminated = true; + this.terminated = true; } while (!terminated) { @@ -47,7 +47,7 @@ public void run() { coordinator.process(); } catch (Exception e) { LOG.error("Coordinator error during process, exiting thread", e); - terminated = true; + this.terminated = true; } } @@ -56,7 +56,6 @@ public void run() { } catch (Exception e) { LOG.error("Coordinator error during stop, ignoring", e); } - coordinator = null; } boolean isTerminated() { @@ -64,6 +63,7 @@ boolean isTerminated() { } void terminate() { - terminated = true; + this.terminated = true; + coordinator.terminate(); } } 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 406a2cba4526..f827d2fd63f4 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 @@ -61,6 +61,7 @@ import org.apache.iceberg.types.Types.TimestampType; import org.apache.iceberg.util.DateTimeUtil; import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.errors.ConnectException; class RecordConverter { @@ -421,7 +422,7 @@ protected LocalDate convertDateValue(Object value) { int days = (int) (((Date) value).getTime() / 1000 / 60 / 60 / 24); return DateTimeUtil.dateFromDays(days); } - throw new RuntimeException("Cannot convert date: " + value); + throw new ConnectException("Cannot convert date: " + value); } @SuppressWarnings("JavaUtilDate") @@ -437,7 +438,7 @@ protected LocalTime convertTimeValue(Object value) { long millis = ((Date) value).getTime(); return DateTimeUtil.timeFromMicros(millis * 1000); } - throw new RuntimeException("Cannot convert time: " + value); + throw new ConnectException("Cannot convert time: " + value); } protected Temporal convertTimestampValue(Object value, TimestampType type) { @@ -461,7 +462,7 @@ private OffsetDateTime convertOffsetDateTime(Object value) { } else if (value instanceof Date) { return DateTimeUtil.timestamptzFromMicros(((Date) value).getTime() * 1000); } - throw new RuntimeException( + throw new ConnectException( "Cannot convert timestamptz: " + value + ", type: " + value.getClass()); } @@ -489,7 +490,7 @@ private LocalDateTime convertLocalDateTime(Object value) { } else if (value instanceof Date) { return DateTimeUtil.timestampFromMicros(((Date) value).getTime() * 1000); } - throw new RuntimeException( + throw new ConnectException( "Cannot convert timestamp: " + value + ", type: " + value.getClass()); } From 1ebda780b8139f937a4d7118bba3c257f467c543 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Wed, 11 Sep 2024 18:08:49 +0800 Subject: [PATCH 0683/1019] Build: Upgrade google-java-format to 1.22.0 (#11050) --- .../main/java/org/apache/iceberg/DataFile.java | 5 ++++- .../org/apache/iceberg/UpdatePartitionSpec.java | 2 +- .../org/apache/iceberg/encryption/KmsClient.java | 4 +++- .../apache/iceberg/transforms/Transforms.java | 4 +++- .../org/apache/iceberg/transforms/Truncate.java | 4 +++- .../java/org/apache/iceberg/TestHelpers.java | 1 + .../apache/iceberg/aws/glue/GlueTestBase.java | 4 +--- .../aws/glue/TestGlueCatalogCommitFailure.java | 3 ++- .../iceberg/aws/s3/TestS3FileIOIntegration.java | 6 ++---- .../apache/iceberg/aws/HttpClientProperties.java | 12 ++++++++++++ .../iceberg/aws/s3/S3FileIOAwsClientFactory.java | 1 + .../org/apache/iceberg/aws/s3/TestS3FileIO.java | 3 +-- baseline.gradle | 16 ++++------------ build.gradle | 2 +- .../apache/iceberg/common/DynConstructors.java | 4 +++- .../org/apache/iceberg/common/DynMethods.java | 4 +++- .../org/apache/iceberg/CatalogProperties.java | 1 + .../org/apache/iceberg/GenericManifestEntry.java | 12 +++++++++--- .../java/org/apache/iceberg/ManifestEntry.java | 1 + .../apache/iceberg/ManifestFilterManager.java | 4 +++- .../java/org/apache/iceberg/ManifestReader.java | 4 +++- .../java/org/apache/iceberg/SystemConfigs.java | 4 +++- .../java/org/apache/iceberg/TableProperties.java | 16 ++++++++++++---- .../apache/iceberg/TestMetadataTableFilters.java | 4 +++- .../apache/iceberg/hadoop/HadoopFileIOTest.java | 6 ++---- .../iceberg/dell/ecs/EcsSeekableInputStream.java | 1 + .../apache/iceberg/flink/FlinkSchemaUtil.java | 4 +++- .../shuffle/AggregatedStatisticsTracker.java | 4 +++- .../shuffle/RequestGlobalStatisticsEvent.java | 4 +++- .../apache/iceberg/flink/source/FlinkSource.java | 4 +++- .../iceberg/flink/source/IcebergSource.java | 4 +++- .../enumerator/ContinuousIcebergEnumerator.java | 1 + .../sink/shuffle/TestMapRangePartitioner.java | 4 +++- .../TestContinuousSplitPlannerImpl.java | 4 +++- .../apache/iceberg/flink/FlinkSchemaUtil.java | 4 +++- .../shuffle/AggregatedStatisticsTracker.java | 4 +++- .../shuffle/RequestGlobalStatisticsEvent.java | 4 +++- .../apache/iceberg/flink/source/FlinkSource.java | 4 +++- .../iceberg/flink/source/IcebergSource.java | 4 +++- .../enumerator/ContinuousIcebergEnumerator.java | 1 + .../iceberg/flink/sink/TestIcebergCommitter.java | 2 +- .../sink/shuffle/TestMapRangePartitioner.java | 4 +++- .../TestContinuousSplitPlannerImpl.java | 4 +++- .../apache/iceberg/flink/FlinkSchemaUtil.java | 4 +++- .../shuffle/AggregatedStatisticsTracker.java | 4 +++- .../shuffle/RequestGlobalStatisticsEvent.java | 4 +++- .../apache/iceberg/flink/source/FlinkSource.java | 4 +++- .../iceberg/flink/source/IcebergSource.java | 4 +++- .../enumerator/ContinuousIcebergEnumerator.java | 1 + .../iceberg/flink/sink/TestIcebergCommitter.java | 2 +- .../sink/shuffle/TestMapRangePartitioner.java | 4 +++- .../TestContinuousSplitPlannerImpl.java | 4 +++- .../org/apache/iceberg/gcp/GCPProperties.java | 1 + .../apache/hadoop/hive/ql/io/orc/OrcSplit.java | 1 + .../main/java/org/apache/iceberg/orc/ORC.java | 4 +++- .../org/apache/iceberg/orc/ORCSchemaUtil.java | 1 + .../java/org/apache/iceberg/parquet/Parquet.java | 8 ++++++-- .../iceberg/snowflake/SnowflakeCatalog.java | 1 + .../extensions/TestSnapshotTableProcedure.java | 1 + .../source/TestIcebergSourceTablesBase.java | 9 +++++++-- .../extensions/TestSnapshotTableProcedure.java | 1 + .../source/TestIcebergSourceTablesBase.java | 9 +++++++-- .../extensions/TestSnapshotTableProcedure.java | 1 + .../source/TestIcebergSourceTablesBase.java | 9 +++++++-- 64 files changed, 188 insertions(+), 77 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/DataFile.java b/api/src/main/java/org/apache/iceberg/DataFile.java index 59b329c500c7..02ad0aff3128 100644 --- a/api/src/main/java/org/apache/iceberg/DataFile.java +++ b/api/src/main/java/org/apache/iceberg/DataFile.java @@ -102,6 +102,7 @@ public interface DataFile extends ContentFile { 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 static StructType getType(StructType partitionType) { @@ -126,7 +127,9 @@ static StructType getType(StructType partitionType) { SORT_ORDER_ID); } - /** @return the content stored in the file; one of DATA, POSITION_DELETES, or EQUALITY_DELETES */ + /** + * @return the content stored in the file; one of DATA, POSITION_DELETES, or EQUALITY_DELETES + */ @Override default FileContent content() { return FileContent.DATA; diff --git a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java index eeb596d42d5c..a4994d22001d 100644 --- a/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java +++ b/api/src/main/java/org/apache/iceberg/UpdatePartitionSpec.java @@ -132,5 +132,5 @@ public interface UpdatePartitionSpec extends PendingUpdate { default UpdatePartitionSpec addNonDefaultSpec() { throw new UnsupportedOperationException( this.getClass().getName() + " doesn't implement addNonDefaultSpec()"); - }; + } } diff --git a/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java b/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java index 3ebda7be27f2..87dd2b286b12 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java +++ b/api/src/main/java/org/apache/iceberg/encryption/KmsClient.java @@ -23,7 +23,9 @@ import java.util.Map; /** A minimum client interface to connect to a key management service (KMS). */ -/** @deprecated the API will be removed in v2.0.0 (replaced with KeyManagementClient interface). */ +/** + * @deprecated the API will be removed in v2.0.0 (replaced with KeyManagementClient interface). + */ @Deprecated public interface KmsClient extends Serializable { diff --git a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java index 11282efdefb1..aacd4d430069 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Transforms.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Transforms.java @@ -67,7 +67,9 @@ private Transforms() {} return new UnknownTransform<>(transform); } - /** @deprecated use {@link #identity()} instead; will be removed in 2.0.0 */ + /** + * @deprecated use {@link #identity()} instead; will be removed in 2.0.0 + */ @Deprecated public static Transform fromString(Type type, String transform) { Matcher widthMatcher = HAS_WIDTH.matcher(transform); diff --git a/api/src/main/java/org/apache/iceberg/transforms/Truncate.java b/api/src/main/java/org/apache/iceberg/transforms/Truncate.java index 670c6002a97a..a111e4ca394b 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/Truncate.java +++ b/api/src/main/java/org/apache/iceberg/transforms/Truncate.java @@ -43,7 +43,9 @@ static Truncate get(int width) { return new Truncate<>(width); } - /** @deprecated will be removed in 2.0.0 */ + /** + * @deprecated will be removed in 2.0.0 + */ @Deprecated @SuppressWarnings("unchecked") static & SerializableFunction> R get(Type type, int width) { diff --git a/api/src/test/java/org/apache/iceberg/TestHelpers.java b/api/src/test/java/org/apache/iceberg/TestHelpers.java index 329a61fb44b3..ca3b1a908ac6 100644 --- a/api/src/test/java/org/apache/iceberg/TestHelpers.java +++ b/api/src/test/java/org/apache/iceberg/TestHelpers.java @@ -222,6 +222,7 @@ public static T deserialize(final InputStream inputStream) return obj; } } + /** * Serializes an {@link Object} to a byte array for storage/serialization. * 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 495c5ca98e94..29076369c8f5 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 @@ -181,9 +181,7 @@ public static void updateTableColumns( .owner(existingTable.owner()) .parameters(existingTable.parameters()) .storageDescriptor( - existingTable - .storageDescriptor() - .toBuilder() + existingTable.storageDescriptor().toBuilder() .columns(updatedColumns) .build()) .build()) diff --git a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java index 42b527a03742..a1df3d21da69 100644 --- a/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java +++ b/aws/src/integration/java/org/apache/iceberg/aws/glue/TestGlueCatalogCommitFailure.java @@ -531,7 +531,8 @@ private int metadataFileCount(TableMetadata metadata) { new File(S3TestUtil.getKeyFromUri(metadata.metadataFileLocation())) .getParent()) .build()) - .contents().stream() + .contents() + .stream() .filter(s3Object -> s3Object.key().endsWith("metadata.json")) .count(); } 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 b8b86fddc596..388260a54657 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 @@ -459,8 +459,7 @@ public void testPrefixList() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); String listPrefix = String.format("s3://%s/%s/%s", bucketName, prefix, "prefix-list-test"); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { String scalePrefix = String.format("%s/%s/", listPrefix, scale); @@ -481,8 +480,7 @@ public void testPrefixDelete() { String deletePrefix = String.format("s3://%s/%s/%s", bucketName, prefix, "prefix-delete-test"); List scaleSizes = Lists.newArrayList(0, 5, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { String scalePrefix = String.format("%s/%s/", deletePrefix, scale); diff --git a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java index 3e615cad7024..438ae5bb0431 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java +++ b/aws/src/main/java/org/apache/iceberg/aws/HttpClientProperties.java @@ -44,6 +44,7 @@ public class HttpClientProperties implements Serializable { public static final String CLIENT_TYPE_APACHE = "apache"; private static final String CLIENT_PREFIX = "http-client."; + /** * If this is set under {@link #CLIENT_TYPE}, {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient} will be used as the HTTP @@ -59,6 +60,7 @@ public class HttpClientProperties implements Serializable { * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder} */ public static final String PROXY_ENDPOINT = "http-client.proxy-endpoint"; + /** * Used to configure the connection timeout in milliseconds for {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only @@ -69,6 +71,7 @@ public class HttpClientProperties implements Serializable { */ public static final String URLCONNECTION_CONNECTION_TIMEOUT_MS = "http-client.urlconnection.connection-timeout-ms"; + /** * Used to configure the socket timeout in milliseconds for {@link * software.amazon.awssdk.http.urlconnection.UrlConnectionHttpClient.Builder}. This flag only @@ -79,6 +82,7 @@ public class HttpClientProperties implements Serializable { */ public static final String URLCONNECTION_SOCKET_TIMEOUT_MS = "http-client.urlconnection.socket-timeout-ms"; + /** * Used to configure the connection timeout in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -89,6 +93,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_TIMEOUT_MS = "http-client.apache.connection-timeout-ms"; + /** * Used to configure the socket timeout in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -98,6 +103,7 @@ public class HttpClientProperties implements Serializable { * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html */ public static final String APACHE_SOCKET_TIMEOUT_MS = "http-client.apache.socket-timeout-ms"; + /** * Used to configure the connection acquisition timeout in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -108,6 +114,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_ACQUISITION_TIMEOUT_MS = "http-client.apache.connection-acquisition-timeout-ms"; + /** * Used to configure the connection max idle time in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -118,6 +125,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_MAX_IDLE_TIME_MS = "http-client.apache.connection-max-idle-time-ms"; + /** * Used to configure the connection time to live in milliseconds for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -128,6 +136,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_CONNECTION_TIME_TO_LIVE_MS = "http-client.apache.connection-time-to-live-ms"; + /** * Used to configure whether to enable the expect continue setting for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -140,6 +149,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_EXPECT_CONTINUE_ENABLED = "http-client.apache.expect-continue-enabled"; + /** * Used to configure the max connections number for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -149,6 +159,7 @@ public class HttpClientProperties implements Serializable { * https://sdk.amazonaws.com/java/api/latest/software/amazon/awssdk/http/apache/ApacheHttpClient.Builder.html */ public static final String APACHE_MAX_CONNECTIONS = "http-client.apache.max-connections"; + /** * Used to configure whether to enable the tcp keep alive setting for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link @@ -161,6 +172,7 @@ public class HttpClientProperties implements Serializable { */ public static final String APACHE_TCP_KEEP_ALIVE_ENABLED = "http-client.apache.tcp-keep-alive-enabled"; + /** * Used to configure whether to use idle connection reaper for {@link * software.amazon.awssdk.http.apache.ApacheHttpClient.Builder}. This flag only works when {@link diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java index 411e79f58fa8..718298818a30 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/S3FileIOAwsClientFactory.java @@ -29,6 +29,7 @@ public interface S3FileIOAwsClientFactory extends Serializable { * @return s3 client */ S3Client s3(); + /** * Initialize AWS client factory from catalog properties. * 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 1cc34a59cc1b..6caa42fb410b 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 @@ -243,8 +243,7 @@ public void testPrefixList() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { String scalePrefix = String.format("%s/%s/", prefix, scale); diff --git a/baseline.gradle b/baseline.gradle index 5e2710be1749..d29100aa3f31 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -50,16 +50,7 @@ subprojects { t.setDuplicatesStrategy(DuplicatesStrategy.WARN); }); apply plugin: 'com.palantir.baseline-exact-dependencies' - // We need to update Google Java Format to 1.17.0+ to run spotless on JDK 8, but that requires dropping support for JDK 8. - if (JavaVersion.current() == JavaVersion.VERSION_21) { - task spotlessApply { - doLast { - throw new GradleException("Spotless plugin is currently disabled when running on JDK 21 (until we drop JDK 8). To run spotlessApply please use a different JDK version.") - } - } - } else { - apply plugin: 'com.diffplug.spotless' - } + apply plugin: 'com.diffplug.spotless' pluginManager.withPlugin('com.palantir.baseline-checkstyle') { checkstyle { @@ -74,8 +65,9 @@ subprojects { spotless { java { target 'src/main/java/**/*.java', 'src/test/java/**/*.java', 'src/testFixtures/java/**/*.java', 'src/jmh/java/**/*.java', 'src/integration/java/**/*.java' - // we use an older version of google-java-format that is compatible with JDK 8 - googleJavaFormat("1.7") + // 1.23.0 has an issue in formatting comments https://github.com/google/google-java-format/issues/1155 + // so we stick to 1.22.0 to produce consistent result for JDK 11/17/21 + googleJavaFormat("1.22.0") removeUnusedImports() licenseHeaderFile "$rootDir/.baseline/copyright/copyright-header-java.txt" } diff --git a/build.gradle b/build.gradle index fab949bf06a6..02758e2a793b 100644 --- a/build.gradle +++ b/build.gradle @@ -28,7 +28,7 @@ buildscript { dependencies { classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' classpath 'com.palantir.baseline:gradle-baseline-java:5.61.0' - classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.13.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' classpath 'gradle.plugin.io.morethan.jmhreport:gradle-jmh-report:0.9.6' diff --git a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java index 7ec8716c86a4..ea2fca931f13 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynConstructors.java +++ b/common/src/main/java/org/apache/iceberg/common/DynConstructors.java @@ -76,7 +76,9 @@ public R invoke(Object target, Object... args) { return (R) newInstance(args); } - /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + /** + * @deprecated since 1.7.0, visibility will be reduced in 1.8.0 + */ @Deprecated // will become package-private @Override @SuppressWarnings("unchecked") diff --git a/common/src/main/java/org/apache/iceberg/common/DynMethods.java b/common/src/main/java/org/apache/iceberg/common/DynMethods.java index 65a69bd0e12c..5972cdf8dc23 100644 --- a/common/src/main/java/org/apache/iceberg/common/DynMethods.java +++ b/common/src/main/java/org/apache/iceberg/common/DynMethods.java @@ -125,7 +125,9 @@ public String toString() { /** Singleton {@link UnboundMethod}, performs no operation and returns null. */ private static final UnboundMethod NOOP = new UnboundMethod(null, "NOOP") { - /** @deprecated since 1.7.0, visibility will be reduced in 1.8.0 */ + /** + * @deprecated since 1.7.0, visibility will be reduced in 1.8.0 + */ @Deprecated // will become package-private @Override public R invokeChecked(Object target, Object... args) { diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index b6fd990f0ac6..339c59b45d1b 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -124,6 +124,7 @@ private CatalogProperties() {} "client.pool.cache.eviction-interval-ms"; public static final long CLIENT_POOL_CACHE_EVICTION_INTERVAL_MS_DEFAULT = TimeUnit.MINUTES.toMillis(5); + /** * A comma separated list of elements used, in addition to the {@link #URI}, to compose the key of * the client pool cache. diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java index 959e2446c710..575ea1f1c518 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java @@ -94,13 +94,17 @@ ManifestEntry wrapDelete( return this; } - /** @return the status of the file, whether EXISTING, ADDED, or DELETED */ + /** + * @return the status of the file, whether EXISTING, ADDED, or DELETED + */ @Override public Status status() { return status; } - /** @return id of the snapshot in which the file was added to the table */ + /** + * @return id of the snapshot in which the file was added to the table + */ @Override public Long snapshotId() { return snapshotId; @@ -116,7 +120,9 @@ public Long fileSequenceNumber() { return fileSequenceNumber; } - /** @return a file */ + /** + * @return a file + */ @Override public F file() { return file; diff --git a/core/src/main/java/org/apache/iceberg/ManifestEntry.java b/core/src/main/java/org/apache/iceberg/ManifestEntry.java index 3eeb5407ab20..4dce92cf5c2f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/ManifestEntry.java @@ -48,6 +48,7 @@ public int id() { Types.NestedField FILE_SEQUENCE_NUMBER = optional(4, "file_sequence_number", Types.LongType.get()); int DATA_FILE_ID = 2; + // next ID to assign: 5 static Schema getSchema(StructType partitionType) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 3b4c5e57e31e..106be74fa3ad 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -289,7 +289,9 @@ private void invalidateFilteredCache() { cleanUncommitted(SnapshotProducer.EMPTY_SET); } - /** @return a ManifestReader that is a filtered version of the input manifest. */ + /** + * @return a ManifestReader that is a filtered version of the input manifest. + */ private ManifestFile filterManifest(Schema tableSchema, ManifestFile manifest) { ManifestFile cached = filteredManifests.get(manifest); if (cached != null) { diff --git a/core/src/main/java/org/apache/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java index b5f85813dd2f..6364603c591f 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -292,7 +292,9 @@ private boolean isLiveEntry(ManifestEntry entry) { return entry != null && entry.status() != ManifestEntry.Status.DELETED; } - /** @return an Iterator of DataFile. Makes defensive copies of files before returning */ + /** + * @return an Iterator of DataFile. Makes defensive copies of files before returning + */ @Override public CloseableIterator iterator() { boolean dropStats = dropStats(columns); diff --git a/core/src/main/java/org/apache/iceberg/SystemConfigs.java b/core/src/main/java/org/apache/iceberg/SystemConfigs.java index 9cb345b44480..6dd447175244 100644 --- a/core/src/main/java/org/apache/iceberg/SystemConfigs.java +++ b/core/src/main/java/org/apache/iceberg/SystemConfigs.java @@ -72,7 +72,9 @@ private SystemConfigs() {} 8, Integer::parseUnsignedInt); - /** @deprecated will be removed in 2.0.0; use name mapping instead */ + /** + * @deprecated will be removed in 2.0.0; use name mapping instead + */ @Deprecated public static final ConfigEntry NETFLIX_UNSAFE_PARQUET_ID_FALLBACK_ENABLED = new ConfigEntry<>( diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index e41230081615..1652e9312f50 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -244,12 +244,16 @@ private TableProperties() {} public static final String OBJECT_STORE_ENABLED = "write.object-storage.enabled"; public static final boolean OBJECT_STORE_ENABLED_DEFAULT = false; - /** @deprecated Use {@link #WRITE_DATA_LOCATION} instead. */ + /** + * @deprecated Use {@link #WRITE_DATA_LOCATION} instead. + */ @Deprecated public static final String OBJECT_STORE_PATH = "write.object-storage.path"; public static final String WRITE_LOCATION_PROVIDER_IMPL = "write.location-provider.impl"; - /** @deprecated Use {@link #WRITE_DATA_LOCATION} instead. */ + /** + * @deprecated Use {@link #WRITE_DATA_LOCATION} instead. + */ @Deprecated public static final String WRITE_FOLDER_STORAGE_LOCATION = "write.folder-storage.path"; @@ -266,10 +270,14 @@ private TableProperties() {} public static final String WRITE_PARTITION_SUMMARY_LIMIT = "write.summary.partition-limit"; public static final int WRITE_PARTITION_SUMMARY_LIMIT_DEFAULT = 0; - /** @deprecated will be removed in 2.0.0, writing manifest lists is always enabled */ + /** + * @deprecated will be removed in 2.0.0, writing manifest lists is always enabled + */ @Deprecated public static final String MANIFEST_LISTS_ENABLED = "write.manifest-lists.enabled"; - /** @deprecated will be removed in 2.0.0, writing manifest lists is always enabled */ + /** + * @deprecated will be removed in 2.0.0, writing manifest lists is always enabled + */ @Deprecated public static final boolean MANIFEST_LISTS_ENABLED_DEFAULT = true; public static final String METADATA_COMPRESSION = "write.metadata.compression-codec"; diff --git a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java index bf6456e85aec..f8c34019875f 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java +++ b/core/src/test/java/org/apache/iceberg/TestMetadataTableFilters.java @@ -169,7 +169,9 @@ private String partitionColumn(String colName) { } } - /** @return a basic expression that always evaluates to true, to test AND logic */ + /** + * @return a basic expression that always evaluates to true, to test AND logic + */ private Expression dummyExpression() { switch (type) { case FILES: 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 109c88daeaa5..1abbd94839de 100644 --- a/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java +++ b/core/src/test/java/org/apache/iceberg/hadoop/HadoopFileIOTest.java @@ -66,8 +66,7 @@ public void testListPrefix() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { Path scalePath = new Path(parent, Integer.toString(scale)); @@ -101,8 +100,7 @@ public void testDeletePrefix() { List scaleSizes = Lists.newArrayList(1, 1000, 2500); - scaleSizes - .parallelStream() + scaleSizes.parallelStream() .forEach( scale -> { Path scalePath = new Path(parent, Integer.toString(scale)); diff --git a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java index e59b03ae44bc..6d32d3ba981d 100644 --- a/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java +++ b/dell/src/main/java/org/apache/iceberg/dell/ecs/EcsSeekableInputStream.java @@ -46,6 +46,7 @@ class EcsSeekableInputStream extends SeekableInputStream { /** Mutable pos set by {@link #seek(long)} */ private long newPos = 0; + /** Current pos of object content */ private long pos = -1; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 4790dc85bf28..afc09a719f77 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -58,7 +58,9 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + /** + * @deprecated Use {@link #convert(ResolvedSchema)} instead. + */ @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 5525f02c873e..95c2328f032a 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -186,7 +186,9 @@ private boolean isComplete() { return subtaskSet.size() == parallelism; } - /** @return false if duplicate */ + /** + * @return false if duplicate + */ private boolean merge(int subtask, DataStatistics taskStatistics) { if (subtaskSet.contains(subtask)) { return false; diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java index ce17e1964392..ce94bec14860 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -27,7 +27,9 @@ class RequestGlobalStatisticsEvent implements OperatorEvent { this.signature = null; } - /** @param signature hashCode of the subtask's existing global statistics */ + /** + * @param signature hashCode of the subtask's existing global statistics + */ RequestGlobalStatisticsEvent(int signature) { this.signature = signature; } 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 b1431a32dd20..228df74412a2 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 @@ -124,7 +124,9 @@ public Builder setAll(Map properties) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); 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 5718f4b93825..294ca2f55bc5 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 @@ -547,7 +547,9 @@ public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index c7021b9c6847..c50c3854ee14 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -37,6 +37,7 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to * control the total number of snapshots worth of splits tracked by assigner. diff --git a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d5a0bebc74e7..c95a9955b104 100644 --- a/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.18/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -380,7 +380,9 @@ private static Map>> runPartitioner( return partitionResults; } - /** @param expectedAssignmentInfo excluding closing cost */ + /** + * @param expectedAssignmentInfo excluding closing cost + */ private void validatePartitionResults( Map> expectedAssignmentInfo, Map>> partitionResults, 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 0690b456e033..5767fa02c822 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 @@ -80,7 +80,9 @@ private void appendTwoSnapshots() throws IOException { snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } - /** @return the last enumerated snapshot id */ + /** + * @return the last enumerated snapshot id + */ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 4790dc85bf28..afc09a719f77 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -58,7 +58,9 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + /** + * @deprecated Use {@link #convert(ResolvedSchema)} instead. + */ @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 5525f02c873e..95c2328f032a 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -186,7 +186,9 @@ private boolean isComplete() { return subtaskSet.size() == parallelism; } - /** @return false if duplicate */ + /** + * @return false if duplicate + */ private boolean merge(int subtask, DataStatistics taskStatistics) { if (subtaskSet.contains(subtask)) { return false; diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java index ce17e1964392..ce94bec14860 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -27,7 +27,9 @@ class RequestGlobalStatisticsEvent implements OperatorEvent { this.signature = null; } - /** @param signature hashCode of the subtask's existing global statistics */ + /** + * @param signature hashCode of the subtask's existing global statistics + */ RequestGlobalStatisticsEvent(int signature) { this.signature = signature; } 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 b1431a32dd20..228df74412a2 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 @@ -124,7 +124,9 @@ public Builder setAll(Map properties) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); 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 5718f4b93825..294ca2f55bc5 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 @@ -547,7 +547,9 @@ public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index c7021b9c6847..c50c3854ee14 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -37,6 +37,7 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to * control the total number of snapshots worth of splits tracked by assigner. diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java index abdb2b5c79ae..f11aae1d6923 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -1441,5 +1441,5 @@ public int hashCode() { public TypeSerializerSnapshot> snapshotConfiguration() { return null; } - }; + } } diff --git a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d5a0bebc74e7..c95a9955b104 100644 --- a/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -380,7 +380,9 @@ private static Map>> runPartitioner( return partitionResults; } - /** @param expectedAssignmentInfo excluding closing cost */ + /** + * @param expectedAssignmentInfo excluding closing cost + */ private void validatePartitionResults( Map> expectedAssignmentInfo, Map>> partitionResults, 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 0690b456e033..5767fa02c822 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 @@ -80,7 +80,9 @@ private void appendTwoSnapshots() throws IOException { snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } - /** @return the last enumerated snapshot id */ + /** + * @return the last enumerated snapshot id + */ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java index 4790dc85bf28..afc09a719f77 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/FlinkSchemaUtil.java @@ -58,7 +58,9 @@ public class FlinkSchemaUtil { private FlinkSchemaUtil() {} - /** @deprecated Use {@link #convert(ResolvedSchema)} instead. */ + /** + * @deprecated Use {@link #convert(ResolvedSchema)} instead. + */ @Deprecated public static Schema convert(TableSchema schema) { LogicalType schemaType = schema.toRowDataType().getLogicalType(); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java index 5525f02c873e..95c2328f032a 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/AggregatedStatisticsTracker.java @@ -186,7 +186,9 @@ private boolean isComplete() { return subtaskSet.size() == parallelism; } - /** @return false if duplicate */ + /** + * @return false if duplicate + */ private boolean merge(int subtask, DataStatistics taskStatistics) { if (subtaskSet.contains(subtask)) { return false; diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java index ce17e1964392..ce94bec14860 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/shuffle/RequestGlobalStatisticsEvent.java @@ -27,7 +27,9 @@ class RequestGlobalStatisticsEvent implements OperatorEvent { this.signature = null; } - /** @param signature hashCode of the subtask's existing global statistics */ + /** + * @param signature hashCode of the subtask's existing global statistics + */ RequestGlobalStatisticsEvent(int signature) { this.signature = signature; } 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 b1431a32dd20..228df74412a2 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 @@ -124,7 +124,9 @@ public Builder setAll(Map properties) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); 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 5718f4b93825..294ca2f55bc5 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 @@ -547,7 +547,9 @@ public Builder watermarkColumnTimeUnit(TimeUnit timeUnit) { return this; } - /** @deprecated Use {@link #setAll} instead. */ + /** + * @deprecated Use {@link #setAll} instead. + */ @Deprecated public Builder properties(Map properties) { readOptions.putAll(properties); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java index c7021b9c6847..c50c3854ee14 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousIcebergEnumerator.java @@ -37,6 +37,7 @@ public class ContinuousIcebergEnumerator extends AbstractIcebergEnumerator { private static final Logger LOG = LoggerFactory.getLogger(ContinuousIcebergEnumerator.class); + /** * This is hardcoded, as {@link ScanContext#maxPlanningSnapshotCount()} could be the knob to * control the total number of snapshots worth of splits tracked by assigner. diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java index abdb2b5c79ae..f11aae1d6923 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/TestIcebergCommitter.java @@ -1441,5 +1441,5 @@ public int hashCode() { public TypeSerializerSnapshot> snapshotConfiguration() { return null; } - }; + } } diff --git a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java index d5a0bebc74e7..c95a9955b104 100644 --- a/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/sink/shuffle/TestMapRangePartitioner.java @@ -380,7 +380,9 @@ private static Map>> runPartitioner( return partitionResults; } - /** @param expectedAssignmentInfo excluding closing cost */ + /** + * @param expectedAssignmentInfo excluding closing cost + */ private void validatePartitionResults( Map> expectedAssignmentInfo, Map>> partitionResults, 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 0690b456e033..5767fa02c822 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 @@ -80,7 +80,9 @@ private void appendTwoSnapshots() throws IOException { snapshot2 = TABLE_RESOURCE.table().currentSnapshot(); } - /** @return the last enumerated snapshot id */ + /** + * @return the last enumerated snapshot id + */ private CycleResult verifyOneCycle( ContinuousSplitPlannerImpl splitPlanner, IcebergEnumeratorPosition lastPosition) throws Exception { 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 4465ee29012a..4f60e2f91f91 100644 --- a/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java +++ b/gcp/src/main/java/org/apache/iceberg/gcp/GCPProperties.java @@ -46,6 +46,7 @@ public class GCPProperties implements Serializable { /** 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"; + /** * Max possible batch size for deletion. Currently, a max of 100 keys is advised, so we default to * a number below that. https://cloud.google.com/storage/docs/batch diff --git a/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java b/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java index 4f2cdf0282e2..4031bfaa20f6 100644 --- a/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java +++ b/hive3/src/main/java/org/apache/hadoop/hive/ql/io/orc/OrcSplit.java @@ -51,6 +51,7 @@ public class OrcSplit extends FileSplit implements ColumnarSplit, LlapAwareSplit private static final Logger LOG = LoggerFactory.getLogger(OrcSplit.class); private OrcTail orcTail; private boolean hasFooter; + /** This means {@link AcidUtils.AcidBaseFileType#ORIGINAL_BASE} */ private boolean isOriginal; diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORC.java b/orc/src/main/java/org/apache/iceberg/orc/ORC.java index 18186fe3f20e..451c670fcd54 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORC.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORC.java @@ -95,7 +95,9 @@ @SuppressWarnings("checkstyle:AbbreviationAsWordInName") public class ORC { - /** @deprecated use {@link TableProperties#ORC_WRITE_BATCH_SIZE} instead */ + /** + * @deprecated use {@link TableProperties#ORC_WRITE_BATCH_SIZE} instead + */ @Deprecated private static final String VECTOR_ROW_BATCH_SIZE = "iceberg.orc.vectorbatch.size"; private ORC() {} diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java index ba6d6eedd1d4..a573d6874851 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java @@ -73,6 +73,7 @@ public TypeDescription type() { * to an ORC binary type. The values for this attribute are denoted in {@code BinaryType}. */ public static final String ICEBERG_BINARY_TYPE_ATTRIBUTE = "iceberg.binary-type"; + /** * The name of the ORC {@link TypeDescription} attribute indicating the Iceberg type corresponding * to an ORC long type. The values for this attribute are denoted in {@code LongType}. diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java index 3421c0b86d3a..a3a59318320b 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/Parquet.java @@ -1090,7 +1090,9 @@ public ReadBuilder filter(Expression newFilter) { return this; } - /** @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead */ + /** + * @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead + */ @Deprecated public ReadBuilder readSupport(ReadSupport newFilterSupport) { this.readSupport = newFilterSupport; @@ -1119,7 +1121,9 @@ public ReadBuilder set(String key, String value) { return this; } - /** @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead */ + /** + * @deprecated will be removed in 2.0.0; use {@link #createReaderFunc(Function)} instead + */ @Deprecated public ReadBuilder callInit() { this.callInit = true; diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java index 7f64b4ca49fd..aa324558068c 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/SnowflakeCatalog.java @@ -55,6 +55,7 @@ public class SnowflakeCatalog extends BaseMetastoreCatalog private static final String APP_IDENTIFIER = "iceberg-snowflake-catalog"; // Specifies the max length of unique id for each catalog initialized session. private static final int UNIQUE_ID_LENGTH = 20; + // Injectable factory for testing purposes. static class FileIOFactory { public FileIO newFileIO(String impl, Map properties, Object hadoopConf) { diff --git a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 5b7b9097755b..06180cc9cf49 100644 --- a/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -35,6 +35,7 @@ public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + // Currently we can only Snapshot only out of the Spark Session Catalog public TestSnapshotTableProcedure( 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 32b96a580da5..12011b66a5be 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 @@ -2021,8 +2021,13 @@ public void testFilesTablePartitionId() throws Exception { .save(loadLocation(tableIdentifier)); List actual = - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")) - .sort(DataFile.SPEC_ID.name()).collectAsList().stream() + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); diff --git a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java index 905cb8fe07fb..421d6efc9389 100644 --- a/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java +++ b/spark/v3.4/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestSnapshotTableProcedure.java @@ -35,6 +35,7 @@ public class TestSnapshotTableProcedure extends SparkExtensionsTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + // Currently we can only Snapshot only out of the Spark Session Catalog public TestSnapshotTableProcedure( 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 b990a37cdd76..f56f1161e624 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 @@ -2020,8 +2020,13 @@ public void testFilesTablePartitionId() { .save(loadLocation(tableIdentifier)); List actual = - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")) - .sort(DataFile.SPEC_ID.name()).collectAsList().stream() + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); 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 a4d0a2dfd3d0..6caff28bb16c 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 @@ -38,6 +38,7 @@ @ExtendWith(ParameterizedTestExtension.class) public class TestSnapshotTableProcedure extends ExtensionsTestBase { private static final String SOURCE_NAME = "spark_catalog.default.source"; + // Currently we can only Snapshot only out of the Spark Session Catalog @AfterEach 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 d37d6a861690..29216150d362 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 @@ -2044,8 +2044,13 @@ public void testFilesTablePartitionId() { .save(loadLocation(tableIdentifier)); List actual = - spark.read().format("iceberg").load(loadLocation(tableIdentifier, "files")) - .sort(DataFile.SPEC_ID.name()).collectAsList().stream() + spark + .read() + .format("iceberg") + .load(loadLocation(tableIdentifier, "files")) + .sort(DataFile.SPEC_ID.name()) + .collectAsList() + .stream() .map(r -> (Integer) r.getAs(DataFile.SPEC_ID.name())) .collect(Collectors.toList()); From 4e75bfcbff59d966cb0186a5927310e9d3a2e64b Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 12 Sep 2024 08:17:58 +0300 Subject: [PATCH 0684/1019] Flink: Maintenance - Lock remover (#11010) --- .../maintenance/operator/JdbcLockFactory.java | 4 + .../maintenance/operator/LockRemover.java | 144 +++++++ .../operator/TableMaintenanceMetrics.java | 5 + .../maintenance/operator/TaskResult.java | 65 +++ .../maintenance/operator/TriggerManager.java | 14 +- .../maintenance/operator/ManualSource.java | 16 +- .../maintenance/operator/TestLockRemover.java | 379 ++++++++++++++++++ 7 files changed, 615 insertions(+), 12 deletions(-) create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java create mode 100644 flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java create mode 100644 flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java 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/operator/JdbcLockFactory.java index f22be33aeae6..085fbfecd270 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/operator/JdbcLockFactory.java @@ -38,6 +38,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * JDBC table backed implementation of the {@link + * org.apache.iceberg.flink.maintenance.operator.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/operator/LockRemover.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java new file mode 100644 index 000000000000..3c3761ef2f4d --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -0,0 +1,144 @@ +/* + * 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.List; +import java.util.concurrent.atomic.AtomicLong; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. + * + *

    The assumptions about the locks are the following: + * + *

      + *
    • Every {@link TaskResult} is followed by a {@link Watermark} for normal {@link Trigger}s + *
    • For the {@link Trigger#recovery(long)} {@link Watermark} there is no element to process + *
    + * + * When processing the inputs there are 3 possibilities: + * + *
      + *
    • Normal execution - we receive a {@link TaskResult} and then a {@link Watermark} - unlocking + * the lock is handled by the {@link #processElement(StreamRecord)} + *
    • Recovery without ongoing execution (unlocking the recoveryLock) - we receive the {@link + * Trigger#recovery(long)} {@link Watermark} without any {@link TaskResult} - unlocking the + * {@link TriggerLockFactory#createRecoveryLock()} and a possible {@link + * TriggerLockFactory#createLock()} is handled by the {@link #processWatermark(Watermark)} + * (the {@link #lastProcessedTaskStartEpoch} is 0 in this case) + *
    • Recovery with an ongoing execution - we receive a {@link TaskResult} and then a {@link + * Watermark} - unlocking the {@link TriggerLockFactory#createLock()} is handled by the {@link + * #processElement(StreamRecord)}, unlocking the {@link + * TriggerLockFactory#createRecoveryLock()} is handled by the {@link + * #processWatermark(Watermark)} (the {@link #lastProcessedTaskStartEpoch} is the start time + * of the old task) + *
    + */ +@Internal +public class LockRemover extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List maintenanceTaskNames; + + private transient List succeededTaskResultCounters; + private transient List failedTaskResultCounters; + private transient List taskLastRunDurationMs; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessedTaskStartEpoch = 0L; + + public LockRemover(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.lockFactory = lockFactory; + this.maintenanceTaskNames = maintenanceTaskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.succeededTaskResultCounters = + Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (String name : maintenanceTaskNames) { + succeededTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); + failedTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); + AtomicLong duration = new AtomicLong(0); + taskLastRunDurationMs.add(duration); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord streamRecord) { + TaskResult taskResult = streamRecord.getValue(); + LOG.info( + "Processing result {} for task {}", + taskResult, + maintenanceTaskNames.get(taskResult.taskIndex())); + long duration = System.currentTimeMillis() - taskResult.startEpoch(); + lock.unlock(); + this.lastProcessedTaskStartEpoch = taskResult.startEpoch(); + + // Update the metrics + taskLastRunDurationMs.get(taskResult.taskIndex()).set(duration); + if (taskResult.success()) { + succeededTaskResultCounters.get(taskResult.taskIndex()).inc(); + } else { + failedTaskResultCounters.get(taskResult.taskIndex()).inc(); + } + } + + @Override + public void processWatermark(Watermark mark) { + if (mark.getTimestamp() > lastProcessedTaskStartEpoch) { + lock.unlock(); + recoveryLock.unlock(); + } + } +} 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 ec0fd920c34b..1a04461aed43 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 @@ -28,6 +28,11 @@ public class TableMaintenanceMetrics { public static final String TRIGGERED = "triggered"; public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + // LockRemover metrics + public static final String SUCCEEDED_TASK_COUNTER = "succeededTasks"; + public static final String FAILED_TASK_COUNTER = "failedTasks"; + public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + private TableMaintenanceMetrics() { // do not instantiate } 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/operator/TaskResult.java new file mode 100644 index 000000000000..06f10f1c1d68 --- /dev/null +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.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.flink.maintenance.operator; + +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 { + private final int taskIndex; + private final long startEpoch; + private final boolean success; + private final List exceptions; + + public TaskResult(int taskIndex, long startEpoch, boolean success, List exceptions) { + this.taskIndex = taskIndex; + this.startEpoch = startEpoch; + this.success = success; + this.exceptions = exceptions; + } + + public int taskIndex() { + return taskIndex; + } + + public long startEpoch() { + return startEpoch; + } + + public boolean success() { + return success; + } + + public List exceptions() { + return exceptions; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("taskIndex", taskIndex) + .add("startEpoch", startEpoch) + .add("success", success) + .add("exceptions", exceptions) + .toString(); + } +} 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 f4c3c1d47cf7..dc95b27af0a6 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 @@ -63,7 +63,7 @@ class TriggerManager extends KeyedProcessFunction private final TableLoader tableLoader; private final TriggerLockFactory lockFactory; - private final List taskNames; + private final List maintenanceTaskNames; private final List evaluators; private final long minFireDelayMs; private final long lockCheckDelayMs; @@ -92,25 +92,27 @@ class TriggerManager extends KeyedProcessFunction TriggerManager( TableLoader tableLoader, TriggerLockFactory lockFactory, - List taskNames, + List maintenanceTaskNames, List evaluators, long minFireDelayMs, long lockCheckDelayMs) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); Preconditions.checkArgument( - taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), + "Invalid maintenance task names: null or empty"); Preconditions.checkArgument( evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); Preconditions.checkArgument( - taskNames.size() == evaluators.size(), "Provide a name and evaluator for all of the tasks"); + maintenanceTaskNames.size() == evaluators.size(), + "Provide a name and evaluator for all of the maintenance tasks"); Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); Preconditions.checkArgument( lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); this.tableLoader = tableLoader; this.lockFactory = lockFactory; - this.taskNames = taskNames; + this.maintenanceTaskNames = maintenanceTaskNames; this.evaluators = evaluators; this.minFireDelayMs = minFireDelayMs; this.lockCheckDelayMs = lockCheckDelayMs; @@ -137,7 +139,7 @@ public void open(Configuration parameters) throws Exception { TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); this.triggerCounters = - taskNames.stream() + maintenanceTaskNames.stream() .map( name -> getRuntimeContext() 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 38bb9c393fa9..679b3ec508a2 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 @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -156,23 +157,26 @@ public SimpleVersionedSerializer getEnumeratorCheckpointSeriali @Override public SourceReader createReader(SourceReaderContext sourceReaderContext) { - return new SourceReader() { + return new SourceReader<>() { @Override public void start() { // Do nothing } + @SuppressWarnings("unchecked") @Override public InputStatus pollNext(ReaderOutput output) { Tuple2 next = (Tuple2) QUEUES.get(index).poll(); if (next != null) { if (next.f0 == null) { - // No more input - return InputStatus.END_OF_INPUT; - } - - if (next.f1 == null) { + if (next.f1 == null) { + // No more input + return InputStatus.END_OF_INPUT; + } else { + output.emitWatermark(new Watermark(next.f1)); + } + } else if (next.f1 == null) { // No event time set output.collect(next.f0); } else { 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 new file mode 100644 index 000000000000..ccb90ec33d9c --- /dev/null +++ b/flink/v1.20/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -0,0 +1,379 @@ +/* + * 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.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.SUCCEEDED_TASK_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.time.Duration; +import java.util.Collection; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +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.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.Timeout; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 10) +class TestLockRemover extends OperatorTestBase { + private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final TriggerLockFactory.Lock LOCK = new TestingLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); + + @TempDir private File checkpointDir; + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testProcess() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testInSink() throws Exception { + String sinkName = "TestSink"; + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + config.set(CheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofMillis(10)); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source.dataStream().global().sinkTo(new SinkTest()).name(sinkName).setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList()), sinkName + ": "); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testMetrics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + 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())); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[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) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + /** + * The test checks if the recovery watermark is only removed if the watermark has arrived from + * both upstream sources. + * + * @throws Exception if any + */ + @Test + void testRecovery() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source1 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + ManualSource source2 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source1 + .dataStream() + .union(source2.dataStream()) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + RECOVERY_LOCK.tryLock(); + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + processAndCheck(source1, new TaskResult(0, 0L, true, Lists.newArrayList())); + + source1.sendRecord(new TaskResult(0, 1L, true, Lists.newArrayList())); + // we receive the second result - this will not happen in real use cases, but with this we can + // be sure that the previous watermark is processed + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + .equals(2L)); + + // We did not remove the recovery lock, as no watermark received from the other source + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + // Recovery arrives + source1.sendWatermark(10L); + source2.sendWatermark(10L); + + Awaitility.await().until(() -> !RECOVERY_LOCK.isHeld()); + } finally { + closeJobClient(jobClient); + } + } + + private void processAndCheck(ManualSource source, TaskResult input) { + processAndCheck(source, input, null); + } + + private void processAndCheck( + ManualSource source, TaskResult input, String counterPrefix) { + 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))); + } + + private static class TestingLockFactory implements TriggerLockFactory { + @Override + public void open() { + // Do nothing + } + + @Override + public Lock createLock() { + return LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // Do nothing + } + } + + private static class TestingLock implements TriggerLockFactory.Lock { + private boolean locked = false; + + @Override + public boolean tryLock() { + if (isHeld()) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } + + private static class SinkTest + implements Sink, + SupportsCommitter, + SupportsPostCommitTopology { + @Override + public SinkWriter createWriter(InitContext initContext) { + return new CommittingSinkWriter() { + private final Collection received = Lists.newArrayList(); + + @Override + public Collection prepareCommit() { + Collection result = Lists.newArrayList(received); + received.clear(); + return result; + } + + @Override + public void write(TaskResult taskResult, Context context) { + received.add(taskResult); + } + + @Override + public void flush(boolean b) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public Committer createCommitter(CommitterInitContext committerInitContext) { + return new Committer<>() { + @Override + public void commit(Collection> collection) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new SimpleVersionedSerializer<>() { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(TaskResult taskResult) { + return new byte[0]; + } + + @Override + public TaskResult deserialize(int i, byte[] bytes) { + return null; + } + }; + } + + @Override + public void addPostCommitTopology(DataStream> committables) { + committables + .flatMap( + new FlatMapFunction, TaskResult>() { + @Override + public void flatMap( + CommittableMessage taskResultCommittableMessage, + Collector collector) { + if (taskResultCommittableMessage instanceof CommittableWithLineage) { + collector.collect( + ((CommittableWithLineage) taskResultCommittableMessage) + .getCommittable()); + } + } + }) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + } + } +} From e7b89743ab8ac8ae51a98aac37b1f2fa216db664 Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Wed, 11 Sep 2024 23:54:10 -0700 Subject: [PATCH 0685/1019] Docs: Update Project links to includ contributing and REST spec (#11114) --- site/nav.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/site/nav.yml b/site/nav.yml index cad34766be4c..c13b7cfaa6b1 100644 --- a/site/nav.yml +++ b/site/nav.yml @@ -39,7 +39,9 @@ nav: - Vendors: vendors.md - Project: - Community: community.md - - Spec: spec.md + - Contributing: contribute.md + - REST Catalog Spec: https://editor-next.swagger.io/?url=https://raw.githubusercontent.com/apache/iceberg/main/open-api/rest-catalog-open-api.yaml + - Table Spec: spec.md - View spec: view-spec.md - Puffin spec: puffin-spec.md - AES GCM Stream spec: gcm-stream-spec.md From 4e01a810450f7bbbacbc5cf9e6f2ce6d908e94e0 Mon Sep 17 00:00:00 2001 From: pvary Date: Thu, 12 Sep 2024 16:56:17 +0300 Subject: [PATCH 0686/1019] Flink: Port #10484 to v1.19 (#11010) (#11117) --- .../maintenance/operator/JdbcLockFactory.java | 4 + .../maintenance/operator/LockRemover.java | 144 +++++++ .../operator/TableMaintenanceMetrics.java | 5 + .../maintenance/operator/TaskResult.java | 65 +++ .../maintenance/operator/TriggerManager.java | 14 +- .../maintenance/operator/ManualSource.java | 16 +- .../maintenance/operator/TestLockRemover.java | 378 ++++++++++++++++++ .../maintenance/operator/TestLockRemover.java | 3 +- 8 files changed, 615 insertions(+), 14 deletions(-) create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java create mode 100644 flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.java create mode 100644 flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java 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/operator/JdbcLockFactory.java index f22be33aeae6..085fbfecd270 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/operator/JdbcLockFactory.java @@ -38,6 +38,10 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * JDBC table backed implementation of the {@link + * org.apache.iceberg.flink.maintenance.operator.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/operator/LockRemover.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java new file mode 100644 index 000000000000..3c3761ef2f4d --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/LockRemover.java @@ -0,0 +1,144 @@ +/* + * 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.List; +import java.util.concurrent.atomic.AtomicLong; +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.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Manages locks and collect {@link org.apache.flink.metrics.Metric} for the Maintenance Tasks. + * + *

    The assumptions about the locks are the following: + * + *

      + *
    • Every {@link TaskResult} is followed by a {@link Watermark} for normal {@link Trigger}s + *
    • For the {@link Trigger#recovery(long)} {@link Watermark} there is no element to process + *
    + * + * When processing the inputs there are 3 possibilities: + * + *
      + *
    • Normal execution - we receive a {@link TaskResult} and then a {@link Watermark} - unlocking + * the lock is handled by the {@link #processElement(StreamRecord)} + *
    • Recovery without ongoing execution (unlocking the recoveryLock) - we receive the {@link + * Trigger#recovery(long)} {@link Watermark} without any {@link TaskResult} - unlocking the + * {@link TriggerLockFactory#createRecoveryLock()} and a possible {@link + * TriggerLockFactory#createLock()} is handled by the {@link #processWatermark(Watermark)} + * (the {@link #lastProcessedTaskStartEpoch} is 0 in this case) + *
    • Recovery with an ongoing execution - we receive a {@link TaskResult} and then a {@link + * Watermark} - unlocking the {@link TriggerLockFactory#createLock()} is handled by the {@link + * #processElement(StreamRecord)}, unlocking the {@link + * TriggerLockFactory#createRecoveryLock()} is handled by the {@link + * #processWatermark(Watermark)} (the {@link #lastProcessedTaskStartEpoch} is the start time + * of the old task) + *
    + */ +@Internal +public class LockRemover extends AbstractStreamOperator + implements OneInputStreamOperator { + private static final Logger LOG = LoggerFactory.getLogger(LockRemover.class); + + private final TriggerLockFactory lockFactory; + private final List maintenanceTaskNames; + + private transient List succeededTaskResultCounters; + private transient List failedTaskResultCounters; + private transient List taskLastRunDurationMs; + private transient TriggerLockFactory.Lock lock; + private transient TriggerLockFactory.Lock recoveryLock; + private transient long lastProcessedTaskStartEpoch = 0L; + + public LockRemover(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.lockFactory = lockFactory; + this.maintenanceTaskNames = maintenanceTaskNames; + } + + @Override + public void open() throws Exception { + super.open(); + this.succeededTaskResultCounters = + Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.failedTaskResultCounters = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + this.taskLastRunDurationMs = Lists.newArrayListWithExpectedSize(maintenanceTaskNames.size()); + for (String name : maintenanceTaskNames) { + succeededTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.SUCCEEDED_TASK_COUNTER)); + failedTaskResultCounters.add( + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .counter(TableMaintenanceMetrics.FAILED_TASK_COUNTER)); + AtomicLong duration = new AtomicLong(0); + taskLastRunDurationMs.add(duration); + getRuntimeContext() + .getMetricGroup() + .addGroup(TableMaintenanceMetrics.GROUP_KEY, name) + .gauge(TableMaintenanceMetrics.LAST_RUN_DURATION_MS, duration::get); + } + + this.lock = lockFactory.createLock(); + this.recoveryLock = lockFactory.createRecoveryLock(); + } + + @Override + public void processElement(StreamRecord streamRecord) { + TaskResult taskResult = streamRecord.getValue(); + LOG.info( + "Processing result {} for task {}", + taskResult, + maintenanceTaskNames.get(taskResult.taskIndex())); + long duration = System.currentTimeMillis() - taskResult.startEpoch(); + lock.unlock(); + this.lastProcessedTaskStartEpoch = taskResult.startEpoch(); + + // Update the metrics + taskLastRunDurationMs.get(taskResult.taskIndex()).set(duration); + if (taskResult.success()) { + succeededTaskResultCounters.get(taskResult.taskIndex()).inc(); + } else { + failedTaskResultCounters.get(taskResult.taskIndex()).inc(); + } + } + + @Override + public void processWatermark(Watermark mark) { + if (mark.getTimestamp() > lastProcessedTaskStartEpoch) { + lock.unlock(); + recoveryLock.unlock(); + } + } +} 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 ec0fd920c34b..1a04461aed43 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 @@ -28,6 +28,11 @@ public class TableMaintenanceMetrics { public static final String TRIGGERED = "triggered"; public static final String NOTHING_TO_TRIGGER = "nothingToTrigger"; + // LockRemover metrics + public static final String SUCCEEDED_TASK_COUNTER = "succeededTasks"; + public static final String FAILED_TASK_COUNTER = "failedTasks"; + public static final String LAST_RUN_DURATION_MS = "lastRunDurationMs"; + private TableMaintenanceMetrics() { // do not instantiate } 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/operator/TaskResult.java new file mode 100644 index 000000000000..06f10f1c1d68 --- /dev/null +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/maintenance/operator/TaskResult.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.flink.maintenance.operator; + +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 { + private final int taskIndex; + private final long startEpoch; + private final boolean success; + private final List exceptions; + + public TaskResult(int taskIndex, long startEpoch, boolean success, List exceptions) { + this.taskIndex = taskIndex; + this.startEpoch = startEpoch; + this.success = success; + this.exceptions = exceptions; + } + + public int taskIndex() { + return taskIndex; + } + + public long startEpoch() { + return startEpoch; + } + + public boolean success() { + return success; + } + + public List exceptions() { + return exceptions; + } + + @Override + public String toString() { + return MoreObjects.toStringHelper(this) + .add("taskIndex", taskIndex) + .add("startEpoch", startEpoch) + .add("success", success) + .add("exceptions", exceptions) + .toString(); + } +} 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 f4c3c1d47cf7..dc95b27af0a6 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 @@ -63,7 +63,7 @@ class TriggerManager extends KeyedProcessFunction private final TableLoader tableLoader; private final TriggerLockFactory lockFactory; - private final List taskNames; + private final List maintenanceTaskNames; private final List evaluators; private final long minFireDelayMs; private final long lockCheckDelayMs; @@ -92,25 +92,27 @@ class TriggerManager extends KeyedProcessFunction TriggerManager( TableLoader tableLoader, TriggerLockFactory lockFactory, - List taskNames, + List maintenanceTaskNames, List evaluators, long minFireDelayMs, long lockCheckDelayMs) { Preconditions.checkNotNull(tableLoader, "Table loader should no be null"); Preconditions.checkNotNull(lockFactory, "Lock factory should no be null"); Preconditions.checkArgument( - taskNames != null && !taskNames.isEmpty(), "Invalid task names: null or empty"); + maintenanceTaskNames != null && !maintenanceTaskNames.isEmpty(), + "Invalid maintenance task names: null or empty"); Preconditions.checkArgument( evaluators != null && !evaluators.isEmpty(), "Invalid evaluators: null or empty"); Preconditions.checkArgument( - taskNames.size() == evaluators.size(), "Provide a name and evaluator for all of the tasks"); + maintenanceTaskNames.size() == evaluators.size(), + "Provide a name and evaluator for all of the maintenance tasks"); Preconditions.checkArgument(minFireDelayMs > 0, "Minimum fire delay should be at least 1."); Preconditions.checkArgument( lockCheckDelayMs > 0, "Minimum lock delay rate should be at least 1 ms."); this.tableLoader = tableLoader; this.lockFactory = lockFactory; - this.taskNames = taskNames; + this.maintenanceTaskNames = maintenanceTaskNames; this.evaluators = evaluators; this.minFireDelayMs = minFireDelayMs; this.lockCheckDelayMs = lockCheckDelayMs; @@ -137,7 +139,7 @@ public void open(Configuration parameters) throws Exception { TableMaintenanceMetrics.GROUP_KEY, TableMaintenanceMetrics.GROUP_VALUE_DEFAULT) .counter(TableMaintenanceMetrics.NOTHING_TO_TRIGGER); this.triggerCounters = - taskNames.stream() + maintenanceTaskNames.stream() .map( name -> getRuntimeContext() 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 38bb9c393fa9..679b3ec508a2 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 @@ -22,6 +22,7 @@ import java.util.Collections; import java.util.List; import java.util.concurrent.CompletableFuture; +import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.common.eventtime.WatermarkStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.connector.source.Boundedness; @@ -156,23 +157,26 @@ public SimpleVersionedSerializer getEnumeratorCheckpointSeriali @Override public SourceReader createReader(SourceReaderContext sourceReaderContext) { - return new SourceReader() { + return new SourceReader<>() { @Override public void start() { // Do nothing } + @SuppressWarnings("unchecked") @Override public InputStatus pollNext(ReaderOutput output) { Tuple2 next = (Tuple2) QUEUES.get(index).poll(); if (next != null) { if (next.f0 == null) { - // No more input - return InputStatus.END_OF_INPUT; - } - - if (next.f1 == null) { + if (next.f1 == null) { + // No more input + return InputStatus.END_OF_INPUT; + } else { + output.emitWatermark(new Watermark(next.f1)); + } + } else if (next.f1 == null) { // No event time set output.collect(next.f0); } else { 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 new file mode 100644 index 000000000000..cffcc4eb0471 --- /dev/null +++ b/flink/v1.19/flink/src/test/java/org/apache/iceberg/flink/maintenance/operator/TestLockRemover.java @@ -0,0 +1,378 @@ +/* + * 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.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.SUCCEEDED_TASK_COUNTER; +import static org.assertj.core.api.Assertions.assertThat; + +import java.io.File; +import java.util.Collection; +import org.apache.flink.api.common.functions.FlatMapFunction; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.connector.sink2.Committer; +import org.apache.flink.api.connector.sink2.CommitterInitContext; +import org.apache.flink.api.connector.sink2.CommittingSinkWriter; +import org.apache.flink.api.connector.sink2.Sink; +import org.apache.flink.api.connector.sink2.SinkWriter; +import org.apache.flink.api.connector.sink2.SupportsCommitter; +import org.apache.flink.configuration.CheckpointingOptions; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.core.execution.JobClient; +import org.apache.flink.core.io.SimpleVersionedSerializer; +import org.apache.flink.streaming.api.connector.sink2.CommittableMessage; +import org.apache.flink.streaming.api.connector.sink2.CommittableWithLineage; +import org.apache.flink.streaming.api.connector.sink2.SupportsPostCommitTopology; +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.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.Timeout; +import org.junit.jupiter.api.io.TempDir; + +@Timeout(value = 10) +class TestLockRemover extends OperatorTestBase { + private static final String[] TASKS = new String[] {"task0", "task1"}; + private static final TriggerLockFactory.Lock LOCK = new TestingLock(); + private static final TriggerLockFactory.Lock RECOVERY_LOCK = new TestingLock(); + + @TempDir private File checkpointDir; + + @BeforeEach + void before() { + MetricsReporterFactoryForTests.reset(); + } + + @Test + void testProcess() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList())); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testInSink() throws Exception { + String sinkName = "TestSink"; + Configuration config = new Configuration(); + config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); + config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(10); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source.dataStream().global().sinkTo(new SinkTest()).name(sinkName).setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + + LOCK.tryLock(); + assertThat(LOCK.isHeld()).isTrue(); + + // Start a successful trigger for task1 and assert the return value is correct + processAndCheck(source, new TaskResult(0, 0L, true, Lists.newArrayList()), sinkName + ": "); + + // Assert that the lock is removed + assertThat(LOCK.isHeld()).isFalse(); + } finally { + closeJobClient(jobClient); + } + } + + @Test + void testMetrics() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source + .dataStream() + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS))) + .setParallelism(1); + + JobClient jobClient = null; + 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())); + + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[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) + .build()); + } finally { + closeJobClient(jobClient); + } + } + + /** + * The test checks if the recovery watermark is only removed if the watermark has arrived from + * both upstream sources. + * + * @throws Exception if any + */ + @Test + void testRecovery() throws Exception { + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + ManualSource source1 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + ManualSource source2 = + new ManualSource<>(env, TypeInformation.of(TaskResult.class)); + source1 + .dataStream() + .union(source2.dataStream()) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))) + .setParallelism(1); + + JobClient jobClient = null; + try { + jobClient = env.executeAsync(); + RECOVERY_LOCK.tryLock(); + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + processAndCheck(source1, new TaskResult(0, 0L, true, Lists.newArrayList())); + + source1.sendRecord(new TaskResult(0, 1L, true, Lists.newArrayList())); + // we receive the second result - this will not happen in real use cases, but with this we can + // be sure that the previous watermark is processed + Awaitility.await() + .until( + () -> + MetricsReporterFactoryForTests.counter( + DUMMY_NAME + "." + TASKS[0] + "." + SUCCEEDED_TASK_COUNTER) + .equals(2L)); + + // We did not remove the recovery lock, as no watermark received from the other source + assertThat(RECOVERY_LOCK.isHeld()).isTrue(); + + // Recovery arrives + source1.sendWatermark(10L); + source2.sendWatermark(10L); + + Awaitility.await().until(() -> !RECOVERY_LOCK.isHeld()); + } finally { + closeJobClient(jobClient); + } + } + + private void processAndCheck(ManualSource source, TaskResult input) { + processAndCheck(source, input, null); + } + + private void processAndCheck( + ManualSource source, TaskResult input, String counterPrefix) { + 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))); + } + + private static class TestingLockFactory implements TriggerLockFactory { + @Override + public void open() { + // Do nothing + } + + @Override + public Lock createLock() { + return LOCK; + } + + @Override + public Lock createRecoveryLock() { + return RECOVERY_LOCK; + } + + @Override + public void close() { + // Do nothing + } + } + + private static class TestingLock implements TriggerLockFactory.Lock { + private boolean locked = false; + + @Override + public boolean tryLock() { + if (isHeld()) { + return false; + } else { + locked = true; + return true; + } + } + + @Override + public boolean isHeld() { + return locked; + } + + @Override + public void unlock() { + locked = false; + } + } + + private static class SinkTest + implements Sink, + SupportsCommitter, + SupportsPostCommitTopology { + @Override + public SinkWriter createWriter(InitContext initContext) { + return new CommittingSinkWriter() { + private final Collection received = Lists.newArrayList(); + + @Override + public Collection prepareCommit() { + Collection result = Lists.newArrayList(received); + received.clear(); + return result; + } + + @Override + public void write(TaskResult taskResult, Context context) { + received.add(taskResult); + } + + @Override + public void flush(boolean b) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public Committer createCommitter(CommitterInitContext committerInitContext) { + return new Committer<>() { + @Override + public void commit(Collection> collection) { + // noop + } + + @Override + public void close() { + // noop + } + }; + } + + @Override + public SimpleVersionedSerializer getCommittableSerializer() { + return new SimpleVersionedSerializer<>() { + @Override + public int getVersion() { + return 0; + } + + @Override + public byte[] serialize(TaskResult taskResult) { + return new byte[0]; + } + + @Override + public TaskResult deserialize(int i, byte[] bytes) { + return null; + } + }; + } + + @Override + public void addPostCommitTopology(DataStream> committables) { + committables + .flatMap( + new FlatMapFunction, TaskResult>() { + @Override + public void flatMap( + CommittableMessage taskResultCommittableMessage, + Collector collector) { + if (taskResultCommittableMessage instanceof CommittableWithLineage) { + collector.collect( + ((CommittableWithLineage) taskResultCommittableMessage) + .getCommittable()); + } + } + }) + .transform( + DUMMY_NAME, + TypeInformation.of(Void.class), + new LockRemover(new TestingLockFactory(), Lists.newArrayList(TASKS[0]))); + } + } +} 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 ccb90ec33d9c..cffcc4eb0471 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 @@ -24,7 +24,6 @@ import static org.assertj.core.api.Assertions.assertThat; import java.io.File; -import java.time.Duration; import java.util.Collection; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.typeinfo.TypeInformation; @@ -100,8 +99,8 @@ void testInSink() throws Exception { Configuration config = new Configuration(); config.set(CheckpointingOptions.CHECKPOINT_STORAGE, "filesystem"); config.set(CheckpointingOptions.CHECKPOINTS_DIRECTORY, "file://" + checkpointDir.getPath()); - config.set(CheckpointingOptions.CHECKPOINTING_INTERVAL, Duration.ofMillis(10)); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(config); + env.enableCheckpointing(10); ManualSource source = new ManualSource<>(env, TypeInformation.of(TaskResult.class)); source.dataStream().global().sinkTo(new SinkTest()).name(sinkName).setParallelism(1); From 48f097835402cf63d507f7bb6abc286b55dad46a Mon Sep 17 00:00:00 2001 From: Daniel Weeks Date: Thu, 12 Sep 2024 14:56:26 -0700 Subject: [PATCH 0687/1019] OpenAPI: Fix YAML example and value json formatting (#11119) --- open-api/rest-catalog-open-api.yaml | 457 ++++++++++++++-------------- 1 file changed, 229 insertions(+), 228 deletions(-) diff --git a/open-api/rest-catalog-open-api.yaml b/open-api/rest-catalog-open-api.yaml index abd937933290..cf4f7d3f9458 100644 --- a/open-api/rest-catalog-open-api.yaml +++ b/open-api/rest-catalog-open-api.yaml @@ -141,20 +141,20 @@ paths: schema: $ref: '#/components/schemas/CatalogConfig' example: { - "overrides": { - "warehouse": "s3://bucket/warehouse/" - }, - "defaults": { - "clients": "4" - }, - "endpoints": [ - "GET /v1/{prefix}/namespaces/{namespace}", - "GET /v1/{prefix}/namespaces", - "POST /v1/{prefix}/namespaces", - "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", - "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" - ] - } + "overrides": { + "warehouse": "s3://bucket/warehouse/" + }, + "defaults": { + "clients": "4" + }, + "endpoints": [ + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] + } 400: $ref: '#/components/responses/BadRequestErrorResponse' 401: @@ -1035,12 +1035,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } } - } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1051,12 +1051,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } } - } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1065,12 +1065,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } } - } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1079,12 +1079,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } } - } delete: tags: @@ -1329,12 +1329,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } } - } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1345,12 +1345,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } } - } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1359,12 +1359,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } } - } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1373,12 +1373,12 @@ paths: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } } - } /v1/{prefix}/namespaces/{namespace}/views: parameters: @@ -1567,12 +1567,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Internal Server Error", - "type": "CommitStateUnknownException", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "CommitStateUnknownException", + "code": 500 + } } - } 503: $ref: '#/components/responses/ServiceUnavailableResponse' 502: @@ -1583,12 +1583,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Invalid response from the upstream server", - "type": "CommitStateUnknownException", - "code": 502 + "error": { + "message": "Invalid response from the upstream server", + "type": "CommitStateUnknownException", + "code": 502 + } } - } 504: description: A server-side gateway timeout occurred; the commit state is unknown. @@ -1597,12 +1597,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Gateway timed out during commit", - "type": "CommitStateUnknownException", - "code": 504 + "error": { + "message": "Gateway timed out during commit", + "type": "CommitStateUnknownException", + "code": 504 + } } - } 5XX: description: A server-side problem that might not be addressable on the client. @@ -1611,12 +1611,12 @@ paths: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "Bad Gateway", - "type": "InternalServerError", - "code": 502 + "error": { + "message": "Bad Gateway", + "type": "InternalServerError", + "code": 502 + } } - } delete: tags: @@ -1892,12 +1892,12 @@ components: description: A list of endpoints that the server supports. The format of each endpoint must be " ". The HTTP verb and the resource path must be separated by a space character. example: [ - "GET /v1/{prefix}/namespaces/{namespace}", - "GET /v1/{prefix}/namespaces", - "POST /v1/{prefix}/namespaces", - "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", - "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" - ] + "GET /v1/{prefix}/namespaces/{namespace}", + "GET /v1/{prefix}/namespaces", + "POST /v1/{prefix}/namespaces", + "GET /v1/{prefix}/namespaces/{namespace}/tables/{table}", + "GET /v1/{prefix}/namespaces/{namespace}/views/{view}" + ] CreateNamespaceRequest: type: object @@ -3546,44 +3546,45 @@ components: type: object additionalProperties: $ref: '#/components/schemas/MetricResult' - example: - "metrics": { - "total-planning-duration": { - "count": 1, - "time-unit": "nanoseconds", - "total-duration": 2644235116 - }, - "result-data-files": { - "unit": "count", - "value": 1, - }, - "result-delete-files": { - "unit": "count", - "value": 0, - }, - "total-data-manifests": { - "unit": "count", - "value": 1, - }, - "total-delete-manifests": { - "unit": "count", - "value": 0, - }, - "scanned-data-manifests": { - "unit": "count", - "value": 1, - }, - "skipped-data-manifests": { - "unit": "count", - "value": 0, - }, - "total-file-size-bytes": { - "unit": "bytes", - "value": 10, - }, - "total-delete-file-size-bytes": { - "unit": "bytes", - "value": 0, + example: { + "metrics": { + "total-planning-duration": { + "count": 1, + "time-unit": "nanoseconds", + "total-duration": 2644235116 + }, + "result-data-files": { + "unit": "count", + "value": 1, + }, + "result-delete-files": { + "unit": "count", + "value": 0, + }, + "total-data-manifests": { + "unit": "count", + "value": 1, + }, + "total-delete-manifests": { + "unit": "count", + "value": 0, + }, + "scanned-data-manifests": { + "unit": "count", + "value": 1, + }, + "skipped-data-manifests": { + "unit": "count", + "value": 0, + }, + "total-file-size-bytes": { + "unit": "bytes", + "value": 10, + }, + "total-delete-file-size-bytes": { + "unit": "bytes", + "value": 0, + } } } @@ -4315,12 +4316,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Malformed request", - "type": "BadRequestException", - "code": 400 + "error": { + "message": "Malformed request", + "type": "BadRequestException", + "code": 400 + } } - } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -4332,12 +4333,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Not authorized to make this request", - "type": "NotAuthorizedException", - "code": 401 + "error": { + "message": "Not authorized to make this request", + "type": "NotAuthorizedException", + "code": 401 + } } - } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -4348,12 +4349,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Not authorized to make this request", - "type": "NotAuthorizedException", - "code": 403 + "error": { + "message": "Not authorized to make this request", + "type": "NotAuthorizedException", + "code": 403 + } } - } # Note that this is a representative example response for use as a shorthand in the spec. # The fields `message` and `type` as indicated here are not presently prescriptive. @@ -4364,12 +4365,12 @@ components: schema: $ref: '#/components/schemas/ErrorModel' example: { - "error": { - "message": "The server does not support this operation", - "type": "UnsupportedOperationException", - "code": 406 + "error": { + "message": "The server does not support this operation", + "type": "UnsupportedOperationException", + "code": 406 + } } - } IcebergErrorResponse: description: JSON wrapper for all error responses (non-2xx) @@ -4395,9 +4396,9 @@ components: schema: $ref: '#/components/schemas/CreateNamespaceResponse' example: { - "namespace": ["accounting", "tax"], - "properties": { "owner": "Ralph", "created_at": "1452120468" } - } + "namespace": ["accounting", "tax"], + "properties": { "owner": "Ralph", "created_at": "1452120468" } + } GetNamespaceResponse: description: @@ -4440,12 +4441,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Credentials have timed out", - "type": "AuthenticationTimeoutException", - "code": 419 + "error": { + "message": "Credentials have timed out", + "type": "AuthenticationTimeoutException", + "code": 419 + } } - } ServiceUnavailableResponse: description: @@ -4458,12 +4459,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Slow down", - "type": "SlowDownException", - "code": 503 + "error": { + "message": "Slow down", + "type": "SlowDownException", + "code": 503 + } } - } ServerErrorResponse: description: @@ -4475,12 +4476,12 @@ components: schema: $ref: '#/components/schemas/IcebergErrorResponse' example: { - "error": { - "message": "Internal Server Error", - "type": "InternalServerError", - "code": 500 + "error": { + "message": "Internal Server Error", + "type": "InternalServerError", + "code": 500 + } } - } UpdateNamespacePropertiesResponse: description: JSON data response for a synchronous update properties request. @@ -4489,10 +4490,10 @@ components: schema: $ref: '#/components/schemas/UpdateNamespacePropertiesResponse' example: { - "updated": [ "owner" ], - "removed": [ "foo" ], - "missing": [ "bar" ] - } + "updated": [ "owner" ], + "removed": [ "foo" ], + "missing": [ "bar" ] + } CreateTableResponse: description: Table metadata result after creating a table @@ -4554,32 +4555,32 @@ components: ListTablesEmptyExample: summary: An empty list for a namespace with no tables value: { - "identifiers": [ ] - } + "identifiers": [ ] + } ListNamespacesEmptyExample: summary: An empty list of namespaces value: { - "namespaces": [ ] - } + "namespaces": [ ] + } ListNamespacesNonEmptyExample: summary: A non-empty list of namespaces value: { - "namespaces": [ - ["accounting", "tax"], - ["accounting", "credits"] - ] - } + "namespaces": [ + ["accounting", "tax"], + ["accounting", "credits"] + ] + } ListTablesNonEmptyExample: summary: A non-empty list of table identifiers value: { - "identifiers": [ - { "namespace": ["accounting", "tax"], "name": "paid" }, - { "namespace": ["accounting", "tax"], "name": "owed" } - ] - } + "identifiers": [ + { "namespace": ["accounting", "tax"], "name": "paid" }, + { "namespace": ["accounting", "tax"], "name": "owed" } + ] + } MultipartNamespaceAsPathVariable: summary: A multi-part namespace, as represented in a path parameter @@ -4592,96 +4593,96 @@ components: NamespaceAlreadyExistsError: summary: The requested namespace already exists value: { - "error": { - "message": "The given namespace already exists", - "type": "AlreadyExistsException", - "code": 409 + "error": { + "message": "The given namespace already exists", + "type": "AlreadyExistsException", + "code": 409 + } } - } NoSuchPlanIdError: summary: The plan id does not exist value: { - "error": { - "message": "The plan id does not exist", - "type": "NoSuchPlanIdException", - "code": 404 + "error": { + "message": "The plan id does not exist", + "type": "NoSuchPlanIdException", + "code": 404 + } } - } NoSuchPlanTaskError: summary: The plan task does not exist value: { - "error": { - "message": "The plan task does not exist", - "type": "NoSuchPlanTaskException", - "code": 404 + "error": { + "message": "The plan task does not exist", + "type": "NoSuchPlanTaskException", + "code": 404 + } } - } NoSuchTableError: summary: The requested table does not exist value: { - "error": { - "message": "The given table does not exist", - "type": "NoSuchTableException", - "code": 404 + "error": { + "message": "The given table does not exist", + "type": "NoSuchTableException", + "code": 404 + } } - } NoSuchViewError: summary: The requested view does not exist value: { - "error": { - "message": "The given view does not exist", - "type": "NoSuchViewException", - "code": 404 + "error": { + "message": "The given view does not exist", + "type": "NoSuchViewException", + "code": 404 + } } - } NoSuchNamespaceError: summary: The requested namespace does not exist value: { - "error": { - "message": "The given namespace does not exist", - "type": "NoSuchNamespaceException", - "code": 404 + "error": { + "message": "The given namespace does not exist", + "type": "NoSuchNamespaceException", + "code": 404 + } } - } RenameTableSameNamespace: summary: Rename a table in the same namespace value: { - "source": { "namespace": ["accounting", "tax"], "name": "paid" }, - "destination": { "namespace": ["accounting", "tax"], "name": "owed" } - } + "source": { "namespace": ["accounting", "tax"], "name": "paid" }, + "destination": { "namespace": ["accounting", "tax"], "name": "owed" } + } RenameViewSameNamespace: summary: Rename a view in the same namespace value: { - "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, - "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } - } + "source": { "namespace": [ "accounting", "tax" ], "name": "paid-view" }, + "destination": { "namespace": [ "accounting", "tax" ], "name": "owed-view" } + } TableAlreadyExistsError: summary: The requested table identifier already exists value: { - "error": { - "message": "The given table already exists", - "type": "AlreadyExistsException", - "code": 409 + "error": { + "message": "The given table already exists", + "type": "AlreadyExistsException", + "code": 409 + } } - } ViewAlreadyExistsError: summary: The requested view identifier already exists value: { - "error": { - "message": "The given view already exists", - "type": "AlreadyExistsException", - "code": 409 + "error": { + "message": "The given view already exists", + "type": "AlreadyExistsException", + "code": 409 + } } - } # This is an example response and is not meant to be prescriptive regarding the message or type. UnprocessableEntityDuplicateKey: @@ -4689,19 +4690,19 @@ components: The request body either has the same key multiple times in what should be a map with unique keys or the request body has keys in two or more fields which should be disjoint sets. value: { - "error": { - "message": "The request cannot be processed as there is a key present multiple times", - "type": "UnprocessableEntityException", - "code": 422 + "error": { + "message": "The request cannot be processed as there is a key present multiple times", + "type": "UnprocessableEntityException", + "code": 422 + } } - } UpdateAndRemoveNamespacePropertiesRequest: summary: An update namespace properties request with both properties to remove and properties to upsert. value: { - "removals": [ "foo", "bar" ], - "updates": { "owner": "Raoul" } - } + "removals": [ "foo", "bar" ], + "updates": { "owner": "Raoul" } + } securitySchemes: OAuth2: From 9b8b81306b3b602b40ff68ac3d6920959e169bba Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 12 Sep 2024 16:41:05 -0700 Subject: [PATCH 0688/1019] Core: Parallelize manifest writing for many new files (#11086) --- .../org/apache/iceberg/AppendBenchmark.java | 13 +- .../java/org/apache/iceberg/FastAppend.java | 9 +- .../iceberg/MergingSnapshotProducer.java | 73 +---------- .../org/apache/iceberg/SnapshotProducer.java | 124 ++++++++++++++++++ .../java/org/apache/iceberg/TestBase.java | 5 + .../org/apache/iceberg/TestFastAppend.java | 19 +++ .../org/apache/iceberg/TestMergeAppend.java | 21 +++ .../apache/iceberg/TestSnapshotProducer.java | 77 +++++++++++ 8 files changed, 261 insertions(+), 80 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java diff --git a/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java index a8bafe413c6f..a444e7ff9c63 100644 --- a/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/AppendBenchmark.java @@ -38,7 +38,6 @@ 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 evaluates the performance of appending files to the table. @@ -66,14 +65,20 @@ public class AppendBenchmark { required(4, "date_col", Types.DateType.get()), required(5, "timestamp_col", Types.TimestampType.withoutZone()), required(6, "timestamp_tz_col", Types.TimestampType.withZone()), - required(7, "str_col", Types.StringType.get())); + required(7, "str_col1", Types.StringType.get()), + required(8, "str_col2", Types.StringType.get()), + required(9, "str_col3", Types.StringType.get()), + required(10, "str_col4", Types.StringType.get()), + required(11, "str_col5", Types.StringType.get()), + required(12, "str_col6", Types.StringType.get()), + required(13, "str_col7", Types.StringType.get())); private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); private static final HadoopTables TABLES = new HadoopTables(); private Table table; private List dataFiles; - @Param({"500000", "1000000", "2500000"}) + @Param({"50000", "100000", "500000", "1000000", "2500000"}) private int numFiles; @Param({"true", "false"}) @@ -92,7 +97,7 @@ public void tearDownBenchmark() { @Benchmark @Threads(1) - public void appendFiles(Blackhole blackhole) { + public void appendFiles() { AppendFiles append = fast ? table.newFastAppend() : table.newAppend(); for (DataFile dataFile : dataFiles) { diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 4976a8081c44..1bae2e2fc5a0 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -215,14 +215,7 @@ private List writeNewManifests() throws IOException { } if (newManifests == null && !newFiles.isEmpty()) { - RollingManifestWriter writer = newRollingManifestWriter(spec); - try { - newFiles.forEach(writer::add); - } finally { - writer.close(); - } - - this.newManifests = writer.toManifestFiles(); + this.newManifests = writeDataManifests(newFiles, spec); hasNewFiles = false; } diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index d1eb5d89da41..6a4da2abc9b6 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -33,7 +33,6 @@ import java.util.stream.Collectors; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.events.CreateSnapshotEvent; -import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.expressions.Expressions; @@ -972,21 +971,9 @@ private List newDataFilesAsManifests() { if (cachedNewDataManifests.isEmpty()) { newDataFilesBySpec.forEach( (dataSpec, newDataFiles) -> { - try { - RollingManifestWriter writer = newRollingManifestWriter(dataSpec); - try { - if (newDataFilesDataSequenceNumber == null) { - newDataFiles.forEach(writer::add); - } else { - newDataFiles.forEach(f -> writer.add(f, newDataFilesDataSequenceNumber)); - } - } finally { - writer.close(); - } - this.cachedNewDataManifests.addAll(writer.toManifestFiles()); - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close manifest writer"); - } + List newDataManifests = + writeDataManifests(newDataFiles, newDataFilesDataSequenceNumber, dataSpec); + cachedNewDataManifests.addAll(newDataManifests); }); this.hasNewDataFiles = false; } @@ -1016,24 +1003,8 @@ private List newDeleteFilesAsManifests() { newDeleteFilesBySpec.forEach( (specId, deleteFiles) -> { PartitionSpec spec = ops.current().spec(specId); - try { - RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); - try { - deleteFiles.forEach( - df -> { - if (df.dataSequenceNumber() != null) { - writer.add(df.deleteFile(), df.dataSequenceNumber()); - } else { - writer.add(df.deleteFile()); - } - }); - } finally { - writer.close(); - } - cachedNewDeleteManifests.addAll(writer.toManifestFiles()); - } catch (IOException e) { - throw new RuntimeIOException(e, "Failed to close manifest writer"); - } + List newDeleteManifests = writeDeleteManifests(deleteFiles, spec); + cachedNewDeleteManifests.addAll(newDeleteManifests); }); this.hasNewDeleteFiles = false; @@ -1147,38 +1118,4 @@ protected ManifestReader newManifestReader(ManifestFile manifest) { return MergingSnapshotProducer.this.newDeleteManifestReader(manifest); } } - - private static class DeleteFileHolder { - private final DeleteFile deleteFile; - private final Long dataSequenceNumber; - - /** - * Wrap a delete file for commit with a given data sequence number - * - * @param deleteFile delete file - * @param dataSequenceNumber data sequence number to apply - */ - DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { - this.deleteFile = deleteFile; - this.dataSequenceNumber = dataSequenceNumber; - } - - /** - * Wrap a delete file for commit with the latest sequence number - * - * @param deleteFile delete file - */ - DeleteFileHolder(DeleteFile deleteFile) { - this.deleteFile = deleteFile; - this.dataSequenceNumber = null; - } - - public DeleteFile deleteFile() { - return deleteFile; - } - - public Long dataSequenceNumber() { - return dataSequenceNumber; - } - } } diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 74997cc89849..22f6ac5e0b74 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -34,15 +34,18 @@ import com.github.benmanes.caffeine.cache.Caffeine; import com.github.benmanes.caffeine.cache.LoadingCache; import java.io.IOException; +import java.math.RoundingMode; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.Queue; import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutorService; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Consumer; +import java.util.function.Function; import org.apache.iceberg.encryption.EncryptedOutputFile; import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.events.CreateSnapshotEvent; @@ -59,10 +62,14 @@ import org.apache.iceberg.metrics.LoggingMetricsReporter; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.metrics.Timer.Timed; +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.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.Queues; import org.apache.iceberg.relocated.com.google.common.collect.Sets; +import org.apache.iceberg.relocated.com.google.common.math.IntMath; import org.apache.iceberg.util.Exceptions; import org.apache.iceberg.util.SnapshotUtil; import org.apache.iceberg.util.Tasks; @@ -73,6 +80,7 @@ @SuppressWarnings("UnnecessaryAnonymousClass") abstract class SnapshotProducer implements SnapshotUpdate { private static final Logger LOG = LoggerFactory.getLogger(SnapshotProducer.class); + static final int MIN_FILE_GROUP_SIZE = 10_000; static final Set EMPTY_SET = Sets.newHashSet(); /** Default callback used to delete files. */ @@ -554,6 +562,88 @@ protected boolean cleanupAfterCommit() { return true; } + protected List writeDataManifests(List files, PartitionSpec spec) { + return writeDataManifests(files, null /* inherit data seq */, spec); + } + + protected List writeDataManifests( + List files, Long dataSeq, PartitionSpec spec) { + return writeManifests(files, group -> writeDataFileGroup(group, dataSeq, spec)); + } + + private List writeDataFileGroup( + List files, Long dataSeq, PartitionSpec spec) { + RollingManifestWriter writer = newRollingManifestWriter(spec); + + try (RollingManifestWriter closableWriter = writer) { + if (dataSeq != null) { + files.forEach(file -> closableWriter.add(file, dataSeq)); + } else { + files.forEach(closableWriter::add); + } + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to write data manifests"); + } + + return writer.toManifestFiles(); + } + + protected List writeDeleteManifests( + List files, PartitionSpec spec) { + return writeManifests(files, group -> writeDeleteFileGroup(group, spec)); + } + + private List writeDeleteFileGroup( + List files, PartitionSpec spec) { + RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); + + try (RollingManifestWriter closableWriter = writer) { + for (DeleteFileHolder file : files) { + if (file.dataSequenceNumber() != null) { + closableWriter.add(file.deleteFile(), file.dataSequenceNumber()); + } else { + closableWriter.add(file.deleteFile()); + } + } + } catch (IOException e) { + throw new RuntimeIOException(e, "Failed to write delete manifests"); + } + + return writer.toManifestFiles(); + } + + private static List writeManifests( + List files, Function, List> writeFunc) { + int parallelism = manifestWriterCount(ThreadPools.WORKER_THREAD_POOL_SIZE, files.size()); + List> groups = divide(files, parallelism); + Queue manifests = Queues.newConcurrentLinkedQueue(); + Tasks.foreach(groups) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(ThreadPools.getWorkerPool()) + .run(group -> manifests.addAll(writeFunc.apply(group))); + return ImmutableList.copyOf(manifests); + } + + private static List> divide(List list, int groupCount) { + int groupSize = IntMath.divide(list.size(), groupCount, RoundingMode.CEILING); + return Lists.partition(list, groupSize); + } + + /** + * Calculates how many manifest writers can be used to concurrently to handle the given number of + * files without creating too small manifests. + * + * @param workerPoolSize the size of the available worker pool + * @param fileCount the total number of files to be processed + * @return the number of manifest writers that can be used concurrently + */ + @VisibleForTesting + static int manifestWriterCount(int workerPoolSize, int fileCount) { + int limit = IntMath.divide(fileCount, MIN_FILE_GROUP_SIZE, RoundingMode.HALF_UP); + return Math.max(1, Math.min(workerPoolSize, limit)); + } + private static ManifestFile addMetadata(TableOperations ops, ManifestFile manifest) { try (ManifestReader reader = ManifestFiles.read(manifest, ops.io(), ops.current().specsById())) { @@ -654,4 +744,38 @@ private static void updateTotal( } } } + + protected static class DeleteFileHolder { + private final DeleteFile deleteFile; + private final Long dataSequenceNumber; + + /** + * Wrap a delete file for commit with a given data sequence number. + * + * @param deleteFile delete file + * @param dataSequenceNumber data sequence number to apply + */ + DeleteFileHolder(DeleteFile deleteFile, long dataSequenceNumber) { + this.deleteFile = deleteFile; + this.dataSequenceNumber = dataSequenceNumber; + } + + /** + * Wrap a delete file for commit with the latest sequence number. + * + * @param deleteFile delete file + */ + DeleteFileHolder(DeleteFile deleteFile) { + this.deleteFile = deleteFile; + this.dataSequenceNumber = null; + } + + public DeleteFile deleteFile() { + return deleteFile; + } + + public Long dataSequenceNumber() { + return dataSequenceNumber; + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index e03a1efd5156..23fabc2a9402 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -27,6 +27,7 @@ import java.nio.file.Path; import java.nio.file.Paths; import java.util.Arrays; +import java.util.Collection; import java.util.Iterator; import java.util.List; import java.util.Set; @@ -460,6 +461,10 @@ void validateSnapshot(Snapshot old, Snapshot snap, Long sequenceNumber, DataFile } void validateTableFiles(Table tbl, DataFile... expectedFiles) { + validateTableFiles(tbl, Arrays.asList(expectedFiles)); + } + + void validateTableFiles(Table tbl, Collection expectedFiles) { Set expectedFilePaths = Sets.newHashSet(); for (DataFile file : expectedFiles) { expectedFilePaths.add(file.path()); diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index 8125c528d9c3..b2f19fbd5ffc 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -42,6 +42,25 @@ protected static List parameters() { return Arrays.asList(1, 2, 3); } + @TestTemplate + public void testAddManyFiles() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + List dataFiles = Lists.newArrayList(); + + for (int ordinal = 0; ordinal < 2 * SnapshotProducer.MIN_FILE_GROUP_SIZE; ordinal++) { + StructLike partition = TestHelpers.Row.of(ordinal % 2); + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + dataFiles.add(dataFile); + } + + AppendFiles append = table.newAppend(); + dataFiles.forEach(append::appendFile); + append.commit(); + + validateTableFiles(table, dataFiles); + } + @TestTemplate public void appendNullFile() { assertThatThrownBy(() -> table.newFastAppend().appendFile(null).commit()) diff --git a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java index abfcb318334b..e079f634014a 100644 --- a/core/src/test/java/org/apache/iceberg/TestMergeAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestMergeAppend.java @@ -33,9 +33,11 @@ import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import org.apache.iceberg.ManifestEntry.Status; +import org.apache.iceberg.TestHelpers.Row; import org.apache.iceberg.exceptions.CommitFailedException; 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.types.Types; import org.junit.jupiter.api.TestTemplate; @@ -62,6 +64,25 @@ public void appendNullFile() { .hasMessage("Invalid data file: null"); } + @TestTemplate + public void testAddManyFiles() { + assertThat(listManifestFiles()).as("Table should start empty").isEmpty(); + + List dataFiles = Lists.newArrayList(); + + for (int ordinal = 0; ordinal < 2 * SnapshotProducer.MIN_FILE_GROUP_SIZE; ordinal++) { + StructLike partition = Row.of(ordinal % 2); + DataFile dataFile = FileGenerationUtil.generateDataFile(table, partition); + dataFiles.add(dataFile); + } + + AppendFiles append = table.newAppend(); + dataFiles.forEach(append::appendFile); + append.commit(); + + validateTableFiles(table, dataFiles); + } + @TestTemplate public void testEmptyTableAppend() { assertThat(listManifestFiles()).isEmpty(); diff --git a/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.java new file mode 100644 index 000000000000..52bffdf18587 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestSnapshotProducer.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; + +import static org.assertj.core.api.Assertions.assertThat; + +import org.junit.jupiter.api.Test; + +public class TestSnapshotProducer { + + @Test + public void testManifestFileGroupSize() { + assertManifestWriterCount( + 4 /* worker pool size */, + 100 /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer if file count is small"); + + assertManifestWriterCount( + 4 /* worker pool size */, + SnapshotProducer.MIN_FILE_GROUP_SIZE /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer if file count matches min group size"); + + assertManifestWriterCount( + 4 /* worker pool size */, + SnapshotProducer.MIN_FILE_GROUP_SIZE + 1 /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer if file count is slightly above min group size"); + + assertManifestWriterCount( + 4 /* worker pool size */, + (int) (1.25 * SnapshotProducer.MIN_FILE_GROUP_SIZE) /* file count */, + 1 /* manifest writer count */, + "Must use 1 writer when file count is < 1.5 * min group size"); + + assertManifestWriterCount( + 4 /* worker pool size */, + (int) (1.5 * SnapshotProducer.MIN_FILE_GROUP_SIZE) /* file count */, + 2 /* manifest writer count */, + "Must use 2 writers when file count is >= 1.5 * min group size"); + + assertManifestWriterCount( + 3 /* worker pool size */, + 100 * SnapshotProducer.MIN_FILE_GROUP_SIZE /* file count */, + 3 /* manifest writer count */, + "Must limit parallelism to worker pool size when file count is large"); + + assertManifestWriterCount( + 32 /* worker pool size */, + 5 * SnapshotProducer.MIN_FILE_GROUP_SIZE /* file count */, + 5 /* manifest writer count */, + "Must limit parallelism to avoid tiny manifests"); + } + + private void assertManifestWriterCount( + int workerPoolSize, int fileCount, int expectedManifestWriterCount, String errMsg) { + int writerCount = SnapshotProducer.manifestWriterCount(workerPoolSize, fileCount); + assertThat(writerCount).withFailMessage(errMsg).isEqualTo(expectedManifestWriterCount); + } +} From 04d801e99f7d4ebdb2f08c1b448c1efe264066c6 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 12 Sep 2024 22:50:54 -0600 Subject: [PATCH 0689/1019] API, Core: Add manifestLocation API to ContentFile (#11044) --- .../java/org/apache/iceberg/ContentFile.java | 8 +++ .../java/org/apache/iceberg/BaseFile.java | 11 ++++ .../iceberg/InheritableMetadataFactory.java | 11 +++- .../java/org/apache/iceberg/V1Metadata.java | 5 ++ .../java/org/apache/iceberg/V2Metadata.java | 5 ++ .../java/org/apache/iceberg/V3Metadata.java | 5 ++ .../apache/iceberg/DataTableScanTestBase.java | 65 ++++++++++++++++++- .../apache/iceberg/TestManifestReader.java | 31 ++++++++- 8 files changed, 137 insertions(+), 4 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/ContentFile.java b/api/src/main/java/org/apache/iceberg/ContentFile.java index 4bb8a78289de..9db19f41ebfb 100644 --- a/api/src/main/java/org/apache/iceberg/ContentFile.java +++ b/api/src/main/java/org/apache/iceberg/ContentFile.java @@ -29,6 +29,14 @@ * @param the concrete Java class of a ContentFile instance. */ public interface ContentFile { + /** + * Returns the path of the manifest which this file is referenced in or null if it was not read + * from a manifest. + */ + default String manifestLocation() { + return null; + } + /** * Returns the ordinal position of the file in a manifest, or null if it was not read from a * manifest. diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index d4583b31c7a1..ddd08f944c98 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -59,6 +59,7 @@ public PartitionData copy() { private Types.StructType partitionType; private Long fileOrdinal = null; + private String manifestLocation = null; private int partitionSpecId = -1; private FileContent content = FileContent.DATA; private String filePath = null; @@ -178,6 +179,7 @@ public PartitionData copy() { */ BaseFile(BaseFile toCopy, boolean copyStats, Set requestedColumnIds) { this.fileOrdinal = toCopy.fileOrdinal; + this.manifestLocation = toCopy.manifestLocation; this.partitionSpecId = toCopy.partitionSpecId; this.content = toCopy.content; this.filePath = toCopy.filePath; @@ -240,6 +242,10 @@ public void setDataSequenceNumber(Long dataSequenceNumber) { this.dataSequenceNumber = dataSequenceNumber; } + void setManifestLocation(String manifestLocation) { + this.manifestLocation = manifestLocation; + } + @Override public Long fileSequenceNumber() { return fileSequenceNumber; @@ -397,6 +403,11 @@ public Long pos() { return fileOrdinal; } + @Override + public String manifestLocation() { + return manifestLocation; + } + @Override public FileContent content() { return content; diff --git a/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java b/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java index 26b09833a0c0..e01346a76f3b 100644 --- a/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java +++ b/core/src/main/java/org/apache/iceberg/InheritableMetadataFactory.java @@ -35,7 +35,10 @@ static InheritableMetadata fromManifest(ManifestFile manifest) { manifest.snapshotId() != null, "Cannot read from ManifestFile with null (unassigned) snapshot ID"); return new BaseInheritableMetadata( - manifest.partitionSpecId(), manifest.snapshotId(), manifest.sequenceNumber()); + manifest.partitionSpecId(), + manifest.snapshotId(), + manifest.sequenceNumber(), + manifest.path()); } static InheritableMetadata forCopy(long snapshotId) { @@ -46,11 +49,14 @@ static class BaseInheritableMetadata implements InheritableMetadata { private final int specId; private final long snapshotId; private final long sequenceNumber; + private final String manifestLocation; - private BaseInheritableMetadata(int specId, long snapshotId, long sequenceNumber) { + private BaseInheritableMetadata( + int specId, long snapshotId, long sequenceNumber, String manifestLocation) { this.specId = specId; this.snapshotId = snapshotId; this.sequenceNumber = sequenceNumber; + this.manifestLocation = manifestLocation; } @Override @@ -78,6 +84,7 @@ public > ManifestEntry apply(ManifestEntry manife file.setSpecId(specId); file.setDataSequenceNumber(manifestEntry.dataSequenceNumber()); file.setFileSequenceNumber(manifestEntry.fileSequenceNumber()); + file.setManifestLocation(manifestLocation); } return manifestEntry; diff --git a/core/src/main/java/org/apache/iceberg/V1Metadata.java b/core/src/main/java/org/apache/iceberg/V1Metadata.java index 30b04cd73124..78fa81cebe52 100644 --- a/core/src/main/java/org/apache/iceberg/V1Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V1Metadata.java @@ -399,6 +399,11 @@ public Long pos() { return null; } + @Override + public String manifestLocation() { + return null; + } + @Override public int specId() { return wrapped.specId(); diff --git a/core/src/main/java/org/apache/iceberg/V2Metadata.java b/core/src/main/java/org/apache/iceberg/V2Metadata.java index 8f3b71d39925..be4c3734e40b 100644 --- a/core/src/main/java/org/apache/iceberg/V2Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V2Metadata.java @@ -462,6 +462,11 @@ public Long pos() { return null; } + @Override + public String manifestLocation() { + return null; + } + @Override public int specId() { return wrapped.specId(); diff --git a/core/src/main/java/org/apache/iceberg/V3Metadata.java b/core/src/main/java/org/apache/iceberg/V3Metadata.java index 94e20ea99858..f295af3e109d 100644 --- a/core/src/main/java/org/apache/iceberg/V3Metadata.java +++ b/core/src/main/java/org/apache/iceberg/V3Metadata.java @@ -457,6 +457,11 @@ public void put(int i, Object v) { throw new UnsupportedOperationException("Cannot modify IndexedDataFile wrapper via put"); } + @Override + public String manifestLocation() { + return null; + } + @Override public Long pos() { return null; diff --git a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java index 04bbcf662225..8bd7a5ea70d9 100644 --- a/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java +++ b/core/src/test/java/org/apache/iceberg/DataTableScanTestBase.java @@ -23,12 +23,15 @@ import static org.assertj.core.api.Assumptions.assumeThat; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.UUID; +import java.util.stream.Collectors; 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.Iterables; import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.util.CharSequenceMap; import org.junit.jupiter.api.TestTemplate; import org.junit.jupiter.api.extension.ExtendWith; @@ -180,12 +183,25 @@ public void testSettingInvalidRefFails() { private void validateExpectedFileScanTasks(ScanT scan, List expectedFileScanPaths) throws IOException { + validateExpectedFileScanTasks(scan, expectedFileScanPaths, null); + } + + private void validateExpectedFileScanTasks( + ScanT scan, + Collection expectedFileScanPaths, + CharSequenceMap fileToManifest) + throws IOException { try (CloseableIterable scanTasks = scan.planFiles()) { assertThat(scanTasks).hasSameSizeAs(expectedFileScanPaths); List actualFiles = Lists.newArrayList(); for (T task : scanTasks) { - actualFiles.add(((FileScanTask) task).file().path()); + DataFile dataFile = ((FileScanTask) task).file(); + actualFiles.add(dataFile.path()); + if (fileToManifest != null) { + assertThat(fileToManifest.get(dataFile.path())).isEqualTo(dataFile.manifestLocation()); + } } + assertThat(actualFiles).containsAll(expectedFileScanPaths); } } @@ -246,4 +262,51 @@ public void testSequenceNumbersThroughPlanFiles() { .isEqualTo(expectedDeleteSequenceNumber); } } + + @TestTemplate + public void testManifestLocationsInScan() throws IOException { + table.newFastAppend().appendFile(FILE_A).commit(); + ManifestFile firstDataManifest = table.currentSnapshot().allManifests(table.io()).get(0); + table.newFastAppend().appendFile(FILE_B).appendFile(FILE_C).commit(); + ManifestFile secondDataManifest = + table.currentSnapshot().dataManifests(table.io()).stream() + .filter(manifest -> manifest.snapshotId() == table.currentSnapshot().snapshotId()) + .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()); + + validateExpectedFileScanTasks(newScan(), fileToManifest.keySet(), fileToManifest); + } + + @TestTemplate + public void testManifestLocationsInScanWithDeleteFiles() throws IOException { + assumeThat(formatVersion).isEqualTo(2); + + table.newFastAppend().appendFile(FILE_A).commit(); + ManifestFile firstManifest = table.currentSnapshot().allManifests(table.io()).get(0); + DeleteFile deleteFile = newDeleteFile("data_bucket=0"); + table.newRowDelta().addDeletes(deleteFile).commit(); + CharSequenceMap fileToManifest = CharSequenceMap.create(); + fileToManifest.put(FILE_A.path(), firstManifest.path()); + ScanT scan = newScan(); + validateExpectedFileScanTasks(scan, ImmutableList.of(FILE_A.path()), fileToManifest); + List deletes = Lists.newArrayList(); + try (CloseableIterable scanTasks = scan.planFiles()) { + for (T task : scanTasks) { + FileScanTask fileScanTask = (FileScanTask) task; + deletes.addAll(fileScanTask.deletes()); + } + } + + assertThat(deletes.size()).isEqualTo(1); + ManifestFile deleteManifest = + table.currentSnapshot().deleteManifests(table.io()).stream() + .filter(manifest -> manifest.snapshotId() == table.currentSnapshot().snapshotId()) + .collect(Collectors.toList()) + .get(0); + assertThat(deletes.get(0).manifestLocation()).isEqualTo(deleteManifest.path()); + } } diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index 999fd0d92eb2..c9d0e292706e 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -40,7 +40,11 @@ public class TestManifestReader extends TestBase { private static final RecursiveComparisonConfiguration FILE_COMPARISON_CONFIG = RecursiveComparisonConfiguration.builder() .withIgnoredFields( - "dataSequenceNumber", "fileOrdinal", "fileSequenceNumber", "fromProjectionPos") + "dataSequenceNumber", + "fileOrdinal", + "fileSequenceNumber", + "fromProjectionPos", + "manifestLocation") .build(); @TestTemplate @@ -133,6 +137,16 @@ public void testDataFilePositions() throws IOException { } } + @TestTemplate + public void testDataFileManifestPaths() throws IOException { + ManifestFile manifest = writeManifest(1000L, FILE_A, FILE_B, FILE_C); + try (ManifestReader reader = ManifestFiles.read(manifest, FILE_IO)) { + for (DataFile file : reader) { + assertThat(file.manifestLocation()).isEqualTo(manifest.path()); + } + } + } + @TestTemplate public void testDeleteFilePositions() throws IOException { assumeThat(formatVersion).as("Delete files only work for format version 2").isEqualTo(2); @@ -151,6 +165,21 @@ public void testDeleteFilePositions() throws IOException { } } + @TestTemplate + public void testDeleteFileManifestPaths() throws IOException { + assumeThat(formatVersion) + .as("Delete files only work for format version 2 or higher") + .isGreaterThanOrEqualTo(2); + ManifestFile manifest = + writeDeleteManifest(formatVersion, 1000L, FILE_A_DELETES, FILE_B_DELETES); + try (ManifestReader reader = + ManifestFiles.readDeleteManifest(manifest, FILE_IO, null)) { + for (DeleteFile file : reader) { + assertThat(file.manifestLocation()).isEqualTo(manifest.path()); + } + } + } + @TestTemplate public void testDataFileSplitOffsetsNullWhenInvalid() throws IOException { DataFile invalidOffset = From 5220a31454fd116cfa38f01ea1d86bf6e827d870 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 13 Sep 2024 18:04:14 +0200 Subject: [PATCH 0690/1019] Core: Allow servers to express supported endpoints via endpoint field in ConfigResponse (#10929) --- .../org/apache/iceberg/rest/Endpoint.java | 161 ++++++++++++++++++ .../iceberg/rest/RESTSessionCatalog.java | 156 +++++++++++++---- .../iceberg/rest/RESTTableOperations.java | 13 +- .../iceberg/rest/RESTViewOperations.java | 11 +- .../apache/iceberg/rest/ResourcePaths.java | 14 ++ .../rest/responses/ConfigResponse.java | 29 +++- .../rest/responses/ConfigResponseParser.java | 16 ++ .../iceberg/rest/RESTCatalogAdapter.java | 103 ++++++----- .../org/apache/iceberg/rest/TestEndpoint.java | 108 ++++++++++++ .../iceberg/rest/TestRESTViewCatalog.java | 8 +- ...RESTViewCatalogWithAssumedViewSupport.java | 95 +++++++++++ .../responses/TestConfigResponseParser.java | 79 +++++++++ 12 files changed, 704 insertions(+), 89 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/Endpoint.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java diff --git a/core/src/main/java/org/apache/iceberg/rest/Endpoint.java b/core/src/main/java/org/apache/iceberg/rest/Endpoint.java new file mode 100644 index 000000000000..2a8e6d633297 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/Endpoint.java @@ -0,0 +1,161 @@ +/* + * 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.Objects; +import java.util.Set; +import org.apache.hc.core5.http.Method; +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.base.Splitter; +import org.apache.iceberg.relocated.com.google.common.base.Strings; +import org.apache.iceberg.relocated.com.google.common.base.Supplier; + +/** + * Holds an endpoint definition that consists of the HTTP method (GET, POST, DELETE, ...) and the + * resource path as defined in the Iceberg OpenAPI REST specification without parameter + * substitution, such as /v1/{prefix}/namespaces/{namespace}. + */ +public class Endpoint { + + // namespace endpoints + public static final Endpoint V1_LIST_NAMESPACES = + Endpoint.create("GET", ResourcePaths.V1_NAMESPACES); + public static final Endpoint V1_LOAD_NAMESPACE = + Endpoint.create("GET", ResourcePaths.V1_NAMESPACE); + public static final Endpoint V1_CREATE_NAMESPACE = + Endpoint.create("POST", ResourcePaths.V1_NAMESPACES); + public static final Endpoint V1_UPDATE_NAMESPACE = + Endpoint.create("POST", ResourcePaths.V1_NAMESPACE_PROPERTIES); + public static final Endpoint V1_DELETE_NAMESPACE = + Endpoint.create("DELETE", ResourcePaths.V1_NAMESPACE); + public static final Endpoint V1_COMMIT_TRANSACTION = + Endpoint.create("POST", ResourcePaths.V1_TRANSACTIONS_COMMIT); + + // 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_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); + public static final Endpoint V1_RENAME_TABLE = + Endpoint.create("POST", ResourcePaths.V1_TABLE_RENAME); + public static final Endpoint V1_REGISTER_TABLE = + Endpoint.create("POST", ResourcePaths.V1_TABLE_REGISTER); + public static final Endpoint V1_REPORT_METRICS = + Endpoint.create("POST", ResourcePaths.V1_TABLE_METRICS); + + // 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_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); + public static final Endpoint V1_RENAME_VIEW = + Endpoint.create("POST", ResourcePaths.V1_VIEW_RENAME); + + private static final Splitter ENDPOINT_SPLITTER = Splitter.on(" "); + private static final Joiner ENDPOINT_JOINER = Joiner.on(" "); + private final String httpMethod; + private final String path; + + private Endpoint(String httpMethod, String path) { + Preconditions.checkArgument( + !Strings.isNullOrEmpty(httpMethod), "Invalid HTTP method: null or empty"); + Preconditions.checkArgument(!Strings.isNullOrEmpty(path), "Invalid path: null or empty"); + this.httpMethod = Method.normalizedValueOf(httpMethod).toString(); + this.path = path; + } + + public String httpMethod() { + return httpMethod; + } + + public String path() { + return path; + } + + public static Endpoint create(String httpMethod, String path) { + return new Endpoint(httpMethod, path); + } + + @Override + public String toString() { + return ENDPOINT_JOINER.join(httpMethod(), path()); + } + + public static Endpoint fromString(String endpoint) { + List elements = ENDPOINT_SPLITTER.splitToList(endpoint); + Preconditions.checkArgument( + elements.size() == 2, + "Invalid endpoint (must consist of two elements separated by a single space): %s", + endpoint); + return create(elements.get(0), elements.get(1)); + } + + /** + * Checks if the set of endpoints support the given {@link Endpoint}. + * + * @param supportedEndpoints The set of supported endpoints to check + * @param endpoint The endpoint to check against the set of supported endpoints + * @throws UnsupportedOperationException if the given {@link Endpoint} is not included in the set + * of endpoints. + */ + public static void check(Set supportedEndpoints, Endpoint endpoint) { + if (!supportedEndpoints.contains(endpoint)) { + throw new UnsupportedOperationException( + String.format("Server does not support endpoint: %s", endpoint)); + } + } + + /** + * Checks if the set of endpoints support the given {@link Endpoint}. + * + * @param supportedEndpoints The set of supported endpoints to check + * @param endpoint The endpoint to check against the set of supported endpoints + * @param supplier The supplier throwing a {@link RuntimeException} if the given {@link Endpoint} + * is not included in the set of endpoints. + */ + public static void check( + Set supportedEndpoints, Endpoint endpoint, Supplier supplier) { + if (!supportedEndpoints.contains(endpoint)) { + throw supplier.get(); + } + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof Endpoint)) { + return false; + } + + Endpoint endpoint = (Endpoint) o; + return Objects.equals(httpMethod, endpoint.httpMethod) && Objects.equals(path, endpoint.path); + } + + @Override + public int hashCode() { + return Objects.hash(httpMethod, path); + } +} 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 53ce45bb0a3f..cc42604f700d 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -58,7 +58,6 @@ import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.NoSuchViewException; -import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.hadoop.Configurable; import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.FileIO; @@ -116,6 +115,9 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private static final String DEFAULT_FILE_IO_IMPL = "org.apache.iceberg.io.ResolvingFileIO"; private static final String REST_METRICS_REPORTING_ENABLED = "rest-metrics-reporting-enabled"; private static final String REST_SNAPSHOT_LOADING_MODE = "snapshot-loading-mode"; + // for backwards compatibility with older REST servers where it can be assumed that a particular + // server supports view endpoints but doesn't send the "endpoints" field in the ConfigResponse + static final String VIEW_ENDPOINTS_SUPPORTED = "view-endpoints-supported"; public static final String REST_PAGE_SIZE = "rest-page-size"; private static final List TOKEN_PREFERENCE_ORDER = ImmutableList.of( @@ -132,6 +134,33 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog .addAll(TOKEN_PREFERENCE_ORDER) .build(); + private static final Set DEFAULT_ENDPOINTS = + ImmutableSet.builder() + .add(Endpoint.V1_LIST_NAMESPACES) + .add(Endpoint.V1_LOAD_NAMESPACE) + .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_CREATE_TABLE) + .add(Endpoint.V1_UPDATE_TABLE) + .add(Endpoint.V1_DELETE_TABLE) + .add(Endpoint.V1_RENAME_TABLE) + .add(Endpoint.V1_REGISTER_TABLE) + .add(Endpoint.V1_REPORT_METRICS) + .build(); + + private static final Set VIEW_ENDPOINTS = + ImmutableSet.builder() + .add(Endpoint.V1_LIST_VIEWS) + .add(Endpoint.V1_LOAD_VIEW) + .add(Endpoint.V1_CREATE_VIEW) + .add(Endpoint.V1_UPDATE_VIEW) + .add(Endpoint.V1_DELETE_VIEW) + .add(Endpoint.V1_RENAME_VIEW) + .build(); + private final Function, RESTClient> clientBuilder; private final BiFunction, FileIO> ioBuilder; private Cache sessions = null; @@ -148,6 +177,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private boolean reportingViaRestEnabled; private Integer pageSize = null; private CloseableGroup closeables = null; + private Set endpoints; // a lazy thread pool for token refresh private volatile ScheduledExecutorService refreshExecutor = null; @@ -173,6 +203,7 @@ public RESTSessionCatalog( this.ioBuilder = ioBuilder; } + @SuppressWarnings("checkstyle:CyclomaticComplexity") @Override public void initialize(String name, Map unresolved) { Preconditions.checkArgument(unresolved != null, "Invalid configuration: null"); @@ -230,6 +261,18 @@ public void initialize(String name, Map unresolved) { Map mergedProps = config.merge(props); Map baseHeaders = configHeaders(mergedProps); + if (config.endpoints().isEmpty()) { + this.endpoints = + PropertyUtil.propertyAsBoolean(mergedProps, VIEW_ENDPOINTS_SUPPORTED, false) + ? ImmutableSet.builder() + .addAll(DEFAULT_ENDPOINTS) + .addAll(VIEW_ENDPOINTS) + .build() + : DEFAULT_ENDPOINTS; + } else { + this.endpoints = ImmutableSet.copyOf(config.endpoints()); + } + this.sessions = newSessionCache(mergedProps); this.tableSessions = newSessionCache(mergedProps); this.keepTokenRefreshed = @@ -316,6 +359,10 @@ public void setConf(Object newConf) { @Override public List listTables(SessionContext context, Namespace ns) { + if (!endpoints.contains(Endpoint.V1_LIST_TABLES)) { + return ImmutableList.of(); + } + checkNamespaceIsValid(ns); Map queryParams = Maps.newHashMap(); ImmutableList.Builder tables = ImmutableList.builder(); @@ -342,6 +389,7 @@ public List listTables(SessionContext context, Namespace ns) { @Override public boolean dropTable(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_TABLE); checkIdentifierIsValid(identifier); try { @@ -355,6 +403,7 @@ public boolean dropTable(SessionContext context, TableIdentifier identifier) { @Override public boolean purgeTable(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_TABLE); checkIdentifierIsValid(identifier); try { @@ -372,6 +421,7 @@ public boolean purgeTable(SessionContext context, TableIdentifier identifier) { @Override public void renameTable(SessionContext context, TableIdentifier from, TableIdentifier to) { + Endpoint.check(endpoints, Endpoint.V1_RENAME_TABLE); checkIdentifierIsValid(from); checkIdentifierIsValid(to); @@ -384,6 +434,7 @@ public void renameTable(SessionContext context, TableIdentifier from, TableIdent private LoadTableResponse loadInternal( SessionContext context, TableIdentifier identifier, SnapshotMode mode) { + Endpoint.check(endpoints, Endpoint.V1_LOAD_TABLE); return client.get( paths.table(identifier), mode.params(), @@ -394,6 +445,14 @@ private LoadTableResponse loadInternal( @Override public Table loadTable(SessionContext context, TableIdentifier identifier) { + Endpoint.check( + endpoints, + Endpoint.V1_LOAD_TABLE, + () -> + new NoSuchTableException( + "Unable to load table %s.%s: Server does not support endpoint %s", + name(), identifier, Endpoint.V1_LOAD_TABLE)); + checkIdentifierIsValid(identifier); MetadataTableType metadataType; @@ -448,7 +507,8 @@ public Table loadTable(SessionContext context, TableIdentifier identifier) { paths.table(finalIdentifier), session::headers, tableFileIO(context, response.config()), - tableMetadata); + tableMetadata, + endpoints); trackFileIO(ops); @@ -472,7 +532,7 @@ private void trackFileIO(RESTTableOperations ops) { private MetricsReporter metricsReporter( String metricsEndpoint, Supplier> headers) { - if (reportingViaRestEnabled) { + if (reportingViaRestEnabled && endpoints.contains(Endpoint.V1_REPORT_METRICS)) { RESTMetricsReporter restMetricsReporter = new RESTMetricsReporter(client, metricsEndpoint, headers); return MetricsReporters.combine(reporter, restMetricsReporter); @@ -493,6 +553,7 @@ public void invalidateTable(SessionContext context, TableIdentifier ident) {} @Override public Table registerTable( SessionContext context, TableIdentifier ident, String metadataFileLocation) { + Endpoint.check(endpoints, Endpoint.V1_REGISTER_TABLE); checkIdentifierIsValid(ident); Preconditions.checkArgument( @@ -521,7 +582,8 @@ public Table registerTable( paths.table(ident), session::headers, tableFileIO(context, response.config()), - response.tableMetadata()); + response.tableMetadata(), + endpoints); trackFileIO(ops); @@ -532,6 +594,7 @@ public Table registerTable( @Override public void createNamespace( SessionContext context, Namespace namespace, Map metadata) { + Endpoint.check(endpoints, Endpoint.V1_CREATE_NAMESPACE); CreateNamespaceRequest request = CreateNamespaceRequest.builder().withNamespace(namespace).setProperties(metadata).build(); @@ -546,6 +609,10 @@ public void createNamespace( @Override public List listNamespaces(SessionContext context, Namespace namespace) { + if (!endpoints.contains(Endpoint.V1_LIST_NAMESPACES)) { + return ImmutableList.of(); + } + Map queryParams = Maps.newHashMap(); if (!namespace.isEmpty()) { queryParams.put("parent", RESTUtil.encodeNamespace(namespace)); @@ -575,6 +642,7 @@ public List listNamespaces(SessionContext context, Namespace namespac @Override public Map loadNamespaceMetadata(SessionContext context, Namespace ns) { + Endpoint.check(endpoints, Endpoint.V1_LOAD_NAMESPACE); checkNamespaceIsValid(ns); // TODO: rename to LoadNamespaceResponse? @@ -589,6 +657,7 @@ public Map loadNamespaceMetadata(SessionContext context, Namespa @Override public boolean dropNamespace(SessionContext context, Namespace ns) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_NAMESPACE); checkNamespaceIsValid(ns); try { @@ -603,6 +672,7 @@ public boolean dropNamespace(SessionContext context, Namespace ns) { @Override public boolean updateNamespaceMetadata( SessionContext context, Namespace ns, Map updates, Set removals) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_NAMESPACE); checkNamespaceIsValid(ns); UpdateNamespacePropertiesRequest request = @@ -719,6 +789,7 @@ public Builder withProperty(String key, String value) { @Override public Table create() { + Endpoint.check(endpoints, Endpoint.V1_CREATE_TABLE); CreateTableRequest request = CreateTableRequest.builder() .withName(ident.name()) @@ -744,7 +815,8 @@ public Table create() { paths.table(ident), session::headers, tableFileIO(context, response.config()), - response.tableMetadata()); + response.tableMetadata(), + endpoints); trackFileIO(ops); @@ -754,6 +826,7 @@ public Table create() { @Override public Transaction createTransaction() { + Endpoint.check(endpoints, Endpoint.V1_CREATE_TABLE); LoadTableResponse response = stageCreate(); String fullName = fullTableName(ident); @@ -768,7 +841,8 @@ public Transaction createTransaction() { tableFileIO(context, response.config()), RESTTableOperations.UpdateType.CREATE, createChanges(meta), - meta); + meta, + endpoints); trackFileIO(ops); @@ -778,15 +852,9 @@ public Transaction createTransaction() { @Override public Transaction replaceTransaction() { - try { - if (viewExists(context, ident)) { - throw new AlreadyExistsException("View with same name already exists: %s", ident); - } - } catch (RESTException | UnsupportedOperationException e) { - // don't fail if the server doesn't support views, which could be due to: - // 1. server or backing catalog doesn't support views - // 2. newer client talks to an older server that doesn't support views - LOG.debug("Failed to check whether view {} exists", ident, e); + Endpoint.check(endpoints, Endpoint.V1_UPDATE_TABLE); + if (viewExists(context, ident)) { + throw new AlreadyExistsException("View with same name already exists: %s", ident); } LoadTableResponse response = loadInternal(context, ident, snapshotMode); @@ -832,7 +900,8 @@ public Transaction replaceTransaction() { tableFileIO(context, response.config()), RESTTableOperations.UpdateType.REPLACE, changes.build(), - base); + base, + endpoints); trackFileIO(ops); @@ -1085,6 +1154,7 @@ private static Cache newSessionCache(Map pr } public void commitTransaction(SessionContext context, List commits) { + Endpoint.check(endpoints, Endpoint.V1_COMMIT_TRANSACTION); List tableChanges = Lists.newArrayListWithCapacity(commits.size()); for (TableCommit commit : commits) { @@ -1102,6 +1172,10 @@ public void commitTransaction(SessionContext context, List commits) @Override public List listViews(SessionContext context, Namespace namespace) { + if (!endpoints.contains(Endpoint.V1_LIST_VIEWS)) { + return ImmutableList.of(); + } + checkNamespaceIsValid(namespace); Map queryParams = Maps.newHashMap(); ImmutableList.Builder views = ImmutableList.builder(); @@ -1128,30 +1202,29 @@ public List listViews(SessionContext context, Namespace namespa @Override public View loadView(SessionContext context, TableIdentifier identifier) { + Endpoint.check( + endpoints, + Endpoint.V1_LOAD_VIEW, + () -> + new NoSuchViewException( + "Unable to load view %s.%s: Server does not support endpoint %s", + name(), identifier, Endpoint.V1_LOAD_VIEW)); + checkViewIdentifierIsValid(identifier); - LoadViewResponse response; - try { - response = - client.get( - paths.view(identifier), - LoadViewResponse.class, - headers(context), - ErrorHandlers.viewErrorHandler()); - } catch (UnsupportedOperationException | RESTException e) { - // Normally, copying an exception message is a bad practice but engines may show just the - // message and suppress the exception cause when the view does not exist. Since 401 and 403 - // responses can trigger this case, including the message increases the chances that the "Not - // authorized" or "Forbidden" message is preserved and shown. - throw new NoSuchViewException( - e, "Unable to load view %s.%s: %s", name(), identifier, e.getMessage()); - } + LoadViewResponse response = + client.get( + paths.view(identifier), + LoadViewResponse.class, + headers(context), + ErrorHandlers.viewErrorHandler()); AuthSession session = tableSession(response.config(), session(context)); ViewMetadata metadata = response.metadata(); RESTViewOperations ops = - new RESTViewOperations(client, paths.view(identifier), session::headers, metadata); + new RESTViewOperations( + client, paths.view(identifier), session::headers, metadata, endpoints); return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } @@ -1163,6 +1236,7 @@ public RESTViewBuilder buildView(SessionContext context, TableIdentifier identif @Override public boolean dropView(SessionContext context, TableIdentifier identifier) { + Endpoint.check(endpoints, Endpoint.V1_DELETE_VIEW); checkViewIdentifierIsValid(identifier); try { @@ -1176,6 +1250,7 @@ public boolean dropView(SessionContext context, TableIdentifier identifier) { @Override public void renameView(SessionContext context, TableIdentifier from, TableIdentifier to) { + Endpoint.check(endpoints, Endpoint.V1_RENAME_VIEW); checkViewIdentifierIsValid(from); checkViewIdentifierIsValid(to); @@ -1247,6 +1322,7 @@ public ViewBuilder withLocation(String newLocation) { @Override public View create() { + Endpoint.check(endpoints, Endpoint.V1_CREATE_VIEW); Preconditions.checkState( !representations.isEmpty(), "Cannot create view without specifying a query"); Preconditions.checkState(null != schema, "Cannot create view without specifying schema"); @@ -1284,7 +1360,7 @@ public View create() { AuthSession session = tableSession(response.config(), session(context)); RESTViewOperations ops = new RESTViewOperations( - client, paths.view(identifier), session::headers, response.metadata()); + client, paths.view(identifier), session::headers, response.metadata(), endpoints); return new BaseView(ops, ViewUtil.fullViewName(name(), identifier)); } @@ -1308,6 +1384,14 @@ public View replace() { } private LoadViewResponse loadView() { + Endpoint.check( + endpoints, + Endpoint.V1_LOAD_VIEW, + () -> + new NoSuchViewException( + "Unable to load view %s.%s: Server does not support endpoint %s", + name(), identifier, Endpoint.V1_LOAD_VIEW)); + return client.get( paths.view(identifier), LoadViewResponse.class, @@ -1316,6 +1400,7 @@ private LoadViewResponse loadView() { } private View replace(LoadViewResponse response) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_VIEW); Preconditions.checkState( !representations.isEmpty(), "Cannot replace view without specifying a query"); Preconditions.checkState(null != schema, "Cannot replace view without specifying schema"); @@ -1354,7 +1439,8 @@ private View replace(LoadViewResponse response) { AuthSession session = tableSession(response.config(), session(context)); RESTViewOperations ops = - new RESTViewOperations(client, paths.view(identifier), session::headers, metadata); + new RESTViewOperations( + client, paths.view(identifier), session::headers, metadata, endpoints); ops.commit(metadata, replacement); diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java index 0ce1afd93a79..5f6c28b32337 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java @@ -21,6 +21,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.Consumer; import java.util.function.Supplier; import org.apache.iceberg.LocationProviders; @@ -56,6 +57,7 @@ enum UpdateType { private final FileIO io; private final List createChanges; private final TableMetadata replaceBase; + private final Set endpoints; private UpdateType updateType; private TableMetadata current; @@ -64,8 +66,9 @@ enum UpdateType { String path, Supplier> headers, FileIO io, - TableMetadata current) { - this(client, path, headers, io, UpdateType.SIMPLE, Lists.newArrayList(), current); + TableMetadata current, + Set endpoints) { + this(client, path, headers, io, UpdateType.SIMPLE, Lists.newArrayList(), current, endpoints); } RESTTableOperations( @@ -75,7 +78,8 @@ enum UpdateType { FileIO io, UpdateType updateType, List createChanges, - TableMetadata current) { + TableMetadata current, + Set endpoints) { this.client = client; this.path = path; this.headers = headers; @@ -88,6 +92,7 @@ enum UpdateType { } else { this.current = current; } + this.endpoints = endpoints; } @Override @@ -97,12 +102,14 @@ public TableMetadata current() { @Override public TableMetadata refresh() { + Endpoint.check(endpoints, Endpoint.V1_LOAD_TABLE); return updateCurrentMetadata( client.get(path, LoadTableResponse.class, headers, ErrorHandlers.tableErrorHandler())); } @Override public void commit(TableMetadata base, TableMetadata metadata) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_TABLE); Consumer errorHandler; List requirements; List updates; diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java index b4dafaa9031b..466a8e66899b 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTViewOperations.java @@ -20,6 +20,7 @@ import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.function.Supplier; import org.apache.iceberg.UpdateRequirements; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -32,15 +33,21 @@ class RESTViewOperations implements ViewOperations { private final RESTClient client; private final String path; private final Supplier> headers; + private final Set endpoints; private ViewMetadata current; RESTViewOperations( - RESTClient client, String path, Supplier> headers, ViewMetadata current) { + RESTClient client, + String path, + Supplier> headers, + ViewMetadata current, + Set endpoints) { Preconditions.checkArgument(null != current, "Invalid view metadata: null"); this.client = client; this.path = path; this.headers = headers; this.current = current; + this.endpoints = endpoints; } @Override @@ -50,12 +57,14 @@ public ViewMetadata current() { @Override public ViewMetadata refresh() { + Endpoint.check(endpoints, Endpoint.V1_LOAD_VIEW); return updateCurrentMetadata( client.get(path, LoadViewResponse.class, headers, ErrorHandlers.viewErrorHandler())); } @Override public void commit(ViewMetadata base, ViewMetadata metadata) { + Endpoint.check(endpoints, Endpoint.V1_UPDATE_VIEW); // this is only used for replacing view metadata Preconditions.checkState(base != null, "Invalid base metadata: null"); 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 c68a4f450843..5ba7eae28262 100644 --- a/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java +++ b/core/src/main/java/org/apache/iceberg/rest/ResourcePaths.java @@ -26,6 +26,20 @@ public class ResourcePaths { private static final Joiner SLASH = Joiner.on("/").skipNulls(); private static final String PREFIX = "prefix"; + public static final String V1_NAMESPACES = "/v1/{prefix}/namespaces"; + public static final String V1_NAMESPACE = "/v1/{prefix}/namespaces/{namespace}"; + public static final String V1_NAMESPACE_PROPERTIES = + "/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_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_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}"; + public static final String V1_VIEW_RENAME = "/v1/{prefix}/views/rename"; public static ResourcePaths forCatalogProperties(Map properties) { return new ResourcePaths(properties.get(PREFIX)); diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java index f4efc0ff281a..da22ca287b30 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponse.java @@ -18,12 +18,16 @@ */ package org.apache.iceberg.rest.responses; +import java.util.List; import java.util.Map; import java.util.Objects; 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.Endpoint; import org.apache.iceberg.rest.RESTResponse; /** @@ -39,20 +43,24 @@ *
      *
    • defaults - properties that should be used as default configuration *
    • overrides - properties that should be used to override client configuration + *
    • endpoints - a list of endpoints that the server supports *
    */ public class ConfigResponse implements RESTResponse { private Map defaults; private Map overrides; + private List endpoints; public ConfigResponse() { // Required for Jackson deserialization } - private ConfigResponse(Map defaults, Map overrides) { + private ConfigResponse( + Map defaults, Map overrides, List endpoints) { this.defaults = defaults; this.overrides = overrides; + this.endpoints = endpoints; validate(); } @@ -80,6 +88,15 @@ public Map overrides() { return overrides != null ? overrides : ImmutableMap.of(); } + /** + * The list of available endpoints that the server supports + * + * @return A list of available endpoints that the server supports + */ + public List endpoints() { + return null != endpoints ? endpoints : ImmutableList.of(); + } + /** * Merge client-provided config with server side provided configuration to return a single * properties map which will be used for instantiating and configuring the REST catalog. @@ -107,6 +124,7 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("defaults", defaults) .add("overrides", overrides) + .add("endpoints", endpoints) .toString(); } @@ -117,10 +135,12 @@ public static Builder builder() { public static class Builder { private final Map defaults; private final Map overrides; + private final List endpoints; private Builder() { this.defaults = Maps.newHashMap(); this.overrides = Maps.newHashMap(); + this.endpoints = Lists.newArrayList(); } public Builder withDefault(String key, String value) { @@ -153,8 +173,13 @@ public Builder withOverrides(Map overridesToAdd) { return this; } + public Builder withEndpoints(List endpointsToAdd) { + endpoints.addAll(endpointsToAdd); + return this; + } + public ConfigResponse build() { - return new ConfigResponse(defaults, overrides); + return new ConfigResponse(defaults, overrides, endpoints); } } } diff --git a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java index 3240840e3e93..acadcce6d4bf 100644 --- a/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java +++ b/core/src/main/java/org/apache/iceberg/rest/responses/ConfigResponseParser.java @@ -21,13 +21,16 @@ import com.fasterxml.jackson.core.JsonGenerator; import com.fasterxml.jackson.databind.JsonNode; import java.io.IOException; +import java.util.stream.Collectors; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.rest.Endpoint; import org.apache.iceberg.util.JsonUtil; public class ConfigResponseParser { private static final String DEFAULTS = "defaults"; private static final String OVERRIDES = "overrides"; + private static final String ENDPOINTS = "endpoints"; private ConfigResponseParser() {} @@ -46,6 +49,12 @@ public static void toJson(ConfigResponse response, JsonGenerator gen) throws IOE JsonUtil.writeStringMap(DEFAULTS, response.defaults(), gen); JsonUtil.writeStringMap(OVERRIDES, response.overrides(), gen); + if (!response.endpoints().isEmpty()) { + JsonUtil.writeStringArray( + ENDPOINTS, + response.endpoints().stream().map(Endpoint::toString).collect(Collectors.toList()), + gen); + } gen.writeEndObject(); } @@ -67,6 +76,13 @@ public static ConfigResponse fromJson(JsonNode json) { builder.withOverrides(JsonUtil.getStringMapNullableValues(OVERRIDES, json)); } + if (json.hasNonNull(ENDPOINTS)) { + builder.withEndpoints( + JsonUtil.getStringList(ENDPOINTS, json).stream() + .map(Endpoint::fromString) + .collect(Collectors.toList())); + } + return builder.build(); } } 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 2c928c06e52b..6477dfcd00eb 100644 --- a/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java +++ b/core/src/test/java/org/apache/iceberg/rest/RESTCatalogAdapter.java @@ -19,9 +19,11 @@ package org.apache.iceberg.rest; import java.io.IOException; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.function.Consumer; +import java.util.stream.Collectors; import org.apache.iceberg.BaseTable; import org.apache.iceberg.BaseTransaction; import org.apache.iceberg.Table; @@ -115,61 +117,51 @@ enum Route { SEPARATE_AUTH_TOKENS_URI( HTTPMethod.POST, "https://auth-server.com/token", null, OAuthTokenResponse.class), CONFIG(HTTPMethod.GET, "v1/config", null, ConfigResponse.class), - LIST_NAMESPACES(HTTPMethod.GET, "v1/namespaces", null, ListNamespacesResponse.class), + LIST_NAMESPACES( + HTTPMethod.GET, ResourcePaths.V1_NAMESPACES, null, ListNamespacesResponse.class), CREATE_NAMESPACE( HTTPMethod.POST, - "v1/namespaces", + ResourcePaths.V1_NAMESPACES, CreateNamespaceRequest.class, CreateNamespaceResponse.class), - LOAD_NAMESPACE(HTTPMethod.GET, "v1/namespaces/{namespace}", null, GetNamespaceResponse.class), - DROP_NAMESPACE(HTTPMethod.DELETE, "v1/namespaces/{namespace}"), + LOAD_NAMESPACE(HTTPMethod.GET, ResourcePaths.V1_NAMESPACE, null, GetNamespaceResponse.class), + DROP_NAMESPACE(HTTPMethod.DELETE, ResourcePaths.V1_NAMESPACE), UPDATE_NAMESPACE( HTTPMethod.POST, - "v1/namespaces/{namespace}/properties", + ResourcePaths.V1_NAMESPACE_PROPERTIES, UpdateNamespacePropertiesRequest.class, UpdateNamespacePropertiesResponse.class), - LIST_TABLES(HTTPMethod.GET, "v1/namespaces/{namespace}/tables", null, ListTablesResponse.class), + LIST_TABLES(HTTPMethod.GET, ResourcePaths.V1_TABLES, null, ListTablesResponse.class), CREATE_TABLE( HTTPMethod.POST, - "v1/namespaces/{namespace}/tables", + ResourcePaths.V1_TABLES, CreateTableRequest.class, LoadTableResponse.class), - LOAD_TABLE( - HTTPMethod.GET, "v1/namespaces/{namespace}/tables/{name}", null, LoadTableResponse.class), + LOAD_TABLE(HTTPMethod.GET, ResourcePaths.V1_TABLE, null, LoadTableResponse.class), REGISTER_TABLE( HTTPMethod.POST, - "v1/namespaces/{namespace}/register", + ResourcePaths.V1_TABLE_REGISTER, RegisterTableRequest.class, LoadTableResponse.class), UPDATE_TABLE( - HTTPMethod.POST, - "v1/namespaces/{namespace}/tables/{name}", - UpdateTableRequest.class, - LoadTableResponse.class), - DROP_TABLE(HTTPMethod.DELETE, "v1/namespaces/{namespace}/tables/{name}"), - RENAME_TABLE(HTTPMethod.POST, "v1/tables/rename", RenameTableRequest.class, null), + HTTPMethod.POST, ResourcePaths.V1_TABLE, UpdateTableRequest.class, LoadTableResponse.class), + DROP_TABLE(HTTPMethod.DELETE, ResourcePaths.V1_TABLE), + RENAME_TABLE(HTTPMethod.POST, ResourcePaths.V1_TABLE_RENAME, RenameTableRequest.class, null), REPORT_METRICS( + HTTPMethod.POST, ResourcePaths.V1_TABLE_METRICS, ReportMetricsRequest.class, null), + COMMIT_TRANSACTION( HTTPMethod.POST, - "v1/namespaces/{namespace}/tables/{name}/metrics", - ReportMetricsRequest.class, + ResourcePaths.V1_TRANSACTIONS_COMMIT, + CommitTransactionRequest.class, null), - COMMIT_TRANSACTION( - HTTPMethod.POST, "v1/transactions/commit", CommitTransactionRequest.class, null), - LIST_VIEWS(HTTPMethod.GET, "v1/namespaces/{namespace}/views", null, ListTablesResponse.class), - LOAD_VIEW( - HTTPMethod.GET, "v1/namespaces/{namespace}/views/{name}", null, LoadViewResponse.class), + LIST_VIEWS(HTTPMethod.GET, ResourcePaths.V1_VIEWS, null, ListTablesResponse.class), + LOAD_VIEW(HTTPMethod.GET, ResourcePaths.V1_VIEW, null, LoadViewResponse.class), CREATE_VIEW( - HTTPMethod.POST, - "v1/namespaces/{namespace}/views", - CreateViewRequest.class, - LoadViewResponse.class), + HTTPMethod.POST, ResourcePaths.V1_VIEWS, CreateViewRequest.class, LoadViewResponse.class), UPDATE_VIEW( - HTTPMethod.POST, - "v1/namespaces/{namespace}/views/{name}", - UpdateTableRequest.class, - LoadViewResponse.class), - RENAME_VIEW(HTTPMethod.POST, "v1/views/rename", RenameTableRequest.class, null), - DROP_VIEW(HTTPMethod.DELETE, "v1/namespaces/{namespace}/views/{name}"); + HTTPMethod.POST, ResourcePaths.V1_VIEW, UpdateTableRequest.class, LoadViewResponse.class), + RENAME_VIEW(HTTPMethod.POST, ResourcePaths.V1_VIEW_RENAME, RenameTableRequest.class, null), + DROP_VIEW(HTTPMethod.DELETE, ResourcePaths.V1_VIEW); private final HTTPMethod method; private final int requiredLength; @@ -177,6 +169,7 @@ enum Route { private final Map variables; private final Class requestClass; private final Class responseClass; + private final String resourcePath; Route(HTTPMethod method, String pattern) { this(method, pattern, null, null); @@ -188,9 +181,11 @@ enum Route { Class requestClass, Class responseClass) { this.method = method; + this.resourcePath = pattern; // parse the pattern into requirements and variables - List parts = SLASH.splitToList(pattern); + List parts = + SLASH.splitToList(pattern.replaceFirst("/v1/", "v1/").replace("/{prefix}", "")); ImmutableMap.Builder requirementsBuilder = ImmutableMap.builder(); ImmutableMap.Builder variablesBuilder = ImmutableMap.builder(); for (int pos = 0; pos < parts.size(); pos += 1) { @@ -245,6 +240,14 @@ public Class requestClass() { public Class responseClass() { return responseClass; } + + HTTPMethod method() { + return method; + } + + String resourcePath() { + return resourcePath; + } } private static OAuthTokenResponse handleOAuthRequest(Object body) { @@ -282,7 +285,14 @@ public T handleRequest( return castResponse(responseType, handleOAuthRequest(body)); case CONFIG: - return castResponse(responseType, ConfigResponse.builder().build()); + return castResponse( + responseType, + ConfigResponse.builder() + .withEndpoints( + Arrays.stream(Route.values()) + .map(r -> Endpoint.create(r.method.name(), r.resourcePath)) + .collect(Collectors.toList())) + .build()); case LIST_NAMESPACES: if (asNamespaceCatalog != null) { @@ -371,16 +381,16 @@ public T handleRequest( case DROP_TABLE: { if (PropertyUtil.propertyAsBoolean(vars, "purgeRequested", false)) { - CatalogHandlers.purgeTable(catalog, identFromPathVars(vars)); + CatalogHandlers.purgeTable(catalog, tableIdentFromPathVars(vars)); } else { - CatalogHandlers.dropTable(catalog, identFromPathVars(vars)); + CatalogHandlers.dropTable(catalog, tableIdentFromPathVars(vars)); } return null; } case LOAD_TABLE: { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = tableIdentFromPathVars(vars); return castResponse(responseType, CatalogHandlers.loadTable(catalog, ident)); } @@ -394,7 +404,7 @@ public T handleRequest( case UPDATE_TABLE: { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = tableIdentFromPathVars(vars); UpdateTableRequest request = castRequest(UpdateTableRequest.class, body); return castResponse(responseType, CatalogHandlers.updateTable(catalog, ident, request)); } @@ -452,7 +462,7 @@ public T handleRequest( case LOAD_VIEW: { if (null != asViewCatalog) { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = viewIdentFromPathVars(vars); return castResponse(responseType, CatalogHandlers.loadView(asViewCatalog, ident)); } break; @@ -461,7 +471,7 @@ public T handleRequest( case UPDATE_VIEW: { if (null != asViewCatalog) { - TableIdentifier ident = identFromPathVars(vars); + TableIdentifier ident = viewIdentFromPathVars(vars); UpdateTableRequest request = castRequest(UpdateTableRequest.class, body); return castResponse( responseType, CatalogHandlers.updateView(asViewCatalog, ident, request)); @@ -482,7 +492,7 @@ public T handleRequest( case DROP_VIEW: { if (null != asViewCatalog) { - CatalogHandlers.dropView(asViewCatalog, identFromPathVars(vars)); + CatalogHandlers.dropView(asViewCatalog, viewIdentFromPathVars(vars)); return null; } break; @@ -668,8 +678,13 @@ private static Namespace namespaceFromPathVars(Map pathVars) { return RESTUtil.decodeNamespace(pathVars.get("namespace")); } - private static TableIdentifier identFromPathVars(Map pathVars) { + private static TableIdentifier tableIdentFromPathVars(Map pathVars) { + return TableIdentifier.of( + namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("table"))); + } + + private static TableIdentifier viewIdentFromPathVars(Map pathVars) { return TableIdentifier.of( - namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("name"))); + namespaceFromPathVars(pathVars), RESTUtil.decodeString(pathVars.get("view"))); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java b/core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java new file mode 100644 index 000000000000..1873d8799894 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestEndpoint.java @@ -0,0 +1,108 @@ +/* + * 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.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +public class TestEndpoint { + @Test + public void invalidValues() { + assertThatThrownBy(() -> Endpoint.create(null, "endpoint")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid HTTP method: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("", "endpoint")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid HTTP method: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("GET", null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid path: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("GET", "")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid path: null or empty"); + + assertThatThrownBy(() -> Endpoint.create("invalid", "/")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("No enum constant org.apache.hc.core5.http.Method.INVALID"); + } + + @ParameterizedTest + @ValueSource(strings = {"/path", " GET /path", "GET /path ", "GET /path", "GET /path /other"}) + public void invalidFromString(String endpoint) { + assertThatThrownBy(() -> Endpoint.fromString(endpoint)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid endpoint (must consist of two elements separated by a single space): %s", + endpoint); + } + + @Test + public void validFromString() { + Endpoint endpoint = Endpoint.fromString("GET /path"); + assertThat(endpoint.httpMethod()).isEqualTo("GET"); + assertThat(endpoint.path()).isEqualTo("/path"); + } + + @Test + public void toStringRepresentation() { + assertThat(Endpoint.create("POST", "/path/of/resource")) + .asString() + .isEqualTo("POST /path/of/resource"); + assertThat(Endpoint.create("GET", "/")).asString().isEqualTo("GET /"); + assertThat(Endpoint.create("PuT", "/")).asString().isEqualTo("PUT /"); + assertThat(Endpoint.create("PUT", "/namespaces/{namespace}/{x}")) + .asString() + .isEqualTo("PUT /namespaces/{namespace}/{x}"); + } + + @Test + public void supportedEndpoints() { + assertThatCode( + () -> Endpoint.check(ImmutableSet.of(Endpoint.V1_LOAD_TABLE), Endpoint.V1_LOAD_TABLE)) + .doesNotThrowAnyException(); + + assertThatCode( + () -> + Endpoint.check( + ImmutableSet.of(Endpoint.V1_LOAD_TABLE, Endpoint.V1_LOAD_VIEW), + Endpoint.V1_LOAD_TABLE)) + .doesNotThrowAnyException(); + } + + @Test + public void unsupportedEndpoints() { + assertThatThrownBy(() -> Endpoint.check(ImmutableSet.of(), Endpoint.V1_LOAD_TABLE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Server does not support endpoint: %s", Endpoint.V1_LOAD_TABLE); + + assertThatThrownBy( + () -> Endpoint.check(ImmutableSet.of(Endpoint.V1_LOAD_VIEW), Endpoint.V1_LOAD_TABLE)) + .isInstanceOf(UnsupportedOperationException.class) + .hasMessage("Server does not support endpoint: %s", Endpoint.V1_LOAD_TABLE); + } +} 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 db0969620dc9..dac2a9d25f4b 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -57,11 +57,11 @@ public class TestRESTViewCatalog extends ViewCatalogTests { private static final ObjectMapper MAPPER = RESTObjectMapper.mapper(); - @TempDir private Path temp; + @TempDir protected Path temp; - private RESTCatalog restCatalog; - private InMemoryCatalog backendCatalog; - private Server httpServer; + protected RESTCatalog restCatalog; + protected InMemoryCatalog backendCatalog; + protected Server httpServer; @BeforeEach public void createCatalog() throws Exception { diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java new file mode 100644 index 000000000000..3d7d64ddb794 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalogWithAssumedViewSupport.java @@ -0,0 +1,95 @@ +/* + * 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.apache.iceberg.rest.RESTCatalogAdapter.Route.CONFIG; + +import java.io.File; +import java.util.Map; +import java.util.UUID; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.catalog.SessionCatalog; +import org.apache.iceberg.inmemory.InMemoryCatalog; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.rest.responses.ConfigResponse; +import org.eclipse.jetty.server.Server; +import org.eclipse.jetty.server.handler.gzip.GzipHandler; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.junit.jupiter.api.BeforeEach; + +public class TestRESTViewCatalogWithAssumedViewSupport extends TestRESTViewCatalog { + + @BeforeEach + public void createCatalog() throws Exception { + File warehouse = temp.toFile(); + + this.backendCatalog = new InMemoryCatalog(); + this.backendCatalog.initialize( + "in-memory", + ImmutableMap.of(CatalogProperties.WAREHOUSE_LOCATION, warehouse.getAbsolutePath())); + + RESTCatalogAdapter adaptor = + new RESTCatalogAdapter(backendCatalog) { + + @Override + public T handleRequest( + Route route, Map vars, Object body, Class responseType) { + if (CONFIG == route) { + // simulate a legacy server that doesn't send back supported endpoints + return castResponse(responseType, ConfigResponse.builder().build()); + } + + return super.handleRequest(route, vars, body, responseType); + } + }; + + ServletContextHandler servletContext = + new ServletContextHandler(ServletContextHandler.NO_SESSIONS); + servletContext.setContextPath("/"); + servletContext.addServlet(new ServletHolder(new RESTCatalogServlet(adaptor)), "/*"); + servletContext.setHandler(new GzipHandler()); + + this.httpServer = new Server(0); + httpServer.setHandler(servletContext); + httpServer.start(); + + SessionCatalog.SessionContext context = + new SessionCatalog.SessionContext( + UUID.randomUUID().toString(), + "user", + ImmutableMap.of("credential", "user:12345"), + ImmutableMap.of()); + + this.restCatalog = + new RESTCatalog( + context, + (config) -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()); + restCatalog.initialize( + "prod", + ImmutableMap.of( + CatalogProperties.URI, + httpServer.getURI().toString(), + "credential", + "catalog:12345", + // assume that the server supports view endpoints + RESTSessionCatalog.VIEW_ENDPOINTS_SUPPORTED, + "true")); + } +} diff --git a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java index ec4c793c279f..81ec7cc5585c 100644 --- a/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestConfigResponseParser.java @@ -22,8 +22,11 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import com.fasterxml.jackson.databind.JsonNode; +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.Maps; +import org.apache.iceberg.rest.Endpoint; import org.junit.jupiter.api.Test; public class TestConfigResponseParser { @@ -106,6 +109,27 @@ public void overridesOnly() { .isEqualTo(expectedJson); } + @Test + public void endpointsOnly() { + List endpoints = + ImmutableList.of( + Endpoint.V1_LOAD_NAMESPACE, Endpoint.V1_LIST_NAMESPACES, Endpoint.V1_CREATE_NAMESPACE); + ConfigResponse response = ConfigResponse.builder().withEndpoints(endpoints).build(); + + String expectedJson = + "{\n" + + " \"defaults\" : { },\n" + + " \"overrides\" : { },\n" + + " \"endpoints\" : [ \"GET /v1/{prefix}/namespaces/{namespace}\", \"GET /v1/{prefix}/namespaces\", \"POST /v1/{prefix}/namespaces\" ]\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + assertThat(ConfigResponseParser.fromJson(json).endpoints()).isEqualTo(response.endpoints()); + } + @Test public void roundTripSerde() { Map defaults = Maps.newHashMap(); @@ -135,4 +159,59 @@ public void roundTripSerde() { assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) .isEqualTo(expectedJson); } + + @Test + public void invalidEndpoint() { + assertThatThrownBy( + () -> + ConfigResponseParser.fromJson( + "{\"endpoints\":[\"GET_v1/namespaces/{namespace}\"]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid endpoint (must consist of two elements separated by a single space): GET_v1/namespaces/{namespace}"); + + assertThatThrownBy( + () -> + ConfigResponseParser.fromJson( + "{\"endpoints\":[\"GET v1/namespaces/{namespace} INVALID\"]}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Invalid endpoint (must consist of two elements separated by a single space): GET v1/namespaces/{namespace} INVALID"); + } + + @Test + public void roundTripSerdeWithEndpoints() { + Map defaults = Maps.newHashMap(); + defaults.put("key1", "1"); + defaults.put("key2", null); + + Map overrides = Maps.newHashMap(); + overrides.put("key3", "23"); + overrides.put("key4", null); + + ConfigResponse response = + ConfigResponse.builder() + .withDefaults(defaults) + .withOverrides(overrides) + .withEndpoints(ImmutableList.of(Endpoint.V1_LOAD_TABLE, Endpoint.V1_LOAD_VIEW)) + .build(); + + String expectedJson = + "{\n" + + " \"defaults\" : {\n" + + " \"key1\" : \"1\",\n" + + " \"key2\" : null\n" + + " },\n" + + " \"overrides\" : {\n" + + " \"key3\" : \"23\",\n" + + " \"key4\" : null\n" + + " },\n" + + " \"endpoints\" : [ \"GET /v1/{prefix}/namespaces/{namespace}/tables/{table}\", \"GET /v1/{prefix}/namespaces/{namespace}/views/{view}\" ]\n" + + "}"; + + String json = ConfigResponseParser.toJson(response, true); + assertThat(json).isEqualTo(expectedJson); + assertThat(ConfigResponseParser.toJson(ConfigResponseParser.fromJson(json), true)) + .isEqualTo(expectedJson); + } } From 100faa5305317194b6a00ca121346b6e74ba8175 Mon Sep 17 00:00:00 2001 From: Naveen Kumar Date: Fri, 13 Sep 2024 21:45:18 +0530 Subject: [PATCH 0691/1019] Hive: Add View support for HIVE catalog (#9852) --- .../NoSuchIcebergViewException.java | 36 ++ .../java/org/apache/iceberg/CatalogUtil.java | 18 + .../org/apache/iceberg/hive/HiveCatalog.java | 200 ++++++- .../iceberg/hive/HiveOperationsBase.java | 13 + .../iceberg/hive/HiveTableOperations.java | 6 +- .../iceberg/hive/HiveViewOperations.java | 389 +++++++++++++ .../apache/iceberg/hive/TestHiveCatalog.java | 15 + .../iceberg/hive/TestHiveViewCatalog.java | 323 +++++++++++ .../iceberg/hive/TestHiveViewCommits.java | 516 ++++++++++++++++++ 9 files changed, 1508 insertions(+), 8 deletions(-) create mode 100644 api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.java create mode 100644 hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java create mode 100644 hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java create mode 100644 hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java diff --git a/api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.java b/api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.java new file mode 100644 index 000000000000..bc5da2aee280 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/exceptions/NoSuchIcebergViewException.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.exceptions; + +import com.google.errorprone.annotations.FormatMethod; + +/** NoSuchIcebergViewException thrown when a view is found, but it is not an Iceberg view. */ +public class NoSuchIcebergViewException extends NoSuchViewException { + @FormatMethod + public NoSuchIcebergViewException(String message, Object... args) { + super(message, args); + } + + @FormatMethod + public static void check(boolean test, String message, Object... args) { + if (!test) { + throw new NoSuchIcebergViewException(message, args); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/CatalogUtil.java b/core/src/main/java/org/apache/iceberg/CatalogUtil.java index d4fcbda0686d..70b10cbaeb62 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogUtil.java +++ b/core/src/main/java/org/apache/iceberg/CatalogUtil.java @@ -47,6 +47,7 @@ import org.apache.iceberg.util.PropertyUtil; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; +import org.apache.iceberg.view.ViewMetadata; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -137,6 +138,23 @@ public static void dropTableData(FileIO io, TableMetadata metadata) { deleteFile(io, metadata.metadataFileLocation(), "metadata"); } + /** + * Drops view metadata files referenced by ViewMetadata. + * + *

    This should be called by dropView implementations + * + * @param io a FileIO to use for deletes + * @param metadata the last valid ViewMetadata instance for a dropped view. + */ + public static void dropViewMetadata(FileIO io, ViewMetadata metadata) { + boolean gcEnabled = + PropertyUtil.propertyAsBoolean(metadata.properties(), GC_ENABLED, GC_ENABLED_DEFAULT); + + if (gcEnabled) { + deleteFile(io, metadata.metadataFileLocation(), "metadata"); + } + } + @SuppressWarnings("DangerousStringInternUsage") private static void deleteFiles(FileIO io, Set allManifests) { // keep track of deleted files in a map that can be cleaned up when memory runs low 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 5c58222f0c01..1cf738d736cb 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 @@ -27,6 +27,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hive.conf.HiveConf; import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; import org.apache.hadoop.hive.metastore.api.AlreadyExistsException; import org.apache.hadoop.hive.metastore.api.Database; import org.apache.hadoop.hive.metastore.api.InvalidOperationException; @@ -34,19 +35,21 @@ import org.apache.hadoop.hive.metastore.api.PrincipalType; import org.apache.hadoop.hive.metastore.api.Table; import org.apache.hadoop.hive.metastore.api.UnknownDBException; -import org.apache.iceberg.BaseMetastoreCatalog; import org.apache.iceberg.BaseMetastoreTableOperations; import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.CatalogUtil; import org.apache.iceberg.ClientPool; +import org.apache.iceberg.Schema; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; +import org.apache.iceberg.Transaction; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.SupportsNamespaces; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.exceptions.NamespaceNotEmptyException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; +import org.apache.iceberg.exceptions.NoSuchViewException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.hadoop.HadoopFileIO; import org.apache.iceberg.io.FileIO; @@ -56,13 +59,21 @@ 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.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.util.LocationUtil; +import org.apache.iceberg.view.BaseMetastoreViewCatalog; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewBuilder; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewOperations; import org.apache.thrift.TException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class HiveCatalog extends BaseMetastoreCatalog implements SupportsNamespaces, Configurable { +public class HiveCatalog extends BaseMetastoreViewCatalog + implements SupportsNamespaces, Configurable { public static final String LIST_ALL_TABLES = "list-all-tables"; public static final String LIST_ALL_TABLES_DEFAULT = "false"; @@ -117,6 +128,16 @@ public void initialize(String inputName, Map properties) { this.fileIOTracker = new FileIOTracker(); } + @Override + public TableBuilder buildTable(TableIdentifier identifier, Schema schema) { + return new ViewAwareTableBuilder(identifier, schema); + } + + @Override + public ViewBuilder buildView(TableIdentifier identifier) { + return new TableAwareViewBuilder(identifier); + } + @Override public List listTables(Namespace namespace) { Preconditions.checkArgument( @@ -156,6 +177,38 @@ public List listTables(Namespace namespace) { } } + @Override + public List listViews(Namespace namespace) { + Preconditions.checkArgument( + isValidateNamespace(namespace), "Missing database in namespace: %s", namespace); + + try { + String database = namespace.level(0); + List viewNames = + clients.run(client -> client.getTables(database, "*", TableType.VIRTUAL_VIEW)); + + // Retrieving the Table objects from HMS in batches to avoid OOM + List filteredTableIdentifiers = Lists.newArrayList(); + Iterable> viewNameSets = Iterables.partition(viewNames, 100); + + for (List viewNameSet : viewNameSets) { + filteredTableIdentifiers.addAll( + listIcebergTables(viewNameSet, namespace, HiveOperationsBase.ICEBERG_VIEW_TYPE_VALUE)); + } + + return filteredTableIdentifiers; + } catch (UnknownDBException e) { + throw new NoSuchNamespaceException("Namespace does not exist: %s", namespace); + + } catch (TException e) { + throw new RuntimeException("Failed to list all views under namespace " + namespace, e); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to listViews", e); + } + } + @Override public String name() { return name; @@ -213,11 +266,57 @@ public boolean dropTable(TableIdentifier identifier, boolean purge) { } } + @Override + public boolean dropView(TableIdentifier identifier) { + if (!isValidIdentifier(identifier)) { + return false; + } + + try { + String database = identifier.namespace().level(0); + String viewName = identifier.name(); + + HiveViewOperations ops = (HiveViewOperations) newViewOps(identifier); + ViewMetadata lastViewMetadata = null; + try { + lastViewMetadata = ops.current(); + } catch (NotFoundException e) { + LOG.warn("Failed to load view metadata for view: {}", identifier, e); + } + + clients.run( + client -> { + client.dropTable(database, viewName, false, false); + return null; + }); + + if (lastViewMetadata != null) { + CatalogUtil.dropViewMetadata(ops.io(), lastViewMetadata); + } + + LOG.info("Dropped view: {}", identifier); + return true; + } catch (NoSuchObjectException e) { + LOG.info("Skipping drop, view does not exist: {}", identifier, e); + return false; + } catch (TException e) { + throw new RuntimeException("Failed to drop view " + identifier, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted in call to dropView", e); + } + } + @Override public void renameTable(TableIdentifier from, TableIdentifier originalTo) { renameTableOrView(from, originalTo, HiveOperationsBase.ContentType.TABLE); } + @Override + public void renameView(TableIdentifier from, TableIdentifier to) { + renameTableOrView(from, to, HiveOperationsBase.ContentType.VIEW); + } + private List listIcebergTables( List tableNames, Namespace namespace, String tableTypeProp) throws TException, InterruptedException { @@ -233,13 +332,12 @@ private List listIcebergTables( .collect(Collectors.toList()); } + @SuppressWarnings("checkstyle:CyclomaticComplexity") private void renameTableOrView( TableIdentifier from, TableIdentifier originalTo, HiveOperationsBase.ContentType contentType) { - if (!isValidIdentifier(from)) { - throw new NoSuchTableException("Invalid identifier: %s", from); - } + Preconditions.checkArgument(isValidIdentifier(from), "Invalid identifier: %s", from); TableIdentifier to = removeCatalogName(originalTo); Preconditions.checkArgument(isValidIdentifier(to), "Invalid identifier: %s", to); @@ -248,6 +346,16 @@ private void renameTableOrView( "Cannot rename %s to %s. Namespace does not exist: %s", from, to, to.namespace()); } + if (tableExists(to)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Cannot rename %s to %s. Table already exists", from, to); + } + + if (viewExists(to)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Cannot rename %s to %s. View already exists", from, to); + } + String toDatabase = to.namespace().level(0); String fromDatabase = from.namespace().level(0); String fromName = from.name(); @@ -268,7 +376,12 @@ private void renameTableOrView( LOG.info("Renamed {} from {}, to {}", contentType.value(), from, to); } catch (NoSuchObjectException e) { - throw new NoSuchTableException("Table does not exist: %s", from); + switch (contentType) { + case TABLE: + throw new NoSuchTableException("Cannot rename %s to %s. Table does not exist", from, to); + case VIEW: + throw new NoSuchViewException("Cannot rename %s to %s. View does not exist", from, to); + } } catch (InvalidOperationException e) { if (e.getMessage() != null @@ -295,7 +408,7 @@ private void validateTableIsIcebergTableOrView( HiveOperationsBase.validateTableIsIceberg(table, fullName); break; case VIEW: - throw new UnsupportedOperationException("View is not supported."); + HiveOperationsBase.validateTableIsIcebergView(table, fullName); } } @@ -522,6 +635,11 @@ public TableOperations newTableOps(TableIdentifier tableIdentifier) { return ops; } + @Override + protected ViewOperations newViewOps(TableIdentifier identifier) { + return new HiveViewOperations(conf, clients, fileIO, name, identifier); + } + @Override protected String defaultWarehouseLocation(TableIdentifier tableIdentifier) { // This is a little edgy since we basically duplicate the HMS location generation logic. @@ -660,4 +778,72 @@ void setListAllTables(boolean listAllTables) { ClientPool clientPool() { return clients; } + + /** + * The purpose of this class is to add view detection only for Hive-Specific tables. Hive catalog + * follows checks at different levels: 1. During refresh, it validates if the table is an iceberg + * table or not. 2. During commit, it validates if there is any concurrent commit with table or + * table-name already exists. This class helps to do the validation on an early basis. + */ + private class ViewAwareTableBuilder extends BaseMetastoreViewCatalogTableBuilder { + + private final TableIdentifier identifier; + + private ViewAwareTableBuilder(TableIdentifier identifier, Schema schema) { + super(identifier, schema); + this.identifier = identifier; + } + + @Override + public Transaction createOrReplaceTransaction() { + if (viewExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "View with same name already exists: %s", identifier); + } + return super.createOrReplaceTransaction(); + } + + @Override + public org.apache.iceberg.Table create() { + if (viewExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "View with same name already exists: %s", identifier); + } + return super.create(); + } + } + + /** + * The purpose of this class is to add table detection only for Hive-Specific view. Hive catalog + * follows checks at different levels: 1. During refresh, it validates if the view is an iceberg + * view or not. 2. During commit, it validates if there is any concurrent commit with view or + * view-name already exists. This class helps to do the validation on an early basis. + */ + private class TableAwareViewBuilder extends BaseViewBuilder { + + private final TableIdentifier identifier; + + private TableAwareViewBuilder(TableIdentifier identifier) { + super(identifier); + this.identifier = identifier; + } + + @Override + public View createOrReplace() { + if (tableExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Table with same name already exists: %s", identifier); + } + return super.createOrReplace(); + } + + @Override + public View create() { + if (tableExists(identifier)) { + throw new org.apache.iceberg.exceptions.AlreadyExistsException( + "Table with same name already exists: %s", identifier); + } + return super.create(); + } + } } diff --git a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java index 6500e724a4f0..4c78c43096fe 100644 --- a/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveOperationsBase.java @@ -33,6 +33,7 @@ import org.apache.iceberg.SchemaParser; import org.apache.iceberg.TableProperties; import org.apache.iceberg.exceptions.NoSuchIcebergTableException; +import org.apache.iceberg.exceptions.NoSuchIcebergViewException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -53,6 +54,7 @@ interface HiveOperationsBase { long HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT = 32672; String NO_LOCK_EXPECTED_KEY = "expected_parameter_key"; String NO_LOCK_EXPECTED_VALUE = "expected_parameter_value"; + String ICEBERG_VIEW_TYPE_VALUE = "iceberg-view"; enum ContentType { TABLE("Table"), @@ -129,6 +131,17 @@ static void validateTableIsIceberg(Table table, String fullName) { tableType); } + static void validateTableIsIcebergView(Table table, String fullName) { + String tableTypeProp = table.getParameters().get(BaseMetastoreTableOperations.TABLE_TYPE_PROP); + NoSuchIcebergViewException.check( + TableType.VIRTUAL_VIEW.name().equalsIgnoreCase(table.getTableType()) + && ICEBERG_VIEW_TYPE_VALUE.equalsIgnoreCase(tableTypeProp), + "Not an iceberg view: %s (type=%s) (tableType=%s)", + fullName, + tableTypeProp, + table.getTableType()); + } + default void persistTable(Table hmsTable, boolean updateHiveTable, String metadataLocation) throws TException, InterruptedException { if (updateHiveTable) { 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 64f091385297..518daaf6acd1 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 @@ -167,7 +167,7 @@ protected void doRefresh() { refreshFromMetadataLocation(metadataLocation, metadataRefreshMaxRetries); } - @SuppressWarnings("checkstyle:CyclomaticComplexity") + @SuppressWarnings({"checkstyle:CyclomaticComplexity", "MethodLength"}) @Override protected void doCommit(TableMetadata base, TableMetadata metadata) { boolean newTable = base == null; @@ -191,6 +191,10 @@ protected void doCommit(TableMetadata base, TableMetadata metadata) { if (newTable && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) != null) { + if (TableType.VIRTUAL_VIEW.name().equalsIgnoreCase(tbl.getTableType())) { + throw new AlreadyExistsException( + "View with same name already exists: %s.%s", database, tableName); + } throw new AlreadyExistsException("Table already exists: %s.%s", database, tableName); } 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 new file mode 100644 index 000000000000..4fc71299d457 --- /dev/null +++ b/hive-metastore/src/main/java/org/apache/iceberg/hive/HiveViewOperations.java @@ -0,0 +1,389 @@ +/* + * 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.hive; + +import static java.util.Collections.emptySet; + +import java.util.Collections; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.TableType; +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.BaseMetastoreTableOperations; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.ClientPool; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.AlreadyExistsException; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.NoSuchViewException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.hadoop.ConfigProperties; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.view.BaseViewOperations; +import org.apache.iceberg.view.SQLViewRepresentation; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.iceberg.view.ViewRepresentation; +import org.apache.thrift.TException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Hive implementation of Iceberg {@link org.apache.iceberg.view.ViewOperations}. */ +final class HiveViewOperations extends BaseViewOperations implements HiveOperationsBase { + private static final Logger LOG = LoggerFactory.getLogger(HiveViewOperations.class); + + private final String fullName; + private final String database; + private final String viewName; + private final FileIO fileIO; + private final ClientPool metaClients; + private final long maxHiveTablePropertySize; + private final Configuration conf; + private final String catalogName; + + HiveViewOperations( + Configuration conf, + ClientPool metaClients, + FileIO fileIO, + String catalogName, + TableIdentifier viewIdentifier) { + this.conf = conf; + this.catalogName = catalogName; + this.metaClients = metaClients; + this.fileIO = fileIO; + this.fullName = CatalogUtil.fullTableName(catalogName, viewIdentifier); + this.database = viewIdentifier.namespace().level(0); + this.viewName = viewIdentifier.name(); + this.maxHiveTablePropertySize = + conf.getLong(HIVE_TABLE_PROPERTY_MAX_SIZE, HIVE_TABLE_PROPERTY_MAX_SIZE_DEFAULT); + } + + @Override + public void doRefresh() { + String metadataLocation = null; + Table table; + + try { + table = metaClients.run(client -> client.getTable(database, viewName)); + HiveOperationsBase.validateTableIsIcebergView(table, fullName); + + metadataLocation = + table.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); + + } catch (NoSuchObjectException e) { + if (currentMetadataLocation() != null) { + throw new NoSuchViewException("View does not exist: %s.%s", database, viewName); + } + } catch (TException e) { + String errMsg = + String.format("Failed to get view info from metastore %s.%s", database, viewName); + throw new RuntimeException(errMsg, e); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during refresh", e); + } + + refreshFromMetadataLocation(metadataLocation); + } + + @SuppressWarnings("checkstyle:CyclomaticComplexity") + @Override + public void doCommit(ViewMetadata base, ViewMetadata metadata) { + boolean newView = base == null; + String newMetadataLocation = writeNewMetadataIfRequired(metadata); + boolean hiveEngineEnabled = false; + + CommitStatus commitStatus = CommitStatus.FAILURE; + boolean updateHiveView = false; + + HiveLock lock = lockObject(); + try { + lock.lock(); + + Table tbl = loadHmsTable(); + + if (tbl != null) { + // If we try to create the view but the metadata location is already set, then we had a + // concurrent commit + if (newView + && tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP) + != null) { + throw new AlreadyExistsException( + "%s already exists: %s.%s", + TableType.VIRTUAL_VIEW.name().equalsIgnoreCase(tbl.getTableType()) + ? ContentType.VIEW.value() + : ContentType.TABLE.value(), + database, + viewName); + } + + updateHiveView = true; + LOG.debug("Committing existing view: {}", fullName); + } else { + tbl = newHMSView(metadata); + LOG.debug("Committing new view: {}", fullName); + } + + tbl.setSd( + HiveOperationsBase.storageDescriptor( + metadata.schema(), + metadata.location(), + hiveEngineEnabled)); // set to pick up any schema changes + + String metadataLocation = + tbl.getParameters().get(BaseMetastoreTableOperations.METADATA_LOCATION_PROP); + String baseMetadataLocation = base != null ? base.metadataFileLocation() : null; + if (!Objects.equals(baseMetadataLocation, metadataLocation)) { + throw new CommitFailedException( + "Cannot commit: Base metadata location '%s' is not same as the current view metadata location '%s' for %s.%s", + baseMetadataLocation, metadataLocation, database, viewName); + } + + // get Iceberg props that have been removed + Set removedProps = emptySet(); + if (base != null) { + removedProps = + base.properties().keySet().stream() + .filter(key -> !metadata.properties().containsKey(key)) + .collect(Collectors.toSet()); + } + + setHmsTableParameters(newMetadataLocation, tbl, metadata, removedProps); + + lock.ensureActive(); + + try { + persistTable(tbl, updateHiveView, hiveLockEnabled(conf) ? null : baseMetadataLocation); + lock.ensureActive(); + + commitStatus = CommitStatus.SUCCESS; + } catch (LockException le) { + commitStatus = CommitStatus.UNKNOWN; + throw new CommitStateUnknownException( + "Failed to heartbeat for hive lock while " + + "committing changes. This can lead to a concurrent commit attempt be able to overwrite this commit. " + + "Please check the commit history. If you are running into this issue, try reducing " + + "iceberg.hive.lock-heartbeat-interval-ms.", + le); + } catch (org.apache.hadoop.hive.metastore.api.AlreadyExistsException e) { + throw new AlreadyExistsException(e, "View already exists: %s.%s", database, viewName); + + } catch (InvalidObjectException e) { + throw new ValidationException(e, "Invalid Hive object for %s.%s", database, viewName); + + } catch (CommitFailedException | CommitStateUnknownException e) { + throw e; + + } catch (Throwable e) { + if (e.getMessage() != null + && e.getMessage() + .contains( + "The table has been modified. The parameter value for key '" + + BaseMetastoreTableOperations.METADATA_LOCATION_PROP + + "' is")) { + throw new CommitFailedException( + e, "The view %s.%s has been modified concurrently", database, viewName); + } + + if (e.getMessage() != null + && e.getMessage().contains("Table/View 'HIVE_LOCKS' does not exist")) { + throw new RuntimeException( + "Failed to acquire locks from metastore because the underlying metastore " + + "view 'HIVE_LOCKS' does not exist. This can occur when using an embedded metastore which does not " + + "support transactions. To fix this use an alternative metastore.", + e); + } + + LOG.error( + "Cannot tell if commit to {}.{} succeeded, attempting to reconnect and check.", + database, + viewName, + e); + commitStatus = + checkCommitStatus( + viewName, + newMetadataLocation, + metadata.properties(), + () -> checkCurrentMetadataLocation(newMetadataLocation)); + switch (commitStatus) { + case SUCCESS: + break; + case FAILURE: + throw e; + case UNKNOWN: + throw new CommitStateUnknownException(e); + } + } + } catch (TException e) { + throw new RuntimeException( + String.format("Metastore operation failed for %s.%s", database, viewName), e); + + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException("Interrupted during commit", e); + + } catch (LockException e) { + throw new CommitFailedException(e); + + } finally { + HiveOperationsBase.cleanupMetadataAndUnlock(io(), commitStatus, newMetadataLocation, lock); + } + + LOG.info( + "Committed to view {} with the new metadata location {}", fullName, newMetadataLocation); + } + + /** + * Validate if the new metadata location is the current metadata location. + * + * @param newMetadataLocation newly written metadata location + * @return true if the new metadata location is the current metadata location + */ + private boolean checkCurrentMetadataLocation(String newMetadataLocation) { + ViewMetadata metadata = refresh(); + return newMetadataLocation.equals(metadata.metadataFileLocation()); + } + + private void setHmsTableParameters( + String newMetadataLocation, Table tbl, ViewMetadata metadata, Set obsoleteProps) { + Map parameters = + Optional.ofNullable(tbl.getParameters()).orElseGet(Maps::newHashMap); + + // push all Iceberg view properties into HMS + metadata.properties().entrySet().stream() + .filter(entry -> !entry.getKey().equalsIgnoreCase(HiveCatalog.HMS_TABLE_OWNER)) + .forEach(entry -> parameters.put(entry.getKey(), entry.getValue())); + if (metadata.uuid() != null) { + parameters.put("uuid", metadata.uuid()); + } + + // remove any props from HMS that are no longer present in Iceberg view props + obsoleteProps.forEach(parameters::remove); + + parameters.put( + BaseMetastoreTableOperations.TABLE_TYPE_PROP, + ICEBERG_VIEW_TYPE_VALUE.toUpperCase(Locale.ENGLISH)); + parameters.put(BaseMetastoreTableOperations.METADATA_LOCATION_PROP, newMetadataLocation); + + if (currentMetadataLocation() != null && !currentMetadataLocation().isEmpty()) { + parameters.put( + BaseMetastoreTableOperations.PREVIOUS_METADATA_LOCATION_PROP, currentMetadataLocation()); + } + + setSchema(metadata.schema(), parameters); + tbl.setParameters(parameters); + } + + private static boolean hiveLockEnabled(Configuration conf) { + return conf.getBoolean(ConfigProperties.LOCK_HIVE_ENABLED, true); + } + + private Table newHMSView(ViewMetadata metadata) { + final long currentTimeMillis = System.currentTimeMillis(); + String hmsTableOwner = + PropertyUtil.propertyAsString( + metadata.properties(), HiveCatalog.HMS_TABLE_OWNER, HiveHadoopUtil.currentUser()); + String sqlQuery = sqlFor(metadata); + + return new Table( + table(), + database(), + hmsTableOwner, + (int) currentTimeMillis / 1000, + (int) currentTimeMillis / 1000, + Integer.MAX_VALUE, + null, + Collections.emptyList(), + Maps.newHashMap(), + sqlQuery, + sqlQuery, + tableType().name()); + } + + private String sqlFor(ViewMetadata metadata) { + SQLViewRepresentation closest = null; + for (ViewRepresentation representation : metadata.currentVersion().representations()) { + if (representation instanceof SQLViewRepresentation) { + SQLViewRepresentation sqlViewRepresentation = (SQLViewRepresentation) representation; + if (sqlViewRepresentation.dialect().equalsIgnoreCase("hive")) { + return sqlViewRepresentation.sql(); + } else if (closest == null) { + closest = sqlViewRepresentation; + } + } + } + + return closest == null ? null : closest.sql(); + } + + @VisibleForTesting + HiveLock lockObject() { + if (hiveLockEnabled(conf)) { + return new MetastoreLock(conf, metaClients, catalogName, database, viewName); + } else { + return new NoLock(); + } + } + + @Override + protected String viewName() { + return fullName; + } + + @Override + public TableType tableType() { + return TableType.VIRTUAL_VIEW; + } + + @Override + public ClientPool metaClients() { + return metaClients; + } + + @Override + public long maxHiveTablePropertySize() { + return maxHiveTablePropertySize; + } + + @Override + public String database() { + return database; + } + + @Override + public String table() { + return viewName; + } + + @Override + public FileIO io() { + return fileIO; + } +} 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 9249deb7598e..7d0eb641a385 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 @@ -155,6 +155,21 @@ private Schema getTestSchema() { required(2, "data", Types.StringType.get())); } + @Test + public void testInvalidIdentifiersWithRename() { + TableIdentifier invalidFrom = TableIdentifier.of(Namespace.of("l1", "l2"), "table1"); + TableIdentifier validTo = TableIdentifier.of(Namespace.of("l1"), "renamedTable"); + assertThatThrownBy(() -> catalog.renameTable(invalidFrom, validTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidFrom); + + TableIdentifier validFrom = TableIdentifier.of(Namespace.of("l1"), "table1"); + TableIdentifier invalidTo = TableIdentifier.of(Namespace.of("l1", "l2"), "renamedTable"); + assertThatThrownBy(() -> catalog.renameTable(validFrom, invalidTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidTo); + } + @Test public void testCreateTableBuilder() throws Exception { Schema schema = getTestSchema(); diff --git a/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.java new file mode 100644 index 000000000000..3c195e256520 --- /dev/null +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCatalog.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.hive; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.nio.file.Files; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.metastore.api.SerDeInfo; +import org.apache.hadoop.hive.metastore.api.StorageDescriptor; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.catalog.Catalog; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.NoSuchIcebergViewException; +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.view.BaseView; +import org.apache.iceberg.view.ViewCatalogTests; +import org.apache.thrift.TException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +public class TestHiveViewCatalog extends ViewCatalogTests { + + private HiveCatalog catalog; + + @RegisterExtension + private static final HiveMetastoreExtension HIVE_METASTORE_EXTENSION = + HiveMetastoreExtension.builder().build(); + + @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()); + } + + @AfterEach + public void cleanup() throws Exception { + HIVE_METASTORE_EXTENSION.metastore().reset(); + } + + @Override + protected HiveCatalog catalog() { + return catalog; + } + + @Override + protected Catalog tableCatalog() { + return catalog; + } + + @Override + protected boolean requiresNamespaceCreate() { + return true; + } + + @Test + public void testHiveViewAndIcebergViewWithSameName() throws TException, IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + String viewName = "test_hive_view"; + TableIdentifier identifier = TableIdentifier.of(ns, viewName); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + assertThat(catalog.listViews(ns)).isEmpty(); + // create a hive table + Table hiveTable = + createHiveView( + viewName, dbName, Files.createTempDirectory("hive-view-tests-name").toString()); + HIVE_METASTORE_EXTENSION.metastoreClient().createTable(hiveTable); + + catalog.setListAllTables(true); + assertThat(catalog.listTables(ns)).containsExactly(identifier).hasSize(1); + + assertThat(catalog.viewExists(identifier)).isFalse(); + + assertThatThrownBy( + () -> + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .create()) + .isInstanceOf(NoSuchIcebergViewException.class) + .hasMessageStartingWith("Not an iceberg view: hive.hivedb.test_hive_view"); + } + + @Test + public void testListViewWithHiveView() throws TException, IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_view"); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + assertThat(catalog.viewExists(identifier)).isFalse(); + assertThat(catalog.listViews(ns)).isEmpty(); + + String hiveViewName = "test_hive_view"; + // create a hive table + Table hiveTable = + createHiveView( + hiveViewName, dbName, Files.createTempDirectory("hive-view-tests-list").toString()); + HIVE_METASTORE_EXTENSION.metastoreClient().createTable(hiveTable); + + catalog.setListAllTables(true); + + assertThat(catalog.listTables(ns)) + .containsExactly(TableIdentifier.of(ns, hiveViewName)) + .hasSize(1); + + assertThat(catalog.listViews(ns)).hasSize(0); + + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .create(); + assertThat(catalog.viewExists(identifier)).isTrue(); + + assertThat(catalog.listViews(ns)).containsExactly(identifier).hasSize(1); + } + + @Test + public void testViewWithHiveParameters() throws TException, IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_view"); + + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + assertThat(catalog.viewExists(identifier)).isFalse(); + String tableQuery = "select * from hivedb.tbl"; + + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", tableQuery) + .create(); + assertThat(catalog.viewExists(identifier)).isTrue(); + + Table hiveTable = + HIVE_METASTORE_EXTENSION.metastoreClient().getTable(dbName, identifier.name()); + assertThat(hiveTable.getViewOriginalText()).isEqualTo(tableQuery); + assertThat(hiveTable.getViewExpandedText()).isEqualTo(tableQuery); + } + + @Test + public void testInvalidIdentifiersWithRename() { + TableIdentifier invalidFrom = TableIdentifier.of(Namespace.of("l1", "l2"), "view"); + TableIdentifier validTo = TableIdentifier.of(Namespace.of("l1"), "renamedView"); + assertThatThrownBy(() -> catalog.renameView(invalidFrom, validTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidFrom); + + TableIdentifier validFrom = TableIdentifier.of(Namespace.of("l1"), "view"); + TableIdentifier invalidTo = TableIdentifier.of(Namespace.of("l1", "l2"), "renamedView"); + assertThatThrownBy(() -> catalog.renameView(validFrom, invalidTo)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid identifier: " + invalidTo); + } + + @Test + public void dropViewShouldNotDropMetadataFileIfGcNotEnabled() throws IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_drop_view_gc_disabled"); + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + BaseView view = + (BaseView) + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .withProperty(TableProperties.GC_ENABLED, "false") + .create(); + + assertThat(catalog.viewExists(identifier)).isTrue(); + + Path viewLocation = new Path(view.location()); + String currentMetadataLocation = view.operations().current().metadataFileLocation(); + + catalog.dropView(identifier); + + assertThat( + viewLocation + .getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()) + .exists(new Path(currentMetadataLocation))) + .isTrue(); + assertThat(catalog.viewExists(identifier)).isFalse(); + } + + @Test + public void dropViewShouldDropMetadataFileIfGcEnabled() throws IOException { + String dbName = "hivedb"; + Namespace ns = Namespace.of(dbName); + TableIdentifier identifier = TableIdentifier.of(ns, "test_iceberg_drop_view_gc_enabled"); + if (requiresNamespaceCreate()) { + catalog.createNamespace(identifier.namespace()); + } + + BaseView view = + (BaseView) + catalog + .buildView(identifier) + .withSchema(SCHEMA) + .withDefaultNamespace(ns) + .withQuery("hive", "select * from hivedb.tbl") + .withProperty(TableProperties.GC_ENABLED, "true") + .create(); + + assertThat(catalog.viewExists(identifier)).isTrue(); + + Path viewLocation = new Path(view.location()); + String currentMetadataLocation = view.operations().current().metadataFileLocation(); + + assertThat( + viewLocation + .getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()) + .exists(new Path(currentMetadataLocation))) + .isTrue(); + + catalog.dropView(identifier); + + assertThat( + viewLocation + .getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()) + .exists(new Path(currentMetadataLocation))) + .isFalse(); + assertThat(catalog.viewExists(identifier)).isFalse(); + } + + private Table createHiveView(String hiveViewName, String dbName, String location) { + Map parameters = Maps.newHashMap(); + parameters.put( + serdeConstants.SERIALIZATION_CLASS, "org.apache.hadoop.hive.serde2.thrift.test.IntString"); + parameters.put( + serdeConstants.SERIALIZATION_FORMAT, "org.apache.thrift.protocol.TBinaryProtocol"); + + SerDeInfo serDeInfo = + new SerDeInfo(null, "org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer", parameters); + + // StorageDescriptor has an empty list of fields - SerDe will report them. + StorageDescriptor sd = + new StorageDescriptor( + Lists.newArrayList(), + location, + "org.apache.hadoop.mapred.TextInputFormat", + "org.apache.hadoop.mapred.TextOutputFormat", + false, + -1, + serDeInfo, + Lists.newArrayList(), + Lists.newArrayList(), + Maps.newHashMap()); + + Table hiveTable = + new Table( + hiveViewName, + dbName, + "test_owner", + 0, + 0, + 0, + sd, + Lists.newArrayList(), + Maps.newHashMap(), + "viewOriginalText", + "viewExpandedText", + TableType.VIRTUAL_VIEW.name()); + return hiveTable; + } +} 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 new file mode 100644 index 000000000000..47abb51602fa --- /dev/null +++ b/hive-metastore/src/test/java/org/apache/iceberg/hive/TestHiveViewCommits.java @@ -0,0 +1,516 @@ +/* + * 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.hive; + +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.Mockito.any; +import static org.mockito.Mockito.anyBoolean; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.doReturn; +import static org.mockito.Mockito.doThrow; +import static org.mockito.Mockito.spy; +import static org.mockito.Mockito.when; + +import java.io.File; +import java.io.IOException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicReference; +import org.apache.hadoop.fs.Path; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.CatalogUtil; +import org.apache.iceberg.Schema; +import org.apache.iceberg.catalog.Namespace; +import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.exceptions.CommitFailedException; +import org.apache.iceberg.exceptions.CommitStateUnknownException; +import org.apache.iceberg.exceptions.ValidationException; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.view.BaseView; +import org.apache.iceberg.view.View; +import org.apache.iceberg.view.ViewMetadata; +import org.apache.thrift.TException; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +/** Test Hive locks and Hive errors and retry during commits. */ +public class TestHiveViewCommits { + + private static final String VIEW_NAME = "test_iceberg_view"; + private static final String DB_NAME = "hivedb"; + private static final Namespace NS = Namespace.of(DB_NAME); + private static final Schema SCHEMA = + new Schema( + 5, + required(3, "id", Types.IntegerType.get(), "unique ID"), + required(4, "data", Types.StringType.get())); + private static final TableIdentifier VIEW_IDENTIFIER = TableIdentifier.of(NS, VIEW_NAME); + + @RegisterExtension + protected static final HiveMetastoreExtension HIVE_METASTORE_EXTENSION = + HiveMetastoreExtension.builder().withDatabase(DB_NAME).build(); + + private View view; + private Path viewLocation; + + private static HiveCatalog catalog; + + @BeforeAll + public static void initCatalog() { + 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()); + } + + @BeforeEach + public void createTestView() { + view = + catalog + .buildView(VIEW_IDENTIFIER) + .withSchema(SCHEMA) + .withDefaultNamespace(NS) + .withQuery("hive", "select * from ns.tbl") + .create(); + viewLocation = new Path(view.location()); + } + + @AfterEach + public void dropTestView() throws IOException { + viewLocation.getFileSystem(HIVE_METASTORE_EXTENSION.hiveConf()).delete(viewLocation, true); + catalog.dropView(VIEW_IDENTIFIER); + } + + @Test + public void testSuppressUnlockExceptions() { + 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); + + AtomicReference lockRef = new AtomicReference<>(); + + when(spyOps.lockObject()) + .thenAnswer( + i -> { + HiveLock lock = (HiveLock) i.callRealMethod(); + lockRef.set(lock); + return lock; + }); + + try { + spyOps.commit(metadataV2, metadataV1); + HiveLock spyLock = spy(lockRef.get()); + doThrow(new RuntimeException()).when(spyLock).unlock(); + } finally { + lockRef.get().unlock(); + } + + ops.refresh(); + + // the commit must succeed + assertThat(ops.current().properties()).hasSize(0).isEqualTo(metadataV1.properties()); + } + + /** + * Pretends we throw an error while persisting, and not found with check state, commit state + * should be treated as unknown, because in reality the persisting may still succeed, just not yet + * by the time of checking. + */ + @Test + public void testThriftExceptionUnknownStateIfNotInHistoryFailureOnCommit() + 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); + + failCommitAndThrowException(spyOps); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on fire"); + + ops.refresh(); + + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as( + "New metadata files should still exist, new location not in history but" + + " the commit may still succeed") + .isEqualTo(2); + } + + /** Pretends we throw an error while persisting that actually does commit serverside. */ + @Test + public void testThriftExceptionSuccessOnCommit() 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 + commitAndThrowException(ops, spyOps); + spyOps.commit(metadataV2, metadataV1); + + assertThat(ops.current()).as("Current metadata should have not changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as("Commit should have been successful and new metadata file should be made") + .isEqualTo(2); + } + + /** + * Pretends we throw an exception while persisting and don't know what happened, can't check to + * find out, but in reality the commit failed + */ + @Test + public void testThriftExceptionUnknownFailedCommit() 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); + + failCommitAndThrowException(spyOps); + breakFallbackCatalogCommitCheck(spyOps); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on fire"); + + ops.refresh(); + + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as("Client could not determine outcome so new metadata file should also exist") + .isEqualTo(2); + } + + /** + * Pretends we throw an exception while persisting and don't know what happened, can't check to + * find out, but in reality the commit succeeded + */ + @Test + public void testThriftExceptionsUnknownSuccessCommit() 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); + + HiveViewOperations spyOps = spy(ops); + + commitAndThrowException(ops, spyOps); + breakFallbackCatalogCommitCheck(spyOps); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("Datacenter on fire"); + + ops.refresh(); + ViewMetadata metadataV3 = ops.current(); + + assertThat(metadataV3).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV3)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataFileCount(metadataV3)) + .as("Commit should have been successful with updated properties at metadataV2") + .isEqualTo(2); + } + + /** + * Pretends we threw an exception while persisting, the commit succeeded, the lock expired, and a + * second committer placed a commit on top of ours before the first committer was able to check if + * their commit succeeded or not + * + *

    Timeline: + * + *

      + *
    • Client 1 commits which throws an exception but succeeded + *
    • Client 1's lock expires while waiting to do the recheck for commit success + *
    • Client 2 acquires a lock, commits successfully on top of client 1's commit and release + * lock + *
    • Client 1 check's to see if their commit was successful + *
    + * + *

    This tests to make sure a disconnected client 1 doesn't think their commit failed just + * because it isn't the current one during the recheck phase. + */ + @Test + public void testThriftExceptionConcurrentCommit() throws TException, InterruptedException { + HiveViewOperations ops = (HiveViewOperations) ((BaseView) view).operations(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k0", "v0").commit(); + ops.refresh(); + ViewMetadata metadataV2 = ops.current(); + assertThat(metadataV2.properties()).hasSize(1).containsEntry("k0", "v0"); + + HiveViewOperations spyOps = spy(ops); + + AtomicReference lock = new AtomicReference<>(); + doAnswer( + l -> { + lock.set(ops.lockObject()); + return lock.get(); + }) + .when(spyOps) + .lockObject(); + + concurrentCommitAndThrowException(ops, spyOps, (BaseView) view, lock); + + // This commit should fail and concurrent commit should succeed even though this commit + // throws an exception after the persist operation succeeds + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageContaining("Datacenter on fire"); + + ops.refresh(); + ViewMetadata metadataV3 = ops.current(); + + assertThat(metadataV3).as("Current metadata should have changed").isNotEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV3)) + .as("Current metadata file should still exist") + .isTrue(); + assertThat(metadataV3.properties()) + .as("The new properties from the concurrent commit should have been successful") + .hasSize(2); + } + + @Test + public void testInvalidObjectException() { + TableIdentifier badTi = TableIdentifier.of(DB_NAME, "`test_iceberg_view`"); + assertThatThrownBy( + () -> + catalog + .buildView(badTi) + .withSchema(SCHEMA) + .withDefaultNamespace(NS) + .withQuery("hive", "select * from ns.tbl") + .create()) + .isInstanceOf(ValidationException.class) + .hasMessage("Invalid Hive object for " + DB_NAME + "." + "`test_iceberg_view`"); + } + + /** Uses NoLock and pretends we throw an error because of a concurrent commit */ + @Test + public void testNoLockThriftExceptionConcurrentCommit() 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); + + // Sets NoLock + doReturn(new NoLock()).when(spyOps).lockObject(); + + // Simulate a concurrent view modification error + doThrow( + new RuntimeException( + "MetaException(message:The table has been modified. The parameter value for key 'metadata_location' is")) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + + // Should throw a CommitFailedException so the commit could be retried + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .isInstanceOf(CommitFailedException.class) + .hasMessageContaining("The view hivedb.test_iceberg_view has been modified concurrently"); + + ops.refresh(); + + assertThat(ops.current()).as("Current metadata should not have changed").isEqualTo(metadataV2); + assertThat(metadataFileExists(metadataV2)).as("Current metadata should still exist").isTrue(); + assertThat(metadataFileCount(metadataV2)) + .as("New metadata files should not exist") + .isEqualTo(1); + } + + @Test + public void testLockExceptionUnknownSuccessCommit() 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 LockException("Datacenter on fire"); + }) + .when(spyOps) + .persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(metadataV2, metadataV1)) + .hasMessageContaining("Failed to heartbeat for hive lock while") + .isInstanceOf(CommitStateUnknownException.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(); + ViewMetadata metadataV1 = ops.current(); + assertThat(metadataV1.properties()).hasSize(0); + + view.updateProperties().set("k1", "v1").commit(); + + ops.refresh(); + + HiveViewOperations spyOps = spy(ops); + doThrow(new RuntimeException()).when(spyOps).persistTable(any(), anyBoolean(), any()); + + assertThatThrownBy(() -> spyOps.commit(ops.current(), metadataV1)) + .isInstanceOf(CommitStateUnknownException.class) + .hasMessageStartingWith("null\nCannot determine whether the commit was successful or not"); + } + + private void commitAndThrowException( + HiveViewOperations realOperations, HiveViewOperations spyOperations) + throws TException, InterruptedException { + // 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); + realOperations.persistTable(tbl, true, location); + throw new TException("Datacenter on fire"); + }) + .when(spyOperations) + .persistTable(any(), anyBoolean(), any()); + } + + private void concurrentCommitAndThrowException( + HiveViewOperations realOperations, + HiveViewOperations spyOperations, + BaseView baseView, + AtomicReference lock) + throws TException, InterruptedException { + // 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); + realOperations.persistTable(tbl, true, location); + // Simulate lock expiration or removal + lock.get().unlock(); + baseView.operations().refresh(); + baseView.updateProperties().set("k1", "v1").set("k2", "v2").commit(); + throw new TException("Datacenter on fire"); + }) + .when(spyOperations) + .persistTable(any(), anyBoolean(), any()); + } + + private void failCommitAndThrowException(HiveViewOperations spyOperations) + throws TException, InterruptedException { + doThrow(new TException("Datacenter on fire")) + .when(spyOperations) + .persistTable(any(), anyBoolean(), any()); + } + + private void breakFallbackCatalogCommitCheck(HiveViewOperations spyOperations) { + when(spyOperations.refresh()) + .thenThrow(new RuntimeException("Still on fire")); // Failure on commit check + } + + private boolean metadataFileExists(ViewMetadata metadata) { + return new File(metadata.metadataFileLocation().replace("file:", "")).exists(); + } + + private int metadataFileCount(ViewMetadata metadata) { + return new File(metadata.metadataFileLocation().replace("file:", "")) + .getParentFile() + .listFiles(file -> file.getName().endsWith("metadata.json")) + .length; + } +} From de12373d035dbe170cb56f5d6458d35a87472502 Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Fri, 13 Sep 2024 11:27:18 -0700 Subject: [PATCH 0692/1019] Spark 3.4: Action to compute table stats (#11106) --- spark/v3.4/build.gradle | 2 + .../actions/ComputeTableStatsSparkAction.java | 179 ++++++++ .../iceberg/spark/actions/NDVSketchUtil.java | 93 ++++ .../iceberg/spark/actions/SparkActions.java | 6 + .../spark/sql/stats/ThetaSketchAgg.scala | 121 +++++ .../actions/TestComputeTableStatsAction.java | 415 ++++++++++++++++++ 6 files changed, 816 insertions(+) create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java create mode 100644 spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java create mode 100644 spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala create mode 100644 spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index 3b1761d39f63..a978cda9db0b 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -59,6 +59,7 @@ project(":iceberg-spark:iceberg-spark-${sparkMajorVersion}_${scalaVersion}") { implementation project(':iceberg-parquet') implementation project(':iceberg-arrow') implementation("org.scala-lang.modules:scala-collection-compat_${scalaVersion}:${libs.versions.scala.collection.compat.get()}") + implementation("org.apache.datasketches:datasketches-java:${libs.versions.datasketches.get()}") if (scalaVersion == '2.12') { // scala-collection-compat_2.12 pulls scala 2.12.17 and we need 2.12.18 for JDK 21 support implementation 'org.scala-lang:scala-library:2.12.18' @@ -289,6 +290,7 @@ project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersio relocate 'com.carrotsearch', 'org.apache.iceberg.shaded.com.carrotsearch' relocate 'org.threeten.extra', 'org.apache.iceberg.shaded.org.threeten.extra' relocate 'org.roaringbitmap', 'org.apache.iceberg.shaded.org.roaringbitmap' + relocate 'org.apache.datasketches', 'org.apache.iceberg.shaded.org.apache.datasketches' archiveClassifier.set(null) } diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.java new file mode 100644 index 000000000000..a508021c1040 --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/ComputeTableStatsSparkAction.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 java.io.IOException; +import java.util.List; +import java.util.UUID; +import java.util.stream.Collectors; +import org.apache.iceberg.GenericBlobMetadata; +import org.apache.iceberg.GenericStatisticsFile; +import org.apache.iceberg.HasTableOperations; +import org.apache.iceberg.IcebergBuild; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.TableOperations; +import org.apache.iceberg.actions.ComputeTableStats; +import org.apache.iceberg.actions.ImmutableComputeTableStats; +import org.apache.iceberg.exceptions.RuntimeIOException; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.Puffin; +import org.apache.iceberg.puffin.PuffinWriter; +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.spark.JobGroupInfo; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.SparkSession; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Computes the statistics of the given columns and stores it as Puffin files. */ +public class ComputeTableStatsSparkAction extends BaseSparkAction + implements ComputeTableStats { + + private static final Logger LOG = LoggerFactory.getLogger(ComputeTableStatsSparkAction.class); + private static final Result EMPTY_RESULT = ImmutableComputeTableStats.Result.builder().build(); + + private final Table table; + private List columns; + private Snapshot snapshot; + + ComputeTableStatsSparkAction(SparkSession spark, Table table) { + super(spark); + this.table = table; + this.snapshot = table.currentSnapshot(); + } + + @Override + protected ComputeTableStatsSparkAction self() { + return this; + } + + @Override + public ComputeTableStats columns(String... newColumns) { + Preconditions.checkArgument( + newColumns != null && newColumns.length > 0, "Columns cannot be null/empty"); + this.columns = ImmutableList.copyOf(ImmutableSet.copyOf(newColumns)); + return this; + } + + @Override + public ComputeTableStats snapshot(long newSnapshotId) { + Snapshot newSnapshot = table.snapshot(newSnapshotId); + Preconditions.checkArgument(newSnapshot != null, "Snapshot not found: %s", newSnapshotId); + this.snapshot = newSnapshot; + return this; + } + + @Override + public Result execute() { + if (snapshot == null) { + LOG.info("No snapshot to compute stats for table {}", table.name()); + return EMPTY_RESULT; + } + validateColumns(); + JobGroupInfo info = newJobGroupInfo("COMPUTE-TABLE-STATS", jobDesc()); + return withJobGroupInfo(info, this::doExecute); + } + + private Result doExecute() { + LOG.info( + "Computing stats for columns {} in {} (snapshot {})", + columns(), + table.name(), + snapshotId()); + List blobs = generateNDVBlobs(); + StatisticsFile statisticsFile = writeStatsFile(blobs); + table.updateStatistics().setStatistics(snapshotId(), statisticsFile).commit(); + return ImmutableComputeTableStats.Result.builder().statisticsFile(statisticsFile).build(); + } + + private StatisticsFile writeStatsFile(List blobs) { + LOG.info("Writing stats for table {} for snapshot {}", table.name(), snapshotId()); + OutputFile outputFile = table.io().newOutputFile(outputPath()); + try (PuffinWriter writer = Puffin.write(outputFile).createdBy(appIdentifier()).build()) { + blobs.forEach(writer::add); + writer.finish(); + return new GenericStatisticsFile( + snapshotId(), + outputFile.location(), + writer.fileSize(), + writer.footerSize(), + GenericBlobMetadata.from(writer.writtenBlobsMetadata())); + } catch (IOException e) { + throw new RuntimeIOException(e); + } + } + + private List generateNDVBlobs() { + return NDVSketchUtil.generateBlobs(spark(), table, snapshot, columns()); + } + + private List columns() { + if (columns == null) { + Schema schema = table.schemas().get(snapshot.schemaId()); + this.columns = + schema.columns().stream() + .filter(nestedField -> nestedField.type().isPrimitiveType()) + .map(Types.NestedField::name) + .collect(Collectors.toList()); + } + return columns; + } + + private void validateColumns() { + Schema schema = table.schemas().get(snapshot.schemaId()); + Preconditions.checkArgument(!columns().isEmpty(), "No columns found to compute stats"); + for (String columnName : columns()) { + Types.NestedField field = schema.findField(columnName); + Preconditions.checkArgument(field != null, "Can't find column %s in %s", columnName, schema); + Preconditions.checkArgument( + field.type().isPrimitiveType(), + "Can't compute stats on non-primitive type column: %s (%s)", + columnName, + field.type()); + } + } + + private String appIdentifier() { + String icebergVersion = IcebergBuild.fullVersion(); + String sparkVersion = spark().version(); + return String.format("Iceberg %s Spark %s", icebergVersion, sparkVersion); + } + + private long snapshotId() { + return snapshot.snapshotId(); + } + + private String jobDesc() { + return String.format( + "Computing table stats for %s (snapshot_id=%s, columns=%s)", + table.name(), snapshotId(), columns()); + } + + private String outputPath() { + TableOperations operations = ((HasTableOperations) table).operations(); + String fileName = String.format("%s-%s.stats", snapshotId(), UUID.randomUUID()); + return operations.metadataFileLocation(fileName); + } +} diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java new file mode 100644 index 000000000000..22055a161e4e --- /dev/null +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -0,0 +1,93 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.spark.actions; + +import java.nio.ByteBuffer; +import java.util.List; +import org.apache.datasketches.memory.Memory; +import org.apache.datasketches.theta.CompactSketch; +import org.apache.datasketches.theta.Sketch; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Snapshot; +import org.apache.iceberg.Table; +import org.apache.iceberg.puffin.Blob; +import org.apache.iceberg.puffin.PuffinCompressionCodec; +import org.apache.iceberg.puffin.StandardBlobTypes; +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.SparkReadOptions; +import org.apache.iceberg.types.Types; +import org.apache.spark.sql.Column; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.stats.ThetaSketchAgg; + +public class NDVSketchUtil { + + private NDVSketchUtil() {} + + public static final String APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY = "ndv"; + + static List generateBlobs( + SparkSession spark, Table table, Snapshot snapshot, List columns) { + Row sketches = computeNDVSketches(spark, table, snapshot, columns); + Schema schema = table.schemas().get(snapshot.schemaId()); + List blobs = Lists.newArrayList(); + for (int i = 0; i < columns.size(); i++) { + Types.NestedField field = schema.findField(columns.get(i)); + Sketch sketch = CompactSketch.wrap(Memory.wrap((byte[]) sketches.get(i))); + blobs.add(toBlob(field, sketch, snapshot)); + } + return blobs; + } + + private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snapshot) { + return new Blob( + StandardBlobTypes.APACHE_DATASKETCHES_THETA_V1, + ImmutableList.of(field.fieldId()), + snapshot.snapshotId(), + snapshot.sequenceNumber(), + ByteBuffer.wrap(sketch.toByteArray()), + PuffinCompressionCodec.ZSTD, + ImmutableMap.of( + APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY, + String.valueOf((long) sketch.getEstimate()))); + } + + private static Row computeNDVSketches( + SparkSession spark, Table table, Snapshot snapshot, List colNames) { + return spark + .read() + .format("iceberg") + .option(SparkReadOptions.SNAPSHOT_ID, snapshot.snapshotId()) + .load(table.name()) + .select(toAggColumns(colNames)) + .first(); + } + + private static Column[] toAggColumns(List colNames) { + return colNames.stream().map(NDVSketchUtil::toAggColumn).toArray(Column[]::new); + } + + private static Column toAggColumn(String colName) { + ThetaSketchAgg agg = new ThetaSketchAgg(colName); + return new Column(agg.toAggregateExpression()); + } +} 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 fb67ded96e35..f845386d30c4 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 @@ -20,6 +20,7 @@ import org.apache.iceberg.Table; import org.apache.iceberg.actions.ActionsProvider; +import org.apache.iceberg.actions.ComputeTableStats; import org.apache.iceberg.spark.Spark3Util; import org.apache.iceberg.spark.Spark3Util.CatalogAndIdentifier; import org.apache.spark.sql.SparkSession; @@ -96,4 +97,9 @@ public DeleteReachableFilesSparkAction deleteReachableFiles(String metadataLocat public RewritePositionDeleteFilesSparkAction rewritePositionDeletes(Table table) { return new RewritePositionDeleteFilesSparkAction(spark, table); } + + @Override + public ComputeTableStats computeTableStats(Table table) { + return new ComputeTableStatsSparkAction(spark, table); + } } diff --git a/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala new file mode 100644 index 000000000000..cca16960f434 --- /dev/null +++ b/spark/v3.4/spark/src/main/scala/org/apache/spark/sql/stats/ThetaSketchAgg.scala @@ -0,0 +1,121 @@ +/* + * 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.spark.sql.stats + +import java.nio.ByteBuffer +import org.apache.datasketches.common.Family +import org.apache.datasketches.memory.Memory +import org.apache.datasketches.theta.CompactSketch +import org.apache.datasketches.theta.SetOperationBuilder +import org.apache.datasketches.theta.Sketch +import org.apache.datasketches.theta.UpdateSketch +import org.apache.iceberg.spark.SparkSchemaUtil +import org.apache.iceberg.types.Conversions +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.aggregate.ImperativeAggregate +import org.apache.spark.sql.catalyst.expressions.aggregate.TypedImperativeAggregate +import org.apache.spark.sql.catalyst.trees.UnaryLike +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.types.BinaryType +import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.Decimal +import org.apache.spark.unsafe.types.UTF8String + +/** + * ThetaSketchAgg generates Alpha family sketch with default seed. + * The values fed to the sketch are converted to bytes using Iceberg's single value serialization. + * The result returned is an array of bytes of Compact Theta sketch of Datasketches library, + * which should be deserialized to Compact sketch before using. + * + * See [[https://iceberg.apache.org/puffin-spec/]] for more information. + * + */ +case class ThetaSketchAgg( + child: Expression, + mutableAggBufferOffset: Int = 0, + inputAggBufferOffset: Int = 0) extends TypedImperativeAggregate[Sketch] with UnaryLike[Expression] { + + private lazy val icebergType = SparkSchemaUtil.convert(child.dataType) + + def this(colName: String) = { + this(col(colName).expr, 0, 0) + } + + override def dataType: DataType = BinaryType + + override def nullable: Boolean = false + + override def createAggregationBuffer(): Sketch = { + UpdateSketch.builder.setFamily(Family.ALPHA).build() + } + + override def update(buffer: Sketch, input: InternalRow): Sketch = { + val value = child.eval(input) + if (value != null) { + val icebergValue = toIcebergValue(value) + val byteBuffer = Conversions.toByteBuffer(icebergType, icebergValue) + buffer.asInstanceOf[UpdateSketch].update(byteBuffer) + } + buffer + } + + private def toIcebergValue(value: Any): Any = { + value match { + case s: UTF8String => s.toString + case d: Decimal => d.toJavaBigDecimal + case b: Array[Byte] => ByteBuffer.wrap(b) + case _ => value + } + } + + override def merge(buffer: Sketch, input: Sketch): Sketch = { + new SetOperationBuilder().buildUnion.union(buffer, input) + } + + override def eval(buffer: Sketch): Any = { + toBytes(buffer) + } + + override def serialize(buffer: Sketch): Array[Byte] = { + toBytes(buffer) + } + + override def deserialize(storageFormat: Array[Byte]): Sketch = { + CompactSketch.wrap(Memory.wrap(storageFormat)) + } + + override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: Int): ImperativeAggregate = { + copy(mutableAggBufferOffset = newMutableAggBufferOffset) + } + + override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): ImperativeAggregate = { + copy(inputAggBufferOffset = newInputAggBufferOffset) + } + + override protected def withNewChildInternal(newChild: Expression): Expression = { + copy(child = newChild) + } + + private def toBytes(sketch: Sketch): Array[Byte] = { + val compactSketch = sketch.compact() + compactSketch.toByteArray + } +} 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 new file mode 100644 index 000000000000..58703d4a90e5 --- /dev/null +++ b/spark/v3.4/spark/src/test/java/org/apache/iceberg/spark/actions/TestComputeTableStatsAction.java @@ -0,0 +1,415 @@ +/* + * 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.spark.actions.NDVSketchUtil.APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY; +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.assertThatCode; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import org.apache.iceberg.BlobMetadata; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.Files; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.StatisticsFile; +import org.apache.iceberg.Table; +import org.apache.iceberg.actions.ComputeTableStats; +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.ImmutableMap; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.spark.Spark3Util; +import org.apache.iceberg.spark.SparkCatalogTestBase; +import org.apache.iceberg.spark.SparkSchemaUtil; +import org.apache.iceberg.spark.SparkWriteOptions; +import org.apache.iceberg.spark.data.RandomData; +import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.types.Types; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.parser.ParseException; +import org.apache.spark.sql.types.StructType; +import org.junit.After; +import org.junit.Test; + +public class TestComputeTableStatsAction extends SparkCatalogTestBase { + + private static final Types.StructType LEAF_STRUCT_TYPE = + Types.StructType.of( + optional(1, "leafLongCol", Types.LongType.get()), + optional(2, "leafDoubleCol", Types.DoubleType.get())); + + private static final Types.StructType NESTED_STRUCT_TYPE = + Types.StructType.of(required(3, "leafStructCol", LEAF_STRUCT_TYPE)); + + private static final Schema NESTED_SCHEMA = + new Schema(required(4, "nestedStructCol", NESTED_STRUCT_TYPE)); + + private static final Schema SCHEMA_WITH_NESTED_COLUMN = + new Schema( + required(4, "nestedStructCol", NESTED_STRUCT_TYPE), + required(5, "stringCol", Types.StringType.get())); + + public TestComputeTableStatsAction( + String catalogName, String implementation, Map config) { + super(catalogName, implementation, config); + } + + @Test + public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + // To create multiple splits on the mapper + table + .updateProperties() + .set("read.split.target-size", "100") + .set("write.parquet.row-group-size-bytes", "100") + .commit(); + List records = + Lists.newArrayList( + new SimpleRecord(1, "a"), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark.createDataset(records, Encoders.bean(SimpleRecord.class)).writeTo(tableName).append(); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns("id", "data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(2); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isEqualTo(String.valueOf(4)); + } + + @Test + public void testComputeTableStatsActionWithoutExplicitColumns() + throws NoSuchTableException, ParseException { + 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, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + assertThat(results).isNotNull(); + + assertThat(table.statisticsFiles().size()).isEqualTo(1); + StatisticsFile statisticsFile = table.statisticsFiles().get(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(2); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat( + Long.parseLong( + statisticsFile + .blobMetadata() + .get(0) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))) + .isEqualTo(4); + assertThat( + Long.parseLong( + statisticsFile + .blobMetadata() + .get(1) + .properties() + .get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY))) + .isEqualTo(4); + } + + @Test + public void testComputeTableStatsForInvalidColumns() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).columns("id1").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column id1 in table"); + } + + @Test + public void testComputeTableStatsWithNoSnapshots() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result result = actions.computeTableStats(table).columns("id").execute(); + assertThat(result.statisticsFile()).isNull(); + } + + @Test + public void testComputeTableStatsWithNullValues() throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + List records = + Lists.newArrayList( + new SimpleRecord(1, null), + new SimpleRecord(1, "a"), + new SimpleRecord(2, "b"), + new SimpleRecord(3, "c"), + new SimpleRecord(4, "d")); + spark + .createDataset(records, Encoders.bean(SimpleRecord.class)) + .coalesce(1) + .writeTo(tableName) + .append(); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isEqualTo(String.valueOf(4)); + } + + @Test + public void testComputeTableStatsWithSnapshotHavingDifferentSchemas() + throws NoSuchTableException, ParseException { + SparkActions actions = SparkActions.get(); + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + long snapshotId1 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + // Snapshot id not specified + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + assertThatCode(() -> actions.computeTableStats(table).columns("data").execute()) + .doesNotThrowAnyException(); + + sql("ALTER TABLE %s DROP COLUMN %s", tableName, "data"); + // Append data to create snapshot + sql("INSERT into %s values(1)", tableName); + table.refresh(); + long snapshotId2 = Spark3Util.loadIcebergTable(spark, tableName).currentSnapshot().snapshotId(); + + // Snapshot id specified + assertThatCode( + () -> actions.computeTableStats(table).snapshot(snapshotId1).columns("data").execute()) + .doesNotThrowAnyException(); + + assertThatThrownBy( + () -> actions.computeTableStats(table).snapshot(snapshotId2).columns("data").execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Can't find column data in table"); + } + + @Test + public void testComputeTableStatsWhenSnapshotIdNotSpecified() + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + // Append data to create snapshot + sql("INSERT into %s values(1, 'abcd')", tableName); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).columns("data").execute(); + + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isEqualTo(String.valueOf(1)); + } + + @Test + public void testComputeTableStatsWithNestedSchema() + throws NoSuchTableException, ParseException, IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, + SCHEMA_WITH_NESTED_COLUMN, + PartitionSpec.unpartitioned(), + ImmutableMap.of()); + DataFile dataFile = + FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + Table tbl = Spark3Util.loadIcebergTable(spark, tableName); + SparkActions actions = SparkActions.get(); + actions.computeTableStats(tbl).execute(); + + tbl.refresh(); + List statisticsFiles = tbl.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + } + + @Test + public void testComputeTableStatsWithNoComputableColumns() throws IOException { + List records = Lists.newArrayList(createNestedRecord()); + Table table = + validationCatalog.createTable( + tableIdent, NESTED_SCHEMA, PartitionSpec.unpartitioned(), ImmutableMap.of()); + DataFile dataFile = + FileHelpers.writeDataFile(table, Files.localOutput(temp.newFile()), records); + table.newAppend().appendFile(dataFile).commit(); + + table.refresh(); + SparkActions actions = SparkActions.get(); + assertThatThrownBy(() -> actions.computeTableStats(table).execute()) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("No columns found to compute stats"); + } + + @Test + public void testComputeTableStatsOnByteColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("byte_col", "TINYINT"); + } + + @Test + public void testComputeTableStatsOnShortColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("short_col", "SMALLINT"); + } + + @Test + public void testComputeTableStatsOnIntColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("int_col", "INT"); + } + + @Test + public void testComputeTableStatsOnLongColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("long_col", "BIGINT"); + } + + @Test + public void testComputeTableStatsOnTimestampColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP"); + } + + @Test + public void testComputeTableStatsOnTimestampNtzColumn() + throws NoSuchTableException, ParseException { + testComputeTableStats("timestamp_col", "TIMESTAMP_NTZ"); + } + + @Test + public void testComputeTableStatsOnDateColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("date_col", "DATE"); + } + + @Test + public void testComputeTableStatsOnDecimalColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("decimal_col", "DECIMAL(20, 2)"); + } + + @Test + public void testComputeTableStatsOnBinaryColumn() throws NoSuchTableException, ParseException { + testComputeTableStats("binary_col", "BINARY"); + } + + public void testComputeTableStats(String columnName, String type) + throws NoSuchTableException, ParseException { + sql("CREATE TABLE %s (id int, %s %s) USING iceberg", tableName, columnName, type); + Table table = Spark3Util.loadIcebergTable(spark, tableName); + + Dataset dataDF = randomDataDF(table.schema()); + append(tableName, dataDF); + + SparkActions actions = SparkActions.get(); + table.refresh(); + ComputeTableStats.Result results = + actions.computeTableStats(table).columns(columnName).execute(); + assertThat(results).isNotNull(); + + List statisticsFiles = table.statisticsFiles(); + assertThat(statisticsFiles.size()).isEqualTo(1); + + StatisticsFile statisticsFile = statisticsFiles.get(0); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(1); + + BlobMetadata blobMetadata = statisticsFile.blobMetadata().get(0); + assertThat(blobMetadata.properties().get(APACHE_DATASKETCHES_THETA_V1_NDV_PROPERTY)) + .isNotNull(); + } + + private GenericRecord createNestedRecord() { + GenericRecord record = GenericRecord.create(SCHEMA_WITH_NESTED_COLUMN); + GenericRecord nested = GenericRecord.create(NESTED_STRUCT_TYPE); + GenericRecord leaf = GenericRecord.create(LEAF_STRUCT_TYPE); + leaf.set(0, 0L); + leaf.set(1, 0.0); + nested.set(0, leaf); + record.set(0, nested); + record.set(1, "data"); + return record; + } + + private Dataset randomDataDF(Schema schema) { + Iterable rows = RandomData.generateSpark(schema, 10, 0); + JavaRDD rowRDD = sparkContext.parallelize(Lists.newArrayList(rows)); + StructType rowSparkType = SparkSchemaUtil.convert(schema); + return spark.internalCreateDataFrame(JavaRDD.toRDD(rowRDD), rowSparkType, false); + } + + private void append(String table, Dataset df) throws NoSuchTableException { + // fanout writes are enabled as write-time clustering is not supported without Spark extensions + df.coalesce(1).writeTo(table).option(SparkWriteOptions.FANOUT_ENABLED, "true").append(); + } + + @After + public void removeTable() { + sql("DROP TABLE IF EXISTS %s", tableName); + } +} From 2c19cfd59952ff1e4d0872fae983ebfaa821c7b7 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Sat, 14 Sep 2024 12:56:21 +0800 Subject: [PATCH 0693/1019] Docs: Fix missing options for remove_orphan_files procedure (#11080) --- docs/docs/spark-procedures.md | 38 +++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/docs/docs/spark-procedures.md b/docs/docs/spark-procedures.md index 1cd14dd1888e..0953e729a77b 100644 --- a/docs/docs/spark-procedures.md +++ b/docs/docs/spark-procedures.md @@ -312,6 +312,10 @@ Used to remove files which are not referenced in any metadata files of an Iceber | `location` | | string | Directory to look for files in (defaults to the table's location) | | `dry_run` | | boolean | When true, don't actually remove files (defaults to false) | | `max_concurrent_deletes` | | int | Size of the thread pool used for delete file actions (by default, no thread pool is used) | +| `file_list_view` | | string | Dataset to look for files in (skipping the directory listing) | +| `equal_schemes` | | map | Mapping of file system schemes to be considered equal. Key is a comma-separated list of schemes and value is a scheme (defaults to `map('s3a,s3n','s3')`). | +| `equal_authorities` | | map | Mapping of file system authorities to be considered equal. Key is a comma-separated list of authorities and value is an authority. | +| `prefix_mismatch_mode` | | string | Action behavior when location prefixes (schemes/authorities) mismatch:

    • ERROR - throw an exception. (default)
    • IGNORE - no action.
    • DELETE - delete files.
    | #### Output @@ -331,6 +335,40 @@ Remove any files in the `tablelocation/data` folder which are not known to the t CALL catalog_name.system.remove_orphan_files(table => 'db.sample', location => 'tablelocation/data'); ``` +Remove any files in the `files_view` view which are not known to the table `db.sample`. +```java +Dataset compareToFileList = + spark + .createDataFrame(allFiles, FilePathLastModifiedRecord.class) + .withColumnRenamed("filePath", "file_path") + .withColumnRenamed("lastModified", "last_modified"); +String fileListViewName = "files_view"; +compareToFileList.createOrReplaceTempView(fileListViewName); +``` +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', file_list_view => 'files_view'); +``` + +When a file matches references in metadata files except for location prefix (scheme/authority), an error is thrown by default. +The error can be ignored and the file will be skipped by setting `prefix_mismatch_mode` to `IGNORE`. +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', prefix_mismatch_mode => 'IGNORE'); +``` + +The file can still be deleted by setting `prefix_mismatch_mode` to `DELETE`. +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', prefix_mismatch_mode => 'DELETE'); +``` + +The file can also be deleted by considering the mismatched prefixes equal. +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', equal_schemes => map('file', 'file1')); +``` + +```sql +CALL catalog_name.system.remove_orphan_files(table => 'db.sample', equal_authorities => map('ns1', 'ns2')); +``` + ### `rewrite_data_files` Iceberg tracks each data file in a table. More data files leads to more metadata stored in manifest files, and small data files causes an unnecessary amount of metadata and less efficient queries from file open costs. From d8a4e1dc191d712720ec6a775c98945e0f3726db Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Mon, 16 Sep 2024 09:44:10 -0700 Subject: [PATCH 0694/1019] Flink: Increase the number of checkpoints from 4 to 6 to fix flakiness. (#11121) 6 checkpoionts cycles seem to be more stable based on the existing TestFlinkIcebergSinkDistributionMode test. --- .../sink/TestFlinkIcebergSinkRangeDistributionBucketing.java | 2 +- .../sink/TestFlinkIcebergSinkRangeDistributionBucketing.java | 2 +- 2 files changed, 2 insertions(+), 2 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 a5f24e09a60b..f4be472ec629 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 @@ -98,7 +98,7 @@ public class TestFlinkIcebergSinkRangeDistributionBucketing { new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); private static final int NUM_BUCKETS = 4; - private static final int NUM_OF_CHECKPOINTS = 4; + private static final int NUM_OF_CHECKPOINTS = 6; private static final int ROW_COUNT_PER_CHECKPOINT = 200; private static final Schema SCHEMA = new Schema( 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 a5f24e09a60b..f4be472ec629 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 @@ -98,7 +98,7 @@ public class TestFlinkIcebergSinkRangeDistributionBucketing { new HadoopCatalogExtension(TestFixtures.DATABASE, TestFixtures.TABLE); private static final int NUM_BUCKETS = 4; - private static final int NUM_OF_CHECKPOINTS = 4; + private static final int NUM_OF_CHECKPOINTS = 6; private static final int ROW_COUNT_PER_CHECKPOINT = 200; private static final Schema SCHEMA = new Schema( From a717bec7060dc2ac9a4aad21ac3d7856bd2c328a Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Mon, 16 Sep 2024 14:46:24 -0700 Subject: [PATCH 0695/1019] Spark 3.4: Add utility to load table state reliably (#11115) --- .../apache/iceberg/spark/SparkTableUtil.java | 19 +++++++++++++++---- .../iceberg/spark/actions/NDVSketchUtil.java | 12 ++++-------- .../actions/TestComputeTableStatsAction.java | 14 ++++++++++++++ 3 files changed, 33 insertions(+), 12 deletions(-) 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 7a96e97fb98a..e103104171f6 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 @@ -851,6 +851,12 @@ private static void deleteManifests(FileIO io, List manifests) { .run(item -> io.deleteFile(item.path())); } + public static Dataset loadTable(SparkSession spark, Table table, long snapshotId) { + SparkTable sparkTable = new SparkTable(table, snapshotId, false); + DataSourceV2Relation relation = createRelation(sparkTable, ImmutableMap.of()); + return Dataset.ofRows(spark, relation); + } + public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type) { return loadMetadataTable(spark, table, type, ImmutableMap.of()); @@ -858,11 +864,16 @@ public static Dataset loadMetadataTable( public static Dataset loadMetadataTable( SparkSession spark, Table table, MetadataTableType type, Map extraOptions) { - SparkTable metadataTable = - new SparkTable(MetadataTableUtils.createMetadataTableInstance(table, type), false); + Table metadataTable = MetadataTableUtils.createMetadataTableInstance(table, type); + SparkTable sparkMetadataTable = new SparkTable(metadataTable, false); + DataSourceV2Relation relation = createRelation(sparkMetadataTable, extraOptions); + return Dataset.ofRows(spark, relation); + } + + private static DataSourceV2Relation createRelation( + SparkTable sparkTable, Map extraOptions) { CaseInsensitiveStringMap options = new CaseInsensitiveStringMap(extraOptions); - return Dataset.ofRows( - spark, DataSourceV2Relation.create(metadataTable, Some.empty(), Some.empty(), options)); + return DataSourceV2Relation.create(sparkTable, Option.empty(), Option.empty(), options); } /** diff --git a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java index 22055a161e4e..c8a20d3cca73 100644 --- a/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java +++ b/spark/v3.4/spark/src/main/java/org/apache/iceberg/spark/actions/NDVSketchUtil.java @@ -32,9 +32,10 @@ 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.SparkReadOptions; +import org.apache.iceberg.spark.SparkTableUtil; 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.stats.ThetaSketchAgg; @@ -73,13 +74,8 @@ private static Blob toBlob(Types.NestedField field, Sketch sketch, Snapshot snap private static Row computeNDVSketches( SparkSession spark, Table table, Snapshot snapshot, List colNames) { - return spark - .read() - .format("iceberg") - .option(SparkReadOptions.SNAPSHOT_ID, snapshot.snapshotId()) - .load(table.name()) - .select(toAggColumns(colNames)) - .first(); + Dataset inputDF = SparkTableUtil.loadTable(spark, table, snapshot.snapshotId()); + return inputDF.select(toAggColumns(colNames)).first(); } private static Column[] toAggColumns(List colNames) { 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 58703d4a90e5..88805a070cb1 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 @@ -82,6 +82,20 @@ public TestComputeTableStatsAction( super(catalogName, implementation, config); } + @Test + public void testLoadingTableDirectly() { + sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); + sql("INSERT into %s values(1, 'abcd')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + + SparkActions actions = SparkActions.get(); + ComputeTableStats.Result results = actions.computeTableStats(table).execute(); + StatisticsFile statisticsFile = results.statisticsFile(); + assertThat(statisticsFile.fileSizeInBytes()).isNotEqualTo(0); + assertThat(statisticsFile.blobMetadata().size()).isEqualTo(2); + } + @Test public void testComputeTableStatsAction() throws NoSuchTableException, ParseException { sql("CREATE TABLE %s (id int, data string) USING iceberg", tableName); From 134628db5fbacbb3373377f95cbaea6da26efa4a Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Tue, 17 Sep 2024 09:13:30 -0700 Subject: [PATCH 0696/1019] Build: switch to slf4j-simple 2.x for test implementation dependency because avro 1.12.0 brings in slf4j-api dependency to 2.x (#11001) --- 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 c7e949fddcc2..ad5f0f406130 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -79,7 +79,7 @@ pig = "0.17.0" roaringbitmap = "1.2.1" s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" -slf4j = "1.7.36" +slf4j = "2.0.16" snowflake-jdbc = "3.18.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" From 6c92aff81366b9ddb65ed93631530400c9dd8138 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 18 Sep 2024 07:42:23 +0200 Subject: [PATCH 0697/1019] Core: Update metadata location without updating lastUpdatedMillis (#11151) Using `TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build()` would do the following things: - update `lastUpdatedMillis` - write a new `metadata-log` entry However, in `LoadTableResponse` / `RestSessionCatalog` the goal is to only update the metadata location without updating the two mentioned things above. --- .../org/apache/iceberg/TableMetadata.java | 9 +++++ .../org/apache/iceberg/TestTableMetadata.java | 34 +++++++++++++++++++ 2 files changed, 43 insertions(+) diff --git a/core/src/main/java/org/apache/iceberg/TableMetadata.java b/core/src/main/java/org/apache/iceberg/TableMetadata.java index c0bb16bc109d..d20dd59d2b97 100644 --- a/core/src/main/java/org/apache/iceberg/TableMetadata.java +++ b/core/src/main/java/org/apache/iceberg/TableMetadata.java @@ -966,6 +966,15 @@ private Builder(TableMetadata base) { public Builder withMetadataLocation(String newMetadataLocation) { this.metadataLocation = newMetadataLocation; + if (null != base) { + // carry over lastUpdatedMillis from base and set previousFileLocation to null to avoid + // writing a new metadata log entry + // this is safe since setting metadata location doesn't cause any changes and no other + // changes can be added when metadata location is configured + this.lastUpdatedMillis = base.lastUpdatedMillis(); + this.previousFileLocation = null; + } + return this; } diff --git a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java index bcde7a7f31d3..5ada35765773 100644 --- a/core/src/test/java/org/apache/iceberg/TestTableMetadata.java +++ b/core/src/test/java/org/apache/iceberg/TestTableMetadata.java @@ -1685,4 +1685,38 @@ public void testV3TimestampNanoTypeSupport() { ImmutableMap.of(), 3); } + + @Test + public void onlyMetadataLocationIsUpdatedWithoutTimestampAndMetadataLogEntry() { + String uuid = "386b9f01-002b-4d8c-b77f-42c3fd3b7c9b"; + TableMetadata metadata = + TableMetadata.buildFromEmpty() + .assignUUID(uuid) + .setLocation("location") + .setCurrentSchema(TEST_SCHEMA, 3) + .addPartitionSpec(PartitionSpec.unpartitioned()) + .addSortOrder(SortOrder.unsorted()) + .discardChanges() + .withMetadataLocation("original-metadata-location") + .build(); + + assertThat(metadata.previousFiles()).isEmpty(); + assertThat(metadata.metadataFileLocation()).isEqualTo("original-metadata-location"); + + // this will only update the metadata location without writing a new metadata log entry or + // updating lastUpdatedMillis + TableMetadata newMetadata = + TableMetadata.buildFrom(metadata).withMetadataLocation("new-metadata-location").build(); + assertThat(newMetadata.lastUpdatedMillis()).isEqualTo(metadata.lastUpdatedMillis()); + assertThat(newMetadata.metadataFileLocation()).isEqualTo("new-metadata-location"); + assertThat(newMetadata.previousFiles()).isEmpty(); + + TableMetadata updatedMetadata = + TableMetadata.buildFrom(newMetadata) + .withMetadataLocation("updated-metadata-location") + .build(); + assertThat(updatedMetadata.lastUpdatedMillis()).isEqualTo(newMetadata.lastUpdatedMillis()); + assertThat(updatedMetadata.metadataFileLocation()).isEqualTo("updated-metadata-location"); + assertThat(updatedMetadata.previousFiles()).isEmpty(); + } } From 6b70663bd598d4cef61da391ab71209832fc3d3f Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Tue, 17 Sep 2024 23:52:16 -0700 Subject: [PATCH 0698/1019] Kafka Connect: separate CI workflow (#11075) --- .github/workflows/delta-conversion-ci.yml | 6 +- .github/workflows/flink-ci.yml | 4 +- .github/workflows/hive-ci.yml | 6 +- .github/workflows/java-ci.yml | 3 +- .github/workflows/kafka-connect-ci.yml | 105 ++++++++++++++++++ .github/workflows/publish-snapshot.yml | 2 +- .github/workflows/spark-ci.yml | 4 +- dev/stage-binaries.sh | 3 +- gradle.properties | 2 + .../iceberg/connect/IntegrationTestBase.java | 18 ++- .../apache/iceberg/connect/TestContext.java | 12 +- settings.gradle | 34 ++++-- 12 files changed, 172 insertions(+), 27 deletions(-) create mode 100644 .github/workflows/kafka-connect-ci.yml diff --git a/.github/workflows/delta-conversion-ci.yml b/.github/workflows/delta-conversion-ci.yml index cd16847cf95a..9326d9d533fd 100644 --- a/.github/workflows/delta-conversion-ci.yml +++ b/.github/workflows/delta-conversion-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/hive-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -51,6 +52,7 @@ on: - 'hive3-orc-bundle/**' - 'hive-runtime/**' - 'flink/**' + - 'kafka-connect/**' - 'pig/**' - 'docs/**' - 'site/**' @@ -88,7 +90,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.12 -DhiveVersions= -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: @@ -117,7 +119,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DhiveVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions=3.5 -DscalaVersion=2.13 -DhiveVersions= -DkafkaVersions= -DflinkVersions= :iceberg-delta-lake:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/flink-ci.yml b/.github/workflows/flink-ci.yml index 370375783cc2..8ed555847861 100644 --- a/.github/workflows/flink-ci.yml +++ b/.github/workflows/flink-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/hive-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -50,6 +51,7 @@ on: - 'hive3/**' - 'hive3-orc-bundle/**' - 'hive-runtime/**' + - 'kafka-connect/**' - 'spark/**' - 'pig/**' - 'docs/**' @@ -91,7 +93,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions= -DhiveVersions= -DflinkVersions=${{ matrix.flink }} :iceberg-flink:iceberg-flink-${{ matrix.flink }}:check :iceberg-flink:iceberg-flink-runtime-${{ matrix.flink }}:check -Pquick=true -x javadoc + - run: ./gradlew -DsparkVersions= -DhiveVersions= -DkafkaVersions= -DflinkVersions=${{ matrix.flink }} :iceberg-flink:iceberg-flink-${{ matrix.flink }}:check :iceberg-flink:iceberg-flink-runtime-${{ matrix.flink }}:check -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/hive-ci.yml b/.github/workflows/hive-ci.yml index 6ad9f58410d6..bcaf62cc07f8 100644 --- a/.github/workflows/hive-ci.yml +++ b/.github/workflows/hive-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/flink-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -49,6 +50,7 @@ on: - 'arrow/**' - 'spark/**' - 'flink/**' + - 'kafka-connect/**' - 'pig/**' - 'docs/**' - 'site/**' @@ -86,7 +88,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions= -DhiveVersions=2 -DflinkVersions= -Pquick=true :iceberg-mr:check :iceberg-hive-runtime:check -x javadoc + - run: ./gradlew -DsparkVersions= -DhiveVersions=2 -DflinkVersions= -DkafkaVersions= -Pquick=true :iceberg-mr:check :iceberg-hive-runtime:check -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: @@ -115,7 +117,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew -DsparkVersions= -DhiveVersions=3 -DflinkVersions= -Pquick=true :iceberg-hive3-orc-bundle:check :iceberg-hive3:check :iceberg-hive-runtime:check -x javadoc + - run: ./gradlew -DsparkVersions= -DhiveVersions=3 -DflinkVersions= -DkafkaVersions= -Pquick=true :iceberg-hive3-orc-bundle:check :iceberg-hive3:check :iceberg-hive-runtime:check -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/java-ci.yml b/.github/workflows/java-ci.yml index 0d39ee8646ad..1da7a673a865 100644 --- a/.github/workflows/java-ci.yml +++ b/.github/workflows/java-ci.yml @@ -35,6 +35,7 @@ on: - '.github/workflows/flink-ci.yml' - '.github/workflows/hive-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -82,7 +83,7 @@ jobs: key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} restore-keys: ${{ runner.os }}-gradle- - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - - run: ./gradlew check -DsparkVersions= -DhiveVersions= -DflinkVersions= -Pquick=true -x javadoc + - run: ./gradlew check -DsparkVersions= -DhiveVersions= -DflinkVersions= -DkafkaVersions= -Pquick=true -x javadoc - uses: actions/upload-artifact@v4 if: failure() with: diff --git a/.github/workflows/kafka-connect-ci.yml b/.github/workflows/kafka-connect-ci.yml new file mode 100644 index 000000000000..98ec18a77953 --- /dev/null +++ b/.github/workflows/kafka-connect-ci.yml @@ -0,0 +1,105 @@ +# +# 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: "Kafka Connect CI" +on: + push: + branches: + - 'main' + - '0.*' + - '1.*' + - '2.*' + tags: + - 'apache-iceberg-**' + pull_request: + paths-ignore: + - '.github/ISSUE_TEMPLATE/**' + - '.github/workflows/api-binary-compatibility.yml' + - '.github/workflows/delta-conversion-ci.yml' + - '.github/workflows/flink-ci.yml' + - '.github/workflows/hive-ci.yml' + - '.github/workflows/java-ci.yml' + - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/labeler.yml' + - '.github/workflows/licence-check.yml' + - '.github/workflows/open-api.yml' + - '.github/workflows/publish-snapshot.yml' + - '.github/workflows/recurring-jmh-benchmarks.yml' + - '.github/workflows/site-ci.yml' + - '.github/workflows/spark-ci.yml' + - '.github/workflows/stale.yml' + - '.gitignore' + - '.asf.yml' + - 'dev/**' + - 'mr/**' + - 'flink/**' + - 'hive3/**' + - 'hive3-orc-bundle/**' + - 'hive-runtime/**' + - 'spark/**' + - 'pig/**' + - 'docs/**' + - 'site/**' + - 'open-api/**' + - 'format/**' + - '.gitattributes' + - 'README.md' + - 'CONTRIBUTING.md' + - 'LICENSE' + - 'NOTICE' + +concurrency: + group: ${{ github.workflow }}-${{ github.ref }} + cancel-in-progress: ${{ github.event_name == 'pull_request' }} + +jobs: + + kafka-connect-tests: + runs-on: ubuntu-22.04 + strategy: + matrix: + jvm: [11, 17, 21] + env: + SPARK_LOCAL_IP: localhost + steps: + - uses: actions/checkout@v4 + - uses: actions/setup-java@v4 + with: + distribution: zulu + java-version: ${{ matrix.jvm }} + - uses: actions/cache@v4 + with: + path: | + ~/.gradle/caches + ~/.gradle/wrapper + key: ${{ runner.os }}-gradle-${{ hashFiles('**/*.gradle*', '**/gradle-wrapper.properties') }} + restore-keys: ${{ runner.os }}-gradle- + - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts + - run: | + ./gradlew -DsparkVersions= -DhiveVersions= -DflinkVersions= -DkafkaVersions=3 \ + :iceberg-kafka-connect:iceberg-kafka-connect-events:check \ + :iceberg-kafka-connect:iceberg-kafka-connect:check \ + :iceberg-kafka-connect:iceberg-kafka-connect-runtime:check \ + -Pquick=true -x javadoc + - uses: actions/upload-artifact@v4 + if: failure() + with: + name: test logs + path: | + **/build/testlogs diff --git a/.github/workflows/publish-snapshot.yml b/.github/workflows/publish-snapshot.yml index 6acee54bde3e..7ff6b56da576 100644 --- a/.github/workflows/publish-snapshot.yml +++ b/.github/workflows/publish-snapshot.yml @@ -41,4 +41,4 @@ jobs: - run: | ./gradlew printVersion ./gradlew -DallModules publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} - ./gradlew -DflinkVersions= -DsparkVersions=3.3,3.4,3.5 -DscalaVersion=2.13 -DhiveVersions= publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} + ./gradlew -DflinkVersions= -DsparkVersions=3.3,3.4,3.5 -DscalaVersion=2.13 -DkafkaVersions=3 -DhiveVersions= publishApachePublicationToMavenRepository -PmavenUser=${{ secrets.NEXUS_USER }} -PmavenPassword=${{ secrets.NEXUS_PW }} diff --git a/.github/workflows/spark-ci.yml b/.github/workflows/spark-ci.yml index 1cc0425b73a3..b5d91d3cc76c 100644 --- a/.github/workflows/spark-ci.yml +++ b/.github/workflows/spark-ci.yml @@ -36,6 +36,7 @@ on: - '.github/workflows/hive-ci.yml' - '.github/workflows/java-ci.yml' - '.github/workflows/jmh-benchmarks-ci.yml' + - '.github/workflows/kafka-connect-ci.yml' - '.github/workflows/labeler.yml' - '.github/workflows/licence-check.yml' - '.github/workflows/open-api.yml' @@ -52,6 +53,7 @@ on: - 'hive3-orc-bundle/**' - 'hive-runtime/**' - 'flink/**' + - 'kafka-connect/**' - 'pig/**' - 'docs/**' - 'open-api/**' @@ -101,7 +103,7 @@ jobs: tool-cache: false - run: echo -e "$(ip addr show eth0 | grep "inet\b" | awk '{print $2}' | cut -d/ -f1)\t$(hostname -f) $(hostname -s)" | sudo tee -a /etc/hosts - run: | - ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=${{ matrix.scala }} -DhiveVersions= -DflinkVersions= \ + ./gradlew -DsparkVersions=${{ matrix.spark }} -DscalaVersion=${{ matrix.scala }} -DhiveVersions= -DflinkVersions= -DkafkaVersions= \ :iceberg-spark:iceberg-spark-${{ matrix.spark }}_${{ matrix.scala }}:check \ :iceberg-spark:iceberg-spark-extensions-${{ matrix.spark }}_${{ matrix.scala }}:check \ :iceberg-spark:iceberg-spark-runtime-${{ matrix.spark }}_${{ matrix.scala }}:check \ diff --git a/dev/stage-binaries.sh b/dev/stage-binaries.sh index 29cf31e5f423..fa09b76c38e9 100755 --- a/dev/stage-binaries.sh +++ b/dev/stage-binaries.sh @@ -22,8 +22,9 @@ SCALA_VERSION=2.12 FLINK_VERSIONS=1.18,1.19,1.20 SPARK_VERSIONS=3.3,3.4,3.5 HIVE_VERSIONS=2,3 +KAFKA_VERSIONS=3 -./gradlew -Prelease -DscalaVersion=$SCALA_VERSION -DflinkVersions=$FLINK_VERSIONS -DsparkVersions=$SPARK_VERSIONS -DhiveVersions=$HIVE_VERSIONS publishApachePublicationToMavenRepository +./gradlew -Prelease -DscalaVersion=$SCALA_VERSION -DflinkVersions=$FLINK_VERSIONS -DsparkVersions=$SPARK_VERSIONS -DhiveVersions=$HIVE_VERSIONS -DkafkaVersions=$KAFKA_VERSIONS publishApachePublicationToMavenRepository # Also publish Scala 2.13 Artifacts for versions that support it. # Flink does not yet support 2.13 (and is largely dropping a user-facing dependency on Scala). Hive doesn't need a Scala specification. diff --git a/gradle.properties b/gradle.properties index fcbe7d8de012..dc1e1a509b01 100644 --- a/gradle.properties +++ b/gradle.properties @@ -22,6 +22,8 @@ systemProp.defaultHiveVersions=2 systemProp.knownHiveVersions=2,3 systemProp.defaultSparkVersions=3.5 systemProp.knownSparkVersions=3.3,3.4,3.5 +systemProp.defaultKafkaVersions=3 +systemProp.knownKafkaVersions=3 systemProp.defaultScalaVersion=2.12 systemProp.knownScalaVersions=2.12,2.13 # enable the Gradle build cache - speeds up builds! 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 f90d4da0379e..247211edb01f 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 @@ -40,11 +40,13 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.assertj.core.api.Condition; import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.BeforeAll; import org.junit.jupiter.api.BeforeEach; public class IntegrationTestBase { - private final TestContext context = TestContext.INSTANCE; + private static TestContext context; + private Catalog catalog; private Admin admin; private String connectorName; @@ -70,12 +72,16 @@ protected String testTopic() { return testTopic; } + @BeforeAll + public static void baseBeforeAll() { + context = TestContext.instance(); + } + @BeforeEach public void baseBefore() { - catalog = context.initLocalCatalog(); - producer = context.initLocalProducer(); - admin = context.initLocalAdmin(); - + this.catalog = context.initLocalCatalog(); + this.producer = context.initLocalProducer(); + this.admin = context.initLocalAdmin(); this.connectorName = "test_connector-" + UUID.randomUUID(); this.testTopic = "test-topic-" + UUID.randomUUID(); } @@ -98,7 +104,7 @@ protected void assertSnapshotProps(TableIdentifier tableIdentifier, String branc Map props = latestSnapshot(table, branch).summary(); assertThat(props) .hasKeySatisfying( - new Condition() { + new Condition<>() { @Override public boolean matches(String str) { return str.startsWith("kafka.connect.offsets."); diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java index 729d4bb264e5..2a1ded6cd8a1 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/TestContext.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import java.io.File; +import java.time.Duration; import java.util.Map; import java.util.UUID; import org.apache.iceberg.CatalogProperties; @@ -37,7 +38,8 @@ public class TestContext { - public static final TestContext INSTANCE = new TestContext(); + private static volatile TestContext instance; + public static final ObjectMapper MAPPER = new ObjectMapper(); public static final int CONNECT_PORT = 8083; @@ -48,9 +50,17 @@ public class TestContext { private static final String AWS_SECRET_KEY = "minioadmin"; private static final String AWS_REGION = "us-east-1"; + public static synchronized TestContext instance() { + if (instance == null) { + instance = new TestContext(); + } + return instance; + } + private TestContext() { ComposeContainer container = new ComposeContainer(new File("./docker/docker-compose.yml")) + .withStartupTimeout(Duration.ofMinutes(2)) .waitingFor("connect", Wait.forHttp("/connectors")); container.start(); } diff --git a/settings.gradle b/settings.gradle index 1e6d92bf1e1f..56a68c384c5f 100644 --- a/settings.gradle +++ b/settings.gradle @@ -41,7 +41,6 @@ include 'gcp-bundle' include 'dell' include 'snowflake' include 'delta-lake' -include 'kafka-connect' include 'open-api' project(':bom').name = 'iceberg-bom' @@ -67,13 +66,13 @@ project(':gcp-bundle').name = 'iceberg-gcp-bundle' project(':dell').name = 'iceberg-dell' project(':snowflake').name = 'iceberg-snowflake' project(':delta-lake').name = 'iceberg-delta-lake' -project(':kafka-connect').name = 'iceberg-kafka-connect' project(':open-api').name = 'iceberg-open-api' if (null != System.getProperty("allModules")) { System.setProperty("flinkVersions", System.getProperty("knownFlinkVersions")) System.setProperty("sparkVersions", System.getProperty("knownSparkVersions")) System.setProperty("hiveVersions", System.getProperty("knownHiveVersions")) + System.setProperty("kafkaVersions", System.getProperty("knownKafkaVersions")) } List knownFlinkVersions = System.getProperty("knownFlinkVersions").split(",") @@ -100,6 +99,14 @@ if (!knownSparkVersions.containsAll(sparkVersions)) { throw new GradleException("Found unsupported Spark versions: " + (sparkVersions - knownSparkVersions)) } +List knownKafkaVersions = System.getProperty("knownKafkaVersions").split(",") +String kafkaVersionsString = System.getProperty("kafkaVersions") != null ? System.getProperty("kafkaVersions") : System.getProperty("defaultKafkaVersions") +List kafkaVersions = kafkaVersionsString != null && !kafkaVersionsString.isEmpty() ? kafkaVersionsString.split(",") : [] + +if (!knownKafkaVersions.containsAll(kafkaVersions)) { + throw new GradleException("Found unsupported Kafka versions: " + (kafkaVersions - knownKafkaVersions)) +} + List knownScalaVersions = System.getProperty("knownScalaVersions").split(",") String scalaVersion = System.getProperty("scalaVersion") != null ? System.getProperty("scalaVersion") : System.getProperty("defaultScalaVersion") @@ -191,14 +198,19 @@ if (hiveVersions.contains("3")) { project(':hive3-orc-bundle').name = 'iceberg-hive3-orc-bundle' } -include ":iceberg-kafka-connect:kafka-connect-events" -project(":iceberg-kafka-connect:kafka-connect-events").projectDir = file('kafka-connect/kafka-connect-events') -project(":iceberg-kafka-connect:kafka-connect-events").name = "iceberg-kafka-connect-events" +if (kafkaVersions.contains("3")) { + include 'kafka-connect' + project(':kafka-connect').name = 'iceberg-kafka-connect' -include ":iceberg-kafka-connect:kafka-connect" -project(":iceberg-kafka-connect:kafka-connect").projectDir = file('kafka-connect/kafka-connect') -project(":iceberg-kafka-connect:kafka-connect").name = "iceberg-kafka-connect" + include ":iceberg-kafka-connect:kafka-connect-events" + project(":iceberg-kafka-connect:kafka-connect-events").projectDir = file('kafka-connect/kafka-connect-events') + project(":iceberg-kafka-connect:kafka-connect-events").name = "iceberg-kafka-connect-events" -include ":iceberg-kafka-connect:kafka-connect-runtime" -project(":iceberg-kafka-connect:kafka-connect-runtime").projectDir = file('kafka-connect/kafka-connect-runtime') -project(":iceberg-kafka-connect:kafka-connect-runtime").name = "iceberg-kafka-connect-runtime" + include ":iceberg-kafka-connect:kafka-connect" + project(":iceberg-kafka-connect:kafka-connect").projectDir = file('kafka-connect/kafka-connect') + project(":iceberg-kafka-connect:kafka-connect").name = "iceberg-kafka-connect" + + include ":iceberg-kafka-connect:kafka-connect-runtime" + project(":iceberg-kafka-connect:kafka-connect-runtime").projectDir = file('kafka-connect/kafka-connect-runtime') + project(":iceberg-kafka-connect:kafka-connect-runtime").name = "iceberg-kafka-connect-runtime" +} From d499df9497ceb8b6b2a5b2244fda7a0822c050f6 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 18 Sep 2024 08:18:41 -0700 Subject: [PATCH 0699/1019] Core: Move internal struct projection to SupportsIndexProjection (#11132) --- .../java/org/apache/iceberg/BaseFile.java | 98 ++++++++++--------- .../org/apache/iceberg/GenericDataFile.java | 5 + .../org/apache/iceberg/GenericDeleteFile.java | 5 + .../apache/iceberg/GenericManifestEntry.java | 5 +- .../apache/iceberg/GenericManifestFile.java | 65 ++++-------- .../org/apache/iceberg/ManifestWriter.java | 3 +- .../iceberg/avro/SupportsIndexProjection.java | 85 ++++++++++++++++ .../java/org/apache/iceberg/TestBase.java | 20 +++- 8 files changed, 192 insertions(+), 94 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java diff --git a/core/src/main/java/org/apache/iceberg/BaseFile.java b/core/src/main/java/org/apache/iceberg/BaseFile.java index ddd08f944c98..8f84eb5737b9 100644 --- a/core/src/main/java/org/apache/iceberg/BaseFile.java +++ b/core/src/main/java/org/apache/iceberg/BaseFile.java @@ -30,8 +30,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.SupportsIndexProjection; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; -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.iceberg.util.ArrayUtil; @@ -39,7 +39,7 @@ import org.apache.iceberg.util.SerializableMap; /** Base class for both {@link DataFile} and {@link DeleteFile}. */ -abstract class BaseFile +abstract class BaseFile extends SupportsIndexProjection implements ContentFile, IndexedRecord, StructLike, @@ -55,7 +55,6 @@ public PartitionData copy() { } }; - private int[] fromProjectionPos; private Types.StructType partitionType; private Long fileOrdinal = null; @@ -85,40 +84,51 @@ public PartitionData copy() { // cached schema private transient Schema avroSchema = null; + // struct type that corresponds to the positions used for internalGet and internalSet + private static final Types.StructType BASE_TYPE = + Types.StructType.of( + DataFile.CONTENT, + DataFile.FILE_PATH, + DataFile.FILE_FORMAT, + DataFile.SPEC_ID, + Types.NestedField.required( + DataFile.PARTITION_ID, + DataFile.PARTITION_NAME, + EMPTY_STRUCT_TYPE, + DataFile.PARTITION_DOC), + DataFile.RECORD_COUNT, + DataFile.FILE_SIZE, + DataFile.COLUMN_SIZES, + DataFile.VALUE_COUNTS, + DataFile.NULL_VALUE_COUNTS, + DataFile.NAN_VALUE_COUNTS, + DataFile.LOWER_BOUNDS, + DataFile.UPPER_BOUNDS, + DataFile.KEY_METADATA, + DataFile.SPLIT_OFFSETS, + DataFile.EQUALITY_IDS, + DataFile.SORT_ORDER_ID, + MetadataColumns.ROW_POSITION); + /** Used by Avro reflection to instantiate this class when reading manifest files. */ BaseFile(Schema avroSchema) { + this(AvroSchemaUtil.convert(avroSchema).asStructType()); this.avroSchema = avroSchema; + } - Types.StructType schema = AvroSchemaUtil.convert(avroSchema).asNestedType().asStructType(); + /** Used by internal readers to instantiate this class with a projection schema. */ + BaseFile(Types.StructType projection) { + super(BASE_TYPE, projection); + this.avroSchema = AvroSchemaUtil.convert(projection, "data_file"); // partition type may be null if the field was not projected - Type partType = schema.fieldType("partition"); + Type partType = projection.fieldType("partition"); if (partType != null) { this.partitionType = partType.asNestedType().asStructType(); } else { this.partitionType = EMPTY_STRUCT_TYPE; } - List fields = schema.fields(); - List allFields = Lists.newArrayList(); - allFields.addAll(DataFile.getType(partitionType).fields()); - allFields.add(MetadataColumns.ROW_POSITION); - - this.fromProjectionPos = new int[fields.size()]; - for (int i = 0; i < fromProjectionPos.length; i += 1) { - boolean found = false; - for (int j = 0; j < allFields.size(); j += 1) { - if (fields.get(i).fieldId() == allFields.get(j).fieldId()) { - found = true; - fromProjectionPos[i] = j; - } - } - - if (!found) { - throw new IllegalArgumentException("Cannot find projected field: " + fields.get(i)); - } - } - this.partitionData = new PartitionData(partitionType); } @@ -140,6 +150,7 @@ public PartitionData copy() { int[] equalityFieldIds, Integer sortOrderId, ByteBuffer keyMetadata) { + super(BASE_TYPE.fields().size()); this.partitionSpecId = specId; this.content = content; this.filePath = filePath; @@ -178,6 +189,7 @@ public PartitionData copy() { * column stat is kept. */ BaseFile(BaseFile toCopy, boolean copyStats, Set requestedColumnIds) { + super(toCopy); this.fileOrdinal = toCopy.fileOrdinal; this.manifestLocation = toCopy.manifestLocation; this.partitionSpecId = toCopy.partitionSpecId; @@ -203,7 +215,6 @@ public PartitionData copy() { this.lowerBounds = null; this.upperBounds = null; } - this.fromProjectionPos = toCopy.fromProjectionPos; this.keyMetadata = toCopy.keyMetadata == null ? null @@ -222,7 +233,9 @@ public PartitionData copy() { } /** Constructor for Java serialization. */ - BaseFile() {} + BaseFile() { + super(BASE_TYPE.fields().size()); + } @Override public int specId() { @@ -266,13 +279,12 @@ public Schema getSchema() { } @Override - @SuppressWarnings("unchecked") public void put(int i, Object value) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } + set(i, value); + } + + @Override + protected void internalSet(int pos, T value) { switch (pos) { case 0: this.content = value != null ? FILE_CONTENT_VALUES[(Integer) value] : FileContent.DATA; @@ -335,18 +347,12 @@ public void put(int i, Object value) { } @Override - public void set(int pos, T value) { - put(pos, value); + protected T internalGet(int pos, Class javaClass) { + return javaClass.cast(getByPos(pos)); } - @Override - public Object get(int i) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } - switch (pos) { + private Object getByPos(int basePos) { + switch (basePos) { case 0: return content.id(); case 1: @@ -384,13 +390,13 @@ public Object get(int i) { case 17: return fileOrdinal; default: - throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); } } @Override - public T get(int pos, Class javaClass) { - return javaClass.cast(get(pos)); + public Object get(int pos) { + return get(pos, Object.class); } @Override diff --git a/core/src/main/java/org/apache/iceberg/GenericDataFile.java b/core/src/main/java/org/apache/iceberg/GenericDataFile.java index 8fe7ec756abf..7b99e7b60ab8 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDataFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDataFile.java @@ -32,6 +32,11 @@ class GenericDataFile extends BaseFile implements DataFile { super(avroSchema); } + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericDataFile(Types.StructType projection) { + super(projection); + } + GenericDataFile( int specId, String filePath, diff --git a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java index 509bf4c16b03..77e0d8505af6 100644 --- a/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericDeleteFile.java @@ -32,6 +32,11 @@ class GenericDeleteFile extends BaseFile implements DeleteFile { super(avroSchema); } + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericDeleteFile(Types.StructType projection) { + super(projection); + } + GenericDeleteFile( int specId, FileContent content, diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java index 575ea1f1c518..752f2785a947 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestEntry.java @@ -38,8 +38,9 @@ class GenericManifestEntry> this.schema = schema; } - GenericManifestEntry(Types.StructType partitionType) { - this.schema = AvroSchemaUtil.convert(V1Metadata.entrySchema(partitionType), "manifest_entry"); + /** Used by internal readers to instantiate this class with a projection schema. */ + GenericManifestEntry(Types.StructType schema) { + this.schema = AvroSchemaUtil.convert(schema, "manifest_entry"); } private GenericManifestEntry(GenericManifestEntry toCopy, boolean fullCopy) { diff --git a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java index d081e0bdd568..7707c57a6905 100644 --- a/core/src/main/java/org/apache/iceberg/GenericManifestFile.java +++ b/core/src/main/java/org/apache/iceberg/GenericManifestFile.java @@ -28,21 +28,20 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.avro.specific.SpecificData.SchemaConstructable; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.avro.SupportsIndexProjection; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Objects; import org.apache.iceberg.relocated.com.google.common.collect.Lists; -import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ByteBuffers; -public class GenericManifestFile +public class GenericManifestFile extends SupportsIndexProjection implements ManifestFile, StructLike, IndexedRecord, SchemaConstructable, Serializable { private static final Schema AVRO_SCHEMA = AvroSchemaUtil.convert(ManifestFile.schema(), "manifest_file"); private static final ManifestContent[] MANIFEST_CONTENT_VALUES = ManifestContent.values(); private transient Schema avroSchema; // not final for Java serialization - private int[] fromProjectionPos; // data fields private InputFile file = null; @@ -64,28 +63,12 @@ public class GenericManifestFile /** Used by Avro reflection to instantiate this class when reading manifest files. */ public GenericManifestFile(Schema avroSchema) { + super(ManifestFile.schema().asStruct(), AvroSchemaUtil.convert(avroSchema).asStructType()); this.avroSchema = avroSchema; - - List fields = AvroSchemaUtil.convert(avroSchema).asStructType().fields(); - List allFields = ManifestFile.schema().asStruct().fields(); - - this.fromProjectionPos = new int[fields.size()]; - for (int i = 0; i < fromProjectionPos.length; i += 1) { - boolean found = false; - for (int j = 0; j < allFields.size(); j += 1) { - if (fields.get(i).fieldId() == allFields.get(j).fieldId()) { - found = true; - fromProjectionPos[i] = j; - } - } - - if (!found) { - throw new IllegalArgumentException("Cannot find projected field: " + fields.get(i)); - } - } } GenericManifestFile(InputFile file, int specId) { + super(ManifestFile.schema().columns().size()); this.avroSchema = AVRO_SCHEMA; this.file = file; this.manifestPath = file.location(); @@ -101,7 +84,6 @@ public GenericManifestFile(Schema avroSchema) { this.deletedFilesCount = null; this.deletedRowsCount = null; this.partitions = null; - this.fromProjectionPos = null; this.keyMetadata = null; } @@ -122,6 +104,7 @@ public GenericManifestFile(Schema avroSchema) { Long existingRowsCount, Integer deletedFilesCount, Long deletedRowsCount) { + super(ManifestFile.schema().columns().size()); this.avroSchema = AVRO_SCHEMA; this.manifestPath = path; this.length = length; @@ -137,7 +120,6 @@ public GenericManifestFile(Schema avroSchema) { this.deletedFilesCount = deletedFilesCount; this.deletedRowsCount = deletedRowsCount; this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); - this.fromProjectionPos = null; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } @@ -157,6 +139,7 @@ public GenericManifestFile( long deletedRowsCount, List partitions, ByteBuffer keyMetadata) { + super(ManifestFile.schema().columns().size()); this.avroSchema = AVRO_SCHEMA; this.manifestPath = path; this.length = length; @@ -172,7 +155,6 @@ public GenericManifestFile( this.deletedFilesCount = deletedFilesCount; this.deletedRowsCount = deletedRowsCount; this.partitions = partitions == null ? null : partitions.toArray(new PartitionFieldSummary[0]); - this.fromProjectionPos = null; this.keyMetadata = ByteBuffers.toByteArray(keyMetadata); } @@ -182,6 +164,7 @@ public GenericManifestFile( * @param toCopy a generic manifest file to copy. */ private GenericManifestFile(GenericManifestFile toCopy) { + super(toCopy); this.avroSchema = toCopy.avroSchema; this.manifestPath = toCopy.manifestPath; this.length = toCopy.length; @@ -204,7 +187,6 @@ private GenericManifestFile(GenericManifestFile toCopy) { } else { this.partitions = null; } - this.fromProjectionPos = toCopy.fromProjectionPos; this.keyMetadata = toCopy.keyMetadata == null ? null @@ -212,7 +194,9 @@ private GenericManifestFile(GenericManifestFile toCopy) { } /** Constructor for Java serialization. */ - GenericManifestFile() {} + GenericManifestFile() { + super(ManifestFile.schema().columns().size()); + } @Override public String path() { @@ -308,18 +292,17 @@ public int size() { } @Override - public T get(int pos, Class javaClass) { - return javaClass.cast(get(pos)); + public Object get(int pos) { + return internalGet(pos, Object.class); } @Override - public Object get(int i) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } - switch (pos) { + protected T internalGet(int pos, Class javaClass) { + return javaClass.cast(getByPos(pos)); + } + + private Object getByPos(int basePos) { + switch (basePos) { case 0: return manifestPath; case 1: @@ -351,19 +334,13 @@ public Object get(int i) { case 14: return keyMetadata(); default: - throw new UnsupportedOperationException("Unknown field ordinal: " + pos); + throw new UnsupportedOperationException("Unknown field ordinal: " + basePos); } } @Override - @SuppressWarnings("unchecked") - public void set(int i, T value) { - int pos = i; - // if the schema was projected, map the incoming ordinal to the expected one - if (fromProjectionPos != null) { - pos = fromProjectionPos[i]; - } - switch (pos) { + protected void internalSet(int basePos, T value) { + switch (basePos) { case 0: // always coerce to String for Serializable this.manifestPath = value.toString(); diff --git a/core/src/main/java/org/apache/iceberg/ManifestWriter.java b/core/src/main/java/org/apache/iceberg/ManifestWriter.java index 88587a1ebc89..fbfc62b94fe4 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestWriter.java +++ b/core/src/main/java/org/apache/iceberg/ManifestWriter.java @@ -60,7 +60,8 @@ private ManifestWriter(PartitionSpec spec, EncryptedOutputFile file, Long snapsh this.specId = spec.specId(); this.writer = newAppender(spec, this.file); this.snapshotId = snapshotId; - this.reused = new GenericManifestEntry<>(spec.partitionType()); + this.reused = + new GenericManifestEntry<>(V1Metadata.entrySchema(spec.partitionType()).asStruct()); this.stats = new PartitionSummary(spec); this.keyMetadataBuffer = (file.keyMetadata() == null) ? null : file.keyMetadata().buffer(); } diff --git a/core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java b/core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java new file mode 100644 index 000000000000..fa4ffa5aec13 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/SupportsIndexProjection.java @@ -0,0 +1,85 @@ +/* + * 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.avro; + +import java.io.Serializable; +import java.util.List; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.types.Types; + +public abstract class SupportsIndexProjection implements StructLike, Serializable { + private final int[] fromProjectionPos; + + /** Noop constructor that does not project fields */ + protected SupportsIndexProjection(int size) { + this.fromProjectionPos = new int[size]; + for (int i = 0; i < fromProjectionPos.length; i++) { + fromProjectionPos[i] = i; + } + } + + /** Base constructor for building the type mapping */ + protected SupportsIndexProjection(Types.StructType baseType, Types.StructType projectionType) { + List allFields = baseType.fields(); + List fields = projectionType.fields(); + + this.fromProjectionPos = new int[fields.size()]; + for (int i = 0; i < fromProjectionPos.length; i += 1) { + boolean found = false; + for (int j = 0; j < allFields.size(); j += 1) { + if (fields.get(i).fieldId() == allFields.get(j).fieldId()) { + found = true; + fromProjectionPos[i] = j; + } + } + + if (!found) { + throw new IllegalArgumentException("Cannot find projected field: " + fields.get(i)); + } + } + } + + /** Copy constructor */ + protected SupportsIndexProjection(SupportsIndexProjection toCopy) { + this.fromProjectionPos = toCopy.fromProjectionPos; + } + + protected abstract T internalGet(int pos, Class javaClass); + + protected abstract void internalSet(int pos, T value); + + private int pos(int basePos) { + return fromProjectionPos[basePos]; + } + + @Override + public int size() { + return fromProjectionPos.length; + } + + @Override + public T get(int basePos, Class javaClass) { + return internalGet(pos(basePos), javaClass); + } + + @Override + public void set(int basePos, T value) { + internalSet(pos(basePos), value); + } +} diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index 23fabc2a9402..a0b52b346bf3 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -32,6 +32,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import org.apache.iceberg.avro.AvroSchemaUtil; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.OutputFile; @@ -350,7 +351,24 @@ > ManifestEntry manifestEntry( Long fileSequenceNumber, F file) { - GenericManifestEntry entry = new GenericManifestEntry<>(table.spec().partitionType()); + Schema manifestEntrySchema; + switch (table.ops().current().formatVersion()) { + case 1: + manifestEntrySchema = V1Metadata.entrySchema(table.spec().partitionType()); + break; + case 2: + manifestEntrySchema = V2Metadata.entrySchema(table.spec().partitionType()); + break; + case 3: + manifestEntrySchema = V3Metadata.entrySchema(table.spec().partitionType()); + break; + default: + throw new IllegalArgumentException( + "Unsupported format version: " + table.ops().current().formatVersion()); + } + + GenericManifestEntry entry = + new GenericManifestEntry<>(AvroSchemaUtil.convert(manifestEntrySchema, "manifest_entry")); switch (status) { case ADDED: if (dataSequenceNumber != null && dataSequenceNumber != 0) { From 51095ef4704c5e18f7f48c79b822235fdb9c4cf6 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 19 Sep 2024 07:48:29 +0200 Subject: [PATCH 0700/1019] Core: Add explicit JSON parser for LoadTableResponse (#11148) --- .../apache/iceberg/rest/RESTSerializers.java | 23 +- .../rest/responses/LoadTableResponse.java | 8 +- .../responses/LoadTableResponseParser.java | 90 ++++++++ .../TestLoadTableResponseParser.java | 203 ++++++++++++++++++ 4 files changed, 322 insertions(+), 2 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java create mode 100644 core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.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 341dda0e3f6c..7f39d0bc1f5e 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSerializers.java @@ -60,6 +60,8 @@ import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.ErrorResponseParser; import org.apache.iceberg.rest.responses.ImmutableLoadViewResponse; +import org.apache.iceberg.rest.responses.LoadTableResponse; +import org.apache.iceberg.rest.responses.LoadTableResponseParser; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.LoadViewResponseParser; import org.apache.iceberg.rest.responses.OAuthTokenResponse; @@ -115,7 +117,9 @@ public static void registerAll(ObjectMapper mapper) { .addDeserializer(LoadViewResponse.class, new LoadViewResponseDeserializer<>()) .addDeserializer(ImmutableLoadViewResponse.class, new LoadViewResponseDeserializer<>()) .addSerializer(ConfigResponse.class, new ConfigResponseSerializer<>()) - .addDeserializer(ConfigResponse.class, new ConfigResponseDeserializer<>()); + .addDeserializer(ConfigResponse.class, new ConfigResponseDeserializer<>()) + .addSerializer(LoadTableResponse.class, new LoadTableResponseSerializer<>()) + .addDeserializer(LoadTableResponse.class, new LoadTableResponseDeserializer<>()); mapper.registerModule(module); } @@ -422,4 +426,21 @@ public T deserialize(JsonParser p, DeserializationContext context) throws IOExce return (T) ConfigResponseParser.fromJson(jsonNode); } } + + static class LoadTableResponseSerializer extends JsonSerializer { + @Override + public void serialize(T request, JsonGenerator gen, SerializerProvider serializers) + throws IOException { + LoadTableResponseParser.toJson(request, gen); + } + } + + static class LoadTableResponseDeserializer + extends JsonDeserializer { + @Override + public T deserialize(JsonParser p, DeserializationContext context) throws IOException { + JsonNode jsonNode = p.getCodec().readTree(p); + return (T) LoadTableResponseParser.fromJson(jsonNode); + } + } } 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 5e5353c3909a..519d1fc34044 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 @@ -39,6 +39,7 @@ public class LoadTableResponse implements RESTResponse { private String metadataLocation; private TableMetadata metadata; private Map config; + private TableMetadata metadataWithLocation; public LoadTableResponse() { // Required for Jackson deserialization @@ -61,7 +62,12 @@ public String metadataLocation() { } public TableMetadata tableMetadata() { - return TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build(); + if (null == metadataWithLocation) { + this.metadataWithLocation = + TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build(); + } + + return metadataWithLocation; } public Map config() { 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 new file mode 100644 index 000000000000..316c5160ddc5 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/rest/responses/LoadTableResponseParser.java @@ -0,0 +1,90 @@ +/* + * 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.TableMetadata; +import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.JsonUtil; + +public class LoadTableResponseParser { + + private static final String METADATA_LOCATION = "metadata-location"; + private static final String METADATA = "metadata"; + private static final String CONFIG = "config"; + + private LoadTableResponseParser() {} + + public static String toJson(LoadTableResponse response) { + return toJson(response, false); + } + + public static String toJson(LoadTableResponse response, boolean pretty) { + return JsonUtil.generate(gen -> toJson(response, gen), pretty); + } + + public static void toJson(LoadTableResponse response, JsonGenerator gen) throws IOException { + Preconditions.checkArgument(null != response, "Invalid load table response: null"); + + gen.writeStartObject(); + + if (null != response.metadataLocation()) { + gen.writeStringField(METADATA_LOCATION, response.metadataLocation()); + } + + gen.writeFieldName(METADATA); + TableMetadataParser.toJson(response.tableMetadata(), gen); + + if (!response.config().isEmpty()) { + JsonUtil.writeStringMap(CONFIG, response.config(), gen); + } + + gen.writeEndObject(); + } + + public static LoadTableResponse fromJson(String json) { + return JsonUtil.parse(json, LoadTableResponseParser::fromJson); + } + + public static LoadTableResponse fromJson(JsonNode json) { + Preconditions.checkArgument(null != json, "Cannot parse load table response from null object"); + + String metadataLocation = null; + if (json.hasNonNull(METADATA_LOCATION)) { + metadataLocation = JsonUtil.getString(METADATA_LOCATION, json); + } + + TableMetadata metadata = TableMetadataParser.fromJson(JsonUtil.get(METADATA, json)); + + if (null != metadataLocation) { + metadata = TableMetadata.buildFrom(metadata).withMetadataLocation(metadataLocation).build(); + } + + LoadTableResponse.Builder builder = LoadTableResponse.builder().withTableMetadata(metadata); + + if (json.hasNonNull(CONFIG)) { + builder.addAllConfig(JsonUtil.getStringMap(CONFIG, json)); + } + + return builder.build(); + } +} 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 new file mode 100644 index 000000000000..b87c66bffe94 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/rest/responses/TestLoadTableResponseParser.java @@ -0,0 +1,203 @@ +/* + * 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.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; + +public class TestLoadTableResponseParser { + + @Test + public void nullAndEmptyCheck() { + assertThatThrownBy(() -> LoadTableResponseParser.toJson(null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Invalid load table response: null"); + + assertThatThrownBy(() -> LoadTableResponseParser.fromJson((JsonNode) null)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse load table response from null object"); + + assertThatThrownBy(() -> LoadTableResponseParser.fromJson("{}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: metadata"); + } + + @Test + public void missingFields() { + assertThatThrownBy( + () -> LoadTableResponseParser.fromJson("{\"metadata-location\": \"custom-location\"}")) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: metadata"); + } + + @Test + public void roundTripSerde() { + 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).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\" : %d,\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" + + "}", + 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); + } + + @Test + public void roundTripSerdeWithConfig() { + 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")) + .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\" : %d,\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" + + "}", + 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); + } +} From 61e7aaf5daede941130e1f135b214b6c33d3916d Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Thu, 19 Sep 2024 14:33:12 +0800 Subject: [PATCH 0701/1019] Build: Add .java-version to gitignore (#11167) --- .gitignore | 3 +++ 1 file changed, 3 insertions(+) diff --git a/.gitignore b/.gitignore index e89662a10c6d..e4c9e1a16a27 100644 --- a/.gitignore +++ b/.gitignore @@ -66,3 +66,6 @@ metastore_db/ # Spark/metastore files spark-warehouse/ derby.log + +# jenv +.java-version From b2aa5e859c81087257f2ce6c8baf8604a59a5976 Mon Sep 17 00:00:00 2001 From: Jason Fehr Date: Thu, 19 Sep 2024 00:15:16 -0700 Subject: [PATCH 0702/1019] Docs: Clarify Partition Transform (#8337) --- docs/docs/partitioning.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/docs/partitioning.md b/docs/docs/partitioning.md index 09c20439d1b1..aa573b33e968 100644 --- a/docs/docs/partitioning.md +++ b/docs/docs/partitioning.md @@ -84,7 +84,7 @@ This leads to several problems: Iceberg produces partition values by taking a column value and optionally transforming it. Iceberg is responsible for converting `event_time` into `event_date`, and keeps track of the relationship. -Table partitioning is configured using these relationships. The `logs` table would be partitioned by `date(event_time)` and `level`. +Table partitioning is configured using these relationships. The `logs` table would be partitioned by `day(event_time)` and `level`. Because Iceberg doesn't require user-maintained partition columns, it can hide partitioning. Partition values are produced correctly every time and always used to speed up queries, when possible. Producers and consumers wouldn't even see `event_date`. From d1a2bbfd6d31b1d9d6f2f244ec564f85b1b7b8f0 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 19 Sep 2024 13:38:03 -0700 Subject: [PATCH 0703/1019] API, Core: Enable removing rewritten delete files in RowDelta (#11166) --- .../java/org/apache/iceberg/RowDelta.java | 11 ++ .../iceberg/ReplaceDeleteFilesBenchmark.java | 130 ++++++++++++++++ .../java/org/apache/iceberg/BaseRowDelta.java | 6 + .../java/org/apache/iceberg/TestRowDelta.java | 147 ++++++++++++++++++ 4 files changed, 294 insertions(+) create mode 100644 core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java diff --git a/api/src/main/java/org/apache/iceberg/RowDelta.java b/api/src/main/java/org/apache/iceberg/RowDelta.java index 624f6c15d20b..a5e3fa477ba9 100644 --- a/api/src/main/java/org/apache/iceberg/RowDelta.java +++ b/api/src/main/java/org/apache/iceberg/RowDelta.java @@ -46,6 +46,17 @@ public interface RowDelta extends SnapshotUpdate { */ RowDelta addDeletes(DeleteFile deletes); + /** + * Removes a rewritten {@link DeleteFile} from the table. + * + * @param deletes a delete file that can be removed from the table + * @return this for method chaining + */ + default RowDelta removeDeletes(DeleteFile deletes) { + throw new UnsupportedOperationException( + getClass().getName() + " does not implement removeDeletes"); + } + /** * Set the snapshot ID used in any reads for this operation. * diff --git a/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.java new file mode 100644 index 000000000000..a899b870a90c --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/ReplaceDeleteFilesBenchmark.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; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.util.List; +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.types.Types; +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; + +/** + * A benchmark that evaluates the performance of replacing delete files in the table. + * + *

    To run this benchmark: + * ./gradlew :iceberg-core:jmh + * -PjmhIncludeRegex=ReplaceDeleteFilesBenchmark + * -PjmhOutputPath=benchmark/replace-delete-files-benchmark.txt + * + */ +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 3) +@Measurement(iterations = 5) +@BenchmarkMode(Mode.SingleShotTime) +@Timeout(time = 10, timeUnit = TimeUnit.MINUTES) +public class ReplaceDeleteFilesBenchmark { + + private static final String TABLE_IDENT = "tbl"; + private static final Schema SCHEMA = + new Schema( + required(1, "int_col", Types.IntegerType.get()), + required(2, "long_col", Types.LongType.get()), + required(3, "decimal_col", Types.DecimalType.of(10, 10)), + required(4, "date_col", Types.DateType.get()), + required(5, "timestamp_col", Types.TimestampType.withoutZone()), + required(6, "timestamp_tz_col", Types.TimestampType.withZone()), + required(7, "str_col", Types.StringType.get())); + private static final PartitionSpec SPEC = PartitionSpec.unpartitioned(); + private static final HadoopTables TABLES = new HadoopTables(); + + private Table table; + private List deleteFiles; + private List pendingDeleteFiles; + + @Param({"50000", "100000", "500000", "1000000", "2500000"}) + private int numFiles; + + @Setup + public void setupBenchmark() { + initTable(); + initFiles(); + } + + @TearDown + public void tearDownBenchmark() { + dropTable(); + } + + @Benchmark + @Threads(1) + public void replaceDeleteFiles() { + RowDelta rowDelta = table.newRowDelta(); + deleteFiles.forEach(rowDelta::removeDeletes); + pendingDeleteFiles.forEach(rowDelta::addDeletes); + rowDelta.commit(); + } + + private void initTable() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + } + + private void dropTable() { + TABLES.dropTable(TABLE_IDENT); + } + + private void initFiles() { + List generatedDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); + List generatedPendingDeleteFiles = Lists.newArrayListWithExpectedSize(numFiles); + + 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); + } + + rowDelta.commit(); + + this.deleteFiles = generatedDeleteFiles; + this.pendingDeleteFiles = generatedPendingDeleteFiles; + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java index 42fd17f0320b..85c2269ee526 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRowDelta.java +++ b/core/src/main/java/org/apache/iceberg/BaseRowDelta.java @@ -62,6 +62,12 @@ public RowDelta addDeletes(DeleteFile deletes) { return this; } + @Override + public RowDelta removeDeletes(DeleteFile deletes) { + delete(deletes); + return this; + } + @Override public RowDelta validateFromSnapshot(long snapshotId) { this.startingSnapshotId = snapshotId; diff --git a/core/src/test/java/org/apache/iceberg/TestRowDelta.java b/core/src/test/java/org/apache/iceberg/TestRowDelta.java index a2a043e630bb..1d67e48a2ce2 100644 --- a/core/src/test/java/org/apache/iceberg/TestRowDelta.java +++ b/core/src/test/java/org/apache/iceberg/TestRowDelta.java @@ -1409,4 +1409,151 @@ public void testRowDeltaCaseSensitivity() { .isInstanceOf(ValidationException.class) .hasMessageStartingWith("Found new conflicting delete files"); } + + @TestTemplate + public void testRewrittenDeleteFiles() { + 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); + + 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)); + + List 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() { + 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); + + // commit the first DELETE operation that replaces `deleteFile` + DeleteFile newDeleteFile1 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta delete1 = + table + .newRowDelta() + .addDeletes(newDeleteFile1) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles(); + Snapshot snapshot1 = commit(table, delete1, branch); + assertThat(snapshot1.operation()).isEqualTo(DataOperations.DELETE); + assertThat(snapshot1.sequenceNumber()).isEqualTo(2L); + + // commit the second DELETE operation that replaces `deleteFile` + DeleteFile newDeleteFile2 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta delete2 = + table + .newRowDelta() + .addDeletes(newDeleteFile2) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles(); + Snapshot snapshot2 = commit(table, delete2, branch); + assertThat(snapshot2.operation()).isEqualTo(DataOperations.DELETE); + assertThat(snapshot2.sequenceNumber()).isEqualTo(3L); + + List dataManifests = snapshot2.dataManifests(table.io()); + assertThat(dataManifests).hasSize(1); + validateManifest( + dataManifests.get(0), + dataSeqs(1L), + fileSeqs(1L), + ids(baseSnapshot.snapshotId()), + files(dataFile), + statuses(Status.ADDED)); + + // verify both new delete files have been added + List deleteManifests = snapshot2.deleteManifests(table.io()); + assertThat(deleteManifests).hasSize(2); + validateDeleteManifest( + deleteManifests.get(0), + dataSeqs(3L), + fileSeqs(3L), + ids(snapshot2.snapshotId()), + files(newDeleteFile2), + statuses(Status.ADDED)); + validateDeleteManifest( + deleteManifests.get(1), + dataSeqs(2L), + fileSeqs(2L), + ids(snapshot1.snapshotId()), + files(newDeleteFile1), + statuses(Status.ADDED)); + } + + @TestTemplate + public void testConcurrentMergeRewriteSameDeleteFile() { + 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); + + // commit a DELETE operation that replaces `deleteFile` + DeleteFile newDeleteFile1 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta delete = + table + .newRowDelta() + .addDeletes(newDeleteFile1) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles(); + commit(table, delete, branch); + + // attempt to commit a MERGE operation that replaces `deleteFile` + DataFile newDataFile2 = newDataFile("data_bucket=0"); + DeleteFile newDeleteFile2 = newDeleteFile(dataFile.specId(), "data_bucket=0"); + RowDelta merge = + table + .newRowDelta() + .addRows(newDataFile2) + .addDeletes(newDeleteFile2) + .removeDeletes(deleteFile) + .validateFromSnapshot(baseSnapshot.snapshotId()) + .validateNoConflictingDataFiles() + .validateNoConflictingDeleteFiles(); + + // MERGE must fail as DELETE could have deleted more positions + assertThatThrownBy(() -> commit(table, merge, branch)) + .isInstanceOf(ValidationException.class) + .hasMessageStartingWith("Found new conflicting delete files that can apply"); + } } From 25e329bd0065ef1a6dd6977c6973ed7bfeab680a Mon Sep 17 00:00:00 2001 From: jonaswk Date: Thu, 19 Sep 2024 22:40:12 +0200 Subject: [PATCH 0704/1019] Docs: `field_id` in name serialisation spec should read `field-id` (#11135) --- format/spec.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/format/spec.md b/format/spec.md index dcddd80de42f..eb5900e27b4b 100644 --- a/format/spec.md +++ b/format/spec.md @@ -1246,7 +1246,7 @@ Name mapping is serialized as a list of field mapping JSON Objects which are ser |Field mapping field|JSON representation|Example| |--- |--- |--- | |**`names`**|`JSON list of strings`|`["latitude", "lat"]`| -|**`field_id`**|`JSON int`|`1`| +|**`field-id`**|`JSON int`|`1`| |**`fields`**|`JSON field mappings (list of objects)`|`[{ `
      `"field-id": 4,`
      `"names": ["latitude", "lat"]`
    `}, {`
      `"field-id": 5,`
      `"names": ["longitude", "long"]`
    `}]`| Example From 257e2924e10732f78faff3a0f47e72d88388a953 Mon Sep 17 00:00:00 2001 From: sullis Date: Fri, 20 Sep 2024 02:03:00 -0700 Subject: [PATCH 0705/1019] AWS: Bump AWS SDK to version 2.28.5 (#11170) --- aws-bundle/LICENSE | 72 ++++++++++++++++++------------------- aws-bundle/NOTICE | 76 +++++++++++++++++++-------------------- gradle/libs.versions.toml | 2 +- 3 files changed, 75 insertions(+), 75 deletions(-) diff --git a/aws-bundle/LICENSE b/aws-bundle/LICENSE index 0a9656c2c7c9..2e7bc0c946ff 100644 --- a/aws-bundle/LICENSE +++ b/aws-bundle/LICENSE @@ -303,210 +303,210 @@ License: MIT License - http://www.opensource.org/licenses/mit-license.php -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: annotations Version: 2.27.7 +Group: software.amazon.awssdk Name: annotations Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: apache-client Version: 2.27.7 +Group: software.amazon.awssdk Name: apache-client Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: arns Version: 2.27.7 +Group: software.amazon.awssdk Name: arns Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: auth Version: 2.27.7 +Group: software.amazon.awssdk Name: auth Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-core Version: 2.27.7 +Group: software.amazon.awssdk Name: aws-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.27.7 +Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.27.7 +Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.27.7 +Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: checksums Version: 2.27.7 +Group: software.amazon.awssdk Name: checksums Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: checksums-spi Version: 2.27.7 +Group: software.amazon.awssdk Name: checksums-spi Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: crt-core Version: 2.27.7 +Group: software.amazon.awssdk Name: crt-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: dynamodb Version: 2.27.7 +Group: software.amazon.awssdk Name: dynamodb Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: endpoints-spi Version: 2.27.7 +Group: software.amazon.awssdk Name: endpoints-spi Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: glue Version: 2.27.7 +Group: software.amazon.awssdk Name: glue Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-auth Version: 2.27.7 +Group: software.amazon.awssdk Name: http-auth Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-auth-aws Version: 2.27.7 +Group: software.amazon.awssdk Name: http-auth-aws Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.27.7 +Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.27.7 +Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-auth-spi Version: 2.27.7 +Group: software.amazon.awssdk Name: http-auth-spi Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: http-client-spi Version: 2.27.7 +Group: software.amazon.awssdk Name: http-client-spi Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: iam Version: 2.27.7 +Group: software.amazon.awssdk Name: iam Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: identity-spi Version: 2.27.7 +Group: software.amazon.awssdk Name: identity-spi Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: json-utils Version: 2.27.7 +Group: software.amazon.awssdk Name: json-utils Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: kms Version: 2.27.7 +Group: software.amazon.awssdk Name: kms Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: lakeformation Version: 2.27.7 +Group: software.amazon.awssdk Name: lakeformation Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: metrics-spi Version: 2.27.7 +Group: software.amazon.awssdk Name: metrics-spi Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: netty-nio-client Version: 2.27.7 +Group: software.amazon.awssdk Name: netty-nio-client Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: profiles Version: 2.27.7 +Group: software.amazon.awssdk Name: profiles Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: protocol-core Version: 2.27.7 +Group: software.amazon.awssdk Name: protocol-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: regions Version: 2.27.7 +Group: software.amazon.awssdk Name: regions Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: s3 Version: 2.27.7 +Group: software.amazon.awssdk Name: s3 Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sdk-core Version: 2.27.7 +Group: software.amazon.awssdk Name: sdk-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sso Version: 2.27.7 +Group: software.amazon.awssdk Name: sso Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: sts Version: 2.27.7 +Group: software.amazon.awssdk Name: sts Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.27.7 +Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.28.5 Project URL: https://aws.amazon.com/sdkforjava License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- -Group: software.amazon.awssdk Name: utils Version: 2.27.7 +Group: software.amazon.awssdk Name: utils Version: 2.28.5 License: Apache License, Version 2.0 - https://aws.amazon.com/apache2.0 -------------------------------------------------------------------------------- diff --git a/aws-bundle/NOTICE b/aws-bundle/NOTICE index 070d109c8e5f..20f84e0efd9b 100644 --- a/aws-bundle/NOTICE +++ b/aws-bundle/NOTICE @@ -23,43 +23,43 @@ Copyright (c) 2008 Alexander Beider & Stephen P. Morse. -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: checksums Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: checksums-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: crt-core Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: http-auth Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: http-auth-aws Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: http-auth-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: identity-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: retries Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: retries-spi Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.27.7 -NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: annotations Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: apache-client Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: arns Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: auth Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-json-protocol Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-query-protocol Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: aws-xml-protocol Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: checksums Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: checksums-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: crt-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: dynamodb Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: endpoints-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: glue Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-crt Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-aws-eventstream Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-auth-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: http-client-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: iam Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: identity-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: json-utils Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: kms Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: lakeformation Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: metrics-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: netty-nio-client Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: profiles Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: protocol-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: regions Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: retries Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: retries-spi Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: s3 Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: sdk-core Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: sso Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: sts Version: 2.28.5 +NOTICE for Group: software.amazon.awssdk Name: utils Version: 2.28.5 AWS SDK for Java 2.0 Copyright Amazon.com, Inc. or its affiliates. All Rights Reserved. @@ -82,7 +82,7 @@ The licenses for these third party components are included in LICENSE.txt -------------------------------------------------------------------------------- -NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.27.7 +NOTICE for Group: software.amazon.awssdk Name: third-party-jackson-core Version: 2.28.5 # Jackson JSON processor diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index ad5f0f406130..bdad45095ce2 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.27.21" +awssdk-bom = "2.28.5" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From f6c2d8c6c2d490f02c1fd2b5514185a87673a36a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 20 Sep 2024 11:03:33 +0200 Subject: [PATCH 0706/1019] Build: Bump org.xerial.snappy:snappy-java from 1.1.10.6 to 1.1.10.7 (#11140) Bumps [org.xerial.snappy:snappy-java](https://github.com/xerial/snappy-java) from 1.1.10.6 to 1.1.10.7. - [Release notes](https://github.com/xerial/snappy-java/releases) - [Commits](https://github.com/xerial/snappy-java/compare/v1.1.10.6...v1.1.10.7) --- updated-dependencies: - dependency-name: org.xerial.snappy:snappy-java 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> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index 785dc0a8c1ee..ad76fe7e1633 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -70,7 +70,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { // force upgrades for dependencies with known vulnerabilities... resolutionStrategy { force 'org.codehaus.jettison:jettison:1.5.4' - force 'org.xerial.snappy:snappy-java:1.1.10.6' + 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.2.0' } From 3aa011fc87da80bc4c8ac59b81c0ceee8e75d24b Mon Sep 17 00:00:00 2001 From: dongwang Date: Fri, 20 Sep 2024 21:17:28 +0800 Subject: [PATCH 0707/1019] Spark 3.3, 3.4, 3.5: Supplement test case for `RollbackToTimestampProcedure` (#11171) --- .../TestRollbackToTimestampProcedure.java | 35 +++++++++++++++++++ .../TestRollbackToTimestampProcedure.java | 35 +++++++++++++++++++ .../TestRollbackToTimestampProcedure.java | 35 +++++++++++++++++++ 3 files changed, 105 insertions(+) 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 281bfbf7ad1b..c65be60cc1a7 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 @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDateTime; import java.util.List; import java.util.Map; @@ -252,6 +254,39 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + @Test public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; 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 82df7567fce6..1cfe1ecf83be 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 @@ -20,6 +20,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; +import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDateTime; import java.util.List; import java.util.Map; @@ -252,6 +254,39 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @Test + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + @Test public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; 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 08e6b1323c79..6b74391898e0 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 @@ -21,6 +21,8 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.assertj.core.api.Assumptions.assumeThat; +import java.sql.Timestamp; +import java.time.Instant; import java.time.LocalDateTime; import java.util.List; import org.apache.iceberg.Snapshot; @@ -246,6 +248,39 @@ public void testRollbackToTimestampWithoutExplicitCatalog() { sql("SELECT * FROM %s ORDER BY id", tableName)); } + @TestTemplate + public void testRollbackToTimestampBeforeOrEqualToOldestSnapshot() { + sql("CREATE TABLE %s (id bigint NOT NULL, data string) USING iceberg", tableName); + sql("INSERT INTO TABLE %s VALUES (1, 'a')", tableName); + + Table table = validationCatalog.loadTable(tableIdent); + Snapshot firstSnapshot = table.currentSnapshot(); + Timestamp beforeFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis() - 1)); + Timestamp exactFirstSnapshot = + Timestamp.from(Instant.ofEpochMilli(firstSnapshot.timestampMillis())); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, beforeFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + beforeFirstSnapshot.toInstant().toEpochMilli()); + + assertThatThrownBy( + () -> + sql( + "CALL %s.system.rollback_to_timestamp(timestamp => TIMESTAMP '%s', table => '%s')", + catalogName, exactFirstSnapshot, tableIdent)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage( + "Cannot roll back, no valid snapshot older than: %s", + exactFirstSnapshot.toInstant().toEpochMilli()); + } + @TestTemplate public void testInvalidRollbackToTimestampCases() { String timestamp = "TIMESTAMP '2007-12-03T10:15:30'"; From 0806ed4a46f1efb83261cacc6ce76716e91d7524 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sat, 21 Sep 2024 04:36:24 +0900 Subject: [PATCH 0708/1019] Docs: Uppercase SQL keywords in branching docs (#11172) --- docs/docs/branching.md | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/docs/docs/branching.md b/docs/docs/branching.md index f00defda665b..bfb976bfa206 100644 --- a/docs/docs/branching.md +++ b/docs/docs/branching.md @@ -127,7 +127,7 @@ Create a table and insert some data: ```sql CREATE TABLE db.table (id bigint, data string, col float); -INSERT INTO db.table values (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 3.0); +INSERT INTO db.table VALUES (1, 'a', 1.0), (2, 'b', 2.0), (3, 'c', 3.0); SELECT * FROM db.table; 1 a 1.0 2 b 2.0 @@ -148,11 +148,11 @@ SELECT * FROM db.table.branch_test_branch; Modify the table's schema by dropping the `col` column and adding a new column named `new_col`: ```sql -ALTER TABLE db.table drop column col; +ALTER TABLE db.table DROP COLUMN col; -ALTER TABLE db.table add column new_col date; +ALTER TABLE db.table ADD COLUMN new_col date; -INSERT INTO db.table values (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05')); +INSERT INTO db.table VALUES (4, 'd', date('2024-04-04')), (5, 'e', date('2024-05-05')); SELECT * FROM db.table; 1 a NULL @@ -195,7 +195,7 @@ When writing to the branch, the **table's schema** is used for validation: ```sql -INSERT INTO db.table.branch_test_branch values (6, 'e', date('2024-06-06')), (7, 'g', date('2024-07-07')); +INSERT INTO db.table.branch_test_branch VALUES (6, 'e', date('2024-06-06')), (7, 'g', date('2024-07-07')); SELECT * FROM db.table.branch_test_branch; 6 e 2024-06-06 From 1b7f43ebd5cbd28a0ca04cbaf639a0a4d27f2624 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:42:47 +0200 Subject: [PATCH 0709/1019] Build: Bump org.apache.httpcomponents.client5:httpclient5 (#11186) Bumps [org.apache.httpcomponents.client5:httpclient5](https://github.com/apache/httpcomponents-client) from 5.3.1 to 5.4. - [Changelog](https://github.com/apache/httpcomponents-client/blob/master/RELEASE_NOTES.txt) - [Commits](https://github.com/apache/httpcomponents-client/compare/rel/v5.3.1...rel/v5.4) --- updated-dependencies: - dependency-name: org.apache.httpcomponents.client5:httpclient5 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 bdad45095ce2..dc23f1247b27 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -47,7 +47,7 @@ google-libraries-bom = "26.44.0" guava = "33.3.0-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" -httpcomponents-httpclient5 = "5.3.1" +httpcomponents-httpclient5 = "5.4" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" From e46a275ac05497074db92ec77dd545b9a2aba0e1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:43:16 +0200 Subject: [PATCH 0710/1019] Build: Bump tez010 from 0.10.3 to 0.10.4 (#11183) Bumps `tez010` from 0.10.3 to 0.10.4. Updates `org.apache.tez:tez-dag` from 0.10.3 to 0.10.4 Updates `org.apache.tez:tez-mapreduce` from 0.10.3 to 0.10.4 --- updated-dependencies: - dependency-name: org.apache.tez:tez-dag dependency-type: direct:production update-type: version-update:semver-patch - dependency-name: org.apache.tez:tez-mapreduce 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 dc23f1247b27..44f9cbd4043c 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -88,7 +88,7 @@ spring-boot = "2.7.18" spring-web = "5.3.39" sqlite-jdbc = "3.46.1.0" testcontainers = "1.20.1" -tez010 = "0.10.3" +tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above [libraries] From 057b07da874775a2ac0244ca3d3935e73abceeae Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:43:30 +0200 Subject: [PATCH 0711/1019] Build: Bump nessie from 0.95.0 to 0.97.1 (#11184) Bumps `nessie` from 0.95.0 to 0.97.1. Updates `org.projectnessie.nessie:nessie-client` from 0.95.0 to 0.97.1 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.95.0 to 0.97.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.95.0 to 0.97.1 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.95.0 to 0.97.1 --- 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 44f9cbd4043c..91918c568213 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -69,7 +69,7 @@ kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.95.0" +nessie = "0.97.1" netty-buffer = "4.1.113.Final" netty-buffer-compat = "4.1.113.Final" object-client-bundle = "3.3.2" From c4a655db27617f0ee78afcf3708cabbbc8253d9c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:43:45 +0200 Subject: [PATCH 0712/1019] Build: Bump com.google.cloud:libraries-bom from 26.44.0 to 26.47.0 (#11185) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.44.0 to 26.47.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.44.0...v26.47.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 91918c568213..bd0dde16f460 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -43,7 +43,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.44.0" +google-libraries-bom = "26.47.0" guava = "33.3.0-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" From 4b8425ef336e3acec9b657055d51baccc15f8ea4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 23 Sep 2024 14:44:07 +0200 Subject: [PATCH 0713/1019] Build: Bump org.roaringbitmap:RoaringBitmap from 1.2.1 to 1.3.0 (#11187) Bumps [org.roaringbitmap:RoaringBitmap](https://github.com/RoaringBitmap/RoaringBitmap) from 1.2.1 to 1.3.0. - [Release notes](https://github.com/RoaringBitmap/RoaringBitmap/releases) - [Commits](https://github.com/RoaringBitmap/RoaringBitmap/compare/1.2.1...1.3.0) --- updated-dependencies: - dependency-name: org.roaringbitmap:RoaringBitmap 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 bd0dde16f460..34471c2c4b75 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -76,7 +76,7 @@ object-client-bundle = "3.3.2" orc = "1.9.4" parquet = "1.13.1" pig = "0.17.0" -roaringbitmap = "1.2.1" +roaringbitmap = "1.3.0" s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" From 334f00c3db1aced79d6e9fb168ff99c900c83c49 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 23 Sep 2024 14:21:38 -0600 Subject: [PATCH 0714/1019] API: Deprecate ContentFile#path API and add location API which returns String (#11092) Co-authored-by: Eduard Tudenhoefner --- .../main/java/org/apache/iceberg/ContentFile.java | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/api/src/main/java/org/apache/iceberg/ContentFile.java b/api/src/main/java/org/apache/iceberg/ContentFile.java index 9db19f41ebfb..a4d97fa93fa1 100644 --- a/api/src/main/java/org/apache/iceberg/ContentFile.java +++ b/api/src/main/java/org/apache/iceberg/ContentFile.java @@ -51,9 +51,19 @@ default String manifestLocation() { */ FileContent content(); - /** Returns fully qualified path to the file, suitable for constructing a Hadoop Path. */ + /** + * Returns fully qualified path to the file, suitable for constructing a Hadoop Path. + * + * @deprecated since 1.7.0, will be removed in 2.0.0; use {@link #location()} instead. + */ + @Deprecated CharSequence path(); + /** Return the fully qualified path to the file. */ + default String location() { + return path().toString(); + } + /** Returns format of the file. */ FileFormat format(); From 3be67be2803e65ec30e89b4bfbdb3d93ae004b60 Mon Sep 17 00:00:00 2001 From: Prashant Singh <35593236+singhpk234@users.noreply.github.com> Date: Mon, 23 Sep 2024 17:41:39 -0700 Subject: [PATCH 0715/1019] Docs: Document AWS Redshift and Amazon Data Firehose support (#11192) Co-authored-by: Prashant Singh --- docs/docs/aws.md | 6 ++++++ docs/mkdocs.yml | 2 ++ 2 files changed, 8 insertions(+) diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 973248a2b87c..b9638bb50dad 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -685,3 +685,9 @@ Search the [Iceberg blogs](../../blogs.md) page for tutorials around running Ice [Amazon Kinesis Data Analytics](https://aws.amazon.com/about-aws/whats-new/2019/11/you-can-now-run-fully-managed-apache-flink-applications-with-apache-kafka/) provides a platform to run fully managed Apache Flink applications. You can include Iceberg in your application Jar and run it in the platform. + +### AWS Redshift +[AWS Redshift Spectrum or Redshift Serverless](https://docs.aws.amazon.com/redshift/latest/dg/querying-iceberg.html) supports querying Apache Iceberg tables cataloged in the AWS Glue Data Catalog. + +### Amazon Data Firehose +You can use [Firehose](https://docs.aws.amazon.com/firehose/latest/dev/apache-iceberg-destination.html) to directly deliver streaming data to Apache Iceberg Tables in Amazon S3. With this feature, you can route records from a single stream into different Apache Iceberg Tables, and automatically apply insert, update, and delete operations to records in the Apache Iceberg Tables. This feature requires using the AWS Glue Data Catalog. \ No newline at end of file diff --git a/docs/mkdocs.yml b/docs/mkdocs.yml index edafb727d3d8..c1c16dc50096 100644 --- a/docs/mkdocs.yml +++ b/docs/mkdocs.yml @@ -59,6 +59,8 @@ nav: - Starrocks: https://docs.starrocks.io/en-us/latest/data_source/catalog/iceberg_catalog - Amazon Athena: https://docs.aws.amazon.com/athena/latest/ug/querying-iceberg.html - Amazon EMR: https://docs.aws.amazon.com/emr/latest/ReleaseGuide/emr-iceberg-use-cluster.html + - Amazon Data Firehose: https://docs.aws.amazon.com/firehose/latest/dev/apache-iceberg-destination.html + - Amazon Redshift: https://docs.aws.amazon.com/redshift/latest/dg/querying-iceberg.html - Google BigQuery: https://cloud.google.com/bigquery/docs/iceberg-tables - Snowflake: https://docs.snowflake.com/en/user-guide/tables-iceberg - Impala: https://impala.apache.org/docs/build/html/topics/impala_iceberg.html From 90163ae5d85de0e3413acd75affb6416f2933e4d Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 24 Sep 2024 10:26:55 -0600 Subject: [PATCH 0716/1019] API, AWS: Retry S3InputStream reads (#10433) Co-authored-by: Jack Ye Co-authored-by: Xiaoxuan Li --- .../apache/iceberg/aws/s3/S3InputStream.java | 71 ++++-- .../aws/s3/TestFlakyS3InputStream.java | 206 ++++++++++++++++++ .../iceberg/aws/s3/TestS3InputStream.java | 24 +- build.gradle | 2 + gradle/libs.versions.toml | 2 + 5 files changed, 285 insertions(+), 20 deletions(-) create mode 100644 aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.java 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 f442a0f04a1c..74e602a27378 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 @@ -18,9 +18,15 @@ */ package org.apache.iceberg.aws.s3; +import dev.failsafe.Failsafe; +import dev.failsafe.FailsafeException; +import dev.failsafe.RetryPolicy; import java.io.IOException; import java.io.InputStream; +import java.net.SocketException; +import java.net.SocketTimeoutException; import java.util.Arrays; +import javax.net.ssl.SSLException; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.io.FileIOMetricsContext; import org.apache.iceberg.io.IOUtil; @@ -31,6 +37,7 @@ import org.apache.iceberg.metrics.MetricsContext.Unit; 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; import org.apache.iceberg.relocated.com.google.common.io.ByteStreams; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,6 +64,14 @@ class S3InputStream extends SeekableInputStream implements RangeReadable { private final Counter readOperations; private int skipSize = 1024 * 1024; + private RetryPolicy retryPolicy = + RetryPolicy.builder() + .handle( + ImmutableList.of( + SSLException.class, SocketTimeoutException.class, SocketException.class)) + .onFailure(failure -> openStream(true)) + .withMaxRetries(3) + .build(); S3InputStream(S3Client s3, S3URI location) { this(s3, location, new S3FileIOProperties(), MetricsContext.nullMetrics()); @@ -92,13 +107,21 @@ public void seek(long newPos) { public int read() throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); + try { + int bytesRead = Failsafe.with(retryPolicy).get(() -> stream.read()); + pos += 1; + next += 1; + readBytes.increment(); + readOperations.increment(); + + return bytesRead; + } catch (FailsafeException ex) { + if (ex.getCause() instanceof IOException) { + throw (IOException) ex.getCause(); + } - pos += 1; - next += 1; - readBytes.increment(); - readOperations.increment(); - - return stream.read(); + throw ex; + } } @Override @@ -106,13 +129,21 @@ public int read(byte[] b, int off, int len) throws IOException { Preconditions.checkState(!closed, "Cannot read: already closed"); positionStream(); - int bytesRead = stream.read(b, off, len); - pos += bytesRead; - next += bytesRead; - readBytes.increment(bytesRead); - readOperations.increment(); + try { + int bytesRead = Failsafe.with(retryPolicy).get(() -> stream.read(b, off, len)); + pos += bytesRead; + next += bytesRead; + readBytes.increment(bytesRead); + readOperations.increment(); + + return bytesRead; + } catch (FailsafeException ex) { + if (ex.getCause() instanceof IOException) { + throw (IOException) ex.getCause(); + } - return bytesRead; + throw ex; + } } @Override @@ -146,7 +177,7 @@ private InputStream readRange(String range) { public void close() throws IOException { super.close(); closed = true; - closeStream(); + closeStream(false); } private void positionStream() throws IOException { @@ -178,6 +209,10 @@ private void positionStream() throws IOException { } private void openStream() throws IOException { + openStream(false); + } + + private void openStream(boolean closeQuietly) throws IOException { GetObjectRequest.Builder requestBuilder = GetObjectRequest.builder() .bucket(location.bucket()) @@ -186,7 +221,7 @@ private void openStream() throws IOException { S3RequestUtil.configureEncryption(s3FileIOProperties, requestBuilder); - closeStream(); + closeStream(closeQuietly); try { stream = s3.getObject(requestBuilder.build(), ResponseTransformer.toInputStream()); @@ -195,7 +230,7 @@ private void openStream() throws IOException { } } - private void closeStream() throws IOException { + 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 // call abort() so we don't read the remaining data with the Apache HTTP client @@ -203,6 +238,12 @@ private void closeStream() throws IOException { try { stream.close(); } catch (IOException e) { + if (closeQuietly) { + stream = null; + LOG.warn("An error occurred while closing the stream", e); + return; + } + // the Apache HTTP client will throw a ConnectionClosedException // when closing an aborted stream, which is expected if (!e.getClass().getSimpleName().equals("ConnectionClosedException")) { 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 new file mode 100644 index 000000000000..08d14512cdc7 --- /dev/null +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestFlakyS3InputStream.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.aws.s3; + +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketTimeoutException; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Stream; +import javax.net.ssl.SSLException; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import software.amazon.awssdk.awscore.exception.AwsServiceException; +import software.amazon.awssdk.core.ResponseInputStream; +import software.amazon.awssdk.core.exception.SdkClientException; +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.GetObjectRequest; +import software.amazon.awssdk.services.s3.model.GetObjectResponse; +import software.amazon.awssdk.services.s3.model.HeadObjectRequest; +import software.amazon.awssdk.services.s3.model.HeadObjectResponse; +import software.amazon.awssdk.services.s3.model.PutObjectRequest; +import software.amazon.awssdk.services.s3.model.PutObjectResponse; + +public class TestFlakyS3InputStream extends TestS3InputStream { + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testReadWithFlakyStreamRetrySucceed(IOException exception) throws Exception { + testRead(flakyStreamClient(new AtomicInteger(3), exception)); + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testReadWithFlakyStreamExhaustedRetries(IOException exception) { + assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(5), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + } + + @ParameterizedTest + @MethodSource("nonRetryableExceptions") + public void testReadWithFlakyStreamNonRetryableException(IOException exception) { + assertThatThrownBy(() -> testRead(flakyStreamClient(new AtomicInteger(3), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testSeekWithFlakyStreamRetrySucceed(IOException exception) throws Exception { + testSeek(flakyStreamClient(new AtomicInteger(3), exception)); + } + + @ParameterizedTest + @MethodSource("retryableExceptions") + public void testSeekWithFlakyStreamExhaustedRetries(IOException exception) { + assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(5), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + } + + @ParameterizedTest + @MethodSource("nonRetryableExceptions") + public void testSeekWithFlakyStreamNonRetryableException(IOException exception) { + assertThatThrownBy(() -> testSeek(flakyStreamClient(new AtomicInteger(3), exception))) + .isInstanceOf(exception.getClass()) + .hasMessage(exception.getMessage()); + } + + private static Stream retryableExceptions() { + return Stream.of( + Arguments.of( + new SocketTimeoutException("socket timeout exception"), + new SSLException("some ssl exception"))); + } + + private static Stream nonRetryableExceptions() { + return Stream.of(Arguments.of(new IOException("some generic non-retryable IO exception"))); + } + + private S3ClientWrapper flakyStreamClient(AtomicInteger counter, IOException failure) { + S3ClientWrapper flakyClient = spy(new S3ClientWrapper(s3Client())); + doAnswer(invocation -> new FlakyInputStream(invocation.callRealMethod(), counter, failure)) + .when(flakyClient) + .getObject(any(GetObjectRequest.class), any(ResponseTransformer.class)); + return flakyClient; + } + + /** Wrapper for S3 client, used to mock the final class DefaultS3Client */ + public static class S3ClientWrapper implements S3Client { + + private final S3Client delegate; + + public S3ClientWrapper(S3Client delegate) { + this.delegate = delegate; + } + + @Override + public String serviceName() { + return delegate.serviceName(); + } + + @Override + public void close() { + delegate.close(); + } + + @Override + public ReturnT getObject( + GetObjectRequest getObjectRequest, + ResponseTransformer responseTransformer) + throws AwsServiceException, SdkClientException { + return delegate.getObject(getObjectRequest, responseTransformer); + } + + @Override + public HeadObjectResponse headObject(HeadObjectRequest headObjectRequest) + throws AwsServiceException, SdkClientException { + return delegate.headObject(headObjectRequest); + } + + @Override + public PutObjectResponse putObject(PutObjectRequest putObjectRequest, RequestBody requestBody) + throws AwsServiceException, SdkClientException { + return delegate.putObject(putObjectRequest, requestBody); + } + + @Override + public CreateBucketResponse createBucket(CreateBucketRequest createBucketRequest) + throws AwsServiceException, SdkClientException { + return delegate.createBucket(createBucketRequest); + } + } + + static class FlakyInputStream extends InputStream { + private final ResponseInputStream delegate; + private final AtomicInteger counter; + private final int round; + private final IOException exception; + + FlakyInputStream(Object invocationResponse, AtomicInteger counter, IOException exception) { + this.delegate = (ResponseInputStream) invocationResponse; + this.counter = counter; + this.round = counter.get(); + this.exception = exception; + } + + private void checkCounter() throws IOException { + // for every round of n invocations, only the last call succeeds + if (counter.decrementAndGet() == 0) { + counter.set(round); + } else { + throw exception; + } + } + + @Override + public int read() throws IOException { + checkCounter(); + return delegate.read(); + } + + @Override + public int read(byte[] b) throws IOException { + checkCounter(); + return delegate.read(b); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + checkCounter(); + return delegate.read(b, off, len); + } + + @Override + public void close() throws IOException { + delegate.close(); + } + } +} 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 ed71e259a26c..0e3f8b2136a6 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 @@ -54,16 +54,18 @@ public void before() { @Test public void testRead() throws Exception { + testRead(s3); + } + + protected void testRead(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/read.dat"); int dataSize = 1024 * 1024 * 10; byte[] data = randomData(dataSize); writeS3Data(uri, data); - try (SeekableInputStream in = new S3InputStream(s3, uri)) { + try (SeekableInputStream in = new S3InputStream(s3Client, uri)) { int readSize = 1024; - byte[] actual = new byte[readSize]; - readAndCheck(in, in.getPos(), readSize, data, false); readAndCheck(in, in.getPos(), readSize, data, true); @@ -111,6 +113,10 @@ private void readAndCheck( @Test public void testRangeRead() throws Exception { + testRangeRead(s3); + } + + protected void testRangeRead(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/range-read.dat"); int dataSize = 1024 * 1024 * 10; byte[] expected = randomData(dataSize); @@ -122,7 +128,7 @@ public void testRangeRead() throws Exception { writeS3Data(uri, expected); - try (RangeReadable in = new S3InputStream(s3, uri)) { + try (RangeReadable in = new S3InputStream(s3Client, uri)) { // first 1k position = 0; offset = 0; @@ -163,12 +169,16 @@ public void testClose() throws Exception { @Test public void testSeek() throws Exception { + testSeek(s3); + } + + protected void testSeek(S3Client s3Client) throws Exception { S3URI uri = new S3URI("s3://bucket/path/to/seek.dat"); byte[] expected = randomData(1024 * 1024); writeS3Data(uri, expected); - try (SeekableInputStream in = new S3InputStream(s3, uri)) { + try (SeekableInputStream in = new S3InputStream(s3Client, uri)) { in.seek(expected.length / 2); byte[] actual = new byte[expected.length / 2]; IOUtil.readFully(in, actual, 0, expected.length / 2); @@ -200,4 +210,8 @@ private void createBucket(String bucketName) { // don't do anything } } + + protected S3Client s3Client() { + return s3; + } } diff --git a/build.gradle b/build.gradle index 02758e2a793b..620641a21f92 100644 --- a/build.gradle +++ b/build.gradle @@ -347,6 +347,7 @@ 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' @@ -462,6 +463,7 @@ project(':iceberg-aws') { annotationProcessor libs.immutables.value compileOnly libs.immutables.value implementation libs.caffeine + implementation libs.failsafe implementation platform(libs.jackson.bom) implementation libs.jackson.core implementation libs.jackson.databind diff --git a/gradle/libs.versions.toml b/gradle/libs.versions.toml index 34471c2c4b75..fad4d49a1e62 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -39,6 +39,7 @@ delta-standalone = "3.2.0" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.31.0" +failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} flink119 = { strictly = "1.19.0"} @@ -109,6 +110,7 @@ calcite-druid = { module = "org.apache.calcite:calcite-druid", version.ref = "ca datasketches = { module = "org.apache.datasketches:datasketches-java", version.ref = "datasketches" } delta-standalone = { module = "io.delta:delta-standalone_2.12", version.ref = "delta-standalone" } errorprone-annotations = { module = "com.google.errorprone:error_prone_annotations", version.ref = "errorprone-annotations" } +failsafe = { module = "dev.failsafe:failsafe", version.ref = "failsafe"} findbugs-jsr305 = { module = "com.google.code.findbugs:jsr305", version.ref = "findbugs-jsr305" } flink118-avro = { module = "org.apache.flink:flink-avro", version.ref = "flink118" } flink118-connector-base = { module = "org.apache.flink:flink-connector-base", version.ref = "flink118" } From a18cda0f54f68a71f377e58a42ce9dd92ffcbb46 Mon Sep 17 00:00:00 2001 From: Laurent Goujon Date: Wed, 25 Sep 2024 02:13:09 -0700 Subject: [PATCH 0717/1019] AWS: Fix AWS doc URL (#11198) Fix AWS doc URL in S3FileIOProperties to point to the international version instead of the Chinese version. --- .../main/java/org/apache/iceberg/aws/s3/S3FileIOProperties.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3414c9df7046..343540d87f95 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 @@ -283,7 +283,7 @@ public class S3FileIOProperties implements Serializable { * catalog property. After set, x-amz-storage-class header will be set to this property * *

    For more details, see - * https://docs.aws.amazon.com/zh_cn/AmazonS3/latest/userguide/storage-class-intro.html + * https://docs.aws.amazon.com/AmazonS3/latest/userguide/storage-class-intro.html * *

    Example: s3.write.storage-class=INTELLIGENT_TIERING */ From 0f819a8898964cadeddd2e6bbd689c6b43147f9b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 25 Sep 2024 11:13:39 +0200 Subject: [PATCH 0718/1019] Build: Bump mkdocs-macros-plugin from 1.0.5 to 1.2.0 (#11189) Bumps [mkdocs-macros-plugin](https://github.com/fralau/mkdocs_macros_plugin) from 1.0.5 to 1.2.0. - [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.0.5...v1.2.0) --- 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 377fdc43fef2..38c2baa1eb02 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.0.5 +mkdocs-macros-plugin==1.2.0 mkdocs-material==9.5.34 mkdocs-material-extensions==1.3.1 mkdocs-monorepo-plugin @ git+https://github.com/bitsondatadev/mkdocs-monorepo-plugin@url-fix From dff2c57d99da4578f4304f142f0456a04da58adf Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 25 Sep 2024 08:30:06 -0700 Subject: [PATCH 0719/1019] Core: Add rewritten delete files to write results (#11203) --- .palantir/revapi.yml | 4 +++ .../iceberg/io/BasePositionDeltaWriter.java | 1 + .../apache/iceberg/io/DeleteWriteResult.java | 18 +++++++++++++ .../org/apache/iceberg/io/WriteResult.java | 27 ++++++++++++++++--- 4 files changed, 47 insertions(+), 3 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 9b8017f0beec..05ec59226fdd 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -1088,6 +1088,10 @@ acceptedBreaks: old: "class org.apache.iceberg.GenericManifestFile" new: "class org.apache.iceberg.GenericManifestFile" justification: "Serialization across versions is not supported" + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.io.WriteResult" + new: "class org.apache.iceberg.io.WriteResult" + justification: "Serialization across versions is not supported" - code: "java.class.removed" old: "enum org.apache.iceberg.BaseMetastoreTableOperations.CommitStatus" justification: "Removing deprecated code" diff --git a/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java b/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java index e098729ba20d..859d339bd993 100644 --- a/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/BasePositionDeltaWriter.java @@ -81,6 +81,7 @@ public WriteResult result() { .addDataFiles(dataFiles()) .addDeleteFiles(deleteWriteResult.deleteFiles()) .addReferencedDataFiles(deleteWriteResult.referencedDataFiles()) + .addRewrittenDeleteFiles(deleteWriteResult.rewrittenDeleteFiles()) .build(); } diff --git a/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java b/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java index 5c8768dcf16e..98ac76bc8060 100644 --- a/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java +++ b/core/src/main/java/org/apache/iceberg/io/DeleteWriteResult.java @@ -32,25 +32,39 @@ public class DeleteWriteResult { private final List deleteFiles; private final CharSequenceSet referencedDataFiles; + private final List rewrittenDeleteFiles; public DeleteWriteResult(DeleteFile deleteFile) { this.deleteFiles = Collections.singletonList(deleteFile); this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Collections.emptyList(); } public DeleteWriteResult(DeleteFile deleteFile, CharSequenceSet referencedDataFiles) { this.deleteFiles = Collections.singletonList(deleteFile); this.referencedDataFiles = referencedDataFiles; + this.rewrittenDeleteFiles = Collections.emptyList(); } public DeleteWriteResult(List deleteFiles) { this.deleteFiles = deleteFiles; this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Collections.emptyList(); } public DeleteWriteResult(List deleteFiles, CharSequenceSet referencedDataFiles) { this.deleteFiles = deleteFiles; this.referencedDataFiles = referencedDataFiles; + this.rewrittenDeleteFiles = Collections.emptyList(); + } + + public DeleteWriteResult( + List deleteFiles, + CharSequenceSet referencedDataFiles, + List rewrittenDeleteFiles) { + this.deleteFiles = deleteFiles; + this.referencedDataFiles = referencedDataFiles; + this.rewrittenDeleteFiles = rewrittenDeleteFiles; } public List deleteFiles() { @@ -64,4 +78,8 @@ public CharSequenceSet referencedDataFiles() { public boolean referencesDataFiles() { return referencedDataFiles != null && !referencedDataFiles.isEmpty(); } + + public List rewrittenDeleteFiles() { + return rewrittenDeleteFiles; + } } diff --git a/core/src/main/java/org/apache/iceberg/io/WriteResult.java b/core/src/main/java/org/apache/iceberg/io/WriteResult.java index c73b09f46c53..39efaec7d34a 100644 --- a/core/src/main/java/org/apache/iceberg/io/WriteResult.java +++ b/core/src/main/java/org/apache/iceberg/io/WriteResult.java @@ -31,12 +31,17 @@ public class WriteResult implements Serializable { private DataFile[] dataFiles; private DeleteFile[] deleteFiles; private CharSequence[] referencedDataFiles; + private DeleteFile[] rewrittenDeleteFiles; private WriteResult( - List dataFiles, List deleteFiles, CharSequenceSet referencedDataFiles) { + List dataFiles, + List deleteFiles, + CharSequenceSet referencedDataFiles, + List rewrittenDeleteFiles) { this.dataFiles = dataFiles.toArray(new DataFile[0]); this.deleteFiles = deleteFiles.toArray(new DeleteFile[0]); this.referencedDataFiles = referencedDataFiles.toArray(new CharSequence[0]); + this.rewrittenDeleteFiles = rewrittenDeleteFiles.toArray(new DeleteFile[0]); } public DataFile[] dataFiles() { @@ -51,6 +56,10 @@ public CharSequence[] referencedDataFiles() { return referencedDataFiles; } + public DeleteFile[] rewrittenDeleteFiles() { + return rewrittenDeleteFiles; + } + public static Builder builder() { return new Builder(); } @@ -59,18 +68,20 @@ public static class Builder { private final List dataFiles; private final List deleteFiles; private final CharSequenceSet referencedDataFiles; + private final List rewrittenDeleteFiles; private Builder() { this.dataFiles = Lists.newArrayList(); this.deleteFiles = Lists.newArrayList(); this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Lists.newArrayList(); } public Builder add(WriteResult result) { addDataFiles(result.dataFiles); addDeleteFiles(result.deleteFiles); addReferencedDataFiles(result.referencedDataFiles); - + addRewrittenDeleteFiles(result.rewrittenDeleteFiles); return this; } @@ -109,8 +120,18 @@ public Builder addReferencedDataFiles(Iterable files) { return this; } + public Builder addRewrittenDeleteFiles(DeleteFile... files) { + Collections.addAll(rewrittenDeleteFiles, files); + return this; + } + + public Builder addRewrittenDeleteFiles(Iterable files) { + Iterables.addAll(rewrittenDeleteFiles, files); + return this; + } + public WriteResult build() { - return new WriteResult(dataFiles, deleteFiles, referencedDataFiles); + return new WriteResult(dataFiles, deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } } } From 052235dba9686de122b14414c8184680945b97c4 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 25 Sep 2024 08:30:56 -0700 Subject: [PATCH 0720/1019] Core: Support iterating over positions in PositionDeleteIndex (#11202) --- .../deletes/BitmapPositionDeleteIndex.java | 6 ++ .../iceberg/deletes/PositionDeleteIndex.java | 13 ++++ .../TestBitmapPositionDeleteIndex.java | 72 +++++++++++++++++++ 3 files changed, 91 insertions(+) create mode 100644 core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java 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 72f1e00e498a..7503d0d83f5c 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.deletes; +import java.util.function.LongConsumer; import org.roaringbitmap.longlong.Roaring64Bitmap; class BitmapPositionDeleteIndex implements PositionDeleteIndex { @@ -50,4 +51,9 @@ public boolean isDeleted(long position) { public boolean isEmpty() { return roaring64Bitmap.isEmpty(); } + + @Override + public void forEach(LongConsumer consumer) { + roaring64Bitmap.forEach(consumer::accept); + } } 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 be05875aeb2a..27c15749add3 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,8 @@ */ package org.apache.iceberg.deletes; +import java.util.function.LongConsumer; + public interface PositionDeleteIndex { /** * Set a deleted row position. @@ -50,6 +52,17 @@ default boolean isNotEmpty() { return !isEmpty(); } + /** + * Traverses all positions in the index in ascending order, applying the provided consumer. + * + * @param consumer a consumer for the positions + */ + default void forEach(LongConsumer consumer) { + if (isNotEmpty()) { + throw new UnsupportedOperationException(getClass().getName() + " does not support forEach"); + } + } + /** 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 new file mode 100644 index 000000000000..279c5b8d1630 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.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.deletes; + +import static org.assertj.core.api.Assertions.assertThat; + +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.junit.jupiter.api.Test; + +public class TestBitmapPositionDeleteIndex { + + @Test + public void testForEach() { + long pos1 = 10L; // Container 0 (high bits = 0) + long pos2 = 1L << 33; // Container 1 (high bits = 1) + long pos3 = pos2 + 1; // Container 1 (high bits = 1) + long pos4 = 2L << 33; // Container 2 (high bits = 2) + long pos5 = pos4 + 1; // Container 2 (high bits = 2) + long pos6 = 3L << 33; // Container 3 (high bits = 3) + + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + + // add in any order + index.delete(pos1); + index.delete(pos6); + index.delete(pos2); + index.delete(pos3); + index.delete(pos5); + index.delete(pos4); + + // output must be sorted in ascending order across containers + List positions = collect(index); + assertThat(positions).containsExactly(pos1, pos2, pos3, pos4, pos5, pos6); + } + + @Test + public void testForEachEmptyBitmapIndex() { + PositionDeleteIndex index = new BitmapPositionDeleteIndex(); + List positions = collect(index); + assertThat(positions).isEmpty(); + } + + @Test + public void testForEachEmptyIndex() { + PositionDeleteIndex index = PositionDeleteIndex.empty(); + List positions = collect(index); + assertThat(positions).isEmpty(); + } + + private List collect(PositionDeleteIndex index) { + List positions = Lists.newArrayList(); + index.forEach(positions::add); + return positions; + } +} From 58ecc968d600f99da7176b71a153927039778b52 Mon Sep 17 00:00:00 2001 From: aleenamg21-1 <155471412+aleenamg21-1@users.noreply.github.com> Date: Thu, 26 Sep 2024 03:36:21 +0530 Subject: [PATCH 0721/1019] Spark: Added merge schema as spark configuration (#9640) --- .../iceberg/spark/SparkSQLProperties.java | 4 +++ .../apache/iceberg/spark/SparkWriteConf.java | 3 +- .../iceberg/spark/SparkWriteOptions.java | 1 - .../spark/source/TestDataFrameWriterV2.java | 35 +++++++++++++++++++ 4 files changed, 41 insertions(+), 2 deletions(-) diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java index 77ae796ffb76..9130e63ba97e 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkSQLProperties.java @@ -87,6 +87,10 @@ private SparkSQLProperties() {} "spark.sql.iceberg.executor-cache.max-total-size"; public static final long EXECUTOR_CACHE_MAX_TOTAL_SIZE_DEFAULT = 128 * 1024 * 1024; // 128 MB + // Controls whether to merge schema during write operation + public static final String MERGE_SCHEMA = "spark.sql.iceberg.merge-schema"; + public static final boolean MERGE_SCHEMA_DEFAULT = false; + 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; 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 07393a67fe31..2c8c26d80977 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 @@ -141,7 +141,8 @@ public boolean mergeSchema() { .booleanConf() .option(SparkWriteOptions.MERGE_SCHEMA) .option(SparkWriteOptions.SPARK_MERGE_SCHEMA) - .defaultValue(SparkWriteOptions.MERGE_SCHEMA_DEFAULT) + .sessionConf(SparkSQLProperties.MERGE_SCHEMA) + .defaultValue(SparkSQLProperties.MERGE_SCHEMA_DEFAULT) .parse(); } diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java index d9c4f66b192b..33db70bae587 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/SparkWriteOptions.java @@ -68,7 +68,6 @@ private SparkWriteOptions() {} public static final String MERGE_SCHEMA = "merge-schema"; public static final String SPARK_MERGE_SCHEMA = "mergeSchema"; - public static final boolean MERGE_SCHEMA_DEFAULT = false; // Identifies snapshot from which to start validating conflicting changes public static final String VALIDATE_FROM_SNAPSHOT_ID = "validate-from-snapshot-id"; 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 ffdaa8b477cb..1c87886241bf 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 @@ -209,4 +209,39 @@ public void testWriteWithCaseSensitiveOption() throws NoSuchTableException, Pars fields = Spark3Util.loadIcebergTable(sparkSession, tableName).schema().asStruct().fields(); assertThat(fields).hasSize(4); } + + @TestTemplate + public void testMergeSchemaSparkConfiguration() throws Exception { + sql( + "ALTER TABLE %s SET TBLPROPERTIES ('%s'='true')", + tableName, TableProperties.SPARK_WRITE_ACCEPT_ANY_SCHEMA); + Dataset twoColDF = + jsonToDF( + "id bigint, data string", + "{ \"id\": 1, \"data\": \"a\" }", + "{ \"id\": 2, \"data\": \"b\" }"); + + twoColDF.writeTo(tableName).append(); + + assertEquals( + "Should have initial 2-column rows", + ImmutableList.of(row(1L, "a"), row(2L, "b")), + sql("select * from %s order by id", tableName)); + spark.conf().set("spark.sql.iceberg.merge-schema", "true"); + Dataset threeColDF = + jsonToDF( + "id bigint, data string, salary float", + "{ \"id\": 3, \"data\": \"c\", \"salary\": 120000.34 }", + "{ \"id\": 4, \"data\": \"d\", \"salary\": 140000.56 }"); + + threeColDF.writeTo(tableName).append(); + assertEquals( + "Should have 3-column rows", + ImmutableList.of( + row(1L, "a", null), + row(2L, "b", null), + row(3L, "c", 120000.34F), + row(4L, "d", 140000.56F)), + sql("select * from %s order by id", tableName)); + } } From 61ce5cb402f2da2b519aee66744535776d3d4b88 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 25 Sep 2024 16:22:42 -0700 Subject: [PATCH 0722/1019] Core: Support merging in PositionDeleteIndex (#11208) --- .../deletes/BitmapPositionDeleteIndex.java | 9 ++++ .../iceberg/deletes/PositionDeleteIndex.java | 9 ++++ .../deletes/PositionDeleteIndexUtil.java | 14 +------ .../TestBitmapPositionDeleteIndex.java | 41 +++++++++++++++++++ 4 files changed, 60 insertions(+), 13 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 7503d0d83f5c..3a044878562a 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -42,6 +42,15 @@ public void delete(long posStart, long posEnd) { roaring64Bitmap.addRange(posStart, posEnd); } + @Override + public void merge(PositionDeleteIndex that) { + if (that instanceof BitmapPositionDeleteIndex) { + merge((BitmapPositionDeleteIndex) that); + } else { + that.forEach(this::delete); + } + } + @Override public boolean isDeleted(long position) { return roaring64Bitmap.contains(position); 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 27c15749add3..c0086fe6aa2e 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java @@ -36,6 +36,15 @@ public interface PositionDeleteIndex { */ void delete(long posStart, long posEnd); + /** + * Adds positions from the other index, modifying this index in place. + * + * @param that the other index to merge + */ + default void merge(PositionDeleteIndex that) { + that.forEach(this::delete); + } + /** * Checks whether a row at the position is deleted. * diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java index 0c3bff28ee6b..7601232ad24f 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndexUtil.java @@ -18,25 +18,13 @@ */ package org.apache.iceberg.deletes; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; - public class PositionDeleteIndexUtil { private PositionDeleteIndexUtil() {} public static PositionDeleteIndex merge(Iterable indexes) { BitmapPositionDeleteIndex result = new BitmapPositionDeleteIndex(); - - for (PositionDeleteIndex index : indexes) { - if (index.isNotEmpty()) { - Preconditions.checkArgument( - index instanceof BitmapPositionDeleteIndex, - "Can merge only bitmap-based indexes, got %s", - index.getClass().getName()); - result.merge((BitmapPositionDeleteIndex) index); - } - } - + indexes.forEach(result::merge); return result; } } 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 279c5b8d1630..c8fc723deb9e 100644 --- a/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java +++ b/core/src/test/java/org/apache/iceberg/deletes/TestBitmapPositionDeleteIndex.java @@ -64,6 +64,47 @@ public void testForEachEmptyIndex() { assertThat(positions).isEmpty(); } + @Test + public void testMergeBitmapIndexWithNonEmpty() { + long pos1 = 10L; // Container 0 (high bits = 0) + long pos2 = 1L << 33; // Container 1 (high bits = 1) + long pos3 = pos2 + 1; // Container 1 (high bits = 1) + long pos4 = 2L << 33; // Container 2 (high bits = 2) + + BitmapPositionDeleteIndex index1 = new BitmapPositionDeleteIndex(); + index1.delete(pos2); + index1.delete(pos1); + + BitmapPositionDeleteIndex index2 = new BitmapPositionDeleteIndex(); + index2.delete(pos4); + index2.delete(pos3); + + index1.merge(index2); + + // output must be sorted in ascending order across containers + List positions = collect(index1); + assertThat(positions).containsExactly(pos1, pos2, pos3, pos4); + } + + @Test + public void testMergeBitmapIndexWithEmpty() { + long pos1 = 10L; // Container 0 (high bits = 0) + long pos2 = 1L << 33; // Container 1 (high bits = 1) + long pos3 = pos2 + 1; // Container 1 (high bits = 1) + long pos4 = 2L << 33; // Container 2 (high bits = 2) + + BitmapPositionDeleteIndex index = new BitmapPositionDeleteIndex(); + index.delete(pos2); + index.delete(pos1); + index.delete(pos3); + index.delete(pos4); + index.merge(PositionDeleteIndex.empty()); + + // output must be sorted in ascending order across containers + List positions = collect(index); + assertThat(positions).containsExactly(pos1, pos2, pos3, pos4); + } + private List collect(PositionDeleteIndex index) { List positions = Lists.newArrayList(); index.forEach(positions::add); From ededae3837605b3cc239895ad48b7f19e59963b5 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 25 Sep 2024 17:34:30 -0700 Subject: [PATCH 0723/1019] Core: Remove unused code for streaming position deletes (#11175) --- .../org/apache/iceberg/deletes/Deletes.java | 171 ++---------- .../iceberg/deletes/TestPositionFilter.java | 254 ------------------ 2 files changed, 19 insertions(+), 406 deletions(-) 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 cef57cd16726..a72e01613040 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -29,10 +29,7 @@ import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; -import org.apache.iceberg.io.CloseableGroup; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; -import org.apache.iceberg.io.FilterIterator; 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; @@ -44,13 +41,9 @@ import org.apache.iceberg.util.SortedMerge; import org.apache.iceberg.util.StructLikeSet; import org.apache.iceberg.util.ThreadPools; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; public class Deletes { - private static final Logger LOG = LoggerFactory.getLogger(Deletes.class); - private static final Schema POSITION_DELETE_SCHEMA = new Schema(MetadataColumns.DELETE_FILE_PATH, MetadataColumns.DELETE_FILE_POS); @@ -192,6 +185,10 @@ public static PositionDeleteIndex toPositionIndex(CloseableIterable posDel } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated public static CloseableIterable streamingFilter( CloseableIterable rows, Function rowToPosition, @@ -199,20 +196,32 @@ public static CloseableIterable streamingFilter( 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) { - return new PositionStreamDeleteFilter<>(rows, rowToPosition, posDeletes, 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 markDeleted) { - return new PositionStreamDeleteMarker<>(rows, rowToPosition, posDeletes, markDeleted); + Consumer markRowDeleted) { + PositionDeleteIndex positionIndex = toPositionIndex(posDeletes); + Predicate isDeleted = row -> positionIndex.isDeleted(rowToPosition.apply(row)); + return markDeleted(rows, isDeleted, markRowDeleted); } public static CloseableIterable deletePositions( @@ -248,148 +257,6 @@ protected boolean shouldKeep(T row) { } } - private abstract static class PositionStreamDeleteIterable extends CloseableGroup - implements CloseableIterable { - private final CloseableIterable rows; - private final CloseableIterator deletePosIterator; - private final Function rowToPosition; - private long nextDeletePos; - - PositionStreamDeleteIterable( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable deletePositions) { - this.rows = rows; - this.rowToPosition = rowToPosition; - this.deletePosIterator = deletePositions.iterator(); - } - - @Override - public CloseableIterator iterator() { - CloseableIterator iter; - if (deletePosIterator.hasNext()) { - nextDeletePos = deletePosIterator.next(); - iter = applyDelete(rows.iterator(), deletePosIterator); - } else { - iter = rows.iterator(); - } - - addCloseable(iter); - addCloseable(deletePosIterator); - - return iter; - } - - boolean isDeleted(T row) { - long currentPos = rowToPosition.apply(row); - if (currentPos < nextDeletePos) { - return false; - } - - // consume delete positions until the next is past the current position - boolean isDeleted = currentPos == nextDeletePos; - while (deletePosIterator.hasNext() && nextDeletePos <= currentPos) { - this.nextDeletePos = deletePosIterator.next(); - if (!isDeleted && currentPos == nextDeletePos) { - // if any delete position matches the current position - isDeleted = true; - } - } - - return isDeleted; - } - - protected abstract CloseableIterator applyDelete( - CloseableIterator items, CloseableIterator deletePositions); - } - - private static class PositionStreamDeleteFilter extends PositionStreamDeleteIterable { - private final DeleteCounter counter; - - PositionStreamDeleteFilter( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable deletePositions, - DeleteCounter counter) { - super(rows, rowToPosition, deletePositions); - this.counter = counter; - } - - @Override - protected CloseableIterator applyDelete( - CloseableIterator items, CloseableIterator deletePositions) { - return new FilterIterator(items) { - @Override - protected boolean shouldKeep(T item) { - boolean deleted = isDeleted(item); - if (deleted) { - counter.increment(); - } - - return !deleted; - } - - @Override - public void close() { - try { - deletePositions.close(); - } catch (IOException e) { - LOG.warn("Error closing delete file", e); - } - super.close(); - } - }; - } - } - - private static class PositionStreamDeleteMarker extends PositionStreamDeleteIterable { - private final Consumer markDeleted; - - PositionStreamDeleteMarker( - CloseableIterable rows, - Function rowToPosition, - CloseableIterable deletePositions, - Consumer markDeleted) { - super(rows, rowToPosition, deletePositions); - this.markDeleted = markDeleted; - } - - @Override - protected CloseableIterator applyDelete( - CloseableIterator items, CloseableIterator deletePositions) { - - return new CloseableIterator() { - @Override - public void close() { - try { - deletePositions.close(); - } catch (IOException e) { - LOG.warn("Error closing delete file", e); - } - try { - items.close(); - } catch (IOException e) { - LOG.warn("Error closing data file", e); - } - } - - @Override - public boolean hasNext() { - return items.hasNext(); - } - - @Override - public T next() { - T row = items.next(); - if (isDeleted(row)) { - markDeleted.accept(row); - } - return row; - } - }; - } - } - private static class DataFileFilter extends Filter { private final CharSequence dataLocation; 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 8e35c8c9fc99..2a629b2dc2b3 100644 --- a/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java +++ b/core/src/test/java/org/apache/iceberg/deletes/TestPositionFilter.java @@ -20,20 +20,16 @@ import static org.assertj.core.api.Assertions.assertThat; -import java.io.IOException; -import java.util.Iterator; import java.util.List; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.ThreadPoolExecutor; -import java.util.concurrent.atomic.AtomicBoolean; 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; import org.apache.iceberg.io.CloseableIterable; -import org.apache.iceberg.io.CloseableIterator; 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; @@ -110,155 +106,6 @@ public void testPositionMerging() { .containsExactly(0L, 3L, 3L, 9L, 16L, 19L, 19L, 22L, 22L, 56L, 63L, 70L, 91L); } - @Test - public void testPositionStreamRowFilter() { - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a"), - Row.of(1L, "b"), - Row.of(2L, "c"), - Row.of(3L, "d"), - Row.of(4L, "e"), - Row.of(5L, "f"), - Row.of(6L, "g"), - Row.of(7L, "h"), - Row.of(8L, "i"), - Row.of(9L, "j"))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 3L, 4L, 7L, 9L)); - - CloseableIterable actual = - Deletes.streamingFilter(rows, row -> row.get(0, Long.class), deletes); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); - } - - @Test - public void testPositionStreamRowDeleteMarker() { - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a", false), - Row.of(1L, "b", false), - Row.of(2L, "c", false), - Row.of(3L, "d", false), - Row.of(4L, "e", false), - Row.of(5L, "f", false), - Row.of(6L, "g", false), - Row.of(7L, "h", false), - Row.of(8L, "i", false), - Row.of(9L, "j", false))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 3L, 4L, 7L, 9L)); - - CloseableIterable actual = - Deletes.streamingMarker( - rows, - row -> row.get(0, Long.class), /* row to position */ - deletes, - row -> row.set(2, true) /* delete marker */); - - assertThat(Iterables.transform(actual, row -> row.get(2, Boolean.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf( - Lists.newArrayList(true, false, false, true, true, false, false, true, false, true)); - } - - @Test - public void testPositionStreamRowFilterWithDuplicates() { - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a"), - Row.of(1L, "b"), - Row.of(2L, "c"), - Row.of(3L, "d"), - Row.of(4L, "e"), - Row.of(5L, "f"), - Row.of(6L, "g"), - Row.of(7L, "h"), - Row.of(8L, "i"), - Row.of(9L, "j"))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 0L, 0L, 3L, 4L, 7L, 7L, 9L, 9L, 9L)); - - CloseableIterable actual = - Deletes.streamingFilter(rows, row -> row.get(0, Long.class), deletes); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); - } - - @Test - public void testPositionStreamRowFilterWithRowGaps() { - // test the case where row position is greater than the delete position - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList(Row.of(2L, "c"), Row.of(3L, "d"), Row.of(5L, "f"), Row.of(6L, "g"))); - - CloseableIterable deletes = - CloseableIterable.withNoopClose(Lists.newArrayList(0L, 2L, 3L, 4L, 7L, 9L)); - - CloseableIterable actual = - Deletes.streamingFilter(rows, row -> row.get(0, Long.class), deletes); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(5L, 6L)); - } - - @Test - public void testCombinedPositionStreamRowFilter() { - CloseableIterable positionDeletes1 = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of("file_a.avro", 0L), - Row.of("file_a.avro", 3L), - Row.of("file_a.avro", 9L), - Row.of("file_b.avro", 5L), - Row.of("file_b.avro", 6L))); - - CloseableIterable positionDeletes2 = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of("file_a.avro", 3L), - Row.of("file_a.avro", 4L), - Row.of("file_a.avro", 7L), - Row.of("file_b.avro", 2L))); - - CloseableIterable rows = - CloseableIterable.withNoopClose( - Lists.newArrayList( - Row.of(0L, "a"), - Row.of(1L, "b"), - Row.of(2L, "c"), - Row.of(3L, "d"), - Row.of(4L, "e"), - Row.of(5L, "f"), - Row.of(6L, "g"), - Row.of(7L, "h"), - Row.of(8L, "i"), - Row.of(9L, "j"))); - - CloseableIterable actual = - Deletes.streamingFilter( - rows, - row -> row.get(0, Long.class), - Deletes.deletePositions( - "file_a.avro", ImmutableList.of(positionDeletes1, positionDeletes2))); - - assertThat(Iterables.transform(actual, row -> row.get(0, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); - } - @Test public void testPositionSetRowFilter() { CloseableIterable rows = @@ -341,105 +188,4 @@ public void testCombinedPositionSetRowFilter(ExecutorService executorService) { .as("Filter should produce expected rows") .containsExactlyElementsOf(Lists.newArrayList(1L, 2L, 5L, 6L, 8L)); } - - @Test - public void testClosePositionStreamRowDeleteMarker() { - List deletes = Lists.newArrayList(1L, 2L); - - List records = - Lists.newArrayList( - Row.of(29, "a", 1L), Row.of(43, "b", 2L), Row.of(61, "c", 3L), Row.of(89, "d", 4L)); - - CheckingClosableIterable data = new CheckingClosableIterable<>(records); - CheckingClosableIterable deletePositions = new CheckingClosableIterable<>(deletes); - - CloseableIterable posDeletesIterable = - Deletes.streamingFilter(data, row -> row.get(2, Long.class), deletePositions); - - // end iterator is always wrapped with FilterIterator - CloseableIterable eqDeletesIterable = - Deletes.filterDeleted(posDeletesIterable, i -> false, new DeleteCounter()); - List result = Lists.newArrayList(eqDeletesIterable.iterator()); - - // as first two records deleted, expect only last two records - assertThat(Iterables.transform(result, row -> row.get(2, Long.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(3L, 4L)); - - assertThat(data.isClosed).isTrue(); - assertThat(deletePositions.isClosed).isTrue(); - } - - @Test - public void testDeleteMarkerFileClosed() { - - List deletes = Lists.newArrayList(1L, 2L); - - List records = - Lists.newArrayList( - Row.of(29, "a", 1L, false), - Row.of(43, "b", 2L, false), - Row.of(61, "c", 3L, false), - Row.of(89, "d", 4L, false)); - - CheckingClosableIterable data = new CheckingClosableIterable<>(records); - CheckingClosableIterable deletePositions = new CheckingClosableIterable<>(deletes); - - CloseableIterable resultIterable = - Deletes.streamingMarker( - data, row -> row.get(2, Long.class), deletePositions, row -> row.set(3, true)); - - // end iterator is always wrapped with FilterIterator - CloseableIterable eqDeletesIterable = - Deletes.filterDeleted(resultIterable, i -> false, new DeleteCounter()); - List result = Lists.newArrayList(eqDeletesIterable.iterator()); - - // as first two records deleted, expect only those two records marked - assertThat(Iterables.transform(result, row -> row.get(3, Boolean.class))) - .as("Filter should produce expected rows") - .containsExactlyElementsOf(Lists.newArrayList(true, true, false, false)); - - assertThat(data.isClosed).isTrue(); - assertThat(deletePositions.isClosed).isTrue(); - } - - private static class CheckingClosableIterable implements CloseableIterable { - AtomicBoolean isClosed = new AtomicBoolean(false); - final Iterable iterable; - - CheckingClosableIterable(Iterable iterable) { - this.iterable = iterable; - } - - public boolean isClosed() { - return isClosed.get(); - } - - @Override - public void close() throws IOException { - isClosed.set(true); - } - - @Override - public CloseableIterator iterator() { - Iterator it = iterable.iterator(); - return new CloseableIterator() { - - @Override - public boolean hasNext() { - return it.hasNext(); - } - - @Override - public E next() { - return it.next(); - } - - @Override - public void close() { - isClosed.set(true); - } - }; - } - } } From f78b3dd8baa2838546f57f946a9205ba96a9248d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?JB=20Onofr=C3=A9?= Date: Thu, 26 Sep 2024 09:31:36 +0200 Subject: [PATCH 0724/1019] Build: Upgrade to Gradle 8.10.2 (#11212) --- 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 8e876e1c5571..fb602ee2af06 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=1541fa36599e12857140465f3c91a97409b4512501c26f9631fb113e392c5bd1 -distributionUrl=https\://services.gradle.org/distributions/gradle-8.10.1-bin.zip +distributionSha256Sum=31c55713e40233a8303827ceb42ca48a47267a0ad4bab9177123121e71524c26 +distributionUrl=https\://services.gradle.org/distributions/gradle-8.10.2-bin.zip networkTimeout=10000 validateDistributionUrl=true zipStoreBase=GRADLE_USER_HOME diff --git a/gradlew b/gradlew index 85a01a52e645..57ebc02755ab 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.1/gradle/wrapper/gradle-wrapper.jar + curl -o $APP_HOME/gradle/wrapper/gradle-wrapper.jar https://raw.githubusercontent.com/gradle/gradle/v8.10.2/gradle/wrapper/gradle-wrapper.jar fi # Use the maximum available, or set MAX_FD != -1 to use that value. From c2bbe8413952fd3f9664b39094c7d3ea009e3e8f Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 26 Sep 2024 21:44:49 +0530 Subject: [PATCH 0725/1019] Core: Add a util to compute partition stats (#11146) --- .../iceberg/PartitionStatsUtilBenchmark.java | 105 +++++ .../java/org/apache/iceberg/BaseScan.java | 17 + .../org/apache/iceberg/PartitionStats.java | 252 ++++++++++ .../apache/iceberg/PartitionStatsUtil.java | 136 ++++++ .../java/org/apache/iceberg/Partitioning.java | 10 + .../org/apache/iceberg/PartitionsTable.java | 13 +- .../apache/iceberg/FileGenerationUtil.java | 15 + .../iceberg/TestPartitionStatsUtil.java | 429 ++++++++++++++++++ 8 files changed, 965 insertions(+), 12 deletions(-) create mode 100644 core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java create mode 100644 core/src/main/java/org/apache/iceberg/PartitionStats.java create mode 100644 core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java create mode 100644 core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java diff --git a/core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java b/core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java new file mode 100644 index 000000000000..539494e34735 --- /dev/null +++ b/core/src/jmh/java/org/apache/iceberg/PartitionStatsUtilBenchmark.java @@ -0,0 +1,105 @@ +/* + * 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.util.Collection; +import java.util.concurrent.TimeUnit; +import org.apache.iceberg.hadoop.HadoopTables; +import org.apache.iceberg.types.Types; +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.TearDown; +import org.openjdk.jmh.annotations.Threads; +import org.openjdk.jmh.annotations.Timeout; +import org.openjdk.jmh.annotations.Warmup; + +@Fork(1) +@State(Scope.Benchmark) +@Warmup(iterations = 2) +@Measurement(iterations = 5) +@Timeout(time = 1000, timeUnit = TimeUnit.HOURS) +@BenchmarkMode(Mode.SingleShotTime) +public class PartitionStatsUtilBenchmark { + + private static final Schema SCHEMA = + new Schema( + required(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + private static final PartitionSpec SPEC = PartitionSpec.builderFor(SCHEMA).identity("c1").build(); + + // Create 10k manifests + private static final int MANIFEST_COUNTER = 10000; + + // each manifest with 100 partition values + private static final int PARTITION_PER_MANIFEST = 100; + + // 20 data files per partition, which results in 2k data files per manifest + private static final int DATA_FILES_PER_PARTITION_COUNT = 20; + + private static final HadoopTables TABLES = new HadoopTables(); + + private static final String TABLE_IDENT = "tbl"; + + private Table table; + + @Setup + public void setupBenchmark() { + this.table = TABLES.create(SCHEMA, SPEC, TABLE_IDENT); + + for (int manifestCount = 0; manifestCount < MANIFEST_COUNTER; manifestCount++) { + AppendFiles appendFiles = table.newFastAppend(); + + for (int partition = 0; partition < PARTITION_PER_MANIFEST; partition++) { + StructLike partitionData = TestHelpers.Row.of(partition); + for (int fileOrdinal = 0; fileOrdinal < DATA_FILES_PER_PARTITION_COUNT; fileOrdinal++) { + appendFiles.appendFile(FileGenerationUtil.generateDataFile(table, partitionData)); + } + } + + appendFiles.commit(); + } + } + + @TearDown + public void tearDownBenchmark() { + TABLES.dropTable(TABLE_IDENT); + } + + @Benchmark + @Threads(1) + public void benchmarkPartitionStats() { + Collection partitionStats = + PartitionStatsUtil.computeStats(table, table.currentSnapshot()); + assertThat(partitionStats).hasSize(PARTITION_PER_MANIFEST); + + PartitionStatsUtil.sortStats(partitionStats, Partitioning.partitionType(table)); + } +} diff --git a/core/src/main/java/org/apache/iceberg/BaseScan.java b/core/src/main/java/org/apache/iceberg/BaseScan.java index 8c309cf69e6d..804df01d31ba 100644 --- a/core/src/main/java/org/apache/iceberg/BaseScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseScan.java @@ -289,4 +289,21 @@ private static Schema lazyColumnProjection(TableScanContext context, Schema sche public ThisT metricsReporter(MetricsReporter reporter) { return newRefinedScan(table, schema, context.reportWith(reporter)); } + + /** + * Retrieves a list of column names based on the type of manifest content provided. + * + * @param content the manifest content type to scan. + * @return a list of column names corresponding to the specified manifest content type. + */ + static List scanColumns(ManifestContent content) { + switch (content) { + case DATA: + return BaseScan.SCAN_COLUMNS; + case DELETES: + return BaseScan.DELETE_SCAN_COLUMNS; + default: + throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content); + } + } } diff --git a/core/src/main/java/org/apache/iceberg/PartitionStats.java b/core/src/main/java/org/apache/iceberg/PartitionStats.java new file mode 100644 index 000000000000..e4cbd1f6b9bd --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/PartitionStats.java @@ -0,0 +1,252 @@ +/* + * 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 PartitionStats implements StructLike { + + private static final int STATS_COUNT = 12; + + private StructLike partition; + private int specId; + private long dataRecordCount; + private int dataFileCount; + private long totalDataFileSizeInBytes; + private long positionDeleteRecordCount; + private int positionDeleteFileCount; + private long equalityDeleteRecordCount; + private int equalityDeleteFileCount; + private long totalRecordCount; + private Long lastUpdatedAt; // null by default + private Long lastUpdatedSnapshotId; // null by default + + public PartitionStats(StructLike partition, int specId) { + this.partition = partition; + this.specId = specId; + } + + public StructLike partition() { + return partition; + } + + public int specId() { + return specId; + } + + public long dataRecordCount() { + return dataRecordCount; + } + + public int dataFileCount() { + return dataFileCount; + } + + public long totalDataFileSizeInBytes() { + return totalDataFileSizeInBytes; + } + + public long positionDeleteRecordCount() { + return positionDeleteRecordCount; + } + + public int positionDeleteFileCount() { + return positionDeleteFileCount; + } + + public long equalityDeleteRecordCount() { + return equalityDeleteRecordCount; + } + + public int equalityDeleteFileCount() { + return equalityDeleteFileCount; + } + + public long totalRecordCount() { + return totalRecordCount; + } + + public Long lastUpdatedAt() { + return lastUpdatedAt; + } + + public Long lastUpdatedSnapshotId() { + return lastUpdatedSnapshotId; + } + + /** + * Updates the partition stats from the data/delete file. + * + * @param file the {@link ContentFile} from the manifest entry. + * @param snapshot the snapshot corresponding to the live entry. + */ + public void liveEntry(ContentFile file, Snapshot snapshot) { + Preconditions.checkArgument(specId == file.specId(), "Spec IDs must match"); + + switch (file.content()) { + case DATA: + this.dataRecordCount += file.recordCount(); + this.dataFileCount += 1; + this.totalDataFileSizeInBytes += file.fileSizeInBytes(); + break; + case POSITION_DELETES: + this.positionDeleteRecordCount += file.recordCount(); + this.positionDeleteFileCount += 1; + break; + case EQUALITY_DELETES: + this.equalityDeleteRecordCount += file.recordCount(); + this.equalityDeleteFileCount += 1; + break; + default: + throw new UnsupportedOperationException("Unsupported file content type: " + file.content()); + } + + if (snapshot != null) { + updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); + } + + // Note: Not computing the `TOTAL_RECORD_COUNT` for now as it needs scanning the data. + } + + /** + * Updates the modified time and snapshot ID for the deleted manifest entry. + * + * @param snapshot the snapshot corresponding to the deleted manifest entry. + */ + public void deletedEntry(Snapshot snapshot) { + if (snapshot != null) { + updateSnapshotInfo(snapshot.snapshotId(), snapshot.timestampMillis()); + } + } + + /** + * Appends statistics from given entry to current entry. + * + * @param entry the entry from which statistics will be sourced. + */ + public void appendStats(PartitionStats entry) { + Preconditions.checkArgument(specId == entry.specId(), "Spec IDs must match"); + + this.dataRecordCount += entry.dataRecordCount; + this.dataFileCount += entry.dataFileCount; + this.totalDataFileSizeInBytes += entry.totalDataFileSizeInBytes; + this.positionDeleteRecordCount += entry.positionDeleteRecordCount; + this.positionDeleteFileCount += entry.positionDeleteFileCount; + this.equalityDeleteRecordCount += entry.equalityDeleteRecordCount; + this.equalityDeleteFileCount += entry.equalityDeleteFileCount; + this.totalRecordCount += entry.totalRecordCount; + + if (entry.lastUpdatedAt != null) { + updateSnapshotInfo(entry.lastUpdatedSnapshotId, entry.lastUpdatedAt); + } + } + + private void updateSnapshotInfo(long snapshotId, long updatedAt) { + if (lastUpdatedAt == null || lastUpdatedAt < updatedAt) { + this.lastUpdatedAt = updatedAt; + this.lastUpdatedSnapshotId = snapshotId; + } + } + + @Override + public int size() { + return STATS_COUNT; + } + + @Override + public T get(int pos, Class javaClass) { + switch (pos) { + case 0: + return javaClass.cast(partition); + case 1: + return javaClass.cast(specId); + case 2: + return javaClass.cast(dataRecordCount); + case 3: + return javaClass.cast(dataFileCount); + case 4: + return javaClass.cast(totalDataFileSizeInBytes); + case 5: + return javaClass.cast(positionDeleteRecordCount); + case 6: + return javaClass.cast(positionDeleteFileCount); + case 7: + return javaClass.cast(equalityDeleteRecordCount); + case 8: + return javaClass.cast(equalityDeleteFileCount); + case 9: + return javaClass.cast(totalRecordCount); + case 10: + return javaClass.cast(lastUpdatedAt); + case 11: + return javaClass.cast(lastUpdatedSnapshotId); + default: + throw new UnsupportedOperationException("Unknown position: " + pos); + } + } + + @Override + public void set(int pos, T value) { + switch (pos) { + case 0: + this.partition = (StructLike) value; + break; + case 1: + this.specId = (int) value; + break; + case 2: + this.dataRecordCount = (long) value; + break; + case 3: + this.dataFileCount = (int) value; + break; + case 4: + this.totalDataFileSizeInBytes = (long) value; + break; + case 5: + // optional field as per spec, implementation initialize to 0 for counters + this.positionDeleteRecordCount = value == null ? 0L : (long) value; + break; + case 6: + // optional field as per spec, implementation initialize to 0 for counters + this.positionDeleteFileCount = value == null ? 0 : (int) value; + break; + case 7: + // optional field as per spec, implementation initialize to 0 for counters + this.equalityDeleteRecordCount = value == null ? 0L : (long) value; + break; + case 8: + // optional field as per spec, implementation initialize to 0 for counters + this.equalityDeleteFileCount = value == null ? 0 : (int) value; + break; + case 9: + // optional field as per spec, implementation initialize to 0 for counters + this.totalRecordCount = value == null ? 0L : (long) value; + break; + case 10: + this.lastUpdatedAt = (Long) value; + break; + case 11: + this.lastUpdatedSnapshotId = (Long) value; + break; + default: + throw new UnsupportedOperationException("Unknown position: " + pos); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java b/core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java new file mode 100644 index 000000000000..1fe4e6767fe6 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/PartitionStatsUtil.java @@ -0,0 +1,136 @@ +/* + * 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 java.io.IOException; +import java.io.UncheckedIOException; +import java.util.Collection; +import java.util.Comparator; +import java.util.List; +import java.util.Map; +import java.util.Queue; +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.Queues; +import org.apache.iceberg.types.Comparators; +import org.apache.iceberg.types.Types.StructType; +import org.apache.iceberg.util.PartitionMap; +import org.apache.iceberg.util.PartitionUtil; +import org.apache.iceberg.util.Tasks; +import org.apache.iceberg.util.ThreadPools; + +public class PartitionStatsUtil { + + private PartitionStatsUtil() {} + + /** + * Computes the partition stats for the given snapshot of the table. + * + * @param table the table for which partition stats to be computed. + * @param snapshot the snapshot for which partition stats is computed. + * @return the collection of {@link PartitionStats} + */ + public static Collection computeStats(Table table, Snapshot snapshot) { + Preconditions.checkArgument(table != null, "table cannot be null"); + Preconditions.checkArgument(Partitioning.isPartitioned(table), "table must be partitioned"); + Preconditions.checkArgument(snapshot != null, "snapshot cannot be null"); + + StructType partitionType = Partitioning.partitionType(table); + List manifests = snapshot.allManifests(table.io()); + Queue> statsByManifest = Queues.newConcurrentLinkedQueue(); + Tasks.foreach(manifests) + .stopOnFailure() + .throwFailureWhenFinished() + .executeWith(ThreadPools.getWorkerPool()) + .run(manifest -> statsByManifest.add(collectStats(table, manifest, partitionType))); + + return mergeStats(statsByManifest, table.specs()); + } + + /** + * Sorts the {@link PartitionStats} based on the partition data. + * + * @param stats collection of {@link PartitionStats} which needs to be sorted. + * @param partitionType unified partition schema. + * @return the list of {@link PartitionStats} + */ + public static List sortStats( + Collection stats, StructType partitionType) { + List entries = Lists.newArrayList(stats); + entries.sort(partitionStatsCmp(partitionType)); + return entries; + } + + private static Comparator partitionStatsCmp(StructType partitionType) { + return Comparator.comparing(PartitionStats::partition, Comparators.forType(partitionType)); + } + + private static PartitionMap collectStats( + Table table, ManifestFile manifest, StructType partitionType) { + try (ManifestReader reader = openManifest(table, manifest)) { + PartitionMap statsMap = PartitionMap.create(table.specs()); + int specId = manifest.partitionSpecId(); + PartitionSpec spec = table.specs().get(specId); + PartitionData keyTemplate = new PartitionData(partitionType); + + for (ManifestEntry entry : reader.entries()) { + ContentFile file = entry.file(); + StructLike coercedPartition = + PartitionUtil.coercePartition(partitionType, spec, file.partition()); + StructLike key = keyTemplate.copyFor(coercedPartition); + Snapshot snapshot = table.snapshot(entry.snapshotId()); + PartitionStats stats = + statsMap.computeIfAbsent(specId, key, () -> new PartitionStats(key, specId)); + if (entry.isLive()) { + stats.liveEntry(file, snapshot); + } else { + stats.deletedEntry(snapshot); + } + } + + return statsMap; + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + private static ManifestReader openManifest(Table table, ManifestFile manifest) { + List projection = BaseScan.scanColumns(manifest.content()); + return ManifestFiles.open(manifest, table.io()).select(projection); + } + + private static Collection mergeStats( + Queue> statsByManifest, Map specs) { + PartitionMap statsMap = PartitionMap.create(specs); + + for (PartitionMap stats : statsByManifest) { + stats.forEach( + (key, value) -> + statsMap.merge( + key, + value, + (existingEntry, newEntry) -> { + existingEntry.appendStats(newEntry); + return existingEntry; + })); + } + + return statsMap.values(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/Partitioning.java b/core/src/main/java/org/apache/iceberg/Partitioning.java index 7e4fcae333d8..832e0b59fe50 100644 --- a/core/src/main/java/org/apache/iceberg/Partitioning.java +++ b/core/src/main/java/org/apache/iceberg/Partitioning.java @@ -242,6 +242,16 @@ public static StructType partitionType(Table table) { return buildPartitionProjectionType("table partition", specs, allFieldIds(specs)); } + /** + * Checks if any of the specs in a table is partitioned. + * + * @param table the table to check. + * @return {@code true} if the table is partitioned, {@code false} otherwise. + */ + public static boolean isPartitioned(Table table) { + return table.specs().values().stream().anyMatch(PartitionSpec::isPartitioned); + } + private static StructType buildPartitionProjectionType( String typeName, Collection specs, Set projectedFieldIds) { diff --git a/core/src/main/java/org/apache/iceberg/PartitionsTable.java b/core/src/main/java/org/apache/iceberg/PartitionsTable.java index 5ff796e95827..6d0fc8c235f9 100644 --- a/core/src/main/java/org/apache/iceberg/PartitionsTable.java +++ b/core/src/main/java/org/apache/iceberg/PartitionsTable.java @@ -201,7 +201,7 @@ private static CloseableIterable> readEntries( return CloseableIterable.transform( ManifestFiles.open(manifest, table.io(), table.specs()) .caseSensitive(scan.isCaseSensitive()) - .select(scanColumns(manifest.content())) // don't select stats columns + .select(BaseScan.scanColumns(manifest.content())) // don't select stats columns .liveEntries(), t -> (ManifestEntry>) @@ -209,17 +209,6 @@ private static CloseableIterable> readEntries( t.copyWithoutStats()); } - private static List scanColumns(ManifestContent content) { - switch (content) { - case DATA: - return BaseScan.SCAN_COLUMNS; - case DELETES: - return BaseScan.DELETE_SCAN_COLUMNS; - default: - throw new UnsupportedOperationException("Cannot read unknown manifest type: " + content); - } - } - private static CloseableIterable filteredManifests( StaticTableScan scan, Table table, List manifestFilesList) { CloseableIterable manifestFiles = diff --git a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java index e48f23ff9a0b..f66496ae6624 100644 --- a/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java +++ b/core/src/test/java/org/apache/iceberg/FileGenerationUtil.java @@ -86,6 +86,21 @@ public static DeleteFile generatePositionDeleteFile(Table table, StructLike part .build(); } + public static DeleteFile generateEqualityDeleteFile(Table table, StructLike partition) { + PartitionSpec spec = table.spec(); + LocationProvider locations = table.locationProvider(); + String path = locations.newDataLocation(spec, partition, generateFileName()); + long fileSize = generateFileSize(); + return FileMetadata.deleteFileBuilder(spec) + .ofEqualityDeletes() + .withPartition(partition) + .withPath(path) + .withFileSizeInBytes(fileSize) + .withFormat(FileFormat.PARQUET) + .withRecordCount(generateRowCount()) + .build(); + } + public static DeleteFile generatePositionDeleteFile(Table table, DataFile dataFile) { PartitionSpec spec = table.spec(); StructLike partition = dataFile.partition(); diff --git a/core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java b/core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java new file mode 100644 index 000000000000..541fcd2ca22d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/TestPartitionStatsUtil.java @@ -0,0 +1,429 @@ +/* + * 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.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.List; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Types; +import org.assertj.core.groups.Tuple; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestPartitionStatsUtil { + private static final Schema SCHEMA = + new Schema( + optional(1, "c1", Types.IntegerType.get()), + optional(2, "c2", Types.StringType.get()), + optional(3, "c3", Types.StringType.get())); + + protected static final PartitionSpec SPEC = + PartitionSpec.builderFor(SCHEMA).identity("c2").identity("c3").build(); + + @TempDir public File temp; + + @Test + public void testPartitionStatsOnEmptyTable() throws Exception { + Table testTable = TestTables.create(tempDir("empty_table"), "empty_table", SCHEMA, SPEC, 2); + assertThatThrownBy( + () -> PartitionStatsUtil.computeStats(testTable, testTable.currentSnapshot())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("snapshot cannot be null"); + } + + @Test + public void testPartitionStatsOnUnPartitionedTable() throws Exception { + Table testTable = + TestTables.create( + tempDir("unpartitioned_table"), + "unpartitioned_table", + SCHEMA, + PartitionSpec.unpartitioned(), + 2); + + List files = prepareDataFiles(testTable); + AppendFiles appendFiles = testTable.newAppend(); + files.forEach(appendFiles::appendFile); + appendFiles.commit(); + + assertThatThrownBy( + () -> PartitionStatsUtil.computeStats(testTable, testTable.currentSnapshot())) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("table must be partitioned"); + } + + @Test + public void testPartitionStats() throws Exception { + Table testTable = + TestTables.create( + tempDir("partition_stats_compute"), "partition_stats_compute", SCHEMA, SPEC, 2); + + List files = prepareDataFiles(testTable); + for (int i = 0; i < 3; i++) { + // insert same set of records thrice to have a new manifest files + AppendFiles appendFiles = testTable.newAppend(); + files.forEach(appendFiles::appendFile); + appendFiles.commit(); + } + + Snapshot snapshot1 = testTable.currentSnapshot(); + Types.StructType partitionType = Partitioning.partitionType(testTable); + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo", "A"), + 0, + 3 * files.get(0).recordCount(), + 3, + 3 * files.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "B"), + 0, + 3 * files.get(1).recordCount(), + 3, + 3 * files.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "A"), + 0, + 3 * files.get(2).recordCount(), + 3, + 3 * files.get(2).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "B"), + 0, + 3 * files.get(3).recordCount(), + 3, + 3 * files.get(3).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId())); + + DeleteFile deleteFile = + FileGenerationUtil.generatePositionDeleteFile(testTable, TestHelpers.Row.of("foo", "A")); + testTable.newRowDelta().addDeletes(deleteFile).commit(); + Snapshot snapshot2 = testTable.currentSnapshot(); + + DeleteFile eqDelete = + FileGenerationUtil.generateEqualityDeleteFile(testTable, TestHelpers.Row.of("bar", "B")); + testTable.newRowDelta().addDeletes(eqDelete).commit(); + Snapshot snapshot3 = testTable.currentSnapshot(); + + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo", "A"), + 0, + 3 * files.get(0).recordCount(), + 3, + 3 * files.get(0).fileSizeInBytes(), + deleteFile.recordCount(), // position delete file count + 1, // one position delete file + 0L, + 0, + 0L, + snapshot2.timestampMillis(), // new snapshot from pos delete commit + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "B"), + 0, + 3 * files.get(1).recordCount(), + 3, + 3 * files.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "A"), + 0, + 3 * files.get(2).recordCount(), + 3, + 3 * files.get(2).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "B"), + 0, + 3 * files.get(3).recordCount(), + 3, + 3 * files.get(3).fileSizeInBytes(), + 0L, + 0, + eqDelete.recordCount(), + 1, // one equality delete file + 0L, + snapshot3.timestampMillis(), // new snapshot from equality delete commit + snapshot3.snapshotId())); + } + + @Test + @SuppressWarnings("MethodLength") + public void testPartitionStatsWithSchemaEvolution() throws Exception { + final PartitionSpec specBefore = PartitionSpec.builderFor(SCHEMA).identity("c2").build(); + + Table testTable = + TestTables.create( + tempDir("partition_stats_schema_evolve"), + "partition_stats_schema_evolve", + SCHEMA, + specBefore, + SortOrder.unsorted(), + 2); + + List dataFiles = prepareDataFilesOnePart(testTable); + for (int i = 0; i < 2; i++) { + AppendFiles appendFiles = testTable.newAppend(); + dataFiles.forEach(appendFiles::appendFile); + appendFiles.commit(); + } + Snapshot snapshot1 = testTable.currentSnapshot(); + Types.StructType partitionType = Partitioning.partitionType(testTable); + + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo"), + 0, + 2 * dataFiles.get(0).recordCount(), + 2, + 2 * dataFiles.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar"), + 0, + 2 * dataFiles.get(1).recordCount(), + 2, + 2 * dataFiles.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId())); + + // Evolve the partition spec to include c3 + testTable.updateSpec().addField("c3").commit(); + List filesWithNewSpec = prepareDataFiles(testTable); + filesWithNewSpec.add( + FileGenerationUtil.generateDataFile(testTable, TestHelpers.Row.of("bar", null))); + partitionType = Partitioning.partitionType(testTable); + + AppendFiles appendFiles = testTable.newAppend(); + filesWithNewSpec.forEach(appendFiles::appendFile); + appendFiles.commit(); + Snapshot snapshot2 = testTable.currentSnapshot(); + + computeAndValidatePartitionStats( + testTable, + Tuple.tuple( + partitionData(partitionType, "foo", null), // unified tuple + 0, // old spec id as the record is unmodified + 2 * dataFiles.get(0).recordCount(), + 2, + 2 * dataFiles.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", null), + 0, // old spec id for "bar, null" before evolution + 2 * dataFiles.get(1).recordCount(), + 2, + 2 * dataFiles.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot1.timestampMillis(), + snapshot1.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", null), + 1, // new spec id for "bar, null" after evolution + filesWithNewSpec.get(4).recordCount(), + 1, + filesWithNewSpec.get(4).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), // new snapshot + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "A"), + 1, // new spec id + filesWithNewSpec.get(0).recordCount(), + 1, + filesWithNewSpec.get(0).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), // new snapshot + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "foo", "B"), + 1, + filesWithNewSpec.get(1).recordCount(), + 1, + filesWithNewSpec.get(1).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "A"), + 1, + filesWithNewSpec.get(2).recordCount(), + 1, + filesWithNewSpec.get(2).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), + snapshot2.snapshotId()), + Tuple.tuple( + partitionData(partitionType, "bar", "B"), + 1, + filesWithNewSpec.get(3).recordCount(), + 1, + filesWithNewSpec.get(3).fileSizeInBytes(), + 0L, + 0, + 0L, + 0, + 0L, + snapshot2.timestampMillis(), + snapshot2.snapshotId())); + } + + private static PartitionData partitionData(Types.StructType partitionType, String c2, String c3) { + PartitionData partitionData = new PartitionData(partitionType); + partitionData.set(0, c2); + partitionData.set(1, c3); + return partitionData; + } + + private static PartitionData partitionData(Types.StructType partitionType, String c2) { + PartitionData partitionData = new PartitionData(partitionType); + partitionData.set(0, c2); + return partitionData; + } + + private static List prepareDataFiles(Table table) { + List dataFiles = Lists.newArrayList(); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("foo", "A"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("foo", "B"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("bar", "A"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("bar", "B"))); + + return dataFiles; + } + + private static List prepareDataFilesOnePart(Table table) { + List dataFiles = Lists.newArrayList(); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("foo"))); + dataFiles.add(FileGenerationUtil.generateDataFile(table, TestHelpers.Row.of("bar"))); + + return dataFiles; + } + + private static void computeAndValidatePartitionStats(Table testTable, Tuple... expectedValues) { + // compute and commit partition stats file + Collection result = + PartitionStatsUtil.computeStats(testTable, testTable.currentSnapshot()); + + assertThat(result) + .extracting( + PartitionStats::partition, + PartitionStats::specId, + PartitionStats::dataRecordCount, + PartitionStats::dataFileCount, + PartitionStats::totalDataFileSizeInBytes, + PartitionStats::positionDeleteRecordCount, + PartitionStats::positionDeleteFileCount, + PartitionStats::equalityDeleteRecordCount, + PartitionStats::equalityDeleteFileCount, + PartitionStats::totalRecordCount, + PartitionStats::lastUpdatedAt, + PartitionStats::lastUpdatedSnapshotId) + .containsExactlyInAnyOrder(expectedValues); + } + + private File tempDir(String folderName) throws IOException { + return java.nio.file.Files.createTempDirectory(temp.toPath(), folderName).toFile(); + } +} From 6d9e326630dd84713f5534bd0c34de773213b7a3 Mon Sep 17 00:00:00 2001 From: Aihua Xu Date: Thu, 26 Sep 2024 09:47:28 -0700 Subject: [PATCH 0726/1019] Parquet: update PruneColumns to inherit from TypeWithSchemaVisitor to have Iceberg type (#11179) --- .../iceberg/parquet/ParquetSchemaUtil.java | 4 +++- .../org/apache/iceberg/parquet/PruneColumns.java | 16 ++++++++++------ 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java index 27ce35882458..a0dc54c1cdd9 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetSchemaUtil.java @@ -78,7 +78,9 @@ private static Schema convertInternal( public static MessageType pruneColumns(MessageType fileSchema, Schema expectedSchema) { // column order must match the incoming type, so it doesn't matter that the ids are unordered Set selectedIds = TypeUtil.getProjectedIds(expectedSchema); - return (MessageType) ParquetTypeVisitor.visit(fileSchema, new PruneColumns(selectedIds)); + return (MessageType) + TypeWithSchemaVisitor.visit( + expectedSchema.asStruct(), fileSchema, new PruneColumns(selectedIds)); } /** diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java index f4f50f1f3efa..d48485305e8a 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PruneColumns.java @@ -24,6 +24,9 @@ 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.Lists; +import org.apache.iceberg.types.Types.ListType; +import org.apache.iceberg.types.Types.MapType; +import org.apache.iceberg.types.Types.StructType; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.LogicalTypeAnnotation; import org.apache.parquet.schema.MessageType; @@ -31,7 +34,7 @@ import org.apache.parquet.schema.Type; import org.apache.parquet.schema.Types; -class PruneColumns extends ParquetTypeVisitor { +class PruneColumns extends TypeWithSchemaVisitor { private final Set selectedIds; PruneColumns(Set selectedIds) { @@ -40,7 +43,7 @@ class PruneColumns extends ParquetTypeVisitor { } @Override - public Type message(MessageType message, List fields) { + public Type message(StructType expected, MessageType message, List fields) { Types.MessageTypeBuilder builder = Types.buildMessage(); boolean hasChange = false; @@ -79,7 +82,7 @@ public Type message(MessageType message, List fields) { } @Override - public Type struct(GroupType struct, List fields) { + public Type struct(StructType expected, GroupType struct, List fields) { boolean hasChange = false; List filteredFields = Lists.newArrayListWithExpectedSize(fields.size()); for (int i = 0; i < fields.size(); i += 1) { @@ -106,7 +109,7 @@ public Type struct(GroupType struct, List fields) { } @Override - public Type list(GroupType list, Type element) { + public Type list(ListType expected, GroupType list, Type element) { Type repeated = list.getType(0); Type originalElement = ParquetSchemaUtil.determineListElementType(list); Integer elementId = getId(originalElement); @@ -128,7 +131,7 @@ public Type list(GroupType list, Type element) { } @Override - public Type map(GroupType map, Type key, Type value) { + public Type map(MapType expected, GroupType map, Type key, Type value) { GroupType repeated = map.getType(0).asGroupType(); Type originalKey = repeated.getType(0); Type originalValue = repeated.getType(1); @@ -150,7 +153,8 @@ public Type map(GroupType map, Type key, Type value) { } @Override - public Type primitive(PrimitiveType primitive) { + public Type primitive( + org.apache.iceberg.types.Type.PrimitiveType expected, PrimitiveType primitive) { return null; } From 2db9ebe1cf4ee7d194cd6e97a0bc11a4280055a1 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 26 Sep 2024 11:44:14 -0600 Subject: [PATCH 0727/1019] Core: Replace use of CharSequenceMap in DeleteFileIndex with String (#11199) --- .../java/org/apache/iceberg/DeleteFileIndex.java | 15 +++++++-------- .../org/apache/iceberg/util/ContentFileUtil.java | 5 +++++ 2 files changed, 12 insertions(+), 8 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java index 794044373471..8444b91eecd4 100644 --- a/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java +++ b/core/src/main/java/org/apache/iceberg/DeleteFileIndex.java @@ -51,7 +51,6 @@ import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.ArrayUtil; -import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.PartitionMap; import org.apache.iceberg.util.PartitionSet; @@ -70,7 +69,7 @@ class DeleteFileIndex { private final EqualityDeletes globalDeletes; private final PartitionMap eqDeletesByPartition; private final PartitionMap posDeletesByPartition; - private final CharSequenceMap posDeletesByPath; + private final Map posDeletesByPath; private final boolean hasEqDeletes; private final boolean hasPosDeletes; private final boolean isEmpty; @@ -79,7 +78,7 @@ private DeleteFileIndex( EqualityDeletes globalDeletes, PartitionMap eqDeletesByPartition, PartitionMap posDeletesByPartition, - CharSequenceMap posDeletesByPath) { + Map posDeletesByPath) { this.globalDeletes = globalDeletes; this.eqDeletesByPartition = eqDeletesByPartition; this.posDeletesByPartition = posDeletesByPartition; @@ -177,7 +176,7 @@ private DeleteFile[] findPathDeletes(long seq, DataFile dataFile) { return EMPTY_DELETES; } - PositionDeletes deletes = posDeletesByPath.get(dataFile.path()); + PositionDeletes deletes = posDeletesByPath.get(dataFile.location()); return deletes == null ? EMPTY_DELETES : deletes.filter(seq); } @@ -434,7 +433,7 @@ DeleteFileIndex build() { EqualityDeletes globalDeletes = new EqualityDeletes(); PartitionMap eqDeletesByPartition = PartitionMap.create(specsById); PartitionMap posDeletesByPartition = PartitionMap.create(specsById); - CharSequenceMap posDeletesByPath = CharSequenceMap.create(); + Map posDeletesByPath = Maps.newHashMap(); for (DeleteFile file : files) { switch (file.content()) { @@ -458,14 +457,14 @@ DeleteFileIndex build() { } private void add( - CharSequenceMap deletesByPath, + Map deletesByPath, PartitionMap deletesByPartition, DeleteFile file) { - CharSequence path = ContentFileUtil.referencedDataFile(file); + String path = ContentFileUtil.referencedDataFileLocation(file); PositionDeletes deletes; if (path != null) { - deletes = deletesByPath.computeIfAbsent(path, PositionDeletes::new); + deletes = deletesByPath.computeIfAbsent(path, ignored -> new PositionDeletes()); } else { int specId = file.specId(); StructLike partition = file.partition(); 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 fc68f14d0971..04fc077d10ea 100644 --- a/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java +++ b/core/src/main/java/org/apache/iceberg/util/ContentFileUtil.java @@ -75,4 +75,9 @@ public static CharSequence referencedDataFile(DeleteFile deleteFile) { return null; } } + + public static String referencedDataFileLocation(DeleteFile deleteFile) { + CharSequence location = referencedDataFile(deleteFile); + return location != null ? location.toString() : null; + } } From 6d3a49ce646837105dd059a171f9b5a2a923e763 Mon Sep 17 00:00:00 2001 From: Anurag Mantripragada Date: Thu, 26 Sep 2024 17:13:26 -0700 Subject: [PATCH 0728/1019] [Core] Fix TestFastAppend.testAddManyFiles() (#11218) --- core/src/test/java/org/apache/iceberg/TestFastAppend.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/test/java/org/apache/iceberg/TestFastAppend.java b/core/src/test/java/org/apache/iceberg/TestFastAppend.java index b2f19fbd5ffc..7a93b9988730 100644 --- a/core/src/test/java/org/apache/iceberg/TestFastAppend.java +++ b/core/src/test/java/org/apache/iceberg/TestFastAppend.java @@ -54,7 +54,7 @@ public void testAddManyFiles() { dataFiles.add(dataFile); } - AppendFiles append = table.newAppend(); + AppendFiles append = table.newFastAppend(); dataFiles.forEach(append::appendFile); append.commit(); From 7810e5fd102685ea514f9f5a31109db6698ed42a Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Fri, 27 Sep 2024 10:06:58 +0530 Subject: [PATCH 0729/1019] Spark: Deprecate SparkAppenderFactory (#11076) --- .../org/apache/iceberg/spark/source/SparkAppenderFactory.java | 4 ++++ .../org/apache/iceberg/spark/source/SparkAppenderFactory.java | 4 ++++ .../org/apache/iceberg/spark/source/SparkAppenderFactory.java | 4 ++++ 3 files changed, 12 insertions(+) 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 index 6372edde0782..2ff4f9156273 100644 --- 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 @@ -48,6 +48,10 @@ 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; 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 index 9df12fc060ae..c822ed743f85 100644 --- 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 @@ -49,6 +49,10 @@ 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; 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 index 9df12fc060ae..c822ed743f85 100644 --- 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 @@ -49,6 +49,10 @@ 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; From d865d38de0bf0fba15d5da5560bcd9ba685588ec Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 12:46:29 +0200 Subject: [PATCH 0730/1019] Build: Bump guava from 33.3.0-jre to 33.3.1-jre (#11230) Bumps `guava` from 33.3.0-jre to 33.3.1-jre. Updates `com.google.guava:guava` from 33.3.0-jre to 33.3.1-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.0-jre to 33.3.1-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-patch - dependency-name: com.google.guava:guava-testlib 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 fad4d49a1e62..6f14342d73a7 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.47.0" -guava = "33.3.0-jre" +guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" httpcomponents-httpclient5 = "5.4" From c12f9630f5e26c69f2bcd3fb87ea6d9746fdb558 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 12:46:42 +0200 Subject: [PATCH 0731/1019] Build: Bump software.amazon.awssdk:bom from 2.28.5 to 2.28.11 (#11229) Bumps software.amazon.awssdk:bom from 2.28.5 to 2.28.11. --- 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 6f14342d73a7..c546492ed95f 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.5" +awssdk-bom = "2.28.11" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 1a056a892e3ca3bb86c7a1716a5adfd8a33352c0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 12:47:12 +0200 Subject: [PATCH 0732/1019] Build: Bump io.delta:delta-standalone_2.12 from 3.2.0 to 3.2.1 (#11228) Bumps [io.delta:delta-standalone_2.12](https://github.com/delta-io/delta) from 3.2.0 to 3.2.1. - [Release notes](https://github.com/delta-io/delta/releases) - [Commits](https://github.com/delta-io/delta/compare/v3.2.0...v3.2.1) --- updated-dependencies: - dependency-name: io.delta:delta-standalone_2.12 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 c546492ed95f..a68e131fa96e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -35,7 +35,7 @@ awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" calcite = "1.10.0" datasketches = "6.0.0" -delta-standalone = "3.2.0" +delta-standalone = "3.2.1" delta-spark = "3.2.0" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.31.0" From dcaf8de46407083577d2281311b4d1f5eb73458c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 12:47:29 +0200 Subject: [PATCH 0733/1019] Build: Bump junit-platform from 1.10.3 to 1.11.1 (#11227) Bumps `junit-platform` from 1.10.3 to 1.11.1. Updates `org.junit.platform:junit-platform-suite-api` from 1.10.3 to 1.11.1 - [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.10.3 to 1.11.1 - [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-minor - dependency-name: org.junit.platform:junit-platform-suite-engine 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 a68e131fa96e..68b9a1850c98 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.10.1" -junit-platform = "1.10.3" +junit-platform = "1.11.1" kafka = "3.8.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.1" From d1940cae89d837e75fd0d0312780506b649f01c8 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 30 Sep 2024 15:02:17 +0200 Subject: [PATCH 0734/1019] Build: Bump io.delta:delta-spark_2.12 from 3.2.0 to 3.2.1 (#11225) Bumps [io.delta:delta-spark_2.12](https://github.com/delta-io/delta) from 3.2.0 to 3.2.1. - [Release notes](https://github.com/delta-io/delta/releases) - [Commits](https://github.com/delta-io/delta/compare/v3.2.0...v3.2.1) --- updated-dependencies: - dependency-name: io.delta:delta-spark_2.12 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 68b9a1850c98..a726d52dad1e 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -36,7 +36,7 @@ caffeine = "2.9.3" calcite = "1.10.0" datasketches = "6.0.0" delta-standalone = "3.2.1" -delta-spark = "3.2.0" +delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" errorprone-annotations = "2.31.0" failsafe = "3.3.2" From 0a057d239167284cd59402da27ca3d26f8ff4a96 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 30 Sep 2024 18:21:56 +0200 Subject: [PATCH 0735/1019] Core: Improve error handling when parsing view representations (#11236) --- .../apache/iceberg/view/ViewVersionParser.java | 10 ++++++---- .../iceberg/view/TestViewVersionParser.java | 18 ++++++++++++++++++ 2 files changed, 24 insertions(+), 4 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java index 2645e40d94ef..06ee3b2648d2 100644 --- a/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java +++ b/core/src/main/java/org/apache/iceberg/view/ViewVersionParser.java @@ -84,12 +84,14 @@ public static ViewVersion fromJson(JsonNode node) { long timestamp = JsonUtil.getLong(TIMESTAMP_MS, node); Map summary = JsonUtil.getStringMap(SUMMARY, node); - JsonNode serializedRepresentations = node.get(REPRESENTATIONS); + JsonNode serializedRepresentations = JsonUtil.get(REPRESENTATIONS, node); + Preconditions.checkArgument( + serializedRepresentations.isArray(), + "Cannot parse representations from non-array: %s", + serializedRepresentations); ImmutableList.Builder representations = ImmutableList.builder(); for (JsonNode serializedRepresentation : serializedRepresentations) { - ViewRepresentation representation = - ViewRepresentationParser.fromJson(serializedRepresentation); - representations.add(representation); + representations.add(ViewRepresentationParser.fromJson(serializedRepresentation)); } String defaultCatalog = JsonUtil.getStringOrNull(DEFAULT_CATALOG, node); diff --git a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java index 1d115c04c393..a68b99a6797b 100644 --- a/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java +++ b/core/src/test/java/org/apache/iceberg/view/TestViewVersionParser.java @@ -129,4 +129,22 @@ public void missingDefaultCatalog() { .isInstanceOf(IllegalArgumentException.class) .hasMessage("Cannot parse missing field: default-namespace"); } + + @Test + public void invalidRepresentations() { + String invalidRepresentations = + "{\"version-id\":1, \"timestamp-ms\":12345, \"schema-id\":1, \"summary\":{\"user\":\"some-user\"}, \"representations\": 23, \"default-namespace\":[\"one\",\"two\"]}"; + assertThatThrownBy(() -> ViewVersionParser.fromJson(invalidRepresentations)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse representations from non-array: 23"); + } + + @Test + public void missingRepresentations() { + String missingRepresentations = + "{\"version-id\":1, \"timestamp-ms\":12345, \"schema-id\":1, \"summary\":{\"user\":\"some-user\"}, \"default-namespace\":[\"one\",\"two\"]}"; + assertThatThrownBy(() -> ViewVersionParser.fromJson(missingRepresentations)) + .isInstanceOf(IllegalArgumentException.class) + .hasMessage("Cannot parse missing field: representations"); + } } From 88c64b772ff04eeff8f4ba8efe039a1c64063c61 Mon Sep 17 00:00:00 2001 From: rcjverhoef <30408627+rcjverhoef@users.noreply.github.com> Date: Tue, 1 Oct 2024 07:07:20 +0900 Subject: [PATCH 0736/1019] Core: Update REST CatalogHandlers to handle page sizes exceeding number of Namespaces/Tables/Views (#11143) --- .../apache/iceberg/rest/CatalogHandlers.java | 58 +++++++++---------- .../apache/iceberg/rest/TestRESTCatalog.java | 12 ++-- .../iceberg/rest/TestRESTViewCatalog.java | 9 +-- 3 files changed, 38 insertions(+), 41 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 f1b7aa32d679..76fafe48f5b6 100644 --- a/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java +++ b/core/src/main/java/org/apache/iceberg/rest/CatalogHandlers.java @@ -25,6 +25,7 @@ import java.time.OffsetDateTime; import java.time.ZoneOffset; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Optional; @@ -71,6 +72,7 @@ import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.rest.responses.LoadViewResponse; import org.apache.iceberg.rest.responses.UpdateNamespacePropertiesResponse; +import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.Tasks; import org.apache.iceberg.view.BaseView; import org.apache.iceberg.view.SQLViewRepresentation; @@ -82,7 +84,7 @@ public class CatalogHandlers { private static final Schema EMPTY_SCHEMA = new Schema(); - private static final String INTIAL_PAGE_TOKEN = ""; + private static final String INITIAL_PAGE_TOKEN = ""; private CatalogHandlers() {} @@ -108,6 +110,19 @@ public CommitFailedException wrapped() { } } + private static Pair, String> paginate(List list, String pageToken, int pageSize) { + int pageStart = INITIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); + if (pageStart >= list.size()) { + return Pair.of(Collections.emptyList(), null); + } + + int end = Math.min(pageStart + pageSize, list.size()); + List subList = list.subList(pageStart, end); + String nextPageToken = end >= list.size() ? null : String.valueOf(end); + + return Pair.of(subList, nextPageToken); + } + public static ListNamespacesResponse listNamespaces( SupportsNamespaces catalog, Namespace parent) { List results; @@ -123,7 +138,6 @@ public static ListNamespacesResponse listNamespaces( public static ListNamespacesResponse listNamespaces( SupportsNamespaces catalog, Namespace parent, String pageToken, String pageSize) { List results; - List subResults; if (parent.isEmpty()) { results = catalog.listNamespaces(); @@ -131,16 +145,12 @@ public static ListNamespacesResponse listNamespaces( results = catalog.listNamespaces(parent); } - int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); - int end = start + Integer.parseInt(pageSize); - subResults = results.subList(start, end); - String nextToken = String.valueOf(end); - - if (end >= results.size()) { - nextToken = null; - } + Pair, String> page = paginate(results, pageToken, Integer.parseInt(pageSize)); - return ListNamespacesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + return ListNamespacesResponse.builder() + .addAll(page.first()) + .nextPageToken(page.second()) + .build(); } public static CreateNamespaceResponse createNamespace( @@ -203,18 +213,11 @@ public static ListTablesResponse listTables(Catalog catalog, Namespace namespace public static ListTablesResponse listTables( Catalog catalog, Namespace namespace, String pageToken, String pageSize) { List results = catalog.listTables(namespace); - List subResults; - - int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); - int end = start + Integer.parseInt(pageSize); - subResults = results.subList(start, end); - String nextToken = String.valueOf(end); - if (end >= results.size()) { - nextToken = null; - } + Pair, String> page = + paginate(results, pageToken, Integer.parseInt(pageSize)); - return ListTablesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + return ListTablesResponse.builder().addAll(page.first()).nextPageToken(page.second()).build(); } public static LoadTableResponse stageTableCreate( @@ -448,18 +451,11 @@ public static ListTablesResponse listViews(ViewCatalog catalog, Namespace namesp public static ListTablesResponse listViews( ViewCatalog catalog, Namespace namespace, String pageToken, String pageSize) { List results = catalog.listViews(namespace); - List subResults; - - int start = INTIAL_PAGE_TOKEN.equals(pageToken) ? 0 : Integer.parseInt(pageToken); - int end = start + Integer.parseInt(pageSize); - subResults = results.subList(start, end); - String nextToken = String.valueOf(end); - if (end >= results.size()) { - nextToken = null; - } + Pair, String> page = + paginate(results, pageToken, Integer.parseInt(pageSize)); - return ListTablesResponse.builder().addAll(subResults).nextPageToken(nextToken).build(); + return ListTablesResponse.builder().addAll(page.first()).nextPageToken(page.second()).build(); } public static LoadViewResponse createView( 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 1c15cfab43a3..06008761eac1 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -2341,13 +2341,13 @@ public void testInvalidPageSize() { RESTSessionCatalog.REST_PAGE_SIZE)); } - @Test - public void testPaginationForListNamespaces() { + @ParameterizedTest + @ValueSource(ints = {21, 30}) + public void testPaginationForListNamespaces(int numberOfItems) { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); - int numberOfItems = 30; String namespaceName = "newdb"; // create several namespaces for listing and verify @@ -2403,13 +2403,13 @@ public void testPaginationForListNamespaces() { eq(ListNamespacesResponse.class)); } - @Test - public void testPaginationForListTables() { + @ParameterizedTest + @ValueSource(ints = {21, 30}) + public void testPaginationForListTables(int numberOfItems) { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); - int numberOfItems = 30; String namespaceName = "newdb"; String tableName = "newtable"; catalog.createNamespace(Namespace.of(namespaceName)); 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 dac2a9d25f4b..85ccdc8f5ddd 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTViewCatalog.java @@ -50,8 +50,9 @@ 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; import org.mockito.Mockito; public class TestRESTViewCatalog extends ViewCatalogTests { @@ -153,14 +154,14 @@ public void closeCatalog() throws Exception { } } - @Test - public void testPaginationForListViews() { + @ParameterizedTest + @ValueSource(ints = {21, 30}) + public void testPaginationForListViews(int numberOfItems) { RESTCatalogAdapter adapter = Mockito.spy(new RESTCatalogAdapter(backendCatalog)); RESTCatalog catalog = new RESTCatalog(SessionCatalog.SessionContext.createEmpty(), (config) -> adapter); catalog.initialize("test", ImmutableMap.of(RESTSessionCatalog.REST_PAGE_SIZE, "10")); - int numberOfItems = 30; String namespaceName = "newdb"; String viewName = "newview"; From 08bfc7f104156d40e70e22504498e7c1117cf530 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 30 Sep 2024 21:12:56 -0700 Subject: [PATCH 0737/1019] Core: Support combining position deletes during writes (#11222) --- .../deletes/BitmapPositionDeleteIndex.java | 25 +++- .../org/apache/iceberg/deletes/Deletes.java | 32 ++++- .../iceberg/deletes/PositionDelete.java | 7 ++ .../iceberg/deletes/PositionDeleteIndex.java | 15 +++ .../SortingPositionOnlyDeleteWriter.java | 58 +++++++-- .../io/FanoutPositionOnlyDeleteWriter.java | 28 ++++- .../java/org/apache/iceberg/TestBase.java | 4 + .../apache/iceberg/data/BaseDeleteLoader.java | 4 +- .../iceberg/io/TestPartitioningWriters.java | 111 ++++++++++++++++++ 9 files changed, 264 insertions(+), 20 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 3a044878562a..a1b57a38666d 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/BitmapPositionDeleteIndex.java @@ -18,18 +18,35 @@ */ package org.apache.iceberg.deletes; +import java.util.Collection; +import java.util.List; 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 List deleteFiles; BitmapPositionDeleteIndex() { - roaring64Bitmap = new Roaring64Bitmap(); + this.roaring64Bitmap = new Roaring64Bitmap(); + this.deleteFiles = Lists.newArrayList(); + } + + BitmapPositionDeleteIndex(Collection deleteFiles) { + this.roaring64Bitmap = new Roaring64Bitmap(); + this.deleteFiles = Lists.newArrayList(deleteFiles); + } + + BitmapPositionDeleteIndex(DeleteFile deleteFile) { + this.roaring64Bitmap = new Roaring64Bitmap(); + this.deleteFiles = deleteFile != null ? Lists.newArrayList(deleteFile) : Lists.newArrayList(); } void merge(BitmapPositionDeleteIndex that) { roaring64Bitmap.or(that.roaring64Bitmap); + deleteFiles.addAll(that.deleteFiles); } @Override @@ -48,6 +65,7 @@ public void merge(PositionDeleteIndex that) { merge((BitmapPositionDeleteIndex) that); } else { that.forEach(this::delete); + deleteFiles.addAll(that.deleteFiles()); } } @@ -65,4 +83,9 @@ public boolean isEmpty() { public void forEach(LongConsumer consumer) { roaring64Bitmap.forEach(consumer::accept); } + + @Override + public Collection deleteFiles() { + return deleteFiles; + } } 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 a72e01613040..2256b378f62a 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -26,6 +26,7 @@ import java.util.function.Function; import java.util.function.Predicate; import org.apache.iceberg.Accessor; +import org.apache.iceberg.DeleteFile; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; @@ -123,6 +124,11 @@ public static StructLikeSet toEqualitySet( } } + public static CharSequenceMap toPositionIndexes( + CloseableIterable posDeletes) { + return toPositionIndexes(posDeletes, null /* unknown delete file */); + } + /** * Builds a map of position delete indexes by path. * @@ -131,10 +137,11 @@ public static StructLikeSet toEqualitySet( * entire delete file content is needed (e.g. caching). * * @param posDeletes position deletes + * @param file the source delete file for the deletes * @return the map of position delete indexes by path */ public static CharSequenceMap toPositionIndexes( - CloseableIterable posDeletes) { + CloseableIterable posDeletes, DeleteFile file) { CharSequenceMap indexes = CharSequenceMap.create(); try (CloseableIterable deletes = posDeletes) { @@ -142,7 +149,7 @@ public static CharSequenceMap toPosi CharSequence filePath = (CharSequence) FILENAME_ACCESSOR.get(delete); long position = (long) POSITION_ACCESSOR.get(delete); PositionDeleteIndex index = - indexes.computeIfAbsent(filePath, key -> new BitmapPositionDeleteIndex()); + indexes.computeIfAbsent(filePath, key -> new BitmapPositionDeleteIndex(file)); index.delete(position); } } catch (IOException e) { @@ -152,6 +159,20 @@ public static CharSequenceMap toPosi return indexes; } + public static PositionDeleteIndex toPositionIndex( + CharSequence dataLocation, CloseableIterable posDeletes, DeleteFile file) { + CloseableIterable positions = extractPositions(dataLocation, posDeletes); + List files = ImmutableList.of(file); + return toPositionIndex(positions, files); + } + + private static CloseableIterable extractPositions( + CharSequence dataLocation, CloseableIterable rows) { + DataFileFilter filter = new DataFileFilter<>(dataLocation); + CloseableIterable filteredRows = filter.filter(rows); + return CloseableIterable.transform(filteredRows, row -> (Long) POSITION_ACCESSOR.get(row)); + } + public static PositionDeleteIndex toPositionIndex( CharSequence dataLocation, List> deleteFiles) { return toPositionIndex(dataLocation, deleteFiles, ThreadPools.getDeleteWorkerPool()); @@ -176,8 +197,13 @@ public static PositionDeleteIndex toPositionIndex( } public static PositionDeleteIndex toPositionIndex(CloseableIterable posDeletes) { + return toPositionIndex(posDeletes, ImmutableList.of()); + } + + private static PositionDeleteIndex toPositionIndex( + CloseableIterable posDeletes, List files) { try (CloseableIterable deletes = posDeletes) { - PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex(); + PositionDeleteIndex positionDeleteIndex = new BitmapPositionDeleteIndex(files); deletes.forEach(positionDeleteIndex::delete); return positionDeleteIndex; } catch (IOException e) { diff --git a/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java b/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java index 655428ce7713..57e188567f68 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDelete.java @@ -31,6 +31,13 @@ public static PositionDelete create() { private PositionDelete() {} + public PositionDelete set(CharSequence newPath, long newPos) { + this.path = newPath; + this.pos = newPos; + this.row = null; + return this; + } + public PositionDelete set(CharSequence newPath, long newPos, R newRow) { this.path = newPath; this.pos = newPos; 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 c0086fe6aa2e..3655b8b7e8eb 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java +++ b/core/src/main/java/org/apache/iceberg/deletes/PositionDeleteIndex.java @@ -18,7 +18,10 @@ */ package org.apache.iceberg.deletes; +import java.util.Collection; import java.util.function.LongConsumer; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; public interface PositionDeleteIndex { /** @@ -42,6 +45,9 @@ public interface PositionDeleteIndex { * @param that the other index to merge */ default void merge(PositionDeleteIndex that) { + if (!that.deleteFiles().isEmpty()) { + throw new UnsupportedOperationException(getClass().getName() + " does not support merge"); + } that.forEach(this::delete); } @@ -72,6 +78,15 @@ default void forEach(LongConsumer consumer) { } } + /** + * Returns delete files that this index was created from or an empty collection if unknown. + * + * @return delete files that this index was created from + */ + default Collection deleteFiles() { + return ImmutableList.of(); + } + /** Returns an empty immutable position delete index. */ static PositionDeleteIndex empty() { return EmptyPositionDeleteIndex.get(); 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 1d4d131dfe6f..818529c02479 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/deletes/SortingPositionOnlyDeleteWriter.java @@ -21,17 +21,18 @@ import java.io.IOException; import java.util.Collection; import java.util.List; +import java.util.function.Function; import java.util.function.Supplier; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.io.DeleteWriteResult; import org.apache.iceberg.io.FileWriter; +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.types.Comparators; import org.apache.iceberg.util.CharSequenceMap; import org.apache.iceberg.util.CharSequenceSet; -import org.roaringbitmap.longlong.PeekableLongIterator; -import org.roaringbitmap.longlong.Roaring64Bitmap; +import org.apache.iceberg.util.ContentFileUtil; /** * A position delete writer that is capable of handling unordered deletes without rows. @@ -41,6 +42,11 @@ * records are not ordered by file and position as required by the spec. If the incoming deletes are * ordered by an external process, use {@link PositionDeleteWriter} instead. * + *

    If configured, this writer can also load previous deletes using the provided function and + * merge them with incoming ones prior to flushing the deletes into a file. Callers must ensure only + * previous file-scoped deletes are loaded because partition-scoped deletes can apply to multiple + * data files and can't be safely discarded. + * *

    Note this writer stores only positions. It does not store deleted records. */ public class SortingPositionOnlyDeleteWriter @@ -48,7 +54,8 @@ public class SortingPositionOnlyDeleteWriter private final Supplier, DeleteWriteResult>> writers; private final DeleteGranularity granularity; - private final CharSequenceMap positionsByPath; + private final CharSequenceMap positionsByPath; + private final Function loadPreviousDeletes; private DeleteWriteResult result = null; public SortingPositionOnlyDeleteWriter(FileWriter, DeleteWriteResult> writer) { @@ -58,17 +65,26 @@ public SortingPositionOnlyDeleteWriter(FileWriter, DeleteWrite public SortingPositionOnlyDeleteWriter( Supplier, DeleteWriteResult>> writers, DeleteGranularity granularity) { + this(writers, granularity, path -> null /* no access to previous deletes */); + } + + public SortingPositionOnlyDeleteWriter( + Supplier, DeleteWriteResult>> writers, + DeleteGranularity granularity, + Function loadPreviousDeletes) { this.writers = writers; this.granularity = granularity; this.positionsByPath = CharSequenceMap.create(); + this.loadPreviousDeletes = loadPreviousDeletes; } @Override public void write(PositionDelete positionDelete) { CharSequence path = positionDelete.path(); long position = positionDelete.pos(); - Roaring64Bitmap positions = positionsByPath.computeIfAbsent(path, Roaring64Bitmap::new); - positions.add(position); + PositionDeleteIndex positions = + positionsByPath.computeIfAbsent(path, key -> new BitmapPositionDeleteIndex()); + positions.delete(position); } @Override @@ -106,14 +122,16 @@ private DeleteWriteResult writePartitionDeletes() throws IOException { private DeleteWriteResult writeFileDeletes() throws IOException { List deleteFiles = Lists.newArrayList(); CharSequenceSet referencedDataFiles = CharSequenceSet.empty(); + List rewrittenDeleteFiles = Lists.newArrayList(); for (CharSequence path : positionsByPath.keySet()) { DeleteWriteResult writeResult = writeDeletes(ImmutableList.of(path)); deleteFiles.addAll(writeResult.deleteFiles()); referencedDataFiles.addAll(writeResult.referencedDataFiles()); + rewrittenDeleteFiles.addAll(writeResult.rewrittenDeleteFiles()); } - return new DeleteWriteResult(deleteFiles, referencedDataFiles); + return new DeleteWriteResult(deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } @SuppressWarnings("CollectionUndefinedEquality") @@ -123,22 +141,38 @@ private DeleteWriteResult writeDeletes(Collection paths) throws IO } FileWriter, DeleteWriteResult> writer = writers.get(); + List rewrittenDeleteFiles = Lists.newArrayList(); try { PositionDelete positionDelete = PositionDelete.create(); for (CharSequence path : sort(paths)) { - // the iterator provides values in ascending sorted order - PeekableLongIterator positions = positionsByPath.get(path).getLongIterator(); - while (positions.hasNext()) { - long position = positions.next(); - writer.write(positionDelete.set(path, position, null /* no row */)); + PositionDeleteIndex positions = positionsByPath.get(path); + PositionDeleteIndex previousPositions = loadPreviousDeletes.apply(path); + if (previousPositions != null && previousPositions.isNotEmpty()) { + validatePreviousDeletes(previousPositions); + positions.merge(previousPositions); + rewrittenDeleteFiles.addAll(previousPositions.deleteFiles()); } + positions.forEach(position -> writer.write(positionDelete.set(path, position))); } } finally { writer.close(); } - return writer.result(); + DeleteWriteResult writerResult = writer.result(); + List deleteFiles = writerResult.deleteFiles(); + CharSequenceSet referencedDataFiles = writerResult.referencedDataFiles(); + return new DeleteWriteResult(deleteFiles, referencedDataFiles, rewrittenDeleteFiles); + } + + private void validatePreviousDeletes(PositionDeleteIndex index) { + Preconditions.checkArgument( + index.deleteFiles().stream().allMatch(this::isFileScoped), + "Previous deletes must be file-scoped"); + } + + private boolean isFileScoped(DeleteFile deleteFile) { + return ContentFileUtil.referencedDataFile(deleteFile) != null; } private Collection sort(Collection paths) { diff --git a/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java b/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java index c6a55064b756..9c527f4b32e5 100644 --- a/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java +++ b/core/src/main/java/org/apache/iceberg/io/FanoutPositionOnlyDeleteWriter.java @@ -19,11 +19,13 @@ package org.apache.iceberg.io; import java.util.List; +import java.util.function.Function; import org.apache.iceberg.DeleteFile; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.StructLike; import org.apache.iceberg.deletes.DeleteGranularity; import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteIndex; import org.apache.iceberg.deletes.SortingPositionOnlyDeleteWriter; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.util.CharSequenceSet; @@ -45,6 +47,8 @@ public class FanoutPositionOnlyDeleteWriter private final DeleteGranularity granularity; private final List deleteFiles; private final CharSequenceSet referencedDataFiles; + private final List rewrittenDeleteFiles; + private final Function loadPreviousDeletes; public FanoutPositionOnlyDeleteWriter( FileWriterFactory writerFactory, @@ -60,6 +64,22 @@ public FanoutPositionOnlyDeleteWriter( FileIO io, long targetFileSizeInBytes, DeleteGranularity granularity) { + this( + writerFactory, + fileFactory, + io, + targetFileSizeInBytes, + granularity, + path -> null /* no access to previous deletes */); + } + + public FanoutPositionOnlyDeleteWriter( + FileWriterFactory writerFactory, + OutputFileFactory fileFactory, + FileIO io, + long targetFileSizeInBytes, + DeleteGranularity granularity, + Function loadPreviousDeletes) { this.writerFactory = writerFactory; this.fileFactory = fileFactory; this.io = io; @@ -67,6 +87,8 @@ public FanoutPositionOnlyDeleteWriter( this.granularity = granularity; this.deleteFiles = Lists.newArrayList(); this.referencedDataFiles = CharSequenceSet.empty(); + this.rewrittenDeleteFiles = Lists.newArrayList(); + this.loadPreviousDeletes = loadPreviousDeletes; } @Override @@ -76,17 +98,19 @@ protected FileWriter, DeleteWriteResult> newWriter( () -> new RollingPositionDeleteWriter<>( writerFactory, fileFactory, io, targetFileSizeInBytes, spec, partition), - granularity); + granularity, + loadPreviousDeletes); } @Override protected void addResult(DeleteWriteResult result) { deleteFiles.addAll(result.deleteFiles()); referencedDataFiles.addAll(result.referencedDataFiles()); + rewrittenDeleteFiles.addAll(result.rewrittenDeleteFiles()); } @Override protected DeleteWriteResult aggregatedResult() { - return new DeleteWriteResult(deleteFiles, referencedDataFiles); + return new DeleteWriteResult(deleteFiles, referencedDataFiles, rewrittenDeleteFiles); } } diff --git a/core/src/test/java/org/apache/iceberg/TestBase.java b/core/src/test/java/org/apache/iceberg/TestBase.java index a0b52b346bf3..f3bbb7979547 100644 --- a/core/src/test/java/org/apache/iceberg/TestBase.java +++ b/core/src/test/java/org/apache/iceberg/TestBase.java @@ -665,6 +665,10 @@ protected DeleteFile newEqualityDeleteFile(int specId, String partitionPath, int .build(); } + protected PositionDelete positionDelete(CharSequence path, long pos) { + return positionDelete(path, pos, null /* no row */); + } + protected PositionDelete positionDelete(CharSequence path, long pos, T row) { PositionDelete positionDelete = PositionDelete.create(); return positionDelete.set(path, pos, row); 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 91b7fd1c1dc1..8a1ebf95abeb 100644 --- a/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java +++ b/data/src/main/java/org/apache/iceberg/data/BaseDeleteLoader.java @@ -166,13 +166,13 @@ private PositionDeleteIndex getOrReadPosDeletes(DeleteFile deleteFile, CharSeque private CharSequenceMap readPosDeletes(DeleteFile deleteFile) { CloseableIterable deletes = openDeletes(deleteFile, POS_DELETE_SCHEMA); - return Deletes.toPositionIndexes(deletes); + return Deletes.toPositionIndexes(deletes, deleteFile); } private PositionDeleteIndex readPosDeletes(DeleteFile deleteFile, CharSequence filePath) { Expression filter = Expressions.equal(MetadataColumns.DELETE_FILE_PATH.name(), filePath); CloseableIterable deletes = openDeletes(deleteFile, POS_DELETE_SCHEMA, filter); - return Deletes.toPositionIndex(filePath, ImmutableList.of(deletes)); + return Deletes.toPositionIndex(filePath, deletes, deleteFile); } private CloseableIterable openDeletes(DeleteFile deleteFile, Schema projection) { 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 8dc031314eda..1c8453bd6a75 100644 --- a/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java +++ b/data/src/test/java/org/apache/iceberg/io/TestPartitioningWriters.java @@ -20,13 +20,17 @@ 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.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.Parameter; import org.apache.iceberg.ParameterizedTestExtension; @@ -34,10 +38,16 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Schema; +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.expressions.Expressions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.Maps; +import org.apache.iceberg.util.ContentFileUtil; import org.apache.iceberg.util.StructLikeSet; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.TestTemplate; @@ -719,4 +729,105 @@ private void checkFanoutPositionOnlyDeleteWriterGranularity(DeleteGranularity de List expectedRows = ImmutableList.of(toRow(11, "aaa"), toRow(12, "aaa")); assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows)); } + + @TestTemplate + public void testRewriteOfPreviousDeletes() throws IOException { + assumeThat(format()).isIn(FileFormat.PARQUET, FileFormat.ORC); + + 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(); + + PartitionSpec spec = table.spec(); + + // init the first delete writer without access to previous deletes + FanoutPositionOnlyDeleteWriter writer1 = + new FanoutPositionOnlyDeleteWriter<>( + 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.close(); + + // verify the writer result + DeleteWriteResult result1 = writer1.result(); + assertThat(result1.deleteFiles()).hasSize(2); + assertThat(result1.referencedDataFiles()).hasSize(2); + assertThat(result1.referencesDataFiles()).isTrue(); + assertThat(result1.rewrittenDeleteFiles()).isEmpty(); + + // commit the initial deletes + RowDelta rowDelta1 = table.newRowDelta(); + result1.deleteFiles().forEach(rowDelta1::addDeletes); + rowDelta1.commit(); + + // verify correctness of the first delete operation + List expectedRows1 = + ImmutableList.of(toRow(1, "aaa"), toRow(3, "aaa"), toRow(11, "aaa"), toRow(12, "aaa")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows1)); + + // populate previous delete mapping + Map previousDeletes = Maps.newHashMap(); + for (DeleteFile deleteFile : result1.deleteFiles()) { + String dataLocation = ContentFileUtil.referencedDataFile(deleteFile).toString(); + previousDeletes.put(dataLocation, deleteFile); + } + + // init the second delete writer with access to previous deletes + FanoutPositionOnlyDeleteWriter writer2 = + new FanoutPositionOnlyDeleteWriter<>( + writerFactory, + fileFactory, + table.io(), + TARGET_FILE_SIZE, + DeleteGranularity.FILE, + 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.close(); + + // verify the writer result + DeleteWriteResult result2 = writer2.result(); + assertThat(result2.deleteFiles()).hasSize(2); + assertThat(result2.referencedDataFiles()).hasSize(2); + assertThat(result2.referencesDataFiles()).isTrue(); + assertThat(result2.rewrittenDeleteFiles()).hasSize(2); + + // add new and remove rewritten delete files + RowDelta rowDelta2 = table.newRowDelta(); + result2.deleteFiles().forEach(rowDelta2::addDeletes); + result2.rewrittenDeleteFiles().forEach(rowDelta2::removeDeletes); + rowDelta2.commit(); + + // verify correctness of the second delete operation + List expectedRows2 = ImmutableList.of(toRow(11, "aaa"), toRow(12, "aaa")); + assertThat(actualRowSet("*")).isEqualTo(toSet(expectedRows2)); + } + + 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(CharSequence path) { + DeleteFile deleteFile = deleteFiles.get(path); + return deleteLoader.loadPositionDeletes(ImmutableList.of(deleteFile), path); + } + } } From 3063969ccd3f64490c74207b84ff88d56b77f21a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 1 Oct 2024 08:14:34 +0200 Subject: [PATCH 0738/1019] Core: Add DataFileSet / DeleteFileSet (#11195) --- .../org/apache/iceberg/util/DataFileSet.java | 113 ++++++ .../apache/iceberg/util/DeleteFileSet.java | 114 +++++++ .../org/apache/iceberg/util/WrapperSet.java | 177 ++++++++++ .../apache/iceberg/util/TestDataFileSet.java | 303 +++++++++++++++++ .../iceberg/util/TestDeleteFileSet.java | 321 ++++++++++++++++++ 5 files changed, 1028 insertions(+) create mode 100644 api/src/main/java/org/apache/iceberg/util/DataFileSet.java create mode 100644 api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java create mode 100644 api/src/main/java/org/apache/iceberg/util/WrapperSet.java create mode 100644 core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java create mode 100644 core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java diff --git a/api/src/main/java/org/apache/iceberg/util/DataFileSet.java b/api/src/main/java/org/apache/iceberg/util/DataFileSet.java new file mode 100644 index 000000000000..27cbee088ad4 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/DataFileSet.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.util; + +import java.util.Objects; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class DataFileSet extends WrapperSet { + private static final ThreadLocal WRAPPERS = + ThreadLocal.withInitial(() -> DataFileWrapper.wrap(null)); + + private DataFileSet() { + // needed for serialization/deserialization + } + + private DataFileSet(Iterable> wrappers) { + super(wrappers); + } + + public static DataFileSet create() { + return new DataFileSet(); + } + + public static DataFileSet of(Iterable iterable) { + return new DataFileSet( + Iterables.transform( + iterable, + obj -> { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return DataFileWrapper.wrap(obj); + })); + } + + @Override + protected Wrapper wrapper() { + return WRAPPERS.get(); + } + + @Override + protected Wrapper wrap(DataFile dataFile) { + return DataFileWrapper.wrap(dataFile); + } + + @Override + protected Class elementClass() { + return DataFile.class; + } + + private static class DataFileWrapper implements Wrapper { + private DataFile file; + + private DataFileWrapper(DataFile file) { + this.file = file; + } + + private static DataFileWrapper wrap(DataFile dataFile) { + return new DataFileWrapper(dataFile); + } + + @Override + public DataFile get() { + return file; + } + + @Override + public Wrapper set(DataFile dataFile) { + this.file = dataFile; + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof DataFileWrapper)) { + return false; + } + + DataFileWrapper that = (DataFileWrapper) o; + return Objects.equals(file.location(), that.file.location()); + } + + @Override + public int hashCode() { + return Objects.hashCode(file.location()); + } + + @Override + public String toString() { + return file.location(); + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java new file mode 100644 index 000000000000..bbe9824963fc --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/DeleteFileSet.java @@ -0,0 +1,114 @@ +/* + * 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 java.util.Objects; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; + +public class DeleteFileSet extends WrapperSet { + private static final ThreadLocal WRAPPERS = + ThreadLocal.withInitial(() -> DeleteFileWrapper.wrap(null)); + + private DeleteFileSet() { + // needed for serialization/deserialization + } + + private DeleteFileSet(Iterable> wrappers) { + super(wrappers); + } + + public static DeleteFileSet create() { + return new DeleteFileSet(); + } + + public static DeleteFileSet of(Iterable iterable) { + return new DeleteFileSet( + Iterables.transform( + iterable, + obj -> { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return DeleteFileWrapper.wrap(obj); + })); + } + + @Override + protected Wrapper wrapper() { + return WRAPPERS.get(); + } + + @Override + protected Wrapper wrap(DeleteFile deleteFile) { + return DeleteFileWrapper.wrap(deleteFile); + } + + @Override + protected Class elementClass() { + return DeleteFile.class; + } + + private static class DeleteFileWrapper implements Wrapper { + private DeleteFile file; + + private DeleteFileWrapper(DeleteFile file) { + this.file = file; + } + + private static DeleteFileWrapper wrap(DeleteFile deleteFile) { + return new DeleteFileWrapper(deleteFile); + } + + @Override + public DeleteFile get() { + return file; + } + + @Override + public Wrapper set(DeleteFile deleteFile) { + this.file = deleteFile; + return this; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (!(o instanceof DeleteFileWrapper)) { + return false; + } + + DeleteFileWrapper that = (DeleteFileWrapper) o; + // this needs to be updated once deletion vector support is added + return Objects.equals(file.location(), that.file.location()); + } + + @Override + public int hashCode() { + return Objects.hashCode(file.location()); + } + + @Override + public String toString() { + return file.location(); + } + } +} diff --git a/api/src/main/java/org/apache/iceberg/util/WrapperSet.java b/api/src/main/java/org/apache/iceberg/util/WrapperSet.java new file mode 100644 index 000000000000..e589f435e158 --- /dev/null +++ b/api/src/main/java/org/apache/iceberg/util/WrapperSet.java @@ -0,0 +1,177 @@ +/* + * 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 java.io.Serializable; +import java.util.Collection; +import java.util.Iterator; +import java.util.Set; +import java.util.stream.Collectors; +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.Iterators; +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.collect.Streams; + +/** + * A custom set for a {@link Wrapper} of the given type that maintains insertion order and does not + * allow null elements. + * + * @param The type to wrap in a {@link Wrapper} instance. + */ +abstract class WrapperSet implements Set, Serializable { + private final Set> set = Sets.newLinkedHashSet(); + + protected WrapperSet(Iterable> wrappers) { + wrappers.forEach(set::add); + } + + protected WrapperSet() {} + + protected abstract Wrapper wrapper(); + + protected abstract Wrapper wrap(T file); + + protected abstract Class elementClass(); + + protected interface Wrapper extends Serializable { + T get(); + + Wrapper set(T object); + } + + @Override + public int size() { + return set.size(); + } + + @Override + public boolean isEmpty() { + return set.isEmpty(); + } + + @Override + public boolean contains(Object obj) { + Preconditions.checkNotNull(obj, "Invalid object: null"); + Wrapper wrapper = wrapper(); + boolean result = set.contains(wrapper.set(elementClass().cast(obj))); + wrapper.set(null); // don't hold a reference to the value + return result; + } + + @Override + public Iterator iterator() { + return Iterators.transform(set.iterator(), Wrapper::get); + } + + @Override + public Object[] toArray() { + return Lists.newArrayList(iterator()).toArray(); + } + + @Override + public X[] toArray(X[] destArray) { + return Lists.newArrayList(iterator()).toArray(destArray); + } + + @Override + public boolean add(T obj) { + Preconditions.checkNotNull(obj, "Invalid object: null"); + return set.add(wrap(obj)); + } + + @Override + public boolean remove(Object obj) { + Preconditions.checkNotNull(obj, "Invalid object: null"); + Wrapper wrapper = wrapper(); + boolean result = set.remove(wrapper.set(elementClass().cast(obj))); + wrapper.set(null); // don't hold a reference to the value + return result; + } + + @Override + public boolean containsAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + return Iterables.all(collection, this::contains); + } + + @Override + public boolean addAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + return collection.stream().filter(this::add).count() != 0; + } + + @Override + public boolean retainAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + Set> toRetain = + collection.stream() + .map(obj -> Preconditions.checkNotNull(obj, "Invalid object: null")) + .map(elementClass()::cast) + .map(this::wrap) + .collect(Collectors.toSet()); + + return Iterables.retainAll(set, toRetain); + } + + @Override + public boolean removeAll(Collection collection) { + Preconditions.checkNotNull(collection, "Invalid collection: null"); + return collection.stream().filter(this::remove).count() != 0; + } + + @Override + public void clear() { + set.clear(); + } + + @Override + public boolean equals(Object other) { + if (this == other) { + return true; + } else if (!(other instanceof Set)) { + return false; + } + + Set that = (Set) other; + + if (size() != that.size()) { + return false; + } + + try { + return containsAll(that); + } catch (ClassCastException | NullPointerException unused) { + return false; + } + } + + @Override + public int hashCode() { + return set.stream().mapToInt(Object::hashCode).sum(); + } + + @Override + public String toString() { + return Streams.stream(iterator()) + .map(Object::toString) + .collect(Collectors.joining(", ", "[", "]")); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java b/core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java new file mode 100644 index 000000000000..0f298ad82e9d --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestDataFileSet.java @@ -0,0 +1,303 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Set; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +/** + * Testing {@link DataFileSet} is easier in iceberg-core since the data file builders are located + * here + */ +public class TestDataFileSet { + + private static final DataFile FILE_A = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-a.parquet") + .withFileSizeInBytes(1) + .withRecordCount(1) + .build(); + private static final DataFile FILE_B = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-b.parquet") + .withFileSizeInBytes(2) + .withRecordCount(2) + .build(); + private static final DataFile FILE_C = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-c.parquet") + .withFileSizeInBytes(3) + .withRecordCount(3) + .build(); + private static final DataFile FILE_D = + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath("/path/to/data-d.parquet") + .withFileSizeInBytes(4) + .withRecordCount(4) + .build(); + + @Test + public void emptySet() { + assertThat(DataFileSet.create()).isEmpty(); + assertThat(DataFileSet.create()).doesNotContain(FILE_A, FILE_B, FILE_C); + } + + @Test + public void insertionOrderIsMaintained() { + DataFileSet set = DataFileSet.create(); + set.addAll(ImmutableList.of(FILE_D, FILE_A, FILE_C)); + set.add(FILE_B); + set.add(FILE_D); + + assertThat(set).hasSize(4).containsExactly(FILE_D, FILE_A, FILE_C, FILE_B); + } + + @Test + public void clear() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + set.clear(); + assertThat(set).isEmpty(); + } + + @Test + public void addAll() { + DataFileSet empty = DataFileSet.create(); + assertThatThrownBy(() -> empty.add(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.addAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DataFileSet set = DataFileSet.create(); + set.addAll(ImmutableList.of(FILE_B, FILE_A, FILE_C, FILE_A)); + assertThat(set).hasSize(3).containsExactly(FILE_B, FILE_A, FILE_C); + } + + @Test + public void contains() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThatThrownBy(() -> set.contains(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set) + .hasSize(2) + .containsExactly(FILE_A, FILE_B) + .doesNotContain(FILE_C) + .doesNotContain(FILE_D); + + assertThatThrownBy(() -> DataFileSet.of(Arrays.asList(FILE_C, FILE_B, null, FILE_A))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + } + + @Test + public void containsAll() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThatThrownBy(() -> set.containsAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> set.containsAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> set.containsAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set.containsAll(ImmutableList.of(FILE_B, FILE_A))).isTrue(); + assertThat(set.containsAll(ImmutableList.of(FILE_B, FILE_A, FILE_C))).isFalse(); + assertThat(set.containsAll(ImmutableList.of(FILE_B))).isTrue(); + } + + @Test + public void toArray() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_B, FILE_A)); + assertThat(set.toArray()).hasSize(2).containsExactly(FILE_B, FILE_A); + + DataFile[] array = new DataFile[1]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B, FILE_A); + + array = new DataFile[0]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B, FILE_A); + + array = new DataFile[5]; + assertThat(set.toArray(array)).hasSize(5).containsExactly(FILE_B, FILE_A, null, null, null); + + array = new DataFile[2]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B, FILE_A); + } + + @Test + public void retainAll() { + DataFileSet empty = DataFileSet.create(); + assertThatThrownBy(() -> empty.retainAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.retainAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.retainAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThat(set.retainAll(ImmutableList.of(FILE_C, FILE_D, FILE_A))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_A); + + set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + + assertThat(set.retainAll(ImmutableList.of(FILE_B, FILE_A))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.retainAll(ImmutableList.of(FILE_C, FILE_D))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void remove() { + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThatThrownBy(() -> set.remove(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + set.remove(FILE_C); + assertThat(set).containsExactly(FILE_A, FILE_B); + set.remove(FILE_B); + assertThat(set).containsExactly(FILE_A); + set.remove(FILE_A); + assertThat(set).isEmpty(); + } + + @Test + public void removeAll() { + DataFileSet empty = DataFileSet.create(); + assertThatThrownBy(() -> empty.removeAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.removeAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.removeAll(Arrays.asList(FILE_A, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DataFileSet set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + + assertThat(set.removeAll(ImmutableList.of(FILE_C, FILE_D, FILE_A))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_B); + + set = DataFileSet.of(ImmutableList.of(FILE_A, FILE_B)); + assertThat(set.removeAll(ImmutableList.of(FILE_C, FILE_D))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.removeAll(ImmutableList.of(FILE_B, FILE_A))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void equalsAndHashCode() { + DataFileSet set1 = DataFileSet.create(); + DataFileSet set2 = DataFileSet.create(); + + assertThat(set1).isEqualTo(set2); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()); + + set1.add(FILE_A); + set1.add(FILE_B); + set1.add(FILE_C); + + // different DataFile instances but all use the same paths as set1 + set2.add( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FILE_A.location()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build()); + set2.add( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FILE_B.location()) + .withFileSizeInBytes(100) + .withRecordCount(10) + .build()); + set2.add( + DataFiles.builder(PartitionSpec.unpartitioned()) + .withPath(FILE_C.location()) + .withFileSizeInBytes(1000) + .withRecordCount(100) + .build()); + + Set set3 = Collections.unmodifiableSet(set2); + + assertThat(set1).isEqualTo(set2).isEqualTo(set3); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()).isEqualTo(set3.hashCode()); + } + + @Test + public void kryoSerialization() throws Exception { + DataFileSet dataFiles = DataFileSet.of(ImmutableList.of(FILE_C, FILE_B, FILE_A)); + assertThat(TestHelpers.KryoHelpers.roundTripSerialize(dataFiles)).isEqualTo(dataFiles); + } + + @Test + public void javaSerialization() throws Exception { + DataFileSet dataFiles = DataFileSet.of(ImmutableList.of(FILE_C, FILE_B, FILE_A)); + DataFileSet deserialized = TestHelpers.deserialize(TestHelpers.serialize(dataFiles)); + assertThat(deserialized).isEqualTo(dataFiles); + } +} diff --git a/core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java b/core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java new file mode 100644 index 000000000000..5f4488a3a1d5 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/util/TestDeleteFileSet.java @@ -0,0 +1,321 @@ +/* + * 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 static org.assertj.core.api.Assertions.assertThatThrownBy; + +import java.util.Arrays; +import java.util.Collections; +import java.util.Set; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileMetadata; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.TestHelpers; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Test; + +/** + * Testing {@link DeleteFileSet} is easier in iceberg-core since the delete file builders are + * located here + */ +public class TestDeleteFileSet { + + private static final DeleteFile FILE_A_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-a-deletes.parquet") + .withFileSizeInBytes(1) + .withRecordCount(1) + .build(); + private static final DeleteFile FILE_B_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-b-deletes.parquet") + .withFileSizeInBytes(2) + .withRecordCount(2) + .build(); + private static final DeleteFile FILE_C_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-c-deletes.parquet") + .withFileSizeInBytes(3) + .withRecordCount(3) + .build(); + private static final DeleteFile FILE_D_DELETES = + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath("/path/to/data-d-deletes.parquet") + .withFileSizeInBytes(4) + .withRecordCount(4) + .build(); + + @Test + public void emptySet() { + assertThat(DeleteFileSet.create()).isEmpty(); + assertThat(DeleteFileSet.create()) + .doesNotContain(FILE_A_DELETES, FILE_B_DELETES, FILE_C_DELETES); + } + + @Test + public void insertionOrderIsMaintained() { + DeleteFileSet set = DeleteFileSet.create(); + set.addAll(ImmutableList.of(FILE_D_DELETES, FILE_A_DELETES, FILE_C_DELETES)); + set.add(FILE_B_DELETES); + set.add(FILE_D_DELETES); + + assertThat(set) + .hasSize(4) + .containsExactly(FILE_D_DELETES, FILE_A_DELETES, FILE_C_DELETES, FILE_B_DELETES); + } + + @Test + public void clear() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + set.clear(); + assertThat(set).isEmpty(); + } + + @Test + public void addAll() { + DeleteFileSet empty = DeleteFileSet.create(); + assertThatThrownBy(() -> empty.add(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.addAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.addAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DeleteFileSet set = DeleteFileSet.create(); + set.addAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES, FILE_C_DELETES, FILE_A_DELETES)); + assertThat(set).hasSize(3).containsExactly(FILE_B_DELETES, FILE_A_DELETES, FILE_C_DELETES); + } + + @Test + public void contains() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThatThrownBy(() -> set.contains(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set) + .hasSize(2) + .containsExactly(FILE_A_DELETES, FILE_B_DELETES) + .doesNotContain(FILE_C_DELETES) + .doesNotContain(FILE_D_DELETES); + + assertThatThrownBy( + () -> + DeleteFileSet.of( + Arrays.asList(FILE_C_DELETES, FILE_B_DELETES, null, FILE_A_DELETES))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + } + + @Test + public void containsAll() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThatThrownBy(() -> set.containsAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> set.containsAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> set.containsAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThat(set.containsAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES))).isTrue(); + assertThat(set.containsAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES, FILE_C_DELETES))) + .isFalse(); + assertThat(set.containsAll(ImmutableList.of(FILE_B_DELETES))).isTrue(); + } + + @Test + public void toArray() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES)); + assertThat(set.toArray()).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + + DeleteFile[] array = new DeleteFile[1]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + + array = new DeleteFile[0]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + + array = new DeleteFile[5]; + assertThat(set.toArray(array)) + .hasSize(5) + .containsExactly(FILE_B_DELETES, FILE_A_DELETES, null, null, null); + + array = new DeleteFile[2]; + assertThat(set.toArray(array)).hasSize(2).containsExactly(FILE_B_DELETES, FILE_A_DELETES); + } + + @Test + public void retainAll() { + DeleteFileSet empty = DeleteFileSet.create(); + assertThatThrownBy(() -> empty.retainAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.retainAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.retainAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThat(set.retainAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES, FILE_A_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_A_DELETES); + + set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + + assertThat(set.retainAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.retainAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void remove() { + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThatThrownBy(() -> set.remove(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + set.remove(FILE_C_DELETES); + assertThat(set).containsExactly(FILE_A_DELETES, FILE_B_DELETES); + assertThat(set).containsExactly(FILE_A_DELETES, FILE_B_DELETES); + set.remove(FILE_B_DELETES); + assertThat(set).containsExactly(FILE_A_DELETES); + set.remove(FILE_A_DELETES); + assertThat(set).isEmpty(); + } + + @Test + public void removeAll() { + DeleteFileSet empty = DeleteFileSet.create(); + assertThatThrownBy(() -> empty.removeAll(null)) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid collection: null"); + + assertThatThrownBy(() -> empty.removeAll(Collections.singletonList(null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + assertThatThrownBy(() -> empty.removeAll(Arrays.asList(FILE_A_DELETES, null))) + .isInstanceOf(NullPointerException.class) + .hasMessage("Invalid object: null"); + + DeleteFileSet set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThat(set.removeAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES, FILE_A_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).hasSize(1).containsExactly(FILE_B_DELETES); + + set = DeleteFileSet.of(ImmutableList.of(FILE_A_DELETES, FILE_B_DELETES)); + assertThat(set.removeAll(ImmutableList.of(FILE_C_DELETES, FILE_D_DELETES))) + .as("Set should not have changed") + .isFalse(); + + assertThat(set.removeAll(ImmutableList.of(FILE_B_DELETES, FILE_A_DELETES))) + .as("Set should have changed") + .isTrue(); + + assertThat(set).isEmpty(); + } + + @Test + public void equalsAndHashCode() { + DeleteFileSet set1 = DeleteFileSet.create(); + DeleteFileSet set2 = DeleteFileSet.create(); + + assertThat(set1).isEqualTo(set2); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()); + + set1.add(FILE_A_DELETES); + set1.add(FILE_B_DELETES); + set1.add(FILE_C_DELETES); + + // different DeleteFile instances but all use the same paths as set1 + set2.add( + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(FILE_A_DELETES.location()) + .withFileSizeInBytes(10) + .withRecordCount(1) + .build()); + set2.add( + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(FILE_B_DELETES.location()) + .withFileSizeInBytes(100) + .withRecordCount(10) + .build()); + set2.add( + FileMetadata.deleteFileBuilder(PartitionSpec.unpartitioned()) + .ofPositionDeletes() + .withPath(FILE_C_DELETES.location()) + .withFileSizeInBytes(1000) + .withRecordCount(100) + .build()); + + Set set3 = Collections.unmodifiableSet(set2); + + assertThat(set1).isEqualTo(set2).isEqualTo(set3); + assertThat(set1.hashCode()).isEqualTo(set2.hashCode()).isEqualTo(set3.hashCode()); + } + + @Test + public void kryoSerialization() throws Exception { + DeleteFileSet deleteFiles = + DeleteFileSet.of(ImmutableList.of(FILE_C_DELETES, FILE_B_DELETES, FILE_A_DELETES)); + assertThat(TestHelpers.KryoHelpers.roundTripSerialize(deleteFiles)).isEqualTo(deleteFiles); + } + + @Test + public void javaSerialization() throws Exception { + DeleteFileSet deleteFiles = + DeleteFileSet.of(ImmutableList.of(FILE_C_DELETES, FILE_B_DELETES, FILE_A_DELETES)); + DeleteFileSet deserialize = TestHelpers.deserialize(TestHelpers.serialize(deleteFiles)); + assertThat(deserialize).isEqualTo(deleteFiles); + } +} From 634c6c5a850177aedaa0035cc42a251d123c401b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 1 Oct 2024 08:18:13 +0200 Subject: [PATCH 0739/1019] Build: Bump nessie from 0.97.1 to 0.99.0 (#11224) Bumps `nessie` from 0.97.1 to 0.99.0. Updates `org.projectnessie.nessie:nessie-client` from 0.97.1 to 0.99.0 Updates `org.projectnessie.nessie:nessie-jaxrs-testextension` from 0.97.1 to 0.99.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-inmemory-tests` from 0.97.1 to 0.99.0 Updates `org.projectnessie.nessie:nessie-versioned-storage-testextension` from 0.97.1 to 0.99.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 a726d52dad1e..9ce8ff67d71d 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.1" mockito = "4.11.0" mockserver = "5.15.0" -nessie = "0.97.1" +nessie = "0.99.0" netty-buffer = "4.1.113.Final" netty-buffer-compat = "4.1.113.Final" object-client-bundle = "3.3.2" From a3795fe86829a72844d7f041bcada47a43a66074 Mon Sep 17 00:00:00 2001 From: fengjiajie Date: Wed, 2 Oct 2024 00:38:32 +0800 Subject: [PATCH 0740/1019] ThreadPools introduce newExitingWorkerPool and newFixedThreadPool for clearer semantics (#11073) --- .../org/apache/iceberg/aws/s3/S3FileIO.java | 2 +- .../iceberg/metrics/CountersBenchmark.java | 2 +- .../iceberg/BaseDistributedDataScan.java | 10 +++- .../apache/iceberg/hadoop/HadoopFileIO.java | 3 +- .../org/apache/iceberg/util/ThreadPools.java | 58 +++++++++++++++++-- .../iceberg/data/TableMigrationUtil.java | 19 +++++- .../iceberg/flink/sink/IcebergCommitter.java | 3 +- .../flink/sink/IcebergFilesCommitter.java | 2 +- .../flink/source/FlinkInputFormat.java | 2 +- .../iceberg/flink/source/IcebergSource.java | 2 +- .../source/StreamingMonitorFunction.java | 2 +- .../ContinuousSplitPlannerImpl.java | 2 +- .../iceberg/connect/channel/Coordinator.java | 2 +- .../mr/mapreduce/IcebergInputFormat.java | 2 +- .../iceberg/mr/TestIcebergInputFormats.java | 6 +- .../apache/iceberg/spark/SparkTableUtil.java | 12 +++- 16 files changed, 103 insertions(+), 26 deletions(-) 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 040d3012ca7a..11a5ce02247f 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 @@ -342,7 +342,7 @@ private ExecutorService executorService() { synchronized (S3FileIO.class) { if (executorService == null) { executorService = - ThreadPools.newWorkerPool( + ThreadPools.newExitingWorkerPool( "iceberg-s3fileio-delete", s3FileIOProperties.deleteThreads()); } } diff --git a/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java b/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java index 1350950d4874..628028d9f8e6 100644 --- a/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java +++ b/core/src/jmh/java/org/apache/iceberg/metrics/CountersBenchmark.java @@ -50,7 +50,7 @@ public class CountersBenchmark { public void defaultCounterMultipleThreads(Blackhole blackhole) { Counter counter = new DefaultCounter(Unit.BYTES); - ExecutorService workerPool = ThreadPools.newWorkerPool("bench-pool", WORKER_POOL_SIZE); + ExecutorService workerPool = ThreadPools.newFixedThreadPool("bench-pool", WORKER_POOL_SIZE); try { Tasks.range(WORKER_POOL_SIZE) diff --git a/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java b/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java index 263830a53b3c..89c7f0b6067e 100644 --- a/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java +++ b/core/src/main/java/org/apache/iceberg/BaseDistributedDataScan.java @@ -401,8 +401,14 @@ private boolean mayHaveEqualityDeletes(Snapshot snapshot) { return count == null || !count.equals("0"); } - // a monitor pool that enables planing data and deletes concurrently if remote planning is used + /** + * Creates a monitor pool that enables planing data and deletes concurrently if remote planning is + * used + * + *

    Important: Callers are responsible for shutting down the returned executor service + * when it is no longer needed + */ private ExecutorService newMonitorPool() { - return ThreadPools.newWorkerPool("iceberg-planning-monitor-service", MONITOR_POOL_SIZE); + return ThreadPools.newFixedThreadPool("iceberg-planning-monitor-service", MONITOR_POOL_SIZE); } } 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 7aaa2b6a75b1..60270749b5d7 100644 --- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java +++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopFileIO.java @@ -192,7 +192,8 @@ private ExecutorService executorService() { if (executorService == null) { synchronized (HadoopFileIO.class) { if (executorService == null) { - executorService = ThreadPools.newWorkerPool(DELETE_FILE_POOL_NAME, deleteThreads()); + executorService = + ThreadPools.newExitingWorkerPool(DELETE_FILE_POOL_NAME, deleteThreads()); } } } diff --git a/core/src/main/java/org/apache/iceberg/util/ThreadPools.java b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java index ced121c03c63..0d7d174a10f5 100644 --- a/core/src/main/java/org/apache/iceberg/util/ThreadPools.java +++ b/core/src/main/java/org/apache/iceberg/util/ThreadPools.java @@ -42,13 +42,14 @@ private ThreadPools() {} public static final int WORKER_THREAD_POOL_SIZE = SystemConfigs.WORKER_THREAD_POOL_SIZE.value(); - private static final ExecutorService WORKER_POOL = newWorkerPool("iceberg-worker-pool"); + private static final ExecutorService WORKER_POOL = + newExitingWorkerPool("iceberg-worker-pool", WORKER_THREAD_POOL_SIZE); public static final int DELETE_WORKER_THREAD_POOL_SIZE = SystemConfigs.DELETE_WORKER_THREAD_POOL_SIZE.value(); private static final ExecutorService DELETE_WORKER_POOL = - newWorkerPool("iceberg-delete-worker-pool", DELETE_WORKER_THREAD_POOL_SIZE); + newExitingWorkerPool("iceberg-delete-worker-pool", DELETE_WORKER_THREAD_POOL_SIZE); /** * Return an {@link ExecutorService} that uses the "worker" thread-pool. @@ -81,14 +82,61 @@ public static ExecutorService getDeleteWorkerPool() { return DELETE_WORKER_POOL; } + /** + * Creates a fixed-size thread pool that uses daemon threads. The pool is wrapped with {@link + * MoreExecutors#getExitingExecutorService(ThreadPoolExecutor)}, which registers a shutdown hook + * to ensure the pool terminates when the JVM exits. Important: Even if the pool is + * explicitly shut down using {@link ExecutorService#shutdown()}, the shutdown hook is not + * removed. This can lead to accumulation of shutdown hooks if this method is used repeatedly for + * short-lived thread pools. + * + *

    For clarity and to avoid potential issues with shutdown hook accumulation, prefer using + * either {@link #newExitingWorkerPool(String, int)} or {@link #newFixedThreadPool(String, int)}, + * depending on the intended lifecycle of the thread pool. + * + * @deprecated will be removed in 2.0.0. Use {@link #newExitingWorkerPool(String, int)} for + * long-lived thread pools that require a shutdown hook, or {@link #newFixedThreadPool(String, + * int)} for short-lived thread pools where you manage the lifecycle. + */ + @Deprecated public static ExecutorService newWorkerPool(String namePrefix) { - return newWorkerPool(namePrefix, WORKER_THREAD_POOL_SIZE); + return newExitingWorkerPool(namePrefix, WORKER_THREAD_POOL_SIZE); } + /** + * Creates a fixed-size thread pool that uses daemon threads. The pool is wrapped with {@link + * MoreExecutors#getExitingExecutorService(ThreadPoolExecutor)}, which registers a shutdown hook + * to ensure the pool terminates when the JVM exits. Important: Even if the pool is + * explicitly shut down using {@link ExecutorService#shutdown()}, the shutdown hook is not + * removed. This can lead to accumulation of shutdown hooks if this method is used repeatedly for + * short-lived thread pools. + * + *

    For clarity and to avoid potential issues with shutdown hook accumulation, prefer using + * either {@link #newExitingWorkerPool(String, int)} or {@link #newFixedThreadPool(String, int)}, + * depending on the intended lifecycle of the thread pool. + * + * @deprecated will be removed in 2.0.0. Use {@link #newExitingWorkerPool(String, int)} for + * long-lived thread pools that require a shutdown hook, or {@link #newFixedThreadPool(String, + * int)} for short-lived thread pools where you manage the lifecycle. + */ + @Deprecated public static ExecutorService newWorkerPool(String namePrefix, int poolSize) { + return newExitingWorkerPool(namePrefix, poolSize); + } + + /** + * Creates a fixed-size thread pool that uses daemon threads and registers a shutdown hook to + * ensure the pool terminates when the JVM exits. This is suitable for long-lived thread pools + * that should be automatically cleaned up on JVM shutdown. + */ + public static ExecutorService newExitingWorkerPool(String namePrefix, int poolSize) { return MoreExecutors.getExitingExecutorService( - (ThreadPoolExecutor) - Executors.newFixedThreadPool(poolSize, newDaemonThreadFactory(namePrefix))); + (ThreadPoolExecutor) newFixedThreadPool(namePrefix, poolSize)); + } + + /** Creates a fixed-size thread pool that uses daemon threads. */ + public static ExecutorService newFixedThreadPool(String namePrefix, int poolSize) { + return Executors.newFixedThreadPool(poolSize, newDaemonThreadFactory(namePrefix)); } /** 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 da6f2bfd1152..0602c9e494c7 100644 --- a/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java +++ b/data/src/main/java/org/apache/iceberg/data/TableMigrationUtil.java @@ -97,7 +97,9 @@ public static List listPartition( * @param conf a Hadoop conf * @param metricsSpec a metrics conf * @param mapping a name mapping - * @param parallelism number of threads to use for file reading + * @param parallelism number of threads to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. * @return a List of DataFile */ public static List listPartition( @@ -137,7 +139,9 @@ public static List listPartition( * @param conf a Hadoop conf * @param metricsSpec a metrics conf * @param mapping a name mapping - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. * @return a List of DataFile */ public static List listPartition( @@ -250,7 +254,16 @@ private static DataFile buildDataFile( .build(); } + /** + * Returns an {@link ExecutorService} for table migration. + * + *

    If parallelism is 1, this method returns null, indicating that no executor service is + * needed. Otherwise, it returns a fixed-size thread pool with the given parallelism. + * + *

    Important: Callers are responsible for shutting down the returned executor service + * when it is no longer needed to prevent resource leaks. + */ public static ExecutorService migrationService(int parallelism) { - return parallelism == 1 ? null : ThreadPools.newWorkerPool("table-migration", parallelism); + return parallelism == 1 ? null : ThreadPools.newFixedThreadPool("table-migration", parallelism); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java index 2245b36f15a7..3048b51c6cb0 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergCommitter.java @@ -102,7 +102,7 @@ class IcebergCommitter implements Committer { Preconditions.checkArgument( maxContinuousEmptyCommits > 0, MAX_CONTINUOUS_EMPTY_COMMITS + " must be positive"); this.workerPool = - ThreadPools.newWorkerPool( + ThreadPools.newFixedThreadPool( "iceberg-committer-pool-" + table.name() + "-" + sinkId, workerPoolSize); this.continuousEmptyCheckpoints = 0; } @@ -307,5 +307,6 @@ private void commitOperation( @Override public void close() throws IOException { tableLoader.close(); + workerPool.shutdown(); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java index 609deb621f43..b510dce28bac 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/IcebergFilesCommitter.java @@ -455,7 +455,7 @@ public void open() throws Exception { final String operatorID = getRuntimeContext().getOperatorUniqueID(); this.workerPool = - ThreadPools.newWorkerPool("iceberg-worker-pool-" + operatorID, workerPoolSize); + ThreadPools.newFixedThreadPool("iceberg-worker-pool-" + operatorID, workerPoolSize); } @Override diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java index 9a5123dc489e..a68f0e50e0d0 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/FlinkInputFormat.java @@ -93,7 +93,7 @@ public FlinkInputSplit[] createInputSplits(int minNumSplits) throws IOException // Called in Job manager, so it is OK to load table from catalog. tableLoader.open(); final ExecutorService workerPool = - ThreadPools.newWorkerPool("iceberg-plan-worker-pool", context.planParallelism()); + ThreadPools.newFixedThreadPool("iceberg-plan-worker-pool", context.planParallelism()); try (TableLoader loader = tableLoader) { Table table = loader.loadTable(); return FlinkSplitPlanner.planInputSplits(table, context, workerPool); 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 294ca2f55bc5..ba22c69c70bd 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 @@ -153,7 +153,7 @@ private List planSplitsForBatch(String threadName) { } ExecutorService workerPool = - ThreadPools.newWorkerPool(threadName, scanContext.planParallelism()); + ThreadPools.newFixedThreadPool(threadName, scanContext.planParallelism()); try (TableLoader loader = tableLoader.clone()) { loader.open(); this.batchSplits = diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java index a07613aee59b..39f615aeacc5 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/StreamingMonitorFunction.java @@ -106,7 +106,7 @@ public void open(Configuration parameters) throws Exception { "context should be instance of StreamingRuntimeContext"); final String operatorID = ((StreamingRuntimeContext) runtimeContext).getOperatorUniqueID(); this.workerPool = - ThreadPools.newWorkerPool( + ThreadPools.newFixedThreadPool( "iceberg-worker-pool-" + operatorID, scanContext.planParallelism()); } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java index fef4ec45ed8a..9c99d442174f 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java @@ -64,7 +64,7 @@ public ContinuousSplitPlannerImpl( this.workerPool = isSharedPool ? ThreadPools.getWorkerPool() - : ThreadPools.newWorkerPool( + : ThreadPools.newFixedThreadPool( "iceberg-plan-worker-pool-" + threadName, scanContext.planParallelism()); } 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 2a00928310b2..b09540fdc9f6 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 @@ -90,7 +90,7 @@ class Coordinator extends Channel { this.snapshotOffsetsProp = String.format( "kafka.connect.offsets.%s.%s", config.controlTopic(), config.connectGroupId()); - this.exec = ThreadPools.newWorkerPool("iceberg-committer", config.commitThreads()); + this.exec = ThreadPools.newFixedThreadPool("iceberg-committer", config.commitThreads()); this.commitState = new CommitState(config); } 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 a222080d71db..7ea2d26891f8 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 @@ -108,7 +108,7 @@ public List getSplits(JobContext context) { HiveIcebergStorageHandler.table(conf, conf.get(InputFormatConfig.TABLE_IDENTIFIER))) .orElseGet(() -> Catalogs.loadTable(conf)); final ExecutorService workerPool = - ThreadPools.newWorkerPool( + ThreadPools.newFixedThreadPool( "iceberg-plan-worker-pool", conf.getInt( SystemConfigs.WORKER_THREAD_POOL_SIZE.propertyKey(), 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 86d390ca9ffe..2b93b276ad94 100644 --- a/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java +++ b/mr/src/test/java/org/apache/iceberg/mr/TestIcebergInputFormats.java @@ -393,8 +393,10 @@ public void testWorkerPool() throws Exception { UserGroupInformation.createUserForTesting("user1", new String[] {}); UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", new String[] {}); - final ExecutorService workerPool1 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); - final ExecutorService workerPool2 = ThreadPools.newWorkerPool("iceberg-plan-worker-pool", 1); + final ExecutorService workerPool1 = + ThreadPools.newFixedThreadPool("iceberg-plan-worker-pool", 1); + final ExecutorService workerPool2 = + ThreadPools.newFixedThreadPool("iceberg-plan-worker-pool", 1); try { assertThat(getUserFromWorkerPool(user1, table, workerPool1)).isEqualTo("user1"); assertThat(getUserFromWorkerPool(user2, table, workerPool1)).isEqualTo("user1"); 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 e103104171f6..c44969c49e39 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 @@ -443,7 +443,9 @@ public static void importSparkTable( * @param sourceTableIdent an identifier of the source Spark table * @param targetTable an Iceberg table where to import the data * @param stagingDir a staging directory to store temporary manifest files - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. * If non-null, the provided ExecutorService will be + * shutdown within this method after file reading is complete. */ public static void importSparkTable( SparkSession spark, @@ -501,7 +503,9 @@ public static void importSparkTable( * @param partitionFilter only import partitions whose values match those in the map, can be * partially defined * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. */ public static void importSparkTable( SparkSession spark, @@ -719,7 +723,9 @@ public static void importSparkPartitions( * @param spec a partition spec * @param stagingDir a staging directory to store temporary manifest files * @param checkDuplicateFiles if true, throw exception if import results in a duplicate data file - * @param service executor service to use for file reading + * @param service executor service to use for file reading. If null, file reading will be + * performed on the current thread. If non-null, the provided ExecutorService will be shutdown + * within this method after file reading is complete. */ public static void importSparkPartitions( SparkSession spark, From ce8324bab2c8ae998c1e58b1fb683a76a94377a2 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 1 Oct 2024 10:44:19 -0700 Subject: [PATCH 0741/1019] Core: Deprecate legacy ways for loading position deletes (#11242) --- .../src/main/java/org/apache/iceberg/deletes/Deletes.java | 8 ++++++++ 1 file changed, 8 insertions(+) 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 2256b378f62a..0838e9c0dd6e 100644 --- a/core/src/main/java/org/apache/iceberg/deletes/Deletes.java +++ b/core/src/main/java/org/apache/iceberg/deletes/Deletes.java @@ -173,11 +173,19 @@ 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, From eecb3010005bde5f81b9dcc25d78b8273703fc13 Mon Sep 17 00:00:00 2001 From: Ozan Okumusoglu Date: Tue, 1 Oct 2024 15:28:52 -0700 Subject: [PATCH 0742/1019] AWS: Add configuration and set defaults for S3 retry behaviour (#11052) Co-authored-by: Drew Schleit --- .../aws/AssumeRoleAwsClientFactory.java | 1 + .../iceberg/aws/AwsClientFactories.java | 1 + .../LakeFormationAwsClientFactory.java | 1 + .../s3/DefaultS3FileIOAwsClientFactory.java | 1 + .../iceberg/aws/s3/S3FileIOProperties.java | 124 ++++++++++++++++++ .../aws/s3/TestS3FileIOProperties.java | 15 +++ docs/docs/aws.md | 14 ++ 7 files changed, 157 insertions(+) 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 d9ea511f9bcd..4c3c305d4bae 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AssumeRoleAwsClientFactory.java @@ -47,6 +47,7 @@ public S3Client s3() { .applyMutation(s3FileIOProperties::applyEndpointConfigurations) .applyMutation(s3FileIOProperties::applyServiceConfigurations) .applyMutation(s3FileIOProperties::applySignerConfiguration) + .applyMutation(s3FileIOProperties::applyRetryConfigurations) .build(); } 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 81c7bd6b4bab..5974e21209e3 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java +++ b/aws/src/main/java/org/apache/iceberg/aws/AwsClientFactories.java @@ -114,6 +114,7 @@ public S3Client s3() { .applyMutation(s3FileIOProperties::applySignerConfiguration) .applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations) .applyMutation(s3FileIOProperties::applyUserAgentConfigurations) + .applyMutation(s3FileIOProperties::applyRetryConfigurations) .build(); } 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 552da4bc949c..5d37470066d2 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 @@ -81,6 +81,7 @@ public S3Client s3() { .applyMutation(httpClientProperties()::applyHttpClientConfigurations) .applyMutation(s3FileIOProperties()::applyEndpointConfigurations) .applyMutation(s3FileIOProperties()::applyServiceConfigurations) + .applyMutation(s3FileIOProperties()::applyRetryConfigurations) .credentialsProvider( new LakeFormationCredentialsProvider(lakeFormation(), buildTableArn())) .region(Region.of(region())) diff --git a/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java b/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java index 18b40000a91a..8687d737a5d7 100644 --- a/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java +++ b/aws/src/main/java/org/apache/iceberg/aws/s3/DefaultS3FileIOAwsClientFactory.java @@ -55,6 +55,7 @@ public S3Client s3() { .applyMutation(s3FileIOProperties::applySignerConfiguration) .applyMutation(s3FileIOProperties::applyS3AccessGrantsConfigurations) .applyMutation(s3FileIOProperties::applyUserAgentConfigurations) + .applyMutation(s3FileIOProperties::applyRetryConfigurations) .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 343540d87f95..6813913a4db0 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 @@ -20,10 +20,12 @@ import java.io.Serializable; import java.net.URI; +import java.time.Duration; import java.util.Collections; import java.util.Map; import java.util.Set; import java.util.stream.Collectors; +import javax.xml.stream.XMLStreamException; import org.apache.iceberg.EnvironmentContext; import org.apache.iceberg.aws.AwsClientProperties; import org.apache.iceberg.aws.glue.GlueCatalog; @@ -38,6 +40,14 @@ import software.amazon.awssdk.auth.credentials.AnonymousCredentialsProvider; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.exception.SdkServiceException; +import software.amazon.awssdk.core.retry.RetryMode; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.core.retry.backoff.EqualJitterBackoffStrategy; +import software.amazon.awssdk.core.retry.conditions.OrRetryCondition; +import software.amazon.awssdk.core.retry.conditions.RetryCondition; +import software.amazon.awssdk.core.retry.conditions.RetryOnExceptionsCondition; +import software.amazon.awssdk.core.retry.conditions.TokenBucketRetryCondition; import software.amazon.awssdk.services.s3.S3ClientBuilder; import software.amazon.awssdk.services.s3.S3Configuration; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; @@ -393,6 +403,21 @@ public class S3FileIOProperties implements Serializable { */ private static final String S3_FILE_IO_USER_AGENT = "s3fileio/" + EnvironmentContext.get(); + /** Number of times to retry S3 operations. */ + public static final String S3_RETRY_NUM_RETRIES = "s3.retry.num-retries"; + + public static final int S3_RETRY_NUM_RETRIES_DEFAULT = 5; + + /** Minimum wait time to retry a S3 operation */ + public static final String S3_RETRY_MIN_WAIT_MS = "s3.retry.min-wait-ms"; + + public static final long S3_RETRY_MIN_WAIT_MS_DEFAULT = 2_000; // 2 seconds + + /** Maximum wait time to retry a S3 read operation */ + public static final String S3_RETRY_MAX_WAIT_MS = "s3.retry.max-wait-ms"; + + public static final long S3_RETRY_MAX_WAIT_MS_DEFAULT = 20_000; // 20 seconds + private String sseType; private String sseKey; private String sseMd5; @@ -423,6 +448,9 @@ public class S3FileIOProperties implements Serializable { private final String endpoint; private final boolean isRemoteSigningEnabled; private String writeStorageClass; + private int s3RetryNumRetries; + private long s3RetryMinWaitMs; + private long s3RetryMaxWaitMs; private final Map allProperties; public S3FileIOProperties() { @@ -455,6 +483,9 @@ public S3FileIOProperties() { this.isRemoteSigningEnabled = REMOTE_SIGNING_ENABLED_DEFAULT; this.isS3AccessGrantsEnabled = S3_ACCESS_GRANTS_ENABLED_DEFAULT; this.isS3AccessGrantsFallbackToIamEnabled = S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED_DEFAULT; + this.s3RetryNumRetries = S3_RETRY_NUM_RETRIES_DEFAULT; + this.s3RetryMinWaitMs = S3_RETRY_MIN_WAIT_MS_DEFAULT; + this.s3RetryMaxWaitMs = S3_RETRY_MAX_WAIT_MS_DEFAULT; this.allProperties = Maps.newHashMap(); ValidationException.check( @@ -553,6 +584,12 @@ public S3FileIOProperties(Map properties) { properties, S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED, S3_ACCESS_GRANTS_FALLBACK_TO_IAM_ENABLED_DEFAULT); + this.s3RetryNumRetries = + PropertyUtil.propertyAsInt(properties, S3_RETRY_NUM_RETRIES, S3_RETRY_NUM_RETRIES_DEFAULT); + this.s3RetryMinWaitMs = + 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); ValidationException.check( keyIdAccessKeyBothConfigured(), @@ -753,6 +790,34 @@ public void setS3AccessGrantsFallbackToIamEnabled(boolean s3AccessGrantsFallback this.isS3AccessGrantsFallbackToIamEnabled = s3AccessGrantsFallbackToIamEnabled; } + public int s3RetryNumRetries() { + return s3RetryNumRetries; + } + + public void setS3RetryNumRetries(int s3RetryNumRetries) { + this.s3RetryNumRetries = s3RetryNumRetries; + } + + public long s3RetryMinWaitMs() { + return s3RetryMinWaitMs; + } + + public void setS3RetryMinWaitMs(long s3RetryMinWaitMs) { + this.s3RetryMinWaitMs = s3RetryMinWaitMs; + } + + public long s3RetryMaxWaitMs() { + return s3RetryMaxWaitMs; + } + + public void setS3RetryMaxWaitMs(long s3RetryMaxWaitMs) { + this.s3RetryMaxWaitMs = s3RetryMaxWaitMs; + } + + public long s3RetryTotalWaitMs() { + return (long) s3RetryNumRetries() * s3RetryMaxWaitMs(); + } + private boolean keyIdAccessKeyBothConfigured() { return (accessKeyId == null) == (secretAccessKey == null); } @@ -824,6 +889,65 @@ public void applyEndpointConfigurations(T builder) { } } + /** + * Override the retry configurations for an S3 client. + * + *

    Sample usage: + * + *

    +   *     S3Client.builder().applyMutation(s3FileIOProperties::applyRetryConfigurations)
    +   * 
    + */ + public void applyRetryConfigurations(T builder) { + builder.overrideConfiguration( + config -> + config.retryPolicy( + // Use a retry strategy which will persistently retry throttled exceptions with + // exponential backoff, to give S3 a chance to autoscale. + // LEGACY mode works best here, as it will allow throttled exceptions to use all of + // the configured retry attempts. + RetryPolicy.builder(RetryMode.LEGACY) + .numRetries(s3RetryNumRetries) + .throttlingBackoffStrategy( + EqualJitterBackoffStrategy.builder() + .baseDelay(Duration.ofMillis(s3RetryMinWaitMs)) + .maxBackoffTime(Duration.ofMillis(s3RetryMaxWaitMs)) + .build()) + + // Workaround: add XMLStreamException as a retryable exception. + // https://github.com/aws/aws-sdk-java-v2/issues/5442 + // Without this workaround, we see SDK failures if there's a socket exception + // while parsing an error XML response. + .retryCondition( + OrRetryCondition.create( + RetryCondition.defaultRetryCondition(), + RetryOnExceptionsCondition.create(XMLStreamException.class))) + + // Workaround: exclude all 503s from consuming retry tokens. + // https://github.com/aws/aws-sdk-java-v2/issues/5414 + // Without this workaround, workloads which see 503s from S3 HEAD will fail + // prematurely. + .retryCapacityCondition( + TokenBucketRetryCondition.builder() + .tokenBucketSize(500) // 500 is the SDK default + .exceptionCostFunction( + e -> { + if (e instanceof SdkServiceException) { + SdkServiceException sdkServiceException = + (SdkServiceException) e; + if (sdkServiceException.isThrottlingException() + || sdkServiceException.statusCode() == 503) { + return 0; + } + } + + // 5 is the SDK default for non-throttling exceptions + return 5; + }) + .build()) + .build())); + } + /** * Add the S3 Access Grants Plugin for an S3 client. * diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index f445a2d22422..a61b9efb9fec 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -36,6 +36,8 @@ import org.mockito.ArgumentCaptor; import org.mockito.Mockito; import software.amazon.awssdk.core.client.config.ClientOverrideConfiguration; +import software.amazon.awssdk.core.retry.RetryPolicy; +import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3ClientBuilder; import software.amazon.awssdk.services.s3.S3Configuration; import software.amazon.awssdk.services.s3.model.ObjectCannedACL; @@ -491,4 +493,17 @@ public void testApplyUserAgentConfigurations() { Mockito.verify(mockS3ClientBuilder) .overrideConfiguration(Mockito.any(ClientOverrideConfiguration.class)); } + + @Test + public void testApplyRetryConfiguration() { + Map properties = Maps.newHashMap(); + properties.put(S3FileIOProperties.S3_RETRY_NUM_RETRIES, "999"); + S3FileIOProperties s3FileIOProperties = new S3FileIOProperties(properties); + + S3ClientBuilder builder = S3Client.builder(); + s3FileIOProperties.applyRetryConfigurations(builder); + + RetryPolicy retryPolicy = builder.overrideConfiguration().retryPolicy().get(); + assertThat(retryPolicy.numRetries()).as("retries was not set").isEqualTo(999); + } } diff --git a/docs/docs/aws.md b/docs/docs/aws.md index b9638bb50dad..5a166c0c9193 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -378,6 +378,20 @@ 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. +### S3 Retries + +Workloads which encounter S3 throttling should persistently retry, with exponential backoff, to make progress while S3 +automatically scales. We provide the configurations below to adjust S3 retries for this purpose. For workloads that encounter +throttling and fail due to retry exhaustion, we recommend retry count to set 32 in order allow S3 to auto-scale. Note that +workloads with exceptionally high throughput against tables that S3 has not yet scaled, it may be necessary to increase the retry count further. + + +| Property | Default | Description | +|----------------------|---------|---------------------------------------------------------------------------------------| +| s3.retry.num-retries | 5 | Number of times to retry S3 operations. Recommended 32 for high-throughput workloads. | +| s3.retry.min-wait-ms | 2s | Minimum wait time to retry a S3 operation. | +| s3.retry.max-wait-ms | 20s | Maximum wait time to retry a S3 read operation. | + ### S3 Strong Consistency In November 2020, S3 announced [strong consistency](https://aws.amazon.com/s3/consistency/) for all read operations, and Iceberg is updated to fully leverage this feature. From 0672ae628bf2c64fd09eb2a467a1cabf548955b2 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Thu, 3 Oct 2024 12:49:41 +0200 Subject: [PATCH 0743/1019] Puffin: Document stats `ndv` value representation (#10793) --- format/puffin-spec.md | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/format/puffin-spec.md b/format/puffin-spec.md index 1fe9e5b9bc56..7b4e3e6d96b3 100644 --- a/format/puffin-spec.md +++ b/format/puffin-spec.md @@ -121,7 +121,9 @@ distinct values converted to bytes using Iceberg's single-value serialization. The blob metadata for this blob may include following properties: -- `ndv`: estimate of number of distinct values, derived from the sketch. +- `ndv`: estimate of number of distinct values, derived from the sketch, + stored as non-negative integer value represented using decimal digits + with no leading or trailing spaces. ### Compression codecs From a4b9f076f46c22eef4d40aadcc8d557452f205b0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 3 Oct 2024 03:59:21 -0700 Subject: [PATCH 0744/1019] Build: Bump mkdocs-material from 9.5.34 to 9.5.38 (#11233) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.34 to 9.5.38. - [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.34...9.5.38) --- 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 38c2baa1eb02..d9e513707e33 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.2.0 -mkdocs-material==9.5.34 +mkdocs-material==9.5.38 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 d39b814741a518eef1fc01c29e8513bc2d0ebc40 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 3 Oct 2024 04:08:28 -0700 Subject: [PATCH 0745/1019] Build: Bump org.eclipse.microprofile.openapi:microprofile-openapi-api (#11182) Bumps [org.eclipse.microprofile.openapi:microprofile-openapi-api](https://github.com/eclipse/microprofile-open-api) from 3.1.1 to 3.1.2. - [Release notes](https://github.com/eclipse/microprofile-open-api/releases) - [Commits](https://github.com/eclipse/microprofile-open-api/compare/3.1.1...3.1.2) --- updated-dependencies: - dependency-name: org.eclipse.microprofile.openapi:microprofile-openapi-api 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 9ce8ff67d71d..033e91eef0ed 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -67,7 +67,7 @@ junit = "5.10.1" junit-platform = "1.11.1" kafka = "3.8.0" kryo-shaded = "4.0.3" -microprofile-openapi-api = "3.1.1" +microprofile-openapi-api = "3.1.2" mockito = "4.11.0" mockserver = "5.15.0" nessie = "0.99.0" From 360ee6692d2b43b4c77b639d19393e7b799931cb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 3 Oct 2024 04:52:59 -0700 Subject: [PATCH 0746/1019] Build: Bump org.xerial:sqlite-jdbc from 3.46.1.0 to 3.46.1.3 (#11231) Bumps [org.xerial:sqlite-jdbc](https://github.com/xerial/sqlite-jdbc) from 3.46.1.0 to 3.46.1.3. - [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.0...3.46.1.3) --- 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 033e91eef0ed..69acd34ac598 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -87,7 +87,7 @@ spark-hive34 = "3.4.3" spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.39" -sqlite-jdbc = "3.46.1.0" +sqlite-jdbc = "3.46.1.3" testcontainers = "1.20.1" tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 86d82bd25765bd9a8e2bcf5f7b46d0adedbc32f1 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 4 Oct 2024 12:51:57 +0200 Subject: [PATCH 0747/1019] Build: Update baseline-java 5.69.0 (#11252) * Build: Bump com.palantir.baseline:gradle-baseline-java Bumps [com.palantir.baseline:gradle-baseline-java](https://github.com/palantir/gradle-baseline) from 5.61.0 to 5.69.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.61.0...5.69.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] * Specify locale when building DateTimeFormatter * Specify locale for locale-sensitive String.format `String.format("%s", foo)` is (generally) not local-specific, but formatting numbers with `%d`, `%x`, etc. is. --------- Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../iceberg/expressions/BoundReference.java | 4 ++- .../iceberg/expressions/ExpressionUtil.java | 7 +++-- .../io/BulkDeletionFailureException.java | 4 ++- .../iceberg/transforms/TransformUtil.java | 19 ++++++++++--- .../java/org/apache/iceberg/types/Types.java | 7 ++--- .../org/apache/iceberg/util/DateTimeUtil.java | 3 ++- .../vectorized/parquet/DecimalVectorUtil.java | 5 +++- build.gradle | 2 +- .../iceberg/BaseMetastoreTableOperations.java | 4 ++- .../java/org/apache/iceberg/MetricsModes.java | 2 +- .../java/org/apache/iceberg/ScanSummary.java | 3 ++- .../org/apache/iceberg/SnapshotProducer.java | 7 ++++- .../iceberg/data/avro/IcebergDecoder.java | 4 ++- .../org/apache/iceberg/io/ContentCache.java | 5 +++- .../apache/iceberg/io/OutputFileFactory.java | 2 ++ .../org/apache/iceberg/jdbc/JdbcCatalog.java | 13 +++++++-- .../iceberg/view/BaseViewOperations.java | 4 ++- .../flink/sink/ManifestOutputFileFactory.java | 2 ++ .../shuffle/DataStatisticsCoordinator.java | 27 ++++++++++++++----- .../flink/sink/shuffle/SortKeySerializer.java | 7 +++-- .../iceberg/flink/source/DataIterator.java | 6 ++++- .../enumerator/AbstractIcebergEnumerator.java | 6 ++++- .../source/reader/RecordAndPosition.java | 3 ++- .../split/IcebergSourceSplitSerializer.java | 2 ++ .../flink/sink/ManifestOutputFileFactory.java | 2 ++ .../shuffle/DataStatisticsCoordinator.java | 27 ++++++++++++++----- .../flink/sink/shuffle/SortKeySerializer.java | 7 +++-- .../iceberg/flink/source/DataIterator.java | 6 ++++- .../enumerator/AbstractIcebergEnumerator.java | 6 ++++- .../source/reader/RecordAndPosition.java | 3 ++- .../split/IcebergSourceSplitSerializer.java | 2 ++ .../flink/sink/ManifestOutputFileFactory.java | 2 ++ .../shuffle/DataStatisticsCoordinator.java | 27 ++++++++++++++----- .../flink/sink/shuffle/SortKeySerializer.java | 7 +++-- .../iceberg/flink/source/DataIterator.java | 6 ++++- .../enumerator/AbstractIcebergEnumerator.java | 6 ++++- .../source/reader/RecordAndPosition.java | 3 ++- .../split/IcebergSourceSplitSerializer.java | 2 ++ .../iceberg/connect/KafkaConnectUtils.java | 16 ++++++++--- .../iceberg/connect/data/IcebergWriter.java | 6 ++++- .../iceberg/connect/data/RecordConverter.java | 3 ++- .../iceberg/nessie/NessieIcebergClient.java | 4 ++- .../org/apache/iceberg/orc/ORCSchemaUtil.java | 7 ++++- .../apache/iceberg/parquet/PageIterator.java | 15 +++++++++-- .../apache/iceberg/pig/PigParquetReader.java | 11 ++++++-- .../iceberg/snowflake/NamespaceHelpers.java | 5 +++- .../action/DeleteOrphanFilesBenchmark.java | 3 ++- .../action/DeleteOrphanFilesBenchmark.java | 3 ++- .../action/DeleteOrphanFilesBenchmark.java | 3 ++- 49 files changed, 256 insertions(+), 74 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java b/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java index ca51b1944c66..0ff73632b1d6 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java +++ b/api/src/main/java/org/apache/iceberg/expressions/BoundReference.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.expressions; +import java.util.Locale; import org.apache.iceberg.Accessor; import org.apache.iceberg.StructLike; import org.apache.iceberg.types.Type; @@ -82,6 +83,7 @@ public Accessor accessor() { @Override public String toString() { - return String.format("ref(id=%d, accessor-type=%s)", field.fieldId(), accessor.type()); + return String.format( + Locale.ROOT, "ref(id=%d, accessor-type=%s)", field.fieldId(), accessor.type()); } } 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 43c97c50df46..82d513ced7dd 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java +++ b/api/src/main/java/org/apache/iceberg/expressions/ExpressionUtil.java @@ -23,6 +23,7 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.List; +import java.util.Locale; import java.util.Set; import java.util.concurrent.TimeUnit; import java.util.function.Function; @@ -500,8 +501,10 @@ private static List abbreviateValues(List sanitizedValues) { abbreviatedList.addAll(distinctValues); abbreviatedList.add( String.format( + Locale.ROOT, "... (%d values hidden, %d in total)", - sanitizedValues.size() - distinctValues.size(), sanitizedValues.size())); + sanitizedValues.size() - distinctValues.size(), + sanitizedValues.size())); return abbreviatedList; } } @@ -633,7 +636,7 @@ private static String sanitizeString(CharSequence value, long now, int today) { private static String sanitizeSimpleString(CharSequence value) { // hash the value and return the hash as hex - return String.format("(hash-%08x)", HASH_FUNC.apply(value)); + return String.format(Locale.ROOT, "(hash-%08x)", HASH_FUNC.apply(value)); } private static PartitionSpec identitySpec(Schema schema, int... ids) { diff --git a/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java b/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java index 535be5f64ec8..4f89d462fe94 100644 --- a/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java +++ b/api/src/main/java/org/apache/iceberg/io/BulkDeletionFailureException.java @@ -18,11 +18,13 @@ */ package org.apache.iceberg.io; +import java.util.Locale; + public class BulkDeletionFailureException extends RuntimeException { private final int numberFailedObjects; public BulkDeletionFailureException(int numberFailedObjects) { - super(String.format("Failed to delete %d files", numberFailedObjects)); + super(String.format(Locale.ROOT, "Failed to delete %d files", numberFailedObjects)); this.numberFailedObjects = numberFailedObjects; } diff --git a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java index dd7f97e950e8..710019225e09 100644 --- a/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java +++ b/api/src/main/java/org/apache/iceberg/transforms/TransformUtil.java @@ -26,6 +26,7 @@ import java.time.ZoneOffset; import java.time.temporal.ChronoUnit; import java.util.Base64; +import java.util.Locale; import org.apache.iceberg.util.DateTimeUtil; class TransformUtil { @@ -36,19 +37,25 @@ private TransformUtil() {} private static final int EPOCH_YEAR = EPOCH.getYear(); static String humanYear(int yearOrdinal) { - return String.format("%04d", EPOCH_YEAR + yearOrdinal); + return String.format(Locale.ROOT, "%04d", EPOCH_YEAR + yearOrdinal); } static String humanMonth(int monthOrdinal) { return String.format( + Locale.ROOT, "%04d-%02d", - EPOCH_YEAR + Math.floorDiv(monthOrdinal, 12), 1 + Math.floorMod(monthOrdinal, 12)); + EPOCH_YEAR + Math.floorDiv(monthOrdinal, 12), + 1 + Math.floorMod(monthOrdinal, 12)); } static String humanDay(int dayOrdinal) { OffsetDateTime day = EPOCH.plusDays(dayOrdinal); return String.format( - "%04d-%02d-%02d", day.getYear(), day.getMonth().getValue(), day.getDayOfMonth()); + Locale.ROOT, + "%04d-%02d-%02d", + day.getYear(), + day.getMonth().getValue(), + day.getDayOfMonth()); } static String humanTime(Long microsFromMidnight) { @@ -74,8 +81,12 @@ static String humanTimestampNanoWithoutZone(Long timestampNanos) { static String humanHour(int hourOrdinal) { OffsetDateTime time = EPOCH.plusHours(hourOrdinal); return String.format( + Locale.ROOT, "%04d-%02d-%02d-%02d", - time.getYear(), time.getMonth().getValue(), time.getDayOfMonth(), time.getHour()); + time.getYear(), + time.getMonth().getValue(), + time.getDayOfMonth(), + time.getHour()); } static String base64encode(ByteBuffer buffer) { 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 2352b9b52f13..8c636221bd14 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -372,7 +372,7 @@ public TypeID typeId() { @Override public String toString() { - return String.format("fixed[%d]", length); + return String.format(Locale.ROOT, "fixed[%d]", length); } @Override @@ -443,7 +443,7 @@ public TypeID typeId() { @Override public String toString() { - return String.format("decimal(%d, %d)", precision, scale); + return String.format(Locale.ROOT, "decimal(%d, %d)", precision, scale); } @Override @@ -552,7 +552,8 @@ public String doc() { @Override public String toString() { - return String.format("%d: %s: %s %s", id, name, isOptional ? "optional" : "required", type) + return String.format( + Locale.ROOT, "%d: %s: %s %s", id, name, isOptional ? "optional" : "required", type) + (doc != null ? " (" + doc + ")" : ""); } diff --git a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java index 3c312486be00..e26e7098cb22 100644 --- a/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/DateTimeUtil.java @@ -27,6 +27,7 @@ import java.time.format.DateTimeFormatter; import java.time.format.DateTimeFormatterBuilder; import java.time.temporal.ChronoUnit; +import java.util.Locale; public class DateTimeUtil { private DateTimeUtil() {} @@ -43,7 +44,7 @@ private DateTimeUtil() {} .parseCaseInsensitive() .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) .appendOffset("+HH:MM:ss", "+00:00") - .toFormatter(); + .toFormatter(Locale.ROOT); public static LocalDate dateFromDays(int daysFromEpoch) { return ChronoUnit.DAYS.addTo(EPOCH_DAY, daysFromEpoch); diff --git a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java index f6fe062b561b..aba051860abf 100644 --- a/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java +++ b/arrow/src/main/java/org/apache/iceberg/arrow/vectorized/parquet/DecimalVectorUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.arrow.vectorized.parquet; import java.util.Arrays; +import java.util.Locale; import org.apache.arrow.vector.DecimalVector; import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting; @@ -62,7 +63,9 @@ static byte[] padBigEndianBytes(byte[] bigEndianBytes, int newLength) { } throw new IllegalArgumentException( String.format( + Locale.ROOT, "Buffer size of %d is larger than requested size of %d", - bigEndianBytes.length, newLength)); + bigEndianBytes.length, + newLength)); } } diff --git a/build.gradle b/build.gradle index 620641a21f92..e5fc8e0b5af2 100644 --- a/build.gradle +++ b/build.gradle @@ -27,7 +27,7 @@ buildscript { } dependencies { classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' - classpath 'com.palantir.baseline:gradle-baseline-java:5.61.0' + classpath 'com.palantir.baseline:gradle-baseline-java:5.69.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' diff --git a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java index 5c82bc877a15..53f3250dc95c 100644 --- a/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/BaseMetastoreTableOperations.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg; +import java.util.Locale; import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; @@ -327,7 +328,8 @@ private String newTableMetadataFilePath(TableMetadata meta, int newVersion) { TableProperties.METADATA_COMPRESSION, TableProperties.METADATA_COMPRESSION_DEFAULT); String fileExtension = TableMetadataParser.getFileExtension(codecName); return metadataFileLocation( - meta, String.format("%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); + meta, + String.format(Locale.ROOT, "%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); } /** diff --git a/core/src/main/java/org/apache/iceberg/MetricsModes.java b/core/src/main/java/org/apache/iceberg/MetricsModes.java index 3dc2434310a5..cd5d5eab1273 100644 --- a/core/src/main/java/org/apache/iceberg/MetricsModes.java +++ b/core/src/main/java/org/apache/iceberg/MetricsModes.java @@ -114,7 +114,7 @@ public int length() { @Override public String toString() { - return String.format("truncate(%d)", length); + return String.format(Locale.ROOT, "truncate(%d)", length); } @Override diff --git a/core/src/main/java/org/apache/iceberg/ScanSummary.java b/core/src/main/java/org/apache/iceberg/ScanSummary.java index f975ef1636df..1ea171c5b2c3 100644 --- a/core/src/main/java/org/apache/iceberg/ScanSummary.java +++ b/core/src/main/java/org/apache/iceberg/ScanSummary.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.util.Comparator; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.NavigableMap; import java.util.Set; @@ -346,7 +347,7 @@ public void update(K key, Function updateFunc) { while (map.size() > maxSize) { if (throwIfLimited) { throw new IllegalStateException( - String.format("Too many matching keys: more than %d", maxSize)); + String.format(Locale.ROOT, "Too many matching keys: more than %d", maxSize)); } this.cut = map.lastKey(); map.remove(cut); diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index 22f6ac5e0b74..f750e88e86d9 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -37,6 +37,7 @@ import java.math.RoundingMode; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Queue; import java.util.Set; @@ -505,7 +506,11 @@ protected OutputFile manifestListPath() { ops.metadataFileLocation( FileFormat.AVRO.addExtension( String.format( - "snap-%d-%d-%s", snapshotId(), attempt.incrementAndGet(), commitUUID)))); + Locale.ROOT, + "snap-%d-%d-%s", + snapshotId(), + attempt.incrementAndGet(), + commitUUID)))); } protected EncryptedOutputFile newManifestOutputFile() { 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 94fedbcfce91..89513b7e0bed 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 @@ -23,6 +23,7 @@ import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.util.Locale; import java.util.Map; import org.apache.avro.AvroRuntimeException; import org.apache.avro.Schema; @@ -138,7 +139,8 @@ public D decode(InputStream stream, D reuse) throws IOException { if (IcebergEncoder.V1_HEADER[0] != header[0] || IcebergEncoder.V1_HEADER[1] != header[1]) { throw new BadHeaderException( - String.format("Unrecognized header bytes: 0x%02X 0x%02X", header[0], header[1])); + String.format( + Locale.ROOT, "Unrecognized header bytes: 0x%02X 0x%02X", header[0], header[1])); } RawDecoder decoder = getDecoder(FP_BUFFER.get().getLong(2)); 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 ce37cfb08934..c8733a904d90 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -28,6 +28,7 @@ import java.nio.ByteBuffer; import java.time.Duration; import java.util.List; +import java.util.Locale; import org.apache.iceberg.exceptions.NotFoundException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; @@ -253,8 +254,10 @@ private static FileContent download(InputFile input) { // IOException and let the caller fallback to non-caching input file. throw new IOException( String.format( + Locale.ROOT, "Failed to read %d bytes: %d bytes in stream", - fileLength, fileLength - totalBytesToRead)); + fileLength, + fileLength - totalBytesToRead)); } else { buffers.add(ByteBuffer.wrap(buf)); } diff --git a/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java b/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java index c46ca132ed0b..50e84143ffb7 100644 --- a/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java +++ b/core/src/main/java/org/apache/iceberg/io/OutputFileFactory.java @@ -21,6 +21,7 @@ import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -92,6 +93,7 @@ public static Builder builderFor(Table table, int partitionId, long taskId) { private String generateFilename() { return format.addExtension( String.format( + Locale.ROOT, "%05d-%d-%s-%05d%s", partitionId, taskId, diff --git a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java index 5e985faafbed..1cce0e2ce77e 100644 --- a/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java +++ b/core/src/main/java/org/apache/iceberg/jdbc/JdbcCatalog.java @@ -31,6 +31,7 @@ import java.util.AbstractMap; import java.util.Arrays; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; import java.util.function.Consumer; @@ -798,7 +799,11 @@ private boolean insertProperties(Namespace namespace, Map proper } throw new IllegalStateException( - String.format("Failed to insert: %d of %d succeeded", insertedRecords, properties.size())); + String.format( + Locale.ROOT, + "Failed to insert: %d of %d succeeded", + insertedRecords, + properties.size())); } private boolean updateProperties(Namespace namespace, Map properties) { @@ -818,7 +823,11 @@ private boolean updateProperties(Namespace namespace, Map proper } throw new IllegalStateException( - String.format("Failed to update: %d of %d succeeded", updatedRecords, properties.size())); + String.format( + Locale.ROOT, + "Failed to update: %d of %d succeeded", + updatedRecords, + properties.size())); } private boolean deleteProperties(Namespace namespace, Set properties) { diff --git a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java index df96b90eb728..d78aeea6aae5 100644 --- a/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java +++ b/core/src/main/java/org/apache/iceberg/view/BaseViewOperations.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.view; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; import java.util.function.Function; @@ -157,7 +158,8 @@ private String newMetadataFilePath(ViewMetadata metadata, int newVersion) { ViewProperties.METADATA_COMPRESSION, ViewProperties.METADATA_COMPRESSION_DEFAULT); String fileExtension = TableMetadataParser.getFileExtension(codecName); return metadataFileLocation( - metadata, String.format("%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); + metadata, + String.format(Locale.ROOT, "%05d-%s%s", newVersion, UUID.randomUUID(), fileExtension)); } private String metadataFileLocation(ViewMetadata metadata, String filename) { diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index da5e6e7627ae..47a74a2018fb 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -59,6 +60,7 @@ class ManifestOutputFileFactory { private String generatePath(long checkpointId) { return FileFormat.AVRO.addExtension( String.format( + Locale.ROOT, "%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, 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 4bfde7204acf..e2a282efd82e 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Comparator; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,8 +266,10 @@ private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); + operatorName, + statistics.checkpointId())); } @SuppressWarnings("FutureReturnValueIgnored") @@ -290,8 +293,11 @@ private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatis } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); + operatorName, + subtask, + globalStatistics.checkpointId())); } else { LOG.info( "Ignore global statistics request from subtask {} as statistics not available", subtask); @@ -318,8 +324,11 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven } }, String.format( + Locale.ROOT, "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); + event.getClass(), + subtask, + attemptNumber)); } @Override @@ -339,7 +348,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r completedStatistics, completedStatisticsSerializer)); } }, - String.format("taking checkpoint %d", checkpointId)); + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); } @Override @@ -381,7 +390,8 @@ public void subtaskReset(int subtask, long checkpointId) { this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.reset(subtask); }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override @@ -397,7 +407,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override @@ -411,7 +421,10 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway subtaskGateways.registerSubtaskGateway(gateway); }, String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); } @VisibleForTesting 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 d1d75019fa2e..61c6973463ef 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 @@ -23,6 +23,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -170,7 +171,8 @@ public void serialize(SortKey record, DataOutputView target) throws IOException default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } } @@ -239,7 +241,8 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index 91d975349b19..3beda960cec8 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Iterator; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; @@ -95,8 +96,11 @@ public void seek(int startingFileOffset, long startingRecordOffset) { } else { throw new IllegalStateException( String.format( + Locale.ROOT, "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); + startingRecordOffset, + startingFileOffset, + combinedTask)); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 280a126a46ce..fc310606dee9 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -76,6 +77,7 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname // Iceberg source uses custom split request event to piggyback finished split ids. throw new UnsupportedOperationException( String.format( + Locale.ROOT, "Received invalid default split request event " + "from subtask %d as Iceberg source uses custom split request event", subtaskId)); @@ -92,8 +94,10 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { throw new IllegalArgumentException( String.format( + Locale.ROOT, "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); + subtaskId, + sourceEvent.getClass().getCanonicalName())); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java index 6ac92592b6aa..10e7d2037a30 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.util.Locale; import org.apache.flink.annotation.Internal; /** @@ -73,6 +74,6 @@ public void record(T nextRecord) { @Override public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + return String.format(Locale.ROOT, "%s @ %d + %d", record, fileOffset, recordOffset); } } diff --git a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index d4b0f9e1977d..d90d1dc88c91 100644 --- a/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.18/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.split; import java.io.IOException; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -54,6 +55,7 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE default: throw new IOException( String.format( + Locale.ROOT, "Failed to deserialize IcebergSourceSplit. " + "Encountered unsupported version: %d. Supported version are [1]", version)); diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 1cb7f4dea1e8..30517cd38216 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -59,6 +60,7 @@ class ManifestOutputFileFactory { private String generatePath(long checkpointId) { return FileFormat.AVRO.addExtension( String.format( + Locale.ROOT, "%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, 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 4bfde7204acf..e2a282efd82e 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Comparator; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,8 +266,10 @@ private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); + operatorName, + statistics.checkpointId())); } @SuppressWarnings("FutureReturnValueIgnored") @@ -290,8 +293,11 @@ private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatis } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); + operatorName, + subtask, + globalStatistics.checkpointId())); } else { LOG.info( "Ignore global statistics request from subtask {} as statistics not available", subtask); @@ -318,8 +324,11 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven } }, String.format( + Locale.ROOT, "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); + event.getClass(), + subtask, + attemptNumber)); } @Override @@ -339,7 +348,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r completedStatistics, completedStatisticsSerializer)); } }, - String.format("taking checkpoint %d", checkpointId)); + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); } @Override @@ -381,7 +390,8 @@ public void subtaskReset(int subtask, long checkpointId) { this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.reset(subtask); }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override @@ -397,7 +407,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override @@ -411,7 +421,10 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway subtaskGateways.registerSubtaskGateway(gateway); }, String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); } @VisibleForTesting 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 b82fc8250763..b3e536bdde52 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 @@ -23,6 +23,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -169,7 +170,8 @@ public void serialize(SortKey record, DataOutputView target) throws IOException default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } } @@ -238,7 +240,8 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index 91d975349b19..3beda960cec8 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Iterator; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; @@ -95,8 +96,11 @@ public void seek(int startingFileOffset, long startingRecordOffset) { } else { throw new IllegalStateException( String.format( + Locale.ROOT, "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); + startingRecordOffset, + startingFileOffset, + combinedTask)); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 280a126a46ce..fc310606dee9 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -76,6 +77,7 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname // Iceberg source uses custom split request event to piggyback finished split ids. throw new UnsupportedOperationException( String.format( + Locale.ROOT, "Received invalid default split request event " + "from subtask %d as Iceberg source uses custom split request event", subtaskId)); @@ -92,8 +94,10 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { throw new IllegalArgumentException( String.format( + Locale.ROOT, "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); + subtaskId, + sourceEvent.getClass().getCanonicalName())); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java index 6ac92592b6aa..10e7d2037a30 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.util.Locale; import org.apache.flink.annotation.Internal; /** @@ -73,6 +74,6 @@ public void record(T nextRecord) { @Override public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + return String.format(Locale.ROOT, "%s @ %d + %d", record, fileOffset, recordOffset); } } diff --git a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index d4b0f9e1977d..d90d1dc88c91 100644 --- a/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.19/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.split; import java.io.IOException; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -54,6 +55,7 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE default: throw new IOException( String.format( + Locale.ROOT, "Failed to deserialize IcebergSourceSplit. " + "Encountered unsupported version: %d. Supported version are [1]", version)); diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java index 1cb7f4dea1e8..30517cd38216 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/sink/ManifestOutputFileFactory.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.sink; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Supplier; @@ -59,6 +60,7 @@ class ManifestOutputFileFactory { private String generatePath(long checkpointId) { return FileFormat.AVRO.addExtension( String.format( + Locale.ROOT, "%s-%s-%05d-%d-%d-%05d", flinkJobId, operatorUniqueId, 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 4bfde7204acf..e2a282efd82e 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 @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.sink.shuffle; import java.util.Comparator; +import java.util.Locale; import java.util.Map; import java.util.concurrent.Callable; import java.util.concurrent.CompletableFuture; @@ -265,8 +266,10 @@ private void sendGlobalStatisticsToSubtasks(GlobalStatistics statistics) { } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics for checkpoint %d", - operatorName, statistics.checkpointId())); + operatorName, + statistics.checkpointId())); } @SuppressWarnings("FutureReturnValueIgnored") @@ -290,8 +293,11 @@ private void handleRequestGlobalStatisticsEvent(int subtask, RequestGlobalStatis } }, String.format( + Locale.ROOT, "Failed to send operator %s coordinator global data statistics to requesting subtask %d for checkpoint %d", - operatorName, subtask, globalStatistics.checkpointId())); + operatorName, + subtask, + globalStatistics.checkpointId())); } else { LOG.info( "Ignore global statistics request from subtask {} as statistics not available", subtask); @@ -318,8 +324,11 @@ public void handleEventFromOperator(int subtask, int attemptNumber, OperatorEven } }, String.format( + Locale.ROOT, "handling operator event %s from subtask %d (#%d)", - event.getClass(), subtask, attemptNumber)); + event.getClass(), + subtask, + attemptNumber)); } @Override @@ -339,7 +348,7 @@ public void checkpointCoordinator(long checkpointId, CompletableFuture r completedStatistics, completedStatisticsSerializer)); } }, - String.format("taking checkpoint %d", checkpointId)); + String.format(Locale.ROOT, "taking checkpoint %d", checkpointId)); } @Override @@ -381,7 +390,8 @@ public void subtaskReset(int subtask, long checkpointId) { this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.reset(subtask); }, - String.format("handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); + String.format( + Locale.ROOT, "handling subtask %d recovery to checkpoint %d", subtask, checkpointId)); } @Override @@ -397,7 +407,7 @@ public void executionAttemptFailed(int subtask, int attemptNumber, @Nullable Thr this.coordinatorThreadFactory.isCurrentThreadCoordinatorThread()); subtaskGateways.unregisterSubtaskGateway(subtask, attemptNumber); }, - String.format("handling subtask %d (#%d) failure", subtask, attemptNumber)); + String.format(Locale.ROOT, "handling subtask %d (#%d) failure", subtask, attemptNumber)); } @Override @@ -411,7 +421,10 @@ public void executionAttemptReady(int subtask, int attemptNumber, SubtaskGateway subtaskGateways.registerSubtaskGateway(gateway); }, String.format( - "making event gateway to subtask %d (#%d) available", subtask, attemptNumber)); + Locale.ROOT, + "making event gateway to subtask %d (#%d) available", + subtask, + attemptNumber)); } @VisibleForTesting 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 b82fc8250763..b3e536bdde52 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 @@ -23,6 +23,7 @@ import java.math.BigInteger; import java.nio.ByteBuffer; import java.util.List; +import java.util.Locale; import java.util.Objects; import java.util.Set; import java.util.UUID; @@ -169,7 +170,8 @@ public void serialize(SortKey record, DataOutputView target) throws IOException default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } } @@ -238,7 +240,8 @@ public SortKey deserialize(SortKey reuse, DataInputView source) throws IOExcepti default: // SortKey transformation is a flattened struct without list and map throw new UnsupportedOperationException( - String.format("Field %d has unsupported field type: %s", fieldId, typeId)); + String.format( + Locale.ROOT, "Field %d has unsupported field type: %s", fieldId, typeId)); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java index 91d975349b19..3beda960cec8 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/DataIterator.java @@ -21,6 +21,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.util.Iterator; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.iceberg.CombinedScanTask; import org.apache.iceberg.FileScanTask; @@ -95,8 +96,11 @@ public void seek(int startingFileOffset, long startingRecordOffset) { } else { throw new IllegalStateException( String.format( + Locale.ROOT, "Invalid starting record offset %d for file %d from CombinedScanTask: %s", - startingRecordOffset, startingFileOffset, combinedTask)); + startingRecordOffset, + startingFileOffset, + combinedTask)); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java index 280a126a46ce..fc310606dee9 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/AbstractIcebergEnumerator.java @@ -22,6 +22,7 @@ import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicReference; @@ -76,6 +77,7 @@ public void handleSplitRequest(int subtaskId, @Nullable String requesterHostname // Iceberg source uses custom split request event to piggyback finished split ids. throw new UnsupportedOperationException( String.format( + Locale.ROOT, "Received invalid default split request event " + "from subtask %d as Iceberg source uses custom split request event", subtaskId)); @@ -92,8 +94,10 @@ public void handleSourceEvent(int subtaskId, SourceEvent sourceEvent) { } else { throw new IllegalArgumentException( String.format( + Locale.ROOT, "Received unknown event from subtask %d: %s", - subtaskId, sourceEvent.getClass().getCanonicalName())); + subtaskId, + sourceEvent.getClass().getCanonicalName())); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java index 6ac92592b6aa..10e7d2037a30 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/reader/RecordAndPosition.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.flink.source.reader; +import java.util.Locale; import org.apache.flink.annotation.Internal; /** @@ -73,6 +74,6 @@ public void record(T nextRecord) { @Override public String toString() { - return String.format("%s @ %d + %d", record, fileOffset, recordOffset); + return String.format(Locale.ROOT, "%s @ %d + %d", record, fileOffset, recordOffset); } } diff --git a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java index d4b0f9e1977d..d90d1dc88c91 100644 --- a/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java +++ b/flink/v1.20/flink/src/main/java/org/apache/iceberg/flink/source/split/IcebergSourceSplitSerializer.java @@ -19,6 +19,7 @@ package org.apache.iceberg.flink.source.split; import java.io.IOException; +import java.util.Locale; import org.apache.flink.annotation.Internal; import org.apache.flink.core.io.SimpleVersionedSerializer; @@ -54,6 +55,7 @@ public IcebergSourceSplit deserialize(int version, byte[] serialized) throws IOE default: throw new IOException( String.format( + Locale.ROOT, "Failed to deserialize IcebergSourceSplit. " + "Encountered unsupported version: %d. Supported version are [1]", version)); diff --git a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java index 098ab2395b34..511923683da8 100644 --- a/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java +++ b/kafka-connect/kafka-connect-runtime/src/integration/java/org/apache/iceberg/connect/KafkaConnectUtils.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.databind.node.ArrayNode; import java.io.IOException; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeUnit; import org.apache.hc.client5.http.classic.HttpClient; @@ -66,7 +67,9 @@ public Config config(String key, Object value) { public static void startConnector(Config config) { try { HttpPost request = - new HttpPost(String.format("http://localhost:%d/connectors", TestContext.CONNECT_PORT)); + new HttpPost( + String.format( + Locale.ROOT, "http://localhost:%d/connectors", TestContext.CONNECT_PORT)); String body = TestContext.MAPPER.writeValueAsString(config); request.setHeader("Content-Type", "application/json"); request.setEntity(new StringEntity(body)); @@ -80,7 +83,10 @@ public static void ensureConnectorRunning(String name) { HttpGet request = new HttpGet( String.format( - "http://localhost:%d/connectors/%s/status", TestContext.CONNECT_PORT, name)); + Locale.ROOT, + "http://localhost:%d/connectors/%s/status", + TestContext.CONNECT_PORT, + name)); Awaitility.await() .atMost(60, TimeUnit.SECONDS) .until( @@ -106,7 +112,11 @@ public static void stopConnector(String name) { try { HttpDelete request = new HttpDelete( - String.format("http://localhost:%d/connectors/%s", TestContext.CONNECT_PORT, name)); + String.format( + Locale.ROOT, + "http://localhost:%d/connectors/%s", + TestContext.CONNECT_PORT, + name)); HTTP.execute(request, response -> null); } catch (IOException e) { throw new RuntimeException(e); diff --git a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java index 6df6b091510b..b5be5b3a0047 100644 --- a/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java +++ b/kafka-connect/kafka-connect/src/main/java/org/apache/iceberg/connect/data/IcebergWriter.java @@ -22,6 +22,7 @@ import java.io.UncheckedIOException; import java.util.Arrays; import java.util.List; +import java.util.Locale; import org.apache.iceberg.Table; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.connect.IcebergSinkConfig; @@ -65,8 +66,11 @@ public void write(SinkRecord record) { } catch (Exception e) { throw new DataException( String.format( + Locale.ROOT, "An error occurred converting record, topic: %s, partition, %d, offset: %d", - record.topic(), record.kafkaPartition(), record.kafkaOffset()), + record.topic(), + record.kafkaPartition(), + record.kafkaOffset()), e); } } 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 f827d2fd63f4..a312e69001c3 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 @@ -37,6 +37,7 @@ import java.util.Base64; import java.util.Date; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.UUID; import java.util.stream.Collectors; @@ -71,7 +72,7 @@ class RecordConverter { new DateTimeFormatterBuilder() .append(DateTimeFormatter.ISO_LOCAL_DATE_TIME) .appendOffset("+HHmm", "Z") - .toFormatter(); + .toFormatter(Locale.ROOT); private final Schema tableSchema; private final NameMapping nameMapping; diff --git a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java index 8bf6bc115d13..f51aa62d02a6 100644 --- a/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java +++ b/nessie/src/main/java/org/apache/iceberg/nessie/NessieIcebergClient.java @@ -272,8 +272,10 @@ public List listNamespaces(Namespace namespace) throws NoSuchNamespac org.projectnessie.model.Namespace.of(namespace.levels()); filter += String.format( + Locale.ROOT, "size(entry.keyElements) == %d && entry.encodedKey.startsWith('%s.')", - root.getElementCount() + 1, root.name()); + root.getElementCount() + 1, + root.name()); } List entries = withReference(api.getEntries()).filter(filter).stream() diff --git a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java index a573d6874851..89a1632d5a1f 100644 --- a/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java +++ b/orc/src/main/java/org/apache/iceberg/orc/ORCSchemaUtil.java @@ -19,6 +19,7 @@ package org.apache.iceberg.orc; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -327,7 +328,11 @@ private static TypeDescription buildOrcProjection( } else { if (isRequired) { throw new IllegalArgumentException( - String.format("Field %d of type %s is required and was not found.", fieldId, type)); + String.format( + Locale.ROOT, + "Field %d of type %s is required and was not found.", + fieldId, + type)); } orcType = convert(fieldId, type, false); diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java b/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java index 34383352bf68..bff13603002f 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/PageIterator.java @@ -19,6 +19,7 @@ package org.apache.iceberg.parquet; import java.io.IOException; +import java.util.Locale; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.parquet.CorruptDeltaByteArrays; import org.apache.parquet.bytes.ByteBufferInputStream; @@ -211,16 +212,26 @@ RuntimeException handleRuntimeException(RuntimeException exception) { "Read failure possibly due to " + "PARQUET-246: try setting parquet.split.files to false", new ParquetDecodingException( String.format( + Locale.ROOT, "Can't read value in column %s at value %d out of %d in current page. " + "repetition level: %d, definition level: %d", - desc, triplesRead, triplesCount, currentRL, currentDL), + desc, + triplesRead, + triplesCount, + currentRL, + currentDL), exception)); } throw new ParquetDecodingException( String.format( + Locale.ROOT, "Can't read value in column %s at value %d out of %d in current page. " + "repetition level: %d, definition level: %d", - desc, triplesRead, triplesCount, currentRL, currentDL), + desc, + triplesRead, + triplesCount, + currentRL, + currentDL), exception); } 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 48f15e05dd70..35b061db4f9e 100644 --- a/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java +++ b/pig/src/main/java/org/apache/iceberg/pig/PigParquetReader.java @@ -24,6 +24,7 @@ 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; @@ -304,7 +305,11 @@ private static class DateReader extends PrimitiveReader { public String read(String reuse) { OffsetDateTime day = EPOCH.plusDays(column.nextInteger()); return String.format( - "%04d-%02d-%02d", day.getYear(), day.getMonth().getValue(), day.getDayOfMonth()); + Locale.ROOT, + "%04d-%02d-%02d", + day.getYear(), + day.getMonth().getValue(), + day.getDayOfMonth()); } } @@ -437,7 +442,9 @@ protected void set(Tuple tuple, int pos, Object value) { tuple.set(pos, value); } catch (ExecException e) { throw new RuntimeException( - String.format("Error setting tuple value for pos: %d, value: %s", pos, value), e); + String.format( + Locale.ROOT, "Error setting tuple value for pos: %d, value: %s", pos, value), + e); } } } diff --git a/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java b/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java index 28dacbca9817..24bd0e75aee2 100644 --- a/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java +++ b/snowflake/src/main/java/org/apache/iceberg/snowflake/NamespaceHelpers.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.snowflake; +import java.util.Locale; import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableIdentifier; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; @@ -47,8 +48,10 @@ public static SnowflakeIdentifier toSnowflakeIdentifier(Namespace namespace) { default: throw new IllegalArgumentException( String.format( + Locale.ROOT, "Snowflake max namespace level is %d, got namespace '%s'", - MAX_NAMESPACE_DEPTH, namespace)); + MAX_NAMESPACE_DEPTH, + namespace)); } } diff --git a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index 5a7df7283728..b6ade2bff3e1 100644 --- a/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v3.3/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.util.List; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.iceberg.AppendFiles; @@ -124,7 +125,7 @@ private void appendData() { for (int i = 0; i < NUM_SNAPSHOTS; i++) { AppendFiles appendFiles = table().newFastAppend(); for (int j = 0; j < NUM_FILES; j++) { - String path = String.format("%s/path/to/data-%d-%d.parquet", location, i, j); + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); validAndOrphanPaths.add(path); DataFile dataFile = DataFiles.builder(partitionSpec) diff --git a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java index 5a7df7283728..b6ade2bff3e1 100644 --- a/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java +++ b/spark/v3.4/spark/src/jmh/java/org/apache/iceberg/spark/action/DeleteOrphanFilesBenchmark.java @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.util.List; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.iceberg.AppendFiles; @@ -124,7 +125,7 @@ private void appendData() { for (int i = 0; i < NUM_SNAPSHOTS; i++) { AppendFiles appendFiles = table().newFastAppend(); for (int j = 0; j < NUM_FILES; j++) { - String path = String.format("%s/path/to/data-%d-%d.parquet", location, i, j); + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); validAndOrphanPaths.add(path); DataFile dataFile = DataFiles.builder(partitionSpec) 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 c012d7a2af1f..fd0dcac6bbcc 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 @@ -22,6 +22,7 @@ import java.sql.Timestamp; import java.util.List; +import java.util.Locale; import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.iceberg.AppendFiles; @@ -124,7 +125,7 @@ private void appendData() { for (int i = 0; i < NUM_SNAPSHOTS; i++) { AppendFiles appendFiles = table().newFastAppend(); for (int j = 0; j < NUM_FILES; j++) { - String path = String.format("%s/path/to/data-%d-%d.parquet", location, i, j); + String path = String.format(Locale.ROOT, "%s/path/to/data-%d-%d.parquet", location, i, j); validAndOrphanPaths.add(path); DataFile dataFile = DataFiles.builder(partitionSpec) From 93b57d10307f22d8e2dfa4fba05664fc4301534a Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Fri, 4 Oct 2024 12:53:06 +0200 Subject: [PATCH 0748/1019] Build: Forbid implicit case fall-through without a comment and enable couple more recommendable error-prone checks (#11251) * Forbid implicit case fall-through without a comment In ordinary switch statement, case branches implicitly fall-through to the next one. This is Java's C legacy. Forbid this unless intent is indicated in the code with a comment like `// fall through`. * Enable couple more recommendable error-prone checks --- baseline.gradle | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/baseline.gradle b/baseline.gradle index d29100aa3f31..07889d4e348c 100644 --- a/baseline.gradle +++ b/baseline.gradle @@ -83,6 +83,8 @@ subprojects { '-Xep:BadComparable:ERROR', '-Xep:BadInstanceof:ERROR', '-Xep:CatchFail:ERROR', + '-Xep:ClassCanBeStatic:ERROR', + '-Xep:ClassNewInstance:ERROR', '-Xep:CollectionUndefinedEquality:ERROR', // specific to Palantir - Uses name `log` but we use name `LOG` '-Xep:ConsistentLoggerName:OFF', @@ -95,8 +97,10 @@ subprojects { '-Xep:EqualsUnsafeCast:ERROR', '-Xep:EqualsUsingHashCode:ERROR', '-Xep:ExtendsObject:ERROR', + '-Xep:FallThrough:ERROR', // specific to Palantir '-Xep:FinalClass:OFF', + '-Xep:Finalize:ERROR', '-Xep:FormatStringAnnotation:ERROR', '-Xep:GetClassOnEnum:ERROR', '-Xep:HidingField:ERROR', @@ -146,10 +150,14 @@ subprojects { '-Xep:StringSplitter:ERROR', '-Xep:TypeParameterShadowing:OFF', '-Xep:TypeParameterUnusedInFormals:OFF', + '-Xep:UnicodeEscape:ERROR', // Palantir's UnnecessarilyQualified may throw during analysis '-Xep:UnnecessarilyQualified:OFF', + '-Xep:UnnecessaryLongToIntConversion:ERROR', + '-Xep:UnnecessaryMethodReference:ERROR', '-Xep:UnusedMethod:ERROR', '-Xep:UnusedVariable:ERROR', + '-Xep:UseEnumSwitch:ERROR', ) } } From 0ade37f5256c364d839334a6ecaed63a6131400a Mon Sep 17 00:00:00 2001 From: Walaa Eldin Moustafa Date: Fri, 4 Oct 2024 14:56:33 -0700 Subject: [PATCH 0749/1019] API, Core: Add default value APIs and Avro implementation (#9502) Co-authored-by: Walaa Eldin Moustafa Co-authored-by: Ryan Blue --- .palantir/revapi.yml | 5 + .../iceberg/expressions/Expressions.java | 12 ++ .../java/org/apache/iceberg/types/Types.java | 124 +++++++++++-- .../iceberg/avro/GenericAvroReader.java | 14 +- .../iceberg/avro/TestReadDefaultValues.java | 166 ++++++++++++++++++ 5 files changed, 304 insertions(+), 17 deletions(-) create mode 100644 core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 05ec59226fdd..fade79326a49 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -873,6 +873,11 @@ acceptedBreaks: new: "method void org.apache.iceberg.encryption.Ciphers::()" justification: "Static utility class - should not have public constructor" "1.4.0": + org.apache.iceberg:iceberg-api: + - code: "java.class.defaultSerializationChanged" + old: "class org.apache.iceberg.types.Types.NestedField" + new: "class org.apache.iceberg.types.Types.NestedField" + justification: "Add default value APIs." org.apache.iceberg:iceberg-core: - code: "java.class.defaultSerializationChanged" old: "class org.apache.iceberg.PartitionData" diff --git a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java index f21a7705968b..deeba664ec07 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/Expressions.java +++ b/api/src/main/java/org/apache/iceberg/expressions/Expressions.java @@ -309,6 +309,18 @@ public static UnboundTerm transform(String name, Transform transfor return new UnboundTransform<>(ref(name), transform); } + /** + * Create a {@link Literal} from an Object. + * + * @param value a value + * @param Java type of value + * @return a Literal for the given value + * @throws IllegalArgumentException if the value has no literal implementation + */ + public static Literal lit(T value) { + return Literals.from(value); + } + public static UnboundAggregate count(String name) { return new UnboundAggregate<>(Operation.COUNT, ref(name)); } 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 8c636221bd14..4bb1674f3be5 100644 --- a/api/src/main/java/org/apache/iceberg/types/Types.java +++ b/api/src/main/java/org/apache/iceberg/types/Types.java @@ -27,6 +27,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; import org.apache.iceberg.Schema; +import org.apache.iceberg.expressions.Expressions; 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; @@ -469,27 +470,94 @@ public int hashCode() { public static class NestedField implements Serializable { public static NestedField optional(int id, String name, Type type) { - return new NestedField(true, id, name, type, null); + return new NestedField(true, id, name, type, null, null, null); } public static NestedField optional(int id, String name, Type type, String doc) { - return new NestedField(true, id, name, type, doc); + return new NestedField(true, id, name, type, doc, null, null); } public static NestedField required(int id, String name, Type type) { - return new NestedField(false, id, name, type, null); + return new NestedField(false, id, name, type, null, null, null); } public static NestedField required(int id, String name, Type type, String doc) { - return new NestedField(false, id, name, type, doc); + return new NestedField(false, id, name, type, doc, null, null); } public static NestedField of(int id, boolean isOptional, String name, Type type) { - return new NestedField(isOptional, id, name, type, null); + return new NestedField(isOptional, id, name, type, null, null, null); } public static NestedField of(int id, boolean isOptional, String name, Type type, String doc) { - return new NestedField(isOptional, id, name, type, doc); + return new NestedField(isOptional, id, name, type, doc, null, null); + } + + public static Builder from(NestedField field) { + return new Builder(field); + } + + public static Builder required(String name) { + return new Builder(false, name); + } + + public static Builder optional(String name) { + return new Builder(true, name); + } + + public static class Builder { + private final boolean isOptional; + private final String name; + private Integer id = null; + private Type type = null; + private String doc = null; + private Object initialDefault = null; + private Object writeDefault = null; + + private Builder(boolean isFieldOptional, String fieldName) { + isOptional = isFieldOptional; + name = fieldName; + } + + private Builder(NestedField toCopy) { + this.isOptional = toCopy.isOptional; + this.name = toCopy.name; + this.id = toCopy.id; + this.type = toCopy.type; + this.doc = toCopy.doc; + this.initialDefault = toCopy.initialDefault; + this.writeDefault = toCopy.writeDefault; + } + + public Builder withId(int fieldId) { + id = fieldId; + return this; + } + + public Builder ofType(Type fieldType) { + type = fieldType; + return this; + } + + public Builder withDoc(String fieldDoc) { + doc = fieldDoc; + return this; + } + + public Builder withInitialDefault(Object fieldInitialDefault) { + initialDefault = fieldInitialDefault; + return this; + } + + public Builder withWriteDefault(Object fieldWriteDefault) { + writeDefault = fieldWriteDefault; + return this; + } + + public NestedField build() { + // the constructor validates the fields + return new NestedField(isOptional, id, name, type, doc, initialDefault, writeDefault); + } } private final boolean isOptional; @@ -497,8 +565,17 @@ public static NestedField of(int id, boolean isOptional, String name, Type type, private final String name; private final Type type; private final String doc; - - private NestedField(boolean isOptional, int id, String name, Type type, String doc) { + private final Object initialDefault; + private final Object writeDefault; + + private NestedField( + boolean isOptional, + int id, + String name, + Type type, + String doc, + Object initialDefault, + Object writeDefault) { Preconditions.checkNotNull(name, "Name cannot be null"); Preconditions.checkNotNull(type, "Type cannot be null"); this.isOptional = isOptional; @@ -506,6 +583,19 @@ private NestedField(boolean isOptional, int id, String name, Type type, String d this.name = name; this.type = type; this.doc = doc; + this.initialDefault = castDefault(initialDefault, type); + this.writeDefault = castDefault(writeDefault, type); + } + + private static Object castDefault(Object defaultValue, Type type) { + if (type.isNestedType() && defaultValue != null) { + throw new IllegalArgumentException( + String.format("Invalid default value for %s: %s (must be null)", type, defaultValue)); + } else if (defaultValue != null) { + return Expressions.lit(defaultValue).to(type).value(); + } + + return null; } public boolean isOptional() { @@ -516,7 +606,7 @@ public NestedField asOptional() { if (isOptional) { return this; } - return new NestedField(true, id, name, type, doc); + return new NestedField(true, id, name, type, doc, initialDefault, writeDefault); } public boolean isRequired() { @@ -527,11 +617,15 @@ public NestedField asRequired() { if (!isOptional) { return this; } - return new NestedField(false, id, name, type, doc); + return new NestedField(false, id, name, type, doc, initialDefault, writeDefault); } + /** + * @deprecated will be removed in 2.0.0; use {@link Builder#withId(int)} instead + */ + @Deprecated public NestedField withFieldId(int newId) { - return new NestedField(isOptional, newId, name, type, doc); + return new NestedField(isOptional, newId, name, type, doc, initialDefault, writeDefault); } public int fieldId() { @@ -550,6 +644,14 @@ public String doc() { return doc; } + public Object initialDefault() { + return initialDefault; + } + + public Object writeDefault() { + return writeDefault; + } + @Override public String toString() { return String.format( diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java index 93bfa2398466..f630129dc50f 100644 --- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java @@ -47,16 +47,16 @@ public class GenericAvroReader private Schema fileSchema = null; private ValueReader reader = null; - public static GenericAvroReader create(org.apache.iceberg.Schema schema) { - return new GenericAvroReader<>(schema); + public static GenericAvroReader create(org.apache.iceberg.Schema expectedSchema) { + return new GenericAvroReader<>(expectedSchema); } - public static GenericAvroReader create(Schema schema) { - return new GenericAvroReader<>(schema); + public static GenericAvroReader create(Schema readSchema) { + return new GenericAvroReader<>(readSchema); } - GenericAvroReader(org.apache.iceberg.Schema readSchema) { - this.expectedType = readSchema.asStruct(); + GenericAvroReader(org.apache.iceberg.Schema expectedSchema) { + this.expectedType = expectedSchema.asStruct(); } GenericAvroReader(Schema readSchema) { @@ -140,6 +140,8 @@ public ValueReader record(Type partner, Schema record, List> f Types.NestedField field = expected.field(fieldId); 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()))); } 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/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java b/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java new file mode 100644 index 000000000000..1cfe88206b1b --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/avro/TestReadDefaultValues.java @@ -0,0 +1,166 @@ +/* + * 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.avro; + +import static org.apache.iceberg.types.Types.NestedField.required; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Path; +import java.util.List; +import org.apache.avro.generic.GenericData.Record; +import org.apache.iceberg.Files; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SingleValueParser; +import org.apache.iceberg.io.FileAppender; +import org.apache.iceberg.relocated.com.google.common.collect.Lists; +import org.apache.iceberg.types.Type; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; + +public class TestReadDefaultValues { + + @TempDir public Path temp; + + private static final Object[][] TYPES_WITH_DEFAULTS = + new Object[][] { + {Types.BooleanType.get(), "true"}, + {Types.IntegerType.get(), "1"}, + {Types.LongType.get(), "9999999"}, + {Types.FloatType.get(), "1.23"}, + {Types.DoubleType.get(), "123.456"}, + {Types.DateType.get(), "\"2007-12-03\""}, + {Types.TimeType.get(), "\"10:15:30\""}, + {Types.TimestampType.withoutZone(), "\"2007-12-03T10:15:30\""}, + {Types.TimestampType.withZone(), "\"2007-12-03T10:15:30+00:00\""}, + {Types.StringType.get(), "\"foo\""}, + {Types.UUIDType.get(), "\"eb26bdb1-a1d8-4aa6-990e-da940875492c\""}, + {Types.FixedType.ofLength(2), "\"111f\""}, + {Types.BinaryType.get(), "\"0000ff\""}, + {Types.DecimalType.of(9, 4), "\"123.4500\""}, + {Types.DecimalType.of(9, 0), "\"2\""}, + // Avro doesn't support negative scale + // {Types.DecimalType.of(9, -20), "\"2E+20\""}, + // Nested type defaults are not currently allowed + }; + + @Test + public void testDefaultAppliedWhenMissingColumn() throws IOException { + for (Object[] typeAndDefault : TYPES_WITH_DEFAULTS) { + Type type = (Type) typeAndDefault[0]; + String defaultValueJson = (String) typeAndDefault[1]; + Object defaultValue = SingleValueParser.fromJson(type, defaultValueJson); + + // note that this schema does not have column "defaulted" + Schema writerSchema = new Schema(required(999, "written", Types.IntegerType.get())); + + File testFile = temp.resolve("test.avro").toFile(); + testFile.delete(); + + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(writerSchema) + .createWriterFunc(GenericAvroWriter::create) + .named("test") + .build()) { + Record record = new Record(AvroSchemaUtil.convert(writerSchema.asStruct())); + record.put(0, 1); + writer.add(record); + } + + Schema readerSchema = + new Schema( + Types.NestedField.required("written") + .withId(999) + .ofType(Types.IntegerType.get()) + .build(), + Types.NestedField.optional("defaulted") + .withId(1000) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + Record expectedRecord = new Record(AvroSchemaUtil.convert(readerSchema.asStruct())); + expectedRecord.put(0, 1); + expectedRecord.put(1, defaultValue); + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .project(readerSchema) + .createResolvingReader(schema -> GenericAvroReader.create(schema)) + .build()) { + rows = Lists.newArrayList(reader); + } + + AvroTestHelpers.assertEquals(readerSchema.asStruct(), expectedRecord, rows.get(0)); + } + } + + @Test + public void testDefaultDoesNotOverrideExplicitValue() throws IOException { + for (Object[] typeAndDefault : TYPES_WITH_DEFAULTS) { + Type type = (Type) typeAndDefault[0]; + String defaultValueJson = (String) typeAndDefault[1]; + Object defaultValue = SingleValueParser.fromJson(type, defaultValueJson); + + Schema readerSchema = + new Schema( + Types.NestedField.required("written_1") + .withId(999) + .ofType(Types.IntegerType.get()) + .build(), + Types.NestedField.optional("written_2") + .withId(1000) + .ofType(type) + .withInitialDefault(defaultValue) + .build()); + + // Create a record with null value for the column with default value + Record expectedRecord = new Record(AvroSchemaUtil.convert(readerSchema.asStruct())); + expectedRecord.put(0, 1); + expectedRecord.put(1, null); + + File testFile = temp.resolve("test.avro").toFile(); + testFile.delete(); + + try (FileAppender writer = + Avro.write(Files.localOutput(testFile)) + .schema(readerSchema) + .createWriterFunc(GenericAvroWriter::create) + .named("test") + .build()) { + writer.add(expectedRecord); + } + + List rows; + try (AvroIterable reader = + Avro.read(Files.localInput(testFile)) + .project(readerSchema) + .createReaderFunc(GenericAvroReader::create) + .build()) { + rows = Lists.newArrayList(reader); + } + + // Existence of default value should not affect the read result + AvroTestHelpers.assertEquals(readerSchema.asStruct(), expectedRecord, rows.get(0)); + } + } +} From 0581ceb0b00cc5298adcca468873e240e994677a Mon Sep 17 00:00:00 2001 From: hsiang-c <137842490+hsiang-c@users.noreply.github.com> Date: Mon, 7 Oct 2024 14:48:53 +0800 Subject: [PATCH 0750/1019] AWS: Make sure overridden configurations are applied (#11274) --- .../apache/iceberg/aws/s3/S3FileIOProperties.java | 12 +++++++++--- .../apache/iceberg/aws/TestS3FileIOProperties.java | 7 ++++++- 2 files changed, 15 insertions(+), 4 deletions(-) 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 6813913a4db0..4ab1514a22d5 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 @@ -899,9 +899,14 @@ public void applyEndpointConfigurations(T builder) { * */ public void applyRetryConfigurations(T builder) { + ClientOverrideConfiguration.Builder configBuilder = + null != builder.overrideConfiguration() + ? builder.overrideConfiguration().toBuilder() + : ClientOverrideConfiguration.builder(); + builder.overrideConfiguration( - config -> - config.retryPolicy( + configBuilder + .retryPolicy( // Use a retry strategy which will persistently retry throttled exceptions with // exponential backoff, to give S3 a chance to autoscale. // LEGACY mode works best here, as it will allow throttled exceptions to use all of @@ -945,7 +950,8 @@ public void applyRetryConfigurations(T builder) { return 5; }) .build()) - .build())); + .build()) + .build()); } /** 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 92ed3648bcfc..e2499e947617 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/TestS3FileIOProperties.java @@ -36,6 +36,7 @@ import software.amazon.awssdk.auth.credentials.AwsSessionCredentials; import software.amazon.awssdk.auth.credentials.DefaultCredentialsProvider; import software.amazon.awssdk.core.client.config.SdkAdvancedClientOption; +import software.amazon.awssdk.core.retry.RetryPolicy; import software.amazon.awssdk.core.signer.Signer; import software.amazon.awssdk.services.s3.S3Client; import software.amazon.awssdk.services.s3.S3ClientBuilder; @@ -239,7 +240,7 @@ public void testS3RemoteSigningEnabled() { } @Test - public void s3RemoteSigningEnabledWithUserAgent() { + public void s3RemoteSigningEnabledWithUserAgentAndRetryPolicy() { String uri = "http://localhost:12345"; Map properties = ImmutableMap.of( @@ -249,6 +250,7 @@ public void s3RemoteSigningEnabledWithUserAgent() { s3Properties.applySignerConfiguration(builder); s3Properties.applyUserAgentConfigurations(builder); + s3Properties.applyRetryConfigurations(builder); Optional userAgent = builder.overrideConfiguration().advancedOption(SdkAdvancedClientOption.USER_AGENT_PREFIX); @@ -260,6 +262,9 @@ public void s3RemoteSigningEnabledWithUserAgent() { S3V4RestSignerClient signerClient = (S3V4RestSignerClient) signer.get(); assertThat(signerClient.baseSignerUri()).isEqualTo(uri); assertThat(signerClient.properties()).isEqualTo(properties); + + Optional retryPolicy = builder.overrideConfiguration().retryPolicy(); + assertThat(retryPolicy).isPresent().get().isInstanceOf(RetryPolicy.class); } @Test From ef6d5c05ad8cd8fa68c1362e31d5a23369e32473 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 7 Oct 2024 09:25:30 +0200 Subject: [PATCH 0751/1019] Build: Bump com.google.cloud:libraries-bom from 26.47.0 to 26.48.0 (#11271) Bumps [com.google.cloud:libraries-bom](https://github.com/googleapis/java-cloud-bom) from 26.47.0 to 26.48.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.47.0...v26.48.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 69acd34ac598..1ee13574af35 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.47.0" +google-libraries-bom = "26.48.0" guava = "33.3.1-jre" hadoop2 = "2.7.3" hadoop3 = "3.3.6" From 890a1ea0c4ed14b1d0d2c3e82f3c6ebd57e9b97b Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Tue, 8 Oct 2024 00:05:59 +0800 Subject: [PATCH 0752/1019] Core: Fix UnicodeUtil#truncateStringMax returns malformed string. (#11161) --- .../org/apache/iceberg/types/Comparators.java | 6 ++-- .../org/apache/iceberg/util/UnicodeUtil.java | 24 +++++++++++-- .../apache/iceberg/TestMetricsTruncation.java | 34 +++++++++++++++++-- 3 files changed, 56 insertions(+), 8 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/types/Comparators.java b/api/src/main/java/org/apache/iceberg/types/Comparators.java index bfbffc64b673..98416c6943db 100644 --- a/api/src/main/java/org/apache/iceberg/types/Comparators.java +++ b/api/src/main/java/org/apache/iceberg/types/Comparators.java @@ -323,9 +323,9 @@ private CharSeqComparator() {} * represented using two Java characters (using UTF-16 surrogate pairs). Character by character * comparison may yield incorrect results while comparing a 4 byte UTF-8 character to a java * char. Character by character comparison works as expected if both characters are <= 3 byte - * UTF-8 character or both characters are 4 byte UTF-8 characters. - * isCharInUTF16HighSurrogateRange method detects a 4-byte character and considers that - * character to be lexicographically greater than any 3 byte or lower UTF-8 character. + * UTF-8 character or both characters are 4 byte UTF-8 characters. isCharHighSurrogate method + * detects a high surrogate (4-byte character) and considers that character to be + * lexicographically greater than any 3 byte or lower UTF-8 character. */ @Override public int compare(CharSequence s1, CharSequence s2) { diff --git a/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java b/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java index 4dd2afa123ac..a1bb3f497196 100644 --- a/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java +++ b/api/src/main/java/org/apache/iceberg/util/UnicodeUtil.java @@ -82,9 +82,9 @@ public static Literal truncateStringMax(Literal inpu for (int i = length - 1; i >= 0; i--) { // Get the offset in the truncated string buffer where the number of unicode characters = i int offsetByCodePoint = truncatedStringBuilder.offsetByCodePoints(0, i); - int nextCodePoint = truncatedStringBuilder.codePointAt(offsetByCodePoint) + 1; + int nextCodePoint = incrementCodePoint(truncatedStringBuilder.codePointAt(offsetByCodePoint)); // No overflow - if (nextCodePoint != 0 && Character.isValidCodePoint(nextCodePoint)) { + if (nextCodePoint != 0) { truncatedStringBuilder.setLength(offsetByCodePoint); // Append next code point to the truncated substring truncatedStringBuilder.appendCodePoint(nextCodePoint); @@ -93,4 +93,24 @@ public static Literal truncateStringMax(Literal inpu } return null; // Cannot find a valid upper bound } + + private static int incrementCodePoint(int codePoint) { + // surrogate code points are not Unicode scalar values, + // any UTF-8 byte sequence that would otherwise map to code points U+D800..U+DFFF is ill-formed. + // see https://www.unicode.org/versions/Unicode16.0.0/core-spec/chapter-3/#G27288 + Preconditions.checkArgument( + codePoint < Character.MIN_SURROGATE || codePoint > Character.MAX_SURROGATE, + "invalid code point: %s", + codePoint); + + if (codePoint == Character.MIN_SURROGATE - 1) { + // increment to the next Unicode scalar value + return Character.MAX_SURROGATE + 1; + } else if (codePoint == Character.MAX_CODE_POINT) { + // overflow + return 0; + } else { + return codePoint + 1; + } + } } diff --git a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java index 5bd1fcd69120..6faf1eeb5bfd 100644 --- a/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java +++ b/core/src/test/java/org/apache/iceberg/TestMetricsTruncation.java @@ -202,11 +202,20 @@ public void testTruncateStringMax() { String test5 = "\uDBFF\uDFFF\uDBFF\uDFFF"; String test6 = "\uD800\uDFFF\uD800\uDFFF"; // Increment the previous character - String test6_2_expected = "\uD801\uDC00"; + String test6_1_expected = "\uD801\uDC00"; String test7 = "\uD83D\uDE02\uD83D\uDE02\uD83D\uDE02"; String test7_2_expected = "\uD83D\uDE02\uD83D\uDE03"; String test7_1_expected = "\uD83D\uDE03"; + // Increment the max UTF-8 character will overflow + String test8 = "a\uDBFF\uDFFFc"; + String test8_2_expected = "b"; + + // Increment skip invalid Unicode scalar values [Character.MIN_SURROGATE, + // Character.MAX_SURROGATE] + String test9 = "a" + (char) (Character.MIN_SURROGATE - 1) + "b"; + String test9_2_expected = "a" + (char) (Character.MAX_SURROGATE + 1); + Comparator cmp = Literal.of(test1).comparator(); assertThat(cmp.compare(truncateStringMax(Literal.of(test1), 4).value(), test1)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") @@ -254,10 +263,10 @@ public void testTruncateStringMax() { assertThat(truncateStringMax(Literal.of(test5), 1)) .as("An upper bound doesn't exist since the first two characters are max UTF-8 characters") .isNull(); - assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 2).value(), test6)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6)) .as("Truncated upper bound should be greater than or equal to the actual upper bound") .isGreaterThanOrEqualTo(0); - assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_2_expected)) + assertThat(cmp.compare(truncateStringMax(Literal.of(test6), 1).value(), test6_1_expected)) .as( "Test 4 byte UTF-8 character increment. Output must have one character with " + "the first character incremented") @@ -273,5 +282,24 @@ public void testTruncateStringMax() { .as( "Test input with multiple 4 byte UTF-8 character where the first unicode character should be incremented") .isEqualTo(0); + + assertThat(cmp.compare(truncateStringMax(Literal.of(test8), 2).value(), test8)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + assertThat(cmp.compare(truncateStringMax(Literal.of(test8), 2).value(), test8_2_expected)) + .as( + "Test the last character is the 4-byte max UTF-8 character after truncated where the second-to-last " + + "character should be incremented") + .isEqualTo(0); + + assertThat(cmp.compare(truncateStringMax(Literal.of(test9), 2).value(), test9)) + .as("Truncated upper bound should be greater than or equal to the actual upper bound") + .isGreaterThanOrEqualTo(0); + + assertThat(cmp.compare(truncateStringMax(Literal.of(test9), 2).value(), test9_2_expected)) + .as( + "Test the last character is `Character.MIN_SURROGATE - 1` after truncated, it should be incremented to " + + "next valid Unicode scalar value `Character.MAX_SURROGATE + 1`") + .isEqualTo(0); } } From 950099e222eb499ffb137dde7f846dc647e90185 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 7 Oct 2024 16:12:39 -0700 Subject: [PATCH 0753/1019] Core: Add internal Avro reader (#11108) --- .../org/apache/iceberg/ManifestReader.java | 26 +- .../iceberg/avro/GenericAvroReader.java | 64 +---- .../apache/iceberg/avro/InternalReader.java | 252 ++++++++++++++++++ .../apache/iceberg/avro/InternalReaders.java | 110 ++++++++ .../org/apache/iceberg/avro/ValueReaders.java | 98 +++++++ .../apache/iceberg/TestManifestReader.java | 3 +- 6 files changed, 483 insertions(+), 70 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/avro/InternalReader.java create mode 100644 core/src/main/java/org/apache/iceberg/avro/InternalReaders.java diff --git a/core/src/main/java/org/apache/iceberg/ManifestReader.java b/core/src/main/java/org/apache/iceberg/ManifestReader.java index 6364603c591f..cf04eb7c472a 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestReader.java +++ b/core/src/main/java/org/apache/iceberg/ManifestReader.java @@ -25,8 +25,10 @@ import java.util.List; import java.util.Map; import java.util.Set; +import org.apache.avro.io.DatumReader; import org.apache.iceberg.avro.Avro; import org.apache.iceberg.avro.AvroIterable; +import org.apache.iceberg.avro.InternalReader; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -65,16 +67,16 @@ public class ManifestReader> extends CloseableGroup "record_count"); protected enum FileType { - DATA_FILES(GenericDataFile.class.getName()), - DELETE_FILES(GenericDeleteFile.class.getName()); + DATA_FILES(GenericDataFile.class), + DELETE_FILES(GenericDeleteFile.class); - private final String fileClass; + private final Class fileClass; - FileType(String fileClass) { + FileType(Class fileClass) { this.fileClass = fileClass; } - private String fileClass() { + private Class fileClass() { return fileClass; } } @@ -261,12 +263,7 @@ private CloseableIterable> open(Schema projection) { AvroIterable> reader = Avro.read(file) .project(ManifestEntry.wrapFileSchema(Types.StructType.of(fields))) - .rename("manifest_entry", GenericManifestEntry.class.getName()) - .rename("partition", PartitionData.class.getName()) - .rename("r102", PartitionData.class.getName()) - .rename("data_file", content.fileClass()) - .rename("r2", content.fileClass()) - .classLoader(GenericManifestEntry.class.getClassLoader()) + .createResolvingReader(this::newReader) .reuseContainers() .build(); @@ -279,6 +276,13 @@ private CloseableIterable> open(Schema projection) { } } + private DatumReader newReader(Schema schema) { + return InternalReader.create(schema) + .setRootType(GenericManifestEntry.class) + .setCustomType(ManifestEntry.DATA_FILE_ID, content.fileClass()) + .setCustomType(DataFile.PARTITION_ID, PartitionData.class); + } + CloseableIterable> liveEntries() { return entries(true /* only live entries */); } diff --git a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java index f630129dc50f..bfdb65acf1c2 100644 --- a/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java +++ b/core/src/main/java/org/apache/iceberg/avro/GenericAvroReader.java @@ -28,11 +28,8 @@ import org.apache.avro.generic.IndexedRecord; import org.apache.avro.io.DatumReader; import org.apache.avro.io.Decoder; -import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.common.DynClasses; 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.types.Type; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.Pair; @@ -43,7 +40,7 @@ public class GenericAvroReader private final Types.StructType expectedType; private ClassLoader loader = Thread.currentThread().getContextClassLoader(); private Map renames = ImmutableMap.of(); - private final Map idToConstant = ImmutableMap.of(); + private final Map idToConstant = ImmutableMap.of(); private Schema fileSchema = null; private ValueReader reader = null; @@ -111,48 +108,13 @@ private ResolvingReadBuilder(Types.StructType expectedType, String rootName) { @Override public ValueReader record(Type partner, Schema record, List> fieldResults) { - Types.StructType expected = partner != null ? partner.asStructType() : null; - Map idToPos = idToPos(expected); - - List>> readPlan = Lists.newArrayList(); - List fileFields = record.getFields(); - for (int pos = 0; pos < fileFields.size(); pos += 1) { - Schema.Field field = fileFields.get(pos); - ValueReader fieldReader = fieldResults.get(pos); - Integer fieldId = AvroSchemaUtil.fieldId(field); - Integer projectionPos = idToPos.remove(fieldId); - - Object constant = idToConstant.get(fieldId); - if (projectionPos != null && constant != null) { - readPlan.add( - Pair.of(projectionPos, ValueReaders.replaceWithConstant(fieldReader, constant))); - } else { - readPlan.add(Pair.of(projectionPos, fieldReader)); - } + if (partner == null) { + return ValueReaders.skipStruct(fieldResults); } - // handle any expected columns that are not in the data file - for (Map.Entry idAndPos : idToPos.entrySet()) { - int fieldId = idAndPos.getKey(); - int pos = idAndPos.getValue(); - - Object constant = idToConstant.get(fieldId); - Types.NestedField field = expected.field(fieldId); - 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()))); - } else if (fieldId == MetadataColumns.IS_DELETED.fieldId()) { - readPlan.add(Pair.of(pos, ValueReaders.constant(false))); - } else if (fieldId == MetadataColumns.ROW_POSITION.fieldId()) { - readPlan.add(Pair.of(pos, ValueReaders.positions())); - } else if (field.isOptional()) { - readPlan.add(Pair.of(pos, ValueReaders.constant(null))); - } else { - throw new IllegalArgumentException( - String.format("Missing required field: %s", field.name())); - } - } + Types.StructType expected = partner.asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant); return recordReader(readPlan, avroSchemas.get(partner), record.getFullName()); } @@ -266,19 +228,5 @@ public ValueReader primitive(Type partner, Schema primitive) { throw new IllegalArgumentException("Unsupported type: " + primitive); } } - - private Map idToPos(Types.StructType struct) { - Map idToPos = Maps.newHashMap(); - - if (struct != null) { - List fields = struct.fields(); - for (int pos = 0; pos < fields.size(); pos += 1) { - Types.NestedField field = fields.get(pos); - idToPos.put(field.fieldId(), pos); - } - } - - return idToPos; - } } } diff --git a/core/src/main/java/org/apache/iceberg/avro/InternalReader.java b/core/src/main/java/org/apache/iceberg/avro/InternalReader.java new file mode 100644 index 000000000000..ca83ce2ba7cd --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/InternalReader.java @@ -0,0 +1,252 @@ +/* + * 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.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.StructLike; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; +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; + +/** + * A reader that produces Iceberg's internal in-memory object model. + * + *

    Iceberg's internal in-memory object model produces the types defined in {@link + * Type.TypeID#javaClass()}. + * + * @param Java type returned by the reader + */ +public class InternalReader implements DatumReader, SupportsRowPosition { + private static final int ROOT_ID = -1; + + private final Types.StructType expectedType; + private final Map> typeMap = Maps.newHashMap(); + private final Map idToConstant = ImmutableMap.of(); + private Schema fileSchema = null; + private ValueReader reader = null; + + public static InternalReader create(org.apache.iceberg.Schema schema) { + return new InternalReader<>(schema); + } + + InternalReader(org.apache.iceberg.Schema readSchema) { + this.expectedType = readSchema.asStruct(); + } + + @SuppressWarnings("unchecked") + private void initReader() { + this.reader = + (ValueReader) + AvroWithPartnerVisitor.visit( + Pair.of(ROOT_ID, expectedType), + fileSchema, + new ResolvingReadBuilder(), + AccessByID.instance()); + } + + @Override + public void setSchema(Schema schema) { + this.fileSchema = schema; + initReader(); + } + + public InternalReader setRootType(Class rootClass) { + typeMap.put(ROOT_ID, rootClass); + return this; + } + + public InternalReader setCustomType(int fieldId, Class structClass) { + typeMap.put(fieldId, structClass); + return this; + } + + @Override + public void setRowPositionSupplier(Supplier posSupplier) { + if (reader instanceof SupportsRowPosition) { + ((SupportsRowPosition) reader).setRowPositionSupplier(posSupplier); + } + } + + @Override + public T read(T reuse, Decoder decoder) throws IOException { + return reader.read(decoder, reuse); + } + + private class ResolvingReadBuilder + extends AvroWithPartnerVisitor, ValueReader> { + @Override + public ValueReader record( + Pair partner, Schema record, List> fieldResults) { + if (partner == null) { + return ValueReaders.skipStruct(fieldResults); + } + + Types.StructType expected = partner.second().asStructType(); + List>> readPlan = + ValueReaders.buildReadPlan(expected, record, fieldResults, idToConstant); + + return structReader(readPlan, partner.first(), expected); + } + + private ValueReader structReader( + List>> readPlan, int fieldId, Types.StructType struct) { + + Class structClass = typeMap.get(fieldId); + if (structClass != null) { + return InternalReaders.struct(struct, structClass, readPlan); + } else { + return InternalReaders.struct(struct, readPlan); + } + } + + @Override + public ValueReader union( + Pair partner, Schema union, List> options) { + return ValueReaders.union(options); + } + + @Override + public ValueReader arrayMap( + Pair partner, + Schema map, + ValueReader keyReader, + ValueReader valueReader) { + return ValueReaders.arrayMap(keyReader, valueReader); + } + + @Override + public ValueReader array( + Pair partner, Schema array, ValueReader elementReader) { + return ValueReaders.array(elementReader); + } + + @Override + public ValueReader map(Pair partner, Schema map, ValueReader valueReader) { + return ValueReaders.map(ValueReaders.strings(), valueReader); + } + + @Override + public ValueReader primitive(Pair partner, Schema primitive) { + LogicalType logicalType = primitive.getLogicalType(); + if (logicalType != null) { + switch (logicalType.getName()) { + case "date": + return ValueReaders.ints(); + + case "time-micros": + return ValueReaders.longs(); + + case "timestamp-millis": + // adjust to microseconds + ValueReader longs = ValueReaders.longs(); + return (ValueReader) (decoder, ignored) -> longs.read(decoder, null) * 1000L; + + case "timestamp-micros": + return ValueReaders.longs(); + + 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.second().typeId() == Type.TypeID.LONG) { + return ValueReaders.intsAsLongs(); + } + return ValueReaders.ints(); + case LONG: + return ValueReaders.longs(); + case FLOAT: + if (partner != null && partner.second().typeId() == Type.TypeID.DOUBLE) { + return ValueReaders.floatsAsDoubles(); + } + return ValueReaders.floats(); + case DOUBLE: + return ValueReaders.doubles(); + case STRING: + return ValueReaders.strings(); + case FIXED: + return ValueReaders.fixed(primitive); + case BYTES: + return ValueReaders.byteBuffers(); + case ENUM: + return ValueReaders.enums(primitive.getEnumSymbols()); + default: + throw new IllegalArgumentException("Unsupported type: " + primitive); + } + } + } + + private static class AccessByID + implements AvroWithPartnerVisitor.PartnerAccessors> { + private static final AccessByID INSTANCE = new AccessByID(); + + public static AccessByID instance() { + return INSTANCE; + } + + @Override + public Pair fieldPartner( + Pair partner, Integer fieldId, String name) { + Types.NestedField field = partner.second().asStructType().field(fieldId); + return field != null ? Pair.of(field.fieldId(), field.type()) : null; + } + + @Override + public Pair mapKeyPartner(Pair partner) { + Types.MapType map = partner.second().asMapType(); + return Pair.of(map.keyId(), map.keyType()); + } + + @Override + public Pair mapValuePartner(Pair partner) { + Types.MapType map = partner.second().asMapType(); + return Pair.of(map.valueId(), map.valueType()); + } + + @Override + public Pair listElementPartner(Pair partner) { + Types.ListType list = partner.second().asListType(); + return Pair.of(list.elementId(), list.elementType()); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/avro/InternalReaders.java b/core/src/main/java/org/apache/iceberg/avro/InternalReaders.java new file mode 100644 index 000000000000..6136bae052ae --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/avro/InternalReaders.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.avro; + +import java.util.List; +import org.apache.iceberg.StructLike; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.Pair; + +class InternalReaders { + private InternalReaders() {} + + static ValueReader struct( + Types.StructType struct, List>> readPlan) { + return new RecordReader(readPlan, struct); + } + + static ValueReader struct( + Types.StructType struct, Class structClass, List>> readPlan) { + return new PlannedStructLikeReader<>(readPlan, struct, structClass); + } + + private static class PlannedStructLikeReader + extends ValueReaders.PlannedStructReader { + private final Types.StructType structType; + private final Class structClass; + private final DynConstructors.Ctor ctor; + + private PlannedStructLikeReader( + List>> readPlan, + Types.StructType structType, + Class structClass) { + super(readPlan); + this.structType = structType; + this.structClass = structClass; + this.ctor = + DynConstructors.builder(StructLike.class) + .hiddenImpl(structClass, Types.StructType.class) + .hiddenImpl(structClass) + .build(); + } + + @Override + protected S reuseOrCreate(Object reuse) { + if (structClass.isInstance(reuse)) { + return structClass.cast(reuse); + } else { + return ctor.newInstance(structType); + } + } + + @Override + protected Object get(S struct, int pos) { + return struct.get(pos, Object.class); + } + + @Override + protected void set(S struct, int pos, Object value) { + struct.set(pos, value); + } + } + + private static class RecordReader extends ValueReaders.PlannedStructReader { + private final Types.StructType structType; + + private RecordReader( + List>> readPlan, Types.StructType structType) { + super(readPlan); + this.structType = structType; + } + + @Override + protected GenericRecord reuseOrCreate(Object reuse) { + if (reuse instanceof GenericRecord) { + return (GenericRecord) reuse; + } else { + return GenericRecord.create(structType); + } + } + + @Override + protected Object get(GenericRecord struct, int pos) { + return struct.get(pos); + } + + @Override + protected void set(GenericRecord struct, int pos, Object value) { + struct.set(pos, value); + } + } +} 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 d530bc1854e1..246671076c31 100644 --- a/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java +++ b/core/src/main/java/org/apache/iceberg/avro/ValueReaders.java @@ -181,6 +181,83 @@ public static ValueReader record( return new PlannedIndexedReader<>(recordSchema, recordClass, readPlan); } + public static ValueReader skipStruct(List> readers) { + return new SkipStructReader(readers); + } + + /** + * 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 + * @return a read plan that is a list of (position, reader) pairs + */ + static List>> buildReadPlan( + Types.StructType expected, + Schema record, + List> fieldReaders, + Map idToConstant) { + Map idToPos = idToPos(expected); + + List>> readPlan = Lists.newArrayList(); + List fileFields = record.getFields(); + for (int pos = 0; pos < fileFields.size(); pos += 1) { + Schema.Field field = fileFields.get(pos); + ValueReader fieldReader = fieldReaders.get(pos); + Integer fieldId = AvroSchemaUtil.fieldId(field); + Integer projectionPos = idToPos.remove(fieldId); + + Object constant = idToConstant.get(fieldId); + if (projectionPos != null && constant != null) { + readPlan.add( + Pair.of(projectionPos, ValueReaders.replaceWithConstant(fieldReader, constant))); + } else { + readPlan.add(Pair.of(projectionPos, fieldReader)); + } + } + + // handle any expected columns that are not in the data file + for (Map.Entry idAndPos : idToPos.entrySet()) { + int fieldId = idAndPos.getKey(); + int pos = idAndPos.getValue(); + + Object constant = idToConstant.get(fieldId); + Types.NestedField field = expected.field(fieldId); + 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()))); + } else if (fieldId == MetadataColumns.IS_DELETED.fieldId()) { + readPlan.add(Pair.of(pos, ValueReaders.constant(false))); + } else if (fieldId == MetadataColumns.ROW_POSITION.fieldId()) { + readPlan.add(Pair.of(pos, ValueReaders.positions())); + } else if (field.isOptional()) { + readPlan.add(Pair.of(pos, ValueReaders.constant(null))); + } else { + throw new IllegalArgumentException( + String.format("Missing required field: %s", field.name())); + } + } + + return readPlan; + } + + private static Map idToPos(Types.StructType struct) { + Map idToPos = Maps.newHashMap(); + + if (struct != null) { + List fields = struct.fields(); + for (int pos = 0; pos < fields.size(); pos += 1) { + Types.NestedField field = fields.get(pos); + idToPos.put(field.fieldId(), pos); + } + } + + return idToPos; + } + private static class NullReader implements ValueReader { private static final NullReader INSTANCE = new NullReader(); @@ -777,6 +854,27 @@ public void skip(Decoder decoder) throws IOException { } } + private static class SkipStructReader implements ValueReader { + private final ValueReader[] readers; + + private SkipStructReader(List> readers) { + this.readers = readers.toArray(ValueReader[]::new); + } + + @Override + public Void read(Decoder decoder, Object reuse) throws IOException { + skip(decoder); + return null; + } + + @Override + public void skip(Decoder decoder) throws IOException { + for (ValueReader reader : readers) { + reader.skip(decoder); + } + } + } + public abstract static class PlannedStructReader implements ValueReader, SupportsRowPosition { private final ValueReader[] readers; diff --git a/core/src/test/java/org/apache/iceberg/TestManifestReader.java b/core/src/test/java/org/apache/iceberg/TestManifestReader.java index c9d0e292706e..e45415f1f2d2 100644 --- a/core/src/test/java/org/apache/iceberg/TestManifestReader.java +++ b/core/src/test/java/org/apache/iceberg/TestManifestReader.java @@ -44,7 +44,8 @@ public class TestManifestReader extends TestBase { "fileOrdinal", "fileSequenceNumber", "fromProjectionPos", - "manifestLocation") + "manifestLocation", + "partitionData.partitionType.fieldsById") .build(); @TestTemplate From 433ba3916102407e5fcd47d38e183da2c40588f5 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Tue, 8 Oct 2024 00:21:50 -0700 Subject: [PATCH 0754/1019] Arrow: Remove unused readers (#11276) --- ...dDictionaryEncodedParquetValuesReader.java | 38 ------ ...ectorizedParquetDefinitionLevelReader.java | 108 ------------------ 2 files changed, 146 deletions(-) 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 55f1d3fd7908..21670f6b2b33 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 @@ -22,7 +22,6 @@ import java.nio.ByteOrder; import org.apache.arrow.vector.BaseVariableWidthVector; import org.apache.arrow.vector.BitVectorHelper; -import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; @@ -151,15 +150,6 @@ protected void nextVal( } } - class FixedLengthDecimalDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - byte[] bytes = dict.decodeToBinary(currentVal).getBytesUnsafe(); - DecimalVectorUtil.setBigEndian((DecimalVector) vector, idx, bytes); - } - } - class VarWidthBinaryDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( @@ -174,22 +164,6 @@ protected void nextVal( } } - class IntBackedDecimalDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - ((DecimalVector) vector).set(idx, dict.decodeToInt(currentVal)); - } - } - - class LongBackedDecimalDictEncodedReader extends BaseDictEncodedReader { - @Override - protected void nextVal( - FieldVector vector, Dictionary dict, int idx, int currentVal, int typeWidth) { - ((DecimalVector) vector).set(idx, dict.decodeToLong(currentVal)); - } - } - class FixedSizeBinaryDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( @@ -233,22 +207,10 @@ public FixedWidthBinaryDictEncodedReader fixedWidthBinaryDictEncodedReader() { return new FixedWidthBinaryDictEncodedReader(); } - public FixedLengthDecimalDictEncodedReader fixedLengthDecimalDictEncodedReader() { - return new FixedLengthDecimalDictEncodedReader(); - } - public VarWidthBinaryDictEncodedReader varWidthBinaryDictEncodedReader() { return new VarWidthBinaryDictEncodedReader(); } - public IntBackedDecimalDictEncodedReader intBackedDecimalDictEncodedReader() { - return new IntBackedDecimalDictEncodedReader(); - } - - public LongBackedDecimalDictEncodedReader longBackedDecimalDictEncodedReader() { - return new LongBackedDecimalDictEncodedReader(); - } - public FixedSizeBinaryDictEncodedReader fixedSizeBinaryDictEncodedReader() { return new FixedSizeBinaryDictEncodedReader(); } 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 20d7f804978c..2d2412c5504a 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 @@ -24,7 +24,6 @@ import org.apache.arrow.vector.BaseVariableWidthVector; import org.apache.arrow.vector.BitVector; import org.apache.arrow.vector.BitVectorHelper; -import org.apache.arrow.vector.DecimalVector; import org.apache.arrow.vector.FieldVector; import org.apache.arrow.vector.FixedSizeBinaryVector; import org.apache.arrow.vector.IntVector; @@ -532,39 +531,6 @@ protected void nextDictEncodedVal( } } - class FixedLengthDecimalReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - valuesReader.getBuffer(typeWidth).get(byteArray, 0, typeWidth); - DecimalVectorUtil.setBigEndian((DecimalVector) vector, idx, byteArray); - } - - @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 - .fixedLengthDecimalDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - byte[] bytes = dict.decodeToBinary(reader.readInteger()).getBytesUnsafe(); - DecimalVectorUtil.setBigEndian((DecimalVector) vector, idx, bytes); - } - } - } - class FixedSizeBinaryReader extends BaseReader { @Override protected void nextVal( @@ -645,68 +611,6 @@ protected void nextDictEncodedVal( } } - class IntBackedDecimalReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - ((DecimalVector) vector).set(idx, valuesReader.getBuffer(Integer.BYTES).getInt()); - } - - @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 - .intBackedDecimalDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - ((DecimalVector) vector).set(idx, dict.decodeToInt(reader.readInteger())); - } - } - } - - class LongBackedDecimalReader extends BaseReader { - @Override - protected void nextVal( - FieldVector vector, - int idx, - ValuesAsBytesReader valuesReader, - int typeWidth, - byte[] byteArray) { - ((DecimalVector) vector).set(idx, valuesReader.getBuffer(Long.BYTES).getLong()); - } - - @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 - .longBackedDecimalDictEncodedReader() - .nextBatch(vector, idx, numValuesToRead, dict, nullabilityHolder, typeWidth); - } else if (Mode.PACKED.equals(mode)) { - ((DecimalVector) vector).set(idx, dict.decodeToLong(reader.readInteger())); - } - } - } - class BooleanReader extends BaseReader { @Override protected void nextVal( @@ -832,10 +736,6 @@ FixedWidthBinaryReader fixedWidthBinaryReader() { return new FixedWidthBinaryReader(); } - FixedLengthDecimalReader fixedLengthDecimalReader() { - return new FixedLengthDecimalReader(); - } - FixedSizeBinaryReader fixedSizeBinaryReader() { return new FixedSizeBinaryReader(); } @@ -844,14 +744,6 @@ VarWidthReader varWidthReader() { return new VarWidthReader(); } - IntBackedDecimalReader intBackedDecimalReader() { - return new IntBackedDecimalReader(); - } - - LongBackedDecimalReader longBackedDecimalReader() { - return new LongBackedDecimalReader(); - } - BooleanReader booleanReader() { return new BooleanReader(); } From f3cda9de8071f96a824c642b20af4aa725daad75 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 9 Oct 2024 14:19:40 -0700 Subject: [PATCH 0755/1019] Spec: Add v3 types and type promotion (#10955) --- format/spec.md | 49 ++++++++++++++++++++++++++++++++++++++++++------- 1 file changed, 42 insertions(+), 7 deletions(-) diff --git a/format/spec.md b/format/spec.md index eb5900e27b4b..2974430a3772 100644 --- a/format/spec.md +++ b/format/spec.md @@ -48,7 +48,7 @@ In addition to row-level deletes, version 2 makes some requirements stricter for Version 3 of the Iceberg spec extends data types and existing metadata structures to add new capabilities: -* New data types: nanosecond timestamp(tz) +* New data types: nanosecond timestamp(tz), unknown * Default value support for columns * Multi-argument transforms for partitioning and sorting @@ -184,6 +184,7 @@ Supported primitive types are defined in the table below. Primitive types added | Added by version | Primitive type | Description | Requirements | |------------------|--------------------|--------------------------------------------------------------------------|--------------------------------------------------| +| [v3](#version-3) | **`unknown`** | Default / null column type used when a more specific type is not known | Must be optional with `null` defaults; not stored in data files | | | **`boolean`** | True or false | | | | **`int`** | 32-bit signed integers | Can promote to `long` | | | **`long`** | 64-bit signed integers | | @@ -221,6 +222,8 @@ The `initial-default` is set only when a field is added to an existing schema. T The `initial-default` and `write-default` produce SQL default value behavior, without rewriting data files. SQL default value behavior when a field is added handles all existing rows as though the rows were written with the new field's default value. Default value changes may only affect future records and all known fields are written into data files. Omitting a known field when writing a data file is never allowed. The write default for a field must be written if a field is not supplied to a write. If the write default for a required field is not set, the writer must fail. +All columns of `unknown` type must default to null. Non-null values for `initial-default` or `write-default` are invalid. + Default values are attributes of fields in schemas and serialized with fields in the JSON format. See [Appendix C](#appendix-c-json-serialization). @@ -230,11 +233,32 @@ Schemas may be evolved by type promotion or adding, deleting, renaming, or reord Evolution applies changes to the table's current schema to produce a new schema that is identified by a unique schema ID, is added to the table's list of schemas, and is set as the table's current schema. -Valid type promotions are: - -* `int` to `long` -* `float` to `double` -* `decimal(P, S)` to `decimal(P', S)` if `P' > P` -- widen the precision of decimal types. +Valid primitive type promotions are: + +| Primitive type | v1, v2 valid type promotions | v3+ valid type promotions | Requirements | +|------------------|------------------------------|------------------------------|--------------| +| `unknown` | | _any type_ | | +| `int` | `long` | `long` | | +| `date` | | `timestamp`, `timestamp_ns` | Promotion to `timestamptz` or `timestamptz_ns` is **not** allowed; values outside the promoted type's range must result in a runtime failure | +| `float` | `double` | `double` | | +| `decimal(P, S)` | `decimal(P', S)` if `P' > P` | `decimal(P', S)` if `P' > P` | Widen precision only | + +Iceberg's Avro manifest format does not store the type of lower and upper bounds, and type promotion does not rewrite existing bounds. For example, when a `float` is promoted to `double`, existing data file bounds are encoded as 4 little-endian bytes rather than 8 little-endian bytes for `double`. To correctly decode the value, the original type at the time the file was written must be inferred according to the following table: + +| Current type | Length of bounds | Inferred type at write time | +|------------------|------------------|-----------------------------| +| `long` | 4 bytes | `int` | +| `long` | 8 bytes | `long` | +| `double` | 4 bytes | `float` | +| `double` | 8 bytes | `double` | +| `timestamp` | 4 bytes | `date` | +| `timestamp` | 8 bytes | `timestamp` | +| `timestamp_ns` | 4 bytes | `date` | +| `timestamp_ns` | 8 bytes | `timestamp_ns` | +| `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. @@ -949,6 +973,7 @@ Maps with non-string keys must use an array representation with the `map` logica |Type|Avro type|Notes| |--- |--- |--- | +|**`unknown`**|`null` or omitted|| |**`boolean`**|`boolean`|| |**`int`**|`int`|| |**`long`**|`long`|| @@ -1002,6 +1027,7 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo | Type | Parquet physical type | Logical type | Notes | |--------------------|--------------------------------------------------------------------|---------------------------------------------|----------------------------------------------------------------| +| **`unknown`** | None | | Omit from data files | | **`boolean`** | `boolean` | | | | **`int`** | `int` | | | | **`long`** | `long` | | | @@ -1023,12 +1049,16 @@ Lists must use the [3-level representation](https://github.com/apache/parquet-fo | **`map`** | `3-level map` | `MAP` | See Parquet docs for 3-level representation. | +When reading an `unknown` column, any corresponding column must be ignored and replaced with `null` values. + + ### ORC **Data Type Mappings** | Type | ORC type | ORC type attributes | Notes | |--------------------|---------------------|------------------------------------------------------|-----------------------------------------------------------------------------------------| +| **`unknown`** | None | | Omit from data files | | **`boolean`** | `boolean` | | | | **`int`** | `int` | | ORC `tinyint` and `smallint` would also map to **`int`**. | | **`long`** | `long` | | | @@ -1089,6 +1119,7 @@ The types below are not currently valid for bucketing, and so are not hashed. Ho | Primitive type | Hash specification | Test value | |--------------------|-------------------------------------------|--------------------------------------------| +| **`unknown`** | always `null` | | | **`boolean`** | `false: hashInt(0)`, `true: hashInt(1)` | `true` → `1392991556` | | **`float`** | `hashLong(doubleToLongBits(double(v))` [5]| `1.0F` → `-142385009`, `0.0F` → `1669671676`, `-0.0F` → `1669671676` | | **`double`** | `hashLong(doubleToLongBits(v))` [5]| `1.0D` → `-142385009`, `0.0D` → `1669671676`, `-0.0D` → `1669671676` | @@ -1119,6 +1150,7 @@ Types are serialized according to this table: |Type|JSON representation|Example| |--- |--- |--- | +|**`unknown`**|`JSON string: "unknown"`|`"unknown"`| |**`boolean`**|`JSON string: "boolean"`|`"boolean"`| |**`int`**|`JSON string: "int"`|`"int"`| |**`long`**|`JSON string: "long"`|`"long"`| @@ -1267,6 +1299,7 @@ This serialization scheme is for storing single values as individual binary valu | Type | Binary serialization | |------------------------------|--------------------------------------------------------------------------------------------------------------| +| **`unknown`** | Not supported | | **`boolean`** | `0x00` for false, non-zero byte for true | | **`int`** | Stored as 4-byte little-endian | | **`long`** | Stored as 8-byte little-endian | @@ -1319,10 +1352,11 @@ This serialization scheme is for storing single values as individual binary valu ### Version 3 Default values are added to struct fields in v3. + * The `write-default` is a forward-compatible change because it is only used at write time. Old writers will fail because the field is missing. * Tables with `initial-default` will be read correctly by older readers if `initial-default` is always null for optional fields. Otherwise, old readers will default optional columns with null. Old readers will fail to read required fields which are populated by `initial-default` because that default is not supported. -Types `timestamp_ns` and `timestamptz_ns` are added in v3. +Types `unknown`, `timestamp_ns`, and `timestamptz_ns` are added in v3. All readers are required to read tables with unknown partition transforms, ignoring the unsupported partition fields when filtering. @@ -1423,3 +1457,4 @@ Iceberg supports two types of histories for tables. A history of previous "curre might indicate different snapshot IDs for a specific timestamp. The discrepancies can be caused by a variety of table operations (e.g. updating the `current-snapshot-id` can be used to set the snapshot of a table to any arbitrary snapshot, which might have a lineage derived from a table branch or no lineage at all). When processing point in time queries implementations should use "snapshot-log" metadata to lookup the table state at the given point in time. This ensures time-travel queries reflect the state of the table at the provided timestamp. For example a SQL query like `SELECT * FROM prod.db.table TIMESTAMP AS OF '1986-10-26 01:21:00Z';` would find the snapshot of the Iceberg table just prior to '1986-10-26 01:21:00 UTC' in the snapshot logs and use the metadata from that snapshot to perform the scan of the table. If no snapshot exists prior to the timestamp given or "snapshot-log" is not populated (it is an optional field), then systems should raise an informative error message about the missing metadata. + From 6b6dd7c43a8d87e8bf6dc8a2b5af86b6d4b50d50 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:06:54 +0200 Subject: [PATCH 0756/1019] Build: Bump mkdocs-material from 9.5.38 to 9.5.39 (#11272) Bumps [mkdocs-material](https://github.com/squidfunk/mkdocs-material) from 9.5.38 to 9.5.39. - [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.38...9.5.39) --- 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 d9e513707e33..a7db0af729d0 100644 --- a/site/requirements.txt +++ b/site/requirements.txt @@ -17,7 +17,7 @@ mkdocs-awesome-pages-plugin==2.9.3 mkdocs-macros-plugin==1.2.0 -mkdocs-material==9.5.38 +mkdocs-material==9.5.39 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 fd31be51a855c59ebdca0d5eac8ec49981c1eadb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:07:15 +0200 Subject: [PATCH 0757/1019] Build: Bump com.google.errorprone:error_prone_annotations (#11270) Bumps [com.google.errorprone:error_prone_annotations](https://github.com/google/error-prone) from 2.31.0 to 2.33.0. - [Release notes](https://github.com/google/error-prone/releases) - [Commits](https://github.com/google/error-prone/compare/v2.31.0...v2.33.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 1ee13574af35..fbf45713034b 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -38,7 +38,7 @@ datasketches = "6.0.0" delta-standalone = "3.2.1" delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" -errorprone-annotations = "2.31.0" +errorprone-annotations = "2.33.0" failsafe = "3.3.2" findbugs-jsr305 = "3.0.2" flink118 = { strictly = "1.18.1"} From 334127007c06cbb4514bbccb9bc4e7c95593627f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:07:58 +0200 Subject: [PATCH 0758/1019] Build: Bump junit-platform from 1.11.1 to 1.11.2 (#11266) Bumps `junit-platform` from 1.11.1 to 1.11.2. Updates `org.junit.platform:junit-platform-suite-api` from 1.11.1 to 1.11.2 - [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.1 to 1.11.2 - [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 fbf45713034b..9e555c8d8ee4 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.10.1" -junit-platform = "1.11.1" +junit-platform = "1.11.2" kafka = "3.8.0" kryo-shaded = "4.0.3" microprofile-openapi-api = "3.1.2" From 8dfb3c3837f27a47ea1cb475b0a8ba2b61429e4a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:08:06 +0200 Subject: [PATCH 0759/1019] Build: Bump org.testcontainers:testcontainers from 1.20.1 to 1.20.2 (#11265) Bumps [org.testcontainers:testcontainers](https://github.com/testcontainers/testcontainers-java) from 1.20.1 to 1.20.2. - [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.1...1.20.2) --- updated-dependencies: - dependency-name: org.testcontainers:testcontainers 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 9e555c8d8ee4..0a6a35861899 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -88,7 +88,7 @@ spark-hive35 = "3.5.2" spring-boot = "2.7.18" spring-web = "5.3.39" sqlite-jdbc = "3.46.1.3" -testcontainers = "1.20.1" +testcontainers = "1.20.2" tez010 = "0.10.4" tez08 = { strictly = "0.8.4"} # see rich version usage explanation above From 29dabe71c5fca738315e2d6cc3ccb9189e04a8db Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:09:18 +0200 Subject: [PATCH 0760/1019] Build: Bump io.netty:netty-buffer from 4.1.113.Final to 4.1.114.Final (#11269) Bumps [io.netty:netty-buffer](https://github.com/netty/netty) from 4.1.113.Final to 4.1.114.Final. - [Commits](https://github.com/netty/netty/compare/netty-4.1.113.Final...netty-4.1.114.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 0a6a35861899..1951c958a62d 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.113.Final" -netty-buffer-compat = "4.1.113.Final" +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" From f1c193a787cae51edc0d2631b1868f373e819764 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:09:25 +0200 Subject: [PATCH 0761/1019] Build: Bump software.amazon.awssdk:bom from 2.28.11 to 2.28.16 (#11268) Bumps software.amazon.awssdk:bom from 2.28.11 to 2.28.16. --- 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 1951c958a62d..280faa02090e 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.11" +awssdk-bom = "2.28.16" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.0.0" caffeine = "2.9.3" From 1d72e86ea96f7b8e7f0ac51de425b2f100a5d4c9 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:09:36 +0200 Subject: [PATCH 0762/1019] Build: Bump jackson-bom from 2.14.2 to 2.18.0 (#11226) Bumps `jackson-bom` from 2.14.2 to 2.18.0. Updates `com.fasterxml.jackson:jackson-bom` from 2.14.2 to 2.18.0 - [Commits](https://github.com/FasterXML/jackson-bom/compare/jackson-bom-2.14.2...jackson-bom-2.18.0) Updates `com.fasterxml.jackson.core:jackson-core` from 2.14.2 to 2.18.0 - [Commits](https://github.com/FasterXML/jackson-core/compare/jackson-core-2.14.2...jackson-core-2.18.0) Updates `com.fasterxml.jackson.core:jackson-databind` from 2.14.2 to 2.18.0 - [Commits](https://github.com/FasterXML/jackson/commits) Updates `com.fasterxml.jackson.core:jackson-annotations` from 2.14.2 to 2.18.0 - [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-minor - dependency-name: com.fasterxml.jackson.core:jackson-core dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: com.fasterxml.jackson.core:jackson-databind dependency-type: direct:production update-type: version-update:semver-minor - dependency-name: com.fasterxml.jackson.core:jackson-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 280faa02090e..f816bb7d0768 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -52,7 +52,7 @@ httpcomponents-httpclient5 = "5.4" hive2 = { strictly = "2.3.9"} # see rich version usage explanation above hive3 = "3.1.3" immutables-value = "2.10.1" -jackson-bom = "2.14.2" +jackson-bom = "2.18.0" jackson211 = { strictly = "2.11.4"} # see rich version usage explanation above jackson212 = { strictly = "2.12.3"} jackson213 = { strictly = "2.13.4"} From d122af766dc7dff414eb980259c9cf67b0150662 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:09:44 +0200 Subject: [PATCH 0763/1019] Build: Bump datamodel-code-generator from 0.25.9 to 0.26.1 (#11234) Bumps [datamodel-code-generator](https://github.com/koxudaxi/datamodel-code-generator) from 0.25.9 to 0.26.1. - [Release notes](https://github.com/koxudaxi/datamodel-code-generator/releases) - [Commits](https://github.com/koxudaxi/datamodel-code-generator/compare/0.25.9...0.26.1) --- updated-dependencies: - dependency-name: datamodel-code-generator 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> --- 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 77d4f18331ca..d15d7b224871 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.25.9 +datamodel-code-generator==0.26.1 From f1b70e1538f9a4dae36c24306d405810738dc10a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:10:14 +0200 Subject: [PATCH 0764/1019] Build: Bump software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin (#9705) Bumps [software.amazon.s3.accessgrants:aws-s3-accessgrants-java-plugin](https://github.com/aws/aws-s3-accessgrants-plugin-java-v2) from 2.0.0 to 2.0.1. - [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-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 f816bb7d0768..44a5d9497c93 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.28.16" azuresdk-bom = "1.2.25" -awssdk-s3accessgrants = "2.0.0" +awssdk-s3accessgrants = "2.2.0" caffeine = "2.9.3" calcite = "1.10.0" datasketches = "6.0.0" From a022e8f52399ba46dc35005853cfa4365e6e87b0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:11:54 +0200 Subject: [PATCH 0765/1019] Build: Bump net.snowflake:snowflake-jdbc from 3.18.0 to 3.19.0 (#11057) Bumps [net.snowflake:snowflake-jdbc](https://github.com/snowflakedb/snowflake-jdbc) from 3.18.0 to 3.19.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.18.0...v3.19.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 44a5d9497c93..4977de04c9f8 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -81,7 +81,7 @@ roaringbitmap = "1.3.0" s3mock-junit5 = "2.17.0" scala-collection-compat = "2.12.0" slf4j = "2.0.16" -snowflake-jdbc = "3.18.0" +snowflake-jdbc = "3.19.0" spark-hive33 = "3.3.4" spark-hive34 = "3.4.3" spark-hive35 = "3.5.2" From 3bd1561f1bd119184a946407fcb23b78f620af5a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Sat, 12 Oct 2024 21:15:41 +0200 Subject: [PATCH 0766/1019] Build: Bump org.apache.hadoop.thirdparty:hadoop-shaded-guava (#11061) Bumps org.apache.hadoop.thirdparty:hadoop-shaded-guava from 1.2.0 to 1.3.0. --- updated-dependencies: - dependency-name: org.apache.hadoop.thirdparty:hadoop-shaded-guava 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> --- kafka-connect/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kafka-connect/build.gradle b/kafka-connect/build.gradle index ad76fe7e1633..bef660fb6580 100644 --- a/kafka-connect/build.gradle +++ b/kafka-connect/build.gradle @@ -72,7 +72,7 @@ project(':iceberg-kafka-connect:iceberg-kafka-connect-runtime') { force 'org.codehaus.jettison:jettison:1.5.4' 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.2.0' + force 'org.apache.hadoop.thirdparty:hadoop-shaded-guava:1.3.0' } } } From a15aea39338f2154e9298c755a73fe6627d58849 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Sun, 13 Oct 2024 22:53:40 -0700 Subject: [PATCH 0767/1019] Arrow: Deprecate unused fixed width binary reader classes (#11292) --- .../vectorized/parquet/VectorizedColumnIterator.java | 8 ++++++++ .../VectorizedDictionaryEncodedParquetValuesReader.java | 8 ++++++++ .../arrow/vectorized/parquet/VectorizedPageIterator.java | 7 +++++++ .../parquet/VectorizedParquetDefinitionLevelReader.java | 8 ++++++++ 4 files changed, 31 insertions(+) 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 822ca8973f54..54445e424508 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,6 +214,10 @@ 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( @@ -278,6 +282,10 @@ 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(); } 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 21670f6b2b33..4499d0536867 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 @@ -141,6 +141,10 @@ protected void nextVal( } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated class FixedWidthBinaryDictEncodedReader extends BaseDictEncodedReader { @Override protected void nextVal( @@ -203,6 +207,10 @@ 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(); } 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 7c862d4c3fd6..d36521273d63 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 @@ -421,7 +421,10 @@ 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( @@ -496,6 +499,10 @@ VarWidthTypePageReader varWidthTypePageReader() { return new VarWidthTypePageReader(); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated FixedWidthBinaryPageReader fixedWidthBinaryPageReader() { return new FixedWidthBinaryPageReader(); } 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 2d2412c5504a..e8ec7bee8f62 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 @@ -493,6 +493,10 @@ protected void nextDictEncodedVal( } } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated class FixedWidthBinaryReader extends BaseReader { @Override protected void nextVal( @@ -732,6 +736,10 @@ TimestampInt96Reader timestampInt96Reader() { return new TimestampInt96Reader(); } + /** + * @deprecated since 1.7.0, will be removed in 1.8.0. + */ + @Deprecated FixedWidthBinaryReader fixedWidthBinaryReader() { return new FixedWidthBinaryReader(); } From 0d34601c714d6018c671ce6277be4ab198fb28e4 Mon Sep 17 00:00:00 2001 From: Yujiang Zhong <42907416+zhongyujiang@users.noreply.github.com> Date: Mon, 14 Oct 2024 14:47:29 +0800 Subject: [PATCH 0768/1019] API, Spark: Make StrictMetricsEvaluator not fail on nested column predicates (#11261) --- .../expressions/StrictMetricsEvaluator.java | 74 +++++++++++-------- .../TestStrictMetricsEvaluator.java | 63 +++++++++++++++- .../iceberg/spark/extensions/TestDelete.java | 22 ++++++ 3 files changed, 124 insertions(+), 35 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java index 4aee75c447d3..1a5a884f651a 100644 --- a/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java +++ b/api/src/main/java/org/apache/iceberg/expressions/StrictMetricsEvaluator.java @@ -29,9 +29,7 @@ import org.apache.iceberg.DataFile; import org.apache.iceberg.Schema; import org.apache.iceberg.expressions.ExpressionVisitors.BoundExpressionVisitor; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.types.Conversions; -import org.apache.iceberg.types.Types; import org.apache.iceberg.types.Types.StructType; import org.apache.iceberg.util.NaNUtil; @@ -51,7 +49,6 @@ * checks for NaN is necessary in order to not include files that may contain rows that don't match. */ public class StrictMetricsEvaluator { - private final Schema schema; private final StructType struct; private final Expression expr; @@ -60,7 +57,6 @@ public StrictMetricsEvaluator(Schema schema, Expression unbound) { } public StrictMetricsEvaluator(Schema schema, Expression unbound, boolean caseSensitive) { - this.schema = schema; this.struct = schema.asStruct(); this.expr = Binder.bind(struct, rewriteNot(unbound), caseSensitive); } @@ -144,8 +140,9 @@ public Boolean isNull(BoundReference ref) { // no need to check whether the field is required because binding evaluates that case // if the column has any non-null values, the expression does not match int id = ref.fieldId(); - Preconditions.checkNotNull( - struct.field(id), "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (containsNullsOnly(id)) { return ROWS_MUST_MATCH; @@ -159,8 +156,9 @@ public Boolean notNull(BoundReference ref) { // no need to check whether the field is required because binding evaluates that case // if the column has any null values, the expression does not match int id = ref.fieldId(); - Preconditions.checkNotNull( - struct.field(id), "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (nullCounts != null && nullCounts.containsKey(id) && nullCounts.get(id) == 0) { return ROWS_MUST_MATCH; @@ -199,15 +197,16 @@ public Boolean notNaN(BoundReference ref) { public Boolean lt(BoundReference ref, Literal lit) { // Rows must match when: <----------Min----Max---X-------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); int cmp = lit.comparator().compare(upper, lit.value()); if (cmp < 0) { @@ -222,15 +221,16 @@ public Boolean lt(BoundReference ref, Literal lit) { public Boolean ltEq(BoundReference ref, Literal lit) { // Rows must match when: <----------Min----Max---X-------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); int cmp = lit.comparator().compare(upper, lit.value()); if (cmp <= 0) { @@ -245,15 +245,16 @@ public Boolean ltEq(BoundReference ref, Literal lit) { public Boolean gt(BoundReference ref, Literal lit) { // Rows must match when: <-------X---Min----Max----------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (lowerBounds != null && lowerBounds.containsKey(id)) { - T lower = Conversions.fromByteBuffer(field.type(), lowerBounds.get(id)); + T lower = Conversions.fromByteBuffer(ref.type(), lowerBounds.get(id)); if (NaNUtil.isNaN(lower)) { // NaN indicates unreliable bounds. See the StrictMetricsEvaluator docs for more. @@ -273,15 +274,16 @@ public Boolean gt(BoundReference ref, Literal lit) { public Boolean gtEq(BoundReference ref, Literal lit) { // Rows must match when: <-------X---Min----Max----------> Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; } if (lowerBounds != null && lowerBounds.containsKey(id)) { - T lower = Conversions.fromByteBuffer(field.type(), lowerBounds.get(id)); + T lower = Conversions.fromByteBuffer(ref.type(), lowerBounds.get(id)); if (NaNUtil.isNaN(lower)) { // NaN indicates unreliable bounds. See the StrictMetricsEvaluator docs for more. @@ -301,8 +303,9 @@ public Boolean gtEq(BoundReference ref, Literal lit) { public Boolean eq(BoundReference ref, Literal lit) { // Rows must match when Min == X == Max Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; @@ -319,7 +322,7 @@ public Boolean eq(BoundReference ref, Literal lit) { return ROWS_MIGHT_NOT_MATCH; } - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); cmp = lit.comparator().compare(upper, lit.value()); if (cmp != 0) { @@ -336,8 +339,9 @@ public Boolean eq(BoundReference ref, Literal lit) { public Boolean notEq(BoundReference ref, Literal lit) { // Rows must match when X < Min or Max < X because it is not in the range Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (containsNullsOnly(id) || containsNaNsOnly(id)) { return ROWS_MUST_MATCH; @@ -358,7 +362,7 @@ public Boolean notEq(BoundReference ref, Literal lit) { } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); int cmp = lit.comparator().compare(upper, lit.value()); if (cmp < 0) { @@ -372,8 +376,9 @@ public Boolean notEq(BoundReference ref, Literal lit) { @Override public Boolean in(BoundReference ref, Set literalSet) { Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (canContainNulls(id) || canContainNaNs(id)) { return ROWS_MIGHT_NOT_MATCH; @@ -390,7 +395,7 @@ public Boolean in(BoundReference ref, Set literalSet) { } // check if the upper bound is in the set - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); if (!literalSet.contains(upper)) { return ROWS_MIGHT_NOT_MATCH; } @@ -411,8 +416,9 @@ public Boolean in(BoundReference ref, Set literalSet) { @Override public Boolean notIn(BoundReference ref, Set literalSet) { Integer id = ref.fieldId(); - Types.NestedField field = struct.field(id); - Preconditions.checkNotNull(field, "Cannot filter by nested column: %s", schema.findField(id)); + if (isNestedColumn(id)) { + return ROWS_MIGHT_NOT_MATCH; + } if (containsNullsOnly(id) || containsNaNsOnly(id)) { return ROWS_MUST_MATCH; @@ -439,7 +445,7 @@ public Boolean notIn(BoundReference ref, Set literalSet) { } if (upperBounds != null && upperBounds.containsKey(id)) { - T upper = Conversions.fromByteBuffer(field.type(), upperBounds.get(id)); + T upper = Conversions.fromByteBuffer(ref.type(), upperBounds.get(id)); literals = literals.stream() .filter(v -> ref.comparator().compare(upper, v) >= 0) @@ -466,6 +472,10 @@ public Boolean notStartsWith(BoundReference ref, Literal lit) { return ROWS_MIGHT_NOT_MATCH; } + private boolean isNestedColumn(int id) { + return struct.field(id) == null; + } + private boolean canContainNulls(Integer id) { return nullCounts == null || (nullCounts.containsKey(id) && nullCounts.get(id) > 0); } diff --git a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java index a23afb15dde5..f34cd730df77 100644 --- a/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java +++ b/api/src/test/java/org/apache/iceberg/expressions/TestStrictMetricsEvaluator.java @@ -66,7 +66,14 @@ public class TestStrictMetricsEvaluator { optional(11, "all_nulls_double", Types.DoubleType.get()), optional(12, "all_nans_v1_stats", Types.FloatType.get()), optional(13, "nan_and_null_only", Types.DoubleType.get()), - optional(14, "no_nan_stats", Types.DoubleType.get())); + optional(14, "no_nan_stats", Types.DoubleType.get()), + optional( + 15, + "struct", + Types.StructType.of( + Types.NestedField.optional(16, "nested_col_no_stats", Types.IntegerType.get()), + Types.NestedField.optional( + 17, "nested_col_with_stats", Types.IntegerType.get())))); private static final int INT_MIN_VALUE = 30; private static final int INT_MAX_VALUE = 79; @@ -88,6 +95,7 @@ public class TestStrictMetricsEvaluator { .put(12, 50L) .put(13, 50L) .put(14, 50L) + .put(17, 50L) .buildOrThrow(), // null value counts ImmutableMap.builder() @@ -97,6 +105,7 @@ public class TestStrictMetricsEvaluator { .put(11, 50L) .put(12, 0L) .put(13, 1L) + .put(17, 0L) .buildOrThrow(), // nan value counts ImmutableMap.of( @@ -108,13 +117,15 @@ public class TestStrictMetricsEvaluator { 1, toByteBuffer(IntegerType.get(), INT_MIN_VALUE), 7, toByteBuffer(IntegerType.get(), 5), 12, toByteBuffer(Types.FloatType.get(), Float.NaN), - 13, toByteBuffer(Types.DoubleType.get(), Double.NaN)), + 13, toByteBuffer(Types.DoubleType.get(), Double.NaN), + 17, toByteBuffer(Types.IntegerType.get(), INT_MIN_VALUE)), // upper bounds ImmutableMap.of( 1, toByteBuffer(IntegerType.get(), INT_MAX_VALUE), 7, toByteBuffer(IntegerType.get(), 5), 12, toByteBuffer(Types.FloatType.get(), Float.NaN), - 13, toByteBuffer(Types.DoubleType.get(), Double.NaN))); + 13, toByteBuffer(Types.DoubleType.get(), Double.NaN), + 17, toByteBuffer(IntegerType.get(), INT_MAX_VALUE))); private static final DataFile FILE_2 = new TestDataFile( @@ -627,4 +638,50 @@ public void testIntegerNotIn() { shouldRead = new StrictMetricsEvaluator(SCHEMA, notIn("no_nulls", "abc", "def")).eval(FILE); assertThat(shouldRead).as("Should not match: no_nulls field does not have bounds").isFalse(); } + + @Test + public void testEvaluateOnNestedColumnWithoutStats() { + boolean shouldRead = + new StrictMetricsEvaluator( + SCHEMA, greaterThanOrEqual("struct.nested_col_no_stats", INT_MIN_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("greaterThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator( + SCHEMA, lessThanOrEqual("struct.nested_col_no_stats", INT_MAX_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("lessThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, isNull("struct.nested_col_no_stats")).eval(FILE); + assertThat(shouldRead).as("isNull nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, notNull("struct.nested_col_no_stats")).eval(FILE); + assertThat(shouldRead).as("notNull nested column should not match").isFalse(); + } + + @Test + public void testEvaluateOnNestedColumnWithStats() { + boolean shouldRead = + new StrictMetricsEvaluator( + SCHEMA, greaterThanOrEqual("struct.nested_col_with_stats", INT_MIN_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("greaterThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator( + SCHEMA, lessThanOrEqual("struct.nested_col_with_stats", INT_MAX_VALUE)) + .eval(FILE); + assertThat(shouldRead).as("lessThanOrEqual nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, isNull("struct.nested_col_with_stats")).eval(FILE); + assertThat(shouldRead).as("isNull nested column should not match").isFalse(); + + shouldRead = + new StrictMetricsEvaluator(SCHEMA, notNull("struct.nested_col_with_stats")).eval(FILE); + assertThat(shouldRead).as("notNull nested column should not match").isFalse(); + } } 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 d258079541cd..42eb2af774e9 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 @@ -1401,6 +1401,28 @@ public void testDeleteToCustomWapBranchWithoutWhereClause() throws NoSuchTableEx }); } + @TestTemplate + public void testDeleteWithFilterOnNestedColumn() { + createAndInitNestedColumnsTable(); + + sql("INSERT INTO TABLE %s VALUES (1, named_struct(\"c1\", 3, \"c2\", \"v1\"))", tableName); + sql("INSERT INTO TABLE %s VALUES (2, named_struct(\"c1\", 2, \"c2\", \"v2\"))", tableName); + + sql("DELETE FROM %s WHERE complex.c1 > 3", tableName); + assertEquals( + "Should have expected rows", + ImmutableList.of(row(1), row(2)), + sql("SELECT id FROM %s order by id", tableName)); + + sql("DELETE FROM %s WHERE complex.c1 = 3", tableName); + assertEquals( + "Should have expected rows", ImmutableList.of(row(2)), sql("SELECT id FROM %s", tableName)); + + sql("DELETE FROM %s t WHERE t.complex.c1 = 2", tableName); + assertEquals( + "Should have expected rows", ImmutableList.of(), sql("SELECT id FROM %s", tableName)); + } + // TODO: multiple stripes for ORC protected void createAndInitPartitionedTable() { From 49ece3f2452c57b6bce44013863716335ce14eb1 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Mon, 14 Oct 2024 16:16:25 +0530 Subject: [PATCH 0769/1019] Build: Use the active shadow plugin (#11315) --- aws-bundle/build.gradle | 2 +- azure-bundle/build.gradle | 2 +- build.gradle | 4 ++-- flink/v1.18/build.gradle | 2 +- flink/v1.19/build.gradle | 2 +- flink/v1.20/build.gradle | 2 +- gcp-bundle/build.gradle | 2 +- hive-runtime/build.gradle | 2 +- hive3-orc-bundle/build.gradle | 2 +- spark/v3.3/build.gradle | 2 +- spark/v3.4/build.gradle | 2 +- spark/v3.5/build.gradle | 2 +- 12 files changed, 13 insertions(+), 13 deletions(-) diff --git a/aws-bundle/build.gradle b/aws-bundle/build.gradle index 16952d78d8e5..82bfda83498b 100644 --- a/aws-bundle/build.gradle +++ b/aws-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-aws-bundle") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/azure-bundle/build.gradle b/azure-bundle/build.gradle index 9b1cd5b42fb8..46d1b3daea5b 100644 --- a/azure-bundle/build.gradle +++ b/azure-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-azure-bundle") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/build.gradle b/build.gradle index e5fc8e0b5af2..e66ea793bbf7 100644 --- a/build.gradle +++ b/build.gradle @@ -26,7 +26,7 @@ buildscript { gradlePluginPortal() } dependencies { - classpath 'io.github.goooler.shadow:shadow-gradle-plugin:8.1.8' + classpath 'com.gradleup.shadow:shadow-gradle-plugin:8.3.3' classpath 'com.palantir.baseline:gradle-baseline-java:5.69.0' classpath 'com.diffplug.spotless:spotless-plugin-gradle:6.25.0' classpath 'gradle.plugin.org.inferred:gradle-processors:3.7.0' @@ -246,7 +246,7 @@ subprojects { } project(':iceberg-bundled-guava') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.18/build.gradle b/flink/v1.18/build.gradle index aac01c9c6931..83dc07523a3c 100644 --- a/flink/v1.18/build.gradle +++ b/flink/v1.18/build.gradle @@ -127,7 +127,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.19/build.gradle b/flink/v1.19/build.gradle index 6327ebf99dbd..50bcadb618e4 100644 --- a/flink/v1.19/build.gradle +++ b/flink/v1.19/build.gradle @@ -128,7 +128,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/flink/v1.20/build.gradle b/flink/v1.20/build.gradle index 294c88d90709..4a1bae660bdb 100644 --- a/flink/v1.20/build.gradle +++ b/flink/v1.20/build.gradle @@ -128,7 +128,7 @@ project(":iceberg-flink:iceberg-flink-${flinkMajorVersion}") { } project(":iceberg-flink:iceberg-flink-runtime-${flinkMajorVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/gcp-bundle/build.gradle b/gcp-bundle/build.gradle index 1b339a66c195..e7d1b5ddb468 100644 --- a/gcp-bundle/build.gradle +++ b/gcp-bundle/build.gradle @@ -19,7 +19,7 @@ project(":iceberg-gcp-bundle") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/hive-runtime/build.gradle b/hive-runtime/build.gradle index 6f891be8087e..a107afcb3777 100644 --- a/hive-runtime/build.gradle +++ b/hive-runtime/build.gradle @@ -20,7 +20,7 @@ def hiveVersions = (System.getProperty("hiveVersions") != null ? System.getProperty("hiveVersions") : System.getProperty("defaultHiveVersions")).split(",") project(':iceberg-hive-runtime') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/hive3-orc-bundle/build.gradle b/hive3-orc-bundle/build.gradle index 3ca89bd47435..4e1e9c5dd222 100644 --- a/hive3-orc-bundle/build.gradle +++ b/hive3-orc-bundle/build.gradle @@ -21,7 +21,7 @@ // name. This is to be used by Hive3 for features including e.g. vectorization. project(':iceberg-hive3-orc-bundle') { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.3/build.gradle b/spark/v3.3/build.gradle index c081cffeb644..87e519177470 100644 --- a/spark/v3.3/build.gradle +++ b/spark/v3.3/build.gradle @@ -187,7 +187,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.4/build.gradle b/spark/v3.4/build.gradle index a978cda9db0b..a34ea6b92826 100644 --- a/spark/v3.4/build.gradle +++ b/spark/v3.4/build.gradle @@ -191,7 +191,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar diff --git a/spark/v3.5/build.gradle b/spark/v3.5/build.gradle index c8d8bbf396a0..e3c9ef4f0230 100644 --- a/spark/v3.5/build.gradle +++ b/spark/v3.5/build.gradle @@ -193,7 +193,7 @@ project(":iceberg-spark:iceberg-spark-extensions-${sparkMajorVersion}_${scalaVer } project(":iceberg-spark:iceberg-spark-runtime-${sparkMajorVersion}_${scalaVersion}") { - apply plugin: 'io.github.goooler.shadow' + apply plugin: 'com.gradleup.shadow' tasks.jar.dependsOn tasks.shadowJar From 10f8e2aa8625dc69e17825c0aa9d2684a1b9a03a Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 14 Oct 2024 12:46:54 +0200 Subject: [PATCH 0770/1019] Build: Bump software.amazon.awssdk:bom from 2.28.16 to 2.28.21 (#11311) Bumps software.amazon.awssdk:bom from 2.28.16 to 2.28.21. --- 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 4977de04c9f8..65352e49d770 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.16" +awssdk-bom = "2.28.21" azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.2.0" caffeine = "2.9.3" From 383b74b0af26a1e69e941cf20f73fb1d9bb33d4f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 14 Oct 2024 14:37:24 +0200 Subject: [PATCH 0771/1019] Build: Bump org.apache.datasketches:datasketches-java (#11307) Bumps org.apache.datasketches:datasketches-java from 6.0.0 to 6.1.1. --- updated-dependencies: - dependency-name: org.apache.datasketches:datasketches-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> --- 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 65352e49d770..d49c082b09d6 100644 --- a/gradle/libs.versions.toml +++ b/gradle/libs.versions.toml @@ -34,7 +34,7 @@ azuresdk-bom = "1.2.25" awssdk-s3accessgrants = "2.2.0" caffeine = "2.9.3" calcite = "1.10.0" -datasketches = "6.0.0" +datasketches = "6.1.1" delta-standalone = "3.2.1" delta-spark = "3.2.1" esotericsoftware-kryo = "4.0.3" From 11fd224c19d34edc79a3fe808c5f7a34e560e7c2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Mon, 14 Oct 2024 18:54:33 +0200 Subject: [PATCH 0772/1019] Core: Switch usage to DataFileSet / DeleteFileSet (#11158) --- .../apache/iceberg/BaseOverwriteFiles.java | 5 +- .../org/apache/iceberg/BaseRewriteFiles.java | 4 +- .../java/org/apache/iceberg/FastAppend.java | 8 +-- .../apache/iceberg/ManifestFilterManager.java | 58 ++++++++++++------- .../iceberg/MergingSnapshotProducer.java | 28 ++++++--- .../org/apache/iceberg/SnapshotProducer.java | 16 ++--- .../RewriteDataFilesCommitManager.java | 7 +-- .../iceberg/actions/RewriteFileGroup.java | 10 ++-- .../actions/RewritePositionDeletesGroup.java | 10 ++-- .../org/apache/iceberg/TestDeleteFiles.java | 15 ++++- .../apache/iceberg/hive/HiveTableTest.java | 2 +- .../iceberg/nessie/TestNessieTable.java | 2 +- .../source/SparkPositionDeletesRewrite.java | 4 +- .../iceberg/spark/source/SparkWrite.java | 4 +- .../spark/TestFileRewriteCoordinator.java | 7 ++- .../iceberg/spark/data/TestHelpers.java | 4 +- .../source/TestPositionDeletesTable.java | 3 +- 17 files changed, 115 insertions(+), 72 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java index d929bc068ec2..16fbc0dd1ebc 100644 --- a/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseOverwriteFiles.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg; -import java.util.Set; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Evaluator; import org.apache.iceberg.expressions.Expression; @@ -26,11 +25,11 @@ import org.apache.iceberg.expressions.Projections; import org.apache.iceberg.expressions.StrictMetricsEvaluator; 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.DataFileSet; public class BaseOverwriteFiles extends MergingSnapshotProducer implements OverwriteFiles { - private final Set deletedDataFiles = Sets.newHashSet(); + private final DataFileSet deletedDataFiles = DataFileSet.create(); private boolean validateAddedFilesMatchOverwriteFilter = false; private Long startingSnapshotId = null; private Expression conflictDetectionFilter = null; diff --git a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java index d231536d0642..b25681de4238 100644 --- a/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java +++ b/core/src/main/java/org/apache/iceberg/BaseRewriteFiles.java @@ -21,10 +21,10 @@ import java.util.Set; 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.Sets; +import org.apache.iceberg.util.DataFileSet; class BaseRewriteFiles extends MergingSnapshotProducer implements RewriteFiles { - private final Set replacedDataFiles = Sets.newHashSet(); + private final DataFileSet replacedDataFiles = DataFileSet.create(); private Long startingSnapshotId = null; BaseRewriteFiles(String tableName, TableOperations ops) { diff --git a/core/src/main/java/org/apache/iceberg/FastAppend.java b/core/src/main/java/org/apache/iceberg/FastAppend.java index 1bae2e2fc5a0..1b6e1b3b52bc 100644 --- a/core/src/main/java/org/apache/iceberg/FastAppend.java +++ b/core/src/main/java/org/apache/iceberg/FastAppend.java @@ -30,7 +30,7 @@ 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.Lists; -import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DataFileSet; /** * {@link AppendFiles Append} implementation that adds a new manifest file for the write. @@ -43,8 +43,7 @@ class FastAppend extends SnapshotProducer implements AppendFiles { private final TableOperations ops; private final PartitionSpec spec; private final SnapshotSummary.Builder summaryBuilder = SnapshotSummary.builder(); - private final List newFiles = Lists.newArrayList(); - private final CharSequenceSet newFilePaths = CharSequenceSet.empty(); + private final DataFileSet newFiles = DataFileSet.create(); private final List appendManifests = Lists.newArrayList(); private final List rewrittenAppendManifests = Lists.newArrayList(); private List newManifests = null; @@ -86,9 +85,8 @@ protected Map summary() { @Override public FastAppend appendFile(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - if (newFilePaths.add(file.path())) { + if (newFiles.add(file)) { this.hasNewFiles = true; - newFiles.add(file); summaryBuilder.addedFile(spec, file); } diff --git a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java index 106be74fa3ad..fddb1a161637 100644 --- a/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java +++ b/core/src/main/java/org/apache/iceberg/ManifestFilterManager.java @@ -25,6 +25,7 @@ import java.util.Set; import java.util.concurrent.ExecutorService; import java.util.function.Supplier; +import java.util.stream.Collectors; import org.apache.iceberg.exceptions.RuntimeIOException; import org.apache.iceberg.exceptions.ValidationException; import org.apache.iceberg.expressions.Expression; @@ -39,9 +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.CharSequenceWrapper; import org.apache.iceberg.util.ManifestFileUtil; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.PartitionSet; @@ -71,9 +70,9 @@ public String partition() { private final PartitionSet deleteFilePartitions; 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 hasPathOnlyDeletes = false; private boolean failAnyDelete = false; private boolean failMissingDeletePaths = false; private int duplicateDeleteCount = 0; @@ -102,6 +101,8 @@ protected ManifestFilterManager( protected abstract ManifestReader newManifestReader(ManifestFile manifest); + protected abstract Set newFileSet(); + protected void failAnyDelete() { this.failAnyDelete = true; } @@ -153,7 +154,7 @@ void caseSensitive(boolean newCaseSensitive) { void delete(F file) { Preconditions.checkNotNull(file, "Cannot delete file: null"); invalidateFilteredCache(); - deletePaths.add(file.path()); + deleteFiles.add(file); deleteFilePartitions.add(file.specId(), file.partition()); } @@ -161,12 +162,12 @@ void delete(F file) { void delete(CharSequence path) { Preconditions.checkNotNull(path, "Cannot delete file path: null"); invalidateFilteredCache(); - this.hasPathOnlyDeletes = true; deletePaths.add(path); } boolean containsDeletes() { return !deletePaths.isEmpty() + || !deleteFiles.isEmpty() || deleteExpression != Expressions.alwaysFalse() || !dropPartitions.isEmpty(); } @@ -233,23 +234,37 @@ SnapshotSummary.Builder buildSummary(Iterable manifests) { @SuppressWarnings("CollectionUndefinedEquality") private void validateRequiredDeletes(ManifestFile... manifests) { if (failMissingDeletePaths) { - CharSequenceSet deletedFiles = deletedFiles(manifests); + Set deletedFiles = deletedFiles(manifests); + ValidationException.check( + deletedFiles.containsAll(deleteFiles), + "Missing required files to delete: %s", + COMMA.join( + deleteFiles.stream() + .filter(f -> !deletedFiles.contains(f)) + .map(ContentFile::location) + .collect(Collectors.toList()))); + + CharSequenceSet deletedFilePaths = + deletedFiles.stream() + .map(ContentFile::path) + .collect(Collectors.toCollection(CharSequenceSet::empty)); + ValidationException.check( - deletedFiles.containsAll(deletePaths), + deletedFilePaths.containsAll(deletePaths), "Missing required files to delete: %s", - COMMA.join(Iterables.filter(deletePaths, path -> !deletedFiles.contains(path)))); + COMMA.join(Iterables.filter(deletePaths, path -> !deletedFilePaths.contains(path)))); } } - private CharSequenceSet deletedFiles(ManifestFile[] manifests) { - CharSequenceSet deletedFiles = CharSequenceSet.empty(); + private Set deletedFiles(ManifestFile[] manifests) { + Set deletedFiles = newFileSet(); if (manifests != null) { for (ManifestFile manifest : manifests) { Iterable manifestDeletes = filteredManifestToDeletedFiles.get(manifest); if (manifestDeletes != null) { for (F file : manifestDeletes) { - deletedFiles.add(file.path()); + deletedFiles.add(file); } } } @@ -345,9 +360,9 @@ private boolean canContainDeletedFiles(ManifestFile manifest) { } boolean canContainDroppedFiles; - if (hasPathOnlyDeletes) { + if (!deletePaths.isEmpty()) { canContainDroppedFiles = true; - } else if (!deletePaths.isEmpty()) { + } 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); @@ -374,6 +389,7 @@ private boolean manifestHasDeletedFiles( F file = entry.file(); boolean markedForDelete = deletePaths.contains(file.path()) + || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete && entry.isLive() @@ -387,7 +403,7 @@ private boolean manifestHasDeletedFiles( || 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.path()); + file.location()); if (allRowsMatch) { if (failAnyDelete) { @@ -409,8 +425,7 @@ private ManifestFile filterManifestWithDeletedFiles( boolean isDelete = reader.isDeleteManifestReader(); // when this point is reached, there is at least one file that will be deleted in the // manifest. produce a copy of the manifest with all deleted files removed. - List deletedFiles = Lists.newArrayList(); - Set deletedPaths = Sets.newHashSet(); + Set deletedFiles = newFileSet(); try { ManifestWriter writer = newManifestWriter(reader.spec()); @@ -422,6 +437,7 @@ private ManifestFile filterManifestWithDeletedFiles( F file = entry.file(); boolean markedForDelete = deletePaths.contains(file.path()) + || deleteFiles.contains(file) || dropPartitions.contains(file.specId(), file.partition()) || (isDelete && entry.isLive() @@ -436,23 +452,21 @@ private ManifestFile filterManifestWithDeletedFiles( // the expression "Cannot delete file where some, but not all, rows match filter %s: %s", this.deleteExpression, - file.path()); + file.location()); if (allRowsMatch) { writer.delete(entry); - CharSequenceWrapper wrapper = CharSequenceWrapper.wrap(entry.file().path()); - if (deletedPaths.contains(wrapper)) { + if (deletedFiles.contains(file)) { LOG.warn( "Deleting a duplicate path from manifest {}: {}", manifest.path(), - wrapper.get()); + 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(entry.file().copyWithoutStats()); - deletedPaths.add(wrapper); + deletedFiles.add(file.copyWithoutStats()); } } else { writer.existing(entry); diff --git a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java index 6a4da2abc9b6..2209b348227d 100644 --- a/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/MergingSnapshotProducer.java @@ -50,6 +50,8 @@ 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.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; @@ -81,8 +83,8 @@ abstract class MergingSnapshotProducer extends SnapshotProducer { // update data private final Map> newDataFilesBySpec = Maps.newHashMap(); - private final CharSequenceSet newDataFilePaths = CharSequenceSet.empty(); - private final CharSequenceSet newDeleteFilePaths = CharSequenceSet.empty(); + private final DataFileSet newDataFiles = DataFileSet.create(); + private final DeleteFileSet newDeleteFiles = DeleteFileSet.create(); private Long newDataFilesDataSequenceNumber; private final Map> newDeleteFilesBySpec = Maps.newHashMap(); private final List appendManifests = Lists.newArrayList(); @@ -234,7 +236,7 @@ protected boolean addsDeleteFiles() { /** Add a data file to the new snapshot. */ protected void add(DataFile file) { Preconditions.checkNotNull(file, "Invalid data file: null"); - if (newDataFilePaths.add(file.path())) { + if (newDataFiles.add(file)) { PartitionSpec fileSpec = ops.current().spec(file.specId()); Preconditions.checkArgument( fileSpec != null, @@ -244,9 +246,9 @@ protected void add(DataFile file) { addedFilesSummary.addedFile(fileSpec, file); hasNewDataFiles = true; - List newDataFiles = + List dataFiles = newDataFilesBySpec.computeIfAbsent(fileSpec, ignored -> Lists.newArrayList()); - newDataFiles.add(file); + dataFiles.add(file); } } @@ -268,7 +270,7 @@ private void add(DeleteFileHolder fileHolder) { List deleteFiles = newDeleteFilesBySpec.computeIfAbsent(specId, s -> Lists.newArrayList()); - if (newDeleteFilePaths.add(fileHolder.deleteFile().path())) { + if (newDeleteFiles.add(fileHolder.deleteFile())) { deleteFiles.add(fileHolder); addedFilesSummary.addedFile(fileSpec, fileHolder.deleteFile()); hasNewDeleteFiles = true; @@ -970,9 +972,9 @@ private List newDataFilesAsManifests() { if (cachedNewDataManifests.isEmpty()) { newDataFilesBySpec.forEach( - (dataSpec, newDataFiles) -> { + (dataSpec, dataFiles) -> { List newDataManifests = - writeDataManifests(newDataFiles, newDataFilesDataSequenceNumber, dataSpec); + writeDataManifests(dataFiles, newDataFilesDataSequenceNumber, dataSpec); cachedNewDataManifests.addAll(newDataManifests); }); this.hasNewDataFiles = false; @@ -1032,6 +1034,11 @@ protected ManifestWriter newManifestWriter(PartitionSpec manifestSpec) protected ManifestReader newManifestReader(ManifestFile manifest) { return MergingSnapshotProducer.this.newManifestReader(manifest); } + + @Override + protected Set newFileSet() { + return DataFileSet.create(); + } } private class DataFileMergeManager extends ManifestMergeManager { @@ -1085,6 +1092,11 @@ protected ManifestWriter newManifestWriter(PartitionSpec manifestSpe protected ManifestReader newManifestReader(ManifestFile manifest) { return MergingSnapshotProducer.this.newDeleteManifestReader(manifest); } + + @Override + protected Set newFileSet() { + return DeleteFileSet.create(); + } } private class DeleteFileMergeManager extends ManifestMergeManager { diff --git a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java index f750e88e86d9..27724f787dd2 100644 --- a/core/src/main/java/org/apache/iceberg/SnapshotProducer.java +++ b/core/src/main/java/org/apache/iceberg/SnapshotProducer.java @@ -36,6 +36,7 @@ import java.io.IOException; import java.math.RoundingMode; import java.util.Arrays; +import java.util.Collection; import java.util.List; import java.util.Locale; import java.util.Map; @@ -567,17 +568,17 @@ protected boolean cleanupAfterCommit() { return true; } - protected List writeDataManifests(List files, PartitionSpec spec) { + protected List writeDataManifests(Collection files, PartitionSpec spec) { return writeDataManifests(files, null /* inherit data seq */, spec); } protected List writeDataManifests( - List files, Long dataSeq, PartitionSpec spec) { + Collection files, Long dataSeq, PartitionSpec spec) { return writeManifests(files, group -> writeDataFileGroup(group, dataSeq, spec)); } private List writeDataFileGroup( - List files, Long dataSeq, PartitionSpec spec) { + Collection files, Long dataSeq, PartitionSpec spec) { RollingManifestWriter writer = newRollingManifestWriter(spec); try (RollingManifestWriter closableWriter = writer) { @@ -594,12 +595,12 @@ private List writeDataFileGroup( } protected List writeDeleteManifests( - List files, PartitionSpec spec) { + Collection files, PartitionSpec spec) { return writeManifests(files, group -> writeDeleteFileGroup(group, spec)); } private List writeDeleteFileGroup( - List files, PartitionSpec spec) { + Collection files, PartitionSpec spec) { RollingManifestWriter writer = newRollingDeleteManifestWriter(spec); try (RollingManifestWriter closableWriter = writer) { @@ -618,7 +619,7 @@ private List writeDeleteFileGroup( } private static List writeManifests( - List files, Function, List> writeFunc) { + Collection files, Function, List> writeFunc) { int parallelism = manifestWriterCount(ThreadPools.WORKER_THREAD_POOL_SIZE, files.size()); List> groups = divide(files, parallelism); Queue manifests = Queues.newConcurrentLinkedQueue(); @@ -630,7 +631,8 @@ private static List writeManifests( return ImmutableList.copyOf(manifests); } - private static List> divide(List list, int groupCount) { + private static List> divide(Collection collection, int groupCount) { + List list = Lists.newArrayList(collection); int groupSize = IntMath.divide(list.size(), groupCount, RoundingMode.CEILING); return Lists.partition(list, groupSize); } 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 45b4bcf0a4d9..03d23231c0f1 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteDataFilesCommitManager.java @@ -20,14 +20,13 @@ import java.util.Map; import java.util.Set; -import org.apache.iceberg.DataFile; import org.apache.iceberg.RewriteFiles; import org.apache.iceberg.Table; import org.apache.iceberg.exceptions.CleanableFailure; import org.apache.iceberg.exceptions.CommitStateUnknownException; 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.Sets; +import org.apache.iceberg.util.DataFileSet; import org.apache.iceberg.util.Tasks; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -73,8 +72,8 @@ public RewriteDataFilesCommitManager( * @param fileGroups fileSets to commit */ public void commitFileGroups(Set fileGroups) { - Set rewrittenDataFiles = Sets.newHashSet(); - Set addedDataFiles = Sets.newHashSet(); + DataFileSet rewrittenDataFiles = DataFileSet.create(); + DataFileSet addedDataFiles = DataFileSet.create(); for (RewriteFileGroup group : fileGroups) { rewrittenDataFiles.addAll(group.rewrittenFiles()); addedDataFiles.addAll(group.addedFiles()); diff --git a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java index dd1358f2ed40..dfc9842780f5 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewriteFileGroup.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.actions; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Set; @@ -29,6 +28,7 @@ import org.apache.iceberg.actions.RewriteDataFiles.FileGroupInfo; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.DataFileSet; /** * Container class representing a set of files to be rewritten by a RewriteAction and the new files @@ -38,7 +38,7 @@ public class RewriteFileGroup { private final FileGroupInfo info; private final List fileScanTasks; - private Set addedFiles = Collections.emptySet(); + private DataFileSet addedFiles = DataFileSet.create(); public RewriteFileGroup(FileGroupInfo info, List fileScanTasks) { this.info = info; @@ -54,11 +54,13 @@ public List fileScans() { } public void setOutputFiles(Set files) { - addedFiles = files; + addedFiles = DataFileSet.of(files); } public Set rewrittenFiles() { - return fileScans().stream().map(FileScanTask::file).collect(Collectors.toSet()); + return fileScans().stream() + .map(FileScanTask::file) + .collect(Collectors.toCollection(DataFileSet::create)); } public Set addedFiles() { diff --git a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java index 2be7145bcd34..d1c688417a64 100644 --- a/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java +++ b/core/src/main/java/org/apache/iceberg/actions/RewritePositionDeletesGroup.java @@ -18,7 +18,6 @@ */ package org.apache.iceberg.actions; -import java.util.Collections; import java.util.Comparator; import java.util.List; import java.util.Set; @@ -30,6 +29,7 @@ import org.apache.iceberg.actions.RewritePositionDeleteFiles.FileGroupRewriteResult; import org.apache.iceberg.relocated.com.google.common.base.MoreObjects; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.DeleteFileSet; /** * Container class representing a set of position delete files to be rewritten by a {@link @@ -40,7 +40,7 @@ public class RewritePositionDeletesGroup { private final List tasks; private final long maxRewrittenDataSequenceNumber; - private Set addedDeleteFiles = Collections.emptySet(); + private DeleteFileSet addedDeleteFiles = DeleteFileSet.create(); public RewritePositionDeletesGroup(FileGroupInfo info, List tasks) { Preconditions.checkArgument(!tasks.isEmpty(), "Tasks must not be empty"); @@ -59,7 +59,7 @@ public List tasks() { } public void setOutputFiles(Set files) { - addedDeleteFiles = files; + addedDeleteFiles = DeleteFileSet.of(files); } public long maxRewrittenDataSequenceNumber() { @@ -67,7 +67,9 @@ public long maxRewrittenDataSequenceNumber() { } public Set rewrittenDeleteFiles() { - return tasks().stream().map(PositionDeletesScanTask::file).collect(Collectors.toSet()); + return tasks().stream() + .map(PositionDeletesScanTask::file) + .collect(Collectors.toCollection(DeleteFileSet::create)); } public Set addedDeleteFiles() { diff --git a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java index 18e3de240170..4928f998f3b1 100644 --- a/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java +++ b/core/src/test/java/org/apache/iceberg/TestDeleteFiles.java @@ -412,7 +412,20 @@ public void testDeleteValidateFileExistence() { assertThatThrownBy( () -> commit(table, table.newDelete().deleteFile(FILE_B).validateFilesExist(), branch)) - .isInstanceOf(ValidationException.class); + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/data-b.parquet"); + + assertThatThrownBy( + () -> + commit( + table, + table + .newDelete() + .deleteFile("/path/to/non-existing.parquet") + .validateFilesExist(), + branch)) + .isInstanceOf(ValidationException.class) + .hasMessage("Missing required files to delete: /path/to/non-existing.parquet"); } @TestTemplate 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 9ae3c97db47c..13c459128dec 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 @@ -213,7 +213,7 @@ public void testDropTable() throws IOException { table.newAppend().appendFile(file1).appendFile(file2).commit(); // delete file2 - table.newDelete().deleteFile(file2.path()).commit(); + table.newDelete().deleteFile(file2).commit(); String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); diff --git a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java index f0f75c842429..ca507eae575a 100644 --- a/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java +++ b/nessie/src/test/java/org/apache/iceberg/nessie/TestNessieTable.java @@ -407,7 +407,7 @@ public void testDropTable() throws IOException { table.newAppend().appendFile(file1).appendFile(file2).commit(); // delete file2 - table.newDelete().deleteFile(file2.path()).commit(); + table.newDelete().deleteFile(file2).commit(); String manifestListLocation = table.currentSnapshot().manifestListLocation().replace("file:", ""); diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java index d91779475845..73e6ab01563c 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkPositionDeletesRewrite.java @@ -36,11 +36,11 @@ import org.apache.iceberg.io.FileIO; 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.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.PositionDeletesRewriteCoordinator; import org.apache.iceberg.spark.ScanTaskSetManager; import org.apache.iceberg.spark.SparkWriteConf; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.SparkSession; @@ -148,7 +148,7 @@ public boolean useCommitCoordinator() { @Override public void commit(WriterCommitMessage[] messages) { PositionDeletesRewriteCoordinator coordinator = PositionDeletesRewriteCoordinator.get(); - coordinator.stageRewrite(table, fileSetId, ImmutableSet.copyOf(files(messages))); + coordinator.stageRewrite(table, fileSetId, DeleteFileSet.of(files(messages))); } @Override diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java index e4a0eb700be6..cc3dc592ecee 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/source/SparkWrite.java @@ -52,12 +52,12 @@ import org.apache.iceberg.io.PartitioningWriter; import org.apache.iceberg.io.RollingDataWriter; 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.Lists; import org.apache.iceberg.spark.CommitMetadata; import org.apache.iceberg.spark.FileRewriteCoordinator; import org.apache.iceberg.spark.SparkWriteConf; import org.apache.iceberg.spark.SparkWriteRequirements; +import org.apache.iceberg.util.DataFileSet; import org.apache.spark.TaskContext; import org.apache.spark.TaskContext$; import org.apache.spark.api.java.JavaSparkContext; @@ -491,7 +491,7 @@ private RewriteFiles(String fileSetID) { @Override public void commit(WriterCommitMessage[] messages) { FileRewriteCoordinator coordinator = FileRewriteCoordinator.get(); - coordinator.stageRewrite(table, fileSetID, ImmutableSet.copyOf(files(messages))); + coordinator.stageRewrite(table, fileSetID, DataFileSet.of(files(messages))); } } diff --git a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java index 3955d0395474..666634a06c02 100644 --- a/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java +++ b/spark/v3.5/spark/src/test/java/org/apache/iceberg/spark/TestFileRewriteCoordinator.java @@ -34,6 +34,7 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableSet; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.spark.source.SimpleRecord; +import org.apache.iceberg.util.DataFileSet; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoders; import org.apache.spark.sql.Row; @@ -93,7 +94,7 @@ public void testBinPackRewrite() throws NoSuchTableException, IOException { Set rewrittenFiles = taskSetManager.fetchTasks(table, fileSetID).stream() .map(t -> t.asFileScanTask().file()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DataFileSet::create)); Set addedFiles = rewriteCoordinator.fetchNewFiles(table, fileSetID); table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); } @@ -165,7 +166,7 @@ public void testSortRewrite() throws NoSuchTableException, IOException { Set rewrittenFiles = taskSetManager.fetchTasks(table, fileSetID).stream() .map(t -> t.asFileScanTask().file()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DataFileSet::create)); Set addedFiles = rewriteCoordinator.fetchNewFiles(table, fileSetID); table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); } @@ -247,7 +248,7 @@ public void testCommitMultipleRewrites() throws NoSuchTableException, IOExceptio Set addedFiles = fileSetIDs.stream() .flatMap(fileSetID -> rewriteCoordinator.fetchNewFiles(table, fileSetID).stream()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DataFileSet::create)); table.newRewrite().rewriteFiles(rewrittenFiles, addedFiles).commit(); table.refresh(); 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 c73ef630ac48..4252838d5f53 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 @@ -54,11 +54,11 @@ import org.apache.iceberg.TableScan; import org.apache.iceberg.io.CloseableIterable; 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.collect.Streams; import org.apache.iceberg.spark.SparkSchemaUtil; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.orc.storage.serde2.io.DateWritable; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -787,7 +787,7 @@ public static List dataFiles(Table table, String branch) { } public static Set deleteFiles(Table table) { - Set deleteFiles = Sets.newHashSet(); + DeleteFileSet deleteFiles = DeleteFileSet.create(); for (FileScanTask task : table.newScan().planFiles()) { deleteFiles.addAll(task.deletes()); 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 a6573171aa6a..ca934772f6af 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 @@ -72,6 +72,7 @@ import org.apache.iceberg.types.TypeUtil; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.CharSequenceSet; +import org.apache.iceberg.util.DeleteFileSet; import org.apache.iceberg.util.Pair; import org.apache.iceberg.util.StructLikeSet; import org.apache.spark.sql.AnalysisException; @@ -1576,7 +1577,7 @@ private void commit( Set rewrittenFiles = ScanTaskSetManager.get().fetchTasks(posDeletesTable, fileSetID).stream() .map(t -> ((PositionDeletesScanTask) t).file()) - .collect(Collectors.toSet()); + .collect(Collectors.toCollection(DeleteFileSet::create)); Set addedFiles = rewriteCoordinator.fetchNewFiles(posDeletesTable, fileSetID); // Assert new files and old files are equal in number but different in paths From 8af6a2a37e247fef994fb879f36ce5a6f2ea4d6e Mon Sep 17 00:00:00 2001 From: S N Munendra <9696252+munendrasn@users.noreply.github.com> Date: Mon, 14 Oct 2024 22:37:44 +0530 Subject: [PATCH 0773/1019] [AWS] S3FileIO - Add Cross-Region Bucket Access (#11259) --- .../aws/s3/TestS3FileIOIntegration.java | 29 ++++++++++++++++++- .../iceberg/aws/s3/S3FileIOProperties.java | 22 +++++++++++++- .../aws/s3/TestS3FileIOProperties.java | 11 +++++++ docs/docs/aws.md | 16 ++++++++++ 4 files changed, 76 insertions(+), 2 deletions(-) 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 388260a54657..41a07401a1e6 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 @@ -182,6 +182,29 @@ public void testNewInputStreamWithAccessPoint() throws Exception { validateRead(s3FileIO); } + @Test + public void testCrossRegionAccessEnabled() throws Exception { + clientFactory.initialize( + ImmutableMap.of(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, "true")); + S3Client s3Client = clientFactory.s3(); + String crossBucketObjectKey = String.format("%s/%s", prefix, UUID.randomUUID()); + String crossBucketObjectUri = + String.format("s3://%s/%s", crossRegionBucketName, crossBucketObjectKey); + try { + s3Client.putObject( + PutObjectRequest.builder() + .bucket(crossRegionBucketName) + .key(crossBucketObjectKey) + .build(), + RequestBody.fromBytes(contentBytes)); + // make a copy in cross-region bucket + S3FileIO s3FileIO = new S3FileIO(clientFactory::s3); + validateRead(s3FileIO, crossBucketObjectUri); + } finally { + AwsIntegTestUtil.cleanS3Bucket(s3Client, crossRegionBucketName, crossBucketObjectKey); + } + } + @Test public void testNewInputStreamWithCrossRegionAccessPoint() throws Exception { clientFactory.initialize(ImmutableMap.of(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true")); @@ -550,7 +573,11 @@ private void write(S3FileIO s3FileIO, String uri) throws Exception { } private void validateRead(S3FileIO s3FileIO) throws Exception { - InputFile file = s3FileIO.newInputFile(objectUri); + validateRead(s3FileIO, objectUri); + } + + private void validateRead(S3FileIO s3FileIO, String s3Uri) throws Exception { + InputFile file = s3FileIO.newInputFile(s3Uri); assertThat(file.getLength()).isEqualTo(contentBytes.length); try (InputStream stream = file.newStream()) { String result = IoUtils.toUtf8String(stream); 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 4ab1514a22d5..3a43880f31ed 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 @@ -376,6 +376,16 @@ public class S3FileIOProperties implements Serializable { public static final boolean DUALSTACK_ENABLED_DEFAULT = false; + /** + * Determines if S3 client will allow Cross-Region bucket access, default to false. + * + *

    For more details, see + * https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/s3-cross-region.html + */ + public static final String CROSS_REGION_ACCESS_ENABLED = "s3.cross-region-access-enabled"; + + public static final boolean CROSS_REGION_ACCESS_ENABLED_DEFAULT = false; + /** * Used by {@link S3FileIO}, prefix used for bucket access point configuration. To set, we can * pass a catalog property. @@ -442,6 +452,7 @@ public class S3FileIOProperties implements Serializable { private final Map bucketToAccessPointMapping; private boolean isPreloadClientEnabled; private final boolean isDualStackEnabled; + private final boolean isCrossRegionAccessEnabled; private final boolean isPathStyleAccess; private final boolean isUseArnRegionEnabled; private final boolean isAccelerationEnabled; @@ -477,6 +488,7 @@ public S3FileIOProperties() { this.bucketToAccessPointMapping = Collections.emptyMap(); this.isPreloadClientEnabled = PRELOAD_CLIENT_ENABLED_DEFAULT; this.isDualStackEnabled = DUALSTACK_ENABLED_DEFAULT; + this.isCrossRegionAccessEnabled = CROSS_REGION_ACCESS_ENABLED_DEFAULT; this.isPathStyleAccess = PATH_STYLE_ACCESS_DEFAULT; this.isUseArnRegionEnabled = USE_ARN_REGION_ENABLED_DEFAULT; this.isAccelerationEnabled = ACCELERATION_ENABLED_DEFAULT; @@ -521,6 +533,9 @@ public S3FileIOProperties(Map properties) { properties, ACCELERATION_ENABLED, ACCELERATION_ENABLED_DEFAULT); this.isDualStackEnabled = PropertyUtil.propertyAsBoolean(properties, DUALSTACK_ENABLED, DUALSTACK_ENABLED_DEFAULT); + this.isCrossRegionAccessEnabled = + PropertyUtil.propertyAsBoolean( + properties, CROSS_REGION_ACCESS_ENABLED, CROSS_REGION_ACCESS_ENABLED_DEFAULT); try { this.multiPartSize = PropertyUtil.propertyAsInt(properties, MULTIPART_SIZE, MULTIPART_SIZE_DEFAULT); @@ -680,6 +695,10 @@ public boolean isDualStackEnabled() { return this.isDualStackEnabled; } + public boolean isCrossRegionAccessEnabled() { + return this.isCrossRegionAccessEnabled; + } + public boolean isPathStyleAccess() { return this.isPathStyleAccess; } @@ -832,7 +851,7 @@ public void applyCredentialConfigurations( /** * Configure services settings for an S3 client. The settings include: s3DualStack, - * s3UseArnRegion, s3PathStyleAccess, and s3Acceleration + * crossRegionAccessEnabled, s3UseArnRegion, s3PathStyleAccess, and s3Acceleration * *

    Sample usage: * @@ -843,6 +862,7 @@ public void applyCredentialConfigurations( public void applyServiceConfigurations(T builder) { builder .dualstackEnabled(isDualStackEnabled) + .crossRegionAccessEnabled(isCrossRegionAccessEnabled) .serviceConfiguration( S3Configuration.builder() .pathStyleAccessEnabled(isPathStyleAccess) diff --git a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java index a61b9efb9fec..71b931257cf5 100644 --- a/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java +++ b/aws/src/test/java/org/apache/iceberg/aws/s3/TestS3FileIOProperties.java @@ -72,6 +72,9 @@ public void testS3FileIOPropertiesDefaultValues() { assertThat(S3FileIOProperties.DUALSTACK_ENABLED_DEFAULT) .isEqualTo(s3FileIOProperties.isDualStackEnabled()); + assertThat(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED_DEFAULT) + .isEqualTo(s3FileIOProperties.isCrossRegionAccessEnabled()); + assertThat(S3FileIOProperties.PATH_STYLE_ACCESS_DEFAULT) .isEqualTo(s3FileIOProperties.isPathStyleAccess()); @@ -155,6 +158,11 @@ public void testS3FileIOProperties() { S3FileIOProperties.DUALSTACK_ENABLED, String.valueOf(s3FileIOProperties.isDualStackEnabled())); + assertThat(map) + .containsEntry( + S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, + String.valueOf(s3FileIOProperties.isCrossRegionAccessEnabled())); + assertThat(map) .containsEntry( S3FileIOProperties.PATH_STYLE_ACCESS, @@ -382,6 +390,7 @@ private Map getTestProperties() { map.put(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true"); map.put(S3FileIOProperties.ACCELERATION_ENABLED, "true"); map.put(S3FileIOProperties.DUALSTACK_ENABLED, "true"); + map.put(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, "true"); map.put( S3FileIOProperties.MULTIPART_SIZE, String.valueOf(S3FileIOProperties.MULTIPART_SIZE_DEFAULT)); @@ -427,6 +436,7 @@ public void testApplyCredentialConfigurations() { public void testApplyS3ServiceConfigurations() { Map properties = Maps.newHashMap(); properties.put(S3FileIOProperties.DUALSTACK_ENABLED, "true"); + properties.put(S3FileIOProperties.CROSS_REGION_ACCESS_ENABLED, "true"); properties.put(S3FileIOProperties.PATH_STYLE_ACCESS, "true"); properties.put(S3FileIOProperties.USE_ARN_REGION_ENABLED, "true"); // acceleration enabled has to be set to false if path style is true @@ -438,6 +448,7 @@ public void testApplyS3ServiceConfigurations() { ArgumentCaptor.forClass(S3Configuration.class); Mockito.doReturn(mockA).when(mockA).dualstackEnabled(Mockito.anyBoolean()); + Mockito.doReturn(mockA).when(mockA).crossRegionAccessEnabled(Mockito.anyBoolean()); Mockito.doReturn(mockA).when(mockA).serviceConfiguration(Mockito.any(S3Configuration.class)); s3FileIOProperties.applyServiceConfigurations(mockA); diff --git a/docs/docs/aws.md b/docs/docs/aws.md index 5a166c0c9193..e408cb5a2ae4 100644 --- a/docs/docs/aws.md +++ b/docs/docs/aws.md @@ -514,6 +514,22 @@ spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCata For more details on using S3 Access Grants, please refer to [Managing access with S3 Access Grants](https://docs.aws.amazon.com/AmazonS3/latest/userguide/access-grants.html). +### S3 Cross-Region Access + +S3 Cross-Region bucket access can be turned on by setting catalog property `s3.cross-region-access-enabled` to `true`. +This is turned off by default to avoid first S3 API call increased latency. + +For example, to enable S3 Cross-Region bucket access with Spark 3.3, you can start the Spark SQL shell with: +``` +spark-sql --conf spark.sql.catalog.my_catalog=org.apache.iceberg.spark.SparkCatalog \ + --conf spark.sql.catalog.my_catalog.warehouse=s3://my-bucket2/my/key/prefix \ + --conf spark.sql.catalog.my_catalog.type=glue \ + --conf spark.sql.catalog.my_catalog.io-impl=org.apache.iceberg.aws.s3.S3FileIO \ + --conf spark.sql.catalog.my_catalog.s3.cross-region-access-enabled=true +``` + +For more details, please refer to [Cross-Region access for Amazon S3](https://docs.aws.amazon.com/sdk-for-java/latest/developer-guide/s3-cross-region.html). + ### S3 Acceleration [S3 Acceleration](https://aws.amazon.com/s3/transfer-acceleration/) can be used to speed up transfers to and from Amazon S3 by as much as 50-500% for long-distance transfer of larger objects. From cf4959fffb42b428f27f1bea33d2d93113109fa6 Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Mon, 14 Oct 2024 21:57:03 +0200 Subject: [PATCH 0774/1019] Core: Deprecate ContentCache.invalidateAll (#10494) * Deprecate ContentCache.invalidateAll 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. This is a similar problem as the one described at https://github.com/google/guava/issues/1881. `ContentCache` doesn't use a Guava Cache, it uses Caffeine. Caffeine offers partial solution to this issue, but not for `invalidateAll` call. To avoid accidental incorrect use of ContentCache, deprecate the `invalidateAll` method, which can be deceptive for the caller and remove it later. * Document ContentCache.invalidate blocking nature * empty --- .../java/org/apache/iceberg/io/ContentCache.java | 13 +++++++++++++ 1 file changed, 13 insertions(+) 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 c8733a904d90..5757c542dc09 100644 --- a/core/src/main/java/org/apache/iceberg/io/ContentCache.java +++ b/core/src/main/java/org/apache/iceberg/io/ContentCache.java @@ -129,10 +129,23 @@ public InputFile tryCache(InputFile input) { return input; } + /** + * Invalidate the cache entry for the given key. + * + *

    Note: if there is ongoing load, this is a blocking operation, i.e. it will wait for the load + * to complete before invalidating the entry. + */ public void invalidate(String key) { cache.invalidate(key); } + /** + * @deprecated since 1.6.0, will be removed in 1.7.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. + */ + @Deprecated public void invalidateAll() { cache.invalidateAll(); } From 6281aad794af35d53c368701386b279beb38bafa Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Tue, 15 Oct 2024 02:59:10 -0700 Subject: [PATCH 0775/1019] 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 765d71fcf1774e1d4ecaec3f8ae014a7660b9543 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 15 Oct 2024 20:02:49 +0200 Subject: [PATCH 0776/1019] 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 46ed80d8187ded5118fc020b7cbb9e4bb7be4a2b Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Wed, 16 Oct 2024 08:46:16 +0200 Subject: [PATCH 0777/1019] 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 9e849f26c9a48b3193069a9295e6159b9e3796fe 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 0778/1019] 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 3c7f0fd163963d269a9cd2532c3ec6aeda7b192d 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 0779/1019] 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 4a11561cd5417738ddcfc932e436c90f8b6b5d46 Mon Sep 17 00:00:00 2001 From: SeungwanJo Date: Thu, 17 Oct 2024 03:04:08 +0900 Subject: [PATCH 0780/1019] 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 5905bc247db3ff4743ed735c2a952a4e7a971a3a 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 0781/1019] 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 dd8efd0fbea337bb9b382557c2e744d1141ed0b9 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 0782/1019] 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 5698814a98c9207bfdc06d642202cd91c025432e Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Thu, 17 Oct 2024 15:44:53 +0900 Subject: [PATCH 0783/1019] 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 cc019cc67a021c2eba53b16b39ab8807d360be86 Mon Sep 17 00:00:00 2001 From: Edgar Rodriguez Date: Thu, 17 Oct 2024 09:48:54 -0400 Subject: [PATCH 0784/1019] 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 792b9300768b7479df1689d253a82ada553ee7c2 Mon Sep 17 00:00:00 2001 From: Aihua Xu Date: Thu, 17 Oct 2024 12:44:06 -0700 Subject: [PATCH 0785/1019] 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 4c8c8b75f98f9a40cc6d3aeade83c8b673659d7c Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Thu, 17 Oct 2024 14:31:02 -0700 Subject: [PATCH 0786/1019] 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 b83e337c73bd8c74b7b2c8dc98b87dc7a989645c Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 18 Oct 2024 17:31:44 +0800 Subject: [PATCH 0787/1019] 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 f72dce40fb8d947d57ac5d0d5f96a90a865c9b66 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 18 Oct 2024 09:06:57 -0700 Subject: [PATCH 0788/1019] 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 f75a8246449431a22491de5cf075fa566151cf43 Mon Sep 17 00:00:00 2001 From: Steven Zhen Wu Date: Fri, 18 Oct 2024 09:36:11 -0700 Subject: [PATCH 0789/1019] 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 8f26282e6da3f7d11c73998b4b30bd5b1885662a Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 18 Oct 2024 19:24:13 +0200 Subject: [PATCH 0790/1019] 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 73b537d5b22aebf8f9d059e08e7e973174ebfe3d Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 18 Oct 2024 19:24:36 +0200 Subject: [PATCH 0791/1019] 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 7945cc161284fbc04a91ce692876913a680c7278 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 0792/1019] 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 16a484e9ae473e237528df0a336b58d5d4b237fc Mon Sep 17 00:00:00 2001 From: Ozan Okumusoglu Date: Fri, 18 Oct 2024 16:48:00 -0700 Subject: [PATCH 0793/1019] 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 b9e4f6834bf0ce17eee5640decd460903cd70673 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 21 Oct 2024 06:35:03 +0200 Subject: [PATCH 0794/1019] 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 62def97a11bf4986c2070cc9ca8321ab7fdbb56c 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 0795/1019] 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 bf13ae60bf90d0ca5154c42c3433c3baf660ef7f 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 0796/1019] 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 2a7aa6f1e88cb87ac5ed29e7c8ed6ba9027e77f3 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 0797/1019] 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 e35dbdf2b90745cc9d3c0751538eb26fd0771cfd 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 0798/1019] 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 c72c076ca9ffce26283ba6806327317823bcfab4 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 0799/1019] 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 c135c99c656f29c742f7ff192f7aab7c506e9786 Mon Sep 17 00:00:00 2001 From: leesf <490081539@qq.com> Date: Mon, 21 Oct 2024 20:35:03 +0800 Subject: [PATCH 0800/1019] 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 fc8175cfd5d87a7ebbe2fb7486f2a876c60b8414 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Mon, 21 Oct 2024 11:53:13 -0700 Subject: [PATCH 0801/1019] 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 39ae45b3c984f3431808ff0e0a33c98639dea615 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Tue, 22 Oct 2024 09:13:01 -0700 Subject: [PATCH 0802/1019] 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 a8cf77efc0d37665175ebd260197bb2c47e1e2af Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Tue, 22 Oct 2024 15:28:57 -0700 Subject: [PATCH 0803/1019] 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 e6e01f0ff5083410eaa7540ebb75c95a179d5321 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 23 Oct 2024 17:32:30 +0200 Subject: [PATCH 0804/1019] 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 1f4d2d7e3e87d9f3db2e0ed5b8787a5c72dea153 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 23 Oct 2024 19:16:39 +0200 Subject: [PATCH 0805/1019] 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 19e6183df3203397a73e57ea8f31a724357288aa Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Wed, 23 Oct 2024 10:23:53 -0700 Subject: [PATCH 0806/1019] 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 d04e26ff1b26a3ec49e11ae1e891e6a96d0685d8 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 23 Oct 2024 12:36:34 -0700 Subject: [PATCH 0807/1019] 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 eb0357cfd0d93ed2af042d9ae478beeacf7f0d13 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 24 Oct 2024 05:59:18 +0800 Subject: [PATCH 0808/1019] 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 08ff7555efb7b72bfbe44312c0f4d8c406294e51 Mon Sep 17 00:00:00 2001 From: stubz151 Date: Thu, 24 Oct 2024 04:45:00 +0100 Subject: [PATCH 0809/1019] 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 5ef7b980e9fa70e19c8ea74cf3ada1bc65bce1d8 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 24 Oct 2024 12:33:27 +0200 Subject: [PATCH 0810/1019] 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 ccb61ccef509514708fdc23c95df2944b2d7d4e2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 24 Oct 2024 16:07:35 +0200 Subject: [PATCH 0811/1019] 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 ebe0bd385fc8dbed4f39e8211412bc9ecb94d356 Mon Sep 17 00:00:00 2001 From: sullis Date: Thu, 24 Oct 2024 08:37:33 -0700 Subject: [PATCH 0812/1019] 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 d9bbfc1dfef428dd1a190b0522a0e74bc8f4ae7b Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Thu, 24 Oct 2024 10:58:56 -0700 Subject: [PATCH 0813/1019] 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 d5696ae104b531880a6daf8f9f9e80cdbcad134e 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 0814/1019] 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 88a627a62a50222dff62fa02510bb7e7af3a852b Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Fri, 25 Oct 2024 16:25:23 +0200 Subject: [PATCH 0815/1019] 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 6174869cc5103d15b31b4e5a091087d572856ee2 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Sat, 26 Oct 2024 03:28:51 +0800 Subject: [PATCH 0816/1019] 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 e86b4da8e00c97944ac7ecfa41ca6fd5de54886e Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Sat, 26 Oct 2024 02:07:05 +0530 Subject: [PATCH 0817/1019] 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 7f175872ed5dd4520852f077af82417404aac5cc 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 0818/1019] [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 e25760fb37ce8abf9e4edd03ee29e4e93655093d Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Fri, 25 Oct 2024 17:23:44 -0400 Subject: [PATCH 0819/1019] 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 3db0af3ad921f65024319d299fe71a1eb6165795 Mon Sep 17 00:00:00 2001 From: erik-grepr Date: Fri, 25 Oct 2024 22:57:24 -0700 Subject: [PATCH 0820/1019] 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 d6f7998b67c5792f634c5c60a2e875e60357894b Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 28 Oct 2024 17:04:24 +0800 Subject: [PATCH 0821/1019] 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 124cc4723d3c466e50a876e7ae4fc406f6f82653 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 0822/1019] 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 cc0d17357f4b9222bff1b148d0919f4bf2e1e459 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 0823/1019] 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 bbeee79d5042751bb5a65a60e92e20fc248d8e04 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Mon, 28 Oct 2024 10:13:50 +0100 Subject: [PATCH 0824/1019] 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 9f16d3408e6fd47bb9b9c041c4ecbfd9a9860f31 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 0825/1019] 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 b28e095cc302ec5b82cdcc3812ded620ba48a757 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 0826/1019] 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 1b2234612e245bf5d88cab7b394702c06e3fd97b 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 0827/1019] 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 fc615f6ad2359a5fbe3c73141024b6eab1606edf 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 0828/1019] 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 24c2f9318262c9ce7266c9d18fffb21636d19e2b 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 0829/1019] 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 a1152972e3039898363ab2f2e7f32d1d56293dac 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 0830/1019] 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 ad35017275b2aba845011e625d9fa1f06f0feb39 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 0831/1019] 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 7ef6e26858b5d6d9442b7dd46b93799e245a3315 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 0832/1019] 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 bedb89774f52013a64275536f8e1cf992889fb8c Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 29 Oct 2024 00:42:58 +0800 Subject: [PATCH 0833/1019] 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 7ad2243494602b8cbe43110a3545b8ee9827b071 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 28 Oct 2024 18:12:06 +0100 Subject: [PATCH 0834/1019] 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 0711a2a060c67b3ed41adb3a08b6f3b47f9ac20f 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 0835/1019] 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 28c4c506fe8c891f718259270abf0925986ae192 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Mon, 28 Oct 2024 13:32:19 -0400 Subject: [PATCH 0836/1019] 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 e0cad0da1ecdf9c6330de17d59679d547e818300 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 0837/1019] 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 0a88b159d248ed4aae9772fd2c4b998923fdfb12 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 0838/1019] 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 34a70028b942917b751eecdd4203ea8f6c08355a Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 29 Oct 2024 03:57:39 +0800 Subject: [PATCH 0839/1019] 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 b3723a640cac0a4b3b5e87098f71a80108bb468f Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 28 Oct 2024 22:02:55 +0100 Subject: [PATCH 0840/1019] 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 2ddb804995b015d27330dc269335aa796f0319a3 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 0841/1019] 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 4003248bab7b90fb7ad6cef11e530935f7e62494 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Tue, 29 Oct 2024 23:49:10 +0530 Subject: [PATCH 0842/1019] 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 6330040c2144518408119f83a6d835855be09b55 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Tue, 29 Oct 2024 13:03:38 -0700 Subject: [PATCH 0843/1019] 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 90bc87e604fc9587f96bf0bb7f9604dd61363b77 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 07:09:42 +0100 Subject: [PATCH 0844/1019] 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 0dd80159a071f9349c72ba89cff152d4ec71ce65 Mon Sep 17 00:00:00 2001 From: Alex Merced Date: Wed, 30 Oct 2024 02:26:10 -0400 Subject: [PATCH 0845/1019] 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 2010347b3267149dd8c5654aa7121f7edc01a9de Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 10:29:45 +0100 Subject: [PATCH 0846/1019] 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 d153c54cc995b27507b32fe4767f3916aa583f2c Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 30 Oct 2024 15:49:37 +0100 Subject: [PATCH 0847/1019] 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 fc8596265824a1fbb7dbd61be66ec028bae3103f Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 18:30:55 +0100 Subject: [PATCH 0848/1019] 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 caa6ee9956dd9d5e5c3ea338563fb6d94e6ed0ff Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 30 Oct 2024 18:52:34 +0100 Subject: [PATCH 0849/1019] 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 f586b96f9704e03d36b77cc9cb01fa1820923065 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Wed, 30 Oct 2024 14:58:25 -0700 Subject: [PATCH 0850/1019] 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 75dc410af0f09d31bf8a3c1d4d316609039c1c08 Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Thu, 31 Oct 2024 13:31:14 -0700 Subject: [PATCH 0851/1019] 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 9cd378d2a9bd86d63f807b49716cbcf7fd191278 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 1 Nov 2024 04:48:45 +0800 Subject: [PATCH 0852/1019] 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 3cafd79982bc78f2ba7036ac8beacf407810a520 Mon Sep 17 00:00:00 2001 From: sullis Date: Thu, 31 Oct 2024 14:33:14 -0700 Subject: [PATCH 0853/1019] 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 b10b709bc7d97fa8be6b80c3fd1db58379a46752 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 1 Nov 2024 20:18:34 +0100 Subject: [PATCH 0854/1019] 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 24f1b1caa66ceb1d29d5b3236ab0d17e0d7031bc Mon Sep 17 00:00:00 2001 From: Hongyue/Steve Zhang Date: Fri, 1 Nov 2024 17:22:44 -0700 Subject: [PATCH 0855/1019] 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 2c3a1d6394e8078eb5606b6ffc1d6dba49d6b9a5 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sat, 2 Nov 2024 10:06:12 +0100 Subject: [PATCH 0856/1019] 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 0f1fb7072c2ae5c4907a69f09df52b8aa85809ea Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 2 Nov 2024 03:04:03 -0700 Subject: [PATCH 0857/1019] 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 7217d8cec3db2dcdf3b643f122755ff33231975a Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 2 Nov 2024 03:18:04 -0700 Subject: [PATCH 0858/1019] 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 906e0e511a6df3e7466d67b2f041d2d3aab5c833 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Sat, 2 Nov 2024 17:23:28 +0100 Subject: [PATCH 0859/1019] 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 2ff5361f7412d85a4d9e3718de7a8bffde5645eb 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 0860/1019] 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 7e6ce6a8b311510d577d508c8d695905fce3af08 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 0861/1019] 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 d33155b485be89e2b61dc1da73900b59d5cdae9f 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 0862/1019] 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 12b9c703e49654648b02f04b3a68555220579238 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 0863/1019] 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 04ed2197fc9528d6508e33aa6aac30dc33b5d118 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 0864/1019] 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 ec81acf2386a36b74be11268a548b87ebcd6c855 Mon Sep 17 00:00:00 2001 From: pvary Date: Mon, 4 Nov 2024 15:06:00 +0100 Subject: [PATCH 0865/1019] 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 f356cf46c18cb885ce5fe8a076463395c51a6bea 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 0866/1019] 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 f5764be2b0de0d4643d22c6857be3a6ad3a95605 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 0867/1019] 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 ffaf39ad2fffdb1c7df71b69ff18e12c2171d40f 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 0868/1019] 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 d6aa4720dc6ed3da3132ab388d1d79f75c5e7724 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 0869/1019] 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 b7bfed1f13f310648858557dc731afd6de0a3b30 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 4 Nov 2024 15:40:21 +0100 Subject: [PATCH 0870/1019] 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 30123eda69de71301956a3d8a4735f7612ed396d Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 4 Nov 2024 19:35:08 +0100 Subject: [PATCH 0871/1019] 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 65c18f5057266f217ef5f6bc26a09c662ca1e864 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Mon, 4 Nov 2024 13:27:40 -0600 Subject: [PATCH 0872/1019] 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 268689f084c029fe8b8130ade6f4a43f00eacef7 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 4 Nov 2024 21:22:50 +0100 Subject: [PATCH 0873/1019] 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 6058c29dabeb..1477d0bfef5b 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 @@ -47,7 +47,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; @@ -70,6 +72,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.ImmutableList; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; @@ -136,6 +139,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(); @@ -1288,6 +1347,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 fa9a6e14da5b908c362bf07f19425482cc2cbf04 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 5 Nov 2024 08:35:42 +0100 Subject: [PATCH 0874/1019] 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 0875/1019] 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 1477d0bfef5b..4655edce16b4 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 @@ -114,7 +114,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} }; } @@ -158,16 +159,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(); @@ -186,10 +187,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); } } } @@ -1347,10 +1351,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() @@ -1409,4 +1421,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 bba0b16a9efdfb6c20ebc88cfd0c5ea8d3762e88 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 5 Nov 2024 18:04:56 +0800 Subject: [PATCH 0876/1019] 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 246c505b60f3394ab42f5ada7391fc37f6b0440d Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 5 Nov 2024 12:56:02 +0100 Subject: [PATCH 0877/1019] 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 4655edce16b4..0153669b5b98 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 @@ -133,10 +133,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 eb3d4e017478dfa4f61f21fc11acfa880107439e Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 5 Nov 2024 15:52:24 +0100 Subject: [PATCH 0878/1019] 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 f9c083222e42bc3f63bdd4439ea781d940f82682 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Tue, 5 Nov 2024 17:02:00 +0100 Subject: [PATCH 0879/1019] 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 877cd69fc5e3f9c7027c2d14e0ca335abf3d6b3b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Tue, 5 Nov 2024 11:15:49 -0700 Subject: [PATCH 0880/1019] 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 8341daeb4727222189184f2a223e1646d18ce16a 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 0881/1019] 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 85c789ed42c00e7d672d4ed83173495313cdc6cf Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Wed, 6 Nov 2024 13:47:35 +0530 Subject: [PATCH 0882/1019] 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 7bea33123002b050ea0d9e508012e77c9a4e272e Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 6 Nov 2024 21:32:07 +0100 Subject: [PATCH 0883/1019] 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 e345525a99a79bbc00ccd25c4383f2d0ab085588 Mon Sep 17 00:00:00 2001 From: Mingliang Liu Date: Wed, 6 Nov 2024 14:45:41 -0800 Subject: [PATCH 0884/1019] 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 ce560b68e36f710cd8825bfd7d2917be91eb0c8d Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 7 Nov 2024 09:47:50 -0600 Subject: [PATCH 0885/1019] 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 72cddd930e6b8a17a8f85d37e60c387d146606c3 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 8 Nov 2024 00:50:10 +0800 Subject: [PATCH 0886/1019] 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 a8b5d5704c2b518f05da3e3e813133e87543205b Mon Sep 17 00:00:00 2001 From: pvary Date: Fri, 8 Nov 2024 07:29:44 +0100 Subject: [PATCH 0887/1019] 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 eab800deab71b1c773a1f9ca2ba3ecc85722b90b Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 8 Nov 2024 14:36:27 +0800 Subject: [PATCH 0888/1019] 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 d18c751902e12a25ed35f35c7c909fe0589b1331 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 8 Nov 2024 18:09:49 +0100 Subject: [PATCH 0889/1019] 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 7394d2a637d681ea8bdec4b2a530b1c38d364635 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:05:42 -0600 Subject: [PATCH 0890/1019] 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 7d33358ea74855ff79023718b131aeb14162c6e5 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:34:08 -0600 Subject: [PATCH 0891/1019] 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 0701c59b256efedec4ffe362f065d7aaf6af455c Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:34:56 -0600 Subject: [PATCH 0892/1019] 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 4f414d892acaf1983095a0651a84cc7b93716abc Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 13:35:12 -0600 Subject: [PATCH 0893/1019] 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 4cd801eb0b64f8b90c6a6ed004e32e9b129722cb Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 14:46:15 -0600 Subject: [PATCH 0894/1019] 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 7bac7169ea7a60e0ade000c7c1b618299e5e9401 Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Fri, 8 Nov 2024 14:47:21 -0600 Subject: [PATCH 0895/1019] 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 01aac10a64c82d4a5bdb41b7775aa9e317cc6917 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 0896/1019] 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 261f892bcef662fa3284a467fdf6846c7e967f23 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 8 Nov 2024 15:57:37 -0800 Subject: [PATCH 0897/1019] 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 3b8a1302b0fba41c252f98aecb00590834075fe6 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 0898/1019] 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 fa58d487e1a8987b05b9c5bcf0f447eff24fa243 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 0899/1019] 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 9f1f99b2f4334978a013ed89a9c33fc34eb947f4 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 0900/1019] 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 68c2eb185deb35e798f20cf45a98a5d21e416944 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Mon, 11 Nov 2024 21:41:03 +0800 Subject: [PATCH 0901/1019] 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 823492b168ccfbc44b708199dedd8deefc8ac6d7 Mon Sep 17 00:00:00 2001 From: dongwang Date: Tue, 12 Nov 2024 00:58:41 +0800 Subject: [PATCH 0902/1019] 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 7e84055ff692054bcf0332e75883addfbf69efa9 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Mon, 11 Nov 2024 22:08:23 +0100 Subject: [PATCH 0903/1019] 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 0153669b5b98..3d45950ecd15 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 @@ -1031,7 +1031,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(); @@ -1104,7 +1104,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(); @@ -1207,7 +1207,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(); @@ -1268,6 +1268,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 9fd170ce22ce508e6a284338e0509be07e09b8d5 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Mon, 11 Nov 2024 18:55:42 -0500 Subject: [PATCH 0904/1019] 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 0c0dce9ed4ae217d8dc5d208654ecbb0c59e21b2 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 0905/1019] 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 3dc56a3e887a5937ffaaa70f8b7e9f696f7198a1 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Wed, 13 Nov 2024 00:29:08 +0800 Subject: [PATCH 0906/1019] 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 abf973211038a7725094e09412946899f84d1980 Mon Sep 17 00:00:00 2001 From: Kevin Liu Date: Tue, 12 Nov 2024 14:30:21 -0500 Subject: [PATCH 0907/1019] 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 28eb1b9cbe436a4e15b6b04e9d8bf0eeb720a866 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 0908/1019] 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 90864fe503f42c1d220cfca98fe7f61ce5991cd2 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 13 Nov 2024 16:42:06 +0100 Subject: [PATCH 0909/1019] 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 3d45950ecd15..d92f11a5a9c6 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 @@ -43,6 +43,7 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.concurrent.ThreadLocalRandom; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -115,6 +116,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} }; } @@ -1420,13 +1422,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) { @@ -1457,7 +1469,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 65784288f14d24db24347e25ab1d5ad168dc0406 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Wed, 13 Nov 2024 13:09:31 -0800 Subject: [PATCH 0910/1019] 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 da1ae35fb8fe54d68b9f58564225bfc4830fe6e6 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 0911/1019] 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 aee615446a020f8d0eac4c98f07131ee8ba4d163 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 14 Nov 2024 18:48:48 +0800 Subject: [PATCH 0912/1019] 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 276ab4df92e42ccddddfed9154c4a50990688542 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 0913/1019] 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 5edf14322abc4791bbf0a2ea571ec76e4bdee1a7 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 0914/1019] 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 121df4a2dcb9a2ca7785432ce61ca52a5b18356b Mon Sep 17 00:00:00 2001 From: Russell Spitzer Date: Thu, 14 Nov 2024 16:04:01 -0600 Subject: [PATCH 0915/1019] 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 1442e7f7772461cbd00f6465a351f6d77b0f32a0 Mon Sep 17 00:00:00 2001 From: Rocco Varela Date: Thu, 14 Nov 2024 22:07:04 -0800 Subject: [PATCH 0916/1019] 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 8dbbac198bfa792c4c6f8c7b8c1f51b75fb7035a Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Fri, 15 Nov 2024 18:19:52 +0800 Subject: [PATCH 0917/1019] 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 620f3da9e0ab771542917ff14e1a4d13efcd7060 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 15 Nov 2024 08:50:29 -0700 Subject: [PATCH 0918/1019] 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 9d95e62ca5d27e6efdfffeb1adeb42e15beae93b Mon Sep 17 00:00:00 2001 From: Aihua Xu Date: Fri, 15 Nov 2024 10:52:23 -0800 Subject: [PATCH 0919/1019] 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 a0eec4e614eb6768de062461121d1e2d1405e40f Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 15 Nov 2024 22:12:45 +0100 Subject: [PATCH 0920/1019] 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 317f2ce45c51a138c0f59febede9a921497c1c32 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 15 Nov 2024 22:14:32 +0100 Subject: [PATCH 0921/1019] 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 892251f2fc166c49a2ba6bd16644fa11c0037f2e Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Fri, 15 Nov 2024 22:15:08 +0100 Subject: [PATCH 0922/1019] 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 0b98372fa4b170b8cdaddab4bf90c3dc9387955d 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 0923/1019] 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 6810d00b7364caeb4f00458217594deed6c11614 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 0924/1019] 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 bab521e13b8e2c246e70a90df736f542b1215b73 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 0925/1019] 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 a1ad2e03cbabbac9e6b7eb2a9dc5e9ab0b7f7b57 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Sun, 17 Nov 2024 17:20:20 +0100 Subject: [PATCH 0926/1019] 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 50e02a44791d7b0600965d2950723f546f04d1f7 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 0927/1019] 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 c288f93ad442dcac6e47eef676c6ea897fb5e092 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 0928/1019] 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 cbd59b14c663c5a237f366bb3cd17275dbc178e5 Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Mon, 18 Nov 2024 08:26:58 -0700 Subject: [PATCH 0929/1019] 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 7ebaa2ab4f5ebc1797df49997514f2998b6bd183 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 18 Nov 2024 18:14:35 +0100 Subject: [PATCH 0930/1019] 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 8bb129d68beeb40c969334e6f6856a5135bbb1ec Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 19 Nov 2024 04:11:17 +0800 Subject: [PATCH 0931/1019] 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 9d6125172cb92645cfa026ca1c90202b6ecd93cc Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Tue, 19 Nov 2024 04:11:41 +0800 Subject: [PATCH 0932/1019] 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 a5837b643d649f45d218b885557d1261689f2962 Mon Sep 17 00:00:00 2001 From: gaborkaszab Date: Tue, 19 Nov 2024 14:23:55 +0100 Subject: [PATCH 0933/1019] 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 fd83be4fdee35c8ded6c87315b0ac6c0ada28259 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 19 Nov 2024 15:38:46 +0100 Subject: [PATCH 0934/1019] =?UTF-8?q?Revert=20"Core:=20Use=20encoding/deco?= =?UTF-8?q?ding=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 4d11d5ba1010f122bbfdf97a33fe28f2045fb490 Mon Sep 17 00:00:00 2001 From: leesf Date: Wed, 20 Nov 2024 14:05:06 +0800 Subject: [PATCH 0935/1019] 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 1fdf8465db7cb569cdc8d8cd6c81abfcbb32e9ac Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 20 Nov 2024 09:35:34 +0100 Subject: [PATCH 0936/1019] 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 613227efde83045ce4b00dd9b7fd0dc0ecd16063 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 20 Nov 2024 09:48:27 +0100 Subject: [PATCH 0937/1019] 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 2702a276b26e9d361077f14968406740dec8aef7 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 20 Nov 2024 15:06:33 +0100 Subject: [PATCH 0938/1019] 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 33586ec36466a6534f50e578103fc60c7666da72 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Wed, 20 Nov 2024 17:32:11 +0100 Subject: [PATCH 0939/1019] 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 70473a45bb0912b28a98bc6f78300f12b56107a5 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 21 Nov 2024 00:39:41 +0800 Subject: [PATCH 0940/1019] 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 3bc8f4a10be595612112f702e04036ff09366313 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Wed, 20 Nov 2024 10:40:48 -0800 Subject: [PATCH 0941/1019] 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 e222727770f2bbc72ee23ec868462beaa25d634f Mon Sep 17 00:00:00 2001 From: Zhendong Bai Date: Thu, 21 Nov 2024 05:26:19 +0800 Subject: [PATCH 0942/1019] 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 cfc3b63b869a5bd0ad2f8c19ba680bcade072254 Mon Sep 17 00:00:00 2001 From: Matt Topol Date: Wed, 20 Nov 2024 17:08:36 -0500 Subject: [PATCH 0943/1019] 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 7c25b278739b8b08135040112806648decbf234f Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Wed, 20 Nov 2024 15:10:26 -0800 Subject: [PATCH 0944/1019] 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 29fb251dc0f57464a485fff441353655020da259 Mon Sep 17 00:00:00 2001 From: Wing Yew Poon Date: Wed, 20 Nov 2024 17:46:57 -0800 Subject: [PATCH 0945/1019] 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 d91dfae2602601abb6f7aa43c7298c6f992c66bd Mon Sep 17 00:00:00 2001 From: Ace Haidrey Date: Wed, 20 Nov 2024 21:15:44 -0500 Subject: [PATCH 0946/1019] 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 354ef179b100a3f6f355ecfd28ee33ef5e552479 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Thu, 21 Nov 2024 23:14:23 +0800 Subject: [PATCH 0947/1019] 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 e09ce27486e2cd511642171ae49a673f96281382 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 0948/1019] 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 977d2ece672f789aaa104aefbedda46dc2e6c76a Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Thu, 21 Nov 2024 08:23:53 -0700 Subject: [PATCH 0949/1019] 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 c186ff750ea6693297113b07643a1c6f3c697e15 Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Thu, 21 Nov 2024 21:09:10 +0100 Subject: [PATCH 0950/1019] 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 8b85536e37422602c47342125c1cc8602b41828d Mon Sep 17 00:00:00 2001 From: Haizhou Zhao Date: Thu, 21 Nov 2024 15:32:26 -0800 Subject: [PATCH 0951/1019] 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 721faae4c0dc9c6c28612dc75b996191d4d4db75 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 22 Nov 2024 22:26:23 +0800 Subject: [PATCH 0952/1019] 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 03ec03fbd445580734feb9b849c8cee306eabf90 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Fri, 22 Nov 2024 22:26:47 +0800 Subject: [PATCH 0953/1019] 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 815e384462497409fa711b90da1e931fe4fc0dcd 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 0954/1019] 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 4d2452cba4255e05b010a902b07641d9a0885192 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:28:24 +0800 Subject: [PATCH 0955/1019] 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 d55dcebcebb3042929e392302b5f027b55ed2da5 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:57:50 +0800 Subject: [PATCH 0956/1019] 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 e9f0ba14eb19d74c518ff343a84696affc893ab7 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:58:39 +0800 Subject: [PATCH 0957/1019] 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 7583238a5c31137c232ec3baaba1934ae7b9fb42 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Sat, 23 Nov 2024 14:59:46 +0800 Subject: [PATCH 0958/1019] 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 ff06e72b1c8e28f7498e902fac9ebd71f30e95ca 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 0959/1019] 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 dc83b48489ae75c40993e50a89e64cf0463021c7 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 0960/1019] 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 e52a2a19b1ba45767698bc7bdc594d272b3c1312 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 15:48:17 +0800 Subject: [PATCH 0961/1019] 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 1db36dcd3865173258d9e2404e5d7b0197b5d880 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 0962/1019] 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 8ec88dbe425f0de1a69fc715ecd0fa99669c2a40 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 25 Nov 2024 10:25:56 +0100 Subject: [PATCH 0963/1019] 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 a7e51b49f4f5f0ddc2be29233dd17323a41ea1e4 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 0964/1019] 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 c1a432daffff83d0093220ea16680280d181bf7b Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Mon, 25 Nov 2024 13:23:10 +0100 Subject: [PATCH 0965/1019] 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 d150f959fe948aabb47ccd1f718dccff933b8805 Mon Sep 17 00:00:00 2001 From: Cheng Pan Date: Mon, 25 Nov 2024 20:45:27 +0800 Subject: [PATCH 0966/1019] 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 3870e5c40b168329230a0b0fe65876b1b58db15f Mon Sep 17 00:00:00 2001 From: Hussein Awala Date: Mon, 25 Nov 2024 14:08:54 +0100 Subject: [PATCH 0967/1019] 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 a139d6cd1d8d6be41fa22ce296611bcceeae01af 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 0968/1019] 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 aa97326e9070aa3bd7fb2a8bdf8a115779b95b51 Mon Sep 17 00:00:00 2001 From: Manu Zhang Date: Tue, 26 Nov 2024 15:37:46 +0800 Subject: [PATCH 0969/1019] 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 45ec035401bd79146a64aa72b0d5d741877ca070 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 0970/1019] 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 6f732703ad6059381c4ef3eb992fe9bc758b630b 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 0971/1019] 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 4134bb60212b6f0995352b4f378edeaf175aa713 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Wed, 27 Nov 2024 13:43:46 +0100 Subject: [PATCH 0972/1019] 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 67950d9590063de02a7bcfcaff0f571d7390eeb7 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 0973/1019] 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 ab574eae06dbc119283f955b1fdbd2bf3a1119e9 Mon Sep 17 00:00:00 2001 From: Arek Burdach Date: Wed, 27 Nov 2024 14:52:19 +0100 Subject: [PATCH 0974/1019] 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 c228d163894e8b73451cb67acdcbfb6180a9f8e1 Mon Sep 17 00:00:00 2001 From: Hugo Friant Date: Wed, 27 Nov 2024 15:47:51 +0100 Subject: [PATCH 0975/1019] 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 2698e012d4526a530d82f6cd92bae7384913f6a3 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 0976/1019] 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 5d1c32a2f562ce5d5f3d89f0359963afa9a10f6d 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 0977/1019] 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 53603ecf611619e2df04a01d9048583cd86bb166 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Thu, 28 Nov 2024 15:14:57 +0530 Subject: [PATCH 0978/1019] 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 6b29e5a5ab9b141218a8bf0b23af2c022fc69dd4 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Thu, 28 Nov 2024 15:59:42 +0100 Subject: [PATCH 0979/1019] 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 05cee15349e8d11517a01fdc1d5be21b5d3001b8 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 28 Nov 2024 11:34:23 -0800 Subject: [PATCH 0980/1019] 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 28c4ae5258fa7376a14a76103ac0fc93a520fbef Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 28 Nov 2024 20:44:49 +0100 Subject: [PATCH 0981/1019] 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 57be178c7424549f1aa3a307aaf25017ff95c562 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Thu, 28 Nov 2024 23:28:34 +0100 Subject: [PATCH 0982/1019] 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 91dd581fe853763667650749a9c1890bf5650e19 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 0983/1019] 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 0d901724d9600f131c70860b2a8be2dbad14e388 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Sat, 30 Nov 2024 00:05:44 +0900 Subject: [PATCH 0984/1019] 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 beb590fd7ff4605d8ae84df04873f583f3d99525 Mon Sep 17 00:00:00 2001 From: Ajantha Bhat Date: Sun, 1 Dec 2024 12:42:30 +0530 Subject: [PATCH 0985/1019] 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 11f2ea06add01d73b92a9d93d1e2c43986376e73 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 0986/1019] 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 ff95425c0c5ee9c187f26f252647eafe7513c81c 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 0987/1019] 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 cdb3413d8d10772c70ca5d9c8ef0b7f1c2945271 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Mon, 2 Dec 2024 14:25:10 +0900 Subject: [PATCH 0988/1019] 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 a1974955e62290435769743fb83c3a33a5f84631 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 0989/1019] 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 2c78118218a181f42f7518410e40787ff88f4a88 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 0990/1019] 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 d7ef1f0e207e1428f7b46d0e81372a45a4bbcee8 Mon Sep 17 00:00:00 2001 From: AGW Date: Tue, 3 Dec 2024 05:58:57 +0800 Subject: [PATCH 0991/1019] 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 45cf8b744be1f495cfd58eb5688939644adb9df3 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Mon, 2 Dec 2024 17:26:27 -0800 Subject: [PATCH 0992/1019] 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 d406604722953766b7f353e146b03cfbdaa9ef36 Mon Sep 17 00:00:00 2001 From: Eduard Tudenhoefner Date: Tue, 3 Dec 2024 18:54:47 +0100 Subject: [PATCH 0993/1019] 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 78daca86d85ec67f2e1e50efea6138a6085dd262 Mon Sep 17 00:00:00 2001 From: Yuya Ebihara Date: Wed, 4 Dec 2024 20:09:22 +0900 Subject: [PATCH 0994/1019] 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 3300d3c2a6a55d308d4d3a4d79d6145d2b4a12c0 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 4 Dec 2024 12:10:24 +0100 Subject: [PATCH 0995/1019] 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 e68434ca0e48b40d5f4c0eeb0050690e4efccd2c Mon Sep 17 00:00:00 2001 From: AGW Date: Wed, 4 Dec 2024 22:39:10 +0800 Subject: [PATCH 0996/1019] 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 e18c55d17d9a26a1068db2e5519256dbfb0a1d78 Mon Sep 17 00:00:00 2001 From: Shohei Okumiya Date: Thu, 5 Dec 2024 00:41:10 +0900 Subject: [PATCH 0997/1019] 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 97ceb53333cd680fb4cb25a416a629fb645087ce Mon Sep 17 00:00:00 2001 From: Karuppayya Date: Wed, 4 Dec 2024 14:34:52 -0800 Subject: [PATCH 0998/1019] 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 8a0c301bc611971d132be62e344e3ceb873cb7fd Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Thu, 5 Dec 2024 18:18:10 -0800 Subject: [PATCH 0999/1019] 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 47004f0de09e7b68dc1e57a8333b0f25ed28bc3b Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Fri, 6 Dec 2024 08:46:57 -0800 Subject: [PATCH 1000/1019] 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 0403c4b6c6634744c8bbf365a10826e0e28431a8 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Fri, 6 Dec 2024 12:09:29 -0800 Subject: [PATCH 1001/1019] 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 07f3d036c7ff673f17fca360b1f415010de9e26a Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Fri, 6 Dec 2024 13:49:27 -0800 Subject: [PATCH 1002/1019] 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 f70af7cc7497da5d01e7ba4b25be695df0bf93df Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 6 Dec 2024 18:15:02 -0800 Subject: [PATCH 1003/1019] 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 3391d3d500464252ca578be119cea846b35e98fb 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 1004/1019] 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 2d493630c907ec6f00520271c226338ba7b46b69 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 1005/1019] 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 cea11cc8512924c06caa4c6af50fb1ee761cf64e 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 1006/1019] 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 ace1ce1740f8cfff34b5212a7c4ab2d4df10d52e Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Mon, 9 Dec 2024 14:07:38 +0100 Subject: [PATCH 1007/1019] 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 8dc8996cd32b40db66a7468da68b9b3c49218f4c 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 1008/1019] 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 d040408a15e7262b4e2ea4f0a0cc9bd786b32d00 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 1009/1019] 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 cffdb7ecf7cbea8baf4d859129ca8d4700557574 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 1010/1019] 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 5d7363c3d112a06412818435ebb3220ed2676b39 Mon Sep 17 00:00:00 2001 From: Bryan Keller Date: Mon, 9 Dec 2024 10:35:56 -0800 Subject: [PATCH 1011/1019] 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 2d54279df73813c385656aff1a910b84d73ab387 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 9 Dec 2024 11:20:45 -0800 Subject: [PATCH 1012/1019] 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 b8f164f5c0c59ad33b7a3858f4e40297886268de Mon Sep 17 00:00:00 2001 From: Piotr Findeisen Date: Tue, 10 Dec 2024 20:36:12 +0100 Subject: [PATCH 1013/1019] 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 6dac99b3982a20348d21050d8ef6507ee8119fc0 Mon Sep 17 00:00:00 2001 From: GuoYu <511955993@qq.com> Date: Wed, 11 Dec 2024 05:33:55 +0800 Subject: [PATCH 1014/1019] 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 bd6049da8ba82a4fa0520b53c819c8896d4aac9c 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 1015/1019] 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 4548bd77d76adcc3769fd17a032c776c598a12ca Mon Sep 17 00:00:00 2001 From: Amogh Jahagirdar Date: Wed, 11 Dec 2024 10:04:59 -0700 Subject: [PATCH 1016/1019] 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 534f99e19a6913f0a72df3b99de24e1ff31eb069 Mon Sep 17 00:00:00 2001 From: Fokko Driesprong Date: Wed, 11 Dec 2024 21:45:52 +0100 Subject: [PATCH 1017/1019] 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 37e4ed2bf418973b968a14b4940af1865db81c20 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Thu, 15 Feb 2024 11:33:15 -0600 Subject: [PATCH 1018/1019] Added custom sorting with partition column order or user supplied function api to RewriteManifests --- .palantir/revapi.yml | 3 +++ .../iceberg/spark/actions/RewriteManifestsSparkAction.java | 1 + .../iceberg/spark/actions/TestRewriteManifestsAction.java | 3 +++ 3 files changed, 7 insertions(+) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 18c63fbe7bb1..e04995861d39 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -878,6 +878,9 @@ acceptedBreaks: old: "class org.apache.iceberg.types.Types.NestedField" new: "class org.apache.iceberg.types.Types.NestedField" justification: "Add default value APIs." + - code: "java.method.addedToInterface" + new: "method org.apache.iceberg.actions.RewriteManifests org.apache.iceberg.actions.RewriteManifests::sort(java.util.List)" + justification: "New method added to Spark RewriteManifests" org.apache.iceberg:iceberg-core: - code: "java.class.defaultSerializationChanged" old: "class org.apache.iceberg.PartitionData" diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 8410fa7bf93e..9faf948b1527 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -65,6 +65,7 @@ import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; +import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; import org.apache.spark.broadcast.Broadcast; 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 d92f11a5a9c6..7d17432a95b1 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 @@ -44,6 +44,9 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; import java.util.concurrent.ThreadLocalRandom; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; From 1659e495d4cc5f28ad67a86e2e3ec2deea923db2 Mon Sep 17 00:00:00 2001 From: Zach Dischner Date: Mon, 23 Dec 2024 09:13:11 -0600 Subject: [PATCH 1019/1019] Removed function-based manifest rewrite --- .../iceberg/actions/RewriteManifests.java | 29 +-- .../actions/RewriteManifestsSparkAction.java | 54 ++---- .../actions/TestRewriteManifestsAction.java | 176 +----------------- 3 files changed, 29 insertions(+), 230 deletions(-) diff --git a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java index 603af96f5800..917ac24ea004 100644 --- a/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java +++ b/api/src/main/java/org/apache/iceberg/actions/RewriteManifests.java @@ -19,9 +19,7 @@ package org.apache.iceberg.actions; import java.util.List; -import java.util.function.Function; import java.util.function.Predicate; -import org.apache.iceberg.DataFile; import org.apache.iceberg.ManifestFile; /** An action that rewrites manifests. */ @@ -51,39 +49,24 @@ public interface RewriteManifests * Rewrite manifests in a given order, based on partition field names * *

    Supply an optional set of partition field names to cluster the rewritten manifests by. For - * example, given a table PARTITIONED BY (a, b, c, d), you may wish to rewrite and cluster - * manifests by ('d', 'b') only, based on your query patterns. Rewriting Manifests in this way + * example, given a table PARTITIONED BY (a, b, c, d), one may wish to rewrite and cluster + * manifests by ('d', 'b') only, based on known query patterns. Rewriting Manifests in this way * will yield manifest_lists that point to manifest_files containing data files for common 'd' and * 'b' partitions. * *

    If not set, manifests will be rewritten in the order of the transforms in the table's * current partition spec. * - * @param partitionFieldClustering Exact transformed column names used for partitioning; not the - * raw column names that partitions are derived from. E.G. supply 'data_bucket' and not 'data' - * for a bucket(N, data) partition * definition + * @param partitionFields Exact transformed column names used for partitioning; not the raw column + * names that partitions are derived from. E.G. supply 'data_bucket' and not 'data' for a + * bucket(N, data) partition * definition * @return this for method chaining */ - default RewriteManifests clusterBy(List partitionFieldClustering) { + default RewriteManifests clusterBy(List partitionFields) { throw new UnsupportedOperationException( this.getClass().getName() + " doesn't implement clusterBy(List)"); } - /** - * Rewrite manifests in a given order, dictated by a custom Function - * - *

    Supply a Function which will apply its own custom clustering logic based on supplied {@link - * org.apache.iceberg.DataFile} attributes. - * - * @param clusterStrategyFunction A Function that returns a String to be used for manifest - * clustering - * @return this method for chaining - */ - default RewriteManifests clusterBy(Function clusterStrategyFunction) { - throw new UnsupportedOperationException( - this.getClass().getName() + " doesn't implement clusterBy(Function)"); - } - /** * Passes a location where the staged manifests should be written. * diff --git a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java index 9faf948b1527..c5f827c915a6 100644 --- a/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java +++ b/spark/v3.5/spark/src/main/java/org/apache/iceberg/spark/actions/RewriteManifestsSparkAction.java @@ -65,7 +65,6 @@ import org.apache.iceberg.spark.source.SerializableTableWithSize; import org.apache.iceberg.types.Types; import org.apache.iceberg.util.PropertyUtil; -import org.apache.iceberg.util.Tasks; import org.apache.iceberg.util.ThreadPools; import org.apache.spark.api.java.function.MapPartitionsFunction; import org.apache.spark.broadcast.Broadcast; @@ -76,9 +75,7 @@ import org.apache.spark.sql.Row; import org.apache.spark.sql.SparkSession; import org.apache.spark.sql.api.java.UDF1; -import org.apache.spark.sql.expressions.UserDefinedFunction; import org.apache.spark.sql.functions; -import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.types.StructType; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -118,7 +115,6 @@ public class RewriteManifestsSparkAction private String outputLocation; private List partitionFieldClustering = null; - private Function partitionClusteringFunction = null; RewriteManifestsSparkAction(SparkSession spark, Table table) { super(spark); @@ -175,28 +171,25 @@ public RewriteManifestsSparkAction stagingLocation(String newStagingLocation) { } @Override - public RewriteManifestsSparkAction clusterBy(List clusteringColumns) { - // Collect set of allowable partition columns to cluster on + public RewriteManifestsSparkAction clusterBy(List partitionFields) { + // Collect set of available partition columns to cluster on Set availablePartitionNames = spec.fields().stream().map(PartitionField::name).collect(Collectors.toSet()); + // Identify specified partition fields that are not available in the spec + List missingFields = + partitionFields.stream() + .filter(field -> !availablePartitionNames.contains(field)) + .collect(Collectors.toList()); + // Check if these partition fields are included in the spec Preconditions.checkArgument( - clusteringColumns.stream().allMatch(availablePartitionNames::contains), - "Cannot use custom clustering to rewrite manifests '%s'. All partition columns must be " - + "defined in the current partition spec: %s. Choose from the available partitionable columns: %s", - clusteringColumns, - this.spec.specId(), - availablePartitionNames); - - this.partitionFieldClustering = clusteringColumns; - return this; - } + missingFields.isEmpty(), + "Cannot set manifest clustering because specified field(s) %s were not found in current partition spec %s.", + missingFields, + this.spec.specId()); - @Override - public RewriteManifests clusterBy(Function clusteringFunction) { - this.partitionClusteringFunction = - (Function & Serializable) clusteringFunction; + this.partitionFieldClustering = partitionFields; return this; } @@ -293,26 +286,7 @@ private List writePartitionedManifests( // Extract desired clustering criteria into a dedicated column Dataset clusteredManifestEntryDF; - if (partitionClusteringFunction != null) { - LOG.info( - "Sorting manifests for specId {} using custom clustering function", - spec.specId(), - partitionClusteringFunction); - Types.StructType partitionType = DataFile.getType(table.spec().partitionType()); - StructType dataFileSchema = manifestEntryDF.select("data_file.*").schema(); - - // Create a UDF to wrap the custom partitionClusteringFunction call - UserDefinedFunction clusteringUdf = - functions.udf( - new CustomDataFileClusteringUdf( - this.partitionClusteringFunction, partitionType, dataFileSchema), - DataTypes.StringType); - // Apply supplied partitionSortFunction function to the data_file datums within this dataframe - // The results are stored as a String in the new __clustering_column__ - clusteredManifestEntryDF = - manifestEntryDF.withColumn( - CUSTOM_CLUSTERING_COLUMN_NAME, clusteringUdf.apply(col("data_file"))); - } else if (partitionFieldClustering != null) { + if (partitionFieldClustering != null) { LOG.info( "Clustering manifests for specId {} by partition columns by {} ", spec.specId(), 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 7d17432a95b1..ef946c9c431a 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 @@ -33,20 +33,14 @@ import java.io.File; import java.io.IOException; -import java.io.Serializable; import java.nio.file.Path; import java.util.Comparator; import java.util.List; import java.util.Map; import java.util.Random; import java.util.UUID; -import java.util.function.Function; -import java.util.stream.Collectors; -import java.util.stream.IntStream; import java.util.concurrent.ThreadLocalRandom; -import java.util.function.Function; import java.util.stream.Collectors; -import java.util.stream.IntStream; import org.apache.hadoop.conf.Configuration; import org.apache.iceberg.DataFile; import org.apache.iceberg.DataFiles; @@ -551,38 +545,35 @@ public void testRewriteManifestsPartitionedTableWithInvalidClusteringColumns() options.put(TableProperties.SNAPSHOT_ID_INHERITANCE_ENABLED, snapshotIdInheritanceEnabled); Table table = TABLES.create(SCHEMA, spec, options, tableLocation); - SparkActions actions = SparkActions.get(); + SparkActions actions = org.apache.iceberg.spark.actions.SparkActions.get(); - // c2 is not a partition column, cannot use for clustering - List badClusteringKeys1 = ImmutableList.of("c1", "c2"); + List hasNonexistentFields = ImmutableList.of("c1", "c2"); assertThatThrownBy( () -> actions .rewriteManifests(table) .rewriteIf(manifest -> true) - .clusterBy(badClusteringKeys1) + .clusterBy(hasNonexistentFields) .execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot use custom clustering to rewrite manifests '[c1, c2]'. All partition columns " - + "must be defined in the current partition spec: 0. Choose from the available " - + "partitionable columns: [c3_bucket, c1]"); + "Cannot set manifest clustering because specified field(s) [c2] were not found in " + + "current partition spec 0."); // c3_bucket is the correct internal partition name to use, c3 is the untransformed column name, // clusterBy() expects the hidden partition column names - List badClusteringKeys2 = ImmutableList.of("c1", "c3"); + List hasIncorrectPartitionFieldNames = ImmutableList.of("c1", "c3"); assertThatThrownBy( () -> actions .rewriteManifests(table) .rewriteIf(manifest -> true) - .clusterBy(badClusteringKeys2) + .clusterBy(hasIncorrectPartitionFieldNames) .execute()) .isInstanceOf(IllegalArgumentException.class) .hasMessage( - "Cannot use custom clustering to rewrite manifests '[c1, c3]'. All partition columns " - + "must be defined in the current partition spec: 0. Choose from the available " - + "partitionable columns: [c3_bucket, c1]"); + "Cannot set manifest clustering because specified field(s) [c3] were not found in " + + "current partition spec 0."); } @TestTemplate @@ -697,155 +688,6 @@ public void testRewriteManifestsPartitionedTableWithCustomClustering() throws IO .isEqualTo(c3PartitionMax); } - @TestTemplate - public void testRewriteManifestsPartitionedTableWithCustomClusteringFunction() - throws IOException { - Random random = new Random(); - - PartitionSpec spec = - PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 3).bucket("c3", 10).build(); - Table table = TABLES.create(SCHEMA, spec, tableLocation); - - List dataFiles = Lists.newArrayList(); - for (int fileOrdinal = 0; fileOrdinal < 1000; fileOrdinal++) { - dataFiles.add( - newDataFile( - table, - TestHelpers.Row.of( - new Object[] { - fileOrdinal, String.valueOf(random.nextInt() * 100), random.nextInt(10) - }))); - } - ManifestFile appendManifest = writeManifest(table, dataFiles); - table.newFastAppend().appendManifest(appendManifest).commit(); - - List manifests = table.currentSnapshot().allManifests(table.io()); - assertThat(manifests).as("Should have 1 manifests before rewrite").hasSize(1); - - // Capture the c3 partition's lower and upper bounds - used for later test assertions - Integer c3PartitionMin = - Conversions.fromByteBuffer( - Types.IntegerType.get(), manifests.get(0).partitions().get(2).lowerBound()); - Integer c3PartitionMax = - Conversions.fromByteBuffer( - Types.IntegerType.get(), manifests.get(0).partitions().get(2).upperBound()); - - // Set the target manifest size to a small value to force splitting records into multiple files - table - .updateProperties() - .set( - TableProperties.MANIFEST_TARGET_SIZE_BYTES, - String.valueOf(manifests.get(0).length() / 3)) - .commit(); - - SparkActions actions = SparkActions.get(); - - // This is the main point of this test! - // Let's say I want to cluster manifests into buckets 0-4 and 5 - 9 - // I know my Table and partition Spec, so can do that programmatically - Function test = - (Function & Serializable) - (dataFile) -> { - StructLike partition = dataFile.partition(); - // Find the ordinal index for the c3 partition column for this data file - int c3Index = - IntStream.range(0, spec.fields().size()) - .filter(i -> spec.fields().get(i).name().contains("c3")) - .findFirst() - .getAsInt(); - Object c3BucketValue = partition.get(c3Index, Object.class); - - // Return one string for the lower values, one for the upper. RewriteManifests - // will cluster datafiles together in manifests according to this value. - return (Integer) c3BucketValue < 5 ? "cluster=LT_5" : "cluster=GTE_5"; - }; - - RewriteManifests.Result result = - actions - .rewriteManifests(table) - .rewriteIf(manifest -> true) - .clusterBy(test) - .option(RewriteManifestsSparkAction.USE_CACHING, useCaching) - .execute(); - - table.refresh(); - List newManifests = table.currentSnapshot().allManifests(table.io()); - - assertThat(result.rewrittenManifests()).hasSize(1); - assertThat(result.addedManifests()).hasSizeGreaterThanOrEqualTo(2); - - assertThat(newManifests).hasSizeGreaterThanOrEqualTo(2); - - // Rewritten manifests are clustered by c3_bucket - each should contain only a subset of the - // lower and upper bounds of the partition 'c3'. - List> c3Boundaries = - newManifests.stream() - .map(manifest -> manifest.partitions().get(2)) - .sorted( - Comparator.comparing( - ptn -> Conversions.fromByteBuffer(Types.IntegerType.get(), ptn.lowerBound()))) - .map( - p -> - Pair.of( - (Integer) - Conversions.fromByteBuffer(Types.IntegerType.get(), p.lowerBound()), - (Integer) - Conversions.fromByteBuffer(Types.IntegerType.get(), p.upperBound()))) - .collect(Collectors.toList()); - - List lowers = c3Boundaries.stream().map(t -> t.first()).collect(Collectors.toList()); - List uppers = c3Boundaries.stream().map(t -> t.second()).collect(Collectors.toList()); - - // The custom function clusters datafiles by having c3 bucket partitions in the range 0-4 and - // 5-9. - // Internally, that looks like - // +--------------------+---------------------+ - // |partition |__clustering_column__| - // +--------------------+---------------------+ - // |{0, -531806488, 0} |cluster=LT_5 | - // |{1, 385955472, 7} |cluster=GTE_5 | - // |{2, 604077840, 6} |cluster=GTE_5 | - // |{3, 1875302972, 4} |cluster=LT_5 | - // |{4, -1772544904, 0} |cluster=LT_5 | - // |{5, 172551248, 7} |cluster=GTE_5 | - // ... - // After rewriting, this looks like - // - manifest 1 -> [lower bound = 0, upper bound = 4] - // - manifest 2 -> [lower bound = 5, upper bound = 9] - // Without the custom clustering, each manifest tracks the full range of c3 upper/lower bounds. - // AKA they look like - // - manifest 1 -> [lower bound = 0, upper bound = 9] - // - manifest 2 -> [lower bound = 0, upper bound = 9] - // So the upper bound of the partitions tracked by the first file should be LEQ the lower bounds - // of the second. Etc - assertThat(uppers.get(0)) - .as("Upper bound of first manifest partition should be LESS THAN the lower bound of second") - .isLessThan(lowers.get(1)); - - // Each file should contain less than the full c3 partition span - c3Boundaries.forEach( - boundary -> { - assertThat(boundary.second() - boundary.first()) - .as("Manifest should contain less than the full range of c3 bucket partitions") - .isLessThan(c3PartitionMax - c3PartitionMin); - }); - - // c3's Bucket(10) partition means our true lower bound = 0 and true upper bound is 9. The first - // manifest should include the lower bound of 0, and the last should have the upper bound of 9 - assertThat(lowers.get(0)) - .as("Lower bound of first manifest c3 bucket partition should be 0") - .isEqualTo(0); - assertThat(uppers.get(0)) - .as("Upper bound of first manifest c3 bucket partition should be 4") - .isEqualTo(4); - assertThat(lowers.get(1)) - .as("Lower bound of second manifest c3 bucket partition should be 5") - .isEqualTo(5); - assertThat(uppers.get(1)) - .as("Upper bound of second manifest c3 bucket partition should be 9") - .isEqualTo(9); - } - @TestTemplate public void testRewriteManifestsWithPredicate() throws IOException { PartitionSpec spec = PartitionSpec.builderFor(SCHEMA).identity("c1").truncate("c2", 2).build();